[VOL-4293] OpenONU Adapter update for gRPC migration

Change-Id: I05300d3b95b878f44576a99a05f53f52fdc0cda1
diff --git a/.gitignore b/.gitignore
index c4b2953..4bb60e7 100644
--- a/.gitignore
+++ b/.gitignore
@@ -2,6 +2,9 @@
 .idea
 exportToHTML
 
+# vscode
+.vscode
+
 # Python
 *.pyc
 
diff --git a/.golangci.yml b/.golangci.yml
index 7478787..b2483cb 100644
--- a/.golangci.yml
+++ b/.golangci.yml
@@ -55,48 +55,8 @@
       linters:
         - gosec
   exclude:
-    - "don't use underscores in Go names; method Adapter_descriptor"
-    - "don't use underscores in Go names; method Device_types"
-    - "don't use underscores in Go names; method Adopt_device"
-    - "don't use underscores in Go names; method Reconcile_device"
-    - "don't use underscores in Go names; method Abandon_device"
-    - "don't use underscores in Go names; method Disable_device"
-    - "don't use underscores in Go names; method Reenable_device"
-    - "don't use underscores in Go names; method Reboot_device"
-    - "don't use underscores in Go names; method Self_test_device"
-    - "don't use underscores in Go names; method Delete_device"
-    - "don't use underscores in Go names; method Get_device_details"
-    - "don't use underscores in Go names; method Update_flows_bulk"
-    - "don't use underscores in Go names; method Update_flows_incrementally"
-    - "don't use underscores in Go names; method Update_pm_config"
-    - "don't use underscores in Go names; method Receive_packet_out"
-    - "don't use underscores in Go names; method Suppress_event"
-    - "don't use underscores in Go names; method Unsuppress_event"
-    - "don't use underscores in Go names; method Get_ofp_device_info"
-    - "don't use underscores in Go names; method Get_ofp_port_info"
-    - "don't use underscores in Go names; method Process_inter_adapter_message"
-    - "don't use underscores in Go names; method Download_image"
-    - "don't use underscores in Go names; method Get_image_download_status"
-    - "don't use underscores in Go names; method Cancel_image_download"
-    - "don't use underscores in Go names; method Activate_image_update"
-    - "don't use underscores in Go names; method Revert_image_update"
-    - "don't use underscores in Go names; method Disable_port"
-    - "don't use underscores in Go names; method Enable_port"
-    - "don't use underscores in Go names; method Child_device_lost"
-    - "don't use underscores in Go names; method Start_omci_test"
-    - "don't use underscores in Go names; method Get_ext_value"
-    - "don't use underscores in Go names; method Single_get_value_request"
-    - "don't use underscores in Go names; method Download_onu_image"
-    - "don't use underscores in Go names; method Get_onu_image_status"
-    - "don't use underscores in Go names; method Abort_onu_image_upgrade"
-    - "don't use underscores in Go names; method Get_onu_images"
-    - "don't use underscores in Go names; method Activate_onu_image"
-    - "don't use underscores in Go names; method Commit_onu_image"
-    - "don't use underscores in Go names; method Process_tech_profile_instance_request"
-    - "Error return value of `dh.coreProxy.PortStateUpdate` is not checked"
     - "Error return value of `rxCallbackEntry.cbFunction` is not checked"
     - "Error return value of `oo.sendNextRequest` is not checked"
     - "Error return value of `oo.pDevOmciCC.send` is not checked"
     - "Error return value of `onuDeviceEntry.mibDbClass` is not checked"
-    - "Error return value of `handler.rebootDevice` is not checked"
   exclude-use-default: false
diff --git a/Makefile b/Makefile
index 40525f1..cd082e0 100755
--- a/Makefile
+++ b/Makefile
@@ -64,18 +64,18 @@
 ## Local Development Helpers
 local-protos: ## Copies a local version of the voltha-protos dependency into the vendor directory
 ifdef LOCAL_PROTOS
-	rm -rf vendor/github.com/opencord/voltha-protos/v4/go
-	mkdir -p vendor/github.com/opencord/voltha-protos/v4/go
-	cp -r ${LOCAL_PROTOS}/go/* vendor/github.com/opencord/voltha-protos/v4/go
-	rm -rf vendor/github.com/opencord/voltha-protos/v4/go/vendor
+	rm -rf vendor/github.com/opencord/voltha-protos/v5/go
+	mkdir -p vendor/github.com/opencord/voltha-protos/v5/go
+	cp -r ${LOCAL_PROTOS}/go/* vendor/github.com/opencord/voltha-protos/v5/go
+	rm -rf vendor/github.com/opencord/voltha-protos/v5/go/vendor
 endif
 
 ## Local Development Helpers
 local-lib-go: ## Copies a local version of the voltha-lib-go dependency into the vendor directory
 ifdef LOCAL_LIB_GO
-	rm -rf vendor/github.com/opencord/voltha-lib-go/v5/pkg
-	mkdir -p vendor/github.com/opencord/voltha-lib-go/v5/pkg
-	cp -r ${LOCAL_LIB_GO}/pkg/* vendor/github.com/opencord/voltha-lib-go/v5/pkg/
+	rm -rf vendor/github.com/opencord/voltha-lib-go/v7/pkg
+	mkdir -p vendor/github.com/opencord/voltha-lib-go/v7/pkg
+	cp -r ${LOCAL_LIB_GO}/pkg/* vendor/github.com/opencord/voltha-lib-go/v7/pkg/
 endif
 
 build: docker-build ## Alias for 'docker build'
diff --git a/VERSION b/VERSION
index 0ed156c..227cea2 100755
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-1.4.1-dev231
+2.0.0
diff --git a/cmd/openonu-adapter/common.go b/cmd/openonu-adapter/common.go
index 1c0be5e..bf1bc68 100644
--- a/cmd/openonu-adapter/common.go
+++ b/cmd/openonu-adapter/common.go
@@ -18,7 +18,7 @@
 package main
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/cmd/openonu-adapter/main.go b/cmd/openonu-adapter/main.go
index c42cef4..ce5dc6f 100644
--- a/cmd/openonu-adapter/main.go
+++ b/cmd/openonu-adapter/main.go
@@ -28,38 +28,46 @@
 	"syscall"
 	"time"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	com "github.com/opencord/voltha-lib-go/v5/pkg/adapters/common"
-	conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-lib-go/v5/pkg/probe"
-	"github.com/opencord/voltha-lib-go/v5/pkg/version"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/golang/protobuf/ptypes/empty"
+	conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-lib-go/v7/pkg/version"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/grpc"
+
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
 
 	"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
 	ac "github.com/opencord/voltha-openonu-adapter-go/internal/pkg/onuadaptercore"
 )
 
+const (
+	clusterMessagingService = "cluster-message-service"
+	onuAdapterService       = "onu-adapter-service"
+	kvService               = "kv-service"
+	coreService             = "core-service"
+)
+
 type adapter struct {
 	//defaultAppName   string
-	instanceID       string
-	config           *config.AdapterFlags
-	iAdapter         adapters.IAdapter // from Voltha interface adapters
-	kafkaClient      kafka.Client
-	kvClient         kvstore.Client
-	kip              kafka.InterContainerProxy
-	coreProxy        adapterif.CoreProxy
-	adapterProxy     adapterif.AdapterProxy
-	eventProxy       eventif.EventProxy
-	halted           bool
-	exitChannel      chan int
-	receiverChannels []<-chan *ic.InterContainerMessage //from inter-container
+	instanceID  string
+	config      *config.AdapterFlags
+	kafkaClient kafka.Client
+	kvClient    kvstore.Client
+	eventProxy  eventif.EventProxy
+	grpcServer  *vgrpc.GrpcServer
+	onuAdapter  *ac.OpenONUAC
+	coreClient  *vgrpc.Client
+	halted      bool
+	exitChannel chan int
 }
 
 func newAdapter(cf *config.AdapterFlags) *adapter {
@@ -68,7 +76,6 @@
 	a.config = cf
 	a.halted = false
 	a.exitChannel = make(chan int, 1)
-	a.receiverChannels = make([]<-chan *ic.InterContainerMessage, 0)
 	return &a
 }
 
@@ -82,11 +89,10 @@
 			p = value.(*probe.Probe)
 			p.RegisterService(
 				ctx,
-				"message-bus",
-				"kv-store",
-				"container-proxy",
-				"core-request-handler",
-				"register-with-core",
+				clusterMessagingService,
+				kvService,
+				onuAdapterService,
+				coreService,
 			)
 		}
 	}
@@ -97,60 +103,84 @@
 		logger.Fatalw(ctx, "error-setting-kv-client", log.Fields{"error": err})
 	}
 
-	if p != nil {
-		p.UpdateStatus(ctx, "kv-store", probe.ServiceStatusRunning)
-	}
-
 	// Setup Log Config
 	cm := conf.NewConfigManager(ctx, a.kvClient, a.config.KVStoreType, a.config.KVStoreAddress, a.config.KVStoreTimeout)
 	go conf.StartLogLevelConfigProcessing(cm, ctx)
 
 	// Setup Kafka Client
-	if a.kafkaClient, err = newKafkaClient(ctx, "sarama", a.config.KafkaAdapterAddress); err != nil {
+	if a.kafkaClient, err = newKafkaClient(ctx, "sarama", a.config.KafkaClusterAddress); err != nil {
 		logger.Fatalw(ctx, "Unsupported-common-client", log.Fields{"error": err})
 	}
 
-	if p != nil {
-		p.UpdateStatus(ctx, "message-bus", probe.ServiceStatusRunning)
+	// Start kafka communication with the broker
+	if err := kafka.StartAndWaitUntilKafkaConnectionIsUp(ctx, a.kafkaClient, a.config.HeartbeatCheckInterval, clusterMessagingService); err != nil {
+		logger.Fatal(ctx, "unable-to-connect-to-kafka")
 	}
 
-	// Start the common InterContainer Proxy - retries as per program arguments or indefinitely per default
-	if a.kip, err = a.startInterContainerProxy(ctx, a.config.KafkaReconnectRetries); err != nil {
-		logger.Fatalw(ctx, "error-starting-inter-container-proxy", log.Fields{"error": err})
-		//aborting the complete processing here (does notmake sense after set Retry number [else use -1 for infinite])
-		return err
+	// Wait until connection to KV store is established
+	if err := WaitUntilKvStoreConnectionIsUp(ctx, a.kvClient, a.config.KVStoreTimeout, kvService); err != nil {
+		logger.Fatal(ctx, "unable-to-connect-to-kv-store")
 	}
 
-	// Create the core proxy to handle requests to the Core
-	a.coreProxy = com.NewCoreProxy(ctx, a.kip, a.config.Topic, a.config.CoreTopic)
-
-	logger.Debugw(ctx, "create adapter proxy", log.Fields{"CoreTopic": a.config.CoreTopic})
-	a.adapterProxy = com.NewAdapterProxy(ctx, a.kip, a.config.CoreTopic, cm.Backend)
-
 	// Create the event proxy to post events to KAFKA
 	a.eventProxy = events.NewEventProxy(events.MsgClient(a.kafkaClient), events.MsgTopic(kafka.Topic{Name: a.config.EventTopic}))
+	go func() {
+		if err := a.eventProxy.Start(); err != nil {
+			logger.Fatalw(ctx, "event-proxy-cannot-start", log.Fields{"error": err})
+		}
+	}()
+
+	// Create the Core client to handle requests to the Core.  Note that the coreClient is an interface and needs to be
+	// cast to the appropriate grpc client by invoking GetCoreGrpcClient on the a.coreClient
+	if a.coreClient, err = vgrpc.NewClient(a.config.CoreEndpoint,
+		a.coreRestarted,
+		vgrpc.ActivityCheck(true)); err != nil {
+		logger.Fatal(ctx, "grpc-client-not-created")
+	}
+	// Start the core grpc client
+	go a.coreClient.Start(ctx, setAndTestCoreServiceHandler)
 
 	// Create the open ONU interface adapter
-	if a.iAdapter, err = a.startVolthaInterfaceAdapter(ctx, a.kip, a.coreProxy, a.adapterProxy, a.eventProxy,
-		a.config, cm); err != nil {
+	if a.onuAdapter, err = a.startVolthaInterfaceAdapter(ctx, a.coreClient, a.eventProxy, a.config, cm); err != nil {
 		logger.Fatalw(ctx, "error-starting-volthaInterfaceAdapter for OpenOnt", log.Fields{"error": err})
 	}
 
-	// Register the core request handler
-	if err = a.setupRequestHandler(ctx, a.instanceID, a.iAdapter); err != nil {
-		logger.Fatalw(ctx, "error-setting-core-request-handler", log.Fields{"error": err})
-	}
+	// Create and start the grpc server
+	a.grpcServer = vgrpc.NewGrpcServer(a.config.GrpcAddress, nil, false, p)
+
+	//Register the  adapter  service
+	a.addAdapterService(ctx, a.grpcServer, a.onuAdapter)
+
+	//Register the onu inter adapter  service
+	a.addOnuInterAdapterService(ctx, a.grpcServer, a.onuAdapter)
+
+	go a.startGRPCService(ctx, a.grpcServer, onuAdapterService)
 
 	// Register this adapter to the Core - retries indefinitely
-	if err = a.registerWithCore(ctx, -1); err != nil {
+	if err = a.registerWithCore(ctx, coreService, -1); err != nil {
 		logger.Fatalw(ctx, "error-registering-with-core", log.Fields{"error": err})
 	}
 
-	// check the readiness and liveliness and update the probe status
+	// Start the readiness and liveliness check and update the probe status
 	a.checkServicesReadiness(ctx)
 	return err
 }
 
+// TODO:  Any action the adapter needs to do following a Core restart?
+func (a *adapter) coreRestarted(ctx context.Context, endPoint string) error {
+	logger.Errorw(ctx, "core-restarted", log.Fields{"endpoint": endPoint})
+	return nil
+}
+
+// setAndTestCoreServiceHandler is used to test whether the remote gRPC service is up
+func setAndTestCoreServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	svc := core.NewCoreServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		return nil
+	}
+	return svc
+}
+
 func (a *adapter) stop(ctx context.Context) {
 	// Stop leadership tracking
 	a.halted = true
@@ -168,9 +198,20 @@
 		a.kvClient.Close(ctx)
 	}
 
-	if a.kip != nil {
-		a.kip.Stop(ctx)
+	if a.eventProxy != nil {
+		a.eventProxy.Stop()
 	}
+
+	if a.kafkaClient != nil {
+		a.kafkaClient.Stop(ctx)
+	}
+
+	// Stop core client
+	if a.coreClient != nil {
+		a.coreClient.Stop(ctx)
+	}
+
+	// TODO:  More cleanup
 }
 
 // #############################################
@@ -214,38 +255,10 @@
 	return nil
 }
 
-func (a *adapter) startInterContainerProxy(ctx context.Context, retries int) (kafka.InterContainerProxy, error) {
-	logger.Infow(ctx, "starting-intercontainer-messaging-proxy", log.Fields{"addr": a.config.KafkaAdapterAddress, "topic": a.config.Topic})
-	var err error
-	/* address config update acc. to [VOL-2736] */
-	kip := kafka.NewInterContainerProxy(
-		kafka.InterContainerAddress(a.config.KafkaAdapterAddress),
-		kafka.MsgClient(a.kafkaClient),
-		kafka.DefaultTopic(&kafka.Topic{Name: a.config.Topic}))
-	count := 0
-	for {
-		if err = kip.Start(ctx); err != nil {
-			logger.Warnw(ctx, "error-starting-messaging-proxy", log.Fields{"error": err, "retry": retries, "count": count})
-			if retries == count {
-				return nil, err
-			}
-			count++
-			// Take a nap before retrying
-			time.Sleep(2 * time.Second)
-		} else {
-			break
-		}
-	}
-	probe.UpdateStatusFromContext(ctx, "container-proxy", probe.ServiceStatusRunning)
-	logger.Info(ctx, "common-messaging-proxy-created")
-	return kip, nil
-}
-
-func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, kip kafka.InterContainerProxy,
-	cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep eventif.EventProxy,
+func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, cc *vgrpc.Client, ep eventif.EventProxy,
 	cfg *config.AdapterFlags, cm *conf.ConfigManager) (*ac.OpenONUAC, error) {
 	var err error
-	sAcONU := ac.NewOpenONUAC(ctx, a.kip, cp, ap, ep, a.kvClient, cfg, cm)
+	sAcONU := ac.NewOpenONUAC(ctx, cc, ep, a.kvClient, cfg, cm)
 
 	if err = sAcONU.Start(ctx); err != nil {
 		logger.Fatalw(ctx, "error-starting-OpenOnuAdapterCore", log.Fields{"error": err})
@@ -256,20 +269,7 @@
 	return sAcONU, nil
 }
 
-func (a *adapter) setupRequestHandler(ctx context.Context, coreInstanceID string, iadapter adapters.IAdapter) error {
-	logger.Info(ctx, "setting-request-handler")
-	requestProxy := com.NewRequestHandlerProxy(coreInstanceID, iadapter, a.coreProxy)
-	if err := a.kip.SubscribeWithRequestHandlerInterface(ctx, kafka.Topic{Name: a.config.Topic}, requestProxy); err != nil {
-		logger.Errorw(ctx, "request-handler-setup-failed", log.Fields{"error": err})
-		return err
-
-	}
-	probe.UpdateStatusFromContext(ctx, "core-request-handler", probe.ServiceStatusRunning)
-	logger.Info(ctx, "request-handler-setup-done")
-	return nil
-}
-
-func (a *adapter) registerWithCore(ctx context.Context, retries int) error {
+func (a *adapter) registerWithCore(ctx context.Context, serviceName string, retries int) error {
 	adapterID := fmt.Sprintf("brcm_openomci_onu_%d", a.config.CurrentReplica)
 	vendorIdsList := strings.Split(a.config.OnuVendorIds, ",")
 	logger.Infow(ctx, "registering-with-core", log.Fields{
@@ -282,43 +282,78 @@
 		Id:             adapterID, // Unique name for the device type ->exact type required for OLT comm????
 		Vendor:         "VOLTHA OpenONUGo",
 		Version:        version.VersionInfo.Version,
-		Endpoint:       a.config.Topic,
+		Endpoint:       a.config.AdapterEndpoint,
 		Type:           "brcm_openomci_onu",
 		CurrentReplica: int32(a.config.CurrentReplica),
 		TotalReplicas:  int32(a.config.TotalReplicas),
 	}
 	types := []*voltha.DeviceType{{Id: "brcm_openomci_onu",
 		VendorIds:                   vendorIdsList,
-		Adapter:                     "brcm_openomci_onu", // Name of the adapter that handles device type
+		AdapterType:                 "brcm_openomci_onu", // Type of adapter that handles this device type
+		Adapter:                     "brcm_openomci_onu", // Deprecated attribute
 		AcceptsBulkFlowUpdate:       false,               // Currently openolt adapter does not support bulk flow handling
 		AcceptsAddRemoveFlowUpdates: true}}
 	deviceTypes := &voltha.DeviceTypes{Items: types}
 	count := 0
 	for {
-		if err := a.coreProxy.RegisterAdapter(context.TODO(), adapterDescription, deviceTypes); err != nil {
-			logger.Warnw(ctx, "registering-with-core-failed", log.Fields{"error": err})
+		gClient, err := a.coreClient.GetCoreServiceClient()
+		if gClient != nil {
+			if gClient != nil {
+				if _, err = gClient.RegisterAdapter(log.WithSpanFromContext(context.TODO(), ctx), &ic.AdapterRegistration{
+					Adapter: adapterDescription,
+					DTypes:  deviceTypes}); err == nil {
+					break
+				}
+			}
+			logger.Warnw(ctx, "registering-with-core-failed", log.Fields{"endpoint": a.config.CoreEndpoint, "error": err, "count": count, "gclient": gClient})
 			if retries == count {
 				return err
 			}
 			count++
-			// Take a nap before retrying
+			// Take a power nap before retrying
 			time.Sleep(2 * time.Second)
-		} else {
-			break
+
 		}
 	}
-	probe.UpdateStatusFromContext(ctx, "register-with-core", probe.ServiceStatusRunning)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
 	logger.Info(ctx, "registered-with-core")
 	return nil
 }
 
+// startGRPCService creates the grpc service handlers, registers it to the grpc server and starts the server
+func (a *adapter) startGRPCService(ctx context.Context, server *vgrpc.GrpcServer, serviceName string) {
+	logger.Infow(ctx, "service-created", log.Fields{"service": serviceName})
+
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+	logger.Infow(ctx, "service-started", log.Fields{"service": serviceName})
+
+	server.Start(ctx)
+	probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusStopped)
+}
+
+func (a *adapter) addAdapterService(ctx context.Context, server *vgrpc.GrpcServer, handler adapter_services.AdapterServiceServer) {
+	logger.Info(ctx, "adding-adapter-service")
+
+	server.AddService(func(gs *grpc.Server) {
+		adapter_services.RegisterAdapterServiceServer(gs, handler)
+	})
+}
+
+func (a *adapter) addOnuInterAdapterService(ctx context.Context, server *vgrpc.GrpcServer, handler adapter_services.OnuInterAdapterServiceServer) {
+	logger.Info(ctx, "adding-onu-inter-adapter-service")
+
+	server.AddService(func(gs *grpc.Server) {
+		adapter_services.RegisterOnuInterAdapterServiceServer(gs, handler)
+	})
+}
+
 /**
 This function checks the liveliness and readiness of the kakfa and kv-client services
 and update the status in the probe.
 */
 func (a *adapter) checkServicesReadiness(ctx context.Context) {
 	// checks the kafka readiness
-	go a.checkKafkaReadiness(ctx)
+	go kafka.MonitorKafkaReadiness(ctx, a.kafkaClient, a.config.LiveProbeInterval, a.config.NotLiveProbeInterval, clusterMessagingService)
 
 	// checks the kv-store readiness
 	go a.checkKvStoreReadiness(ctx)
@@ -333,19 +368,19 @@
 	timeout := a.config.LiveProbeInterval / 2
 	kvStoreChannel := make(chan bool, 1)
 
-	// Default false to check the liveliness.
-	kvStoreChannel <- false
+	// Default true - we are here only after we already had a KV store connection
+	kvStoreChannel <- true
 	for {
 		timeoutTimer := time.NewTimer(timeout)
 		select {
 		case liveliness := <-kvStoreChannel:
 			if !liveliness {
 				// kv-store not reachable or down, updating the status to not ready state
-				probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusNotReady)
+				probe.UpdateStatusFromContext(ctx, kvService, probe.ServiceStatusNotReady)
 				timeout = a.config.NotLiveProbeInterval
 			} else {
 				// kv-store is reachable , updating the status to running state
-				probe.UpdateStatusFromContext(ctx, "kv-store", probe.ServiceStatusRunning)
+				probe.UpdateStatusFromContext(ctx, kvService, probe.ServiceStatusRunning)
 				timeout = a.config.LiveProbeInterval / 2
 			}
 			// Check if the timer has expired or not
@@ -364,48 +399,28 @@
 	}
 }
 
-/**
-This function checks the liveliness and readiness of the kafka service
-and update the status in the probe.
-*/
-func (a *adapter) checkKafkaReadiness(ctx context.Context) {
-	livelinessChannel := a.kafkaClient.EnableLivenessChannel(ctx, true)
-	healthinessChannel := a.kafkaClient.EnableHealthinessChannel(ctx, true)
-	timeout := a.config.LiveProbeInterval
+// WaitUntilKvStoreConnectionIsUp waits until the KV client can establish a connection to the KV server or until the
+// context times out.
+func WaitUntilKvStoreConnectionIsUp(ctx context.Context, kvClient kvstore.Client, connectionRetryInterval time.Duration, serviceName string) error {
+	if kvClient == nil {
+		return errors.New("kvclient-is-nil")
+	}
 	for {
-		timeoutTimer := time.NewTimer(timeout)
-
-		select {
-		case healthiness := <-healthinessChannel:
-			if !healthiness {
-				// logger.Fatal will call os.Exit(1) to terminate
-				logger.Fatal(ctx, "Kafka service has become unhealthy")
-			}
-		case liveliness := <-livelinessChannel:
-			if !liveliness {
-				// kafka not reachable or down, updating the status to not ready state
-				probe.UpdateStatusFromContext(ctx, "message-bus", probe.ServiceStatusNotReady)
-				timeout = a.config.NotLiveProbeInterval
-			} else {
-				// kafka is reachable , updating the status to running state
-				probe.UpdateStatusFromContext(ctx, "message-bus", probe.ServiceStatusRunning)
-				timeout = a.config.LiveProbeInterval
-			}
-			// Check if the timer has expired or not
-			if !timeoutTimer.Stop() {
-				<-timeoutTimer.C
-			}
-		case <-timeoutTimer.C:
-			logger.Info(ctx, "kafka-proxy-liveness-recheck")
-			// send the liveness probe in a goroutine; we don't want to deadlock ourselves as
-			// the liveness probe may wait (and block) writing to our channel.
-			err := a.kafkaClient.SendLiveness(ctx)
-			if err != nil {
-				// Catch possible error case if sending liveness after Sarama has been stopped.
-				logger.Warnw(ctx, "error-kafka-send-liveness", log.Fields{"error": err})
+		if !kvClient.IsConnectionUp(ctx) {
+			probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
+			logger.Warnw(ctx, "kvconnection-down", log.Fields{"service-name": serviceName, "connect-retry-interval": connectionRetryInterval})
+			select {
+			case <-time.After(connectionRetryInterval):
+				continue
+			case <-ctx.Done():
+				return ctx.Err()
 			}
 		}
+		probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+		logger.Info(ctx, "kv-connection-up")
+		break
 	}
+	return nil
 }
 
 // Adapter Utility methods ##### end   #########
@@ -478,9 +493,10 @@
 	ctx, cancel := context.WithCancel(context.Background())
 	defer cancel()
 
-	cf := config.NewAdapterFlags()
+	cf := &config.AdapterFlags{}
+	cf.ParseCommandArguments(os.Args[1:])
+
 	defaultAppName := cf.InstanceID + "_" + getVerifiedCodeVersion(ctx)
-	cf.ParseCommandArguments()
 
 	// Setup logging
 
diff --git a/docker/Dockerfile.openonu b/docker/Dockerfile.openonu
index 34664cc..43c959c 100755
--- a/docker/Dockerfile.openonu
+++ b/docker/Dockerfile.openonu
@@ -40,13 +40,13 @@
 RUN \
 CGO_ENABLED=$CGO_PARAMETER go build $EXTRA_GO_BUILD_TAGS -mod=vendor -o /app/openonu \
 -ldflags \
-"-X github.com/opencord/voltha-lib-go/v5/pkg/version.version=$org_label_schema_version \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.vcsRef=$org_label_schema_vcs_ref \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.vcsDirty=$org_opencord_vcs_dirty \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.goVersion=$(go version 2>&1 | sed -E  's/.*go([0-9]+\.[0-9]+\.[0-9]+).*/\1/g') \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.os=$(go env GOHOSTOS) \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.arch=$(go env GOHOSTARCH) \
--X github.com/opencord/voltha-lib-go/v5/pkg/version.buildTime=$org_label_schema_build_date" \
+"-X github.com/opencord/voltha-lib-go/v7/pkg/version.version=$org_label_schema_version \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.vcsRef=$org_label_schema_vcs_ref \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.vcsDirty=$org_opencord_vcs_dirty \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.goVersion=$(go version 2>&1 | sed -E  's/.*go([0-9]+\.[0-9]+\.[0-9]+).*/\1/g') \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.os=$(go env GOHOSTOS) \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.arch=$(go env GOHOSTARCH) \
+-X github.com/opencord/voltha-lib-go/v7/pkg/version.buildTime=$org_label_schema_build_date" \
 ./cmd/openonu-adapter/
 
 WORKDIR /app
diff --git a/go.mod b/go.mod
index 5a001ff..4e92e23 100644
--- a/go.mod
+++ b/go.mod
@@ -2,16 +2,22 @@
 
 go 1.16
 
+replace (
+	github.com/coreos/bbolt v1.3.4 => go.etcd.io/bbolt v1.3.4
+	go.etcd.io/bbolt v1.3.4 => github.com/coreos/bbolt v1.3.4
+	google.golang.org/grpc => google.golang.org/grpc v1.25.1
+)
+
 require (
 	github.com/boguslaw-wojcik/crc32a v1.0.0
 	github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73
 	github.com/gogo/protobuf v1.3.2
-	github.com/golang/protobuf v1.3.2
+	github.com/golang/protobuf v1.5.2
 	github.com/google/gopacket v1.1.17
 	github.com/looplab/fsm v0.2.0
 	github.com/opencord/omci-lib-go v1.3.3
-	github.com/opencord/voltha-lib-go/v5 v5.0.5
-	github.com/opencord/voltha-protos/v4 v4.2.0
-	github.com/stretchr/testify v1.6.1
-	google.golang.org/grpc v1.25.1 // indirect
+	github.com/opencord/voltha-lib-go/v7 v7.0.0
+	github.com/opencord/voltha-protos/v5 v5.0.0
+	github.com/stretchr/testify v1.7.0
+	google.golang.org/grpc v1.41.0
 )
diff --git a/go.sum b/go.sum
index 4b6ca32..dadc804 100644
--- a/go.sum
+++ b/go.sum
@@ -1,48 +1,47 @@
-cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
-github.com/DataDog/zstd v1.3.6-0.20190409195224-796139022798/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo=
-github.com/DataDog/zstd v1.4.1 h1:3oxKN3wbHibqx897utPC2LTQU4J+IHWWJO+glkAkpFM=
-github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo=
-github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE=
-github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
-github.com/Shopify/sarama v1.23.1 h1:XxJBCZEoWJtoWjf/xRbmGUpAmTZGnuuF0ON0EvxxBrs=
-github.com/Shopify/sarama v1.23.1/go.mod h1:XLH1GYJnLVE0XCr6KdJGVJRTwY30moWNJ4sERjXX6fs=
+cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
+dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
+github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
+github.com/HdrHistogram/hdrhistogram-go v1.1.0 h1:6dpdDPTRoo78HxAJ6T1HfMiKSnqhgRRqzCuPshRkQ7I=
+github.com/HdrHistogram/hdrhistogram-go v1.1.0/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo=
+github.com/Shopify/sarama v1.29.1 h1:wBAacXbYVLmWieEA/0X/JagDdCZ8NVFOfS6l6+2u5S0=
+github.com/Shopify/sarama v1.29.1/go.mod h1:mdtqvCSg8JOxk8PmpTNGyo6wzd4BMm4QXSfDnTXmgkE=
 github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc=
 github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
 github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1 h1:+JkXLHME8vLJafGhOH4aoV2Iu8bR55nU6iKMVfYVLjY=
 github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1/go.mod h1:nuudZmJhzWtx2212z+pkuy7B6nkBqa+xwNXZHL1j8cg=
+github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw=
 github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
 github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
+github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
+github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8=
+github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
 github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
-github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0=
 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
-github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
+github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
+github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
 github.com/boguslaw-wojcik/crc32a v1.0.0 h1:rZUcnG4WkADJBW8tmb5ZWFN30w/mHV8+/hzM66X2ptQ=
 github.com/boguslaw-wojcik/crc32a v1.0.0/go.mod h1:BnG1x2VM7pNqIjOAHQKMu4NZ0Rn7cCxr+BmuXIbNOhM=
 github.com/boljen/go-bitmap v0.0.0-20151001105940-23cd2fb0ce7d/go.mod h1:f1iKL6ZhUWvbk7PdWVmOaak10o86cqMUYEmn1CZNGEI=
 github.com/bsm/sarama-cluster v2.1.15+incompatible h1:RkV6WiNRnqEEbp81druK8zYhmnIgdOjqSVi0+9Cnl2A=
 github.com/bsm/sarama-cluster v2.1.15+incompatible/go.mod h1:r7ao+4tTNXvWm+VRpRJchr2kQhqxgmAp2iEX5W96gMM=
-github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72 h1:fUmDBbSvv1uOzo/t8WaxZMVb7BxJ8JECo5lGoR9c5bA=
-github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72/go.mod h1:OEE5igu/CDjGegM1Jn6ZMo7R6LlV/JChAkjfQQIRLpg=
 github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
-github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
-github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
+github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY=
+github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
 github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73 h1:q1g9lSyo/nOIC3W5E3FK3Unrz8b9LdLXCyuC+ZcpPC0=
 github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73/go.mod h1:507vXsotcZop7NZfBWdhPmVeOse4ko2R7AagJYrpoEg=
-github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
-github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y=
-github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
-github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w=
-github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
-github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY=
-github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
-github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
-github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a h1:W8b4lQ4tFF21aspRGoBuCNV6V2fFJBF+pm1J6OY8Lys=
-github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
-github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
-github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea h1:n2Ltr3SrfQlf/9nOna1DoGKxLx3qTSI8Ttl6Xrqp6mw=
-github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
-github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
+github.com/coreos/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
+github.com/coreos/etcd v3.3.25+incompatible h1:0GQEw6h3YnuOVdtwygkIfJ+Omx0tZ8/QkVyXI4LkbeY=
+github.com/coreos/etcd v3.3.25+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
+github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM=
+github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
+github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf h1:iW4rZ826su+pqaw19uhpSCzhj44qo35pNgKFGqzDKkU=
+github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
 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=
@@ -50,9 +49,8 @@
 github.com/deckarep/golang-set v1.7.1/go.mod h1:93vsz/8Wt4joVM7c2AVqh+YRMiUSc14yDtF28KmMOgQ=
 github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM=
 github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
-github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 h1:qk/FSDDxo05wdJH28W+p5yivv7LuLYLRXPPD8KQCtZs=
-github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
-github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
+github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo=
+github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
 github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q=
 github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
 github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw=
@@ -61,81 +59,116 @@
 github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I=
 github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
 github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
-github.com/frankban/quicktest v1.5.0 h1:Tb4jWdSpdjKzTUicPnY61PZxKbDoGa7ABbrReT3gQVY=
-github.com/frankban/quicktest v1.5.0/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o=
+github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k=
+github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
+github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g=
+github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY=
+github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k=
 github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
+github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4=
+github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
 github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
 github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY=
 github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
+github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
+github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
 github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
+github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE=
 github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
-github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
-github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
 github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
 github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
+github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k=
 github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58=
 github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
-github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903 h1:LbsanbbD6LieFkXbj9YNNBupiGHJgFeLpO0j0Fza1h8=
-github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE=
+github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
 github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
+github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
 github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
 github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs=
 github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4=
-github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo=
-github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
+github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
+github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
+github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
+github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
+github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
+github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
+github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw=
+github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
+github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA=
+github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
+github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4=
+github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA=
 github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
-github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg=
+github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
 github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
+github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU=
+github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
 github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
 github.com/google/gopacket v1.1.17 h1:rMrlX2ZY2UbvT+sdz3+6J+pp2z+msCq9MxTU6ymxbBY=
 github.com/google/gopacket v1.1.17/go.mod h1:UdDNZ1OO62aGYVnPhxT1U6aI7ukYtA/kB8vaU0diBUM=
-github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
-github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c h1:Lh2aW+HnU2Nbe1gqD9SOJLJxW1jBMmQOktN2acDyJk8=
-github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
-github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 h1:z53tR0945TRRQO/fLEVPI6SMv7ZflF0TEaTAoU7tOzg=
-github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
+github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
+github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4=
+github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM=
+github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
+github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw=
+github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y=
 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho=
 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
-github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1QAp/SlnNrZhI=
-github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
-github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE=
-github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
+github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo=
+github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
+github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE=
+github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
 github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
-github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
-github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
+github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8=
+github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs=
+github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo=
+github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM=
 github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8=
 github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
-github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo=
-github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
+github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o=
+github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg=
+github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA=
+github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc=
+github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY=
+github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc=
+github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ=
+github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8=
+github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
 github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
-github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62Fo=
-github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ=
+github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
 github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
-github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
-github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
+github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM=
+github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes=
 github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
 github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
-github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk=
+github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8=
+github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg=
 github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
 github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
 github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
 github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI=
 github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
 github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
-github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
 github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
+github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
 github.com/looplab/fsm v0.2.0 h1:M8hf5EF4AYLcT1FNKVUX8nu7D0xfp291iGeuigSxfrw=
 github.com/looplab/fsm v0.2.0/go.mod h1:p+IElwgCnAByqr2DWMuNbPjgMwqcHvTRZZn3dvKEke0=
-github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU=
-github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
-github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
 github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
 github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
 github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
@@ -145,202 +178,262 @@
 github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
 github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
-github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
-github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw=
+github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno=
+github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
+github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=
+github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU=
 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/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
+github.com/onsi/ginkgo v1.16.4 h1:29JGrr5oVBm5ulCWet69zQkzWipVXIol6ygQUe/EzNc=
+github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0=
+github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
+github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
+github.com/onsi/gomega v1.14.0 h1:ep6kpPVwmr/nTbklSx2nrLNSIO62DoYAhnPNIMhK8gI=
+github.com/onsi/gomega v1.14.0/go.mod h1:cIuvLEne0aoVhAgh/O6ac0Op8WWw9H6eYCriF+tEHG0=
 github.com/opencord/omci-lib-go v1.3.3 h1:BzywkXVHSphhpl9hHHOJxc9IQ7MrzIB3aY3LG1wbwuk=
 github.com/opencord/omci-lib-go v1.3.3/go.mod h1:moNk4j00XaM3olsu4a8lRAqGmcZJoyIbxtSr+VERLq4=
-github.com/opencord/voltha-lib-go/v5 v5.0.5 h1:S3m984EaHNGJY68Hrv9LOf0rwd92UiVLFBBbTaL+fNg=
-github.com/opencord/voltha-lib-go/v5 v5.0.5/go.mod h1:i1fwPMicFccG38L200+IQAlfHSbszWg//jF1pDQxTPQ=
-github.com/opencord/voltha-protos/v4 v4.2.0 h1:QJZqHPRKa1E1xh40F3UA4xSjBI+6EmW7OfIcJqPNc4A=
-github.com/opencord/voltha-protos/v4 v4.2.0/go.mod h1:wNzWqmTwe7+DbYbpmOX6eMlglREtMkNxIDv3lyI2bco=
-github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU=
+github.com/opencord/voltha-lib-go/v7 v7.0.0 h1:xDMFbXdku7GOsJeMlsxw8WHAZINBZhy+8m9Hyqt+zdk=
+github.com/opencord/voltha-lib-go/v7 v7.0.0/go.mod h1:iZueJRS4XJ3rpm3iy0Zdnhz1lG5bWx2pZoPormwgUKk=
+github.com/opencord/voltha-protos/v5 v5.0.0 h1:US2k7qYPMnOueOCrprq9LjuMT3wK9uyxPwAVwjMmKhc=
+github.com/opencord/voltha-protos/v5 v5.0.0/go.mod h1:uVKXQB499Ir6G+rc47dSThNja1S4Vy3h9JLSDuJGmzI=
 github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
+github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc=
 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE=
-github.com/pierrec/lz4 v0.0.0-20190327172049-315a67e90e41/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
-github.com/pierrec/lz4 v2.3.0+incompatible h1:CZzRn4Ut9GbUkHlQ7jqBXeZQV41ZSKWFc302ZU6lUTk=
-github.com/pierrec/lz4 v2.3.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
+github.com/pierrec/lz4 v2.6.0+incompatible h1:Ix9yFKn1nSPBLFl/yZknTp8TU5G4Ps0JDmguYK6iH1A=
+github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
 github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=
 github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
+github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
+github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
 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/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
-github.com/prometheus/client_golang v1.0.0 h1:vrDKnkGzuGvhNAL56c7DBz29ZL+KxnoR0x7enabFceM=
 github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
+github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M=
+github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ=
+github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0=
 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
 github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM=
 github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/common v0.4.1 h1:K0MGApIoQvMw27RTdJkPbr3JZ7DNbtxQNyi5STVM6Kw=
+github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
+github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
 github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo=
+github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ=
+github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc=
 github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.2 h1:6LJUbpNm42llc4HRCuvApCSWB/WfhuNo9K98Q9sNGfs=
 github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
-github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
-github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563 h1:dY6ETXrvDG7Sa4vE8ZQG4yqWg6UnOcbqTAahkV813vQ=
-github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
-github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
-github.com/sirupsen/logrus v1.2.0 h1:juTguoYk5qI21pwyTXY3B3Y5cOTH3ZUyZCg1v/mihuo=
+github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4=
+github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
+github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
 github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
-github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E=
-github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
-github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
-github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg=
-github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
+github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I=
+github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88=
+github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js=
+github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0=
 github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+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/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
 github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
 github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
 github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
-github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0=
 github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
-github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8 h1:ndzgwNDnKIqyCvHTXaCqh9KlOWKvBry6nuXMJmonVsE=
-github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
-github.com/uber/jaeger-client-go v2.23.1+incompatible h1:uArBYHQR0HqLFFAypI7RsWTzPSj/bDpmZZuQjMLSg1A=
-github.com/uber/jaeger-client-go v2.23.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk=
-github.com/uber/jaeger-lib v2.2.0+incompatible h1:MxZXOiR2JuoANZ3J6DE/U0kSFv/eJ/GfSYVCjK7dyaw=
-github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
-github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
-github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
-github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
+github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY=
+github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/uber/jaeger-client-go v2.29.1+incompatible h1:R9ec3zO3sGpzs0abd43Y+fBZRJ9uiH6lXyR/+u6brW4=
+github.com/uber/jaeger-client-go v2.29.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk=
+github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg=
+github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
+github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
+github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
 github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
 github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
 github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
-go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk=
-go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
-go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522 h1:GQU7sDaYW5CN6WpkPCWZQrZ/dEO6NDc2cHfd9bbsqso=
-go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522/go.mod h1:uQccEQvXbbNc3vI3weFUy1S42v0dtl0CtCePpj8fRSk=
-go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
-go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU=
+github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
+go.etcd.io/bbolt v1.3.4 h1:hi1bXHMVrlQh6WwxAy+qZCV/SYIlqo+Ushwdpa4tAKg=
+go.etcd.io/bbolt v1.3.4/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ=
+go.etcd.io/etcd v3.3.25+incompatible h1:V1RzkZJj9LqsJRy+TUBgpWSbZXITLB819lstuTFoZOY=
+go.etcd.io/etcd v3.3.25+incompatible/go.mod h1:yaeTdrJi5lOmYerz05bd8+V7KubZs8YSFZfzsF9A6aI=
 go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
+go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
+go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0=
+go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
 go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
-go.uber.org/multierr v1.2.0 h1:6I+W7f5VwC5SV9dNrZ3qXrDB9mD0dyGOi/ZJmYw03T4=
-go.uber.org/multierr v1.2.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
-go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM=
+go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4=
+go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
 go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
+go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4=
+go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI=
 golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
 golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190404164418-38d8ce5564a5/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
-golang.org/x/crypto v0.0.0-20191001170739-f9e2070545dc/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
 golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI=
 golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e h1:gsTQYXdTw2Gq7RBsWvlQ91b+aEQ6bXFUngBGuR8sPpI=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
+golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
 golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
+golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY=
+golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs=
+golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
+golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
 golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
 golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o=
+golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
 golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
 golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
+golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
 golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
 golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
 golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190930134127-c5a3c61f89f3/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
 golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20201021035429-f5854403a974 h1:IX6qOQeG5uLjB/hjjwjedwfjND0hgjPMMyO1RoIXQNI=
+golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
+golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
 golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
+golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
+golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
+golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
+golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e h1:XpT3nA5TvE525Ne3hInMh6+GETgn27Zfm9dxsThnX2Q=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
 golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
+golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
+golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
 golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
 golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
 golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
 golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
 golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f h1:+Nyd8tzPX9R7BWHguqsrbFdRx3WQ/1ib8I44HXV5yTA=
+golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
 golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
 golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
 golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
-golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k=
 golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM=
-golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
-golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M=
+golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs=
+golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
 golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
 golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
 golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
 golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
 golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
+golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
 golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
+golang.org/x/tools v0.1.1 h1:wGiQel/hW0NnEkJUk8lbzkX2gFJU6PFxf1v5OlCfuOs=
+golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
 golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
 golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
 golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE=
 golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
+gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo=
+gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0=
+gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw=
+gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc=
 google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
-google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
 google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c h1:hrpEMCZ2O7DR5gC1n2AJGVhrwiEjOi35+jxtIuZpTMo=
-google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
-google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
-google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
-google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA=
+google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0 h1:5Tbluzus3QxoAJx4IefGt1W0HQZW4nuMrVk684jI74Q=
+google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc=
 google.golang.org/grpc v1.25.1 h1:wdKvqQk7IttEw92GoRyKG2IDrUIpgpj6H6m81yfeMW0=
 google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
+google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
+google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
+google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
+google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
+google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
+google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
+google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
+google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
+google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ=
+google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
 gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
 gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo=
 gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw=
-gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
+gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
 gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
-gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw=
-gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q=
-gopkg.in/jcmturner/goidentity.v3 v3.0.0 h1:1duIyWiTaYvVx3YX2CYtpJbUFd7/UuPYCfgXtQ3VTbI=
-gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4=
-gopkg.in/jcmturner/gokrb5.v7 v7.2.3/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM=
-gopkg.in/jcmturner/gokrb5.v7 v7.3.0 h1:0709Jtq/6QXEuWRfAm260XqlpcwL1vxtO1tUE2qK8Z4=
-gopkg.in/jcmturner/gokrb5.v7 v7.3.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM=
-gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU=
-gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8=
-gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
 gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
 gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
-gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
 gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
 gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
 gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10=
+gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
 gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
+gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
+gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
 gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
-honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
 honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
 honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs=
-sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
+rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
+sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q=
+sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc=
diff --git a/internal/pkg/config/config.go b/internal/pkg/config/config.go
index b61a909..aff1de3 100644
--- a/internal/pkg/config/config.go
+++ b/internal/pkg/config/config.go
@@ -19,73 +19,24 @@
 
 import (
 	"flag"
-	"fmt"
 	"os"
 	"time"
 )
 
 // Open ONU default constants
 const (
-	etcdStoreName               = "etcd"
-	defaultInstanceid           = "openonu"
-	defaultKafkaadapteraddress  = "127.0.0.1:9092"
-	defaultKafkaclusteraddress  = "127.0.0.1:9092"
-	defaultKvstoretype          = etcdStoreName
-	defaultKvstoretimeout       = 5 * time.Second
-	defaultKvstoreaddress       = "127.0.0.1:2379"
-	defaultLoglevel             = "WARN"
-	defaultBanner               = false
-	defaultDisplayVersionOnly   = false
-	defaultAccIncrEvto          = false
-	defaultTopic                = "openonu"
-	defaultCoreTopic            = "rwcore"
-	defaultEventTopic           = "voltha.events"
-	defaultOnunumber            = 1
-	defaultProbeHost            = ""
-	defaultProbePort            = 8080
-	defaultLiveProbeInterval    = 60 * time.Second
-	defaultNotLiveProbeInterval = 5 * time.Second // Probe more frequently when not alive
-	//defaultHearbeatFailReportInterval is the time in seconds the adapter will keep checking the hardware for heartbeat.
-	defaultHearbeatCheckInterval = 30 * time.Second
-	// defaultHearbeatFailReportInterval is the time adapter will wait before updating the state to the core.
-	defaultHearbeatFailReportInterval = 180 * time.Second
-	//defaultKafkaReconnectRetries -1: reconnect endlessly.
-	defaultKafkaReconnectRetries      = -1
-	defaultCurrentReplica             = 1
-	defaultTotalReplicas              = 1
-	defaultMaxTimeoutInterAdapterComm = 30 * time.Second
-	defaultMaxTimeoutReconciling      = 10 * time.Second
-	defaultOnuVendorIds               = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
-
-	// For Tracing
-	defaultTraceEnabled          = false
-	defaultTraceAgentAddress     = "127.0.0.1:6831"
-	defaultLogCorrelationEnabled = true
-
-	defaultMetricsEnabled     = false
-	defaultMibAuditInterval   = 0
-	defaultAlarmAuditInterval = 300 * time.Second
-
-	defaultOmciTimeout          = 3 * time.Second
-	defaultDlToAdapterTimeout   = 10 * time.Second
-	defaultDlToOnuTimeoutPer4MB = 60 * time.Minute //assumed for 4 MB of the image
-	//Mask to indicate which possibly active ONU UNI state  is really reported to the core
-	// compare python code - at the moment restrict active state to the first ONU UNI port
-	// check is limited to max 16 uni ports - cmp above UNI limit!!!
-	defaultUniPortMask = 0x0001
+	EtcdStoreName = "etcd"
+	OnuVendorIds  = "OPEN,ALCL,BRCM,TWSH,ALPH,ISKT,SFAA,BBSM,SCOM,ARPX,DACM,ERSN,HWTC,CIGG,ADTN,ARCA,AVMG"
 )
 
 // AdapterFlags represents the set of configurations used by the read-write adaptercore service
 type AdapterFlags struct {
 	// Command line parameters
 	InstanceID                  string
-	KafkaAdapterAddress         string
 	KafkaClusterAddress         string // NOTE this is unused across the adapter
 	KVStoreType                 string
 	KVStoreTimeout              time.Duration
 	KVStoreAddress              string
-	Topic                       string
-	CoreTopic                   string
 	EventTopic                  string
 	LogLevel                    string
 	OnuNumber                   int
@@ -114,169 +65,220 @@
 	DownloadToAdapterTimeout    time.Duration
 	DownloadToOnuTimeout4MB     time.Duration
 	UniPortMask                 int
-}
-
-// NewAdapterFlags returns a new RWCore config
-func NewAdapterFlags() *AdapterFlags {
-	var adapterFlags = AdapterFlags{ // Default values
-		InstanceID:                  defaultInstanceid,
-		KafkaAdapterAddress:         defaultKafkaadapteraddress,
-		KafkaClusterAddress:         defaultKafkaclusteraddress,
-		KVStoreType:                 defaultKvstoretype,
-		KVStoreTimeout:              defaultKvstoretimeout,
-		KVStoreAddress:              defaultKvstoreaddress,
-		Topic:                       defaultTopic,
-		CoreTopic:                   defaultCoreTopic,
-		EventTopic:                  defaultEventTopic,
-		LogLevel:                    defaultLoglevel,
-		OnuNumber:                   defaultOnunumber,
-		Banner:                      defaultBanner,
-		DisplayVersionOnly:          defaultDisplayVersionOnly,
-		AccIncrEvto:                 defaultAccIncrEvto,
-		ProbeHost:                   defaultProbeHost,
-		ProbePort:                   defaultProbePort,
-		LiveProbeInterval:           defaultLiveProbeInterval,
-		NotLiveProbeInterval:        defaultNotLiveProbeInterval,
-		HeartbeatCheckInterval:      defaultHearbeatCheckInterval,
-		HeartbeatFailReportInterval: defaultHearbeatFailReportInterval,
-		KafkaReconnectRetries:       defaultKafkaReconnectRetries,
-		CurrentReplica:              defaultCurrentReplica,
-		TotalReplicas:               defaultTotalReplicas,
-		MaxTimeoutInterAdapterComm:  defaultMaxTimeoutInterAdapterComm,
-		MaxTimeoutReconciling:       defaultMaxTimeoutReconciling,
-		TraceEnabled:                defaultTraceEnabled,
-		TraceAgentAddress:           defaultTraceAgentAddress,
-		LogCorrelationEnabled:       defaultLogCorrelationEnabled,
-		OnuVendorIds:                defaultOnuVendorIds,
-		MetricsEnabled:              defaultMetricsEnabled,
-		MibAuditInterval:            defaultMibAuditInterval,
-		AlarmAuditInterval:          defaultAlarmAuditInterval,
-		OmciTimeout:                 defaultOmciTimeout,
-		DownloadToAdapterTimeout:    defaultDlToAdapterTimeout,
-		DownloadToOnuTimeout4MB:     defaultDlToOnuTimeoutPer4MB,
-		UniPortMask:                 defaultUniPortMask,
-	}
-	return &adapterFlags
+	MinBackoffRetryDelay        time.Duration
+	MaxBackoffRetryDelay        time.Duration
+	AdapterEndpoint             string
+	GrpcAddress                 string
+	CoreEndpoint                string
+	RPCTimeout                  time.Duration
 }
 
 // ParseCommandArguments parses the arguments when running read-write adaptercore service
-func (so *AdapterFlags) ParseCommandArguments() {
+func (so *AdapterFlags) ParseCommandArguments(args []string) {
 
-	help := "Kafka - Adapter messaging address"
-	flag.StringVar(&(so.KafkaAdapterAddress), "kafka_adapter_address", defaultKafkaadapteraddress, help)
+	fs := flag.NewFlagSet(os.Args[0], flag.ExitOnError)
 
-	help = "Kafka - Cluster messaging address"
-	flag.StringVar(&(so.KafkaClusterAddress), "kafka_cluster_address", defaultKafkaclusteraddress, help)
+	fs.StringVar(&(so.KafkaClusterAddress),
+		"kafka_cluster_address",
+		"127.0.0.1:9092",
+		"Kafka - Cluster messaging address")
 
-	help = "Open ONU topic"
-	baseAdapterTopic := flag.String("adapter_topic", defaultTopic, help)
+	fs.StringVar(&(so.EventTopic),
+		"event_topic",
+		"voltha.events",
+		"Event topic")
 
-	help = "Core topic"
-	flag.StringVar(&(so.CoreTopic), "core_topic", defaultCoreTopic, help)
+	fs.StringVar(&(so.KVStoreType),
+		"kv_store_type",
+		EtcdStoreName,
+		"KV store type")
 
-	help = "Event topic"
-	flag.StringVar(&(so.EventTopic), "event_topic", defaultEventTopic, help)
+	fs.DurationVar(&(so.KVStoreTimeout),
+		"kv_store_request_timeout",
+		5*time.Second,
+		"The default timeout when making a kv store request")
 
-	help = "KV store type"
-	flag.StringVar(&(so.KVStoreType), "kv_store_type", defaultKvstoretype, help)
+	fs.StringVar(&(so.KVStoreAddress),
+		"kv_store_address",
+		"127.0.0.1:2379",
+		"KV store address")
 
-	help = "The default timeout when making a kv store request"
-	flag.DurationVar(&(so.KVStoreTimeout), "kv_store_request_timeout", defaultKvstoretimeout, help)
+	fs.StringVar(&(so.LogLevel),
+		"log_level",
+		"WARN",
+		"Log level")
 
-	help = "KV store address"
-	flag.StringVar(&(so.KVStoreAddress), "kv_store_address", defaultKvstoreaddress, help)
+	fs.IntVar(&(so.OnuNumber),
+		"onu_number",
+		1,
+		"Number of ONUs")
 
-	help = "Log level"
-	flag.StringVar(&(so.LogLevel), "log_level", defaultLoglevel, help)
+	fs.BoolVar(&(so.Banner),
+		"banner",
+		false,
+		"Show startup banner log lines")
 
-	help = "Number of ONUs"
-	flag.IntVar(&(so.OnuNumber), "onu_number", defaultOnunumber, help)
+	fs.BoolVar(&(so.DisplayVersionOnly),
+		"version",
+		false,
+		"Show version information and exit")
 
-	help = "Show startup banner log lines"
-	flag.BoolVar(&(so.Banner), "banner", defaultBanner, help)
+	fs.BoolVar(&(so.AccIncrEvto),
+		"accept_incr_evto",
+		false,
+		"Acceptance of incremental EVTOCD configuration")
 
-	help = "Show version information and exit"
-	flag.BoolVar(&(so.DisplayVersionOnly), "version", defaultDisplayVersionOnly, help)
+	fs.StringVar(&(so.ProbeHost),
+		"probe_host",
+		"",
+		"The address on which to listen to answer liveness and readiness probe queries over HTTP")
 
-	help = "Acceptance of incremental EVTOCD configuration"
-	flag.BoolVar(&(so.AccIncrEvto), "accept_incr_evto", defaultAccIncrEvto, help)
+	fs.IntVar(&(so.ProbePort),
+		"probe_port",
+		8080,
+		"The port on which to listen to answer liveness and readiness probe queries over HTTP")
 
-	help = "The address on which to listen to answer liveness and readiness probe queries over HTTP"
-	flag.StringVar(&(so.ProbeHost), "probe_host", defaultProbeHost, help)
+	fs.DurationVar(&(so.LiveProbeInterval),
+		"live_probe_interval",
+		60*time.Second,
+		"Number of seconds for the default liveliness check")
 
-	help = "The port on which to listen to answer liveness and readiness probe queries over HTTP"
-	flag.IntVar(&(so.ProbePort), "probe_port", defaultProbePort, help)
+	fs.DurationVar(&(so.NotLiveProbeInterval),
+		"not_live_probe_interval",
+		60*time.Second,
+		"Number of seconds for liveliness check if probe is not running")
 
-	help = "Number of seconds for the default liveliness check"
-	flag.DurationVar(&(so.LiveProbeInterval), "live_probe_interval", defaultLiveProbeInterval, help)
+	fs.DurationVar(&(so.HeartbeatCheckInterval),
+		"hearbeat_check_interval",
+		30*time.Second,
+		"Number of seconds for heartbeat check interval")
 
-	help = "Number of seconds for liveliness check if probe is not running"
-	flag.DurationVar(&(so.NotLiveProbeInterval), "not_live_probe_interval", defaultNotLiveProbeInterval, help)
+	fs.DurationVar(&(so.HeartbeatFailReportInterval),
+		"hearbeat_fail_interval",
+		30*time.Second,
+		"Number of seconds adapter has to wait before reporting core on the hearbeat check failure")
 
-	help = "Number of seconds for heartbeat check interval"
-	flag.DurationVar(&(so.HeartbeatCheckInterval), "hearbeat_check_interval", defaultHearbeatCheckInterval, help)
+	fs.IntVar(&(so.KafkaReconnectRetries),
+		"kafka_reconnect_retries",
+		-1,
+		"Number of retries to connect to Kafka")
 
-	help = "Number of seconds adapter has to wait before reporting core on the hearbeat check failure"
-	flag.DurationVar(&(so.HeartbeatFailReportInterval), "hearbeat_fail_interval", defaultHearbeatFailReportInterval, help)
+	fs.IntVar(&(so.CurrentReplica),
+		"current_replica",
+		1,
+		"Replica number of this particular instance")
 
-	help = "Number of retries to connect to Kafka"
-	flag.IntVar(&(so.KafkaReconnectRetries), "kafka_reconnect_retries", defaultKafkaReconnectRetries, help)
+	fs.IntVar(&(so.TotalReplicas),
+		"total_replica",
+		1,
+		"Total number of instances for this adapter")
 
-	help = "Replica number of this particular instance"
-	flag.IntVar(&(so.CurrentReplica), "current_replica", defaultCurrentReplica, help)
+	fs.DurationVar(&(so.MaxTimeoutInterAdapterComm),
+		"max_timeout_interadapter_comm",
+		30*time.Second,
+		"Maximum Number of seconds for the default interadapter communication timeout")
 
-	help = "Total number of instances for this adapter"
-	flag.IntVar(&(so.TotalReplicas), "total_replica", defaultTotalReplicas, help)
+	fs.DurationVar(&(so.MaxTimeoutReconciling),
+		"max_timeout_reconciling",
+		10*time.Second,
+		"Maximum Number of seconds for the default ONU reconciling timeout")
 
-	help = "Maximum Number of seconds for the default interadapter communication timeout"
-	flag.DurationVar(&(so.MaxTimeoutInterAdapterComm), "max_timeout_interadapter_comm",
-		defaultMaxTimeoutInterAdapterComm, help)
+	fs.BoolVar(&(so.TraceEnabled),
+		"trace_enabled",
+		false,
+		"Whether to send logs to tracing agent")
 
-	help = "Maximum Number of seconds for the default ONU reconciling timeout"
-	flag.DurationVar(&(so.MaxTimeoutReconciling), "max_timeout_reconciling",
-		defaultMaxTimeoutReconciling, help)
+	fs.StringVar(&(so.TraceAgentAddress),
+		"trace_agent_address",
+		"127.0.0.1:6831",
+		"The address of tracing agent to which span info should be sent")
 
-	help = "Whether to send logs to tracing agent"
-	flag.BoolVar(&(so.TraceEnabled), "trace_enabled", defaultTraceEnabled, help)
+	fs.BoolVar(&(so.LogCorrelationEnabled),
+		"log_correlation_enabled",
+		true,
+		"Whether to enrich log statements with fields denoting operation being executed for achieving correlation")
 
-	help = "The address of tracing agent to which span info should be sent"
-	flag.StringVar(&(so.TraceAgentAddress), "trace_agent_address", defaultTraceAgentAddress, help)
+	fs.StringVar(&(so.OnuVendorIds),
+		"allowed_onu_vendors",
+		OnuVendorIds,
+		"List of Allowed ONU Vendor Ids")
 
-	help = "Whether to enrich log statements with fields denoting operation being executed for achieving correlation"
-	flag.BoolVar(&(so.LogCorrelationEnabled), "log_correlation_enabled", defaultLogCorrelationEnabled, help)
+	fs.BoolVar(&(so.MetricsEnabled),
+		"metrics_enabled",
+		false,
+		"Whether to enable metrics collection")
 
-	help = "List of Allowed ONU Vendor Ids"
-	flag.StringVar(&(so.OnuVendorIds), "allowed_onu_vendors", defaultOnuVendorIds, help)
+	fs.DurationVar(&(so.MibAuditInterval),
+		"mib_audit_interval",
+		300*time.Second,
+		"Mib Audit Interval in seconds - the value zero will disable Mib Audit")
 
-	help = "Whether to enable metrics collection"
-	flag.BoolVar(&(so.MetricsEnabled), "metrics_enabled", defaultMetricsEnabled, help)
+	fs.DurationVar(&(so.OmciTimeout),
+		"omci_timeout",
+		3*time.Second,
+		"OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU")
 
-	help = "Mib Audit Interval in seconds - the value zero will disable Mib Audit"
-	flag.DurationVar(&(so.MibAuditInterval), "mib_audit_interval", defaultMibAuditInterval, help)
+	fs.DurationVar(&(so.AlarmAuditInterval),
+		"alarm_audit_interval",
+		300*time.Second,
+		"Alarm Audit Interval in seconds - the value zero will disable alarm audit")
 
-	help = "OMCI timeout duration - this timeout value is used on the OMCI channel for waiting on response from ONU"
-	flag.DurationVar(&(so.OmciTimeout), "omci_timeout", defaultOmciTimeout, help)
+	fs.DurationVar(&(so.DownloadToAdapterTimeout),
+		"download_to_adapter_timeout",
+		10*time.Second,
+		"File download to adapter timeout in seconds")
 
-	help = "Alarm Audit Interval in seconds - the value zero will disable alarm audit"
-	flag.DurationVar(&(so.AlarmAuditInterval), "alarm_audit_interval", defaultAlarmAuditInterval, help)
+	fs.DurationVar(&(so.DownloadToOnuTimeout4MB),
+		"download_to_onu_timeout_4MB",
+		60*time.Minute,
+		"File download to ONU timeout in minutes for a block of 4MB")
 
-	help = "File download to adapter timeout in seconds"
-	flag.DurationVar(&(so.DownloadToAdapterTimeout), "download_to_adapter_timeout", defaultDlToAdapterTimeout, help)
+	//Mask to indicate which possibly active ONU UNI state  is really reported to the core
+	// compare python code - at the moment restrict active state to the first ONU UNI port
+	// check is limited to max 16 uni ports - cmp above UNI limit!!!
+	fs.IntVar(&(so.UniPortMask),
+		"uni_port_mask",
+		0x0001,
+		"The bitmask to identify UNI ports that need to be enabled")
 
-	help = "File download to ONU timeout in minutes for a block of 4MB"
-	flag.DurationVar(&(so.DownloadToOnuTimeout4MB), "download_to_onu_timeout_4MB", defaultDlToOnuTimeoutPer4MB, help)
+	fs.StringVar(&(so.GrpcAddress),
+		"grpc_address",
+		":50060",
+		"Adapter GRPC Server address")
 
-	help = "The bitmask to identify UNI ports that need to be enabled"
-	flag.IntVar(&(so.UniPortMask), "uni_port_mask", defaultUniPortMask, help)
+	fs.StringVar(&(so.CoreEndpoint),
+		"core_endpoint",
+		":55555",
+		"Core endpoint")
 
-	flag.Parse()
+	fs.StringVar(&(so.AdapterEndpoint),
+		"adapter_endpoint",
+		"",
+		"Adapter Endpoint")
+
+	fs.DurationVar(&(so.RPCTimeout),
+		"rpc_timeout",
+		10*time.Second,
+		"The default timeout when making an RPC request")
+
+	fs.DurationVar(&(so.MinBackoffRetryDelay),
+		"min_retry_delay",
+		500*time.Millisecond,
+		"The minimum number of milliseconds to delay before a connection retry attempt")
+
+	fs.DurationVar(&(so.MaxBackoffRetryDelay),
+		"max_retry_delay",
+		10*time.Second,
+		"The maximum number of milliseconds to delay before a connection retry attempt")
+
+	_ = fs.Parse(args)
 	containerName := getContainerInfo()
 	if len(containerName) > 0 {
 		so.InstanceID = containerName
+	} else {
+		so.InstanceID = "openonu"
 	}
 
-	so.Topic = fmt.Sprintf("%s_%d", *baseAdapterTopic, int32(so.CurrentReplica))
-
 }
 
 func getContainerInfo() string {
diff --git a/internal/pkg/onuadaptercore/adapter_download_manager.go b/internal/pkg/onuadaptercore/adapter_download_manager.go
index 6803aed..6328f72 100644
--- a/internal/pkg/onuadaptercore/adapter_download_manager.go
+++ b/internal/pkg/onuadaptercore/adapter_download_manager.go
@@ -30,9 +30,9 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 // ### downloadToAdapter related definitions  ####
diff --git a/internal/pkg/onuadaptercore/alarm_manager.go b/internal/pkg/onuadaptercore/alarm_manager.go
index 0dbc5b8..b9d4d68 100644
--- a/internal/pkg/onuadaptercore/alarm_manager.go
+++ b/internal/pkg/onuadaptercore/alarm_manager.go
@@ -28,9 +28,9 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/common.go b/internal/pkg/onuadaptercore/common.go
index 0a39a7e..b0b10e3 100644
--- a/internal/pkg/onuadaptercore/common.go
+++ b/internal/pkg/onuadaptercore/common.go
@@ -18,7 +18,7 @@
 package adaptercoreonu
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 var logger log.CLogger
diff --git a/internal/pkg/onuadaptercore/device_handler.go b/internal/pkg/onuadaptercore/device_handler.go
index cfe99c9..2c5a4c8 100644
--- a/internal/pkg/onuadaptercore/device_handler.go
+++ b/internal/pkg/onuadaptercore/device_handler.go
@@ -25,25 +25,23 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-protos/v4/go/tech_profile"
+	"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
+	"github.com/opencord/voltha-protos/v5/go/tech_profile"
 
 	"github.com/gogo/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
 	"github.com/looplab/fsm"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	flow "github.com/opencord/voltha-lib-go/v5/pkg/flows"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	vc "github.com/opencord/voltha-protos/v4/go/common"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	oop "github.com/opencord/voltha-protos/v4/go/openolt"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	flow "github.com/opencord/voltha-lib-go/v7/pkg/flows"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	vc "github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	oop "github.com/opencord/voltha-protos/v5/go/openolt"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // Constants for timeouts
@@ -178,11 +176,11 @@
 	parentID         string
 	ponPortNumber    uint32
 
-	coreProxy    adapterif.CoreProxy
-	AdapterProxy adapterif.AdapterProxy
-	EventProxy   eventif.EventProxy
+	coreClient *vgrpc.Client
+	EventProxy eventif.EventProxy
 
 	pmConfigs *voltha.PmConfigs
+	config    *config.AdapterFlags
 
 	pOpenOnuAc      *OpenONUAC
 	pDeviceStateFsm *fsm.FSM
@@ -237,11 +235,11 @@
 }
 
 //newDeviceHandler creates a new device handler
-func newDeviceHandler(ctx context.Context, cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
+func newDeviceHandler(ctx context.Context, cc *vgrpc.Client, ep eventif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *deviceHandler {
 	var dh deviceHandler
-	dh.coreProxy = cp
-	dh.AdapterProxy = ap
+	dh.coreClient = cc
 	dh.EventProxy = ep
+	dh.config = adapter.config
 	cloned := (proto.Clone(device)).(*voltha.Device)
 	dh.deviceID = cloned.Id
 	dh.DeviceType = cloned.Type
@@ -327,7 +325,7 @@
 
 //adoptOrReconcileDevice adopts the ONU device
 func (dh *deviceHandler) adoptOrReconcileDevice(ctx context.Context, device *voltha.Device) {
-	logger.Debugw(ctx, "Adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
+	logger.Debugw(ctx, "adopt_or_reconcile_device", log.Fields{"device-id": device.Id, "Address": device.GetHostAndPort()})
 
 	logger.Debugw(ctx, "Device FSM: ", log.Fields{"state": string(dh.pDeviceStateFsm.Current())})
 	if dh.pDeviceStateFsm.Is(devStNull) {
@@ -338,7 +336,7 @@
 		// device.PmConfigs is not nil in cases when adapter restarts. We should not re-set the core again.
 		if device.PmConfigs == nil {
 			// Now, set the initial PM configuration for that device
-			if err := dh.coreProxy.DevicePMConfigUpdate(ctx, dh.pmConfigs); err != nil {
+			if err := dh.updatePMConfigInCore(ctx, dh.pmConfigs); err != nil {
 				logger.Errorw(ctx, "error updating pm config to core", log.Fields{"device-id": dh.deviceID, "err": err})
 			}
 		}
@@ -348,36 +346,26 @@
 
 }
 
-func (dh *deviceHandler) processInterAdapterOMCIReceiveMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgBody := msg.GetBody()
-	omciMsg := &ic.InterAdapterOmciMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, omciMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-omci-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
+func (dh *deviceHandler) handleOMCIIndication(ctx context.Context, msg *ic.OmciMessage) error {
 	/* msg print moved symmetrically to omci_cc, if wanted here as additional debug, than perhaps only based on additional debug setting!
 	//assuming omci message content is hex coded!
 	// with restricted output of 16(?) bytes would be ...omciMsg.Message[:16]
 	logger.Debugw(ctx, "inter-adapter-recv-omci", log.Fields{
 		"device-id": dh.deviceID, "RxOmciMessage": hex.EncodeToString(omciMsg.Message)})
 	*/
+
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
 	if pDevEntry != nil {
 		if pDevEntry.PDevOmciCC != nil {
-			return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), omciMsg.Message)
+			return pDevEntry.PDevOmciCC.receiveMessage(log.WithSpanFromContext(context.TODO(), ctx), msg.Message)
 		}
-		logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": omciMsg.Message})
+		logger.Debugw(ctx, "omciCC not ready to receive omci messages - incoming omci message ignored", log.Fields{"rxMsg": msg.Message})
 	}
 	logger.Errorw(ctx, "No valid OnuDevice -aborting", log.Fields{"device-id": dh.deviceID})
 	return fmt.Errorf("no valid OnuDevice: %s", dh.deviceID)
 }
 
-func (dh *deviceHandler) processInterAdapterTechProfileDownloadReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
-
+func (dh *deviceHandler) handleTechProfileDownloadRequest(ctx context.Context, techProfMsg *ic.TechProfileDownloadMessage) error {
 	logger.Infow(ctx, "tech-profile-download-request", log.Fields{"device-id": dh.deviceID})
 
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
@@ -400,14 +388,6 @@
 	// at least 'mib-downloaded' should be reached for processing of this specific ONU configuration
 	//  if (dh.deviceReason == "stopping-openomci") || (dh.deviceReason == "omci-admin-lock")
 
-	msgBody := msg.GetBody()
-	techProfMsg := &ic.InterAdapterTechProfileDownloadMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, techProfMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-techprof-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	// we have to lock access to TechProfile processing based on different messageType calls or
 	// even to fast subsequent calls of the same messageType as well as OnuKVStore processing due
 	// to possible concurrent access by flow processing
@@ -429,7 +409,7 @@
 	if bTpModify := pDevEntry.updateOnuUniTpPath(ctx, uniID, uint8(tpID), techProfMsg.TpInstancePath); bTpModify {
 
 		switch tpInst := techProfMsg.TechTpInstance.(type) {
-		case *ic.InterAdapterTechProfileDownloadMessage_TpInstance:
+		case *ic.TechProfileDownloadMessage_TpInstance:
 			logger.Debugw(ctx, "onu-uni-tp-path-modified", log.Fields{"uniID": uniID, "tp-path": techProfMsg.TpInstancePath, "tpID": tpID})
 			//	if there has been some change for some uni TechProfilePath
 			//in order to allow concurrent calls to other dh instances we do not wait for execution here
@@ -475,9 +455,8 @@
 	return nil
 }
 
-func (dh *deviceHandler) processInterAdapterDeleteGemPortReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteGemPortRequest(ctx context.Context, delGemPortMsg *ic.DeleteGemPortMessage) error {
+	logger.Infow(ctx, "delete-gem-port-request", log.Fields{"device-id": dh.deviceID})
 
 	if dh.pOnuTP == nil {
 		//should normally not happen ...
@@ -485,15 +464,6 @@
 			log.Fields{"device-id": dh.deviceID})
 		return fmt.Errorf("techProfile DelGem request while onuTechProf instance not setup: %s", dh.deviceID)
 	}
-
-	msgBody := msg.GetBody()
-	delGemPortMsg := &ic.InterAdapterDeleteGemPortMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, delGemPortMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-delete-gem-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	//compare TECH_PROFILE_DOWNLOAD_REQUEST
 	dh.pOnuTP.lockTpProcMutex()
 	defer dh.pOnuTP.unlockTpProcMutex()
@@ -516,9 +486,8 @@
 
 }
 
-func (dh *deviceHandler) processInterAdapterDeleteTcontReqMessage(
-	ctx context.Context,
-	msg *ic.InterAdapterMessage) error {
+func (dh *deviceHandler) handleDeleteTcontRequest(ctx context.Context, delTcontMsg *ic.DeleteTcontMessage) error {
+	logger.Infow(ctx, "delete-tcont-request", log.Fields{"device-id": dh.deviceID})
 
 	pDevEntry := dh.getOnuDeviceEntry(ctx, true)
 	if pDevEntry == nil {
@@ -532,14 +501,6 @@
 		return fmt.Errorf("techProfile DelTcont request while onuTechProf instance not setup: %s", dh.deviceID)
 	}
 
-	msgBody := msg.GetBody()
-	delTcontMsg := &ic.InterAdapterDeleteTcontMessage{}
-	if err := ptypes.UnmarshalAny(msgBody, delTcontMsg); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-delete-tcont-msg-body", log.Fields{
-			"device-id": dh.deviceID, "error": err})
-		return err
-	}
-
 	//compare TECH_PROFILE_DOWNLOAD_REQUEST
 	dh.pOnuTP.lockTpProcMutex()
 	defer dh.pOnuTP.unlockTpProcMutex()
@@ -616,52 +577,10 @@
 	return nil
 }
 
-//processInterAdapterMessage sends the proxied messages to the target device
-// If the proxy address is not found in the unmarshalled message, it first fetches the onu device for which the message
-// is meant, and then send the unmarshalled omci message to this onu
-func (dh *deviceHandler) processInterAdapterMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgID := msg.Header.Id
-	msgType := msg.Header.Type
-	fromTopic := msg.Header.FromTopic
-	toTopic := msg.Header.ToTopic
-	toDeviceID := msg.Header.ToDeviceId
-	proxyDeviceID := msg.Header.ProxyDeviceId
-	logger.Debugw(ctx, "InterAdapter message header", log.Fields{"msgID": msgID, "msgType": msgType,
-		"fromTopic": fromTopic, "toTopic": toTopic, "toDeviceID": toDeviceID, "proxyDeviceID": proxyDeviceID})
-
-	switch msgType {
-	// case ic.InterAdapterMessageType_ONU_IND_REQUEST: was handled by OpenONUAC already - see comments there
-	//OMCI_RESPONSE also accepted acc. to VOL-3756 (OMCI_REQUEST request was legacy code)
-	case ic.InterAdapterMessageType_OMCI_RESPONSE, ic.InterAdapterMessageType_OMCI_REQUEST:
-		{
-			return dh.processInterAdapterOMCIReceiveMessage(ctx, msg)
-		}
-	case ic.InterAdapterMessageType_TECH_PROFILE_DOWNLOAD_REQUEST:
-		{
-			return dh.processInterAdapterTechProfileDownloadReqMessage(ctx, msg)
-		}
-	case ic.InterAdapterMessageType_DELETE_GEM_PORT_REQUEST:
-		{
-			return dh.processInterAdapterDeleteGemPortReqMessage(ctx, msg)
-
-		}
-	case ic.InterAdapterMessageType_DELETE_TCONT_REQUEST:
-		{
-			return dh.processInterAdapterDeleteTcontReqMessage(ctx, msg)
-		}
-	default:
-		{
-			logger.Errorw(ctx, "inter-adapter-unhandled-type", log.Fields{
-				"msgType": msg.Header.Type, "device-id": dh.deviceID})
-			return fmt.Errorf("inter-adapter-unhandled-type: %d, %s", msg.Header.Type, dh.deviceID)
-		}
-	}
-}
-
 //FlowUpdateIncremental removes and/or adds the flow changes on a given device
 func (dh *deviceHandler) FlowUpdateIncremental(ctx context.Context,
-	apOfFlowChanges *openflow_13.FlowChanges,
-	apOfGroupChanges *openflow_13.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
+	apOfFlowChanges *of.FlowChanges,
+	apOfGroupChanges *of.FlowGroupChanges, apFlowMetaData *voltha.FlowMetadata) error {
 	logger.Debugw(ctx, "FlowUpdateIncremental started", log.Fields{"device-id": dh.deviceID, "metadata": apFlowMetaData})
 	var retError error = nil
 	//Remove flows (always remove flows first - remove old and add new with same cookie may be part of the same request)
@@ -806,8 +725,11 @@
 		} else {
 			logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 				"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
-			if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
-				dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+			if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+				DeviceId:   dh.deviceID,
+				ConnStatus: voltha.ConnectStatus_REACHABLE,
+				OperStatus: voltha.OperStatus_UNKNOWN,
+			}); err != nil {
 				//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 				logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 			}
@@ -914,10 +836,16 @@
 		techProfsFound = true // set to true if we found TP once for any UNI port
 		for tpID := range uniData.PersTpPathMap {
 			// Request the TpInstance again from the openolt adapter in case of reconcile
-			iaTechTpInst, err := dh.AdapterProxy.TechProfileInstanceRequest(ctx, uniData.PersTpPathMap[tpID],
-				dh.device.ParentPortNo, dh.device.ProxyAddress.OnuId, uint32(uniData.PersUniID),
-				dh.pOpenOnuAc.config.Topic, dh.ProxyAddressType,
-				dh.parentID, dh.ProxyAddressID)
+			iaTechTpInst, err := dh.getTechProfileInstanceFromParentAdapter(ctx,
+				dh.device.ProxyAddress.AdapterEndpoint,
+				&ic.TechProfileInstanceRequestMessage{
+					DeviceId:       dh.device.Id,
+					TpInstancePath: uniData.PersTpPathMap[tpID],
+					ParentDeviceId: dh.parentID,
+					ParentPonPort:  dh.device.ParentPortNo,
+					OnuId:          dh.device.ProxyAddress.OnuId,
+					UniId:          uint32(uniData.PersUniID),
+				})
 			if err != nil || iaTechTpInst == nil {
 				logger.Errorw(ctx, "error fetching tp instance",
 					log.Fields{"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID, "err": err})
@@ -926,7 +854,7 @@
 			}
 			var tpInst tech_profile.TechProfileInstance
 			switch techTpInst := iaTechTpInst.TechTpInstance.(type) {
-			case *ic.InterAdapterTechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
+			case *ic.TechProfileDownloadMessage_TpInstance: // supports only GPON, XGPON, XGS-PON
 				tpInst = *techTpInst.TpInstance
 				logger.Debugw(ctx, "received-tp-instance-successfully-after-reconcile", log.Fields{
 					"tp-id": tpID, "tpPath": uniData.PersTpPathMap[tpID], "uni-id": uniData.PersUniID, "device-id": dh.deviceID})
@@ -1152,8 +1080,11 @@
 
 	logger.Debugw(ctx, "call DeviceStateUpdate upon reboot", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
-		voltha.OperStatus_DISCOVERED); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_DISCOVERED,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		return
@@ -1294,8 +1225,7 @@
 	dh.lockUpgradeFsm.RLock()
 	if dh.pOnuUpradeFsm != nil {
 		dh.lockUpgradeFsm.RUnlock()
-		onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-			dh.deviceID, dh.deviceID)
+		onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
 		if getErr != nil || onuVolthaDevice == nil {
 			logger.Errorw(ctx, "Failed to fetch Onu device for image activation", log.Fields{"device-id": dh.deviceID, "err": getErr})
 			return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1358,8 +1288,7 @@
 	dh.lockUpgradeFsm.RLock()
 	if dh.pOnuUpradeFsm != nil {
 		dh.lockUpgradeFsm.RUnlock()
-		onuVolthaDevice, getErr := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-			dh.deviceID, dh.deviceID)
+		onuVolthaDevice, getErr := dh.getDeviceFromCore(ctx, dh.deviceID)
 		if getErr != nil || onuVolthaDevice == nil {
 			logger.Errorw(ctx, "Failed to fetch Onu device for image commitment", log.Fields{"device-id": dh.deviceID, "err": getErr})
 			return nil, fmt.Errorf("could not fetch device for device-id: %s", dh.deviceID)
@@ -1513,7 +1442,9 @@
 
 	if !dh.isReconciling() {
 		logger.Infow(ctx, "DeviceUpdate", log.Fields{"deviceReason": dh.device.Reason, "device-id": dh.deviceID})
-		_ = dh.coreProxy.DeviceUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.device)
+		if err := dh.updateDeviceInCore(ctx, dh.device); err != nil {
+			logger.Errorw(ctx, "device-update-failed", log.Fields{"device-id": dh.device.Id, "error": err})
+		}
 		//TODO Need to Update Device Reason To CORE as part of device update userstory
 	} else {
 		logger.Debugw(ctx, "reconciling - don't notify core about DeviceUpdate",
@@ -1547,6 +1478,7 @@
 		}
 
 		pPonPort := &voltha.Port{
+			DeviceId:   dh.deviceID,
 			PortNo:     ponPortNo,
 			Label:      fmt.Sprintf("pon-%d", ponPortNo),
 			Type:       voltha.Port_PON_ONU,
@@ -1554,7 +1486,7 @@
 			Peers: []*voltha.Port_PeerPort{{DeviceId: dh.parentID, // Peer device  is OLT
 				PortNo: ponPortNo}}, // Peer port is parent's port number
 		}
-		if err = dh.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, pPonPort); err != nil {
+		if err = dh.createPortInCore(ctx, pPonPort); err != nil {
 			logger.Fatalf(ctx, "Device FSM: PortCreated-failed-%s", err)
 			e.Cancel(err)
 			return
@@ -1814,8 +1746,12 @@
 		}
 		logger.Debugw(ctx, "call DeviceStateUpdate upon create interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 			"OperStatus": voltha.OperStatus_ACTIVATING, "device-id": dh.deviceID})
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING); err != nil {
+
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			OperStatus: voltha.OperStatus_ACTIVATING,
+			ConnStatus: voltha.ConnectStatus_REACHABLE,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		}
@@ -2031,8 +1967,11 @@
 		}
 		logger.Debugw(ctx, "call DeviceStateUpdate upon update interface", log.Fields{"ConnectStatus": voltha.ConnectStatus_UNREACHABLE,
 			"OperStatus": voltha.OperStatus_DISCOVERED, "device-id": dh.deviceID})
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_UNREACHABLE, voltha.OperStatus_DISCOVERED); err != nil {
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			ConnStatus: voltha.ConnectStatus_UNREACHABLE,
+			OperStatus: voltha.OperStatus_DISCOVERED,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state unreachable-discovered",
 				log.Fields{"device-id": dh.deviceID, "error": err})
@@ -2208,8 +2147,11 @@
 		// in case of adapter restart connected to an ONU upgrade I would not rely on the image quality
 		// maybe some 'forced' commitment can be done in this situation from system management (or upgrade restarted)
 		dh.checkOnOnuImageCommit(ctx)
-		if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID,
-			voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE); err != nil {
+		if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+			DeviceId:   dh.deviceID,
+			ConnStatus: voltha.ConnectStatus_REACHABLE,
+			OperStatus: voltha.OperStatus_ACTIVE,
+		}); err != nil {
 			//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 			logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 		} else {
@@ -2292,8 +2234,12 @@
 func (dh *deviceHandler) processUniDisableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
 	logger.Debugw(ctx, "DeviceStateUpdate upon disable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_UNKNOWN, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx),
-		dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_UNKNOWN); err != nil {
+
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_UNKNOWN,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 	}
@@ -2322,8 +2268,11 @@
 func (dh *deviceHandler) processUniEnableStateDoneEvent(ctx context.Context, devEvent OnuDeviceEvent) {
 	logger.Debugw(ctx, "DeviceStateUpdate upon re-enable", log.Fields{"ConnectStatus": voltha.ConnectStatus_REACHABLE,
 		"OperStatus": voltha.OperStatus_ACTIVE, "device-id": dh.deviceID})
-	if err := dh.coreProxy.DeviceStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.ConnectStatus_REACHABLE,
-		voltha.OperStatus_ACTIVE); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: voltha.ConnectStatus_REACHABLE,
+		OperStatus: voltha.OperStatus_ACTIVE,
+	}); err != nil {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing
 		logger.Errorw(ctx, "error-updating-device-state", log.Fields{"device-id": dh.deviceID, "error": err})
 	}
@@ -2537,7 +2486,16 @@
 			uniPort.setOperState(vc.OperStatus_ACTIVE)
 			if !dh.isReconciling() {
 				//maybe also use getter functions on uniPort - perhaps later ...
-				go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+				go func(port *onuUniPort) {
+					if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+						DeviceId:   dh.deviceID,
+						PortType:   voltha.Port_ETHERNET_UNI,
+						PortNo:     port.portNo,
+						OperStatus: port.operState,
+					}); err != nil {
+						logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+					}
+				}(uniPort)
 			} else {
 				logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
 			}
@@ -2557,7 +2515,16 @@
 			uniPort.setOperState(vc.OperStatus_UNKNOWN)
 			if !dh.isReconciling() {
 				//maybe also use getter functions on uniPort - perhaps later ...
-				go dh.coreProxy.PortStateUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, voltha.Port_ETHERNET_UNI, uniPort.portNo, uniPort.operState)
+				go func(port *onuUniPort) {
+					if err := dh.updatePortStateInCore(ctx, &ic.PortState{
+						DeviceId:   dh.deviceID,
+						PortType:   voltha.Port_ETHERNET_UNI,
+						PortNo:     port.portNo,
+						OperStatus: port.operState,
+					}); err != nil {
+						logger.Errorw(ctx, "port-state-update-failed", log.Fields{"error": err, "port-no": uniPort.portNo, "device-id": dh.deviceID})
+					}
+				}(uniPort)
 			} else {
 				logger.Debugw(ctx, "reconciling - don't notify core about PortStateUpdate", log.Fields{"device-id": dh.deviceID})
 			}
@@ -2573,7 +2540,7 @@
 	eventContext := make(map[string]string)
 	//Populating event context
 	//  assume giving ParentId in GetDevice twice really gives the ParentDevice (there is no GetParentDevice()...)
-	parentDevice, err := dh.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx), dh.parentID, dh.parentID)
+	parentDevice, err := dh.getDeviceFromCore(ctx, dh.parentID)
 	if err != nil || parentDevice == nil {
 		logger.Errorw(ctx, "Failed to fetch parent device for OnuEvent",
 			log.Fields{"parentID": dh.parentID, "err": err})
@@ -2853,7 +2820,7 @@
 
 	return kvbackend
 }
-func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loMatchVlan *uint16,
+func (dh *deviceHandler) getFlowOfbFields(ctx context.Context, apFlowItem *of.OfpFlowStats, loMatchVlan *uint16,
 	loAddPcp *uint8, loIPProto *uint32) {
 
 	for _, field := range flow.GetOfbFields(apFlowItem) {
@@ -2930,7 +2897,7 @@
 	} //for all OfbFields
 }
 
-func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *ofp.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
+func (dh *deviceHandler) getFlowActions(ctx context.Context, apFlowItem *of.OfpFlowStats, loSetPcp *uint8, loSetVlan *uint16) {
 	for _, action := range flow.GetActions(apFlowItem) {
 		switch action.Type {
 		/* not used:
@@ -2976,7 +2943,7 @@
 }
 
 //addFlowItemToUniPort parses the actual flow item to add it to the UniPort
-func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort,
+func (dh *deviceHandler) addFlowItemToUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort,
 	apFlowMetaData *voltha.FlowMetadata) error {
 	var loSetVlan uint16 = uint16(of.OfpVlanId_OFPVID_NONE)      //noValidEntry
 	var loMatchVlan uint16 = uint16(of.OfpVlanId_OFPVID_PRESENT) //reserved VLANID entry
@@ -3067,7 +3034,7 @@
 }
 
 //removeFlowItemFromUniPort parses the actual flow item to remove it from the UniPort
-func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *ofp.OfpFlowStats, apUniPort *onuUniPort) error {
+func (dh *deviceHandler) removeFlowItemFromUniPort(ctx context.Context, apFlowItem *of.OfpFlowStats, apUniPort *onuUniPort) error {
 	//optimization and assumption: the flow cookie uniquely identifies the flow and with that the internal rule
 	//hence only the cookie is used here to find the relevant flow and possibly remove the rule
 	//no extra check is done on the rule parameters
@@ -3300,7 +3267,10 @@
 	dh.setDeviceReason(deviceReason)
 	if notifyCore {
 		//TODO with VOL-3045/VOL-3046: return the error and stop further processing at calling position
-		if err := dh.coreProxy.DeviceReasonUpdate(log.WithSpanFromContext(context.TODO(), ctx), dh.deviceID, deviceReasonMap[deviceReason]); err != nil {
+		if err := dh.updateDeviceReasonInCore(ctx, &ic.DeviceReason{
+			DeviceId: dh.deviceID,
+			Reason:   deviceReasonMap[deviceReason],
+		}); err != nil {
 			logger.Errorf(ctx, "DeviceReasonUpdate error: %s",
 				log.Fields{"device-id": dh.deviceID, "error": err}, deviceReasonMap[deviceReason])
 			return err
@@ -3715,7 +3685,11 @@
 					}
 					logger.Debugw(ctx, "reconciling has been finished in time",
 						log.Fields{"device-id": dh.deviceID})
-					if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, operState); err != nil {
+					if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+						DeviceId:   dh.deviceID,
+						ConnStatus: connectStatus,
+						OperStatus: operState,
+					}); err != nil {
 						logger.Errorw(ctx, "unable to update device state to core",
 							log.Fields{"device-id": dh.deviceID, "Err": err})
 					}
@@ -3830,8 +3804,130 @@
 	}
 
 	logger.Debugw(ctx, "Core DeviceStateUpdate", log.Fields{"connectStatus": connectStatus, "operState": voltha.OperStatus_RECONCILING_FAILED})
-	if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.deviceID, connectStatus, voltha.OperStatus_RECONCILING_FAILED); err != nil {
+	if err := dh.updateDeviceStateInCore(ctx, &ic.DeviceStateFilter{
+		DeviceId:   dh.deviceID,
+		ConnStatus: connectStatus,
+		OperStatus: voltha.OperStatus_RECONCILING_FAILED,
+	}); err != nil {
 		logger.Errorw(ctx, "unable to update device state to core",
 			log.Fields{"device-id": dh.deviceID, "Err": err})
 	}
 }
+
+/*
+Helper functions to communicate with Core
+*/
+
+func (dh *deviceHandler) getDeviceFromCore(ctx context.Context, deviceID string) (*voltha.Device, error) {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	logger.Debugw(subCtx, "get-device-from-core", log.Fields{"device-id": deviceID})
+	return cClient.GetDevice(subCtx, &vc.ID{Id: deviceID})
+}
+
+func (dh *deviceHandler) updateDeviceStateInCore(ctx context.Context, deviceStateFilter *ic.DeviceStateFilter) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceStateUpdate(subCtx, deviceStateFilter)
+	logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-state": deviceStateFilter, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updatePMConfigInCore(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DevicePMConfigUpdate(subCtx, pmConfigs)
+	logger.Debugw(subCtx, "pmconfig-updated-in-core", log.Fields{"pm-configs": pmConfigs, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updateDeviceInCore(ctx context.Context, device *voltha.Device) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceUpdate(subCtx, device)
+	logger.Debugw(subCtx, "device-updated-in-core", log.Fields{"device-id": device.Id, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) createPortInCore(ctx context.Context, port *voltha.Port) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.PortCreated(subCtx, port)
+	logger.Debugw(subCtx, "port-created-in-core", log.Fields{"port": port, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updatePortStateInCore(ctx context.Context, portState *ic.PortState) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.PortStateUpdate(subCtx, portState)
+	logger.Debugw(subCtx, "port-state-updated-in-core", log.Fields{"port-state": portState, "error": err})
+	return err
+}
+
+func (dh *deviceHandler) updateDeviceReasonInCore(ctx context.Context, reason *ic.DeviceReason) error {
+	cClient, err := dh.coreClient.GetCoreServiceClient()
+	if err != nil || cClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.RPCTimeout)
+	defer cancel()
+	_, err = cClient.DeviceReasonUpdate(subCtx, reason)
+	logger.Debugw(subCtx, "device-reason-updated-in-core", log.Fields{"reason": reason, "error": err})
+	return err
+}
+
+/*
+Helper functions to communicate with parent adapter
+*/
+
+func (dh *deviceHandler) getTechProfileInstanceFromParentAdapter(ctx context.Context, parentEndpoint string,
+	request *ic.TechProfileInstanceRequestMessage) (*ic.TechProfileDownloadMessage, error) {
+	pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+	if err != nil || pgClient == nil {
+		return nil, err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+	defer cancel()
+	logger.Debugw(subCtx, "get-tech-profile-instance", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+	return pgClient.GetTechProfileInstance(subCtx, request)
+}
+
+func (dh *deviceHandler) sendOMCIRequest(ctx context.Context, parentEndpoint string, request *ic.OmciMessage) error {
+	pgClient, err := dh.pOpenOnuAc.getParentAdapterServiceClient(parentEndpoint)
+	if err != nil || pgClient == nil {
+		return err
+	}
+	subCtx, cancel := context.WithTimeout(log.WithSpanFromContext(context.Background(), ctx), dh.config.MaxTimeoutInterAdapterComm)
+	defer cancel()
+	logger.Debugw(subCtx, "send-omci-request", log.Fields{"request": request, "parent-endpoint": parentEndpoint})
+	_, err = pgClient.ProxyOmciRequest(subCtx, request)
+	if err != nil {
+		logger.Errorw(ctx, "omci-failure", log.Fields{"request": request, "error": err, "request-parent": request.ParentDeviceId, "request-child": request.ChildDeviceId, "request-proxy": request.ProxyAddress})
+	}
+	return err
+}
diff --git a/internal/pkg/onuadaptercore/file_download_manager.go b/internal/pkg/onuadaptercore/file_download_manager.go
index 4793e69..5c3226e 100644
--- a/internal/pkg/onuadaptercore/file_download_manager.go
+++ b/internal/pkg/onuadaptercore/file_download_manager.go
@@ -29,7 +29,7 @@
 	"sync"
 	"time"
 
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 const cDefaultLocalDir = "/tmp" //this is the default local dir to download to
diff --git a/internal/pkg/onuadaptercore/mib_download.go b/internal/pkg/onuadaptercore/mib_download.go
index be63ead..5d77931 100644
--- a/internal/pkg/onuadaptercore/mib_download.go
+++ b/internal/pkg/onuadaptercore/mib_download.go
@@ -26,10 +26,10 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	//ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	//"github.com/opencord/voltha-protos/v5/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 func (onuDeviceEntry *OnuDeviceEntry) enterDLStartingState(ctx context.Context, e *fsm.Event) {
diff --git a/internal/pkg/onuadaptercore/mib_sync.go b/internal/pkg/onuadaptercore/mib_sync.go
index 3716352..1660791 100644
--- a/internal/pkg/onuadaptercore/mib_sync.go
+++ b/internal/pkg/onuadaptercore/mib_sync.go
@@ -28,17 +28,12 @@
 
 	"github.com/looplab/fsm"
 
-	//"sync"
 	"time"
 
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 type sLastTxMeParameter struct {
diff --git a/internal/pkg/onuadaptercore/omci_ani_config.go b/internal/pkg/onuadaptercore/omci_ani_config.go
index f617f41..33e323d 100644
--- a/internal/pkg/onuadaptercore/omci_ani_config.go
+++ b/internal/pkg/onuadaptercore/omci_ani_config.go
@@ -30,10 +30,10 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	//ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	//"github.com/opencord/voltha-protos/v5/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/omci_cc.go b/internal/pkg/onuadaptercore/omci_cc.go
index 4006283..be930b8 100644
--- a/internal/pkg/onuadaptercore/omci_cc.go
+++ b/internal/pkg/onuadaptercore/omci_cc.go
@@ -33,14 +33,15 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
 
-	"github.com/opencord/voltha-protos/v4/go/common"
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
+	//"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	//"github.com/opencord/voltha-protos/v5/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 // ### OMCI related definitions - retrieved from Python adapter code/trace ####
@@ -99,8 +100,7 @@
 	pOnuDeviceEntry    *OnuDeviceEntry
 	deviceID           string
 	pBaseDeviceHandler *deviceHandler
-	coreProxy          adapterif.CoreProxy
-	adapterProxy       adapterif.AdapterProxy
+	coreClient         *vgrpc.Client
 	supportExtMsg      bool
 	rxOmciFrameError   tOmciReceiveError
 
@@ -139,15 +139,14 @@
 //mib_db (as well as not inluded alarm_db not really used in this code? VERIFY!!)
 func newOmciCC(ctx context.Context, onuDeviceEntry *OnuDeviceEntry,
 	deviceID string, deviceHandler *deviceHandler,
-	coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy) *omciCC {
+	coreClient *vgrpc.Client) *omciCC {
 	logger.Debugw(ctx, "init-omciCC", log.Fields{"device-id": deviceID})
 	var omciCC omciCC
 	omciCC.enabled = false
 	omciCC.pOnuDeviceEntry = onuDeviceEntry
 	omciCC.deviceID = deviceID
 	omciCC.pBaseDeviceHandler = deviceHandler
-	omciCC.coreProxy = coreProxy
-	omciCC.adapterProxy = adapterProxy
+	omciCC.coreClient = coreClient
 	omciCC.supportExtMsg = false
 	omciCC.rxOmciFrameError = cOmciMessageReceiveNoError
 	omciCC.txFrames = 0
@@ -590,18 +589,18 @@
 				"TxOmciMessage": hex.EncodeToString(omciTxRequest.txFrame),
 				"device-id":     oo.deviceID,
 				"toDeviceType":  oo.pBaseDeviceHandler.ProxyAddressType,
-				"proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID})
+				"proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID,
+				"proxyAddress":  oo.pBaseDeviceHandler.device.ProxyAddress})
 		}
-		omciMsg := &ic.InterAdapterOmciMessage{
-			Message:       omciTxRequest.txFrame,
-			ProxyAddress:  oo.pBaseDeviceHandler.device.ProxyAddress,
-			ConnectStatus: common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
+		omciMsg := &ic.OmciMessage{
+			ParentDeviceId: oo.pBaseDeviceHandler.ProxyAddressID,
+			ChildDeviceId:  oo.deviceID,
+			Message:        omciTxRequest.txFrame,
+			ProxyAddress:   oo.pBaseDeviceHandler.device.ProxyAddress,
+			ConnectStatus:  common.ConnectStatus_REACHABLE, // If we are sending OMCI messages means we are connected, else we should not be here
 		}
-		if sendErr := oo.adapterProxy.SendInterAdapterMessage(log.WithSpanFromContext(context.Background(), ctx), omciMsg,
-			ic.InterAdapterMessageType_OMCI_REQUEST,
-			//fromTopic,toType,toDevId, ProxyDevId
-			oo.pOnuDeviceEntry.baseDeviceHandler.pOpenOnuAc.config.Topic, oo.pBaseDeviceHandler.ProxyAddressType,
-			oo.deviceID, oo.pBaseDeviceHandler.ProxyAddressID, ""); sendErr != nil {
+		sendErr := oo.pBaseDeviceHandler.sendOMCIRequest(ctx, oo.pBaseDeviceHandler.device.ProxyAddress.AdapterEndpoint, omciMsg)
+		if sendErr != nil {
 			logger.Errorw(ctx, "send omci request error", log.Fields{"ChildId": oo.deviceID, "error": sendErr})
 			return sendErr
 		}
diff --git a/internal/pkg/onuadaptercore/omci_onu_upgrade.go b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
index 1c86e97..8e8a82e 100644
--- a/internal/pkg/onuadaptercore/omci_onu_upgrade.go
+++ b/internal/pkg/onuadaptercore/omci_onu_upgrade.go
@@ -29,8 +29,8 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const cMaxUint32 = ^uint32(0)
diff --git a/internal/pkg/onuadaptercore/omci_self_test_handler.go b/internal/pkg/onuadaptercore/omci_self_test_handler.go
index ce82db8..13b6ec3 100644
--- a/internal/pkg/onuadaptercore/omci_self_test_handler.go
+++ b/internal/pkg/onuadaptercore/omci_self_test_handler.go
@@ -20,13 +20,14 @@
 import (
 	"context"
 	"fmt"
+	"sync"
+	"time"
+
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	"github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"sync"
-	"time"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/omci_test_request.go b/internal/pkg/onuadaptercore/omci_test_request.go
index f0b76dc..235f21d 100644
--- a/internal/pkg/onuadaptercore/omci_test_request.go
+++ b/internal/pkg/onuadaptercore/omci_test_request.go
@@ -21,18 +21,11 @@
 	"context"
 	"fmt"
 
-	//"sync"
-	//"time"
-
 	gp "github.com/google/gopacket"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
 
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 //omciTestRequest structure holds the information for the OMCI test
diff --git a/internal/pkg/onuadaptercore/omci_vlan_config.go b/internal/pkg/onuadaptercore/omci_vlan_config.go
index 1751cdb..c828804 100644
--- a/internal/pkg/onuadaptercore/omci_vlan_config.go
+++ b/internal/pkg/onuadaptercore/omci_vlan_config.go
@@ -27,15 +27,15 @@
 	"sync"
 	"time"
 
-	meters "github.com/opencord/voltha-lib-go/v5/pkg/meters"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	meters "github.com/opencord/voltha-lib-go/v7/pkg/meters"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
 	gp "github.com/google/gopacket"
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/onu_device_db.go b/internal/pkg/onuadaptercore/onu_device_db.go
index 83f419b..17f352b 100644
--- a/internal/pkg/onuadaptercore/onu_device_db.go
+++ b/internal/pkg/onuadaptercore/onu_device_db.go
@@ -25,7 +25,7 @@
 	"sync"
 
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 type meDbMap map[me.ClassID]map[uint16]me.AttributeValueMap
diff --git a/internal/pkg/onuadaptercore/onu_device_entry.go b/internal/pkg/onuadaptercore/onu_device_entry.go
index 91510c9..bfb1ab5 100644
--- a/internal/pkg/onuadaptercore/onu_device_entry.go
+++ b/internal/pkg/onuadaptercore/onu_device_entry.go
@@ -25,22 +25,14 @@
 	"sync"
 	"time"
 
+	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
 
-	//"sync"
-	//"time"
-
-	"github.com/looplab/fsm"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 const (
@@ -269,8 +261,7 @@
 	deviceID                   string
 	baseDeviceHandler          *deviceHandler
 	pOpenOnuAc                 *OpenONUAC
-	coreProxy                  adapterif.CoreProxy
-	adapterProxy               adapterif.AdapterProxy
+	coreClient                 *vgrpc.Client
 	PDevOmciCC                 *omciCC
 	pOnuDB                     *onuDeviceDB
 	mibTemplateKVStore         *db.Backend
@@ -319,8 +310,7 @@
 	onuDeviceEntry.deviceID = dh.deviceID
 	onuDeviceEntry.baseDeviceHandler = dh
 	onuDeviceEntry.pOpenOnuAc = dh.pOpenOnuAc
-	onuDeviceEntry.coreProxy = dh.coreProxy
-	onuDeviceEntry.adapterProxy = dh.AdapterProxy
+	onuDeviceEntry.coreClient = dh.coreClient
 	onuDeviceEntry.devState = DeviceStatusInit
 	onuDeviceEntry.sOnuPersistentData.PersUniConfig = make([]uniPersConfig, 0)
 	onuDeviceEntry.sOnuPersistentData.PersTcontMap = make(map[uint16]uint16)
@@ -520,8 +510,7 @@
 func (oo *OnuDeviceEntry) start(ctx context.Context) error {
 	logger.Debugw(ctx, "OnuDeviceEntry-starting", log.Fields{"for device-id": oo.deviceID})
 	if oo.PDevOmciCC == nil {
-		oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler,
-			oo.coreProxy, oo.adapterProxy)
+		oo.PDevOmciCC = newOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler, oo.coreClient)
 		if oo.PDevOmciCC == nil {
 			logger.Errorw(ctx, "Could not create devOmciCc - abort", log.Fields{"for device-id": oo.deviceID})
 			return fmt.Errorf("could not create devOmciCc %s", oo.deviceID)
diff --git a/internal/pkg/onuadaptercore/onu_image_status.go b/internal/pkg/onuadaptercore/onu_image_status.go
index f36a44e..d226134 100755
--- a/internal/pkg/onuadaptercore/onu_image_status.go
+++ b/internal/pkg/onuadaptercore/onu_image_status.go
@@ -26,8 +26,8 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 //OnuImageStatus implements methods to get status info of onu images
diff --git a/internal/pkg/onuadaptercore/onu_metrics_manager.go b/internal/pkg/onuadaptercore/onu_metrics_manager.go
index c33fb99..c2a2ffb 100644
--- a/internal/pkg/onuadaptercore/onu_metrics_manager.go
+++ b/internal/pkg/onuadaptercore/onu_metrics_manager.go
@@ -28,11 +28,11 @@
 	"github.com/looplab/fsm"
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 const (
diff --git a/internal/pkg/onuadaptercore/onu_uni_port.go b/internal/pkg/onuadaptercore/onu_uni_port.go
index 9cfa7de..f4b255f 100644
--- a/internal/pkg/onuadaptercore/onu_uni_port.go
+++ b/internal/pkg/onuadaptercore/onu_uni_port.go
@@ -27,11 +27,11 @@
 	//"sync"
 	//"time"
 
-	//"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	vc "github.com/opencord/voltha-protos/v4/go/common"
-	of "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	//"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	vc "github.com/opencord/voltha-protos/v5/go/common"
+	of "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 )
 
 type uniPortType uint8
@@ -114,6 +114,7 @@
 		"name": name, "hwAddr": ofHwAddr, "OperState": ofUniPortState})
 
 	pUniPort := &voltha.Port{
+		DeviceId:   apDeviceHandler.deviceID,
 		PortNo:     oo.portNo,
 		Label:      oo.name,
 		Type:       voltha.Port_ETHERNET_UNI,
@@ -136,8 +137,7 @@
 	retryCnt := 0
 	var err error
 	for retryCnt = 0; retryCnt < maxRetry; retryCnt++ {
-		if err = apDeviceHandler.coreProxy.PortCreated(log.WithSpanFromContext(context.TODO(), ctx),
-			apDeviceHandler.deviceID, pUniPort); err != nil {
+		if err = apDeviceHandler.createPortInCore(ctx, pUniPort); err != nil {
 			logger.Errorf(ctx, "Device FSM: PortCreated-failed-%s, retrying after a delay", err)
 			// retry after a sleep
 			time.Sleep(2 * time.Second)
diff --git a/internal/pkg/onuadaptercore/onu_uni_tp.go b/internal/pkg/onuadaptercore/onu_uni_tp.go
index 6e365ab..1e3f9af 100644
--- a/internal/pkg/onuadaptercore/onu_uni_tp.go
+++ b/internal/pkg/onuadaptercore/onu_uni_tp.go
@@ -24,9 +24,8 @@
 	"strings"
 	"sync"
 
-	"github.com/opencord/voltha-protos/v4/go/tech_profile"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/tech_profile"
 )
 
 //definitions for TechProfileProcessing - copied from OltAdapter:openolt_flowmgr.go
diff --git a/internal/pkg/onuadaptercore/openonu.go b/internal/pkg/onuadaptercore/openonu.go
index cfa2661..eecd002 100644
--- a/internal/pkg/onuadaptercore/openonu.go
+++ b/internal/pkg/onuadaptercore/openonu.go
@@ -24,19 +24,20 @@
 	"sync"
 	"time"
 
-	conf "github.com/opencord/voltha-lib-go/v5/pkg/config"
+	vgrpc "github.com/opencord/voltha-lib-go/v7/pkg/grpc"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
 
-	"github.com/golang/protobuf/ptypes"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	oop "github.com/opencord/voltha-protos/v4/go/openolt"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
+	conf "github.com/opencord/voltha-lib-go/v7/pkg/config"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"google.golang.org/grpc"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
 
 	"github.com/opencord/voltha-openonu-adapter-go/internal/pkg/config"
 )
@@ -46,10 +47,10 @@
 	deviceHandlers              map[string]*deviceHandler
 	deviceHandlersCreateChan    map[string]chan bool //channels for deviceHandler create events
 	lockDeviceHandlersMap       sync.RWMutex
-	coreProxy                   adapterif.CoreProxy
-	adapterProxy                adapterif.AdapterProxy
+	coreClient                  *vgrpc.Client
+	parentAdapterClients        map[string]*vgrpc.Client
+	lockParentAdapterClients    sync.RWMutex
 	eventProxy                  eventif.EventProxy
-	kafkaICProxy                kafka.InterContainerProxy
 	kvClient                    kvstore.Client
 	cm                          *conf.ConfigManager
 	config                      *config.AdapterFlags
@@ -74,23 +75,22 @@
 	omciTimeout                int // in seconds
 	alarmAuditInterval         time.Duration
 	dlToOnuTimeout4M           time.Duration
+	rpcTimeout                 time.Duration
 }
 
 //NewOpenONUAC returns a new instance of OpenONU_AC
-func NewOpenONUAC(ctx context.Context, kafkaICProxy kafka.InterContainerProxy,
-	coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy,
-	eventProxy eventif.EventProxy, kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
+func NewOpenONUAC(ctx context.Context, coreClient *vgrpc.Client, eventProxy eventif.EventProxy,
+	kvClient kvstore.Client, cfg *config.AdapterFlags, cm *conf.ConfigManager) *OpenONUAC {
 	var openOnuAc OpenONUAC
 	openOnuAc.exitChannel = make(chan int, 1)
 	openOnuAc.deviceHandlers = make(map[string]*deviceHandler)
 	openOnuAc.deviceHandlersCreateChan = make(map[string]chan bool)
+	openOnuAc.parentAdapterClients = make(map[string]*vgrpc.Client)
 	openOnuAc.lockDeviceHandlersMap = sync.RWMutex{}
-	openOnuAc.kafkaICProxy = kafkaICProxy
 	openOnuAc.config = cfg
 	openOnuAc.cm = cm
+	openOnuAc.coreClient = coreClient
 	openOnuAc.numOnus = cfg.OnuNumber
-	openOnuAc.coreProxy = coreProxy
-	openOnuAc.adapterProxy = adapterProxy
 	openOnuAc.eventProxy = eventProxy
 	openOnuAc.kvClient = kvClient
 	openOnuAc.KVStoreAddress = cfg.KVStoreAddress
@@ -110,6 +110,7 @@
 	openOnuAc.omciTimeout = int(cfg.OmciTimeout.Seconds())
 	openOnuAc.alarmAuditInterval = cfg.AlarmAuditInterval
 	openOnuAc.dlToOnuTimeout4M = cfg.DownloadToOnuTimeout4MB
+	openOnuAc.rpcTimeout = cfg.RPCTimeout
 
 	openOnuAc.pSupportedFsms = &OmciDeviceFsms{
 		"mib-synchronizer": {
@@ -202,200 +203,106 @@
 	return agent
 }
 
-func (oo *OpenONUAC) processInterAdapterONUIndReqMessage(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	msgBody := msg.GetBody()
-	onuIndication := &oop.OnuIndication{}
-	if err := ptypes.UnmarshalAny(msgBody, onuIndication); err != nil {
-		logger.Warnw(ctx, "onu-ind-request-cannot-unmarshal-msg-body", log.Fields{"error": err})
-		return err
-	}
-	//ToDeviceId should address a DeviceHandler instance
-	targetDevice := msg.Header.ToDeviceId
-
-	onuOperstate := onuIndication.GetOperState()
-	waitForDhInstPresent := false
-	if onuOperstate == "up" {
-		//Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
-		//ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
-		//ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
-		//Adopt_device() arrived and DeviceHandler instance was created
-		waitForDhInstPresent = true
-	}
-	if handler := oo.getDeviceHandler(ctx, targetDevice, waitForDhInstPresent); handler != nil {
-		logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": targetDevice,
-			"OnuId":      onuIndication.GetOnuId(),
-			"AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
-			"SNR": onuIndication.GetSerialNumber()})
-
-		if onuOperstate == "up" {
-			return handler.createInterface(ctx, onuIndication)
-		} else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
-			return handler.updateInterface(ctx, onuIndication)
-		} else {
-			logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
-			return fmt.Errorf("invalidOperState: %s, %s", onuOperstate, targetDevice)
-		}
-	}
-	logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
-		"msgToDeviceId": targetDevice})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
+// GetHealthStatus is used as a service readiness validation as a grpc connection
+func (oo *OpenONUAC) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+	return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
 }
 
-// Adapter interface required methods ############## begin #########
-// #################################################################
-
-// for original content compare: (needs according deviceHandler methods)
-// /voltha-openolt-adapter/adaptercore/openolt.go
-
-// Adopt_device creates a new device handler if not present already and then adopts the device
-func (oo *OpenONUAC) Adopt_device(ctx context.Context, device *voltha.Device) error {
+// AdoptDevice creates a new device handler if not present already and then adopts the device
+func (oo *OpenONUAC) AdoptDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	if device == nil {
 		logger.Warn(ctx, "voltha-device-is-nil")
-		return errors.New("nil-device")
+		return nil, errors.New("nil-device")
 	}
 	logger.Infow(ctx, "adopt-device", log.Fields{"device-id": device.Id})
 	var handler *deviceHandler
 	if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
-		handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+		handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
 		oo.addDeviceHandlerToMap(ctx, handler)
-		go handler.adoptOrReconcileDevice(ctx, device)
-		// Launch the creation of the device topic
-		// go oo.createDeviceTopic(device)
+
+		// Setup the grpc communication with the parent adapter
+		if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+			// TODO: Cleanup on failure needed
+			return nil, err
+		}
+
+		go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), device)
 	}
-	return nil
+	return &empty.Empty{}, nil
 }
 
-//Get_ofp_device_info returns OFP information for the given device
-func (oo *OpenONUAC) Get_ofp_device_info(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
-	logger.Errorw(ctx, "device-handler-not-set", log.Fields{"device-id": device.Id})
-	return nil, fmt.Errorf("device-handler-not-set %s", device.Id)
-}
-
-//Get_ofp_port_info returns OFP port information for the given device
-//200630: method removed as per [VOL-3202]: OF port info is now to be delivered within UniPort create
-// cmp changes in onu_uni_port.go::CreateVolthaPort()
-
-//Process_inter_adapter_message sends messages to a target device (between adapters)
-func (oo *OpenONUAC) Process_inter_adapter_message(ctx context.Context, msg *ic.InterAdapterMessage) error {
-	logger.Debugw(ctx, "Process_inter_adapter_message", log.Fields{"msgId": msg.Header.Id,
-		"msgProxyDeviceId": msg.Header.ProxyDeviceId, "msgToDeviceId": msg.Header.ToDeviceId, "Type": msg.Header.Type})
-
-	if msg.Header.Type == ic.InterAdapterMessageType_ONU_IND_REQUEST {
-		// we have to handle ONU_IND_REQUEST already here - see comments in processInterAdapterONUIndReqMessage()
-		return oo.processInterAdapterONUIndReqMessage(ctx, msg)
-	}
-	//ToDeviceId should address a DeviceHandler instance
-	targetDevice := msg.Header.ToDeviceId
-	if handler := oo.getDeviceHandler(ctx, targetDevice, false); handler != nil {
-		/* 200724: modification towards synchronous implementation - possible errors within processing shall be
-		 * 	 in the accordingly delayed response, some timing effect might result in Techprofile processing for multiple UNI's
-		 */
-		return handler.processInterAdapterMessage(ctx, msg)
-		/* so far the processing has been in background with according commented error treatment restrictions:
-		go handler.ProcessInterAdapterMessage(msg)
-		// error treatment might be more sophisticated
-		// by now let's just accept the message on 'communication layer'
-		// message content problems have to be evaluated then in the handler
-		//   and are by now not reported to the calling party (to force what reaction there?)
-		return nil
-		*/
-	}
-	logger.Warnw(ctx, "no handler found for received Inter-Proxy-message", log.Fields{
-		"msgToDeviceId": targetDevice})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
-}
-
-//Process_tech_profile_instance_request not implemented
-func (oo *OpenONUAC) Process_tech_profile_instance_request(ctx context.Context, msg *ic.InterAdapterTechProfileInstanceRequestMessage) *ic.InterAdapterTechProfileDownloadMessage {
-	logger.Error(ctx, "unImplemented")
-	return nil
-}
-
-//Adapter_descriptor not implemented
-func (oo *OpenONUAC) Adapter_descriptor(ctx context.Context) error {
-	return errors.New("unImplemented")
-}
-
-//Device_types unimplemented
-func (oo *OpenONUAC) Device_types(ctx context.Context) (*voltha.DeviceTypes, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Health  returns unimplemented
-func (oo *OpenONUAC) Health(ctx context.Context) (*voltha.HealthStatus, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Reconcile_device is called once when the adapter needs to re-create device - usually on core restart
-func (oo *OpenONUAC) Reconcile_device(ctx context.Context, device *voltha.Device) error {
+//ReconcileDevice is called once when the adapter needs to re-create device - usually on core restart
+func (oo *OpenONUAC) ReconcileDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	if device == nil {
 		logger.Warn(ctx, "reconcile-device-voltha-device-is-nil")
-		return errors.New("nil-device")
+		return nil, errors.New("nil-device")
 	}
 	logger.Infow(ctx, "reconcile-device", log.Fields{"device-id": device.Id})
 	var handler *deviceHandler
 	if handler = oo.getDeviceHandler(ctx, device.Id, false); handler == nil {
-		handler := newDeviceHandler(ctx, oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
+		handler := newDeviceHandler(ctx, oo.coreClient, oo.eventProxy, device, oo)
 		oo.addDeviceHandlerToMap(ctx, handler)
 		handler.device = device
-		if err := handler.coreProxy.DeviceStateUpdate(ctx, device.Id, device.ConnectStatus, voltha.OperStatus_RECONCILING); err != nil {
-			return fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
+		if err := handler.updateDeviceStateInCore(log.WithSpanFromContext(context.Background(), ctx), &ic.DeviceStateFilter{
+			DeviceId:   device.Id,
+			OperStatus: voltha.OperStatus_RECONCILING,
+			ConnStatus: device.ConnectStatus,
+		}); err != nil {
+			return nil, fmt.Errorf("not able to update device state to reconciling. Err : %s", err.Error())
 		}
-		handler.startReconciling(ctx, false)
-		go handler.adoptOrReconcileDevice(ctx, handler.device)
+		// Setup the grpc communication with the parent adapter
+		if err := oo.setupParentInterAdapterClient(ctx, device.ProxyAddress.AdapterEndpoint); err != nil {
+			// TODO: Cleanup on failure needed
+			return nil, err
+		}
+
+		handler.startReconciling(log.WithSpanFromContext(context.Background(), ctx), false)
+		go handler.adoptOrReconcileDevice(log.WithSpanFromContext(context.Background(), ctx), handler.device)
 		// reconcilement will be continued after onu-device entry is added
 	} else {
-		return fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
+		return nil, fmt.Errorf(fmt.Sprintf("device-already-reconciled-or-active-%s", device.Id))
 	}
-	return nil
+	return &empty.Empty{}, nil
 }
 
-//Abandon_device unimplemented
-func (oo *OpenONUAC) Abandon_device(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
-
-//Disable_device disables the given device
-func (oo *OpenONUAC) Disable_device(ctx context.Context, device *voltha.Device) error {
+//DisableDevice disables the given device
+func (oo *OpenONUAC) DisableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "disable-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.disableDevice(ctx, device)
-		return nil
+		go handler.disableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-disable", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Reenable_device enables the onu device after disable
-func (oo *OpenONUAC) Reenable_device(ctx context.Context, device *voltha.Device) error {
+//ReEnableDevice enables the onu device after disable
+func (oo *OpenONUAC) ReEnableDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "reenable-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.reEnableDevice(ctx, device)
-		return nil
+		go handler.reEnableDevice(log.WithSpanFromContext(context.Background(), ctx), device)
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-reenable", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Reboot_device reboots the given device
-func (oo *OpenONUAC) Reboot_device(ctx context.Context, device *voltha.Device) error {
+//RebootDevice reboots the given device
+func (oo *OpenONUAC) RebootDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
 	logger.Infow(ctx, "reboot-device", log.Fields{"device-id": device.Id})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		go handler.rebootDevice(ctx, true, device) //reboot request with device checking
-		return nil
+		go handler.rebootDevice(log.WithSpanFromContext(context.Background(), ctx), true, device) //reboot request with device checking
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-reboot", log.Fields{"device-id": device.Id})
-	return fmt.Errorf("handler-not-found-for-device: %s", device.Id)
+	return nil, fmt.Errorf("handler-not-found-for-device: %s", device.Id)
 }
 
-//Self_test_device unimplemented
-func (oo *OpenONUAC) Self_test_device(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
+// DeleteDevice deletes the given device
+func (oo *OpenONUAC) DeleteDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+	nctx := log.WithSpanFromContext(context.Background(), ctx)
 
-// Delete_device deletes the given device
-func (oo *OpenONUAC) Delete_device(ctx context.Context, device *voltha.Device) error {
-	logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber})
+	logger.Infow(ctx, "delete-device", log.Fields{"device-id": device.Id, "SerialNumber": device.SerialNumber, "ctx": ctx, "nctx": nctx})
 	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
 		var errorsList []error
 
@@ -433,162 +340,104 @@
 		oo.deleteDeviceHandlerToMap(handler)
 		if len(errorsList) > 0 {
 			logger.Errorw(ctx, "one-or-more-error-during-device-delete", log.Fields{"device-id": device.Id})
-			return fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
+			return nil, fmt.Errorf("one-or-more-error-during-device-delete, errors:%v", errorsList)
 		}
-		return nil
+		return &empty.Empty{}, nil
 	}
 	logger.Warnw(ctx, "no handler found for device-deletion", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
 }
 
-//Get_device_details unimplemented
-func (oo *OpenONUAC) Get_device_details(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
+//UpdateFlowsIncrementally updates (add/remove) the flows on a given device
+func (oo *OpenONUAC) UpdateFlowsIncrementally(ctx context.Context, incrFlows *ic.IncrementalFlows) (*empty.Empty, error) {
+	logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": incrFlows.Device.Id})
 
-//Update_flows_bulk returns
-func (oo *OpenONUAC) Update_flows_bulk(ctx context.Context, device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) error {
-	return errors.New("unImplemented")
-}
-
-//Update_flows_incrementally updates (add/remove) the flows on a given device
-func (oo *OpenONUAC) Update_flows_incrementally(ctx context.Context, device *voltha.Device,
-	flows *openflow_13.FlowChanges, groups *openflow_13.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) error {
-
-	logger.Infow(ctx, "update-flows-incrementally", log.Fields{"device-id": device.Id})
 	//flow config is relayed to handler even if the device might be in some 'inactive' state
 	// let the handler or related FSM's decide, what to do with the modified flow state info
 	// at least the flow-remove must be done in respect to internal data, while OMCI activity might not be needed here
 
 	// For now, there is no support for group changes (as in the actual Py-adapter code)
 	//   but processing is continued for flowUpdate possibly also set in the request
-	if groups.ToAdd != nil && groups.ToAdd.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToAdd != nil && incrFlows.Groups.ToAdd.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group add not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
-	if groups.ToRemove != nil && groups.ToRemove.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToRemove != nil && incrFlows.Groups.ToRemove.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group remove not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
-	if groups.ToUpdate != nil && groups.ToUpdate.Items != nil {
-		logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": device.Id})
+	if incrFlows.Groups.ToUpdate != nil && incrFlows.Groups.ToUpdate.Items != nil {
+		logger.Warnw(ctx, "Update-flow-incr: group update not supported (ignored)", log.Fields{"device-id": incrFlows.Device.Id})
 	}
 
-	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		err := handler.FlowUpdateIncremental(ctx, flows, groups, flowMetadata)
-		return err
+	if handler := oo.getDeviceHandler(ctx, incrFlows.Device.Id, false); handler != nil {
+		if err := handler.FlowUpdateIncremental(log.WithSpanFromContext(context.Background(), ctx), incrFlows.Flows, incrFlows.Groups, incrFlows.FlowMetadata); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
 	}
-	logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	logger.Warnw(ctx, "no handler found for incremental flow update", log.Fields{"device-id": incrFlows.Device.Id})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", incrFlows.Device.Id))
 }
 
-//Update_pm_config returns PmConfigs nil or error
-func (oo *OpenONUAC) Update_pm_config(ctx context.Context, device *voltha.Device, pmConfigs *voltha.PmConfigs) error {
-	logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": device.Id})
-	if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
-		return handler.updatePmConfig(ctx, pmConfigs)
+//UpdatePmConfig returns PmConfigs nil or error
+func (oo *OpenONUAC) UpdatePmConfig(ctx context.Context, configs *ic.PmConfigsInfo) (*empty.Empty, error) {
+	logger.Infow(ctx, "update-pm-config", log.Fields{"device-id": configs.DeviceId})
+	if handler := oo.getDeviceHandler(ctx, configs.DeviceId, false); handler != nil {
+		if err := handler.updatePmConfig(log.WithSpanFromContext(context.Background(), ctx), configs.PmConfigs); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
 	}
-	logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": device.Id})
-	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", device.Id))
+	logger.Warnw(ctx, "no handler found for update-pm-config", log.Fields{"device-id": configs.DeviceId})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", configs.DeviceId))
 }
 
-//Receive_packet_out sends packet out to the device
-func (oo *OpenONUAC) Receive_packet_out(ctx context.Context, deviceID string, egressPortNo int, packet *openflow_13.OfpPacketOut) error {
-	return errors.New("unImplemented")
-}
-
-//Suppress_event unimplemented
-func (oo *OpenONUAC) Suppress_event(ctx context.Context, filter *voltha.EventFilter) error {
-	return errors.New("unImplemented")
-}
-
-//Unsuppress_event  unimplemented
-func (oo *OpenONUAC) Unsuppress_event(ctx context.Context, filter *voltha.EventFilter) error {
-	return errors.New("unImplemented")
-}
-
-//Download_image requests downloading some image according to indications as given in request
+//DownloadImage requests downloading some image according to indications as given in request
 //The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Download_image(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	if request != nil && (*request).Name != "" {
-		if !oo.pDownloadManager.imageExists(ctx, request) {
-			logger.Debugw(ctx, "start image download", log.Fields{"image-description": request})
+func (oo *OpenONUAC) DownloadImage(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	ctx = log.WithSpanFromContext(context.Background(), ctx)
+	if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+		if !oo.pDownloadManager.imageExists(ctx, imageInfo.Image) {
+			logger.Debugw(ctx, "start image download", log.Fields{"image-description": imageInfo.Image})
 			// Download_image is not supposed to be blocking, anyway let's call the DownloadManager still synchronously to detect 'fast' problems
 			// the download itself is later done in background
-			err := oo.pDownloadManager.startDownload(ctx, request)
-			return request, err
+			if err := oo.pDownloadManager.startDownload(ctx, imageInfo.Image); err != nil {
+				return nil, err
+			}
+			return imageInfo.Image, nil
 		}
 		// image already exists
-		logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": request})
-		return request, nil
+		logger.Debugw(ctx, "image already downloaded", log.Fields{"image-description": imageInfo.Image})
+		return imageInfo.Image, nil
 	}
-	return request, errors.New("invalid image definition")
+
+	return nil, errors.New("invalid image definition")
 }
 
-//Get_image_download_status unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Get_image_download_status(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Cancel_image_download unimplemented
-//The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Cancel_image_download(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Activate_image_update requests downloading some Onu Software image to the ONU via OMCI
+//ActivateImageUpdate requests downloading some Onu Software image to the INU via OMCI
 //  according to indications as given in request and on success activate the image on the ONU
 //The ImageDownload needs to be called `request`due to library reflection requirements
-func (oo *OpenONUAC) Activate_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	if request != nil && (*request).Name != "" {
-		if oo.pDownloadManager.imageLocallyDownloaded(ctx, request) {
-			if handler := oo.getDeviceHandler(ctx, device.Id, false); handler != nil {
+func (oo *OpenONUAC) ActivateImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	if imageInfo != nil && imageInfo.Image != nil && imageInfo.Image.Name != "" {
+		if oo.pDownloadManager.imageLocallyDownloaded(ctx, imageInfo.Image) {
+			if handler := oo.getDeviceHandler(ctx, imageInfo.Device.Id, false); handler != nil {
 				logger.Debugw(ctx, "image download on omci requested", log.Fields{
-					"image-description": request, "device-id": device.Id})
-				err := handler.doOnuSwUpgrade(ctx, request, oo.pDownloadManager)
-				return request, err
+					"image-description": imageInfo.Image, "device-id": imageInfo.Device.Id})
+				if err := handler.doOnuSwUpgrade(ctx, imageInfo.Image, oo.pDownloadManager); err != nil {
+					return nil, err
+				}
+				return imageInfo.Image, nil
 			}
-			logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": device.Id})
-			return request, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", device.Id))
+			logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": imageInfo.Device.Id})
+			return nil, fmt.Errorf(fmt.Sprintf("handler-not-found - device-id: %s", imageInfo.Device.Id))
 		}
-		logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": request})
-		return request, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", device.Id))
+		logger.Debugw(ctx, "image not yet downloaded on activate request", log.Fields{"image-description": imageInfo.Image})
+		return nil, fmt.Errorf(fmt.Sprintf("image-not-yet-downloaded - device-id: %s", imageInfo.Device.Id))
 	}
-	return request, errors.New("invalid image definition")
+	return nil, errors.New("invalid image definition")
 }
 
-//Revert_image_update unimplemented
-func (oo *OpenONUAC) Revert_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
-	return nil, errors.New("unImplemented")
-}
-
-// Enable_port to Enable PON/NNI interface - seems not to be used/required according to python code
-func (oo *OpenONUAC) Enable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
-	return errors.New("unImplemented")
-}
-
-// Disable_port to Disable pon/nni interface  - seems not to be used/required according to python code
-func (oo *OpenONUAC) Disable_port(ctx context.Context, deviceID string, port *voltha.Port) error {
-	return errors.New("unImplemented")
-}
-
-//Child_device_lost - unimplemented
-//needed for if update >= 3.1.x
-func (oo *OpenONUAC) Child_device_lost(ctx context.Context, device *voltha.Device) error {
-	return errors.New("unImplemented")
-}
-
-// Start_omci_test unimplemented
-func (oo *OpenONUAC) Start_omci_test(ctx context.Context, device *voltha.Device, request *voltha.OmciTestRequest) (*voltha.TestResponse, error) {
-	return nil, errors.New("unImplemented")
-}
-
-// Get_ext_value - unimplemented
-func (oo *OpenONUAC) Get_ext_value(ctx context.Context, deviceID string, device *voltha.Device, valueparam voltha.ValueType_Type) (*voltha.ReturnValues, error) {
-	return nil, errors.New("unImplemented")
-}
-
-//Single_get_value_request handles the core request to retrieve uni status
-func (oo *OpenONUAC) Single_get_value_request(ctx context.Context, request extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
+//GetSingleValue handles the core request to retrieve uni status
+func (oo *OpenONUAC) GetSingleValue(ctx context.Context, request *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
 	logger.Infow(ctx, "Single_get_value_request", log.Fields{"request": request})
 
 	if handler := oo.getDeviceHandler(ctx, request.TargetId, false); handler != nil {
@@ -599,7 +448,7 @@
 			commChan := make(chan Message)
 			respChan := make(chan extension.SingleGetValueResponse)
 			// Initiate the self test request
-			if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, request, commChan, respChan); err != nil {
+			if err := handler.pSelfTestHdlr.SelfTestRequestStart(ctx, *request, commChan, respChan); err != nil {
 				return &extension.SingleGetValueResponse{
 					Response: &extension.GetValueResponse{
 						Status:    extension.GetValueResponse_ERROR,
@@ -627,9 +476,9 @@
 //   The reason for that was never finally investigated.
 //   To be on the safe side argument names are left here always as defined in iAdapter.go .
 
-// Download_onu_image downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
+// DownloadOnuImage downloads (and optionally activates and commits) the indicated ONU image to the requested ONU(s)
 //   if the image is not yet present on the adapter it has to be automatically downloaded
-func (oo *OpenONUAC) Download_onu_image(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) DownloadOnuImage(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
 	if request != nil && len((*request).DeviceId) > 0 && (*request).Image.Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*request).Image.Version
@@ -645,8 +494,18 @@
 			loDeviceImageState.ImageState = &loImageState
 			loDeviceImageState.ImageState.Version = (*request).Image.Version
 
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot start ONU download for requested device
+				logger.Warnw(ctx, "no handler found for image activation", log.Fields{"device-id": loDeviceID})
+				loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+				loDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+				loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
+				continue
+			}
+
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device for image download",
 					log.Fields{"device-id": loDeviceID, "err": err})
@@ -718,6 +577,17 @@
 					loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
 				}
 			}
+
+			// start the ONU download activity for each possible device
+			logger.Debugw(ctx, "image download on omci requested", log.Fields{
+				"image-id": imageIdentifier, "device-id": loDeviceID})
+			//onu upgrade handling called in background without immediate error evaluation here
+			//  as the processing can be done for multiple ONU's and an error on one ONU should not stop processing for others
+			//  state/progress/success of the request has to be verified using the Get_onu_image_status() API
+			go handler.onuSwUpgradeAfterDownload(ctx, request, oo.pFileManager, imageIdentifier)
+			loDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_STARTED
+			loDeviceImageState.ImageState.Reason = voltha.ImageState_NO_ERROR
+			loDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, &loDeviceImageState)
 		}
 		pImageResp := &loResponse
@@ -726,9 +596,9 @@
 	return nil, errors.New("invalid image download parameters")
 }
 
-// Get_onu_image_status delivers the adapter-related information about the download/activation/commitment
+// GetOnuImageStatus delivers the adapter-related information about the download/activation/commitment
 //   status for the requested image
-func (oo *OpenONUAC) Get_onu_image_status(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) GetOnuImageStatus(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -737,12 +607,20 @@
 		var vendorID string
 		for _, pCommonID := range (*in).DeviceId {
 			loDeviceID := (*pCommonID).Id
-			pDeviceImageState := &voltha.DeviceImageState{
-				DeviceId: loDeviceID,
+			pDeviceImageState := &voltha.DeviceImageState{DeviceId: loDeviceID}
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot get the handler
+				logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
+				pDeviceImageState.DeviceId = loDeviceID
+				pDeviceImageState.ImageState.Version = (*in).Version
+				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_FAILED
+				pDeviceImageState.ImageState.Reason = voltha.ImageState_UNKNOWN_ERROR
+				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+				continue
 			}
-			vendorIDSet = false
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device to get image status",
 					log.Fields{"device-id": loDeviceID, "err": err})
@@ -780,23 +658,10 @@
 					}
 					pDeviceImageState.ImageState = pImageState
 				} else {
-					// get the handler for the device
-					if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
-						logger.Debugw(ctx, "image status request for", log.Fields{
-							"image-id": imageIdentifier, "device-id": loDeviceID})
-						//status request is called synchronously to collect the indications for all concerned devices
-						pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
-					} else {
-						//cannot get the handler
-						logger.Warnw(ctx, "no handler found for image status request ", log.Fields{"device-id": loDeviceID})
-						pImageState := &voltha.ImageState{
-							Version:       (*in).Version,
-							DownloadState: voltha.ImageState_DOWNLOAD_UNKNOWN, //no statement about last activity possible
-							Reason:        voltha.ImageState_UNKNOWN_ERROR,    //something like "DEVICE_NOT_EXISTS" would be better (proto def)
-							ImageState:    voltha.ImageState_IMAGE_UNKNOWN,
-						}
-						pDeviceImageState.ImageState = pImageState
-					}
+					logger.Debugw(ctx, "image status request for", log.Fields{
+						"image-id": imageIdentifier, "device-id": loDeviceID})
+					//status request is called synchronously to collect the indications for all concerned devices
+					pDeviceImageState.ImageState = handler.requestOnuSwUpgradeState(ctx, imageIdentifier, (*in).Version)
 				}
 			}
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
@@ -807,8 +672,8 @@
 	return nil, errors.New("invalid image status request parameters")
 }
 
-// Abort_onu_image_upgrade stops the actual download/activation/commitment process (on next possibly step)
-func (oo *OpenONUAC) Abort_onu_image_upgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+// AbortOnuImageUpgrade stops the actual download/activation/commitment process (on next possibly step)
+func (oo *OpenONUAC) AbortOnuImageUpgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -816,16 +681,30 @@
 		var vendorID string
 		for _, pCommonID := range (*in).DeviceId {
 			loDeviceID := (*pCommonID).Id
-			onuVolthaDevice, err := oo.coreProxy.GetDevice(log.WithSpanFromContext(context.TODO(), ctx),
-				loDeviceID, loDeviceID)
+			pDeviceImageState := &voltha.DeviceImageState{}
+			loImageState := voltha.ImageState{}
+			pDeviceImageState.ImageState = &loImageState
+
+			handler := oo.getDeviceHandler(ctx, loDeviceID, false)
+			if handler == nil {
+				//cannot start ONU download for requested device
+				logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
+				pDeviceImageState.DeviceId = loDeviceID
+				pDeviceImageState.ImageState.Version = (*in).Version
+				//nolint:misspell
+				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
+				//nolint:misspell
+				pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST
+				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
+				loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
+				continue
+			}
+			onuVolthaDevice, err := handler.getDeviceFromCore(ctx, loDeviceID)
 			if err != nil || onuVolthaDevice == nil {
 				logger.Warnw(ctx, "Failed to fetch Onu device to abort its download",
 					log.Fields{"device-id": loDeviceID, "err": err})
 				continue //try the work with next deviceId
 			}
-			pDeviceImageState := &voltha.DeviceImageState{}
-			loImageState := voltha.ImageState{}
-			pDeviceImageState.ImageState = &loImageState
 
 			if firstDevice {
 				//start/verify download of the image to the adapter based on first found device only
@@ -844,20 +723,10 @@
 			}
 
 			// cancel the ONU upgrade activity for each possible device
-			if handler := oo.getDeviceHandler(ctx, loDeviceID, false); handler != nil {
-				logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
-					"image-id": imageIdentifier, "device-id": loDeviceID})
-				//upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
-				handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
-			} else {
-				//cannot start aborting ONU download for requested device
-				logger.Warnw(ctx, "no handler found for aborting upgrade ", log.Fields{"device-id": loDeviceID})
-				pDeviceImageState.DeviceId = loDeviceID
-				pDeviceImageState.ImageState.Version = (*in).Version
-				pDeviceImageState.ImageState.DownloadState = voltha.ImageState_DOWNLOAD_CANCELLED
-				pDeviceImageState.ImageState.Reason = voltha.ImageState_CANCELLED_ON_REQUEST //something better would be possible with protos modification
-				pDeviceImageState.ImageState.ImageState = voltha.ImageState_IMAGE_UNKNOWN
-			}
+			logger.Debugw(ctx, "image upgrade abort requested", log.Fields{
+				"image-id": imageIdentifier, "device-id": loDeviceID})
+			//upgrade cancel is called synchronously to collect the imageResponse indications for all concerned devices
+			handler.cancelOnuSwUpgrade(ctx, imageIdentifier, (*in).Version, pDeviceImageState)
 			loResponse.DeviceImageStates = append(loResponse.DeviceImageStates, pDeviceImageState)
 		}
 		if !firstDevice {
@@ -871,23 +740,23 @@
 	return nil, errors.New("invalid image upgrade abort parameters")
 }
 
-// Get_onu_images retrieves the ONU SW image status information via OMCI
-func (oo *OpenONUAC) Get_onu_images(ctx context.Context, deviceID string) (*voltha.OnuImages, error) {
-	logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": deviceID})
-	if handler := oo.getDeviceHandler(ctx, deviceID, false); handler != nil {
+// GetOnuImages retrieves the ONU SW image status information via OMCI
+func (oo *OpenONUAC) GetOnuImages(ctx context.Context, id *common.ID) (*voltha.OnuImages, error) {
+	logger.Infow(ctx, "Get_onu_images", log.Fields{"device-id": id.Id})
+	if handler := oo.getDeviceHandler(ctx, id.Id, false); handler != nil {
 		images, err := handler.getOnuImages(ctx)
 		if err == nil {
 			return images, nil
 		}
-		return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, deviceID))
+		return nil, fmt.Errorf(fmt.Sprintf("%s-%s", err, id.Id))
 	}
-	logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": deviceID})
-	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", deviceID))
+	logger.Warnw(ctx, "no handler found for Get_onu_images", log.Fields{"device-id": id.Id})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", id.Id))
 }
 
-// Activate_onu_image initiates the activation of the image for the requested ONU(s)
+// ActivateOnuImage initiates the activation of the image for the requested ONU(s)
 //  precondition: image downloaded and not yet activated or image refers to current inactive image
-func (oo *OpenONUAC) Activate_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) ActivateOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -933,9 +802,9 @@
 	return nil, errors.New("invalid image activation parameters")
 }
 
-// Commit_onu_image enforces the commitment of the image for the requested ONU(s)
+// CommitOnuImage enforces the commitment of the image for the requested ONU(s)
 //  precondition: image activated and not yet committed
-func (oo *OpenONUAC) Commit_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+func (oo *OpenONUAC) CommitOnuImage(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
 	if in != nil && len((*in).DeviceId) > 0 && (*in).Version != "" {
 		loResponse := voltha.DeviceImageResponse{}
 		imageIdentifier := (*in).Version
@@ -983,3 +852,281 @@
 
 // Adapter interface required methods ################ end #########
 // #################################################################
+
+/*
+ *
+ * ONU inter adapter service
+ *
+ */
+
+// OnuIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OnuIndication(ctx context.Context, onuInd *ic.OnuIndicationMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "onu-indication", log.Fields{"onu-indication": onuInd})
+
+	if onuInd == nil || onuInd.OnuIndication == nil {
+		return nil, fmt.Errorf("invalid-onu-indication-%v", onuInd)
+	}
+
+	onuIndication := onuInd.OnuIndication
+	onuOperstate := onuIndication.GetOperState()
+	waitForDhInstPresent := false
+	if onuOperstate == "up" {
+		//Race condition (relevant in BBSIM-environment only): Due to unsynchronized processing of olt-adapter and rw_core,
+		//ONU_IND_REQUEST msg by olt-adapter could arrive a little bit earlier than rw_core was able to announce the corresponding
+		//ONU by RPC of Adopt_device(). Therefore it could be necessary to wait with processing of ONU_IND_REQUEST until call of
+		//Adopt_device() arrived and DeviceHandler instance was created
+		waitForDhInstPresent = true
+	}
+	if handler := oo.getDeviceHandler(ctx, onuInd.DeviceId, waitForDhInstPresent); handler != nil {
+		logger.Infow(ctx, "onu-ind-request", log.Fields{"device-id": onuInd.DeviceId,
+			"OnuId":      onuIndication.GetOnuId(),
+			"AdminState": onuIndication.GetAdminState(), "OperState": onuOperstate,
+			"SNR": onuIndication.GetSerialNumber()})
+
+		if onuOperstate == "up" {
+			if err := handler.createInterface(ctx, onuIndication); err != nil {
+				return nil, err
+			}
+			return &empty.Empty{}, nil
+		} else if (onuOperstate == "down") || (onuOperstate == "unreachable") {
+			return nil, handler.updateInterface(ctx, onuIndication)
+		} else {
+			logger.Errorw(ctx, "unknown-onu-ind-request operState", log.Fields{"OnuId": onuIndication.GetOnuId()})
+			return nil, fmt.Errorf("invalidOperState: %s, %s", onuOperstate, onuInd.DeviceId)
+		}
+	}
+	logger.Warnw(ctx, "no handler found for received onu-ind-request", log.Fields{
+		"msgToDeviceId": onuInd.DeviceId})
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", onuInd.DeviceId))
+}
+
+// OmciIndication is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) OmciIndication(ctx context.Context, msg *ic.OmciMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "omci-response", log.Fields{"parent-device-id": msg.ParentDeviceId, "child-device-id": msg.ChildDeviceId})
+
+	if handler := oo.getDeviceHandler(ctx, msg.ChildDeviceId, false); handler != nil {
+		if err := handler.handleOMCIIndication(log.WithSpanFromContext(context.Background(), ctx), msg); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", msg.ChildDeviceId))
+}
+
+// DownloadTechProfile is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DownloadTechProfile(ctx context.Context, tProfile *ic.TechProfileDownloadMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "download-tech-profile", log.Fields{"uni-id": tProfile.UniId})
+
+	if handler := oo.getDeviceHandler(ctx, tProfile.DeviceId, false); handler != nil {
+		if err := handler.handleTechProfileDownloadRequest(log.WithSpanFromContext(context.Background(), ctx), tProfile); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tProfile.DeviceId))
+}
+
+// DeleteGemPort is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteGemPort(ctx context.Context, gPort *ic.DeleteGemPortMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "delete-gem-port", log.Fields{"device-id": gPort.DeviceId, "uni-id": gPort.UniId})
+
+	if handler := oo.getDeviceHandler(ctx, gPort.DeviceId, false); handler != nil {
+		if err := handler.handleDeleteGemPortRequest(log.WithSpanFromContext(context.Background(), ctx), gPort); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", gPort.DeviceId))
+}
+
+// DeleteTCont is part of the ONU Inter-adapter service API.
+func (oo *OpenONUAC) DeleteTCont(ctx context.Context, tConf *ic.DeleteTcontMessage) (*empty.Empty, error) {
+	logger.Debugw(ctx, "delete-tcont", log.Fields{"tconf": tConf})
+
+	if handler := oo.getDeviceHandler(ctx, tConf.DeviceId, false); handler != nil {
+		if err := handler.handleDeleteTcontRequest(log.WithSpanFromContext(context.Background(), ctx), tConf); err != nil {
+			return nil, err
+		}
+		return &empty.Empty{}, nil
+	}
+	return nil, fmt.Errorf(fmt.Sprintf("handler-not-found-%s", tConf.DeviceId))
+}
+
+/*
+ * Parent GRPC clients
+ */
+
+func (oo *OpenONUAC) setupParentInterAdapterClient(ctx context.Context, endpoint string) error {
+	logger.Infow(ctx, "setting-parent-adapter-connection", log.Fields{"parent-endpoint": endpoint})
+	oo.lockParentAdapterClients.Lock()
+	defer oo.lockParentAdapterClients.Unlock()
+	if _, ok := oo.parentAdapterClients[endpoint]; ok {
+		return nil
+	}
+
+	childClient, err := vgrpc.NewClient(endpoint,
+		oo.oltAdapterRestarted,
+		vgrpc.ActivityCheck(true))
+
+	if err != nil {
+		return err
+	}
+
+	oo.parentAdapterClients[endpoint] = childClient
+
+	go oo.parentAdapterClients[endpoint].Start(log.WithSpanFromContext(context.TODO(), ctx), setAndTestAdapterServiceHandler)
+
+	// Wait until we have a connection to the child adapter.
+	// Unlimited retries or until context expires
+	subCtx := log.WithSpanFromContext(context.TODO(), ctx)
+	backoff := vgrpc.NewBackoff(oo.config.MinBackoffRetryDelay, oo.config.MaxBackoffRetryDelay, 0)
+	for {
+		client, err := oo.parentAdapterClients[endpoint].GetOltInterAdapterServiceClient()
+		if err == nil && client != nil {
+			logger.Infow(subCtx, "connected-to-parent-adapter", log.Fields{"parent-endpoint": endpoint})
+			break
+		}
+		logger.Warnw(subCtx, "connection-to-parent-adapter-not-ready", log.Fields{"error": err, "parent-endpoint": endpoint})
+		// Backoff
+		if err = backoff.Backoff(subCtx); err != nil {
+			logger.Errorw(subCtx, "received-error-on-backoff", log.Fields{"error": err, "parent-endpoint": endpoint})
+			break
+		}
+	}
+	return nil
+}
+
+func (oo *OpenONUAC) getParentAdapterServiceClient(endpoint string) (adapter_services.OltInterAdapterServiceClient, error) {
+	// First check from cache
+	oo.lockParentAdapterClients.RLock()
+	if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+		oo.lockParentAdapterClients.RUnlock()
+		return pgClient.GetOltInterAdapterServiceClient()
+	}
+	oo.lockParentAdapterClients.RUnlock()
+
+	// Set the parent connection - can occur on restarts
+	ctx, cancel := context.WithTimeout(context.Background(), oo.config.RPCTimeout)
+	err := oo.setupParentInterAdapterClient(ctx, endpoint)
+	cancel()
+	if err != nil {
+		return nil, err
+	}
+
+	// Get the parent client now
+	oo.lockParentAdapterClients.RLock()
+	defer oo.lockParentAdapterClients.RUnlock()
+	if pgClient, ok := oo.parentAdapterClients[endpoint]; ok {
+		return pgClient.GetOltInterAdapterServiceClient()
+	}
+
+	return nil, fmt.Errorf("no-client-for-endpoint-%s", endpoint)
+}
+
+// TODO:  Any action the adapter needs to do following an olt adapter restart?
+func (oo *OpenONUAC) oltAdapterRestarted(ctx context.Context, endPoint string) error {
+	logger.Errorw(ctx, "olt-adapter-restarted", log.Fields{"endpoint": endPoint})
+	return nil
+}
+
+// setAndTestAdapterServiceHandler is used to test whether the remote gRPC service is up
+func setAndTestAdapterServiceHandler(ctx context.Context, conn *grpc.ClientConn) interface{} {
+	svc := adapter_services.NewOltInterAdapterServiceClient(conn)
+	if h, err := svc.GetHealthStatus(ctx, &empty.Empty{}); err != nil || h.State != voltha.HealthStatus_HEALTHY {
+		return nil
+	}
+	return svc
+}
+
+/*
+ *
+ * Unimplemented APIs
+ *
+ */
+
+//GetOfpDeviceInfo returns OFP information for the given device.  Method not implemented as per [VOL-3202].
+// OF port info is now to be delivered within UniPort create cmp changes in onu_uni_port.go::CreateVolthaPort()
+//
+func (oo *OpenONUAC) GetOfpDeviceInfo(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SimulateAlarm is unimplemented
+func (oo *OpenONUAC) SimulateAlarm(context.Context, *ic.SimulateAlarmMessage) (*common.OperationResp, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SetExtValue is unimplemented
+func (oo *OpenONUAC) SetExtValue(context.Context, *ic.SetExtValueMessage) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SetSingleValue is unimplemented
+func (oo *OpenONUAC) SetSingleValue(context.Context, *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//StartOmciTest not implemented
+func (oo *OpenONUAC) StartOmciTest(ctx context.Context, test *ic.OMCITest) (*voltha.TestResponse, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SuppressEvent unimplemented
+func (oo *OpenONUAC) SuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//UnSuppressEvent  unimplemented
+func (oo *OpenONUAC) UnSuppressEvent(ctx context.Context, filter *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//GetImageDownloadStatus is unimplemented
+func (oo *OpenONUAC) GetImageDownloadStatus(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//CancelImageDownload is unimplemented
+func (oo *OpenONUAC) CancelImageDownload(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//RevertImageUpdate is unimplemented
+func (oo *OpenONUAC) RevertImageUpdate(ctx context.Context, imageInfo *ic.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// UpdateFlowsBulk is unimplemented
+func (oo *OpenONUAC) UpdateFlowsBulk(ctx context.Context, flows *ic.BulkFlows) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SelfTestDevice unimplented
+func (oo *OpenONUAC) SelfTestDevice(ctx context.Context, device *voltha.Device) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+//SendPacketOut sends packet out to the device
+func (oo *OpenONUAC) SendPacketOut(ctx context.Context, packet *ic.PacketOut) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// EnablePort to Enable PON/NNI interface - seems not to be used/required according to python code
+func (oo *OpenONUAC) EnablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// DisablePort to Disable pon/nni interface  - seems not to be used/required according to python code
+func (oo *OpenONUAC) DisablePort(ctx context.Context, port *voltha.Port) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// GetExtValue - unimplemented
+func (oo *OpenONUAC) GetExtValue(ctx context.Context, extInfo *ic.GetExtValueMessage) (*voltha.ReturnValues, error) {
+	return nil, errors.New("unImplemented")
+}
+
+// ChildDeviceLost - unimplemented
+func (oo *OpenONUAC) ChildDeviceLost(ctx context.Context, childDevice *voltha.Device) (*empty.Empty, error) {
+	return nil, errors.New("unImplemented")
+}
diff --git a/internal/pkg/onuadaptercore/openonuimpl.go b/internal/pkg/onuadaptercore/openonuimpl.go
index 8e6e187..efa4e96 100644
--- a/internal/pkg/onuadaptercore/openonuimpl.go
+++ b/internal/pkg/onuadaptercore/openonuimpl.go
@@ -20,7 +20,6 @@
 import (
 	"context"
 	"errors"
-	//"github.com/opencord/voltha-lib-go/v5/pkg/log"
 )
 
 /*
diff --git a/internal/pkg/onuadaptercore/uniportadmin.go b/internal/pkg/onuadaptercore/uniportadmin.go
index 0c22500..2ceea25 100644
--- a/internal/pkg/onuadaptercore/uniportadmin.go
+++ b/internal/pkg/onuadaptercore/uniportadmin.go
@@ -27,9 +27,7 @@
 
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	//"github.com/opencord/voltha-protos/v4/go/openflow_13"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 //lockStateFsm defines the structure for the state machine to lock/unlock the ONU UNI ports via OMCI
diff --git a/internal/pkg/onuadaptercore/uniportstatus.go b/internal/pkg/onuadaptercore/uniportstatus.go
index fe9af4c..f7cfe4b 100644
--- a/internal/pkg/onuadaptercore/uniportstatus.go
+++ b/internal/pkg/onuadaptercore/uniportstatus.go
@@ -19,11 +19,12 @@
 
 import (
 	"context"
+	"time"
+
 	"github.com/opencord/omci-lib-go"
 	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	"time"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/extension"
 )
 
 const (
diff --git a/pkg/mocks/common.go b/pkg/mocks/common.go
index 72a4c30..be78c34 100644
--- a/pkg/mocks/common.go
+++ b/pkg/mocks/common.go
@@ -18,7 +18,7 @@
 package mocks
 
 import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
 )
 
 func init() {
diff --git a/vendor/github.com/DataDog/zstd/.travis.yml b/vendor/github.com/DataDog/zstd/.travis.yml
deleted file mode 100644
index 629470c..0000000
--- a/vendor/github.com/DataDog/zstd/.travis.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-dist: xenial
-language: go
-
-go:
-  - 1.10.x
-  - 1.11.x
-  - 1.12.x
-
-os:
-  - linux
-  - osx
-
-matrix:
-  include:
-    name: "Go 1.11.x CentOS 32bits"
-    language: go
-    go: 1.11.x
-    os: linux
-    services:
-      - docker
-    script:
-      # Please update Go version in travis_test_32 as needed
-      - "docker run -i -v \"${PWD}:/zstd\" toopher/centos-i386:centos6 /bin/bash -c \"linux32 --32bit i386 /zstd/travis_test_32.sh\""
-
-install:
-  - "wget https://github.com/DataDog/zstd/files/2246767/mr.zip"
-  - "unzip mr.zip"
-script:
-  - "go build"
-  - "PAYLOAD=`pwd`/mr go test -v"
-  - "PAYLOAD=`pwd`/mr go test -bench ."
diff --git a/vendor/github.com/DataDog/zstd/LICENSE b/vendor/github.com/DataDog/zstd/LICENSE
deleted file mode 100644
index 345c1eb..0000000
--- a/vendor/github.com/DataDog/zstd/LICENSE
+++ /dev/null
@@ -1,27 +0,0 @@
-Simplified BSD License
-
-Copyright (c) 2016, Datadog <info@datadoghq.com>
-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 copyright holder 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/DataDog/zstd/README.md b/vendor/github.com/DataDog/zstd/README.md
deleted file mode 100644
index b32c3e7..0000000
--- a/vendor/github.com/DataDog/zstd/README.md
+++ /dev/null
@@ -1,120 +0,0 @@
-# Zstd Go Wrapper
-
-[C Zstd Homepage](https://github.com/Cyan4973/zstd)
-
-The current headers and C files are from *v1.4.1* (Commit
-[52181f8](https://github.com/facebook/zstd/releases/tag/v1.4.1)).
-
-## Usage
-
-There are two main APIs:
-
-* simple Compress/Decompress
-* streaming API (io.Reader/io.Writer)
-
-The compress/decompress APIs mirror that of lz4, while the streaming API was
-designed to be a drop-in replacement for zlib.
-
-### Simple `Compress/Decompress`
-
-
-```go
-// Compress compresses the byte array given in src and writes it to dst.
-// If you already have a buffer allocated, you can pass it to prevent allocation
-// If not, you can pass nil as dst.
-// If the buffer is too small, it will be reallocated, resized, and returned bu the function
-// If dst is nil, this will allocate the worst case size (CompressBound(src))
-Compress(dst, src []byte) ([]byte, error)
-```
-
-```go
-// CompressLevel is the same as Compress but you can pass another compression level
-CompressLevel(dst, src []byte, level int) ([]byte, error)
-```
-
-```go
-// Decompress will decompress your payload into dst.
-// If you already have a buffer allocated, you can pass it to prevent allocation
-// If not, you can pass nil as dst (allocates a 4*src size as default).
-// If the buffer is too small, it will retry 3 times by doubling the dst size
-// After max retries, it will switch to the slower stream API to be sure to be able
-// to decompress. Currently switches if compression ratio > 4*2**3=32.
-Decompress(dst, src []byte) ([]byte, error)
-```
-
-### Stream API
-
-```go
-// NewWriter creates a new object that can optionally be initialized with
-// a precomputed dictionary. If dict is nil, compress without a dictionary.
-// The dictionary array should not be changed during the use of this object.
-// You MUST CALL Close() to write the last bytes of a zstd stream and free C objects.
-NewWriter(w io.Writer) *Writer
-NewWriterLevel(w io.Writer, level int) *Writer
-NewWriterLevelDict(w io.Writer, level int, dict []byte) *Writer
-
-// Write compresses the input data and write it to the underlying writer
-(w *Writer) Write(p []byte) (int, error)
-
-// Close flushes the buffer and frees C zstd objects
-(w *Writer) Close() error
-```
-
-```go
-// NewReader returns a new io.ReadCloser that will decompress data from the
-// underlying reader.  If a dictionary is provided to NewReaderDict, it must
-// not be modified until Close is called.  It is the caller's responsibility
-// to call Close, which frees up C objects.
-NewReader(r io.Reader) io.ReadCloser
-NewReaderDict(r io.Reader, dict []byte) io.ReadCloser
-```
-
-### Benchmarks (benchmarked with v0.5.0)
-
-The author of Zstd also wrote lz4. Zstd is intended to occupy a speed/ratio
-level similar to what zlib currently provides.  In our tests, the can always
-be made to be better than zlib by chosing an appropriate level while still
-keeping compression and decompression time faster than zlib.
-
-You can run the benchmarks against your own payloads by using the Go benchmarks tool.
-Just export your payload filepath as the `PAYLOAD` environment variable and run the benchmarks:
-
-```go
-go test -bench .
-```
-
-Compression of a 7Mb pdf zstd (this wrapper) vs [czlib](https://github.com/DataDog/czlib):
-```
-BenchmarkCompression               5     221056624 ns/op      67.34 MB/s
-BenchmarkDecompression           100      18370416 ns/op     810.32 MB/s
-
-BenchmarkFzlibCompress             2     610156603 ns/op      24.40 MB/s
-BenchmarkFzlibDecompress          20      81195246 ns/op     183.33 MB/s
-```
-
-Ratio is also better by a margin of ~20%.
-Compression speed is always better than zlib on all the payloads we tested;
-However, [czlib](https://github.com/DataDog/czlib) has optimisations that make it
-faster at decompressiong small payloads:
-
-```
-Testing with size: 11... czlib: 8.97 MB/s, zstd: 3.26 MB/s
-Testing with size: 27... czlib: 23.3 MB/s, zstd: 8.22 MB/s
-Testing with size: 62... czlib: 31.6 MB/s, zstd: 19.49 MB/s
-Testing with size: 141... czlib: 74.54 MB/s, zstd: 42.55 MB/s
-Testing with size: 323... czlib: 155.14 MB/s, zstd: 99.39 MB/s
-Testing with size: 739... czlib: 235.9 MB/s, zstd: 216.45 MB/s
-Testing with size: 1689... czlib: 116.45 MB/s, zstd: 345.64 MB/s
-Testing with size: 3858... czlib: 176.39 MB/s, zstd: 617.56 MB/s
-Testing with size: 8811... czlib: 254.11 MB/s, zstd: 824.34 MB/s
-Testing with size: 20121... czlib: 197.43 MB/s, zstd: 1339.11 MB/s
-Testing with size: 45951... czlib: 201.62 MB/s, zstd: 1951.57 MB/s
-```
-
-zstd starts to shine with payloads > 1KB
-
-### Stability - Current state: STABLE
-
-The C library seems to be pretty stable and according to the author has been tested and fuzzed.
-
-For the Go wrapper, the test cover most usual cases and we have succesfully tested it on all staging and prod data.
diff --git a/vendor/github.com/DataDog/zstd/ZSTD_LICENSE b/vendor/github.com/DataDog/zstd/ZSTD_LICENSE
deleted file mode 100644
index a793a80..0000000
--- a/vendor/github.com/DataDog/zstd/ZSTD_LICENSE
+++ /dev/null
@@ -1,30 +0,0 @@
-BSD License
-
-For Zstandard software
-
-Copyright (c) 2016-present, Facebook, 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 Facebook 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.
diff --git a/vendor/github.com/DataDog/zstd/bitstream.h b/vendor/github.com/DataDog/zstd/bitstream.h
deleted file mode 100644
index d955bd6..0000000
--- a/vendor/github.com/DataDog/zstd/bitstream.h
+++ /dev/null
@@ -1,455 +0,0 @@
-/* ******************************************************************
-   bitstream
-   Part of FSE library
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*
-*  This API consists of small unitary functions, which must be inlined for best performance.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-/*-****************************************
-*  Dependencies
-******************************************/
-#include "mem.h"            /* unaligned access routines */
-#include "debug.h"          /* assert(), DEBUGLOG(), RAWLOG() */
-#include "error_private.h"  /* error codes and messages */
-
-
-/*=========================================
-*  Target specific
-=========================================*/
-#if defined(__BMI__) && defined(__GNUC__)
-#  include <immintrin.h>   /* support for bextr (experimental) */
-#endif
-
-#define STREAM_ACCUMULATOR_MIN_32  25
-#define STREAM_ACCUMULATOR_MIN_64  57
-#define STREAM_ACCUMULATOR_MIN    ((U32)(MEM_32bits() ? STREAM_ACCUMULATOR_MIN_32 : STREAM_ACCUMULATOR_MIN_64))
-
-
-/*-******************************************
-*  bitStream encoding API (write forward)
-********************************************/
-/* bitStream can mix input from multiple sources.
- * A critical property of these streams is that they encode and decode in **reverse** direction.
- * So the first bit sequence you add will be the last to be read, like a LIFO stack.
- */
-typedef struct {
-    size_t bitContainer;
-    unsigned bitPos;
-    char*  startPtr;
-    char*  ptr;
-    char*  endPtr;
-} BIT_CStream_t;
-
-MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC, void* dstBuffer, size_t dstCapacity);
-MEM_STATIC void   BIT_addBits(BIT_CStream_t* bitC, size_t value, unsigned nbBits);
-MEM_STATIC void   BIT_flushBits(BIT_CStream_t* bitC);
-MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC);
-
-/* Start with initCStream, providing the size of buffer to write into.
-*  bitStream will never write outside of this buffer.
-*  `dstCapacity` must be >= sizeof(bitD->bitContainer), otherwise @return will be an error code.
-*
-*  bits are first added to a local register.
-*  Local register is size_t, hence 64-bits on 64-bits systems, or 32-bits on 32-bits systems.
-*  Writing data into memory is an explicit operation, performed by the flushBits function.
-*  Hence keep track how many bits are potentially stored into local register to avoid register overflow.
-*  After a flushBits, a maximum of 7 bits might still be stored into local register.
-*
-*  Avoid storing elements of more than 24 bits if you want compatibility with 32-bits bitstream readers.
-*
-*  Last operation is to close the bitStream.
-*  The function returns the final size of CStream in bytes.
-*  If data couldn't fit into `dstBuffer`, it will return a 0 ( == not storable)
-*/
-
-
-/*-********************************************
-*  bitStream decoding API (read backward)
-**********************************************/
-typedef struct {
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-    const char* limitPtr;
-} BIT_DStream_t;
-
-typedef enum { BIT_DStream_unfinished = 0,
-               BIT_DStream_endOfBuffer = 1,
-               BIT_DStream_completed = 2,
-               BIT_DStream_overflow = 3 } BIT_DStream_status;  /* result of BIT_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD);
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD);
-
-
-/* Start by invoking BIT_initDStream().
-*  A chunk of the bitStream is then stored into a local register.
-*  Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
-*  You can then retrieve bitFields stored into the local register, **in reverse order**.
-*  Local register is explicitly reloaded from memory by the BIT_reloadDStream() method.
-*  A reload guarantee a minimum of ((8*sizeof(bitD->bitContainer))-7) bits when its result is BIT_DStream_unfinished.
-*  Otherwise, it can be less than that, so proceed accordingly.
-*  Checking if DStream has reached its end can be performed with BIT_endOfDStream().
-*/
-
-
-/*-****************************************
-*  unsafe API
-******************************************/
-MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC, size_t value, unsigned nbBits);
-/* faster, but works only if value is "clean", meaning all high bits above nbBits are 0 */
-
-MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC);
-/* unsafe version; does not check buffer overflow */
-
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/*-**************************************************************
-*  Internal functions
-****************************************************************/
-MEM_STATIC unsigned BIT_highbit32 (U32 val)
-{
-    assert(val != 0);
-    {
-#   if defined(_MSC_VER)   /* Visual */
-        unsigned long r=0;
-        _BitScanReverse ( &r, val );
-        return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-        return 31 - __builtin_clz (val);
-#   else   /* Software version */
-        static const unsigned DeBruijnClz[32] = { 0,  9,  1, 10, 13, 21,  2, 29,
-                                                 11, 14, 16, 18, 22, 25,  3, 30,
-                                                  8, 12, 20, 28, 15, 17, 24,  7,
-                                                 19, 27, 23,  6, 26,  5,  4, 31 };
-        U32 v = val;
-        v |= v >> 1;
-        v |= v >> 2;
-        v |= v >> 4;
-        v |= v >> 8;
-        v |= v >> 16;
-        return DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-#   endif
-    }
-}
-
-/*=====    Local Constants   =====*/
-static const unsigned BIT_mask[] = {
-    0,          1,         3,         7,         0xF,       0x1F,
-    0x3F,       0x7F,      0xFF,      0x1FF,     0x3FF,     0x7FF,
-    0xFFF,      0x1FFF,    0x3FFF,    0x7FFF,    0xFFFF,    0x1FFFF,
-    0x3FFFF,    0x7FFFF,   0xFFFFF,   0x1FFFFF,  0x3FFFFF,  0x7FFFFF,
-    0xFFFFFF,   0x1FFFFFF, 0x3FFFFFF, 0x7FFFFFF, 0xFFFFFFF, 0x1FFFFFFF,
-    0x3FFFFFFF, 0x7FFFFFFF}; /* up to 31 bits */
-#define BIT_MASK_SIZE (sizeof(BIT_mask) / sizeof(BIT_mask[0]))
-
-/*-**************************************************************
-*  bitStream encoding
-****************************************************************/
-/*! BIT_initCStream() :
- *  `dstCapacity` must be > sizeof(size_t)
- *  @return : 0 if success,
- *            otherwise an error code (can be tested using ERR_isError()) */
-MEM_STATIC size_t BIT_initCStream(BIT_CStream_t* bitC,
-                                  void* startPtr, size_t dstCapacity)
-{
-    bitC->bitContainer = 0;
-    bitC->bitPos = 0;
-    bitC->startPtr = (char*)startPtr;
-    bitC->ptr = bitC->startPtr;
-    bitC->endPtr = bitC->startPtr + dstCapacity - sizeof(bitC->bitContainer);
-    if (dstCapacity <= sizeof(bitC->bitContainer)) return ERROR(dstSize_tooSmall);
-    return 0;
-}
-
-/*! BIT_addBits() :
- *  can add up to 31 bits into `bitC`.
- *  Note : does not check for register overflow ! */
-MEM_STATIC void BIT_addBits(BIT_CStream_t* bitC,
-                            size_t value, unsigned nbBits)
-{
-    MEM_STATIC_ASSERT(BIT_MASK_SIZE == 32);
-    assert(nbBits < BIT_MASK_SIZE);
-    assert(nbBits + bitC->bitPos < sizeof(bitC->bitContainer) * 8);
-    bitC->bitContainer |= (value & BIT_mask[nbBits]) << bitC->bitPos;
-    bitC->bitPos += nbBits;
-}
-
-/*! BIT_addBitsFast() :
- *  works only if `value` is _clean_,
- *  meaning all high bits above nbBits are 0 */
-MEM_STATIC void BIT_addBitsFast(BIT_CStream_t* bitC,
-                                size_t value, unsigned nbBits)
-{
-    assert((value>>nbBits) == 0);
-    assert(nbBits + bitC->bitPos < sizeof(bitC->bitContainer) * 8);
-    bitC->bitContainer |= value << bitC->bitPos;
-    bitC->bitPos += nbBits;
-}
-
-/*! BIT_flushBitsFast() :
- *  assumption : bitContainer has not overflowed
- *  unsafe version; does not check buffer overflow */
-MEM_STATIC void BIT_flushBitsFast(BIT_CStream_t* bitC)
-{
-    size_t const nbBytes = bitC->bitPos >> 3;
-    assert(bitC->bitPos < sizeof(bitC->bitContainer) * 8);
-    MEM_writeLEST(bitC->ptr, bitC->bitContainer);
-    bitC->ptr += nbBytes;
-    assert(bitC->ptr <= bitC->endPtr);
-    bitC->bitPos &= 7;
-    bitC->bitContainer >>= nbBytes*8;
-}
-
-/*! BIT_flushBits() :
- *  assumption : bitContainer has not overflowed
- *  safe version; check for buffer overflow, and prevents it.
- *  note : does not signal buffer overflow.
- *  overflow will be revealed later on using BIT_closeCStream() */
-MEM_STATIC void BIT_flushBits(BIT_CStream_t* bitC)
-{
-    size_t const nbBytes = bitC->bitPos >> 3;
-    assert(bitC->bitPos < sizeof(bitC->bitContainer) * 8);
-    MEM_writeLEST(bitC->ptr, bitC->bitContainer);
-    bitC->ptr += nbBytes;
-    if (bitC->ptr > bitC->endPtr) bitC->ptr = bitC->endPtr;
-    bitC->bitPos &= 7;
-    bitC->bitContainer >>= nbBytes*8;
-}
-
-/*! BIT_closeCStream() :
- *  @return : size of CStream, in bytes,
- *            or 0 if it could not fit into dstBuffer */
-MEM_STATIC size_t BIT_closeCStream(BIT_CStream_t* bitC)
-{
-    BIT_addBitsFast(bitC, 1, 1);   /* endMark */
-    BIT_flushBits(bitC);
-    if (bitC->ptr >= bitC->endPtr) return 0; /* overflow detected */
-    return (bitC->ptr - bitC->startPtr) + (bitC->bitPos > 0);
-}
-
-
-/*-********************************************************
-*  bitStream decoding
-**********************************************************/
-/*! BIT_initDStream() :
- *  Initialize a BIT_DStream_t.
- * `bitD` : a pointer to an already allocated BIT_DStream_t structure.
- * `srcSize` must be the *exact* size of the bitStream, in bytes.
- * @return : size of stream (== srcSize), or an errorCode if a problem is detected
- */
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    bitD->start = (const char*)srcBuffer;
-    bitD->limitPtr = bitD->start + sizeof(bitD->bitContainer);
-
-    if (srcSize >=  sizeof(bitD->bitContainer)) {  /* normal case */
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-          bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0;  /* ensures bitsConsumed is always set */
-          if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ }
-    } else {
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-        case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16);
-                /* fall-through */
-
-        case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24);
-                /* fall-through */
-
-        case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32);
-                /* fall-through */
-
-        case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24;
-                /* fall-through */
-
-        case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16;
-                /* fall-through */
-
-        case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) <<  8;
-                /* fall-through */
-
-        default: break;
-        }
-        {   BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-            bitD->bitsConsumed = lastByte ? 8 - BIT_highbit32(lastByte) : 0;
-            if (lastByte == 0) return ERROR(corruption_detected);  /* endMark not present */
-        }
-        bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-MEM_STATIC size_t BIT_getUpperBits(size_t bitContainer, U32 const start)
-{
-    return bitContainer >> start;
-}
-
-MEM_STATIC size_t BIT_getMiddleBits(size_t bitContainer, U32 const start, U32 const nbBits)
-{
-    U32 const regMask = sizeof(bitContainer)*8 - 1;
-    /* if start > regMask, bitstream is corrupted, and result is undefined */
-    assert(nbBits < BIT_MASK_SIZE);
-    return (bitContainer >> (start & regMask)) & BIT_mask[nbBits];
-}
-
-MEM_STATIC size_t BIT_getLowerBits(size_t bitContainer, U32 const nbBits)
-{
-    assert(nbBits < BIT_MASK_SIZE);
-    return bitContainer & BIT_mask[nbBits];
-}
-
-/*! BIT_lookBits() :
- *  Provides next n bits from local register.
- *  local register is not modified.
- *  On 32-bits, maxNbBits==24.
- *  On 64-bits, maxNbBits==56.
- * @return : value extracted */
-MEM_STATIC size_t BIT_lookBits(const BIT_DStream_t* bitD, U32 nbBits)
-{
-    /* arbitrate between double-shift and shift+mask */
-#if 1
-    /* if bitD->bitsConsumed + nbBits > sizeof(bitD->bitContainer)*8,
-     * bitstream is likely corrupted, and result is undefined */
-    return BIT_getMiddleBits(bitD->bitContainer, (sizeof(bitD->bitContainer)*8) - bitD->bitsConsumed - nbBits, nbBits);
-#else
-    /* this code path is slower on my os-x laptop */
-    U32 const regMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & regMask)) >> 1) >> ((regMask-nbBits) & regMask);
-#endif
-}
-
-/*! BIT_lookBitsFast() :
- *  unsafe version; only works if nbBits >= 1 */
-MEM_STATIC size_t BIT_lookBitsFast(const BIT_DStream_t* bitD, U32 nbBits)
-{
-    U32 const regMask = sizeof(bitD->bitContainer)*8 - 1;
-    assert(nbBits >= 1);
-    return (bitD->bitContainer << (bitD->bitsConsumed & regMask)) >> (((regMask+1)-nbBits) & regMask);
-}
-
-MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-/*! BIT_readBits() :
- *  Read (consume) next n bits from local register and update.
- *  Pay attention to not read more than nbBits contained into local register.
- * @return : extracted value. */
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits)
-{
-    size_t const value = BIT_lookBits(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*! BIT_readBitsFast() :
- *  unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits)
-{
-    size_t const value = BIT_lookBitsFast(bitD, nbBits);
-    assert(nbBits >= 1);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*! BIT_reloadDStream() :
- *  Refill `bitD` from buffer previously set in BIT_initDStream() .
- *  This function is safe, it guarantees it will not read beyond src buffer.
- * @return : status of `BIT_DStream_t` internal register.
- *           when status == BIT_DStream_unfinished, internal register is filled with at least 25 or 57 bits */
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* overflow detected, like end of stream */
-        return BIT_DStream_overflow;
-
-    if (bitD->ptr >= bitD->limitPtr) {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BIT_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start) {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer;
-        return BIT_DStream_completed;
-    }
-    /* start < ptr < limitPtr */
-    {   U32 nbBytes = bitD->bitsConsumed >> 3;
-        BIT_DStream_status result = BIT_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start) {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BIT_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD->bitContainer), otherwise bitD->ptr == bitD->start */
-        return result;
-    }
-}
-
-/*! BIT_endOfDStream() :
- * @return : 1 if DStream has _exactly_ reached its end (all bits consumed).
- */
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/compiler.h b/vendor/github.com/DataDog/zstd/compiler.h
deleted file mode 100644
index 87bf51a..0000000
--- a/vendor/github.com/DataDog/zstd/compiler.h
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_COMPILER_H
-#define ZSTD_COMPILER_H
-
-/*-*******************************************************
-*  Compiler specifics
-*********************************************************/
-/* force inlining */
-
-#if !defined(ZSTD_NO_INLINE)
-#if defined (__GNUC__) || defined(__cplusplus) || defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#  define INLINE_KEYWORD inline
-#else
-#  define INLINE_KEYWORD
-#endif
-
-#if defined(__GNUC__)
-#  define FORCE_INLINE_ATTR __attribute__((always_inline))
-#elif defined(_MSC_VER)
-#  define FORCE_INLINE_ATTR __forceinline
-#else
-#  define FORCE_INLINE_ATTR
-#endif
-
-#else
-
-#define INLINE_KEYWORD
-#define FORCE_INLINE_ATTR
-
-#endif
-
-/**
- * FORCE_INLINE_TEMPLATE is used to define C "templates", which take constant
- * parameters. They must be inlined for the compiler to eliminate the constant
- * branches.
- */
-#define FORCE_INLINE_TEMPLATE static INLINE_KEYWORD FORCE_INLINE_ATTR
-/**
- * HINT_INLINE is used to help the compiler generate better code. It is *not*
- * used for "templates", so it can be tweaked based on the compilers
- * performance.
- *
- * gcc-4.8 and gcc-4.9 have been shown to benefit from leaving off the
- * always_inline attribute.
- *
- * clang up to 5.0.0 (trunk) benefit tremendously from the always_inline
- * attribute.
- */
-#if !defined(__clang__) && defined(__GNUC__) && __GNUC__ >= 4 && __GNUC_MINOR__ >= 8 && __GNUC__ < 5
-#  define HINT_INLINE static INLINE_KEYWORD
-#else
-#  define HINT_INLINE static INLINE_KEYWORD FORCE_INLINE_ATTR
-#endif
-
-/* force no inlining */
-#ifdef _MSC_VER
-#  define FORCE_NOINLINE static __declspec(noinline)
-#else
-#  ifdef __GNUC__
-#    define FORCE_NOINLINE static __attribute__((__noinline__))
-#  else
-#    define FORCE_NOINLINE static
-#  endif
-#endif
-
-/* target attribute */
-#ifndef __has_attribute
-  #define __has_attribute(x) 0  /* Compatibility with non-clang compilers. */
-#endif
-#if defined(__GNUC__)
-#  define TARGET_ATTRIBUTE(target) __attribute__((__target__(target)))
-#else
-#  define TARGET_ATTRIBUTE(target)
-#endif
-
-/* Enable runtime BMI2 dispatch based on the CPU.
- * Enabled for clang & gcc >=4.8 on x86 when BMI2 isn't enabled by default.
- */
-#ifndef DYNAMIC_BMI2
-  #if ((defined(__clang__) && __has_attribute(__target__)) \
-      || (defined(__GNUC__) \
-          && (__GNUC__ >= 5 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 8)))) \
-      && (defined(__x86_64__) || defined(_M_X86)) \
-      && !defined(__BMI2__)
-  #  define DYNAMIC_BMI2 1
-  #else
-  #  define DYNAMIC_BMI2 0
-  #endif
-#endif
-
-/* prefetch
- * can be disabled, by declaring NO_PREFETCH build macro */
-#if defined(NO_PREFETCH)
-#  define PREFETCH_L1(ptr)  (void)(ptr)  /* disabled */
-#  define PREFETCH_L2(ptr)  (void)(ptr)  /* disabled */
-#else
-#  if defined(_MSC_VER) && (defined(_M_X64) || defined(_M_I86))  /* _mm_prefetch() is not defined outside of x86/x64 */
-#    include <mmintrin.h>   /* https://msdn.microsoft.com/fr-fr/library/84szxsww(v=vs.90).aspx */
-#    define PREFETCH_L1(ptr)  _mm_prefetch((const char*)(ptr), _MM_HINT_T0)
-#    define PREFETCH_L2(ptr)  _mm_prefetch((const char*)(ptr), _MM_HINT_T1)
-#  elif defined(__GNUC__) && ( (__GNUC__ >= 4) || ( (__GNUC__ == 3) && (__GNUC_MINOR__ >= 1) ) )
-#    define PREFETCH_L1(ptr)  __builtin_prefetch((ptr), 0 /* rw==read */, 3 /* locality */)
-#    define PREFETCH_L2(ptr)  __builtin_prefetch((ptr), 0 /* rw==read */, 2 /* locality */)
-#  else
-#    define PREFETCH_L1(ptr) (void)(ptr)  /* disabled */
-#    define PREFETCH_L2(ptr) (void)(ptr)  /* disabled */
-#  endif
-#endif  /* NO_PREFETCH */
-
-#define CACHELINE_SIZE 64
-
-#define PREFETCH_AREA(p, s)  {            \
-    const char* const _ptr = (const char*)(p);  \
-    size_t const _size = (size_t)(s);     \
-    size_t _pos;                          \
-    for (_pos=0; _pos<_size; _pos+=CACHELINE_SIZE) {  \
-        PREFETCH_L2(_ptr + _pos);         \
-    }                                     \
-}
-
-/* vectorization */
-#if !defined(__clang__) && defined(__GNUC__)
-#  define DONT_VECTORIZE __attribute__((optimize("no-tree-vectorize")))
-#else
-#  define DONT_VECTORIZE
-#endif
-
-/* disable warnings */
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4100)        /* disable: C4100: unreferenced formal parameter */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4204)        /* disable: C4204: non-constant aggregate initializer */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-#endif /* ZSTD_COMPILER_H */
diff --git a/vendor/github.com/DataDog/zstd/cover.c b/vendor/github.com/DataDog/zstd/cover.c
deleted file mode 100644
index 6219967..0000000
--- a/vendor/github.com/DataDog/zstd/cover.c
+++ /dev/null
@@ -1,1237 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* *****************************************************************************
- * Constructs a dictionary using a heuristic based on the following paper:
- *
- * Liao, Petri, Moffat, Wirth
- * Effective Construction of Relative Lempel-Ziv Dictionaries
- * Published in WWW 2016.
- *
- * Adapted from code originally written by @ot (Giuseppe Ottaviano).
- ******************************************************************************/
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <stdio.h>  /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h>   /* clock */
-
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "cover.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-/*-*************************************
-*  Constants
-***************************************/
-#define COVER_MAX_SAMPLES_SIZE (sizeof(size_t) == 8 ? ((unsigned)-1) : ((unsigned)1 GB))
-#define DEFAULT_SPLITPOINT 1.0
-
-/*-*************************************
-*  Console display
-***************************************/
-static int g_displayLevel = 2;
-#define DISPLAY(...)                                                           \
-  {                                                                            \
-    fprintf(stderr, __VA_ARGS__);                                              \
-    fflush(stderr);                                                            \
-  }
-#define LOCALDISPLAYLEVEL(displayLevel, l, ...)                                \
-  if (displayLevel >= l) {                                                     \
-    DISPLAY(__VA_ARGS__);                                                      \
-  } /* 0 : no display;   1: errors;   2: default;  3: details;  4: debug */
-#define DISPLAYLEVEL(l, ...) LOCALDISPLAYLEVEL(g_displayLevel, l, __VA_ARGS__)
-
-#define LOCALDISPLAYUPDATE(displayLevel, l, ...)                               \
-  if (displayLevel >= l) {                                                     \
-    if ((clock() - g_time > refreshRate) || (displayLevel >= 4)) {             \
-      g_time = clock();                                                        \
-      DISPLAY(__VA_ARGS__);                                                    \
-    }                                                                          \
-  }
-#define DISPLAYUPDATE(l, ...) LOCALDISPLAYUPDATE(g_displayLevel, l, __VA_ARGS__)
-static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100;
-static clock_t g_time = 0;
-
-/*-*************************************
-* Hash table
-***************************************
-* A small specialized hash map for storing activeDmers.
-* The map does not resize, so if it becomes full it will loop forever.
-* Thus, the map must be large enough to store every value.
-* The map implements linear probing and keeps its load less than 0.5.
-*/
-
-#define MAP_EMPTY_VALUE ((U32)-1)
-typedef struct COVER_map_pair_t_s {
-  U32 key;
-  U32 value;
-} COVER_map_pair_t;
-
-typedef struct COVER_map_s {
-  COVER_map_pair_t *data;
-  U32 sizeLog;
-  U32 size;
-  U32 sizeMask;
-} COVER_map_t;
-
-/**
- * Clear the map.
- */
-static void COVER_map_clear(COVER_map_t *map) {
-  memset(map->data, MAP_EMPTY_VALUE, map->size * sizeof(COVER_map_pair_t));
-}
-
-/**
- * Initializes a map of the given size.
- * Returns 1 on success and 0 on failure.
- * The map must be destroyed with COVER_map_destroy().
- * The map is only guaranteed to be large enough to hold size elements.
- */
-static int COVER_map_init(COVER_map_t *map, U32 size) {
-  map->sizeLog = ZSTD_highbit32(size) + 2;
-  map->size = (U32)1 << map->sizeLog;
-  map->sizeMask = map->size - 1;
-  map->data = (COVER_map_pair_t *)malloc(map->size * sizeof(COVER_map_pair_t));
-  if (!map->data) {
-    map->sizeLog = 0;
-    map->size = 0;
-    return 0;
-  }
-  COVER_map_clear(map);
-  return 1;
-}
-
-/**
- * Internal hash function
- */
-static const U32 prime4bytes = 2654435761U;
-static U32 COVER_map_hash(COVER_map_t *map, U32 key) {
-  return (key * prime4bytes) >> (32 - map->sizeLog);
-}
-
-/**
- * Helper function that returns the index that a key should be placed into.
- */
-static U32 COVER_map_index(COVER_map_t *map, U32 key) {
-  const U32 hash = COVER_map_hash(map, key);
-  U32 i;
-  for (i = hash;; i = (i + 1) & map->sizeMask) {
-    COVER_map_pair_t *pos = &map->data[i];
-    if (pos->value == MAP_EMPTY_VALUE) {
-      return i;
-    }
-    if (pos->key == key) {
-      return i;
-    }
-  }
-}
-
-/**
- * Returns the pointer to the value for key.
- * If key is not in the map, it is inserted and the value is set to 0.
- * The map must not be full.
- */
-static U32 *COVER_map_at(COVER_map_t *map, U32 key) {
-  COVER_map_pair_t *pos = &map->data[COVER_map_index(map, key)];
-  if (pos->value == MAP_EMPTY_VALUE) {
-    pos->key = key;
-    pos->value = 0;
-  }
-  return &pos->value;
-}
-
-/**
- * Deletes key from the map if present.
- */
-static void COVER_map_remove(COVER_map_t *map, U32 key) {
-  U32 i = COVER_map_index(map, key);
-  COVER_map_pair_t *del = &map->data[i];
-  U32 shift = 1;
-  if (del->value == MAP_EMPTY_VALUE) {
-    return;
-  }
-  for (i = (i + 1) & map->sizeMask;; i = (i + 1) & map->sizeMask) {
-    COVER_map_pair_t *const pos = &map->data[i];
-    /* If the position is empty we are done */
-    if (pos->value == MAP_EMPTY_VALUE) {
-      del->value = MAP_EMPTY_VALUE;
-      return;
-    }
-    /* If pos can be moved to del do so */
-    if (((i - COVER_map_hash(map, pos->key)) & map->sizeMask) >= shift) {
-      del->key = pos->key;
-      del->value = pos->value;
-      del = pos;
-      shift = 1;
-    } else {
-      ++shift;
-    }
-  }
-}
-
-/**
- * Destroys a map that is inited with COVER_map_init().
- */
-static void COVER_map_destroy(COVER_map_t *map) {
-  if (map->data) {
-    free(map->data);
-  }
-  map->data = NULL;
-  map->size = 0;
-}
-
-/*-*************************************
-* Context
-***************************************/
-
-typedef struct {
-  const BYTE *samples;
-  size_t *offsets;
-  const size_t *samplesSizes;
-  size_t nbSamples;
-  size_t nbTrainSamples;
-  size_t nbTestSamples;
-  U32 *suffix;
-  size_t suffixSize;
-  U32 *freqs;
-  U32 *dmerAt;
-  unsigned d;
-} COVER_ctx_t;
-
-/* We need a global context for qsort... */
-static COVER_ctx_t *g_ctx = NULL;
-
-/*-*************************************
-*  Helper functions
-***************************************/
-
-/**
- * Returns the sum of the sample sizes.
- */
-size_t COVER_sum(const size_t *samplesSizes, unsigned nbSamples) {
-  size_t sum = 0;
-  unsigned i;
-  for (i = 0; i < nbSamples; ++i) {
-    sum += samplesSizes[i];
-  }
-  return sum;
-}
-
-/**
- * Returns -1 if the dmer at lp is less than the dmer at rp.
- * Return 0 if the dmers at lp and rp are equal.
- * Returns 1 if the dmer at lp is greater than the dmer at rp.
- */
-static int COVER_cmp(COVER_ctx_t *ctx, const void *lp, const void *rp) {
-  U32 const lhs = *(U32 const *)lp;
-  U32 const rhs = *(U32 const *)rp;
-  return memcmp(ctx->samples + lhs, ctx->samples + rhs, ctx->d);
-}
-/**
- * Faster version for d <= 8.
- */
-static int COVER_cmp8(COVER_ctx_t *ctx, const void *lp, const void *rp) {
-  U64 const mask = (ctx->d == 8) ? (U64)-1 : (((U64)1 << (8 * ctx->d)) - 1);
-  U64 const lhs = MEM_readLE64(ctx->samples + *(U32 const *)lp) & mask;
-  U64 const rhs = MEM_readLE64(ctx->samples + *(U32 const *)rp) & mask;
-  if (lhs < rhs) {
-    return -1;
-  }
-  return (lhs > rhs);
-}
-
-/**
- * Same as COVER_cmp() except ties are broken by pointer value
- * NOTE: g_ctx must be set to call this function.  A global is required because
- * qsort doesn't take an opaque pointer.
- */
-static int COVER_strict_cmp(const void *lp, const void *rp) {
-  int result = COVER_cmp(g_ctx, lp, rp);
-  if (result == 0) {
-    result = lp < rp ? -1 : 1;
-  }
-  return result;
-}
-/**
- * Faster version for d <= 8.
- */
-static int COVER_strict_cmp8(const void *lp, const void *rp) {
-  int result = COVER_cmp8(g_ctx, lp, rp);
-  if (result == 0) {
-    result = lp < rp ? -1 : 1;
-  }
-  return result;
-}
-
-/**
- * Returns the first pointer in [first, last) whose element does not compare
- * less than value.  If no such element exists it returns last.
- */
-static const size_t *COVER_lower_bound(const size_t *first, const size_t *last,
-                                       size_t value) {
-  size_t count = last - first;
-  while (count != 0) {
-    size_t step = count / 2;
-    const size_t *ptr = first;
-    ptr += step;
-    if (*ptr < value) {
-      first = ++ptr;
-      count -= step + 1;
-    } else {
-      count = step;
-    }
-  }
-  return first;
-}
-
-/**
- * Generic groupBy function.
- * Groups an array sorted by cmp into groups with equivalent values.
- * Calls grp for each group.
- */
-static void
-COVER_groupBy(const void *data, size_t count, size_t size, COVER_ctx_t *ctx,
-              int (*cmp)(COVER_ctx_t *, const void *, const void *),
-              void (*grp)(COVER_ctx_t *, const void *, const void *)) {
-  const BYTE *ptr = (const BYTE *)data;
-  size_t num = 0;
-  while (num < count) {
-    const BYTE *grpEnd = ptr + size;
-    ++num;
-    while (num < count && cmp(ctx, ptr, grpEnd) == 0) {
-      grpEnd += size;
-      ++num;
-    }
-    grp(ctx, ptr, grpEnd);
-    ptr = grpEnd;
-  }
-}
-
-/*-*************************************
-*  Cover functions
-***************************************/
-
-/**
- * Called on each group of positions with the same dmer.
- * Counts the frequency of each dmer and saves it in the suffix array.
- * Fills `ctx->dmerAt`.
- */
-static void COVER_group(COVER_ctx_t *ctx, const void *group,
-                        const void *groupEnd) {
-  /* The group consists of all the positions with the same first d bytes. */
-  const U32 *grpPtr = (const U32 *)group;
-  const U32 *grpEnd = (const U32 *)groupEnd;
-  /* The dmerId is how we will reference this dmer.
-   * This allows us to map the whole dmer space to a much smaller space, the
-   * size of the suffix array.
-   */
-  const U32 dmerId = (U32)(grpPtr - ctx->suffix);
-  /* Count the number of samples this dmer shows up in */
-  U32 freq = 0;
-  /* Details */
-  const size_t *curOffsetPtr = ctx->offsets;
-  const size_t *offsetsEnd = ctx->offsets + ctx->nbSamples;
-  /* Once *grpPtr >= curSampleEnd this occurrence of the dmer is in a
-   * different sample than the last.
-   */
-  size_t curSampleEnd = ctx->offsets[0];
-  for (; grpPtr != grpEnd; ++grpPtr) {
-    /* Save the dmerId for this position so we can get back to it. */
-    ctx->dmerAt[*grpPtr] = dmerId;
-    /* Dictionaries only help for the first reference to the dmer.
-     * After that zstd can reference the match from the previous reference.
-     * So only count each dmer once for each sample it is in.
-     */
-    if (*grpPtr < curSampleEnd) {
-      continue;
-    }
-    freq += 1;
-    /* Binary search to find the end of the sample *grpPtr is in.
-     * In the common case that grpPtr + 1 == grpEnd we can skip the binary
-     * search because the loop is over.
-     */
-    if (grpPtr + 1 != grpEnd) {
-      const size_t *sampleEndPtr =
-          COVER_lower_bound(curOffsetPtr, offsetsEnd, *grpPtr);
-      curSampleEnd = *sampleEndPtr;
-      curOffsetPtr = sampleEndPtr + 1;
-    }
-  }
-  /* At this point we are never going to look at this segment of the suffix
-   * array again.  We take advantage of this fact to save memory.
-   * We store the frequency of the dmer in the first position of the group,
-   * which is dmerId.
-   */
-  ctx->suffix[dmerId] = freq;
-}
-
-
-/**
- * Selects the best segment in an epoch.
- * Segments of are scored according to the function:
- *
- * Let F(d) be the frequency of dmer d.
- * Let S_i be the dmer at position i of segment S which has length k.
- *
- *     Score(S) = F(S_1) + F(S_2) + ... + F(S_{k-d+1})
- *
- * Once the dmer d is in the dictionary we set F(d) = 0.
- */
-static COVER_segment_t COVER_selectSegment(const COVER_ctx_t *ctx, U32 *freqs,
-                                           COVER_map_t *activeDmers, U32 begin,
-                                           U32 end,
-                                           ZDICT_cover_params_t parameters) {
-  /* Constants */
-  const U32 k = parameters.k;
-  const U32 d = parameters.d;
-  const U32 dmersInK = k - d + 1;
-  /* Try each segment (activeSegment) and save the best (bestSegment) */
-  COVER_segment_t bestSegment = {0, 0, 0};
-  COVER_segment_t activeSegment;
-  /* Reset the activeDmers in the segment */
-  COVER_map_clear(activeDmers);
-  /* The activeSegment starts at the beginning of the epoch. */
-  activeSegment.begin = begin;
-  activeSegment.end = begin;
-  activeSegment.score = 0;
-  /* Slide the activeSegment through the whole epoch.
-   * Save the best segment in bestSegment.
-   */
-  while (activeSegment.end < end) {
-    /* The dmerId for the dmer at the next position */
-    U32 newDmer = ctx->dmerAt[activeSegment.end];
-    /* The entry in activeDmers for this dmerId */
-    U32 *newDmerOcc = COVER_map_at(activeDmers, newDmer);
-    /* If the dmer isn't already present in the segment add its score. */
-    if (*newDmerOcc == 0) {
-      /* The paper suggest using the L-0.5 norm, but experiments show that it
-       * doesn't help.
-       */
-      activeSegment.score += freqs[newDmer];
-    }
-    /* Add the dmer to the segment */
-    activeSegment.end += 1;
-    *newDmerOcc += 1;
-
-    /* If the window is now too large, drop the first position */
-    if (activeSegment.end - activeSegment.begin == dmersInK + 1) {
-      U32 delDmer = ctx->dmerAt[activeSegment.begin];
-      U32 *delDmerOcc = COVER_map_at(activeDmers, delDmer);
-      activeSegment.begin += 1;
-      *delDmerOcc -= 1;
-      /* If this is the last occurrence of the dmer, subtract its score */
-      if (*delDmerOcc == 0) {
-        COVER_map_remove(activeDmers, delDmer);
-        activeSegment.score -= freqs[delDmer];
-      }
-    }
-
-    /* If this segment is the best so far save it */
-    if (activeSegment.score > bestSegment.score) {
-      bestSegment = activeSegment;
-    }
-  }
-  {
-    /* Trim off the zero frequency head and tail from the segment. */
-    U32 newBegin = bestSegment.end;
-    U32 newEnd = bestSegment.begin;
-    U32 pos;
-    for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
-      U32 freq = freqs[ctx->dmerAt[pos]];
-      if (freq != 0) {
-        newBegin = MIN(newBegin, pos);
-        newEnd = pos + 1;
-      }
-    }
-    bestSegment.begin = newBegin;
-    bestSegment.end = newEnd;
-  }
-  {
-    /* Zero out the frequency of each dmer covered by the chosen segment. */
-    U32 pos;
-    for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
-      freqs[ctx->dmerAt[pos]] = 0;
-    }
-  }
-  return bestSegment;
-}
-
-/**
- * Check the validity of the parameters.
- * Returns non-zero if the parameters are valid and 0 otherwise.
- */
-static int COVER_checkParameters(ZDICT_cover_params_t parameters,
-                                 size_t maxDictSize) {
-  /* k and d are required parameters */
-  if (parameters.d == 0 || parameters.k == 0) {
-    return 0;
-  }
-  /* k <= maxDictSize */
-  if (parameters.k > maxDictSize) {
-    return 0;
-  }
-  /* d <= k */
-  if (parameters.d > parameters.k) {
-    return 0;
-  }
-  /* 0 < splitPoint <= 1 */
-  if (parameters.splitPoint <= 0 || parameters.splitPoint > 1){
-    return 0;
-  }
-  return 1;
-}
-
-/**
- * Clean up a context initialized with `COVER_ctx_init()`.
- */
-static void COVER_ctx_destroy(COVER_ctx_t *ctx) {
-  if (!ctx) {
-    return;
-  }
-  if (ctx->suffix) {
-    free(ctx->suffix);
-    ctx->suffix = NULL;
-  }
-  if (ctx->freqs) {
-    free(ctx->freqs);
-    ctx->freqs = NULL;
-  }
-  if (ctx->dmerAt) {
-    free(ctx->dmerAt);
-    ctx->dmerAt = NULL;
-  }
-  if (ctx->offsets) {
-    free(ctx->offsets);
-    ctx->offsets = NULL;
-  }
-}
-
-/**
- * Prepare a context for dictionary building.
- * The context is only dependent on the parameter `d` and can used multiple
- * times.
- * Returns 0 on success or error code on error.
- * The context must be destroyed with `COVER_ctx_destroy()`.
- */
-static size_t COVER_ctx_init(COVER_ctx_t *ctx, const void *samplesBuffer,
-                          const size_t *samplesSizes, unsigned nbSamples,
-                          unsigned d, double splitPoint) {
-  const BYTE *const samples = (const BYTE *)samplesBuffer;
-  const size_t totalSamplesSize = COVER_sum(samplesSizes, nbSamples);
-  /* Split samples into testing and training sets */
-  const unsigned nbTrainSamples = splitPoint < 1.0 ? (unsigned)((double)nbSamples * splitPoint) : nbSamples;
-  const unsigned nbTestSamples = splitPoint < 1.0 ? nbSamples - nbTrainSamples : nbSamples;
-  const size_t trainingSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes, nbTrainSamples) : totalSamplesSize;
-  const size_t testSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes + nbTrainSamples, nbTestSamples) : totalSamplesSize;
-  /* Checks */
-  if (totalSamplesSize < MAX(d, sizeof(U64)) ||
-      totalSamplesSize >= (size_t)COVER_MAX_SAMPLES_SIZE) {
-    DISPLAYLEVEL(1, "Total samples size is too large (%u MB), maximum size is %u MB\n",
-                 (unsigned)(totalSamplesSize>>20), (COVER_MAX_SAMPLES_SIZE >> 20));
-    return ERROR(srcSize_wrong);
-  }
-  /* Check if there are at least 5 training samples */
-  if (nbTrainSamples < 5) {
-    DISPLAYLEVEL(1, "Total number of training samples is %u and is invalid.", nbTrainSamples);
-    return ERROR(srcSize_wrong);
-  }
-  /* Check if there's testing sample */
-  if (nbTestSamples < 1) {
-    DISPLAYLEVEL(1, "Total number of testing samples is %u and is invalid.", nbTestSamples);
-    return ERROR(srcSize_wrong);
-  }
-  /* Zero the context */
-  memset(ctx, 0, sizeof(*ctx));
-  DISPLAYLEVEL(2, "Training on %u samples of total size %u\n", nbTrainSamples,
-               (unsigned)trainingSamplesSize);
-  DISPLAYLEVEL(2, "Testing on %u samples of total size %u\n", nbTestSamples,
-               (unsigned)testSamplesSize);
-  ctx->samples = samples;
-  ctx->samplesSizes = samplesSizes;
-  ctx->nbSamples = nbSamples;
-  ctx->nbTrainSamples = nbTrainSamples;
-  ctx->nbTestSamples = nbTestSamples;
-  /* Partial suffix array */
-  ctx->suffixSize = trainingSamplesSize - MAX(d, sizeof(U64)) + 1;
-  ctx->suffix = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
-  /* Maps index to the dmerID */
-  ctx->dmerAt = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
-  /* The offsets of each file */
-  ctx->offsets = (size_t *)malloc((nbSamples + 1) * sizeof(size_t));
-  if (!ctx->suffix || !ctx->dmerAt || !ctx->offsets) {
-    DISPLAYLEVEL(1, "Failed to allocate scratch buffers\n");
-    COVER_ctx_destroy(ctx);
-    return ERROR(memory_allocation);
-  }
-  ctx->freqs = NULL;
-  ctx->d = d;
-
-  /* Fill offsets from the samplesSizes */
-  {
-    U32 i;
-    ctx->offsets[0] = 0;
-    for (i = 1; i <= nbSamples; ++i) {
-      ctx->offsets[i] = ctx->offsets[i - 1] + samplesSizes[i - 1];
-    }
-  }
-  DISPLAYLEVEL(2, "Constructing partial suffix array\n");
-  {
-    /* suffix is a partial suffix array.
-     * It only sorts suffixes by their first parameters.d bytes.
-     * The sort is stable, so each dmer group is sorted by position in input.
-     */
-    U32 i;
-    for (i = 0; i < ctx->suffixSize; ++i) {
-      ctx->suffix[i] = i;
-    }
-    /* qsort doesn't take an opaque pointer, so pass as a global.
-     * On OpenBSD qsort() is not guaranteed to be stable, their mergesort() is.
-     */
-    g_ctx = ctx;
-#if defined(__OpenBSD__)
-    mergesort(ctx->suffix, ctx->suffixSize, sizeof(U32),
-          (ctx->d <= 8 ? &COVER_strict_cmp8 : &COVER_strict_cmp));
-#else
-    qsort(ctx->suffix, ctx->suffixSize, sizeof(U32),
-          (ctx->d <= 8 ? &COVER_strict_cmp8 : &COVER_strict_cmp));
-#endif
-  }
-  DISPLAYLEVEL(2, "Computing frequencies\n");
-  /* For each dmer group (group of positions with the same first d bytes):
-   * 1. For each position we set dmerAt[position] = dmerID.  The dmerID is
-   *    (groupBeginPtr - suffix).  This allows us to go from position to
-   *    dmerID so we can look up values in freq.
-   * 2. We calculate how many samples the dmer occurs in and save it in
-   *    freqs[dmerId].
-   */
-  COVER_groupBy(ctx->suffix, ctx->suffixSize, sizeof(U32), ctx,
-                (ctx->d <= 8 ? &COVER_cmp8 : &COVER_cmp), &COVER_group);
-  ctx->freqs = ctx->suffix;
-  ctx->suffix = NULL;
-  return 0;
-}
-
-void COVER_warnOnSmallCorpus(size_t maxDictSize, size_t nbDmers, int displayLevel)
-{
-  const double ratio = (double)nbDmers / maxDictSize;
-  if (ratio >= 10) {
-      return;
-  }
-  LOCALDISPLAYLEVEL(displayLevel, 1,
-                    "WARNING: The maximum dictionary size %u is too large "
-                    "compared to the source size %u! "
-                    "size(source)/size(dictionary) = %f, but it should be >= "
-                    "10! This may lead to a subpar dictionary! We recommend "
-                    "training on sources at least 10x, and up to 100x the "
-                    "size of the dictionary!\n", (U32)maxDictSize,
-                    (U32)nbDmers, ratio);
-}
-
-COVER_epoch_info_t COVER_computeEpochs(U32 maxDictSize,
-                                       U32 nbDmers, U32 k, U32 passes)
-{
-  const U32 minEpochSize = k * 10;
-  COVER_epoch_info_t epochs;
-  epochs.num = MAX(1, maxDictSize / k / passes);
-  epochs.size = nbDmers / epochs.num;
-  if (epochs.size >= minEpochSize) {
-      assert(epochs.size * epochs.num <= nbDmers);
-      return epochs;
-  }
-  epochs.size = MIN(minEpochSize, nbDmers);
-  epochs.num = nbDmers / epochs.size;
-  assert(epochs.size * epochs.num <= nbDmers);
-  return epochs;
-}
-
-/**
- * Given the prepared context build the dictionary.
- */
-static size_t COVER_buildDictionary(const COVER_ctx_t *ctx, U32 *freqs,
-                                    COVER_map_t *activeDmers, void *dictBuffer,
-                                    size_t dictBufferCapacity,
-                                    ZDICT_cover_params_t parameters) {
-  BYTE *const dict = (BYTE *)dictBuffer;
-  size_t tail = dictBufferCapacity;
-  /* Divide the data into epochs. We will select one segment from each epoch. */
-  const COVER_epoch_info_t epochs = COVER_computeEpochs(
-      (U32)dictBufferCapacity, (U32)ctx->suffixSize, parameters.k, 4);
-  const size_t maxZeroScoreRun = MAX(10, MIN(100, epochs.num >> 3));
-  size_t zeroScoreRun = 0;
-  size_t epoch;
-  DISPLAYLEVEL(2, "Breaking content into %u epochs of size %u\n",
-                (U32)epochs.num, (U32)epochs.size);
-  /* Loop through the epochs until there are no more segments or the dictionary
-   * is full.
-   */
-  for (epoch = 0; tail > 0; epoch = (epoch + 1) % epochs.num) {
-    const U32 epochBegin = (U32)(epoch * epochs.size);
-    const U32 epochEnd = epochBegin + epochs.size;
-    size_t segmentSize;
-    /* Select a segment */
-    COVER_segment_t segment = COVER_selectSegment(
-        ctx, freqs, activeDmers, epochBegin, epochEnd, parameters);
-    /* If the segment covers no dmers, then we are out of content.
-     * There may be new content in other epochs, for continue for some time.
-     */
-    if (segment.score == 0) {
-      if (++zeroScoreRun >= maxZeroScoreRun) {
-          break;
-      }
-      continue;
-    }
-    zeroScoreRun = 0;
-    /* Trim the segment if necessary and if it is too small then we are done */
-    segmentSize = MIN(segment.end - segment.begin + parameters.d - 1, tail);
-    if (segmentSize < parameters.d) {
-      break;
-    }
-    /* We fill the dictionary from the back to allow the best segments to be
-     * referenced with the smallest offsets.
-     */
-    tail -= segmentSize;
-    memcpy(dict + tail, ctx->samples + segment.begin, segmentSize);
-    DISPLAYUPDATE(
-        2, "\r%u%%       ",
-        (unsigned)(((dictBufferCapacity - tail) * 100) / dictBufferCapacity));
-  }
-  DISPLAYLEVEL(2, "\r%79s\r", "");
-  return tail;
-}
-
-ZDICTLIB_API size_t ZDICT_trainFromBuffer_cover(
-    void *dictBuffer, size_t dictBufferCapacity,
-    const void *samplesBuffer, const size_t *samplesSizes, unsigned nbSamples,
-    ZDICT_cover_params_t parameters)
-{
-  BYTE* const dict = (BYTE*)dictBuffer;
-  COVER_ctx_t ctx;
-  COVER_map_t activeDmers;
-  parameters.splitPoint = 1.0;
-  /* Initialize global data */
-  g_displayLevel = parameters.zParams.notificationLevel;
-  /* Checks */
-  if (!COVER_checkParameters(parameters, dictBufferCapacity)) {
-    DISPLAYLEVEL(1, "Cover parameters incorrect\n");
-    return ERROR(parameter_outOfBound);
-  }
-  if (nbSamples == 0) {
-    DISPLAYLEVEL(1, "Cover must have at least one input file\n");
-    return ERROR(srcSize_wrong);
-  }
-  if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
-    DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
-                 ZDICT_DICTSIZE_MIN);
-    return ERROR(dstSize_tooSmall);
-  }
-  /* Initialize context and activeDmers */
-  {
-    size_t const initVal = COVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples,
-                      parameters.d, parameters.splitPoint);
-    if (ZSTD_isError(initVal)) {
-      return initVal;
-    }
-  }
-  COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.suffixSize, g_displayLevel);
-  if (!COVER_map_init(&activeDmers, parameters.k - parameters.d + 1)) {
-    DISPLAYLEVEL(1, "Failed to allocate dmer map: out of memory\n");
-    COVER_ctx_destroy(&ctx);
-    return ERROR(memory_allocation);
-  }
-
-  DISPLAYLEVEL(2, "Building dictionary\n");
-  {
-    const size_t tail =
-        COVER_buildDictionary(&ctx, ctx.freqs, &activeDmers, dictBuffer,
-                              dictBufferCapacity, parameters);
-    const size_t dictionarySize = ZDICT_finalizeDictionary(
-        dict, dictBufferCapacity, dict + tail, dictBufferCapacity - tail,
-        samplesBuffer, samplesSizes, nbSamples, parameters.zParams);
-    if (!ZSTD_isError(dictionarySize)) {
-      DISPLAYLEVEL(2, "Constructed dictionary of size %u\n",
-                   (unsigned)dictionarySize);
-    }
-    COVER_ctx_destroy(&ctx);
-    COVER_map_destroy(&activeDmers);
-    return dictionarySize;
-  }
-}
-
-
-
-size_t COVER_checkTotalCompressedSize(const ZDICT_cover_params_t parameters,
-                                    const size_t *samplesSizes, const BYTE *samples,
-                                    size_t *offsets,
-                                    size_t nbTrainSamples, size_t nbSamples,
-                                    BYTE *const dict, size_t dictBufferCapacity) {
-  size_t totalCompressedSize = ERROR(GENERIC);
-  /* Pointers */
-  ZSTD_CCtx *cctx;
-  ZSTD_CDict *cdict;
-  void *dst;
-  /* Local variables */
-  size_t dstCapacity;
-  size_t i;
-  /* Allocate dst with enough space to compress the maximum sized sample */
-  {
-    size_t maxSampleSize = 0;
-    i = parameters.splitPoint < 1.0 ? nbTrainSamples : 0;
-    for (; i < nbSamples; ++i) {
-      maxSampleSize = MAX(samplesSizes[i], maxSampleSize);
-    }
-    dstCapacity = ZSTD_compressBound(maxSampleSize);
-    dst = malloc(dstCapacity);
-  }
-  /* Create the cctx and cdict */
-  cctx = ZSTD_createCCtx();
-  cdict = ZSTD_createCDict(dict, dictBufferCapacity,
-                           parameters.zParams.compressionLevel);
-  if (!dst || !cctx || !cdict) {
-    goto _compressCleanup;
-  }
-  /* Compress each sample and sum their sizes (or error) */
-  totalCompressedSize = dictBufferCapacity;
-  i = parameters.splitPoint < 1.0 ? nbTrainSamples : 0;
-  for (; i < nbSamples; ++i) {
-    const size_t size = ZSTD_compress_usingCDict(
-        cctx, dst, dstCapacity, samples + offsets[i],
-        samplesSizes[i], cdict);
-    if (ZSTD_isError(size)) {
-      totalCompressedSize = size;
-      goto _compressCleanup;
-    }
-    totalCompressedSize += size;
-  }
-_compressCleanup:
-  ZSTD_freeCCtx(cctx);
-  ZSTD_freeCDict(cdict);
-  if (dst) {
-    free(dst);
-  }
-  return totalCompressedSize;
-}
-
-
-/**
- * Initialize the `COVER_best_t`.
- */
-void COVER_best_init(COVER_best_t *best) {
-  if (best==NULL) return; /* compatible with init on NULL */
-  (void)ZSTD_pthread_mutex_init(&best->mutex, NULL);
-  (void)ZSTD_pthread_cond_init(&best->cond, NULL);
-  best->liveJobs = 0;
-  best->dict = NULL;
-  best->dictSize = 0;
-  best->compressedSize = (size_t)-1;
-  memset(&best->parameters, 0, sizeof(best->parameters));
-}
-
-/**
- * Wait until liveJobs == 0.
- */
-void COVER_best_wait(COVER_best_t *best) {
-  if (!best) {
-    return;
-  }
-  ZSTD_pthread_mutex_lock(&best->mutex);
-  while (best->liveJobs != 0) {
-    ZSTD_pthread_cond_wait(&best->cond, &best->mutex);
-  }
-  ZSTD_pthread_mutex_unlock(&best->mutex);
-}
-
-/**
- * Call COVER_best_wait() and then destroy the COVER_best_t.
- */
-void COVER_best_destroy(COVER_best_t *best) {
-  if (!best) {
-    return;
-  }
-  COVER_best_wait(best);
-  if (best->dict) {
-    free(best->dict);
-  }
-  ZSTD_pthread_mutex_destroy(&best->mutex);
-  ZSTD_pthread_cond_destroy(&best->cond);
-}
-
-/**
- * Called when a thread is about to be launched.
- * Increments liveJobs.
- */
-void COVER_best_start(COVER_best_t *best) {
-  if (!best) {
-    return;
-  }
-  ZSTD_pthread_mutex_lock(&best->mutex);
-  ++best->liveJobs;
-  ZSTD_pthread_mutex_unlock(&best->mutex);
-}
-
-/**
- * Called when a thread finishes executing, both on error or success.
- * Decrements liveJobs and signals any waiting threads if liveJobs == 0.
- * If this dictionary is the best so far save it and its parameters.
- */
-void COVER_best_finish(COVER_best_t *best, ZDICT_cover_params_t parameters,
-                              COVER_dictSelection_t selection) {
-  void* dict = selection.dictContent;
-  size_t compressedSize = selection.totalCompressedSize;
-  size_t dictSize = selection.dictSize;
-  if (!best) {
-    return;
-  }
-  {
-    size_t liveJobs;
-    ZSTD_pthread_mutex_lock(&best->mutex);
-    --best->liveJobs;
-    liveJobs = best->liveJobs;
-    /* If the new dictionary is better */
-    if (compressedSize < best->compressedSize) {
-      /* Allocate space if necessary */
-      if (!best->dict || best->dictSize < dictSize) {
-        if (best->dict) {
-          free(best->dict);
-        }
-        best->dict = malloc(dictSize);
-        if (!best->dict) {
-          best->compressedSize = ERROR(GENERIC);
-          best->dictSize = 0;
-          ZSTD_pthread_cond_signal(&best->cond);
-          ZSTD_pthread_mutex_unlock(&best->mutex);
-          return;
-        }
-      }
-      /* Save the dictionary, parameters, and size */
-      if (!dict) {
-        return;
-      }
-      memcpy(best->dict, dict, dictSize);
-      best->dictSize = dictSize;
-      best->parameters = parameters;
-      best->compressedSize = compressedSize;
-    }
-    if (liveJobs == 0) {
-      ZSTD_pthread_cond_broadcast(&best->cond);
-    }
-    ZSTD_pthread_mutex_unlock(&best->mutex);
-  }
-}
-
-COVER_dictSelection_t COVER_dictSelectionError(size_t error) {
-    COVER_dictSelection_t selection = { NULL, 0, error };
-    return selection;
-}
-
-unsigned COVER_dictSelectionIsError(COVER_dictSelection_t selection) {
-  return (ZSTD_isError(selection.totalCompressedSize) || !selection.dictContent);
-}
-
-void COVER_dictSelectionFree(COVER_dictSelection_t selection){
-  free(selection.dictContent);
-}
-
-COVER_dictSelection_t COVER_selectDict(BYTE* customDictContent,
-        size_t dictContentSize, const BYTE* samplesBuffer, const size_t* samplesSizes, unsigned nbFinalizeSamples,
-        size_t nbCheckSamples, size_t nbSamples, ZDICT_cover_params_t params, size_t* offsets, size_t totalCompressedSize) {
-
-  size_t largestDict = 0;
-  size_t largestCompressed = 0;
-  BYTE* customDictContentEnd = customDictContent + dictContentSize;
-
-  BYTE * largestDictbuffer = (BYTE *)malloc(dictContentSize);
-  BYTE * candidateDictBuffer = (BYTE *)malloc(dictContentSize);
-  double regressionTolerance = ((double)params.shrinkDictMaxRegression / 100.0) + 1.00;
-
-  if (!largestDictbuffer || !candidateDictBuffer) {
-    free(largestDictbuffer);
-    free(candidateDictBuffer);
-    return COVER_dictSelectionError(dictContentSize);
-  }
-
-  /* Initial dictionary size and compressed size */
-  memcpy(largestDictbuffer, customDictContent, dictContentSize);
-  dictContentSize = ZDICT_finalizeDictionary(
-    largestDictbuffer, dictContentSize, customDictContent, dictContentSize,
-    samplesBuffer, samplesSizes, nbFinalizeSamples, params.zParams);
-
-  if (ZDICT_isError(dictContentSize)) {
-    free(largestDictbuffer);
-    free(candidateDictBuffer);
-    return COVER_dictSelectionError(dictContentSize);
-  }
-
-  totalCompressedSize = COVER_checkTotalCompressedSize(params, samplesSizes,
-                                                       samplesBuffer, offsets,
-                                                       nbCheckSamples, nbSamples,
-                                                       largestDictbuffer, dictContentSize);
-
-  if (ZSTD_isError(totalCompressedSize)) {
-    free(largestDictbuffer);
-    free(candidateDictBuffer);
-    return COVER_dictSelectionError(totalCompressedSize);
-  }
-
-  if (params.shrinkDict == 0) {
-    COVER_dictSelection_t selection = { largestDictbuffer, dictContentSize, totalCompressedSize };
-    free(candidateDictBuffer);
-    return selection;
-  }
-
-  largestDict = dictContentSize;
-  largestCompressed = totalCompressedSize;
-  dictContentSize = ZDICT_DICTSIZE_MIN;
-
-  /* Largest dict is initially at least ZDICT_DICTSIZE_MIN */
-  while (dictContentSize < largestDict) {
-    memcpy(candidateDictBuffer, largestDictbuffer, largestDict);
-    dictContentSize = ZDICT_finalizeDictionary(
-      candidateDictBuffer, dictContentSize, customDictContentEnd - dictContentSize, dictContentSize,
-      samplesBuffer, samplesSizes, nbFinalizeSamples, params.zParams);
-
-    if (ZDICT_isError(dictContentSize)) {
-      free(largestDictbuffer);
-      free(candidateDictBuffer);
-      return COVER_dictSelectionError(dictContentSize);
-
-    }
-
-    totalCompressedSize = COVER_checkTotalCompressedSize(params, samplesSizes,
-                                                         samplesBuffer, offsets,
-                                                         nbCheckSamples, nbSamples,
-                                                         candidateDictBuffer, dictContentSize);
-
-    if (ZSTD_isError(totalCompressedSize)) {
-      free(largestDictbuffer);
-      free(candidateDictBuffer);
-      return COVER_dictSelectionError(totalCompressedSize);
-    }
-
-    if (totalCompressedSize <= largestCompressed * regressionTolerance) {
-      COVER_dictSelection_t selection = { candidateDictBuffer, dictContentSize, totalCompressedSize };
-      free(largestDictbuffer);
-      return selection;
-    }
-    dictContentSize *= 2;
-  }
-  dictContentSize = largestDict;
-  totalCompressedSize = largestCompressed;
-  {
-    COVER_dictSelection_t selection = { largestDictbuffer, dictContentSize, totalCompressedSize };
-    free(candidateDictBuffer);
-    return selection;
-  }
-}
-
-/**
- * Parameters for COVER_tryParameters().
- */
-typedef struct COVER_tryParameters_data_s {
-  const COVER_ctx_t *ctx;
-  COVER_best_t *best;
-  size_t dictBufferCapacity;
-  ZDICT_cover_params_t parameters;
-} COVER_tryParameters_data_t;
-
-/**
- * Tries a set of parameters and updates the COVER_best_t with the results.
- * This function is thread safe if zstd is compiled with multithreaded support.
- * It takes its parameters as an *OWNING* opaque pointer to support threading.
- */
-static void COVER_tryParameters(void *opaque) {
-  /* Save parameters as local variables */
-  COVER_tryParameters_data_t *const data = (COVER_tryParameters_data_t *)opaque;
-  const COVER_ctx_t *const ctx = data->ctx;
-  const ZDICT_cover_params_t parameters = data->parameters;
-  size_t dictBufferCapacity = data->dictBufferCapacity;
-  size_t totalCompressedSize = ERROR(GENERIC);
-  /* Allocate space for hash table, dict, and freqs */
-  COVER_map_t activeDmers;
-  BYTE *const dict = (BYTE * const)malloc(dictBufferCapacity);
-  COVER_dictSelection_t selection = COVER_dictSelectionError(ERROR(GENERIC));
-  U32 *freqs = (U32 *)malloc(ctx->suffixSize * sizeof(U32));
-  if (!COVER_map_init(&activeDmers, parameters.k - parameters.d + 1)) {
-    DISPLAYLEVEL(1, "Failed to allocate dmer map: out of memory\n");
-    goto _cleanup;
-  }
-  if (!dict || !freqs) {
-    DISPLAYLEVEL(1, "Failed to allocate buffers: out of memory\n");
-    goto _cleanup;
-  }
-  /* Copy the frequencies because we need to modify them */
-  memcpy(freqs, ctx->freqs, ctx->suffixSize * sizeof(U32));
-  /* Build the dictionary */
-  {
-    const size_t tail = COVER_buildDictionary(ctx, freqs, &activeDmers, dict,
-                                              dictBufferCapacity, parameters);
-    selection = COVER_selectDict(dict + tail, dictBufferCapacity - tail,
-        ctx->samples, ctx->samplesSizes, (unsigned)ctx->nbTrainSamples, ctx->nbTrainSamples, ctx->nbSamples, parameters, ctx->offsets,
-        totalCompressedSize);
-
-    if (COVER_dictSelectionIsError(selection)) {
-      DISPLAYLEVEL(1, "Failed to select dictionary\n");
-      goto _cleanup;
-    }
-  }
-_cleanup:
-  free(dict);
-  COVER_best_finish(data->best, parameters, selection);
-  free(data);
-  COVER_map_destroy(&activeDmers);
-  COVER_dictSelectionFree(selection);
-  if (freqs) {
-    free(freqs);
-  }
-}
-
-ZDICTLIB_API size_t ZDICT_optimizeTrainFromBuffer_cover(
-    void *dictBuffer, size_t dictBufferCapacity, const void *samplesBuffer,
-    const size_t *samplesSizes, unsigned nbSamples,
-    ZDICT_cover_params_t *parameters) {
-  /* constants */
-  const unsigned nbThreads = parameters->nbThreads;
-  const double splitPoint =
-      parameters->splitPoint <= 0.0 ? DEFAULT_SPLITPOINT : parameters->splitPoint;
-  const unsigned kMinD = parameters->d == 0 ? 6 : parameters->d;
-  const unsigned kMaxD = parameters->d == 0 ? 8 : parameters->d;
-  const unsigned kMinK = parameters->k == 0 ? 50 : parameters->k;
-  const unsigned kMaxK = parameters->k == 0 ? 2000 : parameters->k;
-  const unsigned kSteps = parameters->steps == 0 ? 40 : parameters->steps;
-  const unsigned kStepSize = MAX((kMaxK - kMinK) / kSteps, 1);
-  const unsigned kIterations =
-      (1 + (kMaxD - kMinD) / 2) * (1 + (kMaxK - kMinK) / kStepSize);
-  const unsigned shrinkDict = 0;
-  /* Local variables */
-  const int displayLevel = parameters->zParams.notificationLevel;
-  unsigned iteration = 1;
-  unsigned d;
-  unsigned k;
-  COVER_best_t best;
-  POOL_ctx *pool = NULL;
-  int warned = 0;
-
-  /* Checks */
-  if (splitPoint <= 0 || splitPoint > 1) {
-    LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect parameters\n");
-    return ERROR(parameter_outOfBound);
-  }
-  if (kMinK < kMaxD || kMaxK < kMinK) {
-    LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect parameters\n");
-    return ERROR(parameter_outOfBound);
-  }
-  if (nbSamples == 0) {
-    DISPLAYLEVEL(1, "Cover must have at least one input file\n");
-    return ERROR(srcSize_wrong);
-  }
-  if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
-    DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
-                 ZDICT_DICTSIZE_MIN);
-    return ERROR(dstSize_tooSmall);
-  }
-  if (nbThreads > 1) {
-    pool = POOL_create(nbThreads, 1);
-    if (!pool) {
-      return ERROR(memory_allocation);
-    }
-  }
-  /* Initialization */
-  COVER_best_init(&best);
-  /* Turn down global display level to clean up display at level 2 and below */
-  g_displayLevel = displayLevel == 0 ? 0 : displayLevel - 1;
-  /* Loop through d first because each new value needs a new context */
-  LOCALDISPLAYLEVEL(displayLevel, 2, "Trying %u different sets of parameters\n",
-                    kIterations);
-  for (d = kMinD; d <= kMaxD; d += 2) {
-    /* Initialize the context for this value of d */
-    COVER_ctx_t ctx;
-    LOCALDISPLAYLEVEL(displayLevel, 3, "d=%u\n", d);
-    {
-      const size_t initVal = COVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples, d, splitPoint);
-      if (ZSTD_isError(initVal)) {
-        LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to initialize context\n");
-        COVER_best_destroy(&best);
-        POOL_free(pool);
-        return initVal;
-      }
-    }
-    if (!warned) {
-      COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.suffixSize, displayLevel);
-      warned = 1;
-    }
-    /* Loop through k reusing the same context */
-    for (k = kMinK; k <= kMaxK; k += kStepSize) {
-      /* Prepare the arguments */
-      COVER_tryParameters_data_t *data = (COVER_tryParameters_data_t *)malloc(
-          sizeof(COVER_tryParameters_data_t));
-      LOCALDISPLAYLEVEL(displayLevel, 3, "k=%u\n", k);
-      if (!data) {
-        LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to allocate parameters\n");
-        COVER_best_destroy(&best);
-        COVER_ctx_destroy(&ctx);
-        POOL_free(pool);
-        return ERROR(memory_allocation);
-      }
-      data->ctx = &ctx;
-      data->best = &best;
-      data->dictBufferCapacity = dictBufferCapacity;
-      data->parameters = *parameters;
-      data->parameters.k = k;
-      data->parameters.d = d;
-      data->parameters.splitPoint = splitPoint;
-      data->parameters.steps = kSteps;
-      data->parameters.shrinkDict = shrinkDict;
-      data->parameters.zParams.notificationLevel = g_displayLevel;
-      /* Check the parameters */
-      if (!COVER_checkParameters(data->parameters, dictBufferCapacity)) {
-        DISPLAYLEVEL(1, "Cover parameters incorrect\n");
-        free(data);
-        continue;
-      }
-      /* Call the function and pass ownership of data to it */
-      COVER_best_start(&best);
-      if (pool) {
-        POOL_add(pool, &COVER_tryParameters, data);
-      } else {
-        COVER_tryParameters(data);
-      }
-      /* Print status */
-      LOCALDISPLAYUPDATE(displayLevel, 2, "\r%u%%       ",
-                         (unsigned)((iteration * 100) / kIterations));
-      ++iteration;
-    }
-    COVER_best_wait(&best);
-    COVER_ctx_destroy(&ctx);
-  }
-  LOCALDISPLAYLEVEL(displayLevel, 2, "\r%79s\r", "");
-  /* Fill the output buffer and parameters with output of the best parameters */
-  {
-    const size_t dictSize = best.dictSize;
-    if (ZSTD_isError(best.compressedSize)) {
-      const size_t compressedSize = best.compressedSize;
-      COVER_best_destroy(&best);
-      POOL_free(pool);
-      return compressedSize;
-    }
-    *parameters = best.parameters;
-    memcpy(dictBuffer, best.dict, dictSize);
-    COVER_best_destroy(&best);
-    POOL_free(pool);
-    return dictSize;
-  }
-}
diff --git a/vendor/github.com/DataDog/zstd/cover.h b/vendor/github.com/DataDog/zstd/cover.h
deleted file mode 100644
index d9e0636..0000000
--- a/vendor/github.com/DataDog/zstd/cover.h
+++ /dev/null
@@ -1,147 +0,0 @@
-#include <stdio.h>  /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h>   /* clock */
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-/**
- * COVER_best_t is used for two purposes:
- * 1. Synchronizing threads.
- * 2. Saving the best parameters and dictionary.
- *
- * All of the methods except COVER_best_init() are thread safe if zstd is
- * compiled with multithreaded support.
- */
-typedef struct COVER_best_s {
-  ZSTD_pthread_mutex_t mutex;
-  ZSTD_pthread_cond_t cond;
-  size_t liveJobs;
-  void *dict;
-  size_t dictSize;
-  ZDICT_cover_params_t parameters;
-  size_t compressedSize;
-} COVER_best_t;
-
-/**
- * A segment is a range in the source as well as the score of the segment.
- */
-typedef struct {
-  U32 begin;
-  U32 end;
-  U32 score;
-} COVER_segment_t;
-
-/**
- *Number of epochs and size of each epoch.
- */
-typedef struct {
-  U32 num;
-  U32 size;
-} COVER_epoch_info_t;
-
-/**
- * Struct used for the dictionary selection function.
- */
-typedef struct COVER_dictSelection {
-  BYTE* dictContent;
-  size_t dictSize;
-  size_t totalCompressedSize;
-} COVER_dictSelection_t;
-
-/**
- * Computes the number of epochs and the size of each epoch.
- * We will make sure that each epoch gets at least 10 * k bytes.
- *
- * The COVER algorithms divide the data up into epochs of equal size and
- * select one segment from each epoch.
- *
- * @param maxDictSize The maximum allowed dictionary size.
- * @param nbDmers     The number of dmers we are training on.
- * @param k           The parameter k (segment size).
- * @param passes      The target number of passes over the dmer corpus.
- *                    More passes means a better dictionary.
- */
-COVER_epoch_info_t COVER_computeEpochs(U32 maxDictSize, U32 nbDmers,
-                                       U32 k, U32 passes);
-
-/**
- * Warns the user when their corpus is too small.
- */
-void COVER_warnOnSmallCorpus(size_t maxDictSize, size_t nbDmers, int displayLevel);
-
-/**
- *  Checks total compressed size of a dictionary
- */
-size_t COVER_checkTotalCompressedSize(const ZDICT_cover_params_t parameters,
-                                      const size_t *samplesSizes, const BYTE *samples,
-                                      size_t *offsets,
-                                      size_t nbTrainSamples, size_t nbSamples,
-                                      BYTE *const dict, size_t dictBufferCapacity);
-
-/**
- * Returns the sum of the sample sizes.
- */
-size_t COVER_sum(const size_t *samplesSizes, unsigned nbSamples) ;
-
-/**
- * Initialize the `COVER_best_t`.
- */
-void COVER_best_init(COVER_best_t *best);
-
-/**
- * Wait until liveJobs == 0.
- */
-void COVER_best_wait(COVER_best_t *best);
-
-/**
- * Call COVER_best_wait() and then destroy the COVER_best_t.
- */
-void COVER_best_destroy(COVER_best_t *best);
-
-/**
- * Called when a thread is about to be launched.
- * Increments liveJobs.
- */
-void COVER_best_start(COVER_best_t *best);
-
-/**
- * Called when a thread finishes executing, both on error or success.
- * Decrements liveJobs and signals any waiting threads if liveJobs == 0.
- * If this dictionary is the best so far save it and its parameters.
- */
-void COVER_best_finish(COVER_best_t *best, ZDICT_cover_params_t parameters,
-                       COVER_dictSelection_t selection);
-/**
- * Error function for COVER_selectDict function. Checks if the return
- * value is an error.
- */
-unsigned COVER_dictSelectionIsError(COVER_dictSelection_t selection);
-
- /**
-  * Error function for COVER_selectDict function. Returns a struct where
-  * return.totalCompressedSize is a ZSTD error.
-  */
-COVER_dictSelection_t COVER_dictSelectionError(size_t error);
-
-/**
- * Always call after selectDict is called to free up used memory from
- * newly created dictionary.
- */
-void COVER_dictSelectionFree(COVER_dictSelection_t selection);
-
-/**
- * Called to finalize the dictionary and select one based on whether or not
- * the shrink-dict flag was enabled. If enabled the dictionary used is the
- * smallest dictionary within a specified regression of the compressed size
- * from the largest dictionary.
- */
- COVER_dictSelection_t COVER_selectDict(BYTE* customDictContent,
-                       size_t dictContentSize, const BYTE* samplesBuffer, const size_t* samplesSizes, unsigned nbFinalizeSamples,
-                       size_t nbCheckSamples, size_t nbSamples, ZDICT_cover_params_t params, size_t* offsets, size_t totalCompressedSize);
diff --git a/vendor/github.com/DataDog/zstd/cpu.h b/vendor/github.com/DataDog/zstd/cpu.h
deleted file mode 100644
index 5f0923f..0000000
--- a/vendor/github.com/DataDog/zstd/cpu.h
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright (c) 2018-present, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_COMMON_CPU_H
-#define ZSTD_COMMON_CPU_H
-
-/**
- * Implementation taken from folly/CpuId.h
- * https://github.com/facebook/folly/blob/master/folly/CpuId.h
- */
-
-#include <string.h>
-
-#include "mem.h"
-
-#ifdef _MSC_VER
-#include <intrin.h>
-#endif
-
-typedef struct {
-    U32 f1c;
-    U32 f1d;
-    U32 f7b;
-    U32 f7c;
-} ZSTD_cpuid_t;
-
-MEM_STATIC ZSTD_cpuid_t ZSTD_cpuid(void) {
-    U32 f1c = 0;
-    U32 f1d = 0;
-    U32 f7b = 0;
-    U32 f7c = 0;
-#if defined(_MSC_VER) && (defined(_M_X64) || defined(_M_IX86))
-    int reg[4];
-    __cpuid((int*)reg, 0);
-    {
-        int const n = reg[0];
-        if (n >= 1) {
-            __cpuid((int*)reg, 1);
-            f1c = (U32)reg[2];
-            f1d = (U32)reg[3];
-        }
-        if (n >= 7) {
-            __cpuidex((int*)reg, 7, 0);
-            f7b = (U32)reg[1];
-            f7c = (U32)reg[2];
-        }
-    }
-#elif defined(__i386__) && defined(__PIC__) && !defined(__clang__) && defined(__GNUC__)
-    /* The following block like the normal cpuid branch below, but gcc
-     * reserves ebx for use of its pic register so we must specially
-     * handle the save and restore to avoid clobbering the register
-     */
-    U32 n;
-    __asm__(
-        "pushl %%ebx\n\t"
-        "cpuid\n\t"
-        "popl %%ebx\n\t"
-        : "=a"(n)
-        : "a"(0)
-        : "ecx", "edx");
-    if (n >= 1) {
-      U32 f1a;
-      __asm__(
-          "pushl %%ebx\n\t"
-          "cpuid\n\t"
-          "popl %%ebx\n\t"
-          : "=a"(f1a), "=c"(f1c), "=d"(f1d)
-          : "a"(1));
-    }
-    if (n >= 7) {
-      __asm__(
-          "pushl %%ebx\n\t"
-          "cpuid\n\t"
-          "movl %%ebx, %%eax\n\t"
-          "popl %%ebx"
-          : "=a"(f7b), "=c"(f7c)
-          : "a"(7), "c"(0)
-          : "edx");
-    }
-#elif defined(__x86_64__) || defined(_M_X64) || defined(__i386__)
-    U32 n;
-    __asm__("cpuid" : "=a"(n) : "a"(0) : "ebx", "ecx", "edx");
-    if (n >= 1) {
-      U32 f1a;
-      __asm__("cpuid" : "=a"(f1a), "=c"(f1c), "=d"(f1d) : "a"(1) : "ebx");
-    }
-    if (n >= 7) {
-      U32 f7a;
-      __asm__("cpuid"
-              : "=a"(f7a), "=b"(f7b), "=c"(f7c)
-              : "a"(7), "c"(0)
-              : "edx");
-    }
-#endif
-    {
-        ZSTD_cpuid_t cpuid;
-        cpuid.f1c = f1c;
-        cpuid.f1d = f1d;
-        cpuid.f7b = f7b;
-        cpuid.f7c = f7c;
-        return cpuid;
-    }
-}
-
-#define X(name, r, bit)                                                        \
-  MEM_STATIC int ZSTD_cpuid_##name(ZSTD_cpuid_t const cpuid) {                 \
-    return ((cpuid.r) & (1U << bit)) != 0;                                     \
-  }
-
-/* cpuid(1): Processor Info and Feature Bits. */
-#define C(name, bit) X(name, f1c, bit)
-  C(sse3, 0)
-  C(pclmuldq, 1)
-  C(dtes64, 2)
-  C(monitor, 3)
-  C(dscpl, 4)
-  C(vmx, 5)
-  C(smx, 6)
-  C(eist, 7)
-  C(tm2, 8)
-  C(ssse3, 9)
-  C(cnxtid, 10)
-  C(fma, 12)
-  C(cx16, 13)
-  C(xtpr, 14)
-  C(pdcm, 15)
-  C(pcid, 17)
-  C(dca, 18)
-  C(sse41, 19)
-  C(sse42, 20)
-  C(x2apic, 21)
-  C(movbe, 22)
-  C(popcnt, 23)
-  C(tscdeadline, 24)
-  C(aes, 25)
-  C(xsave, 26)
-  C(osxsave, 27)
-  C(avx, 28)
-  C(f16c, 29)
-  C(rdrand, 30)
-#undef C
-#define D(name, bit) X(name, f1d, bit)
-  D(fpu, 0)
-  D(vme, 1)
-  D(de, 2)
-  D(pse, 3)
-  D(tsc, 4)
-  D(msr, 5)
-  D(pae, 6)
-  D(mce, 7)
-  D(cx8, 8)
-  D(apic, 9)
-  D(sep, 11)
-  D(mtrr, 12)
-  D(pge, 13)
-  D(mca, 14)
-  D(cmov, 15)
-  D(pat, 16)
-  D(pse36, 17)
-  D(psn, 18)
-  D(clfsh, 19)
-  D(ds, 21)
-  D(acpi, 22)
-  D(mmx, 23)
-  D(fxsr, 24)
-  D(sse, 25)
-  D(sse2, 26)
-  D(ss, 27)
-  D(htt, 28)
-  D(tm, 29)
-  D(pbe, 31)
-#undef D
-
-/* cpuid(7): Extended Features. */
-#define B(name, bit) X(name, f7b, bit)
-  B(bmi1, 3)
-  B(hle, 4)
-  B(avx2, 5)
-  B(smep, 7)
-  B(bmi2, 8)
-  B(erms, 9)
-  B(invpcid, 10)
-  B(rtm, 11)
-  B(mpx, 14)
-  B(avx512f, 16)
-  B(avx512dq, 17)
-  B(rdseed, 18)
-  B(adx, 19)
-  B(smap, 20)
-  B(avx512ifma, 21)
-  B(pcommit, 22)
-  B(clflushopt, 23)
-  B(clwb, 24)
-  B(avx512pf, 26)
-  B(avx512er, 27)
-  B(avx512cd, 28)
-  B(sha, 29)
-  B(avx512bw, 30)
-  B(avx512vl, 31)
-#undef B
-#define C(name, bit) X(name, f7c, bit)
-  C(prefetchwt1, 0)
-  C(avx512vbmi, 1)
-#undef C
-
-#undef X
-
-#endif /* ZSTD_COMMON_CPU_H */
diff --git a/vendor/github.com/DataDog/zstd/debug.c b/vendor/github.com/DataDog/zstd/debug.c
deleted file mode 100644
index 3ebdd1c..0000000
--- a/vendor/github.com/DataDog/zstd/debug.c
+++ /dev/null
@@ -1,44 +0,0 @@
-/* ******************************************************************
-   debug
-   Part of FSE library
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-
-/*
- * This module only hosts one global variable
- * which can be used to dynamically influence the verbosity of traces,
- * such as DEBUGLOG and RAWLOG
- */
-
-#include "debug.h"
-
-int g_debuglevel = DEBUGLEVEL;
diff --git a/vendor/github.com/DataDog/zstd/debug.h b/vendor/github.com/DataDog/zstd/debug.h
deleted file mode 100644
index b4fc89d..0000000
--- a/vendor/github.com/DataDog/zstd/debug.h
+++ /dev/null
@@ -1,134 +0,0 @@
-/* ******************************************************************
-   debug
-   Part of FSE library
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-
-/*
- * The purpose of this header is to enable debug functions.
- * They regroup assert(), DEBUGLOG() and RAWLOG() for run-time,
- * and DEBUG_STATIC_ASSERT() for compile-time.
- *
- * By default, DEBUGLEVEL==0, which means run-time debug is disabled.
- *
- * Level 1 enables assert() only.
- * Starting level 2, traces can be generated and pushed to stderr.
- * The higher the level, the more verbose the traces.
- *
- * It's possible to dynamically adjust level using variable g_debug_level,
- * which is only declared if DEBUGLEVEL>=2,
- * and is a global variable, not multi-thread protected (use with care)
- */
-
-#ifndef DEBUG_H_12987983217
-#define DEBUG_H_12987983217
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* static assert is triggered at compile time, leaving no runtime artefact.
- * static assert only works with compile-time constants.
- * Also, this variant can only be used inside a function. */
-#define DEBUG_STATIC_ASSERT(c) (void)sizeof(char[(c) ? 1 : -1])
-
-
-/* DEBUGLEVEL is expected to be defined externally,
- * typically through compiler command line.
- * Value must be a number. */
-#ifndef DEBUGLEVEL
-#  define DEBUGLEVEL 0
-#endif
-
-
-/* DEBUGFILE can be defined externally,
- * typically through compiler command line.
- * note : currently useless.
- * Value must be stderr or stdout */
-#ifndef DEBUGFILE
-#  define DEBUGFILE stderr
-#endif
-
-
-/* recommended values for DEBUGLEVEL :
- * 0 : release mode, no debug, all run-time checks disabled
- * 1 : enables assert() only, no display
- * 2 : reserved, for currently active debug path
- * 3 : events once per object lifetime (CCtx, CDict, etc.)
- * 4 : events once per frame
- * 5 : events once per block
- * 6 : events once per sequence (verbose)
- * 7+: events at every position (*very* verbose)
- *
- * It's generally inconvenient to output traces > 5.
- * In which case, it's possible to selectively trigger high verbosity levels
- * by modifying g_debug_level.
- */
-
-#if (DEBUGLEVEL>=1)
-#  include <assert.h>
-#else
-#  ifndef assert   /* assert may be already defined, due to prior #include <assert.h> */
-#    define assert(condition) ((void)0)   /* disable assert (default) */
-#  endif
-#endif
-
-#if (DEBUGLEVEL>=2)
-#  include <stdio.h>
-extern int g_debuglevel; /* the variable is only declared,
-                            it actually lives in debug.c,
-                            and is shared by the whole process.
-                            It's not thread-safe.
-                            It's useful when enabling very verbose levels
-                            on selective conditions (such as position in src) */
-
-#  define RAWLOG(l, ...) {                                      \
-                if (l<=g_debuglevel) {                          \
-                    fprintf(stderr, __VA_ARGS__);               \
-            }   }
-#  define DEBUGLOG(l, ...) {                                    \
-                if (l<=g_debuglevel) {                          \
-                    fprintf(stderr, __FILE__ ": " __VA_ARGS__); \
-                    fprintf(stderr, " \n");                     \
-            }   }
-#else
-#  define RAWLOG(l, ...)      {}    /* disabled */
-#  define DEBUGLOG(l, ...)    {}    /* disabled */
-#endif
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* DEBUG_H_12987983217 */
diff --git a/vendor/github.com/DataDog/zstd/divsufsort.c b/vendor/github.com/DataDog/zstd/divsufsort.c
deleted file mode 100644
index ead9220..0000000
--- a/vendor/github.com/DataDog/zstd/divsufsort.c
+++ /dev/null
@@ -1,1913 +0,0 @@
-/*
- * divsufsort.c for libdivsufsort-lite
- * Copyright (c) 2003-2008 Yuta Mori All Rights Reserved.
- *
- * 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.
- */
-
-/*- Compiler specifics -*/
-#ifdef __clang__
-#pragma clang diagnostic ignored "-Wshorten-64-to-32"
-#endif
-
-#if defined(_MSC_VER)
-#  pragma warning(disable : 4244)
-#  pragma warning(disable : 4127)    /* C4127 : Condition expression is constant */
-#endif
-
-
-/*- Dependencies -*/
-#include <assert.h>
-#include <stdio.h>
-#include <stdlib.h>
-
-#include "divsufsort.h"
-
-/*- Constants -*/
-#if defined(INLINE)
-# undef INLINE
-#endif
-#if !defined(INLINE)
-# define INLINE __inline
-#endif
-#if defined(ALPHABET_SIZE) && (ALPHABET_SIZE < 1)
-# undef ALPHABET_SIZE
-#endif
-#if !defined(ALPHABET_SIZE)
-# define ALPHABET_SIZE (256)
-#endif
-#define BUCKET_A_SIZE (ALPHABET_SIZE)
-#define BUCKET_B_SIZE (ALPHABET_SIZE * ALPHABET_SIZE)
-#if defined(SS_INSERTIONSORT_THRESHOLD)
-# if SS_INSERTIONSORT_THRESHOLD < 1
-#  undef SS_INSERTIONSORT_THRESHOLD
-#  define SS_INSERTIONSORT_THRESHOLD (1)
-# endif
-#else
-# define SS_INSERTIONSORT_THRESHOLD (8)
-#endif
-#if defined(SS_BLOCKSIZE)
-# if SS_BLOCKSIZE < 0
-#  undef SS_BLOCKSIZE
-#  define SS_BLOCKSIZE (0)
-# elif 32768 <= SS_BLOCKSIZE
-#  undef SS_BLOCKSIZE
-#  define SS_BLOCKSIZE (32767)
-# endif
-#else
-# define SS_BLOCKSIZE (1024)
-#endif
-/* minstacksize = log(SS_BLOCKSIZE) / log(3) * 2 */
-#if SS_BLOCKSIZE == 0
-# define SS_MISORT_STACKSIZE (96)
-#elif SS_BLOCKSIZE <= 4096
-# define SS_MISORT_STACKSIZE (16)
-#else
-# define SS_MISORT_STACKSIZE (24)
-#endif
-#define SS_SMERGE_STACKSIZE (32)
-#define TR_INSERTIONSORT_THRESHOLD (8)
-#define TR_STACKSIZE (64)
-
-
-/*- Macros -*/
-#ifndef SWAP
-# define SWAP(_a, _b) do { t = (_a); (_a) = (_b); (_b) = t; } while(0)
-#endif /* SWAP */
-#ifndef MIN
-# define MIN(_a, _b) (((_a) < (_b)) ? (_a) : (_b))
-#endif /* MIN */
-#ifndef MAX
-# define MAX(_a, _b) (((_a) > (_b)) ? (_a) : (_b))
-#endif /* MAX */
-#define STACK_PUSH(_a, _b, _c, _d)\
-  do {\
-    assert(ssize < STACK_SIZE);\
-    stack[ssize].a = (_a), stack[ssize].b = (_b),\
-    stack[ssize].c = (_c), stack[ssize++].d = (_d);\
-  } while(0)
-#define STACK_PUSH5(_a, _b, _c, _d, _e)\
-  do {\
-    assert(ssize < STACK_SIZE);\
-    stack[ssize].a = (_a), stack[ssize].b = (_b),\
-    stack[ssize].c = (_c), stack[ssize].d = (_d), stack[ssize++].e = (_e);\
-  } while(0)
-#define STACK_POP(_a, _b, _c, _d)\
-  do {\
-    assert(0 <= ssize);\
-    if(ssize == 0) { return; }\
-    (_a) = stack[--ssize].a, (_b) = stack[ssize].b,\
-    (_c) = stack[ssize].c, (_d) = stack[ssize].d;\
-  } while(0)
-#define STACK_POP5(_a, _b, _c, _d, _e)\
-  do {\
-    assert(0 <= ssize);\
-    if(ssize == 0) { return; }\
-    (_a) = stack[--ssize].a, (_b) = stack[ssize].b,\
-    (_c) = stack[ssize].c, (_d) = stack[ssize].d, (_e) = stack[ssize].e;\
-  } while(0)
-#define BUCKET_A(_c0) bucket_A[(_c0)]
-#if ALPHABET_SIZE == 256
-#define BUCKET_B(_c0, _c1) (bucket_B[((_c1) << 8) | (_c0)])
-#define BUCKET_BSTAR(_c0, _c1) (bucket_B[((_c0) << 8) | (_c1)])
-#else
-#define BUCKET_B(_c0, _c1) (bucket_B[(_c1) * ALPHABET_SIZE + (_c0)])
-#define BUCKET_BSTAR(_c0, _c1) (bucket_B[(_c0) * ALPHABET_SIZE + (_c1)])
-#endif
-
-
-/*- Private Functions -*/
-
-static const int lg_table[256]= {
- -1,0,1,1,2,2,2,2,3,3,3,3,3,3,3,3,4,4,4,4,4,4,4,4,4,4,4,4,4,4,4,4,
-  5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,5,
-  6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,
-  6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,6,
-  7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
-  7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
-  7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,
-  7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7
-};
-
-#if (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE)
-
-static INLINE
-int
-ss_ilg(int n) {
-#if SS_BLOCKSIZE == 0
-  return (n & 0xffff0000) ?
-          ((n & 0xff000000) ?
-            24 + lg_table[(n >> 24) & 0xff] :
-            16 + lg_table[(n >> 16) & 0xff]) :
-          ((n & 0x0000ff00) ?
-             8 + lg_table[(n >>  8) & 0xff] :
-             0 + lg_table[(n >>  0) & 0xff]);
-#elif SS_BLOCKSIZE < 256
-  return lg_table[n];
-#else
-  return (n & 0xff00) ?
-          8 + lg_table[(n >> 8) & 0xff] :
-          0 + lg_table[(n >> 0) & 0xff];
-#endif
-}
-
-#endif /* (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE) */
-
-#if SS_BLOCKSIZE != 0
-
-static const int sqq_table[256] = {
-  0,  16,  22,  27,  32,  35,  39,  42,  45,  48,  50,  53,  55,  57,  59,  61,
- 64,  65,  67,  69,  71,  73,  75,  76,  78,  80,  81,  83,  84,  86,  87,  89,
- 90,  91,  93,  94,  96,  97,  98,  99, 101, 102, 103, 104, 106, 107, 108, 109,
-110, 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126,
-128, 128, 129, 130, 131, 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, 142,
-143, 144, 144, 145, 146, 147, 148, 149, 150, 150, 151, 152, 153, 154, 155, 155,
-156, 157, 158, 159, 160, 160, 161, 162, 163, 163, 164, 165, 166, 167, 167, 168,
-169, 170, 170, 171, 172, 173, 173, 174, 175, 176, 176, 177, 178, 178, 179, 180,
-181, 181, 182, 183, 183, 184, 185, 185, 186, 187, 187, 188, 189, 189, 190, 191,
-192, 192, 193, 193, 194, 195, 195, 196, 197, 197, 198, 199, 199, 200, 201, 201,
-202, 203, 203, 204, 204, 205, 206, 206, 207, 208, 208, 209, 209, 210, 211, 211,
-212, 212, 213, 214, 214, 215, 215, 216, 217, 217, 218, 218, 219, 219, 220, 221,
-221, 222, 222, 223, 224, 224, 225, 225, 226, 226, 227, 227, 228, 229, 229, 230,
-230, 231, 231, 232, 232, 233, 234, 234, 235, 235, 236, 236, 237, 237, 238, 238,
-239, 240, 240, 241, 241, 242, 242, 243, 243, 244, 244, 245, 245, 246, 246, 247,
-247, 248, 248, 249, 249, 250, 250, 251, 251, 252, 252, 253, 253, 254, 254, 255
-};
-
-static INLINE
-int
-ss_isqrt(int x) {
-  int y, e;
-
-  if(x >= (SS_BLOCKSIZE * SS_BLOCKSIZE)) { return SS_BLOCKSIZE; }
-  e = (x & 0xffff0000) ?
-        ((x & 0xff000000) ?
-          24 + lg_table[(x >> 24) & 0xff] :
-          16 + lg_table[(x >> 16) & 0xff]) :
-        ((x & 0x0000ff00) ?
-           8 + lg_table[(x >>  8) & 0xff] :
-           0 + lg_table[(x >>  0) & 0xff]);
-
-  if(e >= 16) {
-    y = sqq_table[x >> ((e - 6) - (e & 1))] << ((e >> 1) - 7);
-    if(e >= 24) { y = (y + 1 + x / y) >> 1; }
-    y = (y + 1 + x / y) >> 1;
-  } else if(e >= 8) {
-    y = (sqq_table[x >> ((e - 6) - (e & 1))] >> (7 - (e >> 1))) + 1;
-  } else {
-    return sqq_table[x] >> 4;
-  }
-
-  return (x < (y * y)) ? y - 1 : y;
-}
-
-#endif /* SS_BLOCKSIZE != 0 */
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Compares two suffixes. */
-static INLINE
-int
-ss_compare(const unsigned char *T,
-           const int *p1, const int *p2,
-           int depth) {
-  const unsigned char *U1, *U2, *U1n, *U2n;
-
-  for(U1 = T + depth + *p1,
-      U2 = T + depth + *p2,
-      U1n = T + *(p1 + 1) + 2,
-      U2n = T + *(p2 + 1) + 2;
-      (U1 < U1n) && (U2 < U2n) && (*U1 == *U2);
-      ++U1, ++U2) {
-  }
-
-  return U1 < U1n ?
-        (U2 < U2n ? *U1 - *U2 : 1) :
-        (U2 < U2n ? -1 : 0);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-#if (SS_BLOCKSIZE != 1) && (SS_INSERTIONSORT_THRESHOLD != 1)
-
-/* Insertionsort for small size groups */
-static
-void
-ss_insertionsort(const unsigned char *T, const int *PA,
-                 int *first, int *last, int depth) {
-  int *i, *j;
-  int t;
-  int r;
-
-  for(i = last - 2; first <= i; --i) {
-    for(t = *i, j = i + 1; 0 < (r = ss_compare(T, PA + t, PA + *j, depth));) {
-      do { *(j - 1) = *j; } while((++j < last) && (*j < 0));
-      if(last <= j) { break; }
-    }
-    if(r == 0) { *j = ~*j; }
-    *(j - 1) = t;
-  }
-}
-
-#endif /* (SS_BLOCKSIZE != 1) && (SS_INSERTIONSORT_THRESHOLD != 1) */
-
-
-/*---------------------------------------------------------------------------*/
-
-#if (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE)
-
-static INLINE
-void
-ss_fixdown(const unsigned char *Td, const int *PA,
-           int *SA, int i, int size) {
-  int j, k;
-  int v;
-  int c, d, e;
-
-  for(v = SA[i], c = Td[PA[v]]; (j = 2 * i + 1) < size; SA[i] = SA[k], i = k) {
-    d = Td[PA[SA[k = j++]]];
-    if(d < (e = Td[PA[SA[j]]])) { k = j; d = e; }
-    if(d <= c) { break; }
-  }
-  SA[i] = v;
-}
-
-/* Simple top-down heapsort. */
-static
-void
-ss_heapsort(const unsigned char *Td, const int *PA, int *SA, int size) {
-  int i, m;
-  int t;
-
-  m = size;
-  if((size % 2) == 0) {
-    m--;
-    if(Td[PA[SA[m / 2]]] < Td[PA[SA[m]]]) { SWAP(SA[m], SA[m / 2]); }
-  }
-
-  for(i = m / 2 - 1; 0 <= i; --i) { ss_fixdown(Td, PA, SA, i, m); }
-  if((size % 2) == 0) { SWAP(SA[0], SA[m]); ss_fixdown(Td, PA, SA, 0, m); }
-  for(i = m - 1; 0 < i; --i) {
-    t = SA[0], SA[0] = SA[i];
-    ss_fixdown(Td, PA, SA, 0, i);
-    SA[i] = t;
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Returns the median of three elements. */
-static INLINE
-int *
-ss_median3(const unsigned char *Td, const int *PA,
-           int *v1, int *v2, int *v3) {
-  int *t;
-  if(Td[PA[*v1]] > Td[PA[*v2]]) { SWAP(v1, v2); }
-  if(Td[PA[*v2]] > Td[PA[*v3]]) {
-    if(Td[PA[*v1]] > Td[PA[*v3]]) { return v1; }
-    else { return v3; }
-  }
-  return v2;
-}
-
-/* Returns the median of five elements. */
-static INLINE
-int *
-ss_median5(const unsigned char *Td, const int *PA,
-           int *v1, int *v2, int *v3, int *v4, int *v5) {
-  int *t;
-  if(Td[PA[*v2]] > Td[PA[*v3]]) { SWAP(v2, v3); }
-  if(Td[PA[*v4]] > Td[PA[*v5]]) { SWAP(v4, v5); }
-  if(Td[PA[*v2]] > Td[PA[*v4]]) { SWAP(v2, v4); SWAP(v3, v5); }
-  if(Td[PA[*v1]] > Td[PA[*v3]]) { SWAP(v1, v3); }
-  if(Td[PA[*v1]] > Td[PA[*v4]]) { SWAP(v1, v4); SWAP(v3, v5); }
-  if(Td[PA[*v3]] > Td[PA[*v4]]) { return v4; }
-  return v3;
-}
-
-/* Returns the pivot element. */
-static INLINE
-int *
-ss_pivot(const unsigned char *Td, const int *PA, int *first, int *last) {
-  int *middle;
-  int t;
-
-  t = last - first;
-  middle = first + t / 2;
-
-  if(t <= 512) {
-    if(t <= 32) {
-      return ss_median3(Td, PA, first, middle, last - 1);
-    } else {
-      t >>= 2;
-      return ss_median5(Td, PA, first, first + t, middle, last - 1 - t, last - 1);
-    }
-  }
-  t >>= 3;
-  first  = ss_median3(Td, PA, first, first + t, first + (t << 1));
-  middle = ss_median3(Td, PA, middle - t, middle, middle + t);
-  last   = ss_median3(Td, PA, last - 1 - (t << 1), last - 1 - t, last - 1);
-  return ss_median3(Td, PA, first, middle, last);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Binary partition for substrings. */
-static INLINE
-int *
-ss_partition(const int *PA,
-                    int *first, int *last, int depth) {
-  int *a, *b;
-  int t;
-  for(a = first - 1, b = last;;) {
-    for(; (++a < b) && ((PA[*a] + depth) >= (PA[*a + 1] + 1));) { *a = ~*a; }
-    for(; (a < --b) && ((PA[*b] + depth) <  (PA[*b + 1] + 1));) { }
-    if(b <= a) { break; }
-    t = ~*b;
-    *b = *a;
-    *a = t;
-  }
-  if(first < a) { *first = ~*first; }
-  return a;
-}
-
-/* Multikey introsort for medium size groups. */
-static
-void
-ss_mintrosort(const unsigned char *T, const int *PA,
-              int *first, int *last,
-              int depth) {
-#define STACK_SIZE SS_MISORT_STACKSIZE
-  struct { int *a, *b, c; int d; } stack[STACK_SIZE];
-  const unsigned char *Td;
-  int *a, *b, *c, *d, *e, *f;
-  int s, t;
-  int ssize;
-  int limit;
-  int v, x = 0;
-
-  for(ssize = 0, limit = ss_ilg(last - first);;) {
-
-    if((last - first) <= SS_INSERTIONSORT_THRESHOLD) {
-#if 1 < SS_INSERTIONSORT_THRESHOLD
-      if(1 < (last - first)) { ss_insertionsort(T, PA, first, last, depth); }
-#endif
-      STACK_POP(first, last, depth, limit);
-      continue;
-    }
-
-    Td = T + depth;
-    if(limit-- == 0) { ss_heapsort(Td, PA, first, last - first); }
-    if(limit < 0) {
-      for(a = first + 1, v = Td[PA[*first]]; a < last; ++a) {
-        if((x = Td[PA[*a]]) != v) {
-          if(1 < (a - first)) { break; }
-          v = x;
-          first = a;
-        }
-      }
-      if(Td[PA[*first] - 1] < v) {
-        first = ss_partition(PA, first, a, depth);
-      }
-      if((a - first) <= (last - a)) {
-        if(1 < (a - first)) {
-          STACK_PUSH(a, last, depth, -1);
-          last = a, depth += 1, limit = ss_ilg(a - first);
-        } else {
-          first = a, limit = -1;
-        }
-      } else {
-        if(1 < (last - a)) {
-          STACK_PUSH(first, a, depth + 1, ss_ilg(a - first));
-          first = a, limit = -1;
-        } else {
-          last = a, depth += 1, limit = ss_ilg(a - first);
-        }
-      }
-      continue;
-    }
-
-    /* choose pivot */
-    a = ss_pivot(Td, PA, first, last);
-    v = Td[PA[*a]];
-    SWAP(*first, *a);
-
-    /* partition */
-    for(b = first; (++b < last) && ((x = Td[PA[*b]]) == v);) { }
-    if(((a = b) < last) && (x < v)) {
-      for(; (++b < last) && ((x = Td[PA[*b]]) <= v);) {
-        if(x == v) { SWAP(*b, *a); ++a; }
-      }
-    }
-    for(c = last; (b < --c) && ((x = Td[PA[*c]]) == v);) { }
-    if((b < (d = c)) && (x > v)) {
-      for(; (b < --c) && ((x = Td[PA[*c]]) >= v);) {
-        if(x == v) { SWAP(*c, *d); --d; }
-      }
-    }
-    for(; b < c;) {
-      SWAP(*b, *c);
-      for(; (++b < c) && ((x = Td[PA[*b]]) <= v);) {
-        if(x == v) { SWAP(*b, *a); ++a; }
-      }
-      for(; (b < --c) && ((x = Td[PA[*c]]) >= v);) {
-        if(x == v) { SWAP(*c, *d); --d; }
-      }
-    }
-
-    if(a <= d) {
-      c = b - 1;
-
-      if((s = a - first) > (t = b - a)) { s = t; }
-      for(e = first, f = b - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
-      if((s = d - c) > (t = last - d - 1)) { s = t; }
-      for(e = b, f = last - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
-
-      a = first + (b - a), c = last - (d - c);
-      b = (v <= Td[PA[*a] - 1]) ? a : ss_partition(PA, a, c, depth);
-
-      if((a - first) <= (last - c)) {
-        if((last - c) <= (c - b)) {
-          STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
-          STACK_PUSH(c, last, depth, limit);
-          last = a;
-        } else if((a - first) <= (c - b)) {
-          STACK_PUSH(c, last, depth, limit);
-          STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
-          last = a;
-        } else {
-          STACK_PUSH(c, last, depth, limit);
-          STACK_PUSH(first, a, depth, limit);
-          first = b, last = c, depth += 1, limit = ss_ilg(c - b);
-        }
-      } else {
-        if((a - first) <= (c - b)) {
-          STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
-          STACK_PUSH(first, a, depth, limit);
-          first = c;
-        } else if((last - c) <= (c - b)) {
-          STACK_PUSH(first, a, depth, limit);
-          STACK_PUSH(b, c, depth + 1, ss_ilg(c - b));
-          first = c;
-        } else {
-          STACK_PUSH(first, a, depth, limit);
-          STACK_PUSH(c, last, depth, limit);
-          first = b, last = c, depth += 1, limit = ss_ilg(c - b);
-        }
-      }
-    } else {
-      limit += 1;
-      if(Td[PA[*first] - 1] < v) {
-        first = ss_partition(PA, first, last, depth);
-        limit = ss_ilg(last - first);
-      }
-      depth += 1;
-    }
-  }
-#undef STACK_SIZE
-}
-
-#endif /* (SS_BLOCKSIZE == 0) || (SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE) */
-
-
-/*---------------------------------------------------------------------------*/
-
-#if SS_BLOCKSIZE != 0
-
-static INLINE
-void
-ss_blockswap(int *a, int *b, int n) {
-  int t;
-  for(; 0 < n; --n, ++a, ++b) {
-    t = *a, *a = *b, *b = t;
-  }
-}
-
-static INLINE
-void
-ss_rotate(int *first, int *middle, int *last) {
-  int *a, *b, t;
-  int l, r;
-  l = middle - first, r = last - middle;
-  for(; (0 < l) && (0 < r);) {
-    if(l == r) { ss_blockswap(first, middle, l); break; }
-    if(l < r) {
-      a = last - 1, b = middle - 1;
-      t = *a;
-      do {
-        *a-- = *b, *b-- = *a;
-        if(b < first) {
-          *a = t;
-          last = a;
-          if((r -= l + 1) <= l) { break; }
-          a -= 1, b = middle - 1;
-          t = *a;
-        }
-      } while(1);
-    } else {
-      a = first, b = middle;
-      t = *a;
-      do {
-        *a++ = *b, *b++ = *a;
-        if(last <= b) {
-          *a = t;
-          first = a + 1;
-          if((l -= r + 1) <= r) { break; }
-          a += 1, b = middle;
-          t = *a;
-        }
-      } while(1);
-    }
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static
-void
-ss_inplacemerge(const unsigned char *T, const int *PA,
-                int *first, int *middle, int *last,
-                int depth) {
-  const int *p;
-  int *a, *b;
-  int len, half;
-  int q, r;
-  int x;
-
-  for(;;) {
-    if(*(last - 1) < 0) { x = 1; p = PA + ~*(last - 1); }
-    else                { x = 0; p = PA +  *(last - 1); }
-    for(a = first, len = middle - first, half = len >> 1, r = -1;
-        0 < len;
-        len = half, half >>= 1) {
-      b = a + half;
-      q = ss_compare(T, PA + ((0 <= *b) ? *b : ~*b), p, depth);
-      if(q < 0) {
-        a = b + 1;
-        half -= (len & 1) ^ 1;
-      } else {
-        r = q;
-      }
-    }
-    if(a < middle) {
-      if(r == 0) { *a = ~*a; }
-      ss_rotate(a, middle, last);
-      last -= middle - a;
-      middle = a;
-      if(first == middle) { break; }
-    }
-    --last;
-    if(x != 0) { while(*--last < 0) { } }
-    if(middle == last) { break; }
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Merge-forward with internal buffer. */
-static
-void
-ss_mergeforward(const unsigned char *T, const int *PA,
-                int *first, int *middle, int *last,
-                int *buf, int depth) {
-  int *a, *b, *c, *bufend;
-  int t;
-  int r;
-
-  bufend = buf + (middle - first) - 1;
-  ss_blockswap(buf, first, middle - first);
-
-  for(t = *(a = first), b = buf, c = middle;;) {
-    r = ss_compare(T, PA + *b, PA + *c, depth);
-    if(r < 0) {
-      do {
-        *a++ = *b;
-        if(bufend <= b) { *bufend = t; return; }
-        *b++ = *a;
-      } while(*b < 0);
-    } else if(r > 0) {
-      do {
-        *a++ = *c, *c++ = *a;
-        if(last <= c) {
-          while(b < bufend) { *a++ = *b, *b++ = *a; }
-          *a = *b, *b = t;
-          return;
-        }
-      } while(*c < 0);
-    } else {
-      *c = ~*c;
-      do {
-        *a++ = *b;
-        if(bufend <= b) { *bufend = t; return; }
-        *b++ = *a;
-      } while(*b < 0);
-
-      do {
-        *a++ = *c, *c++ = *a;
-        if(last <= c) {
-          while(b < bufend) { *a++ = *b, *b++ = *a; }
-          *a = *b, *b = t;
-          return;
-        }
-      } while(*c < 0);
-    }
-  }
-}
-
-/* Merge-backward with internal buffer. */
-static
-void
-ss_mergebackward(const unsigned char *T, const int *PA,
-                 int *first, int *middle, int *last,
-                 int *buf, int depth) {
-  const int *p1, *p2;
-  int *a, *b, *c, *bufend;
-  int t;
-  int r;
-  int x;
-
-  bufend = buf + (last - middle) - 1;
-  ss_blockswap(buf, middle, last - middle);
-
-  x = 0;
-  if(*bufend < 0)       { p1 = PA + ~*bufend; x |= 1; }
-  else                  { p1 = PA +  *bufend; }
-  if(*(middle - 1) < 0) { p2 = PA + ~*(middle - 1); x |= 2; }
-  else                  { p2 = PA +  *(middle - 1); }
-  for(t = *(a = last - 1), b = bufend, c = middle - 1;;) {
-    r = ss_compare(T, p1, p2, depth);
-    if(0 < r) {
-      if(x & 1) { do { *a-- = *b, *b-- = *a; } while(*b < 0); x ^= 1; }
-      *a-- = *b;
-      if(b <= buf) { *buf = t; break; }
-      *b-- = *a;
-      if(*b < 0) { p1 = PA + ~*b; x |= 1; }
-      else       { p1 = PA +  *b; }
-    } else if(r < 0) {
-      if(x & 2) { do { *a-- = *c, *c-- = *a; } while(*c < 0); x ^= 2; }
-      *a-- = *c, *c-- = *a;
-      if(c < first) {
-        while(buf < b) { *a-- = *b, *b-- = *a; }
-        *a = *b, *b = t;
-        break;
-      }
-      if(*c < 0) { p2 = PA + ~*c; x |= 2; }
-      else       { p2 = PA +  *c; }
-    } else {
-      if(x & 1) { do { *a-- = *b, *b-- = *a; } while(*b < 0); x ^= 1; }
-      *a-- = ~*b;
-      if(b <= buf) { *buf = t; break; }
-      *b-- = *a;
-      if(x & 2) { do { *a-- = *c, *c-- = *a; } while(*c < 0); x ^= 2; }
-      *a-- = *c, *c-- = *a;
-      if(c < first) {
-        while(buf < b) { *a-- = *b, *b-- = *a; }
-        *a = *b, *b = t;
-        break;
-      }
-      if(*b < 0) { p1 = PA + ~*b; x |= 1; }
-      else       { p1 = PA +  *b; }
-      if(*c < 0) { p2 = PA + ~*c; x |= 2; }
-      else       { p2 = PA +  *c; }
-    }
-  }
-}
-
-/* D&C based merge. */
-static
-void
-ss_swapmerge(const unsigned char *T, const int *PA,
-             int *first, int *middle, int *last,
-             int *buf, int bufsize, int depth) {
-#define STACK_SIZE SS_SMERGE_STACKSIZE
-#define GETIDX(a) ((0 <= (a)) ? (a) : (~(a)))
-#define MERGE_CHECK(a, b, c)\
-  do {\
-    if(((c) & 1) ||\
-       (((c) & 2) && (ss_compare(T, PA + GETIDX(*((a) - 1)), PA + *(a), depth) == 0))) {\
-      *(a) = ~*(a);\
-    }\
-    if(((c) & 4) && ((ss_compare(T, PA + GETIDX(*((b) - 1)), PA + *(b), depth) == 0))) {\
-      *(b) = ~*(b);\
-    }\
-  } while(0)
-  struct { int *a, *b, *c; int d; } stack[STACK_SIZE];
-  int *l, *r, *lm, *rm;
-  int m, len, half;
-  int ssize;
-  int check, next;
-
-  for(check = 0, ssize = 0;;) {
-    if((last - middle) <= bufsize) {
-      if((first < middle) && (middle < last)) {
-        ss_mergebackward(T, PA, first, middle, last, buf, depth);
-      }
-      MERGE_CHECK(first, last, check);
-      STACK_POP(first, middle, last, check);
-      continue;
-    }
-
-    if((middle - first) <= bufsize) {
-      if(first < middle) {
-        ss_mergeforward(T, PA, first, middle, last, buf, depth);
-      }
-      MERGE_CHECK(first, last, check);
-      STACK_POP(first, middle, last, check);
-      continue;
-    }
-
-    for(m = 0, len = MIN(middle - first, last - middle), half = len >> 1;
-        0 < len;
-        len = half, half >>= 1) {
-      if(ss_compare(T, PA + GETIDX(*(middle + m + half)),
-                       PA + GETIDX(*(middle - m - half - 1)), depth) < 0) {
-        m += half + 1;
-        half -= (len & 1) ^ 1;
-      }
-    }
-
-    if(0 < m) {
-      lm = middle - m, rm = middle + m;
-      ss_blockswap(lm, middle, m);
-      l = r = middle, next = 0;
-      if(rm < last) {
-        if(*rm < 0) {
-          *rm = ~*rm;
-          if(first < lm) { for(; *--l < 0;) { } next |= 4; }
-          next |= 1;
-        } else if(first < lm) {
-          for(; *r < 0; ++r) { }
-          next |= 2;
-        }
-      }
-
-      if((l - first) <= (last - r)) {
-        STACK_PUSH(r, rm, last, (next & 3) | (check & 4));
-        middle = lm, last = l, check = (check & 3) | (next & 4);
-      } else {
-        if((next & 2) && (r == middle)) { next ^= 6; }
-        STACK_PUSH(first, lm, l, (check & 3) | (next & 4));
-        first = r, middle = rm, check = (next & 3) | (check & 4);
-      }
-    } else {
-      if(ss_compare(T, PA + GETIDX(*(middle - 1)), PA + *middle, depth) == 0) {
-        *middle = ~*middle;
-      }
-      MERGE_CHECK(first, last, check);
-      STACK_POP(first, middle, last, check);
-    }
-  }
-#undef STACK_SIZE
-}
-
-#endif /* SS_BLOCKSIZE != 0 */
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Substring sort */
-static
-void
-sssort(const unsigned char *T, const int *PA,
-       int *first, int *last,
-       int *buf, int bufsize,
-       int depth, int n, int lastsuffix) {
-  int *a;
-#if SS_BLOCKSIZE != 0
-  int *b, *middle, *curbuf;
-  int j, k, curbufsize, limit;
-#endif
-  int i;
-
-  if(lastsuffix != 0) { ++first; }
-
-#if SS_BLOCKSIZE == 0
-  ss_mintrosort(T, PA, first, last, depth);
-#else
-  if((bufsize < SS_BLOCKSIZE) &&
-      (bufsize < (last - first)) &&
-      (bufsize < (limit = ss_isqrt(last - first)))) {
-    if(SS_BLOCKSIZE < limit) { limit = SS_BLOCKSIZE; }
-    buf = middle = last - limit, bufsize = limit;
-  } else {
-    middle = last, limit = 0;
-  }
-  for(a = first, i = 0; SS_BLOCKSIZE < (middle - a); a += SS_BLOCKSIZE, ++i) {
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
-    ss_mintrosort(T, PA, a, a + SS_BLOCKSIZE, depth);
-#elif 1 < SS_BLOCKSIZE
-    ss_insertionsort(T, PA, a, a + SS_BLOCKSIZE, depth);
-#endif
-    curbufsize = last - (a + SS_BLOCKSIZE);
-    curbuf = a + SS_BLOCKSIZE;
-    if(curbufsize <= bufsize) { curbufsize = bufsize, curbuf = buf; }
-    for(b = a, k = SS_BLOCKSIZE, j = i; j & 1; b -= k, k <<= 1, j >>= 1) {
-      ss_swapmerge(T, PA, b - k, b, b + k, curbuf, curbufsize, depth);
-    }
-  }
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
-  ss_mintrosort(T, PA, a, middle, depth);
-#elif 1 < SS_BLOCKSIZE
-  ss_insertionsort(T, PA, a, middle, depth);
-#endif
-  for(k = SS_BLOCKSIZE; i != 0; k <<= 1, i >>= 1) {
-    if(i & 1) {
-      ss_swapmerge(T, PA, a - k, a, middle, buf, bufsize, depth);
-      a -= k;
-    }
-  }
-  if(limit != 0) {
-#if SS_INSERTIONSORT_THRESHOLD < SS_BLOCKSIZE
-    ss_mintrosort(T, PA, middle, last, depth);
-#elif 1 < SS_BLOCKSIZE
-    ss_insertionsort(T, PA, middle, last, depth);
-#endif
-    ss_inplacemerge(T, PA, first, middle, last, depth);
-  }
-#endif
-
-  if(lastsuffix != 0) {
-    /* Insert last type B* suffix. */
-    int PAi[2]; PAi[0] = PA[*(first - 1)], PAi[1] = n - 2;
-    for(a = first, i = *(first - 1);
-        (a < last) && ((*a < 0) || (0 < ss_compare(T, &(PAi[0]), PA + *a, depth)));
-        ++a) {
-      *(a - 1) = *a;
-    }
-    *(a - 1) = i;
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-int
-tr_ilg(int n) {
-  return (n & 0xffff0000) ?
-          ((n & 0xff000000) ?
-            24 + lg_table[(n >> 24) & 0xff] :
-            16 + lg_table[(n >> 16) & 0xff]) :
-          ((n & 0x0000ff00) ?
-             8 + lg_table[(n >>  8) & 0xff] :
-             0 + lg_table[(n >>  0) & 0xff]);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Simple insertionsort for small size groups. */
-static
-void
-tr_insertionsort(const int *ISAd, int *first, int *last) {
-  int *a, *b;
-  int t, r;
-
-  for(a = first + 1; a < last; ++a) {
-    for(t = *a, b = a - 1; 0 > (r = ISAd[t] - ISAd[*b]);) {
-      do { *(b + 1) = *b; } while((first <= --b) && (*b < 0));
-      if(b < first) { break; }
-    }
-    if(r == 0) { *b = ~*b; }
-    *(b + 1) = t;
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-void
-tr_fixdown(const int *ISAd, int *SA, int i, int size) {
-  int j, k;
-  int v;
-  int c, d, e;
-
-  for(v = SA[i], c = ISAd[v]; (j = 2 * i + 1) < size; SA[i] = SA[k], i = k) {
-    d = ISAd[SA[k = j++]];
-    if(d < (e = ISAd[SA[j]])) { k = j; d = e; }
-    if(d <= c) { break; }
-  }
-  SA[i] = v;
-}
-
-/* Simple top-down heapsort. */
-static
-void
-tr_heapsort(const int *ISAd, int *SA, int size) {
-  int i, m;
-  int t;
-
-  m = size;
-  if((size % 2) == 0) {
-    m--;
-    if(ISAd[SA[m / 2]] < ISAd[SA[m]]) { SWAP(SA[m], SA[m / 2]); }
-  }
-
-  for(i = m / 2 - 1; 0 <= i; --i) { tr_fixdown(ISAd, SA, i, m); }
-  if((size % 2) == 0) { SWAP(SA[0], SA[m]); tr_fixdown(ISAd, SA, 0, m); }
-  for(i = m - 1; 0 < i; --i) {
-    t = SA[0], SA[0] = SA[i];
-    tr_fixdown(ISAd, SA, 0, i);
-    SA[i] = t;
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Returns the median of three elements. */
-static INLINE
-int *
-tr_median3(const int *ISAd, int *v1, int *v2, int *v3) {
-  int *t;
-  if(ISAd[*v1] > ISAd[*v2]) { SWAP(v1, v2); }
-  if(ISAd[*v2] > ISAd[*v3]) {
-    if(ISAd[*v1] > ISAd[*v3]) { return v1; }
-    else { return v3; }
-  }
-  return v2;
-}
-
-/* Returns the median of five elements. */
-static INLINE
-int *
-tr_median5(const int *ISAd,
-           int *v1, int *v2, int *v3, int *v4, int *v5) {
-  int *t;
-  if(ISAd[*v2] > ISAd[*v3]) { SWAP(v2, v3); }
-  if(ISAd[*v4] > ISAd[*v5]) { SWAP(v4, v5); }
-  if(ISAd[*v2] > ISAd[*v4]) { SWAP(v2, v4); SWAP(v3, v5); }
-  if(ISAd[*v1] > ISAd[*v3]) { SWAP(v1, v3); }
-  if(ISAd[*v1] > ISAd[*v4]) { SWAP(v1, v4); SWAP(v3, v5); }
-  if(ISAd[*v3] > ISAd[*v4]) { return v4; }
-  return v3;
-}
-
-/* Returns the pivot element. */
-static INLINE
-int *
-tr_pivot(const int *ISAd, int *first, int *last) {
-  int *middle;
-  int t;
-
-  t = last - first;
-  middle = first + t / 2;
-
-  if(t <= 512) {
-    if(t <= 32) {
-      return tr_median3(ISAd, first, middle, last - 1);
-    } else {
-      t >>= 2;
-      return tr_median5(ISAd, first, first + t, middle, last - 1 - t, last - 1);
-    }
-  }
-  t >>= 3;
-  first  = tr_median3(ISAd, first, first + t, first + (t << 1));
-  middle = tr_median3(ISAd, middle - t, middle, middle + t);
-  last   = tr_median3(ISAd, last - 1 - (t << 1), last - 1 - t, last - 1);
-  return tr_median3(ISAd, first, middle, last);
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-typedef struct _trbudget_t trbudget_t;
-struct _trbudget_t {
-  int chance;
-  int remain;
-  int incval;
-  int count;
-};
-
-static INLINE
-void
-trbudget_init(trbudget_t *budget, int chance, int incval) {
-  budget->chance = chance;
-  budget->remain = budget->incval = incval;
-}
-
-static INLINE
-int
-trbudget_check(trbudget_t *budget, int size) {
-  if(size <= budget->remain) { budget->remain -= size; return 1; }
-  if(budget->chance == 0) { budget->count += size; return 0; }
-  budget->remain += budget->incval - size;
-  budget->chance -= 1;
-  return 1;
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-static INLINE
-void
-tr_partition(const int *ISAd,
-             int *first, int *middle, int *last,
-             int **pa, int **pb, int v) {
-  int *a, *b, *c, *d, *e, *f;
-  int t, s;
-  int x = 0;
-
-  for(b = middle - 1; (++b < last) && ((x = ISAd[*b]) == v);) { }
-  if(((a = b) < last) && (x < v)) {
-    for(; (++b < last) && ((x = ISAd[*b]) <= v);) {
-      if(x == v) { SWAP(*b, *a); ++a; }
-    }
-  }
-  for(c = last; (b < --c) && ((x = ISAd[*c]) == v);) { }
-  if((b < (d = c)) && (x > v)) {
-    for(; (b < --c) && ((x = ISAd[*c]) >= v);) {
-      if(x == v) { SWAP(*c, *d); --d; }
-    }
-  }
-  for(; b < c;) {
-    SWAP(*b, *c);
-    for(; (++b < c) && ((x = ISAd[*b]) <= v);) {
-      if(x == v) { SWAP(*b, *a); ++a; }
-    }
-    for(; (b < --c) && ((x = ISAd[*c]) >= v);) {
-      if(x == v) { SWAP(*c, *d); --d; }
-    }
-  }
-
-  if(a <= d) {
-    c = b - 1;
-    if((s = a - first) > (t = b - a)) { s = t; }
-    for(e = first, f = b - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
-    if((s = d - c) > (t = last - d - 1)) { s = t; }
-    for(e = b, f = last - s; 0 < s; --s, ++e, ++f) { SWAP(*e, *f); }
-    first += (b - a), last -= (d - c);
-  }
-  *pa = first, *pb = last;
-}
-
-static
-void
-tr_copy(int *ISA, const int *SA,
-        int *first, int *a, int *b, int *last,
-        int depth) {
-  /* sort suffixes of middle partition
-     by using sorted order of suffixes of left and right partition. */
-  int *c, *d, *e;
-  int s, v;
-
-  v = b - SA - 1;
-  for(c = first, d = a - 1; c <= d; ++c) {
-    if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
-      *++d = s;
-      ISA[s] = d - SA;
-    }
-  }
-  for(c = last - 1, e = d + 1, d = b; e < d; --c) {
-    if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
-      *--d = s;
-      ISA[s] = d - SA;
-    }
-  }
-}
-
-static
-void
-tr_partialcopy(int *ISA, const int *SA,
-               int *first, int *a, int *b, int *last,
-               int depth) {
-  int *c, *d, *e;
-  int s, v;
-  int rank, lastrank, newrank = -1;
-
-  v = b - SA - 1;
-  lastrank = -1;
-  for(c = first, d = a - 1; c <= d; ++c) {
-    if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
-      *++d = s;
-      rank = ISA[s + depth];
-      if(lastrank != rank) { lastrank = rank; newrank = d - SA; }
-      ISA[s] = newrank;
-    }
-  }
-
-  lastrank = -1;
-  for(e = d; first <= e; --e) {
-    rank = ISA[*e];
-    if(lastrank != rank) { lastrank = rank; newrank = e - SA; }
-    if(newrank != rank) { ISA[*e] = newrank; }
-  }
-
-  lastrank = -1;
-  for(c = last - 1, e = d + 1, d = b; e < d; --c) {
-    if((0 <= (s = *c - depth)) && (ISA[s] == v)) {
-      *--d = s;
-      rank = ISA[s + depth];
-      if(lastrank != rank) { lastrank = rank; newrank = d - SA; }
-      ISA[s] = newrank;
-    }
-  }
-}
-
-static
-void
-tr_introsort(int *ISA, const int *ISAd,
-             int *SA, int *first, int *last,
-             trbudget_t *budget) {
-#define STACK_SIZE TR_STACKSIZE
-  struct { const int *a; int *b, *c; int d, e; }stack[STACK_SIZE];
-  int *a, *b, *c;
-  int t;
-  int v, x = 0;
-  int incr = ISAd - ISA;
-  int limit, next;
-  int ssize, trlink = -1;
-
-  for(ssize = 0, limit = tr_ilg(last - first);;) {
-
-    if(limit < 0) {
-      if(limit == -1) {
-        /* tandem repeat partition */
-        tr_partition(ISAd - incr, first, first, last, &a, &b, last - SA - 1);
-
-        /* update ranks */
-        if(a < last) {
-          for(c = first, v = a - SA - 1; c < a; ++c) { ISA[*c] = v; }
-        }
-        if(b < last) {
-          for(c = a, v = b - SA - 1; c < b; ++c) { ISA[*c] = v; }
-        }
-
-        /* push */
-        if(1 < (b - a)) {
-          STACK_PUSH5(NULL, a, b, 0, 0);
-          STACK_PUSH5(ISAd - incr, first, last, -2, trlink);
-          trlink = ssize - 2;
-        }
-        if((a - first) <= (last - b)) {
-          if(1 < (a - first)) {
-            STACK_PUSH5(ISAd, b, last, tr_ilg(last - b), trlink);
-            last = a, limit = tr_ilg(a - first);
-          } else if(1 < (last - b)) {
-            first = b, limit = tr_ilg(last - b);
-          } else {
-            STACK_POP5(ISAd, first, last, limit, trlink);
-          }
-        } else {
-          if(1 < (last - b)) {
-            STACK_PUSH5(ISAd, first, a, tr_ilg(a - first), trlink);
-            first = b, limit = tr_ilg(last - b);
-          } else if(1 < (a - first)) {
-            last = a, limit = tr_ilg(a - first);
-          } else {
-            STACK_POP5(ISAd, first, last, limit, trlink);
-          }
-        }
-      } else if(limit == -2) {
-        /* tandem repeat copy */
-        a = stack[--ssize].b, b = stack[ssize].c;
-        if(stack[ssize].d == 0) {
-          tr_copy(ISA, SA, first, a, b, last, ISAd - ISA);
-        } else {
-          if(0 <= trlink) { stack[trlink].d = -1; }
-          tr_partialcopy(ISA, SA, first, a, b, last, ISAd - ISA);
-        }
-        STACK_POP5(ISAd, first, last, limit, trlink);
-      } else {
-        /* sorted partition */
-        if(0 <= *first) {
-          a = first;
-          do { ISA[*a] = a - SA; } while((++a < last) && (0 <= *a));
-          first = a;
-        }
-        if(first < last) {
-          a = first; do { *a = ~*a; } while(*++a < 0);
-          next = (ISA[*a] != ISAd[*a]) ? tr_ilg(a - first + 1) : -1;
-          if(++a < last) { for(b = first, v = a - SA - 1; b < a; ++b) { ISA[*b] = v; } }
-
-          /* push */
-          if(trbudget_check(budget, a - first)) {
-            if((a - first) <= (last - a)) {
-              STACK_PUSH5(ISAd, a, last, -3, trlink);
-              ISAd += incr, last = a, limit = next;
-            } else {
-              if(1 < (last - a)) {
-                STACK_PUSH5(ISAd + incr, first, a, next, trlink);
-                first = a, limit = -3;
-              } else {
-                ISAd += incr, last = a, limit = next;
-              }
-            }
-          } else {
-            if(0 <= trlink) { stack[trlink].d = -1; }
-            if(1 < (last - a)) {
-              first = a, limit = -3;
-            } else {
-              STACK_POP5(ISAd, first, last, limit, trlink);
-            }
-          }
-        } else {
-          STACK_POP5(ISAd, first, last, limit, trlink);
-        }
-      }
-      continue;
-    }
-
-    if((last - first) <= TR_INSERTIONSORT_THRESHOLD) {
-      tr_insertionsort(ISAd, first, last);
-      limit = -3;
-      continue;
-    }
-
-    if(limit-- == 0) {
-      tr_heapsort(ISAd, first, last - first);
-      for(a = last - 1; first < a; a = b) {
-        for(x = ISAd[*a], b = a - 1; (first <= b) && (ISAd[*b] == x); --b) { *b = ~*b; }
-      }
-      limit = -3;
-      continue;
-    }
-
-    /* choose pivot */
-    a = tr_pivot(ISAd, first, last);
-    SWAP(*first, *a);
-    v = ISAd[*first];
-
-    /* partition */
-    tr_partition(ISAd, first, first + 1, last, &a, &b, v);
-    if((last - first) != (b - a)) {
-      next = (ISA[*a] != v) ? tr_ilg(b - a) : -1;
-
-      /* update ranks */
-      for(c = first, v = a - SA - 1; c < a; ++c) { ISA[*c] = v; }
-      if(b < last) { for(c = a, v = b - SA - 1; c < b; ++c) { ISA[*c] = v; } }
-
-      /* push */
-      if((1 < (b - a)) && (trbudget_check(budget, b - a))) {
-        if((a - first) <= (last - b)) {
-          if((last - b) <= (b - a)) {
-            if(1 < (a - first)) {
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              STACK_PUSH5(ISAd, b, last, limit, trlink);
-              last = a;
-            } else if(1 < (last - b)) {
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              first = b;
-            } else {
-              ISAd += incr, first = a, last = b, limit = next;
-            }
-          } else if((a - first) <= (b - a)) {
-            if(1 < (a - first)) {
-              STACK_PUSH5(ISAd, b, last, limit, trlink);
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              last = a;
-            } else {
-              STACK_PUSH5(ISAd, b, last, limit, trlink);
-              ISAd += incr, first = a, last = b, limit = next;
-            }
-          } else {
-            STACK_PUSH5(ISAd, b, last, limit, trlink);
-            STACK_PUSH5(ISAd, first, a, limit, trlink);
-            ISAd += incr, first = a, last = b, limit = next;
-          }
-        } else {
-          if((a - first) <= (b - a)) {
-            if(1 < (last - b)) {
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              STACK_PUSH5(ISAd, first, a, limit, trlink);
-              first = b;
-            } else if(1 < (a - first)) {
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              last = a;
-            } else {
-              ISAd += incr, first = a, last = b, limit = next;
-            }
-          } else if((last - b) <= (b - a)) {
-            if(1 < (last - b)) {
-              STACK_PUSH5(ISAd, first, a, limit, trlink);
-              STACK_PUSH5(ISAd + incr, a, b, next, trlink);
-              first = b;
-            } else {
-              STACK_PUSH5(ISAd, first, a, limit, trlink);
-              ISAd += incr, first = a, last = b, limit = next;
-            }
-          } else {
-            STACK_PUSH5(ISAd, first, a, limit, trlink);
-            STACK_PUSH5(ISAd, b, last, limit, trlink);
-            ISAd += incr, first = a, last = b, limit = next;
-          }
-        }
-      } else {
-        if((1 < (b - a)) && (0 <= trlink)) { stack[trlink].d = -1; }
-        if((a - first) <= (last - b)) {
-          if(1 < (a - first)) {
-            STACK_PUSH5(ISAd, b, last, limit, trlink);
-            last = a;
-          } else if(1 < (last - b)) {
-            first = b;
-          } else {
-            STACK_POP5(ISAd, first, last, limit, trlink);
-          }
-        } else {
-          if(1 < (last - b)) {
-            STACK_PUSH5(ISAd, first, a, limit, trlink);
-            first = b;
-          } else if(1 < (a - first)) {
-            last = a;
-          } else {
-            STACK_POP5(ISAd, first, last, limit, trlink);
-          }
-        }
-      }
-    } else {
-      if(trbudget_check(budget, last - first)) {
-        limit = tr_ilg(last - first), ISAd += incr;
-      } else {
-        if(0 <= trlink) { stack[trlink].d = -1; }
-        STACK_POP5(ISAd, first, last, limit, trlink);
-      }
-    }
-  }
-#undef STACK_SIZE
-}
-
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Tandem repeat sort */
-static
-void
-trsort(int *ISA, int *SA, int n, int depth) {
-  int *ISAd;
-  int *first, *last;
-  trbudget_t budget;
-  int t, skip, unsorted;
-
-  trbudget_init(&budget, tr_ilg(n) * 2 / 3, n);
-/*  trbudget_init(&budget, tr_ilg(n) * 3 / 4, n); */
-  for(ISAd = ISA + depth; -n < *SA; ISAd += ISAd - ISA) {
-    first = SA;
-    skip = 0;
-    unsorted = 0;
-    do {
-      if((t = *first) < 0) { first -= t; skip += t; }
-      else {
-        if(skip != 0) { *(first + skip) = skip; skip = 0; }
-        last = SA + ISA[t] + 1;
-        if(1 < (last - first)) {
-          budget.count = 0;
-          tr_introsort(ISA, ISAd, SA, first, last, &budget);
-          if(budget.count != 0) { unsorted += budget.count; }
-          else { skip = first - last; }
-        } else if((last - first) == 1) {
-          skip = -1;
-        }
-        first = last;
-      }
-    } while(first < (SA + n));
-    if(skip != 0) { *(first + skip) = skip; }
-    if(unsorted == 0) { break; }
-  }
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/* Sorts suffixes of type B*. */
-static
-int
-sort_typeBstar(const unsigned char *T, int *SA,
-               int *bucket_A, int *bucket_B,
-               int n, int openMP) {
-  int *PAb, *ISAb, *buf;
-#ifdef LIBBSC_OPENMP
-  int *curbuf;
-  int l;
-#endif
-  int i, j, k, t, m, bufsize;
-  int c0, c1;
-#ifdef LIBBSC_OPENMP
-  int d0, d1;
-#endif
-  (void)openMP;
-
-  /* Initialize bucket arrays. */
-  for(i = 0; i < BUCKET_A_SIZE; ++i) { bucket_A[i] = 0; }
-  for(i = 0; i < BUCKET_B_SIZE; ++i) { bucket_B[i] = 0; }
-
-  /* Count the number of occurrences of the first one or two characters of each
-     type A, B and B* suffix. Moreover, store the beginning position of all
-     type B* suffixes into the array SA. */
-  for(i = n - 1, m = n, c0 = T[n - 1]; 0 <= i;) {
-    /* type A suffix. */
-    do { ++BUCKET_A(c1 = c0); } while((0 <= --i) && ((c0 = T[i]) >= c1));
-    if(0 <= i) {
-      /* type B* suffix. */
-      ++BUCKET_BSTAR(c0, c1);
-      SA[--m] = i;
-      /* type B suffix. */
-      for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) <= c1); --i, c1 = c0) {
-        ++BUCKET_B(c0, c1);
-      }
-    }
-  }
-  m = n - m;
-/*
-note:
-  A type B* suffix is lexicographically smaller than a type B suffix that
-  begins with the same first two characters.
-*/
-
-  /* Calculate the index of start/end point of each bucket. */
-  for(c0 = 0, i = 0, j = 0; c0 < ALPHABET_SIZE; ++c0) {
-    t = i + BUCKET_A(c0);
-    BUCKET_A(c0) = i + j; /* start point */
-    i = t + BUCKET_B(c0, c0);
-    for(c1 = c0 + 1; c1 < ALPHABET_SIZE; ++c1) {
-      j += BUCKET_BSTAR(c0, c1);
-      BUCKET_BSTAR(c0, c1) = j; /* end point */
-      i += BUCKET_B(c0, c1);
-    }
-  }
-
-  if(0 < m) {
-    /* Sort the type B* suffixes by their first two characters. */
-    PAb = SA + n - m; ISAb = SA + m;
-    for(i = m - 2; 0 <= i; --i) {
-      t = PAb[i], c0 = T[t], c1 = T[t + 1];
-      SA[--BUCKET_BSTAR(c0, c1)] = i;
-    }
-    t = PAb[m - 1], c0 = T[t], c1 = T[t + 1];
-    SA[--BUCKET_BSTAR(c0, c1)] = m - 1;
-
-    /* Sort the type B* substrings using sssort. */
-#ifdef LIBBSC_OPENMP
-    if (openMP)
-    {
-        buf = SA + m;
-        c0 = ALPHABET_SIZE - 2, c1 = ALPHABET_SIZE - 1, j = m;
-#pragma omp parallel default(shared) private(bufsize, curbuf, k, l, d0, d1)
-        {
-          bufsize = (n - (2 * m)) / omp_get_num_threads();
-          curbuf = buf + omp_get_thread_num() * bufsize;
-          k = 0;
-          for(;;) {
-            #pragma omp critical(sssort_lock)
-            {
-              if(0 < (l = j)) {
-                d0 = c0, d1 = c1;
-                do {
-                  k = BUCKET_BSTAR(d0, d1);
-                  if(--d1 <= d0) {
-                    d1 = ALPHABET_SIZE - 1;
-                    if(--d0 < 0) { break; }
-                  }
-                } while(((l - k) <= 1) && (0 < (l = k)));
-                c0 = d0, c1 = d1, j = k;
-              }
-            }
-            if(l == 0) { break; }
-            sssort(T, PAb, SA + k, SA + l,
-                   curbuf, bufsize, 2, n, *(SA + k) == (m - 1));
-          }
-        }
-    }
-    else
-    {
-        buf = SA + m, bufsize = n - (2 * m);
-        for(c0 = ALPHABET_SIZE - 2, j = m; 0 < j; --c0) {
-          for(c1 = ALPHABET_SIZE - 1; c0 < c1; j = i, --c1) {
-            i = BUCKET_BSTAR(c0, c1);
-            if(1 < (j - i)) {
-              sssort(T, PAb, SA + i, SA + j,
-                     buf, bufsize, 2, n, *(SA + i) == (m - 1));
-            }
-          }
-        }
-    }
-#else
-    buf = SA + m, bufsize = n - (2 * m);
-    for(c0 = ALPHABET_SIZE - 2, j = m; 0 < j; --c0) {
-      for(c1 = ALPHABET_SIZE - 1; c0 < c1; j = i, --c1) {
-        i = BUCKET_BSTAR(c0, c1);
-        if(1 < (j - i)) {
-          sssort(T, PAb, SA + i, SA + j,
-                 buf, bufsize, 2, n, *(SA + i) == (m - 1));
-        }
-      }
-    }
-#endif
-
-    /* Compute ranks of type B* substrings. */
-    for(i = m - 1; 0 <= i; --i) {
-      if(0 <= SA[i]) {
-        j = i;
-        do { ISAb[SA[i]] = i; } while((0 <= --i) && (0 <= SA[i]));
-        SA[i + 1] = i - j;
-        if(i <= 0) { break; }
-      }
-      j = i;
-      do { ISAb[SA[i] = ~SA[i]] = j; } while(SA[--i] < 0);
-      ISAb[SA[i]] = j;
-    }
-
-    /* Construct the inverse suffix array of type B* suffixes using trsort. */
-    trsort(ISAb, SA, m, 1);
-
-    /* Set the sorted order of tyoe B* suffixes. */
-    for(i = n - 1, j = m, c0 = T[n - 1]; 0 <= i;) {
-      for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) >= c1); --i, c1 = c0) { }
-      if(0 <= i) {
-        t = i;
-        for(--i, c1 = c0; (0 <= i) && ((c0 = T[i]) <= c1); --i, c1 = c0) { }
-        SA[ISAb[--j]] = ((t == 0) || (1 < (t - i))) ? t : ~t;
-      }
-    }
-
-    /* Calculate the index of start/end point of each bucket. */
-    BUCKET_B(ALPHABET_SIZE - 1, ALPHABET_SIZE - 1) = n; /* end point */
-    for(c0 = ALPHABET_SIZE - 2, k = m - 1; 0 <= c0; --c0) {
-      i = BUCKET_A(c0 + 1) - 1;
-      for(c1 = ALPHABET_SIZE - 1; c0 < c1; --c1) {
-        t = i - BUCKET_B(c0, c1);
-        BUCKET_B(c0, c1) = i; /* end point */
-
-        /* Move all type B* suffixes to the correct position. */
-        for(i = t, j = BUCKET_BSTAR(c0, c1);
-            j <= k;
-            --i, --k) { SA[i] = SA[k]; }
-      }
-      BUCKET_BSTAR(c0, c0 + 1) = i - BUCKET_B(c0, c0) + 1; /* start point */
-      BUCKET_B(c0, c0) = i; /* end point */
-    }
-  }
-
-  return m;
-}
-
-/* Constructs the suffix array by using the sorted order of type B* suffixes. */
-static
-void
-construct_SA(const unsigned char *T, int *SA,
-             int *bucket_A, int *bucket_B,
-             int n, int m) {
-  int *i, *j, *k;
-  int s;
-  int c0, c1, c2;
-
-  if(0 < m) {
-    /* Construct the sorted order of type B suffixes by using
-       the sorted order of type B* suffixes. */
-    for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
-      /* Scan the suffix array from right to left. */
-      for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
-          j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
-          i <= j;
-          --j) {
-        if(0 < (s = *j)) {
-          assert(T[s] == c1);
-          assert(((s + 1) < n) && (T[s] <= T[s + 1]));
-          assert(T[s - 1] <= T[s]);
-          *j = ~s;
-          c0 = T[--s];
-          if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
-          if(c0 != c2) {
-            if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
-            k = SA + BUCKET_B(c2 = c0, c1);
-          }
-          assert(k < j); assert(k != NULL);
-          *k-- = s;
-        } else {
-          assert(((s == 0) && (T[s] == c1)) || (s < 0));
-          *j = ~s;
-        }
-      }
-    }
-  }
-
-  /* Construct the suffix array by using
-     the sorted order of type B suffixes. */
-  k = SA + BUCKET_A(c2 = T[n - 1]);
-  *k++ = (T[n - 2] < c2) ? ~(n - 1) : (n - 1);
-  /* Scan the suffix array from left to right. */
-  for(i = SA, j = SA + n; i < j; ++i) {
-    if(0 < (s = *i)) {
-      assert(T[s - 1] >= T[s]);
-      c0 = T[--s];
-      if((s == 0) || (T[s - 1] < c0)) { s = ~s; }
-      if(c0 != c2) {
-        BUCKET_A(c2) = k - SA;
-        k = SA + BUCKET_A(c2 = c0);
-      }
-      assert(i < k);
-      *k++ = s;
-    } else {
-      assert(s < 0);
-      *i = ~s;
-    }
-  }
-}
-
-/* Constructs the burrows-wheeler transformed string directly
-   by using the sorted order of type B* suffixes. */
-static
-int
-construct_BWT(const unsigned char *T, int *SA,
-              int *bucket_A, int *bucket_B,
-              int n, int m) {
-  int *i, *j, *k, *orig;
-  int s;
-  int c0, c1, c2;
-
-  if(0 < m) {
-    /* Construct the sorted order of type B suffixes by using
-       the sorted order of type B* suffixes. */
-    for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
-      /* Scan the suffix array from right to left. */
-      for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
-          j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
-          i <= j;
-          --j) {
-        if(0 < (s = *j)) {
-          assert(T[s] == c1);
-          assert(((s + 1) < n) && (T[s] <= T[s + 1]));
-          assert(T[s - 1] <= T[s]);
-          c0 = T[--s];
-          *j = ~((int)c0);
-          if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
-          if(c0 != c2) {
-            if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
-            k = SA + BUCKET_B(c2 = c0, c1);
-          }
-          assert(k < j); assert(k != NULL);
-          *k-- = s;
-        } else if(s != 0) {
-          *j = ~s;
-#ifndef NDEBUG
-        } else {
-          assert(T[s] == c1);
-#endif
-        }
-      }
-    }
-  }
-
-  /* Construct the BWTed string by using
-     the sorted order of type B suffixes. */
-  k = SA + BUCKET_A(c2 = T[n - 1]);
-  *k++ = (T[n - 2] < c2) ? ~((int)T[n - 2]) : (n - 1);
-  /* Scan the suffix array from left to right. */
-  for(i = SA, j = SA + n, orig = SA; i < j; ++i) {
-    if(0 < (s = *i)) {
-      assert(T[s - 1] >= T[s]);
-      c0 = T[--s];
-      *i = c0;
-      if((0 < s) && (T[s - 1] < c0)) { s = ~((int)T[s - 1]); }
-      if(c0 != c2) {
-        BUCKET_A(c2) = k - SA;
-        k = SA + BUCKET_A(c2 = c0);
-      }
-      assert(i < k);
-      *k++ = s;
-    } else if(s != 0) {
-      *i = ~s;
-    } else {
-      orig = i;
-    }
-  }
-
-  return orig - SA;
-}
-
-/* Constructs the burrows-wheeler transformed string directly
-   by using the sorted order of type B* suffixes. */
-static
-int
-construct_BWT_indexes(const unsigned char *T, int *SA,
-                      int *bucket_A, int *bucket_B,
-                      int n, int m,
-                      unsigned char * num_indexes, int * indexes) {
-  int *i, *j, *k, *orig;
-  int s;
-  int c0, c1, c2;
-
-  int mod = n / 8;
-  {
-      mod |= mod >> 1;  mod |= mod >> 2;
-      mod |= mod >> 4;  mod |= mod >> 8;
-      mod |= mod >> 16; mod >>= 1;
-
-      *num_indexes = (unsigned char)((n - 1) / (mod + 1));
-  }
-
-  if(0 < m) {
-    /* Construct the sorted order of type B suffixes by using
-       the sorted order of type B* suffixes. */
-    for(c1 = ALPHABET_SIZE - 2; 0 <= c1; --c1) {
-      /* Scan the suffix array from right to left. */
-      for(i = SA + BUCKET_BSTAR(c1, c1 + 1),
-          j = SA + BUCKET_A(c1 + 1) - 1, k = NULL, c2 = -1;
-          i <= j;
-          --j) {
-        if(0 < (s = *j)) {
-          assert(T[s] == c1);
-          assert(((s + 1) < n) && (T[s] <= T[s + 1]));
-          assert(T[s - 1] <= T[s]);
-
-          if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = j - SA;
-
-          c0 = T[--s];
-          *j = ~((int)c0);
-          if((0 < s) && (T[s - 1] > c0)) { s = ~s; }
-          if(c0 != c2) {
-            if(0 <= c2) { BUCKET_B(c2, c1) = k - SA; }
-            k = SA + BUCKET_B(c2 = c0, c1);
-          }
-          assert(k < j); assert(k != NULL);
-          *k-- = s;
-        } else if(s != 0) {
-          *j = ~s;
-#ifndef NDEBUG
-        } else {
-          assert(T[s] == c1);
-#endif
-        }
-      }
-    }
-  }
-
-  /* Construct the BWTed string by using
-     the sorted order of type B suffixes. */
-  k = SA + BUCKET_A(c2 = T[n - 1]);
-  if (T[n - 2] < c2) {
-    if (((n - 1) & mod) == 0) indexes[(n - 1) / (mod + 1) - 1] = k - SA;
-    *k++ = ~((int)T[n - 2]);
-  }
-  else {
-    *k++ = n - 1;
-  }
-
-  /* Scan the suffix array from left to right. */
-  for(i = SA, j = SA + n, orig = SA; i < j; ++i) {
-    if(0 < (s = *i)) {
-      assert(T[s - 1] >= T[s]);
-
-      if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = i - SA;
-
-      c0 = T[--s];
-      *i = c0;
-      if(c0 != c2) {
-        BUCKET_A(c2) = k - SA;
-        k = SA + BUCKET_A(c2 = c0);
-      }
-      assert(i < k);
-      if((0 < s) && (T[s - 1] < c0)) {
-          if ((s & mod) == 0) indexes[s / (mod + 1) - 1] = k - SA;
-          *k++ = ~((int)T[s - 1]);
-      } else
-        *k++ = s;
-    } else if(s != 0) {
-      *i = ~s;
-    } else {
-      orig = i;
-    }
-  }
-
-  return orig - SA;
-}
-
-
-/*---------------------------------------------------------------------------*/
-
-/*- Function -*/
-
-int
-divsufsort(const unsigned char *T, int *SA, int n, int openMP) {
-  int *bucket_A, *bucket_B;
-  int m;
-  int err = 0;
-
-  /* Check arguments. */
-  if((T == NULL) || (SA == NULL) || (n < 0)) { return -1; }
-  else if(n == 0) { return 0; }
-  else if(n == 1) { SA[0] = 0; return 0; }
-  else if(n == 2) { m = (T[0] < T[1]); SA[m ^ 1] = 0, SA[m] = 1; return 0; }
-
-  bucket_A = (int *)malloc(BUCKET_A_SIZE * sizeof(int));
-  bucket_B = (int *)malloc(BUCKET_B_SIZE * sizeof(int));
-
-  /* Suffixsort. */
-  if((bucket_A != NULL) && (bucket_B != NULL)) {
-    m = sort_typeBstar(T, SA, bucket_A, bucket_B, n, openMP);
-    construct_SA(T, SA, bucket_A, bucket_B, n, m);
-  } else {
-    err = -2;
-  }
-
-  free(bucket_B);
-  free(bucket_A);
-
-  return err;
-}
-
-int
-divbwt(const unsigned char *T, unsigned char *U, int *A, int n, unsigned char * num_indexes, int * indexes, int openMP) {
-  int *B;
-  int *bucket_A, *bucket_B;
-  int m, pidx, i;
-
-  /* Check arguments. */
-  if((T == NULL) || (U == NULL) || (n < 0)) { return -1; }
-  else if(n <= 1) { if(n == 1) { U[0] = T[0]; } return n; }
-
-  if((B = A) == NULL) { B = (int *)malloc((size_t)(n + 1) * sizeof(int)); }
-  bucket_A = (int *)malloc(BUCKET_A_SIZE * sizeof(int));
-  bucket_B = (int *)malloc(BUCKET_B_SIZE * sizeof(int));
-
-  /* Burrows-Wheeler Transform. */
-  if((B != NULL) && (bucket_A != NULL) && (bucket_B != NULL)) {
-    m = sort_typeBstar(T, B, bucket_A, bucket_B, n, openMP);
-
-    if (num_indexes == NULL || indexes == NULL) {
-        pidx = construct_BWT(T, B, bucket_A, bucket_B, n, m);
-    } else {
-        pidx = construct_BWT_indexes(T, B, bucket_A, bucket_B, n, m, num_indexes, indexes);
-    }
-
-    /* Copy to output string. */
-    U[0] = T[n - 1];
-    for(i = 0; i < pidx; ++i) { U[i + 1] = (unsigned char)B[i]; }
-    for(i += 1; i < n; ++i) { U[i] = (unsigned char)B[i]; }
-    pidx += 1;
-  } else {
-    pidx = -2;
-  }
-
-  free(bucket_B);
-  free(bucket_A);
-  if(A == NULL) { free(B); }
-
-  return pidx;
-}
diff --git a/vendor/github.com/DataDog/zstd/divsufsort.h b/vendor/github.com/DataDog/zstd/divsufsort.h
deleted file mode 100644
index 5440994..0000000
--- a/vendor/github.com/DataDog/zstd/divsufsort.h
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * divsufsort.h for libdivsufsort-lite
- * Copyright (c) 2003-2008 Yuta Mori All Rights Reserved.
- *
- * 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.
- */
-
-#ifndef _DIVSUFSORT_H
-#define _DIVSUFSORT_H 1
-
-#ifdef __cplusplus
-extern "C" {
-#endif /* __cplusplus */
-
-
-/*- Prototypes -*/
-
-/**
- * Constructs the suffix array of a given string.
- * @param T [0..n-1] The input string.
- * @param SA [0..n-1] The output array of suffixes.
- * @param n The length of the given string.
- * @param openMP enables OpenMP optimization.
- * @return 0 if no error occurred, -1 or -2 otherwise.
- */
-int
-divsufsort(const unsigned char *T, int *SA, int n, int openMP);
-
-/**
- * Constructs the burrows-wheeler transformed string of a given string.
- * @param T [0..n-1] The input string.
- * @param U [0..n-1] The output string. (can be T)
- * @param A [0..n-1] The temporary array. (can be NULL)
- * @param n The length of the given string.
- * @param num_indexes The length of secondary indexes array. (can be NULL)
- * @param indexes The secondary indexes array. (can be NULL)
- * @param openMP enables OpenMP optimization.
- * @return The primary index if no error occurred, -1 or -2 otherwise.
- */
-int
-divbwt(const unsigned char *T, unsigned char *U, int *A, int n, unsigned char * num_indexes, int * indexes, int openMP);
-
-
-#ifdef __cplusplus
-} /* extern "C" */
-#endif /* __cplusplus */
-
-#endif /* _DIVSUFSORT_H */
diff --git a/vendor/github.com/DataDog/zstd/entropy_common.c b/vendor/github.com/DataDog/zstd/entropy_common.c
deleted file mode 100644
index b12944e..0000000
--- a/vendor/github.com/DataDog/zstd/entropy_common.c
+++ /dev/null
@@ -1,236 +0,0 @@
-/*
-   Common functions of New Generation Entropy library
-   Copyright (C) 2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-*************************************************************************** */
-
-/* *************************************
-*  Dependencies
-***************************************/
-#include "mem.h"
-#include "error_private.h"       /* ERR_*, ERROR */
-#define FSE_STATIC_LINKING_ONLY  /* FSE_MIN_TABLELOG */
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY  /* HUF_TABLELOG_ABSOLUTEMAX */
-#include "huf.h"
-
-
-/*===   Version   ===*/
-unsigned FSE_versionNumber(void) { return FSE_VERSION_NUMBER; }
-
-
-/*===   Error Management   ===*/
-unsigned FSE_isError(size_t code) { return ERR_isError(code); }
-const char* FSE_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-unsigned HUF_isError(size_t code) { return ERR_isError(code); }
-const char* HUF_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/*-**************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) {
-        /* This function only works when hbSize >= 4 */
-        char buffer[4];
-        memset(buffer, 0, sizeof(buffer));
-        memcpy(buffer, headerBuffer, hbSize);
-        {   size_t const countSize = FSE_readNCount(normalizedCounter, maxSVPtr, tableLogPtr,
-                                                    buffer, sizeof(buffer));
-            if (FSE_isError(countSize)) return countSize;
-            if (countSize > hbSize) return ERROR(corruption_detected);
-            return countSize;
-    }   }
-    assert(hbSize >= 4);
-
-    /* init */
-    memset(normalizedCounter, 0, (*maxSVPtr+1) * sizeof(normalizedCounter[0]));   /* all symbols not present in NCount have a frequency of 0 */
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) & (charnum<=*maxSVPtr)) {
-        if (previous0) {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF) {
-                n0 += 24;
-                if (ip < iend-5) {
-                    ip += 2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                } else {
-                    bitStream >>= 16;
-                    bitCount   += 16;
-            }   }
-            while ((bitStream & 3) == 3) {
-                n0 += 3;
-                bitStream >>= 2;
-                bitCount += 2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                assert((bitCount >> 3) <= 3); /* For first condition to work */
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            } else {
-                bitStream >>= 2;
-        }   }
-        {   int const max = (2*threshold-1) - remaining;
-            int count;
-
-            if ((bitStream & (threshold-1)) < (U32)max) {
-                count = bitStream & (threshold-1);
-                bitCount += nbBits-1;
-            } else {
-                count = bitStream & (2*threshold-1);
-                if (count >= threshold) count -= max;
-                bitCount += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= count < 0 ? -count : count;   /* -1 means +1 */
-            normalizedCounter[charnum++] = (short)count;
-            previous0 = !count;
-            while (remaining < threshold) {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-            } else {
-                bitCount -= (int)(8 * (iend - 4 - ip));
-                ip = iend - 4;
-            }
-            bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-    }   }   /* while ((remaining>1) & (charnum<=*maxSVPtr)) */
-    if (remaining != 1) return ERROR(corruption_detected);
-    if (bitCount > 32) return ERROR(corruption_detected);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    return ip-istart;
-}
-
-
-/*! HUF_readStats() :
-    Read compact Huffman tree, saved by HUF_writeCTable().
-    `huffWeight` is destination buffer.
-    `rankStats` is assumed to be a table of at least HUF_TABLELOG_MAX U32.
-    @return : size read from `src` , or an error Code .
-    Note : Needed by HUF_readCTable() and HUF_readDTableX?() .
-*/
-size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                     U32* nbSymbolsPtr, U32* tableLogPtr,
-                     const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    /* memset(huffWeight, 0, hwSize);   *//* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128) {  /* special header */
-        oSize = iSize - 127;
-        iSize = ((oSize+1)/2);
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        if (oSize >= hwSize) return ERROR(corruption_detected);
-        ip += 1;
-        {   U32 n;
-            for (n=0; n<oSize; n+=2) {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-    }   }   }
-    else  {   /* header compressed with FSE (normal case) */
-        FSE_DTable fseWorkspace[FSE_DTABLE_SIZE_U32(6)];  /* 6 is max possible tableLog for HUF header (maybe even 5, to be tested) */
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSE_decompress_wksp(huffWeight, hwSize-1, ip+1, iSize, fseWorkspace, 6);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSE_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUF_TABLELOG_MAX + 1) * sizeof(U32));
-    weightTotal = 0;
-    {   U32 n; for (n=0; n<oSize; n++) {
-            if (huffWeight[n] >= HUF_TABLELOG_MAX) return ERROR(corruption_detected);
-            rankStats[huffWeight[n]]++;
-            weightTotal += (1 << huffWeight[n]) >> 1;
-    }   }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    {   U32 const tableLog = BIT_highbit32(weightTotal) + 1;
-        if (tableLog > HUF_TABLELOG_MAX) return ERROR(corruption_detected);
-        *tableLogPtr = tableLog;
-        /* determine last weight */
-        {   U32 const total = 1 << tableLog;
-            U32 const rest = total - weightTotal;
-            U32 const verif = 1 << BIT_highbit32(rest);
-            U32 const lastWeight = BIT_highbit32(rest) + 1;
-            if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-            huffWeight[oSize] = (BYTE)lastWeight;
-            rankStats[lastWeight]++;
-    }   }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    return iSize+1;
-}
diff --git a/vendor/github.com/DataDog/zstd/error_private.c b/vendor/github.com/DataDog/zstd/error_private.c
deleted file mode 100644
index 7c1bb67..0000000
--- a/vendor/github.com/DataDog/zstd/error_private.c
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* The purpose of this file is to have a single list of error strings embedded in binary */
-
-#include "error_private.h"
-
-const char* ERR_getErrorString(ERR_enum code)
-{
-#ifdef ZSTD_STRIP_ERROR_STRINGS
-    (void)code;
-    return "Error strings stripped";
-#else
-    static const char* const notErrorCode = "Unspecified error code";
-    switch( code )
-    {
-    case PREFIX(no_error): return "No error detected";
-    case PREFIX(GENERIC):  return "Error (generic)";
-    case PREFIX(prefix_unknown): return "Unknown frame descriptor";
-    case PREFIX(version_unsupported): return "Version not supported";
-    case PREFIX(frameParameter_unsupported): return "Unsupported frame parameter";
-    case PREFIX(frameParameter_windowTooLarge): return "Frame requires too much memory for decoding";
-    case PREFIX(corruption_detected): return "Corrupted block detected";
-    case PREFIX(checksum_wrong): return "Restored data doesn't match checksum";
-    case PREFIX(parameter_unsupported): return "Unsupported parameter";
-    case PREFIX(parameter_outOfBound): return "Parameter is out of bound";
-    case PREFIX(init_missing): return "Context should be init first";
-    case PREFIX(memory_allocation): return "Allocation error : not enough memory";
-    case PREFIX(workSpace_tooSmall): return "workSpace buffer is not large enough";
-    case PREFIX(stage_wrong): return "Operation not authorized at current processing stage";
-    case PREFIX(tableLog_tooLarge): return "tableLog requires too much memory : unsupported";
-    case PREFIX(maxSymbolValue_tooLarge): return "Unsupported max Symbol Value : too large";
-    case PREFIX(maxSymbolValue_tooSmall): return "Specified maxSymbolValue is too small";
-    case PREFIX(dictionary_corrupted): return "Dictionary is corrupted";
-    case PREFIX(dictionary_wrong): return "Dictionary mismatch";
-    case PREFIX(dictionaryCreation_failed): return "Cannot create Dictionary from provided samples";
-    case PREFIX(dstSize_tooSmall): return "Destination buffer is too small";
-    case PREFIX(srcSize_wrong): return "Src size is incorrect";
-    case PREFIX(dstBuffer_null): return "Operation on NULL destination buffer";
-        /* following error codes are not stable and may be removed or changed in a future version */
-    case PREFIX(frameIndex_tooLarge): return "Frame index is too large";
-    case PREFIX(seekableIO): return "An I/O error occurred when reading/seeking";
-    case PREFIX(maxCode):
-    default: return notErrorCode;
-    }
-#endif
-}
diff --git a/vendor/github.com/DataDog/zstd/error_private.h b/vendor/github.com/DataDog/zstd/error_private.h
deleted file mode 100644
index 0d2fa7e..0000000
--- a/vendor/github.com/DataDog/zstd/error_private.h
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* Note : this module is expected to remain private, do not expose it */
-
-#ifndef ERROR_H_MODULE
-#define ERROR_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* ****************************************
-*  Dependencies
-******************************************/
-#include <stddef.h>        /* size_t */
-#include "zstd_errors.h"  /* enum list */
-
-
-/* ****************************************
-*  Compiler-specific
-******************************************/
-#if defined(__GNUC__)
-#  define ERR_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define ERR_STATIC static inline
-#elif defined(_MSC_VER)
-#  define ERR_STATIC static __inline
-#else
-#  define ERR_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/*-****************************************
-*  Customization (error_public.h)
-******************************************/
-typedef ZSTD_ErrorCode ERR_enum;
-#define PREFIX(name) ZSTD_error_##name
-
-
-/*-****************************************
-*  Error codes handling
-******************************************/
-#undef ERROR   /* reported already defined on VS 2015 (Rich Geldreich) */
-#define ERROR(name) ZSTD_ERROR(name)
-#define ZSTD_ERROR(name) ((size_t)-PREFIX(name))
-
-ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); }
-
-ERR_STATIC ERR_enum ERR_getErrorCode(size_t code) { if (!ERR_isError(code)) return (ERR_enum)0; return (ERR_enum) (0-code); }
-
-
-/*-****************************************
-*  Error Strings
-******************************************/
-
-const char* ERR_getErrorString(ERR_enum code);   /* error_private.c */
-
-ERR_STATIC const char* ERR_getErrorName(size_t code)
-{
-    return ERR_getErrorString(ERR_getErrorCode(code));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ERROR_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/errors.go b/vendor/github.com/DataDog/zstd/errors.go
deleted file mode 100644
index 38db0d5..0000000
--- a/vendor/github.com/DataDog/zstd/errors.go
+++ /dev/null
@@ -1,35 +0,0 @@
-package zstd
-
-/*
-#define ZSTD_STATIC_LINKING_ONLY
-#include "zstd.h"
-*/
-import "C"
-
-// ErrorCode is an error returned by the zstd library.
-type ErrorCode int
-
-// Error returns the error string given by zstd
-func (e ErrorCode) Error() string {
-	return C.GoString(C.ZSTD_getErrorName(C.size_t(e)))
-}
-
-func cIsError(code int) bool {
-	return int(C.ZSTD_isError(C.size_t(code))) != 0
-}
-
-// getError returns an error for the return code, or nil if it's not an error
-func getError(code int) error {
-	if code < 0 && cIsError(code) {
-		return ErrorCode(code)
-	}
-	return nil
-}
-
-// IsDstSizeTooSmallError returns whether the error correspond to zstd standard sDstSizeTooSmall error
-func IsDstSizeTooSmallError(e error) bool {
-	if e != nil && e.Error() == "Destination buffer is too small" {
-		return true
-	}
-	return false
-}
diff --git a/vendor/github.com/DataDog/zstd/fastcover.c b/vendor/github.com/DataDog/zstd/fastcover.c
deleted file mode 100644
index 941bb5a..0000000
--- a/vendor/github.com/DataDog/zstd/fastcover.c
+++ /dev/null
@@ -1,747 +0,0 @@
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <stdio.h>  /* fprintf */
-#include <stdlib.h> /* malloc, free, qsort */
-#include <string.h> /* memset */
-#include <time.h>   /* clock */
-
-#include "mem.h" /* read */
-#include "pool.h"
-#include "threading.h"
-#include "cover.h"
-#include "zstd_internal.h" /* includes zstd.h */
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define FASTCOVER_MAX_SAMPLES_SIZE (sizeof(size_t) == 8 ? ((unsigned)-1) : ((unsigned)1 GB))
-#define FASTCOVER_MAX_F 31
-#define FASTCOVER_MAX_ACCEL 10
-#define DEFAULT_SPLITPOINT 0.75
-#define DEFAULT_F 20
-#define DEFAULT_ACCEL 1
-
-
-/*-*************************************
-*  Console display
-***************************************/
-static int g_displayLevel = 2;
-#define DISPLAY(...)                                                           \
-  {                                                                            \
-    fprintf(stderr, __VA_ARGS__);                                              \
-    fflush(stderr);                                                            \
-  }
-#define LOCALDISPLAYLEVEL(displayLevel, l, ...)                                \
-  if (displayLevel >= l) {                                                     \
-    DISPLAY(__VA_ARGS__);                                                      \
-  } /* 0 : no display;   1: errors;   2: default;  3: details;  4: debug */
-#define DISPLAYLEVEL(l, ...) LOCALDISPLAYLEVEL(g_displayLevel, l, __VA_ARGS__)
-
-#define LOCALDISPLAYUPDATE(displayLevel, l, ...)                               \
-  if (displayLevel >= l) {                                                     \
-    if ((clock() - g_time > refreshRate) || (displayLevel >= 4)) {             \
-      g_time = clock();                                                        \
-      DISPLAY(__VA_ARGS__);                                                    \
-    }                                                                          \
-  }
-#define DISPLAYUPDATE(l, ...) LOCALDISPLAYUPDATE(g_displayLevel, l, __VA_ARGS__)
-static const clock_t refreshRate = CLOCKS_PER_SEC * 15 / 100;
-static clock_t g_time = 0;
-
-
-/*-*************************************
-* Hash Functions
-***************************************/
-static const U64 prime6bytes = 227718039650203ULL;
-static size_t ZSTD_hash6(U64 u, U32 h) { return (size_t)(((u  << (64-48)) * prime6bytes) >> (64-h)) ; }
-static size_t ZSTD_hash6Ptr(const void* p, U32 h) { return ZSTD_hash6(MEM_readLE64(p), h); }
-
-static const U64 prime8bytes = 0xCF1BBCDCB7A56463ULL;
-static size_t ZSTD_hash8(U64 u, U32 h) { return (size_t)(((u) * prime8bytes) >> (64-h)) ; }
-static size_t ZSTD_hash8Ptr(const void* p, U32 h) { return ZSTD_hash8(MEM_readLE64(p), h); }
-
-
-/**
- * Hash the d-byte value pointed to by p and mod 2^f
- */
-static size_t FASTCOVER_hashPtrToIndex(const void* p, U32 h, unsigned d) {
-  if (d == 6) {
-    return ZSTD_hash6Ptr(p, h) & ((1 << h) - 1);
-  }
-  return ZSTD_hash8Ptr(p, h) & ((1 << h) - 1);
-}
-
-
-/*-*************************************
-* Acceleration
-***************************************/
-typedef struct {
-  unsigned finalize;    /* Percentage of training samples used for ZDICT_finalizeDictionary */
-  unsigned skip;        /* Number of dmer skipped between each dmer counted in computeFrequency */
-} FASTCOVER_accel_t;
-
-
-static const FASTCOVER_accel_t FASTCOVER_defaultAccelParameters[FASTCOVER_MAX_ACCEL+1] = {
-  { 100, 0 },   /* accel = 0, should not happen because accel = 0 defaults to accel = 1 */
-  { 100, 0 },   /* accel = 1 */
-  { 50, 1 },   /* accel = 2 */
-  { 34, 2 },   /* accel = 3 */
-  { 25, 3 },   /* accel = 4 */
-  { 20, 4 },   /* accel = 5 */
-  { 17, 5 },   /* accel = 6 */
-  { 14, 6 },   /* accel = 7 */
-  { 13, 7 },   /* accel = 8 */
-  { 11, 8 },   /* accel = 9 */
-  { 10, 9 },   /* accel = 10 */
-};
-
-
-/*-*************************************
-* Context
-***************************************/
-typedef struct {
-  const BYTE *samples;
-  size_t *offsets;
-  const size_t *samplesSizes;
-  size_t nbSamples;
-  size_t nbTrainSamples;
-  size_t nbTestSamples;
-  size_t nbDmers;
-  U32 *freqs;
-  unsigned d;
-  unsigned f;
-  FASTCOVER_accel_t accelParams;
-} FASTCOVER_ctx_t;
-
-
-/*-*************************************
-*  Helper functions
-***************************************/
-/**
- * Selects the best segment in an epoch.
- * Segments of are scored according to the function:
- *
- * Let F(d) be the frequency of all dmers with hash value d.
- * Let S_i be hash value of the dmer at position i of segment S which has length k.
- *
- *     Score(S) = F(S_1) + F(S_2) + ... + F(S_{k-d+1})
- *
- * Once the dmer with hash value d is in the dictionary we set F(d) = 0.
- */
-static COVER_segment_t FASTCOVER_selectSegment(const FASTCOVER_ctx_t *ctx,
-                                              U32 *freqs, U32 begin, U32 end,
-                                              ZDICT_cover_params_t parameters,
-                                              U16* segmentFreqs) {
-  /* Constants */
-  const U32 k = parameters.k;
-  const U32 d = parameters.d;
-  const U32 f = ctx->f;
-  const U32 dmersInK = k - d + 1;
-
-  /* Try each segment (activeSegment) and save the best (bestSegment) */
-  COVER_segment_t bestSegment = {0, 0, 0};
-  COVER_segment_t activeSegment;
-
-  /* Reset the activeDmers in the segment */
-  /* The activeSegment starts at the beginning of the epoch. */
-  activeSegment.begin = begin;
-  activeSegment.end = begin;
-  activeSegment.score = 0;
-
-  /* Slide the activeSegment through the whole epoch.
-   * Save the best segment in bestSegment.
-   */
-  while (activeSegment.end < end) {
-    /* Get hash value of current dmer */
-    const size_t idx = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.end, f, d);
-
-    /* Add frequency of this index to score if this is the first occurrence of index in active segment */
-    if (segmentFreqs[idx] == 0) {
-      activeSegment.score += freqs[idx];
-    }
-    /* Increment end of segment and segmentFreqs*/
-    activeSegment.end += 1;
-    segmentFreqs[idx] += 1;
-    /* If the window is now too large, drop the first position */
-    if (activeSegment.end - activeSegment.begin == dmersInK + 1) {
-      /* Get hash value of the dmer to be eliminated from active segment */
-      const size_t delIndex = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.begin, f, d);
-      segmentFreqs[delIndex] -= 1;
-      /* Subtract frequency of this index from score if this is the last occurrence of this index in active segment */
-      if (segmentFreqs[delIndex] == 0) {
-        activeSegment.score -= freqs[delIndex];
-      }
-      /* Increment start of segment */
-      activeSegment.begin += 1;
-    }
-
-    /* If this segment is the best so far save it */
-    if (activeSegment.score > bestSegment.score) {
-      bestSegment = activeSegment;
-    }
-  }
-
-  /* Zero out rest of segmentFreqs array */
-  while (activeSegment.begin < end) {
-    const size_t delIndex = FASTCOVER_hashPtrToIndex(ctx->samples + activeSegment.begin, f, d);
-    segmentFreqs[delIndex] -= 1;
-    activeSegment.begin += 1;
-  }
-
-  {
-    /*  Zero the frequency of hash value of each dmer covered by the chosen segment. */
-    U32 pos;
-    for (pos = bestSegment.begin; pos != bestSegment.end; ++pos) {
-      const size_t i = FASTCOVER_hashPtrToIndex(ctx->samples + pos, f, d);
-      freqs[i] = 0;
-    }
-  }
-
-  return bestSegment;
-}
-
-
-static int FASTCOVER_checkParameters(ZDICT_cover_params_t parameters,
-                                     size_t maxDictSize, unsigned f,
-                                     unsigned accel) {
-  /* k, d, and f are required parameters */
-  if (parameters.d == 0 || parameters.k == 0) {
-    return 0;
-  }
-  /* d has to be 6 or 8 */
-  if (parameters.d != 6 && parameters.d != 8) {
-    return 0;
-  }
-  /* k <= maxDictSize */
-  if (parameters.k > maxDictSize) {
-    return 0;
-  }
-  /* d <= k */
-  if (parameters.d > parameters.k) {
-    return 0;
-  }
-  /* 0 < f <= FASTCOVER_MAX_F*/
-  if (f > FASTCOVER_MAX_F || f == 0) {
-    return 0;
-  }
-  /* 0 < splitPoint <= 1 */
-  if (parameters.splitPoint <= 0 || parameters.splitPoint > 1) {
-    return 0;
-  }
-  /* 0 < accel <= 10 */
-  if (accel > 10 || accel == 0) {
-    return 0;
-  }
-  return 1;
-}
-
-
-/**
- * Clean up a context initialized with `FASTCOVER_ctx_init()`.
- */
-static void
-FASTCOVER_ctx_destroy(FASTCOVER_ctx_t* ctx)
-{
-    if (!ctx) return;
-
-    free(ctx->freqs);
-    ctx->freqs = NULL;
-
-    free(ctx->offsets);
-    ctx->offsets = NULL;
-}
-
-
-/**
- * Calculate for frequency of hash value of each dmer in ctx->samples
- */
-static void
-FASTCOVER_computeFrequency(U32* freqs, const FASTCOVER_ctx_t* ctx)
-{
-    const unsigned f = ctx->f;
-    const unsigned d = ctx->d;
-    const unsigned skip = ctx->accelParams.skip;
-    const unsigned readLength = MAX(d, 8);
-    size_t i;
-    assert(ctx->nbTrainSamples >= 5);
-    assert(ctx->nbTrainSamples <= ctx->nbSamples);
-    for (i = 0; i < ctx->nbTrainSamples; i++) {
-        size_t start = ctx->offsets[i];  /* start of current dmer */
-        size_t const currSampleEnd = ctx->offsets[i+1];
-        while (start + readLength <= currSampleEnd) {
-            const size_t dmerIndex = FASTCOVER_hashPtrToIndex(ctx->samples + start, f, d);
-            freqs[dmerIndex]++;
-            start = start + skip + 1;
-        }
-    }
-}
-
-
-/**
- * Prepare a context for dictionary building.
- * The context is only dependent on the parameter `d` and can used multiple
- * times.
- * Returns 0 on success or error code on error.
- * The context must be destroyed with `FASTCOVER_ctx_destroy()`.
- */
-static size_t
-FASTCOVER_ctx_init(FASTCOVER_ctx_t* ctx,
-                   const void* samplesBuffer,
-                   const size_t* samplesSizes, unsigned nbSamples,
-                   unsigned d, double splitPoint, unsigned f,
-                   FASTCOVER_accel_t accelParams)
-{
-    const BYTE* const samples = (const BYTE*)samplesBuffer;
-    const size_t totalSamplesSize = COVER_sum(samplesSizes, nbSamples);
-    /* Split samples into testing and training sets */
-    const unsigned nbTrainSamples = splitPoint < 1.0 ? (unsigned)((double)nbSamples * splitPoint) : nbSamples;
-    const unsigned nbTestSamples = splitPoint < 1.0 ? nbSamples - nbTrainSamples : nbSamples;
-    const size_t trainingSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes, nbTrainSamples) : totalSamplesSize;
-    const size_t testSamplesSize = splitPoint < 1.0 ? COVER_sum(samplesSizes + nbTrainSamples, nbTestSamples) : totalSamplesSize;
-
-    /* Checks */
-    if (totalSamplesSize < MAX(d, sizeof(U64)) ||
-        totalSamplesSize >= (size_t)FASTCOVER_MAX_SAMPLES_SIZE) {
-        DISPLAYLEVEL(1, "Total samples size is too large (%u MB), maximum size is %u MB\n",
-                    (unsigned)(totalSamplesSize >> 20), (FASTCOVER_MAX_SAMPLES_SIZE >> 20));
-        return ERROR(srcSize_wrong);
-    }
-
-    /* Check if there are at least 5 training samples */
-    if (nbTrainSamples < 5) {
-        DISPLAYLEVEL(1, "Total number of training samples is %u and is invalid\n", nbTrainSamples);
-        return ERROR(srcSize_wrong);
-    }
-
-    /* Check if there's testing sample */
-    if (nbTestSamples < 1) {
-        DISPLAYLEVEL(1, "Total number of testing samples is %u and is invalid.\n", nbTestSamples);
-        return ERROR(srcSize_wrong);
-    }
-
-    /* Zero the context */
-    memset(ctx, 0, sizeof(*ctx));
-    DISPLAYLEVEL(2, "Training on %u samples of total size %u\n", nbTrainSamples,
-                    (unsigned)trainingSamplesSize);
-    DISPLAYLEVEL(2, "Testing on %u samples of total size %u\n", nbTestSamples,
-                    (unsigned)testSamplesSize);
-
-    ctx->samples = samples;
-    ctx->samplesSizes = samplesSizes;
-    ctx->nbSamples = nbSamples;
-    ctx->nbTrainSamples = nbTrainSamples;
-    ctx->nbTestSamples = nbTestSamples;
-    ctx->nbDmers = trainingSamplesSize - MAX(d, sizeof(U64)) + 1;
-    ctx->d = d;
-    ctx->f = f;
-    ctx->accelParams = accelParams;
-
-    /* The offsets of each file */
-    ctx->offsets = (size_t*)calloc((nbSamples + 1), sizeof(size_t));
-    if (ctx->offsets == NULL) {
-        DISPLAYLEVEL(1, "Failed to allocate scratch buffers \n");
-        FASTCOVER_ctx_destroy(ctx);
-        return ERROR(memory_allocation);
-    }
-
-    /* Fill offsets from the samplesSizes */
-    {   U32 i;
-        ctx->offsets[0] = 0;
-        assert(nbSamples >= 5);
-        for (i = 1; i <= nbSamples; ++i) {
-            ctx->offsets[i] = ctx->offsets[i - 1] + samplesSizes[i - 1];
-        }
-    }
-
-    /* Initialize frequency array of size 2^f */
-    ctx->freqs = (U32*)calloc(((U64)1 << f), sizeof(U32));
-    if (ctx->freqs == NULL) {
-        DISPLAYLEVEL(1, "Failed to allocate frequency table \n");
-        FASTCOVER_ctx_destroy(ctx);
-        return ERROR(memory_allocation);
-    }
-
-    DISPLAYLEVEL(2, "Computing frequencies\n");
-    FASTCOVER_computeFrequency(ctx->freqs, ctx);
-
-    return 0;
-}
-
-
-/**
- * Given the prepared context build the dictionary.
- */
-static size_t
-FASTCOVER_buildDictionary(const FASTCOVER_ctx_t* ctx,
-                          U32* freqs,
-                          void* dictBuffer, size_t dictBufferCapacity,
-                          ZDICT_cover_params_t parameters,
-                          U16* segmentFreqs)
-{
-  BYTE *const dict = (BYTE *)dictBuffer;
-  size_t tail = dictBufferCapacity;
-  /* Divide the data into epochs. We will select one segment from each epoch. */
-  const COVER_epoch_info_t epochs = COVER_computeEpochs(
-      (U32)dictBufferCapacity, (U32)ctx->nbDmers, parameters.k, 1);
-  const size_t maxZeroScoreRun = 10;
-  size_t zeroScoreRun = 0;
-  size_t epoch;
-  DISPLAYLEVEL(2, "Breaking content into %u epochs of size %u\n",
-                (U32)epochs.num, (U32)epochs.size);
-  /* Loop through the epochs until there are no more segments or the dictionary
-   * is full.
-   */
-  for (epoch = 0; tail > 0; epoch = (epoch + 1) % epochs.num) {
-    const U32 epochBegin = (U32)(epoch * epochs.size);
-    const U32 epochEnd = epochBegin + epochs.size;
-    size_t segmentSize;
-    /* Select a segment */
-    COVER_segment_t segment = FASTCOVER_selectSegment(
-        ctx, freqs, epochBegin, epochEnd, parameters, segmentFreqs);
-
-    /* If the segment covers no dmers, then we are out of content.
-     * There may be new content in other epochs, for continue for some time.
-     */
-    if (segment.score == 0) {
-      if (++zeroScoreRun >= maxZeroScoreRun) {
-          break;
-      }
-      continue;
-    }
-    zeroScoreRun = 0;
-
-    /* Trim the segment if necessary and if it is too small then we are done */
-    segmentSize = MIN(segment.end - segment.begin + parameters.d - 1, tail);
-    if (segmentSize < parameters.d) {
-      break;
-    }
-
-    /* We fill the dictionary from the back to allow the best segments to be
-     * referenced with the smallest offsets.
-     */
-    tail -= segmentSize;
-    memcpy(dict + tail, ctx->samples + segment.begin, segmentSize);
-    DISPLAYUPDATE(
-        2, "\r%u%%       ",
-        (unsigned)(((dictBufferCapacity - tail) * 100) / dictBufferCapacity));
-  }
-  DISPLAYLEVEL(2, "\r%79s\r", "");
-  return tail;
-}
-
-/**
- * Parameters for FASTCOVER_tryParameters().
- */
-typedef struct FASTCOVER_tryParameters_data_s {
-    const FASTCOVER_ctx_t* ctx;
-    COVER_best_t* best;
-    size_t dictBufferCapacity;
-    ZDICT_cover_params_t parameters;
-} FASTCOVER_tryParameters_data_t;
-
-
-/**
- * Tries a set of parameters and updates the COVER_best_t with the results.
- * This function is thread safe if zstd is compiled with multithreaded support.
- * It takes its parameters as an *OWNING* opaque pointer to support threading.
- */
-static void FASTCOVER_tryParameters(void *opaque)
-{
-  /* Save parameters as local variables */
-  FASTCOVER_tryParameters_data_t *const data = (FASTCOVER_tryParameters_data_t *)opaque;
-  const FASTCOVER_ctx_t *const ctx = data->ctx;
-  const ZDICT_cover_params_t parameters = data->parameters;
-  size_t dictBufferCapacity = data->dictBufferCapacity;
-  size_t totalCompressedSize = ERROR(GENERIC);
-  /* Initialize array to keep track of frequency of dmer within activeSegment */
-  U16* segmentFreqs = (U16 *)calloc(((U64)1 << ctx->f), sizeof(U16));
-  /* Allocate space for hash table, dict, and freqs */
-  BYTE *const dict = (BYTE * const)malloc(dictBufferCapacity);
-  COVER_dictSelection_t selection = COVER_dictSelectionError(ERROR(GENERIC));
-  U32 *freqs = (U32*) malloc(((U64)1 << ctx->f) * sizeof(U32));
-  if (!segmentFreqs || !dict || !freqs) {
-    DISPLAYLEVEL(1, "Failed to allocate buffers: out of memory\n");
-    goto _cleanup;
-  }
-  /* Copy the frequencies because we need to modify them */
-  memcpy(freqs, ctx->freqs, ((U64)1 << ctx->f) * sizeof(U32));
-  /* Build the dictionary */
-  { const size_t tail = FASTCOVER_buildDictionary(ctx, freqs, dict, dictBufferCapacity,
-                                                    parameters, segmentFreqs);
-
-    const unsigned nbFinalizeSamples = (unsigned)(ctx->nbTrainSamples * ctx->accelParams.finalize / 100);
-    selection = COVER_selectDict(dict + tail, dictBufferCapacity - tail,
-         ctx->samples, ctx->samplesSizes, nbFinalizeSamples, ctx->nbTrainSamples, ctx->nbSamples, parameters, ctx->offsets,
-         totalCompressedSize);
-
-    if (COVER_dictSelectionIsError(selection)) {
-      DISPLAYLEVEL(1, "Failed to select dictionary\n");
-      goto _cleanup;
-    }
-  }
-_cleanup:
-  free(dict);
-  COVER_best_finish(data->best, parameters, selection);
-  free(data);
-  free(segmentFreqs);
-  COVER_dictSelectionFree(selection);
-  free(freqs);
-}
-
-
-static void
-FASTCOVER_convertToCoverParams(ZDICT_fastCover_params_t fastCoverParams,
-                               ZDICT_cover_params_t* coverParams)
-{
-    coverParams->k = fastCoverParams.k;
-    coverParams->d = fastCoverParams.d;
-    coverParams->steps = fastCoverParams.steps;
-    coverParams->nbThreads = fastCoverParams.nbThreads;
-    coverParams->splitPoint = fastCoverParams.splitPoint;
-    coverParams->zParams = fastCoverParams.zParams;
-    coverParams->shrinkDict = fastCoverParams.shrinkDict;
-}
-
-
-static void
-FASTCOVER_convertToFastCoverParams(ZDICT_cover_params_t coverParams,
-                                   ZDICT_fastCover_params_t* fastCoverParams,
-                                   unsigned f, unsigned accel)
-{
-    fastCoverParams->k = coverParams.k;
-    fastCoverParams->d = coverParams.d;
-    fastCoverParams->steps = coverParams.steps;
-    fastCoverParams->nbThreads = coverParams.nbThreads;
-    fastCoverParams->splitPoint = coverParams.splitPoint;
-    fastCoverParams->f = f;
-    fastCoverParams->accel = accel;
-    fastCoverParams->zParams = coverParams.zParams;
-    fastCoverParams->shrinkDict = coverParams.shrinkDict;
-}
-
-
-ZDICTLIB_API size_t
-ZDICT_trainFromBuffer_fastCover(void* dictBuffer, size_t dictBufferCapacity,
-                                const void* samplesBuffer,
-                                const size_t* samplesSizes, unsigned nbSamples,
-                                ZDICT_fastCover_params_t parameters)
-{
-    BYTE* const dict = (BYTE*)dictBuffer;
-    FASTCOVER_ctx_t ctx;
-    ZDICT_cover_params_t coverParams;
-    FASTCOVER_accel_t accelParams;
-    /* Initialize global data */
-    g_displayLevel = parameters.zParams.notificationLevel;
-    /* Assign splitPoint and f if not provided */
-    parameters.splitPoint = 1.0;
-    parameters.f = parameters.f == 0 ? DEFAULT_F : parameters.f;
-    parameters.accel = parameters.accel == 0 ? DEFAULT_ACCEL : parameters.accel;
-    /* Convert to cover parameter */
-    memset(&coverParams, 0 , sizeof(coverParams));
-    FASTCOVER_convertToCoverParams(parameters, &coverParams);
-    /* Checks */
-    if (!FASTCOVER_checkParameters(coverParams, dictBufferCapacity, parameters.f,
-                                   parameters.accel)) {
-      DISPLAYLEVEL(1, "FASTCOVER parameters incorrect\n");
-      return ERROR(parameter_outOfBound);
-    }
-    if (nbSamples == 0) {
-      DISPLAYLEVEL(1, "FASTCOVER must have at least one input file\n");
-      return ERROR(srcSize_wrong);
-    }
-    if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
-      DISPLAYLEVEL(1, "dictBufferCapacity must be at least %u\n",
-                   ZDICT_DICTSIZE_MIN);
-      return ERROR(dstSize_tooSmall);
-    }
-    /* Assign corresponding FASTCOVER_accel_t to accelParams*/
-    accelParams = FASTCOVER_defaultAccelParameters[parameters.accel];
-    /* Initialize context */
-    {
-      size_t const initVal = FASTCOVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples,
-                            coverParams.d, parameters.splitPoint, parameters.f,
-                            accelParams);
-      if (ZSTD_isError(initVal)) {
-        DISPLAYLEVEL(1, "Failed to initialize context\n");
-        return initVal;
-      }
-    }
-    COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.nbDmers, g_displayLevel);
-    /* Build the dictionary */
-    DISPLAYLEVEL(2, "Building dictionary\n");
-    {
-      /* Initialize array to keep track of frequency of dmer within activeSegment */
-      U16* segmentFreqs = (U16 *)calloc(((U64)1 << parameters.f), sizeof(U16));
-      const size_t tail = FASTCOVER_buildDictionary(&ctx, ctx.freqs, dictBuffer,
-                                                dictBufferCapacity, coverParams, segmentFreqs);
-      const unsigned nbFinalizeSamples = (unsigned)(ctx.nbTrainSamples * ctx.accelParams.finalize / 100);
-      const size_t dictionarySize = ZDICT_finalizeDictionary(
-          dict, dictBufferCapacity, dict + tail, dictBufferCapacity - tail,
-          samplesBuffer, samplesSizes, nbFinalizeSamples, coverParams.zParams);
-      if (!ZSTD_isError(dictionarySize)) {
-          DISPLAYLEVEL(2, "Constructed dictionary of size %u\n",
-                      (unsigned)dictionarySize);
-      }
-      FASTCOVER_ctx_destroy(&ctx);
-      free(segmentFreqs);
-      return dictionarySize;
-    }
-}
-
-
-ZDICTLIB_API size_t
-ZDICT_optimizeTrainFromBuffer_fastCover(
-                    void* dictBuffer, size_t dictBufferCapacity,
-                    const void* samplesBuffer,
-                    const size_t* samplesSizes, unsigned nbSamples,
-                    ZDICT_fastCover_params_t* parameters)
-{
-    ZDICT_cover_params_t coverParams;
-    FASTCOVER_accel_t accelParams;
-    /* constants */
-    const unsigned nbThreads = parameters->nbThreads;
-    const double splitPoint =
-        parameters->splitPoint <= 0.0 ? DEFAULT_SPLITPOINT : parameters->splitPoint;
-    const unsigned kMinD = parameters->d == 0 ? 6 : parameters->d;
-    const unsigned kMaxD = parameters->d == 0 ? 8 : parameters->d;
-    const unsigned kMinK = parameters->k == 0 ? 50 : parameters->k;
-    const unsigned kMaxK = parameters->k == 0 ? 2000 : parameters->k;
-    const unsigned kSteps = parameters->steps == 0 ? 40 : parameters->steps;
-    const unsigned kStepSize = MAX((kMaxK - kMinK) / kSteps, 1);
-    const unsigned kIterations =
-        (1 + (kMaxD - kMinD) / 2) * (1 + (kMaxK - kMinK) / kStepSize);
-    const unsigned f = parameters->f == 0 ? DEFAULT_F : parameters->f;
-    const unsigned accel = parameters->accel == 0 ? DEFAULT_ACCEL : parameters->accel;
-    const unsigned shrinkDict = 0;
-    /* Local variables */
-    const int displayLevel = parameters->zParams.notificationLevel;
-    unsigned iteration = 1;
-    unsigned d;
-    unsigned k;
-    COVER_best_t best;
-    POOL_ctx *pool = NULL;
-    int warned = 0;
-    /* Checks */
-    if (splitPoint <= 0 || splitPoint > 1) {
-      LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect splitPoint\n");
-      return ERROR(parameter_outOfBound);
-    }
-    if (accel == 0 || accel > FASTCOVER_MAX_ACCEL) {
-      LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect accel\n");
-      return ERROR(parameter_outOfBound);
-    }
-    if (kMinK < kMaxD || kMaxK < kMinK) {
-      LOCALDISPLAYLEVEL(displayLevel, 1, "Incorrect k\n");
-      return ERROR(parameter_outOfBound);
-    }
-    if (nbSamples == 0) {
-      LOCALDISPLAYLEVEL(displayLevel, 1, "FASTCOVER must have at least one input file\n");
-      return ERROR(srcSize_wrong);
-    }
-    if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) {
-      LOCALDISPLAYLEVEL(displayLevel, 1, "dictBufferCapacity must be at least %u\n",
-                   ZDICT_DICTSIZE_MIN);
-      return ERROR(dstSize_tooSmall);
-    }
-    if (nbThreads > 1) {
-      pool = POOL_create(nbThreads, 1);
-      if (!pool) {
-        return ERROR(memory_allocation);
-      }
-    }
-    /* Initialization */
-    COVER_best_init(&best);
-    memset(&coverParams, 0 , sizeof(coverParams));
-    FASTCOVER_convertToCoverParams(*parameters, &coverParams);
-    accelParams = FASTCOVER_defaultAccelParameters[accel];
-    /* Turn down global display level to clean up display at level 2 and below */
-    g_displayLevel = displayLevel == 0 ? 0 : displayLevel - 1;
-    /* Loop through d first because each new value needs a new context */
-    LOCALDISPLAYLEVEL(displayLevel, 2, "Trying %u different sets of parameters\n",
-                      kIterations);
-    for (d = kMinD; d <= kMaxD; d += 2) {
-      /* Initialize the context for this value of d */
-      FASTCOVER_ctx_t ctx;
-      LOCALDISPLAYLEVEL(displayLevel, 3, "d=%u\n", d);
-      {
-        size_t const initVal = FASTCOVER_ctx_init(&ctx, samplesBuffer, samplesSizes, nbSamples, d, splitPoint, f, accelParams);
-        if (ZSTD_isError(initVal)) {
-          LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to initialize context\n");
-          COVER_best_destroy(&best);
-          POOL_free(pool);
-          return initVal;
-        }
-      }
-      if (!warned) {
-        COVER_warnOnSmallCorpus(dictBufferCapacity, ctx.nbDmers, displayLevel);
-        warned = 1;
-      }
-      /* Loop through k reusing the same context */
-      for (k = kMinK; k <= kMaxK; k += kStepSize) {
-        /* Prepare the arguments */
-        FASTCOVER_tryParameters_data_t *data = (FASTCOVER_tryParameters_data_t *)malloc(
-            sizeof(FASTCOVER_tryParameters_data_t));
-        LOCALDISPLAYLEVEL(displayLevel, 3, "k=%u\n", k);
-        if (!data) {
-          LOCALDISPLAYLEVEL(displayLevel, 1, "Failed to allocate parameters\n");
-          COVER_best_destroy(&best);
-          FASTCOVER_ctx_destroy(&ctx);
-          POOL_free(pool);
-          return ERROR(memory_allocation);
-        }
-        data->ctx = &ctx;
-        data->best = &best;
-        data->dictBufferCapacity = dictBufferCapacity;
-        data->parameters = coverParams;
-        data->parameters.k = k;
-        data->parameters.d = d;
-        data->parameters.splitPoint = splitPoint;
-        data->parameters.steps = kSteps;
-        data->parameters.shrinkDict = shrinkDict;
-        data->parameters.zParams.notificationLevel = g_displayLevel;
-        /* Check the parameters */
-        if (!FASTCOVER_checkParameters(data->parameters, dictBufferCapacity,
-                                       data->ctx->f, accel)) {
-          DISPLAYLEVEL(1, "FASTCOVER parameters incorrect\n");
-          free(data);
-          continue;
-        }
-        /* Call the function and pass ownership of data to it */
-        COVER_best_start(&best);
-        if (pool) {
-          POOL_add(pool, &FASTCOVER_tryParameters, data);
-        } else {
-          FASTCOVER_tryParameters(data);
-        }
-        /* Print status */
-        LOCALDISPLAYUPDATE(displayLevel, 2, "\r%u%%       ",
-                           (unsigned)((iteration * 100) / kIterations));
-        ++iteration;
-      }
-      COVER_best_wait(&best);
-      FASTCOVER_ctx_destroy(&ctx);
-    }
-    LOCALDISPLAYLEVEL(displayLevel, 2, "\r%79s\r", "");
-    /* Fill the output buffer and parameters with output of the best parameters */
-    {
-      const size_t dictSize = best.dictSize;
-      if (ZSTD_isError(best.compressedSize)) {
-        const size_t compressedSize = best.compressedSize;
-        COVER_best_destroy(&best);
-        POOL_free(pool);
-        return compressedSize;
-      }
-      FASTCOVER_convertToFastCoverParams(best.parameters, parameters, f, accel);
-      memcpy(dictBuffer, best.dict, dictSize);
-      COVER_best_destroy(&best);
-      POOL_free(pool);
-      return dictSize;
-    }
-
-}
diff --git a/vendor/github.com/DataDog/zstd/fse.h b/vendor/github.com/DataDog/zstd/fse.h
deleted file mode 100644
index 811c670..0000000
--- a/vendor/github.com/DataDog/zstd/fse.h
+++ /dev/null
@@ -1,708 +0,0 @@
-/* ******************************************************************
-   FSE : Finite State Entropy codec
-   Public Prototypes declaration
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef FSE_H
-#define FSE_H
-
-
-/*-*****************************************
-*  Dependencies
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-
-
-/*-*****************************************
-*  FSE_PUBLIC_API : control library symbols visibility
-******************************************/
-#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4)
-#  define FSE_PUBLIC_API __attribute__ ((visibility ("default")))
-#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1)   /* Visual expected */
-#  define FSE_PUBLIC_API __declspec(dllexport)
-#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1)
-#  define FSE_PUBLIC_API __declspec(dllimport) /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/
-#else
-#  define FSE_PUBLIC_API
-#endif
-
-/*------   Version   ------*/
-#define FSE_VERSION_MAJOR    0
-#define FSE_VERSION_MINOR    9
-#define FSE_VERSION_RELEASE  0
-
-#define FSE_LIB_VERSION FSE_VERSION_MAJOR.FSE_VERSION_MINOR.FSE_VERSION_RELEASE
-#define FSE_QUOTE(str) #str
-#define FSE_EXPAND_AND_QUOTE(str) FSE_QUOTE(str)
-#define FSE_VERSION_STRING FSE_EXPAND_AND_QUOTE(FSE_LIB_VERSION)
-
-#define FSE_VERSION_NUMBER  (FSE_VERSION_MAJOR *100*100 + FSE_VERSION_MINOR *100 + FSE_VERSION_RELEASE)
-FSE_PUBLIC_API unsigned FSE_versionNumber(void);   /**< library version number; to be used when checking dll version */
-
-
-/*-****************************************
-*  FSE simple functions
-******************************************/
-/*! FSE_compress() :
-    Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'.
-    'dst' buffer must be already allocated. Compression runs faster is dstCapacity >= FSE_compressBound(srcSize).
-    @return : size of compressed data (<= dstCapacity).
-    Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!!
-                     if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression instead.
-                     if FSE_isError(return), compression failed (more details using FSE_getErrorName())
-*/
-FSE_PUBLIC_API size_t FSE_compress(void* dst, size_t dstCapacity,
-                             const void* src, size_t srcSize);
-
-/*! FSE_decompress():
-    Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstCapacity'.
-    @return : size of regenerated data (<= maxDstSize),
-              or an error code, which can be tested using FSE_isError() .
-
-    ** Important ** : FSE_decompress() does not decompress non-compressible nor RLE data !!!
-    Why ? : making this distinction requires a header.
-    Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-FSE_PUBLIC_API size_t FSE_decompress(void* dst,  size_t dstCapacity,
-                               const void* cSrc, size_t cSrcSize);
-
-
-/*-*****************************************
-*  Tool functions
-******************************************/
-FSE_PUBLIC_API size_t FSE_compressBound(size_t size);       /* maximum compressed size */
-
-/* Error Management */
-FSE_PUBLIC_API unsigned    FSE_isError(size_t code);        /* tells if a return value is an error code */
-FSE_PUBLIC_API const char* FSE_getErrorName(size_t code);   /* provides error code string (useful for debugging) */
-
-
-/*-*****************************************
-*  FSE advanced functions
-******************************************/
-/*! FSE_compress2() :
-    Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
-    Both parameters can be defined as '0' to mean : use default value
-    @return : size of compressed data
-    Special values : if return == 0, srcData is not compressible => Nothing is stored within cSrc !!!
-                     if return == 1, srcData is a single byte symbol * srcSize times. Use RLE compression.
-                     if FSE_isError(return), it's an error code.
-*/
-FSE_PUBLIC_API size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
-
-
-/*-*****************************************
-*  FSE detailed API
-******************************************/
-/*!
-FSE_compress() does the following:
-1. count symbol occurrence from source[] into table count[] (see hist.h)
-2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
-3. save normalized counters to memory buffer using writeNCount()
-4. build encoding table 'CTable' from normalized counters
-5. encode the data stream using encoding table 'CTable'
-
-FSE_decompress() does the following:
-1. read normalized counters with readNCount()
-2. build decoding table 'DTable' from normalized counters
-3. decode the data stream using decoding table 'DTable'
-
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and provide normalized distribution using external method.
-*/
-
-/* *** COMPRESSION *** */
-
-/*! FSE_optimalTableLog():
-    dynamically downsize 'tableLog' when conditions are met.
-    It saves CPU time, by using smaller tables, while preserving or even improving compression ratio.
-    @return : recommended tableLog (necessarily <= 'maxTableLog') */
-FSE_PUBLIC_API unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue);
-
-/*! FSE_normalizeCount():
-    normalize counts so that sum(count[]) == Power_of_2 (2^tableLog)
-    'normalizedCounter' is a table of short, of minimum size (maxSymbolValue+1).
-    @return : tableLog,
-              or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog,
-                    const unsigned* count, size_t srcSize, unsigned maxSymbolValue);
-
-/*! FSE_NCountWriteBound():
-    Provides the maximum possible size of an FSE normalized table, given 'maxSymbolValue' and 'tableLog'.
-    Typically useful for allocation purpose. */
-FSE_PUBLIC_API size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_writeNCount():
-    Compactly save 'normalizedCounter' into 'buffer'.
-    @return : size of the compressed table,
-              or an errorCode, which can be tested using FSE_isError(). */
-FSE_PUBLIC_API size_t FSE_writeNCount (void* buffer, size_t bufferSize,
-                                 const short* normalizedCounter,
-                                 unsigned maxSymbolValue, unsigned tableLog);
-
-/*! Constructor and Destructor of FSE_CTable.
-    Note that FSE_CTable size depends on 'tableLog' and 'maxSymbolValue' */
-typedef unsigned FSE_CTable;   /* don't allocate that. It's only meant to be more restrictive than void* */
-FSE_PUBLIC_API FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog);
-FSE_PUBLIC_API void        FSE_freeCTable (FSE_CTable* ct);
-
-/*! FSE_buildCTable():
-    Builds `ct`, which must be already allocated, using FSE_createCTable().
-    @return : 0, or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_compress_usingCTable():
-    Compress `src` using `ct` into `dst` which must be already allocated.
-    @return : size of compressed data (<= `dstCapacity`),
-              or 0 if compressed data could not fit into `dst`,
-              or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_compress_usingCTable (void* dst, size_t dstCapacity, const void* src, size_t srcSize, const FSE_CTable* ct);
-
-/*!
-Tutorial :
-----------
-The first step is to count all symbols. FSE_count() does this job very fast.
-Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have 'maxSymbolValuePtr[0]+1' cells.
-'src' is a table of bytes of size 'srcSize'. All values within 'src' MUST be <= maxSymbolValuePtr[0]
-maxSymbolValuePtr[0] will be updated, with its real value (necessarily <= original value)
-FSE_count() will return the number of occurrence of the most frequent symbol.
-This can be used to know if there is a single symbol within 'src', and to quickly evaluate its compressibility.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
-
-The next step is to normalize the frequencies.
-FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'.
-It also guarantees a minimum of 1 to any Symbol with frequency >= 1.
-You can use 'tableLog'==0 to mean "use default tableLog value".
-If you are unsure of which tableLog value to use, you can ask FSE_optimalTableLog(),
-which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default").
-
-The result of FSE_normalizeCount() will be saved into a table,
-called 'normalizedCounter', which is a table of signed short.
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells.
-The return value is tableLog if everything proceeded as expected.
-It is 0 if there is a single symbol within distribution.
-If there is an error (ex: invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()).
-
-'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeNCount().
-'buffer' must be already allocated.
-For guaranteed success, buffer size must be at least FSE_headerBound().
-The result of the function is the number of bytes written into 'buffer'.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError(); ex : buffer size too small).
-
-'normalizedCounter' can then be used to create the compression table 'CTable'.
-The space required by 'CTable' must be already allocated, using FSE_createCTable().
-You can then use FSE_buildCTable() to fill 'CTable'.
-If there is an error, both functions will return an ErrorCode (which can be tested using FSE_isError()).
-
-'CTable' can then be used to compress 'src', with FSE_compress_usingCTable().
-Similar to FSE_count(), the convention is that 'src' is assumed to be a table of char of size 'srcSize'
-The function returns the size of compressed data (without header), necessarily <= `dstCapacity`.
-If it returns '0', compressed data could not fit into 'dst'.
-If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
-*/
-
-
-/* *** DECOMPRESSION *** */
-
-/*! FSE_readNCount():
-    Read compactly saved 'normalizedCounter' from 'rBuffer'.
-    @return : size read from 'rBuffer',
-              or an errorCode, which can be tested using FSE_isError().
-              maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-FSE_PUBLIC_API size_t FSE_readNCount (short* normalizedCounter,
-                           unsigned* maxSymbolValuePtr, unsigned* tableLogPtr,
-                           const void* rBuffer, size_t rBuffSize);
-
-/*! Constructor and Destructor of FSE_DTable.
-    Note that its size depends on 'tableLog' */
-typedef unsigned FSE_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-FSE_PUBLIC_API FSE_DTable* FSE_createDTable(unsigned tableLog);
-FSE_PUBLIC_API void        FSE_freeDTable(FSE_DTable* dt);
-
-/*! FSE_buildDTable():
-    Builds 'dt', which must be already allocated, using FSE_createDTable().
-    return : 0, or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_buildDTable (FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSE_decompress_usingDTable():
-    Decompress compressed source `cSrc` of size `cSrcSize` using `dt`
-    into `dst` which must be already allocated.
-    @return : size of regenerated data (necessarily <= `dstCapacity`),
-              or an errorCode, which can be tested using FSE_isError() */
-FSE_PUBLIC_API size_t FSE_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt);
-
-/*!
-Tutorial :
-----------
-(Note : these functions only decompress FSE-compressed blocks.
- If block is uncompressed, use memcpy() instead
- If block is a single repeated byte, use memset() instead )
-
-The first step is to obtain the normalized frequencies of symbols.
-This can be performed by FSE_readNCount() if it was saved using FSE_writeNCount().
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short.
-In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
-or size the table to handle worst case situations (typically 256).
-FSE_readNCount() will provide 'tableLog' and 'maxSymbolValue'.
-The result of FSE_readNCount() is the number of bytes read from 'rBuffer'.
-Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that.
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-The next step is to build the decompression tables 'FSE_DTable' from 'normalizedCounter'.
-This is performed by the function FSE_buildDTable().
-The space required by 'FSE_DTable' must be already allocated using FSE_createDTable().
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-`FSE_DTable` can then be used to decompress `cSrc`, with FSE_decompress_usingDTable().
-`cSrcSize` must be strictly correct, otherwise decompression will fail.
-FSE_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`).
-If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small)
-*/
-
-#endif  /* FSE_H */
-
-#if defined(FSE_STATIC_LINKING_ONLY) && !defined(FSE_H_FSE_STATIC_LINKING_ONLY)
-#define FSE_H_FSE_STATIC_LINKING_ONLY
-
-/* *** Dependency *** */
-#include "bitstream.h"
-
-
-/* *****************************************
-*  Static allocation
-*******************************************/
-/* FSE buffer bounds */
-#define FSE_NCOUNTBOUND 512
-#define FSE_BLOCKBOUND(size) (size + (size>>7))
-#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* It is possible to statically allocate FSE CTable/DTable as a table of FSE_CTable/FSE_DTable using below macros */
-#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue)   (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
-#define FSE_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-/* or use the size to malloc() space directly. Pay attention to alignment restrictions though */
-#define FSE_CTABLE_SIZE(maxTableLog, maxSymbolValue)   (FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) * sizeof(FSE_CTable))
-#define FSE_DTABLE_SIZE(maxTableLog)                   (FSE_DTABLE_SIZE_U32(maxTableLog) * sizeof(FSE_DTable))
-
-
-/* *****************************************
- *  FSE advanced API
- ***************************************** */
-
-unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus);
-/**< same as FSE_optimalTableLog(), which used `minus==2` */
-
-/* FSE_compress_wksp() :
- * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`).
- * FSE_WKSP_SIZE_U32() provides the minimum size required for `workSpace` as a table of FSE_CTable.
- */
-#define FSE_WKSP_SIZE_U32(maxTableLog, maxSymbolValue)   ( FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) + ((maxTableLog > 12) ? (1 << (maxTableLog - 2)) : 1024) )
-size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize);
-
-size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits);
-/**< build a fake FSE_CTable, designed for a flat distribution, where each symbol uses nbBits */
-
-size_t FSE_buildCTable_rle (FSE_CTable* ct, unsigned char symbolValue);
-/**< build a fake FSE_CTable, designed to compress always the same symbolValue */
-
-/* FSE_buildCTable_wksp() :
- * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`).
- * `wkspSize` must be >= `(1<<tableLog)`.
- */
-size_t FSE_buildCTable_wksp(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize);
-
-size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
-/**< build a fake FSE_DTable, designed to read a flat distribution where each symbol uses nbBits */
-
-size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
-/**< build a fake FSE_DTable, designed to always generate the same symbolValue */
-
-size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog);
-/**< same as FSE_decompress(), using an externally allocated `workSpace` produced with `FSE_DTABLE_SIZE_U32(maxLog)` */
-
-typedef enum {
-   FSE_repeat_none,  /**< Cannot use the previous table */
-   FSE_repeat_check, /**< Can use the previous table but it must be checked */
-   FSE_repeat_valid  /**< Can use the previous table and it is assumed to be valid */
- } FSE_repeat;
-
-/* *****************************************
-*  FSE symbol compression API
-*******************************************/
-/*!
-   This API consists of small unitary functions, which highly benefit from being inlined.
-   Hence their body are included in next section.
-*/
-typedef struct {
-    ptrdiff_t   value;
-    const void* stateTable;
-    const void* symbolTT;
-    unsigned    stateLog;
-} FSE_CState_t;
-
-static void FSE_initCState(FSE_CState_t* CStatePtr, const FSE_CTable* ct);
-
-static void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned symbol);
-
-static void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* CStatePtr);
-
-/**<
-These functions are inner components of FSE_compress_usingCTable().
-They allow the creation of custom streams, mixing multiple tables and bit sources.
-
-A key property to keep in mind is that encoding and decoding are done **in reverse direction**.
-So the first symbol you will encode is the last you will decode, like a LIFO stack.
-
-You will need a few variables to track your CStream. They are :
-
-FSE_CTable    ct;         // Provided by FSE_buildCTable()
-BIT_CStream_t bitStream;  // bitStream tracking structure
-FSE_CState_t  state;      // State tracking structure (can have several)
-
-
-The first thing to do is to init bitStream and state.
-    size_t errorCode = BIT_initCStream(&bitStream, dstBuffer, maxDstSize);
-    FSE_initCState(&state, ct);
-
-Note that BIT_initCStream() can produce an error code, so its result should be tested, using FSE_isError();
-You can then encode your input data, byte after byte.
-FSE_encodeSymbol() outputs a maximum of 'tableLog' bits at a time.
-Remember decoding will be done in reverse direction.
-    FSE_encodeByte(&bitStream, &state, symbol);
-
-At any time, you can also add any bit sequence.
-Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders
-    BIT_addBits(&bitStream, bitField, nbBits);
-
-The above methods don't commit data to memory, they just store it into local register, for speed.
-Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
-Writing data to memory is a manual operation, performed by the flushBits function.
-    BIT_flushBits(&bitStream);
-
-Your last FSE encoding operation shall be to flush your last state value(s).
-    FSE_flushState(&bitStream, &state);
-
-Finally, you must close the bitStream.
-The function returns the size of CStream in bytes.
-If data couldn't fit into dstBuffer, it will return a 0 ( == not compressible)
-If there is an error, it returns an errorCode (which can be tested using FSE_isError()).
-    size_t size = BIT_closeCStream(&bitStream);
-*/
-
-
-/* *****************************************
-*  FSE symbol decompression API
-*******************************************/
-typedef struct {
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-
-static void     FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt);
-
-static unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
-
-/**<
-Let's now decompose FSE_decompress_usingDTable() into its unitary components.
-You will decode FSE-encoded symbols from the bitStream,
-and also any other bitFields you put in, **in reverse order**.
-
-You will need a few variables to track your bitStream. They are :
-
-BIT_DStream_t DStream;    // Stream context
-FSE_DState_t  DState;     // State context. Multiple ones are possible
-FSE_DTable*   DTablePtr;  // Decoding table, provided by FSE_buildDTable()
-
-The first thing to do is to init the bitStream.
-    errorCode = BIT_initDStream(&DStream, srcBuffer, srcSize);
-
-You should then retrieve your initial state(s)
-(in reverse flushing order if you have several ones) :
-    errorCode = FSE_initDState(&DState, &DStream, DTablePtr);
-
-You can then decode your data, symbol after symbol.
-For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'.
-Keep in mind that symbols are decoded in reverse order, like a LIFO stack (last in, first out).
-    unsigned char symbol = FSE_decodeSymbol(&DState, &DStream);
-
-You can retrieve any bitfield you eventually stored into the bitStream (in reverse order)
-Note : maximum allowed nbBits is 25, for 32-bits compatibility
-    size_t bitField = BIT_readBits(&DStream, nbBits);
-
-All above operations only read from local register (which size depends on size_t).
-Refueling the register from memory is manually performed by the reload method.
-    endSignal = FSE_reloadDStream(&DStream);
-
-BIT_reloadDStream() result tells if there is still some more data to read from DStream.
-BIT_DStream_unfinished : there is still some data left into the DStream.
-BIT_DStream_endOfBuffer : Dstream reached end of buffer. Its container may no longer be completely filled.
-BIT_DStream_completed : Dstream reached its exact end, corresponding in general to decompression completed.
-BIT_DStream_tooFar : Dstream went too far. Decompression result is corrupted.
-
-When reaching end of buffer (BIT_DStream_endOfBuffer), progress slowly, notably if you decode multiple symbols per loop,
-to properly detect the exact end of stream.
-After each decoded symbol, check if DStream is fully consumed using this simple test :
-    BIT_reloadDStream(&DStream) >= BIT_DStream_completed
-
-When it's done, verify decompression is fully completed, by checking both DStream and the relevant states.
-Checking if DStream has reached its end is performed by :
-    BIT_endOfDStream(&DStream);
-Check also the states. There might be some symbols left there, if some high probability ones (>50%) are possible.
-    FSE_endOfDState(&DState);
-*/
-
-
-/* *****************************************
-*  FSE unsafe API
-*******************************************/
-static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* *****************************************
-*  Implementation of inlined functions
-*******************************************/
-typedef struct {
-    int deltaFindState;
-    U32 deltaNbBits;
-} FSE_symbolCompressionTransform; /* total 8 bytes */
-
-MEM_STATIC void FSE_initCState(FSE_CState_t* statePtr, const FSE_CTable* ct)
-{
-    const void* ptr = ct;
-    const U16* u16ptr = (const U16*) ptr;
-    const U32 tableLog = MEM_read16(ptr);
-    statePtr->value = (ptrdiff_t)1<<tableLog;
-    statePtr->stateTable = u16ptr+2;
-    statePtr->symbolTT = ct + 1 + (tableLog ? (1<<(tableLog-1)) : 1);
-    statePtr->stateLog = tableLog;
-}
-
-
-/*! FSE_initCState2() :
-*   Same as FSE_initCState(), but the first symbol to include (which will be the last to be read)
-*   uses the smallest state value possible, saving the cost of this symbol */
-MEM_STATIC void FSE_initCState2(FSE_CState_t* statePtr, const FSE_CTable* ct, U32 symbol)
-{
-    FSE_initCState(statePtr, ct);
-    {   const FSE_symbolCompressionTransform symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
-        const U16* stateTable = (const U16*)(statePtr->stateTable);
-        U32 nbBitsOut  = (U32)((symbolTT.deltaNbBits + (1<<15)) >> 16);
-        statePtr->value = (nbBitsOut << 16) - symbolTT.deltaNbBits;
-        statePtr->value = stateTable[(statePtr->value >> nbBitsOut) + symbolTT.deltaFindState];
-    }
-}
-
-MEM_STATIC void FSE_encodeSymbol(BIT_CStream_t* bitC, FSE_CState_t* statePtr, unsigned symbol)
-{
-    FSE_symbolCompressionTransform const symbolTT = ((const FSE_symbolCompressionTransform*)(statePtr->symbolTT))[symbol];
-    const U16* const stateTable = (const U16*)(statePtr->stateTable);
-    U32 const nbBitsOut  = (U32)((statePtr->value + symbolTT.deltaNbBits) >> 16);
-    BIT_addBits(bitC, statePtr->value, nbBitsOut);
-    statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT.deltaFindState];
-}
-
-MEM_STATIC void FSE_flushCState(BIT_CStream_t* bitC, const FSE_CState_t* statePtr)
-{
-    BIT_addBits(bitC, statePtr->value, statePtr->stateLog);
-    BIT_flushBits(bitC);
-}
-
-
-/* FSE_getMaxNbBits() :
- * Approximate maximum cost of a symbol, in bits.
- * Fractional get rounded up (i.e : a symbol with a normalized frequency of 3 gives the same result as a frequency of 2)
- * note 1 : assume symbolValue is valid (<= maxSymbolValue)
- * note 2 : if freq[symbolValue]==0, @return a fake cost of tableLog+1 bits */
-MEM_STATIC U32 FSE_getMaxNbBits(const void* symbolTTPtr, U32 symbolValue)
-{
-    const FSE_symbolCompressionTransform* symbolTT = (const FSE_symbolCompressionTransform*) symbolTTPtr;
-    return (symbolTT[symbolValue].deltaNbBits + ((1<<16)-1)) >> 16;
-}
-
-/* FSE_bitCost() :
- * Approximate symbol cost, as fractional value, using fixed-point format (accuracyLog fractional bits)
- * note 1 : assume symbolValue is valid (<= maxSymbolValue)
- * note 2 : if freq[symbolValue]==0, @return a fake cost of tableLog+1 bits */
-MEM_STATIC U32 FSE_bitCost(const void* symbolTTPtr, U32 tableLog, U32 symbolValue, U32 accuracyLog)
-{
-    const FSE_symbolCompressionTransform* symbolTT = (const FSE_symbolCompressionTransform*) symbolTTPtr;
-    U32 const minNbBits = symbolTT[symbolValue].deltaNbBits >> 16;
-    U32 const threshold = (minNbBits+1) << 16;
-    assert(tableLog < 16);
-    assert(accuracyLog < 31-tableLog);  /* ensure enough room for renormalization double shift */
-    {   U32 const tableSize = 1 << tableLog;
-        U32 const deltaFromThreshold = threshold - (symbolTT[symbolValue].deltaNbBits + tableSize);
-        U32 const normalizedDeltaFromThreshold = (deltaFromThreshold << accuracyLog) >> tableLog;   /* linear interpolation (very approximate) */
-        U32 const bitMultiplier = 1 << accuracyLog;
-        assert(symbolTT[symbolValue].deltaNbBits + tableSize <= threshold);
-        assert(normalizedDeltaFromThreshold <= bitMultiplier);
-        return (minNbBits+1)*bitMultiplier - normalizedDeltaFromThreshold;
-    }
-}
-
-
-/* ======    Decompression    ====== */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSE_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSE_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSE_DTableHeader* const DTableH = (const FSE_DTableHeader*)ptr;
-    DStatePtr->state = BIT_readBits(bitD, DTableH->tableLog);
-    BIT_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSE_peekSymbol(const FSE_DState_t* DStatePtr)
-{
-    FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    return DInfo.symbol;
-}
-
-MEM_STATIC void FSE_updateState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    size_t const lowBits = BIT_readBits(bitD, nbBits);
-    DStatePtr->state = DInfo.newState + lowBits;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BIT_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-/*! FSE_decodeSymbolFast() :
-    unsafe, only works if no symbol has a probability > 50% */
-MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    FSE_decode_t const DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BIT_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/* **************************************************************
-*  Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#ifndef FSE_MAX_MEMORY_USAGE
-#  define FSE_MAX_MEMORY_USAGE 14
-#endif
-#ifndef FSE_DEFAULT_MEMORY_USAGE
-#  define FSE_DEFAULT_MEMORY_USAGE 13
-#endif
-
-/*!FSE_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#ifndef FSE_MAX_SYMBOL_VALUE
-#  define FSE_MAX_SYMBOL_VALUE 255
-#endif
-
-/* **************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-#define FSE_DECODE_TYPE FSE_decode_t
-
-
-#endif   /* !FSE_COMMONDEFS_ONLY */
-
-
-/* ***************************************************************
-*  Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG  (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-#  error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-#define FSE_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3)
-
-
-#endif /* FSE_STATIC_LINKING_ONLY */
-
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/fse_compress.c b/vendor/github.com/DataDog/zstd/fse_compress.c
deleted file mode 100644
index 68b47e1..0000000
--- a/vendor/github.com/DataDog/zstd/fse_compress.c
+++ /dev/null
@@ -1,721 +0,0 @@
-/* ******************************************************************
-   FSE : Finite State Entropy encoder
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include "compiler.h"
-#include "mem.h"        /* U32, U16, etc. */
-#include "debug.h"      /* assert, DEBUGLOG */
-#include "hist.h"       /* HIST_count_wksp */
-#include "bitstream.h"
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_isError ERR_isError
-
-
-/* **************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-
-/* FSE_buildCTable_wksp() :
- * Same as FSE_buildCTable(), but using an externally allocated scratch buffer (`workSpace`).
- * wkspSize should be sized to handle worst case situation, which is `1<<max_tableLog * sizeof(FSE_FUNCTION_TYPE)`
- * workSpace must also be properly aligned with FSE_FUNCTION_TYPE requirements
- */
-size_t FSE_buildCTable_wksp(FSE_CTable* ct,
-                      const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
-                            void* workSpace, size_t wkspSize)
-{
-    U32 const tableSize = 1 << tableLog;
-    U32 const tableMask = tableSize - 1;
-    void* const ptr = ct;
-    U16* const tableU16 = ( (U16*) ptr) + 2;
-    void* const FSCT = ((U32*)ptr) + 1 /* header */ + (tableLog ? tableSize>>1 : 1) ;
-    FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT);
-    U32 const step = FSE_TABLESTEP(tableSize);
-    U32 cumul[FSE_MAX_SYMBOL_VALUE+2];
-
-    FSE_FUNCTION_TYPE* const tableSymbol = (FSE_FUNCTION_TYPE*)workSpace;
-    U32 highThreshold = tableSize-1;
-
-    /* CTable header */
-    if (((size_t)1 << tableLog) * sizeof(FSE_FUNCTION_TYPE) > wkspSize) return ERROR(tableLog_tooLarge);
-    tableU16[-2] = (U16) tableLog;
-    tableU16[-1] = (U16) maxSymbolValue;
-    assert(tableLog < 16);   /* required for threshold strategy to work */
-
-    /* For explanations on how to distribute symbol values over the table :
-     * http://fastcompression.blogspot.fr/2014/02/fse-distributing-symbol-values.html */
-
-     #ifdef __clang_analyzer__
-     memset(tableSymbol, 0, sizeof(*tableSymbol) * tableSize);   /* useless initialization, just to keep scan-build happy */
-     #endif
-
-    /* symbol start positions */
-    {   U32 u;
-        cumul[0] = 0;
-        for (u=1; u <= maxSymbolValue+1; u++) {
-            if (normalizedCounter[u-1]==-1) {  /* Low proba symbol */
-                cumul[u] = cumul[u-1] + 1;
-                tableSymbol[highThreshold--] = (FSE_FUNCTION_TYPE)(u-1);
-            } else {
-                cumul[u] = cumul[u-1] + normalizedCounter[u-1];
-        }   }
-        cumul[maxSymbolValue+1] = tableSize+1;
-    }
-
-    /* Spread symbols */
-    {   U32 position = 0;
-        U32 symbol;
-        for (symbol=0; symbol<=maxSymbolValue; symbol++) {
-            int nbOccurrences;
-            int const freq = normalizedCounter[symbol];
-            for (nbOccurrences=0; nbOccurrences<freq; nbOccurrences++) {
-                tableSymbol[position] = (FSE_FUNCTION_TYPE)symbol;
-                position = (position + step) & tableMask;
-                while (position > highThreshold)
-                    position = (position + step) & tableMask;   /* Low proba area */
-        }   }
-
-        assert(position==0);  /* Must have initialized all positions */
-    }
-
-    /* Build table */
-    {   U32 u; for (u=0; u<tableSize; u++) {
-        FSE_FUNCTION_TYPE s = tableSymbol[u];   /* note : static analyzer may not understand tableSymbol is properly initialized */
-        tableU16[cumul[s]++] = (U16) (tableSize+u);   /* TableU16 : sorted by symbol order; gives next state value */
-    }   }
-
-    /* Build Symbol Transformation Table */
-    {   unsigned total = 0;
-        unsigned s;
-        for (s=0; s<=maxSymbolValue; s++) {
-            switch (normalizedCounter[s])
-            {
-            case  0:
-                /* filling nonetheless, for compatibility with FSE_getMaxNbBits() */
-                symbolTT[s].deltaNbBits = ((tableLog+1) << 16) - (1<<tableLog);
-                break;
-
-            case -1:
-            case  1:
-                symbolTT[s].deltaNbBits = (tableLog << 16) - (1<<tableLog);
-                symbolTT[s].deltaFindState = total - 1;
-                total ++;
-                break;
-            default :
-                {
-                    U32 const maxBitsOut = tableLog - BIT_highbit32 (normalizedCounter[s]-1);
-                    U32 const minStatePlus = normalizedCounter[s] << maxBitsOut;
-                    symbolTT[s].deltaNbBits = (maxBitsOut << 16) - minStatePlus;
-                    symbolTT[s].deltaFindState = total - normalizedCounter[s];
-                    total +=  normalizedCounter[s];
-    }   }   }   }
-
-#if 0  /* debug : symbol costs */
-    DEBUGLOG(5, "\n --- table statistics : ");
-    {   U32 symbol;
-        for (symbol=0; symbol<=maxSymbolValue; symbol++) {
-            DEBUGLOG(5, "%3u: w=%3i,   maxBits=%u, fracBits=%.2f",
-                symbol, normalizedCounter[symbol],
-                FSE_getMaxNbBits(symbolTT, symbol),
-                (double)FSE_bitCost(symbolTT, tableLog, symbol, 8) / 256);
-        }
-    }
-#endif
-
-    return 0;
-}
-
-
-size_t FSE_buildCTable(FSE_CTable* ct, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    FSE_FUNCTION_TYPE tableSymbol[FSE_MAX_TABLESIZE];   /* memset() is not necessary, even if static analyzer complain about it */
-    return FSE_buildCTable_wksp(ct, normalizedCounter, maxSymbolValue, tableLog, tableSymbol, sizeof(tableSymbol));
-}
-
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-
-/*-**************************************************************
-*  FSE NCount encoding
-****************************************************************/
-size_t FSE_NCountWriteBound(unsigned maxSymbolValue, unsigned tableLog)
-{
-    size_t const maxHeaderSize = (((maxSymbolValue+1) * tableLog) >> 3) + 3;
-    return maxSymbolValue ? maxHeaderSize : FSE_NCOUNTBOUND;  /* maxSymbolValue==0 ? use default */
-}
-
-static size_t
-FSE_writeNCount_generic (void* header, size_t headerBufferSize,
-                   const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog,
-                         unsigned writeIsSafe)
-{
-    BYTE* const ostart = (BYTE*) header;
-    BYTE* out = ostart;
-    BYTE* const oend = ostart + headerBufferSize;
-    int nbBits;
-    const int tableSize = 1 << tableLog;
-    int remaining;
-    int threshold;
-    U32 bitStream = 0;
-    int bitCount = 0;
-    unsigned symbol = 0;
-    unsigned const alphabetSize = maxSymbolValue + 1;
-    int previousIs0 = 0;
-
-    /* Table Size */
-    bitStream += (tableLog-FSE_MIN_TABLELOG) << bitCount;
-    bitCount  += 4;
-
-    /* Init */
-    remaining = tableSize+1;   /* +1 for extra accuracy */
-    threshold = tableSize;
-    nbBits = tableLog+1;
-
-    while ((symbol < alphabetSize) && (remaining>1)) {  /* stops at 1 */
-        if (previousIs0) {
-            unsigned start = symbol;
-            while ((symbol < alphabetSize) && !normalizedCounter[symbol]) symbol++;
-            if (symbol == alphabetSize) break;   /* incorrect distribution */
-            while (symbol >= start+24) {
-                start+=24;
-                bitStream += 0xFFFFU << bitCount;
-                if ((!writeIsSafe) && (out > oend-2))
-                    return ERROR(dstSize_tooSmall);   /* Buffer overflow */
-                out[0] = (BYTE) bitStream;
-                out[1] = (BYTE)(bitStream>>8);
-                out+=2;
-                bitStream>>=16;
-            }
-            while (symbol >= start+3) {
-                start+=3;
-                bitStream += 3 << bitCount;
-                bitCount += 2;
-            }
-            bitStream += (symbol-start) << bitCount;
-            bitCount += 2;
-            if (bitCount>16) {
-                if ((!writeIsSafe) && (out > oend - 2))
-                    return ERROR(dstSize_tooSmall);   /* Buffer overflow */
-                out[0] = (BYTE)bitStream;
-                out[1] = (BYTE)(bitStream>>8);
-                out += 2;
-                bitStream >>= 16;
-                bitCount -= 16;
-        }   }
-        {   int count = normalizedCounter[symbol++];
-            int const max = (2*threshold-1) - remaining;
-            remaining -= count < 0 ? -count : count;
-            count++;   /* +1 for extra accuracy */
-            if (count>=threshold)
-                count += max;   /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */
-            bitStream += count << bitCount;
-            bitCount  += nbBits;
-            bitCount  -= (count<max);
-            previousIs0  = (count==1);
-            if (remaining<1) return ERROR(GENERIC);
-            while (remaining<threshold) { nbBits--; threshold>>=1; }
-        }
-        if (bitCount>16) {
-            if ((!writeIsSafe) && (out > oend - 2))
-                return ERROR(dstSize_tooSmall);   /* Buffer overflow */
-            out[0] = (BYTE)bitStream;
-            out[1] = (BYTE)(bitStream>>8);
-            out += 2;
-            bitStream >>= 16;
-            bitCount -= 16;
-    }   }
-
-    if (remaining != 1)
-        return ERROR(GENERIC);  /* incorrect normalized distribution */
-    assert(symbol <= alphabetSize);
-
-    /* flush remaining bitStream */
-    if ((!writeIsSafe) && (out > oend - 2))
-        return ERROR(dstSize_tooSmall);   /* Buffer overflow */
-    out[0] = (BYTE)bitStream;
-    out[1] = (BYTE)(bitStream>>8);
-    out+= (bitCount+7) /8;
-
-    return (out-ostart);
-}
-
-
-size_t FSE_writeNCount (void* buffer, size_t bufferSize,
-                  const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);   /* Unsupported */
-    if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC);   /* Unsupported */
-
-    if (bufferSize < FSE_NCountWriteBound(maxSymbolValue, tableLog))
-        return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 0);
-
-    return FSE_writeNCount_generic(buffer, bufferSize, normalizedCounter, maxSymbolValue, tableLog, 1 /* write in buffer is safe */);
-}
-
-
-/*-**************************************************************
-*  FSE Compression Code
-****************************************************************/
-
-FSE_CTable* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog)
-{
-    size_t size;
-    if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
-    size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32);
-    return (FSE_CTable*)malloc(size);
-}
-
-void FSE_freeCTable (FSE_CTable* ct) { free(ct); }
-
-/* provides the minimum logSize to safely represent a distribution */
-static unsigned FSE_minTableLog(size_t srcSize, unsigned maxSymbolValue)
-{
-    U32 minBitsSrc = BIT_highbit32((U32)(srcSize)) + 1;
-    U32 minBitsSymbols = BIT_highbit32(maxSymbolValue) + 2;
-    U32 minBits = minBitsSrc < minBitsSymbols ? minBitsSrc : minBitsSymbols;
-    assert(srcSize > 1); /* Not supported, RLE should be used instead */
-    return minBits;
-}
-
-unsigned FSE_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus)
-{
-    U32 maxBitsSrc = BIT_highbit32((U32)(srcSize - 1)) - minus;
-    U32 tableLog = maxTableLog;
-    U32 minBits = FSE_minTableLog(srcSize, maxSymbolValue);
-    assert(srcSize > 1); /* Not supported, RLE should be used instead */
-    if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
-    if (maxBitsSrc < tableLog) tableLog = maxBitsSrc;   /* Accuracy can be reduced */
-    if (minBits > tableLog) tableLog = minBits;   /* Need a minimum to safely represent all symbol values */
-    if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG;
-    if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG;
-    return tableLog;
-}
-
-unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue)
-{
-    return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 2);
-}
-
-
-/* Secondary normalization method.
-   To be used when primary method fails. */
-
-static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count, size_t total, U32 maxSymbolValue)
-{
-    short const NOT_YET_ASSIGNED = -2;
-    U32 s;
-    U32 distributed = 0;
-    U32 ToDistribute;
-
-    /* Init */
-    U32 const lowThreshold = (U32)(total >> tableLog);
-    U32 lowOne = (U32)((total * 3) >> (tableLog + 1));
-
-    for (s=0; s<=maxSymbolValue; s++) {
-        if (count[s] == 0) {
-            norm[s]=0;
-            continue;
-        }
-        if (count[s] <= lowThreshold) {
-            norm[s] = -1;
-            distributed++;
-            total -= count[s];
-            continue;
-        }
-        if (count[s] <= lowOne) {
-            norm[s] = 1;
-            distributed++;
-            total -= count[s];
-            continue;
-        }
-
-        norm[s]=NOT_YET_ASSIGNED;
-    }
-    ToDistribute = (1 << tableLog) - distributed;
-
-    if (ToDistribute == 0)
-        return 0;
-
-    if ((total / ToDistribute) > lowOne) {
-        /* risk of rounding to zero */
-        lowOne = (U32)((total * 3) / (ToDistribute * 2));
-        for (s=0; s<=maxSymbolValue; s++) {
-            if ((norm[s] == NOT_YET_ASSIGNED) && (count[s] <= lowOne)) {
-                norm[s] = 1;
-                distributed++;
-                total -= count[s];
-                continue;
-        }   }
-        ToDistribute = (1 << tableLog) - distributed;
-    }
-
-    if (distributed == maxSymbolValue+1) {
-        /* all values are pretty poor;
-           probably incompressible data (should have already been detected);
-           find max, then give all remaining points to max */
-        U32 maxV = 0, maxC = 0;
-        for (s=0; s<=maxSymbolValue; s++)
-            if (count[s] > maxC) { maxV=s; maxC=count[s]; }
-        norm[maxV] += (short)ToDistribute;
-        return 0;
-    }
-
-    if (total == 0) {
-        /* all of the symbols were low enough for the lowOne or lowThreshold */
-        for (s=0; ToDistribute > 0; s = (s+1)%(maxSymbolValue+1))
-            if (norm[s] > 0) { ToDistribute--; norm[s]++; }
-        return 0;
-    }
-
-    {   U64 const vStepLog = 62 - tableLog;
-        U64 const mid = (1ULL << (vStepLog-1)) - 1;
-        U64 const rStep = ((((U64)1<<vStepLog) * ToDistribute) + mid) / total;   /* scale on remaining */
-        U64 tmpTotal = mid;
-        for (s=0; s<=maxSymbolValue; s++) {
-            if (norm[s]==NOT_YET_ASSIGNED) {
-                U64 const end = tmpTotal + (count[s] * rStep);
-                U32 const sStart = (U32)(tmpTotal >> vStepLog);
-                U32 const sEnd = (U32)(end >> vStepLog);
-                U32 const weight = sEnd - sStart;
-                if (weight < 1)
-                    return ERROR(GENERIC);
-                norm[s] = (short)weight;
-                tmpTotal = end;
-    }   }   }
-
-    return 0;
-}
-
-
-size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog,
-                           const unsigned* count, size_t total,
-                           unsigned maxSymbolValue)
-{
-    /* Sanity checks */
-    if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG;
-    if (tableLog < FSE_MIN_TABLELOG) return ERROR(GENERIC);   /* Unsupported size */
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);   /* Unsupported size */
-    if (tableLog < FSE_minTableLog(total, maxSymbolValue)) return ERROR(GENERIC);   /* Too small tableLog, compression potentially impossible */
-
-    {   static U32 const rtbTable[] = {     0, 473195, 504333, 520860, 550000, 700000, 750000, 830000 };
-        U64 const scale = 62 - tableLog;
-        U64 const step = ((U64)1<<62) / total;   /* <== here, one division ! */
-        U64 const vStep = 1ULL<<(scale-20);
-        int stillToDistribute = 1<<tableLog;
-        unsigned s;
-        unsigned largest=0;
-        short largestP=0;
-        U32 lowThreshold = (U32)(total >> tableLog);
-
-        for (s=0; s<=maxSymbolValue; s++) {
-            if (count[s] == total) return 0;   /* rle special case */
-            if (count[s] == 0) { normalizedCounter[s]=0; continue; }
-            if (count[s] <= lowThreshold) {
-                normalizedCounter[s] = -1;
-                stillToDistribute--;
-            } else {
-                short proba = (short)((count[s]*step) >> scale);
-                if (proba<8) {
-                    U64 restToBeat = vStep * rtbTable[proba];
-                    proba += (count[s]*step) - ((U64)proba<<scale) > restToBeat;
-                }
-                if (proba > largestP) { largestP=proba; largest=s; }
-                normalizedCounter[s] = proba;
-                stillToDistribute -= proba;
-        }   }
-        if (-stillToDistribute >= (normalizedCounter[largest] >> 1)) {
-            /* corner case, need another normalization method */
-            size_t const errorCode = FSE_normalizeM2(normalizedCounter, tableLog, count, total, maxSymbolValue);
-            if (FSE_isError(errorCode)) return errorCode;
-        }
-        else normalizedCounter[largest] += (short)stillToDistribute;
-    }
-
-#if 0
-    {   /* Print Table (debug) */
-        U32 s;
-        U32 nTotal = 0;
-        for (s=0; s<=maxSymbolValue; s++)
-            RAWLOG(2, "%3i: %4i \n", s, normalizedCounter[s]);
-        for (s=0; s<=maxSymbolValue; s++)
-            nTotal += abs(normalizedCounter[s]);
-        if (nTotal != (1U<<tableLog))
-            RAWLOG(2, "Warning !!! Total == %u != %u !!!", nTotal, 1U<<tableLog);
-        getchar();
-    }
-#endif
-
-    return tableLog;
-}
-
-
-/* fake FSE_CTable, for raw (uncompressed) input */
-size_t FSE_buildCTable_raw (FSE_CTable* ct, unsigned nbBits)
-{
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    void* const ptr = ct;
-    U16* const tableU16 = ( (U16*) ptr) + 2;
-    void* const FSCT = ((U32*)ptr) + 1 /* header */ + (tableSize>>1);   /* assumption : tableLog >= 1 */
-    FSE_symbolCompressionTransform* const symbolTT = (FSE_symbolCompressionTransform*) (FSCT);
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);             /* min size */
-
-    /* header */
-    tableU16[-2] = (U16) nbBits;
-    tableU16[-1] = (U16) maxSymbolValue;
-
-    /* Build table */
-    for (s=0; s<tableSize; s++)
-        tableU16[s] = (U16)(tableSize + s);
-
-    /* Build Symbol Transformation Table */
-    {   const U32 deltaNbBits = (nbBits << 16) - (1 << nbBits);
-        for (s=0; s<=maxSymbolValue; s++) {
-            symbolTT[s].deltaNbBits = deltaNbBits;
-            symbolTT[s].deltaFindState = s-1;
-    }   }
-
-    return 0;
-}
-
-/* fake FSE_CTable, for rle input (always same symbol) */
-size_t FSE_buildCTable_rle (FSE_CTable* ct, BYTE symbolValue)
-{
-    void* ptr = ct;
-    U16* tableU16 = ( (U16*) ptr) + 2;
-    void* FSCTptr = (U32*)ptr + 2;
-    FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) FSCTptr;
-
-    /* header */
-    tableU16[-2] = (U16) 0;
-    tableU16[-1] = (U16) symbolValue;
-
-    /* Build table */
-    tableU16[0] = 0;
-    tableU16[1] = 0;   /* just in case */
-
-    /* Build Symbol Transformation Table */
-    symbolTT[symbolValue].deltaNbBits = 0;
-    symbolTT[symbolValue].deltaFindState = 0;
-
-    return 0;
-}
-
-
-static size_t FSE_compress_usingCTable_generic (void* dst, size_t dstSize,
-                           const void* src, size_t srcSize,
-                           const FSE_CTable* ct, const unsigned fast)
-{
-    const BYTE* const istart = (const BYTE*) src;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* ip=iend;
-
-    BIT_CStream_t bitC;
-    FSE_CState_t CState1, CState2;
-
-    /* init */
-    if (srcSize <= 2) return 0;
-    { size_t const initError = BIT_initCStream(&bitC, dst, dstSize);
-      if (FSE_isError(initError)) return 0; /* not enough space available to write a bitstream */ }
-
-#define FSE_FLUSHBITS(s)  (fast ? BIT_flushBitsFast(s) : BIT_flushBits(s))
-
-    if (srcSize & 1) {
-        FSE_initCState2(&CState1, ct, *--ip);
-        FSE_initCState2(&CState2, ct, *--ip);
-        FSE_encodeSymbol(&bitC, &CState1, *--ip);
-        FSE_FLUSHBITS(&bitC);
-    } else {
-        FSE_initCState2(&CState2, ct, *--ip);
-        FSE_initCState2(&CState1, ct, *--ip);
-    }
-
-    /* join to mod 4 */
-    srcSize -= 2;
-    if ((sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) {  /* test bit 2 */
-        FSE_encodeSymbol(&bitC, &CState2, *--ip);
-        FSE_encodeSymbol(&bitC, &CState1, *--ip);
-        FSE_FLUSHBITS(&bitC);
-    }
-
-    /* 2 or 4 encoding per loop */
-    while ( ip>istart ) {
-
-        FSE_encodeSymbol(&bitC, &CState2, *--ip);
-
-        if (sizeof(bitC.bitContainer)*8 < FSE_MAX_TABLELOG*2+7 )   /* this test must be static */
-            FSE_FLUSHBITS(&bitC);
-
-        FSE_encodeSymbol(&bitC, &CState1, *--ip);
-
-        if (sizeof(bitC.bitContainer)*8 > FSE_MAX_TABLELOG*4+7 ) {  /* this test must be static */
-            FSE_encodeSymbol(&bitC, &CState2, *--ip);
-            FSE_encodeSymbol(&bitC, &CState1, *--ip);
-        }
-
-        FSE_FLUSHBITS(&bitC);
-    }
-
-    FSE_flushCState(&bitC, &CState2);
-    FSE_flushCState(&bitC, &CState1);
-    return BIT_closeCStream(&bitC);
-}
-
-size_t FSE_compress_usingCTable (void* dst, size_t dstSize,
-                           const void* src, size_t srcSize,
-                           const FSE_CTable* ct)
-{
-    unsigned const fast = (dstSize >= FSE_BLOCKBOUND(srcSize));
-
-    if (fast)
-        return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 1);
-    else
-        return FSE_compress_usingCTable_generic(dst, dstSize, src, srcSize, ct, 0);
-}
-
-
-size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); }
-
-#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e
-#define CHECK_F(f)   { CHECK_V_F(_var_err__, f); }
-
-/* FSE_compress_wksp() :
- * Same as FSE_compress2(), but using an externally allocated scratch buffer (`workSpace`).
- * `wkspSize` size must be `(1<<tableLog)`.
- */
-size_t FSE_compress_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + dstSize;
-
-    unsigned count[FSE_MAX_SYMBOL_VALUE+1];
-    S16   norm[FSE_MAX_SYMBOL_VALUE+1];
-    FSE_CTable* CTable = (FSE_CTable*)workSpace;
-    size_t const CTableSize = FSE_CTABLE_SIZE_U32(tableLog, maxSymbolValue);
-    void* scratchBuffer = (void*)(CTable + CTableSize);
-    size_t const scratchBufferSize = wkspSize - (CTableSize * sizeof(FSE_CTable));
-
-    /* init conditions */
-    if (wkspSize < FSE_WKSP_SIZE_U32(tableLog, maxSymbolValue)) return ERROR(tableLog_tooLarge);
-    if (srcSize <= 1) return 0;  /* Not compressible */
-    if (!maxSymbolValue) maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-    if (!tableLog) tableLog = FSE_DEFAULT_TABLELOG;
-
-    /* Scan input and build symbol stats */
-    {   CHECK_V_F(maxCount, HIST_count_wksp(count, &maxSymbolValue, src, srcSize, scratchBuffer, scratchBufferSize) );
-        if (maxCount == srcSize) return 1;   /* only a single symbol in src : rle */
-        if (maxCount == 1) return 0;         /* each symbol present maximum once => not compressible */
-        if (maxCount < (srcSize >> 7)) return 0;   /* Heuristic : not compressible enough */
-    }
-
-    tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue);
-    CHECK_F( FSE_normalizeCount(norm, tableLog, count, srcSize, maxSymbolValue) );
-
-    /* Write table description header */
-    {   CHECK_V_F(nc_err, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) );
-        op += nc_err;
-    }
-
-    /* Compress */
-    CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, scratchBufferSize) );
-    {   CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, src, srcSize, CTable) );
-        if (cSize == 0) return 0;   /* not enough space for compressed data */
-        op += cSize;
-    }
-
-    /* check compressibility */
-    if ( (size_t)(op-ostart) >= srcSize-1 ) return 0;
-
-    return op-ostart;
-}
-
-typedef struct {
-    FSE_CTable CTable_max[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)];
-    BYTE scratchBuffer[1 << FSE_MAX_TABLELOG];
-} fseWkspMax_t;
-
-size_t FSE_compress2 (void* dst, size_t dstCapacity, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog)
-{
-    fseWkspMax_t scratchBuffer;
-    DEBUG_STATIC_ASSERT(sizeof(scratchBuffer) >= FSE_WKSP_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE));   /* compilation failures here means scratchBuffer is not large enough */
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    return FSE_compress_wksp(dst, dstCapacity, src, srcSize, maxSymbolValue, tableLog, &scratchBuffer, sizeof(scratchBuffer));
-}
-
-size_t FSE_compress (void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    return FSE_compress2(dst, dstCapacity, src, srcSize, FSE_MAX_SYMBOL_VALUE, FSE_DEFAULT_TABLELOG);
-}
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
diff --git a/vendor/github.com/DataDog/zstd/fse_decompress.c b/vendor/github.com/DataDog/zstd/fse_decompress.c
deleted file mode 100644
index 72bbead..0000000
--- a/vendor/github.com/DataDog/zstd/fse_decompress.c
+++ /dev/null
@@ -1,309 +0,0 @@
-/* ******************************************************************
-   FSE : Finite State Entropy decoder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include "bitstream.h"
-#include "compiler.h"
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_isError ERR_isError
-#define FSE_STATIC_ASSERT(c) DEBUG_STATIC_ASSERT(c)   /* use only *after* variable declarations */
-
-/* check and forward error code */
-#define CHECK_F(f) { size_t const e = f; if (FSE_isError(e)) return e; }
-
-
-/* **************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-FSE_DTable* FSE_createDTable (unsigned tableLog)
-{
-    if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX;
-    return (FSE_DTable*)malloc( FSE_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
-}
-
-void FSE_freeDTable (FSE_DTable* dt)
-{
-    free(dt);
-}
-
-size_t FSE_buildDTable(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* const tdPtr = dt+1;   /* because *dt is unsigned, 32-bits aligned on 32-bits */
-    FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (tdPtr);
-    U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-
-    U32 const maxSV1 = maxSymbolValue + 1;
-    U32 const tableSize = 1 << tableLog;
-    U32 highThreshold = tableSize-1;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    {   FSE_DTableHeader DTableH;
-        DTableH.tableLog = (U16)tableLog;
-        DTableH.fastMode = 1;
-        {   S16 const largeLimit= (S16)(1 << (tableLog-1));
-            U32 s;
-            for (s=0; s<maxSV1; s++) {
-                if (normalizedCounter[s]==-1) {
-                    tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
-                    symbolNext[s] = 1;
-                } else {
-                    if (normalizedCounter[s] >= largeLimit) DTableH.fastMode=0;
-                    symbolNext[s] = normalizedCounter[s];
-        }   }   }
-        memcpy(dt, &DTableH, sizeof(DTableH));
-    }
-
-    /* Spread symbols */
-    {   U32 const tableMask = tableSize-1;
-        U32 const step = FSE_TABLESTEP(tableSize);
-        U32 s, position = 0;
-        for (s=0; s<maxSV1; s++) {
-            int i;
-            for (i=0; i<normalizedCounter[s]; i++) {
-                tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
-                position = (position + step) & tableMask;
-                while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }   }
-        if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-    }
-
-    /* Build Decoding table */
-    {   U32 u;
-        for (u=0; u<tableSize; u++) {
-            FSE_FUNCTION_TYPE const symbol = (FSE_FUNCTION_TYPE)(tableDecode[u].symbol);
-            U32 const nextState = symbolNext[symbol]++;
-            tableDecode[u].nbBits = (BYTE) (tableLog - BIT_highbit32(nextState) );
-            tableDecode[u].newState = (U16) ( (nextState << tableDecode[u].nbBits) - tableSize);
-    }   }
-
-    return 0;
-}
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/*-*******************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSE_decode_t* const cell = (FSE_decode_t*)dPtr;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSE_decode_t* const dinfo = (FSE_decode_t*)dPtr;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSV1 = tableMask+1;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<maxSV1; s++) {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE_TEMPLATE size_t FSE_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSE_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BIT_DStream_t bitD;
-    FSE_DState_t state1;
-    FSE_DState_t state2;
-
-    /* Init */
-    CHECK_F(BIT_initDStream(&bitD, cSrc, cSrcSize));
-
-    FSE_initDState(&state1, &bitD, dt);
-    FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BIT_reloadDStream(&bitD)==BIT_DStream_unfinished) & (op<olimit) ; op+=4) {
-        op[0] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[1] = FSE_GETSYMBOL(&state2);
-
-        if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[3] = FSE_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */
-    while (1) {
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-        *op++ = FSE_GETSYMBOL(&state1);
-        if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) {
-            *op++ = FSE_GETSYMBOL(&state2);
-            break;
-        }
-
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-        *op++ = FSE_GETSYMBOL(&state2);
-        if (BIT_reloadDStream(&bitD)==BIT_DStream_overflow) {
-            *op++ = FSE_GETSYMBOL(&state1);
-            break;
-    }   }
-
-    return op-ostart;
-}
-
-
-size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSE_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSE_DTableHeader* DTableH = (const FSE_DTableHeader*)ptr;
-    const U32 fastMode = DTableH->fastMode;
-
-    /* select fast mode (static) */
-    if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-size_t FSE_decompress_wksp(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, FSE_DTable* workSpace, unsigned maxLog)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSE_MAX_SYMBOL_VALUE+1];
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-
-    /* normal FSE decoding mode */
-    size_t const NCountLength = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSE_isError(NCountLength)) return NCountLength;
-    //if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size; supposed to be already checked in NCountLength, only remaining case : NCountLength==cSrcSize */
-    if (tableLog > maxLog) return ERROR(tableLog_tooLarge);
-    ip += NCountLength;
-    cSrcSize -= NCountLength;
-
-    CHECK_F( FSE_buildDTable (workSpace, counting, maxSymbolValue, tableLog) );
-
-    return FSE_decompress_usingDTable (dst, dstCapacity, ip, cSrcSize, workSpace);   /* always return, even if it is an error code */
-}
-
-
-typedef FSE_DTable DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-size_t FSE_decompress(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize)
-{
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    return FSE_decompress_wksp(dst, dstCapacity, cSrc, cSrcSize, dt, FSE_MAX_TABLELOG);
-}
-
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
diff --git a/vendor/github.com/DataDog/zstd/hist.c b/vendor/github.com/DataDog/zstd/hist.c
deleted file mode 100644
index 45b7bab..0000000
--- a/vendor/github.com/DataDog/zstd/hist.c
+++ /dev/null
@@ -1,203 +0,0 @@
-/* ******************************************************************
-   hist : Histogram functions
-   part of Finite State Entropy project
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* --- dependencies --- */
-#include "mem.h"             /* U32, BYTE, etc. */
-#include "debug.h"           /* assert, DEBUGLOG */
-#include "error_private.h"   /* ERROR */
-#include "hist.h"
-
-
-/* --- Error management --- */
-unsigned HIST_isError(size_t code) { return ERR_isError(code); }
-
-/*-**************************************************************
- *  Histogram functions
- ****************************************************************/
-unsigned HIST_count_simple(unsigned* count, unsigned* maxSymbolValuePtr,
-                           const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* const end = ip + srcSize;
-    unsigned maxSymbolValue = *maxSymbolValuePtr;
-    unsigned largestCount=0;
-
-    memset(count, 0, (maxSymbolValue+1) * sizeof(*count));
-    if (srcSize==0) { *maxSymbolValuePtr = 0; return 0; }
-
-    while (ip<end) {
-        assert(*ip <= maxSymbolValue);
-        count[*ip++]++;
-    }
-
-    while (!count[maxSymbolValue]) maxSymbolValue--;
-    *maxSymbolValuePtr = maxSymbolValue;
-
-    {   U32 s;
-        for (s=0; s<=maxSymbolValue; s++)
-            if (count[s] > largestCount) largestCount = count[s];
-    }
-
-    return largestCount;
-}
-
-typedef enum { trustInput, checkMaxSymbolValue } HIST_checkInput_e;
-
-/* HIST_count_parallel_wksp() :
- * store histogram into 4 intermediate tables, recombined at the end.
- * this design makes better use of OoO cpus,
- * and is noticeably faster when some values are heavily repeated.
- * But it needs some additional workspace for intermediate tables.
- * `workSpace` size must be a table of size >= HIST_WKSP_SIZE_U32.
- * @return : largest histogram frequency,
- *           or an error code (notably when histogram would be larger than *maxSymbolValuePtr). */
-static size_t HIST_count_parallel_wksp(
-                                unsigned* count, unsigned* maxSymbolValuePtr,
-                                const void* source, size_t sourceSize,
-                                HIST_checkInput_e check,
-                                U32* const workSpace)
-{
-    const BYTE* ip = (const BYTE*)source;
-    const BYTE* const iend = ip+sourceSize;
-    unsigned maxSymbolValue = *maxSymbolValuePtr;
-    unsigned max=0;
-    U32* const Counting1 = workSpace;
-    U32* const Counting2 = Counting1 + 256;
-    U32* const Counting3 = Counting2 + 256;
-    U32* const Counting4 = Counting3 + 256;
-
-    memset(workSpace, 0, 4*256*sizeof(unsigned));
-
-    /* safety checks */
-    if (!sourceSize) {
-        memset(count, 0, maxSymbolValue + 1);
-        *maxSymbolValuePtr = 0;
-        return 0;
-    }
-    if (!maxSymbolValue) maxSymbolValue = 255;            /* 0 == default */
-
-    /* by stripes of 16 bytes */
-    {   U32 cached = MEM_read32(ip); ip += 4;
-        while (ip < iend-15) {
-            U32 c = cached; cached = MEM_read32(ip); ip += 4;
-            Counting1[(BYTE) c     ]++;
-            Counting2[(BYTE)(c>>8) ]++;
-            Counting3[(BYTE)(c>>16)]++;
-            Counting4[       c>>24 ]++;
-            c = cached; cached = MEM_read32(ip); ip += 4;
-            Counting1[(BYTE) c     ]++;
-            Counting2[(BYTE)(c>>8) ]++;
-            Counting3[(BYTE)(c>>16)]++;
-            Counting4[       c>>24 ]++;
-            c = cached; cached = MEM_read32(ip); ip += 4;
-            Counting1[(BYTE) c     ]++;
-            Counting2[(BYTE)(c>>8) ]++;
-            Counting3[(BYTE)(c>>16)]++;
-            Counting4[       c>>24 ]++;
-            c = cached; cached = MEM_read32(ip); ip += 4;
-            Counting1[(BYTE) c     ]++;
-            Counting2[(BYTE)(c>>8) ]++;
-            Counting3[(BYTE)(c>>16)]++;
-            Counting4[       c>>24 ]++;
-        }
-        ip-=4;
-    }
-
-    /* finish last symbols */
-    while (ip<iend) Counting1[*ip++]++;
-
-    if (check) {   /* verify stats will fit into destination table */
-        U32 s; for (s=255; s>maxSymbolValue; s--) {
-            Counting1[s] += Counting2[s] + Counting3[s] + Counting4[s];
-            if (Counting1[s]) return ERROR(maxSymbolValue_tooSmall);
-    }   }
-
-    {   U32 s;
-        if (maxSymbolValue > 255) maxSymbolValue = 255;
-        for (s=0; s<=maxSymbolValue; s++) {
-            count[s] = Counting1[s] + Counting2[s] + Counting3[s] + Counting4[s];
-            if (count[s] > max) max = count[s];
-    }   }
-
-    while (!count[maxSymbolValue]) maxSymbolValue--;
-    *maxSymbolValuePtr = maxSymbolValue;
-    return (size_t)max;
-}
-
-/* HIST_countFast_wksp() :
- * Same as HIST_countFast(), but using an externally provided scratch buffer.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
-                          const void* source, size_t sourceSize,
-                          void* workSpace, size_t workSpaceSize)
-{
-    if (sourceSize < 1500) /* heuristic threshold */
-        return HIST_count_simple(count, maxSymbolValuePtr, source, sourceSize);
-    if ((size_t)workSpace & 3) return ERROR(GENERIC);  /* must be aligned on 4-bytes boundaries */
-    if (workSpaceSize < HIST_WKSP_SIZE) return ERROR(workSpace_tooSmall);
-    return HIST_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, trustInput, (U32*)workSpace);
-}
-
-/* fast variant (unsafe : won't check if src contains values beyond count[] limit) */
-size_t HIST_countFast(unsigned* count, unsigned* maxSymbolValuePtr,
-                     const void* source, size_t sourceSize)
-{
-    unsigned tmpCounters[HIST_WKSP_SIZE_U32];
-    return HIST_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, tmpCounters, sizeof(tmpCounters));
-}
-
-/* HIST_count_wksp() :
- * Same as HIST_count(), but using an externally provided scratch buffer.
- * `workSpace` size must be table of >= HIST_WKSP_SIZE_U32 unsigned */
-size_t HIST_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
-                       const void* source, size_t sourceSize,
-                       void* workSpace, size_t workSpaceSize)
-{
-    if ((size_t)workSpace & 3) return ERROR(GENERIC);  /* must be aligned on 4-bytes boundaries */
-    if (workSpaceSize < HIST_WKSP_SIZE) return ERROR(workSpace_tooSmall);
-    if (*maxSymbolValuePtr < 255)
-        return HIST_count_parallel_wksp(count, maxSymbolValuePtr, source, sourceSize, checkMaxSymbolValue, (U32*)workSpace);
-    *maxSymbolValuePtr = 255;
-    return HIST_countFast_wksp(count, maxSymbolValuePtr, source, sourceSize, workSpace, workSpaceSize);
-}
-
-size_t HIST_count(unsigned* count, unsigned* maxSymbolValuePtr,
-                 const void* src, size_t srcSize)
-{
-    unsigned tmpCounters[HIST_WKSP_SIZE_U32];
-    return HIST_count_wksp(count, maxSymbolValuePtr, src, srcSize, tmpCounters, sizeof(tmpCounters));
-}
diff --git a/vendor/github.com/DataDog/zstd/hist.h b/vendor/github.com/DataDog/zstd/hist.h
deleted file mode 100644
index 8b38935..0000000
--- a/vendor/github.com/DataDog/zstd/hist.h
+++ /dev/null
@@ -1,95 +0,0 @@
-/* ******************************************************************
-   hist : Histogram functions
-   part of Finite State Entropy project
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* --- dependencies --- */
-#include <stddef.h>   /* size_t */
-
-
-/* --- simple histogram functions --- */
-
-/*! HIST_count():
- *  Provides the precise count of each byte within a table 'count'.
- * 'count' is a table of unsigned int, of minimum size (*maxSymbolValuePtr+1).
- *  Updates *maxSymbolValuePtr with actual largest symbol value detected.
- * @return : count of the most frequent symbol (which isn't identified).
- *           or an error code, which can be tested using HIST_isError().
- *           note : if return == srcSize, there is only one symbol.
- */
-size_t HIST_count(unsigned* count, unsigned* maxSymbolValuePtr,
-                  const void* src, size_t srcSize);
-
-unsigned HIST_isError(size_t code);  /**< tells if a return value is an error code */
-
-
-/* --- advanced histogram functions --- */
-
-#define HIST_WKSP_SIZE_U32 1024
-#define HIST_WKSP_SIZE    (HIST_WKSP_SIZE_U32 * sizeof(unsigned))
-/** HIST_count_wksp() :
- *  Same as HIST_count(), but using an externally provided scratch buffer.
- *  Benefit is this function will use very little stack space.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_count_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
-                       const void* src, size_t srcSize,
-                       void* workSpace, size_t workSpaceSize);
-
-/** HIST_countFast() :
- *  same as HIST_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr.
- *  This function is unsafe, and will segfault if any value within `src` is `> *maxSymbolValuePtr`
- */
-size_t HIST_countFast(unsigned* count, unsigned* maxSymbolValuePtr,
-                      const void* src, size_t srcSize);
-
-/** HIST_countFast_wksp() :
- *  Same as HIST_countFast(), but using an externally provided scratch buffer.
- * `workSpace` is a writable buffer which must be 4-bytes aligned,
- * `workSpaceSize` must be >= HIST_WKSP_SIZE
- */
-size_t HIST_countFast_wksp(unsigned* count, unsigned* maxSymbolValuePtr,
-                           const void* src, size_t srcSize,
-                           void* workSpace, size_t workSpaceSize);
-
-/*! HIST_count_simple() :
- *  Same as HIST_countFast(), this function is unsafe,
- *  and will segfault if any value within `src` is `> *maxSymbolValuePtr`.
- *  It is also a bit slower for large inputs.
- *  However, it does not need any additional memory (not even on stack).
- * @return : count of the most frequent symbol.
- *  Note this function doesn't produce any error (i.e. it must succeed).
- */
-unsigned HIST_count_simple(unsigned* count, unsigned* maxSymbolValuePtr,
-                           const void* src, size_t srcSize);
diff --git a/vendor/github.com/DataDog/zstd/huf.h b/vendor/github.com/DataDog/zstd/huf.h
deleted file mode 100644
index 6b572c4..0000000
--- a/vendor/github.com/DataDog/zstd/huf.h
+++ /dev/null
@@ -1,358 +0,0 @@
-/* ******************************************************************
-   huff0 huffman codec,
-   part of Finite State Entropy library
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef HUF_H_298734234
-#define HUF_H_298734234
-
-/* *** Dependencies *** */
-#include <stddef.h>    /* size_t */
-
-
-/* *** library symbols visibility *** */
-/* Note : when linking with -fvisibility=hidden on gcc, or by default on Visual,
- *        HUF symbols remain "private" (internal symbols for library only).
- *        Set macro FSE_DLL_EXPORT to 1 if you want HUF symbols visible on DLL interface */
-#if defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1) && defined(__GNUC__) && (__GNUC__ >= 4)
-#  define HUF_PUBLIC_API __attribute__ ((visibility ("default")))
-#elif defined(FSE_DLL_EXPORT) && (FSE_DLL_EXPORT==1)   /* Visual expected */
-#  define HUF_PUBLIC_API __declspec(dllexport)
-#elif defined(FSE_DLL_IMPORT) && (FSE_DLL_IMPORT==1)
-#  define HUF_PUBLIC_API __declspec(dllimport)  /* not required, just to generate faster code (saves a function pointer load from IAT and an indirect jump) */
-#else
-#  define HUF_PUBLIC_API
-#endif
-
-
-/* ========================== */
-/* ***  simple functions  *** */
-/* ========================== */
-
-/** HUF_compress() :
- *  Compress content from buffer 'src', of size 'srcSize', into buffer 'dst'.
- * 'dst' buffer must be already allocated.
- *  Compression runs faster if `dstCapacity` >= HUF_compressBound(srcSize).
- * `srcSize` must be <= `HUF_BLOCKSIZE_MAX` == 128 KB.
- * @return : size of compressed data (<= `dstCapacity`).
- *  Special values : if return == 0, srcData is not compressible => Nothing is stored within dst !!!
- *                   if HUF_isError(return), compression failed (more details using HUF_getErrorName())
- */
-HUF_PUBLIC_API size_t HUF_compress(void* dst, size_t dstCapacity,
-                             const void* src, size_t srcSize);
-
-/** HUF_decompress() :
- *  Decompress HUF data from buffer 'cSrc', of size 'cSrcSize',
- *  into already allocated buffer 'dst', of minimum size 'dstSize'.
- * `originalSize` : **must** be the ***exact*** size of original (uncompressed) data.
- *  Note : in contrast with FSE, HUF_decompress can regenerate
- *         RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data,
- *         because it knows size to regenerate (originalSize).
- * @return : size of regenerated data (== originalSize),
- *           or an error code, which can be tested using HUF_isError()
- */
-HUF_PUBLIC_API size_t HUF_decompress(void* dst,  size_t originalSize,
-                               const void* cSrc, size_t cSrcSize);
-
-
-/* ***   Tool functions *** */
-#define HUF_BLOCKSIZE_MAX (128 * 1024)                  /**< maximum input size for a single block compressed with HUF_compress */
-HUF_PUBLIC_API size_t HUF_compressBound(size_t size);   /**< maximum compressed size (worst case) */
-
-/* Error Management */
-HUF_PUBLIC_API unsigned    HUF_isError(size_t code);       /**< tells if a return value is an error code */
-HUF_PUBLIC_API const char* HUF_getErrorName(size_t code);  /**< provides error code string (useful for debugging) */
-
-
-/* ***   Advanced function   *** */
-
-/** HUF_compress2() :
- *  Same as HUF_compress(), but offers control over `maxSymbolValue` and `tableLog`.
- * `maxSymbolValue` must be <= HUF_SYMBOLVALUE_MAX .
- * `tableLog` must be `<= HUF_TABLELOG_MAX` . */
-HUF_PUBLIC_API size_t HUF_compress2 (void* dst, size_t dstCapacity,
-                               const void* src, size_t srcSize,
-                               unsigned maxSymbolValue, unsigned tableLog);
-
-/** HUF_compress4X_wksp() :
- *  Same as HUF_compress2(), but uses externally allocated `workSpace`.
- * `workspace` must have minimum alignment of 4, and be at least as large as HUF_WORKSPACE_SIZE */
-#define HUF_WORKSPACE_SIZE (6 << 10)
-#define HUF_WORKSPACE_SIZE_U32 (HUF_WORKSPACE_SIZE / sizeof(U32))
-HUF_PUBLIC_API size_t HUF_compress4X_wksp (void* dst, size_t dstCapacity,
-                                     const void* src, size_t srcSize,
-                                     unsigned maxSymbolValue, unsigned tableLog,
-                                     void* workSpace, size_t wkspSize);
-
-#endif   /* HUF_H_298734234 */
-
-/* ******************************************************************
- *  WARNING !!
- *  The following section contains advanced and experimental definitions
- *  which shall never be used in the context of a dynamic library,
- *  because they are not guaranteed to remain stable in the future.
- *  Only consider them in association with static linking.
- * *****************************************************************/
-#if defined(HUF_STATIC_LINKING_ONLY) && !defined(HUF_H_HUF_STATIC_LINKING_ONLY)
-#define HUF_H_HUF_STATIC_LINKING_ONLY
-
-/* *** Dependencies *** */
-#include "mem.h"   /* U32 */
-
-
-/* *** Constants *** */
-#define HUF_TABLELOG_MAX      12      /* max runtime value of tableLog (due to static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */
-#define HUF_TABLELOG_DEFAULT  11      /* default tableLog value when none specified */
-#define HUF_SYMBOLVALUE_MAX  255
-
-#define HUF_TABLELOG_ABSOLUTEMAX  15  /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#if (HUF_TABLELOG_MAX > HUF_TABLELOG_ABSOLUTEMAX)
-#  error "HUF_TABLELOG_MAX is too large !"
-#endif
-
-
-/* ****************************************
-*  Static allocation
-******************************************/
-/* HUF buffer bounds */
-#define HUF_CTABLEBOUND 129
-#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8)   /* only true when incompressible is pre-filtered with fast heuristic */
-#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* static allocation of HUF's Compression Table */
-#define HUF_CTABLE_SIZE_U32(maxSymbolValue)   ((maxSymbolValue)+1)   /* Use tables of U32, for proper alignment */
-#define HUF_CTABLE_SIZE(maxSymbolValue)       (HUF_CTABLE_SIZE_U32(maxSymbolValue) * sizeof(U32))
-#define HUF_CREATE_STATIC_CTABLE(name, maxSymbolValue) \
-    U32 name##hb[HUF_CTABLE_SIZE_U32(maxSymbolValue)]; \
-    void* name##hv = &(name##hb); \
-    HUF_CElt* name = (HUF_CElt*)(name##hv)   /* no final ; */
-
-/* static allocation of HUF's DTable */
-typedef U32 HUF_DTable;
-#define HUF_DTABLE_SIZE(maxTableLog)   (1 + (1<<(maxTableLog)))
-#define HUF_CREATE_STATIC_DTABLEX1(DTable, maxTableLog) \
-        HUF_DTable DTable[HUF_DTABLE_SIZE((maxTableLog)-1)] = { ((U32)((maxTableLog)-1) * 0x01000001) }
-#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        HUF_DTable DTable[HUF_DTABLE_SIZE(maxTableLog)] = { ((U32)(maxTableLog) * 0x01000001) }
-
-
-/* ****************************************
-*  Advanced decompression functions
-******************************************/
-size_t HUF_decompress4X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-#endif
-
-size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< decodes RLE and uncompressed */
-size_t HUF_decompress4X_hufOnly(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< considers RLE and uncompressed as errors */
-size_t HUF_decompress4X_hufOnly_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize); /**< considers RLE and uncompressed as errors */
-size_t HUF_decompress4X1_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-size_t HUF_decompress4X1_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);   /**< single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-size_t HUF_decompress4X2_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);   /**< double-symbols decoder */
-#endif
-
-
-/* ****************************************
- *  HUF detailed API
- * ****************************************/
-
-/*! HUF_compress() does the following:
- *  1. count symbol occurrence from source[] into table count[] using FSE_count() (exposed within "fse.h")
- *  2. (optional) refine tableLog using HUF_optimalTableLog()
- *  3. build Huffman table from count using HUF_buildCTable()
- *  4. save Huffman table to memory buffer using HUF_writeCTable()
- *  5. encode the data stream using HUF_compress4X_usingCTable()
- *
- *  The following API allows targeting specific sub-functions for advanced tasks.
- *  For example, it's possible to compress several blocks using the same 'CTable',
- *  or to save and regenerate 'CTable' using external methods.
- */
-unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue);
-typedef struct HUF_CElt_s HUF_CElt;   /* incomplete type */
-size_t HUF_buildCTable (HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits);   /* @return : maxNbBits; CTable and count can overlap. In which case, CTable will overwrite count content */
-size_t HUF_writeCTable (void* dst, size_t maxDstSize, const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog);
-size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable);
-
-typedef enum {
-   HUF_repeat_none,  /**< Cannot use the previous table */
-   HUF_repeat_check, /**< Can use the previous table but it must be checked. Note : The previous table must have been constructed by HUF_compress{1, 4}X_repeat */
-   HUF_repeat_valid  /**< Can use the previous table and it is assumed to be valid */
- } HUF_repeat;
-/** HUF_compress4X_repeat() :
- *  Same as HUF_compress4X_wksp(), but considers using hufTable if *repeat != HUF_repeat_none.
- *  If it uses hufTable it does not modify hufTable or repeat.
- *  If it doesn't, it sets *repeat = HUF_repeat_none, and it sets hufTable to the table used.
- *  If preferRepeat then the old table will always be used if valid. */
-size_t HUF_compress4X_repeat(void* dst, size_t dstSize,
-                       const void* src, size_t srcSize,
-                       unsigned maxSymbolValue, unsigned tableLog,
-                       void* workSpace, size_t wkspSize,    /**< `workSpace` must be aligned on 4-bytes boundaries, `wkspSize` must be >= HUF_WORKSPACE_SIZE */
-                       HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2);
-
-/** HUF_buildCTable_wksp() :
- *  Same as HUF_buildCTable(), but using externally allocated scratch buffer.
- * `workSpace` must be aligned on 4-bytes boundaries, and its size must be >= HUF_CTABLE_WORKSPACE_SIZE.
- */
-#define HUF_CTABLE_WORKSPACE_SIZE_U32 (2*HUF_SYMBOLVALUE_MAX +1 +1)
-#define HUF_CTABLE_WORKSPACE_SIZE (HUF_CTABLE_WORKSPACE_SIZE_U32 * sizeof(unsigned))
-size_t HUF_buildCTable_wksp (HUF_CElt* tree,
-                       const unsigned* count, U32 maxSymbolValue, U32 maxNbBits,
-                             void* workSpace, size_t wkspSize);
-
-/*! HUF_readStats() :
- *  Read compact Huffman tree, saved by HUF_writeCTable().
- * `huffWeight` is destination buffer.
- * @return : size read from `src` , or an error Code .
- *  Note : Needed by HUF_readCTable() and HUF_readDTableXn() . */
-size_t HUF_readStats(BYTE* huffWeight, size_t hwSize,
-                     U32* rankStats, U32* nbSymbolsPtr, U32* tableLogPtr,
-                     const void* src, size_t srcSize);
-
-/** HUF_readCTable() :
- *  Loading a CTable saved with HUF_writeCTable() */
-size_t HUF_readCTable (HUF_CElt* CTable, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize);
-
-/** HUF_getNbBits() :
- *  Read nbBits from CTable symbolTable, for symbol `symbolValue` presumed <= HUF_SYMBOLVALUE_MAX
- *  Note 1 : is not inlined, as HUF_CElt definition is private
- *  Note 2 : const void* used, so that it can provide a statically allocated table as argument (which uses type U32) */
-U32 HUF_getNbBits(const void* symbolTable, U32 symbolValue);
-
-/*
- * HUF_decompress() does the following:
- * 1. select the decompression algorithm (X1, X2) based on pre-computed heuristics
- * 2. build Huffman table from save, using HUF_readDTableX?()
- * 3. decode 1 or 4 segments in parallel using HUF_decompress?X?_usingDTable()
- */
-
-/** HUF_selectDecoder() :
- *  Tells which decoder is likely to decode faster,
- *  based on a set of pre-computed metrics.
- * @return : 0==HUF_decompress4X1, 1==HUF_decompress4X2 .
- *  Assumption : 0 < dstSize <= 128 KB */
-U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize);
-
-/**
- *  The minimum workspace size for the `workSpace` used in
- *  HUF_readDTableX1_wksp() and HUF_readDTableX2_wksp().
- *
- *  The space used depends on HUF_TABLELOG_MAX, ranging from ~1500 bytes when
- *  HUF_TABLE_LOG_MAX=12 to ~1850 bytes when HUF_TABLE_LOG_MAX=15.
- *  Buffer overflow errors may potentially occur if code modifications result in
- *  a required workspace size greater than that specified in the following
- *  macro.
- */
-#define HUF_DECOMPRESS_WORKSPACE_SIZE (2 << 10)
-#define HUF_DECOMPRESS_WORKSPACE_SIZE_U32 (HUF_DECOMPRESS_WORKSPACE_SIZE / sizeof(U32))
-
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_readDTableX1 (HUF_DTable* DTable, const void* src, size_t srcSize);
-size_t HUF_readDTableX1_wksp (HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_readDTableX2 (HUF_DTable* DTable, const void* src, size_t srcSize);
-size_t HUF_readDTableX2_wksp (HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize);
-#endif
-
-size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress4X1_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-
-
-/* ====================== */
-/* single stream variants */
-/* ====================== */
-
-size_t HUF_compress1X (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
-size_t HUF_compress1X_wksp (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog, void* workSpace, size_t wkspSize);  /**< `workSpace` must be a table of at least HUF_WORKSPACE_SIZE_U32 unsigned */
-size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable);
-/** HUF_compress1X_repeat() :
- *  Same as HUF_compress1X_wksp(), but considers using hufTable if *repeat != HUF_repeat_none.
- *  If it uses hufTable it does not modify hufTable or repeat.
- *  If it doesn't, it sets *repeat = HUF_repeat_none, and it sets hufTable to the table used.
- *  If preferRepeat then the old table will always be used if valid. */
-size_t HUF_compress1X_repeat(void* dst, size_t dstSize,
-                       const void* src, size_t srcSize,
-                       unsigned maxSymbolValue, unsigned tableLog,
-                       void* workSpace, size_t wkspSize,   /**< `workSpace` must be aligned on 4-bytes boundaries, `wkspSize` must be >= HUF_WORKSPACE_SIZE */
-                       HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2);
-
-size_t HUF_decompress1X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbol decoder */
-#endif
-
-size_t HUF_decompress1X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-size_t HUF_decompress1X_DCtx_wksp (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-size_t HUF_decompress1X1_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);   /**< single-symbol decoder */
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-size_t HUF_decompress1X2_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize);   /**< double-symbols decoder */
-#endif
-
-size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);   /**< automatic selection of sing or double symbol decoder, based on DTable */
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-#ifndef HUF_FORCE_DECOMPRESS_X1
-size_t HUF_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable);
-#endif
-
-/* BMI2 variants.
- * If the CPU has BMI2 support, pass bmi2=1, otherwise pass bmi2=0.
- */
-size_t HUF_decompress1X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2);
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_DCtx_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2);
-#endif
-size_t HUF_decompress4X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2);
-size_t HUF_decompress4X_hufOnly_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2);
-
-#endif /* HUF_STATIC_LINKING_ONLY */
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/huf_compress.c b/vendor/github.com/DataDog/zstd/huf_compress.c
deleted file mode 100644
index f074f1e..0000000
--- a/vendor/github.com/DataDog/zstd/huf_compress.c
+++ /dev/null
@@ -1,798 +0,0 @@
-/* ******************************************************************
-   Huffman encoder, part of New Generation Entropy library
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-#include "compiler.h"
-#include "bitstream.h"
-#include "hist.h"
-#define FSE_STATIC_LINKING_ONLY   /* FSE_optimalTableLog_internal */
-#include "fse.h"        /* header compression */
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "error_private.h"
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define HUF_isError ERR_isError
-#define HUF_STATIC_ASSERT(c) DEBUG_STATIC_ASSERT(c)   /* use only *after* variable declarations */
-#define CHECK_V_F(e, f) size_t const e = f; if (ERR_isError(e)) return e
-#define CHECK_F(f)   { CHECK_V_F(_var_err__, f); }
-
-
-/* **************************************************************
-*  Utils
-****************************************************************/
-unsigned HUF_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue)
-{
-    return FSE_optimalTableLog_internal(maxTableLog, srcSize, maxSymbolValue, 1);
-}
-
-
-/* *******************************************************
-*  HUF : Huffman block compression
-*********************************************************/
-/* HUF_compressWeights() :
- * Same as FSE_compress(), but dedicated to huff0's weights compression.
- * The use case needs much less stack memory.
- * Note : all elements within weightTable are supposed to be <= HUF_TABLELOG_MAX.
- */
-#define MAX_FSE_TABLELOG_FOR_HUFF_HEADER 6
-static size_t HUF_compressWeights (void* dst, size_t dstSize, const void* weightTable, size_t wtSize)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + dstSize;
-
-    unsigned maxSymbolValue = HUF_TABLELOG_MAX;
-    U32 tableLog = MAX_FSE_TABLELOG_FOR_HUFF_HEADER;
-
-    FSE_CTable CTable[FSE_CTABLE_SIZE_U32(MAX_FSE_TABLELOG_FOR_HUFF_HEADER, HUF_TABLELOG_MAX)];
-    BYTE scratchBuffer[1<<MAX_FSE_TABLELOG_FOR_HUFF_HEADER];
-
-    unsigned count[HUF_TABLELOG_MAX+1];
-    S16 norm[HUF_TABLELOG_MAX+1];
-
-    /* init conditions */
-    if (wtSize <= 1) return 0;  /* Not compressible */
-
-    /* Scan input and build symbol stats */
-    {   unsigned const maxCount = HIST_count_simple(count, &maxSymbolValue, weightTable, wtSize);   /* never fails */
-        if (maxCount == wtSize) return 1;   /* only a single symbol in src : rle */
-        if (maxCount == 1) return 0;        /* each symbol present maximum once => not compressible */
-    }
-
-    tableLog = FSE_optimalTableLog(tableLog, wtSize, maxSymbolValue);
-    CHECK_F( FSE_normalizeCount(norm, tableLog, count, wtSize, maxSymbolValue) );
-
-    /* Write table description header */
-    {   CHECK_V_F(hSize, FSE_writeNCount(op, oend-op, norm, maxSymbolValue, tableLog) );
-        op += hSize;
-    }
-
-    /* Compress */
-    CHECK_F( FSE_buildCTable_wksp(CTable, norm, maxSymbolValue, tableLog, scratchBuffer, sizeof(scratchBuffer)) );
-    {   CHECK_V_F(cSize, FSE_compress_usingCTable(op, oend - op, weightTable, wtSize, CTable) );
-        if (cSize == 0) return 0;   /* not enough space for compressed data */
-        op += cSize;
-    }
-
-    return op-ostart;
-}
-
-
-struct HUF_CElt_s {
-  U16  val;
-  BYTE nbBits;
-};   /* typedef'd to HUF_CElt within "huf.h" */
-
-/*! HUF_writeCTable() :
-    `CTable` : Huffman tree to save, using huf representation.
-    @return : size of saved CTable */
-size_t HUF_writeCTable (void* dst, size_t maxDstSize,
-                        const HUF_CElt* CTable, unsigned maxSymbolValue, unsigned huffLog)
-{
-    BYTE bitsToWeight[HUF_TABLELOG_MAX + 1];   /* precomputed conversion table */
-    BYTE huffWeight[HUF_SYMBOLVALUE_MAX];
-    BYTE* op = (BYTE*)dst;
-    U32 n;
-
-     /* check conditions */
-    if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
-
-    /* convert to weight */
-    bitsToWeight[0] = 0;
-    for (n=1; n<huffLog+1; n++)
-        bitsToWeight[n] = (BYTE)(huffLog + 1 - n);
-    for (n=0; n<maxSymbolValue; n++)
-        huffWeight[n] = bitsToWeight[CTable[n].nbBits];
-
-    /* attempt weights compression by FSE */
-    {   CHECK_V_F(hSize, HUF_compressWeights(op+1, maxDstSize-1, huffWeight, maxSymbolValue) );
-        if ((hSize>1) & (hSize < maxSymbolValue/2)) {   /* FSE compressed */
-            op[0] = (BYTE)hSize;
-            return hSize+1;
-    }   }
-
-    /* write raw values as 4-bits (max : 15) */
-    if (maxSymbolValue > (256-128)) return ERROR(GENERIC);   /* should not happen : likely means source cannot be compressed */
-    if (((maxSymbolValue+1)/2) + 1 > maxDstSize) return ERROR(dstSize_tooSmall);   /* not enough space within dst buffer */
-    op[0] = (BYTE)(128 /*special case*/ + (maxSymbolValue-1));
-    huffWeight[maxSymbolValue] = 0;   /* to be sure it doesn't cause msan issue in final combination */
-    for (n=0; n<maxSymbolValue; n+=2)
-        op[(n/2)+1] = (BYTE)((huffWeight[n] << 4) + huffWeight[n+1]);
-    return ((maxSymbolValue+1)/2) + 1;
-}
-
-
-size_t HUF_readCTable (HUF_CElt* CTable, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUF_SYMBOLVALUE_MAX + 1];   /* init not required, even though some static analyzer may complain */
-    U32 rankVal[HUF_TABLELOG_ABSOLUTEMAX + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    U32 nbSymbols = 0;
-
-    /* get symbol weights */
-    CHECK_V_F(readSize, HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX+1, rankVal, &nbSymbols, &tableLog, src, srcSize));
-
-    /* check result */
-    if (tableLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge);
-    if (nbSymbols > *maxSymbolValuePtr+1) return ERROR(maxSymbolValue_tooSmall);
-
-    /* Prepare base value per rank */
-    {   U32 n, nextRankStart = 0;
-        for (n=1; n<=tableLog; n++) {
-            U32 current = nextRankStart;
-            nextRankStart += (rankVal[n] << (n-1));
-            rankVal[n] = current;
-    }   }
-
-    /* fill nbBits */
-    {   U32 n; for (n=0; n<nbSymbols; n++) {
-            const U32 w = huffWeight[n];
-            CTable[n].nbBits = (BYTE)(tableLog + 1 - w);
-    }   }
-
-    /* fill val */
-    {   U16 nbPerRank[HUF_TABLELOG_MAX+2]  = {0};  /* support w=0=>n=tableLog+1 */
-        U16 valPerRank[HUF_TABLELOG_MAX+2] = {0};
-        { U32 n; for (n=0; n<nbSymbols; n++) nbPerRank[CTable[n].nbBits]++; }
-        /* determine stating value per rank */
-        valPerRank[tableLog+1] = 0;   /* for w==0 */
-        {   U16 min = 0;
-            U32 n; for (n=tableLog; n>0; n--) {  /* start at n=tablelog <-> w=1 */
-                valPerRank[n] = min;     /* get starting value within each rank */
-                min += nbPerRank[n];
-                min >>= 1;
-        }   }
-        /* assign value within rank, symbol order */
-        { U32 n; for (n=0; n<nbSymbols; n++) CTable[n].val = valPerRank[CTable[n].nbBits]++; }
-    }
-
-    *maxSymbolValuePtr = nbSymbols - 1;
-    return readSize;
-}
-
-U32 HUF_getNbBits(const void* symbolTable, U32 symbolValue)
-{
-    const HUF_CElt* table = (const HUF_CElt*)symbolTable;
-    assert(symbolValue <= HUF_SYMBOLVALUE_MAX);
-    return table[symbolValue].nbBits;
-}
-
-
-typedef struct nodeElt_s {
-    U32 count;
-    U16 parent;
-    BYTE byte;
-    BYTE nbBits;
-} nodeElt;
-
-static U32 HUF_setMaxHeight(nodeElt* huffNode, U32 lastNonNull, U32 maxNbBits)
-{
-    const U32 largestBits = huffNode[lastNonNull].nbBits;
-    if (largestBits <= maxNbBits) return largestBits;   /* early exit : no elt > maxNbBits */
-
-    /* there are several too large elements (at least >= 2) */
-    {   int totalCost = 0;
-        const U32 baseCost = 1 << (largestBits - maxNbBits);
-        U32 n = lastNonNull;
-
-        while (huffNode[n].nbBits > maxNbBits) {
-            totalCost += baseCost - (1 << (largestBits - huffNode[n].nbBits));
-            huffNode[n].nbBits = (BYTE)maxNbBits;
-            n --;
-        }  /* n stops at huffNode[n].nbBits <= maxNbBits */
-        while (huffNode[n].nbBits == maxNbBits) n--;   /* n end at index of smallest symbol using < maxNbBits */
-
-        /* renorm totalCost */
-        totalCost >>= (largestBits - maxNbBits);  /* note : totalCost is necessarily a multiple of baseCost */
-
-        /* repay normalized cost */
-        {   U32 const noSymbol = 0xF0F0F0F0;
-            U32 rankLast[HUF_TABLELOG_MAX+2];
-            int pos;
-
-            /* Get pos of last (smallest) symbol per rank */
-            memset(rankLast, 0xF0, sizeof(rankLast));
-            {   U32 currentNbBits = maxNbBits;
-                for (pos=n ; pos >= 0; pos--) {
-                    if (huffNode[pos].nbBits >= currentNbBits) continue;
-                    currentNbBits = huffNode[pos].nbBits;   /* < maxNbBits */
-                    rankLast[maxNbBits-currentNbBits] = pos;
-            }   }
-
-            while (totalCost > 0) {
-                U32 nBitsToDecrease = BIT_highbit32(totalCost) + 1;
-                for ( ; nBitsToDecrease > 1; nBitsToDecrease--) {
-                    U32 highPos = rankLast[nBitsToDecrease];
-                    U32 lowPos = rankLast[nBitsToDecrease-1];
-                    if (highPos == noSymbol) continue;
-                    if (lowPos == noSymbol) break;
-                    {   U32 const highTotal = huffNode[highPos].count;
-                        U32 const lowTotal = 2 * huffNode[lowPos].count;
-                        if (highTotal <= lowTotal) break;
-                }   }
-                /* only triggered when no more rank 1 symbol left => find closest one (note : there is necessarily at least one !) */
-                /* HUF_MAX_TABLELOG test just to please gcc 5+; but it should not be necessary */
-                while ((nBitsToDecrease<=HUF_TABLELOG_MAX) && (rankLast[nBitsToDecrease] == noSymbol))
-                    nBitsToDecrease ++;
-                totalCost -= 1 << (nBitsToDecrease-1);
-                if (rankLast[nBitsToDecrease-1] == noSymbol)
-                    rankLast[nBitsToDecrease-1] = rankLast[nBitsToDecrease];   /* this rank is no longer empty */
-                huffNode[rankLast[nBitsToDecrease]].nbBits ++;
-                if (rankLast[nBitsToDecrease] == 0)    /* special case, reached largest symbol */
-                    rankLast[nBitsToDecrease] = noSymbol;
-                else {
-                    rankLast[nBitsToDecrease]--;
-                    if (huffNode[rankLast[nBitsToDecrease]].nbBits != maxNbBits-nBitsToDecrease)
-                        rankLast[nBitsToDecrease] = noSymbol;   /* this rank is now empty */
-            }   }   /* while (totalCost > 0) */
-
-            while (totalCost < 0) {  /* Sometimes, cost correction overshoot */
-                if (rankLast[1] == noSymbol) {  /* special case : no rank 1 symbol (using maxNbBits-1); let's create one from largest rank 0 (using maxNbBits) */
-                    while (huffNode[n].nbBits == maxNbBits) n--;
-                    huffNode[n+1].nbBits--;
-                    rankLast[1] = n+1;
-                    totalCost++;
-                    continue;
-                }
-                huffNode[ rankLast[1] + 1 ].nbBits--;
-                rankLast[1]++;
-                totalCost ++;
-    }   }   }   /* there are several too large elements (at least >= 2) */
-
-    return maxNbBits;
-}
-
-
-typedef struct {
-    U32 base;
-    U32 current;
-} rankPos;
-
-static void HUF_sort(nodeElt* huffNode, const unsigned* count, U32 maxSymbolValue)
-{
-    rankPos rank[32];
-    U32 n;
-
-    memset(rank, 0, sizeof(rank));
-    for (n=0; n<=maxSymbolValue; n++) {
-        U32 r = BIT_highbit32(count[n] + 1);
-        rank[r].base ++;
-    }
-    for (n=30; n>0; n--) rank[n-1].base += rank[n].base;
-    for (n=0; n<32; n++) rank[n].current = rank[n].base;
-    for (n=0; n<=maxSymbolValue; n++) {
-        U32 const c = count[n];
-        U32 const r = BIT_highbit32(c+1) + 1;
-        U32 pos = rank[r].current++;
-        while ((pos > rank[r].base) && (c > huffNode[pos-1].count)) {
-            huffNode[pos] = huffNode[pos-1];
-            pos--;
-        }
-        huffNode[pos].count = c;
-        huffNode[pos].byte  = (BYTE)n;
-    }
-}
-
-
-/** HUF_buildCTable_wksp() :
- *  Same as HUF_buildCTable(), but using externally allocated scratch buffer.
- *  `workSpace` must be aligned on 4-bytes boundaries, and be at least as large as a table of HUF_CTABLE_WORKSPACE_SIZE_U32 unsigned.
- */
-#define STARTNODE (HUF_SYMBOLVALUE_MAX+1)
-typedef nodeElt huffNodeTable[HUF_CTABLE_WORKSPACE_SIZE_U32];
-size_t HUF_buildCTable_wksp (HUF_CElt* tree, const unsigned* count, U32 maxSymbolValue, U32 maxNbBits, void* workSpace, size_t wkspSize)
-{
-    nodeElt* const huffNode0 = (nodeElt*)workSpace;
-    nodeElt* const huffNode = huffNode0+1;
-    U32 n, nonNullRank;
-    int lowS, lowN;
-    U16 nodeNb = STARTNODE;
-    U32 nodeRoot;
-
-    /* safety checks */
-    if (((size_t)workSpace & 3) != 0) return ERROR(GENERIC);  /* must be aligned on 4-bytes boundaries */
-    if (wkspSize < sizeof(huffNodeTable)) return ERROR(workSpace_tooSmall);
-    if (maxNbBits == 0) maxNbBits = HUF_TABLELOG_DEFAULT;
-    if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
-    memset(huffNode0, 0, sizeof(huffNodeTable));
-
-    /* sort, decreasing order */
-    HUF_sort(huffNode, count, maxSymbolValue);
-
-    /* init for parents */
-    nonNullRank = maxSymbolValue;
-    while(huffNode[nonNullRank].count == 0) nonNullRank--;
-    lowS = nonNullRank; nodeRoot = nodeNb + lowS - 1; lowN = nodeNb;
-    huffNode[nodeNb].count = huffNode[lowS].count + huffNode[lowS-1].count;
-    huffNode[lowS].parent = huffNode[lowS-1].parent = nodeNb;
-    nodeNb++; lowS-=2;
-    for (n=nodeNb; n<=nodeRoot; n++) huffNode[n].count = (U32)(1U<<30);
-    huffNode0[0].count = (U32)(1U<<31);  /* fake entry, strong barrier */
-
-    /* create parents */
-    while (nodeNb <= nodeRoot) {
-        U32 n1 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++;
-        U32 n2 = (huffNode[lowS].count < huffNode[lowN].count) ? lowS-- : lowN++;
-        huffNode[nodeNb].count = huffNode[n1].count + huffNode[n2].count;
-        huffNode[n1].parent = huffNode[n2].parent = nodeNb;
-        nodeNb++;
-    }
-
-    /* distribute weights (unlimited tree height) */
-    huffNode[nodeRoot].nbBits = 0;
-    for (n=nodeRoot-1; n>=STARTNODE; n--)
-        huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1;
-    for (n=0; n<=nonNullRank; n++)
-        huffNode[n].nbBits = huffNode[ huffNode[n].parent ].nbBits + 1;
-
-    /* enforce maxTableLog */
-    maxNbBits = HUF_setMaxHeight(huffNode, nonNullRank, maxNbBits);
-
-    /* fill result into tree (val, nbBits) */
-    {   U16 nbPerRank[HUF_TABLELOG_MAX+1] = {0};
-        U16 valPerRank[HUF_TABLELOG_MAX+1] = {0};
-        if (maxNbBits > HUF_TABLELOG_MAX) return ERROR(GENERIC);   /* check fit into table */
-        for (n=0; n<=nonNullRank; n++)
-            nbPerRank[huffNode[n].nbBits]++;
-        /* determine stating value per rank */
-        {   U16 min = 0;
-            for (n=maxNbBits; n>0; n--) {
-                valPerRank[n] = min;      /* get starting value within each rank */
-                min += nbPerRank[n];
-                min >>= 1;
-        }   }
-        for (n=0; n<=maxSymbolValue; n++)
-            tree[huffNode[n].byte].nbBits = huffNode[n].nbBits;   /* push nbBits per symbol, symbol order */
-        for (n=0; n<=maxSymbolValue; n++)
-            tree[n].val = valPerRank[tree[n].nbBits]++;   /* assign value within rank, symbol order */
-    }
-
-    return maxNbBits;
-}
-
-/** HUF_buildCTable() :
- * @return : maxNbBits
- *  Note : count is used before tree is written, so they can safely overlap
- */
-size_t HUF_buildCTable (HUF_CElt* tree, const unsigned* count, unsigned maxSymbolValue, unsigned maxNbBits)
-{
-    huffNodeTable nodeTable;
-    return HUF_buildCTable_wksp(tree, count, maxSymbolValue, maxNbBits, nodeTable, sizeof(nodeTable));
-}
-
-static size_t HUF_estimateCompressedSize(HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue)
-{
-    size_t nbBits = 0;
-    int s;
-    for (s = 0; s <= (int)maxSymbolValue; ++s) {
-        nbBits += CTable[s].nbBits * count[s];
-    }
-    return nbBits >> 3;
-}
-
-static int HUF_validateCTable(const HUF_CElt* CTable, const unsigned* count, unsigned maxSymbolValue) {
-  int bad = 0;
-  int s;
-  for (s = 0; s <= (int)maxSymbolValue; ++s) {
-    bad |= (count[s] != 0) & (CTable[s].nbBits == 0);
-  }
-  return !bad;
-}
-
-size_t HUF_compressBound(size_t size) { return HUF_COMPRESSBOUND(size); }
-
-FORCE_INLINE_TEMPLATE void
-HUF_encodeSymbol(BIT_CStream_t* bitCPtr, U32 symbol, const HUF_CElt* CTable)
-{
-    BIT_addBitsFast(bitCPtr, CTable[symbol].val, CTable[symbol].nbBits);
-}
-
-#define HUF_FLUSHBITS(s)  BIT_flushBits(s)
-
-#define HUF_FLUSHBITS_1(stream) \
-    if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*2+7) HUF_FLUSHBITS(stream)
-
-#define HUF_FLUSHBITS_2(stream) \
-    if (sizeof((stream)->bitContainer)*8 < HUF_TABLELOG_MAX*4+7) HUF_FLUSHBITS(stream)
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_compress1X_usingCTable_internal_body(void* dst, size_t dstSize,
-                                   const void* src, size_t srcSize,
-                                   const HUF_CElt* CTable)
-{
-    const BYTE* ip = (const BYTE*) src;
-    BYTE* const ostart = (BYTE*)dst;
-    BYTE* const oend = ostart + dstSize;
-    BYTE* op = ostart;
-    size_t n;
-    BIT_CStream_t bitC;
-
-    /* init */
-    if (dstSize < 8) return 0;   /* not enough space to compress */
-    { size_t const initErr = BIT_initCStream(&bitC, op, oend-op);
-      if (HUF_isError(initErr)) return 0; }
-
-    n = srcSize & ~3;  /* join to mod 4 */
-    switch (srcSize & 3)
-    {
-        case 3 : HUF_encodeSymbol(&bitC, ip[n+ 2], CTable);
-                 HUF_FLUSHBITS_2(&bitC);
-		 /* fall-through */
-        case 2 : HUF_encodeSymbol(&bitC, ip[n+ 1], CTable);
-                 HUF_FLUSHBITS_1(&bitC);
-		 /* fall-through */
-        case 1 : HUF_encodeSymbol(&bitC, ip[n+ 0], CTable);
-                 HUF_FLUSHBITS(&bitC);
-		 /* fall-through */
-        case 0 : /* fall-through */
-        default: break;
-    }
-
-    for (; n>0; n-=4) {  /* note : n&3==0 at this stage */
-        HUF_encodeSymbol(&bitC, ip[n- 1], CTable);
-        HUF_FLUSHBITS_1(&bitC);
-        HUF_encodeSymbol(&bitC, ip[n- 2], CTable);
-        HUF_FLUSHBITS_2(&bitC);
-        HUF_encodeSymbol(&bitC, ip[n- 3], CTable);
-        HUF_FLUSHBITS_1(&bitC);
-        HUF_encodeSymbol(&bitC, ip[n- 4], CTable);
-        HUF_FLUSHBITS(&bitC);
-    }
-
-    return BIT_closeCStream(&bitC);
-}
-
-#if DYNAMIC_BMI2
-
-static TARGET_ATTRIBUTE("bmi2") size_t
-HUF_compress1X_usingCTable_internal_bmi2(void* dst, size_t dstSize,
-                                   const void* src, size_t srcSize,
-                                   const HUF_CElt* CTable)
-{
-    return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-static size_t
-HUF_compress1X_usingCTable_internal_default(void* dst, size_t dstSize,
-                                      const void* src, size_t srcSize,
-                                      const HUF_CElt* CTable)
-{
-    return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-static size_t
-HUF_compress1X_usingCTable_internal(void* dst, size_t dstSize,
-                              const void* src, size_t srcSize,
-                              const HUF_CElt* CTable, const int bmi2)
-{
-    if (bmi2) {
-        return HUF_compress1X_usingCTable_internal_bmi2(dst, dstSize, src, srcSize, CTable);
-    }
-    return HUF_compress1X_usingCTable_internal_default(dst, dstSize, src, srcSize, CTable);
-}
-
-#else
-
-static size_t
-HUF_compress1X_usingCTable_internal(void* dst, size_t dstSize,
-                              const void* src, size_t srcSize,
-                              const HUF_CElt* CTable, const int bmi2)
-{
-    (void)bmi2;
-    return HUF_compress1X_usingCTable_internal_body(dst, dstSize, src, srcSize, CTable);
-}
-
-#endif
-
-size_t HUF_compress1X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable)
-{
-    return HUF_compress1X_usingCTable_internal(dst, dstSize, src, srcSize, CTable, /* bmi2 */ 0);
-}
-
-
-static size_t
-HUF_compress4X_usingCTable_internal(void* dst, size_t dstSize,
-                              const void* src, size_t srcSize,
-                              const HUF_CElt* CTable, int bmi2)
-{
-    size_t const segmentSize = (srcSize+3)/4;   /* first 3 segments */
-    const BYTE* ip = (const BYTE*) src;
-    const BYTE* const iend = ip + srcSize;
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* const oend = ostart + dstSize;
-    BYTE* op = ostart;
-
-    if (dstSize < 6 + 1 + 1 + 1 + 8) return 0;   /* minimum space to compress successfully */
-    if (srcSize < 12) return 0;   /* no saving possible : too small input */
-    op += 6;   /* jumpTable */
-
-    {   CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
-        if (cSize==0) return 0;
-        assert(cSize <= 65535);
-        MEM_writeLE16(ostart, (U16)cSize);
-        op += cSize;
-    }
-
-    ip += segmentSize;
-    {   CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
-        if (cSize==0) return 0;
-        assert(cSize <= 65535);
-        MEM_writeLE16(ostart+2, (U16)cSize);
-        op += cSize;
-    }
-
-    ip += segmentSize;
-    {   CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, segmentSize, CTable, bmi2) );
-        if (cSize==0) return 0;
-        assert(cSize <= 65535);
-        MEM_writeLE16(ostart+4, (U16)cSize);
-        op += cSize;
-    }
-
-    ip += segmentSize;
-    {   CHECK_V_F(cSize, HUF_compress1X_usingCTable_internal(op, oend-op, ip, iend-ip, CTable, bmi2) );
-        if (cSize==0) return 0;
-        op += cSize;
-    }
-
-    return op-ostart;
-}
-
-size_t HUF_compress4X_usingCTable(void* dst, size_t dstSize, const void* src, size_t srcSize, const HUF_CElt* CTable)
-{
-    return HUF_compress4X_usingCTable_internal(dst, dstSize, src, srcSize, CTable, /* bmi2 */ 0);
-}
-
-typedef enum { HUF_singleStream, HUF_fourStreams } HUF_nbStreams_e;
-
-static size_t HUF_compressCTable_internal(
-                BYTE* const ostart, BYTE* op, BYTE* const oend,
-                const void* src, size_t srcSize,
-                HUF_nbStreams_e nbStreams, const HUF_CElt* CTable, const int bmi2)
-{
-    size_t const cSize = (nbStreams==HUF_singleStream) ?
-                         HUF_compress1X_usingCTable_internal(op, oend - op, src, srcSize, CTable, bmi2) :
-                         HUF_compress4X_usingCTable_internal(op, oend - op, src, srcSize, CTable, bmi2);
-    if (HUF_isError(cSize)) { return cSize; }
-    if (cSize==0) { return 0; }   /* uncompressible */
-    op += cSize;
-    /* check compressibility */
-    if ((size_t)(op-ostart) >= srcSize-1) { return 0; }
-    return op-ostart;
-}
-
-typedef struct {
-    unsigned count[HUF_SYMBOLVALUE_MAX + 1];
-    HUF_CElt CTable[HUF_SYMBOLVALUE_MAX + 1];
-    huffNodeTable nodeTable;
-} HUF_compress_tables_t;
-
-/* HUF_compress_internal() :
- * `workSpace` must a table of at least HUF_WORKSPACE_SIZE_U32 unsigned */
-static size_t
-HUF_compress_internal (void* dst, size_t dstSize,
-                 const void* src, size_t srcSize,
-                       unsigned maxSymbolValue, unsigned huffLog,
-                       HUF_nbStreams_e nbStreams,
-                       void* workSpace, size_t wkspSize,
-                       HUF_CElt* oldHufTable, HUF_repeat* repeat, int preferRepeat,
-                 const int bmi2)
-{
-    HUF_compress_tables_t* const table = (HUF_compress_tables_t*)workSpace;
-    BYTE* const ostart = (BYTE*)dst;
-    BYTE* const oend = ostart + dstSize;
-    BYTE* op = ostart;
-
-    /* checks & inits */
-    if (((size_t)workSpace & 3) != 0) return ERROR(GENERIC);  /* must be aligned on 4-bytes boundaries */
-    if (wkspSize < HUF_WORKSPACE_SIZE) return ERROR(workSpace_tooSmall);
-    if (!srcSize) return 0;  /* Uncompressed */
-    if (!dstSize) return 0;  /* cannot fit anything within dst budget */
-    if (srcSize > HUF_BLOCKSIZE_MAX) return ERROR(srcSize_wrong);   /* current block size limit */
-    if (huffLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge);
-    if (maxSymbolValue > HUF_SYMBOLVALUE_MAX) return ERROR(maxSymbolValue_tooLarge);
-    if (!maxSymbolValue) maxSymbolValue = HUF_SYMBOLVALUE_MAX;
-    if (!huffLog) huffLog = HUF_TABLELOG_DEFAULT;
-
-    /* Heuristic : If old table is valid, use it for small inputs */
-    if (preferRepeat && repeat && *repeat == HUF_repeat_valid) {
-        return HUF_compressCTable_internal(ostart, op, oend,
-                                           src, srcSize,
-                                           nbStreams, oldHufTable, bmi2);
-    }
-
-    /* Scan input and build symbol stats */
-    {   CHECK_V_F(largest, HIST_count_wksp (table->count, &maxSymbolValue, (const BYTE*)src, srcSize, workSpace, wkspSize) );
-        if (largest == srcSize) { *ostart = ((const BYTE*)src)[0]; return 1; }   /* single symbol, rle */
-        if (largest <= (srcSize >> 7)+4) return 0;   /* heuristic : probably not compressible enough */
-    }
-
-    /* Check validity of previous table */
-    if ( repeat
-      && *repeat == HUF_repeat_check
-      && !HUF_validateCTable(oldHufTable, table->count, maxSymbolValue)) {
-        *repeat = HUF_repeat_none;
-    }
-    /* Heuristic : use existing table for small inputs */
-    if (preferRepeat && repeat && *repeat != HUF_repeat_none) {
-        return HUF_compressCTable_internal(ostart, op, oend,
-                                           src, srcSize,
-                                           nbStreams, oldHufTable, bmi2);
-    }
-
-    /* Build Huffman Tree */
-    huffLog = HUF_optimalTableLog(huffLog, srcSize, maxSymbolValue);
-    {   size_t const maxBits = HUF_buildCTable_wksp(table->CTable, table->count,
-                                            maxSymbolValue, huffLog,
-                                            table->nodeTable, sizeof(table->nodeTable));
-        CHECK_F(maxBits);
-        huffLog = (U32)maxBits;
-        /* Zero unused symbols in CTable, so we can check it for validity */
-        memset(table->CTable + (maxSymbolValue + 1), 0,
-               sizeof(table->CTable) - ((maxSymbolValue + 1) * sizeof(HUF_CElt)));
-    }
-
-    /* Write table description header */
-    {   CHECK_V_F(hSize, HUF_writeCTable (op, dstSize, table->CTable, maxSymbolValue, huffLog) );
-        /* Check if using previous huffman table is beneficial */
-        if (repeat && *repeat != HUF_repeat_none) {
-            size_t const oldSize = HUF_estimateCompressedSize(oldHufTable, table->count, maxSymbolValue);
-            size_t const newSize = HUF_estimateCompressedSize(table->CTable, table->count, maxSymbolValue);
-            if (oldSize <= hSize + newSize || hSize + 12 >= srcSize) {
-                return HUF_compressCTable_internal(ostart, op, oend,
-                                                   src, srcSize,
-                                                   nbStreams, oldHufTable, bmi2);
-        }   }
-
-        /* Use the new huffman table */
-        if (hSize + 12ul >= srcSize) { return 0; }
-        op += hSize;
-        if (repeat) { *repeat = HUF_repeat_none; }
-        if (oldHufTable)
-            memcpy(oldHufTable, table->CTable, sizeof(table->CTable));  /* Save new table */
-    }
-    return HUF_compressCTable_internal(ostart, op, oend,
-                                       src, srcSize,
-                                       nbStreams, table->CTable, bmi2);
-}
-
-
-size_t HUF_compress1X_wksp (void* dst, size_t dstSize,
-                      const void* src, size_t srcSize,
-                      unsigned maxSymbolValue, unsigned huffLog,
-                      void* workSpace, size_t wkspSize)
-{
-    return HUF_compress_internal(dst, dstSize, src, srcSize,
-                                 maxSymbolValue, huffLog, HUF_singleStream,
-                                 workSpace, wkspSize,
-                                 NULL, NULL, 0, 0 /*bmi2*/);
-}
-
-size_t HUF_compress1X_repeat (void* dst, size_t dstSize,
-                      const void* src, size_t srcSize,
-                      unsigned maxSymbolValue, unsigned huffLog,
-                      void* workSpace, size_t wkspSize,
-                      HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2)
-{
-    return HUF_compress_internal(dst, dstSize, src, srcSize,
-                                 maxSymbolValue, huffLog, HUF_singleStream,
-                                 workSpace, wkspSize, hufTable,
-                                 repeat, preferRepeat, bmi2);
-}
-
-size_t HUF_compress1X (void* dst, size_t dstSize,
-                 const void* src, size_t srcSize,
-                 unsigned maxSymbolValue, unsigned huffLog)
-{
-    unsigned workSpace[HUF_WORKSPACE_SIZE_U32];
-    return HUF_compress1X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace));
-}
-
-/* HUF_compress4X_repeat():
- * compress input using 4 streams.
- * provide workspace to generate compression tables */
-size_t HUF_compress4X_wksp (void* dst, size_t dstSize,
-                      const void* src, size_t srcSize,
-                      unsigned maxSymbolValue, unsigned huffLog,
-                      void* workSpace, size_t wkspSize)
-{
-    return HUF_compress_internal(dst, dstSize, src, srcSize,
-                                 maxSymbolValue, huffLog, HUF_fourStreams,
-                                 workSpace, wkspSize,
-                                 NULL, NULL, 0, 0 /*bmi2*/);
-}
-
-/* HUF_compress4X_repeat():
- * compress input using 4 streams.
- * re-use an existing huffman compression table */
-size_t HUF_compress4X_repeat (void* dst, size_t dstSize,
-                      const void* src, size_t srcSize,
-                      unsigned maxSymbolValue, unsigned huffLog,
-                      void* workSpace, size_t wkspSize,
-                      HUF_CElt* hufTable, HUF_repeat* repeat, int preferRepeat, int bmi2)
-{
-    return HUF_compress_internal(dst, dstSize, src, srcSize,
-                                 maxSymbolValue, huffLog, HUF_fourStreams,
-                                 workSpace, wkspSize,
-                                 hufTable, repeat, preferRepeat, bmi2);
-}
-
-size_t HUF_compress2 (void* dst, size_t dstSize,
-                const void* src, size_t srcSize,
-                unsigned maxSymbolValue, unsigned huffLog)
-{
-    unsigned workSpace[HUF_WORKSPACE_SIZE_U32];
-    return HUF_compress4X_wksp(dst, dstSize, src, srcSize, maxSymbolValue, huffLog, workSpace, sizeof(workSpace));
-}
-
-size_t HUF_compress (void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return HUF_compress2(dst, maxDstSize, src, srcSize, 255, HUF_TABLELOG_DEFAULT);
-}
diff --git a/vendor/github.com/DataDog/zstd/huf_decompress.c b/vendor/github.com/DataDog/zstd/huf_decompress.c
deleted file mode 100644
index 3f8bd29..0000000
--- a/vendor/github.com/DataDog/zstd/huf_decompress.c
+++ /dev/null
@@ -1,1232 +0,0 @@
-/* ******************************************************************
-   huff0 huffman decoder,
-   part of Finite State Entropy library
-   Copyright (C) 2013-present, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-/* **************************************************************
-*  Dependencies
-****************************************************************/
-#include <string.h>     /* memcpy, memset */
-#include "compiler.h"
-#include "bitstream.h"  /* BIT_* */
-#include "fse.h"        /* to compress headers */
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "error_private.h"
-
-/* **************************************************************
-*  Macros
-****************************************************************/
-
-/* These two optional macros force the use one way or another of the two
- * Huffman decompression implementations. You can't force in both directions
- * at the same time.
- */
-#if defined(HUF_FORCE_DECOMPRESS_X1) && \
-    defined(HUF_FORCE_DECOMPRESS_X2)
-#error "Cannot force the use of the X1 and X2 decoders at the same time!"
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define HUF_isError ERR_isError
-#define CHECK_F(f) { size_t const err_ = (f); if (HUF_isError(err_)) return err_; }
-
-
-/* **************************************************************
-*  Byte alignment for workSpace management
-****************************************************************/
-#define HUF_ALIGN(x, a)         HUF_ALIGN_MASK((x), (a) - 1)
-#define HUF_ALIGN_MASK(x, mask) (((x) + (mask)) & ~(mask))
-
-
-/* **************************************************************
-*  BMI2 Variant Wrappers
-****************************************************************/
-#if DYNAMIC_BMI2
-
-#define HUF_DGEN(fn)                                                        \
-                                                                            \
-    static size_t fn##_default(                                             \
-                  void* dst,  size_t dstSize,                               \
-            const void* cSrc, size_t cSrcSize,                              \
-            const HUF_DTable* DTable)                                       \
-    {                                                                       \
-        return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable);             \
-    }                                                                       \
-                                                                            \
-    static TARGET_ATTRIBUTE("bmi2") size_t fn##_bmi2(                       \
-                  void* dst,  size_t dstSize,                               \
-            const void* cSrc, size_t cSrcSize,                              \
-            const HUF_DTable* DTable)                                       \
-    {                                                                       \
-        return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable);             \
-    }                                                                       \
-                                                                            \
-    static size_t fn(void* dst, size_t dstSize, void const* cSrc,           \
-                     size_t cSrcSize, HUF_DTable const* DTable, int bmi2)   \
-    {                                                                       \
-        if (bmi2) {                                                         \
-            return fn##_bmi2(dst, dstSize, cSrc, cSrcSize, DTable);         \
-        }                                                                   \
-        return fn##_default(dst, dstSize, cSrc, cSrcSize, DTable);          \
-    }
-
-#else
-
-#define HUF_DGEN(fn)                                                        \
-    static size_t fn(void* dst, size_t dstSize, void const* cSrc,           \
-                     size_t cSrcSize, HUF_DTable const* DTable, int bmi2)   \
-    {                                                                       \
-        (void)bmi2;                                                         \
-        return fn##_body(dst, dstSize, cSrc, cSrcSize, DTable);             \
-    }
-
-#endif
-
-
-/*-***************************/
-/*  generic DTableDesc       */
-/*-***************************/
-typedef struct { BYTE maxTableLog; BYTE tableType; BYTE tableLog; BYTE reserved; } DTableDesc;
-
-static DTableDesc HUF_getDTableDesc(const HUF_DTable* table)
-{
-    DTableDesc dtd;
-    memcpy(&dtd, table, sizeof(dtd));
-    return dtd;
-}
-
-
-#ifndef HUF_FORCE_DECOMPRESS_X2
-
-/*-***************************/
-/*  single-symbol decoding   */
-/*-***************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX1;   /* single-symbol decoding */
-
-size_t HUF_readDTableX1_wksp(HUF_DTable* DTable, const void* src, size_t srcSize, void* workSpace, size_t wkspSize)
-{
-    U32 tableLog = 0;
-    U32 nbSymbols = 0;
-    size_t iSize;
-    void* const dtPtr = DTable + 1;
-    HUF_DEltX1* const dt = (HUF_DEltX1*)dtPtr;
-
-    U32* rankVal;
-    BYTE* huffWeight;
-    size_t spaceUsed32 = 0;
-
-    rankVal = (U32 *)workSpace + spaceUsed32;
-    spaceUsed32 += HUF_TABLELOG_ABSOLUTEMAX + 1;
-    huffWeight = (BYTE *)((U32 *)workSpace + spaceUsed32);
-    spaceUsed32 += HUF_ALIGN(HUF_SYMBOLVALUE_MAX + 1, sizeof(U32)) >> 2;
-
-    if ((spaceUsed32 << 2) > wkspSize) return ERROR(tableLog_tooLarge);
-
-    DEBUG_STATIC_ASSERT(sizeof(DTableDesc) == sizeof(HUF_DTable));
-    /* memset(huffWeight, 0, sizeof(huffWeight)); */   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(huffWeight, HUF_SYMBOLVALUE_MAX + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* Table header */
-    {   DTableDesc dtd = HUF_getDTableDesc(DTable);
-        if (tableLog > (U32)(dtd.maxTableLog+1)) return ERROR(tableLog_tooLarge);   /* DTable too small, Huffman tree cannot fit in */
-        dtd.tableType = 0;
-        dtd.tableLog = (BYTE)tableLog;
-        memcpy(DTable, &dtd, sizeof(dtd));
-    }
-
-    /* Calculate starting value for each rank */
-    {   U32 n, nextRankStart = 0;
-        for (n=1; n<tableLog+1; n++) {
-            U32 const current = nextRankStart;
-            nextRankStart += (rankVal[n] << (n-1));
-            rankVal[n] = current;
-    }   }
-
-    /* fill DTable */
-    {   U32 n;
-        for (n=0; n<nbSymbols; n++) {
-            U32 const w = huffWeight[n];
-            U32 const length = (1 << w) >> 1;
-            U32 u;
-            HUF_DEltX1 D;
-            D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-            for (u = rankVal[w]; u < rankVal[w] + length; u++)
-                dt[u] = D;
-            rankVal[w] += length;
-    }   }
-
-    return iSize;
-}
-
-size_t HUF_readDTableX1(HUF_DTable* DTable, const void* src, size_t srcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_readDTableX1_wksp(DTable, src, srcSize,
-                                 workSpace, sizeof(workSpace));
-}
-
-FORCE_INLINE_TEMPLATE BYTE
-HUF_decodeSymbolX1(BIT_DStream_t* Dstream, const HUF_DEltX1* dt, const U32 dtLog)
-{
-    size_t const val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-    BYTE const c = dt[val].byte;
-    BIT_skipBits(Dstream, dt[val].nbBits);
-    return c;
-}
-
-#define HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr) \
-    *ptr++ = HUF_decodeSymbolX1(DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX1_1(ptr, DStreamPtr)  \
-    if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \
-        HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX1_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUF_DECODE_SYMBOLX1_0(ptr, DStreamPtr)
-
-HINT_INLINE size_t
-HUF_decodeStreamX1(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX1* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd-3)) {
-        HUF_DECODE_SYMBOLX1_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX1_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX1_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
-    }
-
-    /* [0-3] symbols remaining */
-    if (MEM_32bits())
-        while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd))
-            HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, no need to reload */
-    while (p < pEnd)
-        HUF_DECODE_SYMBOLX1_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress1X1_usingDTable_internal_body(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + dstSize;
-    const void* dtPtr = DTable + 1;
-    const HUF_DEltX1* const dt = (const HUF_DEltX1*)dtPtr;
-    BIT_DStream_t bitD;
-    DTableDesc const dtd = HUF_getDTableDesc(DTable);
-    U32 const dtLog = dtd.tableLog;
-
-    CHECK_F( BIT_initDStream(&bitD, cSrc, cSrcSize) );
-
-    HUF_decodeStreamX1(op, &bitD, oend, dt, dtLog);
-
-    if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    return dstSize;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress4X1_usingDTable_internal_body(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    /* Check */
-    if (cSrcSize < 10) return ERROR(corruption_detected);  /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable + 1;
-        const HUF_DEltX1* const dt = (const HUF_DEltX1*)dtPtr;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        size_t const length1 = MEM_readLE16(istart);
-        size_t const length2 = MEM_readLE16(istart+2);
-        size_t const length3 = MEM_readLE16(istart+4);
-        size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal = BIT_DStream_unfinished;
-        DTableDesc const dtd = HUF_getDTableDesc(DTable);
-        U32 const dtLog = dtd.tableLog;
-
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        CHECK_F( BIT_initDStream(&bitD1, istart1, length1) );
-        CHECK_F( BIT_initDStream(&bitD2, istart2, length2) );
-        CHECK_F( BIT_initDStream(&bitD3, istart3, length3) );
-        CHECK_F( BIT_initDStream(&bitD4, istart4, length4) );
-
-        /* up to 16 symbols per loop (4 symbols per stream) in 64-bit mode */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        while ( (endSignal==BIT_DStream_unfinished) && (op4<(oend-3)) ) {
-            HUF_DECODE_SYMBOLX1_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX1_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX1_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX1_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX1_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX1_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX1_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX1_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX1_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX1_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX1_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX1_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX1_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX1_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX1_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX1_0(op4, &bitD4);
-            BIT_reloadDStream(&bitD1);
-            BIT_reloadDStream(&bitD2);
-            BIT_reloadDStream(&bitD3);
-            BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        /* note : should not be necessary : op# advance in lock step, and we control op4.
-         *        but curiously, binary generated by gcc 7.2 & 7.3 with -mbmi2 runs faster when >=1 test is present */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX1(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX1(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX1(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX1(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        { U32 const endCheck = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-          if (!endCheck) return ERROR(corruption_detected); }
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-typedef size_t (*HUF_decompress_usingDTable_t)(void *dst, size_t dstSize,
-                                               const void *cSrc,
-                                               size_t cSrcSize,
-                                               const HUF_DTable *DTable);
-
-HUF_DGEN(HUF_decompress1X1_usingDTable_internal)
-HUF_DGEN(HUF_decompress4X1_usingDTable_internal)
-
-
-
-size_t HUF_decompress1X1_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    DTableDesc dtd = HUF_getDTableDesc(DTable);
-    if (dtd.tableType != 0) return ERROR(GENERIC);
-    return HUF_decompress1X1_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-}
-
-size_t HUF_decompress1X1_DCtx_wksp(HUF_DTable* DCtx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUF_readDTableX1_wksp(DCtx, cSrc, cSrcSize, workSpace, wkspSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUF_decompress1X1_usingDTable_internal(dst, dstSize, ip, cSrcSize, DCtx, /* bmi2 */ 0);
-}
-
-
-size_t HUF_decompress1X1_DCtx(HUF_DTable* DCtx, void* dst, size_t dstSize,
-                              const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress1X1_DCtx_wksp(DCtx, dst, dstSize, cSrc, cSrcSize,
-                                       workSpace, sizeof(workSpace));
-}
-
-size_t HUF_decompress1X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX1(DTable, HUF_TABLELOG_MAX);
-    return HUF_decompress1X1_DCtx (DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-size_t HUF_decompress4X1_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    DTableDesc dtd = HUF_getDTableDesc(DTable);
-    if (dtd.tableType != 0) return ERROR(GENERIC);
-    return HUF_decompress4X1_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-}
-
-static size_t HUF_decompress4X1_DCtx_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize, int bmi2)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUF_readDTableX1_wksp (dctx, cSrc, cSrcSize,
-                                                workSpace, wkspSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUF_decompress4X1_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx, bmi2);
-}
-
-size_t HUF_decompress4X1_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize)
-{
-    return HUF_decompress4X1_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, 0);
-}
-
-
-size_t HUF_decompress4X1_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress4X1_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize,
-                                       workSpace, sizeof(workSpace));
-}
-size_t HUF_decompress4X1 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX1(DTable, HUF_TABLELOG_MAX);
-    return HUF_decompress4X1_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-#endif /* HUF_FORCE_DECOMPRESS_X2 */
-
-
-#ifndef HUF_FORCE_DECOMPRESS_X1
-
-/* *************************/
-/* double-symbols decoding */
-/* *************************/
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX2;  /* double-symbols decoding */
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-typedef U32 rankValCol_t[HUF_TABLELOG_MAX + 1];
-typedef rankValCol_t rankVal_t[HUF_TABLELOG_MAX];
-
-
-/* HUF_fillDTableX2Level2() :
- * `rankValOrigin` must be a table of at least (HUF_TABLELOG_MAX + 1) U32 */
-static void HUF_fillDTableX2Level2(HUF_DEltX2* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUF_DEltX2 DElt;
-    U32 rankVal[HUF_TABLELOG_MAX + 1];
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1) {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    {   U32 s; for (s=0; s<sortedListSize; s++) {   /* note : sortedSymbols already skipped */
-            const U32 symbol = sortedSymbols[s].symbol;
-            const U32 weight = sortedSymbols[s].weight;
-            const U32 nbBits = nbBitsBaseline - weight;
-            const U32 length = 1 << (sizeLog-nbBits);
-            const U32 start = rankVal[weight];
-            U32 i = start;
-            const U32 end = start + length;
-
-            MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-            DElt.nbBits = (BYTE)(nbBits + consumed);
-            DElt.length = 2;
-            do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-            rankVal[weight] += length;
-    }   }
-}
-
-
-static void HUF_fillDTableX2(HUF_DEltX2* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUF_TABLELOG_MAX + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++) {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits) {   /* enough room for a second symbol */
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUF_fillDTableX2Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        } else {
-            HUF_DEltX2 DElt;
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits = (BYTE)(nbBits);
-            DElt.length = 1;
-            {   U32 const end = start + length;
-                U32 u;
-                for (u = start; u < end; u++) DTable[u] = DElt;
-        }   }
-        rankVal[weight] += length;
-    }
-}
-
-size_t HUF_readDTableX2_wksp(HUF_DTable* DTable,
-                       const void* src, size_t srcSize,
-                             void* workSpace, size_t wkspSize)
-{
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    DTableDesc dtd = HUF_getDTableDesc(DTable);
-    U32 const maxTableLog = dtd.maxTableLog;
-    size_t iSize;
-    void* dtPtr = DTable+1;   /* force compiler to avoid strict-aliasing */
-    HUF_DEltX2* const dt = (HUF_DEltX2*)dtPtr;
-    U32 *rankStart;
-
-    rankValCol_t* rankVal;
-    U32* rankStats;
-    U32* rankStart0;
-    sortedSymbol_t* sortedSymbol;
-    BYTE* weightList;
-    size_t spaceUsed32 = 0;
-
-    rankVal = (rankValCol_t *)((U32 *)workSpace + spaceUsed32);
-    spaceUsed32 += (sizeof(rankValCol_t) * HUF_TABLELOG_MAX) >> 2;
-    rankStats = (U32 *)workSpace + spaceUsed32;
-    spaceUsed32 += HUF_TABLELOG_MAX + 1;
-    rankStart0 = (U32 *)workSpace + spaceUsed32;
-    spaceUsed32 += HUF_TABLELOG_MAX + 2;
-    sortedSymbol = (sortedSymbol_t *)workSpace + (spaceUsed32 * sizeof(U32)) / sizeof(sortedSymbol_t);
-    spaceUsed32 += HUF_ALIGN(sizeof(sortedSymbol_t) * (HUF_SYMBOLVALUE_MAX + 1), sizeof(U32)) >> 2;
-    weightList = (BYTE *)((U32 *)workSpace + spaceUsed32);
-    spaceUsed32 += HUF_ALIGN(HUF_SYMBOLVALUE_MAX + 1, sizeof(U32)) >> 2;
-
-    if ((spaceUsed32 << 2) > wkspSize) return ERROR(tableLog_tooLarge);
-
-    rankStart = rankStart0 + 1;
-    memset(rankStats, 0, sizeof(U32) * (2 * HUF_TABLELOG_MAX + 2 + 1));
-
-    DEBUG_STATIC_ASSERT(sizeof(HUF_DEltX2) == sizeof(HUF_DTable));   /* if compiler fails here, assertion is wrong */
-    if (maxTableLog > HUF_TABLELOG_MAX) return ERROR(tableLog_tooLarge);
-    /* memset(weightList, 0, sizeof(weightList)); */  /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(weightList, HUF_SYMBOLVALUE_MAX + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > maxTableLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--) {}  /* necessarily finds a solution before 0 */
-
-    /* Get start index of each weight */
-    {   U32 w, nextRankStart = 0;
-        for (w=1; w<maxW+1; w++) {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {   U32 s;
-        for (s=0; s<nbSymbols; s++) {
-            U32 const w = weightList[s];
-            U32 const r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {   U32* const rankVal0 = rankVal[0];
-        {   int const rescale = (maxTableLog-tableLog) - 1;   /* tableLog <= maxTableLog */
-            U32 nextRankVal = 0;
-            U32 w;
-            for (w=1; w<maxW+1; w++) {
-                U32 current = nextRankVal;
-                nextRankVal += rankStats[w] << (w+rescale);
-                rankVal0[w] = current;
-        }   }
-        {   U32 const minBits = tableLog+1 - maxW;
-            U32 consumed;
-            for (consumed = minBits; consumed < maxTableLog - minBits + 1; consumed++) {
-                U32* const rankValPtr = rankVal[consumed];
-                U32 w;
-                for (w = 1; w < maxW+1; w++) {
-                    rankValPtr[w] = rankVal0[w] >> consumed;
-    }   }   }   }
-
-    HUF_fillDTableX2(dt, maxTableLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    dtd.tableLog = (BYTE)maxTableLog;
-    dtd.tableType = 1;
-    memcpy(DTable, &dtd, sizeof(dtd));
-    return iSize;
-}
-
-size_t HUF_readDTableX2(HUF_DTable* DTable, const void* src, size_t srcSize)
-{
-  U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-  return HUF_readDTableX2_wksp(DTable, src, srcSize,
-                               workSpace, sizeof(workSpace));
-}
-
-
-FORCE_INLINE_TEMPLATE U32
-HUF_decodeSymbolX2(void* op, BIT_DStream_t* DStream, const HUF_DEltX2* dt, const U32 dtLog)
-{
-    size_t const val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BIT_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-FORCE_INLINE_TEMPLATE U32
-HUF_decodeLastSymbolX2(void* op, BIT_DStream_t* DStream, const HUF_DEltX2* dt, const U32 dtLog)
-{
-    size_t const val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits);
-    else {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) {
-            BIT_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);
-    }   }
-    return 1;
-}
-
-#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    ptr += HUF_decodeSymbolX2(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_TABLELOG_MAX<=12)) \
-        ptr += HUF_decodeSymbolX2(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUF_decodeSymbolX2(ptr, DStreamPtr, dt, dtLog)
-
-HINT_INLINE size_t
-HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd,
-                const HUF_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p < pEnd-(sizeof(bitDPtr->bitContainer)-1))) {
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to end : up to 2 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) & (p <= pEnd-2))
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUF_decodeLastSymbolX2(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress1X2_usingDTable_internal_body(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    BIT_DStream_t bitD;
-
-    /* Init */
-    CHECK_F( BIT_initDStream(&bitD, cSrc, cSrcSize) );
-
-    /* decode */
-    {   BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable+1;   /* force compiler to not use strict-aliasing */
-        const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr;
-        DTableDesc const dtd = HUF_getDTableDesc(DTable);
-        HUF_decodeStreamX2(ostart, &bitD, oend, dt, dtd.tableLog);
-    }
-
-    /* check */
-    if (!BIT_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    /* decoded size */
-    return dstSize;
-}
-
-
-FORCE_INLINE_TEMPLATE size_t
-HUF_decompress4X2_usingDTable_internal_body(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable+1;
-        const HUF_DEltX2* const dt = (const HUF_DEltX2*)dtPtr;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        size_t const length1 = MEM_readLE16(istart);
-        size_t const length2 = MEM_readLE16(istart+2);
-        size_t const length3 = MEM_readLE16(istart+4);
-        size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        size_t const segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-        DTableDesc const dtd = HUF_getDTableDesc(DTable);
-        U32 const dtLog = dtd.tableLog;
-
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        CHECK_F( BIT_initDStream(&bitD1, istart1, length1) );
-        CHECK_F( BIT_initDStream(&bitD2, istart2, length2) );
-        CHECK_F( BIT_initDStream(&bitD3, istart3, length3) );
-        CHECK_F( BIT_initDStream(&bitD4, istart4, length4) );
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) & (op4<(oend-(sizeof(bitD4.bitContainer)-1))) ; ) {
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        { U32 const endCheck = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-          if (!endCheck) return ERROR(corruption_detected); }
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-HUF_DGEN(HUF_decompress1X2_usingDTable_internal)
-HUF_DGEN(HUF_decompress4X2_usingDTable_internal)
-
-size_t HUF_decompress1X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    DTableDesc dtd = HUF_getDTableDesc(DTable);
-    if (dtd.tableType != 1) return ERROR(GENERIC);
-    return HUF_decompress1X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-}
-
-size_t HUF_decompress1X2_DCtx_wksp(HUF_DTable* DCtx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUF_readDTableX2_wksp(DCtx, cSrc, cSrcSize,
-                                               workSpace, wkspSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUF_decompress1X2_usingDTable_internal(dst, dstSize, ip, cSrcSize, DCtx, /* bmi2 */ 0);
-}
-
-
-size_t HUF_decompress1X2_DCtx(HUF_DTable* DCtx, void* dst, size_t dstSize,
-                              const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress1X2_DCtx_wksp(DCtx, dst, dstSize, cSrc, cSrcSize,
-                                       workSpace, sizeof(workSpace));
-}
-
-size_t HUF_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX);
-    return HUF_decompress1X2_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-size_t HUF_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUF_DTable* DTable)
-{
-    DTableDesc dtd = HUF_getDTableDesc(DTable);
-    if (dtd.tableType != 1) return ERROR(GENERIC);
-    return HUF_decompress4X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-}
-
-static size_t HUF_decompress4X2_DCtx_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize, int bmi2)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUF_readDTableX2_wksp(dctx, cSrc, cSrcSize,
-                                         workSpace, wkspSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUF_decompress4X2_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx, bmi2);
-}
-
-size_t HUF_decompress4X2_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize,
-                                   const void* cSrc, size_t cSrcSize,
-                                   void* workSpace, size_t wkspSize)
-{
-    return HUF_decompress4X2_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, /* bmi2 */ 0);
-}
-
-
-size_t HUF_decompress4X2_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize,
-                              const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress4X2_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize,
-                                       workSpace, sizeof(workSpace));
-}
-
-size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_TABLELOG_MAX);
-    return HUF_decompress4X2_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-#endif /* HUF_FORCE_DECOMPRESS_X1 */
-
-
-/* ***********************************/
-/* Universal decompression selectors */
-/* ***********************************/
-
-size_t HUF_decompress1X_usingDTable(void* dst, size_t maxDstSize,
-                                    const void* cSrc, size_t cSrcSize,
-                                    const HUF_DTable* DTable)
-{
-    DTableDesc const dtd = HUF_getDTableDesc(DTable);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-    (void)dtd;
-    assert(dtd.tableType == 0);
-    return HUF_decompress1X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-    (void)dtd;
-    assert(dtd.tableType == 1);
-    return HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#else
-    return dtd.tableType ? HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0) :
-                           HUF_decompress1X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#endif
-}
-
-size_t HUF_decompress4X_usingDTable(void* dst, size_t maxDstSize,
-                                    const void* cSrc, size_t cSrcSize,
-                                    const HUF_DTable* DTable)
-{
-    DTableDesc const dtd = HUF_getDTableDesc(DTable);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-    (void)dtd;
-    assert(dtd.tableType == 0);
-    return HUF_decompress4X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-    (void)dtd;
-    assert(dtd.tableType == 1);
-    return HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#else
-    return dtd.tableType ? HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0) :
-                           HUF_decompress4X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, /* bmi2 */ 0);
-#endif
-}
-
-
-#if !defined(HUF_FORCE_DECOMPRESS_X1) && !defined(HUF_FORCE_DECOMPRESS_X2)
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-#endif
-
-/** HUF_selectDecoder() :
- *  Tells which decoder is likely to decode faster,
- *  based on a set of pre-computed metrics.
- * @return : 0==HUF_decompress4X1, 1==HUF_decompress4X2 .
- *  Assumption : 0 < dstSize <= 128 KB */
-U32 HUF_selectDecoder (size_t dstSize, size_t cSrcSize)
-{
-    assert(dstSize > 0);
-    assert(dstSize <= 128*1024);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-    (void)dstSize;
-    (void)cSrcSize;
-    return 0;
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-    (void)dstSize;
-    (void)cSrcSize;
-    return 1;
-#else
-    /* decoder timing evaluation */
-    {   U32 const Q = (cSrcSize >= dstSize) ? 15 : (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 */
-        U32 const D256 = (U32)(dstSize >> 8);
-        U32 const DTime0 = algoTime[Q][0].tableTime + (algoTime[Q][0].decode256Time * D256);
-        U32 DTime1 = algoTime[Q][1].tableTime + (algoTime[Q][1].decode256Time * D256);
-        DTime1 += DTime1 >> 3;  /* advantage to algorithm using less memory, to reduce cache eviction */
-        return DTime1 < DTime0;
-    }
-#endif
-}
-
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-#if !defined(HUF_FORCE_DECOMPRESS_X1) && !defined(HUF_FORCE_DECOMPRESS_X2)
-    static const decompressionAlgo decompress[2] = { HUF_decompress4X1, HUF_decompress4X2 };
-#endif
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-        (void)algoNb;
-        assert(algoNb == 0);
-        return HUF_decompress4X1(dst, dstSize, cSrc, cSrcSize);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-        (void)algoNb;
-        assert(algoNb == 1);
-        return HUF_decompress4X2(dst, dstSize, cSrc, cSrcSize);
-#else
-        return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-#endif
-    }
-}
-
-size_t HUF_decompress4X_DCtx (HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-        (void)algoNb;
-        assert(algoNb == 0);
-        return HUF_decompress4X1_DCtx(dctx, dst, dstSize, cSrc, cSrcSize);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-        (void)algoNb;
-        assert(algoNb == 1);
-        return HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize);
-#else
-        return algoNb ? HUF_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) :
-                        HUF_decompress4X1_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ;
-#endif
-    }
-}
-
-size_t HUF_decompress4X_hufOnly(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress4X_hufOnly_wksp(dctx, dst, dstSize, cSrc, cSrcSize,
-                                         workSpace, sizeof(workSpace));
-}
-
-
-size_t HUF_decompress4X_hufOnly_wksp(HUF_DTable* dctx, void* dst,
-                                     size_t dstSize, const void* cSrc,
-                                     size_t cSrcSize, void* workSpace,
-                                     size_t wkspSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize == 0) return ERROR(corruption_detected);
-
-    {   U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-        (void)algoNb;
-        assert(algoNb == 0);
-        return HUF_decompress4X1_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-        (void)algoNb;
-        assert(algoNb == 1);
-        return HUF_decompress4X2_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize);
-#else
-        return algoNb ? HUF_decompress4X2_DCtx_wksp(dctx, dst, dstSize, cSrc,
-                            cSrcSize, workSpace, wkspSize):
-                        HUF_decompress4X1_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize);
-#endif
-    }
-}
-
-size_t HUF_decompress1X_DCtx_wksp(HUF_DTable* dctx, void* dst, size_t dstSize,
-                                  const void* cSrc, size_t cSrcSize,
-                                  void* workSpace, size_t wkspSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-        (void)algoNb;
-        assert(algoNb == 0);
-        return HUF_decompress1X1_DCtx_wksp(dctx, dst, dstSize, cSrc,
-                                cSrcSize, workSpace, wkspSize);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-        (void)algoNb;
-        assert(algoNb == 1);
-        return HUF_decompress1X2_DCtx_wksp(dctx, dst, dstSize, cSrc,
-                                cSrcSize, workSpace, wkspSize);
-#else
-        return algoNb ? HUF_decompress1X2_DCtx_wksp(dctx, dst, dstSize, cSrc,
-                                cSrcSize, workSpace, wkspSize):
-                        HUF_decompress1X1_DCtx_wksp(dctx, dst, dstSize, cSrc,
-                                cSrcSize, workSpace, wkspSize);
-#endif
-    }
-}
-
-size_t HUF_decompress1X_DCtx(HUF_DTable* dctx, void* dst, size_t dstSize,
-                             const void* cSrc, size_t cSrcSize)
-{
-    U32 workSpace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];
-    return HUF_decompress1X_DCtx_wksp(dctx, dst, dstSize, cSrc, cSrcSize,
-                                      workSpace, sizeof(workSpace));
-}
-
-
-size_t HUF_decompress1X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2)
-{
-    DTableDesc const dtd = HUF_getDTableDesc(DTable);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-    (void)dtd;
-    assert(dtd.tableType == 0);
-    return HUF_decompress1X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-    (void)dtd;
-    assert(dtd.tableType == 1);
-    return HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#else
-    return dtd.tableType ? HUF_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2) :
-                           HUF_decompress1X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#endif
-}
-
-#ifndef HUF_FORCE_DECOMPRESS_X2
-size_t HUF_decompress1X1_DCtx_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUF_readDTableX1_wksp(dctx, cSrc, cSrcSize, workSpace, wkspSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUF_decompress1X1_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx, bmi2);
-}
-#endif
-
-size_t HUF_decompress4X_usingDTable_bmi2(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUF_DTable* DTable, int bmi2)
-{
-    DTableDesc const dtd = HUF_getDTableDesc(DTable);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-    (void)dtd;
-    assert(dtd.tableType == 0);
-    return HUF_decompress4X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-    (void)dtd;
-    assert(dtd.tableType == 1);
-    return HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#else
-    return dtd.tableType ? HUF_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2) :
-                           HUF_decompress4X1_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable, bmi2);
-#endif
-}
-
-size_t HUF_decompress4X_hufOnly_wksp_bmi2(HUF_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize, void* workSpace, size_t wkspSize, int bmi2)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize == 0) return ERROR(corruption_detected);
-
-    {   U32 const algoNb = HUF_selectDecoder(dstSize, cSrcSize);
-#if defined(HUF_FORCE_DECOMPRESS_X1)
-        (void)algoNb;
-        assert(algoNb == 0);
-        return HUF_decompress4X1_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, bmi2);
-#elif defined(HUF_FORCE_DECOMPRESS_X2)
-        (void)algoNb;
-        assert(algoNb == 1);
-        return HUF_decompress4X2_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, bmi2);
-#else
-        return algoNb ? HUF_decompress4X2_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, bmi2) :
-                        HUF_decompress4X1_DCtx_wksp_bmi2(dctx, dst, dstSize, cSrc, cSrcSize, workSpace, wkspSize, bmi2);
-#endif
-    }
-}
diff --git a/vendor/github.com/DataDog/zstd/mem.h b/vendor/github.com/DataDog/zstd/mem.h
deleted file mode 100644
index 5da2487..0000000
--- a/vendor/github.com/DataDog/zstd/mem.h
+++ /dev/null
@@ -1,380 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*-****************************************
-*  Dependencies
-******************************************/
-#include <stddef.h>     /* size_t, ptrdiff_t */
-#include <string.h>     /* memcpy */
-
-
-/*-****************************************
-*  Compiler specifics
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-#if defined(__GNUC__)
-#  define MEM_STATIC static __inline __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-#ifndef __has_builtin
-#  define __has_builtin(x) 0  /* compat. with non-clang compilers */
-#endif
-
-/* code only tested on 32 and 64 bits systems */
-#define MEM_STATIC_ASSERT(c)   { enum { MEM_static_assert = 1/(int)(!!(c)) }; }
-MEM_STATIC void MEM_check(void) { MEM_STATIC_ASSERT((sizeof(size_t)==4) || (sizeof(size_t)==8)); }
-
-
-/*-**************************************************************
-*  Basic Types
-*****************************************************************/
-#if  !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) )
-# include <stdint.h>
-  typedef   uint8_t BYTE;
-  typedef  uint16_t U16;
-  typedef   int16_t S16;
-  typedef  uint32_t U32;
-  typedef   int32_t S32;
-  typedef  uint64_t U64;
-  typedef   int64_t S64;
-#else
-# include <limits.h>
-#if CHAR_BIT != 8
-#  error "this implementation requires char to be exactly 8-bit type"
-#endif
-  typedef unsigned char      BYTE;
-#if USHRT_MAX != 65535
-#  error "this implementation requires short to be exactly 16-bit type"
-#endif
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-#if UINT_MAX != 4294967295
-#  error "this implementation requires int to be exactly 32-bit type"
-#endif
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-/* note : there are no limits defined for long long type in C90.
- * limits exist in C99, however, in such case, <stdint.h> is preferred */
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/*-**************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS :
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (i.e., not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets depending on alignment.
- *            In some circumstances, it's the only known way to get the most performance (i.e. GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif defined(__INTEL_COMPILER) || defined(__GNUC__)
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(size_t)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(size_t)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard, by lying on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-MEM_STATIC size_t MEM_readST(const void* memPtr) { return *(const size_t*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-MEM_STATIC void MEM_write32(void* memPtr, U32 value) { *(U32*)memPtr = value; }
-MEM_STATIC void MEM_write64(void* memPtr, U64 value) { *(U64*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-#if defined(_MSC_VER) || (defined(__INTEL_COMPILER) && defined(WIN32))
-    __pragma( pack(push, 1) )
-    typedef struct { U16 v; } unalign16;
-    typedef struct { U32 v; } unalign32;
-    typedef struct { U64 v; } unalign64;
-    typedef struct { size_t v; } unalignArch;
-    __pragma( pack(pop) )
-#else
-    typedef struct { U16 v; } __attribute__((packed)) unalign16;
-    typedef struct { U32 v; } __attribute__((packed)) unalign32;
-    typedef struct { U64 v; } __attribute__((packed)) unalign64;
-    typedef struct { size_t v; } __attribute__((packed)) unalignArch;
-#endif
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign16*)ptr)->v; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign32*)ptr)->v; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign64*)ptr)->v; }
-MEM_STATIC size_t MEM_readST(const void* ptr) { return ((const unalignArch*)ptr)->v; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign16*)memPtr)->v = value; }
-MEM_STATIC void MEM_write32(void* memPtr, U32 value) { ((unalign32*)memPtr)->v = value; }
-MEM_STATIC void MEM_write64(void* memPtr, U64 value) { ((unalign64*)memPtr)->v = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC size_t MEM_readST(const void* memPtr)
-{
-    size_t val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-MEM_STATIC void MEM_write32(void* memPtr, U32 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-MEM_STATIC void MEM_write64(void* memPtr, U64 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-#endif /* MEM_FORCE_MEMORY_ACCESS */
-
-MEM_STATIC U32 MEM_swap32(U32 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_ulong(in);
-#elif (defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)) \
-  || (defined(__clang__) && __has_builtin(__builtin_bswap32))
-    return __builtin_bswap32(in);
-#else
-    return  ((in << 24) & 0xff000000 ) |
-            ((in <<  8) & 0x00ff0000 ) |
-            ((in >>  8) & 0x0000ff00 ) |
-            ((in >> 24) & 0x000000ff );
-#endif
-}
-
-MEM_STATIC U64 MEM_swap64(U64 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_uint64(in);
-#elif (defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)) \
-  || (defined(__clang__) && __has_builtin(__builtin_bswap64))
-    return __builtin_bswap64(in);
-#else
-    return  ((in << 56) & 0xff00000000000000ULL) |
-            ((in << 40) & 0x00ff000000000000ULL) |
-            ((in << 24) & 0x0000ff0000000000ULL) |
-            ((in << 8)  & 0x000000ff00000000ULL) |
-            ((in >> 8)  & 0x00000000ff000000ULL) |
-            ((in >> 24) & 0x0000000000ff0000ULL) |
-            ((in >> 40) & 0x000000000000ff00ULL) |
-            ((in >> 56) & 0x00000000000000ffULL);
-#endif
-}
-
-MEM_STATIC size_t MEM_swapST(size_t in)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_swap32((U32)in);
-    else
-        return (size_t)MEM_swap64((U64)in);
-}
-
-/*=== Little endian r/w ===*/
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian()) {
-        MEM_write16(memPtr, val);
-    } else {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE24(const void* memPtr)
-{
-    return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-MEM_STATIC void MEM_writeLE24(void* memPtr, U32 val)
-{
-    MEM_writeLE16(memPtr, (U16)val);
-    ((BYTE*)memPtr)[2] = (BYTE)(val>>16);
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-        return MEM_swap32(MEM_read32(memPtr));
-}
-
-MEM_STATIC void MEM_writeLE32(void* memPtr, U32 val32)
-{
-    if (MEM_isLittleEndian())
-        MEM_write32(memPtr, val32);
-    else
-        MEM_write32(memPtr, MEM_swap32(val32));
-}
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-        return MEM_swap64(MEM_read64(memPtr));
-}
-
-MEM_STATIC void MEM_writeLE64(void* memPtr, U64 val64)
-{
-    if (MEM_isLittleEndian())
-        MEM_write64(memPtr, val64);
-    else
-        MEM_write64(memPtr, MEM_swap64(val64));
-}
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-MEM_STATIC void MEM_writeLEST(void* memPtr, size_t val)
-{
-    if (MEM_32bits())
-        MEM_writeLE32(memPtr, (U32)val);
-    else
-        MEM_writeLE64(memPtr, (U64)val);
-}
-
-/*=== Big endian r/w ===*/
-
-MEM_STATIC U32 MEM_readBE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_swap32(MEM_read32(memPtr));
-    else
-        return MEM_read32(memPtr);
-}
-
-MEM_STATIC void MEM_writeBE32(void* memPtr, U32 val32)
-{
-    if (MEM_isLittleEndian())
-        MEM_write32(memPtr, MEM_swap32(val32));
-    else
-        MEM_write32(memPtr, val32);
-}
-
-MEM_STATIC U64 MEM_readBE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_swap64(MEM_read64(memPtr));
-    else
-        return MEM_read64(memPtr);
-}
-
-MEM_STATIC void MEM_writeBE64(void* memPtr, U64 val64)
-{
-    if (MEM_isLittleEndian())
-        MEM_write64(memPtr, MEM_swap64(val64));
-    else
-        MEM_write64(memPtr, val64);
-}
-
-MEM_STATIC size_t MEM_readBEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readBE32(memPtr);
-    else
-        return (size_t)MEM_readBE64(memPtr);
-}
-
-MEM_STATIC void MEM_writeBEST(void* memPtr, size_t val)
-{
-    if (MEM_32bits())
-        MEM_writeBE32(memPtr, (U32)val);
-    else
-        MEM_writeBE64(memPtr, (U64)val);
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/pool.c b/vendor/github.com/DataDog/zstd/pool.c
deleted file mode 100644
index 7a82945..0000000
--- a/vendor/github.com/DataDog/zstd/pool.c
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/* ======   Dependencies   ======= */
-#include <stddef.h>    /* size_t */
-#include "debug.h"     /* assert */
-#include "zstd_internal.h"  /* ZSTD_malloc, ZSTD_free */
-#include "pool.h"
-
-/* ======   Compiler specifics   ====== */
-#if defined(_MSC_VER)
-#  pragma warning(disable : 4204)        /* disable: C4204: non-constant aggregate initializer */
-#endif
-
-
-#ifdef ZSTD_MULTITHREAD
-
-#include "threading.h"   /* pthread adaptation */
-
-/* A job is a function and an opaque argument */
-typedef struct POOL_job_s {
-    POOL_function function;
-    void *opaque;
-} POOL_job;
-
-struct POOL_ctx_s {
-    ZSTD_customMem customMem;
-    /* Keep track of the threads */
-    ZSTD_pthread_t* threads;
-    size_t threadCapacity;
-    size_t threadLimit;
-
-    /* The queue is a circular buffer */
-    POOL_job *queue;
-    size_t queueHead;
-    size_t queueTail;
-    size_t queueSize;
-
-    /* The number of threads working on jobs */
-    size_t numThreadsBusy;
-    /* Indicates if the queue is empty */
-    int queueEmpty;
-
-    /* The mutex protects the queue */
-    ZSTD_pthread_mutex_t queueMutex;
-    /* Condition variable for pushers to wait on when the queue is full */
-    ZSTD_pthread_cond_t queuePushCond;
-    /* Condition variables for poppers to wait on when the queue is empty */
-    ZSTD_pthread_cond_t queuePopCond;
-    /* Indicates if the queue is shutting down */
-    int shutdown;
-};
-
-/* POOL_thread() :
- * Work thread for the thread pool.
- * Waits for jobs and executes them.
- * @returns : NULL on failure else non-null.
- */
-static void* POOL_thread(void* opaque) {
-    POOL_ctx* const ctx = (POOL_ctx*)opaque;
-    if (!ctx) { return NULL; }
-    for (;;) {
-        /* Lock the mutex and wait for a non-empty queue or until shutdown */
-        ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-
-        while ( ctx->queueEmpty
-            || (ctx->numThreadsBusy >= ctx->threadLimit) ) {
-            if (ctx->shutdown) {
-                /* even if !queueEmpty, (possible if numThreadsBusy >= threadLimit),
-                 * a few threads will be shutdown while !queueEmpty,
-                 * but enough threads will remain active to finish the queue */
-                ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-                return opaque;
-            }
-            ZSTD_pthread_cond_wait(&ctx->queuePopCond, &ctx->queueMutex);
-        }
-        /* Pop a job off the queue */
-        {   POOL_job const job = ctx->queue[ctx->queueHead];
-            ctx->queueHead = (ctx->queueHead + 1) % ctx->queueSize;
-            ctx->numThreadsBusy++;
-            ctx->queueEmpty = ctx->queueHead == ctx->queueTail;
-            /* Unlock the mutex, signal a pusher, and run the job */
-            ZSTD_pthread_cond_signal(&ctx->queuePushCond);
-            ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-
-            job.function(job.opaque);
-
-            /* If the intended queue size was 0, signal after finishing job */
-            ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-            ctx->numThreadsBusy--;
-            if (ctx->queueSize == 1) {
-                ZSTD_pthread_cond_signal(&ctx->queuePushCond);
-            }
-            ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-        }
-    }  /* for (;;) */
-    assert(0);  /* Unreachable */
-}
-
-POOL_ctx* POOL_create(size_t numThreads, size_t queueSize) {
-    return POOL_create_advanced(numThreads, queueSize, ZSTD_defaultCMem);
-}
-
-POOL_ctx* POOL_create_advanced(size_t numThreads, size_t queueSize,
-                               ZSTD_customMem customMem) {
-    POOL_ctx* ctx;
-    /* Check parameters */
-    if (!numThreads) { return NULL; }
-    /* Allocate the context and zero initialize */
-    ctx = (POOL_ctx*)ZSTD_calloc(sizeof(POOL_ctx), customMem);
-    if (!ctx) { return NULL; }
-    /* Initialize the job queue.
-     * It needs one extra space since one space is wasted to differentiate
-     * empty and full queues.
-     */
-    ctx->queueSize = queueSize + 1;
-    ctx->queue = (POOL_job*)ZSTD_malloc(ctx->queueSize * sizeof(POOL_job), customMem);
-    ctx->queueHead = 0;
-    ctx->queueTail = 0;
-    ctx->numThreadsBusy = 0;
-    ctx->queueEmpty = 1;
-    (void)ZSTD_pthread_mutex_init(&ctx->queueMutex, NULL);
-    (void)ZSTD_pthread_cond_init(&ctx->queuePushCond, NULL);
-    (void)ZSTD_pthread_cond_init(&ctx->queuePopCond, NULL);
-    ctx->shutdown = 0;
-    /* Allocate space for the thread handles */
-    ctx->threads = (ZSTD_pthread_t*)ZSTD_malloc(numThreads * sizeof(ZSTD_pthread_t), customMem);
-    ctx->threadCapacity = 0;
-    ctx->customMem = customMem;
-    /* Check for errors */
-    if (!ctx->threads || !ctx->queue) { POOL_free(ctx); return NULL; }
-    /* Initialize the threads */
-    {   size_t i;
-        for (i = 0; i < numThreads; ++i) {
-            if (ZSTD_pthread_create(&ctx->threads[i], NULL, &POOL_thread, ctx)) {
-                ctx->threadCapacity = i;
-                POOL_free(ctx);
-                return NULL;
-        }   }
-        ctx->threadCapacity = numThreads;
-        ctx->threadLimit = numThreads;
-    }
-    return ctx;
-}
-
-/*! POOL_join() :
-    Shutdown the queue, wake any sleeping threads, and join all of the threads.
-*/
-static void POOL_join(POOL_ctx* ctx) {
-    /* Shut down the queue */
-    ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-    ctx->shutdown = 1;
-    ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-    /* Wake up sleeping threads */
-    ZSTD_pthread_cond_broadcast(&ctx->queuePushCond);
-    ZSTD_pthread_cond_broadcast(&ctx->queuePopCond);
-    /* Join all of the threads */
-    {   size_t i;
-        for (i = 0; i < ctx->threadCapacity; ++i) {
-            ZSTD_pthread_join(ctx->threads[i], NULL);  /* note : could fail */
-    }   }
-}
-
-void POOL_free(POOL_ctx *ctx) {
-    if (!ctx) { return; }
-    POOL_join(ctx);
-    ZSTD_pthread_mutex_destroy(&ctx->queueMutex);
-    ZSTD_pthread_cond_destroy(&ctx->queuePushCond);
-    ZSTD_pthread_cond_destroy(&ctx->queuePopCond);
-    ZSTD_free(ctx->queue, ctx->customMem);
-    ZSTD_free(ctx->threads, ctx->customMem);
-    ZSTD_free(ctx, ctx->customMem);
-}
-
-
-
-size_t POOL_sizeof(POOL_ctx *ctx) {
-    if (ctx==NULL) return 0;  /* supports sizeof NULL */
-    return sizeof(*ctx)
-        + ctx->queueSize * sizeof(POOL_job)
-        + ctx->threadCapacity * sizeof(ZSTD_pthread_t);
-}
-
-
-/* @return : 0 on success, 1 on error */
-static int POOL_resize_internal(POOL_ctx* ctx, size_t numThreads)
-{
-    if (numThreads <= ctx->threadCapacity) {
-        if (!numThreads) return 1;
-        ctx->threadLimit = numThreads;
-        return 0;
-    }
-    /* numThreads > threadCapacity */
-    {   ZSTD_pthread_t* const threadPool = (ZSTD_pthread_t*)ZSTD_malloc(numThreads * sizeof(ZSTD_pthread_t), ctx->customMem);
-        if (!threadPool) return 1;
-        /* replace existing thread pool */
-        memcpy(threadPool, ctx->threads, ctx->threadCapacity * sizeof(*threadPool));
-        ZSTD_free(ctx->threads, ctx->customMem);
-        ctx->threads = threadPool;
-        /* Initialize additional threads */
-        {   size_t threadId;
-            for (threadId = ctx->threadCapacity; threadId < numThreads; ++threadId) {
-                if (ZSTD_pthread_create(&threadPool[threadId], NULL, &POOL_thread, ctx)) {
-                    ctx->threadCapacity = threadId;
-                    return 1;
-            }   }
-    }   }
-    /* successfully expanded */
-    ctx->threadCapacity = numThreads;
-    ctx->threadLimit = numThreads;
-    return 0;
-}
-
-/* @return : 0 on success, 1 on error */
-int POOL_resize(POOL_ctx* ctx, size_t numThreads)
-{
-    int result;
-    if (ctx==NULL) return 1;
-    ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-    result = POOL_resize_internal(ctx, numThreads);
-    ZSTD_pthread_cond_broadcast(&ctx->queuePopCond);
-    ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-    return result;
-}
-
-/**
- * Returns 1 if the queue is full and 0 otherwise.
- *
- * When queueSize is 1 (pool was created with an intended queueSize of 0),
- * then a queue is empty if there is a thread free _and_ no job is waiting.
- */
-static int isQueueFull(POOL_ctx const* ctx) {
-    if (ctx->queueSize > 1) {
-        return ctx->queueHead == ((ctx->queueTail + 1) % ctx->queueSize);
-    } else {
-        return (ctx->numThreadsBusy == ctx->threadLimit) ||
-               !ctx->queueEmpty;
-    }
-}
-
-
-static void POOL_add_internal(POOL_ctx* ctx, POOL_function function, void *opaque)
-{
-    POOL_job const job = {function, opaque};
-    assert(ctx != NULL);
-    if (ctx->shutdown) return;
-
-    ctx->queueEmpty = 0;
-    ctx->queue[ctx->queueTail] = job;
-    ctx->queueTail = (ctx->queueTail + 1) % ctx->queueSize;
-    ZSTD_pthread_cond_signal(&ctx->queuePopCond);
-}
-
-void POOL_add(POOL_ctx* ctx, POOL_function function, void* opaque)
-{
-    assert(ctx != NULL);
-    ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-    /* Wait until there is space in the queue for the new job */
-    while (isQueueFull(ctx) && (!ctx->shutdown)) {
-        ZSTD_pthread_cond_wait(&ctx->queuePushCond, &ctx->queueMutex);
-    }
-    POOL_add_internal(ctx, function, opaque);
-    ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-}
-
-
-int POOL_tryAdd(POOL_ctx* ctx, POOL_function function, void* opaque)
-{
-    assert(ctx != NULL);
-    ZSTD_pthread_mutex_lock(&ctx->queueMutex);
-    if (isQueueFull(ctx)) {
-        ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-        return 0;
-    }
-    POOL_add_internal(ctx, function, opaque);
-    ZSTD_pthread_mutex_unlock(&ctx->queueMutex);
-    return 1;
-}
-
-
-#else  /* ZSTD_MULTITHREAD  not defined */
-
-/* ========================== */
-/* No multi-threading support */
-/* ========================== */
-
-
-/* We don't need any data, but if it is empty, malloc() might return NULL. */
-struct POOL_ctx_s {
-    int dummy;
-};
-static POOL_ctx g_ctx;
-
-POOL_ctx* POOL_create(size_t numThreads, size_t queueSize) {
-    return POOL_create_advanced(numThreads, queueSize, ZSTD_defaultCMem);
-}
-
-POOL_ctx* POOL_create_advanced(size_t numThreads, size_t queueSize, ZSTD_customMem customMem) {
-    (void)numThreads;
-    (void)queueSize;
-    (void)customMem;
-    return &g_ctx;
-}
-
-void POOL_free(POOL_ctx* ctx) {
-    assert(!ctx || ctx == &g_ctx);
-    (void)ctx;
-}
-
-int POOL_resize(POOL_ctx* ctx, size_t numThreads) {
-    (void)ctx; (void)numThreads;
-    return 0;
-}
-
-void POOL_add(POOL_ctx* ctx, POOL_function function, void* opaque) {
-    (void)ctx;
-    function(opaque);
-}
-
-int POOL_tryAdd(POOL_ctx* ctx, POOL_function function, void* opaque) {
-    (void)ctx;
-    function(opaque);
-    return 1;
-}
-
-size_t POOL_sizeof(POOL_ctx* ctx) {
-    if (ctx==NULL) return 0;  /* supports sizeof NULL */
-    assert(ctx == &g_ctx);
-    return sizeof(*ctx);
-}
-
-#endif  /* ZSTD_MULTITHREAD */
diff --git a/vendor/github.com/DataDog/zstd/pool.h b/vendor/github.com/DataDog/zstd/pool.h
deleted file mode 100644
index 458d37f..0000000
--- a/vendor/github.com/DataDog/zstd/pool.h
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef POOL_H
-#define POOL_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-#include <stddef.h>   /* size_t */
-#define ZSTD_STATIC_LINKING_ONLY   /* ZSTD_customMem */
-#include "zstd.h"
-
-typedef struct POOL_ctx_s POOL_ctx;
-
-/*! POOL_create() :
- *  Create a thread pool with at most `numThreads` threads.
- * `numThreads` must be at least 1.
- *  The maximum number of queued jobs before blocking is `queueSize`.
- * @return : POOL_ctx pointer on success, else NULL.
-*/
-POOL_ctx* POOL_create(size_t numThreads, size_t queueSize);
-
-POOL_ctx* POOL_create_advanced(size_t numThreads, size_t queueSize,
-                               ZSTD_customMem customMem);
-
-/*! POOL_free() :
- *  Free a thread pool returned by POOL_create().
- */
-void POOL_free(POOL_ctx* ctx);
-
-/*! POOL_resize() :
- *  Expands or shrinks pool's number of threads.
- *  This is more efficient than releasing + creating a new context,
- *  since it tries to preserve and re-use existing threads.
- * `numThreads` must be at least 1.
- * @return : 0 when resize was successful,
- *           !0 (typically 1) if there is an error.
- *    note : only numThreads can be resized, queueSize remains unchanged.
- */
-int POOL_resize(POOL_ctx* ctx, size_t numThreads);
-
-/*! POOL_sizeof() :
- * @return threadpool memory usage
- *  note : compatible with NULL (returns 0 in this case)
- */
-size_t POOL_sizeof(POOL_ctx* ctx);
-
-/*! POOL_function :
- *  The function type that can be added to a thread pool.
- */
-typedef void (*POOL_function)(void*);
-
-/*! POOL_add() :
- *  Add the job `function(opaque)` to the thread pool. `ctx` must be valid.
- *  Possibly blocks until there is room in the queue.
- *  Note : The function may be executed asynchronously,
- *         therefore, `opaque` must live until function has been completed.
- */
-void POOL_add(POOL_ctx* ctx, POOL_function function, void* opaque);
-
-
-/*! POOL_tryAdd() :
- *  Add the job `function(opaque)` to thread pool _if_ a worker is available.
- *  Returns immediately even if not (does not block).
- * @return : 1 if successful, 0 if not.
- */
-int POOL_tryAdd(POOL_ctx* ctx, POOL_function function, void* opaque);
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif
diff --git a/vendor/github.com/DataDog/zstd/threading.c b/vendor/github.com/DataDog/zstd/threading.c
deleted file mode 100644
index f3d4fa8..0000000
--- a/vendor/github.com/DataDog/zstd/threading.c
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Copyright (c) 2016 Tino Reichardt
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- *
- * You can contact the author at:
- * - zstdmt source repository: https://github.com/mcmilk/zstdmt
- */
-
-/**
- * This file will hold wrapper for systems, which do not support pthreads
- */
-
-/* create fake symbol to avoid empty translation unit warning */
-int g_ZSTD_threading_useless_symbol;
-
-#if defined(ZSTD_MULTITHREAD) && defined(_WIN32)
-
-/**
- * Windows minimalist Pthread Wrapper, based on :
- * http://www.cse.wustl.edu/~schmidt/win32-cv-1.html
- */
-
-
-/* ===  Dependencies  === */
-#include <process.h>
-#include <errno.h>
-#include "threading.h"
-
-
-/* ===  Implementation  === */
-
-static unsigned __stdcall worker(void *arg)
-{
-    ZSTD_pthread_t* const thread = (ZSTD_pthread_t*) arg;
-    thread->arg = thread->start_routine(thread->arg);
-    return 0;
-}
-
-int ZSTD_pthread_create(ZSTD_pthread_t* thread, const void* unused,
-            void* (*start_routine) (void*), void* arg)
-{
-    (void)unused;
-    thread->arg = arg;
-    thread->start_routine = start_routine;
-    thread->handle = (HANDLE) _beginthreadex(NULL, 0, worker, thread, 0, NULL);
-
-    if (!thread->handle)
-        return errno;
-    else
-        return 0;
-}
-
-int ZSTD_pthread_join(ZSTD_pthread_t thread, void **value_ptr)
-{
-    DWORD result;
-
-    if (!thread.handle) return 0;
-
-    result = WaitForSingleObject(thread.handle, INFINITE);
-    switch (result) {
-    case WAIT_OBJECT_0:
-        if (value_ptr) *value_ptr = thread.arg;
-        return 0;
-    case WAIT_ABANDONED:
-        return EINVAL;
-    default:
-        return GetLastError();
-    }
-}
-
-#endif   /* ZSTD_MULTITHREAD */
diff --git a/vendor/github.com/DataDog/zstd/threading.h b/vendor/github.com/DataDog/zstd/threading.h
deleted file mode 100644
index d806c89..0000000
--- a/vendor/github.com/DataDog/zstd/threading.h
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Copyright (c) 2016 Tino Reichardt
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- *
- * You can contact the author at:
- * - zstdmt source repository: https://github.com/mcmilk/zstdmt
- */
-
-#ifndef THREADING_H_938743
-#define THREADING_H_938743
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#if defined(ZSTD_MULTITHREAD) && defined(_WIN32)
-
-/**
- * Windows minimalist Pthread Wrapper, based on :
- * http://www.cse.wustl.edu/~schmidt/win32-cv-1.html
- */
-#ifdef WINVER
-#  undef WINVER
-#endif
-#define WINVER       0x0600
-
-#ifdef _WIN32_WINNT
-#  undef _WIN32_WINNT
-#endif
-#define _WIN32_WINNT 0x0600
-
-#ifndef WIN32_LEAN_AND_MEAN
-#  define WIN32_LEAN_AND_MEAN
-#endif
-
-#undef ERROR   /* reported already defined on VS 2015 (Rich Geldreich) */
-#include <windows.h>
-#undef ERROR
-#define ERROR(name) ZSTD_ERROR(name)
-
-
-/* mutex */
-#define ZSTD_pthread_mutex_t           CRITICAL_SECTION
-#define ZSTD_pthread_mutex_init(a, b)  ((void)(b), InitializeCriticalSection((a)), 0)
-#define ZSTD_pthread_mutex_destroy(a)  DeleteCriticalSection((a))
-#define ZSTD_pthread_mutex_lock(a)     EnterCriticalSection((a))
-#define ZSTD_pthread_mutex_unlock(a)   LeaveCriticalSection((a))
-
-/* condition variable */
-#define ZSTD_pthread_cond_t             CONDITION_VARIABLE
-#define ZSTD_pthread_cond_init(a, b)    ((void)(b), InitializeConditionVariable((a)), 0)
-#define ZSTD_pthread_cond_destroy(a)    ((void)(a))
-#define ZSTD_pthread_cond_wait(a, b)    SleepConditionVariableCS((a), (b), INFINITE)
-#define ZSTD_pthread_cond_signal(a)     WakeConditionVariable((a))
-#define ZSTD_pthread_cond_broadcast(a)  WakeAllConditionVariable((a))
-
-/* ZSTD_pthread_create() and ZSTD_pthread_join() */
-typedef struct {
-    HANDLE handle;
-    void* (*start_routine)(void*);
-    void* arg;
-} ZSTD_pthread_t;
-
-int ZSTD_pthread_create(ZSTD_pthread_t* thread, const void* unused,
-                   void* (*start_routine) (void*), void* arg);
-
-int ZSTD_pthread_join(ZSTD_pthread_t thread, void** value_ptr);
-
-/**
- * add here more wrappers as required
- */
-
-
-#elif defined(ZSTD_MULTITHREAD)   /* posix assumed ; need a better detection method */
-/* ===   POSIX Systems   === */
-#  include <pthread.h>
-
-#define ZSTD_pthread_mutex_t            pthread_mutex_t
-#define ZSTD_pthread_mutex_init(a, b)   pthread_mutex_init((a), (b))
-#define ZSTD_pthread_mutex_destroy(a)   pthread_mutex_destroy((a))
-#define ZSTD_pthread_mutex_lock(a)      pthread_mutex_lock((a))
-#define ZSTD_pthread_mutex_unlock(a)    pthread_mutex_unlock((a))
-
-#define ZSTD_pthread_cond_t             pthread_cond_t
-#define ZSTD_pthread_cond_init(a, b)    pthread_cond_init((a), (b))
-#define ZSTD_pthread_cond_destroy(a)    pthread_cond_destroy((a))
-#define ZSTD_pthread_cond_wait(a, b)    pthread_cond_wait((a), (b))
-#define ZSTD_pthread_cond_signal(a)     pthread_cond_signal((a))
-#define ZSTD_pthread_cond_broadcast(a)  pthread_cond_broadcast((a))
-
-#define ZSTD_pthread_t                  pthread_t
-#define ZSTD_pthread_create(a, b, c, d) pthread_create((a), (b), (c), (d))
-#define ZSTD_pthread_join(a, b)         pthread_join((a),(b))
-
-#else  /* ZSTD_MULTITHREAD not defined */
-/* No multithreading support */
-
-typedef int ZSTD_pthread_mutex_t;
-#define ZSTD_pthread_mutex_init(a, b)   ((void)(a), (void)(b), 0)
-#define ZSTD_pthread_mutex_destroy(a)   ((void)(a))
-#define ZSTD_pthread_mutex_lock(a)      ((void)(a))
-#define ZSTD_pthread_mutex_unlock(a)    ((void)(a))
-
-typedef int ZSTD_pthread_cond_t;
-#define ZSTD_pthread_cond_init(a, b)    ((void)(a), (void)(b), 0)
-#define ZSTD_pthread_cond_destroy(a)    ((void)(a))
-#define ZSTD_pthread_cond_wait(a, b)    ((void)(a), (void)(b))
-#define ZSTD_pthread_cond_signal(a)     ((void)(a))
-#define ZSTD_pthread_cond_broadcast(a)  ((void)(a))
-
-/* do not use ZSTD_pthread_t */
-
-#endif /* ZSTD_MULTITHREAD */
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* THREADING_H_938743 */
diff --git a/vendor/github.com/DataDog/zstd/travis_test_32.sh b/vendor/github.com/DataDog/zstd/travis_test_32.sh
deleted file mode 100644
index d29c86c..0000000
--- a/vendor/github.com/DataDog/zstd/travis_test_32.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/bin/bash
-# Get utilities
-yum -y -q -e 0 install wget tar unzip gcc
-
-# Get Go
-wget -q https://dl.google.com/go/go1.11.1.linux-386.tar.gz
-tar -C /usr/local -xzf go1.11.1.linux-386.tar.gz
-export PATH=$PATH:/usr/local/go/bin
-
-# Get payload
-wget -q https://github.com/DataDog/zstd/files/2246767/mr.zip
-unzip mr.zip
-
-# Build and run tests
-cd zstd
-go build
-PAYLOAD=$(pwd)/mr go test -v
-PAYLOAD=$(pwd)/mr go test -bench .
diff --git a/vendor/github.com/DataDog/zstd/update.txt b/vendor/github.com/DataDog/zstd/update.txt
deleted file mode 100644
index 1de939f..0000000
--- a/vendor/github.com/DataDog/zstd/update.txt
+++ /dev/null
@@ -1,56 +0,0 @@
-./lib/common/bitstream.h
-./lib/common/compiler.h
-./lib/compress/zstd_compress_internal.h
-./lib/compress/zstd_fast.h
-./lib/compress/zstd_double_fast.h
-./lib/compress/zstd_lazy.h
-./lib/compress/zstd_ldm.h
-./lib/dictBuilder/cover.c
-./lib/dictBuilder/divsufsort.c
-./lib/dictBuilder/divsufsort.h
-./lib/common/entropy_common.c
-./lib/common/error_private.c
-./lib/common/error_private.h
-./lib/compress/fse_compress.c
-./lib/common/fse_decompress.c
-./lib/common/fse.h
-./lib/compress/huf_compress.c
-./lib/decompress/huf_decompress.c
-./lib/common/huf.h
-./lib/common/mem.h
-./lib/common/pool.c
-./lib/common/pool.h
-./lib/common/threading.c
-./lib/common/threading.h
-./lib/common/xxhash.c
-./lib/common/xxhash.h
-./lib/deprecated/zbuff_common.c
-./lib/deprecated/zbuff_compress.c
-./lib/deprecated/zbuff_decompress.c
-./lib/deprecated/zbuff.h
-./lib/dictBuilder/zdict.c
-./lib/dictBuilder/zdict.h
-./lib/common/zstd_common.c
-./lib/compress/zstd_compress.c
-./lib/decompress/zstd_decompress.c
-./lib/common/zstd_errors.h
-./lib/zstd.h
-./lib/common/zstd_internal.h
-./lib/legacy/zstd_legacy.h
-./lib/compress/zstd_opt.c
-./lib/compress/zstd_opt.h
-./lib/legacy/zstd_v01.c
-./lib/legacy/zstd_v01.h
-./lib/legacy/zstd_v02.c
-./lib/legacy/zstd_v02.h
-./lib/legacy/zstd_v03.c
-./lib/legacy/zstd_v03.h
-./lib/legacy/zstd_v04.c
-./lib/legacy/zstd_v04.h
-./lib/legacy/zstd_v05.c
-./lib/legacy/zstd_v05.h
-./lib/legacy/zstd_v06.c
-./lib/legacy/zstd_v06.h
-./lib/legacy/zstd_v07.c
-./lib/legacy/zstd_v07.h
-
diff --git a/vendor/github.com/DataDog/zstd/xxhash.c b/vendor/github.com/DataDog/zstd/xxhash.c
deleted file mode 100644
index 30599aa..0000000
--- a/vendor/github.com/DataDog/zstd/xxhash.c
+++ /dev/null
@@ -1,876 +0,0 @@
-/*
-*  xxHash - Fast Hash algorithm
-*  Copyright (C) 2012-2016, Yann Collet
-*
-*  BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-*
-*  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
-*  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.
-*
-*  You can contact the author at :
-*  - xxHash homepage: http://www.xxhash.com
-*  - xxHash source repository : https://github.com/Cyan4973/xxHash
-*/
-
-
-/* *************************************
-*  Tuning parameters
-***************************************/
-/*!XXH_FORCE_MEMORY_ACCESS :
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method doesn't depend on compiler but violate C standard.
- *            It can generate buggy code on targets which do not support unaligned memory accesses.
- *            But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://stackoverflow.com/a/32095106/646947 for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef XXH_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define XXH_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define XXH_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-/*!XXH_ACCEPT_NULL_INPUT_POINTER :
- * If the input pointer is a null pointer, xxHash default behavior is to trigger a memory access error, since it is a bad pointer.
- * When this option is enabled, xxHash output for null input pointers will be the same as a null-length input.
- * By default, this option is disabled. To enable it, uncomment below define :
- */
-/* #define XXH_ACCEPT_NULL_INPUT_POINTER 1 */
-
-/*!XXH_FORCE_NATIVE_FORMAT :
- * By default, xxHash library provides endian-independent Hash values, based on little-endian convention.
- * Results are therefore identical for little-endian and big-endian CPU.
- * This comes at a performance cost for big-endian CPU, since some swapping is required to emulate little-endian format.
- * Should endian-independence be of no importance for your application, you may set the #define below to 1,
- * to improve speed for Big-endian CPU.
- * This option has no impact on Little_Endian CPU.
- */
-#ifndef XXH_FORCE_NATIVE_FORMAT   /* can be defined externally */
-#  define XXH_FORCE_NATIVE_FORMAT 0
-#endif
-
-/*!XXH_FORCE_ALIGN_CHECK :
- * This is a minor performance trick, only useful with lots of very small keys.
- * It means : check for aligned/unaligned input.
- * The check costs one initial branch per hash; set to 0 when the input data
- * is guaranteed to be aligned.
- */
-#ifndef XXH_FORCE_ALIGN_CHECK /* can be defined externally */
-#  if defined(__i386) || defined(_M_IX86) || defined(__x86_64__) || defined(_M_X64)
-#    define XXH_FORCE_ALIGN_CHECK 0
-#  else
-#    define XXH_FORCE_ALIGN_CHECK 1
-#  endif
-#endif
-
-
-/* *************************************
-*  Includes & Memory related functions
-***************************************/
-/* Modify the local functions below should you wish to use some other memory routines */
-/* for malloc(), free() */
-#include <stdlib.h>
-#include <stddef.h>     /* size_t */
-static void* XXH_malloc(size_t s) { return malloc(s); }
-static void  XXH_free  (void* p)  { free(p); }
-/* for memcpy() */
-#include <string.h>
-static void* XXH_memcpy(void* dest, const void* src, size_t size) { return memcpy(dest,src,size); }
-
-#ifndef XXH_STATIC_LINKING_ONLY
-#  define XXH_STATIC_LINKING_ONLY
-#endif
-#include "xxhash.h"
-
-
-/* *************************************
-*  Compiler Specific Options
-***************************************/
-#if defined (__GNUC__) || defined(__cplusplus) || defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#  define INLINE_KEYWORD inline
-#else
-#  define INLINE_KEYWORD
-#endif
-
-#if defined(__GNUC__)
-#  define FORCE_INLINE_ATTR __attribute__((always_inline))
-#elif defined(_MSC_VER)
-#  define FORCE_INLINE_ATTR __forceinline
-#else
-#  define FORCE_INLINE_ATTR
-#endif
-
-#define FORCE_INLINE_TEMPLATE static INLINE_KEYWORD FORCE_INLINE_ATTR
-
-
-#ifdef _MSC_VER
-#  pragma warning(disable : 4127)      /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/* *************************************
-*  Basic Types
-***************************************/
-#ifndef MEM_MODULE
-# define MEM_MODULE
-# if !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) )
-#   include <stdint.h>
-    typedef uint8_t  BYTE;
-    typedef uint16_t U16;
-    typedef uint32_t U32;
-    typedef  int32_t S32;
-    typedef uint64_t U64;
-#  else
-    typedef unsigned char      BYTE;
-    typedef unsigned short     U16;
-    typedef unsigned int       U32;
-    typedef   signed int       S32;
-    typedef unsigned long long U64;   /* if your compiler doesn't support unsigned long long, replace by another 64-bit type here. Note that xxhash.h will also need to be updated. */
-#  endif
-#endif
-
-
-#if (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==2))
-
-/* Force direct memory access. Only works on CPU which support unaligned memory access in hardware */
-static U32 XXH_read32(const void* memPtr) { return *(const U32*) memPtr; }
-static U64 XXH_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-#elif (defined(XXH_FORCE_MEMORY_ACCESS) && (XXH_FORCE_MEMORY_ACCESS==1))
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U32 u32; U64 u64; } __attribute__((packed)) unalign;
-
-static U32 XXH_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-static U64 XXH_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-#else
-
-/* portable and safe solution. Generally efficient.
- * see : http://stackoverflow.com/a/32095106/646947
- */
-
-static U32 XXH_read32(const void* memPtr)
-{
-    U32 val;
-    memcpy(&val, memPtr, sizeof(val));
-    return val;
-}
-
-static U64 XXH_read64(const void* memPtr)
-{
-    U64 val;
-    memcpy(&val, memPtr, sizeof(val));
-    return val;
-}
-
-#endif   /* XXH_FORCE_DIRECT_MEMORY_ACCESS */
-
-
-/* ****************************************
-*  Compiler-specific Functions and Macros
-******************************************/
-#define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__)
-
-/* Note : although _rotl exists for minGW (GCC under windows), performance seems poor */
-#if defined(_MSC_VER)
-#  define XXH_rotl32(x,r) _rotl(x,r)
-#  define XXH_rotl64(x,r) _rotl64(x,r)
-#else
-#  define XXH_rotl32(x,r) ((x << r) | (x >> (32 - r)))
-#  define XXH_rotl64(x,r) ((x << r) | (x >> (64 - r)))
-#endif
-
-#if defined(_MSC_VER)     /* Visual Studio */
-#  define XXH_swap32 _byteswap_ulong
-#  define XXH_swap64 _byteswap_uint64
-#elif GCC_VERSION >= 403
-#  define XXH_swap32 __builtin_bswap32
-#  define XXH_swap64 __builtin_bswap64
-#else
-static U32 XXH_swap32 (U32 x)
-{
-    return  ((x << 24) & 0xff000000 ) |
-            ((x <<  8) & 0x00ff0000 ) |
-            ((x >>  8) & 0x0000ff00 ) |
-            ((x >> 24) & 0x000000ff );
-}
-static U64 XXH_swap64 (U64 x)
-{
-    return  ((x << 56) & 0xff00000000000000ULL) |
-            ((x << 40) & 0x00ff000000000000ULL) |
-            ((x << 24) & 0x0000ff0000000000ULL) |
-            ((x << 8)  & 0x000000ff00000000ULL) |
-            ((x >> 8)  & 0x00000000ff000000ULL) |
-            ((x >> 24) & 0x0000000000ff0000ULL) |
-            ((x >> 40) & 0x000000000000ff00ULL) |
-            ((x >> 56) & 0x00000000000000ffULL);
-}
-#endif
-
-
-/* *************************************
-*  Architecture Macros
-***************************************/
-typedef enum { XXH_bigEndian=0, XXH_littleEndian=1 } XXH_endianess;
-
-/* XXH_CPU_LITTLE_ENDIAN can be defined externally, for example on the compiler command line */
-#ifndef XXH_CPU_LITTLE_ENDIAN
-    static const int g_one = 1;
-#   define XXH_CPU_LITTLE_ENDIAN   (*(const char*)(&g_one))
-#endif
-
-
-/* ***************************
-*  Memory reads
-*****************************/
-typedef enum { XXH_aligned, XXH_unaligned } XXH_alignment;
-
-FORCE_INLINE_TEMPLATE U32 XXH_readLE32_align(const void* ptr, XXH_endianess endian, XXH_alignment align)
-{
-    if (align==XXH_unaligned)
-        return endian==XXH_littleEndian ? XXH_read32(ptr) : XXH_swap32(XXH_read32(ptr));
-    else
-        return endian==XXH_littleEndian ? *(const U32*)ptr : XXH_swap32(*(const U32*)ptr);
-}
-
-FORCE_INLINE_TEMPLATE U32 XXH_readLE32(const void* ptr, XXH_endianess endian)
-{
-    return XXH_readLE32_align(ptr, endian, XXH_unaligned);
-}
-
-static U32 XXH_readBE32(const void* ptr)
-{
-    return XXH_CPU_LITTLE_ENDIAN ? XXH_swap32(XXH_read32(ptr)) : XXH_read32(ptr);
-}
-
-FORCE_INLINE_TEMPLATE U64 XXH_readLE64_align(const void* ptr, XXH_endianess endian, XXH_alignment align)
-{
-    if (align==XXH_unaligned)
-        return endian==XXH_littleEndian ? XXH_read64(ptr) : XXH_swap64(XXH_read64(ptr));
-    else
-        return endian==XXH_littleEndian ? *(const U64*)ptr : XXH_swap64(*(const U64*)ptr);
-}
-
-FORCE_INLINE_TEMPLATE U64 XXH_readLE64(const void* ptr, XXH_endianess endian)
-{
-    return XXH_readLE64_align(ptr, endian, XXH_unaligned);
-}
-
-static U64 XXH_readBE64(const void* ptr)
-{
-    return XXH_CPU_LITTLE_ENDIAN ? XXH_swap64(XXH_read64(ptr)) : XXH_read64(ptr);
-}
-
-
-/* *************************************
-*  Macros
-***************************************/
-#define XXH_STATIC_ASSERT(c)   { enum { XXH_static_assert = 1/(int)(!!(c)) }; }    /* use only *after* variable declarations */
-
-
-/* *************************************
-*  Constants
-***************************************/
-static const U32 PRIME32_1 = 2654435761U;
-static const U32 PRIME32_2 = 2246822519U;
-static const U32 PRIME32_3 = 3266489917U;
-static const U32 PRIME32_4 =  668265263U;
-static const U32 PRIME32_5 =  374761393U;
-
-static const U64 PRIME64_1 = 11400714785074694791ULL;
-static const U64 PRIME64_2 = 14029467366897019727ULL;
-static const U64 PRIME64_3 =  1609587929392839161ULL;
-static const U64 PRIME64_4 =  9650029242287828579ULL;
-static const U64 PRIME64_5 =  2870177450012600261ULL;
-
-XXH_PUBLIC_API unsigned XXH_versionNumber (void) { return XXH_VERSION_NUMBER; }
-
-
-/* **************************
-*  Utils
-****************************/
-XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* restrict dstState, const XXH32_state_t* restrict srcState)
-{
-    memcpy(dstState, srcState, sizeof(*dstState));
-}
-
-XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* restrict dstState, const XXH64_state_t* restrict srcState)
-{
-    memcpy(dstState, srcState, sizeof(*dstState));
-}
-
-
-/* ***************************
-*  Simple Hash Functions
-*****************************/
-
-static U32 XXH32_round(U32 seed, U32 input)
-{
-    seed += input * PRIME32_2;
-    seed  = XXH_rotl32(seed, 13);
-    seed *= PRIME32_1;
-    return seed;
-}
-
-FORCE_INLINE_TEMPLATE U32 XXH32_endian_align(const void* input, size_t len, U32 seed, XXH_endianess endian, XXH_alignment align)
-{
-    const BYTE* p = (const BYTE*)input;
-    const BYTE* bEnd = p + len;
-    U32 h32;
-#define XXH_get32bits(p) XXH_readLE32_align(p, endian, align)
-
-#ifdef XXH_ACCEPT_NULL_INPUT_POINTER
-    if (p==NULL) {
-        len=0;
-        bEnd=p=(const BYTE*)(size_t)16;
-    }
-#endif
-
-    if (len>=16) {
-        const BYTE* const limit = bEnd - 16;
-        U32 v1 = seed + PRIME32_1 + PRIME32_2;
-        U32 v2 = seed + PRIME32_2;
-        U32 v3 = seed + 0;
-        U32 v4 = seed - PRIME32_1;
-
-        do {
-            v1 = XXH32_round(v1, XXH_get32bits(p)); p+=4;
-            v2 = XXH32_round(v2, XXH_get32bits(p)); p+=4;
-            v3 = XXH32_round(v3, XXH_get32bits(p)); p+=4;
-            v4 = XXH32_round(v4, XXH_get32bits(p)); p+=4;
-        } while (p<=limit);
-
-        h32 = XXH_rotl32(v1, 1) + XXH_rotl32(v2, 7) + XXH_rotl32(v3, 12) + XXH_rotl32(v4, 18);
-    } else {
-        h32  = seed + PRIME32_5;
-    }
-
-    h32 += (U32) len;
-
-    while (p+4<=bEnd) {
-        h32 += XXH_get32bits(p) * PRIME32_3;
-        h32  = XXH_rotl32(h32, 17) * PRIME32_4 ;
-        p+=4;
-    }
-
-    while (p<bEnd) {
-        h32 += (*p) * PRIME32_5;
-        h32 = XXH_rotl32(h32, 11) * PRIME32_1 ;
-        p++;
-    }
-
-    h32 ^= h32 >> 15;
-    h32 *= PRIME32_2;
-    h32 ^= h32 >> 13;
-    h32 *= PRIME32_3;
-    h32 ^= h32 >> 16;
-
-    return h32;
-}
-
-
-XXH_PUBLIC_API unsigned int XXH32 (const void* input, size_t len, unsigned int seed)
-{
-#if 0
-    /* Simple version, good for code maintenance, but unfortunately slow for small inputs */
-    XXH32_CREATESTATE_STATIC(state);
-    XXH32_reset(state, seed);
-    XXH32_update(state, input, len);
-    return XXH32_digest(state);
-#else
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if (XXH_FORCE_ALIGN_CHECK) {
-        if ((((size_t)input) & 3) == 0) {   /* Input is 4-bytes aligned, leverage the speed benefit */
-            if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-                return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned);
-            else
-                return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned);
-    }   }
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH32_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned);
-    else
-        return XXH32_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned);
-#endif
-}
-
-
-static U64 XXH64_round(U64 acc, U64 input)
-{
-    acc += input * PRIME64_2;
-    acc  = XXH_rotl64(acc, 31);
-    acc *= PRIME64_1;
-    return acc;
-}
-
-static U64 XXH64_mergeRound(U64 acc, U64 val)
-{
-    val  = XXH64_round(0, val);
-    acc ^= val;
-    acc  = acc * PRIME64_1 + PRIME64_4;
-    return acc;
-}
-
-FORCE_INLINE_TEMPLATE U64 XXH64_endian_align(const void* input, size_t len, U64 seed, XXH_endianess endian, XXH_alignment align)
-{
-    const BYTE* p = (const BYTE*)input;
-    const BYTE* const bEnd = p + len;
-    U64 h64;
-#define XXH_get64bits(p) XXH_readLE64_align(p, endian, align)
-
-#ifdef XXH_ACCEPT_NULL_INPUT_POINTER
-    if (p==NULL) {
-        len=0;
-        bEnd=p=(const BYTE*)(size_t)32;
-    }
-#endif
-
-    if (len>=32) {
-        const BYTE* const limit = bEnd - 32;
-        U64 v1 = seed + PRIME64_1 + PRIME64_2;
-        U64 v2 = seed + PRIME64_2;
-        U64 v3 = seed + 0;
-        U64 v4 = seed - PRIME64_1;
-
-        do {
-            v1 = XXH64_round(v1, XXH_get64bits(p)); p+=8;
-            v2 = XXH64_round(v2, XXH_get64bits(p)); p+=8;
-            v3 = XXH64_round(v3, XXH_get64bits(p)); p+=8;
-            v4 = XXH64_round(v4, XXH_get64bits(p)); p+=8;
-        } while (p<=limit);
-
-        h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18);
-        h64 = XXH64_mergeRound(h64, v1);
-        h64 = XXH64_mergeRound(h64, v2);
-        h64 = XXH64_mergeRound(h64, v3);
-        h64 = XXH64_mergeRound(h64, v4);
-
-    } else {
-        h64  = seed + PRIME64_5;
-    }
-
-    h64 += (U64) len;
-
-    while (p+8<=bEnd) {
-        U64 const k1 = XXH64_round(0, XXH_get64bits(p));
-        h64 ^= k1;
-        h64  = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4;
-        p+=8;
-    }
-
-    if (p+4<=bEnd) {
-        h64 ^= (U64)(XXH_get32bits(p)) * PRIME64_1;
-        h64 = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3;
-        p+=4;
-    }
-
-    while (p<bEnd) {
-        h64 ^= (*p) * PRIME64_5;
-        h64 = XXH_rotl64(h64, 11) * PRIME64_1;
-        p++;
-    }
-
-    h64 ^= h64 >> 33;
-    h64 *= PRIME64_2;
-    h64 ^= h64 >> 29;
-    h64 *= PRIME64_3;
-    h64 ^= h64 >> 32;
-
-    return h64;
-}
-
-
-XXH_PUBLIC_API unsigned long long XXH64 (const void* input, size_t len, unsigned long long seed)
-{
-#if 0
-    /* Simple version, good for code maintenance, but unfortunately slow for small inputs */
-    XXH64_CREATESTATE_STATIC(state);
-    XXH64_reset(state, seed);
-    XXH64_update(state, input, len);
-    return XXH64_digest(state);
-#else
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if (XXH_FORCE_ALIGN_CHECK) {
-        if ((((size_t)input) & 7)==0) {  /* Input is aligned, let's leverage the speed advantage */
-            if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-                return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_aligned);
-            else
-                return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_aligned);
-    }   }
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH64_endian_align(input, len, seed, XXH_littleEndian, XXH_unaligned);
-    else
-        return XXH64_endian_align(input, len, seed, XXH_bigEndian, XXH_unaligned);
-#endif
-}
-
-
-/* **************************************************
-*  Advanced Hash Functions
-****************************************************/
-
-XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void)
-{
-    return (XXH32_state_t*)XXH_malloc(sizeof(XXH32_state_t));
-}
-XXH_PUBLIC_API XXH_errorcode XXH32_freeState(XXH32_state_t* statePtr)
-{
-    XXH_free(statePtr);
-    return XXH_OK;
-}
-
-XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void)
-{
-    return (XXH64_state_t*)XXH_malloc(sizeof(XXH64_state_t));
-}
-XXH_PUBLIC_API XXH_errorcode XXH64_freeState(XXH64_state_t* statePtr)
-{
-    XXH_free(statePtr);
-    return XXH_OK;
-}
-
-
-/*** Hash feed ***/
-
-XXH_PUBLIC_API XXH_errorcode XXH32_reset(XXH32_state_t* statePtr, unsigned int seed)
-{
-    XXH32_state_t state;   /* using a local state to memcpy() in order to avoid strict-aliasing warnings */
-    memset(&state, 0, sizeof(state)-4);   /* do not write into reserved, for future removal */
-    state.v1 = seed + PRIME32_1 + PRIME32_2;
-    state.v2 = seed + PRIME32_2;
-    state.v3 = seed + 0;
-    state.v4 = seed - PRIME32_1;
-    memcpy(statePtr, &state, sizeof(state));
-    return XXH_OK;
-}
-
-
-XXH_PUBLIC_API XXH_errorcode XXH64_reset(XXH64_state_t* statePtr, unsigned long long seed)
-{
-    XXH64_state_t state;   /* using a local state to memcpy() in order to avoid strict-aliasing warnings */
-    memset(&state, 0, sizeof(state)-8);   /* do not write into reserved, for future removal */
-    state.v1 = seed + PRIME64_1 + PRIME64_2;
-    state.v2 = seed + PRIME64_2;
-    state.v3 = seed + 0;
-    state.v4 = seed - PRIME64_1;
-    memcpy(statePtr, &state, sizeof(state));
-    return XXH_OK;
-}
-
-
-FORCE_INLINE_TEMPLATE XXH_errorcode XXH32_update_endian (XXH32_state_t* state, const void* input, size_t len, XXH_endianess endian)
-{
-    const BYTE* p = (const BYTE*)input;
-    const BYTE* const bEnd = p + len;
-
-#ifdef XXH_ACCEPT_NULL_INPUT_POINTER
-    if (input==NULL) return XXH_ERROR;
-#endif
-
-    state->total_len_32 += (unsigned)len;
-    state->large_len |= (len>=16) | (state->total_len_32>=16);
-
-    if (state->memsize + len < 16)  {   /* fill in tmp buffer */
-        XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, len);
-        state->memsize += (unsigned)len;
-        return XXH_OK;
-    }
-
-    if (state->memsize) {   /* some data left from previous update */
-        XXH_memcpy((BYTE*)(state->mem32) + state->memsize, input, 16-state->memsize);
-        {   const U32* p32 = state->mem32;
-            state->v1 = XXH32_round(state->v1, XXH_readLE32(p32, endian)); p32++;
-            state->v2 = XXH32_round(state->v2, XXH_readLE32(p32, endian)); p32++;
-            state->v3 = XXH32_round(state->v3, XXH_readLE32(p32, endian)); p32++;
-            state->v4 = XXH32_round(state->v4, XXH_readLE32(p32, endian)); p32++;
-        }
-        p += 16-state->memsize;
-        state->memsize = 0;
-    }
-
-    if (p <= bEnd-16) {
-        const BYTE* const limit = bEnd - 16;
-        U32 v1 = state->v1;
-        U32 v2 = state->v2;
-        U32 v3 = state->v3;
-        U32 v4 = state->v4;
-
-        do {
-            v1 = XXH32_round(v1, XXH_readLE32(p, endian)); p+=4;
-            v2 = XXH32_round(v2, XXH_readLE32(p, endian)); p+=4;
-            v3 = XXH32_round(v3, XXH_readLE32(p, endian)); p+=4;
-            v4 = XXH32_round(v4, XXH_readLE32(p, endian)); p+=4;
-        } while (p<=limit);
-
-        state->v1 = v1;
-        state->v2 = v2;
-        state->v3 = v3;
-        state->v4 = v4;
-    }
-
-    if (p < bEnd) {
-        XXH_memcpy(state->mem32, p, (size_t)(bEnd-p));
-        state->memsize = (unsigned)(bEnd-p);
-    }
-
-    return XXH_OK;
-}
-
-XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* state_in, const void* input, size_t len)
-{
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH32_update_endian(state_in, input, len, XXH_littleEndian);
-    else
-        return XXH32_update_endian(state_in, input, len, XXH_bigEndian);
-}
-
-
-
-FORCE_INLINE_TEMPLATE U32 XXH32_digest_endian (const XXH32_state_t* state, XXH_endianess endian)
-{
-    const BYTE * p = (const BYTE*)state->mem32;
-    const BYTE* const bEnd = (const BYTE*)(state->mem32) + state->memsize;
-    U32 h32;
-
-    if (state->large_len) {
-        h32 = XXH_rotl32(state->v1, 1) + XXH_rotl32(state->v2, 7) + XXH_rotl32(state->v3, 12) + XXH_rotl32(state->v4, 18);
-    } else {
-        h32 = state->v3 /* == seed */ + PRIME32_5;
-    }
-
-    h32 += state->total_len_32;
-
-    while (p+4<=bEnd) {
-        h32 += XXH_readLE32(p, endian) * PRIME32_3;
-        h32  = XXH_rotl32(h32, 17) * PRIME32_4;
-        p+=4;
-    }
-
-    while (p<bEnd) {
-        h32 += (*p) * PRIME32_5;
-        h32  = XXH_rotl32(h32, 11) * PRIME32_1;
-        p++;
-    }
-
-    h32 ^= h32 >> 15;
-    h32 *= PRIME32_2;
-    h32 ^= h32 >> 13;
-    h32 *= PRIME32_3;
-    h32 ^= h32 >> 16;
-
-    return h32;
-}
-
-
-XXH_PUBLIC_API unsigned int XXH32_digest (const XXH32_state_t* state_in)
-{
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH32_digest_endian(state_in, XXH_littleEndian);
-    else
-        return XXH32_digest_endian(state_in, XXH_bigEndian);
-}
-
-
-
-/* **** XXH64 **** */
-
-FORCE_INLINE_TEMPLATE XXH_errorcode XXH64_update_endian (XXH64_state_t* state, const void* input, size_t len, XXH_endianess endian)
-{
-    const BYTE* p = (const BYTE*)input;
-    const BYTE* const bEnd = p + len;
-
-#ifdef XXH_ACCEPT_NULL_INPUT_POINTER
-    if (input==NULL) return XXH_ERROR;
-#endif
-
-    state->total_len += len;
-
-    if (state->memsize + len < 32) {  /* fill in tmp buffer */
-        XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, len);
-        state->memsize += (U32)len;
-        return XXH_OK;
-    }
-
-    if (state->memsize) {   /* tmp buffer is full */
-        XXH_memcpy(((BYTE*)state->mem64) + state->memsize, input, 32-state->memsize);
-        state->v1 = XXH64_round(state->v1, XXH_readLE64(state->mem64+0, endian));
-        state->v2 = XXH64_round(state->v2, XXH_readLE64(state->mem64+1, endian));
-        state->v3 = XXH64_round(state->v3, XXH_readLE64(state->mem64+2, endian));
-        state->v4 = XXH64_round(state->v4, XXH_readLE64(state->mem64+3, endian));
-        p += 32-state->memsize;
-        state->memsize = 0;
-    }
-
-    if (p+32 <= bEnd) {
-        const BYTE* const limit = bEnd - 32;
-        U64 v1 = state->v1;
-        U64 v2 = state->v2;
-        U64 v3 = state->v3;
-        U64 v4 = state->v4;
-
-        do {
-            v1 = XXH64_round(v1, XXH_readLE64(p, endian)); p+=8;
-            v2 = XXH64_round(v2, XXH_readLE64(p, endian)); p+=8;
-            v3 = XXH64_round(v3, XXH_readLE64(p, endian)); p+=8;
-            v4 = XXH64_round(v4, XXH_readLE64(p, endian)); p+=8;
-        } while (p<=limit);
-
-        state->v1 = v1;
-        state->v2 = v2;
-        state->v3 = v3;
-        state->v4 = v4;
-    }
-
-    if (p < bEnd) {
-        XXH_memcpy(state->mem64, p, (size_t)(bEnd-p));
-        state->memsize = (unsigned)(bEnd-p);
-    }
-
-    return XXH_OK;
-}
-
-XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* state_in, const void* input, size_t len)
-{
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH64_update_endian(state_in, input, len, XXH_littleEndian);
-    else
-        return XXH64_update_endian(state_in, input, len, XXH_bigEndian);
-}
-
-
-
-FORCE_INLINE_TEMPLATE U64 XXH64_digest_endian (const XXH64_state_t* state, XXH_endianess endian)
-{
-    const BYTE * p = (const BYTE*)state->mem64;
-    const BYTE* const bEnd = (const BYTE*)state->mem64 + state->memsize;
-    U64 h64;
-
-    if (state->total_len >= 32) {
-        U64 const v1 = state->v1;
-        U64 const v2 = state->v2;
-        U64 const v3 = state->v3;
-        U64 const v4 = state->v4;
-
-        h64 = XXH_rotl64(v1, 1) + XXH_rotl64(v2, 7) + XXH_rotl64(v3, 12) + XXH_rotl64(v4, 18);
-        h64 = XXH64_mergeRound(h64, v1);
-        h64 = XXH64_mergeRound(h64, v2);
-        h64 = XXH64_mergeRound(h64, v3);
-        h64 = XXH64_mergeRound(h64, v4);
-    } else {
-        h64  = state->v3 + PRIME64_5;
-    }
-
-    h64 += (U64) state->total_len;
-
-    while (p+8<=bEnd) {
-        U64 const k1 = XXH64_round(0, XXH_readLE64(p, endian));
-        h64 ^= k1;
-        h64  = XXH_rotl64(h64,27) * PRIME64_1 + PRIME64_4;
-        p+=8;
-    }
-
-    if (p+4<=bEnd) {
-        h64 ^= (U64)(XXH_readLE32(p, endian)) * PRIME64_1;
-        h64  = XXH_rotl64(h64, 23) * PRIME64_2 + PRIME64_3;
-        p+=4;
-    }
-
-    while (p<bEnd) {
-        h64 ^= (*p) * PRIME64_5;
-        h64  = XXH_rotl64(h64, 11) * PRIME64_1;
-        p++;
-    }
-
-    h64 ^= h64 >> 33;
-    h64 *= PRIME64_2;
-    h64 ^= h64 >> 29;
-    h64 *= PRIME64_3;
-    h64 ^= h64 >> 32;
-
-    return h64;
-}
-
-
-XXH_PUBLIC_API unsigned long long XXH64_digest (const XXH64_state_t* state_in)
-{
-    XXH_endianess endian_detected = (XXH_endianess)XXH_CPU_LITTLE_ENDIAN;
-
-    if ((endian_detected==XXH_littleEndian) || XXH_FORCE_NATIVE_FORMAT)
-        return XXH64_digest_endian(state_in, XXH_littleEndian);
-    else
-        return XXH64_digest_endian(state_in, XXH_bigEndian);
-}
-
-
-/* **************************
-*  Canonical representation
-****************************/
-
-/*! Default XXH result types are basic unsigned 32 and 64 bits.
-*   The canonical representation follows human-readable write convention, aka big-endian (large digits first).
-*   These functions allow transformation of hash result into and from its canonical format.
-*   This way, hash values can be written into a file or buffer, and remain comparable across different systems and programs.
-*/
-
-XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash)
-{
-    XXH_STATIC_ASSERT(sizeof(XXH32_canonical_t) == sizeof(XXH32_hash_t));
-    if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap32(hash);
-    memcpy(dst, &hash, sizeof(*dst));
-}
-
-XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash)
-{
-    XXH_STATIC_ASSERT(sizeof(XXH64_canonical_t) == sizeof(XXH64_hash_t));
-    if (XXH_CPU_LITTLE_ENDIAN) hash = XXH_swap64(hash);
-    memcpy(dst, &hash, sizeof(*dst));
-}
-
-XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src)
-{
-    return XXH_readBE32(src);
-}
-
-XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src)
-{
-    return XXH_readBE64(src);
-}
diff --git a/vendor/github.com/DataDog/zstd/xxhash.h b/vendor/github.com/DataDog/zstd/xxhash.h
deleted file mode 100644
index 9bad1f5..0000000
--- a/vendor/github.com/DataDog/zstd/xxhash.h
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
-   xxHash - Extremely Fast Hash algorithm
-   Header File
-   Copyright (C) 2012-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - xxHash source repository : https://github.com/Cyan4973/xxHash
-*/
-
-/* Notice extracted from xxHash homepage :
-
-xxHash is an extremely fast Hash algorithm, running at RAM speed limits.
-It also successfully passes all tests from the SMHasher suite.
-
-Comparison (single thread, Windows Seven 32 bits, using SMHasher on a Core 2 Duo @3GHz)
-
-Name            Speed       Q.Score   Author
-xxHash          5.4 GB/s     10
-CrapWow         3.2 GB/s      2       Andrew
-MumurHash 3a    2.7 GB/s     10       Austin Appleby
-SpookyHash      2.0 GB/s     10       Bob Jenkins
-SBox            1.4 GB/s      9       Bret Mulvey
-Lookup3         1.2 GB/s      9       Bob Jenkins
-SuperFastHash   1.2 GB/s      1       Paul Hsieh
-CityHash64      1.05 GB/s    10       Pike & Alakuijala
-FNV             0.55 GB/s     5       Fowler, Noll, Vo
-CRC32           0.43 GB/s     9
-MD5-32          0.33 GB/s    10       Ronald L. Rivest
-SHA1-32         0.28 GB/s    10
-
-Q.Score is a measure of quality of the hash function.
-It depends on successfully passing SMHasher test set.
-10 is a perfect score.
-
-A 64-bits version, named XXH64, is available since r35.
-It offers much better speed, but for 64-bits applications only.
-Name     Speed on 64 bits    Speed on 32 bits
-XXH64       13.8 GB/s            1.9 GB/s
-XXH32        6.8 GB/s            6.0 GB/s
-*/
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef XXHASH_H_5627135585666179
-#define XXHASH_H_5627135585666179 1
-
-
-/* ****************************
-*  Definitions
-******************************/
-#include <stddef.h>   /* size_t */
-typedef enum { XXH_OK=0, XXH_ERROR } XXH_errorcode;
-
-
-/* ****************************
-*  API modifier
-******************************/
-/** XXH_PRIVATE_API
-*   This is useful if you want to include xxhash functions in `static` mode
-*   in order to inline them, and remove their symbol from the public list.
-*   Methodology :
-*     #define XXH_PRIVATE_API
-*     #include "xxhash.h"
-*   `xxhash.c` is automatically included.
-*   It's not useful to compile and link it as a separate module anymore.
-*/
-#ifdef XXH_PRIVATE_API
-#  ifndef XXH_STATIC_LINKING_ONLY
-#    define XXH_STATIC_LINKING_ONLY
-#  endif
-#  if defined(__GNUC__)
-#    define XXH_PUBLIC_API static __inline __attribute__((unused))
-#  elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#    define XXH_PUBLIC_API static inline
-#  elif defined(_MSC_VER)
-#    define XXH_PUBLIC_API static __inline
-#  else
-#    define XXH_PUBLIC_API static   /* this version may generate warnings for unused static functions; disable the relevant warning */
-#  endif
-#else
-#  define XXH_PUBLIC_API   /* do nothing */
-#endif /* XXH_PRIVATE_API */
-
-/*!XXH_NAMESPACE, aka Namespace Emulation :
-
-If you want to include _and expose_ xxHash functions from within your own library,
-but also want to avoid symbol collisions with another library which also includes xxHash,
-
-you can use XXH_NAMESPACE, to automatically prefix any public symbol from xxhash library
-with the value of XXH_NAMESPACE (so avoid to keep it NULL and avoid numeric values).
-
-Note that no change is required within the calling program as long as it includes `xxhash.h` :
-regular symbol name will be automatically translated by this header.
-*/
-#ifdef XXH_NAMESPACE
-#  define XXH_CAT(A,B) A##B
-#  define XXH_NAME2(A,B) XXH_CAT(A,B)
-#  define XXH32 XXH_NAME2(XXH_NAMESPACE, XXH32)
-#  define XXH64 XXH_NAME2(XXH_NAMESPACE, XXH64)
-#  define XXH_versionNumber XXH_NAME2(XXH_NAMESPACE, XXH_versionNumber)
-#  define XXH32_createState XXH_NAME2(XXH_NAMESPACE, XXH32_createState)
-#  define XXH64_createState XXH_NAME2(XXH_NAMESPACE, XXH64_createState)
-#  define XXH32_freeState XXH_NAME2(XXH_NAMESPACE, XXH32_freeState)
-#  define XXH64_freeState XXH_NAME2(XXH_NAMESPACE, XXH64_freeState)
-#  define XXH32_reset XXH_NAME2(XXH_NAMESPACE, XXH32_reset)
-#  define XXH64_reset XXH_NAME2(XXH_NAMESPACE, XXH64_reset)
-#  define XXH32_update XXH_NAME2(XXH_NAMESPACE, XXH32_update)
-#  define XXH64_update XXH_NAME2(XXH_NAMESPACE, XXH64_update)
-#  define XXH32_digest XXH_NAME2(XXH_NAMESPACE, XXH32_digest)
-#  define XXH64_digest XXH_NAME2(XXH_NAMESPACE, XXH64_digest)
-#  define XXH32_copyState XXH_NAME2(XXH_NAMESPACE, XXH32_copyState)
-#  define XXH64_copyState XXH_NAME2(XXH_NAMESPACE, XXH64_copyState)
-#  define XXH32_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH32_canonicalFromHash)
-#  define XXH64_canonicalFromHash XXH_NAME2(XXH_NAMESPACE, XXH64_canonicalFromHash)
-#  define XXH32_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH32_hashFromCanonical)
-#  define XXH64_hashFromCanonical XXH_NAME2(XXH_NAMESPACE, XXH64_hashFromCanonical)
-#endif
-
-
-/* *************************************
-*  Version
-***************************************/
-#define XXH_VERSION_MAJOR    0
-#define XXH_VERSION_MINOR    6
-#define XXH_VERSION_RELEASE  2
-#define XXH_VERSION_NUMBER  (XXH_VERSION_MAJOR *100*100 + XXH_VERSION_MINOR *100 + XXH_VERSION_RELEASE)
-XXH_PUBLIC_API unsigned XXH_versionNumber (void);
-
-
-/* ****************************
-*  Simple Hash Functions
-******************************/
-typedef unsigned int       XXH32_hash_t;
-typedef unsigned long long XXH64_hash_t;
-
-XXH_PUBLIC_API XXH32_hash_t XXH32 (const void* input, size_t length, unsigned int seed);
-XXH_PUBLIC_API XXH64_hash_t XXH64 (const void* input, size_t length, unsigned long long seed);
-
-/*!
-XXH32() :
-    Calculate the 32-bits hash of sequence "length" bytes stored at memory address "input".
-    The memory between input & input+length must be valid (allocated and read-accessible).
-    "seed" can be used to alter the result predictably.
-    Speed on Core 2 Duo @ 3 GHz (single thread, SMHasher benchmark) : 5.4 GB/s
-XXH64() :
-    Calculate the 64-bits hash of sequence of length "len" stored at memory address "input".
-    "seed" can be used to alter the result predictably.
-    This function runs 2x faster on 64-bits systems, but slower on 32-bits systems (see benchmark).
-*/
-
-
-/* ****************************
-*  Streaming Hash Functions
-******************************/
-typedef struct XXH32_state_s XXH32_state_t;   /* incomplete type */
-typedef struct XXH64_state_s XXH64_state_t;   /* incomplete type */
-
-/*! State allocation, compatible with dynamic libraries */
-
-XXH_PUBLIC_API XXH32_state_t* XXH32_createState(void);
-XXH_PUBLIC_API XXH_errorcode  XXH32_freeState(XXH32_state_t* statePtr);
-
-XXH_PUBLIC_API XXH64_state_t* XXH64_createState(void);
-XXH_PUBLIC_API XXH_errorcode  XXH64_freeState(XXH64_state_t* statePtr);
-
-
-/* hash streaming */
-
-XXH_PUBLIC_API XXH_errorcode XXH32_reset  (XXH32_state_t* statePtr, unsigned int seed);
-XXH_PUBLIC_API XXH_errorcode XXH32_update (XXH32_state_t* statePtr, const void* input, size_t length);
-XXH_PUBLIC_API XXH32_hash_t  XXH32_digest (const XXH32_state_t* statePtr);
-
-XXH_PUBLIC_API XXH_errorcode XXH64_reset  (XXH64_state_t* statePtr, unsigned long long seed);
-XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH64_state_t* statePtr, const void* input, size_t length);
-XXH_PUBLIC_API XXH64_hash_t  XXH64_digest (const XXH64_state_t* statePtr);
-
-/*
-These functions generate the xxHash of an input provided in multiple segments.
-Note that, for small input, they are slower than single-call functions, due to state management.
-For small input, prefer `XXH32()` and `XXH64()` .
-
-XXH state must first be allocated, using XXH*_createState() .
-
-Start a new hash by initializing state with a seed, using XXH*_reset().
-
-Then, feed the hash state by calling XXH*_update() as many times as necessary.
-Obviously, input must be allocated and read accessible.
-The function returns an error code, with 0 meaning OK, and any other value meaning there is an error.
-
-Finally, a hash value can be produced anytime, by using XXH*_digest().
-This function returns the nn-bits hash as an int or long long.
-
-It's still possible to continue inserting input into the hash state after a digest,
-and generate some new hashes later on, by calling again XXH*_digest().
-
-When done, free XXH state space if it was allocated dynamically.
-*/
-
-
-/* **************************
-*  Utils
-****************************/
-#if !(defined(__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L))   /* ! C99 */
-#  define restrict   /* disable restrict */
-#endif
-
-XXH_PUBLIC_API void XXH32_copyState(XXH32_state_t* restrict dst_state, const XXH32_state_t* restrict src_state);
-XXH_PUBLIC_API void XXH64_copyState(XXH64_state_t* restrict dst_state, const XXH64_state_t* restrict src_state);
-
-
-/* **************************
-*  Canonical representation
-****************************/
-/* Default result type for XXH functions are primitive unsigned 32 and 64 bits.
-*  The canonical representation uses human-readable write convention, aka big-endian (large digits first).
-*  These functions allow transformation of hash result into and from its canonical format.
-*  This way, hash values can be written into a file / memory, and remain comparable on different systems and programs.
-*/
-typedef struct { unsigned char digest[4]; } XXH32_canonical_t;
-typedef struct { unsigned char digest[8]; } XXH64_canonical_t;
-
-XXH_PUBLIC_API void XXH32_canonicalFromHash(XXH32_canonical_t* dst, XXH32_hash_t hash);
-XXH_PUBLIC_API void XXH64_canonicalFromHash(XXH64_canonical_t* dst, XXH64_hash_t hash);
-
-XXH_PUBLIC_API XXH32_hash_t XXH32_hashFromCanonical(const XXH32_canonical_t* src);
-XXH_PUBLIC_API XXH64_hash_t XXH64_hashFromCanonical(const XXH64_canonical_t* src);
-
-#endif /* XXHASH_H_5627135585666179 */
-
-
-
-/* ================================================================================================
-   This section contains definitions which are not guaranteed to remain stable.
-   They may change in future versions, becoming incompatible with a different version of the library.
-   They shall only be used with static linking.
-   Never use these definitions in association with dynamic linking !
-=================================================================================================== */
-#if defined(XXH_STATIC_LINKING_ONLY) && !defined(XXH_STATIC_H_3543687687345)
-#define XXH_STATIC_H_3543687687345
-
-/* These definitions are only meant to allow allocation of XXH state
-   statically, on stack, or in a struct for example.
-   Do not use members directly. */
-
-   struct XXH32_state_s {
-       unsigned total_len_32;
-       unsigned large_len;
-       unsigned v1;
-       unsigned v2;
-       unsigned v3;
-       unsigned v4;
-       unsigned mem32[4];   /* buffer defined as U32 for alignment */
-       unsigned memsize;
-       unsigned reserved;   /* never read nor write, will be removed in a future version */
-   };   /* typedef'd to XXH32_state_t */
-
-   struct XXH64_state_s {
-       unsigned long long total_len;
-       unsigned long long v1;
-       unsigned long long v2;
-       unsigned long long v3;
-       unsigned long long v4;
-       unsigned long long mem64[4];   /* buffer defined as U64 for alignment */
-       unsigned memsize;
-       unsigned reserved[2];          /* never read nor write, will be removed in a future version */
-   };   /* typedef'd to XXH64_state_t */
-
-
-#  ifdef XXH_PRIVATE_API
-#    include "xxhash.c"   /* include xxhash functions as `static`, for inlining */
-#  endif
-
-#endif /* XXH_STATIC_LINKING_ONLY && XXH_STATIC_H_3543687687345 */
-
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/zbuff.h b/vendor/github.com/DataDog/zstd/zbuff.h
deleted file mode 100644
index a93115d..0000000
--- a/vendor/github.com/DataDog/zstd/zbuff.h
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* ***************************************************************
-*  NOTES/WARNINGS
-******************************************************************/
-/* The streaming API defined here is deprecated.
- * Consider migrating towards ZSTD_compressStream() API in `zstd.h`
- * See 'lib/README.md'.
- *****************************************************************/
-
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef ZSTD_BUFFERED_H_23987
-#define ZSTD_BUFFERED_H_23987
-
-/* *************************************
-*  Dependencies
-***************************************/
-#include <stddef.h>      /* size_t */
-#include "zstd.h"        /* ZSTD_CStream, ZSTD_DStream, ZSTDLIB_API */
-
-
-/* ***************************************************************
-*  Compiler specifics
-*****************************************************************/
-/* Deprecation warnings */
-/* Should these warnings be a problem,
-   it is generally possible to disable them,
-   typically with -Wno-deprecated-declarations for gcc
-   or _CRT_SECURE_NO_WARNINGS in Visual.
-   Otherwise, it's also possible to define ZBUFF_DISABLE_DEPRECATE_WARNINGS */
-#ifdef ZBUFF_DISABLE_DEPRECATE_WARNINGS
-#  define ZBUFF_DEPRECATED(message) ZSTDLIB_API  /* disable deprecation warnings */
-#else
-#  if defined (__cplusplus) && (__cplusplus >= 201402) /* C++14 or greater */
-#    define ZBUFF_DEPRECATED(message) [[deprecated(message)]] ZSTDLIB_API
-#  elif (defined(__GNUC__) && (__GNUC__ >= 5)) || defined(__clang__)
-#    define ZBUFF_DEPRECATED(message) ZSTDLIB_API __attribute__((deprecated(message)))
-#  elif defined(__GNUC__) && (__GNUC__ >= 3)
-#    define ZBUFF_DEPRECATED(message) ZSTDLIB_API __attribute__((deprecated))
-#  elif defined(_MSC_VER)
-#    define ZBUFF_DEPRECATED(message) ZSTDLIB_API __declspec(deprecated(message))
-#  else
-#    pragma message("WARNING: You need to implement ZBUFF_DEPRECATED for this compiler")
-#    define ZBUFF_DEPRECATED(message) ZSTDLIB_API
-#  endif
-#endif /* ZBUFF_DISABLE_DEPRECATE_WARNINGS */
-
-
-/* *************************************
-*  Streaming functions
-***************************************/
-/* This is the easier "buffered" streaming API,
-*  using an internal buffer to lift all restrictions on user-provided buffers
-*  which can be any size, any place, for both input and output.
-*  ZBUFF and ZSTD are 100% interoperable,
-*  frames created by one can be decoded by the other one */
-
-typedef ZSTD_CStream ZBUFF_CCtx;
-ZBUFF_DEPRECATED("use ZSTD_createCStream") ZBUFF_CCtx* ZBUFF_createCCtx(void);
-ZBUFF_DEPRECATED("use ZSTD_freeCStream")   size_t      ZBUFF_freeCCtx(ZBUFF_CCtx* cctx);
-
-ZBUFF_DEPRECATED("use ZSTD_initCStream")           size_t ZBUFF_compressInit(ZBUFF_CCtx* cctx, int compressionLevel);
-ZBUFF_DEPRECATED("use ZSTD_initCStream_usingDict") size_t ZBUFF_compressInitDictionary(ZBUFF_CCtx* cctx, const void* dict, size_t dictSize, int compressionLevel);
-
-ZBUFF_DEPRECATED("use ZSTD_compressStream") size_t ZBUFF_compressContinue(ZBUFF_CCtx* cctx, void* dst, size_t* dstCapacityPtr, const void* src, size_t* srcSizePtr);
-ZBUFF_DEPRECATED("use ZSTD_flushStream")    size_t ZBUFF_compressFlush(ZBUFF_CCtx* cctx, void* dst, size_t* dstCapacityPtr);
-ZBUFF_DEPRECATED("use ZSTD_endStream")      size_t ZBUFF_compressEnd(ZBUFF_CCtx* cctx, void* dst, size_t* dstCapacityPtr);
-
-/*-*************************************************
-*  Streaming compression - howto
-*
-*  A ZBUFF_CCtx object is required to track streaming operation.
-*  Use ZBUFF_createCCtx() and ZBUFF_freeCCtx() to create/release resources.
-*  ZBUFF_CCtx objects can be reused multiple times.
-*
-*  Start by initializing ZBUF_CCtx.
-*  Use ZBUFF_compressInit() to start a new compression operation.
-*  Use ZBUFF_compressInitDictionary() for a compression which requires a dictionary.
-*
-*  Use ZBUFF_compressContinue() repetitively to consume input stream.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written within *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present again remaining data.
-*  The content of `dst` will be overwritten (up to *dstCapacityPtr) at each call, so save its content if it matters or change @dst .
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's just a hint, to improve latency)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  At any moment, it's possible to flush whatever data remains within buffer, using ZBUFF_compressFlush().
-*  The nb of bytes written into `dst` will be reported into *dstCapacityPtr.
-*  Note that the function cannot output more than *dstCapacityPtr,
-*  therefore, some content might still be left into internal buffer if *dstCapacityPtr is too small.
-*  @return : nb of bytes still present into internal buffer (0 if it's empty)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  ZBUFF_compressEnd() instructs to finish a frame.
-*  It will perform a flush and write frame epilogue.
-*  The epilogue is required for decoders to consider a frame completed.
-*  Similar to ZBUFF_compressFlush(), it may not be able to output the entire internal buffer content if *dstCapacityPtr is too small.
-*  In which case, call again ZBUFF_compressFlush() to complete the flush.
-*  @return : nb of bytes still present into internal buffer (0 if it's empty)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  Hint : _recommended buffer_ sizes (not compulsory) : ZBUFF_recommendedCInSize() / ZBUFF_recommendedCOutSize()
-*  input : ZBUFF_recommendedCInSize==128 KB block size is the internal unit, use this value to reduce intermediate stages (better latency)
-*  output : ZBUFF_recommendedCOutSize==ZSTD_compressBound(128 KB) + 3 + 3 : ensures it's always possible to write/flush/end a full block. Skip some buffering.
-*  By using both, it ensures that input will be entirely consumed, and output will always contain the result, reducing intermediate buffering.
-* **************************************************/
-
-
-typedef ZSTD_DStream ZBUFF_DCtx;
-ZBUFF_DEPRECATED("use ZSTD_createDStream") ZBUFF_DCtx* ZBUFF_createDCtx(void);
-ZBUFF_DEPRECATED("use ZSTD_freeDStream")   size_t      ZBUFF_freeDCtx(ZBUFF_DCtx* dctx);
-
-ZBUFF_DEPRECATED("use ZSTD_initDStream")           size_t ZBUFF_decompressInit(ZBUFF_DCtx* dctx);
-ZBUFF_DEPRECATED("use ZSTD_initDStream_usingDict") size_t ZBUFF_decompressInitDictionary(ZBUFF_DCtx* dctx, const void* dict, size_t dictSize);
-
-ZBUFF_DEPRECATED("use ZSTD_decompressStream") size_t ZBUFF_decompressContinue(ZBUFF_DCtx* dctx,
-                                            void* dst, size_t* dstCapacityPtr,
-                                      const void* src, size_t* srcSizePtr);
-
-/*-***************************************************************************
-*  Streaming decompression howto
-*
-*  A ZBUFF_DCtx object is required to track streaming operations.
-*  Use ZBUFF_createDCtx() and ZBUFF_freeDCtx() to create/release resources.
-*  Use ZBUFF_decompressInit() to start a new decompression operation,
-*   or ZBUFF_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFF_DCtx objects can be re-init multiple times.
-*
-*  Use ZBUFF_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of `dst` will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters, or change `dst`.
-*  @return : 0 when a frame is completely decoded and fully flushed,
-*            1 when there is still some data left within internal buffer to flush,
-*            >1 when more data is expected, with value being a suggested next input size (it's just a hint, which helps latency),
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFF_recommendedDInSize() and ZBUFF_recommendedDOutSize()
-*  output : ZBUFF_recommendedDOutSize== 128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFF_recommendedDInSize == 128KB + 3;
-*           just follow indications from ZBUFF_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-ZBUFF_DEPRECATED("use ZSTD_isError")      unsigned ZBUFF_isError(size_t errorCode);
-ZBUFF_DEPRECATED("use ZSTD_getErrorName") const char* ZBUFF_getErrorName(size_t errorCode);
-
-/** Functions below provide recommended buffer sizes for Compression or Decompression operations.
-*   These sizes are just hints, they tend to offer better latency */
-ZBUFF_DEPRECATED("use ZSTD_CStreamInSize")  size_t ZBUFF_recommendedCInSize(void);
-ZBUFF_DEPRECATED("use ZSTD_CStreamOutSize") size_t ZBUFF_recommendedCOutSize(void);
-ZBUFF_DEPRECATED("use ZSTD_DStreamInSize")  size_t ZBUFF_recommendedDInSize(void);
-ZBUFF_DEPRECATED("use ZSTD_DStreamOutSize") size_t ZBUFF_recommendedDOutSize(void);
-
-#endif  /* ZSTD_BUFFERED_H_23987 */
-
-
-#ifdef ZBUFF_STATIC_LINKING_ONLY
-#ifndef ZBUFF_STATIC_H_30298098432
-#define ZBUFF_STATIC_H_30298098432
-
-/* ====================================================================================
- * The definitions in this section are considered experimental.
- * They should never be used in association with a dynamic library, as they may change in the future.
- * They are provided for advanced usages.
- * Use them only in association with static linking.
- * ==================================================================================== */
-
-/*--- Dependency ---*/
-#define ZSTD_STATIC_LINKING_ONLY   /* ZSTD_parameters, ZSTD_customMem */
-#include "zstd.h"
-
-
-/*--- Custom memory allocator ---*/
-/*! ZBUFF_createCCtx_advanced() :
- *  Create a ZBUFF compression context using external alloc and free functions */
-ZBUFF_DEPRECATED("use ZSTD_createCStream_advanced") ZBUFF_CCtx* ZBUFF_createCCtx_advanced(ZSTD_customMem customMem);
-
-/*! ZBUFF_createDCtx_advanced() :
- *  Create a ZBUFF decompression context using external alloc and free functions */
-ZBUFF_DEPRECATED("use ZSTD_createDStream_advanced") ZBUFF_DCtx* ZBUFF_createDCtx_advanced(ZSTD_customMem customMem);
-
-
-/*--- Advanced Streaming Initialization ---*/
-ZBUFF_DEPRECATED("use ZSTD_initDStream_usingDict") size_t ZBUFF_compressInit_advanced(ZBUFF_CCtx* zbc,
-                                               const void* dict, size_t dictSize,
-                                               ZSTD_parameters params, unsigned long long pledgedSrcSize);
-
-
-#endif    /* ZBUFF_STATIC_H_30298098432 */
-#endif    /* ZBUFF_STATIC_LINKING_ONLY */
-
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/zbuff_common.c b/vendor/github.com/DataDog/zstd/zbuff_common.c
deleted file mode 100644
index 661b9b0..0000000
--- a/vendor/github.com/DataDog/zstd/zbuff_common.c
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include "error_private.h"
-#include "zbuff.h"
-
-/*-****************************************
-*  ZBUFF Error Management  (deprecated)
-******************************************/
-
-/*! ZBUFF_isError() :
-*   tells if a return value is an error code */
-unsigned ZBUFF_isError(size_t errorCode) { return ERR_isError(errorCode); }
-/*! ZBUFF_getErrorName() :
-*   provides error code string from function result (useful for debugging) */
-const char* ZBUFF_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
diff --git a/vendor/github.com/DataDog/zstd/zbuff_compress.c b/vendor/github.com/DataDog/zstd/zbuff_compress.c
deleted file mode 100644
index f39c60d..0000000
--- a/vendor/github.com/DataDog/zstd/zbuff_compress.c
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-
-/* *************************************
-*  Dependencies
-***************************************/
-#define ZBUFF_STATIC_LINKING_ONLY
-#include "zbuff.h"
-
-
-/*-***********************************************************
-*  Streaming compression
-*
-*  A ZBUFF_CCtx object is required to track streaming operation.
-*  Use ZBUFF_createCCtx() and ZBUFF_freeCCtx() to create/release resources.
-*  Use ZBUFF_compressInit() to start a new compression operation.
-*  ZBUFF_CCtx objects can be reused multiple times.
-*
-*  Use ZBUFF_compressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to call again the function with remaining input.
-*  The content of dst will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters or change dst .
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to improve latency)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  ZBUFF_compressFlush() can be used to instruct ZBUFF to compress and output whatever remains within its buffer.
-*  Note that it will not output more than *dstCapacityPtr.
-*  Therefore, some content might still be left into its internal buffer if dst buffer is too small.
-*  @return : nb of bytes still present into internal buffer (0 if it's empty)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  ZBUFF_compressEnd() instructs to finish a frame.
-*  It will perform a flush and write frame epilogue.
-*  Similar to ZBUFF_compressFlush(), it may not be able to output the entire internal buffer content if *dstCapacityPtr is too small.
-*  @return : nb of bytes still present into internal buffer (0 if it's empty)
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory)
-*  input : ZSTD_BLOCKSIZE_MAX (128 KB), internal unit size, it improves latency to use this value.
-*  output : ZSTD_compressBound(ZSTD_BLOCKSIZE_MAX) + ZSTD_blockHeaderSize + ZBUFF_endFrameSize : ensures it's always possible to write/flush/end a full block at best speed.
-* ***********************************************************/
-
-ZBUFF_CCtx* ZBUFF_createCCtx(void)
-{
-    return ZSTD_createCStream();
-}
-
-ZBUFF_CCtx* ZBUFF_createCCtx_advanced(ZSTD_customMem customMem)
-{
-    return ZSTD_createCStream_advanced(customMem);
-}
-
-size_t ZBUFF_freeCCtx(ZBUFF_CCtx* zbc)
-{
-    return ZSTD_freeCStream(zbc);
-}
-
-
-/* ======   Initialization   ====== */
-
-size_t ZBUFF_compressInit_advanced(ZBUFF_CCtx* zbc,
-                                   const void* dict, size_t dictSize,
-                                   ZSTD_parameters params, unsigned long long pledgedSrcSize)
-{
-    if (pledgedSrcSize==0) pledgedSrcSize = ZSTD_CONTENTSIZE_UNKNOWN;  /* preserve "0 == unknown" behavior */
-    return ZSTD_initCStream_advanced(zbc, dict, dictSize, params, pledgedSrcSize);
-}
-
-
-size_t ZBUFF_compressInitDictionary(ZBUFF_CCtx* zbc, const void* dict, size_t dictSize, int compressionLevel)
-{
-    return ZSTD_initCStream_usingDict(zbc, dict, dictSize, compressionLevel);
-}
-
-size_t ZBUFF_compressInit(ZBUFF_CCtx* zbc, int compressionLevel)
-{
-    return ZSTD_initCStream(zbc, compressionLevel);
-}
-
-/* ======   Compression   ====== */
-
-
-size_t ZBUFF_compressContinue(ZBUFF_CCtx* zbc,
-                              void* dst, size_t* dstCapacityPtr,
-                        const void* src, size_t* srcSizePtr)
-{
-    size_t result;
-    ZSTD_outBuffer outBuff;
-    ZSTD_inBuffer inBuff;
-    outBuff.dst = dst;
-    outBuff.pos = 0;
-    outBuff.size = *dstCapacityPtr;
-    inBuff.src = src;
-    inBuff.pos = 0;
-    inBuff.size = *srcSizePtr;
-    result = ZSTD_compressStream(zbc, &outBuff, &inBuff);
-    *dstCapacityPtr = outBuff.pos;
-    *srcSizePtr = inBuff.pos;
-    return result;
-}
-
-
-
-/* ======   Finalize   ====== */
-
-size_t ZBUFF_compressFlush(ZBUFF_CCtx* zbc, void* dst, size_t* dstCapacityPtr)
-{
-    size_t result;
-    ZSTD_outBuffer outBuff;
-    outBuff.dst = dst;
-    outBuff.pos = 0;
-    outBuff.size = *dstCapacityPtr;
-    result = ZSTD_flushStream(zbc, &outBuff);
-    *dstCapacityPtr = outBuff.pos;
-    return result;
-}
-
-
-size_t ZBUFF_compressEnd(ZBUFF_CCtx* zbc, void* dst, size_t* dstCapacityPtr)
-{
-    size_t result;
-    ZSTD_outBuffer outBuff;
-    outBuff.dst = dst;
-    outBuff.pos = 0;
-    outBuff.size = *dstCapacityPtr;
-    result = ZSTD_endStream(zbc, &outBuff);
-    *dstCapacityPtr = outBuff.pos;
-    return result;
-}
-
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-size_t ZBUFF_recommendedCInSize(void)  { return ZSTD_CStreamInSize(); }
-size_t ZBUFF_recommendedCOutSize(void) { return ZSTD_CStreamOutSize(); }
diff --git a/vendor/github.com/DataDog/zstd/zbuff_decompress.c b/vendor/github.com/DataDog/zstd/zbuff_decompress.c
deleted file mode 100644
index 923c22b..0000000
--- a/vendor/github.com/DataDog/zstd/zbuff_decompress.c
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-
-/* *************************************
-*  Dependencies
-***************************************/
-#define ZBUFF_STATIC_LINKING_ONLY
-#include "zbuff.h"
-
-
-ZBUFF_DCtx* ZBUFF_createDCtx(void)
-{
-    return ZSTD_createDStream();
-}
-
-ZBUFF_DCtx* ZBUFF_createDCtx_advanced(ZSTD_customMem customMem)
-{
-    return ZSTD_createDStream_advanced(customMem);
-}
-
-size_t ZBUFF_freeDCtx(ZBUFF_DCtx* zbd)
-{
-    return ZSTD_freeDStream(zbd);
-}
-
-
-/* *** Initialization *** */
-
-size_t ZBUFF_decompressInitDictionary(ZBUFF_DCtx* zbd, const void* dict, size_t dictSize)
-{
-    return ZSTD_initDStream_usingDict(zbd, dict, dictSize);
-}
-
-size_t ZBUFF_decompressInit(ZBUFF_DCtx* zbd)
-{
-    return ZSTD_initDStream(zbd);
-}
-
-
-/* *** Decompression *** */
-
-size_t ZBUFF_decompressContinue(ZBUFF_DCtx* zbd,
-                                void* dst, size_t* dstCapacityPtr,
-                          const void* src, size_t* srcSizePtr)
-{
-    ZSTD_outBuffer outBuff;
-    ZSTD_inBuffer inBuff;
-    size_t result;
-    outBuff.dst  = dst;
-    outBuff.pos  = 0;
-    outBuff.size = *dstCapacityPtr;
-    inBuff.src  = src;
-    inBuff.pos  = 0;
-    inBuff.size = *srcSizePtr;
-    result = ZSTD_decompressStream(zbd, &outBuff, &inBuff);
-    *dstCapacityPtr = outBuff.pos;
-    *srcSizePtr = inBuff.pos;
-    return result;
-}
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-size_t ZBUFF_recommendedDInSize(void)  { return ZSTD_DStreamInSize(); }
-size_t ZBUFF_recommendedDOutSize(void) { return ZSTD_DStreamOutSize(); }
diff --git a/vendor/github.com/DataDog/zstd/zdict.c b/vendor/github.com/DataDog/zstd/zdict.c
deleted file mode 100644
index ee21ee1..0000000
--- a/vendor/github.com/DataDog/zstd/zdict.c
+++ /dev/null
@@ -1,1111 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/*-**************************************
-*  Tuning parameters
-****************************************/
-#define MINRATIO 4   /* minimum nb of apparition to be selected in dictionary */
-#define ZDICT_MAX_SAMPLES_SIZE (2000U << 20)
-#define ZDICT_MIN_SAMPLES_SIZE (ZDICT_CONTENTSIZE_MIN * MINRATIO)
-
-
-/*-**************************************
-*  Compiler Options
-****************************************/
-/* Unix Large Files support (>4GB) */
-#define _FILE_OFFSET_BITS 64
-#if (defined(__sun__) && (!defined(__LP64__)))   /* Sun Solaris 32-bits requires specific definitions */
-#  define _LARGEFILE_SOURCE
-#elif ! defined(__LP64__)                        /* No point defining Large file for 64 bit */
-#  define _LARGEFILE64_SOURCE
-#endif
-
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <stdlib.h>        /* malloc, free */
-#include <string.h>        /* memset */
-#include <stdio.h>         /* fprintf, fopen, ftello64 */
-#include <time.h>          /* clock */
-
-#include "mem.h"           /* read */
-#include "fse.h"           /* FSE_normalizeCount, FSE_writeNCount */
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"           /* HUF_buildCTable, HUF_writeCTable */
-#include "zstd_internal.h" /* includes zstd.h */
-#include "xxhash.h"        /* XXH64 */
-#include "divsufsort.h"
-#ifndef ZDICT_STATIC_LINKING_ONLY
-#  define ZDICT_STATIC_LINKING_ONLY
-#endif
-#include "zdict.h"
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define DICTLISTSIZE_DEFAULT 10000
-
-#define NOISELENGTH 32
-
-static const int g_compressionLevel_default = 3;
-static const U32 g_selectivity_default = 9;
-
-
-/*-*************************************
-*  Console display
-***************************************/
-#define DISPLAY(...)         { fprintf(stderr, __VA_ARGS__); fflush( stderr ); }
-#define DISPLAYLEVEL(l, ...) if (notificationLevel>=l) { DISPLAY(__VA_ARGS__); }    /* 0 : no display;   1: errors;   2: default;  3: details;  4: debug */
-
-static clock_t ZDICT_clockSpan(clock_t nPrevious) { return clock() - nPrevious; }
-
-static void ZDICT_printHex(const void* ptr, size_t length)
-{
-    const BYTE* const b = (const BYTE*)ptr;
-    size_t u;
-    for (u=0; u<length; u++) {
-        BYTE c = b[u];
-        if (c<32 || c>126) c = '.';   /* non-printable char */
-        DISPLAY("%c", c);
-    }
-}
-
-
-/*-********************************************************
-*  Helper functions
-**********************************************************/
-unsigned ZDICT_isError(size_t errorCode) { return ERR_isError(errorCode); }
-
-const char* ZDICT_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
-
-unsigned ZDICT_getDictID(const void* dictBuffer, size_t dictSize)
-{
-    if (dictSize < 8) return 0;
-    if (MEM_readLE32(dictBuffer) != ZSTD_MAGIC_DICTIONARY) return 0;
-    return MEM_readLE32((const char*)dictBuffer + 4);
-}
-
-
-/*-********************************************************
-*  Dictionary training functions
-**********************************************************/
-static unsigned ZDICT_NbCommonBytes (size_t val)
-{
-    if (MEM_isLittleEndian()) {
-        if (MEM_64bits()) {
-#       if defined(_MSC_VER) && defined(_WIN64)
-            unsigned long r = 0;
-            _BitScanForward64( &r, (U64)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_ctzll((U64)val) >> 3);
-#       else
-            static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2, 0, 3, 1, 3, 1, 4, 2, 7, 0, 2, 3, 6, 1, 5, 3, 5, 1, 3, 4, 4, 2, 5, 6, 7, 7, 0, 1, 2, 3, 3, 4, 6, 2, 6, 5, 5, 3, 4, 5, 6, 7, 1, 2, 4, 6, 4, 4, 5, 7, 2, 6, 5, 7, 6, 7, 7 };
-            return DeBruijnBytePos[((U64)((val & -(long long)val) * 0x0218A392CDABBD3FULL)) >> 58];
-#       endif
-        } else { /* 32 bits */
-#       if defined(_MSC_VER)
-            unsigned long r=0;
-            _BitScanForward( &r, (U32)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_ctz((U32)val) >> 3);
-#       else
-            static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0, 3, 2, 2, 1, 3, 2, 0, 1, 3, 3, 1, 2, 2, 2, 2, 0, 3, 1, 2, 0, 1, 0, 1, 1 };
-            return DeBruijnBytePos[((U32)((val & -(S32)val) * 0x077CB531U)) >> 27];
-#       endif
-        }
-    } else {  /* Big Endian CPU */
-        if (MEM_64bits()) {
-#       if defined(_MSC_VER) && defined(_WIN64)
-            unsigned long r = 0;
-            _BitScanReverse64( &r, val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_clzll(val) >> 3);
-#       else
-            unsigned r;
-            const unsigned n32 = sizeof(size_t)*4;   /* calculate this way due to compiler complaining in 32-bits mode */
-            if (!(val>>n32)) { r=4; } else { r=0; val>>=n32; }
-            if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; }
-            r += (!val);
-            return r;
-#       endif
-        } else { /* 32 bits */
-#       if defined(_MSC_VER)
-            unsigned long r = 0;
-            _BitScanReverse( &r, (unsigned long)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_clz((U32)val) >> 3);
-#       else
-            unsigned r;
-            if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; }
-            r += (!val);
-            return r;
-#       endif
-    }   }
-}
-
-
-/*! ZDICT_count() :
-    Count the nb of common bytes between 2 pointers.
-    Note : this function presumes end of buffer followed by noisy guard band.
-*/
-static size_t ZDICT_count(const void* pIn, const void* pMatch)
-{
-    const char* const pStart = (const char*)pIn;
-    for (;;) {
-        size_t const diff = MEM_readST(pMatch) ^ MEM_readST(pIn);
-        if (!diff) {
-            pIn = (const char*)pIn+sizeof(size_t);
-            pMatch = (const char*)pMatch+sizeof(size_t);
-            continue;
-        }
-        pIn = (const char*)pIn+ZDICT_NbCommonBytes(diff);
-        return (size_t)((const char*)pIn - pStart);
-    }
-}
-
-
-typedef struct {
-    U32 pos;
-    U32 length;
-    U32 savings;
-} dictItem;
-
-static void ZDICT_initDictItem(dictItem* d)
-{
-    d->pos = 1;
-    d->length = 0;
-    d->savings = (U32)(-1);
-}
-
-
-#define LLIMIT 64          /* heuristic determined experimentally */
-#define MINMATCHLENGTH 7   /* heuristic determined experimentally */
-static dictItem ZDICT_analyzePos(
-                       BYTE* doneMarks,
-                       const int* suffix, U32 start,
-                       const void* buffer, U32 minRatio, U32 notificationLevel)
-{
-    U32 lengthList[LLIMIT] = {0};
-    U32 cumulLength[LLIMIT] = {0};
-    U32 savings[LLIMIT] = {0};
-    const BYTE* b = (const BYTE*)buffer;
-    size_t maxLength = LLIMIT;
-    size_t pos = suffix[start];
-    U32 end = start;
-    dictItem solution;
-
-    /* init */
-    memset(&solution, 0, sizeof(solution));
-    doneMarks[pos] = 1;
-
-    /* trivial repetition cases */
-    if ( (MEM_read16(b+pos+0) == MEM_read16(b+pos+2))
-       ||(MEM_read16(b+pos+1) == MEM_read16(b+pos+3))
-       ||(MEM_read16(b+pos+2) == MEM_read16(b+pos+4)) ) {
-        /* skip and mark segment */
-        U16 const pattern16 = MEM_read16(b+pos+4);
-        U32 u, patternEnd = 6;
-        while (MEM_read16(b+pos+patternEnd) == pattern16) patternEnd+=2 ;
-        if (b[pos+patternEnd] == b[pos+patternEnd-1]) patternEnd++;
-        for (u=1; u<patternEnd; u++)
-            doneMarks[pos+u] = 1;
-        return solution;
-    }
-
-    /* look forward */
-    {   size_t length;
-        do {
-            end++;
-            length = ZDICT_count(b + pos, b + suffix[end]);
-        } while (length >= MINMATCHLENGTH);
-    }
-
-    /* look backward */
-    {   size_t length;
-        do {
-            length = ZDICT_count(b + pos, b + *(suffix+start-1));
-            if (length >=MINMATCHLENGTH) start--;
-        } while(length >= MINMATCHLENGTH);
-    }
-
-    /* exit if not found a minimum nb of repetitions */
-    if (end-start < minRatio) {
-        U32 idx;
-        for(idx=start; idx<end; idx++)
-            doneMarks[suffix[idx]] = 1;
-        return solution;
-    }
-
-    {   int i;
-        U32 mml;
-        U32 refinedStart = start;
-        U32 refinedEnd = end;
-
-        DISPLAYLEVEL(4, "\n");
-        DISPLAYLEVEL(4, "found %3u matches of length >= %i at pos %7u  ", (unsigned)(end-start), MINMATCHLENGTH, (unsigned)pos);
-        DISPLAYLEVEL(4, "\n");
-
-        for (mml = MINMATCHLENGTH ; ; mml++) {
-            BYTE currentChar = 0;
-            U32 currentCount = 0;
-            U32 currentID = refinedStart;
-            U32 id;
-            U32 selectedCount = 0;
-            U32 selectedID = currentID;
-            for (id =refinedStart; id < refinedEnd; id++) {
-                if (b[suffix[id] + mml] != currentChar) {
-                    if (currentCount > selectedCount) {
-                        selectedCount = currentCount;
-                        selectedID = currentID;
-                    }
-                    currentID = id;
-                    currentChar = b[ suffix[id] + mml];
-                    currentCount = 0;
-                }
-                currentCount ++;
-            }
-            if (currentCount > selectedCount) {  /* for last */
-                selectedCount = currentCount;
-                selectedID = currentID;
-            }
-
-            if (selectedCount < minRatio)
-                break;
-            refinedStart = selectedID;
-            refinedEnd = refinedStart + selectedCount;
-        }
-
-        /* evaluate gain based on new dict */
-        start = refinedStart;
-        pos = suffix[refinedStart];
-        end = start;
-        memset(lengthList, 0, sizeof(lengthList));
-
-        /* look forward */
-        {   size_t length;
-            do {
-                end++;
-                length = ZDICT_count(b + pos, b + suffix[end]);
-                if (length >= LLIMIT) length = LLIMIT-1;
-                lengthList[length]++;
-            } while (length >=MINMATCHLENGTH);
-        }
-
-        /* look backward */
-        {   size_t length = MINMATCHLENGTH;
-            while ((length >= MINMATCHLENGTH) & (start > 0)) {
-                length = ZDICT_count(b + pos, b + suffix[start - 1]);
-                if (length >= LLIMIT) length = LLIMIT - 1;
-                lengthList[length]++;
-                if (length >= MINMATCHLENGTH) start--;
-            }
-        }
-
-        /* largest useful length */
-        memset(cumulLength, 0, sizeof(cumulLength));
-        cumulLength[maxLength-1] = lengthList[maxLength-1];
-        for (i=(int)(maxLength-2); i>=0; i--)
-            cumulLength[i] = cumulLength[i+1] + lengthList[i];
-
-        for (i=LLIMIT-1; i>=MINMATCHLENGTH; i--) if (cumulLength[i]>=minRatio) break;
-        maxLength = i;
-
-        /* reduce maxLength in case of final into repetitive data */
-        {   U32 l = (U32)maxLength;
-            BYTE const c = b[pos + maxLength-1];
-            while (b[pos+l-2]==c) l--;
-            maxLength = l;
-        }
-        if (maxLength < MINMATCHLENGTH) return solution;   /* skip : no long-enough solution */
-
-        /* calculate savings */
-        savings[5] = 0;
-        for (i=MINMATCHLENGTH; i<=(int)maxLength; i++)
-            savings[i] = savings[i-1] + (lengthList[i] * (i-3));
-
-        DISPLAYLEVEL(4, "Selected dict at position %u, of length %u : saves %u (ratio: %.2f)  \n",
-                     (unsigned)pos, (unsigned)maxLength, (unsigned)savings[maxLength], (double)savings[maxLength] / maxLength);
-
-        solution.pos = (U32)pos;
-        solution.length = (U32)maxLength;
-        solution.savings = savings[maxLength];
-
-        /* mark positions done */
-        {   U32 id;
-            for (id=start; id<end; id++) {
-                U32 p, pEnd, length;
-                U32 const testedPos = suffix[id];
-                if (testedPos == pos)
-                    length = solution.length;
-                else {
-                    length = (U32)ZDICT_count(b+pos, b+testedPos);
-                    if (length > solution.length) length = solution.length;
-                }
-                pEnd = (U32)(testedPos + length);
-                for (p=testedPos; p<pEnd; p++)
-                    doneMarks[p] = 1;
-    }   }   }
-
-    return solution;
-}
-
-
-static int isIncluded(const void* in, const void* container, size_t length)
-{
-    const char* const ip = (const char*) in;
-    const char* const into = (const char*) container;
-    size_t u;
-
-    for (u=0; u<length; u++) {  /* works because end of buffer is a noisy guard band */
-        if (ip[u] != into[u]) break;
-    }
-
-    return u==length;
-}
-
-/*! ZDICT_tryMerge() :
-    check if dictItem can be merged, do it if possible
-    @return : id of destination elt, 0 if not merged
-*/
-static U32 ZDICT_tryMerge(dictItem* table, dictItem elt, U32 eltNbToSkip, const void* buffer)
-{
-    const U32 tableSize = table->pos;
-    const U32 eltEnd = elt.pos + elt.length;
-    const char* const buf = (const char*) buffer;
-
-    /* tail overlap */
-    U32 u; for (u=1; u<tableSize; u++) {
-        if (u==eltNbToSkip) continue;
-        if ((table[u].pos > elt.pos) && (table[u].pos <= eltEnd)) {  /* overlap, existing > new */
-            /* append */
-            U32 const addedLength = table[u].pos - elt.pos;
-            table[u].length += addedLength;
-            table[u].pos = elt.pos;
-            table[u].savings += elt.savings * addedLength / elt.length;   /* rough approx */
-            table[u].savings += elt.length / 8;    /* rough approx bonus */
-            elt = table[u];
-            /* sort : improve rank */
-            while ((u>1) && (table[u-1].savings < elt.savings))
-            table[u] = table[u-1], u--;
-            table[u] = elt;
-            return u;
-    }   }
-
-    /* front overlap */
-    for (u=1; u<tableSize; u++) {
-        if (u==eltNbToSkip) continue;
-
-        if ((table[u].pos + table[u].length >= elt.pos) && (table[u].pos < elt.pos)) {  /* overlap, existing < new */
-            /* append */
-            int const addedLength = (int)eltEnd - (table[u].pos + table[u].length);
-            table[u].savings += elt.length / 8;    /* rough approx bonus */
-            if (addedLength > 0) {   /* otherwise, elt fully included into existing */
-                table[u].length += addedLength;
-                table[u].savings += elt.savings * addedLength / elt.length;   /* rough approx */
-            }
-            /* sort : improve rank */
-            elt = table[u];
-            while ((u>1) && (table[u-1].savings < elt.savings))
-                table[u] = table[u-1], u--;
-            table[u] = elt;
-            return u;
-        }
-
-        if (MEM_read64(buf + table[u].pos) == MEM_read64(buf + elt.pos + 1)) {
-            if (isIncluded(buf + table[u].pos, buf + elt.pos + 1, table[u].length)) {
-                size_t const addedLength = MAX( (int)elt.length - (int)table[u].length , 1 );
-                table[u].pos = elt.pos;
-                table[u].savings += (U32)(elt.savings * addedLength / elt.length);
-                table[u].length = MIN(elt.length, table[u].length + 1);
-                return u;
-            }
-        }
-    }
-
-    return 0;
-}
-
-
-static void ZDICT_removeDictItem(dictItem* table, U32 id)
-{
-    /* convention : table[0].pos stores nb of elts */
-    U32 const max = table[0].pos;
-    U32 u;
-    if (!id) return;   /* protection, should never happen */
-    for (u=id; u<max-1; u++)
-        table[u] = table[u+1];
-    table->pos--;
-}
-
-
-static void ZDICT_insertDictItem(dictItem* table, U32 maxSize, dictItem elt, const void* buffer)
-{
-    /* merge if possible */
-    U32 mergeId = ZDICT_tryMerge(table, elt, 0, buffer);
-    if (mergeId) {
-        U32 newMerge = 1;
-        while (newMerge) {
-            newMerge = ZDICT_tryMerge(table, table[mergeId], mergeId, buffer);
-            if (newMerge) ZDICT_removeDictItem(table, mergeId);
-            mergeId = newMerge;
-        }
-        return;
-    }
-
-    /* insert */
-    {   U32 current;
-        U32 nextElt = table->pos;
-        if (nextElt >= maxSize) nextElt = maxSize-1;
-        current = nextElt-1;
-        while (table[current].savings < elt.savings) {
-            table[current+1] = table[current];
-            current--;
-        }
-        table[current+1] = elt;
-        table->pos = nextElt+1;
-    }
-}
-
-
-static U32 ZDICT_dictSize(const dictItem* dictList)
-{
-    U32 u, dictSize = 0;
-    for (u=1; u<dictList[0].pos; u++)
-        dictSize += dictList[u].length;
-    return dictSize;
-}
-
-
-static size_t ZDICT_trainBuffer_legacy(dictItem* dictList, U32 dictListSize,
-                            const void* const buffer, size_t bufferSize,   /* buffer must end with noisy guard band */
-                            const size_t* fileSizes, unsigned nbFiles,
-                            unsigned minRatio, U32 notificationLevel)
-{
-    int* const suffix0 = (int*)malloc((bufferSize+2)*sizeof(*suffix0));
-    int* const suffix = suffix0+1;
-    U32* reverseSuffix = (U32*)malloc((bufferSize)*sizeof(*reverseSuffix));
-    BYTE* doneMarks = (BYTE*)malloc((bufferSize+16)*sizeof(*doneMarks));   /* +16 for overflow security */
-    U32* filePos = (U32*)malloc(nbFiles * sizeof(*filePos));
-    size_t result = 0;
-    clock_t displayClock = 0;
-    clock_t const refreshRate = CLOCKS_PER_SEC * 3 / 10;
-
-#   define DISPLAYUPDATE(l, ...) if (notificationLevel>=l) { \
-            if (ZDICT_clockSpan(displayClock) > refreshRate)  \
-            { displayClock = clock(); DISPLAY(__VA_ARGS__); \
-            if (notificationLevel>=4) fflush(stderr); } }
-
-    /* init */
-    DISPLAYLEVEL(2, "\r%70s\r", "");   /* clean display line */
-    if (!suffix0 || !reverseSuffix || !doneMarks || !filePos) {
-        result = ERROR(memory_allocation);
-        goto _cleanup;
-    }
-    if (minRatio < MINRATIO) minRatio = MINRATIO;
-    memset(doneMarks, 0, bufferSize+16);
-
-    /* limit sample set size (divsufsort limitation)*/
-    if (bufferSize > ZDICT_MAX_SAMPLES_SIZE) DISPLAYLEVEL(3, "sample set too large : reduced to %u MB ...\n", (unsigned)(ZDICT_MAX_SAMPLES_SIZE>>20));
-    while (bufferSize > ZDICT_MAX_SAMPLES_SIZE) bufferSize -= fileSizes[--nbFiles];
-
-    /* sort */
-    DISPLAYLEVEL(2, "sorting %u files of total size %u MB ...\n", nbFiles, (unsigned)(bufferSize>>20));
-    {   int const divSuftSortResult = divsufsort((const unsigned char*)buffer, suffix, (int)bufferSize, 0);
-        if (divSuftSortResult != 0) { result = ERROR(GENERIC); goto _cleanup; }
-    }
-    suffix[bufferSize] = (int)bufferSize;   /* leads into noise */
-    suffix0[0] = (int)bufferSize;           /* leads into noise */
-    /* build reverse suffix sort */
-    {   size_t pos;
-        for (pos=0; pos < bufferSize; pos++)
-            reverseSuffix[suffix[pos]] = (U32)pos;
-        /* note filePos tracks borders between samples.
-           It's not used at this stage, but planned to become useful in a later update */
-        filePos[0] = 0;
-        for (pos=1; pos<nbFiles; pos++)
-            filePos[pos] = (U32)(filePos[pos-1] + fileSizes[pos-1]);
-    }
-
-    DISPLAYLEVEL(2, "finding patterns ... \n");
-    DISPLAYLEVEL(3, "minimum ratio : %u \n", minRatio);
-
-    {   U32 cursor; for (cursor=0; cursor < bufferSize; ) {
-            dictItem solution;
-            if (doneMarks[cursor]) { cursor++; continue; }
-            solution = ZDICT_analyzePos(doneMarks, suffix, reverseSuffix[cursor], buffer, minRatio, notificationLevel);
-            if (solution.length==0) { cursor++; continue; }
-            ZDICT_insertDictItem(dictList, dictListSize, solution, buffer);
-            cursor += solution.length;
-            DISPLAYUPDATE(2, "\r%4.2f %% \r", (double)cursor / bufferSize * 100);
-    }   }
-
-_cleanup:
-    free(suffix0);
-    free(reverseSuffix);
-    free(doneMarks);
-    free(filePos);
-    return result;
-}
-
-
-static void ZDICT_fillNoise(void* buffer, size_t length)
-{
-    unsigned const prime1 = 2654435761U;
-    unsigned const prime2 = 2246822519U;
-    unsigned acc = prime1;
-    size_t p=0;;
-    for (p=0; p<length; p++) {
-        acc *= prime2;
-        ((unsigned char*)buffer)[p] = (unsigned char)(acc >> 21);
-    }
-}
-
-
-typedef struct
-{
-    ZSTD_CDict* dict;    /* dictionary */
-    ZSTD_CCtx* zc;     /* working context */
-    void* workPlace;   /* must be ZSTD_BLOCKSIZE_MAX allocated */
-} EStats_ress_t;
-
-#define MAXREPOFFSET 1024
-
-static void ZDICT_countEStats(EStats_ress_t esr, ZSTD_parameters params,
-                              unsigned* countLit, unsigned* offsetcodeCount, unsigned* matchlengthCount, unsigned* litlengthCount, U32* repOffsets,
-                              const void* src, size_t srcSize,
-                              U32 notificationLevel)
-{
-    size_t const blockSizeMax = MIN (ZSTD_BLOCKSIZE_MAX, 1 << params.cParams.windowLog);
-    size_t cSize;
-
-    if (srcSize > blockSizeMax) srcSize = blockSizeMax;   /* protection vs large samples */
-    {   size_t const errorCode = ZSTD_compressBegin_usingCDict(esr.zc, esr.dict);
-        if (ZSTD_isError(errorCode)) { DISPLAYLEVEL(1, "warning : ZSTD_compressBegin_usingCDict failed \n"); return; }
-
-    }
-    cSize = ZSTD_compressBlock(esr.zc, esr.workPlace, ZSTD_BLOCKSIZE_MAX, src, srcSize);
-    if (ZSTD_isError(cSize)) { DISPLAYLEVEL(3, "warning : could not compress sample size %u \n", (unsigned)srcSize); return; }
-
-    if (cSize) {  /* if == 0; block is not compressible */
-        const seqStore_t* const seqStorePtr = ZSTD_getSeqStore(esr.zc);
-
-        /* literals stats */
-        {   const BYTE* bytePtr;
-            for(bytePtr = seqStorePtr->litStart; bytePtr < seqStorePtr->lit; bytePtr++)
-                countLit[*bytePtr]++;
-        }
-
-        /* seqStats */
-        {   U32 const nbSeq = (U32)(seqStorePtr->sequences - seqStorePtr->sequencesStart);
-            ZSTD_seqToCodes(seqStorePtr);
-
-            {   const BYTE* codePtr = seqStorePtr->ofCode;
-                U32 u;
-                for (u=0; u<nbSeq; u++) offsetcodeCount[codePtr[u]]++;
-            }
-
-            {   const BYTE* codePtr = seqStorePtr->mlCode;
-                U32 u;
-                for (u=0; u<nbSeq; u++) matchlengthCount[codePtr[u]]++;
-            }
-
-            {   const BYTE* codePtr = seqStorePtr->llCode;
-                U32 u;
-                for (u=0; u<nbSeq; u++) litlengthCount[codePtr[u]]++;
-            }
-
-            if (nbSeq >= 2) { /* rep offsets */
-                const seqDef* const seq = seqStorePtr->sequencesStart;
-                U32 offset1 = seq[0].offset - 3;
-                U32 offset2 = seq[1].offset - 3;
-                if (offset1 >= MAXREPOFFSET) offset1 = 0;
-                if (offset2 >= MAXREPOFFSET) offset2 = 0;
-                repOffsets[offset1] += 3;
-                repOffsets[offset2] += 1;
-    }   }   }
-}
-
-static size_t ZDICT_totalSampleSize(const size_t* fileSizes, unsigned nbFiles)
-{
-    size_t total=0;
-    unsigned u;
-    for (u=0; u<nbFiles; u++) total += fileSizes[u];
-    return total;
-}
-
-typedef struct { U32 offset; U32 count; } offsetCount_t;
-
-static void ZDICT_insertSortCount(offsetCount_t table[ZSTD_REP_NUM+1], U32 val, U32 count)
-{
-    U32 u;
-    table[ZSTD_REP_NUM].offset = val;
-    table[ZSTD_REP_NUM].count = count;
-    for (u=ZSTD_REP_NUM; u>0; u--) {
-        offsetCount_t tmp;
-        if (table[u-1].count >= table[u].count) break;
-        tmp = table[u-1];
-        table[u-1] = table[u];
-        table[u] = tmp;
-    }
-}
-
-/* ZDICT_flatLit() :
- * rewrite `countLit` to contain a mostly flat but still compressible distribution of literals.
- * necessary to avoid generating a non-compressible distribution that HUF_writeCTable() cannot encode.
- */
-static void ZDICT_flatLit(unsigned* countLit)
-{
-    int u;
-    for (u=1; u<256; u++) countLit[u] = 2;
-    countLit[0]   = 4;
-    countLit[253] = 1;
-    countLit[254] = 1;
-}
-
-#define OFFCODE_MAX 30  /* only applicable to first block */
-static size_t ZDICT_analyzeEntropy(void*  dstBuffer, size_t maxDstSize,
-                                   unsigned compressionLevel,
-                             const void*  srcBuffer, const size_t* fileSizes, unsigned nbFiles,
-                             const void* dictBuffer, size_t  dictBufferSize,
-                                   unsigned notificationLevel)
-{
-    unsigned countLit[256];
-    HUF_CREATE_STATIC_CTABLE(hufTable, 255);
-    unsigned offcodeCount[OFFCODE_MAX+1];
-    short offcodeNCount[OFFCODE_MAX+1];
-    U32 offcodeMax = ZSTD_highbit32((U32)(dictBufferSize + 128 KB));
-    unsigned matchLengthCount[MaxML+1];
-    short matchLengthNCount[MaxML+1];
-    unsigned litLengthCount[MaxLL+1];
-    short litLengthNCount[MaxLL+1];
-    U32 repOffset[MAXREPOFFSET];
-    offsetCount_t bestRepOffset[ZSTD_REP_NUM+1];
-    EStats_ress_t esr = { NULL, NULL, NULL };
-    ZSTD_parameters params;
-    U32 u, huffLog = 11, Offlog = OffFSELog, mlLog = MLFSELog, llLog = LLFSELog, total;
-    size_t pos = 0, errorCode;
-    size_t eSize = 0;
-    size_t const totalSrcSize = ZDICT_totalSampleSize(fileSizes, nbFiles);
-    size_t const averageSampleSize = totalSrcSize / (nbFiles + !nbFiles);
-    BYTE* dstPtr = (BYTE*)dstBuffer;
-
-    /* init */
-    DEBUGLOG(4, "ZDICT_analyzeEntropy");
-    if (offcodeMax>OFFCODE_MAX) { eSize = ERROR(dictionaryCreation_failed); goto _cleanup; }   /* too large dictionary */
-    for (u=0; u<256; u++) countLit[u] = 1;   /* any character must be described */
-    for (u=0; u<=offcodeMax; u++) offcodeCount[u] = 1;
-    for (u=0; u<=MaxML; u++) matchLengthCount[u] = 1;
-    for (u=0; u<=MaxLL; u++) litLengthCount[u] = 1;
-    memset(repOffset, 0, sizeof(repOffset));
-    repOffset[1] = repOffset[4] = repOffset[8] = 1;
-    memset(bestRepOffset, 0, sizeof(bestRepOffset));
-    if (compressionLevel==0) compressionLevel = g_compressionLevel_default;
-    params = ZSTD_getParams(compressionLevel, averageSampleSize, dictBufferSize);
-
-    esr.dict = ZSTD_createCDict_advanced(dictBuffer, dictBufferSize, ZSTD_dlm_byRef, ZSTD_dct_rawContent, params.cParams, ZSTD_defaultCMem);
-    esr.zc = ZSTD_createCCtx();
-    esr.workPlace = malloc(ZSTD_BLOCKSIZE_MAX);
-    if (!esr.dict || !esr.zc || !esr.workPlace) {
-        eSize = ERROR(memory_allocation);
-        DISPLAYLEVEL(1, "Not enough memory \n");
-        goto _cleanup;
-    }
-
-    /* collect stats on all samples */
-    for (u=0; u<nbFiles; u++) {
-        ZDICT_countEStats(esr, params,
-                          countLit, offcodeCount, matchLengthCount, litLengthCount, repOffset,
-                         (const char*)srcBuffer + pos, fileSizes[u],
-                          notificationLevel);
-        pos += fileSizes[u];
-    }
-
-    /* analyze, build stats, starting with literals */
-    {   size_t maxNbBits = HUF_buildCTable (hufTable, countLit, 255, huffLog);
-        if (HUF_isError(maxNbBits)) {
-            eSize = maxNbBits;
-            DISPLAYLEVEL(1, " HUF_buildCTable error \n");
-            goto _cleanup;
-        }
-        if (maxNbBits==8) {  /* not compressible : will fail on HUF_writeCTable() */
-            DISPLAYLEVEL(2, "warning : pathological dataset : literals are not compressible : samples are noisy or too regular \n");
-            ZDICT_flatLit(countLit);  /* replace distribution by a fake "mostly flat but still compressible" distribution, that HUF_writeCTable() can encode */
-            maxNbBits = HUF_buildCTable (hufTable, countLit, 255, huffLog);
-            assert(maxNbBits==9);
-        }
-        huffLog = (U32)maxNbBits;
-    }
-
-    /* looking for most common first offsets */
-    {   U32 offset;
-        for (offset=1; offset<MAXREPOFFSET; offset++)
-            ZDICT_insertSortCount(bestRepOffset, offset, repOffset[offset]);
-    }
-    /* note : the result of this phase should be used to better appreciate the impact on statistics */
-
-    total=0; for (u=0; u<=offcodeMax; u++) total+=offcodeCount[u];
-    errorCode = FSE_normalizeCount(offcodeNCount, Offlog, offcodeCount, total, offcodeMax);
-    if (FSE_isError(errorCode)) {
-        eSize = errorCode;
-        DISPLAYLEVEL(1, "FSE_normalizeCount error with offcodeCount \n");
-        goto _cleanup;
-    }
-    Offlog = (U32)errorCode;
-
-    total=0; for (u=0; u<=MaxML; u++) total+=matchLengthCount[u];
-    errorCode = FSE_normalizeCount(matchLengthNCount, mlLog, matchLengthCount, total, MaxML);
-    if (FSE_isError(errorCode)) {
-        eSize = errorCode;
-        DISPLAYLEVEL(1, "FSE_normalizeCount error with matchLengthCount \n");
-        goto _cleanup;
-    }
-    mlLog = (U32)errorCode;
-
-    total=0; for (u=0; u<=MaxLL; u++) total+=litLengthCount[u];
-    errorCode = FSE_normalizeCount(litLengthNCount, llLog, litLengthCount, total, MaxLL);
-    if (FSE_isError(errorCode)) {
-        eSize = errorCode;
-        DISPLAYLEVEL(1, "FSE_normalizeCount error with litLengthCount \n");
-        goto _cleanup;
-    }
-    llLog = (U32)errorCode;
-
-    /* write result to buffer */
-    {   size_t const hhSize = HUF_writeCTable(dstPtr, maxDstSize, hufTable, 255, huffLog);
-        if (HUF_isError(hhSize)) {
-            eSize = hhSize;
-            DISPLAYLEVEL(1, "HUF_writeCTable error \n");
-            goto _cleanup;
-        }
-        dstPtr += hhSize;
-        maxDstSize -= hhSize;
-        eSize += hhSize;
-    }
-
-    {   size_t const ohSize = FSE_writeNCount(dstPtr, maxDstSize, offcodeNCount, OFFCODE_MAX, Offlog);
-        if (FSE_isError(ohSize)) {
-            eSize = ohSize;
-            DISPLAYLEVEL(1, "FSE_writeNCount error with offcodeNCount \n");
-            goto _cleanup;
-        }
-        dstPtr += ohSize;
-        maxDstSize -= ohSize;
-        eSize += ohSize;
-    }
-
-    {   size_t const mhSize = FSE_writeNCount(dstPtr, maxDstSize, matchLengthNCount, MaxML, mlLog);
-        if (FSE_isError(mhSize)) {
-            eSize = mhSize;
-            DISPLAYLEVEL(1, "FSE_writeNCount error with matchLengthNCount \n");
-            goto _cleanup;
-        }
-        dstPtr += mhSize;
-        maxDstSize -= mhSize;
-        eSize += mhSize;
-    }
-
-    {   size_t const lhSize = FSE_writeNCount(dstPtr, maxDstSize, litLengthNCount, MaxLL, llLog);
-        if (FSE_isError(lhSize)) {
-            eSize = lhSize;
-            DISPLAYLEVEL(1, "FSE_writeNCount error with litlengthNCount \n");
-            goto _cleanup;
-        }
-        dstPtr += lhSize;
-        maxDstSize -= lhSize;
-        eSize += lhSize;
-    }
-
-    if (maxDstSize<12) {
-        eSize = ERROR(dstSize_tooSmall);
-        DISPLAYLEVEL(1, "not enough space to write RepOffsets \n");
-        goto _cleanup;
-    }
-# if 0
-    MEM_writeLE32(dstPtr+0, bestRepOffset[0].offset);
-    MEM_writeLE32(dstPtr+4, bestRepOffset[1].offset);
-    MEM_writeLE32(dstPtr+8, bestRepOffset[2].offset);
-#else
-    /* at this stage, we don't use the result of "most common first offset",
-       as the impact of statistics is not properly evaluated */
-    MEM_writeLE32(dstPtr+0, repStartValue[0]);
-    MEM_writeLE32(dstPtr+4, repStartValue[1]);
-    MEM_writeLE32(dstPtr+8, repStartValue[2]);
-#endif
-    eSize += 12;
-
-_cleanup:
-    ZSTD_freeCDict(esr.dict);
-    ZSTD_freeCCtx(esr.zc);
-    free(esr.workPlace);
-
-    return eSize;
-}
-
-
-
-size_t ZDICT_finalizeDictionary(void* dictBuffer, size_t dictBufferCapacity,
-                          const void* customDictContent, size_t dictContentSize,
-                          const void* samplesBuffer, const size_t* samplesSizes,
-                          unsigned nbSamples, ZDICT_params_t params)
-{
-    size_t hSize;
-#define HBUFFSIZE 256   /* should prove large enough for all entropy headers */
-    BYTE header[HBUFFSIZE];
-    int const compressionLevel = (params.compressionLevel == 0) ? g_compressionLevel_default : params.compressionLevel;
-    U32 const notificationLevel = params.notificationLevel;
-
-    /* check conditions */
-    DEBUGLOG(4, "ZDICT_finalizeDictionary");
-    if (dictBufferCapacity < dictContentSize) return ERROR(dstSize_tooSmall);
-    if (dictContentSize < ZDICT_CONTENTSIZE_MIN) return ERROR(srcSize_wrong);
-    if (dictBufferCapacity < ZDICT_DICTSIZE_MIN) return ERROR(dstSize_tooSmall);
-
-    /* dictionary header */
-    MEM_writeLE32(header, ZSTD_MAGIC_DICTIONARY);
-    {   U64 const randomID = XXH64(customDictContent, dictContentSize, 0);
-        U32 const compliantID = (randomID % ((1U<<31)-32768)) + 32768;
-        U32 const dictID = params.dictID ? params.dictID : compliantID;
-        MEM_writeLE32(header+4, dictID);
-    }
-    hSize = 8;
-
-    /* entropy tables */
-    DISPLAYLEVEL(2, "\r%70s\r", "");   /* clean display line */
-    DISPLAYLEVEL(2, "statistics ... \n");
-    {   size_t const eSize = ZDICT_analyzeEntropy(header+hSize, HBUFFSIZE-hSize,
-                                  compressionLevel,
-                                  samplesBuffer, samplesSizes, nbSamples,
-                                  customDictContent, dictContentSize,
-                                  notificationLevel);
-        if (ZDICT_isError(eSize)) return eSize;
-        hSize += eSize;
-    }
-
-    /* copy elements in final buffer ; note : src and dst buffer can overlap */
-    if (hSize + dictContentSize > dictBufferCapacity) dictContentSize = dictBufferCapacity - hSize;
-    {   size_t const dictSize = hSize + dictContentSize;
-        char* dictEnd = (char*)dictBuffer + dictSize;
-        memmove(dictEnd - dictContentSize, customDictContent, dictContentSize);
-        memcpy(dictBuffer, header, hSize);
-        return dictSize;
-    }
-}
-
-
-static size_t ZDICT_addEntropyTablesFromBuffer_advanced(
-        void* dictBuffer, size_t dictContentSize, size_t dictBufferCapacity,
-        const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-        ZDICT_params_t params)
-{
-    int const compressionLevel = (params.compressionLevel == 0) ? g_compressionLevel_default : params.compressionLevel;
-    U32 const notificationLevel = params.notificationLevel;
-    size_t hSize = 8;
-
-    /* calculate entropy tables */
-    DISPLAYLEVEL(2, "\r%70s\r", "");   /* clean display line */
-    DISPLAYLEVEL(2, "statistics ... \n");
-    {   size_t const eSize = ZDICT_analyzeEntropy((char*)dictBuffer+hSize, dictBufferCapacity-hSize,
-                                  compressionLevel,
-                                  samplesBuffer, samplesSizes, nbSamples,
-                                  (char*)dictBuffer + dictBufferCapacity - dictContentSize, dictContentSize,
-                                  notificationLevel);
-        if (ZDICT_isError(eSize)) return eSize;
-        hSize += eSize;
-    }
-
-    /* add dictionary header (after entropy tables) */
-    MEM_writeLE32(dictBuffer, ZSTD_MAGIC_DICTIONARY);
-    {   U64 const randomID = XXH64((char*)dictBuffer + dictBufferCapacity - dictContentSize, dictContentSize, 0);
-        U32 const compliantID = (randomID % ((1U<<31)-32768)) + 32768;
-        U32 const dictID = params.dictID ? params.dictID : compliantID;
-        MEM_writeLE32((char*)dictBuffer+4, dictID);
-    }
-
-    if (hSize + dictContentSize < dictBufferCapacity)
-        memmove((char*)dictBuffer + hSize, (char*)dictBuffer + dictBufferCapacity - dictContentSize, dictContentSize);
-    return MIN(dictBufferCapacity, hSize+dictContentSize);
-}
-
-/* Hidden declaration for dbio.c */
-size_t ZDICT_trainFromBuffer_unsafe_legacy(
-                            void* dictBuffer, size_t maxDictSize,
-                            const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-                            ZDICT_legacy_params_t params);
-/*! ZDICT_trainFromBuffer_unsafe_legacy() :
-*   Warning : `samplesBuffer` must be followed by noisy guard band.
-*   @return : size of dictionary, or an error code which can be tested with ZDICT_isError()
-*/
-size_t ZDICT_trainFromBuffer_unsafe_legacy(
-                            void* dictBuffer, size_t maxDictSize,
-                            const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-                            ZDICT_legacy_params_t params)
-{
-    U32 const dictListSize = MAX(MAX(DICTLISTSIZE_DEFAULT, nbSamples), (U32)(maxDictSize/16));
-    dictItem* const dictList = (dictItem*)malloc(dictListSize * sizeof(*dictList));
-    unsigned const selectivity = params.selectivityLevel == 0 ? g_selectivity_default : params.selectivityLevel;
-    unsigned const minRep = (selectivity > 30) ? MINRATIO : nbSamples >> selectivity;
-    size_t const targetDictSize = maxDictSize;
-    size_t const samplesBuffSize = ZDICT_totalSampleSize(samplesSizes, nbSamples);
-    size_t dictSize = 0;
-    U32 const notificationLevel = params.zParams.notificationLevel;
-
-    /* checks */
-    if (!dictList) return ERROR(memory_allocation);
-    if (maxDictSize < ZDICT_DICTSIZE_MIN) { free(dictList); return ERROR(dstSize_tooSmall); }   /* requested dictionary size is too small */
-    if (samplesBuffSize < ZDICT_MIN_SAMPLES_SIZE) { free(dictList); return ERROR(dictionaryCreation_failed); }   /* not enough source to create dictionary */
-
-    /* init */
-    ZDICT_initDictItem(dictList);
-
-    /* build dictionary */
-    ZDICT_trainBuffer_legacy(dictList, dictListSize,
-                       samplesBuffer, samplesBuffSize,
-                       samplesSizes, nbSamples,
-                       minRep, notificationLevel);
-
-    /* display best matches */
-    if (params.zParams.notificationLevel>= 3) {
-        unsigned const nb = MIN(25, dictList[0].pos);
-        unsigned const dictContentSize = ZDICT_dictSize(dictList);
-        unsigned u;
-        DISPLAYLEVEL(3, "\n %u segments found, of total size %u \n", (unsigned)dictList[0].pos-1, dictContentSize);
-        DISPLAYLEVEL(3, "list %u best segments \n", nb-1);
-        for (u=1; u<nb; u++) {
-            unsigned const pos = dictList[u].pos;
-            unsigned const length = dictList[u].length;
-            U32 const printedLength = MIN(40, length);
-            if ((pos > samplesBuffSize) || ((pos + length) > samplesBuffSize)) {
-                free(dictList);
-                return ERROR(GENERIC);   /* should never happen */
-            }
-            DISPLAYLEVEL(3, "%3u:%3u bytes at pos %8u, savings %7u bytes |",
-                         u, length, pos, (unsigned)dictList[u].savings);
-            ZDICT_printHex((const char*)samplesBuffer+pos, printedLength);
-            DISPLAYLEVEL(3, "| \n");
-    }   }
-
-
-    /* create dictionary */
-    {   unsigned dictContentSize = ZDICT_dictSize(dictList);
-        if (dictContentSize < ZDICT_CONTENTSIZE_MIN) { free(dictList); return ERROR(dictionaryCreation_failed); }   /* dictionary content too small */
-        if (dictContentSize < targetDictSize/4) {
-            DISPLAYLEVEL(2, "!  warning : selected content significantly smaller than requested (%u < %u) \n", dictContentSize, (unsigned)maxDictSize);
-            if (samplesBuffSize < 10 * targetDictSize)
-                DISPLAYLEVEL(2, "!  consider increasing the number of samples (total size : %u MB)\n", (unsigned)(samplesBuffSize>>20));
-            if (minRep > MINRATIO) {
-                DISPLAYLEVEL(2, "!  consider increasing selectivity to produce larger dictionary (-s%u) \n", selectivity+1);
-                DISPLAYLEVEL(2, "!  note : larger dictionaries are not necessarily better, test its efficiency on samples \n");
-            }
-        }
-
-        if ((dictContentSize > targetDictSize*3) && (nbSamples > 2*MINRATIO) && (selectivity>1)) {
-            unsigned proposedSelectivity = selectivity-1;
-            while ((nbSamples >> proposedSelectivity) <= MINRATIO) { proposedSelectivity--; }
-            DISPLAYLEVEL(2, "!  note : calculated dictionary significantly larger than requested (%u > %u) \n", dictContentSize, (unsigned)maxDictSize);
-            DISPLAYLEVEL(2, "!  consider increasing dictionary size, or produce denser dictionary (-s%u) \n", proposedSelectivity);
-            DISPLAYLEVEL(2, "!  always test dictionary efficiency on real samples \n");
-        }
-
-        /* limit dictionary size */
-        {   U32 const max = dictList->pos;   /* convention : nb of useful elts within dictList */
-            U32 currentSize = 0;
-            U32 n; for (n=1; n<max; n++) {
-                currentSize += dictList[n].length;
-                if (currentSize > targetDictSize) { currentSize -= dictList[n].length; break; }
-            }
-            dictList->pos = n;
-            dictContentSize = currentSize;
-        }
-
-        /* build dict content */
-        {   U32 u;
-            BYTE* ptr = (BYTE*)dictBuffer + maxDictSize;
-            for (u=1; u<dictList->pos; u++) {
-                U32 l = dictList[u].length;
-                ptr -= l;
-                if (ptr<(BYTE*)dictBuffer) { free(dictList); return ERROR(GENERIC); }   /* should not happen */
-                memcpy(ptr, (const char*)samplesBuffer+dictList[u].pos, l);
-        }   }
-
-        dictSize = ZDICT_addEntropyTablesFromBuffer_advanced(dictBuffer, dictContentSize, maxDictSize,
-                                                             samplesBuffer, samplesSizes, nbSamples,
-                                                             params.zParams);
-    }
-
-    /* clean up */
-    free(dictList);
-    return dictSize;
-}
-
-
-/* ZDICT_trainFromBuffer_legacy() :
- * issue : samplesBuffer need to be followed by a noisy guard band.
- * work around : duplicate the buffer, and add the noise */
-size_t ZDICT_trainFromBuffer_legacy(void* dictBuffer, size_t dictBufferCapacity,
-                              const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-                              ZDICT_legacy_params_t params)
-{
-    size_t result;
-    void* newBuff;
-    size_t const sBuffSize = ZDICT_totalSampleSize(samplesSizes, nbSamples);
-    if (sBuffSize < ZDICT_MIN_SAMPLES_SIZE) return 0;   /* not enough content => no dictionary */
-
-    newBuff = malloc(sBuffSize + NOISELENGTH);
-    if (!newBuff) return ERROR(memory_allocation);
-
-    memcpy(newBuff, samplesBuffer, sBuffSize);
-    ZDICT_fillNoise((char*)newBuff + sBuffSize, NOISELENGTH);   /* guard band, for end of buffer condition */
-
-    result =
-        ZDICT_trainFromBuffer_unsafe_legacy(dictBuffer, dictBufferCapacity, newBuff,
-                                            samplesSizes, nbSamples, params);
-    free(newBuff);
-    return result;
-}
-
-
-size_t ZDICT_trainFromBuffer(void* dictBuffer, size_t dictBufferCapacity,
-                             const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples)
-{
-    ZDICT_fastCover_params_t params;
-    DEBUGLOG(3, "ZDICT_trainFromBuffer");
-    memset(&params, 0, sizeof(params));
-    params.d = 8;
-    params.steps = 4;
-    /* Default to level 6 since no compression level information is available */
-    params.zParams.compressionLevel = 3;
-#if defined(DEBUGLEVEL) && (DEBUGLEVEL>=1)
-    params.zParams.notificationLevel = DEBUGLEVEL;
-#endif
-    return ZDICT_optimizeTrainFromBuffer_fastCover(dictBuffer, dictBufferCapacity,
-                                               samplesBuffer, samplesSizes, nbSamples,
-                                               &params);
-}
-
-size_t ZDICT_addEntropyTablesFromBuffer(void* dictBuffer, size_t dictContentSize, size_t dictBufferCapacity,
-                                  const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples)
-{
-    ZDICT_params_t params;
-    memset(&params, 0, sizeof(params));
-    return ZDICT_addEntropyTablesFromBuffer_advanced(dictBuffer, dictContentSize, dictBufferCapacity,
-                                                     samplesBuffer, samplesSizes, nbSamples,
-                                                     params);
-}
diff --git a/vendor/github.com/DataDog/zstd/zdict.h b/vendor/github.com/DataDog/zstd/zdict.h
deleted file mode 100644
index 37978ec..0000000
--- a/vendor/github.com/DataDog/zstd/zdict.h
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef DICTBUILDER_H_001
-#define DICTBUILDER_H_001
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*======  Dependencies  ======*/
-#include <stddef.h>  /* size_t */
-
-
-/* =====   ZDICTLIB_API : control library symbols visibility   ===== */
-#ifndef ZDICTLIB_VISIBILITY
-#  if defined(__GNUC__) && (__GNUC__ >= 4)
-#    define ZDICTLIB_VISIBILITY __attribute__ ((visibility ("default")))
-#  else
-#    define ZDICTLIB_VISIBILITY
-#  endif
-#endif
-#if defined(ZSTD_DLL_EXPORT) && (ZSTD_DLL_EXPORT==1)
-#  define ZDICTLIB_API __declspec(dllexport) ZDICTLIB_VISIBILITY
-#elif defined(ZSTD_DLL_IMPORT) && (ZSTD_DLL_IMPORT==1)
-#  define ZDICTLIB_API __declspec(dllimport) ZDICTLIB_VISIBILITY /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/
-#else
-#  define ZDICTLIB_API ZDICTLIB_VISIBILITY
-#endif
-
-
-/*! ZDICT_trainFromBuffer():
- *  Train a dictionary from an array of samples.
- *  Redirect towards ZDICT_optimizeTrainFromBuffer_fastCover() single-threaded, with d=8, steps=4,
- *  f=20, and accel=1.
- *  Samples must be stored concatenated in a single flat buffer `samplesBuffer`,
- *  supplied with an array of sizes `samplesSizes`, providing the size of each sample, in order.
- *  The resulting dictionary will be saved into `dictBuffer`.
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *  Note:  Dictionary training will fail if there are not enough samples to construct a
- *         dictionary, or if most of the samples are too small (< 8 bytes being the lower limit).
- *         If dictionary training fails, you should use zstd without a dictionary, as the dictionary
- *         would've been ineffective anyways. If you believe your samples would benefit from a dictionary
- *         please open an issue with details, and we can look into it.
- *  Note: ZDICT_trainFromBuffer()'s memory usage is about 6 MB.
- *  Tips: In general, a reasonable dictionary has a size of ~ 100 KB.
- *        It's possible to select smaller or larger size, just by specifying `dictBufferCapacity`.
- *        In general, it's recommended to provide a few thousands samples, though this can vary a lot.
- *        It's recommended that total size of all samples be about ~x100 times the target size of dictionary.
- */
-ZDICTLIB_API size_t ZDICT_trainFromBuffer(void* dictBuffer, size_t dictBufferCapacity,
-                                    const void* samplesBuffer,
-                                    const size_t* samplesSizes, unsigned nbSamples);
-
-
-/*======   Helper functions   ======*/
-ZDICTLIB_API unsigned ZDICT_getDictID(const void* dictBuffer, size_t dictSize);  /**< extracts dictID; @return zero if error (not a valid dictionary) */
-ZDICTLIB_API unsigned ZDICT_isError(size_t errorCode);
-ZDICTLIB_API const char* ZDICT_getErrorName(size_t errorCode);
-
-
-
-#ifdef ZDICT_STATIC_LINKING_ONLY
-
-/* ====================================================================================
- * The definitions in this section are considered experimental.
- * They should never be used with a dynamic library, as they may change in the future.
- * They are provided for advanced usages.
- * Use them only in association with static linking.
- * ==================================================================================== */
-
-typedef struct {
-    int      compressionLevel;   /* optimize for a specific zstd compression level; 0 means default */
-    unsigned notificationLevel;  /* Write log to stderr; 0 = none (default); 1 = errors; 2 = progression; 3 = details; 4 = debug; */
-    unsigned dictID;             /* force dictID value; 0 means auto mode (32-bits random value) */
-} ZDICT_params_t;
-
-/*! ZDICT_cover_params_t:
- *  k and d are the only required parameters.
- *  For others, value 0 means default.
- */
-typedef struct {
-    unsigned k;                  /* Segment size : constraint: 0 < k : Reasonable range [16, 2048+] */
-    unsigned d;                  /* dmer size : constraint: 0 < d <= k : Reasonable range [6, 16] */
-    unsigned steps;              /* Number of steps : Only used for optimization : 0 means default (40) : Higher means more parameters checked */
-    unsigned nbThreads;          /* Number of threads : constraint: 0 < nbThreads : 1 means single-threaded : Only used for optimization : Ignored if ZSTD_MULTITHREAD is not defined */
-    double splitPoint;           /* Percentage of samples used for training: Only used for optimization : the first nbSamples * splitPoint samples will be used to training, the last nbSamples * (1 - splitPoint) samples will be used for testing, 0 means default (1.0), 1.0 when all samples are used for both training and testing */
-    unsigned shrinkDict;         /* Train dictionaries to shrink in size starting from the minimum size and selects the smallest dictionary that is shrinkDictMaxRegression% worse than the largest dictionary. 0 means no shrinking and 1 means shrinking  */
-    unsigned shrinkDictMaxRegression; /* Sets shrinkDictMaxRegression so that a smaller dictionary can be at worse shrinkDictMaxRegression% worse than the max dict size dictionary. */
-    ZDICT_params_t zParams;
-} ZDICT_cover_params_t;
-
-typedef struct {
-    unsigned k;                  /* Segment size : constraint: 0 < k : Reasonable range [16, 2048+] */
-    unsigned d;                  /* dmer size : constraint: 0 < d <= k : Reasonable range [6, 16] */
-    unsigned f;                  /* log of size of frequency array : constraint: 0 < f <= 31 : 1 means default(20)*/
-    unsigned steps;              /* Number of steps : Only used for optimization : 0 means default (40) : Higher means more parameters checked */
-    unsigned nbThreads;          /* Number of threads : constraint: 0 < nbThreads : 1 means single-threaded : Only used for optimization : Ignored if ZSTD_MULTITHREAD is not defined */
-    double splitPoint;           /* Percentage of samples used for training: Only used for optimization : the first nbSamples * splitPoint samples will be used to training, the last nbSamples * (1 - splitPoint) samples will be used for testing, 0 means default (0.75), 1.0 when all samples are used for both training and testing */
-    unsigned accel;              /* Acceleration level: constraint: 0 < accel <= 10, higher means faster and less accurate, 0 means default(1) */
-    unsigned shrinkDict;         /* Train dictionaries to shrink in size starting from the minimum size and selects the smallest dictionary that is shrinkDictMaxRegression% worse than the largest dictionary. 0 means no shrinking and 1 means shrinking  */
-    unsigned shrinkDictMaxRegression; /* Sets shrinkDictMaxRegression so that a smaller dictionary can be at worse shrinkDictMaxRegression% worse than the max dict size dictionary. */
-
-    ZDICT_params_t zParams;
-} ZDICT_fastCover_params_t;
-
-/*! ZDICT_trainFromBuffer_cover():
- *  Train a dictionary from an array of samples using the COVER algorithm.
- *  Samples must be stored concatenated in a single flat buffer `samplesBuffer`,
- *  supplied with an array of sizes `samplesSizes`, providing the size of each sample, in order.
- *  The resulting dictionary will be saved into `dictBuffer`.
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *          See ZDICT_trainFromBuffer() for details on failure modes.
- *  Note: ZDICT_trainFromBuffer_cover() requires about 9 bytes of memory for each input byte.
- *  Tips: In general, a reasonable dictionary has a size of ~ 100 KB.
- *        It's possible to select smaller or larger size, just by specifying `dictBufferCapacity`.
- *        In general, it's recommended to provide a few thousands samples, though this can vary a lot.
- *        It's recommended that total size of all samples be about ~x100 times the target size of dictionary.
- */
-ZDICTLIB_API size_t ZDICT_trainFromBuffer_cover(
-          void *dictBuffer, size_t dictBufferCapacity,
-    const void *samplesBuffer, const size_t *samplesSizes, unsigned nbSamples,
-          ZDICT_cover_params_t parameters);
-
-/*! ZDICT_optimizeTrainFromBuffer_cover():
- * The same requirements as above hold for all the parameters except `parameters`.
- * This function tries many parameter combinations and picks the best parameters.
- * `*parameters` is filled with the best parameters found,
- * dictionary constructed with those parameters is stored in `dictBuffer`.
- *
- * All of the parameters d, k, steps are optional.
- * If d is non-zero then we don't check multiple values of d, otherwise we check d = {6, 8}.
- * if steps is zero it defaults to its default value.
- * If k is non-zero then we don't check multiple values of k, otherwise we check steps values in [50, 2000].
- *
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *          On success `*parameters` contains the parameters selected.
- *          See ZDICT_trainFromBuffer() for details on failure modes.
- * Note: ZDICT_optimizeTrainFromBuffer_cover() requires about 8 bytes of memory for each input byte and additionally another 5 bytes of memory for each byte of memory for each thread.
- */
-ZDICTLIB_API size_t ZDICT_optimizeTrainFromBuffer_cover(
-          void* dictBuffer, size_t dictBufferCapacity,
-    const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-          ZDICT_cover_params_t* parameters);
-
-/*! ZDICT_trainFromBuffer_fastCover():
- *  Train a dictionary from an array of samples using a modified version of COVER algorithm.
- *  Samples must be stored concatenated in a single flat buffer `samplesBuffer`,
- *  supplied with an array of sizes `samplesSizes`, providing the size of each sample, in order.
- *  d and k are required.
- *  All other parameters are optional, will use default values if not provided
- *  The resulting dictionary will be saved into `dictBuffer`.
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *          See ZDICT_trainFromBuffer() for details on failure modes.
- *  Note: ZDICT_trainFromBuffer_fastCover() requires 6 * 2^f bytes of memory.
- *  Tips: In general, a reasonable dictionary has a size of ~ 100 KB.
- *        It's possible to select smaller or larger size, just by specifying `dictBufferCapacity`.
- *        In general, it's recommended to provide a few thousands samples, though this can vary a lot.
- *        It's recommended that total size of all samples be about ~x100 times the target size of dictionary.
- */
-ZDICTLIB_API size_t ZDICT_trainFromBuffer_fastCover(void *dictBuffer,
-                    size_t dictBufferCapacity, const void *samplesBuffer,
-                    const size_t *samplesSizes, unsigned nbSamples,
-                    ZDICT_fastCover_params_t parameters);
-
-/*! ZDICT_optimizeTrainFromBuffer_fastCover():
- * The same requirements as above hold for all the parameters except `parameters`.
- * This function tries many parameter combinations (specifically, k and d combinations)
- * and picks the best parameters. `*parameters` is filled with the best parameters found,
- * dictionary constructed with those parameters is stored in `dictBuffer`.
- * All of the parameters d, k, steps, f, and accel are optional.
- * If d is non-zero then we don't check multiple values of d, otherwise we check d = {6, 8}.
- * if steps is zero it defaults to its default value.
- * If k is non-zero then we don't check multiple values of k, otherwise we check steps values in [50, 2000].
- * If f is zero, default value of 20 is used.
- * If accel is zero, default value of 1 is used.
- *
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *          On success `*parameters` contains the parameters selected.
- *          See ZDICT_trainFromBuffer() for details on failure modes.
- * Note: ZDICT_optimizeTrainFromBuffer_fastCover() requires about 6 * 2^f bytes of memory for each thread.
- */
-ZDICTLIB_API size_t ZDICT_optimizeTrainFromBuffer_fastCover(void* dictBuffer,
-                    size_t dictBufferCapacity, const void* samplesBuffer,
-                    const size_t* samplesSizes, unsigned nbSamples,
-                    ZDICT_fastCover_params_t* parameters);
-
-/*! ZDICT_finalizeDictionary():
- * Given a custom content as a basis for dictionary, and a set of samples,
- * finalize dictionary by adding headers and statistics.
- *
- * Samples must be stored concatenated in a flat buffer `samplesBuffer`,
- * supplied with an array of sizes `samplesSizes`, providing the size of each sample in order.
- *
- * dictContentSize must be >= ZDICT_CONTENTSIZE_MIN bytes.
- * maxDictSize must be >= dictContentSize, and must be >= ZDICT_DICTSIZE_MIN bytes.
- *
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`),
- *          or an error code, which can be tested by ZDICT_isError().
- * Note: ZDICT_finalizeDictionary() will push notifications into stderr if instructed to, using notificationLevel>0.
- * Note 2: dictBuffer and dictContent can overlap
- */
-#define ZDICT_CONTENTSIZE_MIN 128
-#define ZDICT_DICTSIZE_MIN    256
-ZDICTLIB_API size_t ZDICT_finalizeDictionary(void* dictBuffer, size_t dictBufferCapacity,
-                                const void* dictContent, size_t dictContentSize,
-                                const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples,
-                                ZDICT_params_t parameters);
-
-typedef struct {
-    unsigned selectivityLevel;   /* 0 means default; larger => select more => larger dictionary */
-    ZDICT_params_t zParams;
-} ZDICT_legacy_params_t;
-
-/*! ZDICT_trainFromBuffer_legacy():
- *  Train a dictionary from an array of samples.
- *  Samples must be stored concatenated in a single flat buffer `samplesBuffer`,
- *  supplied with an array of sizes `samplesSizes`, providing the size of each sample, in order.
- *  The resulting dictionary will be saved into `dictBuffer`.
- * `parameters` is optional and can be provided with values set to 0 to mean "default".
- * @return: size of dictionary stored into `dictBuffer` (<= `dictBufferCapacity`)
- *          or an error code, which can be tested with ZDICT_isError().
- *          See ZDICT_trainFromBuffer() for details on failure modes.
- *  Tips: In general, a reasonable dictionary has a size of ~ 100 KB.
- *        It's possible to select smaller or larger size, just by specifying `dictBufferCapacity`.
- *        In general, it's recommended to provide a few thousands samples, though this can vary a lot.
- *        It's recommended that total size of all samples be about ~x100 times the target size of dictionary.
- *  Note: ZDICT_trainFromBuffer_legacy() will send notifications into stderr if instructed to, using notificationLevel>0.
- */
-ZDICTLIB_API size_t ZDICT_trainFromBuffer_legacy(
-    void *dictBuffer, size_t dictBufferCapacity,
-    const void *samplesBuffer, const size_t *samplesSizes, unsigned nbSamples,
-    ZDICT_legacy_params_t parameters);
-
-/* Deprecation warnings */
-/* It is generally possible to disable deprecation warnings from compiler,
-   for example with -Wno-deprecated-declarations for gcc
-   or _CRT_SECURE_NO_WARNINGS in Visual.
-   Otherwise, it's also possible to manually define ZDICT_DISABLE_DEPRECATE_WARNINGS */
-#ifdef ZDICT_DISABLE_DEPRECATE_WARNINGS
-#  define ZDICT_DEPRECATED(message) ZDICTLIB_API   /* disable deprecation warnings */
-#else
-#  define ZDICT_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__)
-#  if defined (__cplusplus) && (__cplusplus >= 201402) /* C++14 or greater */
-#    define ZDICT_DEPRECATED(message) [[deprecated(message)]] ZDICTLIB_API
-#  elif (ZDICT_GCC_VERSION >= 405) || defined(__clang__)
-#    define ZDICT_DEPRECATED(message) ZDICTLIB_API __attribute__((deprecated(message)))
-#  elif (ZDICT_GCC_VERSION >= 301)
-#    define ZDICT_DEPRECATED(message) ZDICTLIB_API __attribute__((deprecated))
-#  elif defined(_MSC_VER)
-#    define ZDICT_DEPRECATED(message) ZDICTLIB_API __declspec(deprecated(message))
-#  else
-#    pragma message("WARNING: You need to implement ZDICT_DEPRECATED for this compiler")
-#    define ZDICT_DEPRECATED(message) ZDICTLIB_API
-#  endif
-#endif /* ZDICT_DISABLE_DEPRECATE_WARNINGS */
-
-ZDICT_DEPRECATED("use ZDICT_finalizeDictionary() instead")
-size_t ZDICT_addEntropyTablesFromBuffer(void* dictBuffer, size_t dictContentSize, size_t dictBufferCapacity,
-                                  const void* samplesBuffer, const size_t* samplesSizes, unsigned nbSamples);
-
-
-#endif   /* ZDICT_STATIC_LINKING_ONLY */
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* DICTBUILDER_H_001 */
diff --git a/vendor/github.com/DataDog/zstd/zstd.go b/vendor/github.com/DataDog/zstd/zstd.go
deleted file mode 100644
index b6af4eb..0000000
--- a/vendor/github.com/DataDog/zstd/zstd.go
+++ /dev/null
@@ -1,147 +0,0 @@
-package zstd
-
-/*
-#define ZSTD_STATIC_LINKING_ONLY
-#include "zstd.h"
-#include "stdint.h"  // for uintptr_t
-
-// The following *_wrapper function are used for removing superflouos
-// memory allocations when calling the wrapped functions from Go code.
-// See https://github.com/golang/go/issues/24450 for details.
-
-static size_t ZSTD_compress_wrapper(uintptr_t dst, size_t maxDstSize, const uintptr_t src, size_t srcSize, int compressionLevel) {
-	return ZSTD_compress((void*)dst, maxDstSize, (const void*)src, srcSize, compressionLevel);
-}
-
-static size_t ZSTD_decompress_wrapper(uintptr_t dst, size_t maxDstSize, uintptr_t src, size_t srcSize) {
-	return ZSTD_decompress((void*)dst, maxDstSize, (const void *)src, srcSize);
-}
-
-*/
-import "C"
-import (
-	"bytes"
-	"errors"
-	"io/ioutil"
-	"runtime"
-	"unsafe"
-)
-
-// Defines best and standard values for zstd cli
-const (
-	BestSpeed          = 1
-	BestCompression    = 20
-	DefaultCompression = 5
-)
-
-var (
-	// ErrEmptySlice is returned when there is nothing to compress
-	ErrEmptySlice = errors.New("Bytes slice is empty")
-)
-
-// CompressBound returns the worst case size needed for a destination buffer,
-// which can be used to preallocate a destination buffer or select a previously
-// allocated buffer from a pool.
-// See zstd.h to mirror implementation of ZSTD_COMPRESSBOUND
-func CompressBound(srcSize int) int {
-	lowLimit := 128 << 10 // 128 kB
-	var margin int
-	if srcSize < lowLimit {
-		margin = (lowLimit - srcSize) >> 11
-	}
-	return srcSize + (srcSize >> 8) + margin
-}
-
-// cCompressBound is a cgo call to check the go implementation above against the c code.
-func cCompressBound(srcSize int) int {
-	return int(C.ZSTD_compressBound(C.size_t(srcSize)))
-}
-
-// Compress src into dst.  If you have a buffer to use, you can pass it to
-// prevent allocation.  If it is too small, or if nil is passed, a new buffer
-// will be allocated and returned.
-func Compress(dst, src []byte) ([]byte, error) {
-	return CompressLevel(dst, src, DefaultCompression)
-}
-
-// CompressLevel is the same as Compress but you can pass a compression level
-func CompressLevel(dst, src []byte, level int) ([]byte, error) {
-	bound := CompressBound(len(src))
-	if cap(dst) >= bound {
-		dst = dst[0:bound] // Reuse dst buffer
-	} else {
-		dst = make([]byte, bound)
-	}
-
-	srcPtr := C.uintptr_t(uintptr(0)) // Do not point anywhere, if src is empty
-	if len(src) > 0 {
-		srcPtr = C.uintptr_t(uintptr(unsafe.Pointer(&src[0])))
-	}
-
-	cWritten := C.ZSTD_compress_wrapper(
-		C.uintptr_t(uintptr(unsafe.Pointer(&dst[0]))),
-		C.size_t(len(dst)),
-		srcPtr,
-		C.size_t(len(src)),
-		C.int(level))
-
-	runtime.KeepAlive(src)
-	written := int(cWritten)
-	// Check if the return is an Error code
-	if err := getError(written); err != nil {
-		return nil, err
-	}
-	return dst[:written], nil
-}
-
-// Decompress src into dst.  If you have a buffer to use, you can pass it to
-// prevent allocation.  If it is too small, or if nil is passed, a new buffer
-// will be allocated and returned.
-func Decompress(dst, src []byte) ([]byte, error) {
-	if len(src) == 0 {
-		return []byte{}, ErrEmptySlice
-	}
-	decompress := func(dst, src []byte) ([]byte, error) {
-
-		cWritten := C.ZSTD_decompress_wrapper(
-			C.uintptr_t(uintptr(unsafe.Pointer(&dst[0]))),
-			C.size_t(len(dst)),
-			C.uintptr_t(uintptr(unsafe.Pointer(&src[0]))),
-			C.size_t(len(src)))
-
-		runtime.KeepAlive(src)
-		written := int(cWritten)
-		// Check error
-		if err := getError(written); err != nil {
-			return nil, err
-		}
-		return dst[:written], nil
-	}
-
-	if len(dst) == 0 {
-		// Attempt to use zStd to determine decompressed size (may result in error or 0)
-		size := int(C.size_t(C.ZSTD_getDecompressedSize(unsafe.Pointer(&src[0]), C.size_t(len(src)))))
-
-		if err := getError(size); err != nil {
-			return nil, err
-		}
-
-		if size > 0 {
-			dst = make([]byte, size)
-		} else {
-			dst = make([]byte, len(src)*3) // starting guess
-		}
-	}
-	for i := 0; i < 3; i++ { // 3 tries to allocate a bigger buffer
-		result, err := decompress(dst, src)
-		if !IsDstSizeTooSmallError(err) {
-			return result, err
-		}
-		dst = make([]byte, len(dst)*2) // Grow buffer by 2
-	}
-
-	// We failed getting a dst buffer of correct size, use stream API
-	r := NewReader(bytes.NewReader(src))
-	defer r.Close()
-	return ioutil.ReadAll(r)
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd.h b/vendor/github.com/DataDog/zstd/zstd.h
deleted file mode 100644
index a1910ee..0000000
--- a/vendor/github.com/DataDog/zstd/zstd.h
+++ /dev/null
@@ -1,1945 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#ifndef ZSTD_H_235446
-#define ZSTD_H_235446
-
-/* ======   Dependency   ======*/
-#include <stddef.h>   /* size_t */
-
-
-/* =====   ZSTDLIB_API : control library symbols visibility   ===== */
-#ifndef ZSTDLIB_VISIBILITY
-#  if defined(__GNUC__) && (__GNUC__ >= 4)
-#    define ZSTDLIB_VISIBILITY __attribute__ ((visibility ("default")))
-#  else
-#    define ZSTDLIB_VISIBILITY
-#  endif
-#endif
-#if defined(ZSTD_DLL_EXPORT) && (ZSTD_DLL_EXPORT==1)
-#  define ZSTDLIB_API __declspec(dllexport) ZSTDLIB_VISIBILITY
-#elif defined(ZSTD_DLL_IMPORT) && (ZSTD_DLL_IMPORT==1)
-#  define ZSTDLIB_API __declspec(dllimport) ZSTDLIB_VISIBILITY /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/
-#else
-#  define ZSTDLIB_API ZSTDLIB_VISIBILITY
-#endif
-
-
-/*******************************************************************************
-  Introduction
-
-  zstd, short for Zstandard, is a fast lossless compression algorithm, targeting
-  real-time compression scenarios at zlib-level and better compression ratios.
-  The zstd compression library provides in-memory compression and decompression
-  functions.
-
-  The library supports regular compression levels from 1 up to ZSTD_maxCLevel(),
-  which is currently 22. Levels >= 20, labeled `--ultra`, should be used with
-  caution, as they require more memory. The library also offers negative
-  compression levels, which extend the range of speed vs. ratio preferences.
-  The lower the level, the faster the speed (at the cost of compression).
-
-  Compression can be done in:
-    - a single step (described as Simple API)
-    - a single step, reusing a context (described as Explicit context)
-    - unbounded multiple steps (described as Streaming compression)
-
-  The compression ratio achievable on small data can be highly improved using
-  a dictionary. Dictionary compression can be performed in:
-    - a single step (described as Simple dictionary API)
-    - a single step, reusing a dictionary (described as Bulk-processing
-      dictionary API)
-
-  Advanced experimental functions can be accessed using
-  `#define ZSTD_STATIC_LINKING_ONLY` before including zstd.h.
-
-  Advanced experimental APIs should never be used with a dynamically-linked
-  library. They are not "stable"; their definitions or signatures may change in
-  the future. Only static linking is allowed.
-*******************************************************************************/
-
-/*------   Version   ------*/
-#define ZSTD_VERSION_MAJOR    1
-#define ZSTD_VERSION_MINOR    4
-#define ZSTD_VERSION_RELEASE  1
-
-#define ZSTD_VERSION_NUMBER  (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
-ZSTDLIB_API unsigned ZSTD_versionNumber(void);   /**< to check runtime library version */
-
-#define ZSTD_LIB_VERSION ZSTD_VERSION_MAJOR.ZSTD_VERSION_MINOR.ZSTD_VERSION_RELEASE
-#define ZSTD_QUOTE(str) #str
-#define ZSTD_EXPAND_AND_QUOTE(str) ZSTD_QUOTE(str)
-#define ZSTD_VERSION_STRING ZSTD_EXPAND_AND_QUOTE(ZSTD_LIB_VERSION)
-ZSTDLIB_API const char* ZSTD_versionString(void);   /* requires v1.3.0+ */
-
-/* *************************************
- *  Default constant
- ***************************************/
-#ifndef ZSTD_CLEVEL_DEFAULT
-#  define ZSTD_CLEVEL_DEFAULT 3
-#endif
-
-/* *************************************
- *  Constants
- ***************************************/
-
-/* All magic numbers are supposed read/written to/from files/memory using little-endian convention */
-#define ZSTD_MAGICNUMBER            0xFD2FB528    /* valid since v0.8.0 */
-#define ZSTD_MAGIC_DICTIONARY       0xEC30A437    /* valid since v0.7.0 */
-#define ZSTD_MAGIC_SKIPPABLE_START  0x184D2A50    /* all 16 values, from 0x184D2A50 to 0x184D2A5F, signal the beginning of a skippable frame */
-#define ZSTD_MAGIC_SKIPPABLE_MASK   0xFFFFFFF0
-
-#define ZSTD_BLOCKSIZELOG_MAX  17
-#define ZSTD_BLOCKSIZE_MAX     (1<<ZSTD_BLOCKSIZELOG_MAX)
-
-
-
-/***************************************
-*  Simple API
-***************************************/
-/*! ZSTD_compress() :
- *  Compresses `src` content as a single zstd compressed frame into already allocated `dst`.
- *  Hint : compression runs faster if `dstCapacity` >=  `ZSTD_compressBound(srcSize)`.
- *  @return : compressed size written into `dst` (<= `dstCapacity),
- *            or an error code if it fails (which can be tested using ZSTD_isError()). */
-ZSTDLIB_API size_t ZSTD_compress( void* dst, size_t dstCapacity,
-                            const void* src, size_t srcSize,
-                                  int compressionLevel);
-
-/*! ZSTD_decompress() :
- *  `compressedSize` : must be the _exact_ size of some number of compressed and/or skippable frames.
- *  `dstCapacity` is an upper bound of originalSize to regenerate.
- *  If user cannot imply a maximum upper bound, it's better to use streaming mode to decompress data.
- *  @return : the number of bytes decompressed into `dst` (<= `dstCapacity`),
- *            or an errorCode if it fails (which can be tested using ZSTD_isError()). */
-ZSTDLIB_API size_t ZSTD_decompress( void* dst, size_t dstCapacity,
-                              const void* src, size_t compressedSize);
-
-/*! ZSTD_getFrameContentSize() : requires v1.3.0+
- *  `src` should point to the start of a ZSTD encoded frame.
- *  `srcSize` must be at least as large as the frame header.
- *            hint : any size >= `ZSTD_frameHeaderSize_max` is large enough.
- *  @return : - decompressed size of `src` frame content, if known
- *            - ZSTD_CONTENTSIZE_UNKNOWN if the size cannot be determined
- *            - ZSTD_CONTENTSIZE_ERROR if an error occurred (e.g. invalid magic number, srcSize too small)
- *   note 1 : a 0 return value means the frame is valid but "empty".
- *   note 2 : decompressed size is an optional field, it may not be present, typically in streaming mode.
- *            When `return==ZSTD_CONTENTSIZE_UNKNOWN`, data to decompress could be any size.
- *            In which case, it's necessary to use streaming mode to decompress data.
- *            Optionally, application can rely on some implicit limit,
- *            as ZSTD_decompress() only needs an upper bound of decompressed size.
- *            (For example, data could be necessarily cut into blocks <= 16 KB).
- *   note 3 : decompressed size is always present when compression is completed using single-pass functions,
- *            such as ZSTD_compress(), ZSTD_compressCCtx() ZSTD_compress_usingDict() or ZSTD_compress_usingCDict().
- *   note 4 : decompressed size can be very large (64-bits value),
- *            potentially larger than what local system can handle as a single memory segment.
- *            In which case, it's necessary to use streaming mode to decompress data.
- *   note 5 : If source is untrusted, decompressed size could be wrong or intentionally modified.
- *            Always ensure return value fits within application's authorized limits.
- *            Each application can set its own limits.
- *   note 6 : This function replaces ZSTD_getDecompressedSize() */
-#define ZSTD_CONTENTSIZE_UNKNOWN (0ULL - 1)
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-ZSTDLIB_API unsigned long long ZSTD_getFrameContentSize(const void *src, size_t srcSize);
-
-/*! ZSTD_getDecompressedSize() :
- *  NOTE: This function is now obsolete, in favor of ZSTD_getFrameContentSize().
- *  Both functions work the same way, but ZSTD_getDecompressedSize() blends
- *  "empty", "unknown" and "error" results to the same return value (0),
- *  while ZSTD_getFrameContentSize() gives them separate return values.
- * @return : decompressed size of `src` frame content _if known and not empty_, 0 otherwise. */
-ZSTDLIB_API unsigned long long ZSTD_getDecompressedSize(const void* src, size_t srcSize);
-
-/*! ZSTD_findFrameCompressedSize() :
- * `src` should point to the start of a ZSTD frame or skippable frame.
- * `srcSize` must be >= first frame size
- * @return : the compressed size of the first frame starting at `src`,
- *           suitable to pass as `srcSize` to `ZSTD_decompress` or similar,
- *        or an error code if input is invalid */
-ZSTDLIB_API size_t ZSTD_findFrameCompressedSize(const void* src, size_t srcSize);
-
-
-/*======  Helper functions  ======*/
-#define ZSTD_COMPRESSBOUND(srcSize)   ((srcSize) + ((srcSize)>>8) + (((srcSize) < (128<<10)) ? (((128<<10) - (srcSize)) >> 11) /* margin, from 64 to 0 */ : 0))  /* this formula ensures that bound(A) + bound(B) <= bound(A+B) as long as A and B >= 128 KB */
-ZSTDLIB_API size_t      ZSTD_compressBound(size_t srcSize); /*!< maximum compressed size in worst case single-pass scenario */
-ZSTDLIB_API unsigned    ZSTD_isError(size_t code);          /*!< tells if a `size_t` function result is an error code */
-ZSTDLIB_API const char* ZSTD_getErrorName(size_t code);     /*!< provides readable string from an error code */
-ZSTDLIB_API int         ZSTD_minCLevel(void);               /*!< minimum negative compression level allowed */
-ZSTDLIB_API int         ZSTD_maxCLevel(void);               /*!< maximum compression level available */
-
-
-/***************************************
-*  Explicit context
-***************************************/
-/*= Compression context
- *  When compressing many times,
- *  it is recommended to allocate a context just once,
- *  and re-use it for each successive compression operation.
- *  This will make workload friendlier for system's memory.
- *  Note : re-using context is just a speed / resource optimization.
- *         It doesn't change the compression ratio, which remains identical.
- *  Note 2 : In multi-threaded environments,
- *         use one different context per thread for parallel execution.
- */
-typedef struct ZSTD_CCtx_s ZSTD_CCtx;
-ZSTDLIB_API ZSTD_CCtx* ZSTD_createCCtx(void);
-ZSTDLIB_API size_t     ZSTD_freeCCtx(ZSTD_CCtx* cctx);
-
-/*! ZSTD_compressCCtx() :
- *  Same as ZSTD_compress(), using an explicit ZSTD_CCtx
- *  The function will compress at requested compression level,
- *  ignoring any other parameter */
-ZSTDLIB_API size_t ZSTD_compressCCtx(ZSTD_CCtx* cctx,
-                                     void* dst, size_t dstCapacity,
-                               const void* src, size_t srcSize,
-                                     int compressionLevel);
-
-/*= Decompression context
- *  When decompressing many times,
- *  it is recommended to allocate a context only once,
- *  and re-use it for each successive compression operation.
- *  This will make workload friendlier for system's memory.
- *  Use one context per thread for parallel execution. */
-typedef struct ZSTD_DCtx_s ZSTD_DCtx;
-ZSTDLIB_API ZSTD_DCtx* ZSTD_createDCtx(void);
-ZSTDLIB_API size_t     ZSTD_freeDCtx(ZSTD_DCtx* dctx);
-
-/*! ZSTD_decompressDCtx() :
- *  Same as ZSTD_decompress(),
- *  requires an allocated ZSTD_DCtx.
- *  Compatible with sticky parameters.
- */
-ZSTDLIB_API size_t ZSTD_decompressDCtx(ZSTD_DCtx* dctx,
-                                       void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize);
-
-
-/***************************************
-*  Advanced compression API
-***************************************/
-
-/* API design :
- *   Parameters are pushed one by one into an existing context,
- *   using ZSTD_CCtx_set*() functions.
- *   Pushed parameters are sticky : they are valid for next compressed frame, and any subsequent frame.
- *   "sticky" parameters are applicable to `ZSTD_compress2()` and `ZSTD_compressStream*()` !
- *   They do not apply to "simple" one-shot variants such as ZSTD_compressCCtx()
- *
- *   It's possible to reset all parameters to "default" using ZSTD_CCtx_reset().
- *
- *   This API supercedes all other "advanced" API entry points in the experimental section.
- *   In the future, we expect to remove from experimental API entry points which are redundant with this API.
- */
-
-
-/* Compression strategies, listed from fastest to strongest */
-typedef enum { ZSTD_fast=1,
-               ZSTD_dfast=2,
-               ZSTD_greedy=3,
-               ZSTD_lazy=4,
-               ZSTD_lazy2=5,
-               ZSTD_btlazy2=6,
-               ZSTD_btopt=7,
-               ZSTD_btultra=8,
-               ZSTD_btultra2=9
-               /* note : new strategies _might_ be added in the future.
-                         Only the order (from fast to strong) is guaranteed */
-} ZSTD_strategy;
-
-
-typedef enum {
-
-    /* compression parameters
-     * Note: When compressing with a ZSTD_CDict these parameters are superseded
-     * by the parameters used to construct the ZSTD_CDict. See ZSTD_CCtx_refCDict()
-     * for more info (superseded-by-cdict). */
-    ZSTD_c_compressionLevel=100, /* Update all compression parameters according to pre-defined cLevel table
-                              * Default level is ZSTD_CLEVEL_DEFAULT==3.
-                              * Special: value 0 means default, which is controlled by ZSTD_CLEVEL_DEFAULT.
-                              * Note 1 : it's possible to pass a negative compression level.
-                              * Note 2 : setting a level sets all default values of other compression parameters */
-    ZSTD_c_windowLog=101,    /* Maximum allowed back-reference distance, expressed as power of 2.
-                              * Must be clamped between ZSTD_WINDOWLOG_MIN and ZSTD_WINDOWLOG_MAX.
-                              * Special: value 0 means "use default windowLog".
-                              * Note: Using a windowLog greater than ZSTD_WINDOWLOG_LIMIT_DEFAULT
-                              *       requires explicitly allowing such window size at decompression stage if using streaming. */
-    ZSTD_c_hashLog=102,      /* Size of the initial probe table, as a power of 2.
-                              * Resulting memory usage is (1 << (hashLog+2)).
-                              * Must be clamped between ZSTD_HASHLOG_MIN and ZSTD_HASHLOG_MAX.
-                              * Larger tables improve compression ratio of strategies <= dFast,
-                              * and improve speed of strategies > dFast.
-                              * Special: value 0 means "use default hashLog". */
-    ZSTD_c_chainLog=103,     /* Size of the multi-probe search table, as a power of 2.
-                              * Resulting memory usage is (1 << (chainLog+2)).
-                              * Must be clamped between ZSTD_CHAINLOG_MIN and ZSTD_CHAINLOG_MAX.
-                              * Larger tables result in better and slower compression.
-                              * This parameter is useless when using "fast" strategy.
-                              * It's still useful when using "dfast" strategy,
-                              * in which case it defines a secondary probe table.
-                              * Special: value 0 means "use default chainLog". */
-    ZSTD_c_searchLog=104,    /* Number of search attempts, as a power of 2.
-                              * More attempts result in better and slower compression.
-                              * This parameter is useless when using "fast" and "dFast" strategies.
-                              * Special: value 0 means "use default searchLog". */
-    ZSTD_c_minMatch=105,     /* Minimum size of searched matches.
-                              * Note that Zstandard can still find matches of smaller size,
-                              * it just tweaks its search algorithm to look for this size and larger.
-                              * Larger values increase compression and decompression speed, but decrease ratio.
-                              * Must be clamped between ZSTD_MINMATCH_MIN and ZSTD_MINMATCH_MAX.
-                              * Note that currently, for all strategies < btopt, effective minimum is 4.
-                              *                    , for all strategies > fast, effective maximum is 6.
-                              * Special: value 0 means "use default minMatchLength". */
-    ZSTD_c_targetLength=106, /* Impact of this field depends on strategy.
-                              * For strategies btopt, btultra & btultra2:
-                              *     Length of Match considered "good enough" to stop search.
-                              *     Larger values make compression stronger, and slower.
-                              * For strategy fast:
-                              *     Distance between match sampling.
-                              *     Larger values make compression faster, and weaker.
-                              * Special: value 0 means "use default targetLength". */
-    ZSTD_c_strategy=107,     /* See ZSTD_strategy enum definition.
-                              * The higher the value of selected strategy, the more complex it is,
-                              * resulting in stronger and slower compression.
-                              * Special: value 0 means "use default strategy". */
-
-    /* LDM mode parameters */
-    ZSTD_c_enableLongDistanceMatching=160, /* Enable long distance matching.
-                                     * This parameter is designed to improve compression ratio
-                                     * for large inputs, by finding large matches at long distance.
-                                     * It increases memory usage and window size.
-                                     * Note: enabling this parameter increases default ZSTD_c_windowLog to 128 MB
-                                     * except when expressly set to a different value. */
-    ZSTD_c_ldmHashLog=161,   /* Size of the table for long distance matching, as a power of 2.
-                              * Larger values increase memory usage and compression ratio,
-                              * but decrease compression speed.
-                              * Must be clamped between ZSTD_HASHLOG_MIN and ZSTD_HASHLOG_MAX
-                              * default: windowlog - 7.
-                              * Special: value 0 means "automatically determine hashlog". */
-    ZSTD_c_ldmMinMatch=162,  /* Minimum match size for long distance matcher.
-                              * Larger/too small values usually decrease compression ratio.
-                              * Must be clamped between ZSTD_LDM_MINMATCH_MIN and ZSTD_LDM_MINMATCH_MAX.
-                              * Special: value 0 means "use default value" (default: 64). */
-    ZSTD_c_ldmBucketSizeLog=163, /* Log size of each bucket in the LDM hash table for collision resolution.
-                              * Larger values improve collision resolution but decrease compression speed.
-                              * The maximum value is ZSTD_LDM_BUCKETSIZELOG_MAX.
-                              * Special: value 0 means "use default value" (default: 3). */
-    ZSTD_c_ldmHashRateLog=164, /* Frequency of inserting/looking up entries into the LDM hash table.
-                              * Must be clamped between 0 and (ZSTD_WINDOWLOG_MAX - ZSTD_HASHLOG_MIN).
-                              * Default is MAX(0, (windowLog - ldmHashLog)), optimizing hash table usage.
-                              * Larger values improve compression speed.
-                              * Deviating far from default value will likely result in a compression ratio decrease.
-                              * Special: value 0 means "automatically determine hashRateLog". */
-
-    /* frame parameters */
-    ZSTD_c_contentSizeFlag=200, /* Content size will be written into frame header _whenever known_ (default:1)
-                              * Content size must be known at the beginning of compression.
-                              * This is automatically the case when using ZSTD_compress2(),
-                              * For streaming variants, content size must be provided with ZSTD_CCtx_setPledgedSrcSize() */
-    ZSTD_c_checksumFlag=201, /* A 32-bits checksum of content is written at end of frame (default:0) */
-    ZSTD_c_dictIDFlag=202,   /* When applicable, dictionary's ID is written into frame header (default:1) */
-
-    /* multi-threading parameters */
-    /* These parameters are only useful if multi-threading is enabled (compiled with build macro ZSTD_MULTITHREAD).
-     * They return an error otherwise. */
-    ZSTD_c_nbWorkers=400,    /* Select how many threads will be spawned to compress in parallel.
-                              * When nbWorkers >= 1, triggers asynchronous mode when used with ZSTD_compressStream*() :
-                              * ZSTD_compressStream*() consumes input and flush output if possible, but immediately gives back control to caller,
-                              * while compression work is performed in parallel, within worker threads.
-                              * (note : a strong exception to this rule is when first invocation of ZSTD_compressStream2() sets ZSTD_e_end :
-                              *  in which case, ZSTD_compressStream2() delegates to ZSTD_compress2(), which is always a blocking call).
-                              * More workers improve speed, but also increase memory usage.
-                              * Default value is `0`, aka "single-threaded mode" : no worker is spawned, compression is performed inside Caller's thread, all invocations are blocking */
-    ZSTD_c_jobSize=401,      /* Size of a compression job. This value is enforced only when nbWorkers >= 1.
-                              * Each compression job is completed in parallel, so this value can indirectly impact the nb of active threads.
-                              * 0 means default, which is dynamically determined based on compression parameters.
-                              * Job size must be a minimum of overlap size, or 1 MB, whichever is largest.
-                              * The minimum size is automatically and transparently enforced */
-    ZSTD_c_overlapLog=402,   /* Control the overlap size, as a fraction of window size.
-                              * The overlap size is an amount of data reloaded from previous job at the beginning of a new job.
-                              * It helps preserve compression ratio, while each job is compressed in parallel.
-                              * This value is enforced only when nbWorkers >= 1.
-                              * Larger values increase compression ratio, but decrease speed.
-                              * Possible values range from 0 to 9 :
-                              * - 0 means "default" : value will be determined by the library, depending on strategy
-                              * - 1 means "no overlap"
-                              * - 9 means "full overlap", using a full window size.
-                              * Each intermediate rank increases/decreases load size by a factor 2 :
-                              * 9: full window;  8: w/2;  7: w/4;  6: w/8;  5:w/16;  4: w/32;  3:w/64;  2:w/128;  1:no overlap;  0:default
-                              * default value varies between 6 and 9, depending on strategy */
-
-    /* note : additional experimental parameters are also available
-     * within the experimental section of the API.
-     * At the time of this writing, they include :
-     * ZSTD_c_rsyncable
-     * ZSTD_c_format
-     * ZSTD_c_forceMaxWindow
-     * ZSTD_c_forceAttachDict
-     * ZSTD_c_literalCompressionMode
-     * ZSTD_c_targetCBlockSize
-     * Because they are not stable, it's necessary to define ZSTD_STATIC_LINKING_ONLY to access them.
-     * note : never ever use experimentalParam? names directly;
-     *        also, the enums values themselves are unstable and can still change.
-     */
-     ZSTD_c_experimentalParam1=500,
-     ZSTD_c_experimentalParam2=10,
-     ZSTD_c_experimentalParam3=1000,
-     ZSTD_c_experimentalParam4=1001,
-     ZSTD_c_experimentalParam5=1002,
-     ZSTD_c_experimentalParam6=1003,
-} ZSTD_cParameter;
-
-typedef struct {
-    size_t error;
-    int lowerBound;
-    int upperBound;
-} ZSTD_bounds;
-
-/*! ZSTD_cParam_getBounds() :
- *  All parameters must belong to an interval with lower and upper bounds,
- *  otherwise they will either trigger an error or be automatically clamped.
- * @return : a structure, ZSTD_bounds, which contains
- *         - an error status field, which must be tested using ZSTD_isError()
- *         - lower and upper bounds, both inclusive
- */
-ZSTDLIB_API ZSTD_bounds ZSTD_cParam_getBounds(ZSTD_cParameter cParam);
-
-/*! ZSTD_CCtx_setParameter() :
- *  Set one compression parameter, selected by enum ZSTD_cParameter.
- *  All parameters have valid bounds. Bounds can be queried using ZSTD_cParam_getBounds().
- *  Providing a value beyond bound will either clamp it, or trigger an error (depending on parameter).
- *  Setting a parameter is generally only possible during frame initialization (before starting compression).
- *  Exception : when using multi-threading mode (nbWorkers >= 1),
- *              the following parameters can be updated _during_ compression (within same frame):
- *              => compressionLevel, hashLog, chainLog, searchLog, minMatch, targetLength and strategy.
- *              new parameters will be active for next job only (after a flush()).
- * @return : an error code (which can be tested using ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_CCtx_setParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, int value);
-
-/*! ZSTD_CCtx_setPledgedSrcSize() :
- *  Total input data size to be compressed as a single frame.
- *  Value will be written in frame header, unless if explicitly forbidden using ZSTD_c_contentSizeFlag.
- *  This value will also be controlled at end of frame, and trigger an error if not respected.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Note 1 : pledgedSrcSize==0 actually means zero, aka an empty frame.
- *           In order to mean "unknown content size", pass constant ZSTD_CONTENTSIZE_UNKNOWN.
- *           ZSTD_CONTENTSIZE_UNKNOWN is default value for any new frame.
- *  Note 2 : pledgedSrcSize is only valid once, for the next frame.
- *           It's discarded at the end of the frame, and replaced by ZSTD_CONTENTSIZE_UNKNOWN.
- *  Note 3 : Whenever all input data is provided and consumed in a single round,
- *           for example with ZSTD_compress2(),
- *           or invoking immediately ZSTD_compressStream2(,,,ZSTD_e_end),
- *           this value is automatically overridden by srcSize instead.
- */
-ZSTDLIB_API size_t ZSTD_CCtx_setPledgedSrcSize(ZSTD_CCtx* cctx, unsigned long long pledgedSrcSize);
-
-typedef enum {
-    ZSTD_reset_session_only = 1,
-    ZSTD_reset_parameters = 2,
-    ZSTD_reset_session_and_parameters = 3
-} ZSTD_ResetDirective;
-
-/*! ZSTD_CCtx_reset() :
- *  There are 2 different things that can be reset, independently or jointly :
- *  - The session : will stop compressing current frame, and make CCtx ready to start a new one.
- *                  Useful after an error, or to interrupt any ongoing compression.
- *                  Any internal data not yet flushed is cancelled.
- *                  Compression parameters and dictionary remain unchanged.
- *                  They will be used to compress next frame.
- *                  Resetting session never fails.
- *  - The parameters : changes all parameters back to "default".
- *                  This removes any reference to any dictionary too.
- *                  Parameters can only be changed between 2 sessions (i.e. no compression is currently ongoing)
- *                  otherwise the reset fails, and function returns an error value (which can be tested using ZSTD_isError())
- *  - Both : similar to resetting the session, followed by resetting parameters.
- */
-ZSTDLIB_API size_t ZSTD_CCtx_reset(ZSTD_CCtx* cctx, ZSTD_ResetDirective reset);
-
-/*! ZSTD_compress2() :
- *  Behave the same as ZSTD_compressCCtx(), but compression parameters are set using the advanced API.
- *  ZSTD_compress2() always starts a new frame.
- *  Should cctx hold data from a previously unfinished frame, everything about it is forgotten.
- *  - Compression parameters are pushed into CCtx before starting compression, using ZSTD_CCtx_set*()
- *  - The function is always blocking, returns when compression is completed.
- *  Hint : compression runs faster if `dstCapacity` >=  `ZSTD_compressBound(srcSize)`.
- * @return : compressed size written into `dst` (<= `dstCapacity),
- *           or an error code if it fails (which can be tested using ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_compress2( ZSTD_CCtx* cctx,
-                                   void* dst, size_t dstCapacity,
-                             const void* src, size_t srcSize);
-
-
-/***************************************
-*  Advanced decompression API
-***************************************/
-
-/* The advanced API pushes parameters one by one into an existing DCtx context.
- * Parameters are sticky, and remain valid for all following frames
- * using the same DCtx context.
- * It's possible to reset parameters to default values using ZSTD_DCtx_reset().
- * Note : This API is compatible with existing ZSTD_decompressDCtx() and ZSTD_decompressStream().
- *        Therefore, no new decompression function is necessary.
- */
-
-typedef enum {
-
-    ZSTD_d_windowLogMax=100, /* Select a size limit (in power of 2) beyond which
-                              * the streaming API will refuse to allocate memory buffer
-                              * in order to protect the host from unreasonable memory requirements.
-                              * This parameter is only useful in streaming mode, since no internal buffer is allocated in single-pass mode.
-                              * By default, a decompression context accepts window sizes <= (1 << ZSTD_WINDOWLOG_LIMIT_DEFAULT).
-                              * Special: value 0 means "use default maximum windowLog". */
-
-    /* note : additional experimental parameters are also available
-     * within the experimental section of the API.
-     * At the time of this writing, they include :
-     * ZSTD_c_format
-     * Because they are not stable, it's necessary to define ZSTD_STATIC_LINKING_ONLY to access them.
-     * note : never ever use experimentalParam? names directly
-     */
-     ZSTD_d_experimentalParam1=1000
-
-} ZSTD_dParameter;
-
-/*! ZSTD_dParam_getBounds() :
- *  All parameters must belong to an interval with lower and upper bounds,
- *  otherwise they will either trigger an error or be automatically clamped.
- * @return : a structure, ZSTD_bounds, which contains
- *         - an error status field, which must be tested using ZSTD_isError()
- *         - both lower and upper bounds, inclusive
- */
-ZSTDLIB_API ZSTD_bounds ZSTD_dParam_getBounds(ZSTD_dParameter dParam);
-
-/*! ZSTD_DCtx_setParameter() :
- *  Set one compression parameter, selected by enum ZSTD_dParameter.
- *  All parameters have valid bounds. Bounds can be queried using ZSTD_dParam_getBounds().
- *  Providing a value beyond bound will either clamp it, or trigger an error (depending on parameter).
- *  Setting a parameter is only possible during frame initialization (before starting decompression).
- * @return : 0, or an error code (which can be tested using ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_DCtx_setParameter(ZSTD_DCtx* dctx, ZSTD_dParameter param, int value);
-
-/*! ZSTD_DCtx_reset() :
- *  Return a DCtx to clean state.
- *  Session and parameters can be reset jointly or separately.
- *  Parameters can only be reset when no active frame is being decompressed.
- * @return : 0, or an error code, which can be tested with ZSTD_isError()
- */
-ZSTDLIB_API size_t ZSTD_DCtx_reset(ZSTD_DCtx* dctx, ZSTD_ResetDirective reset);
-
-
-/****************************
-*  Streaming
-****************************/
-
-typedef struct ZSTD_inBuffer_s {
-  const void* src;    /**< start of input buffer */
-  size_t size;        /**< size of input buffer */
-  size_t pos;         /**< position where reading stopped. Will be updated. Necessarily 0 <= pos <= size */
-} ZSTD_inBuffer;
-
-typedef struct ZSTD_outBuffer_s {
-  void*  dst;         /**< start of output buffer */
-  size_t size;        /**< size of output buffer */
-  size_t pos;         /**< position where writing stopped. Will be updated. Necessarily 0 <= pos <= size */
-} ZSTD_outBuffer;
-
-
-
-/*-***********************************************************************
-*  Streaming compression - HowTo
-*
-*  A ZSTD_CStream object is required to track streaming operation.
-*  Use ZSTD_createCStream() and ZSTD_freeCStream() to create/release resources.
-*  ZSTD_CStream objects can be reused multiple times on consecutive compression operations.
-*  It is recommended to re-use ZSTD_CStream since it will play nicer with system's memory, by re-using already allocated memory.
-*
-*  For parallel execution, use one separate ZSTD_CStream per thread.
-*
-*  note : since v1.3.0, ZSTD_CStream and ZSTD_CCtx are the same thing.
-*
-*  Parameters are sticky : when starting a new compression on the same context,
-*  it will re-use the same sticky parameters as previous compression session.
-*  When in doubt, it's recommended to fully initialize the context before usage.
-*  Use ZSTD_CCtx_reset() to reset the context and ZSTD_CCtx_setParameter(),
-*  ZSTD_CCtx_setPledgedSrcSize(), or ZSTD_CCtx_loadDictionary() and friends to
-*  set more specific parameters, the pledged source size, or load a dictionary.
-*
-*  Use ZSTD_compressStream2() with ZSTD_e_continue as many times as necessary to
-*  consume input stream. The function will automatically update both `pos`
-*  fields within `input` and `output`.
-*  Note that the function may not consume the entire input, for example, because
-*  the output buffer is already full, in which case `input.pos < input.size`.
-*  The caller must check if input has been entirely consumed.
-*  If not, the caller must make some room to receive more compressed data,
-*  and then present again remaining input data.
-*  note: ZSTD_e_continue is guaranteed to make some forward progress when called,
-*        but doesn't guarantee maximal forward progress. This is especially relevant
-*        when compressing with multiple threads. The call won't block if it can
-*        consume some input, but if it can't it will wait for some, but not all,
-*        output to be flushed.
-* @return : provides a minimum amount of data remaining to be flushed from internal buffers
-*           or an error code, which can be tested using ZSTD_isError().
-*
-*  At any moment, it's possible to flush whatever data might remain stuck within internal buffer,
-*  using ZSTD_compressStream2() with ZSTD_e_flush. `output->pos` will be updated.
-*  Note that, if `output->size` is too small, a single invocation with ZSTD_e_flush might not be enough (return code > 0).
-*  In which case, make some room to receive more compressed data, and call again ZSTD_compressStream2() with ZSTD_e_flush.
-*  You must continue calling ZSTD_compressStream2() with ZSTD_e_flush until it returns 0, at which point you can change the
-*  operation.
-*  note: ZSTD_e_flush will flush as much output as possible, meaning when compressing with multiple threads, it will
-*        block until the flush is complete or the output buffer is full.
-*  @return : 0 if internal buffers are entirely flushed,
-*            >0 if some data still present within internal buffer (the value is minimal estimation of remaining size),
-*            or an error code, which can be tested using ZSTD_isError().
-*
-*  Calling ZSTD_compressStream2() with ZSTD_e_end instructs to finish a frame.
-*  It will perform a flush and write frame epilogue.
-*  The epilogue is required for decoders to consider a frame completed.
-*  flush operation is the same, and follows same rules as calling ZSTD_compressStream2() with ZSTD_e_flush.
-*  You must continue calling ZSTD_compressStream2() with ZSTD_e_end until it returns 0, at which point you are free to
-*  start a new frame.
-*  note: ZSTD_e_end will flush as much output as possible, meaning when compressing with multiple threads, it will
-*        block until the flush is complete or the output buffer is full.
-*  @return : 0 if frame fully completed and fully flushed,
-*            >0 if some data still present within internal buffer (the value is minimal estimation of remaining size),
-*            or an error code, which can be tested using ZSTD_isError().
-*
-* *******************************************************************/
-
-typedef ZSTD_CCtx ZSTD_CStream;  /**< CCtx and CStream are now effectively same object (>= v1.3.0) */
-                                 /* Continue to distinguish them for compatibility with older versions <= v1.2.0 */
-/*===== ZSTD_CStream management functions =====*/
-ZSTDLIB_API ZSTD_CStream* ZSTD_createCStream(void);
-ZSTDLIB_API size_t ZSTD_freeCStream(ZSTD_CStream* zcs);
-
-/*===== Streaming compression functions =====*/
-typedef enum {
-    ZSTD_e_continue=0, /* collect more data, encoder decides when to output compressed result, for optimal compression ratio */
-    ZSTD_e_flush=1,    /* flush any data provided so far,
-                        * it creates (at least) one new block, that can be decoded immediately on reception;
-                        * frame will continue: any future data can still reference previously compressed data, improving compression.
-                        * note : multithreaded compression will block to flush as much output as possible. */
-    ZSTD_e_end=2       /* flush any remaining data _and_ close current frame.
-                        * note that frame is only closed after compressed data is fully flushed (return value == 0).
-                        * After that point, any additional data starts a new frame.
-                        * note : each frame is independent (does not reference any content from previous frame).
-                        : note : multithreaded compression will block to flush as much output as possible. */
-} ZSTD_EndDirective;
-
-/*! ZSTD_compressStream2() :
- *  Behaves about the same as ZSTD_compressStream, with additional control on end directive.
- *  - Compression parameters are pushed into CCtx before starting compression, using ZSTD_CCtx_set*()
- *  - Compression parameters cannot be changed once compression is started (save a list of exceptions in multi-threading mode)
- *  - output->pos must be <= dstCapacity, input->pos must be <= srcSize
- *  - output->pos and input->pos will be updated. They are guaranteed to remain below their respective limit.
- *  - When nbWorkers==0 (default), function is blocking : it completes its job before returning to caller.
- *  - When nbWorkers>=1, function is non-blocking : it just acquires a copy of input, and distributes jobs to internal worker threads, flush whatever is available,
- *                                                  and then immediately returns, just indicating that there is some data remaining to be flushed.
- *                                                  The function nonetheless guarantees forward progress : it will return only after it reads or write at least 1+ byte.
- *  - Exception : if the first call requests a ZSTD_e_end directive and provides enough dstCapacity, the function delegates to ZSTD_compress2() which is always blocking.
- *  - @return provides a minimum amount of data remaining to be flushed from internal buffers
- *            or an error code, which can be tested using ZSTD_isError().
- *            if @return != 0, flush is not fully completed, there is still some data left within internal buffers.
- *            This is useful for ZSTD_e_flush, since in this case more flushes are necessary to empty all buffers.
- *            For ZSTD_e_end, @return == 0 when internal buffers are fully flushed and frame is completed.
- *  - after a ZSTD_e_end directive, if internal buffer is not fully flushed (@return != 0),
- *            only ZSTD_e_end or ZSTD_e_flush operations are allowed.
- *            Before starting a new compression job, or changing compression parameters,
- *            it is required to fully flush internal buffers.
- */
-ZSTDLIB_API size_t ZSTD_compressStream2( ZSTD_CCtx* cctx,
-                                         ZSTD_outBuffer* output,
-                                         ZSTD_inBuffer* input,
-                                         ZSTD_EndDirective endOp);
-
-
-/* These buffer sizes are softly recommended.
- * They are not required : ZSTD_compressStream*() happily accepts any buffer size, for both input and output.
- * Respecting the recommended size just makes it a bit easier for ZSTD_compressStream*(),
- * reducing the amount of memory shuffling and buffering, resulting in minor performance savings.
- *
- * However, note that these recommendations are from the perspective of a C caller program.
- * If the streaming interface is invoked from some other language,
- * especially managed ones such as Java or Go, through a foreign function interface such as jni or cgo,
- * a major performance rule is to reduce crossing such interface to an absolute minimum.
- * It's not rare that performance ends being spent more into the interface, rather than compression itself.
- * In which cases, prefer using large buffers, as large as practical,
- * for both input and output, to reduce the nb of roundtrips.
- */
-ZSTDLIB_API size_t ZSTD_CStreamInSize(void);    /**< recommended size for input buffer */
-ZSTDLIB_API size_t ZSTD_CStreamOutSize(void);   /**< recommended size for output buffer. Guarantee to successfully flush at least one complete compressed block. */
-
-
-/* *****************************************************************************
- * This following is a legacy streaming API.
- * It can be replaced by ZSTD_CCtx_reset() and ZSTD_compressStream2().
- * It is redundant, but remains fully supported.
- * Advanced parameters and dictionary compression can only be used through the
- * new API.
- ******************************************************************************/
-
-/*!
- * Equivalent to:
- *
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_refCDict(zcs, NULL); // clear the dictionary (if any)
- *     ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel);
- */
-ZSTDLIB_API size_t ZSTD_initCStream(ZSTD_CStream* zcs, int compressionLevel);
-/*!
- * Alternative for ZSTD_compressStream2(zcs, output, input, ZSTD_e_continue).
- * NOTE: The return value is different. ZSTD_compressStream() returns a hint for
- * the next read size (if non-zero and not an error). ZSTD_compressStream2()
- * returns the minimum nb of bytes left to flush (if non-zero and not an error).
- */
-ZSTDLIB_API size_t ZSTD_compressStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output, ZSTD_inBuffer* input);
-/*! Equivalent to ZSTD_compressStream2(zcs, output, &emptyInput, ZSTD_e_flush). */
-ZSTDLIB_API size_t ZSTD_flushStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output);
-/*! Equivalent to ZSTD_compressStream2(zcs, output, &emptyInput, ZSTD_e_end). */
-ZSTDLIB_API size_t ZSTD_endStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output);
-
-
-/*-***************************************************************************
-*  Streaming decompression - HowTo
-*
-*  A ZSTD_DStream object is required to track streaming operations.
-*  Use ZSTD_createDStream() and ZSTD_freeDStream() to create/release resources.
-*  ZSTD_DStream objects can be re-used multiple times.
-*
-*  Use ZSTD_initDStream() to start a new decompression operation.
-* @return : recommended first input size
-*  Alternatively, use advanced API to set specific properties.
-*
-*  Use ZSTD_decompressStream() repetitively to consume your input.
-*  The function will update both `pos` fields.
-*  If `input.pos < input.size`, some input has not been consumed.
-*  It's up to the caller to present again remaining data.
-*  The function tries to flush all data decoded immediately, respecting output buffer size.
-*  If `output.pos < output.size`, decoder has flushed everything it could.
-*  But if `output.pos == output.size`, there might be some data left within internal buffers.,
-*  In which case, call ZSTD_decompressStream() again to flush whatever remains in the buffer.
-*  Note : with no additional input provided, amount of data flushed is necessarily <= ZSTD_BLOCKSIZE_MAX.
-* @return : 0 when a frame is completely decoded and fully flushed,
-*        or an error code, which can be tested using ZSTD_isError(),
-*        or any other value > 0, which means there is still some decoding or flushing to do to complete current frame :
-*                                the return value is a suggested next input size (just a hint for better latency)
-*                                that will never request more than the remaining frame size.
-* *******************************************************************************/
-
-typedef ZSTD_DCtx ZSTD_DStream;  /**< DCtx and DStream are now effectively same object (>= v1.3.0) */
-                                 /* For compatibility with versions <= v1.2.0, prefer differentiating them. */
-/*===== ZSTD_DStream management functions =====*/
-ZSTDLIB_API ZSTD_DStream* ZSTD_createDStream(void);
-ZSTDLIB_API size_t ZSTD_freeDStream(ZSTD_DStream* zds);
-
-/*===== Streaming decompression functions =====*/
-
-/* This function is redundant with the advanced API and equivalent to:
- *
- *     ZSTD_DCtx_reset(zds);
- *     ZSTD_DCtx_refDDict(zds, NULL);
- */
-ZSTDLIB_API size_t ZSTD_initDStream(ZSTD_DStream* zds);
-
-ZSTDLIB_API size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inBuffer* input);
-
-ZSTDLIB_API size_t ZSTD_DStreamInSize(void);    /*!< recommended size for input buffer */
-ZSTDLIB_API size_t ZSTD_DStreamOutSize(void);   /*!< recommended size for output buffer. Guarantee to successfully flush at least one complete block in all circumstances. */
-
-
-/**************************
-*  Simple dictionary API
-***************************/
-/*! ZSTD_compress_usingDict() :
- *  Compression at an explicit compression level using a Dictionary.
- *  A dictionary can be any arbitrary data segment (also called a prefix),
- *  or a buffer with specified information (see dictBuilder/zdict.h).
- *  Note : This function loads the dictionary, resulting in significant startup delay.
- *         It's intended for a dictionary used only once.
- *  Note 2 : When `dict == NULL || dictSize < 8` no dictionary is used. */
-ZSTDLIB_API size_t ZSTD_compress_usingDict(ZSTD_CCtx* ctx,
-                                           void* dst, size_t dstCapacity,
-                                     const void* src, size_t srcSize,
-                                     const void* dict,size_t dictSize,
-                                           int compressionLevel);
-
-/*! ZSTD_decompress_usingDict() :
- *  Decompression using a known Dictionary.
- *  Dictionary must be identical to the one used during compression.
- *  Note : This function loads the dictionary, resulting in significant startup delay.
- *         It's intended for a dictionary used only once.
- *  Note : When `dict == NULL || dictSize < 8` no dictionary is used. */
-ZSTDLIB_API size_t ZSTD_decompress_usingDict(ZSTD_DCtx* dctx,
-                                             void* dst, size_t dstCapacity,
-                                       const void* src, size_t srcSize,
-                                       const void* dict,size_t dictSize);
-
-
-/***********************************
- *  Bulk processing dictionary API
- **********************************/
-typedef struct ZSTD_CDict_s ZSTD_CDict;
-
-/*! ZSTD_createCDict() :
- *  When compressing multiple messages / blocks using the same dictionary, it's recommended to load it only once.
- *  ZSTD_createCDict() will create a digested dictionary, ready to start future compression operations without startup cost.
- *  ZSTD_CDict can be created once and shared by multiple threads concurrently, since its usage is read-only.
- * `dictBuffer` can be released after ZSTD_CDict creation, because its content is copied within CDict.
- *  Consider experimental function `ZSTD_createCDict_byReference()` if you prefer to not duplicate `dictBuffer` content.
- *  Note : A ZSTD_CDict can be created from an empty dictBuffer, but it is inefficient when used to compress small data. */
-ZSTDLIB_API ZSTD_CDict* ZSTD_createCDict(const void* dictBuffer, size_t dictSize,
-                                         int compressionLevel);
-
-/*! ZSTD_freeCDict() :
- *  Function frees memory allocated by ZSTD_createCDict(). */
-ZSTDLIB_API size_t      ZSTD_freeCDict(ZSTD_CDict* CDict);
-
-/*! ZSTD_compress_usingCDict() :
- *  Compression using a digested Dictionary.
- *  Recommended when same dictionary is used multiple times.
- *  Note : compression level is _decided at dictionary creation time_,
- *     and frame parameters are hardcoded (dictID=yes, contentSize=yes, checksum=no) */
-ZSTDLIB_API size_t ZSTD_compress_usingCDict(ZSTD_CCtx* cctx,
-                                            void* dst, size_t dstCapacity,
-                                      const void* src, size_t srcSize,
-                                      const ZSTD_CDict* cdict);
-
-
-typedef struct ZSTD_DDict_s ZSTD_DDict;
-
-/*! ZSTD_createDDict() :
- *  Create a digested dictionary, ready to start decompression operation without startup delay.
- *  dictBuffer can be released after DDict creation, as its content is copied inside DDict. */
-ZSTDLIB_API ZSTD_DDict* ZSTD_createDDict(const void* dictBuffer, size_t dictSize);
-
-/*! ZSTD_freeDDict() :
- *  Function frees memory allocated with ZSTD_createDDict() */
-ZSTDLIB_API size_t      ZSTD_freeDDict(ZSTD_DDict* ddict);
-
-/*! ZSTD_decompress_usingDDict() :
- *  Decompression using a digested Dictionary.
- *  Recommended when same dictionary is used multiple times. */
-ZSTDLIB_API size_t ZSTD_decompress_usingDDict(ZSTD_DCtx* dctx,
-                                              void* dst, size_t dstCapacity,
-                                        const void* src, size_t srcSize,
-                                        const ZSTD_DDict* ddict);
-
-
-/********************************
- *  Dictionary helper functions
- *******************************/
-
-/*! ZSTD_getDictID_fromDict() :
- *  Provides the dictID stored within dictionary.
- *  if @return == 0, the dictionary is not conformant with Zstandard specification.
- *  It can still be loaded, but as a content-only dictionary. */
-ZSTDLIB_API unsigned ZSTD_getDictID_fromDict(const void* dict, size_t dictSize);
-
-/*! ZSTD_getDictID_fromDDict() :
- *  Provides the dictID of the dictionary loaded into `ddict`.
- *  If @return == 0, the dictionary is not conformant to Zstandard specification, or empty.
- *  Non-conformant dictionaries can still be loaded, but as content-only dictionaries. */
-ZSTDLIB_API unsigned ZSTD_getDictID_fromDDict(const ZSTD_DDict* ddict);
-
-/*! ZSTD_getDictID_fromFrame() :
- *  Provides the dictID required to decompressed the frame stored within `src`.
- *  If @return == 0, the dictID could not be decoded.
- *  This could for one of the following reasons :
- *  - The frame does not require a dictionary to be decoded (most common case).
- *  - The frame was built with dictID intentionally removed. Whatever dictionary is necessary is a hidden information.
- *    Note : this use case also happens when using a non-conformant dictionary.
- *  - `srcSize` is too small, and as a result, the frame header could not be decoded (only possible if `srcSize < ZSTD_FRAMEHEADERSIZE_MAX`).
- *  - This is not a Zstandard frame.
- *  When identifying the exact failure cause, it's possible to use ZSTD_getFrameHeader(), which will provide a more precise error code. */
-ZSTDLIB_API unsigned ZSTD_getDictID_fromFrame(const void* src, size_t srcSize);
-
-
-/*******************************************************************************
- * Advanced dictionary and prefix API
- *
- * This API allows dictionaries to be used with ZSTD_compress2(),
- * ZSTD_compressStream2(), and ZSTD_decompress(). Dictionaries are sticky, and
- * only reset with the context is reset with ZSTD_reset_parameters or
- * ZSTD_reset_session_and_parameters. Prefixes are single-use.
- ******************************************************************************/
-
-
-/*! ZSTD_CCtx_loadDictionary() :
- *  Create an internal CDict from `dict` buffer.
- *  Decompression will have to use same dictionary.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Special: Loading a NULL (or 0-size) dictionary invalidates previous dictionary,
- *           meaning "return to no-dictionary mode".
- *  Note 1 : Dictionary is sticky, it will be used for all future compressed frames.
- *           To return to "no-dictionary" situation, load a NULL dictionary (or reset parameters).
- *  Note 2 : Loading a dictionary involves building tables.
- *           It's also a CPU consuming operation, with non-negligible impact on latency.
- *           Tables are dependent on compression parameters, and for this reason,
- *           compression parameters can no longer be changed after loading a dictionary.
- *  Note 3 :`dict` content will be copied internally.
- *           Use experimental ZSTD_CCtx_loadDictionary_byReference() to reference content instead.
- *           In such a case, dictionary buffer must outlive its users.
- *  Note 4 : Use ZSTD_CCtx_loadDictionary_advanced()
- *           to precisely select how dictionary content must be interpreted. */
-ZSTDLIB_API size_t ZSTD_CCtx_loadDictionary(ZSTD_CCtx* cctx, const void* dict, size_t dictSize);
-
-/*! ZSTD_CCtx_refCDict() :
- *  Reference a prepared dictionary, to be used for all next compressed frames.
- *  Note that compression parameters are enforced from within CDict,
- *  and supersede any compression parameter previously set within CCtx.
- *  The parameters ignored are labled as "superseded-by-cdict" in the ZSTD_cParameter enum docs.
- *  The ignored parameters will be used again if the CCtx is returned to no-dictionary mode.
- *  The dictionary will remain valid for future compressed frames using same CCtx.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Special : Referencing a NULL CDict means "return to no-dictionary mode".
- *  Note 1 : Currently, only one dictionary can be managed.
- *           Referencing a new dictionary effectively "discards" any previous one.
- *  Note 2 : CDict is just referenced, its lifetime must outlive its usage within CCtx. */
-ZSTDLIB_API size_t ZSTD_CCtx_refCDict(ZSTD_CCtx* cctx, const ZSTD_CDict* cdict);
-
-/*! ZSTD_CCtx_refPrefix() :
- *  Reference a prefix (single-usage dictionary) for next compressed frame.
- *  A prefix is **only used once**. Tables are discarded at end of frame (ZSTD_e_end).
- *  Decompression will need same prefix to properly regenerate data.
- *  Compressing with a prefix is similar in outcome as performing a diff and compressing it,
- *  but performs much faster, especially during decompression (compression speed is tunable with compression level).
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Special: Adding any prefix (including NULL) invalidates any previous prefix or dictionary
- *  Note 1 : Prefix buffer is referenced. It **must** outlive compression.
- *           Its content must remain unmodified during compression.
- *  Note 2 : If the intention is to diff some large src data blob with some prior version of itself,
- *           ensure that the window size is large enough to contain the entire source.
- *           See ZSTD_c_windowLog.
- *  Note 3 : Referencing a prefix involves building tables, which are dependent on compression parameters.
- *           It's a CPU consuming operation, with non-negligible impact on latency.
- *           If there is a need to use the same prefix multiple times, consider loadDictionary instead.
- *  Note 4 : By default, the prefix is interpreted as raw content (ZSTD_dm_rawContent).
- *           Use experimental ZSTD_CCtx_refPrefix_advanced() to alter dictionary interpretation. */
-ZSTDLIB_API size_t ZSTD_CCtx_refPrefix(ZSTD_CCtx* cctx,
-                                 const void* prefix, size_t prefixSize);
-
-/*! ZSTD_DCtx_loadDictionary() :
- *  Create an internal DDict from dict buffer,
- *  to be used to decompress next frames.
- *  The dictionary remains valid for all future frames, until explicitly invalidated.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Special : Adding a NULL (or 0-size) dictionary invalidates any previous dictionary,
- *            meaning "return to no-dictionary mode".
- *  Note 1 : Loading a dictionary involves building tables,
- *           which has a non-negligible impact on CPU usage and latency.
- *           It's recommended to "load once, use many times", to amortize the cost
- *  Note 2 :`dict` content will be copied internally, so `dict` can be released after loading.
- *           Use ZSTD_DCtx_loadDictionary_byReference() to reference dictionary content instead.
- *  Note 3 : Use ZSTD_DCtx_loadDictionary_advanced() to take control of
- *           how dictionary content is loaded and interpreted.
- */
-ZSTDLIB_API size_t ZSTD_DCtx_loadDictionary(ZSTD_DCtx* dctx, const void* dict, size_t dictSize);
-
-/*! ZSTD_DCtx_refDDict() :
- *  Reference a prepared dictionary, to be used to decompress next frames.
- *  The dictionary remains active for decompression of future frames using same DCtx.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Note 1 : Currently, only one dictionary can be managed.
- *           Referencing a new dictionary effectively "discards" any previous one.
- *  Special: referencing a NULL DDict means "return to no-dictionary mode".
- *  Note 2 : DDict is just referenced, its lifetime must outlive its usage from DCtx.
- */
-ZSTDLIB_API size_t ZSTD_DCtx_refDDict(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict);
-
-/*! ZSTD_DCtx_refPrefix() :
- *  Reference a prefix (single-usage dictionary) to decompress next frame.
- *  This is the reverse operation of ZSTD_CCtx_refPrefix(),
- *  and must use the same prefix as the one used during compression.
- *  Prefix is **only used once**. Reference is discarded at end of frame.
- *  End of frame is reached when ZSTD_decompressStream() returns 0.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- *  Note 1 : Adding any prefix (including NULL) invalidates any previously set prefix or dictionary
- *  Note 2 : Prefix buffer is referenced. It **must** outlive decompression.
- *           Prefix buffer must remain unmodified up to the end of frame,
- *           reached when ZSTD_decompressStream() returns 0.
- *  Note 3 : By default, the prefix is treated as raw content (ZSTD_dm_rawContent).
- *           Use ZSTD_CCtx_refPrefix_advanced() to alter dictMode (Experimental section)
- *  Note 4 : Referencing a raw content prefix has almost no cpu nor memory cost.
- *           A full dictionary is more costly, as it requires building tables.
- */
-ZSTDLIB_API size_t ZSTD_DCtx_refPrefix(ZSTD_DCtx* dctx,
-                                 const void* prefix, size_t prefixSize);
-
-/* ===   Memory management   === */
-
-/*! ZSTD_sizeof_*() :
- *  These functions give the _current_ memory usage of selected object.
- *  Note that object memory usage can evolve (increase or decrease) over time. */
-ZSTDLIB_API size_t ZSTD_sizeof_CCtx(const ZSTD_CCtx* cctx);
-ZSTDLIB_API size_t ZSTD_sizeof_DCtx(const ZSTD_DCtx* dctx);
-ZSTDLIB_API size_t ZSTD_sizeof_CStream(const ZSTD_CStream* zcs);
-ZSTDLIB_API size_t ZSTD_sizeof_DStream(const ZSTD_DStream* zds);
-ZSTDLIB_API size_t ZSTD_sizeof_CDict(const ZSTD_CDict* cdict);
-ZSTDLIB_API size_t ZSTD_sizeof_DDict(const ZSTD_DDict* ddict);
-
-#endif  /* ZSTD_H_235446 */
-
-
-/* **************************************************************************************
- *   ADVANCED AND EXPERIMENTAL FUNCTIONS
- ****************************************************************************************
- * The definitions in the following section are considered experimental.
- * They are provided for advanced scenarios.
- * They should never be used with a dynamic library, as prototypes may change in the future.
- * Use them only in association with static linking.
- * ***************************************************************************************/
-
-#if defined(ZSTD_STATIC_LINKING_ONLY) && !defined(ZSTD_H_ZSTD_STATIC_LINKING_ONLY)
-#define ZSTD_H_ZSTD_STATIC_LINKING_ONLY
-
-/****************************************************************************************
- *   experimental API (static linking only)
- ****************************************************************************************
- * The following symbols and constants
- * are not planned to join "stable API" status in the near future.
- * They can still change in future versions.
- * Some of them are planned to remain in the static_only section indefinitely.
- * Some of them might be removed in the future (especially when redundant with existing stable functions)
- * ***************************************************************************************/
-
-#define ZSTD_FRAMEHEADERSIZE_PREFIX 5   /* minimum input size required to query frame header size */
-#define ZSTD_FRAMEHEADERSIZE_MIN    6
-#define ZSTD_FRAMEHEADERSIZE_MAX   18   /* can be useful for static allocation */
-#define ZSTD_SKIPPABLEHEADERSIZE    8
-
-/* compression parameter bounds */
-#define ZSTD_WINDOWLOG_MAX_32    30
-#define ZSTD_WINDOWLOG_MAX_64    31
-#define ZSTD_WINDOWLOG_MAX     ((int)(sizeof(size_t) == 4 ? ZSTD_WINDOWLOG_MAX_32 : ZSTD_WINDOWLOG_MAX_64))
-#define ZSTD_WINDOWLOG_MIN       10
-#define ZSTD_HASHLOG_MAX       ((ZSTD_WINDOWLOG_MAX < 30) ? ZSTD_WINDOWLOG_MAX : 30)
-#define ZSTD_HASHLOG_MIN          6
-#define ZSTD_CHAINLOG_MAX_32     29
-#define ZSTD_CHAINLOG_MAX_64     30
-#define ZSTD_CHAINLOG_MAX      ((int)(sizeof(size_t) == 4 ? ZSTD_CHAINLOG_MAX_32 : ZSTD_CHAINLOG_MAX_64))
-#define ZSTD_CHAINLOG_MIN        ZSTD_HASHLOG_MIN
-#define ZSTD_SEARCHLOG_MAX      (ZSTD_WINDOWLOG_MAX-1)
-#define ZSTD_SEARCHLOG_MIN        1
-#define ZSTD_MINMATCH_MAX         7   /* only for ZSTD_fast, other strategies are limited to 6 */
-#define ZSTD_MINMATCH_MIN         3   /* only for ZSTD_btopt+, faster strategies are limited to 4 */
-#define ZSTD_TARGETLENGTH_MAX    ZSTD_BLOCKSIZE_MAX
-#define ZSTD_TARGETLENGTH_MIN     0   /* note : comparing this constant to an unsigned results in a tautological test */
-#define ZSTD_STRATEGY_MIN        ZSTD_fast
-#define ZSTD_STRATEGY_MAX        ZSTD_btultra2
-
-
-#define ZSTD_OVERLAPLOG_MIN       0
-#define ZSTD_OVERLAPLOG_MAX       9
-
-#define ZSTD_WINDOWLOG_LIMIT_DEFAULT 27   /* by default, the streaming decoder will refuse any frame
-                                           * requiring larger than (1<<ZSTD_WINDOWLOG_LIMIT_DEFAULT) window size,
-                                           * to preserve host's memory from unreasonable requirements.
-                                           * This limit can be overridden using ZSTD_DCtx_setParameter(,ZSTD_d_windowLogMax,).
-                                           * The limit does not apply for one-pass decoders (such as ZSTD_decompress()), since no additional memory is allocated */
-
-
-/* LDM parameter bounds */
-#define ZSTD_LDM_HASHLOG_MIN      ZSTD_HASHLOG_MIN
-#define ZSTD_LDM_HASHLOG_MAX      ZSTD_HASHLOG_MAX
-#define ZSTD_LDM_MINMATCH_MIN        4
-#define ZSTD_LDM_MINMATCH_MAX     4096
-#define ZSTD_LDM_BUCKETSIZELOG_MIN   1
-#define ZSTD_LDM_BUCKETSIZELOG_MAX   8
-#define ZSTD_LDM_HASHRATELOG_MIN     0
-#define ZSTD_LDM_HASHRATELOG_MAX (ZSTD_WINDOWLOG_MAX - ZSTD_HASHLOG_MIN)
-
-/* Advanced parameter bounds */
-#define ZSTD_TARGETCBLOCKSIZE_MIN   64
-#define ZSTD_TARGETCBLOCKSIZE_MAX   ZSTD_BLOCKSIZE_MAX
-
-/* internal */
-#define ZSTD_HASHLOG3_MAX           17
-
-
-/* ---  Advanced types  --- */
-
-typedef struct ZSTD_CCtx_params_s ZSTD_CCtx_params;
-
-typedef struct {
-    unsigned windowLog;       /**< largest match distance : larger == more compression, more memory needed during decompression */
-    unsigned chainLog;        /**< fully searched segment : larger == more compression, slower, more memory (useless for fast) */
-    unsigned hashLog;         /**< dispatch table : larger == faster, more memory */
-    unsigned searchLog;       /**< nb of searches : larger == more compression, slower */
-    unsigned minMatch;        /**< match length searched : larger == faster decompression, sometimes less compression */
-    unsigned targetLength;    /**< acceptable match size for optimal parser (only) : larger == more compression, slower */
-    ZSTD_strategy strategy;   /**< see ZSTD_strategy definition above */
-} ZSTD_compressionParameters;
-
-typedef struct {
-    int contentSizeFlag; /**< 1: content size will be in frame header (when known) */
-    int checksumFlag;    /**< 1: generate a 32-bits checksum using XXH64 algorithm at end of frame, for error detection */
-    int noDictIDFlag;    /**< 1: no dictID will be saved into frame header (dictID is only useful for dictionary compression) */
-} ZSTD_frameParameters;
-
-typedef struct {
-    ZSTD_compressionParameters cParams;
-    ZSTD_frameParameters fParams;
-} ZSTD_parameters;
-
-typedef enum {
-    ZSTD_dct_auto = 0,       /* dictionary is "full" when starting with ZSTD_MAGIC_DICTIONARY, otherwise it is "rawContent" */
-    ZSTD_dct_rawContent = 1, /* ensures dictionary is always loaded as rawContent, even if it starts with ZSTD_MAGIC_DICTIONARY */
-    ZSTD_dct_fullDict = 2    /* refuses to load a dictionary if it does not respect Zstandard's specification, starting with ZSTD_MAGIC_DICTIONARY */
-} ZSTD_dictContentType_e;
-
-typedef enum {
-    ZSTD_dlm_byCopy = 0,  /**< Copy dictionary content internally */
-    ZSTD_dlm_byRef = 1,   /**< Reference dictionary content -- the dictionary buffer must outlive its users. */
-} ZSTD_dictLoadMethod_e;
-
-typedef enum {
-    /* Opened question : should we have a format ZSTD_f_auto ?
-     * Today, it would mean exactly the same as ZSTD_f_zstd1.
-     * But, in the future, should several formats become supported,
-     * on the compression side, it would mean "default format".
-     * On the decompression side, it would mean "automatic format detection",
-     * so that ZSTD_f_zstd1 would mean "accept *only* zstd frames".
-     * Since meaning is a little different, another option could be to define different enums for compression and decompression.
-     * This question could be kept for later, when there are actually multiple formats to support,
-     * but there is also the question of pinning enum values, and pinning value `0` is especially important */
-    ZSTD_f_zstd1 = 0,           /* zstd frame format, specified in zstd_compression_format.md (default) */
-    ZSTD_f_zstd1_magicless = 1, /* Variant of zstd frame format, without initial 4-bytes magic number.
-                                 * Useful to save 4 bytes per generated frame.
-                                 * Decoder cannot recognise automatically this format, requiring this instruction. */
-} ZSTD_format_e;
-
-typedef enum {
-    /* Note: this enum and the behavior it controls are effectively internal
-     * implementation details of the compressor. They are expected to continue
-     * to evolve and should be considered only in the context of extremely
-     * advanced performance tuning.
-     *
-     * Zstd currently supports the use of a CDict in two ways:
-     *
-     * - The contents of the CDict can be copied into the working context. This
-     *   means that the compression can search both the dictionary and input
-     *   while operating on a single set of internal tables. This makes
-     *   the compression faster per-byte of input. However, the initial copy of
-     *   the CDict's tables incurs a fixed cost at the beginning of the
-     *   compression. For small compressions (< 8 KB), that copy can dominate
-     *   the cost of the compression.
-     *
-     * - The CDict's tables can be used in-place. In this model, compression is
-     *   slower per input byte, because the compressor has to search two sets of
-     *   tables. However, this model incurs no start-up cost (as long as the
-     *   working context's tables can be reused). For small inputs, this can be
-     *   faster than copying the CDict's tables.
-     *
-     * Zstd has a simple internal heuristic that selects which strategy to use
-     * at the beginning of a compression. However, if experimentation shows that
-     * Zstd is making poor choices, it is possible to override that choice with
-     * this enum.
-     */
-    ZSTD_dictDefaultAttach = 0, /* Use the default heuristic. */
-    ZSTD_dictForceAttach   = 1, /* Never copy the dictionary. */
-    ZSTD_dictForceCopy     = 2, /* Always copy the dictionary. */
-} ZSTD_dictAttachPref_e;
-
-typedef enum {
-  ZSTD_lcm_auto = 0,          /**< Automatically determine the compression mode based on the compression level.
-                               *   Negative compression levels will be uncompressed, and positive compression
-                               *   levels will be compressed. */
-  ZSTD_lcm_huffman = 1,       /**< Always attempt Huffman compression. Uncompressed literals will still be
-                               *   emitted if Huffman compression is not profitable. */
-  ZSTD_lcm_uncompressed = 2,  /**< Always emit uncompressed literals. */
-} ZSTD_literalCompressionMode_e;
-
-
-/***************************************
-*  Frame size functions
-***************************************/
-
-/*! ZSTD_findDecompressedSize() :
- *  `src` should point to the start of a series of ZSTD encoded and/or skippable frames
- *  `srcSize` must be the _exact_ size of this series
- *       (i.e. there should be a frame boundary at `src + srcSize`)
- *  @return : - decompressed size of all data in all successive frames
- *            - if the decompressed size cannot be determined: ZSTD_CONTENTSIZE_UNKNOWN
- *            - if an error occurred: ZSTD_CONTENTSIZE_ERROR
- *
- *   note 1 : decompressed size is an optional field, that may not be present, especially in streaming mode.
- *            When `return==ZSTD_CONTENTSIZE_UNKNOWN`, data to decompress could be any size.
- *            In which case, it's necessary to use streaming mode to decompress data.
- *   note 2 : decompressed size is always present when compression is done with ZSTD_compress()
- *   note 3 : decompressed size can be very large (64-bits value),
- *            potentially larger than what local system can handle as a single memory segment.
- *            In which case, it's necessary to use streaming mode to decompress data.
- *   note 4 : If source is untrusted, decompressed size could be wrong or intentionally modified.
- *            Always ensure result fits within application's authorized limits.
- *            Each application can set its own limits.
- *   note 5 : ZSTD_findDecompressedSize handles multiple frames, and so it must traverse the input to
- *            read each contained frame header.  This is fast as most of the data is skipped,
- *            however it does mean that all frame data must be present and valid. */
-ZSTDLIB_API unsigned long long ZSTD_findDecompressedSize(const void* src, size_t srcSize);
-
-/*! ZSTD_decompressBound() :
- *  `src` should point to the start of a series of ZSTD encoded and/or skippable frames
- *  `srcSize` must be the _exact_ size of this series
- *       (i.e. there should be a frame boundary at `src + srcSize`)
- *  @return : - upper-bound for the decompressed size of all data in all successive frames
- *            - if an error occured: ZSTD_CONTENTSIZE_ERROR
- *
- *  note 1  : an error can occur if `src` contains an invalid or incorrectly formatted frame.
- *  note 2  : the upper-bound is exact when the decompressed size field is available in every ZSTD encoded frame of `src`.
- *            in this case, `ZSTD_findDecompressedSize` and `ZSTD_decompressBound` return the same value.
- *  note 3  : when the decompressed size field isn't available, the upper-bound for that frame is calculated by:
- *              upper-bound = # blocks * min(128 KB, Window_Size)
- */
-ZSTDLIB_API unsigned long long ZSTD_decompressBound(const void* src, size_t srcSize);
-
-/*! ZSTD_frameHeaderSize() :
- *  srcSize must be >= ZSTD_FRAMEHEADERSIZE_PREFIX.
- * @return : size of the Frame Header,
- *           or an error code (if srcSize is too small) */
-ZSTDLIB_API size_t ZSTD_frameHeaderSize(const void* src, size_t srcSize);
-
-
-/***************************************
-*  Memory management
-***************************************/
-
-/*! ZSTD_estimate*() :
- *  These functions make it possible to estimate memory usage
- *  of a future {D,C}Ctx, before its creation.
- *  ZSTD_estimateCCtxSize() will provide a budget large enough for any compression level up to selected one.
- *  It will also consider src size to be arbitrarily "large", which is worst case.
- *  If srcSize is known to always be small, ZSTD_estimateCCtxSize_usingCParams() can provide a tighter estimation.
- *  ZSTD_estimateCCtxSize_usingCParams() can be used in tandem with ZSTD_getCParams() to create cParams from compressionLevel.
- *  ZSTD_estimateCCtxSize_usingCCtxParams() can be used in tandem with ZSTD_CCtxParams_setParameter(). Only single-threaded compression is supported. This function will return an error code if ZSTD_c_nbWorkers is >= 1.
- *  Note : CCtx size estimation is only correct for single-threaded compression. */
-ZSTDLIB_API size_t ZSTD_estimateCCtxSize(int compressionLevel);
-ZSTDLIB_API size_t ZSTD_estimateCCtxSize_usingCParams(ZSTD_compressionParameters cParams);
-ZSTDLIB_API size_t ZSTD_estimateCCtxSize_usingCCtxParams(const ZSTD_CCtx_params* params);
-ZSTDLIB_API size_t ZSTD_estimateDCtxSize(void);
-
-/*! ZSTD_estimateCStreamSize() :
- *  ZSTD_estimateCStreamSize() will provide a budget large enough for any compression level up to selected one.
- *  It will also consider src size to be arbitrarily "large", which is worst case.
- *  If srcSize is known to always be small, ZSTD_estimateCStreamSize_usingCParams() can provide a tighter estimation.
- *  ZSTD_estimateCStreamSize_usingCParams() can be used in tandem with ZSTD_getCParams() to create cParams from compressionLevel.
- *  ZSTD_estimateCStreamSize_usingCCtxParams() can be used in tandem with ZSTD_CCtxParams_setParameter(). Only single-threaded compression is supported. This function will return an error code if ZSTD_c_nbWorkers is >= 1.
- *  Note : CStream size estimation is only correct for single-threaded compression.
- *  ZSTD_DStream memory budget depends on window Size.
- *  This information can be passed manually, using ZSTD_estimateDStreamSize,
- *  or deducted from a valid frame Header, using ZSTD_estimateDStreamSize_fromFrame();
- *  Note : if streaming is init with function ZSTD_init?Stream_usingDict(),
- *         an internal ?Dict will be created, which additional size is not estimated here.
- *         In this case, get total size by adding ZSTD_estimate?DictSize */
-ZSTDLIB_API size_t ZSTD_estimateCStreamSize(int compressionLevel);
-ZSTDLIB_API size_t ZSTD_estimateCStreamSize_usingCParams(ZSTD_compressionParameters cParams);
-ZSTDLIB_API size_t ZSTD_estimateCStreamSize_usingCCtxParams(const ZSTD_CCtx_params* params);
-ZSTDLIB_API size_t ZSTD_estimateDStreamSize(size_t windowSize);
-ZSTDLIB_API size_t ZSTD_estimateDStreamSize_fromFrame(const void* src, size_t srcSize);
-
-/*! ZSTD_estimate?DictSize() :
- *  ZSTD_estimateCDictSize() will bet that src size is relatively "small", and content is copied, like ZSTD_createCDict().
- *  ZSTD_estimateCDictSize_advanced() makes it possible to control compression parameters precisely, like ZSTD_createCDict_advanced().
- *  Note : dictionaries created by reference (`ZSTD_dlm_byRef`) are logically smaller.
- */
-ZSTDLIB_API size_t ZSTD_estimateCDictSize(size_t dictSize, int compressionLevel);
-ZSTDLIB_API size_t ZSTD_estimateCDictSize_advanced(size_t dictSize, ZSTD_compressionParameters cParams, ZSTD_dictLoadMethod_e dictLoadMethod);
-ZSTDLIB_API size_t ZSTD_estimateDDictSize(size_t dictSize, ZSTD_dictLoadMethod_e dictLoadMethod);
-
-/*! ZSTD_initStatic*() :
- *  Initialize an object using a pre-allocated fixed-size buffer.
- *  workspace: The memory area to emplace the object into.
- *             Provided pointer *must be 8-bytes aligned*.
- *             Buffer must outlive object.
- *  workspaceSize: Use ZSTD_estimate*Size() to determine
- *                 how large workspace must be to support target scenario.
- * @return : pointer to object (same address as workspace, just different type),
- *           or NULL if error (size too small, incorrect alignment, etc.)
- *  Note : zstd will never resize nor malloc() when using a static buffer.
- *         If the object requires more memory than available,
- *         zstd will just error out (typically ZSTD_error_memory_allocation).
- *  Note 2 : there is no corresponding "free" function.
- *           Since workspace is allocated externally, it must be freed externally too.
- *  Note 3 : cParams : use ZSTD_getCParams() to convert a compression level
- *           into its associated cParams.
- *  Limitation 1 : currently not compatible with internal dictionary creation, triggered by
- *                 ZSTD_CCtx_loadDictionary(), ZSTD_initCStream_usingDict() or ZSTD_initDStream_usingDict().
- *  Limitation 2 : static cctx currently not compatible with multi-threading.
- *  Limitation 3 : static dctx is incompatible with legacy support.
- */
-ZSTDLIB_API ZSTD_CCtx*    ZSTD_initStaticCCtx(void* workspace, size_t workspaceSize);
-ZSTDLIB_API ZSTD_CStream* ZSTD_initStaticCStream(void* workspace, size_t workspaceSize);    /**< same as ZSTD_initStaticCCtx() */
-
-ZSTDLIB_API ZSTD_DCtx*    ZSTD_initStaticDCtx(void* workspace, size_t workspaceSize);
-ZSTDLIB_API ZSTD_DStream* ZSTD_initStaticDStream(void* workspace, size_t workspaceSize);    /**< same as ZSTD_initStaticDCtx() */
-
-ZSTDLIB_API const ZSTD_CDict* ZSTD_initStaticCDict(
-                                        void* workspace, size_t workspaceSize,
-                                        const void* dict, size_t dictSize,
-                                        ZSTD_dictLoadMethod_e dictLoadMethod,
-                                        ZSTD_dictContentType_e dictContentType,
-                                        ZSTD_compressionParameters cParams);
-
-ZSTDLIB_API const ZSTD_DDict* ZSTD_initStaticDDict(
-                                        void* workspace, size_t workspaceSize,
-                                        const void* dict, size_t dictSize,
-                                        ZSTD_dictLoadMethod_e dictLoadMethod,
-                                        ZSTD_dictContentType_e dictContentType);
-
-
-/*! Custom memory allocation :
- *  These prototypes make it possible to pass your own allocation/free functions.
- *  ZSTD_customMem is provided at creation time, using ZSTD_create*_advanced() variants listed below.
- *  All allocation/free operations will be completed using these custom variants instead of regular <stdlib.h> ones.
- */
-typedef void* (*ZSTD_allocFunction) (void* opaque, size_t size);
-typedef void  (*ZSTD_freeFunction) (void* opaque, void* address);
-typedef struct { ZSTD_allocFunction customAlloc; ZSTD_freeFunction customFree; void* opaque; } ZSTD_customMem;
-static ZSTD_customMem const ZSTD_defaultCMem = { NULL, NULL, NULL };  /**< this constant defers to stdlib's functions */
-
-ZSTDLIB_API ZSTD_CCtx*    ZSTD_createCCtx_advanced(ZSTD_customMem customMem);
-ZSTDLIB_API ZSTD_CStream* ZSTD_createCStream_advanced(ZSTD_customMem customMem);
-ZSTDLIB_API ZSTD_DCtx*    ZSTD_createDCtx_advanced(ZSTD_customMem customMem);
-ZSTDLIB_API ZSTD_DStream* ZSTD_createDStream_advanced(ZSTD_customMem customMem);
-
-ZSTDLIB_API ZSTD_CDict* ZSTD_createCDict_advanced(const void* dict, size_t dictSize,
-                                                  ZSTD_dictLoadMethod_e dictLoadMethod,
-                                                  ZSTD_dictContentType_e dictContentType,
-                                                  ZSTD_compressionParameters cParams,
-                                                  ZSTD_customMem customMem);
-
-ZSTDLIB_API ZSTD_DDict* ZSTD_createDDict_advanced(const void* dict, size_t dictSize,
-                                                  ZSTD_dictLoadMethod_e dictLoadMethod,
-                                                  ZSTD_dictContentType_e dictContentType,
-                                                  ZSTD_customMem customMem);
-
-
-
-/***************************************
-*  Advanced compression functions
-***************************************/
-
-/*! ZSTD_createCDict_byReference() :
- *  Create a digested dictionary for compression
- *  Dictionary content is just referenced, not duplicated.
- *  As a consequence, `dictBuffer` **must** outlive CDict,
- *  and its content must remain unmodified throughout the lifetime of CDict. */
-ZSTDLIB_API ZSTD_CDict* ZSTD_createCDict_byReference(const void* dictBuffer, size_t dictSize, int compressionLevel);
-
-/*! ZSTD_getCParams() :
- * @return ZSTD_compressionParameters structure for a selected compression level and estimated srcSize.
- * `estimatedSrcSize` value is optional, select 0 if not known */
-ZSTDLIB_API ZSTD_compressionParameters ZSTD_getCParams(int compressionLevel, unsigned long long estimatedSrcSize, size_t dictSize);
-
-/*! ZSTD_getParams() :
- *  same as ZSTD_getCParams(), but @return a full `ZSTD_parameters` object instead of sub-component `ZSTD_compressionParameters`.
- *  All fields of `ZSTD_frameParameters` are set to default : contentSize=1, checksum=0, noDictID=0 */
-ZSTDLIB_API ZSTD_parameters ZSTD_getParams(int compressionLevel, unsigned long long estimatedSrcSize, size_t dictSize);
-
-/*! ZSTD_checkCParams() :
- *  Ensure param values remain within authorized range.
- * @return 0 on success, or an error code (can be checked with ZSTD_isError()) */
-ZSTDLIB_API size_t ZSTD_checkCParams(ZSTD_compressionParameters params);
-
-/*! ZSTD_adjustCParams() :
- *  optimize params for a given `srcSize` and `dictSize`.
- * `srcSize` can be unknown, in which case use ZSTD_CONTENTSIZE_UNKNOWN.
- * `dictSize` must be `0` when there is no dictionary.
- *  cPar can be invalid : all parameters will be clamped within valid range in the @return struct.
- *  This function never fails (wide contract) */
-ZSTDLIB_API ZSTD_compressionParameters ZSTD_adjustCParams(ZSTD_compressionParameters cPar, unsigned long long srcSize, size_t dictSize);
-
-/*! ZSTD_compress_advanced() :
- *  Same as ZSTD_compress_usingDict(), with fine-tune control over compression parameters (by structure) */
-ZSTDLIB_API size_t ZSTD_compress_advanced(ZSTD_CCtx* cctx,
-                                          void* dst, size_t dstCapacity,
-                                    const void* src, size_t srcSize,
-                                    const void* dict,size_t dictSize,
-                                          ZSTD_parameters params);
-
-/*! ZSTD_compress_usingCDict_advanced() :
- *  Same as ZSTD_compress_usingCDict(), with fine-tune control over frame parameters */
-ZSTDLIB_API size_t ZSTD_compress_usingCDict_advanced(ZSTD_CCtx* cctx,
-                                              void* dst, size_t dstCapacity,
-                                        const void* src, size_t srcSize,
-                                        const ZSTD_CDict* cdict,
-                                              ZSTD_frameParameters fParams);
-
-
-/*! ZSTD_CCtx_loadDictionary_byReference() :
- *  Same as ZSTD_CCtx_loadDictionary(), but dictionary content is referenced, instead of being copied into CCtx.
- *  It saves some memory, but also requires that `dict` outlives its usage within `cctx` */
-ZSTDLIB_API size_t ZSTD_CCtx_loadDictionary_byReference(ZSTD_CCtx* cctx, const void* dict, size_t dictSize);
-
-/*! ZSTD_CCtx_loadDictionary_advanced() :
- *  Same as ZSTD_CCtx_loadDictionary(), but gives finer control over
- *  how to load the dictionary (by copy ? by reference ?)
- *  and how to interpret it (automatic ? force raw mode ? full mode only ?) */
-ZSTDLIB_API size_t ZSTD_CCtx_loadDictionary_advanced(ZSTD_CCtx* cctx, const void* dict, size_t dictSize, ZSTD_dictLoadMethod_e dictLoadMethod, ZSTD_dictContentType_e dictContentType);
-
-/*! ZSTD_CCtx_refPrefix_advanced() :
- *  Same as ZSTD_CCtx_refPrefix(), but gives finer control over
- *  how to interpret prefix content (automatic ? force raw mode (default) ? full mode only ?) */
-ZSTDLIB_API size_t ZSTD_CCtx_refPrefix_advanced(ZSTD_CCtx* cctx, const void* prefix, size_t prefixSize, ZSTD_dictContentType_e dictContentType);
-
-/* ===   experimental parameters   === */
-/* these parameters can be used with ZSTD_setParameter()
- * they are not guaranteed to remain supported in the future */
-
- /* Enables rsyncable mode,
-  * which makes compressed files more rsync friendly
-  * by adding periodic synchronization points to the compressed data.
-  * The target average block size is ZSTD_c_jobSize / 2.
-  * It's possible to modify the job size to increase or decrease
-  * the granularity of the synchronization point.
-  * Once the jobSize is smaller than the window size,
-  * it will result in compression ratio degradation.
-  * NOTE 1: rsyncable mode only works when multithreading is enabled.
-  * NOTE 2: rsyncable performs poorly in combination with long range mode,
-  * since it will decrease the effectiveness of synchronization points,
-  * though mileage may vary.
-  * NOTE 3: Rsyncable mode limits maximum compression speed to ~400 MB/s.
-  * If the selected compression level is already running significantly slower,
-  * the overall speed won't be significantly impacted.
-  */
- #define ZSTD_c_rsyncable ZSTD_c_experimentalParam1
-
-/* Select a compression format.
- * The value must be of type ZSTD_format_e.
- * See ZSTD_format_e enum definition for details */
-#define ZSTD_c_format ZSTD_c_experimentalParam2
-
-/* Force back-reference distances to remain < windowSize,
- * even when referencing into Dictionary content (default:0) */
-#define ZSTD_c_forceMaxWindow ZSTD_c_experimentalParam3
-
-/* Controls whether the contents of a CDict
- * are used in place, or copied into the working context.
- * Accepts values from the ZSTD_dictAttachPref_e enum.
- * See the comments on that enum for an explanation of the feature. */
-#define ZSTD_c_forceAttachDict ZSTD_c_experimentalParam4
-
-/* Controls how the literals are compressed (default is auto).
- * The value must be of type ZSTD_literalCompressionMode_e.
- * See ZSTD_literalCompressionMode_t enum definition for details.
- */
-#define ZSTD_c_literalCompressionMode ZSTD_c_experimentalParam5
-
-/* Tries to fit compressed block size to be around targetCBlockSize.
- * No target when targetCBlockSize == 0.
- * There is no guarantee on compressed block size (default:0) */
-#define ZSTD_c_targetCBlockSize ZSTD_c_experimentalParam6
-
-/*! ZSTD_CCtx_getParameter() :
- *  Get the requested compression parameter value, selected by enum ZSTD_cParameter,
- *  and store it into int* value.
- * @return : 0, or an error code (which can be tested with ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_CCtx_getParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, int* value);
-
-
-/*! ZSTD_CCtx_params :
- *  Quick howto :
- *  - ZSTD_createCCtxParams() : Create a ZSTD_CCtx_params structure
- *  - ZSTD_CCtxParams_setParameter() : Push parameters one by one into
- *                                     an existing ZSTD_CCtx_params structure.
- *                                     This is similar to
- *                                     ZSTD_CCtx_setParameter().
- *  - ZSTD_CCtx_setParametersUsingCCtxParams() : Apply parameters to
- *                                    an existing CCtx.
- *                                    These parameters will be applied to
- *                                    all subsequent frames.
- *  - ZSTD_compressStream2() : Do compression using the CCtx.
- *  - ZSTD_freeCCtxParams() : Free the memory.
- *
- *  This can be used with ZSTD_estimateCCtxSize_advanced_usingCCtxParams()
- *  for static allocation of CCtx for single-threaded compression.
- */
-ZSTDLIB_API ZSTD_CCtx_params* ZSTD_createCCtxParams(void);
-ZSTDLIB_API size_t ZSTD_freeCCtxParams(ZSTD_CCtx_params* params);
-
-/*! ZSTD_CCtxParams_reset() :
- *  Reset params to default values.
- */
-ZSTDLIB_API size_t ZSTD_CCtxParams_reset(ZSTD_CCtx_params* params);
-
-/*! ZSTD_CCtxParams_init() :
- *  Initializes the compression parameters of cctxParams according to
- *  compression level. All other parameters are reset to their default values.
- */
-ZSTDLIB_API size_t ZSTD_CCtxParams_init(ZSTD_CCtx_params* cctxParams, int compressionLevel);
-
-/*! ZSTD_CCtxParams_init_advanced() :
- *  Initializes the compression and frame parameters of cctxParams according to
- *  params. All other parameters are reset to their default values.
- */
-ZSTDLIB_API size_t ZSTD_CCtxParams_init_advanced(ZSTD_CCtx_params* cctxParams, ZSTD_parameters params);
-
-/*! ZSTD_CCtxParams_setParameter() :
- *  Similar to ZSTD_CCtx_setParameter.
- *  Set one compression parameter, selected by enum ZSTD_cParameter.
- *  Parameters must be applied to a ZSTD_CCtx using ZSTD_CCtx_setParametersUsingCCtxParams().
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_CCtxParams_setParameter(ZSTD_CCtx_params* params, ZSTD_cParameter param, int value);
-
-/*! ZSTD_CCtxParams_getParameter() :
- * Similar to ZSTD_CCtx_getParameter.
- * Get the requested value of one compression parameter, selected by enum ZSTD_cParameter.
- * @result : 0, or an error code (which can be tested with ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_CCtxParams_getParameter(ZSTD_CCtx_params* params, ZSTD_cParameter param, int* value);
-
-/*! ZSTD_CCtx_setParametersUsingCCtxParams() :
- *  Apply a set of ZSTD_CCtx_params to the compression context.
- *  This can be done even after compression is started,
- *    if nbWorkers==0, this will have no impact until a new compression is started.
- *    if nbWorkers>=1, new parameters will be picked up at next job,
- *       with a few restrictions (windowLog, pledgedSrcSize, nbWorkers, jobSize, and overlapLog are not updated).
- */
-ZSTDLIB_API size_t ZSTD_CCtx_setParametersUsingCCtxParams(
-        ZSTD_CCtx* cctx, const ZSTD_CCtx_params* params);
-
-/*! ZSTD_compressStream2_simpleArgs() :
- *  Same as ZSTD_compressStream2(),
- *  but using only integral types as arguments.
- *  This variant might be helpful for binders from dynamic languages
- *  which have troubles handling structures containing memory pointers.
- */
-ZSTDLIB_API size_t ZSTD_compressStream2_simpleArgs (
-                            ZSTD_CCtx* cctx,
-                            void* dst, size_t dstCapacity, size_t* dstPos,
-                      const void* src, size_t srcSize, size_t* srcPos,
-                            ZSTD_EndDirective endOp);
-
-
-/***************************************
-*  Advanced decompression functions
-***************************************/
-
-/*! ZSTD_isFrame() :
- *  Tells if the content of `buffer` starts with a valid Frame Identifier.
- *  Note : Frame Identifier is 4 bytes. If `size < 4`, @return will always be 0.
- *  Note 2 : Legacy Frame Identifiers are considered valid only if Legacy Support is enabled.
- *  Note 3 : Skippable Frame Identifiers are considered valid. */
-ZSTDLIB_API unsigned ZSTD_isFrame(const void* buffer, size_t size);
-
-/*! ZSTD_createDDict_byReference() :
- *  Create a digested dictionary, ready to start decompression operation without startup delay.
- *  Dictionary content is referenced, and therefore stays in dictBuffer.
- *  It is important that dictBuffer outlives DDict,
- *  it must remain read accessible throughout the lifetime of DDict */
-ZSTDLIB_API ZSTD_DDict* ZSTD_createDDict_byReference(const void* dictBuffer, size_t dictSize);
-
-/*! ZSTD_DCtx_loadDictionary_byReference() :
- *  Same as ZSTD_DCtx_loadDictionary(),
- *  but references `dict` content instead of copying it into `dctx`.
- *  This saves memory if `dict` remains around.,
- *  However, it's imperative that `dict` remains accessible (and unmodified) while being used, so it must outlive decompression. */
-ZSTDLIB_API size_t ZSTD_DCtx_loadDictionary_byReference(ZSTD_DCtx* dctx, const void* dict, size_t dictSize);
-
-/*! ZSTD_DCtx_loadDictionary_advanced() :
- *  Same as ZSTD_DCtx_loadDictionary(),
- *  but gives direct control over
- *  how to load the dictionary (by copy ? by reference ?)
- *  and how to interpret it (automatic ? force raw mode ? full mode only ?). */
-ZSTDLIB_API size_t ZSTD_DCtx_loadDictionary_advanced(ZSTD_DCtx* dctx, const void* dict, size_t dictSize, ZSTD_dictLoadMethod_e dictLoadMethod, ZSTD_dictContentType_e dictContentType);
-
-/*! ZSTD_DCtx_refPrefix_advanced() :
- *  Same as ZSTD_DCtx_refPrefix(), but gives finer control over
- *  how to interpret prefix content (automatic ? force raw mode (default) ? full mode only ?) */
-ZSTDLIB_API size_t ZSTD_DCtx_refPrefix_advanced(ZSTD_DCtx* dctx, const void* prefix, size_t prefixSize, ZSTD_dictContentType_e dictContentType);
-
-/*! ZSTD_DCtx_setMaxWindowSize() :
- *  Refuses allocating internal buffers for frames requiring a window size larger than provided limit.
- *  This protects a decoder context from reserving too much memory for itself (potential attack scenario).
- *  This parameter is only useful in streaming mode, since no internal buffer is allocated in single-pass mode.
- *  By default, a decompression context accepts all window sizes <= (1 << ZSTD_WINDOWLOG_LIMIT_DEFAULT)
- * @return : 0, or an error code (which can be tested using ZSTD_isError()).
- */
-ZSTDLIB_API size_t ZSTD_DCtx_setMaxWindowSize(ZSTD_DCtx* dctx, size_t maxWindowSize);
-
-/* ZSTD_d_format
- * experimental parameter,
- * allowing selection between ZSTD_format_e input compression formats
- */
-#define ZSTD_d_format ZSTD_d_experimentalParam1
-
-/*! ZSTD_DCtx_setFormat() :
- *  Instruct the decoder context about what kind of data to decode next.
- *  This instruction is mandatory to decode data without a fully-formed header,
- *  such ZSTD_f_zstd1_magicless for example.
- * @return : 0, or an error code (which can be tested using ZSTD_isError()). */
-ZSTDLIB_API size_t ZSTD_DCtx_setFormat(ZSTD_DCtx* dctx, ZSTD_format_e format);
-
-/*! ZSTD_decompressStream_simpleArgs() :
- *  Same as ZSTD_decompressStream(),
- *  but using only integral types as arguments.
- *  This can be helpful for binders from dynamic languages
- *  which have troubles handling structures containing memory pointers.
- */
-ZSTDLIB_API size_t ZSTD_decompressStream_simpleArgs (
-                            ZSTD_DCtx* dctx,
-                            void* dst, size_t dstCapacity, size_t* dstPos,
-                      const void* src, size_t srcSize, size_t* srcPos);
-
-
-/********************************************************************
-*  Advanced streaming functions
-*  Warning : most of these functions are now redundant with the Advanced API.
-*  Once Advanced API reaches "stable" status,
-*  redundant functions will be deprecated, and then at some point removed.
-********************************************************************/
-
-/*=====   Advanced Streaming compression functions  =====*/
-/**! ZSTD_initCStream_srcSize() :
- * This function is deprecated, and equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_refCDict(zcs, NULL); // clear the dictionary (if any)
- *     ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel);
- *     ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize);
- *
- * pledgedSrcSize must be correct. If it is not known at init time, use
- * ZSTD_CONTENTSIZE_UNKNOWN. Note that, for compatibility with older programs,
- * "0" also disables frame content size field. It may be enabled in the future.
- */
-ZSTDLIB_API size_t ZSTD_initCStream_srcSize(ZSTD_CStream* zcs, int compressionLevel, unsigned long long pledgedSrcSize);
-/**! ZSTD_initCStream_usingDict() :
- * This function is deprecated, and is equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel);
- *     ZSTD_CCtx_loadDictionary(zcs, dict, dictSize);
- *
- * Creates of an internal CDict (incompatible with static CCtx), except if
- * dict == NULL or dictSize < 8, in which case no dict is used.
- * Note: dict is loaded with ZSTD_dm_auto (treated as a full zstd dictionary if
- * it begins with ZSTD_MAGIC_DICTIONARY, else as raw content) and ZSTD_dlm_byCopy.
- */
-ZSTDLIB_API size_t ZSTD_initCStream_usingDict(ZSTD_CStream* zcs, const void* dict, size_t dictSize, int compressionLevel);
-/**! ZSTD_initCStream_advanced() :
- * This function is deprecated, and is approximately equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_setZstdParams(zcs, params); // Set the zstd params and leave the rest as-is
- *     ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize);
- *     ZSTD_CCtx_loadDictionary(zcs, dict, dictSize);
- *
- * pledgedSrcSize must be correct. If srcSize is not known at init time, use
- * value ZSTD_CONTENTSIZE_UNKNOWN. dict is loaded with ZSTD_dm_auto and ZSTD_dlm_byCopy.
- */
-ZSTDLIB_API size_t ZSTD_initCStream_advanced(ZSTD_CStream* zcs, const void* dict, size_t dictSize,
-                                             ZSTD_parameters params, unsigned long long pledgedSrcSize);
-/**! ZSTD_initCStream_usingCDict() :
- * This function is deprecated, and equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_refCDict(zcs, cdict);
- *
- * note : cdict will just be referenced, and must outlive compression session
- */
-ZSTDLIB_API size_t ZSTD_initCStream_usingCDict(ZSTD_CStream* zcs, const ZSTD_CDict* cdict);
-/**! ZSTD_initCStream_usingCDict_advanced() :
- * This function is deprecated, and is approximately equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_setZstdFrameParams(zcs, fParams); // Set the zstd frame params and leave the rest as-is
- *     ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize);
- *     ZSTD_CCtx_refCDict(zcs, cdict);
- *
- * same as ZSTD_initCStream_usingCDict(), with control over frame parameters.
- * pledgedSrcSize must be correct. If srcSize is not known at init time, use
- * value ZSTD_CONTENTSIZE_UNKNOWN.
- */
-ZSTDLIB_API size_t ZSTD_initCStream_usingCDict_advanced(ZSTD_CStream* zcs, const ZSTD_CDict* cdict, ZSTD_frameParameters fParams, unsigned long long pledgedSrcSize);
-
-/*! ZSTD_resetCStream() :
- * This function is deprecated, and is equivalent to:
- *     ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
- *     ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize);
- *
- *  start a new frame, using same parameters from previous frame.
- *  This is typically useful to skip dictionary loading stage, since it will re-use it in-place.
- *  Note that zcs must be init at least once before using ZSTD_resetCStream().
- *  If pledgedSrcSize is not known at reset time, use macro ZSTD_CONTENTSIZE_UNKNOWN.
- *  If pledgedSrcSize > 0, its value must be correct, as it will be written in header, and controlled at the end.
- *  For the time being, pledgedSrcSize==0 is interpreted as "srcSize unknown" for compatibility with older programs,
- *  but it will change to mean "empty" in future version, so use macro ZSTD_CONTENTSIZE_UNKNOWN instead.
- * @return : 0, or an error code (which can be tested using ZSTD_isError())
- */
-ZSTDLIB_API size_t ZSTD_resetCStream(ZSTD_CStream* zcs, unsigned long long pledgedSrcSize);
-
-
-typedef struct {
-    unsigned long long ingested;   /* nb input bytes read and buffered */
-    unsigned long long consumed;   /* nb input bytes actually compressed */
-    unsigned long long produced;   /* nb of compressed bytes generated and buffered */
-    unsigned long long flushed;    /* nb of compressed bytes flushed : not provided; can be tracked from caller side */
-    unsigned currentJobID;         /* MT only : latest started job nb */
-    unsigned nbActiveWorkers;      /* MT only : nb of workers actively compressing at probe time */
-} ZSTD_frameProgression;
-
-/* ZSTD_getFrameProgression() :
- * tells how much data has been ingested (read from input)
- * consumed (input actually compressed) and produced (output) for current frame.
- * Note : (ingested - consumed) is amount of input data buffered internally, not yet compressed.
- * Aggregates progression inside active worker threads.
- */
-ZSTDLIB_API ZSTD_frameProgression ZSTD_getFrameProgression(const ZSTD_CCtx* cctx);
-
-/*! ZSTD_toFlushNow() :
- *  Tell how many bytes are ready to be flushed immediately.
- *  Useful for multithreading scenarios (nbWorkers >= 1).
- *  Probe the oldest active job, defined as oldest job not yet entirely flushed,
- *  and check its output buffer.
- * @return : amount of data stored in oldest job and ready to be flushed immediately.
- *  if @return == 0, it means either :
- *  + there is no active job (could be checked with ZSTD_frameProgression()), or
- *  + oldest job is still actively compressing data,
- *    but everything it has produced has also been flushed so far,
- *    therefore flush speed is limited by production speed of oldest job
- *    irrespective of the speed of concurrent (and newer) jobs.
- */
-ZSTDLIB_API size_t ZSTD_toFlushNow(ZSTD_CCtx* cctx);
-
-
-/*=====   Advanced Streaming decompression functions  =====*/
-/**
- * This function is deprecated, and is equivalent to:
- *
- *     ZSTD_DCtx_reset(zds, ZSTD_reset_session_only);
- *     ZSTD_DCtx_loadDictionary(zds, dict, dictSize);
- *
- * note: no dictionary will be used if dict == NULL or dictSize < 8
- */
-ZSTDLIB_API size_t ZSTD_initDStream_usingDict(ZSTD_DStream* zds, const void* dict, size_t dictSize);
-/**
- * This function is deprecated, and is equivalent to:
- *
- *     ZSTD_DCtx_reset(zds, ZSTD_reset_session_only);
- *     ZSTD_DCtx_refDDict(zds, ddict);
- *
- * note : ddict is referenced, it must outlive decompression session
- */
-ZSTDLIB_API size_t ZSTD_initDStream_usingDDict(ZSTD_DStream* zds, const ZSTD_DDict* ddict);
-/**
- * This function is deprecated, and is equivalent to:
- *
- *     ZSTD_DCtx_reset(zds, ZSTD_reset_session_only);
- *
- * re-use decompression parameters from previous init; saves dictionary loading
- */
-ZSTDLIB_API size_t ZSTD_resetDStream(ZSTD_DStream* zds);
-
-
-/*********************************************************************
-*  Buffer-less and synchronous inner streaming functions
-*
-*  This is an advanced API, giving full control over buffer management, for users which need direct control over memory.
-*  But it's also a complex one, with several restrictions, documented below.
-*  Prefer normal streaming API for an easier experience.
-********************************************************************* */
-
-/**
-  Buffer-less streaming compression (synchronous mode)
-
-  A ZSTD_CCtx object is required to track streaming operations.
-  Use ZSTD_createCCtx() / ZSTD_freeCCtx() to manage resource.
-  ZSTD_CCtx object can be re-used multiple times within successive compression operations.
-
-  Start by initializing a context.
-  Use ZSTD_compressBegin(), or ZSTD_compressBegin_usingDict() for dictionary compression,
-  or ZSTD_compressBegin_advanced(), for finer parameter control.
-  It's also possible to duplicate a reference context which has already been initialized, using ZSTD_copyCCtx()
-
-  Then, consume your input using ZSTD_compressContinue().
-  There are some important considerations to keep in mind when using this advanced function :
-  - ZSTD_compressContinue() has no internal buffer. It uses externally provided buffers only.
-  - Interface is synchronous : input is consumed entirely and produces 1+ compressed blocks.
-  - Caller must ensure there is enough space in `dst` to store compressed data under worst case scenario.
-    Worst case evaluation is provided by ZSTD_compressBound().
-    ZSTD_compressContinue() doesn't guarantee recover after a failed compression.
-  - ZSTD_compressContinue() presumes prior input ***is still accessible and unmodified*** (up to maximum distance size, see WindowLog).
-    It remembers all previous contiguous blocks, plus one separated memory segment (which can itself consists of multiple contiguous blocks)
-  - ZSTD_compressContinue() detects that prior input has been overwritten when `src` buffer overlaps.
-    In which case, it will "discard" the relevant memory section from its history.
-
-  Finish a frame with ZSTD_compressEnd(), which will write the last block(s) and optional checksum.
-  It's possible to use srcSize==0, in which case, it will write a final empty block to end the frame.
-  Without last block mark, frames are considered unfinished (hence corrupted) by compliant decoders.
-
-  `ZSTD_CCtx` object can be re-used (ZSTD_compressBegin()) to compress again.
-*/
-
-/*=====   Buffer-less streaming compression functions  =====*/
-ZSTDLIB_API size_t ZSTD_compressBegin(ZSTD_CCtx* cctx, int compressionLevel);
-ZSTDLIB_API size_t ZSTD_compressBegin_usingDict(ZSTD_CCtx* cctx, const void* dict, size_t dictSize, int compressionLevel);
-ZSTDLIB_API size_t ZSTD_compressBegin_advanced(ZSTD_CCtx* cctx, const void* dict, size_t dictSize, ZSTD_parameters params, unsigned long long pledgedSrcSize); /**< pledgedSrcSize : If srcSize is not known at init time, use ZSTD_CONTENTSIZE_UNKNOWN */
-ZSTDLIB_API size_t ZSTD_compressBegin_usingCDict(ZSTD_CCtx* cctx, const ZSTD_CDict* cdict); /**< note: fails if cdict==NULL */
-ZSTDLIB_API size_t ZSTD_compressBegin_usingCDict_advanced(ZSTD_CCtx* const cctx, const ZSTD_CDict* const cdict, ZSTD_frameParameters const fParams, unsigned long long const pledgedSrcSize);   /* compression parameters are already set within cdict. pledgedSrcSize must be correct. If srcSize is not known, use macro ZSTD_CONTENTSIZE_UNKNOWN */
-ZSTDLIB_API size_t ZSTD_copyCCtx(ZSTD_CCtx* cctx, const ZSTD_CCtx* preparedCCtx, unsigned long long pledgedSrcSize); /**<  note: if pledgedSrcSize is not known, use ZSTD_CONTENTSIZE_UNKNOWN */
-
-ZSTDLIB_API size_t ZSTD_compressContinue(ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-ZSTDLIB_API size_t ZSTD_compressEnd(ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-/*-
-  Buffer-less streaming decompression (synchronous mode)
-
-  A ZSTD_DCtx object is required to track streaming operations.
-  Use ZSTD_createDCtx() / ZSTD_freeDCtx() to manage it.
-  A ZSTD_DCtx object can be re-used multiple times.
-
-  First typical operation is to retrieve frame parameters, using ZSTD_getFrameHeader().
-  Frame header is extracted from the beginning of compressed frame, so providing only the frame's beginning is enough.
-  Data fragment must be large enough to ensure successful decoding.
- `ZSTD_frameHeaderSize_max` bytes is guaranteed to always be large enough.
-  @result : 0 : successful decoding, the `ZSTD_frameHeader` structure is correctly filled.
-           >0 : `srcSize` is too small, please provide at least @result bytes on next attempt.
-           errorCode, which can be tested using ZSTD_isError().
-
-  It fills a ZSTD_frameHeader structure with important information to correctly decode the frame,
-  such as the dictionary ID, content size, or maximum back-reference distance (`windowSize`).
-  Note that these values could be wrong, either because of data corruption, or because a 3rd party deliberately spoofs false information.
-  As a consequence, check that values remain within valid application range.
-  For example, do not allocate memory blindly, check that `windowSize` is within expectation.
-  Each application can set its own limits, depending on local restrictions.
-  For extended interoperability, it is recommended to support `windowSize` of at least 8 MB.
-
-  ZSTD_decompressContinue() needs previous data blocks during decompression, up to `windowSize` bytes.
-  ZSTD_decompressContinue() is very sensitive to contiguity,
-  if 2 blocks don't follow each other, make sure that either the compressor breaks contiguity at the same place,
-  or that previous contiguous segment is large enough to properly handle maximum back-reference distance.
-  There are multiple ways to guarantee this condition.
-
-  The most memory efficient way is to use a round buffer of sufficient size.
-  Sufficient size is determined by invoking ZSTD_decodingBufferSize_min(),
-  which can @return an error code if required value is too large for current system (in 32-bits mode).
-  In a round buffer methodology, ZSTD_decompressContinue() decompresses each block next to previous one,
-  up to the moment there is not enough room left in the buffer to guarantee decoding another full block,
-  which maximum size is provided in `ZSTD_frameHeader` structure, field `blockSizeMax`.
-  At which point, decoding can resume from the beginning of the buffer.
-  Note that already decoded data stored in the buffer should be flushed before being overwritten.
-
-  There are alternatives possible, for example using two or more buffers of size `windowSize` each, though they consume more memory.
-
-  Finally, if you control the compression process, you can also ignore all buffer size rules,
-  as long as the encoder and decoder progress in "lock-step",
-  aka use exactly the same buffer sizes, break contiguity at the same place, etc.
-
-  Once buffers are setup, start decompression, with ZSTD_decompressBegin().
-  If decompression requires a dictionary, use ZSTD_decompressBegin_usingDict() or ZSTD_decompressBegin_usingDDict().
-
-  Then use ZSTD_nextSrcSizeToDecompress() and ZSTD_decompressContinue() alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how many bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() requires this _exact_ amount of bytes, or it will fail.
-
- @result of ZSTD_decompressContinue() is the number of bytes regenerated within 'dst' (necessarily <= dstCapacity).
-  It can be zero : it just means ZSTD_decompressContinue() has decoded some metadata item.
-  It can also be an error code, which can be tested with ZSTD_isError().
-
-  A frame is fully decoded when ZSTD_nextSrcSizeToDecompress() returns zero.
-  Context can then be reset to start a new decompression.
-
-  Note : it's possible to know if next input to present is a header or a block, using ZSTD_nextInputType().
-  This information is not required to properly decode a frame.
-
-  == Special case : skippable frames ==
-
-  Skippable frames allow integration of user-defined data into a flow of concatenated frames.
-  Skippable frames will be ignored (skipped) by decompressor.
-  The format of skippable frames is as follows :
-  a) Skippable frame ID - 4 Bytes, Little endian format, any value from 0x184D2A50 to 0x184D2A5F
-  b) Frame Size - 4 Bytes, Little endian format, unsigned 32-bits
-  c) Frame Content - any content (User Data) of length equal to Frame Size
-  For skippable frames ZSTD_getFrameHeader() returns zfhPtr->frameType==ZSTD_skippableFrame.
-  For skippable frames ZSTD_decompressContinue() always returns 0 : it only skips the content.
-*/
-
-/*=====   Buffer-less streaming decompression functions  =====*/
-typedef enum { ZSTD_frame, ZSTD_skippableFrame } ZSTD_frameType_e;
-typedef struct {
-    unsigned long long frameContentSize; /* if == ZSTD_CONTENTSIZE_UNKNOWN, it means this field is not available. 0 means "empty" */
-    unsigned long long windowSize;       /* can be very large, up to <= frameContentSize */
-    unsigned blockSizeMax;
-    ZSTD_frameType_e frameType;          /* if == ZSTD_skippableFrame, frameContentSize is the size of skippable content */
-    unsigned headerSize;
-    unsigned dictID;
-    unsigned checksumFlag;
-} ZSTD_frameHeader;
-
-/*! ZSTD_getFrameHeader() :
- *  decode Frame Header, or requires larger `srcSize`.
- * @return : 0, `zfhPtr` is correctly filled,
- *          >0, `srcSize` is too small, value is wanted `srcSize` amount,
- *           or an error code, which can be tested using ZSTD_isError() */
-ZSTDLIB_API size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t srcSize);   /**< doesn't consume input */
-/*! ZSTD_getFrameHeader_advanced() :
- *  same as ZSTD_getFrameHeader(),
- *  with added capability to select a format (like ZSTD_f_zstd1_magicless) */
-ZSTDLIB_API size_t ZSTD_getFrameHeader_advanced(ZSTD_frameHeader* zfhPtr, const void* src, size_t srcSize, ZSTD_format_e format);
-ZSTDLIB_API size_t ZSTD_decodingBufferSize_min(unsigned long long windowSize, unsigned long long frameContentSize);  /**< when frame content size is not known, pass in frameContentSize == ZSTD_CONTENTSIZE_UNKNOWN */
-
-ZSTDLIB_API size_t ZSTD_decompressBegin(ZSTD_DCtx* dctx);
-ZSTDLIB_API size_t ZSTD_decompressBegin_usingDict(ZSTD_DCtx* dctx, const void* dict, size_t dictSize);
-ZSTDLIB_API size_t ZSTD_decompressBegin_usingDDict(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict);
-
-ZSTDLIB_API size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx);
-ZSTDLIB_API size_t ZSTD_decompressContinue(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-/* misc */
-ZSTDLIB_API void   ZSTD_copyDCtx(ZSTD_DCtx* dctx, const ZSTD_DCtx* preparedDCtx);
-typedef enum { ZSTDnit_frameHeader, ZSTDnit_blockHeader, ZSTDnit_block, ZSTDnit_lastBlock, ZSTDnit_checksum, ZSTDnit_skippableFrame } ZSTD_nextInputType_e;
-ZSTDLIB_API ZSTD_nextInputType_e ZSTD_nextInputType(ZSTD_DCtx* dctx);
-
-
-
-
-/* ============================ */
-/**       Block level API       */
-/* ============================ */
-
-/*!
-    Block functions produce and decode raw zstd blocks, without frame metadata.
-    Frame metadata cost is typically ~18 bytes, which can be non-negligible for very small blocks (< 100 bytes).
-    User will have to take in charge required information to regenerate data, such as compressed and content sizes.
-
-    A few rules to respect :
-    - Compressing and decompressing require a context structure
-      + Use ZSTD_createCCtx() and ZSTD_createDCtx()
-    - It is necessary to init context before starting
-      + compression : any ZSTD_compressBegin*() variant, including with dictionary
-      + decompression : any ZSTD_decompressBegin*() variant, including with dictionary
-      + copyCCtx() and copyDCtx() can be used too
-    - Block size is limited, it must be <= ZSTD_getBlockSize() <= ZSTD_BLOCKSIZE_MAX == 128 KB
-      + If input is larger than a block size, it's necessary to split input data into multiple blocks
-      + For inputs larger than a single block, really consider using regular ZSTD_compress() instead.
-        Frame metadata is not that costly, and quickly becomes negligible as source size grows larger.
-    - When a block is considered not compressible enough, ZSTD_compressBlock() result will be zero.
-      In which case, nothing is produced into `dst` !
-      + User must test for such outcome and deal directly with uncompressed data
-      + ZSTD_decompressBlock() doesn't accept uncompressed data as input !!!
-      + In case of multiple successive blocks, should some of them be uncompressed,
-        decoder must be informed of their existence in order to follow proper history.
-        Use ZSTD_insertBlock() for such a case.
-*/
-
-/*=====   Raw zstd block functions  =====*/
-ZSTDLIB_API size_t ZSTD_getBlockSize   (const ZSTD_CCtx* cctx);
-ZSTDLIB_API size_t ZSTD_compressBlock  (ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-ZSTDLIB_API size_t ZSTD_decompressBlock(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-ZSTDLIB_API size_t ZSTD_insertBlock    (ZSTD_DCtx* dctx, const void* blockStart, size_t blockSize);  /**< insert uncompressed block into `dctx` history. Useful for multi-blocks decompression. */
-
-
-#endif   /* ZSTD_H_ZSTD_STATIC_LINKING_ONLY */
-
-#if defined (__cplusplus)
-}
-#endif
diff --git a/vendor/github.com/DataDog/zstd/zstd_common.c b/vendor/github.com/DataDog/zstd/zstd_common.c
deleted file mode 100644
index 667f4a2..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_common.c
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <stdlib.h>      /* malloc, calloc, free */
-#include <string.h>      /* memset */
-#include "error_private.h"
-#include "zstd_internal.h"
-
-
-/*-****************************************
-*  Version
-******************************************/
-unsigned ZSTD_versionNumber(void) { return ZSTD_VERSION_NUMBER; }
-
-const char* ZSTD_versionString(void) { return ZSTD_VERSION_STRING; }
-
-
-/*-****************************************
-*  ZSTD Error Management
-******************************************/
-#undef ZSTD_isError   /* defined within zstd_internal.h */
-/*! ZSTD_isError() :
- *  tells if a return value is an error code
- *  symbol is required for external callers */
-unsigned ZSTD_isError(size_t code) { return ERR_isError(code); }
-
-/*! ZSTD_getErrorName() :
- *  provides error code string from function result (useful for debugging) */
-const char* ZSTD_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-/*! ZSTD_getError() :
- *  convert a `size_t` function result into a proper ZSTD_errorCode enum */
-ZSTD_ErrorCode ZSTD_getErrorCode(size_t code) { return ERR_getErrorCode(code); }
-
-/*! ZSTD_getErrorString() :
- *  provides error code string from enum */
-const char* ZSTD_getErrorString(ZSTD_ErrorCode code) { return ERR_getErrorString(code); }
-
-
-
-/*=**************************************************************
-*  Custom allocator
-****************************************************************/
-void* ZSTD_malloc(size_t size, ZSTD_customMem customMem)
-{
-    if (customMem.customAlloc)
-        return customMem.customAlloc(customMem.opaque, size);
-    return malloc(size);
-}
-
-void* ZSTD_calloc(size_t size, ZSTD_customMem customMem)
-{
-    if (customMem.customAlloc) {
-        /* calloc implemented as malloc+memset;
-         * not as efficient as calloc, but next best guess for custom malloc */
-        void* const ptr = customMem.customAlloc(customMem.opaque, size);
-        memset(ptr, 0, size);
-        return ptr;
-    }
-    return calloc(1, size);
-}
-
-void ZSTD_free(void* ptr, ZSTD_customMem customMem)
-{
-    if (ptr!=NULL) {
-        if (customMem.customFree)
-            customMem.customFree(customMem.opaque, ptr);
-        else
-            free(ptr);
-    }
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_compress.c b/vendor/github.com/DataDog/zstd/zstd_compress.c
deleted file mode 100644
index 1476512..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_compress.c
+++ /dev/null
@@ -1,4475 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <limits.h>         /* INT_MAX */
-#include <string.h>         /* memset */
-#include "cpu.h"
-#include "mem.h"
-#include "hist.h"           /* HIST_countFast_wksp */
-#define FSE_STATIC_LINKING_ONLY   /* FSE_encodeSymbol */
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "zstd_compress_internal.h"
-#include "zstd_fast.h"
-#include "zstd_double_fast.h"
-#include "zstd_lazy.h"
-#include "zstd_opt.h"
-#include "zstd_ldm.h"
-
-
-/*-*************************************
-*  Helper functions
-***************************************/
-size_t ZSTD_compressBound(size_t srcSize) {
-    return ZSTD_COMPRESSBOUND(srcSize);
-}
-
-
-/*-*************************************
-*  Context memory management
-***************************************/
-struct ZSTD_CDict_s {
-    void* dictBuffer;
-    const void* dictContent;
-    size_t dictContentSize;
-    void* workspace;
-    size_t workspaceSize;
-    ZSTD_matchState_t matchState;
-    ZSTD_compressedBlockState_t cBlockState;
-    ZSTD_customMem customMem;
-    U32 dictID;
-};  /* typedef'd to ZSTD_CDict within "zstd.h" */
-
-ZSTD_CCtx* ZSTD_createCCtx(void)
-{
-    return ZSTD_createCCtx_advanced(ZSTD_defaultCMem);
-}
-
-static void ZSTD_initCCtx(ZSTD_CCtx* cctx, ZSTD_customMem memManager)
-{
-    assert(cctx != NULL);
-    memset(cctx, 0, sizeof(*cctx));
-    cctx->customMem = memManager;
-    cctx->bmi2 = ZSTD_cpuid_bmi2(ZSTD_cpuid());
-    {   size_t const err = ZSTD_CCtx_reset(cctx, ZSTD_reset_parameters);
-        assert(!ZSTD_isError(err));
-        (void)err;
-    }
-}
-
-ZSTD_CCtx* ZSTD_createCCtx_advanced(ZSTD_customMem customMem)
-{
-    ZSTD_STATIC_ASSERT(zcss_init==0);
-    ZSTD_STATIC_ASSERT(ZSTD_CONTENTSIZE_UNKNOWN==(0ULL - 1));
-    if (!customMem.customAlloc ^ !customMem.customFree) return NULL;
-    {   ZSTD_CCtx* const cctx = (ZSTD_CCtx*)ZSTD_malloc(sizeof(ZSTD_CCtx), customMem);
-        if (!cctx) return NULL;
-        ZSTD_initCCtx(cctx, customMem);
-        return cctx;
-    }
-}
-
-ZSTD_CCtx* ZSTD_initStaticCCtx(void *workspace, size_t workspaceSize)
-{
-    ZSTD_CCtx* const cctx = (ZSTD_CCtx*) workspace;
-    if (workspaceSize <= sizeof(ZSTD_CCtx)) return NULL;  /* minimum size */
-    if ((size_t)workspace & 7) return NULL;  /* must be 8-aligned */
-    memset(workspace, 0, workspaceSize);   /* may be a bit generous, could memset be smaller ? */
-    cctx->staticSize = workspaceSize;
-    cctx->workSpace = (void*)(cctx+1);
-    cctx->workSpaceSize = workspaceSize - sizeof(ZSTD_CCtx);
-
-    /* statically sized space. entropyWorkspace never moves (but prev/next block swap places) */
-    if (cctx->workSpaceSize < HUF_WORKSPACE_SIZE + 2 * sizeof(ZSTD_compressedBlockState_t)) return NULL;
-    assert(((size_t)cctx->workSpace & (sizeof(void*)-1)) == 0);   /* ensure correct alignment */
-    cctx->blockState.prevCBlock = (ZSTD_compressedBlockState_t*)cctx->workSpace;
-    cctx->blockState.nextCBlock = cctx->blockState.prevCBlock + 1;
-    {
-        void* const ptr = cctx->blockState.nextCBlock + 1;
-        cctx->entropyWorkspace = (U32*)ptr;
-    }
-    cctx->bmi2 = ZSTD_cpuid_bmi2(ZSTD_cpuid());
-    return cctx;
-}
-
-/**
- * Clears and frees all of the dictionaries in the CCtx.
- */
-static void ZSTD_clearAllDicts(ZSTD_CCtx* cctx)
-{
-    ZSTD_free(cctx->localDict.dictBuffer, cctx->customMem);
-    ZSTD_freeCDict(cctx->localDict.cdict);
-    memset(&cctx->localDict, 0, sizeof(cctx->localDict));
-    memset(&cctx->prefixDict, 0, sizeof(cctx->prefixDict));
-    cctx->cdict = NULL;
-}
-
-static size_t ZSTD_sizeof_localDict(ZSTD_localDict dict)
-{
-    size_t const bufferSize = dict.dictBuffer != NULL ? dict.dictSize : 0;
-    size_t const cdictSize = ZSTD_sizeof_CDict(dict.cdict);
-    return bufferSize + cdictSize;
-}
-
-static void ZSTD_freeCCtxContent(ZSTD_CCtx* cctx)
-{
-    assert(cctx != NULL);
-    assert(cctx->staticSize == 0);
-    ZSTD_free(cctx->workSpace, cctx->customMem); cctx->workSpace = NULL;
-    ZSTD_clearAllDicts(cctx);
-#ifdef ZSTD_MULTITHREAD
-    ZSTDMT_freeCCtx(cctx->mtctx); cctx->mtctx = NULL;
-#endif
-}
-
-size_t ZSTD_freeCCtx(ZSTD_CCtx* cctx)
-{
-    if (cctx==NULL) return 0;   /* support free on NULL */
-    RETURN_ERROR_IF(cctx->staticSize, memory_allocation,
-                    "not compatible with static CCtx");
-    ZSTD_freeCCtxContent(cctx);
-    ZSTD_free(cctx, cctx->customMem);
-    return 0;
-}
-
-
-static size_t ZSTD_sizeof_mtctx(const ZSTD_CCtx* cctx)
-{
-#ifdef ZSTD_MULTITHREAD
-    return ZSTDMT_sizeof_CCtx(cctx->mtctx);
-#else
-    (void)cctx;
-    return 0;
-#endif
-}
-
-
-size_t ZSTD_sizeof_CCtx(const ZSTD_CCtx* cctx)
-{
-    if (cctx==NULL) return 0;   /* support sizeof on NULL */
-    return sizeof(*cctx) + cctx->workSpaceSize
-           + ZSTD_sizeof_localDict(cctx->localDict)
-           + ZSTD_sizeof_mtctx(cctx);
-}
-
-size_t ZSTD_sizeof_CStream(const ZSTD_CStream* zcs)
-{
-    return ZSTD_sizeof_CCtx(zcs);  /* same object */
-}
-
-/* private API call, for dictBuilder only */
-const seqStore_t* ZSTD_getSeqStore(const ZSTD_CCtx* ctx) { return &(ctx->seqStore); }
-
-static ZSTD_CCtx_params ZSTD_makeCCtxParamsFromCParams(
-        ZSTD_compressionParameters cParams)
-{
-    ZSTD_CCtx_params cctxParams;
-    memset(&cctxParams, 0, sizeof(cctxParams));
-    cctxParams.cParams = cParams;
-    cctxParams.compressionLevel = ZSTD_CLEVEL_DEFAULT;  /* should not matter, as all cParams are presumed properly defined */
-    assert(!ZSTD_checkCParams(cParams));
-    cctxParams.fParams.contentSizeFlag = 1;
-    return cctxParams;
-}
-
-static ZSTD_CCtx_params* ZSTD_createCCtxParams_advanced(
-        ZSTD_customMem customMem)
-{
-    ZSTD_CCtx_params* params;
-    if (!customMem.customAlloc ^ !customMem.customFree) return NULL;
-    params = (ZSTD_CCtx_params*)ZSTD_calloc(
-            sizeof(ZSTD_CCtx_params), customMem);
-    if (!params) { return NULL; }
-    params->customMem = customMem;
-    params->compressionLevel = ZSTD_CLEVEL_DEFAULT;
-    params->fParams.contentSizeFlag = 1;
-    return params;
-}
-
-ZSTD_CCtx_params* ZSTD_createCCtxParams(void)
-{
-    return ZSTD_createCCtxParams_advanced(ZSTD_defaultCMem);
-}
-
-size_t ZSTD_freeCCtxParams(ZSTD_CCtx_params* params)
-{
-    if (params == NULL) { return 0; }
-    ZSTD_free(params, params->customMem);
-    return 0;
-}
-
-size_t ZSTD_CCtxParams_reset(ZSTD_CCtx_params* params)
-{
-    return ZSTD_CCtxParams_init(params, ZSTD_CLEVEL_DEFAULT);
-}
-
-size_t ZSTD_CCtxParams_init(ZSTD_CCtx_params* cctxParams, int compressionLevel) {
-    RETURN_ERROR_IF(!cctxParams, GENERIC);
-    memset(cctxParams, 0, sizeof(*cctxParams));
-    cctxParams->compressionLevel = compressionLevel;
-    cctxParams->fParams.contentSizeFlag = 1;
-    return 0;
-}
-
-size_t ZSTD_CCtxParams_init_advanced(ZSTD_CCtx_params* cctxParams, ZSTD_parameters params)
-{
-    RETURN_ERROR_IF(!cctxParams, GENERIC);
-    FORWARD_IF_ERROR( ZSTD_checkCParams(params.cParams) );
-    memset(cctxParams, 0, sizeof(*cctxParams));
-    cctxParams->cParams = params.cParams;
-    cctxParams->fParams = params.fParams;
-    cctxParams->compressionLevel = ZSTD_CLEVEL_DEFAULT;   /* should not matter, as all cParams are presumed properly defined */
-    assert(!ZSTD_checkCParams(params.cParams));
-    return 0;
-}
-
-/* ZSTD_assignParamsToCCtxParams() :
- * params is presumed valid at this stage */
-static ZSTD_CCtx_params ZSTD_assignParamsToCCtxParams(
-        ZSTD_CCtx_params cctxParams, ZSTD_parameters params)
-{
-    ZSTD_CCtx_params ret = cctxParams;
-    ret.cParams = params.cParams;
-    ret.fParams = params.fParams;
-    ret.compressionLevel = ZSTD_CLEVEL_DEFAULT;   /* should not matter, as all cParams are presumed properly defined */
-    assert(!ZSTD_checkCParams(params.cParams));
-    return ret;
-}
-
-ZSTD_bounds ZSTD_cParam_getBounds(ZSTD_cParameter param)
-{
-    ZSTD_bounds bounds = { 0, 0, 0 };
-
-    switch(param)
-    {
-    case ZSTD_c_compressionLevel:
-        bounds.lowerBound = ZSTD_minCLevel();
-        bounds.upperBound = ZSTD_maxCLevel();
-        return bounds;
-
-    case ZSTD_c_windowLog:
-        bounds.lowerBound = ZSTD_WINDOWLOG_MIN;
-        bounds.upperBound = ZSTD_WINDOWLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_hashLog:
-        bounds.lowerBound = ZSTD_HASHLOG_MIN;
-        bounds.upperBound = ZSTD_HASHLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_chainLog:
-        bounds.lowerBound = ZSTD_CHAINLOG_MIN;
-        bounds.upperBound = ZSTD_CHAINLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_searchLog:
-        bounds.lowerBound = ZSTD_SEARCHLOG_MIN;
-        bounds.upperBound = ZSTD_SEARCHLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_minMatch:
-        bounds.lowerBound = ZSTD_MINMATCH_MIN;
-        bounds.upperBound = ZSTD_MINMATCH_MAX;
-        return bounds;
-
-    case ZSTD_c_targetLength:
-        bounds.lowerBound = ZSTD_TARGETLENGTH_MIN;
-        bounds.upperBound = ZSTD_TARGETLENGTH_MAX;
-        return bounds;
-
-    case ZSTD_c_strategy:
-        bounds.lowerBound = ZSTD_STRATEGY_MIN;
-        bounds.upperBound = ZSTD_STRATEGY_MAX;
-        return bounds;
-
-    case ZSTD_c_contentSizeFlag:
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_checksumFlag:
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_dictIDFlag:
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_nbWorkers:
-        bounds.lowerBound = 0;
-#ifdef ZSTD_MULTITHREAD
-        bounds.upperBound = ZSTDMT_NBWORKERS_MAX;
-#else
-        bounds.upperBound = 0;
-#endif
-        return bounds;
-
-    case ZSTD_c_jobSize:
-        bounds.lowerBound = 0;
-#ifdef ZSTD_MULTITHREAD
-        bounds.upperBound = ZSTDMT_JOBSIZE_MAX;
-#else
-        bounds.upperBound = 0;
-#endif
-        return bounds;
-
-    case ZSTD_c_overlapLog:
-        bounds.lowerBound = ZSTD_OVERLAPLOG_MIN;
-        bounds.upperBound = ZSTD_OVERLAPLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_enableLongDistanceMatching:
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_ldmHashLog:
-        bounds.lowerBound = ZSTD_LDM_HASHLOG_MIN;
-        bounds.upperBound = ZSTD_LDM_HASHLOG_MAX;
-        return bounds;
-
-    case ZSTD_c_ldmMinMatch:
-        bounds.lowerBound = ZSTD_LDM_MINMATCH_MIN;
-        bounds.upperBound = ZSTD_LDM_MINMATCH_MAX;
-        return bounds;
-
-    case ZSTD_c_ldmBucketSizeLog:
-        bounds.lowerBound = ZSTD_LDM_BUCKETSIZELOG_MIN;
-        bounds.upperBound = ZSTD_LDM_BUCKETSIZELOG_MAX;
-        return bounds;
-
-    case ZSTD_c_ldmHashRateLog:
-        bounds.lowerBound = ZSTD_LDM_HASHRATELOG_MIN;
-        bounds.upperBound = ZSTD_LDM_HASHRATELOG_MAX;
-        return bounds;
-
-    /* experimental parameters */
-    case ZSTD_c_rsyncable:
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_forceMaxWindow :
-        bounds.lowerBound = 0;
-        bounds.upperBound = 1;
-        return bounds;
-
-    case ZSTD_c_format:
-        ZSTD_STATIC_ASSERT(ZSTD_f_zstd1 < ZSTD_f_zstd1_magicless);
-        bounds.lowerBound = ZSTD_f_zstd1;
-        bounds.upperBound = ZSTD_f_zstd1_magicless;   /* note : how to ensure at compile time that this is the highest value enum ? */
-        return bounds;
-
-    case ZSTD_c_forceAttachDict:
-        ZSTD_STATIC_ASSERT(ZSTD_dictDefaultAttach < ZSTD_dictForceCopy);
-        bounds.lowerBound = ZSTD_dictDefaultAttach;
-        bounds.upperBound = ZSTD_dictForceCopy;       /* note : how to ensure at compile time that this is the highest value enum ? */
-        return bounds;
-
-    case ZSTD_c_literalCompressionMode:
-        ZSTD_STATIC_ASSERT(ZSTD_lcm_auto < ZSTD_lcm_huffman && ZSTD_lcm_huffman < ZSTD_lcm_uncompressed);
-        bounds.lowerBound = ZSTD_lcm_auto;
-        bounds.upperBound = ZSTD_lcm_uncompressed;
-        return bounds;
-
-    case ZSTD_c_targetCBlockSize:
-        bounds.lowerBound = ZSTD_TARGETCBLOCKSIZE_MIN;
-        bounds.upperBound = ZSTD_TARGETCBLOCKSIZE_MAX;
-        return bounds;
-
-    default:
-        {   ZSTD_bounds const boundError = { ERROR(parameter_unsupported), 0, 0 };
-            return boundError;
-        }
-    }
-}
-
-/* ZSTD_cParam_withinBounds:
- * @return 1 if value is within cParam bounds,
- * 0 otherwise */
-static int ZSTD_cParam_withinBounds(ZSTD_cParameter cParam, int value)
-{
-    ZSTD_bounds const bounds = ZSTD_cParam_getBounds(cParam);
-    if (ZSTD_isError(bounds.error)) return 0;
-    if (value < bounds.lowerBound) return 0;
-    if (value > bounds.upperBound) return 0;
-    return 1;
-}
-
-/* ZSTD_cParam_clampBounds:
- * Clamps the value into the bounded range.
- */
-static size_t ZSTD_cParam_clampBounds(ZSTD_cParameter cParam, int* value)
-{
-    ZSTD_bounds const bounds = ZSTD_cParam_getBounds(cParam);
-    if (ZSTD_isError(bounds.error)) return bounds.error;
-    if (*value < bounds.lowerBound) *value = bounds.lowerBound;
-    if (*value > bounds.upperBound) *value = bounds.upperBound;
-    return 0;
-}
-
-#define BOUNDCHECK(cParam, val) { \
-    RETURN_ERROR_IF(!ZSTD_cParam_withinBounds(cParam,val), \
-                    parameter_outOfBound); \
-}
-
-
-static int ZSTD_isUpdateAuthorized(ZSTD_cParameter param)
-{
-    switch(param)
-    {
-    case ZSTD_c_compressionLevel:
-    case ZSTD_c_hashLog:
-    case ZSTD_c_chainLog:
-    case ZSTD_c_searchLog:
-    case ZSTD_c_minMatch:
-    case ZSTD_c_targetLength:
-    case ZSTD_c_strategy:
-        return 1;
-
-    case ZSTD_c_format:
-    case ZSTD_c_windowLog:
-    case ZSTD_c_contentSizeFlag:
-    case ZSTD_c_checksumFlag:
-    case ZSTD_c_dictIDFlag:
-    case ZSTD_c_forceMaxWindow :
-    case ZSTD_c_nbWorkers:
-    case ZSTD_c_jobSize:
-    case ZSTD_c_overlapLog:
-    case ZSTD_c_rsyncable:
-    case ZSTD_c_enableLongDistanceMatching:
-    case ZSTD_c_ldmHashLog:
-    case ZSTD_c_ldmMinMatch:
-    case ZSTD_c_ldmBucketSizeLog:
-    case ZSTD_c_ldmHashRateLog:
-    case ZSTD_c_forceAttachDict:
-    case ZSTD_c_literalCompressionMode:
-    case ZSTD_c_targetCBlockSize:
-    default:
-        return 0;
-    }
-}
-
-size_t ZSTD_CCtx_setParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, int value)
-{
-    DEBUGLOG(4, "ZSTD_CCtx_setParameter (%i, %i)", (int)param, value);
-    if (cctx->streamStage != zcss_init) {
-        if (ZSTD_isUpdateAuthorized(param)) {
-            cctx->cParamsChanged = 1;
-        } else {
-            RETURN_ERROR(stage_wrong);
-    }   }
-
-    switch(param)
-    {
-    case ZSTD_c_nbWorkers:
-        RETURN_ERROR_IF((value!=0) && cctx->staticSize, parameter_unsupported,
-                        "MT not compatible with static alloc");
-        break;
-
-    case ZSTD_c_compressionLevel:
-    case ZSTD_c_windowLog:
-    case ZSTD_c_hashLog:
-    case ZSTD_c_chainLog:
-    case ZSTD_c_searchLog:
-    case ZSTD_c_minMatch:
-    case ZSTD_c_targetLength:
-    case ZSTD_c_strategy:
-    case ZSTD_c_ldmHashRateLog:
-    case ZSTD_c_format:
-    case ZSTD_c_contentSizeFlag:
-    case ZSTD_c_checksumFlag:
-    case ZSTD_c_dictIDFlag:
-    case ZSTD_c_forceMaxWindow:
-    case ZSTD_c_forceAttachDict:
-    case ZSTD_c_literalCompressionMode:
-    case ZSTD_c_jobSize:
-    case ZSTD_c_overlapLog:
-    case ZSTD_c_rsyncable:
-    case ZSTD_c_enableLongDistanceMatching:
-    case ZSTD_c_ldmHashLog:
-    case ZSTD_c_ldmMinMatch:
-    case ZSTD_c_ldmBucketSizeLog:
-    case ZSTD_c_targetCBlockSize:
-        break;
-
-    default: RETURN_ERROR(parameter_unsupported);
-    }
-    return ZSTD_CCtxParams_setParameter(&cctx->requestedParams, param, value);
-}
-
-size_t ZSTD_CCtxParams_setParameter(ZSTD_CCtx_params* CCtxParams,
-                                    ZSTD_cParameter param, int value)
-{
-    DEBUGLOG(4, "ZSTD_CCtxParams_setParameter (%i, %i)", (int)param, value);
-    switch(param)
-    {
-    case ZSTD_c_format :
-        BOUNDCHECK(ZSTD_c_format, value);
-        CCtxParams->format = (ZSTD_format_e)value;
-        return (size_t)CCtxParams->format;
-
-    case ZSTD_c_compressionLevel : {
-        FORWARD_IF_ERROR(ZSTD_cParam_clampBounds(param, &value));
-        if (value) {  /* 0 : does not change current level */
-            CCtxParams->compressionLevel = value;
-        }
-        if (CCtxParams->compressionLevel >= 0) return CCtxParams->compressionLevel;
-        return 0;  /* return type (size_t) cannot represent negative values */
-    }
-
-    case ZSTD_c_windowLog :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_windowLog, value);
-        CCtxParams->cParams.windowLog = value;
-        return CCtxParams->cParams.windowLog;
-
-    case ZSTD_c_hashLog :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_hashLog, value);
-        CCtxParams->cParams.hashLog = value;
-        return CCtxParams->cParams.hashLog;
-
-    case ZSTD_c_chainLog :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_chainLog, value);
-        CCtxParams->cParams.chainLog = value;
-        return CCtxParams->cParams.chainLog;
-
-    case ZSTD_c_searchLog :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_searchLog, value);
-        CCtxParams->cParams.searchLog = value;
-        return value;
-
-    case ZSTD_c_minMatch :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_minMatch, value);
-        CCtxParams->cParams.minMatch = value;
-        return CCtxParams->cParams.minMatch;
-
-    case ZSTD_c_targetLength :
-        BOUNDCHECK(ZSTD_c_targetLength, value);
-        CCtxParams->cParams.targetLength = value;
-        return CCtxParams->cParams.targetLength;
-
-    case ZSTD_c_strategy :
-        if (value!=0)   /* 0 => use default */
-            BOUNDCHECK(ZSTD_c_strategy, value);
-        CCtxParams->cParams.strategy = (ZSTD_strategy)value;
-        return (size_t)CCtxParams->cParams.strategy;
-
-    case ZSTD_c_contentSizeFlag :
-        /* Content size written in frame header _when known_ (default:1) */
-        DEBUGLOG(4, "set content size flag = %u", (value!=0));
-        CCtxParams->fParams.contentSizeFlag = value != 0;
-        return CCtxParams->fParams.contentSizeFlag;
-
-    case ZSTD_c_checksumFlag :
-        /* A 32-bits content checksum will be calculated and written at end of frame (default:0) */
-        CCtxParams->fParams.checksumFlag = value != 0;
-        return CCtxParams->fParams.checksumFlag;
-
-    case ZSTD_c_dictIDFlag : /* When applicable, dictionary's dictID is provided in frame header (default:1) */
-        DEBUGLOG(4, "set dictIDFlag = %u", (value!=0));
-        CCtxParams->fParams.noDictIDFlag = !value;
-        return !CCtxParams->fParams.noDictIDFlag;
-
-    case ZSTD_c_forceMaxWindow :
-        CCtxParams->forceWindow = (value != 0);
-        return CCtxParams->forceWindow;
-
-    case ZSTD_c_forceAttachDict : {
-        const ZSTD_dictAttachPref_e pref = (ZSTD_dictAttachPref_e)value;
-        BOUNDCHECK(ZSTD_c_forceAttachDict, pref);
-        CCtxParams->attachDictPref = pref;
-        return CCtxParams->attachDictPref;
-    }
-
-    case ZSTD_c_literalCompressionMode : {
-        const ZSTD_literalCompressionMode_e lcm = (ZSTD_literalCompressionMode_e)value;
-        BOUNDCHECK(ZSTD_c_literalCompressionMode, lcm);
-        CCtxParams->literalCompressionMode = lcm;
-        return CCtxParams->literalCompressionMode;
-    }
-
-    case ZSTD_c_nbWorkers :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR_IF(value!=0, parameter_unsupported, "not compiled with multithreading");
-        return 0;
-#else
-        FORWARD_IF_ERROR(ZSTD_cParam_clampBounds(param, &value));
-        CCtxParams->nbWorkers = value;
-        return CCtxParams->nbWorkers;
-#endif
-
-    case ZSTD_c_jobSize :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR_IF(value!=0, parameter_unsupported, "not compiled with multithreading");
-        return 0;
-#else
-        /* Adjust to the minimum non-default value. */
-        if (value != 0 && value < ZSTDMT_JOBSIZE_MIN)
-            value = ZSTDMT_JOBSIZE_MIN;
-        FORWARD_IF_ERROR(ZSTD_cParam_clampBounds(param, &value));
-        assert(value >= 0);
-        CCtxParams->jobSize = value;
-        return CCtxParams->jobSize;
-#endif
-
-    case ZSTD_c_overlapLog :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR_IF(value!=0, parameter_unsupported, "not compiled with multithreading");
-        return 0;
-#else
-        FORWARD_IF_ERROR(ZSTD_cParam_clampBounds(ZSTD_c_overlapLog, &value));
-        CCtxParams->overlapLog = value;
-        return CCtxParams->overlapLog;
-#endif
-
-    case ZSTD_c_rsyncable :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR_IF(value!=0, parameter_unsupported, "not compiled with multithreading");
-        return 0;
-#else
-        FORWARD_IF_ERROR(ZSTD_cParam_clampBounds(ZSTD_c_overlapLog, &value));
-        CCtxParams->rsyncable = value;
-        return CCtxParams->rsyncable;
-#endif
-
-    case ZSTD_c_enableLongDistanceMatching :
-        CCtxParams->ldmParams.enableLdm = (value!=0);
-        return CCtxParams->ldmParams.enableLdm;
-
-    case ZSTD_c_ldmHashLog :
-        if (value!=0)   /* 0 ==> auto */
-            BOUNDCHECK(ZSTD_c_ldmHashLog, value);
-        CCtxParams->ldmParams.hashLog = value;
-        return CCtxParams->ldmParams.hashLog;
-
-    case ZSTD_c_ldmMinMatch :
-        if (value!=0)   /* 0 ==> default */
-            BOUNDCHECK(ZSTD_c_ldmMinMatch, value);
-        CCtxParams->ldmParams.minMatchLength = value;
-        return CCtxParams->ldmParams.minMatchLength;
-
-    case ZSTD_c_ldmBucketSizeLog :
-        if (value!=0)   /* 0 ==> default */
-            BOUNDCHECK(ZSTD_c_ldmBucketSizeLog, value);
-        CCtxParams->ldmParams.bucketSizeLog = value;
-        return CCtxParams->ldmParams.bucketSizeLog;
-
-    case ZSTD_c_ldmHashRateLog :
-        RETURN_ERROR_IF(value > ZSTD_WINDOWLOG_MAX - ZSTD_HASHLOG_MIN,
-                        parameter_outOfBound);
-        CCtxParams->ldmParams.hashRateLog = value;
-        return CCtxParams->ldmParams.hashRateLog;
-
-    case ZSTD_c_targetCBlockSize :
-        if (value!=0)   /* 0 ==> default */
-            BOUNDCHECK(ZSTD_c_targetCBlockSize, value);
-        CCtxParams->targetCBlockSize = value;
-        return CCtxParams->targetCBlockSize;
-
-    default: RETURN_ERROR(parameter_unsupported, "unknown parameter");
-    }
-}
-
-size_t ZSTD_CCtx_getParameter(ZSTD_CCtx* cctx, ZSTD_cParameter param, int* value)
-{
-    return ZSTD_CCtxParams_getParameter(&cctx->requestedParams, param, value);
-}
-
-size_t ZSTD_CCtxParams_getParameter(
-        ZSTD_CCtx_params* CCtxParams, ZSTD_cParameter param, int* value)
-{
-    switch(param)
-    {
-    case ZSTD_c_format :
-        *value = CCtxParams->format;
-        break;
-    case ZSTD_c_compressionLevel :
-        *value = CCtxParams->compressionLevel;
-        break;
-    case ZSTD_c_windowLog :
-        *value = (int)CCtxParams->cParams.windowLog;
-        break;
-    case ZSTD_c_hashLog :
-        *value = (int)CCtxParams->cParams.hashLog;
-        break;
-    case ZSTD_c_chainLog :
-        *value = (int)CCtxParams->cParams.chainLog;
-        break;
-    case ZSTD_c_searchLog :
-        *value = CCtxParams->cParams.searchLog;
-        break;
-    case ZSTD_c_minMatch :
-        *value = CCtxParams->cParams.minMatch;
-        break;
-    case ZSTD_c_targetLength :
-        *value = CCtxParams->cParams.targetLength;
-        break;
-    case ZSTD_c_strategy :
-        *value = (unsigned)CCtxParams->cParams.strategy;
-        break;
-    case ZSTD_c_contentSizeFlag :
-        *value = CCtxParams->fParams.contentSizeFlag;
-        break;
-    case ZSTD_c_checksumFlag :
-        *value = CCtxParams->fParams.checksumFlag;
-        break;
-    case ZSTD_c_dictIDFlag :
-        *value = !CCtxParams->fParams.noDictIDFlag;
-        break;
-    case ZSTD_c_forceMaxWindow :
-        *value = CCtxParams->forceWindow;
-        break;
-    case ZSTD_c_forceAttachDict :
-        *value = CCtxParams->attachDictPref;
-        break;
-    case ZSTD_c_literalCompressionMode :
-        *value = CCtxParams->literalCompressionMode;
-        break;
-    case ZSTD_c_nbWorkers :
-#ifndef ZSTD_MULTITHREAD
-        assert(CCtxParams->nbWorkers == 0);
-#endif
-        *value = CCtxParams->nbWorkers;
-        break;
-    case ZSTD_c_jobSize :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR(parameter_unsupported, "not compiled with multithreading");
-#else
-        assert(CCtxParams->jobSize <= INT_MAX);
-        *value = (int)CCtxParams->jobSize;
-        break;
-#endif
-    case ZSTD_c_overlapLog :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR(parameter_unsupported, "not compiled with multithreading");
-#else
-        *value = CCtxParams->overlapLog;
-        break;
-#endif
-    case ZSTD_c_rsyncable :
-#ifndef ZSTD_MULTITHREAD
-        RETURN_ERROR(parameter_unsupported, "not compiled with multithreading");
-#else
-        *value = CCtxParams->rsyncable;
-        break;
-#endif
-    case ZSTD_c_enableLongDistanceMatching :
-        *value = CCtxParams->ldmParams.enableLdm;
-        break;
-    case ZSTD_c_ldmHashLog :
-        *value = CCtxParams->ldmParams.hashLog;
-        break;
-    case ZSTD_c_ldmMinMatch :
-        *value = CCtxParams->ldmParams.minMatchLength;
-        break;
-    case ZSTD_c_ldmBucketSizeLog :
-        *value = CCtxParams->ldmParams.bucketSizeLog;
-        break;
-    case ZSTD_c_ldmHashRateLog :
-        *value = CCtxParams->ldmParams.hashRateLog;
-        break;
-    case ZSTD_c_targetCBlockSize :
-        *value = (int)CCtxParams->targetCBlockSize;
-        break;
-    default: RETURN_ERROR(parameter_unsupported, "unknown parameter");
-    }
-    return 0;
-}
-
-/** ZSTD_CCtx_setParametersUsingCCtxParams() :
- *  just applies `params` into `cctx`
- *  no action is performed, parameters are merely stored.
- *  If ZSTDMT is enabled, parameters are pushed to cctx->mtctx.
- *    This is possible even if a compression is ongoing.
- *    In which case, new parameters will be applied on the fly, starting with next compression job.
- */
-size_t ZSTD_CCtx_setParametersUsingCCtxParams(
-        ZSTD_CCtx* cctx, const ZSTD_CCtx_params* params)
-{
-    DEBUGLOG(4, "ZSTD_CCtx_setParametersUsingCCtxParams");
-    RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-    RETURN_ERROR_IF(cctx->cdict, stage_wrong);
-
-    cctx->requestedParams = *params;
-    return 0;
-}
-
-ZSTDLIB_API size_t ZSTD_CCtx_setPledgedSrcSize(ZSTD_CCtx* cctx, unsigned long long pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_CCtx_setPledgedSrcSize to %u bytes", (U32)pledgedSrcSize);
-    RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-    cctx->pledgedSrcSizePlusOne = pledgedSrcSize+1;
-    return 0;
-}
-
-/**
- * Initializes the local dict using the requested parameters.
- * NOTE: This does not use the pledged src size, because it may be used for more
- * than one compression.
- */
-static size_t ZSTD_initLocalDict(ZSTD_CCtx* cctx)
-{
-    ZSTD_localDict* const dl = &cctx->localDict;
-    ZSTD_compressionParameters const cParams = ZSTD_getCParamsFromCCtxParams(
-            &cctx->requestedParams, 0, dl->dictSize);
-    if (dl->dict == NULL) {
-        /* No local dictionary. */
-        assert(dl->dictBuffer == NULL);
-        assert(dl->cdict == NULL);
-        assert(dl->dictSize == 0);
-        return 0;
-    }
-    if (dl->cdict != NULL) {
-        assert(cctx->cdict == dl->cdict);
-        /* Local dictionary already initialized. */
-        return 0;
-    }
-    assert(dl->dictSize > 0);
-    assert(cctx->cdict == NULL);
-    assert(cctx->prefixDict.dict == NULL);
-
-    dl->cdict = ZSTD_createCDict_advanced(
-            dl->dict,
-            dl->dictSize,
-            ZSTD_dlm_byRef,
-            dl->dictContentType,
-            cParams,
-            cctx->customMem);
-    RETURN_ERROR_IF(!dl->cdict, memory_allocation);
-    cctx->cdict = dl->cdict;
-    return 0;
-}
-
-size_t ZSTD_CCtx_loadDictionary_advanced(
-        ZSTD_CCtx* cctx, const void* dict, size_t dictSize,
-        ZSTD_dictLoadMethod_e dictLoadMethod, ZSTD_dictContentType_e dictContentType)
-{
-    RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-    RETURN_ERROR_IF(cctx->staticSize, memory_allocation,
-                    "no malloc for static CCtx");
-    DEBUGLOG(4, "ZSTD_CCtx_loadDictionary_advanced (size: %u)", (U32)dictSize);
-    ZSTD_clearAllDicts(cctx);  /* in case one already exists */
-    if (dict == NULL || dictSize == 0)  /* no dictionary mode */
-        return 0;
-    if (dictLoadMethod == ZSTD_dlm_byRef) {
-        cctx->localDict.dict = dict;
-    } else {
-        void* dictBuffer = ZSTD_malloc(dictSize, cctx->customMem);
-        RETURN_ERROR_IF(!dictBuffer, memory_allocation);
-        memcpy(dictBuffer, dict, dictSize);
-        cctx->localDict.dictBuffer = dictBuffer;
-        cctx->localDict.dict = dictBuffer;
-    }
-    cctx->localDict.dictSize = dictSize;
-    cctx->localDict.dictContentType = dictContentType;
-    return 0;
-}
-
-ZSTDLIB_API size_t ZSTD_CCtx_loadDictionary_byReference(
-      ZSTD_CCtx* cctx, const void* dict, size_t dictSize)
-{
-    return ZSTD_CCtx_loadDictionary_advanced(
-            cctx, dict, dictSize, ZSTD_dlm_byRef, ZSTD_dct_auto);
-}
-
-ZSTDLIB_API size_t ZSTD_CCtx_loadDictionary(ZSTD_CCtx* cctx, const void* dict, size_t dictSize)
-{
-    return ZSTD_CCtx_loadDictionary_advanced(
-            cctx, dict, dictSize, ZSTD_dlm_byCopy, ZSTD_dct_auto);
-}
-
-
-size_t ZSTD_CCtx_refCDict(ZSTD_CCtx* cctx, const ZSTD_CDict* cdict)
-{
-    RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-    /* Free the existing local cdict (if any) to save memory. */
-    ZSTD_clearAllDicts(cctx);
-    cctx->cdict = cdict;
-    return 0;
-}
-
-size_t ZSTD_CCtx_refPrefix(ZSTD_CCtx* cctx, const void* prefix, size_t prefixSize)
-{
-    return ZSTD_CCtx_refPrefix_advanced(cctx, prefix, prefixSize, ZSTD_dct_rawContent);
-}
-
-size_t ZSTD_CCtx_refPrefix_advanced(
-        ZSTD_CCtx* cctx, const void* prefix, size_t prefixSize, ZSTD_dictContentType_e dictContentType)
-{
-    RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-    ZSTD_clearAllDicts(cctx);
-    cctx->prefixDict.dict = prefix;
-    cctx->prefixDict.dictSize = prefixSize;
-    cctx->prefixDict.dictContentType = dictContentType;
-    return 0;
-}
-
-/*! ZSTD_CCtx_reset() :
- *  Also dumps dictionary */
-size_t ZSTD_CCtx_reset(ZSTD_CCtx* cctx, ZSTD_ResetDirective reset)
-{
-    if ( (reset == ZSTD_reset_session_only)
-      || (reset == ZSTD_reset_session_and_parameters) ) {
-        cctx->streamStage = zcss_init;
-        cctx->pledgedSrcSizePlusOne = 0;
-    }
-    if ( (reset == ZSTD_reset_parameters)
-      || (reset == ZSTD_reset_session_and_parameters) ) {
-        RETURN_ERROR_IF(cctx->streamStage != zcss_init, stage_wrong);
-        ZSTD_clearAllDicts(cctx);
-        return ZSTD_CCtxParams_reset(&cctx->requestedParams);
-    }
-    return 0;
-}
-
-
-/** ZSTD_checkCParams() :
-    control CParam values remain within authorized range.
-    @return : 0, or an error code if one value is beyond authorized range */
-size_t ZSTD_checkCParams(ZSTD_compressionParameters cParams)
-{
-    BOUNDCHECK(ZSTD_c_windowLog, (int)cParams.windowLog);
-    BOUNDCHECK(ZSTD_c_chainLog,  (int)cParams.chainLog);
-    BOUNDCHECK(ZSTD_c_hashLog,   (int)cParams.hashLog);
-    BOUNDCHECK(ZSTD_c_searchLog, (int)cParams.searchLog);
-    BOUNDCHECK(ZSTD_c_minMatch,  (int)cParams.minMatch);
-    BOUNDCHECK(ZSTD_c_targetLength,(int)cParams.targetLength);
-    BOUNDCHECK(ZSTD_c_strategy,  cParams.strategy);
-    return 0;
-}
-
-/** ZSTD_clampCParams() :
- *  make CParam values within valid range.
- *  @return : valid CParams */
-static ZSTD_compressionParameters
-ZSTD_clampCParams(ZSTD_compressionParameters cParams)
-{
-#   define CLAMP_TYPE(cParam, val, type) {                                \
-        ZSTD_bounds const bounds = ZSTD_cParam_getBounds(cParam);         \
-        if ((int)val<bounds.lowerBound) val=(type)bounds.lowerBound;      \
-        else if ((int)val>bounds.upperBound) val=(type)bounds.upperBound; \
-    }
-#   define CLAMP(cParam, val) CLAMP_TYPE(cParam, val, unsigned)
-    CLAMP(ZSTD_c_windowLog, cParams.windowLog);
-    CLAMP(ZSTD_c_chainLog,  cParams.chainLog);
-    CLAMP(ZSTD_c_hashLog,   cParams.hashLog);
-    CLAMP(ZSTD_c_searchLog, cParams.searchLog);
-    CLAMP(ZSTD_c_minMatch,  cParams.minMatch);
-    CLAMP(ZSTD_c_targetLength,cParams.targetLength);
-    CLAMP_TYPE(ZSTD_c_strategy,cParams.strategy, ZSTD_strategy);
-    return cParams;
-}
-
-/** ZSTD_cycleLog() :
- *  condition for correct operation : hashLog > 1 */
-static U32 ZSTD_cycleLog(U32 hashLog, ZSTD_strategy strat)
-{
-    U32 const btScale = ((U32)strat >= (U32)ZSTD_btlazy2);
-    return hashLog - btScale;
-}
-
-/** ZSTD_adjustCParams_internal() :
- *  optimize `cPar` for a specified input (`srcSize` and `dictSize`).
- *  mostly downsize to reduce memory consumption and initialization latency.
- * `srcSize` can be ZSTD_CONTENTSIZE_UNKNOWN when not known.
- *  note : for the time being, `srcSize==0` means "unknown" too, for compatibility with older convention.
- *  condition : cPar is presumed validated (can be checked using ZSTD_checkCParams()). */
-static ZSTD_compressionParameters
-ZSTD_adjustCParams_internal(ZSTD_compressionParameters cPar,
-                            unsigned long long srcSize,
-                            size_t dictSize)
-{
-    static const U64 minSrcSize = 513; /* (1<<9) + 1 */
-    static const U64 maxWindowResize = 1ULL << (ZSTD_WINDOWLOG_MAX-1);
-    assert(ZSTD_checkCParams(cPar)==0);
-
-    if (dictSize && (srcSize+1<2) /* ZSTD_CONTENTSIZE_UNKNOWN and 0 mean "unknown" */ )
-        srcSize = minSrcSize;  /* presumed small when there is a dictionary */
-    else if (srcSize == 0)
-        srcSize = ZSTD_CONTENTSIZE_UNKNOWN;  /* 0 == unknown : presumed large */
-
-    /* resize windowLog if input is small enough, to use less memory */
-    if ( (srcSize < maxWindowResize)
-      && (dictSize < maxWindowResize) )  {
-        U32 const tSize = (U32)(srcSize + dictSize);
-        static U32 const hashSizeMin = 1 << ZSTD_HASHLOG_MIN;
-        U32 const srcLog = (tSize < hashSizeMin) ? ZSTD_HASHLOG_MIN :
-                            ZSTD_highbit32(tSize-1) + 1;
-        if (cPar.windowLog > srcLog) cPar.windowLog = srcLog;
-    }
-    if (cPar.hashLog > cPar.windowLog+1) cPar.hashLog = cPar.windowLog+1;
-    {   U32 const cycleLog = ZSTD_cycleLog(cPar.chainLog, cPar.strategy);
-        if (cycleLog > cPar.windowLog)
-            cPar.chainLog -= (cycleLog - cPar.windowLog);
-    }
-
-    if (cPar.windowLog < ZSTD_WINDOWLOG_ABSOLUTEMIN)
-        cPar.windowLog = ZSTD_WINDOWLOG_ABSOLUTEMIN;  /* minimum wlog required for valid frame header */
-
-    return cPar;
-}
-
-ZSTD_compressionParameters
-ZSTD_adjustCParams(ZSTD_compressionParameters cPar,
-                   unsigned long long srcSize,
-                   size_t dictSize)
-{
-    cPar = ZSTD_clampCParams(cPar);   /* resulting cPar is necessarily valid (all parameters within range) */
-    return ZSTD_adjustCParams_internal(cPar, srcSize, dictSize);
-}
-
-ZSTD_compressionParameters ZSTD_getCParamsFromCCtxParams(
-        const ZSTD_CCtx_params* CCtxParams, U64 srcSizeHint, size_t dictSize)
-{
-    ZSTD_compressionParameters cParams = ZSTD_getCParams(CCtxParams->compressionLevel, srcSizeHint, dictSize);
-    if (CCtxParams->ldmParams.enableLdm) cParams.windowLog = ZSTD_LDM_DEFAULT_WINDOW_LOG;
-    if (CCtxParams->cParams.windowLog) cParams.windowLog = CCtxParams->cParams.windowLog;
-    if (CCtxParams->cParams.hashLog) cParams.hashLog = CCtxParams->cParams.hashLog;
-    if (CCtxParams->cParams.chainLog) cParams.chainLog = CCtxParams->cParams.chainLog;
-    if (CCtxParams->cParams.searchLog) cParams.searchLog = CCtxParams->cParams.searchLog;
-    if (CCtxParams->cParams.minMatch) cParams.minMatch = CCtxParams->cParams.minMatch;
-    if (CCtxParams->cParams.targetLength) cParams.targetLength = CCtxParams->cParams.targetLength;
-    if (CCtxParams->cParams.strategy) cParams.strategy = CCtxParams->cParams.strategy;
-    assert(!ZSTD_checkCParams(cParams));
-    return ZSTD_adjustCParams_internal(cParams, srcSizeHint, dictSize);
-}
-
-static size_t
-ZSTD_sizeof_matchState(const ZSTD_compressionParameters* const cParams,
-                       const U32 forCCtx)
-{
-    size_t const chainSize = (cParams->strategy == ZSTD_fast) ? 0 : ((size_t)1 << cParams->chainLog);
-    size_t const hSize = ((size_t)1) << cParams->hashLog;
-    U32    const hashLog3 = (forCCtx && cParams->minMatch==3) ? MIN(ZSTD_HASHLOG3_MAX, cParams->windowLog) : 0;
-    size_t const h3Size = ((size_t)1) << hashLog3;
-    size_t const tableSpace = (chainSize + hSize + h3Size) * sizeof(U32);
-    size_t const optPotentialSpace = ((MaxML+1) + (MaxLL+1) + (MaxOff+1) + (1<<Litbits)) * sizeof(U32)
-                          + (ZSTD_OPT_NUM+1) * (sizeof(ZSTD_match_t)+sizeof(ZSTD_optimal_t));
-    size_t const optSpace = (forCCtx && (cParams->strategy >= ZSTD_btopt))
-                                ? optPotentialSpace
-                                : 0;
-    DEBUGLOG(4, "chainSize: %u - hSize: %u - h3Size: %u",
-                (U32)chainSize, (U32)hSize, (U32)h3Size);
-    return tableSpace + optSpace;
-}
-
-size_t ZSTD_estimateCCtxSize_usingCCtxParams(const ZSTD_CCtx_params* params)
-{
-    RETURN_ERROR_IF(params->nbWorkers > 0, GENERIC, "Estimate CCtx size is supported for single-threaded compression only.");
-    {   ZSTD_compressionParameters const cParams =
-                ZSTD_getCParamsFromCCtxParams(params, 0, 0);
-        size_t const blockSize = MIN(ZSTD_BLOCKSIZE_MAX, (size_t)1 << cParams.windowLog);
-        U32    const divider = (cParams.minMatch==3) ? 3 : 4;
-        size_t const maxNbSeq = blockSize / divider;
-        size_t const tokenSpace = WILDCOPY_OVERLENGTH + blockSize + 11*maxNbSeq;
-        size_t const entropySpace = HUF_WORKSPACE_SIZE;
-        size_t const blockStateSpace = 2 * sizeof(ZSTD_compressedBlockState_t);
-        size_t const matchStateSize = ZSTD_sizeof_matchState(&cParams, /* forCCtx */ 1);
-
-        size_t const ldmSpace = ZSTD_ldm_getTableSize(params->ldmParams);
-        size_t const ldmSeqSpace = ZSTD_ldm_getMaxNbSeq(params->ldmParams, blockSize) * sizeof(rawSeq);
-
-        size_t const neededSpace = entropySpace + blockStateSpace + tokenSpace +
-                                   matchStateSize + ldmSpace + ldmSeqSpace;
-
-        DEBUGLOG(5, "sizeof(ZSTD_CCtx) : %u", (U32)sizeof(ZSTD_CCtx));
-        DEBUGLOG(5, "estimate workSpace : %u", (U32)neededSpace);
-        return sizeof(ZSTD_CCtx) + neededSpace;
-    }
-}
-
-size_t ZSTD_estimateCCtxSize_usingCParams(ZSTD_compressionParameters cParams)
-{
-    ZSTD_CCtx_params const params = ZSTD_makeCCtxParamsFromCParams(cParams);
-    return ZSTD_estimateCCtxSize_usingCCtxParams(&params);
-}
-
-static size_t ZSTD_estimateCCtxSize_internal(int compressionLevel)
-{
-    ZSTD_compressionParameters const cParams = ZSTD_getCParams(compressionLevel, 0, 0);
-    return ZSTD_estimateCCtxSize_usingCParams(cParams);
-}
-
-size_t ZSTD_estimateCCtxSize(int compressionLevel)
-{
-    int level;
-    size_t memBudget = 0;
-    for (level=MIN(compressionLevel, 1); level<=compressionLevel; level++) {
-        size_t const newMB = ZSTD_estimateCCtxSize_internal(level);
-        if (newMB > memBudget) memBudget = newMB;
-    }
-    return memBudget;
-}
-
-size_t ZSTD_estimateCStreamSize_usingCCtxParams(const ZSTD_CCtx_params* params)
-{
-    RETURN_ERROR_IF(params->nbWorkers > 0, GENERIC, "Estimate CCtx size is supported for single-threaded compression only.");
-    {   ZSTD_compressionParameters const cParams =
-                ZSTD_getCParamsFromCCtxParams(params, 0, 0);
-        size_t const CCtxSize = ZSTD_estimateCCtxSize_usingCCtxParams(params);
-        size_t const blockSize = MIN(ZSTD_BLOCKSIZE_MAX, (size_t)1 << cParams.windowLog);
-        size_t const inBuffSize = ((size_t)1 << cParams.windowLog) + blockSize;
-        size_t const outBuffSize = ZSTD_compressBound(blockSize) + 1;
-        size_t const streamingSize = inBuffSize + outBuffSize;
-
-        return CCtxSize + streamingSize;
-    }
-}
-
-size_t ZSTD_estimateCStreamSize_usingCParams(ZSTD_compressionParameters cParams)
-{
-    ZSTD_CCtx_params const params = ZSTD_makeCCtxParamsFromCParams(cParams);
-    return ZSTD_estimateCStreamSize_usingCCtxParams(&params);
-}
-
-static size_t ZSTD_estimateCStreamSize_internal(int compressionLevel)
-{
-    ZSTD_compressionParameters const cParams = ZSTD_getCParams(compressionLevel, 0, 0);
-    return ZSTD_estimateCStreamSize_usingCParams(cParams);
-}
-
-size_t ZSTD_estimateCStreamSize(int compressionLevel)
-{
-    int level;
-    size_t memBudget = 0;
-    for (level=MIN(compressionLevel, 1); level<=compressionLevel; level++) {
-        size_t const newMB = ZSTD_estimateCStreamSize_internal(level);
-        if (newMB > memBudget) memBudget = newMB;
-    }
-    return memBudget;
-}
-
-/* ZSTD_getFrameProgression():
- * tells how much data has been consumed (input) and produced (output) for current frame.
- * able to count progression inside worker threads (non-blocking mode).
- */
-ZSTD_frameProgression ZSTD_getFrameProgression(const ZSTD_CCtx* cctx)
-{
-#ifdef ZSTD_MULTITHREAD
-    if (cctx->appliedParams.nbWorkers > 0) {
-        return ZSTDMT_getFrameProgression(cctx->mtctx);
-    }
-#endif
-    {   ZSTD_frameProgression fp;
-        size_t const buffered = (cctx->inBuff == NULL) ? 0 :
-                                cctx->inBuffPos - cctx->inToCompress;
-        if (buffered) assert(cctx->inBuffPos >= cctx->inToCompress);
-        assert(buffered <= ZSTD_BLOCKSIZE_MAX);
-        fp.ingested = cctx->consumedSrcSize + buffered;
-        fp.consumed = cctx->consumedSrcSize;
-        fp.produced = cctx->producedCSize;
-        fp.flushed  = cctx->producedCSize;   /* simplified; some data might still be left within streaming output buffer */
-        fp.currentJobID = 0;
-        fp.nbActiveWorkers = 0;
-        return fp;
-}   }
-
-/*! ZSTD_toFlushNow()
- *  Only useful for multithreading scenarios currently (nbWorkers >= 1).
- */
-size_t ZSTD_toFlushNow(ZSTD_CCtx* cctx)
-{
-#ifdef ZSTD_MULTITHREAD
-    if (cctx->appliedParams.nbWorkers > 0) {
-        return ZSTDMT_toFlushNow(cctx->mtctx);
-    }
-#endif
-    (void)cctx;
-    return 0;   /* over-simplification; could also check if context is currently running in streaming mode, and in which case, report how many bytes are left to be flushed within output buffer */
-}
-
-
-
-static U32 ZSTD_equivalentCParams(ZSTD_compressionParameters cParams1,
-                                  ZSTD_compressionParameters cParams2)
-{
-    return (cParams1.hashLog  == cParams2.hashLog)
-         & (cParams1.chainLog == cParams2.chainLog)
-         & (cParams1.strategy == cParams2.strategy)   /* opt parser space */
-         & ((cParams1.minMatch==3) == (cParams2.minMatch==3));  /* hashlog3 space */
-}
-
-static void ZSTD_assertEqualCParams(ZSTD_compressionParameters cParams1,
-                                    ZSTD_compressionParameters cParams2)
-{
-    (void)cParams1;
-    (void)cParams2;
-    assert(cParams1.windowLog    == cParams2.windowLog);
-    assert(cParams1.chainLog     == cParams2.chainLog);
-    assert(cParams1.hashLog      == cParams2.hashLog);
-    assert(cParams1.searchLog    == cParams2.searchLog);
-    assert(cParams1.minMatch     == cParams2.minMatch);
-    assert(cParams1.targetLength == cParams2.targetLength);
-    assert(cParams1.strategy     == cParams2.strategy);
-}
-
-/** The parameters are equivalent if ldm is not enabled in both sets or
- *  all the parameters are equivalent. */
-static U32 ZSTD_equivalentLdmParams(ldmParams_t ldmParams1,
-                                    ldmParams_t ldmParams2)
-{
-    return (!ldmParams1.enableLdm && !ldmParams2.enableLdm) ||
-           (ldmParams1.enableLdm == ldmParams2.enableLdm &&
-            ldmParams1.hashLog == ldmParams2.hashLog &&
-            ldmParams1.bucketSizeLog == ldmParams2.bucketSizeLog &&
-            ldmParams1.minMatchLength == ldmParams2.minMatchLength &&
-            ldmParams1.hashRateLog == ldmParams2.hashRateLog);
-}
-
-typedef enum { ZSTDb_not_buffered, ZSTDb_buffered } ZSTD_buffered_policy_e;
-
-/* ZSTD_sufficientBuff() :
- * check internal buffers exist for streaming if buffPol == ZSTDb_buffered .
- * Note : they are assumed to be correctly sized if ZSTD_equivalentCParams()==1 */
-static U32 ZSTD_sufficientBuff(size_t bufferSize1, size_t maxNbSeq1,
-                            size_t maxNbLit1,
-                            ZSTD_buffered_policy_e buffPol2,
-                            ZSTD_compressionParameters cParams2,
-                            U64 pledgedSrcSize)
-{
-    size_t const windowSize2 = MAX(1, (size_t)MIN(((U64)1 << cParams2.windowLog), pledgedSrcSize));
-    size_t const blockSize2 = MIN(ZSTD_BLOCKSIZE_MAX, windowSize2);
-    size_t const maxNbSeq2 = blockSize2 / ((cParams2.minMatch == 3) ? 3 : 4);
-    size_t const maxNbLit2 = blockSize2;
-    size_t const neededBufferSize2 = (buffPol2==ZSTDb_buffered) ? windowSize2 + blockSize2 : 0;
-    DEBUGLOG(4, "ZSTD_sufficientBuff: is neededBufferSize2=%u <= bufferSize1=%u",
-                (U32)neededBufferSize2, (U32)bufferSize1);
-    DEBUGLOG(4, "ZSTD_sufficientBuff: is maxNbSeq2=%u <= maxNbSeq1=%u",
-                (U32)maxNbSeq2, (U32)maxNbSeq1);
-    DEBUGLOG(4, "ZSTD_sufficientBuff: is maxNbLit2=%u <= maxNbLit1=%u",
-                (U32)maxNbLit2, (U32)maxNbLit1);
-    return (maxNbLit2 <= maxNbLit1)
-         & (maxNbSeq2 <= maxNbSeq1)
-         & (neededBufferSize2 <= bufferSize1);
-}
-
-/** Equivalence for resetCCtx purposes */
-static U32 ZSTD_equivalentParams(ZSTD_CCtx_params params1,
-                                 ZSTD_CCtx_params params2,
-                                 size_t buffSize1,
-                                 size_t maxNbSeq1, size_t maxNbLit1,
-                                 ZSTD_buffered_policy_e buffPol2,
-                                 U64 pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_equivalentParams: pledgedSrcSize=%u", (U32)pledgedSrcSize);
-    if (!ZSTD_equivalentCParams(params1.cParams, params2.cParams)) {
-      DEBUGLOG(4, "ZSTD_equivalentCParams() == 0");
-      return 0;
-    }
-    if (!ZSTD_equivalentLdmParams(params1.ldmParams, params2.ldmParams)) {
-      DEBUGLOG(4, "ZSTD_equivalentLdmParams() == 0");
-      return 0;
-    }
-    if (!ZSTD_sufficientBuff(buffSize1, maxNbSeq1, maxNbLit1, buffPol2,
-                             params2.cParams, pledgedSrcSize)) {
-      DEBUGLOG(4, "ZSTD_sufficientBuff() == 0");
-      return 0;
-    }
-    return 1;
-}
-
-static void ZSTD_reset_compressedBlockState(ZSTD_compressedBlockState_t* bs)
-{
-    int i;
-    for (i = 0; i < ZSTD_REP_NUM; ++i)
-        bs->rep[i] = repStartValue[i];
-    bs->entropy.huf.repeatMode = HUF_repeat_none;
-    bs->entropy.fse.offcode_repeatMode = FSE_repeat_none;
-    bs->entropy.fse.matchlength_repeatMode = FSE_repeat_none;
-    bs->entropy.fse.litlength_repeatMode = FSE_repeat_none;
-}
-
-/*! ZSTD_invalidateMatchState()
- *  Invalidate all the matches in the match finder tables.
- *  Requires nextSrc and base to be set (can be NULL).
- */
-static void ZSTD_invalidateMatchState(ZSTD_matchState_t* ms)
-{
-    ZSTD_window_clear(&ms->window);
-
-    ms->nextToUpdate = ms->window.dictLimit;
-    ms->loadedDictEnd = 0;
-    ms->opt.litLengthSum = 0;  /* force reset of btopt stats */
-    ms->dictMatchState = NULL;
-}
-
-/*! ZSTD_continueCCtx() :
- *  reuse CCtx without reset (note : requires no dictionary) */
-static size_t ZSTD_continueCCtx(ZSTD_CCtx* cctx, ZSTD_CCtx_params params, U64 pledgedSrcSize)
-{
-    size_t const windowSize = MAX(1, (size_t)MIN(((U64)1 << params.cParams.windowLog), pledgedSrcSize));
-    size_t const blockSize = MIN(ZSTD_BLOCKSIZE_MAX, windowSize);
-    DEBUGLOG(4, "ZSTD_continueCCtx: re-use context in place");
-
-    cctx->blockSize = blockSize;   /* previous block size could be different even for same windowLog, due to pledgedSrcSize */
-    cctx->appliedParams = params;
-    cctx->blockState.matchState.cParams = params.cParams;
-    cctx->pledgedSrcSizePlusOne = pledgedSrcSize+1;
-    cctx->consumedSrcSize = 0;
-    cctx->producedCSize = 0;
-    if (pledgedSrcSize == ZSTD_CONTENTSIZE_UNKNOWN)
-        cctx->appliedParams.fParams.contentSizeFlag = 0;
-    DEBUGLOG(4, "pledged content size : %u ; flag : %u",
-        (U32)pledgedSrcSize, cctx->appliedParams.fParams.contentSizeFlag);
-    cctx->stage = ZSTDcs_init;
-    cctx->dictID = 0;
-    if (params.ldmParams.enableLdm)
-        ZSTD_window_clear(&cctx->ldmState.window);
-    ZSTD_referenceExternalSequences(cctx, NULL, 0);
-    ZSTD_invalidateMatchState(&cctx->blockState.matchState);
-    ZSTD_reset_compressedBlockState(cctx->blockState.prevCBlock);
-    XXH64_reset(&cctx->xxhState, 0);
-    return 0;
-}
-
-typedef enum { ZSTDcrp_continue, ZSTDcrp_noMemset } ZSTD_compResetPolicy_e;
-
-typedef enum { ZSTD_resetTarget_CDict, ZSTD_resetTarget_CCtx } ZSTD_resetTarget_e;
-
-static void*
-ZSTD_reset_matchState(ZSTD_matchState_t* ms,
-                      void* ptr,
-                const ZSTD_compressionParameters* cParams,
-                      ZSTD_compResetPolicy_e const crp, ZSTD_resetTarget_e const forWho)
-{
-    size_t const chainSize = (cParams->strategy == ZSTD_fast) ? 0 : ((size_t)1 << cParams->chainLog);
-    size_t const hSize = ((size_t)1) << cParams->hashLog;
-    U32    const hashLog3 = ((forWho == ZSTD_resetTarget_CCtx) && cParams->minMatch==3) ? MIN(ZSTD_HASHLOG3_MAX, cParams->windowLog) : 0;
-    size_t const h3Size = ((size_t)1) << hashLog3;
-    size_t const tableSpace = (chainSize + hSize + h3Size) * sizeof(U32);
-
-    assert(((size_t)ptr & 3) == 0);
-
-    ms->hashLog3 = hashLog3;
-    memset(&ms->window, 0, sizeof(ms->window));
-    ms->window.dictLimit = 1;    /* start from 1, so that 1st position is valid */
-    ms->window.lowLimit = 1;     /* it ensures first and later CCtx usages compress the same */
-    ms->window.nextSrc = ms->window.base + 1;   /* see issue #1241 */
-    ZSTD_invalidateMatchState(ms);
-
-    /* opt parser space */
-    if ((forWho == ZSTD_resetTarget_CCtx) && (cParams->strategy >= ZSTD_btopt)) {
-        DEBUGLOG(4, "reserving optimal parser space");
-        ms->opt.litFreq = (unsigned*)ptr;
-        ms->opt.litLengthFreq = ms->opt.litFreq + (1<<Litbits);
-        ms->opt.matchLengthFreq = ms->opt.litLengthFreq + (MaxLL+1);
-        ms->opt.offCodeFreq = ms->opt.matchLengthFreq + (MaxML+1);
-        ptr = ms->opt.offCodeFreq + (MaxOff+1);
-        ms->opt.matchTable = (ZSTD_match_t*)ptr;
-        ptr = ms->opt.matchTable + ZSTD_OPT_NUM+1;
-        ms->opt.priceTable = (ZSTD_optimal_t*)ptr;
-        ptr = ms->opt.priceTable + ZSTD_OPT_NUM+1;
-    }
-
-    /* table Space */
-    DEBUGLOG(4, "reset table : %u", crp!=ZSTDcrp_noMemset);
-    assert(((size_t)ptr & 3) == 0);  /* ensure ptr is properly aligned */
-    if (crp!=ZSTDcrp_noMemset) memset(ptr, 0, tableSpace);   /* reset tables only */
-    ms->hashTable = (U32*)(ptr);
-    ms->chainTable = ms->hashTable + hSize;
-    ms->hashTable3 = ms->chainTable + chainSize;
-    ptr = ms->hashTable3 + h3Size;
-
-    ms->cParams = *cParams;
-
-    assert(((size_t)ptr & 3) == 0);
-    return ptr;
-}
-
-/* ZSTD_indexTooCloseToMax() :
- * minor optimization : prefer memset() rather than reduceIndex()
- * which is measurably slow in some circumstances (reported for Visual Studio).
- * Works when re-using a context for a lot of smallish inputs :
- * if all inputs are smaller than ZSTD_INDEXOVERFLOW_MARGIN,
- * memset() will be triggered before reduceIndex().
- */
-#define ZSTD_INDEXOVERFLOW_MARGIN (16 MB)
-static int ZSTD_indexTooCloseToMax(ZSTD_window_t w)
-{
-    return (size_t)(w.nextSrc - w.base) > (ZSTD_CURRENT_MAX - ZSTD_INDEXOVERFLOW_MARGIN);
-}
-
-#define ZSTD_WORKSPACETOOLARGE_FACTOR 3 /* define "workspace is too large" as this number of times larger than needed */
-#define ZSTD_WORKSPACETOOLARGE_MAXDURATION 128  /* when workspace is continuously too large
-                                         * during at least this number of times,
-                                         * context's memory usage is considered wasteful,
-                                         * because it's sized to handle a worst case scenario which rarely happens.
-                                         * In which case, resize it down to free some memory */
-
-/*! ZSTD_resetCCtx_internal() :
-    note : `params` are assumed fully validated at this stage */
-static size_t ZSTD_resetCCtx_internal(ZSTD_CCtx* zc,
-                                      ZSTD_CCtx_params params,
-                                      U64 const pledgedSrcSize,
-                                      ZSTD_compResetPolicy_e const crp,
-                                      ZSTD_buffered_policy_e const zbuff)
-{
-    DEBUGLOG(4, "ZSTD_resetCCtx_internal: pledgedSrcSize=%u, wlog=%u",
-                (U32)pledgedSrcSize, params.cParams.windowLog);
-    assert(!ZSTD_isError(ZSTD_checkCParams(params.cParams)));
-
-    if (crp == ZSTDcrp_continue) {
-        if (ZSTD_equivalentParams(zc->appliedParams, params,
-                                  zc->inBuffSize,
-                                  zc->seqStore.maxNbSeq, zc->seqStore.maxNbLit,
-                                  zbuff, pledgedSrcSize) ) {
-            DEBUGLOG(4, "ZSTD_equivalentParams()==1 -> consider continue mode");
-            zc->workSpaceOversizedDuration += (zc->workSpaceOversizedDuration > 0);   /* if it was too large, it still is */
-            if (zc->workSpaceOversizedDuration <= ZSTD_WORKSPACETOOLARGE_MAXDURATION) {
-                DEBUGLOG(4, "continue mode confirmed (wLog1=%u, blockSize1=%zu)",
-                            zc->appliedParams.cParams.windowLog, zc->blockSize);
-                if (ZSTD_indexTooCloseToMax(zc->blockState.matchState.window)) {
-                    /* prefer a reset, faster than a rescale */
-                    ZSTD_reset_matchState(&zc->blockState.matchState,
-                                           zc->entropyWorkspace + HUF_WORKSPACE_SIZE_U32,
-                                          &params.cParams,
-                                           crp, ZSTD_resetTarget_CCtx);
-                }
-                return ZSTD_continueCCtx(zc, params, pledgedSrcSize);
-    }   }   }
-    DEBUGLOG(4, "ZSTD_equivalentParams()==0 -> reset CCtx");
-
-    if (params.ldmParams.enableLdm) {
-        /* Adjust long distance matching parameters */
-        ZSTD_ldm_adjustParameters(&params.ldmParams, &params.cParams);
-        assert(params.ldmParams.hashLog >= params.ldmParams.bucketSizeLog);
-        assert(params.ldmParams.hashRateLog < 32);
-        zc->ldmState.hashPower = ZSTD_rollingHash_primePower(params.ldmParams.minMatchLength);
-    }
-
-    {   size_t const windowSize = MAX(1, (size_t)MIN(((U64)1 << params.cParams.windowLog), pledgedSrcSize));
-        size_t const blockSize = MIN(ZSTD_BLOCKSIZE_MAX, windowSize);
-        U32    const divider = (params.cParams.minMatch==3) ? 3 : 4;
-        size_t const maxNbSeq = blockSize / divider;
-        size_t const tokenSpace = WILDCOPY_OVERLENGTH + blockSize + 11*maxNbSeq;
-        size_t const buffOutSize = (zbuff==ZSTDb_buffered) ? ZSTD_compressBound(blockSize)+1 : 0;
-        size_t const buffInSize = (zbuff==ZSTDb_buffered) ? windowSize + blockSize : 0;
-        size_t const matchStateSize = ZSTD_sizeof_matchState(&params.cParams, /* forCCtx */ 1);
-        size_t const maxNbLdmSeq = ZSTD_ldm_getMaxNbSeq(params.ldmParams, blockSize);
-        void* ptr;   /* used to partition workSpace */
-
-        /* Check if workSpace is large enough, alloc a new one if needed */
-        {   size_t const entropySpace = HUF_WORKSPACE_SIZE;
-            size_t const blockStateSpace = 2 * sizeof(ZSTD_compressedBlockState_t);
-            size_t const bufferSpace = buffInSize + buffOutSize;
-            size_t const ldmSpace = ZSTD_ldm_getTableSize(params.ldmParams);
-            size_t const ldmSeqSpace = maxNbLdmSeq * sizeof(rawSeq);
-
-            size_t const neededSpace = entropySpace + blockStateSpace + ldmSpace +
-                                       ldmSeqSpace + matchStateSize + tokenSpace +
-                                       bufferSpace;
-
-            int const workSpaceTooSmall = zc->workSpaceSize < neededSpace;
-            int const workSpaceTooLarge = zc->workSpaceSize > ZSTD_WORKSPACETOOLARGE_FACTOR * neededSpace;
-            int const workSpaceWasteful = workSpaceTooLarge && (zc->workSpaceOversizedDuration > ZSTD_WORKSPACETOOLARGE_MAXDURATION);
-            zc->workSpaceOversizedDuration = workSpaceTooLarge ? zc->workSpaceOversizedDuration+1 : 0;
-
-            DEBUGLOG(4, "Need %zuKB workspace, including %zuKB for match state, and %zuKB for buffers",
-                        neededSpace>>10, matchStateSize>>10, bufferSpace>>10);
-            DEBUGLOG(4, "windowSize: %zu - blockSize: %zu", windowSize, blockSize);
-
-            if (workSpaceTooSmall || workSpaceWasteful) {
-                DEBUGLOG(4, "Resize workSpaceSize from %zuKB to %zuKB",
-                            zc->workSpaceSize >> 10,
-                            neededSpace >> 10);
-
-                RETURN_ERROR_IF(zc->staticSize, memory_allocation, "static cctx : no resize");
-
-                zc->workSpaceSize = 0;
-                ZSTD_free(zc->workSpace, zc->customMem);
-                zc->workSpace = ZSTD_malloc(neededSpace, zc->customMem);
-                RETURN_ERROR_IF(zc->workSpace == NULL, memory_allocation);
-                zc->workSpaceSize = neededSpace;
-                zc->workSpaceOversizedDuration = 0;
-
-                /* Statically sized space.
-                 * entropyWorkspace never moves,
-                 * though prev/next block swap places */
-                assert(((size_t)zc->workSpace & 3) == 0);   /* ensure correct alignment */
-                assert(zc->workSpaceSize >= 2 * sizeof(ZSTD_compressedBlockState_t));
-                zc->blockState.prevCBlock = (ZSTD_compressedBlockState_t*)zc->workSpace;
-                zc->blockState.nextCBlock = zc->blockState.prevCBlock + 1;
-                ptr = zc->blockState.nextCBlock + 1;
-                zc->entropyWorkspace = (U32*)ptr;
-        }   }
-
-        /* init params */
-        zc->appliedParams = params;
-        zc->blockState.matchState.cParams = params.cParams;
-        zc->pledgedSrcSizePlusOne = pledgedSrcSize+1;
-        zc->consumedSrcSize = 0;
-        zc->producedCSize = 0;
-        if (pledgedSrcSize == ZSTD_CONTENTSIZE_UNKNOWN)
-            zc->appliedParams.fParams.contentSizeFlag = 0;
-        DEBUGLOG(4, "pledged content size : %u ; flag : %u",
-            (unsigned)pledgedSrcSize, zc->appliedParams.fParams.contentSizeFlag);
-        zc->blockSize = blockSize;
-
-        XXH64_reset(&zc->xxhState, 0);
-        zc->stage = ZSTDcs_init;
-        zc->dictID = 0;
-
-        ZSTD_reset_compressedBlockState(zc->blockState.prevCBlock);
-
-        ptr = ZSTD_reset_matchState(&zc->blockState.matchState,
-                                     zc->entropyWorkspace + HUF_WORKSPACE_SIZE_U32,
-                                    &params.cParams,
-                                     crp, ZSTD_resetTarget_CCtx);
-
-        /* ldm hash table */
-        /* initialize bucketOffsets table later for pointer alignment */
-        if (params.ldmParams.enableLdm) {
-            size_t const ldmHSize = ((size_t)1) << params.ldmParams.hashLog;
-            memset(ptr, 0, ldmHSize * sizeof(ldmEntry_t));
-            assert(((size_t)ptr & 3) == 0); /* ensure ptr is properly aligned */
-            zc->ldmState.hashTable = (ldmEntry_t*)ptr;
-            ptr = zc->ldmState.hashTable + ldmHSize;
-            zc->ldmSequences = (rawSeq*)ptr;
-            ptr = zc->ldmSequences + maxNbLdmSeq;
-            zc->maxNbLdmSequences = maxNbLdmSeq;
-
-            memset(&zc->ldmState.window, 0, sizeof(zc->ldmState.window));
-        }
-        assert(((size_t)ptr & 3) == 0); /* ensure ptr is properly aligned */
-
-        /* sequences storage */
-        zc->seqStore.maxNbSeq = maxNbSeq;
-        zc->seqStore.sequencesStart = (seqDef*)ptr;
-        ptr = zc->seqStore.sequencesStart + maxNbSeq;
-        zc->seqStore.llCode = (BYTE*) ptr;
-        zc->seqStore.mlCode = zc->seqStore.llCode + maxNbSeq;
-        zc->seqStore.ofCode = zc->seqStore.mlCode + maxNbSeq;
-        zc->seqStore.litStart = zc->seqStore.ofCode + maxNbSeq;
-        /* ZSTD_wildcopy() is used to copy into the literals buffer,
-         * so we have to oversize the buffer by WILDCOPY_OVERLENGTH bytes.
-         */
-        zc->seqStore.maxNbLit = blockSize;
-        ptr = zc->seqStore.litStart + blockSize + WILDCOPY_OVERLENGTH;
-
-        /* ldm bucketOffsets table */
-        if (params.ldmParams.enableLdm) {
-            size_t const ldmBucketSize =
-                  ((size_t)1) << (params.ldmParams.hashLog -
-                                  params.ldmParams.bucketSizeLog);
-            memset(ptr, 0, ldmBucketSize);
-            zc->ldmState.bucketOffsets = (BYTE*)ptr;
-            ptr = zc->ldmState.bucketOffsets + ldmBucketSize;
-            ZSTD_window_clear(&zc->ldmState.window);
-        }
-        ZSTD_referenceExternalSequences(zc, NULL, 0);
-
-        /* buffers */
-        zc->inBuffSize = buffInSize;
-        zc->inBuff = (char*)ptr;
-        zc->outBuffSize = buffOutSize;
-        zc->outBuff = zc->inBuff + buffInSize;
-
-        return 0;
-    }
-}
-
-/* ZSTD_invalidateRepCodes() :
- * ensures next compression will not use repcodes from previous block.
- * Note : only works with regular variant;
- *        do not use with extDict variant ! */
-void ZSTD_invalidateRepCodes(ZSTD_CCtx* cctx) {
-    int i;
-    for (i=0; i<ZSTD_REP_NUM; i++) cctx->blockState.prevCBlock->rep[i] = 0;
-    assert(!ZSTD_window_hasExtDict(cctx->blockState.matchState.window));
-}
-
-/* These are the approximate sizes for each strategy past which copying the
- * dictionary tables into the working context is faster than using them
- * in-place.
- */
-static const size_t attachDictSizeCutoffs[ZSTD_STRATEGY_MAX+1] = {
-    8 KB,  /* unused */
-    8 KB,  /* ZSTD_fast */
-    16 KB, /* ZSTD_dfast */
-    32 KB, /* ZSTD_greedy */
-    32 KB, /* ZSTD_lazy */
-    32 KB, /* ZSTD_lazy2 */
-    32 KB, /* ZSTD_btlazy2 */
-    32 KB, /* ZSTD_btopt */
-    8 KB,  /* ZSTD_btultra */
-    8 KB   /* ZSTD_btultra2 */
-};
-
-static int ZSTD_shouldAttachDict(const ZSTD_CDict* cdict,
-                                 ZSTD_CCtx_params params,
-                                 U64 pledgedSrcSize)
-{
-    size_t cutoff = attachDictSizeCutoffs[cdict->matchState.cParams.strategy];
-    return ( pledgedSrcSize <= cutoff
-          || pledgedSrcSize == ZSTD_CONTENTSIZE_UNKNOWN
-          || params.attachDictPref == ZSTD_dictForceAttach )
-        && params.attachDictPref != ZSTD_dictForceCopy
-        && !params.forceWindow; /* dictMatchState isn't correctly
-                                 * handled in _enforceMaxDist */
-}
-
-static size_t
-ZSTD_resetCCtx_byAttachingCDict(ZSTD_CCtx* cctx,
-                        const ZSTD_CDict* cdict,
-                        ZSTD_CCtx_params params,
-                        U64 pledgedSrcSize,
-                        ZSTD_buffered_policy_e zbuff)
-{
-    {   const ZSTD_compressionParameters* const cdict_cParams = &cdict->matchState.cParams;
-        unsigned const windowLog = params.cParams.windowLog;
-        assert(windowLog != 0);
-        /* Resize working context table params for input only, since the dict
-         * has its own tables. */
-        params.cParams = ZSTD_adjustCParams_internal(*cdict_cParams, pledgedSrcSize, 0);
-        params.cParams.windowLog = windowLog;
-        ZSTD_resetCCtx_internal(cctx, params, pledgedSrcSize,
-                                ZSTDcrp_continue, zbuff);
-        assert(cctx->appliedParams.cParams.strategy == cdict_cParams->strategy);
-    }
-
-    {   const U32 cdictEnd = (U32)( cdict->matchState.window.nextSrc
-                                  - cdict->matchState.window.base);
-        const U32 cdictLen = cdictEnd - cdict->matchState.window.dictLimit;
-        if (cdictLen == 0) {
-            /* don't even attach dictionaries with no contents */
-            DEBUGLOG(4, "skipping attaching empty dictionary");
-        } else {
-            DEBUGLOG(4, "attaching dictionary into context");
-            cctx->blockState.matchState.dictMatchState = &cdict->matchState;
-
-            /* prep working match state so dict matches never have negative indices
-             * when they are translated to the working context's index space. */
-            if (cctx->blockState.matchState.window.dictLimit < cdictEnd) {
-                cctx->blockState.matchState.window.nextSrc =
-                    cctx->blockState.matchState.window.base + cdictEnd;
-                ZSTD_window_clear(&cctx->blockState.matchState.window);
-            }
-            /* loadedDictEnd is expressed within the referential of the active context */
-            cctx->blockState.matchState.loadedDictEnd = cctx->blockState.matchState.window.dictLimit;
-    }   }
-
-    cctx->dictID = cdict->dictID;
-
-    /* copy block state */
-    memcpy(cctx->blockState.prevCBlock, &cdict->cBlockState, sizeof(cdict->cBlockState));
-
-    return 0;
-}
-
-static size_t ZSTD_resetCCtx_byCopyingCDict(ZSTD_CCtx* cctx,
-                            const ZSTD_CDict* cdict,
-                            ZSTD_CCtx_params params,
-                            U64 pledgedSrcSize,
-                            ZSTD_buffered_policy_e zbuff)
-{
-    const ZSTD_compressionParameters *cdict_cParams = &cdict->matchState.cParams;
-
-    DEBUGLOG(4, "copying dictionary into context");
-
-    {   unsigned const windowLog = params.cParams.windowLog;
-        assert(windowLog != 0);
-        /* Copy only compression parameters related to tables. */
-        params.cParams = *cdict_cParams;
-        params.cParams.windowLog = windowLog;
-        ZSTD_resetCCtx_internal(cctx, params, pledgedSrcSize,
-                                ZSTDcrp_noMemset, zbuff);
-        assert(cctx->appliedParams.cParams.strategy == cdict_cParams->strategy);
-        assert(cctx->appliedParams.cParams.hashLog == cdict_cParams->hashLog);
-        assert(cctx->appliedParams.cParams.chainLog == cdict_cParams->chainLog);
-    }
-
-    /* copy tables */
-    {   size_t const chainSize = (cdict_cParams->strategy == ZSTD_fast) ? 0 : ((size_t)1 << cdict_cParams->chainLog);
-        size_t const hSize =  (size_t)1 << cdict_cParams->hashLog;
-        size_t const tableSpace = (chainSize + hSize) * sizeof(U32);
-        assert((U32*)cctx->blockState.matchState.chainTable == (U32*)cctx->blockState.matchState.hashTable + hSize);  /* chainTable must follow hashTable */
-        assert((U32*)cctx->blockState.matchState.hashTable3 == (U32*)cctx->blockState.matchState.chainTable + chainSize);
-        assert((U32*)cdict->matchState.chainTable == (U32*)cdict->matchState.hashTable + hSize);  /* chainTable must follow hashTable */
-        assert((U32*)cdict->matchState.hashTable3 == (U32*)cdict->matchState.chainTable + chainSize);
-        memcpy(cctx->blockState.matchState.hashTable, cdict->matchState.hashTable, tableSpace);   /* presumes all tables follow each other */
-    }
-
-    /* Zero the hashTable3, since the cdict never fills it */
-    {   size_t const h3Size = (size_t)1 << cctx->blockState.matchState.hashLog3;
-        assert(cdict->matchState.hashLog3 == 0);
-        memset(cctx->blockState.matchState.hashTable3, 0, h3Size * sizeof(U32));
-    }
-
-    /* copy dictionary offsets */
-    {   ZSTD_matchState_t const* srcMatchState = &cdict->matchState;
-        ZSTD_matchState_t* dstMatchState = &cctx->blockState.matchState;
-        dstMatchState->window       = srcMatchState->window;
-        dstMatchState->nextToUpdate = srcMatchState->nextToUpdate;
-        dstMatchState->loadedDictEnd= srcMatchState->loadedDictEnd;
-    }
-
-    cctx->dictID = cdict->dictID;
-
-    /* copy block state */
-    memcpy(cctx->blockState.prevCBlock, &cdict->cBlockState, sizeof(cdict->cBlockState));
-
-    return 0;
-}
-
-/* We have a choice between copying the dictionary context into the working
- * context, or referencing the dictionary context from the working context
- * in-place. We decide here which strategy to use. */
-static size_t ZSTD_resetCCtx_usingCDict(ZSTD_CCtx* cctx,
-                            const ZSTD_CDict* cdict,
-                            ZSTD_CCtx_params params,
-                            U64 pledgedSrcSize,
-                            ZSTD_buffered_policy_e zbuff)
-{
-
-    DEBUGLOG(4, "ZSTD_resetCCtx_usingCDict (pledgedSrcSize=%u)",
-                (unsigned)pledgedSrcSize);
-
-    if (ZSTD_shouldAttachDict(cdict, params, pledgedSrcSize)) {
-        return ZSTD_resetCCtx_byAttachingCDict(
-            cctx, cdict, params, pledgedSrcSize, zbuff);
-    } else {
-        return ZSTD_resetCCtx_byCopyingCDict(
-            cctx, cdict, params, pledgedSrcSize, zbuff);
-    }
-}
-
-/*! ZSTD_copyCCtx_internal() :
- *  Duplicate an existing context `srcCCtx` into another one `dstCCtx`.
- *  Only works during stage ZSTDcs_init (i.e. after creation, but before first call to ZSTD_compressContinue()).
- *  The "context", in this case, refers to the hash and chain tables,
- *  entropy tables, and dictionary references.
- * `windowLog` value is enforced if != 0, otherwise value is copied from srcCCtx.
- * @return : 0, or an error code */
-static size_t ZSTD_copyCCtx_internal(ZSTD_CCtx* dstCCtx,
-                            const ZSTD_CCtx* srcCCtx,
-                            ZSTD_frameParameters fParams,
-                            U64 pledgedSrcSize,
-                            ZSTD_buffered_policy_e zbuff)
-{
-    DEBUGLOG(5, "ZSTD_copyCCtx_internal");
-    RETURN_ERROR_IF(srcCCtx->stage!=ZSTDcs_init, stage_wrong);
-
-    memcpy(&dstCCtx->customMem, &srcCCtx->customMem, sizeof(ZSTD_customMem));
-    {   ZSTD_CCtx_params params = dstCCtx->requestedParams;
-        /* Copy only compression parameters related to tables. */
-        params.cParams = srcCCtx->appliedParams.cParams;
-        params.fParams = fParams;
-        ZSTD_resetCCtx_internal(dstCCtx, params, pledgedSrcSize,
-                                ZSTDcrp_noMemset, zbuff);
-        assert(dstCCtx->appliedParams.cParams.windowLog == srcCCtx->appliedParams.cParams.windowLog);
-        assert(dstCCtx->appliedParams.cParams.strategy == srcCCtx->appliedParams.cParams.strategy);
-        assert(dstCCtx->appliedParams.cParams.hashLog == srcCCtx->appliedParams.cParams.hashLog);
-        assert(dstCCtx->appliedParams.cParams.chainLog == srcCCtx->appliedParams.cParams.chainLog);
-        assert(dstCCtx->blockState.matchState.hashLog3 == srcCCtx->blockState.matchState.hashLog3);
-    }
-
-    /* copy tables */
-    {   size_t const chainSize = (srcCCtx->appliedParams.cParams.strategy == ZSTD_fast) ? 0 : ((size_t)1 << srcCCtx->appliedParams.cParams.chainLog);
-        size_t const hSize =  (size_t)1 << srcCCtx->appliedParams.cParams.hashLog;
-        size_t const h3Size = (size_t)1 << srcCCtx->blockState.matchState.hashLog3;
-        size_t const tableSpace = (chainSize + hSize + h3Size) * sizeof(U32);
-        assert((U32*)dstCCtx->blockState.matchState.chainTable == (U32*)dstCCtx->blockState.matchState.hashTable + hSize);  /* chainTable must follow hashTable */
-        assert((U32*)dstCCtx->blockState.matchState.hashTable3 == (U32*)dstCCtx->blockState.matchState.chainTable + chainSize);
-        memcpy(dstCCtx->blockState.matchState.hashTable, srcCCtx->blockState.matchState.hashTable, tableSpace);   /* presumes all tables follow each other */
-    }
-
-    /* copy dictionary offsets */
-    {
-        const ZSTD_matchState_t* srcMatchState = &srcCCtx->blockState.matchState;
-        ZSTD_matchState_t* dstMatchState = &dstCCtx->blockState.matchState;
-        dstMatchState->window       = srcMatchState->window;
-        dstMatchState->nextToUpdate = srcMatchState->nextToUpdate;
-        dstMatchState->loadedDictEnd= srcMatchState->loadedDictEnd;
-    }
-    dstCCtx->dictID = srcCCtx->dictID;
-
-    /* copy block state */
-    memcpy(dstCCtx->blockState.prevCBlock, srcCCtx->blockState.prevCBlock, sizeof(*srcCCtx->blockState.prevCBlock));
-
-    return 0;
-}
-
-/*! ZSTD_copyCCtx() :
- *  Duplicate an existing context `srcCCtx` into another one `dstCCtx`.
- *  Only works during stage ZSTDcs_init (i.e. after creation, but before first call to ZSTD_compressContinue()).
- *  pledgedSrcSize==0 means "unknown".
-*   @return : 0, or an error code */
-size_t ZSTD_copyCCtx(ZSTD_CCtx* dstCCtx, const ZSTD_CCtx* srcCCtx, unsigned long long pledgedSrcSize)
-{
-    ZSTD_frameParameters fParams = { 1 /*content*/, 0 /*checksum*/, 0 /*noDictID*/ };
-    ZSTD_buffered_policy_e const zbuff = (ZSTD_buffered_policy_e)(srcCCtx->inBuffSize>0);
-    ZSTD_STATIC_ASSERT((U32)ZSTDb_buffered==1);
-    if (pledgedSrcSize==0) pledgedSrcSize = ZSTD_CONTENTSIZE_UNKNOWN;
-    fParams.contentSizeFlag = (pledgedSrcSize != ZSTD_CONTENTSIZE_UNKNOWN);
-
-    return ZSTD_copyCCtx_internal(dstCCtx, srcCCtx,
-                                fParams, pledgedSrcSize,
-                                zbuff);
-}
-
-
-#define ZSTD_ROWSIZE 16
-/*! ZSTD_reduceTable() :
- *  reduce table indexes by `reducerValue`, or squash to zero.
- *  PreserveMark preserves "unsorted mark" for btlazy2 strategy.
- *  It must be set to a clear 0/1 value, to remove branch during inlining.
- *  Presume table size is a multiple of ZSTD_ROWSIZE
- *  to help auto-vectorization */
-FORCE_INLINE_TEMPLATE void
-ZSTD_reduceTable_internal (U32* const table, U32 const size, U32 const reducerValue, int const preserveMark)
-{
-    int const nbRows = (int)size / ZSTD_ROWSIZE;
-    int cellNb = 0;
-    int rowNb;
-    assert((size & (ZSTD_ROWSIZE-1)) == 0);  /* multiple of ZSTD_ROWSIZE */
-    assert(size < (1U<<31));   /* can be casted to int */
-    for (rowNb=0 ; rowNb < nbRows ; rowNb++) {
-        int column;
-        for (column=0; column<ZSTD_ROWSIZE; column++) {
-            if (preserveMark) {
-                U32 const adder = (table[cellNb] == ZSTD_DUBT_UNSORTED_MARK) ? reducerValue : 0;
-                table[cellNb] += adder;
-            }
-            if (table[cellNb] < reducerValue) table[cellNb] = 0;
-            else table[cellNb] -= reducerValue;
-            cellNb++;
-    }   }
-}
-
-static void ZSTD_reduceTable(U32* const table, U32 const size, U32 const reducerValue)
-{
-    ZSTD_reduceTable_internal(table, size, reducerValue, 0);
-}
-
-static void ZSTD_reduceTable_btlazy2(U32* const table, U32 const size, U32 const reducerValue)
-{
-    ZSTD_reduceTable_internal(table, size, reducerValue, 1);
-}
-
-/*! ZSTD_reduceIndex() :
-*   rescale all indexes to avoid future overflow (indexes are U32) */
-static void ZSTD_reduceIndex (ZSTD_matchState_t* ms, ZSTD_CCtx_params const* params, const U32 reducerValue)
-{
-    {   U32 const hSize = (U32)1 << params->cParams.hashLog;
-        ZSTD_reduceTable(ms->hashTable, hSize, reducerValue);
-    }
-
-    if (params->cParams.strategy != ZSTD_fast) {
-        U32 const chainSize = (U32)1 << params->cParams.chainLog;
-        if (params->cParams.strategy == ZSTD_btlazy2)
-            ZSTD_reduceTable_btlazy2(ms->chainTable, chainSize, reducerValue);
-        else
-            ZSTD_reduceTable(ms->chainTable, chainSize, reducerValue);
-    }
-
-    if (ms->hashLog3) {
-        U32 const h3Size = (U32)1 << ms->hashLog3;
-        ZSTD_reduceTable(ms->hashTable3, h3Size, reducerValue);
-    }
-}
-
-
-/*-*******************************************************
-*  Block entropic compression
-*********************************************************/
-
-/* See doc/zstd_compression_format.md for detailed format description */
-
-static size_t ZSTD_noCompressBlock (void* dst, size_t dstCapacity, const void* src, size_t srcSize, U32 lastBlock)
-{
-    U32 const cBlockHeader24 = lastBlock + (((U32)bt_raw)<<1) + (U32)(srcSize << 3);
-    RETURN_ERROR_IF(srcSize + ZSTD_blockHeaderSize > dstCapacity,
-                    dstSize_tooSmall);
-    MEM_writeLE24(dst, cBlockHeader24);
-    memcpy((BYTE*)dst + ZSTD_blockHeaderSize, src, srcSize);
-    return ZSTD_blockHeaderSize + srcSize;
-}
-
-static size_t ZSTD_noCompressLiterals (void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    BYTE* const ostart = (BYTE* const)dst;
-    U32   const flSize = 1 + (srcSize>31) + (srcSize>4095);
-
-    RETURN_ERROR_IF(srcSize + flSize > dstCapacity, dstSize_tooSmall);
-
-    switch(flSize)
-    {
-        case 1: /* 2 - 1 - 5 */
-            ostart[0] = (BYTE)((U32)set_basic + (srcSize<<3));
-            break;
-        case 2: /* 2 - 2 - 12 */
-            MEM_writeLE16(ostart, (U16)((U32)set_basic + (1<<2) + (srcSize<<4)));
-            break;
-        case 3: /* 2 - 2 - 20 */
-            MEM_writeLE32(ostart, (U32)((U32)set_basic + (3<<2) + (srcSize<<4)));
-            break;
-        default:   /* not necessary : flSize is {1,2,3} */
-            assert(0);
-    }
-
-    memcpy(ostart + flSize, src, srcSize);
-    return srcSize + flSize;
-}
-
-static size_t ZSTD_compressRleLiteralsBlock (void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    BYTE* const ostart = (BYTE* const)dst;
-    U32   const flSize = 1 + (srcSize>31) + (srcSize>4095);
-
-    (void)dstCapacity;  /* dstCapacity already guaranteed to be >=4, hence large enough */
-
-    switch(flSize)
-    {
-        case 1: /* 2 - 1 - 5 */
-            ostart[0] = (BYTE)((U32)set_rle + (srcSize<<3));
-            break;
-        case 2: /* 2 - 2 - 12 */
-            MEM_writeLE16(ostart, (U16)((U32)set_rle + (1<<2) + (srcSize<<4)));
-            break;
-        case 3: /* 2 - 2 - 20 */
-            MEM_writeLE32(ostart, (U32)((U32)set_rle + (3<<2) + (srcSize<<4)));
-            break;
-        default:   /* not necessary : flSize is {1,2,3} */
-            assert(0);
-    }
-
-    ostart[flSize] = *(const BYTE*)src;
-    return flSize+1;
-}
-
-
-/* ZSTD_minGain() :
- * minimum compression required
- * to generate a compress block or a compressed literals section.
- * note : use same formula for both situations */
-static size_t ZSTD_minGain(size_t srcSize, ZSTD_strategy strat)
-{
-    U32 const minlog = (strat>=ZSTD_btultra) ? (U32)(strat) - 1 : 6;
-    ZSTD_STATIC_ASSERT(ZSTD_btultra == 8);
-    assert(ZSTD_cParam_withinBounds(ZSTD_c_strategy, strat));
-    return (srcSize >> minlog) + 2;
-}
-
-static size_t ZSTD_compressLiterals (ZSTD_hufCTables_t const* prevHuf,
-                                     ZSTD_hufCTables_t* nextHuf,
-                                     ZSTD_strategy strategy, int disableLiteralCompression,
-                                     void* dst, size_t dstCapacity,
-                               const void* src, size_t srcSize,
-                                     void* workspace, size_t wkspSize,
-                               const int bmi2)
-{
-    size_t const minGain = ZSTD_minGain(srcSize, strategy);
-    size_t const lhSize = 3 + (srcSize >= 1 KB) + (srcSize >= 16 KB);
-    BYTE*  const ostart = (BYTE*)dst;
-    U32 singleStream = srcSize < 256;
-    symbolEncodingType_e hType = set_compressed;
-    size_t cLitSize;
-
-    DEBUGLOG(5,"ZSTD_compressLiterals (disableLiteralCompression=%i)",
-                disableLiteralCompression);
-
-    /* Prepare nextEntropy assuming reusing the existing table */
-    memcpy(nextHuf, prevHuf, sizeof(*prevHuf));
-
-    if (disableLiteralCompression)
-        return ZSTD_noCompressLiterals(dst, dstCapacity, src, srcSize);
-
-    /* small ? don't even attempt compression (speed opt) */
-#   define COMPRESS_LITERALS_SIZE_MIN 63
-    {   size_t const minLitSize = (prevHuf->repeatMode == HUF_repeat_valid) ? 6 : COMPRESS_LITERALS_SIZE_MIN;
-        if (srcSize <= minLitSize) return ZSTD_noCompressLiterals(dst, dstCapacity, src, srcSize);
-    }
-
-    RETURN_ERROR_IF(dstCapacity < lhSize+1, dstSize_tooSmall, "not enough space for compression");
-    {   HUF_repeat repeat = prevHuf->repeatMode;
-        int const preferRepeat = strategy < ZSTD_lazy ? srcSize <= 1024 : 0;
-        if (repeat == HUF_repeat_valid && lhSize == 3) singleStream = 1;
-        cLitSize = singleStream ? HUF_compress1X_repeat(ostart+lhSize, dstCapacity-lhSize, src, srcSize, 255, 11,
-                                      workspace, wkspSize, (HUF_CElt*)nextHuf->CTable, &repeat, preferRepeat, bmi2)
-                                : HUF_compress4X_repeat(ostart+lhSize, dstCapacity-lhSize, src, srcSize, 255, 11,
-                                      workspace, wkspSize, (HUF_CElt*)nextHuf->CTable, &repeat, preferRepeat, bmi2);
-        if (repeat != HUF_repeat_none) {
-            /* reused the existing table */
-            hType = set_repeat;
-        }
-    }
-
-    if ((cLitSize==0) | (cLitSize >= srcSize - minGain) | ERR_isError(cLitSize)) {
-        memcpy(nextHuf, prevHuf, sizeof(*prevHuf));
-        return ZSTD_noCompressLiterals(dst, dstCapacity, src, srcSize);
-    }
-    if (cLitSize==1) {
-        memcpy(nextHuf, prevHuf, sizeof(*prevHuf));
-        return ZSTD_compressRleLiteralsBlock(dst, dstCapacity, src, srcSize);
-    }
-
-    if (hType == set_compressed) {
-        /* using a newly constructed table */
-        nextHuf->repeatMode = HUF_repeat_check;
-    }
-
-    /* Build header */
-    switch(lhSize)
-    {
-    case 3: /* 2 - 2 - 10 - 10 */
-        {   U32 const lhc = hType + ((!singleStream) << 2) + ((U32)srcSize<<4) + ((U32)cLitSize<<14);
-            MEM_writeLE24(ostart, lhc);
-            break;
-        }
-    case 4: /* 2 - 2 - 14 - 14 */
-        {   U32 const lhc = hType + (2 << 2) + ((U32)srcSize<<4) + ((U32)cLitSize<<18);
-            MEM_writeLE32(ostart, lhc);
-            break;
-        }
-    case 5: /* 2 - 2 - 18 - 18 */
-        {   U32 const lhc = hType + (3 << 2) + ((U32)srcSize<<4) + ((U32)cLitSize<<22);
-            MEM_writeLE32(ostart, lhc);
-            ostart[4] = (BYTE)(cLitSize >> 10);
-            break;
-        }
-    default:  /* not possible : lhSize is {3,4,5} */
-        assert(0);
-    }
-    return lhSize+cLitSize;
-}
-
-
-void ZSTD_seqToCodes(const seqStore_t* seqStorePtr)
-{
-    const seqDef* const sequences = seqStorePtr->sequencesStart;
-    BYTE* const llCodeTable = seqStorePtr->llCode;
-    BYTE* const ofCodeTable = seqStorePtr->ofCode;
-    BYTE* const mlCodeTable = seqStorePtr->mlCode;
-    U32 const nbSeq = (U32)(seqStorePtr->sequences - seqStorePtr->sequencesStart);
-    U32 u;
-    assert(nbSeq <= seqStorePtr->maxNbSeq);
-    for (u=0; u<nbSeq; u++) {
-        U32 const llv = sequences[u].litLength;
-        U32 const mlv = sequences[u].matchLength;
-        llCodeTable[u] = (BYTE)ZSTD_LLcode(llv);
-        ofCodeTable[u] = (BYTE)ZSTD_highbit32(sequences[u].offset);
-        mlCodeTable[u] = (BYTE)ZSTD_MLcode(mlv);
-    }
-    if (seqStorePtr->longLengthID==1)
-        llCodeTable[seqStorePtr->longLengthPos] = MaxLL;
-    if (seqStorePtr->longLengthID==2)
-        mlCodeTable[seqStorePtr->longLengthPos] = MaxML;
-}
-
-
-/**
- * -log2(x / 256) lookup table for x in [0, 256).
- * If x == 0: Return 0
- * Else: Return floor(-log2(x / 256) * 256)
- */
-static unsigned const kInverseProbabilityLog256[256] = {
-    0,    2048, 1792, 1642, 1536, 1453, 1386, 1329, 1280, 1236, 1197, 1162,
-    1130, 1100, 1073, 1047, 1024, 1001, 980,  960,  941,  923,  906,  889,
-    874,  859,  844,  830,  817,  804,  791,  779,  768,  756,  745,  734,
-    724,  714,  704,  694,  685,  676,  667,  658,  650,  642,  633,  626,
-    618,  610,  603,  595,  588,  581,  574,  567,  561,  554,  548,  542,
-    535,  529,  523,  517,  512,  506,  500,  495,  489,  484,  478,  473,
-    468,  463,  458,  453,  448,  443,  438,  434,  429,  424,  420,  415,
-    411,  407,  402,  398,  394,  390,  386,  382,  377,  373,  370,  366,
-    362,  358,  354,  350,  347,  343,  339,  336,  332,  329,  325,  322,
-    318,  315,  311,  308,  305,  302,  298,  295,  292,  289,  286,  282,
-    279,  276,  273,  270,  267,  264,  261,  258,  256,  253,  250,  247,
-    244,  241,  239,  236,  233,  230,  228,  225,  222,  220,  217,  215,
-    212,  209,  207,  204,  202,  199,  197,  194,  192,  190,  187,  185,
-    182,  180,  178,  175,  173,  171,  168,  166,  164,  162,  159,  157,
-    155,  153,  151,  149,  146,  144,  142,  140,  138,  136,  134,  132,
-    130,  128,  126,  123,  121,  119,  117,  115,  114,  112,  110,  108,
-    106,  104,  102,  100,  98,   96,   94,   93,   91,   89,   87,   85,
-    83,   82,   80,   78,   76,   74,   73,   71,   69,   67,   66,   64,
-    62,   61,   59,   57,   55,   54,   52,   50,   49,   47,   46,   44,
-    42,   41,   39,   37,   36,   34,   33,   31,   30,   28,   26,   25,
-    23,   22,   20,   19,   17,   16,   14,   13,   11,   10,   8,    7,
-    5,    4,    2,    1,
-};
-
-
-/**
- * Returns the cost in bits of encoding the distribution described by count
- * using the entropy bound.
- */
-static size_t ZSTD_entropyCost(unsigned const* count, unsigned const max, size_t const total)
-{
-    unsigned cost = 0;
-    unsigned s;
-    for (s = 0; s <= max; ++s) {
-        unsigned norm = (unsigned)((256 * count[s]) / total);
-        if (count[s] != 0 && norm == 0)
-            norm = 1;
-        assert(count[s] < total);
-        cost += count[s] * kInverseProbabilityLog256[norm];
-    }
-    return cost >> 8;
-}
-
-
-/**
- * Returns the cost in bits of encoding the distribution in count using the
- * table described by norm. The max symbol support by norm is assumed >= max.
- * norm must be valid for every symbol with non-zero probability in count.
- */
-static size_t ZSTD_crossEntropyCost(short const* norm, unsigned accuracyLog,
-                                    unsigned const* count, unsigned const max)
-{
-    unsigned const shift = 8 - accuracyLog;
-    size_t cost = 0;
-    unsigned s;
-    assert(accuracyLog <= 8);
-    for (s = 0; s <= max; ++s) {
-        unsigned const normAcc = norm[s] != -1 ? norm[s] : 1;
-        unsigned const norm256 = normAcc << shift;
-        assert(norm256 > 0);
-        assert(norm256 < 256);
-        cost += count[s] * kInverseProbabilityLog256[norm256];
-    }
-    return cost >> 8;
-}
-
-
-static unsigned ZSTD_getFSEMaxSymbolValue(FSE_CTable const* ctable) {
-  void const* ptr = ctable;
-  U16 const* u16ptr = (U16 const*)ptr;
-  U32 const maxSymbolValue = MEM_read16(u16ptr + 1);
-  return maxSymbolValue;
-}
-
-
-/**
- * Returns the cost in bits of encoding the distribution in count using ctable.
- * Returns an error if ctable cannot represent all the symbols in count.
- */
-static size_t ZSTD_fseBitCost(
-    FSE_CTable const* ctable,
-    unsigned const* count,
-    unsigned const max)
-{
-    unsigned const kAccuracyLog = 8;
-    size_t cost = 0;
-    unsigned s;
-    FSE_CState_t cstate;
-    FSE_initCState(&cstate, ctable);
-    RETURN_ERROR_IF(ZSTD_getFSEMaxSymbolValue(ctable) < max, GENERIC,
-                    "Repeat FSE_CTable has maxSymbolValue %u < %u",
-                    ZSTD_getFSEMaxSymbolValue(ctable), max);
-    for (s = 0; s <= max; ++s) {
-        unsigned const tableLog = cstate.stateLog;
-        unsigned const badCost = (tableLog + 1) << kAccuracyLog;
-        unsigned const bitCost = FSE_bitCost(cstate.symbolTT, tableLog, s, kAccuracyLog);
-        if (count[s] == 0)
-            continue;
-        RETURN_ERROR_IF(bitCost >= badCost, GENERIC,
-                        "Repeat FSE_CTable has Prob[%u] == 0", s);
-        cost += count[s] * bitCost;
-    }
-    return cost >> kAccuracyLog;
-}
-
-/**
- * Returns the cost in bytes of encoding the normalized count header.
- * Returns an error if any of the helper functions return an error.
- */
-static size_t ZSTD_NCountCost(unsigned const* count, unsigned const max,
-                              size_t const nbSeq, unsigned const FSELog)
-{
-    BYTE wksp[FSE_NCOUNTBOUND];
-    S16 norm[MaxSeq + 1];
-    const U32 tableLog = FSE_optimalTableLog(FSELog, nbSeq, max);
-    FORWARD_IF_ERROR(FSE_normalizeCount(norm, tableLog, count, nbSeq, max));
-    return FSE_writeNCount(wksp, sizeof(wksp), norm, max, tableLog);
-}
-
-
-typedef enum {
-    ZSTD_defaultDisallowed = 0,
-    ZSTD_defaultAllowed = 1
-} ZSTD_defaultPolicy_e;
-
-MEM_STATIC symbolEncodingType_e
-ZSTD_selectEncodingType(
-        FSE_repeat* repeatMode, unsigned const* count, unsigned const max,
-        size_t const mostFrequent, size_t nbSeq, unsigned const FSELog,
-        FSE_CTable const* prevCTable,
-        short const* defaultNorm, U32 defaultNormLog,
-        ZSTD_defaultPolicy_e const isDefaultAllowed,
-        ZSTD_strategy const strategy)
-{
-    ZSTD_STATIC_ASSERT(ZSTD_defaultDisallowed == 0 && ZSTD_defaultAllowed != 0);
-    if (mostFrequent == nbSeq) {
-        *repeatMode = FSE_repeat_none;
-        if (isDefaultAllowed && nbSeq <= 2) {
-            /* Prefer set_basic over set_rle when there are 2 or less symbols,
-             * since RLE uses 1 byte, but set_basic uses 5-6 bits per symbol.
-             * If basic encoding isn't possible, always choose RLE.
-             */
-            DEBUGLOG(5, "Selected set_basic");
-            return set_basic;
-        }
-        DEBUGLOG(5, "Selected set_rle");
-        return set_rle;
-    }
-    if (strategy < ZSTD_lazy) {
-        if (isDefaultAllowed) {
-            size_t const staticFse_nbSeq_max = 1000;
-            size_t const mult = 10 - strategy;
-            size_t const baseLog = 3;
-            size_t const dynamicFse_nbSeq_min = (((size_t)1 << defaultNormLog) * mult) >> baseLog;  /* 28-36 for offset, 56-72 for lengths */
-            assert(defaultNormLog >= 5 && defaultNormLog <= 6);  /* xx_DEFAULTNORMLOG */
-            assert(mult <= 9 && mult >= 7);
-            if ( (*repeatMode == FSE_repeat_valid)
-              && (nbSeq < staticFse_nbSeq_max) ) {
-                DEBUGLOG(5, "Selected set_repeat");
-                return set_repeat;
-            }
-            if ( (nbSeq < dynamicFse_nbSeq_min)
-              || (mostFrequent < (nbSeq >> (defaultNormLog-1))) ) {
-                DEBUGLOG(5, "Selected set_basic");
-                /* The format allows default tables to be repeated, but it isn't useful.
-                 * When using simple heuristics to select encoding type, we don't want
-                 * to confuse these tables with dictionaries. When running more careful
-                 * analysis, we don't need to waste time checking both repeating tables
-                 * and default tables.
-                 */
-                *repeatMode = FSE_repeat_none;
-                return set_basic;
-            }
-        }
-    } else {
-        size_t const basicCost = isDefaultAllowed ? ZSTD_crossEntropyCost(defaultNorm, defaultNormLog, count, max) : ERROR(GENERIC);
-        size_t const repeatCost = *repeatMode != FSE_repeat_none ? ZSTD_fseBitCost(prevCTable, count, max) : ERROR(GENERIC);
-        size_t const NCountCost = ZSTD_NCountCost(count, max, nbSeq, FSELog);
-        size_t const compressedCost = (NCountCost << 3) + ZSTD_entropyCost(count, max, nbSeq);
-
-        if (isDefaultAllowed) {
-            assert(!ZSTD_isError(basicCost));
-            assert(!(*repeatMode == FSE_repeat_valid && ZSTD_isError(repeatCost)));
-        }
-        assert(!ZSTD_isError(NCountCost));
-        assert(compressedCost < ERROR(maxCode));
-        DEBUGLOG(5, "Estimated bit costs: basic=%u\trepeat=%u\tcompressed=%u",
-                    (unsigned)basicCost, (unsigned)repeatCost, (unsigned)compressedCost);
-        if (basicCost <= repeatCost && basicCost <= compressedCost) {
-            DEBUGLOG(5, "Selected set_basic");
-            assert(isDefaultAllowed);
-            *repeatMode = FSE_repeat_none;
-            return set_basic;
-        }
-        if (repeatCost <= compressedCost) {
-            DEBUGLOG(5, "Selected set_repeat");
-            assert(!ZSTD_isError(repeatCost));
-            return set_repeat;
-        }
-        assert(compressedCost < basicCost && compressedCost < repeatCost);
-    }
-    DEBUGLOG(5, "Selected set_compressed");
-    *repeatMode = FSE_repeat_check;
-    return set_compressed;
-}
-
-MEM_STATIC size_t
-ZSTD_buildCTable(void* dst, size_t dstCapacity,
-                FSE_CTable* nextCTable, U32 FSELog, symbolEncodingType_e type,
-                unsigned* count, U32 max,
-                const BYTE* codeTable, size_t nbSeq,
-                const S16* defaultNorm, U32 defaultNormLog, U32 defaultMax,
-                const FSE_CTable* prevCTable, size_t prevCTableSize,
-                void* workspace, size_t workspaceSize)
-{
-    BYTE* op = (BYTE*)dst;
-    const BYTE* const oend = op + dstCapacity;
-    DEBUGLOG(6, "ZSTD_buildCTable (dstCapacity=%u)", (unsigned)dstCapacity);
-
-    switch (type) {
-    case set_rle:
-        FORWARD_IF_ERROR(FSE_buildCTable_rle(nextCTable, (BYTE)max));
-        RETURN_ERROR_IF(dstCapacity==0, dstSize_tooSmall);
-        *op = codeTable[0];
-        return 1;
-    case set_repeat:
-        memcpy(nextCTable, prevCTable, prevCTableSize);
-        return 0;
-    case set_basic:
-        FORWARD_IF_ERROR(FSE_buildCTable_wksp(nextCTable, defaultNorm, defaultMax, defaultNormLog, workspace, workspaceSize));  /* note : could be pre-calculated */
-        return 0;
-    case set_compressed: {
-        S16 norm[MaxSeq + 1];
-        size_t nbSeq_1 = nbSeq;
-        const U32 tableLog = FSE_optimalTableLog(FSELog, nbSeq, max);
-        if (count[codeTable[nbSeq-1]] > 1) {
-            count[codeTable[nbSeq-1]]--;
-            nbSeq_1--;
-        }
-        assert(nbSeq_1 > 1);
-        FORWARD_IF_ERROR(FSE_normalizeCount(norm, tableLog, count, nbSeq_1, max));
-        {   size_t const NCountSize = FSE_writeNCount(op, oend - op, norm, max, tableLog);   /* overflow protected */
-            FORWARD_IF_ERROR(NCountSize);
-            FORWARD_IF_ERROR(FSE_buildCTable_wksp(nextCTable, norm, max, tableLog, workspace, workspaceSize));
-            return NCountSize;
-        }
-    }
-    default: assert(0); RETURN_ERROR(GENERIC);
-    }
-}
-
-FORCE_INLINE_TEMPLATE size_t
-ZSTD_encodeSequences_body(
-            void* dst, size_t dstCapacity,
-            FSE_CTable const* CTable_MatchLength, BYTE const* mlCodeTable,
-            FSE_CTable const* CTable_OffsetBits, BYTE const* ofCodeTable,
-            FSE_CTable const* CTable_LitLength, BYTE const* llCodeTable,
-            seqDef const* sequences, size_t nbSeq, int longOffsets)
-{
-    BIT_CStream_t blockStream;
-    FSE_CState_t  stateMatchLength;
-    FSE_CState_t  stateOffsetBits;
-    FSE_CState_t  stateLitLength;
-
-    RETURN_ERROR_IF(
-        ERR_isError(BIT_initCStream(&blockStream, dst, dstCapacity)),
-        dstSize_tooSmall, "not enough space remaining");
-    DEBUGLOG(6, "available space for bitstream : %i  (dstCapacity=%u)",
-                (int)(blockStream.endPtr - blockStream.startPtr),
-                (unsigned)dstCapacity);
-
-    /* first symbols */
-    FSE_initCState2(&stateMatchLength, CTable_MatchLength, mlCodeTable[nbSeq-1]);
-    FSE_initCState2(&stateOffsetBits,  CTable_OffsetBits,  ofCodeTable[nbSeq-1]);
-    FSE_initCState2(&stateLitLength,   CTable_LitLength,   llCodeTable[nbSeq-1]);
-    BIT_addBits(&blockStream, sequences[nbSeq-1].litLength, LL_bits[llCodeTable[nbSeq-1]]);
-    if (MEM_32bits()) BIT_flushBits(&blockStream);
-    BIT_addBits(&blockStream, sequences[nbSeq-1].matchLength, ML_bits[mlCodeTable[nbSeq-1]]);
-    if (MEM_32bits()) BIT_flushBits(&blockStream);
-    if (longOffsets) {
-        U32 const ofBits = ofCodeTable[nbSeq-1];
-        int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1);
-        if (extraBits) {
-            BIT_addBits(&blockStream, sequences[nbSeq-1].offset, extraBits);
-            BIT_flushBits(&blockStream);
-        }
-        BIT_addBits(&blockStream, sequences[nbSeq-1].offset >> extraBits,
-                    ofBits - extraBits);
-    } else {
-        BIT_addBits(&blockStream, sequences[nbSeq-1].offset, ofCodeTable[nbSeq-1]);
-    }
-    BIT_flushBits(&blockStream);
-
-    {   size_t n;
-        for (n=nbSeq-2 ; n<nbSeq ; n--) {      /* intentional underflow */
-            BYTE const llCode = llCodeTable[n];
-            BYTE const ofCode = ofCodeTable[n];
-            BYTE const mlCode = mlCodeTable[n];
-            U32  const llBits = LL_bits[llCode];
-            U32  const ofBits = ofCode;
-            U32  const mlBits = ML_bits[mlCode];
-            DEBUGLOG(6, "encoding: litlen:%2u - matchlen:%2u - offCode:%7u",
-                        (unsigned)sequences[n].litLength,
-                        (unsigned)sequences[n].matchLength + MINMATCH,
-                        (unsigned)sequences[n].offset);
-                                                                            /* 32b*/  /* 64b*/
-                                                                            /* (7)*/  /* (7)*/
-            FSE_encodeSymbol(&blockStream, &stateOffsetBits, ofCode);       /* 15 */  /* 15 */
-            FSE_encodeSymbol(&blockStream, &stateMatchLength, mlCode);      /* 24 */  /* 24 */
-            if (MEM_32bits()) BIT_flushBits(&blockStream);                  /* (7)*/
-            FSE_encodeSymbol(&blockStream, &stateLitLength, llCode);        /* 16 */  /* 33 */
-            if (MEM_32bits() || (ofBits+mlBits+llBits >= 64-7-(LLFSELog+MLFSELog+OffFSELog)))
-                BIT_flushBits(&blockStream);                                /* (7)*/
-            BIT_addBits(&blockStream, sequences[n].litLength, llBits);
-            if (MEM_32bits() && ((llBits+mlBits)>24)) BIT_flushBits(&blockStream);
-            BIT_addBits(&blockStream, sequences[n].matchLength, mlBits);
-            if (MEM_32bits() || (ofBits+mlBits+llBits > 56)) BIT_flushBits(&blockStream);
-            if (longOffsets) {
-                int const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN-1);
-                if (extraBits) {
-                    BIT_addBits(&blockStream, sequences[n].offset, extraBits);
-                    BIT_flushBits(&blockStream);                            /* (7)*/
-                }
-                BIT_addBits(&blockStream, sequences[n].offset >> extraBits,
-                            ofBits - extraBits);                            /* 31 */
-            } else {
-                BIT_addBits(&blockStream, sequences[n].offset, ofBits);     /* 31 */
-            }
-            BIT_flushBits(&blockStream);                                    /* (7)*/
-            DEBUGLOG(7, "remaining space : %i", (int)(blockStream.endPtr - blockStream.ptr));
-    }   }
-
-    DEBUGLOG(6, "ZSTD_encodeSequences: flushing ML state with %u bits", stateMatchLength.stateLog);
-    FSE_flushCState(&blockStream, &stateMatchLength);
-    DEBUGLOG(6, "ZSTD_encodeSequences: flushing Off state with %u bits", stateOffsetBits.stateLog);
-    FSE_flushCState(&blockStream, &stateOffsetBits);
-    DEBUGLOG(6, "ZSTD_encodeSequences: flushing LL state with %u bits", stateLitLength.stateLog);
-    FSE_flushCState(&blockStream, &stateLitLength);
-
-    {   size_t const streamSize = BIT_closeCStream(&blockStream);
-        RETURN_ERROR_IF(streamSize==0, dstSize_tooSmall, "not enough space");
-        return streamSize;
-    }
-}
-
-static size_t
-ZSTD_encodeSequences_default(
-            void* dst, size_t dstCapacity,
-            FSE_CTable const* CTable_MatchLength, BYTE const* mlCodeTable,
-            FSE_CTable const* CTable_OffsetBits, BYTE const* ofCodeTable,
-            FSE_CTable const* CTable_LitLength, BYTE const* llCodeTable,
-            seqDef const* sequences, size_t nbSeq, int longOffsets)
-{
-    return ZSTD_encodeSequences_body(dst, dstCapacity,
-                                    CTable_MatchLength, mlCodeTable,
-                                    CTable_OffsetBits, ofCodeTable,
-                                    CTable_LitLength, llCodeTable,
-                                    sequences, nbSeq, longOffsets);
-}
-
-
-#if DYNAMIC_BMI2
-
-static TARGET_ATTRIBUTE("bmi2") size_t
-ZSTD_encodeSequences_bmi2(
-            void* dst, size_t dstCapacity,
-            FSE_CTable const* CTable_MatchLength, BYTE const* mlCodeTable,
-            FSE_CTable const* CTable_OffsetBits, BYTE const* ofCodeTable,
-            FSE_CTable const* CTable_LitLength, BYTE const* llCodeTable,
-            seqDef const* sequences, size_t nbSeq, int longOffsets)
-{
-    return ZSTD_encodeSequences_body(dst, dstCapacity,
-                                    CTable_MatchLength, mlCodeTable,
-                                    CTable_OffsetBits, ofCodeTable,
-                                    CTable_LitLength, llCodeTable,
-                                    sequences, nbSeq, longOffsets);
-}
-
-#endif
-
-static size_t ZSTD_encodeSequences(
-            void* dst, size_t dstCapacity,
-            FSE_CTable const* CTable_MatchLength, BYTE const* mlCodeTable,
-            FSE_CTable const* CTable_OffsetBits, BYTE const* ofCodeTable,
-            FSE_CTable const* CTable_LitLength, BYTE const* llCodeTable,
-            seqDef const* sequences, size_t nbSeq, int longOffsets, int bmi2)
-{
-    DEBUGLOG(5, "ZSTD_encodeSequences: dstCapacity = %u", (unsigned)dstCapacity);
-#if DYNAMIC_BMI2
-    if (bmi2) {
-        return ZSTD_encodeSequences_bmi2(dst, dstCapacity,
-                                         CTable_MatchLength, mlCodeTable,
-                                         CTable_OffsetBits, ofCodeTable,
-                                         CTable_LitLength, llCodeTable,
-                                         sequences, nbSeq, longOffsets);
-    }
-#endif
-    (void)bmi2;
-    return ZSTD_encodeSequences_default(dst, dstCapacity,
-                                        CTable_MatchLength, mlCodeTable,
-                                        CTable_OffsetBits, ofCodeTable,
-                                        CTable_LitLength, llCodeTable,
-                                        sequences, nbSeq, longOffsets);
-}
-
-static int ZSTD_disableLiteralsCompression(const ZSTD_CCtx_params* cctxParams)
-{
-    switch (cctxParams->literalCompressionMode) {
-    case ZSTD_lcm_huffman:
-        return 0;
-    case ZSTD_lcm_uncompressed:
-        return 1;
-    default:
-        assert(0 /* impossible: pre-validated */);
-        /* fall-through */
-    case ZSTD_lcm_auto:
-        return (cctxParams->cParams.strategy == ZSTD_fast) && (cctxParams->cParams.targetLength > 0);
-    }
-}
-
-/* ZSTD_compressSequences_internal():
- * actually compresses both literals and sequences */
-MEM_STATIC size_t
-ZSTD_compressSequences_internal(seqStore_t* seqStorePtr,
-                          const ZSTD_entropyCTables_t* prevEntropy,
-                                ZSTD_entropyCTables_t* nextEntropy,
-                          const ZSTD_CCtx_params* cctxParams,
-                                void* dst, size_t dstCapacity,
-                                void* workspace, size_t wkspSize,
-                          const int bmi2)
-{
-    const int longOffsets = cctxParams->cParams.windowLog > STREAM_ACCUMULATOR_MIN;
-    ZSTD_strategy const strategy = cctxParams->cParams.strategy;
-    unsigned count[MaxSeq+1];
-    FSE_CTable* CTable_LitLength = nextEntropy->fse.litlengthCTable;
-    FSE_CTable* CTable_OffsetBits = nextEntropy->fse.offcodeCTable;
-    FSE_CTable* CTable_MatchLength = nextEntropy->fse.matchlengthCTable;
-    U32 LLtype, Offtype, MLtype;   /* compressed, raw or rle */
-    const seqDef* const sequences = seqStorePtr->sequencesStart;
-    const BYTE* const ofCodeTable = seqStorePtr->ofCode;
-    const BYTE* const llCodeTable = seqStorePtr->llCode;
-    const BYTE* const mlCodeTable = seqStorePtr->mlCode;
-    BYTE* const ostart = (BYTE*)dst;
-    BYTE* const oend = ostart + dstCapacity;
-    BYTE* op = ostart;
-    size_t const nbSeq = seqStorePtr->sequences - seqStorePtr->sequencesStart;
-    BYTE* seqHead;
-    BYTE* lastNCount = NULL;
-
-    ZSTD_STATIC_ASSERT(HUF_WORKSPACE_SIZE >= (1<<MAX(MLFSELog,LLFSELog)));
-    DEBUGLOG(5, "ZSTD_compressSequences_internal");
-
-    /* Compress literals */
-    {   const BYTE* const literals = seqStorePtr->litStart;
-        size_t const litSize = seqStorePtr->lit - literals;
-        size_t const cSize = ZSTD_compressLiterals(
-                                    &prevEntropy->huf, &nextEntropy->huf,
-                                    cctxParams->cParams.strategy,
-                                    ZSTD_disableLiteralsCompression(cctxParams),
-                                    op, dstCapacity,
-                                    literals, litSize,
-                                    workspace, wkspSize,
-                                    bmi2);
-        FORWARD_IF_ERROR(cSize);
-        assert(cSize <= dstCapacity);
-        op += cSize;
-    }
-
-    /* Sequences Header */
-    RETURN_ERROR_IF((oend-op) < 3 /*max nbSeq Size*/ + 1 /*seqHead*/,
-                    dstSize_tooSmall);
-    if (nbSeq < 0x7F)
-        *op++ = (BYTE)nbSeq;
-    else if (nbSeq < LONGNBSEQ)
-        op[0] = (BYTE)((nbSeq>>8) + 0x80), op[1] = (BYTE)nbSeq, op+=2;
-    else
-        op[0]=0xFF, MEM_writeLE16(op+1, (U16)(nbSeq - LONGNBSEQ)), op+=3;
-    assert(op <= oend);
-    if (nbSeq==0) {
-        /* Copy the old tables over as if we repeated them */
-        memcpy(&nextEntropy->fse, &prevEntropy->fse, sizeof(prevEntropy->fse));
-        return op - ostart;
-    }
-
-    /* seqHead : flags for FSE encoding type */
-    seqHead = op++;
-    assert(op <= oend);
-
-    /* convert length/distances into codes */
-    ZSTD_seqToCodes(seqStorePtr);
-    /* build CTable for Literal Lengths */
-    {   unsigned max = MaxLL;
-        size_t const mostFrequent = HIST_countFast_wksp(count, &max, llCodeTable, nbSeq, workspace, wkspSize);   /* can't fail */
-        DEBUGLOG(5, "Building LL table");
-        nextEntropy->fse.litlength_repeatMode = prevEntropy->fse.litlength_repeatMode;
-        LLtype = ZSTD_selectEncodingType(&nextEntropy->fse.litlength_repeatMode,
-                                        count, max, mostFrequent, nbSeq,
-                                        LLFSELog, prevEntropy->fse.litlengthCTable,
-                                        LL_defaultNorm, LL_defaultNormLog,
-                                        ZSTD_defaultAllowed, strategy);
-        assert(set_basic < set_compressed && set_rle < set_compressed);
-        assert(!(LLtype < set_compressed && nextEntropy->fse.litlength_repeatMode != FSE_repeat_none)); /* We don't copy tables */
-        {   size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_LitLength, LLFSELog, (symbolEncodingType_e)LLtype,
-                                                    count, max, llCodeTable, nbSeq, LL_defaultNorm, LL_defaultNormLog, MaxLL,
-                                                    prevEntropy->fse.litlengthCTable, sizeof(prevEntropy->fse.litlengthCTable),
-                                                    workspace, wkspSize);
-            FORWARD_IF_ERROR(countSize);
-            if (LLtype == set_compressed)
-                lastNCount = op;
-            op += countSize;
-            assert(op <= oend);
-    }   }
-    /* build CTable for Offsets */
-    {   unsigned max = MaxOff;
-        size_t const mostFrequent = HIST_countFast_wksp(count, &max, ofCodeTable, nbSeq, workspace, wkspSize);  /* can't fail */
-        /* We can only use the basic table if max <= DefaultMaxOff, otherwise the offsets are too large */
-        ZSTD_defaultPolicy_e const defaultPolicy = (max <= DefaultMaxOff) ? ZSTD_defaultAllowed : ZSTD_defaultDisallowed;
-        DEBUGLOG(5, "Building OF table");
-        nextEntropy->fse.offcode_repeatMode = prevEntropy->fse.offcode_repeatMode;
-        Offtype = ZSTD_selectEncodingType(&nextEntropy->fse.offcode_repeatMode,
-                                        count, max, mostFrequent, nbSeq,
-                                        OffFSELog, prevEntropy->fse.offcodeCTable,
-                                        OF_defaultNorm, OF_defaultNormLog,
-                                        defaultPolicy, strategy);
-        assert(!(Offtype < set_compressed && nextEntropy->fse.offcode_repeatMode != FSE_repeat_none)); /* We don't copy tables */
-        {   size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_OffsetBits, OffFSELog, (symbolEncodingType_e)Offtype,
-                                                    count, max, ofCodeTable, nbSeq, OF_defaultNorm, OF_defaultNormLog, DefaultMaxOff,
-                                                    prevEntropy->fse.offcodeCTable, sizeof(prevEntropy->fse.offcodeCTable),
-                                                    workspace, wkspSize);
-            FORWARD_IF_ERROR(countSize);
-            if (Offtype == set_compressed)
-                lastNCount = op;
-            op += countSize;
-            assert(op <= oend);
-    }   }
-    /* build CTable for MatchLengths */
-    {   unsigned max = MaxML;
-        size_t const mostFrequent = HIST_countFast_wksp(count, &max, mlCodeTable, nbSeq, workspace, wkspSize);   /* can't fail */
-        DEBUGLOG(5, "Building ML table (remaining space : %i)", (int)(oend-op));
-        nextEntropy->fse.matchlength_repeatMode = prevEntropy->fse.matchlength_repeatMode;
-        MLtype = ZSTD_selectEncodingType(&nextEntropy->fse.matchlength_repeatMode,
-                                        count, max, mostFrequent, nbSeq,
-                                        MLFSELog, prevEntropy->fse.matchlengthCTable,
-                                        ML_defaultNorm, ML_defaultNormLog,
-                                        ZSTD_defaultAllowed, strategy);
-        assert(!(MLtype < set_compressed && nextEntropy->fse.matchlength_repeatMode != FSE_repeat_none)); /* We don't copy tables */
-        {   size_t const countSize = ZSTD_buildCTable(op, oend - op, CTable_MatchLength, MLFSELog, (symbolEncodingType_e)MLtype,
-                                                    count, max, mlCodeTable, nbSeq, ML_defaultNorm, ML_defaultNormLog, MaxML,
-                                                    prevEntropy->fse.matchlengthCTable, sizeof(prevEntropy->fse.matchlengthCTable),
-                                                    workspace, wkspSize);
-            FORWARD_IF_ERROR(countSize);
-            if (MLtype == set_compressed)
-                lastNCount = op;
-            op += countSize;
-            assert(op <= oend);
-    }   }
-
-    *seqHead = (BYTE)((LLtype<<6) + (Offtype<<4) + (MLtype<<2));
-
-    {   size_t const bitstreamSize = ZSTD_encodeSequences(
-                                        op, oend - op,
-                                        CTable_MatchLength, mlCodeTable,
-                                        CTable_OffsetBits, ofCodeTable,
-                                        CTable_LitLength, llCodeTable,
-                                        sequences, nbSeq,
-                                        longOffsets, bmi2);
-        FORWARD_IF_ERROR(bitstreamSize);
-        op += bitstreamSize;
-        assert(op <= oend);
-        /* zstd versions <= 1.3.4 mistakenly report corruption when
-         * FSE_readNCount() receives a buffer < 4 bytes.
-         * Fixed by https://github.com/facebook/zstd/pull/1146.
-         * This can happen when the last set_compressed table present is 2
-         * bytes and the bitstream is only one byte.
-         * In this exceedingly rare case, we will simply emit an uncompressed
-         * block, since it isn't worth optimizing.
-         */
-        if (lastNCount && (op - lastNCount) < 4) {
-            /* NCountSize >= 2 && bitstreamSize > 0 ==> lastCountSize == 3 */
-            assert(op - lastNCount == 3);
-            DEBUGLOG(5, "Avoiding bug in zstd decoder in versions <= 1.3.4 by "
-                        "emitting an uncompressed block.");
-            return 0;
-        }
-    }
-
-    DEBUGLOG(5, "compressed block size : %u", (unsigned)(op - ostart));
-    return op - ostart;
-}
-
-MEM_STATIC size_t
-ZSTD_compressSequences(seqStore_t* seqStorePtr,
-                       const ZSTD_entropyCTables_t* prevEntropy,
-                             ZSTD_entropyCTables_t* nextEntropy,
-                       const ZSTD_CCtx_params* cctxParams,
-                             void* dst, size_t dstCapacity,
-                             size_t srcSize,
-                             void* workspace, size_t wkspSize,
-                             int bmi2)
-{
-    size_t const cSize = ZSTD_compressSequences_internal(
-                            seqStorePtr, prevEntropy, nextEntropy, cctxParams,
-                            dst, dstCapacity,
-                            workspace, wkspSize, bmi2);
-    if (cSize == 0) return 0;
-    /* When srcSize <= dstCapacity, there is enough space to write a raw uncompressed block.
-     * Since we ran out of space, block must be not compressible, so fall back to raw uncompressed block.
-     */
-    if ((cSize == ERROR(dstSize_tooSmall)) & (srcSize <= dstCapacity))
-        return 0;  /* block not compressed */
-    FORWARD_IF_ERROR(cSize);
-
-    /* Check compressibility */
-    {   size_t const maxCSize = srcSize - ZSTD_minGain(srcSize, cctxParams->cParams.strategy);
-        if (cSize >= maxCSize) return 0;  /* block not compressed */
-    }
-
-    return cSize;
-}
-
-/* ZSTD_selectBlockCompressor() :
- * Not static, but internal use only (used by long distance matcher)
- * assumption : strat is a valid strategy */
-ZSTD_blockCompressor ZSTD_selectBlockCompressor(ZSTD_strategy strat, ZSTD_dictMode_e dictMode)
-{
-    static const ZSTD_blockCompressor blockCompressor[3][ZSTD_STRATEGY_MAX+1] = {
-        { ZSTD_compressBlock_fast  /* default for 0 */,
-          ZSTD_compressBlock_fast,
-          ZSTD_compressBlock_doubleFast,
-          ZSTD_compressBlock_greedy,
-          ZSTD_compressBlock_lazy,
-          ZSTD_compressBlock_lazy2,
-          ZSTD_compressBlock_btlazy2,
-          ZSTD_compressBlock_btopt,
-          ZSTD_compressBlock_btultra,
-          ZSTD_compressBlock_btultra2 },
-        { ZSTD_compressBlock_fast_extDict  /* default for 0 */,
-          ZSTD_compressBlock_fast_extDict,
-          ZSTD_compressBlock_doubleFast_extDict,
-          ZSTD_compressBlock_greedy_extDict,
-          ZSTD_compressBlock_lazy_extDict,
-          ZSTD_compressBlock_lazy2_extDict,
-          ZSTD_compressBlock_btlazy2_extDict,
-          ZSTD_compressBlock_btopt_extDict,
-          ZSTD_compressBlock_btultra_extDict,
-          ZSTD_compressBlock_btultra_extDict },
-        { ZSTD_compressBlock_fast_dictMatchState  /* default for 0 */,
-          ZSTD_compressBlock_fast_dictMatchState,
-          ZSTD_compressBlock_doubleFast_dictMatchState,
-          ZSTD_compressBlock_greedy_dictMatchState,
-          ZSTD_compressBlock_lazy_dictMatchState,
-          ZSTD_compressBlock_lazy2_dictMatchState,
-          ZSTD_compressBlock_btlazy2_dictMatchState,
-          ZSTD_compressBlock_btopt_dictMatchState,
-          ZSTD_compressBlock_btultra_dictMatchState,
-          ZSTD_compressBlock_btultra_dictMatchState }
-    };
-    ZSTD_blockCompressor selectedCompressor;
-    ZSTD_STATIC_ASSERT((unsigned)ZSTD_fast == 1);
-
-    assert(ZSTD_cParam_withinBounds(ZSTD_c_strategy, strat));
-    selectedCompressor = blockCompressor[(int)dictMode][(int)strat];
-    assert(selectedCompressor != NULL);
-    return selectedCompressor;
-}
-
-static void ZSTD_storeLastLiterals(seqStore_t* seqStorePtr,
-                                   const BYTE* anchor, size_t lastLLSize)
-{
-    memcpy(seqStorePtr->lit, anchor, lastLLSize);
-    seqStorePtr->lit += lastLLSize;
-}
-
-void ZSTD_resetSeqStore(seqStore_t* ssPtr)
-{
-    ssPtr->lit = ssPtr->litStart;
-    ssPtr->sequences = ssPtr->sequencesStart;
-    ssPtr->longLengthID = 0;
-}
-
-typedef enum { ZSTDbss_compress, ZSTDbss_noCompress } ZSTD_buildSeqStore_e;
-
-static size_t ZSTD_buildSeqStore(ZSTD_CCtx* zc, const void* src, size_t srcSize)
-{
-    ZSTD_matchState_t* const ms = &zc->blockState.matchState;
-    DEBUGLOG(5, "ZSTD_buildSeqStore (srcSize=%zu)", srcSize);
-    assert(srcSize <= ZSTD_BLOCKSIZE_MAX);
-    /* Assert that we have correctly flushed the ctx params into the ms's copy */
-    ZSTD_assertEqualCParams(zc->appliedParams.cParams, ms->cParams);
-    if (srcSize < MIN_CBLOCK_SIZE+ZSTD_blockHeaderSize+1) {
-        ZSTD_ldm_skipSequences(&zc->externSeqStore, srcSize, zc->appliedParams.cParams.minMatch);
-        return ZSTDbss_noCompress; /* don't even attempt compression below a certain srcSize */
-    }
-    ZSTD_resetSeqStore(&(zc->seqStore));
-    /* required for optimal parser to read stats from dictionary */
-    ms->opt.symbolCosts = &zc->blockState.prevCBlock->entropy;
-    /* tell the optimal parser how we expect to compress literals */
-    ms->opt.literalCompressionMode = zc->appliedParams.literalCompressionMode;
-    /* a gap between an attached dict and the current window is not safe,
-     * they must remain adjacent,
-     * and when that stops being the case, the dict must be unset */
-    assert(ms->dictMatchState == NULL || ms->loadedDictEnd == ms->window.dictLimit);
-
-    /* limited update after a very long match */
-    {   const BYTE* const base = ms->window.base;
-        const BYTE* const istart = (const BYTE*)src;
-        const U32 current = (U32)(istart-base);
-        if (sizeof(ptrdiff_t)==8) assert(istart - base < (ptrdiff_t)(U32)(-1));   /* ensure no overflow */
-        if (current > ms->nextToUpdate + 384)
-            ms->nextToUpdate = current - MIN(192, (U32)(current - ms->nextToUpdate - 384));
-    }
-
-    /* select and store sequences */
-    {   ZSTD_dictMode_e const dictMode = ZSTD_matchState_dictMode(ms);
-        size_t lastLLSize;
-        {   int i;
-            for (i = 0; i < ZSTD_REP_NUM; ++i)
-                zc->blockState.nextCBlock->rep[i] = zc->blockState.prevCBlock->rep[i];
-        }
-        if (zc->externSeqStore.pos < zc->externSeqStore.size) {
-            assert(!zc->appliedParams.ldmParams.enableLdm);
-            /* Updates ldmSeqStore.pos */
-            lastLLSize =
-                ZSTD_ldm_blockCompress(&zc->externSeqStore,
-                                       ms, &zc->seqStore,
-                                       zc->blockState.nextCBlock->rep,
-                                       src, srcSize);
-            assert(zc->externSeqStore.pos <= zc->externSeqStore.size);
-        } else if (zc->appliedParams.ldmParams.enableLdm) {
-            rawSeqStore_t ldmSeqStore = {NULL, 0, 0, 0};
-
-            ldmSeqStore.seq = zc->ldmSequences;
-            ldmSeqStore.capacity = zc->maxNbLdmSequences;
-            /* Updates ldmSeqStore.size */
-            FORWARD_IF_ERROR(ZSTD_ldm_generateSequences(&zc->ldmState, &ldmSeqStore,
-                                               &zc->appliedParams.ldmParams,
-                                               src, srcSize));
-            /* Updates ldmSeqStore.pos */
-            lastLLSize =
-                ZSTD_ldm_blockCompress(&ldmSeqStore,
-                                       ms, &zc->seqStore,
-                                       zc->blockState.nextCBlock->rep,
-                                       src, srcSize);
-            assert(ldmSeqStore.pos == ldmSeqStore.size);
-        } else {   /* not long range mode */
-            ZSTD_blockCompressor const blockCompressor = ZSTD_selectBlockCompressor(zc->appliedParams.cParams.strategy, dictMode);
-            lastLLSize = blockCompressor(ms, &zc->seqStore, zc->blockState.nextCBlock->rep, src, srcSize);
-        }
-        {   const BYTE* const lastLiterals = (const BYTE*)src + srcSize - lastLLSize;
-            ZSTD_storeLastLiterals(&zc->seqStore, lastLiterals, lastLLSize);
-    }   }
-    return ZSTDbss_compress;
-}
-
-static size_t ZSTD_compressBlock_internal(ZSTD_CCtx* zc,
-                                        void* dst, size_t dstCapacity,
-                                        const void* src, size_t srcSize)
-{
-    size_t cSize;
-    DEBUGLOG(5, "ZSTD_compressBlock_internal (dstCapacity=%u, dictLimit=%u, nextToUpdate=%u)",
-                (unsigned)dstCapacity, (unsigned)zc->blockState.matchState.window.dictLimit, (unsigned)zc->blockState.matchState.nextToUpdate);
-
-    {   const size_t bss = ZSTD_buildSeqStore(zc, src, srcSize);
-        FORWARD_IF_ERROR(bss);
-        if (bss == ZSTDbss_noCompress) { cSize = 0; goto out; }
-    }
-
-    /* encode sequences and literals */
-    cSize = ZSTD_compressSequences(&zc->seqStore,
-            &zc->blockState.prevCBlock->entropy, &zc->blockState.nextCBlock->entropy,
-            &zc->appliedParams,
-            dst, dstCapacity,
-            srcSize,
-            zc->entropyWorkspace, HUF_WORKSPACE_SIZE /* statically allocated in resetCCtx */,
-            zc->bmi2);
-
-out:
-    if (!ZSTD_isError(cSize) && cSize != 0) {
-        /* confirm repcodes and entropy tables when emitting a compressed block */
-        ZSTD_compressedBlockState_t* const tmp = zc->blockState.prevCBlock;
-        zc->blockState.prevCBlock = zc->blockState.nextCBlock;
-        zc->blockState.nextCBlock = tmp;
-    }
-    /* We check that dictionaries have offset codes available for the first
-     * block. After the first block, the offcode table might not have large
-     * enough codes to represent the offsets in the data.
-     */
-    if (zc->blockState.prevCBlock->entropy.fse.offcode_repeatMode == FSE_repeat_valid)
-        zc->blockState.prevCBlock->entropy.fse.offcode_repeatMode = FSE_repeat_check;
-
-    return cSize;
-}
-
-
-static void ZSTD_overflowCorrectIfNeeded(ZSTD_matchState_t* ms, ZSTD_CCtx_params const* params, void const* ip, void const* iend)
-{
-    if (ZSTD_window_needOverflowCorrection(ms->window, iend)) {
-        U32 const maxDist = (U32)1 << params->cParams.windowLog;
-        U32 const cycleLog = ZSTD_cycleLog(params->cParams.chainLog, params->cParams.strategy);
-        U32 const correction = ZSTD_window_correctOverflow(&ms->window, cycleLog, maxDist, ip);
-        ZSTD_STATIC_ASSERT(ZSTD_CHAINLOG_MAX <= 30);
-        ZSTD_STATIC_ASSERT(ZSTD_WINDOWLOG_MAX_32 <= 30);
-        ZSTD_STATIC_ASSERT(ZSTD_WINDOWLOG_MAX <= 31);
-        ZSTD_reduceIndex(ms, params, correction);
-        if (ms->nextToUpdate < correction) ms->nextToUpdate = 0;
-        else ms->nextToUpdate -= correction;
-        /* invalidate dictionaries on overflow correction */
-        ms->loadedDictEnd = 0;
-        ms->dictMatchState = NULL;
-    }
-}
-
-
-/*! ZSTD_compress_frameChunk() :
-*   Compress a chunk of data into one or multiple blocks.
-*   All blocks will be terminated, all input will be consumed.
-*   Function will issue an error if there is not enough `dstCapacity` to hold the compressed content.
-*   Frame is supposed already started (header already produced)
-*   @return : compressed size, or an error code
-*/
-static size_t ZSTD_compress_frameChunk (ZSTD_CCtx* cctx,
-                                     void* dst, size_t dstCapacity,
-                               const void* src, size_t srcSize,
-                                     U32 lastFrameChunk)
-{
-    size_t blockSize = cctx->blockSize;
-    size_t remaining = srcSize;
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* const ostart = (BYTE*)dst;
-    BYTE* op = ostart;
-    U32 const maxDist = (U32)1 << cctx->appliedParams.cParams.windowLog;
-    assert(cctx->appliedParams.cParams.windowLog <= ZSTD_WINDOWLOG_MAX);
-
-    DEBUGLOG(5, "ZSTD_compress_frameChunk (blockSize=%u)", (unsigned)blockSize);
-    if (cctx->appliedParams.fParams.checksumFlag && srcSize)
-        XXH64_update(&cctx->xxhState, src, srcSize);
-
-    while (remaining) {
-        ZSTD_matchState_t* const ms = &cctx->blockState.matchState;
-        U32 const lastBlock = lastFrameChunk & (blockSize >= remaining);
-
-        RETURN_ERROR_IF(dstCapacity < ZSTD_blockHeaderSize + MIN_CBLOCK_SIZE,
-                        dstSize_tooSmall,
-                        "not enough space to store compressed block");
-        if (remaining < blockSize) blockSize = remaining;
-
-        ZSTD_overflowCorrectIfNeeded(ms, &cctx->appliedParams, ip, ip + blockSize);
-        ZSTD_checkDictValidity(&ms->window, ip + blockSize, maxDist, &ms->loadedDictEnd, &ms->dictMatchState);
-
-        /* Ensure hash/chain table insertion resumes no sooner than lowlimit */
-        if (ms->nextToUpdate < ms->window.lowLimit) ms->nextToUpdate = ms->window.lowLimit;
-
-        {   size_t cSize = ZSTD_compressBlock_internal(cctx,
-                                op+ZSTD_blockHeaderSize, dstCapacity-ZSTD_blockHeaderSize,
-                                ip, blockSize);
-            FORWARD_IF_ERROR(cSize);
-
-            if (cSize == 0) {  /* block is not compressible */
-                cSize = ZSTD_noCompressBlock(op, dstCapacity, ip, blockSize, lastBlock);
-                FORWARD_IF_ERROR(cSize);
-            } else {
-                U32 const cBlockHeader24 = lastBlock + (((U32)bt_compressed)<<1) + (U32)(cSize << 3);
-                MEM_writeLE24(op, cBlockHeader24);
-                cSize += ZSTD_blockHeaderSize;
-            }
-
-            ip += blockSize;
-            assert(remaining >= blockSize);
-            remaining -= blockSize;
-            op += cSize;
-            assert(dstCapacity >= cSize);
-            dstCapacity -= cSize;
-            DEBUGLOG(5, "ZSTD_compress_frameChunk: adding a block of size %u",
-                        (unsigned)cSize);
-    }   }
-
-    if (lastFrameChunk && (op>ostart)) cctx->stage = ZSTDcs_ending;
-    return (size_t)(op-ostart);
-}
-
-
-static size_t ZSTD_writeFrameHeader(void* dst, size_t dstCapacity,
-                                    ZSTD_CCtx_params params, U64 pledgedSrcSize, U32 dictID)
-{   BYTE* const op = (BYTE*)dst;
-    U32   const dictIDSizeCodeLength = (dictID>0) + (dictID>=256) + (dictID>=65536);   /* 0-3 */
-    U32   const dictIDSizeCode = params.fParams.noDictIDFlag ? 0 : dictIDSizeCodeLength;   /* 0-3 */
-    U32   const checksumFlag = params.fParams.checksumFlag>0;
-    U32   const windowSize = (U32)1 << params.cParams.windowLog;
-    U32   const singleSegment = params.fParams.contentSizeFlag && (windowSize >= pledgedSrcSize);
-    BYTE  const windowLogByte = (BYTE)((params.cParams.windowLog - ZSTD_WINDOWLOG_ABSOLUTEMIN) << 3);
-    U32   const fcsCode = params.fParams.contentSizeFlag ?
-                     (pledgedSrcSize>=256) + (pledgedSrcSize>=65536+256) + (pledgedSrcSize>=0xFFFFFFFFU) : 0;  /* 0-3 */
-    BYTE  const frameHeaderDescriptionByte = (BYTE)(dictIDSizeCode + (checksumFlag<<2) + (singleSegment<<5) + (fcsCode<<6) );
-    size_t pos=0;
-
-    assert(!(params.fParams.contentSizeFlag && pledgedSrcSize == ZSTD_CONTENTSIZE_UNKNOWN));
-    RETURN_ERROR_IF(dstCapacity < ZSTD_FRAMEHEADERSIZE_MAX, dstSize_tooSmall);
-    DEBUGLOG(4, "ZSTD_writeFrameHeader : dictIDFlag : %u ; dictID : %u ; dictIDSizeCode : %u",
-                !params.fParams.noDictIDFlag, (unsigned)dictID, (unsigned)dictIDSizeCode);
-
-    if (params.format == ZSTD_f_zstd1) {
-        MEM_writeLE32(dst, ZSTD_MAGICNUMBER);
-        pos = 4;
-    }
-    op[pos++] = frameHeaderDescriptionByte;
-    if (!singleSegment) op[pos++] = windowLogByte;
-    switch(dictIDSizeCode)
-    {
-        default:  assert(0); /* impossible */
-        case 0 : break;
-        case 1 : op[pos] = (BYTE)(dictID); pos++; break;
-        case 2 : MEM_writeLE16(op+pos, (U16)dictID); pos+=2; break;
-        case 3 : MEM_writeLE32(op+pos, dictID); pos+=4; break;
-    }
-    switch(fcsCode)
-    {
-        default:  assert(0); /* impossible */
-        case 0 : if (singleSegment) op[pos++] = (BYTE)(pledgedSrcSize); break;
-        case 1 : MEM_writeLE16(op+pos, (U16)(pledgedSrcSize-256)); pos+=2; break;
-        case 2 : MEM_writeLE32(op+pos, (U32)(pledgedSrcSize)); pos+=4; break;
-        case 3 : MEM_writeLE64(op+pos, (U64)(pledgedSrcSize)); pos+=8; break;
-    }
-    return pos;
-}
-
-/* ZSTD_writeLastEmptyBlock() :
- * output an empty Block with end-of-frame mark to complete a frame
- * @return : size of data written into `dst` (== ZSTD_blockHeaderSize (defined in zstd_internal.h))
- *           or an error code if `dstCapacity` is too small (<ZSTD_blockHeaderSize)
- */
-size_t ZSTD_writeLastEmptyBlock(void* dst, size_t dstCapacity)
-{
-    RETURN_ERROR_IF(dstCapacity < ZSTD_blockHeaderSize, dstSize_tooSmall);
-    {   U32 const cBlockHeader24 = 1 /*lastBlock*/ + (((U32)bt_raw)<<1);  /* 0 size */
-        MEM_writeLE24(dst, cBlockHeader24);
-        return ZSTD_blockHeaderSize;
-    }
-}
-
-size_t ZSTD_referenceExternalSequences(ZSTD_CCtx* cctx, rawSeq* seq, size_t nbSeq)
-{
-    RETURN_ERROR_IF(cctx->stage != ZSTDcs_init, stage_wrong);
-    RETURN_ERROR_IF(cctx->appliedParams.ldmParams.enableLdm,
-                    parameter_unsupported);
-    cctx->externSeqStore.seq = seq;
-    cctx->externSeqStore.size = nbSeq;
-    cctx->externSeqStore.capacity = nbSeq;
-    cctx->externSeqStore.pos = 0;
-    return 0;
-}
-
-
-static size_t ZSTD_compressContinue_internal (ZSTD_CCtx* cctx,
-                              void* dst, size_t dstCapacity,
-                        const void* src, size_t srcSize,
-                               U32 frame, U32 lastFrameChunk)
-{
-    ZSTD_matchState_t* const ms = &cctx->blockState.matchState;
-    size_t fhSize = 0;
-
-    DEBUGLOG(5, "ZSTD_compressContinue_internal, stage: %u, srcSize: %u",
-                cctx->stage, (unsigned)srcSize);
-    RETURN_ERROR_IF(cctx->stage==ZSTDcs_created, stage_wrong,
-                    "missing init (ZSTD_compressBegin)");
-
-    if (frame && (cctx->stage==ZSTDcs_init)) {
-        fhSize = ZSTD_writeFrameHeader(dst, dstCapacity, cctx->appliedParams,
-                                       cctx->pledgedSrcSizePlusOne-1, cctx->dictID);
-        FORWARD_IF_ERROR(fhSize);
-        assert(fhSize <= dstCapacity);
-        dstCapacity -= fhSize;
-        dst = (char*)dst + fhSize;
-        cctx->stage = ZSTDcs_ongoing;
-    }
-
-    if (!srcSize) return fhSize;  /* do not generate an empty block if no input */
-
-    if (!ZSTD_window_update(&ms->window, src, srcSize)) {
-        ms->nextToUpdate = ms->window.dictLimit;
-    }
-    if (cctx->appliedParams.ldmParams.enableLdm) {
-        ZSTD_window_update(&cctx->ldmState.window, src, srcSize);
-    }
-
-    if (!frame) {
-        /* overflow check and correction for block mode */
-        ZSTD_overflowCorrectIfNeeded(ms, &cctx->appliedParams, src, (BYTE const*)src + srcSize);
-    }
-
-    DEBUGLOG(5, "ZSTD_compressContinue_internal (blockSize=%u)", (unsigned)cctx->blockSize);
-    {   size_t const cSize = frame ?
-                             ZSTD_compress_frameChunk (cctx, dst, dstCapacity, src, srcSize, lastFrameChunk) :
-                             ZSTD_compressBlock_internal (cctx, dst, dstCapacity, src, srcSize);
-        FORWARD_IF_ERROR(cSize);
-        cctx->consumedSrcSize += srcSize;
-        cctx->producedCSize += (cSize + fhSize);
-        assert(!(cctx->appliedParams.fParams.contentSizeFlag && cctx->pledgedSrcSizePlusOne == 0));
-        if (cctx->pledgedSrcSizePlusOne != 0) {  /* control src size */
-            ZSTD_STATIC_ASSERT(ZSTD_CONTENTSIZE_UNKNOWN == (unsigned long long)-1);
-            RETURN_ERROR_IF(
-                cctx->consumedSrcSize+1 > cctx->pledgedSrcSizePlusOne,
-                srcSize_wrong,
-                "error : pledgedSrcSize = %u, while realSrcSize >= %u",
-                (unsigned)cctx->pledgedSrcSizePlusOne-1,
-                (unsigned)cctx->consumedSrcSize);
-        }
-        return cSize + fhSize;
-    }
-}
-
-size_t ZSTD_compressContinue (ZSTD_CCtx* cctx,
-                              void* dst, size_t dstCapacity,
-                        const void* src, size_t srcSize)
-{
-    DEBUGLOG(5, "ZSTD_compressContinue (srcSize=%u)", (unsigned)srcSize);
-    return ZSTD_compressContinue_internal(cctx, dst, dstCapacity, src, srcSize, 1 /* frame mode */, 0 /* last chunk */);
-}
-
-
-size_t ZSTD_getBlockSize(const ZSTD_CCtx* cctx)
-{
-    ZSTD_compressionParameters const cParams = cctx->appliedParams.cParams;
-    assert(!ZSTD_checkCParams(cParams));
-    return MIN (ZSTD_BLOCKSIZE_MAX, (U32)1 << cParams.windowLog);
-}
-
-size_t ZSTD_compressBlock(ZSTD_CCtx* cctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    size_t const blockSizeMax = ZSTD_getBlockSize(cctx);
-    RETURN_ERROR_IF(srcSize > blockSizeMax, srcSize_wrong);
-
-    return ZSTD_compressContinue_internal(cctx, dst, dstCapacity, src, srcSize, 0 /* frame mode */, 0 /* last chunk */);
-}
-
-/*! ZSTD_loadDictionaryContent() :
- *  @return : 0, or an error code
- */
-static size_t ZSTD_loadDictionaryContent(ZSTD_matchState_t* ms,
-                                         ZSTD_CCtx_params const* params,
-                                         const void* src, size_t srcSize,
-                                         ZSTD_dictTableLoadMethod_e dtlm)
-{
-    const BYTE* ip = (const BYTE*) src;
-    const BYTE* const iend = ip + srcSize;
-
-    ZSTD_window_update(&ms->window, src, srcSize);
-    ms->loadedDictEnd = params->forceWindow ? 0 : (U32)(iend - ms->window.base);
-
-    /* Assert that we the ms params match the params we're being given */
-    ZSTD_assertEqualCParams(params->cParams, ms->cParams);
-
-    if (srcSize <= HASH_READ_SIZE) return 0;
-
-    while (iend - ip > HASH_READ_SIZE) {
-        size_t const remaining = iend - ip;
-        size_t const chunk = MIN(remaining, ZSTD_CHUNKSIZE_MAX);
-        const BYTE* const ichunk = ip + chunk;
-
-        ZSTD_overflowCorrectIfNeeded(ms, params, ip, ichunk);
-
-        switch(params->cParams.strategy)
-        {
-        case ZSTD_fast:
-            ZSTD_fillHashTable(ms, ichunk, dtlm);
-            break;
-        case ZSTD_dfast:
-            ZSTD_fillDoubleHashTable(ms, ichunk, dtlm);
-            break;
-
-        case ZSTD_greedy:
-        case ZSTD_lazy:
-        case ZSTD_lazy2:
-            if (chunk >= HASH_READ_SIZE)
-                ZSTD_insertAndFindFirstIndex(ms, ichunk-HASH_READ_SIZE);
-            break;
-
-        case ZSTD_btlazy2:   /* we want the dictionary table fully sorted */
-        case ZSTD_btopt:
-        case ZSTD_btultra:
-        case ZSTD_btultra2:
-            if (chunk >= HASH_READ_SIZE)
-                ZSTD_updateTree(ms, ichunk-HASH_READ_SIZE, ichunk);
-            break;
-
-        default:
-            assert(0);  /* not possible : not a valid strategy id */
-        }
-
-        ip = ichunk;
-    }
-
-    ms->nextToUpdate = (U32)(iend - ms->window.base);
-    return 0;
-}
-
-
-/* Dictionaries that assign zero probability to symbols that show up causes problems
-   when FSE encoding.  Refuse dictionaries that assign zero probability to symbols
-   that we may encounter during compression.
-   NOTE: This behavior is not standard and could be improved in the future. */
-static size_t ZSTD_checkDictNCount(short* normalizedCounter, unsigned dictMaxSymbolValue, unsigned maxSymbolValue) {
-    U32 s;
-    RETURN_ERROR_IF(dictMaxSymbolValue < maxSymbolValue, dictionary_corrupted);
-    for (s = 0; s <= maxSymbolValue; ++s) {
-        RETURN_ERROR_IF(normalizedCounter[s] == 0, dictionary_corrupted);
-    }
-    return 0;
-}
-
-
-/* Dictionary format :
- * See :
- * https://github.com/facebook/zstd/blob/master/doc/zstd_compression_format.md#dictionary-format
- */
-/*! ZSTD_loadZstdDictionary() :
- * @return : dictID, or an error code
- *  assumptions : magic number supposed already checked
- *                dictSize supposed > 8
- */
-static size_t ZSTD_loadZstdDictionary(ZSTD_compressedBlockState_t* bs,
-                                      ZSTD_matchState_t* ms,
-                                      ZSTD_CCtx_params const* params,
-                                      const void* dict, size_t dictSize,
-                                      ZSTD_dictTableLoadMethod_e dtlm,
-                                      void* workspace)
-{
-    const BYTE* dictPtr = (const BYTE*)dict;
-    const BYTE* const dictEnd = dictPtr + dictSize;
-    short offcodeNCount[MaxOff+1];
-    unsigned offcodeMaxValue = MaxOff;
-    size_t dictID;
-
-    ZSTD_STATIC_ASSERT(HUF_WORKSPACE_SIZE >= (1<<MAX(MLFSELog,LLFSELog)));
-    assert(dictSize > 8);
-    assert(MEM_readLE32(dictPtr) == ZSTD_MAGIC_DICTIONARY);
-
-    dictPtr += 4;   /* skip magic number */
-    dictID = params->fParams.noDictIDFlag ? 0 :  MEM_readLE32(dictPtr);
-    dictPtr += 4;
-
-    {   unsigned maxSymbolValue = 255;
-        size_t const hufHeaderSize = HUF_readCTable((HUF_CElt*)bs->entropy.huf.CTable, &maxSymbolValue, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(HUF_isError(hufHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(maxSymbolValue < 255, dictionary_corrupted);
-        dictPtr += hufHeaderSize;
-    }
-
-    {   unsigned offcodeLog;
-        size_t const offcodeHeaderSize = FSE_readNCount(offcodeNCount, &offcodeMaxValue, &offcodeLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(offcodeHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(offcodeLog > OffFSELog, dictionary_corrupted);
-        /* Defer checking offcodeMaxValue because we need to know the size of the dictionary content */
-        /* fill all offset symbols to avoid garbage at end of table */
-        RETURN_ERROR_IF(FSE_isError(FSE_buildCTable_wksp(
-                bs->entropy.fse.offcodeCTable,
-                offcodeNCount, MaxOff, offcodeLog,
-                workspace, HUF_WORKSPACE_SIZE)),
-            dictionary_corrupted);
-        dictPtr += offcodeHeaderSize;
-    }
-
-    {   short matchlengthNCount[MaxML+1];
-        unsigned matchlengthMaxValue = MaxML, matchlengthLog;
-        size_t const matchlengthHeaderSize = FSE_readNCount(matchlengthNCount, &matchlengthMaxValue, &matchlengthLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(matchlengthHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(matchlengthLog > MLFSELog, dictionary_corrupted);
-        /* Every match length code must have non-zero probability */
-        FORWARD_IF_ERROR( ZSTD_checkDictNCount(matchlengthNCount, matchlengthMaxValue, MaxML));
-        RETURN_ERROR_IF(FSE_isError(FSE_buildCTable_wksp(
-                bs->entropy.fse.matchlengthCTable,
-                matchlengthNCount, matchlengthMaxValue, matchlengthLog,
-                workspace, HUF_WORKSPACE_SIZE)),
-            dictionary_corrupted);
-        dictPtr += matchlengthHeaderSize;
-    }
-
-    {   short litlengthNCount[MaxLL+1];
-        unsigned litlengthMaxValue = MaxLL, litlengthLog;
-        size_t const litlengthHeaderSize = FSE_readNCount(litlengthNCount, &litlengthMaxValue, &litlengthLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(litlengthHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(litlengthLog > LLFSELog, dictionary_corrupted);
-        /* Every literal length code must have non-zero probability */
-        FORWARD_IF_ERROR( ZSTD_checkDictNCount(litlengthNCount, litlengthMaxValue, MaxLL));
-        RETURN_ERROR_IF(FSE_isError(FSE_buildCTable_wksp(
-                bs->entropy.fse.litlengthCTable,
-                litlengthNCount, litlengthMaxValue, litlengthLog,
-                workspace, HUF_WORKSPACE_SIZE)),
-            dictionary_corrupted);
-        dictPtr += litlengthHeaderSize;
-    }
-
-    RETURN_ERROR_IF(dictPtr+12 > dictEnd, dictionary_corrupted);
-    bs->rep[0] = MEM_readLE32(dictPtr+0);
-    bs->rep[1] = MEM_readLE32(dictPtr+4);
-    bs->rep[2] = MEM_readLE32(dictPtr+8);
-    dictPtr += 12;
-
-    {   size_t const dictContentSize = (size_t)(dictEnd - dictPtr);
-        U32 offcodeMax = MaxOff;
-        if (dictContentSize <= ((U32)-1) - 128 KB) {
-            U32 const maxOffset = (U32)dictContentSize + 128 KB; /* The maximum offset that must be supported */
-            offcodeMax = ZSTD_highbit32(maxOffset); /* Calculate minimum offset code required to represent maxOffset */
-        }
-        /* All offset values <= dictContentSize + 128 KB must be representable */
-        FORWARD_IF_ERROR(ZSTD_checkDictNCount(offcodeNCount, offcodeMaxValue, MIN(offcodeMax, MaxOff)));
-        /* All repCodes must be <= dictContentSize and != 0*/
-        {   U32 u;
-            for (u=0; u<3; u++) {
-                RETURN_ERROR_IF(bs->rep[u] == 0, dictionary_corrupted);
-                RETURN_ERROR_IF(bs->rep[u] > dictContentSize, dictionary_corrupted);
-        }   }
-
-        bs->entropy.huf.repeatMode = HUF_repeat_valid;
-        bs->entropy.fse.offcode_repeatMode = FSE_repeat_valid;
-        bs->entropy.fse.matchlength_repeatMode = FSE_repeat_valid;
-        bs->entropy.fse.litlength_repeatMode = FSE_repeat_valid;
-        FORWARD_IF_ERROR(ZSTD_loadDictionaryContent(ms, params, dictPtr, dictContentSize, dtlm));
-        return dictID;
-    }
-}
-
-/** ZSTD_compress_insertDictionary() :
-*   @return : dictID, or an error code */
-static size_t
-ZSTD_compress_insertDictionary(ZSTD_compressedBlockState_t* bs,
-                               ZSTD_matchState_t* ms,
-                         const ZSTD_CCtx_params* params,
-                         const void* dict, size_t dictSize,
-                               ZSTD_dictContentType_e dictContentType,
-                               ZSTD_dictTableLoadMethod_e dtlm,
-                               void* workspace)
-{
-    DEBUGLOG(4, "ZSTD_compress_insertDictionary (dictSize=%u)", (U32)dictSize);
-    if ((dict==NULL) || (dictSize<=8)) return 0;
-
-    ZSTD_reset_compressedBlockState(bs);
-
-    /* dict restricted modes */
-    if (dictContentType == ZSTD_dct_rawContent)
-        return ZSTD_loadDictionaryContent(ms, params, dict, dictSize, dtlm);
-
-    if (MEM_readLE32(dict) != ZSTD_MAGIC_DICTIONARY) {
-        if (dictContentType == ZSTD_dct_auto) {
-            DEBUGLOG(4, "raw content dictionary detected");
-            return ZSTD_loadDictionaryContent(ms, params, dict, dictSize, dtlm);
-        }
-        RETURN_ERROR_IF(dictContentType == ZSTD_dct_fullDict, dictionary_wrong);
-        assert(0);   /* impossible */
-    }
-
-    /* dict as full zstd dictionary */
-    return ZSTD_loadZstdDictionary(bs, ms, params, dict, dictSize, dtlm, workspace);
-}
-
-/*! ZSTD_compressBegin_internal() :
- * @return : 0, or an error code */
-static size_t ZSTD_compressBegin_internal(ZSTD_CCtx* cctx,
-                                    const void* dict, size_t dictSize,
-                                    ZSTD_dictContentType_e dictContentType,
-                                    ZSTD_dictTableLoadMethod_e dtlm,
-                                    const ZSTD_CDict* cdict,
-                                    ZSTD_CCtx_params params, U64 pledgedSrcSize,
-                                    ZSTD_buffered_policy_e zbuff)
-{
-    DEBUGLOG(4, "ZSTD_compressBegin_internal: wlog=%u", params.cParams.windowLog);
-    /* params are supposed to be fully validated at this point */
-    assert(!ZSTD_isError(ZSTD_checkCParams(params.cParams)));
-    assert(!((dict) && (cdict)));  /* either dict or cdict, not both */
-
-    if (cdict && cdict->dictContentSize>0) {
-        return ZSTD_resetCCtx_usingCDict(cctx, cdict, params, pledgedSrcSize, zbuff);
-    }
-
-    FORWARD_IF_ERROR( ZSTD_resetCCtx_internal(cctx, params, pledgedSrcSize,
-                                     ZSTDcrp_continue, zbuff) );
-    {   size_t const dictID = ZSTD_compress_insertDictionary(
-                cctx->blockState.prevCBlock, &cctx->blockState.matchState,
-                &params, dict, dictSize, dictContentType, dtlm, cctx->entropyWorkspace);
-        FORWARD_IF_ERROR(dictID);
-        assert(dictID <= UINT_MAX);
-        cctx->dictID = (U32)dictID;
-    }
-    return 0;
-}
-
-size_t ZSTD_compressBegin_advanced_internal(ZSTD_CCtx* cctx,
-                                    const void* dict, size_t dictSize,
-                                    ZSTD_dictContentType_e dictContentType,
-                                    ZSTD_dictTableLoadMethod_e dtlm,
-                                    const ZSTD_CDict* cdict,
-                                    ZSTD_CCtx_params params,
-                                    unsigned long long pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_compressBegin_advanced_internal: wlog=%u", params.cParams.windowLog);
-    /* compression parameters verification and optimization */
-    FORWARD_IF_ERROR( ZSTD_checkCParams(params.cParams) );
-    return ZSTD_compressBegin_internal(cctx,
-                                       dict, dictSize, dictContentType, dtlm,
-                                       cdict,
-                                       params, pledgedSrcSize,
-                                       ZSTDb_not_buffered);
-}
-
-/*! ZSTD_compressBegin_advanced() :
-*   @return : 0, or an error code */
-size_t ZSTD_compressBegin_advanced(ZSTD_CCtx* cctx,
-                             const void* dict, size_t dictSize,
-                                   ZSTD_parameters params, unsigned long long pledgedSrcSize)
-{
-    ZSTD_CCtx_params const cctxParams =
-            ZSTD_assignParamsToCCtxParams(cctx->requestedParams, params);
-    return ZSTD_compressBegin_advanced_internal(cctx,
-                                            dict, dictSize, ZSTD_dct_auto, ZSTD_dtlm_fast,
-                                            NULL /*cdict*/,
-                                            cctxParams, pledgedSrcSize);
-}
-
-size_t ZSTD_compressBegin_usingDict(ZSTD_CCtx* cctx, const void* dict, size_t dictSize, int compressionLevel)
-{
-    ZSTD_parameters const params = ZSTD_getParams(compressionLevel, ZSTD_CONTENTSIZE_UNKNOWN, dictSize);
-    ZSTD_CCtx_params const cctxParams =
-            ZSTD_assignParamsToCCtxParams(cctx->requestedParams, params);
-    DEBUGLOG(4, "ZSTD_compressBegin_usingDict (dictSize=%u)", (unsigned)dictSize);
-    return ZSTD_compressBegin_internal(cctx, dict, dictSize, ZSTD_dct_auto, ZSTD_dtlm_fast, NULL,
-                                       cctxParams, ZSTD_CONTENTSIZE_UNKNOWN, ZSTDb_not_buffered);
-}
-
-size_t ZSTD_compressBegin(ZSTD_CCtx* cctx, int compressionLevel)
-{
-    return ZSTD_compressBegin_usingDict(cctx, NULL, 0, compressionLevel);
-}
-
-
-/*! ZSTD_writeEpilogue() :
-*   Ends a frame.
-*   @return : nb of bytes written into dst (or an error code) */
-static size_t ZSTD_writeEpilogue(ZSTD_CCtx* cctx, void* dst, size_t dstCapacity)
-{
-    BYTE* const ostart = (BYTE*)dst;
-    BYTE* op = ostart;
-    size_t fhSize = 0;
-
-    DEBUGLOG(4, "ZSTD_writeEpilogue");
-    RETURN_ERROR_IF(cctx->stage == ZSTDcs_created, stage_wrong, "init missing");
-
-    /* special case : empty frame */
-    if (cctx->stage == ZSTDcs_init) {
-        fhSize = ZSTD_writeFrameHeader(dst, dstCapacity, cctx->appliedParams, 0, 0);
-        FORWARD_IF_ERROR(fhSize);
-        dstCapacity -= fhSize;
-        op += fhSize;
-        cctx->stage = ZSTDcs_ongoing;
-    }
-
-    if (cctx->stage != ZSTDcs_ending) {
-        /* write one last empty block, make it the "last" block */
-        U32 const cBlockHeader24 = 1 /* last block */ + (((U32)bt_raw)<<1) + 0;
-        RETURN_ERROR_IF(dstCapacity<4, dstSize_tooSmall);
-        MEM_writeLE32(op, cBlockHeader24);
-        op += ZSTD_blockHeaderSize;
-        dstCapacity -= ZSTD_blockHeaderSize;
-    }
-
-    if (cctx->appliedParams.fParams.checksumFlag) {
-        U32 const checksum = (U32) XXH64_digest(&cctx->xxhState);
-        RETURN_ERROR_IF(dstCapacity<4, dstSize_tooSmall);
-        DEBUGLOG(4, "ZSTD_writeEpilogue: write checksum : %08X", (unsigned)checksum);
-        MEM_writeLE32(op, checksum);
-        op += 4;
-    }
-
-    cctx->stage = ZSTDcs_created;  /* return to "created but no init" status */
-    return op-ostart;
-}
-
-size_t ZSTD_compressEnd (ZSTD_CCtx* cctx,
-                         void* dst, size_t dstCapacity,
-                   const void* src, size_t srcSize)
-{
-    size_t endResult;
-    size_t const cSize = ZSTD_compressContinue_internal(cctx,
-                                dst, dstCapacity, src, srcSize,
-                                1 /* frame mode */, 1 /* last chunk */);
-    FORWARD_IF_ERROR(cSize);
-    endResult = ZSTD_writeEpilogue(cctx, (char*)dst + cSize, dstCapacity-cSize);
-    FORWARD_IF_ERROR(endResult);
-    assert(!(cctx->appliedParams.fParams.contentSizeFlag && cctx->pledgedSrcSizePlusOne == 0));
-    if (cctx->pledgedSrcSizePlusOne != 0) {  /* control src size */
-        ZSTD_STATIC_ASSERT(ZSTD_CONTENTSIZE_UNKNOWN == (unsigned long long)-1);
-        DEBUGLOG(4, "end of frame : controlling src size");
-        RETURN_ERROR_IF(
-            cctx->pledgedSrcSizePlusOne != cctx->consumedSrcSize+1,
-            srcSize_wrong,
-             "error : pledgedSrcSize = %u, while realSrcSize = %u",
-            (unsigned)cctx->pledgedSrcSizePlusOne-1,
-            (unsigned)cctx->consumedSrcSize);
-    }
-    return cSize + endResult;
-}
-
-
-static size_t ZSTD_compress_internal (ZSTD_CCtx* cctx,
-                                      void* dst, size_t dstCapacity,
-                                const void* src, size_t srcSize,
-                                const void* dict,size_t dictSize,
-                                      ZSTD_parameters params)
-{
-    ZSTD_CCtx_params const cctxParams =
-            ZSTD_assignParamsToCCtxParams(cctx->requestedParams, params);
-    DEBUGLOG(4, "ZSTD_compress_internal");
-    return ZSTD_compress_advanced_internal(cctx,
-                                           dst, dstCapacity,
-                                           src, srcSize,
-                                           dict, dictSize,
-                                           cctxParams);
-}
-
-size_t ZSTD_compress_advanced (ZSTD_CCtx* cctx,
-                               void* dst, size_t dstCapacity,
-                         const void* src, size_t srcSize,
-                         const void* dict,size_t dictSize,
-                               ZSTD_parameters params)
-{
-    DEBUGLOG(4, "ZSTD_compress_advanced");
-    FORWARD_IF_ERROR(ZSTD_checkCParams(params.cParams));
-    return ZSTD_compress_internal(cctx,
-                                  dst, dstCapacity,
-                                  src, srcSize,
-                                  dict, dictSize,
-                                  params);
-}
-
-/* Internal */
-size_t ZSTD_compress_advanced_internal(
-        ZSTD_CCtx* cctx,
-        void* dst, size_t dstCapacity,
-        const void* src, size_t srcSize,
-        const void* dict,size_t dictSize,
-        ZSTD_CCtx_params params)
-{
-    DEBUGLOG(4, "ZSTD_compress_advanced_internal (srcSize:%u)", (unsigned)srcSize);
-    FORWARD_IF_ERROR( ZSTD_compressBegin_internal(cctx,
-                         dict, dictSize, ZSTD_dct_auto, ZSTD_dtlm_fast, NULL,
-                         params, srcSize, ZSTDb_not_buffered) );
-    return ZSTD_compressEnd(cctx, dst, dstCapacity, src, srcSize);
-}
-
-size_t ZSTD_compress_usingDict(ZSTD_CCtx* cctx,
-                               void* dst, size_t dstCapacity,
-                         const void* src, size_t srcSize,
-                         const void* dict, size_t dictSize,
-                               int compressionLevel)
-{
-    ZSTD_parameters const params = ZSTD_getParams(compressionLevel, srcSize + (!srcSize), dict ? dictSize : 0);
-    ZSTD_CCtx_params cctxParams = ZSTD_assignParamsToCCtxParams(cctx->requestedParams, params);
-    assert(params.fParams.contentSizeFlag == 1);
-    return ZSTD_compress_advanced_internal(cctx, dst, dstCapacity, src, srcSize, dict, dictSize, cctxParams);
-}
-
-size_t ZSTD_compressCCtx(ZSTD_CCtx* cctx,
-                         void* dst, size_t dstCapacity,
-                   const void* src, size_t srcSize,
-                         int compressionLevel)
-{
-    DEBUGLOG(4, "ZSTD_compressCCtx (srcSize=%u)", (unsigned)srcSize);
-    assert(cctx != NULL);
-    return ZSTD_compress_usingDict(cctx, dst, dstCapacity, src, srcSize, NULL, 0, compressionLevel);
-}
-
-size_t ZSTD_compress(void* dst, size_t dstCapacity,
-               const void* src, size_t srcSize,
-                     int compressionLevel)
-{
-    size_t result;
-    ZSTD_CCtx ctxBody;
-    ZSTD_initCCtx(&ctxBody, ZSTD_defaultCMem);
-    result = ZSTD_compressCCtx(&ctxBody, dst, dstCapacity, src, srcSize, compressionLevel);
-    ZSTD_freeCCtxContent(&ctxBody);   /* can't free ctxBody itself, as it's on stack; free only heap content */
-    return result;
-}
-
-
-/* =====  Dictionary API  ===== */
-
-/*! ZSTD_estimateCDictSize_advanced() :
- *  Estimate amount of memory that will be needed to create a dictionary with following arguments */
-size_t ZSTD_estimateCDictSize_advanced(
-        size_t dictSize, ZSTD_compressionParameters cParams,
-        ZSTD_dictLoadMethod_e dictLoadMethod)
-{
-    DEBUGLOG(5, "sizeof(ZSTD_CDict) : %u", (unsigned)sizeof(ZSTD_CDict));
-    return sizeof(ZSTD_CDict) + HUF_WORKSPACE_SIZE + ZSTD_sizeof_matchState(&cParams, /* forCCtx */ 0)
-           + (dictLoadMethod == ZSTD_dlm_byRef ? 0 : dictSize);
-}
-
-size_t ZSTD_estimateCDictSize(size_t dictSize, int compressionLevel)
-{
-    ZSTD_compressionParameters const cParams = ZSTD_getCParams(compressionLevel, 0, dictSize);
-    return ZSTD_estimateCDictSize_advanced(dictSize, cParams, ZSTD_dlm_byCopy);
-}
-
-size_t ZSTD_sizeof_CDict(const ZSTD_CDict* cdict)
-{
-    if (cdict==NULL) return 0;   /* support sizeof on NULL */
-    DEBUGLOG(5, "sizeof(*cdict) : %u", (unsigned)sizeof(*cdict));
-    return cdict->workspaceSize + (cdict->dictBuffer ? cdict->dictContentSize : 0) + sizeof(*cdict);
-}
-
-static size_t ZSTD_initCDict_internal(
-                    ZSTD_CDict* cdict,
-              const void* dictBuffer, size_t dictSize,
-                    ZSTD_dictLoadMethod_e dictLoadMethod,
-                    ZSTD_dictContentType_e dictContentType,
-                    ZSTD_compressionParameters cParams)
-{
-    DEBUGLOG(3, "ZSTD_initCDict_internal (dictContentType:%u)", (unsigned)dictContentType);
-    assert(!ZSTD_checkCParams(cParams));
-    cdict->matchState.cParams = cParams;
-    if ((dictLoadMethod == ZSTD_dlm_byRef) || (!dictBuffer) || (!dictSize)) {
-        cdict->dictBuffer = NULL;
-        cdict->dictContent = dictBuffer;
-    } else {
-        void* const internalBuffer = ZSTD_malloc(dictSize, cdict->customMem);
-        cdict->dictBuffer = internalBuffer;
-        cdict->dictContent = internalBuffer;
-        RETURN_ERROR_IF(!internalBuffer, memory_allocation);
-        memcpy(internalBuffer, dictBuffer, dictSize);
-    }
-    cdict->dictContentSize = dictSize;
-
-    /* Reset the state to no dictionary */
-    ZSTD_reset_compressedBlockState(&cdict->cBlockState);
-    {   void* const end = ZSTD_reset_matchState(&cdict->matchState,
-                            (U32*)cdict->workspace + HUF_WORKSPACE_SIZE_U32,
-                            &cParams,
-                             ZSTDcrp_continue, ZSTD_resetTarget_CDict);
-        assert(end == (char*)cdict->workspace + cdict->workspaceSize);
-        (void)end;
-    }
-    /* (Maybe) load the dictionary
-     * Skips loading the dictionary if it is <= 8 bytes.
-     */
-    {   ZSTD_CCtx_params params;
-        memset(&params, 0, sizeof(params));
-        params.compressionLevel = ZSTD_CLEVEL_DEFAULT;
-        params.fParams.contentSizeFlag = 1;
-        params.cParams = cParams;
-        {   size_t const dictID = ZSTD_compress_insertDictionary(
-                    &cdict->cBlockState, &cdict->matchState, &params,
-                    cdict->dictContent, cdict->dictContentSize,
-                    dictContentType, ZSTD_dtlm_full, cdict->workspace);
-            FORWARD_IF_ERROR(dictID);
-            assert(dictID <= (size_t)(U32)-1);
-            cdict->dictID = (U32)dictID;
-        }
-    }
-
-    return 0;
-}
-
-ZSTD_CDict* ZSTD_createCDict_advanced(const void* dictBuffer, size_t dictSize,
-                                      ZSTD_dictLoadMethod_e dictLoadMethod,
-                                      ZSTD_dictContentType_e dictContentType,
-                                      ZSTD_compressionParameters cParams, ZSTD_customMem customMem)
-{
-    DEBUGLOG(3, "ZSTD_createCDict_advanced, mode %u", (unsigned)dictContentType);
-    if (!customMem.customAlloc ^ !customMem.customFree) return NULL;
-
-    {   ZSTD_CDict* const cdict = (ZSTD_CDict*)ZSTD_malloc(sizeof(ZSTD_CDict), customMem);
-        size_t const workspaceSize = HUF_WORKSPACE_SIZE + ZSTD_sizeof_matchState(&cParams, /* forCCtx */ 0);
-        void* const workspace = ZSTD_malloc(workspaceSize, customMem);
-
-        if (!cdict || !workspace) {
-            ZSTD_free(cdict, customMem);
-            ZSTD_free(workspace, customMem);
-            return NULL;
-        }
-        cdict->customMem = customMem;
-        cdict->workspace = workspace;
-        cdict->workspaceSize = workspaceSize;
-        if (ZSTD_isError( ZSTD_initCDict_internal(cdict,
-                                        dictBuffer, dictSize,
-                                        dictLoadMethod, dictContentType,
-                                        cParams) )) {
-            ZSTD_freeCDict(cdict);
-            return NULL;
-        }
-
-        return cdict;
-    }
-}
-
-ZSTD_CDict* ZSTD_createCDict(const void* dict, size_t dictSize, int compressionLevel)
-{
-    ZSTD_compressionParameters cParams = ZSTD_getCParams(compressionLevel, 0, dictSize);
-    return ZSTD_createCDict_advanced(dict, dictSize,
-                                     ZSTD_dlm_byCopy, ZSTD_dct_auto,
-                                     cParams, ZSTD_defaultCMem);
-}
-
-ZSTD_CDict* ZSTD_createCDict_byReference(const void* dict, size_t dictSize, int compressionLevel)
-{
-    ZSTD_compressionParameters cParams = ZSTD_getCParams(compressionLevel, 0, dictSize);
-    return ZSTD_createCDict_advanced(dict, dictSize,
-                                     ZSTD_dlm_byRef, ZSTD_dct_auto,
-                                     cParams, ZSTD_defaultCMem);
-}
-
-size_t ZSTD_freeCDict(ZSTD_CDict* cdict)
-{
-    if (cdict==NULL) return 0;   /* support free on NULL */
-    {   ZSTD_customMem const cMem = cdict->customMem;
-        ZSTD_free(cdict->workspace, cMem);
-        ZSTD_free(cdict->dictBuffer, cMem);
-        ZSTD_free(cdict, cMem);
-        return 0;
-    }
-}
-
-/*! ZSTD_initStaticCDict_advanced() :
- *  Generate a digested dictionary in provided memory area.
- *  workspace: The memory area to emplace the dictionary into.
- *             Provided pointer must 8-bytes aligned.
- *             It must outlive dictionary usage.
- *  workspaceSize: Use ZSTD_estimateCDictSize()
- *                 to determine how large workspace must be.
- *  cParams : use ZSTD_getCParams() to transform a compression level
- *            into its relevants cParams.
- * @return : pointer to ZSTD_CDict*, or NULL if error (size too small)
- *  Note : there is no corresponding "free" function.
- *         Since workspace was allocated externally, it must be freed externally.
- */
-const ZSTD_CDict* ZSTD_initStaticCDict(
-                                 void* workspace, size_t workspaceSize,
-                           const void* dict, size_t dictSize,
-                                 ZSTD_dictLoadMethod_e dictLoadMethod,
-                                 ZSTD_dictContentType_e dictContentType,
-                                 ZSTD_compressionParameters cParams)
-{
-    size_t const matchStateSize = ZSTD_sizeof_matchState(&cParams, /* forCCtx */ 0);
-    size_t const neededSize = sizeof(ZSTD_CDict) + (dictLoadMethod == ZSTD_dlm_byRef ? 0 : dictSize)
-                            + HUF_WORKSPACE_SIZE + matchStateSize;
-    ZSTD_CDict* const cdict = (ZSTD_CDict*) workspace;
-    void* ptr;
-    if ((size_t)workspace & 7) return NULL;  /* 8-aligned */
-    DEBUGLOG(4, "(workspaceSize < neededSize) : (%u < %u) => %u",
-        (unsigned)workspaceSize, (unsigned)neededSize, (unsigned)(workspaceSize < neededSize));
-    if (workspaceSize < neededSize) return NULL;
-
-    if (dictLoadMethod == ZSTD_dlm_byCopy) {
-        memcpy(cdict+1, dict, dictSize);
-        dict = cdict+1;
-        ptr = (char*)workspace + sizeof(ZSTD_CDict) + dictSize;
-    } else {
-        ptr = cdict+1;
-    }
-    cdict->workspace = ptr;
-    cdict->workspaceSize = HUF_WORKSPACE_SIZE + matchStateSize;
-
-    if (ZSTD_isError( ZSTD_initCDict_internal(cdict,
-                                              dict, dictSize,
-                                              ZSTD_dlm_byRef, dictContentType,
-                                              cParams) ))
-        return NULL;
-
-    return cdict;
-}
-
-ZSTD_compressionParameters ZSTD_getCParamsFromCDict(const ZSTD_CDict* cdict)
-{
-    assert(cdict != NULL);
-    return cdict->matchState.cParams;
-}
-
-/* ZSTD_compressBegin_usingCDict_advanced() :
- * cdict must be != NULL */
-size_t ZSTD_compressBegin_usingCDict_advanced(
-    ZSTD_CCtx* const cctx, const ZSTD_CDict* const cdict,
-    ZSTD_frameParameters const fParams, unsigned long long const pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_compressBegin_usingCDict_advanced");
-    RETURN_ERROR_IF(cdict==NULL, dictionary_wrong);
-    {   ZSTD_CCtx_params params = cctx->requestedParams;
-        params.cParams = ZSTD_getCParamsFromCDict(cdict);
-        /* Increase window log to fit the entire dictionary and source if the
-         * source size is known. Limit the increase to 19, which is the
-         * window log for compression level 1 with the largest source size.
-         */
-        if (pledgedSrcSize != ZSTD_CONTENTSIZE_UNKNOWN) {
-            U32 const limitedSrcSize = (U32)MIN(pledgedSrcSize, 1U << 19);
-            U32 const limitedSrcLog = limitedSrcSize > 1 ? ZSTD_highbit32(limitedSrcSize - 1) + 1 : 1;
-            params.cParams.windowLog = MAX(params.cParams.windowLog, limitedSrcLog);
-        }
-        params.fParams = fParams;
-        return ZSTD_compressBegin_internal(cctx,
-                                           NULL, 0, ZSTD_dct_auto, ZSTD_dtlm_fast,
-                                           cdict,
-                                           params, pledgedSrcSize,
-                                           ZSTDb_not_buffered);
-    }
-}
-
-/* ZSTD_compressBegin_usingCDict() :
- * pledgedSrcSize=0 means "unknown"
- * if pledgedSrcSize>0, it will enable contentSizeFlag */
-size_t ZSTD_compressBegin_usingCDict(ZSTD_CCtx* cctx, const ZSTD_CDict* cdict)
-{
-    ZSTD_frameParameters const fParams = { 0 /*content*/, 0 /*checksum*/, 0 /*noDictID*/ };
-    DEBUGLOG(4, "ZSTD_compressBegin_usingCDict : dictIDFlag == %u", !fParams.noDictIDFlag);
-    return ZSTD_compressBegin_usingCDict_advanced(cctx, cdict, fParams, ZSTD_CONTENTSIZE_UNKNOWN);
-}
-
-size_t ZSTD_compress_usingCDict_advanced(ZSTD_CCtx* cctx,
-                                void* dst, size_t dstCapacity,
-                                const void* src, size_t srcSize,
-                                const ZSTD_CDict* cdict, ZSTD_frameParameters fParams)
-{
-    FORWARD_IF_ERROR(ZSTD_compressBegin_usingCDict_advanced(cctx, cdict, fParams, srcSize));   /* will check if cdict != NULL */
-    return ZSTD_compressEnd(cctx, dst, dstCapacity, src, srcSize);
-}
-
-/*! ZSTD_compress_usingCDict() :
- *  Compression using a digested Dictionary.
- *  Faster startup than ZSTD_compress_usingDict(), recommended when same dictionary is used multiple times.
- *  Note that compression parameters are decided at CDict creation time
- *  while frame parameters are hardcoded */
-size_t ZSTD_compress_usingCDict(ZSTD_CCtx* cctx,
-                                void* dst, size_t dstCapacity,
-                                const void* src, size_t srcSize,
-                                const ZSTD_CDict* cdict)
-{
-    ZSTD_frameParameters const fParams = { 1 /*content*/, 0 /*checksum*/, 0 /*noDictID*/ };
-    return ZSTD_compress_usingCDict_advanced(cctx, dst, dstCapacity, src, srcSize, cdict, fParams);
-}
-
-
-
-/* ******************************************************************
-*  Streaming
-********************************************************************/
-
-ZSTD_CStream* ZSTD_createCStream(void)
-{
-    DEBUGLOG(3, "ZSTD_createCStream");
-    return ZSTD_createCStream_advanced(ZSTD_defaultCMem);
-}
-
-ZSTD_CStream* ZSTD_initStaticCStream(void *workspace, size_t workspaceSize)
-{
-    return ZSTD_initStaticCCtx(workspace, workspaceSize);
-}
-
-ZSTD_CStream* ZSTD_createCStream_advanced(ZSTD_customMem customMem)
-{   /* CStream and CCtx are now same object */
-    return ZSTD_createCCtx_advanced(customMem);
-}
-
-size_t ZSTD_freeCStream(ZSTD_CStream* zcs)
-{
-    return ZSTD_freeCCtx(zcs);   /* same object */
-}
-
-
-
-/*======   Initialization   ======*/
-
-size_t ZSTD_CStreamInSize(void)  { return ZSTD_BLOCKSIZE_MAX; }
-
-size_t ZSTD_CStreamOutSize(void)
-{
-    return ZSTD_compressBound(ZSTD_BLOCKSIZE_MAX) + ZSTD_blockHeaderSize + 4 /* 32-bits hash */ ;
-}
-
-static size_t ZSTD_resetCStream_internal(ZSTD_CStream* cctx,
-                    const void* const dict, size_t const dictSize, ZSTD_dictContentType_e const dictContentType,
-                    const ZSTD_CDict* const cdict,
-                    ZSTD_CCtx_params params, unsigned long long const pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_resetCStream_internal");
-    /* Finalize the compression parameters */
-    params.cParams = ZSTD_getCParamsFromCCtxParams(&params, pledgedSrcSize, dictSize);
-    /* params are supposed to be fully validated at this point */
-    assert(!ZSTD_isError(ZSTD_checkCParams(params.cParams)));
-    assert(!((dict) && (cdict)));  /* either dict or cdict, not both */
-
-    FORWARD_IF_ERROR( ZSTD_compressBegin_internal(cctx,
-                                         dict, dictSize, dictContentType, ZSTD_dtlm_fast,
-                                         cdict,
-                                         params, pledgedSrcSize,
-                                         ZSTDb_buffered) );
-
-    cctx->inToCompress = 0;
-    cctx->inBuffPos = 0;
-    cctx->inBuffTarget = cctx->blockSize
-                      + (cctx->blockSize == pledgedSrcSize);   /* for small input: avoid automatic flush on reaching end of block, since it would require to add a 3-bytes null block to end frame */
-    cctx->outBuffContentSize = cctx->outBuffFlushedSize = 0;
-    cctx->streamStage = zcss_load;
-    cctx->frameEnded = 0;
-    return 0;   /* ready to go */
-}
-
-/* ZSTD_resetCStream():
- * pledgedSrcSize == 0 means "unknown" */
-size_t ZSTD_resetCStream(ZSTD_CStream* zcs, unsigned long long pss)
-{
-    /* temporary : 0 interpreted as "unknown" during transition period.
-     * Users willing to specify "unknown" **must** use ZSTD_CONTENTSIZE_UNKNOWN.
-     * 0 will be interpreted as "empty" in the future.
-     */
-    U64 const pledgedSrcSize = (pss==0) ? ZSTD_CONTENTSIZE_UNKNOWN : pss;
-    DEBUGLOG(4, "ZSTD_resetCStream: pledgedSrcSize = %u", (unsigned)pledgedSrcSize);
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize) );
-    return 0;
-}
-
-/*! ZSTD_initCStream_internal() :
- *  Note : for lib/compress only. Used by zstdmt_compress.c.
- *  Assumption 1 : params are valid
- *  Assumption 2 : either dict, or cdict, is defined, not both */
-size_t ZSTD_initCStream_internal(ZSTD_CStream* zcs,
-                    const void* dict, size_t dictSize, const ZSTD_CDict* cdict,
-                    ZSTD_CCtx_params params, unsigned long long pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_initCStream_internal");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize) );
-    assert(!ZSTD_isError(ZSTD_checkCParams(params.cParams)));
-    zcs->requestedParams = params;
-    assert(!((dict) && (cdict)));  /* either dict or cdict, not both */
-    if (dict) {
-        FORWARD_IF_ERROR( ZSTD_CCtx_loadDictionary(zcs, dict, dictSize) );
-    } else {
-        /* Dictionary is cleared if !cdict */
-        FORWARD_IF_ERROR( ZSTD_CCtx_refCDict(zcs, cdict) );
-    }
-    return 0;
-}
-
-/* ZSTD_initCStream_usingCDict_advanced() :
- * same as ZSTD_initCStream_usingCDict(), with control over frame parameters */
-size_t ZSTD_initCStream_usingCDict_advanced(ZSTD_CStream* zcs,
-                                            const ZSTD_CDict* cdict,
-                                            ZSTD_frameParameters fParams,
-                                            unsigned long long pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTD_initCStream_usingCDict_advanced");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize) );
-    zcs->requestedParams.fParams = fParams;
-    FORWARD_IF_ERROR( ZSTD_CCtx_refCDict(zcs, cdict) );
-    return 0;
-}
-
-/* note : cdict must outlive compression session */
-size_t ZSTD_initCStream_usingCDict(ZSTD_CStream* zcs, const ZSTD_CDict* cdict)
-{
-    DEBUGLOG(4, "ZSTD_initCStream_usingCDict");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_refCDict(zcs, cdict) );
-    return 0;
-}
-
-
-/* ZSTD_initCStream_advanced() :
- * pledgedSrcSize must be exact.
- * if srcSize is not known at init time, use value ZSTD_CONTENTSIZE_UNKNOWN.
- * dict is loaded with default parameters ZSTD_dm_auto and ZSTD_dlm_byCopy. */
-size_t ZSTD_initCStream_advanced(ZSTD_CStream* zcs,
-                                 const void* dict, size_t dictSize,
-                                 ZSTD_parameters params, unsigned long long pss)
-{
-    /* for compatibility with older programs relying on this behavior.
-     * Users should now specify ZSTD_CONTENTSIZE_UNKNOWN.
-     * This line will be removed in the future.
-     */
-    U64 const pledgedSrcSize = (pss==0 && params.fParams.contentSizeFlag==0) ? ZSTD_CONTENTSIZE_UNKNOWN : pss;
-    DEBUGLOG(4, "ZSTD_initCStream_advanced");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize) );
-    FORWARD_IF_ERROR( ZSTD_checkCParams(params.cParams) );
-    zcs->requestedParams = ZSTD_assignParamsToCCtxParams(zcs->requestedParams, params);
-    FORWARD_IF_ERROR( ZSTD_CCtx_loadDictionary(zcs, dict, dictSize) );
-    return 0;
-}
-
-size_t ZSTD_initCStream_usingDict(ZSTD_CStream* zcs, const void* dict, size_t dictSize, int compressionLevel)
-{
-    DEBUGLOG(4, "ZSTD_initCStream_usingDict");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_loadDictionary(zcs, dict, dictSize) );
-    return 0;
-}
-
-size_t ZSTD_initCStream_srcSize(ZSTD_CStream* zcs, int compressionLevel, unsigned long long pss)
-{
-    /* temporary : 0 interpreted as "unknown" during transition period.
-     * Users willing to specify "unknown" **must** use ZSTD_CONTENTSIZE_UNKNOWN.
-     * 0 will be interpreted as "empty" in the future.
-     */
-    U64 const pledgedSrcSize = (pss==0) ? ZSTD_CONTENTSIZE_UNKNOWN : pss;
-    DEBUGLOG(4, "ZSTD_initCStream_srcSize");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_refCDict(zcs, NULL) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setPledgedSrcSize(zcs, pledgedSrcSize) );
-    return 0;
-}
-
-size_t ZSTD_initCStream(ZSTD_CStream* zcs, int compressionLevel)
-{
-    DEBUGLOG(4, "ZSTD_initCStream");
-    FORWARD_IF_ERROR( ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_refCDict(zcs, NULL) );
-    FORWARD_IF_ERROR( ZSTD_CCtx_setParameter(zcs, ZSTD_c_compressionLevel, compressionLevel) );
-    return 0;
-}
-
-/*======   Compression   ======*/
-
-static size_t ZSTD_nextInputSizeHint(const ZSTD_CCtx* cctx)
-{
-    size_t hintInSize = cctx->inBuffTarget - cctx->inBuffPos;
-    if (hintInSize==0) hintInSize = cctx->blockSize;
-    return hintInSize;
-}
-
-static size_t ZSTD_limitCopy(void* dst, size_t dstCapacity,
-                       const void* src, size_t srcSize)
-{
-    size_t const length = MIN(dstCapacity, srcSize);
-    if (length) memcpy(dst, src, length);
-    return length;
-}
-
-/** ZSTD_compressStream_generic():
- *  internal function for all *compressStream*() variants
- *  non-static, because can be called from zstdmt_compress.c
- * @return : hint size for next input */
-static size_t ZSTD_compressStream_generic(ZSTD_CStream* zcs,
-                                          ZSTD_outBuffer* output,
-                                          ZSTD_inBuffer* input,
-                                          ZSTD_EndDirective const flushMode)
-{
-    const char* const istart = (const char*)input->src;
-    const char* const iend = istart + input->size;
-    const char* ip = istart + input->pos;
-    char* const ostart = (char*)output->dst;
-    char* const oend = ostart + output->size;
-    char* op = ostart + output->pos;
-    U32 someMoreWork = 1;
-
-    /* check expectations */
-    DEBUGLOG(5, "ZSTD_compressStream_generic, flush=%u", (unsigned)flushMode);
-    assert(zcs->inBuff != NULL);
-    assert(zcs->inBuffSize > 0);
-    assert(zcs->outBuff !=  NULL);
-    assert(zcs->outBuffSize > 0);
-    assert(output->pos <= output->size);
-    assert(input->pos <= input->size);
-
-    while (someMoreWork) {
-        switch(zcs->streamStage)
-        {
-        case zcss_init:
-            RETURN_ERROR(init_missing, "call ZSTD_initCStream() first!");
-
-        case zcss_load:
-            if ( (flushMode == ZSTD_e_end)
-              && ((size_t)(oend-op) >= ZSTD_compressBound(iend-ip))  /* enough dstCapacity */
-              && (zcs->inBuffPos == 0) ) {
-                /* shortcut to compression pass directly into output buffer */
-                size_t const cSize = ZSTD_compressEnd(zcs,
-                                                op, oend-op, ip, iend-ip);
-                DEBUGLOG(4, "ZSTD_compressEnd : cSize=%u", (unsigned)cSize);
-                FORWARD_IF_ERROR(cSize);
-                ip = iend;
-                op += cSize;
-                zcs->frameEnded = 1;
-                ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
-                someMoreWork = 0; break;
-            }
-            /* complete loading into inBuffer */
-            {   size_t const toLoad = zcs->inBuffTarget - zcs->inBuffPos;
-                size_t const loaded = ZSTD_limitCopy(
-                                        zcs->inBuff + zcs->inBuffPos, toLoad,
-                                        ip, iend-ip);
-                zcs->inBuffPos += loaded;
-                ip += loaded;
-                if ( (flushMode == ZSTD_e_continue)
-                  && (zcs->inBuffPos < zcs->inBuffTarget) ) {
-                    /* not enough input to fill full block : stop here */
-                    someMoreWork = 0; break;
-                }
-                if ( (flushMode == ZSTD_e_flush)
-                  && (zcs->inBuffPos == zcs->inToCompress) ) {
-                    /* empty */
-                    someMoreWork = 0; break;
-                }
-            }
-            /* compress current block (note : this stage cannot be stopped in the middle) */
-            DEBUGLOG(5, "stream compression stage (flushMode==%u)", flushMode);
-            {   void* cDst;
-                size_t cSize;
-                size_t const iSize = zcs->inBuffPos - zcs->inToCompress;
-                size_t oSize = oend-op;
-                unsigned const lastBlock = (flushMode == ZSTD_e_end) && (ip==iend);
-                if (oSize >= ZSTD_compressBound(iSize))
-                    cDst = op;   /* compress into output buffer, to skip flush stage */
-                else
-                    cDst = zcs->outBuff, oSize = zcs->outBuffSize;
-                cSize = lastBlock ?
-                        ZSTD_compressEnd(zcs, cDst, oSize,
-                                    zcs->inBuff + zcs->inToCompress, iSize) :
-                        ZSTD_compressContinue(zcs, cDst, oSize,
-                                    zcs->inBuff + zcs->inToCompress, iSize);
-                FORWARD_IF_ERROR(cSize);
-                zcs->frameEnded = lastBlock;
-                /* prepare next block */
-                zcs->inBuffTarget = zcs->inBuffPos + zcs->blockSize;
-                if (zcs->inBuffTarget > zcs->inBuffSize)
-                    zcs->inBuffPos = 0, zcs->inBuffTarget = zcs->blockSize;
-                DEBUGLOG(5, "inBuffTarget:%u / inBuffSize:%u",
-                         (unsigned)zcs->inBuffTarget, (unsigned)zcs->inBuffSize);
-                if (!lastBlock)
-                    assert(zcs->inBuffTarget <= zcs->inBuffSize);
-                zcs->inToCompress = zcs->inBuffPos;
-                if (cDst == op) {  /* no need to flush */
-                    op += cSize;
-                    if (zcs->frameEnded) {
-                        DEBUGLOG(5, "Frame completed directly in outBuffer");
-                        someMoreWork = 0;
-                        ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
-                    }
-                    break;
-                }
-                zcs->outBuffContentSize = cSize;
-                zcs->outBuffFlushedSize = 0;
-                zcs->streamStage = zcss_flush; /* pass-through to flush stage */
-            }
-	    /* fall-through */
-        case zcss_flush:
-            DEBUGLOG(5, "flush stage");
-            {   size_t const toFlush = zcs->outBuffContentSize - zcs->outBuffFlushedSize;
-                size_t const flushed = ZSTD_limitCopy(op, (size_t)(oend-op),
-                            zcs->outBuff + zcs->outBuffFlushedSize, toFlush);
-                DEBUGLOG(5, "toFlush: %u into %u ==> flushed: %u",
-                            (unsigned)toFlush, (unsigned)(oend-op), (unsigned)flushed);
-                op += flushed;
-                zcs->outBuffFlushedSize += flushed;
-                if (toFlush!=flushed) {
-                    /* flush not fully completed, presumably because dst is too small */
-                    assert(op==oend);
-                    someMoreWork = 0;
-                    break;
-                }
-                zcs->outBuffContentSize = zcs->outBuffFlushedSize = 0;
-                if (zcs->frameEnded) {
-                    DEBUGLOG(5, "Frame completed on flush");
-                    someMoreWork = 0;
-                    ZSTD_CCtx_reset(zcs, ZSTD_reset_session_only);
-                    break;
-                }
-                zcs->streamStage = zcss_load;
-                break;
-            }
-
-        default: /* impossible */
-            assert(0);
-        }
-    }
-
-    input->pos = ip - istart;
-    output->pos = op - ostart;
-    if (zcs->frameEnded) return 0;
-    return ZSTD_nextInputSizeHint(zcs);
-}
-
-static size_t ZSTD_nextInputSizeHint_MTorST(const ZSTD_CCtx* cctx)
-{
-#ifdef ZSTD_MULTITHREAD
-    if (cctx->appliedParams.nbWorkers >= 1) {
-        assert(cctx->mtctx != NULL);
-        return ZSTDMT_nextInputSizeHint(cctx->mtctx);
-    }
-#endif
-    return ZSTD_nextInputSizeHint(cctx);
-
-}
-
-size_t ZSTD_compressStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output, ZSTD_inBuffer* input)
-{
-    FORWARD_IF_ERROR( ZSTD_compressStream2(zcs, output, input, ZSTD_e_continue) );
-    return ZSTD_nextInputSizeHint_MTorST(zcs);
-}
-
-
-size_t ZSTD_compressStream2( ZSTD_CCtx* cctx,
-                             ZSTD_outBuffer* output,
-                             ZSTD_inBuffer* input,
-                             ZSTD_EndDirective endOp)
-{
-    DEBUGLOG(5, "ZSTD_compressStream2, endOp=%u ", (unsigned)endOp);
-    /* check conditions */
-    RETURN_ERROR_IF(output->pos > output->size, GENERIC);
-    RETURN_ERROR_IF(input->pos  > input->size, GENERIC);
-    assert(cctx!=NULL);
-
-    /* transparent initialization stage */
-    if (cctx->streamStage == zcss_init) {
-        ZSTD_CCtx_params params = cctx->requestedParams;
-        ZSTD_prefixDict const prefixDict = cctx->prefixDict;
-        FORWARD_IF_ERROR( ZSTD_initLocalDict(cctx) ); /* Init the local dict if present. */
-        memset(&cctx->prefixDict, 0, sizeof(cctx->prefixDict));   /* single usage */
-        assert(prefixDict.dict==NULL || cctx->cdict==NULL);    /* only one can be set */
-        DEBUGLOG(4, "ZSTD_compressStream2 : transparent init stage");
-        if (endOp == ZSTD_e_end) cctx->pledgedSrcSizePlusOne = input->size + 1;  /* auto-fix pledgedSrcSize */
-        params.cParams = ZSTD_getCParamsFromCCtxParams(
-                &cctx->requestedParams, cctx->pledgedSrcSizePlusOne-1, 0 /*dictSize*/);
-
-
-#ifdef ZSTD_MULTITHREAD
-        if ((cctx->pledgedSrcSizePlusOne-1) <= ZSTDMT_JOBSIZE_MIN) {
-            params.nbWorkers = 0; /* do not invoke multi-threading when src size is too small */
-        }
-        if (params.nbWorkers > 0) {
-            /* mt context creation */
-            if (cctx->mtctx == NULL) {
-                DEBUGLOG(4, "ZSTD_compressStream2: creating new mtctx for nbWorkers=%u",
-                            params.nbWorkers);
-                cctx->mtctx = ZSTDMT_createCCtx_advanced(params.nbWorkers, cctx->customMem);
-                RETURN_ERROR_IF(cctx->mtctx == NULL, memory_allocation);
-            }
-            /* mt compression */
-            DEBUGLOG(4, "call ZSTDMT_initCStream_internal as nbWorkers=%u", params.nbWorkers);
-            FORWARD_IF_ERROR( ZSTDMT_initCStream_internal(
-                        cctx->mtctx,
-                        prefixDict.dict, prefixDict.dictSize, ZSTD_dct_rawContent,
-                        cctx->cdict, params, cctx->pledgedSrcSizePlusOne-1) );
-            cctx->streamStage = zcss_load;
-            cctx->appliedParams.nbWorkers = params.nbWorkers;
-        } else
-#endif
-        {   FORWARD_IF_ERROR( ZSTD_resetCStream_internal(cctx,
-                            prefixDict.dict, prefixDict.dictSize, prefixDict.dictContentType,
-                            cctx->cdict,
-                            params, cctx->pledgedSrcSizePlusOne-1) );
-            assert(cctx->streamStage == zcss_load);
-            assert(cctx->appliedParams.nbWorkers == 0);
-    }   }
-    /* end of transparent initialization stage */
-
-    /* compression stage */
-#ifdef ZSTD_MULTITHREAD
-    if (cctx->appliedParams.nbWorkers > 0) {
-        int const forceMaxProgress = (endOp == ZSTD_e_flush || endOp == ZSTD_e_end);
-        size_t flushMin;
-        assert(forceMaxProgress || endOp == ZSTD_e_continue /* Protection for a new flush type */);
-        if (cctx->cParamsChanged) {
-            ZSTDMT_updateCParams_whileCompressing(cctx->mtctx, &cctx->requestedParams);
-            cctx->cParamsChanged = 0;
-        }
-        do {
-            flushMin = ZSTDMT_compressStream_generic(cctx->mtctx, output, input, endOp);
-            if ( ZSTD_isError(flushMin)
-              || (endOp == ZSTD_e_end && flushMin == 0) ) { /* compression completed */
-                ZSTD_CCtx_reset(cctx, ZSTD_reset_session_only);
-            }
-            FORWARD_IF_ERROR(flushMin);
-        } while (forceMaxProgress && flushMin != 0 && output->pos < output->size);
-        DEBUGLOG(5, "completed ZSTD_compressStream2 delegating to ZSTDMT_compressStream_generic");
-        /* Either we don't require maximum forward progress, we've finished the
-         * flush, or we are out of output space.
-         */
-        assert(!forceMaxProgress || flushMin == 0 || output->pos == output->size);
-        return flushMin;
-    }
-#endif
-    FORWARD_IF_ERROR( ZSTD_compressStream_generic(cctx, output, input, endOp) );
-    DEBUGLOG(5, "completed ZSTD_compressStream2");
-    return cctx->outBuffContentSize - cctx->outBuffFlushedSize; /* remaining to flush */
-}
-
-size_t ZSTD_compressStream2_simpleArgs (
-                            ZSTD_CCtx* cctx,
-                            void* dst, size_t dstCapacity, size_t* dstPos,
-                      const void* src, size_t srcSize, size_t* srcPos,
-                            ZSTD_EndDirective endOp)
-{
-    ZSTD_outBuffer output = { dst, dstCapacity, *dstPos };
-    ZSTD_inBuffer  input  = { src, srcSize, *srcPos };
-    /* ZSTD_compressStream2() will check validity of dstPos and srcPos */
-    size_t const cErr = ZSTD_compressStream2(cctx, &output, &input, endOp);
-    *dstPos = output.pos;
-    *srcPos = input.pos;
-    return cErr;
-}
-
-size_t ZSTD_compress2(ZSTD_CCtx* cctx,
-                      void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{
-    ZSTD_CCtx_reset(cctx, ZSTD_reset_session_only);
-    {   size_t oPos = 0;
-        size_t iPos = 0;
-        size_t const result = ZSTD_compressStream2_simpleArgs(cctx,
-                                        dst, dstCapacity, &oPos,
-                                        src, srcSize, &iPos,
-                                        ZSTD_e_end);
-        FORWARD_IF_ERROR(result);
-        if (result != 0) {  /* compression not completed, due to lack of output space */
-            assert(oPos == dstCapacity);
-            RETURN_ERROR(dstSize_tooSmall);
-        }
-        assert(iPos == srcSize);   /* all input is expected consumed */
-        return oPos;
-    }
-}
-
-/*======   Finalize   ======*/
-
-/*! ZSTD_flushStream() :
- * @return : amount of data remaining to flush */
-size_t ZSTD_flushStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output)
-{
-    ZSTD_inBuffer input = { NULL, 0, 0 };
-    return ZSTD_compressStream2(zcs, output, &input, ZSTD_e_flush);
-}
-
-
-size_t ZSTD_endStream(ZSTD_CStream* zcs, ZSTD_outBuffer* output)
-{
-    ZSTD_inBuffer input = { NULL, 0, 0 };
-    size_t const remainingToFlush = ZSTD_compressStream2(zcs, output, &input, ZSTD_e_end);
-    FORWARD_IF_ERROR( remainingToFlush );
-    if (zcs->appliedParams.nbWorkers > 0) return remainingToFlush;   /* minimal estimation */
-    /* single thread mode : attempt to calculate remaining to flush more precisely */
-    {   size_t const lastBlockSize = zcs->frameEnded ? 0 : ZSTD_BLOCKHEADERSIZE;
-        size_t const checksumSize = (size_t)(zcs->frameEnded ? 0 : zcs->appliedParams.fParams.checksumFlag * 4);
-        size_t const toFlush = remainingToFlush + lastBlockSize + checksumSize;
-        DEBUGLOG(4, "ZSTD_endStream : remaining to flush : %u", (unsigned)toFlush);
-        return toFlush;
-    }
-}
-
-
-/*-=====  Pre-defined compression levels  =====-*/
-
-#define ZSTD_MAX_CLEVEL     22
-int ZSTD_maxCLevel(void) { return ZSTD_MAX_CLEVEL; }
-int ZSTD_minCLevel(void) { return (int)-ZSTD_TARGETLENGTH_MAX; }
-
-static const ZSTD_compressionParameters ZSTD_defaultCParameters[4][ZSTD_MAX_CLEVEL+1] = {
-{   /* "default" - for any srcSize > 256 KB */
-    /* W,  C,  H,  S,  L, TL, strat */
-    { 19, 12, 13,  1,  6,  1, ZSTD_fast    },  /* base for negative levels */
-    { 19, 13, 14,  1,  7,  0, ZSTD_fast    },  /* level  1 */
-    { 20, 15, 16,  1,  6,  0, ZSTD_fast    },  /* level  2 */
-    { 21, 16, 17,  1,  5,  1, ZSTD_dfast   },  /* level  3 */
-    { 21, 18, 18,  1,  5,  1, ZSTD_dfast   },  /* level  4 */
-    { 21, 18, 19,  2,  5,  2, ZSTD_greedy  },  /* level  5 */
-    { 21, 19, 19,  3,  5,  4, ZSTD_greedy  },  /* level  6 */
-    { 21, 19, 19,  3,  5,  8, ZSTD_lazy    },  /* level  7 */
-    { 21, 19, 19,  3,  5, 16, ZSTD_lazy2   },  /* level  8 */
-    { 21, 19, 20,  4,  5, 16, ZSTD_lazy2   },  /* level  9 */
-    { 22, 20, 21,  4,  5, 16, ZSTD_lazy2   },  /* level 10 */
-    { 22, 21, 22,  4,  5, 16, ZSTD_lazy2   },  /* level 11 */
-    { 22, 21, 22,  5,  5, 16, ZSTD_lazy2   },  /* level 12 */
-    { 22, 21, 22,  5,  5, 32, ZSTD_btlazy2 },  /* level 13 */
-    { 22, 22, 23,  5,  5, 32, ZSTD_btlazy2 },  /* level 14 */
-    { 22, 23, 23,  6,  5, 32, ZSTD_btlazy2 },  /* level 15 */
-    { 22, 22, 22,  5,  5, 48, ZSTD_btopt   },  /* level 16 */
-    { 23, 23, 22,  5,  4, 64, ZSTD_btopt   },  /* level 17 */
-    { 23, 23, 22,  6,  3, 64, ZSTD_btultra },  /* level 18 */
-    { 23, 24, 22,  7,  3,256, ZSTD_btultra2},  /* level 19 */
-    { 25, 25, 23,  7,  3,256, ZSTD_btultra2},  /* level 20 */
-    { 26, 26, 24,  7,  3,512, ZSTD_btultra2},  /* level 21 */
-    { 27, 27, 25,  9,  3,999, ZSTD_btultra2},  /* level 22 */
-},
-{   /* for srcSize <= 256 KB */
-    /* W,  C,  H,  S,  L,  T, strat */
-    { 18, 12, 13,  1,  5,  1, ZSTD_fast    },  /* base for negative levels */
-    { 18, 13, 14,  1,  6,  0, ZSTD_fast    },  /* level  1 */
-    { 18, 14, 14,  1,  5,  1, ZSTD_dfast   },  /* level  2 */
-    { 18, 16, 16,  1,  4,  1, ZSTD_dfast   },  /* level  3 */
-    { 18, 16, 17,  2,  5,  2, ZSTD_greedy  },  /* level  4.*/
-    { 18, 18, 18,  3,  5,  2, ZSTD_greedy  },  /* level  5.*/
-    { 18, 18, 19,  3,  5,  4, ZSTD_lazy    },  /* level  6.*/
-    { 18, 18, 19,  4,  4,  4, ZSTD_lazy    },  /* level  7 */
-    { 18, 18, 19,  4,  4,  8, ZSTD_lazy2   },  /* level  8 */
-    { 18, 18, 19,  5,  4,  8, ZSTD_lazy2   },  /* level  9 */
-    { 18, 18, 19,  6,  4,  8, ZSTD_lazy2   },  /* level 10 */
-    { 18, 18, 19,  5,  4, 12, ZSTD_btlazy2 },  /* level 11.*/
-    { 18, 19, 19,  7,  4, 12, ZSTD_btlazy2 },  /* level 12.*/
-    { 18, 18, 19,  4,  4, 16, ZSTD_btopt   },  /* level 13 */
-    { 18, 18, 19,  4,  3, 32, ZSTD_btopt   },  /* level 14.*/
-    { 18, 18, 19,  6,  3,128, ZSTD_btopt   },  /* level 15.*/
-    { 18, 19, 19,  6,  3,128, ZSTD_btultra },  /* level 16.*/
-    { 18, 19, 19,  8,  3,256, ZSTD_btultra },  /* level 17.*/
-    { 18, 19, 19,  6,  3,128, ZSTD_btultra2},  /* level 18.*/
-    { 18, 19, 19,  8,  3,256, ZSTD_btultra2},  /* level 19.*/
-    { 18, 19, 19, 10,  3,512, ZSTD_btultra2},  /* level 20.*/
-    { 18, 19, 19, 12,  3,512, ZSTD_btultra2},  /* level 21.*/
-    { 18, 19, 19, 13,  3,999, ZSTD_btultra2},  /* level 22.*/
-},
-{   /* for srcSize <= 128 KB */
-    /* W,  C,  H,  S,  L,  T, strat */
-    { 17, 12, 12,  1,  5,  1, ZSTD_fast    },  /* base for negative levels */
-    { 17, 12, 13,  1,  6,  0, ZSTD_fast    },  /* level  1 */
-    { 17, 13, 15,  1,  5,  0, ZSTD_fast    },  /* level  2 */
-    { 17, 15, 16,  2,  5,  1, ZSTD_dfast   },  /* level  3 */
-    { 17, 17, 17,  2,  4,  1, ZSTD_dfast   },  /* level  4 */
-    { 17, 16, 17,  3,  4,  2, ZSTD_greedy  },  /* level  5 */
-    { 17, 17, 17,  3,  4,  4, ZSTD_lazy    },  /* level  6 */
-    { 17, 17, 17,  3,  4,  8, ZSTD_lazy2   },  /* level  7 */
-    { 17, 17, 17,  4,  4,  8, ZSTD_lazy2   },  /* level  8 */
-    { 17, 17, 17,  5,  4,  8, ZSTD_lazy2   },  /* level  9 */
-    { 17, 17, 17,  6,  4,  8, ZSTD_lazy2   },  /* level 10 */
-    { 17, 17, 17,  5,  4,  8, ZSTD_btlazy2 },  /* level 11 */
-    { 17, 18, 17,  7,  4, 12, ZSTD_btlazy2 },  /* level 12 */
-    { 17, 18, 17,  3,  4, 12, ZSTD_btopt   },  /* level 13.*/
-    { 17, 18, 17,  4,  3, 32, ZSTD_btopt   },  /* level 14.*/
-    { 17, 18, 17,  6,  3,256, ZSTD_btopt   },  /* level 15.*/
-    { 17, 18, 17,  6,  3,128, ZSTD_btultra },  /* level 16.*/
-    { 17, 18, 17,  8,  3,256, ZSTD_btultra },  /* level 17.*/
-    { 17, 18, 17, 10,  3,512, ZSTD_btultra },  /* level 18.*/
-    { 17, 18, 17,  5,  3,256, ZSTD_btultra2},  /* level 19.*/
-    { 17, 18, 17,  7,  3,512, ZSTD_btultra2},  /* level 20.*/
-    { 17, 18, 17,  9,  3,512, ZSTD_btultra2},  /* level 21.*/
-    { 17, 18, 17, 11,  3,999, ZSTD_btultra2},  /* level 22.*/
-},
-{   /* for srcSize <= 16 KB */
-    /* W,  C,  H,  S,  L,  T, strat */
-    { 14, 12, 13,  1,  5,  1, ZSTD_fast    },  /* base for negative levels */
-    { 14, 14, 15,  1,  5,  0, ZSTD_fast    },  /* level  1 */
-    { 14, 14, 15,  1,  4,  0, ZSTD_fast    },  /* level  2 */
-    { 14, 14, 15,  2,  4,  1, ZSTD_dfast   },  /* level  3 */
-    { 14, 14, 14,  4,  4,  2, ZSTD_greedy  },  /* level  4 */
-    { 14, 14, 14,  3,  4,  4, ZSTD_lazy    },  /* level  5.*/
-    { 14, 14, 14,  4,  4,  8, ZSTD_lazy2   },  /* level  6 */
-    { 14, 14, 14,  6,  4,  8, ZSTD_lazy2   },  /* level  7 */
-    { 14, 14, 14,  8,  4,  8, ZSTD_lazy2   },  /* level  8.*/
-    { 14, 15, 14,  5,  4,  8, ZSTD_btlazy2 },  /* level  9.*/
-    { 14, 15, 14,  9,  4,  8, ZSTD_btlazy2 },  /* level 10.*/
-    { 14, 15, 14,  3,  4, 12, ZSTD_btopt   },  /* level 11.*/
-    { 14, 15, 14,  4,  3, 24, ZSTD_btopt   },  /* level 12.*/
-    { 14, 15, 14,  5,  3, 32, ZSTD_btultra },  /* level 13.*/
-    { 14, 15, 15,  6,  3, 64, ZSTD_btultra },  /* level 14.*/
-    { 14, 15, 15,  7,  3,256, ZSTD_btultra },  /* level 15.*/
-    { 14, 15, 15,  5,  3, 48, ZSTD_btultra2},  /* level 16.*/
-    { 14, 15, 15,  6,  3,128, ZSTD_btultra2},  /* level 17.*/
-    { 14, 15, 15,  7,  3,256, ZSTD_btultra2},  /* level 18.*/
-    { 14, 15, 15,  8,  3,256, ZSTD_btultra2},  /* level 19.*/
-    { 14, 15, 15,  8,  3,512, ZSTD_btultra2},  /* level 20.*/
-    { 14, 15, 15,  9,  3,512, ZSTD_btultra2},  /* level 21.*/
-    { 14, 15, 15, 10,  3,999, ZSTD_btultra2},  /* level 22.*/
-},
-};
-
-/*! ZSTD_getCParams() :
- * @return ZSTD_compressionParameters structure for a selected compression level, srcSize and dictSize.
- *  Size values are optional, provide 0 if not known or unused */
-ZSTD_compressionParameters ZSTD_getCParams(int compressionLevel, unsigned long long srcSizeHint, size_t dictSize)
-{
-    size_t const addedSize = srcSizeHint ? 0 : 500;
-    U64 const rSize = srcSizeHint+dictSize ? srcSizeHint+dictSize+addedSize : ZSTD_CONTENTSIZE_UNKNOWN;  /* intentional overflow for srcSizeHint == ZSTD_CONTENTSIZE_UNKNOWN */
-    U32 const tableID = (rSize <= 256 KB) + (rSize <= 128 KB) + (rSize <= 16 KB);
-    int row = compressionLevel;
-    DEBUGLOG(5, "ZSTD_getCParams (cLevel=%i)", compressionLevel);
-    if (compressionLevel == 0) row = ZSTD_CLEVEL_DEFAULT;   /* 0 == default */
-    if (compressionLevel < 0) row = 0;   /* entry 0 is baseline for fast mode */
-    if (compressionLevel > ZSTD_MAX_CLEVEL) row = ZSTD_MAX_CLEVEL;
-    {   ZSTD_compressionParameters cp = ZSTD_defaultCParameters[tableID][row];
-        if (compressionLevel < 0) cp.targetLength = (unsigned)(-compressionLevel);   /* acceleration factor */
-        return ZSTD_adjustCParams_internal(cp, srcSizeHint, dictSize);               /* refine parameters based on srcSize & dictSize */
-    }
-}
-
-/*! ZSTD_getParams() :
- *  same idea as ZSTD_getCParams()
- * @return a `ZSTD_parameters` structure (instead of `ZSTD_compressionParameters`).
- *  Fields of `ZSTD_frameParameters` are set to default values */
-ZSTD_parameters ZSTD_getParams(int compressionLevel, unsigned long long srcSizeHint, size_t dictSize) {
-    ZSTD_parameters params;
-    ZSTD_compressionParameters const cParams = ZSTD_getCParams(compressionLevel, srcSizeHint, dictSize);
-    DEBUGLOG(5, "ZSTD_getParams (cLevel=%i)", compressionLevel);
-    memset(&params, 0, sizeof(params));
-    params.cParams = cParams;
-    params.fParams.contentSizeFlag = 1;
-    return params;
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_compress_internal.h b/vendor/github.com/DataDog/zstd/zstd_compress_internal.h
deleted file mode 100644
index 5495899..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_compress_internal.h
+++ /dev/null
@@ -1,907 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* This header contains definitions
- * that shall **only** be used by modules within lib/compress.
- */
-
-#ifndef ZSTD_COMPRESS_H
-#define ZSTD_COMPRESS_H
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include "zstd_internal.h"
-#ifdef ZSTD_MULTITHREAD
-#  include "zstdmt_compress.h"
-#endif
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define kSearchStrength      8
-#define HASH_READ_SIZE       8
-#define ZSTD_DUBT_UNSORTED_MARK 1   /* For btlazy2 strategy, index ZSTD_DUBT_UNSORTED_MARK==1 means "unsorted".
-                                       It could be confused for a real successor at index "1", if sorted as larger than its predecessor.
-                                       It's not a big deal though : candidate will just be sorted again.
-                                       Additionally, candidate position 1 will be lost.
-                                       But candidate 1 cannot hide a large tree of candidates, so it's a minimal loss.
-                                       The benefit is that ZSTD_DUBT_UNSORTED_MARK cannot be mishandled after table re-use with a different strategy.
-                                       This constant is required by ZSTD_compressBlock_btlazy2() and ZSTD_reduceTable_internal() */
-
-
-/*-*************************************
-*  Context memory management
-***************************************/
-typedef enum { ZSTDcs_created=0, ZSTDcs_init, ZSTDcs_ongoing, ZSTDcs_ending } ZSTD_compressionStage_e;
-typedef enum { zcss_init=0, zcss_load, zcss_flush } ZSTD_cStreamStage;
-
-typedef struct ZSTD_prefixDict_s {
-    const void* dict;
-    size_t dictSize;
-    ZSTD_dictContentType_e dictContentType;
-} ZSTD_prefixDict;
-
-typedef struct {
-    void* dictBuffer;
-    void const* dict;
-    size_t dictSize;
-    ZSTD_dictContentType_e dictContentType;
-    ZSTD_CDict* cdict;
-} ZSTD_localDict;
-
-typedef struct {
-    U32 CTable[HUF_CTABLE_SIZE_U32(255)];
-    HUF_repeat repeatMode;
-} ZSTD_hufCTables_t;
-
-typedef struct {
-    FSE_CTable offcodeCTable[FSE_CTABLE_SIZE_U32(OffFSELog, MaxOff)];
-    FSE_CTable matchlengthCTable[FSE_CTABLE_SIZE_U32(MLFSELog, MaxML)];
-    FSE_CTable litlengthCTable[FSE_CTABLE_SIZE_U32(LLFSELog, MaxLL)];
-    FSE_repeat offcode_repeatMode;
-    FSE_repeat matchlength_repeatMode;
-    FSE_repeat litlength_repeatMode;
-} ZSTD_fseCTables_t;
-
-typedef struct {
-    ZSTD_hufCTables_t huf;
-    ZSTD_fseCTables_t fse;
-} ZSTD_entropyCTables_t;
-
-typedef struct {
-    U32 off;
-    U32 len;
-} ZSTD_match_t;
-
-typedef struct {
-    int price;
-    U32 off;
-    U32 mlen;
-    U32 litlen;
-    U32 rep[ZSTD_REP_NUM];
-} ZSTD_optimal_t;
-
-typedef enum { zop_dynamic=0, zop_predef } ZSTD_OptPrice_e;
-
-typedef struct {
-    /* All tables are allocated inside cctx->workspace by ZSTD_resetCCtx_internal() */
-    unsigned* litFreq;           /* table of literals statistics, of size 256 */
-    unsigned* litLengthFreq;     /* table of litLength statistics, of size (MaxLL+1) */
-    unsigned* matchLengthFreq;   /* table of matchLength statistics, of size (MaxML+1) */
-    unsigned* offCodeFreq;       /* table of offCode statistics, of size (MaxOff+1) */
-    ZSTD_match_t* matchTable;    /* list of found matches, of size ZSTD_OPT_NUM+1 */
-    ZSTD_optimal_t* priceTable;  /* All positions tracked by optimal parser, of size ZSTD_OPT_NUM+1 */
-
-    U32  litSum;                 /* nb of literals */
-    U32  litLengthSum;           /* nb of litLength codes */
-    U32  matchLengthSum;         /* nb of matchLength codes */
-    U32  offCodeSum;             /* nb of offset codes */
-    U32  litSumBasePrice;        /* to compare to log2(litfreq) */
-    U32  litLengthSumBasePrice;  /* to compare to log2(llfreq)  */
-    U32  matchLengthSumBasePrice;/* to compare to log2(mlfreq)  */
-    U32  offCodeSumBasePrice;    /* to compare to log2(offreq)  */
-    ZSTD_OptPrice_e priceType;   /* prices can be determined dynamically, or follow a pre-defined cost structure */
-    const ZSTD_entropyCTables_t* symbolCosts;  /* pre-calculated dictionary statistics */
-    ZSTD_literalCompressionMode_e literalCompressionMode;
-} optState_t;
-
-typedef struct {
-  ZSTD_entropyCTables_t entropy;
-  U32 rep[ZSTD_REP_NUM];
-} ZSTD_compressedBlockState_t;
-
-typedef struct {
-    BYTE const* nextSrc;    /* next block here to continue on current prefix */
-    BYTE const* base;       /* All regular indexes relative to this position */
-    BYTE const* dictBase;   /* extDict indexes relative to this position */
-    U32 dictLimit;          /* below that point, need extDict */
-    U32 lowLimit;           /* below that point, no more valid data */
-} ZSTD_window_t;
-
-typedef struct ZSTD_matchState_t ZSTD_matchState_t;
-struct ZSTD_matchState_t {
-    ZSTD_window_t window;   /* State for window round buffer management */
-    U32 loadedDictEnd;      /* index of end of dictionary, within context's referential. When dict referential is copied into active context (i.e. not attached), effectively same value as dictSize, since referential starts from zero */
-    U32 nextToUpdate;       /* index from which to continue table update */
-    U32 hashLog3;           /* dispatch table : larger == faster, more memory */
-    U32* hashTable;
-    U32* hashTable3;
-    U32* chainTable;
-    optState_t opt;         /* optimal parser state */
-    const ZSTD_matchState_t* dictMatchState;
-    ZSTD_compressionParameters cParams;
-};
-
-typedef struct {
-    ZSTD_compressedBlockState_t* prevCBlock;
-    ZSTD_compressedBlockState_t* nextCBlock;
-    ZSTD_matchState_t matchState;
-} ZSTD_blockState_t;
-
-typedef struct {
-    U32 offset;
-    U32 checksum;
-} ldmEntry_t;
-
-typedef struct {
-    ZSTD_window_t window;   /* State for the window round buffer management */
-    ldmEntry_t* hashTable;
-    BYTE* bucketOffsets;    /* Next position in bucket to insert entry */
-    U64 hashPower;          /* Used to compute the rolling hash.
-                             * Depends on ldmParams.minMatchLength */
-} ldmState_t;
-
-typedef struct {
-    U32 enableLdm;          /* 1 if enable long distance matching */
-    U32 hashLog;            /* Log size of hashTable */
-    U32 bucketSizeLog;      /* Log bucket size for collision resolution, at most 8 */
-    U32 minMatchLength;     /* Minimum match length */
-    U32 hashRateLog;       /* Log number of entries to skip */
-    U32 windowLog;          /* Window log for the LDM */
-} ldmParams_t;
-
-typedef struct {
-    U32 offset;
-    U32 litLength;
-    U32 matchLength;
-} rawSeq;
-
-typedef struct {
-  rawSeq* seq;     /* The start of the sequences */
-  size_t pos;      /* The position where reading stopped. <= size. */
-  size_t size;     /* The number of sequences. <= capacity. */
-  size_t capacity; /* The capacity starting from `seq` pointer */
-} rawSeqStore_t;
-
-struct ZSTD_CCtx_params_s {
-    ZSTD_format_e format;
-    ZSTD_compressionParameters cParams;
-    ZSTD_frameParameters fParams;
-
-    int compressionLevel;
-    int forceWindow;           /* force back-references to respect limit of
-                                * 1<<wLog, even for dictionary */
-    size_t targetCBlockSize;   /* Tries to fit compressed block size to be around targetCBlockSize.
-                                * No target when targetCBlockSize == 0.
-                                * There is no guarantee on compressed block size */
-
-    ZSTD_dictAttachPref_e attachDictPref;
-    ZSTD_literalCompressionMode_e literalCompressionMode;
-
-    /* Multithreading: used to pass parameters to mtctx */
-    int nbWorkers;
-    size_t jobSize;
-    int overlapLog;
-    int rsyncable;
-
-    /* Long distance matching parameters */
-    ldmParams_t ldmParams;
-
-    /* Internal use, for createCCtxParams() and freeCCtxParams() only */
-    ZSTD_customMem customMem;
-};  /* typedef'd to ZSTD_CCtx_params within "zstd.h" */
-
-struct ZSTD_CCtx_s {
-    ZSTD_compressionStage_e stage;
-    int cParamsChanged;                  /* == 1 if cParams(except wlog) or compression level are changed in requestedParams. Triggers transmission of new params to ZSTDMT (if available) then reset to 0. */
-    int bmi2;                            /* == 1 if the CPU supports BMI2 and 0 otherwise. CPU support is determined dynamically once per context lifetime. */
-    ZSTD_CCtx_params requestedParams;
-    ZSTD_CCtx_params appliedParams;
-    U32   dictID;
-
-    int workSpaceOversizedDuration;
-    void* workSpace;
-    size_t workSpaceSize;
-    size_t blockSize;
-    unsigned long long pledgedSrcSizePlusOne;  /* this way, 0 (default) == unknown */
-    unsigned long long consumedSrcSize;
-    unsigned long long producedCSize;
-    XXH64_state_t xxhState;
-    ZSTD_customMem customMem;
-    size_t staticSize;
-
-    seqStore_t seqStore;      /* sequences storage ptrs */
-    ldmState_t ldmState;      /* long distance matching state */
-    rawSeq* ldmSequences;     /* Storage for the ldm output sequences */
-    size_t maxNbLdmSequences;
-    rawSeqStore_t externSeqStore; /* Mutable reference to external sequences */
-    ZSTD_blockState_t blockState;
-    U32* entropyWorkspace;  /* entropy workspace of HUF_WORKSPACE_SIZE bytes */
-
-    /* streaming */
-    char*  inBuff;
-    size_t inBuffSize;
-    size_t inToCompress;
-    size_t inBuffPos;
-    size_t inBuffTarget;
-    char*  outBuff;
-    size_t outBuffSize;
-    size_t outBuffContentSize;
-    size_t outBuffFlushedSize;
-    ZSTD_cStreamStage streamStage;
-    U32    frameEnded;
-
-    /* Dictionary */
-    ZSTD_localDict localDict;
-    const ZSTD_CDict* cdict;
-    ZSTD_prefixDict prefixDict;   /* single-usage dictionary */
-
-    /* Multi-threading */
-#ifdef ZSTD_MULTITHREAD
-    ZSTDMT_CCtx* mtctx;
-#endif
-};
-
-typedef enum { ZSTD_dtlm_fast, ZSTD_dtlm_full } ZSTD_dictTableLoadMethod_e;
-
-typedef enum { ZSTD_noDict = 0, ZSTD_extDict = 1, ZSTD_dictMatchState = 2 } ZSTD_dictMode_e;
-
-
-typedef size_t (*ZSTD_blockCompressor) (
-        ZSTD_matchState_t* bs, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-ZSTD_blockCompressor ZSTD_selectBlockCompressor(ZSTD_strategy strat, ZSTD_dictMode_e dictMode);
-
-
-MEM_STATIC U32 ZSTD_LLcode(U32 litLength)
-{
-    static const BYTE LL_Code[64] = {  0,  1,  2,  3,  4,  5,  6,  7,
-                                       8,  9, 10, 11, 12, 13, 14, 15,
-                                      16, 16, 17, 17, 18, 18, 19, 19,
-                                      20, 20, 20, 20, 21, 21, 21, 21,
-                                      22, 22, 22, 22, 22, 22, 22, 22,
-                                      23, 23, 23, 23, 23, 23, 23, 23,
-                                      24, 24, 24, 24, 24, 24, 24, 24,
-                                      24, 24, 24, 24, 24, 24, 24, 24 };
-    static const U32 LL_deltaCode = 19;
-    return (litLength > 63) ? ZSTD_highbit32(litLength) + LL_deltaCode : LL_Code[litLength];
-}
-
-/* ZSTD_MLcode() :
- * note : mlBase = matchLength - MINMATCH;
- *        because it's the format it's stored in seqStore->sequences */
-MEM_STATIC U32 ZSTD_MLcode(U32 mlBase)
-{
-    static const BYTE ML_Code[128] = { 0,  1,  2,  3,  4,  5,  6,  7,  8,  9, 10, 11, 12, 13, 14, 15,
-                                      16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31,
-                                      32, 32, 33, 33, 34, 34, 35, 35, 36, 36, 36, 36, 37, 37, 37, 37,
-                                      38, 38, 38, 38, 38, 38, 38, 38, 39, 39, 39, 39, 39, 39, 39, 39,
-                                      40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40,
-                                      41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41,
-                                      42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42,
-                                      42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42 };
-    static const U32 ML_deltaCode = 36;
-    return (mlBase > 127) ? ZSTD_highbit32(mlBase) + ML_deltaCode : ML_Code[mlBase];
-}
-
-/*! ZSTD_storeSeq() :
- *  Store a sequence (literal length, literals, offset code and match length code) into seqStore_t.
- *  `offsetCode` : distance to match + 3 (values 1-3 are repCodes).
- *  `mlBase` : matchLength - MINMATCH
-*/
-MEM_STATIC void ZSTD_storeSeq(seqStore_t* seqStorePtr, size_t litLength, const void* literals, U32 offsetCode, size_t mlBase)
-{
-#if defined(DEBUGLEVEL) && (DEBUGLEVEL >= 6)
-    static const BYTE* g_start = NULL;
-    if (g_start==NULL) g_start = (const BYTE*)literals;  /* note : index only works for compression within a single segment */
-    {   U32 const pos = (U32)((const BYTE*)literals - g_start);
-        DEBUGLOG(6, "Cpos%7u :%3u literals, match%4u bytes at offCode%7u",
-               pos, (U32)litLength, (U32)mlBase+MINMATCH, (U32)offsetCode);
-    }
-#endif
-    assert((size_t)(seqStorePtr->sequences - seqStorePtr->sequencesStart) < seqStorePtr->maxNbSeq);
-    /* copy Literals */
-    assert(seqStorePtr->maxNbLit <= 128 KB);
-    assert(seqStorePtr->lit + litLength <= seqStorePtr->litStart + seqStorePtr->maxNbLit);
-    ZSTD_wildcopy(seqStorePtr->lit, literals, litLength, ZSTD_no_overlap);
-    seqStorePtr->lit += litLength;
-
-    /* literal Length */
-    if (litLength>0xFFFF) {
-        assert(seqStorePtr->longLengthID == 0); /* there can only be a single long length */
-        seqStorePtr->longLengthID = 1;
-        seqStorePtr->longLengthPos = (U32)(seqStorePtr->sequences - seqStorePtr->sequencesStart);
-    }
-    seqStorePtr->sequences[0].litLength = (U16)litLength;
-
-    /* match offset */
-    seqStorePtr->sequences[0].offset = offsetCode + 1;
-
-    /* match Length */
-    if (mlBase>0xFFFF) {
-        assert(seqStorePtr->longLengthID == 0); /* there can only be a single long length */
-        seqStorePtr->longLengthID = 2;
-        seqStorePtr->longLengthPos = (U32)(seqStorePtr->sequences - seqStorePtr->sequencesStart);
-    }
-    seqStorePtr->sequences[0].matchLength = (U16)mlBase;
-
-    seqStorePtr->sequences++;
-}
-
-
-/*-*************************************
-*  Match length counter
-***************************************/
-static unsigned ZSTD_NbCommonBytes (size_t val)
-{
-    if (MEM_isLittleEndian()) {
-        if (MEM_64bits()) {
-#       if defined(_MSC_VER) && defined(_WIN64)
-            unsigned long r = 0;
-            _BitScanForward64( &r, (U64)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 4)
-            return (__builtin_ctzll((U64)val) >> 3);
-#       else
-            static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2,
-                                                     0, 3, 1, 3, 1, 4, 2, 7,
-                                                     0, 2, 3, 6, 1, 5, 3, 5,
-                                                     1, 3, 4, 4, 2, 5, 6, 7,
-                                                     7, 0, 1, 2, 3, 3, 4, 6,
-                                                     2, 6, 5, 5, 3, 4, 5, 6,
-                                                     7, 1, 2, 4, 6, 4, 4, 5,
-                                                     7, 2, 6, 5, 7, 6, 7, 7 };
-            return DeBruijnBytePos[((U64)((val & -(long long)val) * 0x0218A392CDABBD3FULL)) >> 58];
-#       endif
-        } else { /* 32 bits */
-#       if defined(_MSC_VER)
-            unsigned long r=0;
-            _BitScanForward( &r, (U32)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_ctz((U32)val) >> 3);
-#       else
-            static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0,
-                                                     3, 2, 2, 1, 3, 2, 0, 1,
-                                                     3, 3, 1, 2, 2, 2, 2, 0,
-                                                     3, 1, 2, 0, 1, 0, 1, 1 };
-            return DeBruijnBytePos[((U32)((val & -(S32)val) * 0x077CB531U)) >> 27];
-#       endif
-        }
-    } else {  /* Big Endian CPU */
-        if (MEM_64bits()) {
-#       if defined(_MSC_VER) && defined(_WIN64)
-            unsigned long r = 0;
-            _BitScanReverse64( &r, val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 4)
-            return (__builtin_clzll(val) >> 3);
-#       else
-            unsigned r;
-            const unsigned n32 = sizeof(size_t)*4;   /* calculate this way due to compiler complaining in 32-bits mode */
-            if (!(val>>n32)) { r=4; } else { r=0; val>>=n32; }
-            if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; }
-            r += (!val);
-            return r;
-#       endif
-        } else { /* 32 bits */
-#       if defined(_MSC_VER)
-            unsigned long r = 0;
-            _BitScanReverse( &r, (unsigned long)val );
-            return (unsigned)(r>>3);
-#       elif defined(__GNUC__) && (__GNUC__ >= 3)
-            return (__builtin_clz((U32)val) >> 3);
-#       else
-            unsigned r;
-            if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; }
-            r += (!val);
-            return r;
-#       endif
-    }   }
-}
-
-
-MEM_STATIC size_t ZSTD_count(const BYTE* pIn, const BYTE* pMatch, const BYTE* const pInLimit)
-{
-    const BYTE* const pStart = pIn;
-    const BYTE* const pInLoopLimit = pInLimit - (sizeof(size_t)-1);
-
-    if (pIn < pInLoopLimit) {
-        { size_t const diff = MEM_readST(pMatch) ^ MEM_readST(pIn);
-          if (diff) return ZSTD_NbCommonBytes(diff); }
-        pIn+=sizeof(size_t); pMatch+=sizeof(size_t);
-        while (pIn < pInLoopLimit) {
-            size_t const diff = MEM_readST(pMatch) ^ MEM_readST(pIn);
-            if (!diff) { pIn+=sizeof(size_t); pMatch+=sizeof(size_t); continue; }
-            pIn += ZSTD_NbCommonBytes(diff);
-            return (size_t)(pIn - pStart);
-    }   }
-    if (MEM_64bits() && (pIn<(pInLimit-3)) && (MEM_read32(pMatch) == MEM_read32(pIn))) { pIn+=4; pMatch+=4; }
-    if ((pIn<(pInLimit-1)) && (MEM_read16(pMatch) == MEM_read16(pIn))) { pIn+=2; pMatch+=2; }
-    if ((pIn<pInLimit) && (*pMatch == *pIn)) pIn++;
-    return (size_t)(pIn - pStart);
-}
-
-/** ZSTD_count_2segments() :
- *  can count match length with `ip` & `match` in 2 different segments.
- *  convention : on reaching mEnd, match count continue starting from iStart
- */
-MEM_STATIC size_t
-ZSTD_count_2segments(const BYTE* ip, const BYTE* match,
-                     const BYTE* iEnd, const BYTE* mEnd, const BYTE* iStart)
-{
-    const BYTE* const vEnd = MIN( ip + (mEnd - match), iEnd);
-    size_t const matchLength = ZSTD_count(ip, match, vEnd);
-    if (match + matchLength != mEnd) return matchLength;
-    DEBUGLOG(7, "ZSTD_count_2segments: found a 2-parts match (current length==%zu)", matchLength);
-    DEBUGLOG(7, "distance from match beginning to end dictionary = %zi", mEnd - match);
-    DEBUGLOG(7, "distance from current pos to end buffer = %zi", iEnd - ip);
-    DEBUGLOG(7, "next byte : ip==%02X, istart==%02X", ip[matchLength], *iStart);
-    DEBUGLOG(7, "final match length = %zu", matchLength + ZSTD_count(ip+matchLength, iStart, iEnd));
-    return matchLength + ZSTD_count(ip+matchLength, iStart, iEnd);
-}
-
-
-/*-*************************************
- *  Hashes
- ***************************************/
-static const U32 prime3bytes = 506832829U;
-static U32    ZSTD_hash3(U32 u, U32 h) { return ((u << (32-24)) * prime3bytes)  >> (32-h) ; }
-MEM_STATIC size_t ZSTD_hash3Ptr(const void* ptr, U32 h) { return ZSTD_hash3(MEM_readLE32(ptr), h); } /* only in zstd_opt.h */
-
-static const U32 prime4bytes = 2654435761U;
-static U32    ZSTD_hash4(U32 u, U32 h) { return (u * prime4bytes) >> (32-h) ; }
-static size_t ZSTD_hash4Ptr(const void* ptr, U32 h) { return ZSTD_hash4(MEM_read32(ptr), h); }
-
-static const U64 prime5bytes = 889523592379ULL;
-static size_t ZSTD_hash5(U64 u, U32 h) { return (size_t)(((u  << (64-40)) * prime5bytes) >> (64-h)) ; }
-static size_t ZSTD_hash5Ptr(const void* p, U32 h) { return ZSTD_hash5(MEM_readLE64(p), h); }
-
-static const U64 prime6bytes = 227718039650203ULL;
-static size_t ZSTD_hash6(U64 u, U32 h) { return (size_t)(((u  << (64-48)) * prime6bytes) >> (64-h)) ; }
-static size_t ZSTD_hash6Ptr(const void* p, U32 h) { return ZSTD_hash6(MEM_readLE64(p), h); }
-
-static const U64 prime7bytes = 58295818150454627ULL;
-static size_t ZSTD_hash7(U64 u, U32 h) { return (size_t)(((u  << (64-56)) * prime7bytes) >> (64-h)) ; }
-static size_t ZSTD_hash7Ptr(const void* p, U32 h) { return ZSTD_hash7(MEM_readLE64(p), h); }
-
-static const U64 prime8bytes = 0xCF1BBCDCB7A56463ULL;
-static size_t ZSTD_hash8(U64 u, U32 h) { return (size_t)(((u) * prime8bytes) >> (64-h)) ; }
-static size_t ZSTD_hash8Ptr(const void* p, U32 h) { return ZSTD_hash8(MEM_readLE64(p), h); }
-
-MEM_STATIC size_t ZSTD_hashPtr(const void* p, U32 hBits, U32 mls)
-{
-    switch(mls)
-    {
-    default:
-    case 4: return ZSTD_hash4Ptr(p, hBits);
-    case 5: return ZSTD_hash5Ptr(p, hBits);
-    case 6: return ZSTD_hash6Ptr(p, hBits);
-    case 7: return ZSTD_hash7Ptr(p, hBits);
-    case 8: return ZSTD_hash8Ptr(p, hBits);
-    }
-}
-
-/** ZSTD_ipow() :
- * Return base^exponent.
- */
-static U64 ZSTD_ipow(U64 base, U64 exponent)
-{
-    U64 power = 1;
-    while (exponent) {
-      if (exponent & 1) power *= base;
-      exponent >>= 1;
-      base *= base;
-    }
-    return power;
-}
-
-#define ZSTD_ROLL_HASH_CHAR_OFFSET 10
-
-/** ZSTD_rollingHash_append() :
- * Add the buffer to the hash value.
- */
-static U64 ZSTD_rollingHash_append(U64 hash, void const* buf, size_t size)
-{
-    BYTE const* istart = (BYTE const*)buf;
-    size_t pos;
-    for (pos = 0; pos < size; ++pos) {
-        hash *= prime8bytes;
-        hash += istart[pos] + ZSTD_ROLL_HASH_CHAR_OFFSET;
-    }
-    return hash;
-}
-
-/** ZSTD_rollingHash_compute() :
- * Compute the rolling hash value of the buffer.
- */
-MEM_STATIC U64 ZSTD_rollingHash_compute(void const* buf, size_t size)
-{
-    return ZSTD_rollingHash_append(0, buf, size);
-}
-
-/** ZSTD_rollingHash_primePower() :
- * Compute the primePower to be passed to ZSTD_rollingHash_rotate() for a hash
- * over a window of length bytes.
- */
-MEM_STATIC U64 ZSTD_rollingHash_primePower(U32 length)
-{
-    return ZSTD_ipow(prime8bytes, length - 1);
-}
-
-/** ZSTD_rollingHash_rotate() :
- * Rotate the rolling hash by one byte.
- */
-MEM_STATIC U64 ZSTD_rollingHash_rotate(U64 hash, BYTE toRemove, BYTE toAdd, U64 primePower)
-{
-    hash -= (toRemove + ZSTD_ROLL_HASH_CHAR_OFFSET) * primePower;
-    hash *= prime8bytes;
-    hash += toAdd + ZSTD_ROLL_HASH_CHAR_OFFSET;
-    return hash;
-}
-
-/*-*************************************
-*  Round buffer management
-***************************************/
-#if (ZSTD_WINDOWLOG_MAX_64 > 31)
-# error "ZSTD_WINDOWLOG_MAX is too large : would overflow ZSTD_CURRENT_MAX"
-#endif
-/* Max current allowed */
-#define ZSTD_CURRENT_MAX ((3U << 29) + (1U << ZSTD_WINDOWLOG_MAX))
-/* Maximum chunk size before overflow correction needs to be called again */
-#define ZSTD_CHUNKSIZE_MAX                                                     \
-    ( ((U32)-1)                  /* Maximum ending current index */            \
-    - ZSTD_CURRENT_MAX)          /* Maximum beginning lowLimit */
-
-/**
- * ZSTD_window_clear():
- * Clears the window containing the history by simply setting it to empty.
- */
-MEM_STATIC void ZSTD_window_clear(ZSTD_window_t* window)
-{
-    size_t const endT = (size_t)(window->nextSrc - window->base);
-    U32 const end = (U32)endT;
-
-    window->lowLimit = end;
-    window->dictLimit = end;
-}
-
-/**
- * ZSTD_window_hasExtDict():
- * Returns non-zero if the window has a non-empty extDict.
- */
-MEM_STATIC U32 ZSTD_window_hasExtDict(ZSTD_window_t const window)
-{
-    return window.lowLimit < window.dictLimit;
-}
-
-/**
- * ZSTD_matchState_dictMode():
- * Inspects the provided matchState and figures out what dictMode should be
- * passed to the compressor.
- */
-MEM_STATIC ZSTD_dictMode_e ZSTD_matchState_dictMode(const ZSTD_matchState_t *ms)
-{
-    return ZSTD_window_hasExtDict(ms->window) ?
-        ZSTD_extDict :
-        ms->dictMatchState != NULL ?
-            ZSTD_dictMatchState :
-            ZSTD_noDict;
-}
-
-/**
- * ZSTD_window_needOverflowCorrection():
- * Returns non-zero if the indices are getting too large and need overflow
- * protection.
- */
-MEM_STATIC U32 ZSTD_window_needOverflowCorrection(ZSTD_window_t const window,
-                                                  void const* srcEnd)
-{
-    U32 const current = (U32)((BYTE const*)srcEnd - window.base);
-    return current > ZSTD_CURRENT_MAX;
-}
-
-/**
- * ZSTD_window_correctOverflow():
- * Reduces the indices to protect from index overflow.
- * Returns the correction made to the indices, which must be applied to every
- * stored index.
- *
- * The least significant cycleLog bits of the indices must remain the same,
- * which may be 0. Every index up to maxDist in the past must be valid.
- * NOTE: (maxDist & cycleMask) must be zero.
- */
-MEM_STATIC U32 ZSTD_window_correctOverflow(ZSTD_window_t* window, U32 cycleLog,
-                                           U32 maxDist, void const* src)
-{
-    /* preemptive overflow correction:
-     * 1. correction is large enough:
-     *    lowLimit > (3<<29) ==> current > 3<<29 + 1<<windowLog
-     *    1<<windowLog <= newCurrent < 1<<chainLog + 1<<windowLog
-     *
-     *    current - newCurrent
-     *    > (3<<29 + 1<<windowLog) - (1<<windowLog + 1<<chainLog)
-     *    > (3<<29) - (1<<chainLog)
-     *    > (3<<29) - (1<<30)             (NOTE: chainLog <= 30)
-     *    > 1<<29
-     *
-     * 2. (ip+ZSTD_CHUNKSIZE_MAX - cctx->base) doesn't overflow:
-     *    After correction, current is less than (1<<chainLog + 1<<windowLog).
-     *    In 64-bit mode we are safe, because we have 64-bit ptrdiff_t.
-     *    In 32-bit mode we are safe, because (chainLog <= 29), so
-     *    ip+ZSTD_CHUNKSIZE_MAX - cctx->base < 1<<32.
-     * 3. (cctx->lowLimit + 1<<windowLog) < 1<<32:
-     *    windowLog <= 31 ==> 3<<29 + 1<<windowLog < 7<<29 < 1<<32.
-     */
-    U32 const cycleMask = (1U << cycleLog) - 1;
-    U32 const current = (U32)((BYTE const*)src - window->base);
-    U32 const newCurrent = (current & cycleMask) + maxDist;
-    U32 const correction = current - newCurrent;
-    assert((maxDist & cycleMask) == 0);
-    assert(current > newCurrent);
-    /* Loose bound, should be around 1<<29 (see above) */
-    assert(correction > 1<<28);
-
-    window->base += correction;
-    window->dictBase += correction;
-    window->lowLimit -= correction;
-    window->dictLimit -= correction;
-
-    DEBUGLOG(4, "Correction of 0x%x bytes to lowLimit=0x%x", correction,
-             window->lowLimit);
-    return correction;
-}
-
-/**
- * ZSTD_window_enforceMaxDist():
- * Updates lowLimit so that:
- *    (srcEnd - base) - lowLimit == maxDist + loadedDictEnd
- *
- * It ensures index is valid as long as index >= lowLimit.
- * This must be called before a block compression call.
- *
- * loadedDictEnd is only defined if a dictionary is in use for current compression.
- * As the name implies, loadedDictEnd represents the index at end of dictionary.
- * The value lies within context's referential, it can be directly compared to blockEndIdx.
- *
- * If loadedDictEndPtr is NULL, no dictionary is in use, and we use loadedDictEnd == 0.
- * If loadedDictEndPtr is not NULL, we set it to zero after updating lowLimit.
- * This is because dictionaries are allowed to be referenced fully
- * as long as the last byte of the dictionary is in the window.
- * Once input has progressed beyond window size, dictionary cannot be referenced anymore.
- *
- * In normal dict mode, the dictionary lies between lowLimit and dictLimit.
- * In dictMatchState mode, lowLimit and dictLimit are the same,
- * and the dictionary is below them.
- * forceWindow and dictMatchState are therefore incompatible.
- */
-MEM_STATIC void
-ZSTD_window_enforceMaxDist(ZSTD_window_t* window,
-                     const void* blockEnd,
-                           U32   maxDist,
-                           U32*  loadedDictEndPtr,
-                     const ZSTD_matchState_t** dictMatchStatePtr)
-{
-    U32 const blockEndIdx = (U32)((BYTE const*)blockEnd - window->base);
-    U32 const loadedDictEnd = (loadedDictEndPtr != NULL) ? *loadedDictEndPtr : 0;
-    DEBUGLOG(5, "ZSTD_window_enforceMaxDist: blockEndIdx=%u, maxDist=%u, loadedDictEnd=%u",
-                (unsigned)blockEndIdx, (unsigned)maxDist, (unsigned)loadedDictEnd);
-
-    /* - When there is no dictionary : loadedDictEnd == 0.
-         In which case, the test (blockEndIdx > maxDist) is merely to avoid
-         overflowing next operation `newLowLimit = blockEndIdx - maxDist`.
-       - When there is a standard dictionary :
-         Index referential is copied from the dictionary,
-         which means it starts from 0.
-         In which case, loadedDictEnd == dictSize,
-         and it makes sense to compare `blockEndIdx > maxDist + dictSize`
-         since `blockEndIdx` also starts from zero.
-       - When there is an attached dictionary :
-         loadedDictEnd is expressed within the referential of the context,
-         so it can be directly compared against blockEndIdx.
-    */
-    if (blockEndIdx > maxDist + loadedDictEnd) {
-        U32 const newLowLimit = blockEndIdx - maxDist;
-        if (window->lowLimit < newLowLimit) window->lowLimit = newLowLimit;
-        if (window->dictLimit < window->lowLimit) {
-            DEBUGLOG(5, "Update dictLimit to match lowLimit, from %u to %u",
-                        (unsigned)window->dictLimit, (unsigned)window->lowLimit);
-            window->dictLimit = window->lowLimit;
-        }
-        /* On reaching window size, dictionaries are invalidated */
-        if (loadedDictEndPtr) *loadedDictEndPtr = 0;
-        if (dictMatchStatePtr) *dictMatchStatePtr = NULL;
-    }
-}
-
-/* Similar to ZSTD_window_enforceMaxDist(),
- * but only invalidates dictionary
- * when input progresses beyond window size. */
-MEM_STATIC void
-ZSTD_checkDictValidity(ZSTD_window_t* window,
-                       const void* blockEnd,
-                             U32   maxDist,
-                             U32*  loadedDictEndPtr,
-                       const ZSTD_matchState_t** dictMatchStatePtr)
-{
-    U32 const blockEndIdx = (U32)((BYTE const*)blockEnd - window->base);
-    U32 const loadedDictEnd = (loadedDictEndPtr != NULL) ? *loadedDictEndPtr : 0;
-    DEBUGLOG(5, "ZSTD_checkDictValidity: blockEndIdx=%u, maxDist=%u, loadedDictEnd=%u",
-                (unsigned)blockEndIdx, (unsigned)maxDist, (unsigned)loadedDictEnd);
-
-    if (loadedDictEnd && (blockEndIdx > maxDist + loadedDictEnd)) {
-        /* On reaching window size, dictionaries are invalidated */
-        if (loadedDictEndPtr) *loadedDictEndPtr = 0;
-        if (dictMatchStatePtr) *dictMatchStatePtr = NULL;
-    }
-}
-
-/**
- * ZSTD_window_update():
- * Updates the window by appending [src, src + srcSize) to the window.
- * If it is not contiguous, the current prefix becomes the extDict, and we
- * forget about the extDict. Handles overlap of the prefix and extDict.
- * Returns non-zero if the segment is contiguous.
- */
-MEM_STATIC U32 ZSTD_window_update(ZSTD_window_t* window,
-                                  void const* src, size_t srcSize)
-{
-    BYTE const* const ip = (BYTE const*)src;
-    U32 contiguous = 1;
-    DEBUGLOG(5, "ZSTD_window_update");
-    /* Check if blocks follow each other */
-    if (src != window->nextSrc) {
-        /* not contiguous */
-        size_t const distanceFromBase = (size_t)(window->nextSrc - window->base);
-        DEBUGLOG(5, "Non contiguous blocks, new segment starts at %u", window->dictLimit);
-        window->lowLimit = window->dictLimit;
-        assert(distanceFromBase == (size_t)(U32)distanceFromBase);  /* should never overflow */
-        window->dictLimit = (U32)distanceFromBase;
-        window->dictBase = window->base;
-        window->base = ip - distanceFromBase;
-        // ms->nextToUpdate = window->dictLimit;
-        if (window->dictLimit - window->lowLimit < HASH_READ_SIZE) window->lowLimit = window->dictLimit;   /* too small extDict */
-        contiguous = 0;
-    }
-    window->nextSrc = ip + srcSize;
-    /* if input and dictionary overlap : reduce dictionary (area presumed modified by input) */
-    if ( (ip+srcSize > window->dictBase + window->lowLimit)
-       & (ip < window->dictBase + window->dictLimit)) {
-        ptrdiff_t const highInputIdx = (ip + srcSize) - window->dictBase;
-        U32 const lowLimitMax = (highInputIdx > (ptrdiff_t)window->dictLimit) ? window->dictLimit : (U32)highInputIdx;
-        window->lowLimit = lowLimitMax;
-        DEBUGLOG(5, "Overlapping extDict and input : new lowLimit = %u", window->lowLimit);
-    }
-    return contiguous;
-}
-
-
-/* debug functions */
-#if (DEBUGLEVEL>=2)
-
-MEM_STATIC double ZSTD_fWeight(U32 rawStat)
-{
-    U32 const fp_accuracy = 8;
-    U32 const fp_multiplier = (1 << fp_accuracy);
-    U32 const newStat = rawStat + 1;
-    U32 const hb = ZSTD_highbit32(newStat);
-    U32 const BWeight = hb * fp_multiplier;
-    U32 const FWeight = (newStat << fp_accuracy) >> hb;
-    U32 const weight = BWeight + FWeight;
-    assert(hb + fp_accuracy < 31);
-    return (double)weight / fp_multiplier;
-}
-
-/* display a table content,
- * listing each element, its frequency, and its predicted bit cost */
-MEM_STATIC void ZSTD_debugTable(const U32* table, U32 max)
-{
-    unsigned u, sum;
-    for (u=0, sum=0; u<=max; u++) sum += table[u];
-    DEBUGLOG(2, "total nb elts: %u", sum);
-    for (u=0; u<=max; u++) {
-        DEBUGLOG(2, "%2u: %5u  (%.2f)",
-                u, table[u], ZSTD_fWeight(sum) - ZSTD_fWeight(table[u]) );
-    }
-}
-
-#endif
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-
-/* ==============================================================
- * Private declarations
- * These prototypes shall only be called from within lib/compress
- * ============================================================== */
-
-/* ZSTD_getCParamsFromCCtxParams() :
- * cParams are built depending on compressionLevel, src size hints,
- * LDM and manually set compression parameters.
- */
-ZSTD_compressionParameters ZSTD_getCParamsFromCCtxParams(
-        const ZSTD_CCtx_params* CCtxParams, U64 srcSizeHint, size_t dictSize);
-
-/*! ZSTD_initCStream_internal() :
- *  Private use only. Init streaming operation.
- *  expects params to be valid.
- *  must receive dict, or cdict, or none, but not both.
- *  @return : 0, or an error code */
-size_t ZSTD_initCStream_internal(ZSTD_CStream* zcs,
-                     const void* dict, size_t dictSize,
-                     const ZSTD_CDict* cdict,
-                     ZSTD_CCtx_params  params, unsigned long long pledgedSrcSize);
-
-void ZSTD_resetSeqStore(seqStore_t* ssPtr);
-
-/*! ZSTD_getCParamsFromCDict() :
- *  as the name implies */
-ZSTD_compressionParameters ZSTD_getCParamsFromCDict(const ZSTD_CDict* cdict);
-
-/* ZSTD_compressBegin_advanced_internal() :
- * Private use only. To be called from zstdmt_compress.c. */
-size_t ZSTD_compressBegin_advanced_internal(ZSTD_CCtx* cctx,
-                                    const void* dict, size_t dictSize,
-                                    ZSTD_dictContentType_e dictContentType,
-                                    ZSTD_dictTableLoadMethod_e dtlm,
-                                    const ZSTD_CDict* cdict,
-                                    ZSTD_CCtx_params params,
-                                    unsigned long long pledgedSrcSize);
-
-/* ZSTD_compress_advanced_internal() :
- * Private use only. To be called from zstdmt_compress.c. */
-size_t ZSTD_compress_advanced_internal(ZSTD_CCtx* cctx,
-                                       void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize,
-                                 const void* dict,size_t dictSize,
-                                 ZSTD_CCtx_params params);
-
-
-/* ZSTD_writeLastEmptyBlock() :
- * output an empty Block with end-of-frame mark to complete a frame
- * @return : size of data written into `dst` (== ZSTD_blockHeaderSize (defined in zstd_internal.h))
- *           or an error code if `dstCapacity` is too small (<ZSTD_blockHeaderSize)
- */
-size_t ZSTD_writeLastEmptyBlock(void* dst, size_t dstCapacity);
-
-
-/* ZSTD_referenceExternalSequences() :
- * Must be called before starting a compression operation.
- * seqs must parse a prefix of the source.
- * This cannot be used when long range matching is enabled.
- * Zstd will use these sequences, and pass the literals to a secondary block
- * compressor.
- * @return : An error code on failure.
- * NOTE: seqs are not verified! Invalid sequences can cause out-of-bounds memory
- * access and data corruption.
- */
-size_t ZSTD_referenceExternalSequences(ZSTD_CCtx* cctx, rawSeq* seq, size_t nbSeq);
-
-
-#endif /* ZSTD_COMPRESS_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_ddict.c b/vendor/github.com/DataDog/zstd/zstd_ddict.c
deleted file mode 100644
index 0af3d23..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_ddict.c
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* zstd_ddict.c :
- * concentrates all logic that needs to know the internals of ZSTD_DDict object */
-
-/*-*******************************************************
-*  Dependencies
-*********************************************************/
-#include <string.h>      /* memcpy, memmove, memset */
-#include "cpu.h"         /* bmi2 */
-#include "mem.h"         /* low level memory routines */
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "zstd_decompress_internal.h"
-#include "zstd_ddict.h"
-
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT>=1)
-#  include "zstd_legacy.h"
-#endif
-
-
-
-/*-*******************************************************
-*  Types
-*********************************************************/
-struct ZSTD_DDict_s {
-    void* dictBuffer;
-    const void* dictContent;
-    size_t dictSize;
-    ZSTD_entropyDTables_t entropy;
-    U32 dictID;
-    U32 entropyPresent;
-    ZSTD_customMem cMem;
-};  /* typedef'd to ZSTD_DDict within "zstd.h" */
-
-const void* ZSTD_DDict_dictContent(const ZSTD_DDict* ddict)
-{
-    assert(ddict != NULL);
-    return ddict->dictContent;
-}
-
-size_t ZSTD_DDict_dictSize(const ZSTD_DDict* ddict)
-{
-    assert(ddict != NULL);
-    return ddict->dictSize;
-}
-
-void ZSTD_copyDDictParameters(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict)
-{
-    DEBUGLOG(4, "ZSTD_copyDDictParameters");
-    assert(dctx != NULL);
-    assert(ddict != NULL);
-    dctx->dictID = ddict->dictID;
-    dctx->prefixStart = ddict->dictContent;
-    dctx->virtualStart = ddict->dictContent;
-    dctx->dictEnd = (const BYTE*)ddict->dictContent + ddict->dictSize;
-    dctx->previousDstEnd = dctx->dictEnd;
-    if (ddict->entropyPresent) {
-        dctx->litEntropy = 1;
-        dctx->fseEntropy = 1;
-        dctx->LLTptr = ddict->entropy.LLTable;
-        dctx->MLTptr = ddict->entropy.MLTable;
-        dctx->OFTptr = ddict->entropy.OFTable;
-        dctx->HUFptr = ddict->entropy.hufTable;
-        dctx->entropy.rep[0] = ddict->entropy.rep[0];
-        dctx->entropy.rep[1] = ddict->entropy.rep[1];
-        dctx->entropy.rep[2] = ddict->entropy.rep[2];
-    } else {
-        dctx->litEntropy = 0;
-        dctx->fseEntropy = 0;
-    }
-}
-
-
-static size_t
-ZSTD_loadEntropy_intoDDict(ZSTD_DDict* ddict,
-                           ZSTD_dictContentType_e dictContentType)
-{
-    ddict->dictID = 0;
-    ddict->entropyPresent = 0;
-    if (dictContentType == ZSTD_dct_rawContent) return 0;
-
-    if (ddict->dictSize < 8) {
-        if (dictContentType == ZSTD_dct_fullDict)
-            return ERROR(dictionary_corrupted);   /* only accept specified dictionaries */
-        return 0;   /* pure content mode */
-    }
-    {   U32 const magic = MEM_readLE32(ddict->dictContent);
-        if (magic != ZSTD_MAGIC_DICTIONARY) {
-            if (dictContentType == ZSTD_dct_fullDict)
-                return ERROR(dictionary_corrupted);   /* only accept specified dictionaries */
-            return 0;   /* pure content mode */
-        }
-    }
-    ddict->dictID = MEM_readLE32((const char*)ddict->dictContent + ZSTD_FRAMEIDSIZE);
-
-    /* load entropy tables */
-    RETURN_ERROR_IF(ZSTD_isError(ZSTD_loadDEntropy(
-            &ddict->entropy, ddict->dictContent, ddict->dictSize)),
-        dictionary_corrupted);
-    ddict->entropyPresent = 1;
-    return 0;
-}
-
-
-static size_t ZSTD_initDDict_internal(ZSTD_DDict* ddict,
-                                      const void* dict, size_t dictSize,
-                                      ZSTD_dictLoadMethod_e dictLoadMethod,
-                                      ZSTD_dictContentType_e dictContentType)
-{
-    if ((dictLoadMethod == ZSTD_dlm_byRef) || (!dict) || (!dictSize)) {
-        ddict->dictBuffer = NULL;
-        ddict->dictContent = dict;
-        if (!dict) dictSize = 0;
-    } else {
-        void* const internalBuffer = ZSTD_malloc(dictSize, ddict->cMem);
-        ddict->dictBuffer = internalBuffer;
-        ddict->dictContent = internalBuffer;
-        if (!internalBuffer) return ERROR(memory_allocation);
-        memcpy(internalBuffer, dict, dictSize);
-    }
-    ddict->dictSize = dictSize;
-    ddict->entropy.hufTable[0] = (HUF_DTable)((HufLog)*0x1000001);  /* cover both little and big endian */
-
-    /* parse dictionary content */
-    FORWARD_IF_ERROR( ZSTD_loadEntropy_intoDDict(ddict, dictContentType) );
-
-    return 0;
-}
-
-ZSTD_DDict* ZSTD_createDDict_advanced(const void* dict, size_t dictSize,
-                                      ZSTD_dictLoadMethod_e dictLoadMethod,
-                                      ZSTD_dictContentType_e dictContentType,
-                                      ZSTD_customMem customMem)
-{
-    if (!customMem.customAlloc ^ !customMem.customFree) return NULL;
-
-    {   ZSTD_DDict* const ddict = (ZSTD_DDict*) ZSTD_malloc(sizeof(ZSTD_DDict), customMem);
-        if (ddict == NULL) return NULL;
-        ddict->cMem = customMem;
-        {   size_t const initResult = ZSTD_initDDict_internal(ddict,
-                                            dict, dictSize,
-                                            dictLoadMethod, dictContentType);
-            if (ZSTD_isError(initResult)) {
-                ZSTD_freeDDict(ddict);
-                return NULL;
-        }   }
-        return ddict;
-    }
-}
-
-/*! ZSTD_createDDict() :
-*   Create a digested dictionary, to start decompression without startup delay.
-*   `dict` content is copied inside DDict.
-*   Consequently, `dict` can be released after `ZSTD_DDict` creation */
-ZSTD_DDict* ZSTD_createDDict(const void* dict, size_t dictSize)
-{
-    ZSTD_customMem const allocator = { NULL, NULL, NULL };
-    return ZSTD_createDDict_advanced(dict, dictSize, ZSTD_dlm_byCopy, ZSTD_dct_auto, allocator);
-}
-
-/*! ZSTD_createDDict_byReference() :
- *  Create a digested dictionary, to start decompression without startup delay.
- *  Dictionary content is simply referenced, it will be accessed during decompression.
- *  Warning : dictBuffer must outlive DDict (DDict must be freed before dictBuffer) */
-ZSTD_DDict* ZSTD_createDDict_byReference(const void* dictBuffer, size_t dictSize)
-{
-    ZSTD_customMem const allocator = { NULL, NULL, NULL };
-    return ZSTD_createDDict_advanced(dictBuffer, dictSize, ZSTD_dlm_byRef, ZSTD_dct_auto, allocator);
-}
-
-
-const ZSTD_DDict* ZSTD_initStaticDDict(
-                                void* sBuffer, size_t sBufferSize,
-                                const void* dict, size_t dictSize,
-                                ZSTD_dictLoadMethod_e dictLoadMethod,
-                                ZSTD_dictContentType_e dictContentType)
-{
-    size_t const neededSpace = sizeof(ZSTD_DDict)
-                             + (dictLoadMethod == ZSTD_dlm_byRef ? 0 : dictSize);
-    ZSTD_DDict* const ddict = (ZSTD_DDict*)sBuffer;
-    assert(sBuffer != NULL);
-    assert(dict != NULL);
-    if ((size_t)sBuffer & 7) return NULL;   /* 8-aligned */
-    if (sBufferSize < neededSpace) return NULL;
-    if (dictLoadMethod == ZSTD_dlm_byCopy) {
-        memcpy(ddict+1, dict, dictSize);  /* local copy */
-        dict = ddict+1;
-    }
-    if (ZSTD_isError( ZSTD_initDDict_internal(ddict,
-                                              dict, dictSize,
-                                              ZSTD_dlm_byRef, dictContentType) ))
-        return NULL;
-    return ddict;
-}
-
-
-size_t ZSTD_freeDDict(ZSTD_DDict* ddict)
-{
-    if (ddict==NULL) return 0;   /* support free on NULL */
-    {   ZSTD_customMem const cMem = ddict->cMem;
-        ZSTD_free(ddict->dictBuffer, cMem);
-        ZSTD_free(ddict, cMem);
-        return 0;
-    }
-}
-
-/*! ZSTD_estimateDDictSize() :
- *  Estimate amount of memory that will be needed to create a dictionary for decompression.
- *  Note : dictionary created by reference using ZSTD_dlm_byRef are smaller */
-size_t ZSTD_estimateDDictSize(size_t dictSize, ZSTD_dictLoadMethod_e dictLoadMethod)
-{
-    return sizeof(ZSTD_DDict) + (dictLoadMethod == ZSTD_dlm_byRef ? 0 : dictSize);
-}
-
-size_t ZSTD_sizeof_DDict(const ZSTD_DDict* ddict)
-{
-    if (ddict==NULL) return 0;   /* support sizeof on NULL */
-    return sizeof(*ddict) + (ddict->dictBuffer ? ddict->dictSize : 0) ;
-}
-
-/*! ZSTD_getDictID_fromDDict() :
- *  Provides the dictID of the dictionary loaded into `ddict`.
- *  If @return == 0, the dictionary is not conformant to Zstandard specification, or empty.
- *  Non-conformant dictionaries can still be loaded, but as content-only dictionaries. */
-unsigned ZSTD_getDictID_fromDDict(const ZSTD_DDict* ddict)
-{
-    if (ddict==NULL) return 0;
-    return ZSTD_getDictID_fromDict(ddict->dictContent, ddict->dictSize);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_ddict.h b/vendor/github.com/DataDog/zstd/zstd_ddict.h
deleted file mode 100644
index 0479d11..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_ddict.h
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-#ifndef ZSTD_DDICT_H
-#define ZSTD_DDICT_H
-
-/*-*******************************************************
- *  Dependencies
- *********************************************************/
-#include <stddef.h>   /* size_t */
-#include "zstd.h"     /* ZSTD_DDict, and several public functions */
-
-
-/*-*******************************************************
- *  Interface
- *********************************************************/
-
-/* note: several prototypes are already published in `zstd.h` :
- * ZSTD_createDDict()
- * ZSTD_createDDict_byReference()
- * ZSTD_createDDict_advanced()
- * ZSTD_freeDDict()
- * ZSTD_initStaticDDict()
- * ZSTD_sizeof_DDict()
- * ZSTD_estimateDDictSize()
- * ZSTD_getDictID_fromDict()
- */
-
-const void* ZSTD_DDict_dictContent(const ZSTD_DDict* ddict);
-size_t ZSTD_DDict_dictSize(const ZSTD_DDict* ddict);
-
-void ZSTD_copyDDictParameters(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict);
-
-
-
-#endif /* ZSTD_DDICT_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_decompress.c b/vendor/github.com/DataDog/zstd/zstd_decompress.c
deleted file mode 100644
index e42872a..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_decompress.c
+++ /dev/null
@@ -1,1768 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
- * HEAPMODE :
- * Select how default decompression function ZSTD_decompress() allocates its context,
- * on stack (0), or into heap (1, default; requires malloc()).
- * Note that functions with explicit context such as ZSTD_decompressDCtx() are unaffected.
- */
-#ifndef ZSTD_HEAPMODE
-#  define ZSTD_HEAPMODE 1
-#endif
-
-/*!
-*  LEGACY_SUPPORT :
-*  if set to 1+, ZSTD_decompress() can decode older formats (v0.1+)
-*/
-#ifndef ZSTD_LEGACY_SUPPORT
-#  define ZSTD_LEGACY_SUPPORT 0
-#endif
-
-/*!
- *  MAXWINDOWSIZE_DEFAULT :
- *  maximum window size accepted by DStream __by default__.
- *  Frames requiring more memory will be rejected.
- *  It's possible to set a different limit using ZSTD_DCtx_setMaxWindowSize().
- */
-#ifndef ZSTD_MAXWINDOWSIZE_DEFAULT
-#  define ZSTD_MAXWINDOWSIZE_DEFAULT (((U32)1 << ZSTD_WINDOWLOG_LIMIT_DEFAULT) + 1)
-#endif
-
-/*!
- *  NO_FORWARD_PROGRESS_MAX :
- *  maximum allowed nb of calls to ZSTD_decompressStream()
- *  without any forward progress
- *  (defined as: no byte read from input, and no byte flushed to output)
- *  before triggering an error.
- */
-#ifndef ZSTD_NO_FORWARD_PROGRESS_MAX
-#  define ZSTD_NO_FORWARD_PROGRESS_MAX 16
-#endif
-
-
-/*-*******************************************************
-*  Dependencies
-*********************************************************/
-#include <string.h>      /* memcpy, memmove, memset */
-#include "cpu.h"         /* bmi2 */
-#include "mem.h"         /* low level memory routines */
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "zstd_internal.h"  /* blockProperties_t */
-#include "zstd_decompress_internal.h"   /* ZSTD_DCtx */
-#include "zstd_ddict.h"  /* ZSTD_DDictDictContent */
-#include "zstd_decompress_block.h"   /* ZSTD_decompressBlock_internal */
-
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT>=1)
-#  include "zstd_legacy.h"
-#endif
-
-
-/*-*************************************************************
-*   Context management
-***************************************************************/
-size_t ZSTD_sizeof_DCtx (const ZSTD_DCtx* dctx)
-{
-    if (dctx==NULL) return 0;   /* support sizeof NULL */
-    return sizeof(*dctx)
-           + ZSTD_sizeof_DDict(dctx->ddictLocal)
-           + dctx->inBuffSize + dctx->outBuffSize;
-}
-
-size_t ZSTD_estimateDCtxSize(void) { return sizeof(ZSTD_DCtx); }
-
-
-static size_t ZSTD_startingInputLength(ZSTD_format_e format)
-{
-    size_t const startingInputLength = (format==ZSTD_f_zstd1_magicless) ?
-                    ZSTD_FRAMEHEADERSIZE_PREFIX - ZSTD_FRAMEIDSIZE :
-                    ZSTD_FRAMEHEADERSIZE_PREFIX;
-    ZSTD_STATIC_ASSERT(ZSTD_FRAMEHEADERSIZE_PREFIX >= ZSTD_FRAMEIDSIZE);
-    /* only supports formats ZSTD_f_zstd1 and ZSTD_f_zstd1_magicless */
-    assert( (format == ZSTD_f_zstd1) || (format == ZSTD_f_zstd1_magicless) );
-    return startingInputLength;
-}
-
-static void ZSTD_initDCtx_internal(ZSTD_DCtx* dctx)
-{
-    dctx->format = ZSTD_f_zstd1;  /* ZSTD_decompressBegin() invokes ZSTD_startingInputLength() with argument dctx->format */
-    dctx->staticSize  = 0;
-    dctx->maxWindowSize = ZSTD_MAXWINDOWSIZE_DEFAULT;
-    dctx->ddict       = NULL;
-    dctx->ddictLocal  = NULL;
-    dctx->dictEnd     = NULL;
-    dctx->ddictIsCold = 0;
-    dctx->dictUses = ZSTD_dont_use;
-    dctx->inBuff      = NULL;
-    dctx->inBuffSize  = 0;
-    dctx->outBuffSize = 0;
-    dctx->streamStage = zdss_init;
-    dctx->legacyContext = NULL;
-    dctx->previousLegacyVersion = 0;
-    dctx->noForwardProgress = 0;
-    dctx->bmi2 = ZSTD_cpuid_bmi2(ZSTD_cpuid());
-}
-
-ZSTD_DCtx* ZSTD_initStaticDCtx(void *workspace, size_t workspaceSize)
-{
-    ZSTD_DCtx* const dctx = (ZSTD_DCtx*) workspace;
-
-    if ((size_t)workspace & 7) return NULL;  /* 8-aligned */
-    if (workspaceSize < sizeof(ZSTD_DCtx)) return NULL;  /* minimum size */
-
-    ZSTD_initDCtx_internal(dctx);
-    dctx->staticSize = workspaceSize;
-    dctx->inBuff = (char*)(dctx+1);
-    return dctx;
-}
-
-ZSTD_DCtx* ZSTD_createDCtx_advanced(ZSTD_customMem customMem)
-{
-    if (!customMem.customAlloc ^ !customMem.customFree) return NULL;
-
-    {   ZSTD_DCtx* const dctx = (ZSTD_DCtx*)ZSTD_malloc(sizeof(*dctx), customMem);
-        if (!dctx) return NULL;
-        dctx->customMem = customMem;
-        ZSTD_initDCtx_internal(dctx);
-        return dctx;
-    }
-}
-
-ZSTD_DCtx* ZSTD_createDCtx(void)
-{
-    DEBUGLOG(3, "ZSTD_createDCtx");
-    return ZSTD_createDCtx_advanced(ZSTD_defaultCMem);
-}
-
-static void ZSTD_clearDict(ZSTD_DCtx* dctx)
-{
-    ZSTD_freeDDict(dctx->ddictLocal);
-    dctx->ddictLocal = NULL;
-    dctx->ddict = NULL;
-    dctx->dictUses = ZSTD_dont_use;
-}
-
-size_t ZSTD_freeDCtx(ZSTD_DCtx* dctx)
-{
-    if (dctx==NULL) return 0;   /* support free on NULL */
-    RETURN_ERROR_IF(dctx->staticSize, memory_allocation, "not compatible with static DCtx");
-    {   ZSTD_customMem const cMem = dctx->customMem;
-        ZSTD_clearDict(dctx);
-        ZSTD_free(dctx->inBuff, cMem);
-        dctx->inBuff = NULL;
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1)
-        if (dctx->legacyContext)
-            ZSTD_freeLegacyStreamContext(dctx->legacyContext, dctx->previousLegacyVersion);
-#endif
-        ZSTD_free(dctx, cMem);
-        return 0;
-    }
-}
-
-/* no longer useful */
-void ZSTD_copyDCtx(ZSTD_DCtx* dstDCtx, const ZSTD_DCtx* srcDCtx)
-{
-    size_t const toCopy = (size_t)((char*)(&dstDCtx->inBuff) - (char*)dstDCtx);
-    memcpy(dstDCtx, srcDCtx, toCopy);  /* no need to copy workspace */
-}
-
-
-/*-*************************************************************
- *   Frame header decoding
- ***************************************************************/
-
-/*! ZSTD_isFrame() :
- *  Tells if the content of `buffer` starts with a valid Frame Identifier.
- *  Note : Frame Identifier is 4 bytes. If `size < 4`, @return will always be 0.
- *  Note 2 : Legacy Frame Identifiers are considered valid only if Legacy Support is enabled.
- *  Note 3 : Skippable Frame Identifiers are considered valid. */
-unsigned ZSTD_isFrame(const void* buffer, size_t size)
-{
-    if (size < ZSTD_FRAMEIDSIZE) return 0;
-    {   U32 const magic = MEM_readLE32(buffer);
-        if (magic == ZSTD_MAGICNUMBER) return 1;
-        if ((magic & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) return 1;
-    }
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1)
-    if (ZSTD_isLegacy(buffer, size)) return 1;
-#endif
-    return 0;
-}
-
-/** ZSTD_frameHeaderSize_internal() :
- *  srcSize must be large enough to reach header size fields.
- *  note : only works for formats ZSTD_f_zstd1 and ZSTD_f_zstd1_magicless.
- * @return : size of the Frame Header
- *           or an error code, which can be tested with ZSTD_isError() */
-static size_t ZSTD_frameHeaderSize_internal(const void* src, size_t srcSize, ZSTD_format_e format)
-{
-    size_t const minInputSize = ZSTD_startingInputLength(format);
-    RETURN_ERROR_IF(srcSize < minInputSize, srcSize_wrong);
-
-    {   BYTE const fhd = ((const BYTE*)src)[minInputSize-1];
-        U32 const dictID= fhd & 3;
-        U32 const singleSegment = (fhd >> 5) & 1;
-        U32 const fcsId = fhd >> 6;
-        return minInputSize + !singleSegment
-             + ZSTD_did_fieldSize[dictID] + ZSTD_fcs_fieldSize[fcsId]
-             + (singleSegment && !fcsId);
-    }
-}
-
-/** ZSTD_frameHeaderSize() :
- *  srcSize must be >= ZSTD_frameHeaderSize_prefix.
- * @return : size of the Frame Header,
- *           or an error code (if srcSize is too small) */
-size_t ZSTD_frameHeaderSize(const void* src, size_t srcSize)
-{
-    return ZSTD_frameHeaderSize_internal(src, srcSize, ZSTD_f_zstd1);
-}
-
-
-/** ZSTD_getFrameHeader_advanced() :
- *  decode Frame Header, or require larger `srcSize`.
- *  note : only works for formats ZSTD_f_zstd1 and ZSTD_f_zstd1_magicless
- * @return : 0, `zfhPtr` is correctly filled,
- *          >0, `srcSize` is too small, value is wanted `srcSize` amount,
- *           or an error code, which can be tested using ZSTD_isError() */
-size_t ZSTD_getFrameHeader_advanced(ZSTD_frameHeader* zfhPtr, const void* src, size_t srcSize, ZSTD_format_e format)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t const minInputSize = ZSTD_startingInputLength(format);
-
-    memset(zfhPtr, 0, sizeof(*zfhPtr));   /* not strictly necessary, but static analyzer do not understand that zfhPtr is only going to be read only if return value is zero, since they are 2 different signals */
-    if (srcSize < minInputSize) return minInputSize;
-    RETURN_ERROR_IF(src==NULL, GENERIC, "invalid parameter");
-
-    if ( (format != ZSTD_f_zstd1_magicless)
-      && (MEM_readLE32(src) != ZSTD_MAGICNUMBER) ) {
-        if ((MEM_readLE32(src) & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {
-            /* skippable frame */
-            if (srcSize < ZSTD_SKIPPABLEHEADERSIZE)
-                return ZSTD_SKIPPABLEHEADERSIZE; /* magic number + frame length */
-            memset(zfhPtr, 0, sizeof(*zfhPtr));
-            zfhPtr->frameContentSize = MEM_readLE32((const char *)src + ZSTD_FRAMEIDSIZE);
-            zfhPtr->frameType = ZSTD_skippableFrame;
-            return 0;
-        }
-        RETURN_ERROR(prefix_unknown);
-    }
-
-    /* ensure there is enough `srcSize` to fully read/decode frame header */
-    {   size_t const fhsize = ZSTD_frameHeaderSize_internal(src, srcSize, format);
-        if (srcSize < fhsize) return fhsize;
-        zfhPtr->headerSize = (U32)fhsize;
-    }
-
-    {   BYTE const fhdByte = ip[minInputSize-1];
-        size_t pos = minInputSize;
-        U32 const dictIDSizeCode = fhdByte&3;
-        U32 const checksumFlag = (fhdByte>>2)&1;
-        U32 const singleSegment = (fhdByte>>5)&1;
-        U32 const fcsID = fhdByte>>6;
-        U64 windowSize = 0;
-        U32 dictID = 0;
-        U64 frameContentSize = ZSTD_CONTENTSIZE_UNKNOWN;
-        RETURN_ERROR_IF((fhdByte & 0x08) != 0, frameParameter_unsupported,
-                        "reserved bits, must be zero");
-
-        if (!singleSegment) {
-            BYTE const wlByte = ip[pos++];
-            U32 const windowLog = (wlByte >> 3) + ZSTD_WINDOWLOG_ABSOLUTEMIN;
-            RETURN_ERROR_IF(windowLog > ZSTD_WINDOWLOG_MAX, frameParameter_windowTooLarge);
-            windowSize = (1ULL << windowLog);
-            windowSize += (windowSize >> 3) * (wlByte&7);
-        }
-        switch(dictIDSizeCode)
-        {
-            default: assert(0);  /* impossible */
-            case 0 : break;
-            case 1 : dictID = ip[pos]; pos++; break;
-            case 2 : dictID = MEM_readLE16(ip+pos); pos+=2; break;
-            case 3 : dictID = MEM_readLE32(ip+pos); pos+=4; break;
-        }
-        switch(fcsID)
-        {
-            default: assert(0);  /* impossible */
-            case 0 : if (singleSegment) frameContentSize = ip[pos]; break;
-            case 1 : frameContentSize = MEM_readLE16(ip+pos)+256; break;
-            case 2 : frameContentSize = MEM_readLE32(ip+pos); break;
-            case 3 : frameContentSize = MEM_readLE64(ip+pos); break;
-        }
-        if (singleSegment) windowSize = frameContentSize;
-
-        zfhPtr->frameType = ZSTD_frame;
-        zfhPtr->frameContentSize = frameContentSize;
-        zfhPtr->windowSize = windowSize;
-        zfhPtr->blockSizeMax = (unsigned) MIN(windowSize, ZSTD_BLOCKSIZE_MAX);
-        zfhPtr->dictID = dictID;
-        zfhPtr->checksumFlag = checksumFlag;
-    }
-    return 0;
-}
-
-/** ZSTD_getFrameHeader() :
- *  decode Frame Header, or require larger `srcSize`.
- *  note : this function does not consume input, it only reads it.
- * @return : 0, `zfhPtr` is correctly filled,
- *          >0, `srcSize` is too small, value is wanted `srcSize` amount,
- *           or an error code, which can be tested using ZSTD_isError() */
-size_t ZSTD_getFrameHeader(ZSTD_frameHeader* zfhPtr, const void* src, size_t srcSize)
-{
-    return ZSTD_getFrameHeader_advanced(zfhPtr, src, srcSize, ZSTD_f_zstd1);
-}
-
-
-/** ZSTD_getFrameContentSize() :
- *  compatible with legacy mode
- * @return : decompressed size of the single frame pointed to be `src` if known, otherwise
- *         - ZSTD_CONTENTSIZE_UNKNOWN if the size cannot be determined
- *         - ZSTD_CONTENTSIZE_ERROR if an error occurred (e.g. invalid magic number, srcSize too small) */
-unsigned long long ZSTD_getFrameContentSize(const void *src, size_t srcSize)
-{
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1)
-    if (ZSTD_isLegacy(src, srcSize)) {
-        unsigned long long const ret = ZSTD_getDecompressedSize_legacy(src, srcSize);
-        return ret == 0 ? ZSTD_CONTENTSIZE_UNKNOWN : ret;
-    }
-#endif
-    {   ZSTD_frameHeader zfh;
-        if (ZSTD_getFrameHeader(&zfh, src, srcSize) != 0)
-            return ZSTD_CONTENTSIZE_ERROR;
-        if (zfh.frameType == ZSTD_skippableFrame) {
-            return 0;
-        } else {
-            return zfh.frameContentSize;
-    }   }
-}
-
-static size_t readSkippableFrameSize(void const* src, size_t srcSize)
-{
-    size_t const skippableHeaderSize = ZSTD_SKIPPABLEHEADERSIZE;
-    U32 sizeU32;
-
-    RETURN_ERROR_IF(srcSize < ZSTD_SKIPPABLEHEADERSIZE, srcSize_wrong);
-
-    sizeU32 = MEM_readLE32((BYTE const*)src + ZSTD_FRAMEIDSIZE);
-    RETURN_ERROR_IF((U32)(sizeU32 + ZSTD_SKIPPABLEHEADERSIZE) < sizeU32,
-                    frameParameter_unsupported);
-    {
-        size_t const skippableSize = skippableHeaderSize + sizeU32;
-        RETURN_ERROR_IF(skippableSize > srcSize, srcSize_wrong);
-        return skippableSize;
-    }
-}
-
-/** ZSTD_findDecompressedSize() :
- *  compatible with legacy mode
- *  `srcSize` must be the exact length of some number of ZSTD compressed and/or
- *      skippable frames
- *  @return : decompressed size of the frames contained */
-unsigned long long ZSTD_findDecompressedSize(const void* src, size_t srcSize)
-{
-    unsigned long long totalDstSize = 0;
-
-    while (srcSize >= ZSTD_FRAMEHEADERSIZE_PREFIX) {
-        U32 const magicNumber = MEM_readLE32(src);
-
-        if ((magicNumber & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {
-            size_t const skippableSize = readSkippableFrameSize(src, srcSize);
-            if (ZSTD_isError(skippableSize)) {
-                return ZSTD_CONTENTSIZE_ERROR;
-            }
-            assert(skippableSize <= srcSize);
-
-            src = (const BYTE *)src + skippableSize;
-            srcSize -= skippableSize;
-            continue;
-        }
-
-        {   unsigned long long const ret = ZSTD_getFrameContentSize(src, srcSize);
-            if (ret >= ZSTD_CONTENTSIZE_ERROR) return ret;
-
-            /* check for overflow */
-            if (totalDstSize + ret < totalDstSize) return ZSTD_CONTENTSIZE_ERROR;
-            totalDstSize += ret;
-        }
-        {   size_t const frameSrcSize = ZSTD_findFrameCompressedSize(src, srcSize);
-            if (ZSTD_isError(frameSrcSize)) {
-                return ZSTD_CONTENTSIZE_ERROR;
-            }
-
-            src = (const BYTE *)src + frameSrcSize;
-            srcSize -= frameSrcSize;
-        }
-    }  /* while (srcSize >= ZSTD_frameHeaderSize_prefix) */
-
-    if (srcSize) return ZSTD_CONTENTSIZE_ERROR;
-
-    return totalDstSize;
-}
-
-/** ZSTD_getDecompressedSize() :
- *  compatible with legacy mode
- * @return : decompressed size if known, 0 otherwise
-             note : 0 can mean any of the following :
-                   - frame content is empty
-                   - decompressed size field is not present in frame header
-                   - frame header unknown / not supported
-                   - frame header not complete (`srcSize` too small) */
-unsigned long long ZSTD_getDecompressedSize(const void* src, size_t srcSize)
-{
-    unsigned long long const ret = ZSTD_getFrameContentSize(src, srcSize);
-    ZSTD_STATIC_ASSERT(ZSTD_CONTENTSIZE_ERROR < ZSTD_CONTENTSIZE_UNKNOWN);
-    return (ret >= ZSTD_CONTENTSIZE_ERROR) ? 0 : ret;
-}
-
-
-/** ZSTD_decodeFrameHeader() :
- * `headerSize` must be the size provided by ZSTD_frameHeaderSize().
- * @return : 0 if success, or an error code, which can be tested using ZSTD_isError() */
-static size_t ZSTD_decodeFrameHeader(ZSTD_DCtx* dctx, const void* src, size_t headerSize)
-{
-    size_t const result = ZSTD_getFrameHeader_advanced(&(dctx->fParams), src, headerSize, dctx->format);
-    if (ZSTD_isError(result)) return result;    /* invalid header */
-    RETURN_ERROR_IF(result>0, srcSize_wrong, "headerSize too small");
-#ifndef FUZZING_BUILD_MODE_UNSAFE_FOR_PRODUCTION
-    /* Skip the dictID check in fuzzing mode, because it makes the search
-     * harder.
-     */
-    RETURN_ERROR_IF(dctx->fParams.dictID && (dctx->dictID != dctx->fParams.dictID),
-                    dictionary_wrong);
-#endif
-    if (dctx->fParams.checksumFlag) XXH64_reset(&dctx->xxhState, 0);
-    return 0;
-}
-
-static ZSTD_frameSizeInfo ZSTD_errorFrameSizeInfo(size_t ret)
-{
-    ZSTD_frameSizeInfo frameSizeInfo;
-    frameSizeInfo.compressedSize = ret;
-    frameSizeInfo.decompressedBound = ZSTD_CONTENTSIZE_ERROR;
-    return frameSizeInfo;
-}
-
-static ZSTD_frameSizeInfo ZSTD_findFrameSizeInfo(const void* src, size_t srcSize)
-{
-    ZSTD_frameSizeInfo frameSizeInfo;
-    memset(&frameSizeInfo, 0, sizeof(ZSTD_frameSizeInfo));
-
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1)
-    if (ZSTD_isLegacy(src, srcSize))
-        return ZSTD_findFrameSizeInfoLegacy(src, srcSize);
-#endif
-
-    if ((srcSize >= ZSTD_SKIPPABLEHEADERSIZE)
-        && (MEM_readLE32(src) & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {
-        frameSizeInfo.compressedSize = readSkippableFrameSize(src, srcSize);
-        assert(ZSTD_isError(frameSizeInfo.compressedSize) ||
-               frameSizeInfo.compressedSize <= srcSize);
-        return frameSizeInfo;
-    } else {
-        const BYTE* ip = (const BYTE*)src;
-        const BYTE* const ipstart = ip;
-        size_t remainingSize = srcSize;
-        size_t nbBlocks = 0;
-        ZSTD_frameHeader zfh;
-
-        /* Extract Frame Header */
-        {   size_t const ret = ZSTD_getFrameHeader(&zfh, src, srcSize);
-            if (ZSTD_isError(ret))
-                return ZSTD_errorFrameSizeInfo(ret);
-            if (ret > 0)
-                return ZSTD_errorFrameSizeInfo(ERROR(srcSize_wrong));
-        }
-
-        ip += zfh.headerSize;
-        remainingSize -= zfh.headerSize;
-
-        /* Iterate over each block */
-        while (1) {
-            blockProperties_t blockProperties;
-            size_t const cBlockSize = ZSTD_getcBlockSize(ip, remainingSize, &blockProperties);
-            if (ZSTD_isError(cBlockSize))
-                return ZSTD_errorFrameSizeInfo(cBlockSize);
-
-            if (ZSTD_blockHeaderSize + cBlockSize > remainingSize)
-                return ZSTD_errorFrameSizeInfo(ERROR(srcSize_wrong));
-
-            ip += ZSTD_blockHeaderSize + cBlockSize;
-            remainingSize -= ZSTD_blockHeaderSize + cBlockSize;
-            nbBlocks++;
-
-            if (blockProperties.lastBlock) break;
-        }
-
-        /* Final frame content checksum */
-        if (zfh.checksumFlag) {
-            if (remainingSize < 4)
-                return ZSTD_errorFrameSizeInfo(ERROR(srcSize_wrong));
-            ip += 4;
-        }
-
-        frameSizeInfo.compressedSize = ip - ipstart;
-        frameSizeInfo.decompressedBound = (zfh.frameContentSize != ZSTD_CONTENTSIZE_UNKNOWN)
-                                        ? zfh.frameContentSize
-                                        : nbBlocks * zfh.blockSizeMax;
-        return frameSizeInfo;
-    }
-}
-
-/** ZSTD_findFrameCompressedSize() :
- *  compatible with legacy mode
- *  `src` must point to the start of a ZSTD frame, ZSTD legacy frame, or skippable frame
- *  `srcSize` must be at least as large as the frame contained
- *  @return : the compressed size of the frame starting at `src` */
-size_t ZSTD_findFrameCompressedSize(const void *src, size_t srcSize)
-{
-    ZSTD_frameSizeInfo const frameSizeInfo = ZSTD_findFrameSizeInfo(src, srcSize);
-    return frameSizeInfo.compressedSize;
-}
-
-/** ZSTD_decompressBound() :
- *  compatible with legacy mode
- *  `src` must point to the start of a ZSTD frame or a skippeable frame
- *  `srcSize` must be at least as large as the frame contained
- *  @return : the maximum decompressed size of the compressed source
- */
-unsigned long long ZSTD_decompressBound(const void* src, size_t srcSize)
-{
-    unsigned long long bound = 0;
-    /* Iterate over each frame */
-    while (srcSize > 0) {
-        ZSTD_frameSizeInfo const frameSizeInfo = ZSTD_findFrameSizeInfo(src, srcSize);
-        size_t const compressedSize = frameSizeInfo.compressedSize;
-        unsigned long long const decompressedBound = frameSizeInfo.decompressedBound;
-        if (ZSTD_isError(compressedSize) || decompressedBound == ZSTD_CONTENTSIZE_ERROR)
-            return ZSTD_CONTENTSIZE_ERROR;
-        assert(srcSize >= compressedSize);
-        src = (const BYTE*)src + compressedSize;
-        srcSize -= compressedSize;
-        bound += decompressedBound;
-    }
-    return bound;
-}
-
-
-/*-*************************************************************
- *   Frame decoding
- ***************************************************************/
-
-
-void ZSTD_checkContinuity(ZSTD_DCtx* dctx, const void* dst)
-{
-    if (dst != dctx->previousDstEnd) {   /* not contiguous */
-        dctx->dictEnd = dctx->previousDstEnd;
-        dctx->virtualStart = (const char*)dst - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->prefixStart));
-        dctx->prefixStart = dst;
-        dctx->previousDstEnd = dst;
-    }
-}
-
-/** ZSTD_insertBlock() :
-    insert `src` block into `dctx` history. Useful to track uncompressed blocks. */
-size_t ZSTD_insertBlock(ZSTD_DCtx* dctx, const void* blockStart, size_t blockSize)
-{
-    ZSTD_checkContinuity(dctx, blockStart);
-    dctx->previousDstEnd = (const char*)blockStart + blockSize;
-    return blockSize;
-}
-
-
-static size_t ZSTD_copyRawBlock(void* dst, size_t dstCapacity,
-                          const void* src, size_t srcSize)
-{
-    DEBUGLOG(5, "ZSTD_copyRawBlock");
-    if (dst == NULL) {
-        if (srcSize == 0) return 0;
-        RETURN_ERROR(dstBuffer_null);
-    }
-    RETURN_ERROR_IF(srcSize > dstCapacity, dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-static size_t ZSTD_setRleBlock(void* dst, size_t dstCapacity,
-                               BYTE b,
-                               size_t regenSize)
-{
-    if (dst == NULL) {
-        if (regenSize == 0) return 0;
-        RETURN_ERROR(dstBuffer_null);
-    }
-    RETURN_ERROR_IF(regenSize > dstCapacity, dstSize_tooSmall);
-    memset(dst, b, regenSize);
-    return regenSize;
-}
-
-
-/*! ZSTD_decompressFrame() :
- * @dctx must be properly initialized
- *  will update *srcPtr and *srcSizePtr,
- *  to make *srcPtr progress by one frame. */
-static size_t ZSTD_decompressFrame(ZSTD_DCtx* dctx,
-                                   void* dst, size_t dstCapacity,
-                             const void** srcPtr, size_t *srcSizePtr)
-{
-    const BYTE* ip = (const BYTE*)(*srcPtr);
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + dstCapacity;
-    BYTE* op = ostart;
-    size_t remainingSrcSize = *srcSizePtr;
-
-    DEBUGLOG(4, "ZSTD_decompressFrame (srcSize:%i)", (int)*srcSizePtr);
-
-    /* check */
-    RETURN_ERROR_IF(
-        remainingSrcSize < ZSTD_FRAMEHEADERSIZE_MIN+ZSTD_blockHeaderSize,
-        srcSize_wrong);
-
-    /* Frame Header */
-    {   size_t const frameHeaderSize = ZSTD_frameHeaderSize(ip, ZSTD_FRAMEHEADERSIZE_PREFIX);
-        if (ZSTD_isError(frameHeaderSize)) return frameHeaderSize;
-        RETURN_ERROR_IF(remainingSrcSize < frameHeaderSize+ZSTD_blockHeaderSize,
-                        srcSize_wrong);
-        FORWARD_IF_ERROR( ZSTD_decodeFrameHeader(dctx, ip, frameHeaderSize) );
-        ip += frameHeaderSize; remainingSrcSize -= frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1) {
-        size_t decodedSize;
-        blockProperties_t blockProperties;
-        size_t const cBlockSize = ZSTD_getcBlockSize(ip, remainingSrcSize, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSrcSize -= ZSTD_blockHeaderSize;
-        RETURN_ERROR_IF(cBlockSize > remainingSrcSize, srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTD_decompressBlock_internal(dctx, op, oend-op, ip, cBlockSize, /* frame */ 1);
-            break;
-        case bt_raw :
-            decodedSize = ZSTD_copyRawBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            decodedSize = ZSTD_setRleBlock(op, oend-op, *ip, blockProperties.origSize);
-            break;
-        case bt_reserved :
-        default:
-            RETURN_ERROR(corruption_detected);
-        }
-
-        if (ZSTD_isError(decodedSize)) return decodedSize;
-        if (dctx->fParams.checksumFlag)
-            XXH64_update(&dctx->xxhState, op, decodedSize);
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSrcSize -= cBlockSize;
-        if (blockProperties.lastBlock) break;
-    }
-
-    if (dctx->fParams.frameContentSize != ZSTD_CONTENTSIZE_UNKNOWN) {
-        RETURN_ERROR_IF((U64)(op-ostart) != dctx->fParams.frameContentSize,
-                        corruption_detected);
-    }
-    if (dctx->fParams.checksumFlag) { /* Frame content checksum verification */
-        U32 const checkCalc = (U32)XXH64_digest(&dctx->xxhState);
-        U32 checkRead;
-        RETURN_ERROR_IF(remainingSrcSize<4, checksum_wrong);
-        checkRead = MEM_readLE32(ip);
-        RETURN_ERROR_IF(checkRead != checkCalc, checksum_wrong);
-        ip += 4;
-        remainingSrcSize -= 4;
-    }
-
-    /* Allow caller to get size read */
-    *srcPtr = ip;
-    *srcSizePtr = remainingSrcSize;
-    return op-ostart;
-}
-
-static size_t ZSTD_decompressMultiFrame(ZSTD_DCtx* dctx,
-                                        void* dst, size_t dstCapacity,
-                                  const void* src, size_t srcSize,
-                                  const void* dict, size_t dictSize,
-                                  const ZSTD_DDict* ddict)
-{
-    void* const dststart = dst;
-    int moreThan1Frame = 0;
-
-    DEBUGLOG(5, "ZSTD_decompressMultiFrame");
-    assert(dict==NULL || ddict==NULL);  /* either dict or ddict set, not both */
-
-    if (ddict) {
-        dict = ZSTD_DDict_dictContent(ddict);
-        dictSize = ZSTD_DDict_dictSize(ddict);
-    }
-
-    while (srcSize >= ZSTD_FRAMEHEADERSIZE_PREFIX) {
-
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT >= 1)
-        if (ZSTD_isLegacy(src, srcSize)) {
-            size_t decodedSize;
-            size_t const frameSize = ZSTD_findFrameCompressedSizeLegacy(src, srcSize);
-            if (ZSTD_isError(frameSize)) return frameSize;
-            RETURN_ERROR_IF(dctx->staticSize, memory_allocation,
-                "legacy support is not compatible with static dctx");
-
-            decodedSize = ZSTD_decompressLegacy(dst, dstCapacity, src, frameSize, dict, dictSize);
-            if (ZSTD_isError(decodedSize)) return decodedSize;
-
-            assert(decodedSize <=- dstCapacity);
-            dst = (BYTE*)dst + decodedSize;
-            dstCapacity -= decodedSize;
-
-            src = (const BYTE*)src + frameSize;
-            srcSize -= frameSize;
-
-            continue;
-        }
-#endif
-
-        {   U32 const magicNumber = MEM_readLE32(src);
-            DEBUGLOG(4, "reading magic number %08X (expecting %08X)",
-                        (unsigned)magicNumber, ZSTD_MAGICNUMBER);
-            if ((magicNumber & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {
-                size_t const skippableSize = readSkippableFrameSize(src, srcSize);
-                FORWARD_IF_ERROR(skippableSize);
-                assert(skippableSize <= srcSize);
-
-                src = (const BYTE *)src + skippableSize;
-                srcSize -= skippableSize;
-                continue;
-        }   }
-
-        if (ddict) {
-            /* we were called from ZSTD_decompress_usingDDict */
-            FORWARD_IF_ERROR(ZSTD_decompressBegin_usingDDict(dctx, ddict));
-        } else {
-            /* this will initialize correctly with no dict if dict == NULL, so
-             * use this in all cases but ddict */
-            FORWARD_IF_ERROR(ZSTD_decompressBegin_usingDict(dctx, dict, dictSize));
-        }
-        ZSTD_checkContinuity(dctx, dst);
-
-        {   const size_t res = ZSTD_decompressFrame(dctx, dst, dstCapacity,
-                                                    &src, &srcSize);
-            RETURN_ERROR_IF(
-                (ZSTD_getErrorCode(res) == ZSTD_error_prefix_unknown)
-             && (moreThan1Frame==1),
-                srcSize_wrong,
-                "at least one frame successfully completed, but following "
-                "bytes are garbage: it's more likely to be a srcSize error, "
-                "specifying more bytes than compressed size of frame(s). This "
-                "error message replaces ERROR(prefix_unknown), which would be "
-                "confusing, as the first header is actually correct. Note that "
-                "one could be unlucky, it might be a corruption error instead, "
-                "happening right at the place where we expect zstd magic "
-                "bytes. But this is _much_ less likely than a srcSize field "
-                "error.");
-            if (ZSTD_isError(res)) return res;
-            assert(res <= dstCapacity);
-            dst = (BYTE*)dst + res;
-            dstCapacity -= res;
-        }
-        moreThan1Frame = 1;
-    }  /* while (srcSize >= ZSTD_frameHeaderSize_prefix) */
-
-    RETURN_ERROR_IF(srcSize, srcSize_wrong, "input not entirely consumed");
-
-    return (BYTE*)dst - (BYTE*)dststart;
-}
-
-size_t ZSTD_decompress_usingDict(ZSTD_DCtx* dctx,
-                                 void* dst, size_t dstCapacity,
-                           const void* src, size_t srcSize,
-                           const void* dict, size_t dictSize)
-{
-    return ZSTD_decompressMultiFrame(dctx, dst, dstCapacity, src, srcSize, dict, dictSize, NULL);
-}
-
-
-static ZSTD_DDict const* ZSTD_getDDict(ZSTD_DCtx* dctx)
-{
-    switch (dctx->dictUses) {
-    default:
-        assert(0 /* Impossible */);
-        /* fall-through */
-    case ZSTD_dont_use:
-        ZSTD_clearDict(dctx);
-        return NULL;
-    case ZSTD_use_indefinitely:
-        return dctx->ddict;
-    case ZSTD_use_once:
-        dctx->dictUses = ZSTD_dont_use;
-        return dctx->ddict;
-    }
-}
-
-size_t ZSTD_decompressDCtx(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    return ZSTD_decompress_usingDDict(dctx, dst, dstCapacity, src, srcSize, ZSTD_getDDict(dctx));
-}
-
-
-size_t ZSTD_decompress(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-#if defined(ZSTD_HEAPMODE) && (ZSTD_HEAPMODE>=1)
-    size_t regenSize;
-    ZSTD_DCtx* const dctx = ZSTD_createDCtx();
-    RETURN_ERROR_IF(dctx==NULL, memory_allocation);
-    regenSize = ZSTD_decompressDCtx(dctx, dst, dstCapacity, src, srcSize);
-    ZSTD_freeDCtx(dctx);
-    return regenSize;
-#else   /* stack mode */
-    ZSTD_DCtx dctx;
-    ZSTD_initDCtx_internal(&dctx);
-    return ZSTD_decompressDCtx(&dctx, dst, dstCapacity, src, srcSize);
-#endif
-}
-
-
-/*-**************************************
-*   Advanced Streaming Decompression API
-*   Bufferless and synchronous
-****************************************/
-size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx) { return dctx->expected; }
-
-ZSTD_nextInputType_e ZSTD_nextInputType(ZSTD_DCtx* dctx) {
-    switch(dctx->stage)
-    {
-    default:   /* should not happen */
-        assert(0);
-    case ZSTDds_getFrameHeaderSize:
-    case ZSTDds_decodeFrameHeader:
-        return ZSTDnit_frameHeader;
-    case ZSTDds_decodeBlockHeader:
-        return ZSTDnit_blockHeader;
-    case ZSTDds_decompressBlock:
-        return ZSTDnit_block;
-    case ZSTDds_decompressLastBlock:
-        return ZSTDnit_lastBlock;
-    case ZSTDds_checkChecksum:
-        return ZSTDnit_checksum;
-    case ZSTDds_decodeSkippableHeader:
-    case ZSTDds_skipFrame:
-        return ZSTDnit_skippableFrame;
-    }
-}
-
-static int ZSTD_isSkipFrame(ZSTD_DCtx* dctx) { return dctx->stage == ZSTDds_skipFrame; }
-
-/** ZSTD_decompressContinue() :
- *  srcSize : must be the exact nb of bytes expected (see ZSTD_nextSrcSizeToDecompress())
- *  @return : nb of bytes generated into `dst` (necessarily <= `dstCapacity)
- *            or an error code, which can be tested using ZSTD_isError() */
-size_t ZSTD_decompressContinue(ZSTD_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    DEBUGLOG(5, "ZSTD_decompressContinue (srcSize:%u)", (unsigned)srcSize);
-    /* Sanity check */
-    RETURN_ERROR_IF(srcSize != dctx->expected, srcSize_wrong, "not allowed");
-    if (dstCapacity) ZSTD_checkContinuity(dctx, dst);
-
-    switch (dctx->stage)
-    {
-    case ZSTDds_getFrameHeaderSize :
-        assert(src != NULL);
-        if (dctx->format == ZSTD_f_zstd1) {  /* allows header */
-            assert(srcSize >= ZSTD_FRAMEIDSIZE);  /* to read skippable magic number */
-            if ((MEM_readLE32(src) & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {        /* skippable frame */
-                memcpy(dctx->headerBuffer, src, srcSize);
-                dctx->expected = ZSTD_SKIPPABLEHEADERSIZE - srcSize;  /* remaining to load to get full skippable frame header */
-                dctx->stage = ZSTDds_decodeSkippableHeader;
-                return 0;
-        }   }
-        dctx->headerSize = ZSTD_frameHeaderSize_internal(src, srcSize, dctx->format);
-        if (ZSTD_isError(dctx->headerSize)) return dctx->headerSize;
-        memcpy(dctx->headerBuffer, src, srcSize);
-        dctx->expected = dctx->headerSize - srcSize;
-        dctx->stage = ZSTDds_decodeFrameHeader;
-        return 0;
-
-    case ZSTDds_decodeFrameHeader:
-        assert(src != NULL);
-        memcpy(dctx->headerBuffer + (dctx->headerSize - srcSize), src, srcSize);
-        FORWARD_IF_ERROR(ZSTD_decodeFrameHeader(dctx, dctx->headerBuffer, dctx->headerSize));
-        dctx->expected = ZSTD_blockHeaderSize;
-        dctx->stage = ZSTDds_decodeBlockHeader;
-        return 0;
-
-    case ZSTDds_decodeBlockHeader:
-        {   blockProperties_t bp;
-            size_t const cBlockSize = ZSTD_getcBlockSize(src, ZSTD_blockHeaderSize, &bp);
-            if (ZSTD_isError(cBlockSize)) return cBlockSize;
-            dctx->expected = cBlockSize;
-            dctx->bType = bp.blockType;
-            dctx->rleSize = bp.origSize;
-            if (cBlockSize) {
-                dctx->stage = bp.lastBlock ? ZSTDds_decompressLastBlock : ZSTDds_decompressBlock;
-                return 0;
-            }
-            /* empty block */
-            if (bp.lastBlock) {
-                if (dctx->fParams.checksumFlag) {
-                    dctx->expected = 4;
-                    dctx->stage = ZSTDds_checkChecksum;
-                } else {
-                    dctx->expected = 0; /* end of frame */
-                    dctx->stage = ZSTDds_getFrameHeaderSize;
-                }
-            } else {
-                dctx->expected = ZSTD_blockHeaderSize;  /* jump to next header */
-                dctx->stage = ZSTDds_decodeBlockHeader;
-            }
-            return 0;
-        }
-
-    case ZSTDds_decompressLastBlock:
-    case ZSTDds_decompressBlock:
-        DEBUGLOG(5, "ZSTD_decompressContinue: case ZSTDds_decompressBlock");
-        {   size_t rSize;
-            switch(dctx->bType)
-            {
-            case bt_compressed:
-                DEBUGLOG(5, "ZSTD_decompressContinue: case bt_compressed");
-                rSize = ZSTD_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize, /* frame */ 1);
-                break;
-            case bt_raw :
-                rSize = ZSTD_copyRawBlock(dst, dstCapacity, src, srcSize);
-                break;
-            case bt_rle :
-                rSize = ZSTD_setRleBlock(dst, dstCapacity, *(const BYTE*)src, dctx->rleSize);
-                break;
-            case bt_reserved :   /* should never happen */
-            default:
-                RETURN_ERROR(corruption_detected);
-            }
-            if (ZSTD_isError(rSize)) return rSize;
-            DEBUGLOG(5, "ZSTD_decompressContinue: decoded size from block : %u", (unsigned)rSize);
-            dctx->decodedSize += rSize;
-            if (dctx->fParams.checksumFlag) XXH64_update(&dctx->xxhState, dst, rSize);
-
-            if (dctx->stage == ZSTDds_decompressLastBlock) {   /* end of frame */
-                DEBUGLOG(4, "ZSTD_decompressContinue: decoded size from frame : %u", (unsigned)dctx->decodedSize);
-                RETURN_ERROR_IF(
-                    dctx->fParams.frameContentSize != ZSTD_CONTENTSIZE_UNKNOWN
-                 && dctx->decodedSize != dctx->fParams.frameContentSize,
-                    corruption_detected);
-                if (dctx->fParams.checksumFlag) {  /* another round for frame checksum */
-                    dctx->expected = 4;
-                    dctx->stage = ZSTDds_checkChecksum;
-                } else {
-                    dctx->expected = 0;   /* ends here */
-                    dctx->stage = ZSTDds_getFrameHeaderSize;
-                }
-            } else {
-                dctx->stage = ZSTDds_decodeBlockHeader;
-                dctx->expected = ZSTD_blockHeaderSize;
-                dctx->previousDstEnd = (char*)dst + rSize;
-            }
-            return rSize;
-        }
-
-    case ZSTDds_checkChecksum:
-        assert(srcSize == 4);  /* guaranteed by dctx->expected */
-        {   U32 const h32 = (U32)XXH64_digest(&dctx->xxhState);
-            U32 const check32 = MEM_readLE32(src);
-            DEBUGLOG(4, "ZSTD_decompressContinue: checksum : calculated %08X :: %08X read", (unsigned)h32, (unsigned)check32);
-            RETURN_ERROR_IF(check32 != h32, checksum_wrong);
-            dctx->expected = 0;
-            dctx->stage = ZSTDds_getFrameHeaderSize;
-            return 0;
-        }
-
-    case ZSTDds_decodeSkippableHeader:
-        assert(src != NULL);
-        assert(srcSize <= ZSTD_SKIPPABLEHEADERSIZE);
-        memcpy(dctx->headerBuffer + (ZSTD_SKIPPABLEHEADERSIZE - srcSize), src, srcSize);   /* complete skippable header */
-        dctx->expected = MEM_readLE32(dctx->headerBuffer + ZSTD_FRAMEIDSIZE);   /* note : dctx->expected can grow seriously large, beyond local buffer size */
-        dctx->stage = ZSTDds_skipFrame;
-        return 0;
-
-    case ZSTDds_skipFrame:
-        dctx->expected = 0;
-        dctx->stage = ZSTDds_getFrameHeaderSize;
-        return 0;
-
-    default:
-        assert(0);   /* impossible */
-        RETURN_ERROR(GENERIC);   /* some compiler require default to do something */
-    }
-}
-
-
-static size_t ZSTD_refDictContent(ZSTD_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    dctx->dictEnd = dctx->previousDstEnd;
-    dctx->virtualStart = (const char*)dict - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->prefixStart));
-    dctx->prefixStart = dict;
-    dctx->previousDstEnd = (const char*)dict + dictSize;
-    return 0;
-}
-
-/*! ZSTD_loadDEntropy() :
- *  dict : must point at beginning of a valid zstd dictionary.
- * @return : size of entropy tables read */
-size_t
-ZSTD_loadDEntropy(ZSTD_entropyDTables_t* entropy,
-                  const void* const dict, size_t const dictSize)
-{
-    const BYTE* dictPtr = (const BYTE*)dict;
-    const BYTE* const dictEnd = dictPtr + dictSize;
-
-    RETURN_ERROR_IF(dictSize <= 8, dictionary_corrupted);
-    assert(MEM_readLE32(dict) == ZSTD_MAGIC_DICTIONARY);   /* dict must be valid */
-    dictPtr += 8;   /* skip header = magic + dictID */
-
-    ZSTD_STATIC_ASSERT(offsetof(ZSTD_entropyDTables_t, OFTable) == offsetof(ZSTD_entropyDTables_t, LLTable) + sizeof(entropy->LLTable));
-    ZSTD_STATIC_ASSERT(offsetof(ZSTD_entropyDTables_t, MLTable) == offsetof(ZSTD_entropyDTables_t, OFTable) + sizeof(entropy->OFTable));
-    ZSTD_STATIC_ASSERT(sizeof(entropy->LLTable) + sizeof(entropy->OFTable) + sizeof(entropy->MLTable) >= HUF_DECOMPRESS_WORKSPACE_SIZE);
-    {   void* const workspace = &entropy->LLTable;   /* use fse tables as temporary workspace; implies fse tables are grouped together */
-        size_t const workspaceSize = sizeof(entropy->LLTable) + sizeof(entropy->OFTable) + sizeof(entropy->MLTable);
-#ifdef HUF_FORCE_DECOMPRESS_X1
-        /* in minimal huffman, we always use X1 variants */
-        size_t const hSize = HUF_readDTableX1_wksp(entropy->hufTable,
-                                                dictPtr, dictEnd - dictPtr,
-                                                workspace, workspaceSize);
-#else
-        size_t const hSize = HUF_readDTableX2_wksp(entropy->hufTable,
-                                                dictPtr, dictEnd - dictPtr,
-                                                workspace, workspaceSize);
-#endif
-        RETURN_ERROR_IF(HUF_isError(hSize), dictionary_corrupted);
-        dictPtr += hSize;
-    }
-
-    {   short offcodeNCount[MaxOff+1];
-        unsigned offcodeMaxValue = MaxOff, offcodeLog;
-        size_t const offcodeHeaderSize = FSE_readNCount(offcodeNCount, &offcodeMaxValue, &offcodeLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(offcodeHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(offcodeMaxValue > MaxOff, dictionary_corrupted);
-        RETURN_ERROR_IF(offcodeLog > OffFSELog, dictionary_corrupted);
-        ZSTD_buildFSETable( entropy->OFTable,
-                            offcodeNCount, offcodeMaxValue,
-                            OF_base, OF_bits,
-                            offcodeLog);
-        dictPtr += offcodeHeaderSize;
-    }
-
-    {   short matchlengthNCount[MaxML+1];
-        unsigned matchlengthMaxValue = MaxML, matchlengthLog;
-        size_t const matchlengthHeaderSize = FSE_readNCount(matchlengthNCount, &matchlengthMaxValue, &matchlengthLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(matchlengthHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(matchlengthMaxValue > MaxML, dictionary_corrupted);
-        RETURN_ERROR_IF(matchlengthLog > MLFSELog, dictionary_corrupted);
-        ZSTD_buildFSETable( entropy->MLTable,
-                            matchlengthNCount, matchlengthMaxValue,
-                            ML_base, ML_bits,
-                            matchlengthLog);
-        dictPtr += matchlengthHeaderSize;
-    }
-
-    {   short litlengthNCount[MaxLL+1];
-        unsigned litlengthMaxValue = MaxLL, litlengthLog;
-        size_t const litlengthHeaderSize = FSE_readNCount(litlengthNCount, &litlengthMaxValue, &litlengthLog, dictPtr, dictEnd-dictPtr);
-        RETURN_ERROR_IF(FSE_isError(litlengthHeaderSize), dictionary_corrupted);
-        RETURN_ERROR_IF(litlengthMaxValue > MaxLL, dictionary_corrupted);
-        RETURN_ERROR_IF(litlengthLog > LLFSELog, dictionary_corrupted);
-        ZSTD_buildFSETable( entropy->LLTable,
-                            litlengthNCount, litlengthMaxValue,
-                            LL_base, LL_bits,
-                            litlengthLog);
-        dictPtr += litlengthHeaderSize;
-    }
-
-    RETURN_ERROR_IF(dictPtr+12 > dictEnd, dictionary_corrupted);
-    {   int i;
-        size_t const dictContentSize = (size_t)(dictEnd - (dictPtr+12));
-        for (i=0; i<3; i++) {
-            U32 const rep = MEM_readLE32(dictPtr); dictPtr += 4;
-            RETURN_ERROR_IF(rep==0 || rep >= dictContentSize,
-                            dictionary_corrupted);
-            entropy->rep[i] = rep;
-    }   }
-
-    return dictPtr - (const BYTE*)dict;
-}
-
-static size_t ZSTD_decompress_insertDictionary(ZSTD_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    if (dictSize < 8) return ZSTD_refDictContent(dctx, dict, dictSize);
-    {   U32 const magic = MEM_readLE32(dict);
-        if (magic != ZSTD_MAGIC_DICTIONARY) {
-            return ZSTD_refDictContent(dctx, dict, dictSize);   /* pure content mode */
-    }   }
-    dctx->dictID = MEM_readLE32((const char*)dict + ZSTD_FRAMEIDSIZE);
-
-    /* load entropy tables */
-    {   size_t const eSize = ZSTD_loadDEntropy(&dctx->entropy, dict, dictSize);
-        RETURN_ERROR_IF(ZSTD_isError(eSize), dictionary_corrupted);
-        dict = (const char*)dict + eSize;
-        dictSize -= eSize;
-    }
-    dctx->litEntropy = dctx->fseEntropy = 1;
-
-    /* reference dictionary content */
-    return ZSTD_refDictContent(dctx, dict, dictSize);
-}
-
-size_t ZSTD_decompressBegin(ZSTD_DCtx* dctx)
-{
-    assert(dctx != NULL);
-    dctx->expected = ZSTD_startingInputLength(dctx->format);  /* dctx->format must be properly set */
-    dctx->stage = ZSTDds_getFrameHeaderSize;
-    dctx->decodedSize = 0;
-    dctx->previousDstEnd = NULL;
-    dctx->prefixStart = NULL;
-    dctx->virtualStart = NULL;
-    dctx->dictEnd = NULL;
-    dctx->entropy.hufTable[0] = (HUF_DTable)((HufLog)*0x1000001);  /* cover both little and big endian */
-    dctx->litEntropy = dctx->fseEntropy = 0;
-    dctx->dictID = 0;
-    ZSTD_STATIC_ASSERT(sizeof(dctx->entropy.rep) == sizeof(repStartValue));
-    memcpy(dctx->entropy.rep, repStartValue, sizeof(repStartValue));  /* initial repcodes */
-    dctx->LLTptr = dctx->entropy.LLTable;
-    dctx->MLTptr = dctx->entropy.MLTable;
-    dctx->OFTptr = dctx->entropy.OFTable;
-    dctx->HUFptr = dctx->entropy.hufTable;
-    return 0;
-}
-
-size_t ZSTD_decompressBegin_usingDict(ZSTD_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    FORWARD_IF_ERROR( ZSTD_decompressBegin(dctx) );
-    if (dict && dictSize)
-        RETURN_ERROR_IF(
-            ZSTD_isError(ZSTD_decompress_insertDictionary(dctx, dict, dictSize)),
-            dictionary_corrupted);
-    return 0;
-}
-
-
-/* ======   ZSTD_DDict   ====== */
-
-size_t ZSTD_decompressBegin_usingDDict(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict)
-{
-    DEBUGLOG(4, "ZSTD_decompressBegin_usingDDict");
-    assert(dctx != NULL);
-    if (ddict) {
-        const char* const dictStart = (const char*)ZSTD_DDict_dictContent(ddict);
-        size_t const dictSize = ZSTD_DDict_dictSize(ddict);
-        const void* const dictEnd = dictStart + dictSize;
-        dctx->ddictIsCold = (dctx->dictEnd != dictEnd);
-        DEBUGLOG(4, "DDict is %s",
-                    dctx->ddictIsCold ? "~cold~" : "hot!");
-    }
-    FORWARD_IF_ERROR( ZSTD_decompressBegin(dctx) );
-    if (ddict) {   /* NULL ddict is equivalent to no dictionary */
-        ZSTD_copyDDictParameters(dctx, ddict);
-    }
-    return 0;
-}
-
-/*! ZSTD_getDictID_fromDict() :
- *  Provides the dictID stored within dictionary.
- *  if @return == 0, the dictionary is not conformant with Zstandard specification.
- *  It can still be loaded, but as a content-only dictionary. */
-unsigned ZSTD_getDictID_fromDict(const void* dict, size_t dictSize)
-{
-    if (dictSize < 8) return 0;
-    if (MEM_readLE32(dict) != ZSTD_MAGIC_DICTIONARY) return 0;
-    return MEM_readLE32((const char*)dict + ZSTD_FRAMEIDSIZE);
-}
-
-/*! ZSTD_getDictID_fromFrame() :
- *  Provides the dictID required to decompress frame stored within `src`.
- *  If @return == 0, the dictID could not be decoded.
- *  This could for one of the following reasons :
- *  - The frame does not require a dictionary (most common case).
- *  - The frame was built with dictID intentionally removed.
- *    Needed dictionary is a hidden information.
- *    Note : this use case also happens when using a non-conformant dictionary.
- *  - `srcSize` is too small, and as a result, frame header could not be decoded.
- *    Note : possible if `srcSize < ZSTD_FRAMEHEADERSIZE_MAX`.
- *  - This is not a Zstandard frame.
- *  When identifying the exact failure cause, it's possible to use
- *  ZSTD_getFrameHeader(), which will provide a more precise error code. */
-unsigned ZSTD_getDictID_fromFrame(const void* src, size_t srcSize)
-{
-    ZSTD_frameHeader zfp = { 0, 0, 0, ZSTD_frame, 0, 0, 0 };
-    size_t const hError = ZSTD_getFrameHeader(&zfp, src, srcSize);
-    if (ZSTD_isError(hError)) return 0;
-    return zfp.dictID;
-}
-
-
-/*! ZSTD_decompress_usingDDict() :
-*   Decompression using a pre-digested Dictionary
-*   Use dictionary without significant overhead. */
-size_t ZSTD_decompress_usingDDict(ZSTD_DCtx* dctx,
-                                  void* dst, size_t dstCapacity,
-                            const void* src, size_t srcSize,
-                            const ZSTD_DDict* ddict)
-{
-    /* pass content and size in case legacy frames are encountered */
-    return ZSTD_decompressMultiFrame(dctx, dst, dstCapacity, src, srcSize,
-                                     NULL, 0,
-                                     ddict);
-}
-
-
-/*=====================================
-*   Streaming decompression
-*====================================*/
-
-ZSTD_DStream* ZSTD_createDStream(void)
-{
-    DEBUGLOG(3, "ZSTD_createDStream");
-    return ZSTD_createDStream_advanced(ZSTD_defaultCMem);
-}
-
-ZSTD_DStream* ZSTD_initStaticDStream(void *workspace, size_t workspaceSize)
-{
-    return ZSTD_initStaticDCtx(workspace, workspaceSize);
-}
-
-ZSTD_DStream* ZSTD_createDStream_advanced(ZSTD_customMem customMem)
-{
-    return ZSTD_createDCtx_advanced(customMem);
-}
-
-size_t ZSTD_freeDStream(ZSTD_DStream* zds)
-{
-    return ZSTD_freeDCtx(zds);
-}
-
-
-/* ***  Initialization  *** */
-
-size_t ZSTD_DStreamInSize(void)  { return ZSTD_BLOCKSIZE_MAX + ZSTD_blockHeaderSize; }
-size_t ZSTD_DStreamOutSize(void) { return ZSTD_BLOCKSIZE_MAX; }
-
-size_t ZSTD_DCtx_loadDictionary_advanced(ZSTD_DCtx* dctx,
-                                   const void* dict, size_t dictSize,
-                                         ZSTD_dictLoadMethod_e dictLoadMethod,
-                                         ZSTD_dictContentType_e dictContentType)
-{
-    RETURN_ERROR_IF(dctx->streamStage != zdss_init, stage_wrong);
-    ZSTD_clearDict(dctx);
-    if (dict && dictSize >= 8) {
-        dctx->ddictLocal = ZSTD_createDDict_advanced(dict, dictSize, dictLoadMethod, dictContentType, dctx->customMem);
-        RETURN_ERROR_IF(dctx->ddictLocal == NULL, memory_allocation);
-        dctx->ddict = dctx->ddictLocal;
-        dctx->dictUses = ZSTD_use_indefinitely;
-    }
-    return 0;
-}
-
-size_t ZSTD_DCtx_loadDictionary_byReference(ZSTD_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    return ZSTD_DCtx_loadDictionary_advanced(dctx, dict, dictSize, ZSTD_dlm_byRef, ZSTD_dct_auto);
-}
-
-size_t ZSTD_DCtx_loadDictionary(ZSTD_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    return ZSTD_DCtx_loadDictionary_advanced(dctx, dict, dictSize, ZSTD_dlm_byCopy, ZSTD_dct_auto);
-}
-
-size_t ZSTD_DCtx_refPrefix_advanced(ZSTD_DCtx* dctx, const void* prefix, size_t prefixSize, ZSTD_dictContentType_e dictContentType)
-{
-    FORWARD_IF_ERROR(ZSTD_DCtx_loadDictionary_advanced(dctx, prefix, prefixSize, ZSTD_dlm_byRef, dictContentType));
-    dctx->dictUses = ZSTD_use_once;
-    return 0;
-}
-
-size_t ZSTD_DCtx_refPrefix(ZSTD_DCtx* dctx, const void* prefix, size_t prefixSize)
-{
-    return ZSTD_DCtx_refPrefix_advanced(dctx, prefix, prefixSize, ZSTD_dct_rawContent);
-}
-
-
-/* ZSTD_initDStream_usingDict() :
- * return : expected size, aka ZSTD_FRAMEHEADERSIZE_PREFIX.
- * this function cannot fail */
-size_t ZSTD_initDStream_usingDict(ZSTD_DStream* zds, const void* dict, size_t dictSize)
-{
-    DEBUGLOG(4, "ZSTD_initDStream_usingDict");
-    FORWARD_IF_ERROR( ZSTD_DCtx_reset(zds, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_DCtx_loadDictionary(zds, dict, dictSize) );
-    return ZSTD_FRAMEHEADERSIZE_PREFIX;
-}
-
-/* note : this variant can't fail */
-size_t ZSTD_initDStream(ZSTD_DStream* zds)
-{
-    DEBUGLOG(4, "ZSTD_initDStream");
-    return ZSTD_initDStream_usingDDict(zds, NULL);
-}
-
-/* ZSTD_initDStream_usingDDict() :
- * ddict will just be referenced, and must outlive decompression session
- * this function cannot fail */
-size_t ZSTD_initDStream_usingDDict(ZSTD_DStream* dctx, const ZSTD_DDict* ddict)
-{
-    FORWARD_IF_ERROR( ZSTD_DCtx_reset(dctx, ZSTD_reset_session_only) );
-    FORWARD_IF_ERROR( ZSTD_DCtx_refDDict(dctx, ddict) );
-    return ZSTD_FRAMEHEADERSIZE_PREFIX;
-}
-
-/* ZSTD_resetDStream() :
- * return : expected size, aka ZSTD_FRAMEHEADERSIZE_PREFIX.
- * this function cannot fail */
-size_t ZSTD_resetDStream(ZSTD_DStream* dctx)
-{
-    FORWARD_IF_ERROR(ZSTD_DCtx_reset(dctx, ZSTD_reset_session_only));
-    return ZSTD_FRAMEHEADERSIZE_PREFIX;
-}
-
-
-size_t ZSTD_DCtx_refDDict(ZSTD_DCtx* dctx, const ZSTD_DDict* ddict)
-{
-    RETURN_ERROR_IF(dctx->streamStage != zdss_init, stage_wrong);
-    ZSTD_clearDict(dctx);
-    if (ddict) {
-        dctx->ddict = ddict;
-        dctx->dictUses = ZSTD_use_indefinitely;
-    }
-    return 0;
-}
-
-/* ZSTD_DCtx_setMaxWindowSize() :
- * note : no direct equivalence in ZSTD_DCtx_setParameter,
- * since this version sets windowSize, and the other sets windowLog */
-size_t ZSTD_DCtx_setMaxWindowSize(ZSTD_DCtx* dctx, size_t maxWindowSize)
-{
-    ZSTD_bounds const bounds = ZSTD_dParam_getBounds(ZSTD_d_windowLogMax);
-    size_t const min = (size_t)1 << bounds.lowerBound;
-    size_t const max = (size_t)1 << bounds.upperBound;
-    RETURN_ERROR_IF(dctx->streamStage != zdss_init, stage_wrong);
-    RETURN_ERROR_IF(maxWindowSize < min, parameter_outOfBound);
-    RETURN_ERROR_IF(maxWindowSize > max, parameter_outOfBound);
-    dctx->maxWindowSize = maxWindowSize;
-    return 0;
-}
-
-size_t ZSTD_DCtx_setFormat(ZSTD_DCtx* dctx, ZSTD_format_e format)
-{
-    return ZSTD_DCtx_setParameter(dctx, ZSTD_d_format, format);
-}
-
-ZSTD_bounds ZSTD_dParam_getBounds(ZSTD_dParameter dParam)
-{
-    ZSTD_bounds bounds = { 0, 0, 0 };
-    switch(dParam) {
-        case ZSTD_d_windowLogMax:
-            bounds.lowerBound = ZSTD_WINDOWLOG_ABSOLUTEMIN;
-            bounds.upperBound = ZSTD_WINDOWLOG_MAX;
-            return bounds;
-        case ZSTD_d_format:
-            bounds.lowerBound = (int)ZSTD_f_zstd1;
-            bounds.upperBound = (int)ZSTD_f_zstd1_magicless;
-            ZSTD_STATIC_ASSERT(ZSTD_f_zstd1 < ZSTD_f_zstd1_magicless);
-            return bounds;
-        default:;
-    }
-    bounds.error = ERROR(parameter_unsupported);
-    return bounds;
-}
-
-/* ZSTD_dParam_withinBounds:
- * @return 1 if value is within dParam bounds,
- * 0 otherwise */
-static int ZSTD_dParam_withinBounds(ZSTD_dParameter dParam, int value)
-{
-    ZSTD_bounds const bounds = ZSTD_dParam_getBounds(dParam);
-    if (ZSTD_isError(bounds.error)) return 0;
-    if (value < bounds.lowerBound) return 0;
-    if (value > bounds.upperBound) return 0;
-    return 1;
-}
-
-#define CHECK_DBOUNDS(p,v) {                \
-    RETURN_ERROR_IF(!ZSTD_dParam_withinBounds(p, v), parameter_outOfBound); \
-}
-
-size_t ZSTD_DCtx_setParameter(ZSTD_DCtx* dctx, ZSTD_dParameter dParam, int value)
-{
-    RETURN_ERROR_IF(dctx->streamStage != zdss_init, stage_wrong);
-    switch(dParam) {
-        case ZSTD_d_windowLogMax:
-            if (value == 0) value = ZSTD_WINDOWLOG_LIMIT_DEFAULT;
-            CHECK_DBOUNDS(ZSTD_d_windowLogMax, value);
-            dctx->maxWindowSize = ((size_t)1) << value;
-            return 0;
-        case ZSTD_d_format:
-            CHECK_DBOUNDS(ZSTD_d_format, value);
-            dctx->format = (ZSTD_format_e)value;
-            return 0;
-        default:;
-    }
-    RETURN_ERROR(parameter_unsupported);
-}
-
-size_t ZSTD_DCtx_reset(ZSTD_DCtx* dctx, ZSTD_ResetDirective reset)
-{
-    if ( (reset == ZSTD_reset_session_only)
-      || (reset == ZSTD_reset_session_and_parameters) ) {
-        dctx->streamStage = zdss_init;
-        dctx->noForwardProgress = 0;
-    }
-    if ( (reset == ZSTD_reset_parameters)
-      || (reset == ZSTD_reset_session_and_parameters) ) {
-        RETURN_ERROR_IF(dctx->streamStage != zdss_init, stage_wrong);
-        ZSTD_clearDict(dctx);
-        dctx->format = ZSTD_f_zstd1;
-        dctx->maxWindowSize = ZSTD_MAXWINDOWSIZE_DEFAULT;
-    }
-    return 0;
-}
-
-
-size_t ZSTD_sizeof_DStream(const ZSTD_DStream* dctx)
-{
-    return ZSTD_sizeof_DCtx(dctx);
-}
-
-size_t ZSTD_decodingBufferSize_min(unsigned long long windowSize, unsigned long long frameContentSize)
-{
-    size_t const blockSize = (size_t) MIN(windowSize, ZSTD_BLOCKSIZE_MAX);
-    unsigned long long const neededRBSize = windowSize + blockSize + (WILDCOPY_OVERLENGTH * 2);
-    unsigned long long const neededSize = MIN(frameContentSize, neededRBSize);
-    size_t const minRBSize = (size_t) neededSize;
-    RETURN_ERROR_IF((unsigned long long)minRBSize != neededSize,
-                    frameParameter_windowTooLarge);
-    return minRBSize;
-}
-
-size_t ZSTD_estimateDStreamSize(size_t windowSize)
-{
-    size_t const blockSize = MIN(windowSize, ZSTD_BLOCKSIZE_MAX);
-    size_t const inBuffSize = blockSize;  /* no block can be larger */
-    size_t const outBuffSize = ZSTD_decodingBufferSize_min(windowSize, ZSTD_CONTENTSIZE_UNKNOWN);
-    return ZSTD_estimateDCtxSize() + inBuffSize + outBuffSize;
-}
-
-size_t ZSTD_estimateDStreamSize_fromFrame(const void* src, size_t srcSize)
-{
-    U32 const windowSizeMax = 1U << ZSTD_WINDOWLOG_MAX;   /* note : should be user-selectable, but requires an additional parameter (or a dctx) */
-    ZSTD_frameHeader zfh;
-    size_t const err = ZSTD_getFrameHeader(&zfh, src, srcSize);
-    if (ZSTD_isError(err)) return err;
-    RETURN_ERROR_IF(err>0, srcSize_wrong);
-    RETURN_ERROR_IF(zfh.windowSize > windowSizeMax,
-                    frameParameter_windowTooLarge);
-    return ZSTD_estimateDStreamSize((size_t)zfh.windowSize);
-}
-
-
-/* *****   Decompression   ***** */
-
-MEM_STATIC size_t ZSTD_limitCopy(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    size_t const length = MIN(dstCapacity, srcSize);
-    memcpy(dst, src, length);
-    return length;
-}
-
-
-size_t ZSTD_decompressStream(ZSTD_DStream* zds, ZSTD_outBuffer* output, ZSTD_inBuffer* input)
-{
-    const char* const istart = (const char*)(input->src) + input->pos;
-    const char* const iend = (const char*)(input->src) + input->size;
-    const char* ip = istart;
-    char* const ostart = (char*)(output->dst) + output->pos;
-    char* const oend = (char*)(output->dst) + output->size;
-    char* op = ostart;
-    U32 someMoreWork = 1;
-
-    DEBUGLOG(5, "ZSTD_decompressStream");
-    RETURN_ERROR_IF(
-        input->pos > input->size,
-        srcSize_wrong,
-        "forbidden. in: pos: %u   vs size: %u",
-        (U32)input->pos, (U32)input->size);
-    RETURN_ERROR_IF(
-        output->pos > output->size,
-        dstSize_tooSmall,
-        "forbidden. out: pos: %u   vs size: %u",
-        (U32)output->pos, (U32)output->size);
-    DEBUGLOG(5, "input size : %u", (U32)(input->size - input->pos));
-
-    while (someMoreWork) {
-        switch(zds->streamStage)
-        {
-        case zdss_init :
-            DEBUGLOG(5, "stage zdss_init => transparent reset ");
-            zds->streamStage = zdss_loadHeader;
-            zds->lhSize = zds->inPos = zds->outStart = zds->outEnd = 0;
-            zds->legacyVersion = 0;
-            zds->hostageByte = 0;
-            /* fall-through */
-
-        case zdss_loadHeader :
-            DEBUGLOG(5, "stage zdss_loadHeader (srcSize : %u)", (U32)(iend - ip));
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT>=1)
-            if (zds->legacyVersion) {
-                RETURN_ERROR_IF(zds->staticSize, memory_allocation,
-                    "legacy support is incompatible with static dctx");
-                {   size_t const hint = ZSTD_decompressLegacyStream(zds->legacyContext, zds->legacyVersion, output, input);
-                    if (hint==0) zds->streamStage = zdss_init;
-                    return hint;
-            }   }
-#endif
-            {   size_t const hSize = ZSTD_getFrameHeader_advanced(&zds->fParams, zds->headerBuffer, zds->lhSize, zds->format);
-                DEBUGLOG(5, "header size : %u", (U32)hSize);
-                if (ZSTD_isError(hSize)) {
-#if defined(ZSTD_LEGACY_SUPPORT) && (ZSTD_LEGACY_SUPPORT>=1)
-                    U32 const legacyVersion = ZSTD_isLegacy(istart, iend-istart);
-                    if (legacyVersion) {
-                        ZSTD_DDict const* const ddict = ZSTD_getDDict(zds);
-                        const void* const dict = ddict ? ZSTD_DDict_dictContent(ddict) : NULL;
-                        size_t const dictSize = ddict ? ZSTD_DDict_dictSize(ddict) : 0;
-                        DEBUGLOG(5, "ZSTD_decompressStream: detected legacy version v0.%u", legacyVersion);
-                        RETURN_ERROR_IF(zds->staticSize, memory_allocation,
-                            "legacy support is incompatible with static dctx");
-                        FORWARD_IF_ERROR(ZSTD_initLegacyStream(&zds->legacyContext,
-                                    zds->previousLegacyVersion, legacyVersion,
-                                    dict, dictSize));
-                        zds->legacyVersion = zds->previousLegacyVersion = legacyVersion;
-                        {   size_t const hint = ZSTD_decompressLegacyStream(zds->legacyContext, legacyVersion, output, input);
-                            if (hint==0) zds->streamStage = zdss_init;   /* or stay in stage zdss_loadHeader */
-                            return hint;
-                    }   }
-#endif
-                    return hSize;   /* error */
-                }
-                if (hSize != 0) {   /* need more input */
-                    size_t const toLoad = hSize - zds->lhSize;   /* if hSize!=0, hSize > zds->lhSize */
-                    size_t const remainingInput = (size_t)(iend-ip);
-                    assert(iend >= ip);
-                    if (toLoad > remainingInput) {   /* not enough input to load full header */
-                        if (remainingInput > 0) {
-                            memcpy(zds->headerBuffer + zds->lhSize, ip, remainingInput);
-                            zds->lhSize += remainingInput;
-                        }
-                        input->pos = input->size;
-                        return (MAX(ZSTD_FRAMEHEADERSIZE_MIN, hSize) - zds->lhSize) + ZSTD_blockHeaderSize;   /* remaining header bytes + next block header */
-                    }
-                    assert(ip != NULL);
-                    memcpy(zds->headerBuffer + zds->lhSize, ip, toLoad); zds->lhSize = hSize; ip += toLoad;
-                    break;
-            }   }
-
-            /* check for single-pass mode opportunity */
-            if (zds->fParams.frameContentSize && zds->fParams.windowSize /* skippable frame if == 0 */
-                && (U64)(size_t)(oend-op) >= zds->fParams.frameContentSize) {
-                size_t const cSize = ZSTD_findFrameCompressedSize(istart, iend-istart);
-                if (cSize <= (size_t)(iend-istart)) {
-                    /* shortcut : using single-pass mode */
-                    size_t const decompressedSize = ZSTD_decompress_usingDDict(zds, op, oend-op, istart, cSize, ZSTD_getDDict(zds));
-                    if (ZSTD_isError(decompressedSize)) return decompressedSize;
-                    DEBUGLOG(4, "shortcut to single-pass ZSTD_decompress_usingDDict()")
-                    ip = istart + cSize;
-                    op += decompressedSize;
-                    zds->expected = 0;
-                    zds->streamStage = zdss_init;
-                    someMoreWork = 0;
-                    break;
-            }   }
-
-            /* Consume header (see ZSTDds_decodeFrameHeader) */
-            DEBUGLOG(4, "Consume header");
-            FORWARD_IF_ERROR(ZSTD_decompressBegin_usingDDict(zds, ZSTD_getDDict(zds)));
-
-            if ((MEM_readLE32(zds->headerBuffer) & ZSTD_MAGIC_SKIPPABLE_MASK) == ZSTD_MAGIC_SKIPPABLE_START) {  /* skippable frame */
-                zds->expected = MEM_readLE32(zds->headerBuffer + ZSTD_FRAMEIDSIZE);
-                zds->stage = ZSTDds_skipFrame;
-            } else {
-                FORWARD_IF_ERROR(ZSTD_decodeFrameHeader(zds, zds->headerBuffer, zds->lhSize));
-                zds->expected = ZSTD_blockHeaderSize;
-                zds->stage = ZSTDds_decodeBlockHeader;
-            }
-
-            /* control buffer memory usage */
-            DEBUGLOG(4, "Control max memory usage (%u KB <= max %u KB)",
-                        (U32)(zds->fParams.windowSize >>10),
-                        (U32)(zds->maxWindowSize >> 10) );
-            zds->fParams.windowSize = MAX(zds->fParams.windowSize, 1U << ZSTD_WINDOWLOG_ABSOLUTEMIN);
-            RETURN_ERROR_IF(zds->fParams.windowSize > zds->maxWindowSize,
-                            frameParameter_windowTooLarge);
-
-            /* Adapt buffer sizes to frame header instructions */
-            {   size_t const neededInBuffSize = MAX(zds->fParams.blockSizeMax, 4 /* frame checksum */);
-                size_t const neededOutBuffSize = ZSTD_decodingBufferSize_min(zds->fParams.windowSize, zds->fParams.frameContentSize);
-                if ((zds->inBuffSize < neededInBuffSize) || (zds->outBuffSize < neededOutBuffSize)) {
-                    size_t const bufferSize = neededInBuffSize + neededOutBuffSize;
-                    DEBUGLOG(4, "inBuff  : from %u to %u",
-                                (U32)zds->inBuffSize, (U32)neededInBuffSize);
-                    DEBUGLOG(4, "outBuff : from %u to %u",
-                                (U32)zds->outBuffSize, (U32)neededOutBuffSize);
-                    if (zds->staticSize) {  /* static DCtx */
-                        DEBUGLOG(4, "staticSize : %u", (U32)zds->staticSize);
-                        assert(zds->staticSize >= sizeof(ZSTD_DCtx));  /* controlled at init */
-                        RETURN_ERROR_IF(
-                            bufferSize > zds->staticSize - sizeof(ZSTD_DCtx),
-                            memory_allocation);
-                    } else {
-                        ZSTD_free(zds->inBuff, zds->customMem);
-                        zds->inBuffSize = 0;
-                        zds->outBuffSize = 0;
-                        zds->inBuff = (char*)ZSTD_malloc(bufferSize, zds->customMem);
-                        RETURN_ERROR_IF(zds->inBuff == NULL, memory_allocation);
-                    }
-                    zds->inBuffSize = neededInBuffSize;
-                    zds->outBuff = zds->inBuff + zds->inBuffSize;
-                    zds->outBuffSize = neededOutBuffSize;
-            }   }
-            zds->streamStage = zdss_read;
-            /* fall-through */
-
-        case zdss_read:
-            DEBUGLOG(5, "stage zdss_read");
-            {   size_t const neededInSize = ZSTD_nextSrcSizeToDecompress(zds);
-                DEBUGLOG(5, "neededInSize = %u", (U32)neededInSize);
-                if (neededInSize==0) {  /* end of frame */
-                    zds->streamStage = zdss_init;
-                    someMoreWork = 0;
-                    break;
-                }
-                if ((size_t)(iend-ip) >= neededInSize) {  /* decode directly from src */
-                    int const isSkipFrame = ZSTD_isSkipFrame(zds);
-                    size_t const decodedSize = ZSTD_decompressContinue(zds,
-                        zds->outBuff + zds->outStart, (isSkipFrame ? 0 : zds->outBuffSize - zds->outStart),
-                        ip, neededInSize);
-                    if (ZSTD_isError(decodedSize)) return decodedSize;
-                    ip += neededInSize;
-                    if (!decodedSize && !isSkipFrame) break;   /* this was just a header */
-                    zds->outEnd = zds->outStart + decodedSize;
-                    zds->streamStage = zdss_flush;
-                    break;
-            }   }
-            if (ip==iend) { someMoreWork = 0; break; }   /* no more input */
-            zds->streamStage = zdss_load;
-            /* fall-through */
-
-        case zdss_load:
-            {   size_t const neededInSize = ZSTD_nextSrcSizeToDecompress(zds);
-                size_t const toLoad = neededInSize - zds->inPos;
-                int const isSkipFrame = ZSTD_isSkipFrame(zds);
-                size_t loadedSize;
-                if (isSkipFrame) {
-                    loadedSize = MIN(toLoad, (size_t)(iend-ip));
-                } else {
-                    RETURN_ERROR_IF(toLoad > zds->inBuffSize - zds->inPos,
-                                    corruption_detected,
-                                    "should never happen");
-                    loadedSize = ZSTD_limitCopy(zds->inBuff + zds->inPos, toLoad, ip, iend-ip);
-                }
-                ip += loadedSize;
-                zds->inPos += loadedSize;
-                if (loadedSize < toLoad) { someMoreWork = 0; break; }   /* not enough input, wait for more */
-
-                /* decode loaded input */
-                {   size_t const decodedSize = ZSTD_decompressContinue(zds,
-                        zds->outBuff + zds->outStart, zds->outBuffSize - zds->outStart,
-                        zds->inBuff, neededInSize);
-                    if (ZSTD_isError(decodedSize)) return decodedSize;
-                    zds->inPos = 0;   /* input is consumed */
-                    if (!decodedSize && !isSkipFrame) { zds->streamStage = zdss_read; break; }   /* this was just a header */
-                    zds->outEnd = zds->outStart +  decodedSize;
-            }   }
-            zds->streamStage = zdss_flush;
-            /* fall-through */
-
-        case zdss_flush:
-            {   size_t const toFlushSize = zds->outEnd - zds->outStart;
-                size_t const flushedSize = ZSTD_limitCopy(op, oend-op, zds->outBuff + zds->outStart, toFlushSize);
-                op += flushedSize;
-                zds->outStart += flushedSize;
-                if (flushedSize == toFlushSize) {  /* flush completed */
-                    zds->streamStage = zdss_read;
-                    if ( (zds->outBuffSize < zds->fParams.frameContentSize)
-                      && (zds->outStart + zds->fParams.blockSizeMax > zds->outBuffSize) ) {
-                        DEBUGLOG(5, "restart filling outBuff from beginning (left:%i, needed:%u)",
-                                (int)(zds->outBuffSize - zds->outStart),
-                                (U32)zds->fParams.blockSizeMax);
-                        zds->outStart = zds->outEnd = 0;
-                    }
-                    break;
-            }   }
-            /* cannot complete flush */
-            someMoreWork = 0;
-            break;
-
-        default:
-            assert(0);    /* impossible */
-            RETURN_ERROR(GENERIC);   /* some compiler require default to do something */
-    }   }
-
-    /* result */
-    input->pos = (size_t)(ip - (const char*)(input->src));
-    output->pos = (size_t)(op - (char*)(output->dst));
-    if ((ip==istart) && (op==ostart)) {  /* no forward progress */
-        zds->noForwardProgress ++;
-        if (zds->noForwardProgress >= ZSTD_NO_FORWARD_PROGRESS_MAX) {
-            RETURN_ERROR_IF(op==oend, dstSize_tooSmall);
-            RETURN_ERROR_IF(ip==iend, srcSize_wrong);
-            assert(0);
-        }
-    } else {
-        zds->noForwardProgress = 0;
-    }
-    {   size_t nextSrcSizeHint = ZSTD_nextSrcSizeToDecompress(zds);
-        if (!nextSrcSizeHint) {   /* frame fully decoded */
-            if (zds->outEnd == zds->outStart) {  /* output fully flushed */
-                if (zds->hostageByte) {
-                    if (input->pos >= input->size) {
-                        /* can't release hostage (not present) */
-                        zds->streamStage = zdss_read;
-                        return 1;
-                    }
-                    input->pos++;  /* release hostage */
-                }   /* zds->hostageByte */
-                return 0;
-            }  /* zds->outEnd == zds->outStart */
-            if (!zds->hostageByte) { /* output not fully flushed; keep last byte as hostage; will be released when all output is flushed */
-                input->pos--;   /* note : pos > 0, otherwise, impossible to finish reading last block */
-                zds->hostageByte=1;
-            }
-            return 1;
-        }  /* nextSrcSizeHint==0 */
-        nextSrcSizeHint += ZSTD_blockHeaderSize * (ZSTD_nextInputType(zds) == ZSTDnit_block);   /* preload header of next block */
-        assert(zds->inPos <= nextSrcSizeHint);
-        nextSrcSizeHint -= zds->inPos;   /* part already loaded*/
-        return nextSrcSizeHint;
-    }
-}
-
-size_t ZSTD_decompressStream_simpleArgs (
-                            ZSTD_DCtx* dctx,
-                            void* dst, size_t dstCapacity, size_t* dstPos,
-                      const void* src, size_t srcSize, size_t* srcPos)
-{
-    ZSTD_outBuffer output = { dst, dstCapacity, *dstPos };
-    ZSTD_inBuffer  input  = { src, srcSize, *srcPos };
-    /* ZSTD_compress_generic() will check validity of dstPos and srcPos */
-    size_t const cErr = ZSTD_decompressStream(dctx, &output, &input);
-    *dstPos = output.pos;
-    *srcPos = input.pos;
-    return cErr;
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_decompress_block.c b/vendor/github.com/DataDog/zstd/zstd_decompress_block.c
deleted file mode 100644
index 24f4859..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_decompress_block.c
+++ /dev/null
@@ -1,1322 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-/* zstd_decompress_block :
- * this module takes care of decompressing _compressed_ block */
-
-/*-*******************************************************
-*  Dependencies
-*********************************************************/
-#include <string.h>      /* memcpy, memmove, memset */
-#include "compiler.h"    /* prefetch */
-#include "cpu.h"         /* bmi2 */
-#include "mem.h"         /* low level memory routines */
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#include "zstd_internal.h"
-#include "zstd_decompress_internal.h"   /* ZSTD_DCtx */
-#include "zstd_ddict.h"  /* ZSTD_DDictDictContent */
-#include "zstd_decompress_block.h"
-
-/*_*******************************************************
-*  Macros
-**********************************************************/
-
-/* These two optional macros force the use one way or another of the two
- * ZSTD_decompressSequences implementations. You can't force in both directions
- * at the same time.
- */
-#if defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT) && \
-    defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG)
-#error "Cannot force the use of the short and the long ZSTD_decompressSequences variants!"
-#endif
-
-
-/*_*******************************************************
-*  Memory operations
-**********************************************************/
-static void ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-
-/*-*************************************************************
- *   Block decoding
- ***************************************************************/
-
-/*! ZSTD_getcBlockSize() :
- *  Provides the size of compressed block from block header `src` */
-size_t ZSTD_getcBlockSize(const void* src, size_t srcSize,
-                          blockProperties_t* bpPtr)
-{
-    RETURN_ERROR_IF(srcSize < ZSTD_blockHeaderSize, srcSize_wrong);
-
-    {   U32 const cBlockHeader = MEM_readLE24(src);
-        U32 const cSize = cBlockHeader >> 3;
-        bpPtr->lastBlock = cBlockHeader & 1;
-        bpPtr->blockType = (blockType_e)((cBlockHeader >> 1) & 3);
-        bpPtr->origSize = cSize;   /* only useful for RLE */
-        if (bpPtr->blockType == bt_rle) return 1;
-        RETURN_ERROR_IF(bpPtr->blockType == bt_reserved, corruption_detected);
-        return cSize;
-    }
-}
-
-
-/* Hidden declaration for fullbench */
-size_t ZSTD_decodeLiteralsBlock(ZSTD_DCtx* dctx,
-                          const void* src, size_t srcSize);
-/*! ZSTD_decodeLiteralsBlock() :
- * @return : nb of bytes read from src (< srcSize )
- *  note : symbol not declared but exposed for fullbench */
-size_t ZSTD_decodeLiteralsBlock(ZSTD_DCtx* dctx,
-                          const void* src, size_t srcSize)   /* note : srcSize < BLOCKSIZE */
-{
-    RETURN_ERROR_IF(srcSize < MIN_CBLOCK_SIZE, corruption_detected);
-
-    {   const BYTE* const istart = (const BYTE*) src;
-        symbolEncodingType_e const litEncType = (symbolEncodingType_e)(istart[0] & 3);
-
-        switch(litEncType)
-        {
-        case set_repeat:
-            RETURN_ERROR_IF(dctx->litEntropy==0, dictionary_corrupted);
-            /* fall-through */
-
-        case set_compressed:
-            RETURN_ERROR_IF(srcSize < 5, corruption_detected, "srcSize >= MIN_CBLOCK_SIZE == 3; here we need up to 5 for case 3");
-            {   size_t lhSize, litSize, litCSize;
-                U32 singleStream=0;
-                U32 const lhlCode = (istart[0] >> 2) & 3;
-                U32 const lhc = MEM_readLE32(istart);
-                size_t hufSuccess;
-                switch(lhlCode)
-                {
-                case 0: case 1: default:   /* note : default is impossible, since lhlCode into [0..3] */
-                    /* 2 - 2 - 10 - 10 */
-                    singleStream = !lhlCode;
-                    lhSize = 3;
-                    litSize  = (lhc >> 4) & 0x3FF;
-                    litCSize = (lhc >> 14) & 0x3FF;
-                    break;
-                case 2:
-                    /* 2 - 2 - 14 - 14 */
-                    lhSize = 4;
-                    litSize  = (lhc >> 4) & 0x3FFF;
-                    litCSize = lhc >> 18;
-                    break;
-                case 3:
-                    /* 2 - 2 - 18 - 18 */
-                    lhSize = 5;
-                    litSize  = (lhc >> 4) & 0x3FFFF;
-                    litCSize = (lhc >> 22) + (istart[4] << 10);
-                    break;
-                }
-                RETURN_ERROR_IF(litSize > ZSTD_BLOCKSIZE_MAX, corruption_detected);
-                RETURN_ERROR_IF(litCSize + lhSize > srcSize, corruption_detected);
-
-                /* prefetch huffman table if cold */
-                if (dctx->ddictIsCold && (litSize > 768 /* heuristic */)) {
-                    PREFETCH_AREA(dctx->HUFptr, sizeof(dctx->entropy.hufTable));
-                }
-
-                if (litEncType==set_repeat) {
-                    if (singleStream) {
-                        hufSuccess = HUF_decompress1X_usingDTable_bmi2(
-                            dctx->litBuffer, litSize, istart+lhSize, litCSize,
-                            dctx->HUFptr, dctx->bmi2);
-                    } else {
-                        hufSuccess = HUF_decompress4X_usingDTable_bmi2(
-                            dctx->litBuffer, litSize, istart+lhSize, litCSize,
-                            dctx->HUFptr, dctx->bmi2);
-                    }
-                } else {
-                    if (singleStream) {
-#if defined(HUF_FORCE_DECOMPRESS_X2)
-                        hufSuccess = HUF_decompress1X_DCtx_wksp(
-                            dctx->entropy.hufTable, dctx->litBuffer, litSize,
-                            istart+lhSize, litCSize, dctx->workspace,
-                            sizeof(dctx->workspace));
-#else
-                        hufSuccess = HUF_decompress1X1_DCtx_wksp_bmi2(
-                            dctx->entropy.hufTable, dctx->litBuffer, litSize,
-                            istart+lhSize, litCSize, dctx->workspace,
-                            sizeof(dctx->workspace), dctx->bmi2);
-#endif
-                    } else {
-                        hufSuccess = HUF_decompress4X_hufOnly_wksp_bmi2(
-                            dctx->entropy.hufTable, dctx->litBuffer, litSize,
-                            istart+lhSize, litCSize, dctx->workspace,
-                            sizeof(dctx->workspace), dctx->bmi2);
-                    }
-                }
-
-                RETURN_ERROR_IF(HUF_isError(hufSuccess), corruption_detected);
-
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                dctx->litEntropy = 1;
-                if (litEncType==set_compressed) dctx->HUFptr = dctx->entropy.hufTable;
-                memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-                return litCSize + lhSize;
-            }
-
-        case set_basic:
-            {   size_t litSize, lhSize;
-                U32 const lhlCode = ((istart[0]) >> 2) & 3;
-                switch(lhlCode)
-                {
-                case 0: case 2: default:   /* note : default is impossible, since lhlCode into [0..3] */
-                    lhSize = 1;
-                    litSize = istart[0] >> 3;
-                    break;
-                case 1:
-                    lhSize = 2;
-                    litSize = MEM_readLE16(istart) >> 4;
-                    break;
-                case 3:
-                    lhSize = 3;
-                    litSize = MEM_readLE24(istart) >> 4;
-                    break;
-                }
-
-                if (lhSize+litSize+WILDCOPY_OVERLENGTH > srcSize) {  /* risk reading beyond src buffer with wildcopy */
-                    RETURN_ERROR_IF(litSize+lhSize > srcSize, corruption_detected);
-                    memcpy(dctx->litBuffer, istart+lhSize, litSize);
-                    dctx->litPtr = dctx->litBuffer;
-                    dctx->litSize = litSize;
-                    memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-                    return lhSize+litSize;
-                }
-                /* direct reference into compressed stream */
-                dctx->litPtr = istart+lhSize;
-                dctx->litSize = litSize;
-                return lhSize+litSize;
-            }
-
-        case set_rle:
-            {   U32 const lhlCode = ((istart[0]) >> 2) & 3;
-                size_t litSize, lhSize;
-                switch(lhlCode)
-                {
-                case 0: case 2: default:   /* note : default is impossible, since lhlCode into [0..3] */
-                    lhSize = 1;
-                    litSize = istart[0] >> 3;
-                    break;
-                case 1:
-                    lhSize = 2;
-                    litSize = MEM_readLE16(istart) >> 4;
-                    break;
-                case 3:
-                    lhSize = 3;
-                    litSize = MEM_readLE24(istart) >> 4;
-                    RETURN_ERROR_IF(srcSize<4, corruption_detected, "srcSize >= MIN_CBLOCK_SIZE == 3; here we need lhSize+1 = 4");
-                    break;
-                }
-                RETURN_ERROR_IF(litSize > ZSTD_BLOCKSIZE_MAX, corruption_detected);
-                memset(dctx->litBuffer, istart[lhSize], litSize + WILDCOPY_OVERLENGTH);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                return lhSize+1;
-            }
-        default:
-            RETURN_ERROR(corruption_detected, "impossible");
-        }
-    }
-}
-
-/* Default FSE distribution tables.
- * These are pre-calculated FSE decoding tables using default distributions as defined in specification :
- * https://github.com/facebook/zstd/blob/master/doc/zstd_compression_format.md#default-distributions
- * They were generated programmatically with following method :
- * - start from default distributions, present in /lib/common/zstd_internal.h
- * - generate tables normally, using ZSTD_buildFSETable()
- * - printout the content of tables
- * - pretify output, report below, test with fuzzer to ensure it's correct */
-
-/* Default FSE distribution table for Literal Lengths */
-static const ZSTD_seqSymbol LL_defaultDTable[(1<<LL_DEFAULTNORMLOG)+1] = {
-     {  1,  1,  1, LL_DEFAULTNORMLOG},  /* header : fastMode, tableLog */
-     /* nextState, nbAddBits, nbBits, baseVal */
-     {  0,  0,  4,    0},  { 16,  0,  4,    0},
-     { 32,  0,  5,    1},  {  0,  0,  5,    3},
-     {  0,  0,  5,    4},  {  0,  0,  5,    6},
-     {  0,  0,  5,    7},  {  0,  0,  5,    9},
-     {  0,  0,  5,   10},  {  0,  0,  5,   12},
-     {  0,  0,  6,   14},  {  0,  1,  5,   16},
-     {  0,  1,  5,   20},  {  0,  1,  5,   22},
-     {  0,  2,  5,   28},  {  0,  3,  5,   32},
-     {  0,  4,  5,   48},  { 32,  6,  5,   64},
-     {  0,  7,  5,  128},  {  0,  8,  6,  256},
-     {  0, 10,  6, 1024},  {  0, 12,  6, 4096},
-     { 32,  0,  4,    0},  {  0,  0,  4,    1},
-     {  0,  0,  5,    2},  { 32,  0,  5,    4},
-     {  0,  0,  5,    5},  { 32,  0,  5,    7},
-     {  0,  0,  5,    8},  { 32,  0,  5,   10},
-     {  0,  0,  5,   11},  {  0,  0,  6,   13},
-     { 32,  1,  5,   16},  {  0,  1,  5,   18},
-     { 32,  1,  5,   22},  {  0,  2,  5,   24},
-     { 32,  3,  5,   32},  {  0,  3,  5,   40},
-     {  0,  6,  4,   64},  { 16,  6,  4,   64},
-     { 32,  7,  5,  128},  {  0,  9,  6,  512},
-     {  0, 11,  6, 2048},  { 48,  0,  4,    0},
-     { 16,  0,  4,    1},  { 32,  0,  5,    2},
-     { 32,  0,  5,    3},  { 32,  0,  5,    5},
-     { 32,  0,  5,    6},  { 32,  0,  5,    8},
-     { 32,  0,  5,    9},  { 32,  0,  5,   11},
-     { 32,  0,  5,   12},  {  0,  0,  6,   15},
-     { 32,  1,  5,   18},  { 32,  1,  5,   20},
-     { 32,  2,  5,   24},  { 32,  2,  5,   28},
-     { 32,  3,  5,   40},  { 32,  4,  5,   48},
-     {  0, 16,  6,65536},  {  0, 15,  6,32768},
-     {  0, 14,  6,16384},  {  0, 13,  6, 8192},
-};   /* LL_defaultDTable */
-
-/* Default FSE distribution table for Offset Codes */
-static const ZSTD_seqSymbol OF_defaultDTable[(1<<OF_DEFAULTNORMLOG)+1] = {
-    {  1,  1,  1, OF_DEFAULTNORMLOG},  /* header : fastMode, tableLog */
-    /* nextState, nbAddBits, nbBits, baseVal */
-    {  0,  0,  5,    0},     {  0,  6,  4,   61},
-    {  0,  9,  5,  509},     {  0, 15,  5,32765},
-    {  0, 21,  5,2097149},   {  0,  3,  5,    5},
-    {  0,  7,  4,  125},     {  0, 12,  5, 4093},
-    {  0, 18,  5,262141},    {  0, 23,  5,8388605},
-    {  0,  5,  5,   29},     {  0,  8,  4,  253},
-    {  0, 14,  5,16381},     {  0, 20,  5,1048573},
-    {  0,  2,  5,    1},     { 16,  7,  4,  125},
-    {  0, 11,  5, 2045},     {  0, 17,  5,131069},
-    {  0, 22,  5,4194301},   {  0,  4,  5,   13},
-    { 16,  8,  4,  253},     {  0, 13,  5, 8189},
-    {  0, 19,  5,524285},    {  0,  1,  5,    1},
-    { 16,  6,  4,   61},     {  0, 10,  5, 1021},
-    {  0, 16,  5,65533},     {  0, 28,  5,268435453},
-    {  0, 27,  5,134217725}, {  0, 26,  5,67108861},
-    {  0, 25,  5,33554429},  {  0, 24,  5,16777213},
-};   /* OF_defaultDTable */
-
-
-/* Default FSE distribution table for Match Lengths */
-static const ZSTD_seqSymbol ML_defaultDTable[(1<<ML_DEFAULTNORMLOG)+1] = {
-    {  1,  1,  1, ML_DEFAULTNORMLOG},  /* header : fastMode, tableLog */
-    /* nextState, nbAddBits, nbBits, baseVal */
-    {  0,  0,  6,    3},  {  0,  0,  4,    4},
-    { 32,  0,  5,    5},  {  0,  0,  5,    6},
-    {  0,  0,  5,    8},  {  0,  0,  5,    9},
-    {  0,  0,  5,   11},  {  0,  0,  6,   13},
-    {  0,  0,  6,   16},  {  0,  0,  6,   19},
-    {  0,  0,  6,   22},  {  0,  0,  6,   25},
-    {  0,  0,  6,   28},  {  0,  0,  6,   31},
-    {  0,  0,  6,   34},  {  0,  1,  6,   37},
-    {  0,  1,  6,   41},  {  0,  2,  6,   47},
-    {  0,  3,  6,   59},  {  0,  4,  6,   83},
-    {  0,  7,  6,  131},  {  0,  9,  6,  515},
-    { 16,  0,  4,    4},  {  0,  0,  4,    5},
-    { 32,  0,  5,    6},  {  0,  0,  5,    7},
-    { 32,  0,  5,    9},  {  0,  0,  5,   10},
-    {  0,  0,  6,   12},  {  0,  0,  6,   15},
-    {  0,  0,  6,   18},  {  0,  0,  6,   21},
-    {  0,  0,  6,   24},  {  0,  0,  6,   27},
-    {  0,  0,  6,   30},  {  0,  0,  6,   33},
-    {  0,  1,  6,   35},  {  0,  1,  6,   39},
-    {  0,  2,  6,   43},  {  0,  3,  6,   51},
-    {  0,  4,  6,   67},  {  0,  5,  6,   99},
-    {  0,  8,  6,  259},  { 32,  0,  4,    4},
-    { 48,  0,  4,    4},  { 16,  0,  4,    5},
-    { 32,  0,  5,    7},  { 32,  0,  5,    8},
-    { 32,  0,  5,   10},  { 32,  0,  5,   11},
-    {  0,  0,  6,   14},  {  0,  0,  6,   17},
-    {  0,  0,  6,   20},  {  0,  0,  6,   23},
-    {  0,  0,  6,   26},  {  0,  0,  6,   29},
-    {  0,  0,  6,   32},  {  0, 16,  6,65539},
-    {  0, 15,  6,32771},  {  0, 14,  6,16387},
-    {  0, 13,  6, 8195},  {  0, 12,  6, 4099},
-    {  0, 11,  6, 2051},  {  0, 10,  6, 1027},
-};   /* ML_defaultDTable */
-
-
-static void ZSTD_buildSeqTable_rle(ZSTD_seqSymbol* dt, U32 baseValue, U32 nbAddBits)
-{
-    void* ptr = dt;
-    ZSTD_seqSymbol_header* const DTableH = (ZSTD_seqSymbol_header*)ptr;
-    ZSTD_seqSymbol* const cell = dt + 1;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->nbBits = 0;
-    cell->nextState = 0;
-    assert(nbAddBits < 255);
-    cell->nbAdditionalBits = (BYTE)nbAddBits;
-    cell->baseValue = baseValue;
-}
-
-
-/* ZSTD_buildFSETable() :
- * generate FSE decoding table for one symbol (ll, ml or off)
- * cannot fail if input is valid =>
- * all inputs are presumed validated at this stage */
-void
-ZSTD_buildFSETable(ZSTD_seqSymbol* dt,
-            const short* normalizedCounter, unsigned maxSymbolValue,
-            const U32* baseValue, const U32* nbAdditionalBits,
-            unsigned tableLog)
-{
-    ZSTD_seqSymbol* const tableDecode = dt+1;
-    U16 symbolNext[MaxSeq+1];
-
-    U32 const maxSV1 = maxSymbolValue + 1;
-    U32 const tableSize = 1 << tableLog;
-    U32 highThreshold = tableSize-1;
-
-    /* Sanity Checks */
-    assert(maxSymbolValue <= MaxSeq);
-    assert(tableLog <= MaxFSELog);
-
-    /* Init, lay down lowprob symbols */
-    {   ZSTD_seqSymbol_header DTableH;
-        DTableH.tableLog = tableLog;
-        DTableH.fastMode = 1;
-        {   S16 const largeLimit= (S16)(1 << (tableLog-1));
-            U32 s;
-            for (s=0; s<maxSV1; s++) {
-                if (normalizedCounter[s]==-1) {
-                    tableDecode[highThreshold--].baseValue = s;
-                    symbolNext[s] = 1;
-                } else {
-                    if (normalizedCounter[s] >= largeLimit) DTableH.fastMode=0;
-                    symbolNext[s] = normalizedCounter[s];
-        }   }   }
-        memcpy(dt, &DTableH, sizeof(DTableH));
-    }
-
-    /* Spread symbols */
-    {   U32 const tableMask = tableSize-1;
-        U32 const step = FSE_TABLESTEP(tableSize);
-        U32 s, position = 0;
-        for (s=0; s<maxSV1; s++) {
-            int i;
-            for (i=0; i<normalizedCounter[s]; i++) {
-                tableDecode[position].baseValue = s;
-                position = (position + step) & tableMask;
-                while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }   }
-        assert(position == 0); /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-    }
-
-    /* Build Decoding table */
-    {   U32 u;
-        for (u=0; u<tableSize; u++) {
-            U32 const symbol = tableDecode[u].baseValue;
-            U32 const nextState = symbolNext[symbol]++;
-            tableDecode[u].nbBits = (BYTE) (tableLog - BIT_highbit32(nextState) );
-            tableDecode[u].nextState = (U16) ( (nextState << tableDecode[u].nbBits) - tableSize);
-            assert(nbAdditionalBits[symbol] < 255);
-            tableDecode[u].nbAdditionalBits = (BYTE)nbAdditionalBits[symbol];
-            tableDecode[u].baseValue = baseValue[symbol];
-    }   }
-}
-
-
-/*! ZSTD_buildSeqTable() :
- * @return : nb bytes read from src,
- *           or an error code if it fails */
-static size_t ZSTD_buildSeqTable(ZSTD_seqSymbol* DTableSpace, const ZSTD_seqSymbol** DTablePtr,
-                                 symbolEncodingType_e type, unsigned max, U32 maxLog,
-                                 const void* src, size_t srcSize,
-                                 const U32* baseValue, const U32* nbAdditionalBits,
-                                 const ZSTD_seqSymbol* defaultTable, U32 flagRepeatTable,
-                                 int ddictIsCold, int nbSeq)
-{
-    switch(type)
-    {
-    case set_rle :
-        RETURN_ERROR_IF(!srcSize, srcSize_wrong);
-        RETURN_ERROR_IF((*(const BYTE*)src) > max, corruption_detected);
-        {   U32 const symbol = *(const BYTE*)src;
-            U32 const baseline = baseValue[symbol];
-            U32 const nbBits = nbAdditionalBits[symbol];
-            ZSTD_buildSeqTable_rle(DTableSpace, baseline, nbBits);
-        }
-        *DTablePtr = DTableSpace;
-        return 1;
-    case set_basic :
-        *DTablePtr = defaultTable;
-        return 0;
-    case set_repeat:
-        RETURN_ERROR_IF(!flagRepeatTable, corruption_detected);
-        /* prefetch FSE table if used */
-        if (ddictIsCold && (nbSeq > 24 /* heuristic */)) {
-            const void* const pStart = *DTablePtr;
-            size_t const pSize = sizeof(ZSTD_seqSymbol) * (SEQSYMBOL_TABLE_SIZE(maxLog));
-            PREFETCH_AREA(pStart, pSize);
-        }
-        return 0;
-    case set_compressed :
-        {   unsigned tableLog;
-            S16 norm[MaxSeq+1];
-            size_t const headerSize = FSE_readNCount(norm, &max, &tableLog, src, srcSize);
-            RETURN_ERROR_IF(FSE_isError(headerSize), corruption_detected);
-            RETURN_ERROR_IF(tableLog > maxLog, corruption_detected);
-            ZSTD_buildFSETable(DTableSpace, norm, max, baseValue, nbAdditionalBits, tableLog);
-            *DTablePtr = DTableSpace;
-            return headerSize;
-        }
-    default :
-        assert(0);
-        RETURN_ERROR(GENERIC, "impossible");
-    }
-}
-
-size_t ZSTD_decodeSeqHeaders(ZSTD_DCtx* dctx, int* nbSeqPtr,
-                             const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* ip = istart;
-    int nbSeq;
-    DEBUGLOG(5, "ZSTD_decodeSeqHeaders");
-
-    /* check */
-    RETURN_ERROR_IF(srcSize < MIN_SEQUENCES_SIZE, srcSize_wrong);
-
-    /* SeqHead */
-    nbSeq = *ip++;
-    if (!nbSeq) {
-        *nbSeqPtr=0;
-        RETURN_ERROR_IF(srcSize != 1, srcSize_wrong);
-        return 1;
-    }
-    if (nbSeq > 0x7F) {
-        if (nbSeq == 0xFF) {
-            RETURN_ERROR_IF(ip+2 > iend, srcSize_wrong);
-            nbSeq = MEM_readLE16(ip) + LONGNBSEQ, ip+=2;
-        } else {
-            RETURN_ERROR_IF(ip >= iend, srcSize_wrong);
-            nbSeq = ((nbSeq-0x80)<<8) + *ip++;
-        }
-    }
-    *nbSeqPtr = nbSeq;
-
-    /* FSE table descriptors */
-    RETURN_ERROR_IF(ip+1 > iend, srcSize_wrong); /* minimum possible size: 1 byte for symbol encoding types */
-    {   symbolEncodingType_e const LLtype = (symbolEncodingType_e)(*ip >> 6);
-        symbolEncodingType_e const OFtype = (symbolEncodingType_e)((*ip >> 4) & 3);
-        symbolEncodingType_e const MLtype = (symbolEncodingType_e)((*ip >> 2) & 3);
-        ip++;
-
-        /* Build DTables */
-        {   size_t const llhSize = ZSTD_buildSeqTable(dctx->entropy.LLTable, &dctx->LLTptr,
-                                                      LLtype, MaxLL, LLFSELog,
-                                                      ip, iend-ip,
-                                                      LL_base, LL_bits,
-                                                      LL_defaultDTable, dctx->fseEntropy,
-                                                      dctx->ddictIsCold, nbSeq);
-            RETURN_ERROR_IF(ZSTD_isError(llhSize), corruption_detected);
-            ip += llhSize;
-        }
-
-        {   size_t const ofhSize = ZSTD_buildSeqTable(dctx->entropy.OFTable, &dctx->OFTptr,
-                                                      OFtype, MaxOff, OffFSELog,
-                                                      ip, iend-ip,
-                                                      OF_base, OF_bits,
-                                                      OF_defaultDTable, dctx->fseEntropy,
-                                                      dctx->ddictIsCold, nbSeq);
-            RETURN_ERROR_IF(ZSTD_isError(ofhSize), corruption_detected);
-            ip += ofhSize;
-        }
-
-        {   size_t const mlhSize = ZSTD_buildSeqTable(dctx->entropy.MLTable, &dctx->MLTptr,
-                                                      MLtype, MaxML, MLFSELog,
-                                                      ip, iend-ip,
-                                                      ML_base, ML_bits,
-                                                      ML_defaultDTable, dctx->fseEntropy,
-                                                      dctx->ddictIsCold, nbSeq);
-            RETURN_ERROR_IF(ZSTD_isError(mlhSize), corruption_detected);
-            ip += mlhSize;
-        }
-    }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t matchLength;
-    size_t offset;
-    const BYTE* match;
-} seq_t;
-
-typedef struct {
-    size_t state;
-    const ZSTD_seqSymbol* table;
-} ZSTD_fseState;
-
-typedef struct {
-    BIT_DStream_t DStream;
-    ZSTD_fseState stateLL;
-    ZSTD_fseState stateOffb;
-    ZSTD_fseState stateML;
-    size_t prevOffset[ZSTD_REP_NUM];
-    const BYTE* prefixStart;
-    const BYTE* dictEnd;
-    size_t pos;
-} seqState_t;
-
-
-/* ZSTD_execSequenceLast7():
- * exceptional case : decompress a match starting within last 7 bytes of output buffer.
- * requires more careful checks, to ensure there is no overflow.
- * performance does not matter though.
- * note : this case is supposed to be never generated "naturally" by reference encoder,
- *        since in most cases it needs at least 8 bytes to look for a match.
- *        but it's allowed by the specification. */
-FORCE_NOINLINE
-size_t ZSTD_execSequenceLast7(BYTE* op,
-                              BYTE* const oend, seq_t sequence,
-                              const BYTE** litPtr, const BYTE* const litLimit,
-                              const BYTE* const base, const BYTE* const vBase, const BYTE* const dictEnd)
-{
-    BYTE* const oLitEnd = op + sequence.litLength;
-    size_t const sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    const BYTE* const iLitEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    RETURN_ERROR_IF(oMatchEnd>oend, dstSize_tooSmall, "last match must fit within dstBuffer");
-    RETURN_ERROR_IF(iLitEnd > litLimit, corruption_detected, "try to read beyond literal buffer");
-
-    /* copy literals */
-    while (op < oLitEnd) *op++ = *(*litPtr)++;
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - base)) {
-        /* offset beyond prefix */
-        RETURN_ERROR_IF(sequence.offset > (size_t)(oLitEnd - vBase),corruption_detected);
-        match = dictEnd - (base-match);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {   size_t const length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = base;
-    }   }
-    while (op < oMatchEnd) *op++ = *match++;
-    return sequenceLength;
-}
-
-
-HINT_INLINE
-size_t ZSTD_execSequence(BYTE* op,
-                         BYTE* const oend, seq_t sequence,
-                         const BYTE** litPtr, const BYTE* const litLimit,
-                         const BYTE* const prefixStart, const BYTE* const virtualStart, const BYTE* const dictEnd)
-{
-    BYTE* const oLitEnd = op + sequence.litLength;
-    size_t const sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_w = oend - WILDCOPY_OVERLENGTH;
-    const BYTE* const iLitEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    RETURN_ERROR_IF(oMatchEnd>oend, dstSize_tooSmall, "last match must start at a minimum distance of WILDCOPY_OVERLENGTH from oend");
-    RETURN_ERROR_IF(iLitEnd > litLimit, corruption_detected, "over-read beyond lit buffer");
-    if (oLitEnd>oend_w) return ZSTD_execSequenceLast7(op, oend, sequence, litPtr, litLimit, prefixStart, virtualStart, dictEnd);
-
-    /* copy Literals */
-    if (sequence.litLength > 8)
-        ZSTD_wildcopy_16min(op, (*litPtr), sequence.litLength, ZSTD_no_overlap);   /* note : since oLitEnd <= oend-WILDCOPY_OVERLENGTH, no risk of overwrite beyond oend */
-    else
-        ZSTD_copy8(op, *litPtr);
-    op = oLitEnd;
-    *litPtr = iLitEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - prefixStart)) {
-        /* offset beyond prefix -> go into extDict */
-        RETURN_ERROR_IF(sequence.offset > (size_t)(oLitEnd - virtualStart), corruption_detected);
-        match = dictEnd + (match - prefixStart);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {   size_t const length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = prefixStart;
-            if (op > oend_w || sequence.matchLength < MINMATCH) {
-              U32 i;
-              for (i = 0; i < sequence.matchLength; ++i) op[i] = match[i];
-              return sequenceLength;
-            }
-    }   }
-    /* Requirement: op <= oend_w && sequence.matchLength >= MINMATCH */
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        static const U32 dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-        static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-        int const sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTD_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTD_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH)) {
-        if (op < oend_w) {
-            ZSTD_wildcopy(op, match, oend_w - op, ZSTD_overlap_src_before_dst);
-            match += oend_w - op;
-            op = oend_w;
-        }
-        while (op < oMatchEnd) *op++ = *match++;
-    } else {
-        ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8, ZSTD_overlap_src_before_dst);   /* works even if matchLength < 8 */
-    }
-    return sequenceLength;
-}
-
-
-HINT_INLINE
-size_t ZSTD_execSequenceLong(BYTE* op,
-                             BYTE* const oend, seq_t sequence,
-                             const BYTE** litPtr, const BYTE* const litLimit,
-                             const BYTE* const prefixStart, const BYTE* const dictStart, const BYTE* const dictEnd)
-{
-    BYTE* const oLitEnd = op + sequence.litLength;
-    size_t const sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_w = oend - WILDCOPY_OVERLENGTH;
-    const BYTE* const iLitEnd = *litPtr + sequence.litLength;
-    const BYTE* match = sequence.match;
-
-    /* check */
-    RETURN_ERROR_IF(oMatchEnd > oend, dstSize_tooSmall, "last match must start at a minimum distance of WILDCOPY_OVERLENGTH from oend");
-    RETURN_ERROR_IF(iLitEnd > litLimit, corruption_detected, "over-read beyond lit buffer");
-    if (oLitEnd > oend_w) return ZSTD_execSequenceLast7(op, oend, sequence, litPtr, litLimit, prefixStart, dictStart, dictEnd);
-
-    /* copy Literals */
-    if (sequence.litLength > 8)
-        ZSTD_wildcopy_16min(op, *litPtr, sequence.litLength, ZSTD_no_overlap);   /* note : since oLitEnd <= oend-WILDCOPY_OVERLENGTH, no risk of overwrite beyond oend */
-    else
-        ZSTD_copy8(op, *litPtr);  /* note : op <= oLitEnd <= oend_w == oend - 8 */
-
-    op = oLitEnd;
-    *litPtr = iLitEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - prefixStart)) {
-        /* offset beyond prefix */
-        RETURN_ERROR_IF(sequence.offset > (size_t)(oLitEnd - dictStart), corruption_detected);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {   size_t const length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = prefixStart;
-            if (op > oend_w || sequence.matchLength < MINMATCH) {
-              U32 i;
-              for (i = 0; i < sequence.matchLength; ++i) op[i] = match[i];
-              return sequenceLength;
-            }
-    }   }
-    assert(op <= oend_w);
-    assert(sequence.matchLength >= MINMATCH);
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        static const U32 dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-        static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-        int const sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTD_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTD_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH)) {
-        if (op < oend_w) {
-            ZSTD_wildcopy(op, match, oend_w - op, ZSTD_overlap_src_before_dst);
-            match += oend_w - op;
-            op = oend_w;
-        }
-        while (op < oMatchEnd) *op++ = *match++;
-    } else {
-        ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8, ZSTD_overlap_src_before_dst);   /* works even if matchLength < 8 */
-    }
-    return sequenceLength;
-}
-
-static void
-ZSTD_initFseState(ZSTD_fseState* DStatePtr, BIT_DStream_t* bitD, const ZSTD_seqSymbol* dt)
-{
-    const void* ptr = dt;
-    const ZSTD_seqSymbol_header* const DTableH = (const ZSTD_seqSymbol_header*)ptr;
-    DStatePtr->state = BIT_readBits(bitD, DTableH->tableLog);
-    DEBUGLOG(6, "ZSTD_initFseState : val=%u using %u bits",
-                (U32)DStatePtr->state, DTableH->tableLog);
-    BIT_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-FORCE_INLINE_TEMPLATE void
-ZSTD_updateFseState(ZSTD_fseState* DStatePtr, BIT_DStream_t* bitD)
-{
-    ZSTD_seqSymbol const DInfo = DStatePtr->table[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    size_t const lowBits = BIT_readBits(bitD, nbBits);
-    DStatePtr->state = DInfo.nextState + lowBits;
-}
-
-/* We need to add at most (ZSTD_WINDOWLOG_MAX_32 - 1) bits to read the maximum
- * offset bits. But we can only read at most (STREAM_ACCUMULATOR_MIN_32 - 1)
- * bits before reloading. This value is the maximum number of bytes we read
- * after reloading when we are decoding long offsets.
- */
-#define LONG_OFFSETS_MAX_EXTRA_BITS_32                       \
-    (ZSTD_WINDOWLOG_MAX_32 > STREAM_ACCUMULATOR_MIN_32       \
-        ? ZSTD_WINDOWLOG_MAX_32 - STREAM_ACCUMULATOR_MIN_32  \
-        : 0)
-
-typedef enum { ZSTD_lo_isRegularOffset, ZSTD_lo_isLongOffset=1 } ZSTD_longOffset_e;
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG
-FORCE_INLINE_TEMPLATE seq_t
-ZSTD_decodeSequence(seqState_t* seqState, const ZSTD_longOffset_e longOffsets)
-{
-    seq_t seq;
-    U32 const llBits = seqState->stateLL.table[seqState->stateLL.state].nbAdditionalBits;
-    U32 const mlBits = seqState->stateML.table[seqState->stateML.state].nbAdditionalBits;
-    U32 const ofBits = seqState->stateOffb.table[seqState->stateOffb.state].nbAdditionalBits;
-    U32 const totalBits = llBits+mlBits+ofBits;
-    U32 const llBase = seqState->stateLL.table[seqState->stateLL.state].baseValue;
-    U32 const mlBase = seqState->stateML.table[seqState->stateML.state].baseValue;
-    U32 const ofBase = seqState->stateOffb.table[seqState->stateOffb.state].baseValue;
-
-    /* sequence */
-    {   size_t offset;
-        if (!ofBits)
-            offset = 0;
-        else {
-            ZSTD_STATIC_ASSERT(ZSTD_lo_isLongOffset == 1);
-            ZSTD_STATIC_ASSERT(LONG_OFFSETS_MAX_EXTRA_BITS_32 == 5);
-            assert(ofBits <= MaxOff);
-            if (MEM_32bits() && longOffsets && (ofBits >= STREAM_ACCUMULATOR_MIN_32)) {
-                U32 const extraBits = ofBits - MIN(ofBits, 32 - seqState->DStream.bitsConsumed);
-                offset = ofBase + (BIT_readBitsFast(&seqState->DStream, ofBits - extraBits) << extraBits);
-                BIT_reloadDStream(&seqState->DStream);
-                if (extraBits) offset += BIT_readBitsFast(&seqState->DStream, extraBits);
-                assert(extraBits <= LONG_OFFSETS_MAX_EXTRA_BITS_32);   /* to avoid another reload */
-            } else {
-                offset = ofBase + BIT_readBitsFast(&seqState->DStream, ofBits/*>0*/);   /* <=  (ZSTD_WINDOWLOG_MAX-1) bits */
-                if (MEM_32bits()) BIT_reloadDStream(&seqState->DStream);
-            }
-        }
-
-        if (ofBits <= 1) {
-            offset += (llBase==0);
-            if (offset) {
-                size_t temp = (offset==3) ? seqState->prevOffset[0] - 1 : seqState->prevOffset[offset];
-                temp += !temp;   /* 0 is not valid; input is corrupted; force offset to 1 */
-                if (offset != 1) seqState->prevOffset[2] = seqState->prevOffset[1];
-                seqState->prevOffset[1] = seqState->prevOffset[0];
-                seqState->prevOffset[0] = offset = temp;
-            } else {  /* offset == 0 */
-                offset = seqState->prevOffset[0];
-            }
-        } else {
-            seqState->prevOffset[2] = seqState->prevOffset[1];
-            seqState->prevOffset[1] = seqState->prevOffset[0];
-            seqState->prevOffset[0] = offset;
-        }
-        seq.offset = offset;
-    }
-
-    seq.matchLength = mlBase
-                    + ((mlBits>0) ? BIT_readBitsFast(&seqState->DStream, mlBits/*>0*/) : 0);  /* <=  16 bits */
-    if (MEM_32bits() && (mlBits+llBits >= STREAM_ACCUMULATOR_MIN_32-LONG_OFFSETS_MAX_EXTRA_BITS_32))
-        BIT_reloadDStream(&seqState->DStream);
-    if (MEM_64bits() && (totalBits >= STREAM_ACCUMULATOR_MIN_64-(LLFSELog+MLFSELog+OffFSELog)))
-        BIT_reloadDStream(&seqState->DStream);
-    /* Ensure there are enough bits to read the rest of data in 64-bit mode. */
-    ZSTD_STATIC_ASSERT(16+LLFSELog+MLFSELog+OffFSELog < STREAM_ACCUMULATOR_MIN_64);
-
-    seq.litLength = llBase
-                  + ((llBits>0) ? BIT_readBitsFast(&seqState->DStream, llBits/*>0*/) : 0);    /* <=  16 bits */
-    if (MEM_32bits())
-        BIT_reloadDStream(&seqState->DStream);
-
-    DEBUGLOG(6, "seq: litL=%u, matchL=%u, offset=%u",
-                (U32)seq.litLength, (U32)seq.matchLength, (U32)seq.offset);
-
-    /* ANS state update */
-    ZSTD_updateFseState(&seqState->stateLL, &seqState->DStream);    /* <=  9 bits */
-    ZSTD_updateFseState(&seqState->stateML, &seqState->DStream);    /* <=  9 bits */
-    if (MEM_32bits()) BIT_reloadDStream(&seqState->DStream);    /* <= 18 bits */
-    ZSTD_updateFseState(&seqState->stateOffb, &seqState->DStream);  /* <=  8 bits */
-
-    return seq;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-DONT_VECTORIZE
-ZSTD_decompressSequences_body( ZSTD_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize, int nbSeq,
-                         const ZSTD_longOffset_e isLongOffset)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + maxDstSize;
-    BYTE* op = ostart;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    const BYTE* const prefixStart = (const BYTE*) (dctx->prefixStart);
-    const BYTE* const vBase = (const BYTE*) (dctx->virtualStart);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-    DEBUGLOG(5, "ZSTD_decompressSequences_body");
-
-    /* Regen sequences */
-    if (nbSeq) {
-        seqState_t seqState;
-        dctx->fseEntropy = 1;
-        { U32 i; for (i=0; i<ZSTD_REP_NUM; i++) seqState.prevOffset[i] = dctx->entropy.rep[i]; }
-        RETURN_ERROR_IF(
-            ERR_isError(BIT_initDStream(&seqState.DStream, ip, iend-ip)),
-            corruption_detected);
-        ZSTD_initFseState(&seqState.stateLL, &seqState.DStream, dctx->LLTptr);
-        ZSTD_initFseState(&seqState.stateOffb, &seqState.DStream, dctx->OFTptr);
-        ZSTD_initFseState(&seqState.stateML, &seqState.DStream, dctx->MLTptr);
-
-        ZSTD_STATIC_ASSERT(
-                BIT_DStream_unfinished < BIT_DStream_completed &&
-                BIT_DStream_endOfBuffer < BIT_DStream_completed &&
-                BIT_DStream_completed < BIT_DStream_overflow);
-
-        for ( ; (BIT_reloadDStream(&(seqState.DStream)) <= BIT_DStream_completed) && nbSeq ; ) {
-            nbSeq--;
-            {   seq_t const sequence = ZSTD_decodeSequence(&seqState, isLongOffset);
-                size_t const oneSeqSize = ZSTD_execSequence(op, oend, sequence, &litPtr, litEnd, prefixStart, vBase, dictEnd);
-                DEBUGLOG(6, "regenerated sequence size : %u", (U32)oneSeqSize);
-                if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-                op += oneSeqSize;
-        }   }
-
-        /* check if reached exact end */
-        DEBUGLOG(5, "ZSTD_decompressSequences_body: after decode loop, remaining nbSeq : %i", nbSeq);
-        RETURN_ERROR_IF(nbSeq, corruption_detected);
-        RETURN_ERROR_IF(BIT_reloadDStream(&seqState.DStream) < BIT_DStream_completed, corruption_detected);
-        /* save reps for next block */
-        { U32 i; for (i=0; i<ZSTD_REP_NUM; i++) dctx->entropy.rep[i] = (U32)(seqState.prevOffset[i]); }
-    }
-
-    /* last literal segment */
-    {   size_t const lastLLSize = litEnd - litPtr;
-        RETURN_ERROR_IF(lastLLSize > (size_t)(oend-op), dstSize_tooSmall);
-        memcpy(op, litPtr, lastLLSize);
-        op += lastLLSize;
-    }
-
-    return op-ostart;
-}
-
-static size_t
-ZSTD_decompressSequences_default(ZSTD_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                           const void* seqStart, size_t seqSize, int nbSeq,
-                           const ZSTD_longOffset_e isLongOffset)
-{
-    return ZSTD_decompressSequences_body(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG */
-
-
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT
-FORCE_INLINE_TEMPLATE seq_t
-ZSTD_decodeSequenceLong(seqState_t* seqState, ZSTD_longOffset_e const longOffsets)
-{
-    seq_t seq;
-    U32 const llBits = seqState->stateLL.table[seqState->stateLL.state].nbAdditionalBits;
-    U32 const mlBits = seqState->stateML.table[seqState->stateML.state].nbAdditionalBits;
-    U32 const ofBits = seqState->stateOffb.table[seqState->stateOffb.state].nbAdditionalBits;
-    U32 const totalBits = llBits+mlBits+ofBits;
-    U32 const llBase = seqState->stateLL.table[seqState->stateLL.state].baseValue;
-    U32 const mlBase = seqState->stateML.table[seqState->stateML.state].baseValue;
-    U32 const ofBase = seqState->stateOffb.table[seqState->stateOffb.state].baseValue;
-
-    /* sequence */
-    {   size_t offset;
-        if (!ofBits)
-            offset = 0;
-        else {
-            ZSTD_STATIC_ASSERT(ZSTD_lo_isLongOffset == 1);
-            ZSTD_STATIC_ASSERT(LONG_OFFSETS_MAX_EXTRA_BITS_32 == 5);
-            assert(ofBits <= MaxOff);
-            if (MEM_32bits() && longOffsets) {
-                U32 const extraBits = ofBits - MIN(ofBits, STREAM_ACCUMULATOR_MIN_32-1);
-                offset = ofBase + (BIT_readBitsFast(&seqState->DStream, ofBits - extraBits) << extraBits);
-                if (MEM_32bits() || extraBits) BIT_reloadDStream(&seqState->DStream);
-                if (extraBits) offset += BIT_readBitsFast(&seqState->DStream, extraBits);
-            } else {
-                offset = ofBase + BIT_readBitsFast(&seqState->DStream, ofBits);   /* <=  (ZSTD_WINDOWLOG_MAX-1) bits */
-                if (MEM_32bits()) BIT_reloadDStream(&seqState->DStream);
-            }
-        }
-
-        if (ofBits <= 1) {
-            offset += (llBase==0);
-            if (offset) {
-                size_t temp = (offset==3) ? seqState->prevOffset[0] - 1 : seqState->prevOffset[offset];
-                temp += !temp;   /* 0 is not valid; input is corrupted; force offset to 1 */
-                if (offset != 1) seqState->prevOffset[2] = seqState->prevOffset[1];
-                seqState->prevOffset[1] = seqState->prevOffset[0];
-                seqState->prevOffset[0] = offset = temp;
-            } else {
-                offset = seqState->prevOffset[0];
-            }
-        } else {
-            seqState->prevOffset[2] = seqState->prevOffset[1];
-            seqState->prevOffset[1] = seqState->prevOffset[0];
-            seqState->prevOffset[0] = offset;
-        }
-        seq.offset = offset;
-    }
-
-    seq.matchLength = mlBase + ((mlBits>0) ? BIT_readBitsFast(&seqState->DStream, mlBits) : 0);  /* <=  16 bits */
-    if (MEM_32bits() && (mlBits+llBits >= STREAM_ACCUMULATOR_MIN_32-LONG_OFFSETS_MAX_EXTRA_BITS_32))
-        BIT_reloadDStream(&seqState->DStream);
-    if (MEM_64bits() && (totalBits >= STREAM_ACCUMULATOR_MIN_64-(LLFSELog+MLFSELog+OffFSELog)))
-        BIT_reloadDStream(&seqState->DStream);
-    /* Verify that there is enough bits to read the rest of the data in 64-bit mode. */
-    ZSTD_STATIC_ASSERT(16+LLFSELog+MLFSELog+OffFSELog < STREAM_ACCUMULATOR_MIN_64);
-
-    seq.litLength = llBase + ((llBits>0) ? BIT_readBitsFast(&seqState->DStream, llBits) : 0);    /* <=  16 bits */
-    if (MEM_32bits())
-        BIT_reloadDStream(&seqState->DStream);
-
-    {   size_t const pos = seqState->pos + seq.litLength;
-        const BYTE* const matchBase = (seq.offset > pos) ? seqState->dictEnd : seqState->prefixStart;
-        seq.match = matchBase + pos - seq.offset;  /* note : this operation can overflow when seq.offset is really too large, which can only happen when input is corrupted.
-                                                    * No consequence though : no memory access will occur, overly large offset will be detected in ZSTD_execSequenceLong() */
-        seqState->pos = pos + seq.matchLength;
-    }
-
-    /* ANS state update */
-    ZSTD_updateFseState(&seqState->stateLL, &seqState->DStream);    /* <=  9 bits */
-    ZSTD_updateFseState(&seqState->stateML, &seqState->DStream);    /* <=  9 bits */
-    if (MEM_32bits()) BIT_reloadDStream(&seqState->DStream);    /* <= 18 bits */
-    ZSTD_updateFseState(&seqState->stateOffb, &seqState->DStream);  /* <=  8 bits */
-
-    return seq;
-}
-
-FORCE_INLINE_TEMPLATE size_t
-ZSTD_decompressSequencesLong_body(
-                               ZSTD_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize, int nbSeq,
-                         const ZSTD_longOffset_e isLongOffset)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + maxDstSize;
-    BYTE* op = ostart;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    const BYTE* const prefixStart = (const BYTE*) (dctx->prefixStart);
-    const BYTE* const dictStart = (const BYTE*) (dctx->virtualStart);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-
-    /* Regen sequences */
-    if (nbSeq) {
-#define STORED_SEQS 4
-#define STORED_SEQS_MASK (STORED_SEQS-1)
-#define ADVANCED_SEQS 4
-        seq_t sequences[STORED_SEQS];
-        int const seqAdvance = MIN(nbSeq, ADVANCED_SEQS);
-        seqState_t seqState;
-        int seqNb;
-        dctx->fseEntropy = 1;
-        { int i; for (i=0; i<ZSTD_REP_NUM; i++) seqState.prevOffset[i] = dctx->entropy.rep[i]; }
-        seqState.prefixStart = prefixStart;
-        seqState.pos = (size_t)(op-prefixStart);
-        seqState.dictEnd = dictEnd;
-        assert(iend >= ip);
-        RETURN_ERROR_IF(
-            ERR_isError(BIT_initDStream(&seqState.DStream, ip, iend-ip)),
-            corruption_detected);
-        ZSTD_initFseState(&seqState.stateLL, &seqState.DStream, dctx->LLTptr);
-        ZSTD_initFseState(&seqState.stateOffb, &seqState.DStream, dctx->OFTptr);
-        ZSTD_initFseState(&seqState.stateML, &seqState.DStream, dctx->MLTptr);
-
-        /* prepare in advance */
-        for (seqNb=0; (BIT_reloadDStream(&seqState.DStream) <= BIT_DStream_completed) && (seqNb<seqAdvance); seqNb++) {
-            sequences[seqNb] = ZSTD_decodeSequenceLong(&seqState, isLongOffset);
-            PREFETCH_L1(sequences[seqNb].match); PREFETCH_L1(sequences[seqNb].match + sequences[seqNb].matchLength - 1); /* note : it's safe to invoke PREFETCH() on any memory address, including invalid ones */
-        }
-        RETURN_ERROR_IF(seqNb<seqAdvance, corruption_detected);
-
-        /* decode and decompress */
-        for ( ; (BIT_reloadDStream(&(seqState.DStream)) <= BIT_DStream_completed) && (seqNb<nbSeq) ; seqNb++) {
-            seq_t const sequence = ZSTD_decodeSequenceLong(&seqState, isLongOffset);
-            size_t const oneSeqSize = ZSTD_execSequenceLong(op, oend, sequences[(seqNb-ADVANCED_SEQS) & STORED_SEQS_MASK], &litPtr, litEnd, prefixStart, dictStart, dictEnd);
-            if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-            PREFETCH_L1(sequence.match); PREFETCH_L1(sequence.match + sequence.matchLength - 1); /* note : it's safe to invoke PREFETCH() on any memory address, including invalid ones */
-            sequences[seqNb & STORED_SEQS_MASK] = sequence;
-            op += oneSeqSize;
-        }
-        RETURN_ERROR_IF(seqNb<nbSeq, corruption_detected);
-
-        /* finish queue */
-        seqNb -= seqAdvance;
-        for ( ; seqNb<nbSeq ; seqNb++) {
-            size_t const oneSeqSize = ZSTD_execSequenceLong(op, oend, sequences[seqNb&STORED_SEQS_MASK], &litPtr, litEnd, prefixStart, dictStart, dictEnd);
-            if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* save reps for next block */
-        { U32 i; for (i=0; i<ZSTD_REP_NUM; i++) dctx->entropy.rep[i] = (U32)(seqState.prevOffset[i]); }
-    }
-
-    /* last literal segment */
-    {   size_t const lastLLSize = litEnd - litPtr;
-        RETURN_ERROR_IF(lastLLSize > (size_t)(oend-op), dstSize_tooSmall);
-        memcpy(op, litPtr, lastLLSize);
-        op += lastLLSize;
-    }
-
-    return op-ostart;
-}
-
-static size_t
-ZSTD_decompressSequencesLong_default(ZSTD_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                           const void* seqStart, size_t seqSize, int nbSeq,
-                           const ZSTD_longOffset_e isLongOffset)
-{
-    return ZSTD_decompressSequencesLong_body(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT */
-
-
-
-#if DYNAMIC_BMI2
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG
-static TARGET_ATTRIBUTE("bmi2") size_t
-DONT_VECTORIZE
-ZSTD_decompressSequences_bmi2(ZSTD_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                           const void* seqStart, size_t seqSize, int nbSeq,
-                           const ZSTD_longOffset_e isLongOffset)
-{
-    return ZSTD_decompressSequences_body(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG */
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT
-static TARGET_ATTRIBUTE("bmi2") size_t
-ZSTD_decompressSequencesLong_bmi2(ZSTD_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                           const void* seqStart, size_t seqSize, int nbSeq,
-                           const ZSTD_longOffset_e isLongOffset)
-{
-    return ZSTD_decompressSequencesLong_body(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT */
-
-#endif /* DYNAMIC_BMI2 */
-
-typedef size_t (*ZSTD_decompressSequences_t)(
-                            ZSTD_DCtx* dctx,
-                            void* dst, size_t maxDstSize,
-                            const void* seqStart, size_t seqSize, int nbSeq,
-                            const ZSTD_longOffset_e isLongOffset);
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG
-static size_t
-ZSTD_decompressSequences(ZSTD_DCtx* dctx, void* dst, size_t maxDstSize,
-                   const void* seqStart, size_t seqSize, int nbSeq,
-                   const ZSTD_longOffset_e isLongOffset)
-{
-    DEBUGLOG(5, "ZSTD_decompressSequences");
-#if DYNAMIC_BMI2
-    if (dctx->bmi2) {
-        return ZSTD_decompressSequences_bmi2(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-    }
-#endif
-  return ZSTD_decompressSequences_default(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG */
-
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT
-/* ZSTD_decompressSequencesLong() :
- * decompression function triggered when a minimum share of offsets is considered "long",
- * aka out of cache.
- * note : "long" definition seems overloaded here, sometimes meaning "wider than bitstream register", and sometimes meaning "farther than memory cache distance".
- * This function will try to mitigate main memory latency through the use of prefetching */
-static size_t
-ZSTD_decompressSequencesLong(ZSTD_DCtx* dctx,
-                             void* dst, size_t maxDstSize,
-                             const void* seqStart, size_t seqSize, int nbSeq,
-                             const ZSTD_longOffset_e isLongOffset)
-{
-    DEBUGLOG(5, "ZSTD_decompressSequencesLong");
-#if DYNAMIC_BMI2
-    if (dctx->bmi2) {
-        return ZSTD_decompressSequencesLong_bmi2(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-    }
-#endif
-  return ZSTD_decompressSequencesLong_default(dctx, dst, maxDstSize, seqStart, seqSize, nbSeq, isLongOffset);
-}
-#endif /* ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT */
-
-
-
-#if !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT) && \
-    !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG)
-/* ZSTD_getLongOffsetsShare() :
- * condition : offTable must be valid
- * @return : "share" of long offsets (arbitrarily defined as > (1<<23))
- *           compared to maximum possible of (1<<OffFSELog) */
-static unsigned
-ZSTD_getLongOffsetsShare(const ZSTD_seqSymbol* offTable)
-{
-    const void* ptr = offTable;
-    U32 const tableLog = ((const ZSTD_seqSymbol_header*)ptr)[0].tableLog;
-    const ZSTD_seqSymbol* table = offTable + 1;
-    U32 const max = 1 << tableLog;
-    U32 u, total = 0;
-    DEBUGLOG(5, "ZSTD_getLongOffsetsShare: (tableLog=%u)", tableLog);
-
-    assert(max <= (1 << OffFSELog));  /* max not too large */
-    for (u=0; u<max; u++) {
-        if (table[u].nbAdditionalBits > 22) total += 1;
-    }
-
-    assert(tableLog <= OffFSELog);
-    total <<= (OffFSELog - tableLog);  /* scale to OffFSELog */
-
-    return total;
-}
-#endif
-
-
-size_t
-ZSTD_decompressBlock_internal(ZSTD_DCtx* dctx,
-                              void* dst, size_t dstCapacity,
-                        const void* src, size_t srcSize, const int frame)
-{   /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-    /* isLongOffset must be true if there are long offsets.
-     * Offsets are long if they are larger than 2^STREAM_ACCUMULATOR_MIN.
-     * We don't expect that to be the case in 64-bit mode.
-     * In block mode, window size is not known, so we have to be conservative.
-     * (note: but it could be evaluated from current-lowLimit)
-     */
-    ZSTD_longOffset_e const isLongOffset = (ZSTD_longOffset_e)(MEM_32bits() && (!frame || (dctx->fParams.windowSize > (1ULL << STREAM_ACCUMULATOR_MIN))));
-    DEBUGLOG(5, "ZSTD_decompressBlock_internal (size : %u)", (U32)srcSize);
-
-    RETURN_ERROR_IF(srcSize >= ZSTD_BLOCKSIZE_MAX, srcSize_wrong);
-
-    /* Decode literals section */
-    {   size_t const litCSize = ZSTD_decodeLiteralsBlock(dctx, src, srcSize);
-        DEBUGLOG(5, "ZSTD_decodeLiteralsBlock : %u", (U32)litCSize);
-        if (ZSTD_isError(litCSize)) return litCSize;
-        ip += litCSize;
-        srcSize -= litCSize;
-    }
-
-    /* Build Decoding Tables */
-    {
-        /* These macros control at build-time which decompressor implementation
-         * we use. If neither is defined, we do some inspection and dispatch at
-         * runtime.
-         */
-#if !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT) && \
-    !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG)
-        int usePrefetchDecoder = dctx->ddictIsCold;
-#endif
-        int nbSeq;
-        size_t const seqHSize = ZSTD_decodeSeqHeaders(dctx, &nbSeq, ip, srcSize);
-        if (ZSTD_isError(seqHSize)) return seqHSize;
-        ip += seqHSize;
-        srcSize -= seqHSize;
-
-#if !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT) && \
-    !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG)
-        if ( !usePrefetchDecoder
-          && (!frame || (dctx->fParams.windowSize > (1<<24)))
-          && (nbSeq>ADVANCED_SEQS) ) {  /* could probably use a larger nbSeq limit */
-            U32 const shareLongOffsets = ZSTD_getLongOffsetsShare(dctx->OFTptr);
-            U32 const minShare = MEM_64bits() ? 7 : 20; /* heuristic values, correspond to 2.73% and 7.81% */
-            usePrefetchDecoder = (shareLongOffsets >= minShare);
-        }
-#endif
-
-        dctx->ddictIsCold = 0;
-
-#if !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT) && \
-    !defined(ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG)
-        if (usePrefetchDecoder)
-#endif
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_SHORT
-            return ZSTD_decompressSequencesLong(dctx, dst, dstCapacity, ip, srcSize, nbSeq, isLongOffset);
-#endif
-
-#ifndef ZSTD_FORCE_DECOMPRESS_SEQUENCES_LONG
-        /* else */
-        return ZSTD_decompressSequences(dctx, dst, dstCapacity, ip, srcSize, nbSeq, isLongOffset);
-#endif
-    }
-}
-
-
-size_t ZSTD_decompressBlock(ZSTD_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{
-    size_t dSize;
-    ZSTD_checkContinuity(dctx, dst);
-    dSize = ZSTD_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize, /* frame */ 0);
-    dctx->previousDstEnd = (char*)dst + dSize;
-    return dSize;
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_decompress_block.h b/vendor/github.com/DataDog/zstd/zstd_decompress_block.h
deleted file mode 100644
index 7e92960..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_decompress_block.h
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-#ifndef ZSTD_DEC_BLOCK_H
-#define ZSTD_DEC_BLOCK_H
-
-/*-*******************************************************
- *  Dependencies
- *********************************************************/
-#include <stddef.h>   /* size_t */
-#include "zstd.h"    /* DCtx, and some public functions */
-#include "zstd_internal.h"  /* blockProperties_t, and some public functions */
-#include "zstd_decompress_internal.h"  /* ZSTD_seqSymbol */
-
-
-/* ===   Prototypes   === */
-
-/* note: prototypes already published within `zstd.h` :
- * ZSTD_decompressBlock()
- */
-
-/* note: prototypes already published within `zstd_internal.h` :
- * ZSTD_getcBlockSize()
- * ZSTD_decodeSeqHeaders()
- */
-
-
-/* ZSTD_decompressBlock_internal() :
- * decompress block, starting at `src`,
- * into destination buffer `dst`.
- * @return : decompressed block size,
- *           or an error code (which can be tested using ZSTD_isError())
- */
-size_t ZSTD_decompressBlock_internal(ZSTD_DCtx* dctx,
-                               void* dst, size_t dstCapacity,
-                         const void* src, size_t srcSize, const int frame);
-
-/* ZSTD_buildFSETable() :
- * generate FSE decoding table for one symbol (ll, ml or off)
- * this function must be called with valid parameters only
- * (dt is large enough, normalizedCounter distribution total is a power of 2, max is within range, etc.)
- * in which case it cannot fail.
- * Internal use only.
- */
-void ZSTD_buildFSETable(ZSTD_seqSymbol* dt,
-             const short* normalizedCounter, unsigned maxSymbolValue,
-             const U32* baseValue, const U32* nbAdditionalBits,
-                   unsigned tableLog);
-
-
-#endif /* ZSTD_DEC_BLOCK_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_decompress_internal.h b/vendor/github.com/DataDog/zstd/zstd_decompress_internal.h
deleted file mode 100644
index ccbdfa0..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_decompress_internal.h
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/* zstd_decompress_internal:
- * objects and definitions shared within lib/decompress modules */
-
- #ifndef ZSTD_DECOMPRESS_INTERNAL_H
- #define ZSTD_DECOMPRESS_INTERNAL_H
-
-
-/*-*******************************************************
- *  Dependencies
- *********************************************************/
-#include "mem.h"             /* BYTE, U16, U32 */
-#include "zstd_internal.h"   /* ZSTD_seqSymbol */
-
-
-
-/*-*******************************************************
- *  Constants
- *********************************************************/
-static const U32 LL_base[MaxLL+1] = {
-                 0,    1,    2,     3,     4,     5,     6,      7,
-                 8,    9,   10,    11,    12,    13,    14,     15,
-                16,   18,   20,    22,    24,    28,    32,     40,
-                48,   64, 0x80, 0x100, 0x200, 0x400, 0x800, 0x1000,
-                0x2000, 0x4000, 0x8000, 0x10000 };
-
-static const U32 OF_base[MaxOff+1] = {
-                 0,        1,       1,       5,     0xD,     0x1D,     0x3D,     0x7D,
-                 0xFD,   0x1FD,   0x3FD,   0x7FD,   0xFFD,   0x1FFD,   0x3FFD,   0x7FFD,
-                 0xFFFD, 0x1FFFD, 0x3FFFD, 0x7FFFD, 0xFFFFD, 0x1FFFFD, 0x3FFFFD, 0x7FFFFD,
-                 0xFFFFFD, 0x1FFFFFD, 0x3FFFFFD, 0x7FFFFFD, 0xFFFFFFD, 0x1FFFFFFD, 0x3FFFFFFD, 0x7FFFFFFD };
-
-static const U32 OF_bits[MaxOff+1] = {
-                     0,  1,  2,  3,  4,  5,  6,  7,
-                     8,  9, 10, 11, 12, 13, 14, 15,
-                    16, 17, 18, 19, 20, 21, 22, 23,
-                    24, 25, 26, 27, 28, 29, 30, 31 };
-
-static const U32 ML_base[MaxML+1] = {
-                     3,  4,  5,    6,     7,     8,     9,    10,
-                    11, 12, 13,   14,    15,    16,    17,    18,
-                    19, 20, 21,   22,    23,    24,    25,    26,
-                    27, 28, 29,   30,    31,    32,    33,    34,
-                    35, 37, 39,   41,    43,    47,    51,    59,
-                    67, 83, 99, 0x83, 0x103, 0x203, 0x403, 0x803,
-                    0x1003, 0x2003, 0x4003, 0x8003, 0x10003 };
-
-
-/*-*******************************************************
- *  Decompression types
- *********************************************************/
- typedef struct {
-     U32 fastMode;
-     U32 tableLog;
- } ZSTD_seqSymbol_header;
-
- typedef struct {
-     U16  nextState;
-     BYTE nbAdditionalBits;
-     BYTE nbBits;
-     U32  baseValue;
- } ZSTD_seqSymbol;
-
- #define SEQSYMBOL_TABLE_SIZE(log)   (1 + (1 << (log)))
-
-typedef struct {
-    ZSTD_seqSymbol LLTable[SEQSYMBOL_TABLE_SIZE(LLFSELog)];    /* Note : Space reserved for FSE Tables */
-    ZSTD_seqSymbol OFTable[SEQSYMBOL_TABLE_SIZE(OffFSELog)];   /* is also used as temporary workspace while building hufTable during DDict creation */
-    ZSTD_seqSymbol MLTable[SEQSYMBOL_TABLE_SIZE(MLFSELog)];    /* and therefore must be at least HUF_DECOMPRESS_WORKSPACE_SIZE large */
-    HUF_DTable hufTable[HUF_DTABLE_SIZE(HufLog)];  /* can accommodate HUF_decompress4X */
-    U32 rep[ZSTD_REP_NUM];
-} ZSTD_entropyDTables_t;
-
-typedef enum { ZSTDds_getFrameHeaderSize, ZSTDds_decodeFrameHeader,
-               ZSTDds_decodeBlockHeader, ZSTDds_decompressBlock,
-               ZSTDds_decompressLastBlock, ZSTDds_checkChecksum,
-               ZSTDds_decodeSkippableHeader, ZSTDds_skipFrame } ZSTD_dStage;
-
-typedef enum { zdss_init=0, zdss_loadHeader,
-               zdss_read, zdss_load, zdss_flush } ZSTD_dStreamStage;
-
-typedef enum {
-    ZSTD_use_indefinitely = -1,  /* Use the dictionary indefinitely */
-    ZSTD_dont_use = 0,           /* Do not use the dictionary (if one exists free it) */
-    ZSTD_use_once = 1            /* Use the dictionary once and set to ZSTD_dont_use */
-} ZSTD_dictUses_e;
-
-struct ZSTD_DCtx_s
-{
-    const ZSTD_seqSymbol* LLTptr;
-    const ZSTD_seqSymbol* MLTptr;
-    const ZSTD_seqSymbol* OFTptr;
-    const HUF_DTable* HUFptr;
-    ZSTD_entropyDTables_t entropy;
-    U32 workspace[HUF_DECOMPRESS_WORKSPACE_SIZE_U32];   /* space needed when building huffman tables */
-    const void* previousDstEnd;   /* detect continuity */
-    const void* prefixStart;      /* start of current segment */
-    const void* virtualStart;     /* virtual start of previous segment if it was just before current one */
-    const void* dictEnd;          /* end of previous segment */
-    size_t expected;
-    ZSTD_frameHeader fParams;
-    U64 decodedSize;
-    blockType_e bType;            /* used in ZSTD_decompressContinue(), store blockType between block header decoding and block decompression stages */
-    ZSTD_dStage stage;
-    U32 litEntropy;
-    U32 fseEntropy;
-    XXH64_state_t xxhState;
-    size_t headerSize;
-    ZSTD_format_e format;
-    const BYTE* litPtr;
-    ZSTD_customMem customMem;
-    size_t litSize;
-    size_t rleSize;
-    size_t staticSize;
-    int bmi2;                     /* == 1 if the CPU supports BMI2 and 0 otherwise. CPU support is determined dynamically once per context lifetime. */
-
-    /* dictionary */
-    ZSTD_DDict* ddictLocal;
-    const ZSTD_DDict* ddict;     /* set by ZSTD_initDStream_usingDDict(), or ZSTD_DCtx_refDDict() */
-    U32 dictID;
-    int ddictIsCold;             /* if == 1 : dictionary is "new" for working context, and presumed "cold" (not in cpu cache) */
-    ZSTD_dictUses_e dictUses;
-
-    /* streaming */
-    ZSTD_dStreamStage streamStage;
-    char*  inBuff;
-    size_t inBuffSize;
-    size_t inPos;
-    size_t maxWindowSize;
-    char*  outBuff;
-    size_t outBuffSize;
-    size_t outStart;
-    size_t outEnd;
-    size_t lhSize;
-    void* legacyContext;
-    U32 previousLegacyVersion;
-    U32 legacyVersion;
-    U32 hostageByte;
-    int noForwardProgress;
-
-    /* workspace */
-    BYTE litBuffer[ZSTD_BLOCKSIZE_MAX + WILDCOPY_OVERLENGTH];
-    BYTE headerBuffer[ZSTD_FRAMEHEADERSIZE_MAX];
-};  /* typedef'd to ZSTD_DCtx within "zstd.h" */
-
-
-/*-*******************************************************
- *  Shared internal functions
- *********************************************************/
-
-/*! ZSTD_loadDEntropy() :
- *  dict : must point at beginning of a valid zstd dictionary.
- * @return : size of entropy tables read */
-size_t ZSTD_loadDEntropy(ZSTD_entropyDTables_t* entropy,
-                   const void* const dict, size_t const dictSize);
-
-/*! ZSTD_checkContinuity() :
- *  check if next `dst` follows previous position, where decompression ended.
- *  If yes, do nothing (continue on current segment).
- *  If not, classify previous segment as "external dictionary", and start a new segment.
- *  This function cannot fail. */
-void ZSTD_checkContinuity(ZSTD_DCtx* dctx, const void* dst);
-
-
-#endif /* ZSTD_DECOMPRESS_INTERNAL_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_double_fast.c b/vendor/github.com/DataDog/zstd/zstd_double_fast.c
deleted file mode 100644
index 5957255..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_double_fast.c
+++ /dev/null
@@ -1,519 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#include "zstd_compress_internal.h"
-#include "zstd_double_fast.h"
-
-
-void ZSTD_fillDoubleHashTable(ZSTD_matchState_t* ms,
-                              void const* end, ZSTD_dictTableLoadMethod_e dtlm)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashLarge = ms->hashTable;
-    U32  const hBitsL = cParams->hashLog;
-    U32  const mls = cParams->minMatch;
-    U32* const hashSmall = ms->chainTable;
-    U32  const hBitsS = cParams->chainLog;
-    const BYTE* const base = ms->window.base;
-    const BYTE* ip = base + ms->nextToUpdate;
-    const BYTE* const iend = ((const BYTE*)end) - HASH_READ_SIZE;
-    const U32 fastHashFillStep = 3;
-
-    /* Always insert every fastHashFillStep position into the hash tables.
-     * Insert the other positions into the large hash table if their entry
-     * is empty.
-     */
-    for (; ip + fastHashFillStep - 1 <= iend; ip += fastHashFillStep) {
-        U32 const current = (U32)(ip - base);
-        U32 i;
-        for (i = 0; i < fastHashFillStep; ++i) {
-            size_t const smHash = ZSTD_hashPtr(ip + i, hBitsS, mls);
-            size_t const lgHash = ZSTD_hashPtr(ip + i, hBitsL, 8);
-            if (i == 0)
-                hashSmall[smHash] = current + i;
-            if (i == 0 || hashLarge[lgHash] == 0)
-                hashLarge[lgHash] = current + i;
-            /* Only load extra positions for ZSTD_dtlm_full */
-            if (dtlm == ZSTD_dtlm_fast)
-                break;
-    }   }
-}
-
-
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_compressBlock_doubleFast_generic(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize,
-        U32 const mls /* template */, ZSTD_dictMode_e const dictMode)
-{
-    ZSTD_compressionParameters const* cParams = &ms->cParams;
-    U32* const hashLong = ms->hashTable;
-    const U32 hBitsL = cParams->hashLog;
-    U32* const hashSmall = ms->chainTable;
-    const U32 hBitsS = cParams->chainLog;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const U32 endIndex = (U32)((size_t)(istart - base) + srcSize);
-    const U32 lowestValid = ms->window.dictLimit;
-    const U32 maxDistance = 1U << cParams->windowLog;
-    const U32 prefixLowestIndex = (endIndex - lowestValid > maxDistance) ? endIndex - maxDistance : lowestValid;
-    const BYTE* const prefixLowest = base + prefixLowestIndex;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - HASH_READ_SIZE;
-    U32 offset_1=rep[0], offset_2=rep[1];
-    U32 offsetSaved = 0;
-
-    const ZSTD_matchState_t* const dms = ms->dictMatchState;
-    const ZSTD_compressionParameters* const dictCParams =
-                                     dictMode == ZSTD_dictMatchState ?
-                                     &dms->cParams : NULL;
-    const U32* const dictHashLong  = dictMode == ZSTD_dictMatchState ?
-                                     dms->hashTable : NULL;
-    const U32* const dictHashSmall = dictMode == ZSTD_dictMatchState ?
-                                     dms->chainTable : NULL;
-    const U32 dictStartIndex       = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.dictLimit : 0;
-    const BYTE* const dictBase     = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.base : NULL;
-    const BYTE* const dictStart    = dictMode == ZSTD_dictMatchState ?
-                                     dictBase + dictStartIndex : NULL;
-    const BYTE* const dictEnd      = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.nextSrc : NULL;
-    const U32 dictIndexDelta       = dictMode == ZSTD_dictMatchState ?
-                                     prefixLowestIndex - (U32)(dictEnd - dictBase) :
-                                     0;
-    const U32 dictHBitsL           = dictMode == ZSTD_dictMatchState ?
-                                     dictCParams->hashLog : hBitsL;
-    const U32 dictHBitsS           = dictMode == ZSTD_dictMatchState ?
-                                     dictCParams->chainLog : hBitsS;
-    const U32 dictAndPrefixLength  = (U32)(ip - prefixLowest + dictEnd - dictStart);
-
-    DEBUGLOG(5, "ZSTD_compressBlock_doubleFast_generic");
-
-    assert(dictMode == ZSTD_noDict || dictMode == ZSTD_dictMatchState);
-
-    /* if a dictionary is attached, it must be within window range */
-    if (dictMode == ZSTD_dictMatchState) {
-        assert(lowestValid + maxDistance >= endIndex);
-    }
-
-    /* init */
-    ip += (dictAndPrefixLength == 0);
-    if (dictMode == ZSTD_noDict) {
-        U32 const maxRep = (U32)(ip - prefixLowest);
-        if (offset_2 > maxRep) offsetSaved = offset_2, offset_2 = 0;
-        if (offset_1 > maxRep) offsetSaved = offset_1, offset_1 = 0;
-    }
-    if (dictMode == ZSTD_dictMatchState) {
-        /* dictMatchState repCode checks don't currently handle repCode == 0
-         * disabling. */
-        assert(offset_1 <= dictAndPrefixLength);
-        assert(offset_2 <= dictAndPrefixLength);
-    }
-
-    /* Main Search Loop */
-    while (ip < ilimit) {   /* < instead of <=, because repcode check at (ip+1) */
-        size_t mLength;
-        U32 offset;
-        size_t const h2 = ZSTD_hashPtr(ip, hBitsL, 8);
-        size_t const h = ZSTD_hashPtr(ip, hBitsS, mls);
-        size_t const dictHL = ZSTD_hashPtr(ip, dictHBitsL, 8);
-        size_t const dictHS = ZSTD_hashPtr(ip, dictHBitsS, mls);
-        U32 const current = (U32)(ip-base);
-        U32 const matchIndexL = hashLong[h2];
-        U32 matchIndexS = hashSmall[h];
-        const BYTE* matchLong = base + matchIndexL;
-        const BYTE* match = base + matchIndexS;
-        const U32 repIndex = current + 1 - offset_1;
-        const BYTE* repMatch = (dictMode == ZSTD_dictMatchState
-                            && repIndex < prefixLowestIndex) ?
-                               dictBase + (repIndex - dictIndexDelta) :
-                               base + repIndex;
-        hashLong[h2] = hashSmall[h] = current;   /* update hash tables */
-
-        /* check dictMatchState repcode */
-        if (dictMode == ZSTD_dictMatchState
-            && ((U32)((prefixLowestIndex-1) - repIndex) >= 3 /* intentional underflow */)
-            && (MEM_read32(repMatch) == MEM_read32(ip+1)) ) {
-            const BYTE* repMatchEnd = repIndex < prefixLowestIndex ? dictEnd : iend;
-            mLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repMatchEnd, prefixLowest) + 4;
-            ip++;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, 0, mLength-MINMATCH);
-            goto _match_stored;
-        }
-
-        /* check noDict repcode */
-        if ( dictMode == ZSTD_noDict
-          && ((offset_1 > 0) & (MEM_read32(ip+1-offset_1) == MEM_read32(ip+1)))) {
-            mLength = ZSTD_count(ip+1+4, ip+1+4-offset_1, iend) + 4;
-            ip++;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, 0, mLength-MINMATCH);
-            goto _match_stored;
-        }
-
-        if (matchIndexL > prefixLowestIndex) {
-            /* check prefix long match */
-            if (MEM_read64(matchLong) == MEM_read64(ip)) {
-                mLength = ZSTD_count(ip+8, matchLong+8, iend) + 8;
-                offset = (U32)(ip-matchLong);
-                while (((ip>anchor) & (matchLong>prefixLowest)) && (ip[-1] == matchLong[-1])) { ip--; matchLong--; mLength++; } /* catch up */
-                goto _match_found;
-            }
-        } else if (dictMode == ZSTD_dictMatchState) {
-            /* check dictMatchState long match */
-            U32 const dictMatchIndexL = dictHashLong[dictHL];
-            const BYTE* dictMatchL = dictBase + dictMatchIndexL;
-            assert(dictMatchL < dictEnd);
-
-            if (dictMatchL > dictStart && MEM_read64(dictMatchL) == MEM_read64(ip)) {
-                mLength = ZSTD_count_2segments(ip+8, dictMatchL+8, iend, dictEnd, prefixLowest) + 8;
-                offset = (U32)(current - dictMatchIndexL - dictIndexDelta);
-                while (((ip>anchor) & (dictMatchL>dictStart)) && (ip[-1] == dictMatchL[-1])) { ip--; dictMatchL--; mLength++; } /* catch up */
-                goto _match_found;
-        }   }
-
-        if (matchIndexS > prefixLowestIndex) {
-            /* check prefix short match */
-            if (MEM_read32(match) == MEM_read32(ip)) {
-                goto _search_next_long;
-            }
-        } else if (dictMode == ZSTD_dictMatchState) {
-            /* check dictMatchState short match */
-            U32 const dictMatchIndexS = dictHashSmall[dictHS];
-            match = dictBase + dictMatchIndexS;
-            matchIndexS = dictMatchIndexS + dictIndexDelta;
-
-            if (match > dictStart && MEM_read32(match) == MEM_read32(ip)) {
-                goto _search_next_long;
-        }   }
-
-        ip += ((ip-anchor) >> kSearchStrength) + 1;
-        continue;
-
-_search_next_long:
-
-        {   size_t const hl3 = ZSTD_hashPtr(ip+1, hBitsL, 8);
-            size_t const dictHLNext = ZSTD_hashPtr(ip+1, dictHBitsL, 8);
-            U32 const matchIndexL3 = hashLong[hl3];
-            const BYTE* matchL3 = base + matchIndexL3;
-            hashLong[hl3] = current + 1;
-
-            /* check prefix long +1 match */
-            if (matchIndexL3 > prefixLowestIndex) {
-                if (MEM_read64(matchL3) == MEM_read64(ip+1)) {
-                    mLength = ZSTD_count(ip+9, matchL3+8, iend) + 8;
-                    ip++;
-                    offset = (U32)(ip-matchL3);
-                    while (((ip>anchor) & (matchL3>prefixLowest)) && (ip[-1] == matchL3[-1])) { ip--; matchL3--; mLength++; } /* catch up */
-                    goto _match_found;
-                }
-            } else if (dictMode == ZSTD_dictMatchState) {
-                /* check dict long +1 match */
-                U32 const dictMatchIndexL3 = dictHashLong[dictHLNext];
-                const BYTE* dictMatchL3 = dictBase + dictMatchIndexL3;
-                assert(dictMatchL3 < dictEnd);
-                if (dictMatchL3 > dictStart && MEM_read64(dictMatchL3) == MEM_read64(ip+1)) {
-                    mLength = ZSTD_count_2segments(ip+1+8, dictMatchL3+8, iend, dictEnd, prefixLowest) + 8;
-                    ip++;
-                    offset = (U32)(current + 1 - dictMatchIndexL3 - dictIndexDelta);
-                    while (((ip>anchor) & (dictMatchL3>dictStart)) && (ip[-1] == dictMatchL3[-1])) { ip--; dictMatchL3--; mLength++; } /* catch up */
-                    goto _match_found;
-        }   }   }
-
-        /* if no long +1 match, explore the short match we found */
-        if (dictMode == ZSTD_dictMatchState && matchIndexS < prefixLowestIndex) {
-            mLength = ZSTD_count_2segments(ip+4, match+4, iend, dictEnd, prefixLowest) + 4;
-            offset = (U32)(current - matchIndexS);
-            while (((ip>anchor) & (match>dictStart)) && (ip[-1] == match[-1])) { ip--; match--; mLength++; } /* catch up */
-        } else {
-            mLength = ZSTD_count(ip+4, match+4, iend) + 4;
-            offset = (U32)(ip - match);
-            while (((ip>anchor) & (match>prefixLowest)) && (ip[-1] == match[-1])) { ip--; match--; mLength++; } /* catch up */
-        }
-
-        /* fall-through */
-
-_match_found:
-        offset_2 = offset_1;
-        offset_1 = offset;
-
-        ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-
-_match_stored:
-        /* match found */
-        ip += mLength;
-        anchor = ip;
-
-        if (ip <= ilimit) {
-            /* Complementary insertion */
-            /* done after iLimit test, as candidates could be > iend-8 */
-            {   U32 const indexToInsert = current+2;
-                hashLong[ZSTD_hashPtr(base+indexToInsert, hBitsL, 8)] = indexToInsert;
-                hashLong[ZSTD_hashPtr(ip-2, hBitsL, 8)] = (U32)(ip-2-base);
-                hashSmall[ZSTD_hashPtr(base+indexToInsert, hBitsS, mls)] = indexToInsert;
-                hashSmall[ZSTD_hashPtr(ip-1, hBitsS, mls)] = (U32)(ip-1-base);
-            }
-
-            /* check immediate repcode */
-            if (dictMode == ZSTD_dictMatchState) {
-                while (ip <= ilimit) {
-                    U32 const current2 = (U32)(ip-base);
-                    U32 const repIndex2 = current2 - offset_2;
-                    const BYTE* repMatch2 = dictMode == ZSTD_dictMatchState
-                        && repIndex2 < prefixLowestIndex ?
-                            dictBase - dictIndexDelta + repIndex2 :
-                            base + repIndex2;
-                    if ( ((U32)((prefixLowestIndex-1) - (U32)repIndex2) >= 3 /* intentional overflow */)
-                       && (MEM_read32(repMatch2) == MEM_read32(ip)) ) {
-                        const BYTE* const repEnd2 = repIndex2 < prefixLowestIndex ? dictEnd : iend;
-                        size_t const repLength2 = ZSTD_count_2segments(ip+4, repMatch2+4, iend, repEnd2, prefixLowest) + 4;
-                        U32 tmpOffset = offset_2; offset_2 = offset_1; offset_1 = tmpOffset;   /* swap offset_2 <=> offset_1 */
-                        ZSTD_storeSeq(seqStore, 0, anchor, 0, repLength2-MINMATCH);
-                        hashSmall[ZSTD_hashPtr(ip, hBitsS, mls)] = current2;
-                        hashLong[ZSTD_hashPtr(ip, hBitsL, 8)] = current2;
-                        ip += repLength2;
-                        anchor = ip;
-                        continue;
-                    }
-                    break;
-            }   }
-
-            if (dictMode == ZSTD_noDict) {
-                while ( (ip <= ilimit)
-                     && ( (offset_2>0)
-                        & (MEM_read32(ip) == MEM_read32(ip - offset_2)) )) {
-                    /* store sequence */
-                    size_t const rLength = ZSTD_count(ip+4, ip+4-offset_2, iend) + 4;
-                    U32 const tmpOff = offset_2; offset_2 = offset_1; offset_1 = tmpOff;  /* swap offset_2 <=> offset_1 */
-                    hashSmall[ZSTD_hashPtr(ip, hBitsS, mls)] = (U32)(ip-base);
-                    hashLong[ZSTD_hashPtr(ip, hBitsL, 8)] = (U32)(ip-base);
-                    ZSTD_storeSeq(seqStore, 0, anchor, 0, rLength-MINMATCH);
-                    ip += rLength;
-                    anchor = ip;
-                    continue;   /* faster when present ... (?) */
-        }   }   }
-    }   /* while (ip < ilimit) */
-
-    /* save reps for next block */
-    rep[0] = offset_1 ? offset_1 : offsetSaved;
-    rep[1] = offset_2 ? offset_2 : offsetSaved;
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-
-size_t ZSTD_compressBlock_doubleFast(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    const U32 mls = ms->cParams.minMatch;
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 4, ZSTD_noDict);
-    case 5 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 5, ZSTD_noDict);
-    case 6 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 6, ZSTD_noDict);
-    case 7 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 7, ZSTD_noDict);
-    }
-}
-
-
-size_t ZSTD_compressBlock_doubleFast_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    const U32 mls = ms->cParams.minMatch;
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 4, ZSTD_dictMatchState);
-    case 5 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 5, ZSTD_dictMatchState);
-    case 6 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 6, ZSTD_dictMatchState);
-    case 7 :
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, 7, ZSTD_dictMatchState);
-    }
-}
-
-
-static size_t ZSTD_compressBlock_doubleFast_extDict_generic(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize,
-        U32 const mls /* template */)
-{
-    ZSTD_compressionParameters const* cParams = &ms->cParams;
-    U32* const hashLong = ms->hashTable;
-    U32  const hBitsL = cParams->hashLog;
-    U32* const hashSmall = ms->chainTable;
-    U32  const hBitsS = cParams->chainLog;
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - 8;
-    const BYTE* const base = ms->window.base;
-    const U32   endIndex = (U32)((size_t)(istart - base) + srcSize);
-    const U32   maxDistance = 1U << cParams->windowLog;
-    const U32   lowestValid = ms->window.lowLimit;
-    const U32   lowLimit = (endIndex - lowestValid > maxDistance) ? endIndex - maxDistance : lowestValid;
-    const U32   dictStartIndex = lowLimit;
-    const U32   dictLimit = ms->window.dictLimit;
-    const U32   prefixStartIndex = (dictLimit > lowLimit) ? dictLimit : lowLimit;
-    const BYTE* const prefixStart = base + prefixStartIndex;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const BYTE* const dictStart = dictBase + dictStartIndex;
-    const BYTE* const dictEnd = dictBase + prefixStartIndex;
-    U32 offset_1=rep[0], offset_2=rep[1];
-
-    DEBUGLOG(5, "ZSTD_compressBlock_doubleFast_extDict_generic (srcSize=%zu)", srcSize);
-
-    /* if extDict is invalidated due to maxDistance, switch to "regular" variant */
-    if (prefixStartIndex == dictStartIndex)
-        return ZSTD_compressBlock_doubleFast_generic(ms, seqStore, rep, src, srcSize, mls, ZSTD_noDict);
-
-    /* Search Loop */
-    while (ip < ilimit) {  /* < instead of <=, because (ip+1) */
-        const size_t hSmall = ZSTD_hashPtr(ip, hBitsS, mls);
-        const U32 matchIndex = hashSmall[hSmall];
-        const BYTE* const matchBase = matchIndex < prefixStartIndex ? dictBase : base;
-        const BYTE* match = matchBase + matchIndex;
-
-        const size_t hLong = ZSTD_hashPtr(ip, hBitsL, 8);
-        const U32 matchLongIndex = hashLong[hLong];
-        const BYTE* const matchLongBase = matchLongIndex < prefixStartIndex ? dictBase : base;
-        const BYTE* matchLong = matchLongBase + matchLongIndex;
-
-        const U32 current = (U32)(ip-base);
-        const U32 repIndex = current + 1 - offset_1;   /* offset_1 expected <= current +1 */
-        const BYTE* const repBase = repIndex < prefixStartIndex ? dictBase : base;
-        const BYTE* const repMatch = repBase + repIndex;
-        size_t mLength;
-        hashSmall[hSmall] = hashLong[hLong] = current;   /* update hash table */
-
-        if ((((U32)((prefixStartIndex-1) - repIndex) >= 3) /* intentional underflow : ensure repIndex doesn't overlap dict + prefix */
-            & (repIndex > dictStartIndex))
-          && (MEM_read32(repMatch) == MEM_read32(ip+1)) ) {
-            const BYTE* repMatchEnd = repIndex < prefixStartIndex ? dictEnd : iend;
-            mLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repMatchEnd, prefixStart) + 4;
-            ip++;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, 0, mLength-MINMATCH);
-        } else {
-            if ((matchLongIndex > dictStartIndex) && (MEM_read64(matchLong) == MEM_read64(ip))) {
-                const BYTE* const matchEnd = matchLongIndex < prefixStartIndex ? dictEnd : iend;
-                const BYTE* const lowMatchPtr = matchLongIndex < prefixStartIndex ? dictStart : prefixStart;
-                U32 offset;
-                mLength = ZSTD_count_2segments(ip+8, matchLong+8, iend, matchEnd, prefixStart) + 8;
-                offset = current - matchLongIndex;
-                while (((ip>anchor) & (matchLong>lowMatchPtr)) && (ip[-1] == matchLong[-1])) { ip--; matchLong--; mLength++; }   /* catch up */
-                offset_2 = offset_1;
-                offset_1 = offset;
-                ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-
-            } else if ((matchIndex > dictStartIndex) && (MEM_read32(match) == MEM_read32(ip))) {
-                size_t const h3 = ZSTD_hashPtr(ip+1, hBitsL, 8);
-                U32 const matchIndex3 = hashLong[h3];
-                const BYTE* const match3Base = matchIndex3 < prefixStartIndex ? dictBase : base;
-                const BYTE* match3 = match3Base + matchIndex3;
-                U32 offset;
-                hashLong[h3] = current + 1;
-                if ( (matchIndex3 > dictStartIndex) && (MEM_read64(match3) == MEM_read64(ip+1)) ) {
-                    const BYTE* const matchEnd = matchIndex3 < prefixStartIndex ? dictEnd : iend;
-                    const BYTE* const lowMatchPtr = matchIndex3 < prefixStartIndex ? dictStart : prefixStart;
-                    mLength = ZSTD_count_2segments(ip+9, match3+8, iend, matchEnd, prefixStart) + 8;
-                    ip++;
-                    offset = current+1 - matchIndex3;
-                    while (((ip>anchor) & (match3>lowMatchPtr)) && (ip[-1] == match3[-1])) { ip--; match3--; mLength++; } /* catch up */
-                } else {
-                    const BYTE* const matchEnd = matchIndex < prefixStartIndex ? dictEnd : iend;
-                    const BYTE* const lowMatchPtr = matchIndex < prefixStartIndex ? dictStart : prefixStart;
-                    mLength = ZSTD_count_2segments(ip+4, match+4, iend, matchEnd, prefixStart) + 4;
-                    offset = current - matchIndex;
-                    while (((ip>anchor) & (match>lowMatchPtr)) && (ip[-1] == match[-1])) { ip--; match--; mLength++; }   /* catch up */
-                }
-                offset_2 = offset_1;
-                offset_1 = offset;
-                ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-
-            } else {
-                ip += ((ip-anchor) >> kSearchStrength) + 1;
-                continue;
-        }   }
-
-        /* move to next sequence start */
-        ip += mLength;
-        anchor = ip;
-
-        if (ip <= ilimit) {
-            /* Complementary insertion */
-            /* done after iLimit test, as candidates could be > iend-8 */
-            {   U32 const indexToInsert = current+2;
-                hashLong[ZSTD_hashPtr(base+indexToInsert, hBitsL, 8)] = indexToInsert;
-                hashLong[ZSTD_hashPtr(ip-2, hBitsL, 8)] = (U32)(ip-2-base);
-                hashSmall[ZSTD_hashPtr(base+indexToInsert, hBitsS, mls)] = indexToInsert;
-                hashSmall[ZSTD_hashPtr(ip-1, hBitsS, mls)] = (U32)(ip-1-base);
-            }
-
-            /* check immediate repcode */
-            while (ip <= ilimit) {
-                U32 const current2 = (U32)(ip-base);
-                U32 const repIndex2 = current2 - offset_2;
-                const BYTE* repMatch2 = repIndex2 < prefixStartIndex ? dictBase + repIndex2 : base + repIndex2;
-                if ( (((U32)((prefixStartIndex-1) - repIndex2) >= 3)   /* intentional overflow : ensure repIndex2 doesn't overlap dict + prefix */
-                    & (repIndex2 > dictStartIndex))
-                  && (MEM_read32(repMatch2) == MEM_read32(ip)) ) {
-                    const BYTE* const repEnd2 = repIndex2 < prefixStartIndex ? dictEnd : iend;
-                    size_t const repLength2 = ZSTD_count_2segments(ip+4, repMatch2+4, iend, repEnd2, prefixStart) + 4;
-                    U32 const tmpOffset = offset_2; offset_2 = offset_1; offset_1 = tmpOffset;   /* swap offset_2 <=> offset_1 */
-                    ZSTD_storeSeq(seqStore, 0, anchor, 0, repLength2-MINMATCH);
-                    hashSmall[ZSTD_hashPtr(ip, hBitsS, mls)] = current2;
-                    hashLong[ZSTD_hashPtr(ip, hBitsL, 8)] = current2;
-                    ip += repLength2;
-                    anchor = ip;
-                    continue;
-                }
-                break;
-    }   }   }
-
-    /* save reps for next block */
-    rep[0] = offset_1;
-    rep[1] = offset_2;
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-
-size_t ZSTD_compressBlock_doubleFast_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    U32 const mls = ms->cParams.minMatch;
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_doubleFast_extDict_generic(ms, seqStore, rep, src, srcSize, 4);
-    case 5 :
-        return ZSTD_compressBlock_doubleFast_extDict_generic(ms, seqStore, rep, src, srcSize, 5);
-    case 6 :
-        return ZSTD_compressBlock_doubleFast_extDict_generic(ms, seqStore, rep, src, srcSize, 6);
-    case 7 :
-        return ZSTD_compressBlock_doubleFast_extDict_generic(ms, seqStore, rep, src, srcSize, 7);
-    }
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_double_fast.h b/vendor/github.com/DataDog/zstd/zstd_double_fast.h
deleted file mode 100644
index 4fa31ac..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_double_fast.h
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_DOUBLE_FAST_H
-#define ZSTD_DOUBLE_FAST_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#include "mem.h"      /* U32 */
-#include "zstd_compress_internal.h"     /* ZSTD_CCtx, size_t */
-
-void ZSTD_fillDoubleHashTable(ZSTD_matchState_t* ms,
-                              void const* end, ZSTD_dictTableLoadMethod_e dtlm);
-size_t ZSTD_compressBlock_doubleFast(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_doubleFast_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_doubleFast_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_DOUBLE_FAST_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_errors.h b/vendor/github.com/DataDog/zstd/zstd_errors.h
deleted file mode 100644
index 92a3433..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_errors.h
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_ERRORS_H_398273423
-#define ZSTD_ERRORS_H_398273423
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*===== dependency =====*/
-#include <stddef.h>   /* size_t */
-
-
-/* =====   ZSTDERRORLIB_API : control library symbols visibility   ===== */
-#ifndef ZSTDERRORLIB_VISIBILITY
-#  if defined(__GNUC__) && (__GNUC__ >= 4)
-#    define ZSTDERRORLIB_VISIBILITY __attribute__ ((visibility ("default")))
-#  else
-#    define ZSTDERRORLIB_VISIBILITY
-#  endif
-#endif
-#if defined(ZSTD_DLL_EXPORT) && (ZSTD_DLL_EXPORT==1)
-#  define ZSTDERRORLIB_API __declspec(dllexport) ZSTDERRORLIB_VISIBILITY
-#elif defined(ZSTD_DLL_IMPORT) && (ZSTD_DLL_IMPORT==1)
-#  define ZSTDERRORLIB_API __declspec(dllimport) ZSTDERRORLIB_VISIBILITY /* It isn't required but allows to generate better code, saving a function pointer load from the IAT and an indirect jump.*/
-#else
-#  define ZSTDERRORLIB_API ZSTDERRORLIB_VISIBILITY
-#endif
-
-/*-*********************************************
- *  Error codes list
- *-*********************************************
- *  Error codes _values_ are pinned down since v1.3.1 only.
- *  Therefore, don't rely on values if you may link to any version < v1.3.1.
- *
- *  Only values < 100 are considered stable.
- *
- *  note 1 : this API shall be used with static linking only.
- *           dynamic linking is not yet officially supported.
- *  note 2 : Prefer relying on the enum than on its value whenever possible
- *           This is the only supported way to use the error list < v1.3.1
- *  note 3 : ZSTD_isError() is always correct, whatever the library version.
- **********************************************/
-typedef enum {
-  ZSTD_error_no_error = 0,
-  ZSTD_error_GENERIC  = 1,
-  ZSTD_error_prefix_unknown                = 10,
-  ZSTD_error_version_unsupported           = 12,
-  ZSTD_error_frameParameter_unsupported    = 14,
-  ZSTD_error_frameParameter_windowTooLarge = 16,
-  ZSTD_error_corruption_detected = 20,
-  ZSTD_error_checksum_wrong      = 22,
-  ZSTD_error_dictionary_corrupted      = 30,
-  ZSTD_error_dictionary_wrong          = 32,
-  ZSTD_error_dictionaryCreation_failed = 34,
-  ZSTD_error_parameter_unsupported   = 40,
-  ZSTD_error_parameter_outOfBound    = 42,
-  ZSTD_error_tableLog_tooLarge       = 44,
-  ZSTD_error_maxSymbolValue_tooLarge = 46,
-  ZSTD_error_maxSymbolValue_tooSmall = 48,
-  ZSTD_error_stage_wrong       = 60,
-  ZSTD_error_init_missing      = 62,
-  ZSTD_error_memory_allocation = 64,
-  ZSTD_error_workSpace_tooSmall= 66,
-  ZSTD_error_dstSize_tooSmall = 70,
-  ZSTD_error_srcSize_wrong    = 72,
-  ZSTD_error_dstBuffer_null   = 74,
-  /* following error codes are __NOT STABLE__, they can be removed or changed in future versions */
-  ZSTD_error_frameIndex_tooLarge = 100,
-  ZSTD_error_seekableIO          = 102,
-  ZSTD_error_maxCode = 120  /* never EVER use this value directly, it can change in future versions! Use ZSTD_isError() instead */
-} ZSTD_ErrorCode;
-
-/*! ZSTD_getErrorCode() :
-    convert a `size_t` function result into a `ZSTD_ErrorCode` enum type,
-    which can be used to compare with enum list published above */
-ZSTDERRORLIB_API ZSTD_ErrorCode ZSTD_getErrorCode(size_t functionResult);
-ZSTDERRORLIB_API const char* ZSTD_getErrorString(ZSTD_ErrorCode code);   /**< Same as ZSTD_getErrorName, but using a `ZSTD_ErrorCode` enum argument */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_ERRORS_H_398273423 */
diff --git a/vendor/github.com/DataDog/zstd/zstd_fast.c b/vendor/github.com/DataDog/zstd/zstd_fast.c
deleted file mode 100644
index a05b8a4..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_fast.c
+++ /dev/null
@@ -1,491 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#include "zstd_compress_internal.h"
-#include "zstd_fast.h"
-
-
-void ZSTD_fillHashTable(ZSTD_matchState_t* ms,
-                        const void* const end,
-                        ZSTD_dictTableLoadMethod_e dtlm)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashTable = ms->hashTable;
-    U32  const hBits = cParams->hashLog;
-    U32  const mls = cParams->minMatch;
-    const BYTE* const base = ms->window.base;
-    const BYTE* ip = base + ms->nextToUpdate;
-    const BYTE* const iend = ((const BYTE*)end) - HASH_READ_SIZE;
-    const U32 fastHashFillStep = 3;
-
-    /* Always insert every fastHashFillStep position into the hash table.
-     * Insert the other positions if their hash entry is empty.
-     */
-    for ( ; ip + fastHashFillStep < iend + 2; ip += fastHashFillStep) {
-        U32 const current = (U32)(ip - base);
-        size_t const hash0 = ZSTD_hashPtr(ip, hBits, mls);
-        hashTable[hash0] = current;
-        if (dtlm == ZSTD_dtlm_fast) continue;
-        /* Only load extra positions for ZSTD_dtlm_full */
-        {   U32 p;
-            for (p = 1; p < fastHashFillStep; ++p) {
-                size_t const hash = ZSTD_hashPtr(ip + p, hBits, mls);
-                if (hashTable[hash] == 0) {  /* not yet filled */
-                    hashTable[hash] = current + p;
-    }   }   }   }
-}
-
-
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_compressBlock_fast_generic(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize,
-        U32 const mls)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashTable = ms->hashTable;
-    U32 const hlog = cParams->hashLog;
-    /* support stepSize of 0 */
-    size_t const stepSize = cParams->targetLength + !(cParams->targetLength) + 1;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const istart = (const BYTE*)src;
-    /* We check ip0 (ip + 0) and ip1 (ip + 1) each loop */
-    const BYTE* ip0 = istart;
-    const BYTE* ip1;
-    const BYTE* anchor = istart;
-    const U32   endIndex = (U32)((size_t)(istart - base) + srcSize);
-    const U32   maxDistance = 1U << cParams->windowLog;
-    const U32   validStartIndex = ms->window.dictLimit;
-    const U32   prefixStartIndex = (endIndex - validStartIndex > maxDistance) ? endIndex - maxDistance : validStartIndex;
-    const BYTE* const prefixStart = base + prefixStartIndex;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - HASH_READ_SIZE;
-    U32 offset_1=rep[0], offset_2=rep[1];
-    U32 offsetSaved = 0;
-
-    /* init */
-    ip0 += (ip0 == prefixStart);
-    ip1 = ip0 + 1;
-    {
-        U32 const maxRep = (U32)(ip0 - prefixStart);
-        if (offset_2 > maxRep) offsetSaved = offset_2, offset_2 = 0;
-        if (offset_1 > maxRep) offsetSaved = offset_1, offset_1 = 0;
-    }
-
-    /* Main Search Loop */
-    while (ip1 < ilimit) {   /* < instead of <=, because check at ip0+2 */
-        size_t mLength;
-        BYTE const* ip2 = ip0 + 2;
-        size_t const h0 = ZSTD_hashPtr(ip0, hlog, mls);
-        U32 const val0 = MEM_read32(ip0);
-        size_t const h1 = ZSTD_hashPtr(ip1, hlog, mls);
-        U32 const val1 = MEM_read32(ip1);
-        U32 const current0 = (U32)(ip0-base);
-        U32 const current1 = (U32)(ip1-base);
-        U32 const matchIndex0 = hashTable[h0];
-        U32 const matchIndex1 = hashTable[h1];
-        BYTE const* repMatch = ip2-offset_1;
-        const BYTE* match0 = base + matchIndex0;
-        const BYTE* match1 = base + matchIndex1;
-        U32 offcode;
-        hashTable[h0] = current0;   /* update hash table */
-        hashTable[h1] = current1;   /* update hash table */
-
-        assert(ip0 + 1 == ip1);
-
-        if ((offset_1 > 0) & (MEM_read32(repMatch) == MEM_read32(ip2))) {
-            mLength = ip2[-1] == repMatch[-1] ? 1 : 0;
-            ip0 = ip2 - mLength;
-            match0 = repMatch - mLength;
-            offcode = 0;
-            goto _match;
-        }
-        if ((matchIndex0 > prefixStartIndex) && MEM_read32(match0) == val0) {
-            /* found a regular match */
-            goto _offset;
-        }
-        if ((matchIndex1 > prefixStartIndex) && MEM_read32(match1) == val1) {
-            /* found a regular match after one literal */
-            ip0 = ip1;
-            match0 = match1;
-            goto _offset;
-        }
-        {
-            size_t const step = ((ip0-anchor) >> (kSearchStrength - 1)) + stepSize;
-            assert(step >= 2);
-            ip0 += step;
-            ip1 += step;
-            continue;
-        }
-_offset: /* Requires: ip0, match0 */
-        /* Compute the offset code */
-        offset_2 = offset_1;
-        offset_1 = (U32)(ip0-match0);
-        offcode = offset_1 + ZSTD_REP_MOVE;
-        mLength = 0;
-        /* Count the backwards match length */
-        while (((ip0>anchor) & (match0>prefixStart))
-             && (ip0[-1] == match0[-1])) { ip0--; match0--; mLength++; } /* catch up */
-
-_match: /* Requires: ip0, match0, offcode */
-        /* Count the forward length */
-        mLength += ZSTD_count(ip0+mLength+4, match0+mLength+4, iend) + 4;
-        ZSTD_storeSeq(seqStore, ip0-anchor, anchor, offcode, mLength-MINMATCH);
-        /* match found */
-        ip0 += mLength;
-        anchor = ip0;
-        ip1 = ip0 + 1;
-
-        if (ip0 <= ilimit) {
-            /* Fill Table */
-            assert(base+current0+2 > istart);  /* check base overflow */
-            hashTable[ZSTD_hashPtr(base+current0+2, hlog, mls)] = current0+2;  /* here because current+2 could be > iend-8 */
-            hashTable[ZSTD_hashPtr(ip0-2, hlog, mls)] = (U32)(ip0-2-base);
-
-            while ( (ip0 <= ilimit)
-                 && ( (offset_2>0)
-                    & (MEM_read32(ip0) == MEM_read32(ip0 - offset_2)) )) {
-                /* store sequence */
-                size_t const rLength = ZSTD_count(ip0+4, ip0+4-offset_2, iend) + 4;
-                U32 const tmpOff = offset_2; offset_2 = offset_1; offset_1 = tmpOff;  /* swap offset_2 <=> offset_1 */
-                hashTable[ZSTD_hashPtr(ip0, hlog, mls)] = (U32)(ip0-base);
-                ip0 += rLength;
-                ip1 = ip0 + 1;
-                ZSTD_storeSeq(seqStore, 0, anchor, 0, rLength-MINMATCH);
-                anchor = ip0;
-                continue;   /* faster when present (confirmed on gcc-8) ... (?) */
-            }
-        }
-    }
-
-    /* save reps for next block */
-    rep[0] = offset_1 ? offset_1 : offsetSaved;
-    rep[1] = offset_2 ? offset_2 : offsetSaved;
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-
-size_t ZSTD_compressBlock_fast(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    ZSTD_compressionParameters const* cParams = &ms->cParams;
-    U32 const mls = cParams->minMatch;
-    assert(ms->dictMatchState == NULL);
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_fast_generic(ms, seqStore, rep, src, srcSize, 4);
-    case 5 :
-        return ZSTD_compressBlock_fast_generic(ms, seqStore, rep, src, srcSize, 5);
-    case 6 :
-        return ZSTD_compressBlock_fast_generic(ms, seqStore, rep, src, srcSize, 6);
-    case 7 :
-        return ZSTD_compressBlock_fast_generic(ms, seqStore, rep, src, srcSize, 7);
-    }
-}
-
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_compressBlock_fast_dictMatchState_generic(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize, U32 const mls)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashTable = ms->hashTable;
-    U32 const hlog = cParams->hashLog;
-    /* support stepSize of 0 */
-    U32 const stepSize = cParams->targetLength + !(cParams->targetLength);
-    const BYTE* const base = ms->window.base;
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const U32   prefixStartIndex = ms->window.dictLimit;
-    const BYTE* const prefixStart = base + prefixStartIndex;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - HASH_READ_SIZE;
-    U32 offset_1=rep[0], offset_2=rep[1];
-    U32 offsetSaved = 0;
-
-    const ZSTD_matchState_t* const dms = ms->dictMatchState;
-    const ZSTD_compressionParameters* const dictCParams = &dms->cParams ;
-    const U32* const dictHashTable = dms->hashTable;
-    const U32 dictStartIndex       = dms->window.dictLimit;
-    const BYTE* const dictBase     = dms->window.base;
-    const BYTE* const dictStart    = dictBase + dictStartIndex;
-    const BYTE* const dictEnd      = dms->window.nextSrc;
-    const U32 dictIndexDelta       = prefixStartIndex - (U32)(dictEnd - dictBase);
-    const U32 dictAndPrefixLength  = (U32)(ip - prefixStart + dictEnd - dictStart);
-    const U32 dictHLog             = dictCParams->hashLog;
-
-    /* if a dictionary is still attached, it necessarily means that
-     * it is within window size. So we just check it. */
-    const U32 maxDistance = 1U << cParams->windowLog;
-    const U32 endIndex = (U32)((size_t)(ip - base) + srcSize);
-    assert(endIndex - prefixStartIndex <= maxDistance);
-    (void)maxDistance; (void)endIndex;   /* these variables are not used when assert() is disabled */
-
-    /* ensure there will be no no underflow
-     * when translating a dict index into a local index */
-    assert(prefixStartIndex >= (U32)(dictEnd - dictBase));
-
-    /* init */
-    ip += (dictAndPrefixLength == 0);
-    /* dictMatchState repCode checks don't currently handle repCode == 0
-     * disabling. */
-    assert(offset_1 <= dictAndPrefixLength);
-    assert(offset_2 <= dictAndPrefixLength);
-
-    /* Main Search Loop */
-    while (ip < ilimit) {   /* < instead of <=, because repcode check at (ip+1) */
-        size_t mLength;
-        size_t const h = ZSTD_hashPtr(ip, hlog, mls);
-        U32 const current = (U32)(ip-base);
-        U32 const matchIndex = hashTable[h];
-        const BYTE* match = base + matchIndex;
-        const U32 repIndex = current + 1 - offset_1;
-        const BYTE* repMatch = (repIndex < prefixStartIndex) ?
-                               dictBase + (repIndex - dictIndexDelta) :
-                               base + repIndex;
-        hashTable[h] = current;   /* update hash table */
-
-        if ( ((U32)((prefixStartIndex-1) - repIndex) >= 3) /* intentional underflow : ensure repIndex isn't overlapping dict + prefix */
-          && (MEM_read32(repMatch) == MEM_read32(ip+1)) ) {
-            const BYTE* const repMatchEnd = repIndex < prefixStartIndex ? dictEnd : iend;
-            mLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repMatchEnd, prefixStart) + 4;
-            ip++;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, 0, mLength-MINMATCH);
-        } else if ( (matchIndex <= prefixStartIndex) ) {
-            size_t const dictHash = ZSTD_hashPtr(ip, dictHLog, mls);
-            U32 const dictMatchIndex = dictHashTable[dictHash];
-            const BYTE* dictMatch = dictBase + dictMatchIndex;
-            if (dictMatchIndex <= dictStartIndex ||
-                MEM_read32(dictMatch) != MEM_read32(ip)) {
-                assert(stepSize >= 1);
-                ip += ((ip-anchor) >> kSearchStrength) + stepSize;
-                continue;
-            } else {
-                /* found a dict match */
-                U32 const offset = (U32)(current-dictMatchIndex-dictIndexDelta);
-                mLength = ZSTD_count_2segments(ip+4, dictMatch+4, iend, dictEnd, prefixStart) + 4;
-                while (((ip>anchor) & (dictMatch>dictStart))
-                     && (ip[-1] == dictMatch[-1])) {
-                    ip--; dictMatch--; mLength++;
-                } /* catch up */
-                offset_2 = offset_1;
-                offset_1 = offset;
-                ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-            }
-        } else if (MEM_read32(match) != MEM_read32(ip)) {
-            /* it's not a match, and we're not going to check the dictionary */
-            assert(stepSize >= 1);
-            ip += ((ip-anchor) >> kSearchStrength) + stepSize;
-            continue;
-        } else {
-            /* found a regular match */
-            U32 const offset = (U32)(ip-match);
-            mLength = ZSTD_count(ip+4, match+4, iend) + 4;
-            while (((ip>anchor) & (match>prefixStart))
-                 && (ip[-1] == match[-1])) { ip--; match--; mLength++; } /* catch up */
-            offset_2 = offset_1;
-            offset_1 = offset;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-        }
-
-        /* match found */
-        ip += mLength;
-        anchor = ip;
-
-        if (ip <= ilimit) {
-            /* Fill Table */
-            assert(base+current+2 > istart);  /* check base overflow */
-            hashTable[ZSTD_hashPtr(base+current+2, hlog, mls)] = current+2;  /* here because current+2 could be > iend-8 */
-            hashTable[ZSTD_hashPtr(ip-2, hlog, mls)] = (U32)(ip-2-base);
-
-            /* check immediate repcode */
-            while (ip <= ilimit) {
-                U32 const current2 = (U32)(ip-base);
-                U32 const repIndex2 = current2 - offset_2;
-                const BYTE* repMatch2 = repIndex2 < prefixStartIndex ?
-                        dictBase - dictIndexDelta + repIndex2 :
-                        base + repIndex2;
-                if ( ((U32)((prefixStartIndex-1) - (U32)repIndex2) >= 3 /* intentional overflow */)
-                   && (MEM_read32(repMatch2) == MEM_read32(ip)) ) {
-                    const BYTE* const repEnd2 = repIndex2 < prefixStartIndex ? dictEnd : iend;
-                    size_t const repLength2 = ZSTD_count_2segments(ip+4, repMatch2+4, iend, repEnd2, prefixStart) + 4;
-                    U32 tmpOffset = offset_2; offset_2 = offset_1; offset_1 = tmpOffset;   /* swap offset_2 <=> offset_1 */
-                    ZSTD_storeSeq(seqStore, 0, anchor, 0, repLength2-MINMATCH);
-                    hashTable[ZSTD_hashPtr(ip, hlog, mls)] = current2;
-                    ip += repLength2;
-                    anchor = ip;
-                    continue;
-                }
-                break;
-            }
-        }
-    }
-
-    /* save reps for next block */
-    rep[0] = offset_1 ? offset_1 : offsetSaved;
-    rep[1] = offset_2 ? offset_2 : offsetSaved;
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-size_t ZSTD_compressBlock_fast_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    ZSTD_compressionParameters const* cParams = &ms->cParams;
-    U32 const mls = cParams->minMatch;
-    assert(ms->dictMatchState != NULL);
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_fast_dictMatchState_generic(ms, seqStore, rep, src, srcSize, 4);
-    case 5 :
-        return ZSTD_compressBlock_fast_dictMatchState_generic(ms, seqStore, rep, src, srcSize, 5);
-    case 6 :
-        return ZSTD_compressBlock_fast_dictMatchState_generic(ms, seqStore, rep, src, srcSize, 6);
-    case 7 :
-        return ZSTD_compressBlock_fast_dictMatchState_generic(ms, seqStore, rep, src, srcSize, 7);
-    }
-}
-
-
-static size_t ZSTD_compressBlock_fast_extDict_generic(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize, U32 const mls)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashTable = ms->hashTable;
-    U32 const hlog = cParams->hashLog;
-    /* support stepSize of 0 */
-    U32 const stepSize = cParams->targetLength + !(cParams->targetLength);
-    const BYTE* const base = ms->window.base;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const U32   endIndex = (U32)((size_t)(istart - base) + srcSize);
-    const U32   maxDistance = 1U << cParams->windowLog;
-    const U32   validLow = ms->window.lowLimit;
-    const U32   lowLimit = (endIndex - validLow > maxDistance) ? endIndex - maxDistance : validLow;
-    const U32   dictStartIndex = lowLimit;
-    const BYTE* const dictStart = dictBase + dictStartIndex;
-    const U32   dictLimit = ms->window.dictLimit;
-    const U32   prefixStartIndex = dictLimit < lowLimit ? lowLimit : dictLimit;
-    const BYTE* const prefixStart = base + prefixStartIndex;
-    const BYTE* const dictEnd = dictBase + prefixStartIndex;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - 8;
-    U32 offset_1=rep[0], offset_2=rep[1];
-
-    /* switch to "regular" variant if extDict is invalidated due to maxDistance */
-    if (prefixStartIndex == dictStartIndex)
-        return ZSTD_compressBlock_fast_generic(ms, seqStore, rep, src, srcSize, mls);
-
-    /* Search Loop */
-    while (ip < ilimit) {  /* < instead of <=, because (ip+1) */
-        const size_t h = ZSTD_hashPtr(ip, hlog, mls);
-        const U32    matchIndex = hashTable[h];
-        const BYTE* const matchBase = matchIndex < prefixStartIndex ? dictBase : base;
-        const BYTE*  match = matchBase + matchIndex;
-        const U32    current = (U32)(ip-base);
-        const U32    repIndex = current + 1 - offset_1;
-        const BYTE* const repBase = repIndex < prefixStartIndex ? dictBase : base;
-        const BYTE* const repMatch = repBase + repIndex;
-        size_t mLength;
-        hashTable[h] = current;   /* update hash table */
-        assert(offset_1 <= current +1);   /* check repIndex */
-
-        if ( (((U32)((prefixStartIndex-1) - repIndex) >= 3) /* intentional underflow */ & (repIndex > dictStartIndex))
-           && (MEM_read32(repMatch) == MEM_read32(ip+1)) ) {
-            const BYTE* repMatchEnd = repIndex < prefixStartIndex ? dictEnd : iend;
-            mLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repMatchEnd, prefixStart) + 4;
-            ip++;
-            ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, 0, mLength-MINMATCH);
-        } else {
-            if ( (matchIndex < dictStartIndex) ||
-                 (MEM_read32(match) != MEM_read32(ip)) ) {
-                assert(stepSize >= 1);
-                ip += ((ip-anchor) >> kSearchStrength) + stepSize;
-                continue;
-            }
-            {   const BYTE* matchEnd = matchIndex < prefixStartIndex ? dictEnd : iend;
-                const BYTE* lowMatchPtr = matchIndex < prefixStartIndex ? dictStart : prefixStart;
-                U32 offset;
-                mLength = ZSTD_count_2segments(ip+4, match+4, iend, matchEnd, prefixStart) + 4;
-                while (((ip>anchor) & (match>lowMatchPtr)) && (ip[-1] == match[-1])) { ip--; match--; mLength++; }   /* catch up */
-                offset = current - matchIndex;
-                offset_2 = offset_1;
-                offset_1 = offset;
-                ZSTD_storeSeq(seqStore, (size_t)(ip-anchor), anchor, offset + ZSTD_REP_MOVE, mLength-MINMATCH);
-        }   }
-
-        /* found a match : store it */
-        ip += mLength;
-        anchor = ip;
-
-        if (ip <= ilimit) {
-            /* Fill Table */
-            hashTable[ZSTD_hashPtr(base+current+2, hlog, mls)] = current+2;
-            hashTable[ZSTD_hashPtr(ip-2, hlog, mls)] = (U32)(ip-2-base);
-            /* check immediate repcode */
-            while (ip <= ilimit) {
-                U32 const current2 = (U32)(ip-base);
-                U32 const repIndex2 = current2 - offset_2;
-                const BYTE* repMatch2 = repIndex2 < prefixStartIndex ? dictBase + repIndex2 : base + repIndex2;
-                if ( (((U32)((prefixStartIndex-1) - repIndex2) >= 3) & (repIndex2 > dictStartIndex))  /* intentional overflow */
-                   && (MEM_read32(repMatch2) == MEM_read32(ip)) ) {
-                    const BYTE* const repEnd2 = repIndex2 < prefixStartIndex ? dictEnd : iend;
-                    size_t const repLength2 = ZSTD_count_2segments(ip+4, repMatch2+4, iend, repEnd2, prefixStart) + 4;
-                    U32 tmpOffset = offset_2; offset_2 = offset_1; offset_1 = tmpOffset;   /* swap offset_2 <=> offset_1 */
-                    ZSTD_storeSeq(seqStore, 0, anchor, 0, repLength2-MINMATCH);
-                    hashTable[ZSTD_hashPtr(ip, hlog, mls)] = current2;
-                    ip += repLength2;
-                    anchor = ip;
-                    continue;
-                }
-                break;
-    }   }   }
-
-    /* save reps for next block */
-    rep[0] = offset_1;
-    rep[1] = offset_2;
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-
-size_t ZSTD_compressBlock_fast_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    ZSTD_compressionParameters const* cParams = &ms->cParams;
-    U32 const mls = cParams->minMatch;
-    switch(mls)
-    {
-    default: /* includes case 3 */
-    case 4 :
-        return ZSTD_compressBlock_fast_extDict_generic(ms, seqStore, rep, src, srcSize, 4);
-    case 5 :
-        return ZSTD_compressBlock_fast_extDict_generic(ms, seqStore, rep, src, srcSize, 5);
-    case 6 :
-        return ZSTD_compressBlock_fast_extDict_generic(ms, seqStore, rep, src, srcSize, 6);
-    case 7 :
-        return ZSTD_compressBlock_fast_extDict_generic(ms, seqStore, rep, src, srcSize, 7);
-    }
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_fast.h b/vendor/github.com/DataDog/zstd/zstd_fast.h
deleted file mode 100644
index b74a88c..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_fast.h
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_FAST_H
-#define ZSTD_FAST_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#include "mem.h"      /* U32 */
-#include "zstd_compress_internal.h"
-
-void ZSTD_fillHashTable(ZSTD_matchState_t* ms,
-                        void const* end, ZSTD_dictTableLoadMethod_e dtlm);
-size_t ZSTD_compressBlock_fast(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_fast_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_fast_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_FAST_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_internal.h b/vendor/github.com/DataDog/zstd/zstd_internal.h
deleted file mode 100644
index 81b16ea..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_internal.h
+++ /dev/null
@@ -1,371 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_CCOMMON_H_MODULE
-#define ZSTD_CCOMMON_H_MODULE
-
-/* this module contains definitions which must be identical
- * across compression, decompression and dictBuilder.
- * It also contains a few functions useful to at least 2 of them
- * and which benefit from being inlined */
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include "compiler.h"
-#include "mem.h"
-#include "debug.h"                 /* assert, DEBUGLOG, RAWLOG, g_debuglevel */
-#include "error_private.h"
-#define ZSTD_STATIC_LINKING_ONLY
-#include "zstd.h"
-#define FSE_STATIC_LINKING_ONLY
-#include "fse.h"
-#define HUF_STATIC_LINKING_ONLY
-#include "huf.h"
-#ifndef XXH_STATIC_LINKING_ONLY
-#  define XXH_STATIC_LINKING_ONLY  /* XXH64_state_t */
-#endif
-#include "xxhash.h"                /* XXH_reset, update, digest */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* ---- static assert (debug) --- */
-#define ZSTD_STATIC_ASSERT(c) DEBUG_STATIC_ASSERT(c)
-#define ZSTD_isError ERR_isError   /* for inlining */
-#define FSE_isError  ERR_isError
-#define HUF_isError  ERR_isError
-
-
-/*-*************************************
-*  shared macros
-***************************************/
-#undef MIN
-#undef MAX
-#define MIN(a,b) ((a)<(b) ? (a) : (b))
-#define MAX(a,b) ((a)>(b) ? (a) : (b))
-
-/**
- * Return the specified error if the condition evaluates to true.
- *
- * In debug modes, prints additional information. In order to do that
- * (particularly, printing the conditional that failed), this can't just wrap
- * RETURN_ERROR().
- */
-#define RETURN_ERROR_IF(cond, err, ...) \
-  if (cond) { \
-    RAWLOG(3, "%s:%d: ERROR!: check %s failed, returning %s", __FILE__, __LINE__, ZSTD_QUOTE(cond), ZSTD_QUOTE(ERROR(err))); \
-    RAWLOG(3, ": " __VA_ARGS__); \
-    RAWLOG(3, "\n"); \
-    return ERROR(err); \
-  }
-
-/**
- * Unconditionally return the specified error.
- *
- * In debug modes, prints additional information.
- */
-#define RETURN_ERROR(err, ...) \
-  do { \
-    RAWLOG(3, "%s:%d: ERROR!: unconditional check failed, returning %s", __FILE__, __LINE__, ZSTD_QUOTE(ERROR(err))); \
-    RAWLOG(3, ": " __VA_ARGS__); \
-    RAWLOG(3, "\n"); \
-    return ERROR(err); \
-  } while(0);
-
-/**
- * If the provided expression evaluates to an error code, returns that error code.
- *
- * In debug modes, prints additional information.
- */
-#define FORWARD_IF_ERROR(err, ...) \
-  do { \
-    size_t const err_code = (err); \
-    if (ERR_isError(err_code)) { \
-      RAWLOG(3, "%s:%d: ERROR!: forwarding error in %s: %s", __FILE__, __LINE__, ZSTD_QUOTE(err), ERR_getErrorName(err_code)); \
-      RAWLOG(3, ": " __VA_ARGS__); \
-      RAWLOG(3, "\n"); \
-      return err_code; \
-    } \
-  } while(0);
-
-
-/*-*************************************
-*  Common constants
-***************************************/
-#define ZSTD_OPT_NUM    (1<<12)
-
-#define ZSTD_REP_NUM      3                 /* number of repcodes */
-#define ZSTD_REP_MOVE     (ZSTD_REP_NUM-1)
-static const U32 repStartValue[ZSTD_REP_NUM] = { 1, 4, 8 };
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define ZSTD_WINDOWLOG_ABSOLUTEMIN 10
-static const size_t ZSTD_fcs_fieldSize[4] = { 0, 2, 4, 8 };
-static const size_t ZSTD_did_fieldSize[4] = { 0, 1, 2, 4 };
-
-#define ZSTD_FRAMEIDSIZE 4   /* magic number size */
-
-#define ZSTD_BLOCKHEADERSIZE 3   /* C standard doesn't allow `static const` variable to be init using another `static const` variable */
-static const size_t ZSTD_blockHeaderSize = ZSTD_BLOCKHEADERSIZE;
-typedef enum { bt_raw, bt_rle, bt_compressed, bt_reserved } blockType_e;
-
-#define MIN_SEQUENCES_SIZE 1 /* nbSeq==0 */
-#define MIN_CBLOCK_SIZE (1 /*litCSize*/ + 1 /* RLE or RAW */ + MIN_SEQUENCES_SIZE /* nbSeq==0 */)   /* for a non-null block */
-
-#define HufLog 12
-typedef enum { set_basic, set_rle, set_compressed, set_repeat } symbolEncodingType_e;
-
-#define LONGNBSEQ 0x7F00
-
-#define MINMATCH 3
-
-#define Litbits  8
-#define MaxLit ((1<<Litbits) - 1)
-#define MaxML   52
-#define MaxLL   35
-#define DefaultMaxOff 28
-#define MaxOff  31
-#define MaxSeq MAX(MaxLL, MaxML)   /* Assumption : MaxOff < MaxLL,MaxML */
-#define MLFSELog    9
-#define LLFSELog    9
-#define OffFSELog   8
-#define MaxFSELog  MAX(MAX(MLFSELog, LLFSELog), OffFSELog)
-
-static const U32 LL_bits[MaxLL+1] = { 0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3,
-                                      4, 6, 7, 8, 9,10,11,12,
-                                     13,14,15,16 };
-static const S16 LL_defaultNorm[MaxLL+1] = { 4, 3, 2, 2, 2, 2, 2, 2,
-                                             2, 2, 2, 2, 2, 1, 1, 1,
-                                             2, 2, 2, 2, 2, 2, 2, 2,
-                                             2, 3, 2, 1, 1, 1, 1, 1,
-                                            -1,-1,-1,-1 };
-#define LL_DEFAULTNORMLOG 6  /* for static allocation */
-static const U32 LL_defaultNormLog = LL_DEFAULTNORMLOG;
-
-static const U32 ML_bits[MaxML+1] = { 0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3,
-                                      4, 4, 5, 7, 8, 9,10,11,
-                                     12,13,14,15,16 };
-static const S16 ML_defaultNorm[MaxML+1] = { 1, 4, 3, 2, 2, 2, 2, 2,
-                                             2, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1,-1,-1,
-                                            -1,-1,-1,-1,-1 };
-#define ML_DEFAULTNORMLOG 6  /* for static allocation */
-static const U32 ML_defaultNormLog = ML_DEFAULTNORMLOG;
-
-static const S16 OF_defaultNorm[DefaultMaxOff+1] = { 1, 1, 1, 1, 1, 1, 2, 2,
-                                                     2, 1, 1, 1, 1, 1, 1, 1,
-                                                     1, 1, 1, 1, 1, 1, 1, 1,
-                                                    -1,-1,-1,-1,-1 };
-#define OF_DEFAULTNORMLOG 5  /* for static allocation */
-static const U32 OF_defaultNormLog = OF_DEFAULTNORMLOG;
-
-
-/*-*******************************************
-*  Shared functions to include for inlining
-*********************************************/
-static void ZSTD_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s) { ZSTD_copy8(d,s); d+=8; s+=8; }
-static void ZSTD_copy16(void* dst, const void* src) { memcpy(dst, src, 16); }
-#define COPY16(d,s) { ZSTD_copy16(d,s); d+=16; s+=16; }
-
-#define WILDCOPY_OVERLENGTH 8
-#define VECLEN 16
-
-typedef enum {
-    ZSTD_no_overlap,
-    ZSTD_overlap_src_before_dst,
-    /*  ZSTD_overlap_dst_before_src, */
-} ZSTD_overlap_e;
-
-/*! ZSTD_wildcopy() :
- *  custom version of memcpy(), can overwrite up to WILDCOPY_OVERLENGTH bytes (if length==0) */
-MEM_STATIC FORCE_INLINE_ATTR DONT_VECTORIZE
-void ZSTD_wildcopy(void* dst, const void* src, ptrdiff_t length, ZSTD_overlap_e ovtype)
-{
-    ptrdiff_t diff = (BYTE*)dst - (const BYTE*)src;
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-
-    assert(diff >= 8 || (ovtype == ZSTD_no_overlap && diff < -8));
-    if (length < VECLEN || (ovtype == ZSTD_overlap_src_before_dst && diff < VECLEN)) {
-      do
-          COPY8(op, ip)
-      while (op < oend);
-    }
-    else {
-      if ((length & 8) == 0)
-        COPY8(op, ip);
-      do {
-        COPY16(op, ip);
-      }
-      while (op < oend);
-    }
-}
-
-/*! ZSTD_wildcopy_16min() :
- *  same semantics as ZSTD_wilcopy() except guaranteed to be able to copy 16 bytes at the start */
-MEM_STATIC FORCE_INLINE_ATTR DONT_VECTORIZE
-void ZSTD_wildcopy_16min(void* dst, const void* src, ptrdiff_t length, ZSTD_overlap_e ovtype)
-{
-    ptrdiff_t diff = (BYTE*)dst - (const BYTE*)src;
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-
-    assert(length >= 8);
-    assert(diff >= 8 || (ovtype == ZSTD_no_overlap && diff < -8));
-
-    if (ovtype == ZSTD_overlap_src_before_dst && diff < VECLEN) {
-      do
-          COPY8(op, ip)
-      while (op < oend);
-    }
-    else {
-      if ((length & 8) == 0)
-        COPY8(op, ip);
-      do {
-        COPY16(op, ip);
-      }
-      while (op < oend);
-    }
-}
-
-MEM_STATIC void ZSTD_wildcopy_e(void* dst, const void* src, void* dstEnd)   /* should be faster for decoding, but strangely, not verified on all platform */
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = (BYTE*)dstEnd;
-    do
-        COPY8(op, ip)
-    while (op < oend);
-}
-
-
-/*-*******************************************
-*  Private declarations
-*********************************************/
-typedef struct seqDef_s {
-    U32 offset;
-    U16 litLength;
-    U16 matchLength;
-} seqDef;
-
-typedef struct {
-    seqDef* sequencesStart;
-    seqDef* sequences;
-    BYTE* litStart;
-    BYTE* lit;
-    BYTE* llCode;
-    BYTE* mlCode;
-    BYTE* ofCode;
-    size_t maxNbSeq;
-    size_t maxNbLit;
-    U32   longLengthID;   /* 0 == no longLength; 1 == Lit.longLength; 2 == Match.longLength; */
-    U32   longLengthPos;
-} seqStore_t;
-
-/**
- * Contains the compressed frame size and an upper-bound for the decompressed frame size.
- * Note: before using `compressedSize`, check for errors using ZSTD_isError().
- *       similarly, before using `decompressedBound`, check for errors using:
- *          `decompressedBound != ZSTD_CONTENTSIZE_ERROR`
- */
-typedef struct {
-    size_t compressedSize;
-    unsigned long long decompressedBound;
-} ZSTD_frameSizeInfo;   /* decompress & legacy */
-
-const seqStore_t* ZSTD_getSeqStore(const ZSTD_CCtx* ctx);   /* compress & dictBuilder */
-void ZSTD_seqToCodes(const seqStore_t* seqStorePtr);   /* compress, dictBuilder, decodeCorpus (shouldn't get its definition from here) */
-
-/* custom memory allocation functions */
-void* ZSTD_malloc(size_t size, ZSTD_customMem customMem);
-void* ZSTD_calloc(size_t size, ZSTD_customMem customMem);
-void ZSTD_free(void* ptr, ZSTD_customMem customMem);
-
-
-MEM_STATIC U32 ZSTD_highbit32(U32 val)   /* compress, dictBuilder, decodeCorpus */
-{
-    assert(val != 0);
-    {
-#   if defined(_MSC_VER)   /* Visual */
-        unsigned long r=0;
-        _BitScanReverse(&r, val);
-        return (unsigned)r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* GCC Intrinsic */
-        return 31 - __builtin_clz(val);
-#   else   /* Software version */
-        static const U32 DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-        U32 v = val;
-        v |= v >> 1;
-        v |= v >> 2;
-        v |= v >> 4;
-        v |= v >> 8;
-        v |= v >> 16;
-        return DeBruijnClz[(v * 0x07C4ACDDU) >> 27];
-#   endif
-    }
-}
-
-
-/* ZSTD_invalidateRepCodes() :
- * ensures next compression will not use repcodes from previous block.
- * Note : only works with regular variant;
- *        do not use with extDict variant ! */
-void ZSTD_invalidateRepCodes(ZSTD_CCtx* cctx);   /* zstdmt, adaptive_compression (shouldn't get this definition from here) */
-
-
-typedef struct {
-    blockType_e blockType;
-    U32 lastBlock;
-    U32 origSize;
-} blockProperties_t;   /* declared here for decompress and fullbench */
-
-/*! ZSTD_getcBlockSize() :
- *  Provides the size of compressed block from block header `src` */
-/* Used by: decompress, fullbench (does not get its definition from here) */
-size_t ZSTD_getcBlockSize(const void* src, size_t srcSize,
-                          blockProperties_t* bpPtr);
-
-/*! ZSTD_decodeSeqHeaders() :
- *  decode sequence header from src */
-/* Used by: decompress, fullbench (does not get its definition from here) */
-size_t ZSTD_decodeSeqHeaders(ZSTD_DCtx* dctx, int* nbSeqPtr,
-                       const void* src, size_t srcSize);
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* ZSTD_CCOMMON_H_MODULE */
diff --git a/vendor/github.com/DataDog/zstd/zstd_lazy.c b/vendor/github.com/DataDog/zstd/zstd_lazy.c
deleted file mode 100644
index 94d906c..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_lazy.c
+++ /dev/null
@@ -1,1111 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#include "zstd_compress_internal.h"
-#include "zstd_lazy.h"
-
-
-/*-*************************************
-*  Binary Tree search
-***************************************/
-
-static void
-ZSTD_updateDUBT(ZSTD_matchState_t* ms,
-                const BYTE* ip, const BYTE* iend,
-                U32 mls)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const hashTable = ms->hashTable;
-    U32  const hashLog = cParams->hashLog;
-
-    U32* const bt = ms->chainTable;
-    U32  const btLog  = cParams->chainLog - 1;
-    U32  const btMask = (1 << btLog) - 1;
-
-    const BYTE* const base = ms->window.base;
-    U32 const target = (U32)(ip - base);
-    U32 idx = ms->nextToUpdate;
-
-    if (idx != target)
-        DEBUGLOG(7, "ZSTD_updateDUBT, from %u to %u (dictLimit:%u)",
-                    idx, target, ms->window.dictLimit);
-    assert(ip + 8 <= iend);   /* condition for ZSTD_hashPtr */
-    (void)iend;
-
-    assert(idx >= ms->window.dictLimit);   /* condition for valid base+idx */
-    for ( ; idx < target ; idx++) {
-        size_t const h  = ZSTD_hashPtr(base + idx, hashLog, mls);   /* assumption : ip + 8 <= iend */
-        U32    const matchIndex = hashTable[h];
-
-        U32*   const nextCandidatePtr = bt + 2*(idx&btMask);
-        U32*   const sortMarkPtr  = nextCandidatePtr + 1;
-
-        DEBUGLOG(8, "ZSTD_updateDUBT: insert %u", idx);
-        hashTable[h] = idx;   /* Update Hash Table */
-        *nextCandidatePtr = matchIndex;   /* update BT like a chain */
-        *sortMarkPtr = ZSTD_DUBT_UNSORTED_MARK;
-    }
-    ms->nextToUpdate = target;
-}
-
-
-/** ZSTD_insertDUBT1() :
- *  sort one already inserted but unsorted position
- *  assumption : current >= btlow == (current - btmask)
- *  doesn't fail */
-static void
-ZSTD_insertDUBT1(ZSTD_matchState_t* ms,
-                 U32 current, const BYTE* inputEnd,
-                 U32 nbCompares, U32 btLow,
-                 const ZSTD_dictMode_e dictMode)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const bt = ms->chainTable;
-    U32  const btLog  = cParams->chainLog - 1;
-    U32  const btMask = (1 << btLog) - 1;
-    size_t commonLengthSmaller=0, commonLengthLarger=0;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const U32 dictLimit = ms->window.dictLimit;
-    const BYTE* const ip = (current>=dictLimit) ? base + current : dictBase + current;
-    const BYTE* const iend = (current>=dictLimit) ? inputEnd : dictBase + dictLimit;
-    const BYTE* const dictEnd = dictBase + dictLimit;
-    const BYTE* const prefixStart = base + dictLimit;
-    const BYTE* match;
-    U32* smallerPtr = bt + 2*(current&btMask);
-    U32* largerPtr  = smallerPtr + 1;
-    U32 matchIndex = *smallerPtr;   /* this candidate is unsorted : next sorted candidate is reached through *smallerPtr, while *largerPtr contains previous unsorted candidate (which is already saved and can be overwritten) */
-    U32 dummy32;   /* to be nullified at the end */
-    U32 const windowValid = ms->window.lowLimit;
-    U32 const maxDistance = 1U << cParams->windowLog;
-    U32 const windowLow = (current - windowValid > maxDistance) ? current - maxDistance : windowValid;
-
-
-    DEBUGLOG(8, "ZSTD_insertDUBT1(%u) (dictLimit=%u, lowLimit=%u)",
-                current, dictLimit, windowLow);
-    assert(current >= btLow);
-    assert(ip < iend);   /* condition for ZSTD_count */
-
-    while (nbCompares-- && (matchIndex > windowLow)) {
-        U32* const nextPtr = bt + 2*(matchIndex & btMask);
-        size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-        assert(matchIndex < current);
-        /* note : all candidates are now supposed sorted,
-         * but it's still possible to have nextPtr[1] == ZSTD_DUBT_UNSORTED_MARK
-         * when a real index has the same value as ZSTD_DUBT_UNSORTED_MARK */
-
-        if ( (dictMode != ZSTD_extDict)
-          || (matchIndex+matchLength >= dictLimit)  /* both in current segment*/
-          || (current < dictLimit) /* both in extDict */) {
-            const BYTE* const mBase = ( (dictMode != ZSTD_extDict)
-                                     || (matchIndex+matchLength >= dictLimit)) ?
-                                        base : dictBase;
-            assert( (matchIndex+matchLength >= dictLimit)   /* might be wrong if extDict is incorrectly set to 0 */
-                 || (current < dictLimit) );
-            match = mBase + matchIndex;
-            matchLength += ZSTD_count(ip+matchLength, match+matchLength, iend);
-        } else {
-            match = dictBase + matchIndex;
-            matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iend, dictEnd, prefixStart);
-            if (matchIndex+matchLength >= dictLimit)
-                match = base + matchIndex;   /* preparation for next read of match[matchLength] */
-        }
-
-        DEBUGLOG(8, "ZSTD_insertDUBT1: comparing %u with %u : found %u common bytes ",
-                    current, matchIndex, (U32)matchLength);
-
-        if (ip+matchLength == iend) {   /* equal : no way to know if inf or sup */
-            break;   /* drop , to guarantee consistency ; miss a bit of compression, but other solutions can corrupt tree */
-        }
-
-        if (match[matchLength] < ip[matchLength]) {  /* necessarily within buffer */
-            /* match is smaller than current */
-            *smallerPtr = matchIndex;             /* update smaller idx */
-            commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-            if (matchIndex <= btLow) { smallerPtr=&dummy32; break; }   /* beyond tree size, stop searching */
-            DEBUGLOG(8, "ZSTD_insertDUBT1: %u (>btLow=%u) is smaller : next => %u",
-                        matchIndex, btLow, nextPtr[1]);
-            smallerPtr = nextPtr+1;               /* new "candidate" => larger than match, which was smaller than target */
-            matchIndex = nextPtr[1];              /* new matchIndex, larger than previous and closer to current */
-        } else {
-            /* match is larger than current */
-            *largerPtr = matchIndex;
-            commonLengthLarger = matchLength;
-            if (matchIndex <= btLow) { largerPtr=&dummy32; break; }   /* beyond tree size, stop searching */
-            DEBUGLOG(8, "ZSTD_insertDUBT1: %u (>btLow=%u) is larger => %u",
-                        matchIndex, btLow, nextPtr[0]);
-            largerPtr = nextPtr;
-            matchIndex = nextPtr[0];
-    }   }
-
-    *smallerPtr = *largerPtr = 0;
-}
-
-
-static size_t
-ZSTD_DUBT_findBetterDictMatch (
-        ZSTD_matchState_t* ms,
-        const BYTE* const ip, const BYTE* const iend,
-        size_t* offsetPtr,
-        size_t bestLength,
-        U32 nbCompares,
-        U32 const mls,
-        const ZSTD_dictMode_e dictMode)
-{
-    const ZSTD_matchState_t * const dms = ms->dictMatchState;
-    const ZSTD_compressionParameters* const dmsCParams = &dms->cParams;
-    const U32 * const dictHashTable = dms->hashTable;
-    U32         const hashLog = dmsCParams->hashLog;
-    size_t      const h  = ZSTD_hashPtr(ip, hashLog, mls);
-    U32               dictMatchIndex = dictHashTable[h];
-
-    const BYTE* const base = ms->window.base;
-    const BYTE* const prefixStart = base + ms->window.dictLimit;
-    U32         const current = (U32)(ip-base);
-    const BYTE* const dictBase = dms->window.base;
-    const BYTE* const dictEnd = dms->window.nextSrc;
-    U32         const dictHighLimit = (U32)(dms->window.nextSrc - dms->window.base);
-    U32         const dictLowLimit = dms->window.lowLimit;
-    U32         const dictIndexDelta = ms->window.lowLimit - dictHighLimit;
-
-    U32*        const dictBt = dms->chainTable;
-    U32         const btLog  = dmsCParams->chainLog - 1;
-    U32         const btMask = (1 << btLog) - 1;
-    U32         const btLow = (btMask >= dictHighLimit - dictLowLimit) ? dictLowLimit : dictHighLimit - btMask;
-
-    size_t commonLengthSmaller=0, commonLengthLarger=0;
-
-    (void)dictMode;
-    assert(dictMode == ZSTD_dictMatchState);
-
-    while (nbCompares-- && (dictMatchIndex > dictLowLimit)) {
-        U32* const nextPtr = dictBt + 2*(dictMatchIndex & btMask);
-        size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-        const BYTE* match = dictBase + dictMatchIndex;
-        matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iend, dictEnd, prefixStart);
-        if (dictMatchIndex+matchLength >= dictHighLimit)
-            match = base + dictMatchIndex + dictIndexDelta;   /* to prepare for next usage of match[matchLength] */
-
-        if (matchLength > bestLength) {
-            U32 matchIndex = dictMatchIndex + dictIndexDelta;
-            if ( (4*(int)(matchLength-bestLength)) > (int)(ZSTD_highbit32(current-matchIndex+1) - ZSTD_highbit32((U32)offsetPtr[0]+1)) ) {
-                DEBUGLOG(9, "ZSTD_DUBT_findBetterDictMatch(%u) : found better match length %u -> %u and offsetCode %u -> %u (dictMatchIndex %u, matchIndex %u)",
-                    current, (U32)bestLength, (U32)matchLength, (U32)*offsetPtr, ZSTD_REP_MOVE + current - matchIndex, dictMatchIndex, matchIndex);
-                bestLength = matchLength, *offsetPtr = ZSTD_REP_MOVE + current - matchIndex;
-            }
-            if (ip+matchLength == iend) {   /* reached end of input : ip[matchLength] is not valid, no way to know if it's larger or smaller than match */
-                break;   /* drop, to guarantee consistency (miss a little bit of compression) */
-            }
-        }
-
-        if (match[matchLength] < ip[matchLength]) {
-            if (dictMatchIndex <= btLow) { break; }   /* beyond tree size, stop the search */
-            commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-            dictMatchIndex = nextPtr[1];              /* new matchIndex larger than previous (closer to current) */
-        } else {
-            /* match is larger than current */
-            if (dictMatchIndex <= btLow) { break; }   /* beyond tree size, stop the search */
-            commonLengthLarger = matchLength;
-            dictMatchIndex = nextPtr[0];
-        }
-    }
-
-    if (bestLength >= MINMATCH) {
-        U32 const mIndex = current - ((U32)*offsetPtr - ZSTD_REP_MOVE); (void)mIndex;
-        DEBUGLOG(8, "ZSTD_DUBT_findBetterDictMatch(%u) : found match of length %u and offsetCode %u (pos %u)",
-                    current, (U32)bestLength, (U32)*offsetPtr, mIndex);
-    }
-    return bestLength;
-
-}
-
-
-static size_t
-ZSTD_DUBT_findBestMatch(ZSTD_matchState_t* ms,
-                        const BYTE* const ip, const BYTE* const iend,
-                        size_t* offsetPtr,
-                        U32 const mls,
-                        const ZSTD_dictMode_e dictMode)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32*   const hashTable = ms->hashTable;
-    U32    const hashLog = cParams->hashLog;
-    size_t const h  = ZSTD_hashPtr(ip, hashLog, mls);
-    U32          matchIndex  = hashTable[h];
-
-    const BYTE* const base = ms->window.base;
-    U32    const current = (U32)(ip-base);
-    U32    const maxDistance = 1U << cParams->windowLog;
-    U32    const windowValid = ms->window.lowLimit;
-    U32    const windowLow = (current - windowValid > maxDistance) ? current - maxDistance : windowValid;
-
-    U32*   const bt = ms->chainTable;
-    U32    const btLog  = cParams->chainLog - 1;
-    U32    const btMask = (1 << btLog) - 1;
-    U32    const btLow = (btMask >= current) ? 0 : current - btMask;
-    U32    const unsortLimit = MAX(btLow, windowLow);
-
-    U32*         nextCandidate = bt + 2*(matchIndex&btMask);
-    U32*         unsortedMark = bt + 2*(matchIndex&btMask) + 1;
-    U32          nbCompares = 1U << cParams->searchLog;
-    U32          nbCandidates = nbCompares;
-    U32          previousCandidate = 0;
-
-    DEBUGLOG(7, "ZSTD_DUBT_findBestMatch (%u) ", current);
-    assert(ip <= iend-8);   /* required for h calculation */
-
-    /* reach end of unsorted candidates list */
-    while ( (matchIndex > unsortLimit)
-         && (*unsortedMark == ZSTD_DUBT_UNSORTED_MARK)
-         && (nbCandidates > 1) ) {
-        DEBUGLOG(8, "ZSTD_DUBT_findBestMatch: candidate %u is unsorted",
-                    matchIndex);
-        *unsortedMark = previousCandidate;  /* the unsortedMark becomes a reversed chain, to move up back to original position */
-        previousCandidate = matchIndex;
-        matchIndex = *nextCandidate;
-        nextCandidate = bt + 2*(matchIndex&btMask);
-        unsortedMark = bt + 2*(matchIndex&btMask) + 1;
-        nbCandidates --;
-    }
-
-    /* nullify last candidate if it's still unsorted
-     * simplification, detrimental to compression ratio, beneficial for speed */
-    if ( (matchIndex > unsortLimit)
-      && (*unsortedMark==ZSTD_DUBT_UNSORTED_MARK) ) {
-        DEBUGLOG(7, "ZSTD_DUBT_findBestMatch: nullify last unsorted candidate %u",
-                    matchIndex);
-        *nextCandidate = *unsortedMark = 0;
-    }
-
-    /* batch sort stacked candidates */
-    matchIndex = previousCandidate;
-    while (matchIndex) {  /* will end on matchIndex == 0 */
-        U32* const nextCandidateIdxPtr = bt + 2*(matchIndex&btMask) + 1;
-        U32 const nextCandidateIdx = *nextCandidateIdxPtr;
-        ZSTD_insertDUBT1(ms, matchIndex, iend,
-                         nbCandidates, unsortLimit, dictMode);
-        matchIndex = nextCandidateIdx;
-        nbCandidates++;
-    }
-
-    /* find longest match */
-    {   size_t commonLengthSmaller = 0, commonLengthLarger = 0;
-        const BYTE* const dictBase = ms->window.dictBase;
-        const U32 dictLimit = ms->window.dictLimit;
-        const BYTE* const dictEnd = dictBase + dictLimit;
-        const BYTE* const prefixStart = base + dictLimit;
-        U32* smallerPtr = bt + 2*(current&btMask);
-        U32* largerPtr  = bt + 2*(current&btMask) + 1;
-        U32 matchEndIdx = current + 8 + 1;
-        U32 dummy32;   /* to be nullified at the end */
-        size_t bestLength = 0;
-
-        matchIndex  = hashTable[h];
-        hashTable[h] = current;   /* Update Hash Table */
-
-        while (nbCompares-- && (matchIndex > windowLow)) {
-            U32* const nextPtr = bt + 2*(matchIndex & btMask);
-            size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-            const BYTE* match;
-
-            if ((dictMode != ZSTD_extDict) || (matchIndex+matchLength >= dictLimit)) {
-                match = base + matchIndex;
-                matchLength += ZSTD_count(ip+matchLength, match+matchLength, iend);
-            } else {
-                match = dictBase + matchIndex;
-                matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iend, dictEnd, prefixStart);
-                if (matchIndex+matchLength >= dictLimit)
-                    match = base + matchIndex;   /* to prepare for next usage of match[matchLength] */
-            }
-
-            if (matchLength > bestLength) {
-                if (matchLength > matchEndIdx - matchIndex)
-                    matchEndIdx = matchIndex + (U32)matchLength;
-                if ( (4*(int)(matchLength-bestLength)) > (int)(ZSTD_highbit32(current-matchIndex+1) - ZSTD_highbit32((U32)offsetPtr[0]+1)) )
-                    bestLength = matchLength, *offsetPtr = ZSTD_REP_MOVE + current - matchIndex;
-                if (ip+matchLength == iend) {   /* equal : no way to know if inf or sup */
-                    if (dictMode == ZSTD_dictMatchState) {
-                        nbCompares = 0; /* in addition to avoiding checking any
-                                         * further in this loop, make sure we
-                                         * skip checking in the dictionary. */
-                    }
-                    break;   /* drop, to guarantee consistency (miss a little bit of compression) */
-                }
-            }
-
-            if (match[matchLength] < ip[matchLength]) {
-                /* match is smaller than current */
-                *smallerPtr = matchIndex;             /* update smaller idx */
-                commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-                if (matchIndex <= btLow) { smallerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-                smallerPtr = nextPtr+1;               /* new "smaller" => larger of match */
-                matchIndex = nextPtr[1];              /* new matchIndex larger than previous (closer to current) */
-            } else {
-                /* match is larger than current */
-                *largerPtr = matchIndex;
-                commonLengthLarger = matchLength;
-                if (matchIndex <= btLow) { largerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-                largerPtr = nextPtr;
-                matchIndex = nextPtr[0];
-        }   }
-
-        *smallerPtr = *largerPtr = 0;
-
-        if (dictMode == ZSTD_dictMatchState && nbCompares) {
-            bestLength = ZSTD_DUBT_findBetterDictMatch(
-                    ms, ip, iend,
-                    offsetPtr, bestLength, nbCompares,
-                    mls, dictMode);
-        }
-
-        assert(matchEndIdx > current+8); /* ensure nextToUpdate is increased */
-        ms->nextToUpdate = matchEndIdx - 8;   /* skip repetitive patterns */
-        if (bestLength >= MINMATCH) {
-            U32 const mIndex = current - ((U32)*offsetPtr - ZSTD_REP_MOVE); (void)mIndex;
-            DEBUGLOG(8, "ZSTD_DUBT_findBestMatch(%u) : found match of length %u and offsetCode %u (pos %u)",
-                        current, (U32)bestLength, (U32)*offsetPtr, mIndex);
-        }
-        return bestLength;
-    }
-}
-
-
-/** ZSTD_BtFindBestMatch() : Tree updater, providing best match */
-FORCE_INLINE_TEMPLATE size_t
-ZSTD_BtFindBestMatch( ZSTD_matchState_t* ms,
-                const BYTE* const ip, const BYTE* const iLimit,
-                      size_t* offsetPtr,
-                const U32 mls /* template */,
-                const ZSTD_dictMode_e dictMode)
-{
-    DEBUGLOG(7, "ZSTD_BtFindBestMatch");
-    if (ip < ms->window.base + ms->nextToUpdate) return 0;   /* skipped area */
-    ZSTD_updateDUBT(ms, ip, iLimit, mls);
-    return ZSTD_DUBT_findBestMatch(ms, ip, iLimit, offsetPtr, mls, dictMode);
-}
-
-
-static size_t
-ZSTD_BtFindBestMatch_selectMLS (  ZSTD_matchState_t* ms,
-                            const BYTE* ip, const BYTE* const iLimit,
-                                  size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 4, ZSTD_noDict);
-    case 5 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 5, ZSTD_noDict);
-    case 7 :
-    case 6 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 6, ZSTD_noDict);
-    }
-}
-
-
-static size_t ZSTD_BtFindBestMatch_dictMatchState_selectMLS (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* const iLimit,
-                        size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 4, ZSTD_dictMatchState);
-    case 5 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 5, ZSTD_dictMatchState);
-    case 7 :
-    case 6 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 6, ZSTD_dictMatchState);
-    }
-}
-
-
-static size_t ZSTD_BtFindBestMatch_extDict_selectMLS (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* const iLimit,
-                        size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 4, ZSTD_extDict);
-    case 5 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 5, ZSTD_extDict);
-    case 7 :
-    case 6 : return ZSTD_BtFindBestMatch(ms, ip, iLimit, offsetPtr, 6, ZSTD_extDict);
-    }
-}
-
-
-
-/* *********************************
-*  Hash Chain
-***********************************/
-#define NEXT_IN_CHAIN(d, mask)   chainTable[(d) & (mask)]
-
-/* Update chains up to ip (excluded)
-   Assumption : always within prefix (i.e. not within extDict) */
-static U32 ZSTD_insertAndFindFirstIndex_internal(
-                        ZSTD_matchState_t* ms,
-                        const ZSTD_compressionParameters* const cParams,
-                        const BYTE* ip, U32 const mls)
-{
-    U32* const hashTable  = ms->hashTable;
-    const U32 hashLog = cParams->hashLog;
-    U32* const chainTable = ms->chainTable;
-    const U32 chainMask = (1 << cParams->chainLog) - 1;
-    const BYTE* const base = ms->window.base;
-    const U32 target = (U32)(ip - base);
-    U32 idx = ms->nextToUpdate;
-
-    while(idx < target) { /* catch up */
-        size_t const h = ZSTD_hashPtr(base+idx, hashLog, mls);
-        NEXT_IN_CHAIN(idx, chainMask) = hashTable[h];
-        hashTable[h] = idx;
-        idx++;
-    }
-
-    ms->nextToUpdate = target;
-    return hashTable[ZSTD_hashPtr(ip, hashLog, mls)];
-}
-
-U32 ZSTD_insertAndFindFirstIndex(ZSTD_matchState_t* ms, const BYTE* ip) {
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    return ZSTD_insertAndFindFirstIndex_internal(ms, cParams, ip, ms->cParams.minMatch);
-}
-
-
-/* inlining is important to hardwire a hot branch (template emulation) */
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_HcFindBestMatch_generic (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* const ip, const BYTE* const iLimit,
-                        size_t* offsetPtr,
-                        const U32 mls, const ZSTD_dictMode_e dictMode)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32* const chainTable = ms->chainTable;
-    const U32 chainSize = (1 << cParams->chainLog);
-    const U32 chainMask = chainSize-1;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const U32 dictLimit = ms->window.dictLimit;
-    const BYTE* const prefixStart = base + dictLimit;
-    const BYTE* const dictEnd = dictBase + dictLimit;
-    const U32 current = (U32)(ip-base);
-    const U32 maxDistance = 1U << cParams->windowLog;
-    const U32 lowValid = ms->window.lowLimit;
-    const U32 lowLimit = (current - lowValid > maxDistance) ? current - maxDistance : lowValid;
-    const U32 minChain = current > chainSize ? current - chainSize : 0;
-    U32 nbAttempts = 1U << cParams->searchLog;
-    size_t ml=4-1;
-
-    /* HC4 match finder */
-    U32 matchIndex = ZSTD_insertAndFindFirstIndex_internal(ms, cParams, ip, mls);
-
-    for ( ; (matchIndex>lowLimit) & (nbAttempts>0) ; nbAttempts--) {
-        size_t currentMl=0;
-        if ((dictMode != ZSTD_extDict) || matchIndex >= dictLimit) {
-            const BYTE* const match = base + matchIndex;
-            assert(matchIndex >= dictLimit);   /* ensures this is true if dictMode != ZSTD_extDict */
-            if (match[ml] == ip[ml])   /* potentially better */
-                currentMl = ZSTD_count(ip, match, iLimit);
-        } else {
-            const BYTE* const match = dictBase + matchIndex;
-            assert(match+4 <= dictEnd);
-            if (MEM_read32(match) == MEM_read32(ip))   /* assumption : matchIndex <= dictLimit-4 (by table construction) */
-                currentMl = ZSTD_count_2segments(ip+4, match+4, iLimit, dictEnd, prefixStart) + 4;
-        }
-
-        /* save best solution */
-        if (currentMl > ml) {
-            ml = currentMl;
-            *offsetPtr = current - matchIndex + ZSTD_REP_MOVE;
-            if (ip+currentMl == iLimit) break; /* best possible, avoids read overflow on next attempt */
-        }
-
-        if (matchIndex <= minChain) break;
-        matchIndex = NEXT_IN_CHAIN(matchIndex, chainMask);
-    }
-
-    if (dictMode == ZSTD_dictMatchState) {
-        const ZSTD_matchState_t* const dms = ms->dictMatchState;
-        const U32* const dmsChainTable = dms->chainTable;
-        const U32 dmsChainSize         = (1 << dms->cParams.chainLog);
-        const U32 dmsChainMask         = dmsChainSize - 1;
-        const U32 dmsLowestIndex       = dms->window.dictLimit;
-        const BYTE* const dmsBase      = dms->window.base;
-        const BYTE* const dmsEnd       = dms->window.nextSrc;
-        const U32 dmsSize              = (U32)(dmsEnd - dmsBase);
-        const U32 dmsIndexDelta        = dictLimit - dmsSize;
-        const U32 dmsMinChain = dmsSize > dmsChainSize ? dmsSize - dmsChainSize : 0;
-
-        matchIndex = dms->hashTable[ZSTD_hashPtr(ip, dms->cParams.hashLog, mls)];
-
-        for ( ; (matchIndex>dmsLowestIndex) & (nbAttempts>0) ; nbAttempts--) {
-            size_t currentMl=0;
-            const BYTE* const match = dmsBase + matchIndex;
-            assert(match+4 <= dmsEnd);
-            if (MEM_read32(match) == MEM_read32(ip))   /* assumption : matchIndex <= dictLimit-4 (by table construction) */
-                currentMl = ZSTD_count_2segments(ip+4, match+4, iLimit, dmsEnd, prefixStart) + 4;
-
-            /* save best solution */
-            if (currentMl > ml) {
-                ml = currentMl;
-                *offsetPtr = current - (matchIndex + dmsIndexDelta) + ZSTD_REP_MOVE;
-                if (ip+currentMl == iLimit) break; /* best possible, avoids read overflow on next attempt */
-            }
-
-            if (matchIndex <= dmsMinChain) break;
-            matchIndex = dmsChainTable[matchIndex & dmsChainMask];
-        }
-    }
-
-    return ml;
-}
-
-
-FORCE_INLINE_TEMPLATE size_t ZSTD_HcFindBestMatch_selectMLS (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* const iLimit,
-                        size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 4, ZSTD_noDict);
-    case 5 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 5, ZSTD_noDict);
-    case 7 :
-    case 6 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 6, ZSTD_noDict);
-    }
-}
-
-
-static size_t ZSTD_HcFindBestMatch_dictMatchState_selectMLS (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* const iLimit,
-                        size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 4, ZSTD_dictMatchState);
-    case 5 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 5, ZSTD_dictMatchState);
-    case 7 :
-    case 6 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 6, ZSTD_dictMatchState);
-    }
-}
-
-
-FORCE_INLINE_TEMPLATE size_t ZSTD_HcFindBestMatch_extDict_selectMLS (
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* const iLimit,
-                        size_t* offsetPtr)
-{
-    switch(ms->cParams.minMatch)
-    {
-    default : /* includes case 3 */
-    case 4 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 4, ZSTD_extDict);
-    case 5 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 5, ZSTD_extDict);
-    case 7 :
-    case 6 : return ZSTD_HcFindBestMatch_generic(ms, ip, iLimit, offsetPtr, 6, ZSTD_extDict);
-    }
-}
-
-
-/* *******************************
-*  Common parser - lazy strategy
-*********************************/
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_compressBlock_lazy_generic(
-                        ZSTD_matchState_t* ms, seqStore_t* seqStore,
-                        U32 rep[ZSTD_REP_NUM],
-                        const void* src, size_t srcSize,
-                        const U32 searchMethod, const U32 depth,
-                        ZSTD_dictMode_e const dictMode)
-{
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - 8;
-    const BYTE* const base = ms->window.base;
-    const U32 prefixLowestIndex = ms->window.dictLimit;
-    const BYTE* const prefixLowest = base + prefixLowestIndex;
-
-    typedef size_t (*searchMax_f)(
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* iLimit, size_t* offsetPtr);
-    searchMax_f const searchMax = dictMode == ZSTD_dictMatchState ?
-        (searchMethod ? ZSTD_BtFindBestMatch_dictMatchState_selectMLS : ZSTD_HcFindBestMatch_dictMatchState_selectMLS) :
-        (searchMethod ? ZSTD_BtFindBestMatch_selectMLS : ZSTD_HcFindBestMatch_selectMLS);
-    U32 offset_1 = rep[0], offset_2 = rep[1], savedOffset=0;
-
-    const ZSTD_matchState_t* const dms = ms->dictMatchState;
-    const U32 dictLowestIndex      = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.dictLimit : 0;
-    const BYTE* const dictBase     = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.base : NULL;
-    const BYTE* const dictLowest   = dictMode == ZSTD_dictMatchState ?
-                                     dictBase + dictLowestIndex : NULL;
-    const BYTE* const dictEnd      = dictMode == ZSTD_dictMatchState ?
-                                     dms->window.nextSrc : NULL;
-    const U32 dictIndexDelta       = dictMode == ZSTD_dictMatchState ?
-                                     prefixLowestIndex - (U32)(dictEnd - dictBase) :
-                                     0;
-    const U32 dictAndPrefixLength = (U32)(ip - prefixLowest + dictEnd - dictLowest);
-
-    /* init */
-    ip += (dictAndPrefixLength == 0);
-    if (dictMode == ZSTD_noDict) {
-        U32 const maxRep = (U32)(ip - prefixLowest);
-        if (offset_2 > maxRep) savedOffset = offset_2, offset_2 = 0;
-        if (offset_1 > maxRep) savedOffset = offset_1, offset_1 = 0;
-    }
-    if (dictMode == ZSTD_dictMatchState) {
-        /* dictMatchState repCode checks don't currently handle repCode == 0
-         * disabling. */
-        assert(offset_1 <= dictAndPrefixLength);
-        assert(offset_2 <= dictAndPrefixLength);
-    }
-
-    /* Match Loop */
-    while (ip < ilimit) {
-        size_t matchLength=0;
-        size_t offset=0;
-        const BYTE* start=ip+1;
-
-        /* check repCode */
-        if (dictMode == ZSTD_dictMatchState) {
-            const U32 repIndex = (U32)(ip - base) + 1 - offset_1;
-            const BYTE* repMatch = (dictMode == ZSTD_dictMatchState
-                                && repIndex < prefixLowestIndex) ?
-                                   dictBase + (repIndex - dictIndexDelta) :
-                                   base + repIndex;
-            if (((U32)((prefixLowestIndex-1) - repIndex) >= 3 /* intentional underflow */)
-                && (MEM_read32(repMatch) == MEM_read32(ip+1)) ) {
-                const BYTE* repMatchEnd = repIndex < prefixLowestIndex ? dictEnd : iend;
-                matchLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repMatchEnd, prefixLowest) + 4;
-                if (depth==0) goto _storeSequence;
-            }
-        }
-        if ( dictMode == ZSTD_noDict
-          && ((offset_1 > 0) & (MEM_read32(ip+1-offset_1) == MEM_read32(ip+1)))) {
-            matchLength = ZSTD_count(ip+1+4, ip+1+4-offset_1, iend) + 4;
-            if (depth==0) goto _storeSequence;
-        }
-
-        /* first search (depth 0) */
-        {   size_t offsetFound = 999999999;
-            size_t const ml2 = searchMax(ms, ip, iend, &offsetFound);
-            if (ml2 > matchLength)
-                matchLength = ml2, start = ip, offset=offsetFound;
-        }
-
-        if (matchLength < 4) {
-            ip += ((ip-anchor) >> kSearchStrength) + 1;   /* jump faster over incompressible sections */
-            continue;
-        }
-
-        /* let's try to find a better solution */
-        if (depth>=1)
-        while (ip<ilimit) {
-            ip ++;
-            if ( (dictMode == ZSTD_noDict)
-              && (offset) && ((offset_1>0) & (MEM_read32(ip) == MEM_read32(ip - offset_1)))) {
-                size_t const mlRep = ZSTD_count(ip+4, ip+4-offset_1, iend) + 4;
-                int const gain2 = (int)(mlRep * 3);
-                int const gain1 = (int)(matchLength*3 - ZSTD_highbit32((U32)offset+1) + 1);
-                if ((mlRep >= 4) && (gain2 > gain1))
-                    matchLength = mlRep, offset = 0, start = ip;
-            }
-            if (dictMode == ZSTD_dictMatchState) {
-                const U32 repIndex = (U32)(ip - base) - offset_1;
-                const BYTE* repMatch = repIndex < prefixLowestIndex ?
-                               dictBase + (repIndex - dictIndexDelta) :
-                               base + repIndex;
-                if (((U32)((prefixLowestIndex-1) - repIndex) >= 3 /* intentional underflow */)
-                    && (MEM_read32(repMatch) == MEM_read32(ip)) ) {
-                    const BYTE* repMatchEnd = repIndex < prefixLowestIndex ? dictEnd : iend;
-                    size_t const mlRep = ZSTD_count_2segments(ip+4, repMatch+4, iend, repMatchEnd, prefixLowest) + 4;
-                    int const gain2 = (int)(mlRep * 3);
-                    int const gain1 = (int)(matchLength*3 - ZSTD_highbit32((U32)offset+1) + 1);
-                    if ((mlRep >= 4) && (gain2 > gain1))
-                        matchLength = mlRep, offset = 0, start = ip;
-                }
-            }
-            {   size_t offset2=999999999;
-                size_t const ml2 = searchMax(ms, ip, iend, &offset2);
-                int const gain2 = (int)(ml2*4 - ZSTD_highbit32((U32)offset2+1));   /* raw approx */
-                int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 4);
-                if ((ml2 >= 4) && (gain2 > gain1)) {
-                    matchLength = ml2, offset = offset2, start = ip;
-                    continue;   /* search a better one */
-            }   }
-
-            /* let's find an even better one */
-            if ((depth==2) && (ip<ilimit)) {
-                ip ++;
-                if ( (dictMode == ZSTD_noDict)
-                  && (offset) && ((offset_1>0) & (MEM_read32(ip) == MEM_read32(ip - offset_1)))) {
-                    size_t const mlRep = ZSTD_count(ip+4, ip+4-offset_1, iend) + 4;
-                    int const gain2 = (int)(mlRep * 4);
-                    int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 1);
-                    if ((mlRep >= 4) && (gain2 > gain1))
-                        matchLength = mlRep, offset = 0, start = ip;
-                }
-                if (dictMode == ZSTD_dictMatchState) {
-                    const U32 repIndex = (U32)(ip - base) - offset_1;
-                    const BYTE* repMatch = repIndex < prefixLowestIndex ?
-                                   dictBase + (repIndex - dictIndexDelta) :
-                                   base + repIndex;
-                    if (((U32)((prefixLowestIndex-1) - repIndex) >= 3 /* intentional underflow */)
-                        && (MEM_read32(repMatch) == MEM_read32(ip)) ) {
-                        const BYTE* repMatchEnd = repIndex < prefixLowestIndex ? dictEnd : iend;
-                        size_t const mlRep = ZSTD_count_2segments(ip+4, repMatch+4, iend, repMatchEnd, prefixLowest) + 4;
-                        int const gain2 = (int)(mlRep * 4);
-                        int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 1);
-                        if ((mlRep >= 4) && (gain2 > gain1))
-                            matchLength = mlRep, offset = 0, start = ip;
-                    }
-                }
-                {   size_t offset2=999999999;
-                    size_t const ml2 = searchMax(ms, ip, iend, &offset2);
-                    int const gain2 = (int)(ml2*4 - ZSTD_highbit32((U32)offset2+1));   /* raw approx */
-                    int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 7);
-                    if ((ml2 >= 4) && (gain2 > gain1)) {
-                        matchLength = ml2, offset = offset2, start = ip;
-                        continue;
-            }   }   }
-            break;  /* nothing found : store previous solution */
-        }
-
-        /* NOTE:
-         * start[-offset+ZSTD_REP_MOVE-1] is undefined behavior.
-         * (-offset+ZSTD_REP_MOVE-1) is unsigned, and is added to start, which
-         * overflows the pointer, which is undefined behavior.
-         */
-        /* catch up */
-        if (offset) {
-            if (dictMode == ZSTD_noDict) {
-                while ( ((start > anchor) & (start - (offset-ZSTD_REP_MOVE) > prefixLowest))
-                     && (start[-1] == (start-(offset-ZSTD_REP_MOVE))[-1]) )  /* only search for offset within prefix */
-                    { start--; matchLength++; }
-            }
-            if (dictMode == ZSTD_dictMatchState) {
-                U32 const matchIndex = (U32)((start-base) - (offset - ZSTD_REP_MOVE));
-                const BYTE* match = (matchIndex < prefixLowestIndex) ? dictBase + matchIndex - dictIndexDelta : base + matchIndex;
-                const BYTE* const mStart = (matchIndex < prefixLowestIndex) ? dictLowest : prefixLowest;
-                while ((start>anchor) && (match>mStart) && (start[-1] == match[-1])) { start--; match--; matchLength++; }  /* catch up */
-            }
-            offset_2 = offset_1; offset_1 = (U32)(offset - ZSTD_REP_MOVE);
-        }
-        /* store sequence */
-_storeSequence:
-        {   size_t const litLength = start - anchor;
-            ZSTD_storeSeq(seqStore, litLength, anchor, (U32)offset, matchLength-MINMATCH);
-            anchor = ip = start + matchLength;
-        }
-
-        /* check immediate repcode */
-        if (dictMode == ZSTD_dictMatchState) {
-            while (ip <= ilimit) {
-                U32 const current2 = (U32)(ip-base);
-                U32 const repIndex = current2 - offset_2;
-                const BYTE* repMatch = dictMode == ZSTD_dictMatchState
-                    && repIndex < prefixLowestIndex ?
-                        dictBase - dictIndexDelta + repIndex :
-                        base + repIndex;
-                if ( ((U32)((prefixLowestIndex-1) - (U32)repIndex) >= 3 /* intentional overflow */)
-                   && (MEM_read32(repMatch) == MEM_read32(ip)) ) {
-                    const BYTE* const repEnd2 = repIndex < prefixLowestIndex ? dictEnd : iend;
-                    matchLength = ZSTD_count_2segments(ip+4, repMatch+4, iend, repEnd2, prefixLowest) + 4;
-                    offset = offset_2; offset_2 = offset_1; offset_1 = (U32)offset;   /* swap offset_2 <=> offset_1 */
-                    ZSTD_storeSeq(seqStore, 0, anchor, 0, matchLength-MINMATCH);
-                    ip += matchLength;
-                    anchor = ip;
-                    continue;
-                }
-                break;
-            }
-        }
-
-        if (dictMode == ZSTD_noDict) {
-            while ( ((ip <= ilimit) & (offset_2>0))
-                 && (MEM_read32(ip) == MEM_read32(ip - offset_2)) ) {
-                /* store sequence */
-                matchLength = ZSTD_count(ip+4, ip+4-offset_2, iend) + 4;
-                offset = offset_2; offset_2 = offset_1; offset_1 = (U32)offset; /* swap repcodes */
-                ZSTD_storeSeq(seqStore, 0, anchor, 0, matchLength-MINMATCH);
-                ip += matchLength;
-                anchor = ip;
-                continue;   /* faster when present ... (?) */
-    }   }   }
-
-    /* Save reps for next block */
-    rep[0] = offset_1 ? offset_1 : savedOffset;
-    rep[1] = offset_2 ? offset_2 : savedOffset;
-
-    /* Return the last literals size */
-    return iend - anchor;
-}
-
-
-size_t ZSTD_compressBlock_btlazy2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 1, 2, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_lazy2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 2, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_lazy(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 1, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_greedy(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 0, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_btlazy2_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 1, 2, ZSTD_dictMatchState);
-}
-
-size_t ZSTD_compressBlock_lazy2_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 2, ZSTD_dictMatchState);
-}
-
-size_t ZSTD_compressBlock_lazy_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 1, ZSTD_dictMatchState);
-}
-
-size_t ZSTD_compressBlock_greedy_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_generic(ms, seqStore, rep, src, srcSize, 0, 0, ZSTD_dictMatchState);
-}
-
-
-FORCE_INLINE_TEMPLATE
-size_t ZSTD_compressBlock_lazy_extDict_generic(
-                        ZSTD_matchState_t* ms, seqStore_t* seqStore,
-                        U32 rep[ZSTD_REP_NUM],
-                        const void* src, size_t srcSize,
-                        const U32 searchMethod, const U32 depth)
-{
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - 8;
-    const BYTE* const base = ms->window.base;
-    const U32 dictLimit = ms->window.dictLimit;
-    const U32 lowestIndex = ms->window.lowLimit;
-    const BYTE* const prefixStart = base + dictLimit;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const BYTE* const dictEnd  = dictBase + dictLimit;
-    const BYTE* const dictStart  = dictBase + lowestIndex;
-
-    typedef size_t (*searchMax_f)(
-                        ZSTD_matchState_t* ms,
-                        const BYTE* ip, const BYTE* iLimit, size_t* offsetPtr);
-    searchMax_f searchMax = searchMethod ? ZSTD_BtFindBestMatch_extDict_selectMLS : ZSTD_HcFindBestMatch_extDict_selectMLS;
-
-    U32 offset_1 = rep[0], offset_2 = rep[1];
-
-    /* init */
-    ip += (ip == prefixStart);
-
-    /* Match Loop */
-    while (ip < ilimit) {
-        size_t matchLength=0;
-        size_t offset=0;
-        const BYTE* start=ip+1;
-        U32 current = (U32)(ip-base);
-
-        /* check repCode */
-        {   const U32 repIndex = (U32)(current+1 - offset_1);
-            const BYTE* const repBase = repIndex < dictLimit ? dictBase : base;
-            const BYTE* const repMatch = repBase + repIndex;
-            if (((U32)((dictLimit-1) - repIndex) >= 3) & (repIndex > lowestIndex))   /* intentional overflow */
-            if (MEM_read32(ip+1) == MEM_read32(repMatch)) {
-                /* repcode detected we should take it */
-                const BYTE* const repEnd = repIndex < dictLimit ? dictEnd : iend;
-                matchLength = ZSTD_count_2segments(ip+1+4, repMatch+4, iend, repEnd, prefixStart) + 4;
-                if (depth==0) goto _storeSequence;
-        }   }
-
-        /* first search (depth 0) */
-        {   size_t offsetFound = 999999999;
-            size_t const ml2 = searchMax(ms, ip, iend, &offsetFound);
-            if (ml2 > matchLength)
-                matchLength = ml2, start = ip, offset=offsetFound;
-        }
-
-         if (matchLength < 4) {
-            ip += ((ip-anchor) >> kSearchStrength) + 1;   /* jump faster over incompressible sections */
-            continue;
-        }
-
-        /* let's try to find a better solution */
-        if (depth>=1)
-        while (ip<ilimit) {
-            ip ++;
-            current++;
-            /* check repCode */
-            if (offset) {
-                const U32 repIndex = (U32)(current - offset_1);
-                const BYTE* const repBase = repIndex < dictLimit ? dictBase : base;
-                const BYTE* const repMatch = repBase + repIndex;
-                if (((U32)((dictLimit-1) - repIndex) >= 3) & (repIndex > lowestIndex))  /* intentional overflow */
-                if (MEM_read32(ip) == MEM_read32(repMatch)) {
-                    /* repcode detected */
-                    const BYTE* const repEnd = repIndex < dictLimit ? dictEnd : iend;
-                    size_t const repLength = ZSTD_count_2segments(ip+4, repMatch+4, iend, repEnd, prefixStart) + 4;
-                    int const gain2 = (int)(repLength * 3);
-                    int const gain1 = (int)(matchLength*3 - ZSTD_highbit32((U32)offset+1) + 1);
-                    if ((repLength >= 4) && (gain2 > gain1))
-                        matchLength = repLength, offset = 0, start = ip;
-            }   }
-
-            /* search match, depth 1 */
-            {   size_t offset2=999999999;
-                size_t const ml2 = searchMax(ms, ip, iend, &offset2);
-                int const gain2 = (int)(ml2*4 - ZSTD_highbit32((U32)offset2+1));   /* raw approx */
-                int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 4);
-                if ((ml2 >= 4) && (gain2 > gain1)) {
-                    matchLength = ml2, offset = offset2, start = ip;
-                    continue;   /* search a better one */
-            }   }
-
-            /* let's find an even better one */
-            if ((depth==2) && (ip<ilimit)) {
-                ip ++;
-                current++;
-                /* check repCode */
-                if (offset) {
-                    const U32 repIndex = (U32)(current - offset_1);
-                    const BYTE* const repBase = repIndex < dictLimit ? dictBase : base;
-                    const BYTE* const repMatch = repBase + repIndex;
-                    if (((U32)((dictLimit-1) - repIndex) >= 3) & (repIndex > lowestIndex))  /* intentional overflow */
-                    if (MEM_read32(ip) == MEM_read32(repMatch)) {
-                        /* repcode detected */
-                        const BYTE* const repEnd = repIndex < dictLimit ? dictEnd : iend;
-                        size_t const repLength = ZSTD_count_2segments(ip+4, repMatch+4, iend, repEnd, prefixStart) + 4;
-                        int const gain2 = (int)(repLength * 4);
-                        int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 1);
-                        if ((repLength >= 4) && (gain2 > gain1))
-                            matchLength = repLength, offset = 0, start = ip;
-                }   }
-
-                /* search match, depth 2 */
-                {   size_t offset2=999999999;
-                    size_t const ml2 = searchMax(ms, ip, iend, &offset2);
-                    int const gain2 = (int)(ml2*4 - ZSTD_highbit32((U32)offset2+1));   /* raw approx */
-                    int const gain1 = (int)(matchLength*4 - ZSTD_highbit32((U32)offset+1) + 7);
-                    if ((ml2 >= 4) && (gain2 > gain1)) {
-                        matchLength = ml2, offset = offset2, start = ip;
-                        continue;
-            }   }   }
-            break;  /* nothing found : store previous solution */
-        }
-
-        /* catch up */
-        if (offset) {
-            U32 const matchIndex = (U32)((start-base) - (offset - ZSTD_REP_MOVE));
-            const BYTE* match = (matchIndex < dictLimit) ? dictBase + matchIndex : base + matchIndex;
-            const BYTE* const mStart = (matchIndex < dictLimit) ? dictStart : prefixStart;
-            while ((start>anchor) && (match>mStart) && (start[-1] == match[-1])) { start--; match--; matchLength++; }  /* catch up */
-            offset_2 = offset_1; offset_1 = (U32)(offset - ZSTD_REP_MOVE);
-        }
-
-        /* store sequence */
-_storeSequence:
-        {   size_t const litLength = start - anchor;
-            ZSTD_storeSeq(seqStore, litLength, anchor, (U32)offset, matchLength-MINMATCH);
-            anchor = ip = start + matchLength;
-        }
-
-        /* check immediate repcode */
-        while (ip <= ilimit) {
-            const U32 repIndex = (U32)((ip-base) - offset_2);
-            const BYTE* const repBase = repIndex < dictLimit ? dictBase : base;
-            const BYTE* const repMatch = repBase + repIndex;
-            if (((U32)((dictLimit-1) - repIndex) >= 3) & (repIndex > lowestIndex))  /* intentional overflow */
-            if (MEM_read32(ip) == MEM_read32(repMatch)) {
-                /* repcode detected we should take it */
-                const BYTE* const repEnd = repIndex < dictLimit ? dictEnd : iend;
-                matchLength = ZSTD_count_2segments(ip+4, repMatch+4, iend, repEnd, prefixStart) + 4;
-                offset = offset_2; offset_2 = offset_1; offset_1 = (U32)offset;   /* swap offset history */
-                ZSTD_storeSeq(seqStore, 0, anchor, 0, matchLength-MINMATCH);
-                ip += matchLength;
-                anchor = ip;
-                continue;   /* faster when present ... (?) */
-            }
-            break;
-    }   }
-
-    /* Save reps for next block */
-    rep[0] = offset_1;
-    rep[1] = offset_2;
-
-    /* Return the last literals size */
-    return iend - anchor;
-}
-
-
-size_t ZSTD_compressBlock_greedy_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_lazy_extDict_generic(ms, seqStore, rep, src, srcSize, 0, 0);
-}
-
-size_t ZSTD_compressBlock_lazy_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-
-{
-    return ZSTD_compressBlock_lazy_extDict_generic(ms, seqStore, rep, src, srcSize, 0, 1);
-}
-
-size_t ZSTD_compressBlock_lazy2_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-
-{
-    return ZSTD_compressBlock_lazy_extDict_generic(ms, seqStore, rep, src, srcSize, 0, 2);
-}
-
-size_t ZSTD_compressBlock_btlazy2_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize)
-
-{
-    return ZSTD_compressBlock_lazy_extDict_generic(ms, seqStore, rep, src, srcSize, 1, 2);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_lazy.h b/vendor/github.com/DataDog/zstd/zstd_lazy.h
deleted file mode 100644
index bb17630..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_lazy.h
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_LAZY_H
-#define ZSTD_LAZY_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#include "zstd_compress_internal.h"
-
-U32 ZSTD_insertAndFindFirstIndex(ZSTD_matchState_t* ms, const BYTE* ip);
-
-void ZSTD_preserveUnsortedMark (U32* const table, U32 const size, U32 const reducerValue);  /*! used in ZSTD_reduceIndex(). preemptively increase value of ZSTD_DUBT_UNSORTED_MARK */
-
-size_t ZSTD_compressBlock_btlazy2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_greedy(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-size_t ZSTD_compressBlock_btlazy2_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy2_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_greedy_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-size_t ZSTD_compressBlock_greedy_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_lazy2_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_btlazy2_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_LAZY_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_ldm.c b/vendor/github.com/DataDog/zstd/zstd_ldm.c
deleted file mode 100644
index 3dcf86e..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_ldm.c
+++ /dev/null
@@ -1,597 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- */
-
-#include "zstd_ldm.h"
-
-#include "debug.h"
-#include "zstd_fast.h"          /* ZSTD_fillHashTable() */
-#include "zstd_double_fast.h"   /* ZSTD_fillDoubleHashTable() */
-
-#define LDM_BUCKET_SIZE_LOG 3
-#define LDM_MIN_MATCH_LENGTH 64
-#define LDM_HASH_RLOG 7
-#define LDM_HASH_CHAR_OFFSET 10
-
-void ZSTD_ldm_adjustParameters(ldmParams_t* params,
-                               ZSTD_compressionParameters const* cParams)
-{
-    params->windowLog = cParams->windowLog;
-    ZSTD_STATIC_ASSERT(LDM_BUCKET_SIZE_LOG <= ZSTD_LDM_BUCKETSIZELOG_MAX);
-    DEBUGLOG(4, "ZSTD_ldm_adjustParameters");
-    if (!params->bucketSizeLog) params->bucketSizeLog = LDM_BUCKET_SIZE_LOG;
-    if (!params->minMatchLength) params->minMatchLength = LDM_MIN_MATCH_LENGTH;
-    if (cParams->strategy >= ZSTD_btopt) {
-      /* Get out of the way of the optimal parser */
-      U32 const minMatch = MAX(cParams->targetLength, params->minMatchLength);
-      assert(minMatch >= ZSTD_LDM_MINMATCH_MIN);
-      assert(minMatch <= ZSTD_LDM_MINMATCH_MAX);
-      params->minMatchLength = minMatch;
-    }
-    if (params->hashLog == 0) {
-        params->hashLog = MAX(ZSTD_HASHLOG_MIN, params->windowLog - LDM_HASH_RLOG);
-        assert(params->hashLog <= ZSTD_HASHLOG_MAX);
-    }
-    if (params->hashRateLog == 0) {
-        params->hashRateLog = params->windowLog < params->hashLog
-                                   ? 0
-                                   : params->windowLog - params->hashLog;
-    }
-    params->bucketSizeLog = MIN(params->bucketSizeLog, params->hashLog);
-}
-
-size_t ZSTD_ldm_getTableSize(ldmParams_t params)
-{
-    size_t const ldmHSize = ((size_t)1) << params.hashLog;
-    size_t const ldmBucketSizeLog = MIN(params.bucketSizeLog, params.hashLog);
-    size_t const ldmBucketSize =
-        ((size_t)1) << (params.hashLog - ldmBucketSizeLog);
-    size_t const totalSize = ldmBucketSize + ldmHSize * sizeof(ldmEntry_t);
-    return params.enableLdm ? totalSize : 0;
-}
-
-size_t ZSTD_ldm_getMaxNbSeq(ldmParams_t params, size_t maxChunkSize)
-{
-    return params.enableLdm ? (maxChunkSize / params.minMatchLength) : 0;
-}
-
-/** ZSTD_ldm_getSmallHash() :
- *  numBits should be <= 32
- *  If numBits==0, returns 0.
- *  @return : the most significant numBits of value. */
-static U32 ZSTD_ldm_getSmallHash(U64 value, U32 numBits)
-{
-    assert(numBits <= 32);
-    return numBits == 0 ? 0 : (U32)(value >> (64 - numBits));
-}
-
-/** ZSTD_ldm_getChecksum() :
- *  numBitsToDiscard should be <= 32
- *  @return : the next most significant 32 bits after numBitsToDiscard */
-static U32 ZSTD_ldm_getChecksum(U64 hash, U32 numBitsToDiscard)
-{
-    assert(numBitsToDiscard <= 32);
-    return (hash >> (64 - 32 - numBitsToDiscard)) & 0xFFFFFFFF;
-}
-
-/** ZSTD_ldm_getTag() ;
- *  Given the hash, returns the most significant numTagBits bits
- *  after (32 + hbits) bits.
- *
- *  If there are not enough bits remaining, return the last
- *  numTagBits bits. */
-static U32 ZSTD_ldm_getTag(U64 hash, U32 hbits, U32 numTagBits)
-{
-    assert(numTagBits < 32 && hbits <= 32);
-    if (32 - hbits < numTagBits) {
-        return hash & (((U32)1 << numTagBits) - 1);
-    } else {
-        return (hash >> (32 - hbits - numTagBits)) & (((U32)1 << numTagBits) - 1);
-    }
-}
-
-/** ZSTD_ldm_getBucket() :
- *  Returns a pointer to the start of the bucket associated with hash. */
-static ldmEntry_t* ZSTD_ldm_getBucket(
-        ldmState_t* ldmState, size_t hash, ldmParams_t const ldmParams)
-{
-    return ldmState->hashTable + (hash << ldmParams.bucketSizeLog);
-}
-
-/** ZSTD_ldm_insertEntry() :
- *  Insert the entry with corresponding hash into the hash table */
-static void ZSTD_ldm_insertEntry(ldmState_t* ldmState,
-                                 size_t const hash, const ldmEntry_t entry,
-                                 ldmParams_t const ldmParams)
-{
-    BYTE* const bucketOffsets = ldmState->bucketOffsets;
-    *(ZSTD_ldm_getBucket(ldmState, hash, ldmParams) + bucketOffsets[hash]) = entry;
-    bucketOffsets[hash]++;
-    bucketOffsets[hash] &= ((U32)1 << ldmParams.bucketSizeLog) - 1;
-}
-
-/** ZSTD_ldm_makeEntryAndInsertByTag() :
- *
- *  Gets the small hash, checksum, and tag from the rollingHash.
- *
- *  If the tag matches (1 << ldmParams.hashRateLog)-1, then
- *  creates an ldmEntry from the offset, and inserts it into the hash table.
- *
- *  hBits is the length of the small hash, which is the most significant hBits
- *  of rollingHash. The checksum is the next 32 most significant bits, followed
- *  by ldmParams.hashRateLog bits that make up the tag. */
-static void ZSTD_ldm_makeEntryAndInsertByTag(ldmState_t* ldmState,
-                                             U64 const rollingHash,
-                                             U32 const hBits,
-                                             U32 const offset,
-                                             ldmParams_t const ldmParams)
-{
-    U32 const tag = ZSTD_ldm_getTag(rollingHash, hBits, ldmParams.hashRateLog);
-    U32 const tagMask = ((U32)1 << ldmParams.hashRateLog) - 1;
-    if (tag == tagMask) {
-        U32 const hash = ZSTD_ldm_getSmallHash(rollingHash, hBits);
-        U32 const checksum = ZSTD_ldm_getChecksum(rollingHash, hBits);
-        ldmEntry_t entry;
-        entry.offset = offset;
-        entry.checksum = checksum;
-        ZSTD_ldm_insertEntry(ldmState, hash, entry, ldmParams);
-    }
-}
-
-/** ZSTD_ldm_countBackwardsMatch() :
- *  Returns the number of bytes that match backwards before pIn and pMatch.
- *
- *  We count only bytes where pMatch >= pBase and pIn >= pAnchor. */
-static size_t ZSTD_ldm_countBackwardsMatch(
-            const BYTE* pIn, const BYTE* pAnchor,
-            const BYTE* pMatch, const BYTE* pBase)
-{
-    size_t matchLength = 0;
-    while (pIn > pAnchor && pMatch > pBase && pIn[-1] == pMatch[-1]) {
-        pIn--;
-        pMatch--;
-        matchLength++;
-    }
-    return matchLength;
-}
-
-/** ZSTD_ldm_fillFastTables() :
- *
- *  Fills the relevant tables for the ZSTD_fast and ZSTD_dfast strategies.
- *  This is similar to ZSTD_loadDictionaryContent.
- *
- *  The tables for the other strategies are filled within their
- *  block compressors. */
-static size_t ZSTD_ldm_fillFastTables(ZSTD_matchState_t* ms,
-                                      void const* end)
-{
-    const BYTE* const iend = (const BYTE*)end;
-
-    switch(ms->cParams.strategy)
-    {
-    case ZSTD_fast:
-        ZSTD_fillHashTable(ms, iend, ZSTD_dtlm_fast);
-        break;
-
-    case ZSTD_dfast:
-        ZSTD_fillDoubleHashTable(ms, iend, ZSTD_dtlm_fast);
-        break;
-
-    case ZSTD_greedy:
-    case ZSTD_lazy:
-    case ZSTD_lazy2:
-    case ZSTD_btlazy2:
-    case ZSTD_btopt:
-    case ZSTD_btultra:
-    case ZSTD_btultra2:
-        break;
-    default:
-        assert(0);  /* not possible : not a valid strategy id */
-    }
-
-    return 0;
-}
-
-/** ZSTD_ldm_fillLdmHashTable() :
- *
- *  Fills hashTable from (lastHashed + 1) to iend (non-inclusive).
- *  lastHash is the rolling hash that corresponds to lastHashed.
- *
- *  Returns the rolling hash corresponding to position iend-1. */
-static U64 ZSTD_ldm_fillLdmHashTable(ldmState_t* state,
-                                     U64 lastHash, const BYTE* lastHashed,
-                                     const BYTE* iend, const BYTE* base,
-                                     U32 hBits, ldmParams_t const ldmParams)
-{
-    U64 rollingHash = lastHash;
-    const BYTE* cur = lastHashed + 1;
-
-    while (cur < iend) {
-        rollingHash = ZSTD_rollingHash_rotate(rollingHash, cur[-1],
-                                              cur[ldmParams.minMatchLength-1],
-                                              state->hashPower);
-        ZSTD_ldm_makeEntryAndInsertByTag(state,
-                                         rollingHash, hBits,
-                                         (U32)(cur - base), ldmParams);
-        ++cur;
-    }
-    return rollingHash;
-}
-
-
-/** ZSTD_ldm_limitTableUpdate() :
- *
- *  Sets cctx->nextToUpdate to a position corresponding closer to anchor
- *  if it is far way
- *  (after a long match, only update tables a limited amount). */
-static void ZSTD_ldm_limitTableUpdate(ZSTD_matchState_t* ms, const BYTE* anchor)
-{
-    U32 const current = (U32)(anchor - ms->window.base);
-    if (current > ms->nextToUpdate + 1024) {
-        ms->nextToUpdate =
-            current - MIN(512, current - ms->nextToUpdate - 1024);
-    }
-}
-
-static size_t ZSTD_ldm_generateSequences_internal(
-        ldmState_t* ldmState, rawSeqStore_t* rawSeqStore,
-        ldmParams_t const* params, void const* src, size_t srcSize)
-{
-    /* LDM parameters */
-    int const extDict = ZSTD_window_hasExtDict(ldmState->window);
-    U32 const minMatchLength = params->minMatchLength;
-    U64 const hashPower = ldmState->hashPower;
-    U32 const hBits = params->hashLog - params->bucketSizeLog;
-    U32 const ldmBucketSize = 1U << params->bucketSizeLog;
-    U32 const hashRateLog = params->hashRateLog;
-    U32 const ldmTagMask = (1U << params->hashRateLog) - 1;
-    /* Prefix and extDict parameters */
-    U32 const dictLimit = ldmState->window.dictLimit;
-    U32 const lowestIndex = extDict ? ldmState->window.lowLimit : dictLimit;
-    BYTE const* const base = ldmState->window.base;
-    BYTE const* const dictBase = extDict ? ldmState->window.dictBase : NULL;
-    BYTE const* const dictStart = extDict ? dictBase + lowestIndex : NULL;
-    BYTE const* const dictEnd = extDict ? dictBase + dictLimit : NULL;
-    BYTE const* const lowPrefixPtr = base + dictLimit;
-    /* Input bounds */
-    BYTE const* const istart = (BYTE const*)src;
-    BYTE const* const iend = istart + srcSize;
-    BYTE const* const ilimit = iend - MAX(minMatchLength, HASH_READ_SIZE);
-    /* Input positions */
-    BYTE const* anchor = istart;
-    BYTE const* ip = istart;
-    /* Rolling hash */
-    BYTE const* lastHashed = NULL;
-    U64 rollingHash = 0;
-
-    while (ip <= ilimit) {
-        size_t mLength;
-        U32 const current = (U32)(ip - base);
-        size_t forwardMatchLength = 0, backwardMatchLength = 0;
-        ldmEntry_t* bestEntry = NULL;
-        if (ip != istart) {
-            rollingHash = ZSTD_rollingHash_rotate(rollingHash, lastHashed[0],
-                                                  lastHashed[minMatchLength],
-                                                  hashPower);
-        } else {
-            rollingHash = ZSTD_rollingHash_compute(ip, minMatchLength);
-        }
-        lastHashed = ip;
-
-        /* Do not insert and do not look for a match */
-        if (ZSTD_ldm_getTag(rollingHash, hBits, hashRateLog) != ldmTagMask) {
-           ip++;
-           continue;
-        }
-
-        /* Get the best entry and compute the match lengths */
-        {
-            ldmEntry_t* const bucket =
-                ZSTD_ldm_getBucket(ldmState,
-                                   ZSTD_ldm_getSmallHash(rollingHash, hBits),
-                                   *params);
-            ldmEntry_t* cur;
-            size_t bestMatchLength = 0;
-            U32 const checksum = ZSTD_ldm_getChecksum(rollingHash, hBits);
-
-            for (cur = bucket; cur < bucket + ldmBucketSize; ++cur) {
-                size_t curForwardMatchLength, curBackwardMatchLength,
-                       curTotalMatchLength;
-                if (cur->checksum != checksum || cur->offset <= lowestIndex) {
-                    continue;
-                }
-                if (extDict) {
-                    BYTE const* const curMatchBase =
-                        cur->offset < dictLimit ? dictBase : base;
-                    BYTE const* const pMatch = curMatchBase + cur->offset;
-                    BYTE const* const matchEnd =
-                        cur->offset < dictLimit ? dictEnd : iend;
-                    BYTE const* const lowMatchPtr =
-                        cur->offset < dictLimit ? dictStart : lowPrefixPtr;
-
-                    curForwardMatchLength = ZSTD_count_2segments(
-                                                ip, pMatch, iend,
-                                                matchEnd, lowPrefixPtr);
-                    if (curForwardMatchLength < minMatchLength) {
-                        continue;
-                    }
-                    curBackwardMatchLength =
-                        ZSTD_ldm_countBackwardsMatch(ip, anchor, pMatch,
-                                                     lowMatchPtr);
-                    curTotalMatchLength = curForwardMatchLength +
-                                          curBackwardMatchLength;
-                } else { /* !extDict */
-                    BYTE const* const pMatch = base + cur->offset;
-                    curForwardMatchLength = ZSTD_count(ip, pMatch, iend);
-                    if (curForwardMatchLength < minMatchLength) {
-                        continue;
-                    }
-                    curBackwardMatchLength =
-                        ZSTD_ldm_countBackwardsMatch(ip, anchor, pMatch,
-                                                     lowPrefixPtr);
-                    curTotalMatchLength = curForwardMatchLength +
-                                          curBackwardMatchLength;
-                }
-
-                if (curTotalMatchLength > bestMatchLength) {
-                    bestMatchLength = curTotalMatchLength;
-                    forwardMatchLength = curForwardMatchLength;
-                    backwardMatchLength = curBackwardMatchLength;
-                    bestEntry = cur;
-                }
-            }
-        }
-
-        /* No match found -- continue searching */
-        if (bestEntry == NULL) {
-            ZSTD_ldm_makeEntryAndInsertByTag(ldmState, rollingHash,
-                                             hBits, current,
-                                             *params);
-            ip++;
-            continue;
-        }
-
-        /* Match found */
-        mLength = forwardMatchLength + backwardMatchLength;
-        ip -= backwardMatchLength;
-
-        {
-            /* Store the sequence:
-             * ip = current - backwardMatchLength
-             * The match is at (bestEntry->offset - backwardMatchLength)
-             */
-            U32 const matchIndex = bestEntry->offset;
-            U32 const offset = current - matchIndex;
-            rawSeq* const seq = rawSeqStore->seq + rawSeqStore->size;
-
-            /* Out of sequence storage */
-            if (rawSeqStore->size == rawSeqStore->capacity)
-                return ERROR(dstSize_tooSmall);
-            seq->litLength = (U32)(ip - anchor);
-            seq->matchLength = (U32)mLength;
-            seq->offset = offset;
-            rawSeqStore->size++;
-        }
-
-        /* Insert the current entry into the hash table */
-        ZSTD_ldm_makeEntryAndInsertByTag(ldmState, rollingHash, hBits,
-                                         (U32)(lastHashed - base),
-                                         *params);
-
-        assert(ip + backwardMatchLength == lastHashed);
-
-        /* Fill the hash table from lastHashed+1 to ip+mLength*/
-        /* Heuristic: don't need to fill the entire table at end of block */
-        if (ip + mLength <= ilimit) {
-            rollingHash = ZSTD_ldm_fillLdmHashTable(
-                              ldmState, rollingHash, lastHashed,
-                              ip + mLength, base, hBits, *params);
-            lastHashed = ip + mLength - 1;
-        }
-        ip += mLength;
-        anchor = ip;
-    }
-    return iend - anchor;
-}
-
-/*! ZSTD_ldm_reduceTable() :
- *  reduce table indexes by `reducerValue` */
-static void ZSTD_ldm_reduceTable(ldmEntry_t* const table, U32 const size,
-                                 U32 const reducerValue)
-{
-    U32 u;
-    for (u = 0; u < size; u++) {
-        if (table[u].offset < reducerValue) table[u].offset = 0;
-        else table[u].offset -= reducerValue;
-    }
-}
-
-size_t ZSTD_ldm_generateSequences(
-        ldmState_t* ldmState, rawSeqStore_t* sequences,
-        ldmParams_t const* params, void const* src, size_t srcSize)
-{
-    U32 const maxDist = 1U << params->windowLog;
-    BYTE const* const istart = (BYTE const*)src;
-    BYTE const* const iend = istart + srcSize;
-    size_t const kMaxChunkSize = 1 << 20;
-    size_t const nbChunks = (srcSize / kMaxChunkSize) + ((srcSize % kMaxChunkSize) != 0);
-    size_t chunk;
-    size_t leftoverSize = 0;
-
-    assert(ZSTD_CHUNKSIZE_MAX >= kMaxChunkSize);
-    /* Check that ZSTD_window_update() has been called for this chunk prior
-     * to passing it to this function.
-     */
-    assert(ldmState->window.nextSrc >= (BYTE const*)src + srcSize);
-    /* The input could be very large (in zstdmt), so it must be broken up into
-     * chunks to enforce the maximum distance and handle overflow correction.
-     */
-    assert(sequences->pos <= sequences->size);
-    assert(sequences->size <= sequences->capacity);
-    for (chunk = 0; chunk < nbChunks && sequences->size < sequences->capacity; ++chunk) {
-        BYTE const* const chunkStart = istart + chunk * kMaxChunkSize;
-        size_t const remaining = (size_t)(iend - chunkStart);
-        BYTE const *const chunkEnd =
-            (remaining < kMaxChunkSize) ? iend : chunkStart + kMaxChunkSize;
-        size_t const chunkSize = chunkEnd - chunkStart;
-        size_t newLeftoverSize;
-        size_t const prevSize = sequences->size;
-
-        assert(chunkStart < iend);
-        /* 1. Perform overflow correction if necessary. */
-        if (ZSTD_window_needOverflowCorrection(ldmState->window, chunkEnd)) {
-            U32 const ldmHSize = 1U << params->hashLog;
-            U32 const correction = ZSTD_window_correctOverflow(
-                &ldmState->window, /* cycleLog */ 0, maxDist, chunkStart);
-            ZSTD_ldm_reduceTable(ldmState->hashTable, ldmHSize, correction);
-        }
-        /* 2. We enforce the maximum offset allowed.
-         *
-         * kMaxChunkSize should be small enough that we don't lose too much of
-         * the window through early invalidation.
-         * TODO: * Test the chunk size.
-         *       * Try invalidation after the sequence generation and test the
-         *         the offset against maxDist directly.
-         */
-        ZSTD_window_enforceMaxDist(&ldmState->window, chunkEnd, maxDist, NULL, NULL);
-        /* 3. Generate the sequences for the chunk, and get newLeftoverSize. */
-        newLeftoverSize = ZSTD_ldm_generateSequences_internal(
-            ldmState, sequences, params, chunkStart, chunkSize);
-        if (ZSTD_isError(newLeftoverSize))
-            return newLeftoverSize;
-        /* 4. We add the leftover literals from previous iterations to the first
-         *    newly generated sequence, or add the `newLeftoverSize` if none are
-         *    generated.
-         */
-        /* Prepend the leftover literals from the last call */
-        if (prevSize < sequences->size) {
-            sequences->seq[prevSize].litLength += (U32)leftoverSize;
-            leftoverSize = newLeftoverSize;
-        } else {
-            assert(newLeftoverSize == chunkSize);
-            leftoverSize += chunkSize;
-        }
-    }
-    return 0;
-}
-
-void ZSTD_ldm_skipSequences(rawSeqStore_t* rawSeqStore, size_t srcSize, U32 const minMatch) {
-    while (srcSize > 0 && rawSeqStore->pos < rawSeqStore->size) {
-        rawSeq* seq = rawSeqStore->seq + rawSeqStore->pos;
-        if (srcSize <= seq->litLength) {
-            /* Skip past srcSize literals */
-            seq->litLength -= (U32)srcSize;
-            return;
-        }
-        srcSize -= seq->litLength;
-        seq->litLength = 0;
-        if (srcSize < seq->matchLength) {
-            /* Skip past the first srcSize of the match */
-            seq->matchLength -= (U32)srcSize;
-            if (seq->matchLength < minMatch) {
-                /* The match is too short, omit it */
-                if (rawSeqStore->pos + 1 < rawSeqStore->size) {
-                    seq[1].litLength += seq[0].matchLength;
-                }
-                rawSeqStore->pos++;
-            }
-            return;
-        }
-        srcSize -= seq->matchLength;
-        seq->matchLength = 0;
-        rawSeqStore->pos++;
-    }
-}
-
-/**
- * If the sequence length is longer than remaining then the sequence is split
- * between this block and the next.
- *
- * Returns the current sequence to handle, or if the rest of the block should
- * be literals, it returns a sequence with offset == 0.
- */
-static rawSeq maybeSplitSequence(rawSeqStore_t* rawSeqStore,
-                                 U32 const remaining, U32 const minMatch)
-{
-    rawSeq sequence = rawSeqStore->seq[rawSeqStore->pos];
-    assert(sequence.offset > 0);
-    /* Likely: No partial sequence */
-    if (remaining >= sequence.litLength + sequence.matchLength) {
-        rawSeqStore->pos++;
-        return sequence;
-    }
-    /* Cut the sequence short (offset == 0 ==> rest is literals). */
-    if (remaining <= sequence.litLength) {
-        sequence.offset = 0;
-    } else if (remaining < sequence.litLength + sequence.matchLength) {
-        sequence.matchLength = remaining - sequence.litLength;
-        if (sequence.matchLength < minMatch) {
-            sequence.offset = 0;
-        }
-    }
-    /* Skip past `remaining` bytes for the future sequences. */
-    ZSTD_ldm_skipSequences(rawSeqStore, remaining, minMatch);
-    return sequence;
-}
-
-size_t ZSTD_ldm_blockCompress(rawSeqStore_t* rawSeqStore,
-    ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-    void const* src, size_t srcSize)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    unsigned const minMatch = cParams->minMatch;
-    ZSTD_blockCompressor const blockCompressor =
-        ZSTD_selectBlockCompressor(cParams->strategy, ZSTD_matchState_dictMode(ms));
-    /* Input bounds */
-    BYTE const* const istart = (BYTE const*)src;
-    BYTE const* const iend = istart + srcSize;
-    /* Input positions */
-    BYTE const* ip = istart;
-
-    DEBUGLOG(5, "ZSTD_ldm_blockCompress: srcSize=%zu", srcSize);
-    assert(rawSeqStore->pos <= rawSeqStore->size);
-    assert(rawSeqStore->size <= rawSeqStore->capacity);
-    /* Loop through each sequence and apply the block compressor to the lits */
-    while (rawSeqStore->pos < rawSeqStore->size && ip < iend) {
-        /* maybeSplitSequence updates rawSeqStore->pos */
-        rawSeq const sequence = maybeSplitSequence(rawSeqStore,
-                                                   (U32)(iend - ip), minMatch);
-        int i;
-        /* End signal */
-        if (sequence.offset == 0)
-            break;
-
-        assert(sequence.offset <= (1U << cParams->windowLog));
-        assert(ip + sequence.litLength + sequence.matchLength <= iend);
-
-        /* Fill tables for block compressor */
-        ZSTD_ldm_limitTableUpdate(ms, ip);
-        ZSTD_ldm_fillFastTables(ms, ip);
-        /* Run the block compressor */
-        DEBUGLOG(5, "calling block compressor on segment of size %u", sequence.litLength);
-        {
-            size_t const newLitLength =
-                blockCompressor(ms, seqStore, rep, ip, sequence.litLength);
-            ip += sequence.litLength;
-            /* Update the repcodes */
-            for (i = ZSTD_REP_NUM - 1; i > 0; i--)
-                rep[i] = rep[i-1];
-            rep[0] = sequence.offset;
-            /* Store the sequence */
-            ZSTD_storeSeq(seqStore, newLitLength, ip - newLitLength,
-                          sequence.offset + ZSTD_REP_MOVE,
-                          sequence.matchLength - MINMATCH);
-            ip += sequence.matchLength;
-        }
-    }
-    /* Fill the tables for the block compressor */
-    ZSTD_ldm_limitTableUpdate(ms, ip);
-    ZSTD_ldm_fillFastTables(ms, ip);
-    /* Compress the last literals */
-    return blockCompressor(ms, seqStore, rep, ip, iend - ip);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_ldm.h b/vendor/github.com/DataDog/zstd/zstd_ldm.h
deleted file mode 100644
index a478461..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_ldm.h
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- */
-
-#ifndef ZSTD_LDM_H
-#define ZSTD_LDM_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#include "zstd_compress_internal.h"   /* ldmParams_t, U32 */
-#include "zstd.h"   /* ZSTD_CCtx, size_t */
-
-/*-*************************************
-*  Long distance matching
-***************************************/
-
-#define ZSTD_LDM_DEFAULT_WINDOW_LOG ZSTD_WINDOWLOG_LIMIT_DEFAULT
-
-/**
- * ZSTD_ldm_generateSequences():
- *
- * Generates the sequences using the long distance match finder.
- * Generates long range matching sequences in `sequences`, which parse a prefix
- * of the source. `sequences` must be large enough to store every sequence,
- * which can be checked with `ZSTD_ldm_getMaxNbSeq()`.
- * @returns 0 or an error code.
- *
- * NOTE: The user must have called ZSTD_window_update() for all of the input
- * they have, even if they pass it to ZSTD_ldm_generateSequences() in chunks.
- * NOTE: This function returns an error if it runs out of space to store
- *       sequences.
- */
-size_t ZSTD_ldm_generateSequences(
-            ldmState_t* ldms, rawSeqStore_t* sequences,
-            ldmParams_t const* params, void const* src, size_t srcSize);
-
-/**
- * ZSTD_ldm_blockCompress():
- *
- * Compresses a block using the predefined sequences, along with a secondary
- * block compressor. The literals section of every sequence is passed to the
- * secondary block compressor, and those sequences are interspersed with the
- * predefined sequences. Returns the length of the last literals.
- * Updates `rawSeqStore.pos` to indicate how many sequences have been consumed.
- * `rawSeqStore.seq` may also be updated to split the last sequence between two
- * blocks.
- * @return The length of the last literals.
- *
- * NOTE: The source must be at most the maximum block size, but the predefined
- * sequences can be any size, and may be longer than the block. In the case that
- * they are longer than the block, the last sequences may need to be split into
- * two. We handle that case correctly, and update `rawSeqStore` appropriately.
- * NOTE: This function does not return any errors.
- */
-size_t ZSTD_ldm_blockCompress(rawSeqStore_t* rawSeqStore,
-            ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-            void const* src, size_t srcSize);
-
-/**
- * ZSTD_ldm_skipSequences():
- *
- * Skip past `srcSize` bytes worth of sequences in `rawSeqStore`.
- * Avoids emitting matches less than `minMatch` bytes.
- * Must be called for data with is not passed to ZSTD_ldm_blockCompress().
- */
-void ZSTD_ldm_skipSequences(rawSeqStore_t* rawSeqStore, size_t srcSize,
-    U32 const minMatch);
-
-
-/** ZSTD_ldm_getTableSize() :
- *  Estimate the space needed for long distance matching tables or 0 if LDM is
- *  disabled.
- */
-size_t ZSTD_ldm_getTableSize(ldmParams_t params);
-
-/** ZSTD_ldm_getSeqSpace() :
- *  Return an upper bound on the number of sequences that can be produced by
- *  the long distance matcher, or 0 if LDM is disabled.
- */
-size_t ZSTD_ldm_getMaxNbSeq(ldmParams_t params, size_t maxChunkSize);
-
-/** ZSTD_ldm_adjustParameters() :
- *  If the params->hashRateLog is not set, set it to its default value based on
- *  windowLog and params->hashLog.
- *
- *  Ensures that params->bucketSizeLog is <= params->hashLog (setting it to
- *  params->hashLog if it is not).
- *
- *  Ensures that the minMatchLength >= targetLength during optimal parsing.
- */
-void ZSTD_ldm_adjustParameters(ldmParams_t* params,
-                               ZSTD_compressionParameters const* cParams);
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_FAST_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_legacy.h b/vendor/github.com/DataDog/zstd/zstd_legacy.h
deleted file mode 100644
index 0dbd3c7..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_legacy.h
+++ /dev/null
@@ -1,415 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_LEGACY_H
-#define ZSTD_LEGACY_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include "mem.h"            /* MEM_STATIC */
-#include "error_private.h"  /* ERROR */
-#include "zstd_internal.h"  /* ZSTD_inBuffer, ZSTD_outBuffer, ZSTD_frameSizeInfo */
-
-#if !defined (ZSTD_LEGACY_SUPPORT) || (ZSTD_LEGACY_SUPPORT == 0)
-#  undef ZSTD_LEGACY_SUPPORT
-#  define ZSTD_LEGACY_SUPPORT 8
-#endif
-
-#if (ZSTD_LEGACY_SUPPORT <= 1)
-#  include "zstd_v01.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 2)
-#  include "zstd_v02.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 3)
-#  include "zstd_v03.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-#  include "zstd_v04.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-#  include "zstd_v05.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-#  include "zstd_v06.h"
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-#  include "zstd_v07.h"
-#endif
-
-/** ZSTD_isLegacy() :
-    @return : > 0 if supported by legacy decoder. 0 otherwise.
-              return value is the version.
-*/
-MEM_STATIC unsigned ZSTD_isLegacy(const void* src, size_t srcSize)
-{
-    U32 magicNumberLE;
-    if (srcSize<4) return 0;
-    magicNumberLE = MEM_readLE32(src);
-    switch(magicNumberLE)
-    {
-#if (ZSTD_LEGACY_SUPPORT <= 1)
-        case ZSTDv01_magicNumberLE:return 1;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 2)
-        case ZSTDv02_magicNumber : return 2;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 3)
-        case ZSTDv03_magicNumber : return 3;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case ZSTDv04_magicNumber : return 4;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case ZSTDv05_MAGICNUMBER : return 5;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case ZSTDv06_MAGICNUMBER : return 6;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case ZSTDv07_MAGICNUMBER : return 7;
-#endif
-        default : return 0;
-    }
-}
-
-
-MEM_STATIC unsigned long long ZSTD_getDecompressedSize_legacy(const void* src, size_t srcSize)
-{
-    U32 const version = ZSTD_isLegacy(src, srcSize);
-    if (version < 5) return 0;  /* no decompressed size in frame header, or not a legacy format */
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-    if (version==5) {
-        ZSTDv05_parameters fParams;
-        size_t const frResult = ZSTDv05_getFrameParams(&fParams, src, srcSize);
-        if (frResult != 0) return 0;
-        return fParams.srcSize;
-    }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-    if (version==6) {
-        ZSTDv06_frameParams fParams;
-        size_t const frResult = ZSTDv06_getFrameParams(&fParams, src, srcSize);
-        if (frResult != 0) return 0;
-        return fParams.frameContentSize;
-    }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-    if (version==7) {
-        ZSTDv07_frameParams fParams;
-        size_t const frResult = ZSTDv07_getFrameParams(&fParams, src, srcSize);
-        if (frResult != 0) return 0;
-        return fParams.frameContentSize;
-    }
-#endif
-    return 0;   /* should not be possible */
-}
-
-
-MEM_STATIC size_t ZSTD_decompressLegacy(
-                     void* dst, size_t dstCapacity,
-               const void* src, size_t compressedSize,
-               const void* dict,size_t dictSize)
-{
-    U32 const version = ZSTD_isLegacy(src, compressedSize);
-    (void)dst; (void)dstCapacity; (void)dict; (void)dictSize;  /* unused when ZSTD_LEGACY_SUPPORT >= 8 */
-    switch(version)
-    {
-#if (ZSTD_LEGACY_SUPPORT <= 1)
-        case 1 :
-            return ZSTDv01_decompress(dst, dstCapacity, src, compressedSize);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 2)
-        case 2 :
-            return ZSTDv02_decompress(dst, dstCapacity, src, compressedSize);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 3)
-        case 3 :
-            return ZSTDv03_decompress(dst, dstCapacity, src, compressedSize);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case 4 :
-            return ZSTDv04_decompress(dst, dstCapacity, src, compressedSize);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case 5 :
-            {   size_t result;
-                ZSTDv05_DCtx* const zd = ZSTDv05_createDCtx();
-                if (zd==NULL) return ERROR(memory_allocation);
-                result = ZSTDv05_decompress_usingDict(zd, dst, dstCapacity, src, compressedSize, dict, dictSize);
-                ZSTDv05_freeDCtx(zd);
-                return result;
-            }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case 6 :
-            {   size_t result;
-                ZSTDv06_DCtx* const zd = ZSTDv06_createDCtx();
-                if (zd==NULL) return ERROR(memory_allocation);
-                result = ZSTDv06_decompress_usingDict(zd, dst, dstCapacity, src, compressedSize, dict, dictSize);
-                ZSTDv06_freeDCtx(zd);
-                return result;
-            }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case 7 :
-            {   size_t result;
-                ZSTDv07_DCtx* const zd = ZSTDv07_createDCtx();
-                if (zd==NULL) return ERROR(memory_allocation);
-                result = ZSTDv07_decompress_usingDict(zd, dst, dstCapacity, src, compressedSize, dict, dictSize);
-                ZSTDv07_freeDCtx(zd);
-                return result;
-            }
-#endif
-        default :
-            return ERROR(prefix_unknown);
-    }
-}
-
-MEM_STATIC ZSTD_frameSizeInfo ZSTD_findFrameSizeInfoLegacy(const void *src, size_t srcSize)
-{
-    ZSTD_frameSizeInfo frameSizeInfo;
-    U32 const version = ZSTD_isLegacy(src, srcSize);
-    switch(version)
-    {
-#if (ZSTD_LEGACY_SUPPORT <= 1)
-        case 1 :
-            ZSTDv01_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 2)
-        case 2 :
-            ZSTDv02_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 3)
-        case 3 :
-            ZSTDv03_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case 4 :
-            ZSTDv04_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case 5 :
-            ZSTDv05_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case 6 :
-            ZSTDv06_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case 7 :
-            ZSTDv07_findFrameSizeInfoLegacy(src, srcSize,
-                &frameSizeInfo.compressedSize,
-                &frameSizeInfo.decompressedBound);
-            break;
-#endif
-        default :
-            frameSizeInfo.compressedSize = ERROR(prefix_unknown);
-            frameSizeInfo.decompressedBound = ZSTD_CONTENTSIZE_ERROR;
-            break;
-    }
-    if (!ZSTD_isError(frameSizeInfo.compressedSize) && frameSizeInfo.compressedSize > srcSize) {
-        frameSizeInfo.compressedSize = ERROR(srcSize_wrong);
-        frameSizeInfo.decompressedBound = ZSTD_CONTENTSIZE_ERROR;
-    }
-    return frameSizeInfo;
-}
-
-MEM_STATIC size_t ZSTD_findFrameCompressedSizeLegacy(const void *src, size_t srcSize)
-{
-    ZSTD_frameSizeInfo frameSizeInfo = ZSTD_findFrameSizeInfoLegacy(src, srcSize);
-    return frameSizeInfo.compressedSize;
-}
-
-MEM_STATIC size_t ZSTD_freeLegacyStreamContext(void* legacyContext, U32 version)
-{
-    switch(version)
-    {
-        default :
-        case 1 :
-        case 2 :
-        case 3 :
-            (void)legacyContext;
-            return ERROR(version_unsupported);
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case 4 : return ZBUFFv04_freeDCtx((ZBUFFv04_DCtx*)legacyContext);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case 5 : return ZBUFFv05_freeDCtx((ZBUFFv05_DCtx*)legacyContext);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case 6 : return ZBUFFv06_freeDCtx((ZBUFFv06_DCtx*)legacyContext);
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case 7 : return ZBUFFv07_freeDCtx((ZBUFFv07_DCtx*)legacyContext);
-#endif
-    }
-}
-
-
-MEM_STATIC size_t ZSTD_initLegacyStream(void** legacyContext, U32 prevVersion, U32 newVersion,
-                                        const void* dict, size_t dictSize)
-{
-    DEBUGLOG(5, "ZSTD_initLegacyStream for v0.%u", newVersion);
-    if (prevVersion != newVersion) ZSTD_freeLegacyStreamContext(*legacyContext, prevVersion);
-    switch(newVersion)
-    {
-        default :
-        case 1 :
-        case 2 :
-        case 3 :
-            (void)dict; (void)dictSize;
-            return 0;
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case 4 :
-        {
-            ZBUFFv04_DCtx* dctx = (prevVersion != newVersion) ? ZBUFFv04_createDCtx() : (ZBUFFv04_DCtx*)*legacyContext;
-            if (dctx==NULL) return ERROR(memory_allocation);
-            ZBUFFv04_decompressInit(dctx);
-            ZBUFFv04_decompressWithDictionary(dctx, dict, dictSize);
-            *legacyContext = dctx;
-            return 0;
-        }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case 5 :
-        {
-            ZBUFFv05_DCtx* dctx = (prevVersion != newVersion) ? ZBUFFv05_createDCtx() : (ZBUFFv05_DCtx*)*legacyContext;
-            if (dctx==NULL) return ERROR(memory_allocation);
-            ZBUFFv05_decompressInitDictionary(dctx, dict, dictSize);
-            *legacyContext = dctx;
-            return 0;
-        }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case 6 :
-        {
-            ZBUFFv06_DCtx* dctx = (prevVersion != newVersion) ? ZBUFFv06_createDCtx() : (ZBUFFv06_DCtx*)*legacyContext;
-            if (dctx==NULL) return ERROR(memory_allocation);
-            ZBUFFv06_decompressInitDictionary(dctx, dict, dictSize);
-            *legacyContext = dctx;
-            return 0;
-        }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case 7 :
-        {
-            ZBUFFv07_DCtx* dctx = (prevVersion != newVersion) ? ZBUFFv07_createDCtx() : (ZBUFFv07_DCtx*)*legacyContext;
-            if (dctx==NULL) return ERROR(memory_allocation);
-            ZBUFFv07_decompressInitDictionary(dctx, dict, dictSize);
-            *legacyContext = dctx;
-            return 0;
-        }
-#endif
-    }
-}
-
-
-
-MEM_STATIC size_t ZSTD_decompressLegacyStream(void* legacyContext, U32 version,
-                                              ZSTD_outBuffer* output, ZSTD_inBuffer* input)
-{
-    DEBUGLOG(5, "ZSTD_decompressLegacyStream for v0.%u", version);
-    switch(version)
-    {
-        default :
-        case 1 :
-        case 2 :
-        case 3 :
-            (void)legacyContext; (void)output; (void)input;
-            return ERROR(version_unsupported);
-#if (ZSTD_LEGACY_SUPPORT <= 4)
-        case 4 :
-            {
-                ZBUFFv04_DCtx* dctx = (ZBUFFv04_DCtx*) legacyContext;
-                const void* src = (const char*)input->src + input->pos;
-                size_t readSize = input->size - input->pos;
-                void* dst = (char*)output->dst + output->pos;
-                size_t decodedSize = output->size - output->pos;
-                size_t const hintSize = ZBUFFv04_decompressContinue(dctx, dst, &decodedSize, src, &readSize);
-                output->pos += decodedSize;
-                input->pos += readSize;
-                return hintSize;
-            }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 5)
-        case 5 :
-            {
-                ZBUFFv05_DCtx* dctx = (ZBUFFv05_DCtx*) legacyContext;
-                const void* src = (const char*)input->src + input->pos;
-                size_t readSize = input->size - input->pos;
-                void* dst = (char*)output->dst + output->pos;
-                size_t decodedSize = output->size - output->pos;
-                size_t const hintSize = ZBUFFv05_decompressContinue(dctx, dst, &decodedSize, src, &readSize);
-                output->pos += decodedSize;
-                input->pos += readSize;
-                return hintSize;
-            }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 6)
-        case 6 :
-            {
-                ZBUFFv06_DCtx* dctx = (ZBUFFv06_DCtx*) legacyContext;
-                const void* src = (const char*)input->src + input->pos;
-                size_t readSize = input->size - input->pos;
-                void* dst = (char*)output->dst + output->pos;
-                size_t decodedSize = output->size - output->pos;
-                size_t const hintSize = ZBUFFv06_decompressContinue(dctx, dst, &decodedSize, src, &readSize);
-                output->pos += decodedSize;
-                input->pos += readSize;
-                return hintSize;
-            }
-#endif
-#if (ZSTD_LEGACY_SUPPORT <= 7)
-        case 7 :
-            {
-                ZBUFFv07_DCtx* dctx = (ZBUFFv07_DCtx*) legacyContext;
-                const void* src = (const char*)input->src + input->pos;
-                size_t readSize = input->size - input->pos;
-                void* dst = (char*)output->dst + output->pos;
-                size_t decodedSize = output->size - output->pos;
-                size_t const hintSize = ZBUFFv07_decompressContinue(dctx, dst, &decodedSize, src, &readSize);
-                output->pos += decodedSize;
-                input->pos += readSize;
-                return hintSize;
-            }
-#endif
-    }
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* ZSTD_LEGACY_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_opt.c b/vendor/github.com/DataDog/zstd/zstd_opt.c
deleted file mode 100644
index e32e542..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_opt.c
+++ /dev/null
@@ -1,1246 +0,0 @@
-/*
- * Copyright (c) 2016-present, Przemyslaw Skibinski, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#include "zstd_compress_internal.h"
-#include "hist.h"
-#include "zstd_opt.h"
-
-
-#define ZSTD_LITFREQ_ADD    2   /* scaling factor for litFreq, so that frequencies adapt faster to new stats */
-#define ZSTD_FREQ_DIV       4   /* log factor when using previous stats to init next stats */
-#define ZSTD_MAX_PRICE     (1<<30)
-
-#define ZSTD_PREDEF_THRESHOLD 1024   /* if srcSize < ZSTD_PREDEF_THRESHOLD, symbols' cost is assumed static, directly determined by pre-defined distributions */
-
-
-/*-*************************************
-*  Price functions for optimal parser
-***************************************/
-
-#if 0    /* approximation at bit level */
-#  define BITCOST_ACCURACY 0
-#  define BITCOST_MULTIPLIER (1 << BITCOST_ACCURACY)
-#  define WEIGHT(stat)  ((void)opt, ZSTD_bitWeight(stat))
-#elif 0  /* fractional bit accuracy */
-#  define BITCOST_ACCURACY 8
-#  define BITCOST_MULTIPLIER (1 << BITCOST_ACCURACY)
-#  define WEIGHT(stat,opt) ((void)opt, ZSTD_fracWeight(stat))
-#else    /* opt==approx, ultra==accurate */
-#  define BITCOST_ACCURACY 8
-#  define BITCOST_MULTIPLIER (1 << BITCOST_ACCURACY)
-#  define WEIGHT(stat,opt) (opt ? ZSTD_fracWeight(stat) : ZSTD_bitWeight(stat))
-#endif
-
-MEM_STATIC U32 ZSTD_bitWeight(U32 stat)
-{
-    return (ZSTD_highbit32(stat+1) * BITCOST_MULTIPLIER);
-}
-
-MEM_STATIC U32 ZSTD_fracWeight(U32 rawStat)
-{
-    U32 const stat = rawStat + 1;
-    U32 const hb = ZSTD_highbit32(stat);
-    U32 const BWeight = hb * BITCOST_MULTIPLIER;
-    U32 const FWeight = (stat << BITCOST_ACCURACY) >> hb;
-    U32 const weight = BWeight + FWeight;
-    assert(hb + BITCOST_ACCURACY < 31);
-    return weight;
-}
-
-#if (DEBUGLEVEL>=2)
-/* debugging function,
- * @return price in bytes as fractional value
- * for debug messages only */
-MEM_STATIC double ZSTD_fCost(U32 price)
-{
-    return (double)price / (BITCOST_MULTIPLIER*8);
-}
-#endif
-
-static int ZSTD_compressedLiterals(optState_t const* const optPtr)
-{
-    return optPtr->literalCompressionMode != ZSTD_lcm_uncompressed;
-}
-
-static void ZSTD_setBasePrices(optState_t* optPtr, int optLevel)
-{
-    if (ZSTD_compressedLiterals(optPtr))
-        optPtr->litSumBasePrice = WEIGHT(optPtr->litSum, optLevel);
-    optPtr->litLengthSumBasePrice = WEIGHT(optPtr->litLengthSum, optLevel);
-    optPtr->matchLengthSumBasePrice = WEIGHT(optPtr->matchLengthSum, optLevel);
-    optPtr->offCodeSumBasePrice = WEIGHT(optPtr->offCodeSum, optLevel);
-}
-
-
-/* ZSTD_downscaleStat() :
- * reduce all elements in table by a factor 2^(ZSTD_FREQ_DIV+malus)
- * return the resulting sum of elements */
-static U32 ZSTD_downscaleStat(unsigned* table, U32 lastEltIndex, int malus)
-{
-    U32 s, sum=0;
-    DEBUGLOG(5, "ZSTD_downscaleStat (nbElts=%u)", (unsigned)lastEltIndex+1);
-    assert(ZSTD_FREQ_DIV+malus > 0 && ZSTD_FREQ_DIV+malus < 31);
-    for (s=0; s<lastEltIndex+1; s++) {
-        table[s] = 1 + (table[s] >> (ZSTD_FREQ_DIV+malus));
-        sum += table[s];
-    }
-    return sum;
-}
-
-/* ZSTD_rescaleFreqs() :
- * if first block (detected by optPtr->litLengthSum == 0) : init statistics
- *    take hints from dictionary if there is one
- *    or init from zero, using src for literals stats, or flat 1 for match symbols
- * otherwise downscale existing stats, to be used as seed for next block.
- */
-static void
-ZSTD_rescaleFreqs(optState_t* const optPtr,
-            const BYTE* const src, size_t const srcSize,
-                  int const optLevel)
-{
-    int const compressedLiterals = ZSTD_compressedLiterals(optPtr);
-    DEBUGLOG(5, "ZSTD_rescaleFreqs (srcSize=%u)", (unsigned)srcSize);
-    optPtr->priceType = zop_dynamic;
-
-    if (optPtr->litLengthSum == 0) {  /* first block : init */
-        if (srcSize <= ZSTD_PREDEF_THRESHOLD) {  /* heuristic */
-            DEBUGLOG(5, "(srcSize <= ZSTD_PREDEF_THRESHOLD) => zop_predef");
-            optPtr->priceType = zop_predef;
-        }
-
-        assert(optPtr->symbolCosts != NULL);
-        if (optPtr->symbolCosts->huf.repeatMode == HUF_repeat_valid) {
-            /* huffman table presumed generated by dictionary */
-            optPtr->priceType = zop_dynamic;
-
-            if (compressedLiterals) {
-                unsigned lit;
-                assert(optPtr->litFreq != NULL);
-                optPtr->litSum = 0;
-                for (lit=0; lit<=MaxLit; lit++) {
-                    U32 const scaleLog = 11;   /* scale to 2K */
-                    U32 const bitCost = HUF_getNbBits(optPtr->symbolCosts->huf.CTable, lit);
-                    assert(bitCost <= scaleLog);
-                    optPtr->litFreq[lit] = bitCost ? 1 << (scaleLog-bitCost) : 1 /*minimum to calculate cost*/;
-                    optPtr->litSum += optPtr->litFreq[lit];
-            }   }
-
-            {   unsigned ll;
-                FSE_CState_t llstate;
-                FSE_initCState(&llstate, optPtr->symbolCosts->fse.litlengthCTable);
-                optPtr->litLengthSum = 0;
-                for (ll=0; ll<=MaxLL; ll++) {
-                    U32 const scaleLog = 10;   /* scale to 1K */
-                    U32 const bitCost = FSE_getMaxNbBits(llstate.symbolTT, ll);
-                    assert(bitCost < scaleLog);
-                    optPtr->litLengthFreq[ll] = bitCost ? 1 << (scaleLog-bitCost) : 1 /*minimum to calculate cost*/;
-                    optPtr->litLengthSum += optPtr->litLengthFreq[ll];
-            }   }
-
-            {   unsigned ml;
-                FSE_CState_t mlstate;
-                FSE_initCState(&mlstate, optPtr->symbolCosts->fse.matchlengthCTable);
-                optPtr->matchLengthSum = 0;
-                for (ml=0; ml<=MaxML; ml++) {
-                    U32 const scaleLog = 10;
-                    U32 const bitCost = FSE_getMaxNbBits(mlstate.symbolTT, ml);
-                    assert(bitCost < scaleLog);
-                    optPtr->matchLengthFreq[ml] = bitCost ? 1 << (scaleLog-bitCost) : 1 /*minimum to calculate cost*/;
-                    optPtr->matchLengthSum += optPtr->matchLengthFreq[ml];
-            }   }
-
-            {   unsigned of;
-                FSE_CState_t ofstate;
-                FSE_initCState(&ofstate, optPtr->symbolCosts->fse.offcodeCTable);
-                optPtr->offCodeSum = 0;
-                for (of=0; of<=MaxOff; of++) {
-                    U32 const scaleLog = 10;
-                    U32 const bitCost = FSE_getMaxNbBits(ofstate.symbolTT, of);
-                    assert(bitCost < scaleLog);
-                    optPtr->offCodeFreq[of] = bitCost ? 1 << (scaleLog-bitCost) : 1 /*minimum to calculate cost*/;
-                    optPtr->offCodeSum += optPtr->offCodeFreq[of];
-            }   }
-
-        } else {  /* not a dictionary */
-
-            assert(optPtr->litFreq != NULL);
-            if (compressedLiterals) {
-                unsigned lit = MaxLit;
-                HIST_count_simple(optPtr->litFreq, &lit, src, srcSize);   /* use raw first block to init statistics */
-                optPtr->litSum = ZSTD_downscaleStat(optPtr->litFreq, MaxLit, 1);
-            }
-
-            {   unsigned ll;
-                for (ll=0; ll<=MaxLL; ll++)
-                    optPtr->litLengthFreq[ll] = 1;
-            }
-            optPtr->litLengthSum = MaxLL+1;
-
-            {   unsigned ml;
-                for (ml=0; ml<=MaxML; ml++)
-                    optPtr->matchLengthFreq[ml] = 1;
-            }
-            optPtr->matchLengthSum = MaxML+1;
-
-            {   unsigned of;
-                for (of=0; of<=MaxOff; of++)
-                    optPtr->offCodeFreq[of] = 1;
-            }
-            optPtr->offCodeSum = MaxOff+1;
-
-        }
-
-    } else {   /* new block : re-use previous statistics, scaled down */
-
-        if (compressedLiterals)
-            optPtr->litSum = ZSTD_downscaleStat(optPtr->litFreq, MaxLit, 1);
-        optPtr->litLengthSum = ZSTD_downscaleStat(optPtr->litLengthFreq, MaxLL, 0);
-        optPtr->matchLengthSum = ZSTD_downscaleStat(optPtr->matchLengthFreq, MaxML, 0);
-        optPtr->offCodeSum = ZSTD_downscaleStat(optPtr->offCodeFreq, MaxOff, 0);
-    }
-
-    ZSTD_setBasePrices(optPtr, optLevel);
-}
-
-/* ZSTD_rawLiteralsCost() :
- * price of literals (only) in specified segment (which length can be 0).
- * does not include price of literalLength symbol */
-static U32 ZSTD_rawLiteralsCost(const BYTE* const literals, U32 const litLength,
-                                const optState_t* const optPtr,
-                                int optLevel)
-{
-    if (litLength == 0) return 0;
-
-    if (!ZSTD_compressedLiterals(optPtr))
-        return (litLength << 3) * BITCOST_MULTIPLIER;  /* Uncompressed - 8 bytes per literal. */
-
-    if (optPtr->priceType == zop_predef)
-        return (litLength*6) * BITCOST_MULTIPLIER;  /* 6 bit per literal - no statistic used */
-
-    /* dynamic statistics */
-    {   U32 price = litLength * optPtr->litSumBasePrice;
-        U32 u;
-        for (u=0; u < litLength; u++) {
-            assert(WEIGHT(optPtr->litFreq[literals[u]], optLevel) <= optPtr->litSumBasePrice);   /* literal cost should never be negative */
-            price -= WEIGHT(optPtr->litFreq[literals[u]], optLevel);
-        }
-        return price;
-    }
-}
-
-/* ZSTD_litLengthPrice() :
- * cost of literalLength symbol */
-static U32 ZSTD_litLengthPrice(U32 const litLength, const optState_t* const optPtr, int optLevel)
-{
-    if (optPtr->priceType == zop_predef) return WEIGHT(litLength, optLevel);
-
-    /* dynamic statistics */
-    {   U32 const llCode = ZSTD_LLcode(litLength);
-        return (LL_bits[llCode] * BITCOST_MULTIPLIER)
-             + optPtr->litLengthSumBasePrice
-             - WEIGHT(optPtr->litLengthFreq[llCode], optLevel);
-    }
-}
-
-/* ZSTD_litLengthContribution() :
- * @return ( cost(litlength) - cost(0) )
- * this value can then be added to rawLiteralsCost()
- * to provide a cost which is directly comparable to a match ending at same position */
-static int ZSTD_litLengthContribution(U32 const litLength, const optState_t* const optPtr, int optLevel)
-{
-    if (optPtr->priceType >= zop_predef) return (int)WEIGHT(litLength, optLevel);
-
-    /* dynamic statistics */
-    {   U32 const llCode = ZSTD_LLcode(litLength);
-        int const contribution = (int)(LL_bits[llCode] * BITCOST_MULTIPLIER)
-                               + (int)WEIGHT(optPtr->litLengthFreq[0], optLevel)   /* note: log2litLengthSum cancel out */
-                               - (int)WEIGHT(optPtr->litLengthFreq[llCode], optLevel);
-#if 1
-        return contribution;
-#else
-        return MAX(0, contribution); /* sometimes better, sometimes not ... */
-#endif
-    }
-}
-
-/* ZSTD_literalsContribution() :
- * creates a fake cost for the literals part of a sequence
- * which can be compared to the ending cost of a match
- * should a new match start at this position */
-static int ZSTD_literalsContribution(const BYTE* const literals, U32 const litLength,
-                                     const optState_t* const optPtr,
-                                     int optLevel)
-{
-    int const contribution = (int)ZSTD_rawLiteralsCost(literals, litLength, optPtr, optLevel)
-                           + ZSTD_litLengthContribution(litLength, optPtr, optLevel);
-    return contribution;
-}
-
-/* ZSTD_getMatchPrice() :
- * Provides the cost of the match part (offset + matchLength) of a sequence
- * Must be combined with ZSTD_fullLiteralsCost() to get the full cost of a sequence.
- * optLevel: when <2, favors small offset for decompression speed (improved cache efficiency) */
-FORCE_INLINE_TEMPLATE U32
-ZSTD_getMatchPrice(U32 const offset,
-                   U32 const matchLength,
-             const optState_t* const optPtr,
-                   int const optLevel)
-{
-    U32 price;
-    U32 const offCode = ZSTD_highbit32(offset+1);
-    U32 const mlBase = matchLength - MINMATCH;
-    assert(matchLength >= MINMATCH);
-
-    if (optPtr->priceType == zop_predef)  /* fixed scheme, do not use statistics */
-        return WEIGHT(mlBase, optLevel) + ((16 + offCode) * BITCOST_MULTIPLIER);
-
-    /* dynamic statistics */
-    price = (offCode * BITCOST_MULTIPLIER) + (optPtr->offCodeSumBasePrice - WEIGHT(optPtr->offCodeFreq[offCode], optLevel));
-    if ((optLevel<2) /*static*/ && offCode >= 20)
-        price += (offCode-19)*2 * BITCOST_MULTIPLIER; /* handicap for long distance offsets, favor decompression speed */
-
-    /* match Length */
-    {   U32 const mlCode = ZSTD_MLcode(mlBase);
-        price += (ML_bits[mlCode] * BITCOST_MULTIPLIER) + (optPtr->matchLengthSumBasePrice - WEIGHT(optPtr->matchLengthFreq[mlCode], optLevel));
-    }
-
-    price += BITCOST_MULTIPLIER / 5;   /* heuristic : make matches a bit more costly to favor less sequences -> faster decompression speed */
-
-    DEBUGLOG(8, "ZSTD_getMatchPrice(ml:%u) = %u", matchLength, price);
-    return price;
-}
-
-/* ZSTD_updateStats() :
- * assumption : literals + litLengtn <= iend */
-static void ZSTD_updateStats(optState_t* const optPtr,
-                             U32 litLength, const BYTE* literals,
-                             U32 offsetCode, U32 matchLength)
-{
-    /* literals */
-    if (ZSTD_compressedLiterals(optPtr)) {
-        U32 u;
-        for (u=0; u < litLength; u++)
-            optPtr->litFreq[literals[u]] += ZSTD_LITFREQ_ADD;
-        optPtr->litSum += litLength*ZSTD_LITFREQ_ADD;
-    }
-
-    /* literal Length */
-    {   U32 const llCode = ZSTD_LLcode(litLength);
-        optPtr->litLengthFreq[llCode]++;
-        optPtr->litLengthSum++;
-    }
-
-    /* match offset code (0-2=>repCode; 3+=>offset+2) */
-    {   U32 const offCode = ZSTD_highbit32(offsetCode+1);
-        assert(offCode <= MaxOff);
-        optPtr->offCodeFreq[offCode]++;
-        optPtr->offCodeSum++;
-    }
-
-    /* match Length */
-    {   U32 const mlBase = matchLength - MINMATCH;
-        U32 const mlCode = ZSTD_MLcode(mlBase);
-        optPtr->matchLengthFreq[mlCode]++;
-        optPtr->matchLengthSum++;
-    }
-}
-
-
-/* ZSTD_readMINMATCH() :
- * function safe only for comparisons
- * assumption : memPtr must be at least 4 bytes before end of buffer */
-MEM_STATIC U32 ZSTD_readMINMATCH(const void* memPtr, U32 length)
-{
-    switch (length)
-    {
-    default :
-    case 4 : return MEM_read32(memPtr);
-    case 3 : if (MEM_isLittleEndian())
-                return MEM_read32(memPtr)<<8;
-             else
-                return MEM_read32(memPtr)>>8;
-    }
-}
-
-
-/* Update hashTable3 up to ip (excluded)
-   Assumption : always within prefix (i.e. not within extDict) */
-static U32 ZSTD_insertAndFindFirstIndexHash3 (ZSTD_matchState_t* ms,
-                                              U32* nextToUpdate3,
-                                              const BYTE* const ip)
-{
-    U32* const hashTable3 = ms->hashTable3;
-    U32 const hashLog3 = ms->hashLog3;
-    const BYTE* const base = ms->window.base;
-    U32 idx = *nextToUpdate3;
-    U32 const target = (U32)(ip - base);
-    size_t const hash3 = ZSTD_hash3Ptr(ip, hashLog3);
-    assert(hashLog3 > 0);
-
-    while(idx < target) {
-        hashTable3[ZSTD_hash3Ptr(base+idx, hashLog3)] = idx;
-        idx++;
-    }
-
-    *nextToUpdate3 = target;
-    return hashTable3[hash3];
-}
-
-
-/*-*************************************
-*  Binary Tree search
-***************************************/
-/** ZSTD_insertBt1() : add one or multiple positions to tree.
- *  ip : assumed <= iend-8 .
- * @return : nb of positions added */
-static U32 ZSTD_insertBt1(
-                ZSTD_matchState_t* ms,
-                const BYTE* const ip, const BYTE* const iend,
-                U32 const mls, const int extDict)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32*   const hashTable = ms->hashTable;
-    U32    const hashLog = cParams->hashLog;
-    size_t const h  = ZSTD_hashPtr(ip, hashLog, mls);
-    U32*   const bt = ms->chainTable;
-    U32    const btLog  = cParams->chainLog - 1;
-    U32    const btMask = (1 << btLog) - 1;
-    U32 matchIndex = hashTable[h];
-    size_t commonLengthSmaller=0, commonLengthLarger=0;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const dictBase = ms->window.dictBase;
-    const U32 dictLimit = ms->window.dictLimit;
-    const BYTE* const dictEnd = dictBase + dictLimit;
-    const BYTE* const prefixStart = base + dictLimit;
-    const BYTE* match;
-    const U32 current = (U32)(ip-base);
-    const U32 btLow = btMask >= current ? 0 : current - btMask;
-    U32* smallerPtr = bt + 2*(current&btMask);
-    U32* largerPtr  = smallerPtr + 1;
-    U32 dummy32;   /* to be nullified at the end */
-    U32 const windowLow = ms->window.lowLimit;
-    U32 matchEndIdx = current+8+1;
-    size_t bestLength = 8;
-    U32 nbCompares = 1U << cParams->searchLog;
-#ifdef ZSTD_C_PREDICT
-    U32 predictedSmall = *(bt + 2*((current-1)&btMask) + 0);
-    U32 predictedLarge = *(bt + 2*((current-1)&btMask) + 1);
-    predictedSmall += (predictedSmall>0);
-    predictedLarge += (predictedLarge>0);
-#endif /* ZSTD_C_PREDICT */
-
-    DEBUGLOG(8, "ZSTD_insertBt1 (%u)", current);
-
-    assert(ip <= iend-8);   /* required for h calculation */
-    hashTable[h] = current;   /* Update Hash Table */
-
-    assert(windowLow > 0);
-    while (nbCompares-- && (matchIndex >= windowLow)) {
-        U32* const nextPtr = bt + 2*(matchIndex & btMask);
-        size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-        assert(matchIndex < current);
-
-#ifdef ZSTD_C_PREDICT   /* note : can create issues when hlog small <= 11 */
-        const U32* predictPtr = bt + 2*((matchIndex-1) & btMask);   /* written this way, as bt is a roll buffer */
-        if (matchIndex == predictedSmall) {
-            /* no need to check length, result known */
-            *smallerPtr = matchIndex;
-            if (matchIndex <= btLow) { smallerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-            smallerPtr = nextPtr+1;               /* new "smaller" => larger of match */
-            matchIndex = nextPtr[1];              /* new matchIndex larger than previous (closer to current) */
-            predictedSmall = predictPtr[1] + (predictPtr[1]>0);
-            continue;
-        }
-        if (matchIndex == predictedLarge) {
-            *largerPtr = matchIndex;
-            if (matchIndex <= btLow) { largerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-            largerPtr = nextPtr;
-            matchIndex = nextPtr[0];
-            predictedLarge = predictPtr[0] + (predictPtr[0]>0);
-            continue;
-        }
-#endif
-
-        if (!extDict || (matchIndex+matchLength >= dictLimit)) {
-            assert(matchIndex+matchLength >= dictLimit);   /* might be wrong if actually extDict */
-            match = base + matchIndex;
-            matchLength += ZSTD_count(ip+matchLength, match+matchLength, iend);
-        } else {
-            match = dictBase + matchIndex;
-            matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iend, dictEnd, prefixStart);
-            if (matchIndex+matchLength >= dictLimit)
-                match = base + matchIndex;   /* to prepare for next usage of match[matchLength] */
-        }
-
-        if (matchLength > bestLength) {
-            bestLength = matchLength;
-            if (matchLength > matchEndIdx - matchIndex)
-                matchEndIdx = matchIndex + (U32)matchLength;
-        }
-
-        if (ip+matchLength == iend) {   /* equal : no way to know if inf or sup */
-            break;   /* drop , to guarantee consistency ; miss a bit of compression, but other solutions can corrupt tree */
-        }
-
-        if (match[matchLength] < ip[matchLength]) {  /* necessarily within buffer */
-            /* match is smaller than current */
-            *smallerPtr = matchIndex;             /* update smaller idx */
-            commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-            if (matchIndex <= btLow) { smallerPtr=&dummy32; break; }   /* beyond tree size, stop searching */
-            smallerPtr = nextPtr+1;               /* new "candidate" => larger than match, which was smaller than target */
-            matchIndex = nextPtr[1];              /* new matchIndex, larger than previous and closer to current */
-        } else {
-            /* match is larger than current */
-            *largerPtr = matchIndex;
-            commonLengthLarger = matchLength;
-            if (matchIndex <= btLow) { largerPtr=&dummy32; break; }   /* beyond tree size, stop searching */
-            largerPtr = nextPtr;
-            matchIndex = nextPtr[0];
-    }   }
-
-    *smallerPtr = *largerPtr = 0;
-    {   U32 positions = 0;
-        if (bestLength > 384) positions = MIN(192, (U32)(bestLength - 384));   /* speed optimization */
-        assert(matchEndIdx > current + 8);
-        return MAX(positions, matchEndIdx - (current + 8));
-    }
-}
-
-FORCE_INLINE_TEMPLATE
-void ZSTD_updateTree_internal(
-                ZSTD_matchState_t* ms,
-                const BYTE* const ip, const BYTE* const iend,
-                const U32 mls, const ZSTD_dictMode_e dictMode)
-{
-    const BYTE* const base = ms->window.base;
-    U32 const target = (U32)(ip - base);
-    U32 idx = ms->nextToUpdate;
-    DEBUGLOG(6, "ZSTD_updateTree_internal, from %u to %u  (dictMode:%u)",
-                idx, target, dictMode);
-
-    while(idx < target) {
-        U32 const forward = ZSTD_insertBt1(ms, base+idx, iend, mls, dictMode == ZSTD_extDict);
-        assert(idx < (U32)(idx + forward));
-        idx += forward;
-    }
-    assert((size_t)(ip - base) <= (size_t)(U32)(-1));
-    assert((size_t)(iend - base) <= (size_t)(U32)(-1));
-    ms->nextToUpdate = target;
-}
-
-void ZSTD_updateTree(ZSTD_matchState_t* ms, const BYTE* ip, const BYTE* iend) {
-    ZSTD_updateTree_internal(ms, ip, iend, ms->cParams.minMatch, ZSTD_noDict);
-}
-
-FORCE_INLINE_TEMPLATE
-U32 ZSTD_insertBtAndGetAllMatches (
-                    ZSTD_match_t* matches,   /* store result (found matches) in this table (presumed large enough) */
-                    ZSTD_matchState_t* ms,
-                    U32* nextToUpdate3,
-                    const BYTE* const ip, const BYTE* const iLimit, const ZSTD_dictMode_e dictMode,
-                    const U32 rep[ZSTD_REP_NUM],
-                    U32 const ll0,   /* tells if associated literal length is 0 or not. This value must be 0 or 1 */
-                    const U32 lengthToBeat,
-                    U32 const mls /* template */)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32 const sufficient_len = MIN(cParams->targetLength, ZSTD_OPT_NUM -1);
-    U32 const maxDistance = 1U << cParams->windowLog;
-    const BYTE* const base = ms->window.base;
-    U32 const current = (U32)(ip-base);
-    U32 const hashLog = cParams->hashLog;
-    U32 const minMatch = (mls==3) ? 3 : 4;
-    U32* const hashTable = ms->hashTable;
-    size_t const h  = ZSTD_hashPtr(ip, hashLog, mls);
-    U32 matchIndex  = hashTable[h];
-    U32* const bt   = ms->chainTable;
-    U32 const btLog = cParams->chainLog - 1;
-    U32 const btMask= (1U << btLog) - 1;
-    size_t commonLengthSmaller=0, commonLengthLarger=0;
-    const BYTE* const dictBase = ms->window.dictBase;
-    U32 const dictLimit = ms->window.dictLimit;
-    const BYTE* const dictEnd = dictBase + dictLimit;
-    const BYTE* const prefixStart = base + dictLimit;
-    U32 const btLow = (btMask >= current) ? 0 : current - btMask;
-    U32 const windowValid = ms->window.lowLimit;
-    U32 const windowLow = ((current - windowValid) > maxDistance) ? current - maxDistance : windowValid;
-    U32 const matchLow = windowLow ? windowLow : 1;
-    U32* smallerPtr = bt + 2*(current&btMask);
-    U32* largerPtr  = bt + 2*(current&btMask) + 1;
-    U32 matchEndIdx = current+8+1;   /* farthest referenced position of any match => detects repetitive patterns */
-    U32 dummy32;   /* to be nullified at the end */
-    U32 mnum = 0;
-    U32 nbCompares = 1U << cParams->searchLog;
-
-    const ZSTD_matchState_t* dms    = dictMode == ZSTD_dictMatchState ? ms->dictMatchState : NULL;
-    const ZSTD_compressionParameters* const dmsCParams =
-                                      dictMode == ZSTD_dictMatchState ? &dms->cParams : NULL;
-    const BYTE* const dmsBase       = dictMode == ZSTD_dictMatchState ? dms->window.base : NULL;
-    const BYTE* const dmsEnd        = dictMode == ZSTD_dictMatchState ? dms->window.nextSrc : NULL;
-    U32         const dmsHighLimit  = dictMode == ZSTD_dictMatchState ? (U32)(dmsEnd - dmsBase) : 0;
-    U32         const dmsLowLimit   = dictMode == ZSTD_dictMatchState ? dms->window.lowLimit : 0;
-    U32         const dmsIndexDelta = dictMode == ZSTD_dictMatchState ? windowLow - dmsHighLimit : 0;
-    U32         const dmsHashLog    = dictMode == ZSTD_dictMatchState ? dmsCParams->hashLog : hashLog;
-    U32         const dmsBtLog      = dictMode == ZSTD_dictMatchState ? dmsCParams->chainLog - 1 : btLog;
-    U32         const dmsBtMask     = dictMode == ZSTD_dictMatchState ? (1U << dmsBtLog) - 1 : 0;
-    U32         const dmsBtLow      = dictMode == ZSTD_dictMatchState && dmsBtMask < dmsHighLimit - dmsLowLimit ? dmsHighLimit - dmsBtMask : dmsLowLimit;
-
-    size_t bestLength = lengthToBeat-1;
-    DEBUGLOG(8, "ZSTD_insertBtAndGetAllMatches: current=%u", current);
-
-    /* check repCode */
-    assert(ll0 <= 1);   /* necessarily 1 or 0 */
-    {   U32 const lastR = ZSTD_REP_NUM + ll0;
-        U32 repCode;
-        for (repCode = ll0; repCode < lastR; repCode++) {
-            U32 const repOffset = (repCode==ZSTD_REP_NUM) ? (rep[0] - 1) : rep[repCode];
-            U32 const repIndex = current - repOffset;
-            U32 repLen = 0;
-            assert(current >= dictLimit);
-            if (repOffset-1 /* intentional overflow, discards 0 and -1 */ < current-dictLimit) {  /* equivalent to `current > repIndex >= dictLimit` */
-                if (ZSTD_readMINMATCH(ip, minMatch) == ZSTD_readMINMATCH(ip - repOffset, minMatch)) {
-                    repLen = (U32)ZSTD_count(ip+minMatch, ip+minMatch-repOffset, iLimit) + minMatch;
-                }
-            } else {  /* repIndex < dictLimit || repIndex >= current */
-                const BYTE* const repMatch = dictMode == ZSTD_dictMatchState ?
-                                             dmsBase + repIndex - dmsIndexDelta :
-                                             dictBase + repIndex;
-                assert(current >= windowLow);
-                if ( dictMode == ZSTD_extDict
-                  && ( ((repOffset-1) /*intentional overflow*/ < current - windowLow)  /* equivalent to `current > repIndex >= windowLow` */
-                     & (((U32)((dictLimit-1) - repIndex) >= 3) ) /* intentional overflow : do not test positions overlapping 2 memory segments */)
-                  && (ZSTD_readMINMATCH(ip, minMatch) == ZSTD_readMINMATCH(repMatch, minMatch)) ) {
-                    repLen = (U32)ZSTD_count_2segments(ip+minMatch, repMatch+minMatch, iLimit, dictEnd, prefixStart) + minMatch;
-                }
-                if (dictMode == ZSTD_dictMatchState
-                  && ( ((repOffset-1) /*intentional overflow*/ < current - (dmsLowLimit + dmsIndexDelta))  /* equivalent to `current > repIndex >= dmsLowLimit` */
-                     & ((U32)((dictLimit-1) - repIndex) >= 3) ) /* intentional overflow : do not test positions overlapping 2 memory segments */
-                  && (ZSTD_readMINMATCH(ip, minMatch) == ZSTD_readMINMATCH(repMatch, minMatch)) ) {
-                    repLen = (U32)ZSTD_count_2segments(ip+minMatch, repMatch+minMatch, iLimit, dmsEnd, prefixStart) + minMatch;
-            }   }
-            /* save longer solution */
-            if (repLen > bestLength) {
-                DEBUGLOG(8, "found repCode %u (ll0:%u, offset:%u) of length %u",
-                            repCode, ll0, repOffset, repLen);
-                bestLength = repLen;
-                matches[mnum].off = repCode - ll0;
-                matches[mnum].len = (U32)repLen;
-                mnum++;
-                if ( (repLen > sufficient_len)
-                   | (ip+repLen == iLimit) ) {  /* best possible */
-                    return mnum;
-    }   }   }   }
-
-    /* HC3 match finder */
-    if ((mls == 3) /*static*/ && (bestLength < mls)) {
-        U32 const matchIndex3 = ZSTD_insertAndFindFirstIndexHash3(ms, nextToUpdate3, ip);
-        if ((matchIndex3 >= matchLow)
-          & (current - matchIndex3 < (1<<18)) /*heuristic : longer distance likely too expensive*/ ) {
-            size_t mlen;
-            if ((dictMode == ZSTD_noDict) /*static*/ || (dictMode == ZSTD_dictMatchState) /*static*/ || (matchIndex3 >= dictLimit)) {
-                const BYTE* const match = base + matchIndex3;
-                mlen = ZSTD_count(ip, match, iLimit);
-            } else {
-                const BYTE* const match = dictBase + matchIndex3;
-                mlen = ZSTD_count_2segments(ip, match, iLimit, dictEnd, prefixStart);
-            }
-
-            /* save best solution */
-            if (mlen >= mls /* == 3 > bestLength */) {
-                DEBUGLOG(8, "found small match with hlog3, of length %u",
-                            (U32)mlen);
-                bestLength = mlen;
-                assert(current > matchIndex3);
-                assert(mnum==0);  /* no prior solution */
-                matches[0].off = (current - matchIndex3) + ZSTD_REP_MOVE;
-                matches[0].len = (U32)mlen;
-                mnum = 1;
-                if ( (mlen > sufficient_len) |
-                     (ip+mlen == iLimit) ) {  /* best possible length */
-                    ms->nextToUpdate = current+1;  /* skip insertion */
-                    return 1;
-        }   }   }
-        /* no dictMatchState lookup: dicts don't have a populated HC3 table */
-    }
-
-    hashTable[h] = current;   /* Update Hash Table */
-
-    while (nbCompares-- && (matchIndex >= matchLow)) {
-        U32* const nextPtr = bt + 2*(matchIndex & btMask);
-        size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-        const BYTE* match;
-        assert(current > matchIndex);
-
-        if ((dictMode == ZSTD_noDict) || (dictMode == ZSTD_dictMatchState) || (matchIndex+matchLength >= dictLimit)) {
-            assert(matchIndex+matchLength >= dictLimit);  /* ensure the condition is correct when !extDict */
-            match = base + matchIndex;
-            matchLength += ZSTD_count(ip+matchLength, match+matchLength, iLimit);
-        } else {
-            match = dictBase + matchIndex;
-            matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iLimit, dictEnd, prefixStart);
-            if (matchIndex+matchLength >= dictLimit)
-                match = base + matchIndex;   /* prepare for match[matchLength] */
-        }
-
-        if (matchLength > bestLength) {
-            DEBUGLOG(8, "found match of length %u at distance %u (offCode=%u)",
-                    (U32)matchLength, current - matchIndex, current - matchIndex + ZSTD_REP_MOVE);
-            assert(matchEndIdx > matchIndex);
-            if (matchLength > matchEndIdx - matchIndex)
-                matchEndIdx = matchIndex + (U32)matchLength;
-            bestLength = matchLength;
-            matches[mnum].off = (current - matchIndex) + ZSTD_REP_MOVE;
-            matches[mnum].len = (U32)matchLength;
-            mnum++;
-            if ( (matchLength > ZSTD_OPT_NUM)
-               | (ip+matchLength == iLimit) /* equal : no way to know if inf or sup */) {
-                if (dictMode == ZSTD_dictMatchState) nbCompares = 0; /* break should also skip searching dms */
-                break; /* drop, to preserve bt consistency (miss a little bit of compression) */
-            }
-        }
-
-        if (match[matchLength] < ip[matchLength]) {
-            /* match smaller than current */
-            *smallerPtr = matchIndex;             /* update smaller idx */
-            commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-            if (matchIndex <= btLow) { smallerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-            smallerPtr = nextPtr+1;               /* new candidate => larger than match, which was smaller than current */
-            matchIndex = nextPtr[1];              /* new matchIndex, larger than previous, closer to current */
-        } else {
-            *largerPtr = matchIndex;
-            commonLengthLarger = matchLength;
-            if (matchIndex <= btLow) { largerPtr=&dummy32; break; }   /* beyond tree size, stop the search */
-            largerPtr = nextPtr;
-            matchIndex = nextPtr[0];
-    }   }
-
-    *smallerPtr = *largerPtr = 0;
-
-    if (dictMode == ZSTD_dictMatchState && nbCompares) {
-        size_t const dmsH = ZSTD_hashPtr(ip, dmsHashLog, mls);
-        U32 dictMatchIndex = dms->hashTable[dmsH];
-        const U32* const dmsBt = dms->chainTable;
-        commonLengthSmaller = commonLengthLarger = 0;
-        while (nbCompares-- && (dictMatchIndex > dmsLowLimit)) {
-            const U32* const nextPtr = dmsBt + 2*(dictMatchIndex & dmsBtMask);
-            size_t matchLength = MIN(commonLengthSmaller, commonLengthLarger);   /* guaranteed minimum nb of common bytes */
-            const BYTE* match = dmsBase + dictMatchIndex;
-            matchLength += ZSTD_count_2segments(ip+matchLength, match+matchLength, iLimit, dmsEnd, prefixStart);
-            if (dictMatchIndex+matchLength >= dmsHighLimit)
-                match = base + dictMatchIndex + dmsIndexDelta;   /* to prepare for next usage of match[matchLength] */
-
-            if (matchLength > bestLength) {
-                matchIndex = dictMatchIndex + dmsIndexDelta;
-                DEBUGLOG(8, "found dms match of length %u at distance %u (offCode=%u)",
-                        (U32)matchLength, current - matchIndex, current - matchIndex + ZSTD_REP_MOVE);
-                if (matchLength > matchEndIdx - matchIndex)
-                    matchEndIdx = matchIndex + (U32)matchLength;
-                bestLength = matchLength;
-                matches[mnum].off = (current - matchIndex) + ZSTD_REP_MOVE;
-                matches[mnum].len = (U32)matchLength;
-                mnum++;
-                if ( (matchLength > ZSTD_OPT_NUM)
-                   | (ip+matchLength == iLimit) /* equal : no way to know if inf or sup */) {
-                    break;   /* drop, to guarantee consistency (miss a little bit of compression) */
-                }
-            }
-
-            if (dictMatchIndex <= dmsBtLow) { break; }   /* beyond tree size, stop the search */
-            if (match[matchLength] < ip[matchLength]) {
-                commonLengthSmaller = matchLength;    /* all smaller will now have at least this guaranteed common length */
-                dictMatchIndex = nextPtr[1];              /* new matchIndex larger than previous (closer to current) */
-            } else {
-                /* match is larger than current */
-                commonLengthLarger = matchLength;
-                dictMatchIndex = nextPtr[0];
-            }
-        }
-    }
-
-    assert(matchEndIdx > current+8);
-    ms->nextToUpdate = matchEndIdx - 8;  /* skip repetitive patterns */
-    return mnum;
-}
-
-
-FORCE_INLINE_TEMPLATE U32 ZSTD_BtGetAllMatches (
-                        ZSTD_match_t* matches,   /* store result (match found, increasing size) in this table */
-                        ZSTD_matchState_t* ms,
-                        U32* nextToUpdate3,
-                        const BYTE* ip, const BYTE* const iHighLimit, const ZSTD_dictMode_e dictMode,
-                        const U32 rep[ZSTD_REP_NUM],
-                        U32 const ll0,
-                        U32 const lengthToBeat)
-{
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-    U32 const matchLengthSearch = cParams->minMatch;
-    DEBUGLOG(8, "ZSTD_BtGetAllMatches");
-    if (ip < ms->window.base + ms->nextToUpdate) return 0;   /* skipped area */
-    ZSTD_updateTree_internal(ms, ip, iHighLimit, matchLengthSearch, dictMode);
-    switch(matchLengthSearch)
-    {
-    case 3 : return ZSTD_insertBtAndGetAllMatches(matches, ms, nextToUpdate3, ip, iHighLimit, dictMode, rep, ll0, lengthToBeat, 3);
-    default :
-    case 4 : return ZSTD_insertBtAndGetAllMatches(matches, ms, nextToUpdate3, ip, iHighLimit, dictMode, rep, ll0, lengthToBeat, 4);
-    case 5 : return ZSTD_insertBtAndGetAllMatches(matches, ms, nextToUpdate3, ip, iHighLimit, dictMode, rep, ll0, lengthToBeat, 5);
-    case 7 :
-    case 6 : return ZSTD_insertBtAndGetAllMatches(matches, ms, nextToUpdate3, ip, iHighLimit, dictMode, rep, ll0, lengthToBeat, 6);
-    }
-}
-
-
-/*-*******************************
-*  Optimal parser
-*********************************/
-typedef struct repcodes_s {
-    U32 rep[3];
-} repcodes_t;
-
-static repcodes_t ZSTD_updateRep(U32 const rep[3], U32 const offset, U32 const ll0)
-{
-    repcodes_t newReps;
-    if (offset >= ZSTD_REP_NUM) {  /* full offset */
-        newReps.rep[2] = rep[1];
-        newReps.rep[1] = rep[0];
-        newReps.rep[0] = offset - ZSTD_REP_MOVE;
-    } else {   /* repcode */
-        U32 const repCode = offset + ll0;
-        if (repCode > 0) {  /* note : if repCode==0, no change */
-            U32 const currentOffset = (repCode==ZSTD_REP_NUM) ? (rep[0] - 1) : rep[repCode];
-            newReps.rep[2] = (repCode >= 2) ? rep[1] : rep[2];
-            newReps.rep[1] = rep[0];
-            newReps.rep[0] = currentOffset;
-        } else {   /* repCode == 0 */
-            memcpy(&newReps, rep, sizeof(newReps));
-        }
-    }
-    return newReps;
-}
-
-
-static U32 ZSTD_totalLen(ZSTD_optimal_t sol)
-{
-    return sol.litlen + sol.mlen;
-}
-
-#if 0 /* debug */
-
-static void
-listStats(const U32* table, int lastEltID)
-{
-    int const nbElts = lastEltID + 1;
-    int enb;
-    for (enb=0; enb < nbElts; enb++) {
-        (void)table;
-        //RAWLOG(2, "%3i:%3i,  ", enb, table[enb]);
-        RAWLOG(2, "%4i,", table[enb]);
-    }
-    RAWLOG(2, " \n");
-}
-
-#endif
-
-FORCE_INLINE_TEMPLATE size_t
-ZSTD_compressBlock_opt_generic(ZSTD_matchState_t* ms,
-                               seqStore_t* seqStore,
-                               U32 rep[ZSTD_REP_NUM],
-                         const void* src, size_t srcSize,
-                         const int optLevel,
-                         const ZSTD_dictMode_e dictMode)
-{
-    optState_t* const optStatePtr = &ms->opt;
-    const BYTE* const istart = (const BYTE*)src;
-    const BYTE* ip = istart;
-    const BYTE* anchor = istart;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* const ilimit = iend - 8;
-    const BYTE* const base = ms->window.base;
-    const BYTE* const prefixStart = base + ms->window.dictLimit;
-    const ZSTD_compressionParameters* const cParams = &ms->cParams;
-
-    U32 const sufficient_len = MIN(cParams->targetLength, ZSTD_OPT_NUM -1);
-    U32 const minMatch = (cParams->minMatch == 3) ? 3 : 4;
-    U32 nextToUpdate3 = ms->nextToUpdate;
-
-    ZSTD_optimal_t* const opt = optStatePtr->priceTable;
-    ZSTD_match_t* const matches = optStatePtr->matchTable;
-    ZSTD_optimal_t lastSequence;
-
-    /* init */
-    DEBUGLOG(5, "ZSTD_compressBlock_opt_generic: current=%u, prefix=%u, nextToUpdate=%u",
-                (U32)(ip - base), ms->window.dictLimit, ms->nextToUpdate);
-    assert(optLevel <= 2);
-    ZSTD_rescaleFreqs(optStatePtr, (const BYTE*)src, srcSize, optLevel);
-    ip += (ip==prefixStart);
-
-    /* Match Loop */
-    while (ip < ilimit) {
-        U32 cur, last_pos = 0;
-
-        /* find first match */
-        {   U32 const litlen = (U32)(ip - anchor);
-            U32 const ll0 = !litlen;
-            U32 const nbMatches = ZSTD_BtGetAllMatches(matches, ms, &nextToUpdate3, ip, iend, dictMode, rep, ll0, minMatch);
-            if (!nbMatches) { ip++; continue; }
-
-            /* initialize opt[0] */
-            { U32 i ; for (i=0; i<ZSTD_REP_NUM; i++) opt[0].rep[i] = rep[i]; }
-            opt[0].mlen = 0;  /* means is_a_literal */
-            opt[0].litlen = litlen;
-            opt[0].price = ZSTD_literalsContribution(anchor, litlen, optStatePtr, optLevel);
-
-            /* large match -> immediate encoding */
-            {   U32 const maxML = matches[nbMatches-1].len;
-                U32 const maxOffset = matches[nbMatches-1].off;
-                DEBUGLOG(6, "found %u matches of maxLength=%u and maxOffCode=%u at cPos=%u => start new series",
-                            nbMatches, maxML, maxOffset, (U32)(ip-prefixStart));
-
-                if (maxML > sufficient_len) {
-                    lastSequence.litlen = litlen;
-                    lastSequence.mlen = maxML;
-                    lastSequence.off = maxOffset;
-                    DEBUGLOG(6, "large match (%u>%u), immediate encoding",
-                                maxML, sufficient_len);
-                    cur = 0;
-                    last_pos = ZSTD_totalLen(lastSequence);
-                    goto _shortestPath;
-            }   }
-
-            /* set prices for first matches starting position == 0 */
-            {   U32 const literalsPrice = opt[0].price + ZSTD_litLengthPrice(0, optStatePtr, optLevel);
-                U32 pos;
-                U32 matchNb;
-                for (pos = 1; pos < minMatch; pos++) {
-                    opt[pos].price = ZSTD_MAX_PRICE;   /* mlen, litlen and price will be fixed during forward scanning */
-                }
-                for (matchNb = 0; matchNb < nbMatches; matchNb++) {
-                    U32 const offset = matches[matchNb].off;
-                    U32 const end = matches[matchNb].len;
-                    repcodes_t const repHistory = ZSTD_updateRep(rep, offset, ll0);
-                    for ( ; pos <= end ; pos++ ) {
-                        U32 const matchPrice = ZSTD_getMatchPrice(offset, pos, optStatePtr, optLevel);
-                        U32 const sequencePrice = literalsPrice + matchPrice;
-                        DEBUGLOG(7, "rPos:%u => set initial price : %.2f",
-                                    pos, ZSTD_fCost(sequencePrice));
-                        opt[pos].mlen = pos;
-                        opt[pos].off = offset;
-                        opt[pos].litlen = litlen;
-                        opt[pos].price = sequencePrice;
-                        ZSTD_STATIC_ASSERT(sizeof(opt[pos].rep) == sizeof(repHistory));
-                        memcpy(opt[pos].rep, &repHistory, sizeof(repHistory));
-                }   }
-                last_pos = pos-1;
-            }
-        }
-
-        /* check further positions */
-        for (cur = 1; cur <= last_pos; cur++) {
-            const BYTE* const inr = ip + cur;
-            assert(cur < ZSTD_OPT_NUM);
-            DEBUGLOG(7, "cPos:%zi==rPos:%u", inr-istart, cur)
-
-            /* Fix current position with one literal if cheaper */
-            {   U32 const litlen = (opt[cur-1].mlen == 0) ? opt[cur-1].litlen + 1 : 1;
-                int const price = opt[cur-1].price
-                                + ZSTD_rawLiteralsCost(ip+cur-1, 1, optStatePtr, optLevel)
-                                + ZSTD_litLengthPrice(litlen, optStatePtr, optLevel)
-                                - ZSTD_litLengthPrice(litlen-1, optStatePtr, optLevel);
-                assert(price < 1000000000); /* overflow check */
-                if (price <= opt[cur].price) {
-                    DEBUGLOG(7, "cPos:%zi==rPos:%u : better price (%.2f<=%.2f) using literal (ll==%u) (hist:%u,%u,%u)",
-                                inr-istart, cur, ZSTD_fCost(price), ZSTD_fCost(opt[cur].price), litlen,
-                                opt[cur-1].rep[0], opt[cur-1].rep[1], opt[cur-1].rep[2]);
-                    opt[cur].mlen = 0;
-                    opt[cur].off = 0;
-                    opt[cur].litlen = litlen;
-                    opt[cur].price = price;
-                    memcpy(opt[cur].rep, opt[cur-1].rep, sizeof(opt[cur].rep));
-                } else {
-                    DEBUGLOG(7, "cPos:%zi==rPos:%u : literal would cost more (%.2f>%.2f) (hist:%u,%u,%u)",
-                                inr-istart, cur, ZSTD_fCost(price), ZSTD_fCost(opt[cur].price),
-                                opt[cur].rep[0], opt[cur].rep[1], opt[cur].rep[2]);
-                }
-            }
-
-            /* last match must start at a minimum distance of 8 from oend */
-            if (inr > ilimit) continue;
-
-            if (cur == last_pos) break;
-
-            if ( (optLevel==0) /*static_test*/
-              && (opt[cur+1].price <= opt[cur].price + (BITCOST_MULTIPLIER/2)) ) {
-                DEBUGLOG(7, "move to next rPos:%u : price is <=", cur+1);
-                continue;  /* skip unpromising positions; about ~+6% speed, -0.01 ratio */
-            }
-
-            {   U32 const ll0 = (opt[cur].mlen != 0);
-                U32 const litlen = (opt[cur].mlen == 0) ? opt[cur].litlen : 0;
-                U32 const previousPrice = opt[cur].price;
-                U32 const basePrice = previousPrice + ZSTD_litLengthPrice(0, optStatePtr, optLevel);
-                U32 const nbMatches = ZSTD_BtGetAllMatches(matches, ms, &nextToUpdate3, inr, iend, dictMode, opt[cur].rep, ll0, minMatch);
-                U32 matchNb;
-                if (!nbMatches) {
-                    DEBUGLOG(7, "rPos:%u : no match found", cur);
-                    continue;
-                }
-
-                {   U32 const maxML = matches[nbMatches-1].len;
-                    DEBUGLOG(7, "cPos:%zi==rPos:%u, found %u matches, of maxLength=%u",
-                                inr-istart, cur, nbMatches, maxML);
-
-                    if ( (maxML > sufficient_len)
-                      || (cur + maxML >= ZSTD_OPT_NUM) ) {
-                        lastSequence.mlen = maxML;
-                        lastSequence.off = matches[nbMatches-1].off;
-                        lastSequence.litlen = litlen;
-                        cur -= (opt[cur].mlen==0) ? opt[cur].litlen : 0;  /* last sequence is actually only literals, fix cur to last match - note : may underflow, in which case, it's first sequence, and it's okay */
-                        last_pos = cur + ZSTD_totalLen(lastSequence);
-                        if (cur > ZSTD_OPT_NUM) cur = 0;   /* underflow => first match */
-                        goto _shortestPath;
-                }   }
-
-                /* set prices using matches found at position == cur */
-                for (matchNb = 0; matchNb < nbMatches; matchNb++) {
-                    U32 const offset = matches[matchNb].off;
-                    repcodes_t const repHistory = ZSTD_updateRep(opt[cur].rep, offset, ll0);
-                    U32 const lastML = matches[matchNb].len;
-                    U32 const startML = (matchNb>0) ? matches[matchNb-1].len+1 : minMatch;
-                    U32 mlen;
-
-                    DEBUGLOG(7, "testing match %u => offCode=%4u, mlen=%2u, llen=%2u",
-                                matchNb, matches[matchNb].off, lastML, litlen);
-
-                    for (mlen = lastML; mlen >= startML; mlen--) {  /* scan downward */
-                        U32 const pos = cur + mlen;
-                        int const price = basePrice + ZSTD_getMatchPrice(offset, mlen, optStatePtr, optLevel);
-
-                        if ((pos > last_pos) || (price < opt[pos].price)) {
-                            DEBUGLOG(7, "rPos:%u (ml=%2u) => new better price (%.2f<%.2f)",
-                                        pos, mlen, ZSTD_fCost(price), ZSTD_fCost(opt[pos].price));
-                            while (last_pos < pos) { opt[last_pos+1].price = ZSTD_MAX_PRICE; last_pos++; }   /* fill empty positions */
-                            opt[pos].mlen = mlen;
-                            opt[pos].off = offset;
-                            opt[pos].litlen = litlen;
-                            opt[pos].price = price;
-                            ZSTD_STATIC_ASSERT(sizeof(opt[pos].rep) == sizeof(repHistory));
-                            memcpy(opt[pos].rep, &repHistory, sizeof(repHistory));
-                        } else {
-                            DEBUGLOG(7, "rPos:%u (ml=%2u) => new price is worse (%.2f>=%.2f)",
-                                        pos, mlen, ZSTD_fCost(price), ZSTD_fCost(opt[pos].price));
-                            if (optLevel==0) break;  /* early update abort; gets ~+10% speed for about -0.01 ratio loss */
-                        }
-            }   }   }
-        }  /* for (cur = 1; cur <= last_pos; cur++) */
-
-        lastSequence = opt[last_pos];
-        cur = last_pos > ZSTD_totalLen(lastSequence) ? last_pos - ZSTD_totalLen(lastSequence) : 0;  /* single sequence, and it starts before `ip` */
-        assert(cur < ZSTD_OPT_NUM);  /* control overflow*/
-
-_shortestPath:   /* cur, last_pos, best_mlen, best_off have to be set */
-        assert(opt[0].mlen == 0);
-
-        {   U32 const storeEnd = cur + 1;
-            U32 storeStart = storeEnd;
-            U32 seqPos = cur;
-
-            DEBUGLOG(6, "start reverse traversal (last_pos:%u, cur:%u)",
-                        last_pos, cur); (void)last_pos;
-            assert(storeEnd < ZSTD_OPT_NUM);
-            DEBUGLOG(6, "last sequence copied into pos=%u (llen=%u,mlen=%u,ofc=%u)",
-                        storeEnd, lastSequence.litlen, lastSequence.mlen, lastSequence.off);
-            opt[storeEnd] = lastSequence;
-            while (seqPos > 0) {
-                U32 const backDist = ZSTD_totalLen(opt[seqPos]);
-                storeStart--;
-                DEBUGLOG(6, "sequence from rPos=%u copied into pos=%u (llen=%u,mlen=%u,ofc=%u)",
-                            seqPos, storeStart, opt[seqPos].litlen, opt[seqPos].mlen, opt[seqPos].off);
-                opt[storeStart] = opt[seqPos];
-                seqPos = (seqPos > backDist) ? seqPos - backDist : 0;
-            }
-
-            /* save sequences */
-            DEBUGLOG(6, "sending selected sequences into seqStore")
-            {   U32 storePos;
-                for (storePos=storeStart; storePos <= storeEnd; storePos++) {
-                    U32 const llen = opt[storePos].litlen;
-                    U32 const mlen = opt[storePos].mlen;
-                    U32 const offCode = opt[storePos].off;
-                    U32 const advance = llen + mlen;
-                    DEBUGLOG(6, "considering seq starting at %zi, llen=%u, mlen=%u",
-                                anchor - istart, (unsigned)llen, (unsigned)mlen);
-
-                    if (mlen==0) {  /* only literals => must be last "sequence", actually starting a new stream of sequences */
-                        assert(storePos == storeEnd);   /* must be last sequence */
-                        ip = anchor + llen;     /* last "sequence" is a bunch of literals => don't progress anchor */
-                        continue;   /* will finish */
-                    }
-
-                    /* repcodes update : like ZSTD_updateRep(), but update in place */
-                    if (offCode >= ZSTD_REP_NUM) {  /* full offset */
-                        rep[2] = rep[1];
-                        rep[1] = rep[0];
-                        rep[0] = offCode - ZSTD_REP_MOVE;
-                    } else {   /* repcode */
-                        U32 const repCode = offCode + (llen==0);
-                        if (repCode) {  /* note : if repCode==0, no change */
-                            U32 const currentOffset = (repCode==ZSTD_REP_NUM) ? (rep[0] - 1) : rep[repCode];
-                            if (repCode >= 2) rep[2] = rep[1];
-                            rep[1] = rep[0];
-                            rep[0] = currentOffset;
-                    }   }
-
-                    assert(anchor + llen <= iend);
-                    ZSTD_updateStats(optStatePtr, llen, anchor, offCode, mlen);
-                    ZSTD_storeSeq(seqStore, llen, anchor, offCode, mlen-MINMATCH);
-                    anchor += advance;
-                    ip = anchor;
-            }   }
-            ZSTD_setBasePrices(optStatePtr, optLevel);
-        }
-
-    }   /* while (ip < ilimit) */
-
-    /* Return the last literals size */
-    return (size_t)(iend - anchor);
-}
-
-
-size_t ZSTD_compressBlock_btopt(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    DEBUGLOG(5, "ZSTD_compressBlock_btopt");
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 0 /*optLevel*/, ZSTD_noDict);
-}
-
-
-/* used in 2-pass strategy */
-static U32 ZSTD_upscaleStat(unsigned* table, U32 lastEltIndex, int bonus)
-{
-    U32 s, sum=0;
-    assert(ZSTD_FREQ_DIV+bonus >= 0);
-    for (s=0; s<lastEltIndex+1; s++) {
-        table[s] <<= ZSTD_FREQ_DIV+bonus;
-        table[s]--;
-        sum += table[s];
-    }
-    return sum;
-}
-
-/* used in 2-pass strategy */
-MEM_STATIC void ZSTD_upscaleStats(optState_t* optPtr)
-{
-    if (ZSTD_compressedLiterals(optPtr))
-        optPtr->litSum = ZSTD_upscaleStat(optPtr->litFreq, MaxLit, 0);
-    optPtr->litLengthSum = ZSTD_upscaleStat(optPtr->litLengthFreq, MaxLL, 0);
-    optPtr->matchLengthSum = ZSTD_upscaleStat(optPtr->matchLengthFreq, MaxML, 0);
-    optPtr->offCodeSum = ZSTD_upscaleStat(optPtr->offCodeFreq, MaxOff, 0);
-}
-
-/* ZSTD_initStats_ultra():
- * make a first compression pass, just to seed stats with more accurate starting values.
- * only works on first block, with no dictionary and no ldm.
- * this function cannot error, hence its contract must be respected.
- */
-static void
-ZSTD_initStats_ultra(ZSTD_matchState_t* ms,
-                     seqStore_t* seqStore,
-                     U32 rep[ZSTD_REP_NUM],
-               const void* src, size_t srcSize)
-{
-    U32 tmpRep[ZSTD_REP_NUM];  /* updated rep codes will sink here */
-    memcpy(tmpRep, rep, sizeof(tmpRep));
-
-    DEBUGLOG(4, "ZSTD_initStats_ultra (srcSize=%zu)", srcSize);
-    assert(ms->opt.litLengthSum == 0);    /* first block */
-    assert(seqStore->sequences == seqStore->sequencesStart);   /* no ldm */
-    assert(ms->window.dictLimit == ms->window.lowLimit);   /* no dictionary */
-    assert(ms->window.dictLimit - ms->nextToUpdate <= 1);  /* no prefix (note: intentional overflow, defined as 2-complement) */
-
-    ZSTD_compressBlock_opt_generic(ms, seqStore, tmpRep, src, srcSize, 2 /*optLevel*/, ZSTD_noDict);   /* generate stats into ms->opt*/
-
-    /* invalidate first scan from history */
-    ZSTD_resetSeqStore(seqStore);
-    ms->window.base -= srcSize;
-    ms->window.dictLimit += (U32)srcSize;
-    ms->window.lowLimit = ms->window.dictLimit;
-    ms->nextToUpdate = ms->window.dictLimit;
-
-    /* re-inforce weight of collected statistics */
-    ZSTD_upscaleStats(&ms->opt);
-}
-
-size_t ZSTD_compressBlock_btultra(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    DEBUGLOG(5, "ZSTD_compressBlock_btultra (srcSize=%zu)", srcSize);
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 2 /*optLevel*/, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_btultra2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    U32 const current = (U32)((const BYTE*)src - ms->window.base);
-    DEBUGLOG(5, "ZSTD_compressBlock_btultra2 (srcSize=%zu)", srcSize);
-
-    /* 2-pass strategy:
-     * this strategy makes a first pass over first block to collect statistics
-     * and seed next round's statistics with it.
-     * After 1st pass, function forgets everything, and starts a new block.
-     * Consequently, this can only work if no data has been previously loaded in tables,
-     * aka, no dictionary, no prefix, no ldm preprocessing.
-     * The compression ratio gain is generally small (~0.5% on first block),
-     * the cost is 2x cpu time on first block. */
-    assert(srcSize <= ZSTD_BLOCKSIZE_MAX);
-    if ( (ms->opt.litLengthSum==0)   /* first block */
-      && (seqStore->sequences == seqStore->sequencesStart)  /* no ldm */
-      && (ms->window.dictLimit == ms->window.lowLimit)   /* no dictionary */
-      && (current == ms->window.dictLimit)   /* start of frame, nothing already loaded nor skipped */
-      && (srcSize > ZSTD_PREDEF_THRESHOLD)
-      ) {
-        ZSTD_initStats_ultra(ms, seqStore, rep, src, srcSize);
-    }
-
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 2 /*optLevel*/, ZSTD_noDict);
-}
-
-size_t ZSTD_compressBlock_btopt_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 0 /*optLevel*/, ZSTD_dictMatchState);
-}
-
-size_t ZSTD_compressBlock_btultra_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 2 /*optLevel*/, ZSTD_dictMatchState);
-}
-
-size_t ZSTD_compressBlock_btopt_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 0 /*optLevel*/, ZSTD_extDict);
-}
-
-size_t ZSTD_compressBlock_btultra_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        const void* src, size_t srcSize)
-{
-    return ZSTD_compressBlock_opt_generic(ms, seqStore, rep, src, srcSize, 2 /*optLevel*/, ZSTD_extDict);
-}
-
-/* note : no btultra2 variant for extDict nor dictMatchState,
- * because btultra2 is not meant to work with dictionaries
- * and is only specific for the first block (no prefix) */
diff --git a/vendor/github.com/DataDog/zstd/zstd_opt.h b/vendor/github.com/DataDog/zstd/zstd_opt.h
deleted file mode 100644
index 094f747..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_opt.h
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_OPT_H
-#define ZSTD_OPT_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-#include "zstd_compress_internal.h"
-
-/* used in ZSTD_loadDictionaryContent() */
-void ZSTD_updateTree(ZSTD_matchState_t* ms, const BYTE* ip, const BYTE* iend);
-
-size_t ZSTD_compressBlock_btopt(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_btultra(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_btultra2(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-
-size_t ZSTD_compressBlock_btopt_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_btultra_dictMatchState(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-size_t ZSTD_compressBlock_btopt_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-size_t ZSTD_compressBlock_btultra_extDict(
-        ZSTD_matchState_t* ms, seqStore_t* seqStore, U32 rep[ZSTD_REP_NUM],
-        void const* src, size_t srcSize);
-
-        /* note : no btultra2 variant for extDict nor dictMatchState,
-         * because btultra2 is not meant to work with dictionaries
-         * and is only specific for the first block (no prefix) */
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_OPT_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_stream.go b/vendor/github.com/DataDog/zstd/zstd_stream.go
deleted file mode 100644
index 2330353..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_stream.go
+++ /dev/null
@@ -1,294 +0,0 @@
-package zstd
-
-/*
-#define ZSTD_STATIC_LINKING_ONLY
-#define ZBUFF_DISABLE_DEPRECATE_WARNINGS
-#include "zstd.h"
-#include "zbuff.h"
-*/
-import "C"
-import (
-	"errors"
-	"fmt"
-	"io"
-	"runtime"
-	"unsafe"
-)
-
-var errShortRead = errors.New("short read")
-
-// Writer is an io.WriteCloser that zstd-compresses its input.
-type Writer struct {
-	CompressionLevel int
-
-	ctx              *C.ZSTD_CCtx
-	dict             []byte
-	dstBuffer        []byte
-	firstError       error
-	underlyingWriter io.Writer
-}
-
-func resize(in []byte, newSize int) []byte {
-	if in == nil {
-		return make([]byte, newSize)
-	}
-	if newSize <= cap(in) {
-		return in[:newSize]
-	}
-	toAdd := newSize - len(in)
-	return append(in, make([]byte, toAdd)...)
-}
-
-// NewWriter creates a new Writer with default compression options.  Writes to
-// the writer will be written in compressed form to w.
-func NewWriter(w io.Writer) *Writer {
-	return NewWriterLevelDict(w, DefaultCompression, nil)
-}
-
-// NewWriterLevel is like NewWriter but specifies the compression level instead
-// of assuming default compression.
-//
-// The level can be DefaultCompression or any integer value between BestSpeed
-// and BestCompression inclusive.
-func NewWriterLevel(w io.Writer, level int) *Writer {
-	return NewWriterLevelDict(w, level, nil)
-
-}
-
-// NewWriterLevelDict is like NewWriterLevel but specifies a dictionary to
-// compress with.  If the dictionary is empty or nil it is ignored. The dictionary
-// should not be modified until the writer is closed.
-func NewWriterLevelDict(w io.Writer, level int, dict []byte) *Writer {
-	var err error
-	ctx := C.ZSTD_createCCtx()
-
-	if dict == nil {
-		err = getError(int(C.ZSTD_compressBegin(ctx,
-			C.int(level))))
-	} else {
-		err = getError(int(C.ZSTD_compressBegin_usingDict(
-			ctx,
-			unsafe.Pointer(&dict[0]),
-			C.size_t(len(dict)),
-			C.int(level))))
-	}
-
-	return &Writer{
-		CompressionLevel: level,
-		ctx:              ctx,
-		dict:             dict,
-		dstBuffer:        make([]byte, CompressBound(1024)),
-		firstError:       err,
-		underlyingWriter: w,
-	}
-}
-
-// Write writes a compressed form of p to the underlying io.Writer.
-func (w *Writer) Write(p []byte) (int, error) {
-	if w.firstError != nil {
-		return 0, w.firstError
-	}
-	if len(p) == 0 {
-		return 0, nil
-	}
-	// Check if dstBuffer is enough
-	if len(w.dstBuffer) < CompressBound(len(p)) {
-		w.dstBuffer = make([]byte, CompressBound(len(p)))
-	}
-
-	retCode := C.ZSTD_compressContinue(
-		w.ctx,
-		unsafe.Pointer(&w.dstBuffer[0]),
-		C.size_t(len(w.dstBuffer)),
-		unsafe.Pointer(&p[0]),
-		C.size_t(len(p)))
-
-	if err := getError(int(retCode)); err != nil {
-		return 0, err
-	}
-	written := int(retCode)
-
-	// Write to underlying buffer
-	_, err := w.underlyingWriter.Write(w.dstBuffer[:written])
-
-	// Same behaviour as zlib, we can't know how much data we wrote, only
-	// if there was an error
-	if err != nil {
-		return 0, err
-	}
-	return len(p), err
-}
-
-// Close closes the Writer, flushing any unwritten data to the underlying
-// io.Writer and freeing objects, but does not close the underlying io.Writer.
-func (w *Writer) Close() error {
-	retCode := C.ZSTD_compressEnd(
-		w.ctx,
-		unsafe.Pointer(&w.dstBuffer[0]),
-		C.size_t(len(w.dstBuffer)),
-		unsafe.Pointer(nil),
-		C.size_t(0))
-
-	if err := getError(int(retCode)); err != nil {
-		return err
-	}
-	written := int(retCode)
-	retCode = C.ZSTD_freeCCtx(w.ctx) // Safely close buffer before writing the end
-
-	if err := getError(int(retCode)); err != nil {
-		return err
-	}
-
-	_, err := w.underlyingWriter.Write(w.dstBuffer[:written])
-	if err != nil {
-		return err
-	}
-	return nil
-}
-
-// reader is an io.ReadCloser that decompresses when read from.
-type reader struct {
-	ctx                 *C.ZBUFF_DCtx
-	compressionBuffer   []byte
-	compressionLeft     int
-	decompressionBuffer []byte
-	decompOff           int
-	decompSize          int
-	dict                []byte
-	firstError          error
-	recommendedSrcSize  int
-	underlyingReader    io.Reader
-}
-
-// NewReader creates a new io.ReadCloser.  Reads from the returned ReadCloser
-// read and decompress data from r.  It is the caller's responsibility to call
-// Close on the ReadCloser when done.  If this is not done, underlying objects
-// in the zstd library will not be freed.
-func NewReader(r io.Reader) io.ReadCloser {
-	return NewReaderDict(r, nil)
-}
-
-// NewReaderDict is like NewReader but uses a preset dictionary.  NewReaderDict
-// ignores the dictionary if it is nil.
-func NewReaderDict(r io.Reader, dict []byte) io.ReadCloser {
-	var err error
-	ctx := C.ZBUFF_createDCtx()
-	if len(dict) == 0 {
-		err = getError(int(C.ZBUFF_decompressInit(ctx)))
-	} else {
-		err = getError(int(C.ZBUFF_decompressInitDictionary(
-			ctx,
-			unsafe.Pointer(&dict[0]),
-			C.size_t(len(dict)))))
-	}
-	cSize := int(C.ZBUFF_recommendedDInSize())
-	dSize := int(C.ZBUFF_recommendedDOutSize())
-	if cSize <= 0 {
-		panic(fmt.Errorf("ZBUFF_recommendedDInSize() returned invalid size: %v", cSize))
-	}
-	if dSize <= 0 {
-		panic(fmt.Errorf("ZBUFF_recommendedDOutSize() returned invalid size: %v", dSize))
-	}
-
-	compressionBuffer := make([]byte, cSize)
-	decompressionBuffer := make([]byte, dSize)
-	return &reader{
-		ctx:                 ctx,
-		dict:                dict,
-		compressionBuffer:   compressionBuffer,
-		decompressionBuffer: decompressionBuffer,
-		firstError:          err,
-		recommendedSrcSize:  cSize,
-		underlyingReader:    r,
-	}
-}
-
-// Close frees the allocated C objects
-func (r *reader) Close() error {
-	return getError(int(C.ZBUFF_freeDCtx(r.ctx)))
-}
-
-func (r *reader) Read(p []byte) (int, error) {
-
-	// If we already have enough bytes, return
-	if r.decompSize-r.decompOff >= len(p) {
-		copy(p, r.decompressionBuffer[r.decompOff:])
-		r.decompOff += len(p)
-		return len(p), nil
-	}
-
-	copy(p, r.decompressionBuffer[r.decompOff:r.decompSize])
-	got := r.decompSize - r.decompOff
-	r.decompSize = 0
-	r.decompOff = 0
-
-	for got < len(p) {
-		// Populate src
-		src := r.compressionBuffer
-		reader := r.underlyingReader
-		n, err := TryReadFull(reader, src[r.compressionLeft:])
-		if err != nil && err != errShortRead { // Handle underlying reader errors first
-			return 0, fmt.Errorf("failed to read from underlying reader: %s", err)
-		} else if n == 0 && r.compressionLeft == 0 {
-			return got, io.EOF
-		}
-		src = src[:r.compressionLeft+n]
-
-		// C code
-		cSrcSize := C.size_t(len(src))
-		cDstSize := C.size_t(len(r.decompressionBuffer))
-		retCode := int(C.ZBUFF_decompressContinue(
-			r.ctx,
-			unsafe.Pointer(&r.decompressionBuffer[0]),
-			&cDstSize,
-			unsafe.Pointer(&src[0]),
-			&cSrcSize))
-
-		// Keep src here eventhough, we reuse later, the code might be deleted at some point
-		runtime.KeepAlive(src)
-		if err = getError(retCode); err != nil {
-			return 0, fmt.Errorf("failed to decompress: %s", err)
-		}
-
-		// Put everything in buffer
-		if int(cSrcSize) < len(src) {
-			left := src[int(cSrcSize):]
-			copy(r.compressionBuffer, left)
-		}
-		r.compressionLeft = len(src) - int(cSrcSize)
-		r.decompSize = int(cDstSize)
-		r.decompOff = copy(p[got:], r.decompressionBuffer[:r.decompSize])
-		got += r.decompOff
-
-		// Resize buffers
-		nsize := retCode // Hint for next src buffer size
-		if nsize <= 0 {
-			// Reset to recommended size
-			nsize = r.recommendedSrcSize
-		}
-		if nsize < r.compressionLeft {
-			nsize = r.compressionLeft
-		}
-		r.compressionBuffer = resize(r.compressionBuffer, nsize)
-	}
-	return got, nil
-}
-
-// TryReadFull reads buffer just as ReadFull does
-// Here we expect that buffer may end and we do not return ErrUnexpectedEOF as ReadAtLeast does.
-// We return errShortRead instead to distinguish short reads and failures.
-// We cannot use ReadFull/ReadAtLeast because it masks Reader errors, such as network failures
-// and causes panic instead of error.
-func TryReadFull(r io.Reader, buf []byte) (n int, err error) {
-	for n < len(buf) && err == nil {
-		var nn int
-		nn, err = r.Read(buf[n:])
-		n += nn
-	}
-	if n == len(buf) && err == io.EOF {
-		err = nil // EOF at the end is somewhat expected
-	} else if err == io.EOF {
-		err = errShortRead
-	}
-	return
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_v01.c b/vendor/github.com/DataDog/zstd/zstd_v01.c
deleted file mode 100644
index ae8cba2..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v01.c
+++ /dev/null
@@ -1,2152 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/******************************************
-*  Includes
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include "zstd_v01.h"
-#include "error_private.h"
-
-
-/******************************************
-*  Static allocation
-******************************************/
-/* You can statically allocate FSE CTable/DTable as a table of unsigned using below macro */
-#define FSE_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-/* You can statically allocate Huff0 DTable as a table of unsigned short using below macro */
-#define HUF_DTABLE_SIZE_U16(maxTableLog)   (1 + (1<<maxTableLog))
-#define HUF_CREATE_STATIC_DTABLE(DTable, maxTableLog) \
-        unsigned short DTable[HUF_DTABLE_SIZE_U16(maxTableLog)] = { maxTableLog }
-
-
-/******************************************
-*  Error Management
-******************************************/
-#define FSE_LIST_ERRORS(ITEM) \
-        ITEM(FSE_OK_NoError) ITEM(FSE_ERROR_GENERIC) \
-        ITEM(FSE_ERROR_tableLog_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooSmall) \
-        ITEM(FSE_ERROR_dstSize_tooSmall) ITEM(FSE_ERROR_srcSize_wrong)\
-        ITEM(FSE_ERROR_corruptionDetected) \
-        ITEM(FSE_ERROR_maxCode)
-
-#define FSE_GENERATE_ENUM(ENUM) ENUM,
-typedef enum { FSE_LIST_ERRORS(FSE_GENERATE_ENUM) } FSE_errorCodes;  /* enum is exposed, to detect & handle specific errors; compare function result to -enum value */
-
-
-/******************************************
-*  FSE symbol compression API
-******************************************/
-/*
-   This API consists of small unitary functions, which highly benefit from being inlined.
-   You will want to enable link-time-optimization to ensure these functions are properly inlined in your binary.
-   Visual seems to do it automatically.
-   For gcc or clang, you'll need to add -flto flag at compilation and linking stages.
-   If none of these solutions is applicable, include "fse.c" directly.
-*/
-
-typedef unsigned FSE_CTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-typedef unsigned FSE_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-
-typedef struct
-{
-    size_t bitContainer;
-    int    bitPos;
-    char*  startPtr;
-    char*  ptr;
-    char*  endPtr;
-} FSE_CStream_t;
-
-typedef struct
-{
-    ptrdiff_t   value;
-    const void* stateTable;
-    const void* symbolTT;
-    unsigned    stateLog;
-} FSE_CState_t;
-
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} FSE_DStream_t;
-
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-typedef enum { FSE_DStream_unfinished = 0,
-               FSE_DStream_endOfBuffer = 1,
-               FSE_DStream_completed = 2,
-               FSE_DStream_tooFar = 3 } FSE_DStream_status;  /* result of FSE_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... ?! */
-
-
-/****************************************************************
-*  Tuning parameters
-****************************************************************/
-/* MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSE_MAX_MEMORY_USAGE 14
-#define FSE_DEFAULT_MEMORY_USAGE 13
-
-/* FSE_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSE_MAX_SYMBOL_VALUE 255
-
-
-/****************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-
-
-/****************************************************************
-*  Byte symbol type
-****************************************************************/
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSE_decode_t;   /* size == U32 */
-
-
-
-/****************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__)
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/****************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-
-#ifndef MEM_ACCESS_MODULE
-#define MEM_ACCESS_MODULE
-/****************************************************************
-*  Basic Types
-*****************************************************************/
-#if defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-# include <stdint.h>
-typedef  uint8_t BYTE;
-typedef uint16_t U16;
-typedef  int16_t S16;
-typedef uint32_t U32;
-typedef  int32_t S32;
-typedef uint64_t U64;
-typedef  int64_t S64;
-#else
-typedef unsigned char       BYTE;
-typedef unsigned short      U16;
-typedef   signed short      S16;
-typedef unsigned int        U32;
-typedef   signed int        S32;
-typedef unsigned long long  U64;
-typedef   signed long long  S64;
-#endif
-
-#endif   /* MEM_ACCESS_MODULE */
-
-/****************************************************************
-*  Memory I/O
-*****************************************************************/
-/* FSE_FORCE_MEMORY_ACCESS
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets generating assembly depending on alignment.
- *            But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef FSE_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define FSE_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define FSE_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-
-static unsigned FSE_32bits(void)
-{
-    return sizeof(void*)==4;
-}
-
-static unsigned FSE_isLittleEndian(void)
-{
-    const union { U32 i; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(FSE_FORCE_MEMORY_ACCESS) && (FSE_FORCE_MEMORY_ACCESS==2)
-
-static U16 FSE_read16(const void* memPtr) { return *(const U16*) memPtr; }
-static U32 FSE_read32(const void* memPtr) { return *(const U32*) memPtr; }
-static U64 FSE_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-#elif defined(FSE_FORCE_MEMORY_ACCESS) && (FSE_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; } __attribute__((packed)) unalign;
-
-static U16 FSE_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-static U32 FSE_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-static U64 FSE_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-#else
-
-static U16 FSE_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-static U32 FSE_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-static U64 FSE_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-#endif // FSE_FORCE_MEMORY_ACCESS
-
-static U16 FSE_readLE16(const void* memPtr)
-{
-    if (FSE_isLittleEndian())
-        return FSE_read16(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-static U32 FSE_readLE32(const void* memPtr)
-{
-    if (FSE_isLittleEndian())
-        return FSE_read32(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24));
-    }
-}
-
-
-static U64 FSE_readLE64(const void* memPtr)
-{
-    if (FSE_isLittleEndian())
-        return FSE_read64(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24)
-                     + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56));
-    }
-}
-
-static size_t FSE_readLEST(const void* memPtr)
-{
-    if (FSE_32bits())
-        return (size_t)FSE_readLE32(memPtr);
-    else
-        return (size_t)FSE_readLE64(memPtr);
-}
-
-
-
-/****************************************************************
-*  Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG  (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-#error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-
-/****************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/****************************************************************
-*  Complex types
-****************************************************************/
-typedef struct
-{
-    int deltaFindState;
-    U32 deltaNbBits;
-} FSE_symbolCompressionTransform; /* total 8 bytes */
-
-typedef U32 DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-/****************************************************************
-*  Internal functions
-****************************************************************/
-FORCE_INLINE unsigned FSE_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (GCC_VERSION >= 304)   /* GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-/****************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-
-static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
-
-#define FSE_DECODE_TYPE FSE_decode_t
-
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSE_DTableHeader;   /* sizeof U32 */
-
-static size_t FSE_buildDTable
-(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*)(ptr) + 1;   /* because dt is unsigned, 32-bits aligned on 32-bits */
-    const U32 tableSize = 1 << tableLog;
-    const U32 tableMask = tableSize-1;
-    const U32 step = FSE_tableStep(tableSize);
-    U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-    U32 position = 0;
-    U32 highThreshold = tableSize-1;
-    const S16 largeLimit= (S16)(1 << (tableLog-1));
-    U32 noLarge = 1;
-    U32 s;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return (size_t)-FSE_ERROR_maxSymbolValue_tooLarge;
-    if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_tableLog_tooLarge;
-
-    /* Init, lay down lowprob symbols */
-    DTableH[0].tableLog = (U16)tableLog;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        if (normalizedCounter[s]==-1)
-        {
-            tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
-            symbolNext[s] = 1;
-        }
-        else
-        {
-            if (normalizedCounter[s] >= largeLimit) noLarge=0;
-            symbolNext[s] = normalizedCounter[s];
-        }
-    }
-
-    /* Spread symbols */
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        int i;
-        for (i=0; i<normalizedCounter[s]; i++)
-        {
-            tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
-            position = (position + step) & tableMask;
-            while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }
-    }
-
-    if (position!=0) return (size_t)-FSE_ERROR_GENERIC;   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-
-    /* Build Decoding table */
-    {
-        U32 i;
-        for (i=0; i<tableSize; i++)
-        {
-            FSE_FUNCTION_TYPE symbol = (FSE_FUNCTION_TYPE)(tableDecode[i].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[i].nbBits = (BYTE) (tableLog - FSE_highbit32 ((U32)nextState) );
-            tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
-        }
-    }
-
-    DTableH->fastMode = (U16)noLarge;
-    return 0;
-}
-
-
-/******************************************
-*  FSE byte symbol
-******************************************/
-#ifndef FSE_COMMONDEFS_ONLY
-
-static unsigned FSE_isError(size_t code) { return (code > (size_t)(-FSE_ERROR_maxCode)); }
-
-static short FSE_abs(short a)
-{
-    return a<0? -a : a;
-}
-
-
-/****************************************************************
-*  Header bitstream management
-****************************************************************/
-static size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return (size_t)-FSE_ERROR_srcSize_wrong;
-    bitStream = FSE_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return (size_t)-FSE_ERROR_tableLog_tooLarge;
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr))
-    {
-        if (previous0)
-        {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF)
-            {
-                n0+=24;
-                if (ip < iend-5)
-                {
-                    ip+=2;
-                    bitStream = FSE_readLE32(ip) >> bitCount;
-                }
-                else
-                {
-                    bitStream >>= 16;
-                    bitCount+=16;
-                }
-            }
-            while ((bitStream & 3) == 3)
-            {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return (size_t)-FSE_ERROR_maxSymbolValue_tooSmall;
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-            {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = FSE_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {
-            const short max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max)
-            {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            }
-            else
-            {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSE_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold)
-            {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            {
-                if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-                {
-                    ip += bitCount>>3;
-                    bitCount &= 7;
-                }
-                else
-                {
-                    bitCount -= (int)(8 * (iend - 4 - ip));
-                    ip = iend - 4;
-                }
-                bitStream = FSE_readLE32(ip) >> (bitCount & 31);
-            }
-        }
-    }
-    if (remaining != 1) return (size_t)-FSE_ERROR_GENERIC;
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-    return ip-istart;
-}
-
-
-/*********************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const cell = (FSE_decode_t*)(ptr) + 1;   /* because dt is unsigned */
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const dinfo = (FSE_decode_t*)(ptr) + 1;   /* because dt is unsigned */
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC;             /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-
-/* FSE_initDStream
- * Initialize a FSE_DStream_t.
- * srcBuffer must point at the beginning of an FSE block.
- * The function result is the size of the FSE_block (== srcSize).
- * If srcSize is too small, the function will return an errorCode;
- */
-static size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) return (size_t)-FSE_ERROR_srcSize_wrong;
-
-    if (srcSize >=  sizeof(size_t))
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(size_t);
-        bitD->bitContainer = FSE_readLEST(bitD->ptr);
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC;   /* stop bit not present */
-        bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
-    }
-    else
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);
-                    /* fallthrough */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);
-                    /* fallthrough */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);
-                    /* fallthrough */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24;
-                    /* fallthrough */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16;
-                    /* fallthrough */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) <<  8;
-                    /* fallthrough */
-            default:;
-        }
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC;   /* stop bit not present */
-        bitD->bitsConsumed = 8 - FSE_highbit32(contain32);
-        bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-
-/*!FSE_lookBits
- * Provides next n bits from the bitContainer.
- * bitContainer is not modified (bits are still present for next read/look)
- * On 32-bits, maxNbBits==25
- * On 64-bits, maxNbBits==57
- * return : value extracted.
- */
-static size_t FSE_lookBits(FSE_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-static size_t FSE_lookBitsFast(FSE_DStream_t* bitD, U32 nbBits)   /* only if nbBits >= 1 !! */
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-static void FSE_skipBits(FSE_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-
-/*!FSE_readBits
- * Read next n bits from the bitContainer.
- * On 32-bits, don't read more than maxNbBits==25
- * On 64-bits, don't read more than maxNbBits==57
- * Use the fast variant *only* if n >= 1.
- * return : value extracted.
- */
-static size_t FSE_readBits(FSE_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = FSE_lookBits(bitD, nbBits);
-    FSE_skipBits(bitD, nbBits);
-    return value;
-}
-
-static size_t FSE_readBitsFast(FSE_DStream_t* bitD, U32 nbBits)   /* only if nbBits >= 1 !! */
-{
-    size_t value = FSE_lookBitsFast(bitD, nbBits);
-    FSE_skipBits(bitD, nbBits);
-    return value;
-}
-
-static unsigned FSE_reloadDStream(FSE_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return FSE_DStream_tooFar;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer))
-    {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = FSE_readLEST(bitD->ptr);
-        return FSE_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start)
-    {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return FSE_DStream_endOfBuffer;
-        return FSE_DStream_completed;
-    }
-    {
-        U32 nbBytes = bitD->bitsConsumed >> 3;
-        U32 result = FSE_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start)
-        {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = FSE_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = FSE_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-
-static void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const FSE_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSE_DTableHeader* const DTableH = (const FSE_DTableHeader*)ptr;
-    DStatePtr->state = FSE_readBits(bitD, DTableH->tableLog);
-    FSE_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-static BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32  nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = FSE_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-static BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32 nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = FSE_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-/* FSE_endOfDStream
-   Tells if bitD has reached end of bitStream or not */
-
-static unsigned FSE_endOfDStream(const FSE_DStream_t* bitD)
-{
-    return ((bitD->ptr == bitD->start) && (bitD->bitsConsumed == sizeof(bitD->bitContainer)*8));
-}
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSE_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    FSE_DStream_t bitD;
-    FSE_DState_t state1;
-    FSE_DState_t state2;
-    size_t errorCode;
-
-    /* Init */
-    errorCode = FSE_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-    if (FSE_isError(errorCode)) return errorCode;
-
-    FSE_initDState(&state1, &bitD, dt);
-    FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (FSE_reloadDStream(&bitD)==FSE_DStream_unfinished) && (op<olimit) ; op+=4)
-    {
-        op[0] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            FSE_reloadDStream(&bitD);
-
-        op[1] = FSE_GETSYMBOL(&state2);
-
-        if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (FSE_reloadDStream(&bitD) > FSE_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            FSE_reloadDStream(&bitD);
-
-        op[3] = FSE_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : FSE_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly FSE_DStream_completed */
-    while (1)
-    {
-        if ( (FSE_reloadDStream(&bitD)>FSE_DStream_completed) || (op==omax) || (FSE_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state1))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state1);
-
-        if ( (FSE_reloadDStream(&bitD)>FSE_DStream_completed) || (op==omax) || (FSE_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state2))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state2);
-    }
-
-    /* end ? */
-    if (FSE_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2))
-        return op-ostart;
-
-    if (op==omax) return (size_t)-FSE_ERROR_dstSize_tooSmall;   /* dst buffer is full, but cSrc unfinished */
-
-    return (size_t)-FSE_ERROR_corruptionDetected;
-}
-
-
-static size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));   /* memcpy() into local variable, to avoid strict aliasing warning */
-
-    /* select fast mode (static) */
-    if (DTableH.fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-static size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSE_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-    size_t errorCode;
-
-    if (cSrcSize<2) return (size_t)-FSE_ERROR_srcSize_wrong;   /* too small input size */
-
-    /* normal FSE decoding mode */
-    errorCode = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSE_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong;   /* too small input size */
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    errorCode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
-    if (FSE_isError(errorCode)) return errorCode;
-
-    /* always return, even if it is an error code */
-    return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);
-}
-
-
-
-/* *******************************************************
-*  Huff0 : Huffman block compression
-*********************************************************/
-#define HUF_MAX_SYMBOL_VALUE 255
-#define HUF_DEFAULT_TABLELOG  12       /* used by default, when not specified */
-#define HUF_MAX_TABLELOG  12           /* max possible tableLog; for allocation purpose; can be modified */
-#define HUF_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#if (HUF_MAX_TABLELOG > HUF_ABSOLUTEMAX_TABLELOG)
-#  error "HUF_MAX_TABLELOG is too large !"
-#endif
-
-typedef struct HUF_CElt_s {
-  U16  val;
-  BYTE nbBits;
-} HUF_CElt ;
-
-typedef struct nodeElt_s {
-    U32 count;
-    U16 parent;
-    BYTE byte;
-    BYTE nbBits;
-} nodeElt;
-
-
-/* *******************************************************
-*  Huff0 : Huffman block decompression
-*********************************************************/
-typedef struct {
-    BYTE byte;
-    BYTE nbBits;
-} HUF_DElt;
-
-static size_t HUF_readDTable (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];  /* large enough for values from 0 to 16 */
-    U32 weightTotal;
-    U32 maxBits;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-    U32 n;
-    U32 nextRankStart;
-    void* ptr = DTable+1;
-    HUF_DElt* const dt = (HUF_DElt*)ptr;
-
-    if (!srcSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-    iSize = ip[0];
-
-    FSE_STATIC_ASSERT(sizeof(HUF_DElt) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* should not be necessary, but some analyzer complain ... */
-    if (iSize >= 128)  /* special header */
-    {
-        if (iSize >= (242))   /* RLE */
-        {
-            static int l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, sizeof(huffWeight));
-            iSize = 0;
-        }
-        else   /* Incompressible */
-        {
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-            ip += 1;
-            for (n=0; n<oSize; n+=2)
-            {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-            }
-        }
-    }
-    else  /* header compressed with FSE (normal case) */
-    {
-        if (iSize+1 > srcSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-        oSize = FSE_decompress(huffWeight, HUF_MAX_SYMBOL_VALUE, ip+1, iSize);   /* max 255 values decoded, last one is implied */
-        if (FSE_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankVal, 0, sizeof(rankVal));
-    weightTotal = 0;
-    for (n=0; n<oSize; n++)
-    {
-        if (huffWeight[n] >= HUF_ABSOLUTEMAX_TABLELOG) return (size_t)-FSE_ERROR_corruptionDetected;
-        rankVal[huffWeight[n]]++;
-        weightTotal += (1 << huffWeight[n]) >> 1;
-    }
-    if (weightTotal == 0) return (size_t)-FSE_ERROR_corruptionDetected;
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    maxBits = FSE_highbit32(weightTotal) + 1;
-    if (maxBits > DTable[0]) return (size_t)-FSE_ERROR_tableLog_tooLarge;   /* DTable is too small */
-    DTable[0] = (U16)maxBits;
-    {
-        U32 total = 1 << maxBits;
-        U32 rest = total - weightTotal;
-        U32 verif = 1 << FSE_highbit32(rest);
-        U32 lastWeight = FSE_highbit32(rest) + 1;
-        if (verif != rest) return (size_t)-FSE_ERROR_corruptionDetected;    /* last value must be a clean power of 2 */
-        huffWeight[oSize] = (BYTE)lastWeight;
-        rankVal[lastWeight]++;
-    }
-
-    /* check tree construction validity */
-    if ((rankVal[1] < 2) || (rankVal[1] & 1)) return (size_t)-FSE_ERROR_corruptionDetected;   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<=maxBits; n++)
-    {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<=oSize; n++)
-    {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUF_DElt D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(maxBits + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize+1;
-}
-
-
-static BYTE HUF_decodeSymbol(FSE_DStream_t* Dstream, const HUF_DElt* dt, const U32 dtLog)
-{
-        const size_t val = FSE_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-        const BYTE c = dt[val].byte;
-        FSE_skipBits(Dstream, dt[val].nbBits);
-        return c;
-}
-
-static size_t HUF_decompress_usingDTable(   /* -3% slower when non static */
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    if (cSrcSize < 6) return (size_t)-FSE_ERROR_srcSize_wrong;
-    {
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* op = ostart;
-        BYTE* const omax = op + maxDstSize;
-        BYTE* const olimit = omax-15;
-
-        const void* ptr = DTable;
-        const HUF_DElt* const dt = (const HUF_DElt*)(ptr)+1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-        U32 reloadStatus;
-
-        /* Init */
-
-        const U16* jumpTable = (const U16*)cSrc;
-        const size_t length1 = FSE_readLE16(jumpTable);
-        const size_t length2 = FSE_readLE16(jumpTable+1);
-        const size_t length3 = FSE_readLE16(jumpTable+2);
-        const size_t length4 = cSrcSize - 6 - length1 - length2 - length3;   // check coherency !!
-        const char* const start1 = (const char*)(cSrc) + 6;
-        const char* const start2 = start1 + length1;
-        const char* const start3 = start2 + length2;
-        const char* const start4 = start3 + length3;
-        FSE_DStream_t bitD1, bitD2, bitD3, bitD4;
-
-        if (length1+length2+length3+6 >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-
-        errorCode = FSE_initDStream(&bitD1, start1, length1);
-        if (FSE_isError(errorCode)) return errorCode;
-        errorCode = FSE_initDStream(&bitD2, start2, length2);
-        if (FSE_isError(errorCode)) return errorCode;
-        errorCode = FSE_initDStream(&bitD3, start3, length3);
-        if (FSE_isError(errorCode)) return errorCode;
-        errorCode = FSE_initDStream(&bitD4, start4, length4);
-        if (FSE_isError(errorCode)) return errorCode;
-
-        reloadStatus=FSE_reloadDStream(&bitD2);
-
-        /* 16 symbols per loop */
-        for ( ; (reloadStatus<FSE_DStream_completed) && (op<olimit);  /* D2-3-4 are supposed to be synchronized and finish together */
-            op+=16, reloadStatus = FSE_reloadDStream(&bitD2) | FSE_reloadDStream(&bitD3) | FSE_reloadDStream(&bitD4), FSE_reloadDStream(&bitD1))
-        {
-    #define HUF_DECODE_SYMBOL_0(n, Dstream) \
-            op[n] = HUF_decodeSymbol(&Dstream, dt, dtLog);
-
-    #define HUF_DECODE_SYMBOL_1(n, Dstream) \
-            op[n] = HUF_decodeSymbol(&Dstream, dt, dtLog); \
-            if (FSE_32bits() && (HUF_MAX_TABLELOG>12)) FSE_reloadDStream(&Dstream)
-
-    #define HUF_DECODE_SYMBOL_2(n, Dstream) \
-            op[n] = HUF_decodeSymbol(&Dstream, dt, dtLog); \
-            if (FSE_32bits()) FSE_reloadDStream(&Dstream)
-
-            HUF_DECODE_SYMBOL_1( 0, bitD1);
-            HUF_DECODE_SYMBOL_1( 1, bitD2);
-            HUF_DECODE_SYMBOL_1( 2, bitD3);
-            HUF_DECODE_SYMBOL_1( 3, bitD4);
-            HUF_DECODE_SYMBOL_2( 4, bitD1);
-            HUF_DECODE_SYMBOL_2( 5, bitD2);
-            HUF_DECODE_SYMBOL_2( 6, bitD3);
-            HUF_DECODE_SYMBOL_2( 7, bitD4);
-            HUF_DECODE_SYMBOL_1( 8, bitD1);
-            HUF_DECODE_SYMBOL_1( 9, bitD2);
-            HUF_DECODE_SYMBOL_1(10, bitD3);
-            HUF_DECODE_SYMBOL_1(11, bitD4);
-            HUF_DECODE_SYMBOL_0(12, bitD1);
-            HUF_DECODE_SYMBOL_0(13, bitD2);
-            HUF_DECODE_SYMBOL_0(14, bitD3);
-            HUF_DECODE_SYMBOL_0(15, bitD4);
-        }
-
-        if (reloadStatus!=FSE_DStream_completed)   /* not complete : some bitStream might be FSE_DStream_unfinished */
-            return (size_t)-FSE_ERROR_corruptionDetected;
-
-        /* tail */
-        {
-            // bitTail = bitD1;   // *much* slower : -20% !??!
-            FSE_DStream_t bitTail;
-            bitTail.ptr = bitD1.ptr;
-            bitTail.bitsConsumed = bitD1.bitsConsumed;
-            bitTail.bitContainer = bitD1.bitContainer;   // required in case of FSE_DStream_endOfBuffer
-            bitTail.start = start1;
-            for ( ; (FSE_reloadDStream(&bitTail) < FSE_DStream_completed) && (op<omax) ; op++)
-            {
-                HUF_DECODE_SYMBOL_0(0, bitTail);
-            }
-
-            if (FSE_endOfDStream(&bitTail))
-                return op-ostart;
-        }
-
-        if (op==omax) return (size_t)-FSE_ERROR_dstSize_tooSmall;   /* dst buffer is full, but cSrc unfinished */
-
-        return (size_t)-FSE_ERROR_corruptionDetected;
-    }
-}
-
-
-static size_t HUF_decompress (void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLE(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUF_readDTable (DTable, cSrc, cSrcSize);
-    if (FSE_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong;
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUF_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, DTable);
-}
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
-
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/****************************************************************
-*  Tuning parameters
-*****************************************************************/
-/* MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect */
-#define ZSTD_MEMORY_USAGE 17
-
-
-/**************************************
-   CPU Feature Detection
-**************************************/
-/*
- * Automated efficient unaligned memory access detection
- * Based on known hardware architectures
- * This list will be updated thanks to feedbacks
- */
-#if defined(CPU_HAS_EFFICIENT_UNALIGNED_MEMORY_ACCESS) \
-    || defined(__ARM_FEATURE_UNALIGNED) \
-    || defined(__i386__) || defined(__x86_64__) \
-    || defined(_M_IX86) || defined(_M_X64) \
-    || defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_8__) \
-    || (defined(_M_ARM) && (_M_ARM >= 7))
-#  define ZSTD_UNALIGNED_ACCESS 1
-#else
-#  define ZSTD_UNALIGNED_ACCESS 0
-#endif
-
-
-/********************************************************
-*  Includes
-*********************************************************/
-#include <stdlib.h>      /* calloc */
-#include <string.h>      /* memcpy, memmove */
-#include <stdio.h>       /* debug : printf */
-
-
-/********************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef __AVX2__
-#  include <immintrin.h>   /* AVX2 intrinsics */
-#endif
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-
-#ifndef MEM_ACCESS_MODULE
-#define MEM_ACCESS_MODULE
-/********************************************************
-*  Basic Types
-*********************************************************/
-#if defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-# include <stdint.h>
-typedef  uint8_t BYTE;
-typedef uint16_t U16;
-typedef  int16_t S16;
-typedef uint32_t U32;
-typedef  int32_t S32;
-typedef uint64_t U64;
-#else
-typedef unsigned char       BYTE;
-typedef unsigned short      U16;
-typedef   signed short      S16;
-typedef unsigned int        U32;
-typedef   signed int        S32;
-typedef unsigned long long  U64;
-#endif
-
-#endif   /* MEM_ACCESS_MODULE */
-
-
-/********************************************************
-*  Constants
-*********************************************************/
-static const U32 ZSTD_magicNumber = 0xFD2FB51E;   /* 3rd version : seqNb header */
-
-#define HASH_LOG (ZSTD_MEMORY_USAGE - 2)
-#define HASH_TABLESIZE (1 << HASH_LOG)
-#define HASH_MASK (HASH_TABLESIZE - 1)
-
-#define KNUTH 2654435761
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BLOCKSIZE (128 KB)                 /* define, for static allocation */
-
-#define WORKPLACESIZE (BLOCKSIZE*3)
-#define MINMATCH 4
-#define MLbits   7
-#define LLbits   6
-#define Offbits  5
-#define MaxML  ((1<<MLbits )-1)
-#define MaxLL  ((1<<LLbits )-1)
-#define MaxOff ((1<<Offbits)-1)
-#define LitFSELog  11
-#define MLFSELog   10
-#define LLFSELog   10
-#define OffFSELog   9
-#define MAX(a,b) ((a)<(b)?(b):(a))
-#define MaxSeq MAX(MaxLL, MaxML)
-
-#define LITERAL_NOENTROPY 63
-#define COMMAND_NOENTROPY 7   /* to remove */
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-static const size_t ZSTD_blockHeaderSize = 3;
-static const size_t ZSTD_frameHeaderSize = 4;
-
-
-/********************************************************
-*  Memory operations
-*********************************************************/
-static unsigned ZSTD_32bits(void) { return sizeof(void*)==4; }
-
-static unsigned ZSTD_isLittleEndian(void)
-{
-    const union { U32 i; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-static U16    ZSTD_read16(const void* p) { U16 r; memcpy(&r, p, sizeof(r)); return r; }
-
-static void   ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-static void   ZSTD_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s)    { ZSTD_copy8(d,s); d+=8; s+=8; }
-
-static void ZSTD_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    while (op < oend) COPY8(op, ip);
-}
-
-static U16 ZSTD_readLE16(const void* memPtr)
-{
-    if (ZSTD_isLittleEndian()) return ZSTD_read16(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)((U16)p[0] + ((U16)p[1]<<8));
-    }
-}
-
-static U32 ZSTD_readLE24(const void* memPtr)
-{
-    return ZSTD_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-static U32 ZSTD_readBE32(const void* memPtr)
-{
-    const BYTE* p = (const BYTE*)memPtr;
-    return (U32)(((U32)p[0]<<24) + ((U32)p[1]<<16) + ((U32)p[2]<<8) + ((U32)p[3]<<0));
-}
-
-
-/**************************************
-*  Local structures
-***************************************/
-typedef struct ZSTD_Cctx_s ZSTD_Cctx;
-
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* offCode;
-    BYTE* litStart;
-    BYTE* lit;
-    BYTE* litLengthStart;
-    BYTE* litLength;
-    BYTE* matchLengthStart;
-    BYTE* matchLength;
-    BYTE* dumpsStart;
-    BYTE* dumps;
-} seqStore_t;
-
-
-typedef struct ZSTD_Cctx_s
-{
-    const BYTE* base;
-    U32 current;
-    U32 nextUpdate;
-    seqStore_t seqStore;
-#ifdef __AVX2__
-    __m256i hashTable[HASH_TABLESIZE>>3];
-#else
-    U32 hashTable[HASH_TABLESIZE];
-#endif
-    BYTE buffer[WORKPLACESIZE];
-} cctxi_t;
-
-
-
-
-/**************************************
-*  Error Management
-**************************************/
-/* published entry point */
-unsigned ZSTDv01_isError(size_t code) { return ERR_isError(code); }
-
-
-/**************************************
-*  Tool functions
-**************************************/
-#define ZSTD_VERSION_MAJOR    0    /* for breaking interface changes  */
-#define ZSTD_VERSION_MINOR    1    /* for new (non-breaking) interface capabilities */
-#define ZSTD_VERSION_RELEASE  3    /* for tweaks, bug-fixes, or development */
-#define ZSTD_VERSION_NUMBER  (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
-
-/**************************************************************
-*   Decompression code
-**************************************************************/
-
-static size_t ZSTDv01_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    BYTE headerFlags;
-    U32 cSize;
-
-    if (srcSize < 3) return ERROR(srcSize_wrong);
-
-    headerFlags = *in;
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-
-    bpPtr->blockType = (blockType_t)(headerFlags >> 6);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-
-static size_t ZSTD_copyUncompressedBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    if (srcSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-static size_t ZSTD_decompressLiterals(void* ctx,
-                                      void* dst, size_t maxDstSize,
-                                const void* src, size_t srcSize)
-{
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + maxDstSize;
-    const BYTE* ip = (const BYTE*)src;
-    size_t errorCode;
-    size_t litSize;
-
-    /* check : minimum 2, for litSize, +1, for content */
-    if (srcSize <= 3) return ERROR(corruption_detected);
-
-    litSize = ip[1] + (ip[0]<<8);
-    litSize += ((ip[-3] >> 3) & 7) << 16;   // mmmmh....
-    op = oend - litSize;
-
-    (void)ctx;
-    if (litSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    errorCode = HUF_decompress(op, litSize, ip+2, srcSize-2);
-    if (FSE_isError(errorCode)) return ERROR(GENERIC);
-    return litSize;
-}
-
-
-static size_t ZSTDv01_decodeLiteralsBlock(void* ctx,
-                                void* dst, size_t maxDstSize,
-                          const BYTE** litStart, size_t* litSize,
-                          const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + maxDstSize;
-    blockProperties_t litbp;
-
-    size_t litcSize = ZSTDv01_getcBlockSize(src, srcSize, &litbp);
-    if (ZSTDv01_isError(litcSize)) return litcSize;
-    if (litcSize > srcSize - ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-    ip += ZSTD_blockHeaderSize;
-
-    switch(litbp.blockType)
-    {
-    case bt_raw:
-        *litStart = ip;
-        ip += litcSize;
-        *litSize = litcSize;
-        break;
-    case bt_rle:
-        {
-            size_t rleSize = litbp.origSize;
-            if (rleSize>maxDstSize) return ERROR(dstSize_tooSmall);
-            if (!srcSize) return ERROR(srcSize_wrong);
-            memset(oend - rleSize, *ip, rleSize);
-            *litStart = oend - rleSize;
-            *litSize = rleSize;
-            ip++;
-            break;
-        }
-    case bt_compressed:
-        {
-            size_t decodedLitSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize);
-            if (ZSTDv01_isError(decodedLitSize)) return decodedLitSize;
-            *litStart = oend - decodedLitSize;
-            *litSize = decodedLitSize;
-            ip += litcSize;
-            break;
-        }
-    case bt_end:
-    default:
-        return ERROR(GENERIC);
-    }
-
-    return ip-istart;
-}
-
-
-static size_t ZSTDv01_decodeSeqHeaders(int* nbSeq, const BYTE** dumpsPtr, size_t* dumpsLengthPtr,
-                         FSE_DTable* DTableLL, FSE_DTable* DTableML, FSE_DTable* DTableOffb,
-                         const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    const BYTE* const iend = istart + srcSize;
-    U32 LLtype, Offtype, MLtype;
-    U32 LLlog, Offlog, MLlog;
-    size_t dumpsLength;
-
-    /* check */
-    if (srcSize < 5) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    *nbSeq = ZSTD_readLE16(ip); ip+=2;
-    LLtype  = *ip >> 6;
-    Offtype = (*ip >> 4) & 3;
-    MLtype  = (*ip >> 2) & 3;
-    if (*ip & 2)
-    {
-        dumpsLength  = ip[2];
-        dumpsLength += ip[1] << 8;
-        ip += 3;
-    }
-    else
-    {
-        dumpsLength  = ip[1];
-        dumpsLength += (ip[0] & 1) << 8;
-        ip += 2;
-    }
-    *dumpsPtr = ip;
-    ip += dumpsLength;
-    *dumpsLengthPtr = dumpsLength;
-
-    /* check */
-    if (ip > iend-3) return ERROR(srcSize_wrong); /* min : all 3 are "raw", hence no header, but at least xxLog bits per type */
-
-    /* sequences */
-    {
-        S16 norm[MaxML+1];    /* assumption : MaxML >= MaxLL and MaxOff */
-        size_t headerSize;
-
-        /* Build DTables */
-        switch(LLtype)
-        {
-        case bt_rle :
-            LLlog = 0;
-            FSE_buildDTable_rle(DTableLL, *ip++); break;
-        case bt_raw :
-            LLlog = LLbits;
-            FSE_buildDTable_raw(DTableLL, LLbits); break;
-        default :
-            {   U32 max = MaxLL;
-                headerSize = FSE_readNCount(norm, &max, &LLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (LLlog > LLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableLL, norm, max, LLlog);
-        }   }
-
-        switch(Offtype)
-        {
-        case bt_rle :
-            Offlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableOffb, *ip++); break;
-        case bt_raw :
-            Offlog = Offbits;
-            FSE_buildDTable_raw(DTableOffb, Offbits); break;
-        default :
-            {   U32 max = MaxOff;
-                headerSize = FSE_readNCount(norm, &max, &Offlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (Offlog > OffFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableOffb, norm, max, Offlog);
-        }   }
-
-        switch(MLtype)
-        {
-        case bt_rle :
-            MLlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableML, *ip++); break;
-        case bt_raw :
-            MLlog = MLbits;
-            FSE_buildDTable_raw(DTableML, MLbits); break;
-        default :
-            {   U32 max = MaxML;
-                headerSize = FSE_readNCount(norm, &max, &MLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (MLlog > MLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableML, norm, max, MLlog);
-    }   }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t offset;
-    size_t matchLength;
-} seq_t;
-
-typedef struct {
-    FSE_DStream_t DStream;
-    FSE_DState_t stateLL;
-    FSE_DState_t stateOffb;
-    FSE_DState_t stateML;
-    size_t prevOffset;
-    const BYTE* dumps;
-    const BYTE* dumpsEnd;
-} seqState_t;
-
-
-static void ZSTD_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    size_t litLength;
-    size_t prevOffset;
-    size_t offset;
-    size_t matchLength;
-    const BYTE* dumps = seqState->dumps;
-    const BYTE* const de = seqState->dumpsEnd;
-
-    /* Literal length */
-    litLength = FSE_decodeSymbol(&(seqState->stateLL), &(seqState->DStream));
-    prevOffset = litLength ? seq->offset : seqState->prevOffset;
-    seqState->prevOffset = seq->offset;
-    if (litLength == MaxLL)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) litLength += add;
-        else
-        {
-            if (dumps<=(de-3))
-            {
-                litLength = ZSTD_readLE24(dumps);
-                dumps += 3;
-            }
-        }
-    }
-
-    /* Offset */
-    {
-        U32 offsetCode, nbBits;
-        offsetCode = FSE_decodeSymbol(&(seqState->stateOffb), &(seqState->DStream));
-        if (ZSTD_32bits()) FSE_reloadDStream(&(seqState->DStream));
-        nbBits = offsetCode - 1;
-        if (offsetCode==0) nbBits = 0;   /* cmove */
-        offset = ((size_t)1 << (nbBits & ((sizeof(offset)*8)-1))) + FSE_readBits(&(seqState->DStream), nbBits);
-        if (ZSTD_32bits()) FSE_reloadDStream(&(seqState->DStream));
-        if (offsetCode==0) offset = prevOffset;
-    }
-
-    /* MatchLength */
-    matchLength = FSE_decodeSymbol(&(seqState->stateML), &(seqState->DStream));
-    if (matchLength == MaxML)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) matchLength += add;
-        else
-        {
-            if (dumps<=(de-3))
-            {
-                matchLength = ZSTD_readLE24(dumps);
-                dumps += 3;
-            }
-        }
-    }
-    matchLength += MINMATCH;
-
-    /* save result */
-    seq->litLength = litLength;
-    seq->offset = offset;
-    seq->matchLength = matchLength;
-    seqState->dumps = dumps;
-}
-
-
-static size_t ZSTD_execSequence(BYTE* op,
-                                seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                BYTE* const base, BYTE* const oend)
-{
-    static const int dec32table[] = {0, 1, 2, 1, 4, 4, 4, 4};   /* added */
-    static const int dec64table[] = {8, 8, 8, 7, 8, 9,10,11};   /* subtracted */
-    const BYTE* const ostart = op;
-    const size_t litLength = sequence.litLength;
-    BYTE* const endMatch = op + litLength + sequence.matchLength;    /* risk : address space overflow (32-bits) */
-    const BYTE* const litEnd = *litPtr + litLength;
-
-    /* check */
-    if (endMatch > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (litEnd > litLimit) return ERROR(corruption_detected);
-    if (sequence.matchLength > (size_t)(*litPtr-op))  return ERROR(dstSize_tooSmall);    /* overwrite literal segment */
-
-    /* copy Literals */
-    if (((size_t)(*litPtr - op) < 8) || ((size_t)(oend-litEnd) < 8) || (op+litLength > oend-8))
-        memmove(op, *litPtr, litLength);   /* overwrite risk */
-    else
-        ZSTD_wildcopy(op, *litPtr, litLength);
-    op += litLength;
-    *litPtr = litEnd;   /* update for next sequence */
-
-    /* check : last match must be at a minimum distance of 8 from end of dest buffer */
-    if (oend-op < 8) return ERROR(dstSize_tooSmall);
-
-    /* copy Match */
-    {
-        const U32 overlapRisk = (((size_t)(litEnd - endMatch)) < 12);
-        const BYTE* match = op - sequence.offset;            /* possible underflow at op - offset ? */
-        size_t qutt = 12;
-        U64 saved[2];
-
-        /* check */
-        if (match < base) return ERROR(corruption_detected);
-        if (sequence.offset > (size_t)base) return ERROR(corruption_detected);
-
-        /* save beginning of literal sequence, in case of write overlap */
-        if (overlapRisk)
-        {
-            if ((endMatch + qutt) > oend) qutt = oend-endMatch;
-            memcpy(saved, endMatch, qutt);
-        }
-
-        if (sequence.offset < 8)
-        {
-            const int dec64 = dec64table[sequence.offset];
-            op[0] = match[0];
-            op[1] = match[1];
-            op[2] = match[2];
-            op[3] = match[3];
-            match += dec32table[sequence.offset];
-            ZSTD_copy4(op+4, match);
-            match -= dec64;
-        } else { ZSTD_copy8(op, match); }
-        op += 8; match += 8;
-
-        if (endMatch > oend-(16-MINMATCH))
-        {
-            if (op < oend-8)
-            {
-                ZSTD_wildcopy(op, match, (oend-8) - op);
-                match += (oend-8) - op;
-                op = oend-8;
-            }
-            while (op<endMatch) *op++ = *match++;
-        }
-        else
-            ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-
-        /* restore, in case of overlap */
-        if (overlapRisk) memcpy(endMatch, saved, qutt);
-    }
-
-    return endMatch-ostart;
-}
-
-typedef struct ZSTDv01_Dctx_s
-{
-    U32 LLTable[FSE_DTABLE_SIZE_U32(LLFSELog)];
-    U32 OffTable[FSE_DTABLE_SIZE_U32(OffFSELog)];
-    U32 MLTable[FSE_DTABLE_SIZE_U32(MLFSELog)];
-    void* previousDstEnd;
-    void* base;
-    size_t expected;
-    blockType_t bType;
-    U32 phase;
-} dctx_t;
-
-
-static size_t ZSTD_decompressSequences(
-                               void* ctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize,
-                         const BYTE* litStart, size_t litSize)
-{
-    dctx_t* dctx = (dctx_t*)ctx;
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t errorCode, dumpsLength;
-    const BYTE* litPtr = litStart;
-    const BYTE* const litEnd = litStart + litSize;
-    int nbSeq;
-    const BYTE* dumps;
-    U32* DTableLL = dctx->LLTable;
-    U32* DTableML = dctx->MLTable;
-    U32* DTableOffb = dctx->OffTable;
-    BYTE* const base = (BYTE*) (dctx->base);
-
-    /* Build Decoding Tables */
-    errorCode = ZSTDv01_decodeSeqHeaders(&nbSeq, &dumps, &dumpsLength,
-                                      DTableLL, DTableML, DTableOffb,
-                                      ip, iend-ip);
-    if (ZSTDv01_isError(errorCode)) return errorCode;
-    ip += errorCode;
-
-    /* Regen sequences */
-    {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        seqState.dumps = dumps;
-        seqState.dumpsEnd = dumps + dumpsLength;
-        seqState.prevOffset = 1;
-        errorCode = FSE_initDStream(&(seqState.DStream), ip, iend-ip);
-        if (FSE_isError(errorCode)) return ERROR(corruption_detected);
-        FSE_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSE_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSE_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (FSE_reloadDStream(&(seqState.DStream)) <= FSE_DStream_completed) && (nbSeq>0) ; )
-        {
-            size_t oneSeqSize;
-            nbSeq--;
-            ZSTD_decodeSequence(&sequence, &seqState);
-            oneSeqSize = ZSTD_execSequence(op, sequence, &litPtr, litEnd, base, oend);
-            if (ZSTDv01_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* check if reached exact end */
-        if ( !FSE_endOfDStream(&(seqState.DStream)) ) return ERROR(corruption_detected);   /* requested too much : data is corrupted */
-        if (nbSeq<0) return ERROR(corruption_detected);   /* requested too many sequences : data is corrupted */
-
-        /* last literal segment */
-        {
-            size_t lastLLSize = litEnd - litPtr;
-            if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-            if (op != litPtr) memmove(op, litPtr, lastLLSize);
-            op += lastLLSize;
-        }
-    }
-
-    return op-ostart;
-}
-
-
-static size_t ZSTD_decompressBlock(
-                            void* ctx,
-                            void* dst, size_t maxDstSize,
-                      const void* src, size_t srcSize)
-{
-    /* blockType == blockCompressed, srcSize is trusted */
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* litPtr = NULL;
-    size_t litSize = 0;
-    size_t errorCode;
-
-    /* Decode literals sub-block */
-    errorCode = ZSTDv01_decodeLiteralsBlock(ctx, dst, maxDstSize, &litPtr, &litSize, src, srcSize);
-    if (ZSTDv01_isError(errorCode)) return errorCode;
-    ip += errorCode;
-    srcSize -= errorCode;
-
-    return ZSTD_decompressSequences(ctx, dst, maxDstSize, ip, srcSize, litPtr, litSize);
-}
-
-
-size_t ZSTDv01_decompressDCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t remainingSize = srcSize;
-    U32 magicNumber;
-    size_t errorCode=0;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-    magicNumber = ZSTD_readBE32(src);
-    if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t blockSize = ZSTDv01_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTDv01_isError(blockSize)) return blockSize;
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (blockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            errorCode = ZSTD_decompressBlock(ctx, op, oend-op, ip, blockSize);
-            break;
-        case bt_raw :
-            errorCode = ZSTD_copyUncompressedBlock(op, oend-op, ip, blockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);
-        }
-        if (blockSize == 0) break;   /* bt_end */
-
-        if (ZSTDv01_isError(errorCode)) return errorCode;
-        op += errorCode;
-        ip += blockSize;
-        remainingSize -= blockSize;
-    }
-
-    return op-ostart;
-}
-
-size_t ZSTDv01_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    dctx_t ctx;
-    ctx.base = dst;
-    return ZSTDv01_decompressDCtx(&ctx, dst, maxDstSize, src, srcSize);
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv01_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    U32 magicNumber;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-    magicNumber = ZSTD_readBE32(src);
-    if (magicNumber != ZSTD_magicNumber) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-        return;
-    }
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t blockSize = ZSTDv01_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTDv01_isError(blockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, blockSize);
-            return;
-        }
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (blockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (blockSize == 0) break;   /* bt_end */
-
-        ip += blockSize;
-        remainingSize -= blockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * BLOCKSIZE;
-}
-
-/*******************************
-*  Streaming Decompression API
-*******************************/
-
-size_t ZSTDv01_resetDCtx(ZSTDv01_Dctx* dctx)
-{
-    dctx->expected = ZSTD_frameHeaderSize;
-    dctx->phase = 0;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    return 0;
-}
-
-ZSTDv01_Dctx* ZSTDv01_createDCtx(void)
-{
-    ZSTDv01_Dctx* dctx = (ZSTDv01_Dctx*)malloc(sizeof(ZSTDv01_Dctx));
-    if (dctx==NULL) return NULL;
-    ZSTDv01_resetDCtx(dctx);
-    return dctx;
-}
-
-size_t ZSTDv01_freeDCtx(ZSTDv01_Dctx* dctx)
-{
-    free(dctx);
-    return 0;
-}
-
-size_t ZSTDv01_nextSrcSizeToDecompress(ZSTDv01_Dctx* dctx)
-{
-    return ((dctx_t*)dctx)->expected;
-}
-
-size_t ZSTDv01_decompressContinue(ZSTDv01_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    dctx_t* ctx = (dctx_t*)dctx;
-
-    /* Sanity check */
-    if (srcSize != ctx->expected) return ERROR(srcSize_wrong);
-    if (dst != ctx->previousDstEnd)  /* not contiguous */
-        ctx->base = dst;
-
-    /* Decompress : frame header */
-    if (ctx->phase == 0)
-    {
-        /* Check frame magic header */
-        U32 magicNumber = ZSTD_readBE32(src);
-        if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        return 0;
-    }
-
-    /* Decompress : block header */
-    if (ctx->phase == 1)
-    {
-        blockProperties_t bp;
-        size_t blockSize = ZSTDv01_getcBlockSize(src, ZSTD_blockHeaderSize, &bp);
-        if (ZSTDv01_isError(blockSize)) return blockSize;
-        if (bp.blockType == bt_end)
-        {
-            ctx->expected = 0;
-            ctx->phase = 0;
-        }
-        else
-        {
-            ctx->expected = blockSize;
-            ctx->bType = bp.blockType;
-            ctx->phase = 2;
-        }
-
-        return 0;
-    }
-
-    /* Decompress : block content */
-    {
-        size_t rSize;
-        switch(ctx->bType)
-        {
-        case bt_compressed:
-            rSize = ZSTD_decompressBlock(ctx, dst, maxDstSize, src, srcSize);
-            break;
-        case bt_raw :
-            rSize = ZSTD_copyUncompressedBlock(dst, maxDstSize, src, srcSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet handled */
-            break;
-        case bt_end :   /* should never happen (filtered at phase 1) */
-            rSize = 0;
-            break;
-        default:
-            return ERROR(GENERIC);
-        }
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        ctx->previousDstEnd = (void*)( ((char*)dst) + rSize);
-        return rSize;
-    }
-
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_v01.h b/vendor/github.com/DataDog/zstd/zstd_v01.h
deleted file mode 100644
index 245f9dd..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v01.h
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_V01_H_28739879432
-#define ZSTD_V01_H_28739879432
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Simple one-step function
-***************************************/
-/**
-ZSTDv01_decompress() : decompress ZSTD frames compliant with v0.1.x format
-    compressedSize : is the exact source size
-    maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
-                      It must be equal or larger than originalSize, otherwise decompression will fail.
-    return : the number of bytes decompressed into destination buffer (originalSize)
-             or an errorCode if it fails (which can be tested using ZSTDv01_isError())
-*/
-size_t ZSTDv01_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize);
-
- /**
- ZSTDv01_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.1.x format
-     srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-     cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                 or an error code if it fails (which can be tested using ZSTDv01_isError())
-     dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                 or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-     note : assumes `cSize` and `dBound` are _not_ NULL.
- */
-void ZSTDv01_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                     size_t* cSize, unsigned long long* dBound);
-
-/**
-ZSTDv01_isError() : tells if the result of ZSTDv01_decompress() is an error
-*/
-unsigned ZSTDv01_isError(size_t code);
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTDv01_Dctx_s ZSTDv01_Dctx;
-ZSTDv01_Dctx* ZSTDv01_createDCtx(void);
-size_t ZSTDv01_freeDCtx(ZSTDv01_Dctx* dctx);
-
-size_t ZSTDv01_decompressDCtx(void* ctx,
-                              void* dst, size_t maxOriginalSize,
-                        const void* src, size_t compressedSize);
-
-/* *************************************
-*  Streaming functions
-***************************************/
-size_t ZSTDv01_resetDCtx(ZSTDv01_Dctx* dctx);
-
-size_t ZSTDv01_nextSrcSizeToDecompress(ZSTDv01_Dctx* dctx);
-size_t ZSTDv01_decompressContinue(ZSTDv01_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
-/**
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTDv01_magicNumber   0xFD2FB51E   /* Big Endian version */
-#define ZSTDv01_magicNumberLE 0x1EB52FFD   /* Little Endian version */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_V01_H_28739879432 */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v02.c b/vendor/github.com/DataDog/zstd/zstd_v02.c
deleted file mode 100644
index 793df60..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v02.c
+++ /dev/null
@@ -1,3513 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include "zstd_v02.h"
-#include "error_private.h"
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-
-
-/* ******************************************************************
-   mem.h
-   low-level memory access routines
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/******************************************
-*  Includes
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include <string.h>    /* memcpy */
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/****************************************************************
-*  Basic Types
-*****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/****************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets generating assembly depending on alignment.
- *            But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(void*)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(void*)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-#endif // MEM_FORCE_MEMORY_ACCESS
-
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian())
-    {
-        MEM_write16(memPtr, val);
-    }
-    else
-    {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE24(const void* memPtr)
-{
-    return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24));
-    }
-}
-
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24)
-                     + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56));
-    }
-}
-
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-
-
-/* ******************************************************************
-   bitstream
-   Part of NewGen Entropy library
-   header file (to include)
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which highly benefit from being inlined.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-
-/**********************************************
-*  bitStream decompression API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BIT_DStream_t;
-
-typedef enum { BIT_DStream_unfinished = 0,
-               BIT_DStream_endOfBuffer = 1,
-               BIT_DStream_completed = 2,
-               BIT_DStream_overflow = 3 } BIT_DStream_status;  /* result of BIT_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD);
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD);
-
-
-/******************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/****************************************************************
-*  Helper functions
-****************************************************************/
-MEM_STATIC unsigned BIT_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-
-/**********************************************************
-* bitStream decoding
-**********************************************************/
-
-/*!BIT_initDStream
-*  Initialize a BIT_DStream_t.
-*  @bitD : a pointer to an already allocated BIT_DStream_t structure
-*  @srcBuffer must point at the beginning of a bitStream
-*  @srcSize must be the exact size of the bitStream
-*  @result : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(size_t))   /* normal case */
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(size_t);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-    }
-    else
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);
-                    /* fallthrough */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);
-                    /* fallthrough */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);
-                    /* fallthrough */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24;
-                    /* fallthrough */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16;
-                    /* fallthrough */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) <<  8;
-                    /* fallthrough */
-            default:;
-        }
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-        bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-MEM_STATIC size_t BIT_lookBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BIT_lookBitsFast :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_lookBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBits(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*!BIT_readBitsFast :
-*  unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBitsFast(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return BIT_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer))
-    {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BIT_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start)
-    {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer;
-        return BIT_DStream_completed;
-    }
-    {
-        U32 nbBytes = bitD->bitsConsumed >> 3;
-        BIT_DStream_status result = BIT_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start)
-        {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BIT_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BIT_endOfDStream
-*   @return Tells if DStream has reached its exact end
-*/
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
-/* ******************************************************************
-   Error codes and messages
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef ERROR_H_MODULE
-#define ERROR_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define ERR_STATIC static inline
-#elif defined(_MSC_VER)
-#  define ERR_STATIC static __inline
-#elif defined(__GNUC__)
-#  define ERR_STATIC static __attribute__((unused))
-#else
-#  define ERR_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/******************************************
-*  Error Management
-******************************************/
-#define PREFIX(name) ZSTD_error_##name
-
-#define ERROR(name) (size_t)-PREFIX(name)
-
-#define ERROR_LIST(ITEM) \
-        ITEM(PREFIX(No_Error)) ITEM(PREFIX(GENERIC)) \
-        ITEM(PREFIX(dstSize_tooSmall)) ITEM(PREFIX(srcSize_wrong)) \
-        ITEM(PREFIX(prefix_unknown)) ITEM(PREFIX(corruption_detected)) \
-        ITEM(PREFIX(tableLog_tooLarge)) ITEM(PREFIX(maxSymbolValue_tooLarge)) ITEM(PREFIX(maxSymbolValue_tooSmall)) \
-        ITEM(PREFIX(maxCode))
-
-#define ERROR_GENERATE_ENUM(ENUM) ENUM,
-typedef enum { ERROR_LIST(ERROR_GENERATE_ENUM) } ERR_codes;  /* enum is exposed, to detect & handle specific errors; compare function result to -enum value */
-
-#define ERROR_CONVERTTOSTRING(STRING) #STRING,
-#define ERROR_GENERATE_STRING(EXPR) ERROR_CONVERTTOSTRING(EXPR)
-static const char* ERR_strings[] = { ERROR_LIST(ERROR_GENERATE_STRING) };
-
-ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); }
-
-ERR_STATIC const char* ERR_getErrorName(size_t code)
-{
-    static const char* codeError = "Unspecified error code";
-    if (ERR_isError(code)) return ERR_strings[-(int)(code)];
-    return codeError;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ERROR_H_MODULE */
-/*
-Constructor and Destructor of type FSE_CTable
-    Note that its size depends on 'tableLog' and 'maxSymbolValue' */
-typedef unsigned FSE_CTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-typedef unsigned FSE_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-
-
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/******************************************
-*  Static allocation
-******************************************/
-/* FSE buffer bounds */
-#define FSE_NCOUNTBOUND 512
-#define FSE_BLOCKBOUND(size) (size + (size>>7))
-#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* You can statically allocate FSE CTable/DTable as a table of unsigned using below macro */
-#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue)   (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
-#define FSE_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/******************************************
-*  FSE advanced API
-******************************************/
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
-/* build a fake FSE_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
-/* build a fake FSE_DTable, designed to always generate the same symbolValue */
-
-
-/******************************************
-*  FSE symbol decompression API
-******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-
-static void     FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt);
-
-static unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
-
-
-/******************************************
-*  FSE unsafe API
-******************************************/
-static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/******************************************
-*  Implementation of inline functions
-******************************************/
-
-/* decompression */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSE_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSE_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));
-    DStatePtr->state = BIT_readBits(bitD, DTableH.tableLog);
-    BIT_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32  nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32 nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/******************************************
-*  Static allocation macros
-******************************************/
-/* Huff0 buffer bounds */
-#define HUF_CTABLEBOUND 129
-#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8)   /* only true if incompressible pre-filtered with fast heuristic */
-#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* static allocation of Huff0's DTable */
-#define HUF_DTABLE_SIZE(maxTableLog)   (1 + (1<<maxTableLog))  /* nb Cells; use unsigned short for X2, unsigned int for X4 */
-#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        unsigned short DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX6(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog) * 3 / 2] = { maxTableLog }
-
-
-/******************************************
-*  Advanced functions
-******************************************/
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbols decoder */
-static size_t HUF_decompress4X6 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* quad-symbols decoder */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-/*
-    zstd - standard compression library
-    Header File
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Version
-***************************************/
-#define ZSTD_VERSION_MAJOR    0    /* for breaking interface changes  */
-#define ZSTD_VERSION_MINOR    2    /* for new (non-breaking) interface capabilities */
-#define ZSTD_VERSION_RELEASE  2    /* for tweaks, bug-fixes, or development */
-#define ZSTD_VERSION_NUMBER  (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTD_CCtx_s ZSTD_CCtx;   /* incomplete type */
-
-#if defined (__cplusplus)
-}
-#endif
-/*
-    zstd - standard compression library
-    Header File for static linking only
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* The objects defined into this file should be considered experimental.
- * They are not labelled stable, as their prototype may change in the future.
- * You can use them for tests, provide feedback, or if you can endure risk of future changes.
- */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Streaming functions
-***************************************/
-
-typedef struct ZSTD_DCtx_s ZSTD_DCtx;
-
-/*
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTD_magicNumber 0xFD2FB522   /* v0.2 (current)*/
-
-
-#if defined (__cplusplus)
-}
-#endif
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/****************************************************************
-*  Tuning parameters
-****************************************************************/
-/* MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSE_MAX_MEMORY_USAGE 14
-#define FSE_DEFAULT_MEMORY_USAGE 13
-
-/* FSE_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSE_MAX_SYMBOL_VALUE 255
-
-
-/****************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-
-
-/****************************************************************
-*  Byte symbol type
-****************************************************************/
-#endif   /* !FSE_COMMONDEFS_ONLY */
-
-
-/****************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/****************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-/****************************************************************
-*  Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG  (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-#error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-
-/****************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/****************************************************************
-*  Complex types
-****************************************************************/
-typedef U32 DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-
-/****************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-
-#define FSE_DECODE_TYPE FSE_decode_t
-
-static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
-
-static size_t FSE_buildDTable
-(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* ptr = dt+1;
-    FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*)ptr;
-    FSE_DTableHeader DTableH;
-    const U32 tableSize = 1 << tableLog;
-    const U32 tableMask = tableSize-1;
-    const U32 step = FSE_tableStep(tableSize);
-    U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-    U32 position = 0;
-    U32 highThreshold = tableSize-1;
-    const S16 largeLimit= (S16)(1 << (tableLog-1));
-    U32 noLarge = 1;
-    U32 s;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    DTableH.tableLog = (U16)tableLog;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        if (normalizedCounter[s]==-1)
-        {
-            tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
-            symbolNext[s] = 1;
-        }
-        else
-        {
-            if (normalizedCounter[s] >= largeLimit) noLarge=0;
-            symbolNext[s] = normalizedCounter[s];
-        }
-    }
-
-    /* Spread symbols */
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        int i;
-        for (i=0; i<normalizedCounter[s]; i++)
-        {
-            tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
-            position = (position + step) & tableMask;
-            while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }
-    }
-
-    if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-
-    /* Build Decoding table */
-    {
-        U32 i;
-        for (i=0; i<tableSize; i++)
-        {
-            FSE_FUNCTION_TYPE symbol = (FSE_FUNCTION_TYPE)(tableDecode[i].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[i].nbBits = (BYTE) (tableLog - BIT_highbit32 ((U32)nextState) );
-            tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
-        }
-    }
-
-    DTableH.fastMode = (U16)noLarge;
-    memcpy(dt, &DTableH, sizeof(DTableH));   /* memcpy(), to avoid strict aliasing warnings */
-    return 0;
-}
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-/******************************************
-*  FSE helper functions
-******************************************/
-static unsigned FSE_isError(size_t code) { return ERR_isError(code); }
-
-
-/****************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-static short FSE_abs(short a)
-{
-    return (short)(a<0 ? -a : a);
-}
-
-static size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr))
-    {
-        if (previous0)
-        {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF)
-            {
-                n0+=24;
-                if (ip < iend-5)
-                {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                }
-                else
-                {
-                    bitStream >>= 16;
-                    bitCount+=16;
-                }
-            }
-            while ((bitStream & 3) == 3)
-            {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-            {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {
-            const short max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max)
-            {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            }
-            else
-            {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSE_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold)
-            {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            {
-                if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-                {
-                    ip += bitCount>>3;
-                    bitCount &= 7;
-                }
-                else
-                {
-                    bitCount -= (int)(8 * (iend - 4 - ip));
-                    ip = iend - 4;
-                }
-                bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-            }
-        }
-    }
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-
-
-/*********************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const cell = (FSE_decode_t*)(ptr) + 1;   /* because dt is unsigned */
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const dinfo = (FSE_decode_t*)(ptr) + 1;   /* because dt is unsigned */
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSE_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BIT_DStream_t bitD;
-    FSE_DState_t state1;
-    FSE_DState_t state2;
-    size_t errorCode;
-
-    /* Init */
-    errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-    if (FSE_isError(errorCode)) return errorCode;
-
-    FSE_initDState(&state1, &bitD, dt);
-    FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BIT_reloadDStream(&bitD)==BIT_DStream_unfinished) && (op<olimit) ; op+=4)
-    {
-        op[0] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[1] = FSE_GETSYMBOL(&state2);
-
-        if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[3] = FSE_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */
-    while (1)
-    {
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state1))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state1);
-
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state2))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state2);
-    }
-
-    /* end ? */
-    if (BIT_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2))
-        return op-ostart;
-
-    if (op==omax) return ERROR(dstSize_tooSmall);   /* dst buffer is full, but cSrc unfinished */
-
-    return ERROR(corruption_detected);
-}
-
-
-static size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));
-
-    /* select fast mode (static) */
-    if (DTableH.fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-static size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSE_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-    size_t errorCode;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSE decoding mode */
-    errorCode = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSE_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    errorCode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
-    if (FSE_isError(errorCode)) return errorCode;
-
-    /* always return, even if it is an error code */
-    return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);
-}
-
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+Huff0 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/****************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/****************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-/****************************************************************
-*  Error Management
-****************************************************************/
-#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/******************************************
-*  Helper functions
-******************************************/
-static unsigned HUF_isError(size_t code) { return ERR_isError(code); }
-
-#define HUF_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUF_MAX_TABLELOG  12           /* max configured tableLog (for static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */
-#define HUF_DEFAULT_TABLELOG  HUF_MAX_TABLELOG   /* tableLog by default, when not specified */
-#define HUF_MAX_SYMBOL_VALUE 255
-#if (HUF_MAX_TABLELOG > HUF_ABSOLUTEMAX_TABLELOG)
-#  error "HUF_MAX_TABLELOG is too large !"
-#endif
-
-
-
-/*********************************************************
-*  Huff0 : Huffman block decompression
-*********************************************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX2;   /* single-symbol decoding */
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-/*! HUF_readStats
-    Read compact Huffman tree, saved by HUF_writeCTable
-    @huffWeight : destination buffer
-    @return : size read from `src`
-*/
-static size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                            U32* nbSymbolsPtr, U32* tableLogPtr,
-                            const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    U32 tableLog;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-    U32 n;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  /* special header */
-    {
-        if (iSize >= (242))   /* RLE */
-        {
-            static int l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else   /* Incompressible */
-        {
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            for (n=0; n<oSize; n+=2)
-            {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-            }
-        }
-    }
-    else  /* header compressed with FSE (normal case) */
-    {
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSE_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSE_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUF_ABSOLUTEMAX_TABLELOG + 1) * sizeof(U32));
-    weightTotal = 0;
-    for (n=0; n<oSize; n++)
-    {
-        if (huffWeight[n] >= HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-        rankStats[huffWeight[n]]++;
-        weightTotal += (1 << huffWeight[n]) >> 1;
-    }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    tableLog = BIT_highbit32(weightTotal) + 1;
-    if (tableLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-    {
-        U32 total = 1 << tableLog;
-        U32 rest = total - weightTotal;
-        U32 verif = 1 << BIT_highbit32(rest);
-        U32 lastWeight = BIT_highbit32(rest) + 1;
-        if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-        huffWeight[oSize] = (BYTE)lastWeight;
-        rankStats[lastWeight]++;
-    }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    *tableLogPtr = tableLog;
-    return iSize+1;
-}
-
-
-/**************************/
-/* single-symbol decoding */
-/**************************/
-
-static size_t HUF_readDTableX2 (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize = ip[0];
-    U32 nbSymbols = 0;
-    U32 n;
-    U32 nextRankStart;
-    void* ptr = DTable+1;
-    HUF_DEltX2* const dt = (HUF_DEltX2*)ptr;
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX2) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(huffWeight, HUF_MAX_SYMBOL_VALUE + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > DTable[0]) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-    DTable[0] = (U16)tableLog;   /* maybe should separate sizeof DTable, as allocated, from used size of DTable, in case of DTable re-use */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<=tableLog; n++)
-    {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<nbSymbols; n++)
-    {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUF_DEltX2 D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize;
-}
-
-static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, const U32 dtLog)
-{
-        const size_t val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-        const BYTE c = dt[val].byte;
-        BIT_skipBits(Dstream, dt[val].nbBits);
-        return c;
-}
-
-#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUF_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4))
-    {
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd))
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-
-static size_t HUF_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-
-        const void* ptr = DTable;
-        const HUF_DEltX2* const dt = ((const HUF_DEltX2*)ptr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUF_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUF_decompress4X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/***************************/
-/* double-symbols decoding */
-/***************************/
-
-static void HUF_fillDTableX4Level2(HUF_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUF_DEltX4 DElt;
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    U32 s;
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1)
-    {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)   /* note : sortedSymbols already skipped */
-    {
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }
-}
-
-typedef U32 rankVal_t[HUF_ABSOLUTEMAX_TABLELOG][HUF_ABSOLUTEMAX_TABLELOG + 1];
-
-static void HUF_fillDTableX4(HUF_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)
-    {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits)   /* enough room for a second symbol */
-        {
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUF_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        }
-        else
-        {
-            U32 i;
-            const U32 end = start + length;
-            HUF_DEltX4 DElt;
-
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits   = (BYTE)(nbBits);
-            DElt.length   = 1;
-            for (i = start; i < end; i++)
-                DTable[i] = DElt;
-        }
-        rankVal[weight] += length;
-    }
-}
-
-static size_t HUF_readDTableX4 (U32* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUF_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUF_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUF_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    const U32 memLog = DTable[0];
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize = ip[0];
-    void* ptr = DTable;
-    HUF_DEltX4* const dt = ((HUF_DEltX4*)ptr) + 1;
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX4) == sizeof(U32));   /* if compilation fails here, assertion is false */
-    if (memLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(weightList, HUF_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--)
-        {if (!maxW) return ERROR(GENERIC); }  /* necessarily finds a solution before maxW==0 */
-
-    /* Get start index of each weight */
-    {
-        U32 w, nextRankStart = 0;
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {
-        U32 s;
-        for (s=0; s<nbSymbols; s++)
-        {
-            U32 w = weightList[s];
-            U32 r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {
-        const U32 minBits = tableLog+1 - maxW;
-        U32 nextRankVal = 0;
-        U32 w, consumed;
-        const int rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-        U32* rankVal0 = rankVal[0];
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankVal;
-            nextRankVal += rankStats[w] << (w+rescale);
-            rankVal0[w] = current;
-        }
-        for (consumed = minBits; consumed <= memLog - minBits; consumed++)
-        {
-            U32* rankValPtr = rankVal[consumed];
-            for (w = 1; w <= maxW; w++)
-            {
-                rankValPtr[w] = rankVal0[w] >> consumed;
-            }
-        }
-    }
-
-    HUF_fillDTableX4(dt, memLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    return iSize;
-}
-
-
-static U32 HUF_decodeSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BIT_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits);
-    else
-    {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8))
-        {
-            BIT_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-        }
-    }
-    return 1;
-}
-
-
-#define HUF_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd-7))
-    {
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-2))
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUF_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-
-
-static size_t HUF_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-
-        const void* ptr = DTable;
-        const HUF_DEltX4* const dt = ((const HUF_DEltX4*)ptr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUF_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUF_decompress4X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/**********************************/
-/* quad-symbol decoding           */
-/**********************************/
-typedef struct { BYTE nbBits; BYTE nbBytes; } HUF_DDescX6;
-typedef union { BYTE byte[4]; U32 sequence; } HUF_DSeqX6;
-
-/* recursive, up to level 3; may benefit from <template>-like strategy to nest each level inline */
-static void HUF_fillDTableX6LevelN(HUF_DDescX6* DDescription, HUF_DSeqX6* DSequence, int sizeLog,
-                           const rankVal_t rankValOrigin, const U32 consumed, const int minWeight, const U32 maxWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize, const U32* rankStart,
-                           const U32 nbBitsBaseline, HUF_DSeqX6 baseSeq, HUF_DDescX6 DDesc)
-{
-    const int scaleLog = nbBitsBaseline - sizeLog;   /* note : targetLog >= (nbBitsBaseline-1), hence scaleLog <= 1 */
-    const int minBits  = nbBitsBaseline - maxWeight;
-    const U32 level = DDesc.nbBytes;
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    U32 symbolStartPos, s;
-
-    /* local rankVal, will be modified */
-    memcpy(rankVal, rankValOrigin[consumed], sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1)
-    {
-        U32 i;
-        const U32 skipSize = rankVal[minWeight];
-        for (i = 0; i < skipSize; i++)
-        {
-            DSequence[i] = baseSeq;
-            DDescription[i] = DDesc;
-        }
-    }
-
-    /* fill DTable */
-    DDesc.nbBytes++;
-    symbolStartPos = rankStart[minWeight];
-    for (s=symbolStartPos; s<sortedListSize; s++)
-    {
-        const BYTE symbol = sortedSymbols[s].symbol;
-        const U32  weight = sortedSymbols[s].weight;   /* >= 1 (sorted) */
-        const int  nbBits = nbBitsBaseline - weight;   /* >= 1 (by construction) */
-        const int  totalBits = consumed+nbBits;
-        const U32  start  = rankVal[weight];
-        const U32  length = 1 << (sizeLog-nbBits);
-        baseSeq.byte[level] = symbol;
-        DDesc.nbBits = (BYTE)totalBits;
-
-        if ((level<3) && (sizeLog-totalBits >= minBits))   /* enough room for another symbol */
-        {
-            int nextMinWeight = totalBits + scaleLog;
-            if (nextMinWeight < 1) nextMinWeight = 1;
-            HUF_fillDTableX6LevelN(DDescription+start, DSequence+start, sizeLog-nbBits,
-                           rankValOrigin, totalBits, nextMinWeight, maxWeight,
-                           sortedSymbols, sortedListSize, rankStart,
-                           nbBitsBaseline, baseSeq, DDesc);   /* recursive (max : level 3) */
-        }
-        else
-        {
-            U32 i;
-            const U32 end = start + length;
-            for (i = start; i < end; i++)
-            {
-                DDescription[i] = DDesc;
-                DSequence[i] = baseSeq;
-            }
-        }
-        rankVal[weight] += length;
-    }
-}
-
-
-/* note : same preparation as X4 */
-static size_t HUF_readDTableX6 (U32* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUF_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUF_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUF_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    rankVal_t rankVal;
-    const U32 memLog = DTable[0];
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize = ip[0];
-
-    if (memLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(weightList, HUF_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--)
-        { if (!maxW) return ERROR(GENERIC); }  /* necessarily finds a solution before maxW==0 */
-
-
-    /* Get start index of each weight */
-    {
-        U32 w, nextRankStart = 0;
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {
-        U32 s;
-        for (s=0; s<nbSymbols; s++)
-        {
-            U32 w = weightList[s];
-            U32 r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {
-        const U32 minBits = tableLog+1 - maxW;
-        U32 nextRankVal = 0;
-        U32 w, consumed;
-        const int rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-        U32* rankVal0 = rankVal[0];
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankVal;
-            nextRankVal += rankStats[w] << (w+rescale);
-            rankVal0[w] = current;
-        }
-        for (consumed = minBits; consumed <= memLog - minBits; consumed++)
-        {
-            U32* rankValPtr = rankVal[consumed];
-            for (w = 1; w <= maxW; w++)
-            {
-                rankValPtr[w] = rankVal0[w] >> consumed;
-            }
-        }
-    }
-
-
-    /* fill tables */
-    {
-        void* ptr = DTable+1;
-        HUF_DDescX6* DDescription = (HUF_DDescX6*)(ptr);
-        void* dSeqStart = DTable + 1 + ((size_t)1<<(memLog-1));
-        HUF_DSeqX6* DSequence = (HUF_DSeqX6*)(dSeqStart);
-        HUF_DSeqX6 DSeq;
-        HUF_DDescX6 DDesc;
-        DSeq.sequence = 0;
-        DDesc.nbBits = 0;
-        DDesc.nbBytes = 0;
-        HUF_fillDTableX6LevelN(DDescription, DSequence, memLog,
-                       (const U32 (*)[HUF_ABSOLUTEMAX_TABLELOG + 1])rankVal, 0, 1, maxW,
-                       sortedSymbol, sizeOfSort, rankStart0,
-                       tableLog+1, DSeq, DDesc);
-    }
-
-    return iSize;
-}
-
-
-static U32 HUF_decodeSymbolX6(void* op, BIT_DStream_t* DStream, const HUF_DDescX6* dd, const HUF_DSeqX6* ds, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, ds+val, sizeof(HUF_DSeqX6));
-    BIT_skipBits(DStream, dd[val].nbBits);
-    return dd[val].nbBytes;
-}
-
-static U32 HUF_decodeLastSymbolsX6(void* op, const U32 maxL, BIT_DStream_t* DStream,
-                                  const HUF_DDescX6* dd, const HUF_DSeqX6* ds, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    U32 length = dd[val].nbBytes;
-    if (length <= maxL)
-    {
-        memcpy(op, ds+val, length);
-        BIT_skipBits(DStream, dd[val].nbBits);
-        return length;
-    }
-    memcpy(op, ds+val, maxL);
-    if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8))
-    {
-        BIT_skipBits(DStream, dd[val].nbBits);
-        if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-            DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-    }
-    return maxL;
-}
-
-
-#define HUF_DECODE_SYMBOLX6_0(ptr, DStreamPtr) \
-    ptr += HUF_decodeSymbolX6(ptr, DStreamPtr, dd, ds, dtLog)
-
-#define HUF_DECODE_SYMBOLX6_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        HUF_DECODE_SYMBOLX6_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX6_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUF_DECODE_SYMBOLX6_0(ptr, DStreamPtr)
-
-static inline size_t HUF_decodeStreamX6(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const U32* DTable, const U32 dtLog)
-{
-    const void* ddPtr = DTable+1;
-    const HUF_DDescX6* dd = (const HUF_DDescX6*)(ddPtr);
-    const void* dsPtr = DTable + 1 + ((size_t)1<<(dtLog-1));
-    const HUF_DSeqX6* ds = (const HUF_DSeqX6*)(dsPtr);
-    BYTE* const pStart = p;
-
-    /* up to 16 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-16))
-    {
-        HUF_DECODE_SYMBOLX6_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX6_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX6_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX6_0(p, bitDPtr);
-    }
-
-    /* closer to the end, up to 4 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4))
-        HUF_DECODE_SYMBOLX6_0(p, bitDPtr);
-
-    while (p <= pEnd-4)
-        HUF_DECODE_SYMBOLX6_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    while (p < pEnd)
-        p += HUF_decodeLastSymbolsX6(p, (U32)(pEnd-p), bitDPtr, dd, ds, dtLog);
-
-    return p-pStart;
-}
-
-
-
-static size_t HUF_decompress4X6_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-
-        const U32 dtLog = DTable[0];
-        const void* ddPtr = DTable+1;
-        const HUF_DDescX6* dd = (const HUF_DDescX6*)(ddPtr);
-        const void* dsPtr = DTable + 1 + ((size_t)1<<(dtLog-1));
-        const HUF_DSeqX6* ds = (const HUF_DSeqX6*)(dsPtr);
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-64 symbols per loop (4-16 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (op3 <= opStart4) && (endSignal==BIT_DStream_unfinished) && (op4<=(oend-16)) ; )
-        {
-            HUF_DECODE_SYMBOLX6_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX6_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX6_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX6_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX6_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX6_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX6_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX6_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX6_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX6_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX6_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX6_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX6_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX6_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX6_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX6_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX6(op1, &bitD1, opStart2, DTable, dtLog);
-        HUF_decodeStreamX6(op2, &bitD2, opStart3, DTable, dtLog);
-        HUF_decodeStreamX6(op3, &bitD3, opStart4, DTable, dtLog);
-        HUF_decodeStreamX6(op4, &bitD4, oend,     DTable, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X6 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX6(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUF_readDTableX6 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUF_decompress4X6_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/**********************************/
-/* Generic decompression selector */
-/**********************************/
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-static size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[3] = { HUF_decompress4X2, HUF_decompress4X4, HUF_decompress4X6 };
-    /* estimate decompression time */
-    U32 Q;
-    const U32 D256 = (U32)(dstSize >> 8);
-    U32 Dtime[3];
-    U32 algoNb = 0;
-    int n;
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    /* decoder timing evaluation */
-    Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-    for (n=0; n<3; n++)
-        Dtime[n] = algoTime[Q][n].tableTime + (algoTime[Q][n].decode256Time * D256);
-
-    Dtime[1] += Dtime[1] >> 4; Dtime[2] += Dtime[2] >> 3; /* advantage to algorithms using less memory, for cache eviction */
-
-    if (Dtime[1] < Dtime[0]) algoNb = 1;
-    if (Dtime[2] < Dtime[algoNb]) algoNb = 2;
-
-    return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-
-    //return HUF_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUF_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-    //return HUF_decompress4X6(dst, dstSize, cSrc, cSrcSize);   /* multi-streams quad-symbols decoding */
-}
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
-*  MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*/
-#define ZSTD_MEMORY_USAGE 17
-
-/*!
- * HEAPMODE :
- * Select how default compression functions will allocate memory for their hash table,
- * in memory stack (0, fastest), or in memory heap (1, requires malloc())
- * Note that compression context is fairly large, as a consequence heap memory is recommended.
- */
-#ifndef ZSTD_HEAPMODE
-#  define ZSTD_HEAPMODE 1
-#endif /* ZSTD_HEAPMODE */
-
-/*!
-*  LEGACY_SUPPORT :
-*  decompressor can decode older formats (starting from Zstd 0.1+)
-*/
-#ifndef ZSTD_LEGACY_SUPPORT
-#  define ZSTD_LEGACY_SUPPORT 1
-#endif
-
-
-/* *******************************************************
-*  Includes
-*********************************************************/
-#include <stdlib.h>      /* calloc */
-#include <string.h>      /* memcpy, memmove */
-#include <stdio.h>       /* debug : printf */
-
-
-/* *******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef __AVX2__
-#  include <immintrin.h>   /* AVX2 intrinsics */
-#endif
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-
-/* *******************************************************
-*  Constants
-*********************************************************/
-#define HASH_LOG (ZSTD_MEMORY_USAGE - 2)
-#define HASH_TABLESIZE (1 << HASH_LOG)
-#define HASH_MASK (HASH_TABLESIZE - 1)
-
-#define KNUTH 2654435761
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BLOCKSIZE (128 KB)                 /* define, for static allocation */
-#define MIN_SEQUENCES_SIZE (2 /*seqNb*/ + 2 /*dumps*/ + 3 /*seqTables*/ + 1 /*bitStream*/)
-#define MIN_CBLOCK_SIZE (3 /*litCSize*/ + MIN_SEQUENCES_SIZE)
-#define IS_RAW BIT0
-#define IS_RLE BIT1
-
-#define WORKPLACESIZE (BLOCKSIZE*3)
-#define MINMATCH 4
-#define MLbits   7
-#define LLbits   6
-#define Offbits  5
-#define MaxML  ((1<<MLbits )-1)
-#define MaxLL  ((1<<LLbits )-1)
-#define MaxOff   31
-#define LitFSELog  11
-#define MLFSELog   10
-#define LLFSELog   10
-#define OffFSELog   9
-#define MAX(a,b) ((a)<(b)?(b):(a))
-#define MaxSeq MAX(MaxLL, MaxML)
-
-#define LITERAL_NOENTROPY 63
-#define COMMAND_NOENTROPY 7   /* to remove */
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-static const size_t ZSTD_blockHeaderSize = 3;
-static const size_t ZSTD_frameHeaderSize = 4;
-
-
-/* *******************************************************
-*  Memory operations
-**********************************************************/
-static void   ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-static void   ZSTD_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s) { ZSTD_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTD_wildcopy : custom version of memcpy(), can copy up to 7-8 bytes too many */
-static void ZSTD_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do COPY8(op, ip) while (op < oend);
-}
-
-
-/* **************************************
-*  Local structures
-****************************************/
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* offCode;
-    BYTE* litStart;
-    BYTE* lit;
-    BYTE* litLengthStart;
-    BYTE* litLength;
-    BYTE* matchLengthStart;
-    BYTE* matchLength;
-    BYTE* dumpsStart;
-    BYTE* dumps;
-} seqStore_t;
-
-
-/* *************************************
-*  Error Management
-***************************************/
-/*! ZSTD_isError
-*   tells if a return value is an error code */
-static unsigned ZSTD_isError(size_t code) { return ERR_isError(code); }
-
-
-
-/* *************************************************************
-*   Decompression section
-***************************************************************/
-struct ZSTD_DCtx_s
-{
-    U32 LLTable[FSE_DTABLE_SIZE_U32(LLFSELog)];
-    U32 OffTable[FSE_DTABLE_SIZE_U32(OffFSELog)];
-    U32 MLTable[FSE_DTABLE_SIZE_U32(MLFSELog)];
-    void* previousDstEnd;
-    void* base;
-    size_t expected;
-    blockType_t bType;
-    U32 phase;
-    const BYTE* litPtr;
-    size_t litSize;
-    BYTE litBuffer[BLOCKSIZE + 8 /* margin for wildcopy */];
-};   /* typedef'd to ZSTD_Dctx within "zstd_static.h" */
-
-
-static size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    BYTE headerFlags;
-    U32 cSize;
-
-    if (srcSize < 3) return ERROR(srcSize_wrong);
-
-    headerFlags = *in;
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-
-    bpPtr->blockType = (blockType_t)(headerFlags >> 6);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-static size_t ZSTD_copyUncompressedBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    if (srcSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/** ZSTD_decompressLiterals
-    @return : nb of bytes read from src, or an error code*/
-static size_t ZSTD_decompressLiterals(void* dst, size_t* maxDstSizePtr,
-                                const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-
-    const size_t litSize = (MEM_readLE32(src) & 0x1FFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-    const size_t litCSize = (MEM_readLE32(ip+2) & 0xFFFFFF) >> 5;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-
-    if (litSize > *maxDstSizePtr) return ERROR(corruption_detected);
-    if (litCSize + 5 > srcSize) return ERROR(corruption_detected);
-
-    if (HUF_isError(HUF_decompress(dst, litSize, ip+5, litCSize))) return ERROR(corruption_detected);
-
-    *maxDstSizePtr = litSize;
-    return litCSize + 5;
-}
-
-
-/** ZSTD_decodeLiteralsBlock
-    @return : nb of bytes read from src (< srcSize )*/
-static size_t ZSTD_decodeLiteralsBlock(void* ctx,
-                          const void* src, size_t srcSize)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)ctx;
-    const BYTE* const istart = (const BYTE* const)src;
-
-    /* any compressed block with literals segment must be at least this size */
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch(*istart & 3)
-    {
-    default:
-    case 0:
-        {
-            size_t litSize = BLOCKSIZE;
-            const size_t readSize = ZSTD_decompressLiterals(dctx->litBuffer, &litSize, src, srcSize);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, 8);
-            return readSize;   /* works if it's an error too */
-        }
-    case IS_RAW:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > srcSize-11)   /* risk of reading too far with wildcopy */
-            {
-                if (litSize > srcSize-3) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, 8);
-                return litSize+3;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+3;
-            dctx->litSize = litSize;
-            return litSize+3;
-        }
-    case IS_RLE:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > BLOCKSIZE) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[3], litSize + 8);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return 4;
-        }
-    }
-}
-
-
-static size_t ZSTD_decodeSeqHeaders(int* nbSeq, const BYTE** dumpsPtr, size_t* dumpsLengthPtr,
-                         FSE_DTable* DTableLL, FSE_DTable* DTableML, FSE_DTable* DTableOffb,
-                         const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    const BYTE* const iend = istart + srcSize;
-    U32 LLtype, Offtype, MLtype;
-    U32 LLlog, Offlog, MLlog;
-    size_t dumpsLength;
-
-    /* check */
-    if (srcSize < 5) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    *nbSeq = MEM_readLE16(ip); ip+=2;
-    LLtype  = *ip >> 6;
-    Offtype = (*ip >> 4) & 3;
-    MLtype  = (*ip >> 2) & 3;
-    if (*ip & 2)
-    {
-        dumpsLength  = ip[2];
-        dumpsLength += ip[1] << 8;
-        ip += 3;
-    }
-    else
-    {
-        dumpsLength  = ip[1];
-        dumpsLength += (ip[0] & 1) << 8;
-        ip += 2;
-    }
-    *dumpsPtr = ip;
-    ip += dumpsLength;
-    *dumpsLengthPtr = dumpsLength;
-
-    /* check */
-    if (ip > iend-3) return ERROR(srcSize_wrong); /* min : all 3 are "raw", hence no header, but at least xxLog bits per type */
-
-    /* sequences */
-    {
-        S16 norm[MaxML+1];    /* assumption : MaxML >= MaxLL and MaxOff */
-        size_t headerSize;
-
-        /* Build DTables */
-        switch(LLtype)
-        {
-        case bt_rle :
-            LLlog = 0;
-            FSE_buildDTable_rle(DTableLL, *ip++); break;
-        case bt_raw :
-            LLlog = LLbits;
-            FSE_buildDTable_raw(DTableLL, LLbits); break;
-        default :
-            {   U32 max = MaxLL;
-                headerSize = FSE_readNCount(norm, &max, &LLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (LLlog > LLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableLL, norm, max, LLlog);
-        }   }
-
-        switch(Offtype)
-        {
-        case bt_rle :
-            Offlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong);   /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableOffb, *ip++ & MaxOff); /* if *ip > MaxOff, data is corrupted */
-            break;
-        case bt_raw :
-            Offlog = Offbits;
-            FSE_buildDTable_raw(DTableOffb, Offbits); break;
-        default :
-            {   U32 max = MaxOff;
-                headerSize = FSE_readNCount(norm, &max, &Offlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (Offlog > OffFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableOffb, norm, max, Offlog);
-        }   }
-
-        switch(MLtype)
-        {
-        case bt_rle :
-            MLlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableML, *ip++); break;
-        case bt_raw :
-            MLlog = MLbits;
-            FSE_buildDTable_raw(DTableML, MLbits); break;
-        default :
-            {   U32 max = MaxML;
-                headerSize = FSE_readNCount(norm, &max, &MLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (MLlog > MLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableML, norm, max, MLlog);
-    }   }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t offset;
-    size_t matchLength;
-} seq_t;
-
-typedef struct {
-    BIT_DStream_t DStream;
-    FSE_DState_t stateLL;
-    FSE_DState_t stateOffb;
-    FSE_DState_t stateML;
-    size_t prevOffset;
-    const BYTE* dumps;
-    const BYTE* dumpsEnd;
-} seqState_t;
-
-
-static void ZSTD_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    size_t litLength;
-    size_t prevOffset;
-    size_t offset;
-    size_t matchLength;
-    const BYTE* dumps = seqState->dumps;
-    const BYTE* const de = seqState->dumpsEnd;
-
-    /* Literal length */
-    litLength = FSE_decodeSymbol(&(seqState->stateLL), &(seqState->DStream));
-    prevOffset = litLength ? seq->offset : seqState->prevOffset;
-    seqState->prevOffset = seq->offset;
-    if (litLength == MaxLL)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) litLength += add;
-        else if (dumps + 3 <= de)
-        {
-            litLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) dumps = de-1;   /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-
-    /* Offset */
-    {
-        static const size_t offsetPrefix[MaxOff+1] = {  /* note : size_t faster than U32 */
-                1 /*fake*/, 1, 2, 4, 8, 16, 32, 64, 128, 256,
-                512, 1024, 2048, 4096, 8192, 16384, 32768, 65536, 131072, 262144,
-                524288, 1048576, 2097152, 4194304, 8388608, 16777216, 33554432, /*fake*/ 1, 1, 1, 1, 1 };
-        U32 offsetCode, nbBits;
-        offsetCode = FSE_decodeSymbol(&(seqState->stateOffb), &(seqState->DStream));   /* <= maxOff, by table construction */
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        nbBits = offsetCode - 1;
-        if (offsetCode==0) nbBits = 0;   /* cmove */
-        offset = offsetPrefix[offsetCode] + BIT_readBits(&(seqState->DStream), nbBits);
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        if (offsetCode==0) offset = prevOffset;   /* cmove */
-    }
-
-    /* MatchLength */
-    matchLength = FSE_decodeSymbol(&(seqState->stateML), &(seqState->DStream));
-    if (matchLength == MaxML)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) matchLength += add;
-        else if (dumps + 3 <= de)
-        {
-            matchLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) dumps = de-1;   /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-    matchLength += MINMATCH;
-
-    /* save result */
-    seq->litLength = litLength;
-    seq->offset = offset;
-    seq->matchLength = matchLength;
-    seqState->dumps = dumps;
-}
-
-
-static size_t ZSTD_execSequence(BYTE* op,
-                                seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                BYTE* const base, BYTE* const oend)
-{
-    static const int dec32table[] = {0, 1, 2, 1, 4, 4, 4, 4};   /* added */
-    static const int dec64table[] = {8, 8, 8, 7, 8, 9,10,11};   /* subtracted */
-    const BYTE* const ostart = op;
-    BYTE* const oLitEnd = op + sequence.litLength;
-    BYTE* const oMatchEnd = op + sequence.litLength + sequence.matchLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_8 = oend-8;
-    const BYTE* const litEnd = *litPtr + sequence.litLength;
-
-    /* checks */
-    if (oLitEnd > oend_8) return ERROR(dstSize_tooSmall);   /* last match must start at a minimum distance of 8 from oend */
-    if (oMatchEnd > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (litEnd > litLimit) return ERROR(corruption_detected);   /* overRead beyond lit buffer */
-
-    /* copy Literals */
-    ZSTD_wildcopy(op, *litPtr, sequence.litLength);   /* note : oLitEnd <= oend-8 : no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = litEnd;   /* update for next sequence */
-
-    /* copy Match */
-    {
-        const BYTE* match = op - sequence.offset;
-
-        /* check */
-        if (sequence.offset > (size_t)op) return ERROR(corruption_detected);   /* address space overflow test (this test seems kept by clang optimizer) */
-        //if (match > op) return ERROR(corruption_detected);   /* address space overflow test (is clang optimizer removing this test ?) */
-        if (match < base) return ERROR(corruption_detected);
-
-        /* close range match, overlap */
-        if (sequence.offset < 8)
-        {
-            const int dec64 = dec64table[sequence.offset];
-            op[0] = match[0];
-            op[1] = match[1];
-            op[2] = match[2];
-            op[3] = match[3];
-            match += dec32table[sequence.offset];
-            ZSTD_copy4(op+4, match);
-            match -= dec64;
-        }
-        else
-        {
-            ZSTD_copy8(op, match);
-        }
-        op += 8; match += 8;
-
-        if (oMatchEnd > oend-(16-MINMATCH))
-        {
-            if (op < oend_8)
-            {
-                ZSTD_wildcopy(op, match, oend_8 - op);
-                match += oend_8 - op;
-                op = oend_8;
-            }
-            while (op < oMatchEnd) *op++ = *match++;
-        }
-        else
-        {
-            ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-        }
-    }
-
-    return oMatchEnd - ostart;
-}
-
-static size_t ZSTD_decompressSequences(
-                               void* ctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)ctx;
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t errorCode, dumpsLength;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    int nbSeq;
-    const BYTE* dumps;
-    U32* DTableLL = dctx->LLTable;
-    U32* DTableML = dctx->MLTable;
-    U32* DTableOffb = dctx->OffTable;
-    BYTE* const base = (BYTE*) (dctx->base);
-
-    /* Build Decoding Tables */
-    errorCode = ZSTD_decodeSeqHeaders(&nbSeq, &dumps, &dumpsLength,
-                                      DTableLL, DTableML, DTableOffb,
-                                      ip, iend-ip);
-    if (ZSTD_isError(errorCode)) return errorCode;
-    ip += errorCode;
-
-    /* Regen sequences */
-    {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        seqState.dumps = dumps;
-        seqState.dumpsEnd = dumps + dumpsLength;
-        seqState.prevOffset = 1;
-        errorCode = BIT_initDStream(&(seqState.DStream), ip, iend-ip);
-        if (ERR_isError(errorCode)) return ERROR(corruption_detected);
-        FSE_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSE_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSE_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BIT_reloadDStream(&(seqState.DStream)) <= BIT_DStream_completed) && (nbSeq>0) ; )
-        {
-            size_t oneSeqSize;
-            nbSeq--;
-            ZSTD_decodeSequence(&sequence, &seqState);
-            oneSeqSize = ZSTD_execSequence(op, sequence, &litPtr, litEnd, base, oend);
-            if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* check if reached exact end */
-        if ( !BIT_endOfDStream(&(seqState.DStream)) ) return ERROR(corruption_detected);   /* requested too much : data is corrupted */
-        if (nbSeq<0) return ERROR(corruption_detected);   /* requested too many sequences : data is corrupted */
-
-        /* last literal segment */
-        {
-            size_t lastLLSize = litEnd - litPtr;
-            if (litPtr > litEnd) return ERROR(corruption_detected);
-            if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-            if (op != litPtr) memmove(op, litPtr, lastLLSize);
-            op += lastLLSize;
-        }
-    }
-
-    return op-ostart;
-}
-
-
-static size_t ZSTD_decompressBlock(
-                            void* ctx,
-                            void* dst, size_t maxDstSize,
-                      const void* src, size_t srcSize)
-{
-    /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-
-    /* Decode literals sub-block */
-    size_t litCSize = ZSTD_decodeLiteralsBlock(ctx, src, srcSize);
-    if (ZSTD_isError(litCSize)) return litCSize;
-    ip += litCSize;
-    srcSize -= litCSize;
-
-    return ZSTD_decompressSequences(ctx, dst, maxDstSize, ip, srcSize);
-}
-
-
-static size_t ZSTD_decompressDCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t remainingSize = srcSize;
-    U32 magicNumber;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t decodedSize=0;
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTD_decompressBlock(ctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTD_copyUncompressedBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        if (ZSTD_isError(decodedSize)) return decodedSize;
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-static size_t ZSTD_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    ZSTD_DCtx ctx;
-    ctx.base = dst;
-    return ZSTD_decompressDCtx(&ctx, dst, maxDstSize, src, srcSize);
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv02_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    U32 magicNumber;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_magicNumber) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-        return;
-    }
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * BLOCKSIZE;
-}
-
-/*******************************
-*  Streaming Decompression API
-*******************************/
-
-static size_t ZSTD_resetDCtx(ZSTD_DCtx* dctx)
-{
-    dctx->expected = ZSTD_frameHeaderSize;
-    dctx->phase = 0;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    return 0;
-}
-
-static ZSTD_DCtx* ZSTD_createDCtx(void)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)malloc(sizeof(ZSTD_DCtx));
-    if (dctx==NULL) return NULL;
-    ZSTD_resetDCtx(dctx);
-    return dctx;
-}
-
-static size_t ZSTD_freeDCtx(ZSTD_DCtx* dctx)
-{
-    free(dctx);
-    return 0;
-}
-
-static size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-static size_t ZSTD_decompressContinue(ZSTD_DCtx* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != ctx->expected) return ERROR(srcSize_wrong);
-    if (dst != ctx->previousDstEnd)  /* not contiguous */
-        ctx->base = dst;
-
-    /* Decompress : frame header */
-    if (ctx->phase == 0)
-    {
-        /* Check frame magic header */
-        U32 magicNumber = MEM_readLE32(src);
-        if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        return 0;
-    }
-
-    /* Decompress : block header */
-    if (ctx->phase == 1)
-    {
-        blockProperties_t bp;
-        size_t blockSize = ZSTD_getcBlockSize(src, ZSTD_blockHeaderSize, &bp);
-        if (ZSTD_isError(blockSize)) return blockSize;
-        if (bp.blockType == bt_end)
-        {
-            ctx->expected = 0;
-            ctx->phase = 0;
-        }
-        else
-        {
-            ctx->expected = blockSize;
-            ctx->bType = bp.blockType;
-            ctx->phase = 2;
-        }
-
-        return 0;
-    }
-
-    /* Decompress : block content */
-    {
-        size_t rSize;
-        switch(ctx->bType)
-        {
-        case bt_compressed:
-            rSize = ZSTD_decompressBlock(ctx, dst, maxDstSize, src, srcSize);
-            break;
-        case bt_raw :
-            rSize = ZSTD_copyUncompressedBlock(dst, maxDstSize, src, srcSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet handled */
-            break;
-        case bt_end :   /* should never happen (filtered at phase 1) */
-            rSize = 0;
-            break;
-        default:
-            return ERROR(GENERIC);
-        }
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        ctx->previousDstEnd = (void*)( ((char*)dst) + rSize);
-        return rSize;
-    }
-
-}
-
-
-/* wrapper layer */
-
-unsigned ZSTDv02_isError(size_t code)
-{
-    return ZSTD_isError(code);
-}
-
-size_t ZSTDv02_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize)
-{
-    return ZSTD_decompress(dst, maxOriginalSize, src, compressedSize);
-}
-
-ZSTDv02_Dctx* ZSTDv02_createDCtx(void)
-{
-    return (ZSTDv02_Dctx*)ZSTD_createDCtx();
-}
-
-size_t ZSTDv02_freeDCtx(ZSTDv02_Dctx* dctx)
-{
-    return ZSTD_freeDCtx((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv02_resetDCtx(ZSTDv02_Dctx* dctx)
-{
-    return ZSTD_resetDCtx((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv02_nextSrcSizeToDecompress(ZSTDv02_Dctx* dctx)
-{
-    return ZSTD_nextSrcSizeToDecompress((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv02_decompressContinue(ZSTDv02_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return ZSTD_decompressContinue((ZSTD_DCtx*)dctx, dst, maxDstSize, src, srcSize);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_v02.h b/vendor/github.com/DataDog/zstd/zstd_v02.h
deleted file mode 100644
index 9d7d8d9..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v02.h
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_V02_H_4174539423
-#define ZSTD_V02_H_4174539423
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Simple one-step function
-***************************************/
-/**
-ZSTDv02_decompress() : decompress ZSTD frames compliant with v0.2.x format
-    compressedSize : is the exact source size
-    maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
-                      It must be equal or larger than originalSize, otherwise decompression will fail.
-    return : the number of bytes decompressed into destination buffer (originalSize)
-             or an errorCode if it fails (which can be tested using ZSTDv01_isError())
-*/
-size_t ZSTDv02_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize);
-
- /**
- ZSTDv02_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.2.x format
-     srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-     cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                 or an error code if it fails (which can be tested using ZSTDv01_isError())
-     dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                 or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
- */
-void ZSTDv02_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                     size_t* cSize, unsigned long long* dBound);
-
-/**
-ZSTDv02_isError() : tells if the result of ZSTDv02_decompress() is an error
-*/
-unsigned ZSTDv02_isError(size_t code);
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTDv02_Dctx_s ZSTDv02_Dctx;
-ZSTDv02_Dctx* ZSTDv02_createDCtx(void);
-size_t ZSTDv02_freeDCtx(ZSTDv02_Dctx* dctx);
-
-size_t ZSTDv02_decompressDCtx(void* ctx,
-                              void* dst, size_t maxOriginalSize,
-                        const void* src, size_t compressedSize);
-
-/* *************************************
-*  Streaming functions
-***************************************/
-size_t ZSTDv02_resetDCtx(ZSTDv02_Dctx* dctx);
-
-size_t ZSTDv02_nextSrcSizeToDecompress(ZSTDv02_Dctx* dctx);
-size_t ZSTDv02_decompressContinue(ZSTDv02_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
-/**
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTDv02_magicNumber 0xFD2FB522   /* v0.2 */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_V02_H_4174539423 */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v03.c b/vendor/github.com/DataDog/zstd/zstd_v03.c
deleted file mode 100644
index 7a0e7c9..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v03.c
+++ /dev/null
@@ -1,3155 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include "zstd_v03.h"
-#include "error_private.h"
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-
-
-
-/* ******************************************************************
-   mem.h
-   low-level memory access routines
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/******************************************
-*  Includes
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include <string.h>    /* memcpy */
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/****************************************************************
-*  Basic Types
-*****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/****************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets generating assembly depending on alignment.
- *            But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(void*)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(void*)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-
-#endif // MEM_FORCE_MEMORY_ACCESS
-
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian())
-    {
-        MEM_write16(memPtr, val);
-    }
-    else
-    {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE24(const void* memPtr)
-{
-    return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24));
-    }
-}
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24)
-                     + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56));
-    }
-}
-
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-
-
-/* ******************************************************************
-   bitstream
-   Part of NewGen Entropy library
-   header file (to include)
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which highly benefit from being inlined.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-
-/**********************************************
-*  bitStream decompression API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BIT_DStream_t;
-
-typedef enum { BIT_DStream_unfinished = 0,
-               BIT_DStream_endOfBuffer = 1,
-               BIT_DStream_completed = 2,
-               BIT_DStream_overflow = 3 } BIT_DStream_status;  /* result of BIT_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD);
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD);
-
-
-
-/******************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/****************************************************************
-*  Helper functions
-****************************************************************/
-MEM_STATIC unsigned BIT_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-
-/**********************************************************
-* bitStream decoding
-**********************************************************/
-
-/*!BIT_initDStream
-*  Initialize a BIT_DStream_t.
-*  @bitD : a pointer to an already allocated BIT_DStream_t structure
-*  @srcBuffer must point at the beginning of a bitStream
-*  @srcSize must be the exact size of the bitStream
-*  @result : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(size_t))   /* normal case */
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(size_t);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-    }
-    else
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);
-                    /* fallthrough */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);
-                    /* fallthrough */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);
-                    /* fallthrough */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24;
-                    /* fallthrough */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16;
-                    /* fallthrough */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) <<  8;
-                    /* fallthrough */
-            default:;
-        }
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-        bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-MEM_STATIC size_t BIT_lookBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BIT_lookBitsFast :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_lookBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBits(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*!BIT_readBitsFast :
-*  unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBitsFast(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return BIT_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer))
-    {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BIT_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start)
-    {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer;
-        return BIT_DStream_completed;
-    }
-    {
-        U32 nbBytes = bitD->bitsConsumed >> 3;
-        BIT_DStream_status result = BIT_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start)
-        {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BIT_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BIT_endOfDStream
-*   @return Tells if DStream has reached its exact end
-*/
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
-/* ******************************************************************
-   Error codes and messages
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef ERROR_H_MODULE
-#define ERROR_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define ERR_STATIC static inline
-#elif defined(_MSC_VER)
-#  define ERR_STATIC static __inline
-#elif defined(__GNUC__)
-#  define ERR_STATIC static __attribute__((unused))
-#else
-#  define ERR_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/******************************************
-*  Error Management
-******************************************/
-#define PREFIX(name) ZSTD_error_##name
-
-#define ERROR(name) (size_t)-PREFIX(name)
-
-#define ERROR_LIST(ITEM) \
-        ITEM(PREFIX(No_Error)) ITEM(PREFIX(GENERIC)) \
-        ITEM(PREFIX(dstSize_tooSmall)) ITEM(PREFIX(srcSize_wrong)) \
-        ITEM(PREFIX(prefix_unknown)) ITEM(PREFIX(corruption_detected)) \
-        ITEM(PREFIX(tableLog_tooLarge)) ITEM(PREFIX(maxSymbolValue_tooLarge)) ITEM(PREFIX(maxSymbolValue_tooSmall)) \
-        ITEM(PREFIX(maxCode))
-
-#define ERROR_GENERATE_ENUM(ENUM) ENUM,
-typedef enum { ERROR_LIST(ERROR_GENERATE_ENUM) } ERR_codes;  /* enum is exposed, to detect & handle specific errors; compare function result to -enum value */
-
-#define ERROR_CONVERTTOSTRING(STRING) #STRING,
-#define ERROR_GENERATE_STRING(EXPR) ERROR_CONVERTTOSTRING(EXPR)
-static const char* ERR_strings[] = { ERROR_LIST(ERROR_GENERATE_STRING) };
-
-ERR_STATIC unsigned ERR_isError(size_t code) { return (code > ERROR(maxCode)); }
-
-ERR_STATIC const char* ERR_getErrorName(size_t code)
-{
-    static const char* codeError = "Unspecified error code";
-    if (ERR_isError(code)) return ERR_strings[-(int)(code)];
-    return codeError;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ERROR_H_MODULE */
-/*
-Constructor and Destructor of type FSE_CTable
-    Note that its size depends on 'tableLog' and 'maxSymbolValue' */
-typedef unsigned FSE_CTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-typedef unsigned FSE_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-
-
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/******************************************
-*  Static allocation
-******************************************/
-/* FSE buffer bounds */
-#define FSE_NCOUNTBOUND 512
-#define FSE_BLOCKBOUND(size) (size + (size>>7))
-#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* You can statically allocate FSE CTable/DTable as a table of unsigned using below macro */
-#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue)   (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
-#define FSE_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/******************************************
-*  FSE advanced API
-******************************************/
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
-/* build a fake FSE_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
-/* build a fake FSE_DTable, designed to always generate the same symbolValue */
-
-
-/******************************************
-*  FSE symbol decompression API
-******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-
-static void     FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt);
-
-static unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
-
-
-/******************************************
-*  FSE unsafe API
-******************************************/
-static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/******************************************
-*  Implementation of inline functions
-******************************************/
-
-/* decompression */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSE_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSE_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));
-    DStatePtr->state = BIT_readBits(bitD, DTableH.tableLog);
-    BIT_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32  nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32 nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/******************************************
-*  Static allocation macros
-******************************************/
-/* Huff0 buffer bounds */
-#define HUF_CTABLEBOUND 129
-#define HUF_BLOCKBOUND(size) (size + (size>>8) + 8)   /* only true if incompressible pre-filtered with fast heuristic */
-#define HUF_COMPRESSBOUND(size) (HUF_CTABLEBOUND + HUF_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* static allocation of Huff0's DTable */
-#define HUF_DTABLE_SIZE(maxTableLog)   (1 + (1<<maxTableLog))  /* nb Cells; use unsigned short for X2, unsigned int for X4 */
-#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        unsigned short DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX6(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog) * 3 / 2] = { maxTableLog }
-
-
-/******************************************
-*  Advanced functions
-******************************************/
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbols decoder */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-/*
-    zstd - standard compression library
-    Header File
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Version
-***************************************/
-#define ZSTD_VERSION_MAJOR    0    /* for breaking interface changes  */
-#define ZSTD_VERSION_MINOR    2    /* for new (non-breaking) interface capabilities */
-#define ZSTD_VERSION_RELEASE  2    /* for tweaks, bug-fixes, or development */
-#define ZSTD_VERSION_NUMBER  (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTD_CCtx_s ZSTD_CCtx;   /* incomplete type */
-
-#if defined (__cplusplus)
-}
-#endif
-/*
-    zstd - standard compression library
-    Header File for static linking only
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* The objects defined into this file should be considered experimental.
- * They are not labelled stable, as their prototype may change in the future.
- * You can use them for tests, provide feedback, or if you can endure risk of future changes.
- */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Streaming functions
-***************************************/
-
-typedef struct ZSTD_DCtx_s ZSTD_DCtx;
-
-/*
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTD_magicNumber 0xFD2FB523   /* v0.3 */
-
-
-#if defined (__cplusplus)
-}
-#endif
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/****************************************************************
-*  Tuning parameters
-****************************************************************/
-/* MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSE_MAX_MEMORY_USAGE 14
-#define FSE_DEFAULT_MEMORY_USAGE 13
-
-/* FSE_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSE_MAX_SYMBOL_VALUE 255
-
-
-/****************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-
-
-/****************************************************************
-*  Byte symbol type
-****************************************************************/
-#endif   /* !FSE_COMMONDEFS_ONLY */
-
-
-/****************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/****************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-/****************************************************************
-*  Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG  (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-#error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-
-/****************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/****************************************************************
-*  Complex types
-****************************************************************/
-typedef U32 DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-
-/****************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-
-/* Function templates */
-
-#define FSE_DECODE_TYPE FSE_decode_t
-
-static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
-
-static size_t FSE_buildDTable
-(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* ptr = dt+1;
-    FSE_DTableHeader DTableH;
-    FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*)ptr;
-    const U32 tableSize = 1 << tableLog;
-    const U32 tableMask = tableSize-1;
-    const U32 step = FSE_tableStep(tableSize);
-    U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-    U32 position = 0;
-    U32 highThreshold = tableSize-1;
-    const S16 largeLimit= (S16)(1 << (tableLog-1));
-    U32 noLarge = 1;
-    U32 s;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    DTableH.tableLog = (U16)tableLog;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        if (normalizedCounter[s]==-1)
-        {
-            tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
-            symbolNext[s] = 1;
-        }
-        else
-        {
-            if (normalizedCounter[s] >= largeLimit) noLarge=0;
-            symbolNext[s] = normalizedCounter[s];
-        }
-    }
-
-    /* Spread symbols */
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        int i;
-        for (i=0; i<normalizedCounter[s]; i++)
-        {
-            tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
-            position = (position + step) & tableMask;
-            while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }
-    }
-
-    if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-
-    /* Build Decoding table */
-    {
-        U32 i;
-        for (i=0; i<tableSize; i++)
-        {
-            FSE_FUNCTION_TYPE symbol = (FSE_FUNCTION_TYPE)(tableDecode[i].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[i].nbBits = (BYTE) (tableLog - BIT_highbit32 ((U32)nextState) );
-            tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
-        }
-    }
-
-    DTableH.fastMode = (U16)noLarge;
-    memcpy(dt, &DTableH, sizeof(DTableH));
-    return 0;
-}
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-/******************************************
-*  FSE helper functions
-******************************************/
-static unsigned FSE_isError(size_t code) { return ERR_isError(code); }
-
-
-/****************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-static short FSE_abs(short a)
-{
-    return a<0 ? -a : a;
-}
-
-static size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr))
-    {
-        if (previous0)
-        {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF)
-            {
-                n0+=24;
-                if (ip < iend-5)
-                {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                }
-                else
-                {
-                    bitStream >>= 16;
-                    bitCount+=16;
-                }
-            }
-            while ((bitStream & 3) == 3)
-            {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-            {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {
-            const short max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max)
-            {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            }
-            else
-            {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSE_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold)
-            {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            {
-                if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-                {
-                    ip += bitCount>>3;
-                    bitCount &= 7;
-                }
-                else
-                {
-                    bitCount -= (int)(8 * (iend - 4 - ip));
-                    ip = iend - 4;
-                }
-                bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-            }
-        }
-    }
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-
-
-/*********************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const cell = (FSE_decode_t*)(ptr) + 1;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    FSE_decode_t* const dinfo = (FSE_decode_t*)(ptr) + 1;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSE_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BIT_DStream_t bitD;
-    FSE_DState_t state1;
-    FSE_DState_t state2;
-    size_t errorCode;
-
-    /* Init */
-    errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-    if (FSE_isError(errorCode)) return errorCode;
-
-    FSE_initDState(&state1, &bitD, dt);
-    FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BIT_reloadDStream(&bitD)==BIT_DStream_unfinished) && (op<olimit) ; op+=4)
-    {
-        op[0] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[1] = FSE_GETSYMBOL(&state2);
-
-        if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[3] = FSE_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */
-    while (1)
-    {
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state1))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state1);
-
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state2))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state2);
-    }
-
-    /* end ? */
-    if (BIT_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2))
-        return op-ostart;
-
-    if (op==omax) return ERROR(dstSize_tooSmall);   /* dst buffer is full, but cSrc unfinished */
-
-    return ERROR(corruption_detected);
-}
-
-
-static size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));
-
-    /* select fast mode (static) */
-    if (DTableH.fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-static size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSE_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-    size_t errorCode;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSE decoding mode */
-    errorCode = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSE_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    errorCode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
-    if (FSE_isError(errorCode)) return errorCode;
-
-    /* always return, even if it is an error code */
-    return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);
-}
-
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+Huff0 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/****************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-/****************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-/****************************************************************
-*  Error Management
-****************************************************************/
-#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/******************************************
-*  Helper functions
-******************************************/
-static unsigned HUF_isError(size_t code) { return ERR_isError(code); }
-
-#define HUF_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUF_MAX_TABLELOG  12           /* max configured tableLog (for static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */
-#define HUF_DEFAULT_TABLELOG  HUF_MAX_TABLELOG   /* tableLog by default, when not specified */
-#define HUF_MAX_SYMBOL_VALUE 255
-#if (HUF_MAX_TABLELOG > HUF_ABSOLUTEMAX_TABLELOG)
-#  error "HUF_MAX_TABLELOG is too large !"
-#endif
-
-
-
-/*********************************************************
-*  Huff0 : Huffman block decompression
-*********************************************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX2;   /* single-symbol decoding */
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-/*! HUF_readStats
-    Read compact Huffman tree, saved by HUF_writeCTable
-    @huffWeight : destination buffer
-    @return : size read from `src`
-*/
-static size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                            U32* nbSymbolsPtr, U32* tableLogPtr,
-                            const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    U32 tableLog;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-    U32 n;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  /* special header */
-    {
-        if (iSize >= (242))   /* RLE */
-        {
-            static int l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else   /* Incompressible */
-        {
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            for (n=0; n<oSize; n+=2)
-            {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-            }
-        }
-    }
-    else  /* header compressed with FSE (normal case) */
-    {
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSE_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSE_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUF_ABSOLUTEMAX_TABLELOG + 1) * sizeof(U32));
-    weightTotal = 0;
-    for (n=0; n<oSize; n++)
-    {
-        if (huffWeight[n] >= HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-        rankStats[huffWeight[n]]++;
-        weightTotal += (1 << huffWeight[n]) >> 1;
-    }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    tableLog = BIT_highbit32(weightTotal) + 1;
-    if (tableLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-    {
-        U32 total = 1 << tableLog;
-        U32 rest = total - weightTotal;
-        U32 verif = 1 << BIT_highbit32(rest);
-        U32 lastWeight = BIT_highbit32(rest) + 1;
-        if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-        huffWeight[oSize] = (BYTE)lastWeight;
-        rankStats[lastWeight]++;
-    }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    *tableLogPtr = tableLog;
-    return iSize+1;
-}
-
-
-/**************************/
-/* single-symbol decoding */
-/**************************/
-
-static size_t HUF_readDTableX2 (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize = ip[0];
-    U32 nbSymbols = 0;
-    U32 n;
-    U32 nextRankStart;
-    void* ptr = DTable+1;
-    HUF_DEltX2* const dt = (HUF_DEltX2*)(ptr);
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX2) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(huffWeight, HUF_MAX_SYMBOL_VALUE + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > DTable[0]) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-    DTable[0] = (U16)tableLog;   /* maybe should separate sizeof DTable, as allocated, from used size of DTable, in case of DTable re-use */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<=tableLog; n++)
-    {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<nbSymbols; n++)
-    {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUF_DEltX2 D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize;
-}
-
-static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, const U32 dtLog)
-{
-        const size_t val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-        const BYTE c = dt[val].byte;
-        BIT_skipBits(Dstream, dt[val].nbBits);
-        return c;
-}
-
-#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUF_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4))
-    {
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd))
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-
-static size_t HUF_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-
-        const void* ptr = DTable;
-        const HUF_DEltX2* const dt = ((const HUF_DEltX2*)ptr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUF_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUF_decompress4X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/***************************/
-/* double-symbols decoding */
-/***************************/
-
-static void HUF_fillDTableX4Level2(HUF_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUF_DEltX4 DElt;
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    U32 s;
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1)
-    {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)   /* note : sortedSymbols already skipped */
-    {
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }
-}
-
-typedef U32 rankVal_t[HUF_ABSOLUTEMAX_TABLELOG][HUF_ABSOLUTEMAX_TABLELOG + 1];
-
-static void HUF_fillDTableX4(HUF_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)
-    {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits)   /* enough room for a second symbol */
-        {
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUF_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        }
-        else
-        {
-            U32 i;
-            const U32 end = start + length;
-            HUF_DEltX4 DElt;
-
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits   = (BYTE)(nbBits);
-            DElt.length   = 1;
-            for (i = start; i < end; i++)
-                DTable[i] = DElt;
-        }
-        rankVal[weight] += length;
-    }
-}
-
-static size_t HUF_readDTableX4 (U32* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUF_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUF_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUF_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    const U32 memLog = DTable[0];
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize = ip[0];
-    void* ptr = DTable;
-    HUF_DEltX4* const dt = ((HUF_DEltX4*)ptr) + 1;
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX4) == sizeof(U32));   /* if compilation fails here, assertion is false */
-    if (memLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(weightList, HUF_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--)
-        { if (!maxW) return ERROR(GENERIC); }  /* necessarily finds a solution before maxW==0 */
-
-    /* Get start index of each weight */
-    {
-        U32 w, nextRankStart = 0;
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {
-        U32 s;
-        for (s=0; s<nbSymbols; s++)
-        {
-            U32 w = weightList[s];
-            U32 r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {
-        const U32 minBits = tableLog+1 - maxW;
-        U32 nextRankVal = 0;
-        U32 w, consumed;
-        const int rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-        U32* rankVal0 = rankVal[0];
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankVal;
-            nextRankVal += rankStats[w] << (w+rescale);
-            rankVal0[w] = current;
-        }
-        for (consumed = minBits; consumed <= memLog - minBits; consumed++)
-        {
-            U32* rankValPtr = rankVal[consumed];
-            for (w = 1; w <= maxW; w++)
-            {
-                rankValPtr[w] = rankVal0[w] >> consumed;
-            }
-        }
-    }
-
-    HUF_fillDTableX4(dt, memLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    return iSize;
-}
-
-
-static U32 HUF_decodeSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BIT_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits);
-    else
-    {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8))
-        {
-            BIT_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-        }
-    }
-    return 1;
-}
-
-
-#define HUF_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd-7))
-    {
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-2))
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUF_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-
-
-static size_t HUF_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-
-        const void* ptr = DTable;
-        const HUF_DEltX4* const dt = ((const HUF_DEltX4*)ptr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUF_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUF_decompress4X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/**********************************/
-/* Generic decompression selector */
-/**********************************/
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-static size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[3] = { HUF_decompress4X2, HUF_decompress4X4, NULL };
-    /* estimate decompression time */
-    U32 Q;
-    const U32 D256 = (U32)(dstSize >> 8);
-    U32 Dtime[3];
-    U32 algoNb = 0;
-    int n;
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    /* decoder timing evaluation */
-    Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-    for (n=0; n<3; n++)
-        Dtime[n] = algoTime[Q][n].tableTime + (algoTime[Q][n].decode256Time * D256);
-
-    Dtime[1] += Dtime[1] >> 4; Dtime[2] += Dtime[2] >> 3; /* advantage to algorithms using less memory, for cache eviction */
-
-    if (Dtime[1] < Dtime[0]) algoNb = 1;
-
-    return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-
-    //return HUF_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUF_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-    //return HUF_decompress4X6(dst, dstSize, cSrc, cSrcSize);   /* multi-streams quad-symbols decoding */
-}
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
-*  MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*/
-#define ZSTD_MEMORY_USAGE 17
-
-/*!
- * HEAPMODE :
- * Select how default compression functions will allocate memory for their hash table,
- * in memory stack (0, fastest), or in memory heap (1, requires malloc())
- * Note that compression context is fairly large, as a consequence heap memory is recommended.
- */
-#ifndef ZSTD_HEAPMODE
-#  define ZSTD_HEAPMODE 1
-#endif /* ZSTD_HEAPMODE */
-
-/*!
-*  LEGACY_SUPPORT :
-*  decompressor can decode older formats (starting from Zstd 0.1+)
-*/
-#ifndef ZSTD_LEGACY_SUPPORT
-#  define ZSTD_LEGACY_SUPPORT 1
-#endif
-
-
-/* *******************************************************
-*  Includes
-*********************************************************/
-#include <stdlib.h>      /* calloc */
-#include <string.h>      /* memcpy, memmove */
-#include <stdio.h>       /* debug : printf */
-
-
-/* *******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef __AVX2__
-#  include <immintrin.h>   /* AVX2 intrinsics */
-#endif
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#else
-#  define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__)
-#endif
-
-
-/* *******************************************************
-*  Constants
-*********************************************************/
-#define HASH_LOG (ZSTD_MEMORY_USAGE - 2)
-#define HASH_TABLESIZE (1 << HASH_LOG)
-#define HASH_MASK (HASH_TABLESIZE - 1)
-
-#define KNUTH 2654435761
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BLOCKSIZE (128 KB)                 /* define, for static allocation */
-#define MIN_SEQUENCES_SIZE (2 /*seqNb*/ + 2 /*dumps*/ + 3 /*seqTables*/ + 1 /*bitStream*/)
-#define MIN_CBLOCK_SIZE (3 /*litCSize*/ + MIN_SEQUENCES_SIZE)
-#define IS_RAW BIT0
-#define IS_RLE BIT1
-
-#define WORKPLACESIZE (BLOCKSIZE*3)
-#define MINMATCH 4
-#define MLbits   7
-#define LLbits   6
-#define Offbits  5
-#define MaxML  ((1<<MLbits )-1)
-#define MaxLL  ((1<<LLbits )-1)
-#define MaxOff   31
-#define LitFSELog  11
-#define MLFSELog   10
-#define LLFSELog   10
-#define OffFSELog   9
-#define MAX(a,b) ((a)<(b)?(b):(a))
-#define MaxSeq MAX(MaxLL, MaxML)
-
-#define LITERAL_NOENTROPY 63
-#define COMMAND_NOENTROPY 7   /* to remove */
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-static const size_t ZSTD_blockHeaderSize = 3;
-static const size_t ZSTD_frameHeaderSize = 4;
-
-
-/* *******************************************************
-*  Memory operations
-**********************************************************/
-static void   ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-static void   ZSTD_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s) { ZSTD_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTD_wildcopy : custom version of memcpy(), can copy up to 7-8 bytes too many */
-static void ZSTD_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do COPY8(op, ip) while (op < oend);
-}
-
-
-/* **************************************
-*  Local structures
-****************************************/
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* offCode;
-    BYTE* litStart;
-    BYTE* lit;
-    BYTE* litLengthStart;
-    BYTE* litLength;
-    BYTE* matchLengthStart;
-    BYTE* matchLength;
-    BYTE* dumpsStart;
-    BYTE* dumps;
-} seqStore_t;
-
-
-/* *************************************
-*  Error Management
-***************************************/
-/*! ZSTD_isError
-*   tells if a return value is an error code */
-static unsigned ZSTD_isError(size_t code) { return ERR_isError(code); }
-
-
-
-/* *************************************************************
-*   Decompression section
-***************************************************************/
-struct ZSTD_DCtx_s
-{
-    U32 LLTable[FSE_DTABLE_SIZE_U32(LLFSELog)];
-    U32 OffTable[FSE_DTABLE_SIZE_U32(OffFSELog)];
-    U32 MLTable[FSE_DTABLE_SIZE_U32(MLFSELog)];
-    void* previousDstEnd;
-    void* base;
-    size_t expected;
-    blockType_t bType;
-    U32 phase;
-    const BYTE* litPtr;
-    size_t litSize;
-    BYTE litBuffer[BLOCKSIZE + 8 /* margin for wildcopy */];
-};   /* typedef'd to ZSTD_Dctx within "zstd_static.h" */
-
-
-static size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    BYTE headerFlags;
-    U32 cSize;
-
-    if (srcSize < 3) return ERROR(srcSize_wrong);
-
-    headerFlags = *in;
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-
-    bpPtr->blockType = (blockType_t)(headerFlags >> 6);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-static size_t ZSTD_copyUncompressedBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    if (srcSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/** ZSTD_decompressLiterals
-    @return : nb of bytes read from src, or an error code*/
-static size_t ZSTD_decompressLiterals(void* dst, size_t* maxDstSizePtr,
-                                const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-
-    const size_t litSize = (MEM_readLE32(src) & 0x1FFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-    const size_t litCSize = (MEM_readLE32(ip+2) & 0xFFFFFF) >> 5;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-
-    if (litSize > *maxDstSizePtr) return ERROR(corruption_detected);
-    if (litCSize + 5 > srcSize) return ERROR(corruption_detected);
-
-    if (HUF_isError(HUF_decompress(dst, litSize, ip+5, litCSize))) return ERROR(corruption_detected);
-
-    *maxDstSizePtr = litSize;
-    return litCSize + 5;
-}
-
-
-/** ZSTD_decodeLiteralsBlock
-    @return : nb of bytes read from src (< srcSize )*/
-static size_t ZSTD_decodeLiteralsBlock(void* ctx,
-                          const void* src, size_t srcSize)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)ctx;
-    const BYTE* const istart = (const BYTE* const)src;
-
-    /* any compressed block with literals segment must be at least this size */
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch(*istart & 3)
-    {
-    default:
-    case 0:
-        {
-            size_t litSize = BLOCKSIZE;
-            const size_t readSize = ZSTD_decompressLiterals(dctx->litBuffer, &litSize, src, srcSize);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, 8);
-            return readSize;   /* works if it's an error too */
-        }
-    case IS_RAW:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > srcSize-11)   /* risk of reading too far with wildcopy */
-            {
-                if (litSize > srcSize-3) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, 8);
-                return litSize+3;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+3;
-            dctx->litSize = litSize;
-            return litSize+3;
-        }
-    case IS_RLE:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > BLOCKSIZE) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[3], litSize + 8);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return 4;
-        }
-    }
-}
-
-
-static size_t ZSTD_decodeSeqHeaders(int* nbSeq, const BYTE** dumpsPtr, size_t* dumpsLengthPtr,
-                         FSE_DTable* DTableLL, FSE_DTable* DTableML, FSE_DTable* DTableOffb,
-                         const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    const BYTE* const iend = istart + srcSize;
-    U32 LLtype, Offtype, MLtype;
-    U32 LLlog, Offlog, MLlog;
-    size_t dumpsLength;
-
-    /* check */
-    if (srcSize < 5) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    *nbSeq = MEM_readLE16(ip); ip+=2;
-    LLtype  = *ip >> 6;
-    Offtype = (*ip >> 4) & 3;
-    MLtype  = (*ip >> 2) & 3;
-    if (*ip & 2)
-    {
-        dumpsLength  = ip[2];
-        dumpsLength += ip[1] << 8;
-        ip += 3;
-    }
-    else
-    {
-        dumpsLength  = ip[1];
-        dumpsLength += (ip[0] & 1) << 8;
-        ip += 2;
-    }
-    *dumpsPtr = ip;
-    ip += dumpsLength;
-    *dumpsLengthPtr = dumpsLength;
-
-    /* check */
-    if (ip > iend-3) return ERROR(srcSize_wrong); /* min : all 3 are "raw", hence no header, but at least xxLog bits per type */
-
-    /* sequences */
-    {
-        S16 norm[MaxML+1];    /* assumption : MaxML >= MaxLL and MaxOff */
-        size_t headerSize;
-
-        /* Build DTables */
-        switch(LLtype)
-        {
-        case bt_rle :
-            LLlog = 0;
-            FSE_buildDTable_rle(DTableLL, *ip++); break;
-        case bt_raw :
-            LLlog = LLbits;
-            FSE_buildDTable_raw(DTableLL, LLbits); break;
-        default :
-            {   U32 max = MaxLL;
-                headerSize = FSE_readNCount(norm, &max, &LLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (LLlog > LLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableLL, norm, max, LLlog);
-        }   }
-
-        switch(Offtype)
-        {
-        case bt_rle :
-            Offlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong);   /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableOffb, *ip++ & MaxOff); /* if *ip > MaxOff, data is corrupted */
-            break;
-        case bt_raw :
-            Offlog = Offbits;
-            FSE_buildDTable_raw(DTableOffb, Offbits); break;
-        default :
-            {   U32 max = MaxOff;
-                headerSize = FSE_readNCount(norm, &max, &Offlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (Offlog > OffFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableOffb, norm, max, Offlog);
-        }   }
-
-        switch(MLtype)
-        {
-        case bt_rle :
-            MLlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableML, *ip++); break;
-        case bt_raw :
-            MLlog = MLbits;
-            FSE_buildDTable_raw(DTableML, MLbits); break;
-        default :
-            {   U32 max = MaxML;
-                headerSize = FSE_readNCount(norm, &max, &MLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (MLlog > MLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableML, norm, max, MLlog);
-    }   }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t offset;
-    size_t matchLength;
-} seq_t;
-
-typedef struct {
-    BIT_DStream_t DStream;
-    FSE_DState_t stateLL;
-    FSE_DState_t stateOffb;
-    FSE_DState_t stateML;
-    size_t prevOffset;
-    const BYTE* dumps;
-    const BYTE* dumpsEnd;
-} seqState_t;
-
-
-static void ZSTD_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    size_t litLength;
-    size_t prevOffset;
-    size_t offset;
-    size_t matchLength;
-    const BYTE* dumps = seqState->dumps;
-    const BYTE* const de = seqState->dumpsEnd;
-
-    /* Literal length */
-    litLength = FSE_decodeSymbol(&(seqState->stateLL), &(seqState->DStream));
-    prevOffset = litLength ? seq->offset : seqState->prevOffset;
-    seqState->prevOffset = seq->offset;
-    if (litLength == MaxLL)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) litLength += add;
-        else if (dumps + 3 <= de)
-        {
-            litLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) dumps = de-1;   /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-
-    /* Offset */
-    {
-        static const size_t offsetPrefix[MaxOff+1] = {  /* note : size_t faster than U32 */
-                1 /*fake*/, 1, 2, 4, 8, 16, 32, 64, 128, 256,
-                512, 1024, 2048, 4096, 8192, 16384, 32768, 65536, 131072, 262144,
-                524288, 1048576, 2097152, 4194304, 8388608, 16777216, 33554432, /*fake*/ 1, 1, 1, 1, 1 };
-        U32 offsetCode, nbBits;
-        offsetCode = FSE_decodeSymbol(&(seqState->stateOffb), &(seqState->DStream));   /* <= maxOff, by table construction */
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        nbBits = offsetCode - 1;
-        if (offsetCode==0) nbBits = 0;   /* cmove */
-        offset = offsetPrefix[offsetCode] + BIT_readBits(&(seqState->DStream), nbBits);
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        if (offsetCode==0) offset = prevOffset;   /* cmove */
-    }
-
-    /* MatchLength */
-    matchLength = FSE_decodeSymbol(&(seqState->stateML), &(seqState->DStream));
-    if (matchLength == MaxML)
-    {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) matchLength += add;
-        else if (dumps + 3 <= de)
-        {
-            matchLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) dumps = de-1;   /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-    matchLength += MINMATCH;
-
-    /* save result */
-    seq->litLength = litLength;
-    seq->offset = offset;
-    seq->matchLength = matchLength;
-    seqState->dumps = dumps;
-}
-
-
-static size_t ZSTD_execSequence(BYTE* op,
-                                seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                BYTE* const base, BYTE* const oend)
-{
-    static const int dec32table[] = {0, 1, 2, 1, 4, 4, 4, 4};   /* added */
-    static const int dec64table[] = {8, 8, 8, 7, 8, 9,10,11};   /* subtracted */
-    const BYTE* const ostart = op;
-    BYTE* const oLitEnd = op + sequence.litLength;
-    BYTE* const oMatchEnd = op + sequence.litLength + sequence.matchLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_8 = oend-8;
-    const BYTE* const litEnd = *litPtr + sequence.litLength;
-
-    /* checks */
-    if (oLitEnd > oend_8) return ERROR(dstSize_tooSmall);   /* last match must start at a minimum distance of 8 from oend */
-    if (oMatchEnd > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (litEnd > litLimit) return ERROR(corruption_detected);   /* overRead beyond lit buffer */
-
-    /* copy Literals */
-    ZSTD_wildcopy(op, *litPtr, sequence.litLength);   /* note : oLitEnd <= oend-8 : no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = litEnd;   /* update for next sequence */
-
-    /* copy Match */
-    {
-        const BYTE* match = op - sequence.offset;
-
-        /* check */
-        if (sequence.offset > (size_t)op) return ERROR(corruption_detected);   /* address space overflow test (this test seems kept by clang optimizer) */
-        //if (match > op) return ERROR(corruption_detected);   /* address space overflow test (is clang optimizer removing this test ?) */
-        if (match < base) return ERROR(corruption_detected);
-
-        /* close range match, overlap */
-        if (sequence.offset < 8)
-        {
-            const int dec64 = dec64table[sequence.offset];
-            op[0] = match[0];
-            op[1] = match[1];
-            op[2] = match[2];
-            op[3] = match[3];
-            match += dec32table[sequence.offset];
-            ZSTD_copy4(op+4, match);
-            match -= dec64;
-        }
-        else
-        {
-            ZSTD_copy8(op, match);
-        }
-        op += 8; match += 8;
-
-        if (oMatchEnd > oend-(16-MINMATCH))
-        {
-            if (op < oend_8)
-            {
-                ZSTD_wildcopy(op, match, oend_8 - op);
-                match += oend_8 - op;
-                op = oend_8;
-            }
-            while (op < oMatchEnd) *op++ = *match++;
-        }
-        else
-        {
-            ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-        }
-    }
-
-    return oMatchEnd - ostart;
-}
-
-static size_t ZSTD_decompressSequences(
-                               void* ctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)ctx;
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t errorCode, dumpsLength;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    int nbSeq;
-    const BYTE* dumps;
-    U32* DTableLL = dctx->LLTable;
-    U32* DTableML = dctx->MLTable;
-    U32* DTableOffb = dctx->OffTable;
-    BYTE* const base = (BYTE*) (dctx->base);
-
-    /* Build Decoding Tables */
-    errorCode = ZSTD_decodeSeqHeaders(&nbSeq, &dumps, &dumpsLength,
-                                      DTableLL, DTableML, DTableOffb,
-                                      ip, iend-ip);
-    if (ZSTD_isError(errorCode)) return errorCode;
-    ip += errorCode;
-
-    /* Regen sequences */
-    {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        seqState.dumps = dumps;
-        seqState.dumpsEnd = dumps + dumpsLength;
-        seqState.prevOffset = sequence.offset = 4;
-        errorCode = BIT_initDStream(&(seqState.DStream), ip, iend-ip);
-        if (ERR_isError(errorCode)) return ERROR(corruption_detected);
-        FSE_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSE_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSE_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BIT_reloadDStream(&(seqState.DStream)) <= BIT_DStream_completed) && (nbSeq>0) ; )
-        {
-            size_t oneSeqSize;
-            nbSeq--;
-            ZSTD_decodeSequence(&sequence, &seqState);
-            oneSeqSize = ZSTD_execSequence(op, sequence, &litPtr, litEnd, base, oend);
-            if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* check if reached exact end */
-        if ( !BIT_endOfDStream(&(seqState.DStream)) ) return ERROR(corruption_detected);   /* requested too much : data is corrupted */
-        if (nbSeq<0) return ERROR(corruption_detected);   /* requested too many sequences : data is corrupted */
-
-        /* last literal segment */
-        {
-            size_t lastLLSize = litEnd - litPtr;
-            if (litPtr > litEnd) return ERROR(corruption_detected);
-            if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-            if (op != litPtr) memmove(op, litPtr, lastLLSize);
-            op += lastLLSize;
-        }
-    }
-
-    return op-ostart;
-}
-
-
-static size_t ZSTD_decompressBlock(
-                            void* ctx,
-                            void* dst, size_t maxDstSize,
-                      const void* src, size_t srcSize)
-{
-    /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-
-    /* Decode literals sub-block */
-    size_t litCSize = ZSTD_decodeLiteralsBlock(ctx, src, srcSize);
-    if (ZSTD_isError(litCSize)) return litCSize;
-    ip += litCSize;
-    srcSize -= litCSize;
-
-    return ZSTD_decompressSequences(ctx, dst, maxDstSize, ip, srcSize);
-}
-
-
-static size_t ZSTD_decompressDCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t remainingSize = srcSize;
-    U32 magicNumber;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t decodedSize=0;
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTD_decompressBlock(ctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTD_copyUncompressedBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        if (ZSTD_isError(decodedSize)) return decodedSize;
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-static size_t ZSTD_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    ZSTD_DCtx ctx;
-    ctx.base = dst;
-    return ZSTD_decompressDCtx(&ctx, dst, maxDstSize, src, srcSize);
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-MEM_STATIC void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv03_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    U32 magicNumber;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize+ZSTD_blockHeaderSize) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_magicNumber) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-        return;
-    }
-    ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * BLOCKSIZE;
-}
-
-
-/*******************************
-*  Streaming Decompression API
-*******************************/
-
-static size_t ZSTD_resetDCtx(ZSTD_DCtx* dctx)
-{
-    dctx->expected = ZSTD_frameHeaderSize;
-    dctx->phase = 0;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    return 0;
-}
-
-static ZSTD_DCtx* ZSTD_createDCtx(void)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)malloc(sizeof(ZSTD_DCtx));
-    if (dctx==NULL) return NULL;
-    ZSTD_resetDCtx(dctx);
-    return dctx;
-}
-
-static size_t ZSTD_freeDCtx(ZSTD_DCtx* dctx)
-{
-    free(dctx);
-    return 0;
-}
-
-static size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-static size_t ZSTD_decompressContinue(ZSTD_DCtx* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != ctx->expected) return ERROR(srcSize_wrong);
-    if (dst != ctx->previousDstEnd)  /* not contiguous */
-        ctx->base = dst;
-
-    /* Decompress : frame header */
-    if (ctx->phase == 0)
-    {
-        /* Check frame magic header */
-        U32 magicNumber = MEM_readLE32(src);
-        if (magicNumber != ZSTD_magicNumber) return ERROR(prefix_unknown);
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        return 0;
-    }
-
-    /* Decompress : block header */
-    if (ctx->phase == 1)
-    {
-        blockProperties_t bp;
-        size_t blockSize = ZSTD_getcBlockSize(src, ZSTD_blockHeaderSize, &bp);
-        if (ZSTD_isError(blockSize)) return blockSize;
-        if (bp.blockType == bt_end)
-        {
-            ctx->expected = 0;
-            ctx->phase = 0;
-        }
-        else
-        {
-            ctx->expected = blockSize;
-            ctx->bType = bp.blockType;
-            ctx->phase = 2;
-        }
-
-        return 0;
-    }
-
-    /* Decompress : block content */
-    {
-        size_t rSize;
-        switch(ctx->bType)
-        {
-        case bt_compressed:
-            rSize = ZSTD_decompressBlock(ctx, dst, maxDstSize, src, srcSize);
-            break;
-        case bt_raw :
-            rSize = ZSTD_copyUncompressedBlock(dst, maxDstSize, src, srcSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet handled */
-            break;
-        case bt_end :   /* should never happen (filtered at phase 1) */
-            rSize = 0;
-            break;
-        default:
-            return ERROR(GENERIC);
-        }
-        ctx->phase = 1;
-        ctx->expected = ZSTD_blockHeaderSize;
-        ctx->previousDstEnd = (void*)( ((char*)dst) + rSize);
-        return rSize;
-    }
-
-}
-
-
-/* wrapper layer */
-
-unsigned ZSTDv03_isError(size_t code)
-{
-    return ZSTD_isError(code);
-}
-
-size_t ZSTDv03_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize)
-{
-    return ZSTD_decompress(dst, maxOriginalSize, src, compressedSize);
-}
-
-ZSTDv03_Dctx* ZSTDv03_createDCtx(void)
-{
-    return (ZSTDv03_Dctx*)ZSTD_createDCtx();
-}
-
-size_t ZSTDv03_freeDCtx(ZSTDv03_Dctx* dctx)
-{
-    return ZSTD_freeDCtx((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv03_resetDCtx(ZSTDv03_Dctx* dctx)
-{
-    return ZSTD_resetDCtx((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv03_nextSrcSizeToDecompress(ZSTDv03_Dctx* dctx)
-{
-    return ZSTD_nextSrcSizeToDecompress((ZSTD_DCtx*)dctx);
-}
-
-size_t ZSTDv03_decompressContinue(ZSTDv03_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return ZSTD_decompressContinue((ZSTD_DCtx*)dctx, dst, maxDstSize, src, srcSize);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstd_v03.h b/vendor/github.com/DataDog/zstd/zstd_v03.h
deleted file mode 100644
index efd8c2b..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v03.h
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_V03_H_298734209782
-#define ZSTD_V03_H_298734209782
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Simple one-step function
-***************************************/
-/**
-ZSTDv03_decompress() : decompress ZSTD frames compliant with v0.3.x format
-    compressedSize : is the exact source size
-    maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
-                      It must be equal or larger than originalSize, otherwise decompression will fail.
-    return : the number of bytes decompressed into destination buffer (originalSize)
-             or an errorCode if it fails (which can be tested using ZSTDv01_isError())
-*/
-size_t ZSTDv03_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize);
-
- /**
- ZSTDv03_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.3.x format
-     srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-     cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                 or an error code if it fails (which can be tested using ZSTDv01_isError())
-     dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                 or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
- */
- void ZSTDv03_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                      size_t* cSize, unsigned long long* dBound);
-
-    /**
-ZSTDv03_isError() : tells if the result of ZSTDv03_decompress() is an error
-*/
-unsigned ZSTDv03_isError(size_t code);
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTDv03_Dctx_s ZSTDv03_Dctx;
-ZSTDv03_Dctx* ZSTDv03_createDCtx(void);
-size_t ZSTDv03_freeDCtx(ZSTDv03_Dctx* dctx);
-
-size_t ZSTDv03_decompressDCtx(void* ctx,
-                              void* dst, size_t maxOriginalSize,
-                        const void* src, size_t compressedSize);
-
-/* *************************************
-*  Streaming functions
-***************************************/
-size_t ZSTDv03_resetDCtx(ZSTDv03_Dctx* dctx);
-
-size_t ZSTDv03_nextSrcSizeToDecompress(ZSTDv03_Dctx* dctx);
-size_t ZSTDv03_decompressContinue(ZSTDv03_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
-/**
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTDv03_magicNumber 0xFD2FB523   /* v0.3 */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_V03_H_298734209782 */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v04.c b/vendor/github.com/DataDog/zstd/zstd_v04.c
deleted file mode 100644
index 645a6e3..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v04.c
+++ /dev/null
@@ -1,3637 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
- /******************************************
- *  Includes
- ******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include <string.h>    /* memcpy */
-
-#include "zstd_v04.h"
-#include "error_private.h"
-
-
-/* ******************************************************************
- *   mem.h
- *******************************************************************/
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/******************************************
-*  Compiler-specific
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/****************************************************************
-*  Basic Types
-*****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/*-*************************************
-*  Debug
-***************************************/
-#include "debug.h"
-#ifndef assert
-#  define assert(condition) ((void)0)
-#endif
-
-
-/****************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets generating assembly depending on alignment.
- *            But in some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(void*)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(void*)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-#endif // MEM_FORCE_MEMORY_ACCESS
-
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian())
-    {
-        MEM_write16(memPtr, val);
-    }
-    else
-    {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE24(const void* memPtr)
-{
-    return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24));
-    }
-}
-
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-    {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24)
-                     + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56));
-    }
-}
-
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-
-/*
-    zstd - standard compression library
-    Header File for static linking only
-*/
-#ifndef ZSTD_STATIC_H
-#define ZSTD_STATIC_H
-
-
-/* *************************************
-*  Types
-***************************************/
-#define ZSTD_WINDOWLOG_ABSOLUTEMIN 11
-
-/** from faster to stronger */
-typedef enum { ZSTD_fast, ZSTD_greedy, ZSTD_lazy, ZSTD_lazy2, ZSTD_btlazy2 } ZSTD_strategy;
-
-typedef struct
-{
-    U64 srcSize;       /* optional : tells how much bytes are present in the frame. Use 0 if not known. */
-    U32 windowLog;     /* largest match distance : larger == more compression, more memory needed during decompression */
-    U32 contentLog;    /* full search segment : larger == more compression, slower, more memory (useless for fast) */
-    U32 hashLog;       /* dispatch table : larger == more memory, faster */
-    U32 searchLog;     /* nb of searches : larger == more compression, slower */
-    U32 searchLength;  /* size of matches : larger == faster decompression, sometimes less compression */
-    ZSTD_strategy strategy;
-} ZSTD_parameters;
-
-typedef ZSTDv04_Dctx ZSTD_DCtx;
-
-/* *************************************
-*  Advanced functions
-***************************************/
-/** ZSTD_decompress_usingDict
-*   Same as ZSTD_decompressDCtx, using a Dictionary content as prefix
-*   Note : dict can be NULL, in which case, it's equivalent to ZSTD_decompressDCtx() */
-static size_t ZSTD_decompress_usingDict(ZSTD_DCtx* ctx,
-                                             void* dst, size_t maxDstSize,
-                                       const void* src, size_t srcSize,
-                                       const void* dict,size_t dictSize);
-
-
-/* **************************************
-*  Streaming functions (direct mode)
-****************************************/
-static size_t ZSTD_resetDCtx(ZSTD_DCtx* dctx);
-static size_t ZSTD_getFrameParams(ZSTD_parameters* params, const void* src, size_t srcSize);
-static void   ZSTD_decompress_insertDictionary(ZSTD_DCtx* ctx, const void* src, size_t srcSize);
-
-static size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx);
-static size_t ZSTD_decompressContinue(ZSTD_DCtx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
-
-/**
-  Streaming decompression, bufferless mode
-
-  A ZSTD_DCtx object is required to track streaming operations.
-  Use ZSTD_createDCtx() / ZSTD_freeDCtx() to manage it.
-  A ZSTD_DCtx object can be re-used multiple times. Use ZSTD_resetDCtx() to return to fresh status.
-
-  First operation is to retrieve frame parameters, using ZSTD_getFrameParams().
-  This function doesn't consume its input. It needs enough input data to properly decode the frame header.
-  Objective is to retrieve *params.windowlog, to know minimum amount of memory required during decoding.
-  Result : 0 when successful, it means the ZSTD_parameters structure has been filled.
-           >0 : means there is not enough data into src. Provides the expected size to successfully decode header.
-           errorCode, which can be tested using ZSTD_isError() (For example, if it's not a ZSTD header)
-
-  Then, you can optionally insert a dictionary.
-  This operation must mimic the compressor behavior, otherwise decompression will fail or be corrupted.
-
-  Then it's possible to start decompression.
-  Use ZSTD_nextSrcSizeToDecompress() and ZSTD_decompressContinue() alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() requires this exact amount of bytes, or it will fail.
-  ZSTD_decompressContinue() needs previous data blocks during decompression, up to (1 << windowlog).
-  They should preferably be located contiguously, prior to current block. Alternatively, a round buffer is also possible.
-
-  @result of ZSTD_decompressContinue() is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-
-  A frame is fully decoded when ZSTD_nextSrcSizeToDecompress() returns zero.
-  Context can then be reset to start a new decompression.
-*/
-
-
-
-
-#endif  /* ZSTD_STATIC_H */
-
-
-/*
-    zstd_internal - common functions to include
-    Header File for include
-*/
-#ifndef ZSTD_CCOMMON_H_MODULE
-#define ZSTD_CCOMMON_H_MODULE
-
-/* *************************************
-*  Common macros
-***************************************/
-#define MIN(a,b) ((a)<(b) ? (a) : (b))
-#define MAX(a,b) ((a)>(b) ? (a) : (b))
-
-
-/* *************************************
-*  Common constants
-***************************************/
-#define ZSTD_MAGICNUMBER 0xFD2FB524   /* v0.4 */
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BLOCKSIZE (128 KB)                 /* define, for static allocation */
-
-static const size_t ZSTD_blockHeaderSize = 3;
-static const size_t ZSTD_frameHeaderSize_min = 5;
-#define ZSTD_frameHeaderSize_max 5         /* define, for static allocation */
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define IS_RAW BIT0
-#define IS_RLE BIT1
-
-#define MINMATCH 4
-#define REPCODE_STARTVALUE 4
-
-#define MLbits   7
-#define LLbits   6
-#define Offbits  5
-#define MaxML  ((1<<MLbits) - 1)
-#define MaxLL  ((1<<LLbits) - 1)
-#define MaxOff ((1<<Offbits)- 1)
-#define MLFSELog   10
-#define LLFSELog   10
-#define OffFSELog   9
-#define MaxSeq MAX(MaxLL, MaxML)
-
-#define MIN_SEQUENCES_SIZE (2 /*seqNb*/ + 2 /*dumps*/ + 3 /*seqTables*/ + 1 /*bitStream*/)
-#define MIN_CBLOCK_SIZE (3 /*litCSize*/ + MIN_SEQUENCES_SIZE)
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-
-/* ******************************************
-*  Shared functions to include for inlining
-********************************************/
-static void ZSTD_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s) { ZSTD_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTD_wildcopy : custom version of memcpy(), can copy up to 7-8 bytes too many */
-static void ZSTD_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do
-        COPY8(op, ip)
-    while (op < oend);
-}
-
-
-
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   header file
-****************************************************************** */
-#ifndef FSE_H
-#define FSE_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* *****************************************
-*  Includes
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-
-
-/* *****************************************
-*  FSE simple functions
-******************************************/
-static size_t FSE_decompress(void* dst,  size_t maxDstSize,
-                const void* cSrc, size_t cSrcSize);
-/*!
-FSE_decompress():
-    Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'maxDstSize'.
-    return : size of regenerated data (<= maxDstSize)
-             or an error code, which can be tested using FSE_isError()
-
-    ** Important ** : FSE_decompress() doesn't decompress non-compressible nor RLE data !!!
-    Why ? : making this distinction requires a header.
-    Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-
-
-/* *****************************************
-*  Tool functions
-******************************************/
-/* Error Management */
-static unsigned    FSE_isError(size_t code);        /* tells if a return value is an error code */
-
-
-
-/* *****************************************
-*  FSE detailed API
-******************************************/
-/*!
-FSE_compress() does the following:
-1. count symbol occurrence from source[] into table count[]
-2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
-3. save normalized counters to memory buffer using writeNCount()
-4. build encoding table 'CTable' from normalized counters
-5. encode the data stream using encoding table 'CTable'
-
-FSE_decompress() does the following:
-1. read normalized counters with readNCount()
-2. build decoding table 'DTable' from normalized counters
-3. decode the data stream using decoding table 'DTable'
-
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and provide normalized distribution using external method.
-*/
-
-
-/* *** DECOMPRESSION *** */
-
-/*!
-FSE_readNCount():
-   Read compactly saved 'normalizedCounter' from 'rBuffer'.
-   return : size read from 'rBuffer'
-            or an errorCode, which can be tested using FSE_isError()
-            maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-static  size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize);
-
-/*!
-Constructor and Destructor of type FSE_DTable
-    Note that its size depends on 'tableLog' */
-typedef unsigned FSE_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-
-/*!
-FSE_buildDTable():
-   Builds 'dt', which must be already allocated, using FSE_createDTable()
-   return : 0,
-            or an errorCode, which can be tested using FSE_isError() */
-static size_t FSE_buildDTable ( FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*!
-FSE_decompress_usingDTable():
-   Decompress compressed source 'cSrc' of size 'cSrcSize' using 'dt'
-   into 'dst' which must be already allocated.
-   return : size of regenerated data (necessarily <= maxDstSize)
-            or an errorCode, which can be tested using FSE_isError() */
-static  size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const FSE_DTable* dt);
-
-/*!
-Tutorial :
-----------
-(Note : these functions only decompress FSE-compressed blocks.
- If block is uncompressed, use memcpy() instead
- If block is a single repeated byte, use memset() instead )
-
-The first step is to obtain the normalized frequencies of symbols.
-This can be performed by FSE_readNCount() if it was saved using FSE_writeNCount().
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short.
-In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
-or size the table to handle worst case situations (typically 256).
-FSE_readNCount() will provide 'tableLog' and 'maxSymbolValue'.
-The result of FSE_readNCount() is the number of bytes read from 'rBuffer'.
-Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that.
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-The next step is to build the decompression tables 'FSE_DTable' from 'normalizedCounter'.
-This is performed by the function FSE_buildDTable().
-The space required by 'FSE_DTable' must be already allocated using FSE_createDTable().
-If there is an error, the function will return an error code, which can be tested using FSE_isError().
-
-'FSE_DTable' can then be used to decompress 'cSrc', with FSE_decompress_usingDTable().
-'cSrcSize' must be strictly correct, otherwise decompression will fail.
-FSE_decompress_usingDTable() result will tell how many bytes were regenerated (<=maxDstSize).
-If there is an error, the function will return an error code, which can be tested using FSE_isError(). (ex: dst buffer too small)
-*/
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSE_H */
-
-
-/* ******************************************************************
-   bitstream
-   Part of NewGen Entropy library
-   header file (to include)
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which highly benefit from being inlined.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-/**********************************************
-*  bitStream decompression API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BIT_DStream_t;
-
-typedef enum { BIT_DStream_unfinished = 0,
-               BIT_DStream_endOfBuffer = 1,
-               BIT_DStream_completed = 2,
-               BIT_DStream_overflow = 3 } BIT_DStream_status;  /* result of BIT_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BIT_readBits(BIT_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD);
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* bitD);
-
-
-
-
-/******************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/****************************************************************
-*  Helper functions
-****************************************************************/
-MEM_STATIC unsigned BIT_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-/**********************************************************
-* bitStream decoding
-**********************************************************/
-
-/*!BIT_initDStream
-*  Initialize a BIT_DStream_t.
-*  @bitD : a pointer to an already allocated BIT_DStream_t structure
-*  @srcBuffer must point at the beginning of a bitStream
-*  @srcSize must be the exact size of the bitStream
-*  @result : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BIT_initDStream(BIT_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(size_t))   /* normal case */
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(size_t);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-    }
-    else
-    {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);/* fall-through */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);/* fall-through */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);/* fall-through */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24; /* fall-through */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16; /* fall-through */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) <<  8; /* fall-through */
-            default: break;
-        }
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BIT_highbit32(contain32);
-        bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-MEM_STATIC size_t BIT_lookBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BIT_lookBitsFast :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_lookBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BIT_skipBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BIT_readBits(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBits(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*!BIT_readBitsFast :
-*  unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BIT_readBitsFast(BIT_DStream_t* bitD, U32 nbBits)
-{
-    size_t value = BIT_lookBitsFast(bitD, nbBits);
-    BIT_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BIT_DStream_status BIT_reloadDStream(BIT_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return BIT_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer))
-    {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BIT_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start)
-    {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BIT_DStream_endOfBuffer;
-        return BIT_DStream_completed;
-    }
-    {
-        U32 nbBytes = bitD->bitsConsumed >> 3;
-        BIT_DStream_status result = BIT_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start)
-        {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BIT_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BIT_endOfDStream
-*   @return Tells if DStream has reached its exact end
-*/
-MEM_STATIC unsigned BIT_endOfDStream(const BIT_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
-
-
-
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef FSE_STATIC_H
-#define FSE_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* *****************************************
-*  Static allocation
-*******************************************/
-/* FSE buffer bounds */
-#define FSE_NCOUNTBOUND 512
-#define FSE_BLOCKBOUND(size) (size + (size>>7))
-#define FSE_COMPRESSBOUND(size) (FSE_NCOUNTBOUND + FSE_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* It is possible to statically allocate FSE CTable/DTable as a table of unsigned using below macros */
-#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue)   (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
-#define FSE_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/* *****************************************
-*  FSE advanced API
-*******************************************/
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits);
-/* build a fake FSE_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, unsigned char symbolValue);
-/* build a fake FSE_DTable, designed to always generate the same symbolValue */
-
-
-
-/* *****************************************
-*  FSE symbol decompression API
-*******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSE_DState_t;
-
-
-static void     FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt);
-
-static unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-
-static unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
-
-
-/* *****************************************
-*  FSE unsafe API
-*******************************************/
-static unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* *****************************************
-*  Implementation of inlined functions
-*******************************************/
-/* decompression */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSE_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSE_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSE_initDState(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD, const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    memcpy(&DTableH, dt, sizeof(DTableH));
-    DStatePtr->state = BIT_readBits(bitD, DTableH.tableLog);
-    BIT_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32  nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, BIT_DStream_t* bitD)
-{
-    const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32 nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BIT_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSE_STATIC_H */
-
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#ifndef FSE_COMMONDEFS_ONLY
-
-/* **************************************************************
-*  Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSE_MAX_MEMORY_USAGE 14
-#define FSE_DEFAULT_MEMORY_USAGE 13
-
-/*!FSE_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSE_MAX_SYMBOL_VALUE 255
-
-
-/* **************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSE_FUNCTION_TYPE BYTE
-#define FSE_FUNCTION_EXTENSION
-#define FSE_DECODE_TYPE FSE_decode_t
-
-
-#endif   /* !FSE_COMMONDEFS_ONLY */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/* **************************************************************
-*  Dependencies
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-
-/* ***************************************************************
-*  Constants
-*****************************************************************/
-#define FSE_MAX_TABLELOG  (FSE_MAX_MEMORY_USAGE-2)
-#define FSE_MAX_TABLESIZE (1U<<FSE_MAX_TABLELOG)
-#define FSE_MAXTABLESIZE_MASK (FSE_MAX_TABLESIZE-1)
-#define FSE_DEFAULT_TABLELOG (FSE_DEFAULT_MEMORY_USAGE-2)
-#define FSE_MIN_TABLELOG 5
-
-#define FSE_TABLELOG_ABSOLUTE_MAX 15
-#if FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX
-#error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/* **************************************************************
-*  Complex types
-****************************************************************/
-typedef U32 DTable_max_t[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)];
-
-
-/*-**************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSE_FUNCTION_EXTENSION
-#  error "FSE_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSE_FUNCTION_TYPE
-#  error "FSE_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSE_CAT(X,Y) X##Y
-#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y)
-#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y)
-
-static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
-
-
-static size_t FSE_buildDTable(FSE_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    FSE_DTableHeader DTableH;
-    void* const tdPtr = dt+1;   /* because dt is unsigned, 32-bits aligned on 32-bits */
-    FSE_DECODE_TYPE* const tableDecode = (FSE_DECODE_TYPE*) (tdPtr);
-    const U32 tableSize = 1 << tableLog;
-    const U32 tableMask = tableSize-1;
-    const U32 step = FSE_tableStep(tableSize);
-    U16 symbolNext[FSE_MAX_SYMBOL_VALUE+1];
-    U32 position = 0;
-    U32 highThreshold = tableSize-1;
-    const S16 largeLimit= (S16)(1 << (tableLog-1));
-    U32 noLarge = 1;
-    U32 s;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSE_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    memset(tableDecode, 0, sizeof(FSE_DECODE_TYPE) * (maxSymbolValue+1) );   /* useless init, but keep static analyzer happy, and we don't need to performance optimize legacy decoders */
-    DTableH.tableLog = (U16)tableLog;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        if (normalizedCounter[s]==-1)
-        {
-            tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s;
-            symbolNext[s] = 1;
-        }
-        else
-        {
-            if (normalizedCounter[s] >= largeLimit) noLarge=0;
-            symbolNext[s] = normalizedCounter[s];
-        }
-    }
-
-    /* Spread symbols */
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        int i;
-        for (i=0; i<normalizedCounter[s]; i++)
-        {
-            tableDecode[position].symbol = (FSE_FUNCTION_TYPE)s;
-            position = (position + step) & tableMask;
-            while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }
-    }
-
-    if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-
-    /* Build Decoding table */
-    {
-        U32 i;
-        for (i=0; i<tableSize; i++)
-        {
-            FSE_FUNCTION_TYPE symbol = (FSE_FUNCTION_TYPE)(tableDecode[i].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[i].nbBits = (BYTE) (tableLog - BIT_highbit32 ((U32)nextState) );
-            tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
-        }
-    }
-
-    DTableH.fastMode = (U16)noLarge;
-    memcpy(dt, &DTableH, sizeof(DTableH));
-    return 0;
-}
-
-
-#ifndef FSE_COMMONDEFS_ONLY
-/******************************************
-*  FSE helper functions
-******************************************/
-static unsigned FSE_isError(size_t code) { return ERR_isError(code); }
-
-
-/****************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-static short FSE_abs(short a)
-{
-    return a<0 ? -a : a;
-}
-
-static size_t FSE_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr))
-    {
-        if (previous0)
-        {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF)
-            {
-                n0+=24;
-                if (ip < iend-5)
-                {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                }
-                else
-                {
-                    bitStream >>= 16;
-                    bitCount+=16;
-                }
-            }
-            while ((bitStream & 3) == 3)
-            {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-            {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {
-            const short max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max)
-            {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            }
-            else
-            {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSE_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold)
-            {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            {
-                if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4))
-                {
-                    ip += bitCount>>3;
-                    bitCount &= 7;
-                }
-                else
-                {
-                    bitCount -= (int)(8 * (iend - 4 - ip));
-                    ip = iend - 4;
-                }
-                bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-            }
-        }
-    }
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-
-
-/*********************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-static size_t FSE_buildDTable_rle (FSE_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSE_decode_t* const cell = (FSE_decode_t*)dPtr;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-static size_t FSE_buildDTable_raw (FSE_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSE_DTableHeader* const DTableH = (FSE_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSE_decode_t* const dinfo = (FSE_decode_t*)dPtr;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<=maxSymbolValue; s++)
-    {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSE_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSE_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BIT_DStream_t bitD;
-    FSE_DState_t state1;
-    FSE_DState_t state2;
-    size_t errorCode;
-
-    /* Init */
-    errorCode = BIT_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-    if (FSE_isError(errorCode)) return errorCode;
-
-    FSE_initDState(&state1, &bitD, dt);
-    FSE_initDState(&state2, &bitD, dt);
-
-#define FSE_GETSYMBOL(statePtr) fast ? FSE_decodeSymbolFast(statePtr, &bitD) : FSE_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BIT_reloadDStream(&bitD)==BIT_DStream_unfinished) && (op<olimit) ; op+=4)
-    {
-        op[0] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[1] = FSE_GETSYMBOL(&state2);
-
-        if (FSE_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BIT_reloadDStream(&bitD) > BIT_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSE_GETSYMBOL(&state1);
-
-        if (FSE_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BIT_reloadDStream(&bitD);
-
-        op[3] = FSE_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BIT_reloadDStream(&bitD) >= FSE_DStream_partiallyFilled; Ends at exactly BIT_DStream_completed */
-    while (1)
-    {
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state1))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state1);
-
-        if ( (BIT_reloadDStream(&bitD)>BIT_DStream_completed) || (op==omax) || (BIT_endOfDStream(&bitD) && (fast || FSE_endOfDState(&state2))) )
-            break;
-
-        *op++ = FSE_GETSYMBOL(&state2);
-    }
-
-    /* end ? */
-    if (BIT_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2))
-        return op-ostart;
-
-    if (op==omax) return ERROR(dstSize_tooSmall);   /* dst buffer is full, but cSrc unfinished */
-
-    return ERROR(corruption_detected);
-}
-
-
-static size_t FSE_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSE_DTable* dt)
-{
-    FSE_DTableHeader DTableH;
-    U32 fastMode;
-
-    memcpy(&DTableH, dt, sizeof(DTableH));
-    fastMode = DTableH.fastMode;
-
-    /* select fast mode (static) */
-    if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-static size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSE_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE;
-    size_t errorCode;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSE decoding mode */
-    errorCode = FSE_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSE_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    errorCode = FSE_buildDTable (dt, counting, maxSymbolValue, tableLog);
-    if (FSE_isError(errorCode)) return errorCode;
-
-    /* always return, even if it is an error code */
-    return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);
-}
-
-
-
-#endif   /* FSE_COMMONDEFS_ONLY */
-
-
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   header file
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef HUFF0_H
-#define HUFF0_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* ****************************************
-*  Dependency
-******************************************/
-#include <stddef.h>    /* size_t */
-
-
-/* ****************************************
-*  Huff0 simple functions
-******************************************/
-static size_t HUF_decompress(void* dst,  size_t dstSize,
-                const void* cSrc, size_t cSrcSize);
-/*!
-HUF_decompress():
-    Decompress Huff0 data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstSize'.
-    'dstSize' must be the exact size of original (uncompressed) data.
-    Note : in contrast with FSE, HUF_decompress can regenerate RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data, because it knows size to regenerate.
-    @return : size of regenerated data (== dstSize)
-              or an error code, which can be tested using HUF_isError()
-*/
-
-
-/* ****************************************
-*  Tool functions
-******************************************/
-/* Error Management */
-static unsigned    HUF_isError(size_t code);        /* tells if a return value is an error code */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* HUFF0_H */
-
-
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef HUFF0_STATIC_H
-#define HUFF0_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/* ****************************************
-*  Static allocation macros
-******************************************/
-/* static allocation of Huff0's DTable */
-#define HUF_DTABLE_SIZE(maxTableLog)   (1 + (1<<maxTableLog))  /* nb Cells; use unsigned short for X2, unsigned int for X4 */
-#define HUF_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        unsigned short DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUF_CREATE_STATIC_DTABLEX6(DTable, maxTableLog) \
-        unsigned int DTable[HUF_DTABLE_SIZE(maxTableLog) * 3 / 2] = { maxTableLog }
-
-
-/* ****************************************
-*  Advanced decompression functions
-******************************************/
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbols decoder */
-
-
-/* ****************************************
-*  Huff0 detailed API
-******************************************/
-/*!
-HUF_decompress() does the following:
-1. select the decompression algorithm (X2, X4, X6) based on pre-computed heuristics
-2. build Huffman table from save, using HUF_readDTableXn()
-3. decode 1 or 4 segments in parallel using HUF_decompressSXn_usingDTable
-
-*/
-static size_t HUF_readDTableX2 (unsigned short* DTable, const void* src, size_t srcSize);
-static size_t HUF_readDTableX4 (unsigned* DTable, const void* src, size_t srcSize);
-
-static size_t HUF_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned short* DTable);
-static size_t HUF_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned* DTable);
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* HUFF0_STATIC_H */
-
-
-
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+Huff0 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-
-/* **************************************************************
-*  Constants
-****************************************************************/
-#define HUF_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUF_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUF_MAX_TABLELOG  12           /* max configured tableLog (for static allocation); can be modified up to HUF_ABSOLUTEMAX_TABLELOG */
-#define HUF_DEFAULT_TABLELOG  HUF_MAX_TABLELOG   /* tableLog by default, when not specified */
-#define HUF_MAX_SYMBOL_VALUE 255
-#if (HUF_MAX_TABLELOG > HUF_ABSOLUTEMAX_TABLELOG)
-#  error "HUF_MAX_TABLELOG is too large !"
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-static unsigned HUF_isError(size_t code) { return ERR_isError(code); }
-#define HUF_STATIC_ASSERT(c) { enum { HUF_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-
-/*-*******************************************************
-*  Huff0 : Huffman block decompression
-*********************************************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUF_DEltX2;   /* single-symbol decoding */
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUF_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-/*! HUF_readStats
-    Read compact Huffman tree, saved by HUF_writeCTable
-    @huffWeight : destination buffer
-    @return : size read from `src`
-*/
-static size_t HUF_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                            U32* nbSymbolsPtr, U32* tableLogPtr,
-                            const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    U32 tableLog;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-    U32 n;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  /* special header */
-    {
-        if (iSize >= (242))   /* RLE */
-        {
-            static int l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else   /* Incompressible */
-        {
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            for (n=0; n<oSize; n+=2)
-            {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-            }
-        }
-    }
-    else  /* header compressed with FSE (normal case) */
-    {
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSE_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSE_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUF_ABSOLUTEMAX_TABLELOG + 1) * sizeof(U32));
-    weightTotal = 0;
-    for (n=0; n<oSize; n++)
-    {
-        if (huffWeight[n] >= HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-        rankStats[huffWeight[n]]++;
-        weightTotal += (1 << huffWeight[n]) >> 1;
-    }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    tableLog = BIT_highbit32(weightTotal) + 1;
-    if (tableLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-    {
-        U32 total = 1 << tableLog;
-        U32 rest = total - weightTotal;
-        U32 verif = 1 << BIT_highbit32(rest);
-        U32 lastWeight = BIT_highbit32(rest) + 1;
-        if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-        huffWeight[oSize] = (BYTE)lastWeight;
-        rankStats[lastWeight]++;
-    }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    *tableLogPtr = tableLog;
-    return iSize+1;
-}
-
-
-/**************************/
-/* single-symbol decoding */
-/**************************/
-
-static size_t HUF_readDTableX2 (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    size_t iSize;
-    U32 nbSymbols = 0;
-    U32 n;
-    U32 nextRankStart;
-    void* const dtPtr = DTable + 1;
-    HUF_DEltX2* const dt = (HUF_DEltX2*)dtPtr;
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX2) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(huffWeight, HUF_MAX_SYMBOL_VALUE + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > DTable[0]) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-    DTable[0] = (U16)tableLog;   /* maybe should separate sizeof DTable, as allocated, from used size of DTable, in case of DTable re-use */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<=tableLog; n++)
-    {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<nbSymbols; n++)
-    {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUF_DEltX2 D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize;
-}
-
-static BYTE HUF_decodeSymbolX2(BIT_DStream_t* Dstream, const HUF_DEltX2* dt, const U32 dtLog)
-{
-        const size_t val = BIT_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-        const BYTE c = dt[val].byte;
-        BIT_skipBits(Dstream, dt[val].nbBits);
-        return c;
-}
-
-#define HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUF_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUF_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUF_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUF_decodeStreamX2(BYTE* p, BIT_DStream_t* const bitDPtr, BYTE* const pEnd, const HUF_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-4))
-    {
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd))
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUF_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-
-static size_t HUF_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUF_DEltX2* const dt = ((const HUF_DEltX2*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX2_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX2(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUF_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUF_decompress4X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/***************************/
-/* double-symbols decoding */
-/***************************/
-
-static void HUF_fillDTableX4Level2(HUF_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUF_DEltX4 DElt;
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    U32 s;
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1)
-    {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)   /* note : sortedSymbols already skipped */
-    {
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }
-}
-
-typedef U32 rankVal_t[HUF_ABSOLUTEMAX_TABLELOG][HUF_ABSOLUTEMAX_TABLELOG + 1];
-
-static void HUF_fillDTableX4(HUF_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUF_ABSOLUTEMAX_TABLELOG + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++)
-    {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits)   /* enough room for a second symbol */
-        {
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUF_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        }
-        else
-        {
-            U32 i;
-            const U32 end = start + length;
-            HUF_DEltX4 DElt;
-
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits   = (BYTE)(nbBits);
-            DElt.length   = 1;
-            for (i = start; i < end; i++)
-                DTable[i] = DElt;
-        }
-        rankVal[weight] += length;
-    }
-}
-
-static size_t HUF_readDTableX4 (U32* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUF_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUF_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUF_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUF_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    const U32 memLog = DTable[0];
-    size_t iSize;
-    void* dtPtr = DTable;
-    HUF_DEltX4* const dt = ((HUF_DEltX4*)dtPtr) + 1;
-
-    HUF_STATIC_ASSERT(sizeof(HUF_DEltX4) == sizeof(U32));   /* if compilation fails here, assertion is false */
-    if (memLog > HUF_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUF_readStats(weightList, HUF_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUF_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--)
-        { if (!maxW) return ERROR(GENERIC); }  /* necessarily finds a solution before maxW==0 */
-
-    /* Get start index of each weight */
-    {
-        U32 w, nextRankStart = 0;
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {
-        U32 s;
-        for (s=0; s<nbSymbols; s++)
-        {
-            U32 w = weightList[s];
-            U32 r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {
-        const U32 minBits = tableLog+1 - maxW;
-        U32 nextRankVal = 0;
-        U32 w, consumed;
-        const int rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-        U32* rankVal0 = rankVal[0];
-        for (w=1; w<=maxW; w++)
-        {
-            U32 current = nextRankVal;
-            nextRankVal += rankStats[w] << (w+rescale);
-            rankVal0[w] = current;
-        }
-        for (consumed = minBits; consumed <= memLog - minBits; consumed++)
-        {
-            U32* rankValPtr = rankVal[consumed];
-            for (w = 1; w <= maxW; w++)
-            {
-                rankValPtr[w] = rankVal0[w] >> consumed;
-            }
-        }
-    }
-
-    HUF_fillDTableX4(dt, memLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    return iSize;
-}
-
-
-static U32 HUF_decodeSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BIT_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUF_decodeLastSymbolX4(void* op, BIT_DStream_t* DStream, const HUF_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BIT_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BIT_skipBits(DStream, dt[val].nbBits);
-    else
-    {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8))
-        {
-            BIT_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-        }
-    }
-    return 1;
-}
-
-
-#define HUF_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUF_MAX_TABLELOG<=12)) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUF_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUF_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUF_decodeStreamX4(BYTE* p, BIT_DStream_t* bitDPtr, BYTE* const pEnd, const HUF_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p < pEnd-7))
-    {
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BIT_reloadDStream(bitDPtr) == BIT_DStream_unfinished) && (p <= pEnd-2))
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUF_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUF_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-static size_t HUF_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUF_DEltX4* const dt = ((const HUF_DEltX4*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BIT_DStream_t bitD1;
-        BIT_DStream_t bitD2;
-        BIT_DStream_t bitD3;
-        BIT_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BIT_initDStream(&bitD1, istart1, length1);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD2, istart2, length2);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD3, istart3, length3);
-        if (HUF_isError(errorCode)) return errorCode;
-        errorCode = BIT_initDStream(&bitD4, istart4, length4);
-        if (HUF_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        for ( ; (endSignal==BIT_DStream_unfinished) && (op4<(oend-7)) ; )
-        {
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUF_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUF_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUF_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUF_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BIT_reloadDStream(&bitD1) | BIT_reloadDStream(&bitD2) | BIT_reloadDStream(&bitD3) | BIT_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUF_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUF_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUF_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUF_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BIT_endOfDStream(&bitD1) & BIT_endOfDStream(&bitD2) & BIT_endOfDStream(&bitD3) & BIT_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-static size_t HUF_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUF_CREATE_STATIC_DTABLEX4(DTable, HUF_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUF_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUF_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUF_decompress4X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/**********************************/
-/* Generic decompression selector */
-/**********************************/
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-static size_t HUF_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[3] = { HUF_decompress4X2, HUF_decompress4X4, NULL };
-    /* estimate decompression time */
-    U32 Q;
-    const U32 D256 = (U32)(dstSize >> 8);
-    U32 Dtime[3];
-    U32 algoNb = 0;
-    int n;
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    /* decoder timing evaluation */
-    Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-    for (n=0; n<3; n++)
-        Dtime[n] = algoTime[Q][n].tableTime + (algoTime[Q][n].decode256Time * D256);
-
-    Dtime[1] += Dtime[1] >> 4; Dtime[2] += Dtime[2] >> 3; /* advantage to algorithms using less memory, for cache eviction */
-
-    if (Dtime[1] < Dtime[0]) algoNb = 1;
-
-    return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-
-    //return HUF_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUF_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-    //return HUF_decompress4X6(dst, dstSize, cSrc, cSrcSize);   /* multi-streams quad-symbols decoding */
-}
-
-
-
-#endif   /* ZSTD_CCOMMON_H_MODULE */
-
-
-/*
-    zstd - decompression module fo v0.4 legacy format
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
- * HEAPMODE :
- * Select how default decompression function ZSTD_decompress() will allocate memory,
- * in memory stack (0), or in memory heap (1, requires malloc())
- */
-#ifndef ZSTD_HEAPMODE
-#  define ZSTD_HEAPMODE 1
-#endif
-
-
-/* *******************************************************
-*  Includes
-*********************************************************/
-#include <stdlib.h>      /* calloc */
-#include <string.h>      /* memcpy, memmove */
-#include <stdio.h>       /* debug : printf */
-
-
-/* *******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-
-/* *************************************
-*  Local types
-***************************************/
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-
-/* *******************************************************
-*  Memory operations
-**********************************************************/
-static void ZSTD_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-
-/* *************************************
-*  Error Management
-***************************************/
-
-/*! ZSTD_isError
-*   tells if a return value is an error code */
-static unsigned ZSTD_isError(size_t code) { return ERR_isError(code); }
-
-
-/* *************************************************************
-*   Context management
-***************************************************************/
-typedef enum { ZSTDds_getFrameHeaderSize, ZSTDds_decodeFrameHeader,
-               ZSTDds_decodeBlockHeader, ZSTDds_decompressBlock } ZSTD_dStage;
-
-struct ZSTDv04_Dctx_s
-{
-    U32 LLTable[FSE_DTABLE_SIZE_U32(LLFSELog)];
-    U32 OffTable[FSE_DTABLE_SIZE_U32(OffFSELog)];
-    U32 MLTable[FSE_DTABLE_SIZE_U32(MLFSELog)];
-    const void* previousDstEnd;
-    const void* base;
-    const void* vBase;
-    const void* dictEnd;
-    size_t expected;
-    size_t headerSize;
-    ZSTD_parameters params;
-    blockType_t bType;
-    ZSTD_dStage stage;
-    const BYTE* litPtr;
-    size_t litSize;
-    BYTE litBuffer[BLOCKSIZE + 8 /* margin for wildcopy */];
-    BYTE headerBuffer[ZSTD_frameHeaderSize_max];
-};  /* typedef'd to ZSTD_DCtx within "zstd_static.h" */
-
-static size_t ZSTD_resetDCtx(ZSTD_DCtx* dctx)
-{
-    dctx->expected = ZSTD_frameHeaderSize_min;
-    dctx->stage = ZSTDds_getFrameHeaderSize;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    dctx->vBase = NULL;
-    dctx->dictEnd = NULL;
-    return 0;
-}
-
-static ZSTD_DCtx* ZSTD_createDCtx(void)
-{
-    ZSTD_DCtx* dctx = (ZSTD_DCtx*)malloc(sizeof(ZSTD_DCtx));
-    if (dctx==NULL) return NULL;
-    ZSTD_resetDCtx(dctx);
-    return dctx;
-}
-
-static size_t ZSTD_freeDCtx(ZSTD_DCtx* dctx)
-{
-    free(dctx);
-    return 0;
-}
-
-
-/* *************************************************************
-*   Decompression section
-***************************************************************/
-/** ZSTD_decodeFrameHeader_Part1
-*   decode the 1st part of the Frame Header, which tells Frame Header size.
-*   srcSize must be == ZSTD_frameHeaderSize_min
-*   @return : the full size of the Frame Header */
-static size_t ZSTD_decodeFrameHeader_Part1(ZSTD_DCtx* zc, const void* src, size_t srcSize)
-{
-    U32 magicNumber;
-    if (srcSize != ZSTD_frameHeaderSize_min) return ERROR(srcSize_wrong);
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_MAGICNUMBER) return ERROR(prefix_unknown);
-    zc->headerSize = ZSTD_frameHeaderSize_min;
-    return zc->headerSize;
-}
-
-
-static size_t ZSTD_getFrameParams(ZSTD_parameters* params, const void* src, size_t srcSize)
-{
-    U32 magicNumber;
-    if (srcSize < ZSTD_frameHeaderSize_min) return ZSTD_frameHeaderSize_max;
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTD_MAGICNUMBER) return ERROR(prefix_unknown);
-    memset(params, 0, sizeof(*params));
-    params->windowLog = (((const BYTE*)src)[4] & 15) + ZSTD_WINDOWLOG_ABSOLUTEMIN;
-    if ((((const BYTE*)src)[4] >> 4) != 0) return ERROR(frameParameter_unsupported);   /* reserved bits */
-    return 0;
-}
-
-/** ZSTD_decodeFrameHeader_Part2
-*   decode the full Frame Header
-*   srcSize must be the size provided by ZSTD_decodeFrameHeader_Part1
-*   @return : 0, or an error code, which can be tested using ZSTD_isError() */
-static size_t ZSTD_decodeFrameHeader_Part2(ZSTD_DCtx* zc, const void* src, size_t srcSize)
-{
-    size_t result;
-    if (srcSize != zc->headerSize) return ERROR(srcSize_wrong);
-    result = ZSTD_getFrameParams(&(zc->params), src, srcSize);
-    if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupported);
-    return result;
-}
-
-
-static size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    BYTE headerFlags;
-    U32 cSize;
-
-    if (srcSize < 3) return ERROR(srcSize_wrong);
-
-    headerFlags = *in;
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-
-    bpPtr->blockType = (blockType_t)(headerFlags >> 6);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-static size_t ZSTD_copyRawBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    if (srcSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/** ZSTD_decompressLiterals
-    @return : nb of bytes read from src, or an error code*/
-static size_t ZSTD_decompressLiterals(void* dst, size_t* maxDstSizePtr,
-                                const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-
-    const size_t litSize = (MEM_readLE32(src) & 0x1FFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-    const size_t litCSize = (MEM_readLE32(ip+2) & 0xFFFFFF) >> 5;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-
-    if (litSize > *maxDstSizePtr) return ERROR(corruption_detected);
-    if (litCSize + 5 > srcSize) return ERROR(corruption_detected);
-
-    if (HUF_isError(HUF_decompress(dst, litSize, ip+5, litCSize))) return ERROR(corruption_detected);
-
-    *maxDstSizePtr = litSize;
-    return litCSize + 5;
-}
-
-
-/** ZSTD_decodeLiteralsBlock
-    @return : nb of bytes read from src (< srcSize ) */
-static size_t ZSTD_decodeLiteralsBlock(ZSTD_DCtx* dctx,
-                          const void* src, size_t srcSize)   /* note : srcSize < BLOCKSIZE */
-{
-    const BYTE* const istart = (const BYTE*) src;
-
-    /* any compressed block with literals segment must be at least this size */
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch(*istart & 3)
-    {
-    /* compressed */
-    case 0:
-        {
-            size_t litSize = BLOCKSIZE;
-            const size_t readSize = ZSTD_decompressLiterals(dctx->litBuffer, &litSize, src, srcSize);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, 8);
-            return readSize;   /* works if it's an error too */
-        }
-    case IS_RAW:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > srcSize-11)   /* risk of reading too far with wildcopy */
-            {
-                if (litSize > srcSize-3) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, 8);
-                return litSize+3;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+3;
-            dctx->litSize = litSize;
-            return litSize+3;        }
-    case IS_RLE:
-        {
-            const size_t litSize = (MEM_readLE32(istart) & 0xFFFFFF) >> 2;   /* no buffer issue : srcSize >= MIN_CBLOCK_SIZE */
-            if (litSize > BLOCKSIZE) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[3], litSize + 8);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return 4;
-        }
-    default:
-        return ERROR(corruption_detected);   /* forbidden nominal case */
-    }
-}
-
-
-static size_t ZSTD_decodeSeqHeaders(int* nbSeq, const BYTE** dumpsPtr, size_t* dumpsLengthPtr,
-                         FSE_DTable* DTableLL, FSE_DTable* DTableML, FSE_DTable* DTableOffb,
-                         const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    const BYTE* const iend = istart + srcSize;
-    U32 LLtype, Offtype, MLtype;
-    U32 LLlog, Offlog, MLlog;
-    size_t dumpsLength;
-
-    /* check */
-    if (srcSize < 5) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    *nbSeq = MEM_readLE16(ip); ip+=2;
-    LLtype  = *ip >> 6;
-    Offtype = (*ip >> 4) & 3;
-    MLtype  = (*ip >> 2) & 3;
-    if (*ip & 2)
-    {
-        dumpsLength  = ip[2];
-        dumpsLength += ip[1] << 8;
-        ip += 3;
-    }
-    else
-    {
-        dumpsLength  = ip[1];
-        dumpsLength += (ip[0] & 1) << 8;
-        ip += 2;
-    }
-    *dumpsPtr = ip;
-    ip += dumpsLength;
-    *dumpsLengthPtr = dumpsLength;
-
-    /* check */
-    if (ip > iend-3) return ERROR(srcSize_wrong); /* min : all 3 are "raw", hence no header, but at least xxLog bits per type */
-
-    /* sequences */
-    {
-        S16 norm[MaxML+1];    /* assumption : MaxML >= MaxLL >= MaxOff */
-        size_t headerSize;
-
-        /* Build DTables */
-        switch(LLtype)
-        {
-        case bt_rle :
-            LLlog = 0;
-            FSE_buildDTable_rle(DTableLL, *ip++); break;
-        case bt_raw :
-            LLlog = LLbits;
-            FSE_buildDTable_raw(DTableLL, LLbits); break;
-        default :
-            {   U32 max = MaxLL;
-                headerSize = FSE_readNCount(norm, &max, &LLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (LLlog > LLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableLL, norm, max, LLlog);
-        }   }
-
-        switch(Offtype)
-        {
-        case bt_rle :
-            Offlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong);   /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableOffb, *ip++ & MaxOff); /* if *ip > MaxOff, data is corrupted */
-            break;
-        case bt_raw :
-            Offlog = Offbits;
-            FSE_buildDTable_raw(DTableOffb, Offbits); break;
-        default :
-            {   U32 max = MaxOff;
-                headerSize = FSE_readNCount(norm, &max, &Offlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (Offlog > OffFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableOffb, norm, max, Offlog);
-        }   }
-
-        switch(MLtype)
-        {
-        case bt_rle :
-            MLlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSE_buildDTable_rle(DTableML, *ip++); break;
-        case bt_raw :
-            MLlog = MLbits;
-            FSE_buildDTable_raw(DTableML, MLbits); break;
-        default :
-            {   U32 max = MaxML;
-                headerSize = FSE_readNCount(norm, &max, &MLlog, ip, iend-ip);
-                if (FSE_isError(headerSize)) return ERROR(GENERIC);
-                if (MLlog > MLFSELog) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSE_buildDTable(DTableML, norm, max, MLlog);
-    }   }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t offset;
-    size_t matchLength;
-} seq_t;
-
-typedef struct {
-    BIT_DStream_t DStream;
-    FSE_DState_t stateLL;
-    FSE_DState_t stateOffb;
-    FSE_DState_t stateML;
-    size_t prevOffset;
-    const BYTE* dumps;
-    const BYTE* dumpsEnd;
-} seqState_t;
-
-
-static void ZSTD_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    size_t litLength;
-    size_t prevOffset;
-    size_t offset;
-    size_t matchLength;
-    const BYTE* dumps = seqState->dumps;
-    const BYTE* const de = seqState->dumpsEnd;
-
-    /* Literal length */
-    litLength = FSE_decodeSymbol(&(seqState->stateLL), &(seqState->DStream));
-    prevOffset = litLength ? seq->offset : seqState->prevOffset;
-    if (litLength == MaxLL) {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) litLength += add;
-        else if (dumps + 3 <= de) {
-            litLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) { dumps = de-1; }  /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-
-    /* Offset */
-    {   static const U32 offsetPrefix[MaxOff+1] = {
-                1 /*fake*/, 1, 2, 4, 8, 16, 32, 64, 128, 256,
-                512, 1024, 2048, 4096, 8192, 16384, 32768, 65536, 131072, 262144,
-                524288, 1048576, 2097152, 4194304, 8388608, 16777216, 33554432, /*fake*/ 1, 1, 1, 1, 1 };
-        U32 offsetCode, nbBits;
-        offsetCode = FSE_decodeSymbol(&(seqState->stateOffb), &(seqState->DStream));   /* <= maxOff, by table construction */
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        nbBits = offsetCode - 1;
-        if (offsetCode==0) nbBits = 0;   /* cmove */
-        offset = offsetPrefix[offsetCode] + BIT_readBits(&(seqState->DStream), nbBits);
-        if (MEM_32bits()) BIT_reloadDStream(&(seqState->DStream));
-        if (offsetCode==0) offset = prevOffset;   /* cmove */
-        if (offsetCode | !litLength) seqState->prevOffset = seq->offset;   /* cmove */
-    }
-
-    /* MatchLength */
-    matchLength = FSE_decodeSymbol(&(seqState->stateML), &(seqState->DStream));
-    if (matchLength == MaxML) {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) matchLength += add;
-        else if (dumps + 3 <= de){
-            matchLength = MEM_readLE24(dumps);
-            dumps += 3;
-        }
-        if (dumps >= de) { dumps = de-1; }  /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-    matchLength += MINMATCH;
-
-    /* save result */
-    seq->litLength = litLength;
-    seq->offset = offset;
-    seq->matchLength = matchLength;
-    seqState->dumps = dumps;
-}
-
-
-static size_t ZSTD_execSequence(BYTE* op,
-                                BYTE* const oend, seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                const BYTE* const base, const BYTE* const vBase, const BYTE* const dictEnd)
-{
-    static const int dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-    static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-    BYTE* const oLitEnd = op + sequence.litLength;
-    const size_t sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_8 = oend-8;
-    const BYTE* const litEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    if (oLitEnd > oend_8) return ERROR(dstSize_tooSmall);   /* last match must start at a minimum distance of 8 from oend */
-    if (oMatchEnd > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (litEnd > litLimit) return ERROR(corruption_detected);   /* risk read beyond lit buffer */
-
-    /* copy Literals */
-    ZSTD_wildcopy(op, *litPtr, sequence.litLength);   /* note : oLitEnd <= oend-8 : no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = litEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - base))
-    {
-        /* offset beyond prefix */
-        if (sequence.offset > (size_t)(oLitEnd - vBase))
-            return ERROR(corruption_detected);
-        match = dictEnd - (base-match);
-        if (match + sequence.matchLength <= dictEnd)
-        {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {
-            size_t length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = base;
-            if (op > oend_8 || sequence.matchLength < MINMATCH) {
-              while (op < oMatchEnd) *op++ = *match++;
-              return sequenceLength;
-            }
-        }
-    }
-    /* Requirement: op <= oend_8 */
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        const int sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTD_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTD_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH))
-    {
-        if (op < oend_8)
-        {
-            ZSTD_wildcopy(op, match, oend_8 - op);
-            match += oend_8 - op;
-            op = oend_8;
-        }
-        while (op < oMatchEnd) *op++ = *match++;
-    }
-    else
-    {
-        ZSTD_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8, but must be signed */
-    }
-    return sequenceLength;
-}
-
-
-static size_t ZSTD_decompressSequences(
-                               ZSTD_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t errorCode, dumpsLength;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    int nbSeq;
-    const BYTE* dumps;
-    U32* DTableLL = dctx->LLTable;
-    U32* DTableML = dctx->MLTable;
-    U32* DTableOffb = dctx->OffTable;
-    const BYTE* const base = (const BYTE*) (dctx->base);
-    const BYTE* const vBase = (const BYTE*) (dctx->vBase);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-
-    /* Build Decoding Tables */
-    errorCode = ZSTD_decodeSeqHeaders(&nbSeq, &dumps, &dumpsLength,
-                                      DTableLL, DTableML, DTableOffb,
-                                      ip, iend-ip);
-    if (ZSTD_isError(errorCode)) return errorCode;
-    ip += errorCode;
-
-    /* Regen sequences */
-    {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        sequence.offset = 4;
-        seqState.dumps = dumps;
-        seqState.dumpsEnd = dumps + dumpsLength;
-        seqState.prevOffset = 4;
-        errorCode = BIT_initDStream(&(seqState.DStream), ip, iend-ip);
-        if (ERR_isError(errorCode)) return ERROR(corruption_detected);
-        FSE_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSE_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSE_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BIT_reloadDStream(&(seqState.DStream)) <= BIT_DStream_completed) && nbSeq ; )
-        {
-            size_t oneSeqSize;
-            nbSeq--;
-            ZSTD_decodeSequence(&sequence, &seqState);
-            oneSeqSize = ZSTD_execSequence(op, oend, sequence, &litPtr, litEnd, base, vBase, dictEnd);
-            if (ZSTD_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* check if reached exact end */
-        if ( !BIT_endOfDStream(&(seqState.DStream)) ) return ERROR(corruption_detected);   /* DStream should be entirely and exactly consumed; otherwise data is corrupted */
-
-        /* last literal segment */
-        {
-            size_t lastLLSize = litEnd - litPtr;
-            if (litPtr > litEnd) return ERROR(corruption_detected);
-            if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-            if (op != litPtr) memcpy(op, litPtr, lastLLSize);
-            op += lastLLSize;
-        }
-    }
-
-    return op-ostart;
-}
-
-
-static void ZSTD_checkContinuity(ZSTD_DCtx* dctx, const void* dst)
-{
-    if (dst != dctx->previousDstEnd)   /* not contiguous */
-    {
-        dctx->dictEnd = dctx->previousDstEnd;
-        dctx->vBase = (const char*)dst - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-        dctx->base = dst;
-        dctx->previousDstEnd = dst;
-    }
-}
-
-
-static size_t ZSTD_decompressBlock_internal(ZSTD_DCtx* dctx,
-                            void* dst, size_t maxDstSize,
-                      const void* src, size_t srcSize)
-{
-    /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-
-    /* Decode literals sub-block */
-    size_t litCSize = ZSTD_decodeLiteralsBlock(dctx, src, srcSize);
-    if (ZSTD_isError(litCSize)) return litCSize;
-    ip += litCSize;
-    srcSize -= litCSize;
-
-    return ZSTD_decompressSequences(dctx, dst, maxDstSize, ip, srcSize);
-}
-
-
-static size_t ZSTD_decompress_usingDict(ZSTD_DCtx* ctx,
-                                 void* dst, size_t maxDstSize,
-                                 const void* src, size_t srcSize,
-                                 const void* dict, size_t dictSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t remainingSize = srcSize;
-    blockProperties_t blockProperties;
-
-    /* init */
-    ZSTD_resetDCtx(ctx);
-    if (dict)
-    {
-        ZSTD_decompress_insertDictionary(ctx, dict, dictSize);
-        ctx->dictEnd = ctx->previousDstEnd;
-        ctx->vBase = (const char*)dst - ((const char*)(ctx->previousDstEnd) - (const char*)(ctx->base));
-        ctx->base = dst;
-    }
-    else
-    {
-        ctx->vBase = ctx->base = ctx->dictEnd = dst;
-    }
-
-    /* Frame Header */
-    {
-        size_t frameHeaderSize;
-        if (srcSize < ZSTD_frameHeaderSize_min+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-        frameHeaderSize = ZSTD_decodeFrameHeader_Part1(ctx, src, ZSTD_frameHeaderSize_min);
-        if (ZSTD_isError(frameHeaderSize)) return frameHeaderSize;
-        if (srcSize < frameHeaderSize+ZSTD_blockHeaderSize) return ERROR(srcSize_wrong);
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-        frameHeaderSize = ZSTD_decodeFrameHeader_Part2(ctx, src, frameHeaderSize);
-        if (ZSTD_isError(frameHeaderSize)) return frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t decodedSize=0;
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTD_decompressBlock_internal(ctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTD_copyRawBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        if (ZSTD_isError(decodedSize)) return decodedSize;
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv04_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTD_frameHeaderSize_min) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-    if (MEM_readLE32(src) != ZSTD_MAGICNUMBER) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-        return;
-    }
-    ip += ZSTD_frameHeaderSize_min; remainingSize -= ZSTD_frameHeaderSize_min;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t cBlockSize = ZSTD_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTD_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTD_blockHeaderSize;
-        remainingSize -= ZSTD_blockHeaderSize;
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * BLOCKSIZE;
-}
-
-/* ******************************
-*  Streaming Decompression API
-********************************/
-static size_t ZSTD_nextSrcSizeToDecompress(ZSTD_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-static size_t ZSTD_decompressContinue(ZSTD_DCtx* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != ctx->expected) return ERROR(srcSize_wrong);
-    ZSTD_checkContinuity(ctx, dst);
-
-    /* Decompress : frame header; part 1 */
-    switch (ctx->stage)
-    {
-    case ZSTDds_getFrameHeaderSize :
-        /* get frame header size */
-        if (srcSize != ZSTD_frameHeaderSize_min) return ERROR(srcSize_wrong);   /* impossible */
-        ctx->headerSize = ZSTD_decodeFrameHeader_Part1(ctx, src, ZSTD_frameHeaderSize_min);
-        if (ZSTD_isError(ctx->headerSize)) return ctx->headerSize;
-        memcpy(ctx->headerBuffer, src, ZSTD_frameHeaderSize_min);
-        if (ctx->headerSize > ZSTD_frameHeaderSize_min) return ERROR(GENERIC);   /* impossible */
-        ctx->expected = 0;   /* not necessary to copy more */
-        /* fallthrough */
-    case ZSTDds_decodeFrameHeader:
-        /* get frame header */
-        {   size_t const result = ZSTD_decodeFrameHeader_Part2(ctx, ctx->headerBuffer, ctx->headerSize);
-            if (ZSTD_isError(result)) return result;
-            ctx->expected = ZSTD_blockHeaderSize;
-            ctx->stage = ZSTDds_decodeBlockHeader;
-            return 0;
-        }
-    case ZSTDds_decodeBlockHeader:
-        /* Decode block header */
-        {   blockProperties_t bp;
-            size_t const blockSize = ZSTD_getcBlockSize(src, ZSTD_blockHeaderSize, &bp);
-            if (ZSTD_isError(blockSize)) return blockSize;
-            if (bp.blockType == bt_end)
-            {
-                ctx->expected = 0;
-                ctx->stage = ZSTDds_getFrameHeaderSize;
-            }
-            else
-            {
-                ctx->expected = blockSize;
-                ctx->bType = bp.blockType;
-                ctx->stage = ZSTDds_decompressBlock;
-            }
-            return 0;
-        }
-    case ZSTDds_decompressBlock:
-        {
-            /* Decompress : block content */
-            size_t rSize;
-            switch(ctx->bType)
-            {
-            case bt_compressed:
-                rSize = ZSTD_decompressBlock_internal(ctx, dst, maxDstSize, src, srcSize);
-                break;
-            case bt_raw :
-                rSize = ZSTD_copyRawBlock(dst, maxDstSize, src, srcSize);
-                break;
-            case bt_rle :
-                return ERROR(GENERIC);   /* not yet handled */
-                break;
-            case bt_end :   /* should never happen (filtered at phase 1) */
-                rSize = 0;
-                break;
-            default:
-                return ERROR(GENERIC);
-            }
-            ctx->stage = ZSTDds_decodeBlockHeader;
-            ctx->expected = ZSTD_blockHeaderSize;
-            ctx->previousDstEnd = (char*)dst + rSize;
-            return rSize;
-        }
-    default:
-        return ERROR(GENERIC);   /* impossible */
-    }
-}
-
-
-static void ZSTD_decompress_insertDictionary(ZSTD_DCtx* ctx, const void* dict, size_t dictSize)
-{
-    ctx->dictEnd = ctx->previousDstEnd;
-    ctx->vBase = (const char*)dict - ((const char*)(ctx->previousDstEnd) - (const char*)(ctx->base));
-    ctx->base = dict;
-    ctx->previousDstEnd = (const char*)dict + dictSize;
-}
-
-
-
-/*
-    Buffered version of Zstd compression library
-    Copyright (C) 2015, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* The objects defined into this file should be considered experimental.
- * They are not labelled stable, as their prototype may change in the future.
- * You can use them for tests, provide feedback, or if you can endure risk of future changes.
- */
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stdlib.h>
-
-
-/** ************************************************
-*  Streaming decompression
-*
-*  A ZBUFF_DCtx object is required to track streaming operation.
-*  Use ZBUFF_createDCtx() and ZBUFF_freeDCtx() to create/release resources.
-*  Use ZBUFF_decompressInit() to start a new decompression operation.
-*  ZBUFF_DCtx objects can be reused multiple times.
-*
-*  Use ZBUFF_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *maxDstSizePtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *maxDstSizePtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to call again the function with remaining input.
-*  The content of dst will be overwritten (up to *maxDstSizePtr) at each function call, so save its content if it matters or change dst .
-*  return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to improve latency)
-*            or 0 when a frame is completely decoded
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory)
-*  output : 128 KB block size is the internal unit, it ensures it's always possible to write a full block when it's decoded.
-*  input : just follow indications from ZBUFF_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* **************************************************/
-
-typedef enum { ZBUFFds_init, ZBUFFds_readHeader, ZBUFFds_loadHeader, ZBUFFds_decodeHeader,
-               ZBUFFds_read, ZBUFFds_load, ZBUFFds_flush } ZBUFF_dStage;
-
-/* *** Resource management *** */
-
-#define ZSTD_frameHeaderSize_max 5   /* too magical, should come from reference */
-struct ZBUFFv04_DCtx_s {
-    ZSTD_DCtx* zc;
-    ZSTD_parameters params;
-    char* inBuff;
-    size_t inBuffSize;
-    size_t inPos;
-    char* outBuff;
-    size_t outBuffSize;
-    size_t outStart;
-    size_t outEnd;
-    size_t hPos;
-    const char* dict;
-    size_t dictSize;
-    ZBUFF_dStage stage;
-    unsigned char headerBuffer[ZSTD_frameHeaderSize_max];
-};   /* typedef'd to ZBUFF_DCtx within "zstd_buffered.h" */
-
-typedef ZBUFFv04_DCtx ZBUFF_DCtx;
-
-
-static ZBUFF_DCtx* ZBUFF_createDCtx(void)
-{
-    ZBUFF_DCtx* zbc = (ZBUFF_DCtx*)malloc(sizeof(ZBUFF_DCtx));
-    if (zbc==NULL) return NULL;
-    memset(zbc, 0, sizeof(*zbc));
-    zbc->zc = ZSTD_createDCtx();
-    zbc->stage = ZBUFFds_init;
-    return zbc;
-}
-
-static size_t ZBUFF_freeDCtx(ZBUFF_DCtx* zbc)
-{
-    if (zbc==NULL) return 0;   /* support free on null */
-    ZSTD_freeDCtx(zbc->zc);
-    free(zbc->inBuff);
-    free(zbc->outBuff);
-    free(zbc);
-    return 0;
-}
-
-
-/* *** Initialization *** */
-
-static size_t ZBUFF_decompressInit(ZBUFF_DCtx* zbc)
-{
-    zbc->stage = ZBUFFds_readHeader;
-    zbc->hPos = zbc->inPos = zbc->outStart = zbc->outEnd = zbc->dictSize = 0;
-    return ZSTD_resetDCtx(zbc->zc);
-}
-
-
-static size_t ZBUFF_decompressWithDictionary(ZBUFF_DCtx* zbc, const void* src, size_t srcSize)
-{
-    zbc->dict = (const char*)src;
-    zbc->dictSize = srcSize;
-    return 0;
-}
-
-static size_t ZBUFF_limitCopy(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    size_t length = MIN(maxDstSize, srcSize);
-    memcpy(dst, src, length);
-    return length;
-}
-
-/* *** Decompression *** */
-
-static size_t ZBUFF_decompressContinue(ZBUFF_DCtx* zbc, void* dst, size_t* maxDstSizePtr, const void* src, size_t* srcSizePtr)
-{
-    const char* const istart = (const char*)src;
-    const char* ip = istart;
-    const char* const iend = istart + *srcSizePtr;
-    char* const ostart = (char*)dst;
-    char* op = ostart;
-    char* const oend = ostart + *maxDstSizePtr;
-    U32 notDone = 1;
-
-    DEBUGLOG(5, "ZBUFF_decompressContinue");
-    while (notDone)
-    {
-        switch(zbc->stage)
-        {
-
-        case ZBUFFds_init :
-            DEBUGLOG(5, "ZBUFF_decompressContinue: stage==ZBUFFds_init => ERROR(init_missing)");
-            return ERROR(init_missing);
-
-        case ZBUFFds_readHeader :
-            /* read header from src */
-            {   size_t const headerSize = ZSTD_getFrameParams(&(zbc->params), src, *srcSizePtr);
-                if (ZSTD_isError(headerSize)) return headerSize;
-                if (headerSize) {
-                    /* not enough input to decode header : tell how many bytes would be necessary */
-                    memcpy(zbc->headerBuffer+zbc->hPos, src, *srcSizePtr);
-                    zbc->hPos += *srcSizePtr;
-                    *maxDstSizePtr = 0;
-                    zbc->stage = ZBUFFds_loadHeader;
-                    return headerSize - zbc->hPos;
-                }
-                zbc->stage = ZBUFFds_decodeHeader;
-                break;
-            }
-
-        case ZBUFFds_loadHeader:
-            /* complete header from src */
-            {   size_t headerSize = ZBUFF_limitCopy(
-                    zbc->headerBuffer + zbc->hPos, ZSTD_frameHeaderSize_max - zbc->hPos,
-                    src, *srcSizePtr);
-                zbc->hPos += headerSize;
-                ip += headerSize;
-                headerSize = ZSTD_getFrameParams(&(zbc->params), zbc->headerBuffer, zbc->hPos);
-                if (ZSTD_isError(headerSize)) return headerSize;
-                if (headerSize) {
-                    /* not enough input to decode header : tell how many bytes would be necessary */
-                    *maxDstSizePtr = 0;
-                    return headerSize - zbc->hPos;
-            }   }
-            /* intentional fallthrough */
-
-        case ZBUFFds_decodeHeader:
-                /* apply header to create / resize buffers */
-                {   size_t const neededOutSize = (size_t)1 << zbc->params.windowLog;
-                    size_t const neededInSize = BLOCKSIZE;   /* a block is never > BLOCKSIZE */
-                    if (zbc->inBuffSize < neededInSize) {
-                        free(zbc->inBuff);
-                        zbc->inBuffSize = neededInSize;
-                        zbc->inBuff = (char*)malloc(neededInSize);
-                        if (zbc->inBuff == NULL) return ERROR(memory_allocation);
-                    }
-                    if (zbc->outBuffSize < neededOutSize) {
-                        free(zbc->outBuff);
-                        zbc->outBuffSize = neededOutSize;
-                        zbc->outBuff = (char*)malloc(neededOutSize);
-                        if (zbc->outBuff == NULL) return ERROR(memory_allocation);
-                }   }
-                if (zbc->dictSize)
-                    ZSTD_decompress_insertDictionary(zbc->zc, zbc->dict, zbc->dictSize);
-                if (zbc->hPos) {
-                    /* some data already loaded into headerBuffer : transfer into inBuff */
-                    memcpy(zbc->inBuff, zbc->headerBuffer, zbc->hPos);
-                    zbc->inPos = zbc->hPos;
-                    zbc->hPos = 0;
-                    zbc->stage = ZBUFFds_load;
-                    break;
-                }
-                zbc->stage = ZBUFFds_read;
-		/* fall-through */
-        case ZBUFFds_read:
-            {
-                size_t neededInSize = ZSTD_nextSrcSizeToDecompress(zbc->zc);
-                if (neededInSize==0)   /* end of frame */
-                {
-                    zbc->stage = ZBUFFds_init;
-                    notDone = 0;
-                    break;
-                }
-                if ((size_t)(iend-ip) >= neededInSize)
-                {
-                    /* directly decode from src */
-                    size_t decodedSize = ZSTD_decompressContinue(zbc->zc,
-                        zbc->outBuff + zbc->outStart, zbc->outBuffSize - zbc->outStart,
-                        ip, neededInSize);
-                    if (ZSTD_isError(decodedSize)) return decodedSize;
-                    ip += neededInSize;
-                    if (!decodedSize) break;   /* this was just a header */
-                    zbc->outEnd = zbc->outStart +  decodedSize;
-                    zbc->stage = ZBUFFds_flush;
-                    break;
-                }
-                if (ip==iend) { notDone = 0; break; }   /* no more input */
-                zbc->stage = ZBUFFds_load;
-            }
-	    /* fall-through */
-        case ZBUFFds_load:
-            {
-                size_t neededInSize = ZSTD_nextSrcSizeToDecompress(zbc->zc);
-                size_t toLoad = neededInSize - zbc->inPos;   /* should always be <= remaining space within inBuff */
-                size_t loadedSize;
-                if (toLoad > zbc->inBuffSize - zbc->inPos) return ERROR(corruption_detected);   /* should never happen */
-                loadedSize = ZBUFF_limitCopy(zbc->inBuff + zbc->inPos, toLoad, ip, iend-ip);
-                ip += loadedSize;
-                zbc->inPos += loadedSize;
-                if (loadedSize < toLoad) { notDone = 0; break; }   /* not enough input, wait for more */
-                {
-                    size_t decodedSize = ZSTD_decompressContinue(zbc->zc,
-                        zbc->outBuff + zbc->outStart, zbc->outBuffSize - zbc->outStart,
-                        zbc->inBuff, neededInSize);
-                    if (ZSTD_isError(decodedSize)) return decodedSize;
-                    zbc->inPos = 0;   /* input is consumed */
-                    if (!decodedSize) { zbc->stage = ZBUFFds_read; break; }   /* this was just a header */
-                    zbc->outEnd = zbc->outStart +  decodedSize;
-                    zbc->stage = ZBUFFds_flush;
-                    /* ZBUFFds_flush follows */
-                }
-            }
-	    /* fall-through */
-        case ZBUFFds_flush:
-            {
-                size_t toFlushSize = zbc->outEnd - zbc->outStart;
-                size_t flushedSize = ZBUFF_limitCopy(op, oend-op, zbc->outBuff + zbc->outStart, toFlushSize);
-                op += flushedSize;
-                zbc->outStart += flushedSize;
-                if (flushedSize == toFlushSize)
-                {
-                    zbc->stage = ZBUFFds_read;
-                    if (zbc->outStart + BLOCKSIZE > zbc->outBuffSize)
-                        zbc->outStart = zbc->outEnd = 0;
-                    break;
-                }
-                /* cannot flush everything */
-                notDone = 0;
-                break;
-            }
-        default: return ERROR(GENERIC);   /* impossible */
-        }
-    }
-
-    *srcSizePtr = ip-istart;
-    *maxDstSizePtr = op-ostart;
-
-    {
-        size_t nextSrcSizeHint = ZSTD_nextSrcSizeToDecompress(zbc->zc);
-        if (nextSrcSizeHint > 3) nextSrcSizeHint+= 3;   /* get the next block header while at it */
-        nextSrcSizeHint -= zbc->inPos;   /* already loaded*/
-        return nextSrcSizeHint;
-    }
-}
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-unsigned ZBUFFv04_isError(size_t errorCode) { return ERR_isError(errorCode); }
-const char* ZBUFFv04_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
-
-size_t ZBUFFv04_recommendedDInSize()  { return BLOCKSIZE + 3; }
-size_t ZBUFFv04_recommendedDOutSize() { return BLOCKSIZE; }
-
-
-
-/*- ========================================================================= -*/
-
-/* final wrapping stage */
-
-size_t ZSTDv04_decompressDCtx(ZSTD_DCtx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return ZSTD_decompress_usingDict(dctx, dst, maxDstSize, src, srcSize, NULL, 0);
-}
-
-size_t ZSTDv04_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-#if defined(ZSTD_HEAPMODE) && (ZSTD_HEAPMODE==1)
-    size_t regenSize;
-    ZSTD_DCtx* dctx = ZSTD_createDCtx();
-    if (dctx==NULL) return ERROR(memory_allocation);
-    regenSize = ZSTDv04_decompressDCtx(dctx, dst, maxDstSize, src, srcSize);
-    ZSTD_freeDCtx(dctx);
-    return regenSize;
-#else
-    ZSTD_DCtx dctx;
-    return ZSTDv04_decompressDCtx(&dctx, dst, maxDstSize, src, srcSize);
-#endif
-}
-
-size_t ZSTDv04_resetDCtx(ZSTDv04_Dctx* dctx) { return ZSTD_resetDCtx(dctx); }
-
-size_t ZSTDv04_nextSrcSizeToDecompress(ZSTDv04_Dctx* dctx)
-{
-    return ZSTD_nextSrcSizeToDecompress(dctx);
-}
-
-size_t ZSTDv04_decompressContinue(ZSTDv04_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return ZSTD_decompressContinue(dctx, dst, maxDstSize, src, srcSize);
-}
-
-
-
-ZBUFFv04_DCtx* ZBUFFv04_createDCtx(void) { return ZBUFF_createDCtx(); }
-size_t ZBUFFv04_freeDCtx(ZBUFFv04_DCtx* dctx) { return ZBUFF_freeDCtx(dctx); }
-
-size_t ZBUFFv04_decompressInit(ZBUFFv04_DCtx* dctx) { return ZBUFF_decompressInit(dctx); }
-size_t ZBUFFv04_decompressWithDictionary(ZBUFFv04_DCtx* dctx, const void* src, size_t srcSize)
-{ return ZBUFF_decompressWithDictionary(dctx, src, srcSize); }
-
-size_t ZBUFFv04_decompressContinue(ZBUFFv04_DCtx* dctx, void* dst, size_t* maxDstSizePtr, const void* src, size_t* srcSizePtr)
-{
-    DEBUGLOG(5, "ZBUFFv04_decompressContinue");
-    return ZBUFF_decompressContinue(dctx, dst, maxDstSizePtr, src, srcSizePtr);
-}
-
-ZSTD_DCtx* ZSTDv04_createDCtx(void) { return ZSTD_createDCtx(); }
-size_t ZSTDv04_freeDCtx(ZSTD_DCtx* dctx) { return ZSTD_freeDCtx(dctx); }
diff --git a/vendor/github.com/DataDog/zstd/zstd_v04.h b/vendor/github.com/DataDog/zstd/zstd_v04.h
deleted file mode 100644
index bb5f3b7..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v04.h
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTD_V04_H_91868324769238
-#define ZSTD_V04_H_91868324769238
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/* *************************************
-*  Includes
-***************************************/
-#include <stddef.h>   /* size_t */
-
-
-/* *************************************
-*  Simple one-step function
-***************************************/
-/**
-ZSTDv04_decompress() : decompress ZSTD frames compliant with v0.4.x format
-    compressedSize : is the exact source size
-    maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
-                      It must be equal or larger than originalSize, otherwise decompression will fail.
-    return : the number of bytes decompressed into destination buffer (originalSize)
-             or an errorCode if it fails (which can be tested using ZSTDv01_isError())
-*/
-size_t ZSTDv04_decompress( void* dst, size_t maxOriginalSize,
-                     const void* src, size_t compressedSize);
-
- /**
- ZSTDv04_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.4.x format
-     srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-     cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                 or an error code if it fails (which can be tested using ZSTDv01_isError())
-     dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                 or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
- */
- void ZSTDv04_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                      size_t* cSize, unsigned long long* dBound);
-
-/**
-ZSTDv04_isError() : tells if the result of ZSTDv04_decompress() is an error
-*/
-unsigned ZSTDv04_isError(size_t code);
-
-
-/* *************************************
-*  Advanced functions
-***************************************/
-typedef struct ZSTDv04_Dctx_s ZSTDv04_Dctx;
-ZSTDv04_Dctx* ZSTDv04_createDCtx(void);
-size_t ZSTDv04_freeDCtx(ZSTDv04_Dctx* dctx);
-
-size_t ZSTDv04_decompressDCtx(ZSTDv04_Dctx* dctx,
-                              void* dst, size_t maxOriginalSize,
-                        const void* src, size_t compressedSize);
-
-
-/* *************************************
-*  Direct Streaming
-***************************************/
-size_t ZSTDv04_resetDCtx(ZSTDv04_Dctx* dctx);
-
-size_t ZSTDv04_nextSrcSizeToDecompress(ZSTDv04_Dctx* dctx);
-size_t ZSTDv04_decompressContinue(ZSTDv04_Dctx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize);
-/**
-  Use above functions alternatively.
-  ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTD_decompressContinue().
-  ZSTD_decompressContinue() will use previous data blocks to improve compression if they are located prior to current block.
-  Result is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header.
-*/
-
-
-/* *************************************
-*  Buffered Streaming
-***************************************/
-typedef struct ZBUFFv04_DCtx_s ZBUFFv04_DCtx;
-ZBUFFv04_DCtx* ZBUFFv04_createDCtx(void);
-size_t         ZBUFFv04_freeDCtx(ZBUFFv04_DCtx* dctx);
-
-size_t ZBUFFv04_decompressInit(ZBUFFv04_DCtx* dctx);
-size_t ZBUFFv04_decompressWithDictionary(ZBUFFv04_DCtx* dctx, const void* dict, size_t dictSize);
-
-size_t ZBUFFv04_decompressContinue(ZBUFFv04_DCtx* dctx, void* dst, size_t* maxDstSizePtr, const void* src, size_t* srcSizePtr);
-
-/** ************************************************
-*  Streaming decompression
-*
-*  A ZBUFF_DCtx object is required to track streaming operation.
-*  Use ZBUFF_createDCtx() and ZBUFF_freeDCtx() to create/release resources.
-*  Use ZBUFF_decompressInit() to start a new decompression operation.
-*  ZBUFF_DCtx objects can be reused multiple times.
-*
-*  Optionally, a reference to a static dictionary can be set, using ZBUFF_decompressWithDictionary()
-*  It must be the same content as the one set during compression phase.
-*  Dictionary content must remain accessible during the decompression process.
-*
-*  Use ZBUFF_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *maxDstSizePtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *maxDstSizePtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of dst will be overwritten (up to *maxDstSizePtr) at each function call, so save its content if it matters or change dst.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to improve latency)
-*            or 0 when a frame is completely decoded
-*            or an error code, which can be tested using ZBUFF_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFF_recommendedDInSize / ZBUFF_recommendedDOutSize
-*  output : ZBUFF_recommendedDOutSize==128 KB block size is the internal unit, it ensures it's always possible to write a full block when it's decoded.
-*  input : ZBUFF_recommendedDInSize==128Kb+3; just follow indications from ZBUFF_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* **************************************************/
-unsigned ZBUFFv04_isError(size_t errorCode);
-const char* ZBUFFv04_getErrorName(size_t errorCode);
-
-
-/** The below functions provide recommended buffer sizes for Compression or Decompression operations.
-*   These sizes are not compulsory, they just tend to offer better latency */
-size_t ZBUFFv04_recommendedDInSize(void);
-size_t ZBUFFv04_recommendedDOutSize(void);
-
-
-/* *************************************
-*  Prefix - version detection
-***************************************/
-#define ZSTDv04_magicNumber 0xFD2FB524   /* v0.4 */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* ZSTD_V04_H_91868324769238 */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v05.c b/vendor/github.com/DataDog/zstd/zstd_v05.c
deleted file mode 100644
index a7ea606..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v05.c
+++ /dev/null
@@ -1,4043 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/*- Dependencies -*/
-#include "zstd_v05.h"
-#include "error_private.h"
-
-
-/* ******************************************************************
-   mem.h
-   low-level memory access routines
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSEv05 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*-****************************************
-*  Dependencies
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include <string.h>    /* memcpy */
-
-
-/*-****************************************
-*  Compiler specifics
-******************************************/
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/*-**************************************************************
-*  Basic Types
-*****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/*-**************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS :
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets depending on alignment.
- *            In some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(void*)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(void*)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard, by lying on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-MEM_STATIC void MEM_write32(void* memPtr, U32 value) { *(U32*)memPtr = value; }
-MEM_STATIC void MEM_write64(void* memPtr, U64 value) { *(U64*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; size_t st; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-MEM_STATIC void MEM_write32(void* memPtr, U32 value) { ((unalign*)memPtr)->u32 = value; }
-MEM_STATIC void MEM_write64(void* memPtr, U64 value) { ((unalign*)memPtr)->u64 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-MEM_STATIC void MEM_write32(void* memPtr, U32 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-MEM_STATIC void MEM_write64(void* memPtr, U64 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-#endif /* MEM_FORCE_MEMORY_ACCESS */
-
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian()) {
-        MEM_write16(memPtr, val);
-    } else {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE24(const void* memPtr)
-{
-    return MEM_readLE16(memPtr) + (((const BYTE*)memPtr)[2] << 16);
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24));
-    }
-}
-
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24)
-                     + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56));
-    }
-}
-
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-
-/*
-    zstd - standard compression library
-    Header File for static linking only
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net
-*/
-#ifndef ZSTD_STATIC_H
-#define ZSTD_STATIC_H
-
-/* The prototypes defined within this file are considered experimental.
- * They should not be used in the context DLL as they may change in the future.
- * Prefer static linking if you need them, to control breaking version changes issues.
- */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/*-*************************************
-*  Types
-***************************************/
-#define ZSTDv05_WINDOWLOG_ABSOLUTEMIN 11
-
-
-/*-*************************************
-*  Advanced functions
-***************************************/
-/*- Advanced Decompression functions -*/
-
-/*! ZSTDv05_decompress_usingPreparedDCtx() :
-*   Same as ZSTDv05_decompress_usingDict, but using a reference context `preparedDCtx`, where dictionary has been loaded.
-*   It avoids reloading the dictionary each time.
-*   `preparedDCtx` must have been properly initialized using ZSTDv05_decompressBegin_usingDict().
-*   Requires 2 contexts : 1 for reference, which will not be modified, and 1 to run the decompression operation */
-size_t ZSTDv05_decompress_usingPreparedDCtx(
-                                             ZSTDv05_DCtx* dctx, const ZSTDv05_DCtx* preparedDCtx,
-                                             void* dst, size_t dstCapacity,
-                                       const void* src, size_t srcSize);
-
-
-/* **************************************
-*  Streaming functions (direct mode)
-****************************************/
-size_t ZSTDv05_decompressBegin(ZSTDv05_DCtx* dctx);
-
-/*
-  Streaming decompression, direct mode (bufferless)
-
-  A ZSTDv05_DCtx object is required to track streaming operations.
-  Use ZSTDv05_createDCtx() / ZSTDv05_freeDCtx() to manage it.
-  A ZSTDv05_DCtx object can be re-used multiple times.
-
-  First typical operation is to retrieve frame parameters, using ZSTDv05_getFrameParams().
-  This operation is independent, and just needs enough input data to properly decode the frame header.
-  Objective is to retrieve *params.windowlog, to know minimum amount of memory required during decoding.
-  Result : 0 when successful, it means the ZSTDv05_parameters structure has been filled.
-           >0 : means there is not enough data into src. Provides the expected size to successfully decode header.
-           errorCode, which can be tested using ZSTDv05_isError()
-
-  Start decompression, with ZSTDv05_decompressBegin() or ZSTDv05_decompressBegin_usingDict()
-  Alternatively, you can copy a prepared context, using ZSTDv05_copyDCtx()
-
-  Then use ZSTDv05_nextSrcSizeToDecompress() and ZSTDv05_decompressContinue() alternatively.
-  ZSTDv05_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTDv05_decompressContinue().
-  ZSTDv05_decompressContinue() requires this exact amount of bytes, or it will fail.
-  ZSTDv05_decompressContinue() needs previous data blocks during decompression, up to (1 << windowlog).
-  They should preferably be located contiguously, prior to current block. Alternatively, a round buffer is also possible.
-
-  @result of ZSTDv05_decompressContinue() is the number of bytes regenerated within 'dst'.
-  It can be zero, which is not an error; it just means ZSTDv05_decompressContinue() has decoded some header.
-
-  A frame is fully decoded when ZSTDv05_nextSrcSizeToDecompress() returns zero.
-  Context can then be reset to start a new decompression.
-*/
-
-
-/* **************************************
-*  Block functions
-****************************************/
-/*! Block functions produce and decode raw zstd blocks, without frame metadata.
-    User will have to take in charge required information to regenerate data, such as block sizes.
-
-    A few rules to respect :
-    - Uncompressed block size must be <= 128 KB
-    - Compressing or decompressing requires a context structure
-      + Use ZSTDv05_createCCtx() and ZSTDv05_createDCtx()
-    - It is necessary to init context before starting
-      + compression : ZSTDv05_compressBegin()
-      + decompression : ZSTDv05_decompressBegin()
-      + variants _usingDict() are also allowed
-      + copyCCtx() and copyDCtx() work too
-    - When a block is considered not compressible enough, ZSTDv05_compressBlock() result will be zero.
-      In which case, nothing is produced into `dst`.
-      + User must test for such outcome and deal directly with uncompressed data
-      + ZSTDv05_decompressBlock() doesn't accept uncompressed data as input !!
-*/
-
-size_t ZSTDv05_decompressBlock(ZSTDv05_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* ZSTDv05_STATIC_H */
-
-
-/*
-    zstd_internal - common functions to include
-    Header File for include
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-*/
-#ifndef ZSTD_CCOMMON_H_MODULE
-#define ZSTD_CCOMMON_H_MODULE
-
-
-
-/*-*************************************
-*  Common macros
-***************************************/
-#define MIN(a,b) ((a)<(b) ? (a) : (b))
-#define MAX(a,b) ((a)>(b) ? (a) : (b))
-
-
-/*-*************************************
-*  Common constants
-***************************************/
-#define ZSTDv05_DICT_MAGIC  0xEC30A435
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BLOCKSIZE (128 KB)                 /* define, for static allocation */
-
-static const size_t ZSTDv05_blockHeaderSize = 3;
-static const size_t ZSTDv05_frameHeaderSize_min = 5;
-#define ZSTDv05_frameHeaderSize_max 5         /* define, for static allocation */
-
-#define BITv057 128
-#define BITv056  64
-#define BITv055  32
-#define BITv054  16
-#define BITv051   2
-#define BITv050   1
-
-#define IS_HUFv05 0
-#define IS_PCH 1
-#define IS_RAW 2
-#define IS_RLE 3
-
-#define MINMATCH 4
-#define REPCODE_STARTVALUE 1
-
-#define Litbits  8
-#define MLbits   7
-#define LLbits   6
-#define Offbits  5
-#define MaxLit ((1<<Litbits) - 1)
-#define MaxML  ((1<<MLbits) - 1)
-#define MaxLL  ((1<<LLbits) - 1)
-#define MaxOff ((1<<Offbits)- 1)
-#define MLFSEv05Log   10
-#define LLFSEv05Log   10
-#define OffFSEv05Log   9
-#define MaxSeq MAX(MaxLL, MaxML)
-
-#define FSEv05_ENCODING_RAW     0
-#define FSEv05_ENCODING_RLE     1
-#define FSEv05_ENCODING_STATIC  2
-#define FSEv05_ENCODING_DYNAMIC 3
-
-
-#define HufLog 12
-
-#define MIN_SEQUENCES_SIZE 1 /* nbSeq==0 */
-#define MIN_CBLOCK_SIZE (1 /*litCSize*/ + 1 /* RLE or RAW */ + MIN_SEQUENCES_SIZE /* nbSeq==0 */)   /* for a non-null block */
-
-#define WILDCOPY_OVERLENGTH 8
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-
-/*-*******************************************
-*  Shared functions to include for inlining
-*********************************************/
-static void ZSTDv05_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-
-#define COPY8(d,s) { ZSTDv05_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTDv05_wildcopy() :
-*   custom version of memcpy(), can copy up to 7 bytes too many (8 bytes if length==0) */
-MEM_STATIC void ZSTDv05_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do
-        COPY8(op, ip)
-    while (op < oend);
-}
-
-
-/*-*******************************************
-*  Private interfaces
-*********************************************/
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* offCode;
-    BYTE* litStart;
-    BYTE* lit;
-    BYTE* litLengthStart;
-    BYTE* litLength;
-    BYTE* matchLengthStart;
-    BYTE* matchLength;
-    BYTE* dumpsStart;
-    BYTE* dumps;
-    /* opt */
-    U32* matchLengthFreq;
-    U32* litLengthFreq;
-    U32* litFreq;
-    U32* offCodeFreq;
-    U32  matchLengthSum;
-    U32  litLengthSum;
-    U32  litSum;
-    U32  offCodeSum;
-} seqStore_t;
-
-
-
-#endif   /* ZSTDv05_CCOMMON_H_MODULE */
-/* ******************************************************************
-   FSEv05 : Finite State Entropy coder
-   header file
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef FSEv05_H
-#define FSEv05_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* *****************************************
-*  Includes
-******************************************/
-#include <stddef.h>    /* size_t, ptrdiff_t */
-
-
-/*-****************************************
-*  FSEv05 simple functions
-******************************************/
-size_t FSEv05_decompress(void* dst,  size_t maxDstSize,
-                const void* cSrc, size_t cSrcSize);
-/*!
-FSEv05_decompress():
-    Decompress FSEv05 data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'maxDstSize'.
-    return : size of regenerated data (<= maxDstSize)
-             or an error code, which can be tested using FSEv05_isError()
-
-    ** Important ** : FSEv05_decompress() doesn't decompress non-compressible nor RLE data !!!
-    Why ? : making this distinction requires a header.
-    Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-
-
-/* *****************************************
-*  Tool functions
-******************************************/
-/* Error Management */
-unsigned    FSEv05_isError(size_t code);        /* tells if a return value is an error code */
-const char* FSEv05_getErrorName(size_t code);   /* provides error code string (useful for debugging) */
-
-
-
-
-/* *****************************************
-*  FSEv05 detailed API
-******************************************/
-/* *** DECOMPRESSION *** */
-
-/*!
-FSEv05_readNCount():
-   Read compactly saved 'normalizedCounter' from 'rBuffer'.
-   return : size read from 'rBuffer'
-            or an errorCode, which can be tested using FSEv05_isError()
-            maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-size_t FSEv05_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize);
-
-/*!
-Constructor and Destructor of type FSEv05_DTable
-    Note that its size depends on 'tableLog' */
-typedef unsigned FSEv05_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-FSEv05_DTable* FSEv05_createDTable(unsigned tableLog);
-void        FSEv05_freeDTable(FSEv05_DTable* dt);
-
-/*!
-FSEv05_buildDTable():
-   Builds 'dt', which must be already allocated, using FSEv05_createDTable()
-   @return : 0,
-             or an errorCode, which can be tested using FSEv05_isError() */
-size_t FSEv05_buildDTable (FSEv05_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*!
-FSEv05_decompress_usingDTable():
-   Decompress compressed source @cSrc of size @cSrcSize using `dt`
-   into `dst` which must be already allocated.
-   @return : size of regenerated data (necessarily <= @dstCapacity)
-             or an errorCode, which can be tested using FSEv05_isError() */
-size_t FSEv05_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSEv05_DTable* dt);
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSEv05_H */
-/* ******************************************************************
-   bitstream
-   Part of FSEv05 library
-   header file (to include)
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef BITv05STREAM_H_MODULE
-#define BITv05STREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which highly benefit from being inlined.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-
-
-/*-********************************************
-*  bitStream decoding API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BITv05_DStream_t;
-
-typedef enum { BITv05_DStream_unfinished = 0,
-               BITv05_DStream_endOfBuffer = 1,
-               BITv05_DStream_completed = 2,
-               BITv05_DStream_overflow = 3 } BITv05_DStream_status;  /* result of BITv05_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BITv05_initDStream(BITv05_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BITv05_readBits(BITv05_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BITv05_DStream_status BITv05_reloadDStream(BITv05_DStream_t* bitD);
-MEM_STATIC unsigned BITv05_endOfDStream(const BITv05_DStream_t* bitD);
-
-
-/*-****************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BITv05_readBitsFast(BITv05_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/*-**************************************************************
-*  Helper functions
-****************************************************************/
-MEM_STATIC unsigned BITv05_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-
-/*-********************************************************
-* bitStream decoding
-**********************************************************/
-/*!BITv05_initDStream
-*  Initialize a BITv05_DStream_t.
-*  @bitD : a pointer to an already allocated BITv05_DStream_t structure
-*  @srcBuffer must point at the beginning of a bitStream
-*  @srcSize must be the exact size of the bitStream
-*  @result : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BITv05_initDStream(BITv05_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(size_t)) {  /* normal case */
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(size_t);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BITv05_highbit32(contain32);
-    } else {
-        U32 contain32;
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[6]) << (sizeof(size_t)*8 - 16);/* fall-through */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[5]) << (sizeof(size_t)*8 - 24);/* fall-through */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[4]) << (sizeof(size_t)*8 - 32);/* fall-through */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[3]) << 24; /* fall-through */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[2]) << 16; /* fall-through */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(bitD->start))[1]) <<  8; /* fall-through */
-            default: break;
-        }
-        contain32 = ((const BYTE*)srcBuffer)[srcSize-1];
-        if (contain32 == 0) return ERROR(GENERIC);   /* endMark not present */
-        bitD->bitsConsumed = 8 - BITv05_highbit32(contain32);
-        bitD->bitsConsumed += (U32)(sizeof(size_t) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-MEM_STATIC size_t BITv05_lookBits(BITv05_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BITv05_lookBitsFast :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv05_lookBitsFast(BITv05_DStream_t* bitD, U32 nbBits)
-{
-    const U32 bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BITv05_skipBits(BITv05_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BITv05_readBits(BITv05_DStream_t* bitD, unsigned nbBits)
-{
-    size_t value = BITv05_lookBits(bitD, nbBits);
-    BITv05_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*!BITv05_readBitsFast :
-*  unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv05_readBitsFast(BITv05_DStream_t* bitD, unsigned nbBits)
-{
-    size_t value = BITv05_lookBitsFast(bitD, nbBits);
-    BITv05_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BITv05_DStream_status BITv05_reloadDStream(BITv05_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return BITv05_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer)) {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BITv05_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start) {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BITv05_DStream_endOfBuffer;
-        return BITv05_DStream_completed;
-    }
-    {
-        U32 nbBytes = bitD->bitsConsumed >> 3;
-        BITv05_DStream_status result = BITv05_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start) {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BITv05_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BITv05_endOfDStream
-*   @return Tells if DStream has reached its exact end
-*/
-MEM_STATIC unsigned BITv05_endOfDStream(const BITv05_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITv05STREAM_H_MODULE */
-/* ******************************************************************
-   FSEv05 : Finite State Entropy coder
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef FSEv05_STATIC_H
-#define FSEv05_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/* *****************************************
-*  Static allocation
-*******************************************/
-/* It is possible to statically allocate FSEv05 CTable/DTable as a table of unsigned using below macros */
-#define FSEv05_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/* *****************************************
-*  FSEv05 advanced API
-*******************************************/
-size_t FSEv05_buildDTable_raw (FSEv05_DTable* dt, unsigned nbBits);
-/* build a fake FSEv05_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-size_t FSEv05_buildDTable_rle (FSEv05_DTable* dt, unsigned char symbolValue);
-/* build a fake FSEv05_DTable, designed to always generate the same symbolValue */
-
-
-
-/* *****************************************
-*  FSEv05 symbol decompression API
-*******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSEv05_DState_t;
-
-
-static void     FSEv05_initDState(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD, const FSEv05_DTable* dt);
-
-static unsigned char FSEv05_decodeSymbol(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD);
-
-static unsigned FSEv05_endOfDState(const FSEv05_DState_t* DStatePtr);
-
-
-
-/* *****************************************
-*  FSEv05 unsafe API
-*******************************************/
-static unsigned char FSEv05_decodeSymbolFast(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* *****************************************
-*  Implementation of inlined functions
-*******************************************/
-/* decompression */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSEv05_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSEv05_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSEv05_initDState(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD, const FSEv05_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv05_DTableHeader* const DTableH = (const FSEv05_DTableHeader*)ptr;
-    DStatePtr->state = BITv05_readBits(bitD, DTableH->tableLog);
-    BITv05_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSEv05_peakSymbol(FSEv05_DState_t* DStatePtr)
-{
-    const FSEv05_decode_t DInfo = ((const FSEv05_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    return DInfo.symbol;
-}
-
-MEM_STATIC BYTE FSEv05_decodeSymbol(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD)
-{
-    const FSEv05_decode_t DInfo = ((const FSEv05_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32  nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BITv05_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC BYTE FSEv05_decodeSymbolFast(FSEv05_DState_t* DStatePtr, BITv05_DStream_t* bitD)
-{
-    const FSEv05_decode_t DInfo = ((const FSEv05_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    const U32 nbBits = DInfo.nbBits;
-    BYTE symbol = DInfo.symbol;
-    size_t lowBits = BITv05_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-MEM_STATIC unsigned FSEv05_endOfDState(const FSEv05_DState_t* DStatePtr)
-{
-    return DStatePtr->state == 0;
-}
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSEv05_STATIC_H */
-/* ******************************************************************
-   FSEv05 : Finite State Entropy coder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSEv05 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-#ifndef FSEv05_COMMONDEFS_ONLY
-
-/* **************************************************************
-*  Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSEv05_MAX_MEMORY_USAGE 14
-#define FSEv05_DEFAULT_MEMORY_USAGE 13
-
-/*!FSEv05_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSEv05_MAX_SYMBOL_VALUE 255
-
-
-/* **************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSEv05_FUNCTION_TYPE BYTE
-#define FSEv05_FUNCTION_EXTENSION
-#define FSEv05_DECODE_TYPE FSEv05_decode_t
-
-
-#endif   /* !FSEv05_COMMONDEFS_ONLY */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-
-
-/* ***************************************************************
-*  Constants
-*****************************************************************/
-#define FSEv05_MAX_TABLELOG  (FSEv05_MAX_MEMORY_USAGE-2)
-#define FSEv05_MAX_TABLESIZE (1U<<FSEv05_MAX_TABLELOG)
-#define FSEv05_MAXTABLESIZE_MASK (FSEv05_MAX_TABLESIZE-1)
-#define FSEv05_DEFAULT_TABLELOG (FSEv05_DEFAULT_MEMORY_USAGE-2)
-#define FSEv05_MIN_TABLELOG 5
-
-#define FSEv05_TABLELOG_ABSOLUTE_MAX 15
-#if FSEv05_MAX_TABLELOG > FSEv05_TABLELOG_ABSOLUTE_MAX
-#error "FSEv05_MAX_TABLELOG > FSEv05_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSEv05_STATIC_ASSERT(c) { enum { FSEv05_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/* **************************************************************
-*  Complex types
-****************************************************************/
-typedef unsigned DTable_max_t[FSEv05_DTABLE_SIZE_U32(FSEv05_MAX_TABLELOG)];
-
-
-/* **************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSEv05_FUNCTION_EXTENSION
-#  error "FSEv05_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSEv05_FUNCTION_TYPE
-#  error "FSEv05_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSEv05_CAT(X,Y) X##Y
-#define FSEv05_FUNCTION_NAME(X,Y) FSEv05_CAT(X,Y)
-#define FSEv05_TYPE_NAME(X,Y) FSEv05_CAT(X,Y)
-
-
-/* Function templates */
-static U32 FSEv05_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; }
-
-
-
-FSEv05_DTable* FSEv05_createDTable (unsigned tableLog)
-{
-    if (tableLog > FSEv05_TABLELOG_ABSOLUTE_MAX) tableLog = FSEv05_TABLELOG_ABSOLUTE_MAX;
-    return (FSEv05_DTable*)malloc( FSEv05_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
-}
-
-void FSEv05_freeDTable (FSEv05_DTable* dt)
-{
-    free(dt);
-}
-
-size_t FSEv05_buildDTable(FSEv05_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    FSEv05_DTableHeader DTableH;
-    void* const tdPtr = dt+1;   /* because dt is unsigned, 32-bits aligned on 32-bits */
-    FSEv05_DECODE_TYPE* const tableDecode = (FSEv05_DECODE_TYPE*) (tdPtr);
-    const U32 tableSize = 1 << tableLog;
-    const U32 tableMask = tableSize-1;
-    const U32 step = FSEv05_tableStep(tableSize);
-    U16 symbolNext[FSEv05_MAX_SYMBOL_VALUE+1];
-    U32 position = 0;
-    U32 highThreshold = tableSize-1;
-    const S16 largeLimit= (S16)(1 << (tableLog-1));
-    U32 noLarge = 1;
-    U32 s;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSEv05_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSEv05_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    memset(tableDecode, 0, sizeof(FSEv05_FUNCTION_TYPE) * (maxSymbolValue+1) );   /* useless init, but keep static analyzer happy, and we don't need to performance optimize legacy decoders */
-    DTableH.tableLog = (U16)tableLog;
-    for (s=0; s<=maxSymbolValue; s++) {
-        if (normalizedCounter[s]==-1) {
-            tableDecode[highThreshold--].symbol = (FSEv05_FUNCTION_TYPE)s;
-            symbolNext[s] = 1;
-        } else {
-            if (normalizedCounter[s] >= largeLimit) noLarge=0;
-            symbolNext[s] = normalizedCounter[s];
-    }   }
-
-    /* Spread symbols */
-    for (s=0; s<=maxSymbolValue; s++) {
-        int i;
-        for (i=0; i<normalizedCounter[s]; i++) {
-            tableDecode[position].symbol = (FSEv05_FUNCTION_TYPE)s;
-            position = (position + step) & tableMask;
-            while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-    }   }
-
-    if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-
-    /* Build Decoding table */
-    {
-        U32 i;
-        for (i=0; i<tableSize; i++) {
-            FSEv05_FUNCTION_TYPE symbol = (FSEv05_FUNCTION_TYPE)(tableDecode[i].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[i].nbBits = (BYTE) (tableLog - BITv05_highbit32 ((U32)nextState) );
-            tableDecode[i].newState = (U16) ( (nextState << tableDecode[i].nbBits) - tableSize);
-    }   }
-
-    DTableH.fastMode = (U16)noLarge;
-    memcpy(dt, &DTableH, sizeof(DTableH));
-    return 0;
-}
-
-
-#ifndef FSEv05_COMMONDEFS_ONLY
-/*-****************************************
-*  FSEv05 helper functions
-******************************************/
-unsigned FSEv05_isError(size_t code) { return ERR_isError(code); }
-
-const char* FSEv05_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/*-**************************************************************
-*  FSEv05 NCount encoding-decoding
-****************************************************************/
-static short FSEv05_abs(short a) { return a<0 ? -a : a; }
-
-
-size_t FSEv05_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSEv05_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSEv05_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr)) {
-        if (previous0) {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF) {
-                n0+=24;
-                if (ip < iend-5) {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                } else {
-                    bitStream >>= 16;
-                    bitCount+=16;
-            }   }
-            while ((bitStream & 3) == 3) {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {
-            const short max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max) {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            } else {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSEv05_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold) {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-            } else {
-                bitCount -= (int)(8 * (iend - 4 - ip));
-                ip = iend - 4;
-            }
-            bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-    }   }
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-
-
-
-/*-*******************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-size_t FSEv05_buildDTable_rle (FSEv05_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSEv05_DTableHeader* const DTableH = (FSEv05_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv05_decode_t* const cell = (FSEv05_decode_t*)dPtr;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-size_t FSEv05_buildDTable_raw (FSEv05_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSEv05_DTableHeader* const DTableH = (FSEv05_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv05_decode_t* const dinfo = (FSEv05_decode_t*)dPtr;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSymbolValue = tableMask;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<=maxSymbolValue; s++) {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSEv05_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSEv05_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BITv05_DStream_t bitD;
-    FSEv05_DState_t state1;
-    FSEv05_DState_t state2;
-    size_t errorCode;
-
-    /* Init */
-    errorCode = BITv05_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-    if (FSEv05_isError(errorCode)) return errorCode;
-
-    FSEv05_initDState(&state1, &bitD, dt);
-    FSEv05_initDState(&state2, &bitD, dt);
-
-#define FSEv05_GETSYMBOL(statePtr) fast ? FSEv05_decodeSymbolFast(statePtr, &bitD) : FSEv05_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BITv05_reloadDStream(&bitD)==BITv05_DStream_unfinished) && (op<olimit) ; op+=4) {
-        op[0] = FSEv05_GETSYMBOL(&state1);
-
-        if (FSEv05_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv05_reloadDStream(&bitD);
-
-        op[1] = FSEv05_GETSYMBOL(&state2);
-
-        if (FSEv05_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BITv05_reloadDStream(&bitD) > BITv05_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSEv05_GETSYMBOL(&state1);
-
-        if (FSEv05_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv05_reloadDStream(&bitD);
-
-        op[3] = FSEv05_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BITv05_reloadDStream(&bitD) >= FSEv05_DStream_partiallyFilled; Ends at exactly BITv05_DStream_completed */
-    while (1) {
-        if ( (BITv05_reloadDStream(&bitD)>BITv05_DStream_completed) || (op==omax) || (BITv05_endOfDStream(&bitD) && (fast || FSEv05_endOfDState(&state1))) )
-            break;
-
-        *op++ = FSEv05_GETSYMBOL(&state1);
-
-        if ( (BITv05_reloadDStream(&bitD)>BITv05_DStream_completed) || (op==omax) || (BITv05_endOfDStream(&bitD) && (fast || FSEv05_endOfDState(&state2))) )
-            break;
-
-        *op++ = FSEv05_GETSYMBOL(&state2);
-    }
-
-    /* end ? */
-    if (BITv05_endOfDStream(&bitD) && FSEv05_endOfDState(&state1) && FSEv05_endOfDState(&state2))
-        return op-ostart;
-
-    if (op==omax) return ERROR(dstSize_tooSmall);   /* dst buffer is full, but cSrc unfinished */
-
-    return ERROR(corruption_detected);
-}
-
-
-size_t FSEv05_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSEv05_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv05_DTableHeader* DTableH = (const FSEv05_DTableHeader*)ptr;
-    const U32 fastMode = DTableH->fastMode;
-
-    /* select fast mode (static) */
-    if (fastMode) return FSEv05_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSEv05_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-size_t FSEv05_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSEv05_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSEv05_MAX_SYMBOL_VALUE;
-    size_t errorCode;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSEv05 decoding mode */
-    errorCode = FSEv05_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-    if (FSEv05_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    errorCode = FSEv05_buildDTable (dt, counting, maxSymbolValue, tableLog);
-    if (FSEv05_isError(errorCode)) return errorCode;
-
-    /* always return, even if it is an error code */
-    return FSEv05_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);
-}
-
-
-
-#endif   /* FSEv05_COMMONDEFS_ONLY */
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   header file
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef HUFF0_H
-#define HUFF0_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/* ****************************************
-*  Huff0 simple functions
-******************************************/
-size_t HUFv05_decompress(void* dst,  size_t dstSize,
-                const void* cSrc, size_t cSrcSize);
-/*!
-HUFv05_decompress():
-    Decompress Huff0 data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstSize'.
-    @dstSize : must be the **exact** size of original (uncompressed) data.
-    Note : in contrast with FSEv05, HUFv05_decompress can regenerate
-           RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data,
-           because it knows size to regenerate.
-    @return : size of regenerated data (== dstSize)
-              or an error code, which can be tested using HUFv05_isError()
-*/
-
-
-/* ****************************************
-*  Tool functions
-******************************************/
-/* Error Management */
-unsigned    HUFv05_isError(size_t code);        /* tells if a return value is an error code */
-const char* HUFv05_getErrorName(size_t code);   /* provides error code string (useful for debugging) */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* HUF0_H */
-/* ******************************************************************
-   Huff0 : Huffman codec, part of New Generation Entropy library
-   header file, for static linking only
-   Copyright (C) 2013-2016, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef HUF0_STATIC_H
-#define HUF0_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/* ****************************************
-*  Static allocation
-******************************************/
-/* static allocation of Huff0's DTable */
-#define HUFv05_DTABLE_SIZE(maxTableLog)   (1 + (1<<maxTableLog))
-#define HUFv05_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        unsigned short DTable[HUFv05_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUFv05_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        unsigned int DTable[HUFv05_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUFv05_CREATE_STATIC_DTABLEX6(DTable, maxTableLog) \
-        unsigned int DTable[HUFv05_DTABLE_SIZE(maxTableLog) * 3 / 2] = { maxTableLog }
-
-
-/* ****************************************
-*  Advanced decompression functions
-******************************************/
-size_t HUFv05_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-size_t HUFv05_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbols decoder */
-
-
-/* ****************************************
-*  Huff0 detailed API
-******************************************/
-/*!
-HUFv05_decompress() does the following:
-1. select the decompression algorithm (X2, X4, X6) based on pre-computed heuristics
-2. build Huffman table from save, using HUFv05_readDTableXn()
-3. decode 1 or 4 segments in parallel using HUFv05_decompressSXn_usingDTable
-*/
-size_t HUFv05_readDTableX2 (unsigned short* DTable, const void* src, size_t srcSize);
-size_t HUFv05_readDTableX4 (unsigned* DTable, const void* src, size_t srcSize);
-
-size_t HUFv05_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned short* DTable);
-size_t HUFv05_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned* DTable);
-
-
-/* single stream variants */
-
-size_t HUFv05_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-size_t HUFv05_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbol decoder */
-
-size_t HUFv05_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned short* DTable);
-size_t HUFv05_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned* DTable);
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* HUF0_STATIC_H */
-/* ******************************************************************
-   Huff0 : Huffman coder, part of New Generation Entropy library
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSEv05+Huff0 source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-/* **************************************************************
-*  Includes
-****************************************************************/
-#include <stdlib.h>     /* malloc, free, qsort */
-#include <string.h>     /* memcpy, memset */
-#include <stdio.h>      /* printf (debug) */
-
-
-/* **************************************************************
-*  Constants
-****************************************************************/
-#define HUFv05_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUFv05_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUFv05_MAX_TABLELOG  12           /* max configured tableLog (for static allocation); can be modified up to HUFv05_ABSOLUTEMAX_TABLELOG */
-#define HUFv05_DEFAULT_TABLELOG  HUFv05_MAX_TABLELOG   /* tableLog by default, when not specified */
-#define HUFv05_MAX_SYMBOL_VALUE 255
-#if (HUFv05_MAX_TABLELOG > HUFv05_ABSOLUTEMAX_TABLELOG)
-#  error "HUFv05_MAX_TABLELOG is too large !"
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-unsigned HUFv05_isError(size_t code) { return ERR_isError(code); }
-const char* HUFv05_getErrorName(size_t code) { return ERR_getErrorName(code); }
-#define HUFv05_STATIC_ASSERT(c) { enum { HUFv05_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/* *******************************************************
-*  Huff0 : Huffman block decompression
-*********************************************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUFv05_DEltX2;   /* single-symbol decoding */
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUFv05_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-/*! HUFv05_readStats
-    Read compact Huffman tree, saved by HUFv05_writeCTable
-    @huffWeight : destination buffer
-    @return : size read from `src`
-*/
-static size_t HUFv05_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                            U32* nbSymbolsPtr, U32* tableLogPtr,
-                            const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    U32 tableLog;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-    U32 n;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  { /* special header */
-        if (iSize >= (242)) {  /* RLE */
-            static int l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else {   /* Incompressible */
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            for (n=0; n<oSize; n+=2) {
-                huffWeight[n]   = ip[n/2] >> 4;
-                huffWeight[n+1] = ip[n/2] & 15;
-    }   }   }
-    else  {   /* header compressed with FSEv05 (normal case) */
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSEv05_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSEv05_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUFv05_ABSOLUTEMAX_TABLELOG + 1) * sizeof(U32));
-    weightTotal = 0;
-    for (n=0; n<oSize; n++) {
-        if (huffWeight[n] >= HUFv05_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-        rankStats[huffWeight[n]]++;
-        weightTotal += (1 << huffWeight[n]) >> 1;
-    }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    tableLog = BITv05_highbit32(weightTotal) + 1;
-    if (tableLog > HUFv05_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-    {   /* determine last weight */
-        U32 total = 1 << tableLog;
-        U32 rest = total - weightTotal;
-        U32 verif = 1 << BITv05_highbit32(rest);
-        U32 lastWeight = BITv05_highbit32(rest) + 1;
-        if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-        huffWeight[oSize] = (BYTE)lastWeight;
-        rankStats[lastWeight]++;
-    }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    *tableLogPtr = tableLog;
-    return iSize+1;
-}
-
-
-/*-***************************/
-/*  single-symbol decoding   */
-/*-***************************/
-
-size_t HUFv05_readDTableX2 (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUFv05_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUFv05_ABSOLUTEMAX_TABLELOG + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    size_t iSize;
-    U32 nbSymbols = 0;
-    U32 n;
-    U32 nextRankStart;
-    void* const dtPtr = DTable + 1;
-    HUFv05_DEltX2* const dt = (HUFv05_DEltX2*)dtPtr;
-
-    HUFv05_STATIC_ASSERT(sizeof(HUFv05_DEltX2) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv05_readStats(huffWeight, HUFv05_MAX_SYMBOL_VALUE + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv05_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > DTable[0]) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-    DTable[0] = (U16)tableLog;   /* maybe should separate sizeof allocated DTable, from used size of DTable, in case of re-use */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<=tableLog; n++) {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<nbSymbols; n++) {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUFv05_DEltX2 D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize;
-}
-
-static BYTE HUFv05_decodeSymbolX2(BITv05_DStream_t* Dstream, const HUFv05_DEltX2* dt, const U32 dtLog)
-{
-        const size_t val = BITv05_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-        const BYTE c = dt[val].byte;
-        BITv05_skipBits(Dstream, dt[val].nbBits);
-        return c;
-}
-
-#define HUFv05_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUFv05_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUFv05_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv05_MAX_TABLELOG<=12)) \
-        HUFv05_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUFv05_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUFv05_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUFv05_decodeStreamX2(BYTE* p, BITv05_DStream_t* const bitDPtr, BYTE* const pEnd, const HUFv05_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BITv05_reloadDStream(bitDPtr) == BITv05_DStream_unfinished) && (p <= pEnd-4)) {
-        HUFv05_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BITv05_reloadDStream(bitDPtr) == BITv05_DStream_unfinished) && (p < pEnd))
-        HUFv05_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUFv05_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-size_t HUFv05_decompress1X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + dstSize;
-    const U32 dtLog = DTable[0];
-    const void* dtPtr = DTable;
-    const HUFv05_DEltX2* const dt = ((const HUFv05_DEltX2*)dtPtr)+1;
-    BITv05_DStream_t bitD;
-
-    if (dstSize <= cSrcSize) return ERROR(dstSize_tooSmall);
-    { size_t const errorCode = BITv05_initDStream(&bitD, cSrc, cSrcSize);
-      if (HUFv05_isError(errorCode)) return errorCode; }
-
-    HUFv05_decodeStreamX2(op, &bitD, oend, dt, dtLog);
-
-    /* check */
-    if (!BITv05_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    return dstSize;
-}
-
-size_t HUFv05_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv05_CREATE_STATIC_DTABLEX2(DTable, HUFv05_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUFv05_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUFv05_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUFv05_decompress1X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-size_t HUFv05_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    /* Check */
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUFv05_DEltX2* const dt = ((const HUFv05_DEltX2*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BITv05_DStream_t bitD1;
-        BITv05_DStream_t bitD2;
-        BITv05_DStream_t bitD3;
-        BITv05_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BITv05_initDStream(&bitD1, istart1, length1);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD2, istart2, length2);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD3, istart3, length3);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD4, istart4, length4);
-        if (HUFv05_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv05_reloadDStream(&bitD1) | BITv05_reloadDStream(&bitD2) | BITv05_reloadDStream(&bitD3) | BITv05_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv05_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv05_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX2_0(op4, &bitD4);
-            endSignal = BITv05_reloadDStream(&bitD1) | BITv05_reloadDStream(&bitD2) | BITv05_reloadDStream(&bitD3) | BITv05_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv05_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv05_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv05_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv05_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BITv05_endOfDStream(&bitD1) & BITv05_endOfDStream(&bitD2) & BITv05_endOfDStream(&bitD3) & BITv05_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv05_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv05_CREATE_STATIC_DTABLEX2(DTable, HUFv05_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-    size_t errorCode;
-
-    errorCode = HUFv05_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUFv05_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUFv05_decompress4X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/* *************************/
-/* double-symbols decoding */
-/* *************************/
-
-static void HUFv05_fillDTableX4Level2(HUFv05_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUFv05_DEltX4 DElt;
-    U32 rankVal[HUFv05_ABSOLUTEMAX_TABLELOG + 1];
-    U32 s;
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1) {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++) {   /* note : sortedSymbols already skipped */
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }
-}
-
-typedef U32 rankVal_t[HUFv05_ABSOLUTEMAX_TABLELOG][HUFv05_ABSOLUTEMAX_TABLELOG + 1];
-
-static void HUFv05_fillDTableX4(HUFv05_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUFv05_ABSOLUTEMAX_TABLELOG + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++) {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits) {   /* enough room for a second symbol */
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUFv05_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        } else {
-            U32 i;
-            const U32 end = start + length;
-            HUFv05_DEltX4 DElt;
-
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits   = (BYTE)(nbBits);
-            DElt.length   = 1;
-            for (i = start; i < end; i++)
-                DTable[i] = DElt;
-        }
-        rankVal[weight] += length;
-    }
-}
-
-size_t HUFv05_readDTableX4 (unsigned* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUFv05_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUFv05_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUFv05_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUFv05_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    const U32 memLog = DTable[0];
-    size_t iSize;
-    void* dtPtr = DTable;
-    HUFv05_DEltX4* const dt = ((HUFv05_DEltX4*)dtPtr) + 1;
-
-    HUFv05_STATIC_ASSERT(sizeof(HUFv05_DEltX4) == sizeof(unsigned));   /* if compilation fails here, assertion is false */
-    if (memLog > HUFv05_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv05_readStats(weightList, HUFv05_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv05_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--) {}  /* necessarily finds a solution before 0 */
-
-    /* Get start index of each weight */
-    {
-        U32 w, nextRankStart = 0;
-        for (w=1; w<=maxW; w++) {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {
-        U32 s;
-        for (s=0; s<nbSymbols; s++) {
-            U32 w = weightList[s];
-            U32 r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {
-        const U32 minBits = tableLog+1 - maxW;
-        U32 nextRankVal = 0;
-        U32 w, consumed;
-        const int rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-        U32* rankVal0 = rankVal[0];
-        for (w=1; w<=maxW; w++) {
-            U32 current = nextRankVal;
-            nextRankVal += rankStats[w] << (w+rescale);
-            rankVal0[w] = current;
-        }
-        for (consumed = minBits; consumed <= memLog - minBits; consumed++) {
-            U32* rankValPtr = rankVal[consumed];
-            for (w = 1; w <= maxW; w++) {
-                rankValPtr[w] = rankVal0[w] >> consumed;
-    }   }   }
-
-    HUFv05_fillDTableX4(dt, memLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    return iSize;
-}
-
-
-static U32 HUFv05_decodeSymbolX4(void* op, BITv05_DStream_t* DStream, const HUFv05_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv05_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BITv05_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUFv05_decodeLastSymbolX4(void* op, BITv05_DStream_t* DStream, const HUFv05_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv05_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BITv05_skipBits(DStream, dt[val].nbBits);
-    else {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) {
-            BITv05_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-    }   }
-    return 1;
-}
-
-
-#define HUFv05_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUFv05_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv05_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv05_MAX_TABLELOG<=12)) \
-        ptr += HUFv05_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv05_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUFv05_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUFv05_decodeStreamX4(BYTE* p, BITv05_DStream_t* bitDPtr, BYTE* const pEnd, const HUFv05_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BITv05_reloadDStream(bitDPtr) == BITv05_DStream_unfinished) && (p < pEnd-7)) {
-        HUFv05_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv05_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BITv05_reloadDStream(bitDPtr) == BITv05_DStream_unfinished) && (p <= pEnd-2))
-        HUFv05_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUFv05_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUFv05_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-
-size_t HUFv05_decompress1X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const unsigned* DTable)
-{
-    const BYTE* const istart = (const BYTE*) cSrc;
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* const oend = ostart + dstSize;
-
-    const U32 dtLog = DTable[0];
-    const void* const dtPtr = DTable;
-    const HUFv05_DEltX4* const dt = ((const HUFv05_DEltX4*)dtPtr) +1;
-    size_t errorCode;
-
-    /* Init */
-    BITv05_DStream_t bitD;
-    errorCode = BITv05_initDStream(&bitD, istart, cSrcSize);
-    if (HUFv05_isError(errorCode)) return errorCode;
-
-    /* finish bitStreams one by one */
-    HUFv05_decodeStreamX4(ostart, &bitD, oend,     dt, dtLog);
-
-    /* check */
-    if (!BITv05_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    /* decoded size */
-    return dstSize;
-}
-
-size_t HUFv05_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv05_CREATE_STATIC_DTABLEX4(DTable, HUFv05_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUFv05_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUFv05_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUFv05_decompress1X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-size_t HUFv05_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const unsigned* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {
-        const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUFv05_DEltX4* const dt = ((const HUFv05_DEltX4*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BITv05_DStream_t bitD1;
-        BITv05_DStream_t bitD2;
-        BITv05_DStream_t bitD3;
-        BITv05_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BITv05_initDStream(&bitD1, istart1, length1);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD2, istart2, length2);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD3, istart3, length3);
-        if (HUFv05_isError(errorCode)) return errorCode;
-        errorCode = BITv05_initDStream(&bitD4, istart4, length4);
-        if (HUFv05_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv05_reloadDStream(&bitD1) | BITv05_reloadDStream(&bitD2) | BITv05_reloadDStream(&bitD3) | BITv05_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv05_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv05_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv05_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUFv05_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUFv05_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUFv05_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BITv05_reloadDStream(&bitD1) | BITv05_reloadDStream(&bitD2) | BITv05_reloadDStream(&bitD3) | BITv05_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv05_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv05_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv05_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv05_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BITv05_endOfDStream(&bitD1) & BITv05_endOfDStream(&bitD2) & BITv05_endOfDStream(&bitD3) & BITv05_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv05_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv05_CREATE_STATIC_DTABLEX4(DTable, HUFv05_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUFv05_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUFv05_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUFv05_decompress4X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/* ********************************/
-/* Generic decompression selector */
-/* ********************************/
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-size_t HUFv05_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[3] = { HUFv05_decompress4X2, HUFv05_decompress4X4, NULL };
-    /* estimate decompression time */
-    U32 Q;
-    const U32 D256 = (U32)(dstSize >> 8);
-    U32 Dtime[3];
-    U32 algoNb = 0;
-    int n;
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize >= dstSize) return ERROR(corruption_detected);   /* invalid, or not compressed, but not compressed already dealt with */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    /* decoder timing evaluation */
-    Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-    for (n=0; n<3; n++)
-        Dtime[n] = algoTime[Q][n].tableTime + (algoTime[Q][n].decode256Time * D256);
-
-    Dtime[1] += Dtime[1] >> 4; Dtime[2] += Dtime[2] >> 3; /* advantage to algorithms using less memory, for cache eviction */
-
-    if (Dtime[1] < Dtime[0]) algoNb = 1;
-
-    return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-
-    //return HUFv05_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUFv05_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-    //return HUFv05_decompress4X6(dst, dstSize, cSrc, cSrcSize);   /* multi-streams quad-symbols decoding */
-}
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
- * HEAPMODE :
- * Select how default decompression function ZSTDv05_decompress() will allocate memory,
- * in memory stack (0), or in memory heap (1, requires malloc())
- */
-#ifndef ZSTDv05_HEAPMODE
-#  define ZSTDv05_HEAPMODE 1
-#endif
-
-
-/*-*******************************************************
-*  Dependencies
-*********************************************************/
-#include <stdlib.h>      /* calloc */
-#include <string.h>      /* memcpy, memmove */
-#include <stdio.h>       /* debug only : printf */
-
-
-/*-*******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-
-/*-*************************************
-*  Local types
-***************************************/
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-
-/* *******************************************************
-*  Memory operations
-**********************************************************/
-static void ZSTDv05_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-
-/* *************************************
-*  Error Management
-***************************************/
-/*! ZSTDv05_isError() :
-*   tells if a return value is an error code */
-unsigned ZSTDv05_isError(size_t code) { return ERR_isError(code); }
-
-
-/*! ZSTDv05_getErrorName() :
-*   provides error code string (useful for debugging) */
-const char* ZSTDv05_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/* *************************************************************
-*   Context management
-***************************************************************/
-typedef enum { ZSTDv05ds_getFrameHeaderSize, ZSTDv05ds_decodeFrameHeader,
-               ZSTDv05ds_decodeBlockHeader, ZSTDv05ds_decompressBlock } ZSTDv05_dStage;
-
-struct ZSTDv05_DCtx_s
-{
-    FSEv05_DTable LLTable[FSEv05_DTABLE_SIZE_U32(LLFSEv05Log)];
-    FSEv05_DTable OffTable[FSEv05_DTABLE_SIZE_U32(OffFSEv05Log)];
-    FSEv05_DTable MLTable[FSEv05_DTABLE_SIZE_U32(MLFSEv05Log)];
-    unsigned   hufTableX4[HUFv05_DTABLE_SIZE(HufLog)];
-    const void* previousDstEnd;
-    const void* base;
-    const void* vBase;
-    const void* dictEnd;
-    size_t expected;
-    size_t headerSize;
-    ZSTDv05_parameters params;
-    blockType_t bType;   /* used in ZSTDv05_decompressContinue(), to transfer blockType between header decoding and block decoding stages */
-    ZSTDv05_dStage stage;
-    U32 flagStaticTables;
-    const BYTE* litPtr;
-    size_t litSize;
-    BYTE litBuffer[BLOCKSIZE + WILDCOPY_OVERLENGTH];
-    BYTE headerBuffer[ZSTDv05_frameHeaderSize_max];
-};  /* typedef'd to ZSTDv05_DCtx within "zstd_static.h" */
-
-size_t ZSTDv05_sizeofDCtx (void); /* Hidden declaration */
-size_t ZSTDv05_sizeofDCtx (void) { return sizeof(ZSTDv05_DCtx); }
-
-size_t ZSTDv05_decompressBegin(ZSTDv05_DCtx* dctx)
-{
-    dctx->expected = ZSTDv05_frameHeaderSize_min;
-    dctx->stage = ZSTDv05ds_getFrameHeaderSize;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    dctx->vBase = NULL;
-    dctx->dictEnd = NULL;
-    dctx->hufTableX4[0] = HufLog;
-    dctx->flagStaticTables = 0;
-    return 0;
-}
-
-ZSTDv05_DCtx* ZSTDv05_createDCtx(void)
-{
-    ZSTDv05_DCtx* dctx = (ZSTDv05_DCtx*)malloc(sizeof(ZSTDv05_DCtx));
-    if (dctx==NULL) return NULL;
-    ZSTDv05_decompressBegin(dctx);
-    return dctx;
-}
-
-size_t ZSTDv05_freeDCtx(ZSTDv05_DCtx* dctx)
-{
-    free(dctx);
-    return 0;   /* reserved as a potential error code in the future */
-}
-
-void ZSTDv05_copyDCtx(ZSTDv05_DCtx* dstDCtx, const ZSTDv05_DCtx* srcDCtx)
-{
-    memcpy(dstDCtx, srcDCtx,
-           sizeof(ZSTDv05_DCtx) - (BLOCKSIZE+WILDCOPY_OVERLENGTH + ZSTDv05_frameHeaderSize_max));  /* no need to copy workspace */
-}
-
-
-/* *************************************************************
-*   Decompression section
-***************************************************************/
-
-/* Frame format description
-   Frame Header -  [ Block Header - Block ] - Frame End
-   1) Frame Header
-      - 4 bytes - Magic Number : ZSTDv05_MAGICNUMBER (defined within zstd_internal.h)
-      - 1 byte  - Window Descriptor
-   2) Block Header
-      - 3 bytes, starting with a 2-bits descriptor
-                 Uncompressed, Compressed, Frame End, unused
-   3) Block
-      See Block Format Description
-   4) Frame End
-      - 3 bytes, compatible with Block Header
-*/
-
-/* Block format description
-
-   Block = Literal Section - Sequences Section
-   Prerequisite : size of (compressed) block, maximum size of regenerated data
-
-   1) Literal Section
-
-   1.1) Header : 1-5 bytes
-        flags: 2 bits
-            00 compressed by Huff0
-            01 unused
-            10 is Raw (uncompressed)
-            11 is Rle
-            Note : using 01 => Huff0 with precomputed table ?
-            Note : delta map ? => compressed ?
-
-   1.1.1) Huff0-compressed literal block : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-   1.1.2) Raw (uncompressed) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RAW<<6) + (0<<4) + size
-               12 bits: (IS_RAW<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RAW<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.3) Rle (repeated single byte) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RLE<<6) + (0<<4) + size
-               12 bits: (IS_RLE<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RLE<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.4) Huff0-compressed literal block, using precomputed CTables : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-        1- CTable available (stored into workspace ?)
-        2- Small input (fast heuristic ? Full comparison ? depend on clevel ?)
-
-
-   1.2) Literal block content
-
-   1.2.1) Huff0 block, using sizes from header
-        See Huff0 format
-
-   1.2.2) Huff0 block, using prepared table
-
-   1.2.3) Raw content
-
-   1.2.4) single byte
-
-
-   2) Sequences section
-      TO DO
-*/
-
-
-/** ZSTDv05_decodeFrameHeader_Part1() :
-*   decode the 1st part of the Frame Header, which tells Frame Header size.
-*   srcSize must be == ZSTDv05_frameHeaderSize_min.
-*   @return : the full size of the Frame Header */
-static size_t ZSTDv05_decodeFrameHeader_Part1(ZSTDv05_DCtx* zc, const void* src, size_t srcSize)
-{
-    U32 magicNumber;
-    if (srcSize != ZSTDv05_frameHeaderSize_min)
-        return ERROR(srcSize_wrong);
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTDv05_MAGICNUMBER) return ERROR(prefix_unknown);
-    zc->headerSize = ZSTDv05_frameHeaderSize_min;
-    return zc->headerSize;
-}
-
-
-size_t ZSTDv05_getFrameParams(ZSTDv05_parameters* params, const void* src, size_t srcSize)
-{
-    U32 magicNumber;
-    if (srcSize < ZSTDv05_frameHeaderSize_min) return ZSTDv05_frameHeaderSize_max;
-    magicNumber = MEM_readLE32(src);
-    if (magicNumber != ZSTDv05_MAGICNUMBER) return ERROR(prefix_unknown);
-    memset(params, 0, sizeof(*params));
-    params->windowLog = (((const BYTE*)src)[4] & 15) + ZSTDv05_WINDOWLOG_ABSOLUTEMIN;
-    if ((((const BYTE*)src)[4] >> 4) != 0) return ERROR(frameParameter_unsupported);   /* reserved bits */
-    return 0;
-}
-
-/** ZSTDv05_decodeFrameHeader_Part2() :
-*   decode the full Frame Header.
-*   srcSize must be the size provided by ZSTDv05_decodeFrameHeader_Part1().
-*   @return : 0, or an error code, which can be tested using ZSTDv05_isError() */
-static size_t ZSTDv05_decodeFrameHeader_Part2(ZSTDv05_DCtx* zc, const void* src, size_t srcSize)
-{
-    size_t result;
-    if (srcSize != zc->headerSize)
-        return ERROR(srcSize_wrong);
-    result = ZSTDv05_getFrameParams(&(zc->params), src, srcSize);
-    if ((MEM_32bits()) && (zc->params.windowLog > 25)) return ERROR(frameParameter_unsupported);
-    return result;
-}
-
-
-static size_t ZSTDv05_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    BYTE headerFlags;
-    U32 cSize;
-
-    if (srcSize < 3)
-        return ERROR(srcSize_wrong);
-
-    headerFlags = *in;
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-
-    bpPtr->blockType = (blockType_t)(headerFlags >> 6);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-
-static size_t ZSTDv05_copyRawBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    if (dst==NULL) return ERROR(dstSize_tooSmall);
-    if (srcSize > maxDstSize) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/*! ZSTDv05_decodeLiteralsBlock() :
-    @return : nb of bytes read from src (< srcSize ) */
-static size_t ZSTDv05_decodeLiteralsBlock(ZSTDv05_DCtx* dctx,
-                                    const void* src, size_t srcSize)   /* note : srcSize < BLOCKSIZE */
-{
-    const BYTE* const istart = (const BYTE*) src;
-
-    /* any compressed block with literals segment must be at least this size */
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch(istart[0]>> 6)
-    {
-    case IS_HUFv05:
-        {
-            size_t litSize, litCSize, singleStream=0;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            if (srcSize < 5) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need up to 5 for case 3 */
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                /* 2 - 2 - 10 - 10 */
-                lhSize=3;
-                singleStream = istart[0] & 16;
-                litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-                litCSize = ((istart[1] &  3) << 8) + istart[2];
-                break;
-            case 2:
-                /* 2 - 2 - 14 - 14 */
-                lhSize=4;
-                litSize  = ((istart[0] & 15) << 10) + (istart[1] << 2) + (istart[2] >> 6);
-                litCSize = ((istart[2] & 63) <<  8) + istart[3];
-                break;
-            case 3:
-                /* 2 - 2 - 18 - 18 */
-                lhSize=5;
-                litSize  = ((istart[0] & 15) << 14) + (istart[1] << 6) + (istart[2] >> 2);
-                litCSize = ((istart[2] &  3) << 16) + (istart[3] << 8) + istart[4];
-                break;
-            }
-            if (litSize > BLOCKSIZE) return ERROR(corruption_detected);
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            if (HUFv05_isError(singleStream ?
-                            HUFv05_decompress1X2(dctx->litBuffer, litSize, istart+lhSize, litCSize) :
-                            HUFv05_decompress   (dctx->litBuffer, litSize, istart+lhSize, litCSize) ))
-                return ERROR(corruption_detected);
-
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case IS_PCH:
-        {
-            size_t errorCode;
-            size_t litSize, litCSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            if (lhSize != 1)  /* only case supported for now : small litSize, single stream */
-                return ERROR(corruption_detected);
-            if (!dctx->flagStaticTables)
-                return ERROR(dictionary_corrupted);
-
-            /* 2 - 2 - 10 - 10 */
-            lhSize=3;
-            litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-            litCSize = ((istart[1] &  3) << 8) + istart[2];
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            errorCode = HUFv05_decompress1X4_usingDTable(dctx->litBuffer, litSize, istart+lhSize, litCSize, dctx->hufTableX4);
-            if (HUFv05_isError(errorCode)) return ERROR(corruption_detected);
-
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case IS_RAW:
-        {
-            size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize=1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                break;
-            }
-
-            if (lhSize+litSize+WILDCOPY_OVERLENGTH > srcSize) {  /* risk reading beyond src buffer with wildcopy */
-                if (litSize+lhSize > srcSize) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart+lhSize, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-                return lhSize+litSize;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+lhSize;
-            dctx->litSize = litSize;
-            return lhSize+litSize;
-        }
-    case IS_RLE:
-        {
-            size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize = 1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                if (srcSize<4) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need lhSize+1 = 4 */
-                break;
-            }
-            if (litSize > BLOCKSIZE) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[lhSize], litSize + WILDCOPY_OVERLENGTH);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return lhSize+1;
-        }
-    default:
-        return ERROR(corruption_detected);   /* impossible */
-    }
-}
-
-
-static size_t ZSTDv05_decodeSeqHeaders(int* nbSeq, const BYTE** dumpsPtr, size_t* dumpsLengthPtr,
-                         FSEv05_DTable* DTableLL, FSEv05_DTable* DTableML, FSEv05_DTable* DTableOffb,
-                         const void* src, size_t srcSize, U32 flagStaticTable)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* ip = istart;
-    const BYTE* const iend = istart + srcSize;
-    U32 LLtype, Offtype, MLtype;
-    unsigned LLlog, Offlog, MLlog;
-    size_t dumpsLength;
-
-    /* check */
-    if (srcSize < MIN_SEQUENCES_SIZE)
-        return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    *nbSeq = *ip++;
-    if (*nbSeq==0) return 1;
-    if (*nbSeq >= 128) {
-        if (ip >= iend) return ERROR(srcSize_wrong);
-        *nbSeq = ((nbSeq[0]-128)<<8) + *ip++;
-    }
-
-    if (ip >= iend) return ERROR(srcSize_wrong);
-    LLtype  = *ip >> 6;
-    Offtype = (*ip >> 4) & 3;
-    MLtype  = (*ip >> 2) & 3;
-    if (*ip & 2) {
-        if (ip+3 > iend) return ERROR(srcSize_wrong);
-        dumpsLength  = ip[2];
-        dumpsLength += ip[1] << 8;
-        ip += 3;
-    } else {
-        if (ip+2 > iend) return ERROR(srcSize_wrong);
-        dumpsLength  = ip[1];
-        dumpsLength += (ip[0] & 1) << 8;
-        ip += 2;
-    }
-    *dumpsPtr = ip;
-    ip += dumpsLength;
-    *dumpsLengthPtr = dumpsLength;
-
-    /* check */
-    if (ip > iend-3) return ERROR(srcSize_wrong); /* min : all 3 are "raw", hence no header, but at least xxLog bits per type */
-
-    /* sequences */
-    {
-        S16 norm[MaxML+1];    /* assumption : MaxML >= MaxLL >= MaxOff */
-        size_t headerSize;
-
-        /* Build DTables */
-        switch(LLtype)
-        {
-        case FSEv05_ENCODING_RLE :
-            LLlog = 0;
-            FSEv05_buildDTable_rle(DTableLL, *ip++);
-            break;
-        case FSEv05_ENCODING_RAW :
-            LLlog = LLbits;
-            FSEv05_buildDTable_raw(DTableLL, LLbits);
-            break;
-        case FSEv05_ENCODING_STATIC:
-            if (!flagStaticTable) return ERROR(corruption_detected);
-            break;
-        case FSEv05_ENCODING_DYNAMIC :
-        default :   /* impossible */
-            {   unsigned max = MaxLL;
-                headerSize = FSEv05_readNCount(norm, &max, &LLlog, ip, iend-ip);
-                if (FSEv05_isError(headerSize)) return ERROR(GENERIC);
-                if (LLlog > LLFSEv05Log) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSEv05_buildDTable(DTableLL, norm, max, LLlog);
-        }   }
-
-        switch(Offtype)
-        {
-        case FSEv05_ENCODING_RLE :
-            Offlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong);   /* min : "raw", hence no header, but at least xxLog bits */
-            FSEv05_buildDTable_rle(DTableOffb, *ip++ & MaxOff); /* if *ip > MaxOff, data is corrupted */
-            break;
-        case FSEv05_ENCODING_RAW :
-            Offlog = Offbits;
-            FSEv05_buildDTable_raw(DTableOffb, Offbits);
-            break;
-        case FSEv05_ENCODING_STATIC:
-            if (!flagStaticTable) return ERROR(corruption_detected);
-            break;
-        case FSEv05_ENCODING_DYNAMIC :
-        default :   /* impossible */
-            {   unsigned max = MaxOff;
-                headerSize = FSEv05_readNCount(norm, &max, &Offlog, ip, iend-ip);
-                if (FSEv05_isError(headerSize)) return ERROR(GENERIC);
-                if (Offlog > OffFSEv05Log) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSEv05_buildDTable(DTableOffb, norm, max, Offlog);
-        }   }
-
-        switch(MLtype)
-        {
-        case FSEv05_ENCODING_RLE :
-            MLlog = 0;
-            if (ip > iend-2) return ERROR(srcSize_wrong); /* min : "raw", hence no header, but at least xxLog bits */
-            FSEv05_buildDTable_rle(DTableML, *ip++);
-            break;
-        case FSEv05_ENCODING_RAW :
-            MLlog = MLbits;
-            FSEv05_buildDTable_raw(DTableML, MLbits);
-            break;
-        case FSEv05_ENCODING_STATIC:
-            if (!flagStaticTable) return ERROR(corruption_detected);
-            break;
-        case FSEv05_ENCODING_DYNAMIC :
-        default :   /* impossible */
-            {   unsigned max = MaxML;
-                headerSize = FSEv05_readNCount(norm, &max, &MLlog, ip, iend-ip);
-                if (FSEv05_isError(headerSize)) return ERROR(GENERIC);
-                if (MLlog > MLFSEv05Log) return ERROR(corruption_detected);
-                ip += headerSize;
-                FSEv05_buildDTable(DTableML, norm, max, MLlog);
-    }   }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t matchLength;
-    size_t offset;
-} seq_t;
-
-typedef struct {
-    BITv05_DStream_t DStream;
-    FSEv05_DState_t stateLL;
-    FSEv05_DState_t stateOffb;
-    FSEv05_DState_t stateML;
-    size_t prevOffset;
-    const BYTE* dumps;
-    const BYTE* dumpsEnd;
-} seqState_t;
-
-
-
-static void ZSTDv05_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    size_t litLength;
-    size_t prevOffset;
-    size_t offset;
-    size_t matchLength;
-    const BYTE* dumps = seqState->dumps;
-    const BYTE* const de = seqState->dumpsEnd;
-
-    /* Literal length */
-    litLength = FSEv05_peakSymbol(&(seqState->stateLL));
-    prevOffset = litLength ? seq->offset : seqState->prevOffset;
-    if (litLength == MaxLL) {
-        const U32 add = *dumps++;
-        if (add < 255) litLength += add;
-        else if (dumps + 3 <= de) {
-            litLength = MEM_readLE24(dumps);
-            if (litLength&1) litLength>>=1, dumps += 3;
-            else litLength = (U16)(litLength)>>1, dumps += 2;
-        }
-        if (dumps >= de) { dumps = de-1; }  /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-
-    /* Offset */
-    {
-        static const U32 offsetPrefix[MaxOff+1] = {
-                1 /*fake*/, 1, 2, 4, 8, 16, 32, 64, 128, 256,
-                512, 1024, 2048, 4096, 8192, 16384, 32768, 65536, 131072, 262144,
-                524288, 1048576, 2097152, 4194304, 8388608, 16777216, 33554432, /*fake*/ 1, 1, 1, 1, 1 };
-        U32 offsetCode = FSEv05_peakSymbol(&(seqState->stateOffb));   /* <= maxOff, by table construction */
-        U32 nbBits = offsetCode - 1;
-        if (offsetCode==0) nbBits = 0;   /* cmove */
-        offset = offsetPrefix[offsetCode] + BITv05_readBits(&(seqState->DStream), nbBits);
-        if (MEM_32bits()) BITv05_reloadDStream(&(seqState->DStream));
-        if (offsetCode==0) offset = prevOffset;   /* repcode, cmove */
-        if (offsetCode | !litLength) seqState->prevOffset = seq->offset;   /* cmove */
-        FSEv05_decodeSymbol(&(seqState->stateOffb), &(seqState->DStream));    /* update */
-    }
-
-    /* Literal length update */
-    FSEv05_decodeSymbol(&(seqState->stateLL), &(seqState->DStream));   /* update */
-    if (MEM_32bits()) BITv05_reloadDStream(&(seqState->DStream));
-
-    /* MatchLength */
-    matchLength = FSEv05_decodeSymbol(&(seqState->stateML), &(seqState->DStream));
-    if (matchLength == MaxML) {
-        const U32 add = dumps<de ? *dumps++ : 0;
-        if (add < 255) matchLength += add;
-        else if (dumps + 3 <= de) {
-            matchLength = MEM_readLE24(dumps);
-            if (matchLength&1) matchLength>>=1, dumps += 3;
-            else matchLength = (U16)(matchLength)>>1, dumps += 2;
-        }
-        if (dumps >= de) { dumps = de-1; }  /* late correction, to avoid read overflow (data is now corrupted anyway) */
-    }
-    matchLength += MINMATCH;
-
-    /* save result */
-    seq->litLength = litLength;
-    seq->offset = offset;
-    seq->matchLength = matchLength;
-    seqState->dumps = dumps;
-
-#if 0   /* debug */
-    {
-        static U64 totalDecoded = 0;
-        printf("pos %6u : %3u literals & match %3u bytes at distance %6u \n",
-           (U32)(totalDecoded), (U32)litLength, (U32)matchLength, (U32)offset);
-        totalDecoded += litLength + matchLength;
-    }
-#endif
-}
-
-
-static size_t ZSTDv05_execSequence(BYTE* op,
-                                BYTE* const oend, seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                const BYTE* const base, const BYTE* const vBase, const BYTE* const dictEnd)
-{
-    static const int dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-    static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-    BYTE* const oLitEnd = op + sequence.litLength;
-    const size_t sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_8 = oend-8;
-    const BYTE* const litEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    if (oLitEnd > oend_8) return ERROR(dstSize_tooSmall);   /* last match must start at a minimum distance of 8 from oend */
-    if (oMatchEnd > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (litEnd > litLimit) return ERROR(corruption_detected);   /* risk read beyond lit buffer */
-
-    /* copy Literals */
-    ZSTDv05_wildcopy(op, *litPtr, sequence.litLength);   /* note : oLitEnd <= oend-8 : no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = litEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - base)) {
-        /* offset beyond prefix */
-        if (sequence.offset > (size_t)(oLitEnd - vBase))
-            return ERROR(corruption_detected);
-        match = dictEnd - (base-match);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {
-            size_t length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = base;
-            if (op > oend_8 || sequence.matchLength < MINMATCH) {
-              while (op < oMatchEnd) *op++ = *match++;
-              return sequenceLength;
-            }
-    }   }
-    /* Requirement: op <= oend_8 */
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        const int sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTDv05_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTDv05_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH)) {
-        if (op < oend_8) {
-            ZSTDv05_wildcopy(op, match, oend_8 - op);
-            match += oend_8 - op;
-            op = oend_8;
-        }
-        while (op < oMatchEnd)
-            *op++ = *match++;
-    } else {
-        ZSTDv05_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-    }
-    return sequenceLength;
-}
-
-
-static size_t ZSTDv05_decompressSequences(
-                               ZSTDv05_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t errorCode, dumpsLength=0;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    int nbSeq=0;
-    const BYTE* dumps = NULL;
-    unsigned* DTableLL = dctx->LLTable;
-    unsigned* DTableML = dctx->MLTable;
-    unsigned* DTableOffb = dctx->OffTable;
-    const BYTE* const base = (const BYTE*) (dctx->base);
-    const BYTE* const vBase = (const BYTE*) (dctx->vBase);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-
-    /* Build Decoding Tables */
-    errorCode = ZSTDv05_decodeSeqHeaders(&nbSeq, &dumps, &dumpsLength,
-                                      DTableLL, DTableML, DTableOffb,
-                                      ip, seqSize, dctx->flagStaticTables);
-    if (ZSTDv05_isError(errorCode)) return errorCode;
-    ip += errorCode;
-
-    /* Regen sequences */
-    if (nbSeq) {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        sequence.offset = REPCODE_STARTVALUE;
-        seqState.dumps = dumps;
-        seqState.dumpsEnd = dumps + dumpsLength;
-        seqState.prevOffset = REPCODE_STARTVALUE;
-        errorCode = BITv05_initDStream(&(seqState.DStream), ip, iend-ip);
-        if (ERR_isError(errorCode)) return ERROR(corruption_detected);
-        FSEv05_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSEv05_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSEv05_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BITv05_reloadDStream(&(seqState.DStream)) <= BITv05_DStream_completed) && nbSeq ; ) {
-            size_t oneSeqSize;
-            nbSeq--;
-            ZSTDv05_decodeSequence(&sequence, &seqState);
-            oneSeqSize = ZSTDv05_execSequence(op, oend, sequence, &litPtr, litEnd, base, vBase, dictEnd);
-            if (ZSTDv05_isError(oneSeqSize)) return oneSeqSize;
-            op += oneSeqSize;
-        }
-
-        /* check if reached exact end */
-        if (nbSeq) return ERROR(corruption_detected);
-    }
-
-    /* last literal segment */
-    {
-        size_t lastLLSize = litEnd - litPtr;
-        if (litPtr > litEnd) return ERROR(corruption_detected);   /* too many literals already used */
-        if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-        memcpy(op, litPtr, lastLLSize);
-        op += lastLLSize;
-    }
-
-    return op-ostart;
-}
-
-
-static void ZSTDv05_checkContinuity(ZSTDv05_DCtx* dctx, const void* dst)
-{
-    if (dst != dctx->previousDstEnd) {   /* not contiguous */
-        dctx->dictEnd = dctx->previousDstEnd;
-        dctx->vBase = (const char*)dst - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-        dctx->base = dst;
-        dctx->previousDstEnd = dst;
-    }
-}
-
-
-static size_t ZSTDv05_decompressBlock_internal(ZSTDv05_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{   /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-    size_t litCSize;
-
-    if (srcSize >= BLOCKSIZE) return ERROR(srcSize_wrong);
-
-    /* Decode literals sub-block */
-    litCSize = ZSTDv05_decodeLiteralsBlock(dctx, src, srcSize);
-    if (ZSTDv05_isError(litCSize)) return litCSize;
-    ip += litCSize;
-    srcSize -= litCSize;
-
-    return ZSTDv05_decompressSequences(dctx, dst, dstCapacity, ip, srcSize);
-}
-
-
-size_t ZSTDv05_decompressBlock(ZSTDv05_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{
-    ZSTDv05_checkContinuity(dctx, dst);
-    return ZSTDv05_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-/*! ZSTDv05_decompress_continueDCtx
-*   dctx must have been properly initialized */
-static size_t ZSTDv05_decompress_continueDCtx(ZSTDv05_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                                 const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + maxDstSize;
-    size_t remainingSize = srcSize;
-    blockProperties_t blockProperties;
-    memset(&blockProperties, 0, sizeof(blockProperties));
-
-    /* Frame Header */
-    {   size_t frameHeaderSize;
-        if (srcSize < ZSTDv05_frameHeaderSize_min+ZSTDv05_blockHeaderSize) return ERROR(srcSize_wrong);
-        frameHeaderSize = ZSTDv05_decodeFrameHeader_Part1(dctx, src, ZSTDv05_frameHeaderSize_min);
-        if (ZSTDv05_isError(frameHeaderSize)) return frameHeaderSize;
-        if (srcSize < frameHeaderSize+ZSTDv05_blockHeaderSize) return ERROR(srcSize_wrong);
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-        frameHeaderSize = ZSTDv05_decodeFrameHeader_Part2(dctx, src, frameHeaderSize);
-        if (ZSTDv05_isError(frameHeaderSize)) return frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t decodedSize=0;
-        size_t cBlockSize = ZSTDv05_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTDv05_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTDv05_blockHeaderSize;
-        remainingSize -= ZSTDv05_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTDv05_decompressBlock_internal(dctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTDv05_copyRawBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        if (ZSTDv05_isError(decodedSize)) return decodedSize;
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-
-size_t ZSTDv05_decompress_usingPreparedDCtx(ZSTDv05_DCtx* dctx, const ZSTDv05_DCtx* refDCtx,
-                                         void* dst, size_t maxDstSize,
-                                   const void* src, size_t srcSize)
-{
-    ZSTDv05_copyDCtx(dctx, refDCtx);
-    ZSTDv05_checkContinuity(dctx, dst);
-    return ZSTDv05_decompress_continueDCtx(dctx, dst, maxDstSize, src, srcSize);
-}
-
-
-size_t ZSTDv05_decompress_usingDict(ZSTDv05_DCtx* dctx,
-                                 void* dst, size_t maxDstSize,
-                                 const void* src, size_t srcSize,
-                                 const void* dict, size_t dictSize)
-{
-    ZSTDv05_decompressBegin_usingDict(dctx, dict, dictSize);
-    ZSTDv05_checkContinuity(dctx, dst);
-    return ZSTDv05_decompress_continueDCtx(dctx, dst, maxDstSize, src, srcSize);
-}
-
-
-size_t ZSTDv05_decompressDCtx(ZSTDv05_DCtx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    return ZSTDv05_decompress_usingDict(dctx, dst, maxDstSize, src, srcSize, NULL, 0);
-}
-
-size_t ZSTDv05_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-#if defined(ZSTDv05_HEAPMODE) && (ZSTDv05_HEAPMODE==1)
-    size_t regenSize;
-    ZSTDv05_DCtx* dctx = ZSTDv05_createDCtx();
-    if (dctx==NULL) return ERROR(memory_allocation);
-    regenSize = ZSTDv05_decompressDCtx(dctx, dst, maxDstSize, src, srcSize);
-    ZSTDv05_freeDCtx(dctx);
-    return regenSize;
-#else
-    ZSTDv05_DCtx dctx;
-    return ZSTDv05_decompressDCtx(&dctx, dst, maxDstSize, src, srcSize);
-#endif
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv05_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    blockProperties_t blockProperties;
-
-    /* Frame Header */
-    if (srcSize < ZSTDv05_frameHeaderSize_min) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-    if (MEM_readLE32(src) != ZSTDv05_MAGICNUMBER) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-        return;
-    }
-    ip += ZSTDv05_frameHeaderSize_min; remainingSize -= ZSTDv05_frameHeaderSize_min;
-
-    /* Loop on each block */
-    while (1)
-    {
-        size_t cBlockSize = ZSTDv05_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTDv05_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTDv05_blockHeaderSize;
-        remainingSize -= ZSTDv05_blockHeaderSize;
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * BLOCKSIZE;
-}
-
-/* ******************************
-*  Streaming Decompression API
-********************************/
-size_t ZSTDv05_nextSrcSizeToDecompress(ZSTDv05_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-size_t ZSTDv05_decompressContinue(ZSTDv05_DCtx* dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != dctx->expected) return ERROR(srcSize_wrong);
-    ZSTDv05_checkContinuity(dctx, dst);
-
-    /* Decompress : frame header; part 1 */
-    switch (dctx->stage)
-    {
-    case ZSTDv05ds_getFrameHeaderSize :
-        /* get frame header size */
-        if (srcSize != ZSTDv05_frameHeaderSize_min) return ERROR(srcSize_wrong);   /* impossible */
-        dctx->headerSize = ZSTDv05_decodeFrameHeader_Part1(dctx, src, ZSTDv05_frameHeaderSize_min);
-        if (ZSTDv05_isError(dctx->headerSize)) return dctx->headerSize;
-        memcpy(dctx->headerBuffer, src, ZSTDv05_frameHeaderSize_min);
-        if (dctx->headerSize > ZSTDv05_frameHeaderSize_min) return ERROR(GENERIC); /* should never happen */
-        dctx->expected = 0;   /* not necessary to copy more */
-        /* fallthrough */
-    case ZSTDv05ds_decodeFrameHeader:
-        /* get frame header */
-        {   size_t const result = ZSTDv05_decodeFrameHeader_Part2(dctx, dctx->headerBuffer, dctx->headerSize);
-            if (ZSTDv05_isError(result)) return result;
-            dctx->expected = ZSTDv05_blockHeaderSize;
-            dctx->stage = ZSTDv05ds_decodeBlockHeader;
-            return 0;
-        }
-    case ZSTDv05ds_decodeBlockHeader:
-        {
-            /* Decode block header */
-            blockProperties_t bp;
-            size_t blockSize = ZSTDv05_getcBlockSize(src, ZSTDv05_blockHeaderSize, &bp);
-            if (ZSTDv05_isError(blockSize)) return blockSize;
-            if (bp.blockType == bt_end) {
-                dctx->expected = 0;
-                dctx->stage = ZSTDv05ds_getFrameHeaderSize;
-            }
-            else {
-                dctx->expected = blockSize;
-                dctx->bType = bp.blockType;
-                dctx->stage = ZSTDv05ds_decompressBlock;
-            }
-            return 0;
-        }
-    case ZSTDv05ds_decompressBlock:
-        {
-            /* Decompress : block content */
-            size_t rSize;
-            switch(dctx->bType)
-            {
-            case bt_compressed:
-                rSize = ZSTDv05_decompressBlock_internal(dctx, dst, maxDstSize, src, srcSize);
-                break;
-            case bt_raw :
-                rSize = ZSTDv05_copyRawBlock(dst, maxDstSize, src, srcSize);
-                break;
-            case bt_rle :
-                return ERROR(GENERIC);   /* not yet handled */
-                break;
-            case bt_end :   /* should never happen (filtered at phase 1) */
-                rSize = 0;
-                break;
-            default:
-                return ERROR(GENERIC);   /* impossible */
-            }
-            dctx->stage = ZSTDv05ds_decodeBlockHeader;
-            dctx->expected = ZSTDv05_blockHeaderSize;
-            dctx->previousDstEnd = (char*)dst + rSize;
-            return rSize;
-        }
-    default:
-        return ERROR(GENERIC);   /* impossible */
-    }
-}
-
-
-static void ZSTDv05_refDictContent(ZSTDv05_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    dctx->dictEnd = dctx->previousDstEnd;
-    dctx->vBase = (const char*)dict - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-    dctx->base = dict;
-    dctx->previousDstEnd = (const char*)dict + dictSize;
-}
-
-static size_t ZSTDv05_loadEntropy(ZSTDv05_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    size_t hSize, offcodeHeaderSize, matchlengthHeaderSize, errorCode, litlengthHeaderSize;
-    short offcodeNCount[MaxOff+1];
-    unsigned offcodeMaxValue=MaxOff, offcodeLog;
-    short matchlengthNCount[MaxML+1];
-    unsigned matchlengthMaxValue = MaxML, matchlengthLog;
-    short litlengthNCount[MaxLL+1];
-    unsigned litlengthMaxValue = MaxLL, litlengthLog;
-
-    hSize = HUFv05_readDTableX4(dctx->hufTableX4, dict, dictSize);
-    if (HUFv05_isError(hSize)) return ERROR(dictionary_corrupted);
-    dict = (const char*)dict + hSize;
-    dictSize -= hSize;
-
-    offcodeHeaderSize = FSEv05_readNCount(offcodeNCount, &offcodeMaxValue, &offcodeLog, dict, dictSize);
-    if (FSEv05_isError(offcodeHeaderSize)) return ERROR(dictionary_corrupted);
-    if (offcodeLog > OffFSEv05Log) return ERROR(dictionary_corrupted);
-    errorCode = FSEv05_buildDTable(dctx->OffTable, offcodeNCount, offcodeMaxValue, offcodeLog);
-    if (FSEv05_isError(errorCode)) return ERROR(dictionary_corrupted);
-    dict = (const char*)dict + offcodeHeaderSize;
-    dictSize -= offcodeHeaderSize;
-
-    matchlengthHeaderSize = FSEv05_readNCount(matchlengthNCount, &matchlengthMaxValue, &matchlengthLog, dict, dictSize);
-    if (FSEv05_isError(matchlengthHeaderSize)) return ERROR(dictionary_corrupted);
-    if (matchlengthLog > MLFSEv05Log) return ERROR(dictionary_corrupted);
-    errorCode = FSEv05_buildDTable(dctx->MLTable, matchlengthNCount, matchlengthMaxValue, matchlengthLog);
-    if (FSEv05_isError(errorCode)) return ERROR(dictionary_corrupted);
-    dict = (const char*)dict + matchlengthHeaderSize;
-    dictSize -= matchlengthHeaderSize;
-
-    litlengthHeaderSize = FSEv05_readNCount(litlengthNCount, &litlengthMaxValue, &litlengthLog, dict, dictSize);
-    if (litlengthLog > LLFSEv05Log) return ERROR(dictionary_corrupted);
-    if (FSEv05_isError(litlengthHeaderSize)) return ERROR(dictionary_corrupted);
-    errorCode = FSEv05_buildDTable(dctx->LLTable, litlengthNCount, litlengthMaxValue, litlengthLog);
-    if (FSEv05_isError(errorCode)) return ERROR(dictionary_corrupted);
-
-    dctx->flagStaticTables = 1;
-    return hSize + offcodeHeaderSize + matchlengthHeaderSize + litlengthHeaderSize;
-}
-
-static size_t ZSTDv05_decompress_insertDictionary(ZSTDv05_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    size_t eSize;
-    U32 magic = MEM_readLE32(dict);
-    if (magic != ZSTDv05_DICT_MAGIC) {
-        /* pure content mode */
-        ZSTDv05_refDictContent(dctx, dict, dictSize);
-        return 0;
-    }
-    /* load entropy tables */
-    dict = (const char*)dict + 4;
-    dictSize -= 4;
-    eSize = ZSTDv05_loadEntropy(dctx, dict, dictSize);
-    if (ZSTDv05_isError(eSize)) return ERROR(dictionary_corrupted);
-
-    /* reference dictionary content */
-    dict = (const char*)dict + eSize;
-    dictSize -= eSize;
-    ZSTDv05_refDictContent(dctx, dict, dictSize);
-
-    return 0;
-}
-
-
-size_t ZSTDv05_decompressBegin_usingDict(ZSTDv05_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    size_t errorCode;
-    errorCode = ZSTDv05_decompressBegin(dctx);
-    if (ZSTDv05_isError(errorCode)) return errorCode;
-
-    if (dict && dictSize) {
-        errorCode = ZSTDv05_decompress_insertDictionary(dctx, dict, dictSize);
-        if (ZSTDv05_isError(errorCode)) return ERROR(dictionary_corrupted);
-    }
-
-    return 0;
-}
-
-/*
-    Buffered version of Zstd compression library
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd source repository : https://github.com/Cyan4973/zstd
-    - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
-*/
-
-/* The objects defined into this file should be considered experimental.
- * They are not labelled stable, as their prototype may change in the future.
- * You can use them for tests, provide feedback, or if you can endure risk of future changes.
- */
-
-
-
-/* *************************************
-*  Constants
-***************************************/
-static size_t ZBUFFv05_blockHeaderSize = 3;
-
-
-
-/* *** Compression *** */
-
-static size_t ZBUFFv05_limitCopy(void* dst, size_t maxDstSize, const void* src, size_t srcSize)
-{
-    size_t length = MIN(maxDstSize, srcSize);
-    memcpy(dst, src, length);
-    return length;
-}
-
-
-
-
-/** ************************************************
-*  Streaming decompression
-*
-*  A ZBUFFv05_DCtx object is required to track streaming operation.
-*  Use ZBUFFv05_createDCtx() and ZBUFFv05_freeDCtx() to create/release resources.
-*  Use ZBUFFv05_decompressInit() to start a new decompression operation.
-*  ZBUFFv05_DCtx objects can be reused multiple times.
-*
-*  Use ZBUFFv05_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *maxDstSizePtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *maxDstSizePtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to call again the function with remaining input.
-*  The content of dst will be overwritten (up to *maxDstSizePtr) at each function call, so save its content if it matters or change dst .
-*  return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to improve latency)
-*            or 0 when a frame is completely decoded
-*            or an error code, which can be tested using ZBUFFv05_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory)
-*  output : 128 KB block size is the internal unit, it ensures it's always possible to write a full block when it's decoded.
-*  input : just follow indications from ZBUFFv05_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* **************************************************/
-
-typedef enum { ZBUFFv05ds_init, ZBUFFv05ds_readHeader, ZBUFFv05ds_loadHeader, ZBUFFv05ds_decodeHeader,
-               ZBUFFv05ds_read, ZBUFFv05ds_load, ZBUFFv05ds_flush } ZBUFFv05_dStage;
-
-/* *** Resource management *** */
-
-#define ZSTDv05_frameHeaderSize_max 5   /* too magical, should come from reference */
-struct ZBUFFv05_DCtx_s {
-    ZSTDv05_DCtx* zc;
-    ZSTDv05_parameters params;
-    char* inBuff;
-    size_t inBuffSize;
-    size_t inPos;
-    char* outBuff;
-    size_t outBuffSize;
-    size_t outStart;
-    size_t outEnd;
-    size_t hPos;
-    ZBUFFv05_dStage stage;
-    unsigned char headerBuffer[ZSTDv05_frameHeaderSize_max];
-};   /* typedef'd to ZBUFFv05_DCtx within "zstd_buffered.h" */
-
-
-ZBUFFv05_DCtx* ZBUFFv05_createDCtx(void)
-{
-    ZBUFFv05_DCtx* zbc = (ZBUFFv05_DCtx*)malloc(sizeof(ZBUFFv05_DCtx));
-    if (zbc==NULL) return NULL;
-    memset(zbc, 0, sizeof(*zbc));
-    zbc->zc = ZSTDv05_createDCtx();
-    zbc->stage = ZBUFFv05ds_init;
-    return zbc;
-}
-
-size_t ZBUFFv05_freeDCtx(ZBUFFv05_DCtx* zbc)
-{
-    if (zbc==NULL) return 0;   /* support free on null */
-    ZSTDv05_freeDCtx(zbc->zc);
-    free(zbc->inBuff);
-    free(zbc->outBuff);
-    free(zbc);
-    return 0;
-}
-
-
-/* *** Initialization *** */
-
-size_t ZBUFFv05_decompressInitDictionary(ZBUFFv05_DCtx* zbc, const void* dict, size_t dictSize)
-{
-    zbc->stage = ZBUFFv05ds_readHeader;
-    zbc->hPos = zbc->inPos = zbc->outStart = zbc->outEnd = 0;
-    return ZSTDv05_decompressBegin_usingDict(zbc->zc, dict, dictSize);
-}
-
-size_t ZBUFFv05_decompressInit(ZBUFFv05_DCtx* zbc)
-{
-    return ZBUFFv05_decompressInitDictionary(zbc, NULL, 0);
-}
-
-
-/* *** Decompression *** */
-
-size_t ZBUFFv05_decompressContinue(ZBUFFv05_DCtx* zbc, void* dst, size_t* maxDstSizePtr, const void* src, size_t* srcSizePtr)
-{
-    const char* const istart = (const char*)src;
-    const char* ip = istart;
-    const char* const iend = istart + *srcSizePtr;
-    char* const ostart = (char*)dst;
-    char* op = ostart;
-    char* const oend = ostart + *maxDstSizePtr;
-    U32 notDone = 1;
-
-    while (notDone) {
-        switch(zbc->stage)
-        {
-        case ZBUFFv05ds_init :
-            return ERROR(init_missing);
-
-        case ZBUFFv05ds_readHeader :
-            /* read header from src */
-            {
-                size_t headerSize = ZSTDv05_getFrameParams(&(zbc->params), src, *srcSizePtr);
-                if (ZSTDv05_isError(headerSize)) return headerSize;
-                if (headerSize) {
-                    /* not enough input to decode header : tell how many bytes would be necessary */
-                    memcpy(zbc->headerBuffer+zbc->hPos, src, *srcSizePtr);
-                    zbc->hPos += *srcSizePtr;
-                    *maxDstSizePtr = 0;
-                    zbc->stage = ZBUFFv05ds_loadHeader;
-                    return headerSize - zbc->hPos;
-                }
-                zbc->stage = ZBUFFv05ds_decodeHeader;
-                break;
-            }
-	    /* fall-through */
-        case ZBUFFv05ds_loadHeader:
-            /* complete header from src */
-            {
-                size_t headerSize = ZBUFFv05_limitCopy(
-                    zbc->headerBuffer + zbc->hPos, ZSTDv05_frameHeaderSize_max - zbc->hPos,
-                    src, *srcSizePtr);
-                zbc->hPos += headerSize;
-                ip += headerSize;
-                headerSize = ZSTDv05_getFrameParams(&(zbc->params), zbc->headerBuffer, zbc->hPos);
-                if (ZSTDv05_isError(headerSize)) return headerSize;
-                if (headerSize) {
-                    /* not enough input to decode header : tell how many bytes would be necessary */
-                    *maxDstSizePtr = 0;
-                    return headerSize - zbc->hPos;
-                }
-                // zbc->stage = ZBUFFv05ds_decodeHeader; break;   /* useless : stage follows */
-            }
-	    /* fall-through */
-        case ZBUFFv05ds_decodeHeader:
-                /* apply header to create / resize buffers */
-                {
-                    size_t neededOutSize = (size_t)1 << zbc->params.windowLog;
-                    size_t neededInSize = BLOCKSIZE;   /* a block is never > BLOCKSIZE */
-                    if (zbc->inBuffSize < neededInSize) {
-                        free(zbc->inBuff);
-                        zbc->inBuffSize = neededInSize;
-                        zbc->inBuff = (char*)malloc(neededInSize);
-                        if (zbc->inBuff == NULL) return ERROR(memory_allocation);
-                    }
-                    if (zbc->outBuffSize < neededOutSize) {
-                        free(zbc->outBuff);
-                        zbc->outBuffSize = neededOutSize;
-                        zbc->outBuff = (char*)malloc(neededOutSize);
-                        if (zbc->outBuff == NULL) return ERROR(memory_allocation);
-                }   }
-                if (zbc->hPos) {
-                    /* some data already loaded into headerBuffer : transfer into inBuff */
-                    memcpy(zbc->inBuff, zbc->headerBuffer, zbc->hPos);
-                    zbc->inPos = zbc->hPos;
-                    zbc->hPos = 0;
-                    zbc->stage = ZBUFFv05ds_load;
-                    break;
-                }
-                zbc->stage = ZBUFFv05ds_read;
-		/* fall-through */
-        case ZBUFFv05ds_read:
-            {
-                size_t neededInSize = ZSTDv05_nextSrcSizeToDecompress(zbc->zc);
-                if (neededInSize==0) {  /* end of frame */
-                    zbc->stage = ZBUFFv05ds_init;
-                    notDone = 0;
-                    break;
-                }
-                if ((size_t)(iend-ip) >= neededInSize) {
-                    /* directly decode from src */
-                    size_t decodedSize = ZSTDv05_decompressContinue(zbc->zc,
-                        zbc->outBuff + zbc->outStart, zbc->outBuffSize - zbc->outStart,
-                        ip, neededInSize);
-                    if (ZSTDv05_isError(decodedSize)) return decodedSize;
-                    ip += neededInSize;
-                    if (!decodedSize) break;   /* this was just a header */
-                    zbc->outEnd = zbc->outStart +  decodedSize;
-                    zbc->stage = ZBUFFv05ds_flush;
-                    break;
-                }
-                if (ip==iend) { notDone = 0; break; }   /* no more input */
-                zbc->stage = ZBUFFv05ds_load;
-            }
-	    /* fall-through */
-        case ZBUFFv05ds_load:
-            {
-                size_t neededInSize = ZSTDv05_nextSrcSizeToDecompress(zbc->zc);
-                size_t toLoad = neededInSize - zbc->inPos;   /* should always be <= remaining space within inBuff */
-                size_t loadedSize;
-                if (toLoad > zbc->inBuffSize - zbc->inPos) return ERROR(corruption_detected);   /* should never happen */
-                loadedSize = ZBUFFv05_limitCopy(zbc->inBuff + zbc->inPos, toLoad, ip, iend-ip);
-                ip += loadedSize;
-                zbc->inPos += loadedSize;
-                if (loadedSize < toLoad) { notDone = 0; break; }   /* not enough input, wait for more */
-                {
-                    size_t decodedSize = ZSTDv05_decompressContinue(zbc->zc,
-                        zbc->outBuff + zbc->outStart, zbc->outBuffSize - zbc->outStart,
-                        zbc->inBuff, neededInSize);
-                    if (ZSTDv05_isError(decodedSize)) return decodedSize;
-                    zbc->inPos = 0;   /* input is consumed */
-                    if (!decodedSize) { zbc->stage = ZBUFFv05ds_read; break; }   /* this was just a header */
-                    zbc->outEnd = zbc->outStart +  decodedSize;
-                    zbc->stage = ZBUFFv05ds_flush;
-                    // break; /* ZBUFFv05ds_flush follows */
-                }
-	    }
-	    /* fall-through */
-        case ZBUFFv05ds_flush:
-            {
-                size_t toFlushSize = zbc->outEnd - zbc->outStart;
-                size_t flushedSize = ZBUFFv05_limitCopy(op, oend-op, zbc->outBuff + zbc->outStart, toFlushSize);
-                op += flushedSize;
-                zbc->outStart += flushedSize;
-                if (flushedSize == toFlushSize) {
-                    zbc->stage = ZBUFFv05ds_read;
-                    if (zbc->outStart + BLOCKSIZE > zbc->outBuffSize)
-                        zbc->outStart = zbc->outEnd = 0;
-                    break;
-                }
-                /* cannot flush everything */
-                notDone = 0;
-                break;
-            }
-        default: return ERROR(GENERIC);   /* impossible */
-    }   }
-
-    *srcSizePtr = ip-istart;
-    *maxDstSizePtr = op-ostart;
-
-    {   size_t nextSrcSizeHint = ZSTDv05_nextSrcSizeToDecompress(zbc->zc);
-        if (nextSrcSizeHint > ZBUFFv05_blockHeaderSize) nextSrcSizeHint+= ZBUFFv05_blockHeaderSize;   /* get next block header too */
-        nextSrcSizeHint -= zbc->inPos;   /* already loaded*/
-        return nextSrcSizeHint;
-    }
-}
-
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-unsigned ZBUFFv05_isError(size_t errorCode) { return ERR_isError(errorCode); }
-const char* ZBUFFv05_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
-
-size_t ZBUFFv05_recommendedDInSize(void)  { return BLOCKSIZE + ZBUFFv05_blockHeaderSize /* block header size*/ ; }
-size_t ZBUFFv05_recommendedDOutSize(void) { return BLOCKSIZE; }
diff --git a/vendor/github.com/DataDog/zstd/zstd_v05.h b/vendor/github.com/DataDog/zstd/zstd_v05.h
deleted file mode 100644
index 4a97985..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v05.h
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTDv05_H
-#define ZSTDv05_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*-*************************************
-*  Dependencies
-***************************************/
-#include <stddef.h>   /* size_t */
-#include "mem.h"      /* U64, U32 */
-
-
-/* *************************************
-*  Simple functions
-***************************************/
-/*! ZSTDv05_decompress() :
-    `compressedSize` : is the _exact_ size of the compressed blob, otherwise decompression will fail.
-    `dstCapacity` must be large enough, equal or larger than originalSize.
-    @return : the number of bytes decompressed into `dst` (<= `dstCapacity`),
-              or an errorCode if it fails (which can be tested using ZSTDv05_isError()) */
-size_t ZSTDv05_decompress( void* dst, size_t dstCapacity,
-                     const void* src, size_t compressedSize);
-
- /**
- ZSTDv05_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.5.x format
-     srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-     cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                 or an error code if it fails (which can be tested using ZSTDv01_isError())
-     dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                 or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
- */
-void ZSTDv05_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                     size_t* cSize, unsigned long long* dBound);
-
-/* *************************************
-*  Helper functions
-***************************************/
-/* Error Management */
-unsigned    ZSTDv05_isError(size_t code);          /*!< tells if a `size_t` function result is an error code */
-const char* ZSTDv05_getErrorName(size_t code);     /*!< provides readable string for an error code */
-
-
-/* *************************************
-*  Explicit memory management
-***************************************/
-/** Decompression context */
-typedef struct ZSTDv05_DCtx_s ZSTDv05_DCtx;
-ZSTDv05_DCtx* ZSTDv05_createDCtx(void);
-size_t ZSTDv05_freeDCtx(ZSTDv05_DCtx* dctx);      /*!< @return : errorCode */
-
-/** ZSTDv05_decompressDCtx() :
-*   Same as ZSTDv05_decompress(), but requires an already allocated ZSTDv05_DCtx (see ZSTDv05_createDCtx()) */
-size_t ZSTDv05_decompressDCtx(ZSTDv05_DCtx* ctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-/*-***********************
-*  Simple Dictionary API
-*************************/
-/*! ZSTDv05_decompress_usingDict() :
-*   Decompression using a pre-defined Dictionary content (see dictBuilder).
-*   Dictionary must be identical to the one used during compression, otherwise regenerated data will be corrupted.
-*   Note : dict can be NULL, in which case, it's equivalent to ZSTDv05_decompressDCtx() */
-size_t ZSTDv05_decompress_usingDict(ZSTDv05_DCtx* dctx,
-                                            void* dst, size_t dstCapacity,
-                                      const void* src, size_t srcSize,
-                                      const void* dict,size_t dictSize);
-
-/*-************************
-*  Advanced Streaming API
-***************************/
-typedef enum { ZSTDv05_fast, ZSTDv05_greedy, ZSTDv05_lazy, ZSTDv05_lazy2, ZSTDv05_btlazy2, ZSTDv05_opt, ZSTDv05_btopt } ZSTDv05_strategy;
-typedef struct {
-    U64 srcSize;
-    U32 windowLog;     /* the only useful information to retrieve */
-    U32 contentLog; U32 hashLog; U32 searchLog; U32 searchLength; U32 targetLength; ZSTDv05_strategy strategy;
-} ZSTDv05_parameters;
-size_t ZSTDv05_getFrameParams(ZSTDv05_parameters* params, const void* src, size_t srcSize);
-
-size_t ZSTDv05_decompressBegin_usingDict(ZSTDv05_DCtx* dctx, const void* dict, size_t dictSize);
-void   ZSTDv05_copyDCtx(ZSTDv05_DCtx* dstDCtx, const ZSTDv05_DCtx* srcDCtx);
-size_t ZSTDv05_nextSrcSizeToDecompress(ZSTDv05_DCtx* dctx);
-size_t ZSTDv05_decompressContinue(ZSTDv05_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-/*-***********************
-*  ZBUFF API
-*************************/
-typedef struct ZBUFFv05_DCtx_s ZBUFFv05_DCtx;
-ZBUFFv05_DCtx* ZBUFFv05_createDCtx(void);
-size_t         ZBUFFv05_freeDCtx(ZBUFFv05_DCtx* dctx);
-
-size_t ZBUFFv05_decompressInit(ZBUFFv05_DCtx* dctx);
-size_t ZBUFFv05_decompressInitDictionary(ZBUFFv05_DCtx* dctx, const void* dict, size_t dictSize);
-
-size_t ZBUFFv05_decompressContinue(ZBUFFv05_DCtx* dctx,
-                                            void* dst, size_t* dstCapacityPtr,
-                                      const void* src, size_t* srcSizePtr);
-
-/*-***************************************************************************
-*  Streaming decompression
-*
-*  A ZBUFFv05_DCtx object is required to track streaming operations.
-*  Use ZBUFFv05_createDCtx() and ZBUFFv05_freeDCtx() to create/release resources.
-*  Use ZBUFFv05_decompressInit() to start a new decompression operation,
-*   or ZBUFFv05_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFFv05_DCtx objects can be reused multiple times.
-*
-*  Use ZBUFFv05_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of @dst will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters or change @dst.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to help latency)
-*            or 0 when a frame is completely decoded
-*            or an error code, which can be tested using ZBUFFv05_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFFv05_recommendedDInSize() / ZBUFFv05_recommendedDOutSize()
-*  output : ZBUFFv05_recommendedDOutSize==128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFFv05_recommendedDInSize==128Kb+3; just follow indications from ZBUFFv05_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-unsigned ZBUFFv05_isError(size_t errorCode);
-const char* ZBUFFv05_getErrorName(size_t errorCode);
-
-/** Functions below provide recommended buffer sizes for Compression or Decompression operations.
-*   These sizes are just hints, and tend to offer better latency */
-size_t ZBUFFv05_recommendedDInSize(void);
-size_t ZBUFFv05_recommendedDOutSize(void);
-
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define ZSTDv05_MAGICNUMBER 0xFD2FB525   /* v0.5 */
-
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* ZSTDv0505_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v06.c b/vendor/github.com/DataDog/zstd/zstd_v06.c
deleted file mode 100644
index f907a3a..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v06.c
+++ /dev/null
@@ -1,4150 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/*- Dependencies -*/
-#include "zstd_v06.h"
-#include <stddef.h>    /* size_t, ptrdiff_t */
-#include <string.h>    /* memcpy */
-#include <stdlib.h>    /* malloc, free, qsort */
-#include "error_private.h"
-
-
-
-/* ******************************************************************
-   mem.h
-   low-level memory access routines
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*-****************************************
-*  Compiler specifics
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/*-**************************************************************
-*  Basic Types
-*****************************************************************/
-#if  !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) )
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/*-**************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS :
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets depending on alignment.
- *            In some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(size_t)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(size_t)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard, by lying on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; size_t st; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-
-#endif /* MEM_FORCE_MEMORY_ACCESS */
-
-MEM_STATIC U32 MEM_swap32(U32 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_ulong(in);
-#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)
-    return __builtin_bswap32(in);
-#else
-    return  ((in << 24) & 0xff000000 ) |
-            ((in <<  8) & 0x00ff0000 ) |
-            ((in >>  8) & 0x0000ff00 ) |
-            ((in >> 24) & 0x000000ff );
-#endif
-}
-
-MEM_STATIC U64 MEM_swap64(U64 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_uint64(in);
-#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)
-    return __builtin_bswap64(in);
-#else
-    return  ((in << 56) & 0xff00000000000000ULL) |
-            ((in << 40) & 0x00ff000000000000ULL) |
-            ((in << 24) & 0x0000ff0000000000ULL) |
-            ((in << 8)  & 0x000000ff00000000ULL) |
-            ((in >> 8)  & 0x00000000ff000000ULL) |
-            ((in >> 24) & 0x0000000000ff0000ULL) |
-            ((in >> 40) & 0x000000000000ff00ULL) |
-            ((in >> 56) & 0x00000000000000ffULL);
-#endif
-}
-
-
-/*=== Little endian r/w ===*/
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian()) {
-        MEM_write16(memPtr, val);
-    } else {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-        return MEM_swap32(MEM_read32(memPtr));
-}
-
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-        return MEM_swap64(MEM_read64(memPtr));
-}
-
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-
-/*
-    zstd - standard compression library
-    Header File for static linking only
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net
-*/
-#ifndef ZSTDv06_STATIC_H
-#define ZSTDv06_STATIC_H
-
-/* The prototypes defined within this file are considered experimental.
- * They should not be used in the context DLL as they may change in the future.
- * Prefer static linking if you need them, to control breaking version changes issues.
- */
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/*- Advanced Decompression functions -*/
-
-/*! ZSTDv06_decompress_usingPreparedDCtx() :
-*   Same as ZSTDv06_decompress_usingDict, but using a reference context `preparedDCtx`, where dictionary has been loaded.
-*   It avoids reloading the dictionary each time.
-*   `preparedDCtx` must have been properly initialized using ZSTDv06_decompressBegin_usingDict().
-*   Requires 2 contexts : 1 for reference (preparedDCtx), which will not be modified, and 1 to run the decompression operation (dctx) */
-ZSTDLIBv06_API size_t ZSTDv06_decompress_usingPreparedDCtx(
-                                           ZSTDv06_DCtx* dctx, const ZSTDv06_DCtx* preparedDCtx,
-                                           void* dst, size_t dstCapacity,
-                                     const void* src, size_t srcSize);
-
-
-
-#define ZSTDv06_FRAMEHEADERSIZE_MAX 13    /* for static allocation */
-static const size_t ZSTDv06_frameHeaderSize_min = 5;
-static const size_t ZSTDv06_frameHeaderSize_max = ZSTDv06_FRAMEHEADERSIZE_MAX;
-
-ZSTDLIBv06_API size_t ZSTDv06_decompressBegin(ZSTDv06_DCtx* dctx);
-
-/*
-  Streaming decompression, direct mode (bufferless)
-
-  A ZSTDv06_DCtx object is required to track streaming operations.
-  Use ZSTDv06_createDCtx() / ZSTDv06_freeDCtx() to manage it.
-  A ZSTDv06_DCtx object can be re-used multiple times.
-
-  First optional operation is to retrieve frame parameters, using ZSTDv06_getFrameParams(), which doesn't consume the input.
-  It can provide the minimum size of rolling buffer required to properly decompress data,
-  and optionally the final size of uncompressed content.
-  (Note : content size is an optional info that may not be present. 0 means : content size unknown)
-  Frame parameters are extracted from the beginning of compressed frame.
-  The amount of data to read is variable, from ZSTDv06_frameHeaderSize_min to ZSTDv06_frameHeaderSize_max (so if `srcSize` >= ZSTDv06_frameHeaderSize_max, it will always work)
-  If `srcSize` is too small for operation to succeed, function will return the minimum size it requires to produce a result.
-  Result : 0 when successful, it means the ZSTDv06_frameParams structure has been filled.
-          >0 : means there is not enough data into `src`. Provides the expected size to successfully decode header.
-           errorCode, which can be tested using ZSTDv06_isError()
-
-  Start decompression, with ZSTDv06_decompressBegin() or ZSTDv06_decompressBegin_usingDict().
-  Alternatively, you can copy a prepared context, using ZSTDv06_copyDCtx().
-
-  Then use ZSTDv06_nextSrcSizeToDecompress() and ZSTDv06_decompressContinue() alternatively.
-  ZSTDv06_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTDv06_decompressContinue().
-  ZSTDv06_decompressContinue() requires this exact amount of bytes, or it will fail.
-  ZSTDv06_decompressContinue() needs previous data blocks during decompression, up to (1 << windowlog).
-  They should preferably be located contiguously, prior to current block. Alternatively, a round buffer is also possible.
-
-  @result of ZSTDv06_decompressContinue() is the number of bytes regenerated within 'dst' (necessarily <= dstCapacity)
-  It can be zero, which is not an error; it just means ZSTDv06_decompressContinue() has decoded some header.
-
-  A frame is fully decoded when ZSTDv06_nextSrcSizeToDecompress() returns zero.
-  Context can then be reset to start a new decompression.
-*/
-
-
-/* **************************************
-*  Block functions
-****************************************/
-/*! Block functions produce and decode raw zstd blocks, without frame metadata.
-    User will have to take in charge required information to regenerate data, such as compressed and content sizes.
-
-    A few rules to respect :
-    - Uncompressed block size must be <= ZSTDv06_BLOCKSIZE_MAX (128 KB)
-    - Compressing or decompressing requires a context structure
-      + Use ZSTDv06_createCCtx() and ZSTDv06_createDCtx()
-    - It is necessary to init context before starting
-      + compression : ZSTDv06_compressBegin()
-      + decompression : ZSTDv06_decompressBegin()
-      + variants _usingDict() are also allowed
-      + copyCCtx() and copyDCtx() work too
-    - When a block is considered not compressible enough, ZSTDv06_compressBlock() result will be zero.
-      In which case, nothing is produced into `dst`.
-      + User must test for such outcome and deal directly with uncompressed data
-      + ZSTDv06_decompressBlock() doesn't accept uncompressed data as input !!
-*/
-
-#define ZSTDv06_BLOCKSIZE_MAX (128 * 1024)   /* define, for static allocation */
-ZSTDLIBv06_API size_t ZSTDv06_decompressBlock(ZSTDv06_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* ZSTDv06_STATIC_H */
-/*
-    zstd_internal - common functions to include
-    Header File for include
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : https://www.zstd.net
-*/
-#ifndef ZSTDv06_CCOMMON_H_MODULE
-#define ZSTDv06_CCOMMON_H_MODULE
-
-
-/*-*************************************
-*  Common macros
-***************************************/
-#define MIN(a,b) ((a)<(b) ? (a) : (b))
-#define MAX(a,b) ((a)>(b) ? (a) : (b))
-
-
-/*-*************************************
-*  Common constants
-***************************************/
-#define ZSTDv06_DICT_MAGIC  0xEC30A436
-
-#define ZSTDv06_REP_NUM    3
-#define ZSTDv06_REP_INIT   ZSTDv06_REP_NUM
-#define ZSTDv06_REP_MOVE   (ZSTDv06_REP_NUM-1)
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define ZSTDv06_WINDOWLOG_ABSOLUTEMIN 12
-static const size_t ZSTDv06_fcs_fieldSize[4] = { 0, 1, 2, 8 };
-
-#define ZSTDv06_BLOCKHEADERSIZE 3   /* because C standard does not allow a static const value to be defined using another static const value .... :( */
-static const size_t ZSTDv06_blockHeaderSize = ZSTDv06_BLOCKHEADERSIZE;
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-#define MIN_SEQUENCES_SIZE 1 /* nbSeq==0 */
-#define MIN_CBLOCK_SIZE (1 /*litCSize*/ + 1 /* RLE or RAW */ + MIN_SEQUENCES_SIZE /* nbSeq==0 */)   /* for a non-null block */
-
-#define HufLog 12
-
-#define IS_HUF 0
-#define IS_PCH 1
-#define IS_RAW 2
-#define IS_RLE 3
-
-#define LONGNBSEQ 0x7F00
-
-#define MINMATCH 3
-#define EQUAL_READ32 4
-#define REPCODE_STARTVALUE 1
-
-#define Litbits  8
-#define MaxLit ((1<<Litbits) - 1)
-#define MaxML  52
-#define MaxLL  35
-#define MaxOff 28
-#define MaxSeq MAX(MaxLL, MaxML)   /* Assumption : MaxOff < MaxLL,MaxML */
-#define MLFSELog    9
-#define LLFSELog    9
-#define OffFSELog   8
-
-#define FSEv06_ENCODING_RAW     0
-#define FSEv06_ENCODING_RLE     1
-#define FSEv06_ENCODING_STATIC  2
-#define FSEv06_ENCODING_DYNAMIC 3
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-static const U32 LL_bits[MaxLL+1] = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3, 4, 6, 7, 8, 9,10,11,12,
-                                     13,14,15,16 };
-static const S16 LL_defaultNorm[MaxLL+1] = { 4, 3, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1,
-                                             2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 2, 1, 1, 1, 1, 1,
-                                            -1,-1,-1,-1 };
-static const U32 LL_defaultNormLog = 6;
-
-static const U32 ML_bits[MaxML+1] = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3, 4, 4, 5, 7, 8, 9,10,11,
-                                     12,13,14,15,16 };
-static const S16 ML_defaultNorm[MaxML+1] = { 1, 4, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,-1,-1,
-                                            -1,-1,-1,-1,-1 };
-static const U32 ML_defaultNormLog = 6;
-
-static const S16 OF_defaultNorm[MaxOff+1] = { 1, 1, 1, 1, 1, 1, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
-                                              1, 1, 1, 1, 1, 1, 1, 1,-1,-1,-1,-1,-1 };
-static const U32 OF_defaultNormLog = 5;
-
-
-/*-*******************************************
-*  Shared functions to include for inlining
-*********************************************/
-static void ZSTDv06_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-#define COPY8(d,s) { ZSTDv06_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTDv06_wildcopy() :
-*   custom version of memcpy(), can copy up to 7 bytes too many (8 bytes if length==0) */
-#define WILDCOPY_OVERLENGTH 8
-MEM_STATIC void ZSTDv06_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do
-        COPY8(op, ip)
-    while (op < oend);
-}
-
-
-
-/*-*******************************************
-*  Private interfaces
-*********************************************/
-typedef struct {
-    U32 off;
-    U32 len;
-} ZSTDv06_match_t;
-
-typedef struct {
-    U32 price;
-    U32 off;
-    U32 mlen;
-    U32 litlen;
-    U32 rep[ZSTDv06_REP_INIT];
-} ZSTDv06_optimal_t;
-
-typedef struct { U32  unused; } ZSTDv06_stats_t;
-
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* litStart;
-    BYTE* lit;
-    U16*  litLengthStart;
-    U16*  litLength;
-    BYTE* llCodeStart;
-    U16*  matchLengthStart;
-    U16*  matchLength;
-    BYTE* mlCodeStart;
-    U32   longLengthID;   /* 0 == no longLength; 1 == Lit.longLength; 2 == Match.longLength; */
-    U32   longLengthPos;
-    /* opt */
-    ZSTDv06_optimal_t* priceTable;
-    ZSTDv06_match_t* matchTable;
-    U32* matchLengthFreq;
-    U32* litLengthFreq;
-    U32* litFreq;
-    U32* offCodeFreq;
-    U32  matchLengthSum;
-    U32  matchSum;
-    U32  litLengthSum;
-    U32  litSum;
-    U32  offCodeSum;
-    U32  log2matchLengthSum;
-    U32  log2matchSum;
-    U32  log2litLengthSum;
-    U32  log2litSum;
-    U32  log2offCodeSum;
-    U32  factor;
-    U32  cachedPrice;
-    U32  cachedLitLength;
-    const BYTE* cachedLiterals;
-    ZSTDv06_stats_t stats;
-} seqStore_t;
-
-void ZSTDv06_seqToCodes(const seqStore_t* seqStorePtr, size_t const nbSeq);
-
-
-#endif   /* ZSTDv06_CCOMMON_H_MODULE */
-/* ******************************************************************
-   FSE : Finite State Entropy codec
-   Public Prototypes declaration
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef FSEv06_H
-#define FSEv06_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/*-****************************************
-*  FSE simple functions
-******************************************/
-/*! FSEv06_decompress():
-    Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstCapacity'.
-    @return : size of regenerated data (<= maxDstSize),
-              or an error code, which can be tested using FSEv06_isError() .
-
-    ** Important ** : FSEv06_decompress() does not decompress non-compressible nor RLE data !!!
-    Why ? : making this distinction requires a header.
-    Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-size_t FSEv06_decompress(void* dst,  size_t dstCapacity,
-                const void* cSrc, size_t cSrcSize);
-
-
-/*-*****************************************
-*  Tool functions
-******************************************/
-size_t FSEv06_compressBound(size_t size);       /* maximum compressed size */
-
-/* Error Management */
-unsigned    FSEv06_isError(size_t code);        /* tells if a return value is an error code */
-const char* FSEv06_getErrorName(size_t code);   /* provides error code string (useful for debugging) */
-
-
-
-/*-*****************************************
-*  FSE detailed API
-******************************************/
-/*!
-
-FSEv06_decompress() does the following:
-1. read normalized counters with readNCount()
-2. build decoding table 'DTable' from normalized counters
-3. decode the data stream using decoding table 'DTable'
-
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and provide normalized distribution using external method.
-*/
-
-
-/* *** DECOMPRESSION *** */
-
-/*! FSEv06_readNCount():
-    Read compactly saved 'normalizedCounter' from 'rBuffer'.
-    @return : size read from 'rBuffer',
-              or an errorCode, which can be tested using FSEv06_isError().
-              maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-size_t FSEv06_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize);
-
-/*! Constructor and Destructor of FSEv06_DTable.
-    Note that its size depends on 'tableLog' */
-typedef unsigned FSEv06_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-FSEv06_DTable* FSEv06_createDTable(unsigned tableLog);
-void        FSEv06_freeDTable(FSEv06_DTable* dt);
-
-/*! FSEv06_buildDTable():
-    Builds 'dt', which must be already allocated, using FSEv06_createDTable().
-    return : 0, or an errorCode, which can be tested using FSEv06_isError() */
-size_t FSEv06_buildDTable (FSEv06_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSEv06_decompress_usingDTable():
-    Decompress compressed source `cSrc` of size `cSrcSize` using `dt`
-    into `dst` which must be already allocated.
-    @return : size of regenerated data (necessarily <= `dstCapacity`),
-              or an errorCode, which can be tested using FSEv06_isError() */
-size_t FSEv06_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSEv06_DTable* dt);
-
-/*!
-Tutorial :
-----------
-(Note : these functions only decompress FSE-compressed blocks.
- If block is uncompressed, use memcpy() instead
- If block is a single repeated byte, use memset() instead )
-
-The first step is to obtain the normalized frequencies of symbols.
-This can be performed by FSEv06_readNCount() if it was saved using FSEv06_writeNCount().
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short.
-In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
-or size the table to handle worst case situations (typically 256).
-FSEv06_readNCount() will provide 'tableLog' and 'maxSymbolValue'.
-The result of FSEv06_readNCount() is the number of bytes read from 'rBuffer'.
-Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that.
-If there is an error, the function will return an error code, which can be tested using FSEv06_isError().
-
-The next step is to build the decompression tables 'FSEv06_DTable' from 'normalizedCounter'.
-This is performed by the function FSEv06_buildDTable().
-The space required by 'FSEv06_DTable' must be already allocated using FSEv06_createDTable().
-If there is an error, the function will return an error code, which can be tested using FSEv06_isError().
-
-`FSEv06_DTable` can then be used to decompress `cSrc`, with FSEv06_decompress_usingDTable().
-`cSrcSize` must be strictly correct, otherwise decompression will fail.
-FSEv06_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`).
-If there is an error, the function will return an error code, which can be tested using FSEv06_isError(). (ex: dst buffer too small)
-*/
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSEv06_H */
-/* ******************************************************************
-   bitstream
-   Part of FSE library
-   header file (to include)
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which must be inlined for best performance.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-
-/*=========================================
-*  Target specific
-=========================================*/
-#if defined(__BMI__) && defined(__GNUC__)
-#  include <immintrin.h>   /* support for bextr (experimental) */
-#endif
-
-
-
-/*-********************************************
-*  bitStream decoding API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BITv06_DStream_t;
-
-typedef enum { BITv06_DStream_unfinished = 0,
-               BITv06_DStream_endOfBuffer = 1,
-               BITv06_DStream_completed = 2,
-               BITv06_DStream_overflow = 3 } BITv06_DStream_status;  /* result of BITv06_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BITv06_initDStream(BITv06_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BITv06_readBits(BITv06_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BITv06_DStream_status BITv06_reloadDStream(BITv06_DStream_t* bitD);
-MEM_STATIC unsigned BITv06_endOfDStream(const BITv06_DStream_t* bitD);
-
-
-
-/*-****************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BITv06_readBitsFast(BITv06_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/*-**************************************************************
-*  Internal functions
-****************************************************************/
-MEM_STATIC unsigned BITv06_highbit32 ( U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    unsigned r;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-    return r;
-#   endif
-}
-
-
-
-/*-********************************************************
-* bitStream decoding
-**********************************************************/
-/*! BITv06_initDStream() :
-*   Initialize a BITv06_DStream_t.
-*   `bitD` : a pointer to an already allocated BITv06_DStream_t structure.
-*   `srcSize` must be the *exact* size of the bitStream, in bytes.
-*   @return : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BITv06_initDStream(BITv06_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(bitD->bitContainer)) {  /* normal case */
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-          if (lastByte == 0) return ERROR(GENERIC);   /* endMark not present */
-          bitD->bitsConsumed = 8 - BITv06_highbit32(lastByte); }
-    } else {
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16);/* fall-through */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24);/* fall-through */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32);/* fall-through */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; /* fall-through */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; /* fall-through */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) <<  8; /* fall-through */
-            default: break;
-        }
-        { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-          if (lastByte == 0) return ERROR(GENERIC);   /* endMark not present */
-          bitD->bitsConsumed = 8 - BITv06_highbit32(lastByte); }
-        bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-
- MEM_STATIC size_t BITv06_lookBits(const BITv06_DStream_t* bitD, U32 nbBits)
-{
-    U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BITv06_lookBitsFast() :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv06_lookBitsFast(const BITv06_DStream_t* bitD, U32 nbBits)
-{
-    U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BITv06_skipBits(BITv06_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BITv06_readBits(BITv06_DStream_t* bitD, U32 nbBits)
-{
-    size_t const value = BITv06_lookBits(bitD, nbBits);
-    BITv06_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*! BITv06_readBitsFast() :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv06_readBitsFast(BITv06_DStream_t* bitD, U32 nbBits)
-{
-    size_t const value = BITv06_lookBitsFast(bitD, nbBits);
-    BITv06_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BITv06_DStream_status BITv06_reloadDStream(BITv06_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should never happen */
-        return BITv06_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer)) {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BITv06_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start) {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BITv06_DStream_endOfBuffer;
-        return BITv06_DStream_completed;
-    }
-    {   U32 nbBytes = bitD->bitsConsumed >> 3;
-        BITv06_DStream_status result = BITv06_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start) {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BITv06_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BITv06_endOfDStream() :
-*   @return Tells if DStream has exactly reached its end (all bits consumed).
-*/
-MEM_STATIC unsigned BITv06_endOfDStream(const BITv06_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
-/* ******************************************************************
-   FSE : Finite State Entropy coder
-   header file for static linking (only)
-   Copyright (C) 2013-2015, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-   - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef FSEv06_STATIC_H
-#define FSEv06_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* *****************************************
-*  Static allocation
-*******************************************/
-/* FSE buffer bounds */
-#define FSEv06_NCOUNTBOUND 512
-#define FSEv06_BLOCKBOUND(size) (size + (size>>7))
-#define FSEv06_COMPRESSBOUND(size) (FSEv06_NCOUNTBOUND + FSEv06_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* It is possible to statically allocate FSE CTable/DTable as a table of unsigned using below macros */
-#define FSEv06_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/* *****************************************
-*  FSE advanced API
-*******************************************/
-size_t FSEv06_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize);
-/* same as FSEv06_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr  */
-
-size_t FSEv06_buildDTable_raw (FSEv06_DTable* dt, unsigned nbBits);
-/* build a fake FSEv06_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-size_t FSEv06_buildDTable_rle (FSEv06_DTable* dt, unsigned char symbolValue);
-/* build a fake FSEv06_DTable, designed to always generate the same symbolValue */
-
-
-/* *****************************************
-*  FSE symbol decompression API
-*******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSEv06_DState_t;
-
-
-static void     FSEv06_initDState(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD, const FSEv06_DTable* dt);
-
-static unsigned char FSEv06_decodeSymbol(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD);
-
-
-/* *****************************************
-*  FSE unsafe API
-*******************************************/
-static unsigned char FSEv06_decodeSymbolFast(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* *****************************************
-*  Implementation of inlined functions
-*******************************************/
-
-
-/* ======    Decompression    ====== */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSEv06_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSEv06_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSEv06_initDState(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD, const FSEv06_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv06_DTableHeader* const DTableH = (const FSEv06_DTableHeader*)ptr;
-    DStatePtr->state = BITv06_readBits(bitD, DTableH->tableLog);
-    BITv06_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSEv06_peekSymbol(const FSEv06_DState_t* DStatePtr)
-{
-    FSEv06_decode_t const DInfo = ((const FSEv06_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    return DInfo.symbol;
-}
-
-MEM_STATIC void FSEv06_updateState(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD)
-{
-    FSEv06_decode_t const DInfo = ((const FSEv06_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    size_t const lowBits = BITv06_readBits(bitD, nbBits);
-    DStatePtr->state = DInfo.newState + lowBits;
-}
-
-MEM_STATIC BYTE FSEv06_decodeSymbol(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD)
-{
-    FSEv06_decode_t const DInfo = ((const FSEv06_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BITv06_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-/*! FSEv06_decodeSymbolFast() :
-    unsafe, only works if no symbol has a probability > 50% */
-MEM_STATIC BYTE FSEv06_decodeSymbolFast(FSEv06_DState_t* DStatePtr, BITv06_DStream_t* bitD)
-{
-    FSEv06_decode_t const DInfo = ((const FSEv06_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BITv06_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-
-
-#ifndef FSEv06_COMMONDEFS_ONLY
-
-/* **************************************************************
-*  Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSEv06_MAX_MEMORY_USAGE 14
-#define FSEv06_DEFAULT_MEMORY_USAGE 13
-
-/*!FSEv06_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSEv06_MAX_SYMBOL_VALUE 255
-
-
-/* **************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSEv06_FUNCTION_TYPE BYTE
-#define FSEv06_FUNCTION_EXTENSION
-#define FSEv06_DECODE_TYPE FSEv06_decode_t
-
-
-#endif   /* !FSEv06_COMMONDEFS_ONLY */
-
-
-/* ***************************************************************
-*  Constants
-*****************************************************************/
-#define FSEv06_MAX_TABLELOG  (FSEv06_MAX_MEMORY_USAGE-2)
-#define FSEv06_MAX_TABLESIZE (1U<<FSEv06_MAX_TABLELOG)
-#define FSEv06_MAXTABLESIZE_MASK (FSEv06_MAX_TABLESIZE-1)
-#define FSEv06_DEFAULT_TABLELOG (FSEv06_DEFAULT_MEMORY_USAGE-2)
-#define FSEv06_MIN_TABLELOG 5
-
-#define FSEv06_TABLELOG_ABSOLUTE_MAX 15
-#if FSEv06_MAX_TABLELOG > FSEv06_TABLELOG_ABSOLUTE_MAX
-#error "FSEv06_MAX_TABLELOG > FSEv06_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-#define FSEv06_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3)
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSEv06_STATIC_H */
-/*
-   Common functions of New Generation Entropy library
-   Copyright (C) 2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-*************************************************************************** */
-
-
-/*-****************************************
-*  FSE Error Management
-******************************************/
-unsigned FSEv06_isError(size_t code) { return ERR_isError(code); }
-
-const char* FSEv06_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/* **************************************************************
-*  HUF Error Management
-****************************************************************/
-static unsigned HUFv06_isError(size_t code) { return ERR_isError(code); }
-
-
-/*-**************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-static short FSEv06_abs(short a) { return a<0 ? -a : a; }
-
-size_t FSEv06_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSEv06_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSEv06_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr)) {
-        if (previous0) {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF) {
-                n0+=24;
-                if (ip < iend-5) {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                } else {
-                    bitStream >>= 16;
-                    bitCount+=16;
-            }   }
-            while ((bitStream & 3) == 3) {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {   short const max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max) {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            } else {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSEv06_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold) {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-            } else {
-                bitCount -= (int)(8 * (iend - 4 - ip));
-                ip = iend - 4;
-            }
-            bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-    }   }   /* while ((remaining>1) && (charnum<=*maxSVPtr)) */
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-/* ******************************************************************
-   FSE : Finite State Entropy decoder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSEv06_isError ERR_isError
-#define FSEv06_STATIC_ASSERT(c) { enum { FSEv06_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/* **************************************************************
-*  Complex types
-****************************************************************/
-typedef U32 DTable_max_t[FSEv06_DTABLE_SIZE_U32(FSEv06_MAX_TABLELOG)];
-
-
-/* **************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSEv06_FUNCTION_EXTENSION
-#  error "FSEv06_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSEv06_FUNCTION_TYPE
-#  error "FSEv06_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSEv06_CAT(X,Y) X##Y
-#define FSEv06_FUNCTION_NAME(X,Y) FSEv06_CAT(X,Y)
-#define FSEv06_TYPE_NAME(X,Y) FSEv06_CAT(X,Y)
-
-
-/* Function templates */
-FSEv06_DTable* FSEv06_createDTable (unsigned tableLog)
-{
-    if (tableLog > FSEv06_TABLELOG_ABSOLUTE_MAX) tableLog = FSEv06_TABLELOG_ABSOLUTE_MAX;
-    return (FSEv06_DTable*)malloc( FSEv06_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
-}
-
-void FSEv06_freeDTable (FSEv06_DTable* dt)
-{
-    free(dt);
-}
-
-size_t FSEv06_buildDTable(FSEv06_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* const tdPtr = dt+1;   /* because *dt is unsigned, 32-bits aligned on 32-bits */
-    FSEv06_DECODE_TYPE* const tableDecode = (FSEv06_DECODE_TYPE*) (tdPtr);
-    U16 symbolNext[FSEv06_MAX_SYMBOL_VALUE+1];
-
-    U32 const maxSV1 = maxSymbolValue + 1;
-    U32 const tableSize = 1 << tableLog;
-    U32 highThreshold = tableSize-1;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSEv06_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSEv06_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    {   FSEv06_DTableHeader DTableH;
-        DTableH.tableLog = (U16)tableLog;
-        DTableH.fastMode = 1;
-        {   S16 const largeLimit= (S16)(1 << (tableLog-1));
-            U32 s;
-            for (s=0; s<maxSV1; s++) {
-                if (normalizedCounter[s]==-1) {
-                    tableDecode[highThreshold--].symbol = (FSEv06_FUNCTION_TYPE)s;
-                    symbolNext[s] = 1;
-                } else {
-                    if (normalizedCounter[s] >= largeLimit) DTableH.fastMode=0;
-                    symbolNext[s] = normalizedCounter[s];
-        }   }   }
-        memcpy(dt, &DTableH, sizeof(DTableH));
-    }
-
-    /* Spread symbols */
-    {   U32 const tableMask = tableSize-1;
-        U32 const step = FSEv06_TABLESTEP(tableSize);
-        U32 s, position = 0;
-        for (s=0; s<maxSV1; s++) {
-            int i;
-            for (i=0; i<normalizedCounter[s]; i++) {
-                tableDecode[position].symbol = (FSEv06_FUNCTION_TYPE)s;
-                position = (position + step) & tableMask;
-                while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }   }
-
-        if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-    }
-
-    /* Build Decoding table */
-    {   U32 u;
-        for (u=0; u<tableSize; u++) {
-            FSEv06_FUNCTION_TYPE const symbol = (FSEv06_FUNCTION_TYPE)(tableDecode[u].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[u].nbBits = (BYTE) (tableLog - BITv06_highbit32 ((U32)nextState) );
-            tableDecode[u].newState = (U16) ( (nextState << tableDecode[u].nbBits) - tableSize);
-    }   }
-
-    return 0;
-}
-
-
-
-#ifndef FSEv06_COMMONDEFS_ONLY
-
-/*-*******************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-size_t FSEv06_buildDTable_rle (FSEv06_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSEv06_DTableHeader* const DTableH = (FSEv06_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv06_decode_t* const cell = (FSEv06_decode_t*)dPtr;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-size_t FSEv06_buildDTable_raw (FSEv06_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSEv06_DTableHeader* const DTableH = (FSEv06_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv06_decode_t* const dinfo = (FSEv06_decode_t*)dPtr;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSV1 = tableMask+1;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<maxSV1; s++) {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSEv06_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSEv06_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BITv06_DStream_t bitD;
-    FSEv06_DState_t state1;
-    FSEv06_DState_t state2;
-
-    /* Init */
-    { size_t const errorCode = BITv06_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-      if (FSEv06_isError(errorCode)) return errorCode; }
-
-    FSEv06_initDState(&state1, &bitD, dt);
-    FSEv06_initDState(&state2, &bitD, dt);
-
-#define FSEv06_GETSYMBOL(statePtr) fast ? FSEv06_decodeSymbolFast(statePtr, &bitD) : FSEv06_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BITv06_reloadDStream(&bitD)==BITv06_DStream_unfinished) && (op<olimit) ; op+=4) {
-        op[0] = FSEv06_GETSYMBOL(&state1);
-
-        if (FSEv06_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv06_reloadDStream(&bitD);
-
-        op[1] = FSEv06_GETSYMBOL(&state2);
-
-        if (FSEv06_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BITv06_reloadDStream(&bitD) > BITv06_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSEv06_GETSYMBOL(&state1);
-
-        if (FSEv06_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv06_reloadDStream(&bitD);
-
-        op[3] = FSEv06_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BITv06_reloadDStream(&bitD) >= FSEv06_DStream_partiallyFilled; Ends at exactly BITv06_DStream_completed */
-    while (1) {
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-
-        *op++ = FSEv06_GETSYMBOL(&state1);
-
-        if (BITv06_reloadDStream(&bitD)==BITv06_DStream_overflow) {
-            *op++ = FSEv06_GETSYMBOL(&state2);
-            break;
-        }
-
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-
-        *op++ = FSEv06_GETSYMBOL(&state2);
-
-        if (BITv06_reloadDStream(&bitD)==BITv06_DStream_overflow) {
-            *op++ = FSEv06_GETSYMBOL(&state1);
-            break;
-    }   }
-
-    return op-ostart;
-}
-
-
-size_t FSEv06_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSEv06_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv06_DTableHeader* DTableH = (const FSEv06_DTableHeader*)ptr;
-    const U32 fastMode = DTableH->fastMode;
-
-    /* select fast mode (static) */
-    if (fastMode) return FSEv06_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSEv06_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-size_t FSEv06_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSEv06_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSEv06_MAX_SYMBOL_VALUE;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSE decoding mode */
-    {   size_t const NCountLength = FSEv06_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-        if (FSEv06_isError(NCountLength)) return NCountLength;
-        if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-        ip += NCountLength;
-        cSrcSize -= NCountLength;
-    }
-
-    { size_t const errorCode = FSEv06_buildDTable (dt, counting, maxSymbolValue, tableLog);
-      if (FSEv06_isError(errorCode)) return errorCode; }
-
-    return FSEv06_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);   /* always return, even if it is an error code */
-}
-
-
-
-#endif   /* FSEv06_COMMONDEFS_ONLY */
-/* ******************************************************************
-   Huffman coder, part of New Generation Entropy library
-   header file
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef HUFv06_H
-#define HUFv06_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* ****************************************
-*  HUF simple functions
-******************************************/
-size_t HUFv06_decompress(void* dst,  size_t dstSize,
-                const void* cSrc, size_t cSrcSize);
-/*
-HUFv06_decompress() :
-    Decompress HUF data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstSize'.
-    `dstSize` : must be the **exact** size of original (uncompressed) data.
-    Note : in contrast with FSE, HUFv06_decompress can regenerate
-           RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data,
-           because it knows size to regenerate.
-    @return : size of regenerated data (== dstSize)
-              or an error code, which can be tested using HUFv06_isError()
-*/
-
-
-/* ****************************************
-*  Tool functions
-******************************************/
-size_t HUFv06_compressBound(size_t size);       /**< maximum compressed size */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* HUFv06_H */
-/* ******************************************************************
-   Huffman codec, part of New Generation Entropy library
-   header file, for static linking only
-   Copyright (C) 2013-2016, Yann Collet
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef HUFv06_STATIC_H
-#define HUFv06_STATIC_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/* ****************************************
-*  Static allocation
-******************************************/
-/* HUF buffer bounds */
-#define HUFv06_CTABLEBOUND 129
-#define HUFv06_BLOCKBOUND(size) (size + (size>>8) + 8)   /* only true if incompressible pre-filtered with fast heuristic */
-#define HUFv06_COMPRESSBOUND(size) (HUFv06_CTABLEBOUND + HUFv06_BLOCKBOUND(size))   /* Macro version, useful for static allocation */
-
-/* static allocation of HUF's DTable */
-#define HUFv06_DTABLE_SIZE(maxTableLog)   (1 + (1<<maxTableLog))
-#define HUFv06_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        unsigned short DTable[HUFv06_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUFv06_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        unsigned int DTable[HUFv06_DTABLE_SIZE(maxTableLog)] = { maxTableLog }
-#define HUFv06_CREATE_STATIC_DTABLEX6(DTable, maxTableLog) \
-        unsigned int DTable[HUFv06_DTABLE_SIZE(maxTableLog) * 3 / 2] = { maxTableLog }
-
-
-/* ****************************************
-*  Advanced decompression functions
-******************************************/
-size_t HUFv06_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-size_t HUFv06_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbols decoder */
-
-
-
-/*!
-HUFv06_decompress() does the following:
-1. select the decompression algorithm (X2, X4, X6) based on pre-computed heuristics
-2. build Huffman table from save, using HUFv06_readDTableXn()
-3. decode 1 or 4 segments in parallel using HUFv06_decompressSXn_usingDTable
-*/
-size_t HUFv06_readDTableX2 (unsigned short* DTable, const void* src, size_t srcSize);
-size_t HUFv06_readDTableX4 (unsigned* DTable, const void* src, size_t srcSize);
-
-size_t HUFv06_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned short* DTable);
-size_t HUFv06_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned* DTable);
-
-
-/* single stream variants */
-size_t HUFv06_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-size_t HUFv06_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbol decoder */
-
-size_t HUFv06_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned short* DTable);
-size_t HUFv06_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const unsigned* DTable);
-
-
-
-/* **************************************************************
-*  Constants
-****************************************************************/
-#define HUFv06_ABSOLUTEMAX_TABLELOG  16   /* absolute limit of HUFv06_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUFv06_MAX_TABLELOG  12           /* max configured tableLog (for static allocation); can be modified up to HUFv06_ABSOLUTEMAX_TABLELOG */
-#define HUFv06_DEFAULT_TABLELOG  HUFv06_MAX_TABLELOG   /* tableLog by default, when not specified */
-#define HUFv06_MAX_SYMBOL_VALUE 255
-#if (HUFv06_MAX_TABLELOG > HUFv06_ABSOLUTEMAX_TABLELOG)
-#  error "HUFv06_MAX_TABLELOG is too large !"
-#endif
-
-
-
-/*! HUFv06_readStats() :
-    Read compact Huffman tree, saved by HUFv06_writeCTable().
-    `huffWeight` is destination buffer.
-    @return : size read from `src`
-*/
-MEM_STATIC size_t HUFv06_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                            U32* nbSymbolsPtr, U32* tableLogPtr,
-                            const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  { /* special header */
-        if (iSize >= (242)) {  /* RLE */
-            static U32 l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else {   /* Incompressible */
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            {   U32 n;
-                for (n=0; n<oSize; n+=2) {
-                    huffWeight[n]   = ip[n/2] >> 4;
-                    huffWeight[n+1] = ip[n/2] & 15;
-    }   }   }   }
-    else  {   /* header compressed with FSE (normal case) */
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSEv06_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSEv06_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUFv06_ABSOLUTEMAX_TABLELOG + 1) * sizeof(U32));
-    weightTotal = 0;
-    {   U32 n; for (n=0; n<oSize; n++) {
-            if (huffWeight[n] >= HUFv06_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-            rankStats[huffWeight[n]]++;
-            weightTotal += (1 << huffWeight[n]) >> 1;
-    }   }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    {   U32 const tableLog = BITv06_highbit32(weightTotal) + 1;
-        if (tableLog > HUFv06_ABSOLUTEMAX_TABLELOG) return ERROR(corruption_detected);
-        *tableLogPtr = tableLog;
-        /* determine last weight */
-        {   U32 const total = 1 << tableLog;
-            U32 const rest = total - weightTotal;
-            U32 const verif = 1 << BITv06_highbit32(rest);
-            U32 const lastWeight = BITv06_highbit32(rest) + 1;
-            if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-            huffWeight[oSize] = (BYTE)lastWeight;
-            rankStats[lastWeight]++;
-    }   }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    return iSize+1;
-}
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* HUFv06_STATIC_H */
-/* ******************************************************************
-   Huffman decoder, part of New Generation Entropy library
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define HUFv06_STATIC_ASSERT(c) { enum { HUFv06_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-
-/* *******************************************************
-*  HUF : Huffman block decompression
-*********************************************************/
-typedef struct { BYTE byte; BYTE nbBits; } HUFv06_DEltX2;   /* single-symbol decoding */
-
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUFv06_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-
-
-/*-***************************/
-/*  single-symbol decoding   */
-/*-***************************/
-
-size_t HUFv06_readDTableX2 (U16* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUFv06_MAX_SYMBOL_VALUE + 1];
-    U32 rankVal[HUFv06_ABSOLUTEMAX_TABLELOG + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    size_t iSize;
-    U32 nbSymbols = 0;
-    U32 n;
-    U32 nextRankStart;
-    void* const dtPtr = DTable + 1;
-    HUFv06_DEltX2* const dt = (HUFv06_DEltX2*)dtPtr;
-
-    HUFv06_STATIC_ASSERT(sizeof(HUFv06_DEltX2) == sizeof(U16));   /* if compilation fails here, assertion is false */
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv06_readStats(huffWeight, HUFv06_MAX_SYMBOL_VALUE + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv06_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > DTable[0]) return ERROR(tableLog_tooLarge);   /* DTable is too small */
-    DTable[0] = (U16)tableLog;   /* maybe should separate sizeof allocated DTable, from used size of DTable, in case of re-use */
-
-    /* Prepare ranks */
-    nextRankStart = 0;
-    for (n=1; n<tableLog+1; n++) {
-        U32 current = nextRankStart;
-        nextRankStart += (rankVal[n] << (n-1));
-        rankVal[n] = current;
-    }
-
-    /* fill DTable */
-    for (n=0; n<nbSymbols; n++) {
-        const U32 w = huffWeight[n];
-        const U32 length = (1 << w) >> 1;
-        U32 i;
-        HUFv06_DEltX2 D;
-        D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-        for (i = rankVal[w]; i < rankVal[w] + length; i++)
-            dt[i] = D;
-        rankVal[w] += length;
-    }
-
-    return iSize;
-}
-
-
-static BYTE HUFv06_decodeSymbolX2(BITv06_DStream_t* Dstream, const HUFv06_DEltX2* dt, const U32 dtLog)
-{
-    const size_t val = BITv06_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-    const BYTE c = dt[val].byte;
-    BITv06_skipBits(Dstream, dt[val].nbBits);
-    return c;
-}
-
-#define HUFv06_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUFv06_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUFv06_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv06_MAX_TABLELOG<=12)) \
-        HUFv06_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUFv06_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUFv06_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUFv06_decodeStreamX2(BYTE* p, BITv06_DStream_t* const bitDPtr, BYTE* const pEnd, const HUFv06_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BITv06_reloadDStream(bitDPtr) == BITv06_DStream_unfinished) && (p <= pEnd-4)) {
-        HUFv06_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BITv06_reloadDStream(bitDPtr) == BITv06_DStream_unfinished) && (p < pEnd))
-        HUFv06_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUFv06_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-size_t HUFv06_decompress1X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + dstSize;
-    const U32 dtLog = DTable[0];
-    const void* dtPtr = DTable;
-    const HUFv06_DEltX2* const dt = ((const HUFv06_DEltX2*)dtPtr)+1;
-    BITv06_DStream_t bitD;
-
-    { size_t const errorCode = BITv06_initDStream(&bitD, cSrc, cSrcSize);
-      if (HUFv06_isError(errorCode)) return errorCode; }
-
-    HUFv06_decodeStreamX2(op, &bitD, oend, dt, dtLog);
-
-    /* check */
-    if (!BITv06_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    return dstSize;
-}
-
-size_t HUFv06_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv06_CREATE_STATIC_DTABLEX2(DTable, HUFv06_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const errorCode = HUFv06_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUFv06_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUFv06_decompress1X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-size_t HUFv06_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U16* DTable)
-{
-    /* Check */
-    if (cSrcSize < 10) return ERROR(corruption_detected);  /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUFv06_DEltX2* const dt = ((const HUFv06_DEltX2*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BITv06_DStream_t bitD1;
-        BITv06_DStream_t bitD2;
-        BITv06_DStream_t bitD3;
-        BITv06_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BITv06_initDStream(&bitD1, istart1, length1);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD2, istart2, length2);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD3, istart3, length3);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD4, istart4, length4);
-        if (HUFv06_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv06_reloadDStream(&bitD1) | BITv06_reloadDStream(&bitD2) | BITv06_reloadDStream(&bitD3) | BITv06_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv06_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv06_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX2_0(op4, &bitD4);
-            endSignal = BITv06_reloadDStream(&bitD1) | BITv06_reloadDStream(&bitD2) | BITv06_reloadDStream(&bitD3) | BITv06_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv06_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv06_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv06_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv06_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BITv06_endOfDStream(&bitD1) & BITv06_endOfDStream(&bitD2) & BITv06_endOfDStream(&bitD3) & BITv06_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv06_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv06_CREATE_STATIC_DTABLEX2(DTable, HUFv06_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const errorCode = HUFv06_readDTableX2 (DTable, cSrc, cSrcSize);
-    if (HUFv06_isError(errorCode)) return errorCode;
-    if (errorCode >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += errorCode;
-    cSrcSize -= errorCode;
-
-    return HUFv06_decompress4X2_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-/* *************************/
-/* double-symbols decoding */
-/* *************************/
-
-static void HUFv06_fillDTableX4Level2(HUFv06_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUFv06_DEltX4 DElt;
-    U32 rankVal[HUFv06_ABSOLUTEMAX_TABLELOG + 1];
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1) {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    { U32 s; for (s=0; s<sortedListSize; s++) {   /* note : sortedSymbols already skipped */
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }}
-}
-
-typedef U32 rankVal_t[HUFv06_ABSOLUTEMAX_TABLELOG][HUFv06_ABSOLUTEMAX_TABLELOG + 1];
-
-static void HUFv06_fillDTableX4(HUFv06_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUFv06_ABSOLUTEMAX_TABLELOG + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++) {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits) {   /* enough room for a second symbol */
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUFv06_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        } else {
-            HUFv06_DEltX4 DElt;
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits = (BYTE)(nbBits);
-            DElt.length = 1;
-            {   U32 u;
-                const U32 end = start + length;
-                for (u = start; u < end; u++) DTable[u] = DElt;
-        }   }
-        rankVal[weight] += length;
-    }
-}
-
-size_t HUFv06_readDTableX4 (U32* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUFv06_MAX_SYMBOL_VALUE + 1];
-    sortedSymbol_t sortedSymbol[HUFv06_MAX_SYMBOL_VALUE + 1];
-    U32 rankStats[HUFv06_ABSOLUTEMAX_TABLELOG + 1] = { 0 };
-    U32 rankStart0[HUFv06_ABSOLUTEMAX_TABLELOG + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    const U32 memLog = DTable[0];
-    size_t iSize;
-    void* dtPtr = DTable;
-    HUFv06_DEltX4* const dt = ((HUFv06_DEltX4*)dtPtr) + 1;
-
-    HUFv06_STATIC_ASSERT(sizeof(HUFv06_DEltX4) == sizeof(U32));   /* if compilation fails here, assertion is false */
-    if (memLog > HUFv06_ABSOLUTEMAX_TABLELOG) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv06_readStats(weightList, HUFv06_MAX_SYMBOL_VALUE + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv06_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > memLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--) {}  /* necessarily finds a solution before 0 */
-
-    /* Get start index of each weight */
-    {   U32 w, nextRankStart = 0;
-        for (w=1; w<maxW+1; w++) {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {   U32 s;
-        for (s=0; s<nbSymbols; s++) {
-            U32 const w = weightList[s];
-            U32 const r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {   U32* const rankVal0 = rankVal[0];
-        {   int const rescale = (memLog-tableLog) - 1;   /* tableLog <= memLog */
-            U32 nextRankVal = 0;
-            U32 w;
-            for (w=1; w<maxW+1; w++) {
-                U32 current = nextRankVal;
-                nextRankVal += rankStats[w] << (w+rescale);
-                rankVal0[w] = current;
-        }   }
-        {   U32 const minBits = tableLog+1 - maxW;
-            U32 consumed;
-            for (consumed = minBits; consumed < memLog - minBits + 1; consumed++) {
-                U32* const rankValPtr = rankVal[consumed];
-                U32 w;
-                for (w = 1; w < maxW+1; w++) {
-                    rankValPtr[w] = rankVal0[w] >> consumed;
-    }   }   }   }
-
-    HUFv06_fillDTableX4(dt, memLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    return iSize;
-}
-
-
-static U32 HUFv06_decodeSymbolX4(void* op, BITv06_DStream_t* DStream, const HUFv06_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv06_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BITv06_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUFv06_decodeLastSymbolX4(void* op, BITv06_DStream_t* DStream, const HUFv06_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv06_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BITv06_skipBits(DStream, dt[val].nbBits);
-    else {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) {
-            BITv06_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-    }   }
-    return 1;
-}
-
-
-#define HUFv06_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUFv06_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv06_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv06_MAX_TABLELOG<=12)) \
-        ptr += HUFv06_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv06_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUFv06_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUFv06_decodeStreamX4(BYTE* p, BITv06_DStream_t* bitDPtr, BYTE* const pEnd, const HUFv06_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BITv06_reloadDStream(bitDPtr) == BITv06_DStream_unfinished) && (p < pEnd-7)) {
-        HUFv06_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv06_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BITv06_reloadDStream(bitDPtr) == BITv06_DStream_unfinished) && (p <= pEnd-2))
-        HUFv06_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUFv06_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUFv06_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-
-size_t HUFv06_decompress1X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    const BYTE* const istart = (const BYTE*) cSrc;
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* const oend = ostart + dstSize;
-
-    const U32 dtLog = DTable[0];
-    const void* const dtPtr = DTable;
-    const HUFv06_DEltX4* const dt = ((const HUFv06_DEltX4*)dtPtr) +1;
-
-    /* Init */
-    BITv06_DStream_t bitD;
-    { size_t const errorCode = BITv06_initDStream(&bitD, istart, cSrcSize);
-      if (HUFv06_isError(errorCode)) return errorCode; }
-
-    /* decode */
-    HUFv06_decodeStreamX4(ostart, &bitD, oend, dt, dtLog);
-
-    /* check */
-    if (!BITv06_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    /* decoded size */
-    return dstSize;
-}
-
-size_t HUFv06_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv06_CREATE_STATIC_DTABLEX4(DTable, HUFv06_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUFv06_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUFv06_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUFv06_decompress1X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-size_t HUFv06_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const U32* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable;
-        const HUFv06_DEltX4* const dt = ((const HUFv06_DEltX4*)dtPtr) +1;
-        const U32 dtLog = DTable[0];
-        size_t errorCode;
-
-        /* Init */
-        BITv06_DStream_t bitD1;
-        BITv06_DStream_t bitD2;
-        BITv06_DStream_t bitD3;
-        BITv06_DStream_t bitD4;
-        const size_t length1 = MEM_readLE16(istart);
-        const size_t length2 = MEM_readLE16(istart+2);
-        const size_t length3 = MEM_readLE16(istart+4);
-        size_t length4;
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-
-        length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        errorCode = BITv06_initDStream(&bitD1, istart1, length1);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD2, istart2, length2);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD3, istart3, length3);
-        if (HUFv06_isError(errorCode)) return errorCode;
-        errorCode = BITv06_initDStream(&bitD4, istart4, length4);
-        if (HUFv06_isError(errorCode)) return errorCode;
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv06_reloadDStream(&bitD1) | BITv06_reloadDStream(&bitD2) | BITv06_reloadDStream(&bitD3) | BITv06_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv06_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv06_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv06_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUFv06_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUFv06_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUFv06_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BITv06_reloadDStream(&bitD1) | BITv06_reloadDStream(&bitD2) | BITv06_reloadDStream(&bitD3) | BITv06_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv06_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv06_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv06_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv06_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BITv06_endOfDStream(&bitD1) & BITv06_endOfDStream(&bitD2) & BITv06_endOfDStream(&bitD3) & BITv06_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv06_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv06_CREATE_STATIC_DTABLEX4(DTable, HUFv06_MAX_TABLELOG);
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUFv06_readDTableX4 (DTable, cSrc, cSrcSize);
-    if (HUFv06_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize;
-    cSrcSize -= hSize;
-
-    return HUFv06_decompress4X4_usingDTable (dst, dstSize, ip, cSrcSize, DTable);
-}
-
-
-
-
-/* ********************************/
-/* Generic decompression selector */
-/* ********************************/
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-size_t HUFv06_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[3] = { HUFv06_decompress4X2, HUFv06_decompress4X4, NULL };
-    U32 Dtime[3];   /* decompression time estimation */
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    /* decoder timing evaluation */
-    {   U32 const Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-        U32 const D256 = (U32)(dstSize >> 8);
-        U32 n; for (n=0; n<3; n++)
-            Dtime[n] = algoTime[Q][n].tableTime + (algoTime[Q][n].decode256Time * D256);
-    }
-
-    Dtime[1] += Dtime[1] >> 4; Dtime[2] += Dtime[2] >> 3; /* advantage to algorithms using less memory, for cache eviction */
-
-    {   U32 algoNb = 0;
-        if (Dtime[1] < Dtime[0]) algoNb = 1;
-        // if (Dtime[2] < Dtime[algoNb]) algoNb = 2;   /* current speed of HUFv06_decompress4X6 is not good */
-        return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-    }
-
-    //return HUFv06_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUFv06_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-    //return HUFv06_decompress4X6(dst, dstSize, cSrc, cSrcSize);   /* multi-streams quad-symbols decoding */
-}
-/*
-    Common functions of Zstd compression library
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net/
-*/
-
-
-/*-****************************************
-*  Version
-******************************************/
-
-/*-****************************************
-*  ZSTD Error Management
-******************************************/
-/*! ZSTDv06_isError() :
-*   tells if a return value is an error code */
-unsigned ZSTDv06_isError(size_t code) { return ERR_isError(code); }
-
-/*! ZSTDv06_getErrorName() :
-*   provides error code string from function result (useful for debugging) */
-const char* ZSTDv06_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/* **************************************************************
-*  ZBUFF Error Management
-****************************************************************/
-unsigned ZBUFFv06_isError(size_t errorCode) { return ERR_isError(errorCode); }
-
-const char* ZBUFFv06_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
- * HEAPMODE :
- * Select how default decompression function ZSTDv06_decompress() will allocate memory,
- * in memory stack (0), or in memory heap (1, requires malloc())
- */
-#ifndef ZSTDv06_HEAPMODE
-#  define ZSTDv06_HEAPMODE 1
-#endif
-
-
-
-/*-*******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#endif
-
-
-/*-*************************************
-*  Macros
-***************************************/
-#define ZSTDv06_isError ERR_isError   /* for inlining */
-#define FSEv06_isError  ERR_isError
-#define HUFv06_isError  ERR_isError
-
-
-/*_*******************************************************
-*  Memory operations
-**********************************************************/
-static void ZSTDv06_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-
-/*-*************************************************************
-*   Context management
-***************************************************************/
-typedef enum { ZSTDds_getFrameHeaderSize, ZSTDds_decodeFrameHeader,
-               ZSTDds_decodeBlockHeader, ZSTDds_decompressBlock } ZSTDv06_dStage;
-
-struct ZSTDv06_DCtx_s
-{
-    FSEv06_DTable LLTable[FSEv06_DTABLE_SIZE_U32(LLFSELog)];
-    FSEv06_DTable OffTable[FSEv06_DTABLE_SIZE_U32(OffFSELog)];
-    FSEv06_DTable MLTable[FSEv06_DTABLE_SIZE_U32(MLFSELog)];
-    unsigned   hufTableX4[HUFv06_DTABLE_SIZE(HufLog)];
-    const void* previousDstEnd;
-    const void* base;
-    const void* vBase;
-    const void* dictEnd;
-    size_t expected;
-    size_t headerSize;
-    ZSTDv06_frameParams fParams;
-    blockType_t bType;   /* used in ZSTDv06_decompressContinue(), to transfer blockType between header decoding and block decoding stages */
-    ZSTDv06_dStage stage;
-    U32 flagRepeatTable;
-    const BYTE* litPtr;
-    size_t litSize;
-    BYTE litBuffer[ZSTDv06_BLOCKSIZE_MAX + WILDCOPY_OVERLENGTH];
-    BYTE headerBuffer[ZSTDv06_FRAMEHEADERSIZE_MAX];
-};  /* typedef'd to ZSTDv06_DCtx within "zstd_static.h" */
-
-size_t ZSTDv06_sizeofDCtx (void); /* Hidden declaration */
-size_t ZSTDv06_sizeofDCtx (void) { return sizeof(ZSTDv06_DCtx); }
-
-size_t ZSTDv06_decompressBegin(ZSTDv06_DCtx* dctx)
-{
-    dctx->expected = ZSTDv06_frameHeaderSize_min;
-    dctx->stage = ZSTDds_getFrameHeaderSize;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    dctx->vBase = NULL;
-    dctx->dictEnd = NULL;
-    dctx->hufTableX4[0] = HufLog;
-    dctx->flagRepeatTable = 0;
-    return 0;
-}
-
-ZSTDv06_DCtx* ZSTDv06_createDCtx(void)
-{
-    ZSTDv06_DCtx* dctx = (ZSTDv06_DCtx*)malloc(sizeof(ZSTDv06_DCtx));
-    if (dctx==NULL) return NULL;
-    ZSTDv06_decompressBegin(dctx);
-    return dctx;
-}
-
-size_t ZSTDv06_freeDCtx(ZSTDv06_DCtx* dctx)
-{
-    free(dctx);
-    return 0;   /* reserved as a potential error code in the future */
-}
-
-void ZSTDv06_copyDCtx(ZSTDv06_DCtx* dstDCtx, const ZSTDv06_DCtx* srcDCtx)
-{
-    memcpy(dstDCtx, srcDCtx,
-           sizeof(ZSTDv06_DCtx) - (ZSTDv06_BLOCKSIZE_MAX+WILDCOPY_OVERLENGTH + ZSTDv06_frameHeaderSize_max));  /* no need to copy workspace */
-}
-
-
-/*-*************************************************************
-*   Decompression section
-***************************************************************/
-
-/* Frame format description
-   Frame Header -  [ Block Header - Block ] - Frame End
-   1) Frame Header
-      - 4 bytes - Magic Number : ZSTDv06_MAGICNUMBER (defined within zstd_static.h)
-      - 1 byte  - Frame Descriptor
-   2) Block Header
-      - 3 bytes, starting with a 2-bits descriptor
-                 Uncompressed, Compressed, Frame End, unused
-   3) Block
-      See Block Format Description
-   4) Frame End
-      - 3 bytes, compatible with Block Header
-*/
-
-
-/* Frame descriptor
-
-   1 byte, using :
-   bit 0-3 : windowLog - ZSTDv06_WINDOWLOG_ABSOLUTEMIN   (see zstd_internal.h)
-   bit 4   : minmatch 4(0) or 3(1)
-   bit 5   : reserved (must be zero)
-   bit 6-7 : Frame content size : unknown, 1 byte, 2 bytes, 8 bytes
-
-   Optional : content size (0, 1, 2 or 8 bytes)
-   0 : unknown
-   1 : 0-255 bytes
-   2 : 256 - 65535+256
-   8 : up to 16 exa
-*/
-
-
-/* Compressed Block, format description
-
-   Block = Literal Section - Sequences Section
-   Prerequisite : size of (compressed) block, maximum size of regenerated data
-
-   1) Literal Section
-
-   1.1) Header : 1-5 bytes
-        flags: 2 bits
-            00 compressed by Huff0
-            01 unused
-            10 is Raw (uncompressed)
-            11 is Rle
-            Note : using 01 => Huff0 with precomputed table ?
-            Note : delta map ? => compressed ?
-
-   1.1.1) Huff0-compressed literal block : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-   1.1.2) Raw (uncompressed) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RAW<<6) + (0<<4) + size
-               12 bits: (IS_RAW<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RAW<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.3) Rle (repeated single byte) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RLE<<6) + (0<<4) + size
-               12 bits: (IS_RLE<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RLE<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.4) Huff0-compressed literal block, using precomputed CTables : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-        1- CTable available (stored into workspace ?)
-        2- Small input (fast heuristic ? Full comparison ? depend on clevel ?)
-
-
-   1.2) Literal block content
-
-   1.2.1) Huff0 block, using sizes from header
-        See Huff0 format
-
-   1.2.2) Huff0 block, using prepared table
-
-   1.2.3) Raw content
-
-   1.2.4) single byte
-
-
-   2) Sequences section
-      TO DO
-*/
-
-/** ZSTDv06_frameHeaderSize() :
-*   srcSize must be >= ZSTDv06_frameHeaderSize_min.
-*   @return : size of the Frame Header */
-static size_t ZSTDv06_frameHeaderSize(const void* src, size_t srcSize)
-{
-    if (srcSize < ZSTDv06_frameHeaderSize_min) return ERROR(srcSize_wrong);
-    { U32 const fcsId = (((const BYTE*)src)[4]) >> 6;
-      return ZSTDv06_frameHeaderSize_min + ZSTDv06_fcs_fieldSize[fcsId]; }
-}
-
-
-/** ZSTDv06_getFrameParams() :
-*   decode Frame Header, or provide expected `srcSize`.
-*   @return : 0, `fparamsPtr` is correctly filled,
-*            >0, `srcSize` is too small, result is expected `srcSize`,
-*             or an error code, which can be tested using ZSTDv06_isError() */
-size_t ZSTDv06_getFrameParams(ZSTDv06_frameParams* fparamsPtr, const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-
-    if (srcSize < ZSTDv06_frameHeaderSize_min) return ZSTDv06_frameHeaderSize_min;
-    if (MEM_readLE32(src) != ZSTDv06_MAGICNUMBER) return ERROR(prefix_unknown);
-
-    /* ensure there is enough `srcSize` to fully read/decode frame header */
-    { size_t const fhsize = ZSTDv06_frameHeaderSize(src, srcSize);
-      if (srcSize < fhsize) return fhsize; }
-
-    memset(fparamsPtr, 0, sizeof(*fparamsPtr));
-    {   BYTE const frameDesc = ip[4];
-        fparamsPtr->windowLog = (frameDesc & 0xF) + ZSTDv06_WINDOWLOG_ABSOLUTEMIN;
-        if ((frameDesc & 0x20) != 0) return ERROR(frameParameter_unsupported);   /* reserved 1 bit */
-        switch(frameDesc >> 6)  /* fcsId */
-        {
-            default:   /* impossible */
-            case 0 : fparamsPtr->frameContentSize = 0; break;
-            case 1 : fparamsPtr->frameContentSize = ip[5]; break;
-            case 2 : fparamsPtr->frameContentSize = MEM_readLE16(ip+5)+256; break;
-            case 3 : fparamsPtr->frameContentSize = MEM_readLE64(ip+5); break;
-    }   }
-    return 0;
-}
-
-
-/** ZSTDv06_decodeFrameHeader() :
-*   `srcSize` must be the size provided by ZSTDv06_frameHeaderSize().
-*   @return : 0 if success, or an error code, which can be tested using ZSTDv06_isError() */
-static size_t ZSTDv06_decodeFrameHeader(ZSTDv06_DCtx* zc, const void* src, size_t srcSize)
-{
-    size_t const result = ZSTDv06_getFrameParams(&(zc->fParams), src, srcSize);
-    if ((MEM_32bits()) && (zc->fParams.windowLog > 25)) return ERROR(frameParameter_unsupported);
-    return result;
-}
-
-
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-/*! ZSTDv06_getcBlockSize() :
-*   Provides the size of compressed block from block header `src` */
-static size_t ZSTDv06_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    U32 cSize;
-
-    if (srcSize < ZSTDv06_blockHeaderSize) return ERROR(srcSize_wrong);
-
-    bpPtr->blockType = (blockType_t)((*in) >> 6);
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-
-static size_t ZSTDv06_copyRawBlock(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    if (dst==NULL) return ERROR(dstSize_tooSmall);
-    if (srcSize > dstCapacity) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/*! ZSTDv06_decodeLiteralsBlock() :
-    @return : nb of bytes read from src (< srcSize ) */
-static size_t ZSTDv06_decodeLiteralsBlock(ZSTDv06_DCtx* dctx,
-                          const void* src, size_t srcSize)   /* note : srcSize < BLOCKSIZE */
-{
-    const BYTE* const istart = (const BYTE*) src;
-
-    /* any compressed block with literals segment must be at least this size */
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch(istart[0]>> 6)
-    {
-    case IS_HUF:
-        {   size_t litSize, litCSize, singleStream=0;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            if (srcSize < 5) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need up to 5 for lhSize, + cSize (+nbSeq) */
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                /* 2 - 2 - 10 - 10 */
-                lhSize=3;
-                singleStream = istart[0] & 16;
-                litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-                litCSize = ((istart[1] &  3) << 8) + istart[2];
-                break;
-            case 2:
-                /* 2 - 2 - 14 - 14 */
-                lhSize=4;
-                litSize  = ((istart[0] & 15) << 10) + (istart[1] << 2) + (istart[2] >> 6);
-                litCSize = ((istart[2] & 63) <<  8) + istart[3];
-                break;
-            case 3:
-                /* 2 - 2 - 18 - 18 */
-                lhSize=5;
-                litSize  = ((istart[0] & 15) << 14) + (istart[1] << 6) + (istart[2] >> 2);
-                litCSize = ((istart[2] &  3) << 16) + (istart[3] << 8) + istart[4];
-                break;
-            }
-            if (litSize > ZSTDv06_BLOCKSIZE_MAX) return ERROR(corruption_detected);
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            if (HUFv06_isError(singleStream ?
-                            HUFv06_decompress1X2(dctx->litBuffer, litSize, istart+lhSize, litCSize) :
-                            HUFv06_decompress   (dctx->litBuffer, litSize, istart+lhSize, litCSize) ))
-                return ERROR(corruption_detected);
-
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case IS_PCH:
-        {   size_t litSize, litCSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            if (lhSize != 1)  /* only case supported for now : small litSize, single stream */
-                return ERROR(corruption_detected);
-            if (!dctx->flagRepeatTable)
-                return ERROR(dictionary_corrupted);
-
-            /* 2 - 2 - 10 - 10 */
-            lhSize=3;
-            litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-            litCSize = ((istart[1] &  3) << 8) + istart[2];
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            {   size_t const errorCode = HUFv06_decompress1X4_usingDTable(dctx->litBuffer, litSize, istart+lhSize, litCSize, dctx->hufTableX4);
-                if (HUFv06_isError(errorCode)) return ERROR(corruption_detected);
-            }
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case IS_RAW:
-        {   size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize=1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                break;
-            }
-
-            if (lhSize+litSize+WILDCOPY_OVERLENGTH > srcSize) {  /* risk reading beyond src buffer with wildcopy */
-                if (litSize+lhSize > srcSize) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart+lhSize, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-                return lhSize+litSize;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+lhSize;
-            dctx->litSize = litSize;
-            return lhSize+litSize;
-        }
-    case IS_RLE:
-        {   size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize = 1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                if (srcSize<4) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need lhSize+1 = 4 */
-                break;
-            }
-            if (litSize > ZSTDv06_BLOCKSIZE_MAX) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[lhSize], litSize + WILDCOPY_OVERLENGTH);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return lhSize+1;
-        }
-    default:
-        return ERROR(corruption_detected);   /* impossible */
-    }
-}
-
-
-/*! ZSTDv06_buildSeqTable() :
-    @return : nb bytes read from src,
-              or an error code if it fails, testable with ZSTDv06_isError()
-*/
-static size_t ZSTDv06_buildSeqTable(FSEv06_DTable* DTable, U32 type, U32 max, U32 maxLog,
-                                 const void* src, size_t srcSize,
-                                 const S16* defaultNorm, U32 defaultLog, U32 flagRepeatTable)
-{
-    switch(type)
-    {
-    case FSEv06_ENCODING_RLE :
-        if (!srcSize) return ERROR(srcSize_wrong);
-        if ( (*(const BYTE*)src) > max) return ERROR(corruption_detected);
-        FSEv06_buildDTable_rle(DTable, *(const BYTE*)src);   /* if *src > max, data is corrupted */
-        return 1;
-    case FSEv06_ENCODING_RAW :
-        FSEv06_buildDTable(DTable, defaultNorm, max, defaultLog);
-        return 0;
-    case FSEv06_ENCODING_STATIC:
-        if (!flagRepeatTable) return ERROR(corruption_detected);
-        return 0;
-    default :   /* impossible */
-    case FSEv06_ENCODING_DYNAMIC :
-        {   U32 tableLog;
-            S16 norm[MaxSeq+1];
-            size_t const headerSize = FSEv06_readNCount(norm, &max, &tableLog, src, srcSize);
-            if (FSEv06_isError(headerSize)) return ERROR(corruption_detected);
-            if (tableLog > maxLog) return ERROR(corruption_detected);
-            FSEv06_buildDTable(DTable, norm, max, tableLog);
-            return headerSize;
-    }   }
-}
-
-
-static size_t ZSTDv06_decodeSeqHeaders(int* nbSeqPtr,
-                             FSEv06_DTable* DTableLL, FSEv06_DTable* DTableML, FSEv06_DTable* DTableOffb, U32 flagRepeatTable,
-                             const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* ip = istart;
-
-    /* check */
-    if (srcSize < MIN_SEQUENCES_SIZE) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    {   int nbSeq = *ip++;
-        if (!nbSeq) { *nbSeqPtr=0; return 1; }
-        if (nbSeq > 0x7F) {
-            if (nbSeq == 0xFF) {
-                if (ip+2 > iend) return ERROR(srcSize_wrong);
-                nbSeq = MEM_readLE16(ip) + LONGNBSEQ, ip+=2;
-            } else {
-                if (ip >= iend) return ERROR(srcSize_wrong);
-                nbSeq = ((nbSeq-0x80)<<8) + *ip++;
-            }
-        }
-        *nbSeqPtr = nbSeq;
-    }
-
-    /* FSE table descriptors */
-    if (ip + 4 > iend) return ERROR(srcSize_wrong); /* min : header byte + all 3 are "raw", hence no header, but at least xxLog bits per type */
-    {   U32 const LLtype  = *ip >> 6;
-        U32 const Offtype = (*ip >> 4) & 3;
-        U32 const MLtype  = (*ip >> 2) & 3;
-        ip++;
-
-        /* Build DTables */
-        {   size_t const bhSize = ZSTDv06_buildSeqTable(DTableLL, LLtype, MaxLL, LLFSELog, ip, iend-ip, LL_defaultNorm, LL_defaultNormLog, flagRepeatTable);
-            if (ZSTDv06_isError(bhSize)) return ERROR(corruption_detected);
-            ip += bhSize;
-        }
-        {   size_t const bhSize = ZSTDv06_buildSeqTable(DTableOffb, Offtype, MaxOff, OffFSELog, ip, iend-ip, OF_defaultNorm, OF_defaultNormLog, flagRepeatTable);
-            if (ZSTDv06_isError(bhSize)) return ERROR(corruption_detected);
-            ip += bhSize;
-        }
-        {   size_t const bhSize = ZSTDv06_buildSeqTable(DTableML, MLtype, MaxML, MLFSELog, ip, iend-ip, ML_defaultNorm, ML_defaultNormLog, flagRepeatTable);
-            if (ZSTDv06_isError(bhSize)) return ERROR(corruption_detected);
-            ip += bhSize;
-    }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t matchLength;
-    size_t offset;
-} seq_t;
-
-typedef struct {
-    BITv06_DStream_t DStream;
-    FSEv06_DState_t stateLL;
-    FSEv06_DState_t stateOffb;
-    FSEv06_DState_t stateML;
-    size_t prevOffset[ZSTDv06_REP_INIT];
-} seqState_t;
-
-
-
-static void ZSTDv06_decodeSequence(seq_t* seq, seqState_t* seqState)
-{
-    /* Literal length */
-    U32 const llCode = FSEv06_peekSymbol(&(seqState->stateLL));
-    U32 const mlCode = FSEv06_peekSymbol(&(seqState->stateML));
-    U32 const ofCode = FSEv06_peekSymbol(&(seqState->stateOffb));   /* <= maxOff, by table construction */
-
-    U32 const llBits = LL_bits[llCode];
-    U32 const mlBits = ML_bits[mlCode];
-    U32 const ofBits = ofCode;
-    U32 const totalBits = llBits+mlBits+ofBits;
-
-    static const U32 LL_base[MaxLL+1] = {
-                             0,  1,  2,  3,  4,  5,  6,  7,  8,  9,   10,    11,    12,    13,    14,     15,
-                            16, 18, 20, 22, 24, 28, 32, 40, 48, 64, 0x80, 0x100, 0x200, 0x400, 0x800, 0x1000,
-                            0x2000, 0x4000, 0x8000, 0x10000 };
-
-    static const U32 ML_base[MaxML+1] = {
-                             0,  1,  2,  3,  4,  5,  6,  7,  8,  9, 10,   11,    12,    13,    14,    15,
-                            16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26,   27,    28,    29,    30,    31,
-                            32, 34, 36, 38, 40, 44, 48, 56, 64, 80, 96, 0x80, 0x100, 0x200, 0x400, 0x800,
-                            0x1000, 0x2000, 0x4000, 0x8000, 0x10000 };
-
-    static const U32 OF_base[MaxOff+1] = {
-                 0,        1,       3,       7,     0xF,     0x1F,     0x3F,     0x7F,
-                 0xFF,   0x1FF,   0x3FF,   0x7FF,   0xFFF,   0x1FFF,   0x3FFF,   0x7FFF,
-                 0xFFFF, 0x1FFFF, 0x3FFFF, 0x7FFFF, 0xFFFFF, 0x1FFFFF, 0x3FFFFF, 0x7FFFFF,
-                 0xFFFFFF, 0x1FFFFFF, 0x3FFFFFF, /*fake*/ 1, 1 };
-
-    /* sequence */
-    {   size_t offset;
-        if (!ofCode)
-            offset = 0;
-        else {
-            offset = OF_base[ofCode] + BITv06_readBits(&(seqState->DStream), ofBits);   /* <=  26 bits */
-            if (MEM_32bits()) BITv06_reloadDStream(&(seqState->DStream));
-        }
-
-        if (offset < ZSTDv06_REP_NUM) {
-            if (llCode == 0 && offset <= 1) offset = 1-offset;
-
-            if (offset != 0) {
-                size_t temp = seqState->prevOffset[offset];
-                if (offset != 1) {
-                    seqState->prevOffset[2] = seqState->prevOffset[1];
-                }
-                seqState->prevOffset[1] = seqState->prevOffset[0];
-                seqState->prevOffset[0] = offset = temp;
-
-            } else {
-                offset = seqState->prevOffset[0];
-            }
-        } else {
-            offset -= ZSTDv06_REP_MOVE;
-            seqState->prevOffset[2] = seqState->prevOffset[1];
-            seqState->prevOffset[1] = seqState->prevOffset[0];
-            seqState->prevOffset[0] = offset;
-        }
-        seq->offset = offset;
-    }
-
-    seq->matchLength = ML_base[mlCode] + MINMATCH + ((mlCode>31) ? BITv06_readBits(&(seqState->DStream), mlBits) : 0);   /* <=  16 bits */
-    if (MEM_32bits() && (mlBits+llBits>24)) BITv06_reloadDStream(&(seqState->DStream));
-
-    seq->litLength = LL_base[llCode] + ((llCode>15) ? BITv06_readBits(&(seqState->DStream), llBits) : 0);   /* <=  16 bits */
-    if (MEM_32bits() ||
-       (totalBits > 64 - 7 - (LLFSELog+MLFSELog+OffFSELog)) ) BITv06_reloadDStream(&(seqState->DStream));
-
-    /* ANS state update */
-    FSEv06_updateState(&(seqState->stateLL), &(seqState->DStream));   /* <=  9 bits */
-    FSEv06_updateState(&(seqState->stateML), &(seqState->DStream));   /* <=  9 bits */
-    if (MEM_32bits()) BITv06_reloadDStream(&(seqState->DStream));     /* <= 18 bits */
-    FSEv06_updateState(&(seqState->stateOffb), &(seqState->DStream)); /* <=  8 bits */
-}
-
-
-static size_t ZSTDv06_execSequence(BYTE* op,
-                                BYTE* const oend, seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                const BYTE* const base, const BYTE* const vBase, const BYTE* const dictEnd)
-{
-    BYTE* const oLitEnd = op + sequence.litLength;
-    size_t const sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_8 = oend-8;
-    const BYTE* const iLitEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    if (oLitEnd > oend_8) return ERROR(dstSize_tooSmall);   /* last match must start at a minimum distance of 8 from oend */
-    if (oMatchEnd > oend) return ERROR(dstSize_tooSmall);   /* overwrite beyond dst buffer */
-    if (iLitEnd > litLimit) return ERROR(corruption_detected);   /* over-read beyond lit buffer */
-
-    /* copy Literals */
-    ZSTDv06_wildcopy(op, *litPtr, sequence.litLength);   /* note : oLitEnd <= oend-8 : no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = iLitEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - base)) {
-        /* offset beyond prefix */
-        if (sequence.offset > (size_t)(oLitEnd - vBase)) return ERROR(corruption_detected);
-        match = dictEnd - (base-match);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {   size_t const length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = base;
-            if (op > oend_8 || sequence.matchLength < MINMATCH) {
-              while (op < oMatchEnd) *op++ = *match++;
-              return sequenceLength;
-            }
-    }   }
-    /* Requirement: op <= oend_8 */
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        static const U32 dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-        static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-        int const sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTDv06_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTDv06_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH)) {
-        if (op < oend_8) {
-            ZSTDv06_wildcopy(op, match, oend_8 - op);
-            match += oend_8 - op;
-            op = oend_8;
-        }
-        while (op < oMatchEnd) *op++ = *match++;
-    } else {
-        ZSTDv06_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-    }
-    return sequenceLength;
-}
-
-
-static size_t ZSTDv06_decompressSequences(
-                               ZSTDv06_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + maxDstSize;
-    BYTE* op = ostart;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    FSEv06_DTable* DTableLL = dctx->LLTable;
-    FSEv06_DTable* DTableML = dctx->MLTable;
-    FSEv06_DTable* DTableOffb = dctx->OffTable;
-    const BYTE* const base = (const BYTE*) (dctx->base);
-    const BYTE* const vBase = (const BYTE*) (dctx->vBase);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-    int nbSeq;
-
-    /* Build Decoding Tables */
-    {   size_t const seqHSize = ZSTDv06_decodeSeqHeaders(&nbSeq, DTableLL, DTableML, DTableOffb, dctx->flagRepeatTable, ip, seqSize);
-        if (ZSTDv06_isError(seqHSize)) return seqHSize;
-        ip += seqHSize;
-        dctx->flagRepeatTable = 0;
-    }
-
-    /* Regen sequences */
-    if (nbSeq) {
-        seq_t sequence;
-        seqState_t seqState;
-
-        memset(&sequence, 0, sizeof(sequence));
-        sequence.offset = REPCODE_STARTVALUE;
-        { U32 i; for (i=0; i<ZSTDv06_REP_INIT; i++) seqState.prevOffset[i] = REPCODE_STARTVALUE; }
-        { size_t const errorCode = BITv06_initDStream(&(seqState.DStream), ip, iend-ip);
-          if (ERR_isError(errorCode)) return ERROR(corruption_detected); }
-        FSEv06_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSEv06_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSEv06_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BITv06_reloadDStream(&(seqState.DStream)) <= BITv06_DStream_completed) && nbSeq ; ) {
-            nbSeq--;
-            ZSTDv06_decodeSequence(&sequence, &seqState);
-
-#if 0  /* debug */
-            static BYTE* start = NULL;
-            if (start==NULL) start = op;
-            size_t pos = (size_t)(op-start);
-            if ((pos >= 5810037) && (pos < 5810400))
-                printf("Dpos %6u :%5u literals & match %3u bytes at distance %6u \n",
-                       pos, (U32)sequence.litLength, (U32)sequence.matchLength, (U32)sequence.offset);
-#endif
-
-            {   size_t const oneSeqSize = ZSTDv06_execSequence(op, oend, sequence, &litPtr, litEnd, base, vBase, dictEnd);
-                if (ZSTDv06_isError(oneSeqSize)) return oneSeqSize;
-                op += oneSeqSize;
-        }   }
-
-        /* check if reached exact end */
-        if (nbSeq) return ERROR(corruption_detected);
-    }
-
-    /* last literal segment */
-    {   size_t const lastLLSize = litEnd - litPtr;
-        if (litPtr > litEnd) return ERROR(corruption_detected);   /* too many literals already used */
-        if (op+lastLLSize > oend) return ERROR(dstSize_tooSmall);
-        memcpy(op, litPtr, lastLLSize);
-        op += lastLLSize;
-    }
-
-    return op-ostart;
-}
-
-
-static void ZSTDv06_checkContinuity(ZSTDv06_DCtx* dctx, const void* dst)
-{
-    if (dst != dctx->previousDstEnd) {   /* not contiguous */
-        dctx->dictEnd = dctx->previousDstEnd;
-        dctx->vBase = (const char*)dst - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-        dctx->base = dst;
-        dctx->previousDstEnd = dst;
-    }
-}
-
-
-static size_t ZSTDv06_decompressBlock_internal(ZSTDv06_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{   /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-
-    if (srcSize >= ZSTDv06_BLOCKSIZE_MAX) return ERROR(srcSize_wrong);
-
-    /* Decode literals sub-block */
-    {   size_t const litCSize = ZSTDv06_decodeLiteralsBlock(dctx, src, srcSize);
-        if (ZSTDv06_isError(litCSize)) return litCSize;
-        ip += litCSize;
-        srcSize -= litCSize;
-    }
-    return ZSTDv06_decompressSequences(dctx, dst, dstCapacity, ip, srcSize);
-}
-
-
-size_t ZSTDv06_decompressBlock(ZSTDv06_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{
-    ZSTDv06_checkContinuity(dctx, dst);
-    return ZSTDv06_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-/*! ZSTDv06_decompressFrame() :
-*   `dctx` must be properly initialized */
-static size_t ZSTDv06_decompressFrame(ZSTDv06_DCtx* dctx,
-                                 void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* const iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* op = ostart;
-    BYTE* const oend = ostart + dstCapacity;
-    size_t remainingSize = srcSize;
-    blockProperties_t blockProperties = { bt_compressed, 0 };
-
-    /* check */
-    if (srcSize < ZSTDv06_frameHeaderSize_min+ZSTDv06_blockHeaderSize) return ERROR(srcSize_wrong);
-
-    /* Frame Header */
-    {   size_t const frameHeaderSize = ZSTDv06_frameHeaderSize(src, ZSTDv06_frameHeaderSize_min);
-        if (ZSTDv06_isError(frameHeaderSize)) return frameHeaderSize;
-        if (srcSize < frameHeaderSize+ZSTDv06_blockHeaderSize) return ERROR(srcSize_wrong);
-        if (ZSTDv06_decodeFrameHeader(dctx, src, frameHeaderSize)) return ERROR(corruption_detected);
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1) {
-        size_t decodedSize=0;
-        size_t const cBlockSize = ZSTDv06_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTDv06_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTDv06_blockHeaderSize;
-        remainingSize -= ZSTDv06_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTDv06_decompressBlock_internal(dctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTDv06_copyRawBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            return ERROR(GENERIC);   /* not yet supported */
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        if (ZSTDv06_isError(decodedSize)) return decodedSize;
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-
-size_t ZSTDv06_decompress_usingPreparedDCtx(ZSTDv06_DCtx* dctx, const ZSTDv06_DCtx* refDCtx,
-                                         void* dst, size_t dstCapacity,
-                                   const void* src, size_t srcSize)
-{
-    ZSTDv06_copyDCtx(dctx, refDCtx);
-    ZSTDv06_checkContinuity(dctx, dst);
-    return ZSTDv06_decompressFrame(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-size_t ZSTDv06_decompress_usingDict(ZSTDv06_DCtx* dctx,
-                                 void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize,
-                                 const void* dict, size_t dictSize)
-{
-    ZSTDv06_decompressBegin_usingDict(dctx, dict, dictSize);
-    ZSTDv06_checkContinuity(dctx, dst);
-    return ZSTDv06_decompressFrame(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-size_t ZSTDv06_decompressDCtx(ZSTDv06_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    return ZSTDv06_decompress_usingDict(dctx, dst, dstCapacity, src, srcSize, NULL, 0);
-}
-
-
-size_t ZSTDv06_decompress(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-#if defined(ZSTDv06_HEAPMODE) && (ZSTDv06_HEAPMODE==1)
-    size_t regenSize;
-    ZSTDv06_DCtx* dctx = ZSTDv06_createDCtx();
-    if (dctx==NULL) return ERROR(memory_allocation);
-    regenSize = ZSTDv06_decompressDCtx(dctx, dst, dstCapacity, src, srcSize);
-    ZSTDv06_freeDCtx(dctx);
-    return regenSize;
-#else   /* stack mode */
-    ZSTDv06_DCtx dctx;
-    return ZSTDv06_decompressDCtx(&dctx, dst, dstCapacity, src, srcSize);
-#endif
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv06_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-    blockProperties_t blockProperties = { bt_compressed, 0 };
-
-    /* Frame Header */
-    {   size_t const frameHeaderSize = ZSTDv06_frameHeaderSize(src, srcSize);
-        if (ZSTDv06_isError(frameHeaderSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, frameHeaderSize);
-            return;
-        }
-        if (MEM_readLE32(src) != ZSTDv06_MAGICNUMBER) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-            return;
-        }
-        if (srcSize < frameHeaderSize+ZSTDv06_blockHeaderSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1) {
-        size_t const cBlockSize = ZSTDv06_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTDv06_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTDv06_blockHeaderSize;
-        remainingSize -= ZSTDv06_blockHeaderSize;
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        if (cBlockSize == 0) break;   /* bt_end */
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * ZSTDv06_BLOCKSIZE_MAX;
-}
-
-/*_******************************
-*  Streaming Decompression API
-********************************/
-size_t ZSTDv06_nextSrcSizeToDecompress(ZSTDv06_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-size_t ZSTDv06_decompressContinue(ZSTDv06_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != dctx->expected) return ERROR(srcSize_wrong);
-    if (dstCapacity) ZSTDv06_checkContinuity(dctx, dst);
-
-    /* Decompress : frame header; part 1 */
-    switch (dctx->stage)
-    {
-    case ZSTDds_getFrameHeaderSize :
-        if (srcSize != ZSTDv06_frameHeaderSize_min) return ERROR(srcSize_wrong);   /* impossible */
-        dctx->headerSize = ZSTDv06_frameHeaderSize(src, ZSTDv06_frameHeaderSize_min);
-        if (ZSTDv06_isError(dctx->headerSize)) return dctx->headerSize;
-        memcpy(dctx->headerBuffer, src, ZSTDv06_frameHeaderSize_min);
-        if (dctx->headerSize > ZSTDv06_frameHeaderSize_min) {
-            dctx->expected = dctx->headerSize - ZSTDv06_frameHeaderSize_min;
-            dctx->stage = ZSTDds_decodeFrameHeader;
-            return 0;
-        }
-        dctx->expected = 0;   /* not necessary to copy more */
-	/* fall-through */
-    case ZSTDds_decodeFrameHeader:
-        {   size_t result;
-            memcpy(dctx->headerBuffer + ZSTDv06_frameHeaderSize_min, src, dctx->expected);
-            result = ZSTDv06_decodeFrameHeader(dctx, dctx->headerBuffer, dctx->headerSize);
-            if (ZSTDv06_isError(result)) return result;
-            dctx->expected = ZSTDv06_blockHeaderSize;
-            dctx->stage = ZSTDds_decodeBlockHeader;
-            return 0;
-        }
-    case ZSTDds_decodeBlockHeader:
-        {   blockProperties_t bp;
-            size_t const cBlockSize = ZSTDv06_getcBlockSize(src, ZSTDv06_blockHeaderSize, &bp);
-            if (ZSTDv06_isError(cBlockSize)) return cBlockSize;
-            if (bp.blockType == bt_end) {
-                dctx->expected = 0;
-                dctx->stage = ZSTDds_getFrameHeaderSize;
-            } else {
-                dctx->expected = cBlockSize;
-                dctx->bType = bp.blockType;
-                dctx->stage = ZSTDds_decompressBlock;
-            }
-            return 0;
-        }
-    case ZSTDds_decompressBlock:
-        {   size_t rSize;
-            switch(dctx->bType)
-            {
-            case bt_compressed:
-                rSize = ZSTDv06_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize);
-                break;
-            case bt_raw :
-                rSize = ZSTDv06_copyRawBlock(dst, dstCapacity, src, srcSize);
-                break;
-            case bt_rle :
-                return ERROR(GENERIC);   /* not yet handled */
-                break;
-            case bt_end :   /* should never happen (filtered at phase 1) */
-                rSize = 0;
-                break;
-            default:
-                return ERROR(GENERIC);   /* impossible */
-            }
-            dctx->stage = ZSTDds_decodeBlockHeader;
-            dctx->expected = ZSTDv06_blockHeaderSize;
-            dctx->previousDstEnd = (char*)dst + rSize;
-            return rSize;
-        }
-    default:
-        return ERROR(GENERIC);   /* impossible */
-    }
-}
-
-
-static void ZSTDv06_refDictContent(ZSTDv06_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    dctx->dictEnd = dctx->previousDstEnd;
-    dctx->vBase = (const char*)dict - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-    dctx->base = dict;
-    dctx->previousDstEnd = (const char*)dict + dictSize;
-}
-
-static size_t ZSTDv06_loadEntropy(ZSTDv06_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    size_t hSize, offcodeHeaderSize, matchlengthHeaderSize, litlengthHeaderSize;
-
-    hSize = HUFv06_readDTableX4(dctx->hufTableX4, dict, dictSize);
-    if (HUFv06_isError(hSize)) return ERROR(dictionary_corrupted);
-    dict = (const char*)dict + hSize;
-    dictSize -= hSize;
-
-    {   short offcodeNCount[MaxOff+1];
-        U32 offcodeMaxValue=MaxOff, offcodeLog;
-        offcodeHeaderSize = FSEv06_readNCount(offcodeNCount, &offcodeMaxValue, &offcodeLog, dict, dictSize);
-        if (FSEv06_isError(offcodeHeaderSize)) return ERROR(dictionary_corrupted);
-        if (offcodeLog > OffFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv06_buildDTable(dctx->OffTable, offcodeNCount, offcodeMaxValue, offcodeLog);
-          if (FSEv06_isError(errorCode)) return ERROR(dictionary_corrupted); }
-        dict = (const char*)dict + offcodeHeaderSize;
-        dictSize -= offcodeHeaderSize;
-    }
-
-    {   short matchlengthNCount[MaxML+1];
-        unsigned matchlengthMaxValue = MaxML, matchlengthLog;
-        matchlengthHeaderSize = FSEv06_readNCount(matchlengthNCount, &matchlengthMaxValue, &matchlengthLog, dict, dictSize);
-        if (FSEv06_isError(matchlengthHeaderSize)) return ERROR(dictionary_corrupted);
-        if (matchlengthLog > MLFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv06_buildDTable(dctx->MLTable, matchlengthNCount, matchlengthMaxValue, matchlengthLog);
-          if (FSEv06_isError(errorCode)) return ERROR(dictionary_corrupted); }
-        dict = (const char*)dict + matchlengthHeaderSize;
-        dictSize -= matchlengthHeaderSize;
-    }
-
-    {   short litlengthNCount[MaxLL+1];
-        unsigned litlengthMaxValue = MaxLL, litlengthLog;
-        litlengthHeaderSize = FSEv06_readNCount(litlengthNCount, &litlengthMaxValue, &litlengthLog, dict, dictSize);
-        if (FSEv06_isError(litlengthHeaderSize)) return ERROR(dictionary_corrupted);
-        if (litlengthLog > LLFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv06_buildDTable(dctx->LLTable, litlengthNCount, litlengthMaxValue, litlengthLog);
-          if (FSEv06_isError(errorCode)) return ERROR(dictionary_corrupted); }
-    }
-
-    dctx->flagRepeatTable = 1;
-    return hSize + offcodeHeaderSize + matchlengthHeaderSize + litlengthHeaderSize;
-}
-
-static size_t ZSTDv06_decompress_insertDictionary(ZSTDv06_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    size_t eSize;
-    U32 const magic = MEM_readLE32(dict);
-    if (magic != ZSTDv06_DICT_MAGIC) {
-        /* pure content mode */
-        ZSTDv06_refDictContent(dctx, dict, dictSize);
-        return 0;
-    }
-    /* load entropy tables */
-    dict = (const char*)dict + 4;
-    dictSize -= 4;
-    eSize = ZSTDv06_loadEntropy(dctx, dict, dictSize);
-    if (ZSTDv06_isError(eSize)) return ERROR(dictionary_corrupted);
-
-    /* reference dictionary content */
-    dict = (const char*)dict + eSize;
-    dictSize -= eSize;
-    ZSTDv06_refDictContent(dctx, dict, dictSize);
-
-    return 0;
-}
-
-
-size_t ZSTDv06_decompressBegin_usingDict(ZSTDv06_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    { size_t const errorCode = ZSTDv06_decompressBegin(dctx);
-      if (ZSTDv06_isError(errorCode)) return errorCode; }
-
-    if (dict && dictSize) {
-        size_t const errorCode = ZSTDv06_decompress_insertDictionary(dctx, dict, dictSize);
-        if (ZSTDv06_isError(errorCode)) return ERROR(dictionary_corrupted);
-    }
-
-    return 0;
-}
-
-/*
-    Buffered version of Zstd compression library
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net/
-*/
-
-
-/*-***************************************************************************
-*  Streaming decompression howto
-*
-*  A ZBUFFv06_DCtx object is required to track streaming operations.
-*  Use ZBUFFv06_createDCtx() and ZBUFFv06_freeDCtx() to create/release resources.
-*  Use ZBUFFv06_decompressInit() to start a new decompression operation,
-*   or ZBUFFv06_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFFv06_DCtx objects can be re-init multiple times.
-*
-*  Use ZBUFFv06_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of @dst will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters, or change @dst.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to help latency),
-*            or 0 when a frame is completely decoded,
-*            or an error code, which can be tested using ZBUFFv06_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFFv06_recommendedDInSize() and ZBUFFv06_recommendedDOutSize()
-*  output : ZBUFFv06_recommendedDOutSize==128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFFv06_recommendedDInSize == 128KB + 3;
-*           just follow indications from ZBUFFv06_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-typedef enum { ZBUFFds_init, ZBUFFds_loadHeader,
-               ZBUFFds_read, ZBUFFds_load, ZBUFFds_flush } ZBUFFv06_dStage;
-
-/* *** Resource management *** */
-struct ZBUFFv06_DCtx_s {
-    ZSTDv06_DCtx* zd;
-    ZSTDv06_frameParams fParams;
-    ZBUFFv06_dStage stage;
-    char*  inBuff;
-    size_t inBuffSize;
-    size_t inPos;
-    char*  outBuff;
-    size_t outBuffSize;
-    size_t outStart;
-    size_t outEnd;
-    size_t blockSize;
-    BYTE headerBuffer[ZSTDv06_FRAMEHEADERSIZE_MAX];
-    size_t lhSize;
-};   /* typedef'd to ZBUFFv06_DCtx within "zstd_buffered.h" */
-
-
-ZBUFFv06_DCtx* ZBUFFv06_createDCtx(void)
-{
-    ZBUFFv06_DCtx* zbd = (ZBUFFv06_DCtx*)malloc(sizeof(ZBUFFv06_DCtx));
-    if (zbd==NULL) return NULL;
-    memset(zbd, 0, sizeof(*zbd));
-    zbd->zd = ZSTDv06_createDCtx();
-    zbd->stage = ZBUFFds_init;
-    return zbd;
-}
-
-size_t ZBUFFv06_freeDCtx(ZBUFFv06_DCtx* zbd)
-{
-    if (zbd==NULL) return 0;   /* support free on null */
-    ZSTDv06_freeDCtx(zbd->zd);
-    free(zbd->inBuff);
-    free(zbd->outBuff);
-    free(zbd);
-    return 0;
-}
-
-
-/* *** Initialization *** */
-
-size_t ZBUFFv06_decompressInitDictionary(ZBUFFv06_DCtx* zbd, const void* dict, size_t dictSize)
-{
-    zbd->stage = ZBUFFds_loadHeader;
-    zbd->lhSize = zbd->inPos = zbd->outStart = zbd->outEnd = 0;
-    return ZSTDv06_decompressBegin_usingDict(zbd->zd, dict, dictSize);
-}
-
-size_t ZBUFFv06_decompressInit(ZBUFFv06_DCtx* zbd)
-{
-    return ZBUFFv06_decompressInitDictionary(zbd, NULL, 0);
-}
-
-
-
-MEM_STATIC size_t ZBUFFv06_limitCopy(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    size_t length = MIN(dstCapacity, srcSize);
-    memcpy(dst, src, length);
-    return length;
-}
-
-
-/* *** Decompression *** */
-
-size_t ZBUFFv06_decompressContinue(ZBUFFv06_DCtx* zbd,
-                                void* dst, size_t* dstCapacityPtr,
-                          const void* src, size_t* srcSizePtr)
-{
-    const char* const istart = (const char*)src;
-    const char* const iend = istart + *srcSizePtr;
-    const char* ip = istart;
-    char* const ostart = (char*)dst;
-    char* const oend = ostart + *dstCapacityPtr;
-    char* op = ostart;
-    U32 notDone = 1;
-
-    while (notDone) {
-        switch(zbd->stage)
-        {
-        case ZBUFFds_init :
-            return ERROR(init_missing);
-
-        case ZBUFFds_loadHeader :
-            {   size_t const hSize = ZSTDv06_getFrameParams(&(zbd->fParams), zbd->headerBuffer, zbd->lhSize);
-                if (hSize != 0) {
-                    size_t const toLoad = hSize - zbd->lhSize;   /* if hSize!=0, hSize > zbd->lhSize */
-                    if (ZSTDv06_isError(hSize)) return hSize;
-                    if (toLoad > (size_t)(iend-ip)) {   /* not enough input to load full header */
-                        memcpy(zbd->headerBuffer + zbd->lhSize, ip, iend-ip);
-                        zbd->lhSize += iend-ip;
-                        *dstCapacityPtr = 0;
-                        return (hSize - zbd->lhSize) + ZSTDv06_blockHeaderSize;   /* remaining header bytes + next block header */
-                    }
-                    memcpy(zbd->headerBuffer + zbd->lhSize, ip, toLoad); zbd->lhSize = hSize; ip += toLoad;
-                    break;
-            }   }
-
-            /* Consume header */
-            {   size_t const h1Size = ZSTDv06_nextSrcSizeToDecompress(zbd->zd);  /* == ZSTDv06_frameHeaderSize_min */
-                size_t const h1Result = ZSTDv06_decompressContinue(zbd->zd, NULL, 0, zbd->headerBuffer, h1Size);
-                if (ZSTDv06_isError(h1Result)) return h1Result;
-                if (h1Size < zbd->lhSize) {   /* long header */
-                    size_t const h2Size = ZSTDv06_nextSrcSizeToDecompress(zbd->zd);
-                    size_t const h2Result = ZSTDv06_decompressContinue(zbd->zd, NULL, 0, zbd->headerBuffer+h1Size, h2Size);
-                    if (ZSTDv06_isError(h2Result)) return h2Result;
-            }   }
-
-            /* Frame header instruct buffer sizes */
-            {   size_t const blockSize = MIN(1 << zbd->fParams.windowLog, ZSTDv06_BLOCKSIZE_MAX);
-                zbd->blockSize = blockSize;
-                if (zbd->inBuffSize < blockSize) {
-                    free(zbd->inBuff);
-                    zbd->inBuffSize = blockSize;
-                    zbd->inBuff = (char*)malloc(blockSize);
-                    if (zbd->inBuff == NULL) return ERROR(memory_allocation);
-                }
-                {   size_t const neededOutSize = ((size_t)1 << zbd->fParams.windowLog) + blockSize + WILDCOPY_OVERLENGTH * 2;
-                    if (zbd->outBuffSize < neededOutSize) {
-                        free(zbd->outBuff);
-                        zbd->outBuffSize = neededOutSize;
-                        zbd->outBuff = (char*)malloc(neededOutSize);
-                        if (zbd->outBuff == NULL) return ERROR(memory_allocation);
-            }   }   }
-            zbd->stage = ZBUFFds_read;
-	    /* fall-through */
-        case ZBUFFds_read:
-            {   size_t const neededInSize = ZSTDv06_nextSrcSizeToDecompress(zbd->zd);
-                if (neededInSize==0) {  /* end of frame */
-                    zbd->stage = ZBUFFds_init;
-                    notDone = 0;
-                    break;
-                }
-                if ((size_t)(iend-ip) >= neededInSize) {  /* decode directly from src */
-                    size_t const decodedSize = ZSTDv06_decompressContinue(zbd->zd,
-                        zbd->outBuff + zbd->outStart, zbd->outBuffSize - zbd->outStart,
-                        ip, neededInSize);
-                    if (ZSTDv06_isError(decodedSize)) return decodedSize;
-                    ip += neededInSize;
-                    if (!decodedSize) break;   /* this was just a header */
-                    zbd->outEnd = zbd->outStart +  decodedSize;
-                    zbd->stage = ZBUFFds_flush;
-                    break;
-                }
-                if (ip==iend) { notDone = 0; break; }   /* no more input */
-                zbd->stage = ZBUFFds_load;
-            }
-	    /* fall-through */
-        case ZBUFFds_load:
-            {   size_t const neededInSize = ZSTDv06_nextSrcSizeToDecompress(zbd->zd);
-                size_t const toLoad = neededInSize - zbd->inPos;   /* should always be <= remaining space within inBuff */
-                size_t loadedSize;
-                if (toLoad > zbd->inBuffSize - zbd->inPos) return ERROR(corruption_detected);   /* should never happen */
-                loadedSize = ZBUFFv06_limitCopy(zbd->inBuff + zbd->inPos, toLoad, ip, iend-ip);
-                ip += loadedSize;
-                zbd->inPos += loadedSize;
-                if (loadedSize < toLoad) { notDone = 0; break; }   /* not enough input, wait for more */
-
-                /* decode loaded input */
-                {   size_t const decodedSize = ZSTDv06_decompressContinue(zbd->zd,
-                        zbd->outBuff + zbd->outStart, zbd->outBuffSize - zbd->outStart,
-                        zbd->inBuff, neededInSize);
-                    if (ZSTDv06_isError(decodedSize)) return decodedSize;
-                    zbd->inPos = 0;   /* input is consumed */
-                    if (!decodedSize) { zbd->stage = ZBUFFds_read; break; }   /* this was just a header */
-                    zbd->outEnd = zbd->outStart +  decodedSize;
-                    zbd->stage = ZBUFFds_flush;
-                    // break; /* ZBUFFds_flush follows */
-                }
-	    }
-	    /* fall-through */
-        case ZBUFFds_flush:
-            {   size_t const toFlushSize = zbd->outEnd - zbd->outStart;
-                size_t const flushedSize = ZBUFFv06_limitCopy(op, oend-op, zbd->outBuff + zbd->outStart, toFlushSize);
-                op += flushedSize;
-                zbd->outStart += flushedSize;
-                if (flushedSize == toFlushSize) {
-                    zbd->stage = ZBUFFds_read;
-                    if (zbd->outStart + zbd->blockSize > zbd->outBuffSize)
-                        zbd->outStart = zbd->outEnd = 0;
-                    break;
-                }
-                /* cannot flush everything */
-                notDone = 0;
-                break;
-            }
-        default: return ERROR(GENERIC);   /* impossible */
-    }   }
-
-    /* result */
-    *srcSizePtr = ip-istart;
-    *dstCapacityPtr = op-ostart;
-    {   size_t nextSrcSizeHint = ZSTDv06_nextSrcSizeToDecompress(zbd->zd);
-        if (nextSrcSizeHint > ZSTDv06_blockHeaderSize) nextSrcSizeHint+= ZSTDv06_blockHeaderSize;   /* get following block header too */
-        nextSrcSizeHint -= zbd->inPos;   /* already loaded*/
-        return nextSrcSizeHint;
-    }
-}
-
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-size_t ZBUFFv06_recommendedDInSize(void)  { return ZSTDv06_BLOCKSIZE_MAX + ZSTDv06_blockHeaderSize /* block header size*/ ; }
-size_t ZBUFFv06_recommendedDOutSize(void) { return ZSTDv06_BLOCKSIZE_MAX; }
diff --git a/vendor/github.com/DataDog/zstd/zstd_v06.h b/vendor/github.com/DataDog/zstd/zstd_v06.h
deleted file mode 100644
index 0781857..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v06.h
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTDv06_H
-#define ZSTDv06_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*======  Dependency  ======*/
-#include <stddef.h>   /* size_t */
-
-
-/*======  Export for Windows  ======*/
-/*!
-*  ZSTDv06_DLL_EXPORT :
-*  Enable exporting of functions when building a Windows DLL
-*/
-#if defined(_WIN32) && defined(ZSTDv06_DLL_EXPORT) && (ZSTDv06_DLL_EXPORT==1)
-#  define ZSTDLIBv06_API __declspec(dllexport)
-#else
-#  define ZSTDLIBv06_API
-#endif
-
-
-/* *************************************
-*  Simple functions
-***************************************/
-/*! ZSTDv06_decompress() :
-    `compressedSize` : is the _exact_ size of the compressed blob, otherwise decompression will fail.
-    `dstCapacity` must be large enough, equal or larger than originalSize.
-    @return : the number of bytes decompressed into `dst` (<= `dstCapacity`),
-              or an errorCode if it fails (which can be tested using ZSTDv06_isError()) */
-ZSTDLIBv06_API size_t ZSTDv06_decompress( void* dst, size_t dstCapacity,
-                                    const void* src, size_t compressedSize);
-
-/**
-ZSTDv06_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.6.x format
-    srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-    cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                or an error code if it fails (which can be tested using ZSTDv01_isError())
-    dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
-*/
-void ZSTDv06_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                     size_t* cSize, unsigned long long* dBound);
-
-/* *************************************
-*  Helper functions
-***************************************/
-ZSTDLIBv06_API size_t      ZSTDv06_compressBound(size_t srcSize); /*!< maximum compressed size (worst case scenario) */
-
-/* Error Management */
-ZSTDLIBv06_API unsigned    ZSTDv06_isError(size_t code);          /*!< tells if a `size_t` function result is an error code */
-ZSTDLIBv06_API const char* ZSTDv06_getErrorName(size_t code);     /*!< provides readable string for an error code */
-
-
-/* *************************************
-*  Explicit memory management
-***************************************/
-/** Decompression context */
-typedef struct ZSTDv06_DCtx_s ZSTDv06_DCtx;
-ZSTDLIBv06_API ZSTDv06_DCtx* ZSTDv06_createDCtx(void);
-ZSTDLIBv06_API size_t     ZSTDv06_freeDCtx(ZSTDv06_DCtx* dctx);      /*!< @return : errorCode */
-
-/** ZSTDv06_decompressDCtx() :
-*   Same as ZSTDv06_decompress(), but requires an already allocated ZSTDv06_DCtx (see ZSTDv06_createDCtx()) */
-ZSTDLIBv06_API size_t ZSTDv06_decompressDCtx(ZSTDv06_DCtx* ctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-/*-***********************
-*  Dictionary API
-*************************/
-/*! ZSTDv06_decompress_usingDict() :
-*   Decompression using a pre-defined Dictionary content (see dictBuilder).
-*   Dictionary must be identical to the one used during compression, otherwise regenerated data will be corrupted.
-*   Note : dict can be NULL, in which case, it's equivalent to ZSTDv06_decompressDCtx() */
-ZSTDLIBv06_API size_t ZSTDv06_decompress_usingDict(ZSTDv06_DCtx* dctx,
-                                                   void* dst, size_t dstCapacity,
-                                             const void* src, size_t srcSize,
-                                             const void* dict,size_t dictSize);
-
-
-/*-************************
-*  Advanced Streaming API
-***************************/
-struct ZSTDv06_frameParams_s { unsigned long long frameContentSize; unsigned windowLog; };
-typedef struct ZSTDv06_frameParams_s ZSTDv06_frameParams;
-
-ZSTDLIBv06_API size_t ZSTDv06_getFrameParams(ZSTDv06_frameParams* fparamsPtr, const void* src, size_t srcSize);   /**< doesn't consume input */
-ZSTDLIBv06_API size_t ZSTDv06_decompressBegin_usingDict(ZSTDv06_DCtx* dctx, const void* dict, size_t dictSize);
-ZSTDLIBv06_API void   ZSTDv06_copyDCtx(ZSTDv06_DCtx* dctx, const ZSTDv06_DCtx* preparedDCtx);
-
-ZSTDLIBv06_API size_t ZSTDv06_nextSrcSizeToDecompress(ZSTDv06_DCtx* dctx);
-ZSTDLIBv06_API size_t ZSTDv06_decompressContinue(ZSTDv06_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-
-/* *************************************
-*  ZBUFF API
-***************************************/
-
-typedef struct ZBUFFv06_DCtx_s ZBUFFv06_DCtx;
-ZSTDLIBv06_API ZBUFFv06_DCtx* ZBUFFv06_createDCtx(void);
-ZSTDLIBv06_API size_t         ZBUFFv06_freeDCtx(ZBUFFv06_DCtx* dctx);
-
-ZSTDLIBv06_API size_t ZBUFFv06_decompressInit(ZBUFFv06_DCtx* dctx);
-ZSTDLIBv06_API size_t ZBUFFv06_decompressInitDictionary(ZBUFFv06_DCtx* dctx, const void* dict, size_t dictSize);
-
-ZSTDLIBv06_API size_t ZBUFFv06_decompressContinue(ZBUFFv06_DCtx* dctx,
-                                                  void* dst, size_t* dstCapacityPtr,
-                                            const void* src, size_t* srcSizePtr);
-
-/*-***************************************************************************
-*  Streaming decompression howto
-*
-*  A ZBUFFv06_DCtx object is required to track streaming operations.
-*  Use ZBUFFv06_createDCtx() and ZBUFFv06_freeDCtx() to create/release resources.
-*  Use ZBUFFv06_decompressInit() to start a new decompression operation,
-*   or ZBUFFv06_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFFv06_DCtx objects can be re-init multiple times.
-*
-*  Use ZBUFFv06_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of `dst` will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters, or change `dst`.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to help latency),
-*            or 0 when a frame is completely decoded,
-*            or an error code, which can be tested using ZBUFFv06_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFFv06_recommendedDInSize() and ZBUFFv06_recommendedDOutSize()
-*  output : ZBUFFv06_recommendedDOutSize== 128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFFv06_recommendedDInSize == 128KB + 3;
-*           just follow indications from ZBUFFv06_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-ZSTDLIBv06_API unsigned ZBUFFv06_isError(size_t errorCode);
-ZSTDLIBv06_API const char* ZBUFFv06_getErrorName(size_t errorCode);
-
-/** Functions below provide recommended buffer sizes for Compression or Decompression operations.
-*   These sizes are just hints, they tend to offer better latency */
-ZSTDLIBv06_API size_t ZBUFFv06_recommendedDInSize(void);
-ZSTDLIBv06_API size_t ZBUFFv06_recommendedDOutSize(void);
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define ZSTDv06_MAGICNUMBER 0xFD2FB526   /* v0.6 */
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* ZSTDv06_BUFFERED_H */
diff --git a/vendor/github.com/DataDog/zstd/zstd_v07.c b/vendor/github.com/DataDog/zstd/zstd_v07.c
deleted file mode 100644
index a83ddc9..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v07.c
+++ /dev/null
@@ -1,4533 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/*- Dependencies -*/
-#include <stddef.h>     /* size_t, ptrdiff_t */
-#include <string.h>     /* memcpy */
-#include <stdlib.h>     /* malloc, free, qsort */
-
-#ifndef XXH_STATIC_LINKING_ONLY
-#  define XXH_STATIC_LINKING_ONLY    /* XXH64_state_t */
-#endif
-#include "xxhash.h"                  /* XXH64_* */
-#include "zstd_v07.h"
-
-#define FSEv07_STATIC_LINKING_ONLY   /* FSEv07_MIN_TABLELOG */
-#define HUFv07_STATIC_LINKING_ONLY   /* HUFv07_TABLELOG_ABSOLUTEMAX */
-#define ZSTDv07_STATIC_LINKING_ONLY
-
-#include "error_private.h"
-
-
-#ifdef ZSTDv07_STATIC_LINKING_ONLY
-
-/* ====================================================================================
- * The definitions in this section are considered experimental.
- * They should never be used with a dynamic library, as they may change in the future.
- * They are provided for advanced usages.
- * Use them only in association with static linking.
- * ==================================================================================== */
-
-/*--- Constants ---*/
-#define ZSTDv07_MAGIC_SKIPPABLE_START  0x184D2A50U
-
-#define ZSTDv07_WINDOWLOG_MAX_32  25
-#define ZSTDv07_WINDOWLOG_MAX_64  27
-#define ZSTDv07_WINDOWLOG_MAX    ((U32)(MEM_32bits() ? ZSTDv07_WINDOWLOG_MAX_32 : ZSTDv07_WINDOWLOG_MAX_64))
-#define ZSTDv07_WINDOWLOG_MIN     18
-#define ZSTDv07_CHAINLOG_MAX     (ZSTDv07_WINDOWLOG_MAX+1)
-#define ZSTDv07_CHAINLOG_MIN       4
-#define ZSTDv07_HASHLOG_MAX       ZSTDv07_WINDOWLOG_MAX
-#define ZSTDv07_HASHLOG_MIN       12
-#define ZSTDv07_HASHLOG3_MAX      17
-#define ZSTDv07_SEARCHLOG_MAX    (ZSTDv07_WINDOWLOG_MAX-1)
-#define ZSTDv07_SEARCHLOG_MIN      1
-#define ZSTDv07_SEARCHLENGTH_MAX   7
-#define ZSTDv07_SEARCHLENGTH_MIN   3
-#define ZSTDv07_TARGETLENGTH_MIN   4
-#define ZSTDv07_TARGETLENGTH_MAX 999
-
-#define ZSTDv07_FRAMEHEADERSIZE_MAX 18    /* for static allocation */
-static const size_t ZSTDv07_frameHeaderSize_min = 5;
-static const size_t ZSTDv07_frameHeaderSize_max = ZSTDv07_FRAMEHEADERSIZE_MAX;
-static const size_t ZSTDv07_skippableHeaderSize = 8;  /* magic number + skippable frame length */
-
-
-/* custom memory allocation functions */
-typedef void* (*ZSTDv07_allocFunction) (void* opaque, size_t size);
-typedef void  (*ZSTDv07_freeFunction) (void* opaque, void* address);
-typedef struct { ZSTDv07_allocFunction customAlloc; ZSTDv07_freeFunction customFree; void* opaque; } ZSTDv07_customMem;
-
-
-/*--- Advanced Decompression functions ---*/
-
-/*! ZSTDv07_estimateDCtxSize() :
- *  Gives the potential amount of memory allocated to create a ZSTDv07_DCtx */
-ZSTDLIBv07_API size_t ZSTDv07_estimateDCtxSize(void);
-
-/*! ZSTDv07_createDCtx_advanced() :
- *  Create a ZSTD decompression context using external alloc and free functions */
-ZSTDLIBv07_API ZSTDv07_DCtx* ZSTDv07_createDCtx_advanced(ZSTDv07_customMem customMem);
-
-/*! ZSTDv07_sizeofDCtx() :
- *  Gives the amount of memory used by a given ZSTDv07_DCtx */
-ZSTDLIBv07_API size_t ZSTDv07_sizeofDCtx(const ZSTDv07_DCtx* dctx);
-
-
-/* ******************************************************************
-*  Buffer-less streaming functions (synchronous mode)
-********************************************************************/
-
-ZSTDLIBv07_API size_t ZSTDv07_decompressBegin(ZSTDv07_DCtx* dctx);
-ZSTDLIBv07_API size_t ZSTDv07_decompressBegin_usingDict(ZSTDv07_DCtx* dctx, const void* dict, size_t dictSize);
-ZSTDLIBv07_API void   ZSTDv07_copyDCtx(ZSTDv07_DCtx* dctx, const ZSTDv07_DCtx* preparedDCtx);
-
-ZSTDLIBv07_API size_t ZSTDv07_nextSrcSizeToDecompress(ZSTDv07_DCtx* dctx);
-ZSTDLIBv07_API size_t ZSTDv07_decompressContinue(ZSTDv07_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-/*
-  Buffer-less streaming decompression (synchronous mode)
-
-  A ZSTDv07_DCtx object is required to track streaming operations.
-  Use ZSTDv07_createDCtx() / ZSTDv07_freeDCtx() to manage it.
-  A ZSTDv07_DCtx object can be re-used multiple times.
-
-  First optional operation is to retrieve frame parameters, using ZSTDv07_getFrameParams(), which doesn't consume the input.
-  It can provide the minimum size of rolling buffer required to properly decompress data (`windowSize`),
-  and optionally the final size of uncompressed content.
-  (Note : content size is an optional info that may not be present. 0 means : content size unknown)
-  Frame parameters are extracted from the beginning of compressed frame.
-  The amount of data to read is variable, from ZSTDv07_frameHeaderSize_min to ZSTDv07_frameHeaderSize_max (so if `srcSize` >= ZSTDv07_frameHeaderSize_max, it will always work)
-  If `srcSize` is too small for operation to succeed, function will return the minimum size it requires to produce a result.
-  Result : 0 when successful, it means the ZSTDv07_frameParams structure has been filled.
-          >0 : means there is not enough data into `src`. Provides the expected size to successfully decode header.
-           errorCode, which can be tested using ZSTDv07_isError()
-
-  Start decompression, with ZSTDv07_decompressBegin() or ZSTDv07_decompressBegin_usingDict().
-  Alternatively, you can copy a prepared context, using ZSTDv07_copyDCtx().
-
-  Then use ZSTDv07_nextSrcSizeToDecompress() and ZSTDv07_decompressContinue() alternatively.
-  ZSTDv07_nextSrcSizeToDecompress() tells how much bytes to provide as 'srcSize' to ZSTDv07_decompressContinue().
-  ZSTDv07_decompressContinue() requires this exact amount of bytes, or it will fail.
-
-  @result of ZSTDv07_decompressContinue() is the number of bytes regenerated within 'dst' (necessarily <= dstCapacity).
-  It can be zero, which is not an error; it just means ZSTDv07_decompressContinue() has decoded some header.
-
-  ZSTDv07_decompressContinue() needs previous data blocks during decompression, up to `windowSize`.
-  They should preferably be located contiguously, prior to current block.
-  Alternatively, a round buffer of sufficient size is also possible. Sufficient size is determined by frame parameters.
-  ZSTDv07_decompressContinue() is very sensitive to contiguity,
-  if 2 blocks don't follow each other, make sure that either the compressor breaks contiguity at the same place,
-    or that previous contiguous segment is large enough to properly handle maximum back-reference.
-
-  A frame is fully decoded when ZSTDv07_nextSrcSizeToDecompress() returns zero.
-  Context can then be reset to start a new decompression.
-
-
-  == Special case : skippable frames ==
-
-  Skippable frames allow the integration of user-defined data into a flow of concatenated frames.
-  Skippable frames will be ignored (skipped) by a decompressor. The format of skippable frame is following:
-  a) Skippable frame ID - 4 Bytes, Little endian format, any value from 0x184D2A50 to 0x184D2A5F
-  b) Frame Size - 4 Bytes, Little endian format, unsigned 32-bits
-  c) Frame Content - any content (User Data) of length equal to Frame Size
-  For skippable frames ZSTDv07_decompressContinue() always returns 0.
-  For skippable frames ZSTDv07_getFrameParams() returns fparamsPtr->windowLog==0 what means that a frame is skippable.
-  It also returns Frame Size as fparamsPtr->frameContentSize.
-*/
-
-
-/* **************************************
-*  Block functions
-****************************************/
-/*! Block functions produce and decode raw zstd blocks, without frame metadata.
-    Frame metadata cost is typically ~18 bytes, which can be non-negligible for very small blocks (< 100 bytes).
-    User will have to take in charge required information to regenerate data, such as compressed and content sizes.
-
-    A few rules to respect :
-    - Compressing and decompressing require a context structure
-      + Use ZSTDv07_createCCtx() and ZSTDv07_createDCtx()
-    - It is necessary to init context before starting
-      + compression : ZSTDv07_compressBegin()
-      + decompression : ZSTDv07_decompressBegin()
-      + variants _usingDict() are also allowed
-      + copyCCtx() and copyDCtx() work too
-    - Block size is limited, it must be <= ZSTDv07_getBlockSizeMax()
-      + If you need to compress more, cut data into multiple blocks
-      + Consider using the regular ZSTDv07_compress() instead, as frame metadata costs become negligible when source size is large.
-    - When a block is considered not compressible enough, ZSTDv07_compressBlock() result will be zero.
-      In which case, nothing is produced into `dst`.
-      + User must test for such outcome and deal directly with uncompressed data
-      + ZSTDv07_decompressBlock() doesn't accept uncompressed data as input !!!
-      + In case of multiple successive blocks, decoder must be informed of uncompressed block existence to follow proper history.
-        Use ZSTDv07_insertBlock() in such a case.
-*/
-
-#define ZSTDv07_BLOCKSIZE_ABSOLUTEMAX (128 * 1024)   /* define, for static allocation */
-ZSTDLIBv07_API size_t ZSTDv07_decompressBlock(ZSTDv07_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-ZSTDLIBv07_API size_t ZSTDv07_insertBlock(ZSTDv07_DCtx* dctx, const void* blockStart, size_t blockSize);  /**< insert block into `dctx` history. Useful for uncompressed blocks */
-
-
-#endif   /* ZSTDv07_STATIC_LINKING_ONLY */
-
-
-/* ******************************************************************
-   mem.h
-   low-level memory access routines
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-#ifndef MEM_H_MODULE
-#define MEM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*-****************************************
-*  Compiler specifics
-******************************************/
-#if defined(_MSC_VER)   /* Visual Studio */
-#   include <stdlib.h>  /* _byteswap_ulong */
-#   include <intrin.h>  /* _byteswap_* */
-#endif
-#if defined(__GNUC__)
-#  define MEM_STATIC static __attribute__((unused))
-#elif defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-#  define MEM_STATIC static inline
-#elif defined(_MSC_VER)
-#  define MEM_STATIC static __inline
-#else
-#  define MEM_STATIC static  /* this version may generate warnings for unused static functions; disable the relevant warning */
-#endif
-
-
-/*-**************************************************************
-*  Basic Types
-*****************************************************************/
-#if  !defined (__VMS) && (defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */) )
-# include <stdint.h>
-  typedef  uint8_t BYTE;
-  typedef uint16_t U16;
-  typedef  int16_t S16;
-  typedef uint32_t U32;
-  typedef  int32_t S32;
-  typedef uint64_t U64;
-  typedef  int64_t S64;
-#else
-  typedef unsigned char       BYTE;
-  typedef unsigned short      U16;
-  typedef   signed short      S16;
-  typedef unsigned int        U32;
-  typedef   signed int        S32;
-  typedef unsigned long long  U64;
-  typedef   signed long long  S64;
-#endif
-
-
-/*-**************************************************************
-*  Memory I/O
-*****************************************************************/
-/* MEM_FORCE_MEMORY_ACCESS :
- * By default, access to unaligned memory is controlled by `memcpy()`, which is safe and portable.
- * Unfortunately, on some target/compiler combinations, the generated assembly is sub-optimal.
- * The below switch allow to select different access method for improved performance.
- * Method 0 (default) : use `memcpy()`. Safe and portable.
- * Method 1 : `__packed` statement. It depends on compiler extension (ie, not portable).
- *            This method is safe if your compiler supports it, and *generally* as fast or faster than `memcpy`.
- * Method 2 : direct access. This method is portable but violate C standard.
- *            It can generate buggy code on targets depending on alignment.
- *            In some circumstances, it's the only known way to get the most performance (ie GCC + ARMv6)
- * See http://fastcompression.blogspot.fr/2015/08/accessing-unaligned-memory.html for details.
- * Prefer these methods in priority order (0 > 1 > 2)
- */
-#ifndef MEM_FORCE_MEMORY_ACCESS   /* can be defined externally, on command line for example */
-#  if defined(__GNUC__) && ( defined(__ARM_ARCH_6__) || defined(__ARM_ARCH_6J__) || defined(__ARM_ARCH_6K__) || defined(__ARM_ARCH_6Z__) || defined(__ARM_ARCH_6ZK__) || defined(__ARM_ARCH_6T2__) )
-#    define MEM_FORCE_MEMORY_ACCESS 2
-#  elif (defined(__INTEL_COMPILER) && !defined(WIN32)) || \
-  (defined(__GNUC__) && ( defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_7A__) || defined(__ARM_ARCH_7R__) || defined(__ARM_ARCH_7M__) || defined(__ARM_ARCH_7S__) ))
-#    define MEM_FORCE_MEMORY_ACCESS 1
-#  endif
-#endif
-
-MEM_STATIC unsigned MEM_32bits(void) { return sizeof(size_t)==4; }
-MEM_STATIC unsigned MEM_64bits(void) { return sizeof(size_t)==8; }
-
-MEM_STATIC unsigned MEM_isLittleEndian(void)
-{
-    const union { U32 u; BYTE c[4]; } one = { 1 };   /* don't use static : performance detrimental  */
-    return one.c[0];
-}
-
-#if defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==2)
-
-/* violates C standard, by lying on structure alignment.
-Only use if no other choice to achieve best performance on target platform */
-MEM_STATIC U16 MEM_read16(const void* memPtr) { return *(const U16*) memPtr; }
-MEM_STATIC U32 MEM_read32(const void* memPtr) { return *(const U32*) memPtr; }
-MEM_STATIC U64 MEM_read64(const void* memPtr) { return *(const U64*) memPtr; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { *(U16*)memPtr = value; }
-
-#elif defined(MEM_FORCE_MEMORY_ACCESS) && (MEM_FORCE_MEMORY_ACCESS==1)
-
-/* __pack instructions are safer, but compiler specific, hence potentially problematic for some compilers */
-/* currently only defined for gcc and icc */
-typedef union { U16 u16; U32 u32; U64 u64; size_t st; } __attribute__((packed)) unalign;
-
-MEM_STATIC U16 MEM_read16(const void* ptr) { return ((const unalign*)ptr)->u16; }
-MEM_STATIC U32 MEM_read32(const void* ptr) { return ((const unalign*)ptr)->u32; }
-MEM_STATIC U64 MEM_read64(const void* ptr) { return ((const unalign*)ptr)->u64; }
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value) { ((unalign*)memPtr)->u16 = value; }
-
-#else
-
-/* default method, safe and standard.
-   can sometimes prove slower */
-
-MEM_STATIC U16 MEM_read16(const void* memPtr)
-{
-    U16 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U32 MEM_read32(const void* memPtr)
-{
-    U32 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC U64 MEM_read64(const void* memPtr)
-{
-    U64 val; memcpy(&val, memPtr, sizeof(val)); return val;
-}
-
-MEM_STATIC void MEM_write16(void* memPtr, U16 value)
-{
-    memcpy(memPtr, &value, sizeof(value));
-}
-
-#endif /* MEM_FORCE_MEMORY_ACCESS */
-
-MEM_STATIC U32 MEM_swap32(U32 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_ulong(in);
-#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)
-    return __builtin_bswap32(in);
-#else
-    return  ((in << 24) & 0xff000000 ) |
-            ((in <<  8) & 0x00ff0000 ) |
-            ((in >>  8) & 0x0000ff00 ) |
-            ((in >> 24) & 0x000000ff );
-#endif
-}
-
-MEM_STATIC U64 MEM_swap64(U64 in)
-{
-#if defined(_MSC_VER)     /* Visual Studio */
-    return _byteswap_uint64(in);
-#elif defined (__GNUC__) && (__GNUC__ * 100 + __GNUC_MINOR__ >= 403)
-    return __builtin_bswap64(in);
-#else
-    return  ((in << 56) & 0xff00000000000000ULL) |
-            ((in << 40) & 0x00ff000000000000ULL) |
-            ((in << 24) & 0x0000ff0000000000ULL) |
-            ((in << 8)  & 0x000000ff00000000ULL) |
-            ((in >> 8)  & 0x00000000ff000000ULL) |
-            ((in >> 24) & 0x0000000000ff0000ULL) |
-            ((in >> 40) & 0x000000000000ff00ULL) |
-            ((in >> 56) & 0x00000000000000ffULL);
-#endif
-}
-
-
-/*=== Little endian r/w ===*/
-
-MEM_STATIC U16 MEM_readLE16(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read16(memPtr);
-    else {
-        const BYTE* p = (const BYTE*)memPtr;
-        return (U16)(p[0] + (p[1]<<8));
-    }
-}
-
-MEM_STATIC void MEM_writeLE16(void* memPtr, U16 val)
-{
-    if (MEM_isLittleEndian()) {
-        MEM_write16(memPtr, val);
-    } else {
-        BYTE* p = (BYTE*)memPtr;
-        p[0] = (BYTE)val;
-        p[1] = (BYTE)(val>>8);
-    }
-}
-
-MEM_STATIC U32 MEM_readLE32(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read32(memPtr);
-    else
-        return MEM_swap32(MEM_read32(memPtr));
-}
-
-
-MEM_STATIC U64 MEM_readLE64(const void* memPtr)
-{
-    if (MEM_isLittleEndian())
-        return MEM_read64(memPtr);
-    else
-        return MEM_swap64(MEM_read64(memPtr));
-}
-
-MEM_STATIC size_t MEM_readLEST(const void* memPtr)
-{
-    if (MEM_32bits())
-        return (size_t)MEM_readLE32(memPtr);
-    else
-        return (size_t)MEM_readLE64(memPtr);
-}
-
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* MEM_H_MODULE */
-/* ******************************************************************
-   bitstream
-   Part of FSE library
-   header file (to include)
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef BITSTREAM_H_MODULE
-#define BITSTREAM_H_MODULE
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-/*
-*  This API consists of small unitary functions, which must be inlined for best performance.
-*  Since link-time-optimization is not available for all compilers,
-*  these functions are defined into a .h to be included.
-*/
-
-
-/*=========================================
-*  Target specific
-=========================================*/
-#if defined(__BMI__) && defined(__GNUC__)
-#  include <immintrin.h>   /* support for bextr (experimental) */
-#endif
-
-/*-********************************************
-*  bitStream decoding API (read backward)
-**********************************************/
-typedef struct
-{
-    size_t   bitContainer;
-    unsigned bitsConsumed;
-    const char* ptr;
-    const char* start;
-} BITv07_DStream_t;
-
-typedef enum { BITv07_DStream_unfinished = 0,
-               BITv07_DStream_endOfBuffer = 1,
-               BITv07_DStream_completed = 2,
-               BITv07_DStream_overflow = 3 } BITv07_DStream_status;  /* result of BITv07_reloadDStream() */
-               /* 1,2,4,8 would be better for bitmap combinations, but slows down performance a bit ... :( */
-
-MEM_STATIC size_t   BITv07_initDStream(BITv07_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
-MEM_STATIC size_t   BITv07_readBits(BITv07_DStream_t* bitD, unsigned nbBits);
-MEM_STATIC BITv07_DStream_status BITv07_reloadDStream(BITv07_DStream_t* bitD);
-MEM_STATIC unsigned BITv07_endOfDStream(const BITv07_DStream_t* bitD);
-
-
-
-/*-****************************************
-*  unsafe API
-******************************************/
-MEM_STATIC size_t BITv07_readBitsFast(BITv07_DStream_t* bitD, unsigned nbBits);
-/* faster, but works only if nbBits >= 1 */
-
-
-
-/*-**************************************************************
-*  Internal functions
-****************************************************************/
-MEM_STATIC unsigned BITv07_highbit32 (U32 val)
-{
-#   if defined(_MSC_VER)   /* Visual */
-    unsigned long r=0;
-    _BitScanReverse ( &r, val );
-    return (unsigned) r;
-#   elif defined(__GNUC__) && (__GNUC__ >= 3)   /* Use GCC Intrinsic */
-    return 31 - __builtin_clz (val);
-#   else   /* Software version */
-    static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 };
-    U32 v = val;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    return DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27];
-#   endif
-}
-
-
-
-/*-********************************************************
-* bitStream decoding
-**********************************************************/
-/*! BITv07_initDStream() :
-*   Initialize a BITv07_DStream_t.
-*   `bitD` : a pointer to an already allocated BITv07_DStream_t structure.
-*   `srcSize` must be the *exact* size of the bitStream, in bytes.
-*   @return : size of stream (== srcSize) or an errorCode if a problem is detected
-*/
-MEM_STATIC size_t BITv07_initDStream(BITv07_DStream_t* bitD, const void* srcBuffer, size_t srcSize)
-{
-    if (srcSize < 1) { memset(bitD, 0, sizeof(*bitD)); return ERROR(srcSize_wrong); }
-
-    if (srcSize >=  sizeof(bitD->bitContainer)) {  /* normal case */
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = (const char*)srcBuffer + srcSize - sizeof(bitD->bitContainer);
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-          bitD->bitsConsumed = lastByte ? 8 - BITv07_highbit32(lastByte) : 0;
-          if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ }
-    } else {
-        bitD->start = (const char*)srcBuffer;
-        bitD->ptr   = bitD->start;
-        bitD->bitContainer = *(const BYTE*)(bitD->start);
-        switch(srcSize)
-        {
-            case 7: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[6]) << (sizeof(bitD->bitContainer)*8 - 16);/* fall-through */
-            case 6: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[5]) << (sizeof(bitD->bitContainer)*8 - 24);/* fall-through */
-            case 5: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[4]) << (sizeof(bitD->bitContainer)*8 - 32);/* fall-through */
-            case 4: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[3]) << 24; /* fall-through */
-            case 3: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[2]) << 16; /* fall-through */
-            case 2: bitD->bitContainer += (size_t)(((const BYTE*)(srcBuffer))[1]) <<  8; /* fall-through */
-            default: break;
-        }
-        { BYTE const lastByte = ((const BYTE*)srcBuffer)[srcSize-1];
-          bitD->bitsConsumed = lastByte ? 8 - BITv07_highbit32(lastByte) : 0;
-          if (lastByte == 0) return ERROR(GENERIC); /* endMark not present */ }
-        bitD->bitsConsumed += (U32)(sizeof(bitD->bitContainer) - srcSize)*8;
-    }
-
-    return srcSize;
-}
-
-
- MEM_STATIC size_t BITv07_lookBits(const BITv07_DStream_t* bitD, U32 nbBits)
-{
-    U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return ((bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> 1) >> ((bitMask-nbBits) & bitMask);
-}
-
-/*! BITv07_lookBitsFast() :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv07_lookBitsFast(const BITv07_DStream_t* bitD, U32 nbBits)
-{
-    U32 const bitMask = sizeof(bitD->bitContainer)*8 - 1;
-    return (bitD->bitContainer << (bitD->bitsConsumed & bitMask)) >> (((bitMask+1)-nbBits) & bitMask);
-}
-
-MEM_STATIC void BITv07_skipBits(BITv07_DStream_t* bitD, U32 nbBits)
-{
-    bitD->bitsConsumed += nbBits;
-}
-
-MEM_STATIC size_t BITv07_readBits(BITv07_DStream_t* bitD, U32 nbBits)
-{
-    size_t const value = BITv07_lookBits(bitD, nbBits);
-    BITv07_skipBits(bitD, nbBits);
-    return value;
-}
-
-/*! BITv07_readBitsFast() :
-*   unsafe version; only works only if nbBits >= 1 */
-MEM_STATIC size_t BITv07_readBitsFast(BITv07_DStream_t* bitD, U32 nbBits)
-{
-    size_t const value = BITv07_lookBitsFast(bitD, nbBits);
-    BITv07_skipBits(bitD, nbBits);
-    return value;
-}
-
-MEM_STATIC BITv07_DStream_status BITv07_reloadDStream(BITv07_DStream_t* bitD)
-{
-    if (bitD->bitsConsumed > (sizeof(bitD->bitContainer)*8))  /* should not happen => corruption detected */
-        return BITv07_DStream_overflow;
-
-    if (bitD->ptr >= bitD->start + sizeof(bitD->bitContainer)) {
-        bitD->ptr -= bitD->bitsConsumed >> 3;
-        bitD->bitsConsumed &= 7;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);
-        return BITv07_DStream_unfinished;
-    }
-    if (bitD->ptr == bitD->start) {
-        if (bitD->bitsConsumed < sizeof(bitD->bitContainer)*8) return BITv07_DStream_endOfBuffer;
-        return BITv07_DStream_completed;
-    }
-    {   U32 nbBytes = bitD->bitsConsumed >> 3;
-        BITv07_DStream_status result = BITv07_DStream_unfinished;
-        if (bitD->ptr - nbBytes < bitD->start) {
-            nbBytes = (U32)(bitD->ptr - bitD->start);  /* ptr > start */
-            result = BITv07_DStream_endOfBuffer;
-        }
-        bitD->ptr -= nbBytes;
-        bitD->bitsConsumed -= nbBytes*8;
-        bitD->bitContainer = MEM_readLEST(bitD->ptr);   /* reminder : srcSize > sizeof(bitD) */
-        return result;
-    }
-}
-
-/*! BITv07_endOfDStream() :
-*   @return Tells if DStream has exactly reached its end (all bits consumed).
-*/
-MEM_STATIC unsigned BITv07_endOfDStream(const BITv07_DStream_t* DStream)
-{
-    return ((DStream->ptr == DStream->start) && (DStream->bitsConsumed == sizeof(DStream->bitContainer)*8));
-}
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif /* BITSTREAM_H_MODULE */
-/* ******************************************************************
-   FSE : Finite State Entropy codec
-   Public Prototypes declaration
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef FSEv07_H
-#define FSEv07_H
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/*-****************************************
-*  FSE simple functions
-******************************************/
-
-/*! FSEv07_decompress():
-    Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated destination buffer 'dst', of size 'dstCapacity'.
-    @return : size of regenerated data (<= maxDstSize),
-              or an error code, which can be tested using FSEv07_isError() .
-
-    ** Important ** : FSEv07_decompress() does not decompress non-compressible nor RLE data !!!
-    Why ? : making this distinction requires a header.
-    Header management is intentionally delegated to the user layer, which can better manage special cases.
-*/
-size_t FSEv07_decompress(void* dst,  size_t dstCapacity,
-                const void* cSrc, size_t cSrcSize);
-
-
-/* Error Management */
-unsigned    FSEv07_isError(size_t code);        /* tells if a return value is an error code */
-const char* FSEv07_getErrorName(size_t code);   /* provides error code string (useful for debugging) */
-
-
-/*-*****************************************
-*  FSE detailed API
-******************************************/
-/*!
-FSEv07_decompress() does the following:
-1. read normalized counters with readNCount()
-2. build decoding table 'DTable' from normalized counters
-3. decode the data stream using decoding table 'DTable'
-
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and provide normalized distribution using external method.
-*/
-
-
-/* *** DECOMPRESSION *** */
-
-/*! FSEv07_readNCount():
-    Read compactly saved 'normalizedCounter' from 'rBuffer'.
-    @return : size read from 'rBuffer',
-              or an errorCode, which can be tested using FSEv07_isError().
-              maxSymbolValuePtr[0] and tableLogPtr[0] will also be updated with their respective values */
-size_t FSEv07_readNCount (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* rBuffer, size_t rBuffSize);
-
-/*! Constructor and Destructor of FSEv07_DTable.
-    Note that its size depends on 'tableLog' */
-typedef unsigned FSEv07_DTable;   /* don't allocate that. It's just a way to be more restrictive than void* */
-FSEv07_DTable* FSEv07_createDTable(unsigned tableLog);
-void        FSEv07_freeDTable(FSEv07_DTable* dt);
-
-/*! FSEv07_buildDTable():
-    Builds 'dt', which must be already allocated, using FSEv07_createDTable().
-    return : 0, or an errorCode, which can be tested using FSEv07_isError() */
-size_t FSEv07_buildDTable (FSEv07_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
-
-/*! FSEv07_decompress_usingDTable():
-    Decompress compressed source `cSrc` of size `cSrcSize` using `dt`
-    into `dst` which must be already allocated.
-    @return : size of regenerated data (necessarily <= `dstCapacity`),
-              or an errorCode, which can be tested using FSEv07_isError() */
-size_t FSEv07_decompress_usingDTable(void* dst, size_t dstCapacity, const void* cSrc, size_t cSrcSize, const FSEv07_DTable* dt);
-
-/*!
-Tutorial :
-----------
-(Note : these functions only decompress FSE-compressed blocks.
- If block is uncompressed, use memcpy() instead
- If block is a single repeated byte, use memset() instead )
-
-The first step is to obtain the normalized frequencies of symbols.
-This can be performed by FSEv07_readNCount() if it was saved using FSEv07_writeNCount().
-'normalizedCounter' must be already allocated, and have at least 'maxSymbolValuePtr[0]+1' cells of signed short.
-In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
-or size the table to handle worst case situations (typically 256).
-FSEv07_readNCount() will provide 'tableLog' and 'maxSymbolValue'.
-The result of FSEv07_readNCount() is the number of bytes read from 'rBuffer'.
-Note that 'rBufferSize' must be at least 4 bytes, even if useful information is less than that.
-If there is an error, the function will return an error code, which can be tested using FSEv07_isError().
-
-The next step is to build the decompression tables 'FSEv07_DTable' from 'normalizedCounter'.
-This is performed by the function FSEv07_buildDTable().
-The space required by 'FSEv07_DTable' must be already allocated using FSEv07_createDTable().
-If there is an error, the function will return an error code, which can be tested using FSEv07_isError().
-
-`FSEv07_DTable` can then be used to decompress `cSrc`, with FSEv07_decompress_usingDTable().
-`cSrcSize` must be strictly correct, otherwise decompression will fail.
-FSEv07_decompress_usingDTable() result will tell how many bytes were regenerated (<=`dstCapacity`).
-If there is an error, the function will return an error code, which can be tested using FSEv07_isError(). (ex: dst buffer too small)
-*/
-
-
-#ifdef FSEv07_STATIC_LINKING_ONLY
-
-
-/* *****************************************
-*  Static allocation
-*******************************************/
-/* FSE buffer bounds */
-#define FSEv07_NCOUNTBOUND 512
-#define FSEv07_BLOCKBOUND(size) (size + (size>>7))
-
-/* It is possible to statically allocate FSE CTable/DTable as a table of unsigned using below macros */
-#define FSEv07_DTABLE_SIZE_U32(maxTableLog)                   (1 + (1<<maxTableLog))
-
-
-/* *****************************************
-*  FSE advanced API
-*******************************************/
-size_t FSEv07_countFast(unsigned* count, unsigned* maxSymbolValuePtr, const void* src, size_t srcSize);
-/**< same as FSEv07_count(), but blindly trusts that all byte values within src are <= *maxSymbolValuePtr  */
-
-unsigned FSEv07_optimalTableLog_internal(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue, unsigned minus);
-/**< same as FSEv07_optimalTableLog(), which used `minus==2` */
-
-size_t FSEv07_buildDTable_raw (FSEv07_DTable* dt, unsigned nbBits);
-/**< build a fake FSEv07_DTable, designed to read an uncompressed bitstream where each symbol uses nbBits */
-
-size_t FSEv07_buildDTable_rle (FSEv07_DTable* dt, unsigned char symbolValue);
-/**< build a fake FSEv07_DTable, designed to always generate the same symbolValue */
-
-
-
-/* *****************************************
-*  FSE symbol decompression API
-*******************************************/
-typedef struct
-{
-    size_t      state;
-    const void* table;   /* precise table may vary, depending on U16 */
-} FSEv07_DState_t;
-
-
-static void     FSEv07_initDState(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD, const FSEv07_DTable* dt);
-
-static unsigned char FSEv07_decodeSymbol(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD);
-
-
-
-/* *****************************************
-*  FSE unsafe API
-*******************************************/
-static unsigned char FSEv07_decodeSymbolFast(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD);
-/* faster, but works only if nbBits is always >= 1 (otherwise, result will be corrupted) */
-
-
-/* ======    Decompression    ====== */
-
-typedef struct {
-    U16 tableLog;
-    U16 fastMode;
-} FSEv07_DTableHeader;   /* sizeof U32 */
-
-typedef struct
-{
-    unsigned short newState;
-    unsigned char  symbol;
-    unsigned char  nbBits;
-} FSEv07_decode_t;   /* size == U32 */
-
-MEM_STATIC void FSEv07_initDState(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD, const FSEv07_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv07_DTableHeader* const DTableH = (const FSEv07_DTableHeader*)ptr;
-    DStatePtr->state = BITv07_readBits(bitD, DTableH->tableLog);
-    BITv07_reloadDStream(bitD);
-    DStatePtr->table = dt + 1;
-}
-
-MEM_STATIC BYTE FSEv07_peekSymbol(const FSEv07_DState_t* DStatePtr)
-{
-    FSEv07_decode_t const DInfo = ((const FSEv07_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    return DInfo.symbol;
-}
-
-MEM_STATIC void FSEv07_updateState(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD)
-{
-    FSEv07_decode_t const DInfo = ((const FSEv07_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    size_t const lowBits = BITv07_readBits(bitD, nbBits);
-    DStatePtr->state = DInfo.newState + lowBits;
-}
-
-MEM_STATIC BYTE FSEv07_decodeSymbol(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD)
-{
-    FSEv07_decode_t const DInfo = ((const FSEv07_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BITv07_readBits(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-/*! FSEv07_decodeSymbolFast() :
-    unsafe, only works if no symbol has a probability > 50% */
-MEM_STATIC BYTE FSEv07_decodeSymbolFast(FSEv07_DState_t* DStatePtr, BITv07_DStream_t* bitD)
-{
-    FSEv07_decode_t const DInfo = ((const FSEv07_decode_t*)(DStatePtr->table))[DStatePtr->state];
-    U32 const nbBits = DInfo.nbBits;
-    BYTE const symbol = DInfo.symbol;
-    size_t const lowBits = BITv07_readBitsFast(bitD, nbBits);
-
-    DStatePtr->state = DInfo.newState + lowBits;
-    return symbol;
-}
-
-
-
-#ifndef FSEv07_COMMONDEFS_ONLY
-
-/* **************************************************************
-*  Tuning parameters
-****************************************************************/
-/*!MEMORY_USAGE :
-*  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
-*  Increasing memory usage improves compression ratio
-*  Reduced memory usage can improve speed, due to cache effect
-*  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
-#define FSEv07_MAX_MEMORY_USAGE 14
-#define FSEv07_DEFAULT_MEMORY_USAGE 13
-
-/*!FSEv07_MAX_SYMBOL_VALUE :
-*  Maximum symbol value authorized.
-*  Required for proper stack allocation */
-#define FSEv07_MAX_SYMBOL_VALUE 255
-
-
-/* **************************************************************
-*  template functions type & suffix
-****************************************************************/
-#define FSEv07_FUNCTION_TYPE BYTE
-#define FSEv07_FUNCTION_EXTENSION
-#define FSEv07_DECODE_TYPE FSEv07_decode_t
-
-
-#endif   /* !FSEv07_COMMONDEFS_ONLY */
-
-
-/* ***************************************************************
-*  Constants
-*****************************************************************/
-#define FSEv07_MAX_TABLELOG  (FSEv07_MAX_MEMORY_USAGE-2)
-#define FSEv07_MAX_TABLESIZE (1U<<FSEv07_MAX_TABLELOG)
-#define FSEv07_MAXTABLESIZE_MASK (FSEv07_MAX_TABLESIZE-1)
-#define FSEv07_DEFAULT_TABLELOG (FSEv07_DEFAULT_MEMORY_USAGE-2)
-#define FSEv07_MIN_TABLELOG 5
-
-#define FSEv07_TABLELOG_ABSOLUTE_MAX 15
-#if FSEv07_MAX_TABLELOG > FSEv07_TABLELOG_ABSOLUTE_MAX
-#  error "FSEv07_MAX_TABLELOG > FSEv07_TABLELOG_ABSOLUTE_MAX is not supported"
-#endif
-
-#define FSEv07_TABLESTEP(tableSize) ((tableSize>>1) + (tableSize>>3) + 3)
-
-
-#endif /* FSEv07_STATIC_LINKING_ONLY */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* FSEv07_H */
-/* ******************************************************************
-   Huffman coder, part of New Generation Entropy library
-   header file
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-   You can contact the author at :
-   - Source repository : https://github.com/Cyan4973/FiniteStateEntropy
-****************************************************************** */
-#ifndef HUFv07_H_298734234
-#define HUFv07_H_298734234
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-
-
-/* *** simple functions *** */
-/**
-HUFv07_decompress() :
-    Decompress HUF data from buffer 'cSrc', of size 'cSrcSize',
-    into already allocated buffer 'dst', of minimum size 'dstSize'.
-    `dstSize` : **must** be the ***exact*** size of original (uncompressed) data.
-    Note : in contrast with FSE, HUFv07_decompress can regenerate
-           RLE (cSrcSize==1) and uncompressed (cSrcSize==dstSize) data,
-           because it knows size to regenerate.
-    @return : size of regenerated data (== dstSize),
-              or an error code, which can be tested using HUFv07_isError()
-*/
-size_t HUFv07_decompress(void* dst,  size_t dstSize,
-                const void* cSrc, size_t cSrcSize);
-
-
-/* ****************************************
-*  Tool functions
-******************************************/
-#define HUFv07_BLOCKSIZE_MAX (128 * 1024)
-
-/* Error Management */
-unsigned    HUFv07_isError(size_t code);        /**< tells if a return value is an error code */
-const char* HUFv07_getErrorName(size_t code);   /**< provides error code string (useful for debugging) */
-
-
-/* *** Advanced function *** */
-
-
-#ifdef HUFv07_STATIC_LINKING_ONLY
-
-
-/* *** Constants *** */
-#define HUFv07_TABLELOG_ABSOLUTEMAX  16   /* absolute limit of HUFv07_MAX_TABLELOG. Beyond that value, code does not work */
-#define HUFv07_TABLELOG_MAX  12           /* max configured tableLog (for static allocation); can be modified up to HUFv07_ABSOLUTEMAX_TABLELOG */
-#define HUFv07_TABLELOG_DEFAULT  11       /* tableLog by default, when not specified */
-#define HUFv07_SYMBOLVALUE_MAX 255
-#if (HUFv07_TABLELOG_MAX > HUFv07_TABLELOG_ABSOLUTEMAX)
-#  error "HUFv07_TABLELOG_MAX is too large !"
-#endif
-
-
-/* ****************************************
-*  Static allocation
-******************************************/
-/* HUF buffer bounds */
-#define HUFv07_BLOCKBOUND(size) (size + (size>>8) + 8)   /* only true if incompressible pre-filtered with fast heuristic */
-
-/* static allocation of HUF's DTable */
-typedef U32 HUFv07_DTable;
-#define HUFv07_DTABLE_SIZE(maxTableLog)   (1 + (1<<(maxTableLog)))
-#define HUFv07_CREATE_STATIC_DTABLEX2(DTable, maxTableLog) \
-        HUFv07_DTable DTable[HUFv07_DTABLE_SIZE((maxTableLog)-1)] = { ((U32)((maxTableLog)-1)*0x1000001) }
-#define HUFv07_CREATE_STATIC_DTABLEX4(DTable, maxTableLog) \
-        HUFv07_DTable DTable[HUFv07_DTABLE_SIZE(maxTableLog)] = { ((U32)(maxTableLog)*0x1000001) }
-
-
-/* ****************************************
-*  Advanced decompression functions
-******************************************/
-size_t HUFv07_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-size_t HUFv07_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-
-size_t HUFv07_decompress4X_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< decodes RLE and uncompressed */
-size_t HUFv07_decompress4X_hufOnly(HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize); /**< considers RLE and uncompressed as errors */
-size_t HUFv07_decompress4X2_DCtx(HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-size_t HUFv07_decompress4X4_DCtx(HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-
-size_t HUFv07_decompress1X_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-size_t HUFv07_decompress1X2_DCtx(HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< single-symbol decoder */
-size_t HUFv07_decompress1X4_DCtx(HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /**< double-symbols decoder */
-
-
-/* ****************************************
-*  HUF detailed API
-******************************************/
-/*!
-The following API allows targeting specific sub-functions for advanced tasks.
-For example, it's possible to compress several blocks using the same 'CTable',
-or to save and regenerate 'CTable' using external methods.
-*/
-/* FSEv07_count() : find it within "fse.h" */
-
-/*! HUFv07_readStats() :
-    Read compact Huffman tree, saved by HUFv07_writeCTable().
-    `huffWeight` is destination buffer.
-    @return : size read from `src` , or an error Code .
-    Note : Needed by HUFv07_readCTable() and HUFv07_readDTableXn() . */
-size_t HUFv07_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                     U32* nbSymbolsPtr, U32* tableLogPtr,
-                     const void* src, size_t srcSize);
-
-
-/*
-HUFv07_decompress() does the following:
-1. select the decompression algorithm (X2, X4) based on pre-computed heuristics
-2. build Huffman table from save, using HUFv07_readDTableXn()
-3. decode 1 or 4 segments in parallel using HUFv07_decompressSXn_usingDTable
-*/
-
-/** HUFv07_selectDecoder() :
-*   Tells which decoder is likely to decode faster,
-*   based on a set of pre-determined metrics.
-*   @return : 0==HUFv07_decompress4X2, 1==HUFv07_decompress4X4 .
-*   Assumption : 0 < cSrcSize < dstSize <= 128 KB */
-U32 HUFv07_selectDecoder (size_t dstSize, size_t cSrcSize);
-
-size_t HUFv07_readDTableX2 (HUFv07_DTable* DTable, const void* src, size_t srcSize);
-size_t HUFv07_readDTableX4 (HUFv07_DTable* DTable, const void* src, size_t srcSize);
-
-size_t HUFv07_decompress4X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-size_t HUFv07_decompress4X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-size_t HUFv07_decompress4X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-
-
-/* single stream variants */
-size_t HUFv07_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* single-symbol decoder */
-size_t HUFv07_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);   /* double-symbol decoder */
-
-size_t HUFv07_decompress1X_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-size_t HUFv07_decompress1X2_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-size_t HUFv07_decompress1X4_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const HUFv07_DTable* DTable);
-
-
-#endif /* HUFv07_STATIC_LINKING_ONLY */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* HUFv07_H_298734234 */
-/*
-   Common functions of New Generation Entropy library
-   Copyright (C) 2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-*************************************************************************** */
-
-
-
-/*-****************************************
-*  FSE Error Management
-******************************************/
-unsigned FSEv07_isError(size_t code) { return ERR_isError(code); }
-
-const char* FSEv07_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/* **************************************************************
-*  HUF Error Management
-****************************************************************/
-unsigned HUFv07_isError(size_t code) { return ERR_isError(code); }
-
-const char* HUFv07_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-/*-**************************************************************
-*  FSE NCount encoding-decoding
-****************************************************************/
-static short FSEv07_abs(short a) { return (short)(a<0 ? -a : a); }
-
-size_t FSEv07_readNCount (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr,
-                 const void* headerBuffer, size_t hbSize)
-{
-    const BYTE* const istart = (const BYTE*) headerBuffer;
-    const BYTE* const iend = istart + hbSize;
-    const BYTE* ip = istart;
-    int nbBits;
-    int remaining;
-    int threshold;
-    U32 bitStream;
-    int bitCount;
-    unsigned charnum = 0;
-    int previous0 = 0;
-
-    if (hbSize < 4) return ERROR(srcSize_wrong);
-    bitStream = MEM_readLE32(ip);
-    nbBits = (bitStream & 0xF) + FSEv07_MIN_TABLELOG;   /* extract tableLog */
-    if (nbBits > FSEv07_TABLELOG_ABSOLUTE_MAX) return ERROR(tableLog_tooLarge);
-    bitStream >>= 4;
-    bitCount = 4;
-    *tableLogPtr = nbBits;
-    remaining = (1<<nbBits)+1;
-    threshold = 1<<nbBits;
-    nbBits++;
-
-    while ((remaining>1) && (charnum<=*maxSVPtr)) {
-        if (previous0) {
-            unsigned n0 = charnum;
-            while ((bitStream & 0xFFFF) == 0xFFFF) {
-                n0+=24;
-                if (ip < iend-5) {
-                    ip+=2;
-                    bitStream = MEM_readLE32(ip) >> bitCount;
-                } else {
-                    bitStream >>= 16;
-                    bitCount+=16;
-            }   }
-            while ((bitStream & 3) == 3) {
-                n0+=3;
-                bitStream>>=2;
-                bitCount+=2;
-            }
-            n0 += bitStream & 3;
-            bitCount += 2;
-            if (n0 > *maxSVPtr) return ERROR(maxSymbolValue_tooSmall);
-            while (charnum < n0) normalizedCounter[charnum++] = 0;
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-                bitStream = MEM_readLE32(ip) >> bitCount;
-            }
-            else
-                bitStream >>= 2;
-        }
-        {   short const max = (short)((2*threshold-1)-remaining);
-            short count;
-
-            if ((bitStream & (threshold-1)) < (U32)max) {
-                count = (short)(bitStream & (threshold-1));
-                bitCount   += nbBits-1;
-            } else {
-                count = (short)(bitStream & (2*threshold-1));
-                if (count >= threshold) count -= max;
-                bitCount   += nbBits;
-            }
-
-            count--;   /* extra accuracy */
-            remaining -= FSEv07_abs(count);
-            normalizedCounter[charnum++] = count;
-            previous0 = !count;
-            while (remaining < threshold) {
-                nbBits--;
-                threshold >>= 1;
-            }
-
-            if ((ip <= iend-7) || (ip + (bitCount>>3) <= iend-4)) {
-                ip += bitCount>>3;
-                bitCount &= 7;
-            } else {
-                bitCount -= (int)(8 * (iend - 4 - ip));
-                ip = iend - 4;
-            }
-            bitStream = MEM_readLE32(ip) >> (bitCount & 31);
-    }   }   /* while ((remaining>1) && (charnum<=*maxSVPtr)) */
-    if (remaining != 1) return ERROR(GENERIC);
-    *maxSVPtr = charnum-1;
-
-    ip += (bitCount+7)>>3;
-    if ((size_t)(ip-istart) > hbSize) return ERROR(srcSize_wrong);
-    return ip-istart;
-}
-
-
-/*! HUFv07_readStats() :
-    Read compact Huffman tree, saved by HUFv07_writeCTable().
-    `huffWeight` is destination buffer.
-    @return : size read from `src` , or an error Code .
-    Note : Needed by HUFv07_readCTable() and HUFv07_readDTableXn() .
-*/
-size_t HUFv07_readStats(BYTE* huffWeight, size_t hwSize, U32* rankStats,
-                     U32* nbSymbolsPtr, U32* tableLogPtr,
-                     const void* src, size_t srcSize)
-{
-    U32 weightTotal;
-    const BYTE* ip = (const BYTE*) src;
-    size_t iSize;
-    size_t oSize;
-
-    if (!srcSize) return ERROR(srcSize_wrong);
-    iSize = ip[0];
-    //memset(huffWeight, 0, hwSize);   /* is not necessary, even though some analyzer complain ... */
-
-    if (iSize >= 128)  { /* special header */
-        if (iSize >= (242)) {  /* RLE */
-            static U32 l[14] = { 1, 2, 3, 4, 7, 8, 15, 16, 31, 32, 63, 64, 127, 128 };
-            oSize = l[iSize-242];
-            memset(huffWeight, 1, hwSize);
-            iSize = 0;
-        }
-        else {   /* Incompressible */
-            oSize = iSize - 127;
-            iSize = ((oSize+1)/2);
-            if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-            if (oSize >= hwSize) return ERROR(corruption_detected);
-            ip += 1;
-            {   U32 n;
-                for (n=0; n<oSize; n+=2) {
-                    huffWeight[n]   = ip[n/2] >> 4;
-                    huffWeight[n+1] = ip[n/2] & 15;
-    }   }   }   }
-    else  {   /* header compressed with FSE (normal case) */
-        if (iSize+1 > srcSize) return ERROR(srcSize_wrong);
-        oSize = FSEv07_decompress(huffWeight, hwSize-1, ip+1, iSize);   /* max (hwSize-1) values decoded, as last one is implied */
-        if (FSEv07_isError(oSize)) return oSize;
-    }
-
-    /* collect weight stats */
-    memset(rankStats, 0, (HUFv07_TABLELOG_ABSOLUTEMAX + 1) * sizeof(U32));
-    weightTotal = 0;
-    {   U32 n; for (n=0; n<oSize; n++) {
-            if (huffWeight[n] >= HUFv07_TABLELOG_ABSOLUTEMAX) return ERROR(corruption_detected);
-            rankStats[huffWeight[n]]++;
-            weightTotal += (1 << huffWeight[n]) >> 1;
-    }   }
-    if (weightTotal == 0) return ERROR(corruption_detected);
-
-    /* get last non-null symbol weight (implied, total must be 2^n) */
-    {   U32 const tableLog = BITv07_highbit32(weightTotal) + 1;
-        if (tableLog > HUFv07_TABLELOG_ABSOLUTEMAX) return ERROR(corruption_detected);
-        *tableLogPtr = tableLog;
-        /* determine last weight */
-        {   U32 const total = 1 << tableLog;
-            U32 const rest = total - weightTotal;
-            U32 const verif = 1 << BITv07_highbit32(rest);
-            U32 const lastWeight = BITv07_highbit32(rest) + 1;
-            if (verif != rest) return ERROR(corruption_detected);    /* last value must be a clean power of 2 */
-            huffWeight[oSize] = (BYTE)lastWeight;
-            rankStats[lastWeight]++;
-    }   }
-
-    /* check tree construction validity */
-    if ((rankStats[1] < 2) || (rankStats[1] & 1)) return ERROR(corruption_detected);   /* by construction : at least 2 elts of rank 1, must be even */
-
-    /* results */
-    *nbSymbolsPtr = (U32)(oSize+1);
-    return iSize+1;
-}
-/* ******************************************************************
-   FSE : Finite State Entropy decoder
-   Copyright (C) 2013-2015, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  define FORCE_INLINE static __forceinline
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4214)        /* disable: C4214: non-int bitfields */
-#else
-#  if defined (__cplusplus) || defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L   /* C99 */
-#    ifdef __GNUC__
-#      define FORCE_INLINE static inline __attribute__((always_inline))
-#    else
-#      define FORCE_INLINE static inline
-#    endif
-#  else
-#    define FORCE_INLINE static
-#  endif /* __STDC_VERSION__ */
-#endif
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define FSEv07_isError ERR_isError
-#define FSEv07_STATIC_ASSERT(c) { enum { FSEv07_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/* **************************************************************
-*  Complex types
-****************************************************************/
-typedef U32 DTable_max_t[FSEv07_DTABLE_SIZE_U32(FSEv07_MAX_TABLELOG)];
-
-
-/* **************************************************************
-*  Templates
-****************************************************************/
-/*
-  designed to be included
-  for type-specific functions (template emulation in C)
-  Objective is to write these functions only once, for improved maintenance
-*/
-
-/* safety checks */
-#ifndef FSEv07_FUNCTION_EXTENSION
-#  error "FSEv07_FUNCTION_EXTENSION must be defined"
-#endif
-#ifndef FSEv07_FUNCTION_TYPE
-#  error "FSEv07_FUNCTION_TYPE must be defined"
-#endif
-
-/* Function names */
-#define FSEv07_CAT(X,Y) X##Y
-#define FSEv07_FUNCTION_NAME(X,Y) FSEv07_CAT(X,Y)
-#define FSEv07_TYPE_NAME(X,Y) FSEv07_CAT(X,Y)
-
-
-/* Function templates */
-FSEv07_DTable* FSEv07_createDTable (unsigned tableLog)
-{
-    if (tableLog > FSEv07_TABLELOG_ABSOLUTE_MAX) tableLog = FSEv07_TABLELOG_ABSOLUTE_MAX;
-    return (FSEv07_DTable*)malloc( FSEv07_DTABLE_SIZE_U32(tableLog) * sizeof (U32) );
-}
-
-void FSEv07_freeDTable (FSEv07_DTable* dt)
-{
-    free(dt);
-}
-
-size_t FSEv07_buildDTable(FSEv07_DTable* dt, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog)
-{
-    void* const tdPtr = dt+1;   /* because *dt is unsigned, 32-bits aligned on 32-bits */
-    FSEv07_DECODE_TYPE* const tableDecode = (FSEv07_DECODE_TYPE*) (tdPtr);
-    U16 symbolNext[FSEv07_MAX_SYMBOL_VALUE+1];
-
-    U32 const maxSV1 = maxSymbolValue + 1;
-    U32 const tableSize = 1 << tableLog;
-    U32 highThreshold = tableSize-1;
-
-    /* Sanity Checks */
-    if (maxSymbolValue > FSEv07_MAX_SYMBOL_VALUE) return ERROR(maxSymbolValue_tooLarge);
-    if (tableLog > FSEv07_MAX_TABLELOG) return ERROR(tableLog_tooLarge);
-
-    /* Init, lay down lowprob symbols */
-    {   FSEv07_DTableHeader DTableH;
-        DTableH.tableLog = (U16)tableLog;
-        DTableH.fastMode = 1;
-        {   S16 const largeLimit= (S16)(1 << (tableLog-1));
-            U32 s;
-            for (s=0; s<maxSV1; s++) {
-                if (normalizedCounter[s]==-1) {
-                    tableDecode[highThreshold--].symbol = (FSEv07_FUNCTION_TYPE)s;
-                    symbolNext[s] = 1;
-                } else {
-                    if (normalizedCounter[s] >= largeLimit) DTableH.fastMode=0;
-                    symbolNext[s] = normalizedCounter[s];
-        }   }   }
-        memcpy(dt, &DTableH, sizeof(DTableH));
-    }
-
-    /* Spread symbols */
-    {   U32 const tableMask = tableSize-1;
-        U32 const step = FSEv07_TABLESTEP(tableSize);
-        U32 s, position = 0;
-        for (s=0; s<maxSV1; s++) {
-            int i;
-            for (i=0; i<normalizedCounter[s]; i++) {
-                tableDecode[position].symbol = (FSEv07_FUNCTION_TYPE)s;
-                position = (position + step) & tableMask;
-                while (position > highThreshold) position = (position + step) & tableMask;   /* lowprob area */
-        }   }
-
-        if (position!=0) return ERROR(GENERIC);   /* position must reach all cells once, otherwise normalizedCounter is incorrect */
-    }
-
-    /* Build Decoding table */
-    {   U32 u;
-        for (u=0; u<tableSize; u++) {
-            FSEv07_FUNCTION_TYPE const symbol = (FSEv07_FUNCTION_TYPE)(tableDecode[u].symbol);
-            U16 nextState = symbolNext[symbol]++;
-            tableDecode[u].nbBits = (BYTE) (tableLog - BITv07_highbit32 ((U32)nextState) );
-            tableDecode[u].newState = (U16) ( (nextState << tableDecode[u].nbBits) - tableSize);
-    }   }
-
-    return 0;
-}
-
-
-
-#ifndef FSEv07_COMMONDEFS_ONLY
-
-/*-*******************************************************
-*  Decompression (Byte symbols)
-*********************************************************/
-size_t FSEv07_buildDTable_rle (FSEv07_DTable* dt, BYTE symbolValue)
-{
-    void* ptr = dt;
-    FSEv07_DTableHeader* const DTableH = (FSEv07_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv07_decode_t* const cell = (FSEv07_decode_t*)dPtr;
-
-    DTableH->tableLog = 0;
-    DTableH->fastMode = 0;
-
-    cell->newState = 0;
-    cell->symbol = symbolValue;
-    cell->nbBits = 0;
-
-    return 0;
-}
-
-
-size_t FSEv07_buildDTable_raw (FSEv07_DTable* dt, unsigned nbBits)
-{
-    void* ptr = dt;
-    FSEv07_DTableHeader* const DTableH = (FSEv07_DTableHeader*)ptr;
-    void* dPtr = dt + 1;
-    FSEv07_decode_t* const dinfo = (FSEv07_decode_t*)dPtr;
-    const unsigned tableSize = 1 << nbBits;
-    const unsigned tableMask = tableSize - 1;
-    const unsigned maxSV1 = tableMask+1;
-    unsigned s;
-
-    /* Sanity checks */
-    if (nbBits < 1) return ERROR(GENERIC);         /* min size */
-
-    /* Build Decoding Table */
-    DTableH->tableLog = (U16)nbBits;
-    DTableH->fastMode = 1;
-    for (s=0; s<maxSV1; s++) {
-        dinfo[s].newState = 0;
-        dinfo[s].symbol = (BYTE)s;
-        dinfo[s].nbBits = (BYTE)nbBits;
-    }
-
-    return 0;
-}
-
-FORCE_INLINE size_t FSEv07_decompress_usingDTable_generic(
-          void* dst, size_t maxDstSize,
-    const void* cSrc, size_t cSrcSize,
-    const FSEv07_DTable* dt, const unsigned fast)
-{
-    BYTE* const ostart = (BYTE*) dst;
-    BYTE* op = ostart;
-    BYTE* const omax = op + maxDstSize;
-    BYTE* const olimit = omax-3;
-
-    BITv07_DStream_t bitD;
-    FSEv07_DState_t state1;
-    FSEv07_DState_t state2;
-
-    /* Init */
-    { size_t const errorCode = BITv07_initDStream(&bitD, cSrc, cSrcSize);   /* replaced last arg by maxCompressed Size */
-      if (FSEv07_isError(errorCode)) return errorCode; }
-
-    FSEv07_initDState(&state1, &bitD, dt);
-    FSEv07_initDState(&state2, &bitD, dt);
-
-#define FSEv07_GETSYMBOL(statePtr) fast ? FSEv07_decodeSymbolFast(statePtr, &bitD) : FSEv07_decodeSymbol(statePtr, &bitD)
-
-    /* 4 symbols per loop */
-    for ( ; (BITv07_reloadDStream(&bitD)==BITv07_DStream_unfinished) && (op<olimit) ; op+=4) {
-        op[0] = FSEv07_GETSYMBOL(&state1);
-
-        if (FSEv07_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv07_reloadDStream(&bitD);
-
-        op[1] = FSEv07_GETSYMBOL(&state2);
-
-        if (FSEv07_MAX_TABLELOG*4+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            { if (BITv07_reloadDStream(&bitD) > BITv07_DStream_unfinished) { op+=2; break; } }
-
-        op[2] = FSEv07_GETSYMBOL(&state1);
-
-        if (FSEv07_MAX_TABLELOG*2+7 > sizeof(bitD.bitContainer)*8)    /* This test must be static */
-            BITv07_reloadDStream(&bitD);
-
-        op[3] = FSEv07_GETSYMBOL(&state2);
-    }
-
-    /* tail */
-    /* note : BITv07_reloadDStream(&bitD) >= FSEv07_DStream_partiallyFilled; Ends at exactly BITv07_DStream_completed */
-    while (1) {
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-
-        *op++ = FSEv07_GETSYMBOL(&state1);
-
-        if (BITv07_reloadDStream(&bitD)==BITv07_DStream_overflow) {
-            *op++ = FSEv07_GETSYMBOL(&state2);
-            break;
-        }
-
-        if (op>(omax-2)) return ERROR(dstSize_tooSmall);
-
-        *op++ = FSEv07_GETSYMBOL(&state2);
-
-        if (BITv07_reloadDStream(&bitD)==BITv07_DStream_overflow) {
-            *op++ = FSEv07_GETSYMBOL(&state1);
-            break;
-    }   }
-
-    return op-ostart;
-}
-
-
-size_t FSEv07_decompress_usingDTable(void* dst, size_t originalSize,
-                            const void* cSrc, size_t cSrcSize,
-                            const FSEv07_DTable* dt)
-{
-    const void* ptr = dt;
-    const FSEv07_DTableHeader* DTableH = (const FSEv07_DTableHeader*)ptr;
-    const U32 fastMode = DTableH->fastMode;
-
-    /* select fast mode (static) */
-    if (fastMode) return FSEv07_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 1);
-    return FSEv07_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, dt, 0);
-}
-
-
-size_t FSEv07_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* const istart = (const BYTE*)cSrc;
-    const BYTE* ip = istart;
-    short counting[FSEv07_MAX_SYMBOL_VALUE+1];
-    DTable_max_t dt;   /* Static analyzer seems unable to understand this table will be properly initialized later */
-    unsigned tableLog;
-    unsigned maxSymbolValue = FSEv07_MAX_SYMBOL_VALUE;
-
-    if (cSrcSize<2) return ERROR(srcSize_wrong);   /* too small input size */
-
-    /* normal FSE decoding mode */
-    {   size_t const NCountLength = FSEv07_readNCount (counting, &maxSymbolValue, &tableLog, istart, cSrcSize);
-        if (FSEv07_isError(NCountLength)) return NCountLength;
-        if (NCountLength >= cSrcSize) return ERROR(srcSize_wrong);   /* too small input size */
-        ip += NCountLength;
-        cSrcSize -= NCountLength;
-    }
-
-    { size_t const errorCode = FSEv07_buildDTable (dt, counting, maxSymbolValue, tableLog);
-      if (FSEv07_isError(errorCode)) return errorCode; }
-
-    return FSEv07_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, dt);   /* always return, even if it is an error code */
-}
-
-
-
-#endif   /* FSEv07_COMMONDEFS_ONLY */
-
-/* ******************************************************************
-   Huffman decoder, part of New Generation Entropy library
-   Copyright (C) 2013-2016, Yann Collet.
-
-   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-   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
-   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.
-
-    You can contact the author at :
-    - FSE+HUF source repository : https://github.com/Cyan4973/FiniteStateEntropy
-    - Public forum : https://groups.google.com/forum/#!forum/lz4c
-****************************************************************** */
-
-/* **************************************************************
-*  Compiler specifics
-****************************************************************/
-#if defined (__cplusplus) || (defined (__STDC_VERSION__) && (__STDC_VERSION__ >= 199901L) /* C99 */)
-/* inline is defined */
-#elif defined(_MSC_VER)
-#  define inline __inline
-#else
-#  define inline /* disable inline */
-#endif
-
-
-#ifdef _MSC_VER    /* Visual Studio */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#endif
-
-
-
-/* **************************************************************
-*  Error Management
-****************************************************************/
-#define HUFv07_STATIC_ASSERT(c) { enum { HUFv07_static_assert = 1/(int)(!!(c)) }; }   /* use only *after* variable declarations */
-
-
-/*-***************************/
-/*  generic DTableDesc       */
-/*-***************************/
-
-typedef struct { BYTE maxTableLog; BYTE tableType; BYTE tableLog; BYTE reserved; } DTableDesc;
-
-static DTableDesc HUFv07_getDTableDesc(const HUFv07_DTable* table)
-{
-    DTableDesc dtd;
-    memcpy(&dtd, table, sizeof(dtd));
-    return dtd;
-}
-
-
-/*-***************************/
-/*  single-symbol decoding   */
-/*-***************************/
-
-typedef struct { BYTE byte; BYTE nbBits; } HUFv07_DEltX2;   /* single-symbol decoding */
-
-size_t HUFv07_readDTableX2 (HUFv07_DTable* DTable, const void* src, size_t srcSize)
-{
-    BYTE huffWeight[HUFv07_SYMBOLVALUE_MAX + 1];
-    U32 rankVal[HUFv07_TABLELOG_ABSOLUTEMAX + 1];   /* large enough for values from 0 to 16 */
-    U32 tableLog = 0;
-    U32 nbSymbols = 0;
-    size_t iSize;
-    void* const dtPtr = DTable + 1;
-    HUFv07_DEltX2* const dt = (HUFv07_DEltX2*)dtPtr;
-
-    HUFv07_STATIC_ASSERT(sizeof(DTableDesc) == sizeof(HUFv07_DTable));
-    //memset(huffWeight, 0, sizeof(huffWeight));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv07_readStats(huffWeight, HUFv07_SYMBOLVALUE_MAX + 1, rankVal, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv07_isError(iSize)) return iSize;
-
-    /* Table header */
-    {   DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-        if (tableLog > (U32)(dtd.maxTableLog+1)) return ERROR(tableLog_tooLarge);   /* DTable too small, huffman tree cannot fit in */
-        dtd.tableType = 0;
-        dtd.tableLog = (BYTE)tableLog;
-        memcpy(DTable, &dtd, sizeof(dtd));
-    }
-
-    /* Prepare ranks */
-    {   U32 n, nextRankStart = 0;
-        for (n=1; n<tableLog+1; n++) {
-            U32 current = nextRankStart;
-            nextRankStart += (rankVal[n] << (n-1));
-            rankVal[n] = current;
-    }   }
-
-    /* fill DTable */
-    {   U32 n;
-        for (n=0; n<nbSymbols; n++) {
-            U32 const w = huffWeight[n];
-            U32 const length = (1 << w) >> 1;
-            U32 i;
-            HUFv07_DEltX2 D;
-            D.byte = (BYTE)n; D.nbBits = (BYTE)(tableLog + 1 - w);
-            for (i = rankVal[w]; i < rankVal[w] + length; i++)
-                dt[i] = D;
-            rankVal[w] += length;
-    }   }
-
-    return iSize;
-}
-
-
-static BYTE HUFv07_decodeSymbolX2(BITv07_DStream_t* Dstream, const HUFv07_DEltX2* dt, const U32 dtLog)
-{
-    size_t const val = BITv07_lookBitsFast(Dstream, dtLog); /* note : dtLog >= 1 */
-    BYTE const c = dt[val].byte;
-    BITv07_skipBits(Dstream, dt[val].nbBits);
-    return c;
-}
-
-#define HUFv07_DECODE_SYMBOLX2_0(ptr, DStreamPtr) \
-    *ptr++ = HUFv07_decodeSymbolX2(DStreamPtr, dt, dtLog)
-
-#define HUFv07_DECODE_SYMBOLX2_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv07_TABLELOG_MAX<=12)) \
-        HUFv07_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-#define HUFv07_DECODE_SYMBOLX2_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        HUFv07_DECODE_SYMBOLX2_0(ptr, DStreamPtr)
-
-static inline size_t HUFv07_decodeStreamX2(BYTE* p, BITv07_DStream_t* const bitDPtr, BYTE* const pEnd, const HUFv07_DEltX2* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 4 symbols at a time */
-    while ((BITv07_reloadDStream(bitDPtr) == BITv07_DStream_unfinished) && (p <= pEnd-4)) {
-        HUFv07_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX2_1(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX2_2(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX2_0(p, bitDPtr);
-    }
-
-    /* closer to the end */
-    while ((BITv07_reloadDStream(bitDPtr) == BITv07_DStream_unfinished) && (p < pEnd))
-        HUFv07_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    /* no more data to retrieve from bitstream, hence no need to reload */
-    while (p < pEnd)
-        HUFv07_DECODE_SYMBOLX2_0(p, bitDPtr);
-
-    return pEnd-pStart;
-}
-
-static size_t HUFv07_decompress1X2_usingDTable_internal(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + dstSize;
-    const void* dtPtr = DTable + 1;
-    const HUFv07_DEltX2* const dt = (const HUFv07_DEltX2*)dtPtr;
-    BITv07_DStream_t bitD;
-    DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-    U32 const dtLog = dtd.tableLog;
-
-    { size_t const errorCode = BITv07_initDStream(&bitD, cSrc, cSrcSize);
-      if (HUFv07_isError(errorCode)) return errorCode; }
-
-    HUFv07_decodeStreamX2(op, &bitD, oend, dt, dtLog);
-
-    /* check */
-    if (!BITv07_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    return dstSize;
-}
-
-size_t HUFv07_decompress1X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-    if (dtd.tableType != 0) return ERROR(GENERIC);
-    return HUFv07_decompress1X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable);
-}
-
-size_t HUFv07_decompress1X2_DCtx (HUFv07_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUFv07_readDTableX2 (DCtx, cSrc, cSrcSize);
-    if (HUFv07_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUFv07_decompress1X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx);
-}
-
-size_t HUFv07_decompress1X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv07_CREATE_STATIC_DTABLEX2(DTable, HUFv07_TABLELOG_MAX);
-    return HUFv07_decompress1X2_DCtx (DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-
-static size_t HUFv07_decompress4X2_usingDTable_internal(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    /* Check */
-    if (cSrcSize < 10) return ERROR(corruption_detected);  /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable + 1;
-        const HUFv07_DEltX2* const dt = (const HUFv07_DEltX2*)dtPtr;
-
-        /* Init */
-        BITv07_DStream_t bitD1;
-        BITv07_DStream_t bitD2;
-        BITv07_DStream_t bitD3;
-        BITv07_DStream_t bitD4;
-        size_t const length1 = MEM_readLE16(istart);
-        size_t const length2 = MEM_readLE16(istart+2);
-        size_t const length3 = MEM_readLE16(istart+4);
-        size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        const size_t segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-        DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-        U32 const dtLog = dtd.tableLog;
-
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        { size_t const errorCode = BITv07_initDStream(&bitD1, istart1, length1);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD2, istart2, length2);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD3, istart3, length3);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD4, istart4, length4);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv07_reloadDStream(&bitD1) | BITv07_reloadDStream(&bitD2) | BITv07_reloadDStream(&bitD3) | BITv07_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv07_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv07_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX2_1(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX2_1(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX2_1(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX2_1(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX2_2(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX2_2(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX2_2(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX2_2(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX2_0(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX2_0(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX2_0(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX2_0(op4, &bitD4);
-            endSignal = BITv07_reloadDStream(&bitD1) | BITv07_reloadDStream(&bitD2) | BITv07_reloadDStream(&bitD3) | BITv07_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv07_decodeStreamX2(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv07_decodeStreamX2(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv07_decodeStreamX2(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv07_decodeStreamX2(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        endSignal = BITv07_endOfDStream(&bitD1) & BITv07_endOfDStream(&bitD2) & BITv07_endOfDStream(&bitD3) & BITv07_endOfDStream(&bitD4);
-        if (!endSignal) return ERROR(corruption_detected);
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv07_decompress4X2_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-    if (dtd.tableType != 0) return ERROR(GENERIC);
-    return HUFv07_decompress4X2_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable);
-}
-
-
-size_t HUFv07_decompress4X2_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUFv07_readDTableX2 (dctx, cSrc, cSrcSize);
-    if (HUFv07_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUFv07_decompress4X2_usingDTable_internal (dst, dstSize, ip, cSrcSize, dctx);
-}
-
-size_t HUFv07_decompress4X2 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv07_CREATE_STATIC_DTABLEX2(DTable, HUFv07_TABLELOG_MAX);
-    return HUFv07_decompress4X2_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-
-/* *************************/
-/* double-symbols decoding */
-/* *************************/
-typedef struct { U16 sequence; BYTE nbBits; BYTE length; } HUFv07_DEltX4;  /* double-symbols decoding */
-
-typedef struct { BYTE symbol; BYTE weight; } sortedSymbol_t;
-
-static void HUFv07_fillDTableX4Level2(HUFv07_DEltX4* DTable, U32 sizeLog, const U32 consumed,
-                           const U32* rankValOrigin, const int minWeight,
-                           const sortedSymbol_t* sortedSymbols, const U32 sortedListSize,
-                           U32 nbBitsBaseline, U16 baseSeq)
-{
-    HUFv07_DEltX4 DElt;
-    U32 rankVal[HUFv07_TABLELOG_ABSOLUTEMAX + 1];
-
-    /* get pre-calculated rankVal */
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill skipped values */
-    if (minWeight>1) {
-        U32 i, skipSize = rankVal[minWeight];
-        MEM_writeLE16(&(DElt.sequence), baseSeq);
-        DElt.nbBits   = (BYTE)(consumed);
-        DElt.length   = 1;
-        for (i = 0; i < skipSize; i++)
-            DTable[i] = DElt;
-    }
-
-    /* fill DTable */
-    { U32 s; for (s=0; s<sortedListSize; s++) {   /* note : sortedSymbols already skipped */
-        const U32 symbol = sortedSymbols[s].symbol;
-        const U32 weight = sortedSymbols[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 length = 1 << (sizeLog-nbBits);
-        const U32 start = rankVal[weight];
-        U32 i = start;
-        const U32 end = start + length;
-
-        MEM_writeLE16(&(DElt.sequence), (U16)(baseSeq + (symbol << 8)));
-        DElt.nbBits = (BYTE)(nbBits + consumed);
-        DElt.length = 2;
-        do { DTable[i++] = DElt; } while (i<end);   /* since length >= 1 */
-
-        rankVal[weight] += length;
-    }}
-}
-
-typedef U32 rankVal_t[HUFv07_TABLELOG_ABSOLUTEMAX][HUFv07_TABLELOG_ABSOLUTEMAX + 1];
-
-static void HUFv07_fillDTableX4(HUFv07_DEltX4* DTable, const U32 targetLog,
-                           const sortedSymbol_t* sortedList, const U32 sortedListSize,
-                           const U32* rankStart, rankVal_t rankValOrigin, const U32 maxWeight,
-                           const U32 nbBitsBaseline)
-{
-    U32 rankVal[HUFv07_TABLELOG_ABSOLUTEMAX + 1];
-    const int scaleLog = nbBitsBaseline - targetLog;   /* note : targetLog >= srcLog, hence scaleLog <= 1 */
-    const U32 minBits  = nbBitsBaseline - maxWeight;
-    U32 s;
-
-    memcpy(rankVal, rankValOrigin, sizeof(rankVal));
-
-    /* fill DTable */
-    for (s=0; s<sortedListSize; s++) {
-        const U16 symbol = sortedList[s].symbol;
-        const U32 weight = sortedList[s].weight;
-        const U32 nbBits = nbBitsBaseline - weight;
-        const U32 start = rankVal[weight];
-        const U32 length = 1 << (targetLog-nbBits);
-
-        if (targetLog-nbBits >= minBits) {   /* enough room for a second symbol */
-            U32 sortedRank;
-            int minWeight = nbBits + scaleLog;
-            if (minWeight < 1) minWeight = 1;
-            sortedRank = rankStart[minWeight];
-            HUFv07_fillDTableX4Level2(DTable+start, targetLog-nbBits, nbBits,
-                           rankValOrigin[nbBits], minWeight,
-                           sortedList+sortedRank, sortedListSize-sortedRank,
-                           nbBitsBaseline, symbol);
-        } else {
-            HUFv07_DEltX4 DElt;
-            MEM_writeLE16(&(DElt.sequence), symbol);
-            DElt.nbBits = (BYTE)(nbBits);
-            DElt.length = 1;
-            {   U32 u;
-                const U32 end = start + length;
-                for (u = start; u < end; u++) DTable[u] = DElt;
-        }   }
-        rankVal[weight] += length;
-    }
-}
-
-size_t HUFv07_readDTableX4 (HUFv07_DTable* DTable, const void* src, size_t srcSize)
-{
-    BYTE weightList[HUFv07_SYMBOLVALUE_MAX + 1];
-    sortedSymbol_t sortedSymbol[HUFv07_SYMBOLVALUE_MAX + 1];
-    U32 rankStats[HUFv07_TABLELOG_ABSOLUTEMAX + 1] = { 0 };
-    U32 rankStart0[HUFv07_TABLELOG_ABSOLUTEMAX + 2] = { 0 };
-    U32* const rankStart = rankStart0+1;
-    rankVal_t rankVal;
-    U32 tableLog, maxW, sizeOfSort, nbSymbols;
-    DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-    U32 const maxTableLog = dtd.maxTableLog;
-    size_t iSize;
-    void* dtPtr = DTable+1;   /* force compiler to avoid strict-aliasing */
-    HUFv07_DEltX4* const dt = (HUFv07_DEltX4*)dtPtr;
-
-    HUFv07_STATIC_ASSERT(sizeof(HUFv07_DEltX4) == sizeof(HUFv07_DTable));   /* if compilation fails here, assertion is false */
-    if (maxTableLog > HUFv07_TABLELOG_ABSOLUTEMAX) return ERROR(tableLog_tooLarge);
-    //memset(weightList, 0, sizeof(weightList));   /* is not necessary, even though some analyzer complain ... */
-
-    iSize = HUFv07_readStats(weightList, HUFv07_SYMBOLVALUE_MAX + 1, rankStats, &nbSymbols, &tableLog, src, srcSize);
-    if (HUFv07_isError(iSize)) return iSize;
-
-    /* check result */
-    if (tableLog > maxTableLog) return ERROR(tableLog_tooLarge);   /* DTable can't fit code depth */
-
-    /* find maxWeight */
-    for (maxW = tableLog; rankStats[maxW]==0; maxW--) {}  /* necessarily finds a solution before 0 */
-
-    /* Get start index of each weight */
-    {   U32 w, nextRankStart = 0;
-        for (w=1; w<maxW+1; w++) {
-            U32 current = nextRankStart;
-            nextRankStart += rankStats[w];
-            rankStart[w] = current;
-        }
-        rankStart[0] = nextRankStart;   /* put all 0w symbols at the end of sorted list*/
-        sizeOfSort = nextRankStart;
-    }
-
-    /* sort symbols by weight */
-    {   U32 s;
-        for (s=0; s<nbSymbols; s++) {
-            U32 const w = weightList[s];
-            U32 const r = rankStart[w]++;
-            sortedSymbol[r].symbol = (BYTE)s;
-            sortedSymbol[r].weight = (BYTE)w;
-        }
-        rankStart[0] = 0;   /* forget 0w symbols; this is beginning of weight(1) */
-    }
-
-    /* Build rankVal */
-    {   U32* const rankVal0 = rankVal[0];
-        {   int const rescale = (maxTableLog-tableLog) - 1;   /* tableLog <= maxTableLog */
-            U32 nextRankVal = 0;
-            U32 w;
-            for (w=1; w<maxW+1; w++) {
-                U32 current = nextRankVal;
-                nextRankVal += rankStats[w] << (w+rescale);
-                rankVal0[w] = current;
-        }   }
-        {   U32 const minBits = tableLog+1 - maxW;
-            U32 consumed;
-            for (consumed = minBits; consumed < maxTableLog - minBits + 1; consumed++) {
-                U32* const rankValPtr = rankVal[consumed];
-                U32 w;
-                for (w = 1; w < maxW+1; w++) {
-                    rankValPtr[w] = rankVal0[w] >> consumed;
-    }   }   }   }
-
-    HUFv07_fillDTableX4(dt, maxTableLog,
-                   sortedSymbol, sizeOfSort,
-                   rankStart0, rankVal, maxW,
-                   tableLog+1);
-
-    dtd.tableLog = (BYTE)maxTableLog;
-    dtd.tableType = 1;
-    memcpy(DTable, &dtd, sizeof(dtd));
-    return iSize;
-}
-
-
-static U32 HUFv07_decodeSymbolX4(void* op, BITv07_DStream_t* DStream, const HUFv07_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv07_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 2);
-    BITv07_skipBits(DStream, dt[val].nbBits);
-    return dt[val].length;
-}
-
-static U32 HUFv07_decodeLastSymbolX4(void* op, BITv07_DStream_t* DStream, const HUFv07_DEltX4* dt, const U32 dtLog)
-{
-    const size_t val = BITv07_lookBitsFast(DStream, dtLog);   /* note : dtLog >= 1 */
-    memcpy(op, dt+val, 1);
-    if (dt[val].length==1) BITv07_skipBits(DStream, dt[val].nbBits);
-    else {
-        if (DStream->bitsConsumed < (sizeof(DStream->bitContainer)*8)) {
-            BITv07_skipBits(DStream, dt[val].nbBits);
-            if (DStream->bitsConsumed > (sizeof(DStream->bitContainer)*8))
-                DStream->bitsConsumed = (sizeof(DStream->bitContainer)*8);   /* ugly hack; works only because it's the last symbol. Note : can't easily extract nbBits from just this symbol */
-    }   }
-    return 1;
-}
-
-
-#define HUFv07_DECODE_SYMBOLX4_0(ptr, DStreamPtr) \
-    ptr += HUFv07_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv07_DECODE_SYMBOLX4_1(ptr, DStreamPtr) \
-    if (MEM_64bits() || (HUFv07_TABLELOG_MAX<=12)) \
-        ptr += HUFv07_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-#define HUFv07_DECODE_SYMBOLX4_2(ptr, DStreamPtr) \
-    if (MEM_64bits()) \
-        ptr += HUFv07_decodeSymbolX4(ptr, DStreamPtr, dt, dtLog)
-
-static inline size_t HUFv07_decodeStreamX4(BYTE* p, BITv07_DStream_t* bitDPtr, BYTE* const pEnd, const HUFv07_DEltX4* const dt, const U32 dtLog)
-{
-    BYTE* const pStart = p;
-
-    /* up to 8 symbols at a time */
-    while ((BITv07_reloadDStream(bitDPtr) == BITv07_DStream_unfinished) && (p < pEnd-7)) {
-        HUFv07_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX4_1(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX4_2(p, bitDPtr);
-        HUFv07_DECODE_SYMBOLX4_0(p, bitDPtr);
-    }
-
-    /* closer to end : up to 2 symbols at a time */
-    while ((BITv07_reloadDStream(bitDPtr) == BITv07_DStream_unfinished) && (p <= pEnd-2))
-        HUFv07_DECODE_SYMBOLX4_0(p, bitDPtr);
-
-    while (p <= pEnd-2)
-        HUFv07_DECODE_SYMBOLX4_0(p, bitDPtr);   /* no need to reload : reached the end of DStream */
-
-    if (p < pEnd)
-        p += HUFv07_decodeLastSymbolX4(p, bitDPtr, dt, dtLog);
-
-    return p-pStart;
-}
-
-
-static size_t HUFv07_decompress1X4_usingDTable_internal(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    BITv07_DStream_t bitD;
-
-    /* Init */
-    {   size_t const errorCode = BITv07_initDStream(&bitD, cSrc, cSrcSize);
-        if (HUFv07_isError(errorCode)) return errorCode;
-    }
-
-    /* decode */
-    {   BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable+1;   /* force compiler to not use strict-aliasing */
-        const HUFv07_DEltX4* const dt = (const HUFv07_DEltX4*)dtPtr;
-        DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-        HUFv07_decodeStreamX4(ostart, &bitD, oend, dt, dtd.tableLog);
-    }
-
-    /* check */
-    if (!BITv07_endOfDStream(&bitD)) return ERROR(corruption_detected);
-
-    /* decoded size */
-    return dstSize;
-}
-
-size_t HUFv07_decompress1X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-    if (dtd.tableType != 1) return ERROR(GENERIC);
-    return HUFv07_decompress1X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable);
-}
-
-size_t HUFv07_decompress1X4_DCtx (HUFv07_DTable* DCtx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t const hSize = HUFv07_readDTableX4 (DCtx, cSrc, cSrcSize);
-    if (HUFv07_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUFv07_decompress1X4_usingDTable_internal (dst, dstSize, ip, cSrcSize, DCtx);
-}
-
-size_t HUFv07_decompress1X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv07_CREATE_STATIC_DTABLEX4(DTable, HUFv07_TABLELOG_MAX);
-    return HUFv07_decompress1X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-static size_t HUFv07_decompress4X4_usingDTable_internal(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    if (cSrcSize < 10) return ERROR(corruption_detected);   /* strict minimum : jump table + 1 byte per stream */
-
-    {   const BYTE* const istart = (const BYTE*) cSrc;
-        BYTE* const ostart = (BYTE*) dst;
-        BYTE* const oend = ostart + dstSize;
-        const void* const dtPtr = DTable+1;
-        const HUFv07_DEltX4* const dt = (const HUFv07_DEltX4*)dtPtr;
-
-        /* Init */
-        BITv07_DStream_t bitD1;
-        BITv07_DStream_t bitD2;
-        BITv07_DStream_t bitD3;
-        BITv07_DStream_t bitD4;
-        size_t const length1 = MEM_readLE16(istart);
-        size_t const length2 = MEM_readLE16(istart+2);
-        size_t const length3 = MEM_readLE16(istart+4);
-        size_t const length4 = cSrcSize - (length1 + length2 + length3 + 6);
-        const BYTE* const istart1 = istart + 6;  /* jumpTable */
-        const BYTE* const istart2 = istart1 + length1;
-        const BYTE* const istart3 = istart2 + length2;
-        const BYTE* const istart4 = istart3 + length3;
-        size_t const segmentSize = (dstSize+3) / 4;
-        BYTE* const opStart2 = ostart + segmentSize;
-        BYTE* const opStart3 = opStart2 + segmentSize;
-        BYTE* const opStart4 = opStart3 + segmentSize;
-        BYTE* op1 = ostart;
-        BYTE* op2 = opStart2;
-        BYTE* op3 = opStart3;
-        BYTE* op4 = opStart4;
-        U32 endSignal;
-        DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-        U32 const dtLog = dtd.tableLog;
-
-        if (length4 > cSrcSize) return ERROR(corruption_detected);   /* overflow */
-        { size_t const errorCode = BITv07_initDStream(&bitD1, istart1, length1);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD2, istart2, length2);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD3, istart3, length3);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-        { size_t const errorCode = BITv07_initDStream(&bitD4, istart4, length4);
-          if (HUFv07_isError(errorCode)) return errorCode; }
-
-        /* 16-32 symbols per loop (4-8 symbols per stream) */
-        endSignal = BITv07_reloadDStream(&bitD1) | BITv07_reloadDStream(&bitD2) | BITv07_reloadDStream(&bitD3) | BITv07_reloadDStream(&bitD4);
-        for ( ; (endSignal==BITv07_DStream_unfinished) && (op4<(oend-7)) ; ) {
-            HUFv07_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX4_1(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX4_1(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX4_1(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX4_1(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX4_2(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX4_2(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX4_2(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX4_2(op4, &bitD4);
-            HUFv07_DECODE_SYMBOLX4_0(op1, &bitD1);
-            HUFv07_DECODE_SYMBOLX4_0(op2, &bitD2);
-            HUFv07_DECODE_SYMBOLX4_0(op3, &bitD3);
-            HUFv07_DECODE_SYMBOLX4_0(op4, &bitD4);
-
-            endSignal = BITv07_reloadDStream(&bitD1) | BITv07_reloadDStream(&bitD2) | BITv07_reloadDStream(&bitD3) | BITv07_reloadDStream(&bitD4);
-        }
-
-        /* check corruption */
-        if (op1 > opStart2) return ERROR(corruption_detected);
-        if (op2 > opStart3) return ERROR(corruption_detected);
-        if (op3 > opStart4) return ERROR(corruption_detected);
-        /* note : op4 supposed already verified within main loop */
-
-        /* finish bitStreams one by one */
-        HUFv07_decodeStreamX4(op1, &bitD1, opStart2, dt, dtLog);
-        HUFv07_decodeStreamX4(op2, &bitD2, opStart3, dt, dtLog);
-        HUFv07_decodeStreamX4(op3, &bitD3, opStart4, dt, dtLog);
-        HUFv07_decodeStreamX4(op4, &bitD4, oend,     dt, dtLog);
-
-        /* check */
-        { U32 const endCheck = BITv07_endOfDStream(&bitD1) & BITv07_endOfDStream(&bitD2) & BITv07_endOfDStream(&bitD3) & BITv07_endOfDStream(&bitD4);
-          if (!endCheck) return ERROR(corruption_detected); }
-
-        /* decoded size */
-        return dstSize;
-    }
-}
-
-
-size_t HUFv07_decompress4X4_usingDTable(
-          void* dst,  size_t dstSize,
-    const void* cSrc, size_t cSrcSize,
-    const HUFv07_DTable* DTable)
-{
-    DTableDesc dtd = HUFv07_getDTableDesc(DTable);
-    if (dtd.tableType != 1) return ERROR(GENERIC);
-    return HUFv07_decompress4X4_usingDTable_internal(dst, dstSize, cSrc, cSrcSize, DTable);
-}
-
-
-size_t HUFv07_decompress4X4_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    const BYTE* ip = (const BYTE*) cSrc;
-
-    size_t hSize = HUFv07_readDTableX4 (dctx, cSrc, cSrcSize);
-    if (HUFv07_isError(hSize)) return hSize;
-    if (hSize >= cSrcSize) return ERROR(srcSize_wrong);
-    ip += hSize; cSrcSize -= hSize;
-
-    return HUFv07_decompress4X4_usingDTable_internal(dst, dstSize, ip, cSrcSize, dctx);
-}
-
-size_t HUFv07_decompress4X4 (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    HUFv07_CREATE_STATIC_DTABLEX4(DTable, HUFv07_TABLELOG_MAX);
-    return HUFv07_decompress4X4_DCtx(DTable, dst, dstSize, cSrc, cSrcSize);
-}
-
-
-/* ********************************/
-/* Generic decompression selector */
-/* ********************************/
-
-size_t HUFv07_decompress1X_usingDTable(void* dst, size_t maxDstSize,
-                                    const void* cSrc, size_t cSrcSize,
-                                    const HUFv07_DTable* DTable)
-{
-    DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-    return dtd.tableType ? HUFv07_decompress1X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) :
-                           HUFv07_decompress1X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable);
-}
-
-size_t HUFv07_decompress4X_usingDTable(void* dst, size_t maxDstSize,
-                                    const void* cSrc, size_t cSrcSize,
-                                    const HUFv07_DTable* DTable)
-{
-    DTableDesc const dtd = HUFv07_getDTableDesc(DTable);
-    return dtd.tableType ? HUFv07_decompress4X4_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable) :
-                           HUFv07_decompress4X2_usingDTable_internal(dst, maxDstSize, cSrc, cSrcSize, DTable);
-}
-
-
-typedef struct { U32 tableTime; U32 decode256Time; } algo_time_t;
-static const algo_time_t algoTime[16 /* Quantization */][3 /* single, double, quad */] =
-{
-    /* single, double, quad */
-    {{0,0}, {1,1}, {2,2}},  /* Q==0 : impossible */
-    {{0,0}, {1,1}, {2,2}},  /* Q==1 : impossible */
-    {{  38,130}, {1313, 74}, {2151, 38}},   /* Q == 2 : 12-18% */
-    {{ 448,128}, {1353, 74}, {2238, 41}},   /* Q == 3 : 18-25% */
-    {{ 556,128}, {1353, 74}, {2238, 47}},   /* Q == 4 : 25-32% */
-    {{ 714,128}, {1418, 74}, {2436, 53}},   /* Q == 5 : 32-38% */
-    {{ 883,128}, {1437, 74}, {2464, 61}},   /* Q == 6 : 38-44% */
-    {{ 897,128}, {1515, 75}, {2622, 68}},   /* Q == 7 : 44-50% */
-    {{ 926,128}, {1613, 75}, {2730, 75}},   /* Q == 8 : 50-56% */
-    {{ 947,128}, {1729, 77}, {3359, 77}},   /* Q == 9 : 56-62% */
-    {{1107,128}, {2083, 81}, {4006, 84}},   /* Q ==10 : 62-69% */
-    {{1177,128}, {2379, 87}, {4785, 88}},   /* Q ==11 : 69-75% */
-    {{1242,128}, {2415, 93}, {5155, 84}},   /* Q ==12 : 75-81% */
-    {{1349,128}, {2644,106}, {5260,106}},   /* Q ==13 : 81-87% */
-    {{1455,128}, {2422,124}, {4174,124}},   /* Q ==14 : 87-93% */
-    {{ 722,128}, {1891,145}, {1936,146}},   /* Q ==15 : 93-99% */
-};
-
-/** HUFv07_selectDecoder() :
-*   Tells which decoder is likely to decode faster,
-*   based on a set of pre-determined metrics.
-*   @return : 0==HUFv07_decompress4X2, 1==HUFv07_decompress4X4 .
-*   Assumption : 0 < cSrcSize < dstSize <= 128 KB */
-U32 HUFv07_selectDecoder (size_t dstSize, size_t cSrcSize)
-{
-    /* decoder timing evaluation */
-    U32 const Q = (U32)(cSrcSize * 16 / dstSize);   /* Q < 16 since dstSize > cSrcSize */
-    U32 const D256 = (U32)(dstSize >> 8);
-    U32 const DTime0 = algoTime[Q][0].tableTime + (algoTime[Q][0].decode256Time * D256);
-    U32 DTime1 = algoTime[Q][1].tableTime + (algoTime[Q][1].decode256Time * D256);
-    DTime1 += DTime1 >> 3;  /* advantage to algorithm using less memory, for cache eviction */
-
-    return DTime1 < DTime0;
-}
-
-
-typedef size_t (*decompressionAlgo)(void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize);
-
-size_t HUFv07_decompress (void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    static const decompressionAlgo decompress[2] = { HUFv07_decompress4X2, HUFv07_decompress4X4 };
-
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUFv07_selectDecoder(dstSize, cSrcSize);
-        return decompress[algoNb](dst, dstSize, cSrc, cSrcSize);
-    }
-
-    //return HUFv07_decompress4X2(dst, dstSize, cSrc, cSrcSize);   /* multi-streams single-symbol decoding */
-    //return HUFv07_decompress4X4(dst, dstSize, cSrc, cSrcSize);   /* multi-streams double-symbols decoding */
-}
-
-size_t HUFv07_decompress4X_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUFv07_selectDecoder(dstSize, cSrcSize);
-        return algoNb ? HUFv07_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) :
-                        HUFv07_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ;
-    }
-}
-
-size_t HUFv07_decompress4X_hufOnly (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if ((cSrcSize >= dstSize) || (cSrcSize <= 1)) return ERROR(corruption_detected);   /* invalid */
-
-    {   U32 const algoNb = HUFv07_selectDecoder(dstSize, cSrcSize);
-        return algoNb ? HUFv07_decompress4X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) :
-                        HUFv07_decompress4X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ;
-    }
-}
-
-size_t HUFv07_decompress1X_DCtx (HUFv07_DTable* dctx, void* dst, size_t dstSize, const void* cSrc, size_t cSrcSize)
-{
-    /* validation checks */
-    if (dstSize == 0) return ERROR(dstSize_tooSmall);
-    if (cSrcSize > dstSize) return ERROR(corruption_detected);   /* invalid */
-    if (cSrcSize == dstSize) { memcpy(dst, cSrc, dstSize); return dstSize; }   /* not compressed */
-    if (cSrcSize == 1) { memset(dst, *(const BYTE*)cSrc, dstSize); return dstSize; }   /* RLE */
-
-    {   U32 const algoNb = HUFv07_selectDecoder(dstSize, cSrcSize);
-        return algoNb ? HUFv07_decompress1X4_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) :
-                        HUFv07_decompress1X2_DCtx(dctx, dst, dstSize, cSrc, cSrcSize) ;
-    }
-}
-/*
-    Common functions of Zstd compression library
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net/
-*/
-
-
-
-/*-****************************************
-*  ZSTD Error Management
-******************************************/
-/*! ZSTDv07_isError() :
-*   tells if a return value is an error code */
-unsigned ZSTDv07_isError(size_t code) { return ERR_isError(code); }
-
-/*! ZSTDv07_getErrorName() :
-*   provides error code string from function result (useful for debugging) */
-const char* ZSTDv07_getErrorName(size_t code) { return ERR_getErrorName(code); }
-
-
-
-/* **************************************************************
-*  ZBUFF Error Management
-****************************************************************/
-unsigned ZBUFFv07_isError(size_t errorCode) { return ERR_isError(errorCode); }
-
-const char* ZBUFFv07_getErrorName(size_t errorCode) { return ERR_getErrorName(errorCode); }
-
-
-
-static void* ZSTDv07_defaultAllocFunction(void* opaque, size_t size)
-{
-    void* address = malloc(size);
-    (void)opaque;
-    /* printf("alloc %p, %d opaque=%p \n", address, (int)size, opaque); */
-    return address;
-}
-
-static void ZSTDv07_defaultFreeFunction(void* opaque, void* address)
-{
-    (void)opaque;
-    /* if (address) printf("free %p opaque=%p \n", address, opaque); */
-    free(address);
-}
-/*
-    zstd_internal - common functions to include
-    Header File for include
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : https://www.zstd.net
-*/
-#ifndef ZSTDv07_CCOMMON_H_MODULE
-#define ZSTDv07_CCOMMON_H_MODULE
-
-
-/*-*************************************
-*  Common macros
-***************************************/
-#define MIN(a,b) ((a)<(b) ? (a) : (b))
-#define MAX(a,b) ((a)>(b) ? (a) : (b))
-
-
-/*-*************************************
-*  Common constants
-***************************************/
-#define ZSTDv07_OPT_NUM    (1<<12)
-#define ZSTDv07_DICT_MAGIC  0xEC30A437   /* v0.7 */
-
-#define ZSTDv07_REP_NUM    3
-#define ZSTDv07_REP_INIT   ZSTDv07_REP_NUM
-#define ZSTDv07_REP_MOVE   (ZSTDv07_REP_NUM-1)
-static const U32 repStartValue[ZSTDv07_REP_NUM] = { 1, 4, 8 };
-
-#define KB *(1 <<10)
-#define MB *(1 <<20)
-#define GB *(1U<<30)
-
-#define BIT7 128
-#define BIT6  64
-#define BIT5  32
-#define BIT4  16
-#define BIT1   2
-#define BIT0   1
-
-#define ZSTDv07_WINDOWLOG_ABSOLUTEMIN 10
-static const size_t ZSTDv07_fcs_fieldSize[4] = { 0, 2, 4, 8 };
-static const size_t ZSTDv07_did_fieldSize[4] = { 0, 1, 2, 4 };
-
-#define ZSTDv07_BLOCKHEADERSIZE 3   /* C standard doesn't allow `static const` variable to be init using another `static const` variable */
-static const size_t ZSTDv07_blockHeaderSize = ZSTDv07_BLOCKHEADERSIZE;
-typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t;
-
-#define MIN_SEQUENCES_SIZE 1 /* nbSeq==0 */
-#define MIN_CBLOCK_SIZE (1 /*litCSize*/ + 1 /* RLE or RAW */ + MIN_SEQUENCES_SIZE /* nbSeq==0 */)   /* for a non-null block */
-
-#define HufLog 12
-typedef enum { lbt_huffman, lbt_repeat, lbt_raw, lbt_rle } litBlockType_t;
-
-#define LONGNBSEQ 0x7F00
-
-#define MINMATCH 3
-#define EQUAL_READ32 4
-
-#define Litbits  8
-#define MaxLit ((1<<Litbits) - 1)
-#define MaxML  52
-#define MaxLL  35
-#define MaxOff 28
-#define MaxSeq MAX(MaxLL, MaxML)   /* Assumption : MaxOff < MaxLL,MaxML */
-#define MLFSELog    9
-#define LLFSELog    9
-#define OffFSELog   8
-
-#define FSEv07_ENCODING_RAW     0
-#define FSEv07_ENCODING_RLE     1
-#define FSEv07_ENCODING_STATIC  2
-#define FSEv07_ENCODING_DYNAMIC 3
-
-#define ZSTD_CONTENTSIZE_ERROR   (0ULL - 2)
-
-static const U32 LL_bits[MaxLL+1] = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3, 4, 6, 7, 8, 9,10,11,12,
-                                     13,14,15,16 };
-static const S16 LL_defaultNorm[MaxLL+1] = { 4, 3, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1,
-                                             2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 2, 1, 1, 1, 1, 1,
-                                            -1,-1,-1,-1 };
-static const U32 LL_defaultNormLog = 6;
-
-static const U32 ML_bits[MaxML+1] = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-                                      1, 1, 1, 1, 2, 2, 3, 3, 4, 4, 5, 7, 8, 9,10,11,
-                                     12,13,14,15,16 };
-static const S16 ML_defaultNorm[MaxML+1] = { 1, 4, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
-                                             1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,-1,-1,
-                                            -1,-1,-1,-1,-1 };
-static const U32 ML_defaultNormLog = 6;
-
-static const S16 OF_defaultNorm[MaxOff+1] = { 1, 1, 1, 1, 1, 1, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
-                                              1, 1, 1, 1, 1, 1, 1, 1,-1,-1,-1,-1,-1 };
-static const U32 OF_defaultNormLog = 5;
-
-
-/*-*******************************************
-*  Shared functions to include for inlining
-*********************************************/
-static void ZSTDv07_copy8(void* dst, const void* src) { memcpy(dst, src, 8); }
-#define COPY8(d,s) { ZSTDv07_copy8(d,s); d+=8; s+=8; }
-
-/*! ZSTDv07_wildcopy() :
-*   custom version of memcpy(), can copy up to 7 bytes too many (8 bytes if length==0) */
-#define WILDCOPY_OVERLENGTH 8
-MEM_STATIC void ZSTDv07_wildcopy(void* dst, const void* src, ptrdiff_t length)
-{
-    const BYTE* ip = (const BYTE*)src;
-    BYTE* op = (BYTE*)dst;
-    BYTE* const oend = op + length;
-    do
-        COPY8(op, ip)
-    while (op < oend);
-}
-
-
-/*-*******************************************
-*  Private interfaces
-*********************************************/
-typedef struct ZSTDv07_stats_s ZSTDv07_stats_t;
-
-typedef struct {
-    U32 off;
-    U32 len;
-} ZSTDv07_match_t;
-
-typedef struct {
-    U32 price;
-    U32 off;
-    U32 mlen;
-    U32 litlen;
-    U32 rep[ZSTDv07_REP_INIT];
-} ZSTDv07_optimal_t;
-
-struct ZSTDv07_stats_s { U32 unused; };
-
-typedef struct {
-    void* buffer;
-    U32*  offsetStart;
-    U32*  offset;
-    BYTE* offCodeStart;
-    BYTE* litStart;
-    BYTE* lit;
-    U16*  litLengthStart;
-    U16*  litLength;
-    BYTE* llCodeStart;
-    U16*  matchLengthStart;
-    U16*  matchLength;
-    BYTE* mlCodeStart;
-    U32   longLengthID;   /* 0 == no longLength; 1 == Lit.longLength; 2 == Match.longLength; */
-    U32   longLengthPos;
-    /* opt */
-    ZSTDv07_optimal_t* priceTable;
-    ZSTDv07_match_t* matchTable;
-    U32* matchLengthFreq;
-    U32* litLengthFreq;
-    U32* litFreq;
-    U32* offCodeFreq;
-    U32  matchLengthSum;
-    U32  matchSum;
-    U32  litLengthSum;
-    U32  litSum;
-    U32  offCodeSum;
-    U32  log2matchLengthSum;
-    U32  log2matchSum;
-    U32  log2litLengthSum;
-    U32  log2litSum;
-    U32  log2offCodeSum;
-    U32  factor;
-    U32  cachedPrice;
-    U32  cachedLitLength;
-    const BYTE* cachedLiterals;
-    ZSTDv07_stats_t stats;
-} seqStore_t;
-
-void ZSTDv07_seqToCodes(const seqStore_t* seqStorePtr, size_t const nbSeq);
-
-/* custom memory allocation functions */
-static const ZSTDv07_customMem defaultCustomMem = { ZSTDv07_defaultAllocFunction, ZSTDv07_defaultFreeFunction, NULL };
-
-#endif   /* ZSTDv07_CCOMMON_H_MODULE */
-/*
-    zstd - standard compression library
-    Copyright (C) 2014-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net
-*/
-
-/* ***************************************************************
-*  Tuning parameters
-*****************************************************************/
-/*!
- * HEAPMODE :
- * Select how default decompression function ZSTDv07_decompress() will allocate memory,
- * in memory stack (0), or in memory heap (1, requires malloc())
- */
-#ifndef ZSTDv07_HEAPMODE
-#  define ZSTDv07_HEAPMODE 1
-#endif
-
-
-/*-*******************************************************
-*  Compiler specifics
-*********************************************************/
-#ifdef _MSC_VER    /* Visual Studio */
-#  include <intrin.h>                    /* For Visual 2005 */
-#  pragma warning(disable : 4127)        /* disable: C4127: conditional expression is constant */
-#  pragma warning(disable : 4324)        /* disable: C4324: padded structure */
-#  pragma warning(disable : 4100)        /* disable: C4100: unreferenced formal parameter */
-#endif
-
-
-/*-*************************************
-*  Macros
-***************************************/
-#define ZSTDv07_isError ERR_isError   /* for inlining */
-#define FSEv07_isError  ERR_isError
-#define HUFv07_isError  ERR_isError
-
-
-/*_*******************************************************
-*  Memory operations
-**********************************************************/
-static void ZSTDv07_copy4(void* dst, const void* src) { memcpy(dst, src, 4); }
-
-
-/*-*************************************************************
-*   Context management
-***************************************************************/
-typedef enum { ZSTDds_getFrameHeaderSize, ZSTDds_decodeFrameHeader,
-               ZSTDds_decodeBlockHeader, ZSTDds_decompressBlock,
-               ZSTDds_decodeSkippableHeader, ZSTDds_skipFrame } ZSTDv07_dStage;
-
-struct ZSTDv07_DCtx_s
-{
-    FSEv07_DTable LLTable[FSEv07_DTABLE_SIZE_U32(LLFSELog)];
-    FSEv07_DTable OffTable[FSEv07_DTABLE_SIZE_U32(OffFSELog)];
-    FSEv07_DTable MLTable[FSEv07_DTABLE_SIZE_U32(MLFSELog)];
-    HUFv07_DTable hufTable[HUFv07_DTABLE_SIZE(HufLog)];  /* can accommodate HUFv07_decompress4X */
-    const void* previousDstEnd;
-    const void* base;
-    const void* vBase;
-    const void* dictEnd;
-    size_t expected;
-    U32 rep[3];
-    ZSTDv07_frameParams fParams;
-    blockType_t bType;   /* used in ZSTDv07_decompressContinue(), to transfer blockType between header decoding and block decoding stages */
-    ZSTDv07_dStage stage;
-    U32 litEntropy;
-    U32 fseEntropy;
-    XXH64_state_t xxhState;
-    size_t headerSize;
-    U32 dictID;
-    const BYTE* litPtr;
-    ZSTDv07_customMem customMem;
-    size_t litSize;
-    BYTE litBuffer[ZSTDv07_BLOCKSIZE_ABSOLUTEMAX + WILDCOPY_OVERLENGTH];
-    BYTE headerBuffer[ZSTDv07_FRAMEHEADERSIZE_MAX];
-};  /* typedef'd to ZSTDv07_DCtx within "zstd_static.h" */
-
-int ZSTDv07_isSkipFrame(ZSTDv07_DCtx* dctx);
-
-size_t ZSTDv07_sizeofDCtx (const ZSTDv07_DCtx* dctx) { return sizeof(*dctx); }
-
-size_t ZSTDv07_estimateDCtxSize(void) { return sizeof(ZSTDv07_DCtx); }
-
-size_t ZSTDv07_decompressBegin(ZSTDv07_DCtx* dctx)
-{
-    dctx->expected = ZSTDv07_frameHeaderSize_min;
-    dctx->stage = ZSTDds_getFrameHeaderSize;
-    dctx->previousDstEnd = NULL;
-    dctx->base = NULL;
-    dctx->vBase = NULL;
-    dctx->dictEnd = NULL;
-    dctx->hufTable[0] = (HUFv07_DTable)((HufLog)*0x1000001);
-    dctx->litEntropy = dctx->fseEntropy = 0;
-    dctx->dictID = 0;
-    { int i; for (i=0; i<ZSTDv07_REP_NUM; i++) dctx->rep[i] = repStartValue[i]; }
-    return 0;
-}
-
-ZSTDv07_DCtx* ZSTDv07_createDCtx_advanced(ZSTDv07_customMem customMem)
-{
-    ZSTDv07_DCtx* dctx;
-
-    if (!customMem.customAlloc && !customMem.customFree)
-        customMem = defaultCustomMem;
-
-    if (!customMem.customAlloc || !customMem.customFree)
-        return NULL;
-
-    dctx = (ZSTDv07_DCtx*) customMem.customAlloc(customMem.opaque, sizeof(ZSTDv07_DCtx));
-    if (!dctx) return NULL;
-    memcpy(&dctx->customMem, &customMem, sizeof(ZSTDv07_customMem));
-    ZSTDv07_decompressBegin(dctx);
-    return dctx;
-}
-
-ZSTDv07_DCtx* ZSTDv07_createDCtx(void)
-{
-    return ZSTDv07_createDCtx_advanced(defaultCustomMem);
-}
-
-size_t ZSTDv07_freeDCtx(ZSTDv07_DCtx* dctx)
-{
-    if (dctx==NULL) return 0;   /* support free on NULL */
-    dctx->customMem.customFree(dctx->customMem.opaque, dctx);
-    return 0;   /* reserved as a potential error code in the future */
-}
-
-void ZSTDv07_copyDCtx(ZSTDv07_DCtx* dstDCtx, const ZSTDv07_DCtx* srcDCtx)
-{
-    memcpy(dstDCtx, srcDCtx,
-           sizeof(ZSTDv07_DCtx) - (ZSTDv07_BLOCKSIZE_ABSOLUTEMAX+WILDCOPY_OVERLENGTH + ZSTDv07_frameHeaderSize_max));  /* no need to copy workspace */
-}
-
-
-/*-*************************************************************
-*   Decompression section
-***************************************************************/
-
-/* Frame format description
-   Frame Header -  [ Block Header - Block ] - Frame End
-   1) Frame Header
-      - 4 bytes - Magic Number : ZSTDv07_MAGICNUMBER (defined within zstd.h)
-      - 1 byte  - Frame Descriptor
-   2) Block Header
-      - 3 bytes, starting with a 2-bits descriptor
-                 Uncompressed, Compressed, Frame End, unused
-   3) Block
-      See Block Format Description
-   4) Frame End
-      - 3 bytes, compatible with Block Header
-*/
-
-
-/* Frame Header :
-
-   1 byte - FrameHeaderDescription :
-   bit 0-1 : dictID (0, 1, 2 or 4 bytes)
-   bit 2   : checksumFlag
-   bit 3   : reserved (must be zero)
-   bit 4   : reserved (unused, can be any value)
-   bit 5   : Single Segment (if 1, WindowLog byte is not present)
-   bit 6-7 : FrameContentFieldSize (0, 2, 4, or 8)
-             if (SkippedWindowLog && !FrameContentFieldsize) FrameContentFieldsize=1;
-
-   Optional : WindowLog (0 or 1 byte)
-   bit 0-2 : octal Fractional (1/8th)
-   bit 3-7 : Power of 2, with 0 = 1 KB (up to 2 TB)
-
-   Optional : dictID (0, 1, 2 or 4 bytes)
-   Automatic adaptation
-   0 : no dictID
-   1 : 1 - 255
-   2 : 256 - 65535
-   4 : all other values
-
-   Optional : content size (0, 1, 2, 4 or 8 bytes)
-   0 : unknown          (fcfs==0 and swl==0)
-   1 : 0-255 bytes      (fcfs==0 and swl==1)
-   2 : 256 - 65535+256  (fcfs==1)
-   4 : 0 - 4GB-1        (fcfs==2)
-   8 : 0 - 16EB-1       (fcfs==3)
-*/
-
-
-/* Compressed Block, format description
-
-   Block = Literal Section - Sequences Section
-   Prerequisite : size of (compressed) block, maximum size of regenerated data
-
-   1) Literal Section
-
-   1.1) Header : 1-5 bytes
-        flags: 2 bits
-            00 compressed by Huff0
-            01 unused
-            10 is Raw (uncompressed)
-            11 is Rle
-            Note : using 01 => Huff0 with precomputed table ?
-            Note : delta map ? => compressed ?
-
-   1.1.1) Huff0-compressed literal block : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-   1.1.2) Raw (uncompressed) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RAW<<6) + (0<<4) + size
-               12 bits: (IS_RAW<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RAW<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.3) Rle (repeated single byte) literal block header : 1-3 bytes
-        size :  5 bits: (IS_RLE<<6) + (0<<4) + size
-               12 bits: (IS_RLE<<6) + (2<<4) + (size>>8)
-                        size&255
-               20 bits: (IS_RLE<<6) + (3<<4) + (size>>16)
-                        size>>8&255
-                        size&255
-
-   1.1.4) Huff0-compressed literal block, using precomputed CTables : 3-5 bytes
-            srcSize < 1 KB => 3 bytes (2-2-10-10) => single stream
-            srcSize < 1 KB => 3 bytes (2-2-10-10)
-            srcSize < 16KB => 4 bytes (2-2-14-14)
-            else           => 5 bytes (2-2-18-18)
-            big endian convention
-
-        1- CTable available (stored into workspace ?)
-        2- Small input (fast heuristic ? Full comparison ? depend on clevel ?)
-
-
-   1.2) Literal block content
-
-   1.2.1) Huff0 block, using sizes from header
-        See Huff0 format
-
-   1.2.2) Huff0 block, using prepared table
-
-   1.2.3) Raw content
-
-   1.2.4) single byte
-
-
-   2) Sequences section
-      TO DO
-*/
-
-/** ZSTDv07_frameHeaderSize() :
-*   srcSize must be >= ZSTDv07_frameHeaderSize_min.
-*   @return : size of the Frame Header */
-static size_t ZSTDv07_frameHeaderSize(const void* src, size_t srcSize)
-{
-    if (srcSize < ZSTDv07_frameHeaderSize_min) return ERROR(srcSize_wrong);
-    {   BYTE const fhd = ((const BYTE*)src)[4];
-        U32 const dictID= fhd & 3;
-        U32 const directMode = (fhd >> 5) & 1;
-        U32 const fcsId = fhd >> 6;
-        return ZSTDv07_frameHeaderSize_min + !directMode + ZSTDv07_did_fieldSize[dictID] + ZSTDv07_fcs_fieldSize[fcsId]
-                + (directMode && !ZSTDv07_fcs_fieldSize[fcsId]);
-    }
-}
-
-
-/** ZSTDv07_getFrameParams() :
-*   decode Frame Header, or require larger `srcSize`.
-*   @return : 0, `fparamsPtr` is correctly filled,
-*            >0, `srcSize` is too small, result is expected `srcSize`,
-*             or an error code, which can be tested using ZSTDv07_isError() */
-size_t ZSTDv07_getFrameParams(ZSTDv07_frameParams* fparamsPtr, const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-
-    if (srcSize < ZSTDv07_frameHeaderSize_min) return ZSTDv07_frameHeaderSize_min;
-    memset(fparamsPtr, 0, sizeof(*fparamsPtr));
-    if (MEM_readLE32(src) != ZSTDv07_MAGICNUMBER) {
-        if ((MEM_readLE32(src) & 0xFFFFFFF0U) == ZSTDv07_MAGIC_SKIPPABLE_START) {
-            if (srcSize < ZSTDv07_skippableHeaderSize) return ZSTDv07_skippableHeaderSize; /* magic number + skippable frame length */
-            fparamsPtr->frameContentSize = MEM_readLE32((const char *)src + 4);
-            fparamsPtr->windowSize = 0; /* windowSize==0 means a frame is skippable */
-            return 0;
-        }
-        return ERROR(prefix_unknown);
-    }
-
-    /* ensure there is enough `srcSize` to fully read/decode frame header */
-    { size_t const fhsize = ZSTDv07_frameHeaderSize(src, srcSize);
-      if (srcSize < fhsize) return fhsize; }
-
-    {   BYTE const fhdByte = ip[4];
-        size_t pos = 5;
-        U32 const dictIDSizeCode = fhdByte&3;
-        U32 const checksumFlag = (fhdByte>>2)&1;
-        U32 const directMode = (fhdByte>>5)&1;
-        U32 const fcsID = fhdByte>>6;
-        U32 const windowSizeMax = 1U << ZSTDv07_WINDOWLOG_MAX;
-        U32 windowSize = 0;
-        U32 dictID = 0;
-        U64 frameContentSize = 0;
-        if ((fhdByte & 0x08) != 0)   /* reserved bits, which must be zero */
-            return ERROR(frameParameter_unsupported);
-        if (!directMode) {
-            BYTE const wlByte = ip[pos++];
-            U32 const windowLog = (wlByte >> 3) + ZSTDv07_WINDOWLOG_ABSOLUTEMIN;
-            if (windowLog > ZSTDv07_WINDOWLOG_MAX)
-                return ERROR(frameParameter_unsupported);
-            windowSize = (1U << windowLog);
-            windowSize += (windowSize >> 3) * (wlByte&7);
-        }
-
-        switch(dictIDSizeCode)
-        {
-            default:   /* impossible */
-            case 0 : break;
-            case 1 : dictID = ip[pos]; pos++; break;
-            case 2 : dictID = MEM_readLE16(ip+pos); pos+=2; break;
-            case 3 : dictID = MEM_readLE32(ip+pos); pos+=4; break;
-        }
-        switch(fcsID)
-        {
-            default:   /* impossible */
-            case 0 : if (directMode) frameContentSize = ip[pos]; break;
-            case 1 : frameContentSize = MEM_readLE16(ip+pos)+256; break;
-            case 2 : frameContentSize = MEM_readLE32(ip+pos); break;
-            case 3 : frameContentSize = MEM_readLE64(ip+pos); break;
-        }
-        if (!windowSize) windowSize = (U32)frameContentSize;
-        if (windowSize > windowSizeMax)
-            return ERROR(frameParameter_unsupported);
-        fparamsPtr->frameContentSize = frameContentSize;
-        fparamsPtr->windowSize = windowSize;
-        fparamsPtr->dictID = dictID;
-        fparamsPtr->checksumFlag = checksumFlag;
-    }
-    return 0;
-}
-
-
-/** ZSTDv07_getDecompressedSize() :
-*   compatible with legacy mode
-*   @return : decompressed size if known, 0 otherwise
-              note : 0 can mean any of the following :
-                   - decompressed size is not provided within frame header
-                   - frame header unknown / not supported
-                   - frame header not completely provided (`srcSize` too small) */
-unsigned long long ZSTDv07_getDecompressedSize(const void* src, size_t srcSize)
-{
-    ZSTDv07_frameParams fparams;
-    size_t const frResult = ZSTDv07_getFrameParams(&fparams, src, srcSize);
-    if (frResult!=0) return 0;
-    return fparams.frameContentSize;
-}
-
-
-/** ZSTDv07_decodeFrameHeader() :
-*   `srcSize` must be the size provided by ZSTDv07_frameHeaderSize().
-*   @return : 0 if success, or an error code, which can be tested using ZSTDv07_isError() */
-static size_t ZSTDv07_decodeFrameHeader(ZSTDv07_DCtx* dctx, const void* src, size_t srcSize)
-{
-    size_t const result = ZSTDv07_getFrameParams(&(dctx->fParams), src, srcSize);
-    if (dctx->fParams.dictID && (dctx->dictID != dctx->fParams.dictID)) return ERROR(dictionary_wrong);
-    if (dctx->fParams.checksumFlag) XXH64_reset(&dctx->xxhState, 0);
-    return result;
-}
-
-
-typedef struct
-{
-    blockType_t blockType;
-    U32 origSize;
-} blockProperties_t;
-
-/*! ZSTDv07_getcBlockSize() :
-*   Provides the size of compressed block from block header `src` */
-static size_t ZSTDv07_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr)
-{
-    const BYTE* const in = (const BYTE* const)src;
-    U32 cSize;
-
-    if (srcSize < ZSTDv07_blockHeaderSize) return ERROR(srcSize_wrong);
-
-    bpPtr->blockType = (blockType_t)((*in) >> 6);
-    cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16);
-    bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0;
-
-    if (bpPtr->blockType == bt_end) return 0;
-    if (bpPtr->blockType == bt_rle) return 1;
-    return cSize;
-}
-
-
-static size_t ZSTDv07_copyRawBlock(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    if (srcSize > dstCapacity) return ERROR(dstSize_tooSmall);
-    memcpy(dst, src, srcSize);
-    return srcSize;
-}
-
-
-/*! ZSTDv07_decodeLiteralsBlock() :
-    @return : nb of bytes read from src (< srcSize ) */
-static size_t ZSTDv07_decodeLiteralsBlock(ZSTDv07_DCtx* dctx,
-                          const void* src, size_t srcSize)   /* note : srcSize < BLOCKSIZE */
-{
-    const BYTE* const istart = (const BYTE*) src;
-
-    if (srcSize < MIN_CBLOCK_SIZE) return ERROR(corruption_detected);
-
-    switch((litBlockType_t)(istart[0]>> 6))
-    {
-    case lbt_huffman:
-        {   size_t litSize, litCSize, singleStream=0;
-            U32 lhSize = (istart[0] >> 4) & 3;
-            if (srcSize < 5) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need up to 5 for lhSize, + cSize (+nbSeq) */
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                /* 2 - 2 - 10 - 10 */
-                lhSize=3;
-                singleStream = istart[0] & 16;
-                litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-                litCSize = ((istart[1] &  3) << 8) + istart[2];
-                break;
-            case 2:
-                /* 2 - 2 - 14 - 14 */
-                lhSize=4;
-                litSize  = ((istart[0] & 15) << 10) + (istart[1] << 2) + (istart[2] >> 6);
-                litCSize = ((istart[2] & 63) <<  8) + istart[3];
-                break;
-            case 3:
-                /* 2 - 2 - 18 - 18 */
-                lhSize=5;
-                litSize  = ((istart[0] & 15) << 14) + (istart[1] << 6) + (istart[2] >> 2);
-                litCSize = ((istart[2] &  3) << 16) + (istart[3] << 8) + istart[4];
-                break;
-            }
-            if (litSize > ZSTDv07_BLOCKSIZE_ABSOLUTEMAX) return ERROR(corruption_detected);
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            if (HUFv07_isError(singleStream ?
-                            HUFv07_decompress1X2_DCtx(dctx->hufTable, dctx->litBuffer, litSize, istart+lhSize, litCSize) :
-                            HUFv07_decompress4X_hufOnly (dctx->hufTable, dctx->litBuffer, litSize, istart+lhSize, litCSize) ))
-                return ERROR(corruption_detected);
-
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            dctx->litEntropy = 1;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case lbt_repeat:
-        {   size_t litSize, litCSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            if (lhSize != 1)  /* only case supported for now : small litSize, single stream */
-                return ERROR(corruption_detected);
-            if (dctx->litEntropy==0)
-                return ERROR(dictionary_corrupted);
-
-            /* 2 - 2 - 10 - 10 */
-            lhSize=3;
-            litSize  = ((istart[0] & 15) << 6) + (istart[1] >> 2);
-            litCSize = ((istart[1] &  3) << 8) + istart[2];
-            if (litCSize + lhSize > srcSize) return ERROR(corruption_detected);
-
-            {   size_t const errorCode = HUFv07_decompress1X4_usingDTable(dctx->litBuffer, litSize, istart+lhSize, litCSize, dctx->hufTable);
-                if (HUFv07_isError(errorCode)) return ERROR(corruption_detected);
-            }
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-            return litCSize + lhSize;
-        }
-    case lbt_raw:
-        {   size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize=1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                break;
-            }
-
-            if (lhSize+litSize+WILDCOPY_OVERLENGTH > srcSize) {  /* risk reading beyond src buffer with wildcopy */
-                if (litSize+lhSize > srcSize) return ERROR(corruption_detected);
-                memcpy(dctx->litBuffer, istart+lhSize, litSize);
-                dctx->litPtr = dctx->litBuffer;
-                dctx->litSize = litSize;
-                memset(dctx->litBuffer + dctx->litSize, 0, WILDCOPY_OVERLENGTH);
-                return lhSize+litSize;
-            }
-            /* direct reference into compressed stream */
-            dctx->litPtr = istart+lhSize;
-            dctx->litSize = litSize;
-            return lhSize+litSize;
-        }
-    case lbt_rle:
-        {   size_t litSize;
-            U32 lhSize = ((istart[0]) >> 4) & 3;
-            switch(lhSize)
-            {
-            case 0: case 1: default:   /* note : default is impossible, since lhSize into [0..3] */
-                lhSize = 1;
-                litSize = istart[0] & 31;
-                break;
-            case 2:
-                litSize = ((istart[0] & 15) << 8) + istart[1];
-                break;
-            case 3:
-                litSize = ((istart[0] & 15) << 16) + (istart[1] << 8) + istart[2];
-                if (srcSize<4) return ERROR(corruption_detected);   /* srcSize >= MIN_CBLOCK_SIZE == 3; here we need lhSize+1 = 4 */
-                break;
-            }
-            if (litSize > ZSTDv07_BLOCKSIZE_ABSOLUTEMAX) return ERROR(corruption_detected);
-            memset(dctx->litBuffer, istart[lhSize], litSize + WILDCOPY_OVERLENGTH);
-            dctx->litPtr = dctx->litBuffer;
-            dctx->litSize = litSize;
-            return lhSize+1;
-        }
-    default:
-        return ERROR(corruption_detected);   /* impossible */
-    }
-}
-
-
-/*! ZSTDv07_buildSeqTable() :
-    @return : nb bytes read from src,
-              or an error code if it fails, testable with ZSTDv07_isError()
-*/
-static size_t ZSTDv07_buildSeqTable(FSEv07_DTable* DTable, U32 type, U32 max, U32 maxLog,
-                                 const void* src, size_t srcSize,
-                                 const S16* defaultNorm, U32 defaultLog, U32 flagRepeatTable)
-{
-    switch(type)
-    {
-    case FSEv07_ENCODING_RLE :
-        if (!srcSize) return ERROR(srcSize_wrong);
-        if ( (*(const BYTE*)src) > max) return ERROR(corruption_detected);
-        FSEv07_buildDTable_rle(DTable, *(const BYTE*)src);   /* if *src > max, data is corrupted */
-        return 1;
-    case FSEv07_ENCODING_RAW :
-        FSEv07_buildDTable(DTable, defaultNorm, max, defaultLog);
-        return 0;
-    case FSEv07_ENCODING_STATIC:
-        if (!flagRepeatTable) return ERROR(corruption_detected);
-        return 0;
-    default :   /* impossible */
-    case FSEv07_ENCODING_DYNAMIC :
-        {   U32 tableLog;
-            S16 norm[MaxSeq+1];
-            size_t const headerSize = FSEv07_readNCount(norm, &max, &tableLog, src, srcSize);
-            if (FSEv07_isError(headerSize)) return ERROR(corruption_detected);
-            if (tableLog > maxLog) return ERROR(corruption_detected);
-            FSEv07_buildDTable(DTable, norm, max, tableLog);
-            return headerSize;
-    }   }
-}
-
-
-static size_t ZSTDv07_decodeSeqHeaders(int* nbSeqPtr,
-                             FSEv07_DTable* DTableLL, FSEv07_DTable* DTableML, FSEv07_DTable* DTableOffb, U32 flagRepeatTable,
-                             const void* src, size_t srcSize)
-{
-    const BYTE* const istart = (const BYTE* const)src;
-    const BYTE* const iend = istart + srcSize;
-    const BYTE* ip = istart;
-
-    /* check */
-    if (srcSize < MIN_SEQUENCES_SIZE) return ERROR(srcSize_wrong);
-
-    /* SeqHead */
-    {   int nbSeq = *ip++;
-        if (!nbSeq) { *nbSeqPtr=0; return 1; }
-        if (nbSeq > 0x7F) {
-            if (nbSeq == 0xFF) {
-                if (ip+2 > iend) return ERROR(srcSize_wrong);
-                nbSeq = MEM_readLE16(ip) + LONGNBSEQ, ip+=2;
-            } else {
-                if (ip >= iend) return ERROR(srcSize_wrong);
-                nbSeq = ((nbSeq-0x80)<<8) + *ip++;
-            }
-        }
-        *nbSeqPtr = nbSeq;
-    }
-
-    /* FSE table descriptors */
-    if (ip + 4 > iend) return ERROR(srcSize_wrong); /* min : header byte + all 3 are "raw", hence no header, but at least xxLog bits per type */
-    {   U32 const LLtype  = *ip >> 6;
-        U32 const OFtype = (*ip >> 4) & 3;
-        U32 const MLtype  = (*ip >> 2) & 3;
-        ip++;
-
-        /* Build DTables */
-        {   size_t const llhSize = ZSTDv07_buildSeqTable(DTableLL, LLtype, MaxLL, LLFSELog, ip, iend-ip, LL_defaultNorm, LL_defaultNormLog, flagRepeatTable);
-            if (ZSTDv07_isError(llhSize)) return ERROR(corruption_detected);
-            ip += llhSize;
-        }
-        {   size_t const ofhSize = ZSTDv07_buildSeqTable(DTableOffb, OFtype, MaxOff, OffFSELog, ip, iend-ip, OF_defaultNorm, OF_defaultNormLog, flagRepeatTable);
-            if (ZSTDv07_isError(ofhSize)) return ERROR(corruption_detected);
-            ip += ofhSize;
-        }
-        {   size_t const mlhSize = ZSTDv07_buildSeqTable(DTableML, MLtype, MaxML, MLFSELog, ip, iend-ip, ML_defaultNorm, ML_defaultNormLog, flagRepeatTable);
-            if (ZSTDv07_isError(mlhSize)) return ERROR(corruption_detected);
-            ip += mlhSize;
-    }   }
-
-    return ip-istart;
-}
-
-
-typedef struct {
-    size_t litLength;
-    size_t matchLength;
-    size_t offset;
-} seq_t;
-
-typedef struct {
-    BITv07_DStream_t DStream;
-    FSEv07_DState_t stateLL;
-    FSEv07_DState_t stateOffb;
-    FSEv07_DState_t stateML;
-    size_t prevOffset[ZSTDv07_REP_INIT];
-} seqState_t;
-
-
-static seq_t ZSTDv07_decodeSequence(seqState_t* seqState)
-{
-    seq_t seq;
-
-    U32 const llCode = FSEv07_peekSymbol(&(seqState->stateLL));
-    U32 const mlCode = FSEv07_peekSymbol(&(seqState->stateML));
-    U32 const ofCode = FSEv07_peekSymbol(&(seqState->stateOffb));   /* <= maxOff, by table construction */
-
-    U32 const llBits = LL_bits[llCode];
-    U32 const mlBits = ML_bits[mlCode];
-    U32 const ofBits = ofCode;
-    U32 const totalBits = llBits+mlBits+ofBits;
-
-    static const U32 LL_base[MaxLL+1] = {
-                             0,  1,  2,  3,  4,  5,  6,  7,  8,  9,   10,    11,    12,    13,    14,     15,
-                            16, 18, 20, 22, 24, 28, 32, 40, 48, 64, 0x80, 0x100, 0x200, 0x400, 0x800, 0x1000,
-                            0x2000, 0x4000, 0x8000, 0x10000 };
-
-    static const U32 ML_base[MaxML+1] = {
-                             3,  4,  5,  6,  7,  8,  9, 10, 11, 12, 13,   14,    15,    16,    17,    18,
-                            19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29,   30,    31,    32,    33,    34,
-                            35, 37, 39, 41, 43, 47, 51, 59, 67, 83, 99, 0x83, 0x103, 0x203, 0x403, 0x803,
-                            0x1003, 0x2003, 0x4003, 0x8003, 0x10003 };
-
-    static const U32 OF_base[MaxOff+1] = {
-                 0,        1,       1,       5,     0xD,     0x1D,     0x3D,     0x7D,
-                 0xFD,   0x1FD,   0x3FD,   0x7FD,   0xFFD,   0x1FFD,   0x3FFD,   0x7FFD,
-                 0xFFFD, 0x1FFFD, 0x3FFFD, 0x7FFFD, 0xFFFFD, 0x1FFFFD, 0x3FFFFD, 0x7FFFFD,
-                 0xFFFFFD, 0x1FFFFFD, 0x3FFFFFD, 0x7FFFFFD, 0xFFFFFFD };
-
-    /* sequence */
-    {   size_t offset;
-        if (!ofCode)
-            offset = 0;
-        else {
-            offset = OF_base[ofCode] + BITv07_readBits(&(seqState->DStream), ofBits);   /* <=  (ZSTDv07_WINDOWLOG_MAX-1) bits */
-            if (MEM_32bits()) BITv07_reloadDStream(&(seqState->DStream));
-        }
-
-        if (ofCode <= 1) {
-            if ((llCode == 0) & (offset <= 1)) offset = 1-offset;
-            if (offset) {
-                size_t const temp = seqState->prevOffset[offset];
-                if (offset != 1) seqState->prevOffset[2] = seqState->prevOffset[1];
-                seqState->prevOffset[1] = seqState->prevOffset[0];
-                seqState->prevOffset[0] = offset = temp;
-            } else {
-                offset = seqState->prevOffset[0];
-            }
-        } else {
-            seqState->prevOffset[2] = seqState->prevOffset[1];
-            seqState->prevOffset[1] = seqState->prevOffset[0];
-            seqState->prevOffset[0] = offset;
-        }
-        seq.offset = offset;
-    }
-
-    seq.matchLength = ML_base[mlCode] + ((mlCode>31) ? BITv07_readBits(&(seqState->DStream), mlBits) : 0);   /* <=  16 bits */
-    if (MEM_32bits() && (mlBits+llBits>24)) BITv07_reloadDStream(&(seqState->DStream));
-
-    seq.litLength = LL_base[llCode] + ((llCode>15) ? BITv07_readBits(&(seqState->DStream), llBits) : 0);   /* <=  16 bits */
-    if (MEM_32bits() ||
-       (totalBits > 64 - 7 - (LLFSELog+MLFSELog+OffFSELog)) ) BITv07_reloadDStream(&(seqState->DStream));
-
-    /* ANS state update */
-    FSEv07_updateState(&(seqState->stateLL), &(seqState->DStream));   /* <=  9 bits */
-    FSEv07_updateState(&(seqState->stateML), &(seqState->DStream));   /* <=  9 bits */
-    if (MEM_32bits()) BITv07_reloadDStream(&(seqState->DStream));     /* <= 18 bits */
-    FSEv07_updateState(&(seqState->stateOffb), &(seqState->DStream)); /* <=  8 bits */
-
-    return seq;
-}
-
-
-static
-size_t ZSTDv07_execSequence(BYTE* op,
-                                BYTE* const oend, seq_t sequence,
-                                const BYTE** litPtr, const BYTE* const litLimit,
-                                const BYTE* const base, const BYTE* const vBase, const BYTE* const dictEnd)
-{
-    BYTE* const oLitEnd = op + sequence.litLength;
-    size_t const sequenceLength = sequence.litLength + sequence.matchLength;
-    BYTE* const oMatchEnd = op + sequenceLength;   /* risk : address space overflow (32-bits) */
-    BYTE* const oend_w = oend-WILDCOPY_OVERLENGTH;
-    const BYTE* const iLitEnd = *litPtr + sequence.litLength;
-    const BYTE* match = oLitEnd - sequence.offset;
-
-    /* check */
-    if ((oLitEnd>oend_w) | (oMatchEnd>oend)) return ERROR(dstSize_tooSmall); /* last match must start at a minimum distance of WILDCOPY_OVERLENGTH from oend */
-    if (iLitEnd > litLimit) return ERROR(corruption_detected);   /* over-read beyond lit buffer */
-
-    /* copy Literals */
-    ZSTDv07_wildcopy(op, *litPtr, sequence.litLength);   /* note : since oLitEnd <= oend-WILDCOPY_OVERLENGTH, no risk of overwrite beyond oend */
-    op = oLitEnd;
-    *litPtr = iLitEnd;   /* update for next sequence */
-
-    /* copy Match */
-    if (sequence.offset > (size_t)(oLitEnd - base)) {
-        /* offset beyond prefix */
-        if (sequence.offset > (size_t)(oLitEnd - vBase)) return ERROR(corruption_detected);
-        match = dictEnd - (base-match);
-        if (match + sequence.matchLength <= dictEnd) {
-            memmove(oLitEnd, match, sequence.matchLength);
-            return sequenceLength;
-        }
-        /* span extDict & currentPrefixSegment */
-        {   size_t const length1 = dictEnd - match;
-            memmove(oLitEnd, match, length1);
-            op = oLitEnd + length1;
-            sequence.matchLength -= length1;
-            match = base;
-            if (op > oend_w || sequence.matchLength < MINMATCH) {
-              while (op < oMatchEnd) *op++ = *match++;
-              return sequenceLength;
-            }
-    }   }
-    /* Requirement: op <= oend_w */
-
-    /* match within prefix */
-    if (sequence.offset < 8) {
-        /* close range match, overlap */
-        static const U32 dec32table[] = { 0, 1, 2, 1, 4, 4, 4, 4 };   /* added */
-        static const int dec64table[] = { 8, 8, 8, 7, 8, 9,10,11 };   /* subtracted */
-        int const sub2 = dec64table[sequence.offset];
-        op[0] = match[0];
-        op[1] = match[1];
-        op[2] = match[2];
-        op[3] = match[3];
-        match += dec32table[sequence.offset];
-        ZSTDv07_copy4(op+4, match);
-        match -= sub2;
-    } else {
-        ZSTDv07_copy8(op, match);
-    }
-    op += 8; match += 8;
-
-    if (oMatchEnd > oend-(16-MINMATCH)) {
-        if (op < oend_w) {
-            ZSTDv07_wildcopy(op, match, oend_w - op);
-            match += oend_w - op;
-            op = oend_w;
-        }
-        while (op < oMatchEnd) *op++ = *match++;
-    } else {
-        ZSTDv07_wildcopy(op, match, (ptrdiff_t)sequence.matchLength-8);   /* works even if matchLength < 8 */
-    }
-    return sequenceLength;
-}
-
-
-static size_t ZSTDv07_decompressSequences(
-                               ZSTDv07_DCtx* dctx,
-                               void* dst, size_t maxDstSize,
-                         const void* seqStart, size_t seqSize)
-{
-    const BYTE* ip = (const BYTE*)seqStart;
-    const BYTE* const iend = ip + seqSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + maxDstSize;
-    BYTE* op = ostart;
-    const BYTE* litPtr = dctx->litPtr;
-    const BYTE* const litEnd = litPtr + dctx->litSize;
-    FSEv07_DTable* DTableLL = dctx->LLTable;
-    FSEv07_DTable* DTableML = dctx->MLTable;
-    FSEv07_DTable* DTableOffb = dctx->OffTable;
-    const BYTE* const base = (const BYTE*) (dctx->base);
-    const BYTE* const vBase = (const BYTE*) (dctx->vBase);
-    const BYTE* const dictEnd = (const BYTE*) (dctx->dictEnd);
-    int nbSeq;
-
-    /* Build Decoding Tables */
-    {   size_t const seqHSize = ZSTDv07_decodeSeqHeaders(&nbSeq, DTableLL, DTableML, DTableOffb, dctx->fseEntropy, ip, seqSize);
-        if (ZSTDv07_isError(seqHSize)) return seqHSize;
-        ip += seqHSize;
-    }
-
-    /* Regen sequences */
-    if (nbSeq) {
-        seqState_t seqState;
-        dctx->fseEntropy = 1;
-        { U32 i; for (i=0; i<ZSTDv07_REP_INIT; i++) seqState.prevOffset[i] = dctx->rep[i]; }
-        { size_t const errorCode = BITv07_initDStream(&(seqState.DStream), ip, iend-ip);
-          if (ERR_isError(errorCode)) return ERROR(corruption_detected); }
-        FSEv07_initDState(&(seqState.stateLL), &(seqState.DStream), DTableLL);
-        FSEv07_initDState(&(seqState.stateOffb), &(seqState.DStream), DTableOffb);
-        FSEv07_initDState(&(seqState.stateML), &(seqState.DStream), DTableML);
-
-        for ( ; (BITv07_reloadDStream(&(seqState.DStream)) <= BITv07_DStream_completed) && nbSeq ; ) {
-            nbSeq--;
-            {   seq_t const sequence = ZSTDv07_decodeSequence(&seqState);
-                size_t const oneSeqSize = ZSTDv07_execSequence(op, oend, sequence, &litPtr, litEnd, base, vBase, dictEnd);
-                if (ZSTDv07_isError(oneSeqSize)) return oneSeqSize;
-                op += oneSeqSize;
-        }   }
-
-        /* check if reached exact end */
-        if (nbSeq) return ERROR(corruption_detected);
-        /* save reps for next block */
-        { U32 i; for (i=0; i<ZSTDv07_REP_INIT; i++) dctx->rep[i] = (U32)(seqState.prevOffset[i]); }
-    }
-
-    /* last literal segment */
-    {   size_t const lastLLSize = litEnd - litPtr;
-        //if (litPtr > litEnd) return ERROR(corruption_detected);   /* too many literals already used */
-        if (lastLLSize > (size_t)(oend-op)) return ERROR(dstSize_tooSmall);
-        memcpy(op, litPtr, lastLLSize);
-        op += lastLLSize;
-    }
-
-    return op-ostart;
-}
-
-
-static void ZSTDv07_checkContinuity(ZSTDv07_DCtx* dctx, const void* dst)
-{
-    if (dst != dctx->previousDstEnd) {   /* not contiguous */
-        dctx->dictEnd = dctx->previousDstEnd;
-        dctx->vBase = (const char*)dst - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-        dctx->base = dst;
-        dctx->previousDstEnd = dst;
-    }
-}
-
-
-static size_t ZSTDv07_decompressBlock_internal(ZSTDv07_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{   /* blockType == blockCompressed */
-    const BYTE* ip = (const BYTE*)src;
-
-    if (srcSize >= ZSTDv07_BLOCKSIZE_ABSOLUTEMAX) return ERROR(srcSize_wrong);
-
-    /* Decode literals sub-block */
-    {   size_t const litCSize = ZSTDv07_decodeLiteralsBlock(dctx, src, srcSize);
-        if (ZSTDv07_isError(litCSize)) return litCSize;
-        ip += litCSize;
-        srcSize -= litCSize;
-    }
-    return ZSTDv07_decompressSequences(dctx, dst, dstCapacity, ip, srcSize);
-}
-
-
-size_t ZSTDv07_decompressBlock(ZSTDv07_DCtx* dctx,
-                            void* dst, size_t dstCapacity,
-                      const void* src, size_t srcSize)
-{
-    size_t dSize;
-    ZSTDv07_checkContinuity(dctx, dst);
-    dSize = ZSTDv07_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize);
-    dctx->previousDstEnd = (char*)dst + dSize;
-    return dSize;
-}
-
-
-/** ZSTDv07_insertBlock() :
-    insert `src` block into `dctx` history. Useful to track uncompressed blocks. */
-ZSTDLIBv07_API size_t ZSTDv07_insertBlock(ZSTDv07_DCtx* dctx, const void* blockStart, size_t blockSize)
-{
-    ZSTDv07_checkContinuity(dctx, blockStart);
-    dctx->previousDstEnd = (const char*)blockStart + blockSize;
-    return blockSize;
-}
-
-
-static size_t ZSTDv07_generateNxBytes(void* dst, size_t dstCapacity, BYTE byte, size_t length)
-{
-    if (length > dstCapacity) return ERROR(dstSize_tooSmall);
-    memset(dst, byte, length);
-    return length;
-}
-
-
-/*! ZSTDv07_decompressFrame() :
-*   `dctx` must be properly initialized */
-static size_t ZSTDv07_decompressFrame(ZSTDv07_DCtx* dctx,
-                                 void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize)
-{
-    const BYTE* ip = (const BYTE*)src;
-    const BYTE* const iend = ip + srcSize;
-    BYTE* const ostart = (BYTE* const)dst;
-    BYTE* const oend = ostart + dstCapacity;
-    BYTE* op = ostart;
-    size_t remainingSize = srcSize;
-
-    /* check */
-    if (srcSize < ZSTDv07_frameHeaderSize_min+ZSTDv07_blockHeaderSize) return ERROR(srcSize_wrong);
-
-    /* Frame Header */
-    {   size_t const frameHeaderSize = ZSTDv07_frameHeaderSize(src, ZSTDv07_frameHeaderSize_min);
-        if (ZSTDv07_isError(frameHeaderSize)) return frameHeaderSize;
-        if (srcSize < frameHeaderSize+ZSTDv07_blockHeaderSize) return ERROR(srcSize_wrong);
-        if (ZSTDv07_decodeFrameHeader(dctx, src, frameHeaderSize)) return ERROR(corruption_detected);
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1) {
-        size_t decodedSize;
-        blockProperties_t blockProperties;
-        size_t const cBlockSize = ZSTDv07_getcBlockSize(ip, iend-ip, &blockProperties);
-        if (ZSTDv07_isError(cBlockSize)) return cBlockSize;
-
-        ip += ZSTDv07_blockHeaderSize;
-        remainingSize -= ZSTDv07_blockHeaderSize;
-        if (cBlockSize > remainingSize) return ERROR(srcSize_wrong);
-
-        switch(blockProperties.blockType)
-        {
-        case bt_compressed:
-            decodedSize = ZSTDv07_decompressBlock_internal(dctx, op, oend-op, ip, cBlockSize);
-            break;
-        case bt_raw :
-            decodedSize = ZSTDv07_copyRawBlock(op, oend-op, ip, cBlockSize);
-            break;
-        case bt_rle :
-            decodedSize = ZSTDv07_generateNxBytes(op, oend-op, *ip, blockProperties.origSize);
-            break;
-        case bt_end :
-            /* end of frame */
-            if (remainingSize) return ERROR(srcSize_wrong);
-            decodedSize = 0;
-            break;
-        default:
-            return ERROR(GENERIC);   /* impossible */
-        }
-        if (blockProperties.blockType == bt_end) break;   /* bt_end */
-
-        if (ZSTDv07_isError(decodedSize)) return decodedSize;
-        if (dctx->fParams.checksumFlag) XXH64_update(&dctx->xxhState, op, decodedSize);
-        op += decodedSize;
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-    }
-
-    return op-ostart;
-}
-
-
-/*! ZSTDv07_decompress_usingPreparedDCtx() :
-*   Same as ZSTDv07_decompress_usingDict, but using a reference context `preparedDCtx`, where dictionary has been loaded.
-*   It avoids reloading the dictionary each time.
-*   `preparedDCtx` must have been properly initialized using ZSTDv07_decompressBegin_usingDict().
-*   Requires 2 contexts : 1 for reference (preparedDCtx), which will not be modified, and 1 to run the decompression operation (dctx) */
-static size_t ZSTDv07_decompress_usingPreparedDCtx(ZSTDv07_DCtx* dctx, const ZSTDv07_DCtx* refDCtx,
-                                         void* dst, size_t dstCapacity,
-                                   const void* src, size_t srcSize)
-{
-    ZSTDv07_copyDCtx(dctx, refDCtx);
-    ZSTDv07_checkContinuity(dctx, dst);
-    return ZSTDv07_decompressFrame(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-size_t ZSTDv07_decompress_usingDict(ZSTDv07_DCtx* dctx,
-                                 void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize,
-                                 const void* dict, size_t dictSize)
-{
-    ZSTDv07_decompressBegin_usingDict(dctx, dict, dictSize);
-    ZSTDv07_checkContinuity(dctx, dst);
-    return ZSTDv07_decompressFrame(dctx, dst, dstCapacity, src, srcSize);
-}
-
-
-size_t ZSTDv07_decompressDCtx(ZSTDv07_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    return ZSTDv07_decompress_usingDict(dctx, dst, dstCapacity, src, srcSize, NULL, 0);
-}
-
-
-size_t ZSTDv07_decompress(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-#if defined(ZSTDv07_HEAPMODE) && (ZSTDv07_HEAPMODE==1)
-    size_t regenSize;
-    ZSTDv07_DCtx* const dctx = ZSTDv07_createDCtx();
-    if (dctx==NULL) return ERROR(memory_allocation);
-    regenSize = ZSTDv07_decompressDCtx(dctx, dst, dstCapacity, src, srcSize);
-    ZSTDv07_freeDCtx(dctx);
-    return regenSize;
-#else   /* stack mode */
-    ZSTDv07_DCtx dctx;
-    return ZSTDv07_decompressDCtx(&dctx, dst, dstCapacity, src, srcSize);
-#endif
-}
-
-/* ZSTD_errorFrameSizeInfoLegacy() :
-   assumes `cSize` and `dBound` are _not_ NULL */
-static void ZSTD_errorFrameSizeInfoLegacy(size_t* cSize, unsigned long long* dBound, size_t ret)
-{
-    *cSize = ret;
-    *dBound = ZSTD_CONTENTSIZE_ERROR;
-}
-
-void ZSTDv07_findFrameSizeInfoLegacy(const void *src, size_t srcSize, size_t* cSize, unsigned long long* dBound)
-{
-    const BYTE* ip = (const BYTE*)src;
-    size_t remainingSize = srcSize;
-    size_t nbBlocks = 0;
-
-    /* check */
-    if (srcSize < ZSTDv07_frameHeaderSize_min+ZSTDv07_blockHeaderSize) {
-        ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-        return;
-    }
-
-    /* Frame Header */
-    {   size_t const frameHeaderSize = ZSTDv07_frameHeaderSize(src, srcSize);
-        if (ZSTDv07_isError(frameHeaderSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, frameHeaderSize);
-            return;
-        }
-        if (MEM_readLE32(src) != ZSTDv07_MAGICNUMBER) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(prefix_unknown));
-            return;
-        }
-        if (srcSize < frameHeaderSize+ZSTDv07_blockHeaderSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-        ip += frameHeaderSize; remainingSize -= frameHeaderSize;
-    }
-
-    /* Loop on each block */
-    while (1) {
-        blockProperties_t blockProperties;
-        size_t const cBlockSize = ZSTDv07_getcBlockSize(ip, remainingSize, &blockProperties);
-        if (ZSTDv07_isError(cBlockSize)) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, cBlockSize);
-            return;
-        }
-
-        ip += ZSTDv07_blockHeaderSize;
-        remainingSize -= ZSTDv07_blockHeaderSize;
-
-        if (blockProperties.blockType == bt_end) break;
-
-        if (cBlockSize > remainingSize) {
-            ZSTD_errorFrameSizeInfoLegacy(cSize, dBound, ERROR(srcSize_wrong));
-            return;
-        }
-
-        ip += cBlockSize;
-        remainingSize -= cBlockSize;
-        nbBlocks++;
-    }
-
-    *cSize = ip - (const BYTE*)src;
-    *dBound = nbBlocks * ZSTDv07_BLOCKSIZE_ABSOLUTEMAX;
-}
-
-/*_******************************
-*  Streaming Decompression API
-********************************/
-size_t ZSTDv07_nextSrcSizeToDecompress(ZSTDv07_DCtx* dctx)
-{
-    return dctx->expected;
-}
-
-int ZSTDv07_isSkipFrame(ZSTDv07_DCtx* dctx)
-{
-    return dctx->stage == ZSTDds_skipFrame;
-}
-
-/** ZSTDv07_decompressContinue() :
-*   @return : nb of bytes generated into `dst` (necessarily <= `dstCapacity)
-*             or an error code, which can be tested using ZSTDv07_isError() */
-size_t ZSTDv07_decompressContinue(ZSTDv07_DCtx* dctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    /* Sanity check */
-    if (srcSize != dctx->expected) return ERROR(srcSize_wrong);
-    if (dstCapacity) ZSTDv07_checkContinuity(dctx, dst);
-
-    switch (dctx->stage)
-    {
-    case ZSTDds_getFrameHeaderSize :
-        if (srcSize != ZSTDv07_frameHeaderSize_min) return ERROR(srcSize_wrong);   /* impossible */
-        if ((MEM_readLE32(src) & 0xFFFFFFF0U) == ZSTDv07_MAGIC_SKIPPABLE_START) {
-            memcpy(dctx->headerBuffer, src, ZSTDv07_frameHeaderSize_min);
-            dctx->expected = ZSTDv07_skippableHeaderSize - ZSTDv07_frameHeaderSize_min; /* magic number + skippable frame length */
-            dctx->stage = ZSTDds_decodeSkippableHeader;
-            return 0;
-        }
-        dctx->headerSize = ZSTDv07_frameHeaderSize(src, ZSTDv07_frameHeaderSize_min);
-        if (ZSTDv07_isError(dctx->headerSize)) return dctx->headerSize;
-        memcpy(dctx->headerBuffer, src, ZSTDv07_frameHeaderSize_min);
-        if (dctx->headerSize > ZSTDv07_frameHeaderSize_min) {
-            dctx->expected = dctx->headerSize - ZSTDv07_frameHeaderSize_min;
-            dctx->stage = ZSTDds_decodeFrameHeader;
-            return 0;
-        }
-        dctx->expected = 0;   /* not necessary to copy more */
-	/* fall-through */
-    case ZSTDds_decodeFrameHeader:
-        {   size_t result;
-            memcpy(dctx->headerBuffer + ZSTDv07_frameHeaderSize_min, src, dctx->expected);
-            result = ZSTDv07_decodeFrameHeader(dctx, dctx->headerBuffer, dctx->headerSize);
-            if (ZSTDv07_isError(result)) return result;
-            dctx->expected = ZSTDv07_blockHeaderSize;
-            dctx->stage = ZSTDds_decodeBlockHeader;
-            return 0;
-        }
-    case ZSTDds_decodeBlockHeader:
-        {   blockProperties_t bp;
-            size_t const cBlockSize = ZSTDv07_getcBlockSize(src, ZSTDv07_blockHeaderSize, &bp);
-            if (ZSTDv07_isError(cBlockSize)) return cBlockSize;
-            if (bp.blockType == bt_end) {
-                if (dctx->fParams.checksumFlag) {
-                    U64 const h64 = XXH64_digest(&dctx->xxhState);
-                    U32 const h32 = (U32)(h64>>11) & ((1<<22)-1);
-                    const BYTE* const ip = (const BYTE*)src;
-                    U32 const check32 = ip[2] + (ip[1] << 8) + ((ip[0] & 0x3F) << 16);
-                    if (check32 != h32) return ERROR(checksum_wrong);
-                }
-                dctx->expected = 0;
-                dctx->stage = ZSTDds_getFrameHeaderSize;
-            } else {
-                dctx->expected = cBlockSize;
-                dctx->bType = bp.blockType;
-                dctx->stage = ZSTDds_decompressBlock;
-            }
-            return 0;
-        }
-    case ZSTDds_decompressBlock:
-        {   size_t rSize;
-            switch(dctx->bType)
-            {
-            case bt_compressed:
-                rSize = ZSTDv07_decompressBlock_internal(dctx, dst, dstCapacity, src, srcSize);
-                break;
-            case bt_raw :
-                rSize = ZSTDv07_copyRawBlock(dst, dstCapacity, src, srcSize);
-                break;
-            case bt_rle :
-                return ERROR(GENERIC);   /* not yet handled */
-                break;
-            case bt_end :   /* should never happen (filtered at phase 1) */
-                rSize = 0;
-                break;
-            default:
-                return ERROR(GENERIC);   /* impossible */
-            }
-            dctx->stage = ZSTDds_decodeBlockHeader;
-            dctx->expected = ZSTDv07_blockHeaderSize;
-            dctx->previousDstEnd = (char*)dst + rSize;
-            if (ZSTDv07_isError(rSize)) return rSize;
-            if (dctx->fParams.checksumFlag) XXH64_update(&dctx->xxhState, dst, rSize);
-            return rSize;
-        }
-    case ZSTDds_decodeSkippableHeader:
-        {   memcpy(dctx->headerBuffer + ZSTDv07_frameHeaderSize_min, src, dctx->expected);
-            dctx->expected = MEM_readLE32(dctx->headerBuffer + 4);
-            dctx->stage = ZSTDds_skipFrame;
-            return 0;
-        }
-    case ZSTDds_skipFrame:
-        {   dctx->expected = 0;
-            dctx->stage = ZSTDds_getFrameHeaderSize;
-            return 0;
-        }
-    default:
-        return ERROR(GENERIC);   /* impossible */
-    }
-}
-
-
-static size_t ZSTDv07_refDictContent(ZSTDv07_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    dctx->dictEnd = dctx->previousDstEnd;
-    dctx->vBase = (const char*)dict - ((const char*)(dctx->previousDstEnd) - (const char*)(dctx->base));
-    dctx->base = dict;
-    dctx->previousDstEnd = (const char*)dict + dictSize;
-    return 0;
-}
-
-static size_t ZSTDv07_loadEntropy(ZSTDv07_DCtx* dctx, const void* const dict, size_t const dictSize)
-{
-    const BYTE* dictPtr = (const BYTE*)dict;
-    const BYTE* const dictEnd = dictPtr + dictSize;
-
-    {   size_t const hSize = HUFv07_readDTableX4(dctx->hufTable, dict, dictSize);
-        if (HUFv07_isError(hSize)) return ERROR(dictionary_corrupted);
-        dictPtr += hSize;
-    }
-
-    {   short offcodeNCount[MaxOff+1];
-        U32 offcodeMaxValue=MaxOff, offcodeLog;
-        size_t const offcodeHeaderSize = FSEv07_readNCount(offcodeNCount, &offcodeMaxValue, &offcodeLog, dictPtr, dictEnd-dictPtr);
-        if (FSEv07_isError(offcodeHeaderSize)) return ERROR(dictionary_corrupted);
-        if (offcodeLog > OffFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv07_buildDTable(dctx->OffTable, offcodeNCount, offcodeMaxValue, offcodeLog);
-          if (FSEv07_isError(errorCode)) return ERROR(dictionary_corrupted); }
-        dictPtr += offcodeHeaderSize;
-    }
-
-    {   short matchlengthNCount[MaxML+1];
-        unsigned matchlengthMaxValue = MaxML, matchlengthLog;
-        size_t const matchlengthHeaderSize = FSEv07_readNCount(matchlengthNCount, &matchlengthMaxValue, &matchlengthLog, dictPtr, dictEnd-dictPtr);
-        if (FSEv07_isError(matchlengthHeaderSize)) return ERROR(dictionary_corrupted);
-        if (matchlengthLog > MLFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv07_buildDTable(dctx->MLTable, matchlengthNCount, matchlengthMaxValue, matchlengthLog);
-          if (FSEv07_isError(errorCode)) return ERROR(dictionary_corrupted); }
-        dictPtr += matchlengthHeaderSize;
-    }
-
-    {   short litlengthNCount[MaxLL+1];
-        unsigned litlengthMaxValue = MaxLL, litlengthLog;
-        size_t const litlengthHeaderSize = FSEv07_readNCount(litlengthNCount, &litlengthMaxValue, &litlengthLog, dictPtr, dictEnd-dictPtr);
-        if (FSEv07_isError(litlengthHeaderSize)) return ERROR(dictionary_corrupted);
-        if (litlengthLog > LLFSELog) return ERROR(dictionary_corrupted);
-        { size_t const errorCode = FSEv07_buildDTable(dctx->LLTable, litlengthNCount, litlengthMaxValue, litlengthLog);
-          if (FSEv07_isError(errorCode)) return ERROR(dictionary_corrupted); }
-        dictPtr += litlengthHeaderSize;
-    }
-
-    if (dictPtr+12 > dictEnd) return ERROR(dictionary_corrupted);
-    dctx->rep[0] = MEM_readLE32(dictPtr+0); if (dctx->rep[0] == 0 || dctx->rep[0] >= dictSize) return ERROR(dictionary_corrupted);
-    dctx->rep[1] = MEM_readLE32(dictPtr+4); if (dctx->rep[1] == 0 || dctx->rep[1] >= dictSize) return ERROR(dictionary_corrupted);
-    dctx->rep[2] = MEM_readLE32(dictPtr+8); if (dctx->rep[2] == 0 || dctx->rep[2] >= dictSize) return ERROR(dictionary_corrupted);
-    dictPtr += 12;
-
-    dctx->litEntropy = dctx->fseEntropy = 1;
-    return dictPtr - (const BYTE*)dict;
-}
-
-static size_t ZSTDv07_decompress_insertDictionary(ZSTDv07_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    if (dictSize < 8) return ZSTDv07_refDictContent(dctx, dict, dictSize);
-    {   U32 const magic = MEM_readLE32(dict);
-        if (magic != ZSTDv07_DICT_MAGIC) {
-            return ZSTDv07_refDictContent(dctx, dict, dictSize);   /* pure content mode */
-    }   }
-    dctx->dictID = MEM_readLE32((const char*)dict + 4);
-
-    /* load entropy tables */
-    dict = (const char*)dict + 8;
-    dictSize -= 8;
-    {   size_t const eSize = ZSTDv07_loadEntropy(dctx, dict, dictSize);
-        if (ZSTDv07_isError(eSize)) return ERROR(dictionary_corrupted);
-        dict = (const char*)dict + eSize;
-        dictSize -= eSize;
-    }
-
-    /* reference dictionary content */
-    return ZSTDv07_refDictContent(dctx, dict, dictSize);
-}
-
-
-size_t ZSTDv07_decompressBegin_usingDict(ZSTDv07_DCtx* dctx, const void* dict, size_t dictSize)
-{
-    { size_t const errorCode = ZSTDv07_decompressBegin(dctx);
-      if (ZSTDv07_isError(errorCode)) return errorCode; }
-
-    if (dict && dictSize) {
-        size_t const errorCode = ZSTDv07_decompress_insertDictionary(dctx, dict, dictSize);
-        if (ZSTDv07_isError(errorCode)) return ERROR(dictionary_corrupted);
-    }
-
-    return 0;
-}
-
-
-struct ZSTDv07_DDict_s {
-    void* dict;
-    size_t dictSize;
-    ZSTDv07_DCtx* refContext;
-};  /* typedef'd tp ZSTDv07_CDict within zstd.h */
-
-static ZSTDv07_DDict* ZSTDv07_createDDict_advanced(const void* dict, size_t dictSize, ZSTDv07_customMem customMem)
-{
-    if (!customMem.customAlloc && !customMem.customFree)
-        customMem = defaultCustomMem;
-
-    if (!customMem.customAlloc || !customMem.customFree)
-        return NULL;
-
-    {   ZSTDv07_DDict* const ddict = (ZSTDv07_DDict*) customMem.customAlloc(customMem.opaque, sizeof(*ddict));
-        void* const dictContent = customMem.customAlloc(customMem.opaque, dictSize);
-        ZSTDv07_DCtx* const dctx = ZSTDv07_createDCtx_advanced(customMem);
-
-        if (!dictContent || !ddict || !dctx) {
-            customMem.customFree(customMem.opaque, dictContent);
-            customMem.customFree(customMem.opaque, ddict);
-            customMem.customFree(customMem.opaque, dctx);
-            return NULL;
-        }
-
-        memcpy(dictContent, dict, dictSize);
-        {   size_t const errorCode = ZSTDv07_decompressBegin_usingDict(dctx, dictContent, dictSize);
-            if (ZSTDv07_isError(errorCode)) {
-                customMem.customFree(customMem.opaque, dictContent);
-                customMem.customFree(customMem.opaque, ddict);
-                customMem.customFree(customMem.opaque, dctx);
-                return NULL;
-        }   }
-
-        ddict->dict = dictContent;
-        ddict->dictSize = dictSize;
-        ddict->refContext = dctx;
-        return ddict;
-    }
-}
-
-/*! ZSTDv07_createDDict() :
-*   Create a digested dictionary, ready to start decompression without startup delay.
-*   `dict` can be released after `ZSTDv07_DDict` creation */
-ZSTDv07_DDict* ZSTDv07_createDDict(const void* dict, size_t dictSize)
-{
-    ZSTDv07_customMem const allocator = { NULL, NULL, NULL };
-    return ZSTDv07_createDDict_advanced(dict, dictSize, allocator);
-}
-
-size_t ZSTDv07_freeDDict(ZSTDv07_DDict* ddict)
-{
-    ZSTDv07_freeFunction const cFree = ddict->refContext->customMem.customFree;
-    void* const opaque = ddict->refContext->customMem.opaque;
-    ZSTDv07_freeDCtx(ddict->refContext);
-    cFree(opaque, ddict->dict);
-    cFree(opaque, ddict);
-    return 0;
-}
-
-/*! ZSTDv07_decompress_usingDDict() :
-*   Decompression using a pre-digested Dictionary
-*   Use dictionary without significant overhead. */
-ZSTDLIBv07_API size_t ZSTDv07_decompress_usingDDict(ZSTDv07_DCtx* dctx,
-                                           void* dst, size_t dstCapacity,
-                                     const void* src, size_t srcSize,
-                                     const ZSTDv07_DDict* ddict)
-{
-    return ZSTDv07_decompress_usingPreparedDCtx(dctx, ddict->refContext,
-                                           dst, dstCapacity,
-                                           src, srcSize);
-}
-/*
-    Buffered version of Zstd compression library
-    Copyright (C) 2015-2016, Yann Collet.
-
-    BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
-
-    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
-    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.
-
-    You can contact the author at :
-    - zstd homepage : http://www.zstd.net/
-*/
-
-
-
-/*-***************************************************************************
-*  Streaming decompression howto
-*
-*  A ZBUFFv07_DCtx object is required to track streaming operations.
-*  Use ZBUFFv07_createDCtx() and ZBUFFv07_freeDCtx() to create/release resources.
-*  Use ZBUFFv07_decompressInit() to start a new decompression operation,
-*   or ZBUFFv07_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFFv07_DCtx objects can be re-init multiple times.
-*
-*  Use ZBUFFv07_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of @dst will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters, or change @dst.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to help latency),
-*            or 0 when a frame is completely decoded,
-*            or an error code, which can be tested using ZBUFFv07_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFFv07_recommendedDInSize() and ZBUFFv07_recommendedDOutSize()
-*  output : ZBUFFv07_recommendedDOutSize==128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFFv07_recommendedDInSize == 128KB + 3;
-*           just follow indications from ZBUFFv07_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-typedef enum { ZBUFFds_init, ZBUFFds_loadHeader,
-               ZBUFFds_read, ZBUFFds_load, ZBUFFds_flush } ZBUFFv07_dStage;
-
-/* *** Resource management *** */
-struct ZBUFFv07_DCtx_s {
-    ZSTDv07_DCtx* zd;
-    ZSTDv07_frameParams fParams;
-    ZBUFFv07_dStage stage;
-    char*  inBuff;
-    size_t inBuffSize;
-    size_t inPos;
-    char*  outBuff;
-    size_t outBuffSize;
-    size_t outStart;
-    size_t outEnd;
-    size_t blockSize;
-    BYTE headerBuffer[ZSTDv07_FRAMEHEADERSIZE_MAX];
-    size_t lhSize;
-    ZSTDv07_customMem customMem;
-};   /* typedef'd to ZBUFFv07_DCtx within "zstd_buffered.h" */
-
-ZSTDLIBv07_API ZBUFFv07_DCtx* ZBUFFv07_createDCtx_advanced(ZSTDv07_customMem customMem);
-
-ZBUFFv07_DCtx* ZBUFFv07_createDCtx(void)
-{
-    return ZBUFFv07_createDCtx_advanced(defaultCustomMem);
-}
-
-ZBUFFv07_DCtx* ZBUFFv07_createDCtx_advanced(ZSTDv07_customMem customMem)
-{
-    ZBUFFv07_DCtx* zbd;
-
-    if (!customMem.customAlloc && !customMem.customFree)
-        customMem = defaultCustomMem;
-
-    if (!customMem.customAlloc || !customMem.customFree)
-        return NULL;
-
-    zbd = (ZBUFFv07_DCtx*)customMem.customAlloc(customMem.opaque, sizeof(ZBUFFv07_DCtx));
-    if (zbd==NULL) return NULL;
-    memset(zbd, 0, sizeof(ZBUFFv07_DCtx));
-    memcpy(&zbd->customMem, &customMem, sizeof(ZSTDv07_customMem));
-    zbd->zd = ZSTDv07_createDCtx_advanced(customMem);
-    if (zbd->zd == NULL) { ZBUFFv07_freeDCtx(zbd); return NULL; }
-    zbd->stage = ZBUFFds_init;
-    return zbd;
-}
-
-size_t ZBUFFv07_freeDCtx(ZBUFFv07_DCtx* zbd)
-{
-    if (zbd==NULL) return 0;   /* support free on null */
-    ZSTDv07_freeDCtx(zbd->zd);
-    if (zbd->inBuff) zbd->customMem.customFree(zbd->customMem.opaque, zbd->inBuff);
-    if (zbd->outBuff) zbd->customMem.customFree(zbd->customMem.opaque, zbd->outBuff);
-    zbd->customMem.customFree(zbd->customMem.opaque, zbd);
-    return 0;
-}
-
-
-/* *** Initialization *** */
-
-size_t ZBUFFv07_decompressInitDictionary(ZBUFFv07_DCtx* zbd, const void* dict, size_t dictSize)
-{
-    zbd->stage = ZBUFFds_loadHeader;
-    zbd->lhSize = zbd->inPos = zbd->outStart = zbd->outEnd = 0;
-    return ZSTDv07_decompressBegin_usingDict(zbd->zd, dict, dictSize);
-}
-
-size_t ZBUFFv07_decompressInit(ZBUFFv07_DCtx* zbd)
-{
-    return ZBUFFv07_decompressInitDictionary(zbd, NULL, 0);
-}
-
-
-/* internal util function */
-MEM_STATIC size_t ZBUFFv07_limitCopy(void* dst, size_t dstCapacity, const void* src, size_t srcSize)
-{
-    size_t const length = MIN(dstCapacity, srcSize);
-    memcpy(dst, src, length);
-    return length;
-}
-
-
-/* *** Decompression *** */
-
-size_t ZBUFFv07_decompressContinue(ZBUFFv07_DCtx* zbd,
-                                void* dst, size_t* dstCapacityPtr,
-                          const void* src, size_t* srcSizePtr)
-{
-    const char* const istart = (const char*)src;
-    const char* const iend = istart + *srcSizePtr;
-    const char* ip = istart;
-    char* const ostart = (char*)dst;
-    char* const oend = ostart + *dstCapacityPtr;
-    char* op = ostart;
-    U32 notDone = 1;
-
-    while (notDone) {
-        switch(zbd->stage)
-        {
-        case ZBUFFds_init :
-            return ERROR(init_missing);
-
-        case ZBUFFds_loadHeader :
-            {   size_t const hSize = ZSTDv07_getFrameParams(&(zbd->fParams), zbd->headerBuffer, zbd->lhSize);
-                if (ZSTDv07_isError(hSize)) return hSize;
-                if (hSize != 0) {
-                    size_t const toLoad = hSize - zbd->lhSize;   /* if hSize!=0, hSize > zbd->lhSize */
-                    if (toLoad > (size_t)(iend-ip)) {   /* not enough input to load full header */
-                        memcpy(zbd->headerBuffer + zbd->lhSize, ip, iend-ip);
-                        zbd->lhSize += iend-ip;
-                        *dstCapacityPtr = 0;
-                        return (hSize - zbd->lhSize) + ZSTDv07_blockHeaderSize;   /* remaining header bytes + next block header */
-                    }
-                    memcpy(zbd->headerBuffer + zbd->lhSize, ip, toLoad); zbd->lhSize = hSize; ip += toLoad;
-                    break;
-            }   }
-
-            /* Consume header */
-            {   size_t const h1Size = ZSTDv07_nextSrcSizeToDecompress(zbd->zd);  /* == ZSTDv07_frameHeaderSize_min */
-                size_t const h1Result = ZSTDv07_decompressContinue(zbd->zd, NULL, 0, zbd->headerBuffer, h1Size);
-                if (ZSTDv07_isError(h1Result)) return h1Result;
-                if (h1Size < zbd->lhSize) {   /* long header */
-                    size_t const h2Size = ZSTDv07_nextSrcSizeToDecompress(zbd->zd);
-                    size_t const h2Result = ZSTDv07_decompressContinue(zbd->zd, NULL, 0, zbd->headerBuffer+h1Size, h2Size);
-                    if (ZSTDv07_isError(h2Result)) return h2Result;
-            }   }
-
-            zbd->fParams.windowSize = MAX(zbd->fParams.windowSize, 1U << ZSTDv07_WINDOWLOG_ABSOLUTEMIN);
-
-            /* Frame header instruct buffer sizes */
-            {   size_t const blockSize = MIN(zbd->fParams.windowSize, ZSTDv07_BLOCKSIZE_ABSOLUTEMAX);
-                zbd->blockSize = blockSize;
-                if (zbd->inBuffSize < blockSize) {
-                    zbd->customMem.customFree(zbd->customMem.opaque, zbd->inBuff);
-                    zbd->inBuffSize = blockSize;
-                    zbd->inBuff = (char*)zbd->customMem.customAlloc(zbd->customMem.opaque, blockSize);
-                    if (zbd->inBuff == NULL) return ERROR(memory_allocation);
-                }
-                {   size_t const neededOutSize = zbd->fParams.windowSize + blockSize + WILDCOPY_OVERLENGTH * 2;
-                    if (zbd->outBuffSize < neededOutSize) {
-                        zbd->customMem.customFree(zbd->customMem.opaque, zbd->outBuff);
-                        zbd->outBuffSize = neededOutSize;
-                        zbd->outBuff = (char*)zbd->customMem.customAlloc(zbd->customMem.opaque, neededOutSize);
-                        if (zbd->outBuff == NULL) return ERROR(memory_allocation);
-            }   }   }
-            zbd->stage = ZBUFFds_read;
-            /* pass-through */
-	    /* fall-through */
-        case ZBUFFds_read:
-            {   size_t const neededInSize = ZSTDv07_nextSrcSizeToDecompress(zbd->zd);
-                if (neededInSize==0) {  /* end of frame */
-                    zbd->stage = ZBUFFds_init;
-                    notDone = 0;
-                    break;
-                }
-                if ((size_t)(iend-ip) >= neededInSize) {  /* decode directly from src */
-                    const int isSkipFrame = ZSTDv07_isSkipFrame(zbd->zd);
-                    size_t const decodedSize = ZSTDv07_decompressContinue(zbd->zd,
-                        zbd->outBuff + zbd->outStart, (isSkipFrame ? 0 : zbd->outBuffSize - zbd->outStart),
-                        ip, neededInSize);
-                    if (ZSTDv07_isError(decodedSize)) return decodedSize;
-                    ip += neededInSize;
-                    if (!decodedSize && !isSkipFrame) break;   /* this was just a header */
-                    zbd->outEnd = zbd->outStart +  decodedSize;
-                    zbd->stage = ZBUFFds_flush;
-                    break;
-                }
-                if (ip==iend) { notDone = 0; break; }   /* no more input */
-                zbd->stage = ZBUFFds_load;
-            }
-	    /* fall-through */
-        case ZBUFFds_load:
-            {   size_t const neededInSize = ZSTDv07_nextSrcSizeToDecompress(zbd->zd);
-                size_t const toLoad = neededInSize - zbd->inPos;   /* should always be <= remaining space within inBuff */
-                size_t loadedSize;
-                if (toLoad > zbd->inBuffSize - zbd->inPos) return ERROR(corruption_detected);   /* should never happen */
-                loadedSize = ZBUFFv07_limitCopy(zbd->inBuff + zbd->inPos, toLoad, ip, iend-ip);
-                ip += loadedSize;
-                zbd->inPos += loadedSize;
-                if (loadedSize < toLoad) { notDone = 0; break; }   /* not enough input, wait for more */
-
-                /* decode loaded input */
-                {  const int isSkipFrame = ZSTDv07_isSkipFrame(zbd->zd);
-                   size_t const decodedSize = ZSTDv07_decompressContinue(zbd->zd,
-                        zbd->outBuff + zbd->outStart, zbd->outBuffSize - zbd->outStart,
-                        zbd->inBuff, neededInSize);
-                    if (ZSTDv07_isError(decodedSize)) return decodedSize;
-                    zbd->inPos = 0;   /* input is consumed */
-                    if (!decodedSize && !isSkipFrame) { zbd->stage = ZBUFFds_read; break; }   /* this was just a header */
-                    zbd->outEnd = zbd->outStart +  decodedSize;
-                    zbd->stage = ZBUFFds_flush;
-                    /* break; */
-                    /* pass-through */
-                }
-	    }
-	    /* fall-through */
-        case ZBUFFds_flush:
-            {   size_t const toFlushSize = zbd->outEnd - zbd->outStart;
-                size_t const flushedSize = ZBUFFv07_limitCopy(op, oend-op, zbd->outBuff + zbd->outStart, toFlushSize);
-                op += flushedSize;
-                zbd->outStart += flushedSize;
-                if (flushedSize == toFlushSize) {
-                    zbd->stage = ZBUFFds_read;
-                    if (zbd->outStart + zbd->blockSize > zbd->outBuffSize)
-                        zbd->outStart = zbd->outEnd = 0;
-                    break;
-                }
-                /* cannot flush everything */
-                notDone = 0;
-                break;
-            }
-        default: return ERROR(GENERIC);   /* impossible */
-    }   }
-
-    /* result */
-    *srcSizePtr = ip-istart;
-    *dstCapacityPtr = op-ostart;
-    {   size_t nextSrcSizeHint = ZSTDv07_nextSrcSizeToDecompress(zbd->zd);
-        nextSrcSizeHint -= zbd->inPos;   /* already loaded*/
-        return nextSrcSizeHint;
-    }
-}
-
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-size_t ZBUFFv07_recommendedDInSize(void)  { return ZSTDv07_BLOCKSIZE_ABSOLUTEMAX + ZSTDv07_blockHeaderSize /* block header size*/ ; }
-size_t ZBUFFv07_recommendedDOutSize(void) { return ZSTDv07_BLOCKSIZE_ABSOLUTEMAX; }
diff --git a/vendor/github.com/DataDog/zstd/zstd_v07.h b/vendor/github.com/DataDog/zstd/zstd_v07.h
deleted file mode 100644
index a566c1d..0000000
--- a/vendor/github.com/DataDog/zstd/zstd_v07.h
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-#ifndef ZSTDv07_H_235446
-#define ZSTDv07_H_235446
-
-#if defined (__cplusplus)
-extern "C" {
-#endif
-
-/*======  Dependency  ======*/
-#include <stddef.h>   /* size_t */
-
-
-/*======  Export for Windows  ======*/
-/*!
-*  ZSTDv07_DLL_EXPORT :
-*  Enable exporting of functions when building a Windows DLL
-*/
-#if defined(_WIN32) && defined(ZSTDv07_DLL_EXPORT) && (ZSTDv07_DLL_EXPORT==1)
-#  define ZSTDLIBv07_API __declspec(dllexport)
-#else
-#  define ZSTDLIBv07_API
-#endif
-
-
-/* *************************************
-*  Simple API
-***************************************/
-/*! ZSTDv07_getDecompressedSize() :
-*   @return : decompressed size if known, 0 otherwise.
-       note 1 : if `0`, follow up with ZSTDv07_getFrameParams() to know precise failure cause.
-       note 2 : decompressed size could be wrong or intentionally modified !
-                always ensure results fit within application's authorized limits */
-unsigned long long ZSTDv07_getDecompressedSize(const void* src, size_t srcSize);
-
-/*! ZSTDv07_decompress() :
-    `compressedSize` : must be _exact_ size of compressed input, otherwise decompression will fail.
-    `dstCapacity` must be equal or larger than originalSize.
-    @return : the number of bytes decompressed into `dst` (<= `dstCapacity`),
-              or an errorCode if it fails (which can be tested using ZSTDv07_isError()) */
-ZSTDLIBv07_API size_t ZSTDv07_decompress( void* dst, size_t dstCapacity,
-                                    const void* src, size_t compressedSize);
-
-/**
-ZSTDv07_findFrameSizeInfoLegacy() : get the source length and decompressed bound of a ZSTD frame compliant with v0.7.x format
-    srcSize : The size of the 'src' buffer, at least as large as the frame pointed to by 'src'
-    cSize (output parameter)  : the number of bytes that would be read to decompress this frame
-                                or an error code if it fails (which can be tested using ZSTDv01_isError())
-    dBound (output parameter) : an upper-bound for the decompressed size of the data in the frame
-                                or ZSTD_CONTENTSIZE_ERROR if an error occurs
-
-    note : assumes `cSize` and `dBound` are _not_ NULL.
-*/
-void ZSTDv07_findFrameSizeInfoLegacy(const void *src, size_t srcSize,
-                                     size_t* cSize, unsigned long long* dBound);
-
-/*======  Helper functions  ======*/
-ZSTDLIBv07_API unsigned    ZSTDv07_isError(size_t code);          /*!< tells if a `size_t` function result is an error code */
-ZSTDLIBv07_API const char* ZSTDv07_getErrorName(size_t code);     /*!< provides readable string from an error code */
-
-
-/*-*************************************
-*  Explicit memory management
-***************************************/
-/** Decompression context */
-typedef struct ZSTDv07_DCtx_s ZSTDv07_DCtx;
-ZSTDLIBv07_API ZSTDv07_DCtx* ZSTDv07_createDCtx(void);
-ZSTDLIBv07_API size_t     ZSTDv07_freeDCtx(ZSTDv07_DCtx* dctx);      /*!< @return : errorCode */
-
-/** ZSTDv07_decompressDCtx() :
-*   Same as ZSTDv07_decompress(), requires an allocated ZSTDv07_DCtx (see ZSTDv07_createDCtx()) */
-ZSTDLIBv07_API size_t ZSTDv07_decompressDCtx(ZSTDv07_DCtx* ctx, void* dst, size_t dstCapacity, const void* src, size_t srcSize);
-
-
-/*-************************
-*  Simple dictionary API
-***************************/
-/*! ZSTDv07_decompress_usingDict() :
-*   Decompression using a pre-defined Dictionary content (see dictBuilder).
-*   Dictionary must be identical to the one used during compression.
-*   Note : This function load the dictionary, resulting in a significant startup time */
-ZSTDLIBv07_API size_t ZSTDv07_decompress_usingDict(ZSTDv07_DCtx* dctx,
-                                                   void* dst, size_t dstCapacity,
-                                             const void* src, size_t srcSize,
-                                             const void* dict,size_t dictSize);
-
-
-/*-**************************
-*  Advanced Dictionary API
-****************************/
-/*! ZSTDv07_createDDict() :
-*   Create a digested dictionary, ready to start decompression operation without startup delay.
-*   `dict` can be released after creation */
-typedef struct ZSTDv07_DDict_s ZSTDv07_DDict;
-ZSTDLIBv07_API ZSTDv07_DDict* ZSTDv07_createDDict(const void* dict, size_t dictSize);
-ZSTDLIBv07_API size_t      ZSTDv07_freeDDict(ZSTDv07_DDict* ddict);
-
-/*! ZSTDv07_decompress_usingDDict() :
-*   Decompression using a pre-digested Dictionary
-*   Faster startup than ZSTDv07_decompress_usingDict(), recommended when same dictionary is used multiple times. */
-ZSTDLIBv07_API size_t ZSTDv07_decompress_usingDDict(ZSTDv07_DCtx* dctx,
-                                                    void* dst, size_t dstCapacity,
-                                              const void* src, size_t srcSize,
-                                              const ZSTDv07_DDict* ddict);
-
-typedef struct {
-    unsigned long long frameContentSize;
-    unsigned windowSize;
-    unsigned dictID;
-    unsigned checksumFlag;
-} ZSTDv07_frameParams;
-
-ZSTDLIBv07_API size_t ZSTDv07_getFrameParams(ZSTDv07_frameParams* fparamsPtr, const void* src, size_t srcSize);   /**< doesn't consume input */
-
-
-
-
-/* *************************************
-*  Streaming functions
-***************************************/
-typedef struct ZBUFFv07_DCtx_s ZBUFFv07_DCtx;
-ZSTDLIBv07_API ZBUFFv07_DCtx* ZBUFFv07_createDCtx(void);
-ZSTDLIBv07_API size_t      ZBUFFv07_freeDCtx(ZBUFFv07_DCtx* dctx);
-
-ZSTDLIBv07_API size_t ZBUFFv07_decompressInit(ZBUFFv07_DCtx* dctx);
-ZSTDLIBv07_API size_t ZBUFFv07_decompressInitDictionary(ZBUFFv07_DCtx* dctx, const void* dict, size_t dictSize);
-
-ZSTDLIBv07_API size_t ZBUFFv07_decompressContinue(ZBUFFv07_DCtx* dctx,
-                                            void* dst, size_t* dstCapacityPtr,
-                                      const void* src, size_t* srcSizePtr);
-
-/*-***************************************************************************
-*  Streaming decompression howto
-*
-*  A ZBUFFv07_DCtx object is required to track streaming operations.
-*  Use ZBUFFv07_createDCtx() and ZBUFFv07_freeDCtx() to create/release resources.
-*  Use ZBUFFv07_decompressInit() to start a new decompression operation,
-*   or ZBUFFv07_decompressInitDictionary() if decompression requires a dictionary.
-*  Note that ZBUFFv07_DCtx objects can be re-init multiple times.
-*
-*  Use ZBUFFv07_decompressContinue() repetitively to consume your input.
-*  *srcSizePtr and *dstCapacityPtr can be any size.
-*  The function will report how many bytes were read or written by modifying *srcSizePtr and *dstCapacityPtr.
-*  Note that it may not consume the entire input, in which case it's up to the caller to present remaining input again.
-*  The content of `dst` will be overwritten (up to *dstCapacityPtr) at each function call, so save its content if it matters, or change `dst`.
-*  @return : a hint to preferred nb of bytes to use as input for next function call (it's only a hint, to help latency),
-*            or 0 when a frame is completely decoded,
-*            or an error code, which can be tested using ZBUFFv07_isError().
-*
-*  Hint : recommended buffer sizes (not compulsory) : ZBUFFv07_recommendedDInSize() and ZBUFFv07_recommendedDOutSize()
-*  output : ZBUFFv07_recommendedDOutSize== 128 KB block size is the internal unit, it ensures it's always possible to write a full block when decoded.
-*  input  : ZBUFFv07_recommendedDInSize == 128KB + 3;
-*           just follow indications from ZBUFFv07_decompressContinue() to minimize latency. It should always be <= 128 KB + 3 .
-* *******************************************************************************/
-
-
-/* *************************************
-*  Tool functions
-***************************************/
-ZSTDLIBv07_API unsigned ZBUFFv07_isError(size_t errorCode);
-ZSTDLIBv07_API const char* ZBUFFv07_getErrorName(size_t errorCode);
-
-/** Functions below provide recommended buffer sizes for Compression or Decompression operations.
-*   These sizes are just hints, they tend to offer better latency */
-ZSTDLIBv07_API size_t ZBUFFv07_recommendedDInSize(void);
-ZSTDLIBv07_API size_t ZBUFFv07_recommendedDOutSize(void);
-
-
-/*-*************************************
-*  Constants
-***************************************/
-#define ZSTDv07_MAGICNUMBER            0xFD2FB527   /* v0.7 */
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif  /* ZSTDv07_H_235446 */
diff --git a/vendor/github.com/DataDog/zstd/zstdmt_compress.c b/vendor/github.com/DataDog/zstd/zstdmt_compress.c
deleted file mode 100644
index 9e537b8..0000000
--- a/vendor/github.com/DataDog/zstd/zstdmt_compress.c
+++ /dev/null
@@ -1,2110 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
-
-/* ======   Compiler specifics   ====== */
-#if defined(_MSC_VER)
-#  pragma warning(disable : 4204)   /* disable: C4204: non-constant aggregate initializer */
-#endif
-
-
-/* ======   Constants   ====== */
-#define ZSTDMT_OVERLAPLOG_DEFAULT 0
-
-
-/* ======   Dependencies   ====== */
-#include <string.h>      /* memcpy, memset */
-#include <limits.h>      /* INT_MAX, UINT_MAX */
-#include "mem.h"         /* MEM_STATIC */
-#include "pool.h"        /* threadpool */
-#include "threading.h"   /* mutex */
-#include "zstd_compress_internal.h"  /* MIN, ERROR, ZSTD_*, ZSTD_highbit32 */
-#include "zstd_ldm.h"
-#include "zstdmt_compress.h"
-
-/* Guards code to support resizing the SeqPool.
- * We will want to resize the SeqPool to save memory in the future.
- * Until then, comment the code out since it is unused.
- */
-#define ZSTD_RESIZE_SEQPOOL 0
-
-/* ======   Debug   ====== */
-#if defined(DEBUGLEVEL) && (DEBUGLEVEL>=2) \
-    && !defined(_MSC_VER) \
-    && !defined(__MINGW32__)
-
-#  include <stdio.h>
-#  include <unistd.h>
-#  include <sys/times.h>
-
-#  define DEBUG_PRINTHEX(l,p,n) {            \
-    unsigned debug_u;                        \
-    for (debug_u=0; debug_u<(n); debug_u++)  \
-        RAWLOG(l, "%02X ", ((const unsigned char*)(p))[debug_u]); \
-    RAWLOG(l, " \n");                        \
-}
-
-static unsigned long long GetCurrentClockTimeMicroseconds(void)
-{
-   static clock_t _ticksPerSecond = 0;
-   if (_ticksPerSecond <= 0) _ticksPerSecond = sysconf(_SC_CLK_TCK);
-
-   {   struct tms junk; clock_t newTicks = (clock_t) times(&junk);
-       return ((((unsigned long long)newTicks)*(1000000))/_ticksPerSecond);
-}  }
-
-#define MUTEX_WAIT_TIME_DLEVEL 6
-#define ZSTD_PTHREAD_MUTEX_LOCK(mutex) {          \
-    if (DEBUGLEVEL >= MUTEX_WAIT_TIME_DLEVEL) {   \
-        unsigned long long const beforeTime = GetCurrentClockTimeMicroseconds(); \
-        ZSTD_pthread_mutex_lock(mutex);           \
-        {   unsigned long long const afterTime = GetCurrentClockTimeMicroseconds(); \
-            unsigned long long const elapsedTime = (afterTime-beforeTime); \
-            if (elapsedTime > 1000) {  /* or whatever threshold you like; I'm using 1 millisecond here */ \
-                DEBUGLOG(MUTEX_WAIT_TIME_DLEVEL, "Thread took %llu microseconds to acquire mutex %s \n", \
-                   elapsedTime, #mutex);          \
-        }   }                                     \
-    } else {                                      \
-        ZSTD_pthread_mutex_lock(mutex);           \
-    }                                             \
-}
-
-#else
-
-#  define ZSTD_PTHREAD_MUTEX_LOCK(m) ZSTD_pthread_mutex_lock(m)
-#  define DEBUG_PRINTHEX(l,p,n) {}
-
-#endif
-
-
-/* =====   Buffer Pool   ===== */
-/* a single Buffer Pool can be invoked from multiple threads in parallel */
-
-typedef struct buffer_s {
-    void* start;
-    size_t capacity;
-} buffer_t;
-
-static const buffer_t g_nullBuffer = { NULL, 0 };
-
-typedef struct ZSTDMT_bufferPool_s {
-    ZSTD_pthread_mutex_t poolMutex;
-    size_t bufferSize;
-    unsigned totalBuffers;
-    unsigned nbBuffers;
-    ZSTD_customMem cMem;
-    buffer_t bTable[1];   /* variable size */
-} ZSTDMT_bufferPool;
-
-static ZSTDMT_bufferPool* ZSTDMT_createBufferPool(unsigned nbWorkers, ZSTD_customMem cMem)
-{
-    unsigned const maxNbBuffers = 2*nbWorkers + 3;
-    ZSTDMT_bufferPool* const bufPool = (ZSTDMT_bufferPool*)ZSTD_calloc(
-        sizeof(ZSTDMT_bufferPool) + (maxNbBuffers-1) * sizeof(buffer_t), cMem);
-    if (bufPool==NULL) return NULL;
-    if (ZSTD_pthread_mutex_init(&bufPool->poolMutex, NULL)) {
-        ZSTD_free(bufPool, cMem);
-        return NULL;
-    }
-    bufPool->bufferSize = 64 KB;
-    bufPool->totalBuffers = maxNbBuffers;
-    bufPool->nbBuffers = 0;
-    bufPool->cMem = cMem;
-    return bufPool;
-}
-
-static void ZSTDMT_freeBufferPool(ZSTDMT_bufferPool* bufPool)
-{
-    unsigned u;
-    DEBUGLOG(3, "ZSTDMT_freeBufferPool (address:%08X)", (U32)(size_t)bufPool);
-    if (!bufPool) return;   /* compatibility with free on NULL */
-    for (u=0; u<bufPool->totalBuffers; u++) {
-        DEBUGLOG(4, "free buffer %2u (address:%08X)", u, (U32)(size_t)bufPool->bTable[u].start);
-        ZSTD_free(bufPool->bTable[u].start, bufPool->cMem);
-    }
-    ZSTD_pthread_mutex_destroy(&bufPool->poolMutex);
-    ZSTD_free(bufPool, bufPool->cMem);
-}
-
-/* only works at initialization, not during compression */
-static size_t ZSTDMT_sizeof_bufferPool(ZSTDMT_bufferPool* bufPool)
-{
-    size_t const poolSize = sizeof(*bufPool)
-                          + (bufPool->totalBuffers - 1) * sizeof(buffer_t);
-    unsigned u;
-    size_t totalBufferSize = 0;
-    ZSTD_pthread_mutex_lock(&bufPool->poolMutex);
-    for (u=0; u<bufPool->totalBuffers; u++)
-        totalBufferSize += bufPool->bTable[u].capacity;
-    ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-
-    return poolSize + totalBufferSize;
-}
-
-/* ZSTDMT_setBufferSize() :
- * all future buffers provided by this buffer pool will have _at least_ this size
- * note : it's better for all buffers to have same size,
- * as they become freely interchangeable, reducing malloc/free usages and memory fragmentation */
-static void ZSTDMT_setBufferSize(ZSTDMT_bufferPool* const bufPool, size_t const bSize)
-{
-    ZSTD_pthread_mutex_lock(&bufPool->poolMutex);
-    DEBUGLOG(4, "ZSTDMT_setBufferSize: bSize = %u", (U32)bSize);
-    bufPool->bufferSize = bSize;
-    ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-}
-
-
-static ZSTDMT_bufferPool* ZSTDMT_expandBufferPool(ZSTDMT_bufferPool* srcBufPool, U32 nbWorkers)
-{
-    unsigned const maxNbBuffers = 2*nbWorkers + 3;
-    if (srcBufPool==NULL) return NULL;
-    if (srcBufPool->totalBuffers >= maxNbBuffers) /* good enough */
-        return srcBufPool;
-    /* need a larger buffer pool */
-    {   ZSTD_customMem const cMem = srcBufPool->cMem;
-        size_t const bSize = srcBufPool->bufferSize;   /* forward parameters */
-        ZSTDMT_bufferPool* newBufPool;
-        ZSTDMT_freeBufferPool(srcBufPool);
-        newBufPool = ZSTDMT_createBufferPool(nbWorkers, cMem);
-        if (newBufPool==NULL) return newBufPool;
-        ZSTDMT_setBufferSize(newBufPool, bSize);
-        return newBufPool;
-    }
-}
-
-/** ZSTDMT_getBuffer() :
- *  assumption : bufPool must be valid
- * @return : a buffer, with start pointer and size
- *  note: allocation may fail, in this case, start==NULL and size==0 */
-static buffer_t ZSTDMT_getBuffer(ZSTDMT_bufferPool* bufPool)
-{
-    size_t const bSize = bufPool->bufferSize;
-    DEBUGLOG(5, "ZSTDMT_getBuffer: bSize = %u", (U32)bufPool->bufferSize);
-    ZSTD_pthread_mutex_lock(&bufPool->poolMutex);
-    if (bufPool->nbBuffers) {   /* try to use an existing buffer */
-        buffer_t const buf = bufPool->bTable[--(bufPool->nbBuffers)];
-        size_t const availBufferSize = buf.capacity;
-        bufPool->bTable[bufPool->nbBuffers] = g_nullBuffer;
-        if ((availBufferSize >= bSize) & ((availBufferSize>>3) <= bSize)) {
-            /* large enough, but not too much */
-            DEBUGLOG(5, "ZSTDMT_getBuffer: provide buffer %u of size %u",
-                        bufPool->nbBuffers, (U32)buf.capacity);
-            ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-            return buf;
-        }
-        /* size conditions not respected : scratch this buffer, create new one */
-        DEBUGLOG(5, "ZSTDMT_getBuffer: existing buffer does not meet size conditions => freeing");
-        ZSTD_free(buf.start, bufPool->cMem);
-    }
-    ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-    /* create new buffer */
-    DEBUGLOG(5, "ZSTDMT_getBuffer: create a new buffer");
-    {   buffer_t buffer;
-        void* const start = ZSTD_malloc(bSize, bufPool->cMem);
-        buffer.start = start;   /* note : start can be NULL if malloc fails ! */
-        buffer.capacity = (start==NULL) ? 0 : bSize;
-        if (start==NULL) {
-            DEBUGLOG(5, "ZSTDMT_getBuffer: buffer allocation failure !!");
-        } else {
-            DEBUGLOG(5, "ZSTDMT_getBuffer: created buffer of size %u", (U32)bSize);
-        }
-        return buffer;
-    }
-}
-
-#if ZSTD_RESIZE_SEQPOOL
-/** ZSTDMT_resizeBuffer() :
- * assumption : bufPool must be valid
- * @return : a buffer that is at least the buffer pool buffer size.
- *           If a reallocation happens, the data in the input buffer is copied.
- */
-static buffer_t ZSTDMT_resizeBuffer(ZSTDMT_bufferPool* bufPool, buffer_t buffer)
-{
-    size_t const bSize = bufPool->bufferSize;
-    if (buffer.capacity < bSize) {
-        void* const start = ZSTD_malloc(bSize, bufPool->cMem);
-        buffer_t newBuffer;
-        newBuffer.start = start;
-        newBuffer.capacity = start == NULL ? 0 : bSize;
-        if (start != NULL) {
-            assert(newBuffer.capacity >= buffer.capacity);
-            memcpy(newBuffer.start, buffer.start, buffer.capacity);
-            DEBUGLOG(5, "ZSTDMT_resizeBuffer: created buffer of size %u", (U32)bSize);
-            return newBuffer;
-        }
-        DEBUGLOG(5, "ZSTDMT_resizeBuffer: buffer allocation failure !!");
-    }
-    return buffer;
-}
-#endif
-
-/* store buffer for later re-use, up to pool capacity */
-static void ZSTDMT_releaseBuffer(ZSTDMT_bufferPool* bufPool, buffer_t buf)
-{
-    DEBUGLOG(5, "ZSTDMT_releaseBuffer");
-    if (buf.start == NULL) return;   /* compatible with release on NULL */
-    ZSTD_pthread_mutex_lock(&bufPool->poolMutex);
-    if (bufPool->nbBuffers < bufPool->totalBuffers) {
-        bufPool->bTable[bufPool->nbBuffers++] = buf;  /* stored for later use */
-        DEBUGLOG(5, "ZSTDMT_releaseBuffer: stored buffer of size %u in slot %u",
-                    (U32)buf.capacity, (U32)(bufPool->nbBuffers-1));
-        ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-        return;
-    }
-    ZSTD_pthread_mutex_unlock(&bufPool->poolMutex);
-    /* Reached bufferPool capacity (should not happen) */
-    DEBUGLOG(5, "ZSTDMT_releaseBuffer: pool capacity reached => freeing ");
-    ZSTD_free(buf.start, bufPool->cMem);
-}
-
-
-/* =====   Seq Pool Wrapper   ====== */
-
-static rawSeqStore_t kNullRawSeqStore = {NULL, 0, 0, 0};
-
-typedef ZSTDMT_bufferPool ZSTDMT_seqPool;
-
-static size_t ZSTDMT_sizeof_seqPool(ZSTDMT_seqPool* seqPool)
-{
-    return ZSTDMT_sizeof_bufferPool(seqPool);
-}
-
-static rawSeqStore_t bufferToSeq(buffer_t buffer)
-{
-    rawSeqStore_t seq = {NULL, 0, 0, 0};
-    seq.seq = (rawSeq*)buffer.start;
-    seq.capacity = buffer.capacity / sizeof(rawSeq);
-    return seq;
-}
-
-static buffer_t seqToBuffer(rawSeqStore_t seq)
-{
-    buffer_t buffer;
-    buffer.start = seq.seq;
-    buffer.capacity = seq.capacity * sizeof(rawSeq);
-    return buffer;
-}
-
-static rawSeqStore_t ZSTDMT_getSeq(ZSTDMT_seqPool* seqPool)
-{
-    if (seqPool->bufferSize == 0) {
-        return kNullRawSeqStore;
-    }
-    return bufferToSeq(ZSTDMT_getBuffer(seqPool));
-}
-
-#if ZSTD_RESIZE_SEQPOOL
-static rawSeqStore_t ZSTDMT_resizeSeq(ZSTDMT_seqPool* seqPool, rawSeqStore_t seq)
-{
-  return bufferToSeq(ZSTDMT_resizeBuffer(seqPool, seqToBuffer(seq)));
-}
-#endif
-
-static void ZSTDMT_releaseSeq(ZSTDMT_seqPool* seqPool, rawSeqStore_t seq)
-{
-  ZSTDMT_releaseBuffer(seqPool, seqToBuffer(seq));
-}
-
-static void ZSTDMT_setNbSeq(ZSTDMT_seqPool* const seqPool, size_t const nbSeq)
-{
-  ZSTDMT_setBufferSize(seqPool, nbSeq * sizeof(rawSeq));
-}
-
-static ZSTDMT_seqPool* ZSTDMT_createSeqPool(unsigned nbWorkers, ZSTD_customMem cMem)
-{
-    ZSTDMT_seqPool* const seqPool = ZSTDMT_createBufferPool(nbWorkers, cMem);
-    if (seqPool == NULL) return NULL;
-    ZSTDMT_setNbSeq(seqPool, 0);
-    return seqPool;
-}
-
-static void ZSTDMT_freeSeqPool(ZSTDMT_seqPool* seqPool)
-{
-    ZSTDMT_freeBufferPool(seqPool);
-}
-
-static ZSTDMT_seqPool* ZSTDMT_expandSeqPool(ZSTDMT_seqPool* pool, U32 nbWorkers)
-{
-    return ZSTDMT_expandBufferPool(pool, nbWorkers);
-}
-
-
-/* =====   CCtx Pool   ===== */
-/* a single CCtx Pool can be invoked from multiple threads in parallel */
-
-typedef struct {
-    ZSTD_pthread_mutex_t poolMutex;
-    int totalCCtx;
-    int availCCtx;
-    ZSTD_customMem cMem;
-    ZSTD_CCtx* cctx[1];   /* variable size */
-} ZSTDMT_CCtxPool;
-
-/* note : all CCtx borrowed from the pool should be released back to the pool _before_ freeing the pool */
-static void ZSTDMT_freeCCtxPool(ZSTDMT_CCtxPool* pool)
-{
-    int cid;
-    for (cid=0; cid<pool->totalCCtx; cid++)
-        ZSTD_freeCCtx(pool->cctx[cid]);  /* note : compatible with free on NULL */
-    ZSTD_pthread_mutex_destroy(&pool->poolMutex);
-    ZSTD_free(pool, pool->cMem);
-}
-
-/* ZSTDMT_createCCtxPool() :
- * implies nbWorkers >= 1 , checked by caller ZSTDMT_createCCtx() */
-static ZSTDMT_CCtxPool* ZSTDMT_createCCtxPool(int nbWorkers,
-                                              ZSTD_customMem cMem)
-{
-    ZSTDMT_CCtxPool* const cctxPool = (ZSTDMT_CCtxPool*) ZSTD_calloc(
-        sizeof(ZSTDMT_CCtxPool) + (nbWorkers-1)*sizeof(ZSTD_CCtx*), cMem);
-    assert(nbWorkers > 0);
-    if (!cctxPool) return NULL;
-    if (ZSTD_pthread_mutex_init(&cctxPool->poolMutex, NULL)) {
-        ZSTD_free(cctxPool, cMem);
-        return NULL;
-    }
-    cctxPool->cMem = cMem;
-    cctxPool->totalCCtx = nbWorkers;
-    cctxPool->availCCtx = 1;   /* at least one cctx for single-thread mode */
-    cctxPool->cctx[0] = ZSTD_createCCtx_advanced(cMem);
-    if (!cctxPool->cctx[0]) { ZSTDMT_freeCCtxPool(cctxPool); return NULL; }
-    DEBUGLOG(3, "cctxPool created, with %u workers", nbWorkers);
-    return cctxPool;
-}
-
-static ZSTDMT_CCtxPool* ZSTDMT_expandCCtxPool(ZSTDMT_CCtxPool* srcPool,
-                                              int nbWorkers)
-{
-    if (srcPool==NULL) return NULL;
-    if (nbWorkers <= srcPool->totalCCtx) return srcPool;   /* good enough */
-    /* need a larger cctx pool */
-    {   ZSTD_customMem const cMem = srcPool->cMem;
-        ZSTDMT_freeCCtxPool(srcPool);
-        return ZSTDMT_createCCtxPool(nbWorkers, cMem);
-    }
-}
-
-/* only works during initialization phase, not during compression */
-static size_t ZSTDMT_sizeof_CCtxPool(ZSTDMT_CCtxPool* cctxPool)
-{
-    ZSTD_pthread_mutex_lock(&cctxPool->poolMutex);
-    {   unsigned const nbWorkers = cctxPool->totalCCtx;
-        size_t const poolSize = sizeof(*cctxPool)
-                                + (nbWorkers-1) * sizeof(ZSTD_CCtx*);
-        unsigned u;
-        size_t totalCCtxSize = 0;
-        for (u=0; u<nbWorkers; u++) {
-            totalCCtxSize += ZSTD_sizeof_CCtx(cctxPool->cctx[u]);
-        }
-        ZSTD_pthread_mutex_unlock(&cctxPool->poolMutex);
-        assert(nbWorkers > 0);
-        return poolSize + totalCCtxSize;
-    }
-}
-
-static ZSTD_CCtx* ZSTDMT_getCCtx(ZSTDMT_CCtxPool* cctxPool)
-{
-    DEBUGLOG(5, "ZSTDMT_getCCtx");
-    ZSTD_pthread_mutex_lock(&cctxPool->poolMutex);
-    if (cctxPool->availCCtx) {
-        cctxPool->availCCtx--;
-        {   ZSTD_CCtx* const cctx = cctxPool->cctx[cctxPool->availCCtx];
-            ZSTD_pthread_mutex_unlock(&cctxPool->poolMutex);
-            return cctx;
-    }   }
-    ZSTD_pthread_mutex_unlock(&cctxPool->poolMutex);
-    DEBUGLOG(5, "create one more CCtx");
-    return ZSTD_createCCtx_advanced(cctxPool->cMem);   /* note : can be NULL, when creation fails ! */
-}
-
-static void ZSTDMT_releaseCCtx(ZSTDMT_CCtxPool* pool, ZSTD_CCtx* cctx)
-{
-    if (cctx==NULL) return;   /* compatibility with release on NULL */
-    ZSTD_pthread_mutex_lock(&pool->poolMutex);
-    if (pool->availCCtx < pool->totalCCtx)
-        pool->cctx[pool->availCCtx++] = cctx;
-    else {
-        /* pool overflow : should not happen, since totalCCtx==nbWorkers */
-        DEBUGLOG(4, "CCtx pool overflow : free cctx");
-        ZSTD_freeCCtx(cctx);
-    }
-    ZSTD_pthread_mutex_unlock(&pool->poolMutex);
-}
-
-/* ====   Serial State   ==== */
-
-typedef struct {
-    void const* start;
-    size_t size;
-} range_t;
-
-typedef struct {
-    /* All variables in the struct are protected by mutex. */
-    ZSTD_pthread_mutex_t mutex;
-    ZSTD_pthread_cond_t cond;
-    ZSTD_CCtx_params params;
-    ldmState_t ldmState;
-    XXH64_state_t xxhState;
-    unsigned nextJobID;
-    /* Protects ldmWindow.
-     * Must be acquired after the main mutex when acquiring both.
-     */
-    ZSTD_pthread_mutex_t ldmWindowMutex;
-    ZSTD_pthread_cond_t ldmWindowCond;  /* Signaled when ldmWindow is updated */
-    ZSTD_window_t ldmWindow;  /* A thread-safe copy of ldmState.window */
-} serialState_t;
-
-static int ZSTDMT_serialState_reset(serialState_t* serialState, ZSTDMT_seqPool* seqPool, ZSTD_CCtx_params params, size_t jobSize)
-{
-    /* Adjust parameters */
-    if (params.ldmParams.enableLdm) {
-        DEBUGLOG(4, "LDM window size = %u KB", (1U << params.cParams.windowLog) >> 10);
-        ZSTD_ldm_adjustParameters(&params.ldmParams, &params.cParams);
-        assert(params.ldmParams.hashLog >= params.ldmParams.bucketSizeLog);
-        assert(params.ldmParams.hashRateLog < 32);
-        serialState->ldmState.hashPower =
-                ZSTD_rollingHash_primePower(params.ldmParams.minMatchLength);
-    } else {
-        memset(&params.ldmParams, 0, sizeof(params.ldmParams));
-    }
-    serialState->nextJobID = 0;
-    if (params.fParams.checksumFlag)
-        XXH64_reset(&serialState->xxhState, 0);
-    if (params.ldmParams.enableLdm) {
-        ZSTD_customMem cMem = params.customMem;
-        unsigned const hashLog = params.ldmParams.hashLog;
-        size_t const hashSize = ((size_t)1 << hashLog) * sizeof(ldmEntry_t);
-        unsigned const bucketLog =
-            params.ldmParams.hashLog - params.ldmParams.bucketSizeLog;
-        size_t const bucketSize = (size_t)1 << bucketLog;
-        unsigned const prevBucketLog =
-            serialState->params.ldmParams.hashLog -
-            serialState->params.ldmParams.bucketSizeLog;
-        /* Size the seq pool tables */
-        ZSTDMT_setNbSeq(seqPool, ZSTD_ldm_getMaxNbSeq(params.ldmParams, jobSize));
-        /* Reset the window */
-        ZSTD_window_clear(&serialState->ldmState.window);
-        serialState->ldmWindow = serialState->ldmState.window;
-        /* Resize tables and output space if necessary. */
-        if (serialState->ldmState.hashTable == NULL || serialState->params.ldmParams.hashLog < hashLog) {
-            ZSTD_free(serialState->ldmState.hashTable, cMem);
-            serialState->ldmState.hashTable = (ldmEntry_t*)ZSTD_malloc(hashSize, cMem);
-        }
-        if (serialState->ldmState.bucketOffsets == NULL || prevBucketLog < bucketLog) {
-            ZSTD_free(serialState->ldmState.bucketOffsets, cMem);
-            serialState->ldmState.bucketOffsets = (BYTE*)ZSTD_malloc(bucketSize, cMem);
-        }
-        if (!serialState->ldmState.hashTable || !serialState->ldmState.bucketOffsets)
-            return 1;
-        /* Zero the tables */
-        memset(serialState->ldmState.hashTable, 0, hashSize);
-        memset(serialState->ldmState.bucketOffsets, 0, bucketSize);
-    }
-    serialState->params = params;
-    serialState->params.jobSize = (U32)jobSize;
-    return 0;
-}
-
-static int ZSTDMT_serialState_init(serialState_t* serialState)
-{
-    int initError = 0;
-    memset(serialState, 0, sizeof(*serialState));
-    initError |= ZSTD_pthread_mutex_init(&serialState->mutex, NULL);
-    initError |= ZSTD_pthread_cond_init(&serialState->cond, NULL);
-    initError |= ZSTD_pthread_mutex_init(&serialState->ldmWindowMutex, NULL);
-    initError |= ZSTD_pthread_cond_init(&serialState->ldmWindowCond, NULL);
-    return initError;
-}
-
-static void ZSTDMT_serialState_free(serialState_t* serialState)
-{
-    ZSTD_customMem cMem = serialState->params.customMem;
-    ZSTD_pthread_mutex_destroy(&serialState->mutex);
-    ZSTD_pthread_cond_destroy(&serialState->cond);
-    ZSTD_pthread_mutex_destroy(&serialState->ldmWindowMutex);
-    ZSTD_pthread_cond_destroy(&serialState->ldmWindowCond);
-    ZSTD_free(serialState->ldmState.hashTable, cMem);
-    ZSTD_free(serialState->ldmState.bucketOffsets, cMem);
-}
-
-static void ZSTDMT_serialState_update(serialState_t* serialState,
-                                      ZSTD_CCtx* jobCCtx, rawSeqStore_t seqStore,
-                                      range_t src, unsigned jobID)
-{
-    /* Wait for our turn */
-    ZSTD_PTHREAD_MUTEX_LOCK(&serialState->mutex);
-    while (serialState->nextJobID < jobID) {
-        DEBUGLOG(5, "wait for serialState->cond");
-        ZSTD_pthread_cond_wait(&serialState->cond, &serialState->mutex);
-    }
-    /* A future job may error and skip our job */
-    if (serialState->nextJobID == jobID) {
-        /* It is now our turn, do any processing necessary */
-        if (serialState->params.ldmParams.enableLdm) {
-            size_t error;
-            assert(seqStore.seq != NULL && seqStore.pos == 0 &&
-                   seqStore.size == 0 && seqStore.capacity > 0);
-            assert(src.size <= serialState->params.jobSize);
-            ZSTD_window_update(&serialState->ldmState.window, src.start, src.size);
-            error = ZSTD_ldm_generateSequences(
-                &serialState->ldmState, &seqStore,
-                &serialState->params.ldmParams, src.start, src.size);
-            /* We provide a large enough buffer to never fail. */
-            assert(!ZSTD_isError(error)); (void)error;
-            /* Update ldmWindow to match the ldmState.window and signal the main
-             * thread if it is waiting for a buffer.
-             */
-            ZSTD_PTHREAD_MUTEX_LOCK(&serialState->ldmWindowMutex);
-            serialState->ldmWindow = serialState->ldmState.window;
-            ZSTD_pthread_cond_signal(&serialState->ldmWindowCond);
-            ZSTD_pthread_mutex_unlock(&serialState->ldmWindowMutex);
-        }
-        if (serialState->params.fParams.checksumFlag && src.size > 0)
-            XXH64_update(&serialState->xxhState, src.start, src.size);
-    }
-    /* Now it is the next jobs turn */
-    serialState->nextJobID++;
-    ZSTD_pthread_cond_broadcast(&serialState->cond);
-    ZSTD_pthread_mutex_unlock(&serialState->mutex);
-
-    if (seqStore.size > 0) {
-        size_t const err = ZSTD_referenceExternalSequences(
-            jobCCtx, seqStore.seq, seqStore.size);
-        assert(serialState->params.ldmParams.enableLdm);
-        assert(!ZSTD_isError(err));
-        (void)err;
-    }
-}
-
-static void ZSTDMT_serialState_ensureFinished(serialState_t* serialState,
-                                              unsigned jobID, size_t cSize)
-{
-    ZSTD_PTHREAD_MUTEX_LOCK(&serialState->mutex);
-    if (serialState->nextJobID <= jobID) {
-        assert(ZSTD_isError(cSize)); (void)cSize;
-        DEBUGLOG(5, "Skipping past job %u because of error", jobID);
-        serialState->nextJobID = jobID + 1;
-        ZSTD_pthread_cond_broadcast(&serialState->cond);
-
-        ZSTD_PTHREAD_MUTEX_LOCK(&serialState->ldmWindowMutex);
-        ZSTD_window_clear(&serialState->ldmWindow);
-        ZSTD_pthread_cond_signal(&serialState->ldmWindowCond);
-        ZSTD_pthread_mutex_unlock(&serialState->ldmWindowMutex);
-    }
-    ZSTD_pthread_mutex_unlock(&serialState->mutex);
-
-}
-
-
-/* ------------------------------------------ */
-/* =====          Worker thread         ===== */
-/* ------------------------------------------ */
-
-static const range_t kNullRange = { NULL, 0 };
-
-typedef struct {
-    size_t   consumed;                   /* SHARED - set0 by mtctx, then modified by worker AND read by mtctx */
-    size_t   cSize;                      /* SHARED - set0 by mtctx, then modified by worker AND read by mtctx, then set0 by mtctx */
-    ZSTD_pthread_mutex_t job_mutex;      /* Thread-safe - used by mtctx and worker */
-    ZSTD_pthread_cond_t job_cond;        /* Thread-safe - used by mtctx and worker */
-    ZSTDMT_CCtxPool* cctxPool;           /* Thread-safe - used by mtctx and (all) workers */
-    ZSTDMT_bufferPool* bufPool;          /* Thread-safe - used by mtctx and (all) workers */
-    ZSTDMT_seqPool* seqPool;             /* Thread-safe - used by mtctx and (all) workers */
-    serialState_t* serial;               /* Thread-safe - used by mtctx and (all) workers */
-    buffer_t dstBuff;                    /* set by worker (or mtctx), then read by worker & mtctx, then modified by mtctx => no barrier */
-    range_t prefix;                      /* set by mtctx, then read by worker & mtctx => no barrier */
-    range_t src;                         /* set by mtctx, then read by worker & mtctx => no barrier */
-    unsigned jobID;                      /* set by mtctx, then read by worker => no barrier */
-    unsigned firstJob;                   /* set by mtctx, then read by worker => no barrier */
-    unsigned lastJob;                    /* set by mtctx, then read by worker => no barrier */
-    ZSTD_CCtx_params params;             /* set by mtctx, then read by worker => no barrier */
-    const ZSTD_CDict* cdict;             /* set by mtctx, then read by worker => no barrier */
-    unsigned long long fullFrameSize;    /* set by mtctx, then read by worker => no barrier */
-    size_t   dstFlushed;                 /* used only by mtctx */
-    unsigned frameChecksumNeeded;        /* used only by mtctx */
-} ZSTDMT_jobDescription;
-
-#define JOB_ERROR(e) {                          \
-    ZSTD_PTHREAD_MUTEX_LOCK(&job->job_mutex);   \
-    job->cSize = e;                             \
-    ZSTD_pthread_mutex_unlock(&job->job_mutex); \
-    goto _endJob;                               \
-}
-
-/* ZSTDMT_compressionJob() is a POOL_function type */
-static void ZSTDMT_compressionJob(void* jobDescription)
-{
-    ZSTDMT_jobDescription* const job = (ZSTDMT_jobDescription*)jobDescription;
-    ZSTD_CCtx_params jobParams = job->params;   /* do not modify job->params ! copy it, modify the copy */
-    ZSTD_CCtx* const cctx = ZSTDMT_getCCtx(job->cctxPool);
-    rawSeqStore_t rawSeqStore = ZSTDMT_getSeq(job->seqPool);
-    buffer_t dstBuff = job->dstBuff;
-    size_t lastCBlockSize = 0;
-
-    /* resources */
-    if (cctx==NULL) JOB_ERROR(ERROR(memory_allocation));
-    if (dstBuff.start == NULL) {   /* streaming job : doesn't provide a dstBuffer */
-        dstBuff = ZSTDMT_getBuffer(job->bufPool);
-        if (dstBuff.start==NULL) JOB_ERROR(ERROR(memory_allocation));
-        job->dstBuff = dstBuff;   /* this value can be read in ZSTDMT_flush, when it copies the whole job */
-    }
-    if (jobParams.ldmParams.enableLdm && rawSeqStore.seq == NULL)
-        JOB_ERROR(ERROR(memory_allocation));
-
-    /* Don't compute the checksum for chunks, since we compute it externally,
-     * but write it in the header.
-     */
-    if (job->jobID != 0) jobParams.fParams.checksumFlag = 0;
-    /* Don't run LDM for the chunks, since we handle it externally */
-    jobParams.ldmParams.enableLdm = 0;
-
-
-    /* init */
-    if (job->cdict) {
-        size_t const initError = ZSTD_compressBegin_advanced_internal(cctx, NULL, 0, ZSTD_dct_auto, ZSTD_dtlm_fast, job->cdict, jobParams, job->fullFrameSize);
-        assert(job->firstJob);  /* only allowed for first job */
-        if (ZSTD_isError(initError)) JOB_ERROR(initError);
-    } else {  /* srcStart points at reloaded section */
-        U64 const pledgedSrcSize = job->firstJob ? job->fullFrameSize : job->src.size;
-        {   size_t const forceWindowError = ZSTD_CCtxParams_setParameter(&jobParams, ZSTD_c_forceMaxWindow, !job->firstJob);
-            if (ZSTD_isError(forceWindowError)) JOB_ERROR(forceWindowError);
-        }
-        {   size_t const initError = ZSTD_compressBegin_advanced_internal(cctx,
-                                        job->prefix.start, job->prefix.size, ZSTD_dct_rawContent, /* load dictionary in "content-only" mode (no header analysis) */
-                                        ZSTD_dtlm_fast,
-                                        NULL, /*cdict*/
-                                        jobParams, pledgedSrcSize);
-            if (ZSTD_isError(initError)) JOB_ERROR(initError);
-    }   }
-
-    /* Perform serial step as early as possible, but after CCtx initialization */
-    ZSTDMT_serialState_update(job->serial, cctx, rawSeqStore, job->src, job->jobID);
-
-    if (!job->firstJob) {  /* flush and overwrite frame header when it's not first job */
-        size_t const hSize = ZSTD_compressContinue(cctx, dstBuff.start, dstBuff.capacity, job->src.start, 0);
-        if (ZSTD_isError(hSize)) JOB_ERROR(hSize);
-        DEBUGLOG(5, "ZSTDMT_compressionJob: flush and overwrite %u bytes of frame header (not first job)", (U32)hSize);
-        ZSTD_invalidateRepCodes(cctx);
-    }
-
-    /* compress */
-    {   size_t const chunkSize = 4*ZSTD_BLOCKSIZE_MAX;
-        int const nbChunks = (int)((job->src.size + (chunkSize-1)) / chunkSize);
-        const BYTE* ip = (const BYTE*) job->src.start;
-        BYTE* const ostart = (BYTE*)dstBuff.start;
-        BYTE* op = ostart;
-        BYTE* oend = op + dstBuff.capacity;
-        int chunkNb;
-        if (sizeof(size_t) > sizeof(int)) assert(job->src.size < ((size_t)INT_MAX) * chunkSize);   /* check overflow */
-        DEBUGLOG(5, "ZSTDMT_compressionJob: compress %u bytes in %i blocks", (U32)job->src.size, nbChunks);
-        assert(job->cSize == 0);
-        for (chunkNb = 1; chunkNb < nbChunks; chunkNb++) {
-            size_t const cSize = ZSTD_compressContinue(cctx, op, oend-op, ip, chunkSize);
-            if (ZSTD_isError(cSize)) JOB_ERROR(cSize);
-            ip += chunkSize;
-            op += cSize; assert(op < oend);
-            /* stats */
-            ZSTD_PTHREAD_MUTEX_LOCK(&job->job_mutex);
-            job->cSize += cSize;
-            job->consumed = chunkSize * chunkNb;
-            DEBUGLOG(5, "ZSTDMT_compressionJob: compress new block : cSize==%u bytes (total: %u)",
-                        (U32)cSize, (U32)job->cSize);
-            ZSTD_pthread_cond_signal(&job->job_cond);   /* warns some more data is ready to be flushed */
-            ZSTD_pthread_mutex_unlock(&job->job_mutex);
-        }
-        /* last block */
-        assert(chunkSize > 0);
-        assert((chunkSize & (chunkSize - 1)) == 0);  /* chunkSize must be power of 2 for mask==(chunkSize-1) to work */
-        if ((nbChunks > 0) | job->lastJob /*must output a "last block" flag*/ ) {
-            size_t const lastBlockSize1 = job->src.size & (chunkSize-1);
-            size_t const lastBlockSize = ((lastBlockSize1==0) & (job->src.size>=chunkSize)) ? chunkSize : lastBlockSize1;
-            size_t const cSize = (job->lastJob) ?
-                 ZSTD_compressEnd     (cctx, op, oend-op, ip, lastBlockSize) :
-                 ZSTD_compressContinue(cctx, op, oend-op, ip, lastBlockSize);
-            if (ZSTD_isError(cSize)) JOB_ERROR(cSize);
-            lastCBlockSize = cSize;
-    }   }
-
-_endJob:
-    ZSTDMT_serialState_ensureFinished(job->serial, job->jobID, job->cSize);
-    if (job->prefix.size > 0)
-        DEBUGLOG(5, "Finished with prefix: %zx", (size_t)job->prefix.start);
-    DEBUGLOG(5, "Finished with source: %zx", (size_t)job->src.start);
-    /* release resources */
-    ZSTDMT_releaseSeq(job->seqPool, rawSeqStore);
-    ZSTDMT_releaseCCtx(job->cctxPool, cctx);
-    /* report */
-    ZSTD_PTHREAD_MUTEX_LOCK(&job->job_mutex);
-    if (ZSTD_isError(job->cSize)) assert(lastCBlockSize == 0);
-    job->cSize += lastCBlockSize;
-    job->consumed = job->src.size;  /* when job->consumed == job->src.size , compression job is presumed completed */
-    ZSTD_pthread_cond_signal(&job->job_cond);
-    ZSTD_pthread_mutex_unlock(&job->job_mutex);
-}
-
-
-/* ------------------------------------------ */
-/* =====   Multi-threaded compression   ===== */
-/* ------------------------------------------ */
-
-typedef struct {
-    range_t prefix;         /* read-only non-owned prefix buffer */
-    buffer_t buffer;
-    size_t filled;
-} inBuff_t;
-
-typedef struct {
-  BYTE* buffer;     /* The round input buffer. All jobs get references
-                     * to pieces of the buffer. ZSTDMT_tryGetInputRange()
-                     * handles handing out job input buffers, and makes
-                     * sure it doesn't overlap with any pieces still in use.
-                     */
-  size_t capacity;  /* The capacity of buffer. */
-  size_t pos;       /* The position of the current inBuff in the round
-                     * buffer. Updated past the end if the inBuff once
-                     * the inBuff is sent to the worker thread.
-                     * pos <= capacity.
-                     */
-} roundBuff_t;
-
-static const roundBuff_t kNullRoundBuff = {NULL, 0, 0};
-
-#define RSYNC_LENGTH 32
-
-typedef struct {
-  U64 hash;
-  U64 hitMask;
-  U64 primePower;
-} rsyncState_t;
-
-struct ZSTDMT_CCtx_s {
-    POOL_ctx* factory;
-    ZSTDMT_jobDescription* jobs;
-    ZSTDMT_bufferPool* bufPool;
-    ZSTDMT_CCtxPool* cctxPool;
-    ZSTDMT_seqPool* seqPool;
-    ZSTD_CCtx_params params;
-    size_t targetSectionSize;
-    size_t targetPrefixSize;
-    int jobReady;        /* 1 => one job is already prepared, but pool has shortage of workers. Don't create a new job. */
-    inBuff_t inBuff;
-    roundBuff_t roundBuff;
-    serialState_t serial;
-    rsyncState_t rsync;
-    unsigned singleBlockingThread;
-    unsigned jobIDMask;
-    unsigned doneJobID;
-    unsigned nextJobID;
-    unsigned frameEnded;
-    unsigned allJobsCompleted;
-    unsigned long long frameContentSize;
-    unsigned long long consumed;
-    unsigned long long produced;
-    ZSTD_customMem cMem;
-    ZSTD_CDict* cdictLocal;
-    const ZSTD_CDict* cdict;
-};
-
-static void ZSTDMT_freeJobsTable(ZSTDMT_jobDescription* jobTable, U32 nbJobs, ZSTD_customMem cMem)
-{
-    U32 jobNb;
-    if (jobTable == NULL) return;
-    for (jobNb=0; jobNb<nbJobs; jobNb++) {
-        ZSTD_pthread_mutex_destroy(&jobTable[jobNb].job_mutex);
-        ZSTD_pthread_cond_destroy(&jobTable[jobNb].job_cond);
-    }
-    ZSTD_free(jobTable, cMem);
-}
-
-/* ZSTDMT_allocJobsTable()
- * allocate and init a job table.
- * update *nbJobsPtr to next power of 2 value, as size of table */
-static ZSTDMT_jobDescription* ZSTDMT_createJobsTable(U32* nbJobsPtr, ZSTD_customMem cMem)
-{
-    U32 const nbJobsLog2 = ZSTD_highbit32(*nbJobsPtr) + 1;
-    U32 const nbJobs = 1 << nbJobsLog2;
-    U32 jobNb;
-    ZSTDMT_jobDescription* const jobTable = (ZSTDMT_jobDescription*)
-                ZSTD_calloc(nbJobs * sizeof(ZSTDMT_jobDescription), cMem);
-    int initError = 0;
-    if (jobTable==NULL) return NULL;
-    *nbJobsPtr = nbJobs;
-    for (jobNb=0; jobNb<nbJobs; jobNb++) {
-        initError |= ZSTD_pthread_mutex_init(&jobTable[jobNb].job_mutex, NULL);
-        initError |= ZSTD_pthread_cond_init(&jobTable[jobNb].job_cond, NULL);
-    }
-    if (initError != 0) {
-        ZSTDMT_freeJobsTable(jobTable, nbJobs, cMem);
-        return NULL;
-    }
-    return jobTable;
-}
-
-static size_t ZSTDMT_expandJobsTable (ZSTDMT_CCtx* mtctx, U32 nbWorkers) {
-    U32 nbJobs = nbWorkers + 2;
-    if (nbJobs > mtctx->jobIDMask+1) {  /* need more job capacity */
-        ZSTDMT_freeJobsTable(mtctx->jobs, mtctx->jobIDMask+1, mtctx->cMem);
-        mtctx->jobIDMask = 0;
-        mtctx->jobs = ZSTDMT_createJobsTable(&nbJobs, mtctx->cMem);
-        if (mtctx->jobs==NULL) return ERROR(memory_allocation);
-        assert((nbJobs != 0) && ((nbJobs & (nbJobs - 1)) == 0));  /* ensure nbJobs is a power of 2 */
-        mtctx->jobIDMask = nbJobs - 1;
-    }
-    return 0;
-}
-
-
-/* ZSTDMT_CCtxParam_setNbWorkers():
- * Internal use only */
-size_t ZSTDMT_CCtxParam_setNbWorkers(ZSTD_CCtx_params* params, unsigned nbWorkers)
-{
-    return ZSTD_CCtxParams_setParameter(params, ZSTD_c_nbWorkers, (int)nbWorkers);
-}
-
-MEM_STATIC ZSTDMT_CCtx* ZSTDMT_createCCtx_advanced_internal(unsigned nbWorkers, ZSTD_customMem cMem)
-{
-    ZSTDMT_CCtx* mtctx;
-    U32 nbJobs = nbWorkers + 2;
-    int initError;
-    DEBUGLOG(3, "ZSTDMT_createCCtx_advanced (nbWorkers = %u)", nbWorkers);
-
-    if (nbWorkers < 1) return NULL;
-    nbWorkers = MIN(nbWorkers , ZSTDMT_NBWORKERS_MAX);
-    if ((cMem.customAlloc!=NULL) ^ (cMem.customFree!=NULL))
-        /* invalid custom allocator */
-        return NULL;
-
-    mtctx = (ZSTDMT_CCtx*) ZSTD_calloc(sizeof(ZSTDMT_CCtx), cMem);
-    if (!mtctx) return NULL;
-    ZSTDMT_CCtxParam_setNbWorkers(&mtctx->params, nbWorkers);
-    mtctx->cMem = cMem;
-    mtctx->allJobsCompleted = 1;
-    mtctx->factory = POOL_create_advanced(nbWorkers, 0, cMem);
-    mtctx->jobs = ZSTDMT_createJobsTable(&nbJobs, cMem);
-    assert(nbJobs > 0); assert((nbJobs & (nbJobs - 1)) == 0);  /* ensure nbJobs is a power of 2 */
-    mtctx->jobIDMask = nbJobs - 1;
-    mtctx->bufPool = ZSTDMT_createBufferPool(nbWorkers, cMem);
-    mtctx->cctxPool = ZSTDMT_createCCtxPool(nbWorkers, cMem);
-    mtctx->seqPool = ZSTDMT_createSeqPool(nbWorkers, cMem);
-    initError = ZSTDMT_serialState_init(&mtctx->serial);
-    mtctx->roundBuff = kNullRoundBuff;
-    if (!mtctx->factory | !mtctx->jobs | !mtctx->bufPool | !mtctx->cctxPool | !mtctx->seqPool | initError) {
-        ZSTDMT_freeCCtx(mtctx);
-        return NULL;
-    }
-    DEBUGLOG(3, "mt_cctx created, for %u threads", nbWorkers);
-    return mtctx;
-}
-
-ZSTDMT_CCtx* ZSTDMT_createCCtx_advanced(unsigned nbWorkers, ZSTD_customMem cMem)
-{
-#ifdef ZSTD_MULTITHREAD
-    return ZSTDMT_createCCtx_advanced_internal(nbWorkers, cMem);
-#else
-    (void)nbWorkers;
-    (void)cMem;
-    return NULL;
-#endif
-}
-
-ZSTDMT_CCtx* ZSTDMT_createCCtx(unsigned nbWorkers)
-{
-    return ZSTDMT_createCCtx_advanced(nbWorkers, ZSTD_defaultCMem);
-}
-
-
-/* ZSTDMT_releaseAllJobResources() :
- * note : ensure all workers are killed first ! */
-static void ZSTDMT_releaseAllJobResources(ZSTDMT_CCtx* mtctx)
-{
-    unsigned jobID;
-    DEBUGLOG(3, "ZSTDMT_releaseAllJobResources");
-    for (jobID=0; jobID <= mtctx->jobIDMask; jobID++) {
-        DEBUGLOG(4, "job%02u: release dst address %08X", jobID, (U32)(size_t)mtctx->jobs[jobID].dstBuff.start);
-        ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[jobID].dstBuff);
-        mtctx->jobs[jobID].dstBuff = g_nullBuffer;
-        mtctx->jobs[jobID].cSize = 0;
-    }
-    memset(mtctx->jobs, 0, (mtctx->jobIDMask+1)*sizeof(ZSTDMT_jobDescription));
-    mtctx->inBuff.buffer = g_nullBuffer;
-    mtctx->inBuff.filled = 0;
-    mtctx->allJobsCompleted = 1;
-}
-
-static void ZSTDMT_waitForAllJobsCompleted(ZSTDMT_CCtx* mtctx)
-{
-    DEBUGLOG(4, "ZSTDMT_waitForAllJobsCompleted");
-    while (mtctx->doneJobID < mtctx->nextJobID) {
-        unsigned const jobID = mtctx->doneJobID & mtctx->jobIDMask;
-        ZSTD_PTHREAD_MUTEX_LOCK(&mtctx->jobs[jobID].job_mutex);
-        while (mtctx->jobs[jobID].consumed < mtctx->jobs[jobID].src.size) {
-            DEBUGLOG(4, "waiting for jobCompleted signal from job %u", mtctx->doneJobID);   /* we want to block when waiting for data to flush */
-            ZSTD_pthread_cond_wait(&mtctx->jobs[jobID].job_cond, &mtctx->jobs[jobID].job_mutex);
-        }
-        ZSTD_pthread_mutex_unlock(&mtctx->jobs[jobID].job_mutex);
-        mtctx->doneJobID++;
-    }
-}
-
-size_t ZSTDMT_freeCCtx(ZSTDMT_CCtx* mtctx)
-{
-    if (mtctx==NULL) return 0;   /* compatible with free on NULL */
-    POOL_free(mtctx->factory);   /* stop and free worker threads */
-    ZSTDMT_releaseAllJobResources(mtctx);  /* release job resources into pools first */
-    ZSTDMT_freeJobsTable(mtctx->jobs, mtctx->jobIDMask+1, mtctx->cMem);
-    ZSTDMT_freeBufferPool(mtctx->bufPool);
-    ZSTDMT_freeCCtxPool(mtctx->cctxPool);
-    ZSTDMT_freeSeqPool(mtctx->seqPool);
-    ZSTDMT_serialState_free(&mtctx->serial);
-    ZSTD_freeCDict(mtctx->cdictLocal);
-    if (mtctx->roundBuff.buffer)
-        ZSTD_free(mtctx->roundBuff.buffer, mtctx->cMem);
-    ZSTD_free(mtctx, mtctx->cMem);
-    return 0;
-}
-
-size_t ZSTDMT_sizeof_CCtx(ZSTDMT_CCtx* mtctx)
-{
-    if (mtctx == NULL) return 0;   /* supports sizeof NULL */
-    return sizeof(*mtctx)
-            + POOL_sizeof(mtctx->factory)
-            + ZSTDMT_sizeof_bufferPool(mtctx->bufPool)
-            + (mtctx->jobIDMask+1) * sizeof(ZSTDMT_jobDescription)
-            + ZSTDMT_sizeof_CCtxPool(mtctx->cctxPool)
-            + ZSTDMT_sizeof_seqPool(mtctx->seqPool)
-            + ZSTD_sizeof_CDict(mtctx->cdictLocal)
-            + mtctx->roundBuff.capacity;
-}
-
-/* Internal only */
-size_t
-ZSTDMT_CCtxParam_setMTCtxParameter(ZSTD_CCtx_params* params,
-                                   ZSTDMT_parameter parameter,
-                                   int value)
-{
-    DEBUGLOG(4, "ZSTDMT_CCtxParam_setMTCtxParameter");
-    switch(parameter)
-    {
-    case ZSTDMT_p_jobSize :
-        DEBUGLOG(4, "ZSTDMT_CCtxParam_setMTCtxParameter : set jobSize to %i", value);
-        return ZSTD_CCtxParams_setParameter(params, ZSTD_c_jobSize, value);
-    case ZSTDMT_p_overlapLog :
-        DEBUGLOG(4, "ZSTDMT_p_overlapLog : %i", value);
-        return ZSTD_CCtxParams_setParameter(params, ZSTD_c_overlapLog, value);
-    case ZSTDMT_p_rsyncable :
-        DEBUGLOG(4, "ZSTD_p_rsyncable : %i", value);
-        return ZSTD_CCtxParams_setParameter(params, ZSTD_c_rsyncable, value);
-    default :
-        return ERROR(parameter_unsupported);
-    }
-}
-
-size_t ZSTDMT_setMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSTDMT_parameter parameter, int value)
-{
-    DEBUGLOG(4, "ZSTDMT_setMTCtxParameter");
-    return ZSTDMT_CCtxParam_setMTCtxParameter(&mtctx->params, parameter, value);
-}
-
-size_t ZSTDMT_getMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSTDMT_parameter parameter, int* value)
-{
-    switch (parameter) {
-    case ZSTDMT_p_jobSize:
-        return ZSTD_CCtxParams_getParameter(&mtctx->params, ZSTD_c_jobSize, value);
-    case ZSTDMT_p_overlapLog:
-        return ZSTD_CCtxParams_getParameter(&mtctx->params, ZSTD_c_overlapLog, value);
-    case ZSTDMT_p_rsyncable:
-        return ZSTD_CCtxParams_getParameter(&mtctx->params, ZSTD_c_rsyncable, value);
-    default:
-        return ERROR(parameter_unsupported);
-    }
-}
-
-/* Sets parameters relevant to the compression job,
- * initializing others to default values. */
-static ZSTD_CCtx_params ZSTDMT_initJobCCtxParams(ZSTD_CCtx_params const params)
-{
-    ZSTD_CCtx_params jobParams = params;
-    /* Clear parameters related to multithreading */
-    jobParams.forceWindow = 0;
-    jobParams.nbWorkers = 0;
-    jobParams.jobSize = 0;
-    jobParams.overlapLog = 0;
-    jobParams.rsyncable = 0;
-    memset(&jobParams.ldmParams, 0, sizeof(ldmParams_t));
-    memset(&jobParams.customMem, 0, sizeof(ZSTD_customMem));
-    return jobParams;
-}
-
-
-/* ZSTDMT_resize() :
- * @return : error code if fails, 0 on success */
-static size_t ZSTDMT_resize(ZSTDMT_CCtx* mtctx, unsigned nbWorkers)
-{
-    if (POOL_resize(mtctx->factory, nbWorkers)) return ERROR(memory_allocation);
-    FORWARD_IF_ERROR( ZSTDMT_expandJobsTable(mtctx, nbWorkers) );
-    mtctx->bufPool = ZSTDMT_expandBufferPool(mtctx->bufPool, nbWorkers);
-    if (mtctx->bufPool == NULL) return ERROR(memory_allocation);
-    mtctx->cctxPool = ZSTDMT_expandCCtxPool(mtctx->cctxPool, nbWorkers);
-    if (mtctx->cctxPool == NULL) return ERROR(memory_allocation);
-    mtctx->seqPool = ZSTDMT_expandSeqPool(mtctx->seqPool, nbWorkers);
-    if (mtctx->seqPool == NULL) return ERROR(memory_allocation);
-    ZSTDMT_CCtxParam_setNbWorkers(&mtctx->params, nbWorkers);
-    return 0;
-}
-
-
-/*! ZSTDMT_updateCParams_whileCompressing() :
- *  Updates a selected set of compression parameters, remaining compatible with currently active frame.
- *  New parameters will be applied to next compression job. */
-void ZSTDMT_updateCParams_whileCompressing(ZSTDMT_CCtx* mtctx, const ZSTD_CCtx_params* cctxParams)
-{
-    U32 const saved_wlog = mtctx->params.cParams.windowLog;   /* Do not modify windowLog while compressing */
-    int const compressionLevel = cctxParams->compressionLevel;
-    DEBUGLOG(5, "ZSTDMT_updateCParams_whileCompressing (level:%i)",
-                compressionLevel);
-    mtctx->params.compressionLevel = compressionLevel;
-    {   ZSTD_compressionParameters cParams = ZSTD_getCParamsFromCCtxParams(cctxParams, 0, 0);
-        cParams.windowLog = saved_wlog;
-        mtctx->params.cParams = cParams;
-    }
-}
-
-/* ZSTDMT_getFrameProgression():
- * tells how much data has been consumed (input) and produced (output) for current frame.
- * able to count progression inside worker threads.
- * Note : mutex will be acquired during statistics collection inside workers. */
-ZSTD_frameProgression ZSTDMT_getFrameProgression(ZSTDMT_CCtx* mtctx)
-{
-    ZSTD_frameProgression fps;
-    DEBUGLOG(5, "ZSTDMT_getFrameProgression");
-    fps.ingested = mtctx->consumed + mtctx->inBuff.filled;
-    fps.consumed = mtctx->consumed;
-    fps.produced = fps.flushed = mtctx->produced;
-    fps.currentJobID = mtctx->nextJobID;
-    fps.nbActiveWorkers = 0;
-    {   unsigned jobNb;
-        unsigned lastJobNb = mtctx->nextJobID + mtctx->jobReady; assert(mtctx->jobReady <= 1);
-        DEBUGLOG(6, "ZSTDMT_getFrameProgression: jobs: from %u to <%u (jobReady:%u)",
-                    mtctx->doneJobID, lastJobNb, mtctx->jobReady)
-        for (jobNb = mtctx->doneJobID ; jobNb < lastJobNb ; jobNb++) {
-            unsigned const wJobID = jobNb & mtctx->jobIDMask;
-            ZSTDMT_jobDescription* jobPtr = &mtctx->jobs[wJobID];
-            ZSTD_pthread_mutex_lock(&jobPtr->job_mutex);
-            {   size_t const cResult = jobPtr->cSize;
-                size_t const produced = ZSTD_isError(cResult) ? 0 : cResult;
-                size_t const flushed = ZSTD_isError(cResult) ? 0 : jobPtr->dstFlushed;
-                assert(flushed <= produced);
-                fps.ingested += jobPtr->src.size;
-                fps.consumed += jobPtr->consumed;
-                fps.produced += produced;
-                fps.flushed  += flushed;
-                fps.nbActiveWorkers += (jobPtr->consumed < jobPtr->src.size);
-            }
-            ZSTD_pthread_mutex_unlock(&mtctx->jobs[wJobID].job_mutex);
-        }
-    }
-    return fps;
-}
-
-
-size_t ZSTDMT_toFlushNow(ZSTDMT_CCtx* mtctx)
-{
-    size_t toFlush;
-    unsigned const jobID = mtctx->doneJobID;
-    assert(jobID <= mtctx->nextJobID);
-    if (jobID == mtctx->nextJobID) return 0;   /* no active job => nothing to flush */
-
-    /* look into oldest non-fully-flushed job */
-    {   unsigned const wJobID = jobID & mtctx->jobIDMask;
-        ZSTDMT_jobDescription* const jobPtr = &mtctx->jobs[wJobID];
-        ZSTD_pthread_mutex_lock(&jobPtr->job_mutex);
-        {   size_t const cResult = jobPtr->cSize;
-            size_t const produced = ZSTD_isError(cResult) ? 0 : cResult;
-            size_t const flushed = ZSTD_isError(cResult) ? 0 : jobPtr->dstFlushed;
-            assert(flushed <= produced);
-            assert(jobPtr->consumed <= jobPtr->src.size);
-            toFlush = produced - flushed;
-            /* if toFlush==0, nothing is available to flush.
-             * However, jobID is expected to still be active:
-             * if jobID was already completed and fully flushed,
-             * ZSTDMT_flushProduced() should have already moved onto next job.
-             * Therefore, some input has not yet been consumed. */
-            if (toFlush==0) {
-                assert(jobPtr->consumed < jobPtr->src.size);
-            }
-        }
-        ZSTD_pthread_mutex_unlock(&mtctx->jobs[wJobID].job_mutex);
-    }
-
-    return toFlush;
-}
-
-
-/* ------------------------------------------ */
-/* =====   Multi-threaded compression   ===== */
-/* ------------------------------------------ */
-
-static unsigned ZSTDMT_computeTargetJobLog(ZSTD_CCtx_params const params)
-{
-    unsigned jobLog;
-    if (params.ldmParams.enableLdm) {
-        /* In Long Range Mode, the windowLog is typically oversized.
-         * In which case, it's preferable to determine the jobSize
-         * based on chainLog instead. */
-        jobLog = MAX(21, params.cParams.chainLog + 4);
-    } else {
-        jobLog = MAX(20, params.cParams.windowLog + 2);
-    }
-    return MIN(jobLog, (unsigned)ZSTDMT_JOBLOG_MAX);
-}
-
-static int ZSTDMT_overlapLog_default(ZSTD_strategy strat)
-{
-    switch(strat)
-    {
-        case ZSTD_btultra2:
-            return 9;
-        case ZSTD_btultra:
-        case ZSTD_btopt:
-            return 8;
-        case ZSTD_btlazy2:
-        case ZSTD_lazy2:
-            return 7;
-        case ZSTD_lazy:
-        case ZSTD_greedy:
-        case ZSTD_dfast:
-        case ZSTD_fast:
-        default:;
-    }
-    return 6;
-}
-
-static int ZSTDMT_overlapLog(int ovlog, ZSTD_strategy strat)
-{
-    assert(0 <= ovlog && ovlog <= 9);
-    if (ovlog == 0) return ZSTDMT_overlapLog_default(strat);
-    return ovlog;
-}
-
-static size_t ZSTDMT_computeOverlapSize(ZSTD_CCtx_params const params)
-{
-    int const overlapRLog = 9 - ZSTDMT_overlapLog(params.overlapLog, params.cParams.strategy);
-    int ovLog = (overlapRLog >= 8) ? 0 : (params.cParams.windowLog - overlapRLog);
-    assert(0 <= overlapRLog && overlapRLog <= 8);
-    if (params.ldmParams.enableLdm) {
-        /* In Long Range Mode, the windowLog is typically oversized.
-         * In which case, it's preferable to determine the jobSize
-         * based on chainLog instead.
-         * Then, ovLog becomes a fraction of the jobSize, rather than windowSize */
-        ovLog = MIN(params.cParams.windowLog, ZSTDMT_computeTargetJobLog(params) - 2)
-                - overlapRLog;
-    }
-    assert(0 <= ovLog && ovLog <= ZSTD_WINDOWLOG_MAX);
-    DEBUGLOG(4, "overlapLog : %i", params.overlapLog);
-    DEBUGLOG(4, "overlap size : %i", 1 << ovLog);
-    return (ovLog==0) ? 0 : (size_t)1 << ovLog;
-}
-
-static unsigned
-ZSTDMT_computeNbJobs(ZSTD_CCtx_params params, size_t srcSize, unsigned nbWorkers)
-{
-    assert(nbWorkers>0);
-    {   size_t const jobSizeTarget = (size_t)1 << ZSTDMT_computeTargetJobLog(params);
-        size_t const jobMaxSize = jobSizeTarget << 2;
-        size_t const passSizeMax = jobMaxSize * nbWorkers;
-        unsigned const multiplier = (unsigned)(srcSize / passSizeMax) + 1;
-        unsigned const nbJobsLarge = multiplier * nbWorkers;
-        unsigned const nbJobsMax = (unsigned)(srcSize / jobSizeTarget) + 1;
-        unsigned const nbJobsSmall = MIN(nbJobsMax, nbWorkers);
-        return (multiplier>1) ? nbJobsLarge : nbJobsSmall;
-}   }
-
-/* ZSTDMT_compress_advanced_internal() :
- * This is a blocking function : it will only give back control to caller after finishing its compression job.
- */
-static size_t ZSTDMT_compress_advanced_internal(
-                ZSTDMT_CCtx* mtctx,
-                void* dst, size_t dstCapacity,
-          const void* src, size_t srcSize,
-          const ZSTD_CDict* cdict,
-                ZSTD_CCtx_params params)
-{
-    ZSTD_CCtx_params const jobParams = ZSTDMT_initJobCCtxParams(params);
-    size_t const overlapSize = ZSTDMT_computeOverlapSize(params);
-    unsigned const nbJobs = ZSTDMT_computeNbJobs(params, srcSize, params.nbWorkers);
-    size_t const proposedJobSize = (srcSize + (nbJobs-1)) / nbJobs;
-    size_t const avgJobSize = (((proposedJobSize-1) & 0x1FFFF) < 0x7FFF) ? proposedJobSize + 0xFFFF : proposedJobSize;   /* avoid too small last block */
-    const char* const srcStart = (const char*)src;
-    size_t remainingSrcSize = srcSize;
-    unsigned const compressWithinDst = (dstCapacity >= ZSTD_compressBound(srcSize)) ? nbJobs : (unsigned)(dstCapacity / ZSTD_compressBound(avgJobSize));  /* presumes avgJobSize >= 256 KB, which should be the case */
-    size_t frameStartPos = 0, dstBufferPos = 0;
-    assert(jobParams.nbWorkers == 0);
-    assert(mtctx->cctxPool->totalCCtx == params.nbWorkers);
-
-    params.jobSize = (U32)avgJobSize;
-    DEBUGLOG(4, "ZSTDMT_compress_advanced_internal: nbJobs=%2u (rawSize=%u bytes; fixedSize=%u) ",
-                nbJobs, (U32)proposedJobSize, (U32)avgJobSize);
-
-    if ((nbJobs==1) | (params.nbWorkers<=1)) {   /* fallback to single-thread mode : this is a blocking invocation anyway */
-        ZSTD_CCtx* const cctx = mtctx->cctxPool->cctx[0];
-        DEBUGLOG(4, "ZSTDMT_compress_advanced_internal: fallback to single-thread mode");
-        if (cdict) return ZSTD_compress_usingCDict_advanced(cctx, dst, dstCapacity, src, srcSize, cdict, jobParams.fParams);
-        return ZSTD_compress_advanced_internal(cctx, dst, dstCapacity, src, srcSize, NULL, 0, jobParams);
-    }
-
-    assert(avgJobSize >= 256 KB);  /* condition for ZSTD_compressBound(A) + ZSTD_compressBound(B) <= ZSTD_compressBound(A+B), required to compress directly into Dst (no additional buffer) */
-    ZSTDMT_setBufferSize(mtctx->bufPool, ZSTD_compressBound(avgJobSize) );
-    if (ZSTDMT_serialState_reset(&mtctx->serial, mtctx->seqPool, params, avgJobSize))
-        return ERROR(memory_allocation);
-
-    FORWARD_IF_ERROR( ZSTDMT_expandJobsTable(mtctx, nbJobs) );  /* only expands if necessary */
-
-    {   unsigned u;
-        for (u=0; u<nbJobs; u++) {
-            size_t const jobSize = MIN(remainingSrcSize, avgJobSize);
-            size_t const dstBufferCapacity = ZSTD_compressBound(jobSize);
-            buffer_t const dstAsBuffer = { (char*)dst + dstBufferPos, dstBufferCapacity };
-            buffer_t const dstBuffer = u < compressWithinDst ? dstAsBuffer : g_nullBuffer;
-            size_t dictSize = u ? overlapSize : 0;
-
-            mtctx->jobs[u].prefix.start = srcStart + frameStartPos - dictSize;
-            mtctx->jobs[u].prefix.size = dictSize;
-            mtctx->jobs[u].src.start = srcStart + frameStartPos;
-            mtctx->jobs[u].src.size = jobSize; assert(jobSize > 0);  /* avoid job.src.size == 0 */
-            mtctx->jobs[u].consumed = 0;
-            mtctx->jobs[u].cSize = 0;
-            mtctx->jobs[u].cdict = (u==0) ? cdict : NULL;
-            mtctx->jobs[u].fullFrameSize = srcSize;
-            mtctx->jobs[u].params = jobParams;
-            /* do not calculate checksum within sections, but write it in header for first section */
-            mtctx->jobs[u].dstBuff = dstBuffer;
-            mtctx->jobs[u].cctxPool = mtctx->cctxPool;
-            mtctx->jobs[u].bufPool = mtctx->bufPool;
-            mtctx->jobs[u].seqPool = mtctx->seqPool;
-            mtctx->jobs[u].serial = &mtctx->serial;
-            mtctx->jobs[u].jobID = u;
-            mtctx->jobs[u].firstJob = (u==0);
-            mtctx->jobs[u].lastJob = (u==nbJobs-1);
-
-            DEBUGLOG(5, "ZSTDMT_compress_advanced_internal: posting job %u  (%u bytes)", u, (U32)jobSize);
-            DEBUG_PRINTHEX(6, mtctx->jobs[u].prefix.start, 12);
-            POOL_add(mtctx->factory, ZSTDMT_compressionJob, &mtctx->jobs[u]);
-
-            frameStartPos += jobSize;
-            dstBufferPos += dstBufferCapacity;
-            remainingSrcSize -= jobSize;
-    }   }
-
-    /* collect result */
-    {   size_t error = 0, dstPos = 0;
-        unsigned jobID;
-        for (jobID=0; jobID<nbJobs; jobID++) {
-            DEBUGLOG(5, "waiting for job %u ", jobID);
-            ZSTD_PTHREAD_MUTEX_LOCK(&mtctx->jobs[jobID].job_mutex);
-            while (mtctx->jobs[jobID].consumed < mtctx->jobs[jobID].src.size) {
-                DEBUGLOG(5, "waiting for jobCompleted signal from job %u", jobID);
-                ZSTD_pthread_cond_wait(&mtctx->jobs[jobID].job_cond, &mtctx->jobs[jobID].job_mutex);
-            }
-            ZSTD_pthread_mutex_unlock(&mtctx->jobs[jobID].job_mutex);
-            DEBUGLOG(5, "ready to write job %u ", jobID);
-
-            {   size_t const cSize = mtctx->jobs[jobID].cSize;
-                if (ZSTD_isError(cSize)) error = cSize;
-                if ((!error) && (dstPos + cSize > dstCapacity)) error = ERROR(dstSize_tooSmall);
-                if (jobID) {   /* note : job 0 is written directly at dst, which is correct position */
-                    if (!error)
-                        memmove((char*)dst + dstPos, mtctx->jobs[jobID].dstBuff.start, cSize);  /* may overlap when job compressed within dst */
-                    if (jobID >= compressWithinDst) {  /* job compressed into its own buffer, which must be released */
-                        DEBUGLOG(5, "releasing buffer %u>=%u", jobID, compressWithinDst);
-                        ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[jobID].dstBuff);
-                }   }
-                mtctx->jobs[jobID].dstBuff = g_nullBuffer;
-                mtctx->jobs[jobID].cSize = 0;
-                dstPos += cSize ;
-            }
-        }  /* for (jobID=0; jobID<nbJobs; jobID++) */
-
-        DEBUGLOG(4, "checksumFlag : %u ", params.fParams.checksumFlag);
-        if (params.fParams.checksumFlag) {
-            U32 const checksum = (U32)XXH64_digest(&mtctx->serial.xxhState);
-            if (dstPos + 4 > dstCapacity) {
-                error = ERROR(dstSize_tooSmall);
-            } else {
-                DEBUGLOG(4, "writing checksum : %08X \n", checksum);
-                MEM_writeLE32((char*)dst + dstPos, checksum);
-                dstPos += 4;
-        }   }
-
-        if (!error) DEBUGLOG(4, "compressed size : %u  ", (U32)dstPos);
-        return error ? error : dstPos;
-    }
-}
-
-size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx,
-                                void* dst, size_t dstCapacity,
-                          const void* src, size_t srcSize,
-                          const ZSTD_CDict* cdict,
-                                ZSTD_parameters params,
-                                int overlapLog)
-{
-    ZSTD_CCtx_params cctxParams = mtctx->params;
-    cctxParams.cParams = params.cParams;
-    cctxParams.fParams = params.fParams;
-    assert(ZSTD_OVERLAPLOG_MIN <= overlapLog && overlapLog <= ZSTD_OVERLAPLOG_MAX);
-    cctxParams.overlapLog = overlapLog;
-    return ZSTDMT_compress_advanced_internal(mtctx,
-                                             dst, dstCapacity,
-                                             src, srcSize,
-                                             cdict, cctxParams);
-}
-
-
-size_t ZSTDMT_compressCCtx(ZSTDMT_CCtx* mtctx,
-                           void* dst, size_t dstCapacity,
-                     const void* src, size_t srcSize,
-                           int compressionLevel)
-{
-    ZSTD_parameters params = ZSTD_getParams(compressionLevel, srcSize, 0);
-    int const overlapLog = ZSTDMT_overlapLog_default(params.cParams.strategy);
-    params.fParams.contentSizeFlag = 1;
-    return ZSTDMT_compress_advanced(mtctx, dst, dstCapacity, src, srcSize, NULL, params, overlapLog);
-}
-
-
-/* ====================================== */
-/* =======      Streaming API     ======= */
-/* ====================================== */
-
-size_t ZSTDMT_initCStream_internal(
-        ZSTDMT_CCtx* mtctx,
-        const void* dict, size_t dictSize, ZSTD_dictContentType_e dictContentType,
-        const ZSTD_CDict* cdict, ZSTD_CCtx_params params,
-        unsigned long long pledgedSrcSize)
-{
-    DEBUGLOG(4, "ZSTDMT_initCStream_internal (pledgedSrcSize=%u, nbWorkers=%u, cctxPool=%u)",
-                (U32)pledgedSrcSize, params.nbWorkers, mtctx->cctxPool->totalCCtx);
-
-    /* params supposed partially fully validated at this point */
-    assert(!ZSTD_isError(ZSTD_checkCParams(params.cParams)));
-    assert(!((dict) && (cdict)));  /* either dict or cdict, not both */
-
-    /* init */
-    if (params.nbWorkers != mtctx->params.nbWorkers)
-        FORWARD_IF_ERROR( ZSTDMT_resize(mtctx, params.nbWorkers) );
-
-    if (params.jobSize != 0 && params.jobSize < ZSTDMT_JOBSIZE_MIN) params.jobSize = ZSTDMT_JOBSIZE_MIN;
-    if (params.jobSize > (size_t)ZSTDMT_JOBSIZE_MAX) params.jobSize = (size_t)ZSTDMT_JOBSIZE_MAX;
-
-    mtctx->singleBlockingThread = (pledgedSrcSize <= ZSTDMT_JOBSIZE_MIN);  /* do not trigger multi-threading when srcSize is too small */
-    if (mtctx->singleBlockingThread) {
-        ZSTD_CCtx_params const singleThreadParams = ZSTDMT_initJobCCtxParams(params);
-        DEBUGLOG(5, "ZSTDMT_initCStream_internal: switch to single blocking thread mode");
-        assert(singleThreadParams.nbWorkers == 0);
-        return ZSTD_initCStream_internal(mtctx->cctxPool->cctx[0],
-                                         dict, dictSize, cdict,
-                                         singleThreadParams, pledgedSrcSize);
-    }
-
-    DEBUGLOG(4, "ZSTDMT_initCStream_internal: %u workers", params.nbWorkers);
-
-    if (mtctx->allJobsCompleted == 0) {   /* previous compression not correctly finished */
-        ZSTDMT_waitForAllJobsCompleted(mtctx);
-        ZSTDMT_releaseAllJobResources(mtctx);
-        mtctx->allJobsCompleted = 1;
-    }
-
-    mtctx->params = params;
-    mtctx->frameContentSize = pledgedSrcSize;
-    if (dict) {
-        ZSTD_freeCDict(mtctx->cdictLocal);
-        mtctx->cdictLocal = ZSTD_createCDict_advanced(dict, dictSize,
-                                                    ZSTD_dlm_byCopy, dictContentType, /* note : a loadPrefix becomes an internal CDict */
-                                                    params.cParams, mtctx->cMem);
-        mtctx->cdict = mtctx->cdictLocal;
-        if (mtctx->cdictLocal == NULL) return ERROR(memory_allocation);
-    } else {
-        ZSTD_freeCDict(mtctx->cdictLocal);
-        mtctx->cdictLocal = NULL;
-        mtctx->cdict = cdict;
-    }
-
-    mtctx->targetPrefixSize = ZSTDMT_computeOverlapSize(params);
-    DEBUGLOG(4, "overlapLog=%i => %u KB", params.overlapLog, (U32)(mtctx->targetPrefixSize>>10));
-    mtctx->targetSectionSize = params.jobSize;
-    if (mtctx->targetSectionSize == 0) {
-        mtctx->targetSectionSize = 1ULL << ZSTDMT_computeTargetJobLog(params);
-    }
-    assert(mtctx->targetSectionSize <= (size_t)ZSTDMT_JOBSIZE_MAX);
-
-    if (params.rsyncable) {
-        /* Aim for the targetsectionSize as the average job size. */
-        U32 const jobSizeMB = (U32)(mtctx->targetSectionSize >> 20);
-        U32 const rsyncBits = ZSTD_highbit32(jobSizeMB) + 20;
-        assert(jobSizeMB >= 1);
-        DEBUGLOG(4, "rsyncLog = %u", rsyncBits);
-        mtctx->rsync.hash = 0;
-        mtctx->rsync.hitMask = (1ULL << rsyncBits) - 1;
-        mtctx->rsync.primePower = ZSTD_rollingHash_primePower(RSYNC_LENGTH);
-    }
-    if (mtctx->targetSectionSize < mtctx->targetPrefixSize) mtctx->targetSectionSize = mtctx->targetPrefixSize;  /* job size must be >= overlap size */
-    DEBUGLOG(4, "Job Size : %u KB (note : set to %u)", (U32)(mtctx->targetSectionSize>>10), (U32)params.jobSize);
-    DEBUGLOG(4, "inBuff Size : %u KB", (U32)(mtctx->targetSectionSize>>10));
-    ZSTDMT_setBufferSize(mtctx->bufPool, ZSTD_compressBound(mtctx->targetSectionSize));
-    {
-        /* If ldm is enabled we need windowSize space. */
-        size_t const windowSize = mtctx->params.ldmParams.enableLdm ? (1U << mtctx->params.cParams.windowLog) : 0;
-        /* Two buffers of slack, plus extra space for the overlap
-         * This is the minimum slack that LDM works with. One extra because
-         * flush might waste up to targetSectionSize-1 bytes. Another extra
-         * for the overlap (if > 0), then one to fill which doesn't overlap
-         * with the LDM window.
-         */
-        size_t const nbSlackBuffers = 2 + (mtctx->targetPrefixSize > 0);
-        size_t const slackSize = mtctx->targetSectionSize * nbSlackBuffers;
-        /* Compute the total size, and always have enough slack */
-        size_t const nbWorkers = MAX(mtctx->params.nbWorkers, 1);
-        size_t const sectionsSize = mtctx->targetSectionSize * nbWorkers;
-        size_t const capacity = MAX(windowSize, sectionsSize) + slackSize;
-        if (mtctx->roundBuff.capacity < capacity) {
-            if (mtctx->roundBuff.buffer)
-                ZSTD_free(mtctx->roundBuff.buffer, mtctx->cMem);
-            mtctx->roundBuff.buffer = (BYTE*)ZSTD_malloc(capacity, mtctx->cMem);
-            if (mtctx->roundBuff.buffer == NULL) {
-                mtctx->roundBuff.capacity = 0;
-                return ERROR(memory_allocation);
-            }
-            mtctx->roundBuff.capacity = capacity;
-        }
-    }
-    DEBUGLOG(4, "roundBuff capacity : %u KB", (U32)(mtctx->roundBuff.capacity>>10));
-    mtctx->roundBuff.pos = 0;
-    mtctx->inBuff.buffer = g_nullBuffer;
-    mtctx->inBuff.filled = 0;
-    mtctx->inBuff.prefix = kNullRange;
-    mtctx->doneJobID = 0;
-    mtctx->nextJobID = 0;
-    mtctx->frameEnded = 0;
-    mtctx->allJobsCompleted = 0;
-    mtctx->consumed = 0;
-    mtctx->produced = 0;
-    if (ZSTDMT_serialState_reset(&mtctx->serial, mtctx->seqPool, params, mtctx->targetSectionSize))
-        return ERROR(memory_allocation);
-    return 0;
-}
-
-size_t ZSTDMT_initCStream_advanced(ZSTDMT_CCtx* mtctx,
-                             const void* dict, size_t dictSize,
-                                   ZSTD_parameters params,
-                                   unsigned long long pledgedSrcSize)
-{
-    ZSTD_CCtx_params cctxParams = mtctx->params;  /* retrieve sticky params */
-    DEBUGLOG(4, "ZSTDMT_initCStream_advanced (pledgedSrcSize=%u)", (U32)pledgedSrcSize);
-    cctxParams.cParams = params.cParams;
-    cctxParams.fParams = params.fParams;
-    return ZSTDMT_initCStream_internal(mtctx, dict, dictSize, ZSTD_dct_auto, NULL,
-                                       cctxParams, pledgedSrcSize);
-}
-
-size_t ZSTDMT_initCStream_usingCDict(ZSTDMT_CCtx* mtctx,
-                               const ZSTD_CDict* cdict,
-                                     ZSTD_frameParameters fParams,
-                                     unsigned long long pledgedSrcSize)
-{
-    ZSTD_CCtx_params cctxParams = mtctx->params;
-    if (cdict==NULL) return ERROR(dictionary_wrong);   /* method incompatible with NULL cdict */
-    cctxParams.cParams = ZSTD_getCParamsFromCDict(cdict);
-    cctxParams.fParams = fParams;
-    return ZSTDMT_initCStream_internal(mtctx, NULL, 0 /*dictSize*/, ZSTD_dct_auto, cdict,
-                                       cctxParams, pledgedSrcSize);
-}
-
-
-/* ZSTDMT_resetCStream() :
- * pledgedSrcSize can be zero == unknown (for the time being)
- * prefer using ZSTD_CONTENTSIZE_UNKNOWN,
- * as `0` might mean "empty" in the future */
-size_t ZSTDMT_resetCStream(ZSTDMT_CCtx* mtctx, unsigned long long pledgedSrcSize)
-{
-    if (!pledgedSrcSize) pledgedSrcSize = ZSTD_CONTENTSIZE_UNKNOWN;
-    return ZSTDMT_initCStream_internal(mtctx, NULL, 0, ZSTD_dct_auto, 0, mtctx->params,
-                                       pledgedSrcSize);
-}
-
-size_t ZSTDMT_initCStream(ZSTDMT_CCtx* mtctx, int compressionLevel) {
-    ZSTD_parameters const params = ZSTD_getParams(compressionLevel, ZSTD_CONTENTSIZE_UNKNOWN, 0);
-    ZSTD_CCtx_params cctxParams = mtctx->params;   /* retrieve sticky params */
-    DEBUGLOG(4, "ZSTDMT_initCStream (cLevel=%i)", compressionLevel);
-    cctxParams.cParams = params.cParams;
-    cctxParams.fParams = params.fParams;
-    return ZSTDMT_initCStream_internal(mtctx, NULL, 0, ZSTD_dct_auto, NULL, cctxParams, ZSTD_CONTENTSIZE_UNKNOWN);
-}
-
-
-/* ZSTDMT_writeLastEmptyBlock()
- * Write a single empty block with an end-of-frame to finish a frame.
- * Job must be created from streaming variant.
- * This function is always successful if expected conditions are fulfilled.
- */
-static void ZSTDMT_writeLastEmptyBlock(ZSTDMT_jobDescription* job)
-{
-    assert(job->lastJob == 1);
-    assert(job->src.size == 0);   /* last job is empty -> will be simplified into a last empty block */
-    assert(job->firstJob == 0);   /* cannot be first job, as it also needs to create frame header */
-    assert(job->dstBuff.start == NULL);   /* invoked from streaming variant only (otherwise, dstBuff might be user's output) */
-    job->dstBuff = ZSTDMT_getBuffer(job->bufPool);
-    if (job->dstBuff.start == NULL) {
-      job->cSize = ERROR(memory_allocation);
-      return;
-    }
-    assert(job->dstBuff.capacity >= ZSTD_blockHeaderSize);   /* no buffer should ever be that small */
-    job->src = kNullRange;
-    job->cSize = ZSTD_writeLastEmptyBlock(job->dstBuff.start, job->dstBuff.capacity);
-    assert(!ZSTD_isError(job->cSize));
-    assert(job->consumed == 0);
-}
-
-static size_t ZSTDMT_createCompressionJob(ZSTDMT_CCtx* mtctx, size_t srcSize, ZSTD_EndDirective endOp)
-{
-    unsigned const jobID = mtctx->nextJobID & mtctx->jobIDMask;
-    int const endFrame = (endOp == ZSTD_e_end);
-
-    if (mtctx->nextJobID > mtctx->doneJobID + mtctx->jobIDMask) {
-        DEBUGLOG(5, "ZSTDMT_createCompressionJob: will not create new job : table is full");
-        assert((mtctx->nextJobID & mtctx->jobIDMask) == (mtctx->doneJobID & mtctx->jobIDMask));
-        return 0;
-    }
-
-    if (!mtctx->jobReady) {
-        BYTE const* src = (BYTE const*)mtctx->inBuff.buffer.start;
-        DEBUGLOG(5, "ZSTDMT_createCompressionJob: preparing job %u to compress %u bytes with %u preload ",
-                    mtctx->nextJobID, (U32)srcSize, (U32)mtctx->inBuff.prefix.size);
-        mtctx->jobs[jobID].src.start = src;
-        mtctx->jobs[jobID].src.size = srcSize;
-        assert(mtctx->inBuff.filled >= srcSize);
-        mtctx->jobs[jobID].prefix = mtctx->inBuff.prefix;
-        mtctx->jobs[jobID].consumed = 0;
-        mtctx->jobs[jobID].cSize = 0;
-        mtctx->jobs[jobID].params = mtctx->params;
-        mtctx->jobs[jobID].cdict = mtctx->nextJobID==0 ? mtctx->cdict : NULL;
-        mtctx->jobs[jobID].fullFrameSize = mtctx->frameContentSize;
-        mtctx->jobs[jobID].dstBuff = g_nullBuffer;
-        mtctx->jobs[jobID].cctxPool = mtctx->cctxPool;
-        mtctx->jobs[jobID].bufPool = mtctx->bufPool;
-        mtctx->jobs[jobID].seqPool = mtctx->seqPool;
-        mtctx->jobs[jobID].serial = &mtctx->serial;
-        mtctx->jobs[jobID].jobID = mtctx->nextJobID;
-        mtctx->jobs[jobID].firstJob = (mtctx->nextJobID==0);
-        mtctx->jobs[jobID].lastJob = endFrame;
-        mtctx->jobs[jobID].frameChecksumNeeded = mtctx->params.fParams.checksumFlag && endFrame && (mtctx->nextJobID>0);
-        mtctx->jobs[jobID].dstFlushed = 0;
-
-        /* Update the round buffer pos and clear the input buffer to be reset */
-        mtctx->roundBuff.pos += srcSize;
-        mtctx->inBuff.buffer = g_nullBuffer;
-        mtctx->inBuff.filled = 0;
-        /* Set the prefix */
-        if (!endFrame) {
-            size_t const newPrefixSize = MIN(srcSize, mtctx->targetPrefixSize);
-            mtctx->inBuff.prefix.start = src + srcSize - newPrefixSize;
-            mtctx->inBuff.prefix.size = newPrefixSize;
-        } else {   /* endFrame==1 => no need for another input buffer */
-            mtctx->inBuff.prefix = kNullRange;
-            mtctx->frameEnded = endFrame;
-            if (mtctx->nextJobID == 0) {
-                /* single job exception : checksum is already calculated directly within worker thread */
-                mtctx->params.fParams.checksumFlag = 0;
-        }   }
-
-        if ( (srcSize == 0)
-          && (mtctx->nextJobID>0)/*single job must also write frame header*/ ) {
-            DEBUGLOG(5, "ZSTDMT_createCompressionJob: creating a last empty block to end frame");
-            assert(endOp == ZSTD_e_end);  /* only possible case : need to end the frame with an empty last block */
-            ZSTDMT_writeLastEmptyBlock(mtctx->jobs + jobID);
-            mtctx->nextJobID++;
-            return 0;
-        }
-    }
-
-    DEBUGLOG(5, "ZSTDMT_createCompressionJob: posting job %u : %u bytes  (end:%u, jobNb == %u (mod:%u))",
-                mtctx->nextJobID,
-                (U32)mtctx->jobs[jobID].src.size,
-                mtctx->jobs[jobID].lastJob,
-                mtctx->nextJobID,
-                jobID);
-    if (POOL_tryAdd(mtctx->factory, ZSTDMT_compressionJob, &mtctx->jobs[jobID])) {
-        mtctx->nextJobID++;
-        mtctx->jobReady = 0;
-    } else {
-        DEBUGLOG(5, "ZSTDMT_createCompressionJob: no worker available for job %u", mtctx->nextJobID);
-        mtctx->jobReady = 1;
-    }
-    return 0;
-}
-
-
-/*! ZSTDMT_flushProduced() :
- *  flush whatever data has been produced but not yet flushed in current job.
- *  move to next job if current one is fully flushed.
- * `output` : `pos` will be updated with amount of data flushed .
- * `blockToFlush` : if >0, the function will block and wait if there is no data available to flush .
- * @return : amount of data remaining within internal buffer, 0 if no more, 1 if unknown but > 0, or an error code */
-static size_t ZSTDMT_flushProduced(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output, unsigned blockToFlush, ZSTD_EndDirective end)
-{
-    unsigned const wJobID = mtctx->doneJobID & mtctx->jobIDMask;
-    DEBUGLOG(5, "ZSTDMT_flushProduced (blocking:%u , job %u <= %u)",
-                blockToFlush, mtctx->doneJobID, mtctx->nextJobID);
-    assert(output->size >= output->pos);
-
-    ZSTD_PTHREAD_MUTEX_LOCK(&mtctx->jobs[wJobID].job_mutex);
-    if (  blockToFlush
-      && (mtctx->doneJobID < mtctx->nextJobID) ) {
-        assert(mtctx->jobs[wJobID].dstFlushed <= mtctx->jobs[wJobID].cSize);
-        while (mtctx->jobs[wJobID].dstFlushed == mtctx->jobs[wJobID].cSize) {  /* nothing to flush */
-            if (mtctx->jobs[wJobID].consumed == mtctx->jobs[wJobID].src.size) {
-                DEBUGLOG(5, "job %u is completely consumed (%u == %u) => don't wait for cond, there will be none",
-                            mtctx->doneJobID, (U32)mtctx->jobs[wJobID].consumed, (U32)mtctx->jobs[wJobID].src.size);
-                break;
-            }
-            DEBUGLOG(5, "waiting for something to flush from job %u (currently flushed: %u bytes)",
-                        mtctx->doneJobID, (U32)mtctx->jobs[wJobID].dstFlushed);
-            ZSTD_pthread_cond_wait(&mtctx->jobs[wJobID].job_cond, &mtctx->jobs[wJobID].job_mutex);  /* block when nothing to flush but some to come */
-    }   }
-
-    /* try to flush something */
-    {   size_t cSize = mtctx->jobs[wJobID].cSize;                  /* shared */
-        size_t const srcConsumed = mtctx->jobs[wJobID].consumed;   /* shared */
-        size_t const srcSize = mtctx->jobs[wJobID].src.size;       /* read-only, could be done after mutex lock, but no-declaration-after-statement */
-        ZSTD_pthread_mutex_unlock(&mtctx->jobs[wJobID].job_mutex);
-        if (ZSTD_isError(cSize)) {
-            DEBUGLOG(5, "ZSTDMT_flushProduced: job %u : compression error detected : %s",
-                        mtctx->doneJobID, ZSTD_getErrorName(cSize));
-            ZSTDMT_waitForAllJobsCompleted(mtctx);
-            ZSTDMT_releaseAllJobResources(mtctx);
-            return cSize;
-        }
-        /* add frame checksum if necessary (can only happen once) */
-        assert(srcConsumed <= srcSize);
-        if ( (srcConsumed == srcSize)   /* job completed -> worker no longer active */
-          && mtctx->jobs[wJobID].frameChecksumNeeded ) {
-            U32 const checksum = (U32)XXH64_digest(&mtctx->serial.xxhState);
-            DEBUGLOG(4, "ZSTDMT_flushProduced: writing checksum : %08X \n", checksum);
-            MEM_writeLE32((char*)mtctx->jobs[wJobID].dstBuff.start + mtctx->jobs[wJobID].cSize, checksum);
-            cSize += 4;
-            mtctx->jobs[wJobID].cSize += 4;  /* can write this shared value, as worker is no longer active */
-            mtctx->jobs[wJobID].frameChecksumNeeded = 0;
-        }
-
-        if (cSize > 0) {   /* compression is ongoing or completed */
-            size_t const toFlush = MIN(cSize - mtctx->jobs[wJobID].dstFlushed, output->size - output->pos);
-            DEBUGLOG(5, "ZSTDMT_flushProduced: Flushing %u bytes from job %u (completion:%u/%u, generated:%u)",
-                        (U32)toFlush, mtctx->doneJobID, (U32)srcConsumed, (U32)srcSize, (U32)cSize);
-            assert(mtctx->doneJobID < mtctx->nextJobID);
-            assert(cSize >= mtctx->jobs[wJobID].dstFlushed);
-            assert(mtctx->jobs[wJobID].dstBuff.start != NULL);
-            memcpy((char*)output->dst + output->pos,
-                   (const char*)mtctx->jobs[wJobID].dstBuff.start + mtctx->jobs[wJobID].dstFlushed,
-                   toFlush);
-            output->pos += toFlush;
-            mtctx->jobs[wJobID].dstFlushed += toFlush;  /* can write : this value is only used by mtctx */
-
-            if ( (srcConsumed == srcSize)    /* job is completed */
-              && (mtctx->jobs[wJobID].dstFlushed == cSize) ) {   /* output buffer fully flushed => free this job position */
-                DEBUGLOG(5, "Job %u completed (%u bytes), moving to next one",
-                        mtctx->doneJobID, (U32)mtctx->jobs[wJobID].dstFlushed);
-                ZSTDMT_releaseBuffer(mtctx->bufPool, mtctx->jobs[wJobID].dstBuff);
-                DEBUGLOG(5, "dstBuffer released");
-                mtctx->jobs[wJobID].dstBuff = g_nullBuffer;
-                mtctx->jobs[wJobID].cSize = 0;   /* ensure this job slot is considered "not started" in future check */
-                mtctx->consumed += srcSize;
-                mtctx->produced += cSize;
-                mtctx->doneJobID++;
-        }   }
-
-        /* return value : how many bytes left in buffer ; fake it to 1 when unknown but >0 */
-        if (cSize > mtctx->jobs[wJobID].dstFlushed) return (cSize - mtctx->jobs[wJobID].dstFlushed);
-        if (srcSize > srcConsumed) return 1;   /* current job not completely compressed */
-    }
-    if (mtctx->doneJobID < mtctx->nextJobID) return 1;   /* some more jobs ongoing */
-    if (mtctx->jobReady) return 1;      /* one job is ready to push, just not yet in the list */
-    if (mtctx->inBuff.filled > 0) return 1;   /* input is not empty, and still needs to be converted into a job */
-    mtctx->allJobsCompleted = mtctx->frameEnded;   /* all jobs are entirely flushed => if this one is last one, frame is completed */
-    if (end == ZSTD_e_end) return !mtctx->frameEnded;  /* for ZSTD_e_end, question becomes : is frame completed ? instead of : are internal buffers fully flushed ? */
-    return 0;   /* internal buffers fully flushed */
-}
-
-/**
- * Returns the range of data used by the earliest job that is not yet complete.
- * If the data of the first job is broken up into two segments, we cover both
- * sections.
- */
-static range_t ZSTDMT_getInputDataInUse(ZSTDMT_CCtx* mtctx)
-{
-    unsigned const firstJobID = mtctx->doneJobID;
-    unsigned const lastJobID = mtctx->nextJobID;
-    unsigned jobID;
-
-    for (jobID = firstJobID; jobID < lastJobID; ++jobID) {
-        unsigned const wJobID = jobID & mtctx->jobIDMask;
-        size_t consumed;
-
-        ZSTD_PTHREAD_MUTEX_LOCK(&mtctx->jobs[wJobID].job_mutex);
-        consumed = mtctx->jobs[wJobID].consumed;
-        ZSTD_pthread_mutex_unlock(&mtctx->jobs[wJobID].job_mutex);
-
-        if (consumed < mtctx->jobs[wJobID].src.size) {
-            range_t range = mtctx->jobs[wJobID].prefix;
-            if (range.size == 0) {
-                /* Empty prefix */
-                range = mtctx->jobs[wJobID].src;
-            }
-            /* Job source in multiple segments not supported yet */
-            assert(range.start <= mtctx->jobs[wJobID].src.start);
-            return range;
-        }
-    }
-    return kNullRange;
-}
-
-/**
- * Returns non-zero iff buffer and range overlap.
- */
-static int ZSTDMT_isOverlapped(buffer_t buffer, range_t range)
-{
-    BYTE const* const bufferStart = (BYTE const*)buffer.start;
-    BYTE const* const bufferEnd = bufferStart + buffer.capacity;
-    BYTE const* const rangeStart = (BYTE const*)range.start;
-    BYTE const* const rangeEnd = rangeStart + range.size;
-
-    if (rangeStart == NULL || bufferStart == NULL)
-        return 0;
-    /* Empty ranges cannot overlap */
-    if (bufferStart == bufferEnd || rangeStart == rangeEnd)
-        return 0;
-
-    return bufferStart < rangeEnd && rangeStart < bufferEnd;
-}
-
-static int ZSTDMT_doesOverlapWindow(buffer_t buffer, ZSTD_window_t window)
-{
-    range_t extDict;
-    range_t prefix;
-
-    DEBUGLOG(5, "ZSTDMT_doesOverlapWindow");
-    extDict.start = window.dictBase + window.lowLimit;
-    extDict.size = window.dictLimit - window.lowLimit;
-
-    prefix.start = window.base + window.dictLimit;
-    prefix.size = window.nextSrc - (window.base + window.dictLimit);
-    DEBUGLOG(5, "extDict [0x%zx, 0x%zx)",
-                (size_t)extDict.start,
-                (size_t)extDict.start + extDict.size);
-    DEBUGLOG(5, "prefix  [0x%zx, 0x%zx)",
-                (size_t)prefix.start,
-                (size_t)prefix.start + prefix.size);
-
-    return ZSTDMT_isOverlapped(buffer, extDict)
-        || ZSTDMT_isOverlapped(buffer, prefix);
-}
-
-static void ZSTDMT_waitForLdmComplete(ZSTDMT_CCtx* mtctx, buffer_t buffer)
-{
-    if (mtctx->params.ldmParams.enableLdm) {
-        ZSTD_pthread_mutex_t* mutex = &mtctx->serial.ldmWindowMutex;
-        DEBUGLOG(5, "ZSTDMT_waitForLdmComplete");
-        DEBUGLOG(5, "source  [0x%zx, 0x%zx)",
-                    (size_t)buffer.start,
-                    (size_t)buffer.start + buffer.capacity);
-        ZSTD_PTHREAD_MUTEX_LOCK(mutex);
-        while (ZSTDMT_doesOverlapWindow(buffer, mtctx->serial.ldmWindow)) {
-            DEBUGLOG(5, "Waiting for LDM to finish...");
-            ZSTD_pthread_cond_wait(&mtctx->serial.ldmWindowCond, mutex);
-        }
-        DEBUGLOG(6, "Done waiting for LDM to finish");
-        ZSTD_pthread_mutex_unlock(mutex);
-    }
-}
-
-/**
- * Attempts to set the inBuff to the next section to fill.
- * If any part of the new section is still in use we give up.
- * Returns non-zero if the buffer is filled.
- */
-static int ZSTDMT_tryGetInputRange(ZSTDMT_CCtx* mtctx)
-{
-    range_t const inUse = ZSTDMT_getInputDataInUse(mtctx);
-    size_t const spaceLeft = mtctx->roundBuff.capacity - mtctx->roundBuff.pos;
-    size_t const target = mtctx->targetSectionSize;
-    buffer_t buffer;
-
-    DEBUGLOG(5, "ZSTDMT_tryGetInputRange");
-    assert(mtctx->inBuff.buffer.start == NULL);
-    assert(mtctx->roundBuff.capacity >= target);
-
-    if (spaceLeft < target) {
-        /* ZSTD_invalidateRepCodes() doesn't work for extDict variants.
-         * Simply copy the prefix to the beginning in that case.
-         */
-        BYTE* const start = (BYTE*)mtctx->roundBuff.buffer;
-        size_t const prefixSize = mtctx->inBuff.prefix.size;
-
-        buffer.start = start;
-        buffer.capacity = prefixSize;
-        if (ZSTDMT_isOverlapped(buffer, inUse)) {
-            DEBUGLOG(5, "Waiting for buffer...");
-            return 0;
-        }
-        ZSTDMT_waitForLdmComplete(mtctx, buffer);
-        memmove(start, mtctx->inBuff.prefix.start, prefixSize);
-        mtctx->inBuff.prefix.start = start;
-        mtctx->roundBuff.pos = prefixSize;
-    }
-    buffer.start = mtctx->roundBuff.buffer + mtctx->roundBuff.pos;
-    buffer.capacity = target;
-
-    if (ZSTDMT_isOverlapped(buffer, inUse)) {
-        DEBUGLOG(5, "Waiting for buffer...");
-        return 0;
-    }
-    assert(!ZSTDMT_isOverlapped(buffer, mtctx->inBuff.prefix));
-
-    ZSTDMT_waitForLdmComplete(mtctx, buffer);
-
-    DEBUGLOG(5, "Using prefix range [%zx, %zx)",
-                (size_t)mtctx->inBuff.prefix.start,
-                (size_t)mtctx->inBuff.prefix.start + mtctx->inBuff.prefix.size);
-    DEBUGLOG(5, "Using source range [%zx, %zx)",
-                (size_t)buffer.start,
-                (size_t)buffer.start + buffer.capacity);
-
-
-    mtctx->inBuff.buffer = buffer;
-    mtctx->inBuff.filled = 0;
-    assert(mtctx->roundBuff.pos + buffer.capacity <= mtctx->roundBuff.capacity);
-    return 1;
-}
-
-typedef struct {
-  size_t toLoad;  /* The number of bytes to load from the input. */
-  int flush;      /* Boolean declaring if we must flush because we found a synchronization point. */
-} syncPoint_t;
-
-/**
- * Searches through the input for a synchronization point. If one is found, we
- * will instruct the caller to flush, and return the number of bytes to load.
- * Otherwise, we will load as many bytes as possible and instruct the caller
- * to continue as normal.
- */
-static syncPoint_t
-findSynchronizationPoint(ZSTDMT_CCtx const* mtctx, ZSTD_inBuffer const input)
-{
-    BYTE const* const istart = (BYTE const*)input.src + input.pos;
-    U64 const primePower = mtctx->rsync.primePower;
-    U64 const hitMask = mtctx->rsync.hitMask;
-
-    syncPoint_t syncPoint;
-    U64 hash;
-    BYTE const* prev;
-    size_t pos;
-
-    syncPoint.toLoad = MIN(input.size - input.pos, mtctx->targetSectionSize - mtctx->inBuff.filled);
-    syncPoint.flush = 0;
-    if (!mtctx->params.rsyncable)
-        /* Rsync is disabled. */
-        return syncPoint;
-    if (mtctx->inBuff.filled + syncPoint.toLoad < RSYNC_LENGTH)
-        /* Not enough to compute the hash.
-         * We will miss any synchronization points in this RSYNC_LENGTH byte
-         * window. However, since it depends only in the internal buffers, if the
-         * state is already synchronized, we will remain synchronized.
-         * Additionally, the probability that we miss a synchronization point is
-         * low: RSYNC_LENGTH / targetSectionSize.
-         */
-        return syncPoint;
-    /* Initialize the loop variables. */
-    if (mtctx->inBuff.filled >= RSYNC_LENGTH) {
-        /* We have enough bytes buffered to initialize the hash.
-         * Start scanning at the beginning of the input.
-         */
-        pos = 0;
-        prev = (BYTE const*)mtctx->inBuff.buffer.start + mtctx->inBuff.filled - RSYNC_LENGTH;
-        hash = ZSTD_rollingHash_compute(prev, RSYNC_LENGTH);
-    } else {
-        /* We don't have enough bytes buffered to initialize the hash, but
-         * we know we have at least RSYNC_LENGTH bytes total.
-         * Start scanning after the first RSYNC_LENGTH bytes less the bytes
-         * already buffered.
-         */
-        pos = RSYNC_LENGTH - mtctx->inBuff.filled;
-        prev = (BYTE const*)mtctx->inBuff.buffer.start - pos;
-        hash = ZSTD_rollingHash_compute(mtctx->inBuff.buffer.start, mtctx->inBuff.filled);
-        hash = ZSTD_rollingHash_append(hash, istart, pos);
-    }
-    /* Starting with the hash of the previous RSYNC_LENGTH bytes, roll
-     * through the input. If we hit a synchronization point, then cut the
-     * job off, and tell the compressor to flush the job. Otherwise, load
-     * all the bytes and continue as normal.
-     * If we go too long without a synchronization point (targetSectionSize)
-     * then a block will be emitted anyways, but this is okay, since if we
-     * are already synchronized we will remain synchronized.
-     */
-    for (; pos < syncPoint.toLoad; ++pos) {
-        BYTE const toRemove = pos < RSYNC_LENGTH ? prev[pos] : istart[pos - RSYNC_LENGTH];
-        /* if (pos >= RSYNC_LENGTH) assert(ZSTD_rollingHash_compute(istart + pos - RSYNC_LENGTH, RSYNC_LENGTH) == hash); */
-        hash = ZSTD_rollingHash_rotate(hash, toRemove, istart[pos], primePower);
-        if ((hash & hitMask) == hitMask) {
-            syncPoint.toLoad = pos + 1;
-            syncPoint.flush = 1;
-            break;
-        }
-    }
-    return syncPoint;
-}
-
-size_t ZSTDMT_nextInputSizeHint(const ZSTDMT_CCtx* mtctx)
-{
-    size_t hintInSize = mtctx->targetSectionSize - mtctx->inBuff.filled;
-    if (hintInSize==0) hintInSize = mtctx->targetSectionSize;
-    return hintInSize;
-}
-
-/** ZSTDMT_compressStream_generic() :
- *  internal use only - exposed to be invoked from zstd_compress.c
- *  assumption : output and input are valid (pos <= size)
- * @return : minimum amount of data remaining to flush, 0 if none */
-size_t ZSTDMT_compressStream_generic(ZSTDMT_CCtx* mtctx,
-                                     ZSTD_outBuffer* output,
-                                     ZSTD_inBuffer* input,
-                                     ZSTD_EndDirective endOp)
-{
-    unsigned forwardInputProgress = 0;
-    DEBUGLOG(5, "ZSTDMT_compressStream_generic (endOp=%u, srcSize=%u)",
-                (U32)endOp, (U32)(input->size - input->pos));
-    assert(output->pos <= output->size);
-    assert(input->pos  <= input->size);
-
-    if (mtctx->singleBlockingThread) {  /* delegate to single-thread (synchronous) */
-        return ZSTD_compressStream2(mtctx->cctxPool->cctx[0], output, input, endOp);
-    }
-
-    if ((mtctx->frameEnded) && (endOp==ZSTD_e_continue)) {
-        /* current frame being ended. Only flush/end are allowed */
-        return ERROR(stage_wrong);
-    }
-
-    /* single-pass shortcut (note : synchronous-mode) */
-    if ( (!mtctx->params.rsyncable)   /* rsyncable mode is disabled */
-      && (mtctx->nextJobID == 0)      /* just started */
-      && (mtctx->inBuff.filled == 0)  /* nothing buffered */
-      && (!mtctx->jobReady)           /* no job already created */
-      && (endOp == ZSTD_e_end)        /* end order */
-      && (output->size - output->pos >= ZSTD_compressBound(input->size - input->pos)) ) { /* enough space in dst */
-        size_t const cSize = ZSTDMT_compress_advanced_internal(mtctx,
-                (char*)output->dst + output->pos, output->size - output->pos,
-                (const char*)input->src + input->pos, input->size - input->pos,
-                mtctx->cdict, mtctx->params);
-        if (ZSTD_isError(cSize)) return cSize;
-        input->pos = input->size;
-        output->pos += cSize;
-        mtctx->allJobsCompleted = 1;
-        mtctx->frameEnded = 1;
-        return 0;
-    }
-
-    /* fill input buffer */
-    if ( (!mtctx->jobReady)
-      && (input->size > input->pos) ) {   /* support NULL input */
-        if (mtctx->inBuff.buffer.start == NULL) {
-            assert(mtctx->inBuff.filled == 0); /* Can't fill an empty buffer */
-            if (!ZSTDMT_tryGetInputRange(mtctx)) {
-                /* It is only possible for this operation to fail if there are
-                 * still compression jobs ongoing.
-                 */
-                DEBUGLOG(5, "ZSTDMT_tryGetInputRange failed");
-                assert(mtctx->doneJobID != mtctx->nextJobID);
-            } else
-                DEBUGLOG(5, "ZSTDMT_tryGetInputRange completed successfully : mtctx->inBuff.buffer.start = %p", mtctx->inBuff.buffer.start);
-        }
-        if (mtctx->inBuff.buffer.start != NULL) {
-            syncPoint_t const syncPoint = findSynchronizationPoint(mtctx, *input);
-            if (syncPoint.flush && endOp == ZSTD_e_continue) {
-                endOp = ZSTD_e_flush;
-            }
-            assert(mtctx->inBuff.buffer.capacity >= mtctx->targetSectionSize);
-            DEBUGLOG(5, "ZSTDMT_compressStream_generic: adding %u bytes on top of %u to buffer of size %u",
-                        (U32)syncPoint.toLoad, (U32)mtctx->inBuff.filled, (U32)mtctx->targetSectionSize);
-            memcpy((char*)mtctx->inBuff.buffer.start + mtctx->inBuff.filled, (const char*)input->src + input->pos, syncPoint.toLoad);
-            input->pos += syncPoint.toLoad;
-            mtctx->inBuff.filled += syncPoint.toLoad;
-            forwardInputProgress = syncPoint.toLoad>0;
-        }
-        if ((input->pos < input->size) && (endOp == ZSTD_e_end))
-            endOp = ZSTD_e_flush;   /* can't end now : not all input consumed */
-    }
-
-    if ( (mtctx->jobReady)
-      || (mtctx->inBuff.filled >= mtctx->targetSectionSize)  /* filled enough : let's compress */
-      || ((endOp != ZSTD_e_continue) && (mtctx->inBuff.filled > 0))  /* something to flush : let's go */
-      || ((endOp == ZSTD_e_end) && (!mtctx->frameEnded)) ) {   /* must finish the frame with a zero-size block */
-        size_t const jobSize = mtctx->inBuff.filled;
-        assert(mtctx->inBuff.filled <= mtctx->targetSectionSize);
-        FORWARD_IF_ERROR( ZSTDMT_createCompressionJob(mtctx, jobSize, endOp) );
-    }
-
-    /* check for potential compressed data ready to be flushed */
-    {   size_t const remainingToFlush = ZSTDMT_flushProduced(mtctx, output, !forwardInputProgress, endOp); /* block if there was no forward input progress */
-        if (input->pos < input->size) return MAX(remainingToFlush, 1);  /* input not consumed : do not end flush yet */
-        DEBUGLOG(5, "end of ZSTDMT_compressStream_generic: remainingToFlush = %u", (U32)remainingToFlush);
-        return remainingToFlush;
-    }
-}
-
-
-size_t ZSTDMT_compressStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output, ZSTD_inBuffer* input)
-{
-    FORWARD_IF_ERROR( ZSTDMT_compressStream_generic(mtctx, output, input, ZSTD_e_continue) );
-
-    /* recommended next input size : fill current input buffer */
-    return mtctx->targetSectionSize - mtctx->inBuff.filled;   /* note : could be zero when input buffer is fully filled and no more availability to create new job */
-}
-
-
-static size_t ZSTDMT_flushStream_internal(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output, ZSTD_EndDirective endFrame)
-{
-    size_t const srcSize = mtctx->inBuff.filled;
-    DEBUGLOG(5, "ZSTDMT_flushStream_internal");
-
-    if ( mtctx->jobReady     /* one job ready for a worker to pick up */
-      || (srcSize > 0)       /* still some data within input buffer */
-      || ((endFrame==ZSTD_e_end) && !mtctx->frameEnded)) {  /* need a last 0-size block to end frame */
-           DEBUGLOG(5, "ZSTDMT_flushStream_internal : create a new job (%u bytes, end:%u)",
-                        (U32)srcSize, (U32)endFrame);
-        FORWARD_IF_ERROR( ZSTDMT_createCompressionJob(mtctx, srcSize, endFrame) );
-    }
-
-    /* check if there is any data available to flush */
-    return ZSTDMT_flushProduced(mtctx, output, 1 /* blockToFlush */, endFrame);
-}
-
-
-size_t ZSTDMT_flushStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output)
-{
-    DEBUGLOG(5, "ZSTDMT_flushStream");
-    if (mtctx->singleBlockingThread)
-        return ZSTD_flushStream(mtctx->cctxPool->cctx[0], output);
-    return ZSTDMT_flushStream_internal(mtctx, output, ZSTD_e_flush);
-}
-
-size_t ZSTDMT_endStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output)
-{
-    DEBUGLOG(4, "ZSTDMT_endStream");
-    if (mtctx->singleBlockingThread)
-        return ZSTD_endStream(mtctx->cctxPool->cctx[0], output);
-    return ZSTDMT_flushStream_internal(mtctx, output, ZSTD_e_end);
-}
diff --git a/vendor/github.com/DataDog/zstd/zstdmt_compress.h b/vendor/github.com/DataDog/zstd/zstdmt_compress.h
deleted file mode 100644
index 12a5260..0000000
--- a/vendor/github.com/DataDog/zstd/zstdmt_compress.h
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Copyright (c) 2016-present, Yann Collet, Facebook, Inc.
- * All rights reserved.
- *
- * This source code is licensed under both the BSD-style license (found in the
- * LICENSE file in the root directory of this source tree) and the GPLv2 (found
- * in the COPYING file in the root directory of this source tree).
- * You may select, at your option, one of the above-listed licenses.
- */
-
- #ifndef ZSTDMT_COMPRESS_H
- #define ZSTDMT_COMPRESS_H
-
- #if defined (__cplusplus)
- extern "C" {
- #endif
-
-
-/* Note : This is an internal API.
- *        These APIs used to be exposed with ZSTDLIB_API,
- *        because it used to be the only way to invoke MT compression.
- *        Now, it's recommended to use ZSTD_compress2 and ZSTD_compressStream2()
- *        instead.
- *
- *        If you depend on these APIs and can't switch, then define
- *        ZSTD_LEGACY_MULTITHREADED_API when making the dynamic library.
- *        However, we may completely remove these functions in a future
- *        release, so please switch soon.
- *
- *        This API requires ZSTD_MULTITHREAD to be defined during compilation,
- *        otherwise ZSTDMT_createCCtx*() will fail.
- */
-
-#ifdef ZSTD_LEGACY_MULTITHREADED_API
-#  define ZSTDMT_API ZSTDLIB_API
-#else
-#  define ZSTDMT_API
-#endif
-
-/* ===   Dependencies   === */
-#include <stddef.h>                /* size_t */
-#define ZSTD_STATIC_LINKING_ONLY   /* ZSTD_parameters */
-#include "zstd.h"            /* ZSTD_inBuffer, ZSTD_outBuffer, ZSTDLIB_API */
-
-
-/* ===   Constants   === */
-#ifndef ZSTDMT_NBWORKERS_MAX
-#  define ZSTDMT_NBWORKERS_MAX 200
-#endif
-#ifndef ZSTDMT_JOBSIZE_MIN
-#  define ZSTDMT_JOBSIZE_MIN (1 MB)
-#endif
-#define ZSTDMT_JOBLOG_MAX   (MEM_32bits() ? 29 : 30)
-#define ZSTDMT_JOBSIZE_MAX  (MEM_32bits() ? (512 MB) : (1024 MB))
-
-
-/* ===   Memory management   === */
-typedef struct ZSTDMT_CCtx_s ZSTDMT_CCtx;
-/* Requires ZSTD_MULTITHREAD to be defined during compilation, otherwise it will return NULL. */
-ZSTDMT_API ZSTDMT_CCtx* ZSTDMT_createCCtx(unsigned nbWorkers);
-/* Requires ZSTD_MULTITHREAD to be defined during compilation, otherwise it will return NULL. */
-ZSTDMT_API ZSTDMT_CCtx* ZSTDMT_createCCtx_advanced(unsigned nbWorkers,
-                                                    ZSTD_customMem cMem);
-ZSTDMT_API size_t ZSTDMT_freeCCtx(ZSTDMT_CCtx* mtctx);
-
-ZSTDMT_API size_t ZSTDMT_sizeof_CCtx(ZSTDMT_CCtx* mtctx);
-
-
-/* ===   Simple one-pass compression function   === */
-
-ZSTDMT_API size_t ZSTDMT_compressCCtx(ZSTDMT_CCtx* mtctx,
-                                       void* dst, size_t dstCapacity,
-                                 const void* src, size_t srcSize,
-                                       int compressionLevel);
-
-
-
-/* ===   Streaming functions   === */
-
-ZSTDMT_API size_t ZSTDMT_initCStream(ZSTDMT_CCtx* mtctx, int compressionLevel);
-ZSTDMT_API size_t ZSTDMT_resetCStream(ZSTDMT_CCtx* mtctx, unsigned long long pledgedSrcSize);  /**< if srcSize is not known at reset time, use ZSTD_CONTENTSIZE_UNKNOWN. Note: for compatibility with older programs, 0 means the same as ZSTD_CONTENTSIZE_UNKNOWN, but it will change in the future to mean "empty" */
-
-ZSTDMT_API size_t ZSTDMT_nextInputSizeHint(const ZSTDMT_CCtx* mtctx);
-ZSTDMT_API size_t ZSTDMT_compressStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output, ZSTD_inBuffer* input);
-
-ZSTDMT_API size_t ZSTDMT_flushStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output);   /**< @return : 0 == all flushed; >0 : still some data to be flushed; or an error code (ZSTD_isError()) */
-ZSTDMT_API size_t ZSTDMT_endStream(ZSTDMT_CCtx* mtctx, ZSTD_outBuffer* output);     /**< @return : 0 == all flushed; >0 : still some data to be flushed; or an error code (ZSTD_isError()) */
-
-
-/* ===   Advanced functions and parameters  === */
-
-ZSTDMT_API size_t ZSTDMT_compress_advanced(ZSTDMT_CCtx* mtctx,
-                                          void* dst, size_t dstCapacity,
-                                    const void* src, size_t srcSize,
-                                    const ZSTD_CDict* cdict,
-                                          ZSTD_parameters params,
-                                          int overlapLog);
-
-ZSTDMT_API size_t ZSTDMT_initCStream_advanced(ZSTDMT_CCtx* mtctx,
-                                        const void* dict, size_t dictSize,   /* dict can be released after init, a local copy is preserved within zcs */
-                                        ZSTD_parameters params,
-                                        unsigned long long pledgedSrcSize);  /* pledgedSrcSize is optional and can be zero == unknown */
-
-ZSTDMT_API size_t ZSTDMT_initCStream_usingCDict(ZSTDMT_CCtx* mtctx,
-                                        const ZSTD_CDict* cdict,
-                                        ZSTD_frameParameters fparams,
-                                        unsigned long long pledgedSrcSize);  /* note : zero means empty */
-
-/* ZSTDMT_parameter :
- * List of parameters that can be set using ZSTDMT_setMTCtxParameter() */
-typedef enum {
-    ZSTDMT_p_jobSize,     /* Each job is compressed in parallel. By default, this value is dynamically determined depending on compression parameters. Can be set explicitly here. */
-    ZSTDMT_p_overlapLog,  /* Each job may reload a part of previous job to enhance compression ratio; 0 == no overlap, 6(default) == use 1/8th of window, >=9 == use full window. This is a "sticky" parameter : its value will be re-used on next compression job */
-    ZSTDMT_p_rsyncable    /* Enables rsyncable mode. */
-} ZSTDMT_parameter;
-
-/* ZSTDMT_setMTCtxParameter() :
- * allow setting individual parameters, one at a time, among a list of enums defined in ZSTDMT_parameter.
- * The function must be called typically after ZSTD_createCCtx() but __before ZSTDMT_init*() !__
- * Parameters not explicitly reset by ZSTDMT_init*() remain the same in consecutive compression sessions.
- * @return : 0, or an error code (which can be tested using ZSTD_isError()) */
-ZSTDMT_API size_t ZSTDMT_setMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSTDMT_parameter parameter, int value);
-
-/* ZSTDMT_getMTCtxParameter() :
- * Query the ZSTDMT_CCtx for a parameter value.
- * @return : 0, or an error code (which can be tested using ZSTD_isError()) */
-ZSTDMT_API size_t ZSTDMT_getMTCtxParameter(ZSTDMT_CCtx* mtctx, ZSTDMT_parameter parameter, int* value);
-
-
-/*! ZSTDMT_compressStream_generic() :
- *  Combines ZSTDMT_compressStream() with optional ZSTDMT_flushStream() or ZSTDMT_endStream()
- *  depending on flush directive.
- * @return : minimum amount of data still to be flushed
- *           0 if fully flushed
- *           or an error code
- *  note : needs to be init using any ZSTD_initCStream*() variant */
-ZSTDMT_API size_t ZSTDMT_compressStream_generic(ZSTDMT_CCtx* mtctx,
-                                                ZSTD_outBuffer* output,
-                                                ZSTD_inBuffer* input,
-                                                ZSTD_EndDirective endOp);
-
-
-/* ========================================================
- * ===  Private interface, for use by ZSTD_compress.c   ===
- * ===  Not exposed in libzstd. Never invoke directly   ===
- * ======================================================== */
-
- /*! ZSTDMT_toFlushNow()
-  *  Tell how many bytes are ready to be flushed immediately.
-  *  Probe the oldest active job (not yet entirely flushed) and check its output buffer.
-  *  If return 0, it means there is no active job,
-  *  or, it means oldest job is still active, but everything produced has been flushed so far,
-  *  therefore flushing is limited by speed of oldest job. */
-size_t ZSTDMT_toFlushNow(ZSTDMT_CCtx* mtctx);
-
-/*! ZSTDMT_CCtxParam_setMTCtxParameter()
- *  like ZSTDMT_setMTCtxParameter(), but into a ZSTD_CCtx_Params */
-size_t ZSTDMT_CCtxParam_setMTCtxParameter(ZSTD_CCtx_params* params, ZSTDMT_parameter parameter, int value);
-
-/*! ZSTDMT_CCtxParam_setNbWorkers()
- *  Set nbWorkers, and clamp it.
- *  Also reset jobSize and overlapLog */
-size_t ZSTDMT_CCtxParam_setNbWorkers(ZSTD_CCtx_params* params, unsigned nbWorkers);
-
-/*! ZSTDMT_updateCParams_whileCompressing() :
- *  Updates only a selected set of compression parameters, to remain compatible with current frame.
- *  New parameters will be applied to next compression job. */
-void ZSTDMT_updateCParams_whileCompressing(ZSTDMT_CCtx* mtctx, const ZSTD_CCtx_params* cctxParams);
-
-/*! ZSTDMT_getFrameProgression():
- *  tells how much data has been consumed (input) and produced (output) for current frame.
- *  able to count progression inside worker threads.
- */
-ZSTD_frameProgression ZSTDMT_getFrameProgression(ZSTDMT_CCtx* mtctx);
-
-
-/*! ZSTDMT_initCStream_internal() :
- *  Private use only. Init streaming operation.
- *  expects params to be valid.
- *  must receive dict, or cdict, or none, but not both.
- *  @return : 0, or an error code */
-size_t ZSTDMT_initCStream_internal(ZSTDMT_CCtx* zcs,
-                    const void* dict, size_t dictSize, ZSTD_dictContentType_e dictContentType,
-                    const ZSTD_CDict* cdict,
-                    ZSTD_CCtx_params params, unsigned long long pledgedSrcSize);
-
-
-#if defined (__cplusplus)
-}
-#endif
-
-#endif   /* ZSTDMT_COMPRESS_H */
diff --git a/vendor/github.com/Shopify/sarama/.gitignore b/vendor/github.com/Shopify/sarama/.gitignore
index 6e362e4..2c9adc2 100644
--- a/vendor/github.com/Shopify/sarama/.gitignore
+++ b/vendor/github.com/Shopify/sarama/.gitignore
@@ -25,3 +25,5 @@
 
 coverage.txt
 profile.out
+
+simplest-uncommitted-msg-0.1-jar-with-dependencies.jar
diff --git a/vendor/github.com/Shopify/sarama/.golangci.yml b/vendor/github.com/Shopify/sarama/.golangci.yml
new file mode 100644
index 0000000..09e5c46
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/.golangci.yml
@@ -0,0 +1,74 @@
+run:
+  timeout: 5m
+  deadline: 10m
+
+linters-settings:
+  govet:
+    check-shadowing: false
+  golint:
+    min-confidence: 0
+  gocyclo:
+    min-complexity: 99
+  maligned:
+    suggest-new: true
+  dupl:
+    threshold: 100
+  goconst:
+    min-len: 2
+    min-occurrences: 3
+  misspell:
+    locale: US
+  goimports:
+    local-prefixes: github.com/Shopify/sarama
+  gocritic:
+    enabled-tags:
+      - diagnostic
+      - experimental
+      - opinionated
+      - performance
+      - style
+    disabled-checks:
+      - wrapperFunc
+      - ifElseChain
+  funlen:
+    lines: 300
+    statements: 300
+
+linters:
+  disable-all: true
+  enable:
+    - bodyclose
+    - deadcode
+    - depguard
+    - dogsled
+    # - dupl
+    - errcheck
+    - funlen
+    - gochecknoinits
+    # - goconst
+    # - gocritic
+    - gocyclo
+    - gofmt
+    - goimports
+    # - golint
+    - gosec
+    # - gosimple
+    - govet
+    # - ineffassign
+    # - misspell
+    # - nakedret
+    # - scopelint
+    - staticcheck
+    - structcheck
+    # - stylecheck
+    - typecheck
+    - unconvert
+    - unused
+    - varcheck
+    - whitespace
+
+issues:
+  exclude:
+    - "G404: Use of weak random number generator"
+  # maximum count of issues with the same text. set to 0 for unlimited. default is 3.
+  max-same-issues: 0
diff --git a/vendor/github.com/Shopify/sarama/.travis.yml b/vendor/github.com/Shopify/sarama/.travis.yml
deleted file mode 100644
index 4331fa1..0000000
--- a/vendor/github.com/Shopify/sarama/.travis.yml
+++ /dev/null
@@ -1,37 +0,0 @@
-dist: xenial
-language: go
-go:
-- 1.11.x
-- 1.12.x
-
-env:
-  global:
-  - KAFKA_PEERS=localhost:9091,localhost:9092,localhost:9093,localhost:9094,localhost:9095
-  - TOXIPROXY_ADDR=http://localhost:8474
-  - KAFKA_INSTALL_ROOT=/home/travis/kafka
-  - KAFKA_HOSTNAME=localhost
-  - DEBUG=true
-  matrix:
-  - KAFKA_VERSION=2.1.1 KAFKA_SCALA_VERSION=2.12
-  - KAFKA_VERSION=2.2.1 KAFKA_SCALA_VERSION=2.12
-  - KAFKA_VERSION=2.3.0 KAFKA_SCALA_VERSION=2.12
-
-before_install:
-- export REPOSITORY_ROOT=${TRAVIS_BUILD_DIR}
-- vagrant/install_cluster.sh
-- vagrant/boot_cluster.sh
-- vagrant/create_topics.sh
-- vagrant/run_java_producer.sh
-
-install: make install_dependencies
-
-script:
-- make test
-- make vet
-- make errcheck
-- if [[ "$TRAVIS_GO_VERSION" == 1.12* ]]; then make fmt; fi
-
-after_success:
-- bash <(curl -s https://codecov.io/bash)
-
-after_script: vagrant/halt_cluster.sh
diff --git a/vendor/github.com/Shopify/sarama/CHANGELOG.md b/vendor/github.com/Shopify/sarama/CHANGELOG.md
index 02bd0ff..59ccd1d 100644
--- a/vendor/github.com/Shopify/sarama/CHANGELOG.md
+++ b/vendor/github.com/Shopify/sarama/CHANGELOG.md
@@ -1,5 +1,237 @@
 # Changelog
 
+#### Unreleased
+
+#### Version 1.28.0 (2021-02-15)
+
+**Note that with this release we change `RoundRobinBalancer` strategy to match Java client behavior. See #1788 for details.**
+
+- #1870 - @kvch - Update Kerberos library to latest major
+- #1876 - @bai - Update docs, reference pkg.go.dev
+- #1846 - @wclaeys - Do not ignore Consumer.Offsets.AutoCommit.Enable config on Close
+- #1747 - @XSAM - fix: mock sync producer does not handle the offset while sending messages
+- #1863 - @bai - Add support for Kafka 2.7.0 + update lz4 and klauspost/compress dependencies
+- #1788 - @kzinglzy - feat[balance_strategy]: announcing a new round robin balance strategy
+- #1862 - @bai - Fix CI setenv permissions issues
+- #1832 - @ilyakaznacheev - Update Godoc link to pkg.go.dev
+- #1822 - @danp - KIP-392: Allow consumers to fetch from closest replica
+
+#### Version 1.27.2 (2020-10-21)
+
+# Improvements
+
+#1750 - @krantideep95 Adds missing mock responses for mocking consumer group
+
+# Fixes
+
+#1817 - reverts #1785 - Add private method to Client interface to prevent implementation
+
+#### Version 1.27.1 (2020-10-07)
+
+# Improvements
+
+#1775 - @d1egoaz - Adds a Producer Interceptor example
+#1781 - @justin-chen - Refresh brokers given list of seed brokers
+#1784 - @justin-chen - Add randomize seed broker method
+#1790 - @d1egoaz - remove example binary
+#1798 - @bai - Test against Go 1.15
+#1785 - @justin-chen - Add private method to Client interface to prevent implementation
+#1802 - @uvw - Support Go 1.13 error unwrapping
+
+# Fixes
+
+#1791 - @stanislavkozlovski - bump default version to 1.0.0
+
+#### Version 1.27.0 (2020-08-11)
+
+# Improvements
+
+#1466 - @rubenvp8510  - Expose kerberos fast negotiation configuration
+#1695 - @KJTsanaktsidis - Use docker-compose to run the functional tests
+#1699 - @wclaeys  - Consumer group support for manually comitting offsets
+#1714 - @bai - Bump Go to version 1.14.3, golangci-lint to 1.27.0
+#1726 - @d1egoaz - Include zstd on the functional tests
+#1730 - @d1egoaz - KIP-42 Add producer and consumer interceptors
+#1738 - @varun06 - fixed variable names that are named same as some std lib package names
+#1741 - @varun06 - updated zstd dependency to latest v1.10.10
+#1743 - @varun06 - Fixed declaration dependencies and other lint issues in code base
+#1763 - @alrs - remove deprecated tls options from test
+#1769 - @bai - Add support for Kafka 2.6.0
+
+# Fixes
+
+#1697 - @kvch - Use gofork for encoding/asn1 to fix ASN errors during Kerberos authentication
+#1744 - @alrs  - Fix isBalanced Function Signature
+
+#### Version 1.26.4 (2020-05-19)
+
+# Fixes
+
+- #1701 - @d1egoaz - Set server name only for the current broker
+- #1694 - @dnwe - testfix: set KAFKA_HEAP_OPTS for zk and kafka
+
+#### Version 1.26.3 (2020-05-07)
+
+# Fixes
+
+- #1692 - @d1egoaz - Set tls ServerName to fix issue: either ServerName or InsecureSkipVerify must be specified in the tls.Config
+
+#### Version 1.26.2 (2020-05-06)
+
+# ⚠️ Known Issues
+
+This release has been marked as not ready for production and may be unstable, please use v1.26.4.
+
+# Improvements
+
+- #1560 - @iyacontrol - add sync pool for gzip 1-9
+- #1605 - @dnwe - feat: protocol support for V11 fetch w/ rackID
+- #1617 - @sladkoff / @dwi-di / @random-dwi - Add support for alter/list partition reassignements APIs
+- #1632 - @bai - Add support for Go 1.14
+- #1640 - @random-dwi - Feature/fix list partition reassignments
+- #1646 - @mimaison - Add DescribeLogDirs to admin client
+- #1667 - @bai - Add support for kafka 2.5.0
+
+# Fixes
+
+- #1594 - @sladkoff - Sets ConfigEntry.Default flag in addition to the ConfigEntry.Source for Kafka versions > V1_1_0_0
+- #1601 - @alrs - fix: remove use of testing.T.FailNow() inside goroutine
+- #1602 - @d1egoaz - adds a note about consumer groups Consume method
+- #1607 - @darklore - Fix memory leak when Broker.Open and Broker.Close called repeatedly
+- #1613 - @wblakecaldwell - Updated "retrying" log message when BackoffFunc implemented
+- #1614 - @alrs - produce_response.go: Remove Unused Functions
+- #1619 - @alrs - tools/kafka-producer-performance: prune unused flag variables
+- #1639 - @agriffaut - Handle errors with no message but error code
+- #1643 - @kzinglzy - fix `config.net.keepalive`
+- #1644 - @KJTsanaktsidis - Fix brokers continually allocating new Session IDs
+- #1645 - @Stephan14 - Remove broker(s) which no longer exist in metadata
+- #1650 - @lavoiesl - Return the response error in heartbeatLoop
+- #1661 - @KJTsanaktsidis - Fix "broker received out of order sequence" when brokers die
+- #1666 - @KevinJCross - Bugfix: Allow TLS connections to work over socks proxy.
+
+#### Version 1.26.1 (2020-02-04)
+
+Improvements:
+- Add requests-in-flight metric ([1539](https://github.com/Shopify/sarama/pull/1539))
+- Fix misleading example for cluster admin ([1595](https://github.com/Shopify/sarama/pull/1595))
+- Replace Travis with GitHub Actions, linters housekeeping ([1573](https://github.com/Shopify/sarama/pull/1573))
+- Allow BalanceStrategy to provide custom assignment data ([1592](https://github.com/Shopify/sarama/pull/1592))
+
+Bug Fixes:
+- Adds back Consumer.Offsets.CommitInterval to fix API ([1590](https://github.com/Shopify/sarama/pull/1590))
+- Fix error message s/CommitInterval/AutoCommit.Interval ([1589](https://github.com/Shopify/sarama/pull/1589))
+
+#### Version 1.26.0 (2020-01-24)
+
+New Features:
+- Enable zstd compression
+  ([1574](https://github.com/Shopify/sarama/pull/1574),
+  [1582](https://github.com/Shopify/sarama/pull/1582))
+- Support headers in tools kafka-console-producer
+  ([1549](https://github.com/Shopify/sarama/pull/1549))
+
+Improvements:
+- Add SASL AuthIdentity to SASL frames (authzid)
+  ([1585](https://github.com/Shopify/sarama/pull/1585)).
+
+Bug Fixes:
+- Sending messages with ZStd compression enabled fails in multiple ways
+  ([1252](https://github.com/Shopify/sarama/issues/1252)).
+- Use the broker for any admin on BrokerConfig
+  ([1571](https://github.com/Shopify/sarama/pull/1571)).
+- Set DescribeConfigRequest Version field
+  ([1576](https://github.com/Shopify/sarama/pull/1576)).
+- ConsumerGroup flooding logs with client/metadata update req
+  ([1578](https://github.com/Shopify/sarama/pull/1578)).
+- MetadataRequest version in DescribeCluster
+  ([1580](https://github.com/Shopify/sarama/pull/1580)).
+- Fix deadlock in consumer group handleError
+  ([1581](https://github.com/Shopify/sarama/pull/1581))
+- Fill in the Fetch{Request,Response} protocol
+  ([1582](https://github.com/Shopify/sarama/pull/1582)).
+- Retry topic request on ControllerNotAvailable
+  ([1586](https://github.com/Shopify/sarama/pull/1586)).
+
+#### Version 1.25.0 (2020-01-13)
+
+New Features:
+- Support TLS protocol in kafka-producer-performance
+  ([1538](https://github.com/Shopify/sarama/pull/1538)).
+- Add support for kafka 2.4.0
+  ([1552](https://github.com/Shopify/sarama/pull/1552)).
+
+Improvements:
+- Allow the Consumer to disable auto-commit offsets
+  ([1164](https://github.com/Shopify/sarama/pull/1164)).
+- Produce records with consistent timestamps
+  ([1455](https://github.com/Shopify/sarama/pull/1455)).
+
+Bug Fixes:
+- Fix incorrect SetTopicMetadata name mentions
+  ([1534](https://github.com/Shopify/sarama/pull/1534)).
+- Fix client.tryRefreshMetadata Println
+  ([1535](https://github.com/Shopify/sarama/pull/1535)).
+- Fix panic on calling updateMetadata on closed client
+  ([1531](https://github.com/Shopify/sarama/pull/1531)).
+- Fix possible faulty metrics in TestFuncProducing
+  ([1545](https://github.com/Shopify/sarama/pull/1545)).
+
+#### Version 1.24.1 (2019-10-31)
+
+New Features:
+- Add DescribeLogDirs Request/Response pair
+  ([1520](https://github.com/Shopify/sarama/pull/1520)).
+
+Bug Fixes:
+- Fix ClusterAdmin returning invalid controller ID on DescribeCluster
+  ([1518](https://github.com/Shopify/sarama/pull/1518)).
+- Fix issue with consumergroup not rebalancing when new partition is added
+  ([1525](https://github.com/Shopify/sarama/pull/1525)).
+- Ensure consistent use of read/write deadlines
+  ([1529](https://github.com/Shopify/sarama/pull/1529)).
+
+#### Version 1.24.0 (2019-10-09)
+
+New Features:
+- Add sticky partition assignor
+  ([1416](https://github.com/Shopify/sarama/pull/1416)).
+- Switch from cgo zstd package to pure Go implementation
+  ([1477](https://github.com/Shopify/sarama/pull/1477)).
+
+Improvements:
+- Allow creating ClusterAdmin from client
+  ([1415](https://github.com/Shopify/sarama/pull/1415)).
+- Set KafkaVersion in ListAcls method
+  ([1452](https://github.com/Shopify/sarama/pull/1452)).
+- Set request version in CreateACL ClusterAdmin method
+  ([1458](https://github.com/Shopify/sarama/pull/1458)).
+- Set request version in DeleteACL ClusterAdmin method
+  ([1461](https://github.com/Shopify/sarama/pull/1461)).
+- Handle missed error codes on TopicMetaDataRequest and GroupCoordinatorRequest
+  ([1464](https://github.com/Shopify/sarama/pull/1464)).
+- Remove direct usage of gofork
+  ([1465](https://github.com/Shopify/sarama/pull/1465)).
+- Add support for Go 1.13
+  ([1478](https://github.com/Shopify/sarama/pull/1478)).
+- Improve behavior of NewMockListAclsResponse
+  ([1481](https://github.com/Shopify/sarama/pull/1481)).
+
+Bug Fixes:
+- Fix race condition in consumergroup example
+  ([1434](https://github.com/Shopify/sarama/pull/1434)).
+- Fix brokerProducer goroutine leak
+  ([1442](https://github.com/Shopify/sarama/pull/1442)).
+- Use released version of lz4 library
+  ([1469](https://github.com/Shopify/sarama/pull/1469)).
+- Set correct version in MockDeleteTopicsResponse
+  ([1484](https://github.com/Shopify/sarama/pull/1484)).
+- Fix CLI help message typo
+  ([1494](https://github.com/Shopify/sarama/pull/1494)).
+
+Known Issues:
+- Please **don't** use Zstd, as it doesn't work right now.
+  See https://github.com/Shopify/sarama/issues/1252
+
 #### Version 1.23.1 (2019-07-22)
 
 Bug Fixes:
diff --git a/vendor/github.com/Shopify/sarama/Makefile b/vendor/github.com/Shopify/sarama/Makefile
index 360b220..4714d77 100644
--- a/vendor/github.com/Shopify/sarama/Makefile
+++ b/vendor/github.com/Shopify/sarama/Makefile
@@ -1,52 +1,31 @@
-export GO111MODULE=on
+default: fmt get update test lint
 
-default: fmt vet errcheck test lint
+GO       := go
+GOBUILD  := CGO_ENABLED=0 $(GO) build $(BUILD_FLAG)
+GOTEST   := $(GO) test -gcflags='-l' -p 3 -v -race -timeout 6m -coverprofile=profile.out -covermode=atomic
 
-# Taken from https://github.com/codecov/example-go#caveat-multiple-files
-.PHONY: test
-test:
-	echo "" > coverage.txt
-	for d in `go list ./...`; do \
-		go test -p 1 -v -timeout 240s -race -coverprofile=profile.out -covermode=atomic $$d || exit 1; \
-		if [ -f profile.out ]; then \
-			cat profile.out >> coverage.txt; \
-			rm profile.out; \
-		fi \
-	done
+FILES    := $(shell find . -name '*.go' -type f -not -name '*.pb.go' -not -name '*_generated.go' -not -name '*_test.go')
+TESTS    := $(shell find . -name '*.go' -type f -not -name '*.pb.go' -not -name '*_generated.go' -name '*_test.go')
 
-GOLINT := $(shell command -v golint)
-
-.PHONY: lint
-lint:
-ifndef GOLINT
-	go get golang.org/x/lint/golint
-endif
-	go list ./... | xargs golint
-
-.PHONY: vet
-vet:
-	go vet ./...
-
-ERRCHECK := $(shell command -v errcheck)
-# See https://github.com/kisielk/errcheck/pull/141 for details on ignorepkg
-.PHONY: errcheck
-errcheck:
-ifndef ERRCHECK
-	go get github.com/kisielk/errcheck
-endif
-	errcheck -ignorepkg fmt github.com/Shopify/sarama/...
-
-.PHONY: fmt
-fmt:
-	@if [ -n "$$(go fmt ./...)" ]; then echo 'Please run go fmt on your code.' && exit 1; fi
-
-.PHONY : install_dependencies
-install_dependencies: get
-
-.PHONY: get
 get:
-	go get -t -v ./...
+	$(GO) get ./...
+	$(GO) mod verify
+	$(GO) mod tidy
 
-.PHONY: clean
-clean:
-	go clean ./...
+update:
+	$(GO) get -u -v ./...
+	$(GO) mod verify
+	$(GO) mod tidy
+
+fmt:
+	gofmt -s -l -w $(FILES) $(TESTS)
+
+lint:
+	GOFLAGS="-tags=functional" golangci-lint run
+
+test:
+	$(GOTEST) ./...
+
+.PHONY: test_functional
+test_functional:
+	$(GOTEST) -tags=functional ./...
diff --git a/vendor/github.com/Shopify/sarama/README.md b/vendor/github.com/Shopify/sarama/README.md
index 4cd736b..f2beb73 100644
--- a/vendor/github.com/Shopify/sarama/README.md
+++ b/vendor/github.com/Shopify/sarama/README.md
@@ -1,39 +1,36 @@
-sarama
-======
+# sarama
 
-[![GoDoc](https://godoc.org/github.com/Shopify/sarama?status.svg)](https://godoc.org/github.com/Shopify/sarama)
+[![Go Reference](https://pkg.go.dev/badge/github.com/Shopify/sarama.svg)](https://pkg.go.dev/github.com/Shopify/sarama)
 [![Build Status](https://travis-ci.org/Shopify/sarama.svg?branch=master)](https://travis-ci.org/Shopify/sarama)
 [![Coverage](https://codecov.io/gh/Shopify/sarama/branch/master/graph/badge.svg)](https://codecov.io/gh/Shopify/sarama)
 
 Sarama is an MIT-licensed Go client library for [Apache Kafka](https://kafka.apache.org/) version 0.8 (and later).
 
-### Getting started
+## Getting started
 
-- API documentation and examples are available via [godoc](https://godoc.org/github.com/Shopify/sarama).
+- API documentation and examples are available via [pkg.go.dev](https://pkg.go.dev/github.com/Shopify/sarama).
 - Mocks for testing are available in the [mocks](./mocks) subpackage.
 - The [examples](./examples) directory contains more elaborate example applications.
 - The [tools](./tools) directory contains command line tools that can be useful for testing, diagnostics, and instrumentation.
 
 You might also want to look at the [Frequently Asked Questions](https://github.com/Shopify/sarama/wiki/Frequently-Asked-Questions).
 
-### Compatibility and API stability
+## Compatibility and API stability
 
 Sarama provides a "2 releases + 2 months" compatibility guarantee: we support
 the two latest stable releases of Kafka and Go, and we provide a two month
 grace period for older releases. This means we currently officially support
-Go 1.11 through 1.12, and Kafka 2.0 through 2.3, although older releases are
+Go 1.15 through 1.16, and Kafka 2.6 through 2.8, although older releases are
 still likely to work.
 
 Sarama follows semantic versioning and provides API stability via the gopkg.in service.
 You can import a version with a guaranteed stable API via http://gopkg.in/Shopify/sarama.v1.
 A changelog is available [here](CHANGELOG.md).
 
-### Contributing
+## Contributing
 
-* Get started by checking our [contribution guidelines](https://github.com/Shopify/sarama/blob/master/.github/CONTRIBUTING.md).
-* Read the [Sarama wiki](https://github.com/Shopify/sarama/wiki) for more
-  technical and design details.
-* The [Kafka Protocol Specification](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol)
-  contains a wealth of useful information.
-* For more general issues, there is [a google group](https://groups.google.com/forum/#!forum/kafka-clients) for Kafka client developers.
-* If you have any questions, just ask!
+- Get started by checking our [contribution guidelines](https://github.com/Shopify/sarama/blob/master/.github/CONTRIBUTING.md).
+- Read the [Sarama wiki](https://github.com/Shopify/sarama/wiki) for more technical and design details.
+- The [Kafka Protocol Specification](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) contains a wealth of useful information.
+- For more general issues, there is [a google group](https://groups.google.com/forum/#!forum/kafka-clients) for Kafka client developers.
+- If you have any questions, just ask!
diff --git a/vendor/github.com/Shopify/sarama/Vagrantfile b/vendor/github.com/Shopify/sarama/Vagrantfile
index f4b848a..07d7ffb 100644
--- a/vendor/github.com/Shopify/sarama/Vagrantfile
+++ b/vendor/github.com/Shopify/sarama/Vagrantfile
@@ -1,14 +1,8 @@
-# -*- mode: ruby -*-
-# vi: set ft=ruby :
-
-# Vagrantfile API/syntax version. Don't touch unless you know what you're doing!
-VAGRANTFILE_API_VERSION = "2"
-
 # We have 5 * 192MB ZK processes and 5 * 320MB Kafka processes => 2560MB
 MEMORY = 3072
 
-Vagrant.configure(VAGRANTFILE_API_VERSION) do |config|
-  config.vm.box = "ubuntu/trusty64"
+Vagrant.configure("2") do |config|
+  config.vm.box = "ubuntu/bionic64"
 
   config.vm.provision :shell, path: "vagrant/provision.sh"
 
diff --git a/vendor/github.com/Shopify/sarama/acl_bindings.go b/vendor/github.com/Shopify/sarama/acl_bindings.go
index 50b689d..13440be 100644
--- a/vendor/github.com/Shopify/sarama/acl_bindings.go
+++ b/vendor/github.com/Shopify/sarama/acl_bindings.go
@@ -1,6 +1,6 @@
 package sarama
 
-//Resource holds information about acl resource type
+// Resource holds information about acl resource type
 type Resource struct {
 	ResourceType        AclResourceType
 	ResourceName        string
@@ -46,7 +46,7 @@
 	return nil
 }
 
-//Acl holds information about acl type
+// Acl holds information about acl type
 type Acl struct {
 	Principal      string
 	Host           string
@@ -93,7 +93,7 @@
 	return nil
 }
 
-//ResourceAcls is an acl resource type
+// ResourceAcls is an acl resource type
 type ResourceAcls struct {
 	Resource
 	Acls []*Acl
diff --git a/vendor/github.com/Shopify/sarama/acl_create_request.go b/vendor/github.com/Shopify/sarama/acl_create_request.go
index da1cdef..449102f 100644
--- a/vendor/github.com/Shopify/sarama/acl_create_request.go
+++ b/vendor/github.com/Shopify/sarama/acl_create_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//CreateAclsRequest is an acl creation request
+// CreateAclsRequest is an acl creation request
 type CreateAclsRequest struct {
 	Version      int16
 	AclCreations []*AclCreation
@@ -47,6 +47,10 @@
 	return c.Version
 }
 
+func (c *CreateAclsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (c *CreateAclsRequest) requiredVersion() KafkaVersion {
 	switch c.Version {
 	case 1:
@@ -56,7 +60,7 @@
 	}
 }
 
-//AclCreation is a wrapper around Resource and Acl type
+// AclCreation is a wrapper around Resource and Acl type
 type AclCreation struct {
 	Resource
 	Acl
diff --git a/vendor/github.com/Shopify/sarama/acl_create_response.go b/vendor/github.com/Shopify/sarama/acl_create_response.go
index f5a5e9a..21d6c34 100644
--- a/vendor/github.com/Shopify/sarama/acl_create_response.go
+++ b/vendor/github.com/Shopify/sarama/acl_create_response.go
@@ -2,7 +2,7 @@
 
 import "time"
 
-//CreateAclsResponse is a an acl reponse creation type
+// CreateAclsResponse is a an acl response creation type
 type CreateAclsResponse struct {
 	ThrottleTime         time.Duration
 	AclCreationResponses []*AclCreationResponse
@@ -55,11 +55,15 @@
 	return 0
 }
 
+func (c *CreateAclsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (c *CreateAclsResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
 
-//AclCreationResponse is an acl creation response type
+// AclCreationResponse is an acl creation response type
 type AclCreationResponse struct {
 	Err    KError
 	ErrMsg *string
diff --git a/vendor/github.com/Shopify/sarama/acl_delete_request.go b/vendor/github.com/Shopify/sarama/acl_delete_request.go
index 15908ea..5e5c03b 100644
--- a/vendor/github.com/Shopify/sarama/acl_delete_request.go
+++ b/vendor/github.com/Shopify/sarama/acl_delete_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//DeleteAclsRequest is a delete acl request
+// DeleteAclsRequest is a delete acl request
 type DeleteAclsRequest struct {
 	Version int
 	Filters []*AclFilter
@@ -48,6 +48,10 @@
 	return int16(d.Version)
 }
 
+func (d *DeleteAclsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (d *DeleteAclsRequest) requiredVersion() KafkaVersion {
 	switch d.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/acl_delete_response.go b/vendor/github.com/Shopify/sarama/acl_delete_response.go
index 6529565..cd33749 100644
--- a/vendor/github.com/Shopify/sarama/acl_delete_response.go
+++ b/vendor/github.com/Shopify/sarama/acl_delete_response.go
@@ -2,7 +2,7 @@
 
 import "time"
 
-//DeleteAclsResponse is a delete acl response
+// DeleteAclsResponse is a delete acl response
 type DeleteAclsResponse struct {
 	Version         int16
 	ThrottleTime    time.Duration
@@ -53,14 +53,18 @@
 }
 
 func (d *DeleteAclsResponse) version() int16 {
-	return int16(d.Version)
+	return d.Version
+}
+
+func (d *DeleteAclsResponse) headerVersion() int16 {
+	return 0
 }
 
 func (d *DeleteAclsResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
 
-//FilterResponse is a filter response type
+// FilterResponse is a filter response type
 type FilterResponse struct {
 	Err          KError
 	ErrMsg       *string
@@ -111,7 +115,7 @@
 	return nil
 }
 
-//MatchingAcl is a matching acl type
+// MatchingAcl is a matching acl type
 type MatchingAcl struct {
 	Err    KError
 	ErrMsg *string
diff --git a/vendor/github.com/Shopify/sarama/acl_describe_request.go b/vendor/github.com/Shopify/sarama/acl_describe_request.go
index 5222d46..e0fe902 100644
--- a/vendor/github.com/Shopify/sarama/acl_describe_request.go
+++ b/vendor/github.com/Shopify/sarama/acl_describe_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//DescribeAclsRequest is a secribe acl request type
+// DescribeAclsRequest is a secribe acl request type
 type DescribeAclsRequest struct {
 	Version int
 	AclFilter
@@ -25,6 +25,10 @@
 	return int16(d.Version)
 }
 
+func (d *DescribeAclsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (d *DescribeAclsRequest) requiredVersion() KafkaVersion {
 	switch d.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/acl_describe_response.go b/vendor/github.com/Shopify/sarama/acl_describe_response.go
index 12126e5..3255fd4 100644
--- a/vendor/github.com/Shopify/sarama/acl_describe_response.go
+++ b/vendor/github.com/Shopify/sarama/acl_describe_response.go
@@ -2,7 +2,7 @@
 
 import "time"
 
-//DescribeAclsResponse is a describe acl response type
+// DescribeAclsResponse is a describe acl response type
 type DescribeAclsResponse struct {
 	Version      int16
 	ThrottleTime time.Duration
@@ -74,7 +74,11 @@
 }
 
 func (d *DescribeAclsResponse) version() int16 {
-	return int16(d.Version)
+	return d.Version
+}
+
+func (d *DescribeAclsResponse) headerVersion() int16 {
+	return 0
 }
 
 func (d *DescribeAclsResponse) requiredVersion() KafkaVersion {
diff --git a/vendor/github.com/Shopify/sarama/acl_filter.go b/vendor/github.com/Shopify/sarama/acl_filter.go
index fad5558..b380161 100644
--- a/vendor/github.com/Shopify/sarama/acl_filter.go
+++ b/vendor/github.com/Shopify/sarama/acl_filter.go
@@ -46,7 +46,6 @@
 
 	if a.Version == 1 {
 		pattern, err := pd.getInt8()
-
 		if err != nil {
 			return err
 		}
diff --git a/vendor/github.com/Shopify/sarama/acl_types.go b/vendor/github.com/Shopify/sarama/acl_types.go
index c10ad7b..c3ba8dd 100644
--- a/vendor/github.com/Shopify/sarama/acl_types.go
+++ b/vendor/github.com/Shopify/sarama/acl_types.go
@@ -1,5 +1,10 @@
 package sarama
 
+import (
+	"fmt"
+	"strings"
+)
+
 type (
 	AclOperation int
 
@@ -27,6 +32,61 @@
 	AclOperationIdempotentWrite
 )
 
+func (a *AclOperation) String() string {
+	mapping := map[AclOperation]string{
+		AclOperationUnknown:         "Unknown",
+		AclOperationAny:             "Any",
+		AclOperationAll:             "All",
+		AclOperationRead:            "Read",
+		AclOperationWrite:           "Write",
+		AclOperationCreate:          "Create",
+		AclOperationDelete:          "Delete",
+		AclOperationAlter:           "Alter",
+		AclOperationDescribe:        "Describe",
+		AclOperationClusterAction:   "ClusterAction",
+		AclOperationDescribeConfigs: "DescribeConfigs",
+		AclOperationAlterConfigs:    "AlterConfigs",
+		AclOperationIdempotentWrite: "IdempotentWrite",
+	}
+	s, ok := mapping[*a]
+	if !ok {
+		s = mapping[AclOperationUnknown]
+	}
+	return s
+}
+
+// MarshalText returns the text form of the AclOperation (name without prefix)
+func (a *AclOperation) MarshalText() ([]byte, error) {
+	return []byte(a.String()), nil
+}
+
+// UnmarshalText takes a text reprentation of the operation and converts it to an AclOperation
+func (a *AclOperation) UnmarshalText(text []byte) error {
+	normalized := strings.ToLower(string(text))
+	mapping := map[string]AclOperation{
+		"unknown":         AclOperationUnknown,
+		"any":             AclOperationAny,
+		"all":             AclOperationAll,
+		"read":            AclOperationRead,
+		"write":           AclOperationWrite,
+		"create":          AclOperationCreate,
+		"delete":          AclOperationDelete,
+		"alter":           AclOperationAlter,
+		"describe":        AclOperationDescribe,
+		"clusteraction":   AclOperationClusterAction,
+		"describeconfigs": AclOperationDescribeConfigs,
+		"alterconfigs":    AclOperationAlterConfigs,
+		"idempotentwrite": AclOperationIdempotentWrite,
+	}
+	ao, ok := mapping[normalized]
+	if !ok {
+		*a = AclOperationUnknown
+		return fmt.Errorf("no acl operation with name %s", normalized)
+	}
+	*a = ao
+	return nil
+}
+
 // ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/acl/AclPermissionType.java
 const (
 	AclPermissionUnknown AclPermissionType = iota
@@ -35,6 +95,44 @@
 	AclPermissionAllow
 )
 
+func (a *AclPermissionType) String() string {
+	mapping := map[AclPermissionType]string{
+		AclPermissionUnknown: "Unknown",
+		AclPermissionAny:     "Any",
+		AclPermissionDeny:    "Deny",
+		AclPermissionAllow:   "Allow",
+	}
+	s, ok := mapping[*a]
+	if !ok {
+		s = mapping[AclPermissionUnknown]
+	}
+	return s
+}
+
+// MarshalText returns the text form of the AclPermissionType (name without prefix)
+func (a *AclPermissionType) MarshalText() ([]byte, error) {
+	return []byte(a.String()), nil
+}
+
+// UnmarshalText takes a text reprentation of the permission type and converts it to an AclPermissionType
+func (a *AclPermissionType) UnmarshalText(text []byte) error {
+	normalized := strings.ToLower(string(text))
+	mapping := map[string]AclPermissionType{
+		"unknown": AclPermissionUnknown,
+		"any":     AclPermissionAny,
+		"deny":    AclPermissionDeny,
+		"allow":   AclPermissionAllow,
+	}
+
+	apt, ok := mapping[normalized]
+	if !ok {
+		*a = AclPermissionUnknown
+		return fmt.Errorf("no acl permission with name %s", normalized)
+	}
+	*a = apt
+	return nil
+}
+
 // ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/resource/ResourceType.java
 const (
 	AclResourceUnknown AclResourceType = iota
@@ -43,8 +141,53 @@
 	AclResourceGroup
 	AclResourceCluster
 	AclResourceTransactionalID
+	AclResourceDelegationToken
 )
 
+func (a *AclResourceType) String() string {
+	mapping := map[AclResourceType]string{
+		AclResourceUnknown:         "Unknown",
+		AclResourceAny:             "Any",
+		AclResourceTopic:           "Topic",
+		AclResourceGroup:           "Group",
+		AclResourceCluster:         "Cluster",
+		AclResourceTransactionalID: "TransactionalID",
+		AclResourceDelegationToken: "DelegationToken",
+	}
+	s, ok := mapping[*a]
+	if !ok {
+		s = mapping[AclResourceUnknown]
+	}
+	return s
+}
+
+// MarshalText returns the text form of the AclResourceType (name without prefix)
+func (a *AclResourceType) MarshalText() ([]byte, error) {
+	return []byte(a.String()), nil
+}
+
+// UnmarshalText takes a text reprentation of the resource type and converts it to an AclResourceType
+func (a *AclResourceType) UnmarshalText(text []byte) error {
+	normalized := strings.ToLower(string(text))
+	mapping := map[string]AclResourceType{
+		"unknown":         AclResourceUnknown,
+		"any":             AclResourceAny,
+		"topic":           AclResourceTopic,
+		"group":           AclResourceGroup,
+		"cluster":         AclResourceCluster,
+		"transactionalid": AclResourceTransactionalID,
+		"delegationtoken": AclResourceDelegationToken,
+	}
+
+	art, ok := mapping[normalized]
+	if !ok {
+		*a = AclResourceUnknown
+		return fmt.Errorf("no acl resource with name %s", normalized)
+	}
+	*a = art
+	return nil
+}
+
 // ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/resource/PatternType.java
 const (
 	AclPatternUnknown AclResourcePatternType = iota
@@ -53,3 +196,43 @@
 	AclPatternLiteral
 	AclPatternPrefixed
 )
+
+func (a *AclResourcePatternType) String() string {
+	mapping := map[AclResourcePatternType]string{
+		AclPatternUnknown:  "Unknown",
+		AclPatternAny:      "Any",
+		AclPatternMatch:    "Match",
+		AclPatternLiteral:  "Literal",
+		AclPatternPrefixed: "Prefixed",
+	}
+	s, ok := mapping[*a]
+	if !ok {
+		s = mapping[AclPatternUnknown]
+	}
+	return s
+}
+
+// MarshalText returns the text form of the AclResourcePatternType (name without prefix)
+func (a *AclResourcePatternType) MarshalText() ([]byte, error) {
+	return []byte(a.String()), nil
+}
+
+// UnmarshalText takes a text reprentation of the resource pattern type and converts it to an AclResourcePatternType
+func (a *AclResourcePatternType) UnmarshalText(text []byte) error {
+	normalized := strings.ToLower(string(text))
+	mapping := map[string]AclResourcePatternType{
+		"unknown":  AclPatternUnknown,
+		"any":      AclPatternAny,
+		"match":    AclPatternMatch,
+		"literal":  AclPatternLiteral,
+		"prefixed": AclPatternPrefixed,
+	}
+
+	arpt, ok := mapping[normalized]
+	if !ok {
+		*a = AclPatternUnknown
+		return fmt.Errorf("no acl resource pattern with name %s", normalized)
+	}
+	*a = arpt
+	return nil
+}
diff --git a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go
index fc227ab..a96af93 100644
--- a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go
+++ b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//AddOffsetsToTxnRequest adds offsets to a transaction request
+// AddOffsetsToTxnRequest adds offsets to a transaction request
 type AddOffsetsToTxnRequest struct {
 	TransactionalID string
 	ProducerID      int64
@@ -48,6 +48,10 @@
 	return 0
 }
 
+func (a *AddOffsetsToTxnRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *AddOffsetsToTxnRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go
index c88c1f8..bb61973 100644
--- a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go
+++ b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go
@@ -4,7 +4,7 @@
 	"time"
 )
 
-//AddOffsetsToTxnResponse is a response type for adding offsets to txns
+// AddOffsetsToTxnResponse is a response type for adding offsets to txns
 type AddOffsetsToTxnResponse struct {
 	ThrottleTime time.Duration
 	Err          KError
@@ -40,6 +40,10 @@
 	return 0
 }
 
+func (a *AddOffsetsToTxnResponse) headerVersion() int16 {
+	return 0
+}
+
 func (a *AddOffsetsToTxnResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go
index 8d4b42e..57ecf64 100644
--- a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go
+++ b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//AddPartitionsToTxnRequest is a add paartition request
+// AddPartitionsToTxnRequest is a add paartition request
 type AddPartitionsToTxnRequest struct {
 	TransactionalID string
 	ProducerID      int64
@@ -72,6 +72,10 @@
 	return 0
 }
 
+func (a *AddPartitionsToTxnRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *AddPartitionsToTxnRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go
index eb4f23e..0989565 100644
--- a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go
+++ b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go
@@ -4,7 +4,7 @@
 	"time"
 )
 
-//AddPartitionsToTxnResponse is a partition errors to transaction type
+// AddPartitionsToTxnResponse is a partition errors to transaction type
 type AddPartitionsToTxnResponse struct {
 	ThrottleTime time.Duration
 	Errors       map[string][]*PartitionError
@@ -79,11 +79,15 @@
 	return 0
 }
 
+func (a *AddPartitionsToTxnResponse) headerVersion() int16 {
+	return 0
+}
+
 func (a *AddPartitionsToTxnResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
 
-//PartitionError is a partition error type
+// PartitionError is a partition error type
 type PartitionError struct {
 	Partition int32
 	Err       KError
diff --git a/vendor/github.com/Shopify/sarama/admin.go b/vendor/github.com/Shopify/sarama/admin.go
index 1db6a0e..abe18b1 100644
--- a/vendor/github.com/Shopify/sarama/admin.go
+++ b/vendor/github.com/Shopify/sarama/admin.go
@@ -2,8 +2,11 @@
 
 import (
 	"errors"
+	"fmt"
 	"math/rand"
+	"strconv"
 	"sync"
+	"time"
 )
 
 // ClusterAdmin is the administrative client for Kafka, which supports managing and inspecting topics,
@@ -39,6 +42,14 @@
 	// new partitions. This operation is supported by brokers with version 1.0.0 or higher.
 	CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error
 
+	// Alter the replica assignment for partitions.
+	// This operation is supported by brokers with version 2.4.0.0 or higher.
+	AlterPartitionReassignments(topic string, assignment [][]int32) error
+
+	// Provides info on ongoing partitions replica reassignments.
+	// This operation is supported by brokers with version 2.4.0.0 or higher.
+	ListPartitionReassignments(topics string, partitions []int32) (topicStatus map[string]map[int32]*PartitionReplicaReassignmentsStatus, err error)
+
 	// Delete records whose offset is smaller than the given offset of the corresponding partition.
 	// This operation is supported by brokers with version 0.11.0.0 or higher.
 	DeleteRecords(topic string, partitionOffsets map[int32]int64) error
@@ -90,6 +101,18 @@
 	// Get information about the nodes in the cluster
 	DescribeCluster() (brokers []*Broker, controllerID int32, err error)
 
+	// Get information about all log directories on the given set of brokers
+	DescribeLogDirs(brokers []int32) (map[int32][]DescribeLogDirsResponseDirMetadata, error)
+
+	// Get information about SCRAM users
+	DescribeUserScramCredentials(users []string) ([]*DescribeUserScramCredentialsResult, error)
+
+	// Delete SCRAM users
+	DeleteUserScramCredentials(delete []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error)
+
+	// Upsert SCRAM users
+	UpsertUserScramCredentials(upsert []AlterUserScramCredentialsUpsert) ([]*AlterUserScramCredentialsResult, error)
+
 	// Close shuts down the admin and closes underlying client.
 	Close() error
 }
@@ -105,9 +128,14 @@
 	if err != nil {
 		return nil, err
 	}
+	return NewClusterAdminFromClient(client)
+}
 
-	//make sure we can retrieve the controller
-	_, err = client.Controller()
+// NewClusterAdminFromClient creates a new ClusterAdmin using the given client.
+// Note that underlying client will also be closed on admin's Close() call.
+func NewClusterAdminFromClient(client Client) (ClusterAdmin, error) {
+	// make sure we can retrieve the controller
+	_, err := client.Controller()
 	if err != nil {
 		return nil, err
 	}
@@ -127,8 +155,45 @@
 	return ca.client.Controller()
 }
 
-func (ca *clusterAdmin) CreateTopic(topic string, detail *TopicDetail, validateOnly bool) error {
+func (ca *clusterAdmin) refreshController() (*Broker, error) {
+	return ca.client.RefreshController()
+}
 
+// isErrNoController returns `true` if the given error type unwraps to an
+// `ErrNotController` response from Kafka
+func isErrNoController(err error) bool {
+	switch e := err.(type) {
+	case *TopicError:
+		return e.Err == ErrNotController
+	case *TopicPartitionError:
+		return e.Err == ErrNotController
+	case KError:
+		return e == ErrNotController
+	}
+	return false
+}
+
+// retryOnError will repeatedly call the given (error-returning) func in the
+// case that its response is non-nil and retryable (as determined by the
+// provided retryable func) up to the maximum number of tries permitted by
+// the admin client configuration
+func (ca *clusterAdmin) retryOnError(retryable func(error) bool, fn func() error) error {
+	var err error
+	for attempt := 0; attempt < ca.conf.Admin.Retry.Max; attempt++ {
+		err = fn()
+		if err == nil || !retryable(err) {
+			return err
+		}
+		Logger.Printf(
+			"admin/request retrying after %dms... (%d attempts remaining)\n",
+			ca.conf.Admin.Retry.Backoff/time.Millisecond, ca.conf.Admin.Retry.Max-attempt)
+		time.Sleep(ca.conf.Admin.Retry.Backoff)
+		continue
+	}
+	return err
+}
+
+func (ca *clusterAdmin) CreateTopic(topic string, detail *TopicDetail, validateOnly bool) error {
 	if topic == "" {
 		return ErrInvalidTopic
 	}
@@ -153,26 +218,31 @@
 		request.Version = 2
 	}
 
-	b, err := ca.Controller()
-	if err != nil {
-		return err
-	}
+	return ca.retryOnError(isErrNoController, func() error {
+		b, err := ca.Controller()
+		if err != nil {
+			return err
+		}
 
-	rsp, err := b.CreateTopics(request)
-	if err != nil {
-		return err
-	}
+		rsp, err := b.CreateTopics(request)
+		if err != nil {
+			return err
+		}
 
-	topicErr, ok := rsp.TopicErrors[topic]
-	if !ok {
-		return ErrIncompleteResponse
-	}
+		topicErr, ok := rsp.TopicErrors[topic]
+		if !ok {
+			return ErrIncompleteResponse
+		}
 
-	if topicErr.Err != ErrNoError {
-		return topicErr
-	}
+		if topicErr.Err != ErrNoError {
+			if topicErr.Err == ErrNotController {
+				_, _ = ca.refreshController()
+			}
+			return topicErr
+		}
 
-	return nil
+		return nil
+	})
 }
 
 func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetadata, err error) {
@@ -209,6 +279,10 @@
 		Topics: []string{},
 	}
 
+	if ca.conf.Version.IsAtLeast(V0_10_0_0) {
+		request.Version = 1
+	}
+
 	response, err := controller.GetMetadata(request)
 	if err != nil {
 		return nil, int32(0), err
@@ -217,6 +291,16 @@
 	return response.Brokers, response.ControllerID, nil
 }
 
+func (ca *clusterAdmin) findBroker(id int32) (*Broker, error) {
+	brokers := ca.client.Brokers()
+	for _, b := range brokers {
+		if b.ID() == id {
+			return b, nil
+		}
+	}
+	return nil, fmt.Errorf("could not find broker id %d", id)
+}
+
 func (ca *clusterAdmin) findAnyBroker() (*Broker, error) {
 	brokers := ca.client.Brokers()
 	if len(brokers) > 0 {
@@ -274,6 +358,15 @@
 	describeConfigsReq := &DescribeConfigsRequest{
 		Resources: describeConfigsResources,
 	}
+
+	if ca.conf.Version.IsAtLeast(V1_1_0_0) {
+		describeConfigsReq.Version = 1
+	}
+
+	if ca.conf.Version.IsAtLeast(V2_0_0_0) {
+		describeConfigsReq.Version = 2
+	}
+
 	describeConfigsResp, err := b.DescribeConfigs(describeConfigsReq)
 	if err != nil {
 		return nil, err
@@ -299,7 +392,6 @@
 }
 
 func (ca *clusterAdmin) DeleteTopic(topic string) error {
-
 	if topic == "" {
 		return ErrInvalidTopic
 	}
@@ -313,25 +405,31 @@
 		request.Version = 1
 	}
 
-	b, err := ca.Controller()
-	if err != nil {
-		return err
-	}
+	return ca.retryOnError(isErrNoController, func() error {
+		b, err := ca.Controller()
+		if err != nil {
+			return err
+		}
 
-	rsp, err := b.DeleteTopics(request)
-	if err != nil {
-		return err
-	}
+		rsp, err := b.DeleteTopics(request)
+		if err != nil {
+			return err
+		}
 
-	topicErr, ok := rsp.TopicErrorCodes[topic]
-	if !ok {
-		return ErrIncompleteResponse
-	}
+		topicErr, ok := rsp.TopicErrorCodes[topic]
+		if !ok {
+			return ErrIncompleteResponse
+		}
 
-	if topicErr != ErrNoError {
-		return topicErr
-	}
-	return nil
+		if topicErr != ErrNoError {
+			if topicErr == ErrNotController {
+				_, _ = ca.refreshController()
+			}
+			return topicErr
+		}
+
+		return nil
+	})
 }
 
 func (ca *clusterAdmin) CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error {
@@ -347,30 +445,110 @@
 		Timeout:         ca.conf.Admin.Timeout,
 	}
 
+	return ca.retryOnError(isErrNoController, func() error {
+		b, err := ca.Controller()
+		if err != nil {
+			return err
+		}
+
+		rsp, err := b.CreatePartitions(request)
+		if err != nil {
+			return err
+		}
+
+		topicErr, ok := rsp.TopicPartitionErrors[topic]
+		if !ok {
+			return ErrIncompleteResponse
+		}
+
+		if topicErr.Err != ErrNoError {
+			if topicErr.Err == ErrNotController {
+				_, _ = ca.refreshController()
+			}
+			return topicErr
+		}
+
+		return nil
+	})
+}
+
+func (ca *clusterAdmin) AlterPartitionReassignments(topic string, assignment [][]int32) error {
+	if topic == "" {
+		return ErrInvalidTopic
+	}
+
+	request := &AlterPartitionReassignmentsRequest{
+		TimeoutMs: int32(60000),
+		Version:   int16(0),
+	}
+
+	for i := 0; i < len(assignment); i++ {
+		request.AddBlock(topic, int32(i), assignment[i])
+	}
+
+	return ca.retryOnError(isErrNoController, func() error {
+		b, err := ca.Controller()
+		if err != nil {
+			return err
+		}
+
+		errs := make([]error, 0)
+
+		rsp, err := b.AlterPartitionReassignments(request)
+
+		if err != nil {
+			errs = append(errs, err)
+		} else {
+			if rsp.ErrorCode > 0 {
+				errs = append(errs, errors.New(rsp.ErrorCode.Error()))
+			}
+
+			for topic, topicErrors := range rsp.Errors {
+				for partition, partitionError := range topicErrors {
+					if partitionError.errorCode != ErrNoError {
+						errStr := fmt.Sprintf("[%s-%d]: %s", topic, partition, partitionError.errorCode.Error())
+						errs = append(errs, errors.New(errStr))
+					}
+				}
+			}
+		}
+
+		if len(errs) > 0 {
+			return ErrReassignPartitions{MultiError{&errs}}
+		}
+
+		return nil
+	})
+}
+
+func (ca *clusterAdmin) ListPartitionReassignments(topic string, partitions []int32) (topicStatus map[string]map[int32]*PartitionReplicaReassignmentsStatus, err error) {
+	if topic == "" {
+		return nil, ErrInvalidTopic
+	}
+
+	request := &ListPartitionReassignmentsRequest{
+		TimeoutMs: int32(60000),
+		Version:   int16(0),
+	}
+
+	request.AddBlock(topic, partitions)
+
 	b, err := ca.Controller()
 	if err != nil {
-		return err
+		return nil, err
 	}
+	_ = b.Open(ca.client.Config())
 
-	rsp, err := b.CreatePartitions(request)
-	if err != nil {
-		return err
+	rsp, err := b.ListPartitionReassignments(request)
+
+	if err == nil && rsp != nil {
+		return rsp.TopicStatus, nil
+	} else {
+		return nil, err
 	}
-
-	topicErr, ok := rsp.TopicPartitionErrors[topic]
-	if !ok {
-		return ErrIncompleteResponse
-	}
-
-	if topicErr.Err != ErrNoError {
-		return topicErr
-	}
-
-	return nil
 }
 
 func (ca *clusterAdmin) DeleteRecords(topic string, partitionOffsets map[int32]int64) error {
-
 	if topic == "" {
 		return ErrInvalidTopic
 	}
@@ -380,11 +558,7 @@
 		if err != nil {
 			return err
 		}
-		if _, ok := partitionPerBroker[broker]; ok {
-			partitionPerBroker[broker] = append(partitionPerBroker[broker], partition)
-		} else {
-			partitionPerBroker[broker] = []int32{partition}
-		}
+		partitionPerBroker[broker] = append(partitionPerBroker[broker], partition)
 	}
 	errs := make([]error, 0)
 	for broker, partitions := range partitionPerBroker {
@@ -418,13 +592,19 @@
 	if len(errs) > 0 {
 		return ErrDeleteRecords{MultiError{&errs}}
 	}
-	//todo since we are dealing with couple of partitions it would be good if we return slice of errors
-	//for each partition instead of one error
+	// todo since we are dealing with couple of partitions it would be good if we return slice of errors
+	// for each partition instead of one error
 	return nil
 }
 
-func (ca *clusterAdmin) DescribeConfig(resource ConfigResource) ([]ConfigEntry, error) {
+// Returns a bool indicating whether the resource request needs to go to a
+// specific broker
+func dependsOnSpecificNode(resource ConfigResource) bool {
+	return (resource.Type == BrokerResource && resource.Name != "") ||
+		resource.Type == BrokerLoggerResource
+}
 
+func (ca *clusterAdmin) DescribeConfig(resource ConfigResource) ([]ConfigEntry, error) {
 	var entries []ConfigEntry
 	var resources []*ConfigResource
 	resources = append(resources, &resource)
@@ -433,11 +613,35 @@
 		Resources: resources,
 	}
 
-	b, err := ca.Controller()
+	if ca.conf.Version.IsAtLeast(V1_1_0_0) {
+		request.Version = 1
+	}
+
+	if ca.conf.Version.IsAtLeast(V2_0_0_0) {
+		request.Version = 2
+	}
+
+	var (
+		b   *Broker
+		err error
+	)
+
+	// DescribeConfig of broker/broker logger must be sent to the broker in question
+	if dependsOnSpecificNode(resource) {
+		var id int64
+		id, err = strconv.ParseInt(resource.Name, 10, 32)
+		if err != nil {
+			return nil, err
+		}
+		b, err = ca.findBroker(int32(id))
+	} else {
+		b, err = ca.findAnyBroker()
+	}
 	if err != nil {
 		return nil, err
 	}
 
+	_ = b.Open(ca.client.Config())
 	rsp, err := b.DescribeConfigs(request)
 	if err != nil {
 		return nil, err
@@ -448,6 +652,9 @@
 			if rspResource.ErrorMsg != "" {
 				return nil, errors.New(rspResource.ErrorMsg)
 			}
+			if rspResource.ErrorCode != 0 {
+				return nil, KError(rspResource.ErrorCode)
+			}
 			for _, cfgEntry := range rspResource.Configs {
 				entries = append(entries, *cfgEntry)
 			}
@@ -457,7 +664,6 @@
 }
 
 func (ca *clusterAdmin) AlterConfig(resourceType ConfigResourceType, name string, entries map[string]*string, validateOnly bool) error {
-
 	var resources []*AlterConfigsResource
 	resources = append(resources, &AlterConfigsResource{
 		Type:          resourceType,
@@ -470,11 +676,27 @@
 		ValidateOnly: validateOnly,
 	}
 
-	b, err := ca.Controller()
+	var (
+		b   *Broker
+		err error
+	)
+
+	// AlterConfig of broker/broker logger must be sent to the broker in question
+	if dependsOnSpecificNode(ConfigResource{Name: name, Type: resourceType}) {
+		var id int64
+		id, err = strconv.ParseInt(name, 10, 32)
+		if err != nil {
+			return err
+		}
+		b, err = ca.findBroker(int32(id))
+	} else {
+		b, err = ca.findAnyBroker()
+	}
 	if err != nil {
 		return err
 	}
 
+	_ = b.Open(ca.client.Config())
 	rsp, err := b.AlterConfigs(request)
 	if err != nil {
 		return err
@@ -485,6 +707,9 @@
 			if rspResource.ErrorMsg != "" {
 				return errors.New(rspResource.ErrorMsg)
 			}
+			if rspResource.ErrorCode != 0 {
+				return KError(rspResource.ErrorCode)
+			}
 		}
 	}
 	return nil
@@ -495,6 +720,10 @@
 	acls = append(acls, &AclCreation{resource, acl})
 	request := &CreateAclsRequest{AclCreations: acls}
 
+	if ca.conf.Version.IsAtLeast(V2_0_0_0) {
+		request.Version = 1
+	}
+
 	b, err := ca.Controller()
 	if err != nil {
 		return err
@@ -505,9 +734,12 @@
 }
 
 func (ca *clusterAdmin) ListAcls(filter AclFilter) ([]ResourceAcls, error) {
-
 	request := &DescribeAclsRequest{AclFilter: filter}
 
+	if ca.conf.Version.IsAtLeast(V2_0_0_0) {
+		request.Version = 1
+	}
+
 	b, err := ca.Controller()
 	if err != nil {
 		return nil, err
@@ -530,6 +762,10 @@
 	filters = append(filters, &filter)
 	request := &DeleteAclsRequest{Filters: filters}
 
+	if ca.conf.Version.IsAtLeast(V2_0_0_0) {
+		request.Version = 1
+	}
+
 	b, err := ca.Controller()
 	if err != nil {
 		return nil, err
@@ -545,7 +781,6 @@
 		for _, mACL := range fr.MatchingAcls {
 			mAcls = append(mAcls, *mACL)
 		}
-
 	}
 	return mAcls, nil
 }
@@ -559,7 +794,6 @@
 			return nil, err
 		}
 		groupsPerBroker[controller] = append(groupsPerBroker[controller], group)
-
 	}
 
 	for broker, brokerGroups := range groupsPerBroker {
@@ -581,7 +815,7 @@
 	// Query brokers in parallel, since we have to query *all* brokers
 	brokers := ca.client.Brokers()
 	groupMaps := make(chan map[string]string, len(brokers))
-	errors := make(chan error, len(brokers))
+	errChan := make(chan error, len(brokers))
 	wg := sync.WaitGroup{}
 
 	for _, b := range brokers {
@@ -592,7 +826,7 @@
 
 			response, err := b.ListGroups(&ListGroupsRequest{})
 			if err != nil {
-				errors <- err
+				errChan <- err
 				return
 			}
 
@@ -602,13 +836,12 @@
 			}
 
 			groupMaps <- groups
-
 		}(b, ca.conf)
 	}
 
 	wg.Wait()
 	close(groupMaps)
-	close(errors)
+	close(errChan)
 
 	for groupMap := range groupMaps {
 		for group, protocolType := range groupMap {
@@ -617,7 +850,7 @@
 	}
 
 	// Intentionally return only the first error for simplicity
-	err = <-errors
+	err = <-errChan
 	return
 }
 
@@ -667,3 +900,106 @@
 
 	return nil
 }
+
+func (ca *clusterAdmin) DescribeLogDirs(brokerIds []int32) (allLogDirs map[int32][]DescribeLogDirsResponseDirMetadata, err error) {
+	allLogDirs = make(map[int32][]DescribeLogDirsResponseDirMetadata)
+
+	// Query brokers in parallel, since we may have to query multiple brokers
+	logDirsMaps := make(chan map[int32][]DescribeLogDirsResponseDirMetadata, len(brokerIds))
+	errChan := make(chan error, len(brokerIds))
+	wg := sync.WaitGroup{}
+
+	for _, b := range brokerIds {
+		wg.Add(1)
+		broker, err := ca.findBroker(b)
+		if err != nil {
+			Logger.Printf("Unable to find broker with ID = %v\n", b)
+			continue
+		}
+		go func(b *Broker, conf *Config) {
+			defer wg.Done()
+			_ = b.Open(conf) // Ensure that broker is opened
+
+			response, err := b.DescribeLogDirs(&DescribeLogDirsRequest{})
+			if err != nil {
+				errChan <- err
+				return
+			}
+			logDirs := make(map[int32][]DescribeLogDirsResponseDirMetadata)
+			logDirs[b.ID()] = response.LogDirs
+			logDirsMaps <- logDirs
+		}(broker, ca.conf)
+	}
+
+	wg.Wait()
+	close(logDirsMaps)
+	close(errChan)
+
+	for logDirsMap := range logDirsMaps {
+		for id, logDirs := range logDirsMap {
+			allLogDirs[id] = logDirs
+		}
+	}
+
+	// Intentionally return only the first error for simplicity
+	err = <-errChan
+	return
+}
+
+func (ca *clusterAdmin) DescribeUserScramCredentials(users []string) ([]*DescribeUserScramCredentialsResult, error) {
+	req := &DescribeUserScramCredentialsRequest{}
+	for _, u := range users {
+		req.DescribeUsers = append(req.DescribeUsers, DescribeUserScramCredentialsRequestUser{
+			Name: u,
+		})
+	}
+
+	b, err := ca.Controller()
+	if err != nil {
+		return nil, err
+	}
+
+	rsp, err := b.DescribeUserScramCredentials(req)
+	if err != nil {
+		return nil, err
+	}
+
+	return rsp.Results, nil
+}
+
+func (ca *clusterAdmin) UpsertUserScramCredentials(upsert []AlterUserScramCredentialsUpsert) ([]*AlterUserScramCredentialsResult, error) {
+	res, err := ca.AlterUserScramCredentials(upsert, nil)
+	if err != nil {
+		return nil, err
+	}
+
+	return res, nil
+}
+
+func (ca *clusterAdmin) DeleteUserScramCredentials(delete []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error) {
+	res, err := ca.AlterUserScramCredentials(nil, delete)
+	if err != nil {
+		return nil, err
+	}
+
+	return res, nil
+}
+
+func (ca *clusterAdmin) AlterUserScramCredentials(u []AlterUserScramCredentialsUpsert, d []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error) {
+	req := &AlterUserScramCredentialsRequest{
+		Deletions:  d,
+		Upsertions: u,
+	}
+
+	b, err := ca.Controller()
+	if err != nil {
+		return nil, err
+	}
+
+	rsp, err := b.AlterUserScramCredentials(req)
+	if err != nil {
+		return nil, err
+	}
+
+	return rsp.Results, nil
+}
diff --git a/vendor/github.com/Shopify/sarama/alter_configs_request.go b/vendor/github.com/Shopify/sarama/alter_configs_request.go
index 26c275b..8b94b1f 100644
--- a/vendor/github.com/Shopify/sarama/alter_configs_request.go
+++ b/vendor/github.com/Shopify/sarama/alter_configs_request.go
@@ -1,12 +1,12 @@
 package sarama
 
-//AlterConfigsRequest is an alter config request type
+// AlterConfigsRequest is an alter config request type
 type AlterConfigsRequest struct {
 	Resources    []*AlterConfigsResource
 	ValidateOnly bool
 }
 
-//AlterConfigsResource is an alter config resource type
+// AlterConfigsResource is an alter config resource type
 type AlterConfigsResource struct {
 	Type          ConfigResourceType
 	Name          string
@@ -117,6 +117,10 @@
 	return 0
 }
 
+func (a *AlterConfigsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *AlterConfigsRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/alter_configs_response.go b/vendor/github.com/Shopify/sarama/alter_configs_response.go
index 3893663..cfb6369 100644
--- a/vendor/github.com/Shopify/sarama/alter_configs_response.go
+++ b/vendor/github.com/Shopify/sarama/alter_configs_response.go
@@ -2,13 +2,13 @@
 
 import "time"
 
-//AlterConfigsResponse is a reponse type for alter config
+// AlterConfigsResponse is a response type for alter config
 type AlterConfigsResponse struct {
 	ThrottleTime time.Duration
 	Resources    []*AlterConfigsResourceResponse
 }
 
-//AlterConfigsResourceResponse is a reponse type for alter config resource
+// AlterConfigsResourceResponse is a response type for alter config resource
 type AlterConfigsResourceResponse struct {
 	ErrorCode int16
 	ErrorMsg  string
@@ -23,16 +23,9 @@
 		return err
 	}
 
-	for i := range a.Resources {
-		pe.putInt16(a.Resources[i].ErrorCode)
-		err := pe.putString(a.Resources[i].ErrorMsg)
-		if err != nil {
-			return nil
-		}
-		pe.putInt8(int8(a.Resources[i].Type))
-		err = pe.putString(a.Resources[i].Name)
-		if err != nil {
-			return nil
+	for _, v := range a.Resources {
+		if err := v.encode(pe); err != nil {
+			return err
 		}
 	}
 
@@ -56,34 +49,56 @@
 	for i := range a.Resources {
 		a.Resources[i] = new(AlterConfigsResourceResponse)
 
-		errCode, err := pd.getInt16()
-		if err != nil {
+		if err := a.Resources[i].decode(pd, version); err != nil {
 			return err
 		}
-		a.Resources[i].ErrorCode = errCode
-
-		e, err := pd.getString()
-		if err != nil {
-			return err
-		}
-		a.Resources[i].ErrorMsg = e
-
-		t, err := pd.getInt8()
-		if err != nil {
-			return err
-		}
-		a.Resources[i].Type = ConfigResourceType(t)
-
-		name, err := pd.getString()
-		if err != nil {
-			return err
-		}
-		a.Resources[i].Name = name
 	}
 
 	return nil
 }
 
+func (a *AlterConfigsResourceResponse) encode(pe packetEncoder) error {
+	pe.putInt16(a.ErrorCode)
+	err := pe.putString(a.ErrorMsg)
+	if err != nil {
+		return nil
+	}
+	pe.putInt8(int8(a.Type))
+	err = pe.putString(a.Name)
+	if err != nil {
+		return nil
+	}
+	return nil
+}
+
+func (a *AlterConfigsResourceResponse) decode(pd packetDecoder, version int16) error {
+	errCode, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+	a.ErrorCode = errCode
+
+	e, err := pd.getString()
+	if err != nil {
+		return err
+	}
+	a.ErrorMsg = e
+
+	t, err := pd.getInt8()
+	if err != nil {
+		return err
+	}
+	a.Type = ConfigResourceType(t)
+
+	name, err := pd.getString()
+	if err != nil {
+		return err
+	}
+	a.Name = name
+
+	return nil
+}
+
 func (a *AlterConfigsResponse) key() int16 {
 	return 32
 }
@@ -92,6 +107,10 @@
 	return 0
 }
 
+func (a *AlterConfigsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (a *AlterConfigsResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go
new file mode 100644
index 0000000..f0a2f9d
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go
@@ -0,0 +1,130 @@
+package sarama
+
+type alterPartitionReassignmentsBlock struct {
+	replicas []int32
+}
+
+func (b *alterPartitionReassignmentsBlock) encode(pe packetEncoder) error {
+	if err := pe.putNullableCompactInt32Array(b.replicas); err != nil {
+		return err
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (b *alterPartitionReassignmentsBlock) decode(pd packetDecoder) (err error) {
+	if b.replicas, err = pd.getCompactInt32Array(); err != nil {
+		return err
+	}
+	return nil
+}
+
+type AlterPartitionReassignmentsRequest struct {
+	TimeoutMs int32
+	blocks    map[string]map[int32]*alterPartitionReassignmentsBlock
+	Version   int16
+}
+
+func (r *AlterPartitionReassignmentsRequest) encode(pe packetEncoder) error {
+	pe.putInt32(r.TimeoutMs)
+
+	pe.putCompactArrayLength(len(r.blocks))
+
+	for topic, partitions := range r.blocks {
+		if err := pe.putCompactString(topic); err != nil {
+			return err
+		}
+		pe.putCompactArrayLength(len(partitions))
+		for partition, block := range partitions {
+			pe.putInt32(partition)
+			if err := block.encode(pe); err != nil {
+				return err
+			}
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+
+	return nil
+}
+
+func (r *AlterPartitionReassignmentsRequest) decode(pd packetDecoder, version int16) (err error) {
+	r.Version = version
+
+	if r.TimeoutMs, err = pd.getInt32(); err != nil {
+		return err
+	}
+
+	topicCount, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+	if topicCount > 0 {
+		r.blocks = make(map[string]map[int32]*alterPartitionReassignmentsBlock)
+		for i := 0; i < topicCount; i++ {
+			topic, err := pd.getCompactString()
+			if err != nil {
+				return err
+			}
+			partitionCount, err := pd.getCompactArrayLength()
+			if err != nil {
+				return err
+			}
+			r.blocks[topic] = make(map[int32]*alterPartitionReassignmentsBlock)
+			for j := 0; j < partitionCount; j++ {
+				partition, err := pd.getInt32()
+				if err != nil {
+					return err
+				}
+				block := &alterPartitionReassignmentsBlock{}
+				if err := block.decode(pd); err != nil {
+					return err
+				}
+				r.blocks[topic][partition] = block
+
+				if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+					return err
+				}
+			}
+			if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+
+	return
+}
+
+func (r *AlterPartitionReassignmentsRequest) key() int16 {
+	return 45
+}
+
+func (r *AlterPartitionReassignmentsRequest) version() int16 {
+	return r.Version
+}
+
+func (r *AlterPartitionReassignmentsRequest) headerVersion() int16 {
+	return 2
+}
+
+func (r *AlterPartitionReassignmentsRequest) requiredVersion() KafkaVersion {
+	return V2_4_0_0
+}
+
+func (r *AlterPartitionReassignmentsRequest) AddBlock(topic string, partitionID int32, replicas []int32) {
+	if r.blocks == nil {
+		r.blocks = make(map[string]map[int32]*alterPartitionReassignmentsBlock)
+	}
+
+	if r.blocks[topic] == nil {
+		r.blocks[topic] = make(map[int32]*alterPartitionReassignmentsBlock)
+	}
+
+	r.blocks[topic][partitionID] = &alterPartitionReassignmentsBlock{replicas}
+}
diff --git a/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go
new file mode 100644
index 0000000..b3f9a15
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go
@@ -0,0 +1,157 @@
+package sarama
+
+type alterPartitionReassignmentsErrorBlock struct {
+	errorCode    KError
+	errorMessage *string
+}
+
+func (b *alterPartitionReassignmentsErrorBlock) encode(pe packetEncoder) error {
+	pe.putInt16(int16(b.errorCode))
+	if err := pe.putNullableCompactString(b.errorMessage); err != nil {
+		return err
+	}
+	pe.putEmptyTaggedFieldArray()
+
+	return nil
+}
+
+func (b *alterPartitionReassignmentsErrorBlock) decode(pd packetDecoder) (err error) {
+	errorCode, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+	b.errorCode = KError(errorCode)
+	b.errorMessage, err = pd.getCompactNullableString()
+
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+	return err
+}
+
+type AlterPartitionReassignmentsResponse struct {
+	Version        int16
+	ThrottleTimeMs int32
+	ErrorCode      KError
+	ErrorMessage   *string
+	Errors         map[string]map[int32]*alterPartitionReassignmentsErrorBlock
+}
+
+func (r *AlterPartitionReassignmentsResponse) AddError(topic string, partition int32, kerror KError, message *string) {
+	if r.Errors == nil {
+		r.Errors = make(map[string]map[int32]*alterPartitionReassignmentsErrorBlock)
+	}
+	partitions := r.Errors[topic]
+	if partitions == nil {
+		partitions = make(map[int32]*alterPartitionReassignmentsErrorBlock)
+		r.Errors[topic] = partitions
+	}
+
+	partitions[partition] = &alterPartitionReassignmentsErrorBlock{errorCode: kerror, errorMessage: message}
+}
+
+func (r *AlterPartitionReassignmentsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(r.ThrottleTimeMs)
+	pe.putInt16(int16(r.ErrorCode))
+	if err := pe.putNullableCompactString(r.ErrorMessage); err != nil {
+		return err
+	}
+
+	pe.putCompactArrayLength(len(r.Errors))
+	for topic, partitions := range r.Errors {
+		if err := pe.putCompactString(topic); err != nil {
+			return err
+		}
+		pe.putCompactArrayLength(len(partitions))
+		for partition, block := range partitions {
+			pe.putInt32(partition)
+
+			if err := block.encode(pe); err != nil {
+				return err
+			}
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (r *AlterPartitionReassignmentsResponse) decode(pd packetDecoder, version int16) (err error) {
+	r.Version = version
+
+	if r.ThrottleTimeMs, err = pd.getInt32(); err != nil {
+		return err
+	}
+
+	kerr, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+
+	r.ErrorCode = KError(kerr)
+
+	if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil {
+		return err
+	}
+
+	numTopics, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	if numTopics > 0 {
+		r.Errors = make(map[string]map[int32]*alterPartitionReassignmentsErrorBlock, numTopics)
+		for i := 0; i < numTopics; i++ {
+			topic, err := pd.getCompactString()
+			if err != nil {
+				return err
+			}
+
+			ongoingPartitionReassignments, err := pd.getCompactArrayLength()
+			if err != nil {
+				return err
+			}
+
+			r.Errors[topic] = make(map[int32]*alterPartitionReassignmentsErrorBlock, ongoingPartitionReassignments)
+
+			for j := 0; j < ongoingPartitionReassignments; j++ {
+				partition, err := pd.getInt32()
+				if err != nil {
+					return err
+				}
+				block := &alterPartitionReassignmentsErrorBlock{}
+				if err := block.decode(pd); err != nil {
+					return err
+				}
+
+				r.Errors[topic][partition] = block
+			}
+			if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+func (r *AlterPartitionReassignmentsResponse) key() int16 {
+	return 45
+}
+
+func (r *AlterPartitionReassignmentsResponse) version() int16 {
+	return r.Version
+}
+
+func (r *AlterPartitionReassignmentsResponse) headerVersion() int16 {
+	return 1
+}
+
+func (r *AlterPartitionReassignmentsResponse) requiredVersion() KafkaVersion {
+	return V2_4_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go
new file mode 100644
index 0000000..0530d89
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go
@@ -0,0 +1,142 @@
+package sarama
+
+type AlterUserScramCredentialsRequest struct {
+	Version int16
+
+	// Deletions represent list of SCRAM credentials to remove
+	Deletions []AlterUserScramCredentialsDelete
+
+	// Upsertions represent list of SCRAM credentials to update/insert
+	Upsertions []AlterUserScramCredentialsUpsert
+}
+
+type AlterUserScramCredentialsDelete struct {
+	Name      string
+	Mechanism ScramMechanismType
+}
+
+type AlterUserScramCredentialsUpsert struct {
+	Name           string
+	Mechanism      ScramMechanismType
+	Iterations     int32
+	Salt           []byte
+	saltedPassword []byte
+
+	// This field is never transmitted over the wire
+	// @see: https://tools.ietf.org/html/rfc5802
+	Password []byte
+}
+
+func (r *AlterUserScramCredentialsRequest) encode(pe packetEncoder) error {
+	pe.putCompactArrayLength(len(r.Deletions))
+	for _, d := range r.Deletions {
+		if err := pe.putCompactString(d.Name); err != nil {
+			return err
+		}
+		pe.putInt8(int8(d.Mechanism))
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putCompactArrayLength(len(r.Upsertions))
+	for _, u := range r.Upsertions {
+		if err := pe.putCompactString(u.Name); err != nil {
+			return err
+		}
+		pe.putInt8(int8(u.Mechanism))
+		pe.putInt32(u.Iterations)
+
+		if err := pe.putCompactBytes(u.Salt); err != nil {
+			return err
+		}
+
+		// do not transmit the password over the wire
+		formatter := scramFormatter{mechanism: u.Mechanism}
+		salted, err := formatter.saltedPassword(u.Password, u.Salt, int(u.Iterations))
+		if err != nil {
+			return err
+		}
+
+		if err := pe.putCompactBytes(salted); err != nil {
+			return err
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (r *AlterUserScramCredentialsRequest) decode(pd packetDecoder, version int16) error {
+	numDeletions, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	r.Deletions = make([]AlterUserScramCredentialsDelete, numDeletions)
+	for i := 0; i < numDeletions; i++ {
+		r.Deletions[i] = AlterUserScramCredentialsDelete{}
+		if r.Deletions[i].Name, err = pd.getCompactString(); err != nil {
+			return err
+		}
+		mechanism, err := pd.getInt8()
+		if err != nil {
+			return err
+		}
+		r.Deletions[i].Mechanism = ScramMechanismType(mechanism)
+		if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
+	numUpsertions, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	r.Upsertions = make([]AlterUserScramCredentialsUpsert, numUpsertions)
+	for i := 0; i < numUpsertions; i++ {
+		r.Upsertions[i] = AlterUserScramCredentialsUpsert{}
+		if r.Upsertions[i].Name, err = pd.getCompactString(); err != nil {
+			return err
+		}
+		mechanism, err := pd.getInt8()
+		if err != nil {
+			return err
+		}
+
+		r.Upsertions[i].Mechanism = ScramMechanismType(mechanism)
+		if r.Upsertions[i].Iterations, err = pd.getInt32(); err != nil {
+			return err
+		}
+		if r.Upsertions[i].Salt, err = pd.getCompactBytes(); err != nil {
+			return err
+		}
+		if r.Upsertions[i].saltedPassword, err = pd.getCompactBytes(); err != nil {
+			return err
+		}
+		if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
+	if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (r *AlterUserScramCredentialsRequest) key() int16 {
+	return 51
+}
+
+func (r *AlterUserScramCredentialsRequest) version() int16 {
+	return r.Version
+}
+
+func (r *AlterUserScramCredentialsRequest) headerVersion() int16 {
+	return 2
+}
+
+func (r *AlterUserScramCredentialsRequest) requiredVersion() KafkaVersion {
+	return V2_7_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go
new file mode 100644
index 0000000..31e167b
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go
@@ -0,0 +1,94 @@
+package sarama
+
+import "time"
+
+type AlterUserScramCredentialsResponse struct {
+	Version int16
+
+	ThrottleTime time.Duration
+
+	Results []*AlterUserScramCredentialsResult
+}
+
+type AlterUserScramCredentialsResult struct {
+	User string
+
+	ErrorCode    KError
+	ErrorMessage *string
+}
+
+func (r *AlterUserScramCredentialsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
+	pe.putCompactArrayLength(len(r.Results))
+
+	for _, u := range r.Results {
+		if err := pe.putCompactString(u.User); err != nil {
+			return err
+		}
+		pe.putInt16(int16(u.ErrorCode))
+		if err := pe.putNullableCompactString(u.ErrorMessage); err != nil {
+			return err
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (r *AlterUserScramCredentialsResponse) decode(pd packetDecoder, version int16) error {
+	throttleTime, err := pd.getInt32()
+	if err != nil {
+		return err
+	}
+	r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond
+
+	numResults, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	if numResults > 0 {
+		r.Results = make([]*AlterUserScramCredentialsResult, numResults)
+		for i := 0; i < numResults; i++ {
+			r.Results[i] = &AlterUserScramCredentialsResult{}
+			if r.Results[i].User, err = pd.getCompactString(); err != nil {
+				return err
+			}
+
+			kerr, err := pd.getInt16()
+			if err != nil {
+				return err
+			}
+
+			r.Results[i].ErrorCode = KError(kerr)
+			if r.Results[i].ErrorMessage, err = pd.getCompactNullableString(); err != nil {
+				return err
+			}
+			if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (r *AlterUserScramCredentialsResponse) key() int16 {
+	return 51
+}
+
+func (r *AlterUserScramCredentialsResponse) version() int16 {
+	return r.Version
+}
+
+func (r *AlterUserScramCredentialsResponse) headerVersion() int16 {
+	return 2
+}
+
+func (r *AlterUserScramCredentialsResponse) requiredVersion() KafkaVersion {
+	return V2_7_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/api_versions_request.go b/vendor/github.com/Shopify/sarama/api_versions_request.go
index b33167c..bee92c0 100644
--- a/vendor/github.com/Shopify/sarama/api_versions_request.go
+++ b/vendor/github.com/Shopify/sarama/api_versions_request.go
@@ -1,8 +1,7 @@
 package sarama
 
-//ApiVersionsRequest ...
-type ApiVersionsRequest struct {
-}
+// ApiVersionsRequest ...
+type ApiVersionsRequest struct{}
 
 func (a *ApiVersionsRequest) encode(pe packetEncoder) error {
 	return nil
@@ -20,6 +19,10 @@
 	return 0
 }
 
+func (a *ApiVersionsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *ApiVersionsRequest) requiredVersion() KafkaVersion {
 	return V0_10_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/api_versions_response.go b/vendor/github.com/Shopify/sarama/api_versions_response.go
index bb1f0b3..0e72e39 100644
--- a/vendor/github.com/Shopify/sarama/api_versions_response.go
+++ b/vendor/github.com/Shopify/sarama/api_versions_response.go
@@ -1,6 +1,6 @@
 package sarama
 
-//ApiVersionsResponseBlock is an api version reponse block type
+// ApiVersionsResponseBlock is an api version response block type
 type ApiVersionsResponseBlock struct {
 	ApiKey     int16
 	MinVersion int16
@@ -32,7 +32,7 @@
 	return nil
 }
 
-//ApiVersionsResponse is an api version response type
+// ApiVersionsResponse is an api version response type
 type ApiVersionsResponse struct {
 	Err         KError
 	ApiVersions []*ApiVersionsResponseBlock
@@ -84,6 +84,10 @@
 	return 0
 }
 
+func (a *ApiVersionsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *ApiVersionsResponse) requiredVersion() KafkaVersion {
 	return V0_10_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/async_producer.go b/vendor/github.com/Shopify/sarama/async_producer.go
index 11e0849..5911f7b 100644
--- a/vendor/github.com/Shopify/sarama/async_producer.go
+++ b/vendor/github.com/Shopify/sarama/async_producer.go
@@ -60,13 +60,28 @@
 	noProducerEpoch = -1
 )
 
-func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) int32 {
+func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) (int32, int16) {
 	key := fmt.Sprintf("%s-%d", topic, partition)
 	t.mutex.Lock()
 	defer t.mutex.Unlock()
 	sequence := t.sequenceNumbers[key]
 	t.sequenceNumbers[key] = sequence + 1
-	return sequence
+	return sequence, t.producerEpoch
+}
+
+func (t *transactionManager) bumpEpoch() {
+	t.mutex.Lock()
+	defer t.mutex.Unlock()
+	t.producerEpoch++
+	for k := range t.sequenceNumbers {
+		t.sequenceNumbers[k] = 0
+	}
+}
+
+func (t *transactionManager) getProducerID() (int64, int16) {
+	t.mutex.Lock()
+	defer t.mutex.Unlock()
+	return t.producerID, t.producerEpoch
 }
 
 func newTransactionManager(conf *Config, client Client) (*transactionManager, error) {
@@ -208,6 +223,8 @@
 	flags          flagSet
 	expectation    chan *ProducerError
 	sequenceNumber int32
+	producerEpoch  int16
+	hasSequence    bool
 }
 
 const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc.
@@ -234,6 +251,9 @@
 func (m *ProducerMessage) clear() {
 	m.flags = 0
 	m.retries = 0
+	m.sequenceNumber = 0
+	m.producerEpoch = 0
+	m.hasSequence = false
 }
 
 // ProducerError is the type of error generated when the producer fails to deliver a message.
@@ -247,6 +267,10 @@
 	return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
 }
 
+func (pe ProducerError) Unwrap() error {
+	return pe.Err
+}
+
 // ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
 // It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
 // when closing a producer.
@@ -328,6 +352,10 @@
 			p.inFlight.Add(1)
 		}
 
+		for _, interceptor := range p.conf.Producer.Interceptors {
+			msg.safelyApplyInterceptor(interceptor)
+		}
+
 		version := 1
 		if p.conf.Version.IsAtLeast(V0_11_0_0) {
 			version = 2
@@ -388,10 +416,6 @@
 				continue
 			}
 		}
-		// All messages being retried (sent or not) have already had their retry count updated
-		if tp.parent.conf.Producer.Idempotent && msg.retries == 0 {
-			msg.sequenceNumber = tp.parent.txnmgr.getAndIncrementSequenceNumber(msg.Topic, msg.Partition)
-		}
 
 		handler := tp.handlers[msg.Partition]
 		if handler == nil {
@@ -411,7 +435,7 @@
 	var partitions []int32
 
 	err := tp.breaker.Run(func() (err error) {
-		var requiresConsistency = false
+		requiresConsistency := false
 		if ep, ok := tp.partitioner.(DynamicConsistencyPartitioner); ok {
 			requiresConsistency = ep.MessageRequiresConsistency(msg)
 		} else {
@@ -425,7 +449,6 @@
 		}
 		return
 	})
-
 	if err != nil {
 		return err
 	}
@@ -520,7 +543,6 @@
 	}()
 
 	for msg := range pp.input {
-
 		if pp.brokerProducer != nil && pp.brokerProducer.abandoned != nil {
 			select {
 			case <-pp.brokerProducer.abandoned:
@@ -571,6 +593,15 @@
 			Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
 		}
 
+		// Now that we know we have a broker to actually try and send this message to, generate the sequence
+		// number for it.
+		// All messages being retried (sent or not) have already had their retry count updated
+		// Also, ignore "special" syn/fin messages used to sync the brokerProducer and the topicProducer.
+		if pp.parent.conf.Producer.Idempotent && msg.retries == 0 && msg.flags == 0 {
+			msg.sequenceNumber, msg.producerEpoch = pp.parent.txnmgr.getAndIncrementSequenceNumber(msg.Topic, msg.Partition)
+			msg.hasSequence = true
+		}
+
 		pp.brokerProducer.input <- msg
 	}
 }
@@ -652,6 +683,7 @@
 		input:          input,
 		output:         bridge,
 		responses:      responses,
+		stopchan:       make(chan struct{}),
 		buffer:         newProduceSet(p),
 		currentRetries: make(map[string]map[int32]error),
 	}
@@ -696,6 +728,7 @@
 	output    chan<- *produceSet
 	responses <-chan *brokerProducerResponse
 	abandoned chan struct{}
+	stopchan  chan struct{}
 
 	buffer     *produceSet
 	timer      <-chan time.Time
@@ -711,12 +744,17 @@
 
 	for {
 		select {
-		case msg := <-bp.input:
-			if msg == nil {
+		case msg, ok := <-bp.input:
+			if !ok {
+				Logger.Printf("producer/broker/%d input chan closed\n", bp.broker.ID())
 				bp.shutdown()
 				return
 			}
 
+			if msg == nil {
+				continue
+			}
+
 			if msg.flags&syn == syn {
 				Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n",
 					bp.broker.ID(), msg.Topic, msg.Partition)
@@ -742,12 +780,21 @@
 			}
 
 			if bp.buffer.wouldOverflow(msg) {
-				if err := bp.waitForSpace(msg); err != nil {
+				Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID())
+				if err := bp.waitForSpace(msg, false); err != nil {
 					bp.parent.retryMessage(msg, err)
 					continue
 				}
 			}
 
+			if bp.parent.txnmgr.producerID != noProducerID && bp.buffer.producerEpoch != msg.producerEpoch {
+				// The epoch was reset, need to roll the buffer over
+				Logger.Printf("producer/broker/%d detected epoch rollover, waiting for new buffer\n", bp.broker.ID())
+				if err := bp.waitForSpace(msg, true); err != nil {
+					bp.parent.retryMessage(msg, err)
+					continue
+				}
+			}
 			if err := bp.buffer.add(msg); err != nil {
 				bp.parent.returnError(msg, err)
 				continue
@@ -760,8 +807,14 @@
 			bp.timerFired = true
 		case output <- bp.buffer:
 			bp.rollOver()
-		case response := <-bp.responses:
-			bp.handleResponse(response)
+		case response, ok := <-bp.responses:
+			if ok {
+				bp.handleResponse(response)
+			}
+		case <-bp.stopchan:
+			Logger.Printf(
+				"producer/broker/%d run loop asked to stop\n", bp.broker.ID())
+			return
 		}
 
 		if bp.timerFired || bp.buffer.readyToFlush() {
@@ -785,7 +838,7 @@
 	for response := range bp.responses {
 		bp.handleResponse(response)
 	}
-
+	close(bp.stopchan)
 	Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID())
 }
 
@@ -797,9 +850,7 @@
 	return bp.currentRetries[msg.Topic][msg.Partition]
 }
 
-func (bp *brokerProducer) waitForSpace(msg *ProducerMessage) error {
-	Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID())
-
+func (bp *brokerProducer) waitForSpace(msg *ProducerMessage, forceRollover bool) error {
 	for {
 		select {
 		case response := <-bp.responses:
@@ -807,7 +858,7 @@
 			// handling a response can change our state, so re-check some things
 			if reason := bp.needsRetry(msg); reason != nil {
 				return reason
-			} else if !bp.buffer.wouldOverflow(msg) {
+			} else if !bp.buffer.wouldOverflow(msg) && !forceRollover {
 				return nil
 			}
 		case bp.output <- bp.buffer:
@@ -1018,6 +1069,12 @@
 }
 
 func (p *asyncProducer) returnError(msg *ProducerMessage, err error) {
+	// We need to reset the producer ID epoch if we set a sequence number on it, because the broker
+	// will never see a message with this number, so we can never continue the sequence.
+	if msg.hasSequence {
+		Logger.Printf("producer/txnmanager rolling over epoch due to publish failure on %s/%d", msg.Topic, msg.Partition)
+		p.txnmgr.bumpEpoch()
+	}
 	msg.clear()
 	pErr := &ProducerError{Msg: msg, Err: err}
 	if p.conf.Producer.Return.Errors {
diff --git a/vendor/github.com/Shopify/sarama/balance_strategy.go b/vendor/github.com/Shopify/sarama/balance_strategy.go
index 2fce17f..9855bf4 100644
--- a/vendor/github.com/Shopify/sarama/balance_strategy.go
+++ b/vendor/github.com/Shopify/sarama/balance_strategy.go
@@ -1,8 +1,25 @@
 package sarama
 
 import (
+	"container/heap"
+	"errors"
+	"fmt"
 	"math"
 	"sort"
+	"strings"
+)
+
+const (
+	// RangeBalanceStrategyName identifies strategies that use the range partition assignment strategy
+	RangeBalanceStrategyName = "range"
+
+	// RoundRobinBalanceStrategyName identifies strategies that use the round-robin partition assignment strategy
+	RoundRobinBalanceStrategyName = "roundrobin"
+
+	// StickyBalanceStrategyName identifies strategies that use the sticky-partition assignment strategy
+	StickyBalanceStrategyName = "sticky"
+
+	defaultGeneration = -1
 )
 
 // BalanceStrategyPlan is the results of any BalanceStrategy.Plan attempt.
@@ -32,6 +49,10 @@
 	// Plan accepts a map of `memberID -> metadata` and a map of `topic -> partitions`
 	// and returns a distribution plan.
 	Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error)
+
+	// AssignmentData returns the serialized assignment data for the specified
+	// memberID
+	AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error)
 }
 
 // --------------------------------------------------------------------
@@ -41,7 +62,7 @@
 //   M1: {T: [0, 1, 2]}
 //   M2: {T: [3, 4, 5]}
 var BalanceStrategyRange = &balanceStrategy{
-	name: "range",
+	name: RangeBalanceStrategyName,
 	coreFn: func(plan BalanceStrategyPlan, memberIDs []string, topic string, partitions []int32) {
 		step := float64(len(partitions)) / float64(len(memberIDs))
 
@@ -54,19 +75,18 @@
 	},
 }
 
-// BalanceStrategyRoundRobin assigns partitions to members in alternating order.
+// BalanceStrategySticky assigns partitions to members with an attempt to preserve earlier assignments
+// while maintain a balanced partition distribution.
 // Example with topic T with six partitions (0..5) and two members (M1, M2):
 //   M1: {T: [0, 2, 4]}
 //   M2: {T: [1, 3, 5]}
-var BalanceStrategyRoundRobin = &balanceStrategy{
-	name: "roundrobin",
-	coreFn: func(plan BalanceStrategyPlan, memberIDs []string, topic string, partitions []int32) {
-		for i, part := range partitions {
-			memberID := memberIDs[i%len(memberIDs)]
-			plan.Add(memberID, topic, part)
-		}
-	},
-}
+//
+// On reassignment with an additional consumer, you might get an assignment plan like:
+//   M1: {T: [0, 2]}
+//   M2: {T: [1, 3]}
+//   M3: {T: [4, 5]}
+//
+var BalanceStrategySticky = &stickyBalanceStrategy{}
 
 // --------------------------------------------------------------------
 
@@ -104,6 +124,11 @@
 	return plan, nil
 }
 
+// AssignmentData simple strategies do not require any shared assignment data
+func (s *balanceStrategy) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) {
+	return nil, nil
+}
+
 type balanceStrategySortable struct {
 	topic     string
 	memberIDs []string
@@ -113,6 +138,7 @@
 func (p balanceStrategySortable) Swap(i, j int) {
 	p.memberIDs[i], p.memberIDs[j] = p.memberIDs[j], p.memberIDs[i]
 }
+
 func (p balanceStrategySortable) Less(i, j int) bool {
 	return balanceStrategyHashValue(p.topic, p.memberIDs[i]) < balanceStrategyHashValue(p.topic, p.memberIDs[j])
 }
@@ -127,3 +153,984 @@
 	}
 	return h
 }
+
+type stickyBalanceStrategy struct {
+	movements partitionMovements
+}
+
+// Name implements BalanceStrategy.
+func (s *stickyBalanceStrategy) Name() string { return StickyBalanceStrategyName }
+
+// Plan implements BalanceStrategy.
+func (s *stickyBalanceStrategy) Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) {
+	// track partition movements during generation of the partition assignment plan
+	s.movements = partitionMovements{
+		Movements:                 make(map[topicPartitionAssignment]consumerPair),
+		PartitionMovementsByTopic: make(map[string]map[consumerPair]map[topicPartitionAssignment]bool),
+	}
+
+	// prepopulate the current assignment state from userdata on the consumer group members
+	currentAssignment, prevAssignment, err := prepopulateCurrentAssignments(members)
+	if err != nil {
+		return nil, err
+	}
+
+	// determine if we're dealing with a completely fresh assignment, or if there's existing assignment state
+	isFreshAssignment := false
+	if len(currentAssignment) == 0 {
+		isFreshAssignment = true
+	}
+
+	// create a mapping of all current topic partitions and the consumers that can be assigned to them
+	partition2AllPotentialConsumers := make(map[topicPartitionAssignment][]string)
+	for topic, partitions := range topics {
+		for _, partition := range partitions {
+			partition2AllPotentialConsumers[topicPartitionAssignment{Topic: topic, Partition: partition}] = []string{}
+		}
+	}
+
+	// create a mapping of all consumers to all potential topic partitions that can be assigned to them
+	// also, populate the mapping of partitions to potential consumers
+	consumer2AllPotentialPartitions := make(map[string][]topicPartitionAssignment, len(members))
+	for memberID, meta := range members {
+		consumer2AllPotentialPartitions[memberID] = make([]topicPartitionAssignment, 0)
+		for _, topicSubscription := range meta.Topics {
+			// only evaluate topic subscriptions that are present in the supplied topics map
+			if _, found := topics[topicSubscription]; found {
+				for _, partition := range topics[topicSubscription] {
+					topicPartition := topicPartitionAssignment{Topic: topicSubscription, Partition: partition}
+					consumer2AllPotentialPartitions[memberID] = append(consumer2AllPotentialPartitions[memberID], topicPartition)
+					partition2AllPotentialConsumers[topicPartition] = append(partition2AllPotentialConsumers[topicPartition], memberID)
+				}
+			}
+		}
+
+		// add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist
+		if _, exists := currentAssignment[memberID]; !exists {
+			currentAssignment[memberID] = make([]topicPartitionAssignment, 0)
+		}
+	}
+
+	// create a mapping of each partition to its current consumer, where possible
+	currentPartitionConsumers := make(map[topicPartitionAssignment]string, len(currentAssignment))
+	unvisitedPartitions := make(map[topicPartitionAssignment]bool, len(partition2AllPotentialConsumers))
+	for partition := range partition2AllPotentialConsumers {
+		unvisitedPartitions[partition] = true
+	}
+	var unassignedPartitions []topicPartitionAssignment
+	for memberID, partitions := range currentAssignment {
+		var keepPartitions []topicPartitionAssignment
+		for _, partition := range partitions {
+			// If this partition no longer exists at all, likely due to the
+			// topic being deleted, we remove the partition from the member.
+			if _, exists := partition2AllPotentialConsumers[partition]; !exists {
+				continue
+			}
+			delete(unvisitedPartitions, partition)
+			currentPartitionConsumers[partition] = memberID
+
+			if !strsContains(members[memberID].Topics, partition.Topic) {
+				unassignedPartitions = append(unassignedPartitions, partition)
+				continue
+			}
+			keepPartitions = append(keepPartitions, partition)
+		}
+		currentAssignment[memberID] = keepPartitions
+	}
+	for unvisited := range unvisitedPartitions {
+		unassignedPartitions = append(unassignedPartitions, unvisited)
+	}
+
+	// sort the topic partitions in order of priority for reassignment
+	sortedPartitions := sortPartitions(currentAssignment, prevAssignment, isFreshAssignment, partition2AllPotentialConsumers, consumer2AllPotentialPartitions)
+
+	// at this point we have preserved all valid topic partition to consumer assignments and removed
+	// all invalid topic partitions and invalid consumers. Now we need to assign unassignedPartitions
+	// to consumers so that the topic partition assignments are as balanced as possible.
+
+	// an ascending sorted set of consumers based on how many topic partitions are already assigned to them
+	sortedCurrentSubscriptions := sortMemberIDsByPartitionAssignments(currentAssignment)
+	s.balance(currentAssignment, prevAssignment, sortedPartitions, unassignedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumers)
+
+	// Assemble plan
+	plan := make(BalanceStrategyPlan, len(currentAssignment))
+	for memberID, assignments := range currentAssignment {
+		if len(assignments) == 0 {
+			plan[memberID] = make(map[string][]int32)
+		} else {
+			for _, assignment := range assignments {
+				plan.Add(memberID, assignment.Topic, assignment.Partition)
+			}
+		}
+	}
+	return plan, nil
+}
+
+// AssignmentData serializes the set of topics currently assigned to the
+// specified member as part of the supplied balance plan
+func (s *stickyBalanceStrategy) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) {
+	return encode(&StickyAssignorUserDataV1{
+		Topics:     topics,
+		Generation: generationID,
+	}, nil)
+}
+
+func strsContains(s []string, value string) bool {
+	for _, entry := range s {
+		if entry == value {
+			return true
+		}
+	}
+	return false
+}
+
+// Balance assignments across consumers for maximum fairness and stickiness.
+func (s *stickyBalanceStrategy) balance(currentAssignment map[string][]topicPartitionAssignment, prevAssignment map[topicPartitionAssignment]consumerGenerationPair, sortedPartitions []topicPartitionAssignment, unassignedPartitions []topicPartitionAssignment, sortedCurrentSubscriptions []string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, currentPartitionConsumer map[topicPartitionAssignment]string) {
+	initializing := false
+	if len(sortedCurrentSubscriptions) == 0 || len(currentAssignment[sortedCurrentSubscriptions[0]]) == 0 {
+		initializing = true
+	}
+
+	// assign all unassigned partitions
+	for _, partition := range unassignedPartitions {
+		// skip if there is no potential consumer for the partition
+		if len(partition2AllPotentialConsumers[partition]) == 0 {
+			continue
+		}
+		sortedCurrentSubscriptions = assignPartition(partition, sortedCurrentSubscriptions, currentAssignment, consumer2AllPotentialPartitions, currentPartitionConsumer)
+	}
+
+	// narrow down the reassignment scope to only those partitions that can actually be reassigned
+	for partition := range partition2AllPotentialConsumers {
+		if !canTopicPartitionParticipateInReassignment(partition, partition2AllPotentialConsumers) {
+			sortedPartitions = removeTopicPartitionFromMemberAssignments(sortedPartitions, partition)
+		}
+	}
+
+	// narrow down the reassignment scope to only those consumers that are subject to reassignment
+	fixedAssignments := make(map[string][]topicPartitionAssignment)
+	for memberID := range consumer2AllPotentialPartitions {
+		if !canConsumerParticipateInReassignment(memberID, currentAssignment, consumer2AllPotentialPartitions, partition2AllPotentialConsumers) {
+			fixedAssignments[memberID] = currentAssignment[memberID]
+			delete(currentAssignment, memberID)
+			sortedCurrentSubscriptions = sortMemberIDsByPartitionAssignments(currentAssignment)
+		}
+	}
+
+	// create a deep copy of the current assignment so we can revert to it if we do not get a more balanced assignment later
+	preBalanceAssignment := deepCopyAssignment(currentAssignment)
+	preBalancePartitionConsumers := make(map[topicPartitionAssignment]string, len(currentPartitionConsumer))
+	for k, v := range currentPartitionConsumer {
+		preBalancePartitionConsumers[k] = v
+	}
+
+	reassignmentPerformed := s.performReassignments(sortedPartitions, currentAssignment, prevAssignment, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer)
+
+	// if we are not preserving existing assignments and we have made changes to the current assignment
+	// make sure we are getting a more balanced assignment; otherwise, revert to previous assignment
+	if !initializing && reassignmentPerformed && getBalanceScore(currentAssignment) >= getBalanceScore(preBalanceAssignment) {
+		currentAssignment = deepCopyAssignment(preBalanceAssignment)
+		currentPartitionConsumer = make(map[topicPartitionAssignment]string, len(preBalancePartitionConsumers))
+		for k, v := range preBalancePartitionConsumers {
+			currentPartitionConsumer[k] = v
+		}
+	}
+
+	// add the fixed assignments (those that could not change) back
+	for consumer, assignments := range fixedAssignments {
+		currentAssignment[consumer] = assignments
+	}
+}
+
+// BalanceStrategyRoundRobin assigns partitions to members in alternating order.
+// For example, there are two topics (t0, t1) and two consumer (m0, m1), and each topic has three partitions (p0, p1, p2):
+// M0: [t0p0, t0p2, t1p1]
+// M1: [t0p1, t1p0, t1p2]
+var BalanceStrategyRoundRobin = new(roundRobinBalancer)
+
+type roundRobinBalancer struct{}
+
+func (b *roundRobinBalancer) Name() string {
+	return RoundRobinBalanceStrategyName
+}
+
+func (b *roundRobinBalancer) Plan(memberAndMetadata map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) {
+	if len(memberAndMetadata) == 0 || len(topics) == 0 {
+		return nil, errors.New("members and topics are not provided")
+	}
+	// sort partitions
+	var topicPartitions []topicAndPartition
+	for topic, partitions := range topics {
+		for _, partition := range partitions {
+			topicPartitions = append(topicPartitions, topicAndPartition{topic: topic, partition: partition})
+		}
+	}
+	sort.SliceStable(topicPartitions, func(i, j int) bool {
+		pi := topicPartitions[i]
+		pj := topicPartitions[j]
+		return pi.comparedValue() < pj.comparedValue()
+	})
+
+	// sort members
+	var members []memberAndTopic
+	for memberID, meta := range memberAndMetadata {
+		m := memberAndTopic{
+			memberID: memberID,
+			topics:   make(map[string]struct{}),
+		}
+		for _, t := range meta.Topics {
+			m.topics[t] = struct{}{}
+		}
+		members = append(members, m)
+	}
+	sort.SliceStable(members, func(i, j int) bool {
+		mi := members[i]
+		mj := members[j]
+		return mi.memberID < mj.memberID
+	})
+
+	// assign partitions
+	plan := make(BalanceStrategyPlan, len(members))
+	i := 0
+	n := len(members)
+	for _, tp := range topicPartitions {
+		m := members[i%n]
+		for !m.hasTopic(tp.topic) {
+			i++
+			m = members[i%n]
+		}
+		plan.Add(m.memberID, tp.topic, tp.partition)
+		i++
+	}
+	return plan, nil
+}
+
+func (b *roundRobinBalancer) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) {
+	return nil, nil // do nothing for now
+}
+
+type topicAndPartition struct {
+	topic     string
+	partition int32
+}
+
+func (tp *topicAndPartition) comparedValue() string {
+	return fmt.Sprintf("%s-%d", tp.topic, tp.partition)
+}
+
+type memberAndTopic struct {
+	memberID string
+	topics   map[string]struct{}
+}
+
+func (m *memberAndTopic) hasTopic(topic string) bool {
+	_, isExist := m.topics[topic]
+	return isExist
+}
+
+// Calculate the balance score of the given assignment, as the sum of assigned partitions size difference of all consumer pairs.
+// A perfectly balanced assignment (with all consumers getting the same number of partitions) has a balance score of 0.
+// Lower balance score indicates a more balanced assignment.
+func getBalanceScore(assignment map[string][]topicPartitionAssignment) int {
+	consumer2AssignmentSize := make(map[string]int, len(assignment))
+	for memberID, partitions := range assignment {
+		consumer2AssignmentSize[memberID] = len(partitions)
+	}
+
+	var score float64
+	for memberID, consumerAssignmentSize := range consumer2AssignmentSize {
+		delete(consumer2AssignmentSize, memberID)
+		for _, otherConsumerAssignmentSize := range consumer2AssignmentSize {
+			score += math.Abs(float64(consumerAssignmentSize - otherConsumerAssignmentSize))
+		}
+	}
+	return int(score)
+}
+
+// Determine whether the current assignment plan is balanced.
+func isBalanced(currentAssignment map[string][]topicPartitionAssignment, allSubscriptions map[string][]topicPartitionAssignment) bool {
+	sortedCurrentSubscriptions := sortMemberIDsByPartitionAssignments(currentAssignment)
+	min := len(currentAssignment[sortedCurrentSubscriptions[0]])
+	max := len(currentAssignment[sortedCurrentSubscriptions[len(sortedCurrentSubscriptions)-1]])
+	if min >= max-1 {
+		// if minimum and maximum numbers of partitions assigned to consumers differ by at most one return true
+		return true
+	}
+
+	// create a mapping from partitions to the consumer assigned to them
+	allPartitions := make(map[topicPartitionAssignment]string)
+	for memberID, partitions := range currentAssignment {
+		for _, partition := range partitions {
+			if _, exists := allPartitions[partition]; exists {
+				Logger.Printf("Topic %s Partition %d is assigned more than one consumer", partition.Topic, partition.Partition)
+			}
+			allPartitions[partition] = memberID
+		}
+	}
+
+	// for each consumer that does not have all the topic partitions it can get make sure none of the topic partitions it
+	// could but did not get cannot be moved to it (because that would break the balance)
+	for _, memberID := range sortedCurrentSubscriptions {
+		consumerPartitions := currentAssignment[memberID]
+		consumerPartitionCount := len(consumerPartitions)
+
+		// skip if this consumer already has all the topic partitions it can get
+		if consumerPartitionCount == len(allSubscriptions[memberID]) {
+			continue
+		}
+
+		// otherwise make sure it cannot get any more
+		potentialTopicPartitions := allSubscriptions[memberID]
+		for _, partition := range potentialTopicPartitions {
+			if !memberAssignmentsIncludeTopicPartition(currentAssignment[memberID], partition) {
+				otherConsumer := allPartitions[partition]
+				otherConsumerPartitionCount := len(currentAssignment[otherConsumer])
+				if consumerPartitionCount < otherConsumerPartitionCount {
+					return false
+				}
+			}
+		}
+	}
+	return true
+}
+
+// Reassign all topic partitions that need reassignment until balanced.
+func (s *stickyBalanceStrategy) performReassignments(reassignablePartitions []topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, prevAssignment map[topicPartitionAssignment]consumerGenerationPair, sortedCurrentSubscriptions []string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, currentPartitionConsumer map[topicPartitionAssignment]string) bool {
+	reassignmentPerformed := false
+	modified := false
+
+	// repeat reassignment until no partition can be moved to improve the balance
+	for {
+		modified = false
+		// reassign all reassignable partitions (starting from the partition with least potential consumers and if needed)
+		// until the full list is processed or a balance is achieved
+		for _, partition := range reassignablePartitions {
+			if isBalanced(currentAssignment, consumer2AllPotentialPartitions) {
+				break
+			}
+
+			// the partition must have at least two consumers
+			if len(partition2AllPotentialConsumers[partition]) <= 1 {
+				Logger.Printf("Expected more than one potential consumer for partition %s topic %d", partition.Topic, partition.Partition)
+			}
+
+			// the partition must have a consumer
+			consumer := currentPartitionConsumer[partition]
+			if consumer == "" {
+				Logger.Printf("Expected topic %s partition %d to be assigned to a consumer", partition.Topic, partition.Partition)
+			}
+
+			if _, exists := prevAssignment[partition]; exists {
+				if len(currentAssignment[consumer]) > (len(currentAssignment[prevAssignment[partition].MemberID]) + 1) {
+					sortedCurrentSubscriptions = s.reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, prevAssignment[partition].MemberID)
+					reassignmentPerformed = true
+					modified = true
+					continue
+				}
+			}
+
+			// check if a better-suited consumer exists for the partition; if so, reassign it
+			for _, otherConsumer := range partition2AllPotentialConsumers[partition] {
+				if len(currentAssignment[consumer]) > (len(currentAssignment[otherConsumer]) + 1) {
+					sortedCurrentSubscriptions = s.reassignPartitionToNewConsumer(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, consumer2AllPotentialPartitions)
+					reassignmentPerformed = true
+					modified = true
+					break
+				}
+			}
+		}
+		if !modified {
+			return reassignmentPerformed
+		}
+	}
+}
+
+// Identify a new consumer for a topic partition and reassign it.
+func (s *stickyBalanceStrategy) reassignPartitionToNewConsumer(partition topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) []string {
+	for _, anotherConsumer := range sortedCurrentSubscriptions {
+		if memberAssignmentsIncludeTopicPartition(consumer2AllPotentialPartitions[anotherConsumer], partition) {
+			return s.reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, anotherConsumer)
+		}
+	}
+	return sortedCurrentSubscriptions
+}
+
+// Reassign a specific partition to a new consumer
+func (s *stickyBalanceStrategy) reassignPartition(partition topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string, newConsumer string) []string {
+	consumer := currentPartitionConsumer[partition]
+	// find the correct partition movement considering the stickiness requirement
+	partitionToBeMoved := s.movements.getTheActualPartitionToBeMoved(partition, consumer, newConsumer)
+	return s.processPartitionMovement(partitionToBeMoved, newConsumer, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer)
+}
+
+// Track the movement of a topic partition after assignment
+func (s *stickyBalanceStrategy) processPartitionMovement(partition topicPartitionAssignment, newConsumer string, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string) []string {
+	oldConsumer := currentPartitionConsumer[partition]
+	s.movements.movePartition(partition, oldConsumer, newConsumer)
+
+	currentAssignment[oldConsumer] = removeTopicPartitionFromMemberAssignments(currentAssignment[oldConsumer], partition)
+	currentAssignment[newConsumer] = append(currentAssignment[newConsumer], partition)
+	currentPartitionConsumer[partition] = newConsumer
+	return sortMemberIDsByPartitionAssignments(currentAssignment)
+}
+
+// Determine whether a specific consumer should be considered for topic partition assignment.
+func canConsumerParticipateInReassignment(memberID string, currentAssignment map[string][]topicPartitionAssignment, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) bool {
+	currentPartitions := currentAssignment[memberID]
+	currentAssignmentSize := len(currentPartitions)
+	maxAssignmentSize := len(consumer2AllPotentialPartitions[memberID])
+	if currentAssignmentSize > maxAssignmentSize {
+		Logger.Printf("The consumer %s is assigned more partitions than the maximum possible", memberID)
+	}
+	if currentAssignmentSize < maxAssignmentSize {
+		// if a consumer is not assigned all its potential partitions it is subject to reassignment
+		return true
+	}
+	for _, partition := range currentPartitions {
+		if canTopicPartitionParticipateInReassignment(partition, partition2AllPotentialConsumers) {
+			return true
+		}
+	}
+	return false
+}
+
+// Only consider reassigning those topic partitions that have two or more potential consumers.
+func canTopicPartitionParticipateInReassignment(partition topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) bool {
+	return len(partition2AllPotentialConsumers[partition]) >= 2
+}
+
+// The assignment should improve the overall balance of the partition assignments to consumers.
+func assignPartition(partition topicPartitionAssignment, sortedCurrentSubscriptions []string, currentAssignment map[string][]topicPartitionAssignment, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, currentPartitionConsumer map[topicPartitionAssignment]string) []string {
+	for _, memberID := range sortedCurrentSubscriptions {
+		if memberAssignmentsIncludeTopicPartition(consumer2AllPotentialPartitions[memberID], partition) {
+			currentAssignment[memberID] = append(currentAssignment[memberID], partition)
+			currentPartitionConsumer[partition] = memberID
+			break
+		}
+	}
+	return sortMemberIDsByPartitionAssignments(currentAssignment)
+}
+
+// Deserialize topic partition assignment data to aid with creation of a sticky assignment.
+func deserializeTopicPartitionAssignment(userDataBytes []byte) (StickyAssignorUserData, error) {
+	userDataV1 := &StickyAssignorUserDataV1{}
+	if err := decode(userDataBytes, userDataV1); err != nil {
+		userDataV0 := &StickyAssignorUserDataV0{}
+		if err := decode(userDataBytes, userDataV0); err != nil {
+			return nil, err
+		}
+		return userDataV0, nil
+	}
+	return userDataV1, nil
+}
+
+// filterAssignedPartitions returns a map of consumer group members to their list of previously-assigned topic partitions, limited
+// to those topic partitions currently reported by the Kafka cluster.
+func filterAssignedPartitions(currentAssignment map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) map[string][]topicPartitionAssignment {
+	assignments := deepCopyAssignment(currentAssignment)
+	for memberID, partitions := range assignments {
+		// perform in-place filtering
+		i := 0
+		for _, partition := range partitions {
+			if _, exists := partition2AllPotentialConsumers[partition]; exists {
+				partitions[i] = partition
+				i++
+			}
+		}
+		assignments[memberID] = partitions[:i]
+	}
+	return assignments
+}
+
+func removeTopicPartitionFromMemberAssignments(assignments []topicPartitionAssignment, topic topicPartitionAssignment) []topicPartitionAssignment {
+	for i, assignment := range assignments {
+		if assignment == topic {
+			return append(assignments[:i], assignments[i+1:]...)
+		}
+	}
+	return assignments
+}
+
+func memberAssignmentsIncludeTopicPartition(assignments []topicPartitionAssignment, topic topicPartitionAssignment) bool {
+	for _, assignment := range assignments {
+		if assignment == topic {
+			return true
+		}
+	}
+	return false
+}
+
+func sortPartitions(currentAssignment map[string][]topicPartitionAssignment, partitionsWithADifferentPreviousAssignment map[topicPartitionAssignment]consumerGenerationPair, isFreshAssignment bool, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) []topicPartitionAssignment {
+	unassignedPartitions := make(map[topicPartitionAssignment]bool, len(partition2AllPotentialConsumers))
+	for partition := range partition2AllPotentialConsumers {
+		unassignedPartitions[partition] = true
+	}
+
+	sortedPartitions := make([]topicPartitionAssignment, 0)
+	if !isFreshAssignment && areSubscriptionsIdentical(partition2AllPotentialConsumers, consumer2AllPotentialPartitions) {
+		// if this is a reassignment and the subscriptions are identical (all consumers can consumer from all topics)
+		// then we just need to simply list partitions in a round robin fashion (from consumers with
+		// most assigned partitions to those with least)
+		assignments := filterAssignedPartitions(currentAssignment, partition2AllPotentialConsumers)
+
+		// use priority-queue to evaluate consumer group members in descending-order based on
+		// the number of topic partition assignments (i.e. consumers with most assignments first)
+		pq := make(assignmentPriorityQueue, len(assignments))
+		i := 0
+		for consumerID, consumerAssignments := range assignments {
+			pq[i] = &consumerGroupMember{
+				id:          consumerID,
+				assignments: consumerAssignments,
+			}
+			i++
+		}
+		heap.Init(&pq)
+
+		for {
+			// loop until no consumer-group members remain
+			if pq.Len() == 0 {
+				break
+			}
+			member := pq[0]
+
+			// partitions that were assigned to a different consumer last time
+			var prevPartitionIndex int
+			for i, partition := range member.assignments {
+				if _, exists := partitionsWithADifferentPreviousAssignment[partition]; exists {
+					prevPartitionIndex = i
+					break
+				}
+			}
+
+			if len(member.assignments) > 0 {
+				partition := member.assignments[prevPartitionIndex]
+				sortedPartitions = append(sortedPartitions, partition)
+				delete(unassignedPartitions, partition)
+				if prevPartitionIndex == 0 {
+					member.assignments = member.assignments[1:]
+				} else {
+					member.assignments = append(member.assignments[:prevPartitionIndex], member.assignments[prevPartitionIndex+1:]...)
+				}
+				heap.Fix(&pq, 0)
+			} else {
+				heap.Pop(&pq)
+			}
+		}
+
+		for partition := range unassignedPartitions {
+			sortedPartitions = append(sortedPartitions, partition)
+		}
+	} else {
+		// an ascending sorted set of topic partitions based on how many consumers can potentially use them
+		sortedPartitions = sortPartitionsByPotentialConsumerAssignments(partition2AllPotentialConsumers)
+	}
+	return sortedPartitions
+}
+
+func sortMemberIDsByPartitionAssignments(assignments map[string][]topicPartitionAssignment) []string {
+	// sort the members by the number of partition assignments in ascending order
+	sortedMemberIDs := make([]string, 0, len(assignments))
+	for memberID := range assignments {
+		sortedMemberIDs = append(sortedMemberIDs, memberID)
+	}
+	sort.SliceStable(sortedMemberIDs, func(i, j int) bool {
+		ret := len(assignments[sortedMemberIDs[i]]) - len(assignments[sortedMemberIDs[j]])
+		if ret == 0 {
+			return sortedMemberIDs[i] < sortedMemberIDs[j]
+		}
+		return len(assignments[sortedMemberIDs[i]]) < len(assignments[sortedMemberIDs[j]])
+	})
+	return sortedMemberIDs
+}
+
+func sortPartitionsByPotentialConsumerAssignments(partition2AllPotentialConsumers map[topicPartitionAssignment][]string) []topicPartitionAssignment {
+	// sort the members by the number of partition assignments in descending order
+	sortedPartionIDs := make([]topicPartitionAssignment, len(partition2AllPotentialConsumers))
+	i := 0
+	for partition := range partition2AllPotentialConsumers {
+		sortedPartionIDs[i] = partition
+		i++
+	}
+	sort.Slice(sortedPartionIDs, func(i, j int) bool {
+		if len(partition2AllPotentialConsumers[sortedPartionIDs[i]]) == len(partition2AllPotentialConsumers[sortedPartionIDs[j]]) {
+			ret := strings.Compare(sortedPartionIDs[i].Topic, sortedPartionIDs[j].Topic)
+			if ret == 0 {
+				return sortedPartionIDs[i].Partition < sortedPartionIDs[j].Partition
+			}
+			return ret < 0
+		}
+		return len(partition2AllPotentialConsumers[sortedPartionIDs[i]]) < len(partition2AllPotentialConsumers[sortedPartionIDs[j]])
+	})
+	return sortedPartionIDs
+}
+
+func deepCopyAssignment(assignment map[string][]topicPartitionAssignment) map[string][]topicPartitionAssignment {
+	m := make(map[string][]topicPartitionAssignment, len(assignment))
+	for memberID, subscriptions := range assignment {
+		m[memberID] = append(subscriptions[:0:0], subscriptions...)
+	}
+	return m
+}
+
+func areSubscriptionsIdentical(partition2AllPotentialConsumers map[topicPartitionAssignment][]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) bool {
+	curMembers := make(map[string]int)
+	for _, cur := range partition2AllPotentialConsumers {
+		if len(curMembers) == 0 {
+			for _, curMembersElem := range cur {
+				curMembers[curMembersElem]++
+			}
+			continue
+		}
+
+		if len(curMembers) != len(cur) {
+			return false
+		}
+
+		yMap := make(map[string]int)
+		for _, yElem := range cur {
+			yMap[yElem]++
+		}
+
+		for curMembersMapKey, curMembersMapVal := range curMembers {
+			if yMap[curMembersMapKey] != curMembersMapVal {
+				return false
+			}
+		}
+	}
+
+	curPartitions := make(map[topicPartitionAssignment]int)
+	for _, cur := range consumer2AllPotentialPartitions {
+		if len(curPartitions) == 0 {
+			for _, curPartitionElem := range cur {
+				curPartitions[curPartitionElem]++
+			}
+			continue
+		}
+
+		if len(curPartitions) != len(cur) {
+			return false
+		}
+
+		yMap := make(map[topicPartitionAssignment]int)
+		for _, yElem := range cur {
+			yMap[yElem]++
+		}
+
+		for curMembersMapKey, curMembersMapVal := range curPartitions {
+			if yMap[curMembersMapKey] != curMembersMapVal {
+				return false
+			}
+		}
+	}
+	return true
+}
+
+// We need to process subscriptions' user data with each consumer's reported generation in mind
+// higher generations overwrite lower generations in case of a conflict
+// note that a conflict could exist only if user data is for different generations
+func prepopulateCurrentAssignments(members map[string]ConsumerGroupMemberMetadata) (map[string][]topicPartitionAssignment, map[topicPartitionAssignment]consumerGenerationPair, error) {
+	currentAssignment := make(map[string][]topicPartitionAssignment)
+	prevAssignment := make(map[topicPartitionAssignment]consumerGenerationPair)
+
+	// for each partition we create a sorted map of its consumers by generation
+	sortedPartitionConsumersByGeneration := make(map[topicPartitionAssignment]map[int]string)
+	for memberID, meta := range members {
+		consumerUserData, err := deserializeTopicPartitionAssignment(meta.UserData)
+		if err != nil {
+			return nil, nil, err
+		}
+		for _, partition := range consumerUserData.partitions() {
+			if consumers, exists := sortedPartitionConsumersByGeneration[partition]; exists {
+				if consumerUserData.hasGeneration() {
+					if _, generationExists := consumers[consumerUserData.generation()]; generationExists {
+						// same partition is assigned to two consumers during the same rebalance.
+						// log a warning and skip this record
+						Logger.Printf("Topic %s Partition %d is assigned to multiple consumers following sticky assignment generation %d", partition.Topic, partition.Partition, consumerUserData.generation())
+						continue
+					} else {
+						consumers[consumerUserData.generation()] = memberID
+					}
+				} else {
+					consumers[defaultGeneration] = memberID
+				}
+			} else {
+				generation := defaultGeneration
+				if consumerUserData.hasGeneration() {
+					generation = consumerUserData.generation()
+				}
+				sortedPartitionConsumersByGeneration[partition] = map[int]string{generation: memberID}
+			}
+		}
+	}
+
+	// prevAssignment holds the prior ConsumerGenerationPair (before current) of each partition
+	// current and previous consumers are the last two consumers of each partition in the above sorted map
+	for partition, consumers := range sortedPartitionConsumersByGeneration {
+		// sort consumers by generation in decreasing order
+		var generations []int
+		for generation := range consumers {
+			generations = append(generations, generation)
+		}
+		sort.Sort(sort.Reverse(sort.IntSlice(generations)))
+
+		consumer := consumers[generations[0]]
+		if _, exists := currentAssignment[consumer]; !exists {
+			currentAssignment[consumer] = []topicPartitionAssignment{partition}
+		} else {
+			currentAssignment[consumer] = append(currentAssignment[consumer], partition)
+		}
+
+		// check for previous assignment, if any
+		if len(generations) > 1 {
+			prevAssignment[partition] = consumerGenerationPair{
+				MemberID:   consumers[generations[1]],
+				Generation: generations[1],
+			}
+		}
+	}
+	return currentAssignment, prevAssignment, nil
+}
+
+type consumerGenerationPair struct {
+	MemberID   string
+	Generation int
+}
+
+// consumerPair represents a pair of Kafka consumer ids involved in a partition reassignment.
+type consumerPair struct {
+	SrcMemberID string
+	DstMemberID string
+}
+
+// partitionMovements maintains some data structures to simplify lookup of partition movements among consumers.
+type partitionMovements struct {
+	PartitionMovementsByTopic map[string]map[consumerPair]map[topicPartitionAssignment]bool
+	Movements                 map[topicPartitionAssignment]consumerPair
+}
+
+func (p *partitionMovements) removeMovementRecordOfPartition(partition topicPartitionAssignment) consumerPair {
+	pair := p.Movements[partition]
+	delete(p.Movements, partition)
+
+	partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic]
+	delete(partitionMovementsForThisTopic[pair], partition)
+	if len(partitionMovementsForThisTopic[pair]) == 0 {
+		delete(partitionMovementsForThisTopic, pair)
+	}
+	if len(p.PartitionMovementsByTopic[partition.Topic]) == 0 {
+		delete(p.PartitionMovementsByTopic, partition.Topic)
+	}
+	return pair
+}
+
+func (p *partitionMovements) addPartitionMovementRecord(partition topicPartitionAssignment, pair consumerPair) {
+	p.Movements[partition] = pair
+	if _, exists := p.PartitionMovementsByTopic[partition.Topic]; !exists {
+		p.PartitionMovementsByTopic[partition.Topic] = make(map[consumerPair]map[topicPartitionAssignment]bool)
+	}
+	partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic]
+	if _, exists := partitionMovementsForThisTopic[pair]; !exists {
+		partitionMovementsForThisTopic[pair] = make(map[topicPartitionAssignment]bool)
+	}
+	partitionMovementsForThisTopic[pair][partition] = true
+}
+
+func (p *partitionMovements) movePartition(partition topicPartitionAssignment, oldConsumer, newConsumer string) {
+	pair := consumerPair{
+		SrcMemberID: oldConsumer,
+		DstMemberID: newConsumer,
+	}
+	if _, exists := p.Movements[partition]; exists {
+		// this partition has previously moved
+		existingPair := p.removeMovementRecordOfPartition(partition)
+		if existingPair.DstMemberID != oldConsumer {
+			Logger.Printf("Existing pair DstMemberID %s was not equal to the oldConsumer ID %s", existingPair.DstMemberID, oldConsumer)
+		}
+		if existingPair.SrcMemberID != newConsumer {
+			// the partition is not moving back to its previous consumer
+			p.addPartitionMovementRecord(partition, consumerPair{
+				SrcMemberID: existingPair.SrcMemberID,
+				DstMemberID: newConsumer,
+			})
+		}
+	} else {
+		p.addPartitionMovementRecord(partition, pair)
+	}
+}
+
+func (p *partitionMovements) getTheActualPartitionToBeMoved(partition topicPartitionAssignment, oldConsumer, newConsumer string) topicPartitionAssignment {
+	if _, exists := p.PartitionMovementsByTopic[partition.Topic]; !exists {
+		return partition
+	}
+	if _, exists := p.Movements[partition]; exists {
+		// this partition has previously moved
+		if oldConsumer != p.Movements[partition].DstMemberID {
+			Logger.Printf("Partition movement DstMemberID %s was not equal to the oldConsumer ID %s", p.Movements[partition].DstMemberID, oldConsumer)
+		}
+		oldConsumer = p.Movements[partition].SrcMemberID
+	}
+
+	partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic]
+	reversePair := consumerPair{
+		SrcMemberID: newConsumer,
+		DstMemberID: oldConsumer,
+	}
+	if _, exists := partitionMovementsForThisTopic[reversePair]; !exists {
+		return partition
+	}
+	var reversePairPartition topicPartitionAssignment
+	for otherPartition := range partitionMovementsForThisTopic[reversePair] {
+		reversePairPartition = otherPartition
+	}
+	return reversePairPartition
+}
+
+func (p *partitionMovements) isLinked(src, dst string, pairs []consumerPair, currentPath []string) ([]string, bool) {
+	if src == dst {
+		return currentPath, false
+	}
+	if len(pairs) == 0 {
+		return currentPath, false
+	}
+	for _, pair := range pairs {
+		if src == pair.SrcMemberID && dst == pair.DstMemberID {
+			currentPath = append(currentPath, src, dst)
+			return currentPath, true
+		}
+	}
+
+	for _, pair := range pairs {
+		if pair.SrcMemberID == src {
+			// create a deep copy of the pairs, excluding the current pair
+			reducedSet := make([]consumerPair, len(pairs)-1)
+			i := 0
+			for _, p := range pairs {
+				if p != pair {
+					reducedSet[i] = pair
+					i++
+				}
+			}
+
+			currentPath = append(currentPath, pair.SrcMemberID)
+			return p.isLinked(pair.DstMemberID, dst, reducedSet, currentPath)
+		}
+	}
+	return currentPath, false
+}
+
+func (p *partitionMovements) in(cycle []string, cycles [][]string) bool {
+	superCycle := make([]string, len(cycle)-1)
+	for i := 0; i < len(cycle)-1; i++ {
+		superCycle[i] = cycle[i]
+	}
+	superCycle = append(superCycle, cycle...)
+	for _, foundCycle := range cycles {
+		if len(foundCycle) == len(cycle) && indexOfSubList(superCycle, foundCycle) != -1 {
+			return true
+		}
+	}
+	return false
+}
+
+func (p *partitionMovements) hasCycles(pairs []consumerPair) bool {
+	cycles := make([][]string, 0)
+	for _, pair := range pairs {
+		// create a deep copy of the pairs, excluding the current pair
+		reducedPairs := make([]consumerPair, len(pairs)-1)
+		i := 0
+		for _, p := range pairs {
+			if p != pair {
+				reducedPairs[i] = pair
+				i++
+			}
+		}
+		if path, linked := p.isLinked(pair.DstMemberID, pair.SrcMemberID, reducedPairs, []string{pair.SrcMemberID}); linked {
+			if !p.in(path, cycles) {
+				cycles = append(cycles, path)
+				Logger.Printf("A cycle of length %d was found: %v", len(path)-1, path)
+			}
+		}
+	}
+
+	// for now we want to make sure there is no partition movements of the same topic between a pair of consumers.
+	// the odds of finding a cycle among more than two consumers seem to be very low (according to various randomized
+	// tests with the given sticky algorithm) that it should not worth the added complexity of handling those cases.
+	for _, cycle := range cycles {
+		if len(cycle) == 3 {
+			return true
+		}
+	}
+	return false
+}
+
+func (p *partitionMovements) isSticky() bool {
+	for topic, movements := range p.PartitionMovementsByTopic {
+		movementPairs := make([]consumerPair, len(movements))
+		i := 0
+		for pair := range movements {
+			movementPairs[i] = pair
+			i++
+		}
+		if p.hasCycles(movementPairs) {
+			Logger.Printf("Stickiness is violated for topic %s", topic)
+			Logger.Printf("Partition movements for this topic occurred among the following consumer pairs: %v", movements)
+			return false
+		}
+	}
+	return true
+}
+
+func indexOfSubList(source []string, target []string) int {
+	targetSize := len(target)
+	maxCandidate := len(source) - targetSize
+nextCand:
+	for candidate := 0; candidate <= maxCandidate; candidate++ {
+		j := candidate
+		for i := 0; i < targetSize; i++ {
+			if target[i] != source[j] {
+				// Element mismatch, try next cand
+				continue nextCand
+			}
+			j++
+		}
+		// All elements of candidate matched target
+		return candidate
+	}
+	return -1
+}
+
+type consumerGroupMember struct {
+	id          string
+	assignments []topicPartitionAssignment
+}
+
+// assignmentPriorityQueue is a priority-queue of consumer group members that is sorted
+// in descending order (most assignments to least assignments).
+type assignmentPriorityQueue []*consumerGroupMember
+
+func (pq assignmentPriorityQueue) Len() int { return len(pq) }
+
+func (pq assignmentPriorityQueue) Less(i, j int) bool {
+	// order asssignment priority queue in descending order using assignment-count/member-id
+	if len(pq[i].assignments) == len(pq[j].assignments) {
+		return strings.Compare(pq[i].id, pq[j].id) > 0
+	}
+	return len(pq[i].assignments) > len(pq[j].assignments)
+}
+
+func (pq assignmentPriorityQueue) Swap(i, j int) {
+	pq[i], pq[j] = pq[j], pq[i]
+}
+
+func (pq *assignmentPriorityQueue) Push(x interface{}) {
+	member := x.(*consumerGroupMember)
+	*pq = append(*pq, member)
+}
+
+func (pq *assignmentPriorityQueue) Pop() interface{} {
+	old := *pq
+	n := len(old)
+	member := old[n-1]
+	*pq = old[0 : n-1]
+	return member
+}
diff --git a/vendor/github.com/Shopify/sarama/broker.go b/vendor/github.com/Shopify/sarama/broker.go
index 9c3e5a0..dd01e4e 100644
--- a/vendor/github.com/Shopify/sarama/broker.go
+++ b/vendor/github.com/Shopify/sarama/broker.go
@@ -13,7 +13,7 @@
 	"sync/atomic"
 	"time"
 
-	metrics "github.com/rcrowley/go-metrics"
+	"github.com/rcrowley/go-metrics"
 )
 
 // Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe.
@@ -40,6 +40,7 @@
 	outgoingByteRate       metrics.Meter
 	responseRate           metrics.Meter
 	responseSize           metrics.Histogram
+	requestsInFlight       metrics.Counter
 	brokerIncomingByteRate metrics.Meter
 	brokerRequestRate      metrics.Meter
 	brokerRequestSize      metrics.Histogram
@@ -47,6 +48,7 @@
 	brokerOutgoingByteRate metrics.Meter
 	brokerResponseRate     metrics.Meter
 	brokerResponseSize     metrics.Histogram
+	brokerRequestsInFlight metrics.Counter
 
 	kerberosAuthenticator GSSAPIKerberosAuth
 }
@@ -71,7 +73,7 @@
 	// server negotiate SASL by wrapping tokens with Kafka protocol headers.
 	SASLHandshakeV1 = int16(1)
 	// SASLExtKeyAuth is the reserved extension key name sent as part of the
-	// SASL/OAUTHBEARER intial client response
+	// SASL/OAUTHBEARER initial client response
 	SASLExtKeyAuth = "auth"
 )
 
@@ -117,6 +119,7 @@
 type responsePromise struct {
 	requestTime   time.Time
 	correlationID int32
+	headerVersion int16
 	packets       chan []byte
 	errors        chan error
 }
@@ -151,27 +154,19 @@
 	go withRecover(func() {
 		defer b.lock.Unlock()
 
-		dialer := net.Dialer{
-			Timeout:   conf.Net.DialTimeout,
-			KeepAlive: conf.Net.KeepAlive,
-			LocalAddr: conf.Net.LocalAddr,
-		}
-
-		if conf.Net.TLS.Enable {
-			b.conn, b.connErr = tls.DialWithDialer(&dialer, "tcp", b.addr, conf.Net.TLS.Config)
-		} else if conf.Net.Proxy.Enable {
-			b.conn, b.connErr = conf.Net.Proxy.Dialer.Dial("tcp", b.addr)
-		} else {
-			b.conn, b.connErr = dialer.Dial("tcp", b.addr)
-		}
+		dialer := conf.getDialer()
+		b.conn, b.connErr = dialer.Dial("tcp", b.addr)
 		if b.connErr != nil {
 			Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr)
 			b.conn = nil
 			atomic.StoreInt32(&b.opened, 0)
 			return
 		}
-		b.conn = newBufConn(b.conn)
+		if conf.Net.TLS.Enable {
+			b.conn = tls.Client(b.conn, validServerNameTLS(b.addr, conf.Net.TLS.Config))
+		}
 
+		b.conn = newBufConn(b.conn)
 		b.conf = conf
 
 		// Create or reuse the global metrics shared between brokers
@@ -182,6 +177,7 @@
 		b.outgoingByteRate = metrics.GetOrRegisterMeter("outgoing-byte-rate", conf.MetricRegistry)
 		b.responseRate = metrics.GetOrRegisterMeter("response-rate", conf.MetricRegistry)
 		b.responseSize = getOrRegisterHistogram("response-size", conf.MetricRegistry)
+		b.requestsInFlight = metrics.GetOrRegisterCounter("requests-in-flight", conf.MetricRegistry)
 		// Do not gather metrics for seeded broker (only used during bootstrap) because they share
 		// the same id (-1) and are already exposed through the global metrics above
 		if b.id >= 0 {
@@ -189,7 +185,6 @@
 		}
 
 		if conf.Net.SASL.Enable {
-
 			b.connErr = b.authenticateViaSASL()
 
 			if b.connErr != nil {
@@ -228,7 +223,7 @@
 	return b.conn != nil, b.connErr
 }
 
-//Close closes the broker resources
+// Close closes the broker resources
 func (b *Broker) Close() error {
 	b.lock.Lock()
 	defer b.lock.Unlock()
@@ -281,12 +276,11 @@
 	return *b.rack
 }
 
-//GetMetadata send a metadata request and returns a metadata response or error
+// GetMetadata send a metadata request and returns a metadata response or error
 func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) {
 	response := new(MetadataResponse)
 
 	err := b.sendAndReceive(request, response)
-
 	if err != nil {
 		return nil, err
 	}
@@ -294,12 +288,11 @@
 	return response, nil
 }
 
-//GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error
+// GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error
 func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
 	response := new(ConsumerMetadataResponse)
 
 	err := b.sendAndReceive(request, response)
-
 	if err != nil {
 		return nil, err
 	}
@@ -307,12 +300,11 @@
 	return response, nil
 }
 
-//FindCoordinator sends a find coordinate request and returns a response or error
+// FindCoordinator sends a find coordinate request and returns a response or error
 func (b *Broker) FindCoordinator(request *FindCoordinatorRequest) (*FindCoordinatorResponse, error) {
 	response := new(FindCoordinatorResponse)
 
 	err := b.sendAndReceive(request, response)
-
 	if err != nil {
 		return nil, err
 	}
@@ -320,12 +312,11 @@
 	return response, nil
 }
 
-//GetAvailableOffsets return an offset response or error
+// GetAvailableOffsets return an offset response or error
 func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
 	response := new(OffsetResponse)
 
 	err := b.sendAndReceive(request, response)
-
 	if err != nil {
 		return nil, err
 	}
@@ -333,7 +324,7 @@
 	return response, nil
 }
 
-//Produce returns a produce response or error
+// Produce returns a produce response or error
 func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
 	var (
 		response *ProduceResponse
@@ -354,7 +345,7 @@
 	return response, nil
 }
 
-//Fetch returns a FetchResponse or error
+// Fetch returns a FetchResponse or error
 func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
 	response := new(FetchResponse)
 
@@ -366,7 +357,7 @@
 	return response, nil
 }
 
-//CommitOffset return an Offset commit reponse or error
+// CommitOffset return an Offset commit response or error
 func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
 	response := new(OffsetCommitResponse)
 
@@ -378,9 +369,10 @@
 	return response, nil
 }
 
-//FetchOffset returns an offset fetch response or error
+// FetchOffset returns an offset fetch response or error
 func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
 	response := new(OffsetFetchResponse)
+	response.Version = request.Version // needed to handle the two header versions
 
 	err := b.sendAndReceive(request, response)
 	if err != nil {
@@ -390,7 +382,7 @@
 	return response, nil
 }
 
-//JoinGroup returns a join group response or error
+// JoinGroup returns a join group response or error
 func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) {
 	response := new(JoinGroupResponse)
 
@@ -402,7 +394,7 @@
 	return response, nil
 }
 
-//SyncGroup returns a sync group response or error
+// SyncGroup returns a sync group response or error
 func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) {
 	response := new(SyncGroupResponse)
 
@@ -414,7 +406,7 @@
 	return response, nil
 }
 
-//LeaveGroup return a leave group response or error
+// LeaveGroup return a leave group response or error
 func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) {
 	response := new(LeaveGroupResponse)
 
@@ -426,7 +418,7 @@
 	return response, nil
 }
 
-//Heartbeat returns a heartbeat response or error
+// Heartbeat returns a heartbeat response or error
 func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) {
 	response := new(HeartbeatResponse)
 
@@ -438,7 +430,7 @@
 	return response, nil
 }
 
-//ListGroups return a list group response or error
+// ListGroups return a list group response or error
 func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) {
 	response := new(ListGroupsResponse)
 
@@ -450,7 +442,7 @@
 	return response, nil
 }
 
-//DescribeGroups return describe group response or error
+// DescribeGroups return describe group response or error
 func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) {
 	response := new(DescribeGroupsResponse)
 
@@ -462,7 +454,7 @@
 	return response, nil
 }
 
-//ApiVersions return api version response or error
+// ApiVersions return api version response or error
 func (b *Broker) ApiVersions(request *ApiVersionsRequest) (*ApiVersionsResponse, error) {
 	response := new(ApiVersionsResponse)
 
@@ -474,7 +466,7 @@
 	return response, nil
 }
 
-//CreateTopics send a create topic request and returns create topic response
+// CreateTopics send a create topic request and returns create topic response
 func (b *Broker) CreateTopics(request *CreateTopicsRequest) (*CreateTopicsResponse, error) {
 	response := new(CreateTopicsResponse)
 
@@ -486,7 +478,7 @@
 	return response, nil
 }
 
-//DeleteTopics sends a delete topic request and returns delete topic response
+// DeleteTopics sends a delete topic request and returns delete topic response
 func (b *Broker) DeleteTopics(request *DeleteTopicsRequest) (*DeleteTopicsResponse, error) {
 	response := new(DeleteTopicsResponse)
 
@@ -498,8 +490,8 @@
 	return response, nil
 }
 
-//CreatePartitions sends a create partition request and returns create
-//partitions response or error
+// CreatePartitions sends a create partition request and returns create
+// partitions response or error
 func (b *Broker) CreatePartitions(request *CreatePartitionsRequest) (*CreatePartitionsResponse, error) {
 	response := new(CreatePartitionsResponse)
 
@@ -511,8 +503,34 @@
 	return response, nil
 }
 
-//DeleteRecords send a request to delete records and return delete record
-//response or error
+// AlterPartitionReassignments sends a alter partition reassignments request and
+// returns alter partition reassignments response
+func (b *Broker) AlterPartitionReassignments(request *AlterPartitionReassignmentsRequest) (*AlterPartitionReassignmentsResponse, error) {
+	response := new(AlterPartitionReassignmentsResponse)
+
+	err := b.sendAndReceive(request, response)
+	if err != nil {
+		return nil, err
+	}
+
+	return response, nil
+}
+
+// ListPartitionReassignments sends a list partition reassignments request and
+// returns list partition reassignments response
+func (b *Broker) ListPartitionReassignments(request *ListPartitionReassignmentsRequest) (*ListPartitionReassignmentsResponse, error) {
+	response := new(ListPartitionReassignmentsResponse)
+
+	err := b.sendAndReceive(request, response)
+	if err != nil {
+		return nil, err
+	}
+
+	return response, nil
+}
+
+// DeleteRecords send a request to delete records and return delete record
+// response or error
 func (b *Broker) DeleteRecords(request *DeleteRecordsRequest) (*DeleteRecordsResponse, error) {
 	response := new(DeleteRecordsResponse)
 
@@ -524,7 +542,7 @@
 	return response, nil
 }
 
-//DescribeAcls sends a describe acl request and returns a response or error
+// DescribeAcls sends a describe acl request and returns a response or error
 func (b *Broker) DescribeAcls(request *DescribeAclsRequest) (*DescribeAclsResponse, error) {
 	response := new(DescribeAclsResponse)
 
@@ -536,7 +554,7 @@
 	return response, nil
 }
 
-//CreateAcls sends a create acl request and returns a response or error
+// CreateAcls sends a create acl request and returns a response or error
 func (b *Broker) CreateAcls(request *CreateAclsRequest) (*CreateAclsResponse, error) {
 	response := new(CreateAclsResponse)
 
@@ -548,7 +566,7 @@
 	return response, nil
 }
 
-//DeleteAcls sends a delete acl request and returns a response or error
+// DeleteAcls sends a delete acl request and returns a response or error
 func (b *Broker) DeleteAcls(request *DeleteAclsRequest) (*DeleteAclsResponse, error) {
 	response := new(DeleteAclsResponse)
 
@@ -560,7 +578,7 @@
 	return response, nil
 }
 
-//InitProducerID sends an init producer request and returns a response or error
+// InitProducerID sends an init producer request and returns a response or error
 func (b *Broker) InitProducerID(request *InitProducerIDRequest) (*InitProducerIDResponse, error) {
 	response := new(InitProducerIDResponse)
 
@@ -572,8 +590,8 @@
 	return response, nil
 }
 
-//AddPartitionsToTxn send a request to add partition to txn and returns
-//a response or error
+// AddPartitionsToTxn send a request to add partition to txn and returns
+// a response or error
 func (b *Broker) AddPartitionsToTxn(request *AddPartitionsToTxnRequest) (*AddPartitionsToTxnResponse, error) {
 	response := new(AddPartitionsToTxnResponse)
 
@@ -585,8 +603,8 @@
 	return response, nil
 }
 
-//AddOffsetsToTxn sends a request to add offsets to txn and returns a response
-//or error
+// AddOffsetsToTxn sends a request to add offsets to txn and returns a response
+// or error
 func (b *Broker) AddOffsetsToTxn(request *AddOffsetsToTxnRequest) (*AddOffsetsToTxnResponse, error) {
 	response := new(AddOffsetsToTxnResponse)
 
@@ -598,7 +616,7 @@
 	return response, nil
 }
 
-//EndTxn sends a request to end txn and returns a response or error
+// EndTxn sends a request to end txn and returns a response or error
 func (b *Broker) EndTxn(request *EndTxnRequest) (*EndTxnResponse, error) {
 	response := new(EndTxnResponse)
 
@@ -610,8 +628,8 @@
 	return response, nil
 }
 
-//TxnOffsetCommit sends a request to commit transaction offsets and returns
-//a response or error
+// TxnOffsetCommit sends a request to commit transaction offsets and returns
+// a response or error
 func (b *Broker) TxnOffsetCommit(request *TxnOffsetCommitRequest) (*TxnOffsetCommitResponse, error) {
 	response := new(TxnOffsetCommitResponse)
 
@@ -623,8 +641,8 @@
 	return response, nil
 }
 
-//DescribeConfigs sends a request to describe config and returns a response or
-//error
+// DescribeConfigs sends a request to describe config and returns a response or
+// error
 func (b *Broker) DescribeConfigs(request *DescribeConfigsRequest) (*DescribeConfigsResponse, error) {
 	response := new(DescribeConfigsResponse)
 
@@ -636,7 +654,7 @@
 	return response, nil
 }
 
-//AlterConfigs sends a request to alter config and return a response or error
+// AlterConfigs sends a request to alter config and return a response or error
 func (b *Broker) AlterConfigs(request *AlterConfigsRequest) (*AlterConfigsResponse, error) {
 	response := new(AlterConfigsResponse)
 
@@ -648,7 +666,19 @@
 	return response, nil
 }
 
-//DeleteGroups sends a request to delete groups and returns a response or error
+// IncrementalAlterConfigs sends a request to incremental alter config and return a response or error
+func (b *Broker) IncrementalAlterConfigs(request *IncrementalAlterConfigsRequest) (*IncrementalAlterConfigsResponse, error) {
+	response := new(IncrementalAlterConfigsResponse)
+
+	err := b.sendAndReceive(request, response)
+	if err != nil {
+		return nil, err
+	}
+
+	return response, nil
+}
+
+// DeleteGroups sends a request to delete groups and returns a response or error
 func (b *Broker) DeleteGroups(request *DeleteGroupsRequest) (*DeleteGroupsResponse, error) {
 	response := new(DeleteGroupsResponse)
 
@@ -659,7 +689,62 @@
 	return response, nil
 }
 
-func (b *Broker) send(rb protocolBody, promiseResponse bool) (*responsePromise, error) {
+// DescribeLogDirs sends a request to get the broker's log dir paths and sizes
+func (b *Broker) DescribeLogDirs(request *DescribeLogDirsRequest) (*DescribeLogDirsResponse, error) {
+	response := new(DescribeLogDirsResponse)
+
+	err := b.sendAndReceive(request, response)
+	if err != nil {
+		return nil, err
+	}
+
+	return response, nil
+}
+
+// DescribeUserScramCredentials sends a request to get SCRAM users
+func (b *Broker) DescribeUserScramCredentials(req *DescribeUserScramCredentialsRequest) (*DescribeUserScramCredentialsResponse, error) {
+	res := new(DescribeUserScramCredentialsResponse)
+
+	err := b.sendAndReceive(req, res)
+	if err != nil {
+		return nil, err
+	}
+
+	return res, err
+}
+
+func (b *Broker) AlterUserScramCredentials(req *AlterUserScramCredentialsRequest) (*AlterUserScramCredentialsResponse, error) {
+	res := new(AlterUserScramCredentialsResponse)
+
+	err := b.sendAndReceive(req, res)
+	if err != nil {
+		return nil, err
+	}
+
+	return res, nil
+}
+
+// readFull ensures the conn ReadDeadline has been setup before making a
+// call to io.ReadFull
+func (b *Broker) readFull(buf []byte) (n int, err error) {
+	if err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout)); err != nil {
+		return 0, err
+	}
+
+	return io.ReadFull(b.conn, buf)
+}
+
+// write  ensures the conn WriteDeadline has been setup before making a
+// call to conn.Write
+func (b *Broker) write(buf []byte) (n int, err error) {
+	if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil {
+		return 0, err
+	}
+
+	return b.conn.Write(buf)
+}
+
+func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersion int16) (*responsePromise, error) {
 	b.lock.Lock()
 	defer b.lock.Unlock()
 
@@ -680,33 +765,36 @@
 		return nil, err
 	}
 
-	err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
-	if err != nil {
-		return nil, err
-	}
-
 	requestTime := time.Now()
-	bytes, err := b.conn.Write(buf)
-	b.updateOutgoingCommunicationMetrics(bytes) //TODO: should it be after error check
+	// Will be decremented in responseReceiver (except error or request with NoResponse)
+	b.addRequestInFlightMetrics(1)
+	bytes, err := b.write(buf)
+	b.updateOutgoingCommunicationMetrics(bytes)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		return nil, err
 	}
 	b.correlationID++
 
 	if !promiseResponse {
 		// Record request latency without the response
-		b.updateRequestLatencyMetrics(time.Since(requestTime))
+		b.updateRequestLatencyAndInFlightMetrics(time.Since(requestTime))
 		return nil, nil
 	}
 
-	promise := responsePromise{requestTime, req.correlationID, make(chan []byte), make(chan error)}
+	promise := responsePromise{requestTime, req.correlationID, responseHeaderVersion, make(chan []byte), make(chan error)}
 	b.responses <- promise
 
 	return &promise, nil
 }
 
-func (b *Broker) sendAndReceive(req protocolBody, res versionedDecoder) error {
-	promise, err := b.send(req, res != nil)
+func (b *Broker) sendAndReceive(req protocolBody, res protocolBody) error {
+	responseHeaderVersion := int16(-1)
+	if res != nil {
+		responseHeaderVersion = res.headerVersion()
+	}
+
+	promise, err := b.send(req, res != nil, responseHeaderVersion)
 	if err != nil {
 		return err
 	}
@@ -760,7 +848,7 @@
 		return err
 	}
 
-	port, err := strconv.Atoi(portstr)
+	port, err := strconv.ParseInt(portstr, 10, 32)
 	if err != nil {
 		return err
 	}
@@ -786,22 +874,20 @@
 
 func (b *Broker) responseReceiver() {
 	var dead error
-	header := make([]byte, 8)
 
 	for response := range b.responses {
 		if dead != nil {
+			// This was previously incremented in send() and
+			// we are not calling updateIncomingCommunicationMetrics()
+			b.addRequestInFlightMetrics(-1)
 			response.errors <- dead
 			continue
 		}
 
-		err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout))
-		if err != nil {
-			dead = err
-			response.errors <- err
-			continue
-		}
+		headerLength := getHeaderLength(response.headerVersion)
+		header := make([]byte, headerLength)
 
-		bytesReadHeader, err := io.ReadFull(b.conn, header)
+		bytesReadHeader, err := b.readFull(header)
 		requestLatency := time.Since(response.requestTime)
 		if err != nil {
 			b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
@@ -811,7 +897,7 @@
 		}
 
 		decodedHeader := responseHeader{}
-		err = decode(header, &decodedHeader)
+		err = versionedDecode(header, &decodedHeader, response.headerVersion)
 		if err != nil {
 			b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
 			dead = err
@@ -827,8 +913,8 @@
 			continue
 		}
 
-		buf := make([]byte, decodedHeader.length-4)
-		bytesReadBody, err := io.ReadFull(b.conn, buf)
+		buf := make([]byte, decodedHeader.length-int32(headerLength)+4)
+		bytesReadBody, err := b.readFull(buf)
 		b.updateIncomingCommunicationMetrics(bytesReadHeader+bytesReadBody, requestLatency)
 		if err != nil {
 			dead = err
@@ -841,6 +927,15 @@
 	close(b.done)
 }
 
+func getHeaderLength(headerVersion int16) int8 {
+	if headerVersion < 1 {
+		return 8
+	} else {
+		// header contains additional tagged field length (0), we don't support actual tags yet.
+		return 9
+	}
+}
+
 func (b *Broker) authenticateViaSASL() error {
 	switch b.conf.Net.SASL.Mechanism {
 	case SASLTypeOAuth:
@@ -871,31 +966,31 @@
 		return err
 	}
 
-	err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
-	if err != nil {
-		return err
-	}
-
 	requestTime := time.Now()
-	bytes, err := b.conn.Write(buf)
+	// Will be decremented in updateIncomingCommunicationMetrics (except error)
+	b.addRequestInFlightMetrics(1)
+	bytes, err := b.write(buf)
 	b.updateOutgoingCommunicationMetrics(bytes)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		Logger.Printf("Failed to send SASL handshake %s: %s\n", b.addr, err.Error())
 		return err
 	}
 	b.correlationID++
-	//wait for the response
+
 	header := make([]byte, 8) // response header
-	_, err = io.ReadFull(b.conn, header)
+	_, err = b.readFull(header)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		Logger.Printf("Failed to read SASL handshake header : %s\n", err.Error())
 		return err
 	}
 
 	length := binary.BigEndian.Uint32(header[:4])
 	payload := make([]byte, length-4)
-	n, err := io.ReadFull(b.conn, payload)
+	n, err := b.readFull(payload)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		Logger.Printf("Failed to read SASL handshake payload : %s\n", err.Error())
 		return err
 	}
@@ -943,10 +1038,9 @@
 // When credentials are invalid, Kafka replies with a SaslAuthenticate response
 // containing an error code and message detailing the authentication failure.
 func (b *Broker) sendAndReceiveSASLPlainAuth() error {
-	// default to V0 to allow for backward compatability when SASL is enabled
+	// default to V0 to allow for backward compatibility when SASL is enabled
 	// but not the handshake
 	if b.conf.Net.SASL.Handshake {
-
 		handshakeErr := b.sendAndReceiveSASLHandshake(SASLTypePlaintext, b.conf.Net.SASL.Version)
 		if handshakeErr != nil {
 			Logger.Printf("Error while performing SASL handshake %s\n", b.addr)
@@ -962,28 +1056,24 @@
 
 // sendAndReceiveV0SASLPlainAuth flows the v0 sasl auth NOT wrapped in the kafka protocol
 func (b *Broker) sendAndReceiveV0SASLPlainAuth() error {
-
-	length := 1 + len(b.conf.Net.SASL.User) + 1 + len(b.conf.Net.SASL.Password)
-	authBytes := make([]byte, length+4) //4 byte length header + auth data
+	length := len(b.conf.Net.SASL.AuthIdentity) + 1 + len(b.conf.Net.SASL.User) + 1 + len(b.conf.Net.SASL.Password)
+	authBytes := make([]byte, length+4) // 4 byte length header + auth data
 	binary.BigEndian.PutUint32(authBytes, uint32(length))
-	copy(authBytes[4:], []byte("\x00"+b.conf.Net.SASL.User+"\x00"+b.conf.Net.SASL.Password))
-
-	err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
-	if err != nil {
-		Logger.Printf("Failed to set write deadline when doing SASL auth with broker %s: %s\n", b.addr, err.Error())
-		return err
-	}
+	copy(authBytes[4:], b.conf.Net.SASL.AuthIdentity+"\x00"+b.conf.Net.SASL.User+"\x00"+b.conf.Net.SASL.Password)
 
 	requestTime := time.Now()
-	bytesWritten, err := b.conn.Write(authBytes)
+	// Will be decremented in updateIncomingCommunicationMetrics (except error)
+	b.addRequestInFlightMetrics(1)
+	bytesWritten, err := b.write(authBytes)
 	b.updateOutgoingCommunicationMetrics(bytesWritten)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
 		return err
 	}
 
 	header := make([]byte, 4)
-	n, err := io.ReadFull(b.conn, header)
+	n, err := b.readFull(header)
 	b.updateIncomingCommunicationMetrics(n, time.Since(requestTime))
 	// If the credentials are valid, we would get a 4 byte response filled with null characters.
 	// Otherwise, the broker closes the connection and we get an EOF
@@ -1002,11 +1092,13 @@
 
 	requestTime := time.Now()
 
+	// Will be decremented in updateIncomingCommunicationMetrics (except error)
+	b.addRequestInFlightMetrics(1)
 	bytesWritten, err := b.sendSASLPlainAuthClientResponse(correlationID)
-
 	b.updateOutgoingCommunicationMetrics(bytesWritten)
 
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
 		return err
 	}
@@ -1059,16 +1151,18 @@
 // if the broker responds with a challenge, in which case the token is
 // rejected.
 func (b *Broker) sendClientMessage(message []byte) (bool, error) {
-
 	requestTime := time.Now()
+	// Will be decremented in updateIncomingCommunicationMetrics (except error)
+	b.addRequestInFlightMetrics(1)
 	correlationID := b.correlationID
 
 	bytesWritten, err := b.sendSASLOAuthBearerClientMessage(message, correlationID)
+	b.updateOutgoingCommunicationMetrics(bytesWritten)
 	if err != nil {
+		b.addRequestInFlightMetrics(-1)
 		return false, err
 	}
 
-	b.updateOutgoingCommunicationMetrics(bytesWritten)
 	b.correlationID++
 
 	res := &SaslAuthenticateResponse{}
@@ -1099,22 +1193,25 @@
 	msg, err := scramClient.Step("")
 	if err != nil {
 		return fmt.Errorf("failed to advance the SCRAM exchange: %s", err.Error())
-
 	}
 
 	for !scramClient.Done() {
 		requestTime := time.Now()
+		// Will be decremented in updateIncomingCommunicationMetrics (except error)
+		b.addRequestInFlightMetrics(1)
 		correlationID := b.correlationID
 		bytesWritten, err := b.sendSaslAuthenticateRequest(correlationID, []byte(msg))
+		b.updateOutgoingCommunicationMetrics(bytesWritten)
 		if err != nil {
+			b.addRequestInFlightMetrics(-1)
 			Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
 			return err
 		}
 
-		b.updateOutgoingCommunicationMetrics(bytesWritten)
 		b.correlationID++
 		challenge, err := b.receiveSaslAuthenticateResponse(correlationID)
 		if err != nil {
+			b.addRequestInFlightMetrics(-1)
 			Logger.Printf("Failed to read response while authenticating with SASL to broker %s: %s\n", b.addr, err.Error())
 			return err
 		}
@@ -1139,22 +1236,18 @@
 		return 0, err
 	}
 
-	if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil {
-		return 0, err
-	}
-
-	return b.conn.Write(buf)
+	return b.write(buf)
 }
 
 func (b *Broker) receiveSaslAuthenticateResponse(correlationID int32) ([]byte, error) {
 	buf := make([]byte, responseLengthSize+correlationIDSize)
-	_, err := io.ReadFull(b.conn, buf)
+	_, err := b.readFull(buf)
 	if err != nil {
 		return nil, err
 	}
 
 	header := responseHeader{}
-	err = decode(buf, &header)
+	err = versionedDecode(buf, &header, 0)
 	if err != nil {
 		return nil, err
 	}
@@ -1164,7 +1257,7 @@
 	}
 
 	buf = make([]byte, header.length-correlationIDSize)
-	_, err = io.ReadFull(b.conn, buf)
+	_, err = b.readFull(buf)
 	if err != nil {
 		return nil, err
 	}
@@ -1211,7 +1304,7 @@
 }
 
 func (b *Broker) sendSASLPlainAuthClientResponse(correlationID int32) (int, error) {
-	authBytes := []byte("\x00" + b.conf.Net.SASL.User + "\x00" + b.conf.Net.SASL.Password)
+	authBytes := []byte(b.conf.Net.SASL.AuthIdentity + "\x00" + b.conf.Net.SASL.User + "\x00" + b.conf.Net.SASL.Password)
 	rb := &SaslAuthenticateRequest{authBytes}
 	req := &request{correlationID: correlationID, clientID: b.conf.ClientID, body: rb}
 	buf, err := encode(req, b.conf.MetricRegistry)
@@ -1219,16 +1312,10 @@
 		return 0, err
 	}
 
-	err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
-	if err != nil {
-		Logger.Printf("Failed to set write deadline when doing SASL auth with broker %s: %s\n", b.addr, err.Error())
-		return 0, err
-	}
-	return b.conn.Write(buf)
+	return b.write(buf)
 }
 
 func (b *Broker) sendSASLOAuthBearerClientMessage(initialResp []byte, correlationID int32) (int, error) {
-
 	rb := &SaslAuthenticateRequest{initialResp}
 
 	req := &request{correlationID: correlationID, clientID: b.conf.ClientID, body: rb}
@@ -1238,25 +1325,18 @@
 		return 0, err
 	}
 
-	if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil {
-		return 0, err
-	}
-
-	return b.conn.Write(buf)
+	return b.write(buf)
 }
 
 func (b *Broker) receiveSASLServerResponse(res *SaslAuthenticateResponse, correlationID int32) (int, error) {
-
 	buf := make([]byte, responseLengthSize+correlationIDSize)
-
-	bytesRead, err := io.ReadFull(b.conn, buf)
+	bytesRead, err := b.readFull(buf)
 	if err != nil {
 		return bytesRead, err
 	}
 
 	header := responseHeader{}
-
-	err = decode(buf, &header)
+	err = versionedDecode(buf, &header, 0)
 	if err != nil {
 		return bytesRead, err
 	}
@@ -1266,8 +1346,7 @@
 	}
 
 	buf = make([]byte, header.length-correlationIDSize)
-
-	c, err := io.ReadFull(b.conn, buf)
+	c, err := b.readFull(buf)
 	bytesRead += c
 	if err != nil {
 		return bytesRead, err
@@ -1285,7 +1364,7 @@
 }
 
 func (b *Broker) updateIncomingCommunicationMetrics(bytes int, requestLatency time.Duration) {
-	b.updateRequestLatencyMetrics(requestLatency)
+	b.updateRequestLatencyAndInFlightMetrics(requestLatency)
 	b.responseRate.Mark(1)
 
 	if b.brokerResponseRate != nil {
@@ -1304,7 +1383,7 @@
 	}
 }
 
-func (b *Broker) updateRequestLatencyMetrics(requestLatency time.Duration) {
+func (b *Broker) updateRequestLatencyAndInFlightMetrics(requestLatency time.Duration) {
 	requestLatencyInMs := int64(requestLatency / time.Millisecond)
 	b.requestLatency.Update(requestLatencyInMs)
 
@@ -1312,6 +1391,14 @@
 		b.brokerRequestLatency.Update(requestLatencyInMs)
 	}
 
+	b.addRequestInFlightMetrics(-1)
+}
+
+func (b *Broker) addRequestInFlightMetrics(i int64) {
+	b.requestsInFlight.Inc(i)
+	if b.brokerRequestsInFlight != nil {
+		b.brokerRequestsInFlight.Inc(i)
+	}
 }
 
 func (b *Broker) updateOutgoingCommunicationMetrics(bytes int) {
@@ -1330,7 +1417,6 @@
 	if b.brokerRequestSize != nil {
 		b.brokerRequestSize.Update(requestSize)
 	}
-
 }
 
 func (b *Broker) registerMetrics() {
@@ -1341,12 +1427,14 @@
 	b.brokerOutgoingByteRate = b.registerMeter("outgoing-byte-rate")
 	b.brokerResponseRate = b.registerMeter("response-rate")
 	b.brokerResponseSize = b.registerHistogram("response-size")
+	b.brokerRequestsInFlight = b.registerCounter("requests-in-flight")
 }
 
 func (b *Broker) unregisterMetrics() {
 	for _, name := range b.registeredMetrics {
 		b.conf.MetricRegistry.Unregister(name)
 	}
+	b.registeredMetrics = nil
 }
 
 func (b *Broker) registerMeter(name string) metrics.Meter {
@@ -1360,3 +1448,28 @@
 	b.registeredMetrics = append(b.registeredMetrics, nameForBroker)
 	return getOrRegisterHistogram(nameForBroker, b.conf.MetricRegistry)
 }
+
+func (b *Broker) registerCounter(name string) metrics.Counter {
+	nameForBroker := getMetricNameForBroker(name, b)
+	b.registeredMetrics = append(b.registeredMetrics, nameForBroker)
+	return metrics.GetOrRegisterCounter(nameForBroker, b.conf.MetricRegistry)
+}
+
+func validServerNameTLS(addr string, cfg *tls.Config) *tls.Config {
+	if cfg == nil {
+		cfg = &tls.Config{
+			MinVersion: tls.VersionTLS12,
+		}
+	}
+	if cfg.ServerName != "" {
+		return cfg
+	}
+
+	c := cfg.Clone()
+	sn, _, err := net.SplitHostPort(addr)
+	if err != nil {
+		Logger.Println(fmt.Errorf("failed to get ServerName from addr %w", err))
+	}
+	c.ServerName = sn
+	return c
+}
diff --git a/vendor/github.com/Shopify/sarama/client.go b/vendor/github.com/Shopify/sarama/client.go
index c4c54b2..c0918ba 100644
--- a/vendor/github.com/Shopify/sarama/client.go
+++ b/vendor/github.com/Shopify/sarama/client.go
@@ -17,12 +17,21 @@
 	// altered after it has been created.
 	Config() *Config
 
-	// Controller returns the cluster controller broker. Requires Kafka 0.10 or higher.
+	// Controller returns the cluster controller broker. It will return a
+	// locally cached value if it's available. You can call RefreshController
+	// to update the cached value. Requires Kafka 0.10 or higher.
 	Controller() (*Broker, error)
 
+	// RefreshController retrieves the cluster controller from fresh metadata
+	// and stores it in the local cache. Requires Kafka 0.10 or higher.
+	RefreshController() (*Broker, error)
+
 	// Brokers returns the current set of active brokers as retrieved from cluster metadata.
 	Brokers() []*Broker
 
+	// Broker returns the active Broker if available for the broker ID.
+	Broker(brokerID int32) (*Broker, error)
+
 	// Topics returns the set of available topics as retrieved from cluster metadata.
 	Topics() ([]string, error)
 
@@ -50,6 +59,11 @@
 	// partition. Offline replicas are replicas which are offline
 	OfflineReplicas(topic string, partitionID int32) ([]int32, error)
 
+	// RefreshBrokers takes a list of addresses to be used as seed brokers.
+	// Existing broker connections are closed and the updated list of seed brokers
+	// will be used for the next metadata fetch.
+	RefreshBrokers(addrs []string) error
+
 	// RefreshMetadata takes a list of topics and queries the cluster to refresh the
 	// available metadata for those topics. If no topics are provided, it will refresh
 	// metadata for all topics.
@@ -149,10 +163,7 @@
 		coordinators:            make(map[string]int32),
 	}
 
-	random := rand.New(rand.NewSource(time.Now().UnixNano()))
-	for _, index := range random.Perm(len(addrs)) {
-		client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index]))
-	}
+	client.randomizeSeedBrokers(addrs)
 
 	if conf.Metadata.Full {
 		// do an initial fetch of all cluster metadata by specifying an empty list of topics
@@ -190,10 +201,20 @@
 	return brokers
 }
 
+func (client *client) Broker(brokerID int32) (*Broker, error) {
+	client.lock.RLock()
+	defer client.lock.RUnlock()
+	broker, ok := client.brokers[brokerID]
+	if !ok {
+		return nil, ErrBrokerNotFound
+	}
+	_ = broker.Open(client.conf)
+	return broker, nil
+}
+
 func (client *client) InitProducerID() (*InitProducerIDResponse, error) {
 	var err error
 	for broker := client.any(); broker != nil; broker = client.any() {
-
 		req := &InitProducerIDRequest{}
 
 		response, err := broker.InitProducerID(req)
@@ -242,6 +263,9 @@
 }
 
 func (client *client) Closed() bool {
+	client.lock.RLock()
+	defer client.lock.RUnlock()
+
 	return client.brokers == nil
 }
 
@@ -421,6 +445,27 @@
 	return leader, err
 }
 
+func (client *client) RefreshBrokers(addrs []string) error {
+	if client.Closed() {
+		return ErrClosedClient
+	}
+
+	client.lock.Lock()
+	defer client.lock.Unlock()
+
+	for _, broker := range client.brokers {
+		_ = broker.Close()
+		delete(client.brokers, broker.ID())
+	}
+
+	client.seedBrokers = nil
+	client.deadSeeds = nil
+
+	client.randomizeSeedBrokers(addrs)
+
+	return nil
+}
+
 func (client *client) RefreshMetadata(topics ...string) error {
 	if client.Closed() {
 		return ErrClosedClient
@@ -430,7 +475,7 @@
 	// error. This handles the case by returning an error instead of sending it
 	// off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
 	for _, topic := range topics {
-		if len(topic) == 0 {
+		if topic == "" {
 			return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
 		}
 	}
@@ -448,7 +493,6 @@
 	}
 
 	offset, err := client.getOffset(topic, partitionID, time)
-
 	if err != nil {
 		if err := client.RefreshMetadata(topic); err != nil {
 			return -1, err
@@ -484,6 +528,35 @@
 	return controller, nil
 }
 
+// deregisterController removes the cached controllerID
+func (client *client) deregisterController() {
+	client.lock.Lock()
+	defer client.lock.Unlock()
+	delete(client.brokers, client.controllerID)
+}
+
+// RefreshController retrieves the cluster controller from fresh metadata
+// and stores it in the local cache. Requires Kafka 0.10 or higher.
+func (client *client) RefreshController() (*Broker, error) {
+	if client.Closed() {
+		return nil, ErrClosedClient
+	}
+
+	client.deregisterController()
+
+	if err := client.refreshMetadata(); err != nil {
+		return nil, err
+	}
+
+	controller := client.cachedController()
+	if controller == nil {
+		return nil, ErrControllerNotAvailable
+	}
+
+	_ = controller.Open(client.conf)
+	return controller, nil
+}
+
 func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
 	if client.Closed() {
 		return nil, ErrClosedClient
@@ -525,10 +598,46 @@
 
 // private broker management helpers
 
+func (client *client) randomizeSeedBrokers(addrs []string) {
+	random := rand.New(rand.NewSource(time.Now().UnixNano()))
+	for _, index := range random.Perm(len(addrs)) {
+		client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index]))
+	}
+}
+
+func (client *client) updateBroker(brokers []*Broker) {
+	currentBroker := make(map[int32]*Broker, len(brokers))
+
+	for _, broker := range brokers {
+		currentBroker[broker.ID()] = broker
+		if client.brokers[broker.ID()] == nil { // add new broker
+			client.brokers[broker.ID()] = broker
+			Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
+		} else if broker.Addr() != client.brokers[broker.ID()].Addr() { // replace broker with new address
+			safeAsyncClose(client.brokers[broker.ID()])
+			client.brokers[broker.ID()] = broker
+			Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
+		}
+	}
+
+	for id, broker := range client.brokers {
+		if _, exist := currentBroker[id]; !exist { // remove old broker
+			safeAsyncClose(broker)
+			delete(client.brokers, id)
+			Logger.Printf("client/broker remove invalid broker #%d with %s", broker.ID(), broker.Addr())
+		}
+	}
+}
+
 // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
 // in the brokers map. It returns the broker that is registered, which may be the provided broker,
 // or a previously registered Broker instance. You must hold the write lock before calling this function.
 func (client *client) registerBroker(broker *Broker) {
+	if client.brokers == nil {
+		Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr())
+		return
+	}
+
 	if client.brokers[broker.ID()] == nil {
 		client.brokers[broker.ID()] = broker
 		Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
@@ -722,7 +831,7 @@
 }
 
 func (client *client) refreshMetadata() error {
-	topics := []string{}
+	var topics []string
 
 	if !client.conf.Metadata.Full {
 		if specificTopics, err := client.MetadataTopics(); err != nil {
@@ -756,7 +865,7 @@
 				Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")
 				return err
 			}
-			Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
+			Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)
 			if backoff > 0 {
 				time.Sleep(backoff)
 			}
@@ -782,7 +891,7 @@
 			req.Version = 1
 		}
 		response, err := broker.GetMetadata(req)
-		switch err.(type) {
+		switch err := err.(type) {
 		case nil:
 			allKnownMetaData := len(topics) == 0
 			// valid response, use it
@@ -799,10 +908,15 @@
 
 		case KError:
 			// if SASL auth error return as this _should_ be a non retryable err for all brokers
-			if err.(KError) == ErrSASLAuthenticationFailed {
+			if err == ErrSASLAuthenticationFailed {
 				Logger.Println("client/metadata failed SASL authentication")
 				return err
 			}
+
+			if err == ErrTopicAuthorizationFailed {
+				Logger.Println("client is not authorized to access this topic. The topics were: ", topics)
+				return err
+			}
 			// else remove that broker and try again
 			Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
 			_ = broker.Close()
@@ -817,7 +931,7 @@
 	}
 
 	if broker != nil {
-		Logger.Println("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
+		Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
 		return retry(ErrOutOfBrokers)
 	}
 
@@ -828,16 +942,19 @@
 
 // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
 func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {
+	if client.Closed() {
+		return
+	}
+
 	client.lock.Lock()
 	defer client.lock.Unlock()
 
 	// For all the brokers we received:
 	// - if it is a new ID, save it
 	// - if it is an existing ID, but the address we have is stale, discard the old one and save it
+	// - if some brokers is not exist in it, remove old broker
 	// - otherwise ignore it, replacing our existing one would just bounce the connection
-	for _, broker := range data.Brokers {
-		client.registerBroker(broker)
-	}
+	client.updateBroker(data.Brokers)
 
 	client.controllerID = data.ControllerID
 
@@ -935,7 +1052,6 @@
 		request.CoordinatorType = CoordinatorGroup
 
 		response, err := broker.FindCoordinator(request)
-
 		if err != nil {
 			Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
 
@@ -966,6 +1082,10 @@
 			}
 
 			return retry(ErrConsumerCoordinatorNotAvailable)
+		case ErrGroupAuthorizationFailed:
+			Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup)
+			return retry(ErrGroupAuthorizationFailed)
+
 		default:
 			return nil, response.Err
 		}
diff --git a/vendor/github.com/Shopify/sarama/compress.go b/vendor/github.com/Shopify/sarama/compress.go
index 94b716e..12cd7c3 100644
--- a/vendor/github.com/Shopify/sarama/compress.go
+++ b/vendor/github.com/Shopify/sarama/compress.go
@@ -6,7 +6,7 @@
 	"fmt"
 	"sync"
 
-	"github.com/eapache/go-xerial-snappy"
+	snappy "github.com/eapache/go-xerial-snappy"
 	"github.com/pierrec/lz4"
 )
 
@@ -22,6 +22,87 @@
 			return gzip.NewWriter(nil)
 		},
 	}
+	gzipWriterPoolForCompressionLevel1 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 1)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel2 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 2)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel3 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 3)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel4 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 4)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel5 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 5)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel6 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 6)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel7 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 7)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel8 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 8)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
+	gzipWriterPoolForCompressionLevel9 = sync.Pool{
+		New: func() interface{} {
+			gz, err := gzip.NewWriterLevel(nil, 9)
+			if err != nil {
+				panic(err)
+			}
+			return gz
+		},
+	}
 )
 
 func compress(cc CompressionCodec, level int, data []byte) ([]byte, error) {
@@ -34,15 +115,53 @@
 			buf    bytes.Buffer
 			writer *gzip.Writer
 		)
-		if level != CompressionLevelDefault {
+
+		switch level {
+		case CompressionLevelDefault:
+			writer = gzipWriterPool.Get().(*gzip.Writer)
+			defer gzipWriterPool.Put(writer)
+			writer.Reset(&buf)
+		case 1:
+			writer = gzipWriterPoolForCompressionLevel1.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel1.Put(writer)
+			writer.Reset(&buf)
+		case 2:
+			writer = gzipWriterPoolForCompressionLevel2.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel2.Put(writer)
+			writer.Reset(&buf)
+		case 3:
+			writer = gzipWriterPoolForCompressionLevel3.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel3.Put(writer)
+			writer.Reset(&buf)
+		case 4:
+			writer = gzipWriterPoolForCompressionLevel4.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel4.Put(writer)
+			writer.Reset(&buf)
+		case 5:
+			writer = gzipWriterPoolForCompressionLevel5.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel5.Put(writer)
+			writer.Reset(&buf)
+		case 6:
+			writer = gzipWriterPoolForCompressionLevel6.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel6.Put(writer)
+			writer.Reset(&buf)
+		case 7:
+			writer = gzipWriterPoolForCompressionLevel7.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel7.Put(writer)
+			writer.Reset(&buf)
+		case 8:
+			writer = gzipWriterPoolForCompressionLevel8.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel8.Put(writer)
+			writer.Reset(&buf)
+		case 9:
+			writer = gzipWriterPoolForCompressionLevel9.Get().(*gzip.Writer)
+			defer gzipWriterPoolForCompressionLevel9.Put(writer)
+			writer.Reset(&buf)
+		default:
 			writer, err = gzip.NewWriterLevel(&buf, level)
 			if err != nil {
 				return nil, err
 			}
-		} else {
-			writer = gzipWriterPool.Get().(*gzip.Writer)
-			defer gzipWriterPool.Put(writer)
-			writer.Reset(&buf)
 		}
 		if _, err := writer.Write(data); err != nil {
 			return nil, err
@@ -68,7 +187,7 @@
 		}
 		return buf.Bytes(), nil
 	case CompressionZSTD:
-		return zstdCompressLevel(nil, data, level)
+		return zstdCompress(nil, data)
 	default:
 		return nil, PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", cc)}
 	}
diff --git a/vendor/github.com/Shopify/sarama/config.go b/vendor/github.com/Shopify/sarama/config.go
index e2e6513..43e739c 100644
--- a/vendor/github.com/Shopify/sarama/config.go
+++ b/vendor/github.com/Shopify/sarama/config.go
@@ -21,6 +21,13 @@
 type Config struct {
 	// Admin is the namespace for ClusterAdmin properties used by the administrative Kafka client.
 	Admin struct {
+		Retry struct {
+			// The total number of times to retry sending (retriable) admin requests (default 5).
+			// Similar to the `retries` setting of the JVM AdminClientConfig.
+			Max int
+			// Backoff time between retries of a failed request (default 100ms)
+			Backoff time.Duration
+		}
 		// The maximum duration the administrative Kafka client will wait for ClusterAdmin operations,
 		// including topics, brokers, configurations and ACLs (defaults to 3 seconds).
 		Timeout time.Duration
@@ -65,8 +72,15 @@
 			// (defaults to true). You should only set this to false if you're using
 			// a non-Kafka SASL proxy.
 			Handshake bool
-			//username and password for SASL/PLAIN  or SASL/SCRAM authentication
-			User     string
+			// AuthIdentity is an (optional) authorization identity (authzid) to
+			// use for SASL/PLAIN authentication (if different from User) when
+			// an authenticated user is permitted to act as the presented
+			// alternative user. See RFC4616 for details.
+			AuthIdentity string
+			// User is the authentication identity (authcid) to present for
+			// SASL/PLAIN or SASL/SCRAM authentication
+			User string
+			// Password for SASL/PLAIN authentication
 			Password string
 			// authz id used for SASL/SCRAM authentication
 			SCRAMAuthzID string
@@ -82,8 +96,9 @@
 			GSSAPI GSSAPIConfig
 		}
 
-		// KeepAlive specifies the keep-alive period for an active network connection.
-		// If zero, keep-alives are disabled. (default is 0: disabled).
+		// KeepAlive specifies the keep-alive period for an active network connection (defaults to 0).
+		// If zero or positive, keep-alives are enabled.
+		// If negative, keep-alives are disabled.
 		KeepAlive time.Duration
 
 		// LocalAddr is the local address to use when dialing an
@@ -214,6 +229,14 @@
 			// `Backoff` if set.
 			BackoffFunc func(retries, maxRetries int) time.Duration
 		}
+
+		// Interceptors to be called when the producer dispatcher reads the
+		// message for the first time. Interceptors allows to intercept and
+		// possible mutate the message before they are published to Kafka
+		// cluster. *ProducerMessage modified by the first interceptor's
+		// OnSend() is passed to the second interceptor OnSend(), and so on in
+		// the interceptor chain.
+		Interceptors []ProducerInterceptor
 	}
 
 	// Consumer is the namespace for configuration related to consuming messages,
@@ -312,7 +335,7 @@
 		// than this, that partition will stop fetching more messages until it
 		// can proceed again.
 		// Note that, since the Messages channel is buffered, the actual grace time is
-		// (MaxProcessingTime * ChanneBufferSize). Defaults to 100ms.
+		// (MaxProcessingTime * ChannelBufferSize). Defaults to 100ms.
 		// If a message is not written to the Messages channel between two ticks
 		// of the expiryTicker then a timeout is detected.
 		// Using a ticker instead of a timer to detect timeouts should typically
@@ -338,9 +361,21 @@
 		// offsets. This currently requires the manual use of an OffsetManager
 		// but will eventually be automated.
 		Offsets struct {
-			// How frequently to commit updated offsets. Defaults to 1s.
+			// Deprecated: CommitInterval exists for historical compatibility
+			// and should not be used. Please use Consumer.Offsets.AutoCommit
 			CommitInterval time.Duration
 
+			// AutoCommit specifies configuration for commit messages automatically.
+			AutoCommit struct {
+				// Whether or not to auto-commit updated offsets back to the broker.
+				// (default enabled).
+				Enable bool
+
+				// How frequently to commit updated offsets. Ineffective unless
+				// auto-commit is enabled (default 1s)
+				Interval time.Duration
+			}
+
 			// The initial offset to use if no offset was previously committed.
 			// Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest.
 			Initial int64
@@ -364,12 +399,24 @@
 		// 	- use `ReadUncommitted` (default) to consume and return all messages in message channel
 		//	- use `ReadCommitted` to hide messages that are part of an aborted transaction
 		IsolationLevel IsolationLevel
+
+		// Interceptors to be called just before the record is sent to the
+		// messages channel. Interceptors allows to intercept and possible
+		// mutate the message before they are returned to the client.
+		// *ConsumerMessage modified by the first interceptor's OnConsume() is
+		// passed to the second interceptor OnConsume(), and so on in the
+		// interceptor chain.
+		Interceptors []ConsumerInterceptor
 	}
 
 	// A user-provided string sent with every request to the brokers for logging,
 	// debugging, and auditing purposes. Defaults to "sarama", but you should
 	// probably set it to something specific to your application.
 	ClientID string
+	// A rack identifier for this client. This can be any string value which
+	// indicates where this client is physically located.
+	// It corresponds with the broker config 'broker.rack'
+	RackID string
 	// The number of events to buffer in internal and external channels. This
 	// permits the producer and consumer to continue processing some messages
 	// in the background while user code is working, greatly improving throughput.
@@ -394,6 +441,8 @@
 func NewConfig() *Config {
 	c := &Config{}
 
+	c.Admin.Retry.Max = 5
+	c.Admin.Retry.Backoff = 100 * time.Millisecond
 	c.Admin.Timeout = 3 * time.Second
 
 	c.Net.MaxOpenRequests = 5
@@ -423,7 +472,8 @@
 	c.Consumer.MaxWaitTime = 250 * time.Millisecond
 	c.Consumer.MaxProcessingTime = 100 * time.Millisecond
 	c.Consumer.Return.Errors = false
-	c.Consumer.Offsets.CommitInterval = 1 * time.Second
+	c.Consumer.Offsets.AutoCommit.Enable = true
+	c.Consumer.Offsets.AutoCommit.Interval = 1 * time.Second
 	c.Consumer.Offsets.Initial = OffsetNewest
 	c.Consumer.Offsets.Retry.Max = 3
 
@@ -436,7 +486,7 @@
 
 	c.ClientID = defaultClientID
 	c.ChannelBufferSize = 256
-	c.Version = MinVersion
+	c.Version = DefaultVersion
 	c.MetricRegistry = metrics.NewRegistry()
 
 	return c
@@ -504,8 +554,6 @@
 		return ConfigurationError("Net.ReadTimeout must be > 0")
 	case c.Net.WriteTimeout <= 0:
 		return ConfigurationError("Net.WriteTimeout must be > 0")
-	case c.Net.KeepAlive < 0:
-		return ConfigurationError("Net.KeepAlive must be >= 0")
 	case c.Net.SASL.Enable:
 		if c.Net.SASL.Mechanism == "" {
 			c.Net.SASL.Mechanism = SASLTypePlaintext
@@ -621,6 +669,10 @@
 		}
 	}
 
+	if c.Producer.Compression == CompressionZSTD && !c.Version.IsAtLeast(V2_1_0_0) {
+		return ConfigurationError("zstd compression requires Version >= V2_1_0_0")
+	}
+
 	if c.Producer.Idempotent {
 		if !c.Version.IsAtLeast(V0_11_0_0) {
 			return ConfigurationError("Idempotent producer requires Version >= V0_11_0_0")
@@ -650,8 +702,8 @@
 		return ConfigurationError("Consumer.MaxProcessingTime must be > 0")
 	case c.Consumer.Retry.Backoff < 0:
 		return ConfigurationError("Consumer.Retry.Backoff must be >= 0")
-	case c.Consumer.Offsets.CommitInterval <= 0:
-		return ConfigurationError("Consumer.Offsets.CommitInterval must be > 0")
+	case c.Consumer.Offsets.AutoCommit.Interval <= 0:
+		return ConfigurationError("Consumer.Offsets.AutoCommit.Interval must be > 0")
 	case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest:
 		return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest")
 	case c.Consumer.Offsets.Retry.Max < 0:
@@ -660,6 +712,11 @@
 		return ConfigurationError("Consumer.IsolationLevel must be ReadUncommitted or ReadCommitted")
 	}
 
+	if c.Consumer.Offsets.CommitInterval != 0 {
+		Logger.Println("Deprecation warning: Consumer.Offsets.CommitInterval exists for historical compatibility" +
+			" and should not be used. Please use Consumer.Offsets.AutoCommit, the current value will be ignored")
+	}
+
 	// validate IsolationLevel
 	if c.Consumer.IsolationLevel == ReadCommitted && !c.Version.IsAtLeast(V0_11_0_0) {
 		return ConfigurationError("ReadCommitted requires Version >= V0_11_0_0")
@@ -693,3 +750,16 @@
 
 	return nil
 }
+
+func (c *Config) getDialer() proxy.Dialer {
+	if c.Net.Proxy.Enable {
+		Logger.Printf("using proxy %s", c.Net.Proxy.Dialer)
+		return c.Net.Proxy.Dialer
+	} else {
+		return &net.Dialer{
+			Timeout:   c.Net.DialTimeout,
+			KeepAlive: c.Net.KeepAlive,
+			LocalAddr: c.Net.LocalAddr,
+		}
+	}
+}
diff --git a/vendor/github.com/Shopify/sarama/config_resource_type.go b/vendor/github.com/Shopify/sarama/config_resource_type.go
index 5399d75..bef1053 100644
--- a/vendor/github.com/Shopify/sarama/config_resource_type.go
+++ b/vendor/github.com/Shopify/sarama/config_resource_type.go
@@ -1,22 +1,18 @@
 package sarama
 
-//ConfigResourceType is a type for config resource
+// ConfigResourceType is a type for resources that have configs.
 type ConfigResourceType int8
 
-// Taken from :
-// https://cwiki.apache.org/confluence/display/KAFKA/KIP-133%3A+Describe+and+Alter+Configs+Admin+APIs#KIP-133:DescribeandAlterConfigsAdminAPIs-WireFormattypes
+// Taken from:
+// https://github.com/apache/kafka/blob/ed7c071e07f1f90e4c2895582f61ca090ced3c42/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java#L32-L55
 
 const (
-	//UnknownResource constant type
-	UnknownResource ConfigResourceType = iota
-	//AnyResource constant type
-	AnyResource
-	//TopicResource constant type
-	TopicResource
-	//GroupResource constant type
-	GroupResource
-	//ClusterResource constant type
-	ClusterResource
-	//BrokerResource constant type
-	BrokerResource
+	// UnknownResource constant type
+	UnknownResource ConfigResourceType = 0
+	// TopicResource constant type
+	TopicResource ConfigResourceType = 2
+	// BrokerResource constant type
+	BrokerResource ConfigResourceType = 4
+	// BrokerLoggerResource constant type
+	BrokerLoggerResource ConfigResourceType = 8
 )
diff --git a/vendor/github.com/Shopify/sarama/consumer.go b/vendor/github.com/Shopify/sarama/consumer.go
index 72c4d7c..f9cd172 100644
--- a/vendor/github.com/Shopify/sarama/consumer.go
+++ b/vendor/github.com/Shopify/sarama/consumer.go
@@ -35,6 +35,10 @@
 	return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err)
 }
 
+func (ce ConsumerError) Unwrap() error {
+	return ce.Err
+}
+
 // ConsumerErrors is a type that wraps a batch of errors and implements the Error interface.
 // It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors
 // when stopping.
@@ -299,6 +303,8 @@
 	errors   chan *ConsumerError
 	feeder   chan *FetchResponse
 
+	preferredReadReplica int32
+
 	trigger, dying chan none
 	closeOnce      sync.Once
 	topic          string
@@ -359,18 +365,29 @@
 	close(child.feeder)
 }
 
+func (child *partitionConsumer) preferredBroker() (*Broker, error) {
+	if child.preferredReadReplica >= 0 {
+		broker, err := child.consumer.client.Broker(child.preferredReadReplica)
+		if err == nil {
+			return broker, nil
+		}
+	}
+
+	// if prefered replica cannot be found fallback to leader
+	return child.consumer.client.Leader(child.topic, child.partition)
+}
+
 func (child *partitionConsumer) dispatch() error {
 	if err := child.consumer.client.RefreshMetadata(child.topic); err != nil {
 		return err
 	}
 
-	var leader *Broker
-	var err error
-	if leader, err = child.consumer.client.Leader(child.topic, child.partition); err != nil {
+	broker, err := child.preferredBroker()
+	if err != nil {
 		return err
 	}
 
-	child.broker = child.consumer.refBrokerConsumer(leader)
+	child.broker = child.consumer.refBrokerConsumer(broker)
 
 	child.broker.input <- child
 
@@ -422,13 +439,13 @@
 func (child *partitionConsumer) Close() error {
 	child.AsyncClose()
 
-	var errors ConsumerErrors
+	var consumerErrors ConsumerErrors
 	for err := range child.errors {
-		errors = append(errors, err)
+		consumerErrors = append(consumerErrors, err)
 	}
 
-	if len(errors) > 0 {
-		return errors
+	if len(consumerErrors) > 0 {
+		return consumerErrors
 	}
 	return nil
 }
@@ -451,6 +468,7 @@
 		}
 
 		for i, msg := range msgs {
+			child.interceptors(msg)
 		messageSelect:
 			select {
 			case <-child.dying:
@@ -464,6 +482,7 @@
 					child.broker.acks.Done()
 				remainingLoop:
 					for _, msg = range msgs[i:] {
+						child.interceptors(msg)
 						select {
 						case child.messages <- msg:
 						case <-child.dying:
@@ -586,6 +605,8 @@
 
 	consumerBatchSizeMetric.Update(int64(nRecs))
 
+	child.preferredReadReplica = block.PreferredReadReplica
+
 	if nRecs == 0 {
 		partialTrailingMessage, err := block.isPartial()
 		if err != nil {
@@ -623,7 +644,7 @@
 	abortedProducerIDs := make(map[int64]struct{}, len(block.AbortedTransactions))
 	abortedTransactions := block.getAbortedTransactions()
 
-	messages := []*ConsumerMessage{}
+	var messages []*ConsumerMessage
 	for _, records := range block.RecordsSet {
 		switch records.recordsType {
 		case legacyRecords:
@@ -693,6 +714,12 @@
 	return messages, nil
 }
 
+func (child *partitionConsumer) interceptors(msg *ConsumerMessage) {
+	for _, interceptor := range child.conf.Consumer.Interceptors {
+		msg.safelyApplyInterceptor(interceptor)
+	}
+}
+
 type brokerConsumer struct {
 	consumer         *consumer
 	broker           *Broker
@@ -761,7 +788,7 @@
 	close(bc.newSubscriptions)
 }
 
-//subscriptionConsumer ensures we will get nil right away if no new subscriptions is available
+// subscriptionConsumer ensures we will get nil right away if no new subscriptions is available
 func (bc *brokerConsumer) subscriptionConsumer() {
 	<-bc.wait // wait for our first piece of work
 
@@ -776,7 +803,6 @@
 		}
 
 		response, err := bc.fetchNewMessages()
-
 		if err != nil {
 			Logger.Printf("consumer/broker/%d disconnecting due to error processing FetchRequest: %s\n", bc.broker.ID(), err)
 			bc.abort(err)
@@ -810,15 +836,27 @@
 	}
 }
 
-//handleResponses handles the response codes left for us by our subscriptions, and abandons ones that have been closed
+// handleResponses handles the response codes left for us by our subscriptions, and abandons ones that have been closed
 func (bc *brokerConsumer) handleResponses() {
 	for child := range bc.subscriptions {
 		result := child.responseResult
 		child.responseResult = nil
 
+		if result == nil {
+			if preferredBroker, err := child.preferredBroker(); err == nil {
+				if bc.broker.ID() != preferredBroker.ID() {
+					// not an error but needs redispatching to consume from prefered replica
+					child.trigger <- none{}
+					delete(bc.subscriptions, child)
+				}
+			}
+			continue
+		}
+
+		// Discard any replica preference.
+		child.preferredReadReplica = -1
+
 		switch result {
-		case nil:
-			// no-op
 		case errTimedOut:
 			Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because consuming was taking too long\n",
 				bc.broker.ID(), child.topic, child.partition)
@@ -887,6 +925,21 @@
 		request.Version = 4
 		request.Isolation = bc.consumer.conf.Consumer.IsolationLevel
 	}
+	if bc.consumer.conf.Version.IsAtLeast(V1_1_0_0) {
+		request.Version = 7
+		// We do not currently implement KIP-227 FetchSessions. Setting the id to 0
+		// and the epoch to -1 tells the broker not to generate as session ID we're going
+		// to just ignore anyway.
+		request.SessionID = 0
+		request.SessionEpoch = -1
+	}
+	if bc.consumer.conf.Version.IsAtLeast(V2_1_0_0) {
+		request.Version = 10
+	}
+	if bc.consumer.conf.Version.IsAtLeast(V2_3_0_0) {
+		request.Version = 11
+		request.RackID = bc.consumer.conf.RackID
+	}
 
 	for child := range bc.subscriptions {
 		request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize)
diff --git a/vendor/github.com/Shopify/sarama/consumer_group.go b/vendor/github.com/Shopify/sarama/consumer_group.go
index 8de9513..2bf236a 100644
--- a/vendor/github.com/Shopify/sarama/consumer_group.go
+++ b/vendor/github.com/Shopify/sarama/consumer_group.go
@@ -38,6 +38,9 @@
 	// as quickly as possible to allow time for Cleanup() and the final offset commit. If the timeout
 	// is exceeded, the consumer will be removed from the group by Kafka, which will cause offset
 	// commit failures.
+	// This method should be called inside an infinite loop, when a
+	// server-side rebalance happens, the consumer session will need to be
+	// recreated to get the new claims.
 	Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error
 
 	// Errors returns a read channel of errors that occurred during the consumer life-cycle.
@@ -63,6 +66,8 @@
 	lock      sync.Mutex
 	closed    chan none
 	closeOnce sync.Once
+
+	userData []byte
 }
 
 // NewConsumerGroup creates a new consumer group the given broker addresses and configuration.
@@ -118,9 +123,6 @@
 	c.closeOnce.Do(func() {
 		close(c.closed)
 
-		c.lock.Lock()
-		defer c.lock.Unlock()
-
 		// leave group
 		if e := c.leave(); e != nil {
 			err = e
@@ -171,6 +173,11 @@
 		return err
 	}
 
+	// loop check topic partition numbers changed
+	// will trigger rebalance when any topic partitions number had changed
+	// avoid Consume function called again that will generate more than loopCheckPartitionNumbers coroutine
+	go c.loopCheckPartitionNumbers(topics, sess)
+
 	// Wait for session exit signal
 	<-sess.ctx.Done()
 
@@ -248,40 +255,41 @@
 	}
 
 	// Sync consumer group
-	sync, err := c.syncGroupRequest(coordinator, plan, join.GenerationId)
+	groupRequest, err := c.syncGroupRequest(coordinator, plan, join.GenerationId)
 	if err != nil {
 		_ = coordinator.Close()
 		return nil, err
 	}
-	switch sync.Err {
+	switch groupRequest.Err {
 	case ErrNoError:
 	case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately
 		c.memberID = ""
 		return c.newSession(ctx, topics, handler, retries)
 	case ErrNotCoordinatorForConsumer: // retry after backoff with coordinator refresh
 		if retries <= 0 {
-			return nil, sync.Err
+			return nil, groupRequest.Err
 		}
 
 		return c.retryNewSession(ctx, topics, handler, retries, true)
 	case ErrRebalanceInProgress: // retry after backoff
 		if retries <= 0 {
-			return nil, sync.Err
+			return nil, groupRequest.Err
 		}
 
 		return c.retryNewSession(ctx, topics, handler, retries, false)
 	default:
-		return nil, sync.Err
+		return nil, groupRequest.Err
 	}
 
 	// Retrieve and sort claims
 	var claims map[string][]int32
-	if len(sync.MemberAssignment) > 0 {
-		members, err := sync.GetMemberAssignment()
+	if len(groupRequest.MemberAssignment) > 0 {
+		members, err := groupRequest.GetMemberAssignment()
 		if err != nil {
 			return nil, err
 		}
 		claims = members.Topics
+		c.userData = members.UserData
 
 		for _, partitions := range claims {
 			sort.Sort(int32Slice(partitions))
@@ -303,9 +311,14 @@
 		req.RebalanceTimeout = int32(c.config.Consumer.Group.Rebalance.Timeout / time.Millisecond)
 	}
 
+	// use static user-data if configured, otherwise use consumer-group userdata from the last sync
+	userData := c.config.Consumer.Group.Member.UserData
+	if len(userData) == 0 {
+		userData = c.userData
+	}
 	meta := &ConsumerGroupMemberMetadata{
 		Topics:   topics,
-		UserData: c.config.Consumer.Group.Member.UserData,
+		UserData: userData,
 	}
 	strategy := c.config.Consumer.Group.Rebalance.Strategy
 	if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil {
@@ -321,13 +334,17 @@
 		MemberId:     c.memberID,
 		GenerationId: generationID,
 	}
+	strategy := c.config.Consumer.Group.Rebalance.Strategy
 	for memberID, topics := range plan {
-		err := req.AddGroupAssignmentMember(memberID, &ConsumerGroupMemberAssignment{
-			Topics: topics,
-		})
+		assignment := &ConsumerGroupMemberAssignment{Topics: topics}
+		userDataBytes, err := strategy.AssignmentData(memberID, topics, generationID)
 		if err != nil {
 			return nil, err
 		}
+		assignment.UserData = userDataBytes
+		if err := req.AddGroupAssignmentMember(memberID, assignment); err != nil {
+			return nil, err
+		}
 	}
 	return coordinator.SyncGroup(req)
 }
@@ -362,8 +379,10 @@
 	return strategy.Plan(members, topics)
 }
 
-// Leaves the cluster, called by Close, protected by lock.
+// Leaves the cluster, called by Close.
 func (c *consumerGroup) leave() error {
+	c.lock.Lock()
+	defer c.lock.Unlock()
 	if c.memberID == "" {
 		return nil
 	}
@@ -395,12 +414,6 @@
 }
 
 func (c *consumerGroup) handleError(err error, topic string, partition int32) {
-	select {
-	case <-c.closed:
-		return
-	default:
-	}
-
 	if _, ok := err.(*ConsumerError); !ok && topic != "" && partition > -1 {
 		err = &ConsumerError{
 			Topic:     topic,
@@ -409,14 +422,67 @@
 		}
 	}
 
-	if c.config.Consumer.Return.Errors {
-		select {
-		case c.errors <- err:
-		default:
-		}
-	} else {
+	if !c.config.Consumer.Return.Errors {
 		Logger.Println(err)
+		return
 	}
+
+	select {
+	case <-c.closed:
+		// consumer is closed
+		return
+	default:
+	}
+
+	select {
+	case c.errors <- err:
+	default:
+		// no error listener
+	}
+}
+
+func (c *consumerGroup) loopCheckPartitionNumbers(topics []string, session *consumerGroupSession) {
+	pause := time.NewTicker(c.config.Metadata.RefreshFrequency)
+	defer session.cancel()
+	defer pause.Stop()
+	var oldTopicToPartitionNum map[string]int
+	var err error
+	if oldTopicToPartitionNum, err = c.topicToPartitionNumbers(topics); err != nil {
+		return
+	}
+	for {
+		if newTopicToPartitionNum, err := c.topicToPartitionNumbers(topics); err != nil {
+			return
+		} else {
+			for topic, num := range oldTopicToPartitionNum {
+				if newTopicToPartitionNum[topic] != num {
+					return // trigger the end of the session on exit
+				}
+			}
+		}
+		select {
+		case <-pause.C:
+		case <-session.ctx.Done():
+			Logger.Printf("loop check partition number coroutine will exit, topics %s", topics)
+			// if session closed by other, should be exited
+			return
+		case <-c.closed:
+			return
+		}
+	}
+}
+
+func (c *consumerGroup) topicToPartitionNumbers(topics []string) (map[string]int, error) {
+	topicToPartitionNum := make(map[string]int, len(topics))
+	for _, topic := range topics {
+		if partitionNum, err := c.client.Partitions(topic); err != nil {
+			Logger.Printf("Consumer Group topic %s get partition number failed %v", topic, err)
+			return nil, err
+		} else {
+			topicToPartitionNum[topic] = len(partitionNum)
+		}
+	}
+	return topicToPartitionNum, nil
 }
 
 // --------------------------------------------------------------------
@@ -447,6 +513,11 @@
 	// message twice, and your processing should ideally be idempotent.
 	MarkOffset(topic string, partition int32, offset int64, metadata string)
 
+	// Commit the offset to the backend
+	//
+	// Note: calling Commit performs a blocking synchronous operation.
+	Commit()
+
 	// ResetOffset resets to the provided offset, alongside a metadata string that
 	// represents the state of the partition consumer at that point in time. Reset
 	// acts as a counterpart to MarkOffset, the difference being that it allows to
@@ -558,6 +629,10 @@
 	}
 }
 
+func (s *consumerGroupSession) Commit() {
+	s.offsets.Commit()
+}
+
 func (s *consumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) {
 	if pom := s.offsets.findPOM(topic, partition); pom != nil {
 		pom.ResetOffset(offset, metadata)
@@ -657,6 +732,9 @@
 	pause := time.NewTicker(s.parent.config.Consumer.Group.Heartbeat.Interval)
 	defer pause.Stop()
 
+	retryBackoff := time.NewTimer(s.parent.config.Metadata.Retry.Backoff)
+	defer retryBackoff.Stop()
+
 	retries := s.parent.config.Metadata.Retry.Max
 	for {
 		coordinator, err := s.parent.client.Coordinator(s.parent.groupID)
@@ -665,11 +743,11 @@
 				s.parent.handleError(err, "", -1)
 				return
 			}
-
+			retryBackoff.Reset(s.parent.config.Metadata.Retry.Backoff)
 			select {
 			case <-s.hbDying:
 				return
-			case <-time.After(s.parent.config.Metadata.Retry.Backoff):
+			case <-retryBackoff.C:
 				retries--
 			}
 			continue
@@ -694,7 +772,7 @@
 		case ErrRebalanceInProgress, ErrUnknownMemberId, ErrIllegalGeneration:
 			return
 		default:
-			s.parent.handleError(err, "", -1)
+			s.parent.handleError(resp.Err, "", -1)
 			return
 		}
 
diff --git a/vendor/github.com/Shopify/sarama/consumer_group_members.go b/vendor/github.com/Shopify/sarama/consumer_group_members.go
index 2d02cc3..21b11e9 100644
--- a/vendor/github.com/Shopify/sarama/consumer_group_members.go
+++ b/vendor/github.com/Shopify/sarama/consumer_group_members.go
@@ -1,6 +1,6 @@
 package sarama
 
-//ConsumerGroupMemberMetadata holds the metadata for consumer group
+// ConsumerGroupMemberMetadata holds the metadata for consumer group
 type ConsumerGroupMemberMetadata struct {
 	Version  int16
 	Topics   []string
@@ -37,7 +37,7 @@
 	return nil
 }
 
-//ConsumerGroupMemberAssignment holds the member assignment for a consume group
+// ConsumerGroupMemberAssignment holds the member assignment for a consume group
 type ConsumerGroupMemberAssignment struct {
 	Version  int16
 	Topics   map[string][]int32
diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_request.go b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go
index a8dcaef..5c18e04 100644
--- a/vendor/github.com/Shopify/sarama/consumer_metadata_request.go
+++ b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go
@@ -1,6 +1,6 @@
 package sarama
 
-//ConsumerMetadataRequest is used for metadata requests
+// ConsumerMetadataRequest is used for metadata requests
 type ConsumerMetadataRequest struct {
 	ConsumerGroup string
 }
@@ -29,6 +29,10 @@
 	return 0
 }
 
+func (r *ConsumerMetadataRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *ConsumerMetadataRequest) requiredVersion() KafkaVersion {
 	return V0_8_2_0
 }
diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_response.go b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go
index f39a871..7fe0cf9 100644
--- a/vendor/github.com/Shopify/sarama/consumer_metadata_response.go
+++ b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go
@@ -5,7 +5,7 @@
 	"strconv"
 )
 
-//ConsumerMetadataResponse holds the response for a consumer group meta data requests
+// ConsumerMetadataResponse holds the response for a consumer group meta data requests
 type ConsumerMetadataResponse struct {
 	Err             KError
 	Coordinator     *Broker
@@ -73,6 +73,10 @@
 	return 0
 }
 
+func (r *ConsumerMetadataResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *ConsumerMetadataResponse) requiredVersion() KafkaVersion {
 	return V0_8_2_0
 }
diff --git a/vendor/github.com/Shopify/sarama/control_record.go b/vendor/github.com/Shopify/sarama/control_record.go
index 9b75ab5..244a821 100644
--- a/vendor/github.com/Shopify/sarama/control_record.go
+++ b/vendor/github.com/Shopify/sarama/control_record.go
@@ -1,14 +1,14 @@
 package sarama
 
-//ControlRecordType ...
+// ControlRecordType ...
 type ControlRecordType int
 
 const (
-	//ControlRecordAbort is a control record for abort
+	// ControlRecordAbort is a control record for abort
 	ControlRecordAbort ControlRecordType = iota
-	//ControlRecordCommit is a control record for commit
+	// ControlRecordCommit is a control record for commit
 	ControlRecordCommit
-	//ControlRecordUnknown is a control record of unknown type
+	// ControlRecordUnknown is a control record of unknown type
 	ControlRecordUnknown
 )
 
@@ -23,16 +23,6 @@
 
 func (cr *ControlRecord) decode(key, value packetDecoder) error {
 	var err error
-	cr.Version, err = value.getInt16()
-	if err != nil {
-		return err
-	}
-
-	cr.CoordinatorEpoch, err = value.getInt32()
-	if err != nil {
-		return err
-	}
-
 	// There a version for the value part AND the key part. And I have no idea if they are supposed to match or not
 	// Either way, all these version can only be 0 for now
 	cr.Version, err = key.getInt16()
@@ -55,6 +45,18 @@
 		// UNKNOWN is used to indicate a control type which the client is not aware of and should be ignored
 		cr.Type = ControlRecordUnknown
 	}
+	// we want to parse value only if we are decoding control record of known type
+	if cr.Type != ControlRecordUnknown {
+		cr.Version, err = value.getInt16()
+		if err != nil {
+			return err
+		}
+
+		cr.CoordinatorEpoch, err = value.getInt32()
+		if err != nil {
+			return err
+		}
+	}
 	return nil
 }
 
diff --git a/vendor/github.com/Shopify/sarama/crc32_field.go b/vendor/github.com/Shopify/sarama/crc32_field.go
index 38189a3..32236e5 100644
--- a/vendor/github.com/Shopify/sarama/crc32_field.go
+++ b/vendor/github.com/Shopify/sarama/crc32_field.go
@@ -72,6 +72,7 @@
 
 	return nil
 }
+
 func (c *crc32Field) crc(curOffset int, buf []byte) (uint32, error) {
 	var tab *crc32.Table
 	switch c.polynomial {
diff --git a/vendor/github.com/Shopify/sarama/create_partitions_request.go b/vendor/github.com/Shopify/sarama/create_partitions_request.go
index af321e9..46fb044 100644
--- a/vendor/github.com/Shopify/sarama/create_partitions_request.go
+++ b/vendor/github.com/Shopify/sarama/create_partitions_request.go
@@ -67,6 +67,10 @@
 	return 0
 }
 
+func (r *CreatePartitionsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *CreatePartitionsRequest) requiredVersion() KafkaVersion {
 	return V1_0_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/create_partitions_response.go b/vendor/github.com/Shopify/sarama/create_partitions_response.go
index bb18204..12ce788 100644
--- a/vendor/github.com/Shopify/sarama/create_partitions_response.go
+++ b/vendor/github.com/Shopify/sarama/create_partitions_response.go
@@ -63,6 +63,10 @@
 	return 0
 }
 
+func (r *CreatePartitionsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *CreatePartitionsResponse) requiredVersion() KafkaVersion {
 	return V1_0_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/create_topics_request.go b/vendor/github.com/Shopify/sarama/create_topics_request.go
index 709c0a4..287acd0 100644
--- a/vendor/github.com/Shopify/sarama/create_topics_request.go
+++ b/vendor/github.com/Shopify/sarama/create_topics_request.go
@@ -79,6 +79,10 @@
 	return c.Version
 }
 
+func (r *CreateTopicsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (c *CreateTopicsRequest) requiredVersion() KafkaVersion {
 	switch c.Version {
 	case 2:
diff --git a/vendor/github.com/Shopify/sarama/create_topics_response.go b/vendor/github.com/Shopify/sarama/create_topics_response.go
index a493e02..7e1448a 100644
--- a/vendor/github.com/Shopify/sarama/create_topics_response.go
+++ b/vendor/github.com/Shopify/sarama/create_topics_response.go
@@ -70,6 +70,10 @@
 	return c.Version
 }
 
+func (c *CreateTopicsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (c *CreateTopicsResponse) requiredVersion() KafkaVersion {
 	switch c.Version {
 	case 2:
diff --git a/vendor/github.com/Shopify/sarama/decompress.go b/vendor/github.com/Shopify/sarama/decompress.go
index eaccbfc..af45fda 100644
--- a/vendor/github.com/Shopify/sarama/decompress.go
+++ b/vendor/github.com/Shopify/sarama/decompress.go
@@ -7,7 +7,7 @@
 	"io/ioutil"
 	"sync"
 
-	"github.com/eapache/go-xerial-snappy"
+	snappy "github.com/eapache/go-xerial-snappy"
 	"github.com/pierrec/lz4"
 )
 
@@ -26,34 +26,32 @@
 	case CompressionNone:
 		return data, nil
 	case CompressionGZIP:
-		var (
-			err        error
-			reader     *gzip.Reader
-			readerIntf = gzipReaderPool.Get()
-		)
-		if readerIntf != nil {
-			reader = readerIntf.(*gzip.Reader)
-		} else {
+		var err error
+		reader, ok := gzipReaderPool.Get().(*gzip.Reader)
+		if !ok {
 			reader, err = gzip.NewReader(bytes.NewReader(data))
-			if err != nil {
-				return nil, err
-			}
+		} else {
+			err = reader.Reset(bytes.NewReader(data))
+		}
+
+		if err != nil {
+			return nil, err
 		}
 
 		defer gzipReaderPool.Put(reader)
 
-		if err := reader.Reset(bytes.NewReader(data)); err != nil {
-			return nil, err
-		}
-
 		return ioutil.ReadAll(reader)
 	case CompressionSnappy:
 		return snappy.Decode(data)
 	case CompressionLZ4:
-		reader := lz4ReaderPool.Get().(*lz4.Reader)
+		reader, ok := lz4ReaderPool.Get().(*lz4.Reader)
+		if !ok {
+			reader = lz4.NewReader(bytes.NewReader(data))
+		} else {
+			reader.Reset(bytes.NewReader(data))
+		}
 		defer lz4ReaderPool.Put(reader)
 
-		reader.Reset(bytes.NewReader(data))
 		return ioutil.ReadAll(reader)
 	case CompressionZSTD:
 		return zstdDecompress(nil, data)
diff --git a/vendor/github.com/Shopify/sarama/delete_groups_request.go b/vendor/github.com/Shopify/sarama/delete_groups_request.go
index 305a324..4ac8bbe 100644
--- a/vendor/github.com/Shopify/sarama/delete_groups_request.go
+++ b/vendor/github.com/Shopify/sarama/delete_groups_request.go
@@ -21,6 +21,10 @@
 	return 0
 }
 
+func (r *DeleteGroupsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *DeleteGroupsRequest) requiredVersion() KafkaVersion {
 	return V1_1_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/delete_groups_response.go b/vendor/github.com/Shopify/sarama/delete_groups_response.go
index c067ebb..5e7b1ed 100644
--- a/vendor/github.com/Shopify/sarama/delete_groups_response.go
+++ b/vendor/github.com/Shopify/sarama/delete_groups_response.go
@@ -65,6 +65,10 @@
 	return 0
 }
 
+func (r *DeleteGroupsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *DeleteGroupsResponse) requiredVersion() KafkaVersion {
 	return V1_1_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/delete_records_request.go b/vendor/github.com/Shopify/sarama/delete_records_request.go
index 93efafd..dc106b1 100644
--- a/vendor/github.com/Shopify/sarama/delete_records_request.go
+++ b/vendor/github.com/Shopify/sarama/delete_records_request.go
@@ -77,6 +77,10 @@
 	return 0
 }
 
+func (d *DeleteRecordsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (d *DeleteRecordsRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/delete_records_response.go b/vendor/github.com/Shopify/sarama/delete_records_response.go
index 733a58b..d530b4c 100644
--- a/vendor/github.com/Shopify/sarama/delete_records_response.go
+++ b/vendor/github.com/Shopify/sarama/delete_records_response.go
@@ -80,6 +80,10 @@
 	return 0
 }
 
+func (d *DeleteRecordsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (d *DeleteRecordsResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/delete_topics_request.go b/vendor/github.com/Shopify/sarama/delete_topics_request.go
index 911f67d..ba6780a 100644
--- a/vendor/github.com/Shopify/sarama/delete_topics_request.go
+++ b/vendor/github.com/Shopify/sarama/delete_topics_request.go
@@ -38,6 +38,10 @@
 	return d.Version
 }
 
+func (d *DeleteTopicsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (d *DeleteTopicsRequest) requiredVersion() KafkaVersion {
 	switch d.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/delete_topics_response.go b/vendor/github.com/Shopify/sarama/delete_topics_response.go
index 3422546..733961a 100644
--- a/vendor/github.com/Shopify/sarama/delete_topics_response.go
+++ b/vendor/github.com/Shopify/sarama/delete_topics_response.go
@@ -68,6 +68,10 @@
 	return d.Version
 }
 
+func (d *DeleteTopicsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (d *DeleteTopicsResponse) requiredVersion() KafkaVersion {
 	switch d.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/describe_configs_request.go b/vendor/github.com/Shopify/sarama/describe_configs_request.go
index ccb587b..4c34880 100644
--- a/vendor/github.com/Shopify/sarama/describe_configs_request.go
+++ b/vendor/github.com/Shopify/sarama/describe_configs_request.go
@@ -61,7 +61,6 @@
 		r.Resources[i].Name = name
 
 		confLength, err := pd.getArrayLength()
-
 		if err != nil {
 			return err
 		}
@@ -100,6 +99,10 @@
 	return r.Version
 }
 
+func (r *DescribeConfigsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *DescribeConfigsRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/describe_configs_response.go b/vendor/github.com/Shopify/sarama/describe_configs_response.go
index 5737232..928f5a5 100644
--- a/vendor/github.com/Shopify/sarama/describe_configs_response.go
+++ b/vendor/github.com/Shopify/sarama/describe_configs_response.go
@@ -112,6 +112,10 @@
 	return r.Version
 }
 
+func (r *DescribeConfigsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *DescribeConfigsResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -220,7 +224,7 @@
 	return nil
 }
 
-//https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration
+// https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration
 func (r *ConfigEntry) decode(pd packetDecoder, version int16) (err error) {
 	if version == 0 {
 		r.Source = SourceUnknown
@@ -249,12 +253,16 @@
 			return err
 		}
 		r.Default = defaultB
+		if defaultB {
+			r.Source = SourceDefault
+		}
 	} else {
 		source, err := pd.getInt8()
 		if err != nil {
 			return err
 		}
 		r.Source = ConfigSource(source)
+		r.Default = r.Source == SourceDefault
 	}
 
 	sensitive, err := pd.getBool()
@@ -277,7 +285,6 @@
 			}
 			r.Synonyms[i] = s
 		}
-
 	}
 	return nil
 }
diff --git a/vendor/github.com/Shopify/sarama/describe_groups_request.go b/vendor/github.com/Shopify/sarama/describe_groups_request.go
index 1fb3567..f8962da 100644
--- a/vendor/github.com/Shopify/sarama/describe_groups_request.go
+++ b/vendor/github.com/Shopify/sarama/describe_groups_request.go
@@ -21,6 +21,10 @@
 	return 0
 }
 
+func (r *DescribeGroupsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *DescribeGroupsRequest) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/describe_groups_response.go b/vendor/github.com/Shopify/sarama/describe_groups_response.go
index 542b3a9..bc242e4 100644
--- a/vendor/github.com/Shopify/sarama/describe_groups_response.go
+++ b/vendor/github.com/Shopify/sarama/describe_groups_response.go
@@ -43,6 +43,10 @@
 	return 0
 }
 
+func (r *DescribeGroupsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *DescribeGroupsResponse) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go b/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go
new file mode 100644
index 0000000..c0bf04e
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go
@@ -0,0 +1,87 @@
+package sarama
+
+// DescribeLogDirsRequest is a describe request to get partitions' log size
+type DescribeLogDirsRequest struct {
+	// Version 0 and 1 are equal
+	// The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
+	Version int16
+
+	// If this is an empty array, all topics will be queried
+	DescribeTopics []DescribeLogDirsRequestTopic
+}
+
+// DescribeLogDirsRequestTopic is a describe request about the log dir of one or more partitions within a Topic
+type DescribeLogDirsRequestTopic struct {
+	Topic        string
+	PartitionIDs []int32
+}
+
+func (r *DescribeLogDirsRequest) encode(pe packetEncoder) error {
+	length := len(r.DescribeTopics)
+	if length == 0 {
+		// In order to query all topics we must send null
+		length = -1
+	}
+
+	if err := pe.putArrayLength(length); err != nil {
+		return err
+	}
+
+	for _, d := range r.DescribeTopics {
+		if err := pe.putString(d.Topic); err != nil {
+			return err
+		}
+
+		if err := pe.putInt32Array(d.PartitionIDs); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (r *DescribeLogDirsRequest) decode(pd packetDecoder, version int16) error {
+	n, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+	if n == -1 {
+		n = 0
+	}
+
+	topics := make([]DescribeLogDirsRequestTopic, n)
+	for i := 0; i < n; i++ {
+		topics[i] = DescribeLogDirsRequestTopic{}
+
+		topic, err := pd.getString()
+		if err != nil {
+			return err
+		}
+		topics[i].Topic = topic
+
+		pIDs, err := pd.getInt32Array()
+		if err != nil {
+			return err
+		}
+		topics[i].PartitionIDs = pIDs
+	}
+	r.DescribeTopics = topics
+
+	return nil
+}
+
+func (r *DescribeLogDirsRequest) key() int16 {
+	return 35
+}
+
+func (r *DescribeLogDirsRequest) version() int16 {
+	return r.Version
+}
+
+func (r *DescribeLogDirsRequest) headerVersion() int16 {
+	return 1
+}
+
+func (r *DescribeLogDirsRequest) requiredVersion() KafkaVersion {
+	return V1_0_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go b/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go
new file mode 100644
index 0000000..411da38
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go
@@ -0,0 +1,229 @@
+package sarama
+
+import "time"
+
+type DescribeLogDirsResponse struct {
+	ThrottleTime time.Duration
+
+	// Version 0 and 1 are equal
+	// The version number is bumped to indicate that on quota violation brokers send out responses before throttling.
+	Version int16
+
+	LogDirs []DescribeLogDirsResponseDirMetadata
+}
+
+func (r *DescribeLogDirsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
+
+	if err := pe.putArrayLength(len(r.LogDirs)); err != nil {
+		return err
+	}
+
+	for _, dir := range r.LogDirs {
+		if err := dir.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (r *DescribeLogDirsResponse) decode(pd packetDecoder, version int16) error {
+	throttleTime, err := pd.getInt32()
+	if err != nil {
+		return err
+	}
+	r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond
+
+	// Decode array of DescribeLogDirsResponseDirMetadata
+	n, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+
+	r.LogDirs = make([]DescribeLogDirsResponseDirMetadata, n)
+	for i := 0; i < n; i++ {
+		dir := DescribeLogDirsResponseDirMetadata{}
+		if err := dir.decode(pd, version); err != nil {
+			return err
+		}
+		r.LogDirs[i] = dir
+	}
+
+	return nil
+}
+
+func (r *DescribeLogDirsResponse) key() int16 {
+	return 35
+}
+
+func (r *DescribeLogDirsResponse) version() int16 {
+	return r.Version
+}
+
+func (r *DescribeLogDirsResponse) headerVersion() int16 {
+	return 0
+}
+
+func (r *DescribeLogDirsResponse) requiredVersion() KafkaVersion {
+	return V1_0_0_0
+}
+
+type DescribeLogDirsResponseDirMetadata struct {
+	ErrorCode KError
+
+	// The absolute log directory path
+	Path   string
+	Topics []DescribeLogDirsResponseTopic
+}
+
+func (r *DescribeLogDirsResponseDirMetadata) encode(pe packetEncoder) error {
+	pe.putInt16(int16(r.ErrorCode))
+
+	if err := pe.putString(r.Path); err != nil {
+		return err
+	}
+
+	if err := pe.putArrayLength(len(r.Topics)); err != nil {
+		return err
+	}
+	for _, topic := range r.Topics {
+		if err := topic.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (r *DescribeLogDirsResponseDirMetadata) decode(pd packetDecoder, version int16) error {
+	errCode, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+	r.ErrorCode = KError(errCode)
+
+	path, err := pd.getString()
+	if err != nil {
+		return err
+	}
+	r.Path = path
+
+	// Decode array of DescribeLogDirsResponseTopic
+	n, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+
+	r.Topics = make([]DescribeLogDirsResponseTopic, n)
+	for i := 0; i < n; i++ {
+		t := DescribeLogDirsResponseTopic{}
+
+		if err := t.decode(pd, version); err != nil {
+			return err
+		}
+
+		r.Topics[i] = t
+	}
+
+	return nil
+}
+
+// DescribeLogDirsResponseTopic contains a topic's partitions descriptions
+type DescribeLogDirsResponseTopic struct {
+	Topic      string
+	Partitions []DescribeLogDirsResponsePartition
+}
+
+func (r *DescribeLogDirsResponseTopic) encode(pe packetEncoder) error {
+	if err := pe.putString(r.Topic); err != nil {
+		return err
+	}
+
+	if err := pe.putArrayLength(len(r.Partitions)); err != nil {
+		return err
+	}
+	for _, partition := range r.Partitions {
+		if err := partition.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (r *DescribeLogDirsResponseTopic) decode(pd packetDecoder, version int16) error {
+	t, err := pd.getString()
+	if err != nil {
+		return err
+	}
+	r.Topic = t
+
+	n, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+	r.Partitions = make([]DescribeLogDirsResponsePartition, n)
+	for i := 0; i < n; i++ {
+		p := DescribeLogDirsResponsePartition{}
+		if err := p.decode(pd, version); err != nil {
+			return err
+		}
+		r.Partitions[i] = p
+	}
+
+	return nil
+}
+
+// DescribeLogDirsResponsePartition describes a partition's log directory
+type DescribeLogDirsResponsePartition struct {
+	PartitionID int32
+
+	// The size of the log segments of the partition in bytes.
+	Size int64
+
+	// The lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or
+	// current replica's LEO (if it is the future log for the partition)
+	OffsetLag int64
+
+	// True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of
+	// the replica in the future.
+	IsTemporary bool
+}
+
+func (r *DescribeLogDirsResponsePartition) encode(pe packetEncoder) error {
+	pe.putInt32(r.PartitionID)
+	pe.putInt64(r.Size)
+	pe.putInt64(r.OffsetLag)
+	pe.putBool(r.IsTemporary)
+
+	return nil
+}
+
+func (r *DescribeLogDirsResponsePartition) decode(pd packetDecoder, version int16) error {
+	pID, err := pd.getInt32()
+	if err != nil {
+		return err
+	}
+	r.PartitionID = pID
+
+	size, err := pd.getInt64()
+	if err != nil {
+		return err
+	}
+	r.Size = size
+
+	lag, err := pd.getInt64()
+	if err != nil {
+		return err
+	}
+	r.OffsetLag = lag
+
+	isTemp, err := pd.getBool()
+	if err != nil {
+		return err
+	}
+	r.IsTemporary = isTemp
+
+	return nil
+}
diff --git a/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go
new file mode 100644
index 0000000..b5b5940
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go
@@ -0,0 +1,70 @@
+package sarama
+
+// DescribeUserScramCredentialsRequest is a request to get list of SCRAM user names
+type DescribeUserScramCredentialsRequest struct {
+	// Version 0 is currently only supported
+	Version int16
+
+	// If this is an empty array, all users will be queried
+	DescribeUsers []DescribeUserScramCredentialsRequestUser
+}
+
+// DescribeUserScramCredentialsRequestUser is a describe request about specific user name
+type DescribeUserScramCredentialsRequestUser struct {
+	Name string
+}
+
+func (r *DescribeUserScramCredentialsRequest) encode(pe packetEncoder) error {
+	pe.putCompactArrayLength(len(r.DescribeUsers))
+	for _, d := range r.DescribeUsers {
+		if err := pe.putCompactString(d.Name); err != nil {
+			return err
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (r *DescribeUserScramCredentialsRequest) decode(pd packetDecoder, version int16) error {
+	n, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+	if n == -1 {
+		n = 0
+	}
+
+	r.DescribeUsers = make([]DescribeUserScramCredentialsRequestUser, n)
+	for i := 0; i < n; i++ {
+		r.DescribeUsers[i] = DescribeUserScramCredentialsRequestUser{}
+		if r.DescribeUsers[i].Name, err = pd.getCompactString(); err != nil {
+			return err
+		}
+		if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
+	if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (r *DescribeUserScramCredentialsRequest) key() int16 {
+	return 50
+}
+
+func (r *DescribeUserScramCredentialsRequest) version() int16 {
+	return r.Version
+}
+
+func (r *DescribeUserScramCredentialsRequest) headerVersion() int16 {
+	return 2
+}
+
+func (r *DescribeUserScramCredentialsRequest) requiredVersion() KafkaVersion {
+	return V2_7_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go
new file mode 100644
index 0000000..2656c2f
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go
@@ -0,0 +1,168 @@
+package sarama
+
+import "time"
+
+type ScramMechanismType int8
+
+const (
+	SCRAM_MECHANISM_UNKNOWN ScramMechanismType = iota // 0
+	SCRAM_MECHANISM_SHA_256                           // 1
+	SCRAM_MECHANISM_SHA_512                           // 2
+)
+
+func (s ScramMechanismType) String() string {
+	switch s {
+	case 1:
+		return SASLTypeSCRAMSHA256
+	case 2:
+		return SASLTypeSCRAMSHA512
+	default:
+		return "Unknown"
+	}
+}
+
+type DescribeUserScramCredentialsResponse struct {
+	// Version 0 is currently only supported
+	Version int16
+
+	ThrottleTime time.Duration
+
+	ErrorCode    KError
+	ErrorMessage *string
+
+	Results []*DescribeUserScramCredentialsResult
+}
+
+type DescribeUserScramCredentialsResult struct {
+	User string
+
+	ErrorCode    KError
+	ErrorMessage *string
+
+	CredentialInfos []*UserScramCredentialsResponseInfo
+}
+
+type UserScramCredentialsResponseInfo struct {
+	Mechanism  ScramMechanismType
+	Iterations int32
+}
+
+func (r *DescribeUserScramCredentialsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
+
+	pe.putInt16(int16(r.ErrorCode))
+	if err := pe.putNullableCompactString(r.ErrorMessage); err != nil {
+		return err
+	}
+
+	pe.putCompactArrayLength(len(r.Results))
+	for _, u := range r.Results {
+		if err := pe.putCompactString(u.User); err != nil {
+			return err
+		}
+		pe.putInt16(int16(u.ErrorCode))
+		if err := pe.putNullableCompactString(u.ErrorMessage); err != nil {
+			return err
+		}
+
+		pe.putCompactArrayLength(len(u.CredentialInfos))
+		for _, c := range u.CredentialInfos {
+			pe.putInt8(int8(c.Mechanism))
+			pe.putInt32(c.Iterations)
+			pe.putEmptyTaggedFieldArray()
+		}
+
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+	return nil
+}
+
+func (r *DescribeUserScramCredentialsResponse) decode(pd packetDecoder, version int16) error {
+	throttleTime, err := pd.getInt32()
+	if err != nil {
+		return err
+	}
+	r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond
+
+	kerr, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+
+	r.ErrorCode = KError(kerr)
+	if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil {
+		return err
+	}
+
+	numUsers, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	if numUsers > 0 {
+		r.Results = make([]*DescribeUserScramCredentialsResult, numUsers)
+		for i := 0; i < numUsers; i++ {
+			r.Results[i] = &DescribeUserScramCredentialsResult{}
+			if r.Results[i].User, err = pd.getCompactString(); err != nil {
+				return err
+			}
+
+			errorCode, err := pd.getInt16()
+			if err != nil {
+				return err
+			}
+			r.Results[i].ErrorCode = KError(errorCode)
+			if r.Results[i].ErrorMessage, err = pd.getCompactNullableString(); err != nil {
+				return err
+			}
+
+			numCredentialInfos, err := pd.getCompactArrayLength()
+			if err != nil {
+				return err
+			}
+
+			r.Results[i].CredentialInfos = make([]*UserScramCredentialsResponseInfo, numCredentialInfos)
+			for j := 0; j < numCredentialInfos; j++ {
+				r.Results[i].CredentialInfos[j] = &UserScramCredentialsResponseInfo{}
+				scramMechanism, err := pd.getInt8()
+				if err != nil {
+					return err
+				}
+				r.Results[i].CredentialInfos[j].Mechanism = ScramMechanismType(scramMechanism)
+				if r.Results[i].CredentialInfos[j].Iterations, err = pd.getInt32(); err != nil {
+					return err
+				}
+				if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+					return err
+				}
+			}
+
+			if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if _, err = pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (r *DescribeUserScramCredentialsResponse) key() int16 {
+	return 50
+}
+
+func (r *DescribeUserScramCredentialsResponse) version() int16 {
+	return r.Version
+}
+
+func (r *DescribeUserScramCredentialsResponse) headerVersion() int16 {
+	return 2
+}
+
+func (r *DescribeUserScramCredentialsResponse) requiredVersion() KafkaVersion {
+	return V2_7_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/dev.yml b/vendor/github.com/Shopify/sarama/dev.yml
index 3f4d569..7bf9ff9 100644
--- a/vendor/github.com/Shopify/sarama/dev.yml
+++ b/vendor/github.com/Shopify/sarama/dev.yml
@@ -2,7 +2,7 @@
 
 up:
   - go:
-      version: '1.12'
+      version: '1.16'
 
 commands:
   test:
diff --git a/vendor/github.com/Shopify/sarama/docker-compose.yml b/vendor/github.com/Shopify/sarama/docker-compose.yml
new file mode 100644
index 0000000..8e9c24e
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/docker-compose.yml
@@ -0,0 +1,134 @@
+version: '3.7'
+services:
+  zookeeper-1:
+    image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      ZOOKEEPER_SERVER_ID: '1'
+      ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888'
+      ZOOKEEPER_CLIENT_PORT: '2181'
+      ZOOKEEPER_PEER_PORT: '2888'
+      ZOOKEEPER_LEADER_PORT: '3888'
+      ZOOKEEPER_INIT_LIMIT: '10'
+      ZOOKEEPER_SYNC_LIMIT: '5'
+      ZOOKEEPER_MAX_CLIENT_CONNS: '0'
+  zookeeper-2:
+    image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      ZOOKEEPER_SERVER_ID: '2'
+      ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888'
+      ZOOKEEPER_CLIENT_PORT: '2181'
+      ZOOKEEPER_PEER_PORT: '2888'
+      ZOOKEEPER_LEADER_PORT: '3888'
+      ZOOKEEPER_INIT_LIMIT: '10'
+      ZOOKEEPER_SYNC_LIMIT: '5'
+      ZOOKEEPER_MAX_CLIENT_CONNS: '0'
+  zookeeper-3:
+    image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      ZOOKEEPER_SERVER_ID: '3'
+      ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888'
+      ZOOKEEPER_CLIENT_PORT: '2181'
+      ZOOKEEPER_PEER_PORT: '2888'
+      ZOOKEEPER_LEADER_PORT: '3888'
+      ZOOKEEPER_INIT_LIMIT: '10'
+      ZOOKEEPER_SYNC_LIMIT: '5'
+      ZOOKEEPER_MAX_CLIENT_CONNS: '0'
+  kafka-1:
+    image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181'
+      KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29091'
+      KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-1:9091,LISTENER_LOCAL://localhost:29091'
+      KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL'
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT'
+      KAFKA_DEFAULT_REPLICATION_FACTOR: '2'
+      KAFKA_BROKER_ID: '1'
+      KAFKA_BROKER_RACK: '1'
+      KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000'
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000'
+      KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector'
+      KAFKA_DELETE_TOPIC_ENABLE: 'true'
+      KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'
+  kafka-2:
+    image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181'
+      KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29092'
+      KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-2:9091,LISTENER_LOCAL://localhost:29092'
+      KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL'
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT'
+      KAFKA_DEFAULT_REPLICATION_FACTOR: '2'
+      KAFKA_BROKER_ID: '2'
+      KAFKA_BROKER_RACK: '2'
+      KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000'
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000'
+      KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector'
+      KAFKA_DELETE_TOPIC_ENABLE: 'true'
+      KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'
+  kafka-3:
+    image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181'
+      KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29093'
+      KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-3:9091,LISTENER_LOCAL://localhost:29093'
+      KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL'
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT'
+      KAFKA_DEFAULT_REPLICATION_FACTOR: '2'
+      KAFKA_BROKER_ID: '3'
+      KAFKA_BROKER_RACK: '3'
+      KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000'
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000'
+      KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector'
+      KAFKA_DELETE_TOPIC_ENABLE: 'true'
+      KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'
+  kafka-4:
+    image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181'
+      KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29094'
+      KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-4:9091,LISTENER_LOCAL://localhost:29094'
+      KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL'
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT'
+      KAFKA_DEFAULT_REPLICATION_FACTOR: '2'
+      KAFKA_BROKER_ID: '4'
+      KAFKA_BROKER_RACK: '4'
+      KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000'
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000'
+      KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector'
+      KAFKA_DELETE_TOPIC_ENABLE: 'true'
+      KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'
+  kafka-5:
+    image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.1.1}'
+    restart: always
+    environment:
+      KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181'
+      KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29095'
+      KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-5:9091,LISTENER_LOCAL://localhost:29095'
+      KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL'
+      KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT'
+      KAFKA_DEFAULT_REPLICATION_FACTOR: '2'
+      KAFKA_BROKER_ID: '5'
+      KAFKA_BROKER_RACK: '5'
+      KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000'
+      KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000'
+      KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector'
+      KAFKA_DELETE_TOPIC_ENABLE: 'true'
+      KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false'
+  toxiproxy:
+    image: 'shopify/toxiproxy:2.1.4'
+    ports:
+      # The tests themselves actually start the proxies on these ports
+      - '29091:29091'
+      - '29092:29092'
+      - '29093:29093'
+      - '29094:29094'
+      - '29095:29095'
+      # This is the toxiproxy API port
+      - '8474:8474'
diff --git a/vendor/github.com/Shopify/sarama/encoder_decoder.go b/vendor/github.com/Shopify/sarama/encoder_decoder.go
index 7ce3bc0..dab54f8 100644
--- a/vendor/github.com/Shopify/sarama/encoder_decoder.go
+++ b/vendor/github.com/Shopify/sarama/encoder_decoder.go
@@ -12,6 +12,11 @@
 	encode(pe packetEncoder) error
 }
 
+type encoderWithHeader interface {
+	encoder
+	headerVersion() int16
+}
+
 // Encode takes an Encoder and turns it into bytes while potentially recording metrics.
 func encode(e encoder, metricRegistry metrics.Registry) ([]byte, error) {
 	if e == nil {
@@ -40,7 +45,7 @@
 	return realEnc.raw, nil
 }
 
-// Decoder is the interface that wraps the basic Decode method.
+// decoder is the interface that wraps the basic Decode method.
 // Anything implementing Decoder can be extracted from bytes using Kafka's encoding rules.
 type decoder interface {
 	decode(pd packetDecoder) error
@@ -50,7 +55,7 @@
 	decode(pd packetDecoder, version int16) error
 }
 
-// Decode takes bytes and a Decoder and fills the fields of the decoder from the bytes,
+// decode takes bytes and a decoder and fills the fields of the decoder from the bytes,
 // interpreted using Kafka's encoding rules.
 func decode(buf []byte, in decoder) error {
 	if buf == nil {
diff --git a/vendor/github.com/Shopify/sarama/end_txn_request.go b/vendor/github.com/Shopify/sarama/end_txn_request.go
index 2cd9b50..6635425 100644
--- a/vendor/github.com/Shopify/sarama/end_txn_request.go
+++ b/vendor/github.com/Shopify/sarama/end_txn_request.go
@@ -45,6 +45,10 @@
 	return 0
 }
 
+func (r *EndTxnRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *EndTxnRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/end_txn_response.go b/vendor/github.com/Shopify/sarama/end_txn_response.go
index 33b27e3..7639767 100644
--- a/vendor/github.com/Shopify/sarama/end_txn_response.go
+++ b/vendor/github.com/Shopify/sarama/end_txn_response.go
@@ -39,6 +39,10 @@
 	return 0
 }
 
+func (r *EndTxnResponse) headerVersion() int16 {
+	return 0
+}
+
 func (e *EndTxnResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/errors.go b/vendor/github.com/Shopify/sarama/errors.go
index c6a8be7..0fca0a3 100644
--- a/vendor/github.com/Shopify/sarama/errors.go
+++ b/vendor/github.com/Shopify/sarama/errors.go
@@ -9,6 +9,9 @@
 // or otherwise failed to respond.
 var ErrOutOfBrokers = errors.New("kafka: client has run out of available brokers to talk to (Is your cluster reachable?)")
 
+// ErrBrokerNotFound is the error returned when there's no broker found for the requested ID.
+var ErrBrokerNotFound = errors.New("kafka: broker for ID is not found")
+
 // ErrClosedClient is the error returned when a method is called on a client that has been closed.
 var ErrClosedClient = errors.New("kafka: tried to use a client that was closed")
 
@@ -49,6 +52,9 @@
 // the metadata.
 var ErrNoTopicsToUpdateMetadata = errors.New("kafka: no specific topics to update metadata")
 
+// ErrUnknownScramMechanism is returned when user tries to AlterUserScramCredentials with unknown SCRAM mechanism
+var ErrUnknownScramMechanism = errors.New("kafka: unknown SCRAM mechanism provided")
+
 // PacketEncodingError is returned from a failure while encoding a Kafka packet. This can happen, for example,
 // if you try to encode a string over 2^15 characters in length, since Kafka's encoding rules do not permit that.
 type PacketEncodingError struct {
@@ -87,13 +93,21 @@
 }
 
 func (mErr MultiError) Error() string {
-	var errString = ""
+	errString := ""
 	for _, err := range *mErr.Errors {
 		errString += err.Error() + ","
 	}
 	return errString
 }
 
+func (mErr MultiError) PrettyError() string {
+	errString := ""
+	for _, err := range *mErr.Errors {
+		errString += err.Error() + "\n"
+	}
+	return errString
+}
+
 // ErrDeleteRecords is the type of error returned when fail to delete the required records
 type ErrDeleteRecords struct {
 	MultiError
@@ -103,6 +117,14 @@
 	return "kafka server: failed to delete records " + err.MultiError.Error()
 }
 
+type ErrReassignPartitions struct {
+	MultiError
+}
+
+func (err ErrReassignPartitions) Error() string {
+	return fmt.Sprintf("failed to reassign partitions for topic: \n%s", err.MultiError.PrettyError())
+}
+
 // Numeric error codes returned by the Kafka server.
 const (
 	ErrNoError                            KError = 0
@@ -188,6 +210,13 @@
 	ErrMemberIdRequired                   KError = 79
 	ErrPreferredLeaderNotAvailable        KError = 80
 	ErrGroupMaxSizeReached                KError = 81
+	ErrFencedInstancedId                  KError = 82
+	ErrEligibleLeadersNotAvailable        KError = 83
+	ErrElectionNotNeeded                  KError = 84
+	ErrNoReassignmentInProgress           KError = 85
+	ErrGroupSubscribedToTopic             KError = 86
+	ErrInvalidRecord                      KError = 87
+	ErrUnstableOffsetCommit               KError = 88
 )
 
 func (err KError) Error() string {
@@ -360,6 +389,20 @@
 		return "kafka server: The preferred leader was not available"
 	case ErrGroupMaxSizeReached:
 		return "kafka server: Consumer group The consumer group has reached its max size. already has the configured maximum number of members."
+	case ErrFencedInstancedId:
+		return "kafka server: The broker rejected this static consumer since another consumer with the same group.instance.id has registered with a different member.id."
+	case ErrEligibleLeadersNotAvailable:
+		return "kafka server: Eligible topic partition leaders are not available."
+	case ErrElectionNotNeeded:
+		return "kafka server: Leader election not needed for topic partition."
+	case ErrNoReassignmentInProgress:
+		return "kafka server: No partition reassignment is in progress."
+	case ErrGroupSubscribedToTopic:
+		return "kafka server: Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it."
+	case ErrInvalidRecord:
+		return "kafka server: This record has failed the validation on broker and hence will be rejected."
+	case ErrUnstableOffsetCommit:
+		return "kafka server: There are unstable offsets that need to be cleared."
 	}
 
 	return fmt.Sprintf("Unknown error, how did this happen? Error code = %d", err)
diff --git a/vendor/github.com/Shopify/sarama/fetch_request.go b/vendor/github.com/Shopify/sarama/fetch_request.go
index 4db9ddd..f893aef 100644
--- a/vendor/github.com/Shopify/sarama/fetch_request.go
+++ b/vendor/github.com/Shopify/sarama/fetch_request.go
@@ -1,20 +1,41 @@
 package sarama
 
 type fetchRequestBlock struct {
-	fetchOffset int64
-	maxBytes    int32
+	Version            int16
+	currentLeaderEpoch int32
+	fetchOffset        int64
+	logStartOffset     int64
+	maxBytes           int32
 }
 
-func (b *fetchRequestBlock) encode(pe packetEncoder) error {
+func (b *fetchRequestBlock) encode(pe packetEncoder, version int16) error {
+	b.Version = version
+	if b.Version >= 9 {
+		pe.putInt32(b.currentLeaderEpoch)
+	}
 	pe.putInt64(b.fetchOffset)
+	if b.Version >= 5 {
+		pe.putInt64(b.logStartOffset)
+	}
 	pe.putInt32(b.maxBytes)
 	return nil
 }
 
-func (b *fetchRequestBlock) decode(pd packetDecoder) (err error) {
+func (b *fetchRequestBlock) decode(pd packetDecoder, version int16) (err error) {
+	b.Version = version
+	if b.Version >= 9 {
+		if b.currentLeaderEpoch, err = pd.getInt32(); err != nil {
+			return err
+		}
+	}
 	if b.fetchOffset, err = pd.getInt64(); err != nil {
 		return err
 	}
+	if b.Version >= 5 {
+		if b.logStartOffset, err = pd.getInt64(); err != nil {
+			return err
+		}
+	}
 	if b.maxBytes, err = pd.getInt32(); err != nil {
 		return err
 	}
@@ -25,12 +46,16 @@
 // https://issues.apache.org/jira/browse/KAFKA-2063 for a discussion of the issues leading up to that.  The KIP is at
 // https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes
 type FetchRequest struct {
-	MaxWaitTime int32
-	MinBytes    int32
-	MaxBytes    int32
-	Version     int16
-	Isolation   IsolationLevel
-	blocks      map[string]map[int32]*fetchRequestBlock
+	MaxWaitTime  int32
+	MinBytes     int32
+	MaxBytes     int32
+	Version      int16
+	Isolation    IsolationLevel
+	SessionID    int32
+	SessionEpoch int32
+	blocks       map[string]map[int32]*fetchRequestBlock
+	forgotten    map[string][]int32
+	RackID       string
 }
 
 type IsolationLevel int8
@@ -50,6 +75,10 @@
 	if r.Version >= 4 {
 		pe.putInt8(int8(r.Isolation))
 	}
+	if r.Version >= 7 {
+		pe.putInt32(r.SessionID)
+		pe.putInt32(r.SessionEpoch)
+	}
 	err = pe.putArrayLength(len(r.blocks))
 	if err != nil {
 		return err
@@ -65,17 +94,44 @@
 		}
 		for partition, block := range blocks {
 			pe.putInt32(partition)
-			err = block.encode(pe)
+			err = block.encode(pe, r.Version)
 			if err != nil {
 				return err
 			}
 		}
 	}
+	if r.Version >= 7 {
+		err = pe.putArrayLength(len(r.forgotten))
+		if err != nil {
+			return err
+		}
+		for topic, partitions := range r.forgotten {
+			err = pe.putString(topic)
+			if err != nil {
+				return err
+			}
+			err = pe.putArrayLength(len(partitions))
+			if err != nil {
+				return err
+			}
+			for _, partition := range partitions {
+				pe.putInt32(partition)
+			}
+		}
+	}
+	if r.Version >= 11 {
+		err = pe.putString(r.RackID)
+		if err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
 func (r *FetchRequest) decode(pd packetDecoder, version int16) (err error) {
 	r.Version = version
+
 	if _, err = pd.getInt32(); err != nil {
 		return err
 	}
@@ -97,6 +153,16 @@
 		}
 		r.Isolation = IsolationLevel(isolation)
 	}
+	if r.Version >= 7 {
+		r.SessionID, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+		r.SessionEpoch, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	}
 	topicCount, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -121,12 +187,47 @@
 				return err
 			}
 			fetchBlock := &fetchRequestBlock{}
-			if err = fetchBlock.decode(pd); err != nil {
+			if err = fetchBlock.decode(pd, r.Version); err != nil {
 				return err
 			}
 			r.blocks[topic][partition] = fetchBlock
 		}
 	}
+
+	if r.Version >= 7 {
+		forgottenCount, err := pd.getArrayLength()
+		if err != nil {
+			return err
+		}
+		r.forgotten = make(map[string][]int32)
+		for i := 0; i < forgottenCount; i++ {
+			topic, err := pd.getString()
+			if err != nil {
+				return err
+			}
+			partitionCount, err := pd.getArrayLength()
+			if err != nil {
+				return err
+			}
+			r.forgotten[topic] = make([]int32, partitionCount)
+
+			for j := 0; j < partitionCount; j++ {
+				partition, err := pd.getInt32()
+				if err != nil {
+					return err
+				}
+				r.forgotten[topic][j] = partition
+			}
+		}
+	}
+
+	if r.Version >= 11 {
+		r.RackID, err = pd.getString()
+		if err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
@@ -138,18 +239,34 @@
 	return r.Version
 }
 
+func (r *FetchRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *FetchRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
+	case 0:
+		return MinVersion
 	case 1:
 		return V0_9_0_0
 	case 2:
 		return V0_10_0_0
 	case 3:
 		return V0_10_1_0
-	case 4:
+	case 4, 5:
 		return V0_11_0_0
+	case 6:
+		return V1_0_0_0
+	case 7:
+		return V1_1_0_0
+	case 8:
+		return V2_0_0_0
+	case 9, 10:
+		return V2_1_0_0
+	case 11:
+		return V2_3_0_0
 	default:
-		return MinVersion
+		return MaxVersion
 	}
 }
 
@@ -158,13 +275,21 @@
 		r.blocks = make(map[string]map[int32]*fetchRequestBlock)
 	}
 
+	if r.Version >= 7 && r.forgotten == nil {
+		r.forgotten = make(map[string][]int32)
+	}
+
 	if r.blocks[topic] == nil {
 		r.blocks[topic] = make(map[int32]*fetchRequestBlock)
 	}
 
 	tmp := new(fetchRequestBlock)
+	tmp.Version = r.Version
 	tmp.maxBytes = maxBytes
 	tmp.fetchOffset = fetchOffset
+	if r.Version >= 9 {
+		tmp.currentLeaderEpoch = int32(-1)
+	}
 
 	r.blocks[topic][partitionID] = tmp
 }
diff --git a/vendor/github.com/Shopify/sarama/fetch_response.go b/vendor/github.com/Shopify/sarama/fetch_response.go
index 3afc187..54b8828 100644
--- a/vendor/github.com/Shopify/sarama/fetch_response.go
+++ b/vendor/github.com/Shopify/sarama/fetch_response.go
@@ -30,13 +30,15 @@
 }
 
 type FetchResponseBlock struct {
-	Err                 KError
-	HighWaterMarkOffset int64
-	LastStableOffset    int64
-	AbortedTransactions []*AbortedTransaction
-	Records             *Records // deprecated: use FetchResponseBlock.RecordsSet
-	RecordsSet          []*Records
-	Partial             bool
+	Err                  KError
+	HighWaterMarkOffset  int64
+	LastStableOffset     int64
+	LogStartOffset       int64
+	AbortedTransactions  []*AbortedTransaction
+	PreferredReadReplica int32
+	Records              *Records // deprecated: use FetchResponseBlock.RecordsSet
+	RecordsSet           []*Records
+	Partial              bool
 }
 
 func (b *FetchResponseBlock) decode(pd packetDecoder, version int16) (err error) {
@@ -57,6 +59,13 @@
 			return err
 		}
 
+		if version >= 5 {
+			b.LogStartOffset, err = pd.getInt64()
+			if err != nil {
+				return err
+			}
+		}
+
 		numTransact, err := pd.getArrayLength()
 		if err != nil {
 			return err
@@ -75,6 +84,15 @@
 		}
 	}
 
+	if version >= 11 {
+		b.PreferredReadReplica, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	} else {
+		b.PreferredReadReplica = -1
+	}
+
 	recordsSize, err := pd.getInt32()
 	if err != nil {
 		return err
@@ -166,6 +184,10 @@
 	if version >= 4 {
 		pe.putInt64(b.LastStableOffset)
 
+		if version >= 5 {
+			pe.putInt64(b.LogStartOffset)
+		}
+
 		if err = pe.putArrayLength(len(b.AbortedTransactions)); err != nil {
 			return err
 		}
@@ -176,6 +198,10 @@
 		}
 	}
 
+	if version >= 11 {
+		pe.putInt32(b.PreferredReadReplica)
+	}
+
 	pe.push(&lengthField{})
 	for _, records := range b.RecordsSet {
 		err = records.encode(pe)
@@ -200,7 +226,9 @@
 type FetchResponse struct {
 	Blocks        map[string]map[int32]*FetchResponseBlock
 	ThrottleTime  time.Duration
-	Version       int16 // v1 requires 0.9+, v2 requires 0.10+
+	ErrorCode     int16
+	SessionID     int32
+	Version       int16
 	LogAppendTime bool
 	Timestamp     time.Time
 }
@@ -216,6 +244,17 @@
 		r.ThrottleTime = time.Duration(throttle) * time.Millisecond
 	}
 
+	if r.Version >= 7 {
+		r.ErrorCode, err = pd.getInt16()
+		if err != nil {
+			return err
+		}
+		r.SessionID, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	}
+
 	numTopics, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -258,6 +297,11 @@
 		pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
 	}
 
+	if r.Version >= 7 {
+		pe.putInt16(r.ErrorCode)
+		pe.putInt32(r.SessionID)
+	}
+
 	err = pe.putArrayLength(len(r.Blocks))
 	if err != nil {
 		return err
@@ -281,7 +325,6 @@
 				return err
 			}
 		}
-
 	}
 	return nil
 }
@@ -294,18 +337,34 @@
 	return r.Version
 }
 
+func (r *FetchResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *FetchResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
+	case 0:
+		return MinVersion
 	case 1:
 		return V0_9_0_0
 	case 2:
 		return V0_10_0_0
 	case 3:
 		return V0_10_1_0
-	case 4:
+	case 4, 5:
 		return V0_11_0_0
+	case 6:
+		return V1_0_0_0
+	case 7:
+		return V1_1_0_0
+	case 8:
+		return V2_0_0_0
+	case 9, 10:
+		return V2_1_0_0
+	case 11:
+		return V2_3_0_0
 	default:
-		return MinVersion
+		return MaxVersion
 	}
 }
 
diff --git a/vendor/github.com/Shopify/sarama/find_coordinator_request.go b/vendor/github.com/Shopify/sarama/find_coordinator_request.go
index ff2ad20..597bcbf 100644
--- a/vendor/github.com/Shopify/sarama/find_coordinator_request.go
+++ b/vendor/github.com/Shopify/sarama/find_coordinator_request.go
@@ -51,6 +51,10 @@
 	return f.Version
 }
 
+func (r *FindCoordinatorRequest) headerVersion() int16 {
+	return 1
+}
+
 func (f *FindCoordinatorRequest) requiredVersion() KafkaVersion {
 	switch f.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/find_coordinator_response.go b/vendor/github.com/Shopify/sarama/find_coordinator_response.go
index 9c900e8..83a648a 100644
--- a/vendor/github.com/Shopify/sarama/find_coordinator_response.go
+++ b/vendor/github.com/Shopify/sarama/find_coordinator_response.go
@@ -82,6 +82,10 @@
 	return f.Version
 }
 
+func (r *FindCoordinatorResponse) headerVersion() int16 {
+	return 0
+}
+
 func (f *FindCoordinatorResponse) requiredVersion() KafkaVersion {
 	switch f.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/go.mod b/vendor/github.com/Shopify/sarama/go.mod
index 8c45155..ccbd8e2 100644
--- a/vendor/github.com/Shopify/sarama/go.mod
+++ b/vendor/github.com/Shopify/sarama/go.mod
@@ -1,24 +1,28 @@
 module github.com/Shopify/sarama
 
+go 1.13
+
 require (
-	github.com/DataDog/zstd v1.3.6-0.20190409195224-796139022798
 	github.com/Shopify/toxiproxy v2.1.4+incompatible
 	github.com/davecgh/go-spew v1.1.1
-	github.com/eapache/go-resiliency v1.1.0
+	github.com/eapache/go-resiliency v1.2.0
 	github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21
 	github.com/eapache/queue v1.1.0
-	github.com/golang/snappy v0.0.1 // indirect
-	github.com/hashicorp/go-uuid v1.0.1 // indirect
-	github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03
-	github.com/pierrec/lz4 v0.0.0-20190327172049-315a67e90e41
-	github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a
-	github.com/stretchr/testify v1.3.0
-	github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c
-	github.com/xdg/stringprep v1.0.0 // indirect
-	golang.org/x/crypto v0.0.0-20190404164418-38d8ce5564a5 // indirect
-	golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3
-	gopkg.in/jcmturner/aescts.v1 v1.0.1 // indirect
-	gopkg.in/jcmturner/dnsutils.v1 v1.0.1 // indirect
-	gopkg.in/jcmturner/gokrb5.v7 v7.2.3
-	gopkg.in/jcmturner/rpc.v1 v1.1.0 // indirect
+	github.com/fortytw2/leaktest v1.3.0
+	github.com/frankban/quicktest v1.11.3 // indirect
+	github.com/jcmturner/gofork v1.0.0
+	github.com/jcmturner/gokrb5/v8 v8.4.2
+	github.com/klauspost/compress v1.12.2
+	github.com/kr/text v0.2.0 // indirect
+	github.com/pierrec/lz4 v2.6.0+incompatible
+	github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475
+	github.com/stretchr/testify v1.7.0
+	github.com/xdg/scram v1.0.3
+	github.com/xdg/stringprep v1.0.3 // indirect
+	golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e // indirect
+	golang.org/x/net v0.0.0-20210614182718-04defd469f4e
+	golang.org/x/text v0.3.6 // indirect
+	golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect
+	gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect
+	gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect
 )
diff --git a/vendor/github.com/Shopify/sarama/go.sum b/vendor/github.com/Shopify/sarama/go.sum
index 4dbc6d2..a497760 100644
--- a/vendor/github.com/Shopify/sarama/go.sum
+++ b/vendor/github.com/Shopify/sarama/go.sum
@@ -1,51 +1,93 @@
-github.com/DataDog/zstd v1.3.6-0.20190409195224-796139022798 h1:2T/jmrHeTezcCM58lvEQXs0UpQJCo5SoGAcg+mbSTIg=
-github.com/DataDog/zstd v1.3.6-0.20190409195224-796139022798/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo=
 github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc=
 github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
+github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
 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/eapache/go-resiliency v1.1.0 h1:1NtRmCAqadE2FN4ZcN6g90TP3uk8cg9rn9eNK2197aU=
-github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
+github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q=
+github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
 github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw=
 github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU=
 github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc=
 github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I=
-github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4=
-github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE=
-github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03 h1:FUwcHNlEqkqLjLBdCp5PRlCFijNjvcYANOZXzCfXwCM=
-github.com/jcmturner/gofork v0.0.0-20190328161633-dc7c13fece03/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
-github.com/pierrec/lz4 v0.0.0-20190327172049-315a67e90e41 h1:GeinFsrjWz97fAxVUEd748aV0cYL+I6k44gFJTCVvpU=
-github.com/pierrec/lz4 v0.0.0-20190327172049-315a67e90e41/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
-github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
+github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
+github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g=
+github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY=
+github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k=
+github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA=
+github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
+github.com/google/go-cmp v0.5.4 h1:L8R9j+yAqZuZjsqh/z+F1NCffTKKLShY6zXTItVIZ8M=
+github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ=
+github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4=
+github.com/gorilla/sessions v1.2.1 h1:DHd3rPN5lE3Ts3D8rKkQ8x/0kqfeNmBAaiSi+o7FsgI=
+github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM=
+github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE=
+github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
+github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8=
+github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs=
+github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo=
+github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM=
+github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8=
+github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o=
+github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o=
+github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg=
+github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA=
+github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc=
+github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY=
+github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc=
+github.com/klauspost/compress v1.12.2 h1:2KCfW3I9M7nSc5wOqXAlW2v2U6v+w6cbjvbfp+OykW8=
+github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg=
+github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI=
+github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
+github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
+github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
+github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
+github.com/pierrec/lz4 v2.6.0+incompatible h1:Ix9yFKn1nSPBLFl/yZknTp8TU5G4Ps0JDmguYK6iH1A=
+github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
 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/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a h1:9ZKAASQSHhDYGoxY8uLVpewe1GDZ2vu2Tr/vTdVAkFQ=
-github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
+github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
 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=
-github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c h1:u40Z8hqBAAQyv+vATcGgV0YCnDjqSL7/q/JyPhhJSPk=
-github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
-github.com/xdg/stringprep v1.0.0 h1:d9X0esnoa3dFsV0FG35rAT0RIhYFlPq7MiP+DW89La0=
-github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
-golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY=
+github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/xdg/scram v1.0.3 h1:nTadYh2Fs4BK2xdldEa2g5bbaZp0/+1nJMMPtPxS/to=
+github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I=
+github.com/xdg/stringprep v1.0.3 h1:cmL5Enob4W83ti/ZHuZLuKD/xqJfus4fVPwE+/BDm+4=
+github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y=
 golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190404164418-38d8ce5564a5 h1:bselrhR0Or1vomJZC8ZIjWtbDmn9OYFLX5Ik9alpJpE=
-golang.org/x/crypto v0.0.0-20190404164418-38d8ce5564a5/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
-golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3 h1:0GoQqolDA55aaLxZyTzK/Y2ePZzZTUrRacwib7cNsYQ=
+golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e h1:gsTQYXdTw2Gq7RBsWvlQ91b+aEQ6bXFUngBGuR8sPpI=
+golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
 golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
+golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a h1:njMmldwFTyDLqonHMagNXKBWptTBeDZOdblgaDsNEGQ=
+golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e h1:XpT3nA5TvE525Ne3hInMh6+GETgn27Zfm9dxsThnX2Q=
+golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
 golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
+golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
 golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
-gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw=
-gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM=
-gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q=
-gopkg.in/jcmturner/gokrb5.v7 v7.2.3 h1:hHMV/yKPwMnJhPuPx7pH2Uw/3Qyf+thJYlisUc44010=
-gopkg.in/jcmturner/gokrb5.v7 v7.2.3/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM=
-gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU=
-gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8=
+golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M=
+golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE=
+golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
+gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
diff --git a/vendor/github.com/Shopify/sarama/gssapi_kerberos.go b/vendor/github.com/Shopify/sarama/gssapi_kerberos.go
index 49b632d..ab8b701 100644
--- a/vendor/github.com/Shopify/sarama/gssapi_kerberos.go
+++ b/vendor/github.com/Shopify/sarama/gssapi_kerberos.go
@@ -2,17 +2,20 @@
 
 import (
 	"encoding/binary"
+	"errors"
 	"fmt"
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/gssapi"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
 	"io"
+	"math"
 	"strings"
 	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/gssapi"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
 )
 
 const (
@@ -33,6 +36,7 @@
 	Username           string
 	Password           string
 	Realm              string
+	DisablePAFXFAST    bool
 }
 
 type GSSAPIKerberosAuth struct {
@@ -51,15 +55,14 @@
 	Destroy()
 }
 
-/*
-*
-* Appends length in big endian before payload, and send it to kafka
-*
- */
-
+// writePackage appends length in big endian before the payload, and sends it to kafka
 func (krbAuth *GSSAPIKerberosAuth) writePackage(broker *Broker, payload []byte) (int, error) {
-	length := len(payload)
-	finalPackage := make([]byte, length+4) //4 byte length header + payload
+	length := uint64(len(payload))
+	size := length + 4 // 4 byte length header + payload
+	if size > math.MaxInt32 {
+		return 0, errors.New("payload too large, will overflow int32")
+	}
+	finalPackage := make([]byte, size)
 	copy(finalPackage[4:], payload)
 	binary.BigEndian.PutUint32(finalPackage, uint32(length))
 	bytes, err := broker.conn.Write(finalPackage)
@@ -69,12 +72,7 @@
 	return bytes, nil
 }
 
-/*
-*
-* Read length (4 bytes) and then read the payload
-*
- */
-
+// readPackage reads payload length (4 bytes) and then reads the payload into []byte
 func (krbAuth *GSSAPIKerberosAuth) readPackage(broker *Broker) ([]byte, int, error) {
 	bytesRead := 0
 	lengthInBytes := make([]byte, 4)
@@ -152,7 +150,7 @@
 *
  */
 func (krbAuth *GSSAPIKerberosAuth) appendGSSAPIHeader(payload []byte) ([]byte, error) {
-	oidBytes, err := asn1.Marshal(gssapi.OID(gssapi.OIDKRB5))
+	oidBytes, err := asn1.Marshal(gssapi.OIDKRB5.OID())
 	if err != nil {
 		return nil, err
 	}
@@ -199,7 +197,6 @@
 
 /* This does the handshake for authorization */
 func (krbAuth *GSSAPIKerberosAuth) Authorize(broker *Broker) error {
-
 	kerberosClient, err := krbAuth.NewKerberosClientFunc(krbAuth.Config)
 	if err != nil {
 		Logger.Printf("Kerberos client error: %s", err)
@@ -218,7 +215,6 @@
 	spn := fmt.Sprintf("%s/%s", broker.conf.Net.SASL.GSSAPI.ServiceName, host)
 
 	ticket, encKey, err := kerberosClient.GetServiceTicket(spn)
-
 	if err != nil {
 		Logger.Printf("Error getting Kerberos service ticket : %s", err)
 		return err
@@ -242,7 +238,7 @@
 		}
 		broker.updateOutgoingCommunicationMetrics(bytesWritten)
 		if krbAuth.step == GSS_API_VERIFY {
-			var bytesRead = 0
+			bytesRead := 0
 			receivedBytes, bytesRead, err = krbAuth.readPackage(broker)
 			requestLatency := time.Since(requestTime)
 			broker.updateIncomingCommunicationMetrics(bytesRead, requestLatency)
diff --git a/vendor/github.com/Shopify/sarama/heartbeat_request.go b/vendor/github.com/Shopify/sarama/heartbeat_request.go
index ce49c47..e9d9af1 100644
--- a/vendor/github.com/Shopify/sarama/heartbeat_request.go
+++ b/vendor/github.com/Shopify/sarama/heartbeat_request.go
@@ -42,6 +42,10 @@
 	return 0
 }
 
+func (r *HeartbeatRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *HeartbeatRequest) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/heartbeat_response.go b/vendor/github.com/Shopify/sarama/heartbeat_response.go
index 766f5fd..577ab72 100644
--- a/vendor/github.com/Shopify/sarama/heartbeat_response.go
+++ b/vendor/github.com/Shopify/sarama/heartbeat_response.go
@@ -27,6 +27,10 @@
 	return 0
 }
 
+func (r *HeartbeatResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *HeartbeatResponse) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go b/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go
new file mode 100644
index 0000000..c4d05a9
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go
@@ -0,0 +1,173 @@
+package sarama
+
+type IncrementalAlterConfigsOperation int8
+
+const (
+	IncrementalAlterConfigsOperationSet IncrementalAlterConfigsOperation = iota
+	IncrementalAlterConfigsOperationDelete
+	IncrementalAlterConfigsOperationAppend
+	IncrementalAlterConfigsOperationSubtract
+)
+
+// IncrementalAlterConfigsRequest is an incremental alter config request type
+type IncrementalAlterConfigsRequest struct {
+	Resources    []*IncrementalAlterConfigsResource
+	ValidateOnly bool
+}
+
+type IncrementalAlterConfigsResource struct {
+	Type          ConfigResourceType
+	Name          string
+	ConfigEntries map[string]IncrementalAlterConfigsEntry
+}
+
+type IncrementalAlterConfigsEntry struct {
+	Operation IncrementalAlterConfigsOperation
+	Value     *string
+}
+
+func (a *IncrementalAlterConfigsRequest) encode(pe packetEncoder) error {
+	if err := pe.putArrayLength(len(a.Resources)); err != nil {
+		return err
+	}
+
+	for _, r := range a.Resources {
+		if err := r.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	pe.putBool(a.ValidateOnly)
+	return nil
+}
+
+func (a *IncrementalAlterConfigsRequest) decode(pd packetDecoder, version int16) error {
+	resourceCount, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+
+	a.Resources = make([]*IncrementalAlterConfigsResource, resourceCount)
+	for i := range a.Resources {
+		r := &IncrementalAlterConfigsResource{}
+		err = r.decode(pd, version)
+		if err != nil {
+			return err
+		}
+		a.Resources[i] = r
+	}
+
+	validateOnly, err := pd.getBool()
+	if err != nil {
+		return err
+	}
+
+	a.ValidateOnly = validateOnly
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsResource) encode(pe packetEncoder) error {
+	pe.putInt8(int8(a.Type))
+
+	if err := pe.putString(a.Name); err != nil {
+		return err
+	}
+
+	if err := pe.putArrayLength(len(a.ConfigEntries)); err != nil {
+		return err
+	}
+
+	for name, e := range a.ConfigEntries {
+		if err := pe.putString(name); err != nil {
+			return err
+		}
+
+		if err := e.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsResource) decode(pd packetDecoder, version int16) error {
+	t, err := pd.getInt8()
+	if err != nil {
+		return err
+	}
+	a.Type = ConfigResourceType(t)
+
+	name, err := pd.getString()
+	if err != nil {
+		return err
+	}
+	a.Name = name
+
+	n, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+
+	if n > 0 {
+		a.ConfigEntries = make(map[string]IncrementalAlterConfigsEntry, n)
+		for i := 0; i < n; i++ {
+			name, err := pd.getString()
+			if err != nil {
+				return err
+			}
+
+			var v IncrementalAlterConfigsEntry
+
+			if err := v.decode(pd, version); err != nil {
+				return err
+			}
+
+			a.ConfigEntries[name] = v
+		}
+	}
+	return err
+}
+
+func (a *IncrementalAlterConfigsEntry) encode(pe packetEncoder) error {
+	pe.putInt8(int8(a.Operation))
+
+	if err := pe.putNullableString(a.Value); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsEntry) decode(pd packetDecoder, version int16) error {
+	t, err := pd.getInt8()
+	if err != nil {
+		return err
+	}
+	a.Operation = IncrementalAlterConfigsOperation(t)
+
+	s, err := pd.getNullableString()
+	if err != nil {
+		return err
+	}
+
+	a.Value = s
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsRequest) key() int16 {
+	return 44
+}
+
+func (a *IncrementalAlterConfigsRequest) version() int16 {
+	return 0
+}
+
+func (a *IncrementalAlterConfigsRequest) headerVersion() int16 {
+	return 1
+}
+
+func (a *IncrementalAlterConfigsRequest) requiredVersion() KafkaVersion {
+	return V2_3_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go b/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go
new file mode 100644
index 0000000..3e8c450
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go
@@ -0,0 +1,66 @@
+package sarama
+
+import "time"
+
+// IncrementalAlterConfigsResponse is a response type for incremental alter config
+type IncrementalAlterConfigsResponse struct {
+	ThrottleTime time.Duration
+	Resources    []*AlterConfigsResourceResponse
+}
+
+func (a *IncrementalAlterConfigsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(int32(a.ThrottleTime / time.Millisecond))
+
+	if err := pe.putArrayLength(len(a.Resources)); err != nil {
+		return err
+	}
+
+	for _, v := range a.Resources {
+		if err := v.encode(pe); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsResponse) decode(pd packetDecoder, version int16) error {
+	throttleTime, err := pd.getInt32()
+	if err != nil {
+		return err
+	}
+	a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond
+
+	responseCount, err := pd.getArrayLength()
+	if err != nil {
+		return err
+	}
+
+	a.Resources = make([]*AlterConfigsResourceResponse, responseCount)
+
+	for i := range a.Resources {
+		a.Resources[i] = new(AlterConfigsResourceResponse)
+
+		if err := a.Resources[i].decode(pd, version); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func (a *IncrementalAlterConfigsResponse) key() int16 {
+	return 44
+}
+
+func (a *IncrementalAlterConfigsResponse) version() int16 {
+	return 0
+}
+
+func (a *IncrementalAlterConfigsResponse) headerVersion() int16 {
+	return 0
+}
+
+func (a *IncrementalAlterConfigsResponse) requiredVersion() KafkaVersion {
+	return V2_3_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/init_producer_id_request.go b/vendor/github.com/Shopify/sarama/init_producer_id_request.go
index 8ceb6c2..6894443 100644
--- a/vendor/github.com/Shopify/sarama/init_producer_id_request.go
+++ b/vendor/github.com/Shopify/sarama/init_producer_id_request.go
@@ -38,6 +38,10 @@
 	return 0
 }
 
+func (i *InitProducerIDRequest) headerVersion() int16 {
+	return 1
+}
+
 func (i *InitProducerIDRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/init_producer_id_response.go b/vendor/github.com/Shopify/sarama/init_producer_id_response.go
index 1b32eb0..3e1242b 100644
--- a/vendor/github.com/Shopify/sarama/init_producer_id_response.go
+++ b/vendor/github.com/Shopify/sarama/init_producer_id_response.go
@@ -50,6 +50,10 @@
 	return 0
 }
 
+func (i *InitProducerIDResponse) headerVersion() int16 {
+	return 0
+}
+
 func (i *InitProducerIDResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/interceptors.go b/vendor/github.com/Shopify/sarama/interceptors.go
new file mode 100644
index 0000000..d0d33e5
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/interceptors.go
@@ -0,0 +1,43 @@
+package sarama
+
+// ProducerInterceptor allows you to intercept (and possibly mutate) the records
+// received by the producer before they are published to the Kafka cluster.
+// https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors#KIP42:AddProducerandConsumerInterceptors-Motivation
+type ProducerInterceptor interface {
+
+	// OnSend is called when the producer message is intercepted. Please avoid
+	// modifying the message until it's safe to do so, as this is _not_ a copy
+	// of the message.
+	OnSend(*ProducerMessage)
+}
+
+// ConsumerInterceptor allows you to intercept (and possibly mutate) the records
+// received by the consumer before they are sent to the messages channel.
+// https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors#KIP42:AddProducerandConsumerInterceptors-Motivation
+type ConsumerInterceptor interface {
+
+	// OnConsume is called when the consumed message is intercepted. Please
+	// avoid modifying the message until it's safe to do so, as this is _not_ a
+	// copy of the message.
+	OnConsume(*ConsumerMessage)
+}
+
+func (msg *ProducerMessage) safelyApplyInterceptor(interceptor ProducerInterceptor) {
+	defer func() {
+		if r := recover(); r != nil {
+			Logger.Printf("Error when calling producer interceptor: %s, %w\n", interceptor, r)
+		}
+	}()
+
+	interceptor.OnSend(msg)
+}
+
+func (msg *ConsumerMessage) safelyApplyInterceptor(interceptor ConsumerInterceptor) {
+	defer func() {
+		if r := recover(); r != nil {
+			Logger.Printf("Error when calling consumer interceptor: %s, %w\n", interceptor, r)
+		}
+	}()
+
+	interceptor.OnConsume(msg)
+}
diff --git a/vendor/github.com/Shopify/sarama/join_group_request.go b/vendor/github.com/Shopify/sarama/join_group_request.go
index 97e9299..3734e82 100644
--- a/vendor/github.com/Shopify/sarama/join_group_request.go
+++ b/vendor/github.com/Shopify/sarama/join_group_request.go
@@ -134,6 +134,10 @@
 	return r.Version
 }
 
+func (r *JoinGroupRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *JoinGroupRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 2:
diff --git a/vendor/github.com/Shopify/sarama/join_group_response.go b/vendor/github.com/Shopify/sarama/join_group_response.go
index 5752acc..54b0a45 100644
--- a/vendor/github.com/Shopify/sarama/join_group_response.go
+++ b/vendor/github.com/Shopify/sarama/join_group_response.go
@@ -123,6 +123,10 @@
 	return r.Version
 }
 
+func (r *JoinGroupResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *JoinGroupResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 2:
diff --git a/vendor/github.com/Shopify/sarama/kerberos_client.go b/vendor/github.com/Shopify/sarama/kerberos_client.go
index 91b998f..01a5319 100644
--- a/vendor/github.com/Shopify/sarama/kerberos_client.go
+++ b/vendor/github.com/Shopify/sarama/kerberos_client.go
@@ -1,10 +1,10 @@
 package sarama
 
 import (
-	krb5client "gopkg.in/jcmturner/gokrb5.v7/client"
-	krb5config "gopkg.in/jcmturner/gokrb5.v7/config"
-	"gopkg.in/jcmturner/gokrb5.v7/keytab"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
+	krb5client "github.com/jcmturner/gokrb5/v8/client"
+	krb5config "github.com/jcmturner/gokrb5/v8/config"
+	"github.com/jcmturner/gokrb5/v8/keytab"
+	"github.com/jcmturner/gokrb5/v8/types"
 )
 
 type KerberosGoKrb5Client struct {
@@ -19,14 +19,9 @@
 	return c.Credentials.CName()
 }
 
-/*
-*
-* Create kerberos client used to obtain TGT and TGS tokens
-* used gokrb5 library, which is a pure go kerberos client with
-* some GSS-API capabilities, and SPNEGO support. Kafka does not use SPNEGO
-* it uses pure Kerberos 5 solution (RFC-4121 and RFC-4120).
-*
- */
+// NewKerberosClient creates kerberos client used to obtain TGT and TGS tokens.
+// It uses pure go Kerberos 5 solution (RFC-4121 and RFC-4120).
+// uses gokrb5 library underlying which is a pure go kerberos client with some GSS-API capabilities.
 func NewKerberosClient(config *GSSAPIConfig) (KerberosClient, error) {
 	cfg, err := krb5config.Load(config.KerberosConfigPath)
 	if err != nil {
@@ -42,10 +37,10 @@
 		if err != nil {
 			return nil, err
 		}
-		client = krb5client.NewClientWithKeytab(config.Username, config.Realm, kt, cfg)
+		client = krb5client.NewWithKeytab(config.Username, config.Realm, kt, cfg, krb5client.DisablePAFXFAST(config.DisablePAFXFAST))
 	} else {
-		client = krb5client.NewClientWithPassword(config.Username,
-			config.Realm, config.Password, cfg)
+		client = krb5client.NewWithPassword(config.Username,
+			config.Realm, config.Password, cfg, krb5client.DisablePAFXFAST(config.DisablePAFXFAST))
 	}
 	return &KerberosGoKrb5Client{*client}, nil
 }
diff --git a/vendor/github.com/Shopify/sarama/leave_group_request.go b/vendor/github.com/Shopify/sarama/leave_group_request.go
index e177427..d7789b6 100644
--- a/vendor/github.com/Shopify/sarama/leave_group_request.go
+++ b/vendor/github.com/Shopify/sarama/leave_group_request.go
@@ -35,6 +35,10 @@
 	return 0
 }
 
+func (r *LeaveGroupRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *LeaveGroupRequest) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/leave_group_response.go b/vendor/github.com/Shopify/sarama/leave_group_response.go
index d60c626..25f8d5e 100644
--- a/vendor/github.com/Shopify/sarama/leave_group_response.go
+++ b/vendor/github.com/Shopify/sarama/leave_group_response.go
@@ -27,6 +27,10 @@
 	return 0
 }
 
+func (r *LeaveGroupResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *LeaveGroupResponse) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/list_groups_request.go b/vendor/github.com/Shopify/sarama/list_groups_request.go
index 3b16abf..4553b2d 100644
--- a/vendor/github.com/Shopify/sarama/list_groups_request.go
+++ b/vendor/github.com/Shopify/sarama/list_groups_request.go
@@ -1,7 +1,6 @@
 package sarama
 
-type ListGroupsRequest struct {
-}
+type ListGroupsRequest struct{}
 
 func (r *ListGroupsRequest) encode(pe packetEncoder) error {
 	return nil
@@ -19,6 +18,10 @@
 	return 0
 }
 
+func (r *ListGroupsRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *ListGroupsRequest) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/list_groups_response.go b/vendor/github.com/Shopify/sarama/list_groups_response.go
index 56115d4..777bae7 100644
--- a/vendor/github.com/Shopify/sarama/list_groups_response.go
+++ b/vendor/github.com/Shopify/sarama/list_groups_response.go
@@ -64,6 +64,10 @@
 	return 0
 }
 
+func (r *ListGroupsResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *ListGroupsResponse) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go b/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go
new file mode 100644
index 0000000..c1ffa9b
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go
@@ -0,0 +1,98 @@
+package sarama
+
+type ListPartitionReassignmentsRequest struct {
+	TimeoutMs int32
+	blocks    map[string][]int32
+	Version   int16
+}
+
+func (r *ListPartitionReassignmentsRequest) encode(pe packetEncoder) error {
+	pe.putInt32(r.TimeoutMs)
+
+	pe.putCompactArrayLength(len(r.blocks))
+
+	for topic, partitions := range r.blocks {
+		if err := pe.putCompactString(topic); err != nil {
+			return err
+		}
+
+		if err := pe.putCompactInt32Array(partitions); err != nil {
+			return err
+		}
+
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+
+	return nil
+}
+
+func (r *ListPartitionReassignmentsRequest) decode(pd packetDecoder, version int16) (err error) {
+	r.Version = version
+
+	if r.TimeoutMs, err = pd.getInt32(); err != nil {
+		return err
+	}
+
+	topicCount, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+	if topicCount > 0 {
+		r.blocks = make(map[string][]int32)
+		for i := 0; i < topicCount; i++ {
+			topic, err := pd.getCompactString()
+			if err != nil {
+				return err
+			}
+			partitionCount, err := pd.getCompactArrayLength()
+			if err != nil {
+				return err
+			}
+			r.blocks[topic] = make([]int32, partitionCount)
+			for j := 0; j < partitionCount; j++ {
+				partition, err := pd.getInt32()
+				if err != nil {
+					return err
+				}
+				r.blocks[topic][j] = partition
+			}
+			if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+
+	return
+}
+
+func (r *ListPartitionReassignmentsRequest) key() int16 {
+	return 46
+}
+
+func (r *ListPartitionReassignmentsRequest) version() int16 {
+	return r.Version
+}
+
+func (r *ListPartitionReassignmentsRequest) headerVersion() int16 {
+	return 2
+}
+
+func (r *ListPartitionReassignmentsRequest) requiredVersion() KafkaVersion {
+	return V2_4_0_0
+}
+
+func (r *ListPartitionReassignmentsRequest) AddBlock(topic string, partitionIDs []int32) {
+	if r.blocks == nil {
+		r.blocks = make(map[string][]int32)
+	}
+
+	if r.blocks[topic] == nil {
+		r.blocks[topic] = partitionIDs
+	}
+}
diff --git a/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go b/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go
new file mode 100644
index 0000000..4baa6a0
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go
@@ -0,0 +1,169 @@
+package sarama
+
+type PartitionReplicaReassignmentsStatus struct {
+	Replicas         []int32
+	AddingReplicas   []int32
+	RemovingReplicas []int32
+}
+
+func (b *PartitionReplicaReassignmentsStatus) encode(pe packetEncoder) error {
+	if err := pe.putCompactInt32Array(b.Replicas); err != nil {
+		return err
+	}
+	if err := pe.putCompactInt32Array(b.AddingReplicas); err != nil {
+		return err
+	}
+	if err := pe.putCompactInt32Array(b.RemovingReplicas); err != nil {
+		return err
+	}
+
+	pe.putEmptyTaggedFieldArray()
+
+	return nil
+}
+
+func (b *PartitionReplicaReassignmentsStatus) decode(pd packetDecoder) (err error) {
+	if b.Replicas, err = pd.getCompactInt32Array(); err != nil {
+		return err
+	}
+
+	if b.AddingReplicas, err = pd.getCompactInt32Array(); err != nil {
+		return err
+	}
+
+	if b.RemovingReplicas, err = pd.getCompactInt32Array(); err != nil {
+		return err
+	}
+
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+
+	return err
+}
+
+type ListPartitionReassignmentsResponse struct {
+	Version        int16
+	ThrottleTimeMs int32
+	ErrorCode      KError
+	ErrorMessage   *string
+	TopicStatus    map[string]map[int32]*PartitionReplicaReassignmentsStatus
+}
+
+func (r *ListPartitionReassignmentsResponse) AddBlock(topic string, partition int32, replicas, addingReplicas, removingReplicas []int32) {
+	if r.TopicStatus == nil {
+		r.TopicStatus = make(map[string]map[int32]*PartitionReplicaReassignmentsStatus)
+	}
+	partitions := r.TopicStatus[topic]
+	if partitions == nil {
+		partitions = make(map[int32]*PartitionReplicaReassignmentsStatus)
+		r.TopicStatus[topic] = partitions
+	}
+
+	partitions[partition] = &PartitionReplicaReassignmentsStatus{Replicas: replicas, AddingReplicas: addingReplicas, RemovingReplicas: removingReplicas}
+}
+
+func (r *ListPartitionReassignmentsResponse) encode(pe packetEncoder) error {
+	pe.putInt32(r.ThrottleTimeMs)
+	pe.putInt16(int16(r.ErrorCode))
+	if err := pe.putNullableCompactString(r.ErrorMessage); err != nil {
+		return err
+	}
+
+	pe.putCompactArrayLength(len(r.TopicStatus))
+	for topic, partitions := range r.TopicStatus {
+		if err := pe.putCompactString(topic); err != nil {
+			return err
+		}
+		pe.putCompactArrayLength(len(partitions))
+		for partition, block := range partitions {
+			pe.putInt32(partition)
+
+			if err := block.encode(pe); err != nil {
+				return err
+			}
+		}
+		pe.putEmptyTaggedFieldArray()
+	}
+
+	pe.putEmptyTaggedFieldArray()
+
+	return nil
+}
+
+func (r *ListPartitionReassignmentsResponse) decode(pd packetDecoder, version int16) (err error) {
+	r.Version = version
+
+	if r.ThrottleTimeMs, err = pd.getInt32(); err != nil {
+		return err
+	}
+
+	kerr, err := pd.getInt16()
+	if err != nil {
+		return err
+	}
+
+	r.ErrorCode = KError(kerr)
+
+	if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil {
+		return err
+	}
+
+	numTopics, err := pd.getCompactArrayLength()
+	if err != nil {
+		return err
+	}
+
+	r.TopicStatus = make(map[string]map[int32]*PartitionReplicaReassignmentsStatus, numTopics)
+	for i := 0; i < numTopics; i++ {
+		topic, err := pd.getCompactString()
+		if err != nil {
+			return err
+		}
+
+		ongoingPartitionReassignments, err := pd.getCompactArrayLength()
+		if err != nil {
+			return err
+		}
+
+		r.TopicStatus[topic] = make(map[int32]*PartitionReplicaReassignmentsStatus, ongoingPartitionReassignments)
+
+		for j := 0; j < ongoingPartitionReassignments; j++ {
+			partition, err := pd.getInt32()
+			if err != nil {
+				return err
+			}
+
+			block := &PartitionReplicaReassignmentsStatus{}
+			if err := block.decode(pd); err != nil {
+				return err
+			}
+			r.TopicStatus[topic][partition] = block
+		}
+
+		if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+	if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+func (r *ListPartitionReassignmentsResponse) key() int16 {
+	return 46
+}
+
+func (r *ListPartitionReassignmentsResponse) version() int16 {
+	return r.Version
+}
+
+func (r *ListPartitionReassignmentsResponse) headerVersion() int16 {
+	return 1
+}
+
+func (r *ListPartitionReassignmentsResponse) requiredVersion() KafkaVersion {
+	return V2_4_0_0
+}
diff --git a/vendor/github.com/Shopify/sarama/message.go b/vendor/github.com/Shopify/sarama/message.go
index 7c54748..fd0d1d9 100644
--- a/vendor/github.com/Shopify/sarama/message.go
+++ b/vendor/github.com/Shopify/sarama/message.go
@@ -6,15 +6,15 @@
 )
 
 const (
-	//CompressionNone no compression
+	// CompressionNone no compression
 	CompressionNone CompressionCodec = iota
-	//CompressionGZIP compression using GZIP
+	// CompressionGZIP compression using GZIP
 	CompressionGZIP
-	//CompressionSnappy compression using snappy
+	// CompressionSnappy compression using snappy
 	CompressionSnappy
-	//CompressionLZ4 compression using LZ4
+	// CompressionLZ4 compression using LZ4
 	CompressionLZ4
-	//CompressionZSTD compression using ZSTD
+	// CompressionZSTD compression using ZSTD
 	CompressionZSTD
 
 	// The lowest 3 bits contain the compression codec used for the message
@@ -42,7 +42,7 @@
 	}[int(cc)]
 }
 
-//Message is a kafka message type
+// Message is a kafka message type
 type Message struct {
 	Codec            CompressionCodec // codec used to compress the message contents
 	CompressionLevel int              // compression level
@@ -85,7 +85,6 @@
 		payload = m.compressedCache
 		m.compressedCache = nil
 	} else if m.Value != nil {
-
 		payload, err = compress(m.Codec, m.CompressionLevel, m.Value)
 		if err != nil {
 			return err
@@ -147,18 +146,12 @@
 	// for future metrics about the compression ratio in fetch requests
 	m.compressedSize = len(m.Value)
 
-	switch m.Codec {
-	case CompressionNone:
-		// nothing to do
-	default:
-		if m.Value == nil {
-			break
-		}
-
+	if m.Value != nil && m.Codec != CompressionNone {
 		m.Value, err = decompress(m.Codec, m.Value)
 		if err != nil {
 			return err
 		}
+
 		if err := m.decodeSet(); err != nil {
 			return err
 		}
diff --git a/vendor/github.com/Shopify/sarama/metadata_request.go b/vendor/github.com/Shopify/sarama/metadata_request.go
index 1b590d3..e835f5a 100644
--- a/vendor/github.com/Shopify/sarama/metadata_request.go
+++ b/vendor/github.com/Shopify/sarama/metadata_request.go
@@ -65,6 +65,10 @@
 	return r.Version
 }
 
+func (r *MetadataRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *MetadataRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/metadata_response.go b/vendor/github.com/Shopify/sarama/metadata_response.go
index b2d532e..0bb8702 100644
--- a/vendor/github.com/Shopify/sarama/metadata_response.go
+++ b/vendor/github.com/Shopify/sarama/metadata_response.go
@@ -255,6 +255,10 @@
 	return r.Version
 }
 
+func (r *MetadataResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *MetadataResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -318,5 +322,4 @@
 	pmatch.Isr = isr
 	pmatch.OfflineReplicas = offline
 	pmatch.Err = err
-
 }
diff --git a/vendor/github.com/Shopify/sarama/mockbroker.go b/vendor/github.com/Shopify/sarama/mockbroker.go
index 4ed46a6..c2654d1 100644
--- a/vendor/github.com/Shopify/sarama/mockbroker.go
+++ b/vendor/github.com/Shopify/sarama/mockbroker.go
@@ -20,7 +20,7 @@
 
 type GSSApiHandlerFunc func([]byte) []byte
 
-type requestHandlerFunc func(req *request) (res encoder)
+type requestHandlerFunc func(req *request) (res encoderWithHeader)
 
 // RequestNotifierFunc is invoked when a mock broker processes a request successfully
 // and will provides the number of bytes read and written.
@@ -30,7 +30,7 @@
 // to facilitate testing of higher level or specialized consumers and producers
 // built on top of Sarama. Note that it does not 'mimic' the Kafka API protocol,
 // but rather provides a facility to do that. It takes care of the TCP
-// transport, request unmarshaling, response marshaling, and makes it the test
+// transport, request unmarshalling, response marshalling, and makes it the test
 // writer responsibility to program correct according to the Kafka API protocol
 // MockBroker behaviour.
 //
@@ -55,7 +55,7 @@
 	port          int32
 	closing       chan none
 	stopper       chan none
-	expectations  chan encoder
+	expectations  chan encoderWithHeader
 	listener      net.Listener
 	t             TestReporter
 	latency       time.Duration
@@ -83,7 +83,7 @@
 // and uses the found MockResponse instance to generate an appropriate reply.
 // If the request type is not found in the map then nothing is sent.
 func (b *MockBroker) SetHandlerByMap(handlerMap map[string]MockResponse) {
-	b.setHandler(func(req *request) (res encoder) {
+	b.setHandler(func(req *request) (res encoderWithHeader) {
 		reqTypeName := reflect.TypeOf(req.body).Elem().Name()
 		mockResponse := handlerMap[reqTypeName]
 		if mockResponse == nil {
@@ -213,11 +213,13 @@
 	return buffer[4] == 0x60 || bytes.Equal(buffer[4:6], []byte{0x05, 0x04})
 }
 
-func (b *MockBroker) handleRequests(conn net.Conn, idx int, wg *sync.WaitGroup) {
+func (b *MockBroker) handleRequests(conn io.ReadWriteCloser, idx int, wg *sync.WaitGroup) {
 	defer wg.Done()
 	defer func() {
 		_ = conn.Close()
 	}()
+	s := spew.NewDefaultConfig()
+	s.MaxDepth = 1
 	Logger.Printf("*** mockbroker/%d/%d: connection opened", b.BrokerID(), idx)
 	var err error
 
@@ -231,11 +233,9 @@
 		}
 	}()
 
-	resHeader := make([]byte, 8)
 	var bytesWritten int
 	var bytesRead int
 	for {
-
 		buffer, err := b.readToBytes(conn)
 		if err != nil {
 			Logger.Printf("*** mockbroker/%d/%d: invalid request: err=%+v, %+v", b.brokerID, idx, err, spew.Sdump(buffer))
@@ -245,7 +245,6 @@
 
 		bytesWritten = 0
 		if !b.isGSSAPI(buffer) {
-
 			req, br, err := decodeRequest(bytes.NewReader(buffer))
 			bytesRead = br
 			if err != nil {
@@ -267,7 +266,12 @@
 				Logger.Printf("*** mockbroker/%d/%d: ignored %v", b.brokerID, idx, spew.Sdump(req))
 				continue
 			}
-			Logger.Printf("*** mockbroker/%d/%d: served %v -> %v", b.brokerID, idx, req, res)
+			Logger.Printf(
+				"*** mockbroker/%d/%d: replied to %T with %T\n-> %s\n-> %s",
+				b.brokerID, idx, req.body, res,
+				s.Sprintf("%#v", req.body),
+				s.Sprintf("%#v", res),
+			)
 
 			encodedRes, err := encode(res, nil)
 			if err != nil {
@@ -283,8 +287,7 @@
 				continue
 			}
 
-			binary.BigEndian.PutUint32(resHeader, uint32(len(encodedRes)+4))
-			binary.BigEndian.PutUint32(resHeader[4:], uint32(req.correlationID))
+			resHeader := b.encodeHeader(res.headerVersion(), req.correlationID, uint32(len(encodedRes)))
 			if _, err = conn.Write(resHeader); err != nil {
 				b.serverError(err)
 				break
@@ -294,7 +297,6 @@
 				break
 			}
 			bytesWritten = len(resHeader) + len(encodedRes)
-
 		} else {
 			// GSSAPI is not part of kafka protocol, but is supported for authentication proposes.
 			// Don't support history for this kind of request as is only used for test GSSAPI authentication mechanism
@@ -317,12 +319,29 @@
 			b.notifier(bytesRead, bytesWritten)
 		}
 		b.lock.Unlock()
-
 	}
 	Logger.Printf("*** mockbroker/%d/%d: connection closed, err=%v", b.BrokerID(), idx, err)
 }
 
-func (b *MockBroker) defaultRequestHandler(req *request) (res encoder) {
+func (b *MockBroker) encodeHeader(headerVersion int16, correlationId int32, payloadLength uint32) []byte {
+	headerLength := uint32(8)
+
+	if headerVersion >= 1 {
+		headerLength = 9
+	}
+
+	resHeader := make([]byte, headerLength)
+	binary.BigEndian.PutUint32(resHeader, payloadLength+headerLength-4)
+	binary.BigEndian.PutUint32(resHeader[4:], uint32(correlationId))
+
+	if headerVersion >= 1 {
+		binary.PutUvarint(resHeader[8:], 0)
+	}
+
+	return resHeader
+}
+
+func (b *MockBroker) defaultRequestHandler(req *request) (res encoderWithHeader) {
 	select {
 	case res, ok := <-b.expectations:
 		if !ok {
@@ -377,7 +396,7 @@
 		stopper:      make(chan none),
 		t:            t,
 		brokerID:     brokerID,
-		expectations: make(chan encoder, 512),
+		expectations: make(chan encoderWithHeader, 512),
 		listener:     listener,
 	}
 	broker.handler = broker.defaultRequestHandler
@@ -398,6 +417,6 @@
 	return broker
 }
 
-func (b *MockBroker) Returns(e encoder) {
+func (b *MockBroker) Returns(e encoderWithHeader) {
 	b.expectations <- e
 }
diff --git a/vendor/github.com/Shopify/sarama/mockkerberos.go b/vendor/github.com/Shopify/sarama/mockkerberos.go
index affeb2d..a43607e 100644
--- a/vendor/github.com/Shopify/sarama/mockkerberos.go
+++ b/vendor/github.com/Shopify/sarama/mockkerberos.go
@@ -3,11 +3,12 @@
 import (
 	"encoding/binary"
 	"encoding/hex"
-	"gopkg.in/jcmturner/gokrb5.v7/credentials"
-	"gopkg.in/jcmturner/gokrb5.v7/gssapi"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
+
+	"github.com/jcmturner/gokrb5/v8/credentials"
+	"github.com/jcmturner/gokrb5/v8/gssapi"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
 )
 
 type KafkaGSSAPIHandler struct {
@@ -26,7 +27,7 @@
 		return []byte{0x00, 0x00, 0x00, 0x01, 0xAD}
 	}
 
-	var pack = gssapi.WrapToken{
+	pack := gssapi.WrapToken{
 		Flags:     KRB5_USER_AUTH,
 		EC:        12,
 		RRC:       0,
@@ -55,7 +56,6 @@
 }
 
 type MockKerberosClient struct {
-	asReqBytes  string
 	asRepBytes  string
 	ASRep       messages.ASRep
 	credentials *credentials.Credentials
@@ -108,8 +108,9 @@
 func (c *MockKerberosClient) Domain() string {
 	return "EXAMPLE.COM"
 }
+
 func (c *MockKerberosClient) CName() types.PrincipalName {
-	var p = types.PrincipalName{
+	p := types.PrincipalName{
 		NameType: KRB5_USER_AUTH,
 		NameString: []string{
 			"kafka",
@@ -118,6 +119,7 @@
 	}
 	return p
 }
+
 func (c *MockKerberosClient) Destroy() {
 	// Do nothing.
 }
diff --git a/vendor/github.com/Shopify/sarama/mockresponses.go b/vendor/github.com/Shopify/sarama/mockresponses.go
index c78f0ac..6654ed0 100644
--- a/vendor/github.com/Shopify/sarama/mockresponses.go
+++ b/vendor/github.com/Shopify/sarama/mockresponses.go
@@ -18,20 +18,20 @@
 // allows generating a response based on a request body. MockResponses are used
 // to program behavior of MockBroker in tests.
 type MockResponse interface {
-	For(reqBody versionedDecoder) (res encoder)
+	For(reqBody versionedDecoder) (res encoderWithHeader)
 }
 
 // MockWrapper is a mock response builder that returns a particular concrete
 // response regardless of the actual request passed to the `For` method.
 type MockWrapper struct {
-	res encoder
+	res encoderWithHeader
 }
 
-func (mw *MockWrapper) For(reqBody versionedDecoder) (res encoder) {
+func (mw *MockWrapper) For(reqBody versionedDecoder) (res encoderWithHeader) {
 	return mw.res
 }
 
-func NewMockWrapper(res encoder) *MockWrapper {
+func NewMockWrapper(res encoderWithHeader) *MockWrapper {
 	return &MockWrapper{res: res}
 }
 
@@ -50,7 +50,7 @@
 		switch res := res.(type) {
 		case MockResponse:
 			ms.responses[i] = res
-		case encoder:
+		case encoderWithHeader:
 			ms.responses[i] = NewMockWrapper(res)
 		default:
 			panic(fmt.Sprintf("Unexpected response type: %T", res))
@@ -59,7 +59,7 @@
 	return ms
 }
 
-func (mc *MockSequence) For(reqBody versionedDecoder) (res encoder) {
+func (mc *MockSequence) For(reqBody versionedDecoder) (res encoderWithHeader) {
 	res = mc.responses[0].For(reqBody)
 	if len(mc.responses) > 1 {
 		mc.responses = mc.responses[1:]
@@ -79,7 +79,7 @@
 	}
 }
 
-func (m *MockListGroupsResponse) For(reqBody versionedDecoder) encoder {
+func (m *MockListGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	request := reqBody.(*ListGroupsRequest)
 	_ = request
 	response := &ListGroupsResponse{
@@ -110,7 +110,7 @@
 	return m
 }
 
-func (m *MockDescribeGroupsResponse) For(reqBody versionedDecoder) encoder {
+func (m *MockDescribeGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	request := reqBody.(*DescribeGroupsRequest)
 
 	response := &DescribeGroupsResponse{}
@@ -166,7 +166,7 @@
 	return mmr
 }
 
-func (mmr *MockMetadataResponse) For(reqBody versionedDecoder) encoder {
+func (mmr *MockMetadataResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	metadataRequest := reqBody.(*MetadataRequest)
 	metadataResponse := &MetadataResponse{
 		Version:      metadataRequest.version(),
@@ -177,8 +177,8 @@
 	}
 
 	// Generate set of replicas
-	replicas := []int32{}
-	offlineReplicas := []int32{}
+	var replicas []int32
+	var offlineReplicas []int32
 	for _, brokerID := range mmr.brokers {
 		replicas = append(replicas, brokerID)
 	}
@@ -233,7 +233,7 @@
 	return mor
 }
 
-func (mor *MockOffsetResponse) For(reqBody versionedDecoder) encoder {
+func (mor *MockOffsetResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	offsetRequest := reqBody.(*OffsetRequest)
 	offsetResponse := &OffsetResponse{Version: mor.version}
 	for topic, partitions := range offsetRequest.blocks {
@@ -309,7 +309,7 @@
 	return mfr
 }
 
-func (mfr *MockFetchResponse) For(reqBody versionedDecoder) encoder {
+func (mfr *MockFetchResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	fetchRequest := reqBody.(*FetchRequest)
 	res := &FetchResponse{
 		Version: mfr.version,
@@ -393,7 +393,7 @@
 	return mr
 }
 
-func (mr *MockConsumerMetadataResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockConsumerMetadataResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*ConsumerMetadataRequest)
 	group := req.ConsumerGroup
 	res := &ConsumerMetadataResponse{}
@@ -442,7 +442,7 @@
 	return mr
 }
 
-func (mr *MockFindCoordinatorResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockFindCoordinatorResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*FindCoordinatorRequest)
 	res := &FindCoordinatorResponse{}
 	var v interface{}
@@ -489,7 +489,7 @@
 	return mr
 }
 
-func (mr *MockOffsetCommitResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockOffsetCommitResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*OffsetCommitRequest)
 	group := req.ConsumerGroup
 	res := &OffsetCommitResponse{}
@@ -546,7 +546,7 @@
 	return mr
 }
 
-func (mr *MockProduceResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockProduceResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*ProduceRequest)
 	res := &ProduceResponse{
 		Version: mr.version,
@@ -605,7 +605,7 @@
 	return mr
 }
 
-func (mr *MockOffsetFetchResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockOffsetFetchResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*OffsetFetchRequest)
 	group := req.ConsumerGroup
 	res := &OffsetFetchResponse{Version: req.Version}
@@ -630,7 +630,7 @@
 	return &MockCreateTopicsResponse{t: t}
 }
 
-func (mr *MockCreateTopicsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockCreateTopicsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*CreateTopicsRequest)
 	res := &CreateTopicsResponse{
 		Version: req.Version,
@@ -659,7 +659,7 @@
 	return &MockDeleteTopicsResponse{t: t}
 }
 
-func (mr *MockDeleteTopicsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockDeleteTopicsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*DeleteTopicsRequest)
 	res := &DeleteTopicsResponse{}
 	res.TopicErrorCodes = make(map[string]KError)
@@ -667,6 +667,7 @@
 	for _, topic := range req.Topics {
 		res.TopicErrorCodes[topic] = ErrNoError
 	}
+	res.Version = req.Version
 	return res
 }
 
@@ -678,7 +679,7 @@
 	return &MockCreatePartitionsResponse{t: t}
 }
 
-func (mr *MockCreatePartitionsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockCreatePartitionsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*CreatePartitionsRequest)
 	res := &CreatePartitionsResponse{}
 	res.TopicPartitionErrors = make(map[string]*TopicPartitionError)
@@ -697,6 +698,43 @@
 	return res
 }
 
+type MockAlterPartitionReassignmentsResponse struct {
+	t TestReporter
+}
+
+func NewMockAlterPartitionReassignmentsResponse(t TestReporter) *MockAlterPartitionReassignmentsResponse {
+	return &MockAlterPartitionReassignmentsResponse{t: t}
+}
+
+func (mr *MockAlterPartitionReassignmentsResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	req := reqBody.(*AlterPartitionReassignmentsRequest)
+	_ = req
+	res := &AlterPartitionReassignmentsResponse{}
+	return res
+}
+
+type MockListPartitionReassignmentsResponse struct {
+	t TestReporter
+}
+
+func NewMockListPartitionReassignmentsResponse(t TestReporter) *MockListPartitionReassignmentsResponse {
+	return &MockListPartitionReassignmentsResponse{t: t}
+}
+
+func (mr *MockListPartitionReassignmentsResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	req := reqBody.(*ListPartitionReassignmentsRequest)
+	_ = req
+	res := &ListPartitionReassignmentsResponse{}
+
+	for topic, partitions := range req.blocks {
+		for _, partition := range partitions {
+			res.AddBlock(topic, partition, []int32{0}, []int32{1}, []int32{2})
+		}
+	}
+
+	return res
+}
+
 type MockDeleteRecordsResponse struct {
 	t TestReporter
 }
@@ -705,7 +743,7 @@
 	return &MockDeleteRecordsResponse{t: t}
 }
 
-func (mr *MockDeleteRecordsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockDeleteRecordsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*DeleteRecordsRequest)
 	res := &DeleteRecordsResponse{}
 	res.Topics = make(map[string]*DeleteRecordsResponseTopic)
@@ -728,31 +766,87 @@
 	return &MockDescribeConfigsResponse{t: t}
 }
 
-func (mr *MockDescribeConfigsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockDescribeConfigsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*DescribeConfigsRequest)
-	res := &DescribeConfigsResponse{}
+	res := &DescribeConfigsResponse{
+		Version: req.Version,
+	}
+
+	includeSynonyms := req.Version > 0
+	includeSource := req.Version > 0
 
 	for _, r := range req.Resources {
 		var configEntries []*ConfigEntry
 		switch r.Type {
-		case TopicResource:
+		case BrokerResource:
 			configEntries = append(configEntries,
-				&ConfigEntry{Name: "max.message.bytes",
-					Value:     "1000000",
-					ReadOnly:  false,
-					Default:   true,
-					Sensitive: false,
-				}, &ConfigEntry{Name: "retention.ms",
-					Value:     "5000",
-					ReadOnly:  false,
-					Default:   false,
-					Sensitive: false,
-				}, &ConfigEntry{Name: "password",
-					Value:     "12345",
-					ReadOnly:  false,
-					Default:   false,
-					Sensitive: true,
-				})
+				&ConfigEntry{
+					Name:     "min.insync.replicas",
+					Value:    "2",
+					ReadOnly: false,
+					Default:  false,
+				},
+			)
+			res.Resources = append(res.Resources, &ResourceResponse{
+				Name:    r.Name,
+				Configs: configEntries,
+			})
+		case BrokerLoggerResource:
+			configEntries = append(configEntries,
+				&ConfigEntry{
+					Name:     "kafka.controller.KafkaController",
+					Value:    "DEBUG",
+					ReadOnly: false,
+					Default:  false,
+				},
+			)
+			res.Resources = append(res.Resources, &ResourceResponse{
+				Name:    r.Name,
+				Configs: configEntries,
+			})
+		case TopicResource:
+			maxMessageBytes := &ConfigEntry{
+				Name:      "max.message.bytes",
+				Value:     "1000000",
+				ReadOnly:  false,
+				Default:   !includeSource,
+				Sensitive: false,
+			}
+			if includeSource {
+				maxMessageBytes.Source = SourceDefault
+			}
+			if includeSynonyms {
+				maxMessageBytes.Synonyms = []*ConfigSynonym{
+					{
+						ConfigName:  "max.message.bytes",
+						ConfigValue: "500000",
+					},
+				}
+			}
+			retentionMs := &ConfigEntry{
+				Name:      "retention.ms",
+				Value:     "5000",
+				ReadOnly:  false,
+				Default:   false,
+				Sensitive: false,
+			}
+			if includeSynonyms {
+				retentionMs.Synonyms = []*ConfigSynonym{
+					{
+						ConfigName:  "log.retention.ms",
+						ConfigValue: "2500",
+					},
+				}
+			}
+			password := &ConfigEntry{
+				Name:      "password",
+				Value:     "12345",
+				ReadOnly:  false,
+				Default:   false,
+				Sensitive: true,
+			}
+			configEntries = append(
+				configEntries, maxMessageBytes, retentionMs, password)
 			res.Resources = append(res.Resources, &ResourceResponse{
 				Name:    r.Name,
 				Configs: configEntries,
@@ -762,6 +856,31 @@
 	return res
 }
 
+type MockDescribeConfigsResponseWithErrorCode struct {
+	t TestReporter
+}
+
+func NewMockDescribeConfigsResponseWithErrorCode(t TestReporter) *MockDescribeConfigsResponseWithErrorCode {
+	return &MockDescribeConfigsResponseWithErrorCode{t: t}
+}
+
+func (mr *MockDescribeConfigsResponseWithErrorCode) For(reqBody versionedDecoder) encoderWithHeader {
+	req := reqBody.(*DescribeConfigsRequest)
+	res := &DescribeConfigsResponse{
+		Version: req.Version,
+	}
+
+	for _, r := range req.Resources {
+		res.Resources = append(res.Resources, &ResourceResponse{
+			Name:      r.Name,
+			Type:      r.Type,
+			ErrorCode: 83,
+			ErrorMsg:  "",
+		})
+	}
+	return res
+}
+
 type MockAlterConfigsResponse struct {
 	t TestReporter
 }
@@ -770,19 +889,43 @@
 	return &MockAlterConfigsResponse{t: t}
 }
 
-func (mr *MockAlterConfigsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockAlterConfigsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*AlterConfigsRequest)
 	res := &AlterConfigsResponse{}
 
 	for _, r := range req.Resources {
-		res.Resources = append(res.Resources, &AlterConfigsResourceResponse{Name: r.Name,
-			Type:     TopicResource,
+		res.Resources = append(res.Resources, &AlterConfigsResourceResponse{
+			Name:     r.Name,
+			Type:     r.Type,
 			ErrorMsg: "",
 		})
 	}
 	return res
 }
 
+type MockAlterConfigsResponseWithErrorCode struct {
+	t TestReporter
+}
+
+func NewMockAlterConfigsResponseWithErrorCode(t TestReporter) *MockAlterConfigsResponseWithErrorCode {
+	return &MockAlterConfigsResponseWithErrorCode{t: t}
+}
+
+func (mr *MockAlterConfigsResponseWithErrorCode) For(reqBody versionedDecoder) encoderWithHeader {
+	req := reqBody.(*AlterConfigsRequest)
+	res := &AlterConfigsResponse{}
+
+	for _, r := range req.Resources {
+		res.Resources = append(res.Resources, &AlterConfigsResourceResponse{
+			Name:      r.Name,
+			Type:      r.Type,
+			ErrorCode: 83,
+			ErrorMsg:  "",
+		})
+	}
+	return res
+}
+
 type MockCreateAclsResponse struct {
 	t TestReporter
 }
@@ -791,7 +934,7 @@
 	return &MockCreateAclsResponse{t: t}
 }
 
-func (mr *MockCreateAclsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockCreateAclsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*CreateAclsRequest)
 	res := &CreateAclsResponse{}
 
@@ -809,17 +952,35 @@
 	return &MockListAclsResponse{t: t}
 }
 
-func (mr *MockListAclsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockListAclsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*DescribeAclsRequest)
 	res := &DescribeAclsResponse{}
-
 	res.Err = ErrNoError
 	acl := &ResourceAcls{}
-	acl.Resource.ResourceName = *req.ResourceName
+	if req.ResourceName != nil {
+		acl.Resource.ResourceName = *req.ResourceName
+	}
+	acl.Resource.ResourcePatternType = req.ResourcePatternTypeFilter
 	acl.Resource.ResourceType = req.ResourceType
-	acl.Acls = append(acl.Acls, &Acl{})
-	res.ResourceAcls = append(res.ResourceAcls, acl)
 
+	host := "*"
+	if req.Host != nil {
+		host = *req.Host
+	}
+
+	principal := "User:test"
+	if req.Principal != nil {
+		principal = *req.Principal
+	}
+
+	permissionType := req.PermissionType
+	if permissionType == AclPermissionAny {
+		permissionType = AclPermissionAllow
+	}
+
+	acl.Acls = append(acl.Acls, &Acl{Operation: req.Operation, PermissionType: permissionType, Host: host, Principal: principal})
+	res.ResourceAcls = append(res.ResourceAcls, acl)
+	res.Version = int16(req.Version)
 	return res
 }
 
@@ -833,7 +994,7 @@
 	return &MockSaslAuthenticateResponse{t: t}
 }
 
-func (msar *MockSaslAuthenticateResponse) For(reqBody versionedDecoder) encoder {
+func (msar *MockSaslAuthenticateResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	res := &SaslAuthenticateResponse{}
 	res.Err = msar.kerror
 	res.SaslAuthBytes = msar.saslAuthBytes
@@ -864,7 +1025,7 @@
 	return &MockSaslHandshakeResponse{t: t}
 }
 
-func (mshr *MockSaslHandshakeResponse) For(reqBody versionedDecoder) encoder {
+func (mshr *MockSaslHandshakeResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	res := &SaslHandshakeResponse{}
 	res.Err = mshr.kerror
 	res.EnabledMechanisms = mshr.enabledMechanisms
@@ -885,7 +1046,7 @@
 	return &MockDeleteAclsResponse{t: t}
 }
 
-func (mr *MockDeleteAclsResponse) For(reqBody versionedDecoder) encoder {
+func (mr *MockDeleteAclsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	req := reqBody.(*DeleteAclsRequest)
 	res := &DeleteAclsResponse{}
 
@@ -894,6 +1055,7 @@
 		response.MatchingAcls = append(response.MatchingAcls, &MatchingAcl{Err: ErrNoError})
 		res.FilterResponses = append(res.FilterResponses, response)
 	}
+	res.Version = int16(req.Version)
 	return res
 }
 
@@ -910,7 +1072,7 @@
 	return m
 }
 
-func (m *MockDeleteGroupsResponse) For(reqBody versionedDecoder) encoder {
+func (m *MockDeleteGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader {
 	resp := &DeleteGroupsResponse{
 		GroupErrorCodes: map[string]KError{},
 	}
@@ -919,3 +1081,193 @@
 	}
 	return resp
 }
+
+type MockJoinGroupResponse struct {
+	t TestReporter
+
+	ThrottleTime  int32
+	Err           KError
+	GenerationId  int32
+	GroupProtocol string
+	LeaderId      string
+	MemberId      string
+	Members       map[string][]byte
+}
+
+func NewMockJoinGroupResponse(t TestReporter) *MockJoinGroupResponse {
+	return &MockJoinGroupResponse{
+		t:       t,
+		Members: make(map[string][]byte),
+	}
+}
+
+func (m *MockJoinGroupResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	req := reqBody.(*JoinGroupRequest)
+	resp := &JoinGroupResponse{
+		Version:       req.Version,
+		ThrottleTime:  m.ThrottleTime,
+		Err:           m.Err,
+		GenerationId:  m.GenerationId,
+		GroupProtocol: m.GroupProtocol,
+		LeaderId:      m.LeaderId,
+		MemberId:      m.MemberId,
+		Members:       m.Members,
+	}
+	return resp
+}
+
+func (m *MockJoinGroupResponse) SetThrottleTime(t int32) *MockJoinGroupResponse {
+	m.ThrottleTime = t
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetError(kerr KError) *MockJoinGroupResponse {
+	m.Err = kerr
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetGenerationId(id int32) *MockJoinGroupResponse {
+	m.GenerationId = id
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetGroupProtocol(proto string) *MockJoinGroupResponse {
+	m.GroupProtocol = proto
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetLeaderId(id string) *MockJoinGroupResponse {
+	m.LeaderId = id
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetMemberId(id string) *MockJoinGroupResponse {
+	m.MemberId = id
+	return m
+}
+
+func (m *MockJoinGroupResponse) SetMember(id string, meta *ConsumerGroupMemberMetadata) *MockJoinGroupResponse {
+	bin, err := encode(meta, nil)
+	if err != nil {
+		panic(fmt.Sprintf("error encoding member metadata: %v", err))
+	}
+	m.Members[id] = bin
+	return m
+}
+
+type MockLeaveGroupResponse struct {
+	t TestReporter
+
+	Err KError
+}
+
+func NewMockLeaveGroupResponse(t TestReporter) *MockLeaveGroupResponse {
+	return &MockLeaveGroupResponse{t: t}
+}
+
+func (m *MockLeaveGroupResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	resp := &LeaveGroupResponse{
+		Err: m.Err,
+	}
+	return resp
+}
+
+func (m *MockLeaveGroupResponse) SetError(kerr KError) *MockLeaveGroupResponse {
+	m.Err = kerr
+	return m
+}
+
+type MockSyncGroupResponse struct {
+	t TestReporter
+
+	Err              KError
+	MemberAssignment []byte
+}
+
+func NewMockSyncGroupResponse(t TestReporter) *MockSyncGroupResponse {
+	return &MockSyncGroupResponse{t: t}
+}
+
+func (m *MockSyncGroupResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	resp := &SyncGroupResponse{
+		Err:              m.Err,
+		MemberAssignment: m.MemberAssignment,
+	}
+	return resp
+}
+
+func (m *MockSyncGroupResponse) SetError(kerr KError) *MockSyncGroupResponse {
+	m.Err = kerr
+	return m
+}
+
+func (m *MockSyncGroupResponse) SetMemberAssignment(assignment *ConsumerGroupMemberAssignment) *MockSyncGroupResponse {
+	bin, err := encode(assignment, nil)
+	if err != nil {
+		panic(fmt.Sprintf("error encoding member assignment: %v", err))
+	}
+	m.MemberAssignment = bin
+	return m
+}
+
+type MockHeartbeatResponse struct {
+	t TestReporter
+
+	Err KError
+}
+
+func NewMockHeartbeatResponse(t TestReporter) *MockHeartbeatResponse {
+	return &MockHeartbeatResponse{t: t}
+}
+
+func (m *MockHeartbeatResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	resp := &HeartbeatResponse{}
+	return resp
+}
+
+func (m *MockHeartbeatResponse) SetError(kerr KError) *MockHeartbeatResponse {
+	m.Err = kerr
+	return m
+}
+
+type MockDescribeLogDirsResponse struct {
+	t       TestReporter
+	logDirs []DescribeLogDirsResponseDirMetadata
+}
+
+func NewMockDescribeLogDirsResponse(t TestReporter) *MockDescribeLogDirsResponse {
+	return &MockDescribeLogDirsResponse{t: t}
+}
+
+func (m *MockDescribeLogDirsResponse) SetLogDirs(logDirPath string, topicPartitions map[string]int) *MockDescribeLogDirsResponse {
+	var topics []DescribeLogDirsResponseTopic
+	for topic := range topicPartitions {
+		var partitions []DescribeLogDirsResponsePartition
+		for i := 0; i < topicPartitions[topic]; i++ {
+			partitions = append(partitions, DescribeLogDirsResponsePartition{
+				PartitionID: int32(i),
+				IsTemporary: false,
+				OffsetLag:   int64(0),
+				Size:        int64(1234),
+			})
+		}
+		topics = append(topics, DescribeLogDirsResponseTopic{
+			Topic:      topic,
+			Partitions: partitions,
+		})
+	}
+	logDir := DescribeLogDirsResponseDirMetadata{
+		ErrorCode: ErrNoError,
+		Path:      logDirPath,
+		Topics:    topics,
+	}
+	m.logDirs = []DescribeLogDirsResponseDirMetadata{logDir}
+	return m
+}
+
+func (m *MockDescribeLogDirsResponse) For(reqBody versionedDecoder) encoderWithHeader {
+	resp := &DescribeLogDirsResponse{
+		LogDirs: m.logDirs,
+	}
+	return resp
+}
diff --git a/vendor/github.com/Shopify/sarama/offset_commit_request.go b/vendor/github.com/Shopify/sarama/offset_commit_request.go
index 5732ed9..9931cad 100644
--- a/vendor/github.com/Shopify/sarama/offset_commit_request.go
+++ b/vendor/github.com/Shopify/sarama/offset_commit_request.go
@@ -170,6 +170,10 @@
 	return r.Version
 }
 
+func (r *OffsetCommitRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *OffsetCommitRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/offset_commit_response.go b/vendor/github.com/Shopify/sarama/offset_commit_response.go
index e842298..342260e 100644
--- a/vendor/github.com/Shopify/sarama/offset_commit_response.go
+++ b/vendor/github.com/Shopify/sarama/offset_commit_response.go
@@ -94,6 +94,10 @@
 	return r.Version
 }
 
+func (r *OffsetCommitResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *OffsetCommitResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_request.go b/vendor/github.com/Shopify/sarama/offset_fetch_request.go
index 6860824..7e147eb 100644
--- a/vendor/github.com/Shopify/sarama/offset_fetch_request.go
+++ b/vendor/github.com/Shopify/sarama/offset_fetch_request.go
@@ -3,60 +3,155 @@
 type OffsetFetchRequest struct {
 	Version       int16
 	ConsumerGroup string
+	RequireStable bool // requires v7+
 	partitions    map[string][]int32
 }
 
 func (r *OffsetFetchRequest) encode(pe packetEncoder) (err error) {
-	if r.Version < 0 || r.Version > 5 {
+	if r.Version < 0 || r.Version > 7 {
 		return PacketEncodingError{"invalid or unsupported OffsetFetchRequest version field"}
 	}
 
-	if err = pe.putString(r.ConsumerGroup); err != nil {
+	isFlexible := r.Version >= 6
+
+	if isFlexible {
+		err = pe.putCompactString(r.ConsumerGroup)
+	} else {
+		err = pe.putString(r.ConsumerGroup)
+	}
+	if err != nil {
 		return err
 	}
 
-	if r.Version >= 2 && r.partitions == nil {
-		pe.putInt32(-1)
-	} else {
-		if err = pe.putArrayLength(len(r.partitions)); err != nil {
-			return err
+	if isFlexible {
+		if r.partitions == nil {
+			pe.putUVarint(0)
+		} else {
+			pe.putCompactArrayLength(len(r.partitions))
 		}
-		for topic, partitions := range r.partitions {
-			if err = pe.putString(topic); err != nil {
-				return err
-			}
-			if err = pe.putInt32Array(partitions); err != nil {
+	} else {
+		if r.partitions == nil && r.Version >= 2 {
+			pe.putInt32(-1)
+		} else {
+			if err = pe.putArrayLength(len(r.partitions)); err != nil {
 				return err
 			}
 		}
 	}
+
+	for topic, partitions := range r.partitions {
+		if isFlexible {
+			err = pe.putCompactString(topic)
+		} else {
+			err = pe.putString(topic)
+		}
+		if err != nil {
+			return err
+		}
+
+		//
+
+		if isFlexible {
+			err = pe.putCompactInt32Array(partitions)
+		} else {
+			err = pe.putInt32Array(partitions)
+		}
+		if err != nil {
+			return err
+		}
+
+		if isFlexible {
+			pe.putEmptyTaggedFieldArray()
+		}
+	}
+
+	if r.RequireStable && r.Version < 7 {
+		return PacketEncodingError{"requireStable is not supported. use version 7 or later"}
+	}
+
+	if r.Version >= 7 {
+		pe.putBool(r.RequireStable)
+	}
+
+	if isFlexible {
+		pe.putEmptyTaggedFieldArray()
+	}
+
 	return nil
 }
 
 func (r *OffsetFetchRequest) decode(pd packetDecoder, version int16) (err error) {
 	r.Version = version
-	if r.ConsumerGroup, err = pd.getString(); err != nil {
-		return err
+	isFlexible := r.Version >= 6
+	if isFlexible {
+		r.ConsumerGroup, err = pd.getCompactString()
+	} else {
+		r.ConsumerGroup, err = pd.getString()
 	}
-	partitionCount, err := pd.getArrayLength()
 	if err != nil {
 		return err
 	}
+
+	var partitionCount int
+
+	if isFlexible {
+		partitionCount, err = pd.getCompactArrayLength()
+	} else {
+		partitionCount, err = pd.getArrayLength()
+	}
+	if err != nil {
+		return err
+	}
+
 	if (partitionCount == 0 && version < 2) || partitionCount < 0 {
 		return nil
 	}
-	r.partitions = make(map[string][]int32)
+
+	r.partitions = make(map[string][]int32, partitionCount)
 	for i := 0; i < partitionCount; i++ {
-		topic, err := pd.getString()
+		var topic string
+		if isFlexible {
+			topic, err = pd.getCompactString()
+		} else {
+			topic, err = pd.getString()
+		}
 		if err != nil {
 			return err
 		}
-		partitions, err := pd.getInt32Array()
+
+		var partitions []int32
+		if isFlexible {
+			partitions, err = pd.getCompactInt32Array()
+		} else {
+			partitions, err = pd.getInt32Array()
+		}
 		if err != nil {
 			return err
 		}
+		if isFlexible {
+			_, err = pd.getEmptyTaggedFieldArray()
+			if err != nil {
+				return err
+			}
+		}
+
 		r.partitions[topic] = partitions
 	}
+
+	if r.Version >= 7 {
+		r.RequireStable, err = pd.getBool()
+		if err != nil {
+			return err
+		}
+	}
+
+	if isFlexible {
+		_, err = pd.getEmptyTaggedFieldArray()
+		if err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
@@ -68,6 +163,14 @@
 	return r.Version
 }
 
+func (r *OffsetFetchRequest) headerVersion() int16 {
+	if r.Version >= 6 {
+		return 2
+	}
+
+	return 1
+}
+
 func (r *OffsetFetchRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -80,6 +183,10 @@
 		return V2_0_0_0
 	case 5:
 		return V2_1_0_0
+	case 6:
+		return V2_4_0_0
+	case 7:
+		return V2_5_0_0
 	default:
 		return MinVersion
 	}
diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_response.go b/vendor/github.com/Shopify/sarama/offset_fetch_response.go
index 9e25702..1944922 100644
--- a/vendor/github.com/Shopify/sarama/offset_fetch_response.go
+++ b/vendor/github.com/Shopify/sarama/offset_fetch_response.go
@@ -8,6 +8,8 @@
 }
 
 func (b *OffsetFetchResponseBlock) decode(pd packetDecoder, version int16) (err error) {
+	isFlexible := version >= 6
+
 	b.Offset, err = pd.getInt64()
 	if err != nil {
 		return err
@@ -20,7 +22,11 @@
 		}
 	}
 
-	b.Metadata, err = pd.getString()
+	if isFlexible {
+		b.Metadata, err = pd.getCompactString()
+	} else {
+		b.Metadata, err = pd.getString()
+	}
 	if err != nil {
 		return err
 	}
@@ -31,23 +37,37 @@
 	}
 	b.Err = KError(tmp)
 
+	if isFlexible {
+		if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
 func (b *OffsetFetchResponseBlock) encode(pe packetEncoder, version int16) (err error) {
+	isFlexible := version >= 6
 	pe.putInt64(b.Offset)
 
 	if version >= 5 {
 		pe.putInt32(b.LeaderEpoch)
 	}
-
-	err = pe.putString(b.Metadata)
+	if isFlexible {
+		err = pe.putCompactString(b.Metadata)
+	} else {
+		err = pe.putString(b.Metadata)
+	}
 	if err != nil {
 		return err
 	}
 
 	pe.putInt16(int16(b.Err))
 
+	if isFlexible {
+		pe.putEmptyTaggedFieldArray()
+	}
+
 	return nil
 }
 
@@ -58,19 +78,37 @@
 	Err            KError
 }
 
-func (r *OffsetFetchResponse) encode(pe packetEncoder) error {
+func (r *OffsetFetchResponse) encode(pe packetEncoder) (err error) {
+	isFlexible := r.Version >= 6
+
 	if r.Version >= 3 {
 		pe.putInt32(r.ThrottleTimeMs)
 	}
-
-	if err := pe.putArrayLength(len(r.Blocks)); err != nil {
+	if isFlexible {
+		pe.putCompactArrayLength(len(r.Blocks))
+	} else {
+		err = pe.putArrayLength(len(r.Blocks))
+	}
+	if err != nil {
 		return err
 	}
+
 	for topic, partitions := range r.Blocks {
-		if err := pe.putString(topic); err != nil {
+		if isFlexible {
+			err = pe.putCompactString(topic)
+		} else {
+			err = pe.putString(topic)
+		}
+		if err != nil {
 			return err
 		}
-		if err := pe.putArrayLength(len(partitions)); err != nil {
+
+		if isFlexible {
+			pe.putCompactArrayLength(len(partitions))
+		} else {
+			err = pe.putArrayLength(len(partitions))
+		}
+		if err != nil {
 			return err
 		}
 		for partition, block := range partitions {
@@ -79,15 +117,22 @@
 				return err
 			}
 		}
+		if isFlexible {
+			pe.putEmptyTaggedFieldArray()
+		}
 	}
 	if r.Version >= 2 {
 		pe.putInt16(int16(r.Err))
 	}
+	if isFlexible {
+		pe.putEmptyTaggedFieldArray()
+	}
 	return nil
 }
 
 func (r *OffsetFetchResponse) decode(pd packetDecoder, version int16) (err error) {
 	r.Version = version
+	isFlexible := version >= 6
 
 	if version >= 3 {
 		r.ThrottleTimeMs, err = pd.getInt32()
@@ -96,7 +141,12 @@
 		}
 	}
 
-	numTopics, err := pd.getArrayLength()
+	var numTopics int
+	if isFlexible {
+		numTopics, err = pd.getCompactArrayLength()
+	} else {
+		numTopics, err = pd.getArrayLength()
+	}
 	if err != nil {
 		return err
 	}
@@ -104,22 +154,30 @@
 	if numTopics > 0 {
 		r.Blocks = make(map[string]map[int32]*OffsetFetchResponseBlock, numTopics)
 		for i := 0; i < numTopics; i++ {
-			name, err := pd.getString()
+			var name string
+			if isFlexible {
+				name, err = pd.getCompactString()
+			} else {
+				name, err = pd.getString()
+			}
 			if err != nil {
 				return err
 			}
 
-			numBlocks, err := pd.getArrayLength()
+			var numBlocks int
+			if isFlexible {
+				numBlocks, err = pd.getCompactArrayLength()
+			} else {
+				numBlocks, err = pd.getArrayLength()
+			}
 			if err != nil {
 				return err
 			}
 
-			if numBlocks == 0 {
-				r.Blocks[name] = nil
-				continue
+			r.Blocks[name] = nil
+			if numBlocks > 0 {
+				r.Blocks[name] = make(map[int32]*OffsetFetchResponseBlock, numBlocks)
 			}
-			r.Blocks[name] = make(map[int32]*OffsetFetchResponseBlock, numBlocks)
-
 			for j := 0; j < numBlocks; j++ {
 				id, err := pd.getInt32()
 				if err != nil {
@@ -131,8 +189,15 @@
 				if err != nil {
 					return err
 				}
+
 				r.Blocks[name][id] = block
 			}
+
+			if isFlexible {
+				if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+					return err
+				}
+			}
 		}
 	}
 
@@ -144,6 +209,12 @@
 		r.Err = KError(kerr)
 	}
 
+	if isFlexible {
+		if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
@@ -155,6 +226,14 @@
 	return r.Version
 }
 
+func (r *OffsetFetchResponse) headerVersion() int16 {
+	if r.Version >= 6 {
+		return 1
+	}
+
+	return 0
+}
+
 func (r *OffsetFetchResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -167,6 +246,10 @@
 		return V2_0_0_0
 	case 5:
 		return V2_1_0_0
+	case 6:
+		return V2_4_0_0
+	case 7:
+		return V2_5_0_0
 	default:
 		return MinVersion
 	}
diff --git a/vendor/github.com/Shopify/sarama/offset_manager.go b/vendor/github.com/Shopify/sarama/offset_manager.go
index 923972f..4f480a0 100644
--- a/vendor/github.com/Shopify/sarama/offset_manager.go
+++ b/vendor/github.com/Shopify/sarama/offset_manager.go
@@ -19,6 +19,10 @@
 	// will otherwise leak memory. You must call this after all the
 	// PartitionOffsetManagers are closed.
 	Close() error
+
+	// Commit commits the offsets. This method can be used if AutoCommit.Enable is
+	// set to false.
+	Commit()
 }
 
 type offsetManager struct {
@@ -58,7 +62,6 @@
 		client: client,
 		conf:   conf,
 		group:  group,
-		ticker: time.NewTicker(conf.Consumer.Offsets.CommitInterval),
 		poms:   make(map[string]map[int32]*partitionOffsetManager),
 
 		memberID:   memberID,
@@ -67,7 +70,10 @@
 		closing: make(chan none),
 		closed:  make(chan none),
 	}
-	go withRecover(om.mainLoop)
+	if conf.Consumer.Offsets.AutoCommit.Enable {
+		om.ticker = time.NewTicker(conf.Consumer.Offsets.AutoCommit.Interval)
+		go withRecover(om.mainLoop)
+	}
 
 	return om, nil
 }
@@ -99,16 +105,20 @@
 	om.closeOnce.Do(func() {
 		// exit the mainLoop
 		close(om.closing)
-		<-om.closed
+		if om.conf.Consumer.Offsets.AutoCommit.Enable {
+			<-om.closed
+		}
 
 		// mark all POMs as closed
 		om.asyncClosePOMs()
 
 		// flush one last time
-		for attempt := 0; attempt <= om.conf.Consumer.Offsets.Retry.Max; attempt++ {
-			om.flushToBroker()
-			if om.releasePOMs(false) == 0 {
-				break
+		if om.conf.Consumer.Offsets.AutoCommit.Enable {
+			for attempt := 0; attempt <= om.conf.Consumer.Offsets.Retry.Max; attempt++ {
+				om.flushToBroker()
+				if om.releasePOMs(false) == 0 {
+					break
+				}
 			}
 		}
 
@@ -225,14 +235,18 @@
 	for {
 		select {
 		case <-om.ticker.C:
-			om.flushToBroker()
-			om.releasePOMs(false)
+			om.Commit()
 		case <-om.closing:
 			return
 		}
 	}
 }
 
+func (om *offsetManager) Commit() {
+	om.flushToBroker()
+	om.releasePOMs(false)
+}
+
 func (om *offsetManager) flushToBroker() {
 	req := om.constructRequest()
 	if req == nil {
@@ -275,7 +289,6 @@
 			ConsumerID:              om.memberID,
 			ConsumerGroupGeneration: om.generation,
 		}
-
 	}
 
 	om.pomsLock.RLock()
diff --git a/vendor/github.com/Shopify/sarama/offset_request.go b/vendor/github.com/Shopify/sarama/offset_request.go
index 326c372..4c9ce4d 100644
--- a/vendor/github.com/Shopify/sarama/offset_request.go
+++ b/vendor/github.com/Shopify/sarama/offset_request.go
@@ -6,7 +6,7 @@
 }
 
 func (b *offsetRequestBlock) encode(pe packetEncoder, version int16) error {
-	pe.putInt64(int64(b.time))
+	pe.putInt64(b.time)
 	if version == 0 {
 		pe.putInt32(b.maxOffsets)
 	}
@@ -28,6 +28,7 @@
 
 type OffsetRequest struct {
 	Version        int16
+	IsolationLevel IsolationLevel
 	replicaID      int32
 	isReplicaIDSet bool
 	blocks         map[string]map[int32]*offsetRequestBlock
@@ -41,6 +42,10 @@
 		pe.putInt32(-1)
 	}
 
+	if r.Version >= 2 {
+		pe.putBool(r.IsolationLevel == ReadCommitted)
+	}
+
 	err := pe.putArrayLength(len(r.blocks))
 	if err != nil {
 		return err
@@ -75,6 +80,18 @@
 		r.SetReplicaID(replicaID)
 	}
 
+	if r.Version >= 2 {
+		tmp, err := pd.getBool()
+		if err != nil {
+			return err
+		}
+
+		r.IsolationLevel = ReadUncommitted
+		if tmp {
+			r.IsolationLevel = ReadCommitted
+		}
+	}
+
 	blockCount, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -116,10 +133,16 @@
 	return r.Version
 }
 
+func (r *OffsetRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *OffsetRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
 		return V0_10_1_0
+	case 2:
+		return V0_11_0_0
 	default:
 		return MinVersion
 	}
diff --git a/vendor/github.com/Shopify/sarama/offset_response.go b/vendor/github.com/Shopify/sarama/offset_response.go
index 8b2193f..69349ef 100644
--- a/vendor/github.com/Shopify/sarama/offset_response.go
+++ b/vendor/github.com/Shopify/sarama/offset_response.go
@@ -50,11 +50,19 @@
 }
 
 type OffsetResponse struct {
-	Version int16
-	Blocks  map[string]map[int32]*OffsetResponseBlock
+	Version        int16
+	ThrottleTimeMs int32
+	Blocks         map[string]map[int32]*OffsetResponseBlock
 }
 
 func (r *OffsetResponse) decode(pd packetDecoder, version int16) (err error) {
+	if version >= 2 {
+		r.ThrottleTimeMs, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	}
+
 	numTopics, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -120,6 +128,10 @@
 
 */
 func (r *OffsetResponse) encode(pe packetEncoder) (err error) {
+	if r.Version >= 2 {
+		pe.putInt32(r.ThrottleTimeMs)
+	}
+
 	if err = pe.putArrayLength(len(r.Blocks)); err != nil {
 		return err
 	}
@@ -150,10 +162,16 @@
 	return r.Version
 }
 
+func (r *OffsetResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *OffsetResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
 		return V0_10_1_0
+	case 2:
+		return V0_11_0_0
 	default:
 		return MinVersion
 	}
diff --git a/vendor/github.com/Shopify/sarama/packet_decoder.go b/vendor/github.com/Shopify/sarama/packet_decoder.go
index 9be854c..184bc26 100644
--- a/vendor/github.com/Shopify/sarama/packet_decoder.go
+++ b/vendor/github.com/Shopify/sarama/packet_decoder.go
@@ -10,15 +10,22 @@
 	getInt32() (int32, error)
 	getInt64() (int64, error)
 	getVarint() (int64, error)
+	getUVarint() (uint64, error)
 	getArrayLength() (int, error)
+	getCompactArrayLength() (int, error)
 	getBool() (bool, error)
+	getEmptyTaggedFieldArray() (int, error)
 
 	// Collections
 	getBytes() ([]byte, error)
 	getVarintBytes() ([]byte, error)
+	getCompactBytes() ([]byte, error)
 	getRawBytes(length int) ([]byte, error)
 	getString() (string, error)
 	getNullableString() (*string, error)
+	getCompactString() (string, error)
+	getCompactNullableString() (*string, error)
+	getCompactInt32Array() ([]int32, error)
 	getInt32Array() ([]int32, error)
 	getInt64Array() ([]int64, error)
 	getStringArray() ([]string, error)
diff --git a/vendor/github.com/Shopify/sarama/packet_encoder.go b/vendor/github.com/Shopify/sarama/packet_encoder.go
index 67b8dae..aea53ca 100644
--- a/vendor/github.com/Shopify/sarama/packet_encoder.go
+++ b/vendor/github.com/Shopify/sarama/packet_encoder.go
@@ -12,18 +12,26 @@
 	putInt32(in int32)
 	putInt64(in int64)
 	putVarint(in int64)
+	putUVarint(in uint64)
+	putCompactArrayLength(in int)
 	putArrayLength(in int) error
 	putBool(in bool)
 
 	// Collections
 	putBytes(in []byte) error
 	putVarintBytes(in []byte) error
+	putCompactBytes(in []byte) error
 	putRawBytes(in []byte) error
+	putCompactString(in string) error
+	putNullableCompactString(in *string) error
 	putString(in string) error
 	putNullableString(in *string) error
 	putStringArray(in []string) error
+	putCompactInt32Array(in []int32) error
+	putNullableCompactInt32Array(in []int32) error
 	putInt32Array(in []int32) error
 	putInt64Array(in []int64) error
+	putEmptyTaggedFieldArray()
 
 	// Provide the current offset to record the batch size metric
 	offset() int
diff --git a/vendor/github.com/Shopify/sarama/partitioner.go b/vendor/github.com/Shopify/sarama/partitioner.go
index 6a708e7..a66e11e 100644
--- a/vendor/github.com/Shopify/sarama/partitioner.go
+++ b/vendor/github.com/Shopify/sarama/partitioner.go
@@ -42,7 +42,7 @@
 
 type manualPartitioner struct{}
 
-// HashPartitionOption lets you modify default values of the partitioner
+// HashPartitionerOption lets you modify default values of the partitioner
 type HashPartitionerOption func(*hashPartitioner)
 
 // WithAbsFirst means that the partitioner handles absolute values
diff --git a/vendor/github.com/Shopify/sarama/prep_encoder.go b/vendor/github.com/Shopify/sarama/prep_encoder.go
index b633cd1..0d01374 100644
--- a/vendor/github.com/Shopify/sarama/prep_encoder.go
+++ b/vendor/github.com/Shopify/sarama/prep_encoder.go
@@ -2,6 +2,7 @@
 
 import (
 	"encoding/binary"
+	"errors"
 	"fmt"
 	"math"
 
@@ -36,6 +37,11 @@
 	pe.length += binary.PutVarint(buf[:], in)
 }
 
+func (pe *prepEncoder) putUVarint(in uint64) {
+	var buf [binary.MaxVarintLen64]byte
+	pe.length += binary.PutUvarint(buf[:], in)
+}
+
 func (pe *prepEncoder) putArrayLength(in int) error {
 	if in > math.MaxInt32 {
 		return PacketEncodingError{fmt.Sprintf("array too long (%d)", in)}
@@ -44,6 +50,10 @@
 	return nil
 }
 
+func (pe *prepEncoder) putCompactArrayLength(in int) {
+	pe.putUVarint(uint64(in + 1))
+}
+
 func (pe *prepEncoder) putBool(in bool) {
 	pe.length++
 }
@@ -67,6 +77,25 @@
 	return pe.putRawBytes(in)
 }
 
+func (pe *prepEncoder) putCompactBytes(in []byte) error {
+	pe.putUVarint(uint64(len(in) + 1))
+	return pe.putRawBytes(in)
+}
+
+func (pe *prepEncoder) putCompactString(in string) error {
+	pe.putCompactArrayLength(len(in))
+	return pe.putRawBytes([]byte(in))
+}
+
+func (pe *prepEncoder) putNullableCompactString(in *string) error {
+	if in == nil {
+		pe.putUVarint(0)
+		return nil
+	} else {
+		return pe.putCompactString(*in)
+	}
+}
+
 func (pe *prepEncoder) putRawBytes(in []byte) error {
 	if len(in) > math.MaxInt32 {
 		return PacketEncodingError{fmt.Sprintf("byteslice too long (%d)", len(in))}
@@ -107,6 +136,27 @@
 	return nil
 }
 
+func (pe *prepEncoder) putCompactInt32Array(in []int32) error {
+	if in == nil {
+		return errors.New("expected int32 array to be non null")
+	}
+
+	pe.putUVarint(uint64(len(in)) + 1)
+	pe.length += 4 * len(in)
+	return nil
+}
+
+func (pe *prepEncoder) putNullableCompactInt32Array(in []int32) error {
+	if in == nil {
+		pe.putUVarint(0)
+		return nil
+	}
+
+	pe.putUVarint(uint64(len(in)) + 1)
+	pe.length += 4 * len(in)
+	return nil
+}
+
 func (pe *prepEncoder) putInt32Array(in []int32) error {
 	err := pe.putArrayLength(len(in))
 	if err != nil {
@@ -125,6 +175,10 @@
 	return nil
 }
 
+func (pe *prepEncoder) putEmptyTaggedFieldArray() {
+	pe.putUVarint(0)
+}
+
 func (pe *prepEncoder) offset() int {
 	return pe.length
 }
diff --git a/vendor/github.com/Shopify/sarama/produce_request.go b/vendor/github.com/Shopify/sarama/produce_request.go
index 0c755d0..0034651 100644
--- a/vendor/github.com/Shopify/sarama/produce_request.go
+++ b/vendor/github.com/Shopify/sarama/produce_request.go
@@ -206,6 +206,10 @@
 	return r.Version
 }
 
+func (r *ProduceRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *ProduceRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -214,6 +218,8 @@
 		return V0_10_0_0
 	case 3:
 		return V0_11_0_0
+	case 7:
+		return V2_1_0_0
 	default:
 		return MinVersion
 	}
diff --git a/vendor/github.com/Shopify/sarama/produce_response.go b/vendor/github.com/Shopify/sarama/produce_response.go
index 4c5cd35..edf9787 100644
--- a/vendor/github.com/Shopify/sarama/produce_response.go
+++ b/vendor/github.com/Shopify/sarama/produce_response.go
@@ -5,11 +5,27 @@
 	"time"
 )
 
+// Protocol, http://kafka.apache.org/protocol.html
+// v1
+// v2 = v3 = v4
+// v5 = v6 = v7
+// Produce Response (Version: 7) => [responses] throttle_time_ms
+//   responses => topic [partition_responses]
+//     topic => STRING
+//     partition_responses => partition error_code base_offset log_append_time log_start_offset
+//       partition => INT32
+//       error_code => INT16
+//       base_offset => INT64
+//       log_append_time => INT64
+//       log_start_offset => INT64
+//   throttle_time_ms => INT32
+
+// partition_responses in protocol
 type ProduceResponseBlock struct {
-	Err    KError
-	Offset int64
-	// only provided if Version >= 2 and the broker is configured with `LogAppendTime`
-	Timestamp time.Time
+	Err         KError    // v0, error_code
+	Offset      int64     // v0, base_offset
+	Timestamp   time.Time // v2, log_append_time, and the broker is configured with `LogAppendTime`
+	StartOffset int64     // v5, log_start_offset
 }
 
 func (b *ProduceResponseBlock) decode(pd packetDecoder, version int16) (err error) {
@@ -32,6 +48,13 @@
 		}
 	}
 
+	if version >= 5 {
+		b.StartOffset, err = pd.getInt64()
+		if err != nil {
+			return err
+		}
+	}
+
 	return nil
 }
 
@@ -49,13 +72,17 @@
 		pe.putInt64(timestamp)
 	}
 
+	if version >= 5 {
+		pe.putInt64(b.StartOffset)
+	}
+
 	return nil
 }
 
 type ProduceResponse struct {
-	Blocks       map[string]map[int32]*ProduceResponseBlock
+	Blocks       map[string]map[int32]*ProduceResponseBlock // v0, responses
 	Version      int16
-	ThrottleTime time.Duration // only provided if Version >= 1
+	ThrottleTime time.Duration // v1, throttle_time_ms
 }
 
 func (r *ProduceResponse) decode(pd packetDecoder, version int16) (err error) {
@@ -129,6 +156,7 @@
 			}
 		}
 	}
+
 	if r.Version >= 1 {
 		pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
 	}
@@ -143,17 +171,12 @@
 	return r.Version
 }
 
+func (r *ProduceResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *ProduceResponse) requiredVersion() KafkaVersion {
-	switch r.Version {
-	case 1:
-		return V0_9_0_0
-	case 2:
-		return V0_10_0_0
-	case 3:
-		return V0_11_0_0
-	default:
-		return MinVersion
-	}
+	return MinVersion
 }
 
 func (r *ProduceResponse) GetBlock(topic string, partition int32) *ProduceResponseBlock {
diff --git a/vendor/github.com/Shopify/sarama/produce_set.go b/vendor/github.com/Shopify/sarama/produce_set.go
index bba0f7e..9c70f81 100644
--- a/vendor/github.com/Shopify/sarama/produce_set.go
+++ b/vendor/github.com/Shopify/sarama/produce_set.go
@@ -13,17 +13,22 @@
 }
 
 type produceSet struct {
-	parent *asyncProducer
-	msgs   map[string]map[int32]*partitionSet
+	parent        *asyncProducer
+	msgs          map[string]map[int32]*partitionSet
+	producerID    int64
+	producerEpoch int16
 
 	bufferBytes int
 	bufferCount int
 }
 
 func newProduceSet(parent *asyncProducer) *produceSet {
+	pid, epoch := parent.txnmgr.getProducerID()
 	return &produceSet{
-		msgs:   make(map[string]map[int32]*partitionSet),
-		parent: parent,
+		msgs:          make(map[string]map[int32]*partitionSet),
+		parent:        parent,
+		producerID:    pid,
+		producerEpoch: epoch,
 	}
 }
 
@@ -44,9 +49,10 @@
 	}
 
 	timestamp := msg.Timestamp
-	if msg.Timestamp.IsZero() {
+	if timestamp.IsZero() {
 		timestamp = time.Now()
 	}
+	timestamp = timestamp.Truncate(time.Millisecond)
 
 	partitions := ps.msgs[msg.Topic]
 	if partitions == nil {
@@ -64,8 +70,8 @@
 				Version:          2,
 				Codec:            ps.parent.conf.Producer.Compression,
 				CompressionLevel: ps.parent.conf.Producer.CompressionLevel,
-				ProducerID:       ps.parent.txnmgr.producerID,
-				ProducerEpoch:    ps.parent.txnmgr.producerEpoch,
+				ProducerID:       ps.producerID,
+				ProducerEpoch:    ps.producerEpoch,
 			}
 			if ps.parent.conf.Producer.Idempotent {
 				batch.FirstSequence = msg.sequenceNumber
@@ -77,12 +83,17 @@
 		}
 		partitions[msg.Partition] = set
 	}
-	set.msgs = append(set.msgs, msg)
 
 	if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) {
 		if ps.parent.conf.Producer.Idempotent && msg.sequenceNumber < set.recordsToSend.RecordBatch.FirstSequence {
 			return errors.New("assertion failed: message out of sequence added to a batch")
 		}
+	}
+
+	// Past this point we can't return an error, because we've already added the message to the set.
+	set.msgs = append(set.msgs, msg)
+
+	if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) {
 		// We are being conservative here to avoid having to prep encode the record
 		size += maximumRecordOverhead
 		rec := &Record{
@@ -128,6 +139,10 @@
 		req.Version = 3
 	}
 
+	if ps.parent.conf.Producer.Compression == CompressionZSTD && ps.parent.conf.Version.IsAtLeast(V2_1_0_0) {
+		req.Version = 7
+	}
+
 	for topic, partitionSets := range ps.msgs {
 		for partition, set := range partitionSets {
 			if req.Version >= 3 {
diff --git a/vendor/github.com/Shopify/sarama/real_decoder.go b/vendor/github.com/Shopify/sarama/real_decoder.go
index 085cbb3..2482c63 100644
--- a/vendor/github.com/Shopify/sarama/real_decoder.go
+++ b/vendor/github.com/Shopify/sarama/real_decoder.go
@@ -5,13 +5,15 @@
 	"math"
 )
 
-var errInvalidArrayLength = PacketDecodingError{"invalid array length"}
-var errInvalidByteSliceLength = PacketDecodingError{"invalid byteslice length"}
-var errInvalidByteSliceLengthType = PacketDecodingError{"invalid byteslice length type"}
-var errInvalidStringLength = PacketDecodingError{"invalid string length"}
-var errInvalidSubsetSize = PacketDecodingError{"invalid subset size"}
-var errVarintOverflow = PacketDecodingError{"varint overflow"}
-var errInvalidBool = PacketDecodingError{"invalid bool"}
+var (
+	errInvalidArrayLength      = PacketDecodingError{"invalid array length"}
+	errInvalidByteSliceLength  = PacketDecodingError{"invalid byteslice length"}
+	errInvalidStringLength     = PacketDecodingError{"invalid string length"}
+	errVarintOverflow          = PacketDecodingError{"varint overflow"}
+	errUVarintOverflow         = PacketDecodingError{"uvarint overflow"}
+	errInvalidBool             = PacketDecodingError{"invalid bool"}
+	errUnsupportedTaggedFields = PacketDecodingError{"non-empty tagged fields are not supported yet"}
+)
 
 type realDecoder struct {
 	raw   []byte
@@ -75,6 +77,22 @@
 	return tmp, nil
 }
 
+func (rd *realDecoder) getUVarint() (uint64, error) {
+	tmp, n := binary.Uvarint(rd.raw[rd.off:])
+	if n == 0 {
+		rd.off = len(rd.raw)
+		return 0, ErrInsufficientData
+	}
+
+	if n < 0 {
+		rd.off -= n
+		return 0, errUVarintOverflow
+	}
+
+	rd.off += n
+	return tmp, nil
+}
+
 func (rd *realDecoder) getArrayLength() (int, error) {
 	if rd.remaining() < 4 {
 		rd.off = len(rd.raw)
@@ -91,6 +109,19 @@
 	return tmp, nil
 }
 
+func (rd *realDecoder) getCompactArrayLength() (int, error) {
+	n, err := rd.getUVarint()
+	if err != nil {
+		return 0, err
+	}
+
+	if n == 0 {
+		return 0, nil
+	}
+
+	return int(n) - 1, nil
+}
+
 func (rd *realDecoder) getBool() (bool, error) {
 	b, err := rd.getInt8()
 	if err != nil || b == 0 {
@@ -102,6 +133,19 @@
 	return true, nil
 }
 
+func (rd *realDecoder) getEmptyTaggedFieldArray() (int, error) {
+	tagCount, err := rd.getUVarint()
+	if err != nil {
+		return 0, err
+	}
+
+	if tagCount != 0 {
+		return 0, errUnsupportedTaggedFields
+	}
+
+	return 0, nil
+}
+
 // collections
 
 func (rd *realDecoder) getBytes() ([]byte, error) {
@@ -128,6 +172,16 @@
 	return rd.getRawBytes(int(tmp))
 }
 
+func (rd *realDecoder) getCompactBytes() ([]byte, error) {
+	n, err := rd.getUVarint()
+	if err != nil {
+		return nil, err
+	}
+
+	length := int(n - 1)
+	return rd.getRawBytes(length)
+}
+
 func (rd *realDecoder) getStringLength() (int, error) {
 	length, err := rd.getInt16()
 	if err != nil {
@@ -169,6 +223,57 @@
 	return &tmpStr, err
 }
 
+func (rd *realDecoder) getCompactString() (string, error) {
+	n, err := rd.getUVarint()
+	if err != nil {
+		return "", err
+	}
+
+	length := int(n - 1)
+
+	tmpStr := string(rd.raw[rd.off : rd.off+length])
+	rd.off += length
+	return tmpStr, nil
+}
+
+func (rd *realDecoder) getCompactNullableString() (*string, error) {
+	n, err := rd.getUVarint()
+	if err != nil {
+		return nil, err
+	}
+
+	length := int(n - 1)
+
+	if length < 0 {
+		return nil, err
+	}
+
+	tmpStr := string(rd.raw[rd.off : rd.off+length])
+	rd.off += length
+	return &tmpStr, err
+}
+
+func (rd *realDecoder) getCompactInt32Array() ([]int32, error) {
+	n, err := rd.getUVarint()
+	if err != nil {
+		return nil, err
+	}
+
+	if n == 0 {
+		return nil, nil
+	}
+
+	arrayLength := int(n) - 1
+
+	ret := make([]int32, arrayLength)
+
+	for i := range ret {
+		ret[i] = int32(binary.BigEndian.Uint32(rd.raw[rd.off:]))
+		rd.off += 4
+	}
+	return ret, nil
+}
+
 func (rd *realDecoder) getInt32Array() ([]int32, error) {
 	if rd.remaining() < 4 {
 		rd.off = len(rd.raw)
diff --git a/vendor/github.com/Shopify/sarama/real_encoder.go b/vendor/github.com/Shopify/sarama/real_encoder.go
index 3c75387..c07204c 100644
--- a/vendor/github.com/Shopify/sarama/real_encoder.go
+++ b/vendor/github.com/Shopify/sarama/real_encoder.go
@@ -2,6 +2,7 @@
 
 import (
 	"encoding/binary"
+	"errors"
 
 	"github.com/rcrowley/go-metrics"
 )
@@ -39,11 +40,20 @@
 	re.off += binary.PutVarint(re.raw[re.off:], in)
 }
 
+func (re *realEncoder) putUVarint(in uint64) {
+	re.off += binary.PutUvarint(re.raw[re.off:], in)
+}
+
 func (re *realEncoder) putArrayLength(in int) error {
 	re.putInt32(int32(in))
 	return nil
 }
 
+func (re *realEncoder) putCompactArrayLength(in int) {
+	// 0 represents a null array, so +1 has to be added
+	re.putUVarint(uint64(in + 1))
+}
+
 func (re *realEncoder) putBool(in bool) {
 	if in {
 		re.putInt8(1)
@@ -78,6 +88,24 @@
 	return re.putRawBytes(in)
 }
 
+func (re *realEncoder) putCompactBytes(in []byte) error {
+	re.putUVarint(uint64(len(in) + 1))
+	return re.putRawBytes(in)
+}
+
+func (re *realEncoder) putCompactString(in string) error {
+	re.putCompactArrayLength(len(in))
+	return re.putRawBytes([]byte(in))
+}
+
+func (re *realEncoder) putNullableCompactString(in *string) error {
+	if in == nil {
+		re.putInt8(0)
+		return nil
+	}
+	return re.putCompactString(*in)
+}
+
 func (re *realEncoder) putString(in string) error {
 	re.putInt16(int16(len(in)))
 	copy(re.raw[re.off:], in)
@@ -108,6 +136,31 @@
 	return nil
 }
 
+func (re *realEncoder) putCompactInt32Array(in []int32) error {
+	if in == nil {
+		return errors.New("expected int32 array to be non null")
+	}
+	// 0 represents a null array, so +1 has to be added
+	re.putUVarint(uint64(len(in)) + 1)
+	for _, val := range in {
+		re.putInt32(val)
+	}
+	return nil
+}
+
+func (re *realEncoder) putNullableCompactInt32Array(in []int32) error {
+	if in == nil {
+		re.putUVarint(0)
+		return nil
+	}
+	// 0 represents a null array, so +1 has to be added
+	re.putUVarint(uint64(len(in)) + 1)
+	for _, val := range in {
+		re.putInt32(val)
+	}
+	return nil
+}
+
 func (re *realEncoder) putInt32Array(in []int32) error {
 	err := re.putArrayLength(len(in))
 	if err != nil {
@@ -130,6 +183,10 @@
 	return nil
 }
 
+func (re *realEncoder) putEmptyTaggedFieldArray() {
+	re.putUVarint(0)
+}
+
 func (re *realEncoder) offset() int {
 	return re.off
 }
diff --git a/vendor/github.com/Shopify/sarama/record.go b/vendor/github.com/Shopify/sarama/record.go
index cdccfe3..a3fe8c0 100644
--- a/vendor/github.com/Shopify/sarama/record.go
+++ b/vendor/github.com/Shopify/sarama/record.go
@@ -11,7 +11,7 @@
 	maximumRecordOverhead = 5*binary.MaxVarintLen32 + binary.MaxVarintLen64 + 1
 )
 
-//RecordHeader stores key and value for a record header
+// RecordHeader stores key and value for a record header
 type RecordHeader struct {
 	Key   []byte
 	Value []byte
@@ -35,7 +35,7 @@
 	return nil
 }
 
-//Record is kafka record type
+// Record is kafka record type
 type Record struct {
 	Headers []*RecordHeader
 
diff --git a/vendor/github.com/Shopify/sarama/records.go b/vendor/github.com/Shopify/sarama/records.go
index 98160c7..f4c5e95 100644
--- a/vendor/github.com/Shopify/sarama/records.go
+++ b/vendor/github.com/Shopify/sarama/records.go
@@ -8,7 +8,6 @@
 	defaultRecords
 
 	magicOffset = 16
-	magicLength = 1
 )
 
 // Records implements a union type containing either a RecordBatch or a legacy MessageSet.
diff --git a/vendor/github.com/Shopify/sarama/request.go b/vendor/github.com/Shopify/sarama/request.go
index 5ed8ca4..d899df5 100644
--- a/vendor/github.com/Shopify/sarama/request.go
+++ b/vendor/github.com/Shopify/sarama/request.go
@@ -11,6 +11,7 @@
 	versionedDecoder
 	key() int16
 	version() int16
+	headerVersion() int16
 	requiredVersion() KafkaVersion
 }
 
@@ -26,12 +27,19 @@
 	pe.putInt16(r.body.version())
 	pe.putInt32(r.correlationID)
 
-	err := pe.putString(r.clientID)
-	if err != nil {
-		return err
+	if r.body.headerVersion() >= 1 {
+		err := pe.putString(r.clientID)
+		if err != nil {
+			return err
+		}
 	}
 
-	err = r.body.encode(pe)
+	if r.body.headerVersion() >= 2 {
+		// we don't use tag headers at the moment so we just put an array length of 0
+		pe.putUVarint(0)
+	}
+
+	err := r.body.encode(pe)
 	if err != nil {
 		return err
 	}
@@ -65,6 +73,14 @@
 		return PacketDecodingError{fmt.Sprintf("unknown request key (%d)", key)}
 	}
 
+	if r.body.headerVersion() >= 2 {
+		// tagged field
+		_, err = pd.getUVarint()
+		if err != nil {
+			return err
+		}
+	}
+
 	return r.body.decode(pd, version)
 }
 
@@ -105,7 +121,7 @@
 	case 0:
 		return &ProduceRequest{}
 	case 1:
-		return &FetchRequest{}
+		return &FetchRequest{Version: version}
 	case 2:
 		return &OffsetRequest{Version: version}
 	case 3:
@@ -113,7 +129,7 @@
 	case 8:
 		return &OffsetCommitRequest{Version: version}
 	case 9:
-		return &OffsetFetchRequest{}
+		return &OffsetFetchRequest{Version: version}
 	case 10:
 		return &FindCoordinatorRequest{}
 	case 11:
@@ -158,12 +174,24 @@
 		return &DescribeConfigsRequest{}
 	case 33:
 		return &AlterConfigsRequest{}
+	case 35:
+		return &DescribeLogDirsRequest{}
 	case 36:
 		return &SaslAuthenticateRequest{}
 	case 37:
 		return &CreatePartitionsRequest{}
 	case 42:
 		return &DeleteGroupsRequest{}
+	case 44:
+		return &IncrementalAlterConfigsRequest{}
+	case 45:
+		return &AlterPartitionReassignmentsRequest{}
+	case 46:
+		return &ListPartitionReassignmentsRequest{}
+	case 50:
+		return &DescribeUserScramCredentialsRequest{}
+	case 51:
+		return &AlterUserScramCredentialsRequest{}
 	}
 	return nil
 }
diff --git a/vendor/github.com/Shopify/sarama/response_header.go b/vendor/github.com/Shopify/sarama/response_header.go
index 7a75918..fbcef0b 100644
--- a/vendor/github.com/Shopify/sarama/response_header.go
+++ b/vendor/github.com/Shopify/sarama/response_header.go
@@ -2,15 +2,17 @@
 
 import "fmt"
 
-const responseLengthSize = 4
-const correlationIDSize = 4
+const (
+	responseLengthSize = 4
+	correlationIDSize  = 4
+)
 
 type responseHeader struct {
 	length        int32
 	correlationID int32
 }
 
-func (r *responseHeader) decode(pd packetDecoder) (err error) {
+func (r *responseHeader) decode(pd packetDecoder, version int16) (err error) {
 	r.length, err = pd.getInt32()
 	if err != nil {
 		return err
@@ -20,5 +22,12 @@
 	}
 
 	r.correlationID, err = pd.getInt32()
+
+	if version >= 1 {
+		if _, err := pd.getEmptyTaggedFieldArray(); err != nil {
+			return err
+		}
+	}
+
 	return err
 }
diff --git a/vendor/github.com/Shopify/sarama/sarama.go b/vendor/github.com/Shopify/sarama/sarama.go
index 1e0277a..48f362d 100644
--- a/vendor/github.com/Shopify/sarama/sarama.go
+++ b/vendor/github.com/Shopify/sarama/sarama.go
@@ -39,6 +39,10 @@
 	| response-rate-for-broker-<broker-id>         | meter      | Responses/second received from a given broker                 |
 	| response-size                                | histogram  | Distribution of the response size in bytes for all brokers    |
 	| response-size-for-broker-<broker-id>         | histogram  | Distribution of the response size in bytes for a given broker |
+	| requests-in-flight                           | counter    | The current number of in-flight requests awaiting a response  |
+	|                                              |            | for all brokers                                               |
+	| requests-in-flight-for-broker-<broker-id>    | counter    | The current number of in-flight requests awaiting a response  |
+	|                                              |            | for a given broker                                            |
 	+----------------------------------------------+------------+---------------------------------------------------------------+
 
 Note that we do not gather specific metrics for seed brokers but they are part of the "all brokers" metrics.
diff --git a/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go b/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go
index 54c8b09..90504df 100644
--- a/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go
+++ b/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go
@@ -24,6 +24,10 @@
 	return 0
 }
 
+func (r *SaslAuthenticateRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *SaslAuthenticateRequest) requiredVersion() KafkaVersion {
 	return V1_0_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go b/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go
index 0038c3f..3ef57b5 100644
--- a/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go
+++ b/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go
@@ -39,6 +39,10 @@
 	return 0
 }
 
+func (r *SaslAuthenticateResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *SaslAuthenticateResponse) requiredVersion() KafkaVersion {
 	return V1_0_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/sasl_handshake_request.go b/vendor/github.com/Shopify/sarama/sasl_handshake_request.go
index fe5ba05..74dc307 100644
--- a/vendor/github.com/Shopify/sarama/sasl_handshake_request.go
+++ b/vendor/github.com/Shopify/sarama/sasl_handshake_request.go
@@ -29,6 +29,10 @@
 	return r.Version
 }
 
+func (r *SaslHandshakeRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *SaslHandshakeRequest) requiredVersion() KafkaVersion {
 	return V0_10_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/sasl_handshake_response.go b/vendor/github.com/Shopify/sarama/sasl_handshake_response.go
index ef290d4..69dfc31 100644
--- a/vendor/github.com/Shopify/sarama/sasl_handshake_response.go
+++ b/vendor/github.com/Shopify/sarama/sasl_handshake_response.go
@@ -33,6 +33,10 @@
 	return 0
 }
 
+func (r *SaslHandshakeResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *SaslHandshakeResponse) requiredVersion() KafkaVersion {
 	return V0_10_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/scram_formatter.go b/vendor/github.com/Shopify/sarama/scram_formatter.go
new file mode 100644
index 0000000..2af9e4a
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/scram_formatter.go
@@ -0,0 +1,78 @@
+package sarama
+
+import (
+	"crypto/hmac"
+	"crypto/sha256"
+	"crypto/sha512"
+	"hash"
+)
+
+// ScramFormatter implementation
+// @see: https://github.com/apache/kafka/blob/99b9b3e84f4e98c3f07714e1de6a139a004cbc5b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramFormatter.java#L93
+type scramFormatter struct {
+	mechanism ScramMechanismType
+}
+
+func (s scramFormatter) mac(key []byte) (hash.Hash, error) {
+	var m hash.Hash
+
+	switch s.mechanism {
+	case SCRAM_MECHANISM_SHA_256:
+		m = hmac.New(sha256.New, key)
+
+	case SCRAM_MECHANISM_SHA_512:
+		m = hmac.New(sha512.New, key)
+	default:
+		return nil, ErrUnknownScramMechanism
+	}
+
+	return m, nil
+}
+
+func (s scramFormatter) hmac(key []byte, extra []byte) ([]byte, error) {
+	mac, err := s.mac(key)
+	if err != nil {
+		return nil, err
+	}
+
+	if _, err := mac.Write(extra); err != nil {
+		return nil, err
+	}
+	return mac.Sum(nil), nil
+}
+
+func (s scramFormatter) xor(result []byte, second []byte) {
+	for i := 0; i < len(result); i++ {
+		result[i] = result[i] ^ second[i]
+	}
+}
+
+func (s scramFormatter) saltedPassword(password []byte, salt []byte, iterations int) ([]byte, error) {
+	mac, err := s.mac(password)
+	if err != nil {
+		return nil, err
+	}
+
+	if _, err := mac.Write(salt); err != nil {
+		return nil, err
+	}
+	if _, err := mac.Write([]byte{0, 0, 0, 1}); err != nil {
+		return nil, err
+	}
+
+	u1 := mac.Sum(nil)
+	prev := u1
+	result := u1
+
+	for i := 2; i <= iterations; i++ {
+		ui, err := s.hmac(password, prev)
+		if err != nil {
+			return nil, err
+		}
+
+		s.xor(result, ui)
+		prev = ui
+	}
+
+	return result, nil
+}
diff --git a/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go b/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go
new file mode 100644
index 0000000..161233f
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go
@@ -0,0 +1,124 @@
+package sarama
+
+type topicPartitionAssignment struct {
+	Topic     string
+	Partition int32
+}
+
+type StickyAssignorUserData interface {
+	partitions() []topicPartitionAssignment
+	hasGeneration() bool
+	generation() int
+}
+
+// StickyAssignorUserDataV0 holds topic partition information for an assignment
+type StickyAssignorUserDataV0 struct {
+	Topics map[string][]int32
+
+	topicPartitions []topicPartitionAssignment
+}
+
+func (m *StickyAssignorUserDataV0) encode(pe packetEncoder) error {
+	if err := pe.putArrayLength(len(m.Topics)); err != nil {
+		return err
+	}
+
+	for topic, partitions := range m.Topics {
+		if err := pe.putString(topic); err != nil {
+			return err
+		}
+		if err := pe.putInt32Array(partitions); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (m *StickyAssignorUserDataV0) decode(pd packetDecoder) (err error) {
+	var topicLen int
+	if topicLen, err = pd.getArrayLength(); err != nil {
+		return
+	}
+
+	m.Topics = make(map[string][]int32, topicLen)
+	for i := 0; i < topicLen; i++ {
+		var topic string
+		if topic, err = pd.getString(); err != nil {
+			return
+		}
+		if m.Topics[topic], err = pd.getInt32Array(); err != nil {
+			return
+		}
+	}
+	m.topicPartitions = populateTopicPartitions(m.Topics)
+	return nil
+}
+
+func (m *StickyAssignorUserDataV0) partitions() []topicPartitionAssignment { return m.topicPartitions }
+func (m *StickyAssignorUserDataV0) hasGeneration() bool                    { return false }
+func (m *StickyAssignorUserDataV0) generation() int                        { return defaultGeneration }
+
+// StickyAssignorUserDataV1 holds topic partition information for an assignment
+type StickyAssignorUserDataV1 struct {
+	Topics     map[string][]int32
+	Generation int32
+
+	topicPartitions []topicPartitionAssignment
+}
+
+func (m *StickyAssignorUserDataV1) encode(pe packetEncoder) error {
+	if err := pe.putArrayLength(len(m.Topics)); err != nil {
+		return err
+	}
+
+	for topic, partitions := range m.Topics {
+		if err := pe.putString(topic); err != nil {
+			return err
+		}
+		if err := pe.putInt32Array(partitions); err != nil {
+			return err
+		}
+	}
+
+	pe.putInt32(m.Generation)
+	return nil
+}
+
+func (m *StickyAssignorUserDataV1) decode(pd packetDecoder) (err error) {
+	var topicLen int
+	if topicLen, err = pd.getArrayLength(); err != nil {
+		return
+	}
+
+	m.Topics = make(map[string][]int32, topicLen)
+	for i := 0; i < topicLen; i++ {
+		var topic string
+		if topic, err = pd.getString(); err != nil {
+			return
+		}
+		if m.Topics[topic], err = pd.getInt32Array(); err != nil {
+			return
+		}
+	}
+
+	m.Generation, err = pd.getInt32()
+	if err != nil {
+		return err
+	}
+	m.topicPartitions = populateTopicPartitions(m.Topics)
+	return nil
+}
+
+func (m *StickyAssignorUserDataV1) partitions() []topicPartitionAssignment { return m.topicPartitions }
+func (m *StickyAssignorUserDataV1) hasGeneration() bool                    { return true }
+func (m *StickyAssignorUserDataV1) generation() int                        { return int(m.Generation) }
+
+func populateTopicPartitions(topics map[string][]int32) []topicPartitionAssignment {
+	topicPartitions := make([]topicPartitionAssignment, 0)
+	for topic, partitions := range topics {
+		for _, partition := range partitions {
+			topicPartitions = append(topicPartitions, topicPartitionAssignment{Topic: topic, Partition: partition})
+		}
+	}
+	return topicPartitions
+}
diff --git a/vendor/github.com/Shopify/sarama/sync_group_request.go b/vendor/github.com/Shopify/sarama/sync_group_request.go
index fe20708..ac6ecb1 100644
--- a/vendor/github.com/Shopify/sarama/sync_group_request.go
+++ b/vendor/github.com/Shopify/sarama/sync_group_request.go
@@ -77,6 +77,10 @@
 	return 0
 }
 
+func (r *SyncGroupRequest) headerVersion() int16 {
+	return 1
+}
+
 func (r *SyncGroupRequest) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/sync_group_response.go b/vendor/github.com/Shopify/sarama/sync_group_response.go
index 194b382..af019c4 100644
--- a/vendor/github.com/Shopify/sarama/sync_group_response.go
+++ b/vendor/github.com/Shopify/sarama/sync_group_response.go
@@ -36,6 +36,10 @@
 	return 0
 }
 
+func (r *SyncGroupResponse) headerVersion() int16 {
+	return 0
+}
+
 func (r *SyncGroupResponse) requiredVersion() KafkaVersion {
 	return V0_9_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go b/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go
index 71e95b8..c4043a3 100644
--- a/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go
+++ b/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go
@@ -91,6 +91,10 @@
 	return 0
 }
 
+func (a *TxnOffsetCommitRequest) headerVersion() int16 {
+	return 1
+}
+
 func (a *TxnOffsetCommitRequest) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go b/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go
index 6c980f4..94d8029 100644
--- a/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go
+++ b/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go
@@ -78,6 +78,10 @@
 	return 0
 }
 
+func (a *TxnOffsetCommitResponse) headerVersion() int16 {
+	return 0
+}
+
 func (a *TxnOffsetCommitResponse) requiredVersion() KafkaVersion {
 	return V0_11_0_0
 }
diff --git a/vendor/github.com/Shopify/sarama/utils.go b/vendor/github.com/Shopify/sarama/utils.go
index 7c815cd..1859d29 100644
--- a/vendor/github.com/Shopify/sarama/utils.go
+++ b/vendor/github.com/Shopify/sarama/utils.go
@@ -26,9 +26,7 @@
 
 func dupInt32Slice(input []int32) []int32 {
 	ret := make([]int32, 0, len(input))
-	for _, val := range input {
-		ret = append(ret, val)
-	}
+	ret = append(ret, input...)
 	return ret
 }
 
@@ -161,6 +159,11 @@
 	V2_1_0_0  = newKafkaVersion(2, 1, 0, 0)
 	V2_2_0_0  = newKafkaVersion(2, 2, 0, 0)
 	V2_3_0_0  = newKafkaVersion(2, 3, 0, 0)
+	V2_4_0_0  = newKafkaVersion(2, 4, 0, 0)
+	V2_5_0_0  = newKafkaVersion(2, 5, 0, 0)
+	V2_6_0_0  = newKafkaVersion(2, 6, 0, 0)
+	V2_7_0_0  = newKafkaVersion(2, 7, 0, 0)
+	V2_8_0_0  = newKafkaVersion(2, 8, 0, 0)
 
 	SupportedVersions = []KafkaVersion{
 		V0_8_2_0,
@@ -185,15 +188,21 @@
 		V2_1_0_0,
 		V2_2_0_0,
 		V2_3_0_0,
+		V2_4_0_0,
+		V2_5_0_0,
+		V2_6_0_0,
+		V2_7_0_0,
+		V2_8_0_0,
 	}
-	MinVersion = V0_8_2_0
-	MaxVersion = V2_3_0_0
+	MinVersion     = V0_8_2_0
+	MaxVersion     = V2_8_0_0
+	DefaultVersion = V1_0_0_0
 )
 
-//ParseKafkaVersion parses and returns kafka version or error from a string
+// ParseKafkaVersion parses and returns kafka version or error from a string
 func ParseKafkaVersion(s string) (KafkaVersion, error) {
 	if len(s) < 5 {
-		return MinVersion, fmt.Errorf("invalid version `%s`", s)
+		return DefaultVersion, fmt.Errorf("invalid version `%s`", s)
 	}
 	var major, minor, veryMinor, patch uint
 	var err error
@@ -203,7 +212,7 @@
 		err = scanKafkaVersion(s, `^\d+\.\d+\.\d+$`, "%d.%d.%d", [3]*uint{&major, &minor, &veryMinor})
 	}
 	if err != nil {
-		return MinVersion, err
+		return DefaultVersion, err
 	}
 	return newKafkaVersion(major, minor, veryMinor, patch), nil
 }
diff --git a/vendor/github.com/Shopify/sarama/zstd.go b/vendor/github.com/Shopify/sarama/zstd.go
new file mode 100644
index 0000000..e23bfc4
--- /dev/null
+++ b/vendor/github.com/Shopify/sarama/zstd.go
@@ -0,0 +1,18 @@
+package sarama
+
+import (
+	"github.com/klauspost/compress/zstd"
+)
+
+var (
+	zstdDec, _ = zstd.NewReader(nil)
+	zstdEnc, _ = zstd.NewWriter(nil, zstd.WithZeroFrames(true))
+)
+
+func zstdDecompress(dst, src []byte) ([]byte, error) {
+	return zstdDec.DecodeAll(src, dst)
+}
+
+func zstdCompress(dst, src []byte) ([]byte, error) {
+	return zstdEnc.EncodeAll(src, dst), nil
+}
diff --git a/vendor/github.com/Shopify/sarama/zstd_cgo.go b/vendor/github.com/Shopify/sarama/zstd_cgo.go
deleted file mode 100644
index f5ccb31..0000000
--- a/vendor/github.com/Shopify/sarama/zstd_cgo.go
+++ /dev/null
@@ -1,13 +0,0 @@
-// +build cgo
-
-package sarama
-
-import "github.com/DataDog/zstd"
-
-func zstdDecompress(dst, src []byte) ([]byte, error) {
-	return zstd.Decompress(dst, src)
-}
-
-func zstdCompressLevel(dst, src []byte, level int) ([]byte, error) {
-	return zstd.CompressLevel(dst, src, level)
-}
diff --git a/vendor/github.com/Shopify/sarama/zstd_fallback.go b/vendor/github.com/Shopify/sarama/zstd_fallback.go
deleted file mode 100644
index 381a56b..0000000
--- a/vendor/github.com/Shopify/sarama/zstd_fallback.go
+++ /dev/null
@@ -1,17 +0,0 @@
-// +build !cgo
-
-package sarama
-
-import (
-	"errors"
-)
-
-var errZstdCgo = errors.New("zstd compression requires building with cgo enabled")
-
-func zstdDecompress(dst, src []byte) ([]byte, error) {
-	return nil, errZstdCgo
-}
-
-func zstdCompressLevel(dst, src []byte, level int) ([]byte, error) {
-	return nil, errZstdCgo
-}
diff --git a/vendor/github.com/buraksezer/consistent/.gitignore b/vendor/github.com/buraksezer/consistent/.gitignore
deleted file mode 100644
index a1338d6..0000000
--- a/vendor/github.com/buraksezer/consistent/.gitignore
+++ /dev/null
@@ -1,14 +0,0 @@
-# Binaries for programs and plugins
-*.exe
-*.dll
-*.so
-*.dylib
-
-# Test binary, build with `go test -c`
-*.test
-
-# Output of the go coverage tool, specifically when used with LiteIDE
-*.out
-
-# Project-local glide cache, RE: https://github.com/Masterminds/glide/issues/736
-.glide/
diff --git a/vendor/github.com/buraksezer/consistent/.travis.yml b/vendor/github.com/buraksezer/consistent/.travis.yml
deleted file mode 100644
index 4f2ee4d..0000000
--- a/vendor/github.com/buraksezer/consistent/.travis.yml
+++ /dev/null
@@ -1 +0,0 @@
-language: go
diff --git a/vendor/github.com/buraksezer/consistent/LICENSE b/vendor/github.com/buraksezer/consistent/LICENSE
deleted file mode 100644
index e470334..0000000
--- a/vendor/github.com/buraksezer/consistent/LICENSE
+++ /dev/null
@@ -1,21 +0,0 @@
-MIT License
-
-Copyright (c) 2018 Burak Sezer
-
-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/buraksezer/consistent/README.md b/vendor/github.com/buraksezer/consistent/README.md
deleted file mode 100644
index bde53d1..0000000
--- a/vendor/github.com/buraksezer/consistent/README.md
+++ /dev/null
@@ -1,235 +0,0 @@
-consistent
-==========
-[![GoDoc](http://img.shields.io/badge/godoc-reference-blue.svg?style=flat)](https://godoc.org/github.com/buraksezer/consistent) [![Build Status](https://travis-ci.org/buraksezer/consistent.svg?branch=master)](https://travis-ci.org/buraksezer/consistent) [![Coverage](http://gocover.io/_badge/github.com/buraksezer/consistent)](http://gocover.io/github.com/buraksezer/consistent) [![Go Report Card](https://goreportcard.com/badge/github.com/buraksezer/consistent)](https://goreportcard.com/report/github.com/buraksezer/consistent) [![License: MIT](https://img.shields.io/badge/License-MIT-yellow.svg)](https://opensource.org/licenses/MIT) [![Mentioned in Awesome Go](https://awesome.re/mentioned-badge.svg)](https://github.com/avelino/awesome-go)  
-
-
-This library provides a consistent hashing function which simultaneously achieves both uniformity and consistency. 
-
-For detailed information about the concept, you should take a look at the following resources:
-
-* [Consistent Hashing with Bounded Loads on Google Research Blog](https://research.googleblog.com/2017/04/consistent-hashing-with-bounded-loads.html)
-* [Improving load balancing with a new consistent-hashing algorithm on Vimeo Engineering Blog](https://medium.com/vimeo-engineering-blog/improving-load-balancing-with-a-new-consistent-hashing-algorithm-9f1bd75709ed)
-* [Consistent Hashing with Bounded Loads paper on arXiv](https://arxiv.org/abs/1608.01350)
-
-Table of Content
-----------------
-
-- [Overview](#overview)
-- [Install](#install)
-- [Configuration](#configuration)
-- [Usage](#usage)
-- [Benchmarks](#benchmarks)
-- [Examples](#examples)
-
-Overview
---------
-
-In this package's context, the keys are distributed among partitions and partitions are distributed among members as well. 
-
-When you create a new consistent instance or call `Add/Remove`:
-
-* The member's name is hashed and inserted into the hash ring,
-* Average load is calculated by the algorithm defined in the paper,
-* Partitions are distributed among members by hashing partition IDs and none of them exceed the average load.
-
-Average load cannot be exceeded. So if all members are loaded at the maximum while trying to add a new member, it panics.
-
-When you want to locate a key by calling `LocateKey`:
-
-* The key(byte slice) is hashed,
-* The result of the hash is mod by the number of partitions,
-* The result of this modulo - `MOD(hash result, partition count)` - is the partition in which the key will be located,
-* Owner of the partition is already determined before calling `LocateKey`. So it returns the partition owner immediately.
-
-No memory is allocated by `consistent` except hashing when you want to locate a key.
-
-Note that the number of partitions cannot be changed after creation. 
-
-Install
--------
-
-With a correctly configured Go environment:
-
-```
-go get github.com/buraksezer/consistent
-```
-
-You will find some useful usage samples in [examples](https://github.com/buraksezer/consistent/tree/master/_examples) folder.
-
-Configuration
--------------
-
-```go
-type Config struct {
-	// Hasher is responsible for generating unsigned, 64 bit hash of provided byte slice.
-	Hasher Hasher
-
-	// Keys are distributed among partitions. Prime numbers are good to
-	// distribute keys uniformly. Select a big PartitionCount if you have
-	// too many keys.
-	PartitionCount int
-
-	// Members are replicated on consistent hash ring. This number controls
-	// the number each member is replicated on the ring.
-	ReplicationFactor int
-
-	// Load is used to calculate average load. See the code, the paper and Google's 
-	// blog post to learn about it.
-	Load float64
-}
-```
-
-Any hash algorithm can be used as hasher which implements Hasher interface. Please take a look at the *Sample* section for an example.
-
-Usage
------
-
-`LocateKey` function finds a member in the cluster for your key:
-```go
-// With a properly configured and initialized consistent instance
-key := []byte("my-key")
-member := c.LocateKey(key)
-```
-It returns a thread-safe copy of the member you added before.
-
-The second most frequently used function is `GetClosestN`. 
-
-```go
-// With a properly configured and initialized consistent instance
-
-key := []byte("my-key")
-members, err := c.GetClosestN(key, 2)
-```
-
-This may be useful to find backup nodes to store your key.
-
-Benchmarks
-----------
-On an early 2015 Macbook:
-
-```
-BenchmarkAddRemove-4     	  100000	     22006 ns/op
-BenchmarkLocateKey-4     	 5000000	       252 ns/op
-BenchmarkGetClosestN-4   	  500000	      2974 ns/op
-```
-
-Examples
---------
-
-The most basic use of consistent package should be like this. For detailed list of functions, [visit godoc.org.](https://godoc.org/github.com/buraksezer/consistent)
-More sample code can be found under [_examples](https://github.com/buraksezer/consistent/tree/master/_examples).
-
-```go
-package main
-
-import (
-	"fmt"
-
-	"github.com/buraksezer/consistent"
-	"github.com/cespare/xxhash"
-)
-
-// In your code, you probably have a custom data type 
-// for your cluster members. Just add a String function to implement 
-// consistent.Member interface.
-type myMember string
-
-func (m myMember) String() string {
-	return string(m)
-}
-
-// consistent package doesn't provide a default hashing function. 
-// You should provide a proper one to distribute keys/members uniformly.
-type hasher struct{}
-
-func (h hasher) Sum64(data []byte) uint64 {
-	// you should use a proper hash function for uniformity.
-	return xxhash.Sum64(data)
-}
-
-func main() {
-	// Create a new consistent instance
-	cfg := consistent.Config{
-		PartitionCount:    7,
-		ReplicationFactor: 20,
-		Load:              1.25,
-		Hasher:            hasher{},
-	}
-	c := consistent.New(nil, cfg)
-
-	// Add some members to the consistent hash table.
-	// Add function calculates average load and distributes partitions over members
-	node1 := myMember("node1.olric.com")
-	c.Add(node1)
-
-	node2 := myMember("node2.olric.com")
-	c.Add(node2)
-
-	key := []byte("my-key")
-	// calculates partition id for the given key
-	// partID := hash(key) % partitionCount
-	// the partitions are already distributed among members by Add function.
-	owner := c.LocateKey(key)
-	fmt.Println(owner.String())
-	// Prints node2.olric.com
-}
-```
-
-Another useful example is `_examples/relocation_percentage.go`. It creates a `consistent` object with 8 members and distributes partitions among them. Then adds 9th member, 
-here is the result with a proper configuration and hash function:
-
-```
-bloom:consistent burak$ go run _examples/relocation_percentage.go
-partID: 218 moved to node2.olric from node0.olric
-partID: 173 moved to node9.olric from node3.olric
-partID: 225 moved to node7.olric from node0.olric
-partID:  85 moved to node9.olric from node7.olric
-partID: 220 moved to node5.olric from node0.olric
-partID:  33 moved to node9.olric from node5.olric
-partID: 254 moved to node9.olric from node4.olric
-partID:  71 moved to node9.olric from node3.olric
-partID: 236 moved to node9.olric from node2.olric
-partID: 118 moved to node9.olric from node3.olric
-partID: 233 moved to node3.olric from node0.olric
-partID:  50 moved to node9.olric from node4.olric
-partID: 252 moved to node9.olric from node2.olric
-partID: 121 moved to node9.olric from node2.olric
-partID: 259 moved to node9.olric from node4.olric
-partID:  92 moved to node9.olric from node7.olric
-partID: 152 moved to node9.olric from node3.olric
-partID: 105 moved to node9.olric from node2.olric
-
-6% of the partitions are relocated
-```
-
-Moved partition count is highly dependent on your configuration and quailty of hash function. You should modify the configuration to find an optimum set of configurations
-for your system.
-
-`_examples/load_distribution.go` is also useful to understand load distribution. It creates a `consistent` object with 8 members and locates 1M key. It also calculates average 
-load which cannot be exceeded by any member. Here is the result:
-
-```
-Maximum key count for a member should be around this:  147602
-member: node2.olric, key count: 100362
-member: node5.olric, key count: 99448
-member: node0.olric, key count: 147735
-member: node3.olric, key count: 103455
-member: node6.olric, key count: 147069
-member: node1.olric, key count: 121566
-member: node4.olric, key count: 147932
-member: node7.olric, key count: 132433
-```
-
-Average load can be calculated by using the following formula:
-
-```
-load := (consistent.AverageLoad() * float64(keyCount)) / float64(config.PartitionCount)
-```
-
-Contributions
--------------
-Please don't hesitate to fork the project and send a pull request or just e-mail me to ask questions and share ideas.
-
-License
--------
-MIT License, - see LICENSE for more details.
diff --git a/vendor/github.com/buraksezer/consistent/consistent.go b/vendor/github.com/buraksezer/consistent/consistent.go
deleted file mode 100644
index a1446d6..0000000
--- a/vendor/github.com/buraksezer/consistent/consistent.go
+++ /dev/null
@@ -1,362 +0,0 @@
-// Copyright (c) 2018 Burak Sezer
-// All rights reserved.
-//
-// This code is licensed under the MIT License.
-//
-// 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.
-
-// Package consistent provides a consistent hashing function with bounded loads.
-// For more information about the underlying algorithm, please take a look at
-// https://research.googleblog.com/2017/04/consistent-hashing-with-bounded-loads.html
-//
-// Example Use:
-// 	cfg := consistent.Config{
-// 		PartitionCount:    71,
-// 		ReplicationFactor: 20,
-// 		Load:              1.25,
-// 		Hasher:            hasher{},
-//	}
-//
-//      // Create a new consistent object
-//      // You may call this with a list of members
-//      // instead of adding them one by one.
-//	c := consistent.New(members, cfg)
-//
-//      // myMember struct just needs to implement a String method.
-//      // New/Add/Remove distributes partitions among members using the algorithm
-//      // defined on Google Research Blog.
-//	c.Add(myMember)
-//
-//	key := []byte("my-key")
-//      // LocateKey hashes the key and calculates partition ID with
-//      // this modulo operation: MOD(hash result, partition count)
-//      // The owner of the partition is already calculated by New/Add/Remove.
-//      // LocateKey just returns the member which's responsible for the key.
-//	member := c.LocateKey(key)
-//
-package consistent
-
-import (
-	"encoding/binary"
-	"errors"
-	"fmt"
-	"math"
-	"sort"
-	"sync"
-)
-
-var (
-	//ErrInsufficientMemberCount represents an error which means there are not enough members to complete the task.
-	ErrInsufficientMemberCount = errors.New("insufficient member count")
-
-	// ErrMemberNotFound represents an error which means requested member could not be found in consistent hash ring.
-	ErrMemberNotFound = errors.New("member could not be found in ring")
-)
-
-// Hasher is responsible for generating unsigned, 64 bit hash of provided byte slice.
-// Hasher should minimize collisions (generating same hash for different byte slice)
-// and while performance is also important fast functions are preferable (i.e.
-// you can use FarmHash family).
-type Hasher interface {
-	Sum64([]byte) uint64
-}
-
-// Member interface represents a member in consistent hash ring.
-type Member interface {
-	String() string
-}
-
-// Config represents a structure to control consistent package.
-type Config struct {
-	// Hasher is responsible for generating unsigned, 64 bit hash of provided byte slice.
-	Hasher Hasher
-
-	// Keys are distributed among partitions. Prime numbers are good to
-	// distribute keys uniformly. Select a big PartitionCount if you have
-	// too many keys.
-	PartitionCount int
-
-	// Members are replicated on consistent hash ring. This number means that a member
-	// how many times replicated on the ring.
-	ReplicationFactor int
-
-	// Load is used to calculate average load. See the code, the paper and Google's blog post to learn about it.
-	Load float64
-}
-
-// Consistent holds the information about the members of the consistent hash circle.
-type Consistent struct {
-	mu sync.RWMutex
-
-	config         Config
-	hasher         Hasher
-	sortedSet      []uint64
-	partitionCount uint64
-	loads          map[string]float64
-	members        map[string]*Member
-	partitions     map[int]*Member
-	ring           map[uint64]*Member
-}
-
-// New creates and returns a new Consistent object.
-func New(members []Member, config Config) *Consistent {
-	c := &Consistent{
-		config:         config,
-		members:        make(map[string]*Member),
-		partitionCount: uint64(config.PartitionCount),
-		ring:           make(map[uint64]*Member),
-	}
-	if config.Hasher == nil {
-		panic("Hasher cannot be nil")
-	}
-	// TODO: Check configuration here
-	c.hasher = config.Hasher
-	for _, member := range members {
-		c.add(member)
-	}
-	if members != nil {
-		c.distributePartitions()
-	}
-	return c
-}
-
-// GetMembers returns a thread-safe copy of members.
-func (c *Consistent) GetMembers() []Member {
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-
-	// Create a thread-safe copy of member list.
-	members := make([]Member, 0, len(c.members))
-	for _, member := range c.members {
-		members = append(members, *member)
-	}
-	return members
-}
-
-// AverageLoad exposes the current average load.
-func (c *Consistent) AverageLoad() float64 {
-	avgLoad := float64(c.partitionCount/uint64(len(c.members))) * c.config.Load
-	return math.Ceil(avgLoad)
-}
-
-func (c *Consistent) distributeWithLoad(partID, idx int, partitions map[int]*Member, loads map[string]float64) {
-	avgLoad := c.AverageLoad()
-	var count int
-	for {
-		count++
-		if count >= len(c.sortedSet) {
-			// User needs to decrease partition count, increase member count or increase load factor.
-			panic("not enough room to distribute partitions")
-		}
-		i := c.sortedSet[idx]
-		member := *c.ring[i]
-		load := loads[member.String()]
-		if load+1 <= avgLoad {
-			partitions[partID] = &member
-			loads[member.String()]++
-			return
-		}
-		idx++
-		if idx >= len(c.sortedSet) {
-			idx = 0
-		}
-	}
-}
-
-func (c *Consistent) distributePartitions() {
-	loads := make(map[string]float64)
-	partitions := make(map[int]*Member)
-
-	bs := make([]byte, 8)
-	for partID := uint64(0); partID < c.partitionCount; partID++ {
-		binary.LittleEndian.PutUint64(bs, partID)
-		key := c.hasher.Sum64(bs)
-		idx := sort.Search(len(c.sortedSet), func(i int) bool {
-			return c.sortedSet[i] >= key
-		})
-		if idx >= len(c.sortedSet) {
-			idx = 0
-		}
-		c.distributeWithLoad(int(partID), idx, partitions, loads)
-	}
-	c.partitions = partitions
-	c.loads = loads
-}
-
-func (c *Consistent) add(member Member) {
-	for i := 0; i < c.config.ReplicationFactor; i++ {
-		key := []byte(fmt.Sprintf("%s%d", member.String(), i))
-		h := c.hasher.Sum64(key)
-		c.ring[h] = &member
-		c.sortedSet = append(c.sortedSet, h)
-	}
-	// sort hashes ascendingly
-	sort.Slice(c.sortedSet, func(i int, j int) bool {
-		return c.sortedSet[i] < c.sortedSet[j]
-	})
-	// Storing member at this map is useful to find backup members of a partition.
-	c.members[member.String()] = &member
-}
-
-// Add adds a new member to the consistent hash circle.
-func (c *Consistent) Add(member Member) {
-	c.mu.Lock()
-	defer c.mu.Unlock()
-
-	if _, ok := c.members[member.String()]; ok {
-		// We already have this member. Quit immediately.
-		return
-	}
-	c.add(member)
-	c.distributePartitions()
-}
-
-func (c *Consistent) delSlice(val uint64) {
-	for i := 0; i < len(c.sortedSet); i++ {
-		if c.sortedSet[i] == val {
-			c.sortedSet = append(c.sortedSet[:i], c.sortedSet[i+1:]...)
-			break
-		}
-	}
-}
-
-// Remove removes a member from the consistent hash circle.
-func (c *Consistent) Remove(name string) {
-	c.mu.Lock()
-	defer c.mu.Unlock()
-
-	if _, ok := c.members[name]; !ok {
-		// There is no member with that name. Quit immediately.
-		return
-	}
-
-	for i := 0; i < c.config.ReplicationFactor; i++ {
-		key := []byte(fmt.Sprintf("%s%d", name, i))
-		h := c.hasher.Sum64(key)
-		delete(c.ring, h)
-		c.delSlice(h)
-	}
-	delete(c.members, name)
-	if len(c.members) == 0 {
-		// consistent hash ring is empty now. Reset the partition table.
-		c.partitions = make(map[int]*Member)
-		return
-	}
-	c.distributePartitions()
-}
-
-// LoadDistribution exposes load distribution of members.
-func (c *Consistent) LoadDistribution() map[string]float64 {
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-
-	// Create a thread-safe copy
-	res := make(map[string]float64)
-	for member, load := range c.loads {
-		res[member] = load
-	}
-	return res
-}
-
-// FindPartitionID returns partition id for given key.
-func (c *Consistent) FindPartitionID(key []byte) int {
-	hkey := c.hasher.Sum64(key)
-	return int(hkey % c.partitionCount)
-}
-
-// GetPartitionOwner returns the owner of the given partition.
-func (c *Consistent) GetPartitionOwner(partID int) Member {
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-
-	member, ok := c.partitions[partID]
-	if !ok {
-		return nil
-	}
-	// Create a thread-safe copy of member and return it.
-	return *member
-}
-
-// LocateKey finds a home for given key
-func (c *Consistent) LocateKey(key []byte) Member {
-	partID := c.FindPartitionID(key)
-	return c.GetPartitionOwner(partID)
-}
-
-func (c *Consistent) getClosestN(partID, count int) ([]Member, error) {
-	c.mu.RLock()
-	defer c.mu.RUnlock()
-
-	res := []Member{}
-	if count > len(c.members) {
-		return res, ErrInsufficientMemberCount
-	}
-
-	var ownerKey uint64
-	owner := c.GetPartitionOwner(partID)
-	// Hash and sort all the names.
-	keys := []uint64{}
-	kmems := make(map[uint64]*Member)
-	for name, member := range c.members {
-		key := c.hasher.Sum64([]byte(name))
-		if name == owner.String() {
-			ownerKey = key
-		}
-		keys = append(keys, key)
-		kmems[key] = member
-	}
-	sort.Slice(keys, func(i, j int) bool {
-		return keys[i] < keys[j]
-	})
-
-	// Find the key owner
-	idx := 0
-	for idx < len(keys) {
-		if keys[idx] == ownerKey {
-			key := keys[idx]
-			res = append(res, *kmems[key])
-			break
-		}
-		idx++
-	}
-
-	// Find the closest(replica owners) members.
-	for len(res) < count {
-		idx++
-		if idx >= len(keys) {
-			idx = 0
-		}
-		key := keys[idx]
-		res = append(res, *kmems[key])
-	}
-	return res, nil
-}
-
-// GetClosestN returns the closest N member to a key in the hash ring.
-// This may be useful to find members for replication.
-func (c *Consistent) GetClosestN(key []byte, count int) ([]Member, error) {
-	partID := c.FindPartitionID(key)
-	return c.getClosestN(partID, count)
-}
-
-// GetClosestNForPartition returns the closest N member for given partition.
-// This may be useful to find members for replication.
-func (c *Consistent) GetClosestNForPartition(partID, count int) ([]Member, error) {
-	return c.getClosestN(partID, count)
-}
diff --git a/vendor/github.com/cespare/xxhash/README.md b/vendor/github.com/cespare/xxhash/README.md
deleted file mode 100644
index 0982fd2..0000000
--- a/vendor/github.com/cespare/xxhash/README.md
+++ /dev/null
@@ -1,50 +0,0 @@
-# xxhash
-
-[![GoDoc](https://godoc.org/github.com/cespare/xxhash?status.svg)](https://godoc.org/github.com/cespare/xxhash)
-
-xxhash is a Go implementation of the 64-bit
-[xxHash](http://cyan4973.github.io/xxHash/) algorithm, XXH64. This is a
-high-quality hashing algorithm that is much faster than anything in the Go
-standard library.
-
-The API is very small, taking its cue from the other hashing packages in the
-standard library:
-
-    $ go doc github.com/cespare/xxhash                                                                                                                                                                                              !
-    package xxhash // import "github.com/cespare/xxhash"
-
-    Package xxhash implements the 64-bit variant of xxHash (XXH64) as described
-    at http://cyan4973.github.io/xxHash/.
-
-    func New() hash.Hash64
-    func Sum64(b []byte) uint64
-    func Sum64String(s string) uint64
-
-This implementation provides a fast pure-Go implementation and an even faster
-assembly implementation for amd64.
-
-## Benchmarks
-
-Here are some quick benchmarks comparing the pure-Go and assembly
-implementations of Sum64 against another popular Go XXH64 implementation,
-[github.com/OneOfOne/xxhash](https://github.com/OneOfOne/xxhash):
-
-| input size | OneOfOne | cespare (purego) | cespare |
-| --- | --- | --- | --- |
-| 5 B   |  416 MB/s | 720 MB/s |  872 MB/s  |
-| 100 B | 3980 MB/s | 5013 MB/s | 5252 MB/s  |
-| 4 KB  | 12727 MB/s | 12999 MB/s | 13026 MB/s |
-| 10 MB | 9879 MB/s | 10775 MB/s | 10913 MB/s  |
-
-These numbers were generated with:
-
-```
-$ go test -benchtime 10s -bench '/OneOfOne,'
-$ go test -tags purego -benchtime 10s -bench '/xxhash,'
-$ go test -benchtime 10s -bench '/xxhash,'
-```
-
-## Projects using this package
-
-- [InfluxDB](https://github.com/influxdata/influxdb)
-- [Prometheus](https://github.com/prometheus/prometheus)
diff --git a/vendor/github.com/cespare/xxhash/go.mod b/vendor/github.com/cespare/xxhash/go.mod
deleted file mode 100644
index 10605a6..0000000
--- a/vendor/github.com/cespare/xxhash/go.mod
+++ /dev/null
@@ -1,6 +0,0 @@
-module github.com/cespare/xxhash
-
-require (
-	github.com/OneOfOne/xxhash v1.2.2
-	github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72
-)
diff --git a/vendor/github.com/cespare/xxhash/go.sum b/vendor/github.com/cespare/xxhash/go.sum
deleted file mode 100644
index f6b5542..0000000
--- a/vendor/github.com/cespare/xxhash/go.sum
+++ /dev/null
@@ -1,4 +0,0 @@
-github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE=
-github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
diff --git a/vendor/github.com/cespare/xxhash/rotate.go b/vendor/github.com/cespare/xxhash/rotate.go
deleted file mode 100644
index f3eac5e..0000000
--- a/vendor/github.com/cespare/xxhash/rotate.go
+++ /dev/null
@@ -1,14 +0,0 @@
-// +build !go1.9
-
-package xxhash
-
-// TODO(caleb): After Go 1.10 comes out, remove this fallback code.
-
-func rol1(x uint64) uint64  { return (x << 1) | (x >> (64 - 1)) }
-func rol7(x uint64) uint64  { return (x << 7) | (x >> (64 - 7)) }
-func rol11(x uint64) uint64 { return (x << 11) | (x >> (64 - 11)) }
-func rol12(x uint64) uint64 { return (x << 12) | (x >> (64 - 12)) }
-func rol18(x uint64) uint64 { return (x << 18) | (x >> (64 - 18)) }
-func rol23(x uint64) uint64 { return (x << 23) | (x >> (64 - 23)) }
-func rol27(x uint64) uint64 { return (x << 27) | (x >> (64 - 27)) }
-func rol31(x uint64) uint64 { return (x << 31) | (x >> (64 - 31)) }
diff --git a/vendor/github.com/cespare/xxhash/rotate19.go b/vendor/github.com/cespare/xxhash/rotate19.go
deleted file mode 100644
index b99612b..0000000
--- a/vendor/github.com/cespare/xxhash/rotate19.go
+++ /dev/null
@@ -1,14 +0,0 @@
-// +build go1.9
-
-package xxhash
-
-import "math/bits"
-
-func rol1(x uint64) uint64  { return bits.RotateLeft64(x, 1) }
-func rol7(x uint64) uint64  { return bits.RotateLeft64(x, 7) }
-func rol11(x uint64) uint64 { return bits.RotateLeft64(x, 11) }
-func rol12(x uint64) uint64 { return bits.RotateLeft64(x, 12) }
-func rol18(x uint64) uint64 { return bits.RotateLeft64(x, 18) }
-func rol23(x uint64) uint64 { return bits.RotateLeft64(x, 23) }
-func rol27(x uint64) uint64 { return bits.RotateLeft64(x, 27) }
-func rol31(x uint64) uint64 { return bits.RotateLeft64(x, 31) }
diff --git a/vendor/github.com/cespare/xxhash/xxhash.go b/vendor/github.com/cespare/xxhash/xxhash.go
deleted file mode 100644
index f896bd2..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash.go
+++ /dev/null
@@ -1,168 +0,0 @@
-// Package xxhash implements the 64-bit variant of xxHash (XXH64) as described
-// at http://cyan4973.github.io/xxHash/.
-package xxhash
-
-import (
-	"encoding/binary"
-	"hash"
-)
-
-const (
-	prime1 uint64 = 11400714785074694791
-	prime2 uint64 = 14029467366897019727
-	prime3 uint64 = 1609587929392839161
-	prime4 uint64 = 9650029242287828579
-	prime5 uint64 = 2870177450012600261
-)
-
-// NOTE(caleb): I'm using both consts and vars of the primes. Using consts where
-// possible in the Go code is worth a small (but measurable) performance boost
-// by avoiding some MOVQs. Vars are needed for the asm and also are useful for
-// convenience in the Go code in a few places where we need to intentionally
-// avoid constant arithmetic (e.g., v1 := prime1 + prime2 fails because the
-// result overflows a uint64).
-var (
-	prime1v = prime1
-	prime2v = prime2
-	prime3v = prime3
-	prime4v = prime4
-	prime5v = prime5
-)
-
-type xxh struct {
-	v1    uint64
-	v2    uint64
-	v3    uint64
-	v4    uint64
-	total int
-	mem   [32]byte
-	n     int // how much of mem is used
-}
-
-// New creates a new hash.Hash64 that implements the 64-bit xxHash algorithm.
-func New() hash.Hash64 {
-	var x xxh
-	x.Reset()
-	return &x
-}
-
-func (x *xxh) Reset() {
-	x.n = 0
-	x.total = 0
-	x.v1 = prime1v + prime2
-	x.v2 = prime2
-	x.v3 = 0
-	x.v4 = -prime1v
-}
-
-func (x *xxh) Size() int      { return 8 }
-func (x *xxh) BlockSize() int { return 32 }
-
-// Write adds more data to x. It always returns len(b), nil.
-func (x *xxh) Write(b []byte) (n int, err error) {
-	n = len(b)
-	x.total += len(b)
-
-	if x.n+len(b) < 32 {
-		// This new data doesn't even fill the current block.
-		copy(x.mem[x.n:], b)
-		x.n += len(b)
-		return
-	}
-
-	if x.n > 0 {
-		// Finish off the partial block.
-		copy(x.mem[x.n:], b)
-		x.v1 = round(x.v1, u64(x.mem[0:8]))
-		x.v2 = round(x.v2, u64(x.mem[8:16]))
-		x.v3 = round(x.v3, u64(x.mem[16:24]))
-		x.v4 = round(x.v4, u64(x.mem[24:32]))
-		b = b[32-x.n:]
-		x.n = 0
-	}
-
-	if len(b) >= 32 {
-		// One or more full blocks left.
-		b = writeBlocks(x, b)
-	}
-
-	// Store any remaining partial block.
-	copy(x.mem[:], b)
-	x.n = len(b)
-
-	return
-}
-
-func (x *xxh) Sum(b []byte) []byte {
-	s := x.Sum64()
-	return append(
-		b,
-		byte(s>>56),
-		byte(s>>48),
-		byte(s>>40),
-		byte(s>>32),
-		byte(s>>24),
-		byte(s>>16),
-		byte(s>>8),
-		byte(s),
-	)
-}
-
-func (x *xxh) Sum64() uint64 {
-	var h uint64
-
-	if x.total >= 32 {
-		v1, v2, v3, v4 := x.v1, x.v2, x.v3, x.v4
-		h = rol1(v1) + rol7(v2) + rol12(v3) + rol18(v4)
-		h = mergeRound(h, v1)
-		h = mergeRound(h, v2)
-		h = mergeRound(h, v3)
-		h = mergeRound(h, v4)
-	} else {
-		h = x.v3 + prime5
-	}
-
-	h += uint64(x.total)
-
-	i, end := 0, x.n
-	for ; i+8 <= end; i += 8 {
-		k1 := round(0, u64(x.mem[i:i+8]))
-		h ^= k1
-		h = rol27(h)*prime1 + prime4
-	}
-	if i+4 <= end {
-		h ^= uint64(u32(x.mem[i:i+4])) * prime1
-		h = rol23(h)*prime2 + prime3
-		i += 4
-	}
-	for i < end {
-		h ^= uint64(x.mem[i]) * prime5
-		h = rol11(h) * prime1
-		i++
-	}
-
-	h ^= h >> 33
-	h *= prime2
-	h ^= h >> 29
-	h *= prime3
-	h ^= h >> 32
-
-	return h
-}
-
-func u64(b []byte) uint64 { return binary.LittleEndian.Uint64(b) }
-func u32(b []byte) uint32 { return binary.LittleEndian.Uint32(b) }
-
-func round(acc, input uint64) uint64 {
-	acc += input * prime2
-	acc = rol31(acc)
-	acc *= prime1
-	return acc
-}
-
-func mergeRound(acc, val uint64) uint64 {
-	val = round(0, val)
-	acc ^= val
-	acc = acc*prime1 + prime4
-	return acc
-}
diff --git a/vendor/github.com/cespare/xxhash/xxhash_amd64.go b/vendor/github.com/cespare/xxhash/xxhash_amd64.go
deleted file mode 100644
index d617652..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash_amd64.go
+++ /dev/null
@@ -1,12 +0,0 @@
-// +build !appengine
-// +build gc
-// +build !purego
-
-package xxhash
-
-// Sum64 computes the 64-bit xxHash digest of b.
-//
-//go:noescape
-func Sum64(b []byte) uint64
-
-func writeBlocks(x *xxh, b []byte) []byte
diff --git a/vendor/github.com/cespare/xxhash/xxhash_amd64.s b/vendor/github.com/cespare/xxhash/xxhash_amd64.s
deleted file mode 100644
index 757f201..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash_amd64.s
+++ /dev/null
@@ -1,233 +0,0 @@
-// +build !appengine
-// +build gc
-// +build !purego
-
-#include "textflag.h"
-
-// Register allocation:
-// AX	h
-// CX	pointer to advance through b
-// DX	n
-// BX	loop end
-// R8	v1, k1
-// R9	v2
-// R10	v3
-// R11	v4
-// R12	tmp
-// R13	prime1v
-// R14	prime2v
-// R15	prime4v
-
-// round reads from and advances the buffer pointer in CX.
-// It assumes that R13 has prime1v and R14 has prime2v.
-#define round(r) \
-	MOVQ  (CX), R12 \
-	ADDQ  $8, CX    \
-	IMULQ R14, R12  \
-	ADDQ  R12, r    \
-	ROLQ  $31, r    \
-	IMULQ R13, r
-
-// mergeRound applies a merge round on the two registers acc and val.
-// It assumes that R13 has prime1v, R14 has prime2v, and R15 has prime4v.
-#define mergeRound(acc, val) \
-	IMULQ R14, val \
-	ROLQ  $31, val \
-	IMULQ R13, val \
-	XORQ  val, acc \
-	IMULQ R13, acc \
-	ADDQ  R15, acc
-
-// func Sum64(b []byte) uint64
-TEXT ·Sum64(SB), NOSPLIT, $0-32
-	// Load fixed primes.
-	MOVQ ·prime1v(SB), R13
-	MOVQ ·prime2v(SB), R14
-	MOVQ ·prime4v(SB), R15
-
-	// Load slice.
-	MOVQ b_base+0(FP), CX
-	MOVQ b_len+8(FP), DX
-	LEAQ (CX)(DX*1), BX
-
-	// The first loop limit will be len(b)-32.
-	SUBQ $32, BX
-
-	// Check whether we have at least one block.
-	CMPQ DX, $32
-	JLT  noBlocks
-
-	// Set up initial state (v1, v2, v3, v4).
-	MOVQ R13, R8
-	ADDQ R14, R8
-	MOVQ R14, R9
-	XORQ R10, R10
-	XORQ R11, R11
-	SUBQ R13, R11
-
-	// Loop until CX > BX.
-blockLoop:
-	round(R8)
-	round(R9)
-	round(R10)
-	round(R11)
-
-	CMPQ CX, BX
-	JLE  blockLoop
-
-	MOVQ R8, AX
-	ROLQ $1, AX
-	MOVQ R9, R12
-	ROLQ $7, R12
-	ADDQ R12, AX
-	MOVQ R10, R12
-	ROLQ $12, R12
-	ADDQ R12, AX
-	MOVQ R11, R12
-	ROLQ $18, R12
-	ADDQ R12, AX
-
-	mergeRound(AX, R8)
-	mergeRound(AX, R9)
-	mergeRound(AX, R10)
-	mergeRound(AX, R11)
-
-	JMP afterBlocks
-
-noBlocks:
-	MOVQ ·prime5v(SB), AX
-
-afterBlocks:
-	ADDQ DX, AX
-
-	// Right now BX has len(b)-32, and we want to loop until CX > len(b)-8.
-	ADDQ $24, BX
-
-	CMPQ CX, BX
-	JG   fourByte
-
-wordLoop:
-	// Calculate k1.
-	MOVQ  (CX), R8
-	ADDQ  $8, CX
-	IMULQ R14, R8
-	ROLQ  $31, R8
-	IMULQ R13, R8
-
-	XORQ  R8, AX
-	ROLQ  $27, AX
-	IMULQ R13, AX
-	ADDQ  R15, AX
-
-	CMPQ CX, BX
-	JLE  wordLoop
-
-fourByte:
-	ADDQ $4, BX
-	CMPQ CX, BX
-	JG   singles
-
-	MOVL  (CX), R8
-	ADDQ  $4, CX
-	IMULQ R13, R8
-	XORQ  R8, AX
-
-	ROLQ  $23, AX
-	IMULQ R14, AX
-	ADDQ  ·prime3v(SB), AX
-
-singles:
-	ADDQ $4, BX
-	CMPQ CX, BX
-	JGE  finalize
-
-singlesLoop:
-	MOVBQZX (CX), R12
-	ADDQ    $1, CX
-	IMULQ   ·prime5v(SB), R12
-	XORQ    R12, AX
-
-	ROLQ  $11, AX
-	IMULQ R13, AX
-
-	CMPQ CX, BX
-	JL   singlesLoop
-
-finalize:
-	MOVQ  AX, R12
-	SHRQ  $33, R12
-	XORQ  R12, AX
-	IMULQ R14, AX
-	MOVQ  AX, R12
-	SHRQ  $29, R12
-	XORQ  R12, AX
-	IMULQ ·prime3v(SB), AX
-	MOVQ  AX, R12
-	SHRQ  $32, R12
-	XORQ  R12, AX
-
-	MOVQ AX, ret+24(FP)
-	RET
-
-// writeBlocks uses the same registers as above except that it uses AX to store
-// the x pointer.
-
-// func writeBlocks(x *xxh, b []byte) []byte
-TEXT ·writeBlocks(SB), NOSPLIT, $0-56
-	// Load fixed primes needed for round.
-	MOVQ ·prime1v(SB), R13
-	MOVQ ·prime2v(SB), R14
-
-	// Load slice.
-	MOVQ b_base+8(FP), CX
-	MOVQ CX, ret_base+32(FP) // initialize return base pointer; see NOTE below
-	MOVQ b_len+16(FP), DX
-	LEAQ (CX)(DX*1), BX
-	SUBQ $32, BX
-
-	// Load vN from x.
-	MOVQ x+0(FP), AX
-	MOVQ 0(AX), R8   // v1
-	MOVQ 8(AX), R9   // v2
-	MOVQ 16(AX), R10 // v3
-	MOVQ 24(AX), R11 // v4
-
-	// We don't need to check the loop condition here; this function is
-	// always called with at least one block of data to process.
-blockLoop:
-	round(R8)
-	round(R9)
-	round(R10)
-	round(R11)
-
-	CMPQ CX, BX
-	JLE  blockLoop
-
-	// Copy vN back to x.
-	MOVQ R8, 0(AX)
-	MOVQ R9, 8(AX)
-	MOVQ R10, 16(AX)
-	MOVQ R11, 24(AX)
-
-	// Construct return slice.
-	// NOTE: It's important that we don't construct a slice that has a base
-	// pointer off the end of the original slice, as in Go 1.7+ this will
-	// cause runtime crashes. (See discussion in, for example,
-	// https://github.com/golang/go/issues/16772.)
-	// Therefore, we calculate the length/cap first, and if they're zero, we
-	// keep the old base. This is what the compiler does as well if you
-	// write code like
-	//   b = b[len(b):]
-
-	// New length is 32 - (CX - BX) -> BX+32 - CX.
-	ADDQ $32, BX
-	SUBQ CX, BX
-	JZ   afterSetBase
-
-	MOVQ CX, ret_base+32(FP)
-
-afterSetBase:
-	MOVQ BX, ret_len+40(FP)
-	MOVQ BX, ret_cap+48(FP) // set cap == len
-
-	RET
diff --git a/vendor/github.com/cespare/xxhash/xxhash_other.go b/vendor/github.com/cespare/xxhash/xxhash_other.go
deleted file mode 100644
index c68d13f..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash_other.go
+++ /dev/null
@@ -1,75 +0,0 @@
-// +build !amd64 appengine !gc purego
-
-package xxhash
-
-// Sum64 computes the 64-bit xxHash digest of b.
-func Sum64(b []byte) uint64 {
-	// A simpler version would be
-	//   x := New()
-	//   x.Write(b)
-	//   return x.Sum64()
-	// but this is faster, particularly for small inputs.
-
-	n := len(b)
-	var h uint64
-
-	if n >= 32 {
-		v1 := prime1v + prime2
-		v2 := prime2
-		v3 := uint64(0)
-		v4 := -prime1v
-		for len(b) >= 32 {
-			v1 = round(v1, u64(b[0:8:len(b)]))
-			v2 = round(v2, u64(b[8:16:len(b)]))
-			v3 = round(v3, u64(b[16:24:len(b)]))
-			v4 = round(v4, u64(b[24:32:len(b)]))
-			b = b[32:len(b):len(b)]
-		}
-		h = rol1(v1) + rol7(v2) + rol12(v3) + rol18(v4)
-		h = mergeRound(h, v1)
-		h = mergeRound(h, v2)
-		h = mergeRound(h, v3)
-		h = mergeRound(h, v4)
-	} else {
-		h = prime5
-	}
-
-	h += uint64(n)
-
-	i, end := 0, len(b)
-	for ; i+8 <= end; i += 8 {
-		k1 := round(0, u64(b[i:i+8:len(b)]))
-		h ^= k1
-		h = rol27(h)*prime1 + prime4
-	}
-	if i+4 <= end {
-		h ^= uint64(u32(b[i:i+4:len(b)])) * prime1
-		h = rol23(h)*prime2 + prime3
-		i += 4
-	}
-	for ; i < end; i++ {
-		h ^= uint64(b[i]) * prime5
-		h = rol11(h) * prime1
-	}
-
-	h ^= h >> 33
-	h *= prime2
-	h ^= h >> 29
-	h *= prime3
-	h ^= h >> 32
-
-	return h
-}
-
-func writeBlocks(x *xxh, b []byte) []byte {
-	v1, v2, v3, v4 := x.v1, x.v2, x.v3, x.v4
-	for len(b) >= 32 {
-		v1 = round(v1, u64(b[0:8:len(b)]))
-		v2 = round(v2, u64(b[8:16:len(b)]))
-		v3 = round(v3, u64(b[16:24:len(b)]))
-		v4 = round(v4, u64(b[24:32:len(b)]))
-		b = b[32:len(b):len(b)]
-	}
-	x.v1, x.v2, x.v3, x.v4 = v1, v2, v3, v4
-	return b
-}
diff --git a/vendor/github.com/cespare/xxhash/xxhash_safe.go b/vendor/github.com/cespare/xxhash/xxhash_safe.go
deleted file mode 100644
index dfa15ab..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash_safe.go
+++ /dev/null
@@ -1,10 +0,0 @@
-// +build appengine
-
-// This file contains the safe implementations of otherwise unsafe-using code.
-
-package xxhash
-
-// Sum64String computes the 64-bit xxHash digest of s.
-func Sum64String(s string) uint64 {
-	return Sum64([]byte(s))
-}
diff --git a/vendor/github.com/cespare/xxhash/xxhash_unsafe.go b/vendor/github.com/cespare/xxhash/xxhash_unsafe.go
deleted file mode 100644
index d2b64e8..0000000
--- a/vendor/github.com/cespare/xxhash/xxhash_unsafe.go
+++ /dev/null
@@ -1,30 +0,0 @@
-// +build !appengine
-
-// This file encapsulates usage of unsafe.
-// xxhash_safe.go contains the safe implementations.
-
-package xxhash
-
-import (
-	"reflect"
-	"unsafe"
-)
-
-// Sum64String computes the 64-bit xxHash digest of s.
-// It may be faster than Sum64([]byte(s)) by avoiding a copy.
-//
-// TODO(caleb): Consider removing this if an optimization is ever added to make
-// it unnecessary: https://golang.org/issue/2205.
-//
-// TODO(caleb): We still have a function call; we could instead write Go/asm
-// copies of Sum64 for strings to squeeze out a bit more speed.
-func Sum64String(s string) uint64 {
-	// See https://groups.google.com/d/msg/golang-nuts/dcjzJy-bSpw/tcZYBzQqAQAJ
-	// for some discussion about this unsafe conversion.
-	var b []byte
-	bh := (*reflect.SliceHeader)(unsafe.Pointer(&b))
-	bh.Data = (*reflect.StringHeader)(unsafe.Pointer(&s)).Data
-	bh.Len = len(s)
-	bh.Cap = len(s)
-	return Sum64(b)
-}
diff --git a/vendor/github.com/coreos/etcd/LICENSE b/vendor/github.com/coreos/etcd/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/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/etcd/NOTICE b/vendor/github.com/coreos/etcd/NOTICE
new file mode 100644
index 0000000..b39ddfa
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/NOTICE
@@ -0,0 +1,5 @@
+CoreOS Project
+Copyright 2014 CoreOS, Inc
+
+This product includes software developed at CoreOS, Inc.
+(http://www.coreos.com/).
diff --git a/vendor/github.com/coreos/etcd/auth/authpb/auth.pb.go b/vendor/github.com/coreos/etcd/auth/authpb/auth.pb.go
new file mode 100644
index 0000000..c5faf00
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/auth/authpb/auth.pb.go
@@ -0,0 +1,972 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: auth.proto
+
+package authpb
+
+import (
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+	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.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Permission_Type int32
+
+const (
+	READ      Permission_Type = 0
+	WRITE     Permission_Type = 1
+	READWRITE Permission_Type = 2
+)
+
+var Permission_Type_name = map[int32]string{
+	0: "READ",
+	1: "WRITE",
+	2: "READWRITE",
+}
+
+var Permission_Type_value = map[string]int32{
+	"READ":      0,
+	"WRITE":     1,
+	"READWRITE": 2,
+}
+
+func (x Permission_Type) String() string {
+	return proto.EnumName(Permission_Type_name, int32(x))
+}
+
+func (Permission_Type) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_8bbd6f3875b0e874, []int{1, 0}
+}
+
+// User is a single entry in the bucket authUsers
+type User struct {
+	Name                 []byte   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Password             []byte   `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
+	Roles                []string `protobuf:"bytes,3,rep,name=roles,proto3" json:"roles,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *User) Reset()         { *m = User{} }
+func (m *User) String() string { return proto.CompactTextString(m) }
+func (*User) ProtoMessage()    {}
+func (*User) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8bbd6f3875b0e874, []int{0}
+}
+func (m *User) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *User) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_User.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *User) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_User.Merge(m, src)
+}
+func (m *User) XXX_Size() int {
+	return m.Size()
+}
+func (m *User) XXX_DiscardUnknown() {
+	xxx_messageInfo_User.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_User proto.InternalMessageInfo
+
+// Permission is a single entity
+type Permission struct {
+	PermType             Permission_Type `protobuf:"varint,1,opt,name=permType,proto3,enum=authpb.Permission_Type" json:"permType,omitempty"`
+	Key                  []byte          `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
+	RangeEnd             []byte          `protobuf:"bytes,3,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *Permission) Reset()         { *m = Permission{} }
+func (m *Permission) String() string { return proto.CompactTextString(m) }
+func (*Permission) ProtoMessage()    {}
+func (*Permission) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8bbd6f3875b0e874, []int{1}
+}
+func (m *Permission) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Permission) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Permission.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Permission) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Permission.Merge(m, src)
+}
+func (m *Permission) XXX_Size() int {
+	return m.Size()
+}
+func (m *Permission) XXX_DiscardUnknown() {
+	xxx_messageInfo_Permission.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Permission proto.InternalMessageInfo
+
+// Role is a single entry in the bucket authRoles
+type Role struct {
+	Name                 []byte        `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	KeyPermission        []*Permission `protobuf:"bytes,2,rep,name=keyPermission,proto3" json:"keyPermission,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *Role) Reset()         { *m = Role{} }
+func (m *Role) String() string { return proto.CompactTextString(m) }
+func (*Role) ProtoMessage()    {}
+func (*Role) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8bbd6f3875b0e874, []int{2}
+}
+func (m *Role) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Role) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Role.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Role) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Role.Merge(m, src)
+}
+func (m *Role) XXX_Size() int {
+	return m.Size()
+}
+func (m *Role) XXX_DiscardUnknown() {
+	xxx_messageInfo_Role.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Role proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterEnum("authpb.Permission_Type", Permission_Type_name, Permission_Type_value)
+	proto.RegisterType((*User)(nil), "authpb.User")
+	proto.RegisterType((*Permission)(nil), "authpb.Permission")
+	proto.RegisterType((*Role)(nil), "authpb.Role")
+}
+
+func init() { proto.RegisterFile("auth.proto", fileDescriptor_8bbd6f3875b0e874) }
+
+var fileDescriptor_8bbd6f3875b0e874 = []byte{
+	// 288 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0xc1, 0x4a, 0xc3, 0x30,
+	0x1c, 0xc6, 0x9b, 0xb6, 0x1b, 0xed, 0x5f, 0x27, 0x25, 0x0c, 0x0c, 0x13, 0x42, 0xe9, 0xa9, 0x78,
+	0xa8, 0xb0, 0x5d, 0xbc, 0x2a, 0xf6, 0x20, 0x78, 0x90, 0x50, 0xf1, 0x28, 0x1d, 0x0d, 0x75, 0x6c,
+	0x6d, 0x4a, 0x32, 0x91, 0xbe, 0x89, 0x07, 0x1f, 0x68, 0xc7, 0x3d, 0x82, 0xab, 0x2f, 0x22, 0x4d,
+	0x64, 0x43, 0xdc, 0xed, 0xfb, 0xbe, 0xff, 0x97, 0xe4, 0x97, 0x3f, 0x40, 0xfe, 0xb6, 0x7e, 0x4d,
+	0x1a, 0x29, 0xd6, 0x02, 0x0f, 0x7b, 0xdd, 0xcc, 0x27, 0xe3, 0x52, 0x94, 0x42, 0x47, 0x57, 0xbd,
+	0x32, 0xd3, 0xe8, 0x01, 0xdc, 0x27, 0xc5, 0x25, 0xc6, 0xe0, 0xd6, 0x79, 0xc5, 0x09, 0x0a, 0x51,
+	0x7c, 0xca, 0xb4, 0xc6, 0x13, 0xf0, 0x9a, 0x5c, 0xa9, 0x77, 0x21, 0x0b, 0x62, 0xeb, 0x7c, 0xef,
+	0xf1, 0x18, 0x06, 0x52, 0xac, 0xb8, 0x22, 0x4e, 0xe8, 0xc4, 0x3e, 0x33, 0x26, 0xfa, 0x44, 0x00,
+	0x8f, 0x5c, 0x56, 0x0b, 0xa5, 0x16, 0xa2, 0xc6, 0x33, 0xf0, 0x1a, 0x2e, 0xab, 0xac, 0x6d, 0xcc,
+	0xc5, 0x67, 0xd3, 0xf3, 0xc4, 0xd0, 0x24, 0x87, 0x56, 0xd2, 0x8f, 0xd9, 0xbe, 0x88, 0x03, 0x70,
+	0x96, 0xbc, 0xfd, 0x7d, 0xb0, 0x97, 0xf8, 0x02, 0x7c, 0x99, 0xd7, 0x25, 0x7f, 0xe1, 0x75, 0x41,
+	0x1c, 0x03, 0xa2, 0x83, 0xb4, 0x2e, 0xa2, 0x4b, 0x70, 0xf5, 0x31, 0x0f, 0x5c, 0x96, 0xde, 0xdc,
+	0x05, 0x16, 0xf6, 0x61, 0xf0, 0xcc, 0xee, 0xb3, 0x34, 0x40, 0x78, 0x04, 0x7e, 0x1f, 0x1a, 0x6b,
+	0x47, 0x19, 0xb8, 0x4c, 0xac, 0xf8, 0xd1, 0xcf, 0x5e, 0xc3, 0x68, 0xc9, 0xdb, 0x03, 0x16, 0xb1,
+	0x43, 0x27, 0x3e, 0x99, 0xe2, 0xff, 0xc0, 0xec, 0x6f, 0xf1, 0x96, 0x6c, 0x76, 0xd4, 0xda, 0xee,
+	0xa8, 0xb5, 0xe9, 0x28, 0xda, 0x76, 0x14, 0x7d, 0x75, 0x14, 0x7d, 0x7c, 0x53, 0x6b, 0x3e, 0xd4,
+	0x3b, 0x9e, 0xfd, 0x04, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x76, 0x8d, 0x4f, 0x8f, 0x01, 0x00, 0x00,
+}
+
+func (m *User) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *User) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *User) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Roles) > 0 {
+		for iNdEx := len(m.Roles) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Roles[iNdEx])
+			copy(dAtA[i:], m.Roles[iNdEx])
+			i = encodeVarintAuth(dAtA, i, uint64(len(m.Roles[iNdEx])))
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if len(m.Password) > 0 {
+		i -= len(m.Password)
+		copy(dAtA[i:], m.Password)
+		i = encodeVarintAuth(dAtA, i, uint64(len(m.Password)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintAuth(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Permission) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Permission) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Permission) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintAuth(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintAuth(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.PermType != 0 {
+		i = encodeVarintAuth(dAtA, i, uint64(m.PermType))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Role) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Role) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Role) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.KeyPermission) > 0 {
+		for iNdEx := len(m.KeyPermission) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.KeyPermission[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintAuth(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintAuth(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintAuth(dAtA []byte, offset int, v uint64) int {
+	offset -= sovAuth(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *User) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovAuth(uint64(l))
+	}
+	l = len(m.Password)
+	if l > 0 {
+		n += 1 + l + sovAuth(uint64(l))
+	}
+	if len(m.Roles) > 0 {
+		for _, s := range m.Roles {
+			l = len(s)
+			n += 1 + l + sovAuth(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Permission) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.PermType != 0 {
+		n += 1 + sovAuth(uint64(m.PermType))
+	}
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovAuth(uint64(l))
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 1 + l + sovAuth(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Role) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovAuth(uint64(l))
+	}
+	if len(m.KeyPermission) > 0 {
+		for _, e := range m.KeyPermission {
+			l = e.Size()
+			n += 1 + l + sovAuth(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovAuth(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozAuth(x uint64) (n int) {
+	return sovAuth(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *User) 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 ErrIntOverflowAuth
+			}
+			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: User: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: User: 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 ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 Password", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Password = append(m.Password[:0], dAtA[iNdEx:postIndex]...)
+			if m.Password == nil {
+				m.Password = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipAuth(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 *Permission) 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 ErrIntOverflowAuth
+			}
+			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: Permission: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Permission: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PermType", wireType)
+			}
+			m.PermType = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.PermType |= Permission_Type(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		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 ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
+			if m.RangeEnd == nil {
+				m.RangeEnd = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipAuth(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 *Role) 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 ErrIntOverflowAuth
+			}
+			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: Role: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Role: 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 ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 KeyPermission", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthAuth
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.KeyPermission = append(m.KeyPermission, &Permission{})
+			if err := m.KeyPermission[len(m.KeyPermission)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipAuth(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthAuth
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthAuth
+			}
+			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 skipAuth(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, ErrIntOverflowAuth
+			}
+			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, ErrIntOverflowAuth
+				}
+				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, ErrIntOverflowAuth
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthAuth
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthAuth
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowAuth
+					}
+					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 := skipAuth(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthAuth
+				}
+			}
+			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 (
+	ErrInvalidLengthAuth = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowAuth   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/github.com/coreos/etcd/auth/authpb/auth.proto b/vendor/github.com/coreos/etcd/auth/authpb/auth.proto
new file mode 100644
index 0000000..001d334
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/auth/authpb/auth.proto
@@ -0,0 +1,37 @@
+syntax = "proto3";
+package authpb;
+
+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;
+option (gogoproto.goproto_enum_prefix_all) = false;
+
+// User is a single entry in the bucket authUsers
+message User {
+  bytes name = 1;
+  bytes password = 2;
+  repeated string roles = 3;
+}
+
+// Permission is a single entity
+message Permission {
+  enum Type {
+    READ = 0;
+    WRITE = 1;
+    READWRITE = 2;
+  }
+  Type permType = 1;
+
+  bytes key = 2;
+  bytes range_end = 3;
+}
+
+// Role is a single entry in the bucket authRoles
+message Role {
+  bytes name = 1;
+
+  repeated Permission keyPermission = 2;
+}
diff --git a/vendor/github.com/coreos/etcd/clientv3/balancer/balancer.go b/vendor/github.com/coreos/etcd/clientv3/balancer/balancer.go
new file mode 100644
index 0000000..9306385
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/clientv3/balancer/balancer.go
@@ -0,0 +1,293 @@
+// 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 balancer implements client balancer.
+package balancer
+
+import (
+	"strconv"
+	"sync"
+	"time"
+
+	"github.com/coreos/etcd/clientv3/balancer/connectivity"
+	"github.com/coreos/etcd/clientv3/balancer/picker"
+
+	"go.uber.org/zap"
+	"google.golang.org/grpc/balancer"
+	grpcconnectivity "google.golang.org/grpc/connectivity"
+	"google.golang.org/grpc/resolver"
+	_ "google.golang.org/grpc/resolver/dns"         // register DNS resolver
+	_ "google.golang.org/grpc/resolver/passthrough" // register passthrough resolver
+)
+
+// Config defines balancer configurations.
+type Config struct {
+	// Policy configures balancer policy.
+	Policy picker.Policy
+
+	// Picker implements gRPC picker.
+	// Leave empty if "Policy" field is not custom.
+	// TODO: currently custom policy is not supported.
+	// Picker picker.Picker
+
+	// Name defines an additional name for balancer.
+	// Useful for balancer testing to avoid register conflicts.
+	// If empty, defaults to policy name.
+	Name string
+
+	// Logger configures balancer logging.
+	// If nil, logs are discarded.
+	Logger *zap.Logger
+}
+
+// RegisterBuilder creates and registers a builder. Since this function calls balancer.Register, it
+// must be invoked at initialization time.
+func RegisterBuilder(cfg Config) {
+	bb := &builder{cfg}
+	balancer.Register(bb)
+
+	bb.cfg.Logger.Debug(
+		"registered balancer",
+		zap.String("policy", bb.cfg.Policy.String()),
+		zap.String("name", bb.cfg.Name),
+	)
+}
+
+type builder struct {
+	cfg Config
+}
+
+// Build is called initially when creating "ccBalancerWrapper".
+// "grpc.Dial" is called to this client connection.
+// Then, resolved addresses will be handled via "HandleResolvedAddrs".
+func (b *builder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
+	bb := &baseBalancer{
+		id:     strconv.FormatInt(time.Now().UnixNano(), 36),
+		policy: b.cfg.Policy,
+		name:   b.cfg.Name,
+		lg:     b.cfg.Logger,
+
+		addrToSc: make(map[resolver.Address]balancer.SubConn),
+		scToAddr: make(map[balancer.SubConn]resolver.Address),
+		scToSt:   make(map[balancer.SubConn]grpcconnectivity.State),
+
+		currentConn:          nil,
+		connectivityRecorder: connectivity.New(b.cfg.Logger),
+
+		// initialize picker always returns "ErrNoSubConnAvailable"
+		picker: picker.NewErr(balancer.ErrNoSubConnAvailable),
+	}
+
+	// TODO: support multiple connections
+	bb.mu.Lock()
+	bb.currentConn = cc
+	bb.mu.Unlock()
+
+	bb.lg.Info(
+		"built balancer",
+		zap.String("balancer-id", bb.id),
+		zap.String("policy", bb.policy.String()),
+		zap.String("resolver-target", cc.Target()),
+	)
+	return bb
+}
+
+// Name implements "grpc/balancer.Builder" interface.
+func (b *builder) Name() string { return b.cfg.Name }
+
+// Balancer defines client balancer interface.
+type Balancer interface {
+	// Balancer is called on specified client connection. Client initiates gRPC
+	// connection with "grpc.Dial(addr, grpc.WithBalancerName)", and then those resolved
+	// addresses are passed to "grpc/balancer.Balancer.HandleResolvedAddrs".
+	// For each resolved address, balancer calls "balancer.ClientConn.NewSubConn".
+	// "grpc/balancer.Balancer.HandleSubConnStateChange" is called when connectivity state
+	// changes, thus requires failover logic in this method.
+	balancer.Balancer
+
+	// Picker calls "Pick" for every client request.
+	picker.Picker
+}
+
+type baseBalancer struct {
+	id     string
+	policy picker.Policy
+	name   string
+	lg     *zap.Logger
+
+	mu sync.RWMutex
+
+	addrToSc map[resolver.Address]balancer.SubConn
+	scToAddr map[balancer.SubConn]resolver.Address
+	scToSt   map[balancer.SubConn]grpcconnectivity.State
+
+	currentConn          balancer.ClientConn
+	connectivityRecorder connectivity.Recorder
+
+	picker picker.Picker
+}
+
+// HandleResolvedAddrs implements "grpc/balancer.Balancer" interface.
+// gRPC sends initial or updated resolved addresses from "Build".
+func (bb *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
+	if err != nil {
+		bb.lg.Warn("HandleResolvedAddrs called with error", zap.String("balancer-id", bb.id), zap.Error(err))
+		return
+	}
+	bb.lg.Info("resolved",
+		zap.String("picker", bb.picker.String()),
+		zap.String("balancer-id", bb.id),
+		zap.Strings("addresses", addrsToStrings(addrs)),
+	)
+
+	bb.mu.Lock()
+	defer bb.mu.Unlock()
+
+	resolved := make(map[resolver.Address]struct{})
+	for _, addr := range addrs {
+		resolved[addr] = struct{}{}
+		if _, ok := bb.addrToSc[addr]; !ok {
+			sc, err := bb.currentConn.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{})
+			if err != nil {
+				bb.lg.Warn("NewSubConn failed", zap.String("picker", bb.picker.String()), zap.String("balancer-id", bb.id), zap.Error(err), zap.String("address", addr.Addr))
+				continue
+			}
+			bb.lg.Info("created subconn", zap.String("address", addr.Addr))
+			bb.addrToSc[addr] = sc
+			bb.scToAddr[sc] = addr
+			bb.scToSt[sc] = grpcconnectivity.Idle
+			sc.Connect()
+		}
+	}
+
+	for addr, sc := range bb.addrToSc {
+		if _, ok := resolved[addr]; !ok {
+			// was removed by resolver or failed to create subconn
+			bb.currentConn.RemoveSubConn(sc)
+			delete(bb.addrToSc, addr)
+
+			bb.lg.Info(
+				"removed subconn",
+				zap.String("picker", bb.picker.String()),
+				zap.String("balancer-id", bb.id),
+				zap.String("address", addr.Addr),
+				zap.String("subconn", scToString(sc)),
+			)
+
+			// Keep the state of this sc in bb.scToSt until sc's state becomes Shutdown.
+			// The entry will be deleted in HandleSubConnStateChange.
+			// (DO NOT) delete(bb.scToAddr, sc)
+			// (DO NOT) delete(bb.scToSt, sc)
+		}
+	}
+}
+
+// HandleSubConnStateChange implements "grpc/balancer.Balancer" interface.
+func (bb *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s grpcconnectivity.State) {
+	bb.mu.Lock()
+	defer bb.mu.Unlock()
+
+	old, ok := bb.scToSt[sc]
+	if !ok {
+		bb.lg.Warn(
+			"state change for an unknown subconn",
+			zap.String("picker", bb.picker.String()),
+			zap.String("balancer-id", bb.id),
+			zap.String("subconn", scToString(sc)),
+			zap.Int("subconn-size", len(bb.scToAddr)),
+			zap.String("state", s.String()),
+		)
+		return
+	}
+
+	bb.lg.Info(
+		"state changed",
+		zap.String("picker", bb.picker.String()),
+		zap.String("balancer-id", bb.id),
+		zap.Bool("connected", s == grpcconnectivity.Ready),
+		zap.String("subconn", scToString(sc)),
+		zap.Int("subconn-size", len(bb.scToAddr)),
+		zap.String("address", bb.scToAddr[sc].Addr),
+		zap.String("old-state", old.String()),
+		zap.String("new-state", s.String()),
+	)
+
+	bb.scToSt[sc] = s
+	switch s {
+	case grpcconnectivity.Idle:
+		sc.Connect()
+	case grpcconnectivity.Shutdown:
+		// When an address was removed by resolver, b called RemoveSubConn but
+		// kept the sc's state in scToSt. Remove state for this sc here.
+		delete(bb.scToAddr, sc)
+		delete(bb.scToSt, sc)
+	}
+
+	oldAggrState := bb.connectivityRecorder.GetCurrentState()
+	bb.connectivityRecorder.RecordTransition(old, s)
+
+	// Update balancer picker when one of the following happens:
+	//  - this sc became ready from not-ready
+	//  - this sc became not-ready from ready
+	//  - the aggregated state of balancer became TransientFailure from non-TransientFailure
+	//  - the aggregated state of balancer became non-TransientFailure from TransientFailure
+	if (s == grpcconnectivity.Ready) != (old == grpcconnectivity.Ready) ||
+		(bb.connectivityRecorder.GetCurrentState() == grpcconnectivity.TransientFailure) != (oldAggrState == grpcconnectivity.TransientFailure) {
+		bb.updatePicker()
+	}
+
+	bb.currentConn.UpdateBalancerState(bb.connectivityRecorder.GetCurrentState(), bb.picker)
+}
+
+func (bb *baseBalancer) updatePicker() {
+	if bb.connectivityRecorder.GetCurrentState() == grpcconnectivity.TransientFailure {
+		bb.picker = picker.NewErr(balancer.ErrTransientFailure)
+		bb.lg.Info(
+			"updated picker to transient error picker",
+			zap.String("picker", bb.picker.String()),
+			zap.String("balancer-id", bb.id),
+			zap.String("policy", bb.policy.String()),
+		)
+		return
+	}
+
+	// only pass ready subconns to picker
+	scToAddr := make(map[balancer.SubConn]resolver.Address)
+	for addr, sc := range bb.addrToSc {
+		if st, ok := bb.scToSt[sc]; ok && st == grpcconnectivity.Ready {
+			scToAddr[sc] = addr
+		}
+	}
+
+	bb.picker = picker.New(picker.Config{
+		Policy:                   bb.policy,
+		Logger:                   bb.lg,
+		SubConnToResolverAddress: scToAddr,
+	})
+	bb.lg.Info(
+		"updated picker",
+		zap.String("picker", bb.picker.String()),
+		zap.String("balancer-id", bb.id),
+		zap.String("policy", bb.policy.String()),
+		zap.Strings("subconn-ready", scsToStrings(scToAddr)),
+		zap.Int("subconn-size", len(scToAddr)),
+	)
+}
+
+// Close implements "grpc/balancer.Balancer" interface.
+// Close is a nop because base balancer doesn't have internal state to clean up,
+// and it doesn't need to call RemoveSubConn for the SubConns.
+func (bb *baseBalancer) Close() {
+	// TODO
+}
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/connectivity/connectivity.go b/vendor/github.com/coreos/etcd/clientv3/balancer/connectivity/connectivity.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/connectivity/connectivity.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/connectivity/connectivity.go
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/picker/doc.go b/vendor/github.com/coreos/etcd/clientv3/balancer/picker/doc.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/picker/doc.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/picker/doc.go
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/picker/err.go b/vendor/github.com/coreos/etcd/clientv3/balancer/picker/err.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/picker/err.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/picker/err.go
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/picker/picker.go b/vendor/github.com/coreos/etcd/clientv3/balancer/picker/picker.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/picker/picker.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/picker/picker.go
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/picker/roundrobin_balanced.go b/vendor/github.com/coreos/etcd/clientv3/balancer/picker/roundrobin_balanced.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/picker/roundrobin_balanced.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/picker/roundrobin_balanced.go
diff --git a/vendor/github.com/coreos/etcd/clientv3/balancer/resolver/endpoint/endpoint.go b/vendor/github.com/coreos/etcd/clientv3/balancer/resolver/endpoint/endpoint.go
new file mode 100644
index 0000000..864b5df
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/clientv3/balancer/resolver/endpoint/endpoint.go
@@ -0,0 +1,247 @@
+// 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 endpoint resolves etcd entpoints using grpc targets of the form 'endpoint://<id>/<endpoint>'.
+package endpoint
+
+import (
+	"context"
+	"fmt"
+	"net"
+	"net/url"
+	"strings"
+	"sync"
+
+	"google.golang.org/grpc/resolver"
+)
+
+const scheme = "endpoint"
+
+var (
+	targetPrefix = fmt.Sprintf("%s://", scheme)
+
+	bldr *builder
+)
+
+func init() {
+	bldr = &builder{
+		resolverGroups: make(map[string]*ResolverGroup),
+	}
+	resolver.Register(bldr)
+}
+
+type builder struct {
+	mu             sync.RWMutex
+	resolverGroups map[string]*ResolverGroup
+}
+
+// NewResolverGroup creates a new ResolverGroup with the given id.
+func NewResolverGroup(id string) (*ResolverGroup, error) {
+	return bldr.newResolverGroup(id)
+}
+
+// ResolverGroup keeps all endpoints of resolvers using a common endpoint://<id>/ target
+// up-to-date.
+type ResolverGroup struct {
+	mu        sync.RWMutex
+	id        string
+	endpoints []string
+	resolvers []*Resolver
+}
+
+func (e *ResolverGroup) addResolver(r *Resolver) {
+	e.mu.Lock()
+	addrs := epsToAddrs(e.endpoints...)
+	e.resolvers = append(e.resolvers, r)
+	e.mu.Unlock()
+	r.cc.NewAddress(addrs)
+}
+
+func (e *ResolverGroup) removeResolver(r *Resolver) {
+	e.mu.Lock()
+	for i, er := range e.resolvers {
+		if er == r {
+			e.resolvers = append(e.resolvers[:i], e.resolvers[i+1:]...)
+			break
+		}
+	}
+	e.mu.Unlock()
+}
+
+// SetEndpoints updates the endpoints for ResolverGroup. All registered resolver are updated
+// immediately with the new endpoints.
+func (e *ResolverGroup) SetEndpoints(endpoints []string) {
+	addrs := epsToAddrs(endpoints...)
+	e.mu.Lock()
+	e.endpoints = endpoints
+	for _, r := range e.resolvers {
+		r.cc.NewAddress(addrs)
+	}
+	e.mu.Unlock()
+}
+
+// Target constructs a endpoint target using the endpoint id of the ResolverGroup.
+func (e *ResolverGroup) Target(endpoint string) string {
+	return Target(e.id, endpoint)
+}
+
+// Target constructs a endpoint resolver target.
+func Target(id, endpoint string) string {
+	return fmt.Sprintf("%s://%s/%s", scheme, id, endpoint)
+}
+
+// IsTarget checks if a given target string in an endpoint resolver target.
+func IsTarget(target string) bool {
+	return strings.HasPrefix(target, "endpoint://")
+}
+
+func (e *ResolverGroup) Close() {
+	bldr.close(e.id)
+}
+
+// Build creates or reuses an etcd resolver for the etcd cluster name identified by the authority part of the target.
+func (b *builder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) {
+	if len(target.Authority) < 1 {
+		return nil, fmt.Errorf("'etcd' target scheme requires non-empty authority identifying etcd cluster being routed to")
+	}
+	id := target.Authority
+	es, err := b.getResolverGroup(id)
+	if err != nil {
+		return nil, fmt.Errorf("failed to build resolver: %v", err)
+	}
+	r := &Resolver{
+		endpointID: id,
+		cc:         cc,
+	}
+	es.addResolver(r)
+	return r, nil
+}
+
+func (b *builder) newResolverGroup(id string) (*ResolverGroup, error) {
+	b.mu.RLock()
+	_, ok := b.resolverGroups[id]
+	b.mu.RUnlock()
+	if ok {
+		return nil, fmt.Errorf("Endpoint already exists for id: %s", id)
+	}
+
+	es := &ResolverGroup{id: id}
+	b.mu.Lock()
+	b.resolverGroups[id] = es
+	b.mu.Unlock()
+	return es, nil
+}
+
+func (b *builder) getResolverGroup(id string) (*ResolverGroup, error) {
+	b.mu.RLock()
+	es, ok := b.resolverGroups[id]
+	b.mu.RUnlock()
+	if !ok {
+		return nil, fmt.Errorf("ResolverGroup not found for id: %s", id)
+	}
+	return es, nil
+}
+
+func (b *builder) close(id string) {
+	b.mu.Lock()
+	delete(b.resolverGroups, id)
+	b.mu.Unlock()
+}
+
+func (b *builder) Scheme() string {
+	return scheme
+}
+
+// Resolver provides a resolver for a single etcd cluster, identified by name.
+type Resolver struct {
+	endpointID string
+	cc         resolver.ClientConn
+	sync.RWMutex
+}
+
+// TODO: use balancer.epsToAddrs
+func epsToAddrs(eps ...string) (addrs []resolver.Address) {
+	addrs = make([]resolver.Address, 0, len(eps))
+	for _, ep := range eps {
+		addrs = append(addrs, resolver.Address{Addr: ep})
+	}
+	return addrs
+}
+
+func (*Resolver) ResolveNow(o resolver.ResolveNowOption) {}
+
+func (r *Resolver) Close() {
+	es, err := bldr.getResolverGroup(r.endpointID)
+	if err != nil {
+		return
+	}
+	es.removeResolver(r)
+}
+
+// ParseEndpoint endpoint parses an endpoint of the form
+// (http|https)://<host>*|(unix|unixs)://<path>)
+// and returns a protocol ('tcp' or 'unix'),
+// host (or filepath if a unix socket),
+// scheme (http, https, unix, unixs).
+func ParseEndpoint(endpoint string) (proto string, host string, scheme string) {
+	proto = "tcp"
+	host = endpoint
+	url, uerr := url.Parse(endpoint)
+	if uerr != nil || !strings.Contains(endpoint, "://") {
+		return proto, host, scheme
+	}
+	scheme = url.Scheme
+
+	// strip scheme:// prefix since grpc dials by host
+	host = url.Host
+	switch url.Scheme {
+	case "http", "https":
+	case "unix", "unixs":
+		proto = "unix"
+		host = url.Host + url.Path
+	default:
+		proto, host = "", ""
+	}
+	return proto, host, scheme
+}
+
+// ParseTarget parses a endpoint://<id>/<endpoint> string and returns the parsed id and endpoint.
+// If the target is malformed, an error is returned.
+func ParseTarget(target string) (string, string, error) {
+	noPrefix := strings.TrimPrefix(target, targetPrefix)
+	if noPrefix == target {
+		return "", "", fmt.Errorf("malformed target, %s prefix is required: %s", targetPrefix, target)
+	}
+	parts := strings.SplitN(noPrefix, "/", 2)
+	if len(parts) != 2 {
+		return "", "", fmt.Errorf("malformed target, expected %s://<id>/<endpoint>, but got %s", scheme, target)
+	}
+	return parts[0], parts[1], nil
+}
+
+// Dialer dials a endpoint using net.Dialer.
+// Context cancelation and timeout are supported.
+func Dialer(ctx context.Context, dialEp string) (net.Conn, error) {
+	proto, host, _ := ParseEndpoint(dialEp)
+	select {
+	case <-ctx.Done():
+		return nil, ctx.Err()
+	default:
+	}
+	dialer := &net.Dialer{}
+	if deadline, ok := ctx.Deadline(); ok {
+		dialer.Deadline = deadline
+	}
+	return dialer.DialContext(ctx, proto, host)
+}
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/utils.go b/vendor/github.com/coreos/etcd/clientv3/balancer/utils.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/clientv3/balancer/utils.go
rename to vendor/github.com/coreos/etcd/clientv3/balancer/utils.go
diff --git a/vendor/github.com/coreos/etcd/clientv3/credentials/credentials.go b/vendor/github.com/coreos/etcd/clientv3/credentials/credentials.go
new file mode 100644
index 0000000..2dc2012
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/clientv3/credentials/credentials.go
@@ -0,0 +1,173 @@
+// 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 credentials implements gRPC credential interface with etcd specific logic.
+// e.g., client handshake with custom authority parameter
+package credentials
+
+import (
+	"context"
+	"crypto/tls"
+	"net"
+	"sync"
+
+	"github.com/coreos/etcd/clientv3/balancer/resolver/endpoint"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	grpccredentials "google.golang.org/grpc/credentials"
+)
+
+// Config defines gRPC credential configuration.
+type Config struct {
+	TLSConfig *tls.Config
+}
+
+// Bundle defines gRPC credential interface.
+type Bundle interface {
+	grpccredentials.Bundle
+	UpdateAuthToken(token string)
+}
+
+// NewBundle constructs a new gRPC credential bundle.
+func NewBundle(cfg Config) Bundle {
+	return &bundle{
+		tc: newTransportCredential(cfg.TLSConfig),
+		rc: newPerRPCCredential(),
+	}
+}
+
+// bundle implements "grpccredentials.Bundle" interface.
+type bundle struct {
+	tc *transportCredential
+	rc *perRPCCredential
+}
+
+func (b *bundle) TransportCredentials() grpccredentials.TransportCredentials {
+	return b.tc
+}
+
+func (b *bundle) PerRPCCredentials() grpccredentials.PerRPCCredentials {
+	return b.rc
+}
+
+func (b *bundle) NewWithMode(mode string) (grpccredentials.Bundle, error) {
+	// no-op
+	return nil, nil
+}
+
+// transportCredential implements "grpccredentials.TransportCredentials" interface.
+// transportCredential wraps TransportCredentials to track which
+// addresses are dialed for which endpoints, and then sets the authority when checking the endpoint's cert to the
+// hostname or IP of the dialed endpoint.
+// This is a workaround of a gRPC load balancer issue. gRPC uses the dialed target's service name as the authority when
+// checking all endpoint certs, which does not work for etcd servers using their hostname or IP as the Subject Alternative Name
+// in their TLS certs.
+// To enable, include both WithTransportCredentials(creds) and WithContextDialer(creds.Dialer)
+// when dialing.
+type transportCredential struct {
+	gtc grpccredentials.TransportCredentials
+	mu  sync.Mutex
+	// addrToEndpoint maps from the connection addresses that are dialed to the hostname or IP of the
+	// endpoint provided to the dialer when dialing
+	addrToEndpoint map[string]string
+}
+
+func newTransportCredential(cfg *tls.Config) *transportCredential {
+	return &transportCredential{
+		gtc:            grpccredentials.NewTLS(cfg),
+		addrToEndpoint: map[string]string{},
+	}
+}
+
+func (tc *transportCredential) ClientHandshake(ctx context.Context, authority string, rawConn net.Conn) (net.Conn, grpccredentials.AuthInfo, error) {
+	// Set the authority when checking the endpoint's cert to the hostname or IP of the dialed endpoint
+	tc.mu.Lock()
+	dialEp, ok := tc.addrToEndpoint[rawConn.RemoteAddr().String()]
+	tc.mu.Unlock()
+	if ok {
+		_, host, _ := endpoint.ParseEndpoint(dialEp)
+		authority = host
+	}
+	return tc.gtc.ClientHandshake(ctx, authority, rawConn)
+}
+
+// return true if given string is an IP.
+func isIP(ep string) bool {
+	return net.ParseIP(ep) != nil
+}
+
+func (tc *transportCredential) ServerHandshake(rawConn net.Conn) (net.Conn, grpccredentials.AuthInfo, error) {
+	return tc.gtc.ServerHandshake(rawConn)
+}
+
+func (tc *transportCredential) Info() grpccredentials.ProtocolInfo {
+	return tc.gtc.Info()
+}
+
+func (tc *transportCredential) Clone() grpccredentials.TransportCredentials {
+	copy := map[string]string{}
+	tc.mu.Lock()
+	for k, v := range tc.addrToEndpoint {
+		copy[k] = v
+	}
+	tc.mu.Unlock()
+	return &transportCredential{
+		gtc:            tc.gtc.Clone(),
+		addrToEndpoint: copy,
+	}
+}
+
+func (tc *transportCredential) OverrideServerName(serverNameOverride string) error {
+	return tc.gtc.OverrideServerName(serverNameOverride)
+}
+
+func (tc *transportCredential) Dialer(ctx context.Context, dialEp string) (net.Conn, error) {
+	// Keep track of which addresses are dialed for which endpoints
+	conn, err := endpoint.Dialer(ctx, dialEp)
+	if conn != nil {
+		tc.mu.Lock()
+		tc.addrToEndpoint[conn.RemoteAddr().String()] = dialEp
+		tc.mu.Unlock()
+	}
+	return conn, err
+}
+
+// perRPCCredential implements "grpccredentials.PerRPCCredentials" interface.
+type perRPCCredential struct {
+	authToken   string
+	authTokenMu sync.RWMutex
+}
+
+func newPerRPCCredential() *perRPCCredential { return &perRPCCredential{} }
+
+func (rc *perRPCCredential) RequireTransportSecurity() bool { return false }
+
+func (rc *perRPCCredential) GetRequestMetadata(ctx context.Context, s ...string) (map[string]string, error) {
+	rc.authTokenMu.RLock()
+	authToken := rc.authToken
+	rc.authTokenMu.RUnlock()
+	return map[string]string{rpctypes.TokenFieldNameGRPC: authToken}, nil
+}
+
+func (b *bundle) UpdateAuthToken(token string) {
+	if b.rc == nil {
+		return
+	}
+	b.rc.UpdateAuthToken(token)
+}
+
+func (rc *perRPCCredential) UpdateAuthToken(token string) {
+	rc.authTokenMu.Lock()
+	rc.authToken = token
+	rc.authTokenMu.Unlock()
+}
diff --git a/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/doc.go b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/doc.go
new file mode 100644
index 0000000..f72c6a6
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/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 rpctypes has types and values shared by the etcd server and client for v3 RPC interaction.
+package rpctypes
diff --git a/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/error.go b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/error.go
new file mode 100644
index 0000000..bc1ad7b
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/error.go
@@ -0,0 +1,217 @@
+// 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 rpctypes
+
+import (
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+// server-side error
+var (
+	ErrGRPCEmptyKey      = status.New(codes.InvalidArgument, "etcdserver: key is not provided").Err()
+	ErrGRPCKeyNotFound   = status.New(codes.InvalidArgument, "etcdserver: key not found").Err()
+	ErrGRPCValueProvided = status.New(codes.InvalidArgument, "etcdserver: value is provided").Err()
+	ErrGRPCLeaseProvided = status.New(codes.InvalidArgument, "etcdserver: lease is provided").Err()
+	ErrGRPCTooManyOps    = status.New(codes.InvalidArgument, "etcdserver: too many operations in txn request").Err()
+	ErrGRPCDuplicateKey  = status.New(codes.InvalidArgument, "etcdserver: duplicate key given in txn request").Err()
+	ErrGRPCCompacted     = status.New(codes.OutOfRange, "etcdserver: mvcc: required revision has been compacted").Err()
+	ErrGRPCFutureRev     = status.New(codes.OutOfRange, "etcdserver: mvcc: required revision is a future revision").Err()
+	ErrGRPCNoSpace       = status.New(codes.ResourceExhausted, "etcdserver: mvcc: database space exceeded").Err()
+
+	ErrGRPCLeaseNotFound    = status.New(codes.NotFound, "etcdserver: requested lease not found").Err()
+	ErrGRPCLeaseExist       = status.New(codes.FailedPrecondition, "etcdserver: lease already exists").Err()
+	ErrGRPCLeaseTTLTooLarge = status.New(codes.OutOfRange, "etcdserver: too large lease TTL").Err()
+
+	ErrGRPCMemberExist            = status.New(codes.FailedPrecondition, "etcdserver: member ID already exist").Err()
+	ErrGRPCPeerURLExist           = status.New(codes.FailedPrecondition, "etcdserver: Peer URLs already exists").Err()
+	ErrGRPCMemberNotEnoughStarted = status.New(codes.FailedPrecondition, "etcdserver: re-configuration failed due to not enough started members").Err()
+	ErrGRPCMemberBadURLs          = status.New(codes.InvalidArgument, "etcdserver: given member URLs are invalid").Err()
+	ErrGRPCMemberNotFound         = status.New(codes.NotFound, "etcdserver: member not found").Err()
+
+	ErrGRPCRequestTooLarge        = status.New(codes.InvalidArgument, "etcdserver: request is too large").Err()
+	ErrGRPCRequestTooManyRequests = status.New(codes.ResourceExhausted, "etcdserver: too many requests").Err()
+
+	ErrGRPCRootUserNotExist     = status.New(codes.FailedPrecondition, "etcdserver: root user does not exist").Err()
+	ErrGRPCRootRoleNotExist     = status.New(codes.FailedPrecondition, "etcdserver: root user does not have root role").Err()
+	ErrGRPCUserAlreadyExist     = status.New(codes.FailedPrecondition, "etcdserver: user name already exists").Err()
+	ErrGRPCUserEmpty            = status.New(codes.InvalidArgument, "etcdserver: user name is empty").Err()
+	ErrGRPCUserNotFound         = status.New(codes.FailedPrecondition, "etcdserver: user name not found").Err()
+	ErrGRPCRoleAlreadyExist     = status.New(codes.FailedPrecondition, "etcdserver: role name already exists").Err()
+	ErrGRPCRoleNotFound         = status.New(codes.FailedPrecondition, "etcdserver: role name not found").Err()
+	ErrGRPCAuthFailed           = status.New(codes.InvalidArgument, "etcdserver: authentication failed, invalid user ID or password").Err()
+	ErrGRPCPermissionDenied     = status.New(codes.PermissionDenied, "etcdserver: permission denied").Err()
+	ErrGRPCRoleNotGranted       = status.New(codes.FailedPrecondition, "etcdserver: role is not granted to the user").Err()
+	ErrGRPCPermissionNotGranted = status.New(codes.FailedPrecondition, "etcdserver: permission is not granted to the role").Err()
+	ErrGRPCAuthNotEnabled       = status.New(codes.FailedPrecondition, "etcdserver: authentication is not enabled").Err()
+	ErrGRPCInvalidAuthToken     = status.New(codes.Unauthenticated, "etcdserver: invalid auth token").Err()
+	ErrGRPCInvalidAuthMgmt      = status.New(codes.InvalidArgument, "etcdserver: invalid auth management").Err()
+
+	ErrGRPCNoLeader                   = status.New(codes.Unavailable, "etcdserver: no leader").Err()
+	ErrGRPCNotLeader                  = status.New(codes.FailedPrecondition, "etcdserver: not leader").Err()
+	ErrGRPCLeaderChanged              = status.New(codes.Unavailable, "etcdserver: leader changed").Err()
+	ErrGRPCNotCapable                 = status.New(codes.Unavailable, "etcdserver: not capable").Err()
+	ErrGRPCStopped                    = status.New(codes.Unavailable, "etcdserver: server stopped").Err()
+	ErrGRPCTimeout                    = status.New(codes.Unavailable, "etcdserver: request timed out").Err()
+	ErrGRPCTimeoutDueToLeaderFail     = status.New(codes.Unavailable, "etcdserver: request timed out, possibly due to previous leader failure").Err()
+	ErrGRPCTimeoutDueToConnectionLost = status.New(codes.Unavailable, "etcdserver: request timed out, possibly due to connection lost").Err()
+	ErrGRPCUnhealthy                  = status.New(codes.Unavailable, "etcdserver: unhealthy cluster").Err()
+	ErrGRPCCorrupt                    = status.New(codes.DataLoss, "etcdserver: corrupt cluster").Err()
+
+	errStringToError = map[string]error{
+		ErrorDesc(ErrGRPCEmptyKey):      ErrGRPCEmptyKey,
+		ErrorDesc(ErrGRPCKeyNotFound):   ErrGRPCKeyNotFound,
+		ErrorDesc(ErrGRPCValueProvided): ErrGRPCValueProvided,
+		ErrorDesc(ErrGRPCLeaseProvided): ErrGRPCLeaseProvided,
+
+		ErrorDesc(ErrGRPCTooManyOps):   ErrGRPCTooManyOps,
+		ErrorDesc(ErrGRPCDuplicateKey): ErrGRPCDuplicateKey,
+		ErrorDesc(ErrGRPCCompacted):    ErrGRPCCompacted,
+		ErrorDesc(ErrGRPCFutureRev):    ErrGRPCFutureRev,
+		ErrorDesc(ErrGRPCNoSpace):      ErrGRPCNoSpace,
+
+		ErrorDesc(ErrGRPCLeaseNotFound):    ErrGRPCLeaseNotFound,
+		ErrorDesc(ErrGRPCLeaseExist):       ErrGRPCLeaseExist,
+		ErrorDesc(ErrGRPCLeaseTTLTooLarge): ErrGRPCLeaseTTLTooLarge,
+
+		ErrorDesc(ErrGRPCMemberExist):            ErrGRPCMemberExist,
+		ErrorDesc(ErrGRPCPeerURLExist):           ErrGRPCPeerURLExist,
+		ErrorDesc(ErrGRPCMemberNotEnoughStarted): ErrGRPCMemberNotEnoughStarted,
+		ErrorDesc(ErrGRPCMemberBadURLs):          ErrGRPCMemberBadURLs,
+		ErrorDesc(ErrGRPCMemberNotFound):         ErrGRPCMemberNotFound,
+
+		ErrorDesc(ErrGRPCRequestTooLarge):        ErrGRPCRequestTooLarge,
+		ErrorDesc(ErrGRPCRequestTooManyRequests): ErrGRPCRequestTooManyRequests,
+
+		ErrorDesc(ErrGRPCRootUserNotExist):     ErrGRPCRootUserNotExist,
+		ErrorDesc(ErrGRPCRootRoleNotExist):     ErrGRPCRootRoleNotExist,
+		ErrorDesc(ErrGRPCUserAlreadyExist):     ErrGRPCUserAlreadyExist,
+		ErrorDesc(ErrGRPCUserEmpty):            ErrGRPCUserEmpty,
+		ErrorDesc(ErrGRPCUserNotFound):         ErrGRPCUserNotFound,
+		ErrorDesc(ErrGRPCRoleAlreadyExist):     ErrGRPCRoleAlreadyExist,
+		ErrorDesc(ErrGRPCRoleNotFound):         ErrGRPCRoleNotFound,
+		ErrorDesc(ErrGRPCAuthFailed):           ErrGRPCAuthFailed,
+		ErrorDesc(ErrGRPCPermissionDenied):     ErrGRPCPermissionDenied,
+		ErrorDesc(ErrGRPCRoleNotGranted):       ErrGRPCRoleNotGranted,
+		ErrorDesc(ErrGRPCPermissionNotGranted): ErrGRPCPermissionNotGranted,
+		ErrorDesc(ErrGRPCAuthNotEnabled):       ErrGRPCAuthNotEnabled,
+		ErrorDesc(ErrGRPCInvalidAuthToken):     ErrGRPCInvalidAuthToken,
+		ErrorDesc(ErrGRPCInvalidAuthMgmt):      ErrGRPCInvalidAuthMgmt,
+
+		ErrorDesc(ErrGRPCNoLeader):                   ErrGRPCNoLeader,
+		ErrorDesc(ErrGRPCNotLeader):                  ErrGRPCNotLeader,
+		ErrorDesc(ErrGRPCNotCapable):                 ErrGRPCNotCapable,
+		ErrorDesc(ErrGRPCStopped):                    ErrGRPCStopped,
+		ErrorDesc(ErrGRPCTimeout):                    ErrGRPCTimeout,
+		ErrorDesc(ErrGRPCTimeoutDueToLeaderFail):     ErrGRPCTimeoutDueToLeaderFail,
+		ErrorDesc(ErrGRPCTimeoutDueToConnectionLost): ErrGRPCTimeoutDueToConnectionLost,
+		ErrorDesc(ErrGRPCUnhealthy):                  ErrGRPCUnhealthy,
+		ErrorDesc(ErrGRPCCorrupt):                    ErrGRPCCorrupt,
+	}
+)
+
+// client-side error
+var (
+	ErrEmptyKey      = Error(ErrGRPCEmptyKey)
+	ErrKeyNotFound   = Error(ErrGRPCKeyNotFound)
+	ErrValueProvided = Error(ErrGRPCValueProvided)
+	ErrLeaseProvided = Error(ErrGRPCLeaseProvided)
+	ErrTooManyOps    = Error(ErrGRPCTooManyOps)
+	ErrDuplicateKey  = Error(ErrGRPCDuplicateKey)
+	ErrCompacted     = Error(ErrGRPCCompacted)
+	ErrFutureRev     = Error(ErrGRPCFutureRev)
+	ErrNoSpace       = Error(ErrGRPCNoSpace)
+
+	ErrLeaseNotFound    = Error(ErrGRPCLeaseNotFound)
+	ErrLeaseExist       = Error(ErrGRPCLeaseExist)
+	ErrLeaseTTLTooLarge = Error(ErrGRPCLeaseTTLTooLarge)
+
+	ErrMemberExist            = Error(ErrGRPCMemberExist)
+	ErrPeerURLExist           = Error(ErrGRPCPeerURLExist)
+	ErrMemberNotEnoughStarted = Error(ErrGRPCMemberNotEnoughStarted)
+	ErrMemberBadURLs          = Error(ErrGRPCMemberBadURLs)
+	ErrMemberNotFound         = Error(ErrGRPCMemberNotFound)
+
+	ErrRequestTooLarge = Error(ErrGRPCRequestTooLarge)
+	ErrTooManyRequests = Error(ErrGRPCRequestTooManyRequests)
+
+	ErrRootUserNotExist     = Error(ErrGRPCRootUserNotExist)
+	ErrRootRoleNotExist     = Error(ErrGRPCRootRoleNotExist)
+	ErrUserAlreadyExist     = Error(ErrGRPCUserAlreadyExist)
+	ErrUserEmpty            = Error(ErrGRPCUserEmpty)
+	ErrUserNotFound         = Error(ErrGRPCUserNotFound)
+	ErrRoleAlreadyExist     = Error(ErrGRPCRoleAlreadyExist)
+	ErrRoleNotFound         = Error(ErrGRPCRoleNotFound)
+	ErrAuthFailed           = Error(ErrGRPCAuthFailed)
+	ErrPermissionDenied     = Error(ErrGRPCPermissionDenied)
+	ErrRoleNotGranted       = Error(ErrGRPCRoleNotGranted)
+	ErrPermissionNotGranted = Error(ErrGRPCPermissionNotGranted)
+	ErrAuthNotEnabled       = Error(ErrGRPCAuthNotEnabled)
+	ErrInvalidAuthToken     = Error(ErrGRPCInvalidAuthToken)
+	ErrInvalidAuthMgmt      = Error(ErrGRPCInvalidAuthMgmt)
+
+	ErrNoLeader                   = Error(ErrGRPCNoLeader)
+	ErrNotLeader                  = Error(ErrGRPCNotLeader)
+	ErrLeaderChanged              = Error(ErrGRPCLeaderChanged)
+	ErrNotCapable                 = Error(ErrGRPCNotCapable)
+	ErrStopped                    = Error(ErrGRPCStopped)
+	ErrTimeout                    = Error(ErrGRPCTimeout)
+	ErrTimeoutDueToLeaderFail     = Error(ErrGRPCTimeoutDueToLeaderFail)
+	ErrTimeoutDueToConnectionLost = Error(ErrGRPCTimeoutDueToConnectionLost)
+	ErrUnhealthy                  = Error(ErrGRPCUnhealthy)
+	ErrCorrupt                    = Error(ErrGRPCCorrupt)
+)
+
+// EtcdError defines gRPC server errors.
+// (https://github.com/grpc/grpc-go/blob/master/rpc_util.go#L319-L323)
+type EtcdError struct {
+	code codes.Code
+	desc string
+}
+
+// Code returns grpc/codes.Code.
+// TODO: define clientv3/codes.Code.
+func (e EtcdError) Code() codes.Code {
+	return e.code
+}
+
+func (e EtcdError) Error() string {
+	return e.desc
+}
+
+func Error(err error) error {
+	if err == nil {
+		return nil
+	}
+	verr, ok := errStringToError[ErrorDesc(err)]
+	if !ok { // not gRPC error
+		return err
+	}
+	ev, ok := status.FromError(verr)
+	var desc string
+	if ok {
+		desc = ev.Message()
+	} else {
+		desc = verr.Error()
+	}
+	return EtcdError{code: ev.Code(), desc: desc}
+}
+
+func ErrorDesc(err error) string {
+	if s, ok := status.FromError(err); ok {
+		return s.Message()
+	}
+	return err.Error()
+}
diff --git a/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/md.go b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/md.go
new file mode 100644
index 0000000..90b8b83
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/md.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.
+
+package rpctypes
+
+var (
+	MetadataRequireLeaderKey = "hasleader"
+	MetadataHasLeader        = "true"
+
+	MetadataClientAPIVersionKey = "client-api-version"
+)
diff --git a/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/metadatafields.go b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/metadatafields.go
new file mode 100644
index 0000000..8f8ac60
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/metadatafields.go
@@ -0,0 +1,20 @@
+// 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 rpctypes
+
+var (
+	TokenFieldNameGRPC    = "token"
+	TokenFieldNameSwagger = "authorization"
+)
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.pb.go b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.pb.go
new file mode 100644
index 0000000..12b6763
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.pb.go
@@ -0,0 +1,1034 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: etcdserver.proto
+
+package etcdserverpb
+
+import (
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+	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.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Request struct {
+	ID                   uint64   `protobuf:"varint,1,opt,name=ID" json:"ID"`
+	Method               string   `protobuf:"bytes,2,opt,name=Method" json:"Method"`
+	Path                 string   `protobuf:"bytes,3,opt,name=Path" json:"Path"`
+	Val                  string   `protobuf:"bytes,4,opt,name=Val" json:"Val"`
+	Dir                  bool     `protobuf:"varint,5,opt,name=Dir" json:"Dir"`
+	PrevValue            string   `protobuf:"bytes,6,opt,name=PrevValue" json:"PrevValue"`
+	PrevIndex            uint64   `protobuf:"varint,7,opt,name=PrevIndex" json:"PrevIndex"`
+	PrevExist            *bool    `protobuf:"varint,8,opt,name=PrevExist" json:"PrevExist,omitempty"`
+	Expiration           int64    `protobuf:"varint,9,opt,name=Expiration" json:"Expiration"`
+	Wait                 bool     `protobuf:"varint,10,opt,name=Wait" json:"Wait"`
+	Since                uint64   `protobuf:"varint,11,opt,name=Since" json:"Since"`
+	Recursive            bool     `protobuf:"varint,12,opt,name=Recursive" json:"Recursive"`
+	Sorted               bool     `protobuf:"varint,13,opt,name=Sorted" json:"Sorted"`
+	Quorum               bool     `protobuf:"varint,14,opt,name=Quorum" json:"Quorum"`
+	Time                 int64    `protobuf:"varint,15,opt,name=Time" json:"Time"`
+	Stream               bool     `protobuf:"varint,16,opt,name=Stream" json:"Stream"`
+	Refresh              *bool    `protobuf:"varint,17,opt,name=Refresh" json:"Refresh,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Request) Reset()         { *m = Request{} }
+func (m *Request) String() string { return proto.CompactTextString(m) }
+func (*Request) ProtoMessage()    {}
+func (*Request) Descriptor() ([]byte, []int) {
+	return fileDescriptor_09ffbeb3bebbce7e, []int{0}
+}
+func (m *Request) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Request) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Request.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Request) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Request.Merge(m, src)
+}
+func (m *Request) XXX_Size() int {
+	return m.Size()
+}
+func (m *Request) XXX_DiscardUnknown() {
+	xxx_messageInfo_Request.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Request proto.InternalMessageInfo
+
+type Metadata struct {
+	NodeID               uint64   `protobuf:"varint,1,opt,name=NodeID" json:"NodeID"`
+	ClusterID            uint64   `protobuf:"varint,2,opt,name=ClusterID" json:"ClusterID"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Metadata) Reset()         { *m = Metadata{} }
+func (m *Metadata) String() string { return proto.CompactTextString(m) }
+func (*Metadata) ProtoMessage()    {}
+func (*Metadata) Descriptor() ([]byte, []int) {
+	return fileDescriptor_09ffbeb3bebbce7e, []int{1}
+}
+func (m *Metadata) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Metadata.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Metadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metadata.Merge(m, src)
+}
+func (m *Metadata) XXX_Size() int {
+	return m.Size()
+}
+func (m *Metadata) XXX_DiscardUnknown() {
+	xxx_messageInfo_Metadata.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Metadata proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterType((*Request)(nil), "etcdserverpb.Request")
+	proto.RegisterType((*Metadata)(nil), "etcdserverpb.Metadata")
+}
+
+func init() { proto.RegisterFile("etcdserver.proto", fileDescriptor_09ffbeb3bebbce7e) }
+
+var fileDescriptor_09ffbeb3bebbce7e = []byte{
+	// 380 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0xd2, 0xdd, 0x6e, 0xda, 0x30,
+	0x14, 0x07, 0x70, 0x0c, 0xe1, 0xcb, 0x63, 0x1b, 0xb3, 0xd0, 0x74, 0x84, 0xa6, 0x2c, 0x42, 0xbb,
+	0xc8, 0xd5, 0xf6, 0x0e, 0x2c, 0x5c, 0x44, 0x2a, 0x15, 0x0d, 0x15, 0xbd, 0x76, 0xc9, 0x29, 0x58,
+	0x02, 0x4c, 0x1d, 0x07, 0xf1, 0x06, 0x7d, 0x85, 0x3e, 0x12, 0x97, 0x7d, 0x82, 0xaa, 0xa5, 0x2f,
+	0x52, 0x39, 0x24, 0xc4, 0xed, 0x5d, 0xf4, 0xfb, 0x9f, 0x1c, 0x1f, 0x7f, 0xd0, 0x2e, 0xea, 0x79,
+	0x9c, 0xa0, 0xda, 0xa1, 0xfa, 0xbb, 0x55, 0x52, 0x4b, 0xd6, 0x29, 0x65, 0x7b, 0xdb, 0xef, 0x2d,
+	0xe4, 0x42, 0x66, 0xc1, 0x3f, 0xf3, 0x75, 0xaa, 0x19, 0x3c, 0x38, 0xb4, 0x19, 0xe1, 0x7d, 0x8a,
+	0x89, 0x66, 0x3d, 0x5a, 0x0d, 0x03, 0x20, 0x1e, 0xf1, 0x9d, 0xa1, 0x73, 0x78, 0xfe, 0x5d, 0x89,
+	0xaa, 0x61, 0xc0, 0x7e, 0xd1, 0xc6, 0x18, 0xf5, 0x52, 0xc6, 0x50, 0xf5, 0x88, 0xdf, 0xce, 0x93,
+	0xdc, 0x18, 0x50, 0x67, 0xc2, 0xf5, 0x12, 0x6a, 0x56, 0x96, 0x09, 0xfb, 0x49, 0x6b, 0x33, 0xbe,
+	0x02, 0xc7, 0x0a, 0x0c, 0x18, 0x0f, 0x84, 0x82, 0xba, 0x47, 0xfc, 0x56, 0xe1, 0x81, 0x50, 0x6c,
+	0x40, 0xdb, 0x13, 0x85, 0xbb, 0x19, 0x5f, 0xa5, 0x08, 0x0d, 0xeb, 0xaf, 0x92, 0x8b, 0x9a, 0x70,
+	0x13, 0xe3, 0x1e, 0x9a, 0xd6, 0xa0, 0x25, 0x17, 0x35, 0xa3, 0xbd, 0x48, 0x34, 0xb4, 0xce, 0xab,
+	0x90, 0xa8, 0x64, 0xf6, 0x87, 0xd2, 0xd1, 0x7e, 0x2b, 0x14, 0xd7, 0x42, 0x6e, 0xa0, 0xed, 0x11,
+	0xbf, 0x96, 0x37, 0xb2, 0xdc, 0xec, 0xed, 0x86, 0x0b, 0x0d, 0xd4, 0x1a, 0x35, 0x13, 0xd6, 0xa7,
+	0xf5, 0xa9, 0xd8, 0xcc, 0x11, 0xbe, 0x58, 0x33, 0x9c, 0xc8, 0xac, 0x1f, 0xe1, 0x3c, 0x55, 0x89,
+	0xd8, 0x21, 0x74, 0xac, 0x5f, 0x4b, 0x36, 0x67, 0x3a, 0x95, 0x4a, 0x63, 0x0c, 0x5f, 0xad, 0x82,
+	0xdc, 0x4c, 0x7a, 0x95, 0x4a, 0x95, 0xae, 0xe1, 0x9b, 0x9d, 0x9e, 0xcc, 0x4c, 0x75, 0x2d, 0xd6,
+	0x08, 0xdf, 0xad, 0xa9, 0x33, 0xc9, 0xba, 0x6a, 0x85, 0x7c, 0x0d, 0xdd, 0x0f, 0x5d, 0x33, 0x63,
+	0xae, 0xb9, 0xe8, 0x3b, 0x85, 0xc9, 0x12, 0x7e, 0x58, 0xa7, 0x52, 0xe0, 0xe0, 0x82, 0xb6, 0xc6,
+	0xa8, 0x79, 0xcc, 0x35, 0x37, 0x9d, 0x2e, 0x65, 0x8c, 0x9f, 0x5e, 0x43, 0x6e, 0x66, 0x87, 0xff,
+	0x57, 0x69, 0xa2, 0x51, 0x85, 0x41, 0xf6, 0x28, 0xce, 0xb7, 0x70, 0xe6, 0x61, 0xef, 0xf0, 0xea,
+	0x56, 0x0e, 0x47, 0x97, 0x3c, 0x1d, 0x5d, 0xf2, 0x72, 0x74, 0xc9, 0xe3, 0x9b, 0x5b, 0x79, 0x0f,
+	0x00, 0x00, 0xff, 0xff, 0xee, 0x40, 0xba, 0xd6, 0xa4, 0x02, 0x00, 0x00,
+}
+
+func (m *Request) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Request) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Request) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Refresh != nil {
+		i--
+		if *m.Refresh {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x1
+		i--
+		dAtA[i] = 0x88
+	}
+	i--
+	if m.Stream {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x1
+	i--
+	dAtA[i] = 0x80
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Time))
+	i--
+	dAtA[i] = 0x78
+	i--
+	if m.Quorum {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x70
+	i--
+	if m.Sorted {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x68
+	i--
+	if m.Recursive {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x60
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Since))
+	i--
+	dAtA[i] = 0x58
+	i--
+	if m.Wait {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x50
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Expiration))
+	i--
+	dAtA[i] = 0x48
+	if m.PrevExist != nil {
+		i--
+		if *m.PrevExist {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x40
+	}
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.PrevIndex))
+	i--
+	dAtA[i] = 0x38
+	i -= len(m.PrevValue)
+	copy(dAtA[i:], m.PrevValue)
+	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.PrevValue)))
+	i--
+	dAtA[i] = 0x32
+	i--
+	if m.Dir {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x28
+	i -= len(m.Val)
+	copy(dAtA[i:], m.Val)
+	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Val)))
+	i--
+	dAtA[i] = 0x22
+	i -= len(m.Path)
+	copy(dAtA[i:], m.Path)
+	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Path)))
+	i--
+	dAtA[i] = 0x1a
+	i -= len(m.Method)
+	copy(dAtA[i:], m.Method)
+	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Method)))
+	i--
+	dAtA[i] = 0x12
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.ID))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func (m *Metadata) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Metadata) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.ClusterID))
+	i--
+	dAtA[i] = 0x10
+	i = encodeVarintEtcdserver(dAtA, i, uint64(m.NodeID))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintEtcdserver(dAtA []byte, offset int, v uint64) int {
+	offset -= sovEtcdserver(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *Request) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovEtcdserver(uint64(m.ID))
+	l = len(m.Method)
+	n += 1 + l + sovEtcdserver(uint64(l))
+	l = len(m.Path)
+	n += 1 + l + sovEtcdserver(uint64(l))
+	l = len(m.Val)
+	n += 1 + l + sovEtcdserver(uint64(l))
+	n += 2
+	l = len(m.PrevValue)
+	n += 1 + l + sovEtcdserver(uint64(l))
+	n += 1 + sovEtcdserver(uint64(m.PrevIndex))
+	if m.PrevExist != nil {
+		n += 2
+	}
+	n += 1 + sovEtcdserver(uint64(m.Expiration))
+	n += 2
+	n += 1 + sovEtcdserver(uint64(m.Since))
+	n += 2
+	n += 2
+	n += 2
+	n += 1 + sovEtcdserver(uint64(m.Time))
+	n += 3
+	if m.Refresh != nil {
+		n += 3
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Metadata) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovEtcdserver(uint64(m.NodeID))
+	n += 1 + sovEtcdserver(uint64(m.ClusterID))
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovEtcdserver(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozEtcdserver(x uint64) (n int) {
+	return sovEtcdserver(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Request) 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 ErrIntOverflowEtcdserver
+			}
+			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: Request: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Request: 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 ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Method = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Path = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Val", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Val = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Dir", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Dir = bool(v != 0)
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevValue", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PrevValue = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 7:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevIndex", wireType)
+			}
+			m.PrevIndex = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.PrevIndex |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevExist", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			b := bool(v != 0)
+			m.PrevExist = &b
+		case 9:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expiration", wireType)
+			}
+			m.Expiration = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Expiration |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 10:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Wait", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Wait = bool(v != 0)
+		case 11:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Since", wireType)
+			}
+			m.Since = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Since |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 12:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Recursive = bool(v != 0)
+		case 13:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Sorted", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Sorted = bool(v != 0)
+		case 14:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Quorum", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Quorum = bool(v != 0)
+		case 15:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Time", wireType)
+			}
+			m.Time = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Time |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 16:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Stream", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Stream = bool(v != 0)
+		case 17:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Refresh", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			b := bool(v != 0)
+			m.Refresh = &b
+		default:
+			iNdEx = preIndex
+			skippy, err := skipEtcdserver(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			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 *Metadata) 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 ErrIntOverflowEtcdserver
+			}
+			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: Metadata: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
+			}
+			m.NodeID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.NodeID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ClusterID", wireType)
+			}
+			m.ClusterID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ClusterID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipEtcdserver(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthEtcdserver
+			}
+			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 skipEtcdserver(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, ErrIntOverflowEtcdserver
+			}
+			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, ErrIntOverflowEtcdserver
+				}
+				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, ErrIntOverflowEtcdserver
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthEtcdserver
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthEtcdserver
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowEtcdserver
+					}
+					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 := skipEtcdserver(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthEtcdserver
+				}
+			}
+			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 (
+	ErrInvalidLengthEtcdserver = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowEtcdserver   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/etcdserver.proto b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.proto
similarity index 100%
rename from vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/etcdserver.proto
rename to vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.proto
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.pb.go b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.pb.go
new file mode 100644
index 0000000..b3a199e
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.pb.go
@@ -0,0 +1,2427 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: raft_internal.proto
+
+package etcdserverpb
+
+import (
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+	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.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type RequestHeader struct {
+	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	// username is a username that is associated with an auth token of gRPC connection
+	Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"`
+	// auth_revision is a revision number of auth.authStore. It is not related to mvcc
+	AuthRevision         uint64   `protobuf:"varint,3,opt,name=auth_revision,json=authRevision,proto3" json:"auth_revision,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *RequestHeader) Reset()         { *m = RequestHeader{} }
+func (m *RequestHeader) String() string { return proto.CompactTextString(m) }
+func (*RequestHeader) ProtoMessage()    {}
+func (*RequestHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b4c9a9be0cfca103, []int{0}
+}
+func (m *RequestHeader) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *RequestHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_RequestHeader.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *RequestHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RequestHeader.Merge(m, src)
+}
+func (m *RequestHeader) XXX_Size() int {
+	return m.Size()
+}
+func (m *RequestHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_RequestHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RequestHeader proto.InternalMessageInfo
+
+// An InternalRaftRequest is the union of all requests which can be
+// sent via raft.
+type InternalRaftRequest struct {
+	Header                   *RequestHeader                   `protobuf:"bytes,100,opt,name=header,proto3" json:"header,omitempty"`
+	ID                       uint64                           `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	V2                       *Request                         `protobuf:"bytes,2,opt,name=v2,proto3" json:"v2,omitempty"`
+	Range                    *RangeRequest                    `protobuf:"bytes,3,opt,name=range,proto3" json:"range,omitempty"`
+	Put                      *PutRequest                      `protobuf:"bytes,4,opt,name=put,proto3" json:"put,omitempty"`
+	DeleteRange              *DeleteRangeRequest              `protobuf:"bytes,5,opt,name=delete_range,json=deleteRange,proto3" json:"delete_range,omitempty"`
+	Txn                      *TxnRequest                      `protobuf:"bytes,6,opt,name=txn,proto3" json:"txn,omitempty"`
+	Compaction               *CompactionRequest               `protobuf:"bytes,7,opt,name=compaction,proto3" json:"compaction,omitempty"`
+	LeaseGrant               *LeaseGrantRequest               `protobuf:"bytes,8,opt,name=lease_grant,json=leaseGrant,proto3" json:"lease_grant,omitempty"`
+	LeaseRevoke              *LeaseRevokeRequest              `protobuf:"bytes,9,opt,name=lease_revoke,json=leaseRevoke,proto3" json:"lease_revoke,omitempty"`
+	Alarm                    *AlarmRequest                    `protobuf:"bytes,10,opt,name=alarm,proto3" json:"alarm,omitempty"`
+	AuthEnable               *AuthEnableRequest               `protobuf:"bytes,1000,opt,name=auth_enable,json=authEnable,proto3" json:"auth_enable,omitempty"`
+	AuthDisable              *AuthDisableRequest              `protobuf:"bytes,1011,opt,name=auth_disable,json=authDisable,proto3" json:"auth_disable,omitempty"`
+	Authenticate             *InternalAuthenticateRequest     `protobuf:"bytes,1012,opt,name=authenticate,proto3" json:"authenticate,omitempty"`
+	AuthUserAdd              *AuthUserAddRequest              `protobuf:"bytes,1100,opt,name=auth_user_add,json=authUserAdd,proto3" json:"auth_user_add,omitempty"`
+	AuthUserDelete           *AuthUserDeleteRequest           `protobuf:"bytes,1101,opt,name=auth_user_delete,json=authUserDelete,proto3" json:"auth_user_delete,omitempty"`
+	AuthUserGet              *AuthUserGetRequest              `protobuf:"bytes,1102,opt,name=auth_user_get,json=authUserGet,proto3" json:"auth_user_get,omitempty"`
+	AuthUserChangePassword   *AuthUserChangePasswordRequest   `protobuf:"bytes,1103,opt,name=auth_user_change_password,json=authUserChangePassword,proto3" json:"auth_user_change_password,omitempty"`
+	AuthUserGrantRole        *AuthUserGrantRoleRequest        `protobuf:"bytes,1104,opt,name=auth_user_grant_role,json=authUserGrantRole,proto3" json:"auth_user_grant_role,omitempty"`
+	AuthUserRevokeRole       *AuthUserRevokeRoleRequest       `protobuf:"bytes,1105,opt,name=auth_user_revoke_role,json=authUserRevokeRole,proto3" json:"auth_user_revoke_role,omitempty"`
+	AuthUserList             *AuthUserListRequest             `protobuf:"bytes,1106,opt,name=auth_user_list,json=authUserList,proto3" json:"auth_user_list,omitempty"`
+	AuthRoleList             *AuthRoleListRequest             `protobuf:"bytes,1107,opt,name=auth_role_list,json=authRoleList,proto3" json:"auth_role_list,omitempty"`
+	AuthRoleAdd              *AuthRoleAddRequest              `protobuf:"bytes,1200,opt,name=auth_role_add,json=authRoleAdd,proto3" json:"auth_role_add,omitempty"`
+	AuthRoleDelete           *AuthRoleDeleteRequest           `protobuf:"bytes,1201,opt,name=auth_role_delete,json=authRoleDelete,proto3" json:"auth_role_delete,omitempty"`
+	AuthRoleGet              *AuthRoleGetRequest              `protobuf:"bytes,1202,opt,name=auth_role_get,json=authRoleGet,proto3" json:"auth_role_get,omitempty"`
+	AuthRoleGrantPermission  *AuthRoleGrantPermissionRequest  `protobuf:"bytes,1203,opt,name=auth_role_grant_permission,json=authRoleGrantPermission,proto3" json:"auth_role_grant_permission,omitempty"`
+	AuthRoleRevokePermission *AuthRoleRevokePermissionRequest `protobuf:"bytes,1204,opt,name=auth_role_revoke_permission,json=authRoleRevokePermission,proto3" json:"auth_role_revoke_permission,omitempty"`
+	XXX_NoUnkeyedLiteral     struct{}                         `json:"-"`
+	XXX_unrecognized         []byte                           `json:"-"`
+	XXX_sizecache            int32                            `json:"-"`
+}
+
+func (m *InternalRaftRequest) Reset()         { *m = InternalRaftRequest{} }
+func (m *InternalRaftRequest) String() string { return proto.CompactTextString(m) }
+func (*InternalRaftRequest) ProtoMessage()    {}
+func (*InternalRaftRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b4c9a9be0cfca103, []int{1}
+}
+func (m *InternalRaftRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *InternalRaftRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_InternalRaftRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *InternalRaftRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_InternalRaftRequest.Merge(m, src)
+}
+func (m *InternalRaftRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *InternalRaftRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_InternalRaftRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_InternalRaftRequest proto.InternalMessageInfo
+
+type EmptyResponse struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EmptyResponse) Reset()         { *m = EmptyResponse{} }
+func (m *EmptyResponse) String() string { return proto.CompactTextString(m) }
+func (*EmptyResponse) ProtoMessage()    {}
+func (*EmptyResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b4c9a9be0cfca103, []int{2}
+}
+func (m *EmptyResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *EmptyResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_EmptyResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *EmptyResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EmptyResponse.Merge(m, src)
+}
+func (m *EmptyResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *EmptyResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_EmptyResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EmptyResponse proto.InternalMessageInfo
+
+// What is the difference between AuthenticateRequest (defined in rpc.proto) and InternalAuthenticateRequest?
+// InternalAuthenticateRequest has a member that is filled by etcdserver and shouldn't be user-facing.
+// For avoiding misusage the field, we have an internal version of AuthenticateRequest.
+type InternalAuthenticateRequest struct {
+	Name     string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
+	// simple_token is generated in API layer (etcdserver/v3_server.go)
+	SimpleToken          string   `protobuf:"bytes,3,opt,name=simple_token,json=simpleToken,proto3" json:"simple_token,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *InternalAuthenticateRequest) Reset()         { *m = InternalAuthenticateRequest{} }
+func (m *InternalAuthenticateRequest) String() string { return proto.CompactTextString(m) }
+func (*InternalAuthenticateRequest) ProtoMessage()    {}
+func (*InternalAuthenticateRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b4c9a9be0cfca103, []int{3}
+}
+func (m *InternalAuthenticateRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *InternalAuthenticateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_InternalAuthenticateRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *InternalAuthenticateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_InternalAuthenticateRequest.Merge(m, src)
+}
+func (m *InternalAuthenticateRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *InternalAuthenticateRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_InternalAuthenticateRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_InternalAuthenticateRequest proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterType((*RequestHeader)(nil), "etcdserverpb.RequestHeader")
+	proto.RegisterType((*InternalRaftRequest)(nil), "etcdserverpb.InternalRaftRequest")
+	proto.RegisterType((*EmptyResponse)(nil), "etcdserverpb.EmptyResponse")
+	proto.RegisterType((*InternalAuthenticateRequest)(nil), "etcdserverpb.InternalAuthenticateRequest")
+}
+
+func init() { proto.RegisterFile("raft_internal.proto", fileDescriptor_b4c9a9be0cfca103) }
+
+var fileDescriptor_b4c9a9be0cfca103 = []byte{
+	// 840 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x96, 0xdb, 0x4e, 0xdb, 0x48,
+	0x18, 0xc7, 0x71, 0x38, 0x66, 0x12, 0xb2, 0xec, 0x00, 0xbb, 0xb3, 0x41, 0xca, 0x86, 0xa0, 0xdd,
+	0x65, 0x77, 0x5b, 0x5a, 0x85, 0x07, 0x68, 0x53, 0x82, 0x00, 0x09, 0x21, 0x64, 0x51, 0xa9, 0x52,
+	0x2f, 0xdc, 0x21, 0xfe, 0x48, 0x5c, 0x1c, 0xdb, 0x1d, 0x4f, 0x52, 0xfa, 0x26, 0x7d, 0x8c, 0x9e,
+	0x1e, 0x82, 0x8b, 0x1e, 0x68, 0xfb, 0x02, 0x2d, 0xbd, 0xe9, 0x55, 0x6f, 0xda, 0x07, 0xa8, 0xe6,
+	0x60, 0x3b, 0x4e, 0x1c, 0xee, 0xec, 0x6f, 0xfe, 0xdf, 0xef, 0xfb, 0x0f, 0xf3, 0x37, 0x13, 0xb4,
+	0xc8, 0xe8, 0x09, 0xb7, 0x1c, 0x8f, 0x03, 0xf3, 0xa8, 0xbb, 0x11, 0x30, 0x9f, 0xfb, 0xb8, 0x08,
+	0xbc, 0x65, 0x87, 0xc0, 0xfa, 0xc0, 0x82, 0xe3, 0xf2, 0x52, 0xdb, 0x6f, 0xfb, 0x72, 0xe1, 0x86,
+	0x78, 0x52, 0x9a, 0xf2, 0x42, 0xa2, 0xd1, 0x95, 0x3c, 0x0b, 0x5a, 0xea, 0xb1, 0xf6, 0x00, 0xcd,
+	0x9b, 0xf0, 0xa8, 0x07, 0x21, 0xdf, 0x05, 0x6a, 0x03, 0xc3, 0x25, 0x94, 0xdb, 0x6b, 0x12, 0xa3,
+	0x6a, 0xac, 0x4f, 0x99, 0xb9, 0xbd, 0x26, 0x2e, 0xa3, 0xb9, 0x5e, 0x28, 0x46, 0x76, 0x81, 0xe4,
+	0xaa, 0xc6, 0x7a, 0xde, 0x8c, 0xdf, 0xf1, 0x1a, 0x9a, 0xa7, 0x3d, 0xde, 0xb1, 0x18, 0xf4, 0x9d,
+	0xd0, 0xf1, 0x3d, 0x32, 0x29, 0xdb, 0x8a, 0xa2, 0x68, 0xea, 0x5a, 0xed, 0x5b, 0x09, 0x2d, 0xee,
+	0x69, 0xd7, 0x26, 0x3d, 0xe1, 0x7a, 0x1c, 0xde, 0x44, 0x33, 0x1d, 0x39, 0x92, 0xd8, 0x55, 0x63,
+	0xbd, 0x50, 0x5f, 0xd9, 0x18, 0xdc, 0xcb, 0x46, 0xca, 0x95, 0xa9, 0xa5, 0x23, 0xee, 0xfe, 0x42,
+	0xb9, 0x7e, 0x5d, 0xfa, 0x2a, 0xd4, 0x97, 0x33, 0x01, 0x66, 0xae, 0x5f, 0xc7, 0x37, 0xd1, 0x34,
+	0xa3, 0x5e, 0x1b, 0xa4, 0xc1, 0x42, 0xbd, 0x3c, 0xa4, 0x14, 0x4b, 0x91, 0x5c, 0x09, 0xf1, 0x7f,
+	0x68, 0x32, 0xe8, 0x71, 0x32, 0x25, 0xf5, 0x24, 0xad, 0x3f, 0xec, 0x45, 0x9b, 0x30, 0x85, 0x08,
+	0x6f, 0xa1, 0xa2, 0x0d, 0x2e, 0x70, 0xb0, 0xd4, 0x90, 0x69, 0xd9, 0x54, 0x4d, 0x37, 0x35, 0xa5,
+	0x22, 0x35, 0xaa, 0x60, 0x27, 0x35, 0x31, 0x90, 0x9f, 0x79, 0x64, 0x26, 0x6b, 0xe0, 0xd1, 0x99,
+	0x17, 0x0f, 0xe4, 0x67, 0x1e, 0xbe, 0x85, 0x50, 0xcb, 0xef, 0x06, 0xb4, 0xc5, 0xc5, 0x1f, 0x7d,
+	0x56, 0xb6, 0xfc, 0x99, 0x6e, 0xd9, 0x8a, 0xd7, 0xa3, 0xce, 0x81, 0x16, 0x7c, 0x1b, 0x15, 0x5c,
+	0xa0, 0x21, 0x58, 0x6d, 0x46, 0x3d, 0x4e, 0xe6, 0xb2, 0x08, 0xfb, 0x42, 0xb0, 0x23, 0xd6, 0x63,
+	0x82, 0x1b, 0x97, 0xc4, 0x9e, 0x15, 0x81, 0x41, 0xdf, 0x3f, 0x05, 0x92, 0xcf, 0xda, 0xb3, 0x44,
+	0x98, 0x52, 0x10, 0xef, 0xd9, 0x4d, 0x6a, 0xe2, 0x58, 0xa8, 0x4b, 0x59, 0x97, 0xa0, 0xac, 0x63,
+	0x69, 0x88, 0xa5, 0xf8, 0x58, 0xa4, 0x10, 0x37, 0x50, 0x41, 0x26, 0x0e, 0x3c, 0x7a, 0xec, 0x02,
+	0xf9, 0x9a, 0xb9, 0xf7, 0x46, 0x8f, 0x77, 0xb6, 0xa5, 0x20, 0x76, 0x4e, 0xe3, 0x12, 0x6e, 0x22,
+	0x99, 0x4f, 0xcb, 0x76, 0x42, 0xc9, 0xf8, 0x3e, 0x9b, 0x65, 0x5d, 0x30, 0x9a, 0x4a, 0x11, 0x5b,
+	0xa7, 0x49, 0x0d, 0x1f, 0x28, 0x0a, 0x78, 0xdc, 0x69, 0x51, 0x0e, 0xe4, 0x87, 0xa2, 0xfc, 0x9b,
+	0xa6, 0x44, 0xb9, 0x6f, 0x0c, 0x48, 0x23, 0x5c, 0xaa, 0x1f, 0x6f, 0xeb, 0x4f, 0x49, 0x7c, 0x5b,
+	0x16, 0xb5, 0x6d, 0xf2, 0x7a, 0x6e, 0x9c, 0xad, 0xbb, 0x21, 0xb0, 0x86, 0x6d, 0xa7, 0x6c, 0xe9,
+	0x1a, 0x3e, 0x40, 0x0b, 0x09, 0x46, 0xc5, 0x8b, 0xbc, 0x51, 0xa4, 0xb5, 0x6c, 0x92, 0xce, 0xa5,
+	0x86, 0x95, 0x68, 0xaa, 0x9c, 0xb6, 0xd5, 0x06, 0x4e, 0xde, 0x5e, 0x69, 0x6b, 0x07, 0xf8, 0x88,
+	0xad, 0x1d, 0xe0, 0xb8, 0x8d, 0xfe, 0x48, 0x30, 0xad, 0x8e, 0x08, 0xbc, 0x15, 0xd0, 0x30, 0x7c,
+	0xec, 0x33, 0x9b, 0xbc, 0x53, 0xc8, 0xff, 0xb3, 0x91, 0x5b, 0x52, 0x7d, 0xa8, 0xc5, 0x11, 0xfd,
+	0x37, 0x9a, 0xb9, 0x8c, 0xef, 0xa1, 0xa5, 0x01, 0xbf, 0x22, 0xa9, 0x16, 0xf3, 0x5d, 0x20, 0x17,
+	0x6a, 0xc6, 0xdf, 0x63, 0x6c, 0xcb, 0x94, 0xfb, 0xc9, 0x51, 0xff, 0x4a, 0x87, 0x57, 0xf0, 0x7d,
+	0xb4, 0x9c, 0x90, 0x55, 0xe8, 0x15, 0xfa, 0xbd, 0x42, 0xff, 0x93, 0x8d, 0xd6, 0xe9, 0x1f, 0x60,
+	0x63, 0x3a, 0xb2, 0x84, 0x77, 0x51, 0x29, 0x81, 0xbb, 0x4e, 0xc8, 0xc9, 0x07, 0x45, 0x5d, 0xcd,
+	0xa6, 0xee, 0x3b, 0x21, 0x4f, 0xe5, 0x28, 0x2a, 0xc6, 0x24, 0x61, 0x4d, 0x91, 0x3e, 0x8e, 0x25,
+	0x89, 0xd1, 0x23, 0xa4, 0xa8, 0x18, 0x1f, 0xbd, 0x24, 0x89, 0x44, 0x3e, 0xcb, 0x8f, 0x3b, 0x7a,
+	0xd1, 0x33, 0x9c, 0x48, 0x5d, 0x8b, 0x13, 0x29, 0x31, 0x3a, 0x91, 0xcf, 0xf3, 0xe3, 0x12, 0x29,
+	0xba, 0x32, 0x12, 0x99, 0x94, 0xd3, 0xb6, 0x44, 0x22, 0x5f, 0x5c, 0x69, 0x6b, 0x38, 0x91, 0xba,
+	0x86, 0x1f, 0xa2, 0xf2, 0x00, 0x46, 0x06, 0x25, 0x00, 0xd6, 0x75, 0x42, 0x79, 0x8f, 0xbd, 0x54,
+	0xcc, 0x6b, 0x63, 0x98, 0x42, 0x7e, 0x18, 0xab, 0x23, 0xfe, 0xef, 0x34, 0x7b, 0x1d, 0x77, 0xd1,
+	0x4a, 0x32, 0x4b, 0x47, 0x67, 0x60, 0xd8, 0x2b, 0x35, 0xec, 0x7a, 0xf6, 0x30, 0x95, 0x92, 0xd1,
+	0x69, 0x84, 0x8e, 0x11, 0xd4, 0x7e, 0x41, 0xf3, 0xdb, 0xdd, 0x80, 0x3f, 0x31, 0x21, 0x0c, 0x7c,
+	0x2f, 0x84, 0x5a, 0x80, 0x56, 0xae, 0xf8, 0x47, 0x84, 0x31, 0x9a, 0x92, 0xb7, 0xbb, 0x21, 0x6f,
+	0x77, 0xf9, 0x2c, 0x6e, 0xfd, 0xf8, 0xfb, 0xd4, 0xb7, 0x7e, 0xf4, 0x8e, 0x57, 0x51, 0x31, 0x74,
+	0xba, 0x81, 0x0b, 0x16, 0xf7, 0x4f, 0x41, 0x5d, 0xfa, 0x79, 0xb3, 0xa0, 0x6a, 0x47, 0xa2, 0x74,
+	0x67, 0xe9, 0xfc, 0x73, 0x65, 0xe2, 0xfc, 0xb2, 0x62, 0x5c, 0x5c, 0x56, 0x8c, 0x4f, 0x97, 0x15,
+	0xe3, 0xe9, 0x97, 0xca, 0xc4, 0xf1, 0x8c, 0xfc, 0xc9, 0xb1, 0xf9, 0x33, 0x00, 0x00, 0xff, 0xff,
+	0xa0, 0xbb, 0x20, 0x2c, 0xca, 0x08, 0x00, 0x00,
+}
+
+func (m *RequestHeader) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *RequestHeader) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *RequestHeader) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.AuthRevision != 0 {
+		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRevision))
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.Username) > 0 {
+		i -= len(m.Username)
+		copy(dAtA[i:], m.Username)
+		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Username)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.ID != 0 {
+		i = encodeVarintRaftInternal(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *InternalRaftRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *InternalRaftRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *InternalRaftRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.AuthRoleRevokePermission != nil {
+		{
+			size, err := m.AuthRoleRevokePermission.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4b
+		i--
+		dAtA[i] = 0xa2
+	}
+	if m.AuthRoleGrantPermission != nil {
+		{
+			size, err := m.AuthRoleGrantPermission.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4b
+		i--
+		dAtA[i] = 0x9a
+	}
+	if m.AuthRoleGet != nil {
+		{
+			size, err := m.AuthRoleGet.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4b
+		i--
+		dAtA[i] = 0x92
+	}
+	if m.AuthRoleDelete != nil {
+		{
+			size, err := m.AuthRoleDelete.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4b
+		i--
+		dAtA[i] = 0x8a
+	}
+	if m.AuthRoleAdd != nil {
+		{
+			size, err := m.AuthRoleAdd.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4b
+		i--
+		dAtA[i] = 0x82
+	}
+	if m.AuthRoleList != nil {
+		{
+			size, err := m.AuthRoleList.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x45
+		i--
+		dAtA[i] = 0x9a
+	}
+	if m.AuthUserList != nil {
+		{
+			size, err := m.AuthUserList.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x45
+		i--
+		dAtA[i] = 0x92
+	}
+	if m.AuthUserRevokeRole != nil {
+		{
+			size, err := m.AuthUserRevokeRole.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x45
+		i--
+		dAtA[i] = 0x8a
+	}
+	if m.AuthUserGrantRole != nil {
+		{
+			size, err := m.AuthUserGrantRole.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x45
+		i--
+		dAtA[i] = 0x82
+	}
+	if m.AuthUserChangePassword != nil {
+		{
+			size, err := m.AuthUserChangePassword.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x44
+		i--
+		dAtA[i] = 0xfa
+	}
+	if m.AuthUserGet != nil {
+		{
+			size, err := m.AuthUserGet.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x44
+		i--
+		dAtA[i] = 0xf2
+	}
+	if m.AuthUserDelete != nil {
+		{
+			size, err := m.AuthUserDelete.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x44
+		i--
+		dAtA[i] = 0xea
+	}
+	if m.AuthUserAdd != nil {
+		{
+			size, err := m.AuthUserAdd.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x44
+		i--
+		dAtA[i] = 0xe2
+	}
+	if m.Authenticate != nil {
+		{
+			size, err := m.Authenticate.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x3f
+		i--
+		dAtA[i] = 0xa2
+	}
+	if m.AuthDisable != nil {
+		{
+			size, err := m.AuthDisable.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x3f
+		i--
+		dAtA[i] = 0x9a
+	}
+	if m.AuthEnable != nil {
+		{
+			size, err := m.AuthEnable.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x3e
+		i--
+		dAtA[i] = 0xc2
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x6
+		i--
+		dAtA[i] = 0xa2
+	}
+	if m.Alarm != nil {
+		{
+			size, err := m.Alarm.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x52
+	}
+	if m.LeaseRevoke != nil {
+		{
+			size, err := m.LeaseRevoke.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4a
+	}
+	if m.LeaseGrant != nil {
+		{
+			size, err := m.LeaseGrant.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x42
+	}
+	if m.Compaction != nil {
+		{
+			size, err := m.Compaction.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x3a
+	}
+	if m.Txn != nil {
+		{
+			size, err := m.Txn.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x32
+	}
+	if m.DeleteRange != nil {
+		{
+			size, err := m.DeleteRange.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x2a
+	}
+	if m.Put != nil {
+		{
+			size, err := m.Put.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x22
+	}
+	if m.Range != nil {
+		{
+			size, err := m.Range.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.V2 != nil {
+		{
+			size, err := m.V2.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRaftInternal(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.ID != 0 {
+		i = encodeVarintRaftInternal(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *EmptyResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *EmptyResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *EmptyResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *InternalAuthenticateRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *InternalAuthenticateRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *InternalAuthenticateRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.SimpleToken) > 0 {
+		i -= len(m.SimpleToken)
+		copy(dAtA[i:], m.SimpleToken)
+		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.SimpleToken)))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if len(m.Password) > 0 {
+		i -= len(m.Password)
+		copy(dAtA[i:], m.Password)
+		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Password)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintRaftInternal(dAtA []byte, offset int, v uint64) int {
+	offset -= sovRaftInternal(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *RequestHeader) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRaftInternal(uint64(m.ID))
+	}
+	l = len(m.Username)
+	if l > 0 {
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRevision != 0 {
+		n += 1 + sovRaftInternal(uint64(m.AuthRevision))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *InternalRaftRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRaftInternal(uint64(m.ID))
+	}
+	if m.V2 != nil {
+		l = m.V2.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Range != nil {
+		l = m.Range.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Put != nil {
+		l = m.Put.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.DeleteRange != nil {
+		l = m.DeleteRange.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Txn != nil {
+		l = m.Txn.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Compaction != nil {
+		l = m.Compaction.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.LeaseGrant != nil {
+		l = m.LeaseGrant.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.LeaseRevoke != nil {
+		l = m.LeaseRevoke.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Alarm != nil {
+		l = m.Alarm.Size()
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthEnable != nil {
+		l = m.AuthEnable.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthDisable != nil {
+		l = m.AuthDisable.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.Authenticate != nil {
+		l = m.Authenticate.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserAdd != nil {
+		l = m.AuthUserAdd.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserDelete != nil {
+		l = m.AuthUserDelete.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserGet != nil {
+		l = m.AuthUserGet.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserChangePassword != nil {
+		l = m.AuthUserChangePassword.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserGrantRole != nil {
+		l = m.AuthUserGrantRole.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserRevokeRole != nil {
+		l = m.AuthUserRevokeRole.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthUserList != nil {
+		l = m.AuthUserList.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleList != nil {
+		l = m.AuthRoleList.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleAdd != nil {
+		l = m.AuthRoleAdd.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleDelete != nil {
+		l = m.AuthRoleDelete.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleGet != nil {
+		l = m.AuthRoleGet.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleGrantPermission != nil {
+		l = m.AuthRoleGrantPermission.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.AuthRoleRevokePermission != nil {
+		l = m.AuthRoleRevokePermission.Size()
+		n += 2 + l + sovRaftInternal(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *EmptyResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *InternalAuthenticateRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	l = len(m.Password)
+	if l > 0 {
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	l = len(m.SimpleToken)
+	if l > 0 {
+		n += 1 + l + sovRaftInternal(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovRaftInternal(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozRaftInternal(x uint64) (n int) {
+	return sovRaftInternal(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *RequestHeader) 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 ErrIntOverflowRaftInternal
+			}
+			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: RequestHeader: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: RequestHeader: 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 ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Username", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Username = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRevision", wireType)
+			}
+			m.AuthRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.AuthRevision |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaftInternal(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			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 *InternalRaftRequest) 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 ErrIntOverflowRaftInternal
+			}
+			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: InternalRaftRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: InternalRaftRequest: 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 ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field V2", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.V2 == nil {
+				m.V2 = &Request{}
+			}
+			if err := m.V2.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Range == nil {
+				m.Range = &RangeRequest{}
+			}
+			if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Put", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Put == nil {
+				m.Put = &PutRequest{}
+			}
+			if err := m.Put.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field DeleteRange", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.DeleteRange == nil {
+				m.DeleteRange = &DeleteRangeRequest{}
+			}
+			if err := m.DeleteRange.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Txn", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Txn == nil {
+				m.Txn = &TxnRequest{}
+			}
+			if err := m.Txn.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 7:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Compaction", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Compaction == nil {
+				m.Compaction = &CompactionRequest{}
+			}
+			if err := m.Compaction.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 8:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeaseGrant", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.LeaseGrant == nil {
+				m.LeaseGrant = &LeaseGrantRequest{}
+			}
+			if err := m.LeaseGrant.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 9:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeaseRevoke", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.LeaseRevoke == nil {
+				m.LeaseRevoke = &LeaseRevokeRequest{}
+			}
+			if err := m.LeaseRevoke.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 10:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Alarm == nil {
+				m.Alarm = &AlarmRequest{}
+			}
+			if err := m.Alarm.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 100:
+			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 ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &RequestHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1000:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthEnable", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthEnable == nil {
+				m.AuthEnable = &AuthEnableRequest{}
+			}
+			if err := m.AuthEnable.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1011:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthDisable", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthDisable == nil {
+				m.AuthDisable = &AuthDisableRequest{}
+			}
+			if err := m.AuthDisable.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1012:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Authenticate", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Authenticate == nil {
+				m.Authenticate = &InternalAuthenticateRequest{}
+			}
+			if err := m.Authenticate.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1100:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserAdd", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserAdd == nil {
+				m.AuthUserAdd = &AuthUserAddRequest{}
+			}
+			if err := m.AuthUserAdd.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1101:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserDelete", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserDelete == nil {
+				m.AuthUserDelete = &AuthUserDeleteRequest{}
+			}
+			if err := m.AuthUserDelete.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1102:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserGet", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserGet == nil {
+				m.AuthUserGet = &AuthUserGetRequest{}
+			}
+			if err := m.AuthUserGet.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1103:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserChangePassword", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserChangePassword == nil {
+				m.AuthUserChangePassword = &AuthUserChangePasswordRequest{}
+			}
+			if err := m.AuthUserChangePassword.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1104:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserGrantRole", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserGrantRole == nil {
+				m.AuthUserGrantRole = &AuthUserGrantRoleRequest{}
+			}
+			if err := m.AuthUserGrantRole.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1105:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserRevokeRole", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserRevokeRole == nil {
+				m.AuthUserRevokeRole = &AuthUserRevokeRoleRequest{}
+			}
+			if err := m.AuthUserRevokeRole.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1106:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserList", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthUserList == nil {
+				m.AuthUserList = &AuthUserListRequest{}
+			}
+			if err := m.AuthUserList.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1107:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleList", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleList == nil {
+				m.AuthRoleList = &AuthRoleListRequest{}
+			}
+			if err := m.AuthRoleList.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1200:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleAdd", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleAdd == nil {
+				m.AuthRoleAdd = &AuthRoleAddRequest{}
+			}
+			if err := m.AuthRoleAdd.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1201:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleDelete", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleDelete == nil {
+				m.AuthRoleDelete = &AuthRoleDeleteRequest{}
+			}
+			if err := m.AuthRoleDelete.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1202:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleGet", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleGet == nil {
+				m.AuthRoleGet = &AuthRoleGetRequest{}
+			}
+			if err := m.AuthRoleGet.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1203:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleGrantPermission", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleGrantPermission == nil {
+				m.AuthRoleGrantPermission = &AuthRoleGrantPermissionRequest{}
+			}
+			if err := m.AuthRoleGrantPermission.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 1204:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleRevokePermission", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.AuthRoleRevokePermission == nil {
+				m.AuthRoleRevokePermission = &AuthRoleRevokePermissionRequest{}
+			}
+			if err := m.AuthRoleRevokePermission.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaftInternal(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			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 *EmptyResponse) 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 ErrIntOverflowRaftInternal
+			}
+			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: EmptyResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: EmptyResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaftInternal(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			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 *InternalAuthenticateRequest) 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 ErrIntOverflowRaftInternal
+			}
+			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: InternalAuthenticateRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: InternalAuthenticateRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Password = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field SimpleToken", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.SimpleToken = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaftInternal(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaftInternal
+			}
+			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 skipRaftInternal(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, ErrIntOverflowRaftInternal
+			}
+			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, ErrIntOverflowRaftInternal
+				}
+				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, ErrIntOverflowRaftInternal
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthRaftInternal
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthRaftInternal
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowRaftInternal
+					}
+					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 := skipRaftInternal(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthRaftInternal
+				}
+			}
+			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 (
+	ErrInvalidLengthRaftInternal = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowRaftInternal   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.proto b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.proto
new file mode 100644
index 0000000..25d45d3
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.proto
@@ -0,0 +1,74 @@
+syntax = "proto3";
+package etcdserverpb;
+
+import "gogoproto/gogo.proto";
+import "etcdserver.proto";
+import "rpc.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.sizer_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+option (gogoproto.goproto_getters_all) = false;
+
+message RequestHeader {
+  uint64 ID = 1;
+  // username is a username that is associated with an auth token of gRPC connection
+  string username = 2;
+  // auth_revision is a revision number of auth.authStore. It is not related to mvcc
+  uint64 auth_revision = 3;
+}
+
+// An InternalRaftRequest is the union of all requests which can be
+// sent via raft.
+message InternalRaftRequest {
+  RequestHeader header = 100;
+  uint64 ID = 1;
+
+  Request v2 = 2;
+
+  RangeRequest range = 3;
+  PutRequest put = 4;
+  DeleteRangeRequest delete_range = 5;
+  TxnRequest txn = 6;
+  CompactionRequest compaction = 7;
+
+  LeaseGrantRequest lease_grant = 8;
+  LeaseRevokeRequest lease_revoke = 9;
+
+  AlarmRequest alarm = 10;
+
+  AuthEnableRequest auth_enable = 1000;
+  AuthDisableRequest auth_disable = 1011;
+
+  InternalAuthenticateRequest authenticate = 1012;
+
+  AuthUserAddRequest auth_user_add = 1100;
+  AuthUserDeleteRequest auth_user_delete = 1101;
+  AuthUserGetRequest auth_user_get = 1102;
+  AuthUserChangePasswordRequest auth_user_change_password = 1103;
+  AuthUserGrantRoleRequest auth_user_grant_role = 1104;
+  AuthUserRevokeRoleRequest auth_user_revoke_role = 1105;
+  AuthUserListRequest auth_user_list = 1106;
+  AuthRoleListRequest auth_role_list = 1107;
+
+  AuthRoleAddRequest auth_role_add = 1200;
+  AuthRoleDeleteRequest auth_role_delete = 1201;
+  AuthRoleGetRequest auth_role_get = 1202;
+  AuthRoleGrantPermissionRequest auth_role_grant_permission = 1203;
+  AuthRoleRevokePermissionRequest auth_role_revoke_permission = 1204;
+}
+
+message EmptyResponse {
+}
+
+// What is the difference between AuthenticateRequest (defined in rpc.proto) and InternalAuthenticateRequest?
+// InternalAuthenticateRequest has a member that is filled by etcdserver and shouldn't be user-facing.
+// For avoiding misusage the field, we have an internal version of AuthenticateRequest.
+message InternalAuthenticateRequest {
+  string name = 1;
+  string password = 2;
+
+  // simple_token is generated in API layer (etcdserver/v3_server.go)
+  string simple_token = 3;
+}
+
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go
new file mode 100644
index 0000000..31e121e
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go
@@ -0,0 +1,183 @@
+// 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 etcdserverpb
+
+import (
+	"fmt"
+	"strings"
+
+	proto "github.com/golang/protobuf/proto"
+)
+
+// InternalRaftStringer implements custom proto Stringer:
+// redact password, replace value fields with value_size fields.
+type InternalRaftStringer struct {
+	Request *InternalRaftRequest
+}
+
+func (as *InternalRaftStringer) String() string {
+	switch {
+	case as.Request.LeaseGrant != nil:
+		return fmt.Sprintf("header:<%s> lease_grant:<ttl:%d-second id:%016x>",
+			as.Request.Header.String(),
+			as.Request.LeaseGrant.TTL,
+			as.Request.LeaseGrant.ID,
+		)
+	case as.Request.LeaseRevoke != nil:
+		return fmt.Sprintf("header:<%s> lease_revoke:<id:%016x>",
+			as.Request.Header.String(),
+			as.Request.LeaseRevoke.ID,
+		)
+	case as.Request.Authenticate != nil:
+		return fmt.Sprintf("header:<%s> authenticate:<name:%s simple_token:%s>",
+			as.Request.Header.String(),
+			as.Request.Authenticate.Name,
+			as.Request.Authenticate.SimpleToken,
+		)
+	case as.Request.AuthUserAdd != nil:
+		return fmt.Sprintf("header:<%s> auth_user_add:<name:%s>",
+			as.Request.Header.String(),
+			as.Request.AuthUserAdd.Name,
+		)
+	case as.Request.AuthUserChangePassword != nil:
+		return fmt.Sprintf("header:<%s> auth_user_change_password:<name:%s>",
+			as.Request.Header.String(),
+			as.Request.AuthUserChangePassword.Name,
+		)
+	case as.Request.Put != nil:
+		return fmt.Sprintf("header:<%s> put:<%s>",
+			as.Request.Header.String(),
+			NewLoggablePutRequest(as.Request.Put).String(),
+		)
+	case as.Request.Txn != nil:
+		return fmt.Sprintf("header:<%s> txn:<%s>",
+			as.Request.Header.String(),
+			NewLoggableTxnRequest(as.Request.Txn).String(),
+		)
+	default:
+		// nothing to redact
+	}
+	return as.Request.String()
+}
+
+// txnRequestStringer implements a custom proto String to replace value bytes fields with value size
+// fields in any nested txn and put operations.
+type txnRequestStringer struct {
+	Request *TxnRequest
+}
+
+func NewLoggableTxnRequest(request *TxnRequest) *txnRequestStringer {
+	return &txnRequestStringer{request}
+}
+
+func (as *txnRequestStringer) String() string {
+	var compare []string
+	for _, c := range as.Request.Compare {
+		switch cv := c.TargetUnion.(type) {
+		case *Compare_Value:
+			compare = append(compare, newLoggableValueCompare(c, cv).String())
+		default:
+			// nothing to redact
+			compare = append(compare, c.String())
+		}
+	}
+	var success []string
+	for _, s := range as.Request.Success {
+		success = append(success, newLoggableRequestOp(s).String())
+	}
+	var failure []string
+	for _, f := range as.Request.Failure {
+		failure = append(failure, newLoggableRequestOp(f).String())
+	}
+	return fmt.Sprintf("compare:<%s> success:<%s> failure:<%s>",
+		strings.Join(compare, " "),
+		strings.Join(success, " "),
+		strings.Join(failure, " "),
+	)
+}
+
+// requestOpStringer implements a custom proto String to replace value bytes fields with value
+// size fields in any nested txn and put operations.
+type requestOpStringer struct {
+	Op *RequestOp
+}
+
+func newLoggableRequestOp(op *RequestOp) *requestOpStringer {
+	return &requestOpStringer{op}
+}
+
+func (as *requestOpStringer) String() string {
+	switch op := as.Op.Request.(type) {
+	case *RequestOp_RequestPut:
+		return fmt.Sprintf("request_put:<%s>", NewLoggablePutRequest(op.RequestPut).String())
+	case *RequestOp_RequestTxn:
+		return fmt.Sprintf("request_txn:<%s>", NewLoggableTxnRequest(op.RequestTxn).String())
+	default:
+		// nothing to redact
+	}
+	return as.Op.String()
+}
+
+// loggableValueCompare implements a custom proto String for Compare.Value union member types to
+// replace the value bytes field with a value size field.
+// To preserve proto encoding of the key and range_end bytes, a faked out proto type is used here.
+type loggableValueCompare struct {
+	Result    Compare_CompareResult `protobuf:"varint,1,opt,name=result,proto3,enum=etcdserverpb.Compare_CompareResult"`
+	Target    Compare_CompareTarget `protobuf:"varint,2,opt,name=target,proto3,enum=etcdserverpb.Compare_CompareTarget"`
+	Key       []byte                `protobuf:"bytes,3,opt,name=key,proto3"`
+	ValueSize int64                 `protobuf:"varint,7,opt,name=value_size,proto3"`
+	RangeEnd  []byte                `protobuf:"bytes,64,opt,name=range_end,proto3"`
+}
+
+func newLoggableValueCompare(c *Compare, cv *Compare_Value) *loggableValueCompare {
+	return &loggableValueCompare{
+		c.Result,
+		c.Target,
+		c.Key,
+		int64(len(cv.Value)),
+		c.RangeEnd,
+	}
+}
+
+func (m *loggableValueCompare) Reset()         { *m = loggableValueCompare{} }
+func (m *loggableValueCompare) String() string { return proto.CompactTextString(m) }
+func (*loggableValueCompare) ProtoMessage()    {}
+
+// loggablePutRequest implements a custom proto String to replace value bytes field with a value
+// size field.
+// To preserve proto encoding of the key bytes, a faked out proto type is used here.
+type loggablePutRequest struct {
+	Key         []byte `protobuf:"bytes,1,opt,name=key,proto3"`
+	ValueSize   int64  `protobuf:"varint,2,opt,name=value_size,proto3"`
+	Lease       int64  `protobuf:"varint,3,opt,name=lease,proto3"`
+	PrevKv      bool   `protobuf:"varint,4,opt,name=prev_kv,proto3"`
+	IgnoreValue bool   `protobuf:"varint,5,opt,name=ignore_value,proto3"`
+	IgnoreLease bool   `protobuf:"varint,6,opt,name=ignore_lease,proto3"`
+}
+
+func NewLoggablePutRequest(request *PutRequest) *loggablePutRequest {
+	return &loggablePutRequest{
+		request.Key,
+		int64(len(request.Value)),
+		request.Lease,
+		request.PrevKv,
+		request.IgnoreValue,
+		request.IgnoreLease,
+	}
+}
+
+func (m *loggablePutRequest) Reset()         { *m = loggablePutRequest{} }
+func (m *loggablePutRequest) String() string { return proto.CompactTextString(m) }
+func (*loggablePutRequest) ProtoMessage()    {}
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.pb.go b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.pb.go
new file mode 100644
index 0000000..a0cff8f
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.pb.go
@@ -0,0 +1,24010 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: rpc.proto
+
+package etcdserverpb
+
+import (
+	context "context"
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	authpb "github.com/coreos/etcd/auth/authpb"
+	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
+	_ "github.com/gogo/protobuf/gogoproto"
+	proto "github.com/golang/protobuf/proto"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	grpc "google.golang.org/grpc"
+	codes "google.golang.org/grpc/codes"
+	status "google.golang.org/grpc/status"
+)
+
+// 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 AlarmType int32
+
+const (
+	AlarmType_NONE    AlarmType = 0
+	AlarmType_NOSPACE AlarmType = 1
+	AlarmType_CORRUPT AlarmType = 2
+)
+
+var AlarmType_name = map[int32]string{
+	0: "NONE",
+	1: "NOSPACE",
+	2: "CORRUPT",
+}
+
+var AlarmType_value = map[string]int32{
+	"NONE":    0,
+	"NOSPACE": 1,
+	"CORRUPT": 2,
+}
+
+func (x AlarmType) String() string {
+	return proto.EnumName(AlarmType_name, int32(x))
+}
+
+func (AlarmType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{0}
+}
+
+type RangeRequest_SortOrder int32
+
+const (
+	RangeRequest_NONE    RangeRequest_SortOrder = 0
+	RangeRequest_ASCEND  RangeRequest_SortOrder = 1
+	RangeRequest_DESCEND RangeRequest_SortOrder = 2
+)
+
+var RangeRequest_SortOrder_name = map[int32]string{
+	0: "NONE",
+	1: "ASCEND",
+	2: "DESCEND",
+}
+
+var RangeRequest_SortOrder_value = map[string]int32{
+	"NONE":    0,
+	"ASCEND":  1,
+	"DESCEND": 2,
+}
+
+func (x RangeRequest_SortOrder) String() string {
+	return proto.EnumName(RangeRequest_SortOrder_name, int32(x))
+}
+
+func (RangeRequest_SortOrder) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{1, 0}
+}
+
+type RangeRequest_SortTarget int32
+
+const (
+	RangeRequest_KEY     RangeRequest_SortTarget = 0
+	RangeRequest_VERSION RangeRequest_SortTarget = 1
+	RangeRequest_CREATE  RangeRequest_SortTarget = 2
+	RangeRequest_MOD     RangeRequest_SortTarget = 3
+	RangeRequest_VALUE   RangeRequest_SortTarget = 4
+)
+
+var RangeRequest_SortTarget_name = map[int32]string{
+	0: "KEY",
+	1: "VERSION",
+	2: "CREATE",
+	3: "MOD",
+	4: "VALUE",
+}
+
+var RangeRequest_SortTarget_value = map[string]int32{
+	"KEY":     0,
+	"VERSION": 1,
+	"CREATE":  2,
+	"MOD":     3,
+	"VALUE":   4,
+}
+
+func (x RangeRequest_SortTarget) String() string {
+	return proto.EnumName(RangeRequest_SortTarget_name, int32(x))
+}
+
+func (RangeRequest_SortTarget) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{1, 1}
+}
+
+type Compare_CompareResult int32
+
+const (
+	Compare_EQUAL     Compare_CompareResult = 0
+	Compare_GREATER   Compare_CompareResult = 1
+	Compare_LESS      Compare_CompareResult = 2
+	Compare_NOT_EQUAL Compare_CompareResult = 3
+)
+
+var Compare_CompareResult_name = map[int32]string{
+	0: "EQUAL",
+	1: "GREATER",
+	2: "LESS",
+	3: "NOT_EQUAL",
+}
+
+var Compare_CompareResult_value = map[string]int32{
+	"EQUAL":     0,
+	"GREATER":   1,
+	"LESS":      2,
+	"NOT_EQUAL": 3,
+}
+
+func (x Compare_CompareResult) String() string {
+	return proto.EnumName(Compare_CompareResult_name, int32(x))
+}
+
+func (Compare_CompareResult) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{9, 0}
+}
+
+type Compare_CompareTarget int32
+
+const (
+	Compare_VERSION Compare_CompareTarget = 0
+	Compare_CREATE  Compare_CompareTarget = 1
+	Compare_MOD     Compare_CompareTarget = 2
+	Compare_VALUE   Compare_CompareTarget = 3
+	Compare_LEASE   Compare_CompareTarget = 4
+)
+
+var Compare_CompareTarget_name = map[int32]string{
+	0: "VERSION",
+	1: "CREATE",
+	2: "MOD",
+	3: "VALUE",
+	4: "LEASE",
+}
+
+var Compare_CompareTarget_value = map[string]int32{
+	"VERSION": 0,
+	"CREATE":  1,
+	"MOD":     2,
+	"VALUE":   3,
+	"LEASE":   4,
+}
+
+func (x Compare_CompareTarget) String() string {
+	return proto.EnumName(Compare_CompareTarget_name, int32(x))
+}
+
+func (Compare_CompareTarget) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{9, 1}
+}
+
+type WatchCreateRequest_FilterType int32
+
+const (
+	// filter out put event.
+	WatchCreateRequest_NOPUT WatchCreateRequest_FilterType = 0
+	// filter out delete event.
+	WatchCreateRequest_NODELETE WatchCreateRequest_FilterType = 1
+)
+
+var WatchCreateRequest_FilterType_name = map[int32]string{
+	0: "NOPUT",
+	1: "NODELETE",
+}
+
+var WatchCreateRequest_FilterType_value = map[string]int32{
+	"NOPUT":    0,
+	"NODELETE": 1,
+}
+
+func (x WatchCreateRequest_FilterType) String() string {
+	return proto.EnumName(WatchCreateRequest_FilterType_name, int32(x))
+}
+
+func (WatchCreateRequest_FilterType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{21, 0}
+}
+
+type AlarmRequest_AlarmAction int32
+
+const (
+	AlarmRequest_GET        AlarmRequest_AlarmAction = 0
+	AlarmRequest_ACTIVATE   AlarmRequest_AlarmAction = 1
+	AlarmRequest_DEACTIVATE AlarmRequest_AlarmAction = 2
+)
+
+var AlarmRequest_AlarmAction_name = map[int32]string{
+	0: "GET",
+	1: "ACTIVATE",
+	2: "DEACTIVATE",
+}
+
+var AlarmRequest_AlarmAction_value = map[string]int32{
+	"GET":        0,
+	"ACTIVATE":   1,
+	"DEACTIVATE": 2,
+}
+
+func (x AlarmRequest_AlarmAction) String() string {
+	return proto.EnumName(AlarmRequest_AlarmAction_name, int32(x))
+}
+
+func (AlarmRequest_AlarmAction) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{49, 0}
+}
+
+type ResponseHeader struct {
+	// cluster_id is the ID of the cluster which sent the response.
+	ClusterId uint64 `protobuf:"varint,1,opt,name=cluster_id,json=clusterId,proto3" json:"cluster_id,omitempty"`
+	// member_id is the ID of the member which sent the response.
+	MemberId uint64 `protobuf:"varint,2,opt,name=member_id,json=memberId,proto3" json:"member_id,omitempty"`
+	// revision is the key-value store revision when the request was applied.
+	// For watch progress responses, the header.revision indicates progress. All future events
+	// recieved in this stream are guaranteed to have a higher revision number than the
+	// header.revision number.
+	Revision int64 `protobuf:"varint,3,opt,name=revision,proto3" json:"revision,omitempty"`
+	// raft_term is the raft term when the request was applied.
+	RaftTerm             uint64   `protobuf:"varint,4,opt,name=raft_term,json=raftTerm,proto3" json:"raft_term,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ResponseHeader) Reset()         { *m = ResponseHeader{} }
+func (m *ResponseHeader) String() string { return proto.CompactTextString(m) }
+func (*ResponseHeader) ProtoMessage()    {}
+func (*ResponseHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{0}
+}
+func (m *ResponseHeader) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *ResponseHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_ResponseHeader.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *ResponseHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ResponseHeader.Merge(m, src)
+}
+func (m *ResponseHeader) XXX_Size() int {
+	return m.Size()
+}
+func (m *ResponseHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_ResponseHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ResponseHeader proto.InternalMessageInfo
+
+func (m *ResponseHeader) GetClusterId() uint64 {
+	if m != nil {
+		return m.ClusterId
+	}
+	return 0
+}
+
+func (m *ResponseHeader) GetMemberId() uint64 {
+	if m != nil {
+		return m.MemberId
+	}
+	return 0
+}
+
+func (m *ResponseHeader) GetRevision() int64 {
+	if m != nil {
+		return m.Revision
+	}
+	return 0
+}
+
+func (m *ResponseHeader) GetRaftTerm() uint64 {
+	if m != nil {
+		return m.RaftTerm
+	}
+	return 0
+}
+
+type RangeRequest struct {
+	// key is the first key for the range. If range_end is not given, the request only looks up key.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	// range_end is the upper bound on the requested range [key, range_end).
+	// If range_end is '\0', the range is all keys >= key.
+	// If range_end is key plus one (e.g., "aa"+1 == "ab", "a\xff"+1 == "b"),
+	// then the range request gets all keys prefixed with key.
+	// If both key and range_end are '\0', then the range request returns all keys.
+	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	// limit is a limit on the number of keys returned for the request. When limit is set to 0,
+	// it is treated as no limit.
+	Limit int64 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"`
+	// revision is the point-in-time of the key-value store to use for the range.
+	// If revision is less or equal to zero, the range is over the newest key-value store.
+	// If the revision has been compacted, ErrCompacted is returned as a response.
+	Revision int64 `protobuf:"varint,4,opt,name=revision,proto3" json:"revision,omitempty"`
+	// sort_order is the order for returned sorted results.
+	SortOrder RangeRequest_SortOrder `protobuf:"varint,5,opt,name=sort_order,json=sortOrder,proto3,enum=etcdserverpb.RangeRequest_SortOrder" json:"sort_order,omitempty"`
+	// sort_target is the key-value field to use for sorting.
+	SortTarget RangeRequest_SortTarget `protobuf:"varint,6,opt,name=sort_target,json=sortTarget,proto3,enum=etcdserverpb.RangeRequest_SortTarget" json:"sort_target,omitempty"`
+	// serializable sets the range request to use serializable member-local reads.
+	// Range requests are linearizable by default; linearizable requests have higher
+	// latency and lower throughput than serializable requests but reflect the current
+	// consensus of the cluster. For better performance, in exchange for possible stale reads,
+	// a serializable range request is served locally without needing to reach consensus
+	// with other nodes in the cluster.
+	Serializable bool `protobuf:"varint,7,opt,name=serializable,proto3" json:"serializable,omitempty"`
+	// keys_only when set returns only the keys and not the values.
+	KeysOnly bool `protobuf:"varint,8,opt,name=keys_only,json=keysOnly,proto3" json:"keys_only,omitempty"`
+	// count_only when set returns only the count of the keys in the range.
+	CountOnly bool `protobuf:"varint,9,opt,name=count_only,json=countOnly,proto3" json:"count_only,omitempty"`
+	// min_mod_revision is the lower bound for returned key mod revisions; all keys with
+	// lesser mod revisions will be filtered away.
+	MinModRevision int64 `protobuf:"varint,10,opt,name=min_mod_revision,json=minModRevision,proto3" json:"min_mod_revision,omitempty"`
+	// max_mod_revision is the upper bound for returned key mod revisions; all keys with
+	// greater mod revisions will be filtered away.
+	MaxModRevision int64 `protobuf:"varint,11,opt,name=max_mod_revision,json=maxModRevision,proto3" json:"max_mod_revision,omitempty"`
+	// min_create_revision is the lower bound for returned key create revisions; all keys with
+	// lesser create trevisions will be filtered away.
+	MinCreateRevision int64 `protobuf:"varint,12,opt,name=min_create_revision,json=minCreateRevision,proto3" json:"min_create_revision,omitempty"`
+	// max_create_revision is the upper bound for returned key create revisions; all keys with
+	// greater create revisions will be filtered away.
+	MaxCreateRevision    int64    `protobuf:"varint,13,opt,name=max_create_revision,json=maxCreateRevision,proto3" json:"max_create_revision,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *RangeRequest) Reset()         { *m = RangeRequest{} }
+func (m *RangeRequest) String() string { return proto.CompactTextString(m) }
+func (*RangeRequest) ProtoMessage()    {}
+func (*RangeRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{1}
+}
+func (m *RangeRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *RangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_RangeRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *RangeRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RangeRequest.Merge(m, src)
+}
+func (m *RangeRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *RangeRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_RangeRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RangeRequest proto.InternalMessageInfo
+
+func (m *RangeRequest) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *RangeRequest) GetRangeEnd() []byte {
+	if m != nil {
+		return m.RangeEnd
+	}
+	return nil
+}
+
+func (m *RangeRequest) GetLimit() int64 {
+	if m != nil {
+		return m.Limit
+	}
+	return 0
+}
+
+func (m *RangeRequest) GetRevision() int64 {
+	if m != nil {
+		return m.Revision
+	}
+	return 0
+}
+
+func (m *RangeRequest) GetSortOrder() RangeRequest_SortOrder {
+	if m != nil {
+		return m.SortOrder
+	}
+	return RangeRequest_NONE
+}
+
+func (m *RangeRequest) GetSortTarget() RangeRequest_SortTarget {
+	if m != nil {
+		return m.SortTarget
+	}
+	return RangeRequest_KEY
+}
+
+func (m *RangeRequest) GetSerializable() bool {
+	if m != nil {
+		return m.Serializable
+	}
+	return false
+}
+
+func (m *RangeRequest) GetKeysOnly() bool {
+	if m != nil {
+		return m.KeysOnly
+	}
+	return false
+}
+
+func (m *RangeRequest) GetCountOnly() bool {
+	if m != nil {
+		return m.CountOnly
+	}
+	return false
+}
+
+func (m *RangeRequest) GetMinModRevision() int64 {
+	if m != nil {
+		return m.MinModRevision
+	}
+	return 0
+}
+
+func (m *RangeRequest) GetMaxModRevision() int64 {
+	if m != nil {
+		return m.MaxModRevision
+	}
+	return 0
+}
+
+func (m *RangeRequest) GetMinCreateRevision() int64 {
+	if m != nil {
+		return m.MinCreateRevision
+	}
+	return 0
+}
+
+func (m *RangeRequest) GetMaxCreateRevision() int64 {
+	if m != nil {
+		return m.MaxCreateRevision
+	}
+	return 0
+}
+
+type RangeResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// kvs is the list of key-value pairs matched by the range request.
+	// kvs is empty when count is requested.
+	Kvs []*mvccpb.KeyValue `protobuf:"bytes,2,rep,name=kvs,proto3" json:"kvs,omitempty"`
+	// more indicates if there are more keys to return in the requested range.
+	More bool `protobuf:"varint,3,opt,name=more,proto3" json:"more,omitempty"`
+	// count is set to the number of keys within the range when requested.
+	Count                int64    `protobuf:"varint,4,opt,name=count,proto3" json:"count,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *RangeResponse) Reset()         { *m = RangeResponse{} }
+func (m *RangeResponse) String() string { return proto.CompactTextString(m) }
+func (*RangeResponse) ProtoMessage()    {}
+func (*RangeResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{2}
+}
+func (m *RangeResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *RangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_RangeResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *RangeResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RangeResponse.Merge(m, src)
+}
+func (m *RangeResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *RangeResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_RangeResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RangeResponse proto.InternalMessageInfo
+
+func (m *RangeResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *RangeResponse) GetKvs() []*mvccpb.KeyValue {
+	if m != nil {
+		return m.Kvs
+	}
+	return nil
+}
+
+func (m *RangeResponse) GetMore() bool {
+	if m != nil {
+		return m.More
+	}
+	return false
+}
+
+func (m *RangeResponse) GetCount() int64 {
+	if m != nil {
+		return m.Count
+	}
+	return 0
+}
+
+type PutRequest struct {
+	// key is the key, in bytes, to put into the key-value store.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	// value is the value, in bytes, to associate with the key in the key-value store.
+	Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+	// lease is the lease ID to associate with the key in the key-value store. A lease
+	// value of 0 indicates no lease.
+	Lease int64 `protobuf:"varint,3,opt,name=lease,proto3" json:"lease,omitempty"`
+	// If prev_kv is set, etcd gets the previous key-value pair before changing it.
+	// The previous key-value pair will be returned in the put response.
+	PrevKv bool `protobuf:"varint,4,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
+	// If ignore_value is set, etcd updates the key using its current value.
+	// Returns an error if the key does not exist.
+	IgnoreValue bool `protobuf:"varint,5,opt,name=ignore_value,json=ignoreValue,proto3" json:"ignore_value,omitempty"`
+	// If ignore_lease is set, etcd updates the key using its current lease.
+	// Returns an error if the key does not exist.
+	IgnoreLease          bool     `protobuf:"varint,6,opt,name=ignore_lease,json=ignoreLease,proto3" json:"ignore_lease,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *PutRequest) Reset()         { *m = PutRequest{} }
+func (m *PutRequest) String() string { return proto.CompactTextString(m) }
+func (*PutRequest) ProtoMessage()    {}
+func (*PutRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{3}
+}
+func (m *PutRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *PutRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_PutRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *PutRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PutRequest.Merge(m, src)
+}
+func (m *PutRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *PutRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_PutRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PutRequest proto.InternalMessageInfo
+
+func (m *PutRequest) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *PutRequest) GetValue() []byte {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+func (m *PutRequest) GetLease() int64 {
+	if m != nil {
+		return m.Lease
+	}
+	return 0
+}
+
+func (m *PutRequest) GetPrevKv() bool {
+	if m != nil {
+		return m.PrevKv
+	}
+	return false
+}
+
+func (m *PutRequest) GetIgnoreValue() bool {
+	if m != nil {
+		return m.IgnoreValue
+	}
+	return false
+}
+
+func (m *PutRequest) GetIgnoreLease() bool {
+	if m != nil {
+		return m.IgnoreLease
+	}
+	return false
+}
+
+type PutResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// if prev_kv is set in the request, the previous key-value pair will be returned.
+	PrevKv               *mvccpb.KeyValue `protobuf:"bytes,2,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *PutResponse) Reset()         { *m = PutResponse{} }
+func (m *PutResponse) String() string { return proto.CompactTextString(m) }
+func (*PutResponse) ProtoMessage()    {}
+func (*PutResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{4}
+}
+func (m *PutResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *PutResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_PutResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *PutResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PutResponse.Merge(m, src)
+}
+func (m *PutResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *PutResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_PutResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PutResponse proto.InternalMessageInfo
+
+func (m *PutResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *PutResponse) GetPrevKv() *mvccpb.KeyValue {
+	if m != nil {
+		return m.PrevKv
+	}
+	return nil
+}
+
+type DeleteRangeRequest struct {
+	// key is the first key to delete in the range.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	// range_end is the key following the last key to delete for the range [key, range_end).
+	// If range_end is not given, the range is defined to contain only the key argument.
+	// If range_end is one bit larger than the given key, then the range is all the keys
+	// with the prefix (the given key).
+	// If range_end is '\0', the range is all keys greater than or equal to the key argument.
+	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	// If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
+	// The previous key-value pairs will be returned in the delete response.
+	PrevKv               bool     `protobuf:"varint,3,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeleteRangeRequest) Reset()         { *m = DeleteRangeRequest{} }
+func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) }
+func (*DeleteRangeRequest) ProtoMessage()    {}
+func (*DeleteRangeRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{5}
+}
+func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *DeleteRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_DeleteRangeRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *DeleteRangeRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeleteRangeRequest.Merge(m, src)
+}
+func (m *DeleteRangeRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *DeleteRangeRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeleteRangeRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeleteRangeRequest proto.InternalMessageInfo
+
+func (m *DeleteRangeRequest) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *DeleteRangeRequest) GetRangeEnd() []byte {
+	if m != nil {
+		return m.RangeEnd
+	}
+	return nil
+}
+
+func (m *DeleteRangeRequest) GetPrevKv() bool {
+	if m != nil {
+		return m.PrevKv
+	}
+	return false
+}
+
+type DeleteRangeResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// deleted is the number of keys deleted by the delete range request.
+	Deleted int64 `protobuf:"varint,2,opt,name=deleted,proto3" json:"deleted,omitempty"`
+	// if prev_kv is set in the request, the previous key-value pairs will be returned.
+	PrevKvs              []*mvccpb.KeyValue `protobuf:"bytes,3,rep,name=prev_kvs,json=prevKvs,proto3" json:"prev_kvs,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *DeleteRangeResponse) Reset()         { *m = DeleteRangeResponse{} }
+func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) }
+func (*DeleteRangeResponse) ProtoMessage()    {}
+func (*DeleteRangeResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{6}
+}
+func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *DeleteRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_DeleteRangeResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *DeleteRangeResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeleteRangeResponse.Merge(m, src)
+}
+func (m *DeleteRangeResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *DeleteRangeResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeleteRangeResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeleteRangeResponse proto.InternalMessageInfo
+
+func (m *DeleteRangeResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *DeleteRangeResponse) GetDeleted() int64 {
+	if m != nil {
+		return m.Deleted
+	}
+	return 0
+}
+
+func (m *DeleteRangeResponse) GetPrevKvs() []*mvccpb.KeyValue {
+	if m != nil {
+		return m.PrevKvs
+	}
+	return nil
+}
+
+type RequestOp struct {
+	// request is a union of request types accepted by a transaction.
+	//
+	// Types that are valid to be assigned to Request:
+	//	*RequestOp_RequestRange
+	//	*RequestOp_RequestPut
+	//	*RequestOp_RequestDeleteRange
+	//	*RequestOp_RequestTxn
+	Request              isRequestOp_Request `protobuf_oneof:"request"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *RequestOp) Reset()         { *m = RequestOp{} }
+func (m *RequestOp) String() string { return proto.CompactTextString(m) }
+func (*RequestOp) ProtoMessage()    {}
+func (*RequestOp) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{7}
+}
+func (m *RequestOp) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *RequestOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_RequestOp.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *RequestOp) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RequestOp.Merge(m, src)
+}
+func (m *RequestOp) XXX_Size() int {
+	return m.Size()
+}
+func (m *RequestOp) XXX_DiscardUnknown() {
+	xxx_messageInfo_RequestOp.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RequestOp proto.InternalMessageInfo
+
+type isRequestOp_Request interface {
+	isRequestOp_Request()
+	MarshalTo([]byte) (int, error)
+	Size() int
+}
+
+type RequestOp_RequestRange struct {
+	RequestRange *RangeRequest `protobuf:"bytes,1,opt,name=request_range,json=requestRange,proto3,oneof"`
+}
+type RequestOp_RequestPut struct {
+	RequestPut *PutRequest `protobuf:"bytes,2,opt,name=request_put,json=requestPut,proto3,oneof"`
+}
+type RequestOp_RequestDeleteRange struct {
+	RequestDeleteRange *DeleteRangeRequest `protobuf:"bytes,3,opt,name=request_delete_range,json=requestDeleteRange,proto3,oneof"`
+}
+type RequestOp_RequestTxn struct {
+	RequestTxn *TxnRequest `protobuf:"bytes,4,opt,name=request_txn,json=requestTxn,proto3,oneof"`
+}
+
+func (*RequestOp_RequestRange) isRequestOp_Request()       {}
+func (*RequestOp_RequestPut) isRequestOp_Request()         {}
+func (*RequestOp_RequestDeleteRange) isRequestOp_Request() {}
+func (*RequestOp_RequestTxn) isRequestOp_Request()         {}
+
+func (m *RequestOp) GetRequest() isRequestOp_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (m *RequestOp) GetRequestRange() *RangeRequest {
+	if x, ok := m.GetRequest().(*RequestOp_RequestRange); ok {
+		return x.RequestRange
+	}
+	return nil
+}
+
+func (m *RequestOp) GetRequestPut() *PutRequest {
+	if x, ok := m.GetRequest().(*RequestOp_RequestPut); ok {
+		return x.RequestPut
+	}
+	return nil
+}
+
+func (m *RequestOp) GetRequestDeleteRange() *DeleteRangeRequest {
+	if x, ok := m.GetRequest().(*RequestOp_RequestDeleteRange); ok {
+		return x.RequestDeleteRange
+	}
+	return nil
+}
+
+func (m *RequestOp) GetRequestTxn() *TxnRequest {
+	if x, ok := m.GetRequest().(*RequestOp_RequestTxn); ok {
+		return x.RequestTxn
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*RequestOp) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _RequestOp_OneofMarshaler, _RequestOp_OneofUnmarshaler, _RequestOp_OneofSizer, []interface{}{
+		(*RequestOp_RequestRange)(nil),
+		(*RequestOp_RequestPut)(nil),
+		(*RequestOp_RequestDeleteRange)(nil),
+		(*RequestOp_RequestTxn)(nil),
+	}
+}
+
+func _RequestOp_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*RequestOp)
+	// request
+	switch x := m.Request.(type) {
+	case *RequestOp_RequestRange:
+		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.RequestRange); err != nil {
+			return err
+		}
+	case *RequestOp_RequestPut:
+		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.RequestPut); err != nil {
+			return err
+		}
+	case *RequestOp_RequestDeleteRange:
+		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.RequestDeleteRange); err != nil {
+			return err
+		}
+	case *RequestOp_RequestTxn:
+		_ = b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.RequestTxn); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("RequestOp.Request has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _RequestOp_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*RequestOp)
+	switch tag {
+	case 1: // request.request_range
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(RangeRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &RequestOp_RequestRange{msg}
+		return true, err
+	case 2: // request.request_put
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(PutRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &RequestOp_RequestPut{msg}
+		return true, err
+	case 3: // request.request_delete_range
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(DeleteRangeRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &RequestOp_RequestDeleteRange{msg}
+		return true, err
+	case 4: // request.request_txn
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(TxnRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &RequestOp_RequestTxn{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _RequestOp_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*RequestOp)
+	// request
+	switch x := m.Request.(type) {
+	case *RequestOp_RequestRange:
+		s := proto.Size(x.RequestRange)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *RequestOp_RequestPut:
+		s := proto.Size(x.RequestPut)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *RequestOp_RequestDeleteRange:
+		s := proto.Size(x.RequestDeleteRange)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *RequestOp_RequestTxn:
+		s := proto.Size(x.RequestTxn)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type ResponseOp struct {
+	// response is a union of response types returned by a transaction.
+	//
+	// Types that are valid to be assigned to Response:
+	//	*ResponseOp_ResponseRange
+	//	*ResponseOp_ResponsePut
+	//	*ResponseOp_ResponseDeleteRange
+	//	*ResponseOp_ResponseTxn
+	Response             isResponseOp_Response `protobuf_oneof:"response"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *ResponseOp) Reset()         { *m = ResponseOp{} }
+func (m *ResponseOp) String() string { return proto.CompactTextString(m) }
+func (*ResponseOp) ProtoMessage()    {}
+func (*ResponseOp) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{8}
+}
+func (m *ResponseOp) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *ResponseOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_ResponseOp.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *ResponseOp) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ResponseOp.Merge(m, src)
+}
+func (m *ResponseOp) XXX_Size() int {
+	return m.Size()
+}
+func (m *ResponseOp) XXX_DiscardUnknown() {
+	xxx_messageInfo_ResponseOp.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ResponseOp proto.InternalMessageInfo
+
+type isResponseOp_Response interface {
+	isResponseOp_Response()
+	MarshalTo([]byte) (int, error)
+	Size() int
+}
+
+type ResponseOp_ResponseRange struct {
+	ResponseRange *RangeResponse `protobuf:"bytes,1,opt,name=response_range,json=responseRange,proto3,oneof"`
+}
+type ResponseOp_ResponsePut struct {
+	ResponsePut *PutResponse `protobuf:"bytes,2,opt,name=response_put,json=responsePut,proto3,oneof"`
+}
+type ResponseOp_ResponseDeleteRange struct {
+	ResponseDeleteRange *DeleteRangeResponse `protobuf:"bytes,3,opt,name=response_delete_range,json=responseDeleteRange,proto3,oneof"`
+}
+type ResponseOp_ResponseTxn struct {
+	ResponseTxn *TxnResponse `protobuf:"bytes,4,opt,name=response_txn,json=responseTxn,proto3,oneof"`
+}
+
+func (*ResponseOp_ResponseRange) isResponseOp_Response()       {}
+func (*ResponseOp_ResponsePut) isResponseOp_Response()         {}
+func (*ResponseOp_ResponseDeleteRange) isResponseOp_Response() {}
+func (*ResponseOp_ResponseTxn) isResponseOp_Response()         {}
+
+func (m *ResponseOp) GetResponse() isResponseOp_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (m *ResponseOp) GetResponseRange() *RangeResponse {
+	if x, ok := m.GetResponse().(*ResponseOp_ResponseRange); ok {
+		return x.ResponseRange
+	}
+	return nil
+}
+
+func (m *ResponseOp) GetResponsePut() *PutResponse {
+	if x, ok := m.GetResponse().(*ResponseOp_ResponsePut); ok {
+		return x.ResponsePut
+	}
+	return nil
+}
+
+func (m *ResponseOp) GetResponseDeleteRange() *DeleteRangeResponse {
+	if x, ok := m.GetResponse().(*ResponseOp_ResponseDeleteRange); ok {
+		return x.ResponseDeleteRange
+	}
+	return nil
+}
+
+func (m *ResponseOp) GetResponseTxn() *TxnResponse {
+	if x, ok := m.GetResponse().(*ResponseOp_ResponseTxn); ok {
+		return x.ResponseTxn
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*ResponseOp) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _ResponseOp_OneofMarshaler, _ResponseOp_OneofUnmarshaler, _ResponseOp_OneofSizer, []interface{}{
+		(*ResponseOp_ResponseRange)(nil),
+		(*ResponseOp_ResponsePut)(nil),
+		(*ResponseOp_ResponseDeleteRange)(nil),
+		(*ResponseOp_ResponseTxn)(nil),
+	}
+}
+
+func _ResponseOp_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*ResponseOp)
+	// response
+	switch x := m.Response.(type) {
+	case *ResponseOp_ResponseRange:
+		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ResponseRange); err != nil {
+			return err
+		}
+	case *ResponseOp_ResponsePut:
+		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ResponsePut); err != nil {
+			return err
+		}
+	case *ResponseOp_ResponseDeleteRange:
+		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ResponseDeleteRange); err != nil {
+			return err
+		}
+	case *ResponseOp_ResponseTxn:
+		_ = b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ResponseTxn); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("ResponseOp.Response has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _ResponseOp_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*ResponseOp)
+	switch tag {
+	case 1: // response.response_range
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(RangeResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &ResponseOp_ResponseRange{msg}
+		return true, err
+	case 2: // response.response_put
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(PutResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &ResponseOp_ResponsePut{msg}
+		return true, err
+	case 3: // response.response_delete_range
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(DeleteRangeResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &ResponseOp_ResponseDeleteRange{msg}
+		return true, err
+	case 4: // response.response_txn
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(TxnResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &ResponseOp_ResponseTxn{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _ResponseOp_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*ResponseOp)
+	// response
+	switch x := m.Response.(type) {
+	case *ResponseOp_ResponseRange:
+		s := proto.Size(x.ResponseRange)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *ResponseOp_ResponsePut:
+		s := proto.Size(x.ResponsePut)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *ResponseOp_ResponseDeleteRange:
+		s := proto.Size(x.ResponseDeleteRange)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *ResponseOp_ResponseTxn:
+		s := proto.Size(x.ResponseTxn)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type Compare struct {
+	// result is logical comparison operation for this comparison.
+	Result Compare_CompareResult `protobuf:"varint,1,opt,name=result,proto3,enum=etcdserverpb.Compare_CompareResult" json:"result,omitempty"`
+	// target is the key-value field to inspect for the comparison.
+	Target Compare_CompareTarget `protobuf:"varint,2,opt,name=target,proto3,enum=etcdserverpb.Compare_CompareTarget" json:"target,omitempty"`
+	// key is the subject key for the comparison operation.
+	Key []byte `protobuf:"bytes,3,opt,name=key,proto3" json:"key,omitempty"`
+	// Types that are valid to be assigned to TargetUnion:
+	//	*Compare_Version
+	//	*Compare_CreateRevision
+	//	*Compare_ModRevision
+	//	*Compare_Value
+	//	*Compare_Lease
+	TargetUnion isCompare_TargetUnion `protobuf_oneof:"target_union"`
+	// range_end compares the given target to all keys in the range [key, range_end).
+	// See RangeRequest for more details on key ranges.
+	RangeEnd             []byte   `protobuf:"bytes,64,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Compare) Reset()         { *m = Compare{} }
+func (m *Compare) String() string { return proto.CompactTextString(m) }
+func (*Compare) ProtoMessage()    {}
+func (*Compare) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{9}
+}
+func (m *Compare) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Compare) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Compare.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Compare) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Compare.Merge(m, src)
+}
+func (m *Compare) XXX_Size() int {
+	return m.Size()
+}
+func (m *Compare) XXX_DiscardUnknown() {
+	xxx_messageInfo_Compare.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Compare proto.InternalMessageInfo
+
+type isCompare_TargetUnion interface {
+	isCompare_TargetUnion()
+	MarshalTo([]byte) (int, error)
+	Size() int
+}
+
+type Compare_Version struct {
+	Version int64 `protobuf:"varint,4,opt,name=version,proto3,oneof"`
+}
+type Compare_CreateRevision struct {
+	CreateRevision int64 `protobuf:"varint,5,opt,name=create_revision,json=createRevision,proto3,oneof"`
+}
+type Compare_ModRevision struct {
+	ModRevision int64 `protobuf:"varint,6,opt,name=mod_revision,json=modRevision,proto3,oneof"`
+}
+type Compare_Value struct {
+	Value []byte `protobuf:"bytes,7,opt,name=value,proto3,oneof"`
+}
+type Compare_Lease struct {
+	Lease int64 `protobuf:"varint,8,opt,name=lease,proto3,oneof"`
+}
+
+func (*Compare_Version) isCompare_TargetUnion()        {}
+func (*Compare_CreateRevision) isCompare_TargetUnion() {}
+func (*Compare_ModRevision) isCompare_TargetUnion()    {}
+func (*Compare_Value) isCompare_TargetUnion()          {}
+func (*Compare_Lease) isCompare_TargetUnion()          {}
+
+func (m *Compare) GetTargetUnion() isCompare_TargetUnion {
+	if m != nil {
+		return m.TargetUnion
+	}
+	return nil
+}
+
+func (m *Compare) GetResult() Compare_CompareResult {
+	if m != nil {
+		return m.Result
+	}
+	return Compare_EQUAL
+}
+
+func (m *Compare) GetTarget() Compare_CompareTarget {
+	if m != nil {
+		return m.Target
+	}
+	return Compare_VERSION
+}
+
+func (m *Compare) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *Compare) GetVersion() int64 {
+	if x, ok := m.GetTargetUnion().(*Compare_Version); ok {
+		return x.Version
+	}
+	return 0
+}
+
+func (m *Compare) GetCreateRevision() int64 {
+	if x, ok := m.GetTargetUnion().(*Compare_CreateRevision); ok {
+		return x.CreateRevision
+	}
+	return 0
+}
+
+func (m *Compare) GetModRevision() int64 {
+	if x, ok := m.GetTargetUnion().(*Compare_ModRevision); ok {
+		return x.ModRevision
+	}
+	return 0
+}
+
+func (m *Compare) GetValue() []byte {
+	if x, ok := m.GetTargetUnion().(*Compare_Value); ok {
+		return x.Value
+	}
+	return nil
+}
+
+func (m *Compare) GetLease() int64 {
+	if x, ok := m.GetTargetUnion().(*Compare_Lease); ok {
+		return x.Lease
+	}
+	return 0
+}
+
+func (m *Compare) GetRangeEnd() []byte {
+	if m != nil {
+		return m.RangeEnd
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*Compare) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _Compare_OneofMarshaler, _Compare_OneofUnmarshaler, _Compare_OneofSizer, []interface{}{
+		(*Compare_Version)(nil),
+		(*Compare_CreateRevision)(nil),
+		(*Compare_ModRevision)(nil),
+		(*Compare_Value)(nil),
+		(*Compare_Lease)(nil),
+	}
+}
+
+func _Compare_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*Compare)
+	// target_union
+	switch x := m.TargetUnion.(type) {
+	case *Compare_Version:
+		_ = b.EncodeVarint(4<<3 | proto.WireVarint)
+		_ = b.EncodeVarint(uint64(x.Version))
+	case *Compare_CreateRevision:
+		_ = b.EncodeVarint(5<<3 | proto.WireVarint)
+		_ = b.EncodeVarint(uint64(x.CreateRevision))
+	case *Compare_ModRevision:
+		_ = b.EncodeVarint(6<<3 | proto.WireVarint)
+		_ = b.EncodeVarint(uint64(x.ModRevision))
+	case *Compare_Value:
+		_ = b.EncodeVarint(7<<3 | proto.WireBytes)
+		_ = b.EncodeRawBytes(x.Value)
+	case *Compare_Lease:
+		_ = b.EncodeVarint(8<<3 | proto.WireVarint)
+		_ = b.EncodeVarint(uint64(x.Lease))
+	case nil:
+	default:
+		return fmt.Errorf("Compare.TargetUnion has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _Compare_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*Compare)
+	switch tag {
+	case 4: // target_union.version
+		if wire != proto.WireVarint {
+			return true, proto.ErrInternalBadWireType
+		}
+		x, err := b.DecodeVarint()
+		m.TargetUnion = &Compare_Version{int64(x)}
+		return true, err
+	case 5: // target_union.create_revision
+		if wire != proto.WireVarint {
+			return true, proto.ErrInternalBadWireType
+		}
+		x, err := b.DecodeVarint()
+		m.TargetUnion = &Compare_CreateRevision{int64(x)}
+		return true, err
+	case 6: // target_union.mod_revision
+		if wire != proto.WireVarint {
+			return true, proto.ErrInternalBadWireType
+		}
+		x, err := b.DecodeVarint()
+		m.TargetUnion = &Compare_ModRevision{int64(x)}
+		return true, err
+	case 7: // target_union.value
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		x, err := b.DecodeRawBytes(true)
+		m.TargetUnion = &Compare_Value{x}
+		return true, err
+	case 8: // target_union.lease
+		if wire != proto.WireVarint {
+			return true, proto.ErrInternalBadWireType
+		}
+		x, err := b.DecodeVarint()
+		m.TargetUnion = &Compare_Lease{int64(x)}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _Compare_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*Compare)
+	// target_union
+	switch x := m.TargetUnion.(type) {
+	case *Compare_Version:
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(x.Version))
+	case *Compare_CreateRevision:
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(x.CreateRevision))
+	case *Compare_ModRevision:
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(x.ModRevision))
+	case *Compare_Value:
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(len(x.Value)))
+		n += len(x.Value)
+	case *Compare_Lease:
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(x.Lease))
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// From google paxosdb paper:
+// Our implementation hinges around a powerful primitive which we call MultiOp. All other database
+// operations except for iteration are implemented as a single call to MultiOp. A MultiOp is applied atomically
+// and consists of three components:
+// 1. A list of tests called guard. Each test in guard checks a single entry in the database. It may check
+// for the absence or presence of a value, or compare with a given value. Two different tests in the guard
+// may apply to the same or different entries in the database. All tests in the guard are applied and
+// MultiOp returns the results. If all tests are true, MultiOp executes t op (see item 2 below), otherwise
+// it executes f op (see item 3 below).
+// 2. A list of database operations called t op. Each operation in the list is either an insert, delete, or
+// lookup operation, and applies to a single database entry. Two different operations in the list may apply
+// to the same or different entries in the database. These operations are executed
+// if guard evaluates to
+// true.
+// 3. A list of database operations called f op. Like t op, but executed if guard evaluates to false.
+type TxnRequest struct {
+	// compare is a list of predicates representing a conjunction of terms.
+	// If the comparisons succeed, then the success requests will be processed in order,
+	// and the response will contain their respective responses in order.
+	// If the comparisons fail, then the failure requests will be processed in order,
+	// and the response will contain their respective responses in order.
+	Compare []*Compare `protobuf:"bytes,1,rep,name=compare,proto3" json:"compare,omitempty"`
+	// success is a list of requests which will be applied when compare evaluates to true.
+	Success []*RequestOp `protobuf:"bytes,2,rep,name=success,proto3" json:"success,omitempty"`
+	// failure is a list of requests which will be applied when compare evaluates to false.
+	Failure              []*RequestOp `protobuf:"bytes,3,rep,name=failure,proto3" json:"failure,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *TxnRequest) Reset()         { *m = TxnRequest{} }
+func (m *TxnRequest) String() string { return proto.CompactTextString(m) }
+func (*TxnRequest) ProtoMessage()    {}
+func (*TxnRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{10}
+}
+func (m *TxnRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *TxnRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_TxnRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *TxnRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TxnRequest.Merge(m, src)
+}
+func (m *TxnRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *TxnRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_TxnRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TxnRequest proto.InternalMessageInfo
+
+func (m *TxnRequest) GetCompare() []*Compare {
+	if m != nil {
+		return m.Compare
+	}
+	return nil
+}
+
+func (m *TxnRequest) GetSuccess() []*RequestOp {
+	if m != nil {
+		return m.Success
+	}
+	return nil
+}
+
+func (m *TxnRequest) GetFailure() []*RequestOp {
+	if m != nil {
+		return m.Failure
+	}
+	return nil
+}
+
+type TxnResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// succeeded is set to true if the compare evaluated to true or false otherwise.
+	Succeeded bool `protobuf:"varint,2,opt,name=succeeded,proto3" json:"succeeded,omitempty"`
+	// responses is a list of responses corresponding to the results from applying
+	// success if succeeded is true or failure if succeeded is false.
+	Responses            []*ResponseOp `protobuf:"bytes,3,rep,name=responses,proto3" json:"responses,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *TxnResponse) Reset()         { *m = TxnResponse{} }
+func (m *TxnResponse) String() string { return proto.CompactTextString(m) }
+func (*TxnResponse) ProtoMessage()    {}
+func (*TxnResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{11}
+}
+func (m *TxnResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *TxnResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_TxnResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *TxnResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TxnResponse.Merge(m, src)
+}
+func (m *TxnResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *TxnResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_TxnResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TxnResponse proto.InternalMessageInfo
+
+func (m *TxnResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *TxnResponse) GetSucceeded() bool {
+	if m != nil {
+		return m.Succeeded
+	}
+	return false
+}
+
+func (m *TxnResponse) GetResponses() []*ResponseOp {
+	if m != nil {
+		return m.Responses
+	}
+	return nil
+}
+
+// CompactionRequest compacts the key-value store up to a given revision. All superseded keys
+// with a revision less than the compaction revision will be removed.
+type CompactionRequest struct {
+	// revision is the key-value store revision for the compaction operation.
+	Revision int64 `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"`
+	// physical is set so the RPC will wait until the compaction is physically
+	// applied to the local database such that compacted entries are totally
+	// removed from the backend database.
+	Physical             bool     `protobuf:"varint,2,opt,name=physical,proto3" json:"physical,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *CompactionRequest) Reset()         { *m = CompactionRequest{} }
+func (m *CompactionRequest) String() string { return proto.CompactTextString(m) }
+func (*CompactionRequest) ProtoMessage()    {}
+func (*CompactionRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{12}
+}
+func (m *CompactionRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *CompactionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_CompactionRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *CompactionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CompactionRequest.Merge(m, src)
+}
+func (m *CompactionRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *CompactionRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_CompactionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CompactionRequest proto.InternalMessageInfo
+
+func (m *CompactionRequest) GetRevision() int64 {
+	if m != nil {
+		return m.Revision
+	}
+	return 0
+}
+
+func (m *CompactionRequest) GetPhysical() bool {
+	if m != nil {
+		return m.Physical
+	}
+	return false
+}
+
+type CompactionResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *CompactionResponse) Reset()         { *m = CompactionResponse{} }
+func (m *CompactionResponse) String() string { return proto.CompactTextString(m) }
+func (*CompactionResponse) ProtoMessage()    {}
+func (*CompactionResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{13}
+}
+func (m *CompactionResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *CompactionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_CompactionResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *CompactionResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CompactionResponse.Merge(m, src)
+}
+func (m *CompactionResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *CompactionResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_CompactionResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CompactionResponse proto.InternalMessageInfo
+
+func (m *CompactionResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type HashRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HashRequest) Reset()         { *m = HashRequest{} }
+func (m *HashRequest) String() string { return proto.CompactTextString(m) }
+func (*HashRequest) ProtoMessage()    {}
+func (*HashRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{14}
+}
+func (m *HashRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *HashRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_HashRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *HashRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HashRequest.Merge(m, src)
+}
+func (m *HashRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *HashRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_HashRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HashRequest proto.InternalMessageInfo
+
+type HashKVRequest struct {
+	// revision is the key-value store revision for the hash operation.
+	Revision             int64    `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HashKVRequest) Reset()         { *m = HashKVRequest{} }
+func (m *HashKVRequest) String() string { return proto.CompactTextString(m) }
+func (*HashKVRequest) ProtoMessage()    {}
+func (*HashKVRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{15}
+}
+func (m *HashKVRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *HashKVRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_HashKVRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *HashKVRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HashKVRequest.Merge(m, src)
+}
+func (m *HashKVRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *HashKVRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_HashKVRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HashKVRequest proto.InternalMessageInfo
+
+func (m *HashKVRequest) GetRevision() int64 {
+	if m != nil {
+		return m.Revision
+	}
+	return 0
+}
+
+type HashKVResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// hash is the hash value computed from the responding member's MVCC keys up to a given revision.
+	Hash uint32 `protobuf:"varint,2,opt,name=hash,proto3" json:"hash,omitempty"`
+	// compact_revision is the compacted revision of key-value store when hash begins.
+	CompactRevision      int64    `protobuf:"varint,3,opt,name=compact_revision,json=compactRevision,proto3" json:"compact_revision,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HashKVResponse) Reset()         { *m = HashKVResponse{} }
+func (m *HashKVResponse) String() string { return proto.CompactTextString(m) }
+func (*HashKVResponse) ProtoMessage()    {}
+func (*HashKVResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{16}
+}
+func (m *HashKVResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *HashKVResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_HashKVResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *HashKVResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HashKVResponse.Merge(m, src)
+}
+func (m *HashKVResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *HashKVResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_HashKVResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HashKVResponse proto.InternalMessageInfo
+
+func (m *HashKVResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *HashKVResponse) GetHash() uint32 {
+	if m != nil {
+		return m.Hash
+	}
+	return 0
+}
+
+func (m *HashKVResponse) GetCompactRevision() int64 {
+	if m != nil {
+		return m.CompactRevision
+	}
+	return 0
+}
+
+type HashResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// hash is the hash value computed from the responding member's KV's backend.
+	Hash                 uint32   `protobuf:"varint,2,opt,name=hash,proto3" json:"hash,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HashResponse) Reset()         { *m = HashResponse{} }
+func (m *HashResponse) String() string { return proto.CompactTextString(m) }
+func (*HashResponse) ProtoMessage()    {}
+func (*HashResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{17}
+}
+func (m *HashResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *HashResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_HashResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *HashResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HashResponse.Merge(m, src)
+}
+func (m *HashResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *HashResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_HashResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HashResponse proto.InternalMessageInfo
+
+func (m *HashResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *HashResponse) GetHash() uint32 {
+	if m != nil {
+		return m.Hash
+	}
+	return 0
+}
+
+type SnapshotRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *SnapshotRequest) Reset()         { *m = SnapshotRequest{} }
+func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) }
+func (*SnapshotRequest) ProtoMessage()    {}
+func (*SnapshotRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{18}
+}
+func (m *SnapshotRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *SnapshotRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_SnapshotRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *SnapshotRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SnapshotRequest.Merge(m, src)
+}
+func (m *SnapshotRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *SnapshotRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_SnapshotRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SnapshotRequest proto.InternalMessageInfo
+
+type SnapshotResponse struct {
+	// header has the current key-value store information. The first header in the snapshot
+	// stream indicates the point in time of the snapshot.
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// remaining_bytes is the number of blob bytes to be sent after this message
+	RemainingBytes uint64 `protobuf:"varint,2,opt,name=remaining_bytes,json=remainingBytes,proto3" json:"remaining_bytes,omitempty"`
+	// blob contains the next chunk of the snapshot in the snapshot stream.
+	Blob                 []byte   `protobuf:"bytes,3,opt,name=blob,proto3" json:"blob,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *SnapshotResponse) Reset()         { *m = SnapshotResponse{} }
+func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) }
+func (*SnapshotResponse) ProtoMessage()    {}
+func (*SnapshotResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{19}
+}
+func (m *SnapshotResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *SnapshotResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_SnapshotResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *SnapshotResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SnapshotResponse.Merge(m, src)
+}
+func (m *SnapshotResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *SnapshotResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_SnapshotResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SnapshotResponse proto.InternalMessageInfo
+
+func (m *SnapshotResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *SnapshotResponse) GetRemainingBytes() uint64 {
+	if m != nil {
+		return m.RemainingBytes
+	}
+	return 0
+}
+
+func (m *SnapshotResponse) GetBlob() []byte {
+	if m != nil {
+		return m.Blob
+	}
+	return nil
+}
+
+type WatchRequest struct {
+	// request_union is a request to either create a new watcher or cancel an existing watcher.
+	//
+	// Types that are valid to be assigned to RequestUnion:
+	//	*WatchRequest_CreateRequest
+	//	*WatchRequest_CancelRequest
+	//	*WatchRequest_ProgressRequest
+	RequestUnion         isWatchRequest_RequestUnion `protobuf_oneof:"request_union"`
+	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
+	XXX_unrecognized     []byte                      `json:"-"`
+	XXX_sizecache        int32                       `json:"-"`
+}
+
+func (m *WatchRequest) Reset()         { *m = WatchRequest{} }
+func (m *WatchRequest) String() string { return proto.CompactTextString(m) }
+func (*WatchRequest) ProtoMessage()    {}
+func (*WatchRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{20}
+}
+func (m *WatchRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *WatchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_WatchRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *WatchRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WatchRequest.Merge(m, src)
+}
+func (m *WatchRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *WatchRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_WatchRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WatchRequest proto.InternalMessageInfo
+
+type isWatchRequest_RequestUnion interface {
+	isWatchRequest_RequestUnion()
+	MarshalTo([]byte) (int, error)
+	Size() int
+}
+
+type WatchRequest_CreateRequest struct {
+	CreateRequest *WatchCreateRequest `protobuf:"bytes,1,opt,name=create_request,json=createRequest,proto3,oneof"`
+}
+type WatchRequest_CancelRequest struct {
+	CancelRequest *WatchCancelRequest `protobuf:"bytes,2,opt,name=cancel_request,json=cancelRequest,proto3,oneof"`
+}
+type WatchRequest_ProgressRequest struct {
+	ProgressRequest *WatchProgressRequest `protobuf:"bytes,3,opt,name=progress_request,json=progressRequest,proto3,oneof"`
+}
+
+func (*WatchRequest_CreateRequest) isWatchRequest_RequestUnion()   {}
+func (*WatchRequest_CancelRequest) isWatchRequest_RequestUnion()   {}
+func (*WatchRequest_ProgressRequest) isWatchRequest_RequestUnion() {}
+
+func (m *WatchRequest) GetRequestUnion() isWatchRequest_RequestUnion {
+	if m != nil {
+		return m.RequestUnion
+	}
+	return nil
+}
+
+func (m *WatchRequest) GetCreateRequest() *WatchCreateRequest {
+	if x, ok := m.GetRequestUnion().(*WatchRequest_CreateRequest); ok {
+		return x.CreateRequest
+	}
+	return nil
+}
+
+func (m *WatchRequest) GetCancelRequest() *WatchCancelRequest {
+	if x, ok := m.GetRequestUnion().(*WatchRequest_CancelRequest); ok {
+		return x.CancelRequest
+	}
+	return nil
+}
+
+func (m *WatchRequest) GetProgressRequest() *WatchProgressRequest {
+	if x, ok := m.GetRequestUnion().(*WatchRequest_ProgressRequest); ok {
+		return x.ProgressRequest
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*WatchRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _WatchRequest_OneofMarshaler, _WatchRequest_OneofUnmarshaler, _WatchRequest_OneofSizer, []interface{}{
+		(*WatchRequest_CreateRequest)(nil),
+		(*WatchRequest_CancelRequest)(nil),
+		(*WatchRequest_ProgressRequest)(nil),
+	}
+}
+
+func _WatchRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*WatchRequest)
+	// request_union
+	switch x := m.RequestUnion.(type) {
+	case *WatchRequest_CreateRequest:
+		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.CreateRequest); err != nil {
+			return err
+		}
+	case *WatchRequest_CancelRequest:
+		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.CancelRequest); err != nil {
+			return err
+		}
+	case *WatchRequest_ProgressRequest:
+		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProgressRequest); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("WatchRequest.RequestUnion has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _WatchRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*WatchRequest)
+	switch tag {
+	case 1: // request_union.create_request
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(WatchCreateRequest)
+		err := b.DecodeMessage(msg)
+		m.RequestUnion = &WatchRequest_CreateRequest{msg}
+		return true, err
+	case 2: // request_union.cancel_request
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(WatchCancelRequest)
+		err := b.DecodeMessage(msg)
+		m.RequestUnion = &WatchRequest_CancelRequest{msg}
+		return true, err
+	case 3: // request_union.progress_request
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(WatchProgressRequest)
+		err := b.DecodeMessage(msg)
+		m.RequestUnion = &WatchRequest_ProgressRequest{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _WatchRequest_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*WatchRequest)
+	// request_union
+	switch x := m.RequestUnion.(type) {
+	case *WatchRequest_CreateRequest:
+		s := proto.Size(x.CreateRequest)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *WatchRequest_CancelRequest:
+		s := proto.Size(x.CancelRequest)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *WatchRequest_ProgressRequest:
+		s := proto.Size(x.ProgressRequest)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type WatchCreateRequest struct {
+	// key is the key to register for watching.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	// range_end is the end of the range [key, range_end) to watch. If range_end is not given,
+	// only the key argument is watched. If range_end is equal to '\0', all keys greater than
+	// or equal to the key argument are watched.
+	// If the range_end is one bit larger than the given key,
+	// then all keys with the prefix (the given key) will be watched.
+	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	// start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
+	StartRevision int64 `protobuf:"varint,3,opt,name=start_revision,json=startRevision,proto3" json:"start_revision,omitempty"`
+	// progress_notify is set so that the etcd server will periodically send a WatchResponse with
+	// no events to the new watcher if there are no recent events. It is useful when clients
+	// wish to recover a disconnected watcher starting from a recent known revision.
+	// The etcd server may decide how often it will send notifications based on current load.
+	ProgressNotify bool `protobuf:"varint,4,opt,name=progress_notify,json=progressNotify,proto3" json:"progress_notify,omitempty"`
+	// filters filter the events at server side before it sends back to the watcher.
+	Filters []WatchCreateRequest_FilterType `protobuf:"varint,5,rep,packed,name=filters,proto3,enum=etcdserverpb.WatchCreateRequest_FilterType" json:"filters,omitempty"`
+	// If prev_kv is set, created watcher gets the previous KV before the event happens.
+	// If the previous KV is already compacted, nothing will be returned.
+	PrevKv bool `protobuf:"varint,6,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
+	// If watch_id is provided and non-zero, it will be assigned to this watcher.
+	// Since creating a watcher in etcd is not a synchronous operation,
+	// this can be used ensure that ordering is correct when creating multiple
+	// watchers on the same stream. Creating a watcher with an ID already in
+	// use on the stream will cause an error to be returned.
+	WatchId int64 `protobuf:"varint,7,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
+	// fragment enables splitting large revisions into multiple watch responses.
+	Fragment             bool     `protobuf:"varint,8,opt,name=fragment,proto3" json:"fragment,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *WatchCreateRequest) Reset()         { *m = WatchCreateRequest{} }
+func (m *WatchCreateRequest) String() string { return proto.CompactTextString(m) }
+func (*WatchCreateRequest) ProtoMessage()    {}
+func (*WatchCreateRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{21}
+}
+func (m *WatchCreateRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *WatchCreateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_WatchCreateRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *WatchCreateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WatchCreateRequest.Merge(m, src)
+}
+func (m *WatchCreateRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *WatchCreateRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_WatchCreateRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WatchCreateRequest proto.InternalMessageInfo
+
+func (m *WatchCreateRequest) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *WatchCreateRequest) GetRangeEnd() []byte {
+	if m != nil {
+		return m.RangeEnd
+	}
+	return nil
+}
+
+func (m *WatchCreateRequest) GetStartRevision() int64 {
+	if m != nil {
+		return m.StartRevision
+	}
+	return 0
+}
+
+func (m *WatchCreateRequest) GetProgressNotify() bool {
+	if m != nil {
+		return m.ProgressNotify
+	}
+	return false
+}
+
+func (m *WatchCreateRequest) GetFilters() []WatchCreateRequest_FilterType {
+	if m != nil {
+		return m.Filters
+	}
+	return nil
+}
+
+func (m *WatchCreateRequest) GetPrevKv() bool {
+	if m != nil {
+		return m.PrevKv
+	}
+	return false
+}
+
+func (m *WatchCreateRequest) GetWatchId() int64 {
+	if m != nil {
+		return m.WatchId
+	}
+	return 0
+}
+
+func (m *WatchCreateRequest) GetFragment() bool {
+	if m != nil {
+		return m.Fragment
+	}
+	return false
+}
+
+type WatchCancelRequest struct {
+	// watch_id is the watcher id to cancel so that no more events are transmitted.
+	WatchId              int64    `protobuf:"varint,1,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *WatchCancelRequest) Reset()         { *m = WatchCancelRequest{} }
+func (m *WatchCancelRequest) String() string { return proto.CompactTextString(m) }
+func (*WatchCancelRequest) ProtoMessage()    {}
+func (*WatchCancelRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{22}
+}
+func (m *WatchCancelRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *WatchCancelRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_WatchCancelRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *WatchCancelRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WatchCancelRequest.Merge(m, src)
+}
+func (m *WatchCancelRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *WatchCancelRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_WatchCancelRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WatchCancelRequest proto.InternalMessageInfo
+
+func (m *WatchCancelRequest) GetWatchId() int64 {
+	if m != nil {
+		return m.WatchId
+	}
+	return 0
+}
+
+// Requests the a watch stream progress status be sent in the watch response stream as soon as
+// possible.
+type WatchProgressRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *WatchProgressRequest) Reset()         { *m = WatchProgressRequest{} }
+func (m *WatchProgressRequest) String() string { return proto.CompactTextString(m) }
+func (*WatchProgressRequest) ProtoMessage()    {}
+func (*WatchProgressRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{23}
+}
+func (m *WatchProgressRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *WatchProgressRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_WatchProgressRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *WatchProgressRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WatchProgressRequest.Merge(m, src)
+}
+func (m *WatchProgressRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *WatchProgressRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_WatchProgressRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WatchProgressRequest proto.InternalMessageInfo
+
+type WatchResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// watch_id is the ID of the watcher that corresponds to the response.
+	WatchId int64 `protobuf:"varint,2,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
+	// created is set to true if the response is for a create watch request.
+	// The client should record the watch_id and expect to receive events for
+	// the created watcher from the same stream.
+	// All events sent to the created watcher will attach with the same watch_id.
+	Created bool `protobuf:"varint,3,opt,name=created,proto3" json:"created,omitempty"`
+	// canceled is set to true if the response is for a cancel watch request.
+	// No further events will be sent to the canceled watcher.
+	Canceled bool `protobuf:"varint,4,opt,name=canceled,proto3" json:"canceled,omitempty"`
+	// compact_revision is set to the minimum index if a watcher tries to watch
+	// at a compacted index.
+	//
+	// This happens when creating a watcher at a compacted revision or the watcher cannot
+	// catch up with the progress of the key-value store.
+	//
+	// The client should treat the watcher as canceled and should not try to create any
+	// watcher with the same start_revision again.
+	CompactRevision int64 `protobuf:"varint,5,opt,name=compact_revision,json=compactRevision,proto3" json:"compact_revision,omitempty"`
+	// cancel_reason indicates the reason for canceling the watcher.
+	CancelReason string `protobuf:"bytes,6,opt,name=cancel_reason,json=cancelReason,proto3" json:"cancel_reason,omitempty"`
+	// framgment is true if large watch response was split over multiple responses.
+	Fragment             bool            `protobuf:"varint,7,opt,name=fragment,proto3" json:"fragment,omitempty"`
+	Events               []*mvccpb.Event `protobuf:"bytes,11,rep,name=events,proto3" json:"events,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *WatchResponse) Reset()         { *m = WatchResponse{} }
+func (m *WatchResponse) String() string { return proto.CompactTextString(m) }
+func (*WatchResponse) ProtoMessage()    {}
+func (*WatchResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{24}
+}
+func (m *WatchResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *WatchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_WatchResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *WatchResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WatchResponse.Merge(m, src)
+}
+func (m *WatchResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *WatchResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_WatchResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WatchResponse proto.InternalMessageInfo
+
+func (m *WatchResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *WatchResponse) GetWatchId() int64 {
+	if m != nil {
+		return m.WatchId
+	}
+	return 0
+}
+
+func (m *WatchResponse) GetCreated() bool {
+	if m != nil {
+		return m.Created
+	}
+	return false
+}
+
+func (m *WatchResponse) GetCanceled() bool {
+	if m != nil {
+		return m.Canceled
+	}
+	return false
+}
+
+func (m *WatchResponse) GetCompactRevision() int64 {
+	if m != nil {
+		return m.CompactRevision
+	}
+	return 0
+}
+
+func (m *WatchResponse) GetCancelReason() string {
+	if m != nil {
+		return m.CancelReason
+	}
+	return ""
+}
+
+func (m *WatchResponse) GetFragment() bool {
+	if m != nil {
+		return m.Fragment
+	}
+	return false
+}
+
+func (m *WatchResponse) GetEvents() []*mvccpb.Event {
+	if m != nil {
+		return m.Events
+	}
+	return nil
+}
+
+type LeaseGrantRequest struct {
+	// TTL is the advisory time-to-live in seconds. Expired lease will return -1.
+	TTL int64 `protobuf:"varint,1,opt,name=TTL,proto3" json:"TTL,omitempty"`
+	// ID is the requested ID for the lease. If ID is set to 0, the lessor chooses an ID.
+	ID                   int64    `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseGrantRequest) Reset()         { *m = LeaseGrantRequest{} }
+func (m *LeaseGrantRequest) String() string { return proto.CompactTextString(m) }
+func (*LeaseGrantRequest) ProtoMessage()    {}
+func (*LeaseGrantRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{25}
+}
+func (m *LeaseGrantRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseGrantRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseGrantRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseGrantRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseGrantRequest.Merge(m, src)
+}
+func (m *LeaseGrantRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseGrantRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseGrantRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseGrantRequest proto.InternalMessageInfo
+
+func (m *LeaseGrantRequest) GetTTL() int64 {
+	if m != nil {
+		return m.TTL
+	}
+	return 0
+}
+
+func (m *LeaseGrantRequest) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+type LeaseGrantResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// ID is the lease ID for the granted lease.
+	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
+	// TTL is the server chosen lease time-to-live in seconds.
+	TTL                  int64    `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
+	Error                string   `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseGrantResponse) Reset()         { *m = LeaseGrantResponse{} }
+func (m *LeaseGrantResponse) String() string { return proto.CompactTextString(m) }
+func (*LeaseGrantResponse) ProtoMessage()    {}
+func (*LeaseGrantResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{26}
+}
+func (m *LeaseGrantResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseGrantResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseGrantResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseGrantResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseGrantResponse.Merge(m, src)
+}
+func (m *LeaseGrantResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseGrantResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseGrantResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseGrantResponse proto.InternalMessageInfo
+
+func (m *LeaseGrantResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LeaseGrantResponse) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *LeaseGrantResponse) GetTTL() int64 {
+	if m != nil {
+		return m.TTL
+	}
+	return 0
+}
+
+func (m *LeaseGrantResponse) GetError() string {
+	if m != nil {
+		return m.Error
+	}
+	return ""
+}
+
+type LeaseRevokeRequest struct {
+	// ID is the lease ID to revoke. When the ID is revoked, all associated keys will be deleted.
+	ID                   int64    `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseRevokeRequest) Reset()         { *m = LeaseRevokeRequest{} }
+func (m *LeaseRevokeRequest) String() string { return proto.CompactTextString(m) }
+func (*LeaseRevokeRequest) ProtoMessage()    {}
+func (*LeaseRevokeRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{27}
+}
+func (m *LeaseRevokeRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseRevokeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseRevokeRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseRevokeRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseRevokeRequest.Merge(m, src)
+}
+func (m *LeaseRevokeRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseRevokeRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseRevokeRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseRevokeRequest proto.InternalMessageInfo
+
+func (m *LeaseRevokeRequest) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+type LeaseRevokeResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *LeaseRevokeResponse) Reset()         { *m = LeaseRevokeResponse{} }
+func (m *LeaseRevokeResponse) String() string { return proto.CompactTextString(m) }
+func (*LeaseRevokeResponse) ProtoMessage()    {}
+func (*LeaseRevokeResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{28}
+}
+func (m *LeaseRevokeResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseRevokeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseRevokeResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseRevokeResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseRevokeResponse.Merge(m, src)
+}
+func (m *LeaseRevokeResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseRevokeResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseRevokeResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseRevokeResponse proto.InternalMessageInfo
+
+func (m *LeaseRevokeResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type LeaseKeepAliveRequest struct {
+	// ID is the lease ID for the lease to keep alive.
+	ID                   int64    `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseKeepAliveRequest) Reset()         { *m = LeaseKeepAliveRequest{} }
+func (m *LeaseKeepAliveRequest) String() string { return proto.CompactTextString(m) }
+func (*LeaseKeepAliveRequest) ProtoMessage()    {}
+func (*LeaseKeepAliveRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{29}
+}
+func (m *LeaseKeepAliveRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseKeepAliveRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseKeepAliveRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseKeepAliveRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseKeepAliveRequest.Merge(m, src)
+}
+func (m *LeaseKeepAliveRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseKeepAliveRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseKeepAliveRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseKeepAliveRequest proto.InternalMessageInfo
+
+func (m *LeaseKeepAliveRequest) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+type LeaseKeepAliveResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// ID is the lease ID from the keep alive request.
+	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
+	// TTL is the new time-to-live for the lease.
+	TTL                  int64    `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseKeepAliveResponse) Reset()         { *m = LeaseKeepAliveResponse{} }
+func (m *LeaseKeepAliveResponse) String() string { return proto.CompactTextString(m) }
+func (*LeaseKeepAliveResponse) ProtoMessage()    {}
+func (*LeaseKeepAliveResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{30}
+}
+func (m *LeaseKeepAliveResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseKeepAliveResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseKeepAliveResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseKeepAliveResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseKeepAliveResponse.Merge(m, src)
+}
+func (m *LeaseKeepAliveResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseKeepAliveResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseKeepAliveResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseKeepAliveResponse proto.InternalMessageInfo
+
+func (m *LeaseKeepAliveResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LeaseKeepAliveResponse) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *LeaseKeepAliveResponse) GetTTL() int64 {
+	if m != nil {
+		return m.TTL
+	}
+	return 0
+}
+
+type LeaseTimeToLiveRequest struct {
+	// ID is the lease ID for the lease.
+	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	// keys is true to query all the keys attached to this lease.
+	Keys                 bool     `protobuf:"varint,2,opt,name=keys,proto3" json:"keys,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseTimeToLiveRequest) Reset()         { *m = LeaseTimeToLiveRequest{} }
+func (m *LeaseTimeToLiveRequest) String() string { return proto.CompactTextString(m) }
+func (*LeaseTimeToLiveRequest) ProtoMessage()    {}
+func (*LeaseTimeToLiveRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{31}
+}
+func (m *LeaseTimeToLiveRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseTimeToLiveRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseTimeToLiveRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseTimeToLiveRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseTimeToLiveRequest.Merge(m, src)
+}
+func (m *LeaseTimeToLiveRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseTimeToLiveRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseTimeToLiveRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseTimeToLiveRequest proto.InternalMessageInfo
+
+func (m *LeaseTimeToLiveRequest) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *LeaseTimeToLiveRequest) GetKeys() bool {
+	if m != nil {
+		return m.Keys
+	}
+	return false
+}
+
+type LeaseTimeToLiveResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// ID is the lease ID from the keep alive request.
+	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
+	// TTL is the remaining TTL in seconds for the lease; the lease will expire in under TTL+1 seconds.
+	TTL int64 `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
+	// GrantedTTL is the initial granted time in seconds upon lease creation/renewal.
+	GrantedTTL int64 `protobuf:"varint,4,opt,name=grantedTTL,proto3" json:"grantedTTL,omitempty"`
+	// Keys is the list of keys attached to this lease.
+	Keys                 [][]byte `protobuf:"bytes,5,rep,name=keys,proto3" json:"keys,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseTimeToLiveResponse) Reset()         { *m = LeaseTimeToLiveResponse{} }
+func (m *LeaseTimeToLiveResponse) String() string { return proto.CompactTextString(m) }
+func (*LeaseTimeToLiveResponse) ProtoMessage()    {}
+func (*LeaseTimeToLiveResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{32}
+}
+func (m *LeaseTimeToLiveResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseTimeToLiveResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseTimeToLiveResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseTimeToLiveResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseTimeToLiveResponse.Merge(m, src)
+}
+func (m *LeaseTimeToLiveResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseTimeToLiveResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseTimeToLiveResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseTimeToLiveResponse proto.InternalMessageInfo
+
+func (m *LeaseTimeToLiveResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LeaseTimeToLiveResponse) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *LeaseTimeToLiveResponse) GetTTL() int64 {
+	if m != nil {
+		return m.TTL
+	}
+	return 0
+}
+
+func (m *LeaseTimeToLiveResponse) GetGrantedTTL() int64 {
+	if m != nil {
+		return m.GrantedTTL
+	}
+	return 0
+}
+
+func (m *LeaseTimeToLiveResponse) GetKeys() [][]byte {
+	if m != nil {
+		return m.Keys
+	}
+	return nil
+}
+
+type LeaseLeasesRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseLeasesRequest) Reset()         { *m = LeaseLeasesRequest{} }
+func (m *LeaseLeasesRequest) String() string { return proto.CompactTextString(m) }
+func (*LeaseLeasesRequest) ProtoMessage()    {}
+func (*LeaseLeasesRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{33}
+}
+func (m *LeaseLeasesRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseLeasesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseLeasesRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseLeasesRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseLeasesRequest.Merge(m, src)
+}
+func (m *LeaseLeasesRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseLeasesRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseLeasesRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseLeasesRequest proto.InternalMessageInfo
+
+type LeaseStatus struct {
+	ID                   int64    `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LeaseStatus) Reset()         { *m = LeaseStatus{} }
+func (m *LeaseStatus) String() string { return proto.CompactTextString(m) }
+func (*LeaseStatus) ProtoMessage()    {}
+func (*LeaseStatus) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{34}
+}
+func (m *LeaseStatus) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseStatus.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseStatus.Merge(m, src)
+}
+func (m *LeaseStatus) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseStatus) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseStatus.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseStatus proto.InternalMessageInfo
+
+func (m *LeaseStatus) GetID() int64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+type LeaseLeasesResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Leases               []*LeaseStatus  `protobuf:"bytes,2,rep,name=leases,proto3" json:"leases,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *LeaseLeasesResponse) Reset()         { *m = LeaseLeasesResponse{} }
+func (m *LeaseLeasesResponse) String() string { return proto.CompactTextString(m) }
+func (*LeaseLeasesResponse) ProtoMessage()    {}
+func (*LeaseLeasesResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{35}
+}
+func (m *LeaseLeasesResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *LeaseLeasesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_LeaseLeasesResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *LeaseLeasesResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LeaseLeasesResponse.Merge(m, src)
+}
+func (m *LeaseLeasesResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *LeaseLeasesResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_LeaseLeasesResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LeaseLeasesResponse proto.InternalMessageInfo
+
+func (m *LeaseLeasesResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LeaseLeasesResponse) GetLeases() []*LeaseStatus {
+	if m != nil {
+		return m.Leases
+	}
+	return nil
+}
+
+type Member struct {
+	// ID is the member ID for this member.
+	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	// name is the human-readable name of the member. If the member is not started, the name will be an empty string.
+	Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+	// peerURLs is the list of URLs the member exposes to the cluster for communication.
+	PeerURLs []string `protobuf:"bytes,3,rep,name=peerURLs,proto3" json:"peerURLs,omitempty"`
+	// clientURLs is the list of URLs the member exposes to clients for communication. If the member is not started, clientURLs will be empty.
+	ClientURLs           []string `protobuf:"bytes,4,rep,name=clientURLs,proto3" json:"clientURLs,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Member) Reset()         { *m = Member{} }
+func (m *Member) String() string { return proto.CompactTextString(m) }
+func (*Member) ProtoMessage()    {}
+func (*Member) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{36}
+}
+func (m *Member) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Member) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Member.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Member) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Member.Merge(m, src)
+}
+func (m *Member) XXX_Size() int {
+	return m.Size()
+}
+func (m *Member) XXX_DiscardUnknown() {
+	xxx_messageInfo_Member.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Member proto.InternalMessageInfo
+
+func (m *Member) GetID() uint64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *Member) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *Member) GetPeerURLs() []string {
+	if m != nil {
+		return m.PeerURLs
+	}
+	return nil
+}
+
+func (m *Member) GetClientURLs() []string {
+	if m != nil {
+		return m.ClientURLs
+	}
+	return nil
+}
+
+type MemberAddRequest struct {
+	// peerURLs is the list of URLs the added member will use to communicate with the cluster.
+	PeerURLs             []string `protobuf:"bytes,1,rep,name=peerURLs,proto3" json:"peerURLs,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MemberAddRequest) Reset()         { *m = MemberAddRequest{} }
+func (m *MemberAddRequest) String() string { return proto.CompactTextString(m) }
+func (*MemberAddRequest) ProtoMessage()    {}
+func (*MemberAddRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{37}
+}
+func (m *MemberAddRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberAddRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberAddRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberAddRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberAddRequest.Merge(m, src)
+}
+func (m *MemberAddRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberAddRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberAddRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberAddRequest proto.InternalMessageInfo
+
+func (m *MemberAddRequest) GetPeerURLs() []string {
+	if m != nil {
+		return m.PeerURLs
+	}
+	return nil
+}
+
+type MemberAddResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// member is the member information for the added member.
+	Member *Member `protobuf:"bytes,2,opt,name=member,proto3" json:"member,omitempty"`
+	// members is a list of all members after adding the new member.
+	Members              []*Member `protobuf:"bytes,3,rep,name=members,proto3" json:"members,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *MemberAddResponse) Reset()         { *m = MemberAddResponse{} }
+func (m *MemberAddResponse) String() string { return proto.CompactTextString(m) }
+func (*MemberAddResponse) ProtoMessage()    {}
+func (*MemberAddResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{38}
+}
+func (m *MemberAddResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberAddResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberAddResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberAddResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberAddResponse.Merge(m, src)
+}
+func (m *MemberAddResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberAddResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberAddResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberAddResponse proto.InternalMessageInfo
+
+func (m *MemberAddResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *MemberAddResponse) GetMember() *Member {
+	if m != nil {
+		return m.Member
+	}
+	return nil
+}
+
+func (m *MemberAddResponse) GetMembers() []*Member {
+	if m != nil {
+		return m.Members
+	}
+	return nil
+}
+
+type MemberRemoveRequest struct {
+	// ID is the member ID of the member to remove.
+	ID                   uint64   `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MemberRemoveRequest) Reset()         { *m = MemberRemoveRequest{} }
+func (m *MemberRemoveRequest) String() string { return proto.CompactTextString(m) }
+func (*MemberRemoveRequest) ProtoMessage()    {}
+func (*MemberRemoveRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{39}
+}
+func (m *MemberRemoveRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberRemoveRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberRemoveRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberRemoveRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberRemoveRequest.Merge(m, src)
+}
+func (m *MemberRemoveRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberRemoveRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberRemoveRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberRemoveRequest proto.InternalMessageInfo
+
+func (m *MemberRemoveRequest) GetID() uint64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+type MemberRemoveResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// members is a list of all members after removing the member.
+	Members              []*Member `protobuf:"bytes,2,rep,name=members,proto3" json:"members,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *MemberRemoveResponse) Reset()         { *m = MemberRemoveResponse{} }
+func (m *MemberRemoveResponse) String() string { return proto.CompactTextString(m) }
+func (*MemberRemoveResponse) ProtoMessage()    {}
+func (*MemberRemoveResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{40}
+}
+func (m *MemberRemoveResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberRemoveResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberRemoveResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberRemoveResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberRemoveResponse.Merge(m, src)
+}
+func (m *MemberRemoveResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberRemoveResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberRemoveResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberRemoveResponse proto.InternalMessageInfo
+
+func (m *MemberRemoveResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *MemberRemoveResponse) GetMembers() []*Member {
+	if m != nil {
+		return m.Members
+	}
+	return nil
+}
+
+type MemberUpdateRequest struct {
+	// ID is the member ID of the member to update.
+	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	// peerURLs is the new list of URLs the member will use to communicate with the cluster.
+	PeerURLs             []string `protobuf:"bytes,2,rep,name=peerURLs,proto3" json:"peerURLs,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MemberUpdateRequest) Reset()         { *m = MemberUpdateRequest{} }
+func (m *MemberUpdateRequest) String() string { return proto.CompactTextString(m) }
+func (*MemberUpdateRequest) ProtoMessage()    {}
+func (*MemberUpdateRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{41}
+}
+func (m *MemberUpdateRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberUpdateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberUpdateRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberUpdateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberUpdateRequest.Merge(m, src)
+}
+func (m *MemberUpdateRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberUpdateRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberUpdateRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberUpdateRequest proto.InternalMessageInfo
+
+func (m *MemberUpdateRequest) GetID() uint64 {
+	if m != nil {
+		return m.ID
+	}
+	return 0
+}
+
+func (m *MemberUpdateRequest) GetPeerURLs() []string {
+	if m != nil {
+		return m.PeerURLs
+	}
+	return nil
+}
+
+type MemberUpdateResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// members is a list of all members after updating the member.
+	Members              []*Member `protobuf:"bytes,2,rep,name=members,proto3" json:"members,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *MemberUpdateResponse) Reset()         { *m = MemberUpdateResponse{} }
+func (m *MemberUpdateResponse) String() string { return proto.CompactTextString(m) }
+func (*MemberUpdateResponse) ProtoMessage()    {}
+func (*MemberUpdateResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{42}
+}
+func (m *MemberUpdateResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberUpdateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberUpdateResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberUpdateResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberUpdateResponse.Merge(m, src)
+}
+func (m *MemberUpdateResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberUpdateResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberUpdateResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberUpdateResponse proto.InternalMessageInfo
+
+func (m *MemberUpdateResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *MemberUpdateResponse) GetMembers() []*Member {
+	if m != nil {
+		return m.Members
+	}
+	return nil
+}
+
+type MemberListRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MemberListRequest) Reset()         { *m = MemberListRequest{} }
+func (m *MemberListRequest) String() string { return proto.CompactTextString(m) }
+func (*MemberListRequest) ProtoMessage()    {}
+func (*MemberListRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{43}
+}
+func (m *MemberListRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberListRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberListRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberListRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberListRequest.Merge(m, src)
+}
+func (m *MemberListRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberListRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberListRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberListRequest proto.InternalMessageInfo
+
+type MemberListResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// members is a list of all members associated with the cluster.
+	Members              []*Member `protobuf:"bytes,2,rep,name=members,proto3" json:"members,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *MemberListResponse) Reset()         { *m = MemberListResponse{} }
+func (m *MemberListResponse) String() string { return proto.CompactTextString(m) }
+func (*MemberListResponse) ProtoMessage()    {}
+func (*MemberListResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{44}
+}
+func (m *MemberListResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MemberListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MemberListResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MemberListResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MemberListResponse.Merge(m, src)
+}
+func (m *MemberListResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *MemberListResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_MemberListResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MemberListResponse proto.InternalMessageInfo
+
+func (m *MemberListResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *MemberListResponse) GetMembers() []*Member {
+	if m != nil {
+		return m.Members
+	}
+	return nil
+}
+
+type DefragmentRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DefragmentRequest) Reset()         { *m = DefragmentRequest{} }
+func (m *DefragmentRequest) String() string { return proto.CompactTextString(m) }
+func (*DefragmentRequest) ProtoMessage()    {}
+func (*DefragmentRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{45}
+}
+func (m *DefragmentRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *DefragmentRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_DefragmentRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *DefragmentRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DefragmentRequest.Merge(m, src)
+}
+func (m *DefragmentRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *DefragmentRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_DefragmentRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DefragmentRequest proto.InternalMessageInfo
+
+type DefragmentResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *DefragmentResponse) Reset()         { *m = DefragmentResponse{} }
+func (m *DefragmentResponse) String() string { return proto.CompactTextString(m) }
+func (*DefragmentResponse) ProtoMessage()    {}
+func (*DefragmentResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{46}
+}
+func (m *DefragmentResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *DefragmentResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_DefragmentResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *DefragmentResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DefragmentResponse.Merge(m, src)
+}
+func (m *DefragmentResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *DefragmentResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_DefragmentResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DefragmentResponse proto.InternalMessageInfo
+
+func (m *DefragmentResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type MoveLeaderRequest struct {
+	// targetID is the node ID for the new leader.
+	TargetID             uint64   `protobuf:"varint,1,opt,name=targetID,proto3" json:"targetID,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MoveLeaderRequest) Reset()         { *m = MoveLeaderRequest{} }
+func (m *MoveLeaderRequest) String() string { return proto.CompactTextString(m) }
+func (*MoveLeaderRequest) ProtoMessage()    {}
+func (*MoveLeaderRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{47}
+}
+func (m *MoveLeaderRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MoveLeaderRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MoveLeaderRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MoveLeaderRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MoveLeaderRequest.Merge(m, src)
+}
+func (m *MoveLeaderRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *MoveLeaderRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_MoveLeaderRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MoveLeaderRequest proto.InternalMessageInfo
+
+func (m *MoveLeaderRequest) GetTargetID() uint64 {
+	if m != nil {
+		return m.TargetID
+	}
+	return 0
+}
+
+type MoveLeaderResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *MoveLeaderResponse) Reset()         { *m = MoveLeaderResponse{} }
+func (m *MoveLeaderResponse) String() string { return proto.CompactTextString(m) }
+func (*MoveLeaderResponse) ProtoMessage()    {}
+func (*MoveLeaderResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{48}
+}
+func (m *MoveLeaderResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *MoveLeaderResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_MoveLeaderResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *MoveLeaderResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MoveLeaderResponse.Merge(m, src)
+}
+func (m *MoveLeaderResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *MoveLeaderResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_MoveLeaderResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MoveLeaderResponse proto.InternalMessageInfo
+
+func (m *MoveLeaderResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AlarmRequest struct {
+	// action is the kind of alarm request to issue. The action
+	// may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
+	// raised alarm.
+	Action AlarmRequest_AlarmAction `protobuf:"varint,1,opt,name=action,proto3,enum=etcdserverpb.AlarmRequest_AlarmAction" json:"action,omitempty"`
+	// memberID is the ID of the member associated with the alarm. If memberID is 0, the
+	// alarm request covers all members.
+	MemberID uint64 `protobuf:"varint,2,opt,name=memberID,proto3" json:"memberID,omitempty"`
+	// alarm is the type of alarm to consider for this request.
+	Alarm                AlarmType `protobuf:"varint,3,opt,name=alarm,proto3,enum=etcdserverpb.AlarmType" json:"alarm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *AlarmRequest) Reset()         { *m = AlarmRequest{} }
+func (m *AlarmRequest) String() string { return proto.CompactTextString(m) }
+func (*AlarmRequest) ProtoMessage()    {}
+func (*AlarmRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{49}
+}
+func (m *AlarmRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AlarmRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AlarmRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AlarmRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmRequest.Merge(m, src)
+}
+func (m *AlarmRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AlarmRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmRequest proto.InternalMessageInfo
+
+func (m *AlarmRequest) GetAction() AlarmRequest_AlarmAction {
+	if m != nil {
+		return m.Action
+	}
+	return AlarmRequest_GET
+}
+
+func (m *AlarmRequest) GetMemberID() uint64 {
+	if m != nil {
+		return m.MemberID
+	}
+	return 0
+}
+
+func (m *AlarmRequest) GetAlarm() AlarmType {
+	if m != nil {
+		return m.Alarm
+	}
+	return AlarmType_NONE
+}
+
+type AlarmMember struct {
+	// memberID is the ID of the member associated with the raised alarm.
+	MemberID uint64 `protobuf:"varint,1,opt,name=memberID,proto3" json:"memberID,omitempty"`
+	// alarm is the type of alarm which has been raised.
+	Alarm                AlarmType `protobuf:"varint,2,opt,name=alarm,proto3,enum=etcdserverpb.AlarmType" json:"alarm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *AlarmMember) Reset()         { *m = AlarmMember{} }
+func (m *AlarmMember) String() string { return proto.CompactTextString(m) }
+func (*AlarmMember) ProtoMessage()    {}
+func (*AlarmMember) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{50}
+}
+func (m *AlarmMember) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AlarmMember) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AlarmMember.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AlarmMember) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmMember.Merge(m, src)
+}
+func (m *AlarmMember) XXX_Size() int {
+	return m.Size()
+}
+func (m *AlarmMember) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmMember.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmMember proto.InternalMessageInfo
+
+func (m *AlarmMember) GetMemberID() uint64 {
+	if m != nil {
+		return m.MemberID
+	}
+	return 0
+}
+
+func (m *AlarmMember) GetAlarm() AlarmType {
+	if m != nil {
+		return m.Alarm
+	}
+	return AlarmType_NONE
+}
+
+type AlarmResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// alarms is a list of alarms associated with the alarm request.
+	Alarms               []*AlarmMember `protobuf:"bytes,2,rep,name=alarms,proto3" json:"alarms,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *AlarmResponse) Reset()         { *m = AlarmResponse{} }
+func (m *AlarmResponse) String() string { return proto.CompactTextString(m) }
+func (*AlarmResponse) ProtoMessage()    {}
+func (*AlarmResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{51}
+}
+func (m *AlarmResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AlarmResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AlarmResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AlarmResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmResponse.Merge(m, src)
+}
+func (m *AlarmResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AlarmResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmResponse proto.InternalMessageInfo
+
+func (m *AlarmResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AlarmResponse) GetAlarms() []*AlarmMember {
+	if m != nil {
+		return m.Alarms
+	}
+	return nil
+}
+
+type StatusRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *StatusRequest) Reset()         { *m = StatusRequest{} }
+func (m *StatusRequest) String() string { return proto.CompactTextString(m) }
+func (*StatusRequest) ProtoMessage()    {}
+func (*StatusRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{52}
+}
+func (m *StatusRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *StatusRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_StatusRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *StatusRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StatusRequest.Merge(m, src)
+}
+func (m *StatusRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *StatusRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_StatusRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StatusRequest proto.InternalMessageInfo
+
+type StatusResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// version is the cluster protocol version used by the responding member.
+	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
+	// dbSize is the size of the backend database, in bytes, of the responding member.
+	DbSize int64 `protobuf:"varint,3,opt,name=dbSize,proto3" json:"dbSize,omitempty"`
+	// leader is the member ID which the responding member believes is the current leader.
+	Leader uint64 `protobuf:"varint,4,opt,name=leader,proto3" json:"leader,omitempty"`
+	// raftIndex is the current raft index of the responding member.
+	RaftIndex uint64 `protobuf:"varint,5,opt,name=raftIndex,proto3" json:"raftIndex,omitempty"`
+	// raftTerm is the current raft term of the responding member.
+	RaftTerm             uint64   `protobuf:"varint,6,opt,name=raftTerm,proto3" json:"raftTerm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *StatusResponse) Reset()         { *m = StatusResponse{} }
+func (m *StatusResponse) String() string { return proto.CompactTextString(m) }
+func (*StatusResponse) ProtoMessage()    {}
+func (*StatusResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{53}
+}
+func (m *StatusResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *StatusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_StatusResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *StatusResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StatusResponse.Merge(m, src)
+}
+func (m *StatusResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *StatusResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_StatusResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StatusResponse proto.InternalMessageInfo
+
+func (m *StatusResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *StatusResponse) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *StatusResponse) GetDbSize() int64 {
+	if m != nil {
+		return m.DbSize
+	}
+	return 0
+}
+
+func (m *StatusResponse) GetLeader() uint64 {
+	if m != nil {
+		return m.Leader
+	}
+	return 0
+}
+
+func (m *StatusResponse) GetRaftIndex() uint64 {
+	if m != nil {
+		return m.RaftIndex
+	}
+	return 0
+}
+
+func (m *StatusResponse) GetRaftTerm() uint64 {
+	if m != nil {
+		return m.RaftTerm
+	}
+	return 0
+}
+
+type AuthEnableRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthEnableRequest) Reset()         { *m = AuthEnableRequest{} }
+func (m *AuthEnableRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthEnableRequest) ProtoMessage()    {}
+func (*AuthEnableRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{54}
+}
+func (m *AuthEnableRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthEnableRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthEnableRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthEnableRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthEnableRequest.Merge(m, src)
+}
+func (m *AuthEnableRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthEnableRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthEnableRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthEnableRequest proto.InternalMessageInfo
+
+type AuthDisableRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthDisableRequest) Reset()         { *m = AuthDisableRequest{} }
+func (m *AuthDisableRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthDisableRequest) ProtoMessage()    {}
+func (*AuthDisableRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{55}
+}
+func (m *AuthDisableRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthDisableRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthDisableRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthDisableRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthDisableRequest.Merge(m, src)
+}
+func (m *AuthDisableRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthDisableRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthDisableRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthDisableRequest proto.InternalMessageInfo
+
+type AuthenticateRequest struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Password             string   `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthenticateRequest) Reset()         { *m = AuthenticateRequest{} }
+func (m *AuthenticateRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthenticateRequest) ProtoMessage()    {}
+func (*AuthenticateRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{56}
+}
+func (m *AuthenticateRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthenticateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthenticateRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthenticateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthenticateRequest.Merge(m, src)
+}
+func (m *AuthenticateRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthenticateRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthenticateRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthenticateRequest proto.InternalMessageInfo
+
+func (m *AuthenticateRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AuthenticateRequest) GetPassword() string {
+	if m != nil {
+		return m.Password
+	}
+	return ""
+}
+
+type AuthUserAddRequest struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Password             string   `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserAddRequest) Reset()         { *m = AuthUserAddRequest{} }
+func (m *AuthUserAddRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserAddRequest) ProtoMessage()    {}
+func (*AuthUserAddRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{57}
+}
+func (m *AuthUserAddRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserAddRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserAddRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserAddRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserAddRequest.Merge(m, src)
+}
+func (m *AuthUserAddRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserAddRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserAddRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserAddRequest proto.InternalMessageInfo
+
+func (m *AuthUserAddRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AuthUserAddRequest) GetPassword() string {
+	if m != nil {
+		return m.Password
+	}
+	return ""
+}
+
+type AuthUserGetRequest struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserGetRequest) Reset()         { *m = AuthUserGetRequest{} }
+func (m *AuthUserGetRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserGetRequest) ProtoMessage()    {}
+func (*AuthUserGetRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{58}
+}
+func (m *AuthUserGetRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserGetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserGetRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserGetRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserGetRequest.Merge(m, src)
+}
+func (m *AuthUserGetRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserGetRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserGetRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserGetRequest proto.InternalMessageInfo
+
+func (m *AuthUserGetRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+type AuthUserDeleteRequest struct {
+	// name is the name of the user to delete.
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserDeleteRequest) Reset()         { *m = AuthUserDeleteRequest{} }
+func (m *AuthUserDeleteRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserDeleteRequest) ProtoMessage()    {}
+func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{59}
+}
+func (m *AuthUserDeleteRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserDeleteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserDeleteRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserDeleteRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserDeleteRequest.Merge(m, src)
+}
+func (m *AuthUserDeleteRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserDeleteRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserDeleteRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserDeleteRequest proto.InternalMessageInfo
+
+func (m *AuthUserDeleteRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+type AuthUserChangePasswordRequest struct {
+	// name is the name of the user whose password is being changed.
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// password is the new password for the user.
+	Password             string   `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserChangePasswordRequest) Reset()         { *m = AuthUserChangePasswordRequest{} }
+func (m *AuthUserChangePasswordRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserChangePasswordRequest) ProtoMessage()    {}
+func (*AuthUserChangePasswordRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{60}
+}
+func (m *AuthUserChangePasswordRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserChangePasswordRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserChangePasswordRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserChangePasswordRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserChangePasswordRequest.Merge(m, src)
+}
+func (m *AuthUserChangePasswordRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserChangePasswordRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserChangePasswordRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserChangePasswordRequest proto.InternalMessageInfo
+
+func (m *AuthUserChangePasswordRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AuthUserChangePasswordRequest) GetPassword() string {
+	if m != nil {
+		return m.Password
+	}
+	return ""
+}
+
+type AuthUserGrantRoleRequest struct {
+	// user is the name of the user which should be granted a given role.
+	User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"`
+	// role is the name of the role to grant to the user.
+	Role                 string   `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserGrantRoleRequest) Reset()         { *m = AuthUserGrantRoleRequest{} }
+func (m *AuthUserGrantRoleRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserGrantRoleRequest) ProtoMessage()    {}
+func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{61}
+}
+func (m *AuthUserGrantRoleRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserGrantRoleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserGrantRoleRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserGrantRoleRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserGrantRoleRequest.Merge(m, src)
+}
+func (m *AuthUserGrantRoleRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserGrantRoleRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserGrantRoleRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserGrantRoleRequest proto.InternalMessageInfo
+
+func (m *AuthUserGrantRoleRequest) GetUser() string {
+	if m != nil {
+		return m.User
+	}
+	return ""
+}
+
+func (m *AuthUserGrantRoleRequest) GetRole() string {
+	if m != nil {
+		return m.Role
+	}
+	return ""
+}
+
+type AuthUserRevokeRoleRequest struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Role                 string   `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserRevokeRoleRequest) Reset()         { *m = AuthUserRevokeRoleRequest{} }
+func (m *AuthUserRevokeRoleRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserRevokeRoleRequest) ProtoMessage()    {}
+func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{62}
+}
+func (m *AuthUserRevokeRoleRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserRevokeRoleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserRevokeRoleRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserRevokeRoleRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserRevokeRoleRequest.Merge(m, src)
+}
+func (m *AuthUserRevokeRoleRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserRevokeRoleRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserRevokeRoleRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserRevokeRoleRequest proto.InternalMessageInfo
+
+func (m *AuthUserRevokeRoleRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AuthUserRevokeRoleRequest) GetRole() string {
+	if m != nil {
+		return m.Role
+	}
+	return ""
+}
+
+type AuthRoleAddRequest struct {
+	// name is the name of the role to add to the authentication system.
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthRoleAddRequest) Reset()         { *m = AuthRoleAddRequest{} }
+func (m *AuthRoleAddRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleAddRequest) ProtoMessage()    {}
+func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{63}
+}
+func (m *AuthRoleAddRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleAddRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleAddRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleAddRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleAddRequest.Merge(m, src)
+}
+func (m *AuthRoleAddRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleAddRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleAddRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleAddRequest proto.InternalMessageInfo
+
+func (m *AuthRoleAddRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+type AuthRoleGetRequest struct {
+	Role                 string   `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthRoleGetRequest) Reset()         { *m = AuthRoleGetRequest{} }
+func (m *AuthRoleGetRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleGetRequest) ProtoMessage()    {}
+func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{64}
+}
+func (m *AuthRoleGetRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleGetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleGetRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleGetRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleGetRequest.Merge(m, src)
+}
+func (m *AuthRoleGetRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleGetRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleGetRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleGetRequest proto.InternalMessageInfo
+
+func (m *AuthRoleGetRequest) GetRole() string {
+	if m != nil {
+		return m.Role
+	}
+	return ""
+}
+
+type AuthUserListRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthUserListRequest) Reset()         { *m = AuthUserListRequest{} }
+func (m *AuthUserListRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthUserListRequest) ProtoMessage()    {}
+func (*AuthUserListRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{65}
+}
+func (m *AuthUserListRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserListRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserListRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserListRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserListRequest.Merge(m, src)
+}
+func (m *AuthUserListRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserListRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserListRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserListRequest proto.InternalMessageInfo
+
+type AuthRoleListRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthRoleListRequest) Reset()         { *m = AuthRoleListRequest{} }
+func (m *AuthRoleListRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleListRequest) ProtoMessage()    {}
+func (*AuthRoleListRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{66}
+}
+func (m *AuthRoleListRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleListRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleListRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleListRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleListRequest.Merge(m, src)
+}
+func (m *AuthRoleListRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleListRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleListRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleListRequest proto.InternalMessageInfo
+
+type AuthRoleDeleteRequest struct {
+	Role                 string   `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthRoleDeleteRequest) Reset()         { *m = AuthRoleDeleteRequest{} }
+func (m *AuthRoleDeleteRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleDeleteRequest) ProtoMessage()    {}
+func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{67}
+}
+func (m *AuthRoleDeleteRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleDeleteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleDeleteRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleDeleteRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleDeleteRequest.Merge(m, src)
+}
+func (m *AuthRoleDeleteRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleDeleteRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleDeleteRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleDeleteRequest proto.InternalMessageInfo
+
+func (m *AuthRoleDeleteRequest) GetRole() string {
+	if m != nil {
+		return m.Role
+	}
+	return ""
+}
+
+type AuthRoleGrantPermissionRequest struct {
+	// name is the name of the role which will be granted the permission.
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// perm is the permission to grant to the role.
+	Perm                 *authpb.Permission `protobuf:"bytes,2,opt,name=perm,proto3" json:"perm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *AuthRoleGrantPermissionRequest) Reset()         { *m = AuthRoleGrantPermissionRequest{} }
+func (m *AuthRoleGrantPermissionRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleGrantPermissionRequest) ProtoMessage()    {}
+func (*AuthRoleGrantPermissionRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{68}
+}
+func (m *AuthRoleGrantPermissionRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleGrantPermissionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleGrantPermissionRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleGrantPermissionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleGrantPermissionRequest.Merge(m, src)
+}
+func (m *AuthRoleGrantPermissionRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleGrantPermissionRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleGrantPermissionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleGrantPermissionRequest proto.InternalMessageInfo
+
+func (m *AuthRoleGrantPermissionRequest) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AuthRoleGrantPermissionRequest) GetPerm() *authpb.Permission {
+	if m != nil {
+		return m.Perm
+	}
+	return nil
+}
+
+type AuthRoleRevokePermissionRequest struct {
+	Role                 string   `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
+	Key                  string   `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
+	RangeEnd             string   `protobuf:"bytes,3,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthRoleRevokePermissionRequest) Reset()         { *m = AuthRoleRevokePermissionRequest{} }
+func (m *AuthRoleRevokePermissionRequest) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleRevokePermissionRequest) ProtoMessage()    {}
+func (*AuthRoleRevokePermissionRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{69}
+}
+func (m *AuthRoleRevokePermissionRequest) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleRevokePermissionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleRevokePermissionRequest.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleRevokePermissionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleRevokePermissionRequest.Merge(m, src)
+}
+func (m *AuthRoleRevokePermissionRequest) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleRevokePermissionRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleRevokePermissionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleRevokePermissionRequest proto.InternalMessageInfo
+
+func (m *AuthRoleRevokePermissionRequest) GetRole() string {
+	if m != nil {
+		return m.Role
+	}
+	return ""
+}
+
+func (m *AuthRoleRevokePermissionRequest) GetKey() string {
+	if m != nil {
+		return m.Key
+	}
+	return ""
+}
+
+func (m *AuthRoleRevokePermissionRequest) GetRangeEnd() string {
+	if m != nil {
+		return m.RangeEnd
+	}
+	return ""
+}
+
+type AuthEnableResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthEnableResponse) Reset()         { *m = AuthEnableResponse{} }
+func (m *AuthEnableResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthEnableResponse) ProtoMessage()    {}
+func (*AuthEnableResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{70}
+}
+func (m *AuthEnableResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthEnableResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthEnableResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthEnableResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthEnableResponse.Merge(m, src)
+}
+func (m *AuthEnableResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthEnableResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthEnableResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthEnableResponse proto.InternalMessageInfo
+
+func (m *AuthEnableResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthDisableResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthDisableResponse) Reset()         { *m = AuthDisableResponse{} }
+func (m *AuthDisableResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthDisableResponse) ProtoMessage()    {}
+func (*AuthDisableResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{71}
+}
+func (m *AuthDisableResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthDisableResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthDisableResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthDisableResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthDisableResponse.Merge(m, src)
+}
+func (m *AuthDisableResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthDisableResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthDisableResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthDisableResponse proto.InternalMessageInfo
+
+func (m *AuthDisableResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthenticateResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// token is an authorized token that can be used in succeeding RPCs
+	Token                string   `protobuf:"bytes,2,opt,name=token,proto3" json:"token,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AuthenticateResponse) Reset()         { *m = AuthenticateResponse{} }
+func (m *AuthenticateResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthenticateResponse) ProtoMessage()    {}
+func (*AuthenticateResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{72}
+}
+func (m *AuthenticateResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthenticateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthenticateResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthenticateResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthenticateResponse.Merge(m, src)
+}
+func (m *AuthenticateResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthenticateResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthenticateResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthenticateResponse proto.InternalMessageInfo
+
+func (m *AuthenticateResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AuthenticateResponse) GetToken() string {
+	if m != nil {
+		return m.Token
+	}
+	return ""
+}
+
+type AuthUserAddResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserAddResponse) Reset()         { *m = AuthUserAddResponse{} }
+func (m *AuthUserAddResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserAddResponse) ProtoMessage()    {}
+func (*AuthUserAddResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{73}
+}
+func (m *AuthUserAddResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserAddResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserAddResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserAddResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserAddResponse.Merge(m, src)
+}
+func (m *AuthUserAddResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserAddResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserAddResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserAddResponse proto.InternalMessageInfo
+
+func (m *AuthUserAddResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthUserGetResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Roles                []string        `protobuf:"bytes,2,rep,name=roles,proto3" json:"roles,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserGetResponse) Reset()         { *m = AuthUserGetResponse{} }
+func (m *AuthUserGetResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserGetResponse) ProtoMessage()    {}
+func (*AuthUserGetResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{74}
+}
+func (m *AuthUserGetResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserGetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserGetResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserGetResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserGetResponse.Merge(m, src)
+}
+func (m *AuthUserGetResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserGetResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserGetResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserGetResponse proto.InternalMessageInfo
+
+func (m *AuthUserGetResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AuthUserGetResponse) GetRoles() []string {
+	if m != nil {
+		return m.Roles
+	}
+	return nil
+}
+
+type AuthUserDeleteResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserDeleteResponse) Reset()         { *m = AuthUserDeleteResponse{} }
+func (m *AuthUserDeleteResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserDeleteResponse) ProtoMessage()    {}
+func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{75}
+}
+func (m *AuthUserDeleteResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserDeleteResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserDeleteResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserDeleteResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserDeleteResponse.Merge(m, src)
+}
+func (m *AuthUserDeleteResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserDeleteResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserDeleteResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserDeleteResponse proto.InternalMessageInfo
+
+func (m *AuthUserDeleteResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthUserChangePasswordResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserChangePasswordResponse) Reset()         { *m = AuthUserChangePasswordResponse{} }
+func (m *AuthUserChangePasswordResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserChangePasswordResponse) ProtoMessage()    {}
+func (*AuthUserChangePasswordResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{76}
+}
+func (m *AuthUserChangePasswordResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserChangePasswordResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserChangePasswordResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserChangePasswordResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserChangePasswordResponse.Merge(m, src)
+}
+func (m *AuthUserChangePasswordResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserChangePasswordResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserChangePasswordResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserChangePasswordResponse proto.InternalMessageInfo
+
+func (m *AuthUserChangePasswordResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthUserGrantRoleResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserGrantRoleResponse) Reset()         { *m = AuthUserGrantRoleResponse{} }
+func (m *AuthUserGrantRoleResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserGrantRoleResponse) ProtoMessage()    {}
+func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{77}
+}
+func (m *AuthUserGrantRoleResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserGrantRoleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserGrantRoleResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserGrantRoleResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserGrantRoleResponse.Merge(m, src)
+}
+func (m *AuthUserGrantRoleResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserGrantRoleResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserGrantRoleResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserGrantRoleResponse proto.InternalMessageInfo
+
+func (m *AuthUserGrantRoleResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthUserRevokeRoleResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserRevokeRoleResponse) Reset()         { *m = AuthUserRevokeRoleResponse{} }
+func (m *AuthUserRevokeRoleResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserRevokeRoleResponse) ProtoMessage()    {}
+func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{78}
+}
+func (m *AuthUserRevokeRoleResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserRevokeRoleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserRevokeRoleResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserRevokeRoleResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserRevokeRoleResponse.Merge(m, src)
+}
+func (m *AuthUserRevokeRoleResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserRevokeRoleResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserRevokeRoleResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserRevokeRoleResponse proto.InternalMessageInfo
+
+func (m *AuthUserRevokeRoleResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthRoleAddResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthRoleAddResponse) Reset()         { *m = AuthRoleAddResponse{} }
+func (m *AuthRoleAddResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleAddResponse) ProtoMessage()    {}
+func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{79}
+}
+func (m *AuthRoleAddResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleAddResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleAddResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleAddResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleAddResponse.Merge(m, src)
+}
+func (m *AuthRoleAddResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleAddResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleAddResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleAddResponse proto.InternalMessageInfo
+
+func (m *AuthRoleAddResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthRoleGetResponse struct {
+	Header               *ResponseHeader      `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Perm                 []*authpb.Permission `protobuf:"bytes,2,rep,name=perm,proto3" json:"perm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *AuthRoleGetResponse) Reset()         { *m = AuthRoleGetResponse{} }
+func (m *AuthRoleGetResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleGetResponse) ProtoMessage()    {}
+func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{80}
+}
+func (m *AuthRoleGetResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleGetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleGetResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleGetResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleGetResponse.Merge(m, src)
+}
+func (m *AuthRoleGetResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleGetResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleGetResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleGetResponse proto.InternalMessageInfo
+
+func (m *AuthRoleGetResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AuthRoleGetResponse) GetPerm() []*authpb.Permission {
+	if m != nil {
+		return m.Perm
+	}
+	return nil
+}
+
+type AuthRoleListResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Roles                []string        `protobuf:"bytes,2,rep,name=roles,proto3" json:"roles,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthRoleListResponse) Reset()         { *m = AuthRoleListResponse{} }
+func (m *AuthRoleListResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleListResponse) ProtoMessage()    {}
+func (*AuthRoleListResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{81}
+}
+func (m *AuthRoleListResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleListResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleListResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleListResponse.Merge(m, src)
+}
+func (m *AuthRoleListResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleListResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleListResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleListResponse proto.InternalMessageInfo
+
+func (m *AuthRoleListResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AuthRoleListResponse) GetRoles() []string {
+	if m != nil {
+		return m.Roles
+	}
+	return nil
+}
+
+type AuthUserListResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Users                []string        `protobuf:"bytes,2,rep,name=users,proto3" json:"users,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthUserListResponse) Reset()         { *m = AuthUserListResponse{} }
+func (m *AuthUserListResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthUserListResponse) ProtoMessage()    {}
+func (*AuthUserListResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{82}
+}
+func (m *AuthUserListResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthUserListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthUserListResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthUserListResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthUserListResponse.Merge(m, src)
+}
+func (m *AuthUserListResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthUserListResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthUserListResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthUserListResponse proto.InternalMessageInfo
+
+func (m *AuthUserListResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *AuthUserListResponse) GetUsers() []string {
+	if m != nil {
+		return m.Users
+	}
+	return nil
+}
+
+type AuthRoleDeleteResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthRoleDeleteResponse) Reset()         { *m = AuthRoleDeleteResponse{} }
+func (m *AuthRoleDeleteResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleDeleteResponse) ProtoMessage()    {}
+func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{83}
+}
+func (m *AuthRoleDeleteResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleDeleteResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleDeleteResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleDeleteResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleDeleteResponse.Merge(m, src)
+}
+func (m *AuthRoleDeleteResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleDeleteResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleDeleteResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleDeleteResponse proto.InternalMessageInfo
+
+func (m *AuthRoleDeleteResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthRoleGrantPermissionResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthRoleGrantPermissionResponse) Reset()         { *m = AuthRoleGrantPermissionResponse{} }
+func (m *AuthRoleGrantPermissionResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleGrantPermissionResponse) ProtoMessage()    {}
+func (*AuthRoleGrantPermissionResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{84}
+}
+func (m *AuthRoleGrantPermissionResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleGrantPermissionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleGrantPermissionResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleGrantPermissionResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleGrantPermissionResponse.Merge(m, src)
+}
+func (m *AuthRoleGrantPermissionResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleGrantPermissionResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleGrantPermissionResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleGrantPermissionResponse proto.InternalMessageInfo
+
+func (m *AuthRoleGrantPermissionResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type AuthRoleRevokePermissionResponse struct {
+	Header               *ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *AuthRoleRevokePermissionResponse) Reset()         { *m = AuthRoleRevokePermissionResponse{} }
+func (m *AuthRoleRevokePermissionResponse) String() string { return proto.CompactTextString(m) }
+func (*AuthRoleRevokePermissionResponse) ProtoMessage()    {}
+func (*AuthRoleRevokePermissionResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_77a6da22d6a3feb1, []int{85}
+}
+func (m *AuthRoleRevokePermissionResponse) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AuthRoleRevokePermissionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AuthRoleRevokePermissionResponse.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AuthRoleRevokePermissionResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AuthRoleRevokePermissionResponse.Merge(m, src)
+}
+func (m *AuthRoleRevokePermissionResponse) XXX_Size() int {
+	return m.Size()
+}
+func (m *AuthRoleRevokePermissionResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_AuthRoleRevokePermissionResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AuthRoleRevokePermissionResponse proto.InternalMessageInfo
+
+func (m *AuthRoleRevokePermissionResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("etcdserverpb.AlarmType", AlarmType_name, AlarmType_value)
+	proto.RegisterEnum("etcdserverpb.RangeRequest_SortOrder", RangeRequest_SortOrder_name, RangeRequest_SortOrder_value)
+	proto.RegisterEnum("etcdserverpb.RangeRequest_SortTarget", RangeRequest_SortTarget_name, RangeRequest_SortTarget_value)
+	proto.RegisterEnum("etcdserverpb.Compare_CompareResult", Compare_CompareResult_name, Compare_CompareResult_value)
+	proto.RegisterEnum("etcdserverpb.Compare_CompareTarget", Compare_CompareTarget_name, Compare_CompareTarget_value)
+	proto.RegisterEnum("etcdserverpb.WatchCreateRequest_FilterType", WatchCreateRequest_FilterType_name, WatchCreateRequest_FilterType_value)
+	proto.RegisterEnum("etcdserverpb.AlarmRequest_AlarmAction", AlarmRequest_AlarmAction_name, AlarmRequest_AlarmAction_value)
+	proto.RegisterType((*ResponseHeader)(nil), "etcdserverpb.ResponseHeader")
+	proto.RegisterType((*RangeRequest)(nil), "etcdserverpb.RangeRequest")
+	proto.RegisterType((*RangeResponse)(nil), "etcdserverpb.RangeResponse")
+	proto.RegisterType((*PutRequest)(nil), "etcdserverpb.PutRequest")
+	proto.RegisterType((*PutResponse)(nil), "etcdserverpb.PutResponse")
+	proto.RegisterType((*DeleteRangeRequest)(nil), "etcdserverpb.DeleteRangeRequest")
+	proto.RegisterType((*DeleteRangeResponse)(nil), "etcdserverpb.DeleteRangeResponse")
+	proto.RegisterType((*RequestOp)(nil), "etcdserverpb.RequestOp")
+	proto.RegisterType((*ResponseOp)(nil), "etcdserverpb.ResponseOp")
+	proto.RegisterType((*Compare)(nil), "etcdserverpb.Compare")
+	proto.RegisterType((*TxnRequest)(nil), "etcdserverpb.TxnRequest")
+	proto.RegisterType((*TxnResponse)(nil), "etcdserverpb.TxnResponse")
+	proto.RegisterType((*CompactionRequest)(nil), "etcdserverpb.CompactionRequest")
+	proto.RegisterType((*CompactionResponse)(nil), "etcdserverpb.CompactionResponse")
+	proto.RegisterType((*HashRequest)(nil), "etcdserverpb.HashRequest")
+	proto.RegisterType((*HashKVRequest)(nil), "etcdserverpb.HashKVRequest")
+	proto.RegisterType((*HashKVResponse)(nil), "etcdserverpb.HashKVResponse")
+	proto.RegisterType((*HashResponse)(nil), "etcdserverpb.HashResponse")
+	proto.RegisterType((*SnapshotRequest)(nil), "etcdserverpb.SnapshotRequest")
+	proto.RegisterType((*SnapshotResponse)(nil), "etcdserverpb.SnapshotResponse")
+	proto.RegisterType((*WatchRequest)(nil), "etcdserverpb.WatchRequest")
+	proto.RegisterType((*WatchCreateRequest)(nil), "etcdserverpb.WatchCreateRequest")
+	proto.RegisterType((*WatchCancelRequest)(nil), "etcdserverpb.WatchCancelRequest")
+	proto.RegisterType((*WatchProgressRequest)(nil), "etcdserverpb.WatchProgressRequest")
+	proto.RegisterType((*WatchResponse)(nil), "etcdserverpb.WatchResponse")
+	proto.RegisterType((*LeaseGrantRequest)(nil), "etcdserverpb.LeaseGrantRequest")
+	proto.RegisterType((*LeaseGrantResponse)(nil), "etcdserverpb.LeaseGrantResponse")
+	proto.RegisterType((*LeaseRevokeRequest)(nil), "etcdserverpb.LeaseRevokeRequest")
+	proto.RegisterType((*LeaseRevokeResponse)(nil), "etcdserverpb.LeaseRevokeResponse")
+	proto.RegisterType((*LeaseKeepAliveRequest)(nil), "etcdserverpb.LeaseKeepAliveRequest")
+	proto.RegisterType((*LeaseKeepAliveResponse)(nil), "etcdserverpb.LeaseKeepAliveResponse")
+	proto.RegisterType((*LeaseTimeToLiveRequest)(nil), "etcdserverpb.LeaseTimeToLiveRequest")
+	proto.RegisterType((*LeaseTimeToLiveResponse)(nil), "etcdserverpb.LeaseTimeToLiveResponse")
+	proto.RegisterType((*LeaseLeasesRequest)(nil), "etcdserverpb.LeaseLeasesRequest")
+	proto.RegisterType((*LeaseStatus)(nil), "etcdserverpb.LeaseStatus")
+	proto.RegisterType((*LeaseLeasesResponse)(nil), "etcdserverpb.LeaseLeasesResponse")
+	proto.RegisterType((*Member)(nil), "etcdserverpb.Member")
+	proto.RegisterType((*MemberAddRequest)(nil), "etcdserverpb.MemberAddRequest")
+	proto.RegisterType((*MemberAddResponse)(nil), "etcdserverpb.MemberAddResponse")
+	proto.RegisterType((*MemberRemoveRequest)(nil), "etcdserverpb.MemberRemoveRequest")
+	proto.RegisterType((*MemberRemoveResponse)(nil), "etcdserverpb.MemberRemoveResponse")
+	proto.RegisterType((*MemberUpdateRequest)(nil), "etcdserverpb.MemberUpdateRequest")
+	proto.RegisterType((*MemberUpdateResponse)(nil), "etcdserverpb.MemberUpdateResponse")
+	proto.RegisterType((*MemberListRequest)(nil), "etcdserverpb.MemberListRequest")
+	proto.RegisterType((*MemberListResponse)(nil), "etcdserverpb.MemberListResponse")
+	proto.RegisterType((*DefragmentRequest)(nil), "etcdserverpb.DefragmentRequest")
+	proto.RegisterType((*DefragmentResponse)(nil), "etcdserverpb.DefragmentResponse")
+	proto.RegisterType((*MoveLeaderRequest)(nil), "etcdserverpb.MoveLeaderRequest")
+	proto.RegisterType((*MoveLeaderResponse)(nil), "etcdserverpb.MoveLeaderResponse")
+	proto.RegisterType((*AlarmRequest)(nil), "etcdserverpb.AlarmRequest")
+	proto.RegisterType((*AlarmMember)(nil), "etcdserverpb.AlarmMember")
+	proto.RegisterType((*AlarmResponse)(nil), "etcdserverpb.AlarmResponse")
+	proto.RegisterType((*StatusRequest)(nil), "etcdserverpb.StatusRequest")
+	proto.RegisterType((*StatusResponse)(nil), "etcdserverpb.StatusResponse")
+	proto.RegisterType((*AuthEnableRequest)(nil), "etcdserverpb.AuthEnableRequest")
+	proto.RegisterType((*AuthDisableRequest)(nil), "etcdserverpb.AuthDisableRequest")
+	proto.RegisterType((*AuthenticateRequest)(nil), "etcdserverpb.AuthenticateRequest")
+	proto.RegisterType((*AuthUserAddRequest)(nil), "etcdserverpb.AuthUserAddRequest")
+	proto.RegisterType((*AuthUserGetRequest)(nil), "etcdserverpb.AuthUserGetRequest")
+	proto.RegisterType((*AuthUserDeleteRequest)(nil), "etcdserverpb.AuthUserDeleteRequest")
+	proto.RegisterType((*AuthUserChangePasswordRequest)(nil), "etcdserverpb.AuthUserChangePasswordRequest")
+	proto.RegisterType((*AuthUserGrantRoleRequest)(nil), "etcdserverpb.AuthUserGrantRoleRequest")
+	proto.RegisterType((*AuthUserRevokeRoleRequest)(nil), "etcdserverpb.AuthUserRevokeRoleRequest")
+	proto.RegisterType((*AuthRoleAddRequest)(nil), "etcdserverpb.AuthRoleAddRequest")
+	proto.RegisterType((*AuthRoleGetRequest)(nil), "etcdserverpb.AuthRoleGetRequest")
+	proto.RegisterType((*AuthUserListRequest)(nil), "etcdserverpb.AuthUserListRequest")
+	proto.RegisterType((*AuthRoleListRequest)(nil), "etcdserverpb.AuthRoleListRequest")
+	proto.RegisterType((*AuthRoleDeleteRequest)(nil), "etcdserverpb.AuthRoleDeleteRequest")
+	proto.RegisterType((*AuthRoleGrantPermissionRequest)(nil), "etcdserverpb.AuthRoleGrantPermissionRequest")
+	proto.RegisterType((*AuthRoleRevokePermissionRequest)(nil), "etcdserverpb.AuthRoleRevokePermissionRequest")
+	proto.RegisterType((*AuthEnableResponse)(nil), "etcdserverpb.AuthEnableResponse")
+	proto.RegisterType((*AuthDisableResponse)(nil), "etcdserverpb.AuthDisableResponse")
+	proto.RegisterType((*AuthenticateResponse)(nil), "etcdserverpb.AuthenticateResponse")
+	proto.RegisterType((*AuthUserAddResponse)(nil), "etcdserverpb.AuthUserAddResponse")
+	proto.RegisterType((*AuthUserGetResponse)(nil), "etcdserverpb.AuthUserGetResponse")
+	proto.RegisterType((*AuthUserDeleteResponse)(nil), "etcdserverpb.AuthUserDeleteResponse")
+	proto.RegisterType((*AuthUserChangePasswordResponse)(nil), "etcdserverpb.AuthUserChangePasswordResponse")
+	proto.RegisterType((*AuthUserGrantRoleResponse)(nil), "etcdserverpb.AuthUserGrantRoleResponse")
+	proto.RegisterType((*AuthUserRevokeRoleResponse)(nil), "etcdserverpb.AuthUserRevokeRoleResponse")
+	proto.RegisterType((*AuthRoleAddResponse)(nil), "etcdserverpb.AuthRoleAddResponse")
+	proto.RegisterType((*AuthRoleGetResponse)(nil), "etcdserverpb.AuthRoleGetResponse")
+	proto.RegisterType((*AuthRoleListResponse)(nil), "etcdserverpb.AuthRoleListResponse")
+	proto.RegisterType((*AuthUserListResponse)(nil), "etcdserverpb.AuthUserListResponse")
+	proto.RegisterType((*AuthRoleDeleteResponse)(nil), "etcdserverpb.AuthRoleDeleteResponse")
+	proto.RegisterType((*AuthRoleGrantPermissionResponse)(nil), "etcdserverpb.AuthRoleGrantPermissionResponse")
+	proto.RegisterType((*AuthRoleRevokePermissionResponse)(nil), "etcdserverpb.AuthRoleRevokePermissionResponse")
+}
+
+func init() { proto.RegisterFile("rpc.proto", fileDescriptor_77a6da22d6a3feb1) }
+
+var fileDescriptor_77a6da22d6a3feb1 = []byte{
+	// 3711 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0x5b, 0x73, 0x1b, 0x47,
+	0x76, 0xe6, 0x00, 0x04, 0x40, 0x1c, 0x5c, 0x08, 0x35, 0x29, 0x09, 0x84, 0x24, 0x8a, 0x6a, 0xdd,
+	0xa8, 0x8b, 0x09, 0x9b, 0x76, 0xf2, 0xa0, 0xa4, 0x5c, 0xa6, 0x48, 0x58, 0xa4, 0x49, 0x91, 0xf4,
+	0x10, 0x94, 0x9d, 0x2a, 0x27, 0xac, 0x21, 0xd0, 0x22, 0x11, 0x02, 0x33, 0xc8, 0xcc, 0x00, 0x22,
+	0x15, 0x57, 0x52, 0xe5, 0x38, 0xae, 0x3c, 0xc7, 0x55, 0xa9, 0x24, 0xaf, 0x5b, 0x5b, 0x2e, 0xff,
+	0x82, 0xfd, 0x0b, 0x5b, 0xfb, 0xb2, 0xbb, 0xb5, 0x7f, 0x60, 0xcb, 0xbb, 0x2f, 0xfb, 0x0b, 0xf6,
+	0xf2, 0xb4, 0xd5, 0xb7, 0x99, 0x9e, 0x1b, 0x48, 0x1b, 0xb6, 0x5f, 0xa8, 0xe9, 0xd3, 0xa7, 0xcf,
+	0x39, 0x7d, 0xba, 0xcf, 0x39, 0xdd, 0x5f, 0x43, 0x90, 0xb7, 0xfb, 0xad, 0xa5, 0xbe, 0x6d, 0xb9,
+	0x16, 0x2a, 0x12, 0xb7, 0xd5, 0x76, 0x88, 0x3d, 0x24, 0x76, 0xff, 0xb0, 0x36, 0x7b, 0x64, 0x1d,
+	0x59, 0xac, 0xa3, 0x4e, 0xbf, 0x38, 0x4f, 0x6d, 0x8e, 0xf2, 0xd4, 0x7b, 0xc3, 0x56, 0x8b, 0xfd,
+	0xe9, 0x1f, 0xd6, 0x4f, 0x86, 0xa2, 0xeb, 0x1a, 0xeb, 0x32, 0x06, 0xee, 0x31, 0xfb, 0xd3, 0x3f,
+	0x64, 0xff, 0x88, 0xce, 0xeb, 0x47, 0x96, 0x75, 0xd4, 0x25, 0x75, 0xa3, 0xdf, 0xa9, 0x1b, 0xa6,
+	0x69, 0xb9, 0x86, 0xdb, 0xb1, 0x4c, 0x87, 0xf7, 0xe2, 0xff, 0xd4, 0xa0, 0xac, 0x13, 0xa7, 0x6f,
+	0x99, 0x0e, 0x59, 0x27, 0x46, 0x9b, 0xd8, 0xe8, 0x06, 0x40, 0xab, 0x3b, 0x70, 0x5c, 0x62, 0x1f,
+	0x74, 0xda, 0x55, 0x6d, 0x41, 0x5b, 0x9c, 0xd4, 0xf3, 0x82, 0xb2, 0xd1, 0x46, 0xd7, 0x20, 0xdf,
+	0x23, 0xbd, 0x43, 0xde, 0x9b, 0x62, 0xbd, 0x53, 0x9c, 0xb0, 0xd1, 0x46, 0x35, 0x98, 0xb2, 0xc9,
+	0xb0, 0xe3, 0x74, 0x2c, 0xb3, 0x9a, 0x5e, 0xd0, 0x16, 0xd3, 0xba, 0xd7, 0xa6, 0x03, 0x6d, 0xe3,
+	0xa5, 0x7b, 0xe0, 0x12, 0xbb, 0x57, 0x9d, 0xe4, 0x03, 0x29, 0xa1, 0x49, 0xec, 0x1e, 0xfe, 0x3c,
+	0x03, 0x45, 0xdd, 0x30, 0x8f, 0x88, 0x4e, 0xfe, 0x65, 0x40, 0x1c, 0x17, 0x55, 0x20, 0x7d, 0x42,
+	0xce, 0x98, 0xfa, 0xa2, 0x4e, 0x3f, 0xf9, 0x78, 0xf3, 0x88, 0x1c, 0x10, 0x93, 0x2b, 0x2e, 0xd2,
+	0xf1, 0xe6, 0x11, 0x69, 0x98, 0x6d, 0x34, 0x0b, 0x99, 0x6e, 0xa7, 0xd7, 0x71, 0x85, 0x56, 0xde,
+	0x08, 0x98, 0x33, 0x19, 0x32, 0x67, 0x15, 0xc0, 0xb1, 0x6c, 0xf7, 0xc0, 0xb2, 0xdb, 0xc4, 0xae,
+	0x66, 0x16, 0xb4, 0xc5, 0xf2, 0xf2, 0x9d, 0x25, 0x75, 0x21, 0x96, 0x54, 0x83, 0x96, 0xf6, 0x2c,
+	0xdb, 0xdd, 0xa1, 0xbc, 0x7a, 0xde, 0x91, 0x9f, 0xe8, 0x7d, 0x28, 0x30, 0x21, 0xae, 0x61, 0x1f,
+	0x11, 0xb7, 0x9a, 0x65, 0x52, 0xee, 0x9e, 0x23, 0xa5, 0xc9, 0x98, 0x75, 0xa6, 0x9e, 0x7f, 0x23,
+	0x0c, 0x45, 0x87, 0xd8, 0x1d, 0xa3, 0xdb, 0x79, 0x6d, 0x1c, 0x76, 0x49, 0x35, 0xb7, 0xa0, 0x2d,
+	0x4e, 0xe9, 0x01, 0x1a, 0x9d, 0xff, 0x09, 0x39, 0x73, 0x0e, 0x2c, 0xb3, 0x7b, 0x56, 0x9d, 0x62,
+	0x0c, 0x53, 0x94, 0xb0, 0x63, 0x76, 0xcf, 0xd8, 0xa2, 0x59, 0x03, 0xd3, 0xe5, 0xbd, 0x79, 0xd6,
+	0x9b, 0x67, 0x14, 0xd6, 0xbd, 0x08, 0x95, 0x5e, 0xc7, 0x3c, 0xe8, 0x59, 0xed, 0x03, 0xcf, 0x21,
+	0xc0, 0x1c, 0x52, 0xee, 0x75, 0xcc, 0xe7, 0x56, 0x5b, 0x97, 0x6e, 0xa1, 0x9c, 0xc6, 0x69, 0x90,
+	0xb3, 0x20, 0x38, 0x8d, 0x53, 0x95, 0x73, 0x09, 0x66, 0xa8, 0xcc, 0x96, 0x4d, 0x0c, 0x97, 0xf8,
+	0xcc, 0x45, 0xc6, 0x7c, 0xa9, 0xd7, 0x31, 0x57, 0x59, 0x4f, 0x80, 0xdf, 0x38, 0x8d, 0xf0, 0x97,
+	0x04, 0xbf, 0x71, 0x1a, 0xe4, 0xc7, 0x4b, 0x90, 0xf7, 0x7c, 0x8e, 0xa6, 0x60, 0x72, 0x7b, 0x67,
+	0xbb, 0x51, 0x99, 0x40, 0x00, 0xd9, 0x95, 0xbd, 0xd5, 0xc6, 0xf6, 0x5a, 0x45, 0x43, 0x05, 0xc8,
+	0xad, 0x35, 0x78, 0x23, 0x85, 0x9f, 0x02, 0xf8, 0xde, 0x45, 0x39, 0x48, 0x6f, 0x36, 0xfe, 0xa1,
+	0x32, 0x41, 0x79, 0x5e, 0x34, 0xf4, 0xbd, 0x8d, 0x9d, 0xed, 0x8a, 0x46, 0x07, 0xaf, 0xea, 0x8d,
+	0x95, 0x66, 0xa3, 0x92, 0xa2, 0x1c, 0xcf, 0x77, 0xd6, 0x2a, 0x69, 0x94, 0x87, 0xcc, 0x8b, 0x95,
+	0xad, 0xfd, 0x46, 0x65, 0x12, 0x7f, 0xa9, 0x41, 0x49, 0xac, 0x17, 0x8f, 0x09, 0xf4, 0x0e, 0x64,
+	0x8f, 0x59, 0x5c, 0xb0, 0xad, 0x58, 0x58, 0xbe, 0x1e, 0x5a, 0xdc, 0x40, 0xec, 0xe8, 0x82, 0x17,
+	0x61, 0x48, 0x9f, 0x0c, 0x9d, 0x6a, 0x6a, 0x21, 0xbd, 0x58, 0x58, 0xae, 0x2c, 0xf1, 0x80, 0x5d,
+	0xda, 0x24, 0x67, 0x2f, 0x8c, 0xee, 0x80, 0xe8, 0xb4, 0x13, 0x21, 0x98, 0xec, 0x59, 0x36, 0x61,
+	0x3b, 0x76, 0x4a, 0x67, 0xdf, 0x74, 0x1b, 0xb3, 0x45, 0x13, 0xbb, 0x95, 0x37, 0xf0, 0xd7, 0x1a,
+	0xc0, 0xee, 0xc0, 0x4d, 0x0e, 0x8d, 0x59, 0xc8, 0x0c, 0xa9, 0x60, 0x11, 0x16, 0xbc, 0xc1, 0x62,
+	0x82, 0x18, 0x0e, 0xf1, 0x62, 0x82, 0x36, 0xd0, 0x55, 0xc8, 0xf5, 0x6d, 0x32, 0x3c, 0x38, 0x19,
+	0x32, 0x25, 0x53, 0x7a, 0x96, 0x36, 0x37, 0x87, 0xe8, 0x16, 0x14, 0x3b, 0x47, 0xa6, 0x65, 0x93,
+	0x03, 0x2e, 0x2b, 0xc3, 0x7a, 0x0b, 0x9c, 0xc6, 0xec, 0x56, 0x58, 0xb8, 0xe0, 0xac, 0xca, 0xb2,
+	0x45, 0x49, 0xd8, 0x84, 0x02, 0x33, 0x75, 0x2c, 0xf7, 0x3d, 0xf0, 0x6d, 0x4c, 0xb1, 0x61, 0x51,
+	0x17, 0x0a, 0xab, 0xf1, 0x27, 0x80, 0xd6, 0x48, 0x97, 0xb8, 0x64, 0x9c, 0xec, 0xa1, 0xf8, 0x24,
+	0xad, 0xfa, 0x04, 0xff, 0xb7, 0x06, 0x33, 0x01, 0xf1, 0x63, 0x4d, 0xab, 0x0a, 0xb9, 0x36, 0x13,
+	0xc6, 0x2d, 0x48, 0xeb, 0xb2, 0x89, 0x1e, 0xc1, 0x94, 0x30, 0xc0, 0xa9, 0xa6, 0x13, 0x36, 0x4d,
+	0x8e, 0xdb, 0xe4, 0xe0, 0xaf, 0x53, 0x90, 0x17, 0x13, 0xdd, 0xe9, 0xa3, 0x15, 0x28, 0xd9, 0xbc,
+	0x71, 0xc0, 0xe6, 0x23, 0x2c, 0xaa, 0x25, 0x27, 0xa1, 0xf5, 0x09, 0xbd, 0x28, 0x86, 0x30, 0x32,
+	0xfa, 0x3b, 0x28, 0x48, 0x11, 0xfd, 0x81, 0x2b, 0x5c, 0x5e, 0x0d, 0x0a, 0xf0, 0xf7, 0xdf, 0xfa,
+	0x84, 0x0e, 0x82, 0x7d, 0x77, 0xe0, 0xa2, 0x26, 0xcc, 0xca, 0xc1, 0x7c, 0x36, 0xc2, 0x8c, 0x34,
+	0x93, 0xb2, 0x10, 0x94, 0x12, 0x5d, 0xaa, 0xf5, 0x09, 0x1d, 0x89, 0xf1, 0x4a, 0xa7, 0x6a, 0x92,
+	0x7b, 0xca, 0x93, 0x77, 0xc4, 0xa4, 0xe6, 0xa9, 0x19, 0x35, 0xa9, 0x79, 0x6a, 0x3e, 0xcd, 0x43,
+	0x4e, 0xb4, 0xf0, 0xcf, 0x52, 0x00, 0x72, 0x35, 0x76, 0xfa, 0x68, 0x0d, 0xca, 0xb6, 0x68, 0x05,
+	0xbc, 0x75, 0x2d, 0xd6, 0x5b, 0x62, 0x11, 0x27, 0xf4, 0x92, 0x1c, 0xc4, 0x8d, 0x7b, 0x17, 0x8a,
+	0x9e, 0x14, 0xdf, 0x61, 0x73, 0x31, 0x0e, 0xf3, 0x24, 0x14, 0xe4, 0x00, 0xea, 0xb2, 0x8f, 0xe0,
+	0xb2, 0x37, 0x3e, 0xc6, 0x67, 0xb7, 0x46, 0xf8, 0xcc, 0x13, 0x38, 0x23, 0x25, 0xa8, 0x5e, 0x53,
+	0x0d, 0xf3, 0xdd, 0x36, 0x17, 0xe3, 0xb6, 0xa8, 0x61, 0xd4, 0x71, 0x40, 0xeb, 0x25, 0x6f, 0xe2,
+	0x3f, 0xa4, 0x21, 0xb7, 0x6a, 0xf5, 0xfa, 0x86, 0x4d, 0x57, 0x23, 0x6b, 0x13, 0x67, 0xd0, 0x75,
+	0x99, 0xbb, 0xca, 0xcb, 0xb7, 0x83, 0x12, 0x05, 0x9b, 0xfc, 0x57, 0x67, 0xac, 0xba, 0x18, 0x42,
+	0x07, 0x8b, 0xf2, 0x98, 0xba, 0xc0, 0x60, 0x51, 0x1c, 0xc5, 0x10, 0x19, 0xc8, 0x69, 0x3f, 0x90,
+	0x6b, 0x90, 0x1b, 0x12, 0xdb, 0x2f, 0xe9, 0xeb, 0x13, 0xba, 0x24, 0xa0, 0x07, 0x30, 0x1d, 0x2e,
+	0x2f, 0x19, 0xc1, 0x53, 0x6e, 0x05, 0xab, 0xd1, 0x6d, 0x28, 0x06, 0x6a, 0x5c, 0x56, 0xf0, 0x15,
+	0x7a, 0x4a, 0x89, 0xbb, 0x22, 0xf3, 0x2a, 0xad, 0xc7, 0xc5, 0xf5, 0x09, 0x99, 0x59, 0xaf, 0xc8,
+	0xcc, 0x3a, 0x25, 0x46, 0x89, 0xdc, 0x1a, 0x48, 0x32, 0xef, 0x05, 0x93, 0x0c, 0x7e, 0x0f, 0x4a,
+	0x01, 0x07, 0xd1, 0xba, 0xd3, 0xf8, 0x70, 0x7f, 0x65, 0x8b, 0x17, 0xa9, 0x67, 0xac, 0x2e, 0xe9,
+	0x15, 0x8d, 0xd6, 0xba, 0xad, 0xc6, 0xde, 0x5e, 0x25, 0x85, 0x4a, 0x90, 0xdf, 0xde, 0x69, 0x1e,
+	0x70, 0xae, 0x34, 0x7e, 0xe6, 0x49, 0x10, 0x45, 0x4e, 0xa9, 0x6d, 0x13, 0x4a, 0x6d, 0xd3, 0x64,
+	0x6d, 0x4b, 0xf9, 0xb5, 0x8d, 0x95, 0xb9, 0xad, 0xc6, 0xca, 0x5e, 0xa3, 0x32, 0xf9, 0xb4, 0x0c,
+	0x45, 0xee, 0xdf, 0x83, 0x81, 0x49, 0x4b, 0xed, 0x4f, 0x34, 0x00, 0x3f, 0x9a, 0x50, 0x1d, 0x72,
+	0x2d, 0xae, 0xa7, 0xaa, 0xb1, 0x64, 0x74, 0x39, 0x76, 0xc9, 0x74, 0xc9, 0x85, 0xde, 0x82, 0x9c,
+	0x33, 0x68, 0xb5, 0x88, 0x23, 0x4b, 0xde, 0xd5, 0x70, 0x3e, 0x14, 0xd9, 0x4a, 0x97, 0x7c, 0x74,
+	0xc8, 0x4b, 0xa3, 0xd3, 0x1d, 0xb0, 0x02, 0x38, 0x7a, 0x88, 0xe0, 0xc3, 0xff, 0xa7, 0x41, 0x41,
+	0xd9, 0xbc, 0xdf, 0x31, 0x09, 0x5f, 0x87, 0x3c, 0xb3, 0x81, 0xb4, 0x45, 0x1a, 0x9e, 0xd2, 0x7d,
+	0x02, 0xfa, 0x5b, 0xc8, 0xcb, 0x08, 0x90, 0x99, 0xb8, 0x1a, 0x2f, 0x76, 0xa7, 0xaf, 0xfb, 0xac,
+	0x78, 0x13, 0x2e, 0x31, 0xaf, 0xb4, 0xe8, 0xe1, 0x5a, 0xfa, 0x51, 0x3d, 0x7e, 0x6a, 0xa1, 0xe3,
+	0x67, 0x0d, 0xa6, 0xfa, 0xc7, 0x67, 0x4e, 0xa7, 0x65, 0x74, 0x85, 0x15, 0x5e, 0x1b, 0x7f, 0x00,
+	0x48, 0x15, 0x36, 0xce, 0x74, 0x71, 0x09, 0x0a, 0xeb, 0x86, 0x73, 0x2c, 0x4c, 0xc2, 0x8f, 0xa0,
+	0x44, 0x9b, 0x9b, 0x2f, 0x2e, 0x60, 0x23, 0xbb, 0x1c, 0x48, 0xee, 0xb1, 0x7c, 0x8e, 0x60, 0xf2,
+	0xd8, 0x70, 0x8e, 0xd9, 0x44, 0x4b, 0x3a, 0xfb, 0x46, 0x0f, 0xa0, 0xd2, 0xe2, 0x93, 0x3c, 0x08,
+	0x5d, 0x19, 0xa6, 0x05, 0xdd, 0x3b, 0x09, 0x7e, 0x0c, 0x45, 0x3e, 0x87, 0xef, 0xdb, 0x08, 0x7c,
+	0x09, 0xa6, 0xf7, 0x4c, 0xa3, 0xef, 0x1c, 0x5b, 0xb2, 0xba, 0xd1, 0x49, 0x57, 0x7c, 0xda, 0x58,
+	0x1a, 0xef, 0xc3, 0xb4, 0x4d, 0x7a, 0x46, 0xc7, 0xec, 0x98, 0x47, 0x07, 0x87, 0x67, 0x2e, 0x71,
+	0xc4, 0x85, 0xa9, 0xec, 0x91, 0x9f, 0x52, 0x2a, 0x35, 0xed, 0xb0, 0x6b, 0x1d, 0x8a, 0x34, 0xc7,
+	0xbe, 0xf1, 0x17, 0x29, 0x28, 0x7e, 0x64, 0xb8, 0x2d, 0xb9, 0x74, 0x68, 0x03, 0xca, 0x5e, 0x72,
+	0x63, 0x14, 0x61, 0x4b, 0xa8, 0xc4, 0xb2, 0x31, 0xf2, 0x28, 0x2d, 0xab, 0x63, 0xa9, 0xa5, 0x12,
+	0x98, 0x28, 0xc3, 0x6c, 0x91, 0xae, 0x27, 0x2a, 0x95, 0x2c, 0x8a, 0x31, 0xaa, 0xa2, 0x54, 0x02,
+	0xda, 0x81, 0x4a, 0xdf, 0xb6, 0x8e, 0x6c, 0xe2, 0x38, 0x9e, 0x30, 0x5e, 0xc6, 0x70, 0x8c, 0xb0,
+	0x5d, 0xc1, 0xea, 0x8b, 0x9b, 0xee, 0x07, 0x49, 0x4f, 0xa7, 0xfd, 0xf3, 0x0c, 0x4f, 0x4e, 0xbf,
+	0x4e, 0x01, 0x8a, 0x4e, 0xea, 0xdb, 0x1e, 0xf1, 0xee, 0x42, 0xd9, 0x71, 0x0d, 0x3b, 0xb2, 0xd9,
+	0x4a, 0x8c, 0xea, 0x65, 0xfc, 0xfb, 0xe0, 0x19, 0x74, 0x60, 0x5a, 0x6e, 0xe7, 0xe5, 0x99, 0x38,
+	0x25, 0x97, 0x25, 0x79, 0x9b, 0x51, 0x51, 0x03, 0x72, 0x2f, 0x3b, 0x5d, 0x97, 0xd8, 0x4e, 0x35,
+	0xb3, 0x90, 0x5e, 0x2c, 0x2f, 0x3f, 0x3a, 0x6f, 0x19, 0x96, 0xde, 0x67, 0xfc, 0xcd, 0xb3, 0x3e,
+	0xd1, 0xe5, 0x58, 0xf5, 0xe4, 0x99, 0x0d, 0x9c, 0xc6, 0xe7, 0x60, 0xea, 0x15, 0x15, 0x41, 0x6f,
+	0xd9, 0x39, 0x7e, 0x58, 0x64, 0x6d, 0x7e, 0xc9, 0x7e, 0x69, 0x1b, 0x47, 0x3d, 0x62, 0xba, 0xf2,
+	0x1e, 0x28, 0xdb, 0xf8, 0x2e, 0x80, 0xaf, 0x86, 0xa6, 0xfc, 0xed, 0x9d, 0xdd, 0xfd, 0x66, 0x65,
+	0x02, 0x15, 0x61, 0x6a, 0x7b, 0x67, 0xad, 0xb1, 0xd5, 0xa0, 0xf5, 0x01, 0xd7, 0xa5, 0x4b, 0x03,
+	0x6b, 0xa9, 0xea, 0xd4, 0x02, 0x3a, 0xf1, 0x15, 0x98, 0x8d, 0x5b, 0x40, 0x7a, 0x16, 0x2d, 0x89,
+	0x5d, 0x3a, 0x56, 0xa8, 0xa8, 0xaa, 0x53, 0xc1, 0xe9, 0x56, 0x21, 0xc7, 0x77, 0x6f, 0x5b, 0x1c,
+	0xce, 0x65, 0x93, 0x3a, 0x82, 0x6f, 0x46, 0xd2, 0x16, 0xab, 0xe4, 0xb5, 0x63, 0xd3, 0x4b, 0x26,
+	0x36, 0xbd, 0xa0, 0xdb, 0x50, 0xf2, 0xa2, 0xc1, 0x70, 0xc4, 0x59, 0x20, 0xaf, 0x17, 0xe5, 0x46,
+	0xa7, 0xb4, 0x80, 0xd3, 0x73, 0x41, 0xa7, 0xa3, 0xbb, 0x90, 0x25, 0x43, 0x62, 0xba, 0x4e, 0xb5,
+	0xc0, 0x2a, 0x46, 0x49, 0x9e, 0xdd, 0x1b, 0x94, 0xaa, 0x8b, 0x4e, 0xfc, 0x37, 0x70, 0x89, 0xdd,
+	0x91, 0x9e, 0xd9, 0x86, 0xa9, 0x5e, 0xe6, 0x9a, 0xcd, 0x2d, 0xe1, 0x6e, 0xfa, 0x89, 0xca, 0x90,
+	0xda, 0x58, 0x13, 0x4e, 0x48, 0x6d, 0xac, 0xe1, 0xcf, 0x34, 0x40, 0xea, 0xb8, 0xb1, 0xfc, 0x1c,
+	0x12, 0x2e, 0xd5, 0xa7, 0x7d, 0xf5, 0xb3, 0x90, 0x21, 0xb6, 0x6d, 0xd9, 0xcc, 0xa3, 0x79, 0x9d,
+	0x37, 0xf0, 0x1d, 0x61, 0x83, 0x4e, 0x86, 0xd6, 0x89, 0x17, 0x83, 0x5c, 0x9a, 0xe6, 0x99, 0xba,
+	0x09, 0x33, 0x01, 0xae, 0xb1, 0x2a, 0xd7, 0x7d, 0xb8, 0xcc, 0x84, 0x6d, 0x12, 0xd2, 0x5f, 0xe9,
+	0x76, 0x86, 0x89, 0x5a, 0xfb, 0x70, 0x25, 0xcc, 0xf8, 0xc3, 0xfa, 0x08, 0xff, 0xbd, 0xd0, 0xd8,
+	0xec, 0xf4, 0x48, 0xd3, 0xda, 0x4a, 0xb6, 0x8d, 0x66, 0xf6, 0x13, 0x72, 0xe6, 0x88, 0x12, 0xcf,
+	0xbe, 0xf1, 0x4f, 0x35, 0xb8, 0x1a, 0x19, 0xfe, 0x03, 0xaf, 0xea, 0x3c, 0xc0, 0x11, 0xdd, 0x3e,
+	0xa4, 0x4d, 0x3b, 0x38, 0xba, 0xa0, 0x50, 0x3c, 0x3b, 0x69, 0x2e, 0x2b, 0x0a, 0x3b, 0x67, 0xc5,
+	0x9a, 0xb3, 0x3f, 0x5e, 0xc4, 0xdf, 0x80, 0x02, 0x23, 0xec, 0xb9, 0x86, 0x3b, 0x70, 0x22, 0x8b,
+	0xf1, 0x6f, 0x62, 0x0b, 0xc8, 0x41, 0x63, 0xcd, 0xeb, 0x2d, 0xc8, 0xb2, 0x83, 0xb5, 0x3c, 0x56,
+	0x86, 0x6e, 0x32, 0x8a, 0x1d, 0xba, 0x60, 0xc4, 0xc7, 0x90, 0x7d, 0xce, 0xd0, 0x48, 0xc5, 0xb2,
+	0x49, 0xb9, 0x14, 0xa6, 0xd1, 0xe3, 0x18, 0x49, 0x5e, 0x67, 0xdf, 0xec, 0x14, 0x46, 0x88, 0xbd,
+	0xaf, 0x6f, 0xf1, 0xd3, 0x5e, 0x5e, 0xf7, 0xda, 0xd4, 0x65, 0xad, 0x6e, 0x87, 0x98, 0x2e, 0xeb,
+	0x9d, 0x64, 0xbd, 0x0a, 0x05, 0x2f, 0x41, 0x85, 0x6b, 0x5a, 0x69, 0xb7, 0x95, 0xd3, 0x94, 0x27,
+	0x4f, 0x0b, 0xca, 0xc3, 0x5f, 0x69, 0x70, 0x49, 0x19, 0x30, 0x96, 0x63, 0x1e, 0x43, 0x96, 0x63,
+	0xae, 0xa2, 0x70, 0xcf, 0x06, 0x47, 0x71, 0x35, 0xba, 0xe0, 0x41, 0x4b, 0x90, 0xe3, 0x5f, 0xf2,
+	0x48, 0x1b, 0xcf, 0x2e, 0x99, 0xf0, 0x5d, 0x98, 0x11, 0x24, 0xd2, 0xb3, 0xe2, 0xf6, 0x36, 0x73,
+	0x28, 0xfe, 0x14, 0x66, 0x83, 0x6c, 0x63, 0x4d, 0x49, 0x31, 0x32, 0x75, 0x11, 0x23, 0x57, 0xa4,
+	0x91, 0xfb, 0xfd, 0xb6, 0x72, 0x2c, 0x08, 0xaf, 0xba, 0xba, 0x22, 0xa9, 0xd0, 0x8a, 0x78, 0x13,
+	0x90, 0x22, 0x7e, 0xd4, 0x09, 0xcc, 0xc8, 0xed, 0xb0, 0xd5, 0x71, 0xbc, 0xd3, 0xe7, 0x6b, 0x40,
+	0x2a, 0xf1, 0xc7, 0x36, 0x68, 0x8d, 0xc8, 0xa2, 0x26, 0x0d, 0xfa, 0x00, 0x90, 0x4a, 0x1c, 0x2b,
+	0xa3, 0xd7, 0xe1, 0xd2, 0x73, 0x6b, 0x48, 0x53, 0x03, 0xa5, 0xfa, 0x21, 0xc3, 0xef, 0xa2, 0xde,
+	0xb2, 0x79, 0x6d, 0xaa, 0x5c, 0x1d, 0x30, 0x96, 0xf2, 0x5f, 0x6a, 0x50, 0x5c, 0xe9, 0x1a, 0x76,
+	0x4f, 0x2a, 0x7e, 0x17, 0xb2, 0xfc, 0x86, 0x25, 0x40, 0x8d, 0x7b, 0x41, 0x31, 0x2a, 0x2f, 0x6f,
+	0xac, 0xf0, 0xfb, 0x98, 0x18, 0x45, 0x0d, 0x17, 0xef, 0x1e, 0x6b, 0xa1, 0x77, 0x90, 0x35, 0xf4,
+	0x06, 0x64, 0x0c, 0x3a, 0x84, 0xa5, 0xe0, 0x72, 0xf8, 0x6e, 0xcb, 0xa4, 0xb1, 0x73, 0x20, 0xe7,
+	0xc2, 0xef, 0x40, 0x41, 0xd1, 0x40, 0x6f, 0xef, 0xcf, 0x1a, 0xe2, 0xd0, 0xb6, 0xb2, 0xda, 0xdc,
+	0x78, 0xc1, 0x2f, 0xf5, 0x65, 0x80, 0xb5, 0x86, 0xd7, 0x4e, 0xe1, 0x8f, 0xc5, 0x28, 0x91, 0xef,
+	0x54, 0x7b, 0xb4, 0x24, 0x7b, 0x52, 0x17, 0xb2, 0xe7, 0x14, 0x4a, 0x62, 0xfa, 0xe3, 0xa6, 0x6f,
+	0x26, 0x2f, 0x21, 0x7d, 0x2b, 0xc6, 0xeb, 0x82, 0x11, 0x4f, 0x43, 0x49, 0x24, 0x74, 0xb1, 0xff,
+	0x7e, 0xa1, 0x41, 0x59, 0x52, 0xc6, 0x05, 0x5f, 0x25, 0x6e, 0xc4, 0x2b, 0x80, 0x87, 0x1a, 0x5d,
+	0x81, 0x6c, 0xfb, 0x70, 0xaf, 0xf3, 0x5a, 0x02, 0xe5, 0xa2, 0x45, 0xe9, 0x5d, 0xae, 0x87, 0xbf,
+	0x56, 0x89, 0x16, 0xba, 0xce, 0x1f, 0xb2, 0x36, 0xcc, 0x36, 0x39, 0x65, 0x67, 0xca, 0x49, 0xdd,
+	0x27, 0xb0, 0x0b, 0xb5, 0x78, 0xd5, 0x62, 0x07, 0x49, 0xf5, 0x95, 0x6b, 0x06, 0x2e, 0xad, 0x0c,
+	0xdc, 0xe3, 0x86, 0x69, 0x1c, 0x76, 0x65, 0xc6, 0xa2, 0x65, 0x96, 0x12, 0xd7, 0x3a, 0x8e, 0x4a,
+	0x6d, 0xc0, 0x0c, 0xa5, 0x12, 0xd3, 0xed, 0xb4, 0x94, 0xf4, 0x26, 0x8b, 0x98, 0x16, 0x2a, 0x62,
+	0x86, 0xe3, 0xbc, 0xb2, 0xec, 0xb6, 0x98, 0x9a, 0xd7, 0xc6, 0x6b, 0x5c, 0xf8, 0xbe, 0x13, 0x28,
+	0x53, 0xdf, 0x56, 0xca, 0xa2, 0x2f, 0xe5, 0x19, 0x71, 0x47, 0x48, 0xc1, 0x8f, 0xe0, 0xb2, 0xe4,
+	0x14, 0xc0, 0xe4, 0x08, 0xe6, 0x1d, 0xb8, 0x21, 0x99, 0x57, 0x8f, 0xe9, 0x45, 0x6d, 0x57, 0x28,
+	0xfc, 0xae, 0x76, 0x3e, 0x85, 0xaa, 0x67, 0x27, 0x3b, 0x2c, 0x5b, 0x5d, 0xd5, 0x80, 0x81, 0x23,
+	0xf6, 0x4c, 0x5e, 0x67, 0xdf, 0x94, 0x66, 0x5b, 0x5d, 0xef, 0x48, 0x40, 0xbf, 0xf1, 0x2a, 0xcc,
+	0x49, 0x19, 0xe2, 0x18, 0x1b, 0x14, 0x12, 0x31, 0x28, 0x4e, 0x88, 0x70, 0x18, 0x1d, 0x3a, 0xda,
+	0xed, 0x2a, 0x67, 0xd0, 0xb5, 0x4c, 0xa6, 0xa6, 0xc8, 0xbc, 0xcc, 0x77, 0x04, 0x35, 0x4c, 0xad,
+	0x18, 0x82, 0x4c, 0x05, 0xa8, 0x64, 0xb1, 0x10, 0x94, 0x1c, 0x59, 0x88, 0x88, 0xe8, 0x4f, 0x60,
+	0xde, 0x33, 0x82, 0xfa, 0x6d, 0x97, 0xd8, 0xbd, 0x8e, 0xe3, 0x28, 0x50, 0x56, 0xdc, 0xc4, 0xef,
+	0xc1, 0x64, 0x9f, 0x88, 0x9c, 0x52, 0x58, 0x46, 0x4b, 0xfc, 0xed, 0x79, 0x49, 0x19, 0xcc, 0xfa,
+	0x71, 0x1b, 0x6e, 0x4a, 0xe9, 0xdc, 0xa3, 0xb1, 0xe2, 0xc3, 0x46, 0xc9, 0x0b, 0x3e, 0x77, 0x6b,
+	0xf4, 0x82, 0x9f, 0xe6, 0x6b, 0xef, 0xc1, 0xab, 0x1f, 0x70, 0x47, 0xca, 0xd8, 0x1a, 0xab, 0x56,
+	0x6c, 0x72, 0x9f, 0x7a, 0x21, 0x39, 0x96, 0xb0, 0x43, 0x98, 0x0d, 0x46, 0xf2, 0x58, 0x69, 0x6c,
+	0x16, 0x32, 0xae, 0x75, 0x42, 0x64, 0x12, 0xe3, 0x0d, 0x69, 0xb0, 0x17, 0xe6, 0x63, 0x19, 0x6c,
+	0xf8, 0xc2, 0xd8, 0x96, 0x1c, 0xd7, 0x5e, 0xba, 0x9a, 0xf2, 0xf0, 0xc5, 0x1b, 0x78, 0x1b, 0xae,
+	0x84, 0xd3, 0xc4, 0x58, 0x26, 0xbf, 0xe0, 0x1b, 0x38, 0x2e, 0x93, 0x8c, 0x25, 0xf7, 0x43, 0x3f,
+	0x19, 0x28, 0x09, 0x65, 0x2c, 0x91, 0x3a, 0xd4, 0xe2, 0xf2, 0xcb, 0xf7, 0xb1, 0x5f, 0xbd, 0x74,
+	0x33, 0x96, 0x30, 0xc7, 0x17, 0x36, 0xfe, 0xf2, 0xfb, 0x39, 0x22, 0x3d, 0x32, 0x47, 0x88, 0x20,
+	0xf1, 0xb3, 0xd8, 0x0f, 0xb0, 0xe9, 0x84, 0x0e, 0x3f, 0x81, 0x8e, 0xab, 0x83, 0xd6, 0x10, 0x4f,
+	0x07, 0x6b, 0xc8, 0x8d, 0xad, 0xa6, 0xdd, 0xb1, 0x16, 0xe3, 0x23, 0x3f, 0x77, 0x46, 0x32, 0xf3,
+	0x58, 0x82, 0x3f, 0x86, 0x85, 0xe4, 0xa4, 0x3c, 0x8e, 0xe4, 0x87, 0x75, 0xc8, 0x7b, 0x07, 0x4a,
+	0xe5, 0x77, 0x1b, 0x05, 0xc8, 0x6d, 0xef, 0xec, 0xed, 0xae, 0xac, 0x36, 0xf8, 0x0f, 0x37, 0x56,
+	0x77, 0x74, 0x7d, 0x7f, 0xb7, 0x59, 0x49, 0x2d, 0xff, 0x29, 0x0d, 0xa9, 0xcd, 0x17, 0xe8, 0x1f,
+	0x21, 0xc3, 0x5f, 0x31, 0x47, 0x3c, 0x5d, 0xd7, 0x46, 0x3d, 0xd4, 0xe2, 0x6b, 0x9f, 0xfd, 0xe6,
+	0xf7, 0x5f, 0xa6, 0x2e, 0xe3, 0x4a, 0x7d, 0xf8, 0xf6, 0x21, 0x71, 0x8d, 0xfa, 0xc9, 0xb0, 0xce,
+	0xea, 0xc3, 0x13, 0xed, 0x21, 0xda, 0x87, 0xf4, 0xee, 0xc0, 0x45, 0x89, 0xcf, 0xda, 0xb5, 0xe4,
+	0xf7, 0x5b, 0x3c, 0xc7, 0x04, 0xcf, 0xe0, 0xb2, 0x22, 0xb8, 0x3f, 0x70, 0xa9, 0xd8, 0x01, 0x14,
+	0xd4, 0x17, 0xd8, 0x73, 0xdf, 0xbb, 0x6b, 0xe7, 0xbf, 0xee, 0xe2, 0x5b, 0x4c, 0xdd, 0x35, 0x7c,
+	0x45, 0x51, 0xc7, 0xdf, 0x89, 0xd5, 0xd9, 0x34, 0x4f, 0x4d, 0x94, 0xf8, 0x22, 0x5e, 0x4b, 0x7e,
+	0xf4, 0x8d, 0x9d, 0x8d, 0x7b, 0x6a, 0x52, 0xb1, 0xa6, 0x78, 0xf3, 0x6d, 0xb9, 0xe8, 0x66, 0xcc,
+	0x9b, 0x9f, 0xfa, 0xba, 0x55, 0x5b, 0x48, 0x66, 0x10, 0x8a, 0x16, 0x98, 0xa2, 0x1a, 0xbe, 0xac,
+	0x28, 0x6a, 0x79, 0x6c, 0x4f, 0xb4, 0x87, 0xcb, 0x47, 0x90, 0x61, 0xe8, 0x31, 0xfa, 0x27, 0xf9,
+	0x51, 0x8b, 0x81, 0xd1, 0x13, 0x16, 0x3f, 0x80, 0x3b, 0xe3, 0x2a, 0x53, 0x86, 0x70, 0x49, 0x2a,
+	0x63, 0xf8, 0xf1, 0x13, 0xed, 0xe1, 0xa2, 0xf6, 0xa6, 0xb6, 0xfc, 0xc7, 0x49, 0xc8, 0x30, 0xb8,
+	0x08, 0x59, 0x00, 0x3e, 0x9a, 0x1a, 0x9e, 0x65, 0x04, 0x9f, 0x0d, 0xcf, 0x32, 0x0a, 0xc4, 0xe2,
+	0x79, 0xa6, 0xb8, 0x8a, 0x67, 0xa4, 0x62, 0x86, 0x44, 0xd5, 0x19, 0xb8, 0x46, 0x7d, 0x3a, 0x14,
+	0x80, 0x19, 0x0f, 0x33, 0x14, 0x27, 0x30, 0x80, 0xaa, 0x86, 0x77, 0x48, 0x0c, 0xa2, 0x8a, 0x31,
+	0xd3, 0x79, 0x1d, 0x5f, 0x55, 0x3c, 0xcb, 0xd5, 0xda, 0x8c, 0x91, 0xea, 0xfd, 0x0f, 0x0d, 0xca,
+	0x41, 0x5c, 0x14, 0xdd, 0x8e, 0x91, 0x1c, 0x86, 0x57, 0x6b, 0x77, 0x46, 0x33, 0x25, 0x59, 0xc0,
+	0xd5, 0x9f, 0x10, 0xd2, 0x37, 0x28, 0xa3, 0x70, 0x3c, 0xfa, 0x42, 0x83, 0xe9, 0x10, 0xd8, 0x89,
+	0xe2, 0x34, 0x44, 0xa0, 0xd4, 0xda, 0xdd, 0x73, 0xb8, 0x84, 0x21, 0xf7, 0x98, 0x21, 0x0b, 0xf8,
+	0x5a, 0xc4, 0x15, 0x6e, 0xa7, 0x47, 0x5c, 0x4b, 0x18, 0xe3, 0x2d, 0x03, 0x07, 0x26, 0x63, 0x97,
+	0x21, 0x00, 0x74, 0xc6, 0x2e, 0x43, 0x10, 0xd5, 0x1c, 0xb1, 0x0c, 0x1c, 0x8d, 0xa4, 0x5b, 0xfc,
+	0xcf, 0x69, 0xc8, 0xad, 0xf2, 0x5f, 0x4f, 0x22, 0x07, 0xf2, 0x1e, 0x02, 0x88, 0xe6, 0xe3, 0xd0,
+	0x18, 0xff, 0xb6, 0x50, 0xbb, 0x99, 0xd8, 0x2f, 0xb4, 0xdf, 0x65, 0xda, 0x6f, 0xe2, 0x9a, 0xd4,
+	0x2e, 0x7e, 0xa4, 0x59, 0xe7, 0xd7, 0xfe, 0xba, 0xd1, 0x6e, 0xd3, 0x89, 0xff, 0x3b, 0x14, 0x55,
+	0x98, 0x0e, 0xdd, 0x8a, 0x45, 0x81, 0x54, 0xa4, 0xaf, 0x86, 0x47, 0xb1, 0x08, 0xed, 0x8b, 0x4c,
+	0x3b, 0xc6, 0x37, 0x12, 0xb4, 0xdb, 0x8c, 0x3d, 0x60, 0x00, 0x87, 0xd9, 0xe2, 0x0d, 0x08, 0xa0,
+	0x78, 0xf1, 0x06, 0x04, 0x51, 0xba, 0x73, 0x0d, 0x18, 0x30, 0x76, 0x6a, 0xc0, 0x2b, 0x00, 0x1f,
+	0x54, 0x43, 0xb1, 0x7e, 0x55, 0xae, 0x4e, 0xe1, 0x90, 0x8f, 0xe2, 0x71, 0xd1, 0x3d, 0x17, 0x52,
+	0xdd, 0xed, 0x38, 0x34, 0xf4, 0x97, 0xbf, 0xca, 0x42, 0xe1, 0xb9, 0xd1, 0x31, 0x5d, 0x62, 0x1a,
+	0x66, 0x8b, 0xa0, 0x97, 0x90, 0x61, 0xa5, 0x31, 0x9c, 0xe5, 0x54, 0xac, 0x29, 0x9c, 0xe5, 0x02,
+	0x40, 0x0c, 0xbe, 0xc3, 0x34, 0xcf, 0xe3, 0x39, 0xa9, 0xb9, 0xe7, 0x8b, 0xaf, 0x33, 0x0c, 0x85,
+	0x4e, 0xf8, 0x9f, 0x21, 0x2b, 0xe0, 0xf9, 0x90, 0xb0, 0x00, 0xb6, 0x52, 0xbb, 0x1e, 0xdf, 0x99,
+	0xb4, 0xbd, 0x54, 0x55, 0x0e, 0xe3, 0xa5, 0xba, 0x5e, 0x03, 0xf8, 0x00, 0x61, 0xd8, 0xb9, 0x11,
+	0x3c, 0xb1, 0xb6, 0x90, 0xcc, 0x20, 0xf4, 0x3e, 0x60, 0x7a, 0x6f, 0xe3, 0xf9, 0x38, 0xbd, 0x6d,
+	0x8f, 0x9f, 0xea, 0x3e, 0x84, 0xc9, 0x75, 0xc3, 0x39, 0x46, 0xa1, 0x62, 0xa7, 0xfc, 0xe0, 0xa1,
+	0x56, 0x8b, 0xeb, 0x12, 0x9a, 0x6e, 0x33, 0x4d, 0x37, 0x70, 0x35, 0x4e, 0xd3, 0xb1, 0xe1, 0xd0,
+	0xea, 0x81, 0x8e, 0x21, 0xcb, 0x7f, 0x03, 0x11, 0xf6, 0x65, 0xe0, 0x77, 0x14, 0x61, 0x5f, 0x06,
+	0x7f, 0x36, 0x71, 0x31, 0x4d, 0x2e, 0x4c, 0xc9, 0x1f, 0x1e, 0xa0, 0x1b, 0xa1, 0xa5, 0x09, 0xfe,
+	0x48, 0xa1, 0x36, 0x9f, 0xd4, 0x2d, 0xf4, 0xdd, 0x67, 0xfa, 0x6e, 0xe1, 0xeb, 0xb1, 0x6b, 0x27,
+	0xb8, 0x9f, 0x68, 0x0f, 0xdf, 0xd4, 0x68, 0x99, 0x00, 0x1f, 0x64, 0x8d, 0x44, 0x47, 0x18, 0xaf,
+	0x8d, 0x44, 0x47, 0x04, 0x9f, 0xc5, 0xcb, 0x4c, 0xf9, 0x63, 0x7c, 0x3f, 0x4e, 0xb9, 0x6b, 0x1b,
+	0xa6, 0xf3, 0x92, 0xd8, 0x6f, 0x70, 0x30, 0xcd, 0x39, 0xee, 0xf4, 0x69, 0xa4, 0xfc, 0x65, 0x1a,
+	0x26, 0xe9, 0x79, 0x94, 0x96, 0x67, 0xff, 0x1a, 0x1f, 0xb6, 0x26, 0x02, 0x9e, 0x85, 0xad, 0x89,
+	0x22, 0x00, 0xd1, 0xf2, 0xcc, 0x7e, 0x27, 0x4f, 0x18, 0x13, 0xf5, 0xba, 0x03, 0x05, 0xe5, 0xae,
+	0x8f, 0x62, 0x04, 0x06, 0x91, 0xb9, 0x70, 0x5d, 0x88, 0x01, 0x0a, 0xf0, 0x4d, 0xa6, 0x73, 0x0e,
+	0xcf, 0x06, 0x74, 0xb6, 0x39, 0x17, 0x55, 0xfa, 0xaf, 0x50, 0x54, 0x31, 0x01, 0x14, 0x23, 0x33,
+	0x84, 0xfc, 0x85, 0x53, 0x62, 0x1c, 0xa4, 0x10, 0xcd, 0x0e, 0xde, 0xff, 0x09, 0x90, 0xac, 0x54,
+	0x79, 0x1f, 0x72, 0x02, 0x28, 0x88, 0x9b, 0x6d, 0x10, 0x2a, 0x8c, 0x9b, 0x6d, 0x08, 0x65, 0x88,
+	0x1e, 0xf3, 0x98, 0x56, 0x7a, 0x1f, 0x92, 0x25, 0x48, 0x68, 0x7c, 0x46, 0xdc, 0x24, 0x8d, 0x3e,
+	0xf6, 0x95, 0xa4, 0x51, 0xb9, 0x8b, 0x8e, 0xd2, 0x78, 0x44, 0x5c, 0x11, 0x4b, 0xf2, 0x9e, 0x87,
+	0x12, 0x04, 0xaa, 0x29, 0x1f, 0x8f, 0x62, 0x49, 0x3a, 0x95, 0xfb, 0x4a, 0x45, 0xbe, 0x47, 0x9f,
+	0x02, 0xf8, 0x90, 0x46, 0xf8, 0xb4, 0x15, 0x8b, 0x8b, 0x86, 0x4f, 0x5b, 0xf1, 0xa8, 0x48, 0x34,
+	0x7f, 0xf8, 0xba, 0xf9, 0xc5, 0x80, 0x6a, 0xff, 0x1f, 0x0d, 0x50, 0x14, 0x01, 0x41, 0x8f, 0xe2,
+	0x35, 0xc4, 0x22, 0xae, 0xb5, 0xc7, 0x17, 0x63, 0x4e, 0x2a, 0x11, 0xbe, 0x59, 0x2d, 0x36, 0xa2,
+	0xff, 0x8a, 0x1a, 0xf6, 0xb9, 0x06, 0xa5, 0x00, 0x84, 0x82, 0xee, 0x25, 0xac, 0x71, 0x08, 0xb4,
+	0xad, 0xdd, 0x3f, 0x97, 0x2f, 0xe9, 0x24, 0xa6, 0xec, 0x08, 0x79, 0x10, 0xff, 0x2f, 0x0d, 0xca,
+	0x41, 0xd8, 0x05, 0x25, 0xc8, 0x8f, 0x00, 0xbf, 0xb5, 0xc5, 0xf3, 0x19, 0xcf, 0x5f, 0x2a, 0xff,
+	0x6c, 0xde, 0x87, 0x9c, 0x00, 0x6b, 0xe2, 0x02, 0x22, 0x08, 0x1b, 0xc7, 0x05, 0x44, 0x08, 0xe9,
+	0x49, 0x08, 0x08, 0xdb, 0xea, 0x12, 0x25, 0x04, 0x05, 0xa2, 0x93, 0xa4, 0x71, 0x74, 0x08, 0x86,
+	0xe0, 0xa0, 0x51, 0x1a, 0xfd, 0x10, 0x94, 0x70, 0x0e, 0x4a, 0x10, 0x78, 0x4e, 0x08, 0x86, 0xd1,
+	0xa0, 0x84, 0x10, 0x64, 0x4a, 0x95, 0x10, 0xf4, 0xc1, 0x97, 0xb8, 0x10, 0x8c, 0x20, 0xe2, 0x71,
+	0x21, 0x18, 0xc5, 0x6f, 0x12, 0xd6, 0x95, 0xe9, 0x0e, 0x84, 0xe0, 0x4c, 0x0c, 0x56, 0x83, 0x1e,
+	0x27, 0x38, 0x34, 0x16, 0x6c, 0xaf, 0xbd, 0x71, 0x41, 0xee, 0x91, 0x7b, 0x9f, 0x2f, 0x85, 0xdc,
+	0xfb, 0xff, 0xaf, 0xc1, 0x6c, 0x1c, 0xd6, 0x83, 0x12, 0x74, 0x25, 0x00, 0xf5, 0xb5, 0xa5, 0x8b,
+	0xb2, 0x9f, 0xef, 0x35, 0x2f, 0x1a, 0x9e, 0x56, 0x7e, 0xfe, 0xcd, 0xbc, 0xf6, 0xab, 0x6f, 0xe6,
+	0xb5, 0xdf, 0x7e, 0x33, 0xaf, 0xfd, 0xef, 0xef, 0xe6, 0x27, 0x0e, 0xb3, 0xec, 0x7f, 0xa7, 0xbd,
+	0xfd, 0xd7, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc8, 0x77, 0x6b, 0x63, 0x24, 0x37, 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
+
+// KVClient is the client API for KV service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type KVClient interface {
+	// Range gets the keys in the range from the key-value store.
+	Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error)
+	// Put puts the given key into the key-value store.
+	// A put request increments the revision of the key-value store
+	// and generates one event in the event history.
+	Put(ctx context.Context, in *PutRequest, opts ...grpc.CallOption) (*PutResponse, error)
+	// DeleteRange deletes the given range from the key-value store.
+	// A delete request increments the revision of the key-value store
+	// and generates a delete event in the event history for every deleted key.
+	DeleteRange(ctx context.Context, in *DeleteRangeRequest, opts ...grpc.CallOption) (*DeleteRangeResponse, error)
+	// Txn processes multiple requests in a single transaction.
+	// A txn request increments the revision of the key-value store
+	// and generates events with the same revision for every completed request.
+	// It is not allowed to modify the same key several times within one txn.
+	Txn(ctx context.Context, in *TxnRequest, opts ...grpc.CallOption) (*TxnResponse, error)
+	// Compact compacts the event history in the etcd key-value store. The key-value
+	// store should be periodically compacted or the event history will continue to grow
+	// indefinitely.
+	Compact(ctx context.Context, in *CompactionRequest, opts ...grpc.CallOption) (*CompactionResponse, error)
+}
+
+type kVClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewKVClient(cc *grpc.ClientConn) KVClient {
+	return &kVClient{cc}
+}
+
+func (c *kVClient) Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error) {
+	out := new(RangeResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.KV/Range", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *kVClient) Put(ctx context.Context, in *PutRequest, opts ...grpc.CallOption) (*PutResponse, error) {
+	out := new(PutResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.KV/Put", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *kVClient) DeleteRange(ctx context.Context, in *DeleteRangeRequest, opts ...grpc.CallOption) (*DeleteRangeResponse, error) {
+	out := new(DeleteRangeResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.KV/DeleteRange", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *kVClient) Txn(ctx context.Context, in *TxnRequest, opts ...grpc.CallOption) (*TxnResponse, error) {
+	out := new(TxnResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.KV/Txn", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *kVClient) Compact(ctx context.Context, in *CompactionRequest, opts ...grpc.CallOption) (*CompactionResponse, error) {
+	out := new(CompactionResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.KV/Compact", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// KVServer is the server API for KV service.
+type KVServer interface {
+	// Range gets the keys in the range from the key-value store.
+	Range(context.Context, *RangeRequest) (*RangeResponse, error)
+	// Put puts the given key into the key-value store.
+	// A put request increments the revision of the key-value store
+	// and generates one event in the event history.
+	Put(context.Context, *PutRequest) (*PutResponse, error)
+	// DeleteRange deletes the given range from the key-value store.
+	// A delete request increments the revision of the key-value store
+	// and generates a delete event in the event history for every deleted key.
+	DeleteRange(context.Context, *DeleteRangeRequest) (*DeleteRangeResponse, error)
+	// Txn processes multiple requests in a single transaction.
+	// A txn request increments the revision of the key-value store
+	// and generates events with the same revision for every completed request.
+	// It is not allowed to modify the same key several times within one txn.
+	Txn(context.Context, *TxnRequest) (*TxnResponse, error)
+	// Compact compacts the event history in the etcd key-value store. The key-value
+	// store should be periodically compacted or the event history will continue to grow
+	// indefinitely.
+	Compact(context.Context, *CompactionRequest) (*CompactionResponse, error)
+}
+
+// UnimplementedKVServer can be embedded to have forward compatible implementations.
+type UnimplementedKVServer struct {
+}
+
+func (*UnimplementedKVServer) Range(ctx context.Context, req *RangeRequest) (*RangeResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Range not implemented")
+}
+func (*UnimplementedKVServer) Put(ctx context.Context, req *PutRequest) (*PutResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Put not implemented")
+}
+func (*UnimplementedKVServer) DeleteRange(ctx context.Context, req *DeleteRangeRequest) (*DeleteRangeResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteRange not implemented")
+}
+func (*UnimplementedKVServer) Txn(ctx context.Context, req *TxnRequest) (*TxnResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Txn not implemented")
+}
+func (*UnimplementedKVServer) Compact(ctx context.Context, req *CompactionRequest) (*CompactionResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Compact not implemented")
+}
+
+func RegisterKVServer(s *grpc.Server, srv KVServer) {
+	s.RegisterService(&_KV_serviceDesc, srv)
+}
+
+func _KV_Range_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(RangeRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(KVServer).Range(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.KV/Range",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(KVServer).Range(ctx, req.(*RangeRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _KV_Put_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(PutRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(KVServer).Put(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.KV/Put",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(KVServer).Put(ctx, req.(*PutRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _KV_DeleteRange_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeleteRangeRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(KVServer).DeleteRange(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.KV/DeleteRange",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(KVServer).DeleteRange(ctx, req.(*DeleteRangeRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _KV_Txn_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(TxnRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(KVServer).Txn(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.KV/Txn",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(KVServer).Txn(ctx, req.(*TxnRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _KV_Compact_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(CompactionRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(KVServer).Compact(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.KV/Compact",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(KVServer).Compact(ctx, req.(*CompactionRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _KV_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.KV",
+	HandlerType: (*KVServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Range",
+			Handler:    _KV_Range_Handler,
+		},
+		{
+			MethodName: "Put",
+			Handler:    _KV_Put_Handler,
+		},
+		{
+			MethodName: "DeleteRange",
+			Handler:    _KV_DeleteRange_Handler,
+		},
+		{
+			MethodName: "Txn",
+			Handler:    _KV_Txn_Handler,
+		},
+		{
+			MethodName: "Compact",
+			Handler:    _KV_Compact_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "rpc.proto",
+}
+
+// WatchClient is the client API for Watch service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type WatchClient interface {
+	// Watch watches for events happening or that have happened. Both input and output
+	// are streams; the input stream is for creating and canceling watchers and the output
+	// stream sends events. One watch RPC can watch on multiple key ranges, streaming events
+	// for several watches at once. The entire event history can be watched starting from the
+	// last compaction revision.
+	Watch(ctx context.Context, opts ...grpc.CallOption) (Watch_WatchClient, error)
+}
+
+type watchClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewWatchClient(cc *grpc.ClientConn) WatchClient {
+	return &watchClient{cc}
+}
+
+func (c *watchClient) Watch(ctx context.Context, opts ...grpc.CallOption) (Watch_WatchClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_Watch_serviceDesc.Streams[0], "/etcdserverpb.Watch/Watch", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &watchWatchClient{stream}
+	return x, nil
+}
+
+type Watch_WatchClient interface {
+	Send(*WatchRequest) error
+	Recv() (*WatchResponse, error)
+	grpc.ClientStream
+}
+
+type watchWatchClient struct {
+	grpc.ClientStream
+}
+
+func (x *watchWatchClient) Send(m *WatchRequest) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *watchWatchClient) Recv() (*WatchResponse, error) {
+	m := new(WatchResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// WatchServer is the server API for Watch service.
+type WatchServer interface {
+	// Watch watches for events happening or that have happened. Both input and output
+	// are streams; the input stream is for creating and canceling watchers and the output
+	// stream sends events. One watch RPC can watch on multiple key ranges, streaming events
+	// for several watches at once. The entire event history can be watched starting from the
+	// last compaction revision.
+	Watch(Watch_WatchServer) error
+}
+
+// UnimplementedWatchServer can be embedded to have forward compatible implementations.
+type UnimplementedWatchServer struct {
+}
+
+func (*UnimplementedWatchServer) Watch(srv Watch_WatchServer) error {
+	return status.Errorf(codes.Unimplemented, "method Watch not implemented")
+}
+
+func RegisterWatchServer(s *grpc.Server, srv WatchServer) {
+	s.RegisterService(&_Watch_serviceDesc, srv)
+}
+
+func _Watch_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(WatchServer).Watch(&watchWatchServer{stream})
+}
+
+type Watch_WatchServer interface {
+	Send(*WatchResponse) error
+	Recv() (*WatchRequest, error)
+	grpc.ServerStream
+}
+
+type watchWatchServer struct {
+	grpc.ServerStream
+}
+
+func (x *watchWatchServer) Send(m *WatchResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *watchWatchServer) Recv() (*WatchRequest, error) {
+	m := new(WatchRequest)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+var _Watch_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.Watch",
+	HandlerType: (*WatchServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Watch",
+			Handler:       _Watch_Watch_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+	},
+	Metadata: "rpc.proto",
+}
+
+// LeaseClient is the client API for Lease service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type LeaseClient interface {
+	// LeaseGrant creates a lease which expires if the server does not receive a keepAlive
+	// within a given time to live period. All keys attached to the lease will be expired and
+	// deleted if the lease expires. Each expired key generates a delete event in the event history.
+	LeaseGrant(ctx context.Context, in *LeaseGrantRequest, opts ...grpc.CallOption) (*LeaseGrantResponse, error)
+	// LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
+	LeaseRevoke(ctx context.Context, in *LeaseRevokeRequest, opts ...grpc.CallOption) (*LeaseRevokeResponse, error)
+	// LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
+	// to the server and streaming keep alive responses from the server to the client.
+	LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (Lease_LeaseKeepAliveClient, error)
+	// LeaseTimeToLive retrieves lease information.
+	LeaseTimeToLive(ctx context.Context, in *LeaseTimeToLiveRequest, opts ...grpc.CallOption) (*LeaseTimeToLiveResponse, error)
+	// LeaseLeases lists all existing leases.
+	LeaseLeases(ctx context.Context, in *LeaseLeasesRequest, opts ...grpc.CallOption) (*LeaseLeasesResponse, error)
+}
+
+type leaseClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewLeaseClient(cc *grpc.ClientConn) LeaseClient {
+	return &leaseClient{cc}
+}
+
+func (c *leaseClient) LeaseGrant(ctx context.Context, in *LeaseGrantRequest, opts ...grpc.CallOption) (*LeaseGrantResponse, error) {
+	out := new(LeaseGrantResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Lease/LeaseGrant", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *leaseClient) LeaseRevoke(ctx context.Context, in *LeaseRevokeRequest, opts ...grpc.CallOption) (*LeaseRevokeResponse, error) {
+	out := new(LeaseRevokeResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Lease/LeaseRevoke", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *leaseClient) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (Lease_LeaseKeepAliveClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_Lease_serviceDesc.Streams[0], "/etcdserverpb.Lease/LeaseKeepAlive", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &leaseLeaseKeepAliveClient{stream}
+	return x, nil
+}
+
+type Lease_LeaseKeepAliveClient interface {
+	Send(*LeaseKeepAliveRequest) error
+	Recv() (*LeaseKeepAliveResponse, error)
+	grpc.ClientStream
+}
+
+type leaseLeaseKeepAliveClient struct {
+	grpc.ClientStream
+}
+
+func (x *leaseLeaseKeepAliveClient) Send(m *LeaseKeepAliveRequest) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *leaseLeaseKeepAliveClient) Recv() (*LeaseKeepAliveResponse, error) {
+	m := new(LeaseKeepAliveResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *leaseClient) LeaseTimeToLive(ctx context.Context, in *LeaseTimeToLiveRequest, opts ...grpc.CallOption) (*LeaseTimeToLiveResponse, error) {
+	out := new(LeaseTimeToLiveResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Lease/LeaseTimeToLive", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *leaseClient) LeaseLeases(ctx context.Context, in *LeaseLeasesRequest, opts ...grpc.CallOption) (*LeaseLeasesResponse, error) {
+	out := new(LeaseLeasesResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Lease/LeaseLeases", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// LeaseServer is the server API for Lease service.
+type LeaseServer interface {
+	// LeaseGrant creates a lease which expires if the server does not receive a keepAlive
+	// within a given time to live period. All keys attached to the lease will be expired and
+	// deleted if the lease expires. Each expired key generates a delete event in the event history.
+	LeaseGrant(context.Context, *LeaseGrantRequest) (*LeaseGrantResponse, error)
+	// LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
+	LeaseRevoke(context.Context, *LeaseRevokeRequest) (*LeaseRevokeResponse, error)
+	// LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
+	// to the server and streaming keep alive responses from the server to the client.
+	LeaseKeepAlive(Lease_LeaseKeepAliveServer) error
+	// LeaseTimeToLive retrieves lease information.
+	LeaseTimeToLive(context.Context, *LeaseTimeToLiveRequest) (*LeaseTimeToLiveResponse, error)
+	// LeaseLeases lists all existing leases.
+	LeaseLeases(context.Context, *LeaseLeasesRequest) (*LeaseLeasesResponse, error)
+}
+
+// UnimplementedLeaseServer can be embedded to have forward compatible implementations.
+type UnimplementedLeaseServer struct {
+}
+
+func (*UnimplementedLeaseServer) LeaseGrant(ctx context.Context, req *LeaseGrantRequest) (*LeaseGrantResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method LeaseGrant not implemented")
+}
+func (*UnimplementedLeaseServer) LeaseRevoke(ctx context.Context, req *LeaseRevokeRequest) (*LeaseRevokeResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method LeaseRevoke not implemented")
+}
+func (*UnimplementedLeaseServer) LeaseKeepAlive(srv Lease_LeaseKeepAliveServer) error {
+	return status.Errorf(codes.Unimplemented, "method LeaseKeepAlive not implemented")
+}
+func (*UnimplementedLeaseServer) LeaseTimeToLive(ctx context.Context, req *LeaseTimeToLiveRequest) (*LeaseTimeToLiveResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method LeaseTimeToLive not implemented")
+}
+func (*UnimplementedLeaseServer) LeaseLeases(ctx context.Context, req *LeaseLeasesRequest) (*LeaseLeasesResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method LeaseLeases not implemented")
+}
+
+func RegisterLeaseServer(s *grpc.Server, srv LeaseServer) {
+	s.RegisterService(&_Lease_serviceDesc, srv)
+}
+
+func _Lease_LeaseGrant_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LeaseGrantRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LeaseServer).LeaseGrant(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Lease/LeaseGrant",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LeaseServer).LeaseGrant(ctx, req.(*LeaseGrantRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Lease_LeaseRevoke_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LeaseRevokeRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LeaseServer).LeaseRevoke(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Lease/LeaseRevoke",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LeaseServer).LeaseRevoke(ctx, req.(*LeaseRevokeRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Lease_LeaseKeepAlive_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(LeaseServer).LeaseKeepAlive(&leaseLeaseKeepAliveServer{stream})
+}
+
+type Lease_LeaseKeepAliveServer interface {
+	Send(*LeaseKeepAliveResponse) error
+	Recv() (*LeaseKeepAliveRequest, error)
+	grpc.ServerStream
+}
+
+type leaseLeaseKeepAliveServer struct {
+	grpc.ServerStream
+}
+
+func (x *leaseLeaseKeepAliveServer) Send(m *LeaseKeepAliveResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *leaseLeaseKeepAliveServer) Recv() (*LeaseKeepAliveRequest, error) {
+	m := new(LeaseKeepAliveRequest)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func _Lease_LeaseTimeToLive_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LeaseTimeToLiveRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LeaseServer).LeaseTimeToLive(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Lease/LeaseTimeToLive",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LeaseServer).LeaseTimeToLive(ctx, req.(*LeaseTimeToLiveRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Lease_LeaseLeases_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LeaseLeasesRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LeaseServer).LeaseLeases(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Lease/LeaseLeases",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LeaseServer).LeaseLeases(ctx, req.(*LeaseLeasesRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Lease_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.Lease",
+	HandlerType: (*LeaseServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "LeaseGrant",
+			Handler:    _Lease_LeaseGrant_Handler,
+		},
+		{
+			MethodName: "LeaseRevoke",
+			Handler:    _Lease_LeaseRevoke_Handler,
+		},
+		{
+			MethodName: "LeaseTimeToLive",
+			Handler:    _Lease_LeaseTimeToLive_Handler,
+		},
+		{
+			MethodName: "LeaseLeases",
+			Handler:    _Lease_LeaseLeases_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "LeaseKeepAlive",
+			Handler:       _Lease_LeaseKeepAlive_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+	},
+	Metadata: "rpc.proto",
+}
+
+// ClusterClient is the client API for Cluster service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type ClusterClient interface {
+	// MemberAdd adds a member into the cluster.
+	MemberAdd(ctx context.Context, in *MemberAddRequest, opts ...grpc.CallOption) (*MemberAddResponse, error)
+	// MemberRemove removes an existing member from the cluster.
+	MemberRemove(ctx context.Context, in *MemberRemoveRequest, opts ...grpc.CallOption) (*MemberRemoveResponse, error)
+	// MemberUpdate updates the member configuration.
+	MemberUpdate(ctx context.Context, in *MemberUpdateRequest, opts ...grpc.CallOption) (*MemberUpdateResponse, error)
+	// MemberList lists all the members in the cluster.
+	MemberList(ctx context.Context, in *MemberListRequest, opts ...grpc.CallOption) (*MemberListResponse, error)
+}
+
+type clusterClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewClusterClient(cc *grpc.ClientConn) ClusterClient {
+	return &clusterClient{cc}
+}
+
+func (c *clusterClient) MemberAdd(ctx context.Context, in *MemberAddRequest, opts ...grpc.CallOption) (*MemberAddResponse, error) {
+	out := new(MemberAddResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Cluster/MemberAdd", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *clusterClient) MemberRemove(ctx context.Context, in *MemberRemoveRequest, opts ...grpc.CallOption) (*MemberRemoveResponse, error) {
+	out := new(MemberRemoveResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Cluster/MemberRemove", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *clusterClient) MemberUpdate(ctx context.Context, in *MemberUpdateRequest, opts ...grpc.CallOption) (*MemberUpdateResponse, error) {
+	out := new(MemberUpdateResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Cluster/MemberUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *clusterClient) MemberList(ctx context.Context, in *MemberListRequest, opts ...grpc.CallOption) (*MemberListResponse, error) {
+	out := new(MemberListResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Cluster/MemberList", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// ClusterServer is the server API for Cluster service.
+type ClusterServer interface {
+	// MemberAdd adds a member into the cluster.
+	MemberAdd(context.Context, *MemberAddRequest) (*MemberAddResponse, error)
+	// MemberRemove removes an existing member from the cluster.
+	MemberRemove(context.Context, *MemberRemoveRequest) (*MemberRemoveResponse, error)
+	// MemberUpdate updates the member configuration.
+	MemberUpdate(context.Context, *MemberUpdateRequest) (*MemberUpdateResponse, error)
+	// MemberList lists all the members in the cluster.
+	MemberList(context.Context, *MemberListRequest) (*MemberListResponse, error)
+}
+
+// UnimplementedClusterServer can be embedded to have forward compatible implementations.
+type UnimplementedClusterServer struct {
+}
+
+func (*UnimplementedClusterServer) MemberAdd(ctx context.Context, req *MemberAddRequest) (*MemberAddResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method MemberAdd not implemented")
+}
+func (*UnimplementedClusterServer) MemberRemove(ctx context.Context, req *MemberRemoveRequest) (*MemberRemoveResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method MemberRemove not implemented")
+}
+func (*UnimplementedClusterServer) MemberUpdate(ctx context.Context, req *MemberUpdateRequest) (*MemberUpdateResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method MemberUpdate not implemented")
+}
+func (*UnimplementedClusterServer) MemberList(ctx context.Context, req *MemberListRequest) (*MemberListResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method MemberList not implemented")
+}
+
+func RegisterClusterServer(s *grpc.Server, srv ClusterServer) {
+	s.RegisterService(&_Cluster_serviceDesc, srv)
+}
+
+func _Cluster_MemberAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MemberAddRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ClusterServer).MemberAdd(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Cluster/MemberAdd",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ClusterServer).MemberAdd(ctx, req.(*MemberAddRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Cluster_MemberRemove_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MemberRemoveRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ClusterServer).MemberRemove(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Cluster/MemberRemove",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ClusterServer).MemberRemove(ctx, req.(*MemberRemoveRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Cluster_MemberUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MemberUpdateRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ClusterServer).MemberUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Cluster/MemberUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ClusterServer).MemberUpdate(ctx, req.(*MemberUpdateRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Cluster_MemberList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MemberListRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ClusterServer).MemberList(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Cluster/MemberList",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ClusterServer).MemberList(ctx, req.(*MemberListRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Cluster_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.Cluster",
+	HandlerType: (*ClusterServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "MemberAdd",
+			Handler:    _Cluster_MemberAdd_Handler,
+		},
+		{
+			MethodName: "MemberRemove",
+			Handler:    _Cluster_MemberRemove_Handler,
+		},
+		{
+			MethodName: "MemberUpdate",
+			Handler:    _Cluster_MemberUpdate_Handler,
+		},
+		{
+			MethodName: "MemberList",
+			Handler:    _Cluster_MemberList_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "rpc.proto",
+}
+
+// MaintenanceClient is the client API for Maintenance service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type MaintenanceClient interface {
+	// Alarm activates, deactivates, and queries alarms regarding cluster health.
+	Alarm(ctx context.Context, in *AlarmRequest, opts ...grpc.CallOption) (*AlarmResponse, error)
+	// Status gets the status of the member.
+	Status(ctx context.Context, in *StatusRequest, opts ...grpc.CallOption) (*StatusResponse, error)
+	// Defragment defragments a member's backend database to recover storage space.
+	Defragment(ctx context.Context, in *DefragmentRequest, opts ...grpc.CallOption) (*DefragmentResponse, error)
+	// Hash computes the hash of the KV's backend.
+	// This is designed for testing; do not use this in production when there
+	// are ongoing transactions.
+	Hash(ctx context.Context, in *HashRequest, opts ...grpc.CallOption) (*HashResponse, error)
+	// HashKV computes the hash of all MVCC keys up to a given revision.
+	HashKV(ctx context.Context, in *HashKVRequest, opts ...grpc.CallOption) (*HashKVResponse, error)
+	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
+	Snapshot(ctx context.Context, in *SnapshotRequest, opts ...grpc.CallOption) (Maintenance_SnapshotClient, error)
+	// MoveLeader requests current leader node to transfer its leadership to transferee.
+	MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error)
+}
+
+type maintenanceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewMaintenanceClient(cc *grpc.ClientConn) MaintenanceClient {
+	return &maintenanceClient{cc}
+}
+
+func (c *maintenanceClient) Alarm(ctx context.Context, in *AlarmRequest, opts ...grpc.CallOption) (*AlarmResponse, error) {
+	out := new(AlarmResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/Alarm", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *maintenanceClient) Status(ctx context.Context, in *StatusRequest, opts ...grpc.CallOption) (*StatusResponse, error) {
+	out := new(StatusResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/Status", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *maintenanceClient) Defragment(ctx context.Context, in *DefragmentRequest, opts ...grpc.CallOption) (*DefragmentResponse, error) {
+	out := new(DefragmentResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/Defragment", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *maintenanceClient) Hash(ctx context.Context, in *HashRequest, opts ...grpc.CallOption) (*HashResponse, error) {
+	out := new(HashResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/Hash", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *maintenanceClient) HashKV(ctx context.Context, in *HashKVRequest, opts ...grpc.CallOption) (*HashKVResponse, error) {
+	out := new(HashKVResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/HashKV", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *maintenanceClient) Snapshot(ctx context.Context, in *SnapshotRequest, opts ...grpc.CallOption) (Maintenance_SnapshotClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_Maintenance_serviceDesc.Streams[0], "/etcdserverpb.Maintenance/Snapshot", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &maintenanceSnapshotClient{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 Maintenance_SnapshotClient interface {
+	Recv() (*SnapshotResponse, error)
+	grpc.ClientStream
+}
+
+type maintenanceSnapshotClient struct {
+	grpc.ClientStream
+}
+
+func (x *maintenanceSnapshotClient) Recv() (*SnapshotResponse, error) {
+	m := new(SnapshotResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *maintenanceClient) MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error) {
+	out := new(MoveLeaderResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Maintenance/MoveLeader", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// MaintenanceServer is the server API for Maintenance service.
+type MaintenanceServer interface {
+	// Alarm activates, deactivates, and queries alarms regarding cluster health.
+	Alarm(context.Context, *AlarmRequest) (*AlarmResponse, error)
+	// Status gets the status of the member.
+	Status(context.Context, *StatusRequest) (*StatusResponse, error)
+	// Defragment defragments a member's backend database to recover storage space.
+	Defragment(context.Context, *DefragmentRequest) (*DefragmentResponse, error)
+	// Hash computes the hash of the KV's backend.
+	// This is designed for testing; do not use this in production when there
+	// are ongoing transactions.
+	Hash(context.Context, *HashRequest) (*HashResponse, error)
+	// HashKV computes the hash of all MVCC keys up to a given revision.
+	HashKV(context.Context, *HashKVRequest) (*HashKVResponse, error)
+	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
+	Snapshot(*SnapshotRequest, Maintenance_SnapshotServer) error
+	// MoveLeader requests current leader node to transfer its leadership to transferee.
+	MoveLeader(context.Context, *MoveLeaderRequest) (*MoveLeaderResponse, error)
+}
+
+// UnimplementedMaintenanceServer can be embedded to have forward compatible implementations.
+type UnimplementedMaintenanceServer struct {
+}
+
+func (*UnimplementedMaintenanceServer) Alarm(ctx context.Context, req *AlarmRequest) (*AlarmResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Alarm not implemented")
+}
+func (*UnimplementedMaintenanceServer) Status(ctx context.Context, req *StatusRequest) (*StatusResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Status not implemented")
+}
+func (*UnimplementedMaintenanceServer) Defragment(ctx context.Context, req *DefragmentRequest) (*DefragmentResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Defragment not implemented")
+}
+func (*UnimplementedMaintenanceServer) Hash(ctx context.Context, req *HashRequest) (*HashResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Hash not implemented")
+}
+func (*UnimplementedMaintenanceServer) HashKV(ctx context.Context, req *HashKVRequest) (*HashKVResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method HashKV not implemented")
+}
+func (*UnimplementedMaintenanceServer) Snapshot(req *SnapshotRequest, srv Maintenance_SnapshotServer) error {
+	return status.Errorf(codes.Unimplemented, "method Snapshot not implemented")
+}
+func (*UnimplementedMaintenanceServer) MoveLeader(ctx context.Context, req *MoveLeaderRequest) (*MoveLeaderResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method MoveLeader not implemented")
+}
+
+func RegisterMaintenanceServer(s *grpc.Server, srv MaintenanceServer) {
+	s.RegisterService(&_Maintenance_serviceDesc, srv)
+}
+
+func _Maintenance_Alarm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AlarmRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).Alarm(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/Alarm",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).Alarm(ctx, req.(*AlarmRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Maintenance_Status_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(StatusRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).Status(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/Status",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).Status(ctx, req.(*StatusRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Maintenance_Defragment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DefragmentRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).Defragment(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/Defragment",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).Defragment(ctx, req.(*DefragmentRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Maintenance_Hash_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(HashRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).Hash(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/Hash",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).Hash(ctx, req.(*HashRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Maintenance_HashKV_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(HashKVRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).HashKV(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/HashKV",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).HashKV(ctx, req.(*HashKVRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Maintenance_Snapshot_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(SnapshotRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(MaintenanceServer).Snapshot(m, &maintenanceSnapshotServer{stream})
+}
+
+type Maintenance_SnapshotServer interface {
+	Send(*SnapshotResponse) error
+	grpc.ServerStream
+}
+
+type maintenanceSnapshotServer struct {
+	grpc.ServerStream
+}
+
+func (x *maintenanceSnapshotServer) Send(m *SnapshotResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _Maintenance_MoveLeader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MoveLeaderRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).MoveLeader(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/MoveLeader",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).MoveLeader(ctx, req.(*MoveLeaderRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Maintenance_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.Maintenance",
+	HandlerType: (*MaintenanceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Alarm",
+			Handler:    _Maintenance_Alarm_Handler,
+		},
+		{
+			MethodName: "Status",
+			Handler:    _Maintenance_Status_Handler,
+		},
+		{
+			MethodName: "Defragment",
+			Handler:    _Maintenance_Defragment_Handler,
+		},
+		{
+			MethodName: "Hash",
+			Handler:    _Maintenance_Hash_Handler,
+		},
+		{
+			MethodName: "HashKV",
+			Handler:    _Maintenance_HashKV_Handler,
+		},
+		{
+			MethodName: "MoveLeader",
+			Handler:    _Maintenance_MoveLeader_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Snapshot",
+			Handler:       _Maintenance_Snapshot_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "rpc.proto",
+}
+
+// AuthClient is the client API for Auth service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type AuthClient interface {
+	// AuthEnable enables authentication.
+	AuthEnable(ctx context.Context, in *AuthEnableRequest, opts ...grpc.CallOption) (*AuthEnableResponse, error)
+	// AuthDisable disables authentication.
+	AuthDisable(ctx context.Context, in *AuthDisableRequest, opts ...grpc.CallOption) (*AuthDisableResponse, error)
+	// Authenticate processes an authenticate request.
+	Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error)
+	// UserAdd adds a new user.
+	UserAdd(ctx context.Context, in *AuthUserAddRequest, opts ...grpc.CallOption) (*AuthUserAddResponse, error)
+	// UserGet gets detailed user information.
+	UserGet(ctx context.Context, in *AuthUserGetRequest, opts ...grpc.CallOption) (*AuthUserGetResponse, error)
+	// UserList gets a list of all users.
+	UserList(ctx context.Context, in *AuthUserListRequest, opts ...grpc.CallOption) (*AuthUserListResponse, error)
+	// UserDelete deletes a specified user.
+	UserDelete(ctx context.Context, in *AuthUserDeleteRequest, opts ...grpc.CallOption) (*AuthUserDeleteResponse, error)
+	// UserChangePassword changes the password of a specified user.
+	UserChangePassword(ctx context.Context, in *AuthUserChangePasswordRequest, opts ...grpc.CallOption) (*AuthUserChangePasswordResponse, error)
+	// UserGrant grants a role to a specified user.
+	UserGrantRole(ctx context.Context, in *AuthUserGrantRoleRequest, opts ...grpc.CallOption) (*AuthUserGrantRoleResponse, error)
+	// UserRevokeRole revokes a role of specified user.
+	UserRevokeRole(ctx context.Context, in *AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (*AuthUserRevokeRoleResponse, error)
+	// RoleAdd adds a new role.
+	RoleAdd(ctx context.Context, in *AuthRoleAddRequest, opts ...grpc.CallOption) (*AuthRoleAddResponse, error)
+	// RoleGet gets detailed role information.
+	RoleGet(ctx context.Context, in *AuthRoleGetRequest, opts ...grpc.CallOption) (*AuthRoleGetResponse, error)
+	// RoleList gets lists of all roles.
+	RoleList(ctx context.Context, in *AuthRoleListRequest, opts ...grpc.CallOption) (*AuthRoleListResponse, error)
+	// RoleDelete deletes a specified role.
+	RoleDelete(ctx context.Context, in *AuthRoleDeleteRequest, opts ...grpc.CallOption) (*AuthRoleDeleteResponse, error)
+	// RoleGrantPermission grants a permission of a specified key or range to a specified role.
+	RoleGrantPermission(ctx context.Context, in *AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (*AuthRoleGrantPermissionResponse, error)
+	// RoleRevokePermission revokes a key or range permission of a specified role.
+	RoleRevokePermission(ctx context.Context, in *AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (*AuthRoleRevokePermissionResponse, error)
+}
+
+type authClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewAuthClient(cc *grpc.ClientConn) AuthClient {
+	return &authClient{cc}
+}
+
+func (c *authClient) AuthEnable(ctx context.Context, in *AuthEnableRequest, opts ...grpc.CallOption) (*AuthEnableResponse, error) {
+	out := new(AuthEnableResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/AuthEnable", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) AuthDisable(ctx context.Context, in *AuthDisableRequest, opts ...grpc.CallOption) (*AuthDisableResponse, error) {
+	out := new(AuthDisableResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/AuthDisable", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error) {
+	out := new(AuthenticateResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/Authenticate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserAdd(ctx context.Context, in *AuthUserAddRequest, opts ...grpc.CallOption) (*AuthUserAddResponse, error) {
+	out := new(AuthUserAddResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserAdd", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserGet(ctx context.Context, in *AuthUserGetRequest, opts ...grpc.CallOption) (*AuthUserGetResponse, error) {
+	out := new(AuthUserGetResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserGet", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserList(ctx context.Context, in *AuthUserListRequest, opts ...grpc.CallOption) (*AuthUserListResponse, error) {
+	out := new(AuthUserListResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserList", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserDelete(ctx context.Context, in *AuthUserDeleteRequest, opts ...grpc.CallOption) (*AuthUserDeleteResponse, error) {
+	out := new(AuthUserDeleteResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserDelete", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserChangePassword(ctx context.Context, in *AuthUserChangePasswordRequest, opts ...grpc.CallOption) (*AuthUserChangePasswordResponse, error) {
+	out := new(AuthUserChangePasswordResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserChangePassword", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserGrantRole(ctx context.Context, in *AuthUserGrantRoleRequest, opts ...grpc.CallOption) (*AuthUserGrantRoleResponse, error) {
+	out := new(AuthUserGrantRoleResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserGrantRole", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) UserRevokeRole(ctx context.Context, in *AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (*AuthUserRevokeRoleResponse, error) {
+	out := new(AuthUserRevokeRoleResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/UserRevokeRole", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleAdd(ctx context.Context, in *AuthRoleAddRequest, opts ...grpc.CallOption) (*AuthRoleAddResponse, error) {
+	out := new(AuthRoleAddResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleAdd", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleGet(ctx context.Context, in *AuthRoleGetRequest, opts ...grpc.CallOption) (*AuthRoleGetResponse, error) {
+	out := new(AuthRoleGetResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleGet", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleList(ctx context.Context, in *AuthRoleListRequest, opts ...grpc.CallOption) (*AuthRoleListResponse, error) {
+	out := new(AuthRoleListResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleList", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleDelete(ctx context.Context, in *AuthRoleDeleteRequest, opts ...grpc.CallOption) (*AuthRoleDeleteResponse, error) {
+	out := new(AuthRoleDeleteResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleDelete", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleGrantPermission(ctx context.Context, in *AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (*AuthRoleGrantPermissionResponse, error) {
+	out := new(AuthRoleGrantPermissionResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleGrantPermission", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *authClient) RoleRevokePermission(ctx context.Context, in *AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (*AuthRoleRevokePermissionResponse, error) {
+	out := new(AuthRoleRevokePermissionResponse)
+	err := c.cc.Invoke(ctx, "/etcdserverpb.Auth/RoleRevokePermission", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// AuthServer is the server API for Auth service.
+type AuthServer interface {
+	// AuthEnable enables authentication.
+	AuthEnable(context.Context, *AuthEnableRequest) (*AuthEnableResponse, error)
+	// AuthDisable disables authentication.
+	AuthDisable(context.Context, *AuthDisableRequest) (*AuthDisableResponse, error)
+	// Authenticate processes an authenticate request.
+	Authenticate(context.Context, *AuthenticateRequest) (*AuthenticateResponse, error)
+	// UserAdd adds a new user.
+	UserAdd(context.Context, *AuthUserAddRequest) (*AuthUserAddResponse, error)
+	// UserGet gets detailed user information.
+	UserGet(context.Context, *AuthUserGetRequest) (*AuthUserGetResponse, error)
+	// UserList gets a list of all users.
+	UserList(context.Context, *AuthUserListRequest) (*AuthUserListResponse, error)
+	// UserDelete deletes a specified user.
+	UserDelete(context.Context, *AuthUserDeleteRequest) (*AuthUserDeleteResponse, error)
+	// UserChangePassword changes the password of a specified user.
+	UserChangePassword(context.Context, *AuthUserChangePasswordRequest) (*AuthUserChangePasswordResponse, error)
+	// UserGrant grants a role to a specified user.
+	UserGrantRole(context.Context, *AuthUserGrantRoleRequest) (*AuthUserGrantRoleResponse, error)
+	// UserRevokeRole revokes a role of specified user.
+	UserRevokeRole(context.Context, *AuthUserRevokeRoleRequest) (*AuthUserRevokeRoleResponse, error)
+	// RoleAdd adds a new role.
+	RoleAdd(context.Context, *AuthRoleAddRequest) (*AuthRoleAddResponse, error)
+	// RoleGet gets detailed role information.
+	RoleGet(context.Context, *AuthRoleGetRequest) (*AuthRoleGetResponse, error)
+	// RoleList gets lists of all roles.
+	RoleList(context.Context, *AuthRoleListRequest) (*AuthRoleListResponse, error)
+	// RoleDelete deletes a specified role.
+	RoleDelete(context.Context, *AuthRoleDeleteRequest) (*AuthRoleDeleteResponse, error)
+	// RoleGrantPermission grants a permission of a specified key or range to a specified role.
+	RoleGrantPermission(context.Context, *AuthRoleGrantPermissionRequest) (*AuthRoleGrantPermissionResponse, error)
+	// RoleRevokePermission revokes a key or range permission of a specified role.
+	RoleRevokePermission(context.Context, *AuthRoleRevokePermissionRequest) (*AuthRoleRevokePermissionResponse, error)
+}
+
+// UnimplementedAuthServer can be embedded to have forward compatible implementations.
+type UnimplementedAuthServer struct {
+}
+
+func (*UnimplementedAuthServer) AuthEnable(ctx context.Context, req *AuthEnableRequest) (*AuthEnableResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method AuthEnable not implemented")
+}
+func (*UnimplementedAuthServer) AuthDisable(ctx context.Context, req *AuthDisableRequest) (*AuthDisableResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method AuthDisable not implemented")
+}
+func (*UnimplementedAuthServer) Authenticate(ctx context.Context, req *AuthenticateRequest) (*AuthenticateResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Authenticate not implemented")
+}
+func (*UnimplementedAuthServer) UserAdd(ctx context.Context, req *AuthUserAddRequest) (*AuthUserAddResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserAdd not implemented")
+}
+func (*UnimplementedAuthServer) UserGet(ctx context.Context, req *AuthUserGetRequest) (*AuthUserGetResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserGet not implemented")
+}
+func (*UnimplementedAuthServer) UserList(ctx context.Context, req *AuthUserListRequest) (*AuthUserListResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserList not implemented")
+}
+func (*UnimplementedAuthServer) UserDelete(ctx context.Context, req *AuthUserDeleteRequest) (*AuthUserDeleteResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserDelete not implemented")
+}
+func (*UnimplementedAuthServer) UserChangePassword(ctx context.Context, req *AuthUserChangePasswordRequest) (*AuthUserChangePasswordResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserChangePassword not implemented")
+}
+func (*UnimplementedAuthServer) UserGrantRole(ctx context.Context, req *AuthUserGrantRoleRequest) (*AuthUserGrantRoleResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserGrantRole not implemented")
+}
+func (*UnimplementedAuthServer) UserRevokeRole(ctx context.Context, req *AuthUserRevokeRoleRequest) (*AuthUserRevokeRoleResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UserRevokeRole not implemented")
+}
+func (*UnimplementedAuthServer) RoleAdd(ctx context.Context, req *AuthRoleAddRequest) (*AuthRoleAddResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleAdd not implemented")
+}
+func (*UnimplementedAuthServer) RoleGet(ctx context.Context, req *AuthRoleGetRequest) (*AuthRoleGetResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleGet not implemented")
+}
+func (*UnimplementedAuthServer) RoleList(ctx context.Context, req *AuthRoleListRequest) (*AuthRoleListResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleList not implemented")
+}
+func (*UnimplementedAuthServer) RoleDelete(ctx context.Context, req *AuthRoleDeleteRequest) (*AuthRoleDeleteResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleDelete not implemented")
+}
+func (*UnimplementedAuthServer) RoleGrantPermission(ctx context.Context, req *AuthRoleGrantPermissionRequest) (*AuthRoleGrantPermissionResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleGrantPermission not implemented")
+}
+func (*UnimplementedAuthServer) RoleRevokePermission(ctx context.Context, req *AuthRoleRevokePermissionRequest) (*AuthRoleRevokePermissionResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RoleRevokePermission not implemented")
+}
+
+func RegisterAuthServer(s *grpc.Server, srv AuthServer) {
+	s.RegisterService(&_Auth_serviceDesc, srv)
+}
+
+func _Auth_AuthEnable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthEnableRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).AuthEnable(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/AuthEnable",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).AuthEnable(ctx, req.(*AuthEnableRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_AuthDisable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthDisableRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).AuthDisable(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/AuthDisable",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).AuthDisable(ctx, req.(*AuthDisableRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_Authenticate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthenticateRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).Authenticate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/Authenticate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).Authenticate(ctx, req.(*AuthenticateRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserAddRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserAdd(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserAdd",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserAdd(ctx, req.(*AuthUserAddRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserGetRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserGet(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserGet",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserGet(ctx, req.(*AuthUserGetRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserListRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserList(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserList",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserList(ctx, req.(*AuthUserListRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserDeleteRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserDelete(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserDelete",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserDelete(ctx, req.(*AuthUserDeleteRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserChangePassword_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserChangePasswordRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserChangePassword(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserChangePassword",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserChangePassword(ctx, req.(*AuthUserChangePasswordRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserGrantRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserGrantRoleRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserGrantRole(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserGrantRole",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserGrantRole(ctx, req.(*AuthUserGrantRoleRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_UserRevokeRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthUserRevokeRoleRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).UserRevokeRole(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/UserRevokeRole",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).UserRevokeRole(ctx, req.(*AuthUserRevokeRoleRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleAddRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleAdd(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleAdd",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleAdd(ctx, req.(*AuthRoleAddRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleGetRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleGet(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleGet",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleGet(ctx, req.(*AuthRoleGetRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleListRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleList(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleList",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleList(ctx, req.(*AuthRoleListRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleDeleteRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleDelete(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleDelete",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleDelete(ctx, req.(*AuthRoleDeleteRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleGrantPermission_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleGrantPermissionRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleGrantPermission(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleGrantPermission",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleGrantPermission(ctx, req.(*AuthRoleGrantPermissionRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Auth_RoleRevokePermission_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(AuthRoleRevokePermissionRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AuthServer).RoleRevokePermission(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Auth/RoleRevokePermission",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AuthServer).RoleRevokePermission(ctx, req.(*AuthRoleRevokePermissionRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Auth_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "etcdserverpb.Auth",
+	HandlerType: (*AuthServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "AuthEnable",
+			Handler:    _Auth_AuthEnable_Handler,
+		},
+		{
+			MethodName: "AuthDisable",
+			Handler:    _Auth_AuthDisable_Handler,
+		},
+		{
+			MethodName: "Authenticate",
+			Handler:    _Auth_Authenticate_Handler,
+		},
+		{
+			MethodName: "UserAdd",
+			Handler:    _Auth_UserAdd_Handler,
+		},
+		{
+			MethodName: "UserGet",
+			Handler:    _Auth_UserGet_Handler,
+		},
+		{
+			MethodName: "UserList",
+			Handler:    _Auth_UserList_Handler,
+		},
+		{
+			MethodName: "UserDelete",
+			Handler:    _Auth_UserDelete_Handler,
+		},
+		{
+			MethodName: "UserChangePassword",
+			Handler:    _Auth_UserChangePassword_Handler,
+		},
+		{
+			MethodName: "UserGrantRole",
+			Handler:    _Auth_UserGrantRole_Handler,
+		},
+		{
+			MethodName: "UserRevokeRole",
+			Handler:    _Auth_UserRevokeRole_Handler,
+		},
+		{
+			MethodName: "RoleAdd",
+			Handler:    _Auth_RoleAdd_Handler,
+		},
+		{
+			MethodName: "RoleGet",
+			Handler:    _Auth_RoleGet_Handler,
+		},
+		{
+			MethodName: "RoleList",
+			Handler:    _Auth_RoleList_Handler,
+		},
+		{
+			MethodName: "RoleDelete",
+			Handler:    _Auth_RoleDelete_Handler,
+		},
+		{
+			MethodName: "RoleGrantPermission",
+			Handler:    _Auth_RoleGrantPermission_Handler,
+		},
+		{
+			MethodName: "RoleRevokePermission",
+			Handler:    _Auth_RoleRevokePermission_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "rpc.proto",
+}
+
+func (m *ResponseHeader) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *ResponseHeader) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *ResponseHeader) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.RaftTerm != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.RaftTerm))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.Revision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.MemberId != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MemberId))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.ClusterId != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ClusterId))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *RangeRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *RangeRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *RangeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.MaxCreateRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MaxCreateRevision))
+		i--
+		dAtA[i] = 0x68
+	}
+	if m.MinCreateRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MinCreateRevision))
+		i--
+		dAtA[i] = 0x60
+	}
+	if m.MaxModRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MaxModRevision))
+		i--
+		dAtA[i] = 0x58
+	}
+	if m.MinModRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MinModRevision))
+		i--
+		dAtA[i] = 0x50
+	}
+	if m.CountOnly {
+		i--
+		if m.CountOnly {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x48
+	}
+	if m.KeysOnly {
+		i--
+		if m.KeysOnly {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x40
+	}
+	if m.Serializable {
+		i--
+		if m.Serializable {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x38
+	}
+	if m.SortTarget != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.SortTarget))
+		i--
+		dAtA[i] = 0x30
+	}
+	if m.SortOrder != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.SortOrder))
+		i--
+		dAtA[i] = 0x28
+	}
+	if m.Revision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.Limit != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Limit))
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *RangeResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *RangeResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *RangeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Count != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Count))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.More {
+		i--
+		if m.More {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.Kvs) > 0 {
+		for iNdEx := len(m.Kvs) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Kvs[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *PutRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *PutRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *PutRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.IgnoreLease {
+		i--
+		if m.IgnoreLease {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x30
+	}
+	if m.IgnoreValue {
+		i--
+		if m.IgnoreValue {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x28
+	}
+	if m.PrevKv {
+		i--
+		if m.PrevKv {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.Lease != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Lease))
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.Value) > 0 {
+		i -= len(m.Value)
+		copy(dAtA[i:], m.Value)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Value)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *PutResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *PutResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *PutResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.PrevKv != nil {
+		{
+			size, err := m.PrevKv.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *DeleteRangeRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *DeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *DeleteRangeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.PrevKv {
+		i--
+		if m.PrevKv {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *DeleteRangeResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *DeleteRangeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.PrevKvs) > 0 {
+		for iNdEx := len(m.PrevKvs) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.PrevKvs[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if m.Deleted != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Deleted))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *RequestOp) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *RequestOp) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *RequestOp) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Request != nil {
+		{
+			size := m.Request.Size()
+			i -= size
+			if _, err := m.Request.MarshalTo(dAtA[i:]); err != nil {
+				return 0, err
+			}
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *RequestOp_RequestRange) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *RequestOp_RequestRange) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.RequestRange != nil {
+		{
+			size, err := m.RequestRange.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+func (m *RequestOp_RequestPut) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *RequestOp_RequestPut) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.RequestPut != nil {
+		{
+			size, err := m.RequestPut.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	return len(dAtA) - i, nil
+}
+func (m *RequestOp_RequestDeleteRange) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *RequestOp_RequestDeleteRange) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.RequestDeleteRange != nil {
+		{
+			size, err := m.RequestDeleteRange.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	return len(dAtA) - i, nil
+}
+func (m *RequestOp_RequestTxn) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *RequestOp_RequestTxn) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.RequestTxn != nil {
+		{
+			size, err := m.RequestTxn.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x22
+	}
+	return len(dAtA) - i, nil
+}
+func (m *ResponseOp) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *ResponseOp) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *ResponseOp) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Response != nil {
+		{
+			size := m.Response.Size()
+			i -= size
+			if _, err := m.Response.MarshalTo(dAtA[i:]); err != nil {
+				return 0, err
+			}
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *ResponseOp_ResponseRange) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *ResponseOp_ResponseRange) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.ResponseRange != nil {
+		{
+			size, err := m.ResponseRange.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+func (m *ResponseOp_ResponsePut) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *ResponseOp_ResponsePut) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.ResponsePut != nil {
+		{
+			size, err := m.ResponsePut.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	return len(dAtA) - i, nil
+}
+func (m *ResponseOp_ResponseDeleteRange) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *ResponseOp_ResponseDeleteRange) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.ResponseDeleteRange != nil {
+		{
+			size, err := m.ResponseDeleteRange.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	return len(dAtA) - i, nil
+}
+func (m *ResponseOp_ResponseTxn) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *ResponseOp_ResponseTxn) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.ResponseTxn != nil {
+		{
+			size, err := m.ResponseTxn.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x22
+	}
+	return len(dAtA) - i, nil
+}
+func (m *Compare) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Compare) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Compare) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x4
+		i--
+		dAtA[i] = 0x82
+	}
+	if m.TargetUnion != nil {
+		{
+			size := m.TargetUnion.Size()
+			i -= size
+			if _, err := m.TargetUnion.MarshalTo(dAtA[i:]); err != nil {
+				return 0, err
+			}
+		}
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.Target != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Target))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Result != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Result))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Compare_Version) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *Compare_Version) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	i = encodeVarintRpc(dAtA, i, uint64(m.Version))
+	i--
+	dAtA[i] = 0x20
+	return len(dAtA) - i, nil
+}
+func (m *Compare_CreateRevision) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *Compare_CreateRevision) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	i = encodeVarintRpc(dAtA, i, uint64(m.CreateRevision))
+	i--
+	dAtA[i] = 0x28
+	return len(dAtA) - i, nil
+}
+func (m *Compare_ModRevision) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *Compare_ModRevision) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	i = encodeVarintRpc(dAtA, i, uint64(m.ModRevision))
+	i--
+	dAtA[i] = 0x30
+	return len(dAtA) - i, nil
+}
+func (m *Compare_Value) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *Compare_Value) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.Value != nil {
+		i -= len(m.Value)
+		copy(dAtA[i:], m.Value)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Value)))
+		i--
+		dAtA[i] = 0x3a
+	}
+	return len(dAtA) - i, nil
+}
+func (m *Compare_Lease) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *Compare_Lease) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	i = encodeVarintRpc(dAtA, i, uint64(m.Lease))
+	i--
+	dAtA[i] = 0x40
+	return len(dAtA) - i, nil
+}
+func (m *TxnRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *TxnRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *TxnRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Failure) > 0 {
+		for iNdEx := len(m.Failure) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Failure[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if len(m.Success) > 0 {
+		for iNdEx := len(m.Success) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Success[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if len(m.Compare) > 0 {
+		for iNdEx := len(m.Compare) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Compare[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0xa
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *TxnResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *TxnResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *TxnResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Responses) > 0 {
+		for iNdEx := len(m.Responses) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Responses[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if m.Succeeded {
+		i--
+		if m.Succeeded {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *CompactionRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *CompactionRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *CompactionRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Physical {
+		i--
+		if m.Physical {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Revision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *CompactionResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *CompactionResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *CompactionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *HashRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *HashRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *HashRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *HashKVRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *HashKVRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *HashKVRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Revision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *HashKVResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *HashKVResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *HashKVResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.CompactRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.CompactRevision))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.Hash != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Hash))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *HashResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *HashResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *HashResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Hash != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Hash))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *SnapshotRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *SnapshotRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *SnapshotRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *SnapshotResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *SnapshotResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *SnapshotResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Blob) > 0 {
+		i -= len(m.Blob)
+		copy(dAtA[i:], m.Blob)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Blob)))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.RemainingBytes != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.RemainingBytes))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *WatchRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *WatchRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *WatchRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.RequestUnion != nil {
+		{
+			size := m.RequestUnion.Size()
+			i -= size
+			if _, err := m.RequestUnion.MarshalTo(dAtA[i:]); err != nil {
+				return 0, err
+			}
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *WatchRequest_CreateRequest) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *WatchRequest_CreateRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.CreateRequest != nil {
+		{
+			size, err := m.CreateRequest.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+func (m *WatchRequest_CancelRequest) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *WatchRequest_CancelRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.CancelRequest != nil {
+		{
+			size, err := m.CancelRequest.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	return len(dAtA) - i, nil
+}
+func (m *WatchRequest_ProgressRequest) MarshalTo(dAtA []byte) (int, error) {
+	return m.MarshalToSizedBuffer(dAtA[:m.Size()])
+}
+
+func (m *WatchRequest_ProgressRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	if m.ProgressRequest != nil {
+		{
+			size, err := m.ProgressRequest.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	return len(dAtA) - i, nil
+}
+func (m *WatchCreateRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *WatchCreateRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *WatchCreateRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Fragment {
+		i--
+		if m.Fragment {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x40
+	}
+	if m.WatchId != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
+		i--
+		dAtA[i] = 0x38
+	}
+	if m.PrevKv {
+		i--
+		if m.PrevKv {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x30
+	}
+	if len(m.Filters) > 0 {
+		dAtA22 := make([]byte, len(m.Filters)*10)
+		var j21 int
+		for _, num := range m.Filters {
+			for num >= 1<<7 {
+				dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j21++
+			}
+			dAtA22[j21] = uint8(num)
+			j21++
+		}
+		i -= j21
+		copy(dAtA[i:], dAtA22[:j21])
+		i = encodeVarintRpc(dAtA, i, uint64(j21))
+		i--
+		dAtA[i] = 0x2a
+	}
+	if m.ProgressNotify {
+		i--
+		if m.ProgressNotify {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.StartRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.StartRevision))
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *WatchCancelRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *WatchCancelRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *WatchCancelRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.WatchId != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *WatchProgressRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *WatchProgressRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *WatchProgressRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *WatchResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *WatchResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *WatchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Events) > 0 {
+		for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Events[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x5a
+		}
+	}
+	if m.Fragment {
+		i--
+		if m.Fragment {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x38
+	}
+	if len(m.CancelReason) > 0 {
+		i -= len(m.CancelReason)
+		copy(dAtA[i:], m.CancelReason)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.CancelReason)))
+		i--
+		dAtA[i] = 0x32
+	}
+	if m.CompactRevision != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.CompactRevision))
+		i--
+		dAtA[i] = 0x28
+	}
+	if m.Canceled {
+		i--
+		if m.Canceled {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.Created {
+		i--
+		if m.Created {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.WatchId != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseGrantRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseGrantRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseGrantRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.TTL != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseGrantResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseGrantResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseGrantResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Error) > 0 {
+		i -= len(m.Error)
+		copy(dAtA[i:], m.Error)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Error)))
+		i--
+		dAtA[i] = 0x22
+	}
+	if m.TTL != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseRevokeRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseRevokeRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseRevokeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseRevokeResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseRevokeResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseRevokeResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseKeepAliveRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseKeepAliveRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseKeepAliveRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseKeepAliveResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseKeepAliveResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseKeepAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.TTL != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseTimeToLiveRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseTimeToLiveRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseTimeToLiveRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Keys {
+		i--
+		if m.Keys {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseTimeToLiveResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseTimeToLiveResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseTimeToLiveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Keys) > 0 {
+		for iNdEx := len(m.Keys) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Keys[iNdEx])
+			copy(dAtA[i:], m.Keys[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.Keys[iNdEx])))
+			i--
+			dAtA[i] = 0x2a
+		}
+	}
+	if m.GrantedTTL != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.GrantedTTL))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.TTL != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseLeasesRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseLeasesRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseLeasesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseStatus) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseStatus) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *LeaseLeasesResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *LeaseLeasesResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *LeaseLeasesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Leases) > 0 {
+		for iNdEx := len(m.Leases) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Leases[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Member) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Member) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Member) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.ClientURLs) > 0 {
+		for iNdEx := len(m.ClientURLs) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.ClientURLs[iNdEx])
+			copy(dAtA[i:], m.ClientURLs[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.ClientURLs[iNdEx])))
+			i--
+			dAtA[i] = 0x22
+		}
+	}
+	if len(m.PeerURLs) > 0 {
+		for iNdEx := len(m.PeerURLs) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.PeerURLs[iNdEx])
+			copy(dAtA[i:], m.PeerURLs[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.PeerURLs[iNdEx])))
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberAddRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberAddRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberAddRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.PeerURLs) > 0 {
+		for iNdEx := len(m.PeerURLs) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.PeerURLs[iNdEx])
+			copy(dAtA[i:], m.PeerURLs[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.PeerURLs[iNdEx])))
+			i--
+			dAtA[i] = 0xa
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberAddResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberAddResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberAddResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Members) > 0 {
+		for iNdEx := len(m.Members) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Members[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x1a
+		}
+	}
+	if m.Member != nil {
+		{
+			size, err := m.Member.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberRemoveRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberRemoveRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberRemoveRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberRemoveResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberRemoveResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberRemoveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Members) > 0 {
+		for iNdEx := len(m.Members) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Members[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberUpdateRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberUpdateRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberUpdateRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.PeerURLs) > 0 {
+		for iNdEx := len(m.PeerURLs) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.PeerURLs[iNdEx])
+			copy(dAtA[i:], m.PeerURLs[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.PeerURLs[iNdEx])))
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.ID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberUpdateResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberUpdateResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberUpdateResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Members) > 0 {
+		for iNdEx := len(m.Members) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Members[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberListRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberListRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MemberListResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MemberListResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MemberListResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Members) > 0 {
+		for iNdEx := len(m.Members) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Members[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *DefragmentRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *DefragmentRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *DefragmentRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *DefragmentResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *DefragmentResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *DefragmentResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MoveLeaderRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MoveLeaderRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MoveLeaderRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.TargetID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.TargetID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MoveLeaderResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MoveLeaderResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *MoveLeaderResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AlarmRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AlarmRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AlarmRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Alarm != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Alarm))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.MemberID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MemberID))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Action != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Action))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AlarmMember) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AlarmMember) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AlarmMember) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Alarm != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Alarm))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.MemberID != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.MemberID))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AlarmResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AlarmResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AlarmResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Alarms) > 0 {
+		for iNdEx := len(m.Alarms) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Alarms[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *StatusRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *StatusRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *StatusRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *StatusResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *StatusResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *StatusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.RaftTerm != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.RaftTerm))
+		i--
+		dAtA[i] = 0x30
+	}
+	if m.RaftIndex != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.RaftIndex))
+		i--
+		dAtA[i] = 0x28
+	}
+	if m.Leader != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.Leader))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.DbSize != 0 {
+		i = encodeVarintRpc(dAtA, i, uint64(m.DbSize))
+		i--
+		dAtA[i] = 0x18
+	}
+	if len(m.Version) > 0 {
+		i -= len(m.Version)
+		copy(dAtA[i:], m.Version)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Version)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthEnableRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthEnableRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthEnableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthDisableRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthDisableRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthDisableRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthenticateRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthenticateRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthenticateRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Password) > 0 {
+		i -= len(m.Password)
+		copy(dAtA[i:], m.Password)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserAddRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserAddRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserAddRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Password) > 0 {
+		i -= len(m.Password)
+		copy(dAtA[i:], m.Password)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserGetRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserGetRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserGetRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserDeleteRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserDeleteRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserDeleteRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserChangePasswordRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserChangePasswordRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserChangePasswordRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Password) > 0 {
+		i -= len(m.Password)
+		copy(dAtA[i:], m.Password)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserGrantRoleRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserGrantRoleRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserGrantRoleRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Role) > 0 {
+		i -= len(m.Role)
+		copy(dAtA[i:], m.Role)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.User) > 0 {
+		i -= len(m.User)
+		copy(dAtA[i:], m.User)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.User)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserRevokeRoleRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserRevokeRoleRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserRevokeRoleRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Role) > 0 {
+		i -= len(m.Role)
+		copy(dAtA[i:], m.Role)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleAddRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleAddRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleAddRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleGetRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleGetRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleGetRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Role) > 0 {
+		i -= len(m.Role)
+		copy(dAtA[i:], m.Role)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserListRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserListRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleListRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleListRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleDeleteRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleDeleteRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleDeleteRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Role) > 0 {
+		i -= len(m.Role)
+		copy(dAtA[i:], m.Role)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleGrantPermissionRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleGrantPermissionRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleGrantPermissionRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Perm != nil {
+		{
+			size, err := m.Perm.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Name) > 0 {
+		i -= len(m.Name)
+		copy(dAtA[i:], m.Name)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleRevokePermissionRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleRevokePermissionRequest) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleRevokePermissionRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.RangeEnd) > 0 {
+		i -= len(m.RangeEnd)
+		copy(dAtA[i:], m.RangeEnd)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if len(m.Role) > 0 {
+		i -= len(m.Role)
+		copy(dAtA[i:], m.Role)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthEnableResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthEnableResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthEnableResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthDisableResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthDisableResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthDisableResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthenticateResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthenticateResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthenticateResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Token) > 0 {
+		i -= len(m.Token)
+		copy(dAtA[i:], m.Token)
+		i = encodeVarintRpc(dAtA, i, uint64(len(m.Token)))
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserAddResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserAddResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserAddResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserGetResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserGetResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserGetResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Roles) > 0 {
+		for iNdEx := len(m.Roles) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Roles[iNdEx])
+			copy(dAtA[i:], m.Roles[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.Roles[iNdEx])))
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserDeleteResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserDeleteResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserChangePasswordResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserChangePasswordResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserChangePasswordResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserGrantRoleResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserGrantRoleResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserGrantRoleResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserRevokeRoleResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserRevokeRoleResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserRevokeRoleResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleAddResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleAddResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleAddResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleGetResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleGetResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleGetResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Perm) > 0 {
+		for iNdEx := len(m.Perm) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Perm[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRpc(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleListResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleListResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleListResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Roles) > 0 {
+		for iNdEx := len(m.Roles) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Roles[iNdEx])
+			copy(dAtA[i:], m.Roles[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.Roles[iNdEx])))
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthUserListResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthUserListResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthUserListResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Users) > 0 {
+		for iNdEx := len(m.Users) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Users[iNdEx])
+			copy(dAtA[i:], m.Users[iNdEx])
+			i = encodeVarintRpc(dAtA, i, uint64(len(m.Users[iNdEx])))
+			i--
+			dAtA[i] = 0x12
+		}
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleDeleteResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleDeleteResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleGrantPermissionResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleGrantPermissionResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleGrantPermissionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *AuthRoleRevokePermissionResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *AuthRoleRevokePermissionResponse) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *AuthRoleRevokePermissionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Header != nil {
+		{
+			size, err := m.Header.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintRpc(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintRpc(dAtA []byte, offset int, v uint64) int {
+	offset -= sovRpc(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *ResponseHeader) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ClusterId != 0 {
+		n += 1 + sovRpc(uint64(m.ClusterId))
+	}
+	if m.MemberId != 0 {
+		n += 1 + sovRpc(uint64(m.MemberId))
+	}
+	if m.Revision != 0 {
+		n += 1 + sovRpc(uint64(m.Revision))
+	}
+	if m.RaftTerm != 0 {
+		n += 1 + sovRpc(uint64(m.RaftTerm))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *RangeRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Limit != 0 {
+		n += 1 + sovRpc(uint64(m.Limit))
+	}
+	if m.Revision != 0 {
+		n += 1 + sovRpc(uint64(m.Revision))
+	}
+	if m.SortOrder != 0 {
+		n += 1 + sovRpc(uint64(m.SortOrder))
+	}
+	if m.SortTarget != 0 {
+		n += 1 + sovRpc(uint64(m.SortTarget))
+	}
+	if m.Serializable {
+		n += 2
+	}
+	if m.KeysOnly {
+		n += 2
+	}
+	if m.CountOnly {
+		n += 2
+	}
+	if m.MinModRevision != 0 {
+		n += 1 + sovRpc(uint64(m.MinModRevision))
+	}
+	if m.MaxModRevision != 0 {
+		n += 1 + sovRpc(uint64(m.MaxModRevision))
+	}
+	if m.MinCreateRevision != 0 {
+		n += 1 + sovRpc(uint64(m.MinCreateRevision))
+	}
+	if m.MaxCreateRevision != 0 {
+		n += 1 + sovRpc(uint64(m.MaxCreateRevision))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *RangeResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Kvs) > 0 {
+		for _, e := range m.Kvs {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.More {
+		n += 2
+	}
+	if m.Count != 0 {
+		n += 1 + sovRpc(uint64(m.Count))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *PutRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Value)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Lease != 0 {
+		n += 1 + sovRpc(uint64(m.Lease))
+	}
+	if m.PrevKv {
+		n += 2
+	}
+	if m.IgnoreValue {
+		n += 2
+	}
+	if m.IgnoreLease {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *PutResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.PrevKv != nil {
+		l = m.PrevKv.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *DeleteRangeRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.PrevKv {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *DeleteRangeResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Deleted != 0 {
+		n += 1 + sovRpc(uint64(m.Deleted))
+	}
+	if len(m.PrevKvs) > 0 {
+		for _, e := range m.PrevKvs {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *RequestOp) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Request != nil {
+		n += m.Request.Size()
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *RequestOp_RequestRange) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.RequestRange != nil {
+		l = m.RequestRange.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *RequestOp_RequestPut) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.RequestPut != nil {
+		l = m.RequestPut.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *RequestOp_RequestDeleteRange) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.RequestDeleteRange != nil {
+		l = m.RequestDeleteRange.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *RequestOp_RequestTxn) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.RequestTxn != nil {
+		l = m.RequestTxn.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *ResponseOp) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Response != nil {
+		n += m.Response.Size()
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *ResponseOp_ResponseRange) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ResponseRange != nil {
+		l = m.ResponseRange.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *ResponseOp_ResponsePut) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ResponsePut != nil {
+		l = m.ResponsePut.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *ResponseOp_ResponseDeleteRange) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ResponseDeleteRange != nil {
+		l = m.ResponseDeleteRange.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *ResponseOp_ResponseTxn) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ResponseTxn != nil {
+		l = m.ResponseTxn.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *Compare) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Result != 0 {
+		n += 1 + sovRpc(uint64(m.Result))
+	}
+	if m.Target != 0 {
+		n += 1 + sovRpc(uint64(m.Target))
+	}
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.TargetUnion != nil {
+		n += m.TargetUnion.Size()
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 2 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Compare_Version) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRpc(uint64(m.Version))
+	return n
+}
+func (m *Compare_CreateRevision) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRpc(uint64(m.CreateRevision))
+	return n
+}
+func (m *Compare_ModRevision) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRpc(uint64(m.ModRevision))
+	return n
+}
+func (m *Compare_Value) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Value != nil {
+		l = len(m.Value)
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *Compare_Lease) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRpc(uint64(m.Lease))
+	return n
+}
+func (m *TxnRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if len(m.Compare) > 0 {
+		for _, e := range m.Compare {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if len(m.Success) > 0 {
+		for _, e := range m.Success {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if len(m.Failure) > 0 {
+		for _, e := range m.Failure {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *TxnResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Succeeded {
+		n += 2
+	}
+	if len(m.Responses) > 0 {
+		for _, e := range m.Responses {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *CompactionRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Revision != 0 {
+		n += 1 + sovRpc(uint64(m.Revision))
+	}
+	if m.Physical {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *CompactionResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *HashRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *HashKVRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Revision != 0 {
+		n += 1 + sovRpc(uint64(m.Revision))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *HashKVResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Hash != 0 {
+		n += 1 + sovRpc(uint64(m.Hash))
+	}
+	if m.CompactRevision != 0 {
+		n += 1 + sovRpc(uint64(m.CompactRevision))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *HashResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Hash != 0 {
+		n += 1 + sovRpc(uint64(m.Hash))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *SnapshotRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *SnapshotResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.RemainingBytes != 0 {
+		n += 1 + sovRpc(uint64(m.RemainingBytes))
+	}
+	l = len(m.Blob)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *WatchRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.RequestUnion != nil {
+		n += m.RequestUnion.Size()
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *WatchRequest_CreateRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.CreateRequest != nil {
+		l = m.CreateRequest.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *WatchRequest_CancelRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.CancelRequest != nil {
+		l = m.CancelRequest.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *WatchRequest_ProgressRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ProgressRequest != nil {
+		l = m.ProgressRequest.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+func (m *WatchCreateRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.StartRevision != 0 {
+		n += 1 + sovRpc(uint64(m.StartRevision))
+	}
+	if m.ProgressNotify {
+		n += 2
+	}
+	if len(m.Filters) > 0 {
+		l = 0
+		for _, e := range m.Filters {
+			l += sovRpc(uint64(e))
+		}
+		n += 1 + sovRpc(uint64(l)) + l
+	}
+	if m.PrevKv {
+		n += 2
+	}
+	if m.WatchId != 0 {
+		n += 1 + sovRpc(uint64(m.WatchId))
+	}
+	if m.Fragment {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *WatchCancelRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.WatchId != 0 {
+		n += 1 + sovRpc(uint64(m.WatchId))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *WatchProgressRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *WatchResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.WatchId != 0 {
+		n += 1 + sovRpc(uint64(m.WatchId))
+	}
+	if m.Created {
+		n += 2
+	}
+	if m.Canceled {
+		n += 2
+	}
+	if m.CompactRevision != 0 {
+		n += 1 + sovRpc(uint64(m.CompactRevision))
+	}
+	l = len(m.CancelReason)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Fragment {
+		n += 2
+	}
+	if len(m.Events) > 0 {
+		for _, e := range m.Events {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseGrantRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.TTL != 0 {
+		n += 1 + sovRpc(uint64(m.TTL))
+	}
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseGrantResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.TTL != 0 {
+		n += 1 + sovRpc(uint64(m.TTL))
+	}
+	l = len(m.Error)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseRevokeRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseRevokeResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseKeepAliveRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseKeepAliveResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.TTL != 0 {
+		n += 1 + sovRpc(uint64(m.TTL))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseTimeToLiveRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.Keys {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseTimeToLiveResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.TTL != 0 {
+		n += 1 + sovRpc(uint64(m.TTL))
+	}
+	if m.GrantedTTL != 0 {
+		n += 1 + sovRpc(uint64(m.GrantedTTL))
+	}
+	if len(m.Keys) > 0 {
+		for _, b := range m.Keys {
+			l = len(b)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseLeasesRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseStatus) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *LeaseLeasesResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Leases) > 0 {
+		for _, e := range m.Leases {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Member) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.PeerURLs) > 0 {
+		for _, s := range m.PeerURLs {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if len(m.ClientURLs) > 0 {
+		for _, s := range m.ClientURLs {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberAddRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if len(m.PeerURLs) > 0 {
+		for _, s := range m.PeerURLs {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberAddResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Member != nil {
+		l = m.Member.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Members) > 0 {
+		for _, e := range m.Members {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberRemoveRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberRemoveResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Members) > 0 {
+		for _, e := range m.Members {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberUpdateRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovRpc(uint64(m.ID))
+	}
+	if len(m.PeerURLs) > 0 {
+		for _, s := range m.PeerURLs {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberUpdateResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Members) > 0 {
+		for _, e := range m.Members {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberListRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MemberListResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Members) > 0 {
+		for _, e := range m.Members {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *DefragmentRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *DefragmentResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MoveLeaderRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.TargetID != 0 {
+		n += 1 + sovRpc(uint64(m.TargetID))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MoveLeaderResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AlarmRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Action != 0 {
+		n += 1 + sovRpc(uint64(m.Action))
+	}
+	if m.MemberID != 0 {
+		n += 1 + sovRpc(uint64(m.MemberID))
+	}
+	if m.Alarm != 0 {
+		n += 1 + sovRpc(uint64(m.Alarm))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AlarmMember) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.MemberID != 0 {
+		n += 1 + sovRpc(uint64(m.MemberID))
+	}
+	if m.Alarm != 0 {
+		n += 1 + sovRpc(uint64(m.Alarm))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AlarmResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Alarms) > 0 {
+		for _, e := range m.Alarms {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *StatusRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *StatusResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Version)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.DbSize != 0 {
+		n += 1 + sovRpc(uint64(m.DbSize))
+	}
+	if m.Leader != 0 {
+		n += 1 + sovRpc(uint64(m.Leader))
+	}
+	if m.RaftIndex != 0 {
+		n += 1 + sovRpc(uint64(m.RaftIndex))
+	}
+	if m.RaftTerm != 0 {
+		n += 1 + sovRpc(uint64(m.RaftTerm))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthEnableRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthDisableRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthenticateRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Password)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserAddRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Password)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserGetRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserDeleteRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserChangePasswordRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Password)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserGrantRoleRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.User)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Role)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserRevokeRoleRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Role)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleAddRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleGetRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Role)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserListRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleListRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleDeleteRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Role)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleGrantPermissionRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.Perm != nil {
+		l = m.Perm.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleRevokePermissionRequest) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Role)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.RangeEnd)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthEnableResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthDisableResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthenticateResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	l = len(m.Token)
+	if l > 0 {
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserAddResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserGetResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Roles) > 0 {
+		for _, s := range m.Roles {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserDeleteResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserChangePasswordResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserGrantRoleResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserRevokeRoleResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleAddResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleGetResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Perm) > 0 {
+		for _, e := range m.Perm {
+			l = e.Size()
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleListResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Roles) > 0 {
+		for _, s := range m.Roles {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthUserListResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if len(m.Users) > 0 {
+		for _, s := range m.Users {
+			l = len(s)
+			n += 1 + l + sovRpc(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleDeleteResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleGrantPermissionResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AuthRoleRevokePermissionResponse) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovRpc(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozRpc(x uint64) (n int) {
+	return sovRpc(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *ResponseHeader) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: ResponseHeader: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ResponseHeader: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ClusterId", wireType)
+			}
+			m.ClusterId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ClusterId |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MemberId", wireType)
+			}
+			m.MemberId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MemberId |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
+			}
+			m.Revision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Revision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RaftTerm", wireType)
+			}
+			m.RaftTerm = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RaftTerm |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *RangeRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: RangeRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: RangeRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
+			if m.RangeEnd == nil {
+				m.RangeEnd = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType)
+			}
+			m.Limit = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Limit |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
+			}
+			m.Revision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Revision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field SortOrder", wireType)
+			}
+			m.SortOrder = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.SortOrder |= RangeRequest_SortOrder(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 6:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field SortTarget", wireType)
+			}
+			m.SortTarget = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.SortTarget |= RangeRequest_SortTarget(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 7:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Serializable", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Serializable = bool(v != 0)
+		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field KeysOnly", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.KeysOnly = bool(v != 0)
+		case 9:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CountOnly", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.CountOnly = bool(v != 0)
+		case 10:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MinModRevision", wireType)
+			}
+			m.MinModRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MinModRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 11:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MaxModRevision", wireType)
+			}
+			m.MaxModRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MaxModRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 12:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MinCreateRevision", wireType)
+			}
+			m.MinCreateRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MinCreateRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 13:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MaxCreateRevision", wireType)
+			}
+			m.MaxCreateRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MaxCreateRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *RangeResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: RangeResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: RangeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Kvs", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Kvs = append(m.Kvs, &mvccpb.KeyValue{})
+			if err := m.Kvs[len(m.Kvs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field More", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.More = bool(v != 0)
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType)
+			}
+			m.Count = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Count |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *PutRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: PutRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: PutRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...)
+			if m.Value == nil {
+				m.Value = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Lease |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.PrevKv = bool(v != 0)
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field IgnoreValue", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.IgnoreValue = bool(v != 0)
+		case 6:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field IgnoreLease", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.IgnoreLease = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *PutResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: PutResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: PutResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.PrevKv == nil {
+				m.PrevKv = &mvccpb.KeyValue{}
+			}
+			if err := m.PrevKv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *DeleteRangeRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: DeleteRangeRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: DeleteRangeRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
+			if m.RangeEnd == nil {
+				m.RangeEnd = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.PrevKv = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *DeleteRangeResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: DeleteRangeResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: DeleteRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Deleted", wireType)
+			}
+			m.Deleted = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Deleted |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKvs", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PrevKvs = append(m.PrevKvs, &mvccpb.KeyValue{})
+			if err := m.PrevKvs[len(m.PrevKvs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *RequestOp) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: RequestOp: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: RequestOp: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RequestRange", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &RangeRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Request = &RequestOp_RequestRange{v}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RequestPut", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &PutRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Request = &RequestOp_RequestPut{v}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RequestDeleteRange", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &DeleteRangeRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Request = &RequestOp_RequestDeleteRange{v}
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RequestTxn", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &TxnRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Request = &RequestOp_RequestTxn{v}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *ResponseOp) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: ResponseOp: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ResponseOp: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ResponseRange", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &RangeResponse{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Response = &ResponseOp_ResponseRange{v}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ResponsePut", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &PutResponse{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Response = &ResponseOp_ResponsePut{v}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ResponseDeleteRange", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &DeleteRangeResponse{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Response = &ResponseOp_ResponseDeleteRange{v}
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ResponseTxn", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &TxnResponse{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.Response = &ResponseOp_ResponseTxn{v}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *Compare) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Compare: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Compare: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
+			}
+			m.Result = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Result |= Compare_CompareResult(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType)
+			}
+			m.Target = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Target |= Compare_CompareTarget(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
+			}
+			var v int64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.TargetUnion = &Compare_Version{v}
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType)
+			}
+			var v int64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.TargetUnion = &Compare_CreateRevision{v}
+		case 6:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType)
+			}
+			var v int64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.TargetUnion = &Compare_ModRevision{v}
+		case 7:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := make([]byte, postIndex-iNdEx)
+			copy(v, dAtA[iNdEx:postIndex])
+			m.TargetUnion = &Compare_Value{v}
+			iNdEx = postIndex
+		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Lease", wireType)
+			}
+			var v int64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.TargetUnion = &Compare_Lease{v}
+		case 64:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
+			if m.RangeEnd == nil {
+				m.RangeEnd = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *TxnRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: TxnRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: TxnRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Compare", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Compare = append(m.Compare, &Compare{})
+			if err := m.Compare[len(m.Compare)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Success", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Success = append(m.Success, &RequestOp{})
+			if err := m.Success[len(m.Success)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Failure", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Failure = append(m.Failure, &RequestOp{})
+			if err := m.Failure[len(m.Failure)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *TxnResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: TxnResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: TxnResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Succeeded", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Succeeded = bool(v != 0)
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Responses = append(m.Responses, &ResponseOp{})
+			if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *CompactionRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: CompactionRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: CompactionRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
+			}
+			m.Revision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Revision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Physical", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Physical = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *CompactionResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: CompactionResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: CompactionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *HashRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: HashRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: HashRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *HashKVRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: HashKVRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: HashKVRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
+			}
+			m.Revision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Revision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *HashKVResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: HashKVResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: HashKVResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType)
+			}
+			m.Hash = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Hash |= uint32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CompactRevision", wireType)
+			}
+			m.CompactRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.CompactRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *HashResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: HashResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: HashResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType)
+			}
+			m.Hash = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Hash |= uint32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *SnapshotRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: SnapshotRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SnapshotRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *SnapshotResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: SnapshotResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SnapshotResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RemainingBytes", wireType)
+			}
+			m.RemainingBytes = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RemainingBytes |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Blob", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Blob = append(m.Blob[:0], dAtA[iNdEx:postIndex]...)
+			if m.Blob == nil {
+				m.Blob = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *WatchRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: WatchRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: WatchRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CreateRequest", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &WatchCreateRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.RequestUnion = &WatchRequest_CreateRequest{v}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CancelRequest", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &WatchCancelRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.RequestUnion = &WatchRequest_CancelRequest{v}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ProgressRequest", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			v := &WatchProgressRequest{}
+			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			m.RequestUnion = &WatchRequest_ProgressRequest{v}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *WatchCreateRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: WatchCreateRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: WatchCreateRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
+			if m.RangeEnd == nil {
+				m.RangeEnd = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field StartRevision", wireType)
+			}
+			m.StartRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.StartRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ProgressNotify", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.ProgressNotify = bool(v != 0)
+		case 5:
+			if wireType == 0 {
+				var v WatchCreateRequest_FilterType
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRpc
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= WatchCreateRequest_FilterType(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.Filters = append(m.Filters, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRpc
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthRpc
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthRpc
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				if elementCount != 0 && len(m.Filters) == 0 {
+					m.Filters = make([]WatchCreateRequest_FilterType, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v WatchCreateRequest_FilterType
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowRpc
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= WatchCreateRequest_FilterType(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.Filters = append(m.Filters, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field Filters", wireType)
+			}
+		case 6:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.PrevKv = bool(v != 0)
+		case 7:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
+			}
+			m.WatchId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.WatchId |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Fragment = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *WatchCancelRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: WatchCancelRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: WatchCancelRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
+			}
+			m.WatchId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.WatchId |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *WatchProgressRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: WatchProgressRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: WatchProgressRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *WatchResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: WatchResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: WatchResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
+			}
+			m.WatchId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.WatchId |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Created", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Created = bool(v != 0)
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Canceled", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Canceled = bool(v != 0)
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CompactRevision", wireType)
+			}
+			m.CompactRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.CompactRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CancelReason", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.CancelReason = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 7:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Fragment = bool(v != 0)
+		case 11:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Events = append(m.Events, &mvccpb.Event{})
+			if err := m.Events[len(m.Events)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseGrantRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseGrantRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseGrantRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TTL |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseGrantResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseGrantResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseGrantResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TTL |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Error = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseRevokeRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseRevokeRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseRevokeRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseRevokeResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseRevokeResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseRevokeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseKeepAliveRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseKeepAliveRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseKeepAliveRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseKeepAliveResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseKeepAliveResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseKeepAliveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TTL |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseTimeToLiveRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseTimeToLiveRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseTimeToLiveRequest: 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 ErrIntOverflowRpc
+				}
+				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 Keys", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Keys = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseTimeToLiveResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseTimeToLiveResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseTimeToLiveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TTL |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field GrantedTTL", wireType)
+			}
+			m.GrantedTTL = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.GrantedTTL |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx))
+			copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseLeasesRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseLeasesRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseLeasesRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseStatus) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseStatus: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseStatus: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *LeaseLeasesResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LeaseLeasesResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseLeasesResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Leases", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Leases = append(m.Leases, &LeaseStatus{})
+			if err := m.Leases[len(m.Leases)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *Member) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Member: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Member: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ClientURLs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.ClientURLs = append(m.ClientURLs, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberAddRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberAddRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberAddRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberAddResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberAddResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Member", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Member == nil {
+				m.Member = &Member{}
+			}
+			if err := m.Member.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Members = append(m.Members, &Member{})
+			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberRemoveRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberRemoveRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberRemoveRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberRemoveResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberRemoveResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberRemoveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Members = append(m.Members, &Member{})
+			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberUpdateRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberUpdateRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberUpdateRequest: 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 ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberUpdateResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberUpdateResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberUpdateResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Members = append(m.Members, &Member{})
+			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberListRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberListRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MemberListResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MemberListResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MemberListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Members = append(m.Members, &Member{})
+			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *DefragmentRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: DefragmentRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: DefragmentRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *DefragmentResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: DefragmentResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: DefragmentResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MoveLeaderRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MoveLeaderRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MoveLeaderRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field TargetID", wireType)
+			}
+			m.TargetID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TargetID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *MoveLeaderResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MoveLeaderResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MoveLeaderResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AlarmRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AlarmRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AlarmRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType)
+			}
+			m.Action = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Action |= AlarmRequest_AlarmAction(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MemberID", wireType)
+			}
+			m.MemberID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MemberID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
+			}
+			m.Alarm = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Alarm |= AlarmType(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AlarmMember) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AlarmMember: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AlarmMember: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MemberID", wireType)
+			}
+			m.MemberID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.MemberID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
+			}
+			m.Alarm = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Alarm |= AlarmType(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AlarmResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AlarmResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AlarmResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Alarms", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Alarms = append(m.Alarms, &AlarmMember{})
+			if err := m.Alarms[len(m.Alarms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *StatusRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: StatusRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: StatusRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *StatusResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: StatusResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: StatusResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Version = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field DbSize", wireType)
+			}
+			m.DbSize = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.DbSize |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType)
+			}
+			m.Leader = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Leader |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RaftIndex", wireType)
+			}
+			m.RaftIndex = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RaftIndex |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 6:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RaftTerm", wireType)
+			}
+			m.RaftTerm = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RaftTerm |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthEnableRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthEnableRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthEnableRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthDisableRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthDisableRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthDisableRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthenticateRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthenticateRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthenticateRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Password = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserAddRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserAddRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserAddRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Password = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserGetRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserGetRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserGetRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserDeleteRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserDeleteRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserDeleteRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserChangePasswordRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserChangePasswordRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserChangePasswordRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Password = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserGrantRoleRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserGrantRoleRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserGrantRoleRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.User = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Role = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserRevokeRoleRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserRevokeRoleRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserRevokeRoleRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Role = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleAddRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleAddRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleAddRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleGetRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleGetRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleGetRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Role = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserListRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserListRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleListRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleListRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleDeleteRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleDeleteRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Role = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleGrantPermissionRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleGrantPermissionRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleGrantPermissionRequest: 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 stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Perm", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Perm == nil {
+				m.Perm = &authpb.Permission{}
+			}
+			if err := m.Perm.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleRevokePermissionRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleRevokePermissionRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleRevokePermissionRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Role = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Key = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RangeEnd = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthEnableResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthEnableResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthEnableResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthDisableResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthDisableResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthDisableResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthenticateResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthenticateResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthenticateResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Token", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Token = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserAddResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserAddResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserGetResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserGetResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserGetResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserDeleteResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserDeleteResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserChangePasswordResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserChangePasswordResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserChangePasswordResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserGrantRoleResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserGrantRoleResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserGrantRoleResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserRevokeRoleResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserRevokeRoleResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserRevokeRoleResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleAddResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleAddResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleGetResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleGetResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleGetResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Perm", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Perm = append(m.Perm, &authpb.Permission{})
+			if err := m.Perm[len(m.Perm)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleListResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleListResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthUserListResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthUserListResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthUserListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Users", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Users = append(m.Users, string(dAtA[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleDeleteResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleDeleteResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleGrantPermissionResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleGrantPermissionResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleGrantPermissionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 *AuthRoleRevokePermissionResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AuthRoleRevokePermissionResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AuthRoleRevokePermissionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRpc
+			}
+			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 skipRpc(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, ErrIntOverflowRpc
+			}
+			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, ErrIntOverflowRpc
+				}
+				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, ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthRpc
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthRpc
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowRpc
+					}
+					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 := skipRpc(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthRpc
+				}
+			}
+			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 (
+	ErrInvalidLengthRpc = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowRpc   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.proto b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.proto
new file mode 100644
index 0000000..d9da43c
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.proto
@@ -0,0 +1,1075 @@
+syntax = "proto3";
+package etcdserverpb;
+
+import "gogoproto/gogo.proto";
+import "etcd/mvcc/mvccpb/kv.proto";
+import "etcd/auth/authpb/auth.proto";
+
+// for grpc-gateway
+import "google/api/annotations.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+
+service KV {
+  // Range gets the keys in the range from the key-value store.
+  rpc Range(RangeRequest) returns (RangeResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/range"
+        body: "*"
+    };
+  }
+
+  // Put puts the given key into the key-value store.
+  // A put request increments the revision of the key-value store
+  // and generates one event in the event history.
+  rpc Put(PutRequest) returns (PutResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/put"
+        body: "*"
+    };
+  }
+
+  // DeleteRange deletes the given range from the key-value store.
+  // A delete request increments the revision of the key-value store
+  // and generates a delete event in the event history for every deleted key.
+  rpc DeleteRange(DeleteRangeRequest) returns (DeleteRangeResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/deleterange"
+        body: "*"
+    };
+  }
+
+  // Txn processes multiple requests in a single transaction.
+  // A txn request increments the revision of the key-value store
+  // and generates events with the same revision for every completed request.
+  // It is not allowed to modify the same key several times within one txn.
+  rpc Txn(TxnRequest) returns (TxnResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/txn"
+        body: "*"
+    };
+  }
+
+  // Compact compacts the event history in the etcd key-value store. The key-value
+  // store should be periodically compacted or the event history will continue to grow
+  // indefinitely.
+  rpc Compact(CompactionRequest) returns (CompactionResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/compaction"
+        body: "*"
+    };
+  }
+}
+
+service Watch {
+  // Watch watches for events happening or that have happened. Both input and output
+  // are streams; the input stream is for creating and canceling watchers and the output
+  // stream sends events. One watch RPC can watch on multiple key ranges, streaming events
+  // for several watches at once. The entire event history can be watched starting from the
+  // last compaction revision.
+  rpc Watch(stream WatchRequest) returns (stream WatchResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/watch"
+        body: "*"
+    };
+  }
+}
+
+service Lease {
+  // LeaseGrant creates a lease which expires if the server does not receive a keepAlive
+  // within a given time to live period. All keys attached to the lease will be expired and
+  // deleted if the lease expires. Each expired key generates a delete event in the event history.
+  rpc LeaseGrant(LeaseGrantRequest) returns (LeaseGrantResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/lease/grant"
+        body: "*"
+    };
+  }
+
+  // LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
+  rpc LeaseRevoke(LeaseRevokeRequest) returns (LeaseRevokeResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/lease/revoke"
+        body: "*"
+    };
+  }
+
+  // LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
+  // to the server and streaming keep alive responses from the server to the client.
+  rpc LeaseKeepAlive(stream LeaseKeepAliveRequest) returns (stream LeaseKeepAliveResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/lease/keepalive"
+        body: "*"
+    };
+  }
+
+  // LeaseTimeToLive retrieves lease information.
+  rpc LeaseTimeToLive(LeaseTimeToLiveRequest) returns (LeaseTimeToLiveResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/lease/timetolive"
+        body: "*"
+    };
+  }
+
+  // LeaseLeases lists all existing leases.
+  rpc LeaseLeases(LeaseLeasesRequest) returns (LeaseLeasesResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/kv/lease/leases"
+        body: "*"
+    };
+  }
+}
+
+service Cluster {
+  // MemberAdd adds a member into the cluster.
+  rpc MemberAdd(MemberAddRequest) returns (MemberAddResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/cluster/member/add"
+        body: "*"
+    };
+  }
+
+  // MemberRemove removes an existing member from the cluster.
+  rpc MemberRemove(MemberRemoveRequest) returns (MemberRemoveResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/cluster/member/remove"
+        body: "*"
+    };
+  }
+
+  // MemberUpdate updates the member configuration.
+  rpc MemberUpdate(MemberUpdateRequest) returns (MemberUpdateResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/cluster/member/update"
+        body: "*"
+    };
+  }
+
+  // MemberList lists all the members in the cluster.
+  rpc MemberList(MemberListRequest) returns (MemberListResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/cluster/member/list"
+        body: "*"
+    };
+  }
+}
+
+service Maintenance {
+  // Alarm activates, deactivates, and queries alarms regarding cluster health.
+  rpc Alarm(AlarmRequest) returns (AlarmResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/alarm"
+        body: "*"
+    };
+  }
+
+  // Status gets the status of the member.
+  rpc Status(StatusRequest) returns (StatusResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/status"
+        body: "*"
+    };
+  }
+
+  // Defragment defragments a member's backend database to recover storage space.
+  rpc Defragment(DefragmentRequest) returns (DefragmentResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/defragment"
+        body: "*"
+    };
+  }
+
+  // Hash computes the hash of the KV's backend.
+  // This is designed for testing; do not use this in production when there
+  // are ongoing transactions.
+  rpc Hash(HashRequest) returns (HashResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/hash"
+        body: "*"
+    };
+  }
+
+  // HashKV computes the hash of all MVCC keys up to a given revision.
+  rpc HashKV(HashKVRequest) returns (HashKVResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/hash"
+        body: "*"
+    };
+  }
+
+  // Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
+  rpc Snapshot(SnapshotRequest) returns (stream SnapshotResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/snapshot"
+        body: "*"
+    };
+  }
+
+  // MoveLeader requests current leader node to transfer its leadership to transferee.
+  rpc MoveLeader(MoveLeaderRequest) returns (MoveLeaderResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/maintenance/transfer-leadership"
+        body: "*"
+    };
+  }
+}
+
+service Auth {
+  // AuthEnable enables authentication.
+  rpc AuthEnable(AuthEnableRequest) returns (AuthEnableResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/enable"
+        body: "*"
+    };
+  }
+
+  // AuthDisable disables authentication.
+  rpc AuthDisable(AuthDisableRequest) returns (AuthDisableResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/disable"
+        body: "*"
+    };
+  }
+
+  // Authenticate processes an authenticate request.
+  rpc Authenticate(AuthenticateRequest) returns (AuthenticateResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/authenticate"
+        body: "*"
+    };
+  }
+
+  // UserAdd adds a new user.
+  rpc UserAdd(AuthUserAddRequest) returns (AuthUserAddResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/add"
+        body: "*"
+    };
+  }
+
+  // UserGet gets detailed user information.
+  rpc UserGet(AuthUserGetRequest) returns (AuthUserGetResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/get"
+        body: "*"
+    };
+  }
+
+  // UserList gets a list of all users.
+  rpc UserList(AuthUserListRequest) returns (AuthUserListResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/list"
+        body: "*"
+    };
+  }
+
+  // UserDelete deletes a specified user.
+  rpc UserDelete(AuthUserDeleteRequest) returns (AuthUserDeleteResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/delete"
+        body: "*"
+    };
+  }
+
+  // UserChangePassword changes the password of a specified user.
+  rpc UserChangePassword(AuthUserChangePasswordRequest) returns (AuthUserChangePasswordResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/changepw"
+        body: "*"
+    };
+  }
+
+  // UserGrant grants a role to a specified user.
+  rpc UserGrantRole(AuthUserGrantRoleRequest) returns (AuthUserGrantRoleResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/grant"
+        body: "*"
+    };
+  }
+
+  // UserRevokeRole revokes a role of specified user.
+  rpc UserRevokeRole(AuthUserRevokeRoleRequest) returns (AuthUserRevokeRoleResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/user/revoke"
+        body: "*"
+    };
+  }
+
+  // RoleAdd adds a new role.
+  rpc RoleAdd(AuthRoleAddRequest) returns (AuthRoleAddResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/add"
+        body: "*"
+    };
+  }
+
+  // RoleGet gets detailed role information.
+  rpc RoleGet(AuthRoleGetRequest) returns (AuthRoleGetResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/get"
+        body: "*"
+    };
+  }
+
+  // RoleList gets lists of all roles.
+  rpc RoleList(AuthRoleListRequest) returns (AuthRoleListResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/list"
+        body: "*"
+    };
+  }
+
+  // RoleDelete deletes a specified role.
+  rpc RoleDelete(AuthRoleDeleteRequest) returns (AuthRoleDeleteResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/delete"
+        body: "*"
+    };
+  }
+
+  // RoleGrantPermission grants a permission of a specified key or range to a specified role.
+  rpc RoleGrantPermission(AuthRoleGrantPermissionRequest) returns (AuthRoleGrantPermissionResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/grant"
+        body: "*"
+    };
+  }
+
+  // RoleRevokePermission revokes a key or range permission of a specified role.
+  rpc RoleRevokePermission(AuthRoleRevokePermissionRequest) returns (AuthRoleRevokePermissionResponse) {
+      option (google.api.http) = {
+        post: "/v3beta/auth/role/revoke"
+        body: "*"
+    };
+  }
+}
+
+message ResponseHeader {
+  // cluster_id is the ID of the cluster which sent the response.
+  uint64 cluster_id = 1;
+  // member_id is the ID of the member which sent the response.
+  uint64 member_id = 2;
+  // revision is the key-value store revision when the request was applied.
+  // For watch progress responses, the header.revision indicates progress. All future events
+  // recieved in this stream are guaranteed to have a higher revision number than the
+  // header.revision number.
+  int64 revision = 3;
+  // raft_term is the raft term when the request was applied.
+  uint64 raft_term = 4;
+}
+
+message RangeRequest {
+  enum SortOrder {
+	NONE = 0; // default, no sorting
+	ASCEND = 1; // lowest target value first
+	DESCEND = 2; // highest target value first
+  }
+  enum SortTarget {
+	KEY = 0;
+	VERSION = 1;
+	CREATE = 2;
+	MOD = 3;
+	VALUE = 4;
+  }
+
+  // key is the first key for the range. If range_end is not given, the request only looks up key.
+  bytes key = 1;
+  // range_end is the upper bound on the requested range [key, range_end).
+  // If range_end is '\0', the range is all keys >= key.
+  // If range_end is key plus one (e.g., "aa"+1 == "ab", "a\xff"+1 == "b"),
+  // then the range request gets all keys prefixed with key.
+  // If both key and range_end are '\0', then the range request returns all keys.
+  bytes range_end = 2;
+  // limit is a limit on the number of keys returned for the request. When limit is set to 0,
+  // it is treated as no limit.
+  int64 limit = 3;
+  // revision is the point-in-time of the key-value store to use for the range.
+  // If revision is less or equal to zero, the range is over the newest key-value store.
+  // If the revision has been compacted, ErrCompacted is returned as a response.
+  int64 revision = 4;
+
+  // sort_order is the order for returned sorted results.
+  SortOrder sort_order = 5;
+
+  // sort_target is the key-value field to use for sorting.
+  SortTarget sort_target = 6;
+
+  // serializable sets the range request to use serializable member-local reads.
+  // Range requests are linearizable by default; linearizable requests have higher
+  // latency and lower throughput than serializable requests but reflect the current
+  // consensus of the cluster. For better performance, in exchange for possible stale reads,
+  // a serializable range request is served locally without needing to reach consensus
+  // with other nodes in the cluster.
+  bool serializable = 7;
+
+  // keys_only when set returns only the keys and not the values.
+  bool keys_only = 8;
+
+  // count_only when set returns only the count of the keys in the range.
+  bool count_only = 9;
+
+  // min_mod_revision is the lower bound for returned key mod revisions; all keys with
+  // lesser mod revisions will be filtered away.
+  int64 min_mod_revision = 10;
+
+  // max_mod_revision is the upper bound for returned key mod revisions; all keys with
+  // greater mod revisions will be filtered away.
+  int64 max_mod_revision = 11;
+
+  // min_create_revision is the lower bound for returned key create revisions; all keys with
+  // lesser create trevisions will be filtered away.
+  int64 min_create_revision = 12;
+
+  // max_create_revision is the upper bound for returned key create revisions; all keys with
+  // greater create revisions will be filtered away.
+  int64 max_create_revision = 13;
+}
+
+message RangeResponse {
+  ResponseHeader header = 1;
+  // kvs is the list of key-value pairs matched by the range request.
+  // kvs is empty when count is requested.
+  repeated mvccpb.KeyValue kvs = 2;
+  // more indicates if there are more keys to return in the requested range.
+  bool more = 3;
+  // count is set to the number of keys within the range when requested.
+  int64 count = 4;
+}
+
+message PutRequest {
+  // key is the key, in bytes, to put into the key-value store.
+  bytes key = 1;
+  // value is the value, in bytes, to associate with the key in the key-value store.
+  bytes value = 2;
+  // lease is the lease ID to associate with the key in the key-value store. A lease
+  // value of 0 indicates no lease.
+  int64 lease = 3;
+
+  // If prev_kv is set, etcd gets the previous key-value pair before changing it.
+  // The previous key-value pair will be returned in the put response.
+  bool prev_kv = 4;
+
+  // If ignore_value is set, etcd updates the key using its current value.
+  // Returns an error if the key does not exist.
+  bool ignore_value = 5;
+
+  // If ignore_lease is set, etcd updates the key using its current lease.
+  // Returns an error if the key does not exist.
+  bool ignore_lease = 6;
+}
+
+message PutResponse {
+  ResponseHeader header = 1;
+  // if prev_kv is set in the request, the previous key-value pair will be returned.
+  mvccpb.KeyValue prev_kv = 2;
+}
+
+message DeleteRangeRequest {
+  // key is the first key to delete in the range.
+  bytes key = 1;
+  // range_end is the key following the last key to delete for the range [key, range_end).
+  // If range_end is not given, the range is defined to contain only the key argument.
+  // If range_end is one bit larger than the given key, then the range is all the keys
+  // with the prefix (the given key).
+  // If range_end is '\0', the range is all keys greater than or equal to the key argument.
+  bytes range_end = 2;
+
+  // If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
+  // The previous key-value pairs will be returned in the delete response.
+  bool prev_kv = 3;
+}
+
+message DeleteRangeResponse {
+  ResponseHeader header = 1;
+  // deleted is the number of keys deleted by the delete range request.
+  int64 deleted = 2;
+  // if prev_kv is set in the request, the previous key-value pairs will be returned.
+  repeated mvccpb.KeyValue prev_kvs = 3;
+}
+
+message RequestOp {
+  // request is a union of request types accepted by a transaction.
+  oneof request {
+    RangeRequest request_range = 1;
+    PutRequest request_put = 2;
+    DeleteRangeRequest request_delete_range = 3;
+    TxnRequest request_txn = 4;
+  }
+}
+
+message ResponseOp {
+  // response is a union of response types returned by a transaction.
+  oneof response {
+    RangeResponse response_range = 1;
+    PutResponse response_put = 2;
+    DeleteRangeResponse response_delete_range = 3;
+    TxnResponse response_txn = 4;
+  }
+}
+
+message Compare {
+  enum CompareResult {
+    EQUAL = 0;
+    GREATER = 1;
+    LESS = 2;
+    NOT_EQUAL = 3;
+  }
+  enum CompareTarget {
+    VERSION = 0;
+    CREATE = 1;
+    MOD = 2;
+    VALUE= 3;
+    LEASE = 4;
+  }
+  // result is logical comparison operation for this comparison.
+  CompareResult result = 1;
+  // target is the key-value field to inspect for the comparison.
+  CompareTarget target = 2;
+  // key is the subject key for the comparison operation.
+  bytes key = 3;
+  oneof target_union {
+    // version is the version of the given key
+    int64 version = 4;
+    // create_revision is the creation revision of the given key
+    int64 create_revision = 5;
+    // mod_revision is the last modified revision of the given key.
+    int64 mod_revision = 6;
+    // value is the value of the given key, in bytes.
+    bytes value = 7;
+    // lease is the lease id of the given key.
+    int64 lease = 8;
+    // leave room for more target_union field tags, jump to 64
+  }
+
+  // range_end compares the given target to all keys in the range [key, range_end).
+  // See RangeRequest for more details on key ranges.
+  bytes range_end = 64;
+  // TODO: fill out with most of the rest of RangeRequest fields when needed.
+}
+
+// From google paxosdb paper:
+// Our implementation hinges around a powerful primitive which we call MultiOp. All other database
+// operations except for iteration are implemented as a single call to MultiOp. A MultiOp is applied atomically
+// and consists of three components:
+// 1. A list of tests called guard. Each test in guard checks a single entry in the database. It may check
+// for the absence or presence of a value, or compare with a given value. Two different tests in the guard
+// may apply to the same or different entries in the database. All tests in the guard are applied and
+// MultiOp returns the results. If all tests are true, MultiOp executes t op (see item 2 below), otherwise
+// it executes f op (see item 3 below).
+// 2. A list of database operations called t op. Each operation in the list is either an insert, delete, or
+// lookup operation, and applies to a single database entry. Two different operations in the list may apply
+// to the same or different entries in the database. These operations are executed
+// if guard evaluates to
+// true.
+// 3. A list of database operations called f op. Like t op, but executed if guard evaluates to false.
+message TxnRequest {
+  // compare is a list of predicates representing a conjunction of terms.
+  // If the comparisons succeed, then the success requests will be processed in order,
+  // and the response will contain their respective responses in order.
+  // If the comparisons fail, then the failure requests will be processed in order,
+  // and the response will contain their respective responses in order.
+  repeated Compare compare = 1;
+  // success is a list of requests which will be applied when compare evaluates to true.
+  repeated RequestOp success = 2;
+  // failure is a list of requests which will be applied when compare evaluates to false.
+  repeated RequestOp failure = 3;
+}
+
+message TxnResponse {
+  ResponseHeader header = 1;
+  // succeeded is set to true if the compare evaluated to true or false otherwise.
+  bool succeeded = 2;
+  // responses is a list of responses corresponding to the results from applying
+  // success if succeeded is true or failure if succeeded is false.
+  repeated ResponseOp responses = 3;
+}
+
+// CompactionRequest compacts the key-value store up to a given revision. All superseded keys
+// with a revision less than the compaction revision will be removed.
+message CompactionRequest {
+  // revision is the key-value store revision for the compaction operation.
+  int64 revision = 1;
+  // physical is set so the RPC will wait until the compaction is physically
+  // applied to the local database such that compacted entries are totally
+  // removed from the backend database.
+  bool physical = 2;
+}
+
+message CompactionResponse {
+  ResponseHeader header = 1;
+}
+
+message HashRequest {
+}
+
+message HashKVRequest {
+  // revision is the key-value store revision for the hash operation.
+  int64 revision = 1;
+}
+
+message HashKVResponse {
+  ResponseHeader header = 1;
+  // hash is the hash value computed from the responding member's MVCC keys up to a given revision.
+  uint32 hash = 2;
+  // compact_revision is the compacted revision of key-value store when hash begins.
+  int64 compact_revision = 3;
+}
+
+message HashResponse {
+  ResponseHeader header = 1;
+  // hash is the hash value computed from the responding member's KV's backend.
+  uint32 hash = 2;
+}
+
+message SnapshotRequest {
+}
+
+message SnapshotResponse {
+  // header has the current key-value store information. The first header in the snapshot
+  // stream indicates the point in time of the snapshot.
+  ResponseHeader header = 1;
+
+  // remaining_bytes is the number of blob bytes to be sent after this message
+  uint64 remaining_bytes = 2;
+
+  // blob contains the next chunk of the snapshot in the snapshot stream.
+  bytes blob = 3;
+}
+
+message WatchRequest {
+  // request_union is a request to either create a new watcher or cancel an existing watcher.
+  oneof request_union {
+    WatchCreateRequest create_request = 1;
+    WatchCancelRequest cancel_request = 2;
+    WatchProgressRequest progress_request = 3;
+  }
+}
+
+message WatchCreateRequest {
+  // key is the key to register for watching.
+  bytes key = 1;
+  // range_end is the end of the range [key, range_end) to watch. If range_end is not given,
+  // only the key argument is watched. If range_end is equal to '\0', all keys greater than
+  // or equal to the key argument are watched.
+  // If the range_end is one bit larger than the given key,
+  // then all keys with the prefix (the given key) will be watched.
+  bytes range_end = 2;
+  // start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
+  int64 start_revision = 3;
+  // progress_notify is set so that the etcd server will periodically send a WatchResponse with
+  // no events to the new watcher if there are no recent events. It is useful when clients
+  // wish to recover a disconnected watcher starting from a recent known revision.
+  // The etcd server may decide how often it will send notifications based on current load.
+  bool progress_notify = 4;
+
+  enum FilterType {
+  // filter out put event.
+  NOPUT = 0;
+  // filter out delete event.
+  NODELETE = 1;
+  }
+  // filters filter the events at server side before it sends back to the watcher.
+  repeated FilterType filters = 5;
+
+  // If prev_kv is set, created watcher gets the previous KV before the event happens.
+  // If the previous KV is already compacted, nothing will be returned.
+  bool prev_kv = 6;
+
+  // If watch_id is provided and non-zero, it will be assigned to this watcher.
+  // Since creating a watcher in etcd is not a synchronous operation,
+  // this can be used ensure that ordering is correct when creating multiple
+  // watchers on the same stream. Creating a watcher with an ID already in
+  // use on the stream will cause an error to be returned.
+  int64 watch_id = 7;
+
+  // fragment enables splitting large revisions into multiple watch responses.
+  bool fragment = 8;
+}
+
+message WatchCancelRequest {
+  // watch_id is the watcher id to cancel so that no more events are transmitted.
+  int64 watch_id = 1;
+}
+
+// Requests the a watch stream progress status be sent in the watch response stream as soon as
+// possible.
+message WatchProgressRequest {
+}
+
+message WatchResponse {
+  ResponseHeader header = 1;
+  // watch_id is the ID of the watcher that corresponds to the response.
+  int64 watch_id = 2;
+  // created is set to true if the response is for a create watch request.
+  // The client should record the watch_id and expect to receive events for
+  // the created watcher from the same stream.
+  // All events sent to the created watcher will attach with the same watch_id.
+  bool created = 3;
+  // canceled is set to true if the response is for a cancel watch request.
+  // No further events will be sent to the canceled watcher.
+  bool canceled = 4;
+  // compact_revision is set to the minimum index if a watcher tries to watch
+  // at a compacted index.
+  //
+  // This happens when creating a watcher at a compacted revision or the watcher cannot
+  // catch up with the progress of the key-value store.
+  //
+  // The client should treat the watcher as canceled and should not try to create any
+  // watcher with the same start_revision again.
+  int64 compact_revision  = 5;
+
+  // cancel_reason indicates the reason for canceling the watcher.
+  string cancel_reason = 6;
+
+  // framgment is true if large watch response was split over multiple responses.
+  bool fragment = 7;
+
+  repeated mvccpb.Event events = 11;
+}
+
+message LeaseGrantRequest {
+  // TTL is the advisory time-to-live in seconds. Expired lease will return -1.
+  int64 TTL = 1;
+  // ID is the requested ID for the lease. If ID is set to 0, the lessor chooses an ID.
+  int64 ID = 2;
+}
+
+message LeaseGrantResponse {
+  ResponseHeader header = 1;
+  // ID is the lease ID for the granted lease.
+  int64 ID = 2;
+  // TTL is the server chosen lease time-to-live in seconds.
+  int64 TTL = 3;
+  string error = 4;
+}
+
+message LeaseRevokeRequest {
+  // ID is the lease ID to revoke. When the ID is revoked, all associated keys will be deleted.
+  int64 ID = 1;
+}
+
+message LeaseRevokeResponse {
+  ResponseHeader header = 1;
+}
+
+message LeaseKeepAliveRequest {
+  // ID is the lease ID for the lease to keep alive.
+  int64 ID = 1;
+}
+
+message LeaseKeepAliveResponse {
+  ResponseHeader header = 1;
+  // ID is the lease ID from the keep alive request.
+  int64 ID = 2;
+  // TTL is the new time-to-live for the lease.
+  int64 TTL = 3;
+}
+
+message LeaseTimeToLiveRequest {
+  // ID is the lease ID for the lease.
+  int64 ID = 1;
+  // keys is true to query all the keys attached to this lease.
+  bool keys = 2;
+}
+
+message LeaseTimeToLiveResponse {
+  ResponseHeader header = 1;
+  // ID is the lease ID from the keep alive request.
+  int64 ID = 2;
+  // TTL is the remaining TTL in seconds for the lease; the lease will expire in under TTL+1 seconds.
+  int64 TTL = 3;
+  // GrantedTTL is the initial granted time in seconds upon lease creation/renewal.
+  int64 grantedTTL = 4;
+  // Keys is the list of keys attached to this lease.
+  repeated bytes keys = 5;
+}
+
+message LeaseLeasesRequest {
+}
+
+message LeaseStatus {
+  int64 ID = 1;
+  // TODO: int64 TTL = 2;
+}
+
+message LeaseLeasesResponse {
+  ResponseHeader header = 1;
+  repeated LeaseStatus leases = 2;
+}
+
+message Member {
+  // ID is the member ID for this member.
+  uint64 ID = 1;
+  // name is the human-readable name of the member. If the member is not started, the name will be an empty string.
+  string name = 2;
+  // peerURLs is the list of URLs the member exposes to the cluster for communication.
+  repeated string peerURLs = 3;
+  // clientURLs is the list of URLs the member exposes to clients for communication. If the member is not started, clientURLs will be empty.
+  repeated string clientURLs = 4;
+}
+
+message MemberAddRequest {
+  // peerURLs is the list of URLs the added member will use to communicate with the cluster.
+  repeated string peerURLs = 1;
+}
+
+message MemberAddResponse {
+  ResponseHeader header = 1;
+  // member is the member information for the added member.
+  Member member = 2;
+  // members is a list of all members after adding the new member.
+  repeated Member members = 3;
+}
+
+message MemberRemoveRequest {
+  // ID is the member ID of the member to remove.
+  uint64 ID = 1;
+}
+
+message MemberRemoveResponse {
+  ResponseHeader header = 1;
+  // members is a list of all members after removing the member.
+  repeated Member members = 2;
+}
+
+message MemberUpdateRequest {
+  // ID is the member ID of the member to update.
+  uint64 ID = 1;
+  // peerURLs is the new list of URLs the member will use to communicate with the cluster.
+  repeated string peerURLs = 2;
+}
+
+message MemberUpdateResponse{
+  ResponseHeader header = 1;
+  // members is a list of all members after updating the member.
+  repeated Member members = 2;
+}
+
+message MemberListRequest {
+}
+
+message MemberListResponse {
+  ResponseHeader header = 1;
+  // members is a list of all members associated with the cluster.
+  repeated Member members = 2;
+}
+
+message DefragmentRequest {
+}
+
+message DefragmentResponse {
+  ResponseHeader header = 1;
+}
+
+message MoveLeaderRequest {
+  // targetID is the node ID for the new leader.
+  uint64 targetID = 1;
+}
+
+message MoveLeaderResponse {
+  ResponseHeader header = 1;
+}
+
+enum AlarmType {
+	NONE = 0; // default, used to query if any alarm is active
+	NOSPACE = 1; // space quota is exhausted
+	CORRUPT = 2; // kv store corruption detected
+}
+
+message AlarmRequest {
+  enum AlarmAction {
+	GET = 0;
+	ACTIVATE = 1;
+	DEACTIVATE = 2;
+  }
+  // action is the kind of alarm request to issue. The action
+  // may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
+  // raised alarm.
+  AlarmAction action = 1;
+  // memberID is the ID of the member associated with the alarm. If memberID is 0, the
+  // alarm request covers all members.
+  uint64 memberID = 2;
+  // alarm is the type of alarm to consider for this request.
+  AlarmType alarm = 3;
+}
+
+message AlarmMember {
+  // memberID is the ID of the member associated with the raised alarm.
+  uint64 memberID = 1;
+  // alarm is the type of alarm which has been raised.
+  AlarmType alarm = 2;
+}
+
+message AlarmResponse {
+  ResponseHeader header = 1;
+  // alarms is a list of alarms associated with the alarm request.
+  repeated AlarmMember alarms = 2;
+}
+
+message StatusRequest {
+}
+
+message StatusResponse {
+  ResponseHeader header = 1;
+  // version is the cluster protocol version used by the responding member.
+  string version = 2;
+  // dbSize is the size of the backend database, in bytes, of the responding member.
+  int64 dbSize = 3;
+  // leader is the member ID which the responding member believes is the current leader.
+  uint64 leader = 4;
+  // raftIndex is the current raft index of the responding member.
+  uint64 raftIndex = 5;
+  // raftTerm is the current raft term of the responding member.
+  uint64 raftTerm = 6;
+}
+
+message AuthEnableRequest {
+}
+
+message AuthDisableRequest {
+}
+
+message AuthenticateRequest {
+  string name = 1;
+  string password = 2;
+}
+
+message AuthUserAddRequest {
+  string name = 1;
+  string password = 2;
+}
+
+message AuthUserGetRequest {
+  string name = 1;
+}
+
+message AuthUserDeleteRequest {
+  // name is the name of the user to delete.
+  string name = 1;
+}
+
+message AuthUserChangePasswordRequest {
+  // name is the name of the user whose password is being changed.
+  string name = 1;
+  // password is the new password for the user.
+  string password = 2;
+}
+
+message AuthUserGrantRoleRequest {
+  // user is the name of the user which should be granted a given role.
+  string user = 1;
+  // role is the name of the role to grant to the user.
+  string role = 2;
+}
+
+message AuthUserRevokeRoleRequest {
+  string name = 1;
+  string role = 2;
+}
+
+message AuthRoleAddRequest {
+  // name is the name of the role to add to the authentication system.
+  string name = 1;
+}
+
+message AuthRoleGetRequest {
+  string role = 1;
+}
+
+message AuthUserListRequest {
+}
+
+message AuthRoleListRequest {
+}
+
+message AuthRoleDeleteRequest {
+  string role = 1;
+}
+
+message AuthRoleGrantPermissionRequest {
+  // name is the name of the role which will be granted the permission.
+  string name = 1;
+  // perm is the permission to grant to the role.
+  authpb.Permission perm = 2;
+}
+
+message AuthRoleRevokePermissionRequest {
+  string role = 1;
+  string key = 2;
+  string range_end = 3;
+}
+
+message AuthEnableResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthDisableResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthenticateResponse {
+  ResponseHeader header = 1;
+  // token is an authorized token that can be used in succeeding RPCs
+  string token = 2;
+}
+
+message AuthUserAddResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthUserGetResponse {
+  ResponseHeader header = 1;
+
+  repeated string roles = 2;
+}
+
+message AuthUserDeleteResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthUserChangePasswordResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthUserGrantRoleResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthUserRevokeRoleResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthRoleAddResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthRoleGetResponse {
+  ResponseHeader header = 1;
+
+  repeated authpb.Permission perm = 2;
+}
+
+message AuthRoleListResponse {
+  ResponseHeader header = 1;
+
+  repeated string roles = 2;
+}
+
+message AuthUserListResponse {
+  ResponseHeader header = 1;
+
+  repeated string users = 2;
+}
+
+message AuthRoleDeleteResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthRoleGrantPermissionResponse {
+  ResponseHeader header = 1;
+}
+
+message AuthRoleRevokePermissionResponse {
+  ResponseHeader header = 1;
+}
diff --git a/vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.pb.go b/vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.pb.go
new file mode 100644
index 0000000..4679da5
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.pb.go
@@ -0,0 +1,830 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: kv.proto
+
+package mvccpb
+
+import (
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+	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.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Event_EventType int32
+
+const (
+	PUT    Event_EventType = 0
+	DELETE Event_EventType = 1
+)
+
+var Event_EventType_name = map[int32]string{
+	0: "PUT",
+	1: "DELETE",
+}
+
+var Event_EventType_value = map[string]int32{
+	"PUT":    0,
+	"DELETE": 1,
+}
+
+func (x Event_EventType) String() string {
+	return proto.EnumName(Event_EventType_name, int32(x))
+}
+
+func (Event_EventType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_2216fe83c9c12408, []int{1, 0}
+}
+
+type KeyValue struct {
+	// key is the key in bytes. An empty key is not allowed.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	// create_revision is the revision of last creation on this key.
+	CreateRevision int64 `protobuf:"varint,2,opt,name=create_revision,json=createRevision,proto3" json:"create_revision,omitempty"`
+	// mod_revision is the revision of last modification on this key.
+	ModRevision int64 `protobuf:"varint,3,opt,name=mod_revision,json=modRevision,proto3" json:"mod_revision,omitempty"`
+	// version is the version of the key. A deletion resets
+	// the version to zero and any modification of the key
+	// increases its version.
+	Version int64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"`
+	// value is the value held by the key, in bytes.
+	Value []byte `protobuf:"bytes,5,opt,name=value,proto3" json:"value,omitempty"`
+	// lease is the ID of the lease that attached to key.
+	// When the attached lease expires, the key will be deleted.
+	// If lease is 0, then no lease is attached to the key.
+	Lease                int64    `protobuf:"varint,6,opt,name=lease,proto3" json:"lease,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *KeyValue) Reset()         { *m = KeyValue{} }
+func (m *KeyValue) String() string { return proto.CompactTextString(m) }
+func (*KeyValue) ProtoMessage()    {}
+func (*KeyValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_2216fe83c9c12408, []int{0}
+}
+func (m *KeyValue) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *KeyValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_KeyValue.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *KeyValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_KeyValue.Merge(m, src)
+}
+func (m *KeyValue) XXX_Size() int {
+	return m.Size()
+}
+func (m *KeyValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_KeyValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_KeyValue proto.InternalMessageInfo
+
+type Event struct {
+	// type is the kind of event. If type is a PUT, it indicates
+	// new data has been stored to the key. If type is a DELETE,
+	// it indicates the key was deleted.
+	Type Event_EventType `protobuf:"varint,1,opt,name=type,proto3,enum=mvccpb.Event_EventType" json:"type,omitempty"`
+	// kv holds the KeyValue for the event.
+	// A PUT event contains current kv pair.
+	// A PUT event with kv.Version=1 indicates the creation of a key.
+	// A DELETE/EXPIRE event contains the deleted key with
+	// its modification revision set to the revision of deletion.
+	Kv *KeyValue `protobuf:"bytes,2,opt,name=kv,proto3" json:"kv,omitempty"`
+	// prev_kv holds the key-value pair before the event happens.
+	PrevKv               *KeyValue `protobuf:"bytes,3,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *Event) Reset()         { *m = Event{} }
+func (m *Event) String() string { return proto.CompactTextString(m) }
+func (*Event) ProtoMessage()    {}
+func (*Event) Descriptor() ([]byte, []int) {
+	return fileDescriptor_2216fe83c9c12408, []int{1}
+}
+func (m *Event) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Event.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Event) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Event.Merge(m, src)
+}
+func (m *Event) XXX_Size() int {
+	return m.Size()
+}
+func (m *Event) XXX_DiscardUnknown() {
+	xxx_messageInfo_Event.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Event proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterEnum("mvccpb.Event_EventType", Event_EventType_name, Event_EventType_value)
+	proto.RegisterType((*KeyValue)(nil), "mvccpb.KeyValue")
+	proto.RegisterType((*Event)(nil), "mvccpb.Event")
+}
+
+func init() { proto.RegisterFile("kv.proto", fileDescriptor_2216fe83c9c12408) }
+
+var fileDescriptor_2216fe83c9c12408 = []byte{
+	// 303 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0x41, 0x4e, 0xc2, 0x40,
+	0x14, 0x86, 0x3b, 0x14, 0x0a, 0x3e, 0x08, 0x36, 0x13, 0x12, 0x27, 0x2e, 0x26, 0x95, 0x8d, 0x18,
+	0x13, 0x4c, 0xf0, 0x06, 0xc6, 0xae, 0x70, 0x61, 0x1a, 0x74, 0x4b, 0x4a, 0x79, 0x21, 0xa4, 0x94,
+	0x69, 0x4a, 0x9d, 0xa4, 0x37, 0x71, 0xef, 0xde, 0x73, 0xb0, 0xe4, 0x08, 0x52, 0x2f, 0x62, 0xfa,
+	0xc6, 0xe2, 0xc6, 0xcd, 0xe4, 0xfd, 0xff, 0xff, 0x65, 0xe6, 0x7f, 0x03, 0x9d, 0x58, 0x8f, 0xd3,
+	0x4c, 0xe5, 0x8a, 0x3b, 0x89, 0x8e, 0xa2, 0x74, 0x71, 0x39, 0x58, 0xa9, 0x95, 0x22, 0xeb, 0xae,
+	0x9a, 0x4c, 0x3a, 0xfc, 0x64, 0xd0, 0x99, 0x62, 0xf1, 0x1a, 0x6e, 0xde, 0x90, 0xbb, 0x60, 0xc7,
+	0x58, 0x08, 0xe6, 0xb1, 0x51, 0x2f, 0xa8, 0x46, 0x7e, 0x0d, 0xe7, 0x51, 0x86, 0x61, 0x8e, 0xf3,
+	0x0c, 0xf5, 0x7a, 0xb7, 0x56, 0x5b, 0xd1, 0xf0, 0xd8, 0xc8, 0x0e, 0xfa, 0xc6, 0x0e, 0x7e, 0x5d,
+	0x7e, 0x05, 0xbd, 0x44, 0x2d, 0xff, 0x28, 0x9b, 0xa8, 0x6e, 0xa2, 0x96, 0x27, 0x44, 0x40, 0x5b,
+	0x63, 0x46, 0x69, 0x93, 0xd2, 0x5a, 0xf2, 0x01, 0xb4, 0x74, 0x55, 0x40, 0xb4, 0xe8, 0x65, 0x23,
+	0x2a, 0x77, 0x83, 0xe1, 0x0e, 0x85, 0x43, 0xb4, 0x11, 0xc3, 0x0f, 0x06, 0x2d, 0x5f, 0xe3, 0x36,
+	0xe7, 0xb7, 0xd0, 0xcc, 0x8b, 0x14, 0xa9, 0x6e, 0x7f, 0x72, 0x31, 0x36, 0x7b, 0x8e, 0x29, 0x34,
+	0xe7, 0xac, 0x48, 0x31, 0x20, 0x88, 0x7b, 0xd0, 0x88, 0x35, 0x75, 0xef, 0x4e, 0xdc, 0x1a, 0xad,
+	0x17, 0x0f, 0x1a, 0xb1, 0xe6, 0x37, 0xd0, 0x4e, 0x33, 0xd4, 0xf3, 0x58, 0x53, 0xf9, 0xff, 0x30,
+	0xa7, 0x02, 0xa6, 0x7a, 0xe8, 0xc1, 0xd9, 0xe9, 0x7e, 0xde, 0x06, 0xfb, 0xf9, 0x65, 0xe6, 0x5a,
+	0x1c, 0xc0, 0x79, 0xf4, 0x9f, 0xfc, 0x99, 0xef, 0xb2, 0x07, 0xb1, 0x3f, 0x4a, 0xeb, 0x70, 0x94,
+	0xd6, 0xbe, 0x94, 0xec, 0x50, 0x4a, 0xf6, 0x55, 0x4a, 0xf6, 0xfe, 0x2d, 0xad, 0x85, 0x43, 0xff,
+	0x7e, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x45, 0x92, 0x5d, 0xa1, 0x01, 0x00, 0x00,
+}
+
+func (m *KeyValue) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *KeyValue) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *KeyValue) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Lease != 0 {
+		i = encodeVarintKv(dAtA, i, uint64(m.Lease))
+		i--
+		dAtA[i] = 0x30
+	}
+	if len(m.Value) > 0 {
+		i -= len(m.Value)
+		copy(dAtA[i:], m.Value)
+		i = encodeVarintKv(dAtA, i, uint64(len(m.Value)))
+		i--
+		dAtA[i] = 0x2a
+	}
+	if m.Version != 0 {
+		i = encodeVarintKv(dAtA, i, uint64(m.Version))
+		i--
+		dAtA[i] = 0x20
+	}
+	if m.ModRevision != 0 {
+		i = encodeVarintKv(dAtA, i, uint64(m.ModRevision))
+		i--
+		dAtA[i] = 0x18
+	}
+	if m.CreateRevision != 0 {
+		i = encodeVarintKv(dAtA, i, uint64(m.CreateRevision))
+		i--
+		dAtA[i] = 0x10
+	}
+	if len(m.Key) > 0 {
+		i -= len(m.Key)
+		copy(dAtA[i:], m.Key)
+		i = encodeVarintKv(dAtA, i, uint64(len(m.Key)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Event) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Event) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Event) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.PrevKv != nil {
+		{
+			size, err := m.PrevKv.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintKv(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.Kv != nil {
+		{
+			size, err := m.Kv.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintKv(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x12
+	}
+	if m.Type != 0 {
+		i = encodeVarintKv(dAtA, i, uint64(m.Type))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintKv(dAtA []byte, offset int, v uint64) int {
+	offset -= sovKv(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *KeyValue) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovKv(uint64(l))
+	}
+	if m.CreateRevision != 0 {
+		n += 1 + sovKv(uint64(m.CreateRevision))
+	}
+	if m.ModRevision != 0 {
+		n += 1 + sovKv(uint64(m.ModRevision))
+	}
+	if m.Version != 0 {
+		n += 1 + sovKv(uint64(m.Version))
+	}
+	l = len(m.Value)
+	if l > 0 {
+		n += 1 + l + sovKv(uint64(l))
+	}
+	if m.Lease != 0 {
+		n += 1 + sovKv(uint64(m.Lease))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Event) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Type != 0 {
+		n += 1 + sovKv(uint64(m.Type))
+	}
+	if m.Kv != nil {
+		l = m.Kv.Size()
+		n += 1 + l + sovKv(uint64(l))
+	}
+	if m.PrevKv != nil {
+		l = m.PrevKv.Size()
+		n += 1 + l + sovKv(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovKv(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozKv(x uint64) (n int) {
+	return sovKv(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *KeyValue) 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 ErrIntOverflowKv
+			}
+			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: KeyValue: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: KeyValue: 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 ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthKv
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthKv
+			}
+			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 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType)
+			}
+			m.CreateRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.CreateRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType)
+			}
+			m.ModRevision = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ModRevision |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
+			}
+			m.Version = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Version |= int64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 5:
+			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 ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthKv
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthKv
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...)
+			if m.Value == nil {
+				m.Value = []byte{}
+			}
+			iNdEx = postIndex
+		case 6:
+			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 ErrIntOverflowKv
+				}
+				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 := skipKv(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthKv
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthKv
+			}
+			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 *Event) 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 ErrIntOverflowKv
+			}
+			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: Event: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Event: 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 ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Type |= Event_EventType(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		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 ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthKv
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthKv
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Kv == nil {
+				m.Kv = &KeyValue{}
+			}
+			if err := m.Kv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthKv
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthKv
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.PrevKv == nil {
+				m.PrevKv = &KeyValue{}
+			}
+			if err := m.PrevKv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipKv(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthKv
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthKv
+			}
+			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 skipKv(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, ErrIntOverflowKv
+			}
+			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, ErrIntOverflowKv
+				}
+				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, ErrIntOverflowKv
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthKv
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthKv
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowKv
+					}
+					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 := skipKv(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthKv
+				}
+			}
+			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 (
+	ErrInvalidLengthKv = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowKv   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/go.etcd.io/etcd/mvcc/mvccpb/kv.proto b/vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.proto
similarity index 100%
rename from vendor/go.etcd.io/etcd/mvcc/mvccpb/kv.proto
rename to vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.proto
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/discard_logger.go b/vendor/github.com/coreos/etcd/pkg/logutil/discard_logger.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/discard_logger.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/discard_logger.go
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/doc.go b/vendor/github.com/coreos/etcd/pkg/logutil/doc.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/doc.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/doc.go
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/log_level.go b/vendor/github.com/coreos/etcd/pkg/logutil/log_level.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/log_level.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/log_level.go
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/logger.go b/vendor/github.com/coreos/etcd/pkg/logutil/logger.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/logger.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/logger.go
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/merge_logger.go b/vendor/github.com/coreos/etcd/pkg/logutil/merge_logger.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/merge_logger.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/merge_logger.go
diff --git a/vendor/github.com/coreos/etcd/pkg/logutil/package_logger.go b/vendor/github.com/coreos/etcd/pkg/logutil/package_logger.go
new file mode 100644
index 0000000..378bee0
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/logutil/package_logger.go
@@ -0,0 +1,60 @@
+// 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 logutil
+
+import (
+	"github.com/coreos/pkg/capnslog"
+	"google.golang.org/grpc/grpclog"
+)
+
+// assert that "packageLogger" satisfy "Logger" interface
+var _ Logger = &packageLogger{}
+
+// NewPackageLogger wraps "*capnslog.PackageLogger" that implements "Logger" interface.
+//
+// For example:
+//
+//  var defaultLogger Logger
+//  defaultLogger = NewPackageLogger("github.com/coreos/etcd", "snapshot")
+//
+func NewPackageLogger(repo, pkg string) Logger {
+	return &packageLogger{p: capnslog.NewPackageLogger(repo, pkg)}
+}
+
+type packageLogger struct {
+	p *capnslog.PackageLogger
+}
+
+func (l *packageLogger) Info(args ...interface{})                    { l.p.Info(args...) }
+func (l *packageLogger) Infoln(args ...interface{})                  { l.p.Info(args...) }
+func (l *packageLogger) Infof(format string, args ...interface{})    { l.p.Infof(format, args...) }
+func (l *packageLogger) Warning(args ...interface{})                 { l.p.Warning(args...) }
+func (l *packageLogger) Warningln(args ...interface{})               { l.p.Warning(args...) }
+func (l *packageLogger) Warningf(format string, args ...interface{}) { l.p.Warningf(format, args...) }
+func (l *packageLogger) Error(args ...interface{})                   { l.p.Error(args...) }
+func (l *packageLogger) Errorln(args ...interface{})                 { l.p.Error(args...) }
+func (l *packageLogger) Errorf(format string, args ...interface{})   { l.p.Errorf(format, args...) }
+func (l *packageLogger) Fatal(args ...interface{})                   { l.p.Fatal(args...) }
+func (l *packageLogger) Fatalln(args ...interface{})                 { l.p.Fatal(args...) }
+func (l *packageLogger) Fatalf(format string, args ...interface{})   { l.p.Fatalf(format, args...) }
+func (l *packageLogger) V(lvl int) bool {
+	return l.p.LevelAt(capnslog.LogLevel(lvl))
+}
+func (l *packageLogger) Lvl(lvl int) grpclog.LoggerV2 {
+	if l.p.LevelAt(capnslog.LogLevel(lvl)) {
+		return l
+	}
+	return &discardLogger{}
+}
diff --git a/vendor/github.com/coreos/etcd/pkg/logutil/zap.go b/vendor/github.com/coreos/etcd/pkg/logutil/zap.go
new file mode 100644
index 0000000..2f69223
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/logutil/zap.go
@@ -0,0 +1,97 @@
+// 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 logutil
+
+import (
+	"sort"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+)
+
+// DefaultZapLoggerConfig defines default zap logger configuration.
+var DefaultZapLoggerConfig = zap.Config{
+	Level: zap.NewAtomicLevelAt(ConvertToZapLevel(DefaultLogLevel)),
+
+	Development: false,
+	Sampling: &zap.SamplingConfig{
+		Initial:    100,
+		Thereafter: 100,
+	},
+
+	Encoding: "json",
+
+	// copied from "zap.NewProductionEncoderConfig" with some updates
+	EncoderConfig: zapcore.EncoderConfig{
+		TimeKey:        "ts",
+		LevelKey:       "level",
+		NameKey:        "logger",
+		CallerKey:      "caller",
+		MessageKey:     "msg",
+		StacktraceKey:  "stacktrace",
+		LineEnding:     zapcore.DefaultLineEnding,
+		EncodeLevel:    zapcore.LowercaseLevelEncoder,
+		EncodeTime:     zapcore.ISO8601TimeEncoder,
+		EncodeDuration: zapcore.StringDurationEncoder,
+		EncodeCaller:   zapcore.ShortCallerEncoder,
+	},
+
+	// Use "/dev/null" to discard all
+	OutputPaths:      []string{"stderr"},
+	ErrorOutputPaths: []string{"stderr"},
+}
+
+// AddOutputPaths adds output paths to the existing output paths, resolving conflicts.
+func AddOutputPaths(cfg zap.Config, outputPaths, errorOutputPaths []string) zap.Config {
+	outputs := make(map[string]struct{})
+	for _, v := range cfg.OutputPaths {
+		outputs[v] = struct{}{}
+	}
+	for _, v := range outputPaths {
+		outputs[v] = struct{}{}
+	}
+	outputSlice := make([]string, 0)
+	if _, ok := outputs["/dev/null"]; ok {
+		// "/dev/null" to discard all
+		outputSlice = []string{"/dev/null"}
+	} else {
+		for k := range outputs {
+			outputSlice = append(outputSlice, k)
+		}
+	}
+	cfg.OutputPaths = outputSlice
+	sort.Strings(cfg.OutputPaths)
+
+	errOutputs := make(map[string]struct{})
+	for _, v := range cfg.ErrorOutputPaths {
+		errOutputs[v] = struct{}{}
+	}
+	for _, v := range errorOutputPaths {
+		errOutputs[v] = struct{}{}
+	}
+	errOutputSlice := make([]string, 0)
+	if _, ok := errOutputs["/dev/null"]; ok {
+		// "/dev/null" to discard all
+		errOutputSlice = []string{"/dev/null"}
+	} else {
+		for k := range errOutputs {
+			errOutputSlice = append(errOutputSlice, k)
+		}
+	}
+	cfg.ErrorOutputPaths = errOutputSlice
+	sort.Strings(cfg.ErrorOutputPaths)
+
+	return cfg
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/zap_grpc.go b/vendor/github.com/coreos/etcd/pkg/logutil/zap_grpc.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/logutil/zap_grpc.go
rename to vendor/github.com/coreos/etcd/pkg/logutil/zap_grpc.go
diff --git a/vendor/github.com/coreos/etcd/pkg/logutil/zap_journal.go b/vendor/github.com/coreos/etcd/pkg/logutil/zap_journal.go
new file mode 100644
index 0000000..b1788bc
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/logutil/zap_journal.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.
+
+// +build !windows
+
+package logutil
+
+import (
+	"bytes"
+	"encoding/json"
+	"fmt"
+	"io"
+	"os"
+	"path/filepath"
+
+	"github.com/coreos/etcd/pkg/systemd"
+
+	"github.com/coreos/go-systemd/journal"
+	"go.uber.org/zap/zapcore"
+)
+
+// NewJournalWriter wraps "io.Writer" to redirect log output
+// to the local systemd journal. If journald send fails, it fails
+// back to writing to the original writer.
+// The decode overhead is only <30µs per write.
+// Reference: https://github.com/coreos/pkg/blob/master/capnslog/journald_formatter.go
+func NewJournalWriter(wr io.Writer) (io.Writer, error) {
+	return &journalWriter{Writer: wr}, systemd.DialJournal()
+}
+
+type journalWriter struct {
+	io.Writer
+}
+
+// WARN: assume that etcd uses default field names in zap encoder config
+// make sure to keep this up-to-date!
+type logLine struct {
+	Level  string `json:"level"`
+	Caller string `json:"caller"`
+}
+
+func (w *journalWriter) Write(p []byte) (int, error) {
+	line := &logLine{}
+	if err := json.NewDecoder(bytes.NewReader(p)).Decode(line); err != nil {
+		return 0, err
+	}
+
+	var pri journal.Priority
+	switch line.Level {
+	case zapcore.DebugLevel.String():
+		pri = journal.PriDebug
+	case zapcore.InfoLevel.String():
+		pri = journal.PriInfo
+
+	case zapcore.WarnLevel.String():
+		pri = journal.PriWarning
+	case zapcore.ErrorLevel.String():
+		pri = journal.PriErr
+
+	case zapcore.DPanicLevel.String():
+		pri = journal.PriCrit
+	case zapcore.PanicLevel.String():
+		pri = journal.PriCrit
+	case zapcore.FatalLevel.String():
+		pri = journal.PriCrit
+
+	default:
+		panic(fmt.Errorf("unknown log level: %q", line.Level))
+	}
+
+	err := journal.Send(string(p), pri, map[string]string{
+		"PACKAGE":           filepath.Dir(line.Caller),
+		"SYSLOG_IDENTIFIER": filepath.Base(os.Args[0]),
+	})
+	if err != nil {
+		// "journal" also falls back to stderr
+		// "fmt.Fprintln(os.Stderr, s)"
+		return w.Writer.Write(p)
+	}
+	return 0, nil
+}
diff --git a/vendor/github.com/coreos/etcd/pkg/logutil/zap_raft.go b/vendor/github.com/coreos/etcd/pkg/logutil/zap_raft.go
new file mode 100644
index 0000000..012d688
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/logutil/zap_raft.go
@@ -0,0 +1,102 @@
+// 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 logutil
+
+import (
+	"errors"
+
+	"github.com/coreos/etcd/raft"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+)
+
+// NewRaftLogger builds "raft.Logger" from "*zap.Config".
+func NewRaftLogger(lcfg *zap.Config) (raft.Logger, error) {
+	if lcfg == nil {
+		return nil, errors.New("nil zap.Config")
+	}
+	lg, err := lcfg.Build(zap.AddCallerSkip(1)) // to annotate caller outside of "logutil"
+	if err != nil {
+		return nil, err
+	}
+	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}, nil
+}
+
+// NewRaftLoggerZap converts "*zap.Logger" to "raft.Logger".
+func NewRaftLoggerZap(lg *zap.Logger) raft.Logger {
+	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}
+}
+
+// NewRaftLoggerFromZapCore creates "raft.Logger" from "zap.Core"
+// and "zapcore.WriteSyncer".
+func NewRaftLoggerFromZapCore(cr zapcore.Core, syncer zapcore.WriteSyncer) raft.Logger {
+	// "AddCallerSkip" to annotate caller outside of "logutil"
+	lg := zap.New(cr, zap.AddCaller(), zap.AddCallerSkip(1), zap.ErrorOutput(syncer))
+	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}
+}
+
+type zapRaftLogger struct {
+	lg    *zap.Logger
+	sugar *zap.SugaredLogger
+}
+
+func (zl *zapRaftLogger) Debug(args ...interface{}) {
+	zl.sugar.Debug(args...)
+}
+
+func (zl *zapRaftLogger) Debugf(format string, args ...interface{}) {
+	zl.sugar.Debugf(format, args...)
+}
+
+func (zl *zapRaftLogger) Error(args ...interface{}) {
+	zl.sugar.Error(args...)
+}
+
+func (zl *zapRaftLogger) Errorf(format string, args ...interface{}) {
+	zl.sugar.Errorf(format, args...)
+}
+
+func (zl *zapRaftLogger) Info(args ...interface{}) {
+	zl.sugar.Info(args...)
+}
+
+func (zl *zapRaftLogger) Infof(format string, args ...interface{}) {
+	zl.sugar.Infof(format, args...)
+}
+
+func (zl *zapRaftLogger) Warning(args ...interface{}) {
+	zl.sugar.Warn(args...)
+}
+
+func (zl *zapRaftLogger) Warningf(format string, args ...interface{}) {
+	zl.sugar.Warnf(format, args...)
+}
+
+func (zl *zapRaftLogger) Fatal(args ...interface{}) {
+	zl.sugar.Fatal(args...)
+}
+
+func (zl *zapRaftLogger) Fatalf(format string, args ...interface{}) {
+	zl.sugar.Fatalf(format, args...)
+}
+
+func (zl *zapRaftLogger) Panic(args ...interface{}) {
+	zl.sugar.Panic(args...)
+}
+
+func (zl *zapRaftLogger) Panicf(format string, args ...interface{}) {
+	zl.sugar.Panicf(format, args...)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/systemd/doc.go b/vendor/github.com/coreos/etcd/pkg/systemd/doc.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/systemd/doc.go
rename to vendor/github.com/coreos/etcd/pkg/systemd/doc.go
diff --git a/vendor/go.etcd.io/etcd/pkg/systemd/journal.go b/vendor/github.com/coreos/etcd/pkg/systemd/journal.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/systemd/journal.go
rename to vendor/github.com/coreos/etcd/pkg/systemd/journal.go
diff --git a/vendor/go.etcd.io/etcd/pkg/types/doc.go b/vendor/github.com/coreos/etcd/pkg/types/doc.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/types/doc.go
rename to vendor/github.com/coreos/etcd/pkg/types/doc.go
diff --git a/vendor/github.com/coreos/etcd/pkg/types/id.go b/vendor/github.com/coreos/etcd/pkg/types/id.go
new file mode 100644
index 0000000..1b042d9
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/types/id.go
@@ -0,0 +1,41 @@
+// 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 types
+
+import (
+	"strconv"
+)
+
+// ID represents a generic identifier which is canonically
+// stored as a uint64 but is typically represented as a
+// base-16 string for input/output
+type ID uint64
+
+func (i ID) String() string {
+	return strconv.FormatUint(uint64(i), 16)
+}
+
+// IDFromString attempts to create an ID from a base-16 string.
+func IDFromString(s string) (ID, error) {
+	i, err := strconv.ParseUint(s, 16, 64)
+	return ID(i), err
+}
+
+// IDSlice implements the sort interface
+type IDSlice []ID
+
+func (p IDSlice) Len() int           { return len(p) }
+func (p IDSlice) Less(i, j int) bool { return uint64(p[i]) < uint64(p[j]) }
+func (p IDSlice) Swap(i, j int)      { p[i], p[j] = p[j], p[i] }
diff --git a/vendor/github.com/coreos/etcd/pkg/types/set.go b/vendor/github.com/coreos/etcd/pkg/types/set.go
new file mode 100644
index 0000000..c111b0c
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/pkg/types/set.go
@@ -0,0 +1,178 @@
+// 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 types
+
+import (
+	"reflect"
+	"sort"
+	"sync"
+)
+
+type Set interface {
+	Add(string)
+	Remove(string)
+	Contains(string) bool
+	Equals(Set) bool
+	Length() int
+	Values() []string
+	Copy() Set
+	Sub(Set) Set
+}
+
+func NewUnsafeSet(values ...string) *unsafeSet {
+	set := &unsafeSet{make(map[string]struct{})}
+	for _, v := range values {
+		set.Add(v)
+	}
+	return set
+}
+
+func NewThreadsafeSet(values ...string) *tsafeSet {
+	us := NewUnsafeSet(values...)
+	return &tsafeSet{us, sync.RWMutex{}}
+}
+
+type unsafeSet struct {
+	d map[string]struct{}
+}
+
+// Add adds a new value to the set (no-op if the value is already present)
+func (us *unsafeSet) Add(value string) {
+	us.d[value] = struct{}{}
+}
+
+// Remove removes the given value from the set
+func (us *unsafeSet) Remove(value string) {
+	delete(us.d, value)
+}
+
+// Contains returns whether the set contains the given value
+func (us *unsafeSet) Contains(value string) (exists bool) {
+	_, exists = us.d[value]
+	return exists
+}
+
+// ContainsAll returns whether the set contains all given values
+func (us *unsafeSet) ContainsAll(values []string) bool {
+	for _, s := range values {
+		if !us.Contains(s) {
+			return false
+		}
+	}
+	return true
+}
+
+// Equals returns whether the contents of two sets are identical
+func (us *unsafeSet) Equals(other Set) bool {
+	v1 := sort.StringSlice(us.Values())
+	v2 := sort.StringSlice(other.Values())
+	v1.Sort()
+	v2.Sort()
+	return reflect.DeepEqual(v1, v2)
+}
+
+// Length returns the number of elements in the set
+func (us *unsafeSet) Length() int {
+	return len(us.d)
+}
+
+// Values returns the values of the Set in an unspecified order.
+func (us *unsafeSet) Values() (values []string) {
+	values = make([]string, 0)
+	for val := range us.d {
+		values = append(values, val)
+	}
+	return values
+}
+
+// Copy creates a new Set containing the values of the first
+func (us *unsafeSet) Copy() Set {
+	cp := NewUnsafeSet()
+	for val := range us.d {
+		cp.Add(val)
+	}
+
+	return cp
+}
+
+// Sub removes all elements in other from the set
+func (us *unsafeSet) Sub(other Set) Set {
+	oValues := other.Values()
+	result := us.Copy().(*unsafeSet)
+
+	for _, val := range oValues {
+		if _, ok := result.d[val]; !ok {
+			continue
+		}
+		delete(result.d, val)
+	}
+
+	return result
+}
+
+type tsafeSet struct {
+	us *unsafeSet
+	m  sync.RWMutex
+}
+
+func (ts *tsafeSet) Add(value string) {
+	ts.m.Lock()
+	defer ts.m.Unlock()
+	ts.us.Add(value)
+}
+
+func (ts *tsafeSet) Remove(value string) {
+	ts.m.Lock()
+	defer ts.m.Unlock()
+	ts.us.Remove(value)
+}
+
+func (ts *tsafeSet) Contains(value string) (exists bool) {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	return ts.us.Contains(value)
+}
+
+func (ts *tsafeSet) Equals(other Set) bool {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	return ts.us.Equals(other)
+}
+
+func (ts *tsafeSet) Length() int {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	return ts.us.Length()
+}
+
+func (ts *tsafeSet) Values() (values []string) {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	return ts.us.Values()
+}
+
+func (ts *tsafeSet) Copy() Set {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	usResult := ts.us.Copy().(*unsafeSet)
+	return &tsafeSet{usResult, sync.RWMutex{}}
+}
+
+func (ts *tsafeSet) Sub(other Set) Set {
+	ts.m.RLock()
+	defer ts.m.RUnlock()
+	usResult := ts.us.Sub(other).(*unsafeSet)
+	return &tsafeSet{usResult, sync.RWMutex{}}
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/types/slice.go b/vendor/github.com/coreos/etcd/pkg/types/slice.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/types/slice.go
rename to vendor/github.com/coreos/etcd/pkg/types/slice.go
diff --git a/vendor/go.etcd.io/etcd/pkg/types/urls.go b/vendor/github.com/coreos/etcd/pkg/types/urls.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/types/urls.go
rename to vendor/github.com/coreos/etcd/pkg/types/urls.go
diff --git a/vendor/go.etcd.io/etcd/pkg/types/urlsmap.go b/vendor/github.com/coreos/etcd/pkg/types/urlsmap.go
similarity index 100%
rename from vendor/go.etcd.io/etcd/pkg/types/urlsmap.go
rename to vendor/github.com/coreos/etcd/pkg/types/urlsmap.go
diff --git a/vendor/github.com/coreos/etcd/raft/README.md b/vendor/github.com/coreos/etcd/raft/README.md
new file mode 100644
index 0000000..fde22b1
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/README.md
@@ -0,0 +1,196 @@
+# Raft library
+
+Raft is a protocol with which a cluster of nodes can maintain a replicated state machine.
+The state machine is kept in sync through the use of a replicated log.
+For more details on Raft, see "In Search of an Understandable Consensus Algorithm"
+(https://ramcloud.stanford.edu/raft.pdf) by Diego Ongaro and John Ousterhout.
+
+This Raft library is stable and feature complete. As of 2016, it is **the most widely used** Raft library in production, serving tens of thousands clusters each day. It powers distributed systems such as etcd, Kubernetes, Docker Swarm, Cloud Foundry Diego, CockroachDB, TiDB, Project Calico, Flannel, and more.
+
+Most Raft implementations have a monolithic design, including storage handling, messaging serialization, and network transport. This library instead follows a minimalistic design philosophy by only implementing the core raft algorithm. This minimalism buys flexibility, determinism, and performance.
+
+To keep the codebase small as well as provide flexibility, the library only implements the Raft algorithm; both network and disk IO are left to the user. Library users must implement their own transportation layer for message passing between Raft peers over the wire. Similarly, users must implement their own storage layer to persist the Raft log and state.
+
+In order to easily test the Raft library, its behavior should be deterministic. To achieve this determinism, the library models Raft as a state machine.  The state machine takes a `Message` as input. A message can either be a local timer update or a network message sent from a remote peer. The state machine's output is a 3-tuple `{[]Messages, []LogEntries, NextState}` consisting of an array of `Messages`, `log entries`, and `Raft state changes`. For state machines with the same state, the same state machine input should always generate the same state machine output.
+
+A simple example application, _raftexample_, is also available to help illustrate how to use this package in practice: https://github.com/coreos/etcd/tree/master/contrib/raftexample
+
+# Features
+
+This raft implementation is a full feature implementation of Raft protocol. Features includes:
+
+- Leader election
+- Log replication
+- Log compaction 
+- Membership changes
+- Leadership transfer extension
+- Efficient linearizable read-only queries served by both the leader and followers
+  - leader checks with quorum and bypasses Raft log before processing read-only queries
+  - followers asks leader to get a safe read index before processing read-only queries
+- More efficient lease-based linearizable read-only queries served by both the leader and followers
+  - leader bypasses Raft log and processing read-only queries locally
+  - followers asks leader to get a safe read index before processing read-only queries
+  - this approach relies on the clock of the all the machines in raft group
+
+This raft implementation also includes a few optional enhancements:
+
+- Optimistic pipelining to reduce log replication latency
+- Flow control for log replication
+- Batching Raft messages to reduce synchronized network I/O calls
+- Batching log entries to reduce disk synchronized I/O
+- Writing to leader's disk in parallel
+- Internal proposal redirection from followers to leader
+- Automatic stepping down when the leader loses quorum 
+
+## Notable Users
+
+- [cockroachdb](https://github.com/cockroachdb/cockroach) A Scalable, Survivable, Strongly-Consistent SQL Database
+- [dgraph](https://github.com/dgraph-io/dgraph) A Scalable, Distributed, Low Latency, High Throughput Graph Database
+- [etcd](https://github.com/coreos/etcd) A distributed reliable key-value store
+- [tikv](https://github.com/pingcap/tikv) A Distributed transactional key value database powered by Rust and Raft
+- [swarmkit](https://github.com/docker/swarmkit) A toolkit for orchestrating distributed systems at any scale.
+- [chain core](https://github.com/chain/chain) Software for operating permissioned, multi-asset blockchain networks
+
+## Usage
+
+The primary object in raft is a Node. Either start a Node from scratch using raft.StartNode or start a Node from some initial state using raft.RestartNode.
+
+To start a three-node cluster
+```go
+  storage := raft.NewMemoryStorage()
+  c := &Config{
+    ID:              0x01,
+    ElectionTick:    10,
+    HeartbeatTick:   1,
+    Storage:         storage,
+    MaxSizePerMsg:   4096,
+    MaxInflightMsgs: 256,
+  }
+  // Set peer list to the other nodes in the cluster.
+  // Note that they need to be started separately as well.
+  n := raft.StartNode(c, []raft.Peer{{ID: 0x02}, {ID: 0x03}})
+```
+
+Start a single node cluster, like so:
+```go
+  // Create storage and config as shown above.
+  // Set peer list to itself, so this node can become the leader of this single-node cluster.
+  peers := []raft.Peer{{ID: 0x01}}
+  n := raft.StartNode(c, peers)
+```
+
+To allow a new node to join this cluster, do not pass in any peers. First, add the node to the existing cluster by calling `ProposeConfChange` on any existing node inside the cluster. Then, start the node with an empty peer list, like so:
+```go
+  // Create storage and config as shown above.
+  n := raft.StartNode(c, nil)
+```
+
+To restart a node from previous state:
+```go
+  storage := raft.NewMemoryStorage()
+
+  // Recover the in-memory storage from persistent snapshot, state and entries.
+  storage.ApplySnapshot(snapshot)
+  storage.SetHardState(state)
+  storage.Append(entries)
+
+  c := &Config{
+    ID:              0x01,
+    ElectionTick:    10,
+    HeartbeatTick:   1,
+    Storage:         storage,
+    MaxSizePerMsg:   4096,
+    MaxInflightMsgs: 256,
+  }
+
+  // Restart raft without peer information.
+  // Peer information is already included in the storage.
+  n := raft.RestartNode(c)
+```
+
+After creating a Node, the user has a few responsibilities:
+
+First, read from the Node.Ready() channel and process the updates it contains. These steps may be performed in parallel, except as noted in step 2.
+
+1. Write Entries, HardState and Snapshot to persistent storage in order, i.e. Entries first, then HardState and Snapshot if they are not empty. If persistent storage supports atomic writes then all of them can be written together. Note that when writing an Entry with Index i, any previously-persisted entries with Index >= i must be discarded.
+
+2. Send all Messages to the nodes named in the To field. It is important that no messages be sent until the latest HardState has been persisted to disk, and all Entries written by any previous Ready batch (Messages may be sent while entries from the same batch are being persisted). To reduce the I/O latency, an optimization can be applied to make leader write to disk in parallel with its followers (as explained at section 10.2.1 in Raft thesis). If any Message has type MsgSnap, call Node.ReportSnapshot() after it has been sent (these messages may be large). Note: Marshalling messages is not thread-safe; it is important to make sure that no new entries are persisted while marshalling. The easiest way to achieve this is to serialise the messages directly inside the main raft loop.
+
+3. Apply Snapshot (if any) and CommittedEntries to the state machine. If any committed Entry has Type EntryConfChange, call Node.ApplyConfChange() to apply it to the node. The configuration change may be cancelled at this point by setting the NodeID field to zero before calling ApplyConfChange (but ApplyConfChange must be called one way or the other, and the decision to cancel must be based solely on the state machine and not external information such as the observed health of the node).
+
+4. Call Node.Advance() to signal readiness for the next batch of updates. This may be done at any time after step 1, although all updates must be processed in the order they were returned by Ready.
+
+Second, all persisted log entries must be made available via an implementation of the Storage interface. The provided MemoryStorage type can be used for this (if repopulating its state upon a restart), or a custom disk-backed implementation can be supplied.
+
+Third, after receiving a message from another node, pass it to Node.Step:
+
+```go
+	func recvRaftRPC(ctx context.Context, m raftpb.Message) {
+		n.Step(ctx, m)
+	}
+```
+
+Finally, call `Node.Tick()` at regular intervals (probably via a `time.Ticker`). Raft has two important timeouts: heartbeat and the election timeout. However, internally to the raft package time is represented by an abstract "tick".
+
+The total state machine handling loop will look something like this:
+
+```go
+  for {
+    select {
+    case <-s.Ticker:
+      n.Tick()
+    case rd := <-s.Node.Ready():
+      saveToStorage(rd.State, rd.Entries, rd.Snapshot)
+      send(rd.Messages)
+      if !raft.IsEmptySnap(rd.Snapshot) {
+        processSnapshot(rd.Snapshot)
+      }
+      for _, entry := range rd.CommittedEntries {
+        process(entry)
+        if entry.Type == raftpb.EntryConfChange {
+          var cc raftpb.ConfChange
+          cc.Unmarshal(entry.Data)
+          s.Node.ApplyConfChange(cc)
+        }
+      }
+      s.Node.Advance()
+    case <-s.done:
+      return
+    }
+  }
+```
+
+To propose changes to the state machine from the node to take application data, serialize it into a byte slice and call:
+
+```go
+	n.Propose(ctx, data)
+```
+
+If the proposal is committed, data will appear in committed entries with type raftpb.EntryNormal. There is no guarantee that a proposed command will be committed; the command may have to be reproposed after a timeout. 
+
+To add or remove node in a cluster, build ConfChange struct 'cc' and call:
+
+```go
+	n.ProposeConfChange(ctx, cc)
+```
+
+After config change is committed, some committed entry with type raftpb.EntryConfChange will be returned. This must be applied to node through:
+
+```go
+	var cc raftpb.ConfChange
+	cc.Unmarshal(data)
+	n.ApplyConfChange(cc)
+```
+
+Note: An ID represents a unique node in a cluster for all time. A
+given ID MUST be used only once even if the old node has been removed.
+This means that for example IP addresses make poor node IDs since they
+may be reused. Node IDs must be non-zero.
+
+## Implementation notes
+
+This implementation is up to date with the final Raft thesis (https://ramcloud.stanford.edu/~ongaro/thesis.pdf), although this implementation of the membership change protocol differs somewhat from that described in chapter 4. The key invariant that membership changes happen one node at a time is preserved, but in our implementation the membership change takes effect when its entry is applied, not when it is added to the log (so the entry is committed under the old membership instead of the new). This is equivalent in terms of safety, since the old and new configurations are guaranteed to overlap.
+
+To ensure there is no attempt to commit two membership changes at once by matching log positions (which would be unsafe since they should have different quorum requirements), any proposed membership change is simply disallowed while any uncommitted change appears in the leader's log.
+
+This approach introduces a problem when removing a member from a two-member cluster: If one of the members dies before the other one receives the commit of the confchange entry, then the member cannot be removed any more since the cluster cannot make progress. For this reason it is highly recommended to use three or more nodes in every cluster.
diff --git a/vendor/go.etcd.io/etcd/raft/design.md b/vendor/github.com/coreos/etcd/raft/design.md
similarity index 100%
rename from vendor/go.etcd.io/etcd/raft/design.md
rename to vendor/github.com/coreos/etcd/raft/design.md
diff --git a/vendor/github.com/coreos/etcd/raft/doc.go b/vendor/github.com/coreos/etcd/raft/doc.go
new file mode 100644
index 0000000..b55c591
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/doc.go
@@ -0,0 +1,300 @@
+// 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 raft sends and receives messages in the Protocol Buffer format
+defined in the raftpb package.
+
+Raft is a protocol with which a cluster of nodes can maintain a replicated state machine.
+The state machine is kept in sync through the use of a replicated log.
+For more details on Raft, see "In Search of an Understandable Consensus Algorithm"
+(https://ramcloud.stanford.edu/raft.pdf) by Diego Ongaro and John Ousterhout.
+
+A simple example application, _raftexample_, is also available to help illustrate
+how to use this package in practice:
+https://github.com/coreos/etcd/tree/master/contrib/raftexample
+
+Usage
+
+The primary object in raft is a Node. You either start a Node from scratch
+using raft.StartNode or start a Node from some initial state using raft.RestartNode.
+
+To start a node from scratch:
+
+  storage := raft.NewMemoryStorage()
+  c := &Config{
+    ID:              0x01,
+    ElectionTick:    10,
+    HeartbeatTick:   1,
+    Storage:         storage,
+    MaxSizePerMsg:   4096,
+    MaxInflightMsgs: 256,
+  }
+  n := raft.StartNode(c, []raft.Peer{{ID: 0x02}, {ID: 0x03}})
+
+To restart a node from previous state:
+
+  storage := raft.NewMemoryStorage()
+
+  // recover the in-memory storage from persistent
+  // snapshot, state and entries.
+  storage.ApplySnapshot(snapshot)
+  storage.SetHardState(state)
+  storage.Append(entries)
+
+  c := &Config{
+    ID:              0x01,
+    ElectionTick:    10,
+    HeartbeatTick:   1,
+    Storage:         storage,
+    MaxSizePerMsg:   4096,
+    MaxInflightMsgs: 256,
+  }
+
+  // restart raft without peer information.
+  // peer information is already included in the storage.
+  n := raft.RestartNode(c)
+
+Now that you are holding onto a Node you have a few responsibilities:
+
+First, you must read from the Node.Ready() channel and process the updates
+it contains. These steps may be performed in parallel, except as noted in step
+2.
+
+1. Write HardState, Entries, and Snapshot to persistent storage if they are
+not empty. Note that when writing an Entry with Index i, any
+previously-persisted entries with Index >= i must be discarded.
+
+2. Send all Messages to the nodes named in the To field. It is important that
+no messages be sent until the latest HardState has been persisted to disk,
+and all Entries written by any previous Ready batch (Messages may be sent while
+entries from the same batch are being persisted). To reduce the I/O latency, an
+optimization can be applied to make leader write to disk in parallel with its
+followers (as explained at section 10.2.1 in Raft thesis). If any Message has type
+MsgSnap, call Node.ReportSnapshot() after it has been sent (these messages may be
+large).
+
+Note: Marshalling messages is not thread-safe; it is important that you
+make sure that no new entries are persisted while marshalling.
+The easiest way to achieve this is to serialise the messages directly inside
+your main raft loop.
+
+3. Apply Snapshot (if any) and CommittedEntries to the state machine.
+If any committed Entry has Type EntryConfChange, call Node.ApplyConfChange()
+to apply it to the node. The configuration change may be cancelled at this point
+by setting the NodeID field to zero before calling ApplyConfChange
+(but ApplyConfChange must be called one way or the other, and the decision to cancel
+must be based solely on the state machine and not external information such as
+the observed health of the node).
+
+4. Call Node.Advance() to signal readiness for the next batch of updates.
+This may be done at any time after step 1, although all updates must be processed
+in the order they were returned by Ready.
+
+Second, all persisted log entries must be made available via an
+implementation of the Storage interface. The provided MemoryStorage
+type can be used for this (if you repopulate its state upon a
+restart), or you can supply your own disk-backed implementation.
+
+Third, when you receive a message from another node, pass it to Node.Step:
+
+	func recvRaftRPC(ctx context.Context, m raftpb.Message) {
+		n.Step(ctx, m)
+	}
+
+Finally, you need to call Node.Tick() at regular intervals (probably
+via a time.Ticker). Raft has two important timeouts: heartbeat and the
+election timeout. However, internally to the raft package time is
+represented by an abstract "tick".
+
+The total state machine handling loop will look something like this:
+
+  for {
+    select {
+    case <-s.Ticker:
+      n.Tick()
+    case rd := <-s.Node.Ready():
+      saveToStorage(rd.State, rd.Entries, rd.Snapshot)
+      send(rd.Messages)
+      if !raft.IsEmptySnap(rd.Snapshot) {
+        processSnapshot(rd.Snapshot)
+      }
+      for _, entry := range rd.CommittedEntries {
+        process(entry)
+        if entry.Type == raftpb.EntryConfChange {
+          var cc raftpb.ConfChange
+          cc.Unmarshal(entry.Data)
+          s.Node.ApplyConfChange(cc)
+        }
+      }
+      s.Node.Advance()
+    case <-s.done:
+      return
+    }
+  }
+
+To propose changes to the state machine from your node take your application
+data, serialize it into a byte slice and call:
+
+	n.Propose(ctx, data)
+
+If the proposal is committed, data will appear in committed entries with type
+raftpb.EntryNormal. There is no guarantee that a proposed command will be
+committed; you may have to re-propose after a timeout.
+
+To add or remove node in a cluster, build ConfChange struct 'cc' and call:
+
+	n.ProposeConfChange(ctx, cc)
+
+After config change is committed, some committed entry with type
+raftpb.EntryConfChange will be returned. You must apply it to node through:
+
+	var cc raftpb.ConfChange
+	cc.Unmarshal(data)
+	n.ApplyConfChange(cc)
+
+Note: An ID represents a unique node in a cluster for all time. A
+given ID MUST be used only once even if the old node has been removed.
+This means that for example IP addresses make poor node IDs since they
+may be reused. Node IDs must be non-zero.
+
+Implementation notes
+
+This implementation is up to date with the final Raft thesis
+(https://ramcloud.stanford.edu/~ongaro/thesis.pdf), although our
+implementation of the membership change protocol differs somewhat from
+that described in chapter 4. The key invariant that membership changes
+happen one node at a time is preserved, but in our implementation the
+membership change takes effect when its entry is applied, not when it
+is added to the log (so the entry is committed under the old
+membership instead of the new). This is equivalent in terms of safety,
+since the old and new configurations are guaranteed to overlap.
+
+To ensure that we do not attempt to commit two membership changes at
+once by matching log positions (which would be unsafe since they
+should have different quorum requirements), we simply disallow any
+proposed membership change while any uncommitted change appears in
+the leader's log.
+
+This approach introduces a problem when you try to remove a member
+from a two-member cluster: If one of the members dies before the
+other one receives the commit of the confchange entry, then the member
+cannot be removed any more since the cluster cannot make progress.
+For this reason it is highly recommended to use three or more nodes in
+every cluster.
+
+MessageType
+
+Package raft sends and receives message in Protocol Buffer format (defined
+in raftpb package). Each state (follower, candidate, leader) implements its
+own 'step' method ('stepFollower', 'stepCandidate', 'stepLeader') when
+advancing with the given raftpb.Message. Each step is determined by its
+raftpb.MessageType. Note that every step is checked by one common method
+'Step' that safety-checks the terms of node and incoming message to prevent
+stale log entries:
+
+	'MsgHup' is used for election. If a node is a follower or candidate, the
+	'tick' function in 'raft' struct is set as 'tickElection'. If a follower or
+	candidate has not received any heartbeat before the election timeout, it
+	passes 'MsgHup' to its Step method and becomes (or remains) a candidate to
+	start a new election.
+
+	'MsgBeat' is an internal type that signals the leader to send a heartbeat of
+	the 'MsgHeartbeat' type. If a node is a leader, the 'tick' function in
+	the 'raft' struct is set as 'tickHeartbeat', and triggers the leader to
+	send periodic 'MsgHeartbeat' messages to its followers.
+
+	'MsgProp' proposes to append data to its log entries. This is a special
+	type to redirect proposals to leader. Therefore, send method overwrites
+	raftpb.Message's term with its HardState's term to avoid attaching its
+	local term to 'MsgProp'. When 'MsgProp' is passed to the leader's 'Step'
+	method, the leader first calls the 'appendEntry' method to append entries
+	to its log, and then calls 'bcastAppend' method to send those entries to
+	its peers. When passed to candidate, 'MsgProp' is dropped. When passed to
+	follower, 'MsgProp' is stored in follower's mailbox(msgs) by the send
+	method. It is stored with sender's ID and later forwarded to leader by
+	rafthttp package.
+
+	'MsgApp' contains log entries to replicate. A leader calls bcastAppend,
+	which calls sendAppend, which sends soon-to-be-replicated logs in 'MsgApp'
+	type. When 'MsgApp' is passed to candidate's Step method, candidate reverts
+	back to follower, because it indicates that there is a valid leader sending
+	'MsgApp' messages. Candidate and follower respond to this message in
+	'MsgAppResp' type.
+
+	'MsgAppResp' is response to log replication request('MsgApp'). When
+	'MsgApp' is passed to candidate or follower's Step method, it responds by
+	calling 'handleAppendEntries' method, which sends 'MsgAppResp' to raft
+	mailbox.
+
+	'MsgVote' requests votes for election. When a node is a follower or
+	candidate and 'MsgHup' is passed to its Step method, then the node calls
+	'campaign' method to campaign itself to become a leader. Once 'campaign'
+	method is called, the node becomes candidate and sends 'MsgVote' to peers
+	in cluster to request votes. When passed to leader or candidate's Step
+	method and the message's Term is lower than leader's or candidate's,
+	'MsgVote' will be rejected ('MsgVoteResp' is returned with Reject true).
+	If leader or candidate receives 'MsgVote' with higher term, it will revert
+	back to follower. When 'MsgVote' is passed to follower, it votes for the
+	sender only when sender's last term is greater than MsgVote's term or
+	sender's last term is equal to MsgVote's term but sender's last committed
+	index is greater than or equal to follower's.
+
+	'MsgVoteResp' contains responses from voting request. When 'MsgVoteResp' is
+	passed to candidate, the candidate calculates how many votes it has won. If
+	it's more than majority (quorum), it becomes leader and calls 'bcastAppend'.
+	If candidate receives majority of votes of denials, it reverts back to
+	follower.
+
+	'MsgPreVote' and 'MsgPreVoteResp' are used in an optional two-phase election
+	protocol. When Config.PreVote is true, a pre-election is carried out first
+	(using the same rules as a regular election), and no node increases its term
+	number unless the pre-election indicates that the campaigining node would win.
+	This minimizes disruption when a partitioned node rejoins the cluster.
+
+	'MsgSnap' requests to install a snapshot message. When a node has just
+	become a leader or the leader receives 'MsgProp' message, it calls
+	'bcastAppend' method, which then calls 'sendAppend' method to each
+	follower. In 'sendAppend', if a leader fails to get term or entries,
+	the leader requests snapshot by sending 'MsgSnap' type message.
+
+	'MsgSnapStatus' tells the result of snapshot install message. When a
+	follower rejected 'MsgSnap', it indicates the snapshot request with
+	'MsgSnap' had failed from network issues which causes the network layer
+	to fail to send out snapshots to its followers. Then leader considers
+	follower's progress as probe. When 'MsgSnap' were not rejected, it
+	indicates that the snapshot succeeded and the leader sets follower's
+	progress to probe and resumes its log replication.
+
+	'MsgHeartbeat' sends heartbeat from leader. When 'MsgHeartbeat' is passed
+	to candidate and message's term is higher than candidate's, the candidate
+	reverts back to follower and updates its committed index from the one in
+	this heartbeat. And it sends the message to its mailbox. When
+	'MsgHeartbeat' is passed to follower's Step method and message's term is
+	higher than follower's, the follower updates its leaderID with the ID
+	from the message.
+
+	'MsgHeartbeatResp' is a response to 'MsgHeartbeat'. When 'MsgHeartbeatResp'
+	is passed to leader's Step method, the leader knows which follower
+	responded. And only when the leader's last committed index is greater than
+	follower's Match index, the leader runs 'sendAppend` method.
+
+	'MsgUnreachable' tells that request(message) wasn't delivered. When
+	'MsgUnreachable' is passed to leader's Step method, the leader discovers
+	that the follower that sent this 'MsgUnreachable' is not reachable, often
+	indicating 'MsgApp' is lost. When follower's progress state is replicate,
+	the leader sets it back to probe.
+
+*/
+package raft
diff --git a/vendor/github.com/coreos/etcd/raft/log.go b/vendor/github.com/coreos/etcd/raft/log.go
new file mode 100644
index 0000000..c3036d3
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/log.go
@@ -0,0 +1,358 @@
+// 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 raft
+
+import (
+	"fmt"
+	"log"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+type raftLog struct {
+	// storage contains all stable entries since the last snapshot.
+	storage Storage
+
+	// unstable contains all unstable entries and snapshot.
+	// they will be saved into storage.
+	unstable unstable
+
+	// committed is the highest log position that is known to be in
+	// stable storage on a quorum of nodes.
+	committed uint64
+	// applied is the highest log position that the application has
+	// been instructed to apply to its state machine.
+	// Invariant: applied <= committed
+	applied uint64
+
+	logger Logger
+}
+
+// newLog returns log using the given storage. It recovers the log to the state
+// that it just commits and applies the latest snapshot.
+func newLog(storage Storage, logger Logger) *raftLog {
+	if storage == nil {
+		log.Panic("storage must not be nil")
+	}
+	log := &raftLog{
+		storage: storage,
+		logger:  logger,
+	}
+	firstIndex, err := storage.FirstIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	lastIndex, err := storage.LastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	log.unstable.offset = lastIndex + 1
+	log.unstable.logger = logger
+	// Initialize our committed and applied pointers to the time of the last compaction.
+	log.committed = firstIndex - 1
+	log.applied = firstIndex - 1
+
+	return log
+}
+
+func (l *raftLog) String() string {
+	return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, len(unstable.Entries)=%d", l.committed, l.applied, l.unstable.offset, len(l.unstable.entries))
+}
+
+// maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
+// it returns (last index of new entries, true).
+func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry) (lastnewi uint64, ok bool) {
+	if l.matchTerm(index, logTerm) {
+		lastnewi = index + uint64(len(ents))
+		ci := l.findConflict(ents)
+		switch {
+		case ci == 0:
+		case ci <= l.committed:
+			l.logger.Panicf("entry %d conflict with committed entry [committed(%d)]", ci, l.committed)
+		default:
+			offset := index + 1
+			l.append(ents[ci-offset:]...)
+		}
+		l.commitTo(min(committed, lastnewi))
+		return lastnewi, true
+	}
+	return 0, false
+}
+
+func (l *raftLog) append(ents ...pb.Entry) uint64 {
+	if len(ents) == 0 {
+		return l.lastIndex()
+	}
+	if after := ents[0].Index - 1; after < l.committed {
+		l.logger.Panicf("after(%d) is out of range [committed(%d)]", after, l.committed)
+	}
+	l.unstable.truncateAndAppend(ents)
+	return l.lastIndex()
+}
+
+// findConflict finds the index of the conflict.
+// It returns the first pair of conflicting entries between the existing
+// entries and the given entries, if there are any.
+// If there is no conflicting entries, and the existing entries contains
+// all the given entries, zero will be returned.
+// If there is no conflicting entries, but the given entries contains new
+// entries, the index of the first new entry will be returned.
+// An entry is considered to be conflicting if it has the same index but
+// a different term.
+// The first entry MUST have an index equal to the argument 'from'.
+// The index of the given entries MUST be continuously increasing.
+func (l *raftLog) findConflict(ents []pb.Entry) uint64 {
+	for _, ne := range ents {
+		if !l.matchTerm(ne.Index, ne.Term) {
+			if ne.Index <= l.lastIndex() {
+				l.logger.Infof("found conflict at index %d [existing term: %d, conflicting term: %d]",
+					ne.Index, l.zeroTermOnErrCompacted(l.term(ne.Index)), ne.Term)
+			}
+			return ne.Index
+		}
+	}
+	return 0
+}
+
+func (l *raftLog) unstableEntries() []pb.Entry {
+	if len(l.unstable.entries) == 0 {
+		return nil
+	}
+	return l.unstable.entries
+}
+
+// nextEnts returns all the available entries for execution.
+// If applied is smaller than the index of snapshot, it returns all committed
+// entries after the index of snapshot.
+func (l *raftLog) nextEnts() (ents []pb.Entry) {
+	off := max(l.applied+1, l.firstIndex())
+	if l.committed+1 > off {
+		ents, err := l.slice(off, l.committed+1, noLimit)
+		if err != nil {
+			l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err)
+		}
+		return ents
+	}
+	return nil
+}
+
+// hasNextEnts returns if there is any available entries for execution. This
+// is a fast check without heavy raftLog.slice() in raftLog.nextEnts().
+func (l *raftLog) hasNextEnts() bool {
+	off := max(l.applied+1, l.firstIndex())
+	return l.committed+1 > off
+}
+
+func (l *raftLog) snapshot() (pb.Snapshot, error) {
+	if l.unstable.snapshot != nil {
+		return *l.unstable.snapshot, nil
+	}
+	return l.storage.Snapshot()
+}
+
+func (l *raftLog) firstIndex() uint64 {
+	if i, ok := l.unstable.maybeFirstIndex(); ok {
+		return i
+	}
+	index, err := l.storage.FirstIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	return index
+}
+
+func (l *raftLog) lastIndex() uint64 {
+	if i, ok := l.unstable.maybeLastIndex(); ok {
+		return i
+	}
+	i, err := l.storage.LastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	return i
+}
+
+func (l *raftLog) commitTo(tocommit uint64) {
+	// never decrease commit
+	if l.committed < tocommit {
+		if l.lastIndex() < tocommit {
+			l.logger.Panicf("tocommit(%d) is out of range [lastIndex(%d)]. Was the raft log corrupted, truncated, or lost?", tocommit, l.lastIndex())
+		}
+		l.committed = tocommit
+	}
+}
+
+func (l *raftLog) appliedTo(i uint64) {
+	if i == 0 {
+		return
+	}
+	if l.committed < i || i < l.applied {
+		l.logger.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed)
+	}
+	l.applied = i
+}
+
+func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) }
+
+func (l *raftLog) stableSnapTo(i uint64) { l.unstable.stableSnapTo(i) }
+
+func (l *raftLog) lastTerm() uint64 {
+	t, err := l.term(l.lastIndex())
+	if err != nil {
+		l.logger.Panicf("unexpected error when getting the last term (%v)", err)
+	}
+	return t
+}
+
+func (l *raftLog) term(i uint64) (uint64, error) {
+	// the valid term range is [index of dummy entry, last index]
+	dummyIndex := l.firstIndex() - 1
+	if i < dummyIndex || i > l.lastIndex() {
+		// TODO: return an error instead?
+		return 0, nil
+	}
+
+	if t, ok := l.unstable.maybeTerm(i); ok {
+		return t, nil
+	}
+
+	t, err := l.storage.Term(i)
+	if err == nil {
+		return t, nil
+	}
+	if err == ErrCompacted || err == ErrUnavailable {
+		return 0, err
+	}
+	panic(err) // TODO(bdarnell)
+}
+
+func (l *raftLog) entries(i, maxsize uint64) ([]pb.Entry, error) {
+	if i > l.lastIndex() {
+		return nil, nil
+	}
+	return l.slice(i, l.lastIndex()+1, maxsize)
+}
+
+// allEntries returns all entries in the log.
+func (l *raftLog) allEntries() []pb.Entry {
+	ents, err := l.entries(l.firstIndex(), noLimit)
+	if err == nil {
+		return ents
+	}
+	if err == ErrCompacted { // try again if there was a racing compaction
+		return l.allEntries()
+	}
+	// TODO (xiangli): handle error?
+	panic(err)
+}
+
+// isUpToDate determines if the given (lastIndex,term) log is more up-to-date
+// by comparing the index and term of the last entries in the existing logs.
+// If the logs have last entries with different terms, then the log with the
+// later term is more up-to-date. If the logs end with the same term, then
+// whichever log has the larger lastIndex is more up-to-date. If the logs are
+// the same, the given log is up-to-date.
+func (l *raftLog) isUpToDate(lasti, term uint64) bool {
+	return term > l.lastTerm() || (term == l.lastTerm() && lasti >= l.lastIndex())
+}
+
+func (l *raftLog) matchTerm(i, term uint64) bool {
+	t, err := l.term(i)
+	if err != nil {
+		return false
+	}
+	return t == term
+}
+
+func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
+	if maxIndex > l.committed && l.zeroTermOnErrCompacted(l.term(maxIndex)) == term {
+		l.commitTo(maxIndex)
+		return true
+	}
+	return false
+}
+
+func (l *raftLog) restore(s pb.Snapshot) {
+	l.logger.Infof("log [%s] starts to restore snapshot [index: %d, term: %d]", l, s.Metadata.Index, s.Metadata.Term)
+	l.committed = s.Metadata.Index
+	l.unstable.restore(s)
+}
+
+// slice returns a slice of log entries from lo through hi-1, inclusive.
+func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) {
+	err := l.mustCheckOutOfBounds(lo, hi)
+	if err != nil {
+		return nil, err
+	}
+	if lo == hi {
+		return nil, nil
+	}
+	var ents []pb.Entry
+	if lo < l.unstable.offset {
+		storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), maxSize)
+		if err == ErrCompacted {
+			return nil, err
+		} else if err == ErrUnavailable {
+			l.logger.Panicf("entries[%d:%d) is unavailable from storage", lo, min(hi, l.unstable.offset))
+		} else if err != nil {
+			panic(err) // TODO(bdarnell)
+		}
+
+		// check if ents has reached the size limitation
+		if uint64(len(storedEnts)) < min(hi, l.unstable.offset)-lo {
+			return storedEnts, nil
+		}
+
+		ents = storedEnts
+	}
+	if hi > l.unstable.offset {
+		unstable := l.unstable.slice(max(lo, l.unstable.offset), hi)
+		if len(ents) > 0 {
+			ents = append([]pb.Entry{}, ents...)
+			ents = append(ents, unstable...)
+		} else {
+			ents = unstable
+		}
+	}
+	return limitSize(ents, maxSize), nil
+}
+
+// l.firstIndex <= lo <= hi <= l.firstIndex + len(l.entries)
+func (l *raftLog) mustCheckOutOfBounds(lo, hi uint64) error {
+	if lo > hi {
+		l.logger.Panicf("invalid slice %d > %d", lo, hi)
+	}
+	fi := l.firstIndex()
+	if lo < fi {
+		return ErrCompacted
+	}
+
+	length := l.lastIndex() + 1 - fi
+	if lo < fi || hi > fi+length {
+		l.logger.Panicf("slice[%d,%d) out of bound [%d,%d]", lo, hi, fi, l.lastIndex())
+	}
+	return nil
+}
+
+func (l *raftLog) zeroTermOnErrCompacted(t uint64, err error) uint64 {
+	if err == nil {
+		return t
+	}
+	if err == ErrCompacted {
+		return 0
+	}
+	l.logger.Panicf("unexpected error (%v)", err)
+	return 0
+}
diff --git a/vendor/github.com/coreos/etcd/raft/log_unstable.go b/vendor/github.com/coreos/etcd/raft/log_unstable.go
new file mode 100644
index 0000000..263af9c
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/log_unstable.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 raft
+
+import pb "github.com/coreos/etcd/raft/raftpb"
+
+// unstable.entries[i] has raft log position i+unstable.offset.
+// Note that unstable.offset may be less than the highest log
+// position in storage; this means that the next write to storage
+// might need to truncate the log before persisting unstable.entries.
+type unstable struct {
+	// the incoming unstable snapshot, if any.
+	snapshot *pb.Snapshot
+	// all entries that have not yet been written to storage.
+	entries []pb.Entry
+	offset  uint64
+
+	logger Logger
+}
+
+// maybeFirstIndex returns the index of the first possible entry in entries
+// if it has a snapshot.
+func (u *unstable) maybeFirstIndex() (uint64, bool) {
+	if u.snapshot != nil {
+		return u.snapshot.Metadata.Index + 1, true
+	}
+	return 0, false
+}
+
+// maybeLastIndex returns the last index if it has at least one
+// unstable entry or snapshot.
+func (u *unstable) maybeLastIndex() (uint64, bool) {
+	if l := len(u.entries); l != 0 {
+		return u.offset + uint64(l) - 1, true
+	}
+	if u.snapshot != nil {
+		return u.snapshot.Metadata.Index, true
+	}
+	return 0, false
+}
+
+// maybeTerm returns the term of the entry at index i, if there
+// is any.
+func (u *unstable) maybeTerm(i uint64) (uint64, bool) {
+	if i < u.offset {
+		if u.snapshot == nil {
+			return 0, false
+		}
+		if u.snapshot.Metadata.Index == i {
+			return u.snapshot.Metadata.Term, true
+		}
+		return 0, false
+	}
+
+	last, ok := u.maybeLastIndex()
+	if !ok {
+		return 0, false
+	}
+	if i > last {
+		return 0, false
+	}
+	return u.entries[i-u.offset].Term, true
+}
+
+func (u *unstable) stableTo(i, t uint64) {
+	gt, ok := u.maybeTerm(i)
+	if !ok {
+		return
+	}
+	// if i < offset, term is matched with the snapshot
+	// only update the unstable entries if term is matched with
+	// an unstable entry.
+	if gt == t && i >= u.offset {
+		u.entries = u.entries[i+1-u.offset:]
+		u.offset = i + 1
+		u.shrinkEntriesArray()
+	}
+}
+
+// shrinkEntriesArray discards the underlying array used by the entries slice
+// if most of it isn't being used. This avoids holding references to a bunch of
+// potentially large entries that aren't needed anymore. Simply clearing the
+// entries wouldn't be safe because clients might still be using them.
+func (u *unstable) shrinkEntriesArray() {
+	// We replace the array if we're using less than half of the space in
+	// it. This number is fairly arbitrary, chosen as an attempt to balance
+	// memory usage vs number of allocations. It could probably be improved
+	// with some focused tuning.
+	const lenMultiple = 2
+	if len(u.entries) == 0 {
+		u.entries = nil
+	} else if len(u.entries)*lenMultiple < cap(u.entries) {
+		newEntries := make([]pb.Entry, len(u.entries))
+		copy(newEntries, u.entries)
+		u.entries = newEntries
+	}
+}
+
+func (u *unstable) stableSnapTo(i uint64) {
+	if u.snapshot != nil && u.snapshot.Metadata.Index == i {
+		u.snapshot = nil
+	}
+}
+
+func (u *unstable) restore(s pb.Snapshot) {
+	u.offset = s.Metadata.Index + 1
+	u.entries = nil
+	u.snapshot = &s
+}
+
+func (u *unstable) truncateAndAppend(ents []pb.Entry) {
+	after := ents[0].Index
+	switch {
+	case after == u.offset+uint64(len(u.entries)):
+		// after is the next index in the u.entries
+		// directly append
+		u.entries = append(u.entries, ents...)
+	case after <= u.offset:
+		u.logger.Infof("replace the unstable entries from index %d", after)
+		// The log is being truncated to before our current offset
+		// portion, so set the offset and replace the entries
+		u.offset = after
+		u.entries = ents
+	default:
+		// truncate to after and copy to u.entries
+		// then append
+		u.logger.Infof("truncate the unstable entries before index %d", after)
+		u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...)
+		u.entries = append(u.entries, ents...)
+	}
+}
+
+func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry {
+	u.mustCheckOutOfBounds(lo, hi)
+	return u.entries[lo-u.offset : hi-u.offset]
+}
+
+// u.offset <= lo <= hi <= u.offset+len(u.offset)
+func (u *unstable) mustCheckOutOfBounds(lo, hi uint64) {
+	if lo > hi {
+		u.logger.Panicf("invalid unstable.slice %d > %d", lo, hi)
+	}
+	upper := u.offset + uint64(len(u.entries))
+	if lo < u.offset || hi > upper {
+		u.logger.Panicf("unstable.slice[%d,%d) out of bound [%d,%d]", lo, hi, u.offset, upper)
+	}
+}
diff --git a/vendor/github.com/coreos/etcd/raft/logger.go b/vendor/github.com/coreos/etcd/raft/logger.go
new file mode 100644
index 0000000..426a77d
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/logger.go
@@ -0,0 +1,126 @@
+// 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 raft
+
+import (
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+)
+
+type Logger interface {
+	Debug(v ...interface{})
+	Debugf(format string, v ...interface{})
+
+	Error(v ...interface{})
+	Errorf(format string, v ...interface{})
+
+	Info(v ...interface{})
+	Infof(format string, v ...interface{})
+
+	Warning(v ...interface{})
+	Warningf(format string, v ...interface{})
+
+	Fatal(v ...interface{})
+	Fatalf(format string, v ...interface{})
+
+	Panic(v ...interface{})
+	Panicf(format string, v ...interface{})
+}
+
+func SetLogger(l Logger) { raftLogger = l }
+
+var (
+	defaultLogger = &DefaultLogger{Logger: log.New(os.Stderr, "raft", log.LstdFlags)}
+	discardLogger = &DefaultLogger{Logger: log.New(ioutil.Discard, "", 0)}
+	raftLogger    = Logger(defaultLogger)
+)
+
+const (
+	calldepth = 2
+)
+
+// DefaultLogger is a default implementation of the Logger interface.
+type DefaultLogger struct {
+	*log.Logger
+	debug bool
+}
+
+func (l *DefaultLogger) EnableTimestamps() {
+	l.SetFlags(l.Flags() | log.Ldate | log.Ltime)
+}
+
+func (l *DefaultLogger) EnableDebug() {
+	l.debug = true
+}
+
+func (l *DefaultLogger) Debug(v ...interface{}) {
+	if l.debug {
+		l.Output(calldepth, header("DEBUG", fmt.Sprint(v...)))
+	}
+}
+
+func (l *DefaultLogger) Debugf(format string, v ...interface{}) {
+	if l.debug {
+		l.Output(calldepth, header("DEBUG", fmt.Sprintf(format, v...)))
+	}
+}
+
+func (l *DefaultLogger) Info(v ...interface{}) {
+	l.Output(calldepth, header("INFO", fmt.Sprint(v...)))
+}
+
+func (l *DefaultLogger) Infof(format string, v ...interface{}) {
+	l.Output(calldepth, header("INFO", fmt.Sprintf(format, v...)))
+}
+
+func (l *DefaultLogger) Error(v ...interface{}) {
+	l.Output(calldepth, header("ERROR", fmt.Sprint(v...)))
+}
+
+func (l *DefaultLogger) Errorf(format string, v ...interface{}) {
+	l.Output(calldepth, header("ERROR", fmt.Sprintf(format, v...)))
+}
+
+func (l *DefaultLogger) Warning(v ...interface{}) {
+	l.Output(calldepth, header("WARN", fmt.Sprint(v...)))
+}
+
+func (l *DefaultLogger) Warningf(format string, v ...interface{}) {
+	l.Output(calldepth, header("WARN", fmt.Sprintf(format, v...)))
+}
+
+func (l *DefaultLogger) Fatal(v ...interface{}) {
+	l.Output(calldepth, header("FATAL", fmt.Sprint(v...)))
+	os.Exit(1)
+}
+
+func (l *DefaultLogger) Fatalf(format string, v ...interface{}) {
+	l.Output(calldepth, header("FATAL", fmt.Sprintf(format, v...)))
+	os.Exit(1)
+}
+
+func (l *DefaultLogger) Panic(v ...interface{}) {
+	l.Logger.Panic(v...)
+}
+
+func (l *DefaultLogger) Panicf(format string, v ...interface{}) {
+	l.Logger.Panicf(format, v...)
+}
+
+func header(lvl, msg string) string {
+	return fmt.Sprintf("%s: %s", lvl, msg)
+}
diff --git a/vendor/github.com/coreos/etcd/raft/node.go b/vendor/github.com/coreos/etcd/raft/node.go
new file mode 100644
index 0000000..33a9db8
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/node.go
@@ -0,0 +1,539 @@
+// 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 raft
+
+import (
+	"context"
+	"errors"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+type SnapshotStatus int
+
+const (
+	SnapshotFinish  SnapshotStatus = 1
+	SnapshotFailure SnapshotStatus = 2
+)
+
+var (
+	emptyState = pb.HardState{}
+
+	// ErrStopped is returned by methods on Nodes that have been stopped.
+	ErrStopped = errors.New("raft: stopped")
+)
+
+// SoftState provides state that is useful for logging and debugging.
+// The state is volatile and does not need to be persisted to the WAL.
+type SoftState struct {
+	Lead      uint64 // must use atomic operations to access; keep 64-bit aligned.
+	RaftState StateType
+}
+
+func (a *SoftState) equal(b *SoftState) bool {
+	return a.Lead == b.Lead && a.RaftState == b.RaftState
+}
+
+// Ready encapsulates the entries and messages that are ready to read,
+// be saved to stable storage, committed or sent to other peers.
+// All fields in Ready are read-only.
+type Ready struct {
+	// The current volatile state of a Node.
+	// SoftState will be nil if there is no update.
+	// It is not required to consume or store SoftState.
+	*SoftState
+
+	// The current state of a Node to be saved to stable storage BEFORE
+	// Messages are sent.
+	// HardState will be equal to empty state if there is no update.
+	pb.HardState
+
+	// ReadStates can be used for node to serve linearizable read requests locally
+	// when its applied index is greater than the index in ReadState.
+	// Note that the readState will be returned when raft receives msgReadIndex.
+	// The returned is only valid for the request that requested to read.
+	ReadStates []ReadState
+
+	// Entries specifies entries to be saved to stable storage BEFORE
+	// Messages are sent.
+	Entries []pb.Entry
+
+	// Snapshot specifies the snapshot to be saved to stable storage.
+	Snapshot pb.Snapshot
+
+	// CommittedEntries specifies entries to be committed to a
+	// store/state-machine. These have previously been committed to stable
+	// store.
+	CommittedEntries []pb.Entry
+
+	// Messages specifies outbound messages to be sent AFTER Entries are
+	// committed to stable storage.
+	// If it contains a MsgSnap message, the application MUST report back to raft
+	// when the snapshot has been received or has failed by calling ReportSnapshot.
+	Messages []pb.Message
+
+	// MustSync indicates whether the HardState and Entries must be synchronously
+	// written to disk or if an asynchronous write is permissible.
+	MustSync bool
+}
+
+func isHardStateEqual(a, b pb.HardState) bool {
+	return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
+}
+
+// IsEmptyHardState returns true if the given HardState is empty.
+func IsEmptyHardState(st pb.HardState) bool {
+	return isHardStateEqual(st, emptyState)
+}
+
+// IsEmptySnap returns true if the given Snapshot is empty.
+func IsEmptySnap(sp pb.Snapshot) bool {
+	return sp.Metadata.Index == 0
+}
+
+func (rd Ready) containsUpdates() bool {
+	return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
+		!IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
+		len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0 || len(rd.ReadStates) != 0
+}
+
+// Node represents a node in a raft cluster.
+type Node interface {
+	// Tick increments the internal logical clock for the Node by a single tick. Election
+	// timeouts and heartbeat timeouts are in units of ticks.
+	Tick()
+	// Campaign causes the Node to transition to candidate state and start campaigning to become leader.
+	Campaign(ctx context.Context) error
+	// Propose proposes that data be appended to the log.
+	Propose(ctx context.Context, data []byte) error
+	// ProposeConfChange proposes config change.
+	// At most one ConfChange can be in the process of going through consensus.
+	// Application needs to call ApplyConfChange when applying EntryConfChange type entry.
+	ProposeConfChange(ctx context.Context, cc pb.ConfChange) error
+	// Step advances the state machine using the given message. ctx.Err() will be returned, if any.
+	Step(ctx context.Context, msg pb.Message) error
+
+	// Ready returns a channel that returns the current point-in-time state.
+	// Users of the Node must call Advance after retrieving the state returned by Ready.
+	//
+	// NOTE: No committed entries from the next Ready may be applied until all committed entries
+	// and snapshots from the previous one have finished.
+	Ready() <-chan Ready
+
+	// Advance notifies the Node that the application has saved progress up to the last Ready.
+	// It prepares the node to return the next available Ready.
+	//
+	// The application should generally call Advance after it applies the entries in last Ready.
+	//
+	// However, as an optimization, the application may call Advance while it is applying the
+	// commands. For example. when the last Ready contains a snapshot, the application might take
+	// a long time to apply the snapshot data. To continue receiving Ready without blocking raft
+	// progress, it can call Advance before finishing applying the last ready.
+	Advance()
+	// ApplyConfChange applies config change to the local node.
+	// Returns an opaque ConfState protobuf which must be recorded
+	// in snapshots. Will never return nil; it returns a pointer only
+	// to match MemoryStorage.Compact.
+	ApplyConfChange(cc pb.ConfChange) *pb.ConfState
+
+	// TransferLeadership attempts to transfer leadership to the given transferee.
+	TransferLeadership(ctx context.Context, lead, transferee uint64)
+
+	// ReadIndex request a read state. The read state will be set in the ready.
+	// Read state has a read index. Once the application advances further than the read
+	// index, any linearizable read requests issued before the read request can be
+	// processed safely. The read state will have the same rctx attached.
+	ReadIndex(ctx context.Context, rctx []byte) error
+
+	// Status returns the current status of the raft state machine.
+	Status() Status
+	// ReportUnreachable reports the given node is not reachable for the last send.
+	ReportUnreachable(id uint64)
+	// ReportSnapshot reports the status of the sent snapshot.
+	ReportSnapshot(id uint64, status SnapshotStatus)
+	// Stop performs any necessary termination of the Node.
+	Stop()
+}
+
+type Peer struct {
+	ID      uint64
+	Context []byte
+}
+
+// StartNode returns a new Node given configuration and a list of raft peers.
+// It appends a ConfChangeAddNode entry for each given peer to the initial log.
+func StartNode(c *Config, peers []Peer) Node {
+	r := newRaft(c)
+	// become the follower at term 1 and apply initial configuration
+	// entries of term 1
+	r.becomeFollower(1, None)
+	for _, peer := range peers {
+		cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
+		d, err := cc.Marshal()
+		if err != nil {
+			panic("unexpected marshal error")
+		}
+		e := pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: r.raftLog.lastIndex() + 1, Data: d}
+		r.raftLog.append(e)
+	}
+	// Mark these initial entries as committed.
+	// TODO(bdarnell): These entries are still unstable; do we need to preserve
+	// the invariant that committed < unstable?
+	r.raftLog.committed = r.raftLog.lastIndex()
+	// Now apply them, mainly so that the application can call Campaign
+	// immediately after StartNode in tests. Note that these nodes will
+	// be added to raft twice: here and when the application's Ready
+	// loop calls ApplyConfChange. The calls to addNode must come after
+	// all calls to raftLog.append so progress.next is set after these
+	// bootstrapping entries (it is an error if we try to append these
+	// entries since they have already been committed).
+	// We do not set raftLog.applied so the application will be able
+	// to observe all conf changes via Ready.CommittedEntries.
+	for _, peer := range peers {
+		r.addNode(peer.ID)
+	}
+
+	n := newNode()
+	n.logger = c.Logger
+	go n.run(r)
+	return &n
+}
+
+// RestartNode is similar to StartNode but does not take a list of peers.
+// The current membership of the cluster will be restored from the Storage.
+// If the caller has an existing state machine, pass in the last log index that
+// has been applied to it; otherwise use zero.
+func RestartNode(c *Config) Node {
+	r := newRaft(c)
+
+	n := newNode()
+	n.logger = c.Logger
+	go n.run(r)
+	return &n
+}
+
+// node is the canonical implementation of the Node interface
+type node struct {
+	propc      chan pb.Message
+	recvc      chan pb.Message
+	confc      chan pb.ConfChange
+	confstatec chan pb.ConfState
+	readyc     chan Ready
+	advancec   chan struct{}
+	tickc      chan struct{}
+	done       chan struct{}
+	stop       chan struct{}
+	status     chan chan Status
+
+	logger Logger
+}
+
+func newNode() node {
+	return node{
+		propc:      make(chan pb.Message),
+		recvc:      make(chan pb.Message),
+		confc:      make(chan pb.ConfChange),
+		confstatec: make(chan pb.ConfState),
+		readyc:     make(chan Ready),
+		advancec:   make(chan struct{}),
+		// make tickc a buffered chan, so raft node can buffer some ticks when the node
+		// is busy processing raft messages. Raft node will resume process buffered
+		// ticks when it becomes idle.
+		tickc:  make(chan struct{}, 128),
+		done:   make(chan struct{}),
+		stop:   make(chan struct{}),
+		status: make(chan chan Status),
+	}
+}
+
+func (n *node) Stop() {
+	select {
+	case n.stop <- struct{}{}:
+		// Not already stopped, so trigger it
+	case <-n.done:
+		// Node has already been stopped - no need to do anything
+		return
+	}
+	// Block until the stop has been acknowledged by run()
+	<-n.done
+}
+
+func (n *node) run(r *raft) {
+	var propc chan pb.Message
+	var readyc chan Ready
+	var advancec chan struct{}
+	var prevLastUnstablei, prevLastUnstablet uint64
+	var havePrevLastUnstablei bool
+	var prevSnapi uint64
+	var rd Ready
+
+	lead := None
+	prevSoftSt := r.softState()
+	prevHardSt := emptyState
+
+	for {
+		if advancec != nil {
+			readyc = nil
+		} else {
+			rd = newReady(r, prevSoftSt, prevHardSt)
+			if rd.containsUpdates() {
+				readyc = n.readyc
+			} else {
+				readyc = nil
+			}
+		}
+
+		if lead != r.lead {
+			if r.hasLeader() {
+				if lead == None {
+					r.logger.Infof("raft.node: %x elected leader %x at term %d", r.id, r.lead, r.Term)
+				} else {
+					r.logger.Infof("raft.node: %x changed leader from %x to %x at term %d", r.id, lead, r.lead, r.Term)
+				}
+				propc = n.propc
+			} else {
+				r.logger.Infof("raft.node: %x lost leader %x at term %d", r.id, lead, r.Term)
+				propc = nil
+			}
+			lead = r.lead
+		}
+
+		select {
+		// TODO: maybe buffer the config propose if there exists one (the way
+		// described in raft dissertation)
+		// Currently it is dropped in Step silently.
+		case m := <-propc:
+			m.From = r.id
+			r.Step(m)
+		case m := <-n.recvc:
+			// filter out response message from unknown From.
+			if pr := r.getProgress(m.From); pr != nil || !IsResponseMsg(m.Type) {
+				r.Step(m) // raft never returns an error
+			}
+		case cc := <-n.confc:
+			if cc.NodeID == None {
+				r.resetPendingConf()
+				select {
+				case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
+				case <-n.done:
+				}
+				break
+			}
+			switch cc.Type {
+			case pb.ConfChangeAddNode:
+				r.addNode(cc.NodeID)
+			case pb.ConfChangeAddLearnerNode:
+				r.addLearner(cc.NodeID)
+			case pb.ConfChangeRemoveNode:
+				// block incoming proposal when local node is
+				// removed
+				if cc.NodeID == r.id {
+					propc = nil
+				}
+				r.removeNode(cc.NodeID)
+			case pb.ConfChangeUpdateNode:
+				r.resetPendingConf()
+			default:
+				panic("unexpected conf type")
+			}
+			select {
+			case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
+			case <-n.done:
+			}
+		case <-n.tickc:
+			r.tick()
+		case readyc <- rd:
+			if rd.SoftState != nil {
+				prevSoftSt = rd.SoftState
+			}
+			if len(rd.Entries) > 0 {
+				prevLastUnstablei = rd.Entries[len(rd.Entries)-1].Index
+				prevLastUnstablet = rd.Entries[len(rd.Entries)-1].Term
+				havePrevLastUnstablei = true
+			}
+			if !IsEmptyHardState(rd.HardState) {
+				prevHardSt = rd.HardState
+			}
+			if !IsEmptySnap(rd.Snapshot) {
+				prevSnapi = rd.Snapshot.Metadata.Index
+			}
+
+			r.msgs = nil
+			r.readStates = nil
+			advancec = n.advancec
+		case <-advancec:
+			if prevHardSt.Commit != 0 {
+				r.raftLog.appliedTo(prevHardSt.Commit)
+			}
+			if havePrevLastUnstablei {
+				r.raftLog.stableTo(prevLastUnstablei, prevLastUnstablet)
+				havePrevLastUnstablei = false
+			}
+			r.raftLog.stableSnapTo(prevSnapi)
+			advancec = nil
+		case c := <-n.status:
+			c <- getStatus(r)
+		case <-n.stop:
+			close(n.done)
+			return
+		}
+	}
+}
+
+// Tick increments the internal logical clock for this Node. Election timeouts
+// and heartbeat timeouts are in units of ticks.
+func (n *node) Tick() {
+	select {
+	case n.tickc <- struct{}{}:
+	case <-n.done:
+	default:
+		n.logger.Warningf("A tick missed to fire. Node blocks too long!")
+	}
+}
+
+func (n *node) Campaign(ctx context.Context) error { return n.step(ctx, pb.Message{Type: pb.MsgHup}) }
+
+func (n *node) Propose(ctx context.Context, data []byte) error {
+	return n.step(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
+}
+
+func (n *node) Step(ctx context.Context, m pb.Message) error {
+	// ignore unexpected local messages receiving over network
+	if IsLocalMsg(m.Type) {
+		// TODO: return an error?
+		return nil
+	}
+	return n.step(ctx, m)
+}
+
+func (n *node) ProposeConfChange(ctx context.Context, cc pb.ConfChange) error {
+	data, err := cc.Marshal()
+	if err != nil {
+		return err
+	}
+	return n.Step(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange, Data: data}}})
+}
+
+// Step advances the state machine using msgs. The ctx.Err() will be returned,
+// if any.
+func (n *node) step(ctx context.Context, m pb.Message) error {
+	ch := n.recvc
+	if m.Type == pb.MsgProp {
+		ch = n.propc
+	}
+
+	select {
+	case ch <- m:
+		return nil
+	case <-ctx.Done():
+		return ctx.Err()
+	case <-n.done:
+		return ErrStopped
+	}
+}
+
+func (n *node) Ready() <-chan Ready { return n.readyc }
+
+func (n *node) Advance() {
+	select {
+	case n.advancec <- struct{}{}:
+	case <-n.done:
+	}
+}
+
+func (n *node) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
+	var cs pb.ConfState
+	select {
+	case n.confc <- cc:
+	case <-n.done:
+	}
+	select {
+	case cs = <-n.confstatec:
+	case <-n.done:
+	}
+	return &cs
+}
+
+func (n *node) Status() Status {
+	c := make(chan Status)
+	select {
+	case n.status <- c:
+		return <-c
+	case <-n.done:
+		return Status{}
+	}
+}
+
+func (n *node) ReportUnreachable(id uint64) {
+	select {
+	case n.recvc <- pb.Message{Type: pb.MsgUnreachable, From: id}:
+	case <-n.done:
+	}
+}
+
+func (n *node) ReportSnapshot(id uint64, status SnapshotStatus) {
+	rej := status == SnapshotFailure
+
+	select {
+	case n.recvc <- pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej}:
+	case <-n.done:
+	}
+}
+
+func (n *node) TransferLeadership(ctx context.Context, lead, transferee uint64) {
+	select {
+	// manually set 'from' and 'to', so that leader can voluntarily transfers its leadership
+	case n.recvc <- pb.Message{Type: pb.MsgTransferLeader, From: transferee, To: lead}:
+	case <-n.done:
+	case <-ctx.Done():
+	}
+}
+
+func (n *node) ReadIndex(ctx context.Context, rctx []byte) error {
+	return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
+}
+
+func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
+	rd := Ready{
+		Entries:          r.raftLog.unstableEntries(),
+		CommittedEntries: r.raftLog.nextEnts(),
+		Messages:         r.msgs,
+	}
+	if softSt := r.softState(); !softSt.equal(prevSoftSt) {
+		rd.SoftState = softSt
+	}
+	if hardSt := r.hardState(); !isHardStateEqual(hardSt, prevHardSt) {
+		rd.HardState = hardSt
+	}
+	if r.raftLog.unstable.snapshot != nil {
+		rd.Snapshot = *r.raftLog.unstable.snapshot
+	}
+	if len(r.readStates) != 0 {
+		rd.ReadStates = r.readStates
+	}
+	rd.MustSync = MustSync(rd.HardState, prevHardSt, len(rd.Entries))
+	return rd
+}
+
+// MustSync returns true if the hard state and count of Raft entries indicate
+// that a synchronous write to persistent storage is required.
+func MustSync(st, prevst pb.HardState, entsnum int) bool {
+	// Persistent state on all servers:
+	// (Updated on stable storage before responding to RPCs)
+	// currentTerm
+	// votedFor
+	// log entries[]
+	return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term
+}
diff --git a/vendor/github.com/coreos/etcd/raft/progress.go b/vendor/github.com/coreos/etcd/raft/progress.go
new file mode 100644
index 0000000..ef3787d
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/progress.go
@@ -0,0 +1,284 @@
+// 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 raft
+
+import "fmt"
+
+const (
+	ProgressStateProbe ProgressStateType = iota
+	ProgressStateReplicate
+	ProgressStateSnapshot
+)
+
+type ProgressStateType uint64
+
+var prstmap = [...]string{
+	"ProgressStateProbe",
+	"ProgressStateReplicate",
+	"ProgressStateSnapshot",
+}
+
+func (st ProgressStateType) String() string { return prstmap[uint64(st)] }
+
+// Progress represents a follower’s progress in the view of the leader. Leader maintains
+// progresses of all followers, and sends entries to the follower based on its progress.
+type Progress struct {
+	Match, Next uint64
+	// State defines how the leader should interact with the follower.
+	//
+	// When in ProgressStateProbe, leader sends at most one replication message
+	// per heartbeat interval. It also probes actual progress of the follower.
+	//
+	// When in ProgressStateReplicate, leader optimistically increases next
+	// to the latest entry sent after sending replication message. This is
+	// an optimized state for fast replicating log entries to the follower.
+	//
+	// When in ProgressStateSnapshot, leader should have sent out snapshot
+	// before and stops sending any replication message.
+	State ProgressStateType
+
+	// Paused is used in ProgressStateProbe.
+	// When Paused is true, raft should pause sending replication message to this peer.
+	Paused bool
+	// PendingSnapshot is used in ProgressStateSnapshot.
+	// If there is a pending snapshot, the pendingSnapshot will be set to the
+	// index of the snapshot. If pendingSnapshot is set, the replication process of
+	// this Progress will be paused. raft will not resend snapshot until the pending one
+	// is reported to be failed.
+	PendingSnapshot uint64
+
+	// RecentActive is true if the progress is recently active. Receiving any messages
+	// from the corresponding follower indicates the progress is active.
+	// RecentActive can be reset to false after an election timeout.
+	RecentActive bool
+
+	// inflights is a sliding window for the inflight messages.
+	// Each inflight message contains one or more log entries.
+	// The max number of entries per message is defined in raft config as MaxSizePerMsg.
+	// Thus inflight effectively limits both the number of inflight messages
+	// and the bandwidth each Progress can use.
+	// When inflights is full, no more message should be sent.
+	// When a leader sends out a message, the index of the last
+	// entry should be added to inflights. The index MUST be added
+	// into inflights in order.
+	// When a leader receives a reply, the previous inflights should
+	// be freed by calling inflights.freeTo with the index of the last
+	// received entry.
+	ins *inflights
+
+	// IsLearner is true if this progress is tracked for a learner.
+	IsLearner bool
+}
+
+func (pr *Progress) resetState(state ProgressStateType) {
+	pr.Paused = false
+	pr.PendingSnapshot = 0
+	pr.State = state
+	pr.ins.reset()
+}
+
+func (pr *Progress) becomeProbe() {
+	// If the original state is ProgressStateSnapshot, progress knows that
+	// the pending snapshot has been sent to this peer successfully, then
+	// probes from pendingSnapshot + 1.
+	if pr.State == ProgressStateSnapshot {
+		pendingSnapshot := pr.PendingSnapshot
+		pr.resetState(ProgressStateProbe)
+		pr.Next = max(pr.Match+1, pendingSnapshot+1)
+	} else {
+		pr.resetState(ProgressStateProbe)
+		pr.Next = pr.Match + 1
+	}
+}
+
+func (pr *Progress) becomeReplicate() {
+	pr.resetState(ProgressStateReplicate)
+	pr.Next = pr.Match + 1
+}
+
+func (pr *Progress) becomeSnapshot(snapshoti uint64) {
+	pr.resetState(ProgressStateSnapshot)
+	pr.PendingSnapshot = snapshoti
+}
+
+// maybeUpdate returns false if the given n index comes from an outdated message.
+// Otherwise it updates the progress and returns true.
+func (pr *Progress) maybeUpdate(n uint64) bool {
+	var updated bool
+	if pr.Match < n {
+		pr.Match = n
+		updated = true
+		pr.resume()
+	}
+	if pr.Next < n+1 {
+		pr.Next = n + 1
+	}
+	return updated
+}
+
+func (pr *Progress) optimisticUpdate(n uint64) { pr.Next = n + 1 }
+
+// maybeDecrTo returns false if the given to index comes from an out of order message.
+// Otherwise it decreases the progress next index to min(rejected, last) and returns true.
+func (pr *Progress) maybeDecrTo(rejected, last uint64) bool {
+	if pr.State == ProgressStateReplicate {
+		// the rejection must be stale if the progress has matched and "rejected"
+		// is smaller than "match".
+		if rejected <= pr.Match {
+			return false
+		}
+		// directly decrease next to match + 1
+		pr.Next = pr.Match + 1
+		return true
+	}
+
+	// the rejection must be stale if "rejected" does not match next - 1
+	if pr.Next-1 != rejected {
+		return false
+	}
+
+	if pr.Next = min(rejected, last+1); pr.Next < 1 {
+		pr.Next = 1
+	}
+	pr.resume()
+	return true
+}
+
+func (pr *Progress) pause()  { pr.Paused = true }
+func (pr *Progress) resume() { pr.Paused = false }
+
+// IsPaused returns whether sending log entries to this node has been
+// paused. A node may be paused because it has rejected recent
+// MsgApps, is currently waiting for a snapshot, or has reached the
+// MaxInflightMsgs limit.
+func (pr *Progress) IsPaused() bool {
+	switch pr.State {
+	case ProgressStateProbe:
+		return pr.Paused
+	case ProgressStateReplicate:
+		return pr.ins.full()
+	case ProgressStateSnapshot:
+		return true
+	default:
+		panic("unexpected state")
+	}
+}
+
+func (pr *Progress) snapshotFailure() { pr.PendingSnapshot = 0 }
+
+// needSnapshotAbort returns true if snapshot progress's Match
+// is equal or higher than the pendingSnapshot.
+func (pr *Progress) needSnapshotAbort() bool {
+	return pr.State == ProgressStateSnapshot && pr.Match >= pr.PendingSnapshot
+}
+
+func (pr *Progress) String() string {
+	return fmt.Sprintf("next = %d, match = %d, state = %s, waiting = %v, pendingSnapshot = %d", pr.Next, pr.Match, pr.State, pr.IsPaused(), pr.PendingSnapshot)
+}
+
+type inflights struct {
+	// the starting index in the buffer
+	start int
+	// number of inflights in the buffer
+	count int
+
+	// the size of the buffer
+	size int
+
+	// buffer contains the index of the last entry
+	// inside one message.
+	buffer []uint64
+}
+
+func newInflights(size int) *inflights {
+	return &inflights{
+		size: size,
+	}
+}
+
+// add adds an inflight into inflights
+func (in *inflights) add(inflight uint64) {
+	if in.full() {
+		panic("cannot add into a full inflights")
+	}
+	next := in.start + in.count
+	size := in.size
+	if next >= size {
+		next -= size
+	}
+	if next >= len(in.buffer) {
+		in.growBuf()
+	}
+	in.buffer[next] = inflight
+	in.count++
+}
+
+// grow the inflight buffer by doubling up to inflights.size. We grow on demand
+// instead of preallocating to inflights.size to handle systems which have
+// thousands of Raft groups per process.
+func (in *inflights) growBuf() {
+	newSize := len(in.buffer) * 2
+	if newSize == 0 {
+		newSize = 1
+	} else if newSize > in.size {
+		newSize = in.size
+	}
+	newBuffer := make([]uint64, newSize)
+	copy(newBuffer, in.buffer)
+	in.buffer = newBuffer
+}
+
+// freeTo frees the inflights smaller or equal to the given `to` flight.
+func (in *inflights) freeTo(to uint64) {
+	if in.count == 0 || to < in.buffer[in.start] {
+		// out of the left side of the window
+		return
+	}
+
+	idx := in.start
+	var i int
+	for i = 0; i < in.count; i++ {
+		if to < in.buffer[idx] { // found the first large inflight
+			break
+		}
+
+		// increase index and maybe rotate
+		size := in.size
+		if idx++; idx >= size {
+			idx -= size
+		}
+	}
+	// free i inflights and set new start index
+	in.count -= i
+	in.start = idx
+	if in.count == 0 {
+		// inflights is empty, reset the start index so that we don't grow the
+		// buffer unnecessarily.
+		in.start = 0
+	}
+}
+
+func (in *inflights) freeFirstOne() { in.freeTo(in.buffer[in.start]) }
+
+// full returns true if the inflights is full.
+func (in *inflights) full() bool {
+	return in.count == in.size
+}
+
+// resets frees all inflights.
+func (in *inflights) reset() {
+	in.count = 0
+	in.start = 0
+}
diff --git a/vendor/github.com/coreos/etcd/raft/raft.go b/vendor/github.com/coreos/etcd/raft/raft.go
new file mode 100644
index 0000000..22ff138
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/raft.go
@@ -0,0 +1,1407 @@
+// 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 raft
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"math"
+	"math/rand"
+	"sort"
+	"strings"
+	"sync"
+	"time"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+// None is a placeholder node ID used when there is no leader.
+const None uint64 = 0
+const noLimit = math.MaxUint64
+
+// Possible values for StateType.
+const (
+	StateFollower StateType = iota
+	StateCandidate
+	StateLeader
+	StatePreCandidate
+	numStates
+)
+
+type ReadOnlyOption int
+
+const (
+	// ReadOnlySafe guarantees the linearizability of the read only request by
+	// communicating with the quorum. It is the default and suggested option.
+	ReadOnlySafe ReadOnlyOption = iota
+	// ReadOnlyLeaseBased ensures linearizability of the read only request by
+	// relying on the leader lease. It can be affected by clock drift.
+	// If the clock drift is unbounded, leader might keep the lease longer than it
+	// should (clock can move backward/pause without any bound). ReadIndex is not safe
+	// in that case.
+	ReadOnlyLeaseBased
+)
+
+// Possible values for CampaignType
+const (
+	// campaignPreElection represents the first phase of a normal election when
+	// Config.PreVote is true.
+	campaignPreElection CampaignType = "CampaignPreElection"
+	// campaignElection represents a normal (time-based) election (the second phase
+	// of the election when Config.PreVote is true).
+	campaignElection CampaignType = "CampaignElection"
+	// campaignTransfer represents the type of leader transfer
+	campaignTransfer CampaignType = "CampaignTransfer"
+)
+
+// lockedRand is a small wrapper around rand.Rand to provide
+// synchronization. Only the methods needed by the code are exposed
+// (e.g. Intn).
+type lockedRand struct {
+	mu   sync.Mutex
+	rand *rand.Rand
+}
+
+func (r *lockedRand) Intn(n int) int {
+	r.mu.Lock()
+	v := r.rand.Intn(n)
+	r.mu.Unlock()
+	return v
+}
+
+var globalRand = &lockedRand{
+	rand: rand.New(rand.NewSource(time.Now().UnixNano())),
+}
+
+// CampaignType represents the type of campaigning
+// the reason we use the type of string instead of uint64
+// is because it's simpler to compare and fill in raft entries
+type CampaignType string
+
+// StateType represents the role of a node in a cluster.
+type StateType uint64
+
+var stmap = [...]string{
+	"StateFollower",
+	"StateCandidate",
+	"StateLeader",
+	"StatePreCandidate",
+}
+
+func (st StateType) String() string {
+	return stmap[uint64(st)]
+}
+
+// Config contains the parameters to start a raft.
+type Config struct {
+	// ID is the identity of the local raft. ID cannot be 0.
+	ID uint64
+
+	// peers contains the IDs of all nodes (including self) in the raft cluster. It
+	// should only be set when starting a new raft cluster. Restarting raft from
+	// previous configuration will panic if peers is set. peer is private and only
+	// used for testing right now.
+	peers []uint64
+
+	// learners contains the IDs of all leaner nodes (including self if the local node is a leaner) in the raft cluster.
+	// learners only receives entries from the leader node. It does not vote or promote itself.
+	learners []uint64
+
+	// ElectionTick is the number of Node.Tick invocations that must pass between
+	// elections. That is, if a follower does not receive any message from the
+	// leader of current term before ElectionTick has elapsed, it will become
+	// candidate and start an election. ElectionTick must be greater than
+	// HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
+	// unnecessary leader switching.
+	ElectionTick int
+	// HeartbeatTick is the number of Node.Tick invocations that must pass between
+	// heartbeats. That is, a leader sends heartbeat messages to maintain its
+	// leadership every HeartbeatTick ticks.
+	HeartbeatTick int
+
+	// Storage is the storage for raft. raft generates entries and states to be
+	// stored in storage. raft reads the persisted entries and states out of
+	// Storage when it needs. raft reads out the previous state and configuration
+	// out of storage when restarting.
+	Storage Storage
+	// Applied is the last applied index. It should only be set when restarting
+	// raft. raft will not return entries to the application smaller or equal to
+	// Applied. If Applied is unset when restarting, raft might return previous
+	// applied entries. This is a very application dependent configuration.
+	Applied uint64
+
+	// MaxSizePerMsg limits the max size of each append message. Smaller value
+	// lowers the raft recovery cost(initial probing and message lost during normal
+	// operation). On the other side, it might affect the throughput during normal
+	// replication. Note: math.MaxUint64 for unlimited, 0 for at most one entry per
+	// message.
+	MaxSizePerMsg uint64
+	// MaxInflightMsgs limits the max number of in-flight append messages during
+	// optimistic replication phase. The application transportation layer usually
+	// has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
+	// overflowing that sending buffer. TODO (xiangli): feedback to application to
+	// limit the proposal rate?
+	MaxInflightMsgs int
+
+	// CheckQuorum specifies if the leader should check quorum activity. Leader
+	// steps down when quorum is not active for an electionTimeout.
+	CheckQuorum bool
+
+	// PreVote enables the Pre-Vote algorithm described in raft thesis section
+	// 9.6. This prevents disruption when a node that has been partitioned away
+	// rejoins the cluster.
+	PreVote bool
+
+	// ReadOnlyOption specifies how the read only request is processed.
+	//
+	// ReadOnlySafe guarantees the linearizability of the read only request by
+	// communicating with the quorum. It is the default and suggested option.
+	//
+	// ReadOnlyLeaseBased ensures linearizability of the read only request by
+	// relying on the leader lease. It can be affected by clock drift.
+	// If the clock drift is unbounded, leader might keep the lease longer than it
+	// should (clock can move backward/pause without any bound). ReadIndex is not safe
+	// in that case.
+	// CheckQuorum MUST be enabled if ReadOnlyOption is ReadOnlyLeaseBased.
+	ReadOnlyOption ReadOnlyOption
+
+	// Logger is the logger used for raft log. For multinode which can host
+	// multiple raft group, each raft group can have its own logger
+	Logger Logger
+
+	// DisableProposalForwarding set to true means that followers will drop
+	// proposals, rather than forwarding them to the leader. One use case for
+	// this feature would be in a situation where the Raft leader is used to
+	// compute the data of a proposal, for example, adding a timestamp from a
+	// hybrid logical clock to data in a monotonically increasing way. Forwarding
+	// should be disabled to prevent a follower with an innaccurate hybrid
+	// logical clock from assigning the timestamp and then forwarding the data
+	// to the leader.
+	DisableProposalForwarding bool
+}
+
+func (c *Config) validate() error {
+	if c.ID == None {
+		return errors.New("cannot use none as id")
+	}
+
+	if c.HeartbeatTick <= 0 {
+		return errors.New("heartbeat tick must be greater than 0")
+	}
+
+	if c.ElectionTick <= c.HeartbeatTick {
+		return errors.New("election tick must be greater than heartbeat tick")
+	}
+
+	if c.Storage == nil {
+		return errors.New("storage cannot be nil")
+	}
+
+	if c.MaxInflightMsgs <= 0 {
+		return errors.New("max inflight messages must be greater than 0")
+	}
+
+	if c.Logger == nil {
+		c.Logger = raftLogger
+	}
+
+	if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum {
+		return errors.New("CheckQuorum must be enabled when ReadOnlyOption is ReadOnlyLeaseBased")
+	}
+
+	return nil
+}
+
+type raft struct {
+	id uint64
+
+	Term uint64
+	Vote uint64
+
+	readStates []ReadState
+
+	// the log
+	raftLog *raftLog
+
+	maxInflight int
+	maxMsgSize  uint64
+	prs         map[uint64]*Progress
+	learnerPrs  map[uint64]*Progress
+
+	state StateType
+
+	// isLearner is true if the local raft node is a learner.
+	isLearner bool
+
+	votes map[uint64]bool
+
+	msgs []pb.Message
+
+	// the leader id
+	lead uint64
+	// leadTransferee is id of the leader transfer target when its value is not zero.
+	// Follow the procedure defined in raft thesis 3.10.
+	leadTransferee uint64
+	// New configuration is ignored if there exists unapplied configuration.
+	pendingConf bool
+
+	readOnly *readOnly
+
+	// number of ticks since it reached last electionTimeout when it is leader
+	// or candidate.
+	// number of ticks since it reached last electionTimeout or received a
+	// valid message from current leader when it is a follower.
+	electionElapsed int
+
+	// number of ticks since it reached last heartbeatTimeout.
+	// only leader keeps heartbeatElapsed.
+	heartbeatElapsed int
+
+	checkQuorum bool
+	preVote     bool
+
+	heartbeatTimeout int
+	electionTimeout  int
+	// randomizedElectionTimeout is a random number between
+	// [electiontimeout, 2 * electiontimeout - 1]. It gets reset
+	// when raft changes its state to follower or candidate.
+	randomizedElectionTimeout int
+	disableProposalForwarding bool
+
+	tick func()
+	step stepFunc
+
+	logger Logger
+}
+
+func newRaft(c *Config) *raft {
+	if err := c.validate(); err != nil {
+		panic(err.Error())
+	}
+	raftlog := newLog(c.Storage, c.Logger)
+	hs, cs, err := c.Storage.InitialState()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	peers := c.peers
+	learners := c.learners
+	if len(cs.Nodes) > 0 || len(cs.Learners) > 0 {
+		if len(peers) > 0 || len(learners) > 0 {
+			// TODO(bdarnell): the peers argument is always nil except in
+			// tests; the argument should be removed and these tests should be
+			// updated to specify their nodes through a snapshot.
+			panic("cannot specify both newRaft(peers, learners) and ConfState.(Nodes, Learners)")
+		}
+		peers = cs.Nodes
+		learners = cs.Learners
+	}
+	r := &raft{
+		id:                        c.ID,
+		lead:                      None,
+		isLearner:                 false,
+		raftLog:                   raftlog,
+		maxMsgSize:                c.MaxSizePerMsg,
+		maxInflight:               c.MaxInflightMsgs,
+		prs:                       make(map[uint64]*Progress),
+		learnerPrs:                make(map[uint64]*Progress),
+		electionTimeout:           c.ElectionTick,
+		heartbeatTimeout:          c.HeartbeatTick,
+		logger:                    c.Logger,
+		checkQuorum:               c.CheckQuorum,
+		preVote:                   c.PreVote,
+		readOnly:                  newReadOnly(c.ReadOnlyOption),
+		disableProposalForwarding: c.DisableProposalForwarding,
+	}
+	for _, p := range peers {
+		r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
+	}
+	for _, p := range learners {
+		if _, ok := r.prs[p]; ok {
+			panic(fmt.Sprintf("node %x is in both learner and peer list", p))
+		}
+		r.learnerPrs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight), IsLearner: true}
+		if r.id == p {
+			r.isLearner = true
+		}
+	}
+
+	if !isHardStateEqual(hs, emptyState) {
+		r.loadState(hs)
+	}
+	if c.Applied > 0 {
+		raftlog.appliedTo(c.Applied)
+	}
+	r.becomeFollower(r.Term, None)
+
+	var nodesStrs []string
+	for _, n := range r.nodes() {
+		nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
+	}
+
+	r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
+		r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
+	return r
+}
+
+func (r *raft) hasLeader() bool { return r.lead != None }
+
+func (r *raft) softState() *SoftState { return &SoftState{Lead: r.lead, RaftState: r.state} }
+
+func (r *raft) hardState() pb.HardState {
+	return pb.HardState{
+		Term:   r.Term,
+		Vote:   r.Vote,
+		Commit: r.raftLog.committed,
+	}
+}
+
+func (r *raft) quorum() int { return len(r.prs)/2 + 1 }
+
+func (r *raft) nodes() []uint64 {
+	nodes := make([]uint64, 0, len(r.prs)+len(r.learnerPrs))
+	for id := range r.prs {
+		nodes = append(nodes, id)
+	}
+	for id := range r.learnerPrs {
+		nodes = append(nodes, id)
+	}
+	sort.Sort(uint64Slice(nodes))
+	return nodes
+}
+
+// send persists state to stable storage and then sends to its mailbox.
+func (r *raft) send(m pb.Message) {
+	m.From = r.id
+	if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp {
+		if m.Term == 0 {
+			// All {pre-,}campaign messages need to have the term set when
+			// sending.
+			// - MsgVote: m.Term is the term the node is campaigning for,
+			//   non-zero as we increment the term when campaigning.
+			// - MsgVoteResp: m.Term is the new r.Term if the MsgVote was
+			//   granted, non-zero for the same reason MsgVote is
+			// - MsgPreVote: m.Term is the term the node will campaign,
+			//   non-zero as we use m.Term to indicate the next term we'll be
+			//   campaigning for
+			// - MsgPreVoteResp: m.Term is the term received in the original
+			//   MsgPreVote if the pre-vote was granted, non-zero for the
+			//   same reasons MsgPreVote is
+			panic(fmt.Sprintf("term should be set when sending %s", m.Type))
+		}
+	} else {
+		if m.Term != 0 {
+			panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
+		}
+		// do not attach term to MsgProp, MsgReadIndex
+		// proposals are a way to forward to the leader and
+		// should be treated as local message.
+		// MsgReadIndex is also forwarded to leader.
+		if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
+			m.Term = r.Term
+		}
+	}
+	r.msgs = append(r.msgs, m)
+}
+
+func (r *raft) getProgress(id uint64) *Progress {
+	if pr, ok := r.prs[id]; ok {
+		return pr
+	}
+
+	return r.learnerPrs[id]
+}
+
+// sendAppend sends RPC, with entries to the given peer.
+func (r *raft) sendAppend(to uint64) {
+	pr := r.getProgress(to)
+	if pr.IsPaused() {
+		return
+	}
+	m := pb.Message{}
+	m.To = to
+
+	term, errt := r.raftLog.term(pr.Next - 1)
+	ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
+
+	if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
+		if !pr.RecentActive {
+			r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
+			return
+		}
+
+		m.Type = pb.MsgSnap
+		snapshot, err := r.raftLog.snapshot()
+		if err != nil {
+			if err == ErrSnapshotTemporarilyUnavailable {
+				r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
+				return
+			}
+			panic(err) // TODO(bdarnell)
+		}
+		if IsEmptySnap(snapshot) {
+			panic("need non-empty snapshot")
+		}
+		m.Snapshot = snapshot
+		sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
+		r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
+			r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
+		pr.becomeSnapshot(sindex)
+		r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
+	} else {
+		m.Type = pb.MsgApp
+		m.Index = pr.Next - 1
+		m.LogTerm = term
+		m.Entries = ents
+		m.Commit = r.raftLog.committed
+		if n := len(m.Entries); n != 0 {
+			switch pr.State {
+			// optimistically increase the next when in ProgressStateReplicate
+			case ProgressStateReplicate:
+				last := m.Entries[n-1].Index
+				pr.optimisticUpdate(last)
+				pr.ins.add(last)
+			case ProgressStateProbe:
+				pr.pause()
+			default:
+				r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
+			}
+		}
+	}
+	r.send(m)
+}
+
+// sendHeartbeat sends an empty MsgApp
+func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
+	// Attach the commit as min(to.matched, r.committed).
+	// When the leader sends out heartbeat message,
+	// the receiver(follower) might not be matched with the leader
+	// or it might not have all the committed entries.
+	// The leader MUST NOT forward the follower's commit to
+	// an unmatched index.
+	commit := min(r.getProgress(to).Match, r.raftLog.committed)
+	m := pb.Message{
+		To:      to,
+		Type:    pb.MsgHeartbeat,
+		Commit:  commit,
+		Context: ctx,
+	}
+
+	r.send(m)
+}
+
+func (r *raft) forEachProgress(f func(id uint64, pr *Progress)) {
+	for id, pr := range r.prs {
+		f(id, pr)
+	}
+
+	for id, pr := range r.learnerPrs {
+		f(id, pr)
+	}
+}
+
+// bcastAppend sends RPC, with entries to all peers that are not up-to-date
+// according to the progress recorded in r.prs.
+func (r *raft) bcastAppend() {
+	r.forEachProgress(func(id uint64, _ *Progress) {
+		if id == r.id {
+			return
+		}
+
+		r.sendAppend(id)
+	})
+}
+
+// bcastHeartbeat sends RPC, without entries to all the peers.
+func (r *raft) bcastHeartbeat() {
+	lastCtx := r.readOnly.lastPendingRequestCtx()
+	if len(lastCtx) == 0 {
+		r.bcastHeartbeatWithCtx(nil)
+	} else {
+		r.bcastHeartbeatWithCtx([]byte(lastCtx))
+	}
+}
+
+func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
+	r.forEachProgress(func(id uint64, _ *Progress) {
+		if id == r.id {
+			return
+		}
+		r.sendHeartbeat(id, ctx)
+	})
+}
+
+// maybeCommit attempts to advance the commit index. Returns true if
+// the commit index changed (in which case the caller should call
+// r.bcastAppend).
+func (r *raft) maybeCommit() bool {
+	// TODO(bmizerany): optimize.. Currently naive
+	mis := make(uint64Slice, 0, len(r.prs))
+	for _, p := range r.prs {
+		mis = append(mis, p.Match)
+	}
+	sort.Sort(sort.Reverse(mis))
+	mci := mis[r.quorum()-1]
+	return r.raftLog.maybeCommit(mci, r.Term)
+}
+
+func (r *raft) reset(term uint64) {
+	if r.Term != term {
+		r.Term = term
+		r.Vote = None
+	}
+	r.lead = None
+
+	r.electionElapsed = 0
+	r.heartbeatElapsed = 0
+	r.resetRandomizedElectionTimeout()
+
+	r.abortLeaderTransfer()
+
+	r.votes = make(map[uint64]bool)
+	r.forEachProgress(func(id uint64, pr *Progress) {
+		*pr = Progress{Next: r.raftLog.lastIndex() + 1, ins: newInflights(r.maxInflight), IsLearner: pr.IsLearner}
+		if id == r.id {
+			pr.Match = r.raftLog.lastIndex()
+		}
+	})
+
+	r.pendingConf = false
+	r.readOnly = newReadOnly(r.readOnly.option)
+}
+
+func (r *raft) appendEntry(es ...pb.Entry) {
+	li := r.raftLog.lastIndex()
+	for i := range es {
+		es[i].Term = r.Term
+		es[i].Index = li + 1 + uint64(i)
+	}
+	r.raftLog.append(es...)
+	r.getProgress(r.id).maybeUpdate(r.raftLog.lastIndex())
+	// Regardless of maybeCommit's return, our caller will call bcastAppend.
+	r.maybeCommit()
+}
+
+// tickElection is run by followers and candidates after r.electionTimeout.
+func (r *raft) tickElection() {
+	r.electionElapsed++
+
+	if r.promotable() && r.pastElectionTimeout() {
+		r.electionElapsed = 0
+		r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
+	}
+}
+
+// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
+func (r *raft) tickHeartbeat() {
+	r.heartbeatElapsed++
+	r.electionElapsed++
+
+	if r.electionElapsed >= r.electionTimeout {
+		r.electionElapsed = 0
+		if r.checkQuorum {
+			r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
+		}
+		// If current leader cannot transfer leadership in electionTimeout, it becomes leader again.
+		if r.state == StateLeader && r.leadTransferee != None {
+			r.abortLeaderTransfer()
+		}
+	}
+
+	if r.state != StateLeader {
+		return
+	}
+
+	if r.heartbeatElapsed >= r.heartbeatTimeout {
+		r.heartbeatElapsed = 0
+		r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
+	}
+}
+
+func (r *raft) becomeFollower(term uint64, lead uint64) {
+	r.step = stepFollower
+	r.reset(term)
+	r.tick = r.tickElection
+	r.lead = lead
+	r.state = StateFollower
+	r.logger.Infof("%x became follower at term %d", r.id, r.Term)
+}
+
+func (r *raft) becomeCandidate() {
+	// TODO(xiangli) remove the panic when the raft implementation is stable
+	if r.state == StateLeader {
+		panic("invalid transition [leader -> candidate]")
+	}
+	r.step = stepCandidate
+	r.reset(r.Term + 1)
+	r.tick = r.tickElection
+	r.Vote = r.id
+	r.state = StateCandidate
+	r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
+}
+
+func (r *raft) becomePreCandidate() {
+	// TODO(xiangli) remove the panic when the raft implementation is stable
+	if r.state == StateLeader {
+		panic("invalid transition [leader -> pre-candidate]")
+	}
+	// Becoming a pre-candidate changes our step functions and state,
+	// but doesn't change anything else. In particular it does not increase
+	// r.Term or change r.Vote.
+	r.step = stepCandidate
+	r.votes = make(map[uint64]bool)
+	r.tick = r.tickElection
+	r.lead = None
+	r.state = StatePreCandidate
+	r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
+}
+
+func (r *raft) becomeLeader() {
+	// TODO(xiangli) remove the panic when the raft implementation is stable
+	if r.state == StateFollower {
+		panic("invalid transition [follower -> leader]")
+	}
+	r.step = stepLeader
+	r.reset(r.Term)
+	r.tick = r.tickHeartbeat
+	r.lead = r.id
+	r.state = StateLeader
+	ents, err := r.raftLog.entries(r.raftLog.committed+1, noLimit)
+	if err != nil {
+		r.logger.Panicf("unexpected error getting uncommitted entries (%v)", err)
+	}
+
+	nconf := numOfPendingConf(ents)
+	if nconf > 1 {
+		panic("unexpected multiple uncommitted config entry")
+	}
+	if nconf == 1 {
+		r.pendingConf = true
+	}
+
+	r.appendEntry(pb.Entry{Data: nil})
+	r.logger.Infof("%x became leader at term %d", r.id, r.Term)
+}
+
+func (r *raft) campaign(t CampaignType) {
+	var term uint64
+	var voteMsg pb.MessageType
+	if t == campaignPreElection {
+		r.becomePreCandidate()
+		voteMsg = pb.MsgPreVote
+		// PreVote RPCs are sent for the next term before we've incremented r.Term.
+		term = r.Term + 1
+	} else {
+		r.becomeCandidate()
+		voteMsg = pb.MsgVote
+		term = r.Term
+	}
+	if r.quorum() == r.poll(r.id, voteRespMsgType(voteMsg), true) {
+		// We won the election after voting for ourselves (which must mean that
+		// this is a single-node cluster). Advance to the next state.
+		if t == campaignPreElection {
+			r.campaign(campaignElection)
+		} else {
+			r.becomeLeader()
+		}
+		return
+	}
+	for id := range r.prs {
+		if id == r.id {
+			continue
+		}
+		r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
+			r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
+
+		var ctx []byte
+		if t == campaignTransfer {
+			ctx = []byte(t)
+		}
+		r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
+	}
+}
+
+func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int) {
+	if v {
+		r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
+	} else {
+		r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
+	}
+	if _, ok := r.votes[id]; !ok {
+		r.votes[id] = v
+	}
+	for _, vv := range r.votes {
+		if vv {
+			granted++
+		}
+	}
+	return granted
+}
+
+func (r *raft) Step(m pb.Message) error {
+	// Handle the message term, which may result in our stepping down to a follower.
+	switch {
+	case m.Term == 0:
+		// local message
+	case m.Term > r.Term:
+		if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
+			force := bytes.Equal(m.Context, []byte(campaignTransfer))
+			inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
+			if !force && inLease {
+				// If a server receives a RequestVote request within the minimum election timeout
+				// of hearing from a current leader, it does not update its term or grant its vote
+				r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)",
+					r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
+				return nil
+			}
+		}
+		switch {
+		case m.Type == pb.MsgPreVote:
+			// Never change our term in response to a PreVote
+		case m.Type == pb.MsgPreVoteResp && !m.Reject:
+			// We send pre-vote requests with a term in our future. If the
+			// pre-vote is granted, we will increment our term when we get a
+			// quorum. If it is not, the term comes from the node that
+			// rejected our vote so we should become a follower at the new
+			// term.
+		default:
+			r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
+				r.id, r.Term, m.Type, m.From, m.Term)
+			if m.Type == pb.MsgApp || m.Type == pb.MsgHeartbeat || m.Type == pb.MsgSnap {
+				r.becomeFollower(m.Term, m.From)
+			} else {
+				r.becomeFollower(m.Term, None)
+			}
+		}
+
+	case m.Term < r.Term:
+		if r.checkQuorum && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
+			// We have received messages from a leader at a lower term. It is possible
+			// that these messages were simply delayed in the network, but this could
+			// also mean that this node has advanced its term number during a network
+			// partition, and it is now unable to either win an election or to rejoin
+			// the majority on the old term. If checkQuorum is false, this will be
+			// handled by incrementing term numbers in response to MsgVote with a
+			// higher term, but if checkQuorum is true we may not advance the term on
+			// MsgVote and must generate other messages to advance the term. The net
+			// result of these two features is to minimize the disruption caused by
+			// nodes that have been removed from the cluster's configuration: a
+			// removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
+			// but it will not receive MsgApp or MsgHeartbeat, so it will not create
+			// disruptive term increases
+			r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
+		} else {
+			// ignore other cases
+			r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
+				r.id, r.Term, m.Type, m.From, m.Term)
+		}
+		return nil
+	}
+
+	switch m.Type {
+	case pb.MsgHup:
+		if r.state != StateLeader {
+			ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
+			if err != nil {
+				r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
+			}
+			if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
+				r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
+				return nil
+			}
+
+			r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
+			if r.preVote {
+				r.campaign(campaignPreElection)
+			} else {
+				r.campaign(campaignElection)
+			}
+		} else {
+			r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
+		}
+
+	case pb.MsgVote, pb.MsgPreVote:
+		if r.isLearner {
+			// TODO: learner may need to vote, in case of node down when confchange.
+			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: learner can not vote",
+				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
+			return nil
+		}
+		// The m.Term > r.Term clause is for MsgPreVote. For MsgVote m.Term should
+		// always equal r.Term.
+		if (r.Vote == None || m.Term > r.Term || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
+			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
+				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
+			// When responding to Msg{Pre,}Vote messages we include the term
+			// from the message, not the local term. To see why consider the
+			// case where a single node was previously partitioned away and
+			// it's local term is now of date. If we include the local term
+			// (recall that for pre-votes we don't update the local term), the
+			// (pre-)campaigning node on the other end will proceed to ignore
+			// the message (it ignores all out of date messages).
+			// The term in the original message and current local term are the
+			// same in the case of regular votes, but different for pre-votes.
+			r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)})
+			if m.Type == pb.MsgVote {
+				// Only record real votes.
+				r.electionElapsed = 0
+				r.Vote = m.From
+			}
+		} else {
+			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
+				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
+			r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
+		}
+
+	default:
+		r.step(r, m)
+	}
+	return nil
+}
+
+type stepFunc func(r *raft, m pb.Message)
+
+func stepLeader(r *raft, m pb.Message) {
+	// These message types do not require any progress for m.From.
+	switch m.Type {
+	case pb.MsgBeat:
+		r.bcastHeartbeat()
+		return
+	case pb.MsgCheckQuorum:
+		if !r.checkQuorumActive() {
+			r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
+			r.becomeFollower(r.Term, None)
+		}
+		return
+	case pb.MsgProp:
+		if len(m.Entries) == 0 {
+			r.logger.Panicf("%x stepped empty MsgProp", r.id)
+		}
+		if _, ok := r.prs[r.id]; !ok {
+			// If we are not currently a member of the range (i.e. this node
+			// was removed from the configuration while serving as leader),
+			// drop any new proposals.
+			return
+		}
+		if r.leadTransferee != None {
+			r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
+			return
+		}
+
+		for i, e := range m.Entries {
+			if e.Type == pb.EntryConfChange {
+				if r.pendingConf {
+					r.logger.Infof("propose conf %s ignored since pending unapplied configuration", e.String())
+					m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
+				}
+				r.pendingConf = true
+			}
+		}
+		r.appendEntry(m.Entries...)
+		r.bcastAppend()
+		return
+	case pb.MsgReadIndex:
+		if r.quorum() > 1 {
+			if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
+				// Reject read only request when this leader has not committed any log entry at its term.
+				return
+			}
+
+			// thinking: use an interally defined context instead of the user given context.
+			// We can express this in terms of the term and index instead of a user-supplied value.
+			// This would allow multiple reads to piggyback on the same message.
+			switch r.readOnly.option {
+			case ReadOnlySafe:
+				r.readOnly.addRequest(r.raftLog.committed, m)
+				r.bcastHeartbeatWithCtx(m.Entries[0].Data)
+			case ReadOnlyLeaseBased:
+				ri := r.raftLog.committed
+				if m.From == None || m.From == r.id { // from local member
+					r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
+				} else {
+					r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
+				}
+			}
+		} else {
+			r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
+		}
+
+		return
+	}
+
+	// All other message types require a progress for m.From (pr).
+	pr := r.getProgress(m.From)
+	if pr == nil {
+		r.logger.Debugf("%x no progress available for %x", r.id, m.From)
+		return
+	}
+	switch m.Type {
+	case pb.MsgAppResp:
+		pr.RecentActive = true
+
+		if m.Reject {
+			r.logger.Debugf("%x received msgApp rejection(lastindex: %d) from %x for index %d",
+				r.id, m.RejectHint, m.From, m.Index)
+			if pr.maybeDecrTo(m.Index, m.RejectHint) {
+				r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
+				if pr.State == ProgressStateReplicate {
+					pr.becomeProbe()
+				}
+				r.sendAppend(m.From)
+			}
+		} else {
+			oldPaused := pr.IsPaused()
+			if pr.maybeUpdate(m.Index) {
+				switch {
+				case pr.State == ProgressStateProbe:
+					pr.becomeReplicate()
+				case pr.State == ProgressStateSnapshot && pr.needSnapshotAbort():
+					r.logger.Debugf("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+					pr.becomeProbe()
+				case pr.State == ProgressStateReplicate:
+					pr.ins.freeTo(m.Index)
+				}
+
+				if r.maybeCommit() {
+					r.bcastAppend()
+				} else if oldPaused {
+					// update() reset the wait state on this node. If we had delayed sending
+					// an update before, send it now.
+					r.sendAppend(m.From)
+				}
+				// Transfer leadership is in progress.
+				if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
+					r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
+					r.sendTimeoutNow(m.From)
+				}
+			}
+		}
+	case pb.MsgHeartbeatResp:
+		pr.RecentActive = true
+		pr.resume()
+
+		// free one slot for the full inflights window to allow progress.
+		if pr.State == ProgressStateReplicate && pr.ins.full() {
+			pr.ins.freeFirstOne()
+		}
+		if pr.Match < r.raftLog.lastIndex() {
+			r.sendAppend(m.From)
+		}
+
+		if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
+			return
+		}
+
+		ackCount := r.readOnly.recvAck(m)
+		if ackCount < r.quorum() {
+			return
+		}
+
+		rss := r.readOnly.advance(m)
+		for _, rs := range rss {
+			req := rs.req
+			if req.From == None || req.From == r.id { // from local member
+				r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
+			} else {
+				r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
+			}
+		}
+	case pb.MsgSnapStatus:
+		if pr.State != ProgressStateSnapshot {
+			return
+		}
+		if !m.Reject {
+			pr.becomeProbe()
+			r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+		} else {
+			pr.snapshotFailure()
+			pr.becomeProbe()
+			r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+		}
+		// If snapshot finish, wait for the msgAppResp from the remote node before sending
+		// out the next msgApp.
+		// If snapshot failure, wait for a heartbeat interval before next try
+		pr.pause()
+	case pb.MsgUnreachable:
+		// During optimistic replication, if the remote becomes unreachable,
+		// there is huge probability that a MsgApp is lost.
+		if pr.State == ProgressStateReplicate {
+			pr.becomeProbe()
+		}
+		r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
+	case pb.MsgTransferLeader:
+		if pr.IsLearner {
+			r.logger.Debugf("%x is learner. Ignored transferring leadership", r.id)
+			return
+		}
+		leadTransferee := m.From
+		lastLeadTransferee := r.leadTransferee
+		if lastLeadTransferee != None {
+			if lastLeadTransferee == leadTransferee {
+				r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
+					r.id, r.Term, leadTransferee, leadTransferee)
+				return
+			}
+			r.abortLeaderTransfer()
+			r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
+		}
+		if leadTransferee == r.id {
+			r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
+			return
+		}
+		// Transfer leadership to third party.
+		r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
+		// Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
+		r.electionElapsed = 0
+		r.leadTransferee = leadTransferee
+		if pr.Match == r.raftLog.lastIndex() {
+			r.sendTimeoutNow(leadTransferee)
+			r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
+		} else {
+			r.sendAppend(leadTransferee)
+		}
+	}
+}
+
+// stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
+// whether they respond to MsgVoteResp or MsgPreVoteResp.
+func stepCandidate(r *raft, m pb.Message) {
+	// Only handle vote responses corresponding to our candidacy (while in
+	// StateCandidate, we may get stale MsgPreVoteResp messages in this term from
+	// our pre-candidate state).
+	var myVoteRespType pb.MessageType
+	if r.state == StatePreCandidate {
+		myVoteRespType = pb.MsgPreVoteResp
+	} else {
+		myVoteRespType = pb.MsgVoteResp
+	}
+	switch m.Type {
+	case pb.MsgProp:
+		r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
+		return
+	case pb.MsgApp:
+		r.becomeFollower(r.Term, m.From)
+		r.handleAppendEntries(m)
+	case pb.MsgHeartbeat:
+		r.becomeFollower(r.Term, m.From)
+		r.handleHeartbeat(m)
+	case pb.MsgSnap:
+		r.becomeFollower(m.Term, m.From)
+		r.handleSnapshot(m)
+	case myVoteRespType:
+		gr := r.poll(m.From, m.Type, !m.Reject)
+		r.logger.Infof("%x [quorum:%d] has received %d %s votes and %d vote rejections", r.id, r.quorum(), gr, m.Type, len(r.votes)-gr)
+		switch r.quorum() {
+		case gr:
+			if r.state == StatePreCandidate {
+				r.campaign(campaignElection)
+			} else {
+				r.becomeLeader()
+				r.bcastAppend()
+			}
+		case len(r.votes) - gr:
+			r.becomeFollower(r.Term, None)
+		}
+	case pb.MsgTimeoutNow:
+		r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
+	}
+}
+
+func stepFollower(r *raft, m pb.Message) {
+	switch m.Type {
+	case pb.MsgProp:
+		if r.lead == None {
+			r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
+			return
+		} else if r.disableProposalForwarding {
+			r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
+			return
+		}
+		m.To = r.lead
+		r.send(m)
+	case pb.MsgApp:
+		r.electionElapsed = 0
+		r.lead = m.From
+		r.handleAppendEntries(m)
+	case pb.MsgHeartbeat:
+		r.electionElapsed = 0
+		r.lead = m.From
+		r.handleHeartbeat(m)
+	case pb.MsgSnap:
+		r.electionElapsed = 0
+		r.lead = m.From
+		r.handleSnapshot(m)
+	case pb.MsgTransferLeader:
+		if r.lead == None {
+			r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
+			return
+		}
+		m.To = r.lead
+		r.send(m)
+	case pb.MsgTimeoutNow:
+		if r.promotable() {
+			r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
+			// Leadership transfers never use pre-vote even if r.preVote is true; we
+			// know we are not recovering from a partition so there is no need for the
+			// extra round trip.
+			r.campaign(campaignTransfer)
+		} else {
+			r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
+		}
+	case pb.MsgReadIndex:
+		if r.lead == None {
+			r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
+			return
+		}
+		m.To = r.lead
+		r.send(m)
+	case pb.MsgReadIndexResp:
+		if len(m.Entries) != 1 {
+			r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
+			return
+		}
+		r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
+	}
+}
+
+func (r *raft) handleAppendEntries(m pb.Message) {
+	if m.Index < r.raftLog.committed {
+		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
+		return
+	}
+
+	if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
+		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
+	} else {
+		r.logger.Debugf("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
+			r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
+		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
+	}
+}
+
+func (r *raft) handleHeartbeat(m pb.Message) {
+	r.raftLog.commitTo(m.Commit)
+	r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
+}
+
+func (r *raft) handleSnapshot(m pb.Message) {
+	sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
+	if r.restore(m.Snapshot) {
+		r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
+			r.id, r.raftLog.committed, sindex, sterm)
+		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
+	} else {
+		r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
+			r.id, r.raftLog.committed, sindex, sterm)
+		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
+	}
+}
+
+// restore recovers the state machine from a snapshot. It restores the log and the
+// configuration of state machine.
+func (r *raft) restore(s pb.Snapshot) bool {
+	if s.Metadata.Index <= r.raftLog.committed {
+		return false
+	}
+	if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
+		r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
+			r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
+		r.raftLog.commitTo(s.Metadata.Index)
+		return false
+	}
+
+	// The normal peer can't become learner.
+	if !r.isLearner {
+		for _, id := range s.Metadata.ConfState.Learners {
+			if id == r.id {
+				r.logger.Errorf("%x can't become learner when restores snapshot [index: %d, term: %d]", r.id, s.Metadata.Index, s.Metadata.Term)
+				return false
+			}
+		}
+	}
+
+	r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
+		r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
+
+	r.raftLog.restore(s)
+	r.prs = make(map[uint64]*Progress)
+	r.learnerPrs = make(map[uint64]*Progress)
+	r.restoreNode(s.Metadata.ConfState.Nodes, false)
+	r.restoreNode(s.Metadata.ConfState.Learners, true)
+	return true
+}
+
+func (r *raft) restoreNode(nodes []uint64, isLearner bool) {
+	for _, n := range nodes {
+		match, next := uint64(0), r.raftLog.lastIndex()+1
+		if n == r.id {
+			match = next - 1
+			r.isLearner = isLearner
+		}
+		r.setProgress(n, match, next, isLearner)
+		r.logger.Infof("%x restored progress of %x [%s]", r.id, n, r.getProgress(n))
+	}
+}
+
+// promotable indicates whether state machine can be promoted to leader,
+// which is true when its own id is in progress list.
+func (r *raft) promotable() bool {
+	_, ok := r.prs[r.id]
+	return ok
+}
+
+func (r *raft) addNode(id uint64) {
+	r.addNodeOrLearnerNode(id, false)
+}
+
+func (r *raft) addLearner(id uint64) {
+	r.addNodeOrLearnerNode(id, true)
+}
+
+func (r *raft) addNodeOrLearnerNode(id uint64, isLearner bool) {
+	r.pendingConf = false
+	pr := r.getProgress(id)
+	if pr == nil {
+		r.setProgress(id, 0, r.raftLog.lastIndex()+1, isLearner)
+	} else {
+		if isLearner && !pr.IsLearner {
+			// can only change Learner to Voter
+			r.logger.Infof("%x ignored addLeaner: do not support changing %x from raft peer to learner.", r.id, id)
+			return
+		}
+
+		if isLearner == pr.IsLearner {
+			// Ignore any redundant addNode calls (which can happen because the
+			// initial bootstrapping entries are applied twice).
+			return
+		}
+
+		// change Learner to Voter, use origin Learner progress
+		delete(r.learnerPrs, id)
+		pr.IsLearner = false
+		r.prs[id] = pr
+	}
+
+	if r.id == id {
+		r.isLearner = isLearner
+	}
+
+	// When a node is first added, we should mark it as recently active.
+	// Otherwise, CheckQuorum may cause us to step down if it is invoked
+	// before the added node has a chance to communicate with us.
+	pr = r.getProgress(id)
+	pr.RecentActive = true
+}
+
+func (r *raft) removeNode(id uint64) {
+	r.delProgress(id)
+	r.pendingConf = false
+
+	// do not try to commit or abort transferring if there is no nodes in the cluster.
+	if len(r.prs) == 0 && len(r.learnerPrs) == 0 {
+		return
+	}
+
+	// The quorum size is now smaller, so see if any pending entries can
+	// be committed.
+	if r.maybeCommit() {
+		r.bcastAppend()
+	}
+	// If the removed node is the leadTransferee, then abort the leadership transferring.
+	if r.state == StateLeader && r.leadTransferee == id {
+		r.abortLeaderTransfer()
+	}
+}
+
+func (r *raft) resetPendingConf() { r.pendingConf = false }
+
+func (r *raft) setProgress(id, match, next uint64, isLearner bool) {
+	if !isLearner {
+		delete(r.learnerPrs, id)
+		r.prs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight)}
+		return
+	}
+
+	if _, ok := r.prs[id]; ok {
+		panic(fmt.Sprintf("%x unexpected changing from voter to learner for %x", r.id, id))
+	}
+	r.learnerPrs[id] = &Progress{Next: next, Match: match, ins: newInflights(r.maxInflight), IsLearner: true}
+}
+
+func (r *raft) delProgress(id uint64) {
+	delete(r.prs, id)
+	delete(r.learnerPrs, id)
+}
+
+func (r *raft) loadState(state pb.HardState) {
+	if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
+		r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
+	}
+	r.raftLog.committed = state.Commit
+	r.Term = state.Term
+	r.Vote = state.Vote
+}
+
+// pastElectionTimeout returns true iff r.electionElapsed is greater
+// than or equal to the randomized election timeout in
+// [electiontimeout, 2 * electiontimeout - 1].
+func (r *raft) pastElectionTimeout() bool {
+	return r.electionElapsed >= r.randomizedElectionTimeout
+}
+
+func (r *raft) resetRandomizedElectionTimeout() {
+	r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
+}
+
+// checkQuorumActive returns true if the quorum is active from
+// the view of the local raft state machine. Otherwise, it returns
+// false.
+// checkQuorumActive also resets all RecentActive to false.
+func (r *raft) checkQuorumActive() bool {
+	var act int
+
+	r.forEachProgress(func(id uint64, pr *Progress) {
+		if id == r.id { // self is always active
+			act++
+			return
+		}
+
+		if pr.RecentActive && !pr.IsLearner {
+			act++
+		}
+
+		pr.RecentActive = false
+	})
+
+	return act >= r.quorum()
+}
+
+func (r *raft) sendTimeoutNow(to uint64) {
+	r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
+}
+
+func (r *raft) abortLeaderTransfer() {
+	r.leadTransferee = None
+}
+
+func numOfPendingConf(ents []pb.Entry) int {
+	n := 0
+	for i := range ents {
+		if ents[i].Type == pb.EntryConfChange {
+			n++
+		}
+	}
+	return n
+}
diff --git a/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go b/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go
new file mode 100644
index 0000000..753bd84
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/raftpb/raft.pb.go
@@ -0,0 +1,2365 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: raft.proto
+
+package raftpb
+
+import (
+	fmt "fmt"
+	io "io"
+	math "math"
+	math_bits "math/bits"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+	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.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type EntryType int32
+
+const (
+	EntryNormal     EntryType = 0
+	EntryConfChange EntryType = 1
+)
+
+var EntryType_name = map[int32]string{
+	0: "EntryNormal",
+	1: "EntryConfChange",
+}
+
+var EntryType_value = map[string]int32{
+	"EntryNormal":     0,
+	"EntryConfChange": 1,
+}
+
+func (x EntryType) Enum() *EntryType {
+	p := new(EntryType)
+	*p = x
+	return p
+}
+
+func (x EntryType) String() string {
+	return proto.EnumName(EntryType_name, int32(x))
+}
+
+func (x *EntryType) UnmarshalJSON(data []byte) error {
+	value, err := proto.UnmarshalJSONEnum(EntryType_value, data, "EntryType")
+	if err != nil {
+		return err
+	}
+	*x = EntryType(value)
+	return nil
+}
+
+func (EntryType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{0}
+}
+
+type MessageType int32
+
+const (
+	MsgHup            MessageType = 0
+	MsgBeat           MessageType = 1
+	MsgProp           MessageType = 2
+	MsgApp            MessageType = 3
+	MsgAppResp        MessageType = 4
+	MsgVote           MessageType = 5
+	MsgVoteResp       MessageType = 6
+	MsgSnap           MessageType = 7
+	MsgHeartbeat      MessageType = 8
+	MsgHeartbeatResp  MessageType = 9
+	MsgUnreachable    MessageType = 10
+	MsgSnapStatus     MessageType = 11
+	MsgCheckQuorum    MessageType = 12
+	MsgTransferLeader MessageType = 13
+	MsgTimeoutNow     MessageType = 14
+	MsgReadIndex      MessageType = 15
+	MsgReadIndexResp  MessageType = 16
+	MsgPreVote        MessageType = 17
+	MsgPreVoteResp    MessageType = 18
+)
+
+var MessageType_name = map[int32]string{
+	0:  "MsgHup",
+	1:  "MsgBeat",
+	2:  "MsgProp",
+	3:  "MsgApp",
+	4:  "MsgAppResp",
+	5:  "MsgVote",
+	6:  "MsgVoteResp",
+	7:  "MsgSnap",
+	8:  "MsgHeartbeat",
+	9:  "MsgHeartbeatResp",
+	10: "MsgUnreachable",
+	11: "MsgSnapStatus",
+	12: "MsgCheckQuorum",
+	13: "MsgTransferLeader",
+	14: "MsgTimeoutNow",
+	15: "MsgReadIndex",
+	16: "MsgReadIndexResp",
+	17: "MsgPreVote",
+	18: "MsgPreVoteResp",
+}
+
+var MessageType_value = map[string]int32{
+	"MsgHup":            0,
+	"MsgBeat":           1,
+	"MsgProp":           2,
+	"MsgApp":            3,
+	"MsgAppResp":        4,
+	"MsgVote":           5,
+	"MsgVoteResp":       6,
+	"MsgSnap":           7,
+	"MsgHeartbeat":      8,
+	"MsgHeartbeatResp":  9,
+	"MsgUnreachable":    10,
+	"MsgSnapStatus":     11,
+	"MsgCheckQuorum":    12,
+	"MsgTransferLeader": 13,
+	"MsgTimeoutNow":     14,
+	"MsgReadIndex":      15,
+	"MsgReadIndexResp":  16,
+	"MsgPreVote":        17,
+	"MsgPreVoteResp":    18,
+}
+
+func (x MessageType) Enum() *MessageType {
+	p := new(MessageType)
+	*p = x
+	return p
+}
+
+func (x MessageType) String() string {
+	return proto.EnumName(MessageType_name, int32(x))
+}
+
+func (x *MessageType) UnmarshalJSON(data []byte) error {
+	value, err := proto.UnmarshalJSONEnum(MessageType_value, data, "MessageType")
+	if err != nil {
+		return err
+	}
+	*x = MessageType(value)
+	return nil
+}
+
+func (MessageType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{1}
+}
+
+type ConfChangeType int32
+
+const (
+	ConfChangeAddNode        ConfChangeType = 0
+	ConfChangeRemoveNode     ConfChangeType = 1
+	ConfChangeUpdateNode     ConfChangeType = 2
+	ConfChangeAddLearnerNode ConfChangeType = 3
+)
+
+var ConfChangeType_name = map[int32]string{
+	0: "ConfChangeAddNode",
+	1: "ConfChangeRemoveNode",
+	2: "ConfChangeUpdateNode",
+	3: "ConfChangeAddLearnerNode",
+}
+
+var ConfChangeType_value = map[string]int32{
+	"ConfChangeAddNode":        0,
+	"ConfChangeRemoveNode":     1,
+	"ConfChangeUpdateNode":     2,
+	"ConfChangeAddLearnerNode": 3,
+}
+
+func (x ConfChangeType) Enum() *ConfChangeType {
+	p := new(ConfChangeType)
+	*p = x
+	return p
+}
+
+func (x ConfChangeType) String() string {
+	return proto.EnumName(ConfChangeType_name, int32(x))
+}
+
+func (x *ConfChangeType) UnmarshalJSON(data []byte) error {
+	value, err := proto.UnmarshalJSONEnum(ConfChangeType_value, data, "ConfChangeType")
+	if err != nil {
+		return err
+	}
+	*x = ConfChangeType(value)
+	return nil
+}
+
+func (ConfChangeType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{2}
+}
+
+type Entry struct {
+	Term                 uint64    `protobuf:"varint,2,opt,name=Term" json:"Term"`
+	Index                uint64    `protobuf:"varint,3,opt,name=Index" json:"Index"`
+	Type                 EntryType `protobuf:"varint,1,opt,name=Type,enum=raftpb.EntryType" json:"Type"`
+	Data                 []byte    `protobuf:"bytes,4,opt,name=Data" json:"Data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *Entry) Reset()         { *m = Entry{} }
+func (m *Entry) String() string { return proto.CompactTextString(m) }
+func (*Entry) ProtoMessage()    {}
+func (*Entry) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{0}
+}
+func (m *Entry) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Entry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Entry.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Entry) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Entry.Merge(m, src)
+}
+func (m *Entry) XXX_Size() int {
+	return m.Size()
+}
+func (m *Entry) XXX_DiscardUnknown() {
+	xxx_messageInfo_Entry.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Entry proto.InternalMessageInfo
+
+type SnapshotMetadata struct {
+	ConfState            ConfState `protobuf:"bytes,1,opt,name=conf_state,json=confState" json:"conf_state"`
+	Index                uint64    `protobuf:"varint,2,opt,name=index" json:"index"`
+	Term                 uint64    `protobuf:"varint,3,opt,name=term" json:"term"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *SnapshotMetadata) Reset()         { *m = SnapshotMetadata{} }
+func (m *SnapshotMetadata) String() string { return proto.CompactTextString(m) }
+func (*SnapshotMetadata) ProtoMessage()    {}
+func (*SnapshotMetadata) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{1}
+}
+func (m *SnapshotMetadata) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *SnapshotMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_SnapshotMetadata.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *SnapshotMetadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SnapshotMetadata.Merge(m, src)
+}
+func (m *SnapshotMetadata) XXX_Size() int {
+	return m.Size()
+}
+func (m *SnapshotMetadata) XXX_DiscardUnknown() {
+	xxx_messageInfo_SnapshotMetadata.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SnapshotMetadata proto.InternalMessageInfo
+
+type Snapshot struct {
+	Data                 []byte           `protobuf:"bytes,1,opt,name=data" json:"data,omitempty"`
+	Metadata             SnapshotMetadata `protobuf:"bytes,2,opt,name=metadata" json:"metadata"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `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 fileDescriptor_b042552c306ae59b, []int{2}
+}
+func (m *Snapshot) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Snapshot) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Snapshot.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Snapshot) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Snapshot.Merge(m, src)
+}
+func (m *Snapshot) XXX_Size() int {
+	return m.Size()
+}
+func (m *Snapshot) XXX_DiscardUnknown() {
+	xxx_messageInfo_Snapshot.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Snapshot proto.InternalMessageInfo
+
+type Message struct {
+	Type                 MessageType `protobuf:"varint,1,opt,name=type,enum=raftpb.MessageType" json:"type"`
+	To                   uint64      `protobuf:"varint,2,opt,name=to" json:"to"`
+	From                 uint64      `protobuf:"varint,3,opt,name=from" json:"from"`
+	Term                 uint64      `protobuf:"varint,4,opt,name=term" json:"term"`
+	LogTerm              uint64      `protobuf:"varint,5,opt,name=logTerm" json:"logTerm"`
+	Index                uint64      `protobuf:"varint,6,opt,name=index" json:"index"`
+	Entries              []Entry     `protobuf:"bytes,7,rep,name=entries" json:"entries"`
+	Commit               uint64      `protobuf:"varint,8,opt,name=commit" json:"commit"`
+	Snapshot             Snapshot    `protobuf:"bytes,9,opt,name=snapshot" json:"snapshot"`
+	Reject               bool        `protobuf:"varint,10,opt,name=reject" json:"reject"`
+	RejectHint           uint64      `protobuf:"varint,11,opt,name=rejectHint" json:"rejectHint"`
+	Context              []byte      `protobuf:"bytes,12,opt,name=context" json:"context,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *Message) Reset()         { *m = Message{} }
+func (m *Message) String() string { return proto.CompactTextString(m) }
+func (*Message) ProtoMessage()    {}
+func (*Message) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{3}
+}
+func (m *Message) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Message) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Message.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Message) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Message.Merge(m, src)
+}
+func (m *Message) XXX_Size() int {
+	return m.Size()
+}
+func (m *Message) XXX_DiscardUnknown() {
+	xxx_messageInfo_Message.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Message proto.InternalMessageInfo
+
+type HardState struct {
+	Term                 uint64   `protobuf:"varint,1,opt,name=term" json:"term"`
+	Vote                 uint64   `protobuf:"varint,2,opt,name=vote" json:"vote"`
+	Commit               uint64   `protobuf:"varint,3,opt,name=commit" json:"commit"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HardState) Reset()         { *m = HardState{} }
+func (m *HardState) String() string { return proto.CompactTextString(m) }
+func (*HardState) ProtoMessage()    {}
+func (*HardState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{4}
+}
+func (m *HardState) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *HardState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_HardState.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *HardState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HardState.Merge(m, src)
+}
+func (m *HardState) XXX_Size() int {
+	return m.Size()
+}
+func (m *HardState) XXX_DiscardUnknown() {
+	xxx_messageInfo_HardState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HardState proto.InternalMessageInfo
+
+type ConfState struct {
+	Nodes                []uint64 `protobuf:"varint,1,rep,name=nodes" json:"nodes,omitempty"`
+	Learners             []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ConfState) Reset()         { *m = ConfState{} }
+func (m *ConfState) String() string { return proto.CompactTextString(m) }
+func (*ConfState) ProtoMessage()    {}
+func (*ConfState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{5}
+}
+func (m *ConfState) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *ConfState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_ConfState.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *ConfState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ConfState.Merge(m, src)
+}
+func (m *ConfState) XXX_Size() int {
+	return m.Size()
+}
+func (m *ConfState) XXX_DiscardUnknown() {
+	xxx_messageInfo_ConfState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ConfState proto.InternalMessageInfo
+
+type ConfChange struct {
+	ID                   uint64         `protobuf:"varint,1,opt,name=ID" json:"ID"`
+	Type                 ConfChangeType `protobuf:"varint,2,opt,name=Type,enum=raftpb.ConfChangeType" json:"Type"`
+	NodeID               uint64         `protobuf:"varint,3,opt,name=NodeID" json:"NodeID"`
+	Context              []byte         `protobuf:"bytes,4,opt,name=Context" json:"Context,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *ConfChange) Reset()         { *m = ConfChange{} }
+func (m *ConfChange) String() string { return proto.CompactTextString(m) }
+func (*ConfChange) ProtoMessage()    {}
+func (*ConfChange) Descriptor() ([]byte, []int) {
+	return fileDescriptor_b042552c306ae59b, []int{6}
+}
+func (m *ConfChange) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *ConfChange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_ConfChange.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *ConfChange) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ConfChange.Merge(m, src)
+}
+func (m *ConfChange) XXX_Size() int {
+	return m.Size()
+}
+func (m *ConfChange) XXX_DiscardUnknown() {
+	xxx_messageInfo_ConfChange.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ConfChange proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterEnum("raftpb.EntryType", EntryType_name, EntryType_value)
+	proto.RegisterEnum("raftpb.MessageType", MessageType_name, MessageType_value)
+	proto.RegisterEnum("raftpb.ConfChangeType", ConfChangeType_name, ConfChangeType_value)
+	proto.RegisterType((*Entry)(nil), "raftpb.Entry")
+	proto.RegisterType((*SnapshotMetadata)(nil), "raftpb.SnapshotMetadata")
+	proto.RegisterType((*Snapshot)(nil), "raftpb.Snapshot")
+	proto.RegisterType((*Message)(nil), "raftpb.Message")
+	proto.RegisterType((*HardState)(nil), "raftpb.HardState")
+	proto.RegisterType((*ConfState)(nil), "raftpb.ConfState")
+	proto.RegisterType((*ConfChange)(nil), "raftpb.ConfChange")
+}
+
+func init() { proto.RegisterFile("raft.proto", fileDescriptor_b042552c306ae59b) }
+
+var fileDescriptor_b042552c306ae59b = []byte{
+	// 816 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x54, 0xcd, 0x6e, 0x23, 0x45,
+	0x10, 0xf6, 0x8c, 0xc7, 0x7f, 0x35, 0x8e, 0xd3, 0xa9, 0x35, 0xa8, 0x15, 0x45, 0xc6, 0xb2, 0x38,
+	0x58, 0x41, 0x1b, 0x20, 0x07, 0x0e, 0x48, 0x1c, 0x36, 0x09, 0x52, 0x22, 0xad, 0xa3, 0xc5, 0x9b,
+	0xe5, 0x80, 0x84, 0x50, 0xc7, 0x53, 0x9e, 0x18, 0x32, 0xd3, 0xa3, 0x9e, 0xf6, 0xb2, 0xb9, 0x20,
+	0x1e, 0x80, 0x07, 0xe0, 0xc2, 0xfb, 0xe4, 0xb8, 0x12, 0x77, 0xc4, 0x86, 0x17, 0x41, 0xdd, 0xd3,
+	0x63, 0xcf, 0x24, 0xb7, 0xae, 0xaf, 0x6a, 0xbe, 0xfa, 0xbe, 0xea, 0xea, 0x01, 0x50, 0x62, 0xa9,
+	0x8f, 0x32, 0x25, 0xb5, 0xc4, 0xb6, 0x39, 0x67, 0xd7, 0xfb, 0xc3, 0x58, 0xc6, 0xd2, 0x42, 0x9f,
+	0x9b, 0x53, 0x91, 0x9d, 0xfc, 0x06, 0xad, 0x6f, 0x53, 0xad, 0xee, 0x90, 0x43, 0x70, 0x45, 0x2a,
+	0xe1, 0xfe, 0xd8, 0x9b, 0x06, 0x27, 0xc1, 0xfd, 0x3f, 0x9f, 0x34, 0xe6, 0x16, 0xc1, 0x7d, 0x68,
+	0x5d, 0xa4, 0x11, 0xbd, 0xe3, 0xcd, 0x4a, 0xaa, 0x80, 0xf0, 0x33, 0x08, 0xae, 0xee, 0x32, 0xe2,
+	0xde, 0xd8, 0x9b, 0x0e, 0x8e, 0xf7, 0x8e, 0x8a, 0x5e, 0x47, 0x96, 0xd2, 0x24, 0x36, 0x44, 0x77,
+	0x19, 0x21, 0x42, 0x70, 0x26, 0xb4, 0xe0, 0xc1, 0xd8, 0x9b, 0xf6, 0xe7, 0xf6, 0x3c, 0xf9, 0xdd,
+	0x03, 0xf6, 0x3a, 0x15, 0x59, 0x7e, 0x23, 0xf5, 0x8c, 0xb4, 0x88, 0x84, 0x16, 0xf8, 0x15, 0xc0,
+	0x42, 0xa6, 0xcb, 0x9f, 0x72, 0x2d, 0x74, 0xc1, 0x1d, 0x6e, 0xb9, 0x4f, 0x65, 0xba, 0x7c, 0x6d,
+	0x12, 0x8e, 0xbb, 0xb7, 0x28, 0x01, 0xa3, 0x74, 0x65, 0x95, 0x56, 0x4d, 0x14, 0x90, 0xf1, 0xa7,
+	0x8d, 0xbf, 0xaa, 0x09, 0x8b, 0x4c, 0x7e, 0x80, 0x6e, 0xa9, 0xc0, 0x48, 0x34, 0x0a, 0x6c, 0xcf,
+	0xfe, 0xdc, 0x9e, 0xf1, 0x6b, 0xe8, 0x26, 0x4e, 0x99, 0x25, 0x0e, 0x8f, 0x79, 0xa9, 0xe5, 0xb1,
+	0x72, 0xc7, 0xbb, 0xa9, 0x9f, 0xfc, 0xd5, 0x84, 0xce, 0x8c, 0xf2, 0x5c, 0xc4, 0x84, 0xcf, 0x21,
+	0xd0, 0xdb, 0x59, 0x3d, 0x2b, 0x39, 0x5c, 0xba, 0x3a, 0x2d, 0x53, 0x86, 0x43, 0xf0, 0xb5, 0xac,
+	0x39, 0xf1, 0xb5, 0x34, 0x36, 0x96, 0x4a, 0x3e, 0xb2, 0x61, 0x90, 0x8d, 0xc1, 0xe0, 0xb1, 0x41,
+	0x1c, 0x41, 0xe7, 0x56, 0xc6, 0xf6, 0x76, 0x5b, 0x95, 0x64, 0x09, 0x6e, 0xc7, 0xd6, 0x7e, 0x3a,
+	0xb6, 0xe7, 0xd0, 0xa1, 0x54, 0xab, 0x15, 0xe5, 0xbc, 0x33, 0x6e, 0x4e, 0xc3, 0xe3, 0x9d, 0xda,
+	0x1d, 0x97, 0x54, 0xae, 0x06, 0x0f, 0xa0, 0xbd, 0x90, 0x49, 0xb2, 0xd2, 0xbc, 0x5b, 0xe1, 0x72,
+	0x18, 0x1e, 0x43, 0x37, 0x77, 0x13, 0xe3, 0x3d, 0x3b, 0x49, 0xf6, 0x78, 0x92, 0xe5, 0x04, 0xcb,
+	0x3a, 0xc3, 0xa8, 0xe8, 0x67, 0x5a, 0x68, 0x0e, 0x63, 0x6f, 0xda, 0x2d, 0x19, 0x0b, 0x0c, 0x3f,
+	0x05, 0x28, 0x4e, 0xe7, 0xab, 0x54, 0xf3, 0xb0, 0xd2, 0xb3, 0x82, 0x23, 0x87, 0xce, 0x42, 0xa6,
+	0x9a, 0xde, 0x69, 0xde, 0xb7, 0x17, 0x5b, 0x86, 0x93, 0x1f, 0xa1, 0x77, 0x2e, 0x54, 0x54, 0xac,
+	0x4f, 0x39, 0x41, 0xef, 0xc9, 0x04, 0x39, 0x04, 0x6f, 0xa5, 0xa6, 0xfa, 0xe3, 0x30, 0x48, 0xc5,
+	0x70, 0xf3, 0xa9, 0xe1, 0xc9, 0x37, 0xd0, 0xdb, 0xac, 0x2b, 0x0e, 0xa1, 0x95, 0xca, 0x88, 0x72,
+	0xee, 0x8d, 0x9b, 0xd3, 0x60, 0x5e, 0x04, 0xb8, 0x0f, 0xdd, 0x5b, 0x12, 0x2a, 0x25, 0x95, 0x73,
+	0xdf, 0x26, 0x36, 0xf1, 0xe4, 0x0f, 0x0f, 0xc0, 0x7c, 0x7f, 0x7a, 0x23, 0xd2, 0xd8, 0x6e, 0xc4,
+	0xc5, 0x59, 0x4d, 0x9d, 0x7f, 0x71, 0x86, 0x5f, 0xb8, 0x27, 0xe8, 0xdb, 0xb5, 0xfa, 0xb8, 0xfa,
+	0x4c, 0x8a, 0xef, 0x9e, 0xbc, 0xc3, 0x03, 0x68, 0x5f, 0xca, 0x88, 0x2e, 0xce, 0xea, 0x9a, 0x0b,
+	0xcc, 0x0c, 0xeb, 0xd4, 0x0d, 0xab, 0x78, 0xa8, 0x65, 0x78, 0xf8, 0x25, 0xf4, 0x36, 0x0f, 0x1b,
+	0x77, 0x21, 0xb4, 0xc1, 0xa5, 0x54, 0x89, 0xb8, 0x65, 0x0d, 0x7c, 0x06, 0xbb, 0x16, 0xd8, 0x36,
+	0x66, 0xde, 0xe1, 0xdf, 0x3e, 0x84, 0x95, 0x05, 0x47, 0x80, 0xf6, 0x2c, 0x8f, 0xcf, 0xd7, 0x19,
+	0x6b, 0x60, 0x08, 0x9d, 0x59, 0x1e, 0x9f, 0x90, 0xd0, 0xcc, 0x73, 0xc1, 0x2b, 0x25, 0x33, 0xe6,
+	0xbb, 0xaa, 0x17, 0x59, 0xc6, 0x9a, 0x38, 0x00, 0x28, 0xce, 0x73, 0xca, 0x33, 0x16, 0xb8, 0xc2,
+	0xef, 0xa5, 0x26, 0xd6, 0x32, 0x22, 0x5c, 0x60, 0xb3, 0x6d, 0x97, 0x35, 0xcb, 0xc4, 0x3a, 0xc8,
+	0xa0, 0x6f, 0x9a, 0x91, 0x50, 0xfa, 0xda, 0x74, 0xe9, 0xe2, 0x10, 0x58, 0x15, 0xb1, 0x1f, 0xf5,
+	0x10, 0x61, 0x30, 0xcb, 0xe3, 0x37, 0xa9, 0x22, 0xb1, 0xb8, 0x11, 0xd7, 0xb7, 0xc4, 0x00, 0xf7,
+	0x60, 0xc7, 0x11, 0x99, 0xcb, 0x5b, 0xe7, 0x2c, 0x74, 0x65, 0xa7, 0x37, 0xb4, 0xf8, 0xe5, 0xbb,
+	0xb5, 0x54, 0xeb, 0x84, 0xf5, 0xf1, 0x23, 0xd8, 0x9b, 0xe5, 0xf1, 0x95, 0x12, 0x69, 0xbe, 0x24,
+	0xf5, 0x92, 0x44, 0x44, 0x8a, 0xed, 0xb8, 0xaf, 0xaf, 0x56, 0x09, 0xc9, 0xb5, 0xbe, 0x94, 0xbf,
+	0xb2, 0x81, 0x13, 0x33, 0x27, 0x11, 0xd9, 0x3f, 0x27, 0xdb, 0x75, 0x62, 0x36, 0x88, 0x15, 0xc3,
+	0x9c, 0xdf, 0x57, 0x8a, 0xac, 0xc5, 0x3d, 0xd7, 0xd5, 0xc5, 0xb6, 0x06, 0x0f, 0xef, 0x60, 0x50,
+	0xbf, 0x5e, 0xa3, 0x63, 0x8b, 0xbc, 0x88, 0x22, 0x73, 0x97, 0xac, 0x81, 0x1c, 0x86, 0x5b, 0x78,
+	0x4e, 0x89, 0x7c, 0x4b, 0x36, 0xe3, 0xd5, 0x33, 0x6f, 0xb2, 0x48, 0xe8, 0x22, 0xe3, 0xe3, 0x01,
+	0xf0, 0x1a, 0xd5, 0xcb, 0x62, 0x1b, 0x6d, 0xb6, 0x79, 0xc2, 0xef, 0x3f, 0x8c, 0x1a, 0xef, 0x3f,
+	0x8c, 0x1a, 0xf7, 0x0f, 0x23, 0xef, 0xfd, 0xc3, 0xc8, 0xfb, 0xf7, 0x61, 0xe4, 0xfd, 0xf9, 0xdf,
+	0xa8, 0xf1, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x30, 0xe1, 0x02, 0x69, 0x74, 0x06, 0x00, 0x00,
+}
+
+func (m *Entry) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Entry) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Entry) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Data != nil {
+		i -= len(m.Data)
+		copy(dAtA[i:], m.Data)
+		i = encodeVarintRaft(dAtA, i, uint64(len(m.Data)))
+		i--
+		dAtA[i] = 0x22
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
+	i--
+	dAtA[i] = 0x18
+	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
+	i--
+	dAtA[i] = 0x10
+	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func (m *SnapshotMetadata) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *SnapshotMetadata) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *SnapshotMetadata) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
+	i--
+	dAtA[i] = 0x18
+	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
+	i--
+	dAtA[i] = 0x10
+	{
+		size, err := m.ConfState.MarshalToSizedBuffer(dAtA[:i])
+		if err != nil {
+			return 0, err
+		}
+		i -= size
+		i = encodeVarintRaft(dAtA, i, uint64(size))
+	}
+	i--
+	dAtA[i] = 0xa
+	return len(dAtA) - i, nil
+}
+
+func (m *Snapshot) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Snapshot) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Snapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	{
+		size, err := m.Metadata.MarshalToSizedBuffer(dAtA[:i])
+		if err != nil {
+			return 0, err
+		}
+		i -= size
+		i = encodeVarintRaft(dAtA, i, uint64(size))
+	}
+	i--
+	dAtA[i] = 0x12
+	if m.Data != nil {
+		i -= len(m.Data)
+		copy(dAtA[i:], m.Data)
+		i = encodeVarintRaft(dAtA, i, uint64(len(m.Data)))
+		i--
+		dAtA[i] = 0xa
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *Message) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Message) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Context != nil {
+		i -= len(m.Context)
+		copy(dAtA[i:], m.Context)
+		i = encodeVarintRaft(dAtA, i, uint64(len(m.Context)))
+		i--
+		dAtA[i] = 0x62
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.RejectHint))
+	i--
+	dAtA[i] = 0x58
+	i--
+	if m.Reject {
+		dAtA[i] = 1
+	} else {
+		dAtA[i] = 0
+	}
+	i--
+	dAtA[i] = 0x50
+	{
+		size, err := m.Snapshot.MarshalToSizedBuffer(dAtA[:i])
+		if err != nil {
+			return 0, err
+		}
+		i -= size
+		i = encodeVarintRaft(dAtA, i, uint64(size))
+	}
+	i--
+	dAtA[i] = 0x4a
+	i = encodeVarintRaft(dAtA, i, uint64(m.Commit))
+	i--
+	dAtA[i] = 0x40
+	if len(m.Entries) > 0 {
+		for iNdEx := len(m.Entries) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Entries[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintRaft(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x3a
+		}
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
+	i--
+	dAtA[i] = 0x30
+	i = encodeVarintRaft(dAtA, i, uint64(m.LogTerm))
+	i--
+	dAtA[i] = 0x28
+	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
+	i--
+	dAtA[i] = 0x20
+	i = encodeVarintRaft(dAtA, i, uint64(m.From))
+	i--
+	dAtA[i] = 0x18
+	i = encodeVarintRaft(dAtA, i, uint64(m.To))
+	i--
+	dAtA[i] = 0x10
+	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func (m *HardState) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *HardState) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *HardState) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.Commit))
+	i--
+	dAtA[i] = 0x18
+	i = encodeVarintRaft(dAtA, i, uint64(m.Vote))
+	i--
+	dAtA[i] = 0x10
+	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func (m *ConfState) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *ConfState) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *ConfState) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if len(m.Learners) > 0 {
+		for iNdEx := len(m.Learners) - 1; iNdEx >= 0; iNdEx-- {
+			i = encodeVarintRaft(dAtA, i, uint64(m.Learners[iNdEx]))
+			i--
+			dAtA[i] = 0x10
+		}
+	}
+	if len(m.Nodes) > 0 {
+		for iNdEx := len(m.Nodes) - 1; iNdEx >= 0; iNdEx-- {
+			i = encodeVarintRaft(dAtA, i, uint64(m.Nodes[iNdEx]))
+			i--
+			dAtA[i] = 0x8
+		}
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *ConfChange) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *ConfChange) MarshalTo(dAtA []byte) (int, error) {
+	size := m.Size()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *ConfChange) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.Context != nil {
+		i -= len(m.Context)
+		copy(dAtA[i:], m.Context)
+		i = encodeVarintRaft(dAtA, i, uint64(len(m.Context)))
+		i--
+		dAtA[i] = 0x22
+	}
+	i = encodeVarintRaft(dAtA, i, uint64(m.NodeID))
+	i--
+	dAtA[i] = 0x18
+	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
+	i--
+	dAtA[i] = 0x10
+	i = encodeVarintRaft(dAtA, i, uint64(m.ID))
+	i--
+	dAtA[i] = 0x8
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintRaft(dAtA []byte, offset int, v uint64) int {
+	offset -= sovRaft(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *Entry) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRaft(uint64(m.Type))
+	n += 1 + sovRaft(uint64(m.Term))
+	n += 1 + sovRaft(uint64(m.Index))
+	if m.Data != nil {
+		l = len(m.Data)
+		n += 1 + l + sovRaft(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *SnapshotMetadata) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	l = m.ConfState.Size()
+	n += 1 + l + sovRaft(uint64(l))
+	n += 1 + sovRaft(uint64(m.Index))
+	n += 1 + sovRaft(uint64(m.Term))
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Snapshot) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Data != nil {
+		l = len(m.Data)
+		n += 1 + l + sovRaft(uint64(l))
+	}
+	l = m.Metadata.Size()
+	n += 1 + l + sovRaft(uint64(l))
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Message) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRaft(uint64(m.Type))
+	n += 1 + sovRaft(uint64(m.To))
+	n += 1 + sovRaft(uint64(m.From))
+	n += 1 + sovRaft(uint64(m.Term))
+	n += 1 + sovRaft(uint64(m.LogTerm))
+	n += 1 + sovRaft(uint64(m.Index))
+	if len(m.Entries) > 0 {
+		for _, e := range m.Entries {
+			l = e.Size()
+			n += 1 + l + sovRaft(uint64(l))
+		}
+	}
+	n += 1 + sovRaft(uint64(m.Commit))
+	l = m.Snapshot.Size()
+	n += 1 + l + sovRaft(uint64(l))
+	n += 2
+	n += 1 + sovRaft(uint64(m.RejectHint))
+	if m.Context != nil {
+		l = len(m.Context)
+		n += 1 + l + sovRaft(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *HardState) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRaft(uint64(m.Term))
+	n += 1 + sovRaft(uint64(m.Vote))
+	n += 1 + sovRaft(uint64(m.Commit))
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *ConfState) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if len(m.Nodes) > 0 {
+		for _, e := range m.Nodes {
+			n += 1 + sovRaft(uint64(e))
+		}
+	}
+	if len(m.Learners) > 0 {
+		for _, e := range m.Learners {
+			n += 1 + sovRaft(uint64(e))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *ConfChange) Size() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	n += 1 + sovRaft(uint64(m.ID))
+	n += 1 + sovRaft(uint64(m.Type))
+	n += 1 + sovRaft(uint64(m.NodeID))
+	if m.Context != nil {
+		l = len(m.Context)
+		n += 1 + l + sovRaft(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovRaft(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozRaft(x uint64) (n int) {
+	return sovRaft(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Entry) 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 ErrIntOverflowRaft
+			}
+			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: Entry: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Entry: 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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Type |= EntryType(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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Term |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Index |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 *SnapshotMetadata) 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 ErrIntOverflowRaft
+			}
+			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: SnapshotMetadata: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SnapshotMetadata: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ConfState", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if err := m.ConfState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Index |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			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 ErrIntOverflowRaft
+				}
+				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 := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 ErrIntOverflowRaft
+			}
+			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 != 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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...)
+			if m.Data == nil {
+				m.Data = []byte{}
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if err := m.Metadata.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 *Message) 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 ErrIntOverflowRaft
+			}
+			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: Message: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Message: 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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Type |= MessageType(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field To", wireType)
+			}
+			m.To = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.To |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field From", wireType)
+			}
+			m.From = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.From |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Term |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LogTerm", wireType)
+			}
+			m.LogTerm = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.LogTerm |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 6:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Index |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 7:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Entries", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Entries = append(m.Entries, Entry{})
+			if err := m.Entries[len(m.Entries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Commit", wireType)
+			}
+			m.Commit = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Commit |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 9:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if err := m.Snapshot.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 10:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Reject", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Reject = bool(v != 0)
+		case 11:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RejectHint", wireType)
+			}
+			m.RejectHint = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RejectHint |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 12:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...)
+			if m.Context == nil {
+				m.Context = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 *HardState) 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 ErrIntOverflowRaft
+			}
+			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: HardState: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: HardState: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Term |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Vote", wireType)
+			}
+			m.Vote = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Vote |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Commit", wireType)
+			}
+			m.Commit = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Commit |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 *ConfState) 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 ErrIntOverflowRaft
+			}
+			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: ConfState: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ConfState: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType == 0 {
+				var v uint64
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRaft
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= uint64(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.Nodes = append(m.Nodes, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRaft
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthRaft
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthRaft
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.Nodes) == 0 {
+					m.Nodes = make([]uint64, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v uint64
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowRaft
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= uint64(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.Nodes = append(m.Nodes, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType)
+			}
+		case 2:
+			if wireType == 0 {
+				var v uint64
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRaft
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= uint64(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.Learners = append(m.Learners, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowRaft
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthRaft
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthRaft
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.Learners) == 0 {
+					m.Learners = make([]uint64, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v uint64
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowRaft
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= uint64(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.Learners = append(m.Learners, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field Learners", wireType)
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 *ConfChange) 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 ErrIntOverflowRaft
+			}
+			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: ConfChange: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ConfChange: 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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			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 ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Type |= ConfChangeType(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
+			}
+			m.NodeID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.NodeID |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRaft
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...)
+			if m.Context == nil {
+				m.Context = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRaft(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRaft
+			}
+			if (iNdEx + skippy) < 0 {
+				return ErrInvalidLengthRaft
+			}
+			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 skipRaft(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, ErrIntOverflowRaft
+			}
+			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, ErrIntOverflowRaft
+				}
+				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, ErrIntOverflowRaft
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if length < 0 {
+				return 0, ErrInvalidLengthRaft
+			}
+			iNdEx += length
+			if iNdEx < 0 {
+				return 0, ErrInvalidLengthRaft
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowRaft
+					}
+					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 := skipRaft(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+				if iNdEx < 0 {
+					return 0, ErrInvalidLengthRaft
+				}
+			}
+			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 (
+	ErrInvalidLengthRaft = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowRaft   = fmt.Errorf("proto: integer overflow")
+)
diff --git a/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto b/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto
new file mode 100644
index 0000000..644ce7b
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/raftpb/raft.proto
@@ -0,0 +1,95 @@
+syntax = "proto2";
+package raftpb;
+
+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;
+option (gogoproto.goproto_enum_prefix_all) = false;
+
+enum EntryType {
+	EntryNormal     = 0;
+	EntryConfChange = 1;
+}
+
+message Entry {
+	optional uint64     Term  = 2 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
+	optional uint64     Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
+	optional EntryType  Type  = 1 [(gogoproto.nullable) = false];
+	optional bytes      Data  = 4;
+}
+
+message SnapshotMetadata {
+	optional ConfState conf_state = 1 [(gogoproto.nullable) = false];
+	optional uint64    index      = 2 [(gogoproto.nullable) = false];
+	optional uint64    term       = 3 [(gogoproto.nullable) = false];
+}
+
+message Snapshot {
+	optional bytes            data     = 1;
+	optional SnapshotMetadata metadata = 2 [(gogoproto.nullable) = false];
+}
+
+enum MessageType {
+	MsgHup             = 0;
+	MsgBeat            = 1;
+	MsgProp            = 2;
+	MsgApp             = 3;
+	MsgAppResp         = 4;
+	MsgVote            = 5;
+	MsgVoteResp        = 6;
+	MsgSnap            = 7;
+	MsgHeartbeat       = 8;
+	MsgHeartbeatResp   = 9;
+	MsgUnreachable     = 10;
+	MsgSnapStatus      = 11;
+	MsgCheckQuorum     = 12;
+	MsgTransferLeader  = 13;
+	MsgTimeoutNow      = 14;
+	MsgReadIndex       = 15;
+	MsgReadIndexResp   = 16;
+	MsgPreVote         = 17;
+	MsgPreVoteResp     = 18;
+}
+
+message Message {
+	optional MessageType type        = 1  [(gogoproto.nullable) = false];
+	optional uint64      to          = 2  [(gogoproto.nullable) = false];
+	optional uint64      from        = 3  [(gogoproto.nullable) = false];
+	optional uint64      term        = 4  [(gogoproto.nullable) = false];
+	optional uint64      logTerm     = 5  [(gogoproto.nullable) = false];
+	optional uint64      index       = 6  [(gogoproto.nullable) = false];
+	repeated Entry       entries     = 7  [(gogoproto.nullable) = false];
+	optional uint64      commit      = 8  [(gogoproto.nullable) = false];
+	optional Snapshot    snapshot    = 9  [(gogoproto.nullable) = false];
+	optional bool        reject      = 10 [(gogoproto.nullable) = false];
+	optional uint64      rejectHint  = 11 [(gogoproto.nullable) = false];
+	optional bytes       context     = 12;
+}
+
+message HardState {
+	optional uint64 term   = 1 [(gogoproto.nullable) = false];
+	optional uint64 vote   = 2 [(gogoproto.nullable) = false];
+	optional uint64 commit = 3 [(gogoproto.nullable) = false];
+}
+
+message ConfState {
+	repeated uint64 nodes    = 1;
+	repeated uint64 learners = 2;
+}
+
+enum ConfChangeType {
+	ConfChangeAddNode        = 0;
+	ConfChangeRemoveNode     = 1;
+	ConfChangeUpdateNode     = 2;
+	ConfChangeAddLearnerNode = 3;
+}
+
+message ConfChange {
+	optional uint64          ID      = 1 [(gogoproto.nullable) = false];
+	optional ConfChangeType  Type    = 2 [(gogoproto.nullable) = false];
+	optional uint64          NodeID  = 3 [(gogoproto.nullable) = false];
+	optional bytes           Context = 4;
+}
diff --git a/vendor/github.com/coreos/etcd/raft/rawnode.go b/vendor/github.com/coreos/etcd/raft/rawnode.go
new file mode 100644
index 0000000..925cb85
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/rawnode.go
@@ -0,0 +1,266 @@
+// 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 raft
+
+import (
+	"errors"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+// ErrStepLocalMsg is returned when try to step a local raft message
+var ErrStepLocalMsg = errors.New("raft: cannot step raft local message")
+
+// ErrStepPeerNotFound is returned when try to step a response message
+// but there is no peer found in raft.prs for that node.
+var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found")
+
+// RawNode is a thread-unsafe Node.
+// The methods of this struct correspond to the methods of Node and are described
+// more fully there.
+type RawNode struct {
+	raft       *raft
+	prevSoftSt *SoftState
+	prevHardSt pb.HardState
+}
+
+func (rn *RawNode) newReady() Ready {
+	return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
+}
+
+func (rn *RawNode) commitReady(rd Ready) {
+	if rd.SoftState != nil {
+		rn.prevSoftSt = rd.SoftState
+	}
+	if !IsEmptyHardState(rd.HardState) {
+		rn.prevHardSt = rd.HardState
+	}
+	if rn.prevHardSt.Commit != 0 {
+		// In most cases, prevHardSt and rd.HardState will be the same
+		// because when there are new entries to apply we just sent a
+		// HardState with an updated Commit value. However, on initial
+		// startup the two are different because we don't send a HardState
+		// until something changes, but we do send any un-applied but
+		// committed entries (and previously-committed entries may be
+		// incorporated into the snapshot, even if rd.CommittedEntries is
+		// empty). Therefore we mark all committed entries as applied
+		// whether they were included in rd.HardState or not.
+		rn.raft.raftLog.appliedTo(rn.prevHardSt.Commit)
+	}
+	if len(rd.Entries) > 0 {
+		e := rd.Entries[len(rd.Entries)-1]
+		rn.raft.raftLog.stableTo(e.Index, e.Term)
+	}
+	if !IsEmptySnap(rd.Snapshot) {
+		rn.raft.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
+	}
+	if len(rd.ReadStates) != 0 {
+		rn.raft.readStates = nil
+	}
+}
+
+// NewRawNode returns a new RawNode given configuration and a list of raft peers.
+func NewRawNode(config *Config, peers []Peer) (*RawNode, error) {
+	if config.ID == 0 {
+		panic("config.ID must not be zero")
+	}
+	r := newRaft(config)
+	rn := &RawNode{
+		raft: r,
+	}
+	lastIndex, err := config.Storage.LastIndex()
+	if err != nil {
+		panic(err) // TODO(bdarnell)
+	}
+	// If the log is empty, this is a new RawNode (like StartNode); otherwise it's
+	// restoring an existing RawNode (like RestartNode).
+	// TODO(bdarnell): rethink RawNode initialization and whether the application needs
+	// to be able to tell us when it expects the RawNode to exist.
+	if lastIndex == 0 {
+		r.becomeFollower(1, None)
+		ents := make([]pb.Entry, len(peers))
+		for i, peer := range peers {
+			cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
+			data, err := cc.Marshal()
+			if err != nil {
+				panic("unexpected marshal error")
+			}
+
+			ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
+		}
+		r.raftLog.append(ents...)
+		r.raftLog.committed = uint64(len(ents))
+		for _, peer := range peers {
+			r.addNode(peer.ID)
+		}
+	}
+
+	// Set the initial hard and soft states after performing all initialization.
+	rn.prevSoftSt = r.softState()
+	if lastIndex == 0 {
+		rn.prevHardSt = emptyState
+	} else {
+		rn.prevHardSt = r.hardState()
+	}
+
+	return rn, nil
+}
+
+// Tick advances the internal logical clock by a single tick.
+func (rn *RawNode) Tick() {
+	rn.raft.tick()
+}
+
+// TickQuiesced advances the internal logical clock by a single tick without
+// performing any other state machine processing. It allows the caller to avoid
+// periodic heartbeats and elections when all of the peers in a Raft group are
+// known to be at the same state. Expected usage is to periodically invoke Tick
+// or TickQuiesced depending on whether the group is "active" or "quiesced".
+//
+// WARNING: Be very careful about using this method as it subverts the Raft
+// state machine. You should probably be using Tick instead.
+func (rn *RawNode) TickQuiesced() {
+	rn.raft.electionElapsed++
+}
+
+// Campaign causes this RawNode to transition to candidate state.
+func (rn *RawNode) Campaign() error {
+	return rn.raft.Step(pb.Message{
+		Type: pb.MsgHup,
+	})
+}
+
+// Propose proposes data be appended to the raft log.
+func (rn *RawNode) Propose(data []byte) error {
+	return rn.raft.Step(pb.Message{
+		Type: pb.MsgProp,
+		From: rn.raft.id,
+		Entries: []pb.Entry{
+			{Data: data},
+		}})
+}
+
+// ProposeConfChange proposes a config change.
+func (rn *RawNode) ProposeConfChange(cc pb.ConfChange) error {
+	data, err := cc.Marshal()
+	if err != nil {
+		return err
+	}
+	return rn.raft.Step(pb.Message{
+		Type: pb.MsgProp,
+		Entries: []pb.Entry{
+			{Type: pb.EntryConfChange, Data: data},
+		},
+	})
+}
+
+// ApplyConfChange applies a config change to the local node.
+func (rn *RawNode) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
+	if cc.NodeID == None {
+		rn.raft.resetPendingConf()
+		return &pb.ConfState{Nodes: rn.raft.nodes()}
+	}
+	switch cc.Type {
+	case pb.ConfChangeAddNode:
+		rn.raft.addNode(cc.NodeID)
+	case pb.ConfChangeAddLearnerNode:
+		rn.raft.addLearner(cc.NodeID)
+	case pb.ConfChangeRemoveNode:
+		rn.raft.removeNode(cc.NodeID)
+	case pb.ConfChangeUpdateNode:
+		rn.raft.resetPendingConf()
+	default:
+		panic("unexpected conf type")
+	}
+	return &pb.ConfState{Nodes: rn.raft.nodes()}
+}
+
+// Step advances the state machine using the given message.
+func (rn *RawNode) Step(m pb.Message) error {
+	// ignore unexpected local messages receiving over network
+	if IsLocalMsg(m.Type) {
+		return ErrStepLocalMsg
+	}
+	if pr := rn.raft.getProgress(m.From); pr != nil || !IsResponseMsg(m.Type) {
+		return rn.raft.Step(m)
+	}
+	return ErrStepPeerNotFound
+}
+
+// Ready returns the current point-in-time state of this RawNode.
+func (rn *RawNode) Ready() Ready {
+	rd := rn.newReady()
+	rn.raft.msgs = nil
+	return rd
+}
+
+// HasReady called when RawNode user need to check if any Ready pending.
+// Checking logic in this method should be consistent with Ready.containsUpdates().
+func (rn *RawNode) HasReady() bool {
+	r := rn.raft
+	if !r.softState().equal(rn.prevSoftSt) {
+		return true
+	}
+	if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) {
+		return true
+	}
+	if r.raftLog.unstable.snapshot != nil && !IsEmptySnap(*r.raftLog.unstable.snapshot) {
+		return true
+	}
+	if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
+		return true
+	}
+	if len(r.readStates) != 0 {
+		return true
+	}
+	return false
+}
+
+// Advance notifies the RawNode that the application has applied and saved progress in the
+// last Ready results.
+func (rn *RawNode) Advance(rd Ready) {
+	rn.commitReady(rd)
+}
+
+// Status returns the current status of the given group.
+func (rn *RawNode) Status() *Status {
+	status := getStatus(rn.raft)
+	return &status
+}
+
+// ReportUnreachable reports the given node is not reachable for the last send.
+func (rn *RawNode) ReportUnreachable(id uint64) {
+	_ = rn.raft.Step(pb.Message{Type: pb.MsgUnreachable, From: id})
+}
+
+// ReportSnapshot reports the status of the sent snapshot.
+func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
+	rej := status == SnapshotFailure
+
+	_ = rn.raft.Step(pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej})
+}
+
+// TransferLeader tries to transfer leadership to the given transferee.
+func (rn *RawNode) TransferLeader(transferee uint64) {
+	_ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
+}
+
+// ReadIndex requests a read state. The read state will be set in ready.
+// Read State has a read index. Once the application advances further than the read
+// index, any linearizable read requests issued before the read request can be
+// processed safely. The read state will have the same rctx attached.
+func (rn *RawNode) ReadIndex(rctx []byte) {
+	_ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
+}
diff --git a/vendor/github.com/coreos/etcd/raft/read_only.go b/vendor/github.com/coreos/etcd/raft/read_only.go
new file mode 100644
index 0000000..ae746fa
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/read_only.go
@@ -0,0 +1,118 @@
+// 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 raft
+
+import pb "github.com/coreos/etcd/raft/raftpb"
+
+// ReadState provides state for read only query.
+// It's caller's responsibility to call ReadIndex first before getting
+// this state from ready, it's also caller's duty to differentiate if this
+// state is what it requests through RequestCtx, eg. given a unique id as
+// RequestCtx
+type ReadState struct {
+	Index      uint64
+	RequestCtx []byte
+}
+
+type readIndexStatus struct {
+	req   pb.Message
+	index uint64
+	acks  map[uint64]struct{}
+}
+
+type readOnly struct {
+	option           ReadOnlyOption
+	pendingReadIndex map[string]*readIndexStatus
+	readIndexQueue   []string
+}
+
+func newReadOnly(option ReadOnlyOption) *readOnly {
+	return &readOnly{
+		option:           option,
+		pendingReadIndex: make(map[string]*readIndexStatus),
+	}
+}
+
+// addRequest adds a read only reuqest into readonly struct.
+// `index` is the commit index of the raft state machine when it received
+// the read only request.
+// `m` is the original read only request message from the local or remote node.
+func (ro *readOnly) addRequest(index uint64, m pb.Message) {
+	ctx := string(m.Entries[0].Data)
+	if _, ok := ro.pendingReadIndex[ctx]; ok {
+		return
+	}
+	ro.pendingReadIndex[ctx] = &readIndexStatus{index: index, req: m, acks: make(map[uint64]struct{})}
+	ro.readIndexQueue = append(ro.readIndexQueue, ctx)
+}
+
+// recvAck notifies the readonly struct that the raft state machine received
+// an acknowledgment of the heartbeat that attached with the read only request
+// context.
+func (ro *readOnly) recvAck(m pb.Message) int {
+	rs, ok := ro.pendingReadIndex[string(m.Context)]
+	if !ok {
+		return 0
+	}
+
+	rs.acks[m.From] = struct{}{}
+	// add one to include an ack from local node
+	return len(rs.acks) + 1
+}
+
+// advance advances the read only request queue kept by the readonly struct.
+// It dequeues the requests until it finds the read only request that has
+// the same context as the given `m`.
+func (ro *readOnly) advance(m pb.Message) []*readIndexStatus {
+	var (
+		i     int
+		found bool
+	)
+
+	ctx := string(m.Context)
+	rss := []*readIndexStatus{}
+
+	for _, okctx := range ro.readIndexQueue {
+		i++
+		rs, ok := ro.pendingReadIndex[okctx]
+		if !ok {
+			panic("cannot find corresponding read state from pending map")
+		}
+		rss = append(rss, rs)
+		if okctx == ctx {
+			found = true
+			break
+		}
+	}
+
+	if found {
+		ro.readIndexQueue = ro.readIndexQueue[i:]
+		for _, rs := range rss {
+			delete(ro.pendingReadIndex, string(rs.req.Entries[0].Data))
+		}
+		return rss
+	}
+
+	return nil
+}
+
+// lastPendingRequestCtx returns the context of the last pending read only
+// request in readonly struct.
+func (ro *readOnly) lastPendingRequestCtx() string {
+	if len(ro.readIndexQueue) == 0 {
+		return ""
+	}
+	return ro.readIndexQueue[len(ro.readIndexQueue)-1]
+}
diff --git a/vendor/github.com/coreos/etcd/raft/status.go b/vendor/github.com/coreos/etcd/raft/status.go
new file mode 100644
index 0000000..f4d3d86
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/status.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 raft
+
+import (
+	"fmt"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+type Status struct {
+	ID uint64
+
+	pb.HardState
+	SoftState
+
+	Applied  uint64
+	Progress map[uint64]Progress
+
+	LeadTransferee uint64
+}
+
+// getStatus gets a copy of the current raft status.
+func getStatus(r *raft) Status {
+	s := Status{
+		ID:             r.id,
+		LeadTransferee: r.leadTransferee,
+	}
+
+	s.HardState = r.hardState()
+	s.SoftState = *r.softState()
+
+	s.Applied = r.raftLog.applied
+
+	if s.RaftState == StateLeader {
+		s.Progress = make(map[uint64]Progress)
+		for id, p := range r.prs {
+			s.Progress[id] = *p
+		}
+
+		for id, p := range r.learnerPrs {
+			s.Progress[id] = *p
+		}
+	}
+
+	return s
+}
+
+// MarshalJSON translates the raft status into JSON.
+// TODO: try to simplify this by introducing ID type into raft
+func (s Status) MarshalJSON() ([]byte, error) {
+	j := fmt.Sprintf(`{"id":"%x","term":%d,"vote":"%x","commit":%d,"lead":"%x","raftState":%q,"applied":%d,"progress":{`,
+		s.ID, s.Term, s.Vote, s.Commit, s.Lead, s.RaftState, s.Applied)
+
+	if len(s.Progress) == 0 {
+		j += "},"
+	} else {
+		for k, v := range s.Progress {
+			subj := fmt.Sprintf(`"%x":{"match":%d,"next":%d,"state":%q},`, k, v.Match, v.Next, v.State)
+			j += subj
+		}
+		// remove the trailing ","
+		j = j[:len(j)-1] + "},"
+	}
+
+	j += fmt.Sprintf(`"leadtransferee":"%x"}`, s.LeadTransferee)
+	return []byte(j), nil
+}
+
+func (s Status) String() string {
+	b, err := s.MarshalJSON()
+	if err != nil {
+		raftLogger.Panicf("unexpected error: %v", err)
+	}
+	return string(b)
+}
diff --git a/vendor/github.com/coreos/etcd/raft/storage.go b/vendor/github.com/coreos/etcd/raft/storage.go
new file mode 100644
index 0000000..69c3a7d
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/storage.go
@@ -0,0 +1,271 @@
+// 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 raft
+
+import (
+	"errors"
+	"sync"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+// ErrCompacted is returned by Storage.Entries/Compact when a requested
+// index is unavailable because it predates the last snapshot.
+var ErrCompacted = errors.New("requested index is unavailable due to compaction")
+
+// ErrSnapOutOfDate is returned by Storage.CreateSnapshot when a requested
+// index is older than the existing snapshot.
+var ErrSnapOutOfDate = errors.New("requested index is older than the existing snapshot")
+
+// ErrUnavailable is returned by Storage interface when the requested log entries
+// are unavailable.
+var ErrUnavailable = errors.New("requested entry at index is unavailable")
+
+// ErrSnapshotTemporarilyUnavailable is returned by the Storage interface when the required
+// snapshot is temporarily unavailable.
+var ErrSnapshotTemporarilyUnavailable = errors.New("snapshot is temporarily unavailable")
+
+// Storage is an interface that may be implemented by the application
+// to retrieve log entries from storage.
+//
+// If any Storage method returns an error, the raft instance will
+// become inoperable and refuse to participate in elections; the
+// application is responsible for cleanup and recovery in this case.
+type Storage interface {
+	// InitialState returns the saved HardState and ConfState information.
+	InitialState() (pb.HardState, pb.ConfState, error)
+	// Entries returns a slice of log entries in the range [lo,hi).
+	// MaxSize limits the total size of the log entries returned, but
+	// Entries returns at least one entry if any.
+	Entries(lo, hi, maxSize uint64) ([]pb.Entry, error)
+	// Term returns the term of entry i, which must be in the range
+	// [FirstIndex()-1, LastIndex()]. The term of the entry before
+	// FirstIndex is retained for matching purposes even though the
+	// rest of that entry may not be available.
+	Term(i uint64) (uint64, error)
+	// LastIndex returns the index of the last entry in the log.
+	LastIndex() (uint64, error)
+	// FirstIndex returns the index of the first log entry that is
+	// possibly available via Entries (older entries have been incorporated
+	// into the latest Snapshot; if storage only contains the dummy entry the
+	// first log entry is not available).
+	FirstIndex() (uint64, error)
+	// Snapshot returns the most recent snapshot.
+	// If snapshot is temporarily unavailable, it should return ErrSnapshotTemporarilyUnavailable,
+	// so raft state machine could know that Storage needs some time to prepare
+	// snapshot and call Snapshot later.
+	Snapshot() (pb.Snapshot, error)
+}
+
+// MemoryStorage implements the Storage interface backed by an
+// in-memory array.
+type MemoryStorage struct {
+	// Protects access to all fields. Most methods of MemoryStorage are
+	// run on the raft goroutine, but Append() is run on an application
+	// goroutine.
+	sync.Mutex
+
+	hardState pb.HardState
+	snapshot  pb.Snapshot
+	// ents[i] has raft log position i+snapshot.Metadata.Index
+	ents []pb.Entry
+}
+
+// NewMemoryStorage creates an empty MemoryStorage.
+func NewMemoryStorage() *MemoryStorage {
+	return &MemoryStorage{
+		// When starting from scratch populate the list with a dummy entry at term zero.
+		ents: make([]pb.Entry, 1),
+	}
+}
+
+// InitialState implements the Storage interface.
+func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) {
+	return ms.hardState, ms.snapshot.Metadata.ConfState, nil
+}
+
+// SetHardState saves the current HardState.
+func (ms *MemoryStorage) SetHardState(st pb.HardState) error {
+	ms.Lock()
+	defer ms.Unlock()
+	ms.hardState = st
+	return nil
+}
+
+// Entries implements the Storage interface.
+func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	offset := ms.ents[0].Index
+	if lo <= offset {
+		return nil, ErrCompacted
+	}
+	if hi > ms.lastIndex()+1 {
+		raftLogger.Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex())
+	}
+	// only contains dummy entries.
+	if len(ms.ents) == 1 {
+		return nil, ErrUnavailable
+	}
+
+	ents := ms.ents[lo-offset : hi-offset]
+	return limitSize(ents, maxSize), nil
+}
+
+// Term implements the Storage interface.
+func (ms *MemoryStorage) Term(i uint64) (uint64, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	offset := ms.ents[0].Index
+	if i < offset {
+		return 0, ErrCompacted
+	}
+	if int(i-offset) >= len(ms.ents) {
+		return 0, ErrUnavailable
+	}
+	return ms.ents[i-offset].Term, nil
+}
+
+// LastIndex implements the Storage interface.
+func (ms *MemoryStorage) LastIndex() (uint64, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.lastIndex(), nil
+}
+
+func (ms *MemoryStorage) lastIndex() uint64 {
+	return ms.ents[0].Index + uint64(len(ms.ents)) - 1
+}
+
+// FirstIndex implements the Storage interface.
+func (ms *MemoryStorage) FirstIndex() (uint64, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.firstIndex(), nil
+}
+
+func (ms *MemoryStorage) firstIndex() uint64 {
+	return ms.ents[0].Index + 1
+}
+
+// Snapshot implements the Storage interface.
+func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	return ms.snapshot, nil
+}
+
+// ApplySnapshot overwrites the contents of this Storage object with
+// those of the given snapshot.
+func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
+	ms.Lock()
+	defer ms.Unlock()
+
+	//handle check for old snapshot being applied
+	msIndex := ms.snapshot.Metadata.Index
+	snapIndex := snap.Metadata.Index
+	if msIndex >= snapIndex {
+		return ErrSnapOutOfDate
+	}
+
+	ms.snapshot = snap
+	ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}}
+	return nil
+}
+
+// CreateSnapshot makes a snapshot which can be retrieved with Snapshot() and
+// can be used to reconstruct the state at that point.
+// If any configuration changes have been made since the last compaction,
+// the result of the last ApplyConfChange must be passed in.
+func (ms *MemoryStorage) CreateSnapshot(i uint64, cs *pb.ConfState, data []byte) (pb.Snapshot, error) {
+	ms.Lock()
+	defer ms.Unlock()
+	if i <= ms.snapshot.Metadata.Index {
+		return pb.Snapshot{}, ErrSnapOutOfDate
+	}
+
+	offset := ms.ents[0].Index
+	if i > ms.lastIndex() {
+		raftLogger.Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex())
+	}
+
+	ms.snapshot.Metadata.Index = i
+	ms.snapshot.Metadata.Term = ms.ents[i-offset].Term
+	if cs != nil {
+		ms.snapshot.Metadata.ConfState = *cs
+	}
+	ms.snapshot.Data = data
+	return ms.snapshot, nil
+}
+
+// Compact discards all log entries prior to compactIndex.
+// It is the application's responsibility to not attempt to compact an index
+// greater than raftLog.applied.
+func (ms *MemoryStorage) Compact(compactIndex uint64) error {
+	ms.Lock()
+	defer ms.Unlock()
+	offset := ms.ents[0].Index
+	if compactIndex <= offset {
+		return ErrCompacted
+	}
+	if compactIndex > ms.lastIndex() {
+		raftLogger.Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex())
+	}
+
+	i := compactIndex - offset
+	ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i)
+	ents[0].Index = ms.ents[i].Index
+	ents[0].Term = ms.ents[i].Term
+	ents = append(ents, ms.ents[i+1:]...)
+	ms.ents = ents
+	return nil
+}
+
+// Append the new entries to storage.
+// TODO (xiangli): ensure the entries are continuous and
+// entries[0].Index > ms.entries[0].Index
+func (ms *MemoryStorage) Append(entries []pb.Entry) error {
+	if len(entries) == 0 {
+		return nil
+	}
+
+	ms.Lock()
+	defer ms.Unlock()
+
+	first := ms.firstIndex()
+	last := entries[0].Index + uint64(len(entries)) - 1
+
+	// shortcut if there is no new entry.
+	if last < first {
+		return nil
+	}
+	// truncate compacted entries
+	if first > entries[0].Index {
+		entries = entries[first-entries[0].Index:]
+	}
+
+	offset := entries[0].Index - ms.ents[0].Index
+	switch {
+	case uint64(len(ms.ents)) > offset:
+		ms.ents = append([]pb.Entry{}, ms.ents[:offset]...)
+		ms.ents = append(ms.ents, entries...)
+	case uint64(len(ms.ents)) == offset:
+		ms.ents = append(ms.ents, entries...)
+	default:
+		raftLogger.Panicf("missing log entry [last: %d, append at: %d]",
+			ms.lastIndex(), entries[0].Index)
+	}
+	return nil
+}
diff --git a/vendor/github.com/coreos/etcd/raft/util.go b/vendor/github.com/coreos/etcd/raft/util.go
new file mode 100644
index 0000000..f4141fe
--- /dev/null
+++ b/vendor/github.com/coreos/etcd/raft/util.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 raft
+
+import (
+	"bytes"
+	"fmt"
+
+	pb "github.com/coreos/etcd/raft/raftpb"
+)
+
+func (st StateType) MarshalJSON() ([]byte, error) {
+	return []byte(fmt.Sprintf("%q", st.String())), nil
+}
+
+// uint64Slice implements sort interface
+type uint64Slice []uint64
+
+func (p uint64Slice) Len() int           { return len(p) }
+func (p uint64Slice) Less(i, j int) bool { return p[i] < p[j] }
+func (p uint64Slice) Swap(i, j int)      { p[i], p[j] = p[j], p[i] }
+
+func min(a, b uint64) uint64 {
+	if a > b {
+		return b
+	}
+	return a
+}
+
+func max(a, b uint64) uint64 {
+	if a > b {
+		return a
+	}
+	return b
+}
+
+func IsLocalMsg(msgt pb.MessageType) bool {
+	return msgt == pb.MsgHup || msgt == pb.MsgBeat || msgt == pb.MsgUnreachable ||
+		msgt == pb.MsgSnapStatus || msgt == pb.MsgCheckQuorum
+}
+
+func IsResponseMsg(msgt pb.MessageType) bool {
+	return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable || msgt == pb.MsgPreVoteResp
+}
+
+// voteResponseType maps vote and prevote message types to their corresponding responses.
+func voteRespMsgType(msgt pb.MessageType) pb.MessageType {
+	switch msgt {
+	case pb.MsgVote:
+		return pb.MsgVoteResp
+	case pb.MsgPreVote:
+		return pb.MsgPreVoteResp
+	default:
+		panic(fmt.Sprintf("not a vote message: %s", msgt))
+	}
+}
+
+// EntryFormatter can be implemented by the application to provide human-readable formatting
+// of entry data. Nil is a valid EntryFormatter and will use a default format.
+type EntryFormatter func([]byte) string
+
+// DescribeMessage returns a concise human-readable description of a
+// Message for debugging.
+func DescribeMessage(m pb.Message, f EntryFormatter) string {
+	var buf bytes.Buffer
+	fmt.Fprintf(&buf, "%x->%x %v Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index)
+	if m.Reject {
+		fmt.Fprintf(&buf, " Rejected")
+		if m.RejectHint != 0 {
+			fmt.Fprintf(&buf, "(Hint:%d)", m.RejectHint)
+		}
+	}
+	if m.Commit != 0 {
+		fmt.Fprintf(&buf, " Commit:%d", m.Commit)
+	}
+	if len(m.Entries) > 0 {
+		fmt.Fprintf(&buf, " Entries:[")
+		for i, e := range m.Entries {
+			if i != 0 {
+				buf.WriteString(", ")
+			}
+			buf.WriteString(DescribeEntry(e, f))
+		}
+		fmt.Fprintf(&buf, "]")
+	}
+	if !IsEmptySnap(m.Snapshot) {
+		fmt.Fprintf(&buf, " Snapshot:%v", m.Snapshot)
+	}
+	return buf.String()
+}
+
+// DescribeEntry returns a concise human-readable description of an
+// Entry for debugging.
+func DescribeEntry(e pb.Entry, f EntryFormatter) string {
+	var formatted string
+	if e.Type == pb.EntryNormal && f != nil {
+		formatted = f(e.Data)
+	} else {
+		formatted = fmt.Sprintf("%q", e.Data)
+	}
+	return fmt.Sprintf("%d/%d %s %s", e.Term, e.Index, e.Type, formatted)
+}
+
+func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry {
+	if len(ents) == 0 {
+		return ents
+	}
+	size := ents[0].Size()
+	var limit int
+	for limit = 1; limit < len(ents); limit++ {
+		size += ents[limit].Size()
+		if uint64(size) > maxSize {
+			break
+		}
+	}
+	return ents[:limit]
+}
diff --git a/vendor/github.com/coreos/etcd/version/version.go b/vendor/github.com/coreos/etcd/version/version.go
new file mode 100644
index 0000000..aa96ffa
--- /dev/null
+++ b/vendor/github.com/coreos/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.3.25"
+	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/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/NOTICE b/vendor/github.com/coreos/go-semver/NOTICE
new file mode 100644
index 0000000..23a0ada
--- /dev/null
+++ b/vendor/github.com/coreos/go-semver/NOTICE
@@ -0,0 +1,5 @@
+CoreOS Project
+Copyright 2018 CoreOS, Inc
+
+This product includes software developed at CoreOS, Inc.
+(http://www.coreos.com/).
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..76cf485
--- /dev/null
+++ b/vendor/github.com/coreos/go-semver/semver/semver.go
@@ -0,0 +1,296 @@
+// 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"
+	"regexp"
+	"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)
+	}
+
+	if err := validateIdentifier(string(preRelease)); err != nil {
+		return fmt.Errorf("failed to validate pre-release: %v", err)
+	}
+
+	if err := validateIdentifier(metadata); err != nil {
+		return fmt.Errorf("failed to validate metadata: %v", err)
+	}
+
+	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
+	}
+
+	// Numeric identifiers always have lower precedence than non-numeric identifiers.
+	if aInt && !bInt {
+		return -1
+	} else if !aInt && bInt {
+		return 1
+	}
+
+	// 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 = ""
+}
+
+// validateIdentifier makes sure the provided identifier satisfies semver spec
+func validateIdentifier(id string) error {
+	if id != "" && !reIdentifier.MatchString(id) {
+		return fmt.Errorf("%s is not a valid semver identifier", id)
+	}
+	return nil
+}
+
+// reIdentifier is a regular expression used to check that pre-release and metadata
+// identifiers satisfy the spec requirements
+var reIdentifier = regexp.MustCompile(`^[0-9A-Za-z-]+(\.[0-9A-Za-z-]+)*$`)
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/coreos/pkg/capnslog/init.go b/vendor/github.com/coreos/pkg/capnslog/init.go
index 44b8cd3..38ce6d2 100644
--- a/vendor/github.com/coreos/pkg/capnslog/init.go
+++ b/vendor/github.com/coreos/pkg/capnslog/init.go
@@ -32,7 +32,7 @@
 func init() {
 	initHijack()
 
-	// Go `log` pacakge uses os.Stderr.
+	// Go `log` package uses os.Stderr.
 	SetFormatter(NewDefaultFormatter(os.Stderr))
 	SetGlobalLogLevel(INFO)
 }
diff --git a/vendor/github.com/golang/protobuf/proto/buffer.go b/vendor/github.com/golang/protobuf/proto/buffer.go
new file mode 100644
index 0000000..e810e6f
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/buffer.go
@@ -0,0 +1,324 @@
+// 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.
+
+package proto
+
+import (
+	"errors"
+	"fmt"
+
+	"google.golang.org/protobuf/encoding/prototext"
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	WireVarint     = 0
+	WireFixed32    = 5
+	WireFixed64    = 1
+	WireBytes      = 2
+	WireStartGroup = 3
+	WireEndGroup   = 4
+)
+
+// EncodeVarint returns the varint encoded bytes of v.
+func EncodeVarint(v uint64) []byte {
+	return protowire.AppendVarint(nil, v)
+}
+
+// SizeVarint returns the length of the varint encoded bytes of v.
+// This is equal to len(EncodeVarint(v)).
+func SizeVarint(v uint64) int {
+	return protowire.SizeVarint(v)
+}
+
+// DecodeVarint parses a varint encoded integer from b,
+// returning the integer value and the length of the varint.
+// It returns (0, 0) if there is a parse error.
+func DecodeVarint(b []byte) (uint64, int) {
+	v, n := protowire.ConsumeVarint(b)
+	if n < 0 {
+		return 0, 0
+	}
+	return v, n
+}
+
+// Buffer is a buffer for encoding and decoding the protobuf wire format.
+// It may be reused between invocations to reduce memory usage.
+type Buffer struct {
+	buf           []byte
+	idx           int
+	deterministic bool
+}
+
+// NewBuffer allocates a new Buffer initialized with buf,
+// where the contents of buf are considered the unread portion of the buffer.
+func NewBuffer(buf []byte) *Buffer {
+	return &Buffer{buf: buf}
+}
+
+// SetDeterministic specifies whether to use deterministic serialization.
+//
+// Deterministic serialization guarantees that for a given binary, equal
+// messages will always be serialized to the same bytes. This implies:
+//
+//   - Repeated serialization of a message will return the same bytes.
+//   - Different processes of the same binary (which may be executing on
+//     different machines) will serialize equal messages to the same bytes.
+//
+// Note that the deterministic serialization is NOT canonical across
+// languages. It is not guaranteed to remain stable over time. It is unstable
+// across different builds with schema changes due to unknown fields.
+// Users who need canonical serialization (e.g., persistent storage in a
+// canonical form, fingerprinting, etc.) should define their own
+// canonicalization specification and implement their own serializer rather
+// than relying on this API.
+//
+// If deterministic serialization is requested, map entries will be sorted
+// by keys in lexographical order. This is an implementation detail and
+// subject to change.
+func (b *Buffer) SetDeterministic(deterministic bool) {
+	b.deterministic = deterministic
+}
+
+// SetBuf sets buf as the internal buffer,
+// where the contents of buf are considered the unread portion of the buffer.
+func (b *Buffer) SetBuf(buf []byte) {
+	b.buf = buf
+	b.idx = 0
+}
+
+// Reset clears the internal buffer of all written and unread data.
+func (b *Buffer) Reset() {
+	b.buf = b.buf[:0]
+	b.idx = 0
+}
+
+// Bytes returns the internal buffer.
+func (b *Buffer) Bytes() []byte {
+	return b.buf
+}
+
+// Unread returns the unread portion of the buffer.
+func (b *Buffer) Unread() []byte {
+	return b.buf[b.idx:]
+}
+
+// Marshal appends the wire-format encoding of m to the buffer.
+func (b *Buffer) Marshal(m Message) error {
+	var err error
+	b.buf, err = marshalAppend(b.buf, m, b.deterministic)
+	return err
+}
+
+// Unmarshal parses the wire-format message in the buffer and
+// places the decoded results in m.
+// It does not reset m before unmarshaling.
+func (b *Buffer) Unmarshal(m Message) error {
+	err := UnmarshalMerge(b.Unread(), m)
+	b.idx = len(b.buf)
+	return err
+}
+
+type unknownFields struct{ XXX_unrecognized protoimpl.UnknownFields }
+
+func (m *unknownFields) String() string { panic("not implemented") }
+func (m *unknownFields) Reset()         { panic("not implemented") }
+func (m *unknownFields) ProtoMessage()  { panic("not implemented") }
+
+// DebugPrint dumps the encoded bytes of b with a header and footer including s
+// to stdout. This is only intended for debugging.
+func (*Buffer) DebugPrint(s string, b []byte) {
+	m := MessageReflect(new(unknownFields))
+	m.SetUnknown(b)
+	b, _ = prototext.MarshalOptions{AllowPartial: true, Indent: "\t"}.Marshal(m.Interface())
+	fmt.Printf("==== %s ====\n%s==== %s ====\n", s, b, s)
+}
+
+// EncodeVarint appends an unsigned varint encoding to the buffer.
+func (b *Buffer) EncodeVarint(v uint64) error {
+	b.buf = protowire.AppendVarint(b.buf, v)
+	return nil
+}
+
+// EncodeZigzag32 appends a 32-bit zig-zag varint encoding to the buffer.
+func (b *Buffer) EncodeZigzag32(v uint64) error {
+	return b.EncodeVarint(uint64((uint32(v) << 1) ^ uint32((int32(v) >> 31))))
+}
+
+// EncodeZigzag64 appends a 64-bit zig-zag varint encoding to the buffer.
+func (b *Buffer) EncodeZigzag64(v uint64) error {
+	return b.EncodeVarint(uint64((uint64(v) << 1) ^ uint64((int64(v) >> 63))))
+}
+
+// EncodeFixed32 appends a 32-bit little-endian integer to the buffer.
+func (b *Buffer) EncodeFixed32(v uint64) error {
+	b.buf = protowire.AppendFixed32(b.buf, uint32(v))
+	return nil
+}
+
+// EncodeFixed64 appends a 64-bit little-endian integer to the buffer.
+func (b *Buffer) EncodeFixed64(v uint64) error {
+	b.buf = protowire.AppendFixed64(b.buf, uint64(v))
+	return nil
+}
+
+// EncodeRawBytes appends a length-prefixed raw bytes to the buffer.
+func (b *Buffer) EncodeRawBytes(v []byte) error {
+	b.buf = protowire.AppendBytes(b.buf, v)
+	return nil
+}
+
+// EncodeStringBytes appends a length-prefixed raw bytes to the buffer.
+// It does not validate whether v contains valid UTF-8.
+func (b *Buffer) EncodeStringBytes(v string) error {
+	b.buf = protowire.AppendString(b.buf, v)
+	return nil
+}
+
+// EncodeMessage appends a length-prefixed encoded message to the buffer.
+func (b *Buffer) EncodeMessage(m Message) error {
+	var err error
+	b.buf = protowire.AppendVarint(b.buf, uint64(Size(m)))
+	b.buf, err = marshalAppend(b.buf, m, b.deterministic)
+	return err
+}
+
+// DecodeVarint consumes an encoded unsigned varint from the buffer.
+func (b *Buffer) DecodeVarint() (uint64, error) {
+	v, n := protowire.ConsumeVarint(b.buf[b.idx:])
+	if n < 0 {
+		return 0, protowire.ParseError(n)
+	}
+	b.idx += n
+	return uint64(v), nil
+}
+
+// DecodeZigzag32 consumes an encoded 32-bit zig-zag varint from the buffer.
+func (b *Buffer) DecodeZigzag32() (uint64, error) {
+	v, err := b.DecodeVarint()
+	if err != nil {
+		return 0, err
+	}
+	return uint64((uint32(v) >> 1) ^ uint32((int32(v&1)<<31)>>31)), nil
+}
+
+// DecodeZigzag64 consumes an encoded 64-bit zig-zag varint from the buffer.
+func (b *Buffer) DecodeZigzag64() (uint64, error) {
+	v, err := b.DecodeVarint()
+	if err != nil {
+		return 0, err
+	}
+	return uint64((uint64(v) >> 1) ^ uint64((int64(v&1)<<63)>>63)), nil
+}
+
+// DecodeFixed32 consumes a 32-bit little-endian integer from the buffer.
+func (b *Buffer) DecodeFixed32() (uint64, error) {
+	v, n := protowire.ConsumeFixed32(b.buf[b.idx:])
+	if n < 0 {
+		return 0, protowire.ParseError(n)
+	}
+	b.idx += n
+	return uint64(v), nil
+}
+
+// DecodeFixed64 consumes a 64-bit little-endian integer from the buffer.
+func (b *Buffer) DecodeFixed64() (uint64, error) {
+	v, n := protowire.ConsumeFixed64(b.buf[b.idx:])
+	if n < 0 {
+		return 0, protowire.ParseError(n)
+	}
+	b.idx += n
+	return uint64(v), nil
+}
+
+// DecodeRawBytes consumes a length-prefixed raw bytes from the buffer.
+// If alloc is specified, it returns a copy the raw bytes
+// rather than a sub-slice of the buffer.
+func (b *Buffer) DecodeRawBytes(alloc bool) ([]byte, error) {
+	v, n := protowire.ConsumeBytes(b.buf[b.idx:])
+	if n < 0 {
+		return nil, protowire.ParseError(n)
+	}
+	b.idx += n
+	if alloc {
+		v = append([]byte(nil), v...)
+	}
+	return v, nil
+}
+
+// DecodeStringBytes consumes a length-prefixed raw bytes from the buffer.
+// It does not validate whether the raw bytes contain valid UTF-8.
+func (b *Buffer) DecodeStringBytes() (string, error) {
+	v, n := protowire.ConsumeString(b.buf[b.idx:])
+	if n < 0 {
+		return "", protowire.ParseError(n)
+	}
+	b.idx += n
+	return v, nil
+}
+
+// DecodeMessage consumes a length-prefixed message from the buffer.
+// It does not reset m before unmarshaling.
+func (b *Buffer) DecodeMessage(m Message) error {
+	v, err := b.DecodeRawBytes(false)
+	if err != nil {
+		return err
+	}
+	return UnmarshalMerge(v, m)
+}
+
+// DecodeGroup consumes a message group from the buffer.
+// It assumes that the start group marker has already been consumed and
+// consumes all bytes until (and including the end group marker).
+// It does not reset m before unmarshaling.
+func (b *Buffer) DecodeGroup(m Message) error {
+	v, n, err := consumeGroup(b.buf[b.idx:])
+	if err != nil {
+		return err
+	}
+	b.idx += n
+	return UnmarshalMerge(v, m)
+}
+
+// consumeGroup parses b until it finds an end group marker, returning
+// the raw bytes of the message (excluding the end group marker) and the
+// the total length of the message (including the end group marker).
+func consumeGroup(b []byte) ([]byte, int, error) {
+	b0 := b
+	depth := 1 // assume this follows a start group marker
+	for {
+		_, wtyp, tagLen := protowire.ConsumeTag(b)
+		if tagLen < 0 {
+			return nil, 0, protowire.ParseError(tagLen)
+		}
+		b = b[tagLen:]
+
+		var valLen int
+		switch wtyp {
+		case protowire.VarintType:
+			_, valLen = protowire.ConsumeVarint(b)
+		case protowire.Fixed32Type:
+			_, valLen = protowire.ConsumeFixed32(b)
+		case protowire.Fixed64Type:
+			_, valLen = protowire.ConsumeFixed64(b)
+		case protowire.BytesType:
+			_, valLen = protowire.ConsumeBytes(b)
+		case protowire.StartGroupType:
+			depth++
+		case protowire.EndGroupType:
+			depth--
+		default:
+			return nil, 0, errors.New("proto: cannot parse reserved wire type")
+		}
+		if valLen < 0 {
+			return nil, 0, protowire.ParseError(valLen)
+		}
+		b = b[valLen:]
+
+		if depth == 0 {
+			return b0[:len(b0)-len(b)-tagLen], len(b0) - len(b), nil
+		}
+	}
+}
diff --git a/vendor/github.com/golang/protobuf/proto/clone.go b/vendor/github.com/golang/protobuf/proto/clone.go
deleted file mode 100644
index 3cd3249..0000000
--- a/vendor/github.com/golang/protobuf/proto/clone.go
+++ /dev/null
@@ -1,253 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2011 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.
-
-// Protocol buffer deep copy and merge.
-// TODO: RawMessage.
-
-package proto
-
-import (
-	"fmt"
-	"log"
-	"reflect"
-	"strings"
-)
-
-// Clone returns a deep copy of a protocol buffer.
-func Clone(src Message) Message {
-	in := reflect.ValueOf(src)
-	if in.IsNil() {
-		return src
-	}
-	out := reflect.New(in.Type().Elem())
-	dst := out.Interface().(Message)
-	Merge(dst, src)
-	return dst
-}
-
-// Merger is the interface representing objects that can merge messages of the same type.
-type Merger interface {
-	// Merge merges src into this message.
-	// Required and optional fields that are set in src will be set to that value in dst.
-	// Elements of repeated fields will be appended.
-	//
-	// Merge may panic if called with a different argument type than the receiver.
-	Merge(src Message)
-}
-
-// generatedMerger is the custom merge method that generated protos will have.
-// We must add this method since a generate Merge method will conflict with
-// many existing protos that have a Merge data field already defined.
-type generatedMerger interface {
-	XXX_Merge(src Message)
-}
-
-// Merge merges src into dst.
-// Required and optional fields that are set in src will be set to that value in dst.
-// Elements of repeated fields will be appended.
-// Merge panics if src and dst are not the same type, or if dst is nil.
-func Merge(dst, src Message) {
-	if m, ok := dst.(Merger); ok {
-		m.Merge(src)
-		return
-	}
-
-	in := reflect.ValueOf(src)
-	out := reflect.ValueOf(dst)
-	if out.IsNil() {
-		panic("proto: nil destination")
-	}
-	if in.Type() != out.Type() {
-		panic(fmt.Sprintf("proto.Merge(%T, %T) type mismatch", dst, src))
-	}
-	if in.IsNil() {
-		return // Merge from nil src is a noop
-	}
-	if m, ok := dst.(generatedMerger); ok {
-		m.XXX_Merge(src)
-		return
-	}
-	mergeStruct(out.Elem(), in.Elem())
-}
-
-func mergeStruct(out, in reflect.Value) {
-	sprop := GetProperties(in.Type())
-	for i := 0; i < in.NumField(); i++ {
-		f := in.Type().Field(i)
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-		mergeAny(out.Field(i), in.Field(i), false, sprop.Prop[i])
-	}
-
-	if emIn, err := extendable(in.Addr().Interface()); err == nil {
-		emOut, _ := extendable(out.Addr().Interface())
-		mIn, muIn := emIn.extensionsRead()
-		if mIn != nil {
-			mOut := emOut.extensionsWrite()
-			muIn.Lock()
-			mergeExtension(mOut, mIn)
-			muIn.Unlock()
-		}
-	}
-
-	uf := in.FieldByName("XXX_unrecognized")
-	if !uf.IsValid() {
-		return
-	}
-	uin := uf.Bytes()
-	if len(uin) > 0 {
-		out.FieldByName("XXX_unrecognized").SetBytes(append([]byte(nil), uin...))
-	}
-}
-
-// mergeAny performs a merge between two values of the same type.
-// viaPtr indicates whether the values were indirected through a pointer (implying proto2).
-// prop is set if this is a struct field (it may be nil).
-func mergeAny(out, in reflect.Value, viaPtr bool, prop *Properties) {
-	if in.Type() == protoMessageType {
-		if !in.IsNil() {
-			if out.IsNil() {
-				out.Set(reflect.ValueOf(Clone(in.Interface().(Message))))
-			} else {
-				Merge(out.Interface().(Message), in.Interface().(Message))
-			}
-		}
-		return
-	}
-	switch in.Kind() {
-	case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64,
-		reflect.String, reflect.Uint32, reflect.Uint64:
-		if !viaPtr && isProto3Zero(in) {
-			return
-		}
-		out.Set(in)
-	case reflect.Interface:
-		// Probably a oneof field; copy non-nil values.
-		if in.IsNil() {
-			return
-		}
-		// Allocate destination if it is not set, or set to a different type.
-		// Otherwise we will merge as normal.
-		if out.IsNil() || out.Elem().Type() != in.Elem().Type() {
-			out.Set(reflect.New(in.Elem().Elem().Type())) // interface -> *T -> T -> new(T)
-		}
-		mergeAny(out.Elem(), in.Elem(), false, nil)
-	case reflect.Map:
-		if in.Len() == 0 {
-			return
-		}
-		if out.IsNil() {
-			out.Set(reflect.MakeMap(in.Type()))
-		}
-		// For maps with value types of *T or []byte we need to deep copy each value.
-		elemKind := in.Type().Elem().Kind()
-		for _, key := range in.MapKeys() {
-			var val reflect.Value
-			switch elemKind {
-			case reflect.Ptr:
-				val = reflect.New(in.Type().Elem().Elem())
-				mergeAny(val, in.MapIndex(key), false, nil)
-			case reflect.Slice:
-				val = in.MapIndex(key)
-				val = reflect.ValueOf(append([]byte{}, val.Bytes()...))
-			default:
-				val = in.MapIndex(key)
-			}
-			out.SetMapIndex(key, val)
-		}
-	case reflect.Ptr:
-		if in.IsNil() {
-			return
-		}
-		if out.IsNil() {
-			out.Set(reflect.New(in.Elem().Type()))
-		}
-		mergeAny(out.Elem(), in.Elem(), true, nil)
-	case reflect.Slice:
-		if in.IsNil() {
-			return
-		}
-		if in.Type().Elem().Kind() == reflect.Uint8 {
-			// []byte is a scalar bytes field, not a repeated field.
-
-			// Edge case: if this is in a proto3 message, a zero length
-			// bytes field is considered the zero value, and should not
-			// be merged.
-			if prop != nil && prop.proto3 && in.Len() == 0 {
-				return
-			}
-
-			// Make a deep copy.
-			// Append to []byte{} instead of []byte(nil) so that we never end up
-			// with a nil result.
-			out.SetBytes(append([]byte{}, in.Bytes()...))
-			return
-		}
-		n := in.Len()
-		if out.IsNil() {
-			out.Set(reflect.MakeSlice(in.Type(), 0, n))
-		}
-		switch in.Type().Elem().Kind() {
-		case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64,
-			reflect.String, reflect.Uint32, reflect.Uint64:
-			out.Set(reflect.AppendSlice(out, in))
-		default:
-			for i := 0; i < n; i++ {
-				x := reflect.Indirect(reflect.New(in.Type().Elem()))
-				mergeAny(x, in.Index(i), false, nil)
-				out.Set(reflect.Append(out, x))
-			}
-		}
-	case reflect.Struct:
-		mergeStruct(out, in)
-	default:
-		// unknown type, so not a protocol buffer
-		log.Printf("proto: don't know how to copy %v", in)
-	}
-}
-
-func mergeExtension(out, in map[int32]Extension) {
-	for extNum, eIn := range in {
-		eOut := Extension{desc: eIn.desc}
-		if eIn.value != nil {
-			v := reflect.New(reflect.TypeOf(eIn.value)).Elem()
-			mergeAny(v, reflect.ValueOf(eIn.value), false, nil)
-			eOut.value = v.Interface()
-		}
-		if eIn.enc != nil {
-			eOut.enc = make([]byte, len(eIn.enc))
-			copy(eOut.enc, eIn.enc)
-		}
-
-		out[extNum] = eOut
-	}
-}
diff --git a/vendor/github.com/golang/protobuf/proto/decode.go b/vendor/github.com/golang/protobuf/proto/decode.go
deleted file mode 100644
index 63b0f08..0000000
--- a/vendor/github.com/golang/protobuf/proto/decode.go
+++ /dev/null
@@ -1,427 +0,0 @@
-// 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.
-
-package proto
-
-/*
- * Routines for decoding protocol buffer data to construct in-memory representations.
- */
-
-import (
-	"errors"
-	"fmt"
-	"io"
-)
-
-// errOverflow is returned when an integer is too large to be represented.
-var errOverflow = errors.New("proto: integer overflow")
-
-// ErrInternalBadWireType is returned by generated code when an incorrect
-// wire type is encountered. It does not get returned to user code.
-var ErrInternalBadWireType = errors.New("proto: internal error: bad wiretype for oneof")
-
-// DecodeVarint reads a varint-encoded integer from the slice.
-// It returns the integer and the number of bytes consumed, or
-// zero if there is not enough.
-// This is the format for the
-// int32, int64, uint32, uint64, bool, and enum
-// protocol buffer types.
-func DecodeVarint(buf []byte) (x uint64, n int) {
-	for shift := uint(0); shift < 64; shift += 7 {
-		if n >= len(buf) {
-			return 0, 0
-		}
-		b := uint64(buf[n])
-		n++
-		x |= (b & 0x7F) << shift
-		if (b & 0x80) == 0 {
-			return x, n
-		}
-	}
-
-	// The number is too large to represent in a 64-bit value.
-	return 0, 0
-}
-
-func (p *Buffer) decodeVarintSlow() (x uint64, err error) {
-	i := p.index
-	l := len(p.buf)
-
-	for shift := uint(0); shift < 64; shift += 7 {
-		if i >= l {
-			err = io.ErrUnexpectedEOF
-			return
-		}
-		b := p.buf[i]
-		i++
-		x |= (uint64(b) & 0x7F) << shift
-		if b < 0x80 {
-			p.index = i
-			return
-		}
-	}
-
-	// The number is too large to represent in a 64-bit value.
-	err = errOverflow
-	return
-}
-
-// DecodeVarint reads a varint-encoded integer from the Buffer.
-// This is the format for the
-// int32, int64, uint32, uint64, bool, and enum
-// protocol buffer types.
-func (p *Buffer) DecodeVarint() (x uint64, err error) {
-	i := p.index
-	buf := p.buf
-
-	if i >= len(buf) {
-		return 0, io.ErrUnexpectedEOF
-	} else if buf[i] < 0x80 {
-		p.index++
-		return uint64(buf[i]), nil
-	} else if len(buf)-i < 10 {
-		return p.decodeVarintSlow()
-	}
-
-	var b uint64
-	// we already checked the first byte
-	x = uint64(buf[i]) - 0x80
-	i++
-
-	b = uint64(buf[i])
-	i++
-	x += b << 7
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 7
-
-	b = uint64(buf[i])
-	i++
-	x += b << 14
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 14
-
-	b = uint64(buf[i])
-	i++
-	x += b << 21
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 21
-
-	b = uint64(buf[i])
-	i++
-	x += b << 28
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 28
-
-	b = uint64(buf[i])
-	i++
-	x += b << 35
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 35
-
-	b = uint64(buf[i])
-	i++
-	x += b << 42
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 42
-
-	b = uint64(buf[i])
-	i++
-	x += b << 49
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 49
-
-	b = uint64(buf[i])
-	i++
-	x += b << 56
-	if b&0x80 == 0 {
-		goto done
-	}
-	x -= 0x80 << 56
-
-	b = uint64(buf[i])
-	i++
-	x += b << 63
-	if b&0x80 == 0 {
-		goto done
-	}
-
-	return 0, errOverflow
-
-done:
-	p.index = i
-	return x, nil
-}
-
-// DecodeFixed64 reads a 64-bit integer from the Buffer.
-// This is the format for the
-// fixed64, sfixed64, and double protocol buffer types.
-func (p *Buffer) DecodeFixed64() (x uint64, err error) {
-	// x, err already 0
-	i := p.index + 8
-	if i < 0 || i > len(p.buf) {
-		err = io.ErrUnexpectedEOF
-		return
-	}
-	p.index = i
-
-	x = uint64(p.buf[i-8])
-	x |= uint64(p.buf[i-7]) << 8
-	x |= uint64(p.buf[i-6]) << 16
-	x |= uint64(p.buf[i-5]) << 24
-	x |= uint64(p.buf[i-4]) << 32
-	x |= uint64(p.buf[i-3]) << 40
-	x |= uint64(p.buf[i-2]) << 48
-	x |= uint64(p.buf[i-1]) << 56
-	return
-}
-
-// DecodeFixed32 reads a 32-bit integer from the Buffer.
-// This is the format for the
-// fixed32, sfixed32, and float protocol buffer types.
-func (p *Buffer) DecodeFixed32() (x uint64, err error) {
-	// x, err already 0
-	i := p.index + 4
-	if i < 0 || i > len(p.buf) {
-		err = io.ErrUnexpectedEOF
-		return
-	}
-	p.index = i
-
-	x = uint64(p.buf[i-4])
-	x |= uint64(p.buf[i-3]) << 8
-	x |= uint64(p.buf[i-2]) << 16
-	x |= uint64(p.buf[i-1]) << 24
-	return
-}
-
-// DecodeZigzag64 reads a zigzag-encoded 64-bit integer
-// from the Buffer.
-// This is the format used for the sint64 protocol buffer type.
-func (p *Buffer) DecodeZigzag64() (x uint64, err error) {
-	x, err = p.DecodeVarint()
-	if err != nil {
-		return
-	}
-	x = (x >> 1) ^ uint64((int64(x&1)<<63)>>63)
-	return
-}
-
-// DecodeZigzag32 reads a zigzag-encoded 32-bit integer
-// from  the Buffer.
-// This is the format used for the sint32 protocol buffer type.
-func (p *Buffer) DecodeZigzag32() (x uint64, err error) {
-	x, err = p.DecodeVarint()
-	if err != nil {
-		return
-	}
-	x = uint64((uint32(x) >> 1) ^ uint32((int32(x&1)<<31)>>31))
-	return
-}
-
-// DecodeRawBytes reads a count-delimited byte buffer from the Buffer.
-// This is the format used for the bytes protocol buffer
-// type and for embedded messages.
-func (p *Buffer) DecodeRawBytes(alloc bool) (buf []byte, err error) {
-	n, err := p.DecodeVarint()
-	if err != nil {
-		return nil, err
-	}
-
-	nb := int(n)
-	if nb < 0 {
-		return nil, fmt.Errorf("proto: bad byte length %d", nb)
-	}
-	end := p.index + nb
-	if end < p.index || end > len(p.buf) {
-		return nil, io.ErrUnexpectedEOF
-	}
-
-	if !alloc {
-		// todo: check if can get more uses of alloc=false
-		buf = p.buf[p.index:end]
-		p.index += nb
-		return
-	}
-
-	buf = make([]byte, nb)
-	copy(buf, p.buf[p.index:])
-	p.index += nb
-	return
-}
-
-// DecodeStringBytes reads an encoded string from the Buffer.
-// This is the format used for the proto2 string type.
-func (p *Buffer) DecodeStringBytes() (s string, err error) {
-	buf, err := p.DecodeRawBytes(false)
-	if err != nil {
-		return
-	}
-	return string(buf), nil
-}
-
-// Unmarshaler is the interface representing objects that can
-// unmarshal themselves.  The argument points to data that may be
-// overwritten, so implementations should not keep references to the
-// buffer.
-// Unmarshal implementations should not clear the receiver.
-// Any unmarshaled data should be merged into the receiver.
-// Callers of Unmarshal that do not want to retain existing data
-// should Reset the receiver before calling Unmarshal.
-type Unmarshaler interface {
-	Unmarshal([]byte) error
-}
-
-// newUnmarshaler is the interface representing objects that can
-// unmarshal themselves. The semantics are identical to Unmarshaler.
-//
-// This exists to support protoc-gen-go generated messages.
-// The proto package will stop type-asserting to this interface in the future.
-//
-// DO NOT DEPEND ON THIS.
-type newUnmarshaler interface {
-	XXX_Unmarshal([]byte) error
-}
-
-// Unmarshal parses the protocol buffer representation in buf and places the
-// decoded result in pb.  If the struct underlying pb does not match
-// the data in buf, the results can be unpredictable.
-//
-// Unmarshal resets pb before starting to unmarshal, so any
-// existing data in pb is always removed. Use UnmarshalMerge
-// to preserve and append to existing data.
-func Unmarshal(buf []byte, pb Message) error {
-	pb.Reset()
-	if u, ok := pb.(newUnmarshaler); ok {
-		return u.XXX_Unmarshal(buf)
-	}
-	if u, ok := pb.(Unmarshaler); ok {
-		return u.Unmarshal(buf)
-	}
-	return NewBuffer(buf).Unmarshal(pb)
-}
-
-// UnmarshalMerge parses the protocol buffer representation in buf and
-// writes the decoded result to pb.  If the struct underlying pb does not match
-// the data in buf, the results can be unpredictable.
-//
-// UnmarshalMerge merges into existing data in pb.
-// Most code should use Unmarshal instead.
-func UnmarshalMerge(buf []byte, pb Message) error {
-	if u, ok := pb.(newUnmarshaler); ok {
-		return u.XXX_Unmarshal(buf)
-	}
-	if u, ok := pb.(Unmarshaler); ok {
-		// NOTE: The history of proto have unfortunately been inconsistent
-		// whether Unmarshaler should or should not implicitly clear itself.
-		// Some implementations do, most do not.
-		// Thus, calling this here may or may not do what people want.
-		//
-		// See https://github.com/golang/protobuf/issues/424
-		return u.Unmarshal(buf)
-	}
-	return NewBuffer(buf).Unmarshal(pb)
-}
-
-// DecodeMessage reads a count-delimited message from the Buffer.
-func (p *Buffer) DecodeMessage(pb Message) error {
-	enc, err := p.DecodeRawBytes(false)
-	if err != nil {
-		return err
-	}
-	return NewBuffer(enc).Unmarshal(pb)
-}
-
-// DecodeGroup reads a tag-delimited group from the Buffer.
-// StartGroup tag is already consumed. This function consumes
-// EndGroup tag.
-func (p *Buffer) DecodeGroup(pb Message) error {
-	b := p.buf[p.index:]
-	x, y := findEndGroup(b)
-	if x < 0 {
-		return io.ErrUnexpectedEOF
-	}
-	err := Unmarshal(b[:x], pb)
-	p.index += y
-	return err
-}
-
-// Unmarshal parses the protocol buffer representation in the
-// Buffer and places the decoded result in pb.  If the struct
-// underlying pb does not match the data in the buffer, the results can be
-// unpredictable.
-//
-// Unlike proto.Unmarshal, this does not reset pb before starting to unmarshal.
-func (p *Buffer) Unmarshal(pb Message) error {
-	// If the object can unmarshal itself, let it.
-	if u, ok := pb.(newUnmarshaler); ok {
-		err := u.XXX_Unmarshal(p.buf[p.index:])
-		p.index = len(p.buf)
-		return err
-	}
-	if u, ok := pb.(Unmarshaler); ok {
-		// NOTE: The history of proto have unfortunately been inconsistent
-		// whether Unmarshaler should or should not implicitly clear itself.
-		// Some implementations do, most do not.
-		// Thus, calling this here may or may not do what people want.
-		//
-		// See https://github.com/golang/protobuf/issues/424
-		err := u.Unmarshal(p.buf[p.index:])
-		p.index = len(p.buf)
-		return err
-	}
-
-	// Slow workaround for messages that aren't Unmarshalers.
-	// This includes some hand-coded .pb.go files and
-	// bootstrap protos.
-	// TODO: fix all of those and then add Unmarshal to
-	// the Message interface. Then:
-	// The cast above and code below can be deleted.
-	// The old unmarshaler can be deleted.
-	// Clients can call Unmarshal directly (can already do that, actually).
-	var info InternalMessageInfo
-	err := info.Unmarshal(pb, p.buf[p.index:])
-	p.index = len(p.buf)
-	return err
-}
diff --git a/vendor/github.com/golang/protobuf/proto/defaults.go b/vendor/github.com/golang/protobuf/proto/defaults.go
new file mode 100644
index 0000000..d399bf0
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/defaults.go
@@ -0,0 +1,63 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// SetDefaults sets unpopulated scalar fields to their default values.
+// Fields within a oneof are not set even if they have a default value.
+// SetDefaults is recursively called upon any populated message fields.
+func SetDefaults(m Message) {
+	if m != nil {
+		setDefaults(MessageReflect(m))
+	}
+}
+
+func setDefaults(m protoreflect.Message) {
+	fds := m.Descriptor().Fields()
+	for i := 0; i < fds.Len(); i++ {
+		fd := fds.Get(i)
+		if !m.Has(fd) {
+			if fd.HasDefault() && fd.ContainingOneof() == nil {
+				v := fd.Default()
+				if fd.Kind() == protoreflect.BytesKind {
+					v = protoreflect.ValueOf(append([]byte(nil), v.Bytes()...)) // copy the default bytes
+				}
+				m.Set(fd, v)
+			}
+			continue
+		}
+	}
+
+	m.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		switch {
+		// Handle singular message.
+		case fd.Cardinality() != protoreflect.Repeated:
+			if fd.Message() != nil {
+				setDefaults(m.Get(fd).Message())
+			}
+		// Handle list of messages.
+		case fd.IsList():
+			if fd.Message() != nil {
+				ls := m.Get(fd).List()
+				for i := 0; i < ls.Len(); i++ {
+					setDefaults(ls.Get(i).Message())
+				}
+			}
+		// Handle map of messages.
+		case fd.IsMap():
+			if fd.MapValue().Message() != nil {
+				ms := m.Get(fd).Map()
+				ms.Range(func(_ protoreflect.MapKey, v protoreflect.Value) bool {
+					setDefaults(v.Message())
+					return true
+				})
+			}
+		}
+		return true
+	})
+}
diff --git a/vendor/github.com/golang/protobuf/proto/deprecated.go b/vendor/github.com/golang/protobuf/proto/deprecated.go
index 35b882c..e8db57e 100644
--- a/vendor/github.com/golang/protobuf/proto/deprecated.go
+++ b/vendor/github.com/golang/protobuf/proto/deprecated.go
@@ -1,63 +1,113 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2018 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.
+// 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 proto
 
-import "errors"
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
 
-// Deprecated: do not use.
+	protoV2 "google.golang.org/protobuf/proto"
+)
+
+var (
+	// Deprecated: No longer returned.
+	ErrNil = errors.New("proto: Marshal called with nil")
+
+	// Deprecated: No longer returned.
+	ErrTooLarge = errors.New("proto: message encodes to over 2 GB")
+
+	// Deprecated: No longer returned.
+	ErrInternalBadWireType = errors.New("proto: internal error: bad wiretype for oneof")
+)
+
+// Deprecated: Do not use.
 type Stats struct{ Emalloc, Dmalloc, Encode, Decode, Chit, Cmiss, Size uint64 }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func GetStats() Stats { return Stats{} }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func MarshalMessageSet(interface{}) ([]byte, error) {
 	return nil, errors.New("proto: not implemented")
 }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func UnmarshalMessageSet([]byte, interface{}) error {
 	return errors.New("proto: not implemented")
 }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func MarshalMessageSetJSON(interface{}) ([]byte, error) {
 	return nil, errors.New("proto: not implemented")
 }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func UnmarshalMessageSetJSON([]byte, interface{}) error {
 	return errors.New("proto: not implemented")
 }
 
-// Deprecated: do not use.
+// Deprecated: Do not use.
 func RegisterMessageSetType(Message, int32, string) {}
+
+// Deprecated: Do not use.
+func EnumName(m map[int32]string, v int32) string {
+	s, ok := m[v]
+	if ok {
+		return s
+	}
+	return strconv.Itoa(int(v))
+}
+
+// Deprecated: Do not use.
+func UnmarshalJSONEnum(m map[string]int32, data []byte, enumName string) (int32, error) {
+	if data[0] == '"' {
+		// New style: enums are strings.
+		var repr string
+		if err := json.Unmarshal(data, &repr); err != nil {
+			return -1, err
+		}
+		val, ok := m[repr]
+		if !ok {
+			return 0, fmt.Errorf("unrecognized enum %s value %q", enumName, repr)
+		}
+		return val, nil
+	}
+	// Old style: enums are ints.
+	var val int32
+	if err := json.Unmarshal(data, &val); err != nil {
+		return 0, fmt.Errorf("cannot unmarshal %#q into enum %s", data, enumName)
+	}
+	return val, nil
+}
+
+// Deprecated: Do not use; this type existed for intenal-use only.
+type InternalMessageInfo struct{}
+
+// Deprecated: Do not use; this method existed for intenal-use only.
+func (*InternalMessageInfo) DiscardUnknown(m Message) {
+	DiscardUnknown(m)
+}
+
+// Deprecated: Do not use; this method existed for intenal-use only.
+func (*InternalMessageInfo) Marshal(b []byte, m Message, deterministic bool) ([]byte, error) {
+	return protoV2.MarshalOptions{Deterministic: deterministic}.MarshalAppend(b, MessageV2(m))
+}
+
+// Deprecated: Do not use; this method existed for intenal-use only.
+func (*InternalMessageInfo) Merge(dst, src Message) {
+	protoV2.Merge(MessageV2(dst), MessageV2(src))
+}
+
+// Deprecated: Do not use; this method existed for intenal-use only.
+func (*InternalMessageInfo) Size(m Message) int {
+	return protoV2.Size(MessageV2(m))
+}
+
+// Deprecated: Do not use; this method existed for intenal-use only.
+func (*InternalMessageInfo) Unmarshal(m Message, b []byte) error {
+	return protoV2.UnmarshalOptions{Merge: true}.Unmarshal(b, MessageV2(m))
+}
diff --git a/vendor/github.com/golang/protobuf/proto/discard.go b/vendor/github.com/golang/protobuf/proto/discard.go
index dea2617..2187e87 100644
--- a/vendor/github.com/golang/protobuf/proto/discard.go
+++ b/vendor/github.com/golang/protobuf/proto/discard.go
@@ -1,48 +1,13 @@
-// 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.
+// 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.
 
 package proto
 
 import (
-	"fmt"
-	"reflect"
-	"strings"
-	"sync"
-	"sync/atomic"
+	"google.golang.org/protobuf/reflect/protoreflect"
 )
 
-type generatedDiscarder interface {
-	XXX_DiscardUnknown()
-}
-
 // DiscardUnknown recursively discards all unknown fields from this message
 // and all embedded messages.
 //
@@ -51,300 +16,43 @@
 // marshal to be able to produce a message that continues to have those
 // unrecognized fields. To avoid this, DiscardUnknown is used to
 // explicitly clear the unknown fields after unmarshaling.
-//
-// For proto2 messages, the unknown fields of message extensions are only
-// discarded from messages that have been accessed via GetExtension.
 func DiscardUnknown(m Message) {
-	if m, ok := m.(generatedDiscarder); ok {
-		m.XXX_DiscardUnknown()
-		return
-	}
-	// TODO: Dynamically populate a InternalMessageInfo for legacy messages,
-	// but the master branch has no implementation for InternalMessageInfo,
-	// so it would be more work to replicate that approach.
-	discardLegacy(m)
-}
-
-// DiscardUnknown recursively discards all unknown fields.
-func (a *InternalMessageInfo) DiscardUnknown(m Message) {
-	di := atomicLoadDiscardInfo(&a.discard)
-	if di == nil {
-		di = getDiscardInfo(reflect.TypeOf(m).Elem())
-		atomicStoreDiscardInfo(&a.discard, di)
-	}
-	di.discard(toPointer(&m))
-}
-
-type discardInfo struct {
-	typ reflect.Type
-
-	initialized int32 // 0: only typ is valid, 1: everything is valid
-	lock        sync.Mutex
-
-	fields       []discardFieldInfo
-	unrecognized field
-}
-
-type discardFieldInfo struct {
-	field   field // Offset of field, guaranteed to be valid
-	discard func(src pointer)
-}
-
-var (
-	discardInfoMap  = map[reflect.Type]*discardInfo{}
-	discardInfoLock sync.Mutex
-)
-
-func getDiscardInfo(t reflect.Type) *discardInfo {
-	discardInfoLock.Lock()
-	defer discardInfoLock.Unlock()
-	di := discardInfoMap[t]
-	if di == nil {
-		di = &discardInfo{typ: t}
-		discardInfoMap[t] = di
-	}
-	return di
-}
-
-func (di *discardInfo) discard(src pointer) {
-	if src.isNil() {
-		return // Nothing to do.
-	}
-
-	if atomic.LoadInt32(&di.initialized) == 0 {
-		di.computeDiscardInfo()
-	}
-
-	for _, fi := range di.fields {
-		sfp := src.offset(fi.field)
-		fi.discard(sfp)
-	}
-
-	// For proto2 messages, only discard unknown fields in message extensions
-	// that have been accessed via GetExtension.
-	if em, err := extendable(src.asPointerTo(di.typ).Interface()); err == nil {
-		// Ignore lock since DiscardUnknown is not concurrency safe.
-		emm, _ := em.extensionsRead()
-		for _, mx := range emm {
-			if m, ok := mx.value.(Message); ok {
-				DiscardUnknown(m)
-			}
-		}
-	}
-
-	if di.unrecognized.IsValid() {
-		*src.offset(di.unrecognized).toBytes() = nil
+	if m != nil {
+		discardUnknown(MessageReflect(m))
 	}
 }
 
-func (di *discardInfo) computeDiscardInfo() {
-	di.lock.Lock()
-	defer di.lock.Unlock()
-	if di.initialized != 0 {
-		return
-	}
-	t := di.typ
-	n := t.NumField()
-
-	for i := 0; i < n; i++ {
-		f := t.Field(i)
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-
-		dfi := discardFieldInfo{field: toField(&f)}
-		tf := f.Type
-
-		// Unwrap tf to get its most basic type.
-		var isPointer, isSlice bool
-		if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 {
-			isSlice = true
-			tf = tf.Elem()
-		}
-		if tf.Kind() == reflect.Ptr {
-			isPointer = true
-			tf = tf.Elem()
-		}
-		if isPointer && isSlice && tf.Kind() != reflect.Struct {
-			panic(fmt.Sprintf("%v.%s cannot be a slice of pointers to primitive types", t, f.Name))
-		}
-
-		switch tf.Kind() {
-		case reflect.Struct:
-			switch {
-			case !isPointer:
-				panic(fmt.Sprintf("%v.%s cannot be a direct struct value", t, f.Name))
-			case isSlice: // E.g., []*pb.T
-				di := getDiscardInfo(tf)
-				dfi.discard = func(src pointer) {
-					sps := src.getPointerSlice()
-					for _, sp := range sps {
-						if !sp.isNil() {
-							di.discard(sp)
-						}
-					}
-				}
-			default: // E.g., *pb.T
-				di := getDiscardInfo(tf)
-				dfi.discard = func(src pointer) {
-					sp := src.getPointer()
-					if !sp.isNil() {
-						di.discard(sp)
-					}
+func discardUnknown(m protoreflect.Message) {
+	m.Range(func(fd protoreflect.FieldDescriptor, val protoreflect.Value) bool {
+		switch {
+		// Handle singular message.
+		case fd.Cardinality() != protoreflect.Repeated:
+			if fd.Message() != nil {
+				discardUnknown(m.Get(fd).Message())
+			}
+		// Handle list of messages.
+		case fd.IsList():
+			if fd.Message() != nil {
+				ls := m.Get(fd).List()
+				for i := 0; i < ls.Len(); i++ {
+					discardUnknown(ls.Get(i).Message())
 				}
 			}
-		case reflect.Map:
-			switch {
-			case isPointer || isSlice:
-				panic(fmt.Sprintf("%v.%s cannot be a pointer to a map or a slice of map values", t, f.Name))
-			default: // E.g., map[K]V
-				if tf.Elem().Kind() == reflect.Ptr { // Proto struct (e.g., *T)
-					dfi.discard = func(src pointer) {
-						sm := src.asPointerTo(tf).Elem()
-						if sm.Len() == 0 {
-							return
-						}
-						for _, key := range sm.MapKeys() {
-							val := sm.MapIndex(key)
-							DiscardUnknown(val.Interface().(Message))
-						}
-					}
-				} else {
-					dfi.discard = func(pointer) {} // Noop
-				}
-			}
-		case reflect.Interface:
-			// Must be oneof field.
-			switch {
-			case isPointer || isSlice:
-				panic(fmt.Sprintf("%v.%s cannot be a pointer to a interface or a slice of interface values", t, f.Name))
-			default: // E.g., interface{}
-				// TODO: Make this faster?
-				dfi.discard = func(src pointer) {
-					su := src.asPointerTo(tf).Elem()
-					if !su.IsNil() {
-						sv := su.Elem().Elem().Field(0)
-						if sv.Kind() == reflect.Ptr && sv.IsNil() {
-							return
-						}
-						switch sv.Type().Kind() {
-						case reflect.Ptr: // Proto struct (e.g., *T)
-							DiscardUnknown(sv.Interface().(Message))
-						}
-					}
-				}
-			}
-		default:
-			continue
-		}
-		di.fields = append(di.fields, dfi)
-	}
-
-	di.unrecognized = invalidField
-	if f, ok := t.FieldByName("XXX_unrecognized"); ok {
-		if f.Type != reflect.TypeOf([]byte{}) {
-			panic("expected XXX_unrecognized to be of type []byte")
-		}
-		di.unrecognized = toField(&f)
-	}
-
-	atomic.StoreInt32(&di.initialized, 1)
-}
-
-func discardLegacy(m Message) {
-	v := reflect.ValueOf(m)
-	if v.Kind() != reflect.Ptr || v.IsNil() {
-		return
-	}
-	v = v.Elem()
-	if v.Kind() != reflect.Struct {
-		return
-	}
-	t := v.Type()
-
-	for i := 0; i < v.NumField(); i++ {
-		f := t.Field(i)
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-		vf := v.Field(i)
-		tf := f.Type
-
-		// Unwrap tf to get its most basic type.
-		var isPointer, isSlice bool
-		if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 {
-			isSlice = true
-			tf = tf.Elem()
-		}
-		if tf.Kind() == reflect.Ptr {
-			isPointer = true
-			tf = tf.Elem()
-		}
-		if isPointer && isSlice && tf.Kind() != reflect.Struct {
-			panic(fmt.Sprintf("%T.%s cannot be a slice of pointers to primitive types", m, f.Name))
-		}
-
-		switch tf.Kind() {
-		case reflect.Struct:
-			switch {
-			case !isPointer:
-				panic(fmt.Sprintf("%T.%s cannot be a direct struct value", m, f.Name))
-			case isSlice: // E.g., []*pb.T
-				for j := 0; j < vf.Len(); j++ {
-					discardLegacy(vf.Index(j).Interface().(Message))
-				}
-			default: // E.g., *pb.T
-				discardLegacy(vf.Interface().(Message))
-			}
-		case reflect.Map:
-			switch {
-			case isPointer || isSlice:
-				panic(fmt.Sprintf("%T.%s cannot be a pointer to a map or a slice of map values", m, f.Name))
-			default: // E.g., map[K]V
-				tv := vf.Type().Elem()
-				if tv.Kind() == reflect.Ptr && tv.Implements(protoMessageType) { // Proto struct (e.g., *T)
-					for _, key := range vf.MapKeys() {
-						val := vf.MapIndex(key)
-						discardLegacy(val.Interface().(Message))
-					}
-				}
-			}
-		case reflect.Interface:
-			// Must be oneof field.
-			switch {
-			case isPointer || isSlice:
-				panic(fmt.Sprintf("%T.%s cannot be a pointer to a interface or a slice of interface values", m, f.Name))
-			default: // E.g., test_proto.isCommunique_Union interface
-				if !vf.IsNil() && f.Tag.Get("protobuf_oneof") != "" {
-					vf = vf.Elem() // E.g., *test_proto.Communique_Msg
-					if !vf.IsNil() {
-						vf = vf.Elem()   // E.g., test_proto.Communique_Msg
-						vf = vf.Field(0) // E.g., Proto struct (e.g., *T) or primitive value
-						if vf.Kind() == reflect.Ptr {
-							discardLegacy(vf.Interface().(Message))
-						}
-					}
-				}
+		// Handle map of messages.
+		case fd.IsMap():
+			if fd.MapValue().Message() != nil {
+				ms := m.Get(fd).Map()
+				ms.Range(func(_ protoreflect.MapKey, v protoreflect.Value) bool {
+					discardUnknown(v.Message())
+					return true
+				})
 			}
 		}
-	}
+		return true
+	})
 
-	if vf := v.FieldByName("XXX_unrecognized"); vf.IsValid() {
-		if vf.Type() != reflect.TypeOf([]byte{}) {
-			panic("expected XXX_unrecognized to be of type []byte")
-		}
-		vf.Set(reflect.ValueOf([]byte(nil)))
-	}
-
-	// For proto2 messages, only discard unknown fields in message extensions
-	// that have been accessed via GetExtension.
-	if em, err := extendable(m); err == nil {
-		// Ignore lock since discardLegacy is not concurrency safe.
-		emm, _ := em.extensionsRead()
-		for _, mx := range emm {
-			if m, ok := mx.value.(Message); ok {
-				discardLegacy(m)
-			}
-		}
+	// Discard unknown fields.
+	if len(m.GetUnknown()) > 0 {
+		m.SetUnknown(nil)
 	}
 }
diff --git a/vendor/github.com/golang/protobuf/proto/encode.go b/vendor/github.com/golang/protobuf/proto/encode.go
deleted file mode 100644
index 3abfed2..0000000
--- a/vendor/github.com/golang/protobuf/proto/encode.go
+++ /dev/null
@@ -1,203 +0,0 @@
-// 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.
-
-package proto
-
-/*
- * Routines for encoding data into the wire format for protocol buffers.
- */
-
-import (
-	"errors"
-	"reflect"
-)
-
-var (
-	// errRepeatedHasNil is the error returned if Marshal is called with
-	// a struct with a repeated field containing a nil element.
-	errRepeatedHasNil = errors.New("proto: repeated field has nil element")
-
-	// errOneofHasNil is the error returned if Marshal is called with
-	// a struct with a oneof field containing a nil element.
-	errOneofHasNil = errors.New("proto: oneof field has nil value")
-
-	// ErrNil is the error returned if Marshal is called with nil.
-	ErrNil = errors.New("proto: Marshal called with nil")
-
-	// ErrTooLarge is the error returned if Marshal is called with a
-	// message that encodes to >2GB.
-	ErrTooLarge = errors.New("proto: message encodes to over 2 GB")
-)
-
-// The fundamental encoders that put bytes on the wire.
-// Those that take integer types all accept uint64 and are
-// therefore of type valueEncoder.
-
-const maxVarintBytes = 10 // maximum length of a varint
-
-// EncodeVarint returns the varint encoding of x.
-// This is the format for the
-// int32, int64, uint32, uint64, bool, and enum
-// protocol buffer types.
-// Not used by the package itself, but helpful to clients
-// wishing to use the same encoding.
-func EncodeVarint(x uint64) []byte {
-	var buf [maxVarintBytes]byte
-	var n int
-	for n = 0; x > 127; n++ {
-		buf[n] = 0x80 | uint8(x&0x7F)
-		x >>= 7
-	}
-	buf[n] = uint8(x)
-	n++
-	return buf[0:n]
-}
-
-// EncodeVarint writes a varint-encoded integer to the Buffer.
-// This is the format for the
-// int32, int64, uint32, uint64, bool, and enum
-// protocol buffer types.
-func (p *Buffer) EncodeVarint(x uint64) error {
-	for x >= 1<<7 {
-		p.buf = append(p.buf, uint8(x&0x7f|0x80))
-		x >>= 7
-	}
-	p.buf = append(p.buf, uint8(x))
-	return nil
-}
-
-// SizeVarint returns the varint encoding size of an integer.
-func SizeVarint(x uint64) int {
-	switch {
-	case x < 1<<7:
-		return 1
-	case x < 1<<14:
-		return 2
-	case x < 1<<21:
-		return 3
-	case x < 1<<28:
-		return 4
-	case x < 1<<35:
-		return 5
-	case x < 1<<42:
-		return 6
-	case x < 1<<49:
-		return 7
-	case x < 1<<56:
-		return 8
-	case x < 1<<63:
-		return 9
-	}
-	return 10
-}
-
-// EncodeFixed64 writes a 64-bit integer to the Buffer.
-// This is the format for the
-// fixed64, sfixed64, and double protocol buffer types.
-func (p *Buffer) EncodeFixed64(x uint64) error {
-	p.buf = append(p.buf,
-		uint8(x),
-		uint8(x>>8),
-		uint8(x>>16),
-		uint8(x>>24),
-		uint8(x>>32),
-		uint8(x>>40),
-		uint8(x>>48),
-		uint8(x>>56))
-	return nil
-}
-
-// EncodeFixed32 writes a 32-bit integer to the Buffer.
-// This is the format for the
-// fixed32, sfixed32, and float protocol buffer types.
-func (p *Buffer) EncodeFixed32(x uint64) error {
-	p.buf = append(p.buf,
-		uint8(x),
-		uint8(x>>8),
-		uint8(x>>16),
-		uint8(x>>24))
-	return nil
-}
-
-// EncodeZigzag64 writes a zigzag-encoded 64-bit integer
-// to the Buffer.
-// This is the format used for the sint64 protocol buffer type.
-func (p *Buffer) EncodeZigzag64(x uint64) error {
-	// use signed number to get arithmetic right shift.
-	return p.EncodeVarint(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-
-// EncodeZigzag32 writes a zigzag-encoded 32-bit integer
-// to the Buffer.
-// This is the format used for the sint32 protocol buffer type.
-func (p *Buffer) EncodeZigzag32(x uint64) error {
-	// use signed number to get arithmetic right shift.
-	return p.EncodeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31))))
-}
-
-// EncodeRawBytes writes a count-delimited byte buffer to the Buffer.
-// This is the format used for the bytes protocol buffer
-// type and for embedded messages.
-func (p *Buffer) EncodeRawBytes(b []byte) error {
-	p.EncodeVarint(uint64(len(b)))
-	p.buf = append(p.buf, b...)
-	return nil
-}
-
-// EncodeStringBytes writes an encoded string to the Buffer.
-// This is the format used for the proto2 string type.
-func (p *Buffer) EncodeStringBytes(s string) error {
-	p.EncodeVarint(uint64(len(s)))
-	p.buf = append(p.buf, s...)
-	return nil
-}
-
-// Marshaler is the interface representing objects that can marshal themselves.
-type Marshaler interface {
-	Marshal() ([]byte, error)
-}
-
-// EncodeMessage writes the protocol buffer to the Buffer,
-// prefixed by a varint-encoded length.
-func (p *Buffer) EncodeMessage(pb Message) error {
-	siz := Size(pb)
-	p.EncodeVarint(uint64(siz))
-	return p.Marshal(pb)
-}
-
-// All protocol buffer fields are nillable, but be careful.
-func isNil(v reflect.Value) bool {
-	switch v.Kind() {
-	case reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice:
-		return v.IsNil()
-	}
-	return false
-}
diff --git a/vendor/github.com/golang/protobuf/proto/equal.go b/vendor/github.com/golang/protobuf/proto/equal.go
deleted file mode 100644
index f9b6e41..0000000
--- a/vendor/github.com/golang/protobuf/proto/equal.go
+++ /dev/null
@@ -1,301 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2011 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.
-
-// Protocol buffer comparison.
-
-package proto
-
-import (
-	"bytes"
-	"log"
-	"reflect"
-	"strings"
-)
-
-/*
-Equal returns true iff protocol buffers a and b are equal.
-The arguments must both be pointers to protocol buffer structs.
-
-Equality is defined in this way:
-  - Two messages are equal iff they are the same type,
-    corresponding fields are equal, unknown field sets
-    are equal, and extensions sets are equal.
-  - Two set scalar fields are equal iff their values are equal.
-    If the fields are of a floating-point type, remember that
-    NaN != x for all x, including NaN. If the message is defined
-    in a proto3 .proto file, fields are not "set"; specifically,
-    zero length proto3 "bytes" fields are equal (nil == {}).
-  - Two repeated fields are equal iff their lengths are the same,
-    and their corresponding elements are equal. Note a "bytes" field,
-    although represented by []byte, is not a repeated field and the
-    rule for the scalar fields described above applies.
-  - Two unset fields are equal.
-  - Two unknown field sets are equal if their current
-    encoded state is equal.
-  - Two extension sets are equal iff they have corresponding
-    elements that are pairwise equal.
-  - Two map fields are equal iff their lengths are the same,
-    and they contain the same set of elements. Zero-length map
-    fields are equal.
-  - Every other combination of things are not equal.
-
-The return value is undefined if a and b are not protocol buffers.
-*/
-func Equal(a, b Message) bool {
-	if a == nil || b == nil {
-		return a == b
-	}
-	v1, v2 := reflect.ValueOf(a), reflect.ValueOf(b)
-	if v1.Type() != v2.Type() {
-		return false
-	}
-	if v1.Kind() == reflect.Ptr {
-		if v1.IsNil() {
-			return v2.IsNil()
-		}
-		if v2.IsNil() {
-			return false
-		}
-		v1, v2 = v1.Elem(), v2.Elem()
-	}
-	if v1.Kind() != reflect.Struct {
-		return false
-	}
-	return equalStruct(v1, v2)
-}
-
-// v1 and v2 are known to have the same type.
-func equalStruct(v1, v2 reflect.Value) bool {
-	sprop := GetProperties(v1.Type())
-	for i := 0; i < v1.NumField(); i++ {
-		f := v1.Type().Field(i)
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-		f1, f2 := v1.Field(i), v2.Field(i)
-		if f.Type.Kind() == reflect.Ptr {
-			if n1, n2 := f1.IsNil(), f2.IsNil(); n1 && n2 {
-				// both unset
-				continue
-			} else if n1 != n2 {
-				// set/unset mismatch
-				return false
-			}
-			f1, f2 = f1.Elem(), f2.Elem()
-		}
-		if !equalAny(f1, f2, sprop.Prop[i]) {
-			return false
-		}
-	}
-
-	if em1 := v1.FieldByName("XXX_InternalExtensions"); em1.IsValid() {
-		em2 := v2.FieldByName("XXX_InternalExtensions")
-		if !equalExtensions(v1.Type(), em1.Interface().(XXX_InternalExtensions), em2.Interface().(XXX_InternalExtensions)) {
-			return false
-		}
-	}
-
-	if em1 := v1.FieldByName("XXX_extensions"); em1.IsValid() {
-		em2 := v2.FieldByName("XXX_extensions")
-		if !equalExtMap(v1.Type(), em1.Interface().(map[int32]Extension), em2.Interface().(map[int32]Extension)) {
-			return false
-		}
-	}
-
-	uf := v1.FieldByName("XXX_unrecognized")
-	if !uf.IsValid() {
-		return true
-	}
-
-	u1 := uf.Bytes()
-	u2 := v2.FieldByName("XXX_unrecognized").Bytes()
-	return bytes.Equal(u1, u2)
-}
-
-// v1 and v2 are known to have the same type.
-// prop may be nil.
-func equalAny(v1, v2 reflect.Value, prop *Properties) bool {
-	if v1.Type() == protoMessageType {
-		m1, _ := v1.Interface().(Message)
-		m2, _ := v2.Interface().(Message)
-		return Equal(m1, m2)
-	}
-	switch v1.Kind() {
-	case reflect.Bool:
-		return v1.Bool() == v2.Bool()
-	case reflect.Float32, reflect.Float64:
-		return v1.Float() == v2.Float()
-	case reflect.Int32, reflect.Int64:
-		return v1.Int() == v2.Int()
-	case reflect.Interface:
-		// Probably a oneof field; compare the inner values.
-		n1, n2 := v1.IsNil(), v2.IsNil()
-		if n1 || n2 {
-			return n1 == n2
-		}
-		e1, e2 := v1.Elem(), v2.Elem()
-		if e1.Type() != e2.Type() {
-			return false
-		}
-		return equalAny(e1, e2, nil)
-	case reflect.Map:
-		if v1.Len() != v2.Len() {
-			return false
-		}
-		for _, key := range v1.MapKeys() {
-			val2 := v2.MapIndex(key)
-			if !val2.IsValid() {
-				// This key was not found in the second map.
-				return false
-			}
-			if !equalAny(v1.MapIndex(key), val2, nil) {
-				return false
-			}
-		}
-		return true
-	case reflect.Ptr:
-		// Maps may have nil values in them, so check for nil.
-		if v1.IsNil() && v2.IsNil() {
-			return true
-		}
-		if v1.IsNil() != v2.IsNil() {
-			return false
-		}
-		return equalAny(v1.Elem(), v2.Elem(), prop)
-	case reflect.Slice:
-		if v1.Type().Elem().Kind() == reflect.Uint8 {
-			// short circuit: []byte
-
-			// Edge case: if this is in a proto3 message, a zero length
-			// bytes field is considered the zero value.
-			if prop != nil && prop.proto3 && v1.Len() == 0 && v2.Len() == 0 {
-				return true
-			}
-			if v1.IsNil() != v2.IsNil() {
-				return false
-			}
-			return bytes.Equal(v1.Interface().([]byte), v2.Interface().([]byte))
-		}
-
-		if v1.Len() != v2.Len() {
-			return false
-		}
-		for i := 0; i < v1.Len(); i++ {
-			if !equalAny(v1.Index(i), v2.Index(i), prop) {
-				return false
-			}
-		}
-		return true
-	case reflect.String:
-		return v1.Interface().(string) == v2.Interface().(string)
-	case reflect.Struct:
-		return equalStruct(v1, v2)
-	case reflect.Uint32, reflect.Uint64:
-		return v1.Uint() == v2.Uint()
-	}
-
-	// unknown type, so not a protocol buffer
-	log.Printf("proto: don't know how to compare %v", v1)
-	return false
-}
-
-// base is the struct type that the extensions are based on.
-// x1 and x2 are InternalExtensions.
-func equalExtensions(base reflect.Type, x1, x2 XXX_InternalExtensions) bool {
-	em1, _ := x1.extensionsRead()
-	em2, _ := x2.extensionsRead()
-	return equalExtMap(base, em1, em2)
-}
-
-func equalExtMap(base reflect.Type, em1, em2 map[int32]Extension) bool {
-	if len(em1) != len(em2) {
-		return false
-	}
-
-	for extNum, e1 := range em1 {
-		e2, ok := em2[extNum]
-		if !ok {
-			return false
-		}
-
-		m1 := extensionAsLegacyType(e1.value)
-		m2 := extensionAsLegacyType(e2.value)
-
-		if m1 == nil && m2 == nil {
-			// Both have only encoded form.
-			if bytes.Equal(e1.enc, e2.enc) {
-				continue
-			}
-			// The bytes are different, but the extensions might still be
-			// equal. We need to decode them to compare.
-		}
-
-		if m1 != nil && m2 != nil {
-			// Both are unencoded.
-			if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) {
-				return false
-			}
-			continue
-		}
-
-		// At least one is encoded. To do a semantically correct comparison
-		// we need to unmarshal them first.
-		var desc *ExtensionDesc
-		if m := extensionMaps[base]; m != nil {
-			desc = m[extNum]
-		}
-		if desc == nil {
-			// If both have only encoded form and the bytes are the same,
-			// it is handled above. We get here when the bytes are different.
-			// We don't know how to decode it, so just compare them as byte
-			// slices.
-			log.Printf("proto: don't know how to compare extension %d of %v", extNum, base)
-			return false
-		}
-		var err error
-		if m1 == nil {
-			m1, err = decodeExtension(e1.enc, desc)
-		}
-		if m2 == nil && err == nil {
-			m2, err = decodeExtension(e2.enc, desc)
-		}
-		if err != nil {
-			// The encoded form is invalid.
-			log.Printf("proto: badly encoded extension %d of %v: %v", extNum, base, err)
-			return false
-		}
-		if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) {
-			return false
-		}
-	}
-
-	return true
-}
diff --git a/vendor/github.com/golang/protobuf/proto/extensions.go b/vendor/github.com/golang/protobuf/proto/extensions.go
index fa88add..42fc120 100644
--- a/vendor/github.com/golang/protobuf/proto/extensions.go
+++ b/vendor/github.com/golang/protobuf/proto/extensions.go
@@ -1,607 +1,356 @@
-// 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.
+// 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 proto
 
-/*
- * Types and routines for supporting protocol buffer extensions.
- */
-
 import (
 	"errors"
 	"fmt"
-	"io"
 	"reflect"
-	"strconv"
-	"sync"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+	"google.golang.org/protobuf/runtime/protoiface"
+	"google.golang.org/protobuf/runtime/protoimpl"
 )
 
-// ErrMissingExtension is the error returned by GetExtension if the named extension is not in the message.
+type (
+	// ExtensionDesc represents an extension descriptor and
+	// is used to interact with an extension field in a message.
+	//
+	// Variables of this type are generated in code by protoc-gen-go.
+	ExtensionDesc = protoimpl.ExtensionInfo
+
+	// ExtensionRange represents a range of message extensions.
+	// Used in code generated by protoc-gen-go.
+	ExtensionRange = protoiface.ExtensionRangeV1
+
+	// Deprecated: Do not use; this is an internal type.
+	Extension = protoimpl.ExtensionFieldV1
+
+	// Deprecated: Do not use; this is an internal type.
+	XXX_InternalExtensions = protoimpl.ExtensionFields
+)
+
+// ErrMissingExtension reports whether the extension was not present.
 var ErrMissingExtension = errors.New("proto: missing extension")
 
-// ExtensionRange represents a range of message extensions for a protocol buffer.
-// Used in code generated by the protocol compiler.
-type ExtensionRange struct {
-	Start, End int32 // both inclusive
-}
-
-// extendableProto is an interface implemented by any protocol buffer generated by the current
-// proto compiler that may be extended.
-type extendableProto interface {
-	Message
-	ExtensionRangeArray() []ExtensionRange
-	extensionsWrite() map[int32]Extension
-	extensionsRead() (map[int32]Extension, sync.Locker)
-}
-
-// extendableProtoV1 is an interface implemented by a protocol buffer generated by the previous
-// version of the proto compiler that may be extended.
-type extendableProtoV1 interface {
-	Message
-	ExtensionRangeArray() []ExtensionRange
-	ExtensionMap() map[int32]Extension
-}
-
-// extensionAdapter is a wrapper around extendableProtoV1 that implements extendableProto.
-type extensionAdapter struct {
-	extendableProtoV1
-}
-
-func (e extensionAdapter) extensionsWrite() map[int32]Extension {
-	return e.ExtensionMap()
-}
-
-func (e extensionAdapter) extensionsRead() (map[int32]Extension, sync.Locker) {
-	return e.ExtensionMap(), notLocker{}
-}
-
-// notLocker is a sync.Locker whose Lock and Unlock methods are nops.
-type notLocker struct{}
-
-func (n notLocker) Lock()   {}
-func (n notLocker) Unlock() {}
-
-// extendable returns the extendableProto interface for the given generated proto message.
-// If the proto message has the old extension format, it returns a wrapper that implements
-// the extendableProto interface.
-func extendable(p interface{}) (extendableProto, error) {
-	switch p := p.(type) {
-	case extendableProto:
-		if isNilPtr(p) {
-			return nil, fmt.Errorf("proto: nil %T is not extendable", p)
-		}
-		return p, nil
-	case extendableProtoV1:
-		if isNilPtr(p) {
-			return nil, fmt.Errorf("proto: nil %T is not extendable", p)
-		}
-		return extensionAdapter{p}, nil
-	}
-	// Don't allocate a specific error containing %T:
-	// this is the hot path for Clone and MarshalText.
-	return nil, errNotExtendable
-}
-
 var errNotExtendable = errors.New("proto: not an extendable proto.Message")
 
-func isNilPtr(x interface{}) bool {
-	v := reflect.ValueOf(x)
-	return v.Kind() == reflect.Ptr && v.IsNil()
-}
-
-// XXX_InternalExtensions is an internal representation of proto extensions.
-//
-// Each generated message struct type embeds an anonymous XXX_InternalExtensions field,
-// thus gaining the unexported 'extensions' method, which can be called only from the proto package.
-//
-// The methods of XXX_InternalExtensions are not concurrency safe in general,
-// but calls to logically read-only methods such as has and get may be executed concurrently.
-type XXX_InternalExtensions struct {
-	// The struct must be indirect so that if a user inadvertently copies a
-	// generated message and its embedded XXX_InternalExtensions, they
-	// avoid the mayhem of a copied mutex.
-	//
-	// The mutex serializes all logically read-only operations to p.extensionMap.
-	// It is up to the client to ensure that write operations to p.extensionMap are
-	// mutually exclusive with other accesses.
-	p *struct {
-		mu           sync.Mutex
-		extensionMap map[int32]Extension
+// HasExtension reports whether the extension field is present in m
+// either as an explicitly populated field or as an unknown field.
+func HasExtension(m Message, xt *ExtensionDesc) (has bool) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
+		return false
 	}
-}
 
-// extensionsWrite returns the extension map, creating it on first use.
-func (e *XXX_InternalExtensions) extensionsWrite() map[int32]Extension {
-	if e.p == nil {
-		e.p = new(struct {
-			mu           sync.Mutex
-			extensionMap map[int32]Extension
+	// Check whether any populated known field matches the field number.
+	xtd := xt.TypeDescriptor()
+	if isValidExtension(mr.Descriptor(), xtd) {
+		has = mr.Has(xtd)
+	} else {
+		mr.Range(func(fd protoreflect.FieldDescriptor, _ protoreflect.Value) bool {
+			has = int32(fd.Number()) == xt.Field
+			return !has
 		})
-		e.p.extensionMap = make(map[int32]Extension)
 	}
-	return e.p.extensionMap
-}
 
-// extensionsRead returns the extensions map for read-only use.  It may be nil.
-// The caller must hold the returned mutex's lock when accessing Elements within the map.
-func (e *XXX_InternalExtensions) extensionsRead() (map[int32]Extension, sync.Locker) {
-	if e.p == nil {
-		return nil, nil
+	// Check whether any unknown field matches the field number.
+	for b := mr.GetUnknown(); !has && len(b) > 0; {
+		num, _, n := protowire.ConsumeField(b)
+		has = int32(num) == xt.Field
+		b = b[n:]
 	}
-	return e.p.extensionMap, &e.p.mu
+	return has
 }
 
-// ExtensionDesc represents an extension specification.
-// Used in generated code from the protocol compiler.
-type ExtensionDesc struct {
-	ExtendedType  Message     // nil pointer to the type that is being extended
-	ExtensionType interface{} // nil pointer to the extension type
-	Field         int32       // field number
-	Name          string      // fully-qualified name of extension, for text formatting
-	Tag           string      // protobuf tag style
-	Filename      string      // name of the file in which the extension is defined
-}
-
-func (ed *ExtensionDesc) repeated() bool {
-	t := reflect.TypeOf(ed.ExtensionType)
-	return t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8
-}
-
-// Extension represents an extension in a message.
-type Extension struct {
-	// When an extension is stored in a message using SetExtension
-	// only desc and value are set. When the message is marshaled
-	// enc will be set to the encoded form of the message.
-	//
-	// When a message is unmarshaled and contains extensions, each
-	// extension will have only enc set. When such an extension is
-	// accessed using GetExtension (or GetExtensions) desc and value
-	// will be set.
-	desc *ExtensionDesc
-
-	// value is a concrete value for the extension field. Let the type of
-	// desc.ExtensionType be the "API type" and the type of Extension.value
-	// be the "storage type". The API type and storage type are the same except:
-	//	* For scalars (except []byte), the API type uses *T,
-	//	while the storage type uses T.
-	//	* For repeated fields, the API type uses []T, while the storage type
-	//	uses *[]T.
-	//
-	// The reason for the divergence is so that the storage type more naturally
-	// matches what is expected of when retrieving the values through the
-	// protobuf reflection APIs.
-	//
-	// The value may only be populated if desc is also populated.
-	value interface{}
-
-	// enc is the raw bytes for the extension field.
-	enc []byte
-}
-
-// SetRawExtension is for testing only.
-func SetRawExtension(base Message, id int32, b []byte) {
-	epb, err := extendable(base)
-	if err != nil {
+// ClearExtension removes the extension field from m
+// either as an explicitly populated field or as an unknown field.
+func ClearExtension(m Message, xt *ExtensionDesc) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
 		return
 	}
-	extmap := epb.extensionsWrite()
-	extmap[id] = Extension{enc: b}
-}
 
-// isExtensionField returns true iff the given field number is in an extension range.
-func isExtensionField(pb extendableProto, field int32) bool {
-	for _, er := range pb.ExtensionRangeArray() {
-		if er.Start <= field && field <= er.End {
+	xtd := xt.TypeDescriptor()
+	if isValidExtension(mr.Descriptor(), xtd) {
+		mr.Clear(xtd)
+	} else {
+		mr.Range(func(fd protoreflect.FieldDescriptor, _ protoreflect.Value) bool {
+			if int32(fd.Number()) == xt.Field {
+				mr.Clear(fd)
+				return false
+			}
 			return true
-		}
+		})
 	}
-	return false
+	clearUnknown(mr, fieldNum(xt.Field))
 }
 
-// checkExtensionTypes checks that the given extension is valid for pb.
-func checkExtensionTypes(pb extendableProto, extension *ExtensionDesc) error {
-	var pbi interface{} = pb
-	// Check the extended type.
-	if ea, ok := pbi.(extensionAdapter); ok {
-		pbi = ea.extendableProtoV1
-	}
-	if a, b := reflect.TypeOf(pbi), reflect.TypeOf(extension.ExtendedType); a != b {
-		return fmt.Errorf("proto: bad extended type; %v does not extend %v", b, a)
-	}
-	// Check the range.
-	if !isExtensionField(pb, extension.Field) {
-		return errors.New("proto: bad extension number; not in declared ranges")
-	}
-	return nil
-}
-
-// extPropKey is sufficient to uniquely identify an extension.
-type extPropKey struct {
-	base  reflect.Type
-	field int32
-}
-
-var extProp = struct {
-	sync.RWMutex
-	m map[extPropKey]*Properties
-}{
-	m: make(map[extPropKey]*Properties),
-}
-
-func extensionProperties(ed *ExtensionDesc) *Properties {
-	key := extPropKey{base: reflect.TypeOf(ed.ExtendedType), field: ed.Field}
-
-	extProp.RLock()
-	if prop, ok := extProp.m[key]; ok {
-		extProp.RUnlock()
-		return prop
-	}
-	extProp.RUnlock()
-
-	extProp.Lock()
-	defer extProp.Unlock()
-	// Check again.
-	if prop, ok := extProp.m[key]; ok {
-		return prop
-	}
-
-	prop := new(Properties)
-	prop.Init(reflect.TypeOf(ed.ExtensionType), "unknown_name", ed.Tag, nil)
-	extProp.m[key] = prop
-	return prop
-}
-
-// HasExtension returns whether the given extension is present in pb.
-func HasExtension(pb Message, extension *ExtensionDesc) bool {
-	// TODO: Check types, field numbers, etc.?
-	epb, err := extendable(pb)
-	if err != nil {
-		return false
-	}
-	extmap, mu := epb.extensionsRead()
-	if extmap == nil {
-		return false
-	}
-	mu.Lock()
-	_, ok := extmap[extension.Field]
-	mu.Unlock()
-	return ok
-}
-
-// ClearExtension removes the given extension from pb.
-func ClearExtension(pb Message, extension *ExtensionDesc) {
-	epb, err := extendable(pb)
-	if err != nil {
+// ClearAllExtensions clears all extensions from m.
+// This includes populated fields and unknown fields in the extension range.
+func ClearAllExtensions(m Message) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
 		return
 	}
-	// TODO: Check types, field numbers, etc.?
-	extmap := epb.extensionsWrite()
-	delete(extmap, extension.Field)
+
+	mr.Range(func(fd protoreflect.FieldDescriptor, _ protoreflect.Value) bool {
+		if fd.IsExtension() {
+			mr.Clear(fd)
+		}
+		return true
+	})
+	clearUnknown(mr, mr.Descriptor().ExtensionRanges())
 }
 
-// GetExtension retrieves a proto2 extended field from pb.
+// GetExtension retrieves a proto2 extended field from m.
 //
 // If the descriptor is type complete (i.e., ExtensionDesc.ExtensionType is non-nil),
 // then GetExtension parses the encoded field and returns a Go value of the specified type.
 // If the field is not present, then the default value is returned (if one is specified),
 // otherwise ErrMissingExtension is reported.
 //
-// If the descriptor is not type complete (i.e., ExtensionDesc.ExtensionType is nil),
-// then GetExtension returns the raw encoded bytes of the field extension.
-func GetExtension(pb Message, extension *ExtensionDesc) (interface{}, error) {
-	epb, err := extendable(pb)
-	if err != nil {
-		return nil, err
+// If the descriptor is type incomplete (i.e., ExtensionDesc.ExtensionType is nil),
+// then GetExtension returns the raw encoded bytes for the extension field.
+func GetExtension(m Message, xt *ExtensionDesc) (interface{}, error) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() || mr.Descriptor().ExtensionRanges().Len() == 0 {
+		return nil, errNotExtendable
 	}
 
-	if extension.ExtendedType != nil {
-		// can only check type if this is a complete descriptor
-		if err := checkExtensionTypes(epb, extension); err != nil {
+	// Retrieve the unknown fields for this extension field.
+	var bo protoreflect.RawFields
+	for bi := mr.GetUnknown(); len(bi) > 0; {
+		num, _, n := protowire.ConsumeField(bi)
+		if int32(num) == xt.Field {
+			bo = append(bo, bi[:n]...)
+		}
+		bi = bi[n:]
+	}
+
+	// For type incomplete descriptors, only retrieve the unknown fields.
+	if xt.ExtensionType == nil {
+		return []byte(bo), nil
+	}
+
+	// If the extension field only exists as unknown fields, unmarshal it.
+	// This is rarely done since proto.Unmarshal eagerly unmarshals extensions.
+	xtd := xt.TypeDescriptor()
+	if !isValidExtension(mr.Descriptor(), xtd) {
+		return nil, fmt.Errorf("proto: bad extended type; %T does not extend %T", xt.ExtendedType, m)
+	}
+	if !mr.Has(xtd) && len(bo) > 0 {
+		m2 := mr.New()
+		if err := (proto.UnmarshalOptions{
+			Resolver: extensionResolver{xt},
+		}.Unmarshal(bo, m2.Interface())); err != nil {
 			return nil, err
 		}
-	}
-
-	emap, mu := epb.extensionsRead()
-	if emap == nil {
-		return defaultExtensionValue(extension)
-	}
-	mu.Lock()
-	defer mu.Unlock()
-	e, ok := emap[extension.Field]
-	if !ok {
-		// defaultExtensionValue returns the default value or
-		// ErrMissingExtension if there is no default.
-		return defaultExtensionValue(extension)
-	}
-
-	if e.value != nil {
-		// Already decoded. Check the descriptor, though.
-		if e.desc != extension {
-			// This shouldn't happen. If it does, it means that
-			// GetExtension was called twice with two different
-			// descriptors with the same field number.
-			return nil, errors.New("proto: descriptor conflict")
+		if m2.Has(xtd) {
+			mr.Set(xtd, m2.Get(xtd))
+			clearUnknown(mr, fieldNum(xt.Field))
 		}
-		return extensionAsLegacyType(e.value), nil
 	}
 
-	if extension.ExtensionType == nil {
-		// incomplete descriptor
-		return e.enc, nil
-	}
-
-	v, err := decodeExtension(e.enc, extension)
-	if err != nil {
-		return nil, err
-	}
-
-	// Remember the decoded version and drop the encoded version.
-	// That way it is safe to mutate what we return.
-	e.value = extensionAsStorageType(v)
-	e.desc = extension
-	e.enc = nil
-	emap[extension.Field] = e
-	return extensionAsLegacyType(e.value), nil
-}
-
-// defaultExtensionValue returns the default value for extension.
-// If no default for an extension is defined ErrMissingExtension is returned.
-func defaultExtensionValue(extension *ExtensionDesc) (interface{}, error) {
-	if extension.ExtensionType == nil {
-		// incomplete descriptor, so no default
+	// Check whether the message has the extension field set or a default.
+	var pv protoreflect.Value
+	switch {
+	case mr.Has(xtd):
+		pv = mr.Get(xtd)
+	case xtd.HasDefault():
+		pv = xtd.Default()
+	default:
 		return nil, ErrMissingExtension
 	}
 
-	t := reflect.TypeOf(extension.ExtensionType)
-	props := extensionProperties(extension)
-
-	sf, _, err := fieldDefault(t, props)
-	if err != nil {
-		return nil, err
-	}
-
-	if sf == nil || sf.value == nil {
-		// There is no default value.
-		return nil, ErrMissingExtension
-	}
-
-	if t.Kind() != reflect.Ptr {
-		// We do not need to return a Ptr, we can directly return sf.value.
-		return sf.value, nil
-	}
-
-	// We need to return an interface{} that is a pointer to sf.value.
-	value := reflect.New(t).Elem()
-	value.Set(reflect.New(value.Type().Elem()))
-	if sf.kind == reflect.Int32 {
-		// We may have an int32 or an enum, but the underlying data is int32.
-		// Since we can't set an int32 into a non int32 reflect.value directly
-		// set it as a int32.
-		value.Elem().SetInt(int64(sf.value.(int32)))
-	} else {
-		value.Elem().Set(reflect.ValueOf(sf.value))
-	}
-	return value.Interface(), nil
-}
-
-// decodeExtension decodes an extension encoded in b.
-func decodeExtension(b []byte, extension *ExtensionDesc) (interface{}, error) {
-	t := reflect.TypeOf(extension.ExtensionType)
-	unmarshal := typeUnmarshaler(t, extension.Tag)
-
-	// t is a pointer to a struct, pointer to basic type or a slice.
-	// Allocate space to store the pointer/slice.
-	value := reflect.New(t).Elem()
-
-	var err error
-	for {
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		wire := int(x) & 7
-
-		b, err = unmarshal(b, valToPointer(value.Addr()), wire)
-		if err != nil {
-			return nil, err
-		}
-
-		if len(b) == 0 {
-			break
-		}
-	}
-	return value.Interface(), nil
-}
-
-// GetExtensions returns a slice of the extensions present in pb that are also listed in es.
-// The returned slice has the same length as es; missing extensions will appear as nil elements.
-func GetExtensions(pb Message, es []*ExtensionDesc) (extensions []interface{}, err error) {
-	epb, err := extendable(pb)
-	if err != nil {
-		return nil, err
-	}
-	extensions = make([]interface{}, len(es))
-	for i, e := range es {
-		extensions[i], err = GetExtension(epb, e)
-		if err == ErrMissingExtension {
-			err = nil
-		}
-		if err != nil {
-			return
-		}
-	}
-	return
-}
-
-// ExtensionDescs returns a new slice containing pb's extension descriptors, in undefined order.
-// For non-registered extensions, ExtensionDescs returns an incomplete descriptor containing
-// just the Field field, which defines the extension's field number.
-func ExtensionDescs(pb Message) ([]*ExtensionDesc, error) {
-	epb, err := extendable(pb)
-	if err != nil {
-		return nil, err
-	}
-	registeredExtensions := RegisteredExtensions(pb)
-
-	emap, mu := epb.extensionsRead()
-	if emap == nil {
-		return nil, nil
-	}
-	mu.Lock()
-	defer mu.Unlock()
-	extensions := make([]*ExtensionDesc, 0, len(emap))
-	for extid, e := range emap {
-		desc := e.desc
-		if desc == nil {
-			desc = registeredExtensions[extid]
-			if desc == nil {
-				desc = &ExtensionDesc{Field: extid}
-			}
-		}
-
-		extensions = append(extensions, desc)
-	}
-	return extensions, nil
-}
-
-// SetExtension sets the specified extension of pb to the specified value.
-func SetExtension(pb Message, extension *ExtensionDesc, value interface{}) error {
-	epb, err := extendable(pb)
-	if err != nil {
-		return err
-	}
-	if err := checkExtensionTypes(epb, extension); err != nil {
-		return err
-	}
-	typ := reflect.TypeOf(extension.ExtensionType)
-	if typ != reflect.TypeOf(value) {
-		return fmt.Errorf("proto: bad extension value type. got: %T, want: %T", value, extension.ExtensionType)
-	}
-	// nil extension values need to be caught early, because the
-	// encoder can't distinguish an ErrNil due to a nil extension
-	// from an ErrNil due to a missing field. Extensions are
-	// always optional, so the encoder would just swallow the error
-	// and drop all the extensions from the encoded message.
-	if reflect.ValueOf(value).IsNil() {
-		return fmt.Errorf("proto: SetExtension called with nil value of type %T", value)
-	}
-
-	extmap := epb.extensionsWrite()
-	extmap[extension.Field] = Extension{desc: extension, value: extensionAsStorageType(value)}
-	return nil
-}
-
-// ClearAllExtensions clears all extensions from pb.
-func ClearAllExtensions(pb Message) {
-	epb, err := extendable(pb)
-	if err != nil {
-		return
-	}
-	m := epb.extensionsWrite()
-	for k := range m {
-		delete(m, k)
-	}
-}
-
-// A global registry of extensions.
-// The generated code will register the generated descriptors by calling RegisterExtension.
-
-var extensionMaps = make(map[reflect.Type]map[int32]*ExtensionDesc)
-
-// RegisterExtension is called from the generated code.
-func RegisterExtension(desc *ExtensionDesc) {
-	st := reflect.TypeOf(desc.ExtendedType).Elem()
-	m := extensionMaps[st]
-	if m == nil {
-		m = make(map[int32]*ExtensionDesc)
-		extensionMaps[st] = m
-	}
-	if _, ok := m[desc.Field]; ok {
-		panic("proto: duplicate extension registered: " + st.String() + " " + strconv.Itoa(int(desc.Field)))
-	}
-	m[desc.Field] = desc
-}
-
-// RegisteredExtensions returns a map of the registered extensions of a
-// protocol buffer struct, indexed by the extension number.
-// The argument pb should be a nil pointer to the struct type.
-func RegisteredExtensions(pb Message) map[int32]*ExtensionDesc {
-	return extensionMaps[reflect.TypeOf(pb).Elem()]
-}
-
-// extensionAsLegacyType converts an value in the storage type as the API type.
-// See Extension.value.
-func extensionAsLegacyType(v interface{}) interface{} {
-	switch rv := reflect.ValueOf(v); rv.Kind() {
-	case reflect.Bool, reflect.Int32, reflect.Int64, reflect.Uint32, reflect.Uint64, reflect.Float32, reflect.Float64, reflect.String:
-		// Represent primitive types as a pointer to the value.
+	v := xt.InterfaceOf(pv)
+	rv := reflect.ValueOf(v)
+	if isScalarKind(rv.Kind()) {
 		rv2 := reflect.New(rv.Type())
 		rv2.Elem().Set(rv)
 		v = rv2.Interface()
-	case reflect.Ptr:
-		// Represent slice types as the value itself.
-		switch rv.Type().Elem().Kind() {
-		case reflect.Slice:
-			if rv.IsNil() {
-				v = reflect.Zero(rv.Type().Elem()).Interface()
-			} else {
-				v = rv.Elem().Interface()
-			}
-		}
 	}
-	return v
+	return v, nil
 }
 
-// extensionAsStorageType converts an value in the API type as the storage type.
-// See Extension.value.
-func extensionAsStorageType(v interface{}) interface{} {
-	switch rv := reflect.ValueOf(v); rv.Kind() {
-	case reflect.Ptr:
-		// Represent slice types as the value itself.
-		switch rv.Type().Elem().Kind() {
-		case reflect.Bool, reflect.Int32, reflect.Int64, reflect.Uint32, reflect.Uint64, reflect.Float32, reflect.Float64, reflect.String:
-			if rv.IsNil() {
-				v = reflect.Zero(rv.Type().Elem()).Interface()
-			} else {
-				v = rv.Elem().Interface()
+// extensionResolver is a custom extension resolver that stores a single
+// extension type that takes precedence over the global registry.
+type extensionResolver struct{ xt protoreflect.ExtensionType }
+
+func (r extensionResolver) FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error) {
+	if xtd := r.xt.TypeDescriptor(); xtd.FullName() == field {
+		return r.xt, nil
+	}
+	return protoregistry.GlobalTypes.FindExtensionByName(field)
+}
+
+func (r extensionResolver) FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error) {
+	if xtd := r.xt.TypeDescriptor(); xtd.ContainingMessage().FullName() == message && xtd.Number() == field {
+		return r.xt, nil
+	}
+	return protoregistry.GlobalTypes.FindExtensionByNumber(message, field)
+}
+
+// GetExtensions returns a list of the extensions values present in m,
+// corresponding with the provided list of extension descriptors, xts.
+// If an extension is missing in m, the corresponding value is nil.
+func GetExtensions(m Message, xts []*ExtensionDesc) ([]interface{}, error) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
+		return nil, errNotExtendable
+	}
+
+	vs := make([]interface{}, len(xts))
+	for i, xt := range xts {
+		v, err := GetExtension(m, xt)
+		if err != nil {
+			if err == ErrMissingExtension {
+				continue
 			}
+			return vs, err
 		}
-	case reflect.Slice:
-		// Represent slice types as a pointer to the value.
-		if rv.Type().Elem().Kind() != reflect.Uint8 {
-			rv2 := reflect.New(rv.Type())
-			rv2.Elem().Set(rv)
-			v = rv2.Interface()
+		vs[i] = v
+	}
+	return vs, nil
+}
+
+// SetExtension sets an extension field in m to the provided value.
+func SetExtension(m Message, xt *ExtensionDesc, v interface{}) error {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() || mr.Descriptor().ExtensionRanges().Len() == 0 {
+		return errNotExtendable
+	}
+
+	rv := reflect.ValueOf(v)
+	if reflect.TypeOf(v) != reflect.TypeOf(xt.ExtensionType) {
+		return fmt.Errorf("proto: bad extension value type. got: %T, want: %T", v, xt.ExtensionType)
+	}
+	if rv.Kind() == reflect.Ptr {
+		if rv.IsNil() {
+			return fmt.Errorf("proto: SetExtension called with nil value of type %T", v)
+		}
+		if isScalarKind(rv.Elem().Kind()) {
+			v = rv.Elem().Interface()
 		}
 	}
-	return v
+
+	xtd := xt.TypeDescriptor()
+	if !isValidExtension(mr.Descriptor(), xtd) {
+		return fmt.Errorf("proto: bad extended type; %T does not extend %T", xt.ExtendedType, m)
+	}
+	mr.Set(xtd, xt.ValueOf(v))
+	clearUnknown(mr, fieldNum(xt.Field))
+	return nil
+}
+
+// SetRawExtension inserts b into the unknown fields of m.
+//
+// Deprecated: Use Message.ProtoReflect.SetUnknown instead.
+func SetRawExtension(m Message, fnum int32, b []byte) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
+		return
+	}
+
+	// Verify that the raw field is valid.
+	for b0 := b; len(b0) > 0; {
+		num, _, n := protowire.ConsumeField(b0)
+		if int32(num) != fnum {
+			panic(fmt.Sprintf("mismatching field number: got %d, want %d", num, fnum))
+		}
+		b0 = b0[n:]
+	}
+
+	ClearExtension(m, &ExtensionDesc{Field: fnum})
+	mr.SetUnknown(append(mr.GetUnknown(), b...))
+}
+
+// ExtensionDescs returns a list of extension descriptors found in m,
+// containing descriptors for both populated extension fields in m and
+// also unknown fields of m that are in the extension range.
+// For the later case, an type incomplete descriptor is provided where only
+// the ExtensionDesc.Field field is populated.
+// The order of the extension descriptors is undefined.
+func ExtensionDescs(m Message) ([]*ExtensionDesc, error) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() || mr.Descriptor().ExtensionRanges().Len() == 0 {
+		return nil, errNotExtendable
+	}
+
+	// Collect a set of known extension descriptors.
+	extDescs := make(map[protoreflect.FieldNumber]*ExtensionDesc)
+	mr.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		if fd.IsExtension() {
+			xt := fd.(protoreflect.ExtensionTypeDescriptor)
+			if xd, ok := xt.Type().(*ExtensionDesc); ok {
+				extDescs[fd.Number()] = xd
+			}
+		}
+		return true
+	})
+
+	// Collect a set of unknown extension descriptors.
+	extRanges := mr.Descriptor().ExtensionRanges()
+	for b := mr.GetUnknown(); len(b) > 0; {
+		num, _, n := protowire.ConsumeField(b)
+		if extRanges.Has(num) && extDescs[num] == nil {
+			extDescs[num] = nil
+		}
+		b = b[n:]
+	}
+
+	// Transpose the set of descriptors into a list.
+	var xts []*ExtensionDesc
+	for num, xt := range extDescs {
+		if xt == nil {
+			xt = &ExtensionDesc{Field: int32(num)}
+		}
+		xts = append(xts, xt)
+	}
+	return xts, nil
+}
+
+// isValidExtension reports whether xtd is a valid extension descriptor for md.
+func isValidExtension(md protoreflect.MessageDescriptor, xtd protoreflect.ExtensionTypeDescriptor) bool {
+	return xtd.ContainingMessage() == md && md.ExtensionRanges().Has(xtd.Number())
+}
+
+// isScalarKind reports whether k is a protobuf scalar kind (except bytes).
+// This function exists for historical reasons since the representation of
+// scalars differs between v1 and v2, where v1 uses *T and v2 uses T.
+func isScalarKind(k reflect.Kind) bool {
+	switch k {
+	case reflect.Bool, reflect.Int32, reflect.Int64, reflect.Uint32, reflect.Uint64, reflect.Float32, reflect.Float64, reflect.String:
+		return true
+	default:
+		return false
+	}
+}
+
+// clearUnknown removes unknown fields from m where remover.Has reports true.
+func clearUnknown(m protoreflect.Message, remover interface {
+	Has(protoreflect.FieldNumber) bool
+}) {
+	var bo protoreflect.RawFields
+	for bi := m.GetUnknown(); len(bi) > 0; {
+		num, _, n := protowire.ConsumeField(bi)
+		if !remover.Has(num) {
+			bo = append(bo, bi[:n]...)
+		}
+		bi = bi[n:]
+	}
+	if bi := m.GetUnknown(); len(bi) != len(bo) {
+		m.SetUnknown(bo)
+	}
+}
+
+type fieldNum protoreflect.FieldNumber
+
+func (n1 fieldNum) Has(n2 protoreflect.FieldNumber) bool {
+	return protoreflect.FieldNumber(n1) == n2
 }
diff --git a/vendor/github.com/golang/protobuf/proto/lib.go b/vendor/github.com/golang/protobuf/proto/lib.go
deleted file mode 100644
index fdd328b..0000000
--- a/vendor/github.com/golang/protobuf/proto/lib.go
+++ /dev/null
@@ -1,965 +0,0 @@
-// 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.
-
-/*
-Package proto converts data structures to and from the wire format of
-protocol buffers.  It works in concert with the Go source code generated
-for .proto files by the protocol compiler.
-
-A summary of the properties of the protocol buffer interface
-for a protocol buffer variable v:
-
-  - Names are turned from camel_case to CamelCase for export.
-  - There are no methods on v to set fields; just treat
-	them as structure fields.
-  - There are getters that return a field's value if set,
-	and return the field's default value if unset.
-	The getters work even if the receiver is a nil message.
-  - The zero value for a struct is its correct initialization state.
-	All desired fields must be set before marshaling.
-  - A Reset() method will restore a protobuf struct to its zero state.
-  - Non-repeated fields are pointers to the values; nil means unset.
-	That is, optional or required field int32 f becomes F *int32.
-  - Repeated fields are slices.
-  - Helper functions are available to aid the setting of fields.
-	msg.Foo = proto.String("hello") // set field
-  - Constants are defined to hold the default values of all fields that
-	have them.  They have the form Default_StructName_FieldName.
-	Because the getter methods handle defaulted values,
-	direct use of these constants should be rare.
-  - Enums are given type names and maps from names to values.
-	Enum values are prefixed by the enclosing message's name, or by the
-	enum's type name if it is a top-level enum. Enum types have a String
-	method, and a Enum method to assist in message construction.
-  - Nested messages, groups and enums have type names prefixed with the name of
-	the surrounding message type.
-  - Extensions are given descriptor names that start with E_,
-	followed by an underscore-delimited list of the nested messages
-	that contain it (if any) followed by the CamelCased name of the
-	extension field itself.  HasExtension, ClearExtension, GetExtension
-	and SetExtension are functions for manipulating extensions.
-  - Oneof field sets are given a single field in their message,
-	with distinguished wrapper types for each possible field value.
-  - Marshal and Unmarshal are functions to encode and decode the wire format.
-
-When the .proto file specifies `syntax="proto3"`, there are some differences:
-
-  - Non-repeated fields of non-message type are values instead of pointers.
-  - Enum types do not get an Enum method.
-
-The simplest way to describe this is to see an example.
-Given file test.proto, containing
-
-	package example;
-
-	enum FOO { X = 17; }
-
-	message Test {
-	  required string label = 1;
-	  optional int32 type = 2 [default=77];
-	  repeated int64 reps = 3;
-	  optional group OptionalGroup = 4 {
-	    required string RequiredField = 5;
-	  }
-	  oneof union {
-	    int32 number = 6;
-	    string name = 7;
-	  }
-	}
-
-The resulting file, test.pb.go, is:
-
-	package example
-
-	import proto "github.com/golang/protobuf/proto"
-	import math "math"
-
-	type FOO int32
-	const (
-		FOO_X FOO = 17
-	)
-	var FOO_name = map[int32]string{
-		17: "X",
-	}
-	var FOO_value = map[string]int32{
-		"X": 17,
-	}
-
-	func (x FOO) Enum() *FOO {
-		p := new(FOO)
-		*p = x
-		return p
-	}
-	func (x FOO) String() string {
-		return proto.EnumName(FOO_name, int32(x))
-	}
-	func (x *FOO) UnmarshalJSON(data []byte) error {
-		value, err := proto.UnmarshalJSONEnum(FOO_value, data)
-		if err != nil {
-			return err
-		}
-		*x = FOO(value)
-		return nil
-	}
-
-	type Test struct {
-		Label         *string             `protobuf:"bytes,1,req,name=label" json:"label,omitempty"`
-		Type          *int32              `protobuf:"varint,2,opt,name=type,def=77" json:"type,omitempty"`
-		Reps          []int64             `protobuf:"varint,3,rep,name=reps" json:"reps,omitempty"`
-		Optionalgroup *Test_OptionalGroup `protobuf:"group,4,opt,name=OptionalGroup" json:"optionalgroup,omitempty"`
-		// Types that are valid to be assigned to Union:
-		//	*Test_Number
-		//	*Test_Name
-		Union            isTest_Union `protobuf_oneof:"union"`
-		XXX_unrecognized []byte       `json:"-"`
-	}
-	func (m *Test) Reset()         { *m = Test{} }
-	func (m *Test) String() string { return proto.CompactTextString(m) }
-	func (*Test) ProtoMessage() {}
-
-	type isTest_Union interface {
-		isTest_Union()
-	}
-
-	type Test_Number struct {
-		Number int32 `protobuf:"varint,6,opt,name=number"`
-	}
-	type Test_Name struct {
-		Name string `protobuf:"bytes,7,opt,name=name"`
-	}
-
-	func (*Test_Number) isTest_Union() {}
-	func (*Test_Name) isTest_Union()   {}
-
-	func (m *Test) GetUnion() isTest_Union {
-		if m != nil {
-			return m.Union
-		}
-		return nil
-	}
-	const Default_Test_Type int32 = 77
-
-	func (m *Test) GetLabel() string {
-		if m != nil && m.Label != nil {
-			return *m.Label
-		}
-		return ""
-	}
-
-	func (m *Test) GetType() int32 {
-		if m != nil && m.Type != nil {
-			return *m.Type
-		}
-		return Default_Test_Type
-	}
-
-	func (m *Test) GetOptionalgroup() *Test_OptionalGroup {
-		if m != nil {
-			return m.Optionalgroup
-		}
-		return nil
-	}
-
-	type Test_OptionalGroup struct {
-		RequiredField *string `protobuf:"bytes,5,req" json:"RequiredField,omitempty"`
-	}
-	func (m *Test_OptionalGroup) Reset()         { *m = Test_OptionalGroup{} }
-	func (m *Test_OptionalGroup) String() string { return proto.CompactTextString(m) }
-
-	func (m *Test_OptionalGroup) GetRequiredField() string {
-		if m != nil && m.RequiredField != nil {
-			return *m.RequiredField
-		}
-		return ""
-	}
-
-	func (m *Test) GetNumber() int32 {
-		if x, ok := m.GetUnion().(*Test_Number); ok {
-			return x.Number
-		}
-		return 0
-	}
-
-	func (m *Test) GetName() string {
-		if x, ok := m.GetUnion().(*Test_Name); ok {
-			return x.Name
-		}
-		return ""
-	}
-
-	func init() {
-		proto.RegisterEnum("example.FOO", FOO_name, FOO_value)
-	}
-
-To create and play with a Test object:
-
-	package main
-
-	import (
-		"log"
-
-		"github.com/golang/protobuf/proto"
-		pb "./example.pb"
-	)
-
-	func main() {
-		test := &pb.Test{
-			Label: proto.String("hello"),
-			Type:  proto.Int32(17),
-			Reps:  []int64{1, 2, 3},
-			Optionalgroup: &pb.Test_OptionalGroup{
-				RequiredField: proto.String("good bye"),
-			},
-			Union: &pb.Test_Name{"fred"},
-		}
-		data, err := proto.Marshal(test)
-		if err != nil {
-			log.Fatal("marshaling error: ", err)
-		}
-		newTest := &pb.Test{}
-		err = proto.Unmarshal(data, newTest)
-		if err != nil {
-			log.Fatal("unmarshaling error: ", err)
-		}
-		// Now test and newTest contain the same data.
-		if test.GetLabel() != newTest.GetLabel() {
-			log.Fatalf("data mismatch %q != %q", test.GetLabel(), newTest.GetLabel())
-		}
-		// Use a type switch to determine which oneof was set.
-		switch u := test.Union.(type) {
-		case *pb.Test_Number: // u.Number contains the number.
-		case *pb.Test_Name: // u.Name contains the string.
-		}
-		// etc.
-	}
-*/
-package proto
-
-import (
-	"encoding/json"
-	"fmt"
-	"log"
-	"reflect"
-	"sort"
-	"strconv"
-	"sync"
-)
-
-// RequiredNotSetError is an error type returned by either Marshal or Unmarshal.
-// Marshal reports this when a required field is not initialized.
-// Unmarshal reports this when a required field is missing from the wire data.
-type RequiredNotSetError struct{ field string }
-
-func (e *RequiredNotSetError) Error() string {
-	if e.field == "" {
-		return fmt.Sprintf("proto: required field not set")
-	}
-	return fmt.Sprintf("proto: required field %q not set", e.field)
-}
-func (e *RequiredNotSetError) RequiredNotSet() bool {
-	return true
-}
-
-type invalidUTF8Error struct{ field string }
-
-func (e *invalidUTF8Error) Error() string {
-	if e.field == "" {
-		return "proto: invalid UTF-8 detected"
-	}
-	return fmt.Sprintf("proto: field %q contains invalid UTF-8", e.field)
-}
-func (e *invalidUTF8Error) InvalidUTF8() bool {
-	return true
-}
-
-// errInvalidUTF8 is a sentinel error to identify fields with invalid UTF-8.
-// This error should not be exposed to the external API as such errors should
-// be recreated with the field information.
-var errInvalidUTF8 = &invalidUTF8Error{}
-
-// isNonFatal reports whether the error is either a RequiredNotSet error
-// or a InvalidUTF8 error.
-func isNonFatal(err error) bool {
-	if re, ok := err.(interface{ RequiredNotSet() bool }); ok && re.RequiredNotSet() {
-		return true
-	}
-	if re, ok := err.(interface{ InvalidUTF8() bool }); ok && re.InvalidUTF8() {
-		return true
-	}
-	return false
-}
-
-type nonFatal struct{ E error }
-
-// Merge merges err into nf and reports whether it was successful.
-// Otherwise it returns false for any fatal non-nil errors.
-func (nf *nonFatal) Merge(err error) (ok bool) {
-	if err == nil {
-		return true // not an error
-	}
-	if !isNonFatal(err) {
-		return false // fatal error
-	}
-	if nf.E == nil {
-		nf.E = err // store first instance of non-fatal error
-	}
-	return true
-}
-
-// Message is implemented by generated protocol buffer messages.
-type Message interface {
-	Reset()
-	String() string
-	ProtoMessage()
-}
-
-// A Buffer is a buffer manager for marshaling and unmarshaling
-// protocol buffers.  It may be reused between invocations to
-// reduce memory usage.  It is not necessary to use a Buffer;
-// the global functions Marshal and Unmarshal create a
-// temporary Buffer and are fine for most applications.
-type Buffer struct {
-	buf   []byte // encode/decode byte stream
-	index int    // read point
-
-	deterministic bool
-}
-
-// NewBuffer allocates a new Buffer and initializes its internal data to
-// the contents of the argument slice.
-func NewBuffer(e []byte) *Buffer {
-	return &Buffer{buf: e}
-}
-
-// Reset resets the Buffer, ready for marshaling a new protocol buffer.
-func (p *Buffer) Reset() {
-	p.buf = p.buf[0:0] // for reading/writing
-	p.index = 0        // for reading
-}
-
-// SetBuf replaces the internal buffer with the slice,
-// ready for unmarshaling the contents of the slice.
-func (p *Buffer) SetBuf(s []byte) {
-	p.buf = s
-	p.index = 0
-}
-
-// Bytes returns the contents of the Buffer.
-func (p *Buffer) Bytes() []byte { return p.buf }
-
-// SetDeterministic sets whether to use deterministic serialization.
-//
-// Deterministic serialization guarantees that for a given binary, equal
-// messages will always be serialized to the same bytes. This implies:
-//
-//   - Repeated serialization of a message will return the same bytes.
-//   - Different processes of the same binary (which may be executing on
-//     different machines) will serialize equal messages to the same bytes.
-//
-// Note that the deterministic serialization is NOT canonical across
-// languages. It is not guaranteed to remain stable over time. It is unstable
-// across different builds with schema changes due to unknown fields.
-// Users who need canonical serialization (e.g., persistent storage in a
-// canonical form, fingerprinting, etc.) should define their own
-// canonicalization specification and implement their own serializer rather
-// than relying on this API.
-//
-// If deterministic serialization is requested, map entries will be sorted
-// by keys in lexographical order. This is an implementation detail and
-// subject to change.
-func (p *Buffer) SetDeterministic(deterministic bool) {
-	p.deterministic = deterministic
-}
-
-/*
- * Helper routines for simplifying the creation of optional fields of basic type.
- */
-
-// Bool is a helper routine that allocates a new bool value
-// to store v and returns a pointer to it.
-func Bool(v bool) *bool {
-	return &v
-}
-
-// Int32 is a helper routine that allocates a new int32 value
-// to store v and returns a pointer to it.
-func Int32(v int32) *int32 {
-	return &v
-}
-
-// Int is a helper routine that allocates a new int32 value
-// to store v and returns a pointer to it, but unlike Int32
-// its argument value is an int.
-func Int(v int) *int32 {
-	p := new(int32)
-	*p = int32(v)
-	return p
-}
-
-// Int64 is a helper routine that allocates a new int64 value
-// to store v and returns a pointer to it.
-func Int64(v int64) *int64 {
-	return &v
-}
-
-// Float32 is a helper routine that allocates a new float32 value
-// to store v and returns a pointer to it.
-func Float32(v float32) *float32 {
-	return &v
-}
-
-// Float64 is a helper routine that allocates a new float64 value
-// to store v and returns a pointer to it.
-func Float64(v float64) *float64 {
-	return &v
-}
-
-// Uint32 is a helper routine that allocates a new uint32 value
-// to store v and returns a pointer to it.
-func Uint32(v uint32) *uint32 {
-	return &v
-}
-
-// Uint64 is a helper routine that allocates a new uint64 value
-// to store v and returns a pointer to it.
-func Uint64(v uint64) *uint64 {
-	return &v
-}
-
-// String is a helper routine that allocates a new string value
-// to store v and returns a pointer to it.
-func String(v string) *string {
-	return &v
-}
-
-// EnumName is a helper function to simplify printing protocol buffer enums
-// by name.  Given an enum map and a value, it returns a useful string.
-func EnumName(m map[int32]string, v int32) string {
-	s, ok := m[v]
-	if ok {
-		return s
-	}
-	return strconv.Itoa(int(v))
-}
-
-// UnmarshalJSONEnum is a helper function to simplify recovering enum int values
-// from their JSON-encoded representation. Given a map from the enum's symbolic
-// names to its int values, and a byte buffer containing the JSON-encoded
-// value, it returns an int32 that can be cast to the enum type by the caller.
-//
-// The function can deal with both JSON representations, numeric and symbolic.
-func UnmarshalJSONEnum(m map[string]int32, data []byte, enumName string) (int32, error) {
-	if data[0] == '"' {
-		// New style: enums are strings.
-		var repr string
-		if err := json.Unmarshal(data, &repr); err != nil {
-			return -1, err
-		}
-		val, ok := m[repr]
-		if !ok {
-			return 0, fmt.Errorf("unrecognized enum %s value %q", enumName, repr)
-		}
-		return val, nil
-	}
-	// Old style: enums are ints.
-	var val int32
-	if err := json.Unmarshal(data, &val); err != nil {
-		return 0, fmt.Errorf("cannot unmarshal %#q into enum %s", data, enumName)
-	}
-	return val, nil
-}
-
-// DebugPrint dumps the encoded data in b in a debugging format with a header
-// including the string s. Used in testing but made available for general debugging.
-func (p *Buffer) DebugPrint(s string, b []byte) {
-	var u uint64
-
-	obuf := p.buf
-	index := p.index
-	p.buf = b
-	p.index = 0
-	depth := 0
-
-	fmt.Printf("\n--- %s ---\n", s)
-
-out:
-	for {
-		for i := 0; i < depth; i++ {
-			fmt.Print("  ")
-		}
-
-		index := p.index
-		if index == len(p.buf) {
-			break
-		}
-
-		op, err := p.DecodeVarint()
-		if err != nil {
-			fmt.Printf("%3d: fetching op err %v\n", index, err)
-			break out
-		}
-		tag := op >> 3
-		wire := op & 7
-
-		switch wire {
-		default:
-			fmt.Printf("%3d: t=%3d unknown wire=%d\n",
-				index, tag, wire)
-			break out
-
-		case WireBytes:
-			var r []byte
-
-			r, err = p.DecodeRawBytes(false)
-			if err != nil {
-				break out
-			}
-			fmt.Printf("%3d: t=%3d bytes [%d]", index, tag, len(r))
-			if len(r) <= 6 {
-				for i := 0; i < len(r); i++ {
-					fmt.Printf(" %.2x", r[i])
-				}
-			} else {
-				for i := 0; i < 3; i++ {
-					fmt.Printf(" %.2x", r[i])
-				}
-				fmt.Printf(" ..")
-				for i := len(r) - 3; i < len(r); i++ {
-					fmt.Printf(" %.2x", r[i])
-				}
-			}
-			fmt.Printf("\n")
-
-		case WireFixed32:
-			u, err = p.DecodeFixed32()
-			if err != nil {
-				fmt.Printf("%3d: t=%3d fix32 err %v\n", index, tag, err)
-				break out
-			}
-			fmt.Printf("%3d: t=%3d fix32 %d\n", index, tag, u)
-
-		case WireFixed64:
-			u, err = p.DecodeFixed64()
-			if err != nil {
-				fmt.Printf("%3d: t=%3d fix64 err %v\n", index, tag, err)
-				break out
-			}
-			fmt.Printf("%3d: t=%3d fix64 %d\n", index, tag, u)
-
-		case WireVarint:
-			u, err = p.DecodeVarint()
-			if err != nil {
-				fmt.Printf("%3d: t=%3d varint err %v\n", index, tag, err)
-				break out
-			}
-			fmt.Printf("%3d: t=%3d varint %d\n", index, tag, u)
-
-		case WireStartGroup:
-			fmt.Printf("%3d: t=%3d start\n", index, tag)
-			depth++
-
-		case WireEndGroup:
-			depth--
-			fmt.Printf("%3d: t=%3d end\n", index, tag)
-		}
-	}
-
-	if depth != 0 {
-		fmt.Printf("%3d: start-end not balanced %d\n", p.index, depth)
-	}
-	fmt.Printf("\n")
-
-	p.buf = obuf
-	p.index = index
-}
-
-// SetDefaults sets unset protocol buffer fields to their default values.
-// It only modifies fields that are both unset and have defined defaults.
-// It recursively sets default values in any non-nil sub-messages.
-func SetDefaults(pb Message) {
-	setDefaults(reflect.ValueOf(pb), true, false)
-}
-
-// v is a pointer to a struct.
-func setDefaults(v reflect.Value, recur, zeros bool) {
-	v = v.Elem()
-
-	defaultMu.RLock()
-	dm, ok := defaults[v.Type()]
-	defaultMu.RUnlock()
-	if !ok {
-		dm = buildDefaultMessage(v.Type())
-		defaultMu.Lock()
-		defaults[v.Type()] = dm
-		defaultMu.Unlock()
-	}
-
-	for _, sf := range dm.scalars {
-		f := v.Field(sf.index)
-		if !f.IsNil() {
-			// field already set
-			continue
-		}
-		dv := sf.value
-		if dv == nil && !zeros {
-			// no explicit default, and don't want to set zeros
-			continue
-		}
-		fptr := f.Addr().Interface() // **T
-		// TODO: Consider batching the allocations we do here.
-		switch sf.kind {
-		case reflect.Bool:
-			b := new(bool)
-			if dv != nil {
-				*b = dv.(bool)
-			}
-			*(fptr.(**bool)) = b
-		case reflect.Float32:
-			f := new(float32)
-			if dv != nil {
-				*f = dv.(float32)
-			}
-			*(fptr.(**float32)) = f
-		case reflect.Float64:
-			f := new(float64)
-			if dv != nil {
-				*f = dv.(float64)
-			}
-			*(fptr.(**float64)) = f
-		case reflect.Int32:
-			// might be an enum
-			if ft := f.Type(); ft != int32PtrType {
-				// enum
-				f.Set(reflect.New(ft.Elem()))
-				if dv != nil {
-					f.Elem().SetInt(int64(dv.(int32)))
-				}
-			} else {
-				// int32 field
-				i := new(int32)
-				if dv != nil {
-					*i = dv.(int32)
-				}
-				*(fptr.(**int32)) = i
-			}
-		case reflect.Int64:
-			i := new(int64)
-			if dv != nil {
-				*i = dv.(int64)
-			}
-			*(fptr.(**int64)) = i
-		case reflect.String:
-			s := new(string)
-			if dv != nil {
-				*s = dv.(string)
-			}
-			*(fptr.(**string)) = s
-		case reflect.Uint8:
-			// exceptional case: []byte
-			var b []byte
-			if dv != nil {
-				db := dv.([]byte)
-				b = make([]byte, len(db))
-				copy(b, db)
-			} else {
-				b = []byte{}
-			}
-			*(fptr.(*[]byte)) = b
-		case reflect.Uint32:
-			u := new(uint32)
-			if dv != nil {
-				*u = dv.(uint32)
-			}
-			*(fptr.(**uint32)) = u
-		case reflect.Uint64:
-			u := new(uint64)
-			if dv != nil {
-				*u = dv.(uint64)
-			}
-			*(fptr.(**uint64)) = u
-		default:
-			log.Printf("proto: can't set default for field %v (sf.kind=%v)", f, sf.kind)
-		}
-	}
-
-	for _, ni := range dm.nested {
-		f := v.Field(ni)
-		// f is *T or []*T or map[T]*T
-		switch f.Kind() {
-		case reflect.Ptr:
-			if f.IsNil() {
-				continue
-			}
-			setDefaults(f, recur, zeros)
-
-		case reflect.Slice:
-			for i := 0; i < f.Len(); i++ {
-				e := f.Index(i)
-				if e.IsNil() {
-					continue
-				}
-				setDefaults(e, recur, zeros)
-			}
-
-		case reflect.Map:
-			for _, k := range f.MapKeys() {
-				e := f.MapIndex(k)
-				if e.IsNil() {
-					continue
-				}
-				setDefaults(e, recur, zeros)
-			}
-		}
-	}
-}
-
-var (
-	// defaults maps a protocol buffer struct type to a slice of the fields,
-	// with its scalar fields set to their proto-declared non-zero default values.
-	defaultMu sync.RWMutex
-	defaults  = make(map[reflect.Type]defaultMessage)
-
-	int32PtrType = reflect.TypeOf((*int32)(nil))
-)
-
-// defaultMessage represents information about the default values of a message.
-type defaultMessage struct {
-	scalars []scalarField
-	nested  []int // struct field index of nested messages
-}
-
-type scalarField struct {
-	index int          // struct field index
-	kind  reflect.Kind // element type (the T in *T or []T)
-	value interface{}  // the proto-declared default value, or nil
-}
-
-// t is a struct type.
-func buildDefaultMessage(t reflect.Type) (dm defaultMessage) {
-	sprop := GetProperties(t)
-	for _, prop := range sprop.Prop {
-		fi, ok := sprop.decoderTags.get(prop.Tag)
-		if !ok {
-			// XXX_unrecognized
-			continue
-		}
-		ft := t.Field(fi).Type
-
-		sf, nested, err := fieldDefault(ft, prop)
-		switch {
-		case err != nil:
-			log.Print(err)
-		case nested:
-			dm.nested = append(dm.nested, fi)
-		case sf != nil:
-			sf.index = fi
-			dm.scalars = append(dm.scalars, *sf)
-		}
-	}
-
-	return dm
-}
-
-// fieldDefault returns the scalarField for field type ft.
-// sf will be nil if the field can not have a default.
-// nestedMessage will be true if this is a nested message.
-// Note that sf.index is not set on return.
-func fieldDefault(ft reflect.Type, prop *Properties) (sf *scalarField, nestedMessage bool, err error) {
-	var canHaveDefault bool
-	switch ft.Kind() {
-	case reflect.Ptr:
-		if ft.Elem().Kind() == reflect.Struct {
-			nestedMessage = true
-		} else {
-			canHaveDefault = true // proto2 scalar field
-		}
-
-	case reflect.Slice:
-		switch ft.Elem().Kind() {
-		case reflect.Ptr:
-			nestedMessage = true // repeated message
-		case reflect.Uint8:
-			canHaveDefault = true // bytes field
-		}
-
-	case reflect.Map:
-		if ft.Elem().Kind() == reflect.Ptr {
-			nestedMessage = true // map with message values
-		}
-	}
-
-	if !canHaveDefault {
-		if nestedMessage {
-			return nil, true, nil
-		}
-		return nil, false, nil
-	}
-
-	// We now know that ft is a pointer or slice.
-	sf = &scalarField{kind: ft.Elem().Kind()}
-
-	// scalar fields without defaults
-	if !prop.HasDefault {
-		return sf, false, nil
-	}
-
-	// a scalar field: either *T or []byte
-	switch ft.Elem().Kind() {
-	case reflect.Bool:
-		x, err := strconv.ParseBool(prop.Default)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default bool %q: %v", prop.Default, err)
-		}
-		sf.value = x
-	case reflect.Float32:
-		x, err := strconv.ParseFloat(prop.Default, 32)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default float32 %q: %v", prop.Default, err)
-		}
-		sf.value = float32(x)
-	case reflect.Float64:
-		x, err := strconv.ParseFloat(prop.Default, 64)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default float64 %q: %v", prop.Default, err)
-		}
-		sf.value = x
-	case reflect.Int32:
-		x, err := strconv.ParseInt(prop.Default, 10, 32)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default int32 %q: %v", prop.Default, err)
-		}
-		sf.value = int32(x)
-	case reflect.Int64:
-		x, err := strconv.ParseInt(prop.Default, 10, 64)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default int64 %q: %v", prop.Default, err)
-		}
-		sf.value = x
-	case reflect.String:
-		sf.value = prop.Default
-	case reflect.Uint8:
-		// []byte (not *uint8)
-		sf.value = []byte(prop.Default)
-	case reflect.Uint32:
-		x, err := strconv.ParseUint(prop.Default, 10, 32)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default uint32 %q: %v", prop.Default, err)
-		}
-		sf.value = uint32(x)
-	case reflect.Uint64:
-		x, err := strconv.ParseUint(prop.Default, 10, 64)
-		if err != nil {
-			return nil, false, fmt.Errorf("proto: bad default uint64 %q: %v", prop.Default, err)
-		}
-		sf.value = x
-	default:
-		return nil, false, fmt.Errorf("proto: unhandled def kind %v", ft.Elem().Kind())
-	}
-
-	return sf, false, nil
-}
-
-// mapKeys returns a sort.Interface to be used for sorting the map keys.
-// Map fields may have key types of non-float scalars, strings and enums.
-func mapKeys(vs []reflect.Value) sort.Interface {
-	s := mapKeySorter{vs: vs}
-
-	// Type specialization per https://developers.google.com/protocol-buffers/docs/proto#maps.
-	if len(vs) == 0 {
-		return s
-	}
-	switch vs[0].Kind() {
-	case reflect.Int32, reflect.Int64:
-		s.less = func(a, b reflect.Value) bool { return a.Int() < b.Int() }
-	case reflect.Uint32, reflect.Uint64:
-		s.less = func(a, b reflect.Value) bool { return a.Uint() < b.Uint() }
-	case reflect.Bool:
-		s.less = func(a, b reflect.Value) bool { return !a.Bool() && b.Bool() } // false < true
-	case reflect.String:
-		s.less = func(a, b reflect.Value) bool { return a.String() < b.String() }
-	default:
-		panic(fmt.Sprintf("unsupported map key type: %v", vs[0].Kind()))
-	}
-
-	return s
-}
-
-type mapKeySorter struct {
-	vs   []reflect.Value
-	less func(a, b reflect.Value) bool
-}
-
-func (s mapKeySorter) Len() int      { return len(s.vs) }
-func (s mapKeySorter) Swap(i, j int) { s.vs[i], s.vs[j] = s.vs[j], s.vs[i] }
-func (s mapKeySorter) Less(i, j int) bool {
-	return s.less(s.vs[i], s.vs[j])
-}
-
-// isProto3Zero reports whether v is a zero proto3 value.
-func isProto3Zero(v reflect.Value) bool {
-	switch v.Kind() {
-	case reflect.Bool:
-		return !v.Bool()
-	case reflect.Int32, reflect.Int64:
-		return v.Int() == 0
-	case reflect.Uint32, reflect.Uint64:
-		return v.Uint() == 0
-	case reflect.Float32, reflect.Float64:
-		return v.Float() == 0
-	case reflect.String:
-		return v.String() == ""
-	}
-	return false
-}
-
-const (
-	// ProtoPackageIsVersion3 is referenced from generated protocol buffer files
-	// to assert that that code is compatible with this version of the proto package.
-	ProtoPackageIsVersion3 = true
-
-	// ProtoPackageIsVersion2 is referenced from generated protocol buffer files
-	// to assert that that code is compatible with this version of the proto package.
-	ProtoPackageIsVersion2 = true
-
-	// ProtoPackageIsVersion1 is referenced from generated protocol buffer files
-	// to assert that that code is compatible with this version of the proto package.
-	ProtoPackageIsVersion1 = true
-)
-
-// InternalMessageInfo is a type used internally by generated .pb.go files.
-// This type is not intended to be used by non-generated code.
-// This type is not subject to any compatibility guarantee.
-type InternalMessageInfo struct {
-	marshal   *marshalInfo
-	unmarshal *unmarshalInfo
-	merge     *mergeInfo
-	discard   *discardInfo
-}
diff --git a/vendor/github.com/golang/protobuf/proto/message_set.go b/vendor/github.com/golang/protobuf/proto/message_set.go
deleted file mode 100644
index f48a756..0000000
--- a/vendor/github.com/golang/protobuf/proto/message_set.go
+++ /dev/null
@@ -1,181 +0,0 @@
-// 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.
-
-package proto
-
-/*
- * Support for message sets.
- */
-
-import (
-	"errors"
-)
-
-// errNoMessageTypeID occurs when a protocol buffer does not have a message type ID.
-// A message type ID is required for storing a protocol buffer in a message set.
-var errNoMessageTypeID = errors.New("proto does not have a message type ID")
-
-// The first two types (_MessageSet_Item and messageSet)
-// model what the protocol compiler produces for the following protocol message:
-//   message MessageSet {
-//     repeated group Item = 1 {
-//       required int32 type_id = 2;
-//       required string message = 3;
-//     };
-//   }
-// That is the MessageSet wire format. We can't use a proto to generate these
-// because that would introduce a circular dependency between it and this package.
-
-type _MessageSet_Item struct {
-	TypeId  *int32 `protobuf:"varint,2,req,name=type_id"`
-	Message []byte `protobuf:"bytes,3,req,name=message"`
-}
-
-type messageSet struct {
-	Item             []*_MessageSet_Item `protobuf:"group,1,rep"`
-	XXX_unrecognized []byte
-	// TODO: caching?
-}
-
-// Make sure messageSet is a Message.
-var _ Message = (*messageSet)(nil)
-
-// messageTypeIder is an interface satisfied by a protocol buffer type
-// that may be stored in a MessageSet.
-type messageTypeIder interface {
-	MessageTypeId() int32
-}
-
-func (ms *messageSet) find(pb Message) *_MessageSet_Item {
-	mti, ok := pb.(messageTypeIder)
-	if !ok {
-		return nil
-	}
-	id := mti.MessageTypeId()
-	for _, item := range ms.Item {
-		if *item.TypeId == id {
-			return item
-		}
-	}
-	return nil
-}
-
-func (ms *messageSet) Has(pb Message) bool {
-	return ms.find(pb) != nil
-}
-
-func (ms *messageSet) Unmarshal(pb Message) error {
-	if item := ms.find(pb); item != nil {
-		return Unmarshal(item.Message, pb)
-	}
-	if _, ok := pb.(messageTypeIder); !ok {
-		return errNoMessageTypeID
-	}
-	return nil // TODO: return error instead?
-}
-
-func (ms *messageSet) Marshal(pb Message) error {
-	msg, err := Marshal(pb)
-	if err != nil {
-		return err
-	}
-	if item := ms.find(pb); item != nil {
-		// reuse existing item
-		item.Message = msg
-		return nil
-	}
-
-	mti, ok := pb.(messageTypeIder)
-	if !ok {
-		return errNoMessageTypeID
-	}
-
-	mtid := mti.MessageTypeId()
-	ms.Item = append(ms.Item, &_MessageSet_Item{
-		TypeId:  &mtid,
-		Message: msg,
-	})
-	return nil
-}
-
-func (ms *messageSet) Reset()         { *ms = messageSet{} }
-func (ms *messageSet) String() string { return CompactTextString(ms) }
-func (*messageSet) ProtoMessage()     {}
-
-// Support for the message_set_wire_format message option.
-
-func skipVarint(buf []byte) []byte {
-	i := 0
-	for ; buf[i]&0x80 != 0; i++ {
-	}
-	return buf[i+1:]
-}
-
-// unmarshalMessageSet decodes the extension map encoded in buf in the message set wire format.
-// It is called by Unmarshal methods on protocol buffer messages with the message_set_wire_format option.
-func unmarshalMessageSet(buf []byte, exts interface{}) error {
-	var m map[int32]Extension
-	switch exts := exts.(type) {
-	case *XXX_InternalExtensions:
-		m = exts.extensionsWrite()
-	case map[int32]Extension:
-		m = exts
-	default:
-		return errors.New("proto: not an extension map")
-	}
-
-	ms := new(messageSet)
-	if err := Unmarshal(buf, ms); err != nil {
-		return err
-	}
-	for _, item := range ms.Item {
-		id := *item.TypeId
-		msg := item.Message
-
-		// Restore wire type and field number varint, plus length varint.
-		// Be careful to preserve duplicate items.
-		b := EncodeVarint(uint64(id)<<3 | WireBytes)
-		if ext, ok := m[id]; ok {
-			// Existing data; rip off the tag and length varint
-			// so we join the new data correctly.
-			// We can assume that ext.enc is set because we are unmarshaling.
-			o := ext.enc[len(b):]   // skip wire type and field number
-			_, n := DecodeVarint(o) // calculate length of length varint
-			o = o[n:]               // skip length varint
-			msg = append(o, msg...) // join old data and new data
-		}
-		b = append(b, EncodeVarint(uint64(len(msg)))...)
-		b = append(b, msg...)
-
-		m[id] = Extension{enc: b}
-	}
-	return nil
-}
diff --git a/vendor/github.com/golang/protobuf/proto/pointer_reflect.go b/vendor/github.com/golang/protobuf/proto/pointer_reflect.go
deleted file mode 100644
index 94fa919..0000000
--- a/vendor/github.com/golang/protobuf/proto/pointer_reflect.go
+++ /dev/null
@@ -1,360 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2012 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.
-
-// +build purego appengine js
-
-// This file contains an implementation of proto field accesses using package reflect.
-// It is slower than the code in pointer_unsafe.go but it avoids package unsafe and can
-// be used on App Engine.
-
-package proto
-
-import (
-	"reflect"
-	"sync"
-)
-
-const unsafeAllowed = false
-
-// A field identifies a field in a struct, accessible from a pointer.
-// In this implementation, a field is identified by the sequence of field indices
-// passed to reflect's FieldByIndex.
-type field []int
-
-// toField returns a field equivalent to the given reflect field.
-func toField(f *reflect.StructField) field {
-	return f.Index
-}
-
-// invalidField is an invalid field identifier.
-var invalidField = field(nil)
-
-// zeroField is a noop when calling pointer.offset.
-var zeroField = field([]int{})
-
-// IsValid reports whether the field identifier is valid.
-func (f field) IsValid() bool { return f != nil }
-
-// The pointer type is for the table-driven decoder.
-// The implementation here uses a reflect.Value of pointer type to
-// create a generic pointer. In pointer_unsafe.go we use unsafe
-// instead of reflect to implement the same (but faster) interface.
-type pointer struct {
-	v reflect.Value
-}
-
-// toPointer converts an interface of pointer type to a pointer
-// that points to the same target.
-func toPointer(i *Message) pointer {
-	return pointer{v: reflect.ValueOf(*i)}
-}
-
-// toAddrPointer converts an interface to a pointer that points to
-// the interface data.
-func toAddrPointer(i *interface{}, isptr, deref bool) pointer {
-	v := reflect.ValueOf(*i)
-	u := reflect.New(v.Type())
-	u.Elem().Set(v)
-	if deref {
-		u = u.Elem()
-	}
-	return pointer{v: u}
-}
-
-// valToPointer converts v to a pointer.  v must be of pointer type.
-func valToPointer(v reflect.Value) pointer {
-	return pointer{v: v}
-}
-
-// offset converts from a pointer to a structure to a pointer to
-// one of its fields.
-func (p pointer) offset(f field) pointer {
-	return pointer{v: p.v.Elem().FieldByIndex(f).Addr()}
-}
-
-func (p pointer) isNil() bool {
-	return p.v.IsNil()
-}
-
-// grow updates the slice s in place to make it one element longer.
-// s must be addressable.
-// Returns the (addressable) new element.
-func grow(s reflect.Value) reflect.Value {
-	n, m := s.Len(), s.Cap()
-	if n < m {
-		s.SetLen(n + 1)
-	} else {
-		s.Set(reflect.Append(s, reflect.Zero(s.Type().Elem())))
-	}
-	return s.Index(n)
-}
-
-func (p pointer) toInt64() *int64 {
-	return p.v.Interface().(*int64)
-}
-func (p pointer) toInt64Ptr() **int64 {
-	return p.v.Interface().(**int64)
-}
-func (p pointer) toInt64Slice() *[]int64 {
-	return p.v.Interface().(*[]int64)
-}
-
-var int32ptr = reflect.TypeOf((*int32)(nil))
-
-func (p pointer) toInt32() *int32 {
-	return p.v.Convert(int32ptr).Interface().(*int32)
-}
-
-// The toInt32Ptr/Slice methods don't work because of enums.
-// Instead, we must use set/get methods for the int32ptr/slice case.
-/*
-	func (p pointer) toInt32Ptr() **int32 {
-		return p.v.Interface().(**int32)
-}
-	func (p pointer) toInt32Slice() *[]int32 {
-		return p.v.Interface().(*[]int32)
-}
-*/
-func (p pointer) getInt32Ptr() *int32 {
-	if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) {
-		// raw int32 type
-		return p.v.Elem().Interface().(*int32)
-	}
-	// an enum
-	return p.v.Elem().Convert(int32PtrType).Interface().(*int32)
-}
-func (p pointer) setInt32Ptr(v int32) {
-	// Allocate value in a *int32. Possibly convert that to a *enum.
-	// Then assign it to a **int32 or **enum.
-	// Note: we can convert *int32 to *enum, but we can't convert
-	// **int32 to **enum!
-	p.v.Elem().Set(reflect.ValueOf(&v).Convert(p.v.Type().Elem()))
-}
-
-// getInt32Slice copies []int32 from p as a new slice.
-// This behavior differs from the implementation in pointer_unsafe.go.
-func (p pointer) getInt32Slice() []int32 {
-	if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) {
-		// raw int32 type
-		return p.v.Elem().Interface().([]int32)
-	}
-	// an enum
-	// Allocate a []int32, then assign []enum's values into it.
-	// Note: we can't convert []enum to []int32.
-	slice := p.v.Elem()
-	s := make([]int32, slice.Len())
-	for i := 0; i < slice.Len(); i++ {
-		s[i] = int32(slice.Index(i).Int())
-	}
-	return s
-}
-
-// setInt32Slice copies []int32 into p as a new slice.
-// This behavior differs from the implementation in pointer_unsafe.go.
-func (p pointer) setInt32Slice(v []int32) {
-	if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) {
-		// raw int32 type
-		p.v.Elem().Set(reflect.ValueOf(v))
-		return
-	}
-	// an enum
-	// Allocate a []enum, then assign []int32's values into it.
-	// Note: we can't convert []enum to []int32.
-	slice := reflect.MakeSlice(p.v.Type().Elem(), len(v), cap(v))
-	for i, x := range v {
-		slice.Index(i).SetInt(int64(x))
-	}
-	p.v.Elem().Set(slice)
-}
-func (p pointer) appendInt32Slice(v int32) {
-	grow(p.v.Elem()).SetInt(int64(v))
-}
-
-func (p pointer) toUint64() *uint64 {
-	return p.v.Interface().(*uint64)
-}
-func (p pointer) toUint64Ptr() **uint64 {
-	return p.v.Interface().(**uint64)
-}
-func (p pointer) toUint64Slice() *[]uint64 {
-	return p.v.Interface().(*[]uint64)
-}
-func (p pointer) toUint32() *uint32 {
-	return p.v.Interface().(*uint32)
-}
-func (p pointer) toUint32Ptr() **uint32 {
-	return p.v.Interface().(**uint32)
-}
-func (p pointer) toUint32Slice() *[]uint32 {
-	return p.v.Interface().(*[]uint32)
-}
-func (p pointer) toBool() *bool {
-	return p.v.Interface().(*bool)
-}
-func (p pointer) toBoolPtr() **bool {
-	return p.v.Interface().(**bool)
-}
-func (p pointer) toBoolSlice() *[]bool {
-	return p.v.Interface().(*[]bool)
-}
-func (p pointer) toFloat64() *float64 {
-	return p.v.Interface().(*float64)
-}
-func (p pointer) toFloat64Ptr() **float64 {
-	return p.v.Interface().(**float64)
-}
-func (p pointer) toFloat64Slice() *[]float64 {
-	return p.v.Interface().(*[]float64)
-}
-func (p pointer) toFloat32() *float32 {
-	return p.v.Interface().(*float32)
-}
-func (p pointer) toFloat32Ptr() **float32 {
-	return p.v.Interface().(**float32)
-}
-func (p pointer) toFloat32Slice() *[]float32 {
-	return p.v.Interface().(*[]float32)
-}
-func (p pointer) toString() *string {
-	return p.v.Interface().(*string)
-}
-func (p pointer) toStringPtr() **string {
-	return p.v.Interface().(**string)
-}
-func (p pointer) toStringSlice() *[]string {
-	return p.v.Interface().(*[]string)
-}
-func (p pointer) toBytes() *[]byte {
-	return p.v.Interface().(*[]byte)
-}
-func (p pointer) toBytesSlice() *[][]byte {
-	return p.v.Interface().(*[][]byte)
-}
-func (p pointer) toExtensions() *XXX_InternalExtensions {
-	return p.v.Interface().(*XXX_InternalExtensions)
-}
-func (p pointer) toOldExtensions() *map[int32]Extension {
-	return p.v.Interface().(*map[int32]Extension)
-}
-func (p pointer) getPointer() pointer {
-	return pointer{v: p.v.Elem()}
-}
-func (p pointer) setPointer(q pointer) {
-	p.v.Elem().Set(q.v)
-}
-func (p pointer) appendPointer(q pointer) {
-	grow(p.v.Elem()).Set(q.v)
-}
-
-// getPointerSlice copies []*T from p as a new []pointer.
-// This behavior differs from the implementation in pointer_unsafe.go.
-func (p pointer) getPointerSlice() []pointer {
-	if p.v.IsNil() {
-		return nil
-	}
-	n := p.v.Elem().Len()
-	s := make([]pointer, n)
-	for i := 0; i < n; i++ {
-		s[i] = pointer{v: p.v.Elem().Index(i)}
-	}
-	return s
-}
-
-// setPointerSlice copies []pointer into p as a new []*T.
-// This behavior differs from the implementation in pointer_unsafe.go.
-func (p pointer) setPointerSlice(v []pointer) {
-	if v == nil {
-		p.v.Elem().Set(reflect.New(p.v.Elem().Type()).Elem())
-		return
-	}
-	s := reflect.MakeSlice(p.v.Elem().Type(), 0, len(v))
-	for _, p := range v {
-		s = reflect.Append(s, p.v)
-	}
-	p.v.Elem().Set(s)
-}
-
-// getInterfacePointer returns a pointer that points to the
-// interface data of the interface pointed by p.
-func (p pointer) getInterfacePointer() pointer {
-	if p.v.Elem().IsNil() {
-		return pointer{v: p.v.Elem()}
-	}
-	return pointer{v: p.v.Elem().Elem().Elem().Field(0).Addr()} // *interface -> interface -> *struct -> struct
-}
-
-func (p pointer) asPointerTo(t reflect.Type) reflect.Value {
-	// TODO: check that p.v.Type().Elem() == t?
-	return p.v
-}
-
-func atomicLoadUnmarshalInfo(p **unmarshalInfo) *unmarshalInfo {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	return *p
-}
-func atomicStoreUnmarshalInfo(p **unmarshalInfo, v *unmarshalInfo) {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	*p = v
-}
-func atomicLoadMarshalInfo(p **marshalInfo) *marshalInfo {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	return *p
-}
-func atomicStoreMarshalInfo(p **marshalInfo, v *marshalInfo) {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	*p = v
-}
-func atomicLoadMergeInfo(p **mergeInfo) *mergeInfo {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	return *p
-}
-func atomicStoreMergeInfo(p **mergeInfo, v *mergeInfo) {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	*p = v
-}
-func atomicLoadDiscardInfo(p **discardInfo) *discardInfo {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	return *p
-}
-func atomicStoreDiscardInfo(p **discardInfo, v *discardInfo) {
-	atomicLock.Lock()
-	defer atomicLock.Unlock()
-	*p = v
-}
-
-var atomicLock sync.Mutex
diff --git a/vendor/github.com/golang/protobuf/proto/pointer_unsafe.go b/vendor/github.com/golang/protobuf/proto/pointer_unsafe.go
deleted file mode 100644
index dbfffe0..0000000
--- a/vendor/github.com/golang/protobuf/proto/pointer_unsafe.go
+++ /dev/null
@@ -1,313 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2012 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.
-
-// +build !purego,!appengine,!js
-
-// This file contains the implementation of the proto field accesses using package unsafe.
-
-package proto
-
-import (
-	"reflect"
-	"sync/atomic"
-	"unsafe"
-)
-
-const unsafeAllowed = true
-
-// A field identifies a field in a struct, accessible from a pointer.
-// In this implementation, a field is identified by its byte offset from the start of the struct.
-type field uintptr
-
-// toField returns a field equivalent to the given reflect field.
-func toField(f *reflect.StructField) field {
-	return field(f.Offset)
-}
-
-// invalidField is an invalid field identifier.
-const invalidField = ^field(0)
-
-// zeroField is a noop when calling pointer.offset.
-const zeroField = field(0)
-
-// IsValid reports whether the field identifier is valid.
-func (f field) IsValid() bool {
-	return f != invalidField
-}
-
-// The pointer type below is for the new table-driven encoder/decoder.
-// The implementation here uses unsafe.Pointer to create a generic pointer.
-// In pointer_reflect.go we use reflect instead of unsafe to implement
-// the same (but slower) interface.
-type pointer struct {
-	p unsafe.Pointer
-}
-
-// size of pointer
-var ptrSize = unsafe.Sizeof(uintptr(0))
-
-// toPointer converts an interface of pointer type to a pointer
-// that points to the same target.
-func toPointer(i *Message) pointer {
-	// Super-tricky - read pointer out of data word of interface value.
-	// Saves ~25ns over the equivalent:
-	// return valToPointer(reflect.ValueOf(*i))
-	return pointer{p: (*[2]unsafe.Pointer)(unsafe.Pointer(i))[1]}
-}
-
-// toAddrPointer converts an interface to a pointer that points to
-// the interface data.
-func toAddrPointer(i *interface{}, isptr, deref bool) (p pointer) {
-	// Super-tricky - read or get the address of data word of interface value.
-	if isptr {
-		// The interface is of pointer type, thus it is a direct interface.
-		// The data word is the pointer data itself. We take its address.
-		p = pointer{p: unsafe.Pointer(uintptr(unsafe.Pointer(i)) + ptrSize)}
-	} else {
-		// The interface is not of pointer type. The data word is the pointer
-		// to the data.
-		p = pointer{p: (*[2]unsafe.Pointer)(unsafe.Pointer(i))[1]}
-	}
-	if deref {
-		p.p = *(*unsafe.Pointer)(p.p)
-	}
-	return p
-}
-
-// valToPointer converts v to a pointer. v must be of pointer type.
-func valToPointer(v reflect.Value) pointer {
-	return pointer{p: unsafe.Pointer(v.Pointer())}
-}
-
-// offset converts from a pointer to a structure to a pointer to
-// one of its fields.
-func (p pointer) offset(f field) pointer {
-	// For safety, we should panic if !f.IsValid, however calling panic causes
-	// this to no longer be inlineable, which is a serious performance cost.
-	/*
-		if !f.IsValid() {
-			panic("invalid field")
-		}
-	*/
-	return pointer{p: unsafe.Pointer(uintptr(p.p) + uintptr(f))}
-}
-
-func (p pointer) isNil() bool {
-	return p.p == nil
-}
-
-func (p pointer) toInt64() *int64 {
-	return (*int64)(p.p)
-}
-func (p pointer) toInt64Ptr() **int64 {
-	return (**int64)(p.p)
-}
-func (p pointer) toInt64Slice() *[]int64 {
-	return (*[]int64)(p.p)
-}
-func (p pointer) toInt32() *int32 {
-	return (*int32)(p.p)
-}
-
-// See pointer_reflect.go for why toInt32Ptr/Slice doesn't exist.
-/*
-	func (p pointer) toInt32Ptr() **int32 {
-		return (**int32)(p.p)
-	}
-	func (p pointer) toInt32Slice() *[]int32 {
-		return (*[]int32)(p.p)
-	}
-*/
-func (p pointer) getInt32Ptr() *int32 {
-	return *(**int32)(p.p)
-}
-func (p pointer) setInt32Ptr(v int32) {
-	*(**int32)(p.p) = &v
-}
-
-// getInt32Slice loads a []int32 from p.
-// The value returned is aliased with the original slice.
-// This behavior differs from the implementation in pointer_reflect.go.
-func (p pointer) getInt32Slice() []int32 {
-	return *(*[]int32)(p.p)
-}
-
-// setInt32Slice stores a []int32 to p.
-// The value set is aliased with the input slice.
-// This behavior differs from the implementation in pointer_reflect.go.
-func (p pointer) setInt32Slice(v []int32) {
-	*(*[]int32)(p.p) = v
-}
-
-// TODO: Can we get rid of appendInt32Slice and use setInt32Slice instead?
-func (p pointer) appendInt32Slice(v int32) {
-	s := (*[]int32)(p.p)
-	*s = append(*s, v)
-}
-
-func (p pointer) toUint64() *uint64 {
-	return (*uint64)(p.p)
-}
-func (p pointer) toUint64Ptr() **uint64 {
-	return (**uint64)(p.p)
-}
-func (p pointer) toUint64Slice() *[]uint64 {
-	return (*[]uint64)(p.p)
-}
-func (p pointer) toUint32() *uint32 {
-	return (*uint32)(p.p)
-}
-func (p pointer) toUint32Ptr() **uint32 {
-	return (**uint32)(p.p)
-}
-func (p pointer) toUint32Slice() *[]uint32 {
-	return (*[]uint32)(p.p)
-}
-func (p pointer) toBool() *bool {
-	return (*bool)(p.p)
-}
-func (p pointer) toBoolPtr() **bool {
-	return (**bool)(p.p)
-}
-func (p pointer) toBoolSlice() *[]bool {
-	return (*[]bool)(p.p)
-}
-func (p pointer) toFloat64() *float64 {
-	return (*float64)(p.p)
-}
-func (p pointer) toFloat64Ptr() **float64 {
-	return (**float64)(p.p)
-}
-func (p pointer) toFloat64Slice() *[]float64 {
-	return (*[]float64)(p.p)
-}
-func (p pointer) toFloat32() *float32 {
-	return (*float32)(p.p)
-}
-func (p pointer) toFloat32Ptr() **float32 {
-	return (**float32)(p.p)
-}
-func (p pointer) toFloat32Slice() *[]float32 {
-	return (*[]float32)(p.p)
-}
-func (p pointer) toString() *string {
-	return (*string)(p.p)
-}
-func (p pointer) toStringPtr() **string {
-	return (**string)(p.p)
-}
-func (p pointer) toStringSlice() *[]string {
-	return (*[]string)(p.p)
-}
-func (p pointer) toBytes() *[]byte {
-	return (*[]byte)(p.p)
-}
-func (p pointer) toBytesSlice() *[][]byte {
-	return (*[][]byte)(p.p)
-}
-func (p pointer) toExtensions() *XXX_InternalExtensions {
-	return (*XXX_InternalExtensions)(p.p)
-}
-func (p pointer) toOldExtensions() *map[int32]Extension {
-	return (*map[int32]Extension)(p.p)
-}
-
-// getPointerSlice loads []*T from p as a []pointer.
-// The value returned is aliased with the original slice.
-// This behavior differs from the implementation in pointer_reflect.go.
-func (p pointer) getPointerSlice() []pointer {
-	// Super-tricky - p should point to a []*T where T is a
-	// message type. We load it as []pointer.
-	return *(*[]pointer)(p.p)
-}
-
-// setPointerSlice stores []pointer into p as a []*T.
-// The value set is aliased with the input slice.
-// This behavior differs from the implementation in pointer_reflect.go.
-func (p pointer) setPointerSlice(v []pointer) {
-	// Super-tricky - p should point to a []*T where T is a
-	// message type. We store it as []pointer.
-	*(*[]pointer)(p.p) = v
-}
-
-// getPointer loads the pointer at p and returns it.
-func (p pointer) getPointer() pointer {
-	return pointer{p: *(*unsafe.Pointer)(p.p)}
-}
-
-// setPointer stores the pointer q at p.
-func (p pointer) setPointer(q pointer) {
-	*(*unsafe.Pointer)(p.p) = q.p
-}
-
-// append q to the slice pointed to by p.
-func (p pointer) appendPointer(q pointer) {
-	s := (*[]unsafe.Pointer)(p.p)
-	*s = append(*s, q.p)
-}
-
-// getInterfacePointer returns a pointer that points to the
-// interface data of the interface pointed by p.
-func (p pointer) getInterfacePointer() pointer {
-	// Super-tricky - read pointer out of data word of interface value.
-	return pointer{p: (*(*[2]unsafe.Pointer)(p.p))[1]}
-}
-
-// asPointerTo returns a reflect.Value that is a pointer to an
-// object of type t stored at p.
-func (p pointer) asPointerTo(t reflect.Type) reflect.Value {
-	return reflect.NewAt(t, p.p)
-}
-
-func atomicLoadUnmarshalInfo(p **unmarshalInfo) *unmarshalInfo {
-	return (*unmarshalInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p))))
-}
-func atomicStoreUnmarshalInfo(p **unmarshalInfo, v *unmarshalInfo) {
-	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v))
-}
-func atomicLoadMarshalInfo(p **marshalInfo) *marshalInfo {
-	return (*marshalInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p))))
-}
-func atomicStoreMarshalInfo(p **marshalInfo, v *marshalInfo) {
-	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v))
-}
-func atomicLoadMergeInfo(p **mergeInfo) *mergeInfo {
-	return (*mergeInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p))))
-}
-func atomicStoreMergeInfo(p **mergeInfo, v *mergeInfo) {
-	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v))
-}
-func atomicLoadDiscardInfo(p **discardInfo) *discardInfo {
-	return (*discardInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p))))
-}
-func atomicStoreDiscardInfo(p **discardInfo, v *discardInfo) {
-	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v))
-}
diff --git a/vendor/github.com/golang/protobuf/proto/properties.go b/vendor/github.com/golang/protobuf/proto/properties.go
index a4b8c0c..dcdc220 100644
--- a/vendor/github.com/golang/protobuf/proto/properties.go
+++ b/vendor/github.com/golang/protobuf/proto/properties.go
@@ -1,162 +1,104 @@
-// 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.
+// 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 proto
 
-/*
- * Routines for encoding data into the wire format for protocol buffers.
- */
-
 import (
 	"fmt"
-	"log"
 	"reflect"
-	"sort"
 	"strconv"
 	"strings"
 	"sync"
+
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoimpl"
 )
 
-const debug bool = false
-
-// Constants that identify the encoding of a value on the wire.
-const (
-	WireVarint     = 0
-	WireFixed64    = 1
-	WireBytes      = 2
-	WireStartGroup = 3
-	WireEndGroup   = 4
-	WireFixed32    = 5
-)
-
-// tagMap is an optimization over map[int]int for typical protocol buffer
-// use-cases. Encoded protocol buffers are often in tag order with small tag
-// numbers.
-type tagMap struct {
-	fastTags []int
-	slowTags map[int]int
-}
-
-// tagMapFastLimit is the upper bound on the tag number that will be stored in
-// the tagMap slice rather than its map.
-const tagMapFastLimit = 1024
-
-func (p *tagMap) get(t int) (int, bool) {
-	if t > 0 && t < tagMapFastLimit {
-		if t >= len(p.fastTags) {
-			return 0, false
-		}
-		fi := p.fastTags[t]
-		return fi, fi >= 0
-	}
-	fi, ok := p.slowTags[t]
-	return fi, ok
-}
-
-func (p *tagMap) put(t int, fi int) {
-	if t > 0 && t < tagMapFastLimit {
-		for len(p.fastTags) < t+1 {
-			p.fastTags = append(p.fastTags, -1)
-		}
-		p.fastTags[t] = fi
-		return
-	}
-	if p.slowTags == nil {
-		p.slowTags = make(map[int]int)
-	}
-	p.slowTags[t] = fi
-}
-
-// StructProperties represents properties for all the fields of a struct.
-// decoderTags and decoderOrigNames should only be used by the decoder.
+// StructProperties represents protocol buffer type information for a
+// generated protobuf message in the open-struct API.
+//
+// Deprecated: Do not use.
 type StructProperties struct {
-	Prop             []*Properties  // properties for each field
-	reqCount         int            // required count
-	decoderTags      tagMap         // map from proto tag to struct field number
-	decoderOrigNames map[string]int // map from original name to struct field number
-	order            []int          // list of struct field numbers in tag order
+	// Prop are the properties for each field.
+	//
+	// Fields belonging to a oneof are stored in OneofTypes instead, with a
+	// single Properties representing the parent oneof held here.
+	//
+	// The order of Prop matches the order of fields in the Go struct.
+	// Struct fields that are not related to protobufs have a "XXX_" prefix
+	// in the Properties.Name and must be ignored by the user.
+	Prop []*Properties
 
 	// OneofTypes contains information about the oneof fields in this message.
-	// It is keyed by the original name of a field.
+	// It is keyed by the protobuf field name.
 	OneofTypes map[string]*OneofProperties
 }
 
-// OneofProperties represents information about a specific field in a oneof.
-type OneofProperties struct {
-	Type  reflect.Type // pointer to generated struct type for this oneof field
-	Field int          // struct field number of the containing oneof in the message
-	Prop  *Properties
-}
-
-// Implement the sorting interface so we can sort the fields in tag order, as recommended by the spec.
-// See encode.go, (*Buffer).enc_struct.
-
-func (sp *StructProperties) Len() int { return len(sp.order) }
-func (sp *StructProperties) Less(i, j int) bool {
-	return sp.Prop[sp.order[i]].Tag < sp.Prop[sp.order[j]].Tag
-}
-func (sp *StructProperties) Swap(i, j int) { sp.order[i], sp.order[j] = sp.order[j], sp.order[i] }
-
-// Properties represents the protocol-specific behavior of a single struct field.
+// Properties represents the type information for a protobuf message field.
+//
+// Deprecated: Do not use.
 type Properties struct {
-	Name     string // name of the field, for error messages
-	OrigName string // original name before protocol compiler (always set)
-	JSONName string // name to use for JSON; determined by protoc
-	Wire     string
+	// Name is a placeholder name with little meaningful semantic value.
+	// If the name has an "XXX_" prefix, the entire Properties must be ignored.
+	Name string
+	// OrigName is the protobuf field name or oneof name.
+	OrigName string
+	// JSONName is the JSON name for the protobuf field.
+	JSONName string
+	// Enum is a placeholder name for enums.
+	// For historical reasons, this is neither the Go name for the enum,
+	// nor the protobuf name for the enum.
+	Enum string // Deprecated: Do not use.
+	// Weak contains the full name of the weakly referenced message.
+	Weak string
+	// Wire is a string representation of the wire type.
+	Wire string
+	// WireType is the protobuf wire type for the field.
 	WireType int
-	Tag      int
+	// Tag is the protobuf field number.
+	Tag int
+	// Required reports whether this is a required field.
 	Required bool
+	// Optional reports whether this is a optional field.
 	Optional bool
+	// Repeated reports whether this is a repeated field.
 	Repeated bool
-	Packed   bool   // relevant for repeated primitives only
-	Enum     string // set for enum types only
-	proto3   bool   // whether this is known to be a proto3 field
-	oneof    bool   // whether this is a oneof field
+	// Packed reports whether this is a packed repeated field of scalars.
+	Packed bool
+	// Proto3 reports whether this field operates under the proto3 syntax.
+	Proto3 bool
+	// Oneof reports whether this field belongs within a oneof.
+	Oneof bool
 
-	Default    string // default value
-	HasDefault bool   // whether an explicit default was provided
+	// Default is the default value in string form.
+	Default string
+	// HasDefault reports whether the field has a default value.
+	HasDefault bool
 
-	stype reflect.Type      // set for struct types only
-	sprop *StructProperties // set for struct types only
+	// MapKeyProp is the properties for the key field for a map field.
+	MapKeyProp *Properties
+	// MapValProp is the properties for the value field for a map field.
+	MapValProp *Properties
+}
 
-	mtype      reflect.Type // set for map types only
-	MapKeyProp *Properties  // set for map types only
-	MapValProp *Properties  // set for map types only
+// OneofProperties represents the type information for a protobuf oneof.
+//
+// Deprecated: Do not use.
+type OneofProperties struct {
+	// Type is a pointer to the generated wrapper type for the field value.
+	// This is nil for messages that are not in the open-struct API.
+	Type reflect.Type
+	// Field is the index into StructProperties.Prop for the containing oneof.
+	Field int
+	// Prop is the properties for the field.
+	Prop *Properties
 }
 
 // String formats the properties in the protobuf struct field tag style.
 func (p *Properties) String() string {
 	s := p.Wire
-	s += ","
-	s += strconv.Itoa(p.Tag)
+	s += "," + strconv.Itoa(p.Tag)
 	if p.Required {
 		s += ",req"
 	}
@@ -170,18 +112,21 @@
 		s += ",packed"
 	}
 	s += ",name=" + p.OrigName
-	if p.JSONName != p.OrigName {
+	if p.JSONName != "" {
 		s += ",json=" + p.JSONName
 	}
-	if p.proto3 {
-		s += ",proto3"
-	}
-	if p.oneof {
-		s += ",oneof"
-	}
 	if len(p.Enum) > 0 {
 		s += ",enum=" + p.Enum
 	}
+	if len(p.Weak) > 0 {
+		s += ",weak=" + p.Weak
+	}
+	if p.Proto3 {
+		s += ",proto3"
+	}
+	if p.Oneof {
+		s += ",oneof"
+	}
 	if p.HasDefault {
 		s += ",def=" + p.Default
 	}
@@ -189,356 +134,173 @@
 }
 
 // Parse populates p by parsing a string in the protobuf struct field tag style.
-func (p *Properties) Parse(s string) {
-	// "bytes,49,opt,name=foo,def=hello!"
-	fields := strings.Split(s, ",") // breaks def=, but handled below.
-	if len(fields) < 2 {
-		log.Printf("proto: tag has too few fields: %q", s)
-		return
-	}
-
-	p.Wire = fields[0]
-	switch p.Wire {
-	case "varint":
-		p.WireType = WireVarint
-	case "fixed32":
-		p.WireType = WireFixed32
-	case "fixed64":
-		p.WireType = WireFixed64
-	case "zigzag32":
-		p.WireType = WireVarint
-	case "zigzag64":
-		p.WireType = WireVarint
-	case "bytes", "group":
-		p.WireType = WireBytes
-		// no numeric converter for non-numeric types
-	default:
-		log.Printf("proto: tag has unknown wire type: %q", s)
-		return
-	}
-
-	var err error
-	p.Tag, err = strconv.Atoi(fields[1])
-	if err != nil {
-		return
-	}
-
-outer:
-	for i := 2; i < len(fields); i++ {
-		f := fields[i]
-		switch {
-		case f == "req":
-			p.Required = true
-		case f == "opt":
+func (p *Properties) Parse(tag string) {
+	// For example: "bytes,49,opt,name=foo,def=hello!"
+	for len(tag) > 0 {
+		i := strings.IndexByte(tag, ',')
+		if i < 0 {
+			i = len(tag)
+		}
+		switch s := tag[:i]; {
+		case strings.HasPrefix(s, "name="):
+			p.OrigName = s[len("name="):]
+		case strings.HasPrefix(s, "json="):
+			p.JSONName = s[len("json="):]
+		case strings.HasPrefix(s, "enum="):
+			p.Enum = s[len("enum="):]
+		case strings.HasPrefix(s, "weak="):
+			p.Weak = s[len("weak="):]
+		case strings.Trim(s, "0123456789") == "":
+			n, _ := strconv.ParseUint(s, 10, 32)
+			p.Tag = int(n)
+		case s == "opt":
 			p.Optional = true
-		case f == "rep":
+		case s == "req":
+			p.Required = true
+		case s == "rep":
 			p.Repeated = true
-		case f == "packed":
+		case s == "varint" || s == "zigzag32" || s == "zigzag64":
+			p.Wire = s
+			p.WireType = WireVarint
+		case s == "fixed32":
+			p.Wire = s
+			p.WireType = WireFixed32
+		case s == "fixed64":
+			p.Wire = s
+			p.WireType = WireFixed64
+		case s == "bytes":
+			p.Wire = s
+			p.WireType = WireBytes
+		case s == "group":
+			p.Wire = s
+			p.WireType = WireStartGroup
+		case s == "packed":
 			p.Packed = true
-		case strings.HasPrefix(f, "name="):
-			p.OrigName = f[5:]
-		case strings.HasPrefix(f, "json="):
-			p.JSONName = f[5:]
-		case strings.HasPrefix(f, "enum="):
-			p.Enum = f[5:]
-		case f == "proto3":
-			p.proto3 = true
-		case f == "oneof":
-			p.oneof = true
-		case strings.HasPrefix(f, "def="):
+		case s == "proto3":
+			p.Proto3 = true
+		case s == "oneof":
+			p.Oneof = true
+		case strings.HasPrefix(s, "def="):
+			// The default tag is special in that everything afterwards is the
+			// default regardless of the presence of commas.
 			p.HasDefault = true
-			p.Default = f[4:] // rest of string
-			if i+1 < len(fields) {
-				// Commas aren't escaped, and def is always last.
-				p.Default += "," + strings.Join(fields[i+1:], ",")
-				break outer
-			}
+			p.Default, i = tag[len("def="):], len(tag)
 		}
+		tag = strings.TrimPrefix(tag[i:], ",")
 	}
 }
 
-var protoMessageType = reflect.TypeOf((*Message)(nil)).Elem()
-
-// setFieldProps initializes the field properties for submessages and maps.
-func (p *Properties) setFieldProps(typ reflect.Type, f *reflect.StructField, lockGetProp bool) {
-	switch t1 := typ; t1.Kind() {
-	case reflect.Ptr:
-		if t1.Elem().Kind() == reflect.Struct {
-			p.stype = t1.Elem()
-		}
-
-	case reflect.Slice:
-		if t2 := t1.Elem(); t2.Kind() == reflect.Ptr && t2.Elem().Kind() == reflect.Struct {
-			p.stype = t2.Elem()
-		}
-
-	case reflect.Map:
-		p.mtype = t1
-		p.MapKeyProp = &Properties{}
-		p.MapKeyProp.init(reflect.PtrTo(p.mtype.Key()), "Key", f.Tag.Get("protobuf_key"), nil, lockGetProp)
-		p.MapValProp = &Properties{}
-		vtype := p.mtype.Elem()
-		if vtype.Kind() != reflect.Ptr && vtype.Kind() != reflect.Slice {
-			// The value type is not a message (*T) or bytes ([]byte),
-			// so we need encoders for the pointer to this type.
-			vtype = reflect.PtrTo(vtype)
-		}
-		p.MapValProp.init(vtype, "Value", f.Tag.Get("protobuf_val"), nil, lockGetProp)
-	}
-
-	if p.stype != nil {
-		if lockGetProp {
-			p.sprop = GetProperties(p.stype)
-		} else {
-			p.sprop = getPropertiesLocked(p.stype)
-		}
-	}
-}
-
-var (
-	marshalerType = reflect.TypeOf((*Marshaler)(nil)).Elem()
-)
-
 // Init populates the properties from a protocol buffer struct tag.
+//
+// Deprecated: Do not use.
 func (p *Properties) Init(typ reflect.Type, name, tag string, f *reflect.StructField) {
-	p.init(typ, name, tag, f, true)
-}
-
-func (p *Properties) init(typ reflect.Type, name, tag string, f *reflect.StructField, lockGetProp bool) {
-	// "bytes,49,opt,def=hello!"
 	p.Name = name
 	p.OrigName = name
 	if tag == "" {
 		return
 	}
 	p.Parse(tag)
-	p.setFieldProps(typ, f, lockGetProp)
+
+	if typ != nil && typ.Kind() == reflect.Map {
+		p.MapKeyProp = new(Properties)
+		p.MapKeyProp.Init(nil, "Key", f.Tag.Get("protobuf_key"), nil)
+		p.MapValProp = new(Properties)
+		p.MapValProp.Init(nil, "Value", f.Tag.Get("protobuf_val"), nil)
+	}
 }
 
-var (
-	propertiesMu  sync.RWMutex
-	propertiesMap = make(map[reflect.Type]*StructProperties)
-)
+var propertiesCache sync.Map // map[reflect.Type]*StructProperties
 
-// GetProperties returns the list of properties for the type represented by t.
-// t must represent a generated struct type of a protocol message.
+// GetProperties returns the list of properties for the type represented by t,
+// which must be a generated protocol buffer message in the open-struct API,
+// where protobuf message fields are represented by exported Go struct fields.
+//
+// Deprecated: Use protobuf reflection instead.
 func GetProperties(t reflect.Type) *StructProperties {
-	if t.Kind() != reflect.Struct {
-		panic("proto: type must have kind struct")
+	if p, ok := propertiesCache.Load(t); ok {
+		return p.(*StructProperties)
 	}
-
-	// Most calls to GetProperties in a long-running program will be
-	// retrieving details for types we have seen before.
-	propertiesMu.RLock()
-	sprop, ok := propertiesMap[t]
-	propertiesMu.RUnlock()
-	if ok {
-		return sprop
-	}
-
-	propertiesMu.Lock()
-	sprop = getPropertiesLocked(t)
-	propertiesMu.Unlock()
-	return sprop
+	p, _ := propertiesCache.LoadOrStore(t, newProperties(t))
+	return p.(*StructProperties)
 }
 
-type (
-	oneofFuncsIface interface {
-		XXX_OneofFuncs() (func(Message, *Buffer) error, func(Message, int, int, *Buffer) (bool, error), func(Message) int, []interface{})
-	}
-	oneofWrappersIface interface {
-		XXX_OneofWrappers() []interface{}
-	}
-)
-
-// getPropertiesLocked requires that propertiesMu is held.
-func getPropertiesLocked(t reflect.Type) *StructProperties {
-	if prop, ok := propertiesMap[t]; ok {
-		return prop
+func newProperties(t reflect.Type) *StructProperties {
+	if t.Kind() != reflect.Struct {
+		panic(fmt.Sprintf("%v is not a generated message in the open-struct API", t))
 	}
 
+	var hasOneof bool
 	prop := new(StructProperties)
-	// in case of recursive protos, fill this in now.
-	propertiesMap[t] = prop
 
-	// build properties
-	prop.Prop = make([]*Properties, t.NumField())
-	prop.order = make([]int, t.NumField())
-
+	// Construct a list of properties for each field in the struct.
 	for i := 0; i < t.NumField(); i++ {
-		f := t.Field(i)
 		p := new(Properties)
-		name := f.Name
-		p.init(f.Type, name, f.Tag.Get("protobuf"), &f, false)
+		f := t.Field(i)
+		tagField := f.Tag.Get("protobuf")
+		p.Init(f.Type, f.Name, tagField, &f)
 
-		oneof := f.Tag.Get("protobuf_oneof") // special case
-		if oneof != "" {
-			// Oneof fields don't use the traditional protobuf tag.
-			p.OrigName = oneof
+		tagOneof := f.Tag.Get("protobuf_oneof")
+		if tagOneof != "" {
+			hasOneof = true
+			p.OrigName = tagOneof
 		}
-		prop.Prop[i] = p
-		prop.order[i] = i
-		if debug {
-			print(i, " ", f.Name, " ", t.String(), " ")
-			if p.Tag > 0 {
-				print(p.String())
+
+		// Rename unrelated struct fields with the "XXX_" prefix since so much
+		// user code simply checks for this to exclude special fields.
+		if tagField == "" && tagOneof == "" && !strings.HasPrefix(p.Name, "XXX_") {
+			p.Name = "XXX_" + p.Name
+			p.OrigName = "XXX_" + p.OrigName
+		} else if p.Weak != "" {
+			p.Name = p.OrigName // avoid possible "XXX_" prefix on weak field
+		}
+
+		prop.Prop = append(prop.Prop, p)
+	}
+
+	// Construct a mapping of oneof field names to properties.
+	if hasOneof {
+		var oneofWrappers []interface{}
+		if fn, ok := reflect.PtrTo(t).MethodByName("XXX_OneofFuncs"); ok {
+			oneofWrappers = fn.Func.Call([]reflect.Value{reflect.Zero(fn.Type.In(0))})[3].Interface().([]interface{})
+		}
+		if fn, ok := reflect.PtrTo(t).MethodByName("XXX_OneofWrappers"); ok {
+			oneofWrappers = fn.Func.Call([]reflect.Value{reflect.Zero(fn.Type.In(0))})[0].Interface().([]interface{})
+		}
+		if m, ok := reflect.Zero(reflect.PtrTo(t)).Interface().(protoreflect.ProtoMessage); ok {
+			if m, ok := m.ProtoReflect().(interface{ ProtoMessageInfo() *protoimpl.MessageInfo }); ok {
+				oneofWrappers = m.ProtoMessageInfo().OneofWrappers
 			}
-			print("\n")
 		}
-	}
 
-	// Re-order prop.order.
-	sort.Sort(prop)
-
-	var oots []interface{}
-	switch m := reflect.Zero(reflect.PtrTo(t)).Interface().(type) {
-	case oneofFuncsIface:
-		_, _, _, oots = m.XXX_OneofFuncs()
-	case oneofWrappersIface:
-		oots = m.XXX_OneofWrappers()
-	}
-	if len(oots) > 0 {
-		// Interpret oneof metadata.
 		prop.OneofTypes = make(map[string]*OneofProperties)
-		for _, oot := range oots {
-			oop := &OneofProperties{
-				Type: reflect.ValueOf(oot).Type(), // *T
+		for _, wrapper := range oneofWrappers {
+			p := &OneofProperties{
+				Type: reflect.ValueOf(wrapper).Type(), // *T
 				Prop: new(Properties),
 			}
-			sft := oop.Type.Elem().Field(0)
-			oop.Prop.Name = sft.Name
-			oop.Prop.Parse(sft.Tag.Get("protobuf"))
-			// There will be exactly one interface field that
-			// this new value is assignable to.
-			for i := 0; i < t.NumField(); i++ {
-				f := t.Field(i)
-				if f.Type.Kind() != reflect.Interface {
-					continue
-				}
-				if !oop.Type.AssignableTo(f.Type) {
-					continue
-				}
-				oop.Field = i
-				break
-			}
-			prop.OneofTypes[oop.Prop.OrigName] = oop
-		}
-	}
+			f := p.Type.Elem().Field(0)
+			p.Prop.Name = f.Name
+			p.Prop.Parse(f.Tag.Get("protobuf"))
 
-	// build required counts
-	// build tags
-	reqCount := 0
-	prop.decoderOrigNames = make(map[string]int)
-	for i, p := range prop.Prop {
-		if strings.HasPrefix(p.Name, "XXX_") {
-			// Internal fields should not appear in tags/origNames maps.
-			// They are handled specially when encoding and decoding.
-			continue
+			// Determine the struct field that contains this oneof.
+			// Each wrapper is assignable to exactly one parent field.
+			var foundOneof bool
+			for i := 0; i < t.NumField() && !foundOneof; i++ {
+				if p.Type.AssignableTo(t.Field(i).Type) {
+					p.Field = i
+					foundOneof = true
+				}
+			}
+			if !foundOneof {
+				panic(fmt.Sprintf("%v is not a generated message in the open-struct API", t))
+			}
+			prop.OneofTypes[p.Prop.OrigName] = p
 		}
-		if p.Required {
-			reqCount++
-		}
-		prop.decoderTags.put(p.Tag, i)
-		prop.decoderOrigNames[p.OrigName] = i
 	}
-	prop.reqCount = reqCount
 
 	return prop
 }
 
-// A global registry of enum types.
-// The generated code will register the generated maps by calling RegisterEnum.
-
-var enumValueMaps = make(map[string]map[string]int32)
-
-// RegisterEnum is called from the generated code to install the enum descriptor
-// maps into the global table to aid parsing text format protocol buffers.
-func RegisterEnum(typeName string, unusedNameMap map[int32]string, valueMap map[string]int32) {
-	if _, ok := enumValueMaps[typeName]; ok {
-		panic("proto: duplicate enum registered: " + typeName)
-	}
-	enumValueMaps[typeName] = valueMap
-}
-
-// EnumValueMap returns the mapping from names to integers of the
-// enum type enumType, or a nil if not found.
-func EnumValueMap(enumType string) map[string]int32 {
-	return enumValueMaps[enumType]
-}
-
-// A registry of all linked message types.
-// The string is a fully-qualified proto name ("pkg.Message").
-var (
-	protoTypedNils = make(map[string]Message)      // a map from proto names to typed nil pointers
-	protoMapTypes  = make(map[string]reflect.Type) // a map from proto names to map types
-	revProtoTypes  = make(map[reflect.Type]string)
-)
-
-// RegisterType is called from generated code and maps from the fully qualified
-// proto name to the type (pointer to struct) of the protocol buffer.
-func RegisterType(x Message, name string) {
-	if _, ok := protoTypedNils[name]; ok {
-		// TODO: Some day, make this a panic.
-		log.Printf("proto: duplicate proto type registered: %s", name)
-		return
-	}
-	t := reflect.TypeOf(x)
-	if v := reflect.ValueOf(x); v.Kind() == reflect.Ptr && v.Pointer() == 0 {
-		// Generated code always calls RegisterType with nil x.
-		// This check is just for extra safety.
-		protoTypedNils[name] = x
-	} else {
-		protoTypedNils[name] = reflect.Zero(t).Interface().(Message)
-	}
-	revProtoTypes[t] = name
-}
-
-// RegisterMapType is called from generated code and maps from the fully qualified
-// proto name to the native map type of the proto map definition.
-func RegisterMapType(x interface{}, name string) {
-	if reflect.TypeOf(x).Kind() != reflect.Map {
-		panic(fmt.Sprintf("RegisterMapType(%T, %q); want map", x, name))
-	}
-	if _, ok := protoMapTypes[name]; ok {
-		log.Printf("proto: duplicate proto type registered: %s", name)
-		return
-	}
-	t := reflect.TypeOf(x)
-	protoMapTypes[name] = t
-	revProtoTypes[t] = name
-}
-
-// MessageName returns the fully-qualified proto name for the given message type.
-func MessageName(x Message) string {
-	type xname interface {
-		XXX_MessageName() string
-	}
-	if m, ok := x.(xname); ok {
-		return m.XXX_MessageName()
-	}
-	return revProtoTypes[reflect.TypeOf(x)]
-}
-
-// MessageType returns the message type (pointer to struct) for a named message.
-// The type is not guaranteed to implement proto.Message if the name refers to a
-// map entry.
-func MessageType(name string) reflect.Type {
-	if t, ok := protoTypedNils[name]; ok {
-		return reflect.TypeOf(t)
-	}
-	return protoMapTypes[name]
-}
-
-// A registry of all linked proto files.
-var (
-	protoFiles = make(map[string][]byte) // file name => fileDescriptor
-)
-
-// RegisterFile is called from generated code and maps from the
-// full file name of a .proto file to its compressed FileDescriptorProto.
-func RegisterFile(filename string, fileDescriptor []byte) {
-	protoFiles[filename] = fileDescriptor
-}
-
-// FileDescriptor returns the compressed FileDescriptorProto for a .proto file.
-func FileDescriptor(filename string) []byte { return protoFiles[filename] }
+func (sp *StructProperties) Len() int           { return len(sp.Prop) }
+func (sp *StructProperties) Less(i, j int) bool { return false }
+func (sp *StructProperties) Swap(i, j int)      { return }
diff --git a/vendor/github.com/golang/protobuf/proto/proto.go b/vendor/github.com/golang/protobuf/proto/proto.go
new file mode 100644
index 0000000..5aee89c
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/proto.go
@@ -0,0 +1,167 @@
+// 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.
+
+// Package proto provides functionality for handling protocol buffer messages.
+// In particular, it provides marshaling and unmarshaling between a protobuf
+// message and the binary wire format.
+//
+// See https://developers.google.com/protocol-buffers/docs/gotutorial for
+// more information.
+//
+// Deprecated: Use the "google.golang.org/protobuf/proto" package instead.
+package proto
+
+import (
+	protoV2 "google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+	"google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	ProtoPackageIsVersion1 = true
+	ProtoPackageIsVersion2 = true
+	ProtoPackageIsVersion3 = true
+	ProtoPackageIsVersion4 = true
+)
+
+// GeneratedEnum is any enum type generated by protoc-gen-go
+// which is a named int32 kind.
+// This type exists for documentation purposes.
+type GeneratedEnum interface{}
+
+// GeneratedMessage is any message type generated by protoc-gen-go
+// which is a pointer to a named struct kind.
+// This type exists for documentation purposes.
+type GeneratedMessage interface{}
+
+// Message is a protocol buffer message.
+//
+// This is the v1 version of the message interface and is marginally better
+// than an empty interface as it lacks any method to programatically interact
+// with the contents of the message.
+//
+// A v2 message is declared in "google.golang.org/protobuf/proto".Message and
+// exposes protobuf reflection as a first-class feature of the interface.
+//
+// To convert a v1 message to a v2 message, use the MessageV2 function.
+// To convert a v2 message to a v1 message, use the MessageV1 function.
+type Message = protoiface.MessageV1
+
+// MessageV1 converts either a v1 or v2 message to a v1 message.
+// It returns nil if m is nil.
+func MessageV1(m GeneratedMessage) protoiface.MessageV1 {
+	return protoimpl.X.ProtoMessageV1Of(m)
+}
+
+// MessageV2 converts either a v1 or v2 message to a v2 message.
+// It returns nil if m is nil.
+func MessageV2(m GeneratedMessage) protoV2.Message {
+	return protoimpl.X.ProtoMessageV2Of(m)
+}
+
+// MessageReflect returns a reflective view for a message.
+// It returns nil if m is nil.
+func MessageReflect(m Message) protoreflect.Message {
+	return protoimpl.X.MessageOf(m)
+}
+
+// Marshaler is implemented by messages that can marshal themselves.
+// This interface is used by the following functions: Size, Marshal,
+// Buffer.Marshal, and Buffer.EncodeMessage.
+//
+// Deprecated: Do not implement.
+type Marshaler interface {
+	// Marshal formats the encoded bytes of the message.
+	// It should be deterministic and emit valid protobuf wire data.
+	// The caller takes ownership of the returned buffer.
+	Marshal() ([]byte, error)
+}
+
+// Unmarshaler is implemented by messages that can unmarshal themselves.
+// This interface is used by the following functions: Unmarshal, UnmarshalMerge,
+// Buffer.Unmarshal, Buffer.DecodeMessage, and Buffer.DecodeGroup.
+//
+// Deprecated: Do not implement.
+type Unmarshaler interface {
+	// Unmarshal parses the encoded bytes of the protobuf wire input.
+	// The provided buffer is only valid for during method call.
+	// It should not reset the receiver message.
+	Unmarshal([]byte) error
+}
+
+// Merger is implemented by messages that can merge themselves.
+// This interface is used by the following functions: Clone and Merge.
+//
+// Deprecated: Do not implement.
+type Merger interface {
+	// Merge merges the contents of src into the receiver message.
+	// It clones all data structures in src such that it aliases no mutable
+	// memory referenced by src.
+	Merge(src Message)
+}
+
+// RequiredNotSetError is an error type returned when
+// marshaling or unmarshaling a message with missing required fields.
+type RequiredNotSetError struct {
+	err error
+}
+
+func (e *RequiredNotSetError) Error() string {
+	if e.err != nil {
+		return e.err.Error()
+	}
+	return "proto: required field not set"
+}
+func (e *RequiredNotSetError) RequiredNotSet() bool {
+	return true
+}
+
+func checkRequiredNotSet(m protoV2.Message) error {
+	if err := protoV2.CheckInitialized(m); err != nil {
+		return &RequiredNotSetError{err: err}
+	}
+	return nil
+}
+
+// Clone returns a deep copy of src.
+func Clone(src Message) Message {
+	return MessageV1(protoV2.Clone(MessageV2(src)))
+}
+
+// Merge merges src into dst, which must be messages of the same type.
+//
+// Populated scalar fields in src are copied to dst, while populated
+// singular messages in src are merged into dst by recursively calling Merge.
+// The elements of every list field in src is appended to the corresponded
+// list fields in dst. The entries of every map field in src is copied into
+// the corresponding map field in dst, possibly replacing existing entries.
+// The unknown fields of src are appended to the unknown fields of dst.
+func Merge(dst, src Message) {
+	protoV2.Merge(MessageV2(dst), MessageV2(src))
+}
+
+// Equal reports whether two messages are equal.
+// If two messages marshal to the same bytes under deterministic serialization,
+// then Equal is guaranteed to report true.
+//
+// Two messages are equal if they are the same protobuf message type,
+// have the same set of populated known and extension field values,
+// and the same set of unknown fields values.
+//
+// Scalar values are compared with the equivalent of the == operator in Go,
+// except bytes values which are compared using bytes.Equal and
+// floating point values which specially treat NaNs as equal.
+// Message values are compared by recursively calling Equal.
+// Lists are equal if each element value is also equal.
+// Maps are equal if they have the same set of keys, where the pair of values
+// for each key is also equal.
+func Equal(x, y Message) bool {
+	return protoV2.Equal(MessageV2(x), MessageV2(y))
+}
+
+func isMessageSet(md protoreflect.MessageDescriptor) bool {
+	ms, ok := md.(interface{ IsMessageSet() bool })
+	return ok && ms.IsMessageSet()
+}
diff --git a/vendor/github.com/golang/protobuf/proto/registry.go b/vendor/github.com/golang/protobuf/proto/registry.go
new file mode 100644
index 0000000..066b432
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/registry.go
@@ -0,0 +1,317 @@
+// 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.
+
+package proto
+
+import (
+	"bytes"
+	"compress/gzip"
+	"fmt"
+	"io/ioutil"
+	"reflect"
+	"strings"
+	"sync"
+
+	"google.golang.org/protobuf/reflect/protodesc"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+	"google.golang.org/protobuf/runtime/protoimpl"
+)
+
+// filePath is the path to the proto source file.
+type filePath = string // e.g., "google/protobuf/descriptor.proto"
+
+// fileDescGZIP is the compressed contents of the encoded FileDescriptorProto.
+type fileDescGZIP = []byte
+
+var fileCache sync.Map // map[filePath]fileDescGZIP
+
+// RegisterFile is called from generated code to register the compressed
+// FileDescriptorProto with the file path for a proto source file.
+//
+// Deprecated: Use protoregistry.GlobalFiles.RegisterFile instead.
+func RegisterFile(s filePath, d fileDescGZIP) {
+	// Decompress the descriptor.
+	zr, err := gzip.NewReader(bytes.NewReader(d))
+	if err != nil {
+		panic(fmt.Sprintf("proto: invalid compressed file descriptor: %v", err))
+	}
+	b, err := ioutil.ReadAll(zr)
+	if err != nil {
+		panic(fmt.Sprintf("proto: invalid compressed file descriptor: %v", err))
+	}
+
+	// Construct a protoreflect.FileDescriptor from the raw descriptor.
+	// Note that DescBuilder.Build automatically registers the constructed
+	// file descriptor with the v2 registry.
+	protoimpl.DescBuilder{RawDescriptor: b}.Build()
+
+	// Locally cache the raw descriptor form for the file.
+	fileCache.Store(s, d)
+}
+
+// FileDescriptor returns the compressed FileDescriptorProto given the file path
+// for a proto source file. It returns nil if not found.
+//
+// Deprecated: Use protoregistry.GlobalFiles.FindFileByPath instead.
+func FileDescriptor(s filePath) fileDescGZIP {
+	if v, ok := fileCache.Load(s); ok {
+		return v.(fileDescGZIP)
+	}
+
+	// Find the descriptor in the v2 registry.
+	var b []byte
+	if fd, _ := protoregistry.GlobalFiles.FindFileByPath(s); fd != nil {
+		b, _ = Marshal(protodesc.ToFileDescriptorProto(fd))
+	}
+
+	// Locally cache the raw descriptor form for the file.
+	if len(b) > 0 {
+		v, _ := fileCache.LoadOrStore(s, protoimpl.X.CompressGZIP(b))
+		return v.(fileDescGZIP)
+	}
+	return nil
+}
+
+// enumName is the name of an enum. For historical reasons, the enum name is
+// neither the full Go name nor the full protobuf name of the enum.
+// The name is the dot-separated combination of just the proto package that the
+// enum is declared within followed by the Go type name of the generated enum.
+type enumName = string // e.g., "my.proto.package.GoMessage_GoEnum"
+
+// enumsByName maps enum values by name to their numeric counterpart.
+type enumsByName = map[string]int32
+
+// enumsByNumber maps enum values by number to their name counterpart.
+type enumsByNumber = map[int32]string
+
+var enumCache sync.Map     // map[enumName]enumsByName
+var numFilesCache sync.Map // map[protoreflect.FullName]int
+
+// RegisterEnum is called from the generated code to register the mapping of
+// enum value names to enum numbers for the enum identified by s.
+//
+// Deprecated: Use protoregistry.GlobalTypes.RegisterEnum instead.
+func RegisterEnum(s enumName, _ enumsByNumber, m enumsByName) {
+	if _, ok := enumCache.Load(s); ok {
+		panic("proto: duplicate enum registered: " + s)
+	}
+	enumCache.Store(s, m)
+
+	// This does not forward registration to the v2 registry since this API
+	// lacks sufficient information to construct a complete v2 enum descriptor.
+}
+
+// EnumValueMap returns the mapping from enum value names to enum numbers for
+// the enum of the given name. It returns nil if not found.
+//
+// Deprecated: Use protoregistry.GlobalTypes.FindEnumByName instead.
+func EnumValueMap(s enumName) enumsByName {
+	if v, ok := enumCache.Load(s); ok {
+		return v.(enumsByName)
+	}
+
+	// Check whether the cache is stale. If the number of files in the current
+	// package differs, then it means that some enums may have been recently
+	// registered upstream that we do not know about.
+	var protoPkg protoreflect.FullName
+	if i := strings.LastIndexByte(s, '.'); i >= 0 {
+		protoPkg = protoreflect.FullName(s[:i])
+	}
+	v, _ := numFilesCache.Load(protoPkg)
+	numFiles, _ := v.(int)
+	if protoregistry.GlobalFiles.NumFilesByPackage(protoPkg) == numFiles {
+		return nil // cache is up-to-date; was not found earlier
+	}
+
+	// Update the enum cache for all enums declared in the given proto package.
+	numFiles = 0
+	protoregistry.GlobalFiles.RangeFilesByPackage(protoPkg, func(fd protoreflect.FileDescriptor) bool {
+		walkEnums(fd, func(ed protoreflect.EnumDescriptor) {
+			name := protoimpl.X.LegacyEnumName(ed)
+			if _, ok := enumCache.Load(name); !ok {
+				m := make(enumsByName)
+				evs := ed.Values()
+				for i := evs.Len() - 1; i >= 0; i-- {
+					ev := evs.Get(i)
+					m[string(ev.Name())] = int32(ev.Number())
+				}
+				enumCache.LoadOrStore(name, m)
+			}
+		})
+		numFiles++
+		return true
+	})
+	numFilesCache.Store(protoPkg, numFiles)
+
+	// Check cache again for enum map.
+	if v, ok := enumCache.Load(s); ok {
+		return v.(enumsByName)
+	}
+	return nil
+}
+
+// walkEnums recursively walks all enums declared in d.
+func walkEnums(d interface {
+	Enums() protoreflect.EnumDescriptors
+	Messages() protoreflect.MessageDescriptors
+}, f func(protoreflect.EnumDescriptor)) {
+	eds := d.Enums()
+	for i := eds.Len() - 1; i >= 0; i-- {
+		f(eds.Get(i))
+	}
+	mds := d.Messages()
+	for i := mds.Len() - 1; i >= 0; i-- {
+		walkEnums(mds.Get(i), f)
+	}
+}
+
+// messageName is the full name of protobuf message.
+type messageName = string
+
+var messageTypeCache sync.Map // map[messageName]reflect.Type
+
+// RegisterType is called from generated code to register the message Go type
+// for a message of the given name.
+//
+// Deprecated: Use protoregistry.GlobalTypes.RegisterMessage instead.
+func RegisterType(m Message, s messageName) {
+	mt := protoimpl.X.LegacyMessageTypeOf(m, protoreflect.FullName(s))
+	if err := protoregistry.GlobalTypes.RegisterMessage(mt); err != nil {
+		panic(err)
+	}
+	messageTypeCache.Store(s, reflect.TypeOf(m))
+}
+
+// RegisterMapType is called from generated code to register the Go map type
+// for a protobuf message representing a map entry.
+//
+// Deprecated: Do not use.
+func RegisterMapType(m interface{}, s messageName) {
+	t := reflect.TypeOf(m)
+	if t.Kind() != reflect.Map {
+		panic(fmt.Sprintf("invalid map kind: %v", t))
+	}
+	if _, ok := messageTypeCache.Load(s); ok {
+		panic(fmt.Errorf("proto: duplicate proto message registered: %s", s))
+	}
+	messageTypeCache.Store(s, t)
+}
+
+// MessageType returns the message type for a named message.
+// It returns nil if not found.
+//
+// Deprecated: Use protoregistry.GlobalTypes.FindMessageByName instead.
+func MessageType(s messageName) reflect.Type {
+	if v, ok := messageTypeCache.Load(s); ok {
+		return v.(reflect.Type)
+	}
+
+	// Derive the message type from the v2 registry.
+	var t reflect.Type
+	if mt, _ := protoregistry.GlobalTypes.FindMessageByName(protoreflect.FullName(s)); mt != nil {
+		t = messageGoType(mt)
+	}
+
+	// If we could not get a concrete type, it is possible that it is a
+	// pseudo-message for a map entry.
+	if t == nil {
+		d, _ := protoregistry.GlobalFiles.FindDescriptorByName(protoreflect.FullName(s))
+		if md, _ := d.(protoreflect.MessageDescriptor); md != nil && md.IsMapEntry() {
+			kt := goTypeForField(md.Fields().ByNumber(1))
+			vt := goTypeForField(md.Fields().ByNumber(2))
+			t = reflect.MapOf(kt, vt)
+		}
+	}
+
+	// Locally cache the message type for the given name.
+	if t != nil {
+		v, _ := messageTypeCache.LoadOrStore(s, t)
+		return v.(reflect.Type)
+	}
+	return nil
+}
+
+func goTypeForField(fd protoreflect.FieldDescriptor) reflect.Type {
+	switch k := fd.Kind(); k {
+	case protoreflect.EnumKind:
+		if et, _ := protoregistry.GlobalTypes.FindEnumByName(fd.Enum().FullName()); et != nil {
+			return enumGoType(et)
+		}
+		return reflect.TypeOf(protoreflect.EnumNumber(0))
+	case protoreflect.MessageKind, protoreflect.GroupKind:
+		if mt, _ := protoregistry.GlobalTypes.FindMessageByName(fd.Message().FullName()); mt != nil {
+			return messageGoType(mt)
+		}
+		return reflect.TypeOf((*protoreflect.Message)(nil)).Elem()
+	default:
+		return reflect.TypeOf(fd.Default().Interface())
+	}
+}
+
+func enumGoType(et protoreflect.EnumType) reflect.Type {
+	return reflect.TypeOf(et.New(0))
+}
+
+func messageGoType(mt protoreflect.MessageType) reflect.Type {
+	return reflect.TypeOf(MessageV1(mt.Zero().Interface()))
+}
+
+// MessageName returns the full protobuf name for the given message type.
+//
+// Deprecated: Use protoreflect.MessageDescriptor.FullName instead.
+func MessageName(m Message) messageName {
+	if m == nil {
+		return ""
+	}
+	if m, ok := m.(interface{ XXX_MessageName() messageName }); ok {
+		return m.XXX_MessageName()
+	}
+	return messageName(protoimpl.X.MessageDescriptorOf(m).FullName())
+}
+
+// RegisterExtension is called from the generated code to register
+// the extension descriptor.
+//
+// Deprecated: Use protoregistry.GlobalTypes.RegisterExtension instead.
+func RegisterExtension(d *ExtensionDesc) {
+	if err := protoregistry.GlobalTypes.RegisterExtension(d); err != nil {
+		panic(err)
+	}
+}
+
+type extensionsByNumber = map[int32]*ExtensionDesc
+
+var extensionCache sync.Map // map[messageName]extensionsByNumber
+
+// RegisteredExtensions returns a map of the registered extensions for the
+// provided protobuf message, indexed by the extension field number.
+//
+// Deprecated: Use protoregistry.GlobalTypes.RangeExtensionsByMessage instead.
+func RegisteredExtensions(m Message) extensionsByNumber {
+	// Check whether the cache is stale. If the number of extensions for
+	// the given message differs, then it means that some extensions were
+	// recently registered upstream that we do not know about.
+	s := MessageName(m)
+	v, _ := extensionCache.Load(s)
+	xs, _ := v.(extensionsByNumber)
+	if protoregistry.GlobalTypes.NumExtensionsByMessage(protoreflect.FullName(s)) == len(xs) {
+		return xs // cache is up-to-date
+	}
+
+	// Cache is stale, re-compute the extensions map.
+	xs = make(extensionsByNumber)
+	protoregistry.GlobalTypes.RangeExtensionsByMessage(protoreflect.FullName(s), func(xt protoreflect.ExtensionType) bool {
+		if xd, ok := xt.(*ExtensionDesc); ok {
+			xs[int32(xt.TypeDescriptor().Number())] = xd
+		} else {
+			// TODO: This implies that the protoreflect.ExtensionType is a
+			// custom type not generated by protoc-gen-go. We could try and
+			// convert the type to an ExtensionDesc.
+		}
+		return true
+	})
+	extensionCache.Store(s, xs)
+	return xs
+}
diff --git a/vendor/github.com/golang/protobuf/proto/table_marshal.go b/vendor/github.com/golang/protobuf/proto/table_marshal.go
deleted file mode 100644
index 5cb11fa..0000000
--- a/vendor/github.com/golang/protobuf/proto/table_marshal.go
+++ /dev/null
@@ -1,2776 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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 proto
-
-import (
-	"errors"
-	"fmt"
-	"math"
-	"reflect"
-	"sort"
-	"strconv"
-	"strings"
-	"sync"
-	"sync/atomic"
-	"unicode/utf8"
-)
-
-// a sizer takes a pointer to a field and the size of its tag, computes the size of
-// the encoded data.
-type sizer func(pointer, int) int
-
-// a marshaler takes a byte slice, a pointer to a field, and its tag (in wire format),
-// marshals the field to the end of the slice, returns the slice and error (if any).
-type marshaler func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error)
-
-// marshalInfo is the information used for marshaling a message.
-type marshalInfo struct {
-	typ          reflect.Type
-	fields       []*marshalFieldInfo
-	unrecognized field                      // offset of XXX_unrecognized
-	extensions   field                      // offset of XXX_InternalExtensions
-	v1extensions field                      // offset of XXX_extensions
-	sizecache    field                      // offset of XXX_sizecache
-	initialized  int32                      // 0 -- only typ is set, 1 -- fully initialized
-	messageset   bool                       // uses message set wire format
-	hasmarshaler bool                       // has custom marshaler
-	sync.RWMutex                            // protect extElems map, also for initialization
-	extElems     map[int32]*marshalElemInfo // info of extension elements
-}
-
-// marshalFieldInfo is the information used for marshaling a field of a message.
-type marshalFieldInfo struct {
-	field      field
-	wiretag    uint64 // tag in wire format
-	tagsize    int    // size of tag in wire format
-	sizer      sizer
-	marshaler  marshaler
-	isPointer  bool
-	required   bool                              // field is required
-	name       string                            // name of the field, for error reporting
-	oneofElems map[reflect.Type]*marshalElemInfo // info of oneof elements
-}
-
-// marshalElemInfo is the information used for marshaling an extension or oneof element.
-type marshalElemInfo struct {
-	wiretag   uint64 // tag in wire format
-	tagsize   int    // size of tag in wire format
-	sizer     sizer
-	marshaler marshaler
-	isptr     bool // elem is pointer typed, thus interface of this type is a direct interface (extension only)
-	deref     bool // dereference the pointer before operating on it; implies isptr
-}
-
-var (
-	marshalInfoMap  = map[reflect.Type]*marshalInfo{}
-	marshalInfoLock sync.Mutex
-)
-
-// getMarshalInfo returns the information to marshal a given type of message.
-// The info it returns may not necessarily initialized.
-// t is the type of the message (NOT the pointer to it).
-func getMarshalInfo(t reflect.Type) *marshalInfo {
-	marshalInfoLock.Lock()
-	u, ok := marshalInfoMap[t]
-	if !ok {
-		u = &marshalInfo{typ: t}
-		marshalInfoMap[t] = u
-	}
-	marshalInfoLock.Unlock()
-	return u
-}
-
-// Size is the entry point from generated code,
-// and should be ONLY called by generated code.
-// It computes the size of encoded data of msg.
-// a is a pointer to a place to store cached marshal info.
-func (a *InternalMessageInfo) Size(msg Message) int {
-	u := getMessageMarshalInfo(msg, a)
-	ptr := toPointer(&msg)
-	if ptr.isNil() {
-		// We get here if msg is a typed nil ((*SomeMessage)(nil)),
-		// so it satisfies the interface, and msg == nil wouldn't
-		// catch it. We don't want crash in this case.
-		return 0
-	}
-	return u.size(ptr)
-}
-
-// Marshal is the entry point from generated code,
-// and should be ONLY called by generated code.
-// It marshals msg to the end of b.
-// a is a pointer to a place to store cached marshal info.
-func (a *InternalMessageInfo) Marshal(b []byte, msg Message, deterministic bool) ([]byte, error) {
-	u := getMessageMarshalInfo(msg, a)
-	ptr := toPointer(&msg)
-	if ptr.isNil() {
-		// We get here if msg is a typed nil ((*SomeMessage)(nil)),
-		// so it satisfies the interface, and msg == nil wouldn't
-		// catch it. We don't want crash in this case.
-		return b, ErrNil
-	}
-	return u.marshal(b, ptr, deterministic)
-}
-
-func getMessageMarshalInfo(msg interface{}, a *InternalMessageInfo) *marshalInfo {
-	// u := a.marshal, but atomically.
-	// We use an atomic here to ensure memory consistency.
-	u := atomicLoadMarshalInfo(&a.marshal)
-	if u == nil {
-		// Get marshal information from type of message.
-		t := reflect.ValueOf(msg).Type()
-		if t.Kind() != reflect.Ptr {
-			panic(fmt.Sprintf("cannot handle non-pointer message type %v", t))
-		}
-		u = getMarshalInfo(t.Elem())
-		// Store it in the cache for later users.
-		// a.marshal = u, but atomically.
-		atomicStoreMarshalInfo(&a.marshal, u)
-	}
-	return u
-}
-
-// size is the main function to compute the size of the encoded data of a message.
-// ptr is the pointer to the message.
-func (u *marshalInfo) size(ptr pointer) int {
-	if atomic.LoadInt32(&u.initialized) == 0 {
-		u.computeMarshalInfo()
-	}
-
-	// If the message can marshal itself, let it do it, for compatibility.
-	// NOTE: This is not efficient.
-	if u.hasmarshaler {
-		m := ptr.asPointerTo(u.typ).Interface().(Marshaler)
-		b, _ := m.Marshal()
-		return len(b)
-	}
-
-	n := 0
-	for _, f := range u.fields {
-		if f.isPointer && ptr.offset(f.field).getPointer().isNil() {
-			// nil pointer always marshals to nothing
-			continue
-		}
-		n += f.sizer(ptr.offset(f.field), f.tagsize)
-	}
-	if u.extensions.IsValid() {
-		e := ptr.offset(u.extensions).toExtensions()
-		if u.messageset {
-			n += u.sizeMessageSet(e)
-		} else {
-			n += u.sizeExtensions(e)
-		}
-	}
-	if u.v1extensions.IsValid() {
-		m := *ptr.offset(u.v1extensions).toOldExtensions()
-		n += u.sizeV1Extensions(m)
-	}
-	if u.unrecognized.IsValid() {
-		s := *ptr.offset(u.unrecognized).toBytes()
-		n += len(s)
-	}
-	// cache the result for use in marshal
-	if u.sizecache.IsValid() {
-		atomic.StoreInt32(ptr.offset(u.sizecache).toInt32(), int32(n))
-	}
-	return n
-}
-
-// cachedsize gets the size from cache. If there is no cache (i.e. message is not generated),
-// fall back to compute the size.
-func (u *marshalInfo) cachedsize(ptr pointer) int {
-	if u.sizecache.IsValid() {
-		return int(atomic.LoadInt32(ptr.offset(u.sizecache).toInt32()))
-	}
-	return u.size(ptr)
-}
-
-// marshal is the main function to marshal a message. It takes a byte slice and appends
-// the encoded data to the end of the slice, returns the slice and error (if any).
-// ptr is the pointer to the message.
-// If deterministic is true, map is marshaled in deterministic order.
-func (u *marshalInfo) marshal(b []byte, ptr pointer, deterministic bool) ([]byte, error) {
-	if atomic.LoadInt32(&u.initialized) == 0 {
-		u.computeMarshalInfo()
-	}
-
-	// If the message can marshal itself, let it do it, for compatibility.
-	// NOTE: This is not efficient.
-	if u.hasmarshaler {
-		m := ptr.asPointerTo(u.typ).Interface().(Marshaler)
-		b1, err := m.Marshal()
-		b = append(b, b1...)
-		return b, err
-	}
-
-	var err, errLater error
-	// The old marshaler encodes extensions at beginning.
-	if u.extensions.IsValid() {
-		e := ptr.offset(u.extensions).toExtensions()
-		if u.messageset {
-			b, err = u.appendMessageSet(b, e, deterministic)
-		} else {
-			b, err = u.appendExtensions(b, e, deterministic)
-		}
-		if err != nil {
-			return b, err
-		}
-	}
-	if u.v1extensions.IsValid() {
-		m := *ptr.offset(u.v1extensions).toOldExtensions()
-		b, err = u.appendV1Extensions(b, m, deterministic)
-		if err != nil {
-			return b, err
-		}
-	}
-	for _, f := range u.fields {
-		if f.required {
-			if ptr.offset(f.field).getPointer().isNil() {
-				// Required field is not set.
-				// We record the error but keep going, to give a complete marshaling.
-				if errLater == nil {
-					errLater = &RequiredNotSetError{f.name}
-				}
-				continue
-			}
-		}
-		if f.isPointer && ptr.offset(f.field).getPointer().isNil() {
-			// nil pointer always marshals to nothing
-			continue
-		}
-		b, err = f.marshaler(b, ptr.offset(f.field), f.wiretag, deterministic)
-		if err != nil {
-			if err1, ok := err.(*RequiredNotSetError); ok {
-				// Required field in submessage is not set.
-				// We record the error but keep going, to give a complete marshaling.
-				if errLater == nil {
-					errLater = &RequiredNotSetError{f.name + "." + err1.field}
-				}
-				continue
-			}
-			if err == errRepeatedHasNil {
-				err = errors.New("proto: repeated field " + f.name + " has nil element")
-			}
-			if err == errInvalidUTF8 {
-				if errLater == nil {
-					fullName := revProtoTypes[reflect.PtrTo(u.typ)] + "." + f.name
-					errLater = &invalidUTF8Error{fullName}
-				}
-				continue
-			}
-			return b, err
-		}
-	}
-	if u.unrecognized.IsValid() {
-		s := *ptr.offset(u.unrecognized).toBytes()
-		b = append(b, s...)
-	}
-	return b, errLater
-}
-
-// computeMarshalInfo initializes the marshal info.
-func (u *marshalInfo) computeMarshalInfo() {
-	u.Lock()
-	defer u.Unlock()
-	if u.initialized != 0 { // non-atomic read is ok as it is protected by the lock
-		return
-	}
-
-	t := u.typ
-	u.unrecognized = invalidField
-	u.extensions = invalidField
-	u.v1extensions = invalidField
-	u.sizecache = invalidField
-
-	// If the message can marshal itself, let it do it, for compatibility.
-	// NOTE: This is not efficient.
-	if reflect.PtrTo(t).Implements(marshalerType) {
-		u.hasmarshaler = true
-		atomic.StoreInt32(&u.initialized, 1)
-		return
-	}
-
-	// get oneof implementers
-	var oneofImplementers []interface{}
-	switch m := reflect.Zero(reflect.PtrTo(t)).Interface().(type) {
-	case oneofFuncsIface:
-		_, _, _, oneofImplementers = m.XXX_OneofFuncs()
-	case oneofWrappersIface:
-		oneofImplementers = m.XXX_OneofWrappers()
-	}
-
-	n := t.NumField()
-
-	// deal with XXX fields first
-	for i := 0; i < t.NumField(); i++ {
-		f := t.Field(i)
-		if !strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-		switch f.Name {
-		case "XXX_sizecache":
-			u.sizecache = toField(&f)
-		case "XXX_unrecognized":
-			u.unrecognized = toField(&f)
-		case "XXX_InternalExtensions":
-			u.extensions = toField(&f)
-			u.messageset = f.Tag.Get("protobuf_messageset") == "1"
-		case "XXX_extensions":
-			u.v1extensions = toField(&f)
-		case "XXX_NoUnkeyedLiteral":
-			// nothing to do
-		default:
-			panic("unknown XXX field: " + f.Name)
-		}
-		n--
-	}
-
-	// normal fields
-	fields := make([]marshalFieldInfo, n) // batch allocation
-	u.fields = make([]*marshalFieldInfo, 0, n)
-	for i, j := 0, 0; i < t.NumField(); i++ {
-		f := t.Field(i)
-
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-		field := &fields[j]
-		j++
-		field.name = f.Name
-		u.fields = append(u.fields, field)
-		if f.Tag.Get("protobuf_oneof") != "" {
-			field.computeOneofFieldInfo(&f, oneofImplementers)
-			continue
-		}
-		if f.Tag.Get("protobuf") == "" {
-			// field has no tag (not in generated message), ignore it
-			u.fields = u.fields[:len(u.fields)-1]
-			j--
-			continue
-		}
-		field.computeMarshalFieldInfo(&f)
-	}
-
-	// fields are marshaled in tag order on the wire.
-	sort.Sort(byTag(u.fields))
-
-	atomic.StoreInt32(&u.initialized, 1)
-}
-
-// helper for sorting fields by tag
-type byTag []*marshalFieldInfo
-
-func (a byTag) Len() int           { return len(a) }
-func (a byTag) Swap(i, j int)      { a[i], a[j] = a[j], a[i] }
-func (a byTag) Less(i, j int) bool { return a[i].wiretag < a[j].wiretag }
-
-// getExtElemInfo returns the information to marshal an extension element.
-// The info it returns is initialized.
-func (u *marshalInfo) getExtElemInfo(desc *ExtensionDesc) *marshalElemInfo {
-	// get from cache first
-	u.RLock()
-	e, ok := u.extElems[desc.Field]
-	u.RUnlock()
-	if ok {
-		return e
-	}
-
-	t := reflect.TypeOf(desc.ExtensionType) // pointer or slice to basic type or struct
-	tags := strings.Split(desc.Tag, ",")
-	tag, err := strconv.Atoi(tags[1])
-	if err != nil {
-		panic("tag is not an integer")
-	}
-	wt := wiretype(tags[0])
-	if t.Kind() == reflect.Ptr && t.Elem().Kind() != reflect.Struct {
-		t = t.Elem()
-	}
-	sizer, marshaler := typeMarshaler(t, tags, false, false)
-	var deref bool
-	if t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 {
-		t = reflect.PtrTo(t)
-		deref = true
-	}
-	e = &marshalElemInfo{
-		wiretag:   uint64(tag)<<3 | wt,
-		tagsize:   SizeVarint(uint64(tag) << 3),
-		sizer:     sizer,
-		marshaler: marshaler,
-		isptr:     t.Kind() == reflect.Ptr,
-		deref:     deref,
-	}
-
-	// update cache
-	u.Lock()
-	if u.extElems == nil {
-		u.extElems = make(map[int32]*marshalElemInfo)
-	}
-	u.extElems[desc.Field] = e
-	u.Unlock()
-	return e
-}
-
-// computeMarshalFieldInfo fills up the information to marshal a field.
-func (fi *marshalFieldInfo) computeMarshalFieldInfo(f *reflect.StructField) {
-	// parse protobuf tag of the field.
-	// tag has format of "bytes,49,opt,name=foo,def=hello!"
-	tags := strings.Split(f.Tag.Get("protobuf"), ",")
-	if tags[0] == "" {
-		return
-	}
-	tag, err := strconv.Atoi(tags[1])
-	if err != nil {
-		panic("tag is not an integer")
-	}
-	wt := wiretype(tags[0])
-	if tags[2] == "req" {
-		fi.required = true
-	}
-	fi.setTag(f, tag, wt)
-	fi.setMarshaler(f, tags)
-}
-
-func (fi *marshalFieldInfo) computeOneofFieldInfo(f *reflect.StructField, oneofImplementers []interface{}) {
-	fi.field = toField(f)
-	fi.wiretag = math.MaxInt32 // Use a large tag number, make oneofs sorted at the end. This tag will not appear on the wire.
-	fi.isPointer = true
-	fi.sizer, fi.marshaler = makeOneOfMarshaler(fi, f)
-	fi.oneofElems = make(map[reflect.Type]*marshalElemInfo)
-
-	ityp := f.Type // interface type
-	for _, o := range oneofImplementers {
-		t := reflect.TypeOf(o)
-		if !t.Implements(ityp) {
-			continue
-		}
-		sf := t.Elem().Field(0) // oneof implementer is a struct with a single field
-		tags := strings.Split(sf.Tag.Get("protobuf"), ",")
-		tag, err := strconv.Atoi(tags[1])
-		if err != nil {
-			panic("tag is not an integer")
-		}
-		wt := wiretype(tags[0])
-		sizer, marshaler := typeMarshaler(sf.Type, tags, false, true) // oneof should not omit any zero value
-		fi.oneofElems[t.Elem()] = &marshalElemInfo{
-			wiretag:   uint64(tag)<<3 | wt,
-			tagsize:   SizeVarint(uint64(tag) << 3),
-			sizer:     sizer,
-			marshaler: marshaler,
-		}
-	}
-}
-
-// wiretype returns the wire encoding of the type.
-func wiretype(encoding string) uint64 {
-	switch encoding {
-	case "fixed32":
-		return WireFixed32
-	case "fixed64":
-		return WireFixed64
-	case "varint", "zigzag32", "zigzag64":
-		return WireVarint
-	case "bytes":
-		return WireBytes
-	case "group":
-		return WireStartGroup
-	}
-	panic("unknown wire type " + encoding)
-}
-
-// setTag fills up the tag (in wire format) and its size in the info of a field.
-func (fi *marshalFieldInfo) setTag(f *reflect.StructField, tag int, wt uint64) {
-	fi.field = toField(f)
-	fi.wiretag = uint64(tag)<<3 | wt
-	fi.tagsize = SizeVarint(uint64(tag) << 3)
-}
-
-// setMarshaler fills up the sizer and marshaler in the info of a field.
-func (fi *marshalFieldInfo) setMarshaler(f *reflect.StructField, tags []string) {
-	switch f.Type.Kind() {
-	case reflect.Map:
-		// map field
-		fi.isPointer = true
-		fi.sizer, fi.marshaler = makeMapMarshaler(f)
-		return
-	case reflect.Ptr, reflect.Slice:
-		fi.isPointer = true
-	}
-	fi.sizer, fi.marshaler = typeMarshaler(f.Type, tags, true, false)
-}
-
-// typeMarshaler returns the sizer and marshaler of a given field.
-// t is the type of the field.
-// tags is the generated "protobuf" tag of the field.
-// If nozero is true, zero value is not marshaled to the wire.
-// If oneof is true, it is a oneof field.
-func typeMarshaler(t reflect.Type, tags []string, nozero, oneof bool) (sizer, marshaler) {
-	encoding := tags[0]
-
-	pointer := false
-	slice := false
-	if t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 {
-		slice = true
-		t = t.Elem()
-	}
-	if t.Kind() == reflect.Ptr {
-		pointer = true
-		t = t.Elem()
-	}
-
-	packed := false
-	proto3 := false
-	validateUTF8 := true
-	for i := 2; i < len(tags); i++ {
-		if tags[i] == "packed" {
-			packed = true
-		}
-		if tags[i] == "proto3" {
-			proto3 = true
-		}
-	}
-	validateUTF8 = validateUTF8 && proto3
-
-	switch t.Kind() {
-	case reflect.Bool:
-		if pointer {
-			return sizeBoolPtr, appendBoolPtr
-		}
-		if slice {
-			if packed {
-				return sizeBoolPackedSlice, appendBoolPackedSlice
-			}
-			return sizeBoolSlice, appendBoolSlice
-		}
-		if nozero {
-			return sizeBoolValueNoZero, appendBoolValueNoZero
-		}
-		return sizeBoolValue, appendBoolValue
-	case reflect.Uint32:
-		switch encoding {
-		case "fixed32":
-			if pointer {
-				return sizeFixed32Ptr, appendFixed32Ptr
-			}
-			if slice {
-				if packed {
-					return sizeFixed32PackedSlice, appendFixed32PackedSlice
-				}
-				return sizeFixed32Slice, appendFixed32Slice
-			}
-			if nozero {
-				return sizeFixed32ValueNoZero, appendFixed32ValueNoZero
-			}
-			return sizeFixed32Value, appendFixed32Value
-		case "varint":
-			if pointer {
-				return sizeVarint32Ptr, appendVarint32Ptr
-			}
-			if slice {
-				if packed {
-					return sizeVarint32PackedSlice, appendVarint32PackedSlice
-				}
-				return sizeVarint32Slice, appendVarint32Slice
-			}
-			if nozero {
-				return sizeVarint32ValueNoZero, appendVarint32ValueNoZero
-			}
-			return sizeVarint32Value, appendVarint32Value
-		}
-	case reflect.Int32:
-		switch encoding {
-		case "fixed32":
-			if pointer {
-				return sizeFixedS32Ptr, appendFixedS32Ptr
-			}
-			if slice {
-				if packed {
-					return sizeFixedS32PackedSlice, appendFixedS32PackedSlice
-				}
-				return sizeFixedS32Slice, appendFixedS32Slice
-			}
-			if nozero {
-				return sizeFixedS32ValueNoZero, appendFixedS32ValueNoZero
-			}
-			return sizeFixedS32Value, appendFixedS32Value
-		case "varint":
-			if pointer {
-				return sizeVarintS32Ptr, appendVarintS32Ptr
-			}
-			if slice {
-				if packed {
-					return sizeVarintS32PackedSlice, appendVarintS32PackedSlice
-				}
-				return sizeVarintS32Slice, appendVarintS32Slice
-			}
-			if nozero {
-				return sizeVarintS32ValueNoZero, appendVarintS32ValueNoZero
-			}
-			return sizeVarintS32Value, appendVarintS32Value
-		case "zigzag32":
-			if pointer {
-				return sizeZigzag32Ptr, appendZigzag32Ptr
-			}
-			if slice {
-				if packed {
-					return sizeZigzag32PackedSlice, appendZigzag32PackedSlice
-				}
-				return sizeZigzag32Slice, appendZigzag32Slice
-			}
-			if nozero {
-				return sizeZigzag32ValueNoZero, appendZigzag32ValueNoZero
-			}
-			return sizeZigzag32Value, appendZigzag32Value
-		}
-	case reflect.Uint64:
-		switch encoding {
-		case "fixed64":
-			if pointer {
-				return sizeFixed64Ptr, appendFixed64Ptr
-			}
-			if slice {
-				if packed {
-					return sizeFixed64PackedSlice, appendFixed64PackedSlice
-				}
-				return sizeFixed64Slice, appendFixed64Slice
-			}
-			if nozero {
-				return sizeFixed64ValueNoZero, appendFixed64ValueNoZero
-			}
-			return sizeFixed64Value, appendFixed64Value
-		case "varint":
-			if pointer {
-				return sizeVarint64Ptr, appendVarint64Ptr
-			}
-			if slice {
-				if packed {
-					return sizeVarint64PackedSlice, appendVarint64PackedSlice
-				}
-				return sizeVarint64Slice, appendVarint64Slice
-			}
-			if nozero {
-				return sizeVarint64ValueNoZero, appendVarint64ValueNoZero
-			}
-			return sizeVarint64Value, appendVarint64Value
-		}
-	case reflect.Int64:
-		switch encoding {
-		case "fixed64":
-			if pointer {
-				return sizeFixedS64Ptr, appendFixedS64Ptr
-			}
-			if slice {
-				if packed {
-					return sizeFixedS64PackedSlice, appendFixedS64PackedSlice
-				}
-				return sizeFixedS64Slice, appendFixedS64Slice
-			}
-			if nozero {
-				return sizeFixedS64ValueNoZero, appendFixedS64ValueNoZero
-			}
-			return sizeFixedS64Value, appendFixedS64Value
-		case "varint":
-			if pointer {
-				return sizeVarintS64Ptr, appendVarintS64Ptr
-			}
-			if slice {
-				if packed {
-					return sizeVarintS64PackedSlice, appendVarintS64PackedSlice
-				}
-				return sizeVarintS64Slice, appendVarintS64Slice
-			}
-			if nozero {
-				return sizeVarintS64ValueNoZero, appendVarintS64ValueNoZero
-			}
-			return sizeVarintS64Value, appendVarintS64Value
-		case "zigzag64":
-			if pointer {
-				return sizeZigzag64Ptr, appendZigzag64Ptr
-			}
-			if slice {
-				if packed {
-					return sizeZigzag64PackedSlice, appendZigzag64PackedSlice
-				}
-				return sizeZigzag64Slice, appendZigzag64Slice
-			}
-			if nozero {
-				return sizeZigzag64ValueNoZero, appendZigzag64ValueNoZero
-			}
-			return sizeZigzag64Value, appendZigzag64Value
-		}
-	case reflect.Float32:
-		if pointer {
-			return sizeFloat32Ptr, appendFloat32Ptr
-		}
-		if slice {
-			if packed {
-				return sizeFloat32PackedSlice, appendFloat32PackedSlice
-			}
-			return sizeFloat32Slice, appendFloat32Slice
-		}
-		if nozero {
-			return sizeFloat32ValueNoZero, appendFloat32ValueNoZero
-		}
-		return sizeFloat32Value, appendFloat32Value
-	case reflect.Float64:
-		if pointer {
-			return sizeFloat64Ptr, appendFloat64Ptr
-		}
-		if slice {
-			if packed {
-				return sizeFloat64PackedSlice, appendFloat64PackedSlice
-			}
-			return sizeFloat64Slice, appendFloat64Slice
-		}
-		if nozero {
-			return sizeFloat64ValueNoZero, appendFloat64ValueNoZero
-		}
-		return sizeFloat64Value, appendFloat64Value
-	case reflect.String:
-		if validateUTF8 {
-			if pointer {
-				return sizeStringPtr, appendUTF8StringPtr
-			}
-			if slice {
-				return sizeStringSlice, appendUTF8StringSlice
-			}
-			if nozero {
-				return sizeStringValueNoZero, appendUTF8StringValueNoZero
-			}
-			return sizeStringValue, appendUTF8StringValue
-		}
-		if pointer {
-			return sizeStringPtr, appendStringPtr
-		}
-		if slice {
-			return sizeStringSlice, appendStringSlice
-		}
-		if nozero {
-			return sizeStringValueNoZero, appendStringValueNoZero
-		}
-		return sizeStringValue, appendStringValue
-	case reflect.Slice:
-		if slice {
-			return sizeBytesSlice, appendBytesSlice
-		}
-		if oneof {
-			// Oneof bytes field may also have "proto3" tag.
-			// We want to marshal it as a oneof field. Do this
-			// check before the proto3 check.
-			return sizeBytesOneof, appendBytesOneof
-		}
-		if proto3 {
-			return sizeBytes3, appendBytes3
-		}
-		return sizeBytes, appendBytes
-	case reflect.Struct:
-		switch encoding {
-		case "group":
-			if slice {
-				return makeGroupSliceMarshaler(getMarshalInfo(t))
-			}
-			return makeGroupMarshaler(getMarshalInfo(t))
-		case "bytes":
-			if slice {
-				return makeMessageSliceMarshaler(getMarshalInfo(t))
-			}
-			return makeMessageMarshaler(getMarshalInfo(t))
-		}
-	}
-	panic(fmt.Sprintf("unknown or mismatched type: type: %v, wire type: %v", t, encoding))
-}
-
-// Below are functions to size/marshal a specific type of a field.
-// They are stored in the field's info, and called by function pointers.
-// They have type sizer or marshaler.
-
-func sizeFixed32Value(_ pointer, tagsize int) int {
-	return 4 + tagsize
-}
-func sizeFixed32ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toUint32()
-	if v == 0 {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFixed32Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toUint32Ptr()
-	if p == nil {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFixed32Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint32Slice()
-	return (4 + tagsize) * len(s)
-}
-func sizeFixed32PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize
-}
-func sizeFixedS32Value(_ pointer, tagsize int) int {
-	return 4 + tagsize
-}
-func sizeFixedS32ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFixedS32Ptr(ptr pointer, tagsize int) int {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFixedS32Slice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	return (4 + tagsize) * len(s)
-}
-func sizeFixedS32PackedSlice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize
-}
-func sizeFloat32Value(_ pointer, tagsize int) int {
-	return 4 + tagsize
-}
-func sizeFloat32ValueNoZero(ptr pointer, tagsize int) int {
-	v := math.Float32bits(*ptr.toFloat32())
-	if v == 0 {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFloat32Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toFloat32Ptr()
-	if p == nil {
-		return 0
-	}
-	return 4 + tagsize
-}
-func sizeFloat32Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toFloat32Slice()
-	return (4 + tagsize) * len(s)
-}
-func sizeFloat32PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toFloat32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize
-}
-func sizeFixed64Value(_ pointer, tagsize int) int {
-	return 8 + tagsize
-}
-func sizeFixed64ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toUint64()
-	if v == 0 {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFixed64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toUint64Ptr()
-	if p == nil {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFixed64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint64Slice()
-	return (8 + tagsize) * len(s)
-}
-func sizeFixed64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize
-}
-func sizeFixedS64Value(_ pointer, tagsize int) int {
-	return 8 + tagsize
-}
-func sizeFixedS64ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFixedS64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFixedS64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	return (8 + tagsize) * len(s)
-}
-func sizeFixedS64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize
-}
-func sizeFloat64Value(_ pointer, tagsize int) int {
-	return 8 + tagsize
-}
-func sizeFloat64ValueNoZero(ptr pointer, tagsize int) int {
-	v := math.Float64bits(*ptr.toFloat64())
-	if v == 0 {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFloat64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toFloat64Ptr()
-	if p == nil {
-		return 0
-	}
-	return 8 + tagsize
-}
-func sizeFloat64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toFloat64Slice()
-	return (8 + tagsize) * len(s)
-}
-func sizeFloat64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toFloat64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize
-}
-func sizeVarint32Value(ptr pointer, tagsize int) int {
-	v := *ptr.toUint32()
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarint32ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toUint32()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarint32Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toUint32Ptr()
-	if p == nil {
-		return 0
-	}
-	return SizeVarint(uint64(*p)) + tagsize
-}
-func sizeVarint32Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint32Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v)) + tagsize
-	}
-	return n
-}
-func sizeVarint32PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeVarintS32Value(ptr pointer, tagsize int) int {
-	v := *ptr.toInt32()
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarintS32ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarintS32Ptr(ptr pointer, tagsize int) int {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return 0
-	}
-	return SizeVarint(uint64(*p)) + tagsize
-}
-func sizeVarintS32Slice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v)) + tagsize
-	}
-	return n
-}
-func sizeVarintS32PackedSlice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeVarint64Value(ptr pointer, tagsize int) int {
-	v := *ptr.toUint64()
-	return SizeVarint(v) + tagsize
-}
-func sizeVarint64ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toUint64()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(v) + tagsize
-}
-func sizeVarint64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toUint64Ptr()
-	if p == nil {
-		return 0
-	}
-	return SizeVarint(*p) + tagsize
-}
-func sizeVarint64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint64Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(v) + tagsize
-	}
-	return n
-}
-func sizeVarint64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toUint64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(v)
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeVarintS64Value(ptr pointer, tagsize int) int {
-	v := *ptr.toInt64()
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarintS64ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(uint64(v)) + tagsize
-}
-func sizeVarintS64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return 0
-	}
-	return SizeVarint(uint64(*p)) + tagsize
-}
-func sizeVarintS64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v)) + tagsize
-	}
-	return n
-}
-func sizeVarintS64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeZigzag32Value(ptr pointer, tagsize int) int {
-	v := *ptr.toInt32()
-	return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize
-}
-func sizeZigzag32ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize
-}
-func sizeZigzag32Ptr(ptr pointer, tagsize int) int {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return 0
-	}
-	v := *p
-	return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize
-}
-func sizeZigzag32Slice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize
-	}
-	return n
-}
-func sizeZigzag32PackedSlice(ptr pointer, tagsize int) int {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64((uint32(v) << 1) ^ uint32((int32(v) >> 31))))
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeZigzag64Value(ptr pointer, tagsize int) int {
-	v := *ptr.toInt64()
-	return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize
-}
-func sizeZigzag64ValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return 0
-	}
-	return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize
-}
-func sizeZigzag64Ptr(ptr pointer, tagsize int) int {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return 0
-	}
-	v := *p
-	return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize
-}
-func sizeZigzag64Slice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize
-	}
-	return n
-}
-func sizeZigzag64PackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return 0
-	}
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v<<1) ^ uint64((int64(v) >> 63)))
-	}
-	return n + SizeVarint(uint64(n)) + tagsize
-}
-func sizeBoolValue(_ pointer, tagsize int) int {
-	return 1 + tagsize
-}
-func sizeBoolValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toBool()
-	if !v {
-		return 0
-	}
-	return 1 + tagsize
-}
-func sizeBoolPtr(ptr pointer, tagsize int) int {
-	p := *ptr.toBoolPtr()
-	if p == nil {
-		return 0
-	}
-	return 1 + tagsize
-}
-func sizeBoolSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toBoolSlice()
-	return (1 + tagsize) * len(s)
-}
-func sizeBoolPackedSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toBoolSlice()
-	if len(s) == 0 {
-		return 0
-	}
-	return len(s) + SizeVarint(uint64(len(s))) + tagsize
-}
-func sizeStringValue(ptr pointer, tagsize int) int {
-	v := *ptr.toString()
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeStringValueNoZero(ptr pointer, tagsize int) int {
-	v := *ptr.toString()
-	if v == "" {
-		return 0
-	}
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeStringPtr(ptr pointer, tagsize int) int {
-	p := *ptr.toStringPtr()
-	if p == nil {
-		return 0
-	}
-	v := *p
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeStringSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toStringSlice()
-	n := 0
-	for _, v := range s {
-		n += len(v) + SizeVarint(uint64(len(v))) + tagsize
-	}
-	return n
-}
-func sizeBytes(ptr pointer, tagsize int) int {
-	v := *ptr.toBytes()
-	if v == nil {
-		return 0
-	}
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeBytes3(ptr pointer, tagsize int) int {
-	v := *ptr.toBytes()
-	if len(v) == 0 {
-		return 0
-	}
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeBytesOneof(ptr pointer, tagsize int) int {
-	v := *ptr.toBytes()
-	return len(v) + SizeVarint(uint64(len(v))) + tagsize
-}
-func sizeBytesSlice(ptr pointer, tagsize int) int {
-	s := *ptr.toBytesSlice()
-	n := 0
-	for _, v := range s {
-		n += len(v) + SizeVarint(uint64(len(v))) + tagsize
-	}
-	return n
-}
-
-// appendFixed32 appends an encoded fixed32 to b.
-func appendFixed32(b []byte, v uint32) []byte {
-	b = append(b,
-		byte(v),
-		byte(v>>8),
-		byte(v>>16),
-		byte(v>>24))
-	return b
-}
-
-// appendFixed64 appends an encoded fixed64 to b.
-func appendFixed64(b []byte, v uint64) []byte {
-	b = append(b,
-		byte(v),
-		byte(v>>8),
-		byte(v>>16),
-		byte(v>>24),
-		byte(v>>32),
-		byte(v>>40),
-		byte(v>>48),
-		byte(v>>56))
-	return b
-}
-
-// appendVarint appends an encoded varint to b.
-func appendVarint(b []byte, v uint64) []byte {
-	// TODO: make 1-byte (maybe 2-byte) case inline-able, once we
-	// have non-leaf inliner.
-	switch {
-	case v < 1<<7:
-		b = append(b, byte(v))
-	case v < 1<<14:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte(v>>7))
-	case v < 1<<21:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte(v>>14))
-	case v < 1<<28:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte(v>>21))
-	case v < 1<<35:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte(v>>28))
-	case v < 1<<42:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte((v>>28)&0x7f|0x80),
-			byte(v>>35))
-	case v < 1<<49:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte((v>>28)&0x7f|0x80),
-			byte((v>>35)&0x7f|0x80),
-			byte(v>>42))
-	case v < 1<<56:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte((v>>28)&0x7f|0x80),
-			byte((v>>35)&0x7f|0x80),
-			byte((v>>42)&0x7f|0x80),
-			byte(v>>49))
-	case v < 1<<63:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte((v>>28)&0x7f|0x80),
-			byte((v>>35)&0x7f|0x80),
-			byte((v>>42)&0x7f|0x80),
-			byte((v>>49)&0x7f|0x80),
-			byte(v>>56))
-	default:
-		b = append(b,
-			byte(v&0x7f|0x80),
-			byte((v>>7)&0x7f|0x80),
-			byte((v>>14)&0x7f|0x80),
-			byte((v>>21)&0x7f|0x80),
-			byte((v>>28)&0x7f|0x80),
-			byte((v>>35)&0x7f|0x80),
-			byte((v>>42)&0x7f|0x80),
-			byte((v>>49)&0x7f|0x80),
-			byte((v>>56)&0x7f|0x80),
-			1)
-	}
-	return b
-}
-
-func appendFixed32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint32()
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, v)
-	return b, nil
-}
-func appendFixed32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint32()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, v)
-	return b, nil
-}
-func appendFixed32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toUint32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, *p)
-	return b, nil
-}
-func appendFixed32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed32(b, v)
-	}
-	return b, nil
-}
-func appendFixed32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(4*len(s)))
-	for _, v := range s {
-		b = appendFixed32(b, v)
-	}
-	return b, nil
-}
-func appendFixedS32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, uint32(v))
-	return b, nil
-}
-func appendFixedS32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, uint32(v))
-	return b, nil
-}
-func appendFixedS32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, uint32(*p))
-	return b, nil
-}
-func appendFixedS32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed32(b, uint32(v))
-	}
-	return b, nil
-}
-func appendFixedS32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(4*len(s)))
-	for _, v := range s {
-		b = appendFixed32(b, uint32(v))
-	}
-	return b, nil
-}
-func appendFloat32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := math.Float32bits(*ptr.toFloat32())
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, v)
-	return b, nil
-}
-func appendFloat32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := math.Float32bits(*ptr.toFloat32())
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, v)
-	return b, nil
-}
-func appendFloat32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toFloat32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed32(b, math.Float32bits(*p))
-	return b, nil
-}
-func appendFloat32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toFloat32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed32(b, math.Float32bits(v))
-	}
-	return b, nil
-}
-func appendFloat32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toFloat32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(4*len(s)))
-	for _, v := range s {
-		b = appendFixed32(b, math.Float32bits(v))
-	}
-	return b, nil
-}
-func appendFixed64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint64()
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, v)
-	return b, nil
-}
-func appendFixed64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint64()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, v)
-	return b, nil
-}
-func appendFixed64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toUint64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, *p)
-	return b, nil
-}
-func appendFixed64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed64(b, v)
-	}
-	return b, nil
-}
-func appendFixed64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(8*len(s)))
-	for _, v := range s {
-		b = appendFixed64(b, v)
-	}
-	return b, nil
-}
-func appendFixedS64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, uint64(v))
-	return b, nil
-}
-func appendFixedS64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, uint64(v))
-	return b, nil
-}
-func appendFixedS64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, uint64(*p))
-	return b, nil
-}
-func appendFixedS64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed64(b, uint64(v))
-	}
-	return b, nil
-}
-func appendFixedS64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(8*len(s)))
-	for _, v := range s {
-		b = appendFixed64(b, uint64(v))
-	}
-	return b, nil
-}
-func appendFloat64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := math.Float64bits(*ptr.toFloat64())
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, v)
-	return b, nil
-}
-func appendFloat64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := math.Float64bits(*ptr.toFloat64())
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, v)
-	return b, nil
-}
-func appendFloat64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toFloat64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendFixed64(b, math.Float64bits(*p))
-	return b, nil
-}
-func appendFloat64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toFloat64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendFixed64(b, math.Float64bits(v))
-	}
-	return b, nil
-}
-func appendFloat64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toFloat64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(8*len(s)))
-	for _, v := range s {
-		b = appendFixed64(b, math.Float64bits(v))
-	}
-	return b, nil
-}
-func appendVarint32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint32()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarint32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint32()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarint32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toUint32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(*p))
-	return b, nil
-}
-func appendVarint32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendVarint32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendVarintS32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarintS32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarintS32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(*p))
-	return b, nil
-}
-func appendVarintS32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendVarintS32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendVarint64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint64()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, v)
-	return b, nil
-}
-func appendVarint64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toUint64()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, v)
-	return b, nil
-}
-func appendVarint64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toUint64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, *p)
-	return b, nil
-}
-func appendVarint64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, v)
-	}
-	return b, nil
-}
-func appendVarint64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toUint64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(v)
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, v)
-	}
-	return b, nil
-}
-func appendVarintS64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarintS64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v))
-	return b, nil
-}
-func appendVarintS64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(*p))
-	return b, nil
-}
-func appendVarintS64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendVarintS64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v))
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, uint64(v))
-	}
-	return b, nil
-}
-func appendZigzag32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31))))
-	return b, nil
-}
-func appendZigzag32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt32()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31))))
-	return b, nil
-}
-func appendZigzag32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := ptr.getInt32Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	v := *p
-	b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31))))
-	return b, nil
-}
-func appendZigzag32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31))))
-	}
-	return b, nil
-}
-func appendZigzag32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := ptr.getInt32Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64((uint32(v) << 1) ^ uint32((int32(v) >> 31))))
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31))))
-	}
-	return b, nil
-}
-func appendZigzag64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63)))
-	return b, nil
-}
-func appendZigzag64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toInt64()
-	if v == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63)))
-	return b, nil
-}
-func appendZigzag64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toInt64Ptr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	v := *p
-	b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63)))
-	return b, nil
-}
-func appendZigzag64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63)))
-	}
-	return b, nil
-}
-func appendZigzag64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toInt64Slice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	// compute size
-	n := 0
-	for _, v := range s {
-		n += SizeVarint(uint64(v<<1) ^ uint64((int64(v) >> 63)))
-	}
-	b = appendVarint(b, uint64(n))
-	for _, v := range s {
-		b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63)))
-	}
-	return b, nil
-}
-func appendBoolValue(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toBool()
-	b = appendVarint(b, wiretag)
-	if v {
-		b = append(b, 1)
-	} else {
-		b = append(b, 0)
-	}
-	return b, nil
-}
-func appendBoolValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toBool()
-	if !v {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = append(b, 1)
-	return b, nil
-}
-
-func appendBoolPtr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toBoolPtr()
-	if p == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	if *p {
-		b = append(b, 1)
-	} else {
-		b = append(b, 0)
-	}
-	return b, nil
-}
-func appendBoolSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toBoolSlice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		if v {
-			b = append(b, 1)
-		} else {
-			b = append(b, 0)
-		}
-	}
-	return b, nil
-}
-func appendBoolPackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toBoolSlice()
-	if len(s) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag&^7|WireBytes)
-	b = appendVarint(b, uint64(len(s)))
-	for _, v := range s {
-		if v {
-			b = append(b, 1)
-		} else {
-			b = append(b, 0)
-		}
-	}
-	return b, nil
-}
-func appendStringValue(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toString()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendStringValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toString()
-	if v == "" {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendStringPtr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	p := *ptr.toStringPtr()
-	if p == nil {
-		return b, nil
-	}
-	v := *p
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendStringSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toStringSlice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(len(v)))
-		b = append(b, v...)
-	}
-	return b, nil
-}
-func appendUTF8StringValue(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	var invalidUTF8 bool
-	v := *ptr.toString()
-	if !utf8.ValidString(v) {
-		invalidUTF8 = true
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	if invalidUTF8 {
-		return b, errInvalidUTF8
-	}
-	return b, nil
-}
-func appendUTF8StringValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	var invalidUTF8 bool
-	v := *ptr.toString()
-	if v == "" {
-		return b, nil
-	}
-	if !utf8.ValidString(v) {
-		invalidUTF8 = true
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	if invalidUTF8 {
-		return b, errInvalidUTF8
-	}
-	return b, nil
-}
-func appendUTF8StringPtr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	var invalidUTF8 bool
-	p := *ptr.toStringPtr()
-	if p == nil {
-		return b, nil
-	}
-	v := *p
-	if !utf8.ValidString(v) {
-		invalidUTF8 = true
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	if invalidUTF8 {
-		return b, errInvalidUTF8
-	}
-	return b, nil
-}
-func appendUTF8StringSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	var invalidUTF8 bool
-	s := *ptr.toStringSlice()
-	for _, v := range s {
-		if !utf8.ValidString(v) {
-			invalidUTF8 = true
-		}
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(len(v)))
-		b = append(b, v...)
-	}
-	if invalidUTF8 {
-		return b, errInvalidUTF8
-	}
-	return b, nil
-}
-func appendBytes(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toBytes()
-	if v == nil {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendBytes3(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toBytes()
-	if len(v) == 0 {
-		return b, nil
-	}
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendBytesOneof(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	v := *ptr.toBytes()
-	b = appendVarint(b, wiretag)
-	b = appendVarint(b, uint64(len(v)))
-	b = append(b, v...)
-	return b, nil
-}
-func appendBytesSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) {
-	s := *ptr.toBytesSlice()
-	for _, v := range s {
-		b = appendVarint(b, wiretag)
-		b = appendVarint(b, uint64(len(v)))
-		b = append(b, v...)
-	}
-	return b, nil
-}
-
-// makeGroupMarshaler returns the sizer and marshaler for a group.
-// u is the marshal info of the underlying message.
-func makeGroupMarshaler(u *marshalInfo) (sizer, marshaler) {
-	return func(ptr pointer, tagsize int) int {
-			p := ptr.getPointer()
-			if p.isNil() {
-				return 0
-			}
-			return u.size(p) + 2*tagsize
-		},
-		func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) {
-			p := ptr.getPointer()
-			if p.isNil() {
-				return b, nil
-			}
-			var err error
-			b = appendVarint(b, wiretag) // start group
-			b, err = u.marshal(b, p, deterministic)
-			b = appendVarint(b, wiretag+(WireEndGroup-WireStartGroup)) // end group
-			return b, err
-		}
-}
-
-// makeGroupSliceMarshaler returns the sizer and marshaler for a group slice.
-// u is the marshal info of the underlying message.
-func makeGroupSliceMarshaler(u *marshalInfo) (sizer, marshaler) {
-	return func(ptr pointer, tagsize int) int {
-			s := ptr.getPointerSlice()
-			n := 0
-			for _, v := range s {
-				if v.isNil() {
-					continue
-				}
-				n += u.size(v) + 2*tagsize
-			}
-			return n
-		},
-		func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) {
-			s := ptr.getPointerSlice()
-			var err error
-			var nerr nonFatal
-			for _, v := range s {
-				if v.isNil() {
-					return b, errRepeatedHasNil
-				}
-				b = appendVarint(b, wiretag) // start group
-				b, err = u.marshal(b, v, deterministic)
-				b = appendVarint(b, wiretag+(WireEndGroup-WireStartGroup)) // end group
-				if !nerr.Merge(err) {
-					if err == ErrNil {
-						err = errRepeatedHasNil
-					}
-					return b, err
-				}
-			}
-			return b, nerr.E
-		}
-}
-
-// makeMessageMarshaler returns the sizer and marshaler for a message field.
-// u is the marshal info of the message.
-func makeMessageMarshaler(u *marshalInfo) (sizer, marshaler) {
-	return func(ptr pointer, tagsize int) int {
-			p := ptr.getPointer()
-			if p.isNil() {
-				return 0
-			}
-			siz := u.size(p)
-			return siz + SizeVarint(uint64(siz)) + tagsize
-		},
-		func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) {
-			p := ptr.getPointer()
-			if p.isNil() {
-				return b, nil
-			}
-			b = appendVarint(b, wiretag)
-			siz := u.cachedsize(p)
-			b = appendVarint(b, uint64(siz))
-			return u.marshal(b, p, deterministic)
-		}
-}
-
-// makeMessageSliceMarshaler returns the sizer and marshaler for a message slice.
-// u is the marshal info of the message.
-func makeMessageSliceMarshaler(u *marshalInfo) (sizer, marshaler) {
-	return func(ptr pointer, tagsize int) int {
-			s := ptr.getPointerSlice()
-			n := 0
-			for _, v := range s {
-				if v.isNil() {
-					continue
-				}
-				siz := u.size(v)
-				n += siz + SizeVarint(uint64(siz)) + tagsize
-			}
-			return n
-		},
-		func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) {
-			s := ptr.getPointerSlice()
-			var err error
-			var nerr nonFatal
-			for _, v := range s {
-				if v.isNil() {
-					return b, errRepeatedHasNil
-				}
-				b = appendVarint(b, wiretag)
-				siz := u.cachedsize(v)
-				b = appendVarint(b, uint64(siz))
-				b, err = u.marshal(b, v, deterministic)
-
-				if !nerr.Merge(err) {
-					if err == ErrNil {
-						err = errRepeatedHasNil
-					}
-					return b, err
-				}
-			}
-			return b, nerr.E
-		}
-}
-
-// makeMapMarshaler returns the sizer and marshaler for a map field.
-// f is the pointer to the reflect data structure of the field.
-func makeMapMarshaler(f *reflect.StructField) (sizer, marshaler) {
-	// figure out key and value type
-	t := f.Type
-	keyType := t.Key()
-	valType := t.Elem()
-	keyTags := strings.Split(f.Tag.Get("protobuf_key"), ",")
-	valTags := strings.Split(f.Tag.Get("protobuf_val"), ",")
-	keySizer, keyMarshaler := typeMarshaler(keyType, keyTags, false, false) // don't omit zero value in map
-	valSizer, valMarshaler := typeMarshaler(valType, valTags, false, false) // don't omit zero value in map
-	keyWireTag := 1<<3 | wiretype(keyTags[0])
-	valWireTag := 2<<3 | wiretype(valTags[0])
-
-	// We create an interface to get the addresses of the map key and value.
-	// If value is pointer-typed, the interface is a direct interface, the
-	// idata itself is the value. Otherwise, the idata is the pointer to the
-	// value.
-	// Key cannot be pointer-typed.
-	valIsPtr := valType.Kind() == reflect.Ptr
-
-	// If value is a message with nested maps, calling
-	// valSizer in marshal may be quadratic. We should use
-	// cached version in marshal (but not in size).
-	// If value is not message type, we don't have size cache,
-	// but it cannot be nested either. Just use valSizer.
-	valCachedSizer := valSizer
-	if valIsPtr && valType.Elem().Kind() == reflect.Struct {
-		u := getMarshalInfo(valType.Elem())
-		valCachedSizer = func(ptr pointer, tagsize int) int {
-			// Same as message sizer, but use cache.
-			p := ptr.getPointer()
-			if p.isNil() {
-				return 0
-			}
-			siz := u.cachedsize(p)
-			return siz + SizeVarint(uint64(siz)) + tagsize
-		}
-	}
-	return func(ptr pointer, tagsize int) int {
-			m := ptr.asPointerTo(t).Elem() // the map
-			n := 0
-			for _, k := range m.MapKeys() {
-				ki := k.Interface()
-				vi := m.MapIndex(k).Interface()
-				kaddr := toAddrPointer(&ki, false, false)      // pointer to key
-				vaddr := toAddrPointer(&vi, valIsPtr, false)   // pointer to value
-				siz := keySizer(kaddr, 1) + valSizer(vaddr, 1) // tag of key = 1 (size=1), tag of val = 2 (size=1)
-				n += siz + SizeVarint(uint64(siz)) + tagsize
-			}
-			return n
-		},
-		func(b []byte, ptr pointer, tag uint64, deterministic bool) ([]byte, error) {
-			m := ptr.asPointerTo(t).Elem() // the map
-			var err error
-			keys := m.MapKeys()
-			if len(keys) > 1 && deterministic {
-				sort.Sort(mapKeys(keys))
-			}
-
-			var nerr nonFatal
-			for _, k := range keys {
-				ki := k.Interface()
-				vi := m.MapIndex(k).Interface()
-				kaddr := toAddrPointer(&ki, false, false)    // pointer to key
-				vaddr := toAddrPointer(&vi, valIsPtr, false) // pointer to value
-				b = appendVarint(b, tag)
-				siz := keySizer(kaddr, 1) + valCachedSizer(vaddr, 1) // tag of key = 1 (size=1), tag of val = 2 (size=1)
-				b = appendVarint(b, uint64(siz))
-				b, err = keyMarshaler(b, kaddr, keyWireTag, deterministic)
-				if !nerr.Merge(err) {
-					return b, err
-				}
-				b, err = valMarshaler(b, vaddr, valWireTag, deterministic)
-				if err != ErrNil && !nerr.Merge(err) { // allow nil value in map
-					return b, err
-				}
-			}
-			return b, nerr.E
-		}
-}
-
-// makeOneOfMarshaler returns the sizer and marshaler for a oneof field.
-// fi is the marshal info of the field.
-// f is the pointer to the reflect data structure of the field.
-func makeOneOfMarshaler(fi *marshalFieldInfo, f *reflect.StructField) (sizer, marshaler) {
-	// Oneof field is an interface. We need to get the actual data type on the fly.
-	t := f.Type
-	return func(ptr pointer, _ int) int {
-			p := ptr.getInterfacePointer()
-			if p.isNil() {
-				return 0
-			}
-			v := ptr.asPointerTo(t).Elem().Elem().Elem() // *interface -> interface -> *struct -> struct
-			telem := v.Type()
-			e := fi.oneofElems[telem]
-			return e.sizer(p, e.tagsize)
-		},
-		func(b []byte, ptr pointer, _ uint64, deterministic bool) ([]byte, error) {
-			p := ptr.getInterfacePointer()
-			if p.isNil() {
-				return b, nil
-			}
-			v := ptr.asPointerTo(t).Elem().Elem().Elem() // *interface -> interface -> *struct -> struct
-			telem := v.Type()
-			if telem.Field(0).Type.Kind() == reflect.Ptr && p.getPointer().isNil() {
-				return b, errOneofHasNil
-			}
-			e := fi.oneofElems[telem]
-			return e.marshaler(b, p, e.wiretag, deterministic)
-		}
-}
-
-// sizeExtensions computes the size of encoded data for a XXX_InternalExtensions field.
-func (u *marshalInfo) sizeExtensions(ext *XXX_InternalExtensions) int {
-	m, mu := ext.extensionsRead()
-	if m == nil {
-		return 0
-	}
-	mu.Lock()
-
-	n := 0
-	for _, e := range m {
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			n += len(e.enc)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		n += ei.sizer(p, ei.tagsize)
-	}
-	mu.Unlock()
-	return n
-}
-
-// appendExtensions marshals a XXX_InternalExtensions field to the end of byte slice b.
-func (u *marshalInfo) appendExtensions(b []byte, ext *XXX_InternalExtensions, deterministic bool) ([]byte, error) {
-	m, mu := ext.extensionsRead()
-	if m == nil {
-		return b, nil
-	}
-	mu.Lock()
-	defer mu.Unlock()
-
-	var err error
-	var nerr nonFatal
-
-	// Fast-path for common cases: zero or one extensions.
-	// Don't bother sorting the keys.
-	if len(m) <= 1 {
-		for _, e := range m {
-			if e.value == nil || e.desc == nil {
-				// Extension is only in its encoded form.
-				b = append(b, e.enc...)
-				continue
-			}
-
-			// We don't skip extensions that have an encoded form set,
-			// because the extension value may have been mutated after
-			// the last time this function was called.
-
-			ei := u.getExtElemInfo(e.desc)
-			v := e.value
-			p := toAddrPointer(&v, ei.isptr, ei.deref)
-			b, err = ei.marshaler(b, p, ei.wiretag, deterministic)
-			if !nerr.Merge(err) {
-				return b, err
-			}
-		}
-		return b, nerr.E
-	}
-
-	// Sort the keys to provide a deterministic encoding.
-	// Not sure this is required, but the old code does it.
-	keys := make([]int, 0, len(m))
-	for k := range m {
-		keys = append(keys, int(k))
-	}
-	sort.Ints(keys)
-
-	for _, k := range keys {
-		e := m[int32(k)]
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			b = append(b, e.enc...)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		b, err = ei.marshaler(b, p, ei.wiretag, deterministic)
-		if !nerr.Merge(err) {
-			return b, err
-		}
-	}
-	return b, nerr.E
-}
-
-// message set format is:
-//   message MessageSet {
-//     repeated group Item = 1 {
-//       required int32 type_id = 2;
-//       required string message = 3;
-//     };
-//   }
-
-// sizeMessageSet computes the size of encoded data for a XXX_InternalExtensions field
-// in message set format (above).
-func (u *marshalInfo) sizeMessageSet(ext *XXX_InternalExtensions) int {
-	m, mu := ext.extensionsRead()
-	if m == nil {
-		return 0
-	}
-	mu.Lock()
-
-	n := 0
-	for id, e := range m {
-		n += 2                          // start group, end group. tag = 1 (size=1)
-		n += SizeVarint(uint64(id)) + 1 // type_id, tag = 2 (size=1)
-
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint
-			siz := len(msgWithLen)
-			n += siz + 1 // message, tag = 3 (size=1)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		n += ei.sizer(p, 1) // message, tag = 3 (size=1)
-	}
-	mu.Unlock()
-	return n
-}
-
-// appendMessageSet marshals a XXX_InternalExtensions field in message set format (above)
-// to the end of byte slice b.
-func (u *marshalInfo) appendMessageSet(b []byte, ext *XXX_InternalExtensions, deterministic bool) ([]byte, error) {
-	m, mu := ext.extensionsRead()
-	if m == nil {
-		return b, nil
-	}
-	mu.Lock()
-	defer mu.Unlock()
-
-	var err error
-	var nerr nonFatal
-
-	// Fast-path for common cases: zero or one extensions.
-	// Don't bother sorting the keys.
-	if len(m) <= 1 {
-		for id, e := range m {
-			b = append(b, 1<<3|WireStartGroup)
-			b = append(b, 2<<3|WireVarint)
-			b = appendVarint(b, uint64(id))
-
-			if e.value == nil || e.desc == nil {
-				// Extension is only in its encoded form.
-				msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint
-				b = append(b, 3<<3|WireBytes)
-				b = append(b, msgWithLen...)
-				b = append(b, 1<<3|WireEndGroup)
-				continue
-			}
-
-			// We don't skip extensions that have an encoded form set,
-			// because the extension value may have been mutated after
-			// the last time this function was called.
-
-			ei := u.getExtElemInfo(e.desc)
-			v := e.value
-			p := toAddrPointer(&v, ei.isptr, ei.deref)
-			b, err = ei.marshaler(b, p, 3<<3|WireBytes, deterministic)
-			if !nerr.Merge(err) {
-				return b, err
-			}
-			b = append(b, 1<<3|WireEndGroup)
-		}
-		return b, nerr.E
-	}
-
-	// Sort the keys to provide a deterministic encoding.
-	keys := make([]int, 0, len(m))
-	for k := range m {
-		keys = append(keys, int(k))
-	}
-	sort.Ints(keys)
-
-	for _, id := range keys {
-		e := m[int32(id)]
-		b = append(b, 1<<3|WireStartGroup)
-		b = append(b, 2<<3|WireVarint)
-		b = appendVarint(b, uint64(id))
-
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint
-			b = append(b, 3<<3|WireBytes)
-			b = append(b, msgWithLen...)
-			b = append(b, 1<<3|WireEndGroup)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		b, err = ei.marshaler(b, p, 3<<3|WireBytes, deterministic)
-		b = append(b, 1<<3|WireEndGroup)
-		if !nerr.Merge(err) {
-			return b, err
-		}
-	}
-	return b, nerr.E
-}
-
-// sizeV1Extensions computes the size of encoded data for a V1-API extension field.
-func (u *marshalInfo) sizeV1Extensions(m map[int32]Extension) int {
-	if m == nil {
-		return 0
-	}
-
-	n := 0
-	for _, e := range m {
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			n += len(e.enc)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		n += ei.sizer(p, ei.tagsize)
-	}
-	return n
-}
-
-// appendV1Extensions marshals a V1-API extension field to the end of byte slice b.
-func (u *marshalInfo) appendV1Extensions(b []byte, m map[int32]Extension, deterministic bool) ([]byte, error) {
-	if m == nil {
-		return b, nil
-	}
-
-	// Sort the keys to provide a deterministic encoding.
-	keys := make([]int, 0, len(m))
-	for k := range m {
-		keys = append(keys, int(k))
-	}
-	sort.Ints(keys)
-
-	var err error
-	var nerr nonFatal
-	for _, k := range keys {
-		e := m[int32(k)]
-		if e.value == nil || e.desc == nil {
-			// Extension is only in its encoded form.
-			b = append(b, e.enc...)
-			continue
-		}
-
-		// We don't skip extensions that have an encoded form set,
-		// because the extension value may have been mutated after
-		// the last time this function was called.
-
-		ei := u.getExtElemInfo(e.desc)
-		v := e.value
-		p := toAddrPointer(&v, ei.isptr, ei.deref)
-		b, err = ei.marshaler(b, p, ei.wiretag, deterministic)
-		if !nerr.Merge(err) {
-			return b, err
-		}
-	}
-	return b, nerr.E
-}
-
-// newMarshaler is the interface representing objects that can marshal themselves.
-//
-// This exists to support protoc-gen-go generated messages.
-// The proto package will stop type-asserting to this interface in the future.
-//
-// DO NOT DEPEND ON THIS.
-type newMarshaler interface {
-	XXX_Size() int
-	XXX_Marshal(b []byte, deterministic bool) ([]byte, error)
-}
-
-// Size returns the encoded size of a protocol buffer message.
-// This is the main entry point.
-func Size(pb Message) int {
-	if m, ok := pb.(newMarshaler); ok {
-		return m.XXX_Size()
-	}
-	if m, ok := pb.(Marshaler); ok {
-		// If the message can marshal itself, let it do it, for compatibility.
-		// NOTE: This is not efficient.
-		b, _ := m.Marshal()
-		return len(b)
-	}
-	// in case somehow we didn't generate the wrapper
-	if pb == nil {
-		return 0
-	}
-	var info InternalMessageInfo
-	return info.Size(pb)
-}
-
-// Marshal takes a protocol buffer message
-// and encodes it into the wire format, returning the data.
-// This is the main entry point.
-func Marshal(pb Message) ([]byte, error) {
-	if m, ok := pb.(newMarshaler); ok {
-		siz := m.XXX_Size()
-		b := make([]byte, 0, siz)
-		return m.XXX_Marshal(b, false)
-	}
-	if m, ok := pb.(Marshaler); ok {
-		// If the message can marshal itself, let it do it, for compatibility.
-		// NOTE: This is not efficient.
-		return m.Marshal()
-	}
-	// in case somehow we didn't generate the wrapper
-	if pb == nil {
-		return nil, ErrNil
-	}
-	var info InternalMessageInfo
-	siz := info.Size(pb)
-	b := make([]byte, 0, siz)
-	return info.Marshal(b, pb, false)
-}
-
-// Marshal takes a protocol buffer message
-// and encodes it into the wire format, writing the result to the
-// Buffer.
-// This is an alternative entry point. It is not necessary to use
-// a Buffer for most applications.
-func (p *Buffer) Marshal(pb Message) error {
-	var err error
-	if m, ok := pb.(newMarshaler); ok {
-		siz := m.XXX_Size()
-		p.grow(siz) // make sure buf has enough capacity
-		p.buf, err = m.XXX_Marshal(p.buf, p.deterministic)
-		return err
-	}
-	if m, ok := pb.(Marshaler); ok {
-		// If the message can marshal itself, let it do it, for compatibility.
-		// NOTE: This is not efficient.
-		b, err := m.Marshal()
-		p.buf = append(p.buf, b...)
-		return err
-	}
-	// in case somehow we didn't generate the wrapper
-	if pb == nil {
-		return ErrNil
-	}
-	var info InternalMessageInfo
-	siz := info.Size(pb)
-	p.grow(siz) // make sure buf has enough capacity
-	p.buf, err = info.Marshal(p.buf, pb, p.deterministic)
-	return err
-}
-
-// grow grows the buffer's capacity, if necessary, to guarantee space for
-// another n bytes. After grow(n), at least n bytes can be written to the
-// buffer without another allocation.
-func (p *Buffer) grow(n int) {
-	need := len(p.buf) + n
-	if need <= cap(p.buf) {
-		return
-	}
-	newCap := len(p.buf) * 2
-	if newCap < need {
-		newCap = need
-	}
-	p.buf = append(make([]byte, 0, newCap), p.buf...)
-}
diff --git a/vendor/github.com/golang/protobuf/proto/table_merge.go b/vendor/github.com/golang/protobuf/proto/table_merge.go
deleted file mode 100644
index 5525def..0000000
--- a/vendor/github.com/golang/protobuf/proto/table_merge.go
+++ /dev/null
@@ -1,654 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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 proto
-
-import (
-	"fmt"
-	"reflect"
-	"strings"
-	"sync"
-	"sync/atomic"
-)
-
-// Merge merges the src message into dst.
-// This assumes that dst and src of the same type and are non-nil.
-func (a *InternalMessageInfo) Merge(dst, src Message) {
-	mi := atomicLoadMergeInfo(&a.merge)
-	if mi == nil {
-		mi = getMergeInfo(reflect.TypeOf(dst).Elem())
-		atomicStoreMergeInfo(&a.merge, mi)
-	}
-	mi.merge(toPointer(&dst), toPointer(&src))
-}
-
-type mergeInfo struct {
-	typ reflect.Type
-
-	initialized int32 // 0: only typ is valid, 1: everything is valid
-	lock        sync.Mutex
-
-	fields       []mergeFieldInfo
-	unrecognized field // Offset of XXX_unrecognized
-}
-
-type mergeFieldInfo struct {
-	field field // Offset of field, guaranteed to be valid
-
-	// isPointer reports whether the value in the field is a pointer.
-	// This is true for the following situations:
-	//	* Pointer to struct
-	//	* Pointer to basic type (proto2 only)
-	//	* Slice (first value in slice header is a pointer)
-	//	* String (first value in string header is a pointer)
-	isPointer bool
-
-	// basicWidth reports the width of the field assuming that it is directly
-	// embedded in the struct (as is the case for basic types in proto3).
-	// The possible values are:
-	// 	0: invalid
-	//	1: bool
-	//	4: int32, uint32, float32
-	//	8: int64, uint64, float64
-	basicWidth int
-
-	// Where dst and src are pointers to the types being merged.
-	merge func(dst, src pointer)
-}
-
-var (
-	mergeInfoMap  = map[reflect.Type]*mergeInfo{}
-	mergeInfoLock sync.Mutex
-)
-
-func getMergeInfo(t reflect.Type) *mergeInfo {
-	mergeInfoLock.Lock()
-	defer mergeInfoLock.Unlock()
-	mi := mergeInfoMap[t]
-	if mi == nil {
-		mi = &mergeInfo{typ: t}
-		mergeInfoMap[t] = mi
-	}
-	return mi
-}
-
-// merge merges src into dst assuming they are both of type *mi.typ.
-func (mi *mergeInfo) merge(dst, src pointer) {
-	if dst.isNil() {
-		panic("proto: nil destination")
-	}
-	if src.isNil() {
-		return // Nothing to do.
-	}
-
-	if atomic.LoadInt32(&mi.initialized) == 0 {
-		mi.computeMergeInfo()
-	}
-
-	for _, fi := range mi.fields {
-		sfp := src.offset(fi.field)
-
-		// As an optimization, we can avoid the merge function call cost
-		// if we know for sure that the source will have no effect
-		// by checking if it is the zero value.
-		if unsafeAllowed {
-			if fi.isPointer && sfp.getPointer().isNil() { // Could be slice or string
-				continue
-			}
-			if fi.basicWidth > 0 {
-				switch {
-				case fi.basicWidth == 1 && !*sfp.toBool():
-					continue
-				case fi.basicWidth == 4 && *sfp.toUint32() == 0:
-					continue
-				case fi.basicWidth == 8 && *sfp.toUint64() == 0:
-					continue
-				}
-			}
-		}
-
-		dfp := dst.offset(fi.field)
-		fi.merge(dfp, sfp)
-	}
-
-	// TODO: Make this faster?
-	out := dst.asPointerTo(mi.typ).Elem()
-	in := src.asPointerTo(mi.typ).Elem()
-	if emIn, err := extendable(in.Addr().Interface()); err == nil {
-		emOut, _ := extendable(out.Addr().Interface())
-		mIn, muIn := emIn.extensionsRead()
-		if mIn != nil {
-			mOut := emOut.extensionsWrite()
-			muIn.Lock()
-			mergeExtension(mOut, mIn)
-			muIn.Unlock()
-		}
-	}
-
-	if mi.unrecognized.IsValid() {
-		if b := *src.offset(mi.unrecognized).toBytes(); len(b) > 0 {
-			*dst.offset(mi.unrecognized).toBytes() = append([]byte(nil), b...)
-		}
-	}
-}
-
-func (mi *mergeInfo) computeMergeInfo() {
-	mi.lock.Lock()
-	defer mi.lock.Unlock()
-	if mi.initialized != 0 {
-		return
-	}
-	t := mi.typ
-	n := t.NumField()
-
-	props := GetProperties(t)
-	for i := 0; i < n; i++ {
-		f := t.Field(i)
-		if strings.HasPrefix(f.Name, "XXX_") {
-			continue
-		}
-
-		mfi := mergeFieldInfo{field: toField(&f)}
-		tf := f.Type
-
-		// As an optimization, we can avoid the merge function call cost
-		// if we know for sure that the source will have no effect
-		// by checking if it is the zero value.
-		if unsafeAllowed {
-			switch tf.Kind() {
-			case reflect.Ptr, reflect.Slice, reflect.String:
-				// As a special case, we assume slices and strings are pointers
-				// since we know that the first field in the SliceSlice or
-				// StringHeader is a data pointer.
-				mfi.isPointer = true
-			case reflect.Bool:
-				mfi.basicWidth = 1
-			case reflect.Int32, reflect.Uint32, reflect.Float32:
-				mfi.basicWidth = 4
-			case reflect.Int64, reflect.Uint64, reflect.Float64:
-				mfi.basicWidth = 8
-			}
-		}
-
-		// Unwrap tf to get at its most basic type.
-		var isPointer, isSlice bool
-		if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 {
-			isSlice = true
-			tf = tf.Elem()
-		}
-		if tf.Kind() == reflect.Ptr {
-			isPointer = true
-			tf = tf.Elem()
-		}
-		if isPointer && isSlice && tf.Kind() != reflect.Struct {
-			panic("both pointer and slice for basic type in " + tf.Name())
-		}
-
-		switch tf.Kind() {
-		case reflect.Int32:
-			switch {
-			case isSlice: // E.g., []int32
-				mfi.merge = func(dst, src pointer) {
-					// NOTE: toInt32Slice is not defined (see pointer_reflect.go).
-					/*
-						sfsp := src.toInt32Slice()
-						if *sfsp != nil {
-							dfsp := dst.toInt32Slice()
-							*dfsp = append(*dfsp, *sfsp...)
-							if *dfsp == nil {
-								*dfsp = []int64{}
-							}
-						}
-					*/
-					sfs := src.getInt32Slice()
-					if sfs != nil {
-						dfs := dst.getInt32Slice()
-						dfs = append(dfs, sfs...)
-						if dfs == nil {
-							dfs = []int32{}
-						}
-						dst.setInt32Slice(dfs)
-					}
-				}
-			case isPointer: // E.g., *int32
-				mfi.merge = func(dst, src pointer) {
-					// NOTE: toInt32Ptr is not defined (see pointer_reflect.go).
-					/*
-						sfpp := src.toInt32Ptr()
-						if *sfpp != nil {
-							dfpp := dst.toInt32Ptr()
-							if *dfpp == nil {
-								*dfpp = Int32(**sfpp)
-							} else {
-								**dfpp = **sfpp
-							}
-						}
-					*/
-					sfp := src.getInt32Ptr()
-					if sfp != nil {
-						dfp := dst.getInt32Ptr()
-						if dfp == nil {
-							dst.setInt32Ptr(*sfp)
-						} else {
-							*dfp = *sfp
-						}
-					}
-				}
-			default: // E.g., int32
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toInt32(); v != 0 {
-						*dst.toInt32() = v
-					}
-				}
-			}
-		case reflect.Int64:
-			switch {
-			case isSlice: // E.g., []int64
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toInt64Slice()
-					if *sfsp != nil {
-						dfsp := dst.toInt64Slice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []int64{}
-						}
-					}
-				}
-			case isPointer: // E.g., *int64
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toInt64Ptr()
-					if *sfpp != nil {
-						dfpp := dst.toInt64Ptr()
-						if *dfpp == nil {
-							*dfpp = Int64(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., int64
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toInt64(); v != 0 {
-						*dst.toInt64() = v
-					}
-				}
-			}
-		case reflect.Uint32:
-			switch {
-			case isSlice: // E.g., []uint32
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toUint32Slice()
-					if *sfsp != nil {
-						dfsp := dst.toUint32Slice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []uint32{}
-						}
-					}
-				}
-			case isPointer: // E.g., *uint32
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toUint32Ptr()
-					if *sfpp != nil {
-						dfpp := dst.toUint32Ptr()
-						if *dfpp == nil {
-							*dfpp = Uint32(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., uint32
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toUint32(); v != 0 {
-						*dst.toUint32() = v
-					}
-				}
-			}
-		case reflect.Uint64:
-			switch {
-			case isSlice: // E.g., []uint64
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toUint64Slice()
-					if *sfsp != nil {
-						dfsp := dst.toUint64Slice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []uint64{}
-						}
-					}
-				}
-			case isPointer: // E.g., *uint64
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toUint64Ptr()
-					if *sfpp != nil {
-						dfpp := dst.toUint64Ptr()
-						if *dfpp == nil {
-							*dfpp = Uint64(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., uint64
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toUint64(); v != 0 {
-						*dst.toUint64() = v
-					}
-				}
-			}
-		case reflect.Float32:
-			switch {
-			case isSlice: // E.g., []float32
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toFloat32Slice()
-					if *sfsp != nil {
-						dfsp := dst.toFloat32Slice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []float32{}
-						}
-					}
-				}
-			case isPointer: // E.g., *float32
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toFloat32Ptr()
-					if *sfpp != nil {
-						dfpp := dst.toFloat32Ptr()
-						if *dfpp == nil {
-							*dfpp = Float32(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., float32
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toFloat32(); v != 0 {
-						*dst.toFloat32() = v
-					}
-				}
-			}
-		case reflect.Float64:
-			switch {
-			case isSlice: // E.g., []float64
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toFloat64Slice()
-					if *sfsp != nil {
-						dfsp := dst.toFloat64Slice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []float64{}
-						}
-					}
-				}
-			case isPointer: // E.g., *float64
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toFloat64Ptr()
-					if *sfpp != nil {
-						dfpp := dst.toFloat64Ptr()
-						if *dfpp == nil {
-							*dfpp = Float64(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., float64
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toFloat64(); v != 0 {
-						*dst.toFloat64() = v
-					}
-				}
-			}
-		case reflect.Bool:
-			switch {
-			case isSlice: // E.g., []bool
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toBoolSlice()
-					if *sfsp != nil {
-						dfsp := dst.toBoolSlice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []bool{}
-						}
-					}
-				}
-			case isPointer: // E.g., *bool
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toBoolPtr()
-					if *sfpp != nil {
-						dfpp := dst.toBoolPtr()
-						if *dfpp == nil {
-							*dfpp = Bool(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., bool
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toBool(); v {
-						*dst.toBool() = v
-					}
-				}
-			}
-		case reflect.String:
-			switch {
-			case isSlice: // E.g., []string
-				mfi.merge = func(dst, src pointer) {
-					sfsp := src.toStringSlice()
-					if *sfsp != nil {
-						dfsp := dst.toStringSlice()
-						*dfsp = append(*dfsp, *sfsp...)
-						if *dfsp == nil {
-							*dfsp = []string{}
-						}
-					}
-				}
-			case isPointer: // E.g., *string
-				mfi.merge = func(dst, src pointer) {
-					sfpp := src.toStringPtr()
-					if *sfpp != nil {
-						dfpp := dst.toStringPtr()
-						if *dfpp == nil {
-							*dfpp = String(**sfpp)
-						} else {
-							**dfpp = **sfpp
-						}
-					}
-				}
-			default: // E.g., string
-				mfi.merge = func(dst, src pointer) {
-					if v := *src.toString(); v != "" {
-						*dst.toString() = v
-					}
-				}
-			}
-		case reflect.Slice:
-			isProto3 := props.Prop[i].proto3
-			switch {
-			case isPointer:
-				panic("bad pointer in byte slice case in " + tf.Name())
-			case tf.Elem().Kind() != reflect.Uint8:
-				panic("bad element kind in byte slice case in " + tf.Name())
-			case isSlice: // E.g., [][]byte
-				mfi.merge = func(dst, src pointer) {
-					sbsp := src.toBytesSlice()
-					if *sbsp != nil {
-						dbsp := dst.toBytesSlice()
-						for _, sb := range *sbsp {
-							if sb == nil {
-								*dbsp = append(*dbsp, nil)
-							} else {
-								*dbsp = append(*dbsp, append([]byte{}, sb...))
-							}
-						}
-						if *dbsp == nil {
-							*dbsp = [][]byte{}
-						}
-					}
-				}
-			default: // E.g., []byte
-				mfi.merge = func(dst, src pointer) {
-					sbp := src.toBytes()
-					if *sbp != nil {
-						dbp := dst.toBytes()
-						if !isProto3 || len(*sbp) > 0 {
-							*dbp = append([]byte{}, *sbp...)
-						}
-					}
-				}
-			}
-		case reflect.Struct:
-			switch {
-			case !isPointer:
-				panic(fmt.Sprintf("message field %s without pointer", tf))
-			case isSlice: // E.g., []*pb.T
-				mi := getMergeInfo(tf)
-				mfi.merge = func(dst, src pointer) {
-					sps := src.getPointerSlice()
-					if sps != nil {
-						dps := dst.getPointerSlice()
-						for _, sp := range sps {
-							var dp pointer
-							if !sp.isNil() {
-								dp = valToPointer(reflect.New(tf))
-								mi.merge(dp, sp)
-							}
-							dps = append(dps, dp)
-						}
-						if dps == nil {
-							dps = []pointer{}
-						}
-						dst.setPointerSlice(dps)
-					}
-				}
-			default: // E.g., *pb.T
-				mi := getMergeInfo(tf)
-				mfi.merge = func(dst, src pointer) {
-					sp := src.getPointer()
-					if !sp.isNil() {
-						dp := dst.getPointer()
-						if dp.isNil() {
-							dp = valToPointer(reflect.New(tf))
-							dst.setPointer(dp)
-						}
-						mi.merge(dp, sp)
-					}
-				}
-			}
-		case reflect.Map:
-			switch {
-			case isPointer || isSlice:
-				panic("bad pointer or slice in map case in " + tf.Name())
-			default: // E.g., map[K]V
-				mfi.merge = func(dst, src pointer) {
-					sm := src.asPointerTo(tf).Elem()
-					if sm.Len() == 0 {
-						return
-					}
-					dm := dst.asPointerTo(tf).Elem()
-					if dm.IsNil() {
-						dm.Set(reflect.MakeMap(tf))
-					}
-
-					switch tf.Elem().Kind() {
-					case reflect.Ptr: // Proto struct (e.g., *T)
-						for _, key := range sm.MapKeys() {
-							val := sm.MapIndex(key)
-							val = reflect.ValueOf(Clone(val.Interface().(Message)))
-							dm.SetMapIndex(key, val)
-						}
-					case reflect.Slice: // E.g. Bytes type (e.g., []byte)
-						for _, key := range sm.MapKeys() {
-							val := sm.MapIndex(key)
-							val = reflect.ValueOf(append([]byte{}, val.Bytes()...))
-							dm.SetMapIndex(key, val)
-						}
-					default: // Basic type (e.g., string)
-						for _, key := range sm.MapKeys() {
-							val := sm.MapIndex(key)
-							dm.SetMapIndex(key, val)
-						}
-					}
-				}
-			}
-		case reflect.Interface:
-			// Must be oneof field.
-			switch {
-			case isPointer || isSlice:
-				panic("bad pointer or slice in interface case in " + tf.Name())
-			default: // E.g., interface{}
-				// TODO: Make this faster?
-				mfi.merge = func(dst, src pointer) {
-					su := src.asPointerTo(tf).Elem()
-					if !su.IsNil() {
-						du := dst.asPointerTo(tf).Elem()
-						typ := su.Elem().Type()
-						if du.IsNil() || du.Elem().Type() != typ {
-							du.Set(reflect.New(typ.Elem())) // Initialize interface if empty
-						}
-						sv := su.Elem().Elem().Field(0)
-						if sv.Kind() == reflect.Ptr && sv.IsNil() {
-							return
-						}
-						dv := du.Elem().Elem().Field(0)
-						if dv.Kind() == reflect.Ptr && dv.IsNil() {
-							dv.Set(reflect.New(sv.Type().Elem())) // Initialize proto message if empty
-						}
-						switch sv.Type().Kind() {
-						case reflect.Ptr: // Proto struct (e.g., *T)
-							Merge(dv.Interface().(Message), sv.Interface().(Message))
-						case reflect.Slice: // E.g. Bytes type (e.g., []byte)
-							dv.Set(reflect.ValueOf(append([]byte{}, sv.Bytes()...)))
-						default: // Basic type (e.g., string)
-							dv.Set(sv)
-						}
-					}
-				}
-			}
-		default:
-			panic(fmt.Sprintf("merger not found for type:%s", tf))
-		}
-		mi.fields = append(mi.fields, mfi)
-	}
-
-	mi.unrecognized = invalidField
-	if f, ok := t.FieldByName("XXX_unrecognized"); ok {
-		if f.Type != reflect.TypeOf([]byte{}) {
-			panic("expected XXX_unrecognized to be of type []byte")
-		}
-		mi.unrecognized = toField(&f)
-	}
-
-	atomic.StoreInt32(&mi.initialized, 1)
-}
diff --git a/vendor/github.com/golang/protobuf/proto/table_unmarshal.go b/vendor/github.com/golang/protobuf/proto/table_unmarshal.go
deleted file mode 100644
index acee2fc..0000000
--- a/vendor/github.com/golang/protobuf/proto/table_unmarshal.go
+++ /dev/null
@@ -1,2053 +0,0 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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 proto
-
-import (
-	"errors"
-	"fmt"
-	"io"
-	"math"
-	"reflect"
-	"strconv"
-	"strings"
-	"sync"
-	"sync/atomic"
-	"unicode/utf8"
-)
-
-// Unmarshal is the entry point from the generated .pb.go files.
-// This function is not intended to be used by non-generated code.
-// This function is not subject to any compatibility guarantee.
-// msg contains a pointer to a protocol buffer struct.
-// b is the data to be unmarshaled into the protocol buffer.
-// a is a pointer to a place to store cached unmarshal information.
-func (a *InternalMessageInfo) Unmarshal(msg Message, b []byte) error {
-	// Load the unmarshal information for this message type.
-	// The atomic load ensures memory consistency.
-	u := atomicLoadUnmarshalInfo(&a.unmarshal)
-	if u == nil {
-		// Slow path: find unmarshal info for msg, update a with it.
-		u = getUnmarshalInfo(reflect.TypeOf(msg).Elem())
-		atomicStoreUnmarshalInfo(&a.unmarshal, u)
-	}
-	// Then do the unmarshaling.
-	err := u.unmarshal(toPointer(&msg), b)
-	return err
-}
-
-type unmarshalInfo struct {
-	typ reflect.Type // type of the protobuf struct
-
-	// 0 = only typ field is initialized
-	// 1 = completely initialized
-	initialized     int32
-	lock            sync.Mutex                    // prevents double initialization
-	dense           []unmarshalFieldInfo          // fields indexed by tag #
-	sparse          map[uint64]unmarshalFieldInfo // fields indexed by tag #
-	reqFields       []string                      // names of required fields
-	reqMask         uint64                        // 1<<len(reqFields)-1
-	unrecognized    field                         // offset of []byte to put unrecognized data (or invalidField if we should throw it away)
-	extensions      field                         // offset of extensions field (of type proto.XXX_InternalExtensions), or invalidField if it does not exist
-	oldExtensions   field                         // offset of old-form extensions field (of type map[int]Extension)
-	extensionRanges []ExtensionRange              // if non-nil, implies extensions field is valid
-	isMessageSet    bool                          // if true, implies extensions field is valid
-}
-
-// An unmarshaler takes a stream of bytes and a pointer to a field of a message.
-// It decodes the field, stores it at f, and returns the unused bytes.
-// w is the wire encoding.
-// b is the data after the tag and wire encoding have been read.
-type unmarshaler func(b []byte, f pointer, w int) ([]byte, error)
-
-type unmarshalFieldInfo struct {
-	// location of the field in the proto message structure.
-	field field
-
-	// function to unmarshal the data for the field.
-	unmarshal unmarshaler
-
-	// if a required field, contains a single set bit at this field's index in the required field list.
-	reqMask uint64
-
-	name string // name of the field, for error reporting
-}
-
-var (
-	unmarshalInfoMap  = map[reflect.Type]*unmarshalInfo{}
-	unmarshalInfoLock sync.Mutex
-)
-
-// getUnmarshalInfo returns the data structure which can be
-// subsequently used to unmarshal a message of the given type.
-// t is the type of the message (note: not pointer to message).
-func getUnmarshalInfo(t reflect.Type) *unmarshalInfo {
-	// It would be correct to return a new unmarshalInfo
-	// unconditionally. We would end up allocating one
-	// per occurrence of that type as a message or submessage.
-	// We use a cache here just to reduce memory usage.
-	unmarshalInfoLock.Lock()
-	defer unmarshalInfoLock.Unlock()
-	u := unmarshalInfoMap[t]
-	if u == nil {
-		u = &unmarshalInfo{typ: t}
-		// Note: we just set the type here. The rest of the fields
-		// will be initialized on first use.
-		unmarshalInfoMap[t] = u
-	}
-	return u
-}
-
-// unmarshal does the main work of unmarshaling a message.
-// u provides type information used to unmarshal the message.
-// m is a pointer to a protocol buffer message.
-// b is a byte stream to unmarshal into m.
-// This is top routine used when recursively unmarshaling submessages.
-func (u *unmarshalInfo) unmarshal(m pointer, b []byte) error {
-	if atomic.LoadInt32(&u.initialized) == 0 {
-		u.computeUnmarshalInfo()
-	}
-	if u.isMessageSet {
-		return unmarshalMessageSet(b, m.offset(u.extensions).toExtensions())
-	}
-	var reqMask uint64 // bitmask of required fields we've seen.
-	var errLater error
-	for len(b) > 0 {
-		// Read tag and wire type.
-		// Special case 1 and 2 byte varints.
-		var x uint64
-		if b[0] < 128 {
-			x = uint64(b[0])
-			b = b[1:]
-		} else if len(b) >= 2 && b[1] < 128 {
-			x = uint64(b[0]&0x7f) + uint64(b[1])<<7
-			b = b[2:]
-		} else {
-			var n int
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-		}
-		tag := x >> 3
-		wire := int(x) & 7
-
-		// Dispatch on the tag to one of the unmarshal* functions below.
-		var f unmarshalFieldInfo
-		if tag < uint64(len(u.dense)) {
-			f = u.dense[tag]
-		} else {
-			f = u.sparse[tag]
-		}
-		if fn := f.unmarshal; fn != nil {
-			var err error
-			b, err = fn(b, m.offset(f.field), wire)
-			if err == nil {
-				reqMask |= f.reqMask
-				continue
-			}
-			if r, ok := err.(*RequiredNotSetError); ok {
-				// Remember this error, but keep parsing. We need to produce
-				// a full parse even if a required field is missing.
-				if errLater == nil {
-					errLater = r
-				}
-				reqMask |= f.reqMask
-				continue
-			}
-			if err != errInternalBadWireType {
-				if err == errInvalidUTF8 {
-					if errLater == nil {
-						fullName := revProtoTypes[reflect.PtrTo(u.typ)] + "." + f.name
-						errLater = &invalidUTF8Error{fullName}
-					}
-					continue
-				}
-				return err
-			}
-			// Fragments with bad wire type are treated as unknown fields.
-		}
-
-		// Unknown tag.
-		if !u.unrecognized.IsValid() {
-			// Don't keep unrecognized data; just skip it.
-			var err error
-			b, err = skipField(b, wire)
-			if err != nil {
-				return err
-			}
-			continue
-		}
-		// Keep unrecognized data around.
-		// maybe in extensions, maybe in the unrecognized field.
-		z := m.offset(u.unrecognized).toBytes()
-		var emap map[int32]Extension
-		var e Extension
-		for _, r := range u.extensionRanges {
-			if uint64(r.Start) <= tag && tag <= uint64(r.End) {
-				if u.extensions.IsValid() {
-					mp := m.offset(u.extensions).toExtensions()
-					emap = mp.extensionsWrite()
-					e = emap[int32(tag)]
-					z = &e.enc
-					break
-				}
-				if u.oldExtensions.IsValid() {
-					p := m.offset(u.oldExtensions).toOldExtensions()
-					emap = *p
-					if emap == nil {
-						emap = map[int32]Extension{}
-						*p = emap
-					}
-					e = emap[int32(tag)]
-					z = &e.enc
-					break
-				}
-				panic("no extensions field available")
-			}
-		}
-
-		// Use wire type to skip data.
-		var err error
-		b0 := b
-		b, err = skipField(b, wire)
-		if err != nil {
-			return err
-		}
-		*z = encodeVarint(*z, tag<<3|uint64(wire))
-		*z = append(*z, b0[:len(b0)-len(b)]...)
-
-		if emap != nil {
-			emap[int32(tag)] = e
-		}
-	}
-	if reqMask != u.reqMask && errLater == nil {
-		// A required field of this message is missing.
-		for _, n := range u.reqFields {
-			if reqMask&1 == 0 {
-				errLater = &RequiredNotSetError{n}
-			}
-			reqMask >>= 1
-		}
-	}
-	return errLater
-}
-
-// computeUnmarshalInfo fills in u with information for use
-// in unmarshaling protocol buffers of type u.typ.
-func (u *unmarshalInfo) computeUnmarshalInfo() {
-	u.lock.Lock()
-	defer u.lock.Unlock()
-	if u.initialized != 0 {
-		return
-	}
-	t := u.typ
-	n := t.NumField()
-
-	// Set up the "not found" value for the unrecognized byte buffer.
-	// This is the default for proto3.
-	u.unrecognized = invalidField
-	u.extensions = invalidField
-	u.oldExtensions = invalidField
-
-	// List of the generated type and offset for each oneof field.
-	type oneofField struct {
-		ityp  reflect.Type // interface type of oneof field
-		field field        // offset in containing message
-	}
-	var oneofFields []oneofField
-
-	for i := 0; i < n; i++ {
-		f := t.Field(i)
-		if f.Name == "XXX_unrecognized" {
-			// The byte slice used to hold unrecognized input is special.
-			if f.Type != reflect.TypeOf(([]byte)(nil)) {
-				panic("bad type for XXX_unrecognized field: " + f.Type.Name())
-			}
-			u.unrecognized = toField(&f)
-			continue
-		}
-		if f.Name == "XXX_InternalExtensions" {
-			// Ditto here.
-			if f.Type != reflect.TypeOf(XXX_InternalExtensions{}) {
-				panic("bad type for XXX_InternalExtensions field: " + f.Type.Name())
-			}
-			u.extensions = toField(&f)
-			if f.Tag.Get("protobuf_messageset") == "1" {
-				u.isMessageSet = true
-			}
-			continue
-		}
-		if f.Name == "XXX_extensions" {
-			// An older form of the extensions field.
-			if f.Type != reflect.TypeOf((map[int32]Extension)(nil)) {
-				panic("bad type for XXX_extensions field: " + f.Type.Name())
-			}
-			u.oldExtensions = toField(&f)
-			continue
-		}
-		if f.Name == "XXX_NoUnkeyedLiteral" || f.Name == "XXX_sizecache" {
-			continue
-		}
-
-		oneof := f.Tag.Get("protobuf_oneof")
-		if oneof != "" {
-			oneofFields = append(oneofFields, oneofField{f.Type, toField(&f)})
-			// The rest of oneof processing happens below.
-			continue
-		}
-
-		tags := f.Tag.Get("protobuf")
-		tagArray := strings.Split(tags, ",")
-		if len(tagArray) < 2 {
-			panic("protobuf tag not enough fields in " + t.Name() + "." + f.Name + ": " + tags)
-		}
-		tag, err := strconv.Atoi(tagArray[1])
-		if err != nil {
-			panic("protobuf tag field not an integer: " + tagArray[1])
-		}
-
-		name := ""
-		for _, tag := range tagArray[3:] {
-			if strings.HasPrefix(tag, "name=") {
-				name = tag[5:]
-			}
-		}
-
-		// Extract unmarshaling function from the field (its type and tags).
-		unmarshal := fieldUnmarshaler(&f)
-
-		// Required field?
-		var reqMask uint64
-		if tagArray[2] == "req" {
-			bit := len(u.reqFields)
-			u.reqFields = append(u.reqFields, name)
-			reqMask = uint64(1) << uint(bit)
-			// TODO: if we have more than 64 required fields, we end up
-			// not verifying that all required fields are present.
-			// Fix this, perhaps using a count of required fields?
-		}
-
-		// Store the info in the correct slot in the message.
-		u.setTag(tag, toField(&f), unmarshal, reqMask, name)
-	}
-
-	// Find any types associated with oneof fields.
-	var oneofImplementers []interface{}
-	switch m := reflect.Zero(reflect.PtrTo(t)).Interface().(type) {
-	case oneofFuncsIface:
-		_, _, _, oneofImplementers = m.XXX_OneofFuncs()
-	case oneofWrappersIface:
-		oneofImplementers = m.XXX_OneofWrappers()
-	}
-	for _, v := range oneofImplementers {
-		tptr := reflect.TypeOf(v) // *Msg_X
-		typ := tptr.Elem()        // Msg_X
-
-		f := typ.Field(0) // oneof implementers have one field
-		baseUnmarshal := fieldUnmarshaler(&f)
-		tags := strings.Split(f.Tag.Get("protobuf"), ",")
-		fieldNum, err := strconv.Atoi(tags[1])
-		if err != nil {
-			panic("protobuf tag field not an integer: " + tags[1])
-		}
-		var name string
-		for _, tag := range tags {
-			if strings.HasPrefix(tag, "name=") {
-				name = strings.TrimPrefix(tag, "name=")
-				break
-			}
-		}
-
-		// Find the oneof field that this struct implements.
-		// Might take O(n^2) to process all of the oneofs, but who cares.
-		for _, of := range oneofFields {
-			if tptr.Implements(of.ityp) {
-				// We have found the corresponding interface for this struct.
-				// That lets us know where this struct should be stored
-				// when we encounter it during unmarshaling.
-				unmarshal := makeUnmarshalOneof(typ, of.ityp, baseUnmarshal)
-				u.setTag(fieldNum, of.field, unmarshal, 0, name)
-			}
-		}
-
-	}
-
-	// Get extension ranges, if any.
-	fn := reflect.Zero(reflect.PtrTo(t)).MethodByName("ExtensionRangeArray")
-	if fn.IsValid() {
-		if !u.extensions.IsValid() && !u.oldExtensions.IsValid() {
-			panic("a message with extensions, but no extensions field in " + t.Name())
-		}
-		u.extensionRanges = fn.Call(nil)[0].Interface().([]ExtensionRange)
-	}
-
-	// Explicitly disallow tag 0. This will ensure we flag an error
-	// when decoding a buffer of all zeros. Without this code, we
-	// would decode and skip an all-zero buffer of even length.
-	// [0 0] is [tag=0/wiretype=varint varint-encoded-0].
-	u.setTag(0, zeroField, func(b []byte, f pointer, w int) ([]byte, error) {
-		return nil, fmt.Errorf("proto: %s: illegal tag 0 (wire type %d)", t, w)
-	}, 0, "")
-
-	// Set mask for required field check.
-	u.reqMask = uint64(1)<<uint(len(u.reqFields)) - 1
-
-	atomic.StoreInt32(&u.initialized, 1)
-}
-
-// setTag stores the unmarshal information for the given tag.
-// tag = tag # for field
-// field/unmarshal = unmarshal info for that field.
-// reqMask = if required, bitmask for field position in required field list. 0 otherwise.
-// name = short name of the field.
-func (u *unmarshalInfo) setTag(tag int, field field, unmarshal unmarshaler, reqMask uint64, name string) {
-	i := unmarshalFieldInfo{field: field, unmarshal: unmarshal, reqMask: reqMask, name: name}
-	n := u.typ.NumField()
-	if tag >= 0 && (tag < 16 || tag < 2*n) { // TODO: what are the right numbers here?
-		for len(u.dense) <= tag {
-			u.dense = append(u.dense, unmarshalFieldInfo{})
-		}
-		u.dense[tag] = i
-		return
-	}
-	if u.sparse == nil {
-		u.sparse = map[uint64]unmarshalFieldInfo{}
-	}
-	u.sparse[uint64(tag)] = i
-}
-
-// fieldUnmarshaler returns an unmarshaler for the given field.
-func fieldUnmarshaler(f *reflect.StructField) unmarshaler {
-	if f.Type.Kind() == reflect.Map {
-		return makeUnmarshalMap(f)
-	}
-	return typeUnmarshaler(f.Type, f.Tag.Get("protobuf"))
-}
-
-// typeUnmarshaler returns an unmarshaler for the given field type / field tag pair.
-func typeUnmarshaler(t reflect.Type, tags string) unmarshaler {
-	tagArray := strings.Split(tags, ",")
-	encoding := tagArray[0]
-	name := "unknown"
-	proto3 := false
-	validateUTF8 := true
-	for _, tag := range tagArray[3:] {
-		if strings.HasPrefix(tag, "name=") {
-			name = tag[5:]
-		}
-		if tag == "proto3" {
-			proto3 = true
-		}
-	}
-	validateUTF8 = validateUTF8 && proto3
-
-	// Figure out packaging (pointer, slice, or both)
-	slice := false
-	pointer := false
-	if t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 {
-		slice = true
-		t = t.Elem()
-	}
-	if t.Kind() == reflect.Ptr {
-		pointer = true
-		t = t.Elem()
-	}
-
-	// We'll never have both pointer and slice for basic types.
-	if pointer && slice && t.Kind() != reflect.Struct {
-		panic("both pointer and slice for basic type in " + t.Name())
-	}
-
-	switch t.Kind() {
-	case reflect.Bool:
-		if pointer {
-			return unmarshalBoolPtr
-		}
-		if slice {
-			return unmarshalBoolSlice
-		}
-		return unmarshalBoolValue
-	case reflect.Int32:
-		switch encoding {
-		case "fixed32":
-			if pointer {
-				return unmarshalFixedS32Ptr
-			}
-			if slice {
-				return unmarshalFixedS32Slice
-			}
-			return unmarshalFixedS32Value
-		case "varint":
-			// this could be int32 or enum
-			if pointer {
-				return unmarshalInt32Ptr
-			}
-			if slice {
-				return unmarshalInt32Slice
-			}
-			return unmarshalInt32Value
-		case "zigzag32":
-			if pointer {
-				return unmarshalSint32Ptr
-			}
-			if slice {
-				return unmarshalSint32Slice
-			}
-			return unmarshalSint32Value
-		}
-	case reflect.Int64:
-		switch encoding {
-		case "fixed64":
-			if pointer {
-				return unmarshalFixedS64Ptr
-			}
-			if slice {
-				return unmarshalFixedS64Slice
-			}
-			return unmarshalFixedS64Value
-		case "varint":
-			if pointer {
-				return unmarshalInt64Ptr
-			}
-			if slice {
-				return unmarshalInt64Slice
-			}
-			return unmarshalInt64Value
-		case "zigzag64":
-			if pointer {
-				return unmarshalSint64Ptr
-			}
-			if slice {
-				return unmarshalSint64Slice
-			}
-			return unmarshalSint64Value
-		}
-	case reflect.Uint32:
-		switch encoding {
-		case "fixed32":
-			if pointer {
-				return unmarshalFixed32Ptr
-			}
-			if slice {
-				return unmarshalFixed32Slice
-			}
-			return unmarshalFixed32Value
-		case "varint":
-			if pointer {
-				return unmarshalUint32Ptr
-			}
-			if slice {
-				return unmarshalUint32Slice
-			}
-			return unmarshalUint32Value
-		}
-	case reflect.Uint64:
-		switch encoding {
-		case "fixed64":
-			if pointer {
-				return unmarshalFixed64Ptr
-			}
-			if slice {
-				return unmarshalFixed64Slice
-			}
-			return unmarshalFixed64Value
-		case "varint":
-			if pointer {
-				return unmarshalUint64Ptr
-			}
-			if slice {
-				return unmarshalUint64Slice
-			}
-			return unmarshalUint64Value
-		}
-	case reflect.Float32:
-		if pointer {
-			return unmarshalFloat32Ptr
-		}
-		if slice {
-			return unmarshalFloat32Slice
-		}
-		return unmarshalFloat32Value
-	case reflect.Float64:
-		if pointer {
-			return unmarshalFloat64Ptr
-		}
-		if slice {
-			return unmarshalFloat64Slice
-		}
-		return unmarshalFloat64Value
-	case reflect.Map:
-		panic("map type in typeUnmarshaler in " + t.Name())
-	case reflect.Slice:
-		if pointer {
-			panic("bad pointer in slice case in " + t.Name())
-		}
-		if slice {
-			return unmarshalBytesSlice
-		}
-		return unmarshalBytesValue
-	case reflect.String:
-		if validateUTF8 {
-			if pointer {
-				return unmarshalUTF8StringPtr
-			}
-			if slice {
-				return unmarshalUTF8StringSlice
-			}
-			return unmarshalUTF8StringValue
-		}
-		if pointer {
-			return unmarshalStringPtr
-		}
-		if slice {
-			return unmarshalStringSlice
-		}
-		return unmarshalStringValue
-	case reflect.Struct:
-		// message or group field
-		if !pointer {
-			panic(fmt.Sprintf("message/group field %s:%s without pointer", t, encoding))
-		}
-		switch encoding {
-		case "bytes":
-			if slice {
-				return makeUnmarshalMessageSlicePtr(getUnmarshalInfo(t), name)
-			}
-			return makeUnmarshalMessagePtr(getUnmarshalInfo(t), name)
-		case "group":
-			if slice {
-				return makeUnmarshalGroupSlicePtr(getUnmarshalInfo(t), name)
-			}
-			return makeUnmarshalGroupPtr(getUnmarshalInfo(t), name)
-		}
-	}
-	panic(fmt.Sprintf("unmarshaler not found type:%s encoding:%s", t, encoding))
-}
-
-// Below are all the unmarshalers for individual fields of various types.
-
-func unmarshalInt64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x)
-	*f.toInt64() = v
-	return b, nil
-}
-
-func unmarshalInt64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x)
-	*f.toInt64Ptr() = &v
-	return b, nil
-}
-
-func unmarshalInt64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := int64(x)
-			s := f.toInt64Slice()
-			*s = append(*s, v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x)
-	s := f.toInt64Slice()
-	*s = append(*s, v)
-	return b, nil
-}
-
-func unmarshalSint64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x>>1) ^ int64(x)<<63>>63
-	*f.toInt64() = v
-	return b, nil
-}
-
-func unmarshalSint64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x>>1) ^ int64(x)<<63>>63
-	*f.toInt64Ptr() = &v
-	return b, nil
-}
-
-func unmarshalSint64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := int64(x>>1) ^ int64(x)<<63>>63
-			s := f.toInt64Slice()
-			*s = append(*s, v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int64(x>>1) ^ int64(x)<<63>>63
-	s := f.toInt64Slice()
-	*s = append(*s, v)
-	return b, nil
-}
-
-func unmarshalUint64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint64(x)
-	*f.toUint64() = v
-	return b, nil
-}
-
-func unmarshalUint64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint64(x)
-	*f.toUint64Ptr() = &v
-	return b, nil
-}
-
-func unmarshalUint64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := uint64(x)
-			s := f.toUint64Slice()
-			*s = append(*s, v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint64(x)
-	s := f.toUint64Slice()
-	*s = append(*s, v)
-	return b, nil
-}
-
-func unmarshalInt32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x)
-	*f.toInt32() = v
-	return b, nil
-}
-
-func unmarshalInt32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x)
-	f.setInt32Ptr(v)
-	return b, nil
-}
-
-func unmarshalInt32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := int32(x)
-			f.appendInt32Slice(v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x)
-	f.appendInt32Slice(v)
-	return b, nil
-}
-
-func unmarshalSint32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x>>1) ^ int32(x)<<31>>31
-	*f.toInt32() = v
-	return b, nil
-}
-
-func unmarshalSint32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x>>1) ^ int32(x)<<31>>31
-	f.setInt32Ptr(v)
-	return b, nil
-}
-
-func unmarshalSint32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := int32(x>>1) ^ int32(x)<<31>>31
-			f.appendInt32Slice(v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := int32(x>>1) ^ int32(x)<<31>>31
-	f.appendInt32Slice(v)
-	return b, nil
-}
-
-func unmarshalUint32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint32(x)
-	*f.toUint32() = v
-	return b, nil
-}
-
-func unmarshalUint32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint32(x)
-	*f.toUint32Ptr() = &v
-	return b, nil
-}
-
-func unmarshalUint32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			b = b[n:]
-			v := uint32(x)
-			s := f.toUint32Slice()
-			*s = append(*s, v)
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	v := uint32(x)
-	s := f.toUint32Slice()
-	*s = append(*s, v)
-	return b, nil
-}
-
-func unmarshalFixed64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
-	*f.toUint64() = v
-	return b[8:], nil
-}
-
-func unmarshalFixed64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
-	*f.toUint64Ptr() = &v
-	return b[8:], nil
-}
-
-func unmarshalFixed64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 8 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
-			s := f.toUint64Slice()
-			*s = append(*s, v)
-			b = b[8:]
-		}
-		return res, nil
-	}
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
-	s := f.toUint64Slice()
-	*s = append(*s, v)
-	return b[8:], nil
-}
-
-func unmarshalFixedS64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56
-	*f.toInt64() = v
-	return b[8:], nil
-}
-
-func unmarshalFixedS64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56
-	*f.toInt64Ptr() = &v
-	return b[8:], nil
-}
-
-func unmarshalFixedS64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 8 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56
-			s := f.toInt64Slice()
-			*s = append(*s, v)
-			b = b[8:]
-		}
-		return res, nil
-	}
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56
-	s := f.toInt64Slice()
-	*s = append(*s, v)
-	return b[8:], nil
-}
-
-func unmarshalFixed32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24
-	*f.toUint32() = v
-	return b[4:], nil
-}
-
-func unmarshalFixed32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24
-	*f.toUint32Ptr() = &v
-	return b[4:], nil
-}
-
-func unmarshalFixed32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 4 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24
-			s := f.toUint32Slice()
-			*s = append(*s, v)
-			b = b[4:]
-		}
-		return res, nil
-	}
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24
-	s := f.toUint32Slice()
-	*s = append(*s, v)
-	return b[4:], nil
-}
-
-func unmarshalFixedS32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24
-	*f.toInt32() = v
-	return b[4:], nil
-}
-
-func unmarshalFixedS32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24
-	f.setInt32Ptr(v)
-	return b[4:], nil
-}
-
-func unmarshalFixedS32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 4 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24
-			f.appendInt32Slice(v)
-			b = b[4:]
-		}
-		return res, nil
-	}
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24
-	f.appendInt32Slice(v)
-	return b[4:], nil
-}
-
-func unmarshalBoolValue(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	// Note: any length varint is allowed, even though any sane
-	// encoder will use one byte.
-	// See https://github.com/golang/protobuf/issues/76
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	// TODO: check if x>1? Tests seem to indicate no.
-	v := x != 0
-	*f.toBool() = v
-	return b[n:], nil
-}
-
-func unmarshalBoolPtr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := x != 0
-	*f.toBoolPtr() = &v
-	return b[n:], nil
-}
-
-func unmarshalBoolSlice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			x, n = decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := x != 0
-			s := f.toBoolSlice()
-			*s = append(*s, v)
-			b = b[n:]
-		}
-		return res, nil
-	}
-	if w != WireVarint {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := x != 0
-	s := f.toBoolSlice()
-	*s = append(*s, v)
-	return b[n:], nil
-}
-
-func unmarshalFloat64Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56)
-	*f.toFloat64() = v
-	return b[8:], nil
-}
-
-func unmarshalFloat64Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56)
-	*f.toFloat64Ptr() = &v
-	return b[8:], nil
-}
-
-func unmarshalFloat64Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 8 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56)
-			s := f.toFloat64Slice()
-			*s = append(*s, v)
-			b = b[8:]
-		}
-		return res, nil
-	}
-	if w != WireFixed64 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 8 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56)
-	s := f.toFloat64Slice()
-	*s = append(*s, v)
-	return b[8:], nil
-}
-
-func unmarshalFloat32Value(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24)
-	*f.toFloat32() = v
-	return b[4:], nil
-}
-
-func unmarshalFloat32Ptr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24)
-	*f.toFloat32Ptr() = &v
-	return b[4:], nil
-}
-
-func unmarshalFloat32Slice(b []byte, f pointer, w int) ([]byte, error) {
-	if w == WireBytes { // packed
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		res := b[x:]
-		b = b[:x]
-		for len(b) > 0 {
-			if len(b) < 4 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24)
-			s := f.toFloat32Slice()
-			*s = append(*s, v)
-			b = b[4:]
-		}
-		return res, nil
-	}
-	if w != WireFixed32 {
-		return b, errInternalBadWireType
-	}
-	if len(b) < 4 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24)
-	s := f.toFloat32Slice()
-	*s = append(*s, v)
-	return b[4:], nil
-}
-
-func unmarshalStringValue(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	*f.toString() = v
-	return b[x:], nil
-}
-
-func unmarshalStringPtr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	*f.toStringPtr() = &v
-	return b[x:], nil
-}
-
-func unmarshalStringSlice(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	s := f.toStringSlice()
-	*s = append(*s, v)
-	return b[x:], nil
-}
-
-func unmarshalUTF8StringValue(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	*f.toString() = v
-	if !utf8.ValidString(v) {
-		return b[x:], errInvalidUTF8
-	}
-	return b[x:], nil
-}
-
-func unmarshalUTF8StringPtr(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	*f.toStringPtr() = &v
-	if !utf8.ValidString(v) {
-		return b[x:], errInvalidUTF8
-	}
-	return b[x:], nil
-}
-
-func unmarshalUTF8StringSlice(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := string(b[:x])
-	s := f.toStringSlice()
-	*s = append(*s, v)
-	if !utf8.ValidString(v) {
-		return b[x:], errInvalidUTF8
-	}
-	return b[x:], nil
-}
-
-var emptyBuf [0]byte
-
-func unmarshalBytesValue(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	// The use of append here is a trick which avoids the zeroing
-	// that would be required if we used a make/copy pair.
-	// We append to emptyBuf instead of nil because we want
-	// a non-nil result even when the length is 0.
-	v := append(emptyBuf[:], b[:x]...)
-	*f.toBytes() = v
-	return b[x:], nil
-}
-
-func unmarshalBytesSlice(b []byte, f pointer, w int) ([]byte, error) {
-	if w != WireBytes {
-		return b, errInternalBadWireType
-	}
-	x, n := decodeVarint(b)
-	if n == 0 {
-		return nil, io.ErrUnexpectedEOF
-	}
-	b = b[n:]
-	if x > uint64(len(b)) {
-		return nil, io.ErrUnexpectedEOF
-	}
-	v := append(emptyBuf[:], b[:x]...)
-	s := f.toBytesSlice()
-	*s = append(*s, v)
-	return b[x:], nil
-}
-
-func makeUnmarshalMessagePtr(sub *unmarshalInfo, name string) unmarshaler {
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		if w != WireBytes {
-			return b, errInternalBadWireType
-		}
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		// First read the message field to see if something is there.
-		// The semantics of multiple submessages are weird.  Instead of
-		// the last one winning (as it is for all other fields), multiple
-		// submessages are merged.
-		v := f.getPointer()
-		if v.isNil() {
-			v = valToPointer(reflect.New(sub.typ))
-			f.setPointer(v)
-		}
-		err := sub.unmarshal(v, b[:x])
-		if err != nil {
-			if r, ok := err.(*RequiredNotSetError); ok {
-				r.field = name + "." + r.field
-			} else {
-				return nil, err
-			}
-		}
-		return b[x:], err
-	}
-}
-
-func makeUnmarshalMessageSlicePtr(sub *unmarshalInfo, name string) unmarshaler {
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		if w != WireBytes {
-			return b, errInternalBadWireType
-		}
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		v := valToPointer(reflect.New(sub.typ))
-		err := sub.unmarshal(v, b[:x])
-		if err != nil {
-			if r, ok := err.(*RequiredNotSetError); ok {
-				r.field = name + "." + r.field
-			} else {
-				return nil, err
-			}
-		}
-		f.appendPointer(v)
-		return b[x:], err
-	}
-}
-
-func makeUnmarshalGroupPtr(sub *unmarshalInfo, name string) unmarshaler {
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		if w != WireStartGroup {
-			return b, errInternalBadWireType
-		}
-		x, y := findEndGroup(b)
-		if x < 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		v := f.getPointer()
-		if v.isNil() {
-			v = valToPointer(reflect.New(sub.typ))
-			f.setPointer(v)
-		}
-		err := sub.unmarshal(v, b[:x])
-		if err != nil {
-			if r, ok := err.(*RequiredNotSetError); ok {
-				r.field = name + "." + r.field
-			} else {
-				return nil, err
-			}
-		}
-		return b[y:], err
-	}
-}
-
-func makeUnmarshalGroupSlicePtr(sub *unmarshalInfo, name string) unmarshaler {
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		if w != WireStartGroup {
-			return b, errInternalBadWireType
-		}
-		x, y := findEndGroup(b)
-		if x < 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		v := valToPointer(reflect.New(sub.typ))
-		err := sub.unmarshal(v, b[:x])
-		if err != nil {
-			if r, ok := err.(*RequiredNotSetError); ok {
-				r.field = name + "." + r.field
-			} else {
-				return nil, err
-			}
-		}
-		f.appendPointer(v)
-		return b[y:], err
-	}
-}
-
-func makeUnmarshalMap(f *reflect.StructField) unmarshaler {
-	t := f.Type
-	kt := t.Key()
-	vt := t.Elem()
-	unmarshalKey := typeUnmarshaler(kt, f.Tag.Get("protobuf_key"))
-	unmarshalVal := typeUnmarshaler(vt, f.Tag.Get("protobuf_val"))
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		// The map entry is a submessage. Figure out how big it is.
-		if w != WireBytes {
-			return nil, fmt.Errorf("proto: bad wiretype for map field: got %d want %d", w, WireBytes)
-		}
-		x, n := decodeVarint(b)
-		if n == 0 {
-			return nil, io.ErrUnexpectedEOF
-		}
-		b = b[n:]
-		if x > uint64(len(b)) {
-			return nil, io.ErrUnexpectedEOF
-		}
-		r := b[x:] // unused data to return
-		b = b[:x]  // data for map entry
-
-		// Note: we could use #keys * #values ~= 200 functions
-		// to do map decoding without reflection. Probably not worth it.
-		// Maps will be somewhat slow. Oh well.
-
-		// Read key and value from data.
-		var nerr nonFatal
-		k := reflect.New(kt)
-		v := reflect.New(vt)
-		for len(b) > 0 {
-			x, n := decodeVarint(b)
-			if n == 0 {
-				return nil, io.ErrUnexpectedEOF
-			}
-			wire := int(x) & 7
-			b = b[n:]
-
-			var err error
-			switch x >> 3 {
-			case 1:
-				b, err = unmarshalKey(b, valToPointer(k), wire)
-			case 2:
-				b, err = unmarshalVal(b, valToPointer(v), wire)
-			default:
-				err = errInternalBadWireType // skip unknown tag
-			}
-
-			if nerr.Merge(err) {
-				continue
-			}
-			if err != errInternalBadWireType {
-				return nil, err
-			}
-
-			// Skip past unknown fields.
-			b, err = skipField(b, wire)
-			if err != nil {
-				return nil, err
-			}
-		}
-
-		// Get map, allocate if needed.
-		m := f.asPointerTo(t).Elem() // an addressable map[K]T
-		if m.IsNil() {
-			m.Set(reflect.MakeMap(t))
-		}
-
-		// Insert into map.
-		m.SetMapIndex(k.Elem(), v.Elem())
-
-		return r, nerr.E
-	}
-}
-
-// makeUnmarshalOneof makes an unmarshaler for oneof fields.
-// for:
-// message Msg {
-//   oneof F {
-//     int64 X = 1;
-//     float64 Y = 2;
-//   }
-// }
-// typ is the type of the concrete entry for a oneof case (e.g. Msg_X).
-// ityp is the interface type of the oneof field (e.g. isMsg_F).
-// unmarshal is the unmarshaler for the base type of the oneof case (e.g. int64).
-// Note that this function will be called once for each case in the oneof.
-func makeUnmarshalOneof(typ, ityp reflect.Type, unmarshal unmarshaler) unmarshaler {
-	sf := typ.Field(0)
-	field0 := toField(&sf)
-	return func(b []byte, f pointer, w int) ([]byte, error) {
-		// Allocate holder for value.
-		v := reflect.New(typ)
-
-		// Unmarshal data into holder.
-		// We unmarshal into the first field of the holder object.
-		var err error
-		var nerr nonFatal
-		b, err = unmarshal(b, valToPointer(v).offset(field0), w)
-		if !nerr.Merge(err) {
-			return nil, err
-		}
-
-		// Write pointer to holder into target field.
-		f.asPointerTo(ityp).Elem().Set(v)
-
-		return b, nerr.E
-	}
-}
-
-// Error used by decode internally.
-var errInternalBadWireType = errors.New("proto: internal error: bad wiretype")
-
-// skipField skips past a field of type wire and returns the remaining bytes.
-func skipField(b []byte, wire int) ([]byte, error) {
-	switch wire {
-	case WireVarint:
-		_, k := decodeVarint(b)
-		if k == 0 {
-			return b, io.ErrUnexpectedEOF
-		}
-		b = b[k:]
-	case WireFixed32:
-		if len(b) < 4 {
-			return b, io.ErrUnexpectedEOF
-		}
-		b = b[4:]
-	case WireFixed64:
-		if len(b) < 8 {
-			return b, io.ErrUnexpectedEOF
-		}
-		b = b[8:]
-	case WireBytes:
-		m, k := decodeVarint(b)
-		if k == 0 || uint64(len(b)-k) < m {
-			return b, io.ErrUnexpectedEOF
-		}
-		b = b[uint64(k)+m:]
-	case WireStartGroup:
-		_, i := findEndGroup(b)
-		if i == -1 {
-			return b, io.ErrUnexpectedEOF
-		}
-		b = b[i:]
-	default:
-		return b, fmt.Errorf("proto: can't skip unknown wire type %d", wire)
-	}
-	return b, nil
-}
-
-// findEndGroup finds the index of the next EndGroup tag.
-// Groups may be nested, so the "next" EndGroup tag is the first
-// unpaired EndGroup.
-// findEndGroup returns the indexes of the start and end of the EndGroup tag.
-// Returns (-1,-1) if it can't find one.
-func findEndGroup(b []byte) (int, int) {
-	depth := 1
-	i := 0
-	for {
-		x, n := decodeVarint(b[i:])
-		if n == 0 {
-			return -1, -1
-		}
-		j := i
-		i += n
-		switch x & 7 {
-		case WireVarint:
-			_, k := decodeVarint(b[i:])
-			if k == 0 {
-				return -1, -1
-			}
-			i += k
-		case WireFixed32:
-			if len(b)-4 < i {
-				return -1, -1
-			}
-			i += 4
-		case WireFixed64:
-			if len(b)-8 < i {
-				return -1, -1
-			}
-			i += 8
-		case WireBytes:
-			m, k := decodeVarint(b[i:])
-			if k == 0 {
-				return -1, -1
-			}
-			i += k
-			if uint64(len(b)-i) < m {
-				return -1, -1
-			}
-			i += int(m)
-		case WireStartGroup:
-			depth++
-		case WireEndGroup:
-			depth--
-			if depth == 0 {
-				return j, i
-			}
-		default:
-			return -1, -1
-		}
-	}
-}
-
-// encodeVarint appends a varint-encoded integer to b and returns the result.
-func encodeVarint(b []byte, x uint64) []byte {
-	for x >= 1<<7 {
-		b = append(b, byte(x&0x7f|0x80))
-		x >>= 7
-	}
-	return append(b, byte(x))
-}
-
-// decodeVarint reads a varint-encoded integer from b.
-// Returns the decoded integer and the number of bytes read.
-// If there is an error, it returns 0,0.
-func decodeVarint(b []byte) (uint64, int) {
-	var x, y uint64
-	if len(b) == 0 {
-		goto bad
-	}
-	x = uint64(b[0])
-	if x < 0x80 {
-		return x, 1
-	}
-	x -= 0x80
-
-	if len(b) <= 1 {
-		goto bad
-	}
-	y = uint64(b[1])
-	x += y << 7
-	if y < 0x80 {
-		return x, 2
-	}
-	x -= 0x80 << 7
-
-	if len(b) <= 2 {
-		goto bad
-	}
-	y = uint64(b[2])
-	x += y << 14
-	if y < 0x80 {
-		return x, 3
-	}
-	x -= 0x80 << 14
-
-	if len(b) <= 3 {
-		goto bad
-	}
-	y = uint64(b[3])
-	x += y << 21
-	if y < 0x80 {
-		return x, 4
-	}
-	x -= 0x80 << 21
-
-	if len(b) <= 4 {
-		goto bad
-	}
-	y = uint64(b[4])
-	x += y << 28
-	if y < 0x80 {
-		return x, 5
-	}
-	x -= 0x80 << 28
-
-	if len(b) <= 5 {
-		goto bad
-	}
-	y = uint64(b[5])
-	x += y << 35
-	if y < 0x80 {
-		return x, 6
-	}
-	x -= 0x80 << 35
-
-	if len(b) <= 6 {
-		goto bad
-	}
-	y = uint64(b[6])
-	x += y << 42
-	if y < 0x80 {
-		return x, 7
-	}
-	x -= 0x80 << 42
-
-	if len(b) <= 7 {
-		goto bad
-	}
-	y = uint64(b[7])
-	x += y << 49
-	if y < 0x80 {
-		return x, 8
-	}
-	x -= 0x80 << 49
-
-	if len(b) <= 8 {
-		goto bad
-	}
-	y = uint64(b[8])
-	x += y << 56
-	if y < 0x80 {
-		return x, 9
-	}
-	x -= 0x80 << 56
-
-	if len(b) <= 9 {
-		goto bad
-	}
-	y = uint64(b[9])
-	x += y << 63
-	if y < 2 {
-		return x, 10
-	}
-
-bad:
-	return 0, 0
-}
diff --git a/vendor/github.com/golang/protobuf/proto/text.go b/vendor/github.com/golang/protobuf/proto/text.go
deleted file mode 100644
index 1aaee72..0000000
--- a/vendor/github.com/golang/protobuf/proto/text.go
+++ /dev/null
@@ -1,843 +0,0 @@
-// 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.
-
-package proto
-
-// Functions for writing the text protocol buffer format.
-
-import (
-	"bufio"
-	"bytes"
-	"encoding"
-	"errors"
-	"fmt"
-	"io"
-	"log"
-	"math"
-	"reflect"
-	"sort"
-	"strings"
-)
-
-var (
-	newline         = []byte("\n")
-	spaces          = []byte("                                        ")
-	endBraceNewline = []byte("}\n")
-	backslashN      = []byte{'\\', 'n'}
-	backslashR      = []byte{'\\', 'r'}
-	backslashT      = []byte{'\\', 't'}
-	backslashDQ     = []byte{'\\', '"'}
-	backslashBS     = []byte{'\\', '\\'}
-	posInf          = []byte("inf")
-	negInf          = []byte("-inf")
-	nan             = []byte("nan")
-)
-
-type writer interface {
-	io.Writer
-	WriteByte(byte) error
-}
-
-// textWriter is an io.Writer that tracks its indentation level.
-type textWriter struct {
-	ind      int
-	complete bool // if the current position is a complete line
-	compact  bool // whether to write out as a one-liner
-	w        writer
-}
-
-func (w *textWriter) WriteString(s string) (n int, err error) {
-	if !strings.Contains(s, "\n") {
-		if !w.compact && w.complete {
-			w.writeIndent()
-		}
-		w.complete = false
-		return io.WriteString(w.w, s)
-	}
-	// WriteString is typically called without newlines, so this
-	// codepath and its copy are rare.  We copy to avoid
-	// duplicating all of Write's logic here.
-	return w.Write([]byte(s))
-}
-
-func (w *textWriter) Write(p []byte) (n int, err error) {
-	newlines := bytes.Count(p, newline)
-	if newlines == 0 {
-		if !w.compact && w.complete {
-			w.writeIndent()
-		}
-		n, err = w.w.Write(p)
-		w.complete = false
-		return n, err
-	}
-
-	frags := bytes.SplitN(p, newline, newlines+1)
-	if w.compact {
-		for i, frag := range frags {
-			if i > 0 {
-				if err := w.w.WriteByte(' '); err != nil {
-					return n, err
-				}
-				n++
-			}
-			nn, err := w.w.Write(frag)
-			n += nn
-			if err != nil {
-				return n, err
-			}
-		}
-		return n, nil
-	}
-
-	for i, frag := range frags {
-		if w.complete {
-			w.writeIndent()
-		}
-		nn, err := w.w.Write(frag)
-		n += nn
-		if err != nil {
-			return n, err
-		}
-		if i+1 < len(frags) {
-			if err := w.w.WriteByte('\n'); err != nil {
-				return n, err
-			}
-			n++
-		}
-	}
-	w.complete = len(frags[len(frags)-1]) == 0
-	return n, nil
-}
-
-func (w *textWriter) WriteByte(c byte) error {
-	if w.compact && c == '\n' {
-		c = ' '
-	}
-	if !w.compact && w.complete {
-		w.writeIndent()
-	}
-	err := w.w.WriteByte(c)
-	w.complete = c == '\n'
-	return err
-}
-
-func (w *textWriter) indent() { w.ind++ }
-
-func (w *textWriter) unindent() {
-	if w.ind == 0 {
-		log.Print("proto: textWriter unindented too far")
-		return
-	}
-	w.ind--
-}
-
-func writeName(w *textWriter, props *Properties) error {
-	if _, err := w.WriteString(props.OrigName); err != nil {
-		return err
-	}
-	if props.Wire != "group" {
-		return w.WriteByte(':')
-	}
-	return nil
-}
-
-func requiresQuotes(u string) bool {
-	// When type URL contains any characters except [0-9A-Za-z./\-]*, it must be quoted.
-	for _, ch := range u {
-		switch {
-		case ch == '.' || ch == '/' || ch == '_':
-			continue
-		case '0' <= ch && ch <= '9':
-			continue
-		case 'A' <= ch && ch <= 'Z':
-			continue
-		case 'a' <= ch && ch <= 'z':
-			continue
-		default:
-			return true
-		}
-	}
-	return false
-}
-
-// isAny reports whether sv is a google.protobuf.Any message
-func isAny(sv reflect.Value) bool {
-	type wkt interface {
-		XXX_WellKnownType() string
-	}
-	t, ok := sv.Addr().Interface().(wkt)
-	return ok && t.XXX_WellKnownType() == "Any"
-}
-
-// writeProto3Any writes an expanded google.protobuf.Any message.
-//
-// It returns (false, nil) if sv value can't be unmarshaled (e.g. because
-// required messages are not linked in).
-//
-// It returns (true, error) when sv was written in expanded format or an error
-// was encountered.
-func (tm *TextMarshaler) writeProto3Any(w *textWriter, sv reflect.Value) (bool, error) {
-	turl := sv.FieldByName("TypeUrl")
-	val := sv.FieldByName("Value")
-	if !turl.IsValid() || !val.IsValid() {
-		return true, errors.New("proto: invalid google.protobuf.Any message")
-	}
-
-	b, ok := val.Interface().([]byte)
-	if !ok {
-		return true, errors.New("proto: invalid google.protobuf.Any message")
-	}
-
-	parts := strings.Split(turl.String(), "/")
-	mt := MessageType(parts[len(parts)-1])
-	if mt == nil {
-		return false, nil
-	}
-	m := reflect.New(mt.Elem())
-	if err := Unmarshal(b, m.Interface().(Message)); err != nil {
-		return false, nil
-	}
-	w.Write([]byte("["))
-	u := turl.String()
-	if requiresQuotes(u) {
-		writeString(w, u)
-	} else {
-		w.Write([]byte(u))
-	}
-	if w.compact {
-		w.Write([]byte("]:<"))
-	} else {
-		w.Write([]byte("]: <\n"))
-		w.ind++
-	}
-	if err := tm.writeStruct(w, m.Elem()); err != nil {
-		return true, err
-	}
-	if w.compact {
-		w.Write([]byte("> "))
-	} else {
-		w.ind--
-		w.Write([]byte(">\n"))
-	}
-	return true, nil
-}
-
-func (tm *TextMarshaler) writeStruct(w *textWriter, sv reflect.Value) error {
-	if tm.ExpandAny && isAny(sv) {
-		if canExpand, err := tm.writeProto3Any(w, sv); canExpand {
-			return err
-		}
-	}
-	st := sv.Type()
-	sprops := GetProperties(st)
-	for i := 0; i < sv.NumField(); i++ {
-		fv := sv.Field(i)
-		props := sprops.Prop[i]
-		name := st.Field(i).Name
-
-		if name == "XXX_NoUnkeyedLiteral" {
-			continue
-		}
-
-		if strings.HasPrefix(name, "XXX_") {
-			// There are two XXX_ fields:
-			//   XXX_unrecognized []byte
-			//   XXX_extensions   map[int32]proto.Extension
-			// The first is handled here;
-			// the second is handled at the bottom of this function.
-			if name == "XXX_unrecognized" && !fv.IsNil() {
-				if err := writeUnknownStruct(w, fv.Interface().([]byte)); err != nil {
-					return err
-				}
-			}
-			continue
-		}
-		if fv.Kind() == reflect.Ptr && fv.IsNil() {
-			// Field not filled in. This could be an optional field or
-			// a required field that wasn't filled in. Either way, there
-			// isn't anything we can show for it.
-			continue
-		}
-		if fv.Kind() == reflect.Slice && fv.IsNil() {
-			// Repeated field that is empty, or a bytes field that is unused.
-			continue
-		}
-
-		if props.Repeated && fv.Kind() == reflect.Slice {
-			// Repeated field.
-			for j := 0; j < fv.Len(); j++ {
-				if err := writeName(w, props); err != nil {
-					return err
-				}
-				if !w.compact {
-					if err := w.WriteByte(' '); err != nil {
-						return err
-					}
-				}
-				v := fv.Index(j)
-				if v.Kind() == reflect.Ptr && v.IsNil() {
-					// A nil message in a repeated field is not valid,
-					// but we can handle that more gracefully than panicking.
-					if _, err := w.Write([]byte("<nil>\n")); err != nil {
-						return err
-					}
-					continue
-				}
-				if err := tm.writeAny(w, v, props); err != nil {
-					return err
-				}
-				if err := w.WriteByte('\n'); err != nil {
-					return err
-				}
-			}
-			continue
-		}
-		if fv.Kind() == reflect.Map {
-			// Map fields are rendered as a repeated struct with key/value fields.
-			keys := fv.MapKeys()
-			sort.Sort(mapKeys(keys))
-			for _, key := range keys {
-				val := fv.MapIndex(key)
-				if err := writeName(w, props); err != nil {
-					return err
-				}
-				if !w.compact {
-					if err := w.WriteByte(' '); err != nil {
-						return err
-					}
-				}
-				// open struct
-				if err := w.WriteByte('<'); err != nil {
-					return err
-				}
-				if !w.compact {
-					if err := w.WriteByte('\n'); err != nil {
-						return err
-					}
-				}
-				w.indent()
-				// key
-				if _, err := w.WriteString("key:"); err != nil {
-					return err
-				}
-				if !w.compact {
-					if err := w.WriteByte(' '); err != nil {
-						return err
-					}
-				}
-				if err := tm.writeAny(w, key, props.MapKeyProp); err != nil {
-					return err
-				}
-				if err := w.WriteByte('\n'); err != nil {
-					return err
-				}
-				// nil values aren't legal, but we can avoid panicking because of them.
-				if val.Kind() != reflect.Ptr || !val.IsNil() {
-					// value
-					if _, err := w.WriteString("value:"); err != nil {
-						return err
-					}
-					if !w.compact {
-						if err := w.WriteByte(' '); err != nil {
-							return err
-						}
-					}
-					if err := tm.writeAny(w, val, props.MapValProp); err != nil {
-						return err
-					}
-					if err := w.WriteByte('\n'); err != nil {
-						return err
-					}
-				}
-				// close struct
-				w.unindent()
-				if err := w.WriteByte('>'); err != nil {
-					return err
-				}
-				if err := w.WriteByte('\n'); err != nil {
-					return err
-				}
-			}
-			continue
-		}
-		if props.proto3 && fv.Kind() == reflect.Slice && fv.Len() == 0 {
-			// empty bytes field
-			continue
-		}
-		if fv.Kind() != reflect.Ptr && fv.Kind() != reflect.Slice {
-			// proto3 non-repeated scalar field; skip if zero value
-			if isProto3Zero(fv) {
-				continue
-			}
-		}
-
-		if fv.Kind() == reflect.Interface {
-			// Check if it is a oneof.
-			if st.Field(i).Tag.Get("protobuf_oneof") != "" {
-				// fv is nil, or holds a pointer to generated struct.
-				// That generated struct has exactly one field,
-				// which has a protobuf struct tag.
-				if fv.IsNil() {
-					continue
-				}
-				inner := fv.Elem().Elem() // interface -> *T -> T
-				tag := inner.Type().Field(0).Tag.Get("protobuf")
-				props = new(Properties) // Overwrite the outer props var, but not its pointee.
-				props.Parse(tag)
-				// Write the value in the oneof, not the oneof itself.
-				fv = inner.Field(0)
-
-				// Special case to cope with malformed messages gracefully:
-				// If the value in the oneof is a nil pointer, don't panic
-				// in writeAny.
-				if fv.Kind() == reflect.Ptr && fv.IsNil() {
-					// Use errors.New so writeAny won't render quotes.
-					msg := errors.New("/* nil */")
-					fv = reflect.ValueOf(&msg).Elem()
-				}
-			}
-		}
-
-		if err := writeName(w, props); err != nil {
-			return err
-		}
-		if !w.compact {
-			if err := w.WriteByte(' '); err != nil {
-				return err
-			}
-		}
-
-		// Enums have a String method, so writeAny will work fine.
-		if err := tm.writeAny(w, fv, props); err != nil {
-			return err
-		}
-
-		if err := w.WriteByte('\n'); err != nil {
-			return err
-		}
-	}
-
-	// Extensions (the XXX_extensions field).
-	pv := sv.Addr()
-	if _, err := extendable(pv.Interface()); err == nil {
-		if err := tm.writeExtensions(w, pv); err != nil {
-			return err
-		}
-	}
-
-	return nil
-}
-
-// writeAny writes an arbitrary field.
-func (tm *TextMarshaler) writeAny(w *textWriter, v reflect.Value, props *Properties) error {
-	v = reflect.Indirect(v)
-
-	// Floats have special cases.
-	if v.Kind() == reflect.Float32 || v.Kind() == reflect.Float64 {
-		x := v.Float()
-		var b []byte
-		switch {
-		case math.IsInf(x, 1):
-			b = posInf
-		case math.IsInf(x, -1):
-			b = negInf
-		case math.IsNaN(x):
-			b = nan
-		}
-		if b != nil {
-			_, err := w.Write(b)
-			return err
-		}
-		// Other values are handled below.
-	}
-
-	// We don't attempt to serialise every possible value type; only those
-	// that can occur in protocol buffers.
-	switch v.Kind() {
-	case reflect.Slice:
-		// Should only be a []byte; repeated fields are handled in writeStruct.
-		if err := writeString(w, string(v.Bytes())); err != nil {
-			return err
-		}
-	case reflect.String:
-		if err := writeString(w, v.String()); err != nil {
-			return err
-		}
-	case reflect.Struct:
-		// Required/optional group/message.
-		var bra, ket byte = '<', '>'
-		if props != nil && props.Wire == "group" {
-			bra, ket = '{', '}'
-		}
-		if err := w.WriteByte(bra); err != nil {
-			return err
-		}
-		if !w.compact {
-			if err := w.WriteByte('\n'); err != nil {
-				return err
-			}
-		}
-		w.indent()
-		if v.CanAddr() {
-			// Calling v.Interface on a struct causes the reflect package to
-			// copy the entire struct. This is racy with the new Marshaler
-			// since we atomically update the XXX_sizecache.
-			//
-			// Thus, we retrieve a pointer to the struct if possible to avoid
-			// a race since v.Interface on the pointer doesn't copy the struct.
-			//
-			// If v is not addressable, then we are not worried about a race
-			// since it implies that the binary Marshaler cannot possibly be
-			// mutating this value.
-			v = v.Addr()
-		}
-		if etm, ok := v.Interface().(encoding.TextMarshaler); ok {
-			text, err := etm.MarshalText()
-			if err != nil {
-				return err
-			}
-			if _, err = w.Write(text); err != nil {
-				return err
-			}
-		} else {
-			if v.Kind() == reflect.Ptr {
-				v = v.Elem()
-			}
-			if err := tm.writeStruct(w, v); err != nil {
-				return err
-			}
-		}
-		w.unindent()
-		if err := w.WriteByte(ket); err != nil {
-			return err
-		}
-	default:
-		_, err := fmt.Fprint(w, v.Interface())
-		return err
-	}
-	return nil
-}
-
-// equivalent to C's isprint.
-func isprint(c byte) bool {
-	return c >= 0x20 && c < 0x7f
-}
-
-// writeString writes a string in the protocol buffer text format.
-// It is similar to strconv.Quote except we don't use Go escape sequences,
-// we treat the string as a byte sequence, and we use octal escapes.
-// These differences are to maintain interoperability with the other
-// languages' implementations of the text format.
-func writeString(w *textWriter, s string) error {
-	// use WriteByte here to get any needed indent
-	if err := w.WriteByte('"'); err != nil {
-		return err
-	}
-	// Loop over the bytes, not the runes.
-	for i := 0; i < len(s); i++ {
-		var err error
-		// Divergence from C++: we don't escape apostrophes.
-		// There's no need to escape them, and the C++ parser
-		// copes with a naked apostrophe.
-		switch c := s[i]; c {
-		case '\n':
-			_, err = w.w.Write(backslashN)
-		case '\r':
-			_, err = w.w.Write(backslashR)
-		case '\t':
-			_, err = w.w.Write(backslashT)
-		case '"':
-			_, err = w.w.Write(backslashDQ)
-		case '\\':
-			_, err = w.w.Write(backslashBS)
-		default:
-			if isprint(c) {
-				err = w.w.WriteByte(c)
-			} else {
-				_, err = fmt.Fprintf(w.w, "\\%03o", c)
-			}
-		}
-		if err != nil {
-			return err
-		}
-	}
-	return w.WriteByte('"')
-}
-
-func writeUnknownStruct(w *textWriter, data []byte) (err error) {
-	if !w.compact {
-		if _, err := fmt.Fprintf(w, "/* %d unknown bytes */\n", len(data)); err != nil {
-			return err
-		}
-	}
-	b := NewBuffer(data)
-	for b.index < len(b.buf) {
-		x, err := b.DecodeVarint()
-		if err != nil {
-			_, err := fmt.Fprintf(w, "/* %v */\n", err)
-			return err
-		}
-		wire, tag := x&7, x>>3
-		if wire == WireEndGroup {
-			w.unindent()
-			if _, err := w.Write(endBraceNewline); err != nil {
-				return err
-			}
-			continue
-		}
-		if _, err := fmt.Fprint(w, tag); err != nil {
-			return err
-		}
-		if wire != WireStartGroup {
-			if err := w.WriteByte(':'); err != nil {
-				return err
-			}
-		}
-		if !w.compact || wire == WireStartGroup {
-			if err := w.WriteByte(' '); err != nil {
-				return err
-			}
-		}
-		switch wire {
-		case WireBytes:
-			buf, e := b.DecodeRawBytes(false)
-			if e == nil {
-				_, err = fmt.Fprintf(w, "%q", buf)
-			} else {
-				_, err = fmt.Fprintf(w, "/* %v */", e)
-			}
-		case WireFixed32:
-			x, err = b.DecodeFixed32()
-			err = writeUnknownInt(w, x, err)
-		case WireFixed64:
-			x, err = b.DecodeFixed64()
-			err = writeUnknownInt(w, x, err)
-		case WireStartGroup:
-			err = w.WriteByte('{')
-			w.indent()
-		case WireVarint:
-			x, err = b.DecodeVarint()
-			err = writeUnknownInt(w, x, err)
-		default:
-			_, err = fmt.Fprintf(w, "/* unknown wire type %d */", wire)
-		}
-		if err != nil {
-			return err
-		}
-		if err = w.WriteByte('\n'); err != nil {
-			return err
-		}
-	}
-	return nil
-}
-
-func writeUnknownInt(w *textWriter, x uint64, err error) error {
-	if err == nil {
-		_, err = fmt.Fprint(w, x)
-	} else {
-		_, err = fmt.Fprintf(w, "/* %v */", err)
-	}
-	return err
-}
-
-type int32Slice []int32
-
-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] }
-
-// writeExtensions writes all the extensions in pv.
-// pv is assumed to be a pointer to a protocol message struct that is extendable.
-func (tm *TextMarshaler) writeExtensions(w *textWriter, pv reflect.Value) error {
-	emap := extensionMaps[pv.Type().Elem()]
-	ep, _ := extendable(pv.Interface())
-
-	// Order the extensions by ID.
-	// This isn't strictly necessary, but it will give us
-	// canonical output, which will also make testing easier.
-	m, mu := ep.extensionsRead()
-	if m == nil {
-		return nil
-	}
-	mu.Lock()
-	ids := make([]int32, 0, len(m))
-	for id := range m {
-		ids = append(ids, id)
-	}
-	sort.Sort(int32Slice(ids))
-	mu.Unlock()
-
-	for _, extNum := range ids {
-		ext := m[extNum]
-		var desc *ExtensionDesc
-		if emap != nil {
-			desc = emap[extNum]
-		}
-		if desc == nil {
-			// Unknown extension.
-			if err := writeUnknownStruct(w, ext.enc); err != nil {
-				return err
-			}
-			continue
-		}
-
-		pb, err := GetExtension(ep, desc)
-		if err != nil {
-			return fmt.Errorf("failed getting extension: %v", err)
-		}
-
-		// Repeated extensions will appear as a slice.
-		if !desc.repeated() {
-			if err := tm.writeExtension(w, desc.Name, pb); err != nil {
-				return err
-			}
-		} else {
-			v := reflect.ValueOf(pb)
-			for i := 0; i < v.Len(); i++ {
-				if err := tm.writeExtension(w, desc.Name, v.Index(i).Interface()); err != nil {
-					return err
-				}
-			}
-		}
-	}
-	return nil
-}
-
-func (tm *TextMarshaler) writeExtension(w *textWriter, name string, pb interface{}) error {
-	if _, err := fmt.Fprintf(w, "[%s]:", name); err != nil {
-		return err
-	}
-	if !w.compact {
-		if err := w.WriteByte(' '); err != nil {
-			return err
-		}
-	}
-	if err := tm.writeAny(w, reflect.ValueOf(pb), nil); err != nil {
-		return err
-	}
-	if err := w.WriteByte('\n'); err != nil {
-		return err
-	}
-	return nil
-}
-
-func (w *textWriter) writeIndent() {
-	if !w.complete {
-		return
-	}
-	remain := w.ind * 2
-	for remain > 0 {
-		n := remain
-		if n > len(spaces) {
-			n = len(spaces)
-		}
-		w.w.Write(spaces[:n])
-		remain -= n
-	}
-	w.complete = false
-}
-
-// TextMarshaler is a configurable text format marshaler.
-type TextMarshaler struct {
-	Compact   bool // use compact text format (one line).
-	ExpandAny bool // expand google.protobuf.Any messages of known types
-}
-
-// Marshal writes a given protocol buffer in text format.
-// The only errors returned are from w.
-func (tm *TextMarshaler) Marshal(w io.Writer, pb Message) error {
-	val := reflect.ValueOf(pb)
-	if pb == nil || val.IsNil() {
-		w.Write([]byte("<nil>"))
-		return nil
-	}
-	var bw *bufio.Writer
-	ww, ok := w.(writer)
-	if !ok {
-		bw = bufio.NewWriter(w)
-		ww = bw
-	}
-	aw := &textWriter{
-		w:        ww,
-		complete: true,
-		compact:  tm.Compact,
-	}
-
-	if etm, ok := pb.(encoding.TextMarshaler); ok {
-		text, err := etm.MarshalText()
-		if err != nil {
-			return err
-		}
-		if _, err = aw.Write(text); err != nil {
-			return err
-		}
-		if bw != nil {
-			return bw.Flush()
-		}
-		return nil
-	}
-	// Dereference the received pointer so we don't have outer < and >.
-	v := reflect.Indirect(val)
-	if err := tm.writeStruct(aw, v); err != nil {
-		return err
-	}
-	if bw != nil {
-		return bw.Flush()
-	}
-	return nil
-}
-
-// Text is the same as Marshal, but returns the string directly.
-func (tm *TextMarshaler) Text(pb Message) string {
-	var buf bytes.Buffer
-	tm.Marshal(&buf, pb)
-	return buf.String()
-}
-
-var (
-	defaultTextMarshaler = TextMarshaler{}
-	compactTextMarshaler = TextMarshaler{Compact: true}
-)
-
-// TODO: consider removing some of the Marshal functions below.
-
-// MarshalText writes a given protocol buffer in text format.
-// The only errors returned are from w.
-func MarshalText(w io.Writer, pb Message) error { return defaultTextMarshaler.Marshal(w, pb) }
-
-// MarshalTextString is the same as MarshalText, but returns the string directly.
-func MarshalTextString(pb Message) string { return defaultTextMarshaler.Text(pb) }
-
-// CompactText writes a given protocol buffer in compact text format (one line).
-func CompactText(w io.Writer, pb Message) error { return compactTextMarshaler.Marshal(w, pb) }
-
-// CompactTextString is the same as CompactText, but returns the string directly.
-func CompactTextString(pb Message) string { return compactTextMarshaler.Text(pb) }
diff --git a/vendor/github.com/golang/protobuf/proto/text_decode.go b/vendor/github.com/golang/protobuf/proto/text_decode.go
new file mode 100644
index 0000000..47eb3e4
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/text_decode.go
@@ -0,0 +1,801 @@
+// 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 proto
+
+import (
+	"encoding"
+	"errors"
+	"fmt"
+	"reflect"
+	"strconv"
+	"strings"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/prototext"
+	protoV2 "google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+const wrapTextUnmarshalV2 = false
+
+// ParseError is returned by UnmarshalText.
+type ParseError struct {
+	Message string
+
+	// Deprecated: Do not use.
+	Line, Offset int
+}
+
+func (e *ParseError) Error() string {
+	if wrapTextUnmarshalV2 {
+		return e.Message
+	}
+	if e.Line == 1 {
+		return fmt.Sprintf("line 1.%d: %v", e.Offset, e.Message)
+	}
+	return fmt.Sprintf("line %d: %v", e.Line, e.Message)
+}
+
+// UnmarshalText parses a proto text formatted string into m.
+func UnmarshalText(s string, m Message) error {
+	if u, ok := m.(encoding.TextUnmarshaler); ok {
+		return u.UnmarshalText([]byte(s))
+	}
+
+	m.Reset()
+	mi := MessageV2(m)
+
+	if wrapTextUnmarshalV2 {
+		err := prototext.UnmarshalOptions{
+			AllowPartial: true,
+		}.Unmarshal([]byte(s), mi)
+		if err != nil {
+			return &ParseError{Message: err.Error()}
+		}
+		return checkRequiredNotSet(mi)
+	} else {
+		if err := newTextParser(s).unmarshalMessage(mi.ProtoReflect(), ""); err != nil {
+			return err
+		}
+		return checkRequiredNotSet(mi)
+	}
+}
+
+type textParser struct {
+	s            string // remaining input
+	done         bool   // whether the parsing is finished (success or error)
+	backed       bool   // whether back() was called
+	offset, line int
+	cur          token
+}
+
+type token struct {
+	value    string
+	err      *ParseError
+	line     int    // line number
+	offset   int    // byte number from start of input, not start of line
+	unquoted string // the unquoted version of value, if it was a quoted string
+}
+
+func newTextParser(s string) *textParser {
+	p := new(textParser)
+	p.s = s
+	p.line = 1
+	p.cur.line = 1
+	return p
+}
+
+func (p *textParser) unmarshalMessage(m protoreflect.Message, terminator string) (err error) {
+	md := m.Descriptor()
+	fds := md.Fields()
+
+	// A struct is a sequence of "name: value", terminated by one of
+	// '>' or '}', or the end of the input.  A name may also be
+	// "[extension]" or "[type/url]".
+	//
+	// The whole struct can also be an expanded Any message, like:
+	// [type/url] < ... struct contents ... >
+	seen := make(map[protoreflect.FieldNumber]bool)
+	for {
+		tok := p.next()
+		if tok.err != nil {
+			return tok.err
+		}
+		if tok.value == terminator {
+			break
+		}
+		if tok.value == "[" {
+			if err := p.unmarshalExtensionOrAny(m, seen); err != nil {
+				return err
+			}
+			continue
+		}
+
+		// This is a normal, non-extension field.
+		name := protoreflect.Name(tok.value)
+		fd := fds.ByName(name)
+		switch {
+		case fd == nil:
+			gd := fds.ByName(protoreflect.Name(strings.ToLower(string(name))))
+			if gd != nil && gd.Kind() == protoreflect.GroupKind && gd.Message().Name() == name {
+				fd = gd
+			}
+		case fd.Kind() == protoreflect.GroupKind && fd.Message().Name() != name:
+			fd = nil
+		case fd.IsWeak() && fd.Message().IsPlaceholder():
+			fd = nil
+		}
+		if fd == nil {
+			typeName := string(md.FullName())
+			if m, ok := m.Interface().(Message); ok {
+				t := reflect.TypeOf(m)
+				if t.Kind() == reflect.Ptr {
+					typeName = t.Elem().String()
+				}
+			}
+			return p.errorf("unknown field name %q in %v", name, typeName)
+		}
+		if od := fd.ContainingOneof(); od != nil && m.WhichOneof(od) != nil {
+			return p.errorf("field '%s' would overwrite already parsed oneof '%s'", name, od.Name())
+		}
+		if fd.Cardinality() != protoreflect.Repeated && seen[fd.Number()] {
+			return p.errorf("non-repeated field %q was repeated", fd.Name())
+		}
+		seen[fd.Number()] = true
+
+		// Consume any colon.
+		if err := p.checkForColon(fd); err != nil {
+			return err
+		}
+
+		// Parse into the field.
+		v := m.Get(fd)
+		if !m.Has(fd) && (fd.IsList() || fd.IsMap() || fd.Message() != nil) {
+			v = m.Mutable(fd)
+		}
+		if v, err = p.unmarshalValue(v, fd); err != nil {
+			return err
+		}
+		m.Set(fd, v)
+
+		if err := p.consumeOptionalSeparator(); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (p *textParser) unmarshalExtensionOrAny(m protoreflect.Message, seen map[protoreflect.FieldNumber]bool) error {
+	name, err := p.consumeExtensionOrAnyName()
+	if err != nil {
+		return err
+	}
+
+	// If it contains a slash, it's an Any type URL.
+	if slashIdx := strings.LastIndex(name, "/"); slashIdx >= 0 {
+		tok := p.next()
+		if tok.err != nil {
+			return tok.err
+		}
+		// consume an optional colon
+		if tok.value == ":" {
+			tok = p.next()
+			if tok.err != nil {
+				return tok.err
+			}
+		}
+
+		var terminator string
+		switch tok.value {
+		case "<":
+			terminator = ">"
+		case "{":
+			terminator = "}"
+		default:
+			return p.errorf("expected '{' or '<', found %q", tok.value)
+		}
+
+		mt, err := protoregistry.GlobalTypes.FindMessageByURL(name)
+		if err != nil {
+			return p.errorf("unrecognized message %q in google.protobuf.Any", name[slashIdx+len("/"):])
+		}
+		m2 := mt.New()
+		if err := p.unmarshalMessage(m2, terminator); err != nil {
+			return err
+		}
+		b, err := protoV2.Marshal(m2.Interface())
+		if err != nil {
+			return p.errorf("failed to marshal message of type %q: %v", name[slashIdx+len("/"):], err)
+		}
+
+		urlFD := m.Descriptor().Fields().ByName("type_url")
+		valFD := m.Descriptor().Fields().ByName("value")
+		if seen[urlFD.Number()] {
+			return p.errorf("Any message unpacked multiple times, or %q already set", urlFD.Name())
+		}
+		if seen[valFD.Number()] {
+			return p.errorf("Any message unpacked multiple times, or %q already set", valFD.Name())
+		}
+		m.Set(urlFD, protoreflect.ValueOfString(name))
+		m.Set(valFD, protoreflect.ValueOfBytes(b))
+		seen[urlFD.Number()] = true
+		seen[valFD.Number()] = true
+		return nil
+	}
+
+	xname := protoreflect.FullName(name)
+	xt, _ := protoregistry.GlobalTypes.FindExtensionByName(xname)
+	if xt == nil && isMessageSet(m.Descriptor()) {
+		xt, _ = protoregistry.GlobalTypes.FindExtensionByName(xname.Append("message_set_extension"))
+	}
+	if xt == nil {
+		return p.errorf("unrecognized extension %q", name)
+	}
+	fd := xt.TypeDescriptor()
+	if fd.ContainingMessage().FullName() != m.Descriptor().FullName() {
+		return p.errorf("extension field %q does not extend message %q", name, m.Descriptor().FullName())
+	}
+
+	if err := p.checkForColon(fd); err != nil {
+		return err
+	}
+
+	v := m.Get(fd)
+	if !m.Has(fd) && (fd.IsList() || fd.IsMap() || fd.Message() != nil) {
+		v = m.Mutable(fd)
+	}
+	v, err = p.unmarshalValue(v, fd)
+	if err != nil {
+		return err
+	}
+	m.Set(fd, v)
+	return p.consumeOptionalSeparator()
+}
+
+func (p *textParser) unmarshalValue(v protoreflect.Value, fd protoreflect.FieldDescriptor) (protoreflect.Value, error) {
+	tok := p.next()
+	if tok.err != nil {
+		return v, tok.err
+	}
+	if tok.value == "" {
+		return v, p.errorf("unexpected EOF")
+	}
+
+	switch {
+	case fd.IsList():
+		lv := v.List()
+		var err error
+		if tok.value == "[" {
+			// Repeated field with list notation, like [1,2,3].
+			for {
+				vv := lv.NewElement()
+				vv, err = p.unmarshalSingularValue(vv, fd)
+				if err != nil {
+					return v, err
+				}
+				lv.Append(vv)
+
+				tok := p.next()
+				if tok.err != nil {
+					return v, tok.err
+				}
+				if tok.value == "]" {
+					break
+				}
+				if tok.value != "," {
+					return v, p.errorf("Expected ']' or ',' found %q", tok.value)
+				}
+			}
+			return v, nil
+		}
+
+		// One value of the repeated field.
+		p.back()
+		vv := lv.NewElement()
+		vv, err = p.unmarshalSingularValue(vv, fd)
+		if err != nil {
+			return v, err
+		}
+		lv.Append(vv)
+		return v, nil
+	case fd.IsMap():
+		// The map entry should be this sequence of tokens:
+		//	< key : KEY value : VALUE >
+		// However, implementations may omit key or value, and technically
+		// we should support them in any order.
+		var terminator string
+		switch tok.value {
+		case "<":
+			terminator = ">"
+		case "{":
+			terminator = "}"
+		default:
+			return v, p.errorf("expected '{' or '<', found %q", tok.value)
+		}
+
+		keyFD := fd.MapKey()
+		valFD := fd.MapValue()
+
+		mv := v.Map()
+		kv := keyFD.Default()
+		vv := mv.NewValue()
+		for {
+			tok := p.next()
+			if tok.err != nil {
+				return v, tok.err
+			}
+			if tok.value == terminator {
+				break
+			}
+			var err error
+			switch tok.value {
+			case "key":
+				if err := p.consumeToken(":"); err != nil {
+					return v, err
+				}
+				if kv, err = p.unmarshalSingularValue(kv, keyFD); err != nil {
+					return v, err
+				}
+				if err := p.consumeOptionalSeparator(); err != nil {
+					return v, err
+				}
+			case "value":
+				if err := p.checkForColon(valFD); err != nil {
+					return v, err
+				}
+				if vv, err = p.unmarshalSingularValue(vv, valFD); err != nil {
+					return v, err
+				}
+				if err := p.consumeOptionalSeparator(); err != nil {
+					return v, err
+				}
+			default:
+				p.back()
+				return v, p.errorf(`expected "key", "value", or %q, found %q`, terminator, tok.value)
+			}
+		}
+		mv.Set(kv.MapKey(), vv)
+		return v, nil
+	default:
+		p.back()
+		return p.unmarshalSingularValue(v, fd)
+	}
+}
+
+func (p *textParser) unmarshalSingularValue(v protoreflect.Value, fd protoreflect.FieldDescriptor) (protoreflect.Value, error) {
+	tok := p.next()
+	if tok.err != nil {
+		return v, tok.err
+	}
+	if tok.value == "" {
+		return v, p.errorf("unexpected EOF")
+	}
+
+	switch fd.Kind() {
+	case protoreflect.BoolKind:
+		switch tok.value {
+		case "true", "1", "t", "True":
+			return protoreflect.ValueOfBool(true), nil
+		case "false", "0", "f", "False":
+			return protoreflect.ValueOfBool(false), nil
+		}
+	case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind:
+		if x, err := strconv.ParseInt(tok.value, 0, 32); err == nil {
+			return protoreflect.ValueOfInt32(int32(x)), nil
+		}
+
+		// The C++ parser accepts large positive hex numbers that uses
+		// two's complement arithmetic to represent negative numbers.
+		// This feature is here for backwards compatibility with C++.
+		if strings.HasPrefix(tok.value, "0x") {
+			if x, err := strconv.ParseUint(tok.value, 0, 32); err == nil {
+				return protoreflect.ValueOfInt32(int32(-(int64(^x) + 1))), nil
+			}
+		}
+	case protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Sfixed64Kind:
+		if x, err := strconv.ParseInt(tok.value, 0, 64); err == nil {
+			return protoreflect.ValueOfInt64(int64(x)), nil
+		}
+
+		// The C++ parser accepts large positive hex numbers that uses
+		// two's complement arithmetic to represent negative numbers.
+		// This feature is here for backwards compatibility with C++.
+		if strings.HasPrefix(tok.value, "0x") {
+			if x, err := strconv.ParseUint(tok.value, 0, 64); err == nil {
+				return protoreflect.ValueOfInt64(int64(-(int64(^x) + 1))), nil
+			}
+		}
+	case protoreflect.Uint32Kind, protoreflect.Fixed32Kind:
+		if x, err := strconv.ParseUint(tok.value, 0, 32); err == nil {
+			return protoreflect.ValueOfUint32(uint32(x)), nil
+		}
+	case protoreflect.Uint64Kind, protoreflect.Fixed64Kind:
+		if x, err := strconv.ParseUint(tok.value, 0, 64); err == nil {
+			return protoreflect.ValueOfUint64(uint64(x)), nil
+		}
+	case protoreflect.FloatKind:
+		// Ignore 'f' for compatibility with output generated by C++,
+		// but don't remove 'f' when the value is "-inf" or "inf".
+		v := tok.value
+		if strings.HasSuffix(v, "f") && v != "-inf" && v != "inf" {
+			v = v[:len(v)-len("f")]
+		}
+		if x, err := strconv.ParseFloat(v, 32); err == nil {
+			return protoreflect.ValueOfFloat32(float32(x)), nil
+		}
+	case protoreflect.DoubleKind:
+		// Ignore 'f' for compatibility with output generated by C++,
+		// but don't remove 'f' when the value is "-inf" or "inf".
+		v := tok.value
+		if strings.HasSuffix(v, "f") && v != "-inf" && v != "inf" {
+			v = v[:len(v)-len("f")]
+		}
+		if x, err := strconv.ParseFloat(v, 64); err == nil {
+			return protoreflect.ValueOfFloat64(float64(x)), nil
+		}
+	case protoreflect.StringKind:
+		if isQuote(tok.value[0]) {
+			return protoreflect.ValueOfString(tok.unquoted), nil
+		}
+	case protoreflect.BytesKind:
+		if isQuote(tok.value[0]) {
+			return protoreflect.ValueOfBytes([]byte(tok.unquoted)), nil
+		}
+	case protoreflect.EnumKind:
+		if x, err := strconv.ParseInt(tok.value, 0, 32); err == nil {
+			return protoreflect.ValueOfEnum(protoreflect.EnumNumber(x)), nil
+		}
+		vd := fd.Enum().Values().ByName(protoreflect.Name(tok.value))
+		if vd != nil {
+			return protoreflect.ValueOfEnum(vd.Number()), nil
+		}
+	case protoreflect.MessageKind, protoreflect.GroupKind:
+		var terminator string
+		switch tok.value {
+		case "{":
+			terminator = "}"
+		case "<":
+			terminator = ">"
+		default:
+			return v, p.errorf("expected '{' or '<', found %q", tok.value)
+		}
+		err := p.unmarshalMessage(v.Message(), terminator)
+		return v, err
+	default:
+		panic(fmt.Sprintf("invalid kind %v", fd.Kind()))
+	}
+	return v, p.errorf("invalid %v: %v", fd.Kind(), tok.value)
+}
+
+// Consume a ':' from the input stream (if the next token is a colon),
+// returning an error if a colon is needed but not present.
+func (p *textParser) checkForColon(fd protoreflect.FieldDescriptor) *ParseError {
+	tok := p.next()
+	if tok.err != nil {
+		return tok.err
+	}
+	if tok.value != ":" {
+		if fd.Message() == nil {
+			return p.errorf("expected ':', found %q", tok.value)
+		}
+		p.back()
+	}
+	return nil
+}
+
+// consumeExtensionOrAnyName consumes an extension name or an Any type URL and
+// the following ']'. It returns the name or URL consumed.
+func (p *textParser) consumeExtensionOrAnyName() (string, error) {
+	tok := p.next()
+	if tok.err != nil {
+		return "", tok.err
+	}
+
+	// If extension name or type url is quoted, it's a single token.
+	if len(tok.value) > 2 && isQuote(tok.value[0]) && tok.value[len(tok.value)-1] == tok.value[0] {
+		name, err := unquoteC(tok.value[1:len(tok.value)-1], rune(tok.value[0]))
+		if err != nil {
+			return "", err
+		}
+		return name, p.consumeToken("]")
+	}
+
+	// Consume everything up to "]"
+	var parts []string
+	for tok.value != "]" {
+		parts = append(parts, tok.value)
+		tok = p.next()
+		if tok.err != nil {
+			return "", p.errorf("unrecognized type_url or extension name: %s", tok.err)
+		}
+		if p.done && tok.value != "]" {
+			return "", p.errorf("unclosed type_url or extension name")
+		}
+	}
+	return strings.Join(parts, ""), nil
+}
+
+// consumeOptionalSeparator consumes an optional semicolon or comma.
+// It is used in unmarshalMessage to provide backward compatibility.
+func (p *textParser) consumeOptionalSeparator() error {
+	tok := p.next()
+	if tok.err != nil {
+		return tok.err
+	}
+	if tok.value != ";" && tok.value != "," {
+		p.back()
+	}
+	return nil
+}
+
+func (p *textParser) errorf(format string, a ...interface{}) *ParseError {
+	pe := &ParseError{fmt.Sprintf(format, a...), p.cur.line, p.cur.offset}
+	p.cur.err = pe
+	p.done = true
+	return pe
+}
+
+func (p *textParser) skipWhitespace() {
+	i := 0
+	for i < len(p.s) && (isWhitespace(p.s[i]) || p.s[i] == '#') {
+		if p.s[i] == '#' {
+			// comment; skip to end of line or input
+			for i < len(p.s) && p.s[i] != '\n' {
+				i++
+			}
+			if i == len(p.s) {
+				break
+			}
+		}
+		if p.s[i] == '\n' {
+			p.line++
+		}
+		i++
+	}
+	p.offset += i
+	p.s = p.s[i:len(p.s)]
+	if len(p.s) == 0 {
+		p.done = true
+	}
+}
+
+func (p *textParser) advance() {
+	// Skip whitespace
+	p.skipWhitespace()
+	if p.done {
+		return
+	}
+
+	// Start of non-whitespace
+	p.cur.err = nil
+	p.cur.offset, p.cur.line = p.offset, p.line
+	p.cur.unquoted = ""
+	switch p.s[0] {
+	case '<', '>', '{', '}', ':', '[', ']', ';', ',', '/':
+		// Single symbol
+		p.cur.value, p.s = p.s[0:1], p.s[1:len(p.s)]
+	case '"', '\'':
+		// Quoted string
+		i := 1
+		for i < len(p.s) && p.s[i] != p.s[0] && p.s[i] != '\n' {
+			if p.s[i] == '\\' && i+1 < len(p.s) {
+				// skip escaped char
+				i++
+			}
+			i++
+		}
+		if i >= len(p.s) || p.s[i] != p.s[0] {
+			p.errorf("unmatched quote")
+			return
+		}
+		unq, err := unquoteC(p.s[1:i], rune(p.s[0]))
+		if err != nil {
+			p.errorf("invalid quoted string %s: %v", p.s[0:i+1], err)
+			return
+		}
+		p.cur.value, p.s = p.s[0:i+1], p.s[i+1:len(p.s)]
+		p.cur.unquoted = unq
+	default:
+		i := 0
+		for i < len(p.s) && isIdentOrNumberChar(p.s[i]) {
+			i++
+		}
+		if i == 0 {
+			p.errorf("unexpected byte %#x", p.s[0])
+			return
+		}
+		p.cur.value, p.s = p.s[0:i], p.s[i:len(p.s)]
+	}
+	p.offset += len(p.cur.value)
+}
+
+// Back off the parser by one token. Can only be done between calls to next().
+// It makes the next advance() a no-op.
+func (p *textParser) back() { p.backed = true }
+
+// Advances the parser and returns the new current token.
+func (p *textParser) next() *token {
+	if p.backed || p.done {
+		p.backed = false
+		return &p.cur
+	}
+	p.advance()
+	if p.done {
+		p.cur.value = ""
+	} else if len(p.cur.value) > 0 && isQuote(p.cur.value[0]) {
+		// Look for multiple quoted strings separated by whitespace,
+		// and concatenate them.
+		cat := p.cur
+		for {
+			p.skipWhitespace()
+			if p.done || !isQuote(p.s[0]) {
+				break
+			}
+			p.advance()
+			if p.cur.err != nil {
+				return &p.cur
+			}
+			cat.value += " " + p.cur.value
+			cat.unquoted += p.cur.unquoted
+		}
+		p.done = false // parser may have seen EOF, but we want to return cat
+		p.cur = cat
+	}
+	return &p.cur
+}
+
+func (p *textParser) consumeToken(s string) error {
+	tok := p.next()
+	if tok.err != nil {
+		return tok.err
+	}
+	if tok.value != s {
+		p.back()
+		return p.errorf("expected %q, found %q", s, tok.value)
+	}
+	return nil
+}
+
+var errBadUTF8 = errors.New("proto: bad UTF-8")
+
+func unquoteC(s string, quote rune) (string, error) {
+	// This is based on C++'s tokenizer.cc.
+	// Despite its name, this is *not* parsing C syntax.
+	// For instance, "\0" is an invalid quoted string.
+
+	// Avoid allocation in trivial cases.
+	simple := true
+	for _, r := range s {
+		if r == '\\' || r == quote {
+			simple = false
+			break
+		}
+	}
+	if simple {
+		return s, nil
+	}
+
+	buf := make([]byte, 0, 3*len(s)/2)
+	for len(s) > 0 {
+		r, n := utf8.DecodeRuneInString(s)
+		if r == utf8.RuneError && n == 1 {
+			return "", errBadUTF8
+		}
+		s = s[n:]
+		if r != '\\' {
+			if r < utf8.RuneSelf {
+				buf = append(buf, byte(r))
+			} else {
+				buf = append(buf, string(r)...)
+			}
+			continue
+		}
+
+		ch, tail, err := unescape(s)
+		if err != nil {
+			return "", err
+		}
+		buf = append(buf, ch...)
+		s = tail
+	}
+	return string(buf), nil
+}
+
+func unescape(s string) (ch string, tail string, err error) {
+	r, n := utf8.DecodeRuneInString(s)
+	if r == utf8.RuneError && n == 1 {
+		return "", "", errBadUTF8
+	}
+	s = s[n:]
+	switch r {
+	case 'a':
+		return "\a", s, nil
+	case 'b':
+		return "\b", s, nil
+	case 'f':
+		return "\f", s, nil
+	case 'n':
+		return "\n", s, nil
+	case 'r':
+		return "\r", s, nil
+	case 't':
+		return "\t", s, nil
+	case 'v':
+		return "\v", s, nil
+	case '?':
+		return "?", s, nil // trigraph workaround
+	case '\'', '"', '\\':
+		return string(r), s, nil
+	case '0', '1', '2', '3', '4', '5', '6', '7':
+		if len(s) < 2 {
+			return "", "", fmt.Errorf(`\%c requires 2 following digits`, r)
+		}
+		ss := string(r) + s[:2]
+		s = s[2:]
+		i, err := strconv.ParseUint(ss, 8, 8)
+		if err != nil {
+			return "", "", fmt.Errorf(`\%s contains non-octal digits`, ss)
+		}
+		return string([]byte{byte(i)}), s, nil
+	case 'x', 'X', 'u', 'U':
+		var n int
+		switch r {
+		case 'x', 'X':
+			n = 2
+		case 'u':
+			n = 4
+		case 'U':
+			n = 8
+		}
+		if len(s) < n {
+			return "", "", fmt.Errorf(`\%c requires %d following digits`, r, n)
+		}
+		ss := s[:n]
+		s = s[n:]
+		i, err := strconv.ParseUint(ss, 16, 64)
+		if err != nil {
+			return "", "", fmt.Errorf(`\%c%s contains non-hexadecimal digits`, r, ss)
+		}
+		if r == 'x' || r == 'X' {
+			return string([]byte{byte(i)}), s, nil
+		}
+		if i > utf8.MaxRune {
+			return "", "", fmt.Errorf(`\%c%s is not a valid Unicode code point`, r, ss)
+		}
+		return string(rune(i)), s, nil
+	}
+	return "", "", fmt.Errorf(`unknown escape \%c`, r)
+}
+
+func isIdentOrNumberChar(c byte) bool {
+	switch {
+	case 'A' <= c && c <= 'Z', 'a' <= c && c <= 'z':
+		return true
+	case '0' <= c && c <= '9':
+		return true
+	}
+	switch c {
+	case '-', '+', '.', '_':
+		return true
+	}
+	return false
+}
+
+func isWhitespace(c byte) bool {
+	switch c {
+	case ' ', '\t', '\n', '\r':
+		return true
+	}
+	return false
+}
+
+func isQuote(c byte) bool {
+	switch c {
+	case '"', '\'':
+		return true
+	}
+	return false
+}
diff --git a/vendor/github.com/golang/protobuf/proto/text_encode.go b/vendor/github.com/golang/protobuf/proto/text_encode.go
new file mode 100644
index 0000000..a31134e
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/text_encode.go
@@ -0,0 +1,560 @@
+// 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 proto
+
+import (
+	"bytes"
+	"encoding"
+	"fmt"
+	"io"
+	"math"
+	"sort"
+	"strings"
+
+	"google.golang.org/protobuf/encoding/prototext"
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+const wrapTextMarshalV2 = false
+
+// TextMarshaler is a configurable text format marshaler.
+type TextMarshaler struct {
+	Compact   bool // use compact text format (one line)
+	ExpandAny bool // expand google.protobuf.Any messages of known types
+}
+
+// Marshal writes the proto text format of m to w.
+func (tm *TextMarshaler) Marshal(w io.Writer, m Message) error {
+	b, err := tm.marshal(m)
+	if len(b) > 0 {
+		if _, err := w.Write(b); err != nil {
+			return err
+		}
+	}
+	return err
+}
+
+// Text returns a proto text formatted string of m.
+func (tm *TextMarshaler) Text(m Message) string {
+	b, _ := tm.marshal(m)
+	return string(b)
+}
+
+func (tm *TextMarshaler) marshal(m Message) ([]byte, error) {
+	mr := MessageReflect(m)
+	if mr == nil || !mr.IsValid() {
+		return []byte("<nil>"), nil
+	}
+
+	if wrapTextMarshalV2 {
+		if m, ok := m.(encoding.TextMarshaler); ok {
+			return m.MarshalText()
+		}
+
+		opts := prototext.MarshalOptions{
+			AllowPartial: true,
+			EmitUnknown:  true,
+		}
+		if !tm.Compact {
+			opts.Indent = "  "
+		}
+		if !tm.ExpandAny {
+			opts.Resolver = (*protoregistry.Types)(nil)
+		}
+		return opts.Marshal(mr.Interface())
+	} else {
+		w := &textWriter{
+			compact:   tm.Compact,
+			expandAny: tm.ExpandAny,
+			complete:  true,
+		}
+
+		if m, ok := m.(encoding.TextMarshaler); ok {
+			b, err := m.MarshalText()
+			if err != nil {
+				return nil, err
+			}
+			w.Write(b)
+			return w.buf, nil
+		}
+
+		err := w.writeMessage(mr)
+		return w.buf, err
+	}
+}
+
+var (
+	defaultTextMarshaler = TextMarshaler{}
+	compactTextMarshaler = TextMarshaler{Compact: true}
+)
+
+// MarshalText writes the proto text format of m to w.
+func MarshalText(w io.Writer, m Message) error { return defaultTextMarshaler.Marshal(w, m) }
+
+// MarshalTextString returns a proto text formatted string of m.
+func MarshalTextString(m Message) string { return defaultTextMarshaler.Text(m) }
+
+// CompactText writes the compact proto text format of m to w.
+func CompactText(w io.Writer, m Message) error { return compactTextMarshaler.Marshal(w, m) }
+
+// CompactTextString returns a compact proto text formatted string of m.
+func CompactTextString(m Message) string { return compactTextMarshaler.Text(m) }
+
+var (
+	newline         = []byte("\n")
+	endBraceNewline = []byte("}\n")
+	posInf          = []byte("inf")
+	negInf          = []byte("-inf")
+	nan             = []byte("nan")
+)
+
+// textWriter is an io.Writer that tracks its indentation level.
+type textWriter struct {
+	compact   bool // same as TextMarshaler.Compact
+	expandAny bool // same as TextMarshaler.ExpandAny
+	complete  bool // whether the current position is a complete line
+	indent    int  // indentation level; never negative
+	buf       []byte
+}
+
+func (w *textWriter) Write(p []byte) (n int, _ error) {
+	newlines := bytes.Count(p, newline)
+	if newlines == 0 {
+		if !w.compact && w.complete {
+			w.writeIndent()
+		}
+		w.buf = append(w.buf, p...)
+		w.complete = false
+		return len(p), nil
+	}
+
+	frags := bytes.SplitN(p, newline, newlines+1)
+	if w.compact {
+		for i, frag := range frags {
+			if i > 0 {
+				w.buf = append(w.buf, ' ')
+				n++
+			}
+			w.buf = append(w.buf, frag...)
+			n += len(frag)
+		}
+		return n, nil
+	}
+
+	for i, frag := range frags {
+		if w.complete {
+			w.writeIndent()
+		}
+		w.buf = append(w.buf, frag...)
+		n += len(frag)
+		if i+1 < len(frags) {
+			w.buf = append(w.buf, '\n')
+			n++
+		}
+	}
+	w.complete = len(frags[len(frags)-1]) == 0
+	return n, nil
+}
+
+func (w *textWriter) WriteByte(c byte) error {
+	if w.compact && c == '\n' {
+		c = ' '
+	}
+	if !w.compact && w.complete {
+		w.writeIndent()
+	}
+	w.buf = append(w.buf, c)
+	w.complete = c == '\n'
+	return nil
+}
+
+func (w *textWriter) writeName(fd protoreflect.FieldDescriptor) {
+	if !w.compact && w.complete {
+		w.writeIndent()
+	}
+	w.complete = false
+
+	if fd.Kind() != protoreflect.GroupKind {
+		w.buf = append(w.buf, fd.Name()...)
+		w.WriteByte(':')
+	} else {
+		// Use message type name for group field name.
+		w.buf = append(w.buf, fd.Message().Name()...)
+	}
+
+	if !w.compact {
+		w.WriteByte(' ')
+	}
+}
+
+func requiresQuotes(u string) bool {
+	// When type URL contains any characters except [0-9A-Za-z./\-]*, it must be quoted.
+	for _, ch := range u {
+		switch {
+		case ch == '.' || ch == '/' || ch == '_':
+			continue
+		case '0' <= ch && ch <= '9':
+			continue
+		case 'A' <= ch && ch <= 'Z':
+			continue
+		case 'a' <= ch && ch <= 'z':
+			continue
+		default:
+			return true
+		}
+	}
+	return false
+}
+
+// writeProto3Any writes an expanded google.protobuf.Any message.
+//
+// It returns (false, nil) if sv value can't be unmarshaled (e.g. because
+// required messages are not linked in).
+//
+// It returns (true, error) when sv was written in expanded format or an error
+// was encountered.
+func (w *textWriter) writeProto3Any(m protoreflect.Message) (bool, error) {
+	md := m.Descriptor()
+	fdURL := md.Fields().ByName("type_url")
+	fdVal := md.Fields().ByName("value")
+
+	url := m.Get(fdURL).String()
+	mt, err := protoregistry.GlobalTypes.FindMessageByURL(url)
+	if err != nil {
+		return false, nil
+	}
+
+	b := m.Get(fdVal).Bytes()
+	m2 := mt.New()
+	if err := proto.Unmarshal(b, m2.Interface()); err != nil {
+		return false, nil
+	}
+	w.Write([]byte("["))
+	if requiresQuotes(url) {
+		w.writeQuotedString(url)
+	} else {
+		w.Write([]byte(url))
+	}
+	if w.compact {
+		w.Write([]byte("]:<"))
+	} else {
+		w.Write([]byte("]: <\n"))
+		w.indent++
+	}
+	if err := w.writeMessage(m2); err != nil {
+		return true, err
+	}
+	if w.compact {
+		w.Write([]byte("> "))
+	} else {
+		w.indent--
+		w.Write([]byte(">\n"))
+	}
+	return true, nil
+}
+
+func (w *textWriter) writeMessage(m protoreflect.Message) error {
+	md := m.Descriptor()
+	if w.expandAny && md.FullName() == "google.protobuf.Any" {
+		if canExpand, err := w.writeProto3Any(m); canExpand {
+			return err
+		}
+	}
+
+	fds := md.Fields()
+	for i := 0; i < fds.Len(); {
+		fd := fds.Get(i)
+		if od := fd.ContainingOneof(); od != nil {
+			fd = m.WhichOneof(od)
+			i += od.Fields().Len()
+		} else {
+			i++
+		}
+		if fd == nil || !m.Has(fd) {
+			continue
+		}
+
+		switch {
+		case fd.IsList():
+			lv := m.Get(fd).List()
+			for j := 0; j < lv.Len(); j++ {
+				w.writeName(fd)
+				v := lv.Get(j)
+				if err := w.writeSingularValue(v, fd); err != nil {
+					return err
+				}
+				w.WriteByte('\n')
+			}
+		case fd.IsMap():
+			kfd := fd.MapKey()
+			vfd := fd.MapValue()
+			mv := m.Get(fd).Map()
+
+			type entry struct{ key, val protoreflect.Value }
+			var entries []entry
+			mv.Range(func(k protoreflect.MapKey, v protoreflect.Value) bool {
+				entries = append(entries, entry{k.Value(), v})
+				return true
+			})
+			sort.Slice(entries, func(i, j int) bool {
+				switch kfd.Kind() {
+				case protoreflect.BoolKind:
+					return !entries[i].key.Bool() && entries[j].key.Bool()
+				case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind, protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Sfixed64Kind:
+					return entries[i].key.Int() < entries[j].key.Int()
+				case protoreflect.Uint32Kind, protoreflect.Fixed32Kind, protoreflect.Uint64Kind, protoreflect.Fixed64Kind:
+					return entries[i].key.Uint() < entries[j].key.Uint()
+				case protoreflect.StringKind:
+					return entries[i].key.String() < entries[j].key.String()
+				default:
+					panic("invalid kind")
+				}
+			})
+			for _, entry := range entries {
+				w.writeName(fd)
+				w.WriteByte('<')
+				if !w.compact {
+					w.WriteByte('\n')
+				}
+				w.indent++
+				w.writeName(kfd)
+				if err := w.writeSingularValue(entry.key, kfd); err != nil {
+					return err
+				}
+				w.WriteByte('\n')
+				w.writeName(vfd)
+				if err := w.writeSingularValue(entry.val, vfd); err != nil {
+					return err
+				}
+				w.WriteByte('\n')
+				w.indent--
+				w.WriteByte('>')
+				w.WriteByte('\n')
+			}
+		default:
+			w.writeName(fd)
+			if err := w.writeSingularValue(m.Get(fd), fd); err != nil {
+				return err
+			}
+			w.WriteByte('\n')
+		}
+	}
+
+	if b := m.GetUnknown(); len(b) > 0 {
+		w.writeUnknownFields(b)
+	}
+	return w.writeExtensions(m)
+}
+
+func (w *textWriter) writeSingularValue(v protoreflect.Value, fd protoreflect.FieldDescriptor) error {
+	switch fd.Kind() {
+	case protoreflect.FloatKind, protoreflect.DoubleKind:
+		switch vf := v.Float(); {
+		case math.IsInf(vf, +1):
+			w.Write(posInf)
+		case math.IsInf(vf, -1):
+			w.Write(negInf)
+		case math.IsNaN(vf):
+			w.Write(nan)
+		default:
+			fmt.Fprint(w, v.Interface())
+		}
+	case protoreflect.StringKind:
+		// NOTE: This does not validate UTF-8 for historical reasons.
+		w.writeQuotedString(string(v.String()))
+	case protoreflect.BytesKind:
+		w.writeQuotedString(string(v.Bytes()))
+	case protoreflect.MessageKind, protoreflect.GroupKind:
+		var bra, ket byte = '<', '>'
+		if fd.Kind() == protoreflect.GroupKind {
+			bra, ket = '{', '}'
+		}
+		w.WriteByte(bra)
+		if !w.compact {
+			w.WriteByte('\n')
+		}
+		w.indent++
+		m := v.Message()
+		if m2, ok := m.Interface().(encoding.TextMarshaler); ok {
+			b, err := m2.MarshalText()
+			if err != nil {
+				return err
+			}
+			w.Write(b)
+		} else {
+			w.writeMessage(m)
+		}
+		w.indent--
+		w.WriteByte(ket)
+	case protoreflect.EnumKind:
+		if ev := fd.Enum().Values().ByNumber(v.Enum()); ev != nil {
+			fmt.Fprint(w, ev.Name())
+		} else {
+			fmt.Fprint(w, v.Enum())
+		}
+	default:
+		fmt.Fprint(w, v.Interface())
+	}
+	return nil
+}
+
+// writeQuotedString writes a quoted string in the protocol buffer text format.
+func (w *textWriter) writeQuotedString(s string) {
+	w.WriteByte('"')
+	for i := 0; i < len(s); i++ {
+		switch c := s[i]; c {
+		case '\n':
+			w.buf = append(w.buf, `\n`...)
+		case '\r':
+			w.buf = append(w.buf, `\r`...)
+		case '\t':
+			w.buf = append(w.buf, `\t`...)
+		case '"':
+			w.buf = append(w.buf, `\"`...)
+		case '\\':
+			w.buf = append(w.buf, `\\`...)
+		default:
+			if isPrint := c >= 0x20 && c < 0x7f; isPrint {
+				w.buf = append(w.buf, c)
+			} else {
+				w.buf = append(w.buf, fmt.Sprintf(`\%03o`, c)...)
+			}
+		}
+	}
+	w.WriteByte('"')
+}
+
+func (w *textWriter) writeUnknownFields(b []byte) {
+	if !w.compact {
+		fmt.Fprintf(w, "/* %d unknown bytes */\n", len(b))
+	}
+
+	for len(b) > 0 {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return
+		}
+		b = b[n:]
+
+		if wtyp == protowire.EndGroupType {
+			w.indent--
+			w.Write(endBraceNewline)
+			continue
+		}
+		fmt.Fprint(w, num)
+		if wtyp != protowire.StartGroupType {
+			w.WriteByte(':')
+		}
+		if !w.compact || wtyp == protowire.StartGroupType {
+			w.WriteByte(' ')
+		}
+		switch wtyp {
+		case protowire.VarintType:
+			v, n := protowire.ConsumeVarint(b)
+			if n < 0 {
+				return
+			}
+			b = b[n:]
+			fmt.Fprint(w, v)
+		case protowire.Fixed32Type:
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return
+			}
+			b = b[n:]
+			fmt.Fprint(w, v)
+		case protowire.Fixed64Type:
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return
+			}
+			b = b[n:]
+			fmt.Fprint(w, v)
+		case protowire.BytesType:
+			v, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return
+			}
+			b = b[n:]
+			fmt.Fprintf(w, "%q", v)
+		case protowire.StartGroupType:
+			w.WriteByte('{')
+			w.indent++
+		default:
+			fmt.Fprintf(w, "/* unknown wire type %d */", wtyp)
+		}
+		w.WriteByte('\n')
+	}
+}
+
+// writeExtensions writes all the extensions in m.
+func (w *textWriter) writeExtensions(m protoreflect.Message) error {
+	md := m.Descriptor()
+	if md.ExtensionRanges().Len() == 0 {
+		return nil
+	}
+
+	type ext struct {
+		desc protoreflect.FieldDescriptor
+		val  protoreflect.Value
+	}
+	var exts []ext
+	m.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		if fd.IsExtension() {
+			exts = append(exts, ext{fd, v})
+		}
+		return true
+	})
+	sort.Slice(exts, func(i, j int) bool {
+		return exts[i].desc.Number() < exts[j].desc.Number()
+	})
+
+	for _, ext := range exts {
+		// For message set, use the name of the message as the extension name.
+		name := string(ext.desc.FullName())
+		if isMessageSet(ext.desc.ContainingMessage()) {
+			name = strings.TrimSuffix(name, ".message_set_extension")
+		}
+
+		if !ext.desc.IsList() {
+			if err := w.writeSingularExtension(name, ext.val, ext.desc); err != nil {
+				return err
+			}
+		} else {
+			lv := ext.val.List()
+			for i := 0; i < lv.Len(); i++ {
+				if err := w.writeSingularExtension(name, lv.Get(i), ext.desc); err != nil {
+					return err
+				}
+			}
+		}
+	}
+	return nil
+}
+
+func (w *textWriter) writeSingularExtension(name string, v protoreflect.Value, fd protoreflect.FieldDescriptor) error {
+	fmt.Fprintf(w, "[%s]:", name)
+	if !w.compact {
+		w.WriteByte(' ')
+	}
+	if err := w.writeSingularValue(v, fd); err != nil {
+		return err
+	}
+	w.WriteByte('\n')
+	return nil
+}
+
+func (w *textWriter) writeIndent() {
+	if !w.complete {
+		return
+	}
+	for i := 0; i < w.indent*2; i++ {
+		w.buf = append(w.buf, ' ')
+	}
+	w.complete = false
+}
diff --git a/vendor/github.com/golang/protobuf/proto/text_parser.go b/vendor/github.com/golang/protobuf/proto/text_parser.go
deleted file mode 100644
index bb55a3a..0000000
--- a/vendor/github.com/golang/protobuf/proto/text_parser.go
+++ /dev/null
@@ -1,880 +0,0 @@
-// 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.
-
-package proto
-
-// Functions for parsing the Text protocol buffer format.
-// TODO: message sets.
-
-import (
-	"encoding"
-	"errors"
-	"fmt"
-	"reflect"
-	"strconv"
-	"strings"
-	"unicode/utf8"
-)
-
-// Error string emitted when deserializing Any and fields are already set
-const anyRepeatedlyUnpacked = "Any message unpacked multiple times, or %q already set"
-
-type ParseError struct {
-	Message string
-	Line    int // 1-based line number
-	Offset  int // 0-based byte offset from start of input
-}
-
-func (p *ParseError) Error() string {
-	if p.Line == 1 {
-		// show offset only for first line
-		return fmt.Sprintf("line 1.%d: %v", p.Offset, p.Message)
-	}
-	return fmt.Sprintf("line %d: %v", p.Line, p.Message)
-}
-
-type token struct {
-	value    string
-	err      *ParseError
-	line     int    // line number
-	offset   int    // byte number from start of input, not start of line
-	unquoted string // the unquoted version of value, if it was a quoted string
-}
-
-func (t *token) String() string {
-	if t.err == nil {
-		return fmt.Sprintf("%q (line=%d, offset=%d)", t.value, t.line, t.offset)
-	}
-	return fmt.Sprintf("parse error: %v", t.err)
-}
-
-type textParser struct {
-	s            string // remaining input
-	done         bool   // whether the parsing is finished (success or error)
-	backed       bool   // whether back() was called
-	offset, line int
-	cur          token
-}
-
-func newTextParser(s string) *textParser {
-	p := new(textParser)
-	p.s = s
-	p.line = 1
-	p.cur.line = 1
-	return p
-}
-
-func (p *textParser) errorf(format string, a ...interface{}) *ParseError {
-	pe := &ParseError{fmt.Sprintf(format, a...), p.cur.line, p.cur.offset}
-	p.cur.err = pe
-	p.done = true
-	return pe
-}
-
-// Numbers and identifiers are matched by [-+._A-Za-z0-9]
-func isIdentOrNumberChar(c byte) bool {
-	switch {
-	case 'A' <= c && c <= 'Z', 'a' <= c && c <= 'z':
-		return true
-	case '0' <= c && c <= '9':
-		return true
-	}
-	switch c {
-	case '-', '+', '.', '_':
-		return true
-	}
-	return false
-}
-
-func isWhitespace(c byte) bool {
-	switch c {
-	case ' ', '\t', '\n', '\r':
-		return true
-	}
-	return false
-}
-
-func isQuote(c byte) bool {
-	switch c {
-	case '"', '\'':
-		return true
-	}
-	return false
-}
-
-func (p *textParser) skipWhitespace() {
-	i := 0
-	for i < len(p.s) && (isWhitespace(p.s[i]) || p.s[i] == '#') {
-		if p.s[i] == '#' {
-			// comment; skip to end of line or input
-			for i < len(p.s) && p.s[i] != '\n' {
-				i++
-			}
-			if i == len(p.s) {
-				break
-			}
-		}
-		if p.s[i] == '\n' {
-			p.line++
-		}
-		i++
-	}
-	p.offset += i
-	p.s = p.s[i:len(p.s)]
-	if len(p.s) == 0 {
-		p.done = true
-	}
-}
-
-func (p *textParser) advance() {
-	// Skip whitespace
-	p.skipWhitespace()
-	if p.done {
-		return
-	}
-
-	// Start of non-whitespace
-	p.cur.err = nil
-	p.cur.offset, p.cur.line = p.offset, p.line
-	p.cur.unquoted = ""
-	switch p.s[0] {
-	case '<', '>', '{', '}', ':', '[', ']', ';', ',', '/':
-		// Single symbol
-		p.cur.value, p.s = p.s[0:1], p.s[1:len(p.s)]
-	case '"', '\'':
-		// Quoted string
-		i := 1
-		for i < len(p.s) && p.s[i] != p.s[0] && p.s[i] != '\n' {
-			if p.s[i] == '\\' && i+1 < len(p.s) {
-				// skip escaped char
-				i++
-			}
-			i++
-		}
-		if i >= len(p.s) || p.s[i] != p.s[0] {
-			p.errorf("unmatched quote")
-			return
-		}
-		unq, err := unquoteC(p.s[1:i], rune(p.s[0]))
-		if err != nil {
-			p.errorf("invalid quoted string %s: %v", p.s[0:i+1], err)
-			return
-		}
-		p.cur.value, p.s = p.s[0:i+1], p.s[i+1:len(p.s)]
-		p.cur.unquoted = unq
-	default:
-		i := 0
-		for i < len(p.s) && isIdentOrNumberChar(p.s[i]) {
-			i++
-		}
-		if i == 0 {
-			p.errorf("unexpected byte %#x", p.s[0])
-			return
-		}
-		p.cur.value, p.s = p.s[0:i], p.s[i:len(p.s)]
-	}
-	p.offset += len(p.cur.value)
-}
-
-var (
-	errBadUTF8 = errors.New("proto: bad UTF-8")
-)
-
-func unquoteC(s string, quote rune) (string, error) {
-	// This is based on C++'s tokenizer.cc.
-	// Despite its name, this is *not* parsing C syntax.
-	// For instance, "\0" is an invalid quoted string.
-
-	// Avoid allocation in trivial cases.
-	simple := true
-	for _, r := range s {
-		if r == '\\' || r == quote {
-			simple = false
-			break
-		}
-	}
-	if simple {
-		return s, nil
-	}
-
-	buf := make([]byte, 0, 3*len(s)/2)
-	for len(s) > 0 {
-		r, n := utf8.DecodeRuneInString(s)
-		if r == utf8.RuneError && n == 1 {
-			return "", errBadUTF8
-		}
-		s = s[n:]
-		if r != '\\' {
-			if r < utf8.RuneSelf {
-				buf = append(buf, byte(r))
-			} else {
-				buf = append(buf, string(r)...)
-			}
-			continue
-		}
-
-		ch, tail, err := unescape(s)
-		if err != nil {
-			return "", err
-		}
-		buf = append(buf, ch...)
-		s = tail
-	}
-	return string(buf), nil
-}
-
-func unescape(s string) (ch string, tail string, err error) {
-	r, n := utf8.DecodeRuneInString(s)
-	if r == utf8.RuneError && n == 1 {
-		return "", "", errBadUTF8
-	}
-	s = s[n:]
-	switch r {
-	case 'a':
-		return "\a", s, nil
-	case 'b':
-		return "\b", s, nil
-	case 'f':
-		return "\f", s, nil
-	case 'n':
-		return "\n", s, nil
-	case 'r':
-		return "\r", s, nil
-	case 't':
-		return "\t", s, nil
-	case 'v':
-		return "\v", s, nil
-	case '?':
-		return "?", s, nil // trigraph workaround
-	case '\'', '"', '\\':
-		return string(r), s, nil
-	case '0', '1', '2', '3', '4', '5', '6', '7':
-		if len(s) < 2 {
-			return "", "", fmt.Errorf(`\%c requires 2 following digits`, r)
-		}
-		ss := string(r) + s[:2]
-		s = s[2:]
-		i, err := strconv.ParseUint(ss, 8, 8)
-		if err != nil {
-			return "", "", fmt.Errorf(`\%s contains non-octal digits`, ss)
-		}
-		return string([]byte{byte(i)}), s, nil
-	case 'x', 'X', 'u', 'U':
-		var n int
-		switch r {
-		case 'x', 'X':
-			n = 2
-		case 'u':
-			n = 4
-		case 'U':
-			n = 8
-		}
-		if len(s) < n {
-			return "", "", fmt.Errorf(`\%c requires %d following digits`, r, n)
-		}
-		ss := s[:n]
-		s = s[n:]
-		i, err := strconv.ParseUint(ss, 16, 64)
-		if err != nil {
-			return "", "", fmt.Errorf(`\%c%s contains non-hexadecimal digits`, r, ss)
-		}
-		if r == 'x' || r == 'X' {
-			return string([]byte{byte(i)}), s, nil
-		}
-		if i > utf8.MaxRune {
-			return "", "", fmt.Errorf(`\%c%s is not a valid Unicode code point`, r, ss)
-		}
-		return string(i), s, nil
-	}
-	return "", "", fmt.Errorf(`unknown escape \%c`, r)
-}
-
-// Back off the parser by one token. Can only be done between calls to next().
-// It makes the next advance() a no-op.
-func (p *textParser) back() { p.backed = true }
-
-// Advances the parser and returns the new current token.
-func (p *textParser) next() *token {
-	if p.backed || p.done {
-		p.backed = false
-		return &p.cur
-	}
-	p.advance()
-	if p.done {
-		p.cur.value = ""
-	} else if len(p.cur.value) > 0 && isQuote(p.cur.value[0]) {
-		// Look for multiple quoted strings separated by whitespace,
-		// and concatenate them.
-		cat := p.cur
-		for {
-			p.skipWhitespace()
-			if p.done || !isQuote(p.s[0]) {
-				break
-			}
-			p.advance()
-			if p.cur.err != nil {
-				return &p.cur
-			}
-			cat.value += " " + p.cur.value
-			cat.unquoted += p.cur.unquoted
-		}
-		p.done = false // parser may have seen EOF, but we want to return cat
-		p.cur = cat
-	}
-	return &p.cur
-}
-
-func (p *textParser) consumeToken(s string) error {
-	tok := p.next()
-	if tok.err != nil {
-		return tok.err
-	}
-	if tok.value != s {
-		p.back()
-		return p.errorf("expected %q, found %q", s, tok.value)
-	}
-	return nil
-}
-
-// Return a RequiredNotSetError indicating which required field was not set.
-func (p *textParser) missingRequiredFieldError(sv reflect.Value) *RequiredNotSetError {
-	st := sv.Type()
-	sprops := GetProperties(st)
-	for i := 0; i < st.NumField(); i++ {
-		if !isNil(sv.Field(i)) {
-			continue
-		}
-
-		props := sprops.Prop[i]
-		if props.Required {
-			return &RequiredNotSetError{fmt.Sprintf("%v.%v", st, props.OrigName)}
-		}
-	}
-	return &RequiredNotSetError{fmt.Sprintf("%v.<unknown field name>", st)} // should not happen
-}
-
-// Returns the index in the struct for the named field, as well as the parsed tag properties.
-func structFieldByName(sprops *StructProperties, name string) (int, *Properties, bool) {
-	i, ok := sprops.decoderOrigNames[name]
-	if ok {
-		return i, sprops.Prop[i], true
-	}
-	return -1, nil, false
-}
-
-// Consume a ':' from the input stream (if the next token is a colon),
-// returning an error if a colon is needed but not present.
-func (p *textParser) checkForColon(props *Properties, typ reflect.Type) *ParseError {
-	tok := p.next()
-	if tok.err != nil {
-		return tok.err
-	}
-	if tok.value != ":" {
-		// Colon is optional when the field is a group or message.
-		needColon := true
-		switch props.Wire {
-		case "group":
-			needColon = false
-		case "bytes":
-			// A "bytes" field is either a message, a string, or a repeated field;
-			// those three become *T, *string and []T respectively, so we can check for
-			// this field being a pointer to a non-string.
-			if typ.Kind() == reflect.Ptr {
-				// *T or *string
-				if typ.Elem().Kind() == reflect.String {
-					break
-				}
-			} else if typ.Kind() == reflect.Slice {
-				// []T or []*T
-				if typ.Elem().Kind() != reflect.Ptr {
-					break
-				}
-			} else if typ.Kind() == reflect.String {
-				// The proto3 exception is for a string field,
-				// which requires a colon.
-				break
-			}
-			needColon = false
-		}
-		if needColon {
-			return p.errorf("expected ':', found %q", tok.value)
-		}
-		p.back()
-	}
-	return nil
-}
-
-func (p *textParser) readStruct(sv reflect.Value, terminator string) error {
-	st := sv.Type()
-	sprops := GetProperties(st)
-	reqCount := sprops.reqCount
-	var reqFieldErr error
-	fieldSet := make(map[string]bool)
-	// A struct is a sequence of "name: value", terminated by one of
-	// '>' or '}', or the end of the input.  A name may also be
-	// "[extension]" or "[type/url]".
-	//
-	// The whole struct can also be an expanded Any message, like:
-	// [type/url] < ... struct contents ... >
-	for {
-		tok := p.next()
-		if tok.err != nil {
-			return tok.err
-		}
-		if tok.value == terminator {
-			break
-		}
-		if tok.value == "[" {
-			// Looks like an extension or an Any.
-			//
-			// TODO: Check whether we need to handle
-			// namespace rooted names (e.g. ".something.Foo").
-			extName, err := p.consumeExtName()
-			if err != nil {
-				return err
-			}
-
-			if s := strings.LastIndex(extName, "/"); s >= 0 {
-				// If it contains a slash, it's an Any type URL.
-				messageName := extName[s+1:]
-				mt := MessageType(messageName)
-				if mt == nil {
-					return p.errorf("unrecognized message %q in google.protobuf.Any", messageName)
-				}
-				tok = p.next()
-				if tok.err != nil {
-					return tok.err
-				}
-				// consume an optional colon
-				if tok.value == ":" {
-					tok = p.next()
-					if tok.err != nil {
-						return tok.err
-					}
-				}
-				var terminator string
-				switch tok.value {
-				case "<":
-					terminator = ">"
-				case "{":
-					terminator = "}"
-				default:
-					return p.errorf("expected '{' or '<', found %q", tok.value)
-				}
-				v := reflect.New(mt.Elem())
-				if pe := p.readStruct(v.Elem(), terminator); pe != nil {
-					return pe
-				}
-				b, err := Marshal(v.Interface().(Message))
-				if err != nil {
-					return p.errorf("failed to marshal message of type %q: %v", messageName, err)
-				}
-				if fieldSet["type_url"] {
-					return p.errorf(anyRepeatedlyUnpacked, "type_url")
-				}
-				if fieldSet["value"] {
-					return p.errorf(anyRepeatedlyUnpacked, "value")
-				}
-				sv.FieldByName("TypeUrl").SetString(extName)
-				sv.FieldByName("Value").SetBytes(b)
-				fieldSet["type_url"] = true
-				fieldSet["value"] = true
-				continue
-			}
-
-			var desc *ExtensionDesc
-			// This could be faster, but it's functional.
-			// TODO: Do something smarter than a linear scan.
-			for _, d := range RegisteredExtensions(reflect.New(st).Interface().(Message)) {
-				if d.Name == extName {
-					desc = d
-					break
-				}
-			}
-			if desc == nil {
-				return p.errorf("unrecognized extension %q", extName)
-			}
-
-			props := &Properties{}
-			props.Parse(desc.Tag)
-
-			typ := reflect.TypeOf(desc.ExtensionType)
-			if err := p.checkForColon(props, typ); err != nil {
-				return err
-			}
-
-			rep := desc.repeated()
-
-			// Read the extension structure, and set it in
-			// the value we're constructing.
-			var ext reflect.Value
-			if !rep {
-				ext = reflect.New(typ).Elem()
-			} else {
-				ext = reflect.New(typ.Elem()).Elem()
-			}
-			if err := p.readAny(ext, props); err != nil {
-				if _, ok := err.(*RequiredNotSetError); !ok {
-					return err
-				}
-				reqFieldErr = err
-			}
-			ep := sv.Addr().Interface().(Message)
-			if !rep {
-				SetExtension(ep, desc, ext.Interface())
-			} else {
-				old, err := GetExtension(ep, desc)
-				var sl reflect.Value
-				if err == nil {
-					sl = reflect.ValueOf(old) // existing slice
-				} else {
-					sl = reflect.MakeSlice(typ, 0, 1)
-				}
-				sl = reflect.Append(sl, ext)
-				SetExtension(ep, desc, sl.Interface())
-			}
-			if err := p.consumeOptionalSeparator(); err != nil {
-				return err
-			}
-			continue
-		}
-
-		// This is a normal, non-extension field.
-		name := tok.value
-		var dst reflect.Value
-		fi, props, ok := structFieldByName(sprops, name)
-		if ok {
-			dst = sv.Field(fi)
-		} else if oop, ok := sprops.OneofTypes[name]; ok {
-			// It is a oneof.
-			props = oop.Prop
-			nv := reflect.New(oop.Type.Elem())
-			dst = nv.Elem().Field(0)
-			field := sv.Field(oop.Field)
-			if !field.IsNil() {
-				return p.errorf("field '%s' would overwrite already parsed oneof '%s'", name, sv.Type().Field(oop.Field).Name)
-			}
-			field.Set(nv)
-		}
-		if !dst.IsValid() {
-			return p.errorf("unknown field name %q in %v", name, st)
-		}
-
-		if dst.Kind() == reflect.Map {
-			// Consume any colon.
-			if err := p.checkForColon(props, dst.Type()); err != nil {
-				return err
-			}
-
-			// Construct the map if it doesn't already exist.
-			if dst.IsNil() {
-				dst.Set(reflect.MakeMap(dst.Type()))
-			}
-			key := reflect.New(dst.Type().Key()).Elem()
-			val := reflect.New(dst.Type().Elem()).Elem()
-
-			// The map entry should be this sequence of tokens:
-			//	< key : KEY value : VALUE >
-			// However, implementations may omit key or value, and technically
-			// we should support them in any order.  See b/28924776 for a time
-			// this went wrong.
-
-			tok := p.next()
-			var terminator string
-			switch tok.value {
-			case "<":
-				terminator = ">"
-			case "{":
-				terminator = "}"
-			default:
-				return p.errorf("expected '{' or '<', found %q", tok.value)
-			}
-			for {
-				tok := p.next()
-				if tok.err != nil {
-					return tok.err
-				}
-				if tok.value == terminator {
-					break
-				}
-				switch tok.value {
-				case "key":
-					if err := p.consumeToken(":"); err != nil {
-						return err
-					}
-					if err := p.readAny(key, props.MapKeyProp); err != nil {
-						return err
-					}
-					if err := p.consumeOptionalSeparator(); err != nil {
-						return err
-					}
-				case "value":
-					if err := p.checkForColon(props.MapValProp, dst.Type().Elem()); err != nil {
-						return err
-					}
-					if err := p.readAny(val, props.MapValProp); err != nil {
-						return err
-					}
-					if err := p.consumeOptionalSeparator(); err != nil {
-						return err
-					}
-				default:
-					p.back()
-					return p.errorf(`expected "key", "value", or %q, found %q`, terminator, tok.value)
-				}
-			}
-
-			dst.SetMapIndex(key, val)
-			continue
-		}
-
-		// Check that it's not already set if it's not a repeated field.
-		if !props.Repeated && fieldSet[name] {
-			return p.errorf("non-repeated field %q was repeated", name)
-		}
-
-		if err := p.checkForColon(props, dst.Type()); err != nil {
-			return err
-		}
-
-		// Parse into the field.
-		fieldSet[name] = true
-		if err := p.readAny(dst, props); err != nil {
-			if _, ok := err.(*RequiredNotSetError); !ok {
-				return err
-			}
-			reqFieldErr = err
-		}
-		if props.Required {
-			reqCount--
-		}
-
-		if err := p.consumeOptionalSeparator(); err != nil {
-			return err
-		}
-
-	}
-
-	if reqCount > 0 {
-		return p.missingRequiredFieldError(sv)
-	}
-	return reqFieldErr
-}
-
-// consumeExtName consumes extension name or expanded Any type URL and the
-// following ']'. It returns the name or URL consumed.
-func (p *textParser) consumeExtName() (string, error) {
-	tok := p.next()
-	if tok.err != nil {
-		return "", tok.err
-	}
-
-	// If extension name or type url is quoted, it's a single token.
-	if len(tok.value) > 2 && isQuote(tok.value[0]) && tok.value[len(tok.value)-1] == tok.value[0] {
-		name, err := unquoteC(tok.value[1:len(tok.value)-1], rune(tok.value[0]))
-		if err != nil {
-			return "", err
-		}
-		return name, p.consumeToken("]")
-	}
-
-	// Consume everything up to "]"
-	var parts []string
-	for tok.value != "]" {
-		parts = append(parts, tok.value)
-		tok = p.next()
-		if tok.err != nil {
-			return "", p.errorf("unrecognized type_url or extension name: %s", tok.err)
-		}
-		if p.done && tok.value != "]" {
-			return "", p.errorf("unclosed type_url or extension name")
-		}
-	}
-	return strings.Join(parts, ""), nil
-}
-
-// consumeOptionalSeparator consumes an optional semicolon or comma.
-// It is used in readStruct to provide backward compatibility.
-func (p *textParser) consumeOptionalSeparator() error {
-	tok := p.next()
-	if tok.err != nil {
-		return tok.err
-	}
-	if tok.value != ";" && tok.value != "," {
-		p.back()
-	}
-	return nil
-}
-
-func (p *textParser) readAny(v reflect.Value, props *Properties) error {
-	tok := p.next()
-	if tok.err != nil {
-		return tok.err
-	}
-	if tok.value == "" {
-		return p.errorf("unexpected EOF")
-	}
-
-	switch fv := v; fv.Kind() {
-	case reflect.Slice:
-		at := v.Type()
-		if at.Elem().Kind() == reflect.Uint8 {
-			// Special case for []byte
-			if tok.value[0] != '"' && tok.value[0] != '\'' {
-				// Deliberately written out here, as the error after
-				// this switch statement would write "invalid []byte: ...",
-				// which is not as user-friendly.
-				return p.errorf("invalid string: %v", tok.value)
-			}
-			bytes := []byte(tok.unquoted)
-			fv.Set(reflect.ValueOf(bytes))
-			return nil
-		}
-		// Repeated field.
-		if tok.value == "[" {
-			// Repeated field with list notation, like [1,2,3].
-			for {
-				fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem()))
-				err := p.readAny(fv.Index(fv.Len()-1), props)
-				if err != nil {
-					return err
-				}
-				tok := p.next()
-				if tok.err != nil {
-					return tok.err
-				}
-				if tok.value == "]" {
-					break
-				}
-				if tok.value != "," {
-					return p.errorf("Expected ']' or ',' found %q", tok.value)
-				}
-			}
-			return nil
-		}
-		// One value of the repeated field.
-		p.back()
-		fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem()))
-		return p.readAny(fv.Index(fv.Len()-1), props)
-	case reflect.Bool:
-		// true/1/t/True or false/f/0/False.
-		switch tok.value {
-		case "true", "1", "t", "True":
-			fv.SetBool(true)
-			return nil
-		case "false", "0", "f", "False":
-			fv.SetBool(false)
-			return nil
-		}
-	case reflect.Float32, reflect.Float64:
-		v := tok.value
-		// Ignore 'f' for compatibility with output generated by C++, but don't
-		// remove 'f' when the value is "-inf" or "inf".
-		if strings.HasSuffix(v, "f") && tok.value != "-inf" && tok.value != "inf" {
-			v = v[:len(v)-1]
-		}
-		if f, err := strconv.ParseFloat(v, fv.Type().Bits()); err == nil {
-			fv.SetFloat(f)
-			return nil
-		}
-	case reflect.Int32:
-		if x, err := strconv.ParseInt(tok.value, 0, 32); err == nil {
-			fv.SetInt(x)
-			return nil
-		}
-
-		if len(props.Enum) == 0 {
-			break
-		}
-		m, ok := enumValueMaps[props.Enum]
-		if !ok {
-			break
-		}
-		x, ok := m[tok.value]
-		if !ok {
-			break
-		}
-		fv.SetInt(int64(x))
-		return nil
-	case reflect.Int64:
-		if x, err := strconv.ParseInt(tok.value, 0, 64); err == nil {
-			fv.SetInt(x)
-			return nil
-		}
-
-	case reflect.Ptr:
-		// A basic field (indirected through pointer), or a repeated message/group
-		p.back()
-		fv.Set(reflect.New(fv.Type().Elem()))
-		return p.readAny(fv.Elem(), props)
-	case reflect.String:
-		if tok.value[0] == '"' || tok.value[0] == '\'' {
-			fv.SetString(tok.unquoted)
-			return nil
-		}
-	case reflect.Struct:
-		var terminator string
-		switch tok.value {
-		case "{":
-			terminator = "}"
-		case "<":
-			terminator = ">"
-		default:
-			return p.errorf("expected '{' or '<', found %q", tok.value)
-		}
-		// TODO: Handle nested messages which implement encoding.TextUnmarshaler.
-		return p.readStruct(fv, terminator)
-	case reflect.Uint32:
-		if x, err := strconv.ParseUint(tok.value, 0, 32); err == nil {
-			fv.SetUint(uint64(x))
-			return nil
-		}
-	case reflect.Uint64:
-		if x, err := strconv.ParseUint(tok.value, 0, 64); err == nil {
-			fv.SetUint(x)
-			return nil
-		}
-	}
-	return p.errorf("invalid %v: %v", v.Type(), tok.value)
-}
-
-// UnmarshalText reads a protocol buffer in Text format. UnmarshalText resets pb
-// before starting to unmarshal, so any existing data in pb is always removed.
-// If a required field is not set and no other error occurs,
-// UnmarshalText returns *RequiredNotSetError.
-func UnmarshalText(s string, pb Message) error {
-	if um, ok := pb.(encoding.TextUnmarshaler); ok {
-		return um.UnmarshalText([]byte(s))
-	}
-	pb.Reset()
-	v := reflect.ValueOf(pb)
-	return newTextParser(s).readStruct(v.Elem(), "")
-}
diff --git a/vendor/github.com/golang/protobuf/proto/wire.go b/vendor/github.com/golang/protobuf/proto/wire.go
new file mode 100644
index 0000000..d7c28da
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/wire.go
@@ -0,0 +1,78 @@
+// 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.
+
+package proto
+
+import (
+	protoV2 "google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// Size returns the size in bytes of the wire-format encoding of m.
+func Size(m Message) int {
+	if m == nil {
+		return 0
+	}
+	mi := MessageV2(m)
+	return protoV2.Size(mi)
+}
+
+// Marshal returns the wire-format encoding of m.
+func Marshal(m Message) ([]byte, error) {
+	b, err := marshalAppend(nil, m, false)
+	if b == nil {
+		b = zeroBytes
+	}
+	return b, err
+}
+
+var zeroBytes = make([]byte, 0, 0)
+
+func marshalAppend(buf []byte, m Message, deterministic bool) ([]byte, error) {
+	if m == nil {
+		return nil, ErrNil
+	}
+	mi := MessageV2(m)
+	nbuf, err := protoV2.MarshalOptions{
+		Deterministic: deterministic,
+		AllowPartial:  true,
+	}.MarshalAppend(buf, mi)
+	if err != nil {
+		return buf, err
+	}
+	if len(buf) == len(nbuf) {
+		if !mi.ProtoReflect().IsValid() {
+			return buf, ErrNil
+		}
+	}
+	return nbuf, checkRequiredNotSet(mi)
+}
+
+// Unmarshal parses a wire-format message in b and places the decoded results in m.
+//
+// Unmarshal resets m before starting to unmarshal, so any existing data in m is always
+// removed. Use UnmarshalMerge to preserve and append to existing data.
+func Unmarshal(b []byte, m Message) error {
+	m.Reset()
+	return UnmarshalMerge(b, m)
+}
+
+// UnmarshalMerge parses a wire-format message in b and places the decoded results in m.
+func UnmarshalMerge(b []byte, m Message) error {
+	mi := MessageV2(m)
+	out, err := protoV2.UnmarshalOptions{
+		AllowPartial: true,
+		Merge:        true,
+	}.UnmarshalState(protoiface.UnmarshalInput{
+		Buf:     b,
+		Message: mi.ProtoReflect(),
+	})
+	if err != nil {
+		return err
+	}
+	if out.Flags&protoiface.UnmarshalInitialized > 0 {
+		return nil
+	}
+	return checkRequiredNotSet(mi)
+}
diff --git a/vendor/github.com/golang/protobuf/proto/wrappers.go b/vendor/github.com/golang/protobuf/proto/wrappers.go
new file mode 100644
index 0000000..398e348
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/proto/wrappers.go
@@ -0,0 +1,34 @@
+// 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.
+
+package proto
+
+// Bool stores v in a new bool value and returns a pointer to it.
+func Bool(v bool) *bool { return &v }
+
+// Int stores v in a new int32 value and returns a pointer to it.
+//
+// Deprecated: Use Int32 instead.
+func Int(v int) *int32 { return Int32(int32(v)) }
+
+// Int32 stores v in a new int32 value and returns a pointer to it.
+func Int32(v int32) *int32 { return &v }
+
+// Int64 stores v in a new int64 value and returns a pointer to it.
+func Int64(v int64) *int64 { return &v }
+
+// Uint32 stores v in a new uint32 value and returns a pointer to it.
+func Uint32(v uint32) *uint32 { return &v }
+
+// Uint64 stores v in a new uint64 value and returns a pointer to it.
+func Uint64(v uint64) *uint64 { return &v }
+
+// Float32 stores v in a new float32 value and returns a pointer to it.
+func Float32(v float32) *float32 { return &v }
+
+// Float64 stores v in a new float64 value and returns a pointer to it.
+func Float64(v float64) *float64 { return &v }
+
+// String stores v in a new string value and returns a pointer to it.
+func String(v string) *string { return &v }
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.pb.go b/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.pb.go
index 1ded05b..63dc057 100644
--- a/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.pb.go
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.pb.go
@@ -1,2887 +1,200 @@
 // Code generated by protoc-gen-go. DO NOT EDIT.
-// source: google/protobuf/descriptor.proto
+// source: github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.proto
 
 package descriptor
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
+	reflect "reflect"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+// Symbols defined in public import of google/protobuf/descriptor.proto.
 
-// 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 FieldDescriptorProto_Type = descriptorpb.FieldDescriptorProto_Type
 
-type FieldDescriptorProto_Type int32
+const FieldDescriptorProto_TYPE_DOUBLE = descriptorpb.FieldDescriptorProto_TYPE_DOUBLE
+const FieldDescriptorProto_TYPE_FLOAT = descriptorpb.FieldDescriptorProto_TYPE_FLOAT
+const FieldDescriptorProto_TYPE_INT64 = descriptorpb.FieldDescriptorProto_TYPE_INT64
+const FieldDescriptorProto_TYPE_UINT64 = descriptorpb.FieldDescriptorProto_TYPE_UINT64
+const FieldDescriptorProto_TYPE_INT32 = descriptorpb.FieldDescriptorProto_TYPE_INT32
+const FieldDescriptorProto_TYPE_FIXED64 = descriptorpb.FieldDescriptorProto_TYPE_FIXED64
+const FieldDescriptorProto_TYPE_FIXED32 = descriptorpb.FieldDescriptorProto_TYPE_FIXED32
+const FieldDescriptorProto_TYPE_BOOL = descriptorpb.FieldDescriptorProto_TYPE_BOOL
+const FieldDescriptorProto_TYPE_STRING = descriptorpb.FieldDescriptorProto_TYPE_STRING
+const FieldDescriptorProto_TYPE_GROUP = descriptorpb.FieldDescriptorProto_TYPE_GROUP
+const FieldDescriptorProto_TYPE_MESSAGE = descriptorpb.FieldDescriptorProto_TYPE_MESSAGE
+const FieldDescriptorProto_TYPE_BYTES = descriptorpb.FieldDescriptorProto_TYPE_BYTES
+const FieldDescriptorProto_TYPE_UINT32 = descriptorpb.FieldDescriptorProto_TYPE_UINT32
+const FieldDescriptorProto_TYPE_ENUM = descriptorpb.FieldDescriptorProto_TYPE_ENUM
+const FieldDescriptorProto_TYPE_SFIXED32 = descriptorpb.FieldDescriptorProto_TYPE_SFIXED32
+const FieldDescriptorProto_TYPE_SFIXED64 = descriptorpb.FieldDescriptorProto_TYPE_SFIXED64
+const FieldDescriptorProto_TYPE_SINT32 = descriptorpb.FieldDescriptorProto_TYPE_SINT32
+const FieldDescriptorProto_TYPE_SINT64 = descriptorpb.FieldDescriptorProto_TYPE_SINT64
 
-const (
-	// 0 is reserved for errors.
-	// Order is weird for historical reasons.
-	FieldDescriptorProto_TYPE_DOUBLE FieldDescriptorProto_Type = 1
-	FieldDescriptorProto_TYPE_FLOAT  FieldDescriptorProto_Type = 2
-	// Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT64 if
-	// negative values are likely.
-	FieldDescriptorProto_TYPE_INT64  FieldDescriptorProto_Type = 3
-	FieldDescriptorProto_TYPE_UINT64 FieldDescriptorProto_Type = 4
-	// Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT32 if
-	// negative values are likely.
-	FieldDescriptorProto_TYPE_INT32   FieldDescriptorProto_Type = 5
-	FieldDescriptorProto_TYPE_FIXED64 FieldDescriptorProto_Type = 6
-	FieldDescriptorProto_TYPE_FIXED32 FieldDescriptorProto_Type = 7
-	FieldDescriptorProto_TYPE_BOOL    FieldDescriptorProto_Type = 8
-	FieldDescriptorProto_TYPE_STRING  FieldDescriptorProto_Type = 9
-	// Tag-delimited aggregate.
-	// Group type is deprecated and not supported in proto3. However, Proto3
-	// implementations should still be able to parse the group wire format and
-	// treat group fields as unknown fields.
-	FieldDescriptorProto_TYPE_GROUP   FieldDescriptorProto_Type = 10
-	FieldDescriptorProto_TYPE_MESSAGE FieldDescriptorProto_Type = 11
-	// New in version 2.
-	FieldDescriptorProto_TYPE_BYTES    FieldDescriptorProto_Type = 12
-	FieldDescriptorProto_TYPE_UINT32   FieldDescriptorProto_Type = 13
-	FieldDescriptorProto_TYPE_ENUM     FieldDescriptorProto_Type = 14
-	FieldDescriptorProto_TYPE_SFIXED32 FieldDescriptorProto_Type = 15
-	FieldDescriptorProto_TYPE_SFIXED64 FieldDescriptorProto_Type = 16
-	FieldDescriptorProto_TYPE_SINT32   FieldDescriptorProto_Type = 17
-	FieldDescriptorProto_TYPE_SINT64   FieldDescriptorProto_Type = 18
-)
+var FieldDescriptorProto_Type_name = descriptorpb.FieldDescriptorProto_Type_name
+var FieldDescriptorProto_Type_value = descriptorpb.FieldDescriptorProto_Type_value
 
-var FieldDescriptorProto_Type_name = map[int32]string{
-	1:  "TYPE_DOUBLE",
-	2:  "TYPE_FLOAT",
-	3:  "TYPE_INT64",
-	4:  "TYPE_UINT64",
-	5:  "TYPE_INT32",
-	6:  "TYPE_FIXED64",
-	7:  "TYPE_FIXED32",
-	8:  "TYPE_BOOL",
-	9:  "TYPE_STRING",
-	10: "TYPE_GROUP",
-	11: "TYPE_MESSAGE",
-	12: "TYPE_BYTES",
-	13: "TYPE_UINT32",
-	14: "TYPE_ENUM",
-	15: "TYPE_SFIXED32",
-	16: "TYPE_SFIXED64",
-	17: "TYPE_SINT32",
-	18: "TYPE_SINT64",
-}
+type FieldDescriptorProto_Label = descriptorpb.FieldDescriptorProto_Label
 
-var FieldDescriptorProto_Type_value = map[string]int32{
-	"TYPE_DOUBLE":   1,
-	"TYPE_FLOAT":    2,
-	"TYPE_INT64":    3,
-	"TYPE_UINT64":   4,
-	"TYPE_INT32":    5,
-	"TYPE_FIXED64":  6,
-	"TYPE_FIXED32":  7,
-	"TYPE_BOOL":     8,
-	"TYPE_STRING":   9,
-	"TYPE_GROUP":    10,
-	"TYPE_MESSAGE":  11,
-	"TYPE_BYTES":    12,
-	"TYPE_UINT32":   13,
-	"TYPE_ENUM":     14,
-	"TYPE_SFIXED32": 15,
-	"TYPE_SFIXED64": 16,
-	"TYPE_SINT32":   17,
-	"TYPE_SINT64":   18,
-}
+const FieldDescriptorProto_LABEL_OPTIONAL = descriptorpb.FieldDescriptorProto_LABEL_OPTIONAL
+const FieldDescriptorProto_LABEL_REQUIRED = descriptorpb.FieldDescriptorProto_LABEL_REQUIRED
+const FieldDescriptorProto_LABEL_REPEATED = descriptorpb.FieldDescriptorProto_LABEL_REPEATED
 
-func (x FieldDescriptorProto_Type) Enum() *FieldDescriptorProto_Type {
-	p := new(FieldDescriptorProto_Type)
-	*p = x
-	return p
-}
+var FieldDescriptorProto_Label_name = descriptorpb.FieldDescriptorProto_Label_name
+var FieldDescriptorProto_Label_value = descriptorpb.FieldDescriptorProto_Label_value
 
-func (x FieldDescriptorProto_Type) String() string {
-	return proto.EnumName(FieldDescriptorProto_Type_name, int32(x))
-}
+type FileOptions_OptimizeMode = descriptorpb.FileOptions_OptimizeMode
 
-func (x *FieldDescriptorProto_Type) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(FieldDescriptorProto_Type_value, data, "FieldDescriptorProto_Type")
-	if err != nil {
-		return err
-	}
-	*x = FieldDescriptorProto_Type(value)
-	return nil
-}
+const FileOptions_SPEED = descriptorpb.FileOptions_SPEED
+const FileOptions_CODE_SIZE = descriptorpb.FileOptions_CODE_SIZE
+const FileOptions_LITE_RUNTIME = descriptorpb.FileOptions_LITE_RUNTIME
 
-func (FieldDescriptorProto_Type) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{4, 0}
-}
+var FileOptions_OptimizeMode_name = descriptorpb.FileOptions_OptimizeMode_name
+var FileOptions_OptimizeMode_value = descriptorpb.FileOptions_OptimizeMode_value
 
-type FieldDescriptorProto_Label int32
+type FieldOptions_CType = descriptorpb.FieldOptions_CType
 
-const (
-	// 0 is reserved for errors
-	FieldDescriptorProto_LABEL_OPTIONAL FieldDescriptorProto_Label = 1
-	FieldDescriptorProto_LABEL_REQUIRED FieldDescriptorProto_Label = 2
-	FieldDescriptorProto_LABEL_REPEATED FieldDescriptorProto_Label = 3
-)
+const FieldOptions_STRING = descriptorpb.FieldOptions_STRING
+const FieldOptions_CORD = descriptorpb.FieldOptions_CORD
+const FieldOptions_STRING_PIECE = descriptorpb.FieldOptions_STRING_PIECE
 
-var FieldDescriptorProto_Label_name = map[int32]string{
-	1: "LABEL_OPTIONAL",
-	2: "LABEL_REQUIRED",
-	3: "LABEL_REPEATED",
-}
+var FieldOptions_CType_name = descriptorpb.FieldOptions_CType_name
+var FieldOptions_CType_value = descriptorpb.FieldOptions_CType_value
 
-var FieldDescriptorProto_Label_value = map[string]int32{
-	"LABEL_OPTIONAL": 1,
-	"LABEL_REQUIRED": 2,
-	"LABEL_REPEATED": 3,
-}
+type FieldOptions_JSType = descriptorpb.FieldOptions_JSType
 
-func (x FieldDescriptorProto_Label) Enum() *FieldDescriptorProto_Label {
-	p := new(FieldDescriptorProto_Label)
-	*p = x
-	return p
-}
+const FieldOptions_JS_NORMAL = descriptorpb.FieldOptions_JS_NORMAL
+const FieldOptions_JS_STRING = descriptorpb.FieldOptions_JS_STRING
+const FieldOptions_JS_NUMBER = descriptorpb.FieldOptions_JS_NUMBER
 
-func (x FieldDescriptorProto_Label) String() string {
-	return proto.EnumName(FieldDescriptorProto_Label_name, int32(x))
-}
+var FieldOptions_JSType_name = descriptorpb.FieldOptions_JSType_name
+var FieldOptions_JSType_value = descriptorpb.FieldOptions_JSType_value
 
-func (x *FieldDescriptorProto_Label) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(FieldDescriptorProto_Label_value, data, "FieldDescriptorProto_Label")
-	if err != nil {
-		return err
-	}
-	*x = FieldDescriptorProto_Label(value)
-	return nil
-}
+type MethodOptions_IdempotencyLevel = descriptorpb.MethodOptions_IdempotencyLevel
 
-func (FieldDescriptorProto_Label) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{4, 1}
-}
-
-// Generated classes can be optimized for speed or code size.
-type FileOptions_OptimizeMode int32
+const MethodOptions_IDEMPOTENCY_UNKNOWN = descriptorpb.MethodOptions_IDEMPOTENCY_UNKNOWN
+const MethodOptions_NO_SIDE_EFFECTS = descriptorpb.MethodOptions_NO_SIDE_EFFECTS
+const MethodOptions_IDEMPOTENT = descriptorpb.MethodOptions_IDEMPOTENT
 
-const (
-	FileOptions_SPEED FileOptions_OptimizeMode = 1
-	// etc.
-	FileOptions_CODE_SIZE    FileOptions_OptimizeMode = 2
-	FileOptions_LITE_RUNTIME FileOptions_OptimizeMode = 3
-)
+var MethodOptions_IdempotencyLevel_name = descriptorpb.MethodOptions_IdempotencyLevel_name
+var MethodOptions_IdempotencyLevel_value = descriptorpb.MethodOptions_IdempotencyLevel_value
 
-var FileOptions_OptimizeMode_name = map[int32]string{
-	1: "SPEED",
-	2: "CODE_SIZE",
-	3: "LITE_RUNTIME",
-}
+type FileDescriptorSet = descriptorpb.FileDescriptorSet
+type FileDescriptorProto = descriptorpb.FileDescriptorProto
+type DescriptorProto = descriptorpb.DescriptorProto
+type ExtensionRangeOptions = descriptorpb.ExtensionRangeOptions
+type FieldDescriptorProto = descriptorpb.FieldDescriptorProto
+type OneofDescriptorProto = descriptorpb.OneofDescriptorProto
+type EnumDescriptorProto = descriptorpb.EnumDescriptorProto
+type EnumValueDescriptorProto = descriptorpb.EnumValueDescriptorProto
+type ServiceDescriptorProto = descriptorpb.ServiceDescriptorProto
+type MethodDescriptorProto = descriptorpb.MethodDescriptorProto
 
-var FileOptions_OptimizeMode_value = map[string]int32{
-	"SPEED":        1,
-	"CODE_SIZE":    2,
-	"LITE_RUNTIME": 3,
-}
+const Default_MethodDescriptorProto_ClientStreaming = descriptorpb.Default_MethodDescriptorProto_ClientStreaming
+const Default_MethodDescriptorProto_ServerStreaming = descriptorpb.Default_MethodDescriptorProto_ServerStreaming
 
-func (x FileOptions_OptimizeMode) Enum() *FileOptions_OptimizeMode {
-	p := new(FileOptions_OptimizeMode)
-	*p = x
-	return p
-}
+type FileOptions = descriptorpb.FileOptions
 
-func (x FileOptions_OptimizeMode) String() string {
-	return proto.EnumName(FileOptions_OptimizeMode_name, int32(x))
-}
+const Default_FileOptions_JavaMultipleFiles = descriptorpb.Default_FileOptions_JavaMultipleFiles
+const Default_FileOptions_JavaStringCheckUtf8 = descriptorpb.Default_FileOptions_JavaStringCheckUtf8
+const Default_FileOptions_OptimizeFor = descriptorpb.Default_FileOptions_OptimizeFor
+const Default_FileOptions_CcGenericServices = descriptorpb.Default_FileOptions_CcGenericServices
+const Default_FileOptions_JavaGenericServices = descriptorpb.Default_FileOptions_JavaGenericServices
+const Default_FileOptions_PyGenericServices = descriptorpb.Default_FileOptions_PyGenericServices
+const Default_FileOptions_PhpGenericServices = descriptorpb.Default_FileOptions_PhpGenericServices
+const Default_FileOptions_Deprecated = descriptorpb.Default_FileOptions_Deprecated
+const Default_FileOptions_CcEnableArenas = descriptorpb.Default_FileOptions_CcEnableArenas
 
-func (x *FileOptions_OptimizeMode) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(FileOptions_OptimizeMode_value, data, "FileOptions_OptimizeMode")
-	if err != nil {
-		return err
-	}
-	*x = FileOptions_OptimizeMode(value)
-	return nil
-}
+type MessageOptions = descriptorpb.MessageOptions
 
-func (FileOptions_OptimizeMode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{10, 0}
-}
+const Default_MessageOptions_MessageSetWireFormat = descriptorpb.Default_MessageOptions_MessageSetWireFormat
+const Default_MessageOptions_NoStandardDescriptorAccessor = descriptorpb.Default_MessageOptions_NoStandardDescriptorAccessor
+const Default_MessageOptions_Deprecated = descriptorpb.Default_MessageOptions_Deprecated
 
-type FieldOptions_CType int32
+type FieldOptions = descriptorpb.FieldOptions
 
-const (
-	// Default mode.
-	FieldOptions_STRING       FieldOptions_CType = 0
-	FieldOptions_CORD         FieldOptions_CType = 1
-	FieldOptions_STRING_PIECE FieldOptions_CType = 2
-)
+const Default_FieldOptions_Ctype = descriptorpb.Default_FieldOptions_Ctype
+const Default_FieldOptions_Jstype = descriptorpb.Default_FieldOptions_Jstype
+const Default_FieldOptions_Lazy = descriptorpb.Default_FieldOptions_Lazy
+const Default_FieldOptions_Deprecated = descriptorpb.Default_FieldOptions_Deprecated
+const Default_FieldOptions_Weak = descriptorpb.Default_FieldOptions_Weak
 
-var FieldOptions_CType_name = map[int32]string{
-	0: "STRING",
-	1: "CORD",
-	2: "STRING_PIECE",
-}
+type OneofOptions = descriptorpb.OneofOptions
+type EnumOptions = descriptorpb.EnumOptions
 
-var FieldOptions_CType_value = map[string]int32{
-	"STRING":       0,
-	"CORD":         1,
-	"STRING_PIECE": 2,
-}
+const Default_EnumOptions_Deprecated = descriptorpb.Default_EnumOptions_Deprecated
 
-func (x FieldOptions_CType) Enum() *FieldOptions_CType {
-	p := new(FieldOptions_CType)
-	*p = x
-	return p
-}
+type EnumValueOptions = descriptorpb.EnumValueOptions
 
-func (x FieldOptions_CType) String() string {
-	return proto.EnumName(FieldOptions_CType_name, int32(x))
-}
+const Default_EnumValueOptions_Deprecated = descriptorpb.Default_EnumValueOptions_Deprecated
 
-func (x *FieldOptions_CType) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(FieldOptions_CType_value, data, "FieldOptions_CType")
-	if err != nil {
-		return err
-	}
-	*x = FieldOptions_CType(value)
-	return nil
-}
+type ServiceOptions = descriptorpb.ServiceOptions
 
-func (FieldOptions_CType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{12, 0}
-}
+const Default_ServiceOptions_Deprecated = descriptorpb.Default_ServiceOptions_Deprecated
 
-type FieldOptions_JSType int32
+type MethodOptions = descriptorpb.MethodOptions
 
-const (
-	// Use the default type.
-	FieldOptions_JS_NORMAL FieldOptions_JSType = 0
-	// Use JavaScript strings.
-	FieldOptions_JS_STRING FieldOptions_JSType = 1
-	// Use JavaScript numbers.
-	FieldOptions_JS_NUMBER FieldOptions_JSType = 2
-)
+const Default_MethodOptions_Deprecated = descriptorpb.Default_MethodOptions_Deprecated
+const Default_MethodOptions_IdempotencyLevel = descriptorpb.Default_MethodOptions_IdempotencyLevel
 
-var FieldOptions_JSType_name = map[int32]string{
-	0: "JS_NORMAL",
-	1: "JS_STRING",
-	2: "JS_NUMBER",
-}
+type UninterpretedOption = descriptorpb.UninterpretedOption
+type SourceCodeInfo = descriptorpb.SourceCodeInfo
+type GeneratedCodeInfo = descriptorpb.GeneratedCodeInfo
+type DescriptorProto_ExtensionRange = descriptorpb.DescriptorProto_ExtensionRange
+type DescriptorProto_ReservedRange = descriptorpb.DescriptorProto_ReservedRange
+type EnumDescriptorProto_EnumReservedRange = descriptorpb.EnumDescriptorProto_EnumReservedRange
+type UninterpretedOption_NamePart = descriptorpb.UninterpretedOption_NamePart
+type SourceCodeInfo_Location = descriptorpb.SourceCodeInfo_Location
+type GeneratedCodeInfo_Annotation = descriptorpb.GeneratedCodeInfo_Annotation
 
-var FieldOptions_JSType_value = map[string]int32{
-	"JS_NORMAL": 0,
-	"JS_STRING": 1,
-	"JS_NUMBER": 2,
-}
+var File_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto protoreflect.FileDescriptor
 
-func (x FieldOptions_JSType) Enum() *FieldOptions_JSType {
-	p := new(FieldOptions_JSType)
-	*p = x
-	return p
+var file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_rawDesc = []byte{
+	0x0a, 0x44, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x67, 0x6f, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x42, 0x40, 0x5a, 0x3e, 0x67, 0x69, 0x74, 0x68,
+	0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65,
+	0x6e, 0x2d, 0x67, 0x6f, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x3b,
+	0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x00, 0x62, 0x06, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x32,
 }
 
-func (x FieldOptions_JSType) String() string {
-	return proto.EnumName(FieldOptions_JSType_name, int32(x))
+var file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_goTypes = []interface{}{}
+var file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func (x *FieldOptions_JSType) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(FieldOptions_JSType_value, data, "FieldOptions_JSType")
-	if err != nil {
-		return err
+func init() { file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_init() }
+func file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_init() {
+	if File_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto != nil {
+		return
 	}
-	*x = FieldOptions_JSType(value)
-	return nil
-}
-
-func (FieldOptions_JSType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{12, 1}
-}
-
-// Is this method side-effect-free (or safe in HTTP parlance), or idempotent,
-// or neither? HTTP based RPC implementation may choose GET verb for safe
-// methods, and PUT verb for idempotent methods instead of the default POST.
-type MethodOptions_IdempotencyLevel int32
-
-const (
-	MethodOptions_IDEMPOTENCY_UNKNOWN MethodOptions_IdempotencyLevel = 0
-	MethodOptions_NO_SIDE_EFFECTS     MethodOptions_IdempotencyLevel = 1
-	MethodOptions_IDEMPOTENT          MethodOptions_IdempotencyLevel = 2
-)
-
-var MethodOptions_IdempotencyLevel_name = map[int32]string{
-	0: "IDEMPOTENCY_UNKNOWN",
-	1: "NO_SIDE_EFFECTS",
-	2: "IDEMPOTENT",
-}
-
-var MethodOptions_IdempotencyLevel_value = map[string]int32{
-	"IDEMPOTENCY_UNKNOWN": 0,
-	"NO_SIDE_EFFECTS":     1,
-	"IDEMPOTENT":          2,
-}
-
-func (x MethodOptions_IdempotencyLevel) Enum() *MethodOptions_IdempotencyLevel {
-	p := new(MethodOptions_IdempotencyLevel)
-	*p = x
-	return p
-}
-
-func (x MethodOptions_IdempotencyLevel) String() string {
-	return proto.EnumName(MethodOptions_IdempotencyLevel_name, int32(x))
-}
-
-func (x *MethodOptions_IdempotencyLevel) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(MethodOptions_IdempotencyLevel_value, data, "MethodOptions_IdempotencyLevel")
-	if err != nil {
-		return err
-	}
-	*x = MethodOptions_IdempotencyLevel(value)
-	return nil
-}
-
-func (MethodOptions_IdempotencyLevel) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{17, 0}
-}
-
-// The protocol compiler can output a FileDescriptorSet containing the .proto
-// files it parses.
-type FileDescriptorSet struct {
-	File                 []*FileDescriptorProto `protobuf:"bytes,1,rep,name=file" json:"file,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *FileDescriptorSet) Reset()         { *m = FileDescriptorSet{} }
-func (m *FileDescriptorSet) String() string { return proto.CompactTextString(m) }
-func (*FileDescriptorSet) ProtoMessage()    {}
-func (*FileDescriptorSet) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{0}
-}
-
-func (m *FileDescriptorSet) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FileDescriptorSet.Unmarshal(m, b)
-}
-func (m *FileDescriptorSet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FileDescriptorSet.Marshal(b, m, deterministic)
-}
-func (m *FileDescriptorSet) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FileDescriptorSet.Merge(m, src)
-}
-func (m *FileDescriptorSet) XXX_Size() int {
-	return xxx_messageInfo_FileDescriptorSet.Size(m)
-}
-func (m *FileDescriptorSet) XXX_DiscardUnknown() {
-	xxx_messageInfo_FileDescriptorSet.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FileDescriptorSet proto.InternalMessageInfo
-
-func (m *FileDescriptorSet) GetFile() []*FileDescriptorProto {
-	if m != nil {
-		return m.File
-	}
-	return nil
-}
-
-// Describes a complete .proto file.
-type FileDescriptorProto struct {
-	Name    *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Package *string `protobuf:"bytes,2,opt,name=package" json:"package,omitempty"`
-	// Names of files imported by this file.
-	Dependency []string `protobuf:"bytes,3,rep,name=dependency" json:"dependency,omitempty"`
-	// Indexes of the public imported files in the dependency list above.
-	PublicDependency []int32 `protobuf:"varint,10,rep,name=public_dependency,json=publicDependency" json:"public_dependency,omitempty"`
-	// Indexes of the weak imported files in the dependency list.
-	// For Google-internal migration only. Do not use.
-	WeakDependency []int32 `protobuf:"varint,11,rep,name=weak_dependency,json=weakDependency" json:"weak_dependency,omitempty"`
-	// All top-level definitions in this file.
-	MessageType []*DescriptorProto        `protobuf:"bytes,4,rep,name=message_type,json=messageType" json:"message_type,omitempty"`
-	EnumType    []*EnumDescriptorProto    `protobuf:"bytes,5,rep,name=enum_type,json=enumType" json:"enum_type,omitempty"`
-	Service     []*ServiceDescriptorProto `protobuf:"bytes,6,rep,name=service" json:"service,omitempty"`
-	Extension   []*FieldDescriptorProto   `protobuf:"bytes,7,rep,name=extension" json:"extension,omitempty"`
-	Options     *FileOptions              `protobuf:"bytes,8,opt,name=options" json:"options,omitempty"`
-	// This field contains optional information about the original source code.
-	// You may safely remove this entire field without harming runtime
-	// functionality of the descriptors -- the information is needed only by
-	// development tools.
-	SourceCodeInfo *SourceCodeInfo `protobuf:"bytes,9,opt,name=source_code_info,json=sourceCodeInfo" json:"source_code_info,omitempty"`
-	// The syntax of the proto file.
-	// The supported values are "proto2" and "proto3".
-	Syntax               *string  `protobuf:"bytes,12,opt,name=syntax" json:"syntax,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *FileDescriptorProto) Reset()         { *m = FileDescriptorProto{} }
-func (m *FileDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*FileDescriptorProto) ProtoMessage()    {}
-func (*FileDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{1}
-}
-
-func (m *FileDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FileDescriptorProto.Unmarshal(m, b)
-}
-func (m *FileDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FileDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *FileDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FileDescriptorProto.Merge(m, src)
-}
-func (m *FileDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_FileDescriptorProto.Size(m)
-}
-func (m *FileDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_FileDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FileDescriptorProto proto.InternalMessageInfo
-
-func (m *FileDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *FileDescriptorProto) GetPackage() string {
-	if m != nil && m.Package != nil {
-		return *m.Package
-	}
-	return ""
-}
-
-func (m *FileDescriptorProto) GetDependency() []string {
-	if m != nil {
-		return m.Dependency
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetPublicDependency() []int32 {
-	if m != nil {
-		return m.PublicDependency
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetWeakDependency() []int32 {
-	if m != nil {
-		return m.WeakDependency
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetMessageType() []*DescriptorProto {
-	if m != nil {
-		return m.MessageType
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetEnumType() []*EnumDescriptorProto {
-	if m != nil {
-		return m.EnumType
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetService() []*ServiceDescriptorProto {
-	if m != nil {
-		return m.Service
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetExtension() []*FieldDescriptorProto {
-	if m != nil {
-		return m.Extension
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetOptions() *FileOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetSourceCodeInfo() *SourceCodeInfo {
-	if m != nil {
-		return m.SourceCodeInfo
-	}
-	return nil
-}
-
-func (m *FileDescriptorProto) GetSyntax() string {
-	if m != nil && m.Syntax != nil {
-		return *m.Syntax
-	}
-	return ""
-}
-
-// Describes a message type.
-type DescriptorProto struct {
-	Name           *string                           `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Field          []*FieldDescriptorProto           `protobuf:"bytes,2,rep,name=field" json:"field,omitempty"`
-	Extension      []*FieldDescriptorProto           `protobuf:"bytes,6,rep,name=extension" json:"extension,omitempty"`
-	NestedType     []*DescriptorProto                `protobuf:"bytes,3,rep,name=nested_type,json=nestedType" json:"nested_type,omitempty"`
-	EnumType       []*EnumDescriptorProto            `protobuf:"bytes,4,rep,name=enum_type,json=enumType" json:"enum_type,omitempty"`
-	ExtensionRange []*DescriptorProto_ExtensionRange `protobuf:"bytes,5,rep,name=extension_range,json=extensionRange" json:"extension_range,omitempty"`
-	OneofDecl      []*OneofDescriptorProto           `protobuf:"bytes,8,rep,name=oneof_decl,json=oneofDecl" json:"oneof_decl,omitempty"`
-	Options        *MessageOptions                   `protobuf:"bytes,7,opt,name=options" json:"options,omitempty"`
-	ReservedRange  []*DescriptorProto_ReservedRange  `protobuf:"bytes,9,rep,name=reserved_range,json=reservedRange" json:"reserved_range,omitempty"`
-	// Reserved field names, which may not be used by fields in the same message.
-	// A given name may only be reserved once.
-	ReservedName         []string `protobuf:"bytes,10,rep,name=reserved_name,json=reservedName" json:"reserved_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DescriptorProto) Reset()         { *m = DescriptorProto{} }
-func (m *DescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*DescriptorProto) ProtoMessage()    {}
-func (*DescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{2}
-}
-
-func (m *DescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DescriptorProto.Unmarshal(m, b)
-}
-func (m *DescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *DescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DescriptorProto.Merge(m, src)
-}
-func (m *DescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_DescriptorProto.Size(m)
-}
-func (m *DescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_DescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DescriptorProto proto.InternalMessageInfo
-
-func (m *DescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *DescriptorProto) GetField() []*FieldDescriptorProto {
-	if m != nil {
-		return m.Field
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetExtension() []*FieldDescriptorProto {
-	if m != nil {
-		return m.Extension
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetNestedType() []*DescriptorProto {
-	if m != nil {
-		return m.NestedType
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetEnumType() []*EnumDescriptorProto {
-	if m != nil {
-		return m.EnumType
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetExtensionRange() []*DescriptorProto_ExtensionRange {
-	if m != nil {
-		return m.ExtensionRange
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetOneofDecl() []*OneofDescriptorProto {
-	if m != nil {
-		return m.OneofDecl
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetOptions() *MessageOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetReservedRange() []*DescriptorProto_ReservedRange {
-	if m != nil {
-		return m.ReservedRange
-	}
-	return nil
-}
-
-func (m *DescriptorProto) GetReservedName() []string {
-	if m != nil {
-		return m.ReservedName
-	}
-	return nil
-}
-
-type DescriptorProto_ExtensionRange struct {
-	Start                *int32                 `protobuf:"varint,1,opt,name=start" json:"start,omitempty"`
-	End                  *int32                 `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`
-	Options              *ExtensionRangeOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *DescriptorProto_ExtensionRange) Reset()         { *m = DescriptorProto_ExtensionRange{} }
-func (m *DescriptorProto_ExtensionRange) String() string { return proto.CompactTextString(m) }
-func (*DescriptorProto_ExtensionRange) ProtoMessage()    {}
-func (*DescriptorProto_ExtensionRange) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{2, 0}
-}
-
-func (m *DescriptorProto_ExtensionRange) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DescriptorProto_ExtensionRange.Unmarshal(m, b)
-}
-func (m *DescriptorProto_ExtensionRange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DescriptorProto_ExtensionRange.Marshal(b, m, deterministic)
-}
-func (m *DescriptorProto_ExtensionRange) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DescriptorProto_ExtensionRange.Merge(m, src)
-}
-func (m *DescriptorProto_ExtensionRange) XXX_Size() int {
-	return xxx_messageInfo_DescriptorProto_ExtensionRange.Size(m)
-}
-func (m *DescriptorProto_ExtensionRange) XXX_DiscardUnknown() {
-	xxx_messageInfo_DescriptorProto_ExtensionRange.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DescriptorProto_ExtensionRange proto.InternalMessageInfo
-
-func (m *DescriptorProto_ExtensionRange) GetStart() int32 {
-	if m != nil && m.Start != nil {
-		return *m.Start
-	}
-	return 0
-}
-
-func (m *DescriptorProto_ExtensionRange) GetEnd() int32 {
-	if m != nil && m.End != nil {
-		return *m.End
-	}
-	return 0
-}
-
-func (m *DescriptorProto_ExtensionRange) GetOptions() *ExtensionRangeOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-// Range of reserved tag numbers. Reserved tag numbers may not be used by
-// fields or extension ranges in the same message. Reserved ranges may
-// not overlap.
-type DescriptorProto_ReservedRange struct {
-	Start                *int32   `protobuf:"varint,1,opt,name=start" json:"start,omitempty"`
-	End                  *int32   `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DescriptorProto_ReservedRange) Reset()         { *m = DescriptorProto_ReservedRange{} }
-func (m *DescriptorProto_ReservedRange) String() string { return proto.CompactTextString(m) }
-func (*DescriptorProto_ReservedRange) ProtoMessage()    {}
-func (*DescriptorProto_ReservedRange) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{2, 1}
-}
-
-func (m *DescriptorProto_ReservedRange) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DescriptorProto_ReservedRange.Unmarshal(m, b)
-}
-func (m *DescriptorProto_ReservedRange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DescriptorProto_ReservedRange.Marshal(b, m, deterministic)
-}
-func (m *DescriptorProto_ReservedRange) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DescriptorProto_ReservedRange.Merge(m, src)
-}
-func (m *DescriptorProto_ReservedRange) XXX_Size() int {
-	return xxx_messageInfo_DescriptorProto_ReservedRange.Size(m)
-}
-func (m *DescriptorProto_ReservedRange) XXX_DiscardUnknown() {
-	xxx_messageInfo_DescriptorProto_ReservedRange.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DescriptorProto_ReservedRange proto.InternalMessageInfo
-
-func (m *DescriptorProto_ReservedRange) GetStart() int32 {
-	if m != nil && m.Start != nil {
-		return *m.Start
-	}
-	return 0
-}
-
-func (m *DescriptorProto_ReservedRange) GetEnd() int32 {
-	if m != nil && m.End != nil {
-		return *m.End
-	}
-	return 0
-}
-
-type ExtensionRangeOptions struct {
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *ExtensionRangeOptions) Reset()         { *m = ExtensionRangeOptions{} }
-func (m *ExtensionRangeOptions) String() string { return proto.CompactTextString(m) }
-func (*ExtensionRangeOptions) ProtoMessage()    {}
-func (*ExtensionRangeOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{3}
-}
-
-var extRange_ExtensionRangeOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*ExtensionRangeOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_ExtensionRangeOptions
-}
-
-func (m *ExtensionRangeOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExtensionRangeOptions.Unmarshal(m, b)
-}
-func (m *ExtensionRangeOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExtensionRangeOptions.Marshal(b, m, deterministic)
-}
-func (m *ExtensionRangeOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExtensionRangeOptions.Merge(m, src)
-}
-func (m *ExtensionRangeOptions) XXX_Size() int {
-	return xxx_messageInfo_ExtensionRangeOptions.Size(m)
-}
-func (m *ExtensionRangeOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExtensionRangeOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExtensionRangeOptions proto.InternalMessageInfo
-
-func (m *ExtensionRangeOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-// Describes a field within a message.
-type FieldDescriptorProto struct {
-	Name   *string                     `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Number *int32                      `protobuf:"varint,3,opt,name=number" json:"number,omitempty"`
-	Label  *FieldDescriptorProto_Label `protobuf:"varint,4,opt,name=label,enum=google.protobuf.FieldDescriptorProto_Label" json:"label,omitempty"`
-	// If type_name is set, this need not be set.  If both this and type_name
-	// are set, this must be one of TYPE_ENUM, TYPE_MESSAGE or TYPE_GROUP.
-	Type *FieldDescriptorProto_Type `protobuf:"varint,5,opt,name=type,enum=google.protobuf.FieldDescriptorProto_Type" json:"type,omitempty"`
-	// For message and enum types, this is the name of the type.  If the name
-	// starts with a '.', it is fully-qualified.  Otherwise, C++-like scoping
-	// rules are used to find the type (i.e. first the nested types within this
-	// message are searched, then within the parent, on up to the root
-	// namespace).
-	TypeName *string `protobuf:"bytes,6,opt,name=type_name,json=typeName" json:"type_name,omitempty"`
-	// For extensions, this is the name of the type being extended.  It is
-	// resolved in the same manner as type_name.
-	Extendee *string `protobuf:"bytes,2,opt,name=extendee" json:"extendee,omitempty"`
-	// For numeric types, contains the original text representation of the value.
-	// For booleans, "true" or "false".
-	// For strings, contains the default text contents (not escaped in any way).
-	// For bytes, contains the C escaped value.  All bytes >= 128 are escaped.
-	// TODO(kenton):  Base-64 encode?
-	DefaultValue *string `protobuf:"bytes,7,opt,name=default_value,json=defaultValue" json:"default_value,omitempty"`
-	// If set, gives the index of a oneof in the containing type's oneof_decl
-	// list.  This field is a member of that oneof.
-	OneofIndex *int32 `protobuf:"varint,9,opt,name=oneof_index,json=oneofIndex" json:"oneof_index,omitempty"`
-	// JSON name of this field. The value is set by protocol compiler. If the
-	// user has set a "json_name" option on this field, that option's value
-	// will be used. Otherwise, it's deduced from the field's name by converting
-	// it to camelCase.
-	JsonName             *string       `protobuf:"bytes,10,opt,name=json_name,json=jsonName" json:"json_name,omitempty"`
-	Options              *FieldOptions `protobuf:"bytes,8,opt,name=options" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *FieldDescriptorProto) Reset()         { *m = FieldDescriptorProto{} }
-func (m *FieldDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*FieldDescriptorProto) ProtoMessage()    {}
-func (*FieldDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{4}
-}
-
-func (m *FieldDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FieldDescriptorProto.Unmarshal(m, b)
-}
-func (m *FieldDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FieldDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *FieldDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FieldDescriptorProto.Merge(m, src)
-}
-func (m *FieldDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_FieldDescriptorProto.Size(m)
-}
-func (m *FieldDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_FieldDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FieldDescriptorProto proto.InternalMessageInfo
-
-func (m *FieldDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *FieldDescriptorProto) GetNumber() int32 {
-	if m != nil && m.Number != nil {
-		return *m.Number
-	}
-	return 0
-}
-
-func (m *FieldDescriptorProto) GetLabel() FieldDescriptorProto_Label {
-	if m != nil && m.Label != nil {
-		return *m.Label
-	}
-	return FieldDescriptorProto_LABEL_OPTIONAL
-}
-
-func (m *FieldDescriptorProto) GetType() FieldDescriptorProto_Type {
-	if m != nil && m.Type != nil {
-		return *m.Type
-	}
-	return FieldDescriptorProto_TYPE_DOUBLE
-}
-
-func (m *FieldDescriptorProto) GetTypeName() string {
-	if m != nil && m.TypeName != nil {
-		return *m.TypeName
-	}
-	return ""
-}
-
-func (m *FieldDescriptorProto) GetExtendee() string {
-	if m != nil && m.Extendee != nil {
-		return *m.Extendee
-	}
-	return ""
-}
-
-func (m *FieldDescriptorProto) GetDefaultValue() string {
-	if m != nil && m.DefaultValue != nil {
-		return *m.DefaultValue
-	}
-	return ""
-}
-
-func (m *FieldDescriptorProto) GetOneofIndex() int32 {
-	if m != nil && m.OneofIndex != nil {
-		return *m.OneofIndex
-	}
-	return 0
-}
-
-func (m *FieldDescriptorProto) GetJsonName() string {
-	if m != nil && m.JsonName != nil {
-		return *m.JsonName
-	}
-	return ""
-}
-
-func (m *FieldDescriptorProto) GetOptions() *FieldOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-// Describes a oneof.
-type OneofDescriptorProto struct {
-	Name                 *string       `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Options              *OneofOptions `protobuf:"bytes,2,opt,name=options" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *OneofDescriptorProto) Reset()         { *m = OneofDescriptorProto{} }
-func (m *OneofDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*OneofDescriptorProto) ProtoMessage()    {}
-func (*OneofDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{5}
-}
-
-func (m *OneofDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OneofDescriptorProto.Unmarshal(m, b)
-}
-func (m *OneofDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OneofDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *OneofDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OneofDescriptorProto.Merge(m, src)
-}
-func (m *OneofDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_OneofDescriptorProto.Size(m)
-}
-func (m *OneofDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_OneofDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OneofDescriptorProto proto.InternalMessageInfo
-
-func (m *OneofDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *OneofDescriptorProto) GetOptions() *OneofOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-// Describes an enum type.
-type EnumDescriptorProto struct {
-	Name    *string                     `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Value   []*EnumValueDescriptorProto `protobuf:"bytes,2,rep,name=value" json:"value,omitempty"`
-	Options *EnumOptions                `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
-	// Range of reserved numeric values. Reserved numeric values may not be used
-	// by enum values in the same enum declaration. Reserved ranges may not
-	// overlap.
-	ReservedRange []*EnumDescriptorProto_EnumReservedRange `protobuf:"bytes,4,rep,name=reserved_range,json=reservedRange" json:"reserved_range,omitempty"`
-	// Reserved enum value names, which may not be reused. A given name may only
-	// be reserved once.
-	ReservedName         []string `protobuf:"bytes,5,rep,name=reserved_name,json=reservedName" json:"reserved_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EnumDescriptorProto) Reset()         { *m = EnumDescriptorProto{} }
-func (m *EnumDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*EnumDescriptorProto) ProtoMessage()    {}
-func (*EnumDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{6}
-}
-
-func (m *EnumDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EnumDescriptorProto.Unmarshal(m, b)
-}
-func (m *EnumDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EnumDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *EnumDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EnumDescriptorProto.Merge(m, src)
-}
-func (m *EnumDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_EnumDescriptorProto.Size(m)
-}
-func (m *EnumDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_EnumDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EnumDescriptorProto proto.InternalMessageInfo
-
-func (m *EnumDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *EnumDescriptorProto) GetValue() []*EnumValueDescriptorProto {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func (m *EnumDescriptorProto) GetOptions() *EnumOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-func (m *EnumDescriptorProto) GetReservedRange() []*EnumDescriptorProto_EnumReservedRange {
-	if m != nil {
-		return m.ReservedRange
-	}
-	return nil
-}
-
-func (m *EnumDescriptorProto) GetReservedName() []string {
-	if m != nil {
-		return m.ReservedName
-	}
-	return nil
-}
-
-// Range of reserved numeric values. Reserved values may not be used by
-// entries in the same enum. Reserved ranges may not overlap.
-//
-// Note that this is distinct from DescriptorProto.ReservedRange in that it
-// is inclusive such that it can appropriately represent the entire int32
-// domain.
-type EnumDescriptorProto_EnumReservedRange struct {
-	Start                *int32   `protobuf:"varint,1,opt,name=start" json:"start,omitempty"`
-	End                  *int32   `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EnumDescriptorProto_EnumReservedRange) Reset()         { *m = EnumDescriptorProto_EnumReservedRange{} }
-func (m *EnumDescriptorProto_EnumReservedRange) String() string { return proto.CompactTextString(m) }
-func (*EnumDescriptorProto_EnumReservedRange) ProtoMessage()    {}
-func (*EnumDescriptorProto_EnumReservedRange) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{6, 0}
-}
-
-func (m *EnumDescriptorProto_EnumReservedRange) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EnumDescriptorProto_EnumReservedRange.Unmarshal(m, b)
-}
-func (m *EnumDescriptorProto_EnumReservedRange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EnumDescriptorProto_EnumReservedRange.Marshal(b, m, deterministic)
-}
-func (m *EnumDescriptorProto_EnumReservedRange) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EnumDescriptorProto_EnumReservedRange.Merge(m, src)
-}
-func (m *EnumDescriptorProto_EnumReservedRange) XXX_Size() int {
-	return xxx_messageInfo_EnumDescriptorProto_EnumReservedRange.Size(m)
-}
-func (m *EnumDescriptorProto_EnumReservedRange) XXX_DiscardUnknown() {
-	xxx_messageInfo_EnumDescriptorProto_EnumReservedRange.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EnumDescriptorProto_EnumReservedRange proto.InternalMessageInfo
-
-func (m *EnumDescriptorProto_EnumReservedRange) GetStart() int32 {
-	if m != nil && m.Start != nil {
-		return *m.Start
-	}
-	return 0
-}
-
-func (m *EnumDescriptorProto_EnumReservedRange) GetEnd() int32 {
-	if m != nil && m.End != nil {
-		return *m.End
-	}
-	return 0
-}
-
-// Describes a value within an enum.
-type EnumValueDescriptorProto struct {
-	Name                 *string           `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Number               *int32            `protobuf:"varint,2,opt,name=number" json:"number,omitempty"`
-	Options              *EnumValueOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *EnumValueDescriptorProto) Reset()         { *m = EnumValueDescriptorProto{} }
-func (m *EnumValueDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*EnumValueDescriptorProto) ProtoMessage()    {}
-func (*EnumValueDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{7}
-}
-
-func (m *EnumValueDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EnumValueDescriptorProto.Unmarshal(m, b)
-}
-func (m *EnumValueDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EnumValueDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *EnumValueDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EnumValueDescriptorProto.Merge(m, src)
-}
-func (m *EnumValueDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_EnumValueDescriptorProto.Size(m)
-}
-func (m *EnumValueDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_EnumValueDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EnumValueDescriptorProto proto.InternalMessageInfo
-
-func (m *EnumValueDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *EnumValueDescriptorProto) GetNumber() int32 {
-	if m != nil && m.Number != nil {
-		return *m.Number
-	}
-	return 0
-}
-
-func (m *EnumValueDescriptorProto) GetOptions() *EnumValueOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-// Describes a service.
-type ServiceDescriptorProto struct {
-	Name                 *string                  `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	Method               []*MethodDescriptorProto `protobuf:"bytes,2,rep,name=method" json:"method,omitempty"`
-	Options              *ServiceOptions          `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
-}
-
-func (m *ServiceDescriptorProto) Reset()         { *m = ServiceDescriptorProto{} }
-func (m *ServiceDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*ServiceDescriptorProto) ProtoMessage()    {}
-func (*ServiceDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{8}
-}
-
-func (m *ServiceDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ServiceDescriptorProto.Unmarshal(m, b)
-}
-func (m *ServiceDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ServiceDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *ServiceDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ServiceDescriptorProto.Merge(m, src)
-}
-func (m *ServiceDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_ServiceDescriptorProto.Size(m)
-}
-func (m *ServiceDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_ServiceDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ServiceDescriptorProto proto.InternalMessageInfo
-
-func (m *ServiceDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *ServiceDescriptorProto) GetMethod() []*MethodDescriptorProto {
-	if m != nil {
-		return m.Method
-	}
-	return nil
-}
-
-func (m *ServiceDescriptorProto) GetOptions() *ServiceOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-// Describes a method of a service.
-type MethodDescriptorProto struct {
-	Name *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
-	// Input and output type names.  These are resolved in the same way as
-	// FieldDescriptorProto.type_name, but must refer to a message type.
-	InputType  *string        `protobuf:"bytes,2,opt,name=input_type,json=inputType" json:"input_type,omitempty"`
-	OutputType *string        `protobuf:"bytes,3,opt,name=output_type,json=outputType" json:"output_type,omitempty"`
-	Options    *MethodOptions `protobuf:"bytes,4,opt,name=options" json:"options,omitempty"`
-	// Identifies if client streams multiple client messages
-	ClientStreaming *bool `protobuf:"varint,5,opt,name=client_streaming,json=clientStreaming,def=0" json:"client_streaming,omitempty"`
-	// Identifies if server streams multiple server messages
-	ServerStreaming      *bool    `protobuf:"varint,6,opt,name=server_streaming,json=serverStreaming,def=0" json:"server_streaming,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *MethodDescriptorProto) Reset()         { *m = MethodDescriptorProto{} }
-func (m *MethodDescriptorProto) String() string { return proto.CompactTextString(m) }
-func (*MethodDescriptorProto) ProtoMessage()    {}
-func (*MethodDescriptorProto) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{9}
-}
-
-func (m *MethodDescriptorProto) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MethodDescriptorProto.Unmarshal(m, b)
-}
-func (m *MethodDescriptorProto) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MethodDescriptorProto.Marshal(b, m, deterministic)
-}
-func (m *MethodDescriptorProto) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MethodDescriptorProto.Merge(m, src)
-}
-func (m *MethodDescriptorProto) XXX_Size() int {
-	return xxx_messageInfo_MethodDescriptorProto.Size(m)
-}
-func (m *MethodDescriptorProto) XXX_DiscardUnknown() {
-	xxx_messageInfo_MethodDescriptorProto.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MethodDescriptorProto proto.InternalMessageInfo
-
-const Default_MethodDescriptorProto_ClientStreaming bool = false
-const Default_MethodDescriptorProto_ServerStreaming bool = false
-
-func (m *MethodDescriptorProto) GetName() string {
-	if m != nil && m.Name != nil {
-		return *m.Name
-	}
-	return ""
-}
-
-func (m *MethodDescriptorProto) GetInputType() string {
-	if m != nil && m.InputType != nil {
-		return *m.InputType
-	}
-	return ""
-}
-
-func (m *MethodDescriptorProto) GetOutputType() string {
-	if m != nil && m.OutputType != nil {
-		return *m.OutputType
-	}
-	return ""
-}
-
-func (m *MethodDescriptorProto) GetOptions() *MethodOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-func (m *MethodDescriptorProto) GetClientStreaming() bool {
-	if m != nil && m.ClientStreaming != nil {
-		return *m.ClientStreaming
-	}
-	return Default_MethodDescriptorProto_ClientStreaming
-}
-
-func (m *MethodDescriptorProto) GetServerStreaming() bool {
-	if m != nil && m.ServerStreaming != nil {
-		return *m.ServerStreaming
-	}
-	return Default_MethodDescriptorProto_ServerStreaming
-}
-
-type FileOptions struct {
-	// Sets the Java package where classes generated from this .proto will be
-	// placed.  By default, the proto package is used, but this is often
-	// inappropriate because proto packages do not normally start with backwards
-	// domain names.
-	JavaPackage *string `protobuf:"bytes,1,opt,name=java_package,json=javaPackage" json:"java_package,omitempty"`
-	// If set, all the classes from the .proto file are wrapped in a single
-	// outer class with the given name.  This applies to both Proto1
-	// (equivalent to the old "--one_java_file" option) and Proto2 (where
-	// a .proto always translates to a single class, but you may want to
-	// explicitly choose the class name).
-	JavaOuterClassname *string `protobuf:"bytes,8,opt,name=java_outer_classname,json=javaOuterClassname" json:"java_outer_classname,omitempty"`
-	// If set true, then the Java code generator will generate a separate .java
-	// file for each top-level message, enum, and service defined in the .proto
-	// file.  Thus, these types will *not* be nested inside the outer class
-	// named by java_outer_classname.  However, the outer class will still be
-	// generated to contain the file's getDescriptor() method as well as any
-	// top-level extensions defined in the file.
-	JavaMultipleFiles *bool `protobuf:"varint,10,opt,name=java_multiple_files,json=javaMultipleFiles,def=0" json:"java_multiple_files,omitempty"`
-	// This option does nothing.
-	JavaGenerateEqualsAndHash *bool `protobuf:"varint,20,opt,name=java_generate_equals_and_hash,json=javaGenerateEqualsAndHash" json:"java_generate_equals_and_hash,omitempty"` // Deprecated: Do not use.
-	// If set true, then the Java2 code generator will generate code that
-	// throws an exception whenever an attempt is made to assign a non-UTF-8
-	// byte sequence to a string field.
-	// Message reflection will do the same.
-	// However, an extension field still accepts non-UTF-8 byte sequences.
-	// This option has no effect on when used with the lite runtime.
-	JavaStringCheckUtf8 *bool                     `protobuf:"varint,27,opt,name=java_string_check_utf8,json=javaStringCheckUtf8,def=0" json:"java_string_check_utf8,omitempty"`
-	OptimizeFor         *FileOptions_OptimizeMode `protobuf:"varint,9,opt,name=optimize_for,json=optimizeFor,enum=google.protobuf.FileOptions_OptimizeMode,def=1" json:"optimize_for,omitempty"`
-	// Sets the Go package where structs generated from this .proto will be
-	// placed. If omitted, the Go package will be derived from the following:
-	//   - The basename of the package import path, if provided.
-	//   - Otherwise, the package statement in the .proto file, if present.
-	//   - Otherwise, the basename of the .proto file, without extension.
-	GoPackage *string `protobuf:"bytes,11,opt,name=go_package,json=goPackage" json:"go_package,omitempty"`
-	// Should generic services be generated in each language?  "Generic" services
-	// are not specific to any particular RPC system.  They are generated by the
-	// main code generators in each language (without additional plugins).
-	// Generic services were the only kind of service generation supported by
-	// early versions of google.protobuf.
-	//
-	// Generic services are now considered deprecated in favor of using plugins
-	// that generate code specific to your particular RPC system.  Therefore,
-	// these default to false.  Old code which depends on generic services should
-	// explicitly set them to true.
-	CcGenericServices   *bool `protobuf:"varint,16,opt,name=cc_generic_services,json=ccGenericServices,def=0" json:"cc_generic_services,omitempty"`
-	JavaGenericServices *bool `protobuf:"varint,17,opt,name=java_generic_services,json=javaGenericServices,def=0" json:"java_generic_services,omitempty"`
-	PyGenericServices   *bool `protobuf:"varint,18,opt,name=py_generic_services,json=pyGenericServices,def=0" json:"py_generic_services,omitempty"`
-	PhpGenericServices  *bool `protobuf:"varint,42,opt,name=php_generic_services,json=phpGenericServices,def=0" json:"php_generic_services,omitempty"`
-	// Is this file deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for everything in the file, or it will be completely ignored; in the very
-	// least, this is a formalization for deprecating files.
-	Deprecated *bool `protobuf:"varint,23,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// Enables the use of arenas for the proto messages in this file. This applies
-	// only to generated classes for C++.
-	CcEnableArenas *bool `protobuf:"varint,31,opt,name=cc_enable_arenas,json=ccEnableArenas,def=0" json:"cc_enable_arenas,omitempty"`
-	// Sets the objective c class prefix which is prepended to all objective c
-	// generated classes from this .proto. There is no default.
-	ObjcClassPrefix *string `protobuf:"bytes,36,opt,name=objc_class_prefix,json=objcClassPrefix" json:"objc_class_prefix,omitempty"`
-	// Namespace for generated classes; defaults to the package.
-	CsharpNamespace *string `protobuf:"bytes,37,opt,name=csharp_namespace,json=csharpNamespace" json:"csharp_namespace,omitempty"`
-	// By default Swift generators will take the proto package and CamelCase it
-	// replacing '.' with underscore and use that to prefix the types/symbols
-	// defined. When this options is provided, they will use this value instead
-	// to prefix the types/symbols defined.
-	SwiftPrefix *string `protobuf:"bytes,39,opt,name=swift_prefix,json=swiftPrefix" json:"swift_prefix,omitempty"`
-	// Sets the php class prefix which is prepended to all php generated classes
-	// from this .proto. Default is empty.
-	PhpClassPrefix *string `protobuf:"bytes,40,opt,name=php_class_prefix,json=phpClassPrefix" json:"php_class_prefix,omitempty"`
-	// Use this option to change the namespace of php generated classes. Default
-	// is empty. When this option is empty, the package name will be used for
-	// determining the namespace.
-	PhpNamespace *string `protobuf:"bytes,41,opt,name=php_namespace,json=phpNamespace" json:"php_namespace,omitempty"`
-	// Use this option to change the namespace of php generated metadata classes.
-	// Default is empty. When this option is empty, the proto file name will be used
-	// for determining the namespace.
-	PhpMetadataNamespace *string `protobuf:"bytes,44,opt,name=php_metadata_namespace,json=phpMetadataNamespace" json:"php_metadata_namespace,omitempty"`
-	// Use this option to change the package of ruby generated classes. Default
-	// is empty. When this option is not set, the package name will be used for
-	// determining the ruby package.
-	RubyPackage *string `protobuf:"bytes,45,opt,name=ruby_package,json=rubyPackage" json:"ruby_package,omitempty"`
-	// The parser stores options it doesn't recognize here.
-	// See the documentation for the "Options" section above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *FileOptions) Reset()         { *m = FileOptions{} }
-func (m *FileOptions) String() string { return proto.CompactTextString(m) }
-func (*FileOptions) ProtoMessage()    {}
-func (*FileOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{10}
-}
-
-var extRange_FileOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*FileOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_FileOptions
-}
-
-func (m *FileOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FileOptions.Unmarshal(m, b)
-}
-func (m *FileOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FileOptions.Marshal(b, m, deterministic)
-}
-func (m *FileOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FileOptions.Merge(m, src)
-}
-func (m *FileOptions) XXX_Size() int {
-	return xxx_messageInfo_FileOptions.Size(m)
-}
-func (m *FileOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_FileOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FileOptions proto.InternalMessageInfo
-
-const Default_FileOptions_JavaMultipleFiles bool = false
-const Default_FileOptions_JavaStringCheckUtf8 bool = false
-const Default_FileOptions_OptimizeFor FileOptions_OptimizeMode = FileOptions_SPEED
-const Default_FileOptions_CcGenericServices bool = false
-const Default_FileOptions_JavaGenericServices bool = false
-const Default_FileOptions_PyGenericServices bool = false
-const Default_FileOptions_PhpGenericServices bool = false
-const Default_FileOptions_Deprecated bool = false
-const Default_FileOptions_CcEnableArenas bool = false
-
-func (m *FileOptions) GetJavaPackage() string {
-	if m != nil && m.JavaPackage != nil {
-		return *m.JavaPackage
-	}
-	return ""
-}
-
-func (m *FileOptions) GetJavaOuterClassname() string {
-	if m != nil && m.JavaOuterClassname != nil {
-		return *m.JavaOuterClassname
-	}
-	return ""
-}
-
-func (m *FileOptions) GetJavaMultipleFiles() bool {
-	if m != nil && m.JavaMultipleFiles != nil {
-		return *m.JavaMultipleFiles
-	}
-	return Default_FileOptions_JavaMultipleFiles
-}
-
-// Deprecated: Do not use.
-func (m *FileOptions) GetJavaGenerateEqualsAndHash() bool {
-	if m != nil && m.JavaGenerateEqualsAndHash != nil {
-		return *m.JavaGenerateEqualsAndHash
-	}
-	return false
-}
-
-func (m *FileOptions) GetJavaStringCheckUtf8() bool {
-	if m != nil && m.JavaStringCheckUtf8 != nil {
-		return *m.JavaStringCheckUtf8
-	}
-	return Default_FileOptions_JavaStringCheckUtf8
-}
-
-func (m *FileOptions) GetOptimizeFor() FileOptions_OptimizeMode {
-	if m != nil && m.OptimizeFor != nil {
-		return *m.OptimizeFor
-	}
-	return Default_FileOptions_OptimizeFor
-}
-
-func (m *FileOptions) GetGoPackage() string {
-	if m != nil && m.GoPackage != nil {
-		return *m.GoPackage
-	}
-	return ""
-}
-
-func (m *FileOptions) GetCcGenericServices() bool {
-	if m != nil && m.CcGenericServices != nil {
-		return *m.CcGenericServices
-	}
-	return Default_FileOptions_CcGenericServices
-}
-
-func (m *FileOptions) GetJavaGenericServices() bool {
-	if m != nil && m.JavaGenericServices != nil {
-		return *m.JavaGenericServices
-	}
-	return Default_FileOptions_JavaGenericServices
-}
-
-func (m *FileOptions) GetPyGenericServices() bool {
-	if m != nil && m.PyGenericServices != nil {
-		return *m.PyGenericServices
-	}
-	return Default_FileOptions_PyGenericServices
-}
-
-func (m *FileOptions) GetPhpGenericServices() bool {
-	if m != nil && m.PhpGenericServices != nil {
-		return *m.PhpGenericServices
-	}
-	return Default_FileOptions_PhpGenericServices
-}
-
-func (m *FileOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_FileOptions_Deprecated
-}
-
-func (m *FileOptions) GetCcEnableArenas() bool {
-	if m != nil && m.CcEnableArenas != nil {
-		return *m.CcEnableArenas
-	}
-	return Default_FileOptions_CcEnableArenas
-}
-
-func (m *FileOptions) GetObjcClassPrefix() string {
-	if m != nil && m.ObjcClassPrefix != nil {
-		return *m.ObjcClassPrefix
-	}
-	return ""
-}
-
-func (m *FileOptions) GetCsharpNamespace() string {
-	if m != nil && m.CsharpNamespace != nil {
-		return *m.CsharpNamespace
-	}
-	return ""
-}
-
-func (m *FileOptions) GetSwiftPrefix() string {
-	if m != nil && m.SwiftPrefix != nil {
-		return *m.SwiftPrefix
-	}
-	return ""
-}
-
-func (m *FileOptions) GetPhpClassPrefix() string {
-	if m != nil && m.PhpClassPrefix != nil {
-		return *m.PhpClassPrefix
-	}
-	return ""
-}
-
-func (m *FileOptions) GetPhpNamespace() string {
-	if m != nil && m.PhpNamespace != nil {
-		return *m.PhpNamespace
-	}
-	return ""
-}
-
-func (m *FileOptions) GetPhpMetadataNamespace() string {
-	if m != nil && m.PhpMetadataNamespace != nil {
-		return *m.PhpMetadataNamespace
-	}
-	return ""
-}
-
-func (m *FileOptions) GetRubyPackage() string {
-	if m != nil && m.RubyPackage != nil {
-		return *m.RubyPackage
-	}
-	return ""
-}
-
-func (m *FileOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type MessageOptions struct {
-	// Set true to use the old proto1 MessageSet wire format for extensions.
-	// This is provided for backwards-compatibility with the MessageSet wire
-	// format.  You should not use this for any other reason:  It's less
-	// efficient, has fewer features, and is more complicated.
-	//
-	// The message must be defined exactly as follows:
-	//   message Foo {
-	//     option message_set_wire_format = true;
-	//     extensions 4 to max;
-	//   }
-	// Note that the message cannot have any defined fields; MessageSets only
-	// have extensions.
-	//
-	// All extensions of your type must be singular messages; e.g. they cannot
-	// be int32s, enums, or repeated messages.
-	//
-	// Because this is an option, the above two restrictions are not enforced by
-	// the protocol compiler.
-	MessageSetWireFormat *bool `protobuf:"varint,1,opt,name=message_set_wire_format,json=messageSetWireFormat,def=0" json:"message_set_wire_format,omitempty"`
-	// Disables the generation of the standard "descriptor()" accessor, which can
-	// conflict with a field of the same name.  This is meant to make migration
-	// from proto1 easier; new code should avoid fields named "descriptor".
-	NoStandardDescriptorAccessor *bool `protobuf:"varint,2,opt,name=no_standard_descriptor_accessor,json=noStandardDescriptorAccessor,def=0" json:"no_standard_descriptor_accessor,omitempty"`
-	// Is this message deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for the message, or it will be completely ignored; in the very least,
-	// this is a formalization for deprecating messages.
-	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// Whether the message is an automatically generated map entry type for the
-	// maps field.
-	//
-	// For maps fields:
-	//     map<KeyType, ValueType> map_field = 1;
-	// The parsed descriptor looks like:
-	//     message MapFieldEntry {
-	//         option map_entry = true;
-	//         optional KeyType key = 1;
-	//         optional ValueType value = 2;
-	//     }
-	//     repeated MapFieldEntry map_field = 1;
-	//
-	// Implementations may choose not to generate the map_entry=true message, but
-	// use a native map in the target language to hold the keys and values.
-	// The reflection APIs in such implementions still need to work as
-	// if the field is a repeated message field.
-	//
-	// NOTE: Do not set the option in .proto files. Always use the maps syntax
-	// instead. The option should only be implicitly set by the proto compiler
-	// parser.
-	MapEntry *bool `protobuf:"varint,7,opt,name=map_entry,json=mapEntry" json:"map_entry,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *MessageOptions) Reset()         { *m = MessageOptions{} }
-func (m *MessageOptions) String() string { return proto.CompactTextString(m) }
-func (*MessageOptions) ProtoMessage()    {}
-func (*MessageOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{11}
-}
-
-var extRange_MessageOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*MessageOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_MessageOptions
-}
-
-func (m *MessageOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MessageOptions.Unmarshal(m, b)
-}
-func (m *MessageOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MessageOptions.Marshal(b, m, deterministic)
-}
-func (m *MessageOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MessageOptions.Merge(m, src)
-}
-func (m *MessageOptions) XXX_Size() int {
-	return xxx_messageInfo_MessageOptions.Size(m)
-}
-func (m *MessageOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_MessageOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MessageOptions proto.InternalMessageInfo
-
-const Default_MessageOptions_MessageSetWireFormat bool = false
-const Default_MessageOptions_NoStandardDescriptorAccessor bool = false
-const Default_MessageOptions_Deprecated bool = false
-
-func (m *MessageOptions) GetMessageSetWireFormat() bool {
-	if m != nil && m.MessageSetWireFormat != nil {
-		return *m.MessageSetWireFormat
-	}
-	return Default_MessageOptions_MessageSetWireFormat
-}
-
-func (m *MessageOptions) GetNoStandardDescriptorAccessor() bool {
-	if m != nil && m.NoStandardDescriptorAccessor != nil {
-		return *m.NoStandardDescriptorAccessor
-	}
-	return Default_MessageOptions_NoStandardDescriptorAccessor
-}
-
-func (m *MessageOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_MessageOptions_Deprecated
-}
-
-func (m *MessageOptions) GetMapEntry() bool {
-	if m != nil && m.MapEntry != nil {
-		return *m.MapEntry
-	}
-	return false
-}
-
-func (m *MessageOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type FieldOptions struct {
-	// The ctype option instructs the C++ code generator to use a different
-	// representation of the field than it normally would.  See the specific
-	// options below.  This option is not yet implemented in the open source
-	// release -- sorry, we'll try to include it in a future version!
-	Ctype *FieldOptions_CType `protobuf:"varint,1,opt,name=ctype,enum=google.protobuf.FieldOptions_CType,def=0" json:"ctype,omitempty"`
-	// The packed option can be enabled for repeated primitive fields to enable
-	// a more efficient representation on the wire. Rather than repeatedly
-	// writing the tag and type for each element, the entire array is encoded as
-	// a single length-delimited blob. In proto3, only explicit setting it to
-	// false will avoid using packed encoding.
-	Packed *bool `protobuf:"varint,2,opt,name=packed" json:"packed,omitempty"`
-	// The jstype option determines the JavaScript type used for values of the
-	// field.  The option is permitted only for 64 bit integral and fixed types
-	// (int64, uint64, sint64, fixed64, sfixed64).  A field with jstype JS_STRING
-	// is represented as JavaScript string, which avoids loss of precision that
-	// can happen when a large value is converted to a floating point JavaScript.
-	// Specifying JS_NUMBER for the jstype causes the generated JavaScript code to
-	// use the JavaScript "number" type.  The behavior of the default option
-	// JS_NORMAL is implementation dependent.
-	//
-	// This option is an enum to permit additional types to be added, e.g.
-	// goog.math.Integer.
-	Jstype *FieldOptions_JSType `protobuf:"varint,6,opt,name=jstype,enum=google.protobuf.FieldOptions_JSType,def=0" json:"jstype,omitempty"`
-	// Should this field be parsed lazily?  Lazy applies only to message-type
-	// fields.  It means that when the outer message is initially parsed, the
-	// inner message's contents will not be parsed but instead stored in encoded
-	// form.  The inner message will actually be parsed when it is first accessed.
-	//
-	// This is only a hint.  Implementations are free to choose whether to use
-	// eager or lazy parsing regardless of the value of this option.  However,
-	// setting this option true suggests that the protocol author believes that
-	// using lazy parsing on this field is worth the additional bookkeeping
-	// overhead typically needed to implement it.
-	//
-	// This option does not affect the public interface of any generated code;
-	// all method signatures remain the same.  Furthermore, thread-safety of the
-	// interface is not affected by this option; const methods remain safe to
-	// call from multiple threads concurrently, while non-const methods continue
-	// to require exclusive access.
-	//
-	//
-	// Note that implementations may choose not to check required fields within
-	// a lazy sub-message.  That is, calling IsInitialized() on the outer message
-	// may return true even if the inner message has missing required fields.
-	// This is necessary because otherwise the inner message would have to be
-	// parsed in order to perform the check, defeating the purpose of lazy
-	// parsing.  An implementation which chooses not to check required fields
-	// must be consistent about it.  That is, for any particular sub-message, the
-	// implementation must either *always* check its required fields, or *never*
-	// check its required fields, regardless of whether or not the message has
-	// been parsed.
-	Lazy *bool `protobuf:"varint,5,opt,name=lazy,def=0" json:"lazy,omitempty"`
-	// Is this field deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for accessors, or it will be completely ignored; in the very least, this
-	// is a formalization for deprecating fields.
-	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// For Google-internal migration only. Do not use.
-	Weak *bool `protobuf:"varint,10,opt,name=weak,def=0" json:"weak,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *FieldOptions) Reset()         { *m = FieldOptions{} }
-func (m *FieldOptions) String() string { return proto.CompactTextString(m) }
-func (*FieldOptions) ProtoMessage()    {}
-func (*FieldOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{12}
-}
-
-var extRange_FieldOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*FieldOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_FieldOptions
-}
-
-func (m *FieldOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FieldOptions.Unmarshal(m, b)
-}
-func (m *FieldOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FieldOptions.Marshal(b, m, deterministic)
-}
-func (m *FieldOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FieldOptions.Merge(m, src)
-}
-func (m *FieldOptions) XXX_Size() int {
-	return xxx_messageInfo_FieldOptions.Size(m)
-}
-func (m *FieldOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_FieldOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FieldOptions proto.InternalMessageInfo
-
-const Default_FieldOptions_Ctype FieldOptions_CType = FieldOptions_STRING
-const Default_FieldOptions_Jstype FieldOptions_JSType = FieldOptions_JS_NORMAL
-const Default_FieldOptions_Lazy bool = false
-const Default_FieldOptions_Deprecated bool = false
-const Default_FieldOptions_Weak bool = false
-
-func (m *FieldOptions) GetCtype() FieldOptions_CType {
-	if m != nil && m.Ctype != nil {
-		return *m.Ctype
-	}
-	return Default_FieldOptions_Ctype
-}
-
-func (m *FieldOptions) GetPacked() bool {
-	if m != nil && m.Packed != nil {
-		return *m.Packed
-	}
-	return false
-}
-
-func (m *FieldOptions) GetJstype() FieldOptions_JSType {
-	if m != nil && m.Jstype != nil {
-		return *m.Jstype
-	}
-	return Default_FieldOptions_Jstype
-}
-
-func (m *FieldOptions) GetLazy() bool {
-	if m != nil && m.Lazy != nil {
-		return *m.Lazy
-	}
-	return Default_FieldOptions_Lazy
-}
-
-func (m *FieldOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_FieldOptions_Deprecated
-}
-
-func (m *FieldOptions) GetWeak() bool {
-	if m != nil && m.Weak != nil {
-		return *m.Weak
-	}
-	return Default_FieldOptions_Weak
-}
-
-func (m *FieldOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type OneofOptions struct {
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *OneofOptions) Reset()         { *m = OneofOptions{} }
-func (m *OneofOptions) String() string { return proto.CompactTextString(m) }
-func (*OneofOptions) ProtoMessage()    {}
-func (*OneofOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{13}
-}
-
-var extRange_OneofOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*OneofOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_OneofOptions
-}
-
-func (m *OneofOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OneofOptions.Unmarshal(m, b)
-}
-func (m *OneofOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OneofOptions.Marshal(b, m, deterministic)
-}
-func (m *OneofOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OneofOptions.Merge(m, src)
-}
-func (m *OneofOptions) XXX_Size() int {
-	return xxx_messageInfo_OneofOptions.Size(m)
-}
-func (m *OneofOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_OneofOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OneofOptions proto.InternalMessageInfo
-
-func (m *OneofOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type EnumOptions struct {
-	// Set this option to true to allow mapping different tag names to the same
-	// value.
-	AllowAlias *bool `protobuf:"varint,2,opt,name=allow_alias,json=allowAlias" json:"allow_alias,omitempty"`
-	// Is this enum deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for the enum, or it will be completely ignored; in the very least, this
-	// is a formalization for deprecating enums.
-	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *EnumOptions) Reset()         { *m = EnumOptions{} }
-func (m *EnumOptions) String() string { return proto.CompactTextString(m) }
-func (*EnumOptions) ProtoMessage()    {}
-func (*EnumOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{14}
-}
-
-var extRange_EnumOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*EnumOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_EnumOptions
-}
-
-func (m *EnumOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EnumOptions.Unmarshal(m, b)
-}
-func (m *EnumOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EnumOptions.Marshal(b, m, deterministic)
-}
-func (m *EnumOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EnumOptions.Merge(m, src)
-}
-func (m *EnumOptions) XXX_Size() int {
-	return xxx_messageInfo_EnumOptions.Size(m)
-}
-func (m *EnumOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_EnumOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EnumOptions proto.InternalMessageInfo
-
-const Default_EnumOptions_Deprecated bool = false
-
-func (m *EnumOptions) GetAllowAlias() bool {
-	if m != nil && m.AllowAlias != nil {
-		return *m.AllowAlias
-	}
-	return false
-}
-
-func (m *EnumOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_EnumOptions_Deprecated
-}
-
-func (m *EnumOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type EnumValueOptions struct {
-	// Is this enum value deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for the enum value, or it will be completely ignored; in the very least,
-	// this is a formalization for deprecating enum values.
-	Deprecated *bool `protobuf:"varint,1,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *EnumValueOptions) Reset()         { *m = EnumValueOptions{} }
-func (m *EnumValueOptions) String() string { return proto.CompactTextString(m) }
-func (*EnumValueOptions) ProtoMessage()    {}
-func (*EnumValueOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{15}
-}
-
-var extRange_EnumValueOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*EnumValueOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_EnumValueOptions
-}
-
-func (m *EnumValueOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EnumValueOptions.Unmarshal(m, b)
-}
-func (m *EnumValueOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EnumValueOptions.Marshal(b, m, deterministic)
-}
-func (m *EnumValueOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EnumValueOptions.Merge(m, src)
-}
-func (m *EnumValueOptions) XXX_Size() int {
-	return xxx_messageInfo_EnumValueOptions.Size(m)
-}
-func (m *EnumValueOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_EnumValueOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EnumValueOptions proto.InternalMessageInfo
-
-const Default_EnumValueOptions_Deprecated bool = false
-
-func (m *EnumValueOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_EnumValueOptions_Deprecated
-}
-
-func (m *EnumValueOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type ServiceOptions struct {
-	// Is this service deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for the service, or it will be completely ignored; in the very least,
-	// this is a formalization for deprecating services.
-	Deprecated *bool `protobuf:"varint,33,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *ServiceOptions) Reset()         { *m = ServiceOptions{} }
-func (m *ServiceOptions) String() string { return proto.CompactTextString(m) }
-func (*ServiceOptions) ProtoMessage()    {}
-func (*ServiceOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{16}
-}
-
-var extRange_ServiceOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*ServiceOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_ServiceOptions
-}
-
-func (m *ServiceOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ServiceOptions.Unmarshal(m, b)
-}
-func (m *ServiceOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ServiceOptions.Marshal(b, m, deterministic)
-}
-func (m *ServiceOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ServiceOptions.Merge(m, src)
-}
-func (m *ServiceOptions) XXX_Size() int {
-	return xxx_messageInfo_ServiceOptions.Size(m)
-}
-func (m *ServiceOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_ServiceOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ServiceOptions proto.InternalMessageInfo
-
-const Default_ServiceOptions_Deprecated bool = false
-
-func (m *ServiceOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_ServiceOptions_Deprecated
-}
-
-func (m *ServiceOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-type MethodOptions struct {
-	// Is this method deprecated?
-	// Depending on the target platform, this can emit Deprecated annotations
-	// for the method, or it will be completely ignored; in the very least,
-	// this is a formalization for deprecating methods.
-	Deprecated       *bool                           `protobuf:"varint,33,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
-	IdempotencyLevel *MethodOptions_IdempotencyLevel `protobuf:"varint,34,opt,name=idempotency_level,json=idempotencyLevel,enum=google.protobuf.MethodOptions_IdempotencyLevel,def=0" json:"idempotency_level,omitempty"`
-	// The parser stores options it doesn't recognize here. See above.
-	UninterpretedOption          []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	proto.XXX_InternalExtensions `json:"-"`
-	XXX_unrecognized             []byte `json:"-"`
-	XXX_sizecache                int32  `json:"-"`
-}
-
-func (m *MethodOptions) Reset()         { *m = MethodOptions{} }
-func (m *MethodOptions) String() string { return proto.CompactTextString(m) }
-func (*MethodOptions) ProtoMessage()    {}
-func (*MethodOptions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{17}
-}
-
-var extRange_MethodOptions = []proto.ExtensionRange{
-	{Start: 1000, End: 536870911},
-}
-
-func (*MethodOptions) ExtensionRangeArray() []proto.ExtensionRange {
-	return extRange_MethodOptions
-}
-
-func (m *MethodOptions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MethodOptions.Unmarshal(m, b)
-}
-func (m *MethodOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MethodOptions.Marshal(b, m, deterministic)
-}
-func (m *MethodOptions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MethodOptions.Merge(m, src)
-}
-func (m *MethodOptions) XXX_Size() int {
-	return xxx_messageInfo_MethodOptions.Size(m)
-}
-func (m *MethodOptions) XXX_DiscardUnknown() {
-	xxx_messageInfo_MethodOptions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MethodOptions proto.InternalMessageInfo
-
-const Default_MethodOptions_Deprecated bool = false
-const Default_MethodOptions_IdempotencyLevel MethodOptions_IdempotencyLevel = MethodOptions_IDEMPOTENCY_UNKNOWN
-
-func (m *MethodOptions) GetDeprecated() bool {
-	if m != nil && m.Deprecated != nil {
-		return *m.Deprecated
-	}
-	return Default_MethodOptions_Deprecated
-}
-
-func (m *MethodOptions) GetIdempotencyLevel() MethodOptions_IdempotencyLevel {
-	if m != nil && m.IdempotencyLevel != nil {
-		return *m.IdempotencyLevel
-	}
-	return Default_MethodOptions_IdempotencyLevel
-}
-
-func (m *MethodOptions) GetUninterpretedOption() []*UninterpretedOption {
-	if m != nil {
-		return m.UninterpretedOption
-	}
-	return nil
-}
-
-// A message representing a option the parser does not recognize. This only
-// appears in options protos created by the compiler::Parser class.
-// DescriptorPool resolves these when building Descriptor objects. Therefore,
-// options protos in descriptor objects (e.g. returned by Descriptor::options(),
-// or produced by Descriptor::CopyTo()) will never have UninterpretedOptions
-// in them.
-type UninterpretedOption struct {
-	Name []*UninterpretedOption_NamePart `protobuf:"bytes,2,rep,name=name" json:"name,omitempty"`
-	// The value of the uninterpreted option, in whatever type the tokenizer
-	// identified it as during parsing. Exactly one of these should be set.
-	IdentifierValue      *string  `protobuf:"bytes,3,opt,name=identifier_value,json=identifierValue" json:"identifier_value,omitempty"`
-	PositiveIntValue     *uint64  `protobuf:"varint,4,opt,name=positive_int_value,json=positiveIntValue" json:"positive_int_value,omitempty"`
-	NegativeIntValue     *int64   `protobuf:"varint,5,opt,name=negative_int_value,json=negativeIntValue" json:"negative_int_value,omitempty"`
-	DoubleValue          *float64 `protobuf:"fixed64,6,opt,name=double_value,json=doubleValue" json:"double_value,omitempty"`
-	StringValue          []byte   `protobuf:"bytes,7,opt,name=string_value,json=stringValue" json:"string_value,omitempty"`
-	AggregateValue       *string  `protobuf:"bytes,8,opt,name=aggregate_value,json=aggregateValue" json:"aggregate_value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *UninterpretedOption) Reset()         { *m = UninterpretedOption{} }
-func (m *UninterpretedOption) String() string { return proto.CompactTextString(m) }
-func (*UninterpretedOption) ProtoMessage()    {}
-func (*UninterpretedOption) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{18}
-}
-
-func (m *UninterpretedOption) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_UninterpretedOption.Unmarshal(m, b)
-}
-func (m *UninterpretedOption) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_UninterpretedOption.Marshal(b, m, deterministic)
-}
-func (m *UninterpretedOption) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_UninterpretedOption.Merge(m, src)
-}
-func (m *UninterpretedOption) XXX_Size() int {
-	return xxx_messageInfo_UninterpretedOption.Size(m)
-}
-func (m *UninterpretedOption) XXX_DiscardUnknown() {
-	xxx_messageInfo_UninterpretedOption.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_UninterpretedOption proto.InternalMessageInfo
-
-func (m *UninterpretedOption) GetName() []*UninterpretedOption_NamePart {
-	if m != nil {
-		return m.Name
-	}
-	return nil
-}
-
-func (m *UninterpretedOption) GetIdentifierValue() string {
-	if m != nil && m.IdentifierValue != nil {
-		return *m.IdentifierValue
-	}
-	return ""
-}
-
-func (m *UninterpretedOption) GetPositiveIntValue() uint64 {
-	if m != nil && m.PositiveIntValue != nil {
-		return *m.PositiveIntValue
-	}
-	return 0
-}
-
-func (m *UninterpretedOption) GetNegativeIntValue() int64 {
-	if m != nil && m.NegativeIntValue != nil {
-		return *m.NegativeIntValue
-	}
-	return 0
-}
-
-func (m *UninterpretedOption) GetDoubleValue() float64 {
-	if m != nil && m.DoubleValue != nil {
-		return *m.DoubleValue
-	}
-	return 0
-}
-
-func (m *UninterpretedOption) GetStringValue() []byte {
-	if m != nil {
-		return m.StringValue
-	}
-	return nil
-}
-
-func (m *UninterpretedOption) GetAggregateValue() string {
-	if m != nil && m.AggregateValue != nil {
-		return *m.AggregateValue
-	}
-	return ""
-}
-
-// The name of the uninterpreted option.  Each string represents a segment in
-// a dot-separated name.  is_extension is true iff a segment represents an
-// extension (denoted with parentheses in options specs in .proto files).
-// E.g.,{ ["foo", false], ["bar.baz", true], ["qux", false] } represents
-// "foo.(bar.baz).qux".
-type UninterpretedOption_NamePart struct {
-	NamePart             *string  `protobuf:"bytes,1,req,name=name_part,json=namePart" json:"name_part,omitempty"`
-	IsExtension          *bool    `protobuf:"varint,2,req,name=is_extension,json=isExtension" json:"is_extension,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *UninterpretedOption_NamePart) Reset()         { *m = UninterpretedOption_NamePart{} }
-func (m *UninterpretedOption_NamePart) String() string { return proto.CompactTextString(m) }
-func (*UninterpretedOption_NamePart) ProtoMessage()    {}
-func (*UninterpretedOption_NamePart) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{18, 0}
-}
-
-func (m *UninterpretedOption_NamePart) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_UninterpretedOption_NamePart.Unmarshal(m, b)
-}
-func (m *UninterpretedOption_NamePart) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_UninterpretedOption_NamePart.Marshal(b, m, deterministic)
-}
-func (m *UninterpretedOption_NamePart) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_UninterpretedOption_NamePart.Merge(m, src)
-}
-func (m *UninterpretedOption_NamePart) XXX_Size() int {
-	return xxx_messageInfo_UninterpretedOption_NamePart.Size(m)
-}
-func (m *UninterpretedOption_NamePart) XXX_DiscardUnknown() {
-	xxx_messageInfo_UninterpretedOption_NamePart.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_UninterpretedOption_NamePart proto.InternalMessageInfo
-
-func (m *UninterpretedOption_NamePart) GetNamePart() string {
-	if m != nil && m.NamePart != nil {
-		return *m.NamePart
-	}
-	return ""
-}
-
-func (m *UninterpretedOption_NamePart) GetIsExtension() bool {
-	if m != nil && m.IsExtension != nil {
-		return *m.IsExtension
-	}
-	return false
-}
-
-// Encapsulates information about the original source file from which a
-// FileDescriptorProto was generated.
-type SourceCodeInfo struct {
-	// A Location identifies a piece of source code in a .proto file which
-	// corresponds to a particular definition.  This information is intended
-	// to be useful to IDEs, code indexers, documentation generators, and similar
-	// tools.
-	//
-	// For example, say we have a file like:
-	//   message Foo {
-	//     optional string foo = 1;
-	//   }
-	// Let's look at just the field definition:
-	//   optional string foo = 1;
-	//   ^       ^^     ^^  ^  ^^^
-	//   a       bc     de  f  ghi
-	// We have the following locations:
-	//   span   path               represents
-	//   [a,i)  [ 4, 0, 2, 0 ]     The whole field definition.
-	//   [a,b)  [ 4, 0, 2, 0, 4 ]  The label (optional).
-	//   [c,d)  [ 4, 0, 2, 0, 5 ]  The type (string).
-	//   [e,f)  [ 4, 0, 2, 0, 1 ]  The name (foo).
-	//   [g,h)  [ 4, 0, 2, 0, 3 ]  The number (1).
-	//
-	// Notes:
-	// - A location may refer to a repeated field itself (i.e. not to any
-	//   particular index within it).  This is used whenever a set of elements are
-	//   logically enclosed in a single code segment.  For example, an entire
-	//   extend block (possibly containing multiple extension definitions) will
-	//   have an outer location whose path refers to the "extensions" repeated
-	//   field without an index.
-	// - Multiple locations may have the same path.  This happens when a single
-	//   logical declaration is spread out across multiple places.  The most
-	//   obvious example is the "extend" block again -- there may be multiple
-	//   extend blocks in the same scope, each of which will have the same path.
-	// - A location's span is not always a subset of its parent's span.  For
-	//   example, the "extendee" of an extension declaration appears at the
-	//   beginning of the "extend" block and is shared by all extensions within
-	//   the block.
-	// - Just because a location's span is a subset of some other location's span
-	//   does not mean that it is a descendent.  For example, a "group" defines
-	//   both a type and a field in a single declaration.  Thus, the locations
-	//   corresponding to the type and field and their components will overlap.
-	// - Code which tries to interpret locations should probably be designed to
-	//   ignore those that it doesn't understand, as more types of locations could
-	//   be recorded in the future.
-	Location             []*SourceCodeInfo_Location `protobuf:"bytes,1,rep,name=location" json:"location,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *SourceCodeInfo) Reset()         { *m = SourceCodeInfo{} }
-func (m *SourceCodeInfo) String() string { return proto.CompactTextString(m) }
-func (*SourceCodeInfo) ProtoMessage()    {}
-func (*SourceCodeInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{19}
-}
-
-func (m *SourceCodeInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SourceCodeInfo.Unmarshal(m, b)
-}
-func (m *SourceCodeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SourceCodeInfo.Marshal(b, m, deterministic)
-}
-func (m *SourceCodeInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SourceCodeInfo.Merge(m, src)
-}
-func (m *SourceCodeInfo) XXX_Size() int {
-	return xxx_messageInfo_SourceCodeInfo.Size(m)
-}
-func (m *SourceCodeInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_SourceCodeInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SourceCodeInfo proto.InternalMessageInfo
-
-func (m *SourceCodeInfo) GetLocation() []*SourceCodeInfo_Location {
-	if m != nil {
-		return m.Location
-	}
-	return nil
-}
-
-type SourceCodeInfo_Location struct {
-	// Identifies which part of the FileDescriptorProto was defined at this
-	// location.
-	//
-	// Each element is a field number or an index.  They form a path from
-	// the root FileDescriptorProto to the place where the definition.  For
-	// example, this path:
-	//   [ 4, 3, 2, 7, 1 ]
-	// refers to:
-	//   file.message_type(3)  // 4, 3
-	//       .field(7)         // 2, 7
-	//       .name()           // 1
-	// This is because FileDescriptorProto.message_type has field number 4:
-	//   repeated DescriptorProto message_type = 4;
-	// and DescriptorProto.field has field number 2:
-	//   repeated FieldDescriptorProto field = 2;
-	// and FieldDescriptorProto.name has field number 1:
-	//   optional string name = 1;
-	//
-	// Thus, the above path gives the location of a field name.  If we removed
-	// the last element:
-	//   [ 4, 3, 2, 7 ]
-	// this path refers to the whole field declaration (from the beginning
-	// of the label to the terminating semicolon).
-	Path []int32 `protobuf:"varint,1,rep,packed,name=path" json:"path,omitempty"`
-	// Always has exactly three or four elements: start line, start column,
-	// end line (optional, otherwise assumed same as start line), end column.
-	// These are packed into a single field for efficiency.  Note that line
-	// and column numbers are zero-based -- typically you will want to add
-	// 1 to each before displaying to a user.
-	Span []int32 `protobuf:"varint,2,rep,packed,name=span" json:"span,omitempty"`
-	// If this SourceCodeInfo represents a complete declaration, these are any
-	// comments appearing before and after the declaration which appear to be
-	// attached to the declaration.
-	//
-	// A series of line comments appearing on consecutive lines, with no other
-	// tokens appearing on those lines, will be treated as a single comment.
-	//
-	// leading_detached_comments will keep paragraphs of comments that appear
-	// before (but not connected to) the current element. Each paragraph,
-	// separated by empty lines, will be one comment element in the repeated
-	// field.
-	//
-	// Only the comment content is provided; comment markers (e.g. //) are
-	// stripped out.  For block comments, leading whitespace and an asterisk
-	// will be stripped from the beginning of each line other than the first.
-	// Newlines are included in the output.
-	//
-	// Examples:
-	//
-	//   optional int32 foo = 1;  // Comment attached to foo.
-	//   // Comment attached to bar.
-	//   optional int32 bar = 2;
-	//
-	//   optional string baz = 3;
-	//   // Comment attached to baz.
-	//   // Another line attached to baz.
-	//
-	//   // Comment attached to qux.
-	//   //
-	//   // Another line attached to qux.
-	//   optional double qux = 4;
-	//
-	//   // Detached comment for corge. This is not leading or trailing comments
-	//   // to qux or corge because there are blank lines separating it from
-	//   // both.
-	//
-	//   // Detached comment for corge paragraph 2.
-	//
-	//   optional string corge = 5;
-	//   /* Block comment attached
-	//    * to corge.  Leading asterisks
-	//    * will be removed. */
-	//   /* Block comment attached to
-	//    * grault. */
-	//   optional int32 grault = 6;
-	//
-	//   // ignored detached comments.
-	LeadingComments         *string  `protobuf:"bytes,3,opt,name=leading_comments,json=leadingComments" json:"leading_comments,omitempty"`
-	TrailingComments        *string  `protobuf:"bytes,4,opt,name=trailing_comments,json=trailingComments" json:"trailing_comments,omitempty"`
-	LeadingDetachedComments []string `protobuf:"bytes,6,rep,name=leading_detached_comments,json=leadingDetachedComments" json:"leading_detached_comments,omitempty"`
-	XXX_NoUnkeyedLiteral    struct{} `json:"-"`
-	XXX_unrecognized        []byte   `json:"-"`
-	XXX_sizecache           int32    `json:"-"`
-}
-
-func (m *SourceCodeInfo_Location) Reset()         { *m = SourceCodeInfo_Location{} }
-func (m *SourceCodeInfo_Location) String() string { return proto.CompactTextString(m) }
-func (*SourceCodeInfo_Location) ProtoMessage()    {}
-func (*SourceCodeInfo_Location) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{19, 0}
-}
-
-func (m *SourceCodeInfo_Location) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SourceCodeInfo_Location.Unmarshal(m, b)
-}
-func (m *SourceCodeInfo_Location) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SourceCodeInfo_Location.Marshal(b, m, deterministic)
-}
-func (m *SourceCodeInfo_Location) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SourceCodeInfo_Location.Merge(m, src)
-}
-func (m *SourceCodeInfo_Location) XXX_Size() int {
-	return xxx_messageInfo_SourceCodeInfo_Location.Size(m)
-}
-func (m *SourceCodeInfo_Location) XXX_DiscardUnknown() {
-	xxx_messageInfo_SourceCodeInfo_Location.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SourceCodeInfo_Location proto.InternalMessageInfo
-
-func (m *SourceCodeInfo_Location) GetPath() []int32 {
-	if m != nil {
-		return m.Path
-	}
-	return nil
-}
-
-func (m *SourceCodeInfo_Location) GetSpan() []int32 {
-	if m != nil {
-		return m.Span
-	}
-	return nil
-}
-
-func (m *SourceCodeInfo_Location) GetLeadingComments() string {
-	if m != nil && m.LeadingComments != nil {
-		return *m.LeadingComments
-	}
-	return ""
-}
-
-func (m *SourceCodeInfo_Location) GetTrailingComments() string {
-	if m != nil && m.TrailingComments != nil {
-		return *m.TrailingComments
-	}
-	return ""
-}
-
-func (m *SourceCodeInfo_Location) GetLeadingDetachedComments() []string {
-	if m != nil {
-		return m.LeadingDetachedComments
-	}
-	return nil
-}
-
-// Describes the relationship between generated code and its original source
-// file. A GeneratedCodeInfo message is associated with only one generated
-// source file, but may contain references to different source .proto files.
-type GeneratedCodeInfo struct {
-	// An Annotation connects some span of text in generated code to an element
-	// of its generating .proto file.
-	Annotation           []*GeneratedCodeInfo_Annotation `protobuf:"bytes,1,rep,name=annotation" json:"annotation,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *GeneratedCodeInfo) Reset()         { *m = GeneratedCodeInfo{} }
-func (m *GeneratedCodeInfo) String() string { return proto.CompactTextString(m) }
-func (*GeneratedCodeInfo) ProtoMessage()    {}
-func (*GeneratedCodeInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{20}
-}
-
-func (m *GeneratedCodeInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GeneratedCodeInfo.Unmarshal(m, b)
-}
-func (m *GeneratedCodeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GeneratedCodeInfo.Marshal(b, m, deterministic)
-}
-func (m *GeneratedCodeInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GeneratedCodeInfo.Merge(m, src)
-}
-func (m *GeneratedCodeInfo) XXX_Size() int {
-	return xxx_messageInfo_GeneratedCodeInfo.Size(m)
-}
-func (m *GeneratedCodeInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_GeneratedCodeInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GeneratedCodeInfo proto.InternalMessageInfo
-
-func (m *GeneratedCodeInfo) GetAnnotation() []*GeneratedCodeInfo_Annotation {
-	if m != nil {
-		return m.Annotation
-	}
-	return nil
-}
-
-type GeneratedCodeInfo_Annotation struct {
-	// Identifies the element in the original source .proto file. This field
-	// is formatted the same as SourceCodeInfo.Location.path.
-	Path []int32 `protobuf:"varint,1,rep,packed,name=path" json:"path,omitempty"`
-	// Identifies the filesystem path to the original source .proto.
-	SourceFile *string `protobuf:"bytes,2,opt,name=source_file,json=sourceFile" json:"source_file,omitempty"`
-	// Identifies the starting offset in bytes in the generated code
-	// that relates to the identified object.
-	Begin *int32 `protobuf:"varint,3,opt,name=begin" json:"begin,omitempty"`
-	// Identifies the ending offset in bytes in the generated code that
-	// relates to the identified offset. The end offset should be one past
-	// the last relevant byte (so the length of the text = end - begin).
-	End                  *int32   `protobuf:"varint,4,opt,name=end" json:"end,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GeneratedCodeInfo_Annotation) Reset()         { *m = GeneratedCodeInfo_Annotation{} }
-func (m *GeneratedCodeInfo_Annotation) String() string { return proto.CompactTextString(m) }
-func (*GeneratedCodeInfo_Annotation) ProtoMessage()    {}
-func (*GeneratedCodeInfo_Annotation) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e5baabe45344a177, []int{20, 0}
-}
-
-func (m *GeneratedCodeInfo_Annotation) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GeneratedCodeInfo_Annotation.Unmarshal(m, b)
-}
-func (m *GeneratedCodeInfo_Annotation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GeneratedCodeInfo_Annotation.Marshal(b, m, deterministic)
-}
-func (m *GeneratedCodeInfo_Annotation) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GeneratedCodeInfo_Annotation.Merge(m, src)
-}
-func (m *GeneratedCodeInfo_Annotation) XXX_Size() int {
-	return xxx_messageInfo_GeneratedCodeInfo_Annotation.Size(m)
-}
-func (m *GeneratedCodeInfo_Annotation) XXX_DiscardUnknown() {
-	xxx_messageInfo_GeneratedCodeInfo_Annotation.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GeneratedCodeInfo_Annotation proto.InternalMessageInfo
-
-func (m *GeneratedCodeInfo_Annotation) GetPath() []int32 {
-	if m != nil {
-		return m.Path
-	}
-	return nil
-}
-
-func (m *GeneratedCodeInfo_Annotation) GetSourceFile() string {
-	if m != nil && m.SourceFile != nil {
-		return *m.SourceFile
-	}
-	return ""
-}
-
-func (m *GeneratedCodeInfo_Annotation) GetBegin() int32 {
-	if m != nil && m.Begin != nil {
-		return *m.Begin
-	}
-	return 0
-}
-
-func (m *GeneratedCodeInfo_Annotation) GetEnd() int32 {
-	if m != nil && m.End != nil {
-		return *m.End
-	}
-	return 0
-}
-
-func init() {
-	proto.RegisterEnum("google.protobuf.FieldDescriptorProto_Type", FieldDescriptorProto_Type_name, FieldDescriptorProto_Type_value)
-	proto.RegisterEnum("google.protobuf.FieldDescriptorProto_Label", FieldDescriptorProto_Label_name, FieldDescriptorProto_Label_value)
-	proto.RegisterEnum("google.protobuf.FileOptions_OptimizeMode", FileOptions_OptimizeMode_name, FileOptions_OptimizeMode_value)
-	proto.RegisterEnum("google.protobuf.FieldOptions_CType", FieldOptions_CType_name, FieldOptions_CType_value)
-	proto.RegisterEnum("google.protobuf.FieldOptions_JSType", FieldOptions_JSType_name, FieldOptions_JSType_value)
-	proto.RegisterEnum("google.protobuf.MethodOptions_IdempotencyLevel", MethodOptions_IdempotencyLevel_name, MethodOptions_IdempotencyLevel_value)
-	proto.RegisterType((*FileDescriptorSet)(nil), "google.protobuf.FileDescriptorSet")
-	proto.RegisterType((*FileDescriptorProto)(nil), "google.protobuf.FileDescriptorProto")
-	proto.RegisterType((*DescriptorProto)(nil), "google.protobuf.DescriptorProto")
-	proto.RegisterType((*DescriptorProto_ExtensionRange)(nil), "google.protobuf.DescriptorProto.ExtensionRange")
-	proto.RegisterType((*DescriptorProto_ReservedRange)(nil), "google.protobuf.DescriptorProto.ReservedRange")
-	proto.RegisterType((*ExtensionRangeOptions)(nil), "google.protobuf.ExtensionRangeOptions")
-	proto.RegisterType((*FieldDescriptorProto)(nil), "google.protobuf.FieldDescriptorProto")
-	proto.RegisterType((*OneofDescriptorProto)(nil), "google.protobuf.OneofDescriptorProto")
-	proto.RegisterType((*EnumDescriptorProto)(nil), "google.protobuf.EnumDescriptorProto")
-	proto.RegisterType((*EnumDescriptorProto_EnumReservedRange)(nil), "google.protobuf.EnumDescriptorProto.EnumReservedRange")
-	proto.RegisterType((*EnumValueDescriptorProto)(nil), "google.protobuf.EnumValueDescriptorProto")
-	proto.RegisterType((*ServiceDescriptorProto)(nil), "google.protobuf.ServiceDescriptorProto")
-	proto.RegisterType((*MethodDescriptorProto)(nil), "google.protobuf.MethodDescriptorProto")
-	proto.RegisterType((*FileOptions)(nil), "google.protobuf.FileOptions")
-	proto.RegisterType((*MessageOptions)(nil), "google.protobuf.MessageOptions")
-	proto.RegisterType((*FieldOptions)(nil), "google.protobuf.FieldOptions")
-	proto.RegisterType((*OneofOptions)(nil), "google.protobuf.OneofOptions")
-	proto.RegisterType((*EnumOptions)(nil), "google.protobuf.EnumOptions")
-	proto.RegisterType((*EnumValueOptions)(nil), "google.protobuf.EnumValueOptions")
-	proto.RegisterType((*ServiceOptions)(nil), "google.protobuf.ServiceOptions")
-	proto.RegisterType((*MethodOptions)(nil), "google.protobuf.MethodOptions")
-	proto.RegisterType((*UninterpretedOption)(nil), "google.protobuf.UninterpretedOption")
-	proto.RegisterType((*UninterpretedOption_NamePart)(nil), "google.protobuf.UninterpretedOption.NamePart")
-	proto.RegisterType((*SourceCodeInfo)(nil), "google.protobuf.SourceCodeInfo")
-	proto.RegisterType((*SourceCodeInfo_Location)(nil), "google.protobuf.SourceCodeInfo.Location")
-	proto.RegisterType((*GeneratedCodeInfo)(nil), "google.protobuf.GeneratedCodeInfo")
-	proto.RegisterType((*GeneratedCodeInfo_Annotation)(nil), "google.protobuf.GeneratedCodeInfo.Annotation")
-}
-
-func init() { proto.RegisterFile("google/protobuf/descriptor.proto", fileDescriptor_e5baabe45344a177) }
-
-var fileDescriptor_e5baabe45344a177 = []byte{
-	// 2589 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xdd, 0x8e, 0xdb, 0xc6,
-	0x15, 0x0e, 0xf5, 0xb7, 0xd2, 0x91, 0x56, 0x3b, 0x3b, 0xbb, 0xb1, 0xe9, 0xcd, 0x8f, 0xd7, 0xca,
-	0x8f, 0xd7, 0x4e, 0xac, 0x0d, 0x1c, 0xdb, 0x71, 0xd6, 0x45, 0x5a, 0xad, 0x44, 0x6f, 0xe4, 0xee,
-	0x4a, 0x2a, 0xa5, 0x6d, 0x7e, 0x80, 0x82, 0x98, 0x25, 0x47, 0x12, 0x6d, 0x8a, 0x64, 0x48, 0xca,
-	0xf6, 0x06, 0xbd, 0x30, 0xd0, 0xab, 0x5e, 0x15, 0xe8, 0x55, 0x51, 0x14, 0xbd, 0xe8, 0x4d, 0x80,
-	0x3e, 0x40, 0x81, 0xde, 0xf5, 0x09, 0x0a, 0xe4, 0x0d, 0x8a, 0xb6, 0x40, 0xfb, 0x08, 0xbd, 0x2c,
-	0x66, 0x86, 0xa4, 0x48, 0x49, 0x1b, 0x6f, 0x02, 0xc4, 0xb9, 0x92, 0xe6, 0x3b, 0xdf, 0x39, 0x73,
-	0xe6, 0xcc, 0x99, 0x99, 0x33, 0x43, 0xd8, 0x1e, 0x39, 0xce, 0xc8, 0xa2, 0xbb, 0xae, 0xe7, 0x04,
-	0xce, 0xc9, 0x74, 0xb8, 0x6b, 0x50, 0x5f, 0xf7, 0x4c, 0x37, 0x70, 0xbc, 0x3a, 0xc7, 0xf0, 0x9a,
-	0x60, 0xd4, 0x23, 0x46, 0xed, 0x08, 0xd6, 0xef, 0x9b, 0x16, 0x6d, 0xc5, 0xc4, 0x3e, 0x0d, 0xf0,
-	0x5d, 0xc8, 0x0d, 0x4d, 0x8b, 0xca, 0xd2, 0x76, 0x76, 0xa7, 0x7c, 0xf3, 0xcd, 0xfa, 0x9c, 0x52,
-	0x3d, 0xad, 0xd1, 0x63, 0xb0, 0xca, 0x35, 0x6a, 0xff, 0xce, 0xc1, 0xc6, 0x12, 0x29, 0xc6, 0x90,
-	0xb3, 0xc9, 0x84, 0x59, 0x94, 0x76, 0x4a, 0x2a, 0xff, 0x8f, 0x65, 0x58, 0x71, 0x89, 0xfe, 0x88,
-	0x8c, 0xa8, 0x9c, 0xe1, 0x70, 0xd4, 0xc4, 0xaf, 0x03, 0x18, 0xd4, 0xa5, 0xb6, 0x41, 0x6d, 0xfd,
-	0x54, 0xce, 0x6e, 0x67, 0x77, 0x4a, 0x6a, 0x02, 0xc1, 0xef, 0xc0, 0xba, 0x3b, 0x3d, 0xb1, 0x4c,
-	0x5d, 0x4b, 0xd0, 0x60, 0x3b, 0xbb, 0x93, 0x57, 0x91, 0x10, 0xb4, 0x66, 0xe4, 0xab, 0xb0, 0xf6,
-	0x84, 0x92, 0x47, 0x49, 0x6a, 0x99, 0x53, 0xab, 0x0c, 0x4e, 0x10, 0x9b, 0x50, 0x99, 0x50, 0xdf,
-	0x27, 0x23, 0xaa, 0x05, 0xa7, 0x2e, 0x95, 0x73, 0x7c, 0xf4, 0xdb, 0x0b, 0xa3, 0x9f, 0x1f, 0x79,
-	0x39, 0xd4, 0x1a, 0x9c, 0xba, 0x14, 0x37, 0xa0, 0x44, 0xed, 0xe9, 0x44, 0x58, 0xc8, 0x9f, 0x11,
-	0x3f, 0xc5, 0x9e, 0x4e, 0xe6, 0xad, 0x14, 0x99, 0x5a, 0x68, 0x62, 0xc5, 0xa7, 0xde, 0x63, 0x53,
-	0xa7, 0x72, 0x81, 0x1b, 0xb8, 0xba, 0x60, 0xa0, 0x2f, 0xe4, 0xf3, 0x36, 0x22, 0x3d, 0xdc, 0x84,
-	0x12, 0x7d, 0x1a, 0x50, 0xdb, 0x37, 0x1d, 0x5b, 0x5e, 0xe1, 0x46, 0xde, 0x5a, 0x32, 0x8b, 0xd4,
-	0x32, 0xe6, 0x4d, 0xcc, 0xf4, 0xf0, 0x1d, 0x58, 0x71, 0xdc, 0xc0, 0x74, 0x6c, 0x5f, 0x2e, 0x6e,
-	0x4b, 0x3b, 0xe5, 0x9b, 0xaf, 0x2e, 0x4d, 0x84, 0xae, 0xe0, 0xa8, 0x11, 0x19, 0xb7, 0x01, 0xf9,
-	0xce, 0xd4, 0xd3, 0xa9, 0xa6, 0x3b, 0x06, 0xd5, 0x4c, 0x7b, 0xe8, 0xc8, 0x25, 0x6e, 0xe0, 0xf2,
-	0xe2, 0x40, 0x38, 0xb1, 0xe9, 0x18, 0xb4, 0x6d, 0x0f, 0x1d, 0xb5, 0xea, 0xa7, 0xda, 0xf8, 0x02,
-	0x14, 0xfc, 0x53, 0x3b, 0x20, 0x4f, 0xe5, 0x0a, 0xcf, 0x90, 0xb0, 0x55, 0xfb, 0x6b, 0x01, 0xd6,
-	0xce, 0x93, 0x62, 0xf7, 0x20, 0x3f, 0x64, 0xa3, 0x94, 0x33, 0xdf, 0x26, 0x06, 0x42, 0x27, 0x1d,
-	0xc4, 0xc2, 0x77, 0x0c, 0x62, 0x03, 0xca, 0x36, 0xf5, 0x03, 0x6a, 0x88, 0x8c, 0xc8, 0x9e, 0x33,
-	0xa7, 0x40, 0x28, 0x2d, 0xa6, 0x54, 0xee, 0x3b, 0xa5, 0xd4, 0xa7, 0xb0, 0x16, 0xbb, 0xa4, 0x79,
-	0xc4, 0x1e, 0x45, 0xb9, 0xb9, 0xfb, 0x3c, 0x4f, 0xea, 0x4a, 0xa4, 0xa7, 0x32, 0x35, 0xb5, 0x4a,
-	0x53, 0x6d, 0xdc, 0x02, 0x70, 0x6c, 0xea, 0x0c, 0x35, 0x83, 0xea, 0x96, 0x5c, 0x3c, 0x23, 0x4a,
-	0x5d, 0x46, 0x59, 0x88, 0x92, 0x23, 0x50, 0xdd, 0xc2, 0x1f, 0xce, 0x52, 0x6d, 0xe5, 0x8c, 0x4c,
-	0x39, 0x12, 0x8b, 0x6c, 0x21, 0xdb, 0x8e, 0xa1, 0xea, 0x51, 0x96, 0xf7, 0xd4, 0x08, 0x47, 0x56,
-	0xe2, 0x4e, 0xd4, 0x9f, 0x3b, 0x32, 0x35, 0x54, 0x13, 0x03, 0x5b, 0xf5, 0x92, 0x4d, 0xfc, 0x06,
-	0xc4, 0x80, 0xc6, 0xd3, 0x0a, 0xf8, 0x2e, 0x54, 0x89, 0xc0, 0x0e, 0x99, 0xd0, 0xad, 0x2f, 0xa1,
-	0x9a, 0x0e, 0x0f, 0xde, 0x84, 0xbc, 0x1f, 0x10, 0x2f, 0xe0, 0x59, 0x98, 0x57, 0x45, 0x03, 0x23,
-	0xc8, 0x52, 0xdb, 0xe0, 0xbb, 0x5c, 0x5e, 0x65, 0x7f, 0xf1, 0x4f, 0x66, 0x03, 0xce, 0xf2, 0x01,
-	0xbf, 0xbd, 0x38, 0xa3, 0x29, 0xcb, 0xf3, 0xe3, 0xde, 0xfa, 0x00, 0x56, 0x53, 0x03, 0x38, 0x6f,
-	0xd7, 0xb5, 0x5f, 0xc2, 0xcb, 0x4b, 0x4d, 0xe3, 0x4f, 0x61, 0x73, 0x6a, 0x9b, 0x76, 0x40, 0x3d,
-	0xd7, 0xa3, 0x2c, 0x63, 0x45, 0x57, 0xf2, 0x7f, 0x56, 0xce, 0xc8, 0xb9, 0xe3, 0x24, 0x5b, 0x58,
-	0x51, 0x37, 0xa6, 0x8b, 0xe0, 0xf5, 0x52, 0xf1, 0xbf, 0x2b, 0xe8, 0xd9, 0xb3, 0x67, 0xcf, 0x32,
-	0xb5, 0xdf, 0x15, 0x60, 0x73, 0xd9, 0x9a, 0x59, 0xba, 0x7c, 0x2f, 0x40, 0xc1, 0x9e, 0x4e, 0x4e,
-	0xa8, 0xc7, 0x83, 0x94, 0x57, 0xc3, 0x16, 0x6e, 0x40, 0xde, 0x22, 0x27, 0xd4, 0x92, 0x73, 0xdb,
-	0xd2, 0x4e, 0xf5, 0xe6, 0x3b, 0xe7, 0x5a, 0x95, 0xf5, 0x43, 0xa6, 0xa2, 0x0a, 0x4d, 0xfc, 0x11,
-	0xe4, 0xc2, 0x2d, 0x9a, 0x59, 0xb8, 0x7e, 0x3e, 0x0b, 0x6c, 0x2d, 0xa9, 0x5c, 0x0f, 0xbf, 0x02,
-	0x25, 0xf6, 0x2b, 0x72, 0xa3, 0xc0, 0x7d, 0x2e, 0x32, 0x80, 0xe5, 0x05, 0xde, 0x82, 0x22, 0x5f,
-	0x26, 0x06, 0x8d, 0x8e, 0xb6, 0xb8, 0xcd, 0x12, 0xcb, 0xa0, 0x43, 0x32, 0xb5, 0x02, 0xed, 0x31,
-	0xb1, 0xa6, 0x94, 0x27, 0x7c, 0x49, 0xad, 0x84, 0xe0, 0xcf, 0x19, 0x86, 0x2f, 0x43, 0x59, 0xac,
-	0x2a, 0xd3, 0x36, 0xe8, 0x53, 0xbe, 0x7b, 0xe6, 0x55, 0xb1, 0xd0, 0xda, 0x0c, 0x61, 0xdd, 0x3f,
-	0xf4, 0x1d, 0x3b, 0x4a, 0x4d, 0xde, 0x05, 0x03, 0x78, 0xf7, 0x1f, 0xcc, 0x6f, 0xdc, 0xaf, 0x2d,
-	0x1f, 0xde, 0x7c, 0x4e, 0xd5, 0xfe, 0x92, 0x81, 0x1c, 0xdf, 0x2f, 0xd6, 0xa0, 0x3c, 0xf8, 0xac,
-	0xa7, 0x68, 0xad, 0xee, 0xf1, 0xfe, 0xa1, 0x82, 0x24, 0x5c, 0x05, 0xe0, 0xc0, 0xfd, 0xc3, 0x6e,
-	0x63, 0x80, 0x32, 0x71, 0xbb, 0xdd, 0x19, 0xdc, 0xb9, 0x85, 0xb2, 0xb1, 0xc2, 0xb1, 0x00, 0x72,
-	0x49, 0xc2, 0xfb, 0x37, 0x51, 0x1e, 0x23, 0xa8, 0x08, 0x03, 0xed, 0x4f, 0x95, 0xd6, 0x9d, 0x5b,
-	0xa8, 0x90, 0x46, 0xde, 0xbf, 0x89, 0x56, 0xf0, 0x2a, 0x94, 0x38, 0xb2, 0xdf, 0xed, 0x1e, 0xa2,
-	0x62, 0x6c, 0xb3, 0x3f, 0x50, 0xdb, 0x9d, 0x03, 0x54, 0x8a, 0x6d, 0x1e, 0xa8, 0xdd, 0xe3, 0x1e,
-	0x82, 0xd8, 0xc2, 0x91, 0xd2, 0xef, 0x37, 0x0e, 0x14, 0x54, 0x8e, 0x19, 0xfb, 0x9f, 0x0d, 0x94,
-	0x3e, 0xaa, 0xa4, 0xdc, 0x7a, 0xff, 0x26, 0x5a, 0x8d, 0xbb, 0x50, 0x3a, 0xc7, 0x47, 0xa8, 0x8a,
-	0xd7, 0x61, 0x55, 0x74, 0x11, 0x39, 0xb1, 0x36, 0x07, 0xdd, 0xb9, 0x85, 0xd0, 0xcc, 0x11, 0x61,
-	0x65, 0x3d, 0x05, 0xdc, 0xb9, 0x85, 0x70, 0xad, 0x09, 0x79, 0x9e, 0x5d, 0x18, 0x43, 0xf5, 0xb0,
-	0xb1, 0xaf, 0x1c, 0x6a, 0xdd, 0xde, 0xa0, 0xdd, 0xed, 0x34, 0x0e, 0x91, 0x34, 0xc3, 0x54, 0xe5,
-	0x67, 0xc7, 0x6d, 0x55, 0x69, 0xa1, 0x4c, 0x12, 0xeb, 0x29, 0x8d, 0x81, 0xd2, 0x42, 0xd9, 0x9a,
-	0x0e, 0x9b, 0xcb, 0xf6, 0xc9, 0xa5, 0x2b, 0x23, 0x31, 0xc5, 0x99, 0x33, 0xa6, 0x98, 0xdb, 0x5a,
-	0x98, 0xe2, 0x7f, 0x65, 0x60, 0x63, 0xc9, 0x59, 0xb1, 0xb4, 0x93, 0x1f, 0x43, 0x5e, 0xa4, 0xa8,
-	0x38, 0x3d, 0xaf, 0x2d, 0x3d, 0x74, 0x78, 0xc2, 0x2e, 0x9c, 0xa0, 0x5c, 0x2f, 0x59, 0x41, 0x64,
-	0xcf, 0xa8, 0x20, 0x98, 0x89, 0x85, 0x3d, 0xfd, 0x17, 0x0b, 0x7b, 0xba, 0x38, 0xf6, 0xee, 0x9c,
-	0xe7, 0xd8, 0xe3, 0xd8, 0xb7, 0xdb, 0xdb, 0xf3, 0x4b, 0xf6, 0xf6, 0x7b, 0xb0, 0xbe, 0x60, 0xe8,
-	0xdc, 0x7b, 0xec, 0xaf, 0x24, 0x90, 0xcf, 0x0a, 0xce, 0x73, 0x76, 0xba, 0x4c, 0x6a, 0xa7, 0xbb,
-	0x37, 0x1f, 0xc1, 0x2b, 0x67, 0x4f, 0xc2, 0xc2, 0x5c, 0x7f, 0x25, 0xc1, 0x85, 0xe5, 0x95, 0xe2,
-	0x52, 0x1f, 0x3e, 0x82, 0xc2, 0x84, 0x06, 0x63, 0x27, 0xaa, 0x96, 0xde, 0x5e, 0x72, 0x06, 0x33,
-	0xf1, 0xfc, 0x64, 0x87, 0x5a, 0xc9, 0x43, 0x3c, 0x7b, 0x56, 0xb9, 0x27, 0xbc, 0x59, 0xf0, 0xf4,
-	0xd7, 0x19, 0x78, 0x79, 0xa9, 0xf1, 0xa5, 0x8e, 0xbe, 0x06, 0x60, 0xda, 0xee, 0x34, 0x10, 0x15,
-	0x91, 0xd8, 0x60, 0x4b, 0x1c, 0xe1, 0x9b, 0x17, 0xdb, 0x3c, 0xa7, 0x41, 0x2c, 0xcf, 0x72, 0x39,
-	0x08, 0x88, 0x13, 0xee, 0xce, 0x1c, 0xcd, 0x71, 0x47, 0x5f, 0x3f, 0x63, 0xa4, 0x0b, 0x89, 0xf9,
-	0x1e, 0x20, 0xdd, 0x32, 0xa9, 0x1d, 0x68, 0x7e, 0xe0, 0x51, 0x32, 0x31, 0xed, 0x11, 0x3f, 0x41,
-	0x8a, 0x7b, 0xf9, 0x21, 0xb1, 0x7c, 0xaa, 0xae, 0x09, 0x71, 0x3f, 0x92, 0x32, 0x0d, 0x9e, 0x40,
-	0x5e, 0x42, 0xa3, 0x90, 0xd2, 0x10, 0xe2, 0x58, 0xa3, 0xf6, 0xdb, 0x12, 0x94, 0x13, 0x75, 0x35,
-	0xbe, 0x02, 0x95, 0x87, 0xe4, 0x31, 0xd1, 0xa2, 0xbb, 0x92, 0x88, 0x44, 0x99, 0x61, 0xbd, 0xf0,
-	0xbe, 0xf4, 0x1e, 0x6c, 0x72, 0x8a, 0x33, 0x0d, 0xa8, 0xa7, 0xe9, 0x16, 0xf1, 0x7d, 0x1e, 0xb4,
-	0x22, 0xa7, 0x62, 0x26, 0xeb, 0x32, 0x51, 0x33, 0x92, 0xe0, 0xdb, 0xb0, 0xc1, 0x35, 0x26, 0x53,
-	0x2b, 0x30, 0x5d, 0x8b, 0x6a, 0xec, 0xf6, 0xe6, 0xf3, 0x93, 0x24, 0xf6, 0x6c, 0x9d, 0x31, 0x8e,
-	0x42, 0x02, 0xf3, 0xc8, 0xc7, 0x2d, 0x78, 0x8d, 0xab, 0x8d, 0xa8, 0x4d, 0x3d, 0x12, 0x50, 0x8d,
-	0x7e, 0x31, 0x25, 0x96, 0xaf, 0x11, 0xdb, 0xd0, 0xc6, 0xc4, 0x1f, 0xcb, 0x9b, 0xcc, 0xc0, 0x7e,
-	0x46, 0x96, 0xd4, 0x4b, 0x8c, 0x78, 0x10, 0xf2, 0x14, 0x4e, 0x6b, 0xd8, 0xc6, 0xc7, 0xc4, 0x1f,
-	0xe3, 0x3d, 0xb8, 0xc0, 0xad, 0xf8, 0x81, 0x67, 0xda, 0x23, 0x4d, 0x1f, 0x53, 0xfd, 0x91, 0x36,
-	0x0d, 0x86, 0x77, 0xe5, 0x57, 0x92, 0xfd, 0x73, 0x0f, 0xfb, 0x9c, 0xd3, 0x64, 0x94, 0xe3, 0x60,
-	0x78, 0x17, 0xf7, 0xa1, 0xc2, 0x26, 0x63, 0x62, 0x7e, 0x49, 0xb5, 0xa1, 0xe3, 0xf1, 0xa3, 0xb1,
-	0xba, 0x64, 0x6b, 0x4a, 0x44, 0xb0, 0xde, 0x0d, 0x15, 0x8e, 0x1c, 0x83, 0xee, 0xe5, 0xfb, 0x3d,
-	0x45, 0x69, 0xa9, 0xe5, 0xc8, 0xca, 0x7d, 0xc7, 0x63, 0x09, 0x35, 0x72, 0xe2, 0x00, 0x97, 0x45,
-	0x42, 0x8d, 0x9c, 0x28, 0xbc, 0xb7, 0x61, 0x43, 0xd7, 0xc5, 0x98, 0x4d, 0x5d, 0x0b, 0xef, 0x58,
-	0xbe, 0x8c, 0x52, 0xc1, 0xd2, 0xf5, 0x03, 0x41, 0x08, 0x73, 0xdc, 0xc7, 0x1f, 0xc2, 0xcb, 0xb3,
-	0x60, 0x25, 0x15, 0xd7, 0x17, 0x46, 0x39, 0xaf, 0x7a, 0x1b, 0x36, 0xdc, 0xd3, 0x45, 0x45, 0x9c,
-	0xea, 0xd1, 0x3d, 0x9d, 0x57, 0xfb, 0x00, 0x36, 0xdd, 0xb1, 0xbb, 0xa8, 0x77, 0x3d, 0xa9, 0x87,
-	0xdd, 0xb1, 0x3b, 0xaf, 0xf8, 0x16, 0xbf, 0x70, 0x7b, 0x54, 0x27, 0x01, 0x35, 0xe4, 0x8b, 0x49,
-	0x7a, 0x42, 0x80, 0x77, 0x01, 0xe9, 0xba, 0x46, 0x6d, 0x72, 0x62, 0x51, 0x8d, 0x78, 0xd4, 0x26,
-	0xbe, 0x7c, 0x39, 0x49, 0xae, 0xea, 0xba, 0xc2, 0xa5, 0x0d, 0x2e, 0xc4, 0xd7, 0x61, 0xdd, 0x39,
-	0x79, 0xa8, 0x8b, 0x94, 0xd4, 0x5c, 0x8f, 0x0e, 0xcd, 0xa7, 0xf2, 0x9b, 0x3c, 0xbe, 0x6b, 0x4c,
-	0xc0, 0x13, 0xb2, 0xc7, 0x61, 0x7c, 0x0d, 0x90, 0xee, 0x8f, 0x89, 0xe7, 0xf2, 0x3d, 0xd9, 0x77,
-	0x89, 0x4e, 0xe5, 0xb7, 0x04, 0x55, 0xe0, 0x9d, 0x08, 0x66, 0x4b, 0xc2, 0x7f, 0x62, 0x0e, 0x83,
-	0xc8, 0xe2, 0x55, 0xb1, 0x24, 0x38, 0x16, 0x5a, 0xdb, 0x01, 0xc4, 0x42, 0x91, 0xea, 0x78, 0x87,
-	0xd3, 0xaa, 0xee, 0xd8, 0x4d, 0xf6, 0xfb, 0x06, 0xac, 0x32, 0xe6, 0xac, 0xd3, 0x6b, 0xa2, 0x20,
-	0x73, 0xc7, 0x89, 0x1e, 0x6f, 0xc1, 0x05, 0x46, 0x9a, 0xd0, 0x80, 0x18, 0x24, 0x20, 0x09, 0xf6,
-	0xbb, 0x9c, 0xcd, 0xe2, 0x7e, 0x14, 0x0a, 0x53, 0x7e, 0x7a, 0xd3, 0x93, 0xd3, 0x38, 0xb3, 0x6e,
-	0x08, 0x3f, 0x19, 0x16, 0xe5, 0xd6, 0xf7, 0x56, 0x74, 0xd7, 0xf6, 0xa0, 0x92, 0x4c, 0x7c, 0x5c,
-	0x02, 0x91, 0xfa, 0x48, 0x62, 0x55, 0x50, 0xb3, 0xdb, 0x62, 0xf5, 0xcb, 0xe7, 0x0a, 0xca, 0xb0,
-	0x3a, 0xea, 0xb0, 0x3d, 0x50, 0x34, 0xf5, 0xb8, 0x33, 0x68, 0x1f, 0x29, 0x28, 0x9b, 0x28, 0xd8,
-	0x1f, 0xe4, 0x8a, 0x6f, 0xa3, 0xab, 0xb5, 0xaf, 0x33, 0x50, 0x4d, 0xdf, 0xc0, 0xf0, 0x8f, 0xe0,
-	0x62, 0xf4, 0x5c, 0xe2, 0xd3, 0x40, 0x7b, 0x62, 0x7a, 0x7c, 0x45, 0x4e, 0x88, 0x38, 0x1d, 0xe3,
-	0x9c, 0xd8, 0x0c, 0x59, 0x7d, 0x1a, 0x7c, 0x62, 0x7a, 0x6c, 0xbd, 0x4d, 0x48, 0x80, 0x0f, 0xe1,
-	0xb2, 0xed, 0x68, 0x7e, 0x40, 0x6c, 0x83, 0x78, 0x86, 0x36, 0x7b, 0xa8, 0xd2, 0x88, 0xae, 0x53,
-	0xdf, 0x77, 0xc4, 0x49, 0x18, 0x5b, 0x79, 0xd5, 0x76, 0xfa, 0x21, 0x79, 0x76, 0x44, 0x34, 0x42,
-	0xea, 0x5c, 0xfe, 0x66, 0xcf, 0xca, 0xdf, 0x57, 0xa0, 0x34, 0x21, 0xae, 0x46, 0xed, 0xc0, 0x3b,
-	0xe5, 0x75, 0x77, 0x51, 0x2d, 0x4e, 0x88, 0xab, 0xb0, 0xf6, 0x0b, 0xb9, 0xfe, 0x3c, 0xc8, 0x15,
-	0x8b, 0xa8, 0xf4, 0x20, 0x57, 0x2c, 0x21, 0xa8, 0xfd, 0x33, 0x0b, 0x95, 0x64, 0x1d, 0xce, 0xae,
-	0x35, 0x3a, 0x3f, 0xb2, 0x24, 0xbe, 0xa9, 0xbd, 0xf1, 0x8d, 0x55, 0x7b, 0xbd, 0xc9, 0xce, 0xb2,
-	0xbd, 0x82, 0xa8, 0x8e, 0x55, 0xa1, 0xc9, 0xea, 0x08, 0x96, 0x6c, 0x54, 0x54, 0x23, 0x45, 0x35,
-	0x6c, 0xe1, 0x03, 0x28, 0x3c, 0xf4, 0xb9, 0xed, 0x02, 0xb7, 0xfd, 0xe6, 0x37, 0xdb, 0x7e, 0xd0,
-	0xe7, 0xc6, 0x4b, 0x0f, 0xfa, 0x5a, 0xa7, 0xab, 0x1e, 0x35, 0x0e, 0xd5, 0x50, 0x1d, 0x5f, 0x82,
-	0x9c, 0x45, 0xbe, 0x3c, 0x4d, 0x9f, 0x7a, 0x1c, 0x3a, 0xef, 0x24, 0x5c, 0x82, 0xdc, 0x13, 0x4a,
-	0x1e, 0xa5, 0xcf, 0x1a, 0x0e, 0x7d, 0x8f, 0x8b, 0x61, 0x17, 0xf2, 0x3c, 0x5e, 0x18, 0x20, 0x8c,
-	0x18, 0x7a, 0x09, 0x17, 0x21, 0xd7, 0xec, 0xaa, 0x6c, 0x41, 0x20, 0xa8, 0x08, 0x54, 0xeb, 0xb5,
-	0x95, 0xa6, 0x82, 0x32, 0xb5, 0xdb, 0x50, 0x10, 0x41, 0x60, 0x8b, 0x25, 0x0e, 0x03, 0x7a, 0x29,
-	0x6c, 0x86, 0x36, 0xa4, 0x48, 0x7a, 0x7c, 0xb4, 0xaf, 0xa8, 0x28, 0x93, 0x9e, 0xea, 0x1c, 0xca,
-	0xd7, 0x7c, 0xa8, 0x24, 0x0b, 0xf1, 0x17, 0x73, 0xc9, 0xfe, 0x9b, 0x04, 0xe5, 0x44, 0x61, 0xcd,
-	0x2a, 0x22, 0x62, 0x59, 0xce, 0x13, 0x8d, 0x58, 0x26, 0xf1, 0xc3, 0xd4, 0x00, 0x0e, 0x35, 0x18,
-	0x72, 0xde, 0xa9, 0x7b, 0x41, 0x4b, 0x24, 0x8f, 0x0a, 0xb5, 0x3f, 0x4a, 0x80, 0xe6, 0x2b, 0xdb,
-	0x39, 0x37, 0xa5, 0x1f, 0xd2, 0xcd, 0xda, 0x1f, 0x24, 0xa8, 0xa6, 0xcb, 0xd9, 0x39, 0xf7, 0xae,
-	0xfc, 0xa0, 0xee, 0xfd, 0x23, 0x03, 0xab, 0xa9, 0x22, 0xf6, 0xbc, 0xde, 0x7d, 0x01, 0xeb, 0xa6,
-	0x41, 0x27, 0xae, 0x13, 0x50, 0x5b, 0x3f, 0xd5, 0x2c, 0xfa, 0x98, 0x5a, 0x72, 0x8d, 0x6f, 0x1a,
-	0xbb, 0xdf, 0x5c, 0x26, 0xd7, 0xdb, 0x33, 0xbd, 0x43, 0xa6, 0xb6, 0xb7, 0xd1, 0x6e, 0x29, 0x47,
-	0xbd, 0xee, 0x40, 0xe9, 0x34, 0x3f, 0xd3, 0x8e, 0x3b, 0x3f, 0xed, 0x74, 0x3f, 0xe9, 0xa8, 0xc8,
-	0x9c, 0xa3, 0x7d, 0x8f, 0xcb, 0xbe, 0x07, 0x68, 0xde, 0x29, 0x7c, 0x11, 0x96, 0xb9, 0x85, 0x5e,
-	0xc2, 0x1b, 0xb0, 0xd6, 0xe9, 0x6a, 0xfd, 0x76, 0x4b, 0xd1, 0x94, 0xfb, 0xf7, 0x95, 0xe6, 0xa0,
-	0x2f, 0x1e, 0x3e, 0x62, 0xf6, 0x20, 0xb5, 0xc0, 0x6b, 0xbf, 0xcf, 0xc2, 0xc6, 0x12, 0x4f, 0x70,
-	0x23, 0xbc, 0xb2, 0x88, 0x5b, 0xd4, 0x8d, 0xf3, 0x78, 0x5f, 0x67, 0x35, 0x43, 0x8f, 0x78, 0x41,
-	0x78, 0xc3, 0xb9, 0x06, 0x2c, 0x4a, 0x76, 0x60, 0x0e, 0x4d, 0xea, 0x85, 0xef, 0x44, 0xe2, 0x1e,
-	0xb3, 0x36, 0xc3, 0xc5, 0x53, 0xd1, 0xbb, 0x80, 0x5d, 0xc7, 0x37, 0x03, 0xf3, 0x31, 0xd5, 0x4c,
-	0x3b, 0x7a, 0x54, 0x62, 0xf7, 0x9a, 0x9c, 0x8a, 0x22, 0x49, 0xdb, 0x0e, 0x62, 0xb6, 0x4d, 0x47,
-	0x64, 0x8e, 0xcd, 0x36, 0xf3, 0xac, 0x8a, 0x22, 0x49, 0xcc, 0xbe, 0x02, 0x15, 0xc3, 0x99, 0xb2,
-	0x62, 0x4f, 0xf0, 0xd8, 0xd9, 0x21, 0xa9, 0x65, 0x81, 0xc5, 0x94, 0xb0, 0x8c, 0x9f, 0xbd, 0x66,
-	0x55, 0xd4, 0xb2, 0xc0, 0x04, 0xe5, 0x2a, 0xac, 0x91, 0xd1, 0xc8, 0x63, 0xc6, 0x23, 0x43, 0xe2,
-	0x62, 0x52, 0x8d, 0x61, 0x4e, 0xdc, 0x7a, 0x00, 0xc5, 0x28, 0x0e, 0xec, 0xa8, 0x66, 0x91, 0xd0,
-	0x5c, 0x71, 0xdb, 0xce, 0xec, 0x94, 0xd4, 0xa2, 0x1d, 0x09, 0xaf, 0x40, 0xc5, 0xf4, 0xb5, 0xd9,
-	0xe3, 0x7c, 0x66, 0x3b, 0xb3, 0x53, 0x54, 0xcb, 0xa6, 0x1f, 0x3f, 0x6c, 0xd6, 0xbe, 0xca, 0x40,
-	0x35, 0xfd, 0x71, 0x01, 0xb7, 0xa0, 0x68, 0x39, 0x3a, 0xe1, 0xa9, 0x25, 0xbe, 0x6c, 0xed, 0x3c,
-	0xe7, 0x7b, 0x44, 0xfd, 0x30, 0xe4, 0xab, 0xb1, 0xe6, 0xd6, 0xdf, 0x25, 0x28, 0x46, 0x30, 0xbe,
-	0x00, 0x39, 0x97, 0x04, 0x63, 0x6e, 0x2e, 0xbf, 0x9f, 0x41, 0x92, 0xca, 0xdb, 0x0c, 0xf7, 0x5d,
-	0x62, 0xf3, 0x14, 0x08, 0x71, 0xd6, 0x66, 0xf3, 0x6a, 0x51, 0x62, 0xf0, 0x5b, 0x8f, 0x33, 0x99,
-	0x50, 0x3b, 0xf0, 0xa3, 0x79, 0x0d, 0xf1, 0x66, 0x08, 0xe3, 0x77, 0x60, 0x3d, 0xf0, 0x88, 0x69,
-	0xa5, 0xb8, 0x39, 0xce, 0x45, 0x91, 0x20, 0x26, 0xef, 0xc1, 0xa5, 0xc8, 0xae, 0x41, 0x03, 0xa2,
-	0x8f, 0xa9, 0x31, 0x53, 0x2a, 0xf0, 0xd7, 0x8d, 0x8b, 0x21, 0xa1, 0x15, 0xca, 0x23, 0xdd, 0xda,
-	0xd7, 0x12, 0xac, 0x47, 0xf7, 0x34, 0x23, 0x0e, 0xd6, 0x11, 0x00, 0xb1, 0x6d, 0x27, 0x48, 0x86,
-	0x6b, 0x31, 0x95, 0x17, 0xf4, 0xea, 0x8d, 0x58, 0x49, 0x4d, 0x18, 0xd8, 0x9a, 0x00, 0xcc, 0x24,
-	0x67, 0x86, 0xed, 0x32, 0x94, 0xc3, 0x2f, 0x47, 0xfc, 0xf3, 0xa3, 0xb8, 0xd9, 0x83, 0x80, 0xd8,
-	0x85, 0x0e, 0x6f, 0x42, 0xfe, 0x84, 0x8e, 0x4c, 0x3b, 0x7c, 0x0f, 0x16, 0x8d, 0xe8, 0xfd, 0x25,
-	0x17, 0xbf, 0xbf, 0xec, 0xff, 0x46, 0x82, 0x0d, 0xdd, 0x99, 0xcc, 0xfb, 0xbb, 0x8f, 0xe6, 0x9e,
-	0x17, 0xfc, 0x8f, 0xa5, 0xcf, 0x3f, 0x1a, 0x99, 0xc1, 0x78, 0x7a, 0x52, 0xd7, 0x9d, 0xc9, 0xee,
-	0xc8, 0xb1, 0x88, 0x3d, 0x9a, 0x7d, 0x3f, 0xe5, 0x7f, 0xf4, 0x1b, 0x23, 0x6a, 0xdf, 0x18, 0x39,
-	0x89, 0xaf, 0xa9, 0xf7, 0x66, 0x7f, 0xff, 0x27, 0x49, 0x7f, 0xca, 0x64, 0x0f, 0x7a, 0xfb, 0x7f,
-	0xce, 0x6c, 0x1d, 0x88, 0xee, 0x7a, 0x51, 0x78, 0x54, 0x3a, 0xb4, 0xa8, 0xce, 0x86, 0xfc, 0xff,
-	0x00, 0x00, 0x00, 0xff, 0xff, 0x3e, 0xe8, 0xef, 0xc4, 0x9b, 0x1d, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_goTypes,
+		DependencyIndexes: file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_depIdxs,
+	}.Build()
+	File_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto = out.File
+	file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_rawDesc = nil
+	file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_goTypes = nil
+	file_github_com_golang_protobuf_protoc_gen_go_descriptor_descriptor_proto_depIdxs = nil
 }
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.proto b/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.proto
deleted file mode 100644
index ed08fcb..0000000
--- a/vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.proto
+++ /dev/null
@@ -1,883 +0,0 @@
-// 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)
-//  Based on original Protocol Buffers design by
-//  Sanjay Ghemawat, Jeff Dean, and others.
-//
-// The messages in this file describe the definitions found in .proto files.
-// A valid .proto file can be translated directly to a FileDescriptorProto
-// without any other information (e.g. without reading its imports).
-
-
-syntax = "proto2";
-
-package google.protobuf;
-option go_package = "github.com/golang/protobuf/protoc-gen-go/descriptor;descriptor";
-option java_package = "com.google.protobuf";
-option java_outer_classname = "DescriptorProtos";
-option csharp_namespace = "Google.Protobuf.Reflection";
-option objc_class_prefix = "GPB";
-option cc_enable_arenas = true;
-
-// descriptor.proto must be optimized for speed because reflection-based
-// algorithms don't work during bootstrapping.
-option optimize_for = SPEED;
-
-// The protocol compiler can output a FileDescriptorSet containing the .proto
-// files it parses.
-message FileDescriptorSet {
-  repeated FileDescriptorProto file = 1;
-}
-
-// Describes a complete .proto file.
-message FileDescriptorProto {
-  optional string name = 1;       // file name, relative to root of source tree
-  optional string package = 2;    // e.g. "foo", "foo.bar", etc.
-
-  // Names of files imported by this file.
-  repeated string dependency = 3;
-  // Indexes of the public imported files in the dependency list above.
-  repeated int32 public_dependency = 10;
-  // Indexes of the weak imported files in the dependency list.
-  // For Google-internal migration only. Do not use.
-  repeated int32 weak_dependency = 11;
-
-  // All top-level definitions in this file.
-  repeated DescriptorProto message_type = 4;
-  repeated EnumDescriptorProto enum_type = 5;
-  repeated ServiceDescriptorProto service = 6;
-  repeated FieldDescriptorProto extension = 7;
-
-  optional FileOptions options = 8;
-
-  // This field contains optional information about the original source code.
-  // You may safely remove this entire field without harming runtime
-  // functionality of the descriptors -- the information is needed only by
-  // development tools.
-  optional SourceCodeInfo source_code_info = 9;
-
-  // The syntax of the proto file.
-  // The supported values are "proto2" and "proto3".
-  optional string syntax = 12;
-}
-
-// Describes a message type.
-message DescriptorProto {
-  optional string name = 1;
-
-  repeated FieldDescriptorProto field = 2;
-  repeated FieldDescriptorProto extension = 6;
-
-  repeated DescriptorProto nested_type = 3;
-  repeated EnumDescriptorProto enum_type = 4;
-
-  message ExtensionRange {
-    optional int32 start = 1;
-    optional int32 end = 2;
-
-    optional ExtensionRangeOptions options = 3;
-  }
-  repeated ExtensionRange extension_range = 5;
-
-  repeated OneofDescriptorProto oneof_decl = 8;
-
-  optional MessageOptions options = 7;
-
-  // Range of reserved tag numbers. Reserved tag numbers may not be used by
-  // fields or extension ranges in the same message. Reserved ranges may
-  // not overlap.
-  message ReservedRange {
-    optional int32 start = 1; // Inclusive.
-    optional int32 end = 2;   // Exclusive.
-  }
-  repeated ReservedRange reserved_range = 9;
-  // Reserved field names, which may not be used by fields in the same message.
-  // A given name may only be reserved once.
-  repeated string reserved_name = 10;
-}
-
-message ExtensionRangeOptions {
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-// Describes a field within a message.
-message FieldDescriptorProto {
-  enum Type {
-    // 0 is reserved for errors.
-    // Order is weird for historical reasons.
-    TYPE_DOUBLE         = 1;
-    TYPE_FLOAT          = 2;
-    // Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT64 if
-    // negative values are likely.
-    TYPE_INT64          = 3;
-    TYPE_UINT64         = 4;
-    // Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT32 if
-    // negative values are likely.
-    TYPE_INT32          = 5;
-    TYPE_FIXED64        = 6;
-    TYPE_FIXED32        = 7;
-    TYPE_BOOL           = 8;
-    TYPE_STRING         = 9;
-    // Tag-delimited aggregate.
-    // Group type is deprecated and not supported in proto3. However, Proto3
-    // implementations should still be able to parse the group wire format and
-    // treat group fields as unknown fields.
-    TYPE_GROUP          = 10;
-    TYPE_MESSAGE        = 11;  // Length-delimited aggregate.
-
-    // New in version 2.
-    TYPE_BYTES          = 12;
-    TYPE_UINT32         = 13;
-    TYPE_ENUM           = 14;
-    TYPE_SFIXED32       = 15;
-    TYPE_SFIXED64       = 16;
-    TYPE_SINT32         = 17;  // Uses ZigZag encoding.
-    TYPE_SINT64         = 18;  // Uses ZigZag encoding.
-  };
-
-  enum Label {
-    // 0 is reserved for errors
-    LABEL_OPTIONAL      = 1;
-    LABEL_REQUIRED      = 2;
-    LABEL_REPEATED      = 3;
-  };
-
-  optional string name = 1;
-  optional int32 number = 3;
-  optional Label label = 4;
-
-  // If type_name is set, this need not be set.  If both this and type_name
-  // are set, this must be one of TYPE_ENUM, TYPE_MESSAGE or TYPE_GROUP.
-  optional Type type = 5;
-
-  // For message and enum types, this is the name of the type.  If the name
-  // starts with a '.', it is fully-qualified.  Otherwise, C++-like scoping
-  // rules are used to find the type (i.e. first the nested types within this
-  // message are searched, then within the parent, on up to the root
-  // namespace).
-  optional string type_name = 6;
-
-  // For extensions, this is the name of the type being extended.  It is
-  // resolved in the same manner as type_name.
-  optional string extendee = 2;
-
-  // For numeric types, contains the original text representation of the value.
-  // For booleans, "true" or "false".
-  // For strings, contains the default text contents (not escaped in any way).
-  // For bytes, contains the C escaped value.  All bytes >= 128 are escaped.
-  // TODO(kenton):  Base-64 encode?
-  optional string default_value = 7;
-
-  // If set, gives the index of a oneof in the containing type's oneof_decl
-  // list.  This field is a member of that oneof.
-  optional int32 oneof_index = 9;
-
-  // JSON name of this field. The value is set by protocol compiler. If the
-  // user has set a "json_name" option on this field, that option's value
-  // will be used. Otherwise, it's deduced from the field's name by converting
-  // it to camelCase.
-  optional string json_name = 10;
-
-  optional FieldOptions options = 8;
-}
-
-// Describes a oneof.
-message OneofDescriptorProto {
-  optional string name = 1;
-  optional OneofOptions options = 2;
-}
-
-// Describes an enum type.
-message EnumDescriptorProto {
-  optional string name = 1;
-
-  repeated EnumValueDescriptorProto value = 2;
-
-  optional EnumOptions options = 3;
-
-  // Range of reserved numeric values. Reserved values may not be used by
-  // entries in the same enum. Reserved ranges may not overlap.
-  //
-  // Note that this is distinct from DescriptorProto.ReservedRange in that it
-  // is inclusive such that it can appropriately represent the entire int32
-  // domain.
-  message EnumReservedRange {
-    optional int32 start = 1; // Inclusive.
-    optional int32 end = 2;   // Inclusive.
-  }
-
-  // Range of reserved numeric values. Reserved numeric values may not be used
-  // by enum values in the same enum declaration. Reserved ranges may not
-  // overlap.
-  repeated EnumReservedRange reserved_range = 4;
-
-  // Reserved enum value names, which may not be reused. A given name may only
-  // be reserved once.
-  repeated string reserved_name = 5;
-}
-
-// Describes a value within an enum.
-message EnumValueDescriptorProto {
-  optional string name = 1;
-  optional int32 number = 2;
-
-  optional EnumValueOptions options = 3;
-}
-
-// Describes a service.
-message ServiceDescriptorProto {
-  optional string name = 1;
-  repeated MethodDescriptorProto method = 2;
-
-  optional ServiceOptions options = 3;
-}
-
-// Describes a method of a service.
-message MethodDescriptorProto {
-  optional string name = 1;
-
-  // Input and output type names.  These are resolved in the same way as
-  // FieldDescriptorProto.type_name, but must refer to a message type.
-  optional string input_type = 2;
-  optional string output_type = 3;
-
-  optional MethodOptions options = 4;
-
-  // Identifies if client streams multiple client messages
-  optional bool client_streaming = 5 [default=false];
-  // Identifies if server streams multiple server messages
-  optional bool server_streaming = 6 [default=false];
-}
-
-
-// ===================================================================
-// Options
-
-// Each of the definitions above may have "options" attached.  These are
-// just annotations which may cause code to be generated slightly differently
-// or may contain hints for code that manipulates protocol messages.
-//
-// Clients may define custom options as extensions of the *Options messages.
-// These extensions may not yet be known at parsing time, so the parser cannot
-// store the values in them.  Instead it stores them in a field in the *Options
-// message called uninterpreted_option. This field must have the same name
-// across all *Options messages. We then use this field to populate the
-// extensions when we build a descriptor, at which point all protos have been
-// parsed and so all extensions are known.
-//
-// Extension numbers for custom options may be chosen as follows:
-// * For options which will only be used within a single application or
-//   organization, or for experimental options, use field numbers 50000
-//   through 99999.  It is up to you to ensure that you do not use the
-//   same number for multiple options.
-// * For options which will be published and used publicly by multiple
-//   independent entities, e-mail protobuf-global-extension-registry@google.com
-//   to reserve extension numbers. Simply provide your project name (e.g.
-//   Objective-C plugin) and your project website (if available) -- there's no
-//   need to explain how you intend to use them. Usually you only need one
-//   extension number. You can declare multiple options with only one extension
-//   number by putting them in a sub-message. See the Custom Options section of
-//   the docs for examples:
-//   https://developers.google.com/protocol-buffers/docs/proto#options
-//   If this turns out to be popular, a web service will be set up
-//   to automatically assign option numbers.
-
-
-message FileOptions {
-
-  // Sets the Java package where classes generated from this .proto will be
-  // placed.  By default, the proto package is used, but this is often
-  // inappropriate because proto packages do not normally start with backwards
-  // domain names.
-  optional string java_package = 1;
-
-
-  // If set, all the classes from the .proto file are wrapped in a single
-  // outer class with the given name.  This applies to both Proto1
-  // (equivalent to the old "--one_java_file" option) and Proto2 (where
-  // a .proto always translates to a single class, but you may want to
-  // explicitly choose the class name).
-  optional string java_outer_classname = 8;
-
-  // If set true, then the Java code generator will generate a separate .java
-  // file for each top-level message, enum, and service defined in the .proto
-  // file.  Thus, these types will *not* be nested inside the outer class
-  // named by java_outer_classname.  However, the outer class will still be
-  // generated to contain the file's getDescriptor() method as well as any
-  // top-level extensions defined in the file.
-  optional bool java_multiple_files = 10 [default=false];
-
-  // This option does nothing.
-  optional bool java_generate_equals_and_hash = 20 [deprecated=true];
-
-  // If set true, then the Java2 code generator will generate code that
-  // throws an exception whenever an attempt is made to assign a non-UTF-8
-  // byte sequence to a string field.
-  // Message reflection will do the same.
-  // However, an extension field still accepts non-UTF-8 byte sequences.
-  // This option has no effect on when used with the lite runtime.
-  optional bool java_string_check_utf8 = 27 [default=false];
-
-
-  // Generated classes can be optimized for speed or code size.
-  enum OptimizeMode {
-    SPEED = 1;        // Generate complete code for parsing, serialization,
-                      // etc.
-    CODE_SIZE = 2;    // Use ReflectionOps to implement these methods.
-    LITE_RUNTIME = 3; // Generate code using MessageLite and the lite runtime.
-  }
-  optional OptimizeMode optimize_for = 9 [default=SPEED];
-
-  // Sets the Go package where structs generated from this .proto will be
-  // placed. If omitted, the Go package will be derived from the following:
-  //   - The basename of the package import path, if provided.
-  //   - Otherwise, the package statement in the .proto file, if present.
-  //   - Otherwise, the basename of the .proto file, without extension.
-  optional string go_package = 11;
-
-
-
-  // Should generic services be generated in each language?  "Generic" services
-  // are not specific to any particular RPC system.  They are generated by the
-  // main code generators in each language (without additional plugins).
-  // Generic services were the only kind of service generation supported by
-  // early versions of google.protobuf.
-  //
-  // Generic services are now considered deprecated in favor of using plugins
-  // that generate code specific to your particular RPC system.  Therefore,
-  // these default to false.  Old code which depends on generic services should
-  // explicitly set them to true.
-  optional bool cc_generic_services = 16 [default=false];
-  optional bool java_generic_services = 17 [default=false];
-  optional bool py_generic_services = 18 [default=false];
-  optional bool php_generic_services = 42 [default=false];
-
-  // Is this file deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for everything in the file, or it will be completely ignored; in the very
-  // least, this is a formalization for deprecating files.
-  optional bool deprecated = 23 [default=false];
-
-  // Enables the use of arenas for the proto messages in this file. This applies
-  // only to generated classes for C++.
-  optional bool cc_enable_arenas = 31 [default=false];
-
-
-  // Sets the objective c class prefix which is prepended to all objective c
-  // generated classes from this .proto. There is no default.
-  optional string objc_class_prefix = 36;
-
-  // Namespace for generated classes; defaults to the package.
-  optional string csharp_namespace = 37;
-
-  // By default Swift generators will take the proto package and CamelCase it
-  // replacing '.' with underscore and use that to prefix the types/symbols
-  // defined. When this options is provided, they will use this value instead
-  // to prefix the types/symbols defined.
-  optional string swift_prefix = 39;
-
-  // Sets the php class prefix which is prepended to all php generated classes
-  // from this .proto. Default is empty.
-  optional string php_class_prefix = 40;
-
-  // Use this option to change the namespace of php generated classes. Default
-  // is empty. When this option is empty, the package name will be used for
-  // determining the namespace.
-  optional string php_namespace = 41;
-
-
-  // Use this option to change the namespace of php generated metadata classes.
-  // Default is empty. When this option is empty, the proto file name will be used
-  // for determining the namespace.
-  optional string php_metadata_namespace = 44;
-
-  // Use this option to change the package of ruby generated classes. Default
-  // is empty. When this option is not set, the package name will be used for
-  // determining the ruby package.
-  optional string ruby_package = 45;
-
-  // The parser stores options it doesn't recognize here.
-  // See the documentation for the "Options" section above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message.
-  // See the documentation for the "Options" section above.
-  extensions 1000 to max;
-
-  reserved 38;
-}
-
-message MessageOptions {
-  // Set true to use the old proto1 MessageSet wire format for extensions.
-  // This is provided for backwards-compatibility with the MessageSet wire
-  // format.  You should not use this for any other reason:  It's less
-  // efficient, has fewer features, and is more complicated.
-  //
-  // The message must be defined exactly as follows:
-  //   message Foo {
-  //     option message_set_wire_format = true;
-  //     extensions 4 to max;
-  //   }
-  // Note that the message cannot have any defined fields; MessageSets only
-  // have extensions.
-  //
-  // All extensions of your type must be singular messages; e.g. they cannot
-  // be int32s, enums, or repeated messages.
-  //
-  // Because this is an option, the above two restrictions are not enforced by
-  // the protocol compiler.
-  optional bool message_set_wire_format = 1 [default=false];
-
-  // Disables the generation of the standard "descriptor()" accessor, which can
-  // conflict with a field of the same name.  This is meant to make migration
-  // from proto1 easier; new code should avoid fields named "descriptor".
-  optional bool no_standard_descriptor_accessor = 2 [default=false];
-
-  // Is this message deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for the message, or it will be completely ignored; in the very least,
-  // this is a formalization for deprecating messages.
-  optional bool deprecated = 3 [default=false];
-
-  // Whether the message is an automatically generated map entry type for the
-  // maps field.
-  //
-  // For maps fields:
-  //     map<KeyType, ValueType> map_field = 1;
-  // The parsed descriptor looks like:
-  //     message MapFieldEntry {
-  //         option map_entry = true;
-  //         optional KeyType key = 1;
-  //         optional ValueType value = 2;
-  //     }
-  //     repeated MapFieldEntry map_field = 1;
-  //
-  // Implementations may choose not to generate the map_entry=true message, but
-  // use a native map in the target language to hold the keys and values.
-  // The reflection APIs in such implementions still need to work as
-  // if the field is a repeated message field.
-  //
-  // NOTE: Do not set the option in .proto files. Always use the maps syntax
-  // instead. The option should only be implicitly set by the proto compiler
-  // parser.
-  optional bool map_entry = 7;
-
-  reserved 8;  // javalite_serializable
-  reserved 9;  // javanano_as_lite
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-message FieldOptions {
-  // The ctype option instructs the C++ code generator to use a different
-  // representation of the field than it normally would.  See the specific
-  // options below.  This option is not yet implemented in the open source
-  // release -- sorry, we'll try to include it in a future version!
-  optional CType ctype = 1 [default = STRING];
-  enum CType {
-    // Default mode.
-    STRING = 0;
-
-    CORD = 1;
-
-    STRING_PIECE = 2;
-  }
-  // The packed option can be enabled for repeated primitive fields to enable
-  // a more efficient representation on the wire. Rather than repeatedly
-  // writing the tag and type for each element, the entire array is encoded as
-  // a single length-delimited blob. In proto3, only explicit setting it to
-  // false will avoid using packed encoding.
-  optional bool packed = 2;
-
-  // The jstype option determines the JavaScript type used for values of the
-  // field.  The option is permitted only for 64 bit integral and fixed types
-  // (int64, uint64, sint64, fixed64, sfixed64).  A field with jstype JS_STRING
-  // is represented as JavaScript string, which avoids loss of precision that
-  // can happen when a large value is converted to a floating point JavaScript.
-  // Specifying JS_NUMBER for the jstype causes the generated JavaScript code to
-  // use the JavaScript "number" type.  The behavior of the default option
-  // JS_NORMAL is implementation dependent.
-  //
-  // This option is an enum to permit additional types to be added, e.g.
-  // goog.math.Integer.
-  optional JSType jstype = 6 [default = JS_NORMAL];
-  enum JSType {
-    // Use the default type.
-    JS_NORMAL = 0;
-
-    // Use JavaScript strings.
-    JS_STRING = 1;
-
-    // Use JavaScript numbers.
-    JS_NUMBER = 2;
-  }
-
-  // Should this field be parsed lazily?  Lazy applies only to message-type
-  // fields.  It means that when the outer message is initially parsed, the
-  // inner message's contents will not be parsed but instead stored in encoded
-  // form.  The inner message will actually be parsed when it is first accessed.
-  //
-  // This is only a hint.  Implementations are free to choose whether to use
-  // eager or lazy parsing regardless of the value of this option.  However,
-  // setting this option true suggests that the protocol author believes that
-  // using lazy parsing on this field is worth the additional bookkeeping
-  // overhead typically needed to implement it.
-  //
-  // This option does not affect the public interface of any generated code;
-  // all method signatures remain the same.  Furthermore, thread-safety of the
-  // interface is not affected by this option; const methods remain safe to
-  // call from multiple threads concurrently, while non-const methods continue
-  // to require exclusive access.
-  //
-  //
-  // Note that implementations may choose not to check required fields within
-  // a lazy sub-message.  That is, calling IsInitialized() on the outer message
-  // may return true even if the inner message has missing required fields.
-  // This is necessary because otherwise the inner message would have to be
-  // parsed in order to perform the check, defeating the purpose of lazy
-  // parsing.  An implementation which chooses not to check required fields
-  // must be consistent about it.  That is, for any particular sub-message, the
-  // implementation must either *always* check its required fields, or *never*
-  // check its required fields, regardless of whether or not the message has
-  // been parsed.
-  optional bool lazy = 5 [default=false];
-
-  // Is this field deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for accessors, or it will be completely ignored; in the very least, this
-  // is a formalization for deprecating fields.
-  optional bool deprecated = 3 [default=false];
-
-  // For Google-internal migration only. Do not use.
-  optional bool weak = 10 [default=false];
-
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-
-  reserved 4;  // removed jtype
-}
-
-message OneofOptions {
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-message EnumOptions {
-
-  // Set this option to true to allow mapping different tag names to the same
-  // value.
-  optional bool allow_alias = 2;
-
-  // Is this enum deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for the enum, or it will be completely ignored; in the very least, this
-  // is a formalization for deprecating enums.
-  optional bool deprecated = 3 [default=false];
-
-  reserved 5;  // javanano_as_lite
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-message EnumValueOptions {
-  // Is this enum value deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for the enum value, or it will be completely ignored; in the very least,
-  // this is a formalization for deprecating enum values.
-  optional bool deprecated = 1 [default=false];
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-message ServiceOptions {
-
-  // Note:  Field numbers 1 through 32 are reserved for Google's internal RPC
-  //   framework.  We apologize for hoarding these numbers to ourselves, but
-  //   we were already using them long before we decided to release Protocol
-  //   Buffers.
-
-  // Is this service deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for the service, or it will be completely ignored; in the very least,
-  // this is a formalization for deprecating services.
-  optional bool deprecated = 33 [default=false];
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-message MethodOptions {
-
-  // Note:  Field numbers 1 through 32 are reserved for Google's internal RPC
-  //   framework.  We apologize for hoarding these numbers to ourselves, but
-  //   we were already using them long before we decided to release Protocol
-  //   Buffers.
-
-  // Is this method deprecated?
-  // Depending on the target platform, this can emit Deprecated annotations
-  // for the method, or it will be completely ignored; in the very least,
-  // this is a formalization for deprecating methods.
-  optional bool deprecated = 33 [default=false];
-
-  // Is this method side-effect-free (or safe in HTTP parlance), or idempotent,
-  // or neither? HTTP based RPC implementation may choose GET verb for safe
-  // methods, and PUT verb for idempotent methods instead of the default POST.
-  enum IdempotencyLevel {
-    IDEMPOTENCY_UNKNOWN = 0;
-    NO_SIDE_EFFECTS     = 1; // implies idempotent
-    IDEMPOTENT          = 2; // idempotent, but may have side effects
-  }
-  optional IdempotencyLevel idempotency_level =
-      34 [default=IDEMPOTENCY_UNKNOWN];
-
-  // The parser stores options it doesn't recognize here. See above.
-  repeated UninterpretedOption uninterpreted_option = 999;
-
-  // Clients can define custom options in extensions of this message. See above.
-  extensions 1000 to max;
-}
-
-
-// A message representing a option the parser does not recognize. This only
-// appears in options protos created by the compiler::Parser class.
-// DescriptorPool resolves these when building Descriptor objects. Therefore,
-// options protos in descriptor objects (e.g. returned by Descriptor::options(),
-// or produced by Descriptor::CopyTo()) will never have UninterpretedOptions
-// in them.
-message UninterpretedOption {
-  // The name of the uninterpreted option.  Each string represents a segment in
-  // a dot-separated name.  is_extension is true iff a segment represents an
-  // extension (denoted with parentheses in options specs in .proto files).
-  // E.g.,{ ["foo", false], ["bar.baz", true], ["qux", false] } represents
-  // "foo.(bar.baz).qux".
-  message NamePart {
-    required string name_part = 1;
-    required bool is_extension = 2;
-  }
-  repeated NamePart name = 2;
-
-  // The value of the uninterpreted option, in whatever type the tokenizer
-  // identified it as during parsing. Exactly one of these should be set.
-  optional string identifier_value = 3;
-  optional uint64 positive_int_value = 4;
-  optional int64 negative_int_value = 5;
-  optional double double_value = 6;
-  optional bytes string_value = 7;
-  optional string aggregate_value = 8;
-}
-
-// ===================================================================
-// Optional source code info
-
-// Encapsulates information about the original source file from which a
-// FileDescriptorProto was generated.
-message SourceCodeInfo {
-  // A Location identifies a piece of source code in a .proto file which
-  // corresponds to a particular definition.  This information is intended
-  // to be useful to IDEs, code indexers, documentation generators, and similar
-  // tools.
-  //
-  // For example, say we have a file like:
-  //   message Foo {
-  //     optional string foo = 1;
-  //   }
-  // Let's look at just the field definition:
-  //   optional string foo = 1;
-  //   ^       ^^     ^^  ^  ^^^
-  //   a       bc     de  f  ghi
-  // We have the following locations:
-  //   span   path               represents
-  //   [a,i)  [ 4, 0, 2, 0 ]     The whole field definition.
-  //   [a,b)  [ 4, 0, 2, 0, 4 ]  The label (optional).
-  //   [c,d)  [ 4, 0, 2, 0, 5 ]  The type (string).
-  //   [e,f)  [ 4, 0, 2, 0, 1 ]  The name (foo).
-  //   [g,h)  [ 4, 0, 2, 0, 3 ]  The number (1).
-  //
-  // Notes:
-  // - A location may refer to a repeated field itself (i.e. not to any
-  //   particular index within it).  This is used whenever a set of elements are
-  //   logically enclosed in a single code segment.  For example, an entire
-  //   extend block (possibly containing multiple extension definitions) will
-  //   have an outer location whose path refers to the "extensions" repeated
-  //   field without an index.
-  // - Multiple locations may have the same path.  This happens when a single
-  //   logical declaration is spread out across multiple places.  The most
-  //   obvious example is the "extend" block again -- there may be multiple
-  //   extend blocks in the same scope, each of which will have the same path.
-  // - A location's span is not always a subset of its parent's span.  For
-  //   example, the "extendee" of an extension declaration appears at the
-  //   beginning of the "extend" block and is shared by all extensions within
-  //   the block.
-  // - Just because a location's span is a subset of some other location's span
-  //   does not mean that it is a descendent.  For example, a "group" defines
-  //   both a type and a field in a single declaration.  Thus, the locations
-  //   corresponding to the type and field and their components will overlap.
-  // - Code which tries to interpret locations should probably be designed to
-  //   ignore those that it doesn't understand, as more types of locations could
-  //   be recorded in the future.
-  repeated Location location = 1;
-  message Location {
-    // Identifies which part of the FileDescriptorProto was defined at this
-    // location.
-    //
-    // Each element is a field number or an index.  They form a path from
-    // the root FileDescriptorProto to the place where the definition.  For
-    // example, this path:
-    //   [ 4, 3, 2, 7, 1 ]
-    // refers to:
-    //   file.message_type(3)  // 4, 3
-    //       .field(7)         // 2, 7
-    //       .name()           // 1
-    // This is because FileDescriptorProto.message_type has field number 4:
-    //   repeated DescriptorProto message_type = 4;
-    // and DescriptorProto.field has field number 2:
-    //   repeated FieldDescriptorProto field = 2;
-    // and FieldDescriptorProto.name has field number 1:
-    //   optional string name = 1;
-    //
-    // Thus, the above path gives the location of a field name.  If we removed
-    // the last element:
-    //   [ 4, 3, 2, 7 ]
-    // this path refers to the whole field declaration (from the beginning
-    // of the label to the terminating semicolon).
-    repeated int32 path = 1 [packed=true];
-
-    // Always has exactly three or four elements: start line, start column,
-    // end line (optional, otherwise assumed same as start line), end column.
-    // These are packed into a single field for efficiency.  Note that line
-    // and column numbers are zero-based -- typically you will want to add
-    // 1 to each before displaying to a user.
-    repeated int32 span = 2 [packed=true];
-
-    // If this SourceCodeInfo represents a complete declaration, these are any
-    // comments appearing before and after the declaration which appear to be
-    // attached to the declaration.
-    //
-    // A series of line comments appearing on consecutive lines, with no other
-    // tokens appearing on those lines, will be treated as a single comment.
-    //
-    // leading_detached_comments will keep paragraphs of comments that appear
-    // before (but not connected to) the current element. Each paragraph,
-    // separated by empty lines, will be one comment element in the repeated
-    // field.
-    //
-    // Only the comment content is provided; comment markers (e.g. //) are
-    // stripped out.  For block comments, leading whitespace and an asterisk
-    // will be stripped from the beginning of each line other than the first.
-    // Newlines are included in the output.
-    //
-    // Examples:
-    //
-    //   optional int32 foo = 1;  // Comment attached to foo.
-    //   // Comment attached to bar.
-    //   optional int32 bar = 2;
-    //
-    //   optional string baz = 3;
-    //   // Comment attached to baz.
-    //   // Another line attached to baz.
-    //
-    //   // Comment attached to qux.
-    //   //
-    //   // Another line attached to qux.
-    //   optional double qux = 4;
-    //
-    //   // Detached comment for corge. This is not leading or trailing comments
-    //   // to qux or corge because there are blank lines separating it from
-    //   // both.
-    //
-    //   // Detached comment for corge paragraph 2.
-    //
-    //   optional string corge = 5;
-    //   /* Block comment attached
-    //    * to corge.  Leading asterisks
-    //    * will be removed. */
-    //   /* Block comment attached to
-    //    * grault. */
-    //   optional int32 grault = 6;
-    //
-    //   // ignored detached comments.
-    optional string leading_comments = 3;
-    optional string trailing_comments = 4;
-    repeated string leading_detached_comments = 6;
-  }
-}
-
-// Describes the relationship between generated code and its original source
-// file. A GeneratedCodeInfo message is associated with only one generated
-// source file, but may contain references to different source .proto files.
-message GeneratedCodeInfo {
-  // An Annotation connects some span of text in generated code to an element
-  // of its generating .proto file.
-  repeated Annotation annotation = 1;
-  message Annotation {
-    // Identifies the element in the original source .proto file. This field
-    // is formatted the same as SourceCodeInfo.Location.path.
-    repeated int32 path = 1 [packed=true];
-
-    // Identifies the filesystem path to the original source .proto.
-    optional string source_file = 2;
-
-    // Identifies the starting offset in bytes in the generated code
-    // that relates to the identified object.
-    optional int32 begin = 3;
-
-    // Identifies the ending offset in bytes in the generated code that
-    // relates to the identified offset. The end offset should be one past
-    // the last relevant byte (so the length of the text = end - begin).
-    optional int32 end = 4;
-  }
-}
diff --git a/vendor/github.com/golang/protobuf/ptypes/any.go b/vendor/github.com/golang/protobuf/ptypes/any.go
index 70276e8..85f9f57 100644
--- a/vendor/github.com/golang/protobuf/ptypes/any.go
+++ b/vendor/github.com/golang/protobuf/ptypes/any.go
@@ -1,141 +1,179 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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.
+// 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 ptypes
 
-// This file implements functions to marshal proto.Message to/from
-// google.protobuf.Any message.
-
 import (
 	"fmt"
-	"reflect"
 	"strings"
 
 	"github.com/golang/protobuf/proto"
-	"github.com/golang/protobuf/ptypes/any"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+
+	anypb "github.com/golang/protobuf/ptypes/any"
 )
 
-const googleApis = "type.googleapis.com/"
+const urlPrefix = "type.googleapis.com/"
 
-// AnyMessageName returns the name of the message contained in a google.protobuf.Any message.
+// AnyMessageName returns the message name contained in an anypb.Any message.
+// Most type assertions should use the Is function instead.
 //
-// Note that regular type assertions should be done using the Is
-// function. AnyMessageName is provided for less common use cases like filtering a
-// sequence of Any messages based on a set of allowed message type names.
-func AnyMessageName(any *any.Any) (string, error) {
+// Deprecated: Call the any.MessageName method instead.
+func AnyMessageName(any *anypb.Any) (string, error) {
+	name, err := anyMessageName(any)
+	return string(name), err
+}
+func anyMessageName(any *anypb.Any) (protoreflect.FullName, error) {
 	if any == nil {
 		return "", fmt.Errorf("message is nil")
 	}
-	slash := strings.LastIndex(any.TypeUrl, "/")
-	if slash < 0 {
+	name := protoreflect.FullName(any.TypeUrl)
+	if i := strings.LastIndex(any.TypeUrl, "/"); i >= 0 {
+		name = name[i+len("/"):]
+	}
+	if !name.IsValid() {
 		return "", fmt.Errorf("message type url %q is invalid", any.TypeUrl)
 	}
-	return any.TypeUrl[slash+1:], nil
+	return name, nil
 }
 
-// MarshalAny takes the protocol buffer and encodes it into google.protobuf.Any.
-func MarshalAny(pb proto.Message) (*any.Any, error) {
-	value, err := proto.Marshal(pb)
+// MarshalAny marshals the given message m into an anypb.Any message.
+//
+// Deprecated: Call the anypb.New function instead.
+func MarshalAny(m proto.Message) (*anypb.Any, error) {
+	switch dm := m.(type) {
+	case DynamicAny:
+		m = dm.Message
+	case *DynamicAny:
+		if dm == nil {
+			return nil, proto.ErrNil
+		}
+		m = dm.Message
+	}
+	b, err := proto.Marshal(m)
 	if err != nil {
 		return nil, err
 	}
-	return &any.Any{TypeUrl: googleApis + proto.MessageName(pb), Value: value}, nil
+	return &anypb.Any{TypeUrl: urlPrefix + proto.MessageName(m), Value: b}, nil
 }
 
-// DynamicAny is a value that can be passed to UnmarshalAny to automatically
-// allocate a proto.Message for the type specified in a google.protobuf.Any
-// message. The allocated message is stored in the embedded proto.Message.
+// Empty returns a new message of the type specified in an anypb.Any message.
+// It returns protoregistry.NotFound if the corresponding message type could not
+// be resolved in the global registry.
 //
-// Example:
-//
-//   var x ptypes.DynamicAny
-//   if err := ptypes.UnmarshalAny(a, &x); err != nil { ... }
-//   fmt.Printf("unmarshaled message: %v", x.Message)
-type DynamicAny struct {
-	proto.Message
-}
-
-// Empty returns a new proto.Message of the type specified in a
-// google.protobuf.Any message. It returns an error if corresponding message
-// type isn't linked in.
-func Empty(any *any.Any) (proto.Message, error) {
-	aname, err := AnyMessageName(any)
+// Deprecated: Use protoregistry.GlobalTypes.FindMessageByName instead
+// to resolve the message name and create a new instance of it.
+func Empty(any *anypb.Any) (proto.Message, error) {
+	name, err := anyMessageName(any)
 	if err != nil {
 		return nil, err
 	}
-
-	t := proto.MessageType(aname)
-	if t == nil {
-		return nil, fmt.Errorf("any: message type %q isn't linked in", aname)
+	mt, err := protoregistry.GlobalTypes.FindMessageByName(name)
+	if err != nil {
+		return nil, err
 	}
-	return reflect.New(t.Elem()).Interface().(proto.Message), nil
+	return proto.MessageV1(mt.New().Interface()), nil
 }
 
-// UnmarshalAny parses the protocol buffer representation in a google.protobuf.Any
-// message and places the decoded result in pb. It returns an error if type of
-// contents of Any message does not match type of pb message.
+// UnmarshalAny unmarshals the encoded value contained in the anypb.Any message
+// into the provided message m. It returns an error if the target message
+// does not match the type in the Any message or if an unmarshal error occurs.
 //
-// pb can be a proto.Message, or a *DynamicAny.
-func UnmarshalAny(any *any.Any, pb proto.Message) error {
-	if d, ok := pb.(*DynamicAny); ok {
-		if d.Message == nil {
+// The target message m may be a *DynamicAny message. If the underlying message
+// type could not be resolved, then this returns protoregistry.NotFound.
+//
+// Deprecated: Call the any.UnmarshalTo method instead.
+func UnmarshalAny(any *anypb.Any, m proto.Message) error {
+	if dm, ok := m.(*DynamicAny); ok {
+		if dm.Message == nil {
 			var err error
-			d.Message, err = Empty(any)
+			dm.Message, err = Empty(any)
 			if err != nil {
 				return err
 			}
 		}
-		return UnmarshalAny(any, d.Message)
+		m = dm.Message
 	}
 
-	aname, err := AnyMessageName(any)
+	anyName, err := AnyMessageName(any)
 	if err != nil {
 		return err
 	}
-
-	mname := proto.MessageName(pb)
-	if aname != mname {
-		return fmt.Errorf("mismatched message type: got %q want %q", aname, mname)
+	msgName := proto.MessageName(m)
+	if anyName != msgName {
+		return fmt.Errorf("mismatched message type: got %q want %q", anyName, msgName)
 	}
-	return proto.Unmarshal(any.Value, pb)
+	return proto.Unmarshal(any.Value, m)
 }
 
-// Is returns true if any value contains a given message type.
-func Is(any *any.Any, pb proto.Message) bool {
-	// The following is equivalent to AnyMessageName(any) == proto.MessageName(pb),
-	// but it avoids scanning TypeUrl for the slash.
-	if any == nil {
+// Is reports whether the Any message contains a message of the specified type.
+//
+// Deprecated: Call the any.MessageIs method instead.
+func Is(any *anypb.Any, m proto.Message) bool {
+	if any == nil || m == nil {
 		return false
 	}
-	name := proto.MessageName(pb)
-	prefix := len(any.TypeUrl) - len(name)
-	return prefix >= 1 && any.TypeUrl[prefix-1] == '/' && any.TypeUrl[prefix:] == name
+	name := proto.MessageName(m)
+	if !strings.HasSuffix(any.TypeUrl, name) {
+		return false
+	}
+	return len(any.TypeUrl) == len(name) || any.TypeUrl[len(any.TypeUrl)-len(name)-1] == '/'
+}
+
+// DynamicAny is a value that can be passed to UnmarshalAny to automatically
+// allocate a proto.Message for the type specified in an anypb.Any message.
+// The allocated message is stored in the embedded proto.Message.
+//
+// Example:
+//   var x ptypes.DynamicAny
+//   if err := ptypes.UnmarshalAny(a, &x); err != nil { ... }
+//   fmt.Printf("unmarshaled message: %v", x.Message)
+//
+// Deprecated: Use the any.UnmarshalNew method instead to unmarshal
+// the any message contents into a new instance of the underlying message.
+type DynamicAny struct{ proto.Message }
+
+func (m DynamicAny) String() string {
+	if m.Message == nil {
+		return "<nil>"
+	}
+	return m.Message.String()
+}
+func (m DynamicAny) Reset() {
+	if m.Message == nil {
+		return
+	}
+	m.Message.Reset()
+}
+func (m DynamicAny) ProtoMessage() {
+	return
+}
+func (m DynamicAny) ProtoReflect() protoreflect.Message {
+	if m.Message == nil {
+		return nil
+	}
+	return dynamicAny{proto.MessageReflect(m.Message)}
+}
+
+type dynamicAny struct{ protoreflect.Message }
+
+func (m dynamicAny) Type() protoreflect.MessageType {
+	return dynamicAnyType{m.Message.Type()}
+}
+func (m dynamicAny) New() protoreflect.Message {
+	return dynamicAnyType{m.Message.Type()}.New()
+}
+func (m dynamicAny) Interface() protoreflect.ProtoMessage {
+	return DynamicAny{proto.MessageV1(m.Message.Interface())}
+}
+
+type dynamicAnyType struct{ protoreflect.MessageType }
+
+func (t dynamicAnyType) New() protoreflect.Message {
+	return dynamicAny{t.MessageType.New()}
+}
+func (t dynamicAnyType) Zero() protoreflect.Message {
+	return dynamicAny{t.MessageType.Zero()}
 }
diff --git a/vendor/github.com/golang/protobuf/ptypes/any/any.pb.go b/vendor/github.com/golang/protobuf/ptypes/any/any.pb.go
index 78ee523..0ef27d3 100644
--- a/vendor/github.com/golang/protobuf/ptypes/any/any.pb.go
+++ b/vendor/github.com/golang/protobuf/ptypes/any/any.pb.go
@@ -1,200 +1,62 @@
 // Code generated by protoc-gen-go. DO NOT EDIT.
-// source: google/protobuf/any.proto
+// source: github.com/golang/protobuf/ptypes/any/any.proto
 
 package any
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	anypb "google.golang.org/protobuf/types/known/anypb"
+	reflect "reflect"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+// Symbols defined in public import of google/protobuf/any.proto.
 
-// 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 Any = anypb.Any
 
-// `Any` contains an arbitrary serialized protocol buffer message along with a
-// URL that describes the type of the serialized message.
-//
-// Protobuf library provides support to pack/unpack Any values in the form
-// of utility functions or additional generated methods of the Any type.
-//
-// Example 1: Pack and unpack a message in C++.
-//
-//     Foo foo = ...;
-//     Any any;
-//     any.PackFrom(foo);
-//     ...
-//     if (any.UnpackTo(&foo)) {
-//       ...
-//     }
-//
-// Example 2: Pack and unpack a message in Java.
-//
-//     Foo foo = ...;
-//     Any any = Any.pack(foo);
-//     ...
-//     if (any.is(Foo.class)) {
-//       foo = any.unpack(Foo.class);
-//     }
-//
-//  Example 3: Pack and unpack a message in Python.
-//
-//     foo = Foo(...)
-//     any = Any()
-//     any.Pack(foo)
-//     ...
-//     if any.Is(Foo.DESCRIPTOR):
-//       any.Unpack(foo)
-//       ...
-//
-//  Example 4: Pack and unpack a message in Go
-//
-//      foo := &pb.Foo{...}
-//      any, err := ptypes.MarshalAny(foo)
-//      ...
-//      foo := &pb.Foo{}
-//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
-//        ...
-//      }
-//
-// The pack methods provided by protobuf library will by default use
-// 'type.googleapis.com/full.type.name' as the type URL and the unpack
-// methods only use the fully qualified type name after the last '/'
-// in the type URL, for example "foo.bar.com/x/y.z" will yield type
-// name "y.z".
-//
-//
-// JSON
-// ====
-// The JSON representation of an `Any` value uses the regular
-// representation of the deserialized, embedded message, with an
-// additional field `@type` which contains the type URL. Example:
-//
-//     package google.profile;
-//     message Person {
-//       string first_name = 1;
-//       string last_name = 2;
-//     }
-//
-//     {
-//       "@type": "type.googleapis.com/google.profile.Person",
-//       "firstName": <string>,
-//       "lastName": <string>
-//     }
-//
-// If the embedded message type is well-known and has a custom JSON
-// representation, that representation will be embedded adding a field
-// `value` which holds the custom JSON in addition to the `@type`
-// field. Example (for message [google.protobuf.Duration][]):
-//
-//     {
-//       "@type": "type.googleapis.com/google.protobuf.Duration",
-//       "value": "1.212s"
-//     }
-//
-type Any struct {
-	// A URL/resource name that uniquely identifies the type of the serialized
-	// protocol buffer message. The last segment of the URL's path must represent
-	// the fully qualified name of the type (as in
-	// `path/google.protobuf.Duration`). The name should be in a canonical form
-	// (e.g., leading "." is not accepted).
-	//
-	// In practice, teams usually precompile into the binary all types that they
-	// expect it to use in the context of Any. However, for URLs which use the
-	// scheme `http`, `https`, or no scheme, one can optionally set up a type
-	// server that maps type URLs to message definitions as follows:
-	//
-	// * If no scheme is provided, `https` is assumed.
-	// * An HTTP GET on the URL must yield a [google.protobuf.Type][]
-	//   value in binary format, or produce an error.
-	// * Applications are allowed to cache lookup results based on the
-	//   URL, or have them precompiled into a binary to avoid any
-	//   lookup. Therefore, binary compatibility needs to be preserved
-	//   on changes to types. (Use versioned type names to manage
-	//   breaking changes.)
-	//
-	// Note: this functionality is not currently available in the official
-	// protobuf release, and it is not used for type URLs beginning with
-	// type.googleapis.com.
-	//
-	// Schemes other than `http`, `https` (or the empty scheme) might be
-	// used with implementation specific semantics.
-	//
-	TypeUrl string `protobuf:"bytes,1,opt,name=type_url,json=typeUrl,proto3" json:"type_url,omitempty"`
-	// Must be a valid serialized protocol buffer of the above specified type.
-	Value                []byte   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+var File_github_com_golang_protobuf_ptypes_any_any_proto protoreflect.FileDescriptor
+
+var file_github_com_golang_protobuf_ptypes_any_any_proto_rawDesc = []byte{
+	0x0a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79,
+	0x70, 0x65, 0x73, 0x2f, 0x61, 0x6e, 0x79, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x42, 0x2b, 0x5a, 0x29,
+	0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c, 0x61, 0x6e,
+	0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79, 0x70, 0x65,
+	0x73, 0x2f, 0x61, 0x6e, 0x79, 0x3b, 0x61, 0x6e, 0x79, 0x50, 0x00, 0x62, 0x06, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x33,
 }
 
-func (m *Any) Reset()         { *m = Any{} }
-func (m *Any) String() string { return proto.CompactTextString(m) }
-func (*Any) ProtoMessage()    {}
-func (*Any) Descriptor() ([]byte, []int) {
-	return fileDescriptor_b53526c13ae22eb4, []int{0}
+var file_github_com_golang_protobuf_ptypes_any_any_proto_goTypes = []interface{}{}
+var file_github_com_golang_protobuf_ptypes_any_any_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func (*Any) XXX_WellKnownType() string { return "Any" }
-
-func (m *Any) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Any.Unmarshal(m, b)
-}
-func (m *Any) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Any.Marshal(b, m, deterministic)
-}
-func (m *Any) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Any.Merge(m, src)
-}
-func (m *Any) XXX_Size() int {
-	return xxx_messageInfo_Any.Size(m)
-}
-func (m *Any) XXX_DiscardUnknown() {
-	xxx_messageInfo_Any.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Any proto.InternalMessageInfo
-
-func (m *Any) GetTypeUrl() string {
-	if m != nil {
-		return m.TypeUrl
+func init() { file_github_com_golang_protobuf_ptypes_any_any_proto_init() }
+func file_github_com_golang_protobuf_ptypes_any_any_proto_init() {
+	if File_github_com_golang_protobuf_ptypes_any_any_proto != nil {
+		return
 	}
-	return ""
-}
-
-func (m *Any) GetValue() []byte {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterType((*Any)(nil), "google.protobuf.Any")
-}
-
-func init() { proto.RegisterFile("google/protobuf/any.proto", fileDescriptor_b53526c13ae22eb4) }
-
-var fileDescriptor_b53526c13ae22eb4 = []byte{
-	// 185 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4c, 0xcf, 0xcf, 0x4f,
-	0xcf, 0x49, 0xd5, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0x2a, 0x4d, 0xd3, 0x4f, 0xcc, 0xab, 0xd4,
-	0x03, 0x73, 0x84, 0xf8, 0x21, 0x52, 0x7a, 0x30, 0x29, 0x25, 0x33, 0x2e, 0x66, 0xc7, 0xbc, 0x4a,
-	0x21, 0x49, 0x2e, 0x8e, 0x92, 0xca, 0x82, 0xd4, 0xf8, 0xd2, 0xa2, 0x1c, 0x09, 0x46, 0x05, 0x46,
-	0x0d, 0xce, 0x20, 0x76, 0x10, 0x3f, 0xb4, 0x28, 0x47, 0x48, 0x84, 0x8b, 0xb5, 0x2c, 0x31, 0xa7,
-	0x34, 0x55, 0x82, 0x49, 0x81, 0x51, 0x83, 0x27, 0x08, 0xc2, 0x71, 0xca, 0xe7, 0x12, 0x4e, 0xce,
-	0xcf, 0xd5, 0x43, 0x33, 0xce, 0x89, 0xc3, 0x31, 0xaf, 0x32, 0x00, 0xc4, 0x09, 0x60, 0x8c, 0x52,
-	0x4d, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf, 0xd5, 0x4f, 0xcf, 0xcf, 0x49, 0xcc,
-	0x4b, 0x47, 0xb8, 0xa8, 0x00, 0x64, 0x7a, 0x31, 0xc8, 0x61, 0x8b, 0x98, 0x98, 0xdd, 0x03, 0x9c,
-	0x56, 0x31, 0xc9, 0xb9, 0x43, 0x8c, 0x0a, 0x80, 0x2a, 0xd1, 0x0b, 0x4f, 0xcd, 0xc9, 0xf1, 0xce,
-	0xcb, 0x2f, 0xcf, 0x0b, 0x01, 0x29, 0x4d, 0x62, 0x03, 0xeb, 0x35, 0x06, 0x04, 0x00, 0x00, 0xff,
-	0xff, 0x13, 0xf8, 0xe8, 0x42, 0xdd, 0x00, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_github_com_golang_protobuf_ptypes_any_any_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_github_com_golang_protobuf_ptypes_any_any_proto_goTypes,
+		DependencyIndexes: file_github_com_golang_protobuf_ptypes_any_any_proto_depIdxs,
+	}.Build()
+	File_github_com_golang_protobuf_ptypes_any_any_proto = out.File
+	file_github_com_golang_protobuf_ptypes_any_any_proto_rawDesc = nil
+	file_github_com_golang_protobuf_ptypes_any_any_proto_goTypes = nil
+	file_github_com_golang_protobuf_ptypes_any_any_proto_depIdxs = nil
 }
diff --git a/vendor/github.com/golang/protobuf/ptypes/any/any.proto b/vendor/github.com/golang/protobuf/ptypes/any/any.proto
deleted file mode 100644
index 4932942..0000000
--- a/vendor/github.com/golang/protobuf/ptypes/any/any.proto
+++ /dev/null
@@ -1,154 +0,0 @@
-// 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 go_package = "github.com/golang/protobuf/ptypes/any";
-option java_package = "com.google.protobuf";
-option java_outer_classname = "AnyProto";
-option java_multiple_files = true;
-option objc_class_prefix = "GPB";
-
-// `Any` contains an arbitrary serialized protocol buffer message along with a
-// URL that describes the type of the serialized message.
-//
-// Protobuf library provides support to pack/unpack Any values in the form
-// of utility functions or additional generated methods of the Any type.
-//
-// Example 1: Pack and unpack a message in C++.
-//
-//     Foo foo = ...;
-//     Any any;
-//     any.PackFrom(foo);
-//     ...
-//     if (any.UnpackTo(&foo)) {
-//       ...
-//     }
-//
-// Example 2: Pack and unpack a message in Java.
-//
-//     Foo foo = ...;
-//     Any any = Any.pack(foo);
-//     ...
-//     if (any.is(Foo.class)) {
-//       foo = any.unpack(Foo.class);
-//     }
-//
-//  Example 3: Pack and unpack a message in Python.
-//
-//     foo = Foo(...)
-//     any = Any()
-//     any.Pack(foo)
-//     ...
-//     if any.Is(Foo.DESCRIPTOR):
-//       any.Unpack(foo)
-//       ...
-//
-//  Example 4: Pack and unpack a message in Go
-//
-//      foo := &pb.Foo{...}
-//      any, err := ptypes.MarshalAny(foo)
-//      ...
-//      foo := &pb.Foo{}
-//      if err := ptypes.UnmarshalAny(any, foo); err != nil {
-//        ...
-//      }
-//
-// The pack methods provided by protobuf library will by default use
-// 'type.googleapis.com/full.type.name' as the type URL and the unpack
-// methods only use the fully qualified type name after the last '/'
-// in the type URL, for example "foo.bar.com/x/y.z" will yield type
-// name "y.z".
-//
-//
-// JSON
-// ====
-// The JSON representation of an `Any` value uses the regular
-// representation of the deserialized, embedded message, with an
-// additional field `@type` which contains the type URL. Example:
-//
-//     package google.profile;
-//     message Person {
-//       string first_name = 1;
-//       string last_name = 2;
-//     }
-//
-//     {
-//       "@type": "type.googleapis.com/google.profile.Person",
-//       "firstName": <string>,
-//       "lastName": <string>
-//     }
-//
-// If the embedded message type is well-known and has a custom JSON
-// representation, that representation will be embedded adding a field
-// `value` which holds the custom JSON in addition to the `@type`
-// field. Example (for message [google.protobuf.Duration][]):
-//
-//     {
-//       "@type": "type.googleapis.com/google.protobuf.Duration",
-//       "value": "1.212s"
-//     }
-//
-message Any {
-  // A URL/resource name that uniquely identifies the type of the serialized
-  // protocol buffer message. The last segment of the URL's path must represent
-  // the fully qualified name of the type (as in
-  // `path/google.protobuf.Duration`). The name should be in a canonical form
-  // (e.g., leading "." is not accepted).
-  //
-  // In practice, teams usually precompile into the binary all types that they
-  // expect it to use in the context of Any. However, for URLs which use the
-  // scheme `http`, `https`, or no scheme, one can optionally set up a type
-  // server that maps type URLs to message definitions as follows:
-  //
-  // * If no scheme is provided, `https` is assumed.
-  // * An HTTP GET on the URL must yield a [google.protobuf.Type][]
-  //   value in binary format, or produce an error.
-  // * Applications are allowed to cache lookup results based on the
-  //   URL, or have them precompiled into a binary to avoid any
-  //   lookup. Therefore, binary compatibility needs to be preserved
-  //   on changes to types. (Use versioned type names to manage
-  //   breaking changes.)
-  //
-  // Note: this functionality is not currently available in the official
-  // protobuf release, and it is not used for type URLs beginning with
-  // type.googleapis.com.
-  //
-  // Schemes other than `http`, `https` (or the empty scheme) might be
-  // used with implementation specific semantics.
-  //
-  string type_url = 1;
-
-  // Must be a valid serialized protocol buffer of the above specified type.
-  bytes value = 2;
-}
diff --git a/vendor/github.com/golang/protobuf/ptypes/doc.go b/vendor/github.com/golang/protobuf/ptypes/doc.go
index c0d595d..d3c3325 100644
--- a/vendor/github.com/golang/protobuf/ptypes/doc.go
+++ b/vendor/github.com/golang/protobuf/ptypes/doc.go
@@ -1,35 +1,10 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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.
+// 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 ptypes contains code for interacting with well-known types.
-*/
+// Package ptypes provides functionality for interacting with well-known types.
+//
+// Deprecated: Well-known types have specialized functionality directly
+// injected into the generated packages for each message type.
+// See the deprecation notice for each function for the suggested alternative.
 package ptypes
diff --git a/vendor/github.com/golang/protobuf/ptypes/duration.go b/vendor/github.com/golang/protobuf/ptypes/duration.go
index 26d1ca2..b2b55dd 100644
--- a/vendor/github.com/golang/protobuf/ptypes/duration.go
+++ b/vendor/github.com/golang/protobuf/ptypes/duration.go
@@ -1,102 +1,76 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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.
+// 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 ptypes
 
-// This file implements conversions between google.protobuf.Duration
-// and time.Duration.
-
 import (
 	"errors"
 	"fmt"
 	"time"
 
-	durpb "github.com/golang/protobuf/ptypes/duration"
+	durationpb "github.com/golang/protobuf/ptypes/duration"
 )
 
+// Range of google.protobuf.Duration as specified in duration.proto.
+// This is about 10,000 years in seconds.
 const (
-	// Range of a durpb.Duration in seconds, as specified in
-	// google/protobuf/duration.proto. This is about 10,000 years in seconds.
 	maxSeconds = int64(10000 * 365.25 * 24 * 60 * 60)
 	minSeconds = -maxSeconds
 )
 
-// validateDuration determines whether the durpb.Duration is valid according to the
-// definition in google/protobuf/duration.proto. A valid durpb.Duration
-// may still be too large to fit into a time.Duration (the range of durpb.Duration
-// is about 10,000 years, and the range of time.Duration is about 290).
-func validateDuration(d *durpb.Duration) error {
-	if d == nil {
-		return errors.New("duration: nil Duration")
-	}
-	if d.Seconds < minSeconds || d.Seconds > maxSeconds {
-		return fmt.Errorf("duration: %v: seconds out of range", d)
-	}
-	if d.Nanos <= -1e9 || d.Nanos >= 1e9 {
-		return fmt.Errorf("duration: %v: nanos out of range", d)
-	}
-	// Seconds and Nanos must have the same sign, unless d.Nanos is zero.
-	if (d.Seconds < 0 && d.Nanos > 0) || (d.Seconds > 0 && d.Nanos < 0) {
-		return fmt.Errorf("duration: %v: seconds and nanos have different signs", d)
-	}
-	return nil
-}
-
-// Duration converts a durpb.Duration to a time.Duration. Duration
-// returns an error if the durpb.Duration is invalid or is too large to be
-// represented in a time.Duration.
-func Duration(p *durpb.Duration) (time.Duration, error) {
-	if err := validateDuration(p); err != nil {
+// Duration converts a durationpb.Duration to a time.Duration.
+// Duration returns an error if dur is invalid or overflows a time.Duration.
+//
+// Deprecated: Call the dur.AsDuration and dur.CheckValid methods instead.
+func Duration(dur *durationpb.Duration) (time.Duration, error) {
+	if err := validateDuration(dur); err != nil {
 		return 0, err
 	}
-	d := time.Duration(p.Seconds) * time.Second
-	if int64(d/time.Second) != p.Seconds {
-		return 0, fmt.Errorf("duration: %v is out of range for time.Duration", p)
+	d := time.Duration(dur.Seconds) * time.Second
+	if int64(d/time.Second) != dur.Seconds {
+		return 0, fmt.Errorf("duration: %v is out of range for time.Duration", dur)
 	}
-	if p.Nanos != 0 {
-		d += time.Duration(p.Nanos) * time.Nanosecond
-		if (d < 0) != (p.Nanos < 0) {
-			return 0, fmt.Errorf("duration: %v is out of range for time.Duration", p)
+	if dur.Nanos != 0 {
+		d += time.Duration(dur.Nanos) * time.Nanosecond
+		if (d < 0) != (dur.Nanos < 0) {
+			return 0, fmt.Errorf("duration: %v is out of range for time.Duration", dur)
 		}
 	}
 	return d, nil
 }
 
-// DurationProto converts a time.Duration to a durpb.Duration.
-func DurationProto(d time.Duration) *durpb.Duration {
+// DurationProto converts a time.Duration to a durationpb.Duration.
+//
+// Deprecated: Call the durationpb.New function instead.
+func DurationProto(d time.Duration) *durationpb.Duration {
 	nanos := d.Nanoseconds()
 	secs := nanos / 1e9
 	nanos -= secs * 1e9
-	return &durpb.Duration{
-		Seconds: secs,
+	return &durationpb.Duration{
+		Seconds: int64(secs),
 		Nanos:   int32(nanos),
 	}
 }
+
+// validateDuration determines whether the durationpb.Duration is valid
+// according to the definition in google/protobuf/duration.proto.
+// A valid durpb.Duration may still be too large to fit into a time.Duration
+// Note that the range of durationpb.Duration is about 10,000 years,
+// while the range of time.Duration is about 290 years.
+func validateDuration(dur *durationpb.Duration) error {
+	if dur == nil {
+		return errors.New("duration: nil Duration")
+	}
+	if dur.Seconds < minSeconds || dur.Seconds > maxSeconds {
+		return fmt.Errorf("duration: %v: seconds out of range", dur)
+	}
+	if dur.Nanos <= -1e9 || dur.Nanos >= 1e9 {
+		return fmt.Errorf("duration: %v: nanos out of range", dur)
+	}
+	// Seconds and Nanos must have the same sign, unless d.Nanos is zero.
+	if (dur.Seconds < 0 && dur.Nanos > 0) || (dur.Seconds > 0 && dur.Nanos < 0) {
+		return fmt.Errorf("duration: %v: seconds and nanos have different signs", dur)
+	}
+	return nil
+}
diff --git a/vendor/github.com/golang/protobuf/ptypes/duration/duration.pb.go b/vendor/github.com/golang/protobuf/ptypes/duration/duration.pb.go
index 0d681ee..d0079ee 100644
--- a/vendor/github.com/golang/protobuf/ptypes/duration/duration.pb.go
+++ b/vendor/github.com/golang/protobuf/ptypes/duration/duration.pb.go
@@ -1,161 +1,63 @@
 // Code generated by protoc-gen-go. DO NOT EDIT.
-// source: google/protobuf/duration.proto
+// source: github.com/golang/protobuf/ptypes/duration/duration.proto
 
 package duration
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	durationpb "google.golang.org/protobuf/types/known/durationpb"
+	reflect "reflect"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+// Symbols defined in public import of google/protobuf/duration.proto.
 
-// 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 Duration = durationpb.Duration
 
-// A Duration represents a signed, fixed-length span of time represented
-// as a count of seconds and fractions of seconds at nanosecond
-// resolution. It is independent of any calendar and concepts like "day"
-// or "month". It is related to Timestamp in that the difference between
-// two Timestamp values is a Duration and it can be added or subtracted
-// from a Timestamp. Range is approximately +-10,000 years.
-//
-// # Examples
-//
-// Example 1: Compute Duration from two Timestamps in pseudo code.
-//
-//     Timestamp start = ...;
-//     Timestamp end = ...;
-//     Duration duration = ...;
-//
-//     duration.seconds = end.seconds - start.seconds;
-//     duration.nanos = end.nanos - start.nanos;
-//
-//     if (duration.seconds < 0 && duration.nanos > 0) {
-//       duration.seconds += 1;
-//       duration.nanos -= 1000000000;
-//     } else if (durations.seconds > 0 && duration.nanos < 0) {
-//       duration.seconds -= 1;
-//       duration.nanos += 1000000000;
-//     }
-//
-// Example 2: Compute Timestamp from Timestamp + Duration in pseudo code.
-//
-//     Timestamp start = ...;
-//     Duration duration = ...;
-//     Timestamp end = ...;
-//
-//     end.seconds = start.seconds + duration.seconds;
-//     end.nanos = start.nanos + duration.nanos;
-//
-//     if (end.nanos < 0) {
-//       end.seconds -= 1;
-//       end.nanos += 1000000000;
-//     } else if (end.nanos >= 1000000000) {
-//       end.seconds += 1;
-//       end.nanos -= 1000000000;
-//     }
-//
-// Example 3: Compute Duration from datetime.timedelta in Python.
-//
-//     td = datetime.timedelta(days=3, minutes=10)
-//     duration = Duration()
-//     duration.FromTimedelta(td)
-//
-// # JSON Mapping
-//
-// In JSON format, the Duration type is encoded as a string rather than an
-// object, where the string ends in the suffix "s" (indicating seconds) and
-// is preceded by the number of seconds, with nanoseconds expressed as
-// fractional seconds. For example, 3 seconds with 0 nanoseconds should be
-// encoded in JSON format as "3s", while 3 seconds and 1 nanosecond should
-// be expressed in JSON format as "3.000000001s", and 3 seconds and 1
-// microsecond should be expressed in JSON format as "3.000001s".
-//
-//
-type Duration struct {
-	// Signed seconds of the span of time. Must be from -315,576,000,000
-	// to +315,576,000,000 inclusive. Note: these bounds are computed from:
-	// 60 sec/min * 60 min/hr * 24 hr/day * 365.25 days/year * 10000 years
-	Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"`
-	// Signed fractions of a second at nanosecond resolution of the span
-	// of time. Durations less than one second are represented with a 0
-	// `seconds` field and a positive or negative `nanos` field. For durations
-	// of one second or more, a non-zero value for the `nanos` field must be
-	// of the same sign as the `seconds` field. Must be from -999,999,999
-	// to +999,999,999 inclusive.
-	Nanos                int32    `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+var File_github_com_golang_protobuf_ptypes_duration_duration_proto protoreflect.FileDescriptor
+
+var file_github_com_golang_protobuf_ptypes_duration_duration_proto_rawDesc = []byte{
+	0x0a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79,
+	0x70, 0x65, 0x73, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x64, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x42, 0x35, 0x5a, 0x33, 0x67,
+	0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67,
+	0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79, 0x70, 0x65, 0x73,
+	0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3b, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69,
+	0x6f, 0x6e, 0x50, 0x00, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func (m *Duration) Reset()         { *m = Duration{} }
-func (m *Duration) String() string { return proto.CompactTextString(m) }
-func (*Duration) ProtoMessage()    {}
-func (*Duration) Descriptor() ([]byte, []int) {
-	return fileDescriptor_23597b2ebd7ac6c5, []int{0}
+var file_github_com_golang_protobuf_ptypes_duration_duration_proto_goTypes = []interface{}{}
+var file_github_com_golang_protobuf_ptypes_duration_duration_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func (*Duration) XXX_WellKnownType() string { return "Duration" }
-
-func (m *Duration) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Duration.Unmarshal(m, b)
-}
-func (m *Duration) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Duration.Marshal(b, m, deterministic)
-}
-func (m *Duration) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Duration.Merge(m, src)
-}
-func (m *Duration) XXX_Size() int {
-	return xxx_messageInfo_Duration.Size(m)
-}
-func (m *Duration) XXX_DiscardUnknown() {
-	xxx_messageInfo_Duration.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Duration proto.InternalMessageInfo
-
-func (m *Duration) GetSeconds() int64 {
-	if m != nil {
-		return m.Seconds
+func init() { file_github_com_golang_protobuf_ptypes_duration_duration_proto_init() }
+func file_github_com_golang_protobuf_ptypes_duration_duration_proto_init() {
+	if File_github_com_golang_protobuf_ptypes_duration_duration_proto != nil {
+		return
 	}
-	return 0
-}
-
-func (m *Duration) GetNanos() int32 {
-	if m != nil {
-		return m.Nanos
-	}
-	return 0
-}
-
-func init() {
-	proto.RegisterType((*Duration)(nil), "google.protobuf.Duration")
-}
-
-func init() { proto.RegisterFile("google/protobuf/duration.proto", fileDescriptor_23597b2ebd7ac6c5) }
-
-var fileDescriptor_23597b2ebd7ac6c5 = []byte{
-	// 190 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, 0x4f, 0x29, 0x2d, 0x4a,
-	0x2c, 0xc9, 0xcc, 0xcf, 0xd3, 0x03, 0x8b, 0x08, 0xf1, 0x43, 0xe4, 0xf5, 0x60, 0xf2, 0x4a, 0x56,
-	0x5c, 0x1c, 0x2e, 0x50, 0x25, 0x42, 0x12, 0x5c, 0xec, 0xc5, 0xa9, 0xc9, 0xf9, 0x79, 0x29, 0xc5,
-	0x12, 0x8c, 0x0a, 0x8c, 0x1a, 0xcc, 0x41, 0x30, 0xae, 0x90, 0x08, 0x17, 0x6b, 0x5e, 0x62, 0x5e,
-	0x7e, 0xb1, 0x04, 0x93, 0x02, 0xa3, 0x06, 0x6b, 0x10, 0x84, 0xe3, 0x54, 0xc3, 0x25, 0x9c, 0x9c,
-	0x9f, 0xab, 0x87, 0x66, 0xa4, 0x13, 0x2f, 0xcc, 0xc0, 0x00, 0x90, 0x48, 0x00, 0x63, 0x94, 0x56,
-	0x7a, 0x66, 0x49, 0x46, 0x69, 0x92, 0x5e, 0x72, 0x7e, 0xae, 0x7e, 0x7a, 0x7e, 0x4e, 0x62, 0x5e,
-	0x3a, 0xc2, 0x7d, 0x05, 0x25, 0x95, 0x05, 0xa9, 0xc5, 0x70, 0x67, 0xfe, 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, 0xdc, 0x84, 0x30, 0xff, 0xf3, 0x00, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_github_com_golang_protobuf_ptypes_duration_duration_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_github_com_golang_protobuf_ptypes_duration_duration_proto_goTypes,
+		DependencyIndexes: file_github_com_golang_protobuf_ptypes_duration_duration_proto_depIdxs,
+	}.Build()
+	File_github_com_golang_protobuf_ptypes_duration_duration_proto = out.File
+	file_github_com_golang_protobuf_ptypes_duration_duration_proto_rawDesc = nil
+	file_github_com_golang_protobuf_ptypes_duration_duration_proto_goTypes = nil
+	file_github_com_golang_protobuf_ptypes_duration_duration_proto_depIdxs = nil
 }
diff --git a/vendor/github.com/golang/protobuf/ptypes/duration/duration.proto b/vendor/github.com/golang/protobuf/ptypes/duration/duration.proto
deleted file mode 100644
index 975fce4..0000000
--- a/vendor/github.com/golang/protobuf/ptypes/duration/duration.proto
+++ /dev/null
@@ -1,117 +0,0 @@
-// 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/duration";
-option java_package = "com.google.protobuf";
-option java_outer_classname = "DurationProto";
-option java_multiple_files = true;
-option objc_class_prefix = "GPB";
-
-// A Duration represents a signed, fixed-length span of time represented
-// as a count of seconds and fractions of seconds at nanosecond
-// resolution. It is independent of any calendar and concepts like "day"
-// or "month". It is related to Timestamp in that the difference between
-// two Timestamp values is a Duration and it can be added or subtracted
-// from a Timestamp. Range is approximately +-10,000 years.
-//
-// # Examples
-//
-// Example 1: Compute Duration from two Timestamps in pseudo code.
-//
-//     Timestamp start = ...;
-//     Timestamp end = ...;
-//     Duration duration = ...;
-//
-//     duration.seconds = end.seconds - start.seconds;
-//     duration.nanos = end.nanos - start.nanos;
-//
-//     if (duration.seconds < 0 && duration.nanos > 0) {
-//       duration.seconds += 1;
-//       duration.nanos -= 1000000000;
-//     } else if (durations.seconds > 0 && duration.nanos < 0) {
-//       duration.seconds -= 1;
-//       duration.nanos += 1000000000;
-//     }
-//
-// Example 2: Compute Timestamp from Timestamp + Duration in pseudo code.
-//
-//     Timestamp start = ...;
-//     Duration duration = ...;
-//     Timestamp end = ...;
-//
-//     end.seconds = start.seconds + duration.seconds;
-//     end.nanos = start.nanos + duration.nanos;
-//
-//     if (end.nanos < 0) {
-//       end.seconds -= 1;
-//       end.nanos += 1000000000;
-//     } else if (end.nanos >= 1000000000) {
-//       end.seconds += 1;
-//       end.nanos -= 1000000000;
-//     }
-//
-// Example 3: Compute Duration from datetime.timedelta in Python.
-//
-//     td = datetime.timedelta(days=3, minutes=10)
-//     duration = Duration()
-//     duration.FromTimedelta(td)
-//
-// # JSON Mapping
-//
-// In JSON format, the Duration type is encoded as a string rather than an
-// object, where the string ends in the suffix "s" (indicating seconds) and
-// is preceded by the number of seconds, with nanoseconds expressed as
-// fractional seconds. For example, 3 seconds with 0 nanoseconds should be
-// encoded in JSON format as "3s", while 3 seconds and 1 nanosecond should
-// be expressed in JSON format as "3.000000001s", and 3 seconds and 1
-// microsecond should be expressed in JSON format as "3.000001s".
-//
-//
-message Duration {
-
-  // Signed seconds of the span of time. Must be from -315,576,000,000
-  // to +315,576,000,000 inclusive. Note: these bounds are computed from:
-  // 60 sec/min * 60 min/hr * 24 hr/day * 365.25 days/year * 10000 years
-  int64 seconds = 1;
-
-  // Signed fractions of a second at nanosecond resolution of the span
-  // of time. Durations less than one second are represented with a 0
-  // `seconds` field and a positive or negative `nanos` field. For durations
-  // of one second or more, a non-zero value for the `nanos` field must be
-  // of the same sign as the `seconds` field. Must be from -999,999,999
-  // to +999,999,999 inclusive.
-  int32 nanos = 2;
-}
diff --git a/vendor/github.com/golang/protobuf/ptypes/empty/empty.pb.go b/vendor/github.com/golang/protobuf/ptypes/empty/empty.pb.go
index b4eb03e..16686a6 100644
--- a/vendor/github.com/golang/protobuf/ptypes/empty/empty.pb.go
+++ b/vendor/github.com/golang/protobuf/ptypes/empty/empty.pb.go
@@ -1,83 +1,62 @@
 // Code generated by protoc-gen-go. DO NOT EDIT.
-// source: google/protobuf/empty.proto
+// source: github.com/golang/protobuf/ptypes/empty/empty.proto
 
 package empty
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	emptypb "google.golang.org/protobuf/types/known/emptypb"
+	reflect "reflect"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+// Symbols defined in public import of google/protobuf/empty.proto.
 
-// 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 Empty = emptypb.Empty
 
-// A generic empty message that you can re-use to avoid defining duplicated
-// empty messages in your APIs. A typical example is to use it as the request
-// or the response type of an API method. For instance:
-//
-//     service Foo {
-//       rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty);
-//     }
-//
-// The JSON representation for `Empty` is empty JSON object `{}`.
-type Empty struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+var File_github_com_golang_protobuf_ptypes_empty_empty_proto protoreflect.FileDescriptor
+
+var file_github_com_golang_protobuf_ptypes_empty_empty_proto_rawDesc = []byte{
+	0x0a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79,
+	0x70, 0x65, 0x73, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x42, 0x2f, 0x5a, 0x2d, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
+	0x2f, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
+	0x2f, 0x70, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x3b, 0x65, 0x6d,
+	0x70, 0x74, 0x79, 0x50, 0x00, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func (m *Empty) Reset()         { *m = Empty{} }
-func (m *Empty) String() string { return proto.CompactTextString(m) }
-func (*Empty) ProtoMessage()    {}
-func (*Empty) Descriptor() ([]byte, []int) {
-	return fileDescriptor_900544acb223d5b8, []int{0}
+var file_github_com_golang_protobuf_ptypes_empty_empty_proto_goTypes = []interface{}{}
+var file_github_com_golang_protobuf_ptypes_empty_empty_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func (*Empty) XXX_WellKnownType() string { return "Empty" }
-
-func (m *Empty) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Empty.Unmarshal(m, b)
-}
-func (m *Empty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Empty.Marshal(b, m, deterministic)
-}
-func (m *Empty) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Empty.Merge(m, src)
-}
-func (m *Empty) XXX_Size() int {
-	return xxx_messageInfo_Empty.Size(m)
-}
-func (m *Empty) XXX_DiscardUnknown() {
-	xxx_messageInfo_Empty.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Empty proto.InternalMessageInfo
-
-func init() {
-	proto.RegisterType((*Empty)(nil), "google.protobuf.Empty")
-}
-
-func init() { proto.RegisterFile("google/protobuf/empty.proto", fileDescriptor_900544acb223d5b8) }
-
-var fileDescriptor_900544acb223d5b8 = []byte{
-	// 148 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4e, 0xcf, 0xcf, 0x4f,
-	0xcf, 0x49, 0xd5, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0x2a, 0x4d, 0xd3, 0x4f, 0xcd, 0x2d, 0x28,
-	0xa9, 0xd4, 0x03, 0x73, 0x85, 0xf8, 0x21, 0x92, 0x7a, 0x30, 0x49, 0x25, 0x76, 0x2e, 0x56, 0x57,
-	0x90, 0xbc, 0x53, 0x19, 0x97, 0x70, 0x72, 0x7e, 0xae, 0x1e, 0x9a, 0xbc, 0x13, 0x17, 0x58, 0x36,
-	0x00, 0xc4, 0x0d, 0x60, 0x8c, 0x52, 0x4f, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b, 0xce, 0xcf,
-	0xd5, 0x4f, 0xcf, 0xcf, 0x49, 0xcc, 0x4b, 0x47, 0x58, 0x53, 0x50, 0x52, 0x59, 0x90, 0x5a, 0x0c,
-	0xb1, 0xed, 0x07, 0x23, 0xe3, 0x22, 0x26, 0x66, 0xf7, 0x00, 0xa7, 0x55, 0x4c, 0x72, 0xee, 0x10,
-	0x13, 0x03, 0xa0, 0xea, 0xf4, 0xc2, 0x53, 0x73, 0x72, 0xbc, 0xf3, 0xf2, 0xcb, 0xf3, 0x42, 0x40,
-	0xea, 0x93, 0xd8, 0xc0, 0x06, 0x18, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x64, 0xd4, 0xb3, 0xa6,
-	0xb7, 0x00, 0x00, 0x00,
+func init() { file_github_com_golang_protobuf_ptypes_empty_empty_proto_init() }
+func file_github_com_golang_protobuf_ptypes_empty_empty_proto_init() {
+	if File_github_com_golang_protobuf_ptypes_empty_empty_proto != nil {
+		return
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_github_com_golang_protobuf_ptypes_empty_empty_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_github_com_golang_protobuf_ptypes_empty_empty_proto_goTypes,
+		DependencyIndexes: file_github_com_golang_protobuf_ptypes_empty_empty_proto_depIdxs,
+	}.Build()
+	File_github_com_golang_protobuf_ptypes_empty_empty_proto = out.File
+	file_github_com_golang_protobuf_ptypes_empty_empty_proto_rawDesc = nil
+	file_github_com_golang_protobuf_ptypes_empty_empty_proto_goTypes = nil
+	file_github_com_golang_protobuf_ptypes_empty_empty_proto_depIdxs = nil
 }
diff --git a/vendor/github.com/golang/protobuf/ptypes/empty/empty.proto b/vendor/github.com/golang/protobuf/ptypes/empty/empty.proto
deleted file mode 100644
index 03cacd2..0000000
--- a/vendor/github.com/golang/protobuf/ptypes/empty/empty.proto
+++ /dev/null
@@ -1,52 +0,0 @@
-// 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 go_package = "github.com/golang/protobuf/ptypes/empty";
-option java_package = "com.google.protobuf";
-option java_outer_classname = "EmptyProto";
-option java_multiple_files = true;
-option objc_class_prefix = "GPB";
-option cc_enable_arenas = true;
-
-// A generic empty message that you can re-use to avoid defining duplicated
-// empty messages in your APIs. A typical example is to use it as the request
-// or the response type of an API method. For instance:
-//
-//     service Foo {
-//       rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty);
-//     }
-//
-// The JSON representation for `Empty` is empty JSON object `{}`.
-message Empty {}
diff --git a/vendor/github.com/golang/protobuf/ptypes/timestamp.go b/vendor/github.com/golang/protobuf/ptypes/timestamp.go
index 8da0df0..8368a3f 100644
--- a/vendor/github.com/golang/protobuf/ptypes/timestamp.go
+++ b/vendor/github.com/golang/protobuf/ptypes/timestamp.go
@@ -1,46 +1,18 @@
-// Go support for Protocol Buffers - Google's data interchange format
-//
-// Copyright 2016 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.
+// 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 ptypes
 
-// This file implements operations on google.protobuf.Timestamp.
-
 import (
 	"errors"
 	"fmt"
 	"time"
 
-	tspb "github.com/golang/protobuf/ptypes/timestamp"
+	timestamppb "github.com/golang/protobuf/ptypes/timestamp"
 )
 
+// Range of google.protobuf.Duration as specified in timestamp.proto.
 const (
 	// Seconds field of the earliest valid Timestamp.
 	// This is time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC).Unix().
@@ -50,17 +22,80 @@
 	maxValidSeconds = 253402300800
 )
 
+// Timestamp converts a timestamppb.Timestamp to a time.Time.
+// It returns an error if the argument is invalid.
+//
+// Unlike most Go functions, if Timestamp returns an error, the first return
+// value is not the zero time.Time. Instead, it is the value obtained from the
+// time.Unix function when passed the contents of the Timestamp, in the UTC
+// locale. This may or may not be a meaningful time; many invalid Timestamps
+// do map to valid time.Times.
+//
+// A nil Timestamp returns an error. The first return value in that case is
+// undefined.
+//
+// Deprecated: Call the ts.AsTime and ts.CheckValid methods instead.
+func Timestamp(ts *timestamppb.Timestamp) (time.Time, error) {
+	// Don't return the zero value on error, because corresponds to a valid
+	// timestamp. Instead return whatever time.Unix gives us.
+	var t time.Time
+	if ts == nil {
+		t = time.Unix(0, 0).UTC() // treat nil like the empty Timestamp
+	} else {
+		t = time.Unix(ts.Seconds, int64(ts.Nanos)).UTC()
+	}
+	return t, validateTimestamp(ts)
+}
+
+// TimestampNow returns a google.protobuf.Timestamp for the current time.
+//
+// Deprecated: Call the timestamppb.Now function instead.
+func TimestampNow() *timestamppb.Timestamp {
+	ts, err := TimestampProto(time.Now())
+	if err != nil {
+		panic("ptypes: time.Now() out of Timestamp range")
+	}
+	return ts
+}
+
+// TimestampProto converts the time.Time to a google.protobuf.Timestamp proto.
+// It returns an error if the resulting Timestamp is invalid.
+//
+// Deprecated: Call the timestamppb.New function instead.
+func TimestampProto(t time.Time) (*timestamppb.Timestamp, error) {
+	ts := &timestamppb.Timestamp{
+		Seconds: t.Unix(),
+		Nanos:   int32(t.Nanosecond()),
+	}
+	if err := validateTimestamp(ts); err != nil {
+		return nil, err
+	}
+	return ts, nil
+}
+
+// TimestampString returns the RFC 3339 string for valid Timestamps.
+// For invalid Timestamps, it returns an error message in parentheses.
+//
+// Deprecated: Call the ts.AsTime method instead,
+// followed by a call to the Format method on the time.Time value.
+func TimestampString(ts *timestamppb.Timestamp) string {
+	t, err := Timestamp(ts)
+	if err != nil {
+		return fmt.Sprintf("(%v)", err)
+	}
+	return t.Format(time.RFC3339Nano)
+}
+
 // validateTimestamp determines whether a Timestamp is valid.
-// A valid timestamp represents a time in the range
-// [0001-01-01, 10000-01-01) and has a Nanos field
-// in the range [0, 1e9).
+// A valid timestamp represents a time in the range [0001-01-01, 10000-01-01)
+// and has a Nanos field in the range [0, 1e9).
 //
 // If the Timestamp is valid, validateTimestamp returns nil.
-// Otherwise, it returns an error that describes
-// the problem.
+// Otherwise, it returns an error that describes the problem.
 //
-// Every valid Timestamp can be represented by a time.Time, but the converse is not true.
-func validateTimestamp(ts *tspb.Timestamp) error {
+// Every valid Timestamp can be represented by a time.Time,
+// but the converse is not true.
+func validateTimestamp(ts *timestamppb.Timestamp) error {
 	if ts == nil {
 		return errors.New("timestamp: nil Timestamp")
 	}
@@ -75,58 +110,3 @@
 	}
 	return nil
 }
-
-// Timestamp converts a google.protobuf.Timestamp proto to a time.Time.
-// It returns an error if the argument is invalid.
-//
-// Unlike most Go functions, if Timestamp returns an error, the first return value
-// is not the zero time.Time. Instead, it is the value obtained from the
-// time.Unix function when passed the contents of the Timestamp, in the UTC
-// locale. This may or may not be a meaningful time; many invalid Timestamps
-// do map to valid time.Times.
-//
-// A nil Timestamp returns an error. The first return value in that case is
-// undefined.
-func Timestamp(ts *tspb.Timestamp) (time.Time, error) {
-	// Don't return the zero value on error, because corresponds to a valid
-	// timestamp. Instead return whatever time.Unix gives us.
-	var t time.Time
-	if ts == nil {
-		t = time.Unix(0, 0).UTC() // treat nil like the empty Timestamp
-	} else {
-		t = time.Unix(ts.Seconds, int64(ts.Nanos)).UTC()
-	}
-	return t, validateTimestamp(ts)
-}
-
-// TimestampNow returns a google.protobuf.Timestamp for the current time.
-func TimestampNow() *tspb.Timestamp {
-	ts, err := TimestampProto(time.Now())
-	if err != nil {
-		panic("ptypes: time.Now() out of Timestamp range")
-	}
-	return ts
-}
-
-// TimestampProto converts the time.Time to a google.protobuf.Timestamp proto.
-// It returns an error if the resulting Timestamp is invalid.
-func TimestampProto(t time.Time) (*tspb.Timestamp, error) {
-	ts := &tspb.Timestamp{
-		Seconds: t.Unix(),
-		Nanos:   int32(t.Nanosecond()),
-	}
-	if err := validateTimestamp(ts); err != nil {
-		return nil, err
-	}
-	return ts, nil
-}
-
-// TimestampString returns the RFC 3339 string for valid Timestamps. For invalid
-// Timestamps, it returns an error message in parentheses.
-func TimestampString(ts *tspb.Timestamp) string {
-	t, err := Timestamp(ts)
-	if err != nil {
-		return fmt.Sprintf("(%v)", err)
-	}
-	return t.Format(time.RFC3339Nano)
-}
diff --git a/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.pb.go b/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.pb.go
index 31cd846..a76f807 100644
--- a/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.pb.go
+++ b/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.pb.go
@@ -1,179 +1,64 @@
 // Code generated by protoc-gen-go. DO NOT EDIT.
-// source: google/protobuf/timestamp.proto
+// source: github.com/golang/protobuf/ptypes/timestamp/timestamp.proto
 
 package timestamp
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	timestamppb "google.golang.org/protobuf/types/known/timestamppb"
+	reflect "reflect"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+// Symbols defined in public import of google/protobuf/timestamp.proto.
 
-// 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 Timestamp = timestamppb.Timestamp
 
-// A Timestamp represents a point in time independent of any time zone
-// or calendar, represented as seconds and fractions of seconds at
-// nanosecond resolution in UTC Epoch time. It is encoded using the
-// Proleptic Gregorian Calendar which extends the Gregorian calendar
-// backwards to year one. It is encoded assuming all minutes are 60
-// seconds long, i.e. leap seconds are "smeared" so that no leap second
-// table is needed for interpretation. Range is from
-// 0001-01-01T00:00:00Z to 9999-12-31T23:59:59.999999999Z.
-// By restricting to that range, we ensure that we can convert to
-// and from  RFC 3339 date strings.
-// See [https://www.ietf.org/rfc/rfc3339.txt](https://www.ietf.org/rfc/rfc3339.txt).
-//
-// # Examples
-//
-// Example 1: Compute Timestamp from POSIX `time()`.
-//
-//     Timestamp timestamp;
-//     timestamp.set_seconds(time(NULL));
-//     timestamp.set_nanos(0);
-//
-// Example 2: Compute Timestamp from POSIX `gettimeofday()`.
-//
-//     struct timeval tv;
-//     gettimeofday(&tv, NULL);
-//
-//     Timestamp timestamp;
-//     timestamp.set_seconds(tv.tv_sec);
-//     timestamp.set_nanos(tv.tv_usec * 1000);
-//
-// Example 3: Compute Timestamp from Win32 `GetSystemTimeAsFileTime()`.
-//
-//     FILETIME ft;
-//     GetSystemTimeAsFileTime(&ft);
-//     UINT64 ticks = (((UINT64)ft.dwHighDateTime) << 32) | ft.dwLowDateTime;
-//
-//     // A Windows tick is 100 nanoseconds. Windows epoch 1601-01-01T00:00:00Z
-//     // is 11644473600 seconds before Unix epoch 1970-01-01T00:00:00Z.
-//     Timestamp timestamp;
-//     timestamp.set_seconds((INT64) ((ticks / 10000000) - 11644473600LL));
-//     timestamp.set_nanos((INT32) ((ticks % 10000000) * 100));
-//
-// Example 4: Compute Timestamp from Java `System.currentTimeMillis()`.
-//
-//     long millis = System.currentTimeMillis();
-//
-//     Timestamp timestamp = Timestamp.newBuilder().setSeconds(millis / 1000)
-//         .setNanos((int) ((millis % 1000) * 1000000)).build();
-//
-//
-// Example 5: Compute Timestamp from current time in Python.
-//
-//     timestamp = Timestamp()
-//     timestamp.GetCurrentTime()
-//
-// # JSON Mapping
-//
-// In JSON format, the Timestamp type is encoded as a string in the
-// [RFC 3339](https://www.ietf.org/rfc/rfc3339.txt) format. That is, the
-// format is "{year}-{month}-{day}T{hour}:{min}:{sec}[.{frac_sec}]Z"
-// where {year} is always expressed using four digits while {month}, {day},
-// {hour}, {min}, and {sec} are zero-padded to two digits each. The fractional
-// seconds, which can go up to 9 digits (i.e. up to 1 nanosecond resolution),
-// are optional. The "Z" suffix indicates the timezone ("UTC"); the timezone
-// is required. A proto3 JSON serializer should always use UTC (as indicated by
-// "Z") when printing the Timestamp type and a proto3 JSON parser should be
-// able to accept both UTC and other timezones (as indicated by an offset).
-//
-// For example, "2017-01-15T01:30:15.01Z" encodes 15.01 seconds past
-// 01:30 UTC on January 15, 2017.
-//
-// In JavaScript, one can convert a Date object to this format using the
-// standard [toISOString()](https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Date/toISOString]
-// method. In Python, a standard `datetime.datetime` object can be converted
-// to this format using [`strftime`](https://docs.python.org/2/library/time.html#time.strftime)
-// with the time format spec '%Y-%m-%dT%H:%M:%S.%fZ'. Likewise, in Java, one
-// can use the Joda Time's [`ISODateTimeFormat.dateTime()`](
-// http://www.joda.org/joda-time/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime--
-// ) to obtain a formatter capable of generating timestamps in this format.
-//
-//
-type Timestamp struct {
-	// Represents seconds of UTC time since Unix epoch
-	// 1970-01-01T00:00:00Z. Must be from 0001-01-01T00:00:00Z to
-	// 9999-12-31T23:59:59Z inclusive.
-	Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"`
-	// Non-negative fractions of a second at nanosecond resolution. Negative
-	// second values with fractions must still have non-negative nanos values
-	// that count forward in time. Must be from 0 to 999,999,999
-	// inclusive.
-	Nanos                int32    `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+var File_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto protoreflect.FileDescriptor
+
+var file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_rawDesc = []byte{
+	0x0a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79,
+	0x70, 0x65, 0x73, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2f, 0x74, 0x69,
+	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74,
+	0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x42, 0x37,
+	0x5a, 0x35, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x6f, 0x6c,
+	0x61, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x70, 0x74, 0x79,
+	0x70, 0x65, 0x73, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x3b, 0x74, 0x69,
+	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x00, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x33,
 }
 
-func (m *Timestamp) Reset()         { *m = Timestamp{} }
-func (m *Timestamp) String() string { return proto.CompactTextString(m) }
-func (*Timestamp) ProtoMessage()    {}
-func (*Timestamp) Descriptor() ([]byte, []int) {
-	return fileDescriptor_292007bbfe81227e, []int{0}
+var file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_goTypes = []interface{}{}
+var file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func (*Timestamp) XXX_WellKnownType() string { return "Timestamp" }
-
-func (m *Timestamp) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Timestamp.Unmarshal(m, b)
-}
-func (m *Timestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Timestamp.Marshal(b, m, deterministic)
-}
-func (m *Timestamp) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Timestamp.Merge(m, src)
-}
-func (m *Timestamp) XXX_Size() int {
-	return xxx_messageInfo_Timestamp.Size(m)
-}
-func (m *Timestamp) XXX_DiscardUnknown() {
-	xxx_messageInfo_Timestamp.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Timestamp proto.InternalMessageInfo
-
-func (m *Timestamp) GetSeconds() int64 {
-	if m != nil {
-		return m.Seconds
+func init() { file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_init() }
+func file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_init() {
+	if File_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto != nil {
+		return
 	}
-	return 0
-}
-
-func (m *Timestamp) GetNanos() int32 {
-	if m != nil {
-		return m.Nanos
-	}
-	return 0
-}
-
-func init() {
-	proto.RegisterType((*Timestamp)(nil), "google.protobuf.Timestamp")
-}
-
-func init() { proto.RegisterFile("google/protobuf/timestamp.proto", fileDescriptor_292007bbfe81227e) }
-
-var fileDescriptor_292007bbfe81227e = []byte{
-	// 191 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4f, 0xcf, 0xcf, 0x4f,
-	0xcf, 0x49, 0xd5, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0x2a, 0x4d, 0xd3, 0x2f, 0xc9, 0xcc, 0x4d,
-	0x2d, 0x2e, 0x49, 0xcc, 0x2d, 0xd0, 0x03, 0x0b, 0x09, 0xf1, 0x43, 0x14, 0xe8, 0xc1, 0x14, 0x28,
-	0x59, 0x73, 0x71, 0x86, 0xc0, 0xd4, 0x08, 0x49, 0x70, 0xb1, 0x17, 0xa7, 0x26, 0xe7, 0xe7, 0xa5,
-	0x14, 0x4b, 0x30, 0x2a, 0x30, 0x6a, 0x30, 0x07, 0xc1, 0xb8, 0x42, 0x22, 0x5c, 0xac, 0x79, 0x89,
-	0x79, 0xf9, 0xc5, 0x12, 0x4c, 0x0a, 0x8c, 0x1a, 0xac, 0x41, 0x10, 0x8e, 0x53, 0x1d, 0x97, 0x70,
-	0x72, 0x7e, 0xae, 0x1e, 0x9a, 0x99, 0x4e, 0x7c, 0x70, 0x13, 0x03, 0x40, 0x42, 0x01, 0x8c, 0x51,
-	0xda, 0xe9, 0x99, 0x25, 0x19, 0xa5, 0x49, 0x7a, 0xc9, 0xf9, 0xb9, 0xfa, 0xe9, 0xf9, 0x39, 0x89,
-	0x79, 0xe9, 0x08, 0x27, 0x16, 0x94, 0x54, 0x16, 0xa4, 0x16, 0x23, 0x5c, 0xfa, 0x83, 0x91, 0x71,
-	0x11, 0x13, 0xb3, 0x7b, 0x80, 0xd3, 0x2a, 0x26, 0x39, 0x77, 0x88, 0xc9, 0x01, 0x50, 0xb5, 0x7a,
-	0xe1, 0xa9, 0x39, 0x39, 0xde, 0x79, 0xf9, 0xe5, 0x79, 0x21, 0x20, 0x3d, 0x49, 0x6c, 0x60, 0x43,
-	0x8c, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0xbc, 0x77, 0x4a, 0x07, 0xf7, 0x00, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_goTypes,
+		DependencyIndexes: file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_depIdxs,
+	}.Build()
+	File_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto = out.File
+	file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_rawDesc = nil
+	file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_goTypes = nil
+	file_github_com_golang_protobuf_ptypes_timestamp_timestamp_proto_depIdxs = nil
 }
diff --git a/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.proto b/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.proto
deleted file mode 100644
index eafb3fa..0000000
--- a/vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.proto
+++ /dev/null
@@ -1,135 +0,0 @@
-// 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/timestamp";
-option java_package = "com.google.protobuf";
-option java_outer_classname = "TimestampProto";
-option java_multiple_files = true;
-option objc_class_prefix = "GPB";
-
-// A Timestamp represents a point in time independent of any time zone
-// or calendar, represented as seconds and fractions of seconds at
-// nanosecond resolution in UTC Epoch time. It is encoded using the
-// Proleptic Gregorian Calendar which extends the Gregorian calendar
-// backwards to year one. It is encoded assuming all minutes are 60
-// seconds long, i.e. leap seconds are "smeared" so that no leap second
-// table is needed for interpretation. Range is from
-// 0001-01-01T00:00:00Z to 9999-12-31T23:59:59.999999999Z.
-// By restricting to that range, we ensure that we can convert to
-// and from  RFC 3339 date strings.
-// See [https://www.ietf.org/rfc/rfc3339.txt](https://www.ietf.org/rfc/rfc3339.txt).
-//
-// # Examples
-//
-// Example 1: Compute Timestamp from POSIX `time()`.
-//
-//     Timestamp timestamp;
-//     timestamp.set_seconds(time(NULL));
-//     timestamp.set_nanos(0);
-//
-// Example 2: Compute Timestamp from POSIX `gettimeofday()`.
-//
-//     struct timeval tv;
-//     gettimeofday(&tv, NULL);
-//
-//     Timestamp timestamp;
-//     timestamp.set_seconds(tv.tv_sec);
-//     timestamp.set_nanos(tv.tv_usec * 1000);
-//
-// Example 3: Compute Timestamp from Win32 `GetSystemTimeAsFileTime()`.
-//
-//     FILETIME ft;
-//     GetSystemTimeAsFileTime(&ft);
-//     UINT64 ticks = (((UINT64)ft.dwHighDateTime) << 32) | ft.dwLowDateTime;
-//
-//     // A Windows tick is 100 nanoseconds. Windows epoch 1601-01-01T00:00:00Z
-//     // is 11644473600 seconds before Unix epoch 1970-01-01T00:00:00Z.
-//     Timestamp timestamp;
-//     timestamp.set_seconds((INT64) ((ticks / 10000000) - 11644473600LL));
-//     timestamp.set_nanos((INT32) ((ticks % 10000000) * 100));
-//
-// Example 4: Compute Timestamp from Java `System.currentTimeMillis()`.
-//
-//     long millis = System.currentTimeMillis();
-//
-//     Timestamp timestamp = Timestamp.newBuilder().setSeconds(millis / 1000)
-//         .setNanos((int) ((millis % 1000) * 1000000)).build();
-//
-//
-// Example 5: Compute Timestamp from current time in Python.
-//
-//     timestamp = Timestamp()
-//     timestamp.GetCurrentTime()
-//
-// # JSON Mapping
-//
-// In JSON format, the Timestamp type is encoded as a string in the
-// [RFC 3339](https://www.ietf.org/rfc/rfc3339.txt) format. That is, the
-// format is "{year}-{month}-{day}T{hour}:{min}:{sec}[.{frac_sec}]Z"
-// where {year} is always expressed using four digits while {month}, {day},
-// {hour}, {min}, and {sec} are zero-padded to two digits each. The fractional
-// seconds, which can go up to 9 digits (i.e. up to 1 nanosecond resolution),
-// are optional. The "Z" suffix indicates the timezone ("UTC"); the timezone
-// is required. A proto3 JSON serializer should always use UTC (as indicated by
-// "Z") when printing the Timestamp type and a proto3 JSON parser should be
-// able to accept both UTC and other timezones (as indicated by an offset).
-//
-// For example, "2017-01-15T01:30:15.01Z" encodes 15.01 seconds past
-// 01:30 UTC on January 15, 2017.
-//
-// In JavaScript, one can convert a Date object to this format using the
-// standard [toISOString()](https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Date/toISOString]
-// method. In Python, a standard `datetime.datetime` object can be converted
-// to this format using [`strftime`](https://docs.python.org/2/library/time.html#time.strftime)
-// with the time format spec '%Y-%m-%dT%H:%M:%S.%fZ'. Likewise, in Java, one
-// can use the Joda Time's [`ISODateTimeFormat.dateTime()`](
-// http://www.joda.org/joda-time/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime--
-// ) to obtain a formatter capable of generating timestamps in this format.
-//
-//
-message Timestamp {
-
-  // Represents seconds of UTC time since Unix epoch
-  // 1970-01-01T00:00:00Z. Must be from 0001-01-01T00:00:00Z to
-  // 9999-12-31T23:59:59Z inclusive.
-  int64 seconds = 1;
-
-  // Non-negative fractions of a second at nanosecond resolution. Negative
-  // second values with fractions must still have non-negative nanos values
-  // that count forward in time. Must be from 0 to 999,999,999
-  // inclusive.
-  int32 nanos = 2;
-}
diff --git a/vendor/github.com/golang/snappy/AUTHORS b/vendor/github.com/golang/snappy/AUTHORS
index bcfa195..203e84e 100644
--- a/vendor/github.com/golang/snappy/AUTHORS
+++ b/vendor/github.com/golang/snappy/AUTHORS
@@ -8,8 +8,10 @@
 
 # Please keep the list sorted.
 
+Amazon.com, Inc
 Damian Gryski <dgryski@gmail.com>
 Google Inc.
 Jan Mercl <0xjnml@gmail.com>
+Klaus Post <klauspost@gmail.com>
 Rodolfo Carvalho <rhcarvalho@gmail.com>
 Sebastien Binet <seb.binet@gmail.com>
diff --git a/vendor/github.com/golang/snappy/CONTRIBUTORS b/vendor/github.com/golang/snappy/CONTRIBUTORS
index 931ae31..d991473 100644
--- a/vendor/github.com/golang/snappy/CONTRIBUTORS
+++ b/vendor/github.com/golang/snappy/CONTRIBUTORS
@@ -28,7 +28,9 @@
 
 Damian Gryski <dgryski@gmail.com>
 Jan Mercl <0xjnml@gmail.com>
+Jonathan Swinney <jswinney@amazon.com>
 Kai Backman <kaib@golang.org>
+Klaus Post <klauspost@gmail.com>
 Marc-Antoine Ruel <maruel@chromium.org>
 Nigel Tao <nigeltao@golang.org>
 Rob Pike <r@golang.org>
diff --git a/vendor/github.com/golang/snappy/decode.go b/vendor/github.com/golang/snappy/decode.go
index 72efb03..f1e04b1 100644
--- a/vendor/github.com/golang/snappy/decode.go
+++ b/vendor/github.com/golang/snappy/decode.go
@@ -52,6 +52,8 @@
 // Otherwise, a newly allocated slice will be returned.
 //
 // The dst and src must not overlap. It is valid to pass a nil dst.
+//
+// Decode handles the Snappy block format, not the Snappy stream format.
 func Decode(dst, src []byte) ([]byte, error) {
 	dLen, s, err := decodedLen(src)
 	if err != nil {
@@ -83,6 +85,8 @@
 }
 
 // Reader is an io.Reader that can read Snappy-compressed bytes.
+//
+// Reader handles the Snappy stream format, not the Snappy block format.
 type Reader struct {
 	r       io.Reader
 	err     error
diff --git a/vendor/github.com/golang/snappy/decode_amd64.go b/vendor/github.com/golang/snappy/decode_amd64.go
deleted file mode 100644
index fcd192b..0000000
--- a/vendor/github.com/golang/snappy/decode_amd64.go
+++ /dev/null
@@ -1,14 +0,0 @@
-// Copyright 2016 The Snappy-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 !appengine
-// +build gc
-// +build !noasm
-
-package snappy
-
-// decode has the same semantics as in decode_other.go.
-//
-//go:noescape
-func decode(dst, src []byte) int
diff --git a/vendor/github.com/golang/snappy/decode_arm64.s b/vendor/github.com/golang/snappy/decode_arm64.s
new file mode 100644
index 0000000..7a3ead1
--- /dev/null
+++ b/vendor/github.com/golang/snappy/decode_arm64.s
@@ -0,0 +1,494 @@
+// Copyright 2020 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 !appengine
+// +build gc
+// +build !noasm
+
+#include "textflag.h"
+
+// The asm code generally follows the pure Go code in decode_other.go, except
+// where marked with a "!!!".
+
+// func decode(dst, src []byte) int
+//
+// All local variables fit into registers. The non-zero stack size is only to
+// spill registers and push args when issuing a CALL. The register allocation:
+//	- R2	scratch
+//	- R3	scratch
+//	- R4	length or x
+//	- R5	offset
+//	- R6	&src[s]
+//	- R7	&dst[d]
+//	+ R8	dst_base
+//	+ R9	dst_len
+//	+ R10	dst_base + dst_len
+//	+ R11	src_base
+//	+ R12	src_len
+//	+ R13	src_base + src_len
+//	- R14	used by doCopy
+//	- R15	used by doCopy
+//
+// The registers R8-R13 (marked with a "+") are set at the start of the
+// function, and after a CALL returns, and are not otherwise modified.
+//
+// The d variable is implicitly R7 - R8,  and len(dst)-d is R10 - R7.
+// The s variable is implicitly R6 - R11, and len(src)-s is R13 - R6.
+TEXT ·decode(SB), NOSPLIT, $56-56
+	// Initialize R6, R7 and R8-R13.
+	MOVD dst_base+0(FP), R8
+	MOVD dst_len+8(FP), R9
+	MOVD R8, R7
+	MOVD R8, R10
+	ADD  R9, R10, R10
+	MOVD src_base+24(FP), R11
+	MOVD src_len+32(FP), R12
+	MOVD R11, R6
+	MOVD R11, R13
+	ADD  R12, R13, R13
+
+loop:
+	// for s < len(src)
+	CMP R13, R6
+	BEQ end
+
+	// R4 = uint32(src[s])
+	//
+	// switch src[s] & 0x03
+	MOVBU (R6), R4
+	MOVW  R4, R3
+	ANDW  $3, R3
+	MOVW  $1, R1
+	CMPW  R1, R3
+	BGE   tagCopy
+
+	// ----------------------------------------
+	// The code below handles literal tags.
+
+	// case tagLiteral:
+	// x := uint32(src[s] >> 2)
+	// switch
+	MOVW $60, R1
+	LSRW $2, R4, R4
+	CMPW R4, R1
+	BLS  tagLit60Plus
+
+	// case x < 60:
+	// s++
+	ADD $1, R6, R6
+
+doLit:
+	// This is the end of the inner "switch", when we have a literal tag.
+	//
+	// We assume that R4 == x and x fits in a uint32, where x is the variable
+	// used in the pure Go decode_other.go code.
+
+	// length = int(x) + 1
+	//
+	// Unlike the pure Go code, we don't need to check if length <= 0 because
+	// R4 can hold 64 bits, so the increment cannot overflow.
+	ADD $1, R4, R4
+
+	// Prepare to check if copying length bytes will run past the end of dst or
+	// src.
+	//
+	// R2 = len(dst) - d
+	// R3 = len(src) - s
+	MOVD R10, R2
+	SUB  R7, R2, R2
+	MOVD R13, R3
+	SUB  R6, R3, R3
+
+	// !!! Try a faster technique for short (16 or fewer bytes) copies.
+	//
+	// if length > 16 || len(dst)-d < 16 || len(src)-s < 16 {
+	//   goto callMemmove // Fall back on calling runtime·memmove.
+	// }
+	//
+	// The C++ snappy code calls this TryFastAppend. It also checks len(src)-s
+	// against 21 instead of 16, because it cannot assume that all of its input
+	// is contiguous in memory and so it needs to leave enough source bytes to
+	// read the next tag without refilling buffers, but Go's Decode assumes
+	// contiguousness (the src argument is a []byte).
+	CMP $16, R4
+	BGT callMemmove
+	CMP $16, R2
+	BLT callMemmove
+	CMP $16, R3
+	BLT callMemmove
+
+	// !!! Implement the copy from src to dst as a 16-byte load and store.
+	// (Decode's documentation says that dst and src must not overlap.)
+	//
+	// This always copies 16 bytes, instead of only length bytes, but that's
+	// OK. If the input is a valid Snappy encoding then subsequent iterations
+	// will fix up the overrun. Otherwise, Decode returns a nil []byte (and a
+	// non-nil error), so the overrun will be ignored.
+	//
+	// Note that on arm64, it is legal and cheap to issue unaligned 8-byte or
+	// 16-byte loads and stores. This technique probably wouldn't be as
+	// effective on architectures that are fussier about alignment.
+	LDP 0(R6), (R14, R15)
+	STP (R14, R15), 0(R7)
+
+	// d += length
+	// s += length
+	ADD R4, R7, R7
+	ADD R4, R6, R6
+	B   loop
+
+callMemmove:
+	// if length > len(dst)-d || length > len(src)-s { etc }
+	CMP R2, R4
+	BGT errCorrupt
+	CMP R3, R4
+	BGT errCorrupt
+
+	// copy(dst[d:], src[s:s+length])
+	//
+	// This means calling runtime·memmove(&dst[d], &src[s], length), so we push
+	// R7, R6 and R4 as arguments. Coincidentally, we also need to spill those
+	// three registers to the stack, to save local variables across the CALL.
+	MOVD R7, 8(RSP)
+	MOVD R6, 16(RSP)
+	MOVD R4, 24(RSP)
+	MOVD R7, 32(RSP)
+	MOVD R6, 40(RSP)
+	MOVD R4, 48(RSP)
+	CALL runtime·memmove(SB)
+
+	// Restore local variables: unspill registers from the stack and
+	// re-calculate R8-R13.
+	MOVD 32(RSP), R7
+	MOVD 40(RSP), R6
+	MOVD 48(RSP), R4
+	MOVD dst_base+0(FP), R8
+	MOVD dst_len+8(FP), R9
+	MOVD R8, R10
+	ADD  R9, R10, R10
+	MOVD src_base+24(FP), R11
+	MOVD src_len+32(FP), R12
+	MOVD R11, R13
+	ADD  R12, R13, R13
+
+	// d += length
+	// s += length
+	ADD R4, R7, R7
+	ADD R4, R6, R6
+	B   loop
+
+tagLit60Plus:
+	// !!! This fragment does the
+	//
+	// s += x - 58; if uint(s) > uint(len(src)) { etc }
+	//
+	// checks. In the asm version, we code it once instead of once per switch case.
+	ADD  R4, R6, R6
+	SUB  $58, R6, R6
+	MOVD R6, R3
+	SUB  R11, R3, R3
+	CMP  R12, R3
+	BGT  errCorrupt
+
+	// case x == 60:
+	MOVW $61, R1
+	CMPW R1, R4
+	BEQ  tagLit61
+	BGT  tagLit62Plus
+
+	// x = uint32(src[s-1])
+	MOVBU -1(R6), R4
+	B     doLit
+
+tagLit61:
+	// case x == 61:
+	// x = uint32(src[s-2]) | uint32(src[s-1])<<8
+	MOVHU -2(R6), R4
+	B     doLit
+
+tagLit62Plus:
+	CMPW $62, R4
+	BHI  tagLit63
+
+	// case x == 62:
+	// x = uint32(src[s-3]) | uint32(src[s-2])<<8 | uint32(src[s-1])<<16
+	MOVHU -3(R6), R4
+	MOVBU -1(R6), R3
+	ORR   R3<<16, R4
+	B     doLit
+
+tagLit63:
+	// case x == 63:
+	// x = uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24
+	MOVWU -4(R6), R4
+	B     doLit
+
+	// The code above handles literal tags.
+	// ----------------------------------------
+	// The code below handles copy tags.
+
+tagCopy4:
+	// case tagCopy4:
+	// s += 5
+	ADD $5, R6, R6
+
+	// if uint(s) > uint(len(src)) { etc }
+	MOVD R6, R3
+	SUB  R11, R3, R3
+	CMP  R12, R3
+	BGT  errCorrupt
+
+	// length = 1 + int(src[s-5])>>2
+	MOVD $1, R1
+	ADD  R4>>2, R1, R4
+
+	// offset = int(uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24)
+	MOVWU -4(R6), R5
+	B     doCopy
+
+tagCopy2:
+	// case tagCopy2:
+	// s += 3
+	ADD $3, R6, R6
+
+	// if uint(s) > uint(len(src)) { etc }
+	MOVD R6, R3
+	SUB  R11, R3, R3
+	CMP  R12, R3
+	BGT  errCorrupt
+
+	// length = 1 + int(src[s-3])>>2
+	MOVD $1, R1
+	ADD  R4>>2, R1, R4
+
+	// offset = int(uint32(src[s-2]) | uint32(src[s-1])<<8)
+	MOVHU -2(R6), R5
+	B     doCopy
+
+tagCopy:
+	// We have a copy tag. We assume that:
+	//	- R3 == src[s] & 0x03
+	//	- R4 == src[s]
+	CMP $2, R3
+	BEQ tagCopy2
+	BGT tagCopy4
+
+	// case tagCopy1:
+	// s += 2
+	ADD $2, R6, R6
+
+	// if uint(s) > uint(len(src)) { etc }
+	MOVD R6, R3
+	SUB  R11, R3, R3
+	CMP  R12, R3
+	BGT  errCorrupt
+
+	// offset = int(uint32(src[s-2])&0xe0<<3 | uint32(src[s-1]))
+	MOVD  R4, R5
+	AND   $0xe0, R5
+	MOVBU -1(R6), R3
+	ORR   R5<<3, R3, R5
+
+	// length = 4 + int(src[s-2])>>2&0x7
+	MOVD $7, R1
+	AND  R4>>2, R1, R4
+	ADD  $4, R4, R4
+
+doCopy:
+	// This is the end of the outer "switch", when we have a copy tag.
+	//
+	// We assume that:
+	//	- R4 == length && R4 > 0
+	//	- R5 == offset
+
+	// if offset <= 0 { etc }
+	MOVD $0, R1
+	CMP  R1, R5
+	BLE  errCorrupt
+
+	// if d < offset { etc }
+	MOVD R7, R3
+	SUB  R8, R3, R3
+	CMP  R5, R3
+	BLT  errCorrupt
+
+	// if length > len(dst)-d { etc }
+	MOVD R10, R3
+	SUB  R7, R3, R3
+	CMP  R3, R4
+	BGT  errCorrupt
+
+	// forwardCopy(dst[d:d+length], dst[d-offset:]); d += length
+	//
+	// Set:
+	//	- R14 = len(dst)-d
+	//	- R15 = &dst[d-offset]
+	MOVD R10, R14
+	SUB  R7, R14, R14
+	MOVD R7, R15
+	SUB  R5, R15, R15
+
+	// !!! Try a faster technique for short (16 or fewer bytes) forward copies.
+	//
+	// First, try using two 8-byte load/stores, similar to the doLit technique
+	// above. Even if dst[d:d+length] and dst[d-offset:] can overlap, this is
+	// still OK if offset >= 8. Note that this has to be two 8-byte load/stores
+	// and not one 16-byte load/store, and the first store has to be before the
+	// second load, due to the overlap if offset is in the range [8, 16).
+	//
+	// if length > 16 || offset < 8 || len(dst)-d < 16 {
+	//   goto slowForwardCopy
+	// }
+	// copy 16 bytes
+	// d += length
+	CMP  $16, R4
+	BGT  slowForwardCopy
+	CMP  $8, R5
+	BLT  slowForwardCopy
+	CMP  $16, R14
+	BLT  slowForwardCopy
+	MOVD 0(R15), R2
+	MOVD R2, 0(R7)
+	MOVD 8(R15), R3
+	MOVD R3, 8(R7)
+	ADD  R4, R7, R7
+	B    loop
+
+slowForwardCopy:
+	// !!! If the forward copy is longer than 16 bytes, or if offset < 8, we
+	// can still try 8-byte load stores, provided we can overrun up to 10 extra
+	// bytes. As above, the overrun will be fixed up by subsequent iterations
+	// of the outermost loop.
+	//
+	// The C++ snappy code calls this technique IncrementalCopyFastPath. Its
+	// commentary says:
+	//
+	// ----
+	//
+	// The main part of this loop is a simple copy of eight bytes at a time
+	// until we've copied (at least) the requested amount of bytes.  However,
+	// if d and d-offset are less than eight bytes apart (indicating a
+	// repeating pattern of length < 8), we first need to expand the pattern in
+	// order to get the correct results. For instance, if the buffer looks like
+	// this, with the eight-byte <d-offset> and <d> patterns marked as
+	// intervals:
+	//
+	//    abxxxxxxxxxxxx
+	//    [------]           d-offset
+	//      [------]         d
+	//
+	// a single eight-byte copy from <d-offset> to <d> will repeat the pattern
+	// once, after which we can move <d> two bytes without moving <d-offset>:
+	//
+	//    ababxxxxxxxxxx
+	//    [------]           d-offset
+	//        [------]       d
+	//
+	// and repeat the exercise until the two no longer overlap.
+	//
+	// This allows us to do very well in the special case of one single byte
+	// repeated many times, without taking a big hit for more general cases.
+	//
+	// The worst case of extra writing past the end of the match occurs when
+	// offset == 1 and length == 1; the last copy will read from byte positions
+	// [0..7] and write to [4..11], whereas it was only supposed to write to
+	// position 1. Thus, ten excess bytes.
+	//
+	// ----
+	//
+	// That "10 byte overrun" worst case is confirmed by Go's
+	// TestSlowForwardCopyOverrun, which also tests the fixUpSlowForwardCopy
+	// and finishSlowForwardCopy algorithm.
+	//
+	// if length > len(dst)-d-10 {
+	//   goto verySlowForwardCopy
+	// }
+	SUB $10, R14, R14
+	CMP R14, R4
+	BGT verySlowForwardCopy
+
+makeOffsetAtLeast8:
+	// !!! As above, expand the pattern so that offset >= 8 and we can use
+	// 8-byte load/stores.
+	//
+	// for offset < 8 {
+	//   copy 8 bytes from dst[d-offset:] to dst[d:]
+	//   length -= offset
+	//   d      += offset
+	//   offset += offset
+	//   // The two previous lines together means that d-offset, and therefore
+	//   // R15, is unchanged.
+	// }
+	CMP  $8, R5
+	BGE  fixUpSlowForwardCopy
+	MOVD (R15), R3
+	MOVD R3, (R7)
+	SUB  R5, R4, R4
+	ADD  R5, R7, R7
+	ADD  R5, R5, R5
+	B    makeOffsetAtLeast8
+
+fixUpSlowForwardCopy:
+	// !!! Add length (which might be negative now) to d (implied by R7 being
+	// &dst[d]) so that d ends up at the right place when we jump back to the
+	// top of the loop. Before we do that, though, we save R7 to R2 so that, if
+	// length is positive, copying the remaining length bytes will write to the
+	// right place.
+	MOVD R7, R2
+	ADD  R4, R7, R7
+
+finishSlowForwardCopy:
+	// !!! Repeat 8-byte load/stores until length <= 0. Ending with a negative
+	// length means that we overrun, but as above, that will be fixed up by
+	// subsequent iterations of the outermost loop.
+	MOVD $0, R1
+	CMP  R1, R4
+	BLE  loop
+	MOVD (R15), R3
+	MOVD R3, (R2)
+	ADD  $8, R15, R15
+	ADD  $8, R2, R2
+	SUB  $8, R4, R4
+	B    finishSlowForwardCopy
+
+verySlowForwardCopy:
+	// verySlowForwardCopy is a simple implementation of forward copy. In C
+	// parlance, this is a do/while loop instead of a while loop, since we know
+	// that length > 0. In Go syntax:
+	//
+	// for {
+	//   dst[d] = dst[d - offset]
+	//   d++
+	//   length--
+	//   if length == 0 {
+	//     break
+	//   }
+	// }
+	MOVB (R15), R3
+	MOVB R3, (R7)
+	ADD  $1, R15, R15
+	ADD  $1, R7, R7
+	SUB  $1, R4, R4
+	CBNZ R4, verySlowForwardCopy
+	B    loop
+
+	// The code above handles copy tags.
+	// ----------------------------------------
+
+end:
+	// This is the end of the "for s < len(src)".
+	//
+	// if d != len(dst) { etc }
+	CMP R10, R7
+	BNE errCorrupt
+
+	// return 0
+	MOVD $0, ret+48(FP)
+	RET
+
+errCorrupt:
+	// return decodeErrCodeCorrupt
+	MOVD $1, R2
+	MOVD R2, ret+48(FP)
+	RET
diff --git a/vendor/github.com/golang/snappy/decode_asm.go b/vendor/github.com/golang/snappy/decode_asm.go
new file mode 100644
index 0000000..7082b34
--- /dev/null
+++ b/vendor/github.com/golang/snappy/decode_asm.go
@@ -0,0 +1,15 @@
+// Copyright 2016 The Snappy-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 !appengine
+// +build gc
+// +build !noasm
+// +build amd64 arm64
+
+package snappy
+
+// decode has the same semantics as in decode_other.go.
+//
+//go:noescape
+func decode(dst, src []byte) int
diff --git a/vendor/github.com/golang/snappy/decode_other.go b/vendor/github.com/golang/snappy/decode_other.go
index 8c9f204..2f672be 100644
--- a/vendor/github.com/golang/snappy/decode_other.go
+++ b/vendor/github.com/golang/snappy/decode_other.go
@@ -2,7 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !amd64 appengine !gc noasm
+// +build !amd64,!arm64 appengine !gc noasm
 
 package snappy
 
@@ -85,14 +85,28 @@
 		if offset <= 0 || d < offset || length > len(dst)-d {
 			return decodeErrCodeCorrupt
 		}
-		// Copy from an earlier sub-slice of dst to a later sub-slice. Unlike
-		// the built-in copy function, this byte-by-byte copy always runs
+		// Copy from an earlier sub-slice of dst to a later sub-slice.
+		// If no overlap, use the built-in copy:
+		if offset >= length {
+			copy(dst[d:d+length], dst[d-offset:])
+			d += length
+			continue
+		}
+
+		// Unlike the built-in copy function, this byte-by-byte copy always runs
 		// forwards, even if the slices overlap. Conceptually, this is:
 		//
 		// d += forwardCopy(dst[d:d+length], dst[d-offset:])
-		for end := d + length; d != end; d++ {
-			dst[d] = dst[d-offset]
+		//
+		// We align the slices into a and b and show the compiler they are the same size.
+		// This allows the loop to run without bounds checks.
+		a := dst[d : d+length]
+		b := dst[d-offset:]
+		b = b[:len(a)]
+		for i := range a {
+			a[i] = b[i]
 		}
+		d += length
 	}
 	if d != len(dst) {
 		return decodeErrCodeCorrupt
diff --git a/vendor/github.com/golang/snappy/encode.go b/vendor/github.com/golang/snappy/encode.go
index 8d393e9..7f23657 100644
--- a/vendor/github.com/golang/snappy/encode.go
+++ b/vendor/github.com/golang/snappy/encode.go
@@ -15,6 +15,8 @@
 // Otherwise, a newly allocated slice will be returned.
 //
 // The dst and src must not overlap. It is valid to pass a nil dst.
+//
+// Encode handles the Snappy block format, not the Snappy stream format.
 func Encode(dst, src []byte) []byte {
 	if n := MaxEncodedLen(len(src)); n < 0 {
 		panic(ErrTooLarge)
@@ -139,6 +141,8 @@
 }
 
 // Writer is an io.Writer that can write Snappy-compressed bytes.
+//
+// Writer handles the Snappy stream format, not the Snappy block format.
 type Writer struct {
 	w   io.Writer
 	err error
diff --git a/vendor/github.com/golang/snappy/encode_amd64.go b/vendor/github.com/golang/snappy/encode_amd64.go
deleted file mode 100644
index 150d91b..0000000
--- a/vendor/github.com/golang/snappy/encode_amd64.go
+++ /dev/null
@@ -1,29 +0,0 @@
-// Copyright 2016 The Snappy-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 !appengine
-// +build gc
-// +build !noasm
-
-package snappy
-
-// emitLiteral has the same semantics as in encode_other.go.
-//
-//go:noescape
-func emitLiteral(dst, lit []byte) int
-
-// emitCopy has the same semantics as in encode_other.go.
-//
-//go:noescape
-func emitCopy(dst []byte, offset, length int) int
-
-// extendMatch has the same semantics as in encode_other.go.
-//
-//go:noescape
-func extendMatch(src []byte, i, j int) int
-
-// encodeBlock has the same semantics as in encode_other.go.
-//
-//go:noescape
-func encodeBlock(dst, src []byte) (d int)
diff --git a/vendor/github.com/golang/snappy/encode_arm64.s b/vendor/github.com/golang/snappy/encode_arm64.s
new file mode 100644
index 0000000..bf83667
--- /dev/null
+++ b/vendor/github.com/golang/snappy/encode_arm64.s
@@ -0,0 +1,722 @@
+// Copyright 2020 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 !appengine
+// +build gc
+// +build !noasm
+
+#include "textflag.h"
+
+// The asm code generally follows the pure Go code in encode_other.go, except
+// where marked with a "!!!".
+
+// ----------------------------------------------------------------------------
+
+// func emitLiteral(dst, lit []byte) int
+//
+// All local variables fit into registers. The register allocation:
+//	- R3	len(lit)
+//	- R4	n
+//	- R6	return value
+//	- R8	&dst[i]
+//	- R10	&lit[0]
+//
+// The 32 bytes of stack space is to call runtime·memmove.
+//
+// The unusual register allocation of local variables, such as R10 for the
+// source pointer, matches the allocation used at the call site in encodeBlock,
+// which makes it easier to manually inline this function.
+TEXT ·emitLiteral(SB), NOSPLIT, $32-56
+	MOVD dst_base+0(FP), R8
+	MOVD lit_base+24(FP), R10
+	MOVD lit_len+32(FP), R3
+	MOVD R3, R6
+	MOVW R3, R4
+	SUBW $1, R4, R4
+
+	CMPW $60, R4
+	BLT  oneByte
+	CMPW $256, R4
+	BLT  twoBytes
+
+threeBytes:
+	MOVD $0xf4, R2
+	MOVB R2, 0(R8)
+	MOVW R4, 1(R8)
+	ADD  $3, R8, R8
+	ADD  $3, R6, R6
+	B    memmove
+
+twoBytes:
+	MOVD $0xf0, R2
+	MOVB R2, 0(R8)
+	MOVB R4, 1(R8)
+	ADD  $2, R8, R8
+	ADD  $2, R6, R6
+	B    memmove
+
+oneByte:
+	LSLW $2, R4, R4
+	MOVB R4, 0(R8)
+	ADD  $1, R8, R8
+	ADD  $1, R6, R6
+
+memmove:
+	MOVD R6, ret+48(FP)
+
+	// copy(dst[i:], lit)
+	//
+	// This means calling runtime·memmove(&dst[i], &lit[0], len(lit)), so we push
+	// R8, R10 and R3 as arguments.
+	MOVD R8, 8(RSP)
+	MOVD R10, 16(RSP)
+	MOVD R3, 24(RSP)
+	CALL runtime·memmove(SB)
+	RET
+
+// ----------------------------------------------------------------------------
+
+// func emitCopy(dst []byte, offset, length int) int
+//
+// All local variables fit into registers. The register allocation:
+//	- R3	length
+//	- R7	&dst[0]
+//	- R8	&dst[i]
+//	- R11	offset
+//
+// The unusual register allocation of local variables, such as R11 for the
+// offset, matches the allocation used at the call site in encodeBlock, which
+// makes it easier to manually inline this function.
+TEXT ·emitCopy(SB), NOSPLIT, $0-48
+	MOVD dst_base+0(FP), R8
+	MOVD R8, R7
+	MOVD offset+24(FP), R11
+	MOVD length+32(FP), R3
+
+loop0:
+	// for length >= 68 { etc }
+	CMPW $68, R3
+	BLT  step1
+
+	// Emit a length 64 copy, encoded as 3 bytes.
+	MOVD $0xfe, R2
+	MOVB R2, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+	SUB  $64, R3, R3
+	B    loop0
+
+step1:
+	// if length > 64 { etc }
+	CMP $64, R3
+	BLE step2
+
+	// Emit a length 60 copy, encoded as 3 bytes.
+	MOVD $0xee, R2
+	MOVB R2, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+	SUB  $60, R3, R3
+
+step2:
+	// if length >= 12 || offset >= 2048 { goto step3 }
+	CMP  $12, R3
+	BGE  step3
+	CMPW $2048, R11
+	BGE  step3
+
+	// Emit the remaining copy, encoded as 2 bytes.
+	MOVB R11, 1(R8)
+	LSRW $3, R11, R11
+	AND  $0xe0, R11, R11
+	SUB  $4, R3, R3
+	LSLW $2, R3
+	AND  $0xff, R3, R3
+	ORRW R3, R11, R11
+	ORRW $1, R11, R11
+	MOVB R11, 0(R8)
+	ADD  $2, R8, R8
+
+	// Return the number of bytes written.
+	SUB  R7, R8, R8
+	MOVD R8, ret+40(FP)
+	RET
+
+step3:
+	// Emit the remaining copy, encoded as 3 bytes.
+	SUB  $1, R3, R3
+	AND  $0xff, R3, R3
+	LSLW $2, R3, R3
+	ORRW $2, R3, R3
+	MOVB R3, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+
+	// Return the number of bytes written.
+	SUB  R7, R8, R8
+	MOVD R8, ret+40(FP)
+	RET
+
+// ----------------------------------------------------------------------------
+
+// func extendMatch(src []byte, i, j int) int
+//
+// All local variables fit into registers. The register allocation:
+//	- R6	&src[0]
+//	- R7	&src[j]
+//	- R13	&src[len(src) - 8]
+//	- R14	&src[len(src)]
+//	- R15	&src[i]
+//
+// The unusual register allocation of local variables, such as R15 for a source
+// pointer, matches the allocation used at the call site in encodeBlock, which
+// makes it easier to manually inline this function.
+TEXT ·extendMatch(SB), NOSPLIT, $0-48
+	MOVD src_base+0(FP), R6
+	MOVD src_len+8(FP), R14
+	MOVD i+24(FP), R15
+	MOVD j+32(FP), R7
+	ADD  R6, R14, R14
+	ADD  R6, R15, R15
+	ADD  R6, R7, R7
+	MOVD R14, R13
+	SUB  $8, R13, R13
+
+cmp8:
+	// As long as we are 8 or more bytes before the end of src, we can load and
+	// compare 8 bytes at a time. If those 8 bytes are equal, repeat.
+	CMP  R13, R7
+	BHI  cmp1
+	MOVD (R15), R3
+	MOVD (R7), R4
+	CMP  R4, R3
+	BNE  bsf
+	ADD  $8, R15, R15
+	ADD  $8, R7, R7
+	B    cmp8
+
+bsf:
+	// If those 8 bytes were not equal, XOR the two 8 byte values, and return
+	// the index of the first byte that differs.
+	// RBIT reverses the bit order, then CLZ counts the leading zeros, the
+	// combination of which finds the least significant bit which is set.
+	// The arm64 architecture is little-endian, and the shift by 3 converts
+	// a bit index to a byte index.
+	EOR  R3, R4, R4
+	RBIT R4, R4
+	CLZ  R4, R4
+	ADD  R4>>3, R7, R7
+
+	// Convert from &src[ret] to ret.
+	SUB  R6, R7, R7
+	MOVD R7, ret+40(FP)
+	RET
+
+cmp1:
+	// In src's tail, compare 1 byte at a time.
+	CMP  R7, R14
+	BLS  extendMatchEnd
+	MOVB (R15), R3
+	MOVB (R7), R4
+	CMP  R4, R3
+	BNE  extendMatchEnd
+	ADD  $1, R15, R15
+	ADD  $1, R7, R7
+	B    cmp1
+
+extendMatchEnd:
+	// Convert from &src[ret] to ret.
+	SUB  R6, R7, R7
+	MOVD R7, ret+40(FP)
+	RET
+
+// ----------------------------------------------------------------------------
+
+// func encodeBlock(dst, src []byte) (d int)
+//
+// All local variables fit into registers, other than "var table". The register
+// allocation:
+//	- R3	.	.
+//	- R4	.	.
+//	- R5	64	shift
+//	- R6	72	&src[0], tableSize
+//	- R7	80	&src[s]
+//	- R8	88	&dst[d]
+//	- R9	96	sLimit
+//	- R10	.	&src[nextEmit]
+//	- R11	104	prevHash, currHash, nextHash, offset
+//	- R12	112	&src[base], skip
+//	- R13	.	&src[nextS], &src[len(src) - 8]
+//	- R14	.	len(src), bytesBetweenHashLookups, &src[len(src)], x
+//	- R15	120	candidate
+//	- R16	.	hash constant, 0x1e35a7bd
+//	- R17	.	&table
+//	- .  	128	table
+//
+// The second column (64, 72, etc) is the stack offset to spill the registers
+// when calling other functions. We could pack this slightly tighter, but it's
+// simpler to have a dedicated spill map independent of the function called.
+//
+// "var table [maxTableSize]uint16" takes up 32768 bytes of stack space. An
+// extra 64 bytes, to call other functions, and an extra 64 bytes, to spill
+// local variables (registers) during calls gives 32768 + 64 + 64 = 32896.
+TEXT ·encodeBlock(SB), 0, $32896-56
+	MOVD dst_base+0(FP), R8
+	MOVD src_base+24(FP), R7
+	MOVD src_len+32(FP), R14
+
+	// shift, tableSize := uint32(32-8), 1<<8
+	MOVD  $24, R5
+	MOVD  $256, R6
+	MOVW  $0xa7bd, R16
+	MOVKW $(0x1e35<<16), R16
+
+calcShift:
+	// for ; tableSize < maxTableSize && tableSize < len(src); tableSize *= 2 {
+	//	shift--
+	// }
+	MOVD $16384, R2
+	CMP  R2, R6
+	BGE  varTable
+	CMP  R14, R6
+	BGE  varTable
+	SUB  $1, R5, R5
+	LSL  $1, R6, R6
+	B    calcShift
+
+varTable:
+	// var table [maxTableSize]uint16
+	//
+	// In the asm code, unlike the Go code, we can zero-initialize only the
+	// first tableSize elements. Each uint16 element is 2 bytes and each
+	// iterations writes 64 bytes, so we can do only tableSize/32 writes
+	// instead of the 2048 writes that would zero-initialize all of table's
+	// 32768 bytes. This clear could overrun the first tableSize elements, but
+	// it won't overrun the allocated stack size.
+	ADD  $128, RSP, R17
+	MOVD R17, R4
+
+	// !!! R6 = &src[tableSize]
+	ADD R6<<1, R17, R6
+
+memclr:
+	STP.P (ZR, ZR), 64(R4)
+	STP   (ZR, ZR), -48(R4)
+	STP   (ZR, ZR), -32(R4)
+	STP   (ZR, ZR), -16(R4)
+	CMP   R4, R6
+	BHI   memclr
+
+	// !!! R6 = &src[0]
+	MOVD R7, R6
+
+	// sLimit := len(src) - inputMargin
+	MOVD R14, R9
+	SUB  $15, R9, R9
+
+	// !!! Pre-emptively spill R5, R6 and R9 to the stack. Their values don't
+	// change for the rest of the function.
+	MOVD R5, 64(RSP)
+	MOVD R6, 72(RSP)
+	MOVD R9, 96(RSP)
+
+	// nextEmit := 0
+	MOVD R6, R10
+
+	// s := 1
+	ADD $1, R7, R7
+
+	// nextHash := hash(load32(src, s), shift)
+	MOVW 0(R7), R11
+	MULW R16, R11, R11
+	LSRW R5, R11, R11
+
+outer:
+	// for { etc }
+
+	// skip := 32
+	MOVD $32, R12
+
+	// nextS := s
+	MOVD R7, R13
+
+	// candidate := 0
+	MOVD $0, R15
+
+inner0:
+	// for { etc }
+
+	// s := nextS
+	MOVD R13, R7
+
+	// bytesBetweenHashLookups := skip >> 5
+	MOVD R12, R14
+	LSR  $5, R14, R14
+
+	// nextS = s + bytesBetweenHashLookups
+	ADD R14, R13, R13
+
+	// skip += bytesBetweenHashLookups
+	ADD R14, R12, R12
+
+	// if nextS > sLimit { goto emitRemainder }
+	MOVD R13, R3
+	SUB  R6, R3, R3
+	CMP  R9, R3
+	BHI  emitRemainder
+
+	// candidate = int(table[nextHash])
+	MOVHU 0(R17)(R11<<1), R15
+
+	// table[nextHash] = uint16(s)
+	MOVD R7, R3
+	SUB  R6, R3, R3
+
+	MOVH R3, 0(R17)(R11<<1)
+
+	// nextHash = hash(load32(src, nextS), shift)
+	MOVW 0(R13), R11
+	MULW R16, R11
+	LSRW R5, R11, R11
+
+	// if load32(src, s) != load32(src, candidate) { continue } break
+	MOVW 0(R7), R3
+	MOVW (R6)(R15*1), R4
+	CMPW R4, R3
+	BNE  inner0
+
+fourByteMatch:
+	// As per the encode_other.go code:
+	//
+	// A 4-byte match has been found. We'll later see etc.
+
+	// !!! Jump to a fast path for short (<= 16 byte) literals. See the comment
+	// on inputMargin in encode.go.
+	MOVD R7, R3
+	SUB  R10, R3, R3
+	CMP  $16, R3
+	BLE  emitLiteralFastPath
+
+	// ----------------------------------------
+	// Begin inline of the emitLiteral call.
+	//
+	// d += emitLiteral(dst[d:], src[nextEmit:s])
+
+	MOVW R3, R4
+	SUBW $1, R4, R4
+
+	MOVW $60, R2
+	CMPW R2, R4
+	BLT  inlineEmitLiteralOneByte
+	MOVW $256, R2
+	CMPW R2, R4
+	BLT  inlineEmitLiteralTwoBytes
+
+inlineEmitLiteralThreeBytes:
+	MOVD $0xf4, R1
+	MOVB R1, 0(R8)
+	MOVW R4, 1(R8)
+	ADD  $3, R8, R8
+	B    inlineEmitLiteralMemmove
+
+inlineEmitLiteralTwoBytes:
+	MOVD $0xf0, R1
+	MOVB R1, 0(R8)
+	MOVB R4, 1(R8)
+	ADD  $2, R8, R8
+	B    inlineEmitLiteralMemmove
+
+inlineEmitLiteralOneByte:
+	LSLW $2, R4, R4
+	MOVB R4, 0(R8)
+	ADD  $1, R8, R8
+
+inlineEmitLiteralMemmove:
+	// Spill local variables (registers) onto the stack; call; unspill.
+	//
+	// copy(dst[i:], lit)
+	//
+	// This means calling runtime·memmove(&dst[i], &lit[0], len(lit)), so we push
+	// R8, R10 and R3 as arguments.
+	MOVD R8, 8(RSP)
+	MOVD R10, 16(RSP)
+	MOVD R3, 24(RSP)
+
+	// Finish the "d +=" part of "d += emitLiteral(etc)".
+	ADD   R3, R8, R8
+	MOVD  R7, 80(RSP)
+	MOVD  R8, 88(RSP)
+	MOVD  R15, 120(RSP)
+	CALL  runtime·memmove(SB)
+	MOVD  64(RSP), R5
+	MOVD  72(RSP), R6
+	MOVD  80(RSP), R7
+	MOVD  88(RSP), R8
+	MOVD  96(RSP), R9
+	MOVD  120(RSP), R15
+	ADD   $128, RSP, R17
+	MOVW  $0xa7bd, R16
+	MOVKW $(0x1e35<<16), R16
+	B     inner1
+
+inlineEmitLiteralEnd:
+	// End inline of the emitLiteral call.
+	// ----------------------------------------
+
+emitLiteralFastPath:
+	// !!! Emit the 1-byte encoding "uint8(len(lit)-1)<<2".
+	MOVB R3, R4
+	SUBW $1, R4, R4
+	AND  $0xff, R4, R4
+	LSLW $2, R4, R4
+	MOVB R4, (R8)
+	ADD  $1, R8, R8
+
+	// !!! Implement the copy from lit to dst as a 16-byte load and store.
+	// (Encode's documentation says that dst and src must not overlap.)
+	//
+	// This always copies 16 bytes, instead of only len(lit) bytes, but that's
+	// OK. Subsequent iterations will fix up the overrun.
+	//
+	// Note that on arm64, it is legal and cheap to issue unaligned 8-byte or
+	// 16-byte loads and stores. This technique probably wouldn't be as
+	// effective on architectures that are fussier about alignment.
+	LDP 0(R10), (R0, R1)
+	STP (R0, R1), 0(R8)
+	ADD R3, R8, R8
+
+inner1:
+	// for { etc }
+
+	// base := s
+	MOVD R7, R12
+
+	// !!! offset := base - candidate
+	MOVD R12, R11
+	SUB  R15, R11, R11
+	SUB  R6, R11, R11
+
+	// ----------------------------------------
+	// Begin inline of the extendMatch call.
+	//
+	// s = extendMatch(src, candidate+4, s+4)
+
+	// !!! R14 = &src[len(src)]
+	MOVD src_len+32(FP), R14
+	ADD  R6, R14, R14
+
+	// !!! R13 = &src[len(src) - 8]
+	MOVD R14, R13
+	SUB  $8, R13, R13
+
+	// !!! R15 = &src[candidate + 4]
+	ADD $4, R15, R15
+	ADD R6, R15, R15
+
+	// !!! s += 4
+	ADD $4, R7, R7
+
+inlineExtendMatchCmp8:
+	// As long as we are 8 or more bytes before the end of src, we can load and
+	// compare 8 bytes at a time. If those 8 bytes are equal, repeat.
+	CMP  R13, R7
+	BHI  inlineExtendMatchCmp1
+	MOVD (R15), R3
+	MOVD (R7), R4
+	CMP  R4, R3
+	BNE  inlineExtendMatchBSF
+	ADD  $8, R15, R15
+	ADD  $8, R7, R7
+	B    inlineExtendMatchCmp8
+
+inlineExtendMatchBSF:
+	// If those 8 bytes were not equal, XOR the two 8 byte values, and return
+	// the index of the first byte that differs.
+	// RBIT reverses the bit order, then CLZ counts the leading zeros, the
+	// combination of which finds the least significant bit which is set.
+	// The arm64 architecture is little-endian, and the shift by 3 converts
+	// a bit index to a byte index.
+	EOR  R3, R4, R4
+	RBIT R4, R4
+	CLZ  R4, R4
+	ADD  R4>>3, R7, R7
+	B    inlineExtendMatchEnd
+
+inlineExtendMatchCmp1:
+	// In src's tail, compare 1 byte at a time.
+	CMP  R7, R14
+	BLS  inlineExtendMatchEnd
+	MOVB (R15), R3
+	MOVB (R7), R4
+	CMP  R4, R3
+	BNE  inlineExtendMatchEnd
+	ADD  $1, R15, R15
+	ADD  $1, R7, R7
+	B    inlineExtendMatchCmp1
+
+inlineExtendMatchEnd:
+	// End inline of the extendMatch call.
+	// ----------------------------------------
+
+	// ----------------------------------------
+	// Begin inline of the emitCopy call.
+	//
+	// d += emitCopy(dst[d:], base-candidate, s-base)
+
+	// !!! length := s - base
+	MOVD R7, R3
+	SUB  R12, R3, R3
+
+inlineEmitCopyLoop0:
+	// for length >= 68 { etc }
+	MOVW $68, R2
+	CMPW R2, R3
+	BLT  inlineEmitCopyStep1
+
+	// Emit a length 64 copy, encoded as 3 bytes.
+	MOVD $0xfe, R1
+	MOVB R1, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+	SUBW $64, R3, R3
+	B    inlineEmitCopyLoop0
+
+inlineEmitCopyStep1:
+	// if length > 64 { etc }
+	MOVW $64, R2
+	CMPW R2, R3
+	BLE  inlineEmitCopyStep2
+
+	// Emit a length 60 copy, encoded as 3 bytes.
+	MOVD $0xee, R1
+	MOVB R1, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+	SUBW $60, R3, R3
+
+inlineEmitCopyStep2:
+	// if length >= 12 || offset >= 2048 { goto inlineEmitCopyStep3 }
+	MOVW $12, R2
+	CMPW R2, R3
+	BGE  inlineEmitCopyStep3
+	MOVW $2048, R2
+	CMPW R2, R11
+	BGE  inlineEmitCopyStep3
+
+	// Emit the remaining copy, encoded as 2 bytes.
+	MOVB R11, 1(R8)
+	LSRW $8, R11, R11
+	LSLW $5, R11, R11
+	SUBW $4, R3, R3
+	AND  $0xff, R3, R3
+	LSLW $2, R3, R3
+	ORRW R3, R11, R11
+	ORRW $1, R11, R11
+	MOVB R11, 0(R8)
+	ADD  $2, R8, R8
+	B    inlineEmitCopyEnd
+
+inlineEmitCopyStep3:
+	// Emit the remaining copy, encoded as 3 bytes.
+	SUBW $1, R3, R3
+	LSLW $2, R3, R3
+	ORRW $2, R3, R3
+	MOVB R3, 0(R8)
+	MOVW R11, 1(R8)
+	ADD  $3, R8, R8
+
+inlineEmitCopyEnd:
+	// End inline of the emitCopy call.
+	// ----------------------------------------
+
+	// nextEmit = s
+	MOVD R7, R10
+
+	// if s >= sLimit { goto emitRemainder }
+	MOVD R7, R3
+	SUB  R6, R3, R3
+	CMP  R3, R9
+	BLS  emitRemainder
+
+	// As per the encode_other.go code:
+	//
+	// We could immediately etc.
+
+	// x := load64(src, s-1)
+	MOVD -1(R7), R14
+
+	// prevHash := hash(uint32(x>>0), shift)
+	MOVW R14, R11
+	MULW R16, R11, R11
+	LSRW R5, R11, R11
+
+	// table[prevHash] = uint16(s-1)
+	MOVD R7, R3
+	SUB  R6, R3, R3
+	SUB  $1, R3, R3
+
+	MOVHU R3, 0(R17)(R11<<1)
+
+	// currHash := hash(uint32(x>>8), shift)
+	LSR  $8, R14, R14
+	MOVW R14, R11
+	MULW R16, R11, R11
+	LSRW R5, R11, R11
+
+	// candidate = int(table[currHash])
+	MOVHU 0(R17)(R11<<1), R15
+
+	// table[currHash] = uint16(s)
+	ADD   $1, R3, R3
+	MOVHU R3, 0(R17)(R11<<1)
+
+	// if uint32(x>>8) == load32(src, candidate) { continue }
+	MOVW (R6)(R15*1), R4
+	CMPW R4, R14
+	BEQ  inner1
+
+	// nextHash = hash(uint32(x>>16), shift)
+	LSR  $8, R14, R14
+	MOVW R14, R11
+	MULW R16, R11, R11
+	LSRW R5, R11, R11
+
+	// s++
+	ADD $1, R7, R7
+
+	// break out of the inner1 for loop, i.e. continue the outer loop.
+	B outer
+
+emitRemainder:
+	// if nextEmit < len(src) { etc }
+	MOVD src_len+32(FP), R3
+	ADD  R6, R3, R3
+	CMP  R3, R10
+	BEQ  encodeBlockEnd
+
+	// d += emitLiteral(dst[d:], src[nextEmit:])
+	//
+	// Push args.
+	MOVD R8, 8(RSP)
+	MOVD $0, 16(RSP)  // Unnecessary, as the callee ignores it, but conservative.
+	MOVD $0, 24(RSP)  // Unnecessary, as the callee ignores it, but conservative.
+	MOVD R10, 32(RSP)
+	SUB  R10, R3, R3
+	MOVD R3, 40(RSP)
+	MOVD R3, 48(RSP)  // Unnecessary, as the callee ignores it, but conservative.
+
+	// Spill local variables (registers) onto the stack; call; unspill.
+	MOVD R8, 88(RSP)
+	CALL ·emitLiteral(SB)
+	MOVD 88(RSP), R8
+
+	// Finish the "d +=" part of "d += emitLiteral(etc)".
+	MOVD 56(RSP), R1
+	ADD  R1, R8, R8
+
+encodeBlockEnd:
+	MOVD dst_base+0(FP), R3
+	SUB  R3, R8, R8
+	MOVD R8, d+48(FP)
+	RET
diff --git a/vendor/github.com/golang/snappy/encode_asm.go b/vendor/github.com/golang/snappy/encode_asm.go
new file mode 100644
index 0000000..107c1e7
--- /dev/null
+++ b/vendor/github.com/golang/snappy/encode_asm.go
@@ -0,0 +1,30 @@
+// Copyright 2016 The Snappy-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 !appengine
+// +build gc
+// +build !noasm
+// +build amd64 arm64
+
+package snappy
+
+// emitLiteral has the same semantics as in encode_other.go.
+//
+//go:noescape
+func emitLiteral(dst, lit []byte) int
+
+// emitCopy has the same semantics as in encode_other.go.
+//
+//go:noescape
+func emitCopy(dst []byte, offset, length int) int
+
+// extendMatch has the same semantics as in encode_other.go.
+//
+//go:noescape
+func extendMatch(src []byte, i, j int) int
+
+// encodeBlock has the same semantics as in encode_other.go.
+//
+//go:noescape
+func encodeBlock(dst, src []byte) (d int)
diff --git a/vendor/github.com/golang/snappy/encode_other.go b/vendor/github.com/golang/snappy/encode_other.go
index dbcae90..296d7f0 100644
--- a/vendor/github.com/golang/snappy/encode_other.go
+++ b/vendor/github.com/golang/snappy/encode_other.go
@@ -2,7 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !amd64 appengine !gc noasm
+// +build !amd64,!arm64 appengine !gc noasm
 
 package snappy
 
diff --git a/vendor/github.com/google/uuid/README.md b/vendor/github.com/google/uuid/README.md
index 9d92c11..f765a46 100644
--- a/vendor/github.com/google/uuid/README.md
+++ b/vendor/github.com/google/uuid/README.md
@@ -16,4 +16,4 @@
 
 Full `go doc` style documentation for the package can be viewed online without
 installing this package by using the GoDoc site here: 
-http://godoc.org/github.com/google/uuid
+http://pkg.go.dev/github.com/google/uuid
diff --git a/vendor/github.com/google/uuid/hash.go b/vendor/github.com/google/uuid/hash.go
index b174616..b404f4b 100644
--- a/vendor/github.com/google/uuid/hash.go
+++ b/vendor/github.com/google/uuid/hash.go
@@ -26,8 +26,8 @@
 // NewMD5 and NewSHA1.
 func NewHash(h hash.Hash, space UUID, data []byte, version int) UUID {
 	h.Reset()
-	h.Write(space[:])
-	h.Write(data)
+	h.Write(space[:]) //nolint:errcheck
+	h.Write(data)     //nolint:errcheck
 	s := h.Sum(nil)
 	var uuid UUID
 	copy(uuid[:], s)
diff --git a/vendor/github.com/google/uuid/marshal.go b/vendor/github.com/google/uuid/marshal.go
index 7f9e0c6..14bd340 100644
--- a/vendor/github.com/google/uuid/marshal.go
+++ b/vendor/github.com/google/uuid/marshal.go
@@ -16,10 +16,11 @@
 // UnmarshalText implements encoding.TextUnmarshaler.
 func (uuid *UUID) UnmarshalText(data []byte) error {
 	id, err := ParseBytes(data)
-	if err == nil {
-		*uuid = id
+	if err != nil {
+		return err
 	}
-	return err
+	*uuid = id
+	return nil
 }
 
 // MarshalBinary implements encoding.BinaryMarshaler.
diff --git a/vendor/github.com/google/uuid/null.go b/vendor/github.com/google/uuid/null.go
new file mode 100644
index 0000000..d7fcbf2
--- /dev/null
+++ b/vendor/github.com/google/uuid/null.go
@@ -0,0 +1,118 @@
+// Copyright 2021 Google Inc.  All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package uuid
+
+import (
+	"bytes"
+	"database/sql/driver"
+	"encoding/json"
+	"fmt"
+)
+
+var jsonNull = []byte("null")
+
+// NullUUID represents a UUID that may be null.
+// NullUUID implements the SQL driver.Scanner interface so
+// it can be used as a scan destination:
+//
+//  var u uuid.NullUUID
+//  err := db.QueryRow("SELECT name FROM foo WHERE id=?", id).Scan(&u)
+//  ...
+//  if u.Valid {
+//     // use u.UUID
+//  } else {
+//     // NULL value
+//  }
+//
+type NullUUID struct {
+	UUID  UUID
+	Valid bool // Valid is true if UUID is not NULL
+}
+
+// Scan implements the SQL driver.Scanner interface.
+func (nu *NullUUID) Scan(value interface{}) error {
+	if value == nil {
+		nu.UUID, nu.Valid = Nil, false
+		return nil
+	}
+
+	err := nu.UUID.Scan(value)
+	if err != nil {
+		nu.Valid = false
+		return err
+	}
+
+	nu.Valid = true
+	return nil
+}
+
+// Value implements the driver Valuer interface.
+func (nu NullUUID) Value() (driver.Value, error) {
+	if !nu.Valid {
+		return nil, nil
+	}
+	// Delegate to UUID Value function
+	return nu.UUID.Value()
+}
+
+// MarshalBinary implements encoding.BinaryMarshaler.
+func (nu NullUUID) MarshalBinary() ([]byte, error) {
+	if nu.Valid {
+		return nu.UUID[:], nil
+	}
+
+	return []byte(nil), nil
+}
+
+// UnmarshalBinary implements encoding.BinaryUnmarshaler.
+func (nu *NullUUID) UnmarshalBinary(data []byte) error {
+	if len(data) != 16 {
+		return fmt.Errorf("invalid UUID (got %d bytes)", len(data))
+	}
+	copy(nu.UUID[:], data)
+	nu.Valid = true
+	return nil
+}
+
+// MarshalText implements encoding.TextMarshaler.
+func (nu NullUUID) MarshalText() ([]byte, error) {
+	if nu.Valid {
+		return nu.UUID.MarshalText()
+	}
+
+	return jsonNull, nil
+}
+
+// UnmarshalText implements encoding.TextUnmarshaler.
+func (nu *NullUUID) UnmarshalText(data []byte) error {
+	id, err := ParseBytes(data)
+	if err != nil {
+		nu.Valid = false
+		return err
+	}
+	nu.UUID = id
+	nu.Valid = true
+	return nil
+}
+
+// MarshalJSON implements json.Marshaler.
+func (nu NullUUID) MarshalJSON() ([]byte, error) {
+	if nu.Valid {
+		return json.Marshal(nu.UUID)
+	}
+
+	return jsonNull, nil
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (nu *NullUUID) UnmarshalJSON(data []byte) error {
+	if bytes.Equal(data, jsonNull) {
+		*nu = NullUUID{}
+		return nil // valid null UUID
+	}
+	err := json.Unmarshal(data, &nu.UUID)
+	nu.Valid = err == nil
+	return err
+}
diff --git a/vendor/github.com/google/uuid/sql.go b/vendor/github.com/google/uuid/sql.go
index f326b54..2e02ec0 100644
--- a/vendor/github.com/google/uuid/sql.go
+++ b/vendor/github.com/google/uuid/sql.go
@@ -9,7 +9,7 @@
 	"fmt"
 )
 
-// Scan implements sql.Scanner so UUIDs can be read from databases transparently
+// Scan implements sql.Scanner so UUIDs can be read from databases transparently.
 // Currently, database types that map to string and []byte are supported. Please
 // consult database-specific driver documentation for matching types.
 func (uuid *UUID) Scan(src interface{}) error {
diff --git a/vendor/github.com/google/uuid/uuid.go b/vendor/github.com/google/uuid/uuid.go
index 524404c..a57207a 100644
--- a/vendor/github.com/google/uuid/uuid.go
+++ b/vendor/github.com/google/uuid/uuid.go
@@ -12,6 +12,7 @@
 	"fmt"
 	"io"
 	"strings"
+	"sync"
 )
 
 // A UUID is a 128 bit (16 byte) Universal Unique IDentifier as defined in RFC
@@ -33,7 +34,27 @@
 	Future                    // Reserved for future definition.
 )
 
-var rander = rand.Reader // random function
+const randPoolSize = 16 * 16
+
+var (
+	rander      = rand.Reader // random function
+	poolEnabled = false
+	poolMu      sync.Mutex
+	poolPos     = randPoolSize     // protected with poolMu
+	pool        [randPoolSize]byte // protected with poolMu
+)
+
+type invalidLengthError struct{ len int }
+
+func (err invalidLengthError) Error() string {
+	return fmt.Sprintf("invalid UUID length: %d", err.len)
+}
+
+// IsInvalidLengthError is matcher function for custom error invalidLengthError
+func IsInvalidLengthError(err error) bool {
+	_, ok := err.(invalidLengthError)
+	return ok
+}
 
 // Parse decodes s into a UUID or returns an error.  Both the standard UUID
 // forms of xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx and
@@ -68,7 +89,7 @@
 		}
 		return uuid, nil
 	default:
-		return uuid, fmt.Errorf("invalid UUID length: %d", len(s))
+		return uuid, invalidLengthError{len(s)}
 	}
 	// s is now at least 36 bytes long
 	// it must be of the form  xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx
@@ -112,7 +133,7 @@
 		}
 		return uuid, nil
 	default:
-		return uuid, fmt.Errorf("invalid UUID length: %d", len(b))
+		return uuid, invalidLengthError{len(b)}
 	}
 	// s is now at least 36 bytes long
 	// it must be of the form  xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx
@@ -243,3 +264,31 @@
 	}
 	rander = r
 }
+
+// EnableRandPool enables internal randomness pool used for Random
+// (Version 4) UUID generation. The pool contains random bytes read from
+// the random number generator on demand in batches. Enabling the pool
+// may improve the UUID generation throughput significantly.
+//
+// Since the pool is stored on the Go heap, this feature may be a bad fit
+// for security sensitive applications.
+//
+// Both EnableRandPool and DisableRandPool are not thread-safe and should
+// only be called when there is no possibility that New or any other
+// UUID Version 4 generation function will be called concurrently.
+func EnableRandPool() {
+	poolEnabled = true
+}
+
+// DisableRandPool disables the randomness pool if it was previously
+// enabled with EnableRandPool.
+//
+// Both EnableRandPool and DisableRandPool are not thread-safe and should
+// only be called when there is no possibility that New or any other
+// UUID Version 4 generation function will be called concurrently.
+func DisableRandPool() {
+	poolEnabled = false
+	defer poolMu.Unlock()
+	poolMu.Lock()
+	poolPos = randPoolSize
+}
diff --git a/vendor/github.com/google/uuid/version1.go b/vendor/github.com/google/uuid/version1.go
index 199a1ac..4631096 100644
--- a/vendor/github.com/google/uuid/version1.go
+++ b/vendor/github.com/google/uuid/version1.go
@@ -17,12 +17,6 @@
 //
 // In most cases, New should be used.
 func NewUUID() (UUID, error) {
-	nodeMu.Lock()
-	if nodeID == zeroID {
-		setNodeInterface("")
-	}
-	nodeMu.Unlock()
-
 	var uuid UUID
 	now, seq, err := GetTime()
 	if err != nil {
@@ -38,7 +32,13 @@
 	binary.BigEndian.PutUint16(uuid[4:], timeMid)
 	binary.BigEndian.PutUint16(uuid[6:], timeHi)
 	binary.BigEndian.PutUint16(uuid[8:], seq)
+
+	nodeMu.Lock()
+	if nodeID == zeroID {
+		setNodeInterface("")
+	}
 	copy(uuid[10:], nodeID[:])
+	nodeMu.Unlock()
 
 	return uuid, nil
 }
diff --git a/vendor/github.com/google/uuid/version4.go b/vendor/github.com/google/uuid/version4.go
index 84af91c..7697802 100644
--- a/vendor/github.com/google/uuid/version4.go
+++ b/vendor/github.com/google/uuid/version4.go
@@ -14,11 +14,21 @@
 	return Must(NewRandom())
 }
 
+// NewString creates a new random UUID and returns it as a string or panics.
+// NewString is equivalent to the expression
+//
+//    uuid.New().String()
+func NewString() string {
+	return Must(NewRandom()).String()
+}
+
 // NewRandom returns a Random (Version 4) UUID.
 //
 // The strength of the UUIDs is based on the strength of the crypto/rand
 // package.
 //
+// Uses the randomness pool if it was enabled with EnableRandPool.
+//
 // A note about uniqueness derived from the UUID Wikipedia entry:
 //
 //  Randomly generated UUIDs have 122 random bits.  One's annual risk of being
@@ -27,8 +37,16 @@
 //  equivalent to the odds of creating a few tens of trillions of UUIDs in a
 //  year and having one duplicate.
 func NewRandom() (UUID, error) {
+	if !poolEnabled {
+		return NewRandomFromReader(rander)
+	}
+	return newRandomFromPool()
+}
+
+// NewRandomFromReader returns a UUID based on bytes read from a given io.Reader.
+func NewRandomFromReader(r io.Reader) (UUID, error) {
 	var uuid UUID
-	_, err := io.ReadFull(rander, uuid[:])
+	_, err := io.ReadFull(r, uuid[:])
 	if err != nil {
 		return Nil, err
 	}
@@ -36,3 +54,23 @@
 	uuid[8] = (uuid[8] & 0x3f) | 0x80 // Variant is 10
 	return uuid, nil
 }
+
+func newRandomFromPool() (UUID, error) {
+	var uuid UUID
+	poolMu.Lock()
+	if poolPos == randPoolSize {
+		_, err := io.ReadFull(rander, pool[:])
+		if err != nil {
+			poolMu.Unlock()
+			return Nil, err
+		}
+		poolPos = 0
+	}
+	copy(uuid[:], pool[poolPos:(poolPos+16)])
+	poolPos += 16
+	poolMu.Unlock()
+
+	uuid[6] = (uuid[6] & 0x0f) | 0x40 // Version 4
+	uuid[8] = (uuid[8] & 0x3f) | 0x80 // Variant is 10
+	return uuid, nil
+}
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..826caa3
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.gitignore
@@ -0,0 +1,204 @@
+# 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/
+
+.envrc
\ No newline at end of file
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..fc198d8
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.travis.yml
@@ -0,0 +1,16 @@
+sudo: false
+language: go
+go:
+  - 1.13.x
+  - 1.14.x
+  - 1.15.x
+
+env:
+  global:
+    - GO111MODULE=on
+
+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..6eeb7e2
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CHANGELOG.md
@@ -0,0 +1,51 @@
+# 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
+
+- [#223](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/223) Add go-kit logging middleware - [adrien-f](https://github.com/adrien-f)
+
+## [v1.1.0] - 2019-09-12
+### Added
+- [#226](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/226) Support for go modules.
+- [#221](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/221) logging/zap add support for gRPC LoggerV2  - [kush-patel-hs](https://github.com/kush-patel-hs)
+- [#181](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/181) Rate Limit support - [ceshihao](https://github.com/ceshihao)
+- [#161](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/161) Retry on server stream call - [lonnblad](https://github.com/lonnblad)
+- [#152](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/152) Exponential backoff functions - [polyfloyd](https://github.com/polyfloyd)
+- [#147](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/147) Jaeger support for ctxtags extraction - [vporoshok](https://github.com/vporoshok)
+- [#184](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/184) ctxTags identifies if the call was sampled
+
+### Deprecated
+- [#201](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/201) `golang.org/x/net/context` - [houz42](https://github.com/houz42)
+- [#183](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/183) Documentation Generation in favour of <godoc.org>.
+
+### Fixed
+- [172](https://github.com/grpc-ecosystem/go-grpc-middleware/pull/172) Passing ctx into retry and recover - [johanbrandhorst](https://github.com/johanbrandhorst)
+- Numerious documentation fixes.
+
+## v1.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.1.0...HEAD 
+[v1.1.0]: https://github.com/grpc-ecosystem/go-grpc-middleware/compare/v1.0.0...v1.1.0 
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/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..814e155
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/README.md
@@ -0,0 +1,86 @@
+# 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](https://img.shields.io/badge/slack-%23grpc--middleware-brightgreen)](https://slack.com/share/IRUQCFC23/9Tm7hxRFVKKNoajQfMOcUiIk/enQtODc4ODI4NTIyMDcxLWM5NDA0ZTE4Njg5YjRjYWZkMTI5MzQwNDY3YzBjMzE1YzdjOGM5ZjI1NDNiM2JmNzI2YjM5ODE5OTRiNTEyOWE)
+
+[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 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_recovery.StreamServerInterceptor(),
+        grpc_ctxtags.StreamServerInterceptor(),
+        grpc_opentracing.StreamServerInterceptor(),
+        grpc_prometheus.StreamServerInterceptor,
+        grpc_zap.StreamServerInterceptor(zapLogger),
+        grpc_auth.StreamServerInterceptor(myAuthFunction),
+    )),
+    grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
+        grpc_recovery.UnaryServerInterceptor(),
+        grpc_ctxtags.UnaryServerInterceptor(),
+        grpc_opentracing.UnaryServerInterceptor(),
+        grpc_prometheus.UnaryServerInterceptor,
+        grpc_zap.UnaryServerInterceptor(zapLogger),
+        grpc_auth.UnaryServerInterceptor(myAuthFunction),
+    )),
+)
+```
+
+## 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.
+   * [`grpc_kit`](logging/kit/) - integration of [go-kit](https://github.com/go-kit/kit/tree/master/log) logging library into gRPC handlers.
+   * [`grpc_grpc_logsettable`](logging/settable/) - a wrapper around `grpclog.LoggerV2` that allows to replace loggers in runtime (thread-safe).
+
+#### 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
+   * [`ratelimit`](ratelimit/) - grpc rate limiting by your own limiter
+
+
+## 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..ea3738b
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go
@@ -0,0 +1,120 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+// gRPC Server Interceptor chaining middleware.
+
+package grpc_middleware
+
+import (
+	"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)
+
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		chainer := func(currentInter grpc.UnaryServerInterceptor, currentHandler grpc.UnaryHandler) grpc.UnaryHandler {
+			return func(currentCtx context.Context, currentReq interface{}) (interface{}, error) {
+				return currentInter(currentCtx, currentReq, info, currentHandler)
+			}
+		}
+
+		chainedHandler := handler
+		for i := n - 1; i >= 0; i-- {
+			chainedHandler = chainer(interceptors[i], chainedHandler)
+		}
+
+		return chainedHandler(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)
+
+	return func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		chainer := func(currentInter grpc.StreamServerInterceptor, currentHandler grpc.StreamHandler) grpc.StreamHandler {
+			return func(currentSrv interface{}, currentStream grpc.ServerStream) error {
+				return currentInter(currentSrv, currentStream, info, currentHandler)
+			}
+		}
+
+		chainedHandler := handler
+		for i := n - 1; i >= 0; i-- {
+			chainedHandler = chainer(interceptors[i], chainedHandler)
+		}
+
+		return chainedHandler(srv, ss)
+	}
+}
+
+// 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)
+
+	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		chainer := func(currentInter grpc.UnaryClientInterceptor, currentInvoker grpc.UnaryInvoker) grpc.UnaryInvoker {
+			return func(currentCtx context.Context, currentMethod string, currentReq, currentRepl interface{}, currentConn *grpc.ClientConn, currentOpts ...grpc.CallOption) error {
+				return currentInter(currentCtx, currentMethod, currentReq, currentRepl, currentConn, currentInvoker, currentOpts...)
+			}
+		}
+
+		chainedInvoker := invoker
+		for i := n - 1; i >= 0; i-- {
+			chainedInvoker = chainer(interceptors[i], chainedInvoker)
+		}
+
+		return chainedInvoker(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)
+
+	return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		chainer := func(currentInter grpc.StreamClientInterceptor, currentStreamer grpc.Streamer) grpc.Streamer {
+			return func(currentCtx context.Context, currentDesc *grpc.StreamDesc, currentConn *grpc.ClientConn, currentMethod string, currentOpts ...grpc.CallOption) (grpc.ClientStream, error) {
+				return currentInter(currentCtx, currentDesc, currentConn, currentMethod, currentStreamer, currentOpts...)
+			}
+		}
+
+		chainedStreamer := streamer
+		for i := n - 1; i >= 0; i-- {
+			chainedStreamer = chainer(interceptors[i], chainedStreamer)
+		}
+
+		return chainedStreamer(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..718e100
--- /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, newStream)
+	}
+*/
+package grpc_middleware
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.mod b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.mod
new file mode 100644
index 0000000..7dc62e5
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.mod
@@ -0,0 +1,22 @@
+module github.com/grpc-ecosystem/go-grpc-middleware
+
+require (
+	github.com/go-kit/kit v0.9.0
+	github.com/go-logfmt/logfmt v0.4.0 // indirect
+	github.com/go-stack/stack v1.8.0 // indirect
+	github.com/gogo/protobuf v1.3.2
+	github.com/golang/protobuf v1.3.3
+	github.com/opentracing/opentracing-go v1.1.0
+	github.com/pkg/errors v0.8.1 // indirect
+	github.com/sirupsen/logrus v1.4.2
+	github.com/stretchr/testify v1.4.0
+	go.uber.org/atomic v1.4.0 // indirect
+	go.uber.org/multierr v1.1.0 // indirect
+	go.uber.org/zap v1.10.0
+	golang.org/x/net v0.0.0-20201021035429-f5854403a974
+	golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be
+	google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215 // indirect
+	google.golang.org/grpc v1.29.1
+)
+
+go 1.14
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.sum b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.sum
new file mode 100644
index 0000000..ee522cd
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/go.sum
@@ -0,0 +1,122 @@
+cloud.google.com/go v0.26.0 h1:e0WKqKTd5BnrG8aKH3J3h+QvEIQtSUcf2n5UZ5ZgLtQ=
+cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
+github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
+github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
+github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
+github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
+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/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
+github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
+github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
+github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
+github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk=
+github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-logfmt/logfmt v0.4.0 h1:MP4Eh7ZCb31lleYCFuwm0oe4/YGak+5l1vA2NOE80nA=
+github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
+github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk=
+github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
+github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
+github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
+github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
+github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
+github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.3 h1:gyjaxf+svBWX08ZjK86iN9geUJF0H6gp2IRKX6Nf6/I=
+github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
+github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ=
+github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
+github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
+github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515 h1:T+h1c/A9Gawja4Y9mFVWj2vyii2bbUNDw3kt9VxK2EY=
+github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
+github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU=
+github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=
+github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+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/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4=
+github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
+github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
+github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
+go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU=
+go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI=
+go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
+go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM=
+go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
+golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
+golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
+golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
+golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
+golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20201021035429-f5854403a974 h1:IX6qOQeG5uLjB/hjjwjedwfjND0hgjPMMyO1RoIXQNI=
+golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
+golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be h1:vEDujvNQGv4jgYKudGeI/+DAX4Jffq6hpD55MmoEvKs=
+golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
+golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f h1:+Nyd8tzPX9R7BWHguqsrbFdRx3WQ/1ib8I44HXV5yTA=
+golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k=
+golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
+golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
+golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
+golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
+golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
+google.golang.org/appengine v1.4.0 h1:/wp5JvzpHIxhs/dumFmF7BXTf3Z+dd4uXta4kVyO508=
+google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
+google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
+google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
+google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215 h1:0Uz5jLJQioKgVozXa1gzGbzYxbb/rhQEVvSWxzw5oUs=
+google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
+google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
+google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
+google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
+google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4=
+google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
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..b18d2d2
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/makefile
@@ -0,0 +1,17 @@
+SHELL=/bin/bash
+
+GOFILES_NOVENDOR = $(shell go list ./... | grep -v /vendor/)
+
+all: vet fmt test
+
+fmt:
+	go fmt $(GOFILES_NOVENDOR)
+
+vet:
+	# do not check lostcancel, they are intentional.
+	go vet -lostcancel=false $(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/tags/context.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/context.go
new file mode 100644
index 0000000..0da1658
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/context.go
@@ -0,0 +1,78 @@
+package grpc_ctxtags
+
+import (
+	"context"
+)
+
+type ctxMarker struct{}
+
+var (
+	// ctxMarkerKey is the Context value marker used by *all* logging middleware.
+	// The logging middleware object must interf
+	ctxMarkerKey = &ctxMarker{}
+	// NoopTags is a trivial, minimum overhead implementation of Tags for which all operations are no-ops.
+	NoopTags = &noopTags{}
+)
+
+// Tags is the interface used for storing request tags between Context calls.
+// The default implementation is *not* thread safe, and should be handled only in the context of the request.
+type Tags interface {
+	// Set sets the given key in the metadata tags.
+	Set(key string, value interface{}) Tags
+	// Has checks if the given key exists.
+	Has(key string) bool
+	// Values returns a map of key to values.
+	// Do not modify the underlying map, please use Set instead.
+	Values() map[string]interface{}
+}
+
+type mapTags struct {
+	values map[string]interface{}
+}
+
+func (t *mapTags) Set(key string, value interface{}) Tags {
+	t.values[key] = value
+	return t
+}
+
+func (t *mapTags) Has(key string) bool {
+	_, ok := t.values[key]
+	return ok
+}
+
+func (t *mapTags) Values() map[string]interface{} {
+	return t.values
+}
+
+type noopTags struct{}
+
+func (t *noopTags) Set(key string, value interface{}) Tags {
+	return t
+}
+
+func (t *noopTags) Has(key string) bool {
+	return false
+}
+
+func (t *noopTags) Values() map[string]interface{} {
+	return nil
+}
+
+// Extracts returns a pre-existing Tags object in the Context.
+// If the context wasn't set in a tag interceptor, a no-op Tag storage is returned that will *not* be propagated in context.
+func Extract(ctx context.Context) Tags {
+	t, ok := ctx.Value(ctxMarkerKey).(Tags)
+	if !ok {
+		return NoopTags
+	}
+
+	return t
+}
+
+func SetInContext(ctx context.Context, tags Tags) context.Context {
+	return context.WithValue(ctx, ctxMarkerKey, tags)
+}
+
+func NewTags() Tags {
+	return &mapTags{values: make(map[string]interface{})}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/doc.go
new file mode 100644
index 0000000..960638d
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/doc.go
@@ -0,0 +1,22 @@
+/*
+`grpc_ctxtags` adds a Tag object to the context that can be used by other middleware to add context about a request.
+
+Request Context Tags
+
+Tags describe information about the request, and can be set and used by other middleware, or handlers. Tags are used
+for logging and tracing of requests. Tags are populated both upwards, *and* downwards in the interceptor-handler stack.
+
+You can automatically extract tags (in `grpc.request.<field_name>`) from request payloads.
+
+For unary and server-streaming methods, pass in the `WithFieldExtractor` option. For client-streams and bidirectional-streams, you can
+use `WithFieldExtractorForInitialReq` which will extract the tags from the first message passed from client to server.
+Note the tags will not be modified for subsequent requests, so this option only makes sense when the initial message
+establishes the meta-data for the stream.
+
+If a user doesn't use the interceptors that initialize the `Tags` object, all operations following from an `Extract(ctx)`
+will be no-ops. This is to ensure that code doesn't panic if the interceptors weren't used.
+
+Tags fields are typed, and shallow and should follow the OpenTracing semantics convention:
+https://github.com/opentracing/specification/blob/master/semantic_conventions.md
+*/
+package grpc_ctxtags
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/fieldextractor.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/fieldextractor.go
new file mode 100644
index 0000000..a4073ab
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/fieldextractor.go
@@ -0,0 +1,85 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_ctxtags
+
+import (
+	"reflect"
+)
+
+// RequestFieldExtractorFunc is a user-provided function that extracts field information from a gRPC request.
+// It is called from tags middleware on arrival of unary request or a server-stream request.
+// Keys and values will be added to the context tags of the request. If there are no fields, you should return a nil.
+type RequestFieldExtractorFunc func(fullMethod string, req interface{}) map[string]interface{}
+
+type requestFieldsExtractor interface {
+	// ExtractRequestFields is a method declared on a Protobuf message that extracts fields from the interface.
+	// The values from the extracted fields should be set in the appendToMap, in order to avoid allocations.
+	ExtractRequestFields(appendToMap map[string]interface{})
+}
+
+// CodeGenRequestFieldExtractor is a function that relies on code-generated functions that export log fields from requests.
+// These are usually coming from a protoc-plugin that generates additional information based on custom field options.
+func CodeGenRequestFieldExtractor(fullMethod string, req interface{}) map[string]interface{} {
+	if ext, ok := req.(requestFieldsExtractor); ok {
+		retMap := make(map[string]interface{})
+		ext.ExtractRequestFields(retMap)
+		if len(retMap) == 0 {
+			return nil
+		}
+		return retMap
+	}
+	return nil
+}
+
+// TagBasedRequestFieldExtractor is a function that relies on Go struct tags to export log fields from requests.
+// These are usually coming from a protoc-plugin, such as Gogo protobuf.
+//
+//  message Metadata {
+//     repeated string tags = 1 [ (gogoproto.moretags) = "log_field:\"meta_tags\"" ];
+//  }
+//
+// The tagName is configurable using the tagName variable. Here it would be "log_field".
+func TagBasedRequestFieldExtractor(tagName string) RequestFieldExtractorFunc {
+	return func(fullMethod string, req interface{}) map[string]interface{} {
+		retMap := make(map[string]interface{})
+		reflectMessageTags(req, retMap, tagName)
+		if len(retMap) == 0 {
+			return nil
+		}
+		return retMap
+	}
+}
+
+func reflectMessageTags(msg interface{}, existingMap map[string]interface{}, tagName string) {
+	v := reflect.ValueOf(msg)
+	// Only deal with pointers to structs.
+	if v.Kind() != reflect.Ptr || v.Elem().Kind() != reflect.Struct {
+		return
+	}
+	// Deref the pointer get to the struct.
+	v = v.Elem()
+	t := v.Type()
+	for i := 0; i < v.NumField(); i++ {
+		field := v.Field(i)
+		kind := field.Kind()
+		// Only recurse down direct pointers, which should only be to nested structs.
+		if (kind == reflect.Ptr || kind == reflect.Interface) && field.CanInterface() {
+			reflectMessageTags(field.Interface(), existingMap, tagName)
+		}
+		// In case of arrays/slices (repeated fields) go down to the concrete type.
+		if kind == reflect.Array || kind == reflect.Slice {
+			if field.Len() == 0 {
+				continue
+			}
+			kind = field.Index(0).Kind()
+		}
+		// Only be interested in
+		if (kind >= reflect.Bool && kind <= reflect.Float64) || kind == reflect.String {
+			if tag := t.Field(i).Tag.Get(tagName); tag != "" {
+				existingMap[tag] = field.Interface()
+			}
+		}
+	}
+	return
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/interceptors.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/interceptors.go
new file mode 100644
index 0000000..a7ced60
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/interceptors.go
@@ -0,0 +1,85 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_ctxtags
+
+import (
+	"context"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/peer"
+
+	"github.com/grpc-ecosystem/go-grpc-middleware"
+)
+
+// UnaryServerInterceptor returns a new unary server interceptors that sets the values for request tags.
+func UnaryServerInterceptor(opts ...Option) grpc.UnaryServerInterceptor {
+	o := evaluateOptions(opts)
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		newCtx := newTagsForCtx(ctx)
+		if o.requestFieldsFunc != nil {
+			setRequestFieldTags(newCtx, o.requestFieldsFunc, info.FullMethod, req)
+		}
+		return handler(newCtx, req)
+	}
+}
+
+// StreamServerInterceptor returns a new streaming server interceptor that sets the values for request tags.
+func StreamServerInterceptor(opts ...Option) grpc.StreamServerInterceptor {
+	o := evaluateOptions(opts)
+	return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		newCtx := newTagsForCtx(stream.Context())
+		if o.requestFieldsFunc == nil {
+			// Short-circuit, don't do the expensive bit of allocating a wrappedStream.
+			wrappedStream := grpc_middleware.WrapServerStream(stream)
+			wrappedStream.WrappedContext = newCtx
+			return handler(srv, wrappedStream)
+		}
+		wrapped := &wrappedStream{stream, info, o, newCtx, true}
+		err := handler(srv, wrapped)
+		return err
+	}
+}
+
+// wrappedStream is a thin wrapper around grpc.ServerStream that allows modifying context and extracts log fields from the initial message.
+type wrappedStream struct {
+	grpc.ServerStream
+	info *grpc.StreamServerInfo
+	opts *options
+	// WrappedContext is the wrapper's own Context. You can assign it.
+	WrappedContext context.Context
+	initial        bool
+}
+
+// Context returns the wrapper's WrappedContext, overwriting the nested grpc.ServerStream.Context()
+func (w *wrappedStream) Context() context.Context {
+	return w.WrappedContext
+}
+
+func (w *wrappedStream) RecvMsg(m interface{}) error {
+	err := w.ServerStream.RecvMsg(m)
+	// We only do log fields extraction on the single-request of a server-side stream.
+	if !w.info.IsClientStream || w.opts.requestFieldsFromInitial && w.initial {
+		w.initial = false
+
+		setRequestFieldTags(w.Context(), w.opts.requestFieldsFunc, w.info.FullMethod, m)
+	}
+	return err
+}
+
+func newTagsForCtx(ctx context.Context) context.Context {
+	t := NewTags()
+	if peer, ok := peer.FromContext(ctx); ok {
+		t.Set("peer.address", peer.Addr.String())
+	}
+	return SetInContext(ctx, t)
+}
+
+func setRequestFieldTags(ctx context.Context, f RequestFieldExtractorFunc, fullMethodName string, req interface{}) {
+	if valMap := f(fullMethodName, req); valMap != nil {
+		t := Extract(ctx)
+		for k, v := range valMap {
+			t.Set("grpc.request."+k, v)
+		}
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/options.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/options.go
new file mode 100644
index 0000000..952775f
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tags/options.go
@@ -0,0 +1,44 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_ctxtags
+
+var (
+	defaultOptions = &options{
+		requestFieldsFunc: nil,
+	}
+)
+
+type options struct {
+	requestFieldsFunc        RequestFieldExtractorFunc
+	requestFieldsFromInitial bool
+}
+
+func evaluateOptions(opts []Option) *options {
+	optCopy := &options{}
+	*optCopy = *defaultOptions
+	for _, o := range opts {
+		o(optCopy)
+	}
+	return optCopy
+}
+
+type Option func(*options)
+
+// WithFieldExtractor customizes the function for extracting log fields from protobuf messages, for
+// unary and server-streamed methods only.
+func WithFieldExtractor(f RequestFieldExtractorFunc) Option {
+	return func(o *options) {
+		o.requestFieldsFunc = f
+	}
+}
+
+// WithFieldExtractorForInitialReq customizes the function for extracting log fields from protobuf messages,
+// for all unary and streaming methods. For client-streams and bidirectional-streams, the tags will be
+// extracted from the first message from the client.
+func WithFieldExtractorForInitialReq(f RequestFieldExtractorFunc) Option {
+	return func(o *options) {
+		o.requestFieldsFunc = f
+		o.requestFieldsFromInitial = true
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/client_interceptors.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/client_interceptors.go
new file mode 100644
index 0000000..2e9cafd
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/client_interceptors.go
@@ -0,0 +1,143 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_opentracing
+
+import (
+	"context"
+	"io"
+	"sync"
+
+	"github.com/grpc-ecosystem/go-grpc-middleware/util/metautils"
+	opentracing "github.com/opentracing/opentracing-go"
+	"github.com/opentracing/opentracing-go/ext"
+	"github.com/opentracing/opentracing-go/log"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/metadata"
+)
+
+// UnaryClientInterceptor returns a new unary client interceptor for OpenTracing.
+func UnaryClientInterceptor(opts ...Option) grpc.UnaryClientInterceptor {
+	o := evaluateOptions(opts)
+	return func(parentCtx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		if o.filterOutFunc != nil && !o.filterOutFunc(parentCtx, method) {
+			return invoker(parentCtx, method, req, reply, cc, opts...)
+		}
+		newCtx, clientSpan := newClientSpanFromContext(parentCtx, o.tracer, method)
+		if o.unaryRequestHandlerFunc != nil {
+			o.unaryRequestHandlerFunc(clientSpan, req)
+		}
+		err := invoker(newCtx, method, req, reply, cc, opts...)
+		finishClientSpan(clientSpan, err)
+		return err
+	}
+}
+
+// StreamClientInterceptor returns a new streaming client interceptor for OpenTracing.
+func StreamClientInterceptor(opts ...Option) grpc.StreamClientInterceptor {
+	o := evaluateOptions(opts)
+	return func(parentCtx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		if o.filterOutFunc != nil && !o.filterOutFunc(parentCtx, method) {
+			return streamer(parentCtx, desc, cc, method, opts...)
+		}
+		newCtx, clientSpan := newClientSpanFromContext(parentCtx, o.tracer, method)
+		clientStream, err := streamer(newCtx, desc, cc, method, opts...)
+		if err != nil {
+			finishClientSpan(clientSpan, err)
+			return nil, err
+		}
+		return &tracedClientStream{ClientStream: clientStream, clientSpan: clientSpan}, nil
+	}
+}
+
+// type serverStreamingRetryingStream is the implementation of grpc.ClientStream that acts as a
+// proxy to the underlying call. If any of the RecvMsg() calls fail, it will try to reestablish
+// a new ClientStream according to the retry policy.
+type tracedClientStream struct {
+	grpc.ClientStream
+	mu              sync.Mutex
+	alreadyFinished bool
+	clientSpan      opentracing.Span
+}
+
+func (s *tracedClientStream) Header() (metadata.MD, error) {
+	h, err := s.ClientStream.Header()
+	if err != nil {
+		s.finishClientSpan(err)
+	}
+	return h, err
+}
+
+func (s *tracedClientStream) SendMsg(m interface{}) error {
+	err := s.ClientStream.SendMsg(m)
+	if err != nil {
+		s.finishClientSpan(err)
+	}
+	return err
+}
+
+func (s *tracedClientStream) CloseSend() error {
+	err := s.ClientStream.CloseSend()
+	s.finishClientSpan(err)
+	return err
+}
+
+func (s *tracedClientStream) RecvMsg(m interface{}) error {
+	err := s.ClientStream.RecvMsg(m)
+	if err != nil {
+		s.finishClientSpan(err)
+	}
+	return err
+}
+
+func (s *tracedClientStream) finishClientSpan(err error) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	if !s.alreadyFinished {
+		finishClientSpan(s.clientSpan, err)
+		s.alreadyFinished = true
+	}
+}
+
+// ClientAddContextTags returns a context with specified opentracing tags, which
+// are used by UnaryClientInterceptor/StreamClientInterceptor when creating a
+// new span.
+func ClientAddContextTags(ctx context.Context, tags opentracing.Tags) context.Context {
+	return context.WithValue(ctx, clientSpanTagKey{}, tags)
+}
+
+type clientSpanTagKey struct{}
+
+func newClientSpanFromContext(ctx context.Context, tracer opentracing.Tracer, fullMethodName string) (context.Context, opentracing.Span) {
+	var parentSpanCtx opentracing.SpanContext
+	if parent := opentracing.SpanFromContext(ctx); parent != nil {
+		parentSpanCtx = parent.Context()
+	}
+	opts := []opentracing.StartSpanOption{
+		opentracing.ChildOf(parentSpanCtx),
+		ext.SpanKindRPCClient,
+		grpcTag,
+	}
+	if tagx := ctx.Value(clientSpanTagKey{}); tagx != nil {
+		if opt, ok := tagx.(opentracing.StartSpanOption); ok {
+			opts = append(opts, opt)
+		}
+	}
+	clientSpan := tracer.StartSpan(fullMethodName, opts...)
+	// Make sure we add this to the metadata of the call, so it gets propagated:
+	md := metautils.ExtractOutgoing(ctx).Clone()
+	if err := tracer.Inject(clientSpan.Context(), opentracing.HTTPHeaders, metadataTextMap(md)); err != nil {
+		grpclog.Infof("grpc_opentracing: failed serializing trace information: %v", err)
+	}
+	ctxWithMetadata := md.ToOutgoing(ctx)
+	return opentracing.ContextWithSpan(ctxWithMetadata, clientSpan), clientSpan
+}
+
+func finishClientSpan(clientSpan opentracing.Span, err error) {
+	if err != nil && err != io.EOF {
+		ext.Error.Set(clientSpan, true)
+		clientSpan.LogFields(log.String("event", "error"), log.String("message", err.Error()))
+	}
+	clientSpan.Finish()
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/doc.go
new file mode 100644
index 0000000..7a58efc
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/doc.go
@@ -0,0 +1,22 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+/*
+`grpc_opentracing` adds OpenTracing
+
+OpenTracing Interceptors
+
+These are both client-side and server-side interceptors for OpenTracing. They are a provider-agnostic, with backends
+such as Zipkin, or Google Stackdriver Trace.
+
+For a service that sends out requests and receives requests, you *need* to use both, otherwise downstream requests will
+not have the appropriate requests propagated.
+
+All server-side spans are tagged with grpc_ctxtags information.
+
+For more information see:
+http://opentracing.io/documentation/
+https://github.com/opentracing/specification/blob/master/semantic_conventions.md
+
+*/
+package grpc_opentracing
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/id_extract.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/id_extract.go
new file mode 100644
index 0000000..bc7302e
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/id_extract.go
@@ -0,0 +1,82 @@
+package grpc_opentracing
+
+import (
+	"strings"
+
+	grpc_ctxtags "github.com/grpc-ecosystem/go-grpc-middleware/tags"
+	opentracing "github.com/opentracing/opentracing-go"
+	"google.golang.org/grpc/grpclog"
+)
+
+const (
+	TagTraceId           = "trace.traceid"
+	TagSpanId            = "trace.spanid"
+	TagSampled           = "trace.sampled"
+	jaegerNotSampledFlag = "0"
+)
+
+// injectOpentracingIdsToTags writes trace data to ctxtags.
+// This is done in an incredibly hacky way, because the public-facing interface of opentracing doesn't give access to
+// the TraceId and SpanId of the SpanContext. Only the Tracer's Inject/Extract methods know what these are.
+// Most tracers have them encoded as keys with 'traceid' and 'spanid':
+// https://github.com/openzipkin/zipkin-go-opentracing/blob/594640b9ef7e5c994e8d9499359d693c032d738c/propagation_ot.go#L29
+// https://github.com/opentracing/basictracer-go/blob/1b32af207119a14b1b231d451df3ed04a72efebf/propagation_ot.go#L26
+// Jaeger from Uber use one-key schema with next format '{trace-id}:{span-id}:{parent-span-id}:{flags}'
+// https://www.jaegertracing.io/docs/client-libraries/#trace-span-identity
+// Datadog uses keys ending with 'trace-id' and 'parent-id' (for span) by default:
+// https://github.com/DataDog/dd-trace-go/blob/v1/ddtrace/tracer/textmap.go#L77
+func injectOpentracingIdsToTags(traceHeaderName string, span opentracing.Span, tags grpc_ctxtags.Tags) {
+	if err := span.Tracer().Inject(span.Context(), opentracing.HTTPHeaders,
+		&tagsCarrier{Tags: tags, traceHeaderName: traceHeaderName}); err != nil {
+		grpclog.Infof("grpc_opentracing: failed extracting trace info into ctx %v", err)
+	}
+}
+
+// tagsCarrier is a really hacky way of
+type tagsCarrier struct {
+	grpc_ctxtags.Tags
+	traceHeaderName string
+}
+
+func (t *tagsCarrier) Set(key, val string) {
+	key = strings.ToLower(key)
+
+	if key == t.traceHeaderName {
+		parts := strings.Split(val, ":")
+		if len(parts) == 4 {
+			t.Tags.Set(TagTraceId, parts[0])
+			t.Tags.Set(TagSpanId, parts[1])
+
+			if parts[3] != jaegerNotSampledFlag {
+				t.Tags.Set(TagSampled, "true")
+			} else {
+				t.Tags.Set(TagSampled, "false")
+			}
+
+			return
+		}
+	}
+
+	if strings.Contains(key, "traceid") {
+		t.Tags.Set(TagTraceId, val) // this will most likely be base-16 (hex) encoded
+	}
+
+	if strings.Contains(key, "spanid") && !strings.Contains(strings.ToLower(key), "parent") {
+		t.Tags.Set(TagSpanId, val) // this will most likely be base-16 (hex) encoded
+	}
+
+	if strings.Contains(key, "sampled") {
+		switch val {
+		case "true", "false":
+			t.Tags.Set(TagSampled, val)
+		}
+	}
+
+	if strings.HasSuffix(key, "trace-id") {
+		t.Tags.Set(TagTraceId, val)
+	}
+
+	if strings.HasSuffix(key, "parent-id") {
+		t.Tags.Set(TagSpanId, val)
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/metadata.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/metadata.go
new file mode 100644
index 0000000..3649fb5
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/metadata.go
@@ -0,0 +1,50 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_opentracing
+
+import (
+	"encoding/base64"
+	"strings"
+
+	"google.golang.org/grpc/metadata"
+)
+
+const (
+	binHdrSuffix = "-bin"
+)
+
+// metadataTextMap extends a metadata.MD to be an opentracing textmap
+type metadataTextMap metadata.MD
+
+// Set is a opentracing.TextMapReader interface that extracts values.
+func (m metadataTextMap) Set(key, val string) {
+	// gRPC allows for complex binary values to be written.
+	encodedKey, encodedVal := encodeKeyValue(key, val)
+	// The metadata object is a multimap, and previous values may exist, but for opentracing headers, we do not append
+	// we just override.
+	m[encodedKey] = []string{encodedVal}
+}
+
+// ForeachKey is a opentracing.TextMapReader interface that extracts values.
+func (m metadataTextMap) ForeachKey(callback func(key, val string) error) error {
+	for k, vv := range m {
+		for _, v := range vv {
+			if err := callback(k, v); err != nil {
+				return err
+			}
+		}
+	}
+	return nil
+}
+
+// encodeKeyValue encodes key and value qualified for transmission via gRPC.
+// note: copy pasted from private values of grpc.metadata
+func encodeKeyValue(k, v string) (string, string) {
+	k = strings.ToLower(k)
+	if strings.HasSuffix(k, binHdrSuffix) {
+		val := base64.StdEncoding.EncodeToString([]byte(v))
+		v = string(val)
+	}
+	return k, v
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/options.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/options.go
new file mode 100644
index 0000000..430fe56
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/options.go
@@ -0,0 +1,89 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_opentracing
+
+import (
+	"context"
+
+	"github.com/opentracing/opentracing-go"
+)
+
+var (
+	defaultOptions = &options{
+		filterOutFunc: nil,
+		tracer:        nil,
+	}
+)
+
+// FilterFunc allows users to provide a function that filters out certain methods from being traced.
+//
+// If it returns false, the given request will not be traced.
+type FilterFunc func(ctx context.Context, fullMethodName string) bool
+
+// UnaryRequestHandlerFunc is a custom request handler
+type UnaryRequestHandlerFunc func(span opentracing.Span, req interface{})
+
+// OpNameFunc is a func that allows custom operation names instead of the gRPC method.
+type OpNameFunc func(method string) string
+
+type options struct {
+	filterOutFunc           FilterFunc
+	tracer                  opentracing.Tracer
+	traceHeaderName         string
+	unaryRequestHandlerFunc UnaryRequestHandlerFunc
+	opNameFunc              OpNameFunc
+}
+
+func evaluateOptions(opts []Option) *options {
+	optCopy := &options{}
+	*optCopy = *defaultOptions
+	for _, o := range opts {
+		o(optCopy)
+	}
+	if optCopy.tracer == nil {
+		optCopy.tracer = opentracing.GlobalTracer()
+	}
+	if optCopy.traceHeaderName == "" {
+		optCopy.traceHeaderName = "uber-trace-id"
+	}
+	return optCopy
+}
+
+type Option func(*options)
+
+// WithFilterFunc customizes the function used for deciding whether a given call is traced or not.
+func WithFilterFunc(f FilterFunc) Option {
+	return func(o *options) {
+		o.filterOutFunc = f
+	}
+}
+
+// WithTraceHeaderName customizes the trace header name where trace metadata passed with requests.
+// Default one is `uber-trace-id`
+func WithTraceHeaderName(name string) Option {
+	return func(o *options) {
+		o.traceHeaderName = name
+	}
+}
+
+// WithTracer sets a custom tracer to be used for this middleware, otherwise the opentracing.GlobalTracer is used.
+func WithTracer(tracer opentracing.Tracer) Option {
+	return func(o *options) {
+		o.tracer = tracer
+	}
+}
+
+// WithUnaryRequestHandlerFunc sets a custom handler for the request
+func WithUnaryRequestHandlerFunc(f UnaryRequestHandlerFunc) Option {
+	return func(o *options) {
+		o.unaryRequestHandlerFunc = f
+	}
+}
+
+// WithOpName customizes the trace Operation name
+func WithOpName(f OpNameFunc) Option {
+	return func(o *options) {
+		o.opNameFunc = f
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/server_interceptors.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/server_interceptors.go
new file mode 100644
index 0000000..186b108
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/server_interceptors.go
@@ -0,0 +1,98 @@
+// Copyright 2017 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_opentracing
+
+import (
+	"context"
+
+	"github.com/grpc-ecosystem/go-grpc-middleware"
+	"github.com/grpc-ecosystem/go-grpc-middleware/tags"
+	"github.com/grpc-ecosystem/go-grpc-middleware/util/metautils"
+	"github.com/opentracing/opentracing-go"
+	"github.com/opentracing/opentracing-go/ext"
+	"github.com/opentracing/opentracing-go/log"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/grpclog"
+)
+
+var (
+	grpcTag = opentracing.Tag{Key: string(ext.Component), Value: "gRPC"}
+)
+
+// UnaryServerInterceptor returns a new unary server interceptor for OpenTracing.
+func UnaryServerInterceptor(opts ...Option) grpc.UnaryServerInterceptor {
+	o := evaluateOptions(opts)
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		if o.filterOutFunc != nil && !o.filterOutFunc(ctx, info.FullMethod) {
+			return handler(ctx, req)
+		}
+		opName := info.FullMethod
+		if o.opNameFunc != nil {
+			opName = o.opNameFunc(info.FullMethod)
+		}
+		newCtx, serverSpan := newServerSpanFromInbound(ctx, o.tracer, o.traceHeaderName, opName)
+		if o.unaryRequestHandlerFunc != nil {
+			o.unaryRequestHandlerFunc(serverSpan, req)
+		}
+		resp, err := handler(newCtx, req)
+		finishServerSpan(ctx, serverSpan, err)
+		return resp, err
+	}
+}
+
+// StreamServerInterceptor returns a new streaming server interceptor for OpenTracing.
+func StreamServerInterceptor(opts ...Option) grpc.StreamServerInterceptor {
+	o := evaluateOptions(opts)
+	return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		if o.filterOutFunc != nil && !o.filterOutFunc(stream.Context(), info.FullMethod) {
+			return handler(srv, stream)
+		}
+		opName := info.FullMethod
+		if o.opNameFunc != nil {
+			opName = o.opNameFunc(info.FullMethod)
+		}
+		newCtx, serverSpan := newServerSpanFromInbound(stream.Context(), o.tracer, o.traceHeaderName, opName)
+		wrappedStream := grpc_middleware.WrapServerStream(stream)
+		wrappedStream.WrappedContext = newCtx
+		err := handler(srv, wrappedStream)
+		finishServerSpan(newCtx, serverSpan, err)
+		return err
+	}
+}
+
+func newServerSpanFromInbound(ctx context.Context, tracer opentracing.Tracer, traceHeaderName, opName string) (context.Context, opentracing.Span) {
+	md := metautils.ExtractIncoming(ctx)
+	parentSpanContext, err := tracer.Extract(opentracing.HTTPHeaders, metadataTextMap(md))
+	if err != nil && err != opentracing.ErrSpanContextNotFound {
+		grpclog.Infof("grpc_opentracing: failed parsing trace information: %v", err)
+	}
+
+	serverSpan := tracer.StartSpan(
+		opName,
+		// this is magical, it attaches the new span to the parent parentSpanContext, and creates an unparented one if empty.
+		ext.RPCServerOption(parentSpanContext),
+		grpcTag,
+	)
+
+	injectOpentracingIdsToTags(traceHeaderName, serverSpan, grpc_ctxtags.Extract(ctx))
+	return opentracing.ContextWithSpan(ctx, serverSpan), serverSpan
+}
+
+func finishServerSpan(ctx context.Context, serverSpan opentracing.Span, err error) {
+	// Log context information
+	tags := grpc_ctxtags.Extract(ctx)
+	for k, v := range tags.Values() {
+		// Don't tag errors, log them instead.
+		if vErr, ok := v.(error); ok {
+			serverSpan.LogKV(k, vErr.Error())
+		} else {
+			serverSpan.SetTag(k, v)
+		}
+	}
+	if err != nil {
+		ext.Error.Set(serverSpan, true)
+		serverSpan.LogFields(log.String("event", "error"), log.String("message", err.Error()))
+	}
+	serverSpan.Finish()
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go
new file mode 100644
index 0000000..1ed9bb4
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/doc.go
@@ -0,0 +1,19 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+/*
+Package `metautils` provides convenience functions for dealing with gRPC metadata.MD objects inside
+Context handlers.
+
+While the upstream grpc-go package contains decent functionality (see https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md)
+they are hard to use.
+
+The majority of functions center around the NiceMD, which is a convenience wrapper around metadata.MD. For example
+the following code allows you to easily extract incoming metadata (server handler) and put it into a new client context
+metadata.
+
+  nmd := metautils.ExtractIncoming(serverCtx).Clone(":authorization", ":custom")
+  clientCtx := nmd.Set("x-client-header", "2").Set("x-another", "3").ToOutgoing(ctx)
+*/
+
+package metautils
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go
new file mode 100644
index 0000000..1c60585
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd.go
@@ -0,0 +1,126 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package metautils
+
+import (
+	"context"
+	"strings"
+
+	"google.golang.org/grpc/metadata"
+)
+
+// NiceMD is a convenience wrapper definiting extra functions on the metadata.
+type NiceMD metadata.MD
+
+// ExtractIncoming extracts an inbound metadata from the server-side context.
+//
+// This function always returns a NiceMD wrapper of the metadata.MD, in case the context doesn't have metadata it returns
+// a new empty NiceMD.
+func ExtractIncoming(ctx context.Context) NiceMD {
+	md, ok := metadata.FromIncomingContext(ctx)
+	if !ok {
+		return NiceMD(metadata.Pairs())
+	}
+	return NiceMD(md)
+}
+
+// ExtractOutgoing extracts an outbound metadata from the client-side context.
+//
+// This function always returns a NiceMD wrapper of the metadata.MD, in case the context doesn't have metadata it returns
+// a new empty NiceMD.
+func ExtractOutgoing(ctx context.Context) NiceMD {
+	md, ok := metadata.FromOutgoingContext(ctx)
+	if !ok {
+		return NiceMD(metadata.Pairs())
+	}
+	return NiceMD(md)
+}
+
+// Clone performs a *deep* copy of the metadata.MD.
+//
+// You can specify the lower-case copiedKeys to only copy certain whitelisted keys. If no keys are explicitly whitelisted
+// all keys get copied.
+func (m NiceMD) Clone(copiedKeys ...string) NiceMD {
+	newMd := NiceMD(metadata.Pairs())
+	for k, vv := range m {
+		found := false
+		if len(copiedKeys) == 0 {
+			found = true
+		} else {
+			for _, allowedKey := range copiedKeys {
+				if strings.EqualFold(allowedKey, k) {
+					found = true
+					break
+				}
+			}
+		}
+		if !found {
+			continue
+		}
+		newMd[k] = make([]string, len(vv))
+		copy(newMd[k], vv)
+	}
+	return NiceMD(newMd)
+}
+
+// ToOutgoing sets the given NiceMD as a client-side context for dispatching.
+func (m NiceMD) ToOutgoing(ctx context.Context) context.Context {
+	return metadata.NewOutgoingContext(ctx, metadata.MD(m))
+}
+
+// ToIncoming sets the given NiceMD as a server-side context for dispatching.
+//
+// This is mostly useful in ServerInterceptors..
+func (m NiceMD) ToIncoming(ctx context.Context) context.Context {
+	return metadata.NewIncomingContext(ctx, metadata.MD(m))
+}
+
+// Get retrieves a single value from the metadata.
+//
+// It works analogously to http.Header.Get, returning the first value if there are many set. If the value is not set,
+// an empty string is returned.
+//
+// The function is binary-key safe.
+func (m NiceMD) Get(key string) string {
+	k := strings.ToLower(key)
+	vv, ok := m[k]
+	if !ok {
+		return ""
+	}
+	return vv[0]
+}
+
+// Del retrieves a single value from the metadata.
+//
+// It works analogously to http.Header.Del, deleting all values if they exist.
+//
+// The function is binary-key safe.
+
+func (m NiceMD) Del(key string) NiceMD {
+	k := strings.ToLower(key)
+	delete(m, k)
+	return m
+}
+
+// Set sets the given value in a metadata.
+//
+// It works analogously to http.Header.Set, overwriting all previous metadata values.
+//
+// The function is binary-key safe.
+func (m NiceMD) Set(key string, value string) NiceMD {
+	k := strings.ToLower(key)
+	m[k] = []string{value}
+	return m
+}
+
+// Add retrieves a single value from the metadata.
+//
+// It works analogously to http.Header.Add, as it appends to any existing values associated with key.
+//
+// The function is binary-key safe.
+func (m NiceMD) Add(key string, value string) NiceMD {
+	k := strings.ToLower(key)
+	m[k] = append(m[k], value)
+	return m
+}
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..05ccfb3
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go
@@ -0,0 +1,30 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_middleware
+
+import (
+	"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/hashicorp/go-uuid/uuid.go b/vendor/github.com/hashicorp/go-uuid/uuid.go
index 911227f..0c10c4e 100644
--- a/vendor/github.com/hashicorp/go-uuid/uuid.go
+++ b/vendor/github.com/hashicorp/go-uuid/uuid.go
@@ -4,22 +4,40 @@
 	"crypto/rand"
 	"encoding/hex"
 	"fmt"
+	"io"
 )
 
 // GenerateRandomBytes is used to generate random bytes of given size.
 func GenerateRandomBytes(size int) ([]byte, error) {
+	return GenerateRandomBytesWithReader(size, rand.Reader)
+}
+
+// GenerateRandomBytesWithReader is used to generate random bytes of given size read from a given reader.
+func GenerateRandomBytesWithReader(size int, reader io.Reader) ([]byte, error) {
+	if reader == nil {
+		return nil, fmt.Errorf("provided reader is nil")
+	}
 	buf := make([]byte, size)
-	if _, err := rand.Read(buf); err != nil {
+	if _, err := io.ReadFull(reader, buf); err != nil {
 		return nil, fmt.Errorf("failed to read random bytes: %v", err)
 	}
 	return buf, nil
 }
 
+
 const uuidLen = 16
 
 // GenerateUUID is used to generate a random UUID
 func GenerateUUID() (string, error) {
-	buf, err := GenerateRandomBytes(uuidLen)
+	return GenerateUUIDWithReader(rand.Reader)
+}
+
+// GenerateUUIDWithReader is used to generate a random UUID with a given Reader
+func GenerateUUIDWithReader(reader io.Reader) (string, error) {
+	if reader == nil {
+		return "", fmt.Errorf("provided reader is nil")
+	}
+	buf, err := GenerateRandomBytesWithReader(uuidLen, reader)
 	if err != nil {
 		return "", err
 	}
diff --git a/vendor/gopkg.in/jcmturner/aescts.v1/LICENSE b/vendor/github.com/jcmturner/aescts/v2/LICENSE
similarity index 100%
rename from vendor/gopkg.in/jcmturner/aescts.v1/LICENSE
rename to vendor/github.com/jcmturner/aescts/v2/LICENSE
diff --git a/vendor/github.com/jcmturner/aescts/v2/aescts.go b/vendor/github.com/jcmturner/aescts/v2/aescts.go
new file mode 100644
index 0000000..fee3b43
--- /dev/null
+++ b/vendor/github.com/jcmturner/aescts/v2/aescts.go
@@ -0,0 +1,186 @@
+// Package aescts provides AES CBC CipherText Stealing encryption and decryption methods
+package aescts
+
+import (
+	"crypto/aes"
+	"crypto/cipher"
+	"errors"
+	"fmt"
+)
+
+// Encrypt the message with the key and the initial vector.
+// Returns: next iv, ciphertext bytes, error
+func Encrypt(key, iv, plaintext []byte) ([]byte, []byte, error) {
+	l := len(plaintext)
+
+	block, err := aes.NewCipher(key)
+	if err != nil {
+		return []byte{}, []byte{}, fmt.Errorf("error creating cipher: %v", err)
+	}
+	mode := cipher.NewCBCEncrypter(block, iv)
+
+	m := make([]byte, len(plaintext))
+	copy(m, plaintext)
+
+	/*For consistency, ciphertext stealing is always used for the last two
+	blocks of the data to be encrypted, as in [RC5].  If the data length
+	is a multiple of the block size, this is equivalent to plain CBC mode
+	with the last two ciphertext blocks swapped.*/
+	/*The initial vector carried out from one encryption for use in a
+	subsequent encryption is the next-to-last block of the encryption
+	output; this is the encrypted form of the last plaintext block.*/
+	if l <= aes.BlockSize {
+		m, _ = zeroPad(m, aes.BlockSize)
+		mode.CryptBlocks(m, m)
+		return m, m, nil
+	}
+	if l%aes.BlockSize == 0 {
+		mode.CryptBlocks(m, m)
+		iv = m[len(m)-aes.BlockSize:]
+		rb, _ := swapLastTwoBlocks(m, aes.BlockSize)
+		return iv, rb, nil
+	}
+	m, _ = zeroPad(m, aes.BlockSize)
+	rb, pb, lb, err := tailBlocks(m, aes.BlockSize)
+	if err != nil {
+		return []byte{}, []byte{}, fmt.Errorf("error tailing blocks: %v", err)
+	}
+	var ct []byte
+	if rb != nil {
+		// Encrpt all but the lats 2 blocks and update the rolling iv
+		mode.CryptBlocks(rb, rb)
+		iv = rb[len(rb)-aes.BlockSize:]
+		mode = cipher.NewCBCEncrypter(block, iv)
+		ct = append(ct, rb...)
+	}
+	mode.CryptBlocks(pb, pb)
+	mode = cipher.NewCBCEncrypter(block, pb)
+	mode.CryptBlocks(lb, lb)
+	// Cipher Text Stealing (CTS) - Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing
+	// Swap the last two cipher blocks
+	// Truncate the ciphertext to the length of the original plaintext
+	ct = append(ct, lb...)
+	ct = append(ct, pb...)
+	return lb, ct[:l], nil
+}
+
+// Decrypt the ciphertext with the key and the initial vector.
+func Decrypt(key, iv, ciphertext []byte) ([]byte, error) {
+	// Copy the cipher text as golang slices even when passed by value to this method can result in the backing arrays of the calling code value being updated.
+	ct := make([]byte, len(ciphertext))
+	copy(ct, ciphertext)
+	if len(ct) < aes.BlockSize {
+		return []byte{}, fmt.Errorf("ciphertext is not large enough. It is less that one block size. Blocksize:%v; Ciphertext:%v", aes.BlockSize, len(ct))
+	}
+	// Configure the CBC
+	block, err := aes.NewCipher(key)
+	if err != nil {
+		return nil, fmt.Errorf("error creating cipher: %v", err)
+	}
+	var mode cipher.BlockMode
+
+	//If ciphertext is multiple of blocksize we just need to swap back the last two blocks and then do CBC
+	//If the ciphertext is just one block we can't swap so we just decrypt
+	if len(ct)%aes.BlockSize == 0 {
+		if len(ct) > aes.BlockSize {
+			ct, _ = swapLastTwoBlocks(ct, aes.BlockSize)
+		}
+		mode = cipher.NewCBCDecrypter(block, iv)
+		message := make([]byte, len(ct))
+		mode.CryptBlocks(message, ct)
+		return message[:len(ct)], nil
+	}
+
+	// Cipher Text Stealing (CTS) using CBC interface. Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing
+	// Get ciphertext of the 2nd to last (penultimate) block (cpb), the last block (clb) and the rest (crb)
+	crb, cpb, clb, _ := tailBlocks(ct, aes.BlockSize)
+	v := make([]byte, len(iv), len(iv))
+	copy(v, iv)
+	var message []byte
+	if crb != nil {
+		//If there is more than just the last and the penultimate block we decrypt it and the last bloc of this becomes the iv for later
+		rb := make([]byte, len(crb))
+		mode = cipher.NewCBCDecrypter(block, v)
+		v = crb[len(crb)-aes.BlockSize:]
+		mode.CryptBlocks(rb, crb)
+		message = append(message, rb...)
+	}
+
+	// We need to modify the cipher text
+	// Decryt the 2nd to last (penultimate) block with a the original iv
+	pb := make([]byte, aes.BlockSize)
+	mode = cipher.NewCBCDecrypter(block, iv)
+	mode.CryptBlocks(pb, cpb)
+	// number of byte needed to pad
+	npb := aes.BlockSize - len(ct)%aes.BlockSize
+	//pad last block using the number of bytes needed from the tail of the plaintext 2nd to last (penultimate) block
+	clb = append(clb, pb[len(pb)-npb:]...)
+
+	// Now decrypt the last block in the penultimate position (iv will be from the crb, if the is no crb it's zeros)
+	// iv for the penultimate block decrypted in the last position becomes the modified last block
+	lb := make([]byte, aes.BlockSize)
+	mode = cipher.NewCBCDecrypter(block, v)
+	v = clb
+	mode.CryptBlocks(lb, clb)
+	message = append(message, lb...)
+
+	// Now decrypt the penultimate block in the last position (iv will be from the modified last block)
+	mode = cipher.NewCBCDecrypter(block, v)
+	mode.CryptBlocks(cpb, cpb)
+	message = append(message, cpb...)
+
+	// Truncate to the size of the original cipher text
+	return message[:len(ct)], nil
+}
+
+func tailBlocks(b []byte, c int) ([]byte, []byte, []byte, error) {
+	if len(b) <= c {
+		return []byte{}, []byte{}, []byte{}, errors.New("bytes slice is not larger than one block so cannot tail")
+	}
+	// Get size of last block
+	var lbs int
+	if l := len(b) % aes.BlockSize; l == 0 {
+		lbs = aes.BlockSize
+	} else {
+		lbs = l
+	}
+	// Get last block
+	lb := b[len(b)-lbs:]
+	// Get 2nd to last (penultimate) block
+	pb := b[len(b)-lbs-c : len(b)-lbs]
+	if len(b) > 2*c {
+		rb := b[:len(b)-lbs-c]
+		return rb, pb, lb, nil
+	}
+	return nil, pb, lb, nil
+}
+
+func swapLastTwoBlocks(b []byte, c int) ([]byte, error) {
+	rb, pb, lb, err := tailBlocks(b, c)
+	if err != nil {
+		return nil, err
+	}
+	var out []byte
+	if rb != nil {
+		out = append(out, rb...)
+	}
+	out = append(out, lb...)
+	out = append(out, pb...)
+	return out, nil
+}
+
+// zeroPad pads bytes with zeros to nearest multiple of message size m.
+func zeroPad(b []byte, m int) ([]byte, error) {
+	if m <= 0 {
+		return nil, errors.New("invalid message block size when padding")
+	}
+	if b == nil || len(b) == 0 {
+		return nil, errors.New("data not valid to pad: Zero size")
+	}
+	if l := len(b) % m; l != 0 {
+		n := m - l
+		z := make([]byte, n)
+		b = append(b, z...)
+	}
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/aescts/v2/go.mod b/vendor/github.com/jcmturner/aescts/v2/go.mod
new file mode 100644
index 0000000..034c3ce
--- /dev/null
+++ b/vendor/github.com/jcmturner/aescts/v2/go.mod
@@ -0,0 +1,5 @@
+module github.com/jcmturner/aescts/v2
+
+go 1.13
+
+require github.com/stretchr/testify v1.4.0
diff --git a/vendor/github.com/jcmturner/aescts/v2/go.sum b/vendor/github.com/jcmturner/aescts/v2/go.sum
new file mode 100644
index 0000000..e863f51
--- /dev/null
+++ b/vendor/github.com/jcmturner/aescts/v2/go.sum
@@ -0,0 +1,10 @@
+github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+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.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
diff --git a/vendor/gopkg.in/jcmturner/dnsutils.v1/LICENSE b/vendor/github.com/jcmturner/dnsutils/v2/LICENSE
similarity index 100%
rename from vendor/gopkg.in/jcmturner/dnsutils.v1/LICENSE
rename to vendor/github.com/jcmturner/dnsutils/v2/LICENSE
diff --git a/vendor/github.com/jcmturner/dnsutils/v2/go.mod b/vendor/github.com/jcmturner/dnsutils/v2/go.mod
new file mode 100644
index 0000000..f75ac6d
--- /dev/null
+++ b/vendor/github.com/jcmturner/dnsutils/v2/go.mod
@@ -0,0 +1,5 @@
+module github.com/jcmturner/dnsutils/v2
+
+go 1.13
+
+require github.com/stretchr/testify v1.4.0
diff --git a/vendor/github.com/jcmturner/dnsutils/v2/go.sum b/vendor/github.com/jcmturner/dnsutils/v2/go.sum
new file mode 100644
index 0000000..e863f51
--- /dev/null
+++ b/vendor/github.com/jcmturner/dnsutils/v2/go.sum
@@ -0,0 +1,10 @@
+github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+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.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
diff --git a/vendor/gopkg.in/jcmturner/dnsutils.v1/srv.go b/vendor/github.com/jcmturner/dnsutils/v2/srv.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/dnsutils.v1/srv.go
rename to vendor/github.com/jcmturner/dnsutils/v2/srv.go
diff --git a/vendor/gopkg.in/jcmturner/aescts.v1/LICENSE b/vendor/github.com/jcmturner/gokrb5/v8/LICENSE
similarity index 100%
copy from vendor/gopkg.in/jcmturner/aescts.v1/LICENSE
copy to vendor/github.com/jcmturner/gokrb5/v8/LICENSE
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/asn1tools/tools.go b/vendor/github.com/jcmturner/gokrb5/v8/asn1tools/tools.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/asn1tools/tools.go
rename to vendor/github.com/jcmturner/gokrb5/v8/asn1tools/tools.go
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go b/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go
new file mode 100644
index 0000000..5becccc
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go
@@ -0,0 +1,182 @@
+package client
+
+import (
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/patype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// ASExchange performs an AS exchange for the client to retrieve a TGT.
+func (cl *Client) ASExchange(realm string, ASReq messages.ASReq, referral int) (messages.ASRep, error) {
+	if ok, err := cl.IsConfigured(); !ok {
+		return messages.ASRep{}, krberror.Errorf(err, krberror.ConfigError, "AS Exchange cannot be performed")
+	}
+
+	// Set PAData if required
+	err := setPAData(cl, nil, &ASReq)
+	if err != nil {
+		return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: issue with setting PAData on AS_REQ")
+	}
+
+	b, err := ASReq.Marshal()
+	if err != nil {
+		return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ")
+	}
+	var ASRep messages.ASRep
+
+	rb, err := cl.sendToKDC(b, realm)
+	if err != nil {
+		if e, ok := err.(messages.KRBError); ok {
+			switch e.ErrorCode {
+			case errorcode.KDC_ERR_PREAUTH_REQUIRED, errorcode.KDC_ERR_PREAUTH_FAILED:
+				// From now on assume this client will need to do this pre-auth and set the PAData
+				cl.settings.assumePreAuthentication = true
+				err = setPAData(cl, &e, &ASReq)
+				if err != nil {
+					return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: failed setting AS_REQ PAData for pre-authentication required")
+				}
+				b, err := ASReq.Marshal()
+				if err != nil {
+					return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ with PAData")
+				}
+				rb, err = cl.sendToKDC(b, realm)
+				if err != nil {
+					if _, ok := err.(messages.KRBError); ok {
+						return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC")
+					}
+					return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC")
+				}
+			case errorcode.KDC_ERR_WRONG_REALM:
+				// Client referral https://tools.ietf.org/html/rfc6806.html#section-7
+				if referral > 5 {
+					return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "maximum number of client referrals exceeded")
+				}
+				referral++
+				return cl.ASExchange(e.CRealm, ASReq, referral)
+			default:
+				return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC")
+			}
+		} else {
+			return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC")
+		}
+	}
+	err = ASRep.Unmarshal(rb)
+	if err != nil {
+		return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed to process the AS_REP")
+	}
+	if ok, err := ASRep.Verify(cl.Config, cl.Credentials, ASReq); !ok {
+		return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: AS_REP is not valid or client password/keytab incorrect")
+	}
+	return ASRep, nil
+}
+
+// setPAData adds pre-authentication data to the AS_REQ.
+func setPAData(cl *Client, krberr *messages.KRBError, ASReq *messages.ASReq) error {
+	if !cl.settings.DisablePAFXFAST() {
+		pa := types.PAData{PADataType: patype.PA_REQ_ENC_PA_REP}
+		ASReq.PAData = append(ASReq.PAData, pa)
+	}
+	if cl.settings.AssumePreAuthentication() {
+		// Identify the etype to use to encrypt the PA Data
+		var et etype.EType
+		var err error
+		var key types.EncryptionKey
+		var kvno int
+		if krberr == nil {
+			// This is not in response to an error from the KDC. It is preemptive or renewal
+			// There is no KRB Error that tells us the etype to use
+			etn := cl.settings.preAuthEType // Use the etype that may have previously been negotiated
+			if etn == 0 {
+				etn = int32(cl.Config.LibDefaults.PreferredPreauthTypes[0]) // Resort to config
+			}
+			et, err = crypto.GetEtype(etn)
+			if err != nil {
+				return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption")
+			}
+			key, kvno, err = cl.Key(et, 0, nil)
+			if err != nil {
+				return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials")
+			}
+		} else {
+			// Get the etype to use from the PA data in the KRBError e-data
+			et, err = preAuthEType(krberr)
+			if err != nil {
+				return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption")
+			}
+			cl.settings.preAuthEType = et.GetETypeID() // Set the etype that has been defined for potential future use
+			key, kvno, err = cl.Key(et, 0, krberr)
+			if err != nil {
+				return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials")
+			}
+		}
+		// Generate the PA data
+		paTSb, err := types.GetPAEncTSEncAsnMarshalled()
+		if err != nil {
+			return krberror.Errorf(err, krberror.KRBMsgError, "error creating PAEncTSEnc for Pre-Authentication")
+		}
+		paEncTS, err := crypto.GetEncryptedData(paTSb, key, keyusage.AS_REQ_PA_ENC_TIMESTAMP, kvno)
+		if err != nil {
+			return krberror.Errorf(err, krberror.EncryptingError, "error encrypting pre-authentication timestamp")
+		}
+		pb, err := paEncTS.Marshal()
+		if err != nil {
+			return krberror.Errorf(err, krberror.EncodingError, "error marshaling the PAEncTSEnc encrypted data")
+		}
+		pa := types.PAData{
+			PADataType:  patype.PA_ENC_TIMESTAMP,
+			PADataValue: pb,
+		}
+		// Look for and delete any exiting patype.PA_ENC_TIMESTAMP
+		for i, pa := range ASReq.PAData {
+			if pa.PADataType == patype.PA_ENC_TIMESTAMP {
+				ASReq.PAData[i] = ASReq.PAData[len(ASReq.PAData)-1]
+				ASReq.PAData = ASReq.PAData[:len(ASReq.PAData)-1]
+			}
+		}
+		ASReq.PAData = append(ASReq.PAData, pa)
+	}
+	return nil
+}
+
+// preAuthEType establishes what encryption type to use for pre-authentication from the KRBError returned from the KDC.
+func preAuthEType(krberr *messages.KRBError) (etype etype.EType, err error) {
+	//RFC 4120 5.2.7.5 covers the preference order of ETYPE-INFO2 and ETYPE-INFO.
+	var etypeID int32
+	var pas types.PADataSequence
+	e := pas.Unmarshal(krberr.EData)
+	if e != nil {
+		err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling KRBError data")
+		return
+	}
+Loop:
+	for _, pa := range pas {
+		switch pa.PADataType {
+		case patype.PA_ETYPE_INFO2:
+			info, e := pa.GetETypeInfo2()
+			if e != nil {
+				err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO2 data")
+				return
+			}
+			etypeID = info[0].EType
+			break Loop
+		case patype.PA_ETYPE_INFO:
+			info, e := pa.GetETypeInfo()
+			if e != nil {
+				err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO data")
+				return
+			}
+			etypeID = info[0].EType
+		}
+	}
+	etype, e = crypto.GetEtype(etypeID)
+	if e != nil {
+		err = krberror.Errorf(e, krberror.EncryptingError, "error creating etype")
+		return
+	}
+	return etype, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go b/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go
new file mode 100644
index 0000000..e4571ce
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go
@@ -0,0 +1,103 @@
+package client
+
+import (
+	"github.com/jcmturner/gokrb5/v8/iana/flags"
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// TGSREQGenerateAndExchange generates the TGS_REQ and performs a TGS exchange to retrieve a ticket to the specified SPN.
+func (cl *Client) TGSREQGenerateAndExchange(spn types.PrincipalName, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, renewal bool) (tgsReq messages.TGSReq, tgsRep messages.TGSRep, err error) {
+	tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, spn, renewal)
+	if err != nil {
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: failed to generate a new TGS_REQ")
+	}
+	return cl.TGSExchange(tgsReq, kdcRealm, tgsRep.Ticket, sessionKey, 0)
+}
+
+// TGSExchange exchanges the provided TGS_REQ with the KDC to retrieve a TGS_REP.
+// Referrals are automatically handled.
+// The client's cache is updated with the ticket received.
+func (cl *Client) TGSExchange(tgsReq messages.TGSReq, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, referral int) (messages.TGSReq, messages.TGSRep, error) {
+	var tgsRep messages.TGSRep
+	b, err := tgsReq.Marshal()
+	if err != nil {
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to marshal TGS_REQ")
+	}
+	r, err := cl.sendToKDC(b, kdcRealm)
+	if err != nil {
+		if _, ok := err.(messages.KRBError); ok {
+			return tgsReq, tgsRep, krberror.Errorf(err, krberror.KDCError, "TGS Exchange Error: kerberos error response from KDC when requesting for %s", tgsReq.ReqBody.SName.PrincipalNameString())
+		}
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.NetworkingError, "TGS Exchange Error: issue sending TGS_REQ to KDC")
+	}
+	err = tgsRep.Unmarshal(r)
+	if err != nil {
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP")
+	}
+	err = tgsRep.DecryptEncPart(sessionKey)
+	if err != nil {
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP")
+	}
+	if ok, err := tgsRep.Verify(cl.Config, tgsReq); !ok {
+		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: TGS_REP is not valid")
+	}
+
+	if tgsRep.Ticket.SName.NameString[0] == "krbtgt" && !tgsRep.Ticket.SName.Equal(tgsReq.ReqBody.SName) {
+		if referral > 5 {
+			return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: maximum number of referrals exceeded")
+		}
+		// Server referral https://tools.ietf.org/html/rfc6806.html#section-8
+		// The TGS Rep contains a TGT for another domain as the service resides in that domain.
+		cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart)
+		realm := tgsRep.Ticket.SName.NameString[len(tgsRep.Ticket.SName.NameString)-1]
+		referral++
+		if types.IsFlagSet(&tgsReq.ReqBody.KDCOptions, flags.EncTktInSkey) && len(tgsReq.ReqBody.AdditionalTickets) > 0 {
+			tgsReq, err = messages.NewUser2UserTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, tgsReq.ReqBody.SName, tgsReq.Renewal, tgsReq.ReqBody.AdditionalTickets[0])
+			if err != nil {
+				return tgsReq, tgsRep, err
+			}
+		}
+		tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), realm, cl.Config, tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, tgsReq.ReqBody.SName, tgsReq.Renewal)
+		if err != nil {
+			return tgsReq, tgsRep, err
+		}
+		return cl.TGSExchange(tgsReq, realm, tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, referral)
+	}
+	cl.cache.addEntry(
+		tgsRep.Ticket,
+		tgsRep.DecryptedEncPart.AuthTime,
+		tgsRep.DecryptedEncPart.StartTime,
+		tgsRep.DecryptedEncPart.EndTime,
+		tgsRep.DecryptedEncPart.RenewTill,
+		tgsRep.DecryptedEncPart.Key,
+	)
+	cl.Log("ticket added to cache for %s (EndTime: %v)", tgsRep.Ticket.SName.PrincipalNameString(), tgsRep.DecryptedEncPart.EndTime)
+	return tgsReq, tgsRep, err
+}
+
+// GetServiceTicket makes a request to get a service ticket for the SPN specified
+// SPN format: <SERVICE>/<FQDN> Eg. HTTP/www.example.com
+// The ticket will be added to the client's ticket cache
+func (cl *Client) GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) {
+	var tkt messages.Ticket
+	var skey types.EncryptionKey
+	if tkt, skey, ok := cl.GetCachedTicket(spn); ok {
+		// Already a valid ticket in the cache
+		return tkt, skey, nil
+	}
+	princ := types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn)
+	realm := cl.Config.ResolveRealm(princ.NameString[len(princ.NameString)-1])
+
+	tgt, skey, err := cl.sessionTGT(realm)
+	if err != nil {
+		return tkt, skey, err
+	}
+	_, tgsRep, err := cl.TGSREQGenerateAndExchange(princ, realm, tgt, skey, false)
+	if err != nil {
+		return tkt, skey, err
+	}
+	return tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go b/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go
new file mode 100644
index 0000000..552e73e
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go
@@ -0,0 +1,134 @@
+package client
+
+import (
+	"encoding/json"
+	"errors"
+	"sort"
+	"sync"
+	"time"
+
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// Cache for service tickets held by the client.
+type Cache struct {
+	Entries map[string]CacheEntry
+	mux     sync.RWMutex
+}
+
+// CacheEntry holds details for a cache entry.
+type CacheEntry struct {
+	SPN        string
+	Ticket     messages.Ticket `json:"-"`
+	AuthTime   time.Time
+	StartTime  time.Time
+	EndTime    time.Time
+	RenewTill  time.Time
+	SessionKey types.EncryptionKey `json:"-"`
+}
+
+// NewCache creates a new client ticket cache instance.
+func NewCache() *Cache {
+	return &Cache{
+		Entries: map[string]CacheEntry{},
+	}
+}
+
+// getEntry returns a cache entry that matches the SPN.
+func (c *Cache) getEntry(spn string) (CacheEntry, bool) {
+	c.mux.RLock()
+	defer c.mux.RUnlock()
+	e, ok := (*c).Entries[spn]
+	return e, ok
+}
+
+// JSON returns information about the cached service tickets in a JSON format.
+func (c *Cache) JSON() (string, error) {
+	c.mux.RLock()
+	defer c.mux.RUnlock()
+	var es []CacheEntry
+	keys := make([]string, 0, len(c.Entries))
+	for k := range c.Entries {
+		keys = append(keys, k)
+	}
+	sort.Strings(keys)
+	for _, k := range keys {
+		es = append(es, c.Entries[k])
+	}
+	b, err := json.MarshalIndent(&es, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+}
+
+// addEntry adds a ticket to the cache.
+func (c *Cache) addEntry(tkt messages.Ticket, authTime, startTime, endTime, renewTill time.Time, sessionKey types.EncryptionKey) CacheEntry {
+	spn := tkt.SName.PrincipalNameString()
+	c.mux.Lock()
+	defer c.mux.Unlock()
+	(*c).Entries[spn] = CacheEntry{
+		SPN:        spn,
+		Ticket:     tkt,
+		AuthTime:   authTime,
+		StartTime:  startTime,
+		EndTime:    endTime,
+		RenewTill:  renewTill,
+		SessionKey: sessionKey,
+	}
+	return c.Entries[spn]
+}
+
+// clear deletes all the cache entries
+func (c *Cache) clear() {
+	c.mux.Lock()
+	defer c.mux.Unlock()
+	for k := range c.Entries {
+		delete(c.Entries, k)
+	}
+}
+
+// RemoveEntry removes the cache entry for the defined SPN.
+func (c *Cache) RemoveEntry(spn string) {
+	c.mux.Lock()
+	defer c.mux.Unlock()
+	delete(c.Entries, spn)
+}
+
+// GetCachedTicket returns a ticket from the cache for the SPN.
+// Only a ticket that is currently valid will be returned.
+func (cl *Client) GetCachedTicket(spn string) (messages.Ticket, types.EncryptionKey, bool) {
+	if e, ok := cl.cache.getEntry(spn); ok {
+		//If within time window of ticket return it
+		if time.Now().UTC().After(e.StartTime) && time.Now().UTC().Before(e.EndTime) {
+			cl.Log("ticket received from cache for %s", spn)
+			return e.Ticket, e.SessionKey, true
+		} else if time.Now().UTC().Before(e.RenewTill) {
+			e, err := cl.renewTicket(e)
+			if err != nil {
+				return e.Ticket, e.SessionKey, false
+			}
+			return e.Ticket, e.SessionKey, true
+		}
+	}
+	var tkt messages.Ticket
+	var key types.EncryptionKey
+	return tkt, key, false
+}
+
+// renewTicket renews a cache entry ticket.
+// To renew from outside the client package use GetCachedTicket
+func (cl *Client) renewTicket(e CacheEntry) (CacheEntry, error) {
+	spn := e.Ticket.SName
+	_, _, err := cl.TGSREQGenerateAndExchange(spn, e.Ticket.Realm, e.Ticket, e.SessionKey, true)
+	if err != nil {
+		return e, err
+	}
+	e, ok := cl.cache.getEntry(e.Ticket.SName.PrincipalNameString())
+	if !ok {
+		return e, errors.New("ticket was not added to cache")
+	}
+	cl.Log("ticket renewed for %s (EndTime: %v)", spn.PrincipalNameString(), e.EndTime)
+	return e, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/client.go b/vendor/github.com/jcmturner/gokrb5/v8/client/client.go
new file mode 100644
index 0000000..074e3f1
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/client.go
@@ -0,0 +1,329 @@
+// Package client provides a client library and methods for Kerberos 5 authentication.
+package client
+
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io"
+	"strings"
+	"time"
+
+	"github.com/jcmturner/gokrb5/v8/config"
+	"github.com/jcmturner/gokrb5/v8/credentials"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+	"github.com/jcmturner/gokrb5/v8/keytab"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// Client side configuration and state.
+type Client struct {
+	Credentials *credentials.Credentials
+	Config      *config.Config
+	settings    *Settings
+	sessions    *sessions
+	cache       *Cache
+}
+
+// NewWithPassword creates a new client from a password credential.
+// Set the realm to empty string to use the default realm from config.
+func NewWithPassword(username, realm, password string, krb5conf *config.Config, settings ...func(*Settings)) *Client {
+	creds := credentials.New(username, realm)
+	return &Client{
+		Credentials: creds.WithPassword(password),
+		Config:      krb5conf,
+		settings:    NewSettings(settings...),
+		sessions: &sessions{
+			Entries: make(map[string]*session),
+		},
+		cache: NewCache(),
+	}
+}
+
+// NewWithKeytab creates a new client from a keytab credential.
+func NewWithKeytab(username, realm string, kt *keytab.Keytab, krb5conf *config.Config, settings ...func(*Settings)) *Client {
+	creds := credentials.New(username, realm)
+	return &Client{
+		Credentials: creds.WithKeytab(kt),
+		Config:      krb5conf,
+		settings:    NewSettings(settings...),
+		sessions: &sessions{
+			Entries: make(map[string]*session),
+		},
+		cache: NewCache(),
+	}
+}
+
+// NewFromCCache create a client from a populated client cache.
+//
+// WARNING: A client created from CCache does not automatically renew TGTs and a failure will occur after the TGT expires.
+func NewFromCCache(c *credentials.CCache, krb5conf *config.Config, settings ...func(*Settings)) (*Client, error) {
+	cl := &Client{
+		Credentials: c.GetClientCredentials(),
+		Config:      krb5conf,
+		settings:    NewSettings(settings...),
+		sessions: &sessions{
+			Entries: make(map[string]*session),
+		},
+		cache: NewCache(),
+	}
+	spn := types.PrincipalName{
+		NameType:   nametype.KRB_NT_SRV_INST,
+		NameString: []string{"krbtgt", c.DefaultPrincipal.Realm},
+	}
+	cred, ok := c.GetEntry(spn)
+	if !ok {
+		return cl, errors.New("TGT not found in CCache")
+	}
+	var tgt messages.Ticket
+	err := tgt.Unmarshal(cred.Ticket)
+	if err != nil {
+		return cl, fmt.Errorf("TGT bytes in cache are not valid: %v", err)
+	}
+	cl.sessions.Entries[c.DefaultPrincipal.Realm] = &session{
+		realm:      c.DefaultPrincipal.Realm,
+		authTime:   cred.AuthTime,
+		endTime:    cred.EndTime,
+		renewTill:  cred.RenewTill,
+		tgt:        tgt,
+		sessionKey: cred.Key,
+	}
+	for _, cred := range c.GetEntries() {
+		var tkt messages.Ticket
+		err = tkt.Unmarshal(cred.Ticket)
+		if err != nil {
+			return cl, fmt.Errorf("cache entry ticket bytes are not valid: %v", err)
+		}
+		cl.cache.addEntry(
+			tkt,
+			cred.AuthTime,
+			cred.StartTime,
+			cred.EndTime,
+			cred.RenewTill,
+			cred.Key,
+		)
+	}
+	return cl, nil
+}
+
+// Key returns the client's encryption key for the specified encryption type and its kvno (kvno of zero will find latest).
+// The key can be retrieved either from the keytab or generated from the client's password.
+// If the client has both a keytab and a password defined the keytab is favoured as the source for the key
+// A KRBError can be passed in the event the KDC returns one of type KDC_ERR_PREAUTH_REQUIRED and is required to derive
+// the key for pre-authentication from the client's password. If a KRBError is not available, pass nil to this argument.
+func (cl *Client) Key(etype etype.EType, kvno int, krberr *messages.KRBError) (types.EncryptionKey, int, error) {
+	if cl.Credentials.HasKeytab() && etype != nil {
+		return cl.Credentials.Keytab().GetEncryptionKey(cl.Credentials.CName(), cl.Credentials.Domain(), kvno, etype.GetETypeID())
+	} else if cl.Credentials.HasPassword() {
+		if krberr != nil && krberr.ErrorCode == errorcode.KDC_ERR_PREAUTH_REQUIRED {
+			var pas types.PADataSequence
+			err := pas.Unmarshal(krberr.EData)
+			if err != nil {
+				return types.EncryptionKey{}, 0, fmt.Errorf("could not get PAData from KRBError to generate key from password: %v", err)
+			}
+			key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), krberr.CName, krberr.CRealm, etype.GetETypeID(), pas)
+			return key, 0, err
+		}
+		key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), cl.Credentials.CName(), cl.Credentials.Domain(), etype.GetETypeID(), types.PADataSequence{})
+		return key, 0, err
+	}
+	return types.EncryptionKey{}, 0, errors.New("credential has neither keytab or password to generate key")
+}
+
+// IsConfigured indicates if the client has the values required set.
+func (cl *Client) IsConfigured() (bool, error) {
+	if cl.Credentials.UserName() == "" {
+		return false, errors.New("client does not have a username")
+	}
+	if cl.Credentials.Domain() == "" {
+		return false, errors.New("client does not have a define realm")
+	}
+	// Client needs to have either a password, keytab or a session already (later when loading from CCache)
+	if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() {
+		authTime, _, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
+		if err != nil || authTime.IsZero() {
+			return false, errors.New("client has neither a keytab nor a password set and no session")
+		}
+	}
+	if !cl.Config.LibDefaults.DNSLookupKDC {
+		for _, r := range cl.Config.Realms {
+			if r.Realm == cl.Credentials.Domain() {
+				if len(r.KDC) > 0 {
+					return true, nil
+				}
+				return false, errors.New("client krb5 config does not have any defined KDCs for the default realm")
+			}
+		}
+	}
+	return true, nil
+}
+
+// Login the client with the KDC via an AS exchange.
+func (cl *Client) Login() error {
+	if ok, err := cl.IsConfigured(); !ok {
+		return err
+	}
+	if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() {
+		_, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
+		if err != nil {
+			return krberror.Errorf(err, krberror.KRBMsgError, "no user credentials available and error getting any existing session")
+		}
+		if time.Now().UTC().After(endTime) {
+			return krberror.New(krberror.KRBMsgError, "cannot login, no user credentials available and no valid existing session")
+		}
+		// no credentials but there is a session with tgt already
+		return nil
+	}
+	ASReq, err := messages.NewASReqForTGT(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName())
+	if err != nil {
+		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AS_REQ")
+	}
+	ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0)
+	if err != nil {
+		return err
+	}
+	cl.addSession(ASRep.Ticket, ASRep.DecryptedEncPart)
+	return nil
+}
+
+// AffirmLogin will only perform an AS exchange with the KDC if the client does not already have a TGT.
+func (cl *Client) AffirmLogin() error {
+	_, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
+	if err != nil || time.Now().UTC().After(endTime) {
+		err := cl.Login()
+		if err != nil {
+			return fmt.Errorf("could not get valid TGT for client's realm: %v", err)
+		}
+	}
+	return nil
+}
+
+// realmLogin obtains or renews a TGT and establishes a session for the realm specified.
+func (cl *Client) realmLogin(realm string) error {
+	if realm == cl.Credentials.Domain() {
+		return cl.Login()
+	}
+	_, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
+	if err != nil || time.Now().UTC().After(endTime) {
+		err := cl.Login()
+		if err != nil {
+			return fmt.Errorf("could not get valid TGT for client's realm: %v", err)
+		}
+	}
+	tgt, skey, err := cl.sessionTGT(cl.Credentials.Domain())
+	if err != nil {
+		return err
+	}
+
+	spn := types.PrincipalName{
+		NameType:   nametype.KRB_NT_SRV_INST,
+		NameString: []string{"krbtgt", realm},
+	}
+
+	_, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, false)
+	if err != nil {
+		return err
+	}
+	cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart)
+
+	return nil
+}
+
+// Destroy stops the auto-renewal of all sessions and removes the sessions and cache entries from the client.
+func (cl *Client) Destroy() {
+	creds := credentials.New("", "")
+	cl.sessions.destroy()
+	cl.cache.clear()
+	cl.Credentials = creds
+	cl.Log("client destroyed")
+}
+
+// Diagnostics runs a set of checks that the client is properly configured and writes details to the io.Writer provided.
+func (cl *Client) Diagnostics(w io.Writer) error {
+	cl.Print(w)
+	var errs []string
+	if cl.Credentials.HasKeytab() {
+		var loginRealmEncTypes []int32
+		for _, e := range cl.Credentials.Keytab().Entries {
+			if e.Principal.Realm == cl.Credentials.Realm() {
+				loginRealmEncTypes = append(loginRealmEncTypes, e.Key.KeyType)
+			}
+		}
+		for _, et := range cl.Config.LibDefaults.DefaultTktEnctypeIDs {
+			var etInKt bool
+			for _, val := range loginRealmEncTypes {
+				if val == et {
+					etInKt = true
+					break
+				}
+			}
+			if !etInKt {
+				errs = append(errs, fmt.Sprintf("default_tkt_enctypes specifies %d but this enctype is not available in the client's keytab", et))
+			}
+		}
+		for _, et := range cl.Config.LibDefaults.PreferredPreauthTypes {
+			var etInKt bool
+			for _, val := range loginRealmEncTypes {
+				if int(val) == et {
+					etInKt = true
+					break
+				}
+			}
+			if !etInKt {
+				errs = append(errs, fmt.Sprintf("preferred_preauth_types specifies %d but this enctype is not available in the client's keytab", et))
+			}
+		}
+	}
+	udpCnt, udpKDC, err := cl.Config.GetKDCs(cl.Credentials.Realm(), false)
+	if err != nil {
+		errs = append(errs, fmt.Sprintf("error when resolving KDCs for UDP communication: %v", err))
+	}
+	if udpCnt < 1 {
+		errs = append(errs, "no KDCs resolved for communication via UDP.")
+	} else {
+		b, _ := json.MarshalIndent(&udpKDC, "", "  ")
+		fmt.Fprintf(w, "UDP KDCs: %s\n", string(b))
+	}
+	tcpCnt, tcpKDC, err := cl.Config.GetKDCs(cl.Credentials.Realm(), false)
+	if err != nil {
+		errs = append(errs, fmt.Sprintf("error when resolving KDCs for TCP communication: %v", err))
+	}
+	if tcpCnt < 1 {
+		errs = append(errs, "no KDCs resolved for communication via TCP.")
+	} else {
+		b, _ := json.MarshalIndent(&tcpKDC, "", "  ")
+		fmt.Fprintf(w, "TCP KDCs: %s\n", string(b))
+	}
+
+	if errs == nil || len(errs) < 1 {
+		return nil
+	}
+	err = fmt.Errorf(strings.Join(errs, "\n"))
+	return err
+}
+
+// Print writes the details of the client to the io.Writer provided.
+func (cl *Client) Print(w io.Writer) {
+	c, _ := cl.Credentials.JSON()
+	fmt.Fprintf(w, "Credentials:\n%s\n", c)
+
+	s, _ := cl.sessions.JSON()
+	fmt.Fprintf(w, "TGT Sessions:\n%s\n", s)
+
+	c, _ = cl.cache.JSON()
+	fmt.Fprintf(w, "Service ticket cache:\n%s\n", c)
+
+	s, _ = cl.settings.JSON()
+	fmt.Fprintf(w, "Settings:\n%s\n", s)
+
+	j, _ := cl.Config.JSON()
+	fmt.Fprintf(w, "Krb5 config:\n%s\n", j)
+
+	k, _ := cl.Credentials.Keytab().JSON()
+	fmt.Fprintf(w, "Keytab:\n%s\n", k)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/network.go b/vendor/github.com/jcmturner/gokrb5/v8/client/network.go
new file mode 100644
index 0000000..634f015
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/network.go
@@ -0,0 +1,218 @@
+package client
+
+import (
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"io"
+	"net"
+	"strings"
+	"time"
+
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/messages"
+)
+
+// SendToKDC performs network actions to send data to the KDC.
+func (cl *Client) sendToKDC(b []byte, realm string) ([]byte, error) {
+	var rb []byte
+	if cl.Config.LibDefaults.UDPPreferenceLimit == 1 {
+		//1 means we should always use TCP
+		rb, errtcp := cl.sendKDCTCP(realm, b)
+		if errtcp != nil {
+			if e, ok := errtcp.(messages.KRBError); ok {
+				return rb, e
+			}
+			return rb, fmt.Errorf("communication error with KDC via TCP: %v", errtcp)
+		}
+		return rb, nil
+	}
+	if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit {
+		//Try UDP first, TCP second
+		rb, errudp := cl.sendKDCUDP(realm, b)
+		if errudp != nil {
+			if e, ok := errudp.(messages.KRBError); ok && e.ErrorCode != errorcode.KRB_ERR_RESPONSE_TOO_BIG {
+				// Got a KRBError from KDC
+				// If this is not a KRB_ERR_RESPONSE_TOO_BIG we will return immediately otherwise will try TCP.
+				return rb, e
+			}
+			// Try TCP
+			r, errtcp := cl.sendKDCTCP(realm, b)
+			if errtcp != nil {
+				if e, ok := errtcp.(messages.KRBError); ok {
+					// Got a KRBError
+					return r, e
+				}
+				return r, fmt.Errorf("failed to communicate with KDC. Attempts made with UDP (%v) and then TCP (%v)", errudp, errtcp)
+			}
+			rb = r
+		}
+		return rb, nil
+	}
+	//Try TCP first, UDP second
+	rb, errtcp := cl.sendKDCTCP(realm, b)
+	if errtcp != nil {
+		if e, ok := errtcp.(messages.KRBError); ok {
+			// Got a KRBError from KDC so returning and not trying UDP.
+			return rb, e
+		}
+		rb, errudp := cl.sendKDCUDP(realm, b)
+		if errudp != nil {
+			if e, ok := errudp.(messages.KRBError); ok {
+				// Got a KRBError
+				return rb, e
+			}
+			return rb, fmt.Errorf("failed to communicate with KDC. Attempts made with TCP (%v) and then UDP (%v)", errtcp, errudp)
+		}
+	}
+	return rb, nil
+}
+
+// sendKDCUDP sends bytes to the KDC via UDP.
+func (cl *Client) sendKDCUDP(realm string, b []byte) ([]byte, error) {
+	var r []byte
+	_, kdcs, err := cl.Config.GetKDCs(realm, false)
+	if err != nil {
+		return r, err
+	}
+	r, err = dialSendUDP(kdcs, b)
+	if err != nil {
+		return r, err
+	}
+	return checkForKRBError(r)
+}
+
+// dialSendUDP establishes a UDP connection to a KDC.
+func dialSendUDP(kdcs map[int]string, b []byte) ([]byte, error) {
+	var errs []string
+	for i := 1; i <= len(kdcs); i++ {
+		udpAddr, err := net.ResolveUDPAddr("udp", kdcs[i])
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error resolving KDC address: %v", err))
+			continue
+		}
+
+		conn, err := net.DialTimeout("udp", udpAddr.String(), 5*time.Second)
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error setting dial timeout on connection to %s: %v", kdcs[i], err))
+			continue
+		}
+		if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil {
+			errs = append(errs, fmt.Sprintf("error setting deadline on connection to %s: %v", kdcs[i], err))
+			continue
+		}
+		// conn is guaranteed to be a UDPConn
+		rb, err := sendUDP(conn.(*net.UDPConn), b)
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error sneding to %s: %v", kdcs[i], err))
+			continue
+		}
+		return rb, nil
+	}
+	return nil, fmt.Errorf("error sending to a KDC: %s", strings.Join(errs, "; "))
+}
+
+// sendUDP sends bytes to connection over UDP.
+func sendUDP(conn *net.UDPConn, b []byte) ([]byte, error) {
+	var r []byte
+	defer conn.Close()
+	_, err := conn.Write(b)
+	if err != nil {
+		return r, fmt.Errorf("error sending to (%s): %v", conn.RemoteAddr().String(), err)
+	}
+	udpbuf := make([]byte, 4096)
+	n, _, err := conn.ReadFrom(udpbuf)
+	r = udpbuf[:n]
+	if err != nil {
+		return r, fmt.Errorf("sending over UDP failed to %s: %v", conn.RemoteAddr().String(), err)
+	}
+	if len(r) < 1 {
+		return r, fmt.Errorf("no response data from %s", conn.RemoteAddr().String())
+	}
+	return r, nil
+}
+
+// sendKDCTCP sends bytes to the KDC via TCP.
+func (cl *Client) sendKDCTCP(realm string, b []byte) ([]byte, error) {
+	var r []byte
+	_, kdcs, err := cl.Config.GetKDCs(realm, true)
+	if err != nil {
+		return r, err
+	}
+	r, err = dialSendTCP(kdcs, b)
+	if err != nil {
+		return r, err
+	}
+	return checkForKRBError(r)
+}
+
+// dialKDCTCP establishes a TCP connection to a KDC.
+func dialSendTCP(kdcs map[int]string, b []byte) ([]byte, error) {
+	var errs []string
+	for i := 1; i <= len(kdcs); i++ {
+		tcpAddr, err := net.ResolveTCPAddr("tcp", kdcs[i])
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error resolving KDC address: %v", err))
+			continue
+		}
+
+		conn, err := net.DialTimeout("tcp", tcpAddr.String(), 5*time.Second)
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error setting dial timeout on connection to %s: %v", kdcs[i], err))
+			continue
+		}
+		if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil {
+			errs = append(errs, fmt.Sprintf("error setting deadline on connection to %s: %v", kdcs[i], err))
+			continue
+		}
+		// conn is guaranteed to be a TCPConn
+		rb, err := sendTCP(conn.(*net.TCPConn), b)
+		if err != nil {
+			errs = append(errs, fmt.Sprintf("error sneding to %s: %v", kdcs[i], err))
+			continue
+		}
+		return rb, nil
+	}
+	return nil, errors.New("error in getting a TCP connection to any of the KDCs")
+}
+
+// sendTCP sends bytes to connection over TCP.
+func sendTCP(conn *net.TCPConn, b []byte) ([]byte, error) {
+	defer conn.Close()
+	var r []byte
+	// RFC 4120 7.2.2 specifies the first 4 bytes indicate the length of the message in big endian order.
+	hb := make([]byte, 4, 4)
+	binary.BigEndian.PutUint32(hb, uint32(len(b)))
+	b = append(hb, b...)
+
+	_, err := conn.Write(b)
+	if err != nil {
+		return r, fmt.Errorf("error sending to KDC (%s): %v", conn.RemoteAddr().String(), err)
+	}
+
+	sh := make([]byte, 4, 4)
+	_, err = conn.Read(sh)
+	if err != nil {
+		return r, fmt.Errorf("error reading response size header: %v", err)
+	}
+	s := binary.BigEndian.Uint32(sh)
+
+	rb := make([]byte, s, s)
+	_, err = io.ReadFull(conn, rb)
+	if err != nil {
+		return r, fmt.Errorf("error reading response: %v", err)
+	}
+	if len(rb) < 1 {
+		return r, fmt.Errorf("no response data from KDC %s", conn.RemoteAddr().String())
+	}
+	return rb, nil
+}
+
+// checkForKRBError checks if the response bytes from the KDC are a KRBError.
+func checkForKRBError(b []byte) ([]byte, error) {
+	var KRBErr messages.KRBError
+	if err := KRBErr.Unmarshal(b); err == nil {
+		return b, KRBErr
+	}
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go b/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go
new file mode 100644
index 0000000..fe20559
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go
@@ -0,0 +1,75 @@
+package client
+
+import (
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/kadmin"
+	"github.com/jcmturner/gokrb5/v8/messages"
+)
+
+// Kpasswd server response codes.
+const (
+	KRB5_KPASSWD_SUCCESS             = 0
+	KRB5_KPASSWD_MALFORMED           = 1
+	KRB5_KPASSWD_HARDERROR           = 2
+	KRB5_KPASSWD_AUTHERROR           = 3
+	KRB5_KPASSWD_SOFTERROR           = 4
+	KRB5_KPASSWD_ACCESSDENIED        = 5
+	KRB5_KPASSWD_BAD_VERSION         = 6
+	KRB5_KPASSWD_INITIAL_FLAG_NEEDED = 7
+)
+
+// ChangePasswd changes the password of the client to the value provided.
+func (cl *Client) ChangePasswd(newPasswd string) (bool, error) {
+	ASReq, err := messages.NewASReqForChgPasswd(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName())
+	if err != nil {
+		return false, err
+	}
+	ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0)
+	if err != nil {
+		return false, err
+	}
+
+	msg, key, err := kadmin.ChangePasswdMsg(cl.Credentials.CName(), cl.Credentials.Domain(), newPasswd, ASRep.Ticket, ASRep.DecryptedEncPart.Key)
+	if err != nil {
+		return false, err
+	}
+	r, err := cl.sendToKPasswd(msg)
+	if err != nil {
+		return false, err
+	}
+	err = r.Decrypt(key)
+	if err != nil {
+		return false, err
+	}
+	if r.ResultCode != KRB5_KPASSWD_SUCCESS {
+		return false, fmt.Errorf("error response from kadmin: code: %d; result: %s; krberror: %v", r.ResultCode, r.Result, r.KRBError)
+	}
+	cl.Credentials.WithPassword(newPasswd)
+	return true, nil
+}
+
+func (cl *Client) sendToKPasswd(msg kadmin.Request) (r kadmin.Reply, err error) {
+	_, kps, err := cl.Config.GetKpasswdServers(cl.Credentials.Domain(), true)
+	if err != nil {
+		return
+	}
+	b, err := msg.Marshal()
+	if err != nil {
+		return
+	}
+	var rb []byte
+	if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit {
+		rb, err = dialSendUDP(kps, b)
+		if err != nil {
+			return
+		}
+	} else {
+		rb, err = dialSendTCP(kps, b)
+		if err != nil {
+			return
+		}
+	}
+	err = r.Unmarshal(rb)
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/session.go b/vendor/github.com/jcmturner/gokrb5/v8/client/session.go
new file mode 100644
index 0000000..f7654d0
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/session.go
@@ -0,0 +1,295 @@
+package client
+
+import (
+	"encoding/json"
+	"fmt"
+	"sort"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// sessions hold TGTs and are keyed on the realm name
+type sessions struct {
+	Entries map[string]*session
+	mux     sync.RWMutex
+}
+
+// destroy erases all sessions
+func (s *sessions) destroy() {
+	s.mux.Lock()
+	defer s.mux.Unlock()
+	for k, e := range s.Entries {
+		e.destroy()
+		delete(s.Entries, k)
+	}
+}
+
+// update replaces a session with the one provided or adds it as a new one
+func (s *sessions) update(sess *session) {
+	s.mux.Lock()
+	defer s.mux.Unlock()
+	// if a session already exists for this, cancel its auto renew.
+	if i, ok := s.Entries[sess.realm]; ok {
+		if i != sess {
+			// Session in the sessions cache is not the same as one provided.
+			// Cancel the one in the cache and add this one.
+			i.mux.Lock()
+			defer i.mux.Unlock()
+			i.cancel <- true
+			s.Entries[sess.realm] = sess
+			return
+		}
+	}
+	// No session for this realm was found so just add it
+	s.Entries[sess.realm] = sess
+}
+
+// get returns the session for the realm specified
+func (s *sessions) get(realm string) (*session, bool) {
+	s.mux.RLock()
+	defer s.mux.RUnlock()
+	sess, ok := s.Entries[realm]
+	return sess, ok
+}
+
+// session holds the TGT details for a realm
+type session struct {
+	realm                string
+	authTime             time.Time
+	endTime              time.Time
+	renewTill            time.Time
+	tgt                  messages.Ticket
+	sessionKey           types.EncryptionKey
+	sessionKeyExpiration time.Time
+	cancel               chan bool
+	mux                  sync.RWMutex
+}
+
+// jsonSession is used to enable marshaling some information of a session in a JSON format
+type jsonSession struct {
+	Realm                string
+	AuthTime             time.Time
+	EndTime              time.Time
+	RenewTill            time.Time
+	SessionKeyExpiration time.Time
+}
+
+// AddSession adds a session for a realm with a TGT to the client's session cache.
+// A goroutine is started to automatically renew the TGT before expiry.
+func (cl *Client) addSession(tgt messages.Ticket, dep messages.EncKDCRepPart) {
+	if strings.ToLower(tgt.SName.NameString[0]) != "krbtgt" {
+		// Not a TGT
+		return
+	}
+	realm := tgt.SName.NameString[len(tgt.SName.NameString)-1]
+	s := &session{
+		realm:                realm,
+		authTime:             dep.AuthTime,
+		endTime:              dep.EndTime,
+		renewTill:            dep.RenewTill,
+		tgt:                  tgt,
+		sessionKey:           dep.Key,
+		sessionKeyExpiration: dep.KeyExpiration,
+	}
+	cl.sessions.update(s)
+	cl.enableAutoSessionRenewal(s)
+	cl.Log("TGT session added for %s (EndTime: %v)", realm, dep.EndTime)
+}
+
+// update overwrites the session details with those from the TGT and decrypted encPart
+func (s *session) update(tgt messages.Ticket, dep messages.EncKDCRepPart) {
+	s.mux.Lock()
+	defer s.mux.Unlock()
+	s.authTime = dep.AuthTime
+	s.endTime = dep.EndTime
+	s.renewTill = dep.RenewTill
+	s.tgt = tgt
+	s.sessionKey = dep.Key
+	s.sessionKeyExpiration = dep.KeyExpiration
+}
+
+// destroy will cancel any auto renewal of the session and set the expiration times to the current time
+func (s *session) destroy() {
+	s.mux.Lock()
+	defer s.mux.Unlock()
+	if s.cancel != nil {
+		s.cancel <- true
+	}
+	s.endTime = time.Now().UTC()
+	s.renewTill = s.endTime
+	s.sessionKeyExpiration = s.endTime
+}
+
+// valid informs if the TGT is still within the valid time window
+func (s *session) valid() bool {
+	s.mux.RLock()
+	defer s.mux.RUnlock()
+	t := time.Now().UTC()
+	if t.Before(s.endTime) && s.authTime.Before(t) {
+		return true
+	}
+	return false
+}
+
+// tgtDetails is a thread safe way to get the session's realm, TGT and session key values
+func (s *session) tgtDetails() (string, messages.Ticket, types.EncryptionKey) {
+	s.mux.RLock()
+	defer s.mux.RUnlock()
+	return s.realm, s.tgt, s.sessionKey
+}
+
+// timeDetails is a thread safe way to get the session's validity time values
+func (s *session) timeDetails() (string, time.Time, time.Time, time.Time, time.Time) {
+	s.mux.RLock()
+	defer s.mux.RUnlock()
+	return s.realm, s.authTime, s.endTime, s.renewTill, s.sessionKeyExpiration
+}
+
+// JSON return information about the held sessions in a JSON format.
+func (s *sessions) JSON() (string, error) {
+	s.mux.RLock()
+	defer s.mux.RUnlock()
+	var js []jsonSession
+	keys := make([]string, 0, len(s.Entries))
+	for k := range s.Entries {
+		keys = append(keys, k)
+	}
+	sort.Strings(keys)
+	for _, k := range keys {
+		r, at, et, rt, kt := s.Entries[k].timeDetails()
+		j := jsonSession{
+			Realm:                r,
+			AuthTime:             at,
+			EndTime:              et,
+			RenewTill:            rt,
+			SessionKeyExpiration: kt,
+		}
+		js = append(js, j)
+	}
+	b, err := json.MarshalIndent(js, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+}
+
+// enableAutoSessionRenewal turns on the automatic renewal for the client's TGT session.
+func (cl *Client) enableAutoSessionRenewal(s *session) {
+	var timer *time.Timer
+	s.mux.Lock()
+	s.cancel = make(chan bool, 1)
+	s.mux.Unlock()
+	go func(s *session) {
+		for {
+			s.mux.RLock()
+			w := (s.endTime.Sub(time.Now().UTC()) * 5) / 6
+			s.mux.RUnlock()
+			if w < 0 {
+				return
+			}
+			timer = time.NewTimer(w)
+			select {
+			case <-timer.C:
+				renewal, err := cl.refreshSession(s)
+				if err != nil {
+					cl.Log("error refreshing session: %v", err)
+				}
+				if !renewal && err == nil {
+					// end this goroutine as there will have been a new login and new auto renewal goroutine created.
+					return
+				}
+			case <-s.cancel:
+				// cancel has been called. Stop the timer and exit.
+				timer.Stop()
+				return
+			}
+		}
+	}(s)
+}
+
+// renewTGT renews the client's TGT session.
+func (cl *Client) renewTGT(s *session) error {
+	realm, tgt, skey := s.tgtDetails()
+	spn := types.PrincipalName{
+		NameType:   nametype.KRB_NT_SRV_INST,
+		NameString: []string{"krbtgt", realm},
+	}
+	_, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, true)
+	if err != nil {
+		return krberror.Errorf(err, krberror.KRBMsgError, "error renewing TGT for %s", realm)
+	}
+	s.update(tgsRep.Ticket, tgsRep.DecryptedEncPart)
+	cl.sessions.update(s)
+	cl.Log("TGT session renewed for %s (EndTime: %v)", realm, tgsRep.DecryptedEncPart.EndTime)
+	return nil
+}
+
+// refreshSession updates either through renewal or creating a new login.
+// The boolean indicates if the update was a renewal.
+func (cl *Client) refreshSession(s *session) (bool, error) {
+	s.mux.RLock()
+	realm := s.realm
+	renewTill := s.renewTill
+	s.mux.RUnlock()
+	cl.Log("refreshing TGT session for %s", realm)
+	if time.Now().UTC().Before(renewTill) {
+		err := cl.renewTGT(s)
+		return true, err
+	}
+	err := cl.realmLogin(realm)
+	return false, err
+}
+
+// ensureValidSession makes sure there is a valid session for the realm
+func (cl *Client) ensureValidSession(realm string) error {
+	s, ok := cl.sessions.get(realm)
+	if ok {
+		s.mux.RLock()
+		d := s.endTime.Sub(s.authTime) / 6
+		if s.endTime.Sub(time.Now().UTC()) > d {
+			s.mux.RUnlock()
+			return nil
+		}
+		s.mux.RUnlock()
+		_, err := cl.refreshSession(s)
+		return err
+	}
+	return cl.realmLogin(realm)
+}
+
+// sessionTGTDetails is a thread safe way to get the TGT and session key values for a realm
+func (cl *Client) sessionTGT(realm string) (tgt messages.Ticket, sessionKey types.EncryptionKey, err error) {
+	err = cl.ensureValidSession(realm)
+	if err != nil {
+		return
+	}
+	s, ok := cl.sessions.get(realm)
+	if !ok {
+		err = fmt.Errorf("could not find TGT session for %s", realm)
+		return
+	}
+	_, tgt, sessionKey = s.tgtDetails()
+	return
+}
+
+// sessionTimes provides the timing information with regards to a session for the realm specified.
+func (cl *Client) sessionTimes(realm string) (authTime, endTime, renewTime, sessionExp time.Time, err error) {
+	s, ok := cl.sessions.get(realm)
+	if !ok {
+		err = fmt.Errorf("could not find TGT session for %s", realm)
+		return
+	}
+	_, authTime, endTime, renewTime, sessionExp = s.timeDetails()
+	return
+}
+
+// spnRealm resolves the realm name of a service principal name
+func (cl *Client) spnRealm(spn types.PrincipalName) string {
+	return cl.Config.ResolveRealm(spn.NameString[len(spn.NameString)-1])
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go b/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go
new file mode 100644
index 0000000..bcd3945
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go
@@ -0,0 +1,93 @@
+package client
+
+import (
+	"encoding/json"
+	"fmt"
+	"log"
+)
+
+// Settings holds optional client settings.
+type Settings struct {
+	disablePAFXFast         bool
+	assumePreAuthentication bool
+	preAuthEType            int32
+	logger                  *log.Logger
+}
+
+// jsonSettings is used when marshaling the Settings details to JSON format.
+type jsonSettings struct {
+	DisablePAFXFast         bool
+	AssumePreAuthentication bool
+}
+
+// NewSettings creates a new client settings struct.
+func NewSettings(settings ...func(*Settings)) *Settings {
+	s := new(Settings)
+	for _, set := range settings {
+		set(s)
+	}
+	return s
+}
+
+// DisablePAFXFAST used to configure the client to not use PA_FX_FAST.
+//
+// s := NewSettings(DisablePAFXFAST(true))
+func DisablePAFXFAST(b bool) func(*Settings) {
+	return func(s *Settings) {
+		s.disablePAFXFast = b
+	}
+}
+
+// DisablePAFXFAST indicates is the client should disable the use of PA_FX_FAST.
+func (s *Settings) DisablePAFXFAST() bool {
+	return s.disablePAFXFast
+}
+
+// AssumePreAuthentication used to configure the client to assume pre-authentication is required.
+//
+// s := NewSettings(AssumePreAuthentication(true))
+func AssumePreAuthentication(b bool) func(*Settings) {
+	return func(s *Settings) {
+		s.assumePreAuthentication = b
+	}
+}
+
+// AssumePreAuthentication indicates if the client should proactively assume using pre-authentication.
+func (s *Settings) AssumePreAuthentication() bool {
+	return s.assumePreAuthentication
+}
+
+// Logger used to configure client with a logger.
+//
+// s := NewSettings(kt, Logger(l))
+func Logger(l *log.Logger) func(*Settings) {
+	return func(s *Settings) {
+		s.logger = l
+	}
+}
+
+// Logger returns the client logger instance.
+func (s *Settings) Logger() *log.Logger {
+	return s.logger
+}
+
+// Log will write to the service's logger if it is configured.
+func (cl *Client) Log(format string, v ...interface{}) {
+	if cl.settings.Logger() != nil {
+		cl.settings.Logger().Output(2, fmt.Sprintf(format, v...))
+	}
+}
+
+// JSON returns a JSON representation of the settings.
+func (s *Settings) JSON() (string, error) {
+	js := jsonSettings{
+		DisablePAFXFast:         s.disablePAFXFast,
+		AssumePreAuthentication: s.assumePreAuthentication,
+	}
+	b, err := json.MarshalIndent(js, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/config/error.go b/vendor/github.com/jcmturner/gokrb5/v8/config/error.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/config/error.go
rename to vendor/github.com/jcmturner/gokrb5/v8/config/error.go
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go b/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go
new file mode 100644
index 0000000..3f22c70
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go
@@ -0,0 +1,141 @@
+package config
+
+import (
+	"fmt"
+	"math/rand"
+	"net"
+	"strconv"
+	"strings"
+
+	"github.com/jcmturner/dnsutils/v2"
+)
+
+// GetKDCs returns the count of KDCs available and a map of KDC host names keyed on preference order.
+func (c *Config) GetKDCs(realm string, tcp bool) (int, map[int]string, error) {
+	if realm == "" {
+		realm = c.LibDefaults.DefaultRealm
+	}
+	kdcs := make(map[int]string)
+	var count int
+
+	// Get the KDCs from the krb5.conf.
+	var ks []string
+	for _, r := range c.Realms {
+		if r.Realm != realm {
+			continue
+		}
+		ks = r.KDC
+	}
+	count = len(ks)
+
+	if count > 0 {
+		// Order the kdcs randomly for preference.
+		kdcs = randServOrder(ks)
+		return count, kdcs, nil
+	}
+
+	if !c.LibDefaults.DNSLookupKDC {
+		return count, kdcs, fmt.Errorf("no KDCs defined in configuration for realm %s", realm)
+	}
+
+	// Use DNS to resolve kerberos SRV records.
+	proto := "udp"
+	if tcp {
+		proto = "tcp"
+	}
+	index, addrs, err := dnsutils.OrderedSRV("kerberos", proto, realm)
+	if err != nil {
+		return count, kdcs, err
+	}
+	if len(addrs) < 1 {
+		return count, kdcs, fmt.Errorf("no KDC SRV records found for realm %s", realm)
+	}
+	count = index
+	for k, v := range addrs {
+		kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port))
+	}
+	return count, kdcs, nil
+}
+
+// GetKpasswdServers returns the count of kpasswd servers available and a map of kpasswd host names keyed on preference order.
+// https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html#realms - see kpasswd_server section
+func (c *Config) GetKpasswdServers(realm string, tcp bool) (int, map[int]string, error) {
+	kdcs := make(map[int]string)
+	var count int
+
+	// Use DNS to resolve kerberos SRV records if configured to do so in krb5.conf.
+	if c.LibDefaults.DNSLookupKDC {
+		proto := "udp"
+		if tcp {
+			proto = "tcp"
+		}
+		c, addrs, err := dnsutils.OrderedSRV("kpasswd", proto, realm)
+		if err != nil {
+			return count, kdcs, err
+		}
+		if c < 1 {
+			c, addrs, err = dnsutils.OrderedSRV("kerberos-adm", proto, realm)
+			if err != nil {
+				return count, kdcs, err
+			}
+		}
+		if len(addrs) < 1 {
+			return count, kdcs, fmt.Errorf("no kpasswd or kadmin SRV records found for realm %s", realm)
+		}
+		count = c
+		for k, v := range addrs {
+			kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port))
+		}
+	} else {
+		// Get the KDCs from the krb5.conf an order them randomly for preference.
+		var ks []string
+		var ka []string
+		for _, r := range c.Realms {
+			if r.Realm == realm {
+				ks = r.KPasswdServer
+				ka = r.AdminServer
+				break
+			}
+		}
+		if len(ks) < 1 {
+			for _, k := range ka {
+				h, _, err := net.SplitHostPort(k)
+				if err != nil {
+					continue
+				}
+				ks = append(ks, h+":464")
+			}
+		}
+		count = len(ks)
+		if count < 1 {
+			return count, kdcs, fmt.Errorf("no kpasswd or kadmin defined in configuration for realm %s", realm)
+		}
+		kdcs = randServOrder(ks)
+	}
+	return count, kdcs, nil
+}
+
+func randServOrder(ks []string) map[int]string {
+	kdcs := make(map[int]string)
+	count := len(ks)
+	i := 1
+	if count > 1 {
+		l := len(ks)
+		for l > 0 {
+			ri := rand.Intn(l)
+			kdcs[i] = ks[ri]
+			if l > 1 {
+				// Remove the entry from the source slice by swapping with the last entry and truncating
+				ks[len(ks)-1], ks[ri] = ks[ri], ks[len(ks)-1]
+				ks = ks[:len(ks)-1]
+				l = len(ks)
+			} else {
+				l = 0
+			}
+			i++
+		}
+	} else {
+		kdcs[i] = ks[0]
+	}
+	return kdcs
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go b/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go
new file mode 100644
index 0000000..a763843
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go
@@ -0,0 +1,728 @@
+// Package config implements KRB5 client and service configuration as described at https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html
+package config
+
+import (
+	"bufio"
+	"encoding/hex"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io"
+	"net"
+	"os"
+	"os/user"
+	"regexp"
+	"strconv"
+	"strings"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// Config represents the KRB5 configuration.
+type Config struct {
+	LibDefaults LibDefaults
+	Realms      []Realm
+	DomainRealm DomainRealm
+	//CaPaths
+	//AppDefaults
+	//Plugins
+}
+
+// WeakETypeList is a list of encryption types that have been deemed weak.
+const WeakETypeList = "des-cbc-crc des-cbc-md4 des-cbc-md5 des-cbc-raw des3-cbc-raw des-hmac-sha1 arcfour-hmac-exp rc4-hmac-exp arcfour-hmac-md5-exp des"
+
+// New creates a new config struct instance.
+func New() *Config {
+	d := make(DomainRealm)
+	return &Config{
+		LibDefaults: newLibDefaults(),
+		DomainRealm: d,
+	}
+}
+
+// LibDefaults represents the [libdefaults] section of the configuration.
+type LibDefaults struct {
+	AllowWeakCrypto bool //default false
+	// ap_req_checksum_type int //unlikely to support this
+	Canonicalize bool          //default false
+	CCacheType   int           //default is 4. unlikely to implement older
+	Clockskew    time.Duration //max allowed skew in seconds, default 300
+	//Default_ccache_name string // default /tmp/krb5cc_%{uid} //Not implementing as will hold in memory
+	DefaultClientKeytabName string //default /usr/local/var/krb5/user/%{euid}/client.keytab
+	DefaultKeytabName       string //default /etc/krb5.keytab
+	DefaultRealm            string
+	DefaultTGSEnctypes      []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
+	DefaultTktEnctypes      []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
+	DefaultTGSEnctypeIDs    []int32  //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
+	DefaultTktEnctypeIDs    []int32  //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
+	DNSCanonicalizeHostname bool     //default true
+	DNSLookupKDC            bool     //default false
+	DNSLookupRealm          bool
+	ExtraAddresses          []net.IP       //Not implementing yet
+	Forwardable             bool           //default false
+	IgnoreAcceptorHostname  bool           //default false
+	K5LoginAuthoritative    bool           //default false
+	K5LoginDirectory        string         //default user's home directory. Must be owned by the user or root
+	KDCDefaultOptions       asn1.BitString //default 0x00000010 (KDC_OPT_RENEWABLE_OK)
+	KDCTimeSync             int            //default 1
+	//kdc_req_checksum_type int //unlikely to implement as for very old KDCs
+	NoAddresses         bool     //default true
+	PermittedEnctypes   []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
+	PermittedEnctypeIDs []int32
+	//plugin_base_dir string //not supporting plugins
+	PreferredPreauthTypes []int         //default “17, 16, 15, 14”, which forces libkrb5 to attempt to use PKINIT if it is supported
+	Proxiable             bool          //default false
+	RDNS                  bool          //default true
+	RealmTryDomains       int           //default -1
+	RenewLifetime         time.Duration //default 0
+	SafeChecksumType      int           //default 8
+	TicketLifetime        time.Duration //default 1 day
+	UDPPreferenceLimit    int           // 1 means to always use tcp. MIT krb5 has a default value of 1465, and it prevents user setting more than 32700.
+	VerifyAPReqNofail     bool          //default false
+}
+
+// Create a new LibDefaults struct.
+func newLibDefaults() LibDefaults {
+	uid := "0"
+	var hdir string
+	usr, _ := user.Current()
+	if usr != nil {
+		uid = usr.Uid
+		hdir = usr.HomeDir
+	}
+	opts := asn1.BitString{}
+	opts.Bytes, _ = hex.DecodeString("00000010")
+	opts.BitLength = len(opts.Bytes) * 8
+	return LibDefaults{
+		CCacheType:              4,
+		Clockskew:               time.Duration(300) * time.Second,
+		DefaultClientKeytabName: fmt.Sprintf("/usr/local/var/krb5/user/%s/client.keytab", uid),
+		DefaultKeytabName:       "/etc/krb5.keytab",
+		DefaultTGSEnctypes:      []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
+		DefaultTktEnctypes:      []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
+		DNSCanonicalizeHostname: true,
+		K5LoginDirectory:        hdir,
+		KDCDefaultOptions:       opts,
+		KDCTimeSync:             1,
+		NoAddresses:             true,
+		PermittedEnctypes:       []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
+		RDNS:                    true,
+		RealmTryDomains:         -1,
+		SafeChecksumType:        8,
+		TicketLifetime:          time.Duration(24) * time.Hour,
+		UDPPreferenceLimit:      1465,
+		PreferredPreauthTypes:   []int{17, 16, 15, 14},
+	}
+}
+
+// Parse the lines of the [libdefaults] section of the configuration into the LibDefaults struct.
+func (l *LibDefaults) parseLines(lines []string) error {
+	for _, line := range lines {
+		//Remove comments after the values
+		if idx := strings.IndexAny(line, "#;"); idx != -1 {
+			line = line[:idx]
+		}
+		line = strings.TrimSpace(line)
+		if line == "" {
+			continue
+		}
+		if !strings.Contains(line, "=") {
+			return InvalidErrorf("libdefaults section line (%s)", line)
+		}
+
+		p := strings.Split(line, "=")
+		key := strings.TrimSpace(strings.ToLower(p[0]))
+		switch key {
+		case "allow_weak_crypto":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.AllowWeakCrypto = v
+		case "canonicalize":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.Canonicalize = v
+		case "ccache_type":
+			p[1] = strings.TrimSpace(p[1])
+			v, err := strconv.ParseUint(p[1], 10, 32)
+			if err != nil || v < 0 || v > 4 {
+				return InvalidErrorf("libdefaults section line (%s)", line)
+			}
+			l.CCacheType = int(v)
+		case "clockskew":
+			d, err := parseDuration(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.Clockskew = d
+		case "default_client_keytab_name":
+			l.DefaultClientKeytabName = strings.TrimSpace(p[1])
+		case "default_keytab_name":
+			l.DefaultKeytabName = strings.TrimSpace(p[1])
+		case "default_realm":
+			l.DefaultRealm = strings.TrimSpace(p[1])
+		case "default_tgs_enctypes":
+			l.DefaultTGSEnctypes = strings.Fields(p[1])
+		case "default_tkt_enctypes":
+			l.DefaultTktEnctypes = strings.Fields(p[1])
+		case "dns_canonicalize_hostname":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.DNSCanonicalizeHostname = v
+		case "dns_lookup_kdc":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.DNSLookupKDC = v
+		case "dns_lookup_realm":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.DNSLookupRealm = v
+		case "extra_addresses":
+			ipStr := strings.TrimSpace(p[1])
+			for _, ip := range strings.Split(ipStr, ",") {
+				if eip := net.ParseIP(ip); eip != nil {
+					l.ExtraAddresses = append(l.ExtraAddresses, eip)
+				}
+			}
+		case "forwardable":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.Forwardable = v
+		case "ignore_acceptor_hostname":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.IgnoreAcceptorHostname = v
+		case "k5login_authoritative":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.K5LoginAuthoritative = v
+		case "k5login_directory":
+			l.K5LoginDirectory = strings.TrimSpace(p[1])
+		case "kdc_default_options":
+			v := strings.TrimSpace(p[1])
+			v = strings.Replace(v, "0x", "", -1)
+			b, err := hex.DecodeString(v)
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.KDCDefaultOptions.Bytes = b
+			l.KDCDefaultOptions.BitLength = len(b) * 8
+		case "kdc_timesync":
+			p[1] = strings.TrimSpace(p[1])
+			v, err := strconv.ParseInt(p[1], 10, 32)
+			if err != nil || v < 0 {
+				return InvalidErrorf("libdefaults section line (%s)", line)
+			}
+			l.KDCTimeSync = int(v)
+		case "noaddresses":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.NoAddresses = v
+		case "permitted_enctypes":
+			l.PermittedEnctypes = strings.Fields(p[1])
+		case "preferred_preauth_types":
+			p[1] = strings.TrimSpace(p[1])
+			t := strings.Split(p[1], ",")
+			var v []int
+			for _, s := range t {
+				i, err := strconv.ParseInt(s, 10, 32)
+				if err != nil {
+					return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+				}
+				v = append(v, int(i))
+			}
+			l.PreferredPreauthTypes = v
+		case "proxiable":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.Proxiable = v
+		case "rdns":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.RDNS = v
+		case "realm_try_domains":
+			p[1] = strings.TrimSpace(p[1])
+			v, err := strconv.ParseInt(p[1], 10, 32)
+			if err != nil || v < -1 {
+				return InvalidErrorf("libdefaults section line (%s)", line)
+			}
+			l.RealmTryDomains = int(v)
+		case "renew_lifetime":
+			d, err := parseDuration(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.RenewLifetime = d
+		case "safe_checksum_type":
+			p[1] = strings.TrimSpace(p[1])
+			v, err := strconv.ParseInt(p[1], 10, 32)
+			if err != nil || v < 0 {
+				return InvalidErrorf("libdefaults section line (%s)", line)
+			}
+			l.SafeChecksumType = int(v)
+		case "ticket_lifetime":
+			d, err := parseDuration(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.TicketLifetime = d
+		case "udp_preference_limit":
+			p[1] = strings.TrimSpace(p[1])
+			v, err := strconv.ParseUint(p[1], 10, 32)
+			if err != nil || v > 32700 {
+				return InvalidErrorf("libdefaults section line (%s)", line)
+			}
+			l.UDPPreferenceLimit = int(v)
+		case "verify_ap_req_nofail":
+			v, err := parseBoolean(p[1])
+			if err != nil {
+				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
+			}
+			l.VerifyAPReqNofail = v
+		}
+	}
+	l.DefaultTGSEnctypeIDs = parseETypes(l.DefaultTGSEnctypes, l.AllowWeakCrypto)
+	l.DefaultTktEnctypeIDs = parseETypes(l.DefaultTktEnctypes, l.AllowWeakCrypto)
+	l.PermittedEnctypeIDs = parseETypes(l.PermittedEnctypes, l.AllowWeakCrypto)
+	return nil
+}
+
+// Realm represents an entry in the [realms] section of the configuration.
+type Realm struct {
+	Realm       string
+	AdminServer []string
+	//auth_to_local //Not implementing for now
+	//auth_to_local_names //Not implementing for now
+	DefaultDomain string
+	KDC           []string
+	KPasswdServer []string //default admin_server:464
+	MasterKDC     []string
+}
+
+// Parse the lines of a [realms] entry into the Realm struct.
+func (r *Realm) parseLines(name string, lines []string) (err error) {
+	r.Realm = name
+	var adminServerFinal bool
+	var KDCFinal bool
+	var kpasswdServerFinal bool
+	var masterKDCFinal bool
+	var ignore bool
+	var c int // counts the depth of blocks within brackets { }
+	for _, line := range lines {
+		if ignore && c > 0 && !strings.Contains(line, "{") && !strings.Contains(line, "}") {
+			continue
+		}
+		//Remove comments after the values
+		if idx := strings.IndexAny(line, "#;"); idx != -1 {
+			line = line[:idx]
+		}
+		line = strings.TrimSpace(line)
+		if line == "" {
+			continue
+		}
+		if !strings.Contains(line, "=") && !strings.Contains(line, "}") {
+			return InvalidErrorf("realms section line (%s)", line)
+		}
+		if strings.Contains(line, "v4_") {
+			ignore = true
+			err = UnsupportedDirective{"v4 configurations are not supported"}
+		}
+		if strings.Contains(line, "{") {
+			c++
+			if ignore {
+				continue
+			}
+		}
+		if strings.Contains(line, "}") {
+			c--
+			if c < 0 {
+				return InvalidErrorf("unpaired curly brackets")
+			}
+			if ignore {
+				if c < 1 {
+					c = 0
+					ignore = false
+				}
+				continue
+			}
+		}
+
+		p := strings.Split(line, "=")
+		key := strings.TrimSpace(strings.ToLower(p[0]))
+		v := strings.TrimSpace(p[1])
+		switch key {
+		case "admin_server":
+			appendUntilFinal(&r.AdminServer, v, &adminServerFinal)
+		case "default_domain":
+			r.DefaultDomain = v
+		case "kdc":
+			if !strings.Contains(v, ":") {
+				// No port number specified default to 88
+				if strings.HasSuffix(v, `*`) {
+					v = strings.TrimSpace(strings.TrimSuffix(v, `*`)) + ":88*"
+				} else {
+					v = strings.TrimSpace(v) + ":88"
+				}
+			}
+			appendUntilFinal(&r.KDC, v, &KDCFinal)
+		case "kpasswd_server":
+			appendUntilFinal(&r.KPasswdServer, v, &kpasswdServerFinal)
+		case "master_kdc":
+			appendUntilFinal(&r.MasterKDC, v, &masterKDCFinal)
+		}
+	}
+	//default for Kpasswd_server = admin_server:464
+	if len(r.KPasswdServer) < 1 {
+		for _, a := range r.AdminServer {
+			s := strings.Split(a, ":")
+			r.KPasswdServer = append(r.KPasswdServer, s[0]+":464")
+		}
+	}
+	return
+}
+
+// Parse the lines of the [realms] section of the configuration into an slice of Realm structs.
+func parseRealms(lines []string) (realms []Realm, err error) {
+	var name string
+	var start int
+	var c int
+	for i, l := range lines {
+		//Remove comments after the values
+		if idx := strings.IndexAny(l, "#;"); idx != -1 {
+			l = l[:idx]
+		}
+		l = strings.TrimSpace(l)
+		if l == "" {
+			continue
+		}
+		//if strings.Contains(l, "v4_") {
+		//	return nil, errors.New("v4 configurations are not supported in Realms section")
+		//}
+		if strings.Contains(l, "{") {
+			c++
+			if !strings.Contains(l, "=") {
+				return nil, fmt.Errorf("realm configuration line invalid: %s", l)
+			}
+			if c == 1 {
+				start = i
+				p := strings.Split(l, "=")
+				name = strings.TrimSpace(p[0])
+			}
+		}
+		if strings.Contains(l, "}") {
+			if c < 1 {
+				// but not started a block!!!
+				return nil, errors.New("invalid Realms section in configuration")
+			}
+			c--
+			if c == 0 {
+				var r Realm
+				e := r.parseLines(name, lines[start+1:i])
+				if e != nil {
+					if _, ok := e.(UnsupportedDirective); !ok {
+						err = e
+						return
+					}
+					err = e
+				}
+				realms = append(realms, r)
+			}
+		}
+	}
+	return
+}
+
+// DomainRealm maps the domains to realms representing the [domain_realm] section of the configuration.
+type DomainRealm map[string]string
+
+// Parse the lines of the [domain_realm] section of the configuration and add to the mapping.
+func (d *DomainRealm) parseLines(lines []string) error {
+	for _, line := range lines {
+		//Remove comments after the values
+		if idx := strings.IndexAny(line, "#;"); idx != -1 {
+			line = line[:idx]
+		}
+		if strings.TrimSpace(line) == "" {
+			continue
+		}
+		if !strings.Contains(line, "=") {
+			return InvalidErrorf("realm line (%s)", line)
+		}
+		p := strings.Split(line, "=")
+		domain := strings.TrimSpace(strings.ToLower(p[0]))
+		realm := strings.TrimSpace(p[1])
+		d.addMapping(domain, realm)
+	}
+	return nil
+}
+
+// Add a domain to realm mapping.
+func (d *DomainRealm) addMapping(domain, realm string) {
+	(*d)[domain] = realm
+}
+
+// Delete a domain to realm mapping.
+func (d *DomainRealm) deleteMapping(domain, realm string) {
+	delete(*d, domain)
+}
+
+// ResolveRealm resolves the kerberos realm for the specified domain name from the domain to realm mapping.
+// The most specific mapping is returned.
+func (c *Config) ResolveRealm(domainName string) string {
+	domainName = strings.TrimSuffix(domainName, ".")
+
+	// Try to match the entire hostname first
+	if r, ok := c.DomainRealm[domainName]; ok {
+		return r
+	}
+
+	// Try to match all DNS domain parts
+	periods := strings.Count(domainName, ".") + 1
+	for i := 2; i <= periods; i++ {
+		z := strings.SplitN(domainName, ".", i)
+		if r, ok := c.DomainRealm["."+z[len(z)-1]]; ok {
+			return r
+		}
+	}
+	return c.LibDefaults.DefaultRealm
+}
+
+// Load the KRB5 configuration from the specified file path.
+func Load(cfgPath string) (*Config, error) {
+	fh, err := os.Open(cfgPath)
+	if err != nil {
+		return nil, errors.New("configuration file could not be opened: " + cfgPath + " " + err.Error())
+	}
+	defer fh.Close()
+	scanner := bufio.NewScanner(fh)
+	return NewFromScanner(scanner)
+}
+
+// NewFromString creates a new Config struct from a string.
+func NewFromString(s string) (*Config, error) {
+	reader := strings.NewReader(s)
+	return NewFromReader(reader)
+}
+
+// NewFromReader creates a new Config struct from an io.Reader.
+func NewFromReader(r io.Reader) (*Config, error) {
+	scanner := bufio.NewScanner(r)
+	return NewFromScanner(scanner)
+}
+
+// NewFromScanner creates a new Config struct from a bufio.Scanner.
+func NewFromScanner(scanner *bufio.Scanner) (*Config, error) {
+	c := New()
+	var e error
+	sections := make(map[int]string)
+	var sectionLineNum []int
+	var lines []string
+	for scanner.Scan() {
+		// Skip comments and blank lines
+		if matched, _ := regexp.MatchString(`^\s*(#|;|\n)`, scanner.Text()); matched {
+			continue
+		}
+		if matched, _ := regexp.MatchString(`^\s*\[libdefaults\]\s*`, scanner.Text()); matched {
+			sections[len(lines)] = "libdefaults"
+			sectionLineNum = append(sectionLineNum, len(lines))
+			continue
+		}
+		if matched, _ := regexp.MatchString(`^\s*\[realms\]\s*`, scanner.Text()); matched {
+			sections[len(lines)] = "realms"
+			sectionLineNum = append(sectionLineNum, len(lines))
+			continue
+		}
+		if matched, _ := regexp.MatchString(`^\s*\[domain_realm\]\s*`, scanner.Text()); matched {
+			sections[len(lines)] = "domain_realm"
+			sectionLineNum = append(sectionLineNum, len(lines))
+			continue
+		}
+		if matched, _ := regexp.MatchString(`^\s*\[.*\]\s*`, scanner.Text()); matched {
+			sections[len(lines)] = "unknown_section"
+			sectionLineNum = append(sectionLineNum, len(lines))
+			continue
+		}
+		lines = append(lines, scanner.Text())
+	}
+	for i, start := range sectionLineNum {
+		var end int
+		if i+1 >= len(sectionLineNum) {
+			end = len(lines)
+		} else {
+			end = sectionLineNum[i+1]
+		}
+		switch section := sections[start]; section {
+		case "libdefaults":
+			err := c.LibDefaults.parseLines(lines[start:end])
+			if err != nil {
+				if _, ok := err.(UnsupportedDirective); !ok {
+					return nil, fmt.Errorf("error processing libdefaults section: %v", err)
+				}
+				e = err
+			}
+		case "realms":
+			realms, err := parseRealms(lines[start:end])
+			if err != nil {
+				if _, ok := err.(UnsupportedDirective); !ok {
+					return nil, fmt.Errorf("error processing realms section: %v", err)
+				}
+				e = err
+			}
+			c.Realms = realms
+		case "domain_realm":
+			err := c.DomainRealm.parseLines(lines[start:end])
+			if err != nil {
+				if _, ok := err.(UnsupportedDirective); !ok {
+					return nil, fmt.Errorf("error processing domaain_realm section: %v", err)
+				}
+				e = err
+			}
+		}
+	}
+	return c, e
+}
+
+// Parse a space delimited list of ETypes into a list of EType numbers optionally filtering out weak ETypes.
+func parseETypes(s []string, w bool) []int32 {
+	var eti []int32
+	for _, et := range s {
+		if !w {
+			var weak bool
+			for _, wet := range strings.Fields(WeakETypeList) {
+				if et == wet {
+					weak = true
+					break
+				}
+			}
+			if weak {
+				continue
+			}
+		}
+		i := etypeID.EtypeSupported(et)
+		if i != 0 {
+			eti = append(eti, i)
+		}
+	}
+	return eti
+}
+
+// Parse a time duration string in the configuration to a golang time.Duration.
+func parseDuration(s string) (time.Duration, error) {
+	s = strings.Replace(strings.TrimSpace(s), " ", "", -1)
+
+	// handle Nd[NmNs]
+	if strings.Contains(s, "d") {
+		ds := strings.SplitN(s, "d", 2)
+		dn, err := strconv.ParseUint(ds[0], 10, 32)
+		if err != nil {
+			return time.Duration(0), errors.New("invalid time duration")
+		}
+		d := time.Duration(dn*24) * time.Hour
+		if ds[1] != "" {
+			dp, err := time.ParseDuration(ds[1])
+			if err != nil {
+				return time.Duration(0), errors.New("invalid time duration")
+			}
+			d = d + dp
+		}
+		return d, nil
+	}
+
+	// handle Nm[Ns]
+	d, err := time.ParseDuration(s)
+	if err == nil {
+		return d, nil
+	}
+
+	// handle N
+	v, err := strconv.ParseUint(s, 10, 32)
+	if err == nil && v > 0 {
+		return time.Duration(v) * time.Second, nil
+	}
+
+	// handle h:m[:s]
+	if strings.Contains(s, ":") {
+		t := strings.Split(s, ":")
+		if 2 > len(t) || len(t) > 3 {
+			return time.Duration(0), errors.New("invalid time duration value")
+		}
+		var i []int
+		for _, n := range t {
+			j, err := strconv.ParseInt(n, 10, 16)
+			if err != nil {
+				return time.Duration(0), errors.New("invalid time duration value")
+			}
+			i = append(i, int(j))
+		}
+		d := time.Duration(i[0])*time.Hour + time.Duration(i[1])*time.Minute
+		if len(i) == 3 {
+			d = d + time.Duration(i[2])*time.Second
+		}
+		return d, nil
+	}
+	return time.Duration(0), errors.New("invalid time duration value")
+}
+
+// Parse possible boolean values to golang bool.
+func parseBoolean(s string) (bool, error) {
+	s = strings.TrimSpace(s)
+	v, err := strconv.ParseBool(s)
+	if err == nil {
+		return v, nil
+	}
+	switch strings.ToLower(s) {
+	case "yes":
+		return true, nil
+	case "y":
+		return true, nil
+	case "no":
+		return false, nil
+	case "n":
+		return false, nil
+	}
+	return false, errors.New("invalid boolean value")
+}
+
+// Parse array of strings but stop if an asterisk is placed at the end of a line.
+func appendUntilFinal(s *[]string, value string, final *bool) {
+	if *final {
+		return
+	}
+	if last := len(value) - 1; last >= 0 && value[last] == '*' {
+		*final = true
+		value = value[:len(value)-1]
+	}
+	*s = append(*s, value)
+}
+
+// JSON return details of the config in a JSON format.
+func (c *Config) JSON() (string, error) {
+	b, err := json.MarshalIndent(c, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go b/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go
new file mode 100644
index 0000000..c3b35c7
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go
@@ -0,0 +1,333 @@
+package credentials
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"io/ioutil"
+	"strings"
+	"time"
+	"unsafe"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+const (
+	headerFieldTagKDCOffset = 1
+)
+
+// CCache is the file credentials cache as define here: https://web.mit.edu/kerberos/krb5-latest/doc/formats/ccache_file_format.html
+type CCache struct {
+	Version          uint8
+	Header           header
+	DefaultPrincipal principal
+	Credentials      []*Credential
+	Path             string
+}
+
+type header struct {
+	length uint16
+	fields []headerField
+}
+
+type headerField struct {
+	tag    uint16
+	length uint16
+	value  []byte
+}
+
+// Credential cache entry principal struct.
+type principal struct {
+	Realm         string
+	PrincipalName types.PrincipalName
+}
+
+// Credential holds a Kerberos client's ccache credential information.
+type Credential struct {
+	Client       principal
+	Server       principal
+	Key          types.EncryptionKey
+	AuthTime     time.Time
+	StartTime    time.Time
+	EndTime      time.Time
+	RenewTill    time.Time
+	IsSKey       bool
+	TicketFlags  asn1.BitString
+	Addresses    []types.HostAddress
+	AuthData     []types.AuthorizationDataEntry
+	Ticket       []byte
+	SecondTicket []byte
+}
+
+// LoadCCache loads a credential cache file into a CCache type.
+func LoadCCache(cpath string) (*CCache, error) {
+	c := new(CCache)
+	b, err := ioutil.ReadFile(cpath)
+	if err != nil {
+		return c, err
+	}
+	err = c.Unmarshal(b)
+	return c, err
+}
+
+// Unmarshal a byte slice of credential cache data into CCache type.
+func (c *CCache) Unmarshal(b []byte) error {
+	p := 0
+	//The first byte of the file always has the value 5
+	if int8(b[p]) != 5 {
+		return errors.New("Invalid credential cache data. First byte does not equal 5")
+	}
+	p++
+	//Get credential cache version
+	//The second byte contains the version number (1 to 4)
+	c.Version = b[p]
+	if c.Version < 1 || c.Version > 4 {
+		return errors.New("Invalid credential cache data. Keytab version is not within 1 to 4")
+	}
+	p++
+	//Version 1 or 2 of the file format uses native byte order for integer representations. Versions 3 & 4 always uses big-endian byte order
+	var endian binary.ByteOrder
+	endian = binary.BigEndian
+	if (c.Version == 1 || c.Version == 2) && isNativeEndianLittle() {
+		endian = binary.LittleEndian
+	}
+	if c.Version == 4 {
+		err := parseHeader(b, &p, c, &endian)
+		if err != nil {
+			return err
+		}
+	}
+	c.DefaultPrincipal = parsePrincipal(b, &p, c, &endian)
+	for p < len(b) {
+		cred, err := parseCredential(b, &p, c, &endian)
+		if err != nil {
+			return err
+		}
+		c.Credentials = append(c.Credentials, cred)
+	}
+	return nil
+}
+
+func parseHeader(b []byte, p *int, c *CCache, e *binary.ByteOrder) error {
+	if c.Version != 4 {
+		return errors.New("Credentials cache version is not 4 so there is no header to parse.")
+	}
+	h := header{}
+	h.length = uint16(readInt16(b, p, e))
+	for *p <= int(h.length) {
+		f := headerField{}
+		f.tag = uint16(readInt16(b, p, e))
+		f.length = uint16(readInt16(b, p, e))
+		f.value = b[*p : *p+int(f.length)]
+		*p += int(f.length)
+		if !f.valid() {
+			return errors.New("Invalid credential cache header found")
+		}
+		h.fields = append(h.fields, f)
+	}
+	c.Header = h
+	return nil
+}
+
+// Parse the Keytab bytes of a principal into a Keytab entry's principal.
+func parsePrincipal(b []byte, p *int, c *CCache, e *binary.ByteOrder) (princ principal) {
+	if c.Version != 1 {
+		//Name Type is omitted in version 1
+		princ.PrincipalName.NameType = readInt32(b, p, e)
+	}
+	nc := int(readInt32(b, p, e))
+	if c.Version == 1 {
+		//In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2
+		nc--
+	}
+	lenRealm := readInt32(b, p, e)
+	princ.Realm = string(readBytes(b, p, int(lenRealm), e))
+	for i := 0; i < nc; i++ {
+		l := readInt32(b, p, e)
+		princ.PrincipalName.NameString = append(princ.PrincipalName.NameString, string(readBytes(b, p, int(l), e)))
+	}
+	return princ
+}
+
+func parseCredential(b []byte, p *int, c *CCache, e *binary.ByteOrder) (cred *Credential, err error) {
+	cred = new(Credential)
+	cred.Client = parsePrincipal(b, p, c, e)
+	cred.Server = parsePrincipal(b, p, c, e)
+	key := types.EncryptionKey{}
+	key.KeyType = int32(readInt16(b, p, e))
+	if c.Version == 3 {
+		//repeated twice in version 3
+		key.KeyType = int32(readInt16(b, p, e))
+	}
+	key.KeyValue = readData(b, p, e)
+	cred.Key = key
+	cred.AuthTime = readTimestamp(b, p, e)
+	cred.StartTime = readTimestamp(b, p, e)
+	cred.EndTime = readTimestamp(b, p, e)
+	cred.RenewTill = readTimestamp(b, p, e)
+	if ik := readInt8(b, p, e); ik == 0 {
+		cred.IsSKey = false
+	} else {
+		cred.IsSKey = true
+	}
+	cred.TicketFlags = types.NewKrbFlags()
+	cred.TicketFlags.Bytes = readBytes(b, p, 4, e)
+	l := int(readInt32(b, p, e))
+	cred.Addresses = make([]types.HostAddress, l, l)
+	for i := range cred.Addresses {
+		cred.Addresses[i] = readAddress(b, p, e)
+	}
+	l = int(readInt32(b, p, e))
+	cred.AuthData = make([]types.AuthorizationDataEntry, l, l)
+	for i := range cred.AuthData {
+		cred.AuthData[i] = readAuthDataEntry(b, p, e)
+	}
+	cred.Ticket = readData(b, p, e)
+	cred.SecondTicket = readData(b, p, e)
+	return
+}
+
+// GetClientPrincipalName returns a PrincipalName type for the client the credentials cache is for.
+func (c *CCache) GetClientPrincipalName() types.PrincipalName {
+	return c.DefaultPrincipal.PrincipalName
+}
+
+// GetClientRealm returns the reals of the client the credentials cache is for.
+func (c *CCache) GetClientRealm() string {
+	return c.DefaultPrincipal.Realm
+}
+
+// GetClientCredentials returns a Credentials object representing the client of the credentials cache.
+func (c *CCache) GetClientCredentials() *Credentials {
+	return &Credentials{
+		username: c.DefaultPrincipal.PrincipalName.PrincipalNameString(),
+		realm:    c.GetClientRealm(),
+		cname:    c.DefaultPrincipal.PrincipalName,
+	}
+}
+
+// Contains tests if the cache contains a credential for the provided server PrincipalName
+func (c *CCache) Contains(p types.PrincipalName) bool {
+	for _, cred := range c.Credentials {
+		if cred.Server.PrincipalName.Equal(p) {
+			return true
+		}
+	}
+	return false
+}
+
+// GetEntry returns a specific credential for the PrincipalName provided.
+func (c *CCache) GetEntry(p types.PrincipalName) (*Credential, bool) {
+	cred := new(Credential)
+	var found bool
+	for i := range c.Credentials {
+		if c.Credentials[i].Server.PrincipalName.Equal(p) {
+			cred = c.Credentials[i]
+			found = true
+			break
+		}
+	}
+	if !found {
+		return cred, false
+	}
+	return cred, true
+}
+
+// GetEntries filters out configuration entries an returns a slice of credentials.
+func (c *CCache) GetEntries() []*Credential {
+	creds := make([]*Credential, 0)
+	for _, cred := range c.Credentials {
+		// Filter out configuration entries
+		if strings.HasPrefix(cred.Server.Realm, "X-CACHECONF") {
+			continue
+		}
+		creds = append(creds, cred)
+	}
+	return creds
+}
+
+func (h *headerField) valid() bool {
+	// See https://web.mit.edu/kerberos/krb5-latest/doc/formats/ccache_file_format.html - Header format
+	switch h.tag {
+	case headerFieldTagKDCOffset:
+		if h.length != 8 || len(h.value) != 8 {
+			return false
+		}
+		return true
+	}
+	return false
+}
+
+func readData(b []byte, p *int, e *binary.ByteOrder) []byte {
+	l := readInt32(b, p, e)
+	return readBytes(b, p, int(l), e)
+}
+
+func readAddress(b []byte, p *int, e *binary.ByteOrder) types.HostAddress {
+	a := types.HostAddress{}
+	a.AddrType = int32(readInt16(b, p, e))
+	a.Address = readData(b, p, e)
+	return a
+}
+
+func readAuthDataEntry(b []byte, p *int, e *binary.ByteOrder) types.AuthorizationDataEntry {
+	a := types.AuthorizationDataEntry{}
+	a.ADType = int32(readInt16(b, p, e))
+	a.ADData = readData(b, p, e)
+	return a
+}
+
+// Read bytes representing a timestamp.
+func readTimestamp(b []byte, p *int, e *binary.ByteOrder) time.Time {
+	return time.Unix(int64(readInt32(b, p, e)), 0)
+}
+
+// Read bytes representing an eight bit integer.
+func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8) {
+	buf := bytes.NewBuffer(b[*p : *p+1])
+	binary.Read(buf, *e, &i)
+	*p++
+	return
+}
+
+// Read bytes representing a sixteen bit integer.
+func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16) {
+	buf := bytes.NewBuffer(b[*p : *p+2])
+	binary.Read(buf, *e, &i)
+	*p += 2
+	return
+}
+
+// Read bytes representing a thirty two bit integer.
+func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32) {
+	buf := bytes.NewBuffer(b[*p : *p+4])
+	binary.Read(buf, *e, &i)
+	*p += 4
+	return
+}
+
+func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) []byte {
+	buf := bytes.NewBuffer(b[*p : *p+s])
+	r := make([]byte, s)
+	binary.Read(buf, *e, &r)
+	*p += s
+	return r
+}
+
+func isNativeEndianLittle() bool {
+	var x = 0x012345678
+	var p = unsafe.Pointer(&x)
+	var bp = (*[4]byte)(p)
+
+	var endian bool
+	if 0x01 == bp[0] {
+		endian = false
+	} else if (0x78 & 0xff) == (bp[0] & 0xff) {
+		endian = true
+	} else {
+		// Default to big endian
+		endian = false
+	}
+	return endian
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go b/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go
new file mode 100644
index 0000000..bddbc7e
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go
@@ -0,0 +1,405 @@
+// Package credentials provides credentials management for Kerberos 5 authentication.
+package credentials
+
+import (
+	"bytes"
+	"encoding/gob"
+	"encoding/json"
+	"time"
+
+	"github.com/hashicorp/go-uuid"
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+	"github.com/jcmturner/gokrb5/v8/keytab"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+const (
+	// AttributeKeyADCredentials assigned number for AD credentials.
+	AttributeKeyADCredentials = "gokrb5AttributeKeyADCredentials"
+)
+
+// Credentials struct for a user.
+// Contains either a keytab, password or both.
+// Keytabs are used over passwords if both are defined.
+type Credentials struct {
+	username        string
+	displayName     string
+	realm           string
+	cname           types.PrincipalName
+	keytab          *keytab.Keytab
+	password        string
+	attributes      map[string]interface{}
+	validUntil      time.Time
+	authenticated   bool
+	human           bool
+	authTime        time.Time
+	groupMembership map[string]bool
+	sessionID       string
+}
+
+// marshalCredentials is used to enable marshaling and unmarshaling of credentials
+// without having exported fields on the Credentials struct
+type marshalCredentials struct {
+	Username        string
+	DisplayName     string
+	Realm           string
+	CName           types.PrincipalName `json:"-"`
+	Keytab          bool
+	Password        bool
+	Attributes      map[string]interface{} `json:"-"`
+	ValidUntil      time.Time
+	Authenticated   bool
+	Human           bool
+	AuthTime        time.Time
+	GroupMembership map[string]bool `json:"-"`
+	SessionID       string
+}
+
+// ADCredentials contains information obtained from the PAC.
+type ADCredentials struct {
+	EffectiveName       string
+	FullName            string
+	UserID              int
+	PrimaryGroupID      int
+	LogOnTime           time.Time
+	LogOffTime          time.Time
+	PasswordLastSet     time.Time
+	GroupMembershipSIDs []string
+	LogonDomainName     string
+	LogonDomainID       string
+	LogonServer         string
+}
+
+// New creates a new Credentials instance.
+func New(username string, realm string) *Credentials {
+	uid, err := uuid.GenerateUUID()
+	if err != nil {
+		uid = "00unique-sess-ions-uuid-unavailable0"
+	}
+	return &Credentials{
+		username:        username,
+		displayName:     username,
+		realm:           realm,
+		cname:           types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, username),
+		keytab:          keytab.New(),
+		attributes:      make(map[string]interface{}),
+		groupMembership: make(map[string]bool),
+		sessionID:       uid,
+		human:           true,
+	}
+}
+
+// NewFromPrincipalName creates a new Credentials instance with the user details provides as a PrincipalName type.
+func NewFromPrincipalName(cname types.PrincipalName, realm string) *Credentials {
+	c := New(cname.PrincipalNameString(), realm)
+	c.cname = cname
+	return c
+}
+
+// WithKeytab sets the Keytab in the Credentials struct.
+func (c *Credentials) WithKeytab(kt *keytab.Keytab) *Credentials {
+	c.keytab = kt
+	c.password = ""
+	return c
+}
+
+// Keytab returns the credential's Keytab.
+func (c *Credentials) Keytab() *keytab.Keytab {
+	return c.keytab
+}
+
+// HasKeytab queries if the Credentials has a keytab defined.
+func (c *Credentials) HasKeytab() bool {
+	if c.keytab != nil && len(c.keytab.Entries) > 0 {
+		return true
+	}
+	return false
+}
+
+// WithPassword sets the password in the Credentials struct.
+func (c *Credentials) WithPassword(password string) *Credentials {
+	c.password = password
+	c.keytab = keytab.New() // clear any keytab
+	return c
+}
+
+// Password returns the credential's password.
+func (c *Credentials) Password() string {
+	return c.password
+}
+
+// HasPassword queries if the Credentials has a password defined.
+func (c *Credentials) HasPassword() bool {
+	if c.password != "" {
+		return true
+	}
+	return false
+}
+
+// SetValidUntil sets the expiry time of the credentials
+func (c *Credentials) SetValidUntil(t time.Time) {
+	c.validUntil = t
+}
+
+// SetADCredentials adds ADCredentials attributes to the credentials
+func (c *Credentials) SetADCredentials(a ADCredentials) {
+	c.SetAttribute(AttributeKeyADCredentials, a)
+	if a.FullName != "" {
+		c.SetDisplayName(a.FullName)
+	}
+	if a.EffectiveName != "" {
+		c.SetUserName(a.EffectiveName)
+	}
+	for i := range a.GroupMembershipSIDs {
+		c.AddAuthzAttribute(a.GroupMembershipSIDs[i])
+	}
+}
+
+// GetADCredentials returns ADCredentials attributes sorted in the credential
+func (c *Credentials) GetADCredentials() ADCredentials {
+	if a, ok := c.attributes[AttributeKeyADCredentials].(ADCredentials); ok {
+		return a
+	}
+	return ADCredentials{}
+}
+
+// Methods to implement goidentity.Identity interface
+
+// UserName returns the credential's username.
+func (c *Credentials) UserName() string {
+	return c.username
+}
+
+// SetUserName sets the username value on the credential.
+func (c *Credentials) SetUserName(s string) {
+	c.username = s
+}
+
+// CName returns the credential's client principal name.
+func (c *Credentials) CName() types.PrincipalName {
+	return c.cname
+}
+
+// SetCName sets the client principal name on the credential.
+func (c *Credentials) SetCName(pn types.PrincipalName) {
+	c.cname = pn
+}
+
+// Domain returns the credential's domain.
+func (c *Credentials) Domain() string {
+	return c.realm
+}
+
+// SetDomain sets the domain value on the credential.
+func (c *Credentials) SetDomain(s string) {
+	c.realm = s
+}
+
+// Realm returns the credential's realm. Same as the domain.
+func (c *Credentials) Realm() string {
+	return c.Domain()
+}
+
+// SetRealm sets the realm value on the credential. Same as the domain
+func (c *Credentials) SetRealm(s string) {
+	c.SetDomain(s)
+}
+
+// DisplayName returns the credential's display name.
+func (c *Credentials) DisplayName() string {
+	return c.displayName
+}
+
+// SetDisplayName sets the display name value on the credential.
+func (c *Credentials) SetDisplayName(s string) {
+	c.displayName = s
+}
+
+// Human returns if the  credential represents a human or not.
+func (c *Credentials) Human() bool {
+	return c.human
+}
+
+// SetHuman sets the credential as human.
+func (c *Credentials) SetHuman(b bool) {
+	c.human = b
+}
+
+// AuthTime returns the time the credential was authenticated.
+func (c *Credentials) AuthTime() time.Time {
+	return c.authTime
+}
+
+// SetAuthTime sets the time the credential was authenticated.
+func (c *Credentials) SetAuthTime(t time.Time) {
+	c.authTime = t
+}
+
+// AuthzAttributes returns the credentials authorizing attributes.
+func (c *Credentials) AuthzAttributes() []string {
+	s := make([]string, len(c.groupMembership))
+	i := 0
+	for a := range c.groupMembership {
+		s[i] = a
+		i++
+	}
+	return s
+}
+
+// Authenticated indicates if the credential has been successfully authenticated or not.
+func (c *Credentials) Authenticated() bool {
+	return c.authenticated
+}
+
+// SetAuthenticated sets the credential as having been successfully authenticated.
+func (c *Credentials) SetAuthenticated(b bool) {
+	c.authenticated = b
+}
+
+// AddAuthzAttribute adds an authorization attribute to the credential.
+func (c *Credentials) AddAuthzAttribute(a string) {
+	c.groupMembership[a] = true
+}
+
+// RemoveAuthzAttribute removes an authorization attribute from the credential.
+func (c *Credentials) RemoveAuthzAttribute(a string) {
+	if _, ok := c.groupMembership[a]; !ok {
+		return
+	}
+	delete(c.groupMembership, a)
+}
+
+// EnableAuthzAttribute toggles an authorization attribute to an enabled state on the credential.
+func (c *Credentials) EnableAuthzAttribute(a string) {
+	if enabled, ok := c.groupMembership[a]; ok && !enabled {
+		c.groupMembership[a] = true
+	}
+}
+
+// DisableAuthzAttribute toggles an authorization attribute to a disabled state on the credential.
+func (c *Credentials) DisableAuthzAttribute(a string) {
+	if enabled, ok := c.groupMembership[a]; ok && enabled {
+		c.groupMembership[a] = false
+	}
+}
+
+// Authorized indicates if the credential has the specified authorizing attribute.
+func (c *Credentials) Authorized(a string) bool {
+	if enabled, ok := c.groupMembership[a]; ok && enabled {
+		return true
+	}
+	return false
+}
+
+// SessionID returns the credential's session ID.
+func (c *Credentials) SessionID() string {
+	return c.sessionID
+}
+
+// Expired indicates if the credential has expired.
+func (c *Credentials) Expired() bool {
+	if !c.validUntil.IsZero() && time.Now().UTC().After(c.validUntil) {
+		return true
+	}
+	return false
+}
+
+// ValidUntil returns the credential's valid until date
+func (c *Credentials) ValidUntil() time.Time {
+	return c.validUntil
+}
+
+// Attributes returns the Credentials' attributes map.
+func (c *Credentials) Attributes() map[string]interface{} {
+	return c.attributes
+}
+
+// SetAttribute sets the value of an attribute.
+func (c *Credentials) SetAttribute(k string, v interface{}) {
+	c.attributes[k] = v
+}
+
+// SetAttributes replaces the attributes map with the one provided.
+func (c *Credentials) SetAttributes(a map[string]interface{}) {
+	c.attributes = a
+}
+
+// RemoveAttribute deletes an attribute from the attribute map that has the key provided.
+func (c *Credentials) RemoveAttribute(k string) {
+	delete(c.attributes, k)
+}
+
+// Marshal the Credentials into a byte slice
+func (c *Credentials) Marshal() ([]byte, error) {
+	gob.Register(map[string]interface{}{})
+	gob.Register(ADCredentials{})
+	buf := new(bytes.Buffer)
+	enc := gob.NewEncoder(buf)
+	mc := marshalCredentials{
+		Username:        c.username,
+		DisplayName:     c.displayName,
+		Realm:           c.realm,
+		CName:           c.cname,
+		Keytab:          c.HasKeytab(),
+		Password:        c.HasPassword(),
+		Attributes:      c.attributes,
+		ValidUntil:      c.validUntil,
+		Authenticated:   c.authenticated,
+		Human:           c.human,
+		AuthTime:        c.authTime,
+		GroupMembership: c.groupMembership,
+		SessionID:       c.sessionID,
+	}
+	err := enc.Encode(&mc)
+	if err != nil {
+		return []byte{}, err
+	}
+	return buf.Bytes(), nil
+}
+
+// Unmarshal a byte slice into Credentials
+func (c *Credentials) Unmarshal(b []byte) error {
+	gob.Register(map[string]interface{}{})
+	gob.Register(ADCredentials{})
+	mc := new(marshalCredentials)
+	buf := bytes.NewBuffer(b)
+	dec := gob.NewDecoder(buf)
+	err := dec.Decode(mc)
+	if err != nil {
+		return err
+	}
+	c.username = mc.Username
+	c.displayName = mc.DisplayName
+	c.realm = mc.Realm
+	c.cname = mc.CName
+	c.attributes = mc.Attributes
+	c.validUntil = mc.ValidUntil
+	c.authenticated = mc.Authenticated
+	c.human = mc.Human
+	c.authTime = mc.AuthTime
+	c.groupMembership = mc.GroupMembership
+	c.sessionID = mc.SessionID
+	return nil
+}
+
+// JSON return details of the Credentials in a JSON format.
+func (c *Credentials) JSON() (string, error) {
+	mc := marshalCredentials{
+		Username:      c.username,
+		DisplayName:   c.displayName,
+		Realm:         c.realm,
+		CName:         c.cname,
+		Keytab:        c.HasKeytab(),
+		Password:      c.HasPassword(),
+		ValidUntil:    c.validUntil,
+		Authenticated: c.authenticated,
+		Human:         c.human,
+		AuthTime:      c.authTime,
+		SessionID:     c.sessionID,
+	}
+	b, err := json.MarshalIndent(mc, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go
new file mode 100644
index 0000000..dd8babd
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go
@@ -0,0 +1,129 @@
+package crypto
+
+import (
+	"crypto/aes"
+	"crypto/hmac"
+	"crypto/sha1"
+	"hash"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3961"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3962"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// RFC 3962
+
+// Aes128CtsHmacSha96 implements Kerberos encryption type aes128-cts-hmac-sha1-96
+type Aes128CtsHmacSha96 struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e Aes128CtsHmacSha96) GetETypeID() int32 {
+	return etypeID.AES128_CTS_HMAC_SHA1_96
+}
+
+// GetHashID returns the checksum type ID number.
+func (e Aes128CtsHmacSha96) GetHashID() int32 {
+	return chksumtype.HMAC_SHA1_96_AES128
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e Aes128CtsHmacSha96) GetKeyByteSize() int {
+	return 128 / 8
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e Aes128CtsHmacSha96) GetKeySeedBitLength() int {
+	return e.GetKeyByteSize() * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e Aes128CtsHmacSha96) GetHashFunc() func() hash.Hash {
+	return sha1.New
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e Aes128CtsHmacSha96) GetMessageBlockByteSize() int {
+	return 1
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e Aes128CtsHmacSha96) GetDefaultStringToKeyParams() string {
+	return "00001000"
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e Aes128CtsHmacSha96) GetConfounderByteSize() int {
+	return aes.BlockSize
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e Aes128CtsHmacSha96) GetHMACBitLength() int {
+	return 96
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e Aes128CtsHmacSha96) GetCypherBlockBitLength() int {
+	return aes.BlockSize * 8
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e Aes128CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	return rfc3962.StringToKey(secret, salt, s2kparams, e)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e Aes128CtsHmacSha96) RandomToKey(b []byte) []byte {
+	return rfc3961.RandomToKey(b)
+}
+
+// EncryptData encrypts the data provided.
+func (e Aes128CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	return rfc3962.EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e Aes128CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	return rfc3962.EncryptMessage(key, message, usage, e)
+}
+
+// DecryptData decrypts the data provided.
+func (e Aes128CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc3962.DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e Aes128CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc3962.DecryptMessage(key, ciphertext, usage, e)
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e Aes128CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	return rfc3961.DeriveKey(protocolKey, usage, e)
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e Aes128CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	return rfc3961.DeriveRandom(protocolKey, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the plaintext message.
+func (e Aes128CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e Aes128CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e Aes128CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	c, err := e.GetChecksumHash(protocolKey, data, usage)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(chksum, c)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go
new file mode 100644
index 0000000..b05af7d
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go
@@ -0,0 +1,132 @@
+package crypto
+
+import (
+	"crypto/aes"
+	"crypto/hmac"
+	"crypto/sha256"
+	"hash"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc8009"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// RFC https://tools.ietf.org/html/rfc8009
+
+// Aes128CtsHmacSha256128 implements Kerberos encryption type aes128-cts-hmac-sha256-128
+type Aes128CtsHmacSha256128 struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e Aes128CtsHmacSha256128) GetETypeID() int32 {
+	return etypeID.AES128_CTS_HMAC_SHA256_128
+}
+
+// GetHashID returns the checksum type ID number.
+func (e Aes128CtsHmacSha256128) GetHashID() int32 {
+	return chksumtype.HMAC_SHA256_128_AES128
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e Aes128CtsHmacSha256128) GetKeyByteSize() int {
+	return 128 / 8
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e Aes128CtsHmacSha256128) GetKeySeedBitLength() int {
+	return e.GetKeyByteSize() * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e Aes128CtsHmacSha256128) GetHashFunc() func() hash.Hash {
+	return sha256.New
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e Aes128CtsHmacSha256128) GetMessageBlockByteSize() int {
+	return 1
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e Aes128CtsHmacSha256128) GetDefaultStringToKeyParams() string {
+	return "00008000"
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e Aes128CtsHmacSha256128) GetConfounderByteSize() int {
+	return aes.BlockSize
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e Aes128CtsHmacSha256128) GetHMACBitLength() int {
+	return 128
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e Aes128CtsHmacSha256128) GetCypherBlockBitLength() int {
+	return aes.BlockSize * 8
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e Aes128CtsHmacSha256128) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	saltp := rfc8009.GetSaltP(salt, "aes128-cts-hmac-sha256-128")
+	return rfc8009.StringToKey(secret, saltp, s2kparams, e)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e Aes128CtsHmacSha256128) RandomToKey(b []byte) []byte {
+	return rfc8009.RandomToKey(b)
+}
+
+// EncryptData encrypts the data provided.
+func (e Aes128CtsHmacSha256128) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	return rfc8009.EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e Aes128CtsHmacSha256128) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	return rfc8009.EncryptMessage(key, message, usage, e)
+}
+
+// DecryptData decrypts the data provided.
+func (e Aes128CtsHmacSha256128) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc8009.DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e Aes128CtsHmacSha256128) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc8009.DecryptMessage(key, ciphertext, usage, e)
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e Aes128CtsHmacSha256128) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	return rfc8009.DeriveKey(protocolKey, usage, e), nil
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e Aes128CtsHmacSha256128) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	return rfc8009.DeriveRandom(protocolKey, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the ciphertext message.
+// As the hash is calculated over the iv concatenated with the AES cipher output not the plaintext the pt value to this
+// interface method is not use. Pass any []byte.
+func (e Aes128CtsHmacSha256128) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	// We don't need ib just there for the interface
+	return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e Aes128CtsHmacSha256128) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e Aes128CtsHmacSha256128) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	c, err := e.GetChecksumHash(protocolKey, data, usage)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(chksum, c)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go
new file mode 100644
index 0000000..45e439a
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go
@@ -0,0 +1,129 @@
+package crypto
+
+import (
+	"crypto/aes"
+	"crypto/hmac"
+	"crypto/sha1"
+	"hash"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3961"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3962"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// RFC 3962
+
+// Aes256CtsHmacSha96 implements Kerberos encryption type aes256-cts-hmac-sha1-96
+type Aes256CtsHmacSha96 struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e Aes256CtsHmacSha96) GetETypeID() int32 {
+	return etypeID.AES256_CTS_HMAC_SHA1_96
+}
+
+// GetHashID returns the checksum type ID number.
+func (e Aes256CtsHmacSha96) GetHashID() int32 {
+	return chksumtype.HMAC_SHA1_96_AES256
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e Aes256CtsHmacSha96) GetKeyByteSize() int {
+	return 256 / 8
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e Aes256CtsHmacSha96) GetKeySeedBitLength() int {
+	return e.GetKeyByteSize() * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e Aes256CtsHmacSha96) GetHashFunc() func() hash.Hash {
+	return sha1.New
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e Aes256CtsHmacSha96) GetMessageBlockByteSize() int {
+	return 1
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e Aes256CtsHmacSha96) GetDefaultStringToKeyParams() string {
+	return "00001000"
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e Aes256CtsHmacSha96) GetConfounderByteSize() int {
+	return aes.BlockSize
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e Aes256CtsHmacSha96) GetHMACBitLength() int {
+	return 96
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e Aes256CtsHmacSha96) GetCypherBlockBitLength() int {
+	return aes.BlockSize * 8
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e Aes256CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	return rfc3962.StringToKey(secret, salt, s2kparams, e)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e Aes256CtsHmacSha96) RandomToKey(b []byte) []byte {
+	return rfc3961.RandomToKey(b)
+}
+
+// EncryptData encrypts the data provided.
+func (e Aes256CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	return rfc3962.EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e Aes256CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	return rfc3962.EncryptMessage(key, message, usage, e)
+}
+
+// DecryptData decrypts the data provided.
+func (e Aes256CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc3962.DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e Aes256CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc3962.DecryptMessage(key, ciphertext, usage, e)
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e Aes256CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	return rfc3961.DeriveKey(protocolKey, usage, e)
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e Aes256CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	return rfc3961.DeriveRandom(protocolKey, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the plaintext message.
+func (e Aes256CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e Aes256CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e Aes256CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	c, err := e.GetChecksumHash(protocolKey, data, usage)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(chksum, c)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go
new file mode 100644
index 0000000..6a54475
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go
@@ -0,0 +1,132 @@
+package crypto
+
+import (
+	"crypto/aes"
+	"crypto/hmac"
+	"crypto/sha512"
+	"hash"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc8009"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// RFC https://tools.ietf.org/html/rfc8009
+
+// Aes256CtsHmacSha384192 implements Kerberos encryption type aes256-cts-hmac-sha384-192
+type Aes256CtsHmacSha384192 struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e Aes256CtsHmacSha384192) GetETypeID() int32 {
+	return etypeID.AES256_CTS_HMAC_SHA384_192
+}
+
+// GetHashID returns the checksum type ID number.
+func (e Aes256CtsHmacSha384192) GetHashID() int32 {
+	return chksumtype.HMAC_SHA384_192_AES256
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e Aes256CtsHmacSha384192) GetKeyByteSize() int {
+	return 192 / 8
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e Aes256CtsHmacSha384192) GetKeySeedBitLength() int {
+	return e.GetKeyByteSize() * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e Aes256CtsHmacSha384192) GetHashFunc() func() hash.Hash {
+	return sha512.New384
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e Aes256CtsHmacSha384192) GetMessageBlockByteSize() int {
+	return 1
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e Aes256CtsHmacSha384192) GetDefaultStringToKeyParams() string {
+	return "00008000"
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e Aes256CtsHmacSha384192) GetConfounderByteSize() int {
+	return aes.BlockSize
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e Aes256CtsHmacSha384192) GetHMACBitLength() int {
+	return 192
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e Aes256CtsHmacSha384192) GetCypherBlockBitLength() int {
+	return aes.BlockSize * 8
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e Aes256CtsHmacSha384192) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	saltp := rfc8009.GetSaltP(salt, "aes256-cts-hmac-sha384-192")
+	return rfc8009.StringToKey(secret, saltp, s2kparams, e)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e Aes256CtsHmacSha384192) RandomToKey(b []byte) []byte {
+	return rfc8009.RandomToKey(b)
+}
+
+// EncryptData encrypts the data provided.
+func (e Aes256CtsHmacSha384192) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	return rfc8009.EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e Aes256CtsHmacSha384192) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	return rfc8009.EncryptMessage(key, message, usage, e)
+}
+
+// DecryptData decrypts the data provided.
+func (e Aes256CtsHmacSha384192) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc8009.DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e Aes256CtsHmacSha384192) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc8009.DecryptMessage(key, ciphertext, usage, e)
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e Aes256CtsHmacSha384192) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	return rfc8009.DeriveKey(protocolKey, usage, e), nil
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e Aes256CtsHmacSha384192) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	return rfc8009.DeriveRandom(protocolKey, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the ciphertext message.
+// As the hash is calculated over the iv concatenated with the AES cipher output not the plaintext the pt value to this
+// interface method is not use. Pass any []byte.
+func (e Aes256CtsHmacSha384192) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	// We don't need ib just there for the interface
+	return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e Aes256CtsHmacSha384192) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e Aes256CtsHmacSha384192) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	c, err := e.GetChecksumHash(protocolKey, data, usage)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(chksum, c)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go
new file mode 100644
index 0000000..dab55be
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go
@@ -0,0 +1,132 @@
+// Package common provides encryption methods common across encryption types
+package common
+
+import (
+	"bytes"
+	"crypto/hmac"
+	"encoding/binary"
+	"encoding/hex"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+// ZeroPad pads bytes with zeros to nearest multiple of message size m.
+func ZeroPad(b []byte, m int) ([]byte, error) {
+	if m <= 0 {
+		return nil, errors.New("Invalid message block size when padding")
+	}
+	if b == nil || len(b) == 0 {
+		return nil, errors.New("Data not valid to pad: Zero size")
+	}
+	if l := len(b) % m; l != 0 {
+		n := m - l
+		z := make([]byte, n)
+		b = append(b, z...)
+	}
+	return b, nil
+}
+
+// PKCS7Pad pads bytes according to RFC 2315 to nearest multiple of message size m.
+func PKCS7Pad(b []byte, m int) ([]byte, error) {
+	if m <= 0 {
+		return nil, errors.New("Invalid message block size when padding")
+	}
+	if b == nil || len(b) == 0 {
+		return nil, errors.New("Data not valid to pad: Zero size")
+	}
+	n := m - (len(b) % m)
+	pb := make([]byte, len(b)+n)
+	copy(pb, b)
+	copy(pb[len(b):], bytes.Repeat([]byte{byte(n)}, n))
+	return pb, nil
+}
+
+// PKCS7Unpad removes RFC 2315 padding from byes where message size is m.
+func PKCS7Unpad(b []byte, m int) ([]byte, error) {
+	if m <= 0 {
+		return nil, errors.New("invalid message block size when unpadding")
+	}
+	if b == nil || len(b) == 0 {
+		return nil, errors.New("padded data not valid: Zero size")
+	}
+	if len(b)%m != 0 {
+		return nil, errors.New("padded data not valid: Not multiple of message block size")
+	}
+	c := b[len(b)-1]
+	n := int(c)
+	if n == 0 || n > len(b) {
+		return nil, errors.New("padded data not valid: Data may not have been padded")
+	}
+	for i := 0; i < n; i++ {
+		if b[len(b)-n+i] != c {
+			return nil, errors.New("padded data not valid")
+		}
+	}
+	return b[:len(b)-n], nil
+}
+
+// GetHash generates the keyed hash value according to the etype's hash function.
+func GetHash(pt, key []byte, usage []byte, etype etype.EType) ([]byte, error) {
+	k, err := etype.DeriveKey(key, usage)
+	if err != nil {
+		return nil, fmt.Errorf("unable to derive key for checksum: %v", err)
+	}
+	mac := hmac.New(etype.GetHashFunc(), k)
+	p := make([]byte, len(pt))
+	copy(p, pt)
+	mac.Write(p)
+	return mac.Sum(nil)[:etype.GetHMACBitLength()/8], nil
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func GetChecksumHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) {
+	return GetHash(b, key, GetUsageKc(usage), etype)
+}
+
+// GetIntegrityHash returns a keyed integrity hash of the bytes provided.
+func GetIntegrityHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) {
+	return GetHash(b, key, GetUsageKi(usage), etype)
+}
+
+// VerifyChecksum compares the checksum of the msg bytes is the same as the checksum provided.
+func VerifyChecksum(key, chksum, msg []byte, usage uint32, etype etype.EType) bool {
+	//The encrypted message is a concatenation of the encrypted output and the hash HMAC.
+	expectedMAC, _ := GetChecksumHash(msg, key, usage, etype)
+	return hmac.Equal(chksum, expectedMAC)
+}
+
+// GetUsageKc returns the checksum key usage value for the usage number un.
+//
+// See RFC 3961 5.3 key-derivation function definition.
+func GetUsageKc(un uint32) []byte {
+	return getUsage(un, 0x99)
+}
+
+// GetUsageKe returns the encryption key usage value for the usage number un
+//
+// See RFC 3961 5.3 key-derivation function definition.
+func GetUsageKe(un uint32) []byte {
+	return getUsage(un, 0xAA)
+}
+
+// GetUsageKi returns the integrity key usage value for the usage number un
+//
+// See RFC 3961 5.3 key-derivation function definition.
+func GetUsageKi(un uint32) []byte {
+	return getUsage(un, 0x55)
+}
+
+func getUsage(un uint32, o byte) []byte {
+	var buf bytes.Buffer
+	binary.Write(&buf, binary.BigEndian, un)
+	return append(buf.Bytes(), o)
+}
+
+// IterationsToS2Kparams converts the number of iterations as an integer to a string representation.
+func IterationsToS2Kparams(i uint32) string {
+	b := make([]byte, 4, 4)
+	binary.BigEndian.PutUint32(b, i)
+	return hex.EncodeToString(b)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go
new file mode 100644
index 0000000..5c96ddf
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go
@@ -0,0 +1,175 @@
+// Package crypto implements cryptographic functions for Kerberos 5 implementation.
+package crypto
+
+import (
+	"encoding/hex"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+	"github.com/jcmturner/gokrb5/v8/iana/patype"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// GetEtype returns an instances of the required etype struct for the etype ID.
+func GetEtype(id int32) (etype.EType, error) {
+	switch id {
+	case etypeID.AES128_CTS_HMAC_SHA1_96:
+		var et Aes128CtsHmacSha96
+		return et, nil
+	case etypeID.AES256_CTS_HMAC_SHA1_96:
+		var et Aes256CtsHmacSha96
+		return et, nil
+	case etypeID.AES128_CTS_HMAC_SHA256_128:
+		var et Aes128CtsHmacSha256128
+		return et, nil
+	case etypeID.AES256_CTS_HMAC_SHA384_192:
+		var et Aes256CtsHmacSha384192
+		return et, nil
+	case etypeID.DES3_CBC_SHA1_KD:
+		var et Des3CbcSha1Kd
+		return et, nil
+	case etypeID.RC4_HMAC:
+		var et RC4HMAC
+		return et, nil
+	default:
+		return nil, fmt.Errorf("unknown or unsupported EType: %d", id)
+	}
+}
+
+// GetChksumEtype returns an instances of the required etype struct for the checksum ID.
+func GetChksumEtype(id int32) (etype.EType, error) {
+	switch id {
+	case chksumtype.HMAC_SHA1_96_AES128:
+		var et Aes128CtsHmacSha96
+		return et, nil
+	case chksumtype.HMAC_SHA1_96_AES256:
+		var et Aes256CtsHmacSha96
+		return et, nil
+	case chksumtype.HMAC_SHA256_128_AES128:
+		var et Aes128CtsHmacSha256128
+		return et, nil
+	case chksumtype.HMAC_SHA384_192_AES256:
+		var et Aes256CtsHmacSha384192
+		return et, nil
+	case chksumtype.HMAC_SHA1_DES3_KD:
+		var et Des3CbcSha1Kd
+		return et, nil
+	case chksumtype.KERB_CHECKSUM_HMAC_MD5:
+		var et RC4HMAC
+		return et, nil
+	//case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED:
+	//	var et RC4HMAC
+	//	return et, nil
+	default:
+		return nil, fmt.Errorf("unknown or unsupported checksum type: %d", id)
+	}
+}
+
+// GetKeyFromPassword generates an encryption key from the principal's password.
+func GetKeyFromPassword(passwd string, cname types.PrincipalName, realm string, etypeID int32, pas types.PADataSequence) (types.EncryptionKey, etype.EType, error) {
+	var key types.EncryptionKey
+	et, err := GetEtype(etypeID)
+	if err != nil {
+		return key, et, fmt.Errorf("error getting encryption type: %v", err)
+	}
+	sk2p := et.GetDefaultStringToKeyParams()
+	var salt string
+	var paID int32
+	for _, pa := range pas {
+		switch pa.PADataType {
+		case patype.PA_PW_SALT:
+			if paID > pa.PADataType {
+				continue
+			}
+			salt = string(pa.PADataValue)
+		case patype.PA_ETYPE_INFO:
+			if paID > pa.PADataType {
+				continue
+			}
+			var eti types.ETypeInfo
+			err := eti.Unmarshal(pa.PADataValue)
+			if err != nil {
+				return key, et, fmt.Errorf("error unmashaling PA Data to PA-ETYPE-INFO2: %v", err)
+			}
+			if etypeID != eti[0].EType {
+				et, err = GetEtype(eti[0].EType)
+				if err != nil {
+					return key, et, fmt.Errorf("error getting encryption type: %v", err)
+				}
+			}
+			salt = string(eti[0].Salt)
+		case patype.PA_ETYPE_INFO2:
+			if paID > pa.PADataType {
+				continue
+			}
+			var et2 types.ETypeInfo2
+			err := et2.Unmarshal(pa.PADataValue)
+			if err != nil {
+				return key, et, fmt.Errorf("error unmashalling PA Data to PA-ETYPE-INFO2: %v", err)
+			}
+			if etypeID != et2[0].EType {
+				et, err = GetEtype(et2[0].EType)
+				if err != nil {
+					return key, et, fmt.Errorf("error getting encryption type: %v", err)
+				}
+			}
+			if len(et2[0].S2KParams) == 4 {
+				sk2p = hex.EncodeToString(et2[0].S2KParams)
+			}
+			salt = et2[0].Salt
+		}
+	}
+	if salt == "" {
+		salt = cname.GetSalt(realm)
+	}
+	k, err := et.StringToKey(passwd, salt, sk2p)
+	if err != nil {
+		return key, et, fmt.Errorf("error deriving key from string: %+v", err)
+	}
+	key = types.EncryptionKey{
+		KeyType:  etypeID,
+		KeyValue: k,
+	}
+	return key, et, nil
+}
+
+// GetEncryptedData encrypts the data provided and returns and EncryptedData type.
+// Pass a usage value of zero to use the key provided directly rather than deriving one.
+func GetEncryptedData(plainBytes []byte, key types.EncryptionKey, usage uint32, kvno int) (types.EncryptedData, error) {
+	var ed types.EncryptedData
+	et, err := GetEtype(key.KeyType)
+	if err != nil {
+		return ed, fmt.Errorf("error getting etype: %v", err)
+	}
+	_, b, err := et.EncryptMessage(key.KeyValue, plainBytes, usage)
+	if err != nil {
+		return ed, err
+	}
+
+	ed = types.EncryptedData{
+		EType:  key.KeyType,
+		Cipher: b,
+		KVNO:   kvno,
+	}
+	return ed, nil
+}
+
+// DecryptEncPart decrypts the EncryptedData.
+func DecryptEncPart(ed types.EncryptedData, key types.EncryptionKey, usage uint32) ([]byte, error) {
+	return DecryptMessage(ed.Cipher, key, usage)
+}
+
+// DecryptMessage decrypts the ciphertext and verifies the integrity.
+func DecryptMessage(ciphertext []byte, key types.EncryptionKey, usage uint32) ([]byte, error) {
+	et, err := GetEtype(key.KeyType)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error decrypting: %v", err)
+	}
+	b, err := et.DecryptMessage(key.KeyValue, ciphertext, usage)
+	if err != nil {
+		return nil, fmt.Errorf("error decrypting: %v", err)
+	}
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go
new file mode 100644
index 0000000..6e650eb
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go
@@ -0,0 +1,139 @@
+package crypto
+
+import (
+	"crypto/des"
+	"crypto/hmac"
+	"crypto/sha1"
+	"errors"
+	"hash"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3961"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+//RFC: 3961 Section 6.3
+
+// Des3CbcSha1Kd implements Kerberos encryption type des3-cbc-hmac-sha1-kd
+type Des3CbcSha1Kd struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e Des3CbcSha1Kd) GetETypeID() int32 {
+	return etypeID.DES3_CBC_SHA1_KD
+}
+
+// GetHashID returns the checksum type ID number.
+func (e Des3CbcSha1Kd) GetHashID() int32 {
+	return chksumtype.HMAC_SHA1_DES3_KD
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e Des3CbcSha1Kd) GetKeyByteSize() int {
+	return 24
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e Des3CbcSha1Kd) GetKeySeedBitLength() int {
+	return 21 * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e Des3CbcSha1Kd) GetHashFunc() func() hash.Hash {
+	return sha1.New
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e Des3CbcSha1Kd) GetMessageBlockByteSize() int {
+	//For traditional CBC mode with padding, it would be the underlying cipher's block size
+	return des.BlockSize
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e Des3CbcSha1Kd) GetDefaultStringToKeyParams() string {
+	var s string
+	return s
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e Des3CbcSha1Kd) GetConfounderByteSize() int {
+	return des.BlockSize
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e Des3CbcSha1Kd) GetHMACBitLength() int {
+	return e.GetHashFunc()().Size() * 8
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e Des3CbcSha1Kd) GetCypherBlockBitLength() int {
+	return des.BlockSize * 8
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e Des3CbcSha1Kd) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	if s2kparams != "" {
+		return []byte{}, errors.New("s2kparams must be an empty string")
+	}
+	return rfc3961.DES3StringToKey(secret, salt, e)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e Des3CbcSha1Kd) RandomToKey(b []byte) []byte {
+	return rfc3961.DES3RandomToKey(b)
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e Des3CbcSha1Kd) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	r, err := rfc3961.DeriveRandom(protocolKey, usage, e)
+	return r, err
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e Des3CbcSha1Kd) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	r, err := e.DeriveRandom(protocolKey, usage)
+	if err != nil {
+		return nil, err
+	}
+	return e.RandomToKey(r), nil
+}
+
+// EncryptData encrypts the data provided.
+func (e Des3CbcSha1Kd) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	return rfc3961.DES3EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e Des3CbcSha1Kd) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	return rfc3961.DES3EncryptMessage(key, message, usage, e)
+}
+
+// DecryptData decrypts the data provided.
+func (e Des3CbcSha1Kd) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc3961.DES3DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e Des3CbcSha1Kd) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc3961.DES3DecryptMessage(key, ciphertext, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the plaintext message.
+func (e Des3CbcSha1Kd) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e Des3CbcSha1Kd) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e Des3CbcSha1Kd) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	c, err := e.GetChecksumHash(protocolKey, data, usage)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(chksum, c)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go
new file mode 100644
index 0000000..ab1496d
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go
@@ -0,0 +1,29 @@
+// Package etype provides the Kerberos Encryption Type interface
+package etype
+
+import "hash"
+
+// EType is the interface defining the Encryption Type.
+type EType interface {
+	GetETypeID() int32
+	GetHashID() int32
+	GetKeyByteSize() int
+	GetKeySeedBitLength() int
+	GetDefaultStringToKeyParams() string
+	StringToKey(string, salt, s2kparams string) ([]byte, error)
+	RandomToKey(b []byte) []byte
+	GetHMACBitLength() int
+	GetMessageBlockByteSize() int
+	EncryptData(key, data []byte) ([]byte, []byte, error)
+	EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error)
+	DecryptData(key, data []byte) ([]byte, error)
+	DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error)
+	GetCypherBlockBitLength() int
+	GetConfounderByteSize() int
+	DeriveKey(protocolKey, usage []byte) ([]byte, error)
+	DeriveRandom(protocolKey, usage []byte) ([]byte, error)
+	VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool
+	GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error)
+	VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool
+	GetHashFunc() func() hash.Hash
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go
new file mode 100644
index 0000000..42f84b8
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go
@@ -0,0 +1,133 @@
+package crypto
+
+import (
+	"bytes"
+	"crypto/hmac"
+	"crypto/md5"
+	"hash"
+	"io"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc3961"
+	"github.com/jcmturner/gokrb5/v8/crypto/rfc4757"
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+	"golang.org/x/crypto/md4"
+)
+
+// RC4HMAC implements Kerberos encryption type rc4-hmac
+type RC4HMAC struct {
+}
+
+// GetETypeID returns the EType ID number.
+func (e RC4HMAC) GetETypeID() int32 {
+	return etypeID.RC4_HMAC
+}
+
+// GetHashID returns the checksum type ID number.
+func (e RC4HMAC) GetHashID() int32 {
+	return chksumtype.KERB_CHECKSUM_HMAC_MD5
+}
+
+// GetKeyByteSize returns the number of bytes for key of this etype.
+func (e RC4HMAC) GetKeyByteSize() int {
+	return 16
+}
+
+// GetKeySeedBitLength returns the number of bits for the seed for key generation.
+func (e RC4HMAC) GetKeySeedBitLength() int {
+	return e.GetKeyByteSize() * 8
+}
+
+// GetHashFunc returns the hash function for this etype.
+func (e RC4HMAC) GetHashFunc() func() hash.Hash {
+	return md5.New
+}
+
+// GetMessageBlockByteSize returns the block size for the etype's messages.
+func (e RC4HMAC) GetMessageBlockByteSize() int {
+	return 1
+}
+
+// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
+func (e RC4HMAC) GetDefaultStringToKeyParams() string {
+	return ""
+}
+
+// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
+func (e RC4HMAC) GetConfounderByteSize() int {
+	return 8
+}
+
+// GetHMACBitLength returns the bit count size of the integrity hash.
+func (e RC4HMAC) GetHMACBitLength() int {
+	return md5.Size * 8
+}
+
+// GetCypherBlockBitLength returns the bit count size of the cypher block.
+func (e RC4HMAC) GetCypherBlockBitLength() int {
+	return 8 // doesn't really apply
+}
+
+// StringToKey returns a key derived from the string provided.
+func (e RC4HMAC) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
+	return rfc4757.StringToKey(secret)
+}
+
+// RandomToKey returns a key from the bytes provided.
+func (e RC4HMAC) RandomToKey(b []byte) []byte {
+	r := bytes.NewReader(b)
+	h := md4.New()
+	io.Copy(h, r)
+	return h.Sum(nil)
+}
+
+// EncryptData encrypts the data provided.
+func (e RC4HMAC) EncryptData(key, data []byte) ([]byte, []byte, error) {
+	b, err := rfc4757.EncryptData(key, data, e)
+	return []byte{}, b, err
+}
+
+// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
+func (e RC4HMAC) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
+	b, err := rfc4757.EncryptMessage(key, message, usage, false, e)
+	return []byte{}, b, err
+}
+
+// DecryptData decrypts the data provided.
+func (e RC4HMAC) DecryptData(key, data []byte) ([]byte, error) {
+	return rfc4757.DecryptData(key, data, e)
+}
+
+// DecryptMessage decrypts the message provided and verifies the integrity of the message.
+func (e RC4HMAC) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
+	return rfc4757.DecryptMessage(key, ciphertext, usage, false, e)
+}
+
+// DeriveKey derives a key from the protocol key based on the usage value.
+func (e RC4HMAC) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
+	return rfc4757.HMAC(protocolKey, usage), nil
+}
+
+// DeriveRandom generates data needed for key generation.
+func (e RC4HMAC) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
+	return rfc3961.DeriveRandom(protocolKey, usage, e)
+}
+
+// VerifyIntegrity checks the integrity of the plaintext message.
+func (e RC4HMAC) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
+	return rfc4757.VerifyIntegrity(protocolKey, pt, ct, e)
+}
+
+// GetChecksumHash returns a keyed checksum hash of the bytes provided.
+func (e RC4HMAC) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
+	return rfc4757.Checksum(protocolKey, usage, data)
+}
+
+// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
+func (e RC4HMAC) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
+	checksum, err := rfc4757.Checksum(protocolKey, usage, data)
+	if err != nil {
+		return false
+	}
+	return hmac.Equal(checksum, chksum)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go
new file mode 100644
index 0000000..1383258
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go
@@ -0,0 +1,119 @@
+// Package rfc3961 provides encryption and checksum methods as specified in RFC 3961
+package rfc3961
+
+import (
+	"crypto/cipher"
+	"crypto/des"
+	"crypto/hmac"
+	"crypto/rand"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+// DES3EncryptData encrypts the data provided using DES3 and methods specific to the etype provided.
+func DES3EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return nil, nil, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	data, _ = common.ZeroPad(data, e.GetMessageBlockByteSize())
+
+	block, err := des.NewTripleDESCipher(key)
+	if err != nil {
+		return nil, nil, fmt.Errorf("error creating cipher: %v", err)
+	}
+
+	//RFC 3961: initial cipher state      All bits zero
+	ivz := make([]byte, des.BlockSize)
+
+	ct := make([]byte, len(data))
+	mode := cipher.NewCBCEncrypter(block, ivz)
+	mode.CryptBlocks(ct, data)
+	return ct[len(ct)-e.GetMessageBlockByteSize():], ct, nil
+}
+
+// DES3EncryptMessage encrypts the message provided using DES3 and methods specific to the etype provided.
+// The encrypted data is concatenated with its integrity hash to create an encrypted message.
+func DES3EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
+	//confounder
+	c := make([]byte, e.GetConfounderByteSize())
+	_, err := rand.Read(c)
+	if err != nil {
+		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
+	}
+	plainBytes := append(c, message...)
+	plainBytes, _ = common.ZeroPad(plainBytes, e.GetMessageBlockByteSize())
+
+	// Derive key for encryption from usage
+	var k []byte
+	if usage != 0 {
+		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
+		if err != nil {
+			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
+		}
+	}
+
+	iv, b, err := e.EncryptData(k, plainBytes)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+
+	// Generate and append integrity hash
+	ih, err := common.GetIntegrityHash(plainBytes, key, usage, e)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+	b = append(b, ih...)
+	return iv, b, nil
+}
+
+// DES3DecryptData decrypts the data provided using DES3 and methods specific to the etype provided.
+func DES3DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+
+	if len(data) < des.BlockSize || len(data)%des.BlockSize != 0 {
+		return []byte{}, errors.New("ciphertext is not a multiple of the block size")
+	}
+	block, err := des.NewTripleDESCipher(key)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error creating cipher: %v", err)
+	}
+	pt := make([]byte, len(data))
+	ivz := make([]byte, des.BlockSize)
+	mode := cipher.NewCBCDecrypter(block, ivz)
+	mode.CryptBlocks(pt, data)
+	return pt, nil
+}
+
+// DES3DecryptMessage decrypts the message provided using DES3 and methods specific to the etype provided.
+// The integrity of the message is also verified.
+func DES3DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
+	//Derive the key
+	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
+	if err != nil {
+		return nil, fmt.Errorf("error deriving key: %v", err)
+	}
+	// Strip off the checksum from the end
+	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
+	if err != nil {
+		return nil, fmt.Errorf("error decrypting: %v", err)
+	}
+	//Verify checksum
+	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
+		return nil, errors.New("error decrypting: integrity verification failed")
+	}
+	//Remove the confounder bytes
+	return b[e.GetConfounderByteSize():], nil
+}
+
+// VerifyIntegrity verifies the integrity of cipertext bytes ct.
+func VerifyIntegrity(key, ct, pt []byte, usage uint32, etype etype.EType) bool {
+	h := make([]byte, etype.GetHMACBitLength()/8)
+	copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:])
+	expectedMAC, _ := common.GetIntegrityHash(pt, key, usage, etype)
+	return hmac.Equal(h, expectedMAC)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go
new file mode 100644
index 0000000..ed9b169
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go
@@ -0,0 +1,169 @@
+package rfc3961
+
+import (
+	"bytes"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+const (
+	prfconstant = "prf"
+)
+
+// DeriveRandom implements the RFC 3961 defined function: DR(Key, Constant) = k-truncate(E(Key, Constant, initial-cipher-state)).
+//
+// key: base key or protocol key. Likely to be a key from a keytab file.
+//
+// usage: a constant.
+//
+// n: block size in bits (not bytes) - note if you use something like aes.BlockSize this is in bytes.
+//
+// k: key length / key seed length in bits. Eg. for AES256 this value is 256.
+//
+// e: the encryption etype function to use.
+func DeriveRandom(key, usage []byte, e etype.EType) ([]byte, error) {
+	n := e.GetCypherBlockBitLength()
+	k := e.GetKeySeedBitLength()
+	//Ensure the usage constant is at least the size of the cypher block size. Pass it through the nfold algorithm that will "stretch" it if needs be.
+	nFoldUsage := Nfold(usage, n)
+	//k-truncate implemented by creating a byte array the size of k (k is in bits hence /8)
+	out := make([]byte, k/8)
+	// Keep feeding the output back into the encryption function until it is no longer short than k.
+	_, K, err := e.EncryptData(key, nFoldUsage)
+	if err != nil {
+		return out, err
+	}
+	for i := copy(out, K); i < len(out); {
+		_, K, _ = e.EncryptData(key, K)
+		i = i + copy(out[i:], K)
+	}
+	return out, nil
+}
+
+// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods.
+func DeriveKey(protocolKey, usage []byte, e etype.EType) ([]byte, error) {
+	r, err := e.DeriveRandom(protocolKey, usage)
+	if err != nil {
+		return nil, err
+	}
+	return e.RandomToKey(r), nil
+}
+
+// RandomToKey returns a key from the bytes provided according to the definition in RFC 3961.
+func RandomToKey(b []byte) []byte {
+	return b
+}
+
+// DES3RandomToKey returns a key from the bytes provided according to the definition in RFC 3961 for DES3 etypes.
+func DES3RandomToKey(b []byte) []byte {
+	r := fixWeakKey(stretch56Bits(b[:7]))
+	r2 := fixWeakKey(stretch56Bits(b[7:14]))
+	r = append(r, r2...)
+	r3 := fixWeakKey(stretch56Bits(b[14:21]))
+	r = append(r, r3...)
+	return r
+}
+
+// DES3StringToKey returns a key derived from the string provided according to the definition in RFC 3961 for DES3 etypes.
+func DES3StringToKey(secret, salt string, e etype.EType) ([]byte, error) {
+	s := secret + salt
+	tkey := e.RandomToKey(Nfold([]byte(s), e.GetKeySeedBitLength()))
+	return e.DeriveKey(tkey, []byte("kerberos"))
+}
+
+// PseudoRandom function as defined in RFC 3961
+func PseudoRandom(key, b []byte, e etype.EType) ([]byte, error) {
+	h := e.GetHashFunc()()
+	h.Write(b)
+	tmp := h.Sum(nil)[:e.GetMessageBlockByteSize()]
+	k, err := e.DeriveKey(key, []byte(prfconstant))
+	if err != nil {
+		return []byte{}, err
+	}
+	_, prf, err := e.EncryptData(k, tmp)
+	if err != nil {
+		return []byte{}, err
+	}
+	return prf, nil
+}
+
+func stretch56Bits(b []byte) []byte {
+	d := make([]byte, len(b), len(b))
+	copy(d, b)
+	var lb byte
+	for i, v := range d {
+		bv, nb := calcEvenParity(v)
+		d[i] = nb
+		if bv != 0 {
+			lb = lb | (1 << uint(i+1))
+		} else {
+			lb = lb &^ (1 << uint(i+1))
+		}
+	}
+	_, lb = calcEvenParity(lb)
+	d = append(d, lb)
+	return d
+}
+
+func calcEvenParity(b byte) (uint8, uint8) {
+	lowestbit := b & 0x01
+	// c counter of 1s in the first 7 bits of the byte
+	var c int
+	// Iterate over the highest 7 bits (hence p starts at 1 not zero) and count the 1s.
+	for p := 1; p < 8; p++ {
+		v := b & (1 << uint(p))
+		if v != 0 {
+			c++
+		}
+	}
+	if c%2 == 0 {
+		//Even number of 1s so set parity to 1
+		b = b | 1
+	} else {
+		//Odd number of 1s so set parity to 0
+		b = b &^ 1
+	}
+	return lowestbit, b
+}
+
+func fixWeakKey(b []byte) []byte {
+	if weak(b) {
+		b[7] ^= 0xF0
+	}
+	return b
+}
+
+func weak(b []byte) bool {
+	// weak keys from https://nvlpubs.nist.gov/nistpubs/Legacy/SP/nistspecialpublication800-67r1.pdf
+	weakKeys := [4][]byte{
+		{0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01},
+		{0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE},
+		{0xE0, 0xE0, 0xE0, 0xE0, 0xF1, 0xF1, 0xF1, 0xF1},
+		{0x1F, 0x1F, 0x1F, 0x1F, 0x0E, 0x0E, 0x0E, 0x0E},
+	}
+	semiWeakKeys := [12][]byte{
+		{0x01, 0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E},
+		{0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E, 0x01},
+		{0x01, 0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1},
+		{0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1, 0x01},
+		{0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE},
+		{0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01},
+		{0x1F, 0xE0, 0x1F, 0xE0, 0x0E, 0xF1, 0x0E, 0xF1},
+		{0xE0, 0x1F, 0xE0, 0x1F, 0xF1, 0x0E, 0xF1, 0x0E},
+		{0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E, 0xFE},
+		{0xFE, 0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E},
+		{0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1, 0xFE},
+		{0xFE, 0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1},
+	}
+	for _, k := range weakKeys {
+		if bytes.Equal(b, k) {
+			return true
+		}
+	}
+	for _, k := range semiWeakKeys {
+		if bytes.Equal(b, k) {
+			return true
+		}
+	}
+	return false
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go
new file mode 100644
index 0000000..9536b1e
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go
@@ -0,0 +1,107 @@
+package rfc3961
+
+// Implementation of the n-fold algorithm as defined in RFC 3961.
+
+/* Credits
+This golang implementation of nfold used the following project for help with implementation detail.
+Although their source is in java it was helpful as a reference implementation of the RFC.
+You can find the source code of their open source project along with license information below.
+We acknowledge and are grateful to these developers for their contributions to open source
+
+Project: Apache Directory (http://http://directory.apache.org/)
+https://svn.apache.org/repos/asf/directory/apacheds/tags/1.5.1/kerberos-shared/src/main/java/org/apache/directory/server/kerberos/shared/crypto/encryption/NFold.java
+License: http://www.apache.org/licenses/LICENSE-2.0
+*/
+
+// Nfold expands the key to ensure it is not smaller than one cipher block.
+// Defined in RFC 3961.
+//
+// m input bytes that will be "stretched" to the least common multiple of n bits and the bit length of m.
+func Nfold(m []byte, n int) []byte {
+	k := len(m) * 8
+
+	//Get the lowest common multiple of the two bit sizes
+	lcm := lcm(n, k)
+	relicate := lcm / k
+	var sumBytes []byte
+
+	for i := 0; i < relicate; i++ {
+		rotation := 13 * i
+		sumBytes = append(sumBytes, rotateRight(m, rotation)...)
+	}
+
+	nfold := make([]byte, n/8)
+	sum := make([]byte, n/8)
+	for i := 0; i < lcm/n; i++ {
+		for j := 0; j < n/8; j++ {
+			sum[j] = sumBytes[j+(i*len(sum))]
+		}
+		nfold = onesComplementAddition(nfold, sum)
+	}
+	return nfold
+}
+
+func onesComplementAddition(n1, n2 []byte) []byte {
+	numBits := len(n1) * 8
+	out := make([]byte, numBits/8)
+	carry := 0
+	for i := numBits - 1; i > -1; i-- {
+		n1b := getBit(&n1, i)
+		n2b := getBit(&n2, i)
+		s := n1b + n2b + carry
+
+		if s == 0 || s == 1 {
+			setBit(&out, i, s)
+			carry = 0
+		} else if s == 2 {
+			carry = 1
+		} else if s == 3 {
+			setBit(&out, i, 1)
+			carry = 1
+		}
+	}
+	if carry == 1 {
+		carryArray := make([]byte, len(n1))
+		carryArray[len(carryArray)-1] = 1
+		out = onesComplementAddition(out, carryArray)
+	}
+	return out
+}
+
+func rotateRight(b []byte, step int) []byte {
+	out := make([]byte, len(b))
+	bitLen := len(b) * 8
+	for i := 0; i < bitLen; i++ {
+		v := getBit(&b, i)
+		setBit(&out, (i+step)%bitLen, v)
+	}
+	return out
+}
+
+func lcm(x, y int) int {
+	return (x * y) / gcd(x, y)
+}
+
+func gcd(x, y int) int {
+	for y != 0 {
+		x, y = y, x%y
+	}
+	return x
+}
+
+func getBit(b *[]byte, p int) int {
+	pByte := p / 8
+	pBit := uint(p % 8)
+	vByte := (*b)[pByte]
+	vInt := int(vByte >> (8 - (pBit + 1)) & 0x0001)
+	return vInt
+}
+
+func setBit(b *[]byte, p, v int) {
+	pByte := p / 8
+	pBit := uint(p % 8)
+	oldByte := (*b)[pByte]
+	var newByte byte
+	newByte = byte(v<<(8-(pBit+1))) | oldByte
+	(*b)[pByte] = newByte
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go
new file mode 100644
index 0000000..5ff89e8
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go
@@ -0,0 +1,89 @@
+// Package rfc3962 provides encryption and checksum methods as specified in RFC 3962
+package rfc3962
+
+import (
+	"crypto/rand"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/aescts/v2"
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 3962.
+func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	ivz := make([]byte, e.GetCypherBlockBitLength()/8)
+	return aescts.Encrypt(key, ivz, data)
+}
+
+// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 3962.
+// The encrypted data is concatenated with its integrity hash to create an encrypted message.
+func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	//confounder
+	c := make([]byte, e.GetConfounderByteSize())
+	_, err := rand.Read(c)
+	if err != nil {
+		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
+	}
+	plainBytes := append(c, message...)
+
+	// Derive key for encryption from usage
+	var k []byte
+	if usage != 0 {
+		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
+		if err != nil {
+			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
+		}
+	}
+
+	// Encrypt the data
+	iv, b, err := e.EncryptData(k, plainBytes)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+
+	// Generate and append integrity hash
+	ih, err := common.GetIntegrityHash(plainBytes, key, usage, e)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+	b = append(b, ih...)
+	return iv, b, nil
+}
+
+// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 3962.
+func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	ivz := make([]byte, e.GetCypherBlockBitLength()/8)
+	return aescts.Decrypt(key, ivz, data)
+}
+
+// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 3962.
+// The integrity of the message is also verified.
+func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
+	//Derive the key
+	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
+	if err != nil {
+		return nil, fmt.Errorf("error deriving key: %v", err)
+	}
+	// Strip off the checksum from the end
+	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
+	if err != nil {
+		return nil, err
+	}
+	//Verify checksum
+	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
+		return nil, errors.New("integrity verification failed")
+	}
+	//Remove the confounder bytes
+	return b[e.GetConfounderByteSize():], nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go
new file mode 100644
index 0000000..fb402d9
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go
@@ -0,0 +1,51 @@
+package rfc3962
+
+import (
+	"encoding/binary"
+	"encoding/hex"
+	"errors"
+
+	"github.com/jcmturner/gofork/x/crypto/pbkdf2"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+const (
+	s2kParamsZero = 4294967296
+)
+
+// StringToKey returns a key derived from the string provided according to the definition in RFC 3961.
+func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) {
+	i, err := S2KparamsToItertions(s2kparams)
+	if err != nil {
+		return nil, err
+	}
+	return StringToKeyIter(secret, salt, i, e)
+}
+
+// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0
+func StringToPBKDF2(secret, salt string, iterations int64, e etype.EType) []byte {
+	return pbkdf2.Key64([]byte(secret), []byte(salt), iterations, int64(e.GetKeyByteSize()), e.GetHashFunc())
+}
+
+// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 3961.
+func StringToKeyIter(secret, salt string, iterations int64, e etype.EType) ([]byte, error) {
+	tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e))
+	return e.DeriveKey(tkey, []byte("kerberos"))
+}
+
+// S2KparamsToItertions converts the string representation of iterations to an integer
+func S2KparamsToItertions(s2kparams string) (int64, error) {
+	//The s2kparams string should be hex string representing 4 bytes
+	//The 4 bytes represent a number in big endian order
+	//If the value is zero then the number of iterations should be 4,294,967,296 (2^32)
+	var i uint32
+	if len(s2kparams) != 8 {
+		return int64(s2kParamsZero), errors.New("invalid s2kparams length")
+	}
+	b, err := hex.DecodeString(s2kparams)
+	if err != nil {
+		return int64(s2kParamsZero), errors.New("invalid s2kparams, cannot decode string to bytes")
+	}
+	i = binary.BigEndian.Uint32(b)
+	return int64(i), nil
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/checksum.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/checksum.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/checksum.go
rename to vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/checksum.go
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go
new file mode 100644
index 0000000..fdebe73
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go
@@ -0,0 +1,80 @@
+// Package rfc4757 provides encryption and checksum methods as specified in RFC 4757
+package rfc4757
+
+import (
+	"crypto/hmac"
+	"crypto/rand"
+	"crypto/rc4"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 4757.
+func EncryptData(key, data []byte, e etype.EType) ([]byte, error) {
+	if len(key) != e.GetKeyByteSize() {
+		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	rc4Cipher, err := rc4.NewCipher(key)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error creating RC4 cipher: %v", err)
+	}
+	ed := make([]byte, len(data))
+	copy(ed, data)
+	rc4Cipher.XORKeyStream(ed, ed)
+	rc4Cipher.Reset()
+	return ed, nil
+}
+
+// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 4757.
+func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
+	return EncryptData(key, data, e)
+}
+
+// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 4757.
+// The encrypted data is concatenated with its RC4 header containing integrity checksum and confounder to create an encrypted message.
+func EncryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) {
+	confounder := make([]byte, e.GetConfounderByteSize()) // size = 8
+	_, err := rand.Read(confounder)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error generating confounder: %v", err)
+	}
+	k1 := key
+	k2 := HMAC(k1, UsageToMSMsgType(usage))
+	toenc := append(confounder, data...)
+	chksum := HMAC(k2, toenc)
+	k3 := HMAC(k2, chksum)
+
+	ed, err := EncryptData(k3, toenc, e)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error encrypting data: %v", err)
+	}
+
+	msg := append(chksum, ed...)
+	return msg, nil
+}
+
+// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 4757.
+// The integrity of the message is also verified.
+func DecryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) {
+	checksum := data[:e.GetHMACBitLength()/8]
+	ct := data[e.GetHMACBitLength()/8:]
+	_, k2, k3 := deriveKeys(key, checksum, usage, export)
+
+	pt, err := DecryptData(k3, ct, e)
+	if err != nil {
+		return []byte{}, fmt.Errorf("error decrypting data: %v", err)
+	}
+
+	if !VerifyIntegrity(k2, pt, data, e) {
+		return []byte{}, errors.New("integrity checksum incorrect")
+	}
+	return pt[e.GetConfounderByteSize():], nil
+}
+
+// VerifyIntegrity checks the integrity checksum of the data matches that calculated from the decrypted data.
+func VerifyIntegrity(key, pt, data []byte, e etype.EType) bool {
+	chksum := HMAC(key, pt)
+	return hmac.Equal(chksum, data[:e.GetHMACBitLength()/8])
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go
new file mode 100644
index 0000000..d1f90c0
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go
@@ -0,0 +1,40 @@
+package rfc4757
+
+import (
+	"bytes"
+	"encoding/hex"
+	"errors"
+	"fmt"
+	"io"
+
+	"golang.org/x/crypto/md4"
+)
+
+// StringToKey returns a key derived from the string provided according to the definition in RFC 4757.
+func StringToKey(secret string) ([]byte, error) {
+	b := make([]byte, len(secret)*2, len(secret)*2)
+	for i, r := range secret {
+		u := fmt.Sprintf("%04x", r)
+		c, err := hex.DecodeString(u)
+		if err != nil {
+			return []byte{}, errors.New("character could not be encoded")
+		}
+		// Swap round the two bytes to make little endian as we put into byte slice
+		b[2*i] = c[1]
+		b[2*i+1] = c[0]
+	}
+	r := bytes.NewReader(b)
+	h := md4.New()
+	_, err := io.Copy(h, r)
+	if err != nil {
+		return []byte{}, err
+	}
+	return h.Sum(nil), nil
+}
+
+func deriveKeys(key, checksum []byte, usage uint32, export bool) (k1, k2, k3 []byte) {
+	k1 = key
+	k2 = HMAC(k1, UsageToMSMsgType(usage))
+	k3 = HMAC(k2, checksum)
+	return
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/msgtype.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/msgtype.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/msgtype.go
rename to vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/msgtype.go
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go
new file mode 100644
index 0000000..54cff7b
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go
@@ -0,0 +1,125 @@
+// Package rfc8009 provides encryption and checksum methods as specified in RFC 8009
+package rfc8009
+
+import (
+	"crypto/aes"
+	"crypto/hmac"
+	"crypto/rand"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/aescts/v2"
+	"github.com/jcmturner/gokrb5/v8/crypto/common"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+)
+
+// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 8009.
+func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
+	kl := e.GetKeyByteSize()
+	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
+		kl = 32
+	}
+	if len(key) != kl {
+		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
+	}
+	ivz := make([]byte, aes.BlockSize)
+	return aescts.Encrypt(key, ivz, data)
+}
+
+// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 8009.
+// The encrypted data is concatenated with its integrity hash to create an encrypted message.
+func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
+	kl := e.GetKeyByteSize()
+	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
+		kl = 32
+	}
+	if len(key) != kl {
+		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key))
+	}
+	if len(key) != e.GetKeyByteSize() {
+	}
+	//confounder
+	c := make([]byte, e.GetConfounderByteSize())
+	_, err := rand.Read(c)
+	if err != nil {
+		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
+	}
+	plainBytes := append(c, message...)
+
+	// Derive key for encryption from usage
+	var k []byte
+	if usage != 0 {
+		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
+		if err != nil {
+			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
+		}
+	}
+
+	// Encrypt the data
+	iv, b, err := e.EncryptData(k, plainBytes)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+
+	ivz := make([]byte, e.GetConfounderByteSize())
+	ih, err := GetIntegityHash(ivz, b, key, usage, e)
+	if err != nil {
+		return iv, b, fmt.Errorf("error encrypting data: %v", err)
+	}
+	b = append(b, ih...)
+	return iv, b, nil
+}
+
+// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 8009.
+func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
+	kl := e.GetKeyByteSize()
+	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
+		kl = 32
+	}
+	if len(key) != kl {
+		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key))
+	}
+	ivz := make([]byte, aes.BlockSize)
+	return aescts.Decrypt(key, ivz, data)
+}
+
+// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 8009.
+// The integrity of the message is also verified.
+func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
+	//Derive the key
+	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
+	if err != nil {
+		return nil, fmt.Errorf("error deriving key: %v", err)
+	}
+	// Strip off the checksum from the end
+	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
+	if err != nil {
+		return nil, err
+	}
+	//Verify checksum
+	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
+		return nil, errors.New("integrity verification failed")
+	}
+	//Remove the confounder bytes
+	return b[e.GetConfounderByteSize():], nil
+}
+
+// GetIntegityHash returns a keyed integrity hash of the bytes provided as defined in RFC 8009
+func GetIntegityHash(iv, c, key []byte, usage uint32, e etype.EType) ([]byte, error) {
+	// Generate and append integrity hash
+	// Rather than calculating the hash over the confounder and plaintext
+	// it is calculated over the iv concatenated with the AES cipher output.
+	ib := append(iv, c...)
+	return common.GetIntegrityHash(ib, key, usage, e)
+}
+
+// VerifyIntegrity verifies the integrity of cipertext bytes ct.
+func VerifyIntegrity(key, ct []byte, usage uint32, etype etype.EType) bool {
+	h := make([]byte, etype.GetHMACBitLength()/8)
+	copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:])
+	ivz := make([]byte, etype.GetConfounderByteSize())
+	ib := append(ivz, ct[:len(ct)-(etype.GetHMACBitLength()/8)]...)
+	expectedMAC, _ := common.GetIntegrityHash(ib, key, usage, etype)
+	return hmac.Equal(h, expectedMAC)
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go
new file mode 100644
index 0000000..e947322
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go
@@ -0,0 +1,135 @@
+package rfc8009
+
+import (
+	"crypto/hmac"
+	"encoding/binary"
+	"encoding/hex"
+	"errors"
+
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+	"github.com/jcmturner/gokrb5/v8/iana/etypeID"
+	"golang.org/x/crypto/pbkdf2"
+)
+
+const (
+	s2kParamsZero = 32768
+)
+
+// DeriveRandom for key derivation as defined in RFC 8009
+func DeriveRandom(protocolKey, usage []byte, e etype.EType) ([]byte, error) {
+	h := e.GetHashFunc()()
+	return KDF_HMAC_SHA2(protocolKey, []byte("prf"), usage, h.Size(), e), nil
+}
+
+// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods.
+//
+// https://tools.ietf.org/html/rfc8009#section-5
+func DeriveKey(protocolKey, label []byte, e etype.EType) []byte {
+	var context []byte
+	var kl int
+	// Key length is longer for aes256-cts-hmac-sha384-192 is it is a Ke or from StringToKey (where label is "kerberos")
+	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
+	Swtch:
+		switch label[len(label)-1] {
+		case 0x73:
+			// 0x73 is "s" so label could be kerberos meaning StringToKey so now check if the label is "kerberos"
+			kerblabel := []byte("kerberos")
+			if len(label) != len(kerblabel) {
+				break
+			}
+			for i, b := range label {
+				if b != kerblabel[i] {
+					kl = e.GetKeySeedBitLength()
+					break Swtch
+				}
+			}
+			if kl == 0 {
+				// This is StringToKey
+				kl = 256
+			}
+		case 0xAA:
+			// This is a Ke
+			kl = 256
+		}
+	}
+	if kl == 0 {
+		kl = e.GetKeySeedBitLength()
+	}
+	return e.RandomToKey(KDF_HMAC_SHA2(protocolKey, label, context, kl, e))
+}
+
+// RandomToKey returns a key from the bytes provided according to the definition in RFC 8009.
+func RandomToKey(b []byte) []byte {
+	return b
+}
+
+// StringToKey returns a key derived from the string provided according to the definition in RFC 8009.
+func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) {
+	i, err := S2KparamsToItertions(s2kparams)
+	if err != nil {
+		return nil, err
+	}
+	return StringToKeyIter(secret, salt, i, e)
+}
+
+// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 8009.
+func StringToKeyIter(secret, salt string, iterations int, e etype.EType) ([]byte, error) {
+	tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e))
+	return e.DeriveKey(tkey, []byte("kerberos"))
+}
+
+// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0
+func StringToPBKDF2(secret, salt string, iterations int, e etype.EType) []byte {
+	kl := e.GetKeyByteSize()
+	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
+		kl = 32
+	}
+	return pbkdf2.Key([]byte(secret), []byte(salt), iterations, kl, e.GetHashFunc())
+}
+
+// KDF_HMAC_SHA2 key derivation: https://tools.ietf.org/html/rfc8009#section-3
+func KDF_HMAC_SHA2(protocolKey, label, context []byte, kl int, e etype.EType) []byte {
+	//k: Length in bits of the key to be outputted, expressed in big-endian binary representation in 4 bytes.
+	k := make([]byte, 4, 4)
+	binary.BigEndian.PutUint32(k, uint32(kl))
+
+	c := make([]byte, 4, 4)
+	binary.BigEndian.PutUint32(c, uint32(1))
+	c = append(c, label...)
+	c = append(c, byte(0))
+	if len(context) > 0 {
+		c = append(c, context...)
+	}
+	c = append(c, k...)
+
+	mac := hmac.New(e.GetHashFunc(), protocolKey)
+	mac.Write(c)
+	return mac.Sum(nil)[:(kl / 8)]
+}
+
+// GetSaltP returns the salt value based on the etype name: https://tools.ietf.org/html/rfc8009#section-4
+func GetSaltP(salt, ename string) string {
+	b := []byte(ename)
+	b = append(b, byte(0))
+	b = append(b, []byte(salt)...)
+	return string(b)
+}
+
+// S2KparamsToItertions converts the string representation of iterations to an integer for RFC 8009.
+func S2KparamsToItertions(s2kparams string) (int, error) {
+	var i uint32
+	if len(s2kparams) != 8 {
+		return s2kParamsZero, errors.New("Invalid s2kparams length")
+	}
+	b, err := hex.DecodeString(s2kparams)
+	if err != nil {
+		return s2kParamsZero, errors.New("Invalid s2kparams, cannot decode string to bytes")
+	}
+	i = binary.BigEndian.Uint32(b)
+	//buf := bytes.NewBuffer(b)
+	//err = binary.Read(buf, binary.BigEndian, &i)
+	if err != nil {
+		return s2kParamsZero, errors.New("Invalid s2kparams, cannot convert to big endian int32")
+	}
+	return int(i), nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go
new file mode 100644
index 0000000..ab8daa2
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go
@@ -0,0 +1,174 @@
+package gssapi
+
+import (
+	"bytes"
+	"crypto/hmac"
+	"encoding/binary"
+	"encoding/hex"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// RFC 4121, section 4.2.6.1
+
+const (
+	// MICTokenFlagSentByAcceptor - this flag indicates the sender is the context acceptor.  When not set, it indicates the sender is the context initiator
+	MICTokenFlagSentByAcceptor = 1 << iota
+	// MICTokenFlagSealed - this flag indicates confidentiality is provided for.  It SHALL NOT be set in MIC tokens
+	MICTokenFlagSealed
+	// MICTokenFlagAcceptorSubkey - a subkey asserted by the context acceptor is used to protect the message
+	MICTokenFlagAcceptorSubkey
+)
+
+const (
+	micHdrLen = 16 // Length of the MIC Token's header
+)
+
+// MICToken represents a GSS API MIC token, as defined in RFC 4121.
+// It contains the header fields, the payload (this is not transmitted) and
+// the checksum, and provides the logic for converting to/from bytes plus
+// computing and verifying checksums
+type MICToken struct {
+	// const GSS Token ID: 0x0404
+	Flags byte // contains three flags: acceptor, sealed, acceptor subkey
+	// const Filler: 0xFF 0xFF 0xFF 0xFF 0xFF
+	SndSeqNum uint64 // sender's sequence number. big-endian
+	Payload   []byte // your data! :)
+	Checksum  []byte // checksum of { payload | header }
+}
+
+// Return the 2 bytes identifying a GSS API MIC token
+func getGSSMICTokenID() *[2]byte {
+	return &[2]byte{0x04, 0x04}
+}
+
+// Return the filler bytes used in header
+func fillerBytes() *[5]byte {
+	return &[5]byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF}
+}
+
+// Marshal the MICToken into a byte slice.
+// The payload should have been set and the checksum computed, otherwise an error is returned.
+func (mt *MICToken) Marshal() ([]byte, error) {
+	if mt.Checksum == nil {
+		return nil, errors.New("checksum has not been set")
+	}
+
+	bytes := make([]byte, micHdrLen+len(mt.Checksum))
+	copy(bytes[0:micHdrLen], mt.getMICChecksumHeader()[:])
+	copy(bytes[micHdrLen:], mt.Checksum)
+
+	return bytes, nil
+}
+
+// SetChecksum uses the passed encryption key and key usage to compute the checksum over the payload and
+// the header, and sets the Checksum field of this MICToken.
+// If the payload has not been set or the checksum has already been set, an error is returned.
+func (mt *MICToken) SetChecksum(key types.EncryptionKey, keyUsage uint32) error {
+	if mt.Checksum != nil {
+		return errors.New("checksum has already been computed")
+	}
+	checksum, err := mt.checksum(key, keyUsage)
+	if err != nil {
+		return err
+	}
+	mt.Checksum = checksum
+	return nil
+}
+
+// Compute and return the checksum of this token, computed using the passed key and key usage.
+// Note: This will NOT update the struct's Checksum field.
+func (mt *MICToken) checksum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) {
+	if mt.Payload == nil {
+		return nil, errors.New("cannot compute checksum with uninitialized payload")
+	}
+	d := make([]byte, micHdrLen+len(mt.Payload))
+	copy(d[0:], mt.Payload)
+	copy(d[len(mt.Payload):], mt.getMICChecksumHeader())
+
+	encType, err := crypto.GetEtype(key.KeyType)
+	if err != nil {
+		return nil, err
+	}
+	return encType.GetChecksumHash(key.KeyValue, d, keyUsage)
+}
+
+// Build a header suitable for a checksum computation
+func (mt *MICToken) getMICChecksumHeader() []byte {
+	header := make([]byte, micHdrLen)
+	copy(header[0:2], getGSSMICTokenID()[:])
+	header[2] = mt.Flags
+	copy(header[3:8], fillerBytes()[:])
+	binary.BigEndian.PutUint64(header[8:16], mt.SndSeqNum)
+	return header
+}
+
+// Verify computes the token's checksum with the provided key and usage,
+// and compares it to the checksum present in the token.
+// In case of any failure, (false, err) is returned, with err an explanatory error.
+func (mt *MICToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) {
+	computed, err := mt.checksum(key, keyUsage)
+	if err != nil {
+		return false, err
+	}
+	if !hmac.Equal(computed, mt.Checksum) {
+		return false, fmt.Errorf(
+			"checksum mismatch. Computed: %s, Contained in token: %s",
+			hex.EncodeToString(computed), hex.EncodeToString(mt.Checksum))
+	}
+	return true, nil
+}
+
+// Unmarshal bytes into the corresponding MICToken.
+// If expectFromAcceptor is true we expect the token to have been emitted by the gss acceptor,
+// and will check the according flag, returning an error if the token does not match the expectation.
+func (mt *MICToken) Unmarshal(b []byte, expectFromAcceptor bool) error {
+	if len(b) < micHdrLen {
+		return errors.New("bytes shorter than header length")
+	}
+	if !bytes.Equal(getGSSMICTokenID()[:], b[0:2]) {
+		return fmt.Errorf("wrong Token ID, Expected %s, was %s",
+			hex.EncodeToString(getGSSMICTokenID()[:]),
+			hex.EncodeToString(b[0:2]))
+	}
+	flags := b[2]
+	isFromAcceptor := flags&MICTokenFlagSentByAcceptor != 0
+	if isFromAcceptor && !expectFromAcceptor {
+		return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor")
+	}
+	if !isFromAcceptor && expectFromAcceptor {
+		return errors.New("unexpected acceptor flag is not set: expecting a token from the acceptor, not in the initiator")
+	}
+	if !bytes.Equal(b[3:8], fillerBytes()[:]) {
+		return fmt.Errorf("unexpected filler bytes: expecting %s, was %s",
+			hex.EncodeToString(fillerBytes()[:]),
+			hex.EncodeToString(b[3:8]))
+	}
+
+	mt.Flags = flags
+	mt.SndSeqNum = binary.BigEndian.Uint64(b[8:16])
+	mt.Checksum = b[micHdrLen:]
+	return nil
+}
+
+// NewInitiatorMICToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum.
+// Other flags are set to 0.
+// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier.
+// This is currently not supported.
+func NewInitiatorMICToken(payload []byte, key types.EncryptionKey) (*MICToken, error) {
+	token := MICToken{
+		Flags:     0x00,
+		SndSeqNum: 0,
+		Payload:   payload,
+	}
+
+	if err := token.SetChecksum(key, keyusage.GSSAPI_INITIATOR_SIGN); err != nil {
+		return nil, err
+	}
+
+	return &token, nil
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/README.md b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/README.md
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/README.md
rename to vendor/github.com/jcmturner/gokrb5/v8/gssapi/README.md
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go
new file mode 100644
index 0000000..8c91859
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go
@@ -0,0 +1,27 @@
+package gssapi
+
+import "github.com/jcmturner/gofork/encoding/asn1"
+
+// GSS-API context flags assigned numbers.
+const (
+	ContextFlagDeleg    = 1
+	ContextFlagMutual   = 2
+	ContextFlagReplay   = 4
+	ContextFlagSequence = 8
+	ContextFlagConf     = 16
+	ContextFlagInteg    = 32
+	ContextFlagAnon     = 64
+)
+
+// ContextFlags flags for GSSAPI
+// DEPRECATED - do not use
+type ContextFlags asn1.BitString
+
+// NewContextFlags creates a new ContextFlags instance
+// DEPRECATED - do not use
+func NewContextFlags() ContextFlags {
+	var c ContextFlags
+	c.BitLength = 32
+	c.Bytes = make([]byte, 4)
+	return c
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go
new file mode 100644
index 0000000..8082231
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go
@@ -0,0 +1,202 @@
+// Package gssapi implements Generic Security Services Application Program Interface required for SPNEGO kerberos authentication.
+package gssapi
+
+import (
+	"context"
+	"fmt"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+)
+
+// GSS-API OID names
+const (
+	// GSS-API OID names
+	OIDKRB5         OIDName = "KRB5"         // MechType OID for Kerberos 5
+	OIDMSLegacyKRB5 OIDName = "MSLegacyKRB5" // MechType OID for Kerberos 5
+	OIDSPNEGO       OIDName = "SPNEGO"
+	OIDGSSIAKerb    OIDName = "GSSIAKerb" // Indicates the client cannot get a service ticket and asks the server to serve as an intermediate to the target KDC. http://k5wiki.kerberos.org/wiki/Projects/IAKERB#IAKERB_mech
+)
+
+// GSS-API status values
+const (
+	StatusBadBindings = 1 << iota
+	StatusBadMech
+	StatusBadName
+	StatusBadNameType
+	StatusBadStatus
+	StatusBadSig
+	StatusBadMIC
+	StatusContextExpired
+	StatusCredentialsExpired
+	StatusDefectiveCredential
+	StatusDefectiveToken
+	StatusFailure
+	StatusNoContext
+	StatusNoCred
+	StatusBadQOP
+	StatusUnauthorized
+	StatusUnavailable
+	StatusDuplicateElement
+	StatusNameNotMN
+	StatusComplete
+	StatusContinueNeeded
+	StatusDuplicateToken
+	StatusOldToken
+	StatusUnseqToken
+	StatusGapToken
+)
+
+// ContextToken is an interface for a GSS-API context token.
+type ContextToken interface {
+	Marshal() ([]byte, error)
+	Unmarshal(b []byte) error
+	Verify() (bool, Status)
+	Context() context.Context
+}
+
+/*
+CREDENTIAL MANAGEMENT
+
+GSS_Acquire_cred             acquire credentials for use
+GSS_Release_cred             release credentials after use
+GSS_Inquire_cred             display information about credentials
+GSS_Add_cred                 construct credentials incrementally
+GSS_Inquire_cred_by_mech     display per-mechanism credential information
+
+CONTEXT-LEVEL CALLS
+
+GSS_Init_sec_context         initiate outbound security context
+GSS_Accept_sec_context       accept inbound security context
+GSS_Delete_sec_context       flush context when no longer needed
+GSS_Process_context_token    process received control token on context
+GSS_Context_time             indicate validity time remaining on context
+GSS_Inquire_context          display information about context
+GSS_Wrap_size_limit          determine GSS_Wrap token size limit
+GSS_Export_sec_context       transfer context to other process
+GSS_Import_sec_context       import transferred context
+
+PER-MESSAGE CALLS
+
+GSS_GetMIC                   apply integrity check, receive as token separate from message
+GSS_VerifyMIC                validate integrity check token along with message
+GSS_Wrap                     sign, optionally encrypt, encapsulate
+GSS_Unwrap                   decapsulate, decrypt if needed, validate integrity check
+
+SUPPORT CALLS
+
+GSS_Display_status           translate status codes to printable form
+GSS_Indicate_mechs           indicate mech_types supported on local system
+GSS_Compare_name             compare two names for equality
+GSS_Display_name             translate name to printable form
+GSS_Import_name              convert printable name to normalized form
+GSS_Release_name             free storage of normalized-form name
+GSS_Release_buffer           free storage of general GSS-allocated object
+GSS_Release_OID_set          free storage of OID set object
+GSS_Create_empty_OID_set     create empty OID set
+GSS_Add_OID_set_member       add member to OID set
+GSS_Test_OID_set_member      test if OID is member of OID set
+GSS_Inquire_names_for_mech   indicate name types supported by mechanism
+GSS_Inquire_mechs_for_name   indicates mechanisms supporting name type
+GSS_Canonicalize_name        translate name to per-mechanism form
+GSS_Export_name              externalize per-mechanism name
+GSS_Duplicate_name           duplicate name object
+*/
+
+// Mechanism is the GSS-API interface for authentication mechanisms.
+type Mechanism interface {
+	OID() asn1.ObjectIdentifier
+	AcquireCred() error                                               // acquire credentials for use (eg. AS exchange for KRB5)
+	InitSecContext() (ContextToken, error)                            // initiate outbound security context (eg TGS exchange builds AP_REQ to go into ContextToken to send to service)
+	AcceptSecContext(ct ContextToken) (bool, context.Context, Status) // service verifies the token server side to establish a context
+	MIC() MICToken                                                    // apply integrity check, receive as token separate from message
+	VerifyMIC(mt MICToken) (bool, error)                              // validate integrity check token along with message
+	Wrap(msg []byte) WrapToken                                        // sign, optionally encrypt, encapsulate
+	Unwrap(wt WrapToken) []byte                                       // decapsulate, decrypt if needed, validate integrity check
+}
+
+// OIDName is the type for defined GSS-API OIDs.
+type OIDName string
+
+// OID returns the OID for the provided OID name.
+func (o OIDName) OID() asn1.ObjectIdentifier {
+	switch o {
+	case OIDSPNEGO:
+		return asn1.ObjectIdentifier{1, 3, 6, 1, 5, 5, 2}
+	case OIDKRB5:
+		return asn1.ObjectIdentifier{1, 2, 840, 113554, 1, 2, 2}
+	case OIDMSLegacyKRB5:
+		return asn1.ObjectIdentifier{1, 2, 840, 48018, 1, 2, 2}
+	case OIDGSSIAKerb:
+		return asn1.ObjectIdentifier{1, 3, 6, 1, 5, 2, 5}
+	}
+	return asn1.ObjectIdentifier{}
+}
+
+// Status is the GSS-API status and implements the error interface.
+type Status struct {
+	Code    int
+	Message string
+}
+
+// Error returns the Status description.
+func (s Status) Error() string {
+	var str string
+	switch s.Code {
+	case StatusBadBindings:
+		str = "channel binding mismatch"
+	case StatusBadMech:
+		str = "unsupported mechanism requested"
+	case StatusBadName:
+		str = "invalid name provided"
+	case StatusBadNameType:
+		str = "name of unsupported type provided"
+	case StatusBadStatus:
+		str = "invalid input status selector"
+	case StatusBadSig:
+		str = "token had invalid integrity check"
+	case StatusBadMIC:
+		str = "preferred alias for GSS_S_BAD_SIG"
+	case StatusContextExpired:
+		str = "specified security context expired"
+	case StatusCredentialsExpired:
+		str = "expired credentials detected"
+	case StatusDefectiveCredential:
+		str = "defective credential detected"
+	case StatusDefectiveToken:
+		str = "defective token detected"
+	case StatusFailure:
+		str = "failure, unspecified at GSS-API level"
+	case StatusNoContext:
+		str = "no valid security context specified"
+	case StatusNoCred:
+		str = "no valid credentials provided"
+	case StatusBadQOP:
+		str = "unsupported QOP valu"
+	case StatusUnauthorized:
+		str = "operation unauthorized"
+	case StatusUnavailable:
+		str = "operation unavailable"
+	case StatusDuplicateElement:
+		str = "duplicate credential element requested"
+	case StatusNameNotMN:
+		str = "name contains multi-mechanism elements"
+	case StatusComplete:
+		str = "normal completion"
+	case StatusContinueNeeded:
+		str = "continuation call to routine required"
+	case StatusDuplicateToken:
+		str = "duplicate per-message token detected"
+	case StatusOldToken:
+		str = "timed-out per-message token detected"
+	case StatusUnseqToken:
+		str = "reordered (early) per-message token detected"
+	case StatusGapToken:
+		str = "skipped predecessor token(s) detected"
+	default:
+		str = "unknown GSS-API error status"
+	}
+	if s.Message != "" {
+		return fmt.Sprintf("%s: %s", str, s.Message)
+	}
+	return str
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go
new file mode 100644
index 0000000..ea7d054
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go
@@ -0,0 +1,195 @@
+package gssapi
+
+import (
+	"bytes"
+	"crypto/hmac"
+	"encoding/binary"
+	"encoding/hex"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// RFC 4121, section 4.2.6.2
+
+const (
+	// HdrLen is the length of the Wrap Token's header
+	HdrLen = 16
+	// FillerByte is a filler in the WrapToken structure
+	FillerByte byte = 0xFF
+)
+
+// WrapToken represents a GSS API Wrap token, as defined in RFC 4121.
+// It contains the header fields, the payload and the checksum, and provides
+// the logic for converting to/from bytes plus computing and verifying checksums
+type WrapToken struct {
+	// const GSS Token ID: 0x0504
+	Flags byte // contains three flags: acceptor, sealed, acceptor subkey
+	// const Filler: 0xFF
+	EC        uint16 // checksum length. big-endian
+	RRC       uint16 // right rotation count. big-endian
+	SndSeqNum uint64 // sender's sequence number. big-endian
+	Payload   []byte // your data! :)
+	CheckSum  []byte // authenticated checksum of { payload | header }
+}
+
+// Return the 2 bytes identifying a GSS API Wrap token
+func getGssWrapTokenId() *[2]byte {
+	return &[2]byte{0x05, 0x04}
+}
+
+// Marshal the WrapToken into a byte slice.
+// The payload should have been set and the checksum computed, otherwise an error is returned.
+func (wt *WrapToken) Marshal() ([]byte, error) {
+	if wt.CheckSum == nil {
+		return nil, errors.New("checksum has not been set")
+	}
+	if wt.Payload == nil {
+		return nil, errors.New("payload has not been set")
+	}
+
+	pldOffset := HdrLen                    // Offset of the payload in the token
+	chkSOffset := HdrLen + len(wt.Payload) // Offset of the checksum in the token
+
+	bytes := make([]byte, chkSOffset+int(wt.EC))
+	copy(bytes[0:], getGssWrapTokenId()[:])
+	bytes[2] = wt.Flags
+	bytes[3] = FillerByte
+	binary.BigEndian.PutUint16(bytes[4:6], wt.EC)
+	binary.BigEndian.PutUint16(bytes[6:8], wt.RRC)
+	binary.BigEndian.PutUint64(bytes[8:16], wt.SndSeqNum)
+	copy(bytes[pldOffset:], wt.Payload)
+	copy(bytes[chkSOffset:], wt.CheckSum)
+	return bytes, nil
+}
+
+// SetCheckSum uses the passed encryption key and key usage to compute the checksum over the payload and
+// the header, and sets the CheckSum field of this WrapToken.
+// If the payload has not been set or the checksum has already been set, an error is returned.
+func (wt *WrapToken) SetCheckSum(key types.EncryptionKey, keyUsage uint32) error {
+	if wt.Payload == nil {
+		return errors.New("payload has not been set")
+	}
+	if wt.CheckSum != nil {
+		return errors.New("checksum has already been computed")
+	}
+	chkSum, cErr := wt.computeCheckSum(key, keyUsage)
+	if cErr != nil {
+		return cErr
+	}
+	wt.CheckSum = chkSum
+	return nil
+}
+
+// ComputeCheckSum computes and returns the checksum of this token, computed using the passed key and key usage.
+// Note: This will NOT update the struct's Checksum field.
+func (wt *WrapToken) computeCheckSum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) {
+	if wt.Payload == nil {
+		return nil, errors.New("cannot compute checksum with uninitialized payload")
+	}
+	// Build a slice containing { payload | header }
+	checksumMe := make([]byte, HdrLen+len(wt.Payload))
+	copy(checksumMe[0:], wt.Payload)
+	copy(checksumMe[len(wt.Payload):], getChecksumHeader(wt.Flags, wt.SndSeqNum))
+
+	encType, err := crypto.GetEtype(key.KeyType)
+	if err != nil {
+		return nil, err
+	}
+	return encType.GetChecksumHash(key.KeyValue, checksumMe, keyUsage)
+}
+
+// Build a header suitable for a checksum computation
+func getChecksumHeader(flags byte, senderSeqNum uint64) []byte {
+	header := make([]byte, 16)
+	copy(header[0:], []byte{0x05, 0x04, flags, 0xFF, 0x00, 0x00, 0x00, 0x00})
+	binary.BigEndian.PutUint64(header[8:], senderSeqNum)
+	return header
+}
+
+// Verify computes the token's checksum with the provided key and usage,
+// and compares it to the checksum present in the token.
+// In case of any failure, (false, Err) is returned, with Err an explanatory error.
+func (wt *WrapToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) {
+	computed, cErr := wt.computeCheckSum(key, keyUsage)
+	if cErr != nil {
+		return false, cErr
+	}
+	if !hmac.Equal(computed, wt.CheckSum) {
+		return false, fmt.Errorf(
+			"checksum mismatch. Computed: %s, Contained in token: %s",
+			hex.EncodeToString(computed), hex.EncodeToString(wt.CheckSum))
+	}
+	return true, nil
+}
+
+// Unmarshal bytes into the corresponding WrapToken.
+// If expectFromAcceptor is true, we expect the token to have been emitted by the gss acceptor,
+// and will check the according flag, returning an error if the token does not match the expectation.
+func (wt *WrapToken) Unmarshal(b []byte, expectFromAcceptor bool) error {
+	// Check if we can read a whole header
+	if len(b) < 16 {
+		return errors.New("bytes shorter than header length")
+	}
+	// Is the Token ID correct?
+	if !bytes.Equal(getGssWrapTokenId()[:], b[0:2]) {
+		return fmt.Errorf("wrong Token ID. Expected %s, was %s",
+			hex.EncodeToString(getGssWrapTokenId()[:]),
+			hex.EncodeToString(b[0:2]))
+	}
+	// Check the acceptor flag
+	flags := b[2]
+	isFromAcceptor := flags&0x01 == 1
+	if isFromAcceptor && !expectFromAcceptor {
+		return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor")
+	}
+	if !isFromAcceptor && expectFromAcceptor {
+		return errors.New("expected acceptor flag is not set: expecting a token from the acceptor, not the initiator")
+	}
+	// Check the filler byte
+	if b[3] != FillerByte {
+		return fmt.Errorf("unexpected filler byte: expecting 0xFF, was %s ", hex.EncodeToString(b[3:4]))
+	}
+	checksumL := binary.BigEndian.Uint16(b[4:6])
+	// Sanity check on the checksum length
+	if int(checksumL) > len(b)-HdrLen {
+		return fmt.Errorf("inconsistent checksum length: %d bytes to parse, checksum length is %d", len(b), checksumL)
+	}
+
+	wt.Flags = flags
+	wt.EC = checksumL
+	wt.RRC = binary.BigEndian.Uint16(b[6:8])
+	wt.SndSeqNum = binary.BigEndian.Uint64(b[8:16])
+	wt.Payload = b[16 : len(b)-int(checksumL)]
+	wt.CheckSum = b[len(b)-int(checksumL):]
+	return nil
+}
+
+// NewInitiatorWrapToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum.
+// Other flags are set to 0, and the RRC and sequence number are initialized to 0.
+// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier.
+// This is currently not supported.
+func NewInitiatorWrapToken(payload []byte, key types.EncryptionKey) (*WrapToken, error) {
+	encType, err := crypto.GetEtype(key.KeyType)
+	if err != nil {
+		return nil, err
+	}
+
+	token := WrapToken{
+		Flags: 0x00, // all zeroed out (this is a token sent by the initiator)
+		// Checksum size: length of output of the HMAC function, in bytes.
+		EC:        uint16(encType.GetHMACBitLength() / 8),
+		RRC:       0,
+		SndSeqNum: 0,
+		Payload:   payload,
+	}
+
+	if err := token.SetCheckSum(key, keyusage.GSSAPI_INITIATOR_SEAL); err != nil {
+		return nil, err
+	}
+
+	return &token, nil
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/addrtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/addrtype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/addrtype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/addrtype/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/adtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/adtype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/adtype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/adtype/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/asnAppTag/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/asnAppTag/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/chksumtype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/chksumtype/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/errorcode/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/errorcode/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/errorcode/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/errorcode/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/etypeID/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/etypeID/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/etypeID/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/etypeID/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/flags/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/flags/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/flags/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/flags/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/keyusage/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/keyusage/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/keyusage/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/keyusage/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/msgtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/msgtype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/msgtype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/msgtype/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/nametype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/nametype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/nametype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/nametype/constants.go
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/iana/patype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/patype/constants.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/iana/patype/constants.go
rename to vendor/github.com/jcmturner/gokrb5/v8/iana/patype/constants.go
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go
new file mode 100644
index 0000000..2d68eda
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go
@@ -0,0 +1,23 @@
+package kadmin
+
+import (
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// ChangePasswdData is the payload to a password change message.
+type ChangePasswdData struct {
+	NewPasswd []byte              `asn1:"explicit,tag:0"`
+	TargName  types.PrincipalName `asn1:"explicit,optional,tag:1"`
+	TargRealm string              `asn1:"generalstring,optional,explicit,tag:2"`
+}
+
+// Marshal ChangePasswdData into a byte slice.
+func (c *ChangePasswdData) Marshal() ([]byte, error) {
+	b, err := asn1.Marshal(*c)
+	if err != nil {
+		return []byte{}, err
+	}
+	//b = asn1tools.AddASNAppTag(b, asnAppTag.)
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go
new file mode 100644
index 0000000..d1864c9
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go
@@ -0,0 +1,114 @@
+package kadmin
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"math"
+
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+const (
+	verisonHex = "ff80"
+)
+
+// Request message for changing password.
+type Request struct {
+	APREQ   messages.APReq
+	KRBPriv messages.KRBPriv
+}
+
+// Reply message for a password change.
+type Reply struct {
+	MessageLength int
+	Version       int
+	APREPLength   int
+	APREP         messages.APRep
+	KRBPriv       messages.KRBPriv
+	KRBError      messages.KRBError
+	IsKRBError    bool
+	ResultCode    uint16
+	Result        string
+}
+
+// Marshal a Request into a byte slice.
+func (m *Request) Marshal() (b []byte, err error) {
+	b = []byte{255, 128} // protocol version number: contains the hex constant 0xff80 (big-endian integer).
+	ab, e := m.APREQ.Marshal()
+	if e != nil {
+		err = fmt.Errorf("error marshaling AP_REQ: %v", e)
+		return
+	}
+	if len(ab) > math.MaxUint16 {
+		err = errors.New("length of AP_REQ greater then max Uint16 size")
+		return
+	}
+	al := make([]byte, 2)
+	binary.BigEndian.PutUint16(al, uint16(len(ab)))
+	b = append(b, al...)
+	b = append(b, ab...)
+	pb, e := m.KRBPriv.Marshal()
+	if e != nil {
+		err = fmt.Errorf("error marshaling KRB_Priv: %v", e)
+		return
+	}
+	b = append(b, pb...)
+	if len(b)+2 > math.MaxUint16 {
+		err = errors.New("length of message greater then max Uint16 size")
+		return
+	}
+	ml := make([]byte, 2)
+	binary.BigEndian.PutUint16(ml, uint16(len(b)+2))
+	b = append(ml, b...)
+	return
+}
+
+// Unmarshal a byte slice into a Reply.
+func (m *Reply) Unmarshal(b []byte) error {
+	m.MessageLength = int(binary.BigEndian.Uint16(b[0:2]))
+	m.Version = int(binary.BigEndian.Uint16(b[2:4]))
+	if m.Version != 1 {
+		return fmt.Errorf("kadmin reply has incorrect protocol version number: %d", m.Version)
+	}
+	m.APREPLength = int(binary.BigEndian.Uint16(b[4:6]))
+	if m.APREPLength != 0 {
+		err := m.APREP.Unmarshal(b[6 : 6+m.APREPLength])
+		if err != nil {
+			return err
+		}
+		err = m.KRBPriv.Unmarshal(b[6+m.APREPLength : m.MessageLength])
+		if err != nil {
+			return err
+		}
+	} else {
+		m.IsKRBError = true
+		m.KRBError.Unmarshal(b[6:m.MessageLength])
+		m.ResultCode, m.Result = parseResponse(m.KRBError.EData)
+	}
+	return nil
+}
+
+func parseResponse(b []byte) (c uint16, s string) {
+	c = binary.BigEndian.Uint16(b[0:2])
+	buf := bytes.NewBuffer(b[2:])
+	m := make([]byte, len(b)-2)
+	binary.Read(buf, binary.BigEndian, &m)
+	s = string(m)
+	return
+}
+
+// Decrypt the encrypted part of the KRBError within the change password Reply.
+func (m *Reply) Decrypt(key types.EncryptionKey) error {
+	if m.IsKRBError {
+		return m.KRBError
+	}
+	err := m.KRBPriv.DecryptEncPart(key)
+	if err != nil {
+		return err
+	}
+	m.ResultCode, m.Result = parseResponse(m.KRBPriv.DecryptedEncPart.UserData)
+	return nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go
new file mode 100644
index 0000000..db199bf
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go
@@ -0,0 +1,68 @@
+// Package kadmin provides Kerberos administration capabilities.
+package kadmin
+
+import (
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/messages"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// ChangePasswdMsg generate a change password request and also return the key needed to decrypt the reply.
+func ChangePasswdMsg(cname types.PrincipalName, realm, password string, tkt messages.Ticket, sessionKey types.EncryptionKey) (r Request, k types.EncryptionKey, err error) {
+	// Create change password data struct and marshal to bytes
+	chgpasswd := ChangePasswdData{
+		NewPasswd: []byte(password),
+		TargName:  cname,
+		TargRealm: realm,
+	}
+	chpwdb, err := chgpasswd.Marshal()
+	if err != nil {
+		err = krberror.Errorf(err, krberror.KRBMsgError, "error marshaling change passwd data")
+		return
+	}
+
+	// Generate authenticator
+	auth, err := types.NewAuthenticator(realm, cname)
+	if err != nil {
+		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator")
+		return
+	}
+	etype, err := crypto.GetEtype(sessionKey.KeyType)
+	if err != nil {
+		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey etype")
+		return
+	}
+	err = auth.GenerateSeqNumberAndSubKey(etype.GetETypeID(), etype.GetKeyByteSize())
+	if err != nil {
+		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey")
+		return
+	}
+	k = auth.SubKey
+
+	// Generate AP_REQ
+	APreq, err := messages.NewAPReq(tkt, sessionKey, auth)
+	if err != nil {
+		return
+	}
+
+	// Form the KRBPriv encpart data
+	kp := messages.EncKrbPrivPart{
+		UserData:       chpwdb,
+		Timestamp:      auth.CTime,
+		Usec:           auth.Cusec,
+		SequenceNumber: auth.SeqNumber,
+	}
+	kpriv := messages.NewKRBPriv(kp)
+	err = kpriv.EncryptEncPart(k)
+	if err != nil {
+		err = krberror.Errorf(err, krberror.EncryptingError, "error encrypting change passwd data")
+		return
+	}
+
+	r = Request{
+		APREQ:   APreq,
+		KRBPriv: kpriv,
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go b/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go
new file mode 100644
index 0000000..5c2e9d7
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go
@@ -0,0 +1,530 @@
+// Package keytab implements Kerberos keytabs: https://web.mit.edu/kerberos/krb5-devel/doc/formats/keytab_file_format.html.
+package keytab
+
+import (
+	"bytes"
+	"encoding/binary"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"strings"
+	"time"
+	"unsafe"
+
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+const (
+	keytabFirstByte byte = 05
+)
+
+// Keytab struct.
+type Keytab struct {
+	version uint8
+	Entries []entry
+}
+
+// Keytab entry struct.
+type entry struct {
+	Principal principal
+	Timestamp time.Time
+	KVNO8     uint8
+	Key       types.EncryptionKey
+	KVNO      uint32
+}
+
+func (e entry) String() string {
+	return fmt.Sprintf("% 4d %s %-56s %2d %-64x",
+		e.KVNO8,
+		e.Timestamp.Format("02/01/06 15:04:05"),
+		e.Principal.String(),
+		e.Key.KeyType,
+		e.Key.KeyValue,
+	)
+}
+
+// Keytab entry principal struct.
+type principal struct {
+	NumComponents int16 `json:"-"`
+	Realm         string
+	Components    []string
+	NameType      int32
+}
+
+func (p principal) String() string {
+	return fmt.Sprintf("%s@%s", strings.Join(p.Components, "/"), p.Realm)
+}
+
+// New creates new, empty Keytab type.
+func New() *Keytab {
+	var e []entry
+	return &Keytab{
+		version: 2,
+		Entries: e,
+	}
+}
+
+// GetEncryptionKey returns the EncryptionKey from the Keytab for the newest entry with the required kvno, etype and matching principal.
+// If the kvno is zero then the latest kvno will be returned. The kvno is also returned for
+func (kt *Keytab) GetEncryptionKey(princName types.PrincipalName, realm string, kvno int, etype int32) (types.EncryptionKey, int, error) {
+	var key types.EncryptionKey
+	var t time.Time
+	var kv int
+	for _, k := range kt.Entries {
+		if k.Principal.Realm == realm && len(k.Principal.Components) == len(princName.NameString) &&
+			k.Key.KeyType == etype &&
+			(k.KVNO == uint32(kvno) || kvno == 0) &&
+			k.Timestamp.After(t) {
+			p := true
+			for i, n := range k.Principal.Components {
+				if princName.NameString[i] != n {
+					p = false
+					break
+				}
+			}
+			if p {
+				key = k.Key
+				kv = int(k.KVNO)
+				t = k.Timestamp
+			}
+		}
+	}
+	if len(key.KeyValue) < 1 {
+		return key, 0, fmt.Errorf("matching key not found in keytab. Looking for %v realm: %v kvno: %v etype: %v", princName.NameString, realm, kvno, etype)
+	}
+	return key, kv, nil
+}
+
+// Create a new Keytab entry.
+func newEntry() entry {
+	var b []byte
+	return entry{
+		Principal: newPrincipal(),
+		Timestamp: time.Time{},
+		KVNO8:     0,
+		Key: types.EncryptionKey{
+			KeyType:  0,
+			KeyValue: b,
+		},
+		KVNO: 0,
+	}
+}
+
+func (kt Keytab) String() string {
+	var s string
+	s = `KVNO Timestamp         Principal                                                ET Key
+---- ----------------- -------------------------------------------------------- -- ----------------------------------------------------------------
+`
+	for _, entry := range kt.Entries {
+		s += entry.String() + "\n"
+	}
+	return s
+}
+
+// AddEntry adds an entry to the keytab. The password should be provided in plain text and it will be converted using the defined enctype to be stored.
+func (kt *Keytab) AddEntry(principalName, realm, password string, ts time.Time, KVNO uint8, encType int32) error {
+	// Generate a key from the password
+	princ, _ := types.ParseSPNString(principalName)
+	key, _, err := crypto.GetKeyFromPassword(password, princ, realm, encType, types.PADataSequence{})
+	if err != nil {
+		return err
+	}
+
+	// Populate the keytab entry principal
+	ktep := newPrincipal()
+	ktep.NumComponents = int16(len(princ.NameString))
+	if kt.version == 1 {
+		ktep.NumComponents += 1
+	}
+
+	ktep.Realm = realm
+	ktep.Components = princ.NameString
+	ktep.NameType = princ.NameType
+
+	// Populate the keytab entry
+	e := newEntry()
+	e.Principal = ktep
+	e.Timestamp = ts
+	e.KVNO8 = KVNO
+	e.KVNO = uint32(KVNO)
+	e.Key = key
+
+	kt.Entries = append(kt.Entries, e)
+	return nil
+}
+
+// Create a new principal.
+func newPrincipal() principal {
+	var c []string
+	return principal{
+		NumComponents: 0,
+		Realm:         "",
+		Components:    c,
+		NameType:      0,
+	}
+}
+
+// Load a Keytab file into a Keytab type.
+func Load(ktPath string) (*Keytab, error) {
+	kt := new(Keytab)
+	b, err := ioutil.ReadFile(ktPath)
+	if err != nil {
+		return kt, err
+	}
+	err = kt.Unmarshal(b)
+	return kt, err
+}
+
+// Marshal keytab into byte slice
+func (kt *Keytab) Marshal() ([]byte, error) {
+	b := []byte{keytabFirstByte, kt.version}
+	for _, e := range kt.Entries {
+		eb, err := e.marshal(int(kt.version))
+		if err != nil {
+			return b, err
+		}
+		b = append(b, eb...)
+	}
+	return b, nil
+}
+
+// Write the keytab bytes to io.Writer.
+// Returns the number of bytes written
+func (kt *Keytab) Write(w io.Writer) (int, error) {
+	b, err := kt.Marshal()
+	if err != nil {
+		return 0, fmt.Errorf("error marshaling keytab: %v", err)
+	}
+	return w.Write(b)
+}
+
+// Unmarshal byte slice of Keytab data into Keytab type.
+func (kt *Keytab) Unmarshal(b []byte) error {
+	if len(b) < 2 {
+		return fmt.Errorf("byte array is less than 2 bytes: %d", len(b))
+	}
+
+	//The first byte of the file always has the value 5
+	if b[0] != keytabFirstByte {
+		return errors.New("invalid keytab data. First byte does not equal 5")
+	}
+	//Get keytab version
+	//The 2nd byte contains the version number (1 or 2)
+	kt.version = b[1]
+	if kt.version != 1 && kt.version != 2 {
+		return errors.New("invalid keytab data. Keytab version is neither 1 nor 2")
+	}
+	//Version 1 of the file format uses native byte order for integer representations. Version 2 always uses big-endian byte order
+	var endian binary.ByteOrder
+	endian = binary.BigEndian
+	if kt.version == 1 && isNativeEndianLittle() {
+		endian = binary.LittleEndian
+	}
+	// n tracks position in the byte array
+	n := 2
+	l, err := readInt32(b, &n, &endian)
+	if err != nil {
+		return err
+	}
+	for l != 0 {
+		if l < 0 {
+			//Zero padded so skip over
+			l = l * -1
+			n = n + int(l)
+		} else {
+			if n < 0 {
+				return fmt.Errorf("%d can't be less than zero", n)
+			}
+			if n+int(l) > len(b) {
+				return fmt.Errorf("%s's length is less than %d", b, n+int(l))
+			}
+			eb := b[n : n+int(l)]
+			n = n + int(l)
+			ke := newEntry()
+			// p keeps track as to where we are in the byte stream
+			var p int
+			var err error
+			parsePrincipal(eb, &p, kt, &ke, &endian)
+			ke.Timestamp, err = readTimestamp(eb, &p, &endian)
+			if err != nil {
+				return err
+			}
+			rei8, err := readInt8(eb, &p, &endian)
+			if err != nil {
+				return err
+			}
+			ke.KVNO8 = uint8(rei8)
+			rei16, err := readInt16(eb, &p, &endian)
+			if err != nil {
+				return err
+			}
+			ke.Key.KeyType = int32(rei16)
+			rei16, err = readInt16(eb, &p, &endian)
+			if err != nil {
+				return err
+			}
+			kl := int(rei16)
+			ke.Key.KeyValue, err = readBytes(eb, &p, kl, &endian)
+			if err != nil {
+				return err
+			}
+			// The 32-bit key version overrides the 8-bit key version.
+			// If at least 4 bytes are left after the other fields are read and they are non-zero
+			// this indicates the 32-bit version is present.
+			if len(eb)-p >= 4 {
+				// The 32-bit key may be present
+				ri32, err := readInt32(eb, &p, &endian)
+				if err != nil {
+					return err
+				}
+				ke.KVNO = uint32(ri32)
+			}
+			if ke.KVNO == 0 {
+				// Handles if the value from the last 4 bytes was zero and also if there are not the 4 bytes present. Makes sense to put the same value here as KVNO8
+				ke.KVNO = uint32(ke.KVNO8)
+			}
+			// Add the entry to the keytab
+			kt.Entries = append(kt.Entries, ke)
+		}
+		// Check if there are still 4 bytes left to read
+		// Also check that n is greater than zero
+		if n < 0 || n > len(b) || len(b[n:]) < 4 {
+			break
+		}
+		// Read the size of the next entry
+		l, err = readInt32(b, &n, &endian)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (e entry) marshal(v int) ([]byte, error) {
+	var b []byte
+	pb, err := e.Principal.marshal(v)
+	if err != nil {
+		return b, err
+	}
+	b = append(b, pb...)
+
+	var endian binary.ByteOrder
+	endian = binary.BigEndian
+	if v == 1 && isNativeEndianLittle() {
+		endian = binary.LittleEndian
+	}
+
+	t := make([]byte, 9)
+	endian.PutUint32(t[0:4], uint32(e.Timestamp.Unix()))
+	t[4] = e.KVNO8
+	endian.PutUint16(t[5:7], uint16(e.Key.KeyType))
+	endian.PutUint16(t[7:9], uint16(len(e.Key.KeyValue)))
+	b = append(b, t...)
+
+	buf := new(bytes.Buffer)
+	err = binary.Write(buf, endian, e.Key.KeyValue)
+	if err != nil {
+		return b, err
+	}
+	b = append(b, buf.Bytes()...)
+
+	t = make([]byte, 4)
+	endian.PutUint32(t, e.KVNO)
+	b = append(b, t...)
+
+	// Add the length header
+	t = make([]byte, 4)
+	endian.PutUint32(t, uint32(len(b)))
+	b = append(t, b...)
+	return b, nil
+}
+
+// Parse the Keytab bytes of a principal into a Keytab entry's principal.
+func parsePrincipal(b []byte, p *int, kt *Keytab, ke *entry, e *binary.ByteOrder) error {
+	var err error
+	ke.Principal.NumComponents, err = readInt16(b, p, e)
+	if err != nil {
+		return err
+	}
+	if kt.version == 1 {
+		//In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2
+		ke.Principal.NumComponents--
+	}
+	lenRealm, err := readInt16(b, p, e)
+	if err != nil {
+		return err
+	}
+	realmB, err := readBytes(b, p, int(lenRealm), e)
+	if err != nil {
+		return err
+	}
+	ke.Principal.Realm = string(realmB)
+	for i := 0; i < int(ke.Principal.NumComponents); i++ {
+		l, err := readInt16(b, p, e)
+		if err != nil {
+			return err
+		}
+		compB, err := readBytes(b, p, int(l), e)
+		if err != nil {
+			return err
+		}
+		ke.Principal.Components = append(ke.Principal.Components, string(compB))
+	}
+	if kt.version != 1 {
+		//Name Type is omitted in version 1
+		ke.Principal.NameType, err = readInt32(b, p, e)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (p principal) marshal(v int) ([]byte, error) {
+	//var b []byte
+	b := make([]byte, 2)
+	var endian binary.ByteOrder
+	endian = binary.BigEndian
+	if v == 1 && isNativeEndianLittle() {
+		endian = binary.LittleEndian
+	}
+	endian.PutUint16(b[0:], uint16(p.NumComponents))
+	realm, err := marshalString(p.Realm, v)
+	if err != nil {
+		return b, err
+	}
+	b = append(b, realm...)
+	for _, c := range p.Components {
+		cb, err := marshalString(c, v)
+		if err != nil {
+			return b, err
+		}
+		b = append(b, cb...)
+	}
+	if v != 1 {
+		t := make([]byte, 4)
+		endian.PutUint32(t, uint32(p.NameType))
+		b = append(b, t...)
+	}
+	return b, nil
+}
+
+func marshalString(s string, v int) ([]byte, error) {
+	sb := []byte(s)
+	b := make([]byte, 2)
+	var endian binary.ByteOrder
+	endian = binary.BigEndian
+	if v == 1 && isNativeEndianLittle() {
+		endian = binary.LittleEndian
+	}
+	endian.PutUint16(b[0:], uint16(len(sb)))
+	buf := new(bytes.Buffer)
+	err := binary.Write(buf, endian, sb)
+	if err != nil {
+		return b, err
+	}
+	b = append(b, buf.Bytes()...)
+	return b, err
+}
+
+// Read bytes representing a timestamp.
+func readTimestamp(b []byte, p *int, e *binary.ByteOrder) (time.Time, error) {
+	i32, err := readInt32(b, p, e)
+	if err != nil {
+		return time.Time{}, err
+	}
+	return time.Unix(int64(i32), 0), nil
+}
+
+// Read bytes representing an eight bit integer.
+func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8, err error) {
+	if *p < 0 {
+		return 0, fmt.Errorf("%d cannot be less than zero", *p)
+	}
+
+	if (*p + 1) > len(b) {
+		return 0, fmt.Errorf("%s's length is less than %d", b, *p+1)
+	}
+	buf := bytes.NewBuffer(b[*p : *p+1])
+	binary.Read(buf, *e, &i)
+	*p++
+	return
+}
+
+// Read bytes representing a sixteen bit integer.
+func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16, err error) {
+	if *p < 0 {
+		return 0, fmt.Errorf("%d cannot be less than zero", *p)
+	}
+
+	if (*p + 2) > len(b) {
+		return 0, fmt.Errorf("%s's length is less than %d", b, *p+2)
+	}
+
+	buf := bytes.NewBuffer(b[*p : *p+2])
+	binary.Read(buf, *e, &i)
+	*p += 2
+	return
+}
+
+// Read bytes representing a thirty two bit integer.
+func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32, err error) {
+	if *p < 0 {
+		return 0, fmt.Errorf("%d cannot be less than zero", *p)
+	}
+
+	if (*p + 4) > len(b) {
+		return 0, fmt.Errorf("%s's length is less than %d", b, *p+4)
+	}
+
+	buf := bytes.NewBuffer(b[*p : *p+4])
+	binary.Read(buf, *e, &i)
+	*p += 4
+	return
+}
+
+func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) ([]byte, error) {
+	if s < 0 {
+		return nil, fmt.Errorf("%d cannot be less than zero", s)
+	}
+	i := *p + s
+	if i > len(b) {
+		return nil, fmt.Errorf("%s's length is greater than %d", b, i)
+	}
+	buf := bytes.NewBuffer(b[*p:i])
+	r := make([]byte, s)
+	if err := binary.Read(buf, *e, &r); err != nil {
+		return nil, err
+	}
+	*p += s
+	return r, nil
+}
+
+func isNativeEndianLittle() bool {
+	var x = 0x012345678
+	var p = unsafe.Pointer(&x)
+	var bp = (*[4]byte)(p)
+
+	var endian bool
+	if 0x01 == bp[0] {
+		endian = false
+	} else if (0x78 & 0xff) == (bp[0] & 0xff) {
+		endian = true
+	} else {
+		// Default to big endian
+		endian = false
+	}
+	return endian
+}
+
+// JSON return information about the keys held in the keytab in a JSON format.
+func (kt *Keytab) JSON() (string, error) {
+	b, err := json.MarshalIndent(kt, "", "  ")
+	if err != nil {
+		return "", err
+	}
+	return string(b), nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go b/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go
new file mode 100644
index 0000000..01c6d99
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go
@@ -0,0 +1,67 @@
+// Package krberror provides error type and functions for gokrb5.
+package krberror
+
+import (
+	"fmt"
+	"strings"
+)
+
+// Error type descriptions.
+const (
+	separator       = " < "
+	EncodingError   = "Encoding_Error"
+	NetworkingError = "Networking_Error"
+	DecryptingError = "Decrypting_Error"
+	EncryptingError = "Encrypting_Error"
+	ChksumError     = "Checksum_Error"
+	KRBMsgError     = "KRBMessage_Handling_Error"
+	ConfigError     = "Configuration_Error"
+	KDCError        = "KDC_Error"
+)
+
+// Krberror is an error type for gokrb5
+type Krberror struct {
+	RootCause string
+	EText     []string
+}
+
+// Error function to implement the error interface.
+func (e Krberror) Error() string {
+	return fmt.Sprintf("[Root cause: %s] ", e.RootCause) + strings.Join(e.EText, separator)
+}
+
+// Add another error statement to the error.
+func (e *Krberror) Add(et string, s string) {
+	e.EText = append([]string{fmt.Sprintf("%s: %s", et, s)}, e.EText...)
+}
+
+// New creates a new instance of Krberror.
+func New(et, s string) Krberror {
+	return Krberror{
+		RootCause: et,
+		EText:     []string{s},
+	}
+}
+
+// Errorf appends to or creates a new Krberror.
+func Errorf(err error, et, format string, a ...interface{}) Krberror {
+	if e, ok := err.(Krberror); ok {
+		e.Add(et, fmt.Sprintf(format, a...))
+		return e
+	}
+	return NewErrorf(et, format+": %s", append(a, err)...)
+}
+
+// NewErrorf creates a new Krberror from a formatted string.
+func NewErrorf(et, format string, a ...interface{}) Krberror {
+	var s string
+	if len(a) > 0 {
+		s = fmt.Sprintf("%s: %s", et, fmt.Sprintf(format, a...))
+	} else {
+		s = fmt.Sprintf("%s: %s", et, format)
+	}
+	return Krberror{
+		RootCause: et,
+		EText:     []string{s},
+	}
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go
new file mode 100644
index 0000000..555fb80
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go
@@ -0,0 +1,49 @@
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// APRep implements RFC 4120 KRB_AP_REP: https://tools.ietf.org/html/rfc4120#section-5.5.2.
+type APRep struct {
+	PVNO    int                 `asn1:"explicit,tag:0"`
+	MsgType int                 `asn1:"explicit,tag:1"`
+	EncPart types.EncryptedData `asn1:"explicit,tag:2"`
+}
+
+// EncAPRepPart is the encrypted part of KRB_AP_REP.
+type EncAPRepPart struct {
+	CTime          time.Time           `asn1:"generalized,explicit,tag:0"`
+	Cusec          int                 `asn1:"explicit,tag:1"`
+	Subkey         types.EncryptionKey `asn1:"optional,explicit,tag:2"`
+	SequenceNumber int64               `asn1:"optional,explicit,tag:3"`
+}
+
+// Unmarshal bytes b into the APRep struct.
+func (a *APRep) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREP))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	expectedMsgType := msgtype.KRB_AP_REP
+	if a.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_AP_REP. Expected: %v; Actual: %v", expectedMsgType, a.MsgType)
+	}
+	return nil
+}
+
+// Unmarshal bytes b into the APRep encrypted part struct.
+func (a *EncAPRepPart) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncAPRepPart))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "AP_REP unmarshal error")
+	}
+	return nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go
new file mode 100644
index 0000000..1836079
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go
@@ -0,0 +1,199 @@
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/keytab"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+type marshalAPReq struct {
+	PVNO      int            `asn1:"explicit,tag:0"`
+	MsgType   int            `asn1:"explicit,tag:1"`
+	APOptions asn1.BitString `asn1:"explicit,tag:2"`
+	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
+	Ticket                 asn1.RawValue       `asn1:"explicit,tag:3"`
+	EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"`
+}
+
+// APReq implements RFC 4120 KRB_AP_REQ: https://tools.ietf.org/html/rfc4120#section-5.5.1.
+type APReq struct {
+	PVNO                   int                 `asn1:"explicit,tag:0"`
+	MsgType                int                 `asn1:"explicit,tag:1"`
+	APOptions              asn1.BitString      `asn1:"explicit,tag:2"`
+	Ticket                 Ticket              `asn1:"explicit,tag:3"`
+	EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"`
+	Authenticator          types.Authenticator `asn1:"optional"`
+}
+
+// NewAPReq generates a new KRB_AP_REQ struct.
+func NewAPReq(tkt Ticket, sessionKey types.EncryptionKey, auth types.Authenticator) (APReq, error) {
+	var a APReq
+	ed, err := encryptAuthenticator(auth, sessionKey, tkt)
+	if err != nil {
+		return a, krberror.Errorf(err, krberror.KRBMsgError, "error creating Authenticator for AP_REQ")
+	}
+	a = APReq{
+		PVNO:                   iana.PVNO,
+		MsgType:                msgtype.KRB_AP_REQ,
+		APOptions:              types.NewKrbFlags(),
+		Ticket:                 tkt,
+		EncryptedAuthenticator: ed,
+	}
+	return a, nil
+}
+
+// Encrypt Authenticator
+func encryptAuthenticator(a types.Authenticator, sessionKey types.EncryptionKey, tkt Ticket) (types.EncryptedData, error) {
+	var ed types.EncryptedData
+	m, err := a.Marshal()
+	if err != nil {
+		return ed, krberror.Errorf(err, krberror.EncodingError, "marshaling error of EncryptedData form of Authenticator")
+	}
+	usage := authenticatorKeyUsage(tkt.SName)
+	ed, err = crypto.GetEncryptedData(m, sessionKey, uint32(usage), tkt.EncPart.KVNO)
+	if err != nil {
+		return ed, krberror.Errorf(err, krberror.EncryptingError, "error encrypting Authenticator")
+	}
+	return ed, nil
+}
+
+// DecryptAuthenticator decrypts the Authenticator within the AP_REQ.
+// sessionKey may simply be the key within the decrypted EncPart of the ticket within the AP_REQ.
+func (a *APReq) DecryptAuthenticator(sessionKey types.EncryptionKey) error {
+	usage := authenticatorKeyUsage(a.Ticket.SName)
+	ab, e := crypto.DecryptEncPart(a.EncryptedAuthenticator, sessionKey, uint32(usage))
+	if e != nil {
+		return fmt.Errorf("error decrypting authenticator: %v", e)
+	}
+	err := a.Authenticator.Unmarshal(ab)
+	if err != nil {
+		return fmt.Errorf("error unmarshaling authenticator: %v", err)
+	}
+	return nil
+}
+
+func authenticatorKeyUsage(pn types.PrincipalName) int {
+	if pn.NameString[0] == "krbtgt" {
+		return keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR
+	}
+	return keyusage.AP_REQ_AUTHENTICATOR
+}
+
+// Unmarshal bytes b into the APReq struct.
+func (a *APReq) Unmarshal(b []byte) error {
+	var m marshalAPReq
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREQ))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "unmarshal error of AP_REQ")
+	}
+	if m.MsgType != msgtype.KRB_AP_REQ {
+		return NewKRBError(types.PrincipalName{}, "", errorcode.KRB_AP_ERR_MSG_TYPE, errorcode.Lookup(errorcode.KRB_AP_ERR_MSG_TYPE))
+	}
+	a.PVNO = m.PVNO
+	a.MsgType = m.MsgType
+	a.APOptions = m.APOptions
+	a.EncryptedAuthenticator = m.EncryptedAuthenticator
+	a.Ticket, err = unmarshalTicket(m.Ticket.Bytes)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "unmarshaling error of Ticket within AP_REQ")
+	}
+	return nil
+}
+
+// Marshal APReq struct.
+func (a *APReq) Marshal() ([]byte, error) {
+	m := marshalAPReq{
+		PVNO:                   a.PVNO,
+		MsgType:                a.MsgType,
+		APOptions:              a.APOptions,
+		EncryptedAuthenticator: a.EncryptedAuthenticator,
+	}
+	var b []byte
+	b, err := a.Ticket.Marshal()
+	if err != nil {
+		return b, err
+	}
+	m.Ticket = asn1.RawValue{
+		Class:      asn1.ClassContextSpecific,
+		IsCompound: true,
+		Tag:        3,
+		Bytes:      b,
+	}
+	mk, err := asn1.Marshal(m)
+	if err != nil {
+		return mk, krberror.Errorf(err, krberror.EncodingError, "marshaling error of AP_REQ")
+	}
+	mk = asn1tools.AddASNAppTag(mk, asnAppTag.APREQ)
+	return mk, nil
+}
+
+// Verify an AP_REQ using service's keytab, spn and max acceptable clock skew duration.
+// The service ticket encrypted part and authenticator will be decrypted as part of this operation.
+func (a *APReq) Verify(kt *keytab.Keytab, d time.Duration, cAddr types.HostAddress, snameOverride *types.PrincipalName) (bool, error) {
+	// Decrypt ticket's encrypted part with service key
+	//TODO decrypt with service's session key from its TGT is use-to-user. Need to figure out how to get TGT.
+	//if types.IsFlagSet(&a.APOptions, flags.APOptionUseSessionKey) {
+	//	err := a.Ticket.Decrypt(tgt.DecryptedEncPart.Key)
+	//	if err != nil {
+	//		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of ticket provided using session key")
+	//	}
+	//} else {
+	//	err := a.Ticket.DecryptEncPart(*kt, &a.Ticket.SName)
+	//	if err != nil {
+	//		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided")
+	//	}
+	//}
+	sname := &a.Ticket.SName
+	if snameOverride != nil {
+		sname = snameOverride
+	}
+	err := a.Ticket.DecryptEncPart(kt, sname)
+	if err != nil {
+		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided")
+	}
+
+	// Check time validity of ticket
+	ok, err := a.Ticket.Valid(d)
+	if err != nil || !ok {
+		return ok, err
+	}
+
+	// Check client's address is listed in the client addresses in the ticket
+	if len(a.Ticket.DecryptedEncPart.CAddr) > 0 {
+		//If client addresses are present check if any of them match the source IP that sent the APReq
+		//If there is no match return KRB_AP_ERR_BADADDR error.
+		if !types.HostAddressesContains(a.Ticket.DecryptedEncPart.CAddr, cAddr) {
+			return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADADDR, "client address not within the list contained in the service ticket")
+		}
+	}
+
+	// Decrypt authenticator with session key from ticket's encrypted part
+	err = a.DecryptAuthenticator(a.Ticket.DecryptedEncPart.Key)
+	if err != nil {
+		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BAD_INTEGRITY, "could not decrypt authenticator")
+	}
+
+	// Check CName in authenticator is the same as that in the ticket
+	if !a.Authenticator.CName.Equal(a.Ticket.DecryptedEncPart.CName) {
+		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADMATCH, "CName in Authenticator does not match that in service ticket")
+	}
+
+	// Check the clock skew between the client and the service server
+	ct := a.Authenticator.CTime.Add(time.Duration(a.Authenticator.Cusec) * time.Microsecond)
+	t := time.Now().UTC()
+	if t.Sub(ct) > d || ct.Sub(t) > d {
+		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_SKEW, fmt.Sprintf("clock skew with client too large. greater than %v seconds", d))
+	}
+	return true, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go
new file mode 100644
index 0000000..69df9f0
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go
@@ -0,0 +1,360 @@
+package messages
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.4.2
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/config"
+	"github.com/jcmturner/gokrb5/v8/credentials"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/flags"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/iana/patype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+type marshalKDCRep struct {
+	PVNO    int                  `asn1:"explicit,tag:0"`
+	MsgType int                  `asn1:"explicit,tag:1"`
+	PAData  types.PADataSequence `asn1:"explicit,optional,tag:2"`
+	CRealm  string               `asn1:"generalstring,explicit,tag:3"`
+	CName   types.PrincipalName  `asn1:"explicit,tag:4"`
+	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
+	Ticket  asn1.RawValue       `asn1:"explicit,tag:5"`
+	EncPart types.EncryptedData `asn1:"explicit,tag:6"`
+}
+
+// KDCRepFields represents the KRB_KDC_REP fields.
+type KDCRepFields struct {
+	PVNO             int
+	MsgType          int
+	PAData           []types.PAData
+	CRealm           string
+	CName            types.PrincipalName
+	Ticket           Ticket
+	EncPart          types.EncryptedData
+	DecryptedEncPart EncKDCRepPart
+}
+
+// ASRep implements RFC 4120 KRB_AS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2.
+type ASRep struct {
+	KDCRepFields
+}
+
+// TGSRep implements RFC 4120 KRB_TGS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2.
+type TGSRep struct {
+	KDCRepFields
+}
+
+// EncKDCRepPart is the encrypted part of KRB_KDC_REP.
+type EncKDCRepPart struct {
+	Key           types.EncryptionKey  `asn1:"explicit,tag:0"`
+	LastReqs      []LastReq            `asn1:"explicit,tag:1"`
+	Nonce         int                  `asn1:"explicit,tag:2"`
+	KeyExpiration time.Time            `asn1:"generalized,explicit,optional,tag:3"`
+	Flags         asn1.BitString       `asn1:"explicit,tag:4"`
+	AuthTime      time.Time            `asn1:"generalized,explicit,tag:5"`
+	StartTime     time.Time            `asn1:"generalized,explicit,optional,tag:6"`
+	EndTime       time.Time            `asn1:"generalized,explicit,tag:7"`
+	RenewTill     time.Time            `asn1:"generalized,explicit,optional,tag:8"`
+	SRealm        string               `asn1:"generalstring,explicit,tag:9"`
+	SName         types.PrincipalName  `asn1:"explicit,tag:10"`
+	CAddr         []types.HostAddress  `asn1:"explicit,optional,tag:11"`
+	EncPAData     types.PADataSequence `asn1:"explicit,optional,tag:12"`
+}
+
+// LastReq part of KRB_KDC_REP.
+type LastReq struct {
+	LRType  int32     `asn1:"explicit,tag:0"`
+	LRValue time.Time `asn1:"generalized,explicit,tag:1"`
+}
+
+// Unmarshal bytes b into the ASRep struct.
+func (k *ASRep) Unmarshal(b []byte) error {
+	var m marshalKDCRep
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREP))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	if m.MsgType != msgtype.KRB_AS_REP {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an AS_REP. Expected: %v; Actual: %v", msgtype.KRB_AS_REP, m.MsgType)
+	}
+	//Process the raw ticket within
+	tkt, err := unmarshalTicket(m.Ticket.Bytes)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within AS_REP")
+	}
+	k.KDCRepFields = KDCRepFields{
+		PVNO:    m.PVNO,
+		MsgType: m.MsgType,
+		PAData:  m.PAData,
+		CRealm:  m.CRealm,
+		CName:   m.CName,
+		Ticket:  tkt,
+		EncPart: m.EncPart,
+	}
+	return nil
+}
+
+// Marshal ASRep struct.
+func (k *ASRep) Marshal() ([]byte, error) {
+	m := marshalKDCRep{
+		PVNO:    k.PVNO,
+		MsgType: k.MsgType,
+		PAData:  k.PAData,
+		CRealm:  k.CRealm,
+		CName:   k.CName,
+		EncPart: k.EncPart,
+	}
+	b, err := k.Ticket.Marshal()
+	if err != nil {
+		return []byte{}, err
+	}
+	m.Ticket = asn1.RawValue{
+		Class:      asn1.ClassContextSpecific,
+		IsCompound: true,
+		Tag:        5,
+		Bytes:      b,
+	}
+	mk, err := asn1.Marshal(m)
+	if err != nil {
+		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REP")
+	}
+	mk = asn1tools.AddASNAppTag(mk, asnAppTag.ASREP)
+	return mk, nil
+}
+
+// Unmarshal bytes b into the TGSRep struct.
+func (k *TGSRep) Unmarshal(b []byte) error {
+	var m marshalKDCRep
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREP))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	if m.MsgType != msgtype.KRB_TGS_REP {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an TGS_REP. Expected: %v; Actual: %v", msgtype.KRB_TGS_REP, m.MsgType)
+	}
+	//Process the raw ticket within
+	tkt, err := unmarshalTicket(m.Ticket.Bytes)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within TGS_REP")
+	}
+	k.KDCRepFields = KDCRepFields{
+		PVNO:    m.PVNO,
+		MsgType: m.MsgType,
+		PAData:  m.PAData,
+		CRealm:  m.CRealm,
+		CName:   m.CName,
+		Ticket:  tkt,
+		EncPart: m.EncPart,
+	}
+	return nil
+}
+
+// Marshal TGSRep struct.
+func (k *TGSRep) Marshal() ([]byte, error) {
+	m := marshalKDCRep{
+		PVNO:    k.PVNO,
+		MsgType: k.MsgType,
+		PAData:  k.PAData,
+		CRealm:  k.CRealm,
+		CName:   k.CName,
+		EncPart: k.EncPart,
+	}
+	b, err := k.Ticket.Marshal()
+	if err != nil {
+		return []byte{}, err
+	}
+	m.Ticket = asn1.RawValue{
+		Class:      asn1.ClassContextSpecific,
+		IsCompound: true,
+		Tag:        5,
+		Bytes:      b,
+	}
+	mk, err := asn1.Marshal(m)
+	if err != nil {
+		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling TGS_REP")
+	}
+	mk = asn1tools.AddASNAppTag(mk, asnAppTag.TGSREP)
+	return mk, nil
+}
+
+// Unmarshal bytes b into encrypted part of KRB_KDC_REP.
+func (e *EncKDCRepPart) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncASRepPart))
+	if err != nil {
+		// Try using tag 26
+		// Ref: RFC 4120 - mentions that some implementations use application tag number 26 wether or not the reply is
+		// a AS-REP or a TGS-REP.
+		_, err = asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncTGSRepPart))
+		if err != nil {
+			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part within KDC_REP")
+		}
+	}
+	return nil
+}
+
+// Marshal encrypted part of KRB_KDC_REP.
+func (e *EncKDCRepPart) Marshal() ([]byte, error) {
+	b, err := asn1.Marshal(*e)
+	if err != nil {
+		return b, krberror.Errorf(err, krberror.EncodingError, "marshaling error of AS_REP encpart")
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.EncASRepPart)
+	return b, nil
+}
+
+// DecryptEncPart decrypts the encrypted part of an AS_REP.
+func (k *ASRep) DecryptEncPart(c *credentials.Credentials) (types.EncryptionKey, error) {
+	var key types.EncryptionKey
+	var err error
+	if c.HasKeytab() {
+		key, _, err = c.Keytab().GetEncryptionKey(k.CName, k.CRealm, k.EncPart.KVNO, k.EncPart.EType)
+		if err != nil {
+			return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
+		}
+	}
+	if c.HasPassword() {
+		key, _, err = crypto.GetKeyFromPassword(c.Password(), k.CName, k.CRealm, k.EncPart.EType, k.PAData)
+		if err != nil {
+			return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
+		}
+	}
+	if !c.HasKeytab() && !c.HasPassword() {
+		return key, krberror.NewErrorf(krberror.DecryptingError, "no secret available in credentials to perform decryption of AS_REP encrypted part")
+	}
+	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.AS_REP_ENCPART)
+	if err != nil {
+		return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
+	}
+	var denc EncKDCRepPart
+	err = denc.Unmarshal(b)
+	if err != nil {
+		return key, krberror.Errorf(err, krberror.EncodingError, "error unmarshaling decrypted encpart of AS_REP")
+	}
+	k.DecryptedEncPart = denc
+	return key, nil
+}
+
+// Verify checks the validity of AS_REP message.
+func (k *ASRep) Verify(cfg *config.Config, creds *credentials.Credentials, asReq ASReq) (bool, error) {
+	//Ref RFC 4120 Section 3.1.5
+	if !k.CName.Equal(asReq.ReqBody.CName) {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", asReq.ReqBody.CName, k.CName)
+	}
+	if k.CRealm != asReq.ReqBody.Realm {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "CRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.CRealm)
+	}
+	key, err := k.DecryptEncPart(creds)
+	if err != nil {
+		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting EncPart of AS_REP")
+	}
+	if k.DecryptedEncPart.Nonce != asReq.ReqBody.Nonce {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request")
+	}
+	if !k.DecryptedEncPart.SName.Equal(asReq.ReqBody.SName) {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", asReq.ReqBody.SName, k.DecryptedEncPart.SName)
+	}
+	if k.DecryptedEncPart.SRealm != asReq.ReqBody.Realm {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.DecryptedEncPart.SRealm)
+	}
+	if len(asReq.ReqBody.Addresses) > 0 {
+		if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, asReq.ReqBody.Addresses) {
+			return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the AS_REP does not match those listed in the AS_REQ")
+		}
+	}
+	t := time.Now().UTC()
+	if t.Sub(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(t) > cfg.LibDefaults.Clockskew {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds", cfg.LibDefaults.Clockskew.Seconds())
+	}
+	// RFC 6806 https://tools.ietf.org/html/rfc6806.html#section-11
+	if asReq.PAData.Contains(patype.PA_REQ_ENC_PA_REP) && types.IsFlagSet(&k.DecryptedEncPart.Flags, flags.EncPARep) {
+		if len(k.DecryptedEncPart.EncPAData) < 2 || !k.DecryptedEncPart.EncPAData.Contains(patype.PA_FX_FAST) {
+			return false, krberror.NewErrorf(krberror.KRBMsgError, "KDC did not respond appropriately to FAST negotiation")
+		}
+		for _, pa := range k.DecryptedEncPart.EncPAData {
+			if pa.PADataType == patype.PA_REQ_ENC_PA_REP {
+				var pafast types.PAReqEncPARep
+				err := pafast.Unmarshal(pa.PADataValue)
+				if err != nil {
+					return false, krberror.Errorf(err, krberror.EncodingError, "KDC FAST negotiation response error, could not unmarshal PA_REQ_ENC_PA_REP")
+				}
+				etype, err := crypto.GetChksumEtype(pafast.ChksumType)
+				if err != nil {
+					return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response error")
+				}
+				ab, _ := asReq.Marshal()
+				if !etype.VerifyChecksum(key.KeyValue, ab, pafast.Chksum, keyusage.KEY_USAGE_AS_REQ) {
+					return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response checksum invalid")
+				}
+			}
+		}
+	}
+	return true, nil
+}
+
+// DecryptEncPart decrypts the encrypted part of an TGS_REP.
+func (k *TGSRep) DecryptEncPart(key types.EncryptionKey) error {
+	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.TGS_REP_ENCPART_SESSION_KEY)
+	if err != nil {
+		return krberror.Errorf(err, krberror.DecryptingError, "error decrypting TGS_REP EncPart")
+	}
+	var denc EncKDCRepPart
+	err = denc.Unmarshal(b)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part")
+	}
+	k.DecryptedEncPart = denc
+	return nil
+}
+
+// Verify checks the validity of the TGS_REP message.
+func (k *TGSRep) Verify(cfg *config.Config, tgsReq TGSReq) (bool, error) {
+	if !k.CName.Equal(tgsReq.ReqBody.CName) {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.CName, k.CName)
+	}
+	if k.Ticket.Realm != tgsReq.ReqBody.Realm {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "realm in response ticket does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.Ticket.Realm)
+	}
+	if k.DecryptedEncPart.Nonce != tgsReq.ReqBody.Nonce {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request")
+	}
+	//if k.Ticket.SName.NameType != tgsReq.ReqBody.SName.NameType || k.Ticket.SName.NameString == nil {
+	//	return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.Ticket.SName)
+	//}
+	//for i := range k.Ticket.SName.NameString {
+	//	if k.Ticket.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] {
+	//		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.Ticket.SName)
+	//	}
+	//}
+	//if k.DecryptedEncPart.SName.NameType != tgsReq.ReqBody.SName.NameType || k.DecryptedEncPart.SName.NameString == nil {
+	//	return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName)
+	//}
+	//for i := range k.DecryptedEncPart.SName.NameString {
+	//	if k.DecryptedEncPart.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] {
+	//		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName)
+	//	}
+	//}
+	if k.DecryptedEncPart.SRealm != tgsReq.ReqBody.Realm {
+		return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.DecryptedEncPart.SRealm)
+	}
+	if len(k.DecryptedEncPart.CAddr) > 0 {
+		if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, tgsReq.ReqBody.Addresses) {
+			return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the TGS_REP does not match those listed in the TGS_REQ")
+		}
+	}
+	if time.Since(k.DecryptedEncPart.StartTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.StartTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew {
+		if time.Since(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew {
+			return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds.", cfg.LibDefaults.Clockskew.Seconds())
+		}
+	}
+	return true, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go
new file mode 100644
index 0000000..3745afe
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go
@@ -0,0 +1,432 @@
+package messages
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.4.1
+
+import (
+	"crypto/rand"
+	"fmt"
+	"math"
+	"math/big"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/config"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/flags"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+	"github.com/jcmturner/gokrb5/v8/iana/patype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+type marshalKDCReq struct {
+	PVNO    int                  `asn1:"explicit,tag:1"`
+	MsgType int                  `asn1:"explicit,tag:2"`
+	PAData  types.PADataSequence `asn1:"explicit,optional,tag:3"`
+	ReqBody asn1.RawValue        `asn1:"explicit,tag:4"`
+}
+
+// KDCReqFields represents the KRB_KDC_REQ fields.
+type KDCReqFields struct {
+	PVNO    int
+	MsgType int
+	PAData  types.PADataSequence
+	ReqBody KDCReqBody
+	Renewal bool
+}
+
+// ASReq implements RFC 4120 KRB_AS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1.
+type ASReq struct {
+	KDCReqFields
+}
+
+// TGSReq implements RFC 4120 KRB_TGS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1.
+type TGSReq struct {
+	KDCReqFields
+}
+
+type marshalKDCReqBody struct {
+	KDCOptions  asn1.BitString      `asn1:"explicit,tag:0"`
+	CName       types.PrincipalName `asn1:"explicit,optional,tag:1"`
+	Realm       string              `asn1:"generalstring,explicit,tag:2"`
+	SName       types.PrincipalName `asn1:"explicit,optional,tag:3"`
+	From        time.Time           `asn1:"generalized,explicit,optional,tag:4"`
+	Till        time.Time           `asn1:"generalized,explicit,tag:5"`
+	RTime       time.Time           `asn1:"generalized,explicit,optional,tag:6"`
+	Nonce       int                 `asn1:"explicit,tag:7"`
+	EType       []int32             `asn1:"explicit,tag:8"`
+	Addresses   []types.HostAddress `asn1:"explicit,optional,tag:9"`
+	EncAuthData types.EncryptedData `asn1:"explicit,optional,tag:10"`
+	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
+	AdditionalTickets asn1.RawValue `asn1:"explicit,optional,tag:11"`
+}
+
+// KDCReqBody implements the KRB_KDC_REQ request body.
+type KDCReqBody struct {
+	KDCOptions        asn1.BitString      `asn1:"explicit,tag:0"`
+	CName             types.PrincipalName `asn1:"explicit,optional,tag:1"`
+	Realm             string              `asn1:"generalstring,explicit,tag:2"`
+	SName             types.PrincipalName `asn1:"explicit,optional,tag:3"`
+	From              time.Time           `asn1:"generalized,explicit,optional,tag:4"`
+	Till              time.Time           `asn1:"generalized,explicit,tag:5"`
+	RTime             time.Time           `asn1:"generalized,explicit,optional,tag:6"`
+	Nonce             int                 `asn1:"explicit,tag:7"`
+	EType             []int32             `asn1:"explicit,tag:8"`
+	Addresses         []types.HostAddress `asn1:"explicit,optional,tag:9"`
+	EncAuthData       types.EncryptedData `asn1:"explicit,optional,tag:10"`
+	AdditionalTickets []Ticket            `asn1:"explicit,optional,tag:11"`
+}
+
+// NewASReqForTGT generates a new KRB_AS_REQ struct for a TGT request.
+func NewASReqForTGT(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) {
+	sname := types.PrincipalName{
+		NameType:   nametype.KRB_NT_SRV_INST,
+		NameString: []string{"krbtgt", realm},
+	}
+	return NewASReq(realm, c, cname, sname)
+}
+
+// NewASReqForChgPasswd generates a new KRB_AS_REQ struct for a change password request.
+func NewASReqForChgPasswd(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) {
+	sname := types.PrincipalName{
+		NameType:   nametype.KRB_NT_PRINCIPAL,
+		NameString: []string{"kadmin", "changepw"},
+	}
+	return NewASReq(realm, c, cname, sname)
+}
+
+// NewASReq generates a new KRB_AS_REQ struct for a given SNAME.
+func NewASReq(realm string, c *config.Config, cname, sname types.PrincipalName) (ASReq, error) {
+	nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32))
+	if err != nil {
+		return ASReq{}, err
+	}
+	t := time.Now().UTC()
+	// Copy the default options to make this thread safe
+	kopts := types.NewKrbFlags()
+	copy(kopts.Bytes, c.LibDefaults.KDCDefaultOptions.Bytes)
+	kopts.BitLength = c.LibDefaults.KDCDefaultOptions.BitLength
+	a := ASReq{
+		KDCReqFields{
+			PVNO:    iana.PVNO,
+			MsgType: msgtype.KRB_AS_REQ,
+			PAData:  types.PADataSequence{},
+			ReqBody: KDCReqBody{
+				KDCOptions: kopts,
+				Realm:      realm,
+				CName:      cname,
+				SName:      sname,
+				Till:       t.Add(c.LibDefaults.TicketLifetime),
+				Nonce:      int(nonce.Int64()),
+				EType:      c.LibDefaults.DefaultTktEnctypeIDs,
+			},
+		},
+	}
+	if c.LibDefaults.Forwardable {
+		types.SetFlag(&a.ReqBody.KDCOptions, flags.Forwardable)
+	}
+	if c.LibDefaults.Canonicalize {
+		types.SetFlag(&a.ReqBody.KDCOptions, flags.Canonicalize)
+	}
+	if c.LibDefaults.Proxiable {
+		types.SetFlag(&a.ReqBody.KDCOptions, flags.Proxiable)
+	}
+	if c.LibDefaults.RenewLifetime != 0 {
+		types.SetFlag(&a.ReqBody.KDCOptions, flags.Renewable)
+		a.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime)
+		a.ReqBody.RTime = t.Add(time.Duration(48) * time.Hour)
+	}
+	if !c.LibDefaults.NoAddresses {
+		ha, err := types.LocalHostAddresses()
+		if err != nil {
+			return a, fmt.Errorf("could not get local addresses: %v", err)
+		}
+		ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...)
+		a.ReqBody.Addresses = ha
+	}
+	return a, nil
+}
+
+// NewTGSReq generates a new KRB_TGS_REQ struct.
+func NewTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, tgt Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool) (TGSReq, error) {
+	a, err := tgsReq(cname, sname, kdcRealm, renewal, c)
+	if err != nil {
+		return a, err
+	}
+	err = a.setPAData(tgt, sessionKey)
+	return a, err
+}
+
+// NewUser2UserTGSReq returns a TGS-REQ suitable for user-to-user authentication (https://tools.ietf.org/html/rfc4120#section-3.7)
+func NewUser2UserTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, clientTGT Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool, verifyingTGT Ticket) (TGSReq, error) {
+	a, err := tgsReq(cname, sname, kdcRealm, renewal, c)
+	if err != nil {
+		return a, err
+	}
+	a.ReqBody.AdditionalTickets = []Ticket{verifyingTGT}
+	types.SetFlag(&a.ReqBody.KDCOptions, flags.EncTktInSkey)
+	err = a.setPAData(clientTGT, sessionKey)
+	return a, err
+}
+
+// tgsReq populates the fields for a TGS_REQ
+func tgsReq(cname, sname types.PrincipalName, kdcRealm string, renewal bool, c *config.Config) (TGSReq, error) {
+	nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32))
+	if err != nil {
+		return TGSReq{}, err
+	}
+	t := time.Now().UTC()
+	k := KDCReqFields{
+		PVNO:    iana.PVNO,
+		MsgType: msgtype.KRB_TGS_REQ,
+		ReqBody: KDCReqBody{
+			KDCOptions: types.NewKrbFlags(),
+			Realm:      kdcRealm,
+			CName:      cname, // Add the CName to make validation of the reply easier
+			SName:      sname,
+			Till:       t.Add(c.LibDefaults.TicketLifetime),
+			Nonce:      int(nonce.Int64()),
+			EType:      c.LibDefaults.DefaultTGSEnctypeIDs,
+		},
+		Renewal: renewal,
+	}
+	if c.LibDefaults.Forwardable {
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Forwardable)
+	}
+	if c.LibDefaults.Canonicalize {
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Canonicalize)
+	}
+	if c.LibDefaults.Proxiable {
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Proxiable)
+	}
+	if c.LibDefaults.RenewLifetime > time.Duration(0) {
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable)
+		k.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime)
+	}
+	if !c.LibDefaults.NoAddresses {
+		ha, err := types.LocalHostAddresses()
+		if err != nil {
+			return TGSReq{}, fmt.Errorf("could not get local addresses: %v", err)
+		}
+		ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...)
+		k.ReqBody.Addresses = ha
+	}
+	if renewal {
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renew)
+		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable)
+	}
+	return TGSReq{
+		k,
+	}, nil
+}
+
+func (k *TGSReq) setPAData(tgt Ticket, sessionKey types.EncryptionKey) error {
+	// Marshal the request and calculate checksum
+	b, err := k.ReqBody.Marshal()
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error marshaling TGS_REQ body")
+	}
+	etype, err := crypto.GetEtype(sessionKey.KeyType)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncryptingError, "error getting etype to encrypt authenticator")
+	}
+	cb, err := etype.GetChecksumHash(sessionKey.KeyValue, b, keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR_CHKSUM)
+	if err != nil {
+		return krberror.Errorf(err, krberror.ChksumError, "error getting etype checksum hash")
+	}
+
+	// Form PAData for TGS_REQ
+	// Create authenticator
+	auth, err := types.NewAuthenticator(tgt.Realm, k.ReqBody.CName)
+	if err != nil {
+		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator")
+	}
+	auth.Cksum = types.Checksum{
+		CksumType: etype.GetHashID(),
+		Checksum:  cb,
+	}
+	// Create AP_REQ
+	apReq, err := NewAPReq(tgt, sessionKey, auth)
+	if err != nil {
+		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AP_REQ")
+	}
+	apb, err := apReq.Marshal()
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error marshaling AP_REQ for pre-authentication data")
+	}
+	k.PAData = types.PADataSequence{
+		types.PAData{
+			PADataType:  patype.PA_TGS_REQ,
+			PADataValue: apb,
+		},
+	}
+	return nil
+}
+
+// Unmarshal bytes b into the ASReq struct.
+func (k *ASReq) Unmarshal(b []byte) error {
+	var m marshalKDCReq
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREQ))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling AS_REQ")
+	}
+	expectedMsgType := msgtype.KRB_AS_REQ
+	if m.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a AS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
+	}
+	var reqb KDCReqBody
+	err = reqb.Unmarshal(m.ReqBody.Bytes)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error processing AS_REQ body")
+	}
+	k.MsgType = m.MsgType
+	k.PAData = m.PAData
+	k.PVNO = m.PVNO
+	k.ReqBody = reqb
+	return nil
+}
+
+// Unmarshal bytes b into the TGSReq struct.
+func (k *TGSReq) Unmarshal(b []byte) error {
+	var m marshalKDCReq
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREQ))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling TGS_REQ")
+	}
+	expectedMsgType := msgtype.KRB_TGS_REQ
+	if m.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a TGS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
+	}
+	var reqb KDCReqBody
+	err = reqb.Unmarshal(m.ReqBody.Bytes)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error processing TGS_REQ body")
+	}
+	k.MsgType = m.MsgType
+	k.PAData = m.PAData
+	k.PVNO = m.PVNO
+	k.ReqBody = reqb
+	return nil
+}
+
+// Unmarshal bytes b into the KRB_KDC_REQ body struct.
+func (k *KDCReqBody) Unmarshal(b []byte) error {
+	var m marshalKDCReqBody
+	_, err := asn1.Unmarshal(b, &m)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling KDC_REQ body")
+	}
+	k.KDCOptions = m.KDCOptions
+	if len(k.KDCOptions.Bytes) < 4 {
+		tb := make([]byte, 4-len(k.KDCOptions.Bytes))
+		k.KDCOptions.Bytes = append(tb, k.KDCOptions.Bytes...)
+		k.KDCOptions.BitLength = len(k.KDCOptions.Bytes) * 8
+	}
+	k.CName = m.CName
+	k.Realm = m.Realm
+	k.SName = m.SName
+	k.From = m.From
+	k.Till = m.Till
+	k.RTime = m.RTime
+	k.Nonce = m.Nonce
+	k.EType = m.EType
+	k.Addresses = m.Addresses
+	k.EncAuthData = m.EncAuthData
+	if len(m.AdditionalTickets.Bytes) > 0 {
+		k.AdditionalTickets, err = unmarshalTicketsSequence(m.AdditionalTickets)
+		if err != nil {
+			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling additional tickets")
+		}
+	}
+	return nil
+}
+
+// Marshal ASReq struct.
+func (k *ASReq) Marshal() ([]byte, error) {
+	m := marshalKDCReq{
+		PVNO:    k.PVNO,
+		MsgType: k.MsgType,
+		PAData:  k.PAData,
+	}
+	b, err := k.ReqBody.Marshal()
+	if err != nil {
+		var mk []byte
+		return mk, err
+	}
+	m.ReqBody = asn1.RawValue{
+		Class:      asn1.ClassContextSpecific,
+		IsCompound: true,
+		Tag:        4,
+		Bytes:      b,
+	}
+	mk, err := asn1.Marshal(m)
+	if err != nil {
+		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ")
+	}
+	mk = asn1tools.AddASNAppTag(mk, asnAppTag.ASREQ)
+	return mk, nil
+}
+
+// Marshal TGSReq struct.
+func (k *TGSReq) Marshal() ([]byte, error) {
+	m := marshalKDCReq{
+		PVNO:    k.PVNO,
+		MsgType: k.MsgType,
+		PAData:  k.PAData,
+	}
+	b, err := k.ReqBody.Marshal()
+	if err != nil {
+		var mk []byte
+		return mk, err
+	}
+	m.ReqBody = asn1.RawValue{
+		Class:      asn1.ClassContextSpecific,
+		IsCompound: true,
+		Tag:        4,
+		Bytes:      b,
+	}
+	mk, err := asn1.Marshal(m)
+	if err != nil {
+		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ")
+	}
+	mk = asn1tools.AddASNAppTag(mk, asnAppTag.TGSREQ)
+	return mk, nil
+}
+
+// Marshal KRB_KDC_REQ body struct.
+func (k *KDCReqBody) Marshal() ([]byte, error) {
+	var b []byte
+	m := marshalKDCReqBody{
+		KDCOptions:  k.KDCOptions,
+		CName:       k.CName,
+		Realm:       k.Realm,
+		SName:       k.SName,
+		From:        k.From,
+		Till:        k.Till,
+		RTime:       k.RTime,
+		Nonce:       k.Nonce,
+		EType:       k.EType,
+		Addresses:   k.Addresses,
+		EncAuthData: k.EncAuthData,
+	}
+	rawtkts, err := MarshalTicketSequence(k.AdditionalTickets)
+	if err != nil {
+		return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body additional tickets")
+	}
+	//The asn1.rawValue needs the tag setting on it for where it is in the KDCReqBody
+	rawtkts.Tag = 11
+	if len(rawtkts.Bytes) > 0 {
+		m.AdditionalTickets = rawtkts
+	}
+	b, err = asn1.Marshal(m)
+	if err != nil {
+		return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body")
+	}
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go
new file mode 100644
index 0000000..536fdb9
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go
@@ -0,0 +1,102 @@
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+type marshalKRBCred struct {
+	PVNO    int                 `asn1:"explicit,tag:0"`
+	MsgType int                 `asn1:"explicit,tag:1"`
+	Tickets asn1.RawValue       `asn1:"explicit,tag:2"`
+	EncPart types.EncryptedData `asn1:"explicit,tag:3"`
+}
+
+// KRBCred implements RFC 4120 KRB_CRED: https://tools.ietf.org/html/rfc4120#section-5.8.1.
+type KRBCred struct {
+	PVNO             int
+	MsgType          int
+	Tickets          []Ticket
+	EncPart          types.EncryptedData
+	DecryptedEncPart EncKrbCredPart
+}
+
+// EncKrbCredPart is the encrypted part of KRB_CRED.
+type EncKrbCredPart struct {
+	TicketInfo []KrbCredInfo     `asn1:"explicit,tag:0"`
+	Nouce      int               `asn1:"optional,explicit,tag:1"`
+	Timestamp  time.Time         `asn1:"generalized,optional,explicit,tag:2"`
+	Usec       int               `asn1:"optional,explicit,tag:3"`
+	SAddress   types.HostAddress `asn1:"optional,explicit,tag:4"`
+	RAddress   types.HostAddress `asn1:"optional,explicit,tag:5"`
+}
+
+// KrbCredInfo is the KRB_CRED_INFO part of KRB_CRED.
+type KrbCredInfo struct {
+	Key       types.EncryptionKey `asn1:"explicit,tag:0"`
+	PRealm    string              `asn1:"generalstring,optional,explicit,tag:1"`
+	PName     types.PrincipalName `asn1:"optional,explicit,tag:2"`
+	Flags     asn1.BitString      `asn1:"optional,explicit,tag:3"`
+	AuthTime  time.Time           `asn1:"generalized,optional,explicit,tag:4"`
+	StartTime time.Time           `asn1:"generalized,optional,explicit,tag:5"`
+	EndTime   time.Time           `asn1:"generalized,optional,explicit,tag:6"`
+	RenewTill time.Time           `asn1:"generalized,optional,explicit,tag:7"`
+	SRealm    string              `asn1:"optional,explicit,ia5,tag:8"`
+	SName     types.PrincipalName `asn1:"optional,explicit,tag:9"`
+	CAddr     types.HostAddresses `asn1:"optional,explicit,tag:10"`
+}
+
+// Unmarshal bytes b into the KRBCred struct.
+func (k *KRBCred) Unmarshal(b []byte) error {
+	var m marshalKRBCred
+	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBCred))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	expectedMsgType := msgtype.KRB_CRED
+	if m.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_CRED. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
+	}
+	k.PVNO = m.PVNO
+	k.MsgType = m.MsgType
+	k.EncPart = m.EncPart
+	if len(m.Tickets.Bytes) > 0 {
+		k.Tickets, err = unmarshalTicketsSequence(m.Tickets)
+		if err != nil {
+			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling tickets within KRB_CRED")
+		}
+	}
+	return nil
+}
+
+// DecryptEncPart decrypts the encrypted part of a KRB_CRED.
+func (k *KRBCred) DecryptEncPart(key types.EncryptionKey) error {
+	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_CRED_ENCPART)
+	if err != nil {
+		return krberror.Errorf(err, krberror.DecryptingError, "error decrypting KRB_CRED EncPart")
+	}
+	var denc EncKrbCredPart
+	err = denc.Unmarshal(b)
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part of KRB_CRED")
+	}
+	k.DecryptedEncPart = denc
+	return nil
+}
+
+// Unmarshal bytes b into the encrypted part of KRB_CRED.
+func (k *EncKrbCredPart) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbCredPart))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling EncKrbCredPart")
+	}
+	return nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go
new file mode 100644
index 0000000..d2cf32d
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go
@@ -0,0 +1,94 @@
+// Package messages implements Kerberos 5 message types and methods.
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// KRBError implements RFC 4120 KRB_ERROR: https://tools.ietf.org/html/rfc4120#section-5.9.1.
+type KRBError struct {
+	PVNO      int                 `asn1:"explicit,tag:0"`
+	MsgType   int                 `asn1:"explicit,tag:1"`
+	CTime     time.Time           `asn1:"generalized,optional,explicit,tag:2"`
+	Cusec     int                 `asn1:"optional,explicit,tag:3"`
+	STime     time.Time           `asn1:"generalized,explicit,tag:4"`
+	Susec     int                 `asn1:"explicit,tag:5"`
+	ErrorCode int32               `asn1:"explicit,tag:6"`
+	CRealm    string              `asn1:"generalstring,optional,explicit,tag:7"`
+	CName     types.PrincipalName `asn1:"optional,explicit,tag:8"`
+	Realm     string              `asn1:"generalstring,explicit,tag:9"`
+	SName     types.PrincipalName `asn1:"explicit,tag:10"`
+	EText     string              `asn1:"generalstring,optional,explicit,tag:11"`
+	EData     []byte              `asn1:"optional,explicit,tag:12"`
+}
+
+// NewKRBError creates a new KRBError.
+func NewKRBError(sname types.PrincipalName, realm string, code int32, etext string) KRBError {
+	t := time.Now().UTC()
+	return KRBError{
+		PVNO:      iana.PVNO,
+		MsgType:   msgtype.KRB_ERROR,
+		STime:     t,
+		Susec:     int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
+		ErrorCode: code,
+		SName:     sname,
+		Realm:     realm,
+		EText:     etext,
+	}
+}
+
+// Unmarshal bytes b into the KRBError struct.
+func (k *KRBError) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBError))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "KRB_ERROR unmarshal error")
+	}
+	expectedMsgType := msgtype.KRB_ERROR
+	if k.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_ERROR. Expected: %v; Actual: %v", expectedMsgType, k.MsgType)
+	}
+	return nil
+}
+
+// Marshal a KRBError into bytes.
+func (k *KRBError) Marshal() ([]byte, error) {
+	b, err := asn1.Marshal(*k)
+	if err != nil {
+		return b, krberror.Errorf(err, krberror.EncodingError, "error marshaling KRBError")
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.KRBError)
+	return b, nil
+}
+
+// Error method implementing error interface on KRBError struct.
+func (k KRBError) Error() string {
+	etxt := fmt.Sprintf("KRB Error: %s", errorcode.Lookup(k.ErrorCode))
+	if k.EText != "" {
+		etxt = fmt.Sprintf("%s - %s", etxt, k.EText)
+	}
+	return etxt
+}
+
+func processUnmarshalReplyError(b []byte, err error) error {
+	switch err.(type) {
+	case asn1.StructuralError:
+		var krberr KRBError
+		tmperr := krberr.Unmarshal(b)
+		if tmperr != nil {
+			return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply")
+		}
+		return krberr
+	default:
+		return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply")
+	}
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go
new file mode 100644
index 0000000..0ca6149
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go
@@ -0,0 +1,108 @@
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// KRBPriv implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.7.1.
+type KRBPriv struct {
+	PVNO             int                 `asn1:"explicit,tag:0"`
+	MsgType          int                 `asn1:"explicit,tag:1"`
+	EncPart          types.EncryptedData `asn1:"explicit,tag:3"`
+	DecryptedEncPart EncKrbPrivPart      `asn1:"optional,omitempty"` // Not part of ASN1 bytes so marked as optional so unmarshalling works
+}
+
+// EncKrbPrivPart is the encrypted part of KRB_PRIV.
+type EncKrbPrivPart struct {
+	UserData       []byte            `asn1:"explicit,tag:0"`
+	Timestamp      time.Time         `asn1:"generalized,optional,explicit,tag:1"`
+	Usec           int               `asn1:"optional,explicit,tag:2"`
+	SequenceNumber int64             `asn1:"optional,explicit,tag:3"`
+	SAddress       types.HostAddress `asn1:"explicit,tag:4"`
+	RAddress       types.HostAddress `asn1:"optional,explicit,tag:5"`
+}
+
+// NewKRBPriv returns a new KRBPriv type.
+func NewKRBPriv(part EncKrbPrivPart) KRBPriv {
+	return KRBPriv{
+		PVNO:             iana.PVNO,
+		MsgType:          msgtype.KRB_PRIV,
+		DecryptedEncPart: part,
+	}
+}
+
+// Unmarshal bytes b into the KRBPriv struct.
+func (k *KRBPriv) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBPriv))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	expectedMsgType := msgtype.KRB_PRIV
+	if k.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_PRIV. Expected: %v; Actual: %v", expectedMsgType, k.MsgType)
+	}
+	return nil
+}
+
+// Unmarshal bytes b into the EncKrbPrivPart struct.
+func (k *EncKrbPrivPart) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbPrivPart))
+	if err != nil {
+		return krberror.Errorf(err, krberror.EncodingError, "KRB_PRIV unmarshal error")
+	}
+	return nil
+}
+
+// Marshal the KRBPriv.
+func (k *KRBPriv) Marshal() ([]byte, error) {
+	tk := KRBPriv{
+		PVNO:    k.PVNO,
+		MsgType: k.MsgType,
+		EncPart: k.EncPart,
+	}
+	b, err := asn1.Marshal(tk)
+	if err != nil {
+		return []byte{}, err
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.KRBPriv)
+	return b, nil
+}
+
+// EncryptEncPart encrypts the DecryptedEncPart within the KRBPriv.
+// Use to prepare for marshaling.
+func (k *KRBPriv) EncryptEncPart(key types.EncryptionKey) error {
+	b, err := asn1.Marshal(k.DecryptedEncPart)
+	if err != nil {
+		return err
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.EncKrbPrivPart)
+	k.EncPart, err = crypto.GetEncryptedData(b, key, keyusage.KRB_PRIV_ENCPART, 1)
+	if err != nil {
+		return err
+	}
+	return nil
+}
+
+// DecryptEncPart decrypts the encrypted part of the KRBPriv message.
+func (k *KRBPriv) DecryptEncPart(key types.EncryptionKey) error {
+	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_PRIV_ENCPART)
+	if err != nil {
+		return fmt.Errorf("error decrypting KRBPriv EncPart: %v", err)
+	}
+	err = k.DecryptedEncPart.Unmarshal(b)
+	if err != nil {
+		return fmt.Errorf("error unmarshaling encrypted part: %v", err)
+	}
+	return nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go
new file mode 100644
index 0000000..52cd284
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go
@@ -0,0 +1,43 @@
+package messages
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/msgtype"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// KRBSafe implements RFC 4120 KRB_SAFE: https://tools.ietf.org/html/rfc4120#section-5.6.1.
+type KRBSafe struct {
+	PVNO     int            `asn1:"explicit,tag:0"`
+	MsgType  int            `asn1:"explicit,tag:1"`
+	SafeBody KRBSafeBody    `asn1:"explicit,tag:2"`
+	Cksum    types.Checksum `asn1:"explicit,tag:3"`
+}
+
+// KRBSafeBody implements the KRB_SAFE_BODY of KRB_SAFE.
+type KRBSafeBody struct {
+	UserData       []byte            `asn1:"explicit,tag:0"`
+	Timestamp      time.Time         `asn1:"generalized,optional,explicit,tag:1"`
+	Usec           int               `asn1:"optional,explicit,tag:2"`
+	SequenceNumber int64             `asn1:"optional,explicit,tag:3"`
+	SAddress       types.HostAddress `asn1:"explicit,tag:4"`
+	RAddress       types.HostAddress `asn1:"optional,explicit,tag:5"`
+}
+
+// Unmarshal bytes b into the KRBSafe struct.
+func (s *KRBSafe) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, s, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBSafe))
+	if err != nil {
+		return processUnmarshalReplyError(b, err)
+	}
+	expectedMsgType := msgtype.KRB_SAFE
+	if s.MsgType != expectedMsgType {
+		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_SAFE. Expected: %v; Actual: %v", expectedMsgType, s.MsgType)
+	}
+	return nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go
new file mode 100644
index 0000000..11efad6
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go
@@ -0,0 +1,262 @@
+package messages
+
+import (
+	"fmt"
+	"log"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/adtype"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+	"github.com/jcmturner/gokrb5/v8/iana/errorcode"
+	"github.com/jcmturner/gokrb5/v8/iana/flags"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/keytab"
+	"github.com/jcmturner/gokrb5/v8/krberror"
+	"github.com/jcmturner/gokrb5/v8/pac"
+	"github.com/jcmturner/gokrb5/v8/types"
+)
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.3
+
+// Ticket implements the Kerberos ticket.
+type Ticket struct {
+	TktVNO           int                 `asn1:"explicit,tag:0"`
+	Realm            string              `asn1:"generalstring,explicit,tag:1"`
+	SName            types.PrincipalName `asn1:"explicit,tag:2"`
+	EncPart          types.EncryptedData `asn1:"explicit,tag:3"`
+	DecryptedEncPart EncTicketPart       `asn1:"optional"` // Not part of ASN1 bytes so marked as optional so unmarshalling works
+}
+
+// EncTicketPart is the encrypted part of the Ticket.
+type EncTicketPart struct {
+	Flags             asn1.BitString          `asn1:"explicit,tag:0"`
+	Key               types.EncryptionKey     `asn1:"explicit,tag:1"`
+	CRealm            string                  `asn1:"generalstring,explicit,tag:2"`
+	CName             types.PrincipalName     `asn1:"explicit,tag:3"`
+	Transited         TransitedEncoding       `asn1:"explicit,tag:4"`
+	AuthTime          time.Time               `asn1:"generalized,explicit,tag:5"`
+	StartTime         time.Time               `asn1:"generalized,explicit,optional,tag:6"`
+	EndTime           time.Time               `asn1:"generalized,explicit,tag:7"`
+	RenewTill         time.Time               `asn1:"generalized,explicit,optional,tag:8"`
+	CAddr             types.HostAddresses     `asn1:"explicit,optional,tag:9"`
+	AuthorizationData types.AuthorizationData `asn1:"explicit,optional,tag:10"`
+}
+
+// TransitedEncoding part of the ticket's encrypted part.
+type TransitedEncoding struct {
+	TRType   int32  `asn1:"explicit,tag:0"`
+	Contents []byte `asn1:"explicit,tag:1"`
+}
+
+// NewTicket creates a new Ticket instance.
+func NewTicket(cname types.PrincipalName, crealm string, sname types.PrincipalName, srealm string, flags asn1.BitString, sktab *keytab.Keytab, eTypeID int32, kvno int, authTime, startTime, endTime, renewTill time.Time) (Ticket, types.EncryptionKey, error) {
+	etype, err := crypto.GetEtype(eTypeID)
+	if err != nil {
+		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting etype for new ticket")
+	}
+	sessionKey, err := types.GenerateEncryptionKey(etype)
+	if err != nil {
+		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error generating session key")
+	}
+
+	etp := EncTicketPart{
+		Flags:     flags,
+		Key:       sessionKey,
+		CRealm:    crealm,
+		CName:     cname,
+		Transited: TransitedEncoding{},
+		AuthTime:  authTime,
+		StartTime: startTime,
+		EndTime:   endTime,
+		RenewTill: renewTill,
+	}
+	b, err := asn1.Marshal(etp)
+	if err != nil {
+		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncodingError, "error marshalling ticket encpart")
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.EncTicketPart)
+	skey, _, err := sktab.GetEncryptionKey(sname, srealm, kvno, eTypeID)
+	if err != nil {
+		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting encryption key for new ticket")
+	}
+	ed, err := crypto.GetEncryptedData(b, skey, keyusage.KDC_REP_TICKET, kvno)
+	if err != nil {
+		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error encrypting ticket encpart")
+	}
+	tkt := Ticket{
+		TktVNO:  iana.PVNO,
+		Realm:   srealm,
+		SName:   sname,
+		EncPart: ed,
+	}
+	return tkt, sessionKey, nil
+}
+
+// Unmarshal bytes b into a Ticket struct.
+func (t *Ticket) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.Ticket))
+	return err
+}
+
+// Marshal the Ticket.
+func (t *Ticket) Marshal() ([]byte, error) {
+	b, err := asn1.Marshal(*t)
+	if err != nil {
+		return nil, err
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.Ticket)
+	return b, nil
+}
+
+// Unmarshal bytes b into the EncTicketPart struct.
+func (t *EncTicketPart) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.EncTicketPart))
+	return err
+}
+
+// unmarshalTicket returns a ticket from the bytes provided.
+func unmarshalTicket(b []byte) (t Ticket, err error) {
+	err = t.Unmarshal(b)
+	return
+}
+
+// UnmarshalTicketsSequence returns a slice of Tickets from a raw ASN1 value.
+func unmarshalTicketsSequence(in asn1.RawValue) ([]Ticket, error) {
+	//This is a workaround to a asn1 decoding issue in golang - https://github.com/golang/go/issues/17321. It's not pretty I'm afraid
+	//We pull out raw values from the larger raw value (that is actually the data of the sequence of raw values) and track our position moving along the data.
+	b := in.Bytes
+	// Ignore the head of the asn1 stream (1 byte for tag and those for the length) as this is what tells us its a sequence but we're handling it ourselves
+	p := 1 + asn1tools.GetNumberBytesInLengthHeader(in.Bytes)
+	var tkts []Ticket
+	var raw asn1.RawValue
+	for p < (len(b)) {
+		_, err := asn1.UnmarshalWithParams(b[p:], &raw, fmt.Sprintf("application,tag:%d", asnAppTag.Ticket))
+		if err != nil {
+			return nil, fmt.Errorf("unmarshaling sequence of tickets failed getting length of ticket: %v", err)
+		}
+		t, err := unmarshalTicket(b[p:])
+		if err != nil {
+			return nil, fmt.Errorf("unmarshaling sequence of tickets failed: %v", err)
+		}
+		p += len(raw.FullBytes)
+		tkts = append(tkts, t)
+	}
+	MarshalTicketSequence(tkts)
+	return tkts, nil
+}
+
+// MarshalTicketSequence marshals a slice of Tickets returning an ASN1 raw value containing the ticket sequence.
+func MarshalTicketSequence(tkts []Ticket) (asn1.RawValue, error) {
+	raw := asn1.RawValue{
+		Class:      2,
+		IsCompound: true,
+	}
+	if len(tkts) < 1 {
+		// There are no tickets to marshal
+		return raw, nil
+	}
+	var btkts []byte
+	for i, t := range tkts {
+		b, err := t.Marshal()
+		if err != nil {
+			return raw, fmt.Errorf("error marshaling ticket number %d in sequence of tickets", i+1)
+		}
+		btkts = append(btkts, b...)
+	}
+	// The ASN1 wrapping consists of 2 bytes:
+	// 1st byte -> Identifier Octet - In this case an OCTET STRING (ASN TAG
+	// 2nd byte -> The length (this will be the size indicated in the input bytes + 2 for the additional bytes we add here.
+	// Application Tag:
+	//| Byte:       | 8                            | 7                          | 6                                         | 5 | 4 | 3 | 2 | 1             |
+	//| Value:      | 0                            | 1                          | 1                                         | From the RFC spec 4120        |
+	//| Explanation | Defined by the ASN1 encoding rules for an application tag | A value of 1 indicates a constructed type | The ASN Application tag value |
+	btkts = append(asn1tools.MarshalLengthBytes(len(btkts)), btkts...)
+	btkts = append([]byte{byte(32 + asn1.TagSequence)}, btkts...)
+	raw.Bytes = btkts
+	// If we need to create the full bytes then identifier octet is "context-specific" = 128 + "constructed" + 32 + the wrapping explicit tag (11)
+	//fmt.Fprintf(os.Stderr, "mRaw fb: %v\n", raw.FullBytes)
+	return raw, nil
+}
+
+// DecryptEncPart decrypts the encrypted part of the ticket.
+// The sname argument can be used to specify which service principal's key should be used to decrypt the ticket.
+// If nil is passed as the sname then the service principal specified within the ticket it used.
+func (t *Ticket) DecryptEncPart(keytab *keytab.Keytab, sname *types.PrincipalName) error {
+	if sname == nil {
+		sname = &t.SName
+	}
+	key, _, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType)
+	if err != nil {
+		return NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err))
+	}
+	return t.Decrypt(key)
+}
+
+// Decrypt decrypts the encrypted part of the ticket using the key provided.
+func (t *Ticket) Decrypt(key types.EncryptionKey) error {
+	b, err := crypto.DecryptEncPart(t.EncPart, key, keyusage.KDC_REP_TICKET)
+	if err != nil {
+		return fmt.Errorf("error decrypting Ticket EncPart: %v", err)
+	}
+	var denc EncTicketPart
+	err = denc.Unmarshal(b)
+	if err != nil {
+		return fmt.Errorf("error unmarshaling encrypted part: %v", err)
+	}
+	t.DecryptedEncPart = denc
+	return nil
+}
+
+// GetPACType returns a Microsoft PAC that has been extracted from the ticket and processed.
+func (t *Ticket) GetPACType(keytab *keytab.Keytab, sname *types.PrincipalName, l *log.Logger) (bool, pac.PACType, error) {
+	var isPAC bool
+	for _, ad := range t.DecryptedEncPart.AuthorizationData {
+		if ad.ADType == adtype.ADIfRelevant {
+			var ad2 types.AuthorizationData
+			err := ad2.Unmarshal(ad.ADData)
+			if err != nil {
+				l.Printf("PAC authorization data could not be unmarshaled: %v", err)
+				continue
+			}
+			if ad2[0].ADType == adtype.ADWin2KPAC {
+				isPAC = true
+				var p pac.PACType
+				err = p.Unmarshal(ad2[0].ADData)
+				if err != nil {
+					return isPAC, p, fmt.Errorf("error unmarshaling PAC: %v", err)
+				}
+				if sname == nil {
+					sname = &t.SName
+				}
+				key, _, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType)
+				if err != nil {
+					return isPAC, p, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err))
+				}
+				err = p.ProcessPACInfoBuffers(key, l)
+				return isPAC, p, err
+			}
+		}
+	}
+	return isPAC, pac.PACType{}, nil
+}
+
+// Valid checks it the ticket is currently valid. Max duration passed endtime passed in as argument.
+func (t *Ticket) Valid(d time.Duration) (bool, error) {
+	// Check for future tickets or invalid tickets
+	time := time.Now().UTC()
+	if t.DecryptedEncPart.StartTime.Sub(time) > d || types.IsFlagSet(&t.DecryptedEncPart.Flags, flags.Invalid) {
+		return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_NYV, "service ticket provided is not yet valid")
+	}
+
+	// Check for expired ticket
+	if time.Sub(t.DecryptedEncPart.EndTime) > d {
+		return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_EXPIRED, "service ticket provided has expired")
+	}
+
+	return true, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go
new file mode 100644
index 0000000..36871e0
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go
@@ -0,0 +1,34 @@
+package pac
+
+import (
+	"bytes"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx
+
+// ClientClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh536365.aspx
+type ClientClaimsInfo struct {
+	ClaimsSetMetadata mstypes.ClaimsSetMetadata
+	ClaimsSet         mstypes.ClaimsSet
+}
+
+// Unmarshal bytes into the ClientClaimsInfo struct
+func (k *ClientClaimsInfo) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	m := new(mstypes.ClaimsSetMetadata)
+	err = dec.Decode(m)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err)
+		return
+	}
+	k.ClaimsSetMetadata = *m
+	k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet()
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go
new file mode 100644
index 0000000..ddd9578
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go
@@ -0,0 +1,31 @@
+package pac
+
+import (
+	"bytes"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+)
+
+// ClientInfo implements https://msdn.microsoft.com/en-us/library/cc237951.aspx
+type ClientInfo struct {
+	ClientID   mstypes.FileTime // A FILETIME structure in little-endian format that contains the Kerberos initial ticket-granting ticket TGT authentication time
+	NameLength uint16           // An unsigned 16-bit integer in little-endian format that specifies the length, in bytes, of the Name field.
+	Name       string           // An array of 16-bit Unicode characters in little-endian format that contains the client's account name.
+}
+
+// Unmarshal bytes into the ClientInfo struct
+func (k *ClientInfo) Unmarshal(b []byte) (err error) {
+	//The PAC_CLIENT_INFO structure is a simple structure that is not NDR-encoded.
+	r := mstypes.NewReader(bytes.NewReader(b))
+
+	k.ClientID, err = r.FileTime()
+	if err != nil {
+		return
+	}
+	k.NameLength, err = r.Uint16()
+	if err != nil {
+		return
+	}
+	k.Name, err = r.UTF16String(int(k.NameLength))
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go
new file mode 100644
index 0000000..0c7ccd4
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go
@@ -0,0 +1,86 @@
+package pac
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/types"
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// https://msdn.microsoft.com/en-us/library/cc237931.aspx
+
+// CredentialsInfo implements https://msdn.microsoft.com/en-us/library/cc237953.aspx
+type CredentialsInfo struct {
+	Version                    uint32 // A 32-bit unsigned integer in little-endian format that defines the version. MUST be 0x00000000.
+	EType                      uint32
+	PACCredentialDataEncrypted []byte // Key usage number for encryption: KERB_NON_KERB_SALT (16)
+	PACCredentialData          CredentialData
+}
+
+// Unmarshal bytes into the CredentialsInfo struct
+func (c *CredentialsInfo) Unmarshal(b []byte, k types.EncryptionKey) (err error) {
+	//The CredentialsInfo structure is a simple structure that is not NDR-encoded.
+	r := mstypes.NewReader(bytes.NewReader(b))
+
+	c.Version, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	if c.Version != 0 {
+		err = errors.New("credentials info version is not zero")
+		return
+	}
+	c.EType, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	c.PACCredentialDataEncrypted, err = r.ReadBytes(len(b) - 8)
+	if err != nil {
+		err = fmt.Errorf("error reading PAC Credetials Data: %v", err)
+		return
+	}
+
+	err = c.DecryptEncPart(k)
+	if err != nil {
+		err = fmt.Errorf("error decrypting PAC Credentials Data: %v", err)
+		return
+	}
+	return
+}
+
+// DecryptEncPart decrypts the encrypted part of the CredentialsInfo.
+func (c *CredentialsInfo) DecryptEncPart(k types.EncryptionKey) error {
+	if k.KeyType != int32(c.EType) {
+		return fmt.Errorf("key provided is not the correct type. Type needed: %d, type provided: %d", c.EType, k.KeyType)
+	}
+	pt, err := crypto.DecryptMessage(c.PACCredentialDataEncrypted, k, keyusage.KERB_NON_KERB_SALT)
+	if err != nil {
+		return err
+	}
+	err = c.PACCredentialData.Unmarshal(pt)
+	if err != nil {
+		return err
+	}
+	return nil
+}
+
+// CredentialData implements https://msdn.microsoft.com/en-us/library/cc237952.aspx
+type CredentialData struct {
+	CredentialCount uint32
+	Credentials     []SECPKGSupplementalCred // Size is the value of CredentialCount
+}
+
+// Unmarshal converts the bytes provided into a CredentialData type.
+func (c *CredentialData) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	err = dec.Decode(c)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go
new file mode 100644
index 0000000..6eb2926
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go
@@ -0,0 +1,34 @@
+package pac
+
+import (
+	"bytes"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx
+
+// DeviceClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh554226.aspx
+type DeviceClaimsInfo struct {
+	ClaimsSetMetadata mstypes.ClaimsSetMetadata
+	ClaimsSet         mstypes.ClaimsSet
+}
+
+// Unmarshal bytes into the ClientClaimsInfo struct
+func (k *DeviceClaimsInfo) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	m := new(mstypes.ClaimsSetMetadata)
+	err = dec.Decode(m)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err)
+		return
+	}
+	k.ClaimsSetMetadata = *m
+	k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet()
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go
new file mode 100644
index 0000000..ce82daa
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go
@@ -0,0 +1,32 @@
+package pac
+
+import (
+	"bytes"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// DeviceInfo implements https://msdn.microsoft.com/en-us/library/hh536402.aspx
+type DeviceInfo struct {
+	UserID            uint32                          // A 32-bit unsigned integer that contains the RID of the account. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account.
+	PrimaryGroupID    uint32                          // A 32-bit unsigned integer that contains the RID for the primary group to which this account belongs.
+	AccountDomainID   mstypes.RPCSID                  `ndr:"pointer"` // A SID structure that contains the SID for the domain of the account.This member is used in conjunction with the UserId, and GroupIds members to create the user and group SIDs for the client.
+	AccountGroupCount uint32                          // A 32-bit unsigned integer that contains the number of groups within the account domain to which the account belongs
+	AccountGroupIDs   []mstypes.GroupMembership       `ndr:"pointer,conformant"` // A pointer to a list of GROUP_MEMBERSHIP (section 2.2.2) structures that contains the groups to which the account belongs in the account domain. The number of groups in this list MUST be equal to GroupCount.
+	SIDCount          uint32                          // A 32-bit unsigned integer that contains the total number of SIDs present in the ExtraSids member.
+	ExtraSIDs         []mstypes.KerbSidAndAttributes  `ndr:"pointer,conformant"` // A pointer to a list of KERB_SID_AND_ATTRIBUTES structures that contain a list of SIDs corresponding to groups not in domains. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account.
+	DomainGroupCount  uint32                          // A 32-bit unsigned integer that contains the number of domains with groups to which the account belongs.
+	DomainGroup       []mstypes.DomainGroupMembership `ndr:"pointer,conformant"` // A pointer to a list of DOMAIN_GROUP_MEMBERSHIP structures (section 2.2.3) that contains the domains to which the account belongs to a group. The number of sets in this list MUST be equal to DomainCount.
+}
+
+// Unmarshal bytes into the DeviceInfo struct
+func (k *DeviceInfo) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	err = dec.Decode(k)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling DeviceInfo: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go
new file mode 100644
index 0000000..dde7861
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go
@@ -0,0 +1,110 @@
+// Package pac implements Microsoft Privilege Attribute Certificate (PAC) processing.
+package pac
+
+import (
+	"bytes"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// KERB_VALIDATION_INFO flags.
+const (
+	USERFLAG_GUEST                                    = 31 // Authentication was done via the GUEST account; no password was used.
+	USERFLAG_NO_ENCRYPTION_AVAILABLE                  = 30 // No encryption is available.
+	USERFLAG_LAN_MANAGER_KEY                          = 28 // LAN Manager key was used for authentication.
+	USERFLAG_SUB_AUTH                                 = 25 // Sub-authentication used; session key came from the sub-authentication package.
+	USERFLAG_EXTRA_SIDS                               = 26 // Indicates that the ExtraSids field is populated and contains additional SIDs.
+	USERFLAG_MACHINE_ACCOUNT                          = 24 // Indicates that the account is a machine account.
+	USERFLAG_DC_NTLM2                                 = 23 // Indicates that the domain controller understands NTLMv2.
+	USERFLAG_RESOURCE_GROUPIDS                        = 22 // Indicates that the ResourceGroupIds field is populated.
+	USERFLAG_PROFILEPATH                              = 21 // Indicates that ProfilePath is populated.
+	USERFLAG_NTLM2_NTCHALLENGERESP                    = 20 // The NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation.
+	USERFLAG_LM2_LMCHALLENGERESP                      = 19 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation.
+	USERFLAG_AUTH_LMCHALLENGERESP_KEY_NTCHALLENGERESP = 18 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and the NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used session key generation.
+)
+
+// KerbValidationInfo implement https://msdn.microsoft.com/en-us/library/cc237948.aspx
+type KerbValidationInfo struct {
+	LogOnTime              mstypes.FileTime
+	LogOffTime             mstypes.FileTime
+	KickOffTime            mstypes.FileTime
+	PasswordLastSet        mstypes.FileTime
+	PasswordCanChange      mstypes.FileTime
+	PasswordMustChange     mstypes.FileTime
+	EffectiveName          mstypes.RPCUnicodeString
+	FullName               mstypes.RPCUnicodeString
+	LogonScript            mstypes.RPCUnicodeString
+	ProfilePath            mstypes.RPCUnicodeString
+	HomeDirectory          mstypes.RPCUnicodeString
+	HomeDirectoryDrive     mstypes.RPCUnicodeString
+	LogonCount             uint16
+	BadPasswordCount       uint16
+	UserID                 uint32
+	PrimaryGroupID         uint32
+	GroupCount             uint32
+	GroupIDs               []mstypes.GroupMembership `ndr:"pointer,conformant"`
+	UserFlags              uint32
+	UserSessionKey         mstypes.UserSessionKey
+	LogonServer            mstypes.RPCUnicodeString
+	LogonDomainName        mstypes.RPCUnicodeString
+	LogonDomainID          mstypes.RPCSID `ndr:"pointer"`
+	Reserved1              [2]uint32      // Has 2 elements
+	UserAccountControl     uint32
+	SubAuthStatus          uint32
+	LastSuccessfulILogon   mstypes.FileTime
+	LastFailedILogon       mstypes.FileTime
+	FailedILogonCount      uint32
+	Reserved3              uint32
+	SIDCount               uint32
+	ExtraSIDs              []mstypes.KerbSidAndAttributes `ndr:"pointer,conformant"`
+	ResourceGroupDomainSID mstypes.RPCSID                 `ndr:"pointer"`
+	ResourceGroupCount     uint32
+	ResourceGroupIDs       []mstypes.GroupMembership `ndr:"pointer,conformant"`
+}
+
+// Unmarshal bytes into the DeviceInfo struct
+func (k *KerbValidationInfo) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	err = dec.Decode(k)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err)
+	}
+	return
+}
+
+// GetGroupMembershipSIDs returns a slice of strings containing the group membership SIDs found in the PAC.
+func (k *KerbValidationInfo) GetGroupMembershipSIDs() []string {
+	var g []string
+	lSID := k.LogonDomainID.String()
+	for i := range k.GroupIDs {
+		g = append(g, fmt.Sprintf("%s-%d", lSID, k.GroupIDs[i].RelativeID))
+	}
+	for _, s := range k.ExtraSIDs {
+		var exists = false
+		for _, es := range g {
+			if es == s.SID.String() {
+				exists = true
+				break
+			}
+		}
+		if !exists {
+			g = append(g, s.SID.String())
+		}
+	}
+	for _, r := range k.ResourceGroupIDs {
+		var exists = false
+		s := fmt.Sprintf("%s-%d", k.ResourceGroupDomainSID.String(), r.RelativeID)
+		for _, es := range g {
+			if es == s {
+				exists = true
+				break
+			}
+		}
+		if !exists {
+			g = append(g, s)
+		}
+	}
+	return g
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go
new file mode 100644
index 0000000..fab2ad7
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go
@@ -0,0 +1,251 @@
+package pac
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"log"
+
+	"github.com/jcmturner/gokrb5/v8/crypto"
+	"github.com/jcmturner/gokrb5/v8/iana/keyusage"
+	"github.com/jcmturner/gokrb5/v8/types"
+	"github.com/jcmturner/rpc/v2/mstypes"
+)
+
+const (
+	infoTypeKerbValidationInfo     uint32 = 1
+	infoTypeCredentials            uint32 = 2
+	infoTypePACServerSignatureData uint32 = 6
+	infoTypePACKDCSignatureData    uint32 = 7
+	infoTypePACClientInfo          uint32 = 10
+	infoTypeS4UDelegationInfo      uint32 = 11
+	infoTypeUPNDNSInfo             uint32 = 12
+	infoTypePACClientClaimsInfo    uint32 = 13
+	infoTypePACDeviceInfo          uint32 = 14
+	infoTypePACDeviceClaimsInfo    uint32 = 15
+)
+
+// PACType implements: https://msdn.microsoft.com/en-us/library/cc237950.aspx
+type PACType struct {
+	CBuffers           uint32
+	Version            uint32
+	Buffers            []InfoBuffer
+	Data               []byte
+	KerbValidationInfo *KerbValidationInfo
+	CredentialsInfo    *CredentialsInfo
+	ServerChecksum     *SignatureData
+	KDCChecksum        *SignatureData
+	ClientInfo         *ClientInfo
+	S4UDelegationInfo  *S4UDelegationInfo
+	UPNDNSInfo         *UPNDNSInfo
+	ClientClaimsInfo   *ClientClaimsInfo
+	DeviceInfo         *DeviceInfo
+	DeviceClaimsInfo   *DeviceClaimsInfo
+	ZeroSigData        []byte
+}
+
+// InfoBuffer implements the PAC Info Buffer: https://msdn.microsoft.com/en-us/library/cc237954.aspx
+type InfoBuffer struct {
+	ULType       uint32 // A 32-bit unsigned integer in little-endian format that describes the type of data present in the buffer contained at Offset.
+	CBBufferSize uint32 // A 32-bit unsigned integer in little-endian format that contains the size, in bytes, of the buffer in the PAC located at Offset.
+	Offset       uint64 // A 64-bit unsigned integer in little-endian format that contains the offset to the beginning of the buffer, in bytes, from the beginning of the PACTYPE structure. The data offset MUST be a multiple of eight. The following sections specify the format of each type of element.
+}
+
+// Unmarshal bytes into the PACType struct
+func (pac *PACType) Unmarshal(b []byte) (err error) {
+	pac.Data = b
+	zb := make([]byte, len(b), len(b))
+	copy(zb, b)
+	pac.ZeroSigData = zb
+	r := mstypes.NewReader(bytes.NewReader(b))
+	pac.CBuffers, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	pac.Version, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	buf := make([]InfoBuffer, pac.CBuffers, pac.CBuffers)
+	for i := range buf {
+		buf[i].ULType, err = r.Uint32()
+		if err != nil {
+			return
+		}
+		buf[i].CBBufferSize, err = r.Uint32()
+		if err != nil {
+			return
+		}
+		buf[i].Offset, err = r.Uint64()
+		if err != nil {
+			return
+		}
+	}
+	pac.Buffers = buf
+	return nil
+}
+
+// ProcessPACInfoBuffers processes the PAC Info Buffers.
+// https://msdn.microsoft.com/en-us/library/cc237954.aspx
+func (pac *PACType) ProcessPACInfoBuffers(key types.EncryptionKey, l *log.Logger) error {
+	for _, buf := range pac.Buffers {
+		p := make([]byte, buf.CBBufferSize, buf.CBBufferSize)
+		copy(p, pac.Data[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)])
+		switch buf.ULType {
+		case infoTypeKerbValidationInfo:
+			if pac.KerbValidationInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k KerbValidationInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				return fmt.Errorf("error processing KerbValidationInfo: %v", err)
+			}
+			pac.KerbValidationInfo = &k
+		case infoTypeCredentials:
+			// Currently PAC parsing is only useful on the service side in gokrb5
+			// The CredentialsInfo are only useful when gokrb5 has implemented RFC4556 and only applied on the client side.
+			// Skipping CredentialsInfo - will be revisited under RFC4556 implementation.
+			continue
+			//if pac.CredentialsInfo != nil {
+			//	//Must ignore subsequent buffers of this type
+			//	continue
+			//}
+			//var k CredentialsInfo
+			//err := k.Unmarshal(p, key) // The encryption key used is the AS reply key only available to the client.
+			//if err != nil {
+			//	return fmt.Errorf("error processing CredentialsInfo: %v", err)
+			//}
+			//pac.CredentialsInfo = &k
+		case infoTypePACServerSignatureData:
+			if pac.ServerChecksum != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k SignatureData
+			zb, err := k.Unmarshal(p)
+			copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb)
+			if err != nil {
+				return fmt.Errorf("error processing ServerChecksum: %v", err)
+			}
+			pac.ServerChecksum = &k
+		case infoTypePACKDCSignatureData:
+			if pac.KDCChecksum != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k SignatureData
+			zb, err := k.Unmarshal(p)
+			copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb)
+			if err != nil {
+				return fmt.Errorf("error processing KDCChecksum: %v", err)
+			}
+			pac.KDCChecksum = &k
+		case infoTypePACClientInfo:
+			if pac.ClientInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k ClientInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				return fmt.Errorf("error processing ClientInfo: %v", err)
+			}
+			pac.ClientInfo = &k
+		case infoTypeS4UDelegationInfo:
+			if pac.S4UDelegationInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k S4UDelegationInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				l.Printf("could not process S4U_DelegationInfo: %v", err)
+				continue
+			}
+			pac.S4UDelegationInfo = &k
+		case infoTypeUPNDNSInfo:
+			if pac.UPNDNSInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k UPNDNSInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				l.Printf("could not process UPN_DNSInfo: %v", err)
+				continue
+			}
+			pac.UPNDNSInfo = &k
+		case infoTypePACClientClaimsInfo:
+			if pac.ClientClaimsInfo != nil || len(p) < 1 {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k ClientClaimsInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				l.Printf("could not process ClientClaimsInfo: %v", err)
+				continue
+			}
+			pac.ClientClaimsInfo = &k
+		case infoTypePACDeviceInfo:
+			if pac.DeviceInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k DeviceInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				l.Printf("could not process DeviceInfo: %v", err)
+				continue
+			}
+			pac.DeviceInfo = &k
+		case infoTypePACDeviceClaimsInfo:
+			if pac.DeviceClaimsInfo != nil {
+				//Must ignore subsequent buffers of this type
+				continue
+			}
+			var k DeviceClaimsInfo
+			err := k.Unmarshal(p)
+			if err != nil {
+				l.Printf("could not process DeviceClaimsInfo: %v", err)
+				continue
+			}
+			pac.DeviceClaimsInfo = &k
+		}
+	}
+
+	if ok, err := pac.verify(key); !ok {
+		return err
+	}
+
+	return nil
+}
+
+func (pac *PACType) verify(key types.EncryptionKey) (bool, error) {
+	if pac.KerbValidationInfo == nil {
+		return false, errors.New("PAC Info Buffers does not contain a KerbValidationInfo")
+	}
+	if pac.ServerChecksum == nil {
+		return false, errors.New("PAC Info Buffers does not contain a ServerChecksum")
+	}
+	if pac.KDCChecksum == nil {
+		return false, errors.New("PAC Info Buffers does not contain a KDCChecksum")
+	}
+	if pac.ClientInfo == nil {
+		return false, errors.New("PAC Info Buffers does not contain a ClientInfo")
+	}
+	etype, err := crypto.GetChksumEtype(int32(pac.ServerChecksum.SignatureType))
+	if err != nil {
+		return false, err
+	}
+	if ok := etype.VerifyChecksum(key.KeyValue,
+		pac.ZeroSigData,
+		pac.ServerChecksum.Signature,
+		keyusage.KERB_NON_KERB_CKSUM_SALT); !ok {
+		return false, errors.New("PAC service checksum verification failed")
+	}
+
+	return true, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go
new file mode 100644
index 0000000..da837d4
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go
@@ -0,0 +1,26 @@
+package pac
+
+import (
+	"bytes"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+// S4UDelegationInfo implements https://msdn.microsoft.com/en-us/library/cc237944.aspx
+type S4UDelegationInfo struct {
+	S4U2proxyTarget      mstypes.RPCUnicodeString // The name of the principal to whom the application can forward the ticket.
+	TransitedListSize    uint32
+	S4UTransitedServices []mstypes.RPCUnicodeString `ndr:"pointer,conformant"` // List of all services that have been delegated through by this client and subsequent services or servers.. Size is value of TransitedListSize
+}
+
+// Unmarshal bytes into the S4UDelegationInfo struct
+func (k *S4UDelegationInfo) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	err = dec.Decode(k)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling S4UDelegationInfo: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go
new file mode 100644
index 0000000..8f6aa58
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go
@@ -0,0 +1,67 @@
+package pac
+
+import (
+	"bytes"
+
+	"github.com/jcmturner/gokrb5/v8/iana/chksumtype"
+	"github.com/jcmturner/rpc/v2/mstypes"
+)
+
+/*
+https://msdn.microsoft.com/en-us/library/cc237955.aspx
+
+The Key Usage Value MUST be KERB_NON_KERB_CKSUM_SALT (17) [MS-KILE] (section 3.1.5.9).
+
+Server Signature (SignatureType = 0x00000006)
+https://msdn.microsoft.com/en-us/library/cc237957.aspx
+
+KDC Signature (SignatureType = 0x00000007)
+https://msdn.microsoft.com/en-us/library/dd357117.aspx
+*/
+
+// SignatureData implements https://msdn.microsoft.com/en-us/library/cc237955.aspx
+type SignatureData struct {
+	SignatureType  uint32 // A 32-bit unsigned integer value in little-endian format that defines the cryptographic system used to calculate the checksum. This MUST be one of the following checksum types: KERB_CHECKSUM_HMAC_MD5 (signature size = 16), HMAC_SHA1_96_AES128 (signature size = 12), HMAC_SHA1_96_AES256 (signature size = 12).
+	Signature      []byte // Size depends on the type. See comment above.
+	RODCIdentifier uint16 // A 16-bit unsigned integer value in little-endian format that contains the first 16 bits of the key version number ([MS-KILE] section 3.1.5.8) when the KDC is an RODC. When the KDC is not an RODC, this field does not exist.
+}
+
+// Unmarshal bytes into the SignatureData struct
+func (k *SignatureData) Unmarshal(b []byte) (rb []byte, err error) {
+	r := mstypes.NewReader(bytes.NewReader(b))
+
+	k.SignatureType, err = r.Uint32()
+	if err != nil {
+		return
+	}
+
+	var c int
+	switch k.SignatureType {
+	case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED:
+		c = 16
+	case uint32(chksumtype.HMAC_SHA1_96_AES128):
+		c = 12
+	case uint32(chksumtype.HMAC_SHA1_96_AES256):
+		c = 12
+	}
+	k.Signature, err = r.ReadBytes(c)
+	if err != nil {
+		return
+	}
+
+	// When the KDC is not an Read Only Domain Controller (RODC), this field does not exist.
+	if len(b) >= 4+c+2 {
+		k.RODCIdentifier, err = r.Uint16()
+		if err != nil {
+			return
+		}
+	}
+
+	// Create bytes with zeroed signature needed for checksum verification
+	rb = make([]byte, len(b), len(b))
+	copy(rb, b)
+	z := make([]byte, len(b), len(b))
+	copy(rb[4:4+c], z)
+
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go
new file mode 100644
index 0000000..d40679d
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go
@@ -0,0 +1,87 @@
+package pac
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+	"github.com/jcmturner/rpc/v2/ndr"
+)
+
+const (
+	// NTLMSupCredLMOWF indicates that the LM OWF member is present and valid.
+	NTLMSupCredLMOWF uint32 = 31
+	// NTLMSupCredNTOWF indicates that the NT OWF member is present and valid.
+	NTLMSupCredNTOWF uint32 = 30
+)
+
+// NTLMSupplementalCred implements https://msdn.microsoft.com/en-us/library/cc237949.aspx
+type NTLMSupplementalCred struct {
+	Version    uint32 // A 32-bit unsigned integer that defines the credential version.This field MUST be 0x00000000.
+	Flags      uint32
+	LMPassword []byte // A 16-element array of unsigned 8-bit integers that define the LM OWF. The LMPassword member MUST be ignored if the L flag is not set in the Flags member.
+	NTPassword []byte // A 16-element array of unsigned 8-bit integers that define the NT OWF. The NTPassword member MUST be ignored if the N flag is not set in the Flags member.
+}
+
+// Unmarshal converts the bytes provided into a NTLMSupplementalCred.
+func (c *NTLMSupplementalCred) Unmarshal(b []byte) (err error) {
+	r := mstypes.NewReader(bytes.NewReader(b))
+	c.Version, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	if c.Version != 0 {
+		err = errors.New("NTLMSupplementalCred version is not zero")
+		return
+	}
+	c.Flags, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	if isFlagSet(c.Flags, NTLMSupCredLMOWF) {
+		c.LMPassword, err = r.ReadBytes(16)
+		if err != nil {
+			return
+		}
+	}
+	if isFlagSet(c.Flags, NTLMSupCredNTOWF) {
+		c.NTPassword, err = r.ReadBytes(16)
+		if err != nil {
+			return
+		}
+	}
+	return
+}
+
+// isFlagSet tests if a flag is set in the uint32 little endian flag
+func isFlagSet(f uint32, i uint32) bool {
+	//Which byte?
+	b := int(i / 8)
+	//Which bit in byte
+	p := uint(7 - (int(i) - 8*b))
+	fb := make([]byte, 4)
+	binary.LittleEndian.PutUint32(fb, f)
+	if fb[b]&(1<<p) != 0 {
+		return true
+	}
+	return false
+}
+
+// SECPKGSupplementalCred implements https://msdn.microsoft.com/en-us/library/cc237956.aspx
+type SECPKGSupplementalCred struct {
+	PackageName    mstypes.RPCUnicodeString
+	CredentialSize uint32
+	Credentials    []uint8 `ndr:"pointer,conformant"` // Is a ptr. Size is the value of CredentialSize
+}
+
+// Unmarshal converts the bytes provided into a SECPKGSupplementalCred.
+func (c *SECPKGSupplementalCred) Unmarshal(b []byte) (err error) {
+	dec := ndr.NewDecoder(bytes.NewReader(b))
+	err = dec.Decode(c)
+	if err != nil {
+		err = fmt.Errorf("error unmarshaling SECPKGSupplementalCred: %v", err)
+	}
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/upn_dns_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/upn_dns_info.go
new file mode 100644
index 0000000..d374b96
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/upn_dns_info.go
@@ -0,0 +1,73 @@
+package pac
+
+import (
+	"bytes"
+
+	"github.com/jcmturner/rpc/v2/mstypes"
+)
+
+// UPNDNSInfo implements https://msdn.microsoft.com/en-us/library/dd240468.aspx
+type UPNDNSInfo struct {
+	UPNLength           uint16 // An unsigned 16-bit integer in little-endian format that specifies the length, in bytes, of the UPN field.
+	UPNOffset           uint16 // An unsigned 16-bit integer in little-endian format that contains the offset to the beginning of the buffer, in bytes, from the beginning of the UPN_DNS_INFO structure.
+	DNSDomainNameLength uint16
+	DNSDomainNameOffset uint16
+	Flags               uint32
+	UPN                 string
+	DNSDomain           string
+}
+
+const (
+	upnNoUPNAttr = 31 // The user account object does not have the userPrincipalName attribute ([MS-ADA3] section 2.349) set. A UPN constructed by concatenating the user name with the DNS domain name of the account domain is provided.
+)
+
+// Unmarshal bytes into the UPN_DNSInfo struct
+func (k *UPNDNSInfo) Unmarshal(b []byte) (err error) {
+	//The UPN_DNS_INFO structure is a simple structure that is not NDR-encoded.
+	r := mstypes.NewReader(bytes.NewReader(b))
+	k.UPNLength, err = r.Uint16()
+	if err != nil {
+		return
+	}
+	k.UPNOffset, err = r.Uint16()
+	if err != nil {
+		return
+	}
+	k.DNSDomainNameLength, err = r.Uint16()
+	if err != nil {
+		return
+	}
+	k.DNSDomainNameOffset, err = r.Uint16()
+	if err != nil {
+		return
+	}
+	k.Flags, err = r.Uint32()
+	if err != nil {
+		return
+	}
+	ub := mstypes.NewReader(bytes.NewReader(b[k.UPNOffset : k.UPNOffset+k.UPNLength]))
+	db := mstypes.NewReader(bytes.NewReader(b[k.DNSDomainNameOffset : k.DNSDomainNameOffset+k.DNSDomainNameLength]))
+
+	u := make([]rune, k.UPNLength/2, k.UPNLength/2)
+	for i := 0; i < len(u); i++ {
+		var r uint16
+		r, err = ub.Uint16()
+		if err != nil {
+			return
+		}
+		u[i] = rune(r)
+	}
+	k.UPN = string(u)
+	d := make([]rune, k.DNSDomainNameLength/2, k.DNSDomainNameLength/2)
+	for i := 0; i < len(d); i++ {
+		var r uint16
+		r, err = db.Uint16()
+		if err != nil {
+			return
+		}
+		d[i] = rune(r)
+	}
+	k.DNSDomain = string(d)
+
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/Authenticator.go b/vendor/github.com/jcmturner/gokrb5/v8/types/Authenticator.go
new file mode 100644
index 0000000..1fdba78
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/Authenticator.go
@@ -0,0 +1,81 @@
+// Package types provides Kerberos 5 data types.
+package types
+
+import (
+	"crypto/rand"
+	"fmt"
+	"math"
+	"math/big"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/asn1tools"
+	"github.com/jcmturner/gokrb5/v8/iana"
+	"github.com/jcmturner/gokrb5/v8/iana/asnAppTag"
+)
+
+// Authenticator - A record containing information that can be shown to have been recently generated using the session
+// key known only by the client and server.
+// https://tools.ietf.org/html/rfc4120#section-5.5.1
+type Authenticator struct {
+	AVNO              int               `asn1:"explicit,tag:0"`
+	CRealm            string            `asn1:"generalstring,explicit,tag:1"`
+	CName             PrincipalName     `asn1:"explicit,tag:2"`
+	Cksum             Checksum          `asn1:"explicit,optional,tag:3"`
+	Cusec             int               `asn1:"explicit,tag:4"`
+	CTime             time.Time         `asn1:"generalized,explicit,tag:5"`
+	SubKey            EncryptionKey     `asn1:"explicit,optional,tag:6"`
+	SeqNumber         int64             `asn1:"explicit,optional,tag:7"`
+	AuthorizationData AuthorizationData `asn1:"explicit,optional,tag:8"`
+}
+
+// NewAuthenticator creates a new Authenticator.
+func NewAuthenticator(realm string, cname PrincipalName) (Authenticator, error) {
+	seq, err := rand.Int(rand.Reader, big.NewInt(math.MaxUint32))
+	if err != nil {
+		return Authenticator{}, err
+	}
+	t := time.Now().UTC()
+	return Authenticator{
+		AVNO:      iana.PVNO,
+		CRealm:    realm,
+		CName:     cname,
+		Cksum:     Checksum{},
+		Cusec:     int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
+		CTime:     t,
+		SeqNumber: seq.Int64(),
+	}, nil
+}
+
+// GenerateSeqNumberAndSubKey sets the Authenticator's sequence number and subkey.
+func (a *Authenticator) GenerateSeqNumberAndSubKey(keyType int32, keySize int) error {
+	seq, err := rand.Int(rand.Reader, big.NewInt(math.MaxUint32))
+	if err != nil {
+		return err
+	}
+	a.SeqNumber = seq.Int64()
+	//Generate subkey value
+	sk := make([]byte, keySize, keySize)
+	rand.Read(sk)
+	a.SubKey = EncryptionKey{
+		KeyType:  keyType,
+		KeyValue: sk,
+	}
+	return nil
+}
+
+// Unmarshal bytes into the Authenticator.
+func (a *Authenticator) Unmarshal(b []byte) error {
+	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.Authenticator))
+	return err
+}
+
+// Marshal the Authenticator.
+func (a *Authenticator) Marshal() ([]byte, error) {
+	b, err := asn1.Marshal(*a)
+	if err != nil {
+		return nil, err
+	}
+	b = asn1tools.AddASNAppTag(b, asnAppTag.Authenticator)
+	return b, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/AuthorizationData.go b/vendor/github.com/jcmturner/gokrb5/v8/types/AuthorizationData.go
new file mode 100644
index 0000000..80c477c
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/AuthorizationData.go
@@ -0,0 +1,55 @@
+package types
+
+import (
+	"github.com/jcmturner/gofork/encoding/asn1"
+)
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.6
+
+// AuthorizationData implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6
+type AuthorizationData []AuthorizationDataEntry
+
+// AuthorizationDataEntry implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6
+type AuthorizationDataEntry struct {
+	ADType int32  `asn1:"explicit,tag:0"`
+	ADData []byte `asn1:"explicit,tag:1"`
+}
+
+// ADIfRelevant implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.1
+type ADIfRelevant AuthorizationData
+
+// ADKDCIssued implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.2
+type ADKDCIssued struct {
+	ADChecksum Checksum          `asn1:"explicit,tag:0"`
+	IRealm     string            `asn1:"optional,generalstring,explicit,tag:1"`
+	Isname     PrincipalName     `asn1:"optional,explicit,tag:2"`
+	Elements   AuthorizationData `asn1:"explicit,tag:3"`
+}
+
+// ADAndOr implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.3
+type ADAndOr struct {
+	ConditionCount int32             `asn1:"explicit,tag:0"`
+	Elements       AuthorizationData `asn1:"explicit,tag:1"`
+}
+
+// ADMandatoryForKDC implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.4
+type ADMandatoryForKDC AuthorizationData
+
+// Unmarshal bytes into the ADKDCIssued.
+func (a *ADKDCIssued) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the AuthorizationData.
+func (a *AuthorizationData) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the AuthorizationDataEntry.
+func (a *AuthorizationDataEntry) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/Cryptosystem.go b/vendor/github.com/jcmturner/gokrb5/v8/types/Cryptosystem.go
new file mode 100644
index 0000000..2f354ea
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/Cryptosystem.go
@@ -0,0 +1,72 @@
+package types
+
+import (
+	"crypto/rand"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/crypto/etype"
+)
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.9
+
+// EncryptedData implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
+type EncryptedData struct {
+	EType  int32  `asn1:"explicit,tag:0"`
+	KVNO   int    `asn1:"explicit,optional,tag:1"`
+	Cipher []byte `asn1:"explicit,tag:2"`
+}
+
+// EncryptionKey implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
+// AKA KeyBlock
+type EncryptionKey struct {
+	KeyType  int32  `asn1:"explicit,tag:0"`
+	KeyValue []byte `asn1:"explicit,tag:1" json:"-"`
+}
+
+// Checksum implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
+type Checksum struct {
+	CksumType int32  `asn1:"explicit,tag:0"`
+	Checksum  []byte `asn1:"explicit,tag:1"`
+}
+
+// Unmarshal bytes into the EncryptedData.
+func (a *EncryptedData) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Marshal the EncryptedData.
+func (a *EncryptedData) Marshal() ([]byte, error) {
+	edb, err := asn1.Marshal(*a)
+	if err != nil {
+		return edb, err
+	}
+	return edb, nil
+}
+
+// Unmarshal bytes into the EncryptionKey.
+func (a *EncryptionKey) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the Checksum.
+func (a *Checksum) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// GenerateEncryptionKey creates a new EncryptionKey with a random key value.
+func GenerateEncryptionKey(etype etype.EType) (EncryptionKey, error) {
+	k := EncryptionKey{
+		KeyType: etype.GetETypeID(),
+	}
+	b := make([]byte, etype.GetKeyByteSize(), etype.GetKeyByteSize())
+	_, err := rand.Read(b)
+	if err != nil {
+		return k, err
+	}
+	k.KeyValue = b
+	return k, nil
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/HostAddress.go b/vendor/github.com/jcmturner/gokrb5/v8/types/HostAddress.go
new file mode 100644
index 0000000..895fe80
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/HostAddress.go
@@ -0,0 +1,180 @@
+package types
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.5
+
+import (
+	"bytes"
+	"fmt"
+	"net"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/iana/addrtype"
+)
+
+// HostAddresses implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.5
+type HostAddresses []HostAddress
+
+// HostAddress implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.5
+type HostAddress struct {
+	AddrType int32  `asn1:"explicit,tag:0"`
+	Address  []byte `asn1:"explicit,tag:1"`
+}
+
+// GetHostAddress returns a HostAddress struct from a string in the format <hostname>:<port>
+func GetHostAddress(s string) (HostAddress, error) {
+	var h HostAddress
+	cAddr, _, err := net.SplitHostPort(s)
+	if err != nil {
+		return h, fmt.Errorf("invalid format of client address: %v", err)
+	}
+	ip := net.ParseIP(cAddr)
+	var ht int32
+	if ip.To4() != nil {
+		ht = addrtype.IPv4
+		ip = ip.To4()
+	} else if ip.To16() != nil {
+		ht = addrtype.IPv6
+		ip = ip.To16()
+	} else {
+		return h, fmt.Errorf("could not determine client's address types: %v", err)
+	}
+	h = HostAddress{
+		AddrType: ht,
+		Address:  ip,
+	}
+	return h, nil
+}
+
+// GetAddress returns a string representation of the HostAddress.
+func (h *HostAddress) GetAddress() (string, error) {
+	var b []byte
+	_, err := asn1.Unmarshal(h.Address, &b)
+	return string(b), err
+}
+
+// LocalHostAddresses returns a HostAddresses struct for the local machines interface IP addresses.
+func LocalHostAddresses() (ha HostAddresses, err error) {
+	ifs, err := net.Interfaces()
+	if err != nil {
+		return
+	}
+	for _, iface := range ifs {
+		if iface.Flags&net.FlagLoopback != 0 || iface.Flags&net.FlagUp == 0 {
+			// Interface is either loopback of not up
+			continue
+		}
+		addrs, err := iface.Addrs()
+		if err != nil {
+			continue
+		}
+		for _, addr := range addrs {
+			var ip net.IP
+			switch v := addr.(type) {
+			case *net.IPNet:
+				ip = v.IP
+			case *net.IPAddr:
+				ip = v.IP
+			}
+			var a HostAddress
+			if ip.To16() == nil {
+				//neither IPv4 or IPv6
+				continue
+			}
+			if ip.To4() != nil {
+				//Is IPv4
+				a.AddrType = addrtype.IPv4
+				a.Address = ip.To4()
+			} else {
+				a.AddrType = addrtype.IPv6
+				a.Address = ip.To16()
+			}
+			ha = append(ha, a)
+		}
+	}
+	return ha, nil
+}
+
+// HostAddressesFromNetIPs returns a HostAddresses type from a slice of net.IP
+func HostAddressesFromNetIPs(ips []net.IP) (ha HostAddresses) {
+	for _, ip := range ips {
+		ha = append(ha, HostAddressFromNetIP(ip))
+	}
+	return ha
+}
+
+// HostAddressFromNetIP returns a HostAddress type from a net.IP
+func HostAddressFromNetIP(ip net.IP) HostAddress {
+	if ip.To4() != nil {
+		//Is IPv4
+		return HostAddress{
+			AddrType: addrtype.IPv4,
+			Address:  ip.To4(),
+		}
+	}
+	return HostAddress{
+		AddrType: addrtype.IPv6,
+		Address:  ip.To16(),
+	}
+}
+
+// HostAddressesEqual tests if two HostAddress slices are equal.
+func HostAddressesEqual(h, a []HostAddress) bool {
+	if len(h) != len(a) {
+		return false
+	}
+	for _, e := range a {
+		var found bool
+		for _, i := range h {
+			if e.Equal(i) {
+				found = true
+				break
+			}
+		}
+		if !found {
+			return false
+		}
+	}
+	return true
+}
+
+// HostAddressesContains tests if a HostAddress is contained in a HostAddress slice.
+func HostAddressesContains(h []HostAddress, a HostAddress) bool {
+	for _, e := range h {
+		if e.Equal(a) {
+			return true
+		}
+	}
+	return false
+}
+
+// Equal tests if the HostAddress is equal to another HostAddress provided.
+func (h *HostAddress) Equal(a HostAddress) bool {
+	if h.AddrType != a.AddrType {
+		return false
+	}
+	return bytes.Equal(h.Address, a.Address)
+}
+
+// Contains tests if a HostAddress is contained within the HostAddresses struct.
+func (h *HostAddresses) Contains(a HostAddress) bool {
+	for _, e := range *h {
+		if e.Equal(a) {
+			return true
+		}
+	}
+	return false
+}
+
+// Equal tests if a HostAddress slice is equal to the HostAddresses struct.
+func (h *HostAddresses) Equal(a []HostAddress) bool {
+	if len(*h) != len(a) {
+		return false
+	}
+	for _, e := range a {
+		if !h.Contains(e) {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go b/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go
new file mode 100644
index 0000000..0f20383
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go
@@ -0,0 +1,68 @@
+package types
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.8
+
+import (
+	"github.com/jcmturner/gofork/encoding/asn1"
+)
+
+// NewKrbFlags returns an ASN1 BitString struct of the right size for KrbFlags.
+func NewKrbFlags() asn1.BitString {
+	f := asn1.BitString{}
+	f.Bytes = make([]byte, 4)
+	f.BitLength = len(f.Bytes) * 8
+	return f
+}
+
+// SetFlags sets the flags of an ASN1 BitString.
+func SetFlags(f *asn1.BitString, j []int) {
+	for _, i := range j {
+		SetFlag(f, i)
+	}
+}
+
+// SetFlag sets a flag in an ASN1 BitString.
+func SetFlag(f *asn1.BitString, i int) {
+	for l := len(f.Bytes); l < 4; l++ {
+		(*f).Bytes = append((*f).Bytes, byte(0))
+		(*f).BitLength = len((*f).Bytes) * 8
+	}
+	//Which byte?
+	b := i / 8
+	//Which bit in byte
+	p := uint(7 - (i - 8*b))
+	(*f).Bytes[b] = (*f).Bytes[b] | (1 << p)
+}
+
+// UnsetFlags unsets flags in an ASN1 BitString.
+func UnsetFlags(f *asn1.BitString, j []int) {
+	for _, i := range j {
+		UnsetFlag(f, i)
+	}
+}
+
+// UnsetFlag unsets a flag in an ASN1 BitString.
+func UnsetFlag(f *asn1.BitString, i int) {
+	for l := len(f.Bytes); l < 4; l++ {
+		(*f).Bytes = append((*f).Bytes, byte(0))
+		(*f).BitLength = len((*f).Bytes) * 8
+	}
+	//Which byte?
+	b := i / 8
+	//Which bit in byte
+	p := uint(7 - (i - 8*b))
+	(*f).Bytes[b] = (*f).Bytes[b] &^ (1 << p)
+}
+
+// IsFlagSet tests if a flag is set in the ASN1 BitString.
+func IsFlagSet(f *asn1.BitString, i int) bool {
+	//Which byte?
+	b := i / 8
+	//Which bit in byte
+	p := uint(7 - (i - 8*b))
+	if (*f).Bytes[b]&(1<<p) != 0 {
+		return true
+	}
+	return false
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/PAData.go b/vendor/github.com/jcmturner/gokrb5/v8/types/PAData.go
new file mode 100644
index 0000000..41645ec
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/PAData.go
@@ -0,0 +1,155 @@
+package types
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.7
+import (
+	"fmt"
+	"time"
+
+	"github.com/jcmturner/gofork/encoding/asn1"
+	"github.com/jcmturner/gokrb5/v8/iana/patype"
+)
+
+// PAData implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7
+type PAData struct {
+	PADataType  int32  `asn1:"explicit,tag:1"`
+	PADataValue []byte `asn1:"explicit,tag:2"`
+}
+
+// PADataSequence implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7
+type PADataSequence []PAData
+
+// MethodData implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.9.1
+type MethodData []PAData
+
+// PAEncTimestamp implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.2
+type PAEncTimestamp EncryptedData
+
+// PAEncTSEnc implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.2
+type PAEncTSEnc struct {
+	PATimestamp time.Time `asn1:"generalized,explicit,tag:0"`
+	PAUSec      int       `asn1:"explicit,optional,tag:1"`
+}
+
+// Contains tests if a PADataSequence contains PA Data of a certain type.
+func (pas *PADataSequence) Contains(patype int32) bool {
+	for _, pa := range *pas {
+		if pa.PADataType == patype {
+			return true
+		}
+	}
+	return false
+}
+
+// GetPAEncTSEncAsnMarshalled returns the bytes of a PAEncTSEnc.
+func GetPAEncTSEncAsnMarshalled() ([]byte, error) {
+	t := time.Now().UTC()
+	p := PAEncTSEnc{
+		PATimestamp: t,
+		PAUSec:      int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
+	}
+	b, err := asn1.Marshal(p)
+	if err != nil {
+		return b, fmt.Errorf("error mashaling PAEncTSEnc: %v", err)
+	}
+	return b, nil
+}
+
+// ETypeInfoEntry implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.4
+type ETypeInfoEntry struct {
+	EType int32  `asn1:"explicit,tag:0"`
+	Salt  []byte `asn1:"explicit,optional,tag:1"`
+}
+
+// ETypeInfo implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.4
+type ETypeInfo []ETypeInfoEntry
+
+// ETypeInfo2Entry implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.5
+type ETypeInfo2Entry struct {
+	EType     int32  `asn1:"explicit,tag:0"`
+	Salt      string `asn1:"explicit,optional,generalstring,tag:1"`
+	S2KParams []byte `asn1:"explicit,optional,tag:2"`
+}
+
+// ETypeInfo2 implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.5
+type ETypeInfo2 []ETypeInfo2Entry
+
+// PAReqEncPARep PA Data Type
+type PAReqEncPARep struct {
+	ChksumType int32  `asn1:"explicit,tag:0"`
+	Chksum     []byte `asn1:"explicit,tag:1"`
+}
+
+// Unmarshal bytes into the PAData
+func (pa *PAData) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, pa)
+	return err
+}
+
+// Unmarshal bytes into the PADataSequence
+func (pas *PADataSequence) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, pas)
+	return err
+}
+
+// Unmarshal bytes into the PAReqEncPARep
+func (pa *PAReqEncPARep) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, pa)
+	return err
+}
+
+// Unmarshal bytes into the PAEncTimestamp
+func (pa *PAEncTimestamp) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, pa)
+	return err
+}
+
+// Unmarshal bytes into the PAEncTSEnc
+func (pa *PAEncTSEnc) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, pa)
+	return err
+}
+
+// Unmarshal bytes into the ETypeInfo
+func (a *ETypeInfo) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the ETypeInfoEntry
+func (a *ETypeInfoEntry) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the ETypeInfo2
+func (a *ETypeInfo2) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// Unmarshal bytes into the ETypeInfo2Entry
+func (a *ETypeInfo2Entry) Unmarshal(b []byte) error {
+	_, err := asn1.Unmarshal(b, a)
+	return err
+}
+
+// GetETypeInfo returns an ETypeInfo from the PAData.
+func (pa *PAData) GetETypeInfo() (d ETypeInfo, err error) {
+	if pa.PADataType != patype.PA_ETYPE_INFO {
+		err = fmt.Errorf("PAData does not contain PA EType Info data. TypeID Expected: %v; Actual: %v", patype.PA_ETYPE_INFO, pa.PADataType)
+		return
+	}
+	_, err = asn1.Unmarshal(pa.PADataValue, &d)
+	return
+}
+
+// GetETypeInfo2 returns an ETypeInfo2 from the PAData.
+func (pa *PAData) GetETypeInfo2() (d ETypeInfo2, err error) {
+	if pa.PADataType != patype.PA_ETYPE_INFO2 {
+		err = fmt.Errorf("PAData does not contain PA EType Info 2 data. TypeID Expected: %v; Actual: %v", patype.PA_ETYPE_INFO2, pa.PADataType)
+		return
+	}
+	_, err = asn1.Unmarshal(pa.PADataValue, &d)
+	return
+}
diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/PrincipalName.go b/vendor/github.com/jcmturner/gokrb5/v8/types/PrincipalName.go
new file mode 100644
index 0000000..d48d2c1
--- /dev/null
+++ b/vendor/github.com/jcmturner/gokrb5/v8/types/PrincipalName.go
@@ -0,0 +1,67 @@
+package types
+
+import (
+	"strings"
+
+	"github.com/jcmturner/gokrb5/v8/iana/nametype"
+)
+
+// Reference: https://www.ietf.org/rfc/rfc4120.txt
+// Section: 5.2.2
+
+// PrincipalName implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.2
+type PrincipalName struct {
+	NameType   int32    `asn1:"explicit,tag:0"`
+	NameString []string `asn1:"generalstring,explicit,tag:1"`
+}
+
+// NewPrincipalName creates a new PrincipalName from the name type int32 and name string provided.
+func NewPrincipalName(ntype int32, spn string) PrincipalName {
+	return PrincipalName{
+		NameType:   ntype,
+		NameString: strings.Split(spn, "/"),
+	}
+}
+
+// GetSalt returns a salt derived from the PrincipalName.
+func (pn PrincipalName) GetSalt(realm string) string {
+	var sb []byte
+	sb = append(sb, realm...)
+	for _, n := range pn.NameString {
+		sb = append(sb, n...)
+	}
+	return string(sb)
+}
+
+// Equal tests if the PrincipalName is equal to the one provided.
+func (pn PrincipalName) Equal(n PrincipalName) bool {
+	if len(pn.NameString) != len(n.NameString) {
+		return false
+	}
+	//https://tools.ietf.org/html/rfc4120#section-6.2 - the name type is not significant when checking for equivalence
+	for i, s := range pn.NameString {
+		if n.NameString[i] != s {
+			return false
+		}
+	}
+	return true
+}
+
+// PrincipalNameString returns the PrincipalName in string form.
+func (pn PrincipalName) PrincipalNameString() string {
+	return strings.Join(pn.NameString, "/")
+}
+
+// ParseSPNString will parse a string in the format <service>/<name>@<realm>
+// a PrincipalName type will be returned with the name type set to KRB_NT_PRINCIPAL(1)
+// and the realm will be returned as a string. If the "@<realm>" suffix
+// is not included in the SPN then the value of realm string returned will be ""
+func ParseSPNString(spn string) (pn PrincipalName, realm string) {
+	if strings.Contains(spn, "@") {
+		s := strings.Split(spn, "@")
+		realm = s[len(s)-1]
+		spn = strings.TrimSuffix(spn, "@"+realm)
+	}
+	pn = NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn)
+	return
+}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/TypedData.go b/vendor/github.com/jcmturner/gokrb5/v8/types/TypedData.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/gokrb5.v7/types/TypedData.go
rename to vendor/github.com/jcmturner/gokrb5/v8/types/TypedData.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/LICENSE b/vendor/github.com/jcmturner/rpc/v2/LICENSE
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/LICENSE
rename to vendor/github.com/jcmturner/rpc/v2/LICENSE
diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go
new file mode 100644
index 0000000..b9f535f
--- /dev/null
+++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go
@@ -0,0 +1,152 @@
+package mstypes
+
+import (
+	"bytes"
+	"encoding/hex"
+	"errors"
+	"fmt"
+
+	"github.com/jcmturner/rpc/v2/ndr"
+	"golang.org/x/net/http2/hpack"
+)
+
+// Compression format assigned numbers. https://docs.microsoft.com/en-us/openspecs/windows_protocols/ms-xca/a8b7cb0a-92a6-4187-a23b-5e14273b96f8
+const (
+	CompressionFormatNone       uint16 = 0
+	CompressionFormatLZNT1      uint16 = 2 // LZNT1 aka ntfs compression
+	CompressionFormatXPress     uint16 = 3 // plain LZ77
+	CompressionFormatXPressHuff uint16 = 4 // LZ77+Huffman - The Huffman variant of the XPRESS compression format uses LZ77-style dictionary compression combined with Huffman coding.
+)
+
+// ClaimsSourceTypeAD https://msdn.microsoft.com/en-us/library/hh553809.aspx
+const ClaimsSourceTypeAD uint16 = 1
+
+// Claim Type assigned numbers
+const (
+	ClaimTypeIDInt64    uint16 = 1
+	ClaimTypeIDUInt64   uint16 = 2
+	ClaimTypeIDString   uint16 = 3
+	ClaimsTypeIDBoolean uint16 = 6
+)
+
+// ClaimsBlob implements https://msdn.microsoft.com/en-us/library/hh554119.aspx
+type ClaimsBlob struct {
+	Size        uint32
+	EncodedBlob EncodedBlob
+}
+
+// EncodedBlob are the bytes of the encoded Claims
+type EncodedBlob []byte
+
+// Size returns the size of the bytes of the encoded Claims
+func (b EncodedBlob) Size(c interface{}) int {
+	cb := c.(ClaimsBlob)
+	return int(cb.Size)
+}
+
+// ClaimsSetMetadata implements https://msdn.microsoft.com/en-us/library/hh554073.aspx
+type ClaimsSetMetadata struct {
+	ClaimsSetSize             uint32
+	ClaimsSetBytes            []byte `ndr:"pointer,conformant"`
+	CompressionFormat         uint16 // Enum see constants for options
+	UncompressedClaimsSetSize uint32
+	ReservedType              uint16
+	ReservedFieldSize         uint32
+	ReservedField             []byte `ndr:"pointer,conformant"`
+}
+
+// ClaimsSet reads the ClaimsSet type from the NDR encoded ClaimsSetBytes in the ClaimsSetMetadata
+func (m *ClaimsSetMetadata) ClaimsSet() (c ClaimsSet, err error) {
+	if len(m.ClaimsSetBytes) < 1 {
+		err = errors.New("no bytes available for ClaimsSet")
+		return
+	}
+	// TODO switch statement to decompress ClaimsSetBytes
+	switch m.CompressionFormat {
+	case CompressionFormatLZNT1:
+		s := hex.EncodeToString(m.ClaimsSetBytes)
+		err = fmt.Errorf("ClaimsSet compressed, format LZNT1 not currently supported: %s", s)
+		return
+	case CompressionFormatXPress:
+		s := hex.EncodeToString(m.ClaimsSetBytes)
+		err = fmt.Errorf("ClaimsSet compressed, format XPress not currently supported: %s", s)
+		return
+	case CompressionFormatXPressHuff:
+		var b []byte
+		buff := bytes.NewBuffer(b)
+		_, e := hpack.HuffmanDecode(buff, m.ClaimsSetBytes)
+		if e != nil {
+			err = fmt.Errorf("error deflating: %v", e)
+			return
+		}
+		m.ClaimsSetBytes = buff.Bytes()
+	}
+	dec := ndr.NewDecoder(bytes.NewReader(m.ClaimsSetBytes))
+	err = dec.Decode(&c)
+	return
+}
+
+// ClaimsSet implements https://msdn.microsoft.com/en-us/library/hh554122.aspx
+type ClaimsSet struct {
+	ClaimsArrayCount  uint32
+	ClaimsArrays      []ClaimsArray `ndr:"pointer,conformant"`
+	ReservedType      uint16
+	ReservedFieldSize uint32
+	ReservedField     []byte `ndr:"pointer,conformant"`
+}
+
+// ClaimsArray implements https://msdn.microsoft.com/en-us/library/hh536458.aspx
+type ClaimsArray struct {
+	ClaimsSourceType uint16
+	ClaimsCount      uint32
+	ClaimEntries     []ClaimEntry `ndr:"pointer,conformant"`
+}
+
+// ClaimEntry is a NDR union that implements https://msdn.microsoft.com/en-us/library/hh536374.aspx
+type ClaimEntry struct {
+	ID         string           `ndr:"pointer,conformant,varying"`
+	Type       uint16           `ndr:"unionTag"`
+	TypeInt64  ClaimTypeInt64   `ndr:"unionField"`
+	TypeUInt64 ClaimTypeUInt64  `ndr:"unionField"`
+	TypeString ClaimTypeString  `ndr:"unionField"`
+	TypeBool   ClaimTypeBoolean `ndr:"unionField"`
+}
+
+// SwitchFunc is the ClaimEntry union field selection function
+func (u ClaimEntry) SwitchFunc(_ interface{}) string {
+	switch u.Type {
+	case ClaimTypeIDInt64:
+		return "TypeInt64"
+	case ClaimTypeIDUInt64:
+		return "TypeUInt64"
+	case ClaimTypeIDString:
+		return "TypeString"
+	case ClaimsTypeIDBoolean:
+		return "TypeBool"
+	}
+	return ""
+}
+
+// ClaimTypeInt64 is a claim of type int64
+type ClaimTypeInt64 struct {
+	ValueCount uint32
+	Value      []int64 `ndr:"pointer,conformant"`
+}
+
+// ClaimTypeUInt64 is a claim of type uint64
+type ClaimTypeUInt64 struct {
+	ValueCount uint32
+	Value      []uint64 `ndr:"pointer,conformant"`
+}
+
+// ClaimTypeString is a claim of type string
+type ClaimTypeString struct {
+	ValueCount uint32
+	Value      []LPWSTR `ndr:"pointer,conformant"`
+}
+
+// ClaimTypeBoolean is a claim of type bool
+type ClaimTypeBoolean struct {
+	ValueCount uint32
+	Value      []bool `ndr:"pointer,conformant"`
+}
diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go
new file mode 100644
index 0000000..fb6510d
--- /dev/null
+++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go
@@ -0,0 +1,12 @@
+// Package mstypes provides implemnations of some Microsoft data types [MS-DTYP] https://msdn.microsoft.com/en-us/library/cc230283.aspx
+package mstypes
+
+// LPWSTR implements https://msdn.microsoft.com/en-us/library/cc230355.aspx
+type LPWSTR struct {
+	Value string `ndr:"pointer,conformant,varying"`
+}
+
+// String returns the string representation of LPWSTR data type.
+func (s *LPWSTR) String() string {
+	return s.Value
+}
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/filetime.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/filetime.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/filetime.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/filetime.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/group_membership.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/group_membership.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/group_membership.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/group_membership.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/kerb_sid_and_attributes.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/kerb_sid_and_attributes.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/kerb_sid_and_attributes.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/kerb_sid_and_attributes.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/reader.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/reader.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/reader.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/reader.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/rpc_unicode_string.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/rpc_unicode_string.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/rpc_unicode_string.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/rpc_unicode_string.go
diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go
new file mode 100644
index 0000000..8e34705
--- /dev/null
+++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go
@@ -0,0 +1,36 @@
+package mstypes
+
+import (
+	"encoding/binary"
+	"encoding/hex"
+	"fmt"
+	"math"
+	"strings"
+)
+
+// RPCSID implements https://msdn.microsoft.com/en-us/library/cc230364.aspx
+type RPCSID struct {
+	Revision            uint8    // An 8-bit unsigned integer that specifies the revision level of the SID. This value MUST be set to 0x01.
+	SubAuthorityCount   uint8    // An 8-bit unsigned integer that specifies the number of elements in the SubAuthority array. The maximum number of elements allowed is 15.
+	IdentifierAuthority [6]byte  // An RPC_SID_IDENTIFIER_AUTHORITY structure that indicates the authority under which the SID was created. It describes the entity that created the SID. The Identifier Authority value {0,0,0,0,0,5} denotes SIDs created by the NT SID authority.
+	SubAuthority        []uint32 `ndr:"conformant"` // A variable length array of unsigned 32-bit integers that uniquely identifies a principal relative to the IdentifierAuthority. Its length is determined by SubAuthorityCount.
+}
+
+// String returns the string representation of the RPC_SID.
+func (s *RPCSID) String() string {
+	var strb strings.Builder
+	strb.WriteString("S-1-")
+
+	b := append(make([]byte, 2, 2), s.IdentifierAuthority[:]...)
+	// For a strange reason this is read big endian: https://msdn.microsoft.com/en-us/library/dd302645.aspx
+	i := binary.BigEndian.Uint64(b)
+	if i > math.MaxUint32 {
+		fmt.Fprintf(&strb, "0x%s", hex.EncodeToString(s.IdentifierAuthority[:]))
+	} else {
+		fmt.Fprintf(&strb, "%d", i)
+	}
+	for _, sub := range s.SubAuthority {
+		fmt.Fprintf(&strb, "-%d", sub)
+	}
+	return strb.String()
+}
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/user_session_key.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/user_session_key.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/mstypes/user_session_key.go
rename to vendor/github.com/jcmturner/rpc/v2/mstypes/user_session_key.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/arrays.go b/vendor/github.com/jcmturner/rpc/v2/ndr/arrays.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/arrays.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/arrays.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/decoder.go b/vendor/github.com/jcmturner/rpc/v2/ndr/decoder.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/decoder.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/decoder.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/error.go b/vendor/github.com/jcmturner/rpc/v2/ndr/error.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/error.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/error.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/header.go b/vendor/github.com/jcmturner/rpc/v2/ndr/header.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/header.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/header.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/pipe.go b/vendor/github.com/jcmturner/rpc/v2/ndr/pipe.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/pipe.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/pipe.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/primitives.go b/vendor/github.com/jcmturner/rpc/v2/ndr/primitives.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/primitives.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/primitives.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/rawbytes.go b/vendor/github.com/jcmturner/rpc/v2/ndr/rawbytes.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/rawbytes.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/rawbytes.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/strings.go b/vendor/github.com/jcmturner/rpc/v2/ndr/strings.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/strings.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/strings.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/tags.go b/vendor/github.com/jcmturner/rpc/v2/ndr/tags.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/tags.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/tags.go
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/ndr/union.go b/vendor/github.com/jcmturner/rpc/v2/ndr/union.go
similarity index 100%
rename from vendor/gopkg.in/jcmturner/rpc.v1/ndr/union.go
rename to vendor/github.com/jcmturner/rpc/v2/ndr/union.go
diff --git a/vendor/github.com/klauspost/compress/LICENSE b/vendor/github.com/klauspost/compress/LICENSE
new file mode 100644
index 0000000..1eb75ef
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/LICENSE
@@ -0,0 +1,28 @@
+Copyright (c) 2012 The Go Authors. All rights reserved.
+Copyright (c) 2019 Klaus Post. 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/klauspost/compress/fse/README.md b/vendor/github.com/klauspost/compress/fse/README.md
new file mode 100644
index 0000000..ea7324d
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/README.md
@@ -0,0 +1,79 @@
+# Finite State Entropy

+

+This package provides Finite State Entropy encoding and decoding.

+            

+Finite State Entropy (also referenced as [tANS](https://en.wikipedia.org/wiki/Asymmetric_numeral_systems#tANS)) 

+encoding provides a fast near-optimal symbol encoding/decoding

+for byte blocks as implemented in [zstandard](https://github.com/facebook/zstd).

+

+This can be used for compressing input with a lot of similar input values to the smallest number of bytes.

+This does not perform any multi-byte [dictionary coding](https://en.wikipedia.org/wiki/Dictionary_coder) as LZ coders,

+but it can be used as a secondary step to compressors (like Snappy) that does not do entropy encoding. 

+

+* [Godoc documentation](https://godoc.org/github.com/klauspost/compress/fse)

+

+## News

+

+ * Feb 2018: First implementation released. Consider this beta software for now.

+

+# Usage

+

+This package provides a low level interface that allows to compress single independent blocks. 

+

+Each block is separate, and there is no built in integrity checks. 

+This means that the caller should keep track of block sizes and also do checksums if needed.  

+

+Compressing a block is done via the [`Compress`](https://godoc.org/github.com/klauspost/compress/fse#Compress) function.

+You must provide input and will receive the output and maybe an error.

+

+These error values can be returned:

+

+| Error               | Description                                                                 |

+|---------------------|-----------------------------------------------------------------------------|

+| `<nil>`             | Everything ok, output is returned                                           |

+| `ErrIncompressible` | Returned when input is judged to be too hard to compress                    |

+| `ErrUseRLE`         | Returned from the compressor when the input is a single byte value repeated |

+| `(error)`           | An internal error occurred.                                                 |

+

+As can be seen above there are errors that will be returned even under normal operation so it is important to handle these.

+

+To reduce allocations you can provide a [`Scratch`](https://godoc.org/github.com/klauspost/compress/fse#Scratch) object 

+that can be re-used for successive calls. Both compression and decompression accepts a `Scratch` object, and the same 

+object can be used for both.   

+

+Be aware, that when re-using a `Scratch` object that the *output* buffer is also re-used, so if you are still using this

+you must set the `Out` field in the scratch to nil. The same buffer is used for compression and decompression output.

+

+Decompressing is done by calling the [`Decompress`](https://godoc.org/github.com/klauspost/compress/fse#Decompress) function.

+You must provide the output from the compression stage, at exactly the size you got back. If you receive an error back

+your input was likely corrupted. 

+

+It is important to note that a successful decoding does *not* mean your output matches your original input. 

+There are no integrity checks, so relying on errors from the decompressor does not assure your data is valid.

+

+For more detailed usage, see examples in the [godoc documentation](https://godoc.org/github.com/klauspost/compress/fse#pkg-examples).

+

+# Performance

+

+A lot of factors are affecting speed. Block sizes and compressibility of the material are primary factors.  

+All compression functions are currently only running on the calling goroutine so only one core will be used per block.  

+

+The compressor is significantly faster if symbols are kept as small as possible. The highest byte value of the input

+is used to reduce some of the processing, so if all your input is above byte value 64 for instance, it may be 

+beneficial to transpose all your input values down by 64.   

+

+With moderate block sizes around 64k speed are typically 200MB/s per core for compression and 

+around 300MB/s decompression speed. 

+

+The same hardware typically does Huffman (deflate) encoding at 125MB/s and decompression at 100MB/s. 

+

+# Plans

+

+At one point, more internals will be exposed to facilitate more "expert" usage of the components. 

+

+A streaming interface is also likely to be implemented. Likely compatible with [FSE stream format](https://github.com/Cyan4973/FiniteStateEntropy/blob/dev/programs/fileio.c#L261).  

+

+# Contributing

+

+Contributions are always welcome. Be aware that adding public functions will require good justification and breaking 

+changes will likely not be accepted. If in doubt open an issue before writing the PR.  
\ No newline at end of file
diff --git a/vendor/github.com/klauspost/compress/fse/bitreader.go b/vendor/github.com/klauspost/compress/fse/bitreader.go
new file mode 100644
index 0000000..f65eb39
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/bitreader.go
@@ -0,0 +1,122 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package fse
+
+import (
+	"encoding/binary"
+	"errors"
+	"io"
+)
+
+// bitReader reads a bitstream in reverse.
+// The last set bit indicates the start of the stream and is used
+// for aligning the input.
+type bitReader struct {
+	in       []byte
+	off      uint // next byte to read is at in[off - 1]
+	value    uint64
+	bitsRead uint8
+}
+
+// init initializes and resets the bit reader.
+func (b *bitReader) init(in []byte) error {
+	if len(in) < 1 {
+		return errors.New("corrupt stream: too short")
+	}
+	b.in = in
+	b.off = uint(len(in))
+	// The highest bit of the last byte indicates where to start
+	v := in[len(in)-1]
+	if v == 0 {
+		return errors.New("corrupt stream, did not find end of stream")
+	}
+	b.bitsRead = 64
+	b.value = 0
+	if len(in) >= 8 {
+		b.fillFastStart()
+	} else {
+		b.fill()
+		b.fill()
+	}
+	b.bitsRead += 8 - uint8(highBits(uint32(v)))
+	return nil
+}
+
+// getBits will return n bits. n can be 0.
+func (b *bitReader) getBits(n uint8) uint16 {
+	if n == 0 || b.bitsRead >= 64 {
+		return 0
+	}
+	return b.getBitsFast(n)
+}
+
+// getBitsFast requires that at least one bit is requested every time.
+// There are no checks if the buffer is filled.
+func (b *bitReader) getBitsFast(n uint8) uint16 {
+	const regMask = 64 - 1
+	v := uint16((b.value << (b.bitsRead & regMask)) >> ((regMask + 1 - n) & regMask))
+	b.bitsRead += n
+	return v
+}
+
+// fillFast() will make sure at least 32 bits are available.
+// There must be at least 4 bytes available.
+func (b *bitReader) fillFast() {
+	if b.bitsRead < 32 {
+		return
+	}
+	// 2 bounds checks.
+	v := b.in[b.off-4:]
+	v = v[:4]
+	low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+	b.value = (b.value << 32) | uint64(low)
+	b.bitsRead -= 32
+	b.off -= 4
+}
+
+// fill() will make sure at least 32 bits are available.
+func (b *bitReader) fill() {
+	if b.bitsRead < 32 {
+		return
+	}
+	if b.off > 4 {
+		v := b.in[b.off-4:]
+		v = v[:4]
+		low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+		b.value = (b.value << 32) | uint64(low)
+		b.bitsRead -= 32
+		b.off -= 4
+		return
+	}
+	for b.off > 0 {
+		b.value = (b.value << 8) | uint64(b.in[b.off-1])
+		b.bitsRead -= 8
+		b.off--
+	}
+}
+
+// fillFastStart() assumes the bitreader is empty and there is at least 8 bytes to read.
+func (b *bitReader) fillFastStart() {
+	// Do single re-slice to avoid bounds checks.
+	b.value = binary.LittleEndian.Uint64(b.in[b.off-8:])
+	b.bitsRead = 0
+	b.off -= 8
+}
+
+// finished returns true if all bits have been read from the bit stream.
+func (b *bitReader) finished() bool {
+	return b.bitsRead >= 64 && b.off == 0
+}
+
+// close the bitstream and returns an error if out-of-buffer reads occurred.
+func (b *bitReader) close() error {
+	// Release reference.
+	b.in = nil
+	if b.bitsRead > 64 {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
diff --git a/vendor/github.com/klauspost/compress/fse/bitwriter.go b/vendor/github.com/klauspost/compress/fse/bitwriter.go
new file mode 100644
index 0000000..43e4636
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/bitwriter.go
@@ -0,0 +1,168 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package fse
+
+import "fmt"
+
+// bitWriter will write bits.
+// First bit will be LSB of the first byte of output.
+type bitWriter struct {
+	bitContainer uint64
+	nBits        uint8
+	out          []byte
+}
+
+// bitMask16 is bitmasks. Has extra to avoid bounds check.
+var bitMask16 = [32]uint16{
+	0, 1, 3, 7, 0xF, 0x1F,
+	0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF,
+	0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF} /* up to 16 bits */
+
+// addBits16NC will add up to 16 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16NC(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value&bitMask16[bits&31]) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// addBits16Clean will add up to 16 bits. value may not contain more set bits than indicated.
+// It will not check if there is space for them, so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16Clean(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// addBits16ZeroNC will add up to 16 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+// This is fastest if bits can be zero.
+func (b *bitWriter) addBits16ZeroNC(value uint16, bits uint8) {
+	if bits == 0 {
+		return
+	}
+	value <<= (16 - bits) & 15
+	value >>= (16 - bits) & 15
+	b.bitContainer |= uint64(value) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// flush will flush all pending full bytes.
+// There will be at least 56 bits available for writing when this has been called.
+// Using flush32 is faster, but leaves less space for writing.
+func (b *bitWriter) flush() {
+	v := b.nBits >> 3
+	switch v {
+	case 0:
+	case 1:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+		)
+	case 2:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+		)
+	case 3:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+		)
+	case 4:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+		)
+	case 5:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+		)
+	case 6:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+		)
+	case 7:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+		)
+	case 8:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+			byte(b.bitContainer>>56),
+		)
+	default:
+		panic(fmt.Errorf("bits (%d) > 64", b.nBits))
+	}
+	b.bitContainer >>= v << 3
+	b.nBits &= 7
+}
+
+// flush32 will flush out, so there are at least 32 bits available for writing.
+func (b *bitWriter) flush32() {
+	if b.nBits < 32 {
+		return
+	}
+	b.out = append(b.out,
+		byte(b.bitContainer),
+		byte(b.bitContainer>>8),
+		byte(b.bitContainer>>16),
+		byte(b.bitContainer>>24))
+	b.nBits -= 32
+	b.bitContainer >>= 32
+}
+
+// flushAlign will flush remaining full bytes and align to next byte boundary.
+func (b *bitWriter) flushAlign() {
+	nbBytes := (b.nBits + 7) >> 3
+	for i := uint8(0); i < nbBytes; i++ {
+		b.out = append(b.out, byte(b.bitContainer>>(i*8)))
+	}
+	b.nBits = 0
+	b.bitContainer = 0
+}
+
+// close will write the alignment bit and write the final byte(s)
+// to the output.
+func (b *bitWriter) close() error {
+	// End mark
+	b.addBits16Clean(1, 1)
+	// flush until next byte.
+	b.flushAlign()
+	return nil
+}
+
+// reset and continue writing by appending to out.
+func (b *bitWriter) reset(out []byte) {
+	b.bitContainer = 0
+	b.nBits = 0
+	b.out = out
+}
diff --git a/vendor/github.com/klauspost/compress/fse/bytereader.go b/vendor/github.com/klauspost/compress/fse/bytereader.go
new file mode 100644
index 0000000..abade2d
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/bytereader.go
@@ -0,0 +1,47 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package fse
+
+// byteReader provides a byte reader that reads
+// little endian values from a byte stream.
+// The input stream is manually advanced.
+// The reader performs no bounds checks.
+type byteReader struct {
+	b   []byte
+	off int
+}
+
+// init will initialize the reader and set the input.
+func (b *byteReader) init(in []byte) {
+	b.b = in
+	b.off = 0
+}
+
+// advance the stream b n bytes.
+func (b *byteReader) advance(n uint) {
+	b.off += int(n)
+}
+
+// Uint32 returns a little endian uint32 starting at current offset.
+func (b byteReader) Uint32() uint32 {
+	b2 := b.b[b.off:]
+	b2 = b2[:4]
+	v3 := uint32(b2[3])
+	v2 := uint32(b2[2])
+	v1 := uint32(b2[1])
+	v0 := uint32(b2[0])
+	return v0 | (v1 << 8) | (v2 << 16) | (v3 << 24)
+}
+
+// unread returns the unread portion of the input.
+func (b byteReader) unread() []byte {
+	return b.b[b.off:]
+}
+
+// remain will return the number of bytes remaining.
+func (b byteReader) remain() int {
+	return len(b.b) - b.off
+}
diff --git a/vendor/github.com/klauspost/compress/fse/compress.go b/vendor/github.com/klauspost/compress/fse/compress.go
new file mode 100644
index 0000000..6f34191
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/compress.go
@@ -0,0 +1,683 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package fse
+
+import (
+	"errors"
+	"fmt"
+)
+
+// Compress the input bytes. Input must be < 2GB.
+// Provide a Scratch buffer to avoid memory allocations.
+// Note that the output is also kept in the scratch buffer.
+// If input is too hard to compress, ErrIncompressible is returned.
+// If input is a single byte value repeated ErrUseRLE is returned.
+func Compress(in []byte, s *Scratch) ([]byte, error) {
+	if len(in) <= 1 {
+		return nil, ErrIncompressible
+	}
+	if len(in) > (2<<30)-1 {
+		return nil, errors.New("input too big, must be < 2GB")
+	}
+	s, err := s.prepare(in)
+	if err != nil {
+		return nil, err
+	}
+
+	// Create histogram, if none was provided.
+	maxCount := s.maxCount
+	if maxCount == 0 {
+		maxCount = s.countSimple(in)
+	}
+	// Reset for next run.
+	s.clearCount = true
+	s.maxCount = 0
+	if maxCount == len(in) {
+		// One symbol, use RLE
+		return nil, ErrUseRLE
+	}
+	if maxCount == 1 || maxCount < (len(in)>>7) {
+		// Each symbol present maximum once or too well distributed.
+		return nil, ErrIncompressible
+	}
+	s.optimalTableLog()
+	err = s.normalizeCount()
+	if err != nil {
+		return nil, err
+	}
+	err = s.writeCount()
+	if err != nil {
+		return nil, err
+	}
+
+	if false {
+		err = s.validateNorm()
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	err = s.buildCTable()
+	if err != nil {
+		return nil, err
+	}
+	err = s.compress(in)
+	if err != nil {
+		return nil, err
+	}
+	s.Out = s.bw.out
+	// Check if we compressed.
+	if len(s.Out) >= len(in) {
+		return nil, ErrIncompressible
+	}
+	return s.Out, nil
+}
+
+// cState contains the compression state of a stream.
+type cState struct {
+	bw         *bitWriter
+	stateTable []uint16
+	state      uint16
+}
+
+// init will initialize the compression state to the first symbol of the stream.
+func (c *cState) init(bw *bitWriter, ct *cTable, tableLog uint8, first symbolTransform) {
+	c.bw = bw
+	c.stateTable = ct.stateTable
+
+	nbBitsOut := (first.deltaNbBits + (1 << 15)) >> 16
+	im := int32((nbBitsOut << 16) - first.deltaNbBits)
+	lu := (im >> nbBitsOut) + first.deltaFindState
+	c.state = c.stateTable[lu]
+}
+
+// encode the output symbol provided and write it to the bitstream.
+func (c *cState) encode(symbolTT symbolTransform) {
+	nbBitsOut := (uint32(c.state) + symbolTT.deltaNbBits) >> 16
+	dstState := int32(c.state>>(nbBitsOut&15)) + symbolTT.deltaFindState
+	c.bw.addBits16NC(c.state, uint8(nbBitsOut))
+	c.state = c.stateTable[dstState]
+}
+
+// encode the output symbol provided and write it to the bitstream.
+func (c *cState) encodeZero(symbolTT symbolTransform) {
+	nbBitsOut := (uint32(c.state) + symbolTT.deltaNbBits) >> 16
+	dstState := int32(c.state>>(nbBitsOut&15)) + symbolTT.deltaFindState
+	c.bw.addBits16ZeroNC(c.state, uint8(nbBitsOut))
+	c.state = c.stateTable[dstState]
+}
+
+// flush will write the tablelog to the output and flush the remaining full bytes.
+func (c *cState) flush(tableLog uint8) {
+	c.bw.flush32()
+	c.bw.addBits16NC(c.state, tableLog)
+	c.bw.flush()
+}
+
+// compress is the main compression loop that will encode the input from the last byte to the first.
+func (s *Scratch) compress(src []byte) error {
+	if len(src) <= 2 {
+		return errors.New("compress: src too small")
+	}
+	tt := s.ct.symbolTT[:256]
+	s.bw.reset(s.Out)
+
+	// Our two states each encodes every second byte.
+	// Last byte encoded (first byte decoded) will always be encoded by c1.
+	var c1, c2 cState
+
+	// Encode so remaining size is divisible by 4.
+	ip := len(src)
+	if ip&1 == 1 {
+		c1.init(&s.bw, &s.ct, s.actualTableLog, tt[src[ip-1]])
+		c2.init(&s.bw, &s.ct, s.actualTableLog, tt[src[ip-2]])
+		c1.encodeZero(tt[src[ip-3]])
+		ip -= 3
+	} else {
+		c2.init(&s.bw, &s.ct, s.actualTableLog, tt[src[ip-1]])
+		c1.init(&s.bw, &s.ct, s.actualTableLog, tt[src[ip-2]])
+		ip -= 2
+	}
+	if ip&2 != 0 {
+		c2.encodeZero(tt[src[ip-1]])
+		c1.encodeZero(tt[src[ip-2]])
+		ip -= 2
+	}
+
+	// Main compression loop.
+	switch {
+	case !s.zeroBits && s.actualTableLog <= 8:
+		// We can encode 4 symbols without requiring a flush.
+		// We do not need to check if any output is 0 bits.
+		for ip >= 4 {
+			s.bw.flush32()
+			v3, v2, v1, v0 := src[ip-4], src[ip-3], src[ip-2], src[ip-1]
+			c2.encode(tt[v0])
+			c1.encode(tt[v1])
+			c2.encode(tt[v2])
+			c1.encode(tt[v3])
+			ip -= 4
+		}
+	case !s.zeroBits:
+		// We do not need to check if any output is 0 bits.
+		for ip >= 4 {
+			s.bw.flush32()
+			v3, v2, v1, v0 := src[ip-4], src[ip-3], src[ip-2], src[ip-1]
+			c2.encode(tt[v0])
+			c1.encode(tt[v1])
+			s.bw.flush32()
+			c2.encode(tt[v2])
+			c1.encode(tt[v3])
+			ip -= 4
+		}
+	case s.actualTableLog <= 8:
+		// We can encode 4 symbols without requiring a flush
+		for ip >= 4 {
+			s.bw.flush32()
+			v3, v2, v1, v0 := src[ip-4], src[ip-3], src[ip-2], src[ip-1]
+			c2.encodeZero(tt[v0])
+			c1.encodeZero(tt[v1])
+			c2.encodeZero(tt[v2])
+			c1.encodeZero(tt[v3])
+			ip -= 4
+		}
+	default:
+		for ip >= 4 {
+			s.bw.flush32()
+			v3, v2, v1, v0 := src[ip-4], src[ip-3], src[ip-2], src[ip-1]
+			c2.encodeZero(tt[v0])
+			c1.encodeZero(tt[v1])
+			s.bw.flush32()
+			c2.encodeZero(tt[v2])
+			c1.encodeZero(tt[v3])
+			ip -= 4
+		}
+	}
+
+	// Flush final state.
+	// Used to initialize state when decoding.
+	c2.flush(s.actualTableLog)
+	c1.flush(s.actualTableLog)
+
+	return s.bw.close()
+}
+
+// writeCount will write the normalized histogram count to header.
+// This is read back by readNCount.
+func (s *Scratch) writeCount() error {
+	var (
+		tableLog  = s.actualTableLog
+		tableSize = 1 << tableLog
+		previous0 bool
+		charnum   uint16
+
+		maxHeaderSize = ((int(s.symbolLen) * int(tableLog)) >> 3) + 3
+
+		// Write Table Size
+		bitStream = uint32(tableLog - minTablelog)
+		bitCount  = uint(4)
+		remaining = int16(tableSize + 1) /* +1 for extra accuracy */
+		threshold = int16(tableSize)
+		nbBits    = uint(tableLog + 1)
+	)
+	if cap(s.Out) < maxHeaderSize {
+		s.Out = make([]byte, 0, s.br.remain()+maxHeaderSize)
+	}
+	outP := uint(0)
+	out := s.Out[:maxHeaderSize]
+
+	// stops at 1
+	for remaining > 1 {
+		if previous0 {
+			start := charnum
+			for s.norm[charnum] == 0 {
+				charnum++
+			}
+			for charnum >= start+24 {
+				start += 24
+				bitStream += uint32(0xFFFF) << bitCount
+				out[outP] = byte(bitStream)
+				out[outP+1] = byte(bitStream >> 8)
+				outP += 2
+				bitStream >>= 16
+			}
+			for charnum >= start+3 {
+				start += 3
+				bitStream += 3 << bitCount
+				bitCount += 2
+			}
+			bitStream += uint32(charnum-start) << bitCount
+			bitCount += 2
+			if bitCount > 16 {
+				out[outP] = byte(bitStream)
+				out[outP+1] = byte(bitStream >> 8)
+				outP += 2
+				bitStream >>= 16
+				bitCount -= 16
+			}
+		}
+
+		count := s.norm[charnum]
+		charnum++
+		max := (2*threshold - 1) - remaining
+		if count < 0 {
+			remaining += count
+		} else {
+			remaining -= count
+		}
+		count++ // +1 for extra accuracy
+		if count >= threshold {
+			count += max // [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[
+		}
+		bitStream += uint32(count) << bitCount
+		bitCount += nbBits
+		if count < max {
+			bitCount--
+		}
+
+		previous0 = count == 1
+		if remaining < 1 {
+			return errors.New("internal error: remaining<1")
+		}
+		for remaining < threshold {
+			nbBits--
+			threshold >>= 1
+		}
+
+		if bitCount > 16 {
+			out[outP] = byte(bitStream)
+			out[outP+1] = byte(bitStream >> 8)
+			outP += 2
+			bitStream >>= 16
+			bitCount -= 16
+		}
+	}
+
+	out[outP] = byte(bitStream)
+	out[outP+1] = byte(bitStream >> 8)
+	outP += (bitCount + 7) / 8
+
+	if charnum > s.symbolLen {
+		return errors.New("internal error: charnum > s.symbolLen")
+	}
+	s.Out = out[:outP]
+	return nil
+}
+
+// symbolTransform contains the state transform for a symbol.
+type symbolTransform struct {
+	deltaFindState int32
+	deltaNbBits    uint32
+}
+
+// String prints values as a human readable string.
+func (s symbolTransform) String() string {
+	return fmt.Sprintf("dnbits: %08x, fs:%d", s.deltaNbBits, s.deltaFindState)
+}
+
+// cTable contains tables used for compression.
+type cTable struct {
+	tableSymbol []byte
+	stateTable  []uint16
+	symbolTT    []symbolTransform
+}
+
+// allocCtable will allocate tables needed for compression.
+// If existing tables a re big enough, they are simply re-used.
+func (s *Scratch) allocCtable() {
+	tableSize := 1 << s.actualTableLog
+	// get tableSymbol that is big enough.
+	if cap(s.ct.tableSymbol) < tableSize {
+		s.ct.tableSymbol = make([]byte, tableSize)
+	}
+	s.ct.tableSymbol = s.ct.tableSymbol[:tableSize]
+
+	ctSize := tableSize
+	if cap(s.ct.stateTable) < ctSize {
+		s.ct.stateTable = make([]uint16, ctSize)
+	}
+	s.ct.stateTable = s.ct.stateTable[:ctSize]
+
+	if cap(s.ct.symbolTT) < 256 {
+		s.ct.symbolTT = make([]symbolTransform, 256)
+	}
+	s.ct.symbolTT = s.ct.symbolTT[:256]
+}
+
+// buildCTable will populate the compression table so it is ready to be used.
+func (s *Scratch) buildCTable() error {
+	tableSize := uint32(1 << s.actualTableLog)
+	highThreshold := tableSize - 1
+	var cumul [maxSymbolValue + 2]int16
+
+	s.allocCtable()
+	tableSymbol := s.ct.tableSymbol[:tableSize]
+	// symbol start positions
+	{
+		cumul[0] = 0
+		for ui, v := range s.norm[:s.symbolLen-1] {
+			u := byte(ui) // one less than reference
+			if v == -1 {
+				// Low proba symbol
+				cumul[u+1] = cumul[u] + 1
+				tableSymbol[highThreshold] = u
+				highThreshold--
+			} else {
+				cumul[u+1] = cumul[u] + v
+			}
+		}
+		// Encode last symbol separately to avoid overflowing u
+		u := int(s.symbolLen - 1)
+		v := s.norm[s.symbolLen-1]
+		if v == -1 {
+			// Low proba symbol
+			cumul[u+1] = cumul[u] + 1
+			tableSymbol[highThreshold] = byte(u)
+			highThreshold--
+		} else {
+			cumul[u+1] = cumul[u] + v
+		}
+		if uint32(cumul[s.symbolLen]) != tableSize {
+			return fmt.Errorf("internal error: expected cumul[s.symbolLen] (%d) == tableSize (%d)", cumul[s.symbolLen], tableSize)
+		}
+		cumul[s.symbolLen] = int16(tableSize) + 1
+	}
+	// Spread symbols
+	s.zeroBits = false
+	{
+		step := tableStep(tableSize)
+		tableMask := tableSize - 1
+		var position uint32
+		// if any symbol > largeLimit, we may have 0 bits output.
+		largeLimit := int16(1 << (s.actualTableLog - 1))
+		for ui, v := range s.norm[:s.symbolLen] {
+			symbol := byte(ui)
+			if v > largeLimit {
+				s.zeroBits = true
+			}
+			for nbOccurrences := int16(0); nbOccurrences < v; nbOccurrences++ {
+				tableSymbol[position] = symbol
+				position = (position + step) & tableMask
+				for position > highThreshold {
+					position = (position + step) & tableMask
+				} /* Low proba area */
+			}
+		}
+
+		// Check if we have gone through all positions
+		if position != 0 {
+			return errors.New("position!=0")
+		}
+	}
+
+	// Build table
+	table := s.ct.stateTable
+	{
+		tsi := int(tableSize)
+		for u, v := range tableSymbol {
+			// TableU16 : sorted by symbol order; gives next state value
+			table[cumul[v]] = uint16(tsi + u)
+			cumul[v]++
+		}
+	}
+
+	// Build Symbol Transformation Table
+	{
+		total := int16(0)
+		symbolTT := s.ct.symbolTT[:s.symbolLen]
+		tableLog := s.actualTableLog
+		tl := (uint32(tableLog) << 16) - (1 << tableLog)
+		for i, v := range s.norm[:s.symbolLen] {
+			switch v {
+			case 0:
+			case -1, 1:
+				symbolTT[i].deltaNbBits = tl
+				symbolTT[i].deltaFindState = int32(total - 1)
+				total++
+			default:
+				maxBitsOut := uint32(tableLog) - highBits(uint32(v-1))
+				minStatePlus := uint32(v) << maxBitsOut
+				symbolTT[i].deltaNbBits = (maxBitsOut << 16) - minStatePlus
+				symbolTT[i].deltaFindState = int32(total - v)
+				total += v
+			}
+		}
+		if total != int16(tableSize) {
+			return fmt.Errorf("total mismatch %d (got) != %d (want)", total, tableSize)
+		}
+	}
+	return nil
+}
+
+// countSimple will create a simple histogram in s.count.
+// Returns the biggest count.
+// Does not update s.clearCount.
+func (s *Scratch) countSimple(in []byte) (max int) {
+	for _, v := range in {
+		s.count[v]++
+	}
+	m := uint32(0)
+	for i, v := range s.count[:] {
+		if v > m {
+			m = v
+		}
+		if v > 0 {
+			s.symbolLen = uint16(i) + 1
+		}
+	}
+	return int(m)
+}
+
+// minTableLog provides the minimum logSize to safely represent a distribution.
+func (s *Scratch) minTableLog() uint8 {
+	minBitsSrc := highBits(uint32(s.br.remain()-1)) + 1
+	minBitsSymbols := highBits(uint32(s.symbolLen-1)) + 2
+	if minBitsSrc < minBitsSymbols {
+		return uint8(minBitsSrc)
+	}
+	return uint8(minBitsSymbols)
+}
+
+// optimalTableLog calculates and sets the optimal tableLog in s.actualTableLog
+func (s *Scratch) optimalTableLog() {
+	tableLog := s.TableLog
+	minBits := s.minTableLog()
+	maxBitsSrc := uint8(highBits(uint32(s.br.remain()-1))) - 2
+	if maxBitsSrc < tableLog {
+		// Accuracy can be reduced
+		tableLog = maxBitsSrc
+	}
+	if minBits > tableLog {
+		tableLog = minBits
+	}
+	// Need a minimum to safely represent all symbol values
+	if tableLog < minTablelog {
+		tableLog = minTablelog
+	}
+	if tableLog > maxTableLog {
+		tableLog = maxTableLog
+	}
+	s.actualTableLog = tableLog
+}
+
+var rtbTable = [...]uint32{0, 473195, 504333, 520860, 550000, 700000, 750000, 830000}
+
+// normalizeCount will normalize the count of the symbols so
+// the total is equal to the table size.
+func (s *Scratch) normalizeCount() error {
+	var (
+		tableLog          = s.actualTableLog
+		scale             = 62 - uint64(tableLog)
+		step              = (1 << 62) / uint64(s.br.remain())
+		vStep             = uint64(1) << (scale - 20)
+		stillToDistribute = int16(1 << tableLog)
+		largest           int
+		largestP          int16
+		lowThreshold      = (uint32)(s.br.remain() >> tableLog)
+	)
+
+	for i, cnt := range s.count[:s.symbolLen] {
+		// already handled
+		// if (count[s] == s.length) return 0;   /* rle special case */
+
+		if cnt == 0 {
+			s.norm[i] = 0
+			continue
+		}
+		if cnt <= lowThreshold {
+			s.norm[i] = -1
+			stillToDistribute--
+		} else {
+			proba := (int16)((uint64(cnt) * step) >> scale)
+			if proba < 8 {
+				restToBeat := vStep * uint64(rtbTable[proba])
+				v := uint64(cnt)*step - (uint64(proba) << scale)
+				if v > restToBeat {
+					proba++
+				}
+			}
+			if proba > largestP {
+				largestP = proba
+				largest = i
+			}
+			s.norm[i] = proba
+			stillToDistribute -= proba
+		}
+	}
+
+	if -stillToDistribute >= (s.norm[largest] >> 1) {
+		// corner case, need another normalization method
+		return s.normalizeCount2()
+	}
+	s.norm[largest] += stillToDistribute
+	return nil
+}
+
+// Secondary normalization method.
+// To be used when primary method fails.
+func (s *Scratch) normalizeCount2() error {
+	const notYetAssigned = -2
+	var (
+		distributed  uint32
+		total        = uint32(s.br.remain())
+		tableLog     = s.actualTableLog
+		lowThreshold = total >> tableLog
+		lowOne       = (total * 3) >> (tableLog + 1)
+	)
+	for i, cnt := range s.count[:s.symbolLen] {
+		if cnt == 0 {
+			s.norm[i] = 0
+			continue
+		}
+		if cnt <= lowThreshold {
+			s.norm[i] = -1
+			distributed++
+			total -= cnt
+			continue
+		}
+		if cnt <= lowOne {
+			s.norm[i] = 1
+			distributed++
+			total -= cnt
+			continue
+		}
+		s.norm[i] = notYetAssigned
+	}
+	toDistribute := (1 << tableLog) - distributed
+
+	if (total / toDistribute) > lowOne {
+		// risk of rounding to zero
+		lowOne = (total * 3) / (toDistribute * 2)
+		for i, cnt := range s.count[:s.symbolLen] {
+			if (s.norm[i] == notYetAssigned) && (cnt <= lowOne) {
+				s.norm[i] = 1
+				distributed++
+				total -= cnt
+				continue
+			}
+		}
+		toDistribute = (1 << tableLog) - distributed
+	}
+	if distributed == uint32(s.symbolLen)+1 {
+		// all values are pretty poor;
+		//   probably incompressible data (should have already been detected);
+		//   find max, then give all remaining points to max
+		var maxV int
+		var maxC uint32
+		for i, cnt := range s.count[:s.symbolLen] {
+			if cnt > maxC {
+				maxV = i
+				maxC = cnt
+			}
+		}
+		s.norm[maxV] += int16(toDistribute)
+		return nil
+	}
+
+	if total == 0 {
+		// all of the symbols were low enough for the lowOne or lowThreshold
+		for i := uint32(0); toDistribute > 0; i = (i + 1) % (uint32(s.symbolLen)) {
+			if s.norm[i] > 0 {
+				toDistribute--
+				s.norm[i]++
+			}
+		}
+		return nil
+	}
+
+	var (
+		vStepLog = 62 - uint64(tableLog)
+		mid      = uint64((1 << (vStepLog - 1)) - 1)
+		rStep    = (((1 << vStepLog) * uint64(toDistribute)) + mid) / uint64(total) // scale on remaining
+		tmpTotal = mid
+	)
+	for i, cnt := range s.count[:s.symbolLen] {
+		if s.norm[i] == notYetAssigned {
+			var (
+				end    = tmpTotal + uint64(cnt)*rStep
+				sStart = uint32(tmpTotal >> vStepLog)
+				sEnd   = uint32(end >> vStepLog)
+				weight = sEnd - sStart
+			)
+			if weight < 1 {
+				return errors.New("weight < 1")
+			}
+			s.norm[i] = int16(weight)
+			tmpTotal = end
+		}
+	}
+	return nil
+}
+
+// validateNorm validates the normalized histogram table.
+func (s *Scratch) validateNorm() (err error) {
+	var total int
+	for _, v := range s.norm[:s.symbolLen] {
+		if v >= 0 {
+			total += int(v)
+		} else {
+			total -= int(v)
+		}
+	}
+	defer func() {
+		if err == nil {
+			return
+		}
+		fmt.Printf("selected TableLog: %d, Symbol length: %d\n", s.actualTableLog, s.symbolLen)
+		for i, v := range s.norm[:s.symbolLen] {
+			fmt.Printf("%3d: %5d -> %4d \n", i, s.count[i], v)
+		}
+	}()
+	if total != (1 << s.actualTableLog) {
+		return fmt.Errorf("warning: Total == %d != %d", total, 1<<s.actualTableLog)
+	}
+	for i, v := range s.count[s.symbolLen:] {
+		if v != 0 {
+			return fmt.Errorf("warning: Found symbol out of range, %d after cut", i)
+		}
+	}
+	return nil
+}
diff --git a/vendor/github.com/klauspost/compress/fse/decompress.go b/vendor/github.com/klauspost/compress/fse/decompress.go
new file mode 100644
index 0000000..926f5f1
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/decompress.go
@@ -0,0 +1,374 @@
+package fse
+
+import (
+	"errors"
+	"fmt"
+)
+
+const (
+	tablelogAbsoluteMax = 15
+)
+
+// Decompress a block of data.
+// You can provide a scratch buffer to avoid allocations.
+// If nil is provided a temporary one will be allocated.
+// It is possible, but by no way guaranteed that corrupt data will
+// return an error.
+// It is up to the caller to verify integrity of the returned data.
+// Use a predefined Scrach to set maximum acceptable output size.
+func Decompress(b []byte, s *Scratch) ([]byte, error) {
+	s, err := s.prepare(b)
+	if err != nil {
+		return nil, err
+	}
+	s.Out = s.Out[:0]
+	err = s.readNCount()
+	if err != nil {
+		return nil, err
+	}
+	err = s.buildDtable()
+	if err != nil {
+		return nil, err
+	}
+	err = s.decompress()
+	if err != nil {
+		return nil, err
+	}
+
+	return s.Out, nil
+}
+
+// readNCount will read the symbol distribution so decoding tables can be constructed.
+func (s *Scratch) readNCount() error {
+	var (
+		charnum   uint16
+		previous0 bool
+		b         = &s.br
+	)
+	iend := b.remain()
+	if iend < 4 {
+		return errors.New("input too small")
+	}
+	bitStream := b.Uint32()
+	nbBits := uint((bitStream & 0xF) + minTablelog) // extract tableLog
+	if nbBits > tablelogAbsoluteMax {
+		return errors.New("tableLog too large")
+	}
+	bitStream >>= 4
+	bitCount := uint(4)
+
+	s.actualTableLog = uint8(nbBits)
+	remaining := int32((1 << nbBits) + 1)
+	threshold := int32(1 << nbBits)
+	gotTotal := int32(0)
+	nbBits++
+
+	for remaining > 1 {
+		if previous0 {
+			n0 := charnum
+			for (bitStream & 0xFFFF) == 0xFFFF {
+				n0 += 24
+				if b.off < iend-5 {
+					b.advance(2)
+					bitStream = b.Uint32() >> bitCount
+				} else {
+					bitStream >>= 16
+					bitCount += 16
+				}
+			}
+			for (bitStream & 3) == 3 {
+				n0 += 3
+				bitStream >>= 2
+				bitCount += 2
+			}
+			n0 += uint16(bitStream & 3)
+			bitCount += 2
+			if n0 > maxSymbolValue {
+				return errors.New("maxSymbolValue too small")
+			}
+			for charnum < n0 {
+				s.norm[charnum&0xff] = 0
+				charnum++
+			}
+
+			if b.off <= iend-7 || b.off+int(bitCount>>3) <= iend-4 {
+				b.advance(bitCount >> 3)
+				bitCount &= 7
+				bitStream = b.Uint32() >> bitCount
+			} else {
+				bitStream >>= 2
+			}
+		}
+
+		max := (2*(threshold) - 1) - (remaining)
+		var count int32
+
+		if (int32(bitStream) & (threshold - 1)) < max {
+			count = int32(bitStream) & (threshold - 1)
+			bitCount += nbBits - 1
+		} else {
+			count = int32(bitStream) & (2*threshold - 1)
+			if count >= threshold {
+				count -= max
+			}
+			bitCount += nbBits
+		}
+
+		count-- // extra accuracy
+		if count < 0 {
+			// -1 means +1
+			remaining += count
+			gotTotal -= count
+		} else {
+			remaining -= count
+			gotTotal += count
+		}
+		s.norm[charnum&0xff] = int16(count)
+		charnum++
+		previous0 = count == 0
+		for remaining < threshold {
+			nbBits--
+			threshold >>= 1
+		}
+		if b.off <= iend-7 || b.off+int(bitCount>>3) <= iend-4 {
+			b.advance(bitCount >> 3)
+			bitCount &= 7
+		} else {
+			bitCount -= (uint)(8 * (len(b.b) - 4 - b.off))
+			b.off = len(b.b) - 4
+		}
+		bitStream = b.Uint32() >> (bitCount & 31)
+	}
+	s.symbolLen = charnum
+
+	if s.symbolLen <= 1 {
+		return fmt.Errorf("symbolLen (%d) too small", s.symbolLen)
+	}
+	if s.symbolLen > maxSymbolValue+1 {
+		return fmt.Errorf("symbolLen (%d) too big", s.symbolLen)
+	}
+	if remaining != 1 {
+		return fmt.Errorf("corruption detected (remaining %d != 1)", remaining)
+	}
+	if bitCount > 32 {
+		return fmt.Errorf("corruption detected (bitCount %d > 32)", bitCount)
+	}
+	if gotTotal != 1<<s.actualTableLog {
+		return fmt.Errorf("corruption detected (total %d != %d)", gotTotal, 1<<s.actualTableLog)
+	}
+	b.advance((bitCount + 7) >> 3)
+	return nil
+}
+
+// decSymbol contains information about a state entry,
+// Including the state offset base, the output symbol and
+// the number of bits to read for the low part of the destination state.
+type decSymbol struct {
+	newState uint16
+	symbol   uint8
+	nbBits   uint8
+}
+
+// allocDtable will allocate decoding tables if they are not big enough.
+func (s *Scratch) allocDtable() {
+	tableSize := 1 << s.actualTableLog
+	if cap(s.decTable) < tableSize {
+		s.decTable = make([]decSymbol, tableSize)
+	}
+	s.decTable = s.decTable[:tableSize]
+
+	if cap(s.ct.tableSymbol) < 256 {
+		s.ct.tableSymbol = make([]byte, 256)
+	}
+	s.ct.tableSymbol = s.ct.tableSymbol[:256]
+
+	if cap(s.ct.stateTable) < 256 {
+		s.ct.stateTable = make([]uint16, 256)
+	}
+	s.ct.stateTable = s.ct.stateTable[:256]
+}
+
+// buildDtable will build the decoding table.
+func (s *Scratch) buildDtable() error {
+	tableSize := uint32(1 << s.actualTableLog)
+	highThreshold := tableSize - 1
+	s.allocDtable()
+	symbolNext := s.ct.stateTable[:256]
+
+	// Init, lay down lowprob symbols
+	s.zeroBits = false
+	{
+		largeLimit := int16(1 << (s.actualTableLog - 1))
+		for i, v := range s.norm[:s.symbolLen] {
+			if v == -1 {
+				s.decTable[highThreshold].symbol = uint8(i)
+				highThreshold--
+				symbolNext[i] = 1
+			} else {
+				if v >= largeLimit {
+					s.zeroBits = true
+				}
+				symbolNext[i] = uint16(v)
+			}
+		}
+	}
+	// Spread symbols
+	{
+		tableMask := tableSize - 1
+		step := tableStep(tableSize)
+		position := uint32(0)
+		for ss, v := range s.norm[:s.symbolLen] {
+			for i := 0; i < int(v); i++ {
+				s.decTable[position].symbol = uint8(ss)
+				position = (position + step) & tableMask
+				for position > highThreshold {
+					// lowprob area
+					position = (position + step) & tableMask
+				}
+			}
+		}
+		if position != 0 {
+			// position must reach all cells once, otherwise normalizedCounter is incorrect
+			return errors.New("corrupted input (position != 0)")
+		}
+	}
+
+	// Build Decoding table
+	{
+		tableSize := uint16(1 << s.actualTableLog)
+		for u, v := range s.decTable {
+			symbol := v.symbol
+			nextState := symbolNext[symbol]
+			symbolNext[symbol] = nextState + 1
+			nBits := s.actualTableLog - byte(highBits(uint32(nextState)))
+			s.decTable[u].nbBits = nBits
+			newState := (nextState << nBits) - tableSize
+			if newState >= tableSize {
+				return fmt.Errorf("newState (%d) outside table size (%d)", newState, tableSize)
+			}
+			if newState == uint16(u) && nBits == 0 {
+				// Seems weird that this is possible with nbits > 0.
+				return fmt.Errorf("newState (%d) == oldState (%d) and no bits", newState, u)
+			}
+			s.decTable[u].newState = newState
+		}
+	}
+	return nil
+}
+
+// decompress will decompress the bitstream.
+// If the buffer is over-read an error is returned.
+func (s *Scratch) decompress() error {
+	br := &s.bits
+	br.init(s.br.unread())
+
+	var s1, s2 decoder
+	// Initialize and decode first state and symbol.
+	s1.init(br, s.decTable, s.actualTableLog)
+	s2.init(br, s.decTable, s.actualTableLog)
+
+	// Use temp table to avoid bound checks/append penalty.
+	var tmp = s.ct.tableSymbol[:256]
+	var off uint8
+
+	// Main part
+	if !s.zeroBits {
+		for br.off >= 8 {
+			br.fillFast()
+			tmp[off+0] = s1.nextFast()
+			tmp[off+1] = s2.nextFast()
+			br.fillFast()
+			tmp[off+2] = s1.nextFast()
+			tmp[off+3] = s2.nextFast()
+			off += 4
+			// When off is 0, we have overflowed and should write.
+			if off == 0 {
+				s.Out = append(s.Out, tmp...)
+				if len(s.Out) >= s.DecompressLimit {
+					return fmt.Errorf("output size (%d) > DecompressLimit (%d)", len(s.Out), s.DecompressLimit)
+				}
+			}
+		}
+	} else {
+		for br.off >= 8 {
+			br.fillFast()
+			tmp[off+0] = s1.next()
+			tmp[off+1] = s2.next()
+			br.fillFast()
+			tmp[off+2] = s1.next()
+			tmp[off+3] = s2.next()
+			off += 4
+			if off == 0 {
+				s.Out = append(s.Out, tmp...)
+				// When off is 0, we have overflowed and should write.
+				if len(s.Out) >= s.DecompressLimit {
+					return fmt.Errorf("output size (%d) > DecompressLimit (%d)", len(s.Out), s.DecompressLimit)
+				}
+			}
+		}
+	}
+	s.Out = append(s.Out, tmp[:off]...)
+
+	// Final bits, a bit more expensive check
+	for {
+		if s1.finished() {
+			s.Out = append(s.Out, s1.final(), s2.final())
+			break
+		}
+		br.fill()
+		s.Out = append(s.Out, s1.next())
+		if s2.finished() {
+			s.Out = append(s.Out, s2.final(), s1.final())
+			break
+		}
+		s.Out = append(s.Out, s2.next())
+		if len(s.Out) >= s.DecompressLimit {
+			return fmt.Errorf("output size (%d) > DecompressLimit (%d)", len(s.Out), s.DecompressLimit)
+		}
+	}
+	return br.close()
+}
+
+// decoder keeps track of the current state and updates it from the bitstream.
+type decoder struct {
+	state uint16
+	br    *bitReader
+	dt    []decSymbol
+}
+
+// init will initialize the decoder and read the first state from the stream.
+func (d *decoder) init(in *bitReader, dt []decSymbol, tableLog uint8) {
+	d.dt = dt
+	d.br = in
+	d.state = in.getBits(tableLog)
+}
+
+// next returns the next symbol and sets the next state.
+// At least tablelog bits must be available in the bit reader.
+func (d *decoder) next() uint8 {
+	n := &d.dt[d.state]
+	lowBits := d.br.getBits(n.nbBits)
+	d.state = n.newState + lowBits
+	return n.symbol
+}
+
+// finished returns true if all bits have been read from the bitstream
+// and the next state would require reading bits from the input.
+func (d *decoder) finished() bool {
+	return d.br.finished() && d.dt[d.state].nbBits > 0
+}
+
+// final returns the current state symbol without decoding the next.
+func (d *decoder) final() uint8 {
+	return d.dt[d.state].symbol
+}
+
+// nextFast returns the next symbol and sets the next state.
+// This can only be used if no symbols are 0 bits.
+// At least tablelog bits must be available in the bit reader.
+func (d *decoder) nextFast() uint8 {
+	n := d.dt[d.state]
+	lowBits := d.br.getBitsFast(n.nbBits)
+	d.state = n.newState + lowBits
+	return n.symbol
+}
diff --git a/vendor/github.com/klauspost/compress/fse/fse.go b/vendor/github.com/klauspost/compress/fse/fse.go
new file mode 100644
index 0000000..535cbad
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/fse/fse.go
@@ -0,0 +1,144 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+// Package fse provides Finite State Entropy encoding and decoding.
+//
+// Finite State Entropy encoding provides a fast near-optimal symbol encoding/decoding
+// for byte blocks as implemented in zstd.
+//
+// See https://github.com/klauspost/compress/tree/master/fse for more information.
+package fse
+
+import (
+	"errors"
+	"fmt"
+	"math/bits"
+)
+
+const (
+	/*!MEMORY_USAGE :
+	 *  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
+	 *  Increasing memory usage improves compression ratio
+	 *  Reduced memory usage can improve speed, due to cache effect
+	 *  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
+	maxMemoryUsage     = 14
+	defaultMemoryUsage = 13
+
+	maxTableLog     = maxMemoryUsage - 2
+	maxTablesize    = 1 << maxTableLog
+	defaultTablelog = defaultMemoryUsage - 2
+	minTablelog     = 5
+	maxSymbolValue  = 255
+)
+
+var (
+	// ErrIncompressible is returned when input is judged to be too hard to compress.
+	ErrIncompressible = errors.New("input is not compressible")
+
+	// ErrUseRLE is returned from the compressor when the input is a single byte value repeated.
+	ErrUseRLE = errors.New("input is single value repeated")
+)
+
+// Scratch provides temporary storage for compression and decompression.
+type Scratch struct {
+	// Private
+	count    [maxSymbolValue + 1]uint32
+	norm     [maxSymbolValue + 1]int16
+	br       byteReader
+	bits     bitReader
+	bw       bitWriter
+	ct       cTable      // Compression tables.
+	decTable []decSymbol // Decompression table.
+	maxCount int         // count of the most probable symbol
+
+	// Per block parameters.
+	// These can be used to override compression parameters of the block.
+	// Do not touch, unless you know what you are doing.
+
+	// Out is output buffer.
+	// If the scratch is re-used before the caller is done processing the output,
+	// set this field to nil.
+	// Otherwise the output buffer will be re-used for next Compression/Decompression step
+	// and allocation will be avoided.
+	Out []byte
+
+	// DecompressLimit limits the maximum decoded size acceptable.
+	// If > 0 decompression will stop when approximately this many bytes
+	// has been decoded.
+	// If 0, maximum size will be 2GB.
+	DecompressLimit int
+
+	symbolLen      uint16 // Length of active part of the symbol table.
+	actualTableLog uint8  // Selected tablelog.
+	zeroBits       bool   // no bits has prob > 50%.
+	clearCount     bool   // clear count
+
+	// MaxSymbolValue will override the maximum symbol value of the next block.
+	MaxSymbolValue uint8
+
+	// TableLog will attempt to override the tablelog for the next block.
+	TableLog uint8
+}
+
+// Histogram allows to populate the histogram and skip that step in the compression,
+// It otherwise allows to inspect the histogram when compression is done.
+// To indicate that you have populated the histogram call HistogramFinished
+// with the value of the highest populated symbol, as well as the number of entries
+// in the most populated entry. These are accepted at face value.
+// The returned slice will always be length 256.
+func (s *Scratch) Histogram() []uint32 {
+	return s.count[:]
+}
+
+// HistogramFinished can be called to indicate that the histogram has been populated.
+// maxSymbol is the index of the highest set symbol of the next data segment.
+// maxCount is the number of entries in the most populated entry.
+// These are accepted at face value.
+func (s *Scratch) HistogramFinished(maxSymbol uint8, maxCount int) {
+	s.maxCount = maxCount
+	s.symbolLen = uint16(maxSymbol) + 1
+	s.clearCount = maxCount != 0
+}
+
+// prepare will prepare and allocate scratch tables used for both compression and decompression.
+func (s *Scratch) prepare(in []byte) (*Scratch, error) {
+	if s == nil {
+		s = &Scratch{}
+	}
+	if s.MaxSymbolValue == 0 {
+		s.MaxSymbolValue = 255
+	}
+	if s.TableLog == 0 {
+		s.TableLog = defaultTablelog
+	}
+	if s.TableLog > maxTableLog {
+		return nil, fmt.Errorf("tableLog (%d) > maxTableLog (%d)", s.TableLog, maxTableLog)
+	}
+	if cap(s.Out) == 0 {
+		s.Out = make([]byte, 0, len(in))
+	}
+	if s.clearCount && s.maxCount == 0 {
+		for i := range s.count {
+			s.count[i] = 0
+		}
+		s.clearCount = false
+	}
+	s.br.init(in)
+	if s.DecompressLimit == 0 {
+		// Max size 2GB.
+		s.DecompressLimit = (2 << 30) - 1
+	}
+
+	return s, nil
+}
+
+// tableStep returns the next table index.
+func tableStep(tableSize uint32) uint32 {
+	return (tableSize >> 1) + (tableSize >> 3) + 3
+}
+
+func highBits(val uint32) (n uint32) {
+	return uint32(bits.Len32(val) - 1)
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/.gitignore b/vendor/github.com/klauspost/compress/huff0/.gitignore
new file mode 100644
index 0000000..b3d2629
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/.gitignore
@@ -0,0 +1 @@
+/huff0-fuzz.zip
diff --git a/vendor/github.com/klauspost/compress/huff0/README.md b/vendor/github.com/klauspost/compress/huff0/README.md
new file mode 100644
index 0000000..8b6e5c6
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/README.md
@@ -0,0 +1,89 @@
+# Huff0 entropy compression

+

+This package provides Huff0 encoding and decoding as used in zstd.

+            

+[Huff0](https://github.com/Cyan4973/FiniteStateEntropy#new-generation-entropy-coders), 

+a Huffman codec designed for modern CPU, featuring OoO (Out of Order) operations on multiple ALU 

+(Arithmetic Logic Unit), achieving extremely fast compression and decompression speeds.

+

+This can be used for compressing input with a lot of similar input values to the smallest number of bytes.

+This does not perform any multi-byte [dictionary coding](https://en.wikipedia.org/wiki/Dictionary_coder) as LZ coders,

+but it can be used as a secondary step to compressors (like Snappy) that does not do entropy encoding. 

+

+* [Godoc documentation](https://godoc.org/github.com/klauspost/compress/huff0)

+

+## News

+

+This is used as part of the [zstandard](https://github.com/klauspost/compress/tree/master/zstd#zstd) compression and decompression package.

+

+This ensures that most functionality is well tested.

+

+# Usage

+

+This package provides a low level interface that allows to compress single independent blocks. 

+

+Each block is separate, and there is no built in integrity checks. 

+This means that the caller should keep track of block sizes and also do checksums if needed.  

+

+Compressing a block is done via the [`Compress1X`](https://godoc.org/github.com/klauspost/compress/huff0#Compress1X) and 

+[`Compress4X`](https://godoc.org/github.com/klauspost/compress/huff0#Compress4X) functions.

+You must provide input and will receive the output and maybe an error.

+

+These error values can be returned:

+

+| Error               | Description                                                                 |

+|---------------------|-----------------------------------------------------------------------------|

+| `<nil>`             | Everything ok, output is returned                                           |

+| `ErrIncompressible` | Returned when input is judged to be too hard to compress                    |

+| `ErrUseRLE`         | Returned from the compressor when the input is a single byte value repeated |

+| `ErrTooBig`         | Returned if the input block exceeds the maximum allowed size (128 Kib)      |

+| `(error)`           | An internal error occurred.                                                 |

+

+

+As can be seen above some of there are errors that will be returned even under normal operation so it is important to handle these.

+

+To reduce allocations you can provide a [`Scratch`](https://godoc.org/github.com/klauspost/compress/huff0#Scratch) object 

+that can be re-used for successive calls. Both compression and decompression accepts a `Scratch` object, and the same 

+object can be used for both.   

+

+Be aware, that when re-using a `Scratch` object that the *output* buffer is also re-used, so if you are still using this

+you must set the `Out` field in the scratch to nil. The same buffer is used for compression and decompression output.

+

+The `Scratch` object will retain state that allows to re-use previous tables for encoding and decoding.  

+

+## Tables and re-use

+

+Huff0 allows for reusing tables from the previous block to save space if that is expected to give better/faster results. 

+

+The Scratch object allows you to set a [`ReusePolicy`](https://godoc.org/github.com/klauspost/compress/huff0#ReusePolicy) 

+that controls this behaviour. See the documentation for details. This can be altered between each block.

+

+Do however note that this information is *not* stored in the output block and it is up to the users of the package to

+record whether [`ReadTable`](https://godoc.org/github.com/klauspost/compress/huff0#ReadTable) should be called,

+based on the boolean reported back from the CompressXX call. 

+

+If you want to store the table separate from the data, you can access them as `OutData` and `OutTable` on the 

+[`Scratch`](https://godoc.org/github.com/klauspost/compress/huff0#Scratch) object.

+

+## Decompressing

+

+The first part of decoding is to initialize the decoding table through [`ReadTable`](https://godoc.org/github.com/klauspost/compress/huff0#ReadTable).

+This will initialize the decoding tables. 

+You can supply the complete block to `ReadTable` and it will return the data part of the block 

+which can be given to the decompressor. 

+

+Decompressing is done by calling the [`Decompress1X`](https://godoc.org/github.com/klauspost/compress/huff0#Scratch.Decompress1X) 

+or [`Decompress4X`](https://godoc.org/github.com/klauspost/compress/huff0#Scratch.Decompress4X) function.

+

+For concurrently decompressing content with a fixed table a stateless [`Decoder`](https://godoc.org/github.com/klauspost/compress/huff0#Decoder) can be requested which will remain correct as long as the scratch is unchanged. The capacity of the provided slice indicates the expected output size.

+

+You must provide the output from the compression stage, at exactly the size you got back. If you receive an error back

+your input was likely corrupted. 

+

+It is important to note that a successful decoding does *not* mean your output matches your original input. 

+There are no integrity checks, so relying on errors from the decompressor does not assure your data is valid.

+

+# Contributing

+

+Contributions are always welcome. Be aware that adding public functions will require good justification and breaking 

+changes will likely not be accepted. If in doubt open an issue before writing the PR.

diff --git a/vendor/github.com/klauspost/compress/huff0/bitreader.go b/vendor/github.com/klauspost/compress/huff0/bitreader.go
new file mode 100644
index 0000000..a4979e8
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/bitreader.go
@@ -0,0 +1,329 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package huff0
+
+import (
+	"encoding/binary"
+	"errors"
+	"io"
+)
+
+// bitReader reads a bitstream in reverse.
+// The last set bit indicates the start of the stream and is used
+// for aligning the input.
+type bitReader struct {
+	in       []byte
+	off      uint // next byte to read is at in[off - 1]
+	value    uint64
+	bitsRead uint8
+}
+
+// init initializes and resets the bit reader.
+func (b *bitReader) init(in []byte) error {
+	if len(in) < 1 {
+		return errors.New("corrupt stream: too short")
+	}
+	b.in = in
+	b.off = uint(len(in))
+	// The highest bit of the last byte indicates where to start
+	v := in[len(in)-1]
+	if v == 0 {
+		return errors.New("corrupt stream, did not find end of stream")
+	}
+	b.bitsRead = 64
+	b.value = 0
+	if len(in) >= 8 {
+		b.fillFastStart()
+	} else {
+		b.fill()
+		b.fill()
+	}
+	b.bitsRead += 8 - uint8(highBit32(uint32(v)))
+	return nil
+}
+
+// peekBitsFast requires that at least one bit is requested every time.
+// There are no checks if the buffer is filled.
+func (b *bitReader) peekBitsFast(n uint8) uint16 {
+	const regMask = 64 - 1
+	v := uint16((b.value << (b.bitsRead & regMask)) >> ((regMask + 1 - n) & regMask))
+	return v
+}
+
+// fillFast() will make sure at least 32 bits are available.
+// There must be at least 4 bytes available.
+func (b *bitReader) fillFast() {
+	if b.bitsRead < 32 {
+		return
+	}
+
+	// 2 bounds checks.
+	v := b.in[b.off-4 : b.off]
+	v = v[:4]
+	low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+	b.value = (b.value << 32) | uint64(low)
+	b.bitsRead -= 32
+	b.off -= 4
+}
+
+func (b *bitReader) advance(n uint8) {
+	b.bitsRead += n
+}
+
+// fillFastStart() assumes the bitreader is empty and there is at least 8 bytes to read.
+func (b *bitReader) fillFastStart() {
+	// Do single re-slice to avoid bounds checks.
+	b.value = binary.LittleEndian.Uint64(b.in[b.off-8:])
+	b.bitsRead = 0
+	b.off -= 8
+}
+
+// fill() will make sure at least 32 bits are available.
+func (b *bitReader) fill() {
+	if b.bitsRead < 32 {
+		return
+	}
+	if b.off > 4 {
+		v := b.in[b.off-4:]
+		v = v[:4]
+		low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+		b.value = (b.value << 32) | uint64(low)
+		b.bitsRead -= 32
+		b.off -= 4
+		return
+	}
+	for b.off > 0 {
+		b.value = (b.value << 8) | uint64(b.in[b.off-1])
+		b.bitsRead -= 8
+		b.off--
+	}
+}
+
+// finished returns true if all bits have been read from the bit stream.
+func (b *bitReader) finished() bool {
+	return b.off == 0 && b.bitsRead >= 64
+}
+
+// close the bitstream and returns an error if out-of-buffer reads occurred.
+func (b *bitReader) close() error {
+	// Release reference.
+	b.in = nil
+	if b.bitsRead > 64 {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+
+// bitReader reads a bitstream in reverse.
+// The last set bit indicates the start of the stream and is used
+// for aligning the input.
+type bitReaderBytes struct {
+	in       []byte
+	off      uint // next byte to read is at in[off - 1]
+	value    uint64
+	bitsRead uint8
+}
+
+// init initializes and resets the bit reader.
+func (b *bitReaderBytes) init(in []byte) error {
+	if len(in) < 1 {
+		return errors.New("corrupt stream: too short")
+	}
+	b.in = in
+	b.off = uint(len(in))
+	// The highest bit of the last byte indicates where to start
+	v := in[len(in)-1]
+	if v == 0 {
+		return errors.New("corrupt stream, did not find end of stream")
+	}
+	b.bitsRead = 64
+	b.value = 0
+	if len(in) >= 8 {
+		b.fillFastStart()
+	} else {
+		b.fill()
+		b.fill()
+	}
+	b.advance(8 - uint8(highBit32(uint32(v))))
+	return nil
+}
+
+// peekBitsFast requires that at least one bit is requested every time.
+// There are no checks if the buffer is filled.
+func (b *bitReaderBytes) peekByteFast() uint8 {
+	got := uint8(b.value >> 56)
+	return got
+}
+
+func (b *bitReaderBytes) advance(n uint8) {
+	b.bitsRead += n
+	b.value <<= n & 63
+}
+
+// fillFast() will make sure at least 32 bits are available.
+// There must be at least 4 bytes available.
+func (b *bitReaderBytes) fillFast() {
+	if b.bitsRead < 32 {
+		return
+	}
+
+	// 2 bounds checks.
+	v := b.in[b.off-4 : b.off]
+	v = v[:4]
+	low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+	b.value |= uint64(low) << (b.bitsRead - 32)
+	b.bitsRead -= 32
+	b.off -= 4
+}
+
+// fillFastStart() assumes the bitReaderBytes is empty and there is at least 8 bytes to read.
+func (b *bitReaderBytes) fillFastStart() {
+	// Do single re-slice to avoid bounds checks.
+	b.value = binary.LittleEndian.Uint64(b.in[b.off-8:])
+	b.bitsRead = 0
+	b.off -= 8
+}
+
+// fill() will make sure at least 32 bits are available.
+func (b *bitReaderBytes) fill() {
+	if b.bitsRead < 32 {
+		return
+	}
+	if b.off > 4 {
+		v := b.in[b.off-4:]
+		v = v[:4]
+		low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+		b.value |= uint64(low) << (b.bitsRead - 32)
+		b.bitsRead -= 32
+		b.off -= 4
+		return
+	}
+	for b.off > 0 {
+		b.value |= uint64(b.in[b.off-1]) << (b.bitsRead - 8)
+		b.bitsRead -= 8
+		b.off--
+	}
+}
+
+// finished returns true if all bits have been read from the bit stream.
+func (b *bitReaderBytes) finished() bool {
+	return b.off == 0 && b.bitsRead >= 64
+}
+
+// close the bitstream and returns an error if out-of-buffer reads occurred.
+func (b *bitReaderBytes) close() error {
+	// Release reference.
+	b.in = nil
+	if b.bitsRead > 64 {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+
+// bitReaderShifted reads a bitstream in reverse.
+// The last set bit indicates the start of the stream and is used
+// for aligning the input.
+type bitReaderShifted struct {
+	in       []byte
+	off      uint // next byte to read is at in[off - 1]
+	value    uint64
+	bitsRead uint8
+}
+
+// init initializes and resets the bit reader.
+func (b *bitReaderShifted) init(in []byte) error {
+	if len(in) < 1 {
+		return errors.New("corrupt stream: too short")
+	}
+	b.in = in
+	b.off = uint(len(in))
+	// The highest bit of the last byte indicates where to start
+	v := in[len(in)-1]
+	if v == 0 {
+		return errors.New("corrupt stream, did not find end of stream")
+	}
+	b.bitsRead = 64
+	b.value = 0
+	if len(in) >= 8 {
+		b.fillFastStart()
+	} else {
+		b.fill()
+		b.fill()
+	}
+	b.advance(8 - uint8(highBit32(uint32(v))))
+	return nil
+}
+
+// peekBitsFast requires that at least one bit is requested every time.
+// There are no checks if the buffer is filled.
+func (b *bitReaderShifted) peekBitsFast(n uint8) uint16 {
+	return uint16(b.value >> ((64 - n) & 63))
+}
+
+func (b *bitReaderShifted) advance(n uint8) {
+	b.bitsRead += n
+	b.value <<= n & 63
+}
+
+// fillFast() will make sure at least 32 bits are available.
+// There must be at least 4 bytes available.
+func (b *bitReaderShifted) fillFast() {
+	if b.bitsRead < 32 {
+		return
+	}
+
+	// 2 bounds checks.
+	v := b.in[b.off-4 : b.off]
+	v = v[:4]
+	low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+	b.value |= uint64(low) << ((b.bitsRead - 32) & 63)
+	b.bitsRead -= 32
+	b.off -= 4
+}
+
+// fillFastStart() assumes the bitReaderShifted is empty and there is at least 8 bytes to read.
+func (b *bitReaderShifted) fillFastStart() {
+	// Do single re-slice to avoid bounds checks.
+	b.value = binary.LittleEndian.Uint64(b.in[b.off-8:])
+	b.bitsRead = 0
+	b.off -= 8
+}
+
+// fill() will make sure at least 32 bits are available.
+func (b *bitReaderShifted) fill() {
+	if b.bitsRead < 32 {
+		return
+	}
+	if b.off > 4 {
+		v := b.in[b.off-4:]
+		v = v[:4]
+		low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+		b.value |= uint64(low) << ((b.bitsRead - 32) & 63)
+		b.bitsRead -= 32
+		b.off -= 4
+		return
+	}
+	for b.off > 0 {
+		b.value |= uint64(b.in[b.off-1]) << ((b.bitsRead - 8) & 63)
+		b.bitsRead -= 8
+		b.off--
+	}
+}
+
+// finished returns true if all bits have been read from the bit stream.
+func (b *bitReaderShifted) finished() bool {
+	return b.off == 0 && b.bitsRead >= 64
+}
+
+// close the bitstream and returns an error if out-of-buffer reads occurred.
+func (b *bitReaderShifted) close() error {
+	// Release reference.
+	b.in = nil
+	if b.bitsRead > 64 {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/bitwriter.go b/vendor/github.com/klauspost/compress/huff0/bitwriter.go
new file mode 100644
index 0000000..6bce4e8
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/bitwriter.go
@@ -0,0 +1,210 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package huff0
+
+import "fmt"
+
+// bitWriter will write bits.
+// First bit will be LSB of the first byte of output.
+type bitWriter struct {
+	bitContainer uint64
+	nBits        uint8
+	out          []byte
+}
+
+// bitMask16 is bitmasks. Has extra to avoid bounds check.
+var bitMask16 = [32]uint16{
+	0, 1, 3, 7, 0xF, 0x1F,
+	0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF,
+	0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF} /* up to 16 bits */
+
+// addBits16NC will add up to 16 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16NC(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value&bitMask16[bits&31]) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// addBits16Clean will add up to 16 bits. value may not contain more set bits than indicated.
+// It will not check if there is space for them, so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16Clean(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// encSymbol will add up to 16 bits. value may not contain more set bits than indicated.
+// It will not check if there is space for them, so the caller must ensure that it has flushed recently.
+func (b *bitWriter) encSymbol(ct cTable, symbol byte) {
+	enc := ct[symbol]
+	b.bitContainer |= uint64(enc.val) << (b.nBits & 63)
+	if false {
+		if enc.nBits == 0 {
+			panic("nbits 0")
+		}
+	}
+	b.nBits += enc.nBits
+}
+
+// encTwoSymbols will add up to 32 bits. value may not contain more set bits than indicated.
+// It will not check if there is space for them, so the caller must ensure that it has flushed recently.
+func (b *bitWriter) encTwoSymbols(ct cTable, av, bv byte) {
+	encA := ct[av]
+	encB := ct[bv]
+	sh := b.nBits & 63
+	combined := uint64(encA.val) | (uint64(encB.val) << (encA.nBits & 63))
+	b.bitContainer |= combined << sh
+	if false {
+		if encA.nBits == 0 {
+			panic("nbitsA 0")
+		}
+		if encB.nBits == 0 {
+			panic("nbitsB 0")
+		}
+	}
+	b.nBits += encA.nBits + encB.nBits
+}
+
+// addBits16ZeroNC will add up to 16 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+// This is fastest if bits can be zero.
+func (b *bitWriter) addBits16ZeroNC(value uint16, bits uint8) {
+	if bits == 0 {
+		return
+	}
+	value <<= (16 - bits) & 15
+	value >>= (16 - bits) & 15
+	b.bitContainer |= uint64(value) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// flush will flush all pending full bytes.
+// There will be at least 56 bits available for writing when this has been called.
+// Using flush32 is faster, but leaves less space for writing.
+func (b *bitWriter) flush() {
+	v := b.nBits >> 3
+	switch v {
+	case 0:
+		return
+	case 1:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+		)
+		b.bitContainer >>= 1 << 3
+	case 2:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+		)
+		b.bitContainer >>= 2 << 3
+	case 3:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+		)
+		b.bitContainer >>= 3 << 3
+	case 4:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+		)
+		b.bitContainer >>= 4 << 3
+	case 5:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+		)
+		b.bitContainer >>= 5 << 3
+	case 6:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+		)
+		b.bitContainer >>= 6 << 3
+	case 7:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+		)
+		b.bitContainer >>= 7 << 3
+	case 8:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+			byte(b.bitContainer>>56),
+		)
+		b.bitContainer = 0
+		b.nBits = 0
+		return
+	default:
+		panic(fmt.Errorf("bits (%d) > 64", b.nBits))
+	}
+	b.nBits &= 7
+}
+
+// flush32 will flush out, so there are at least 32 bits available for writing.
+func (b *bitWriter) flush32() {
+	if b.nBits < 32 {
+		return
+	}
+	b.out = append(b.out,
+		byte(b.bitContainer),
+		byte(b.bitContainer>>8),
+		byte(b.bitContainer>>16),
+		byte(b.bitContainer>>24))
+	b.nBits -= 32
+	b.bitContainer >>= 32
+}
+
+// flushAlign will flush remaining full bytes and align to next byte boundary.
+func (b *bitWriter) flushAlign() {
+	nbBytes := (b.nBits + 7) >> 3
+	for i := uint8(0); i < nbBytes; i++ {
+		b.out = append(b.out, byte(b.bitContainer>>(i*8)))
+	}
+	b.nBits = 0
+	b.bitContainer = 0
+}
+
+// close will write the alignment bit and write the final byte(s)
+// to the output.
+func (b *bitWriter) close() error {
+	// End mark
+	b.addBits16Clean(1, 1)
+	// flush until next byte.
+	b.flushAlign()
+	return nil
+}
+
+// reset and continue writing by appending to out.
+func (b *bitWriter) reset(out []byte) {
+	b.bitContainer = 0
+	b.nBits = 0
+	b.out = out
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/bytereader.go b/vendor/github.com/klauspost/compress/huff0/bytereader.go
new file mode 100644
index 0000000..50bcdf6
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/bytereader.go
@@ -0,0 +1,54 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package huff0
+
+// byteReader provides a byte reader that reads
+// little endian values from a byte stream.
+// The input stream is manually advanced.
+// The reader performs no bounds checks.
+type byteReader struct {
+	b   []byte
+	off int
+}
+
+// init will initialize the reader and set the input.
+func (b *byteReader) init(in []byte) {
+	b.b = in
+	b.off = 0
+}
+
+// advance the stream b n bytes.
+func (b *byteReader) advance(n uint) {
+	b.off += int(n)
+}
+
+// Int32 returns a little endian int32 starting at current offset.
+func (b byteReader) Int32() int32 {
+	v3 := int32(b.b[b.off+3])
+	v2 := int32(b.b[b.off+2])
+	v1 := int32(b.b[b.off+1])
+	v0 := int32(b.b[b.off])
+	return (v3 << 24) | (v2 << 16) | (v1 << 8) | v0
+}
+
+// Uint32 returns a little endian uint32 starting at current offset.
+func (b byteReader) Uint32() uint32 {
+	v3 := uint32(b.b[b.off+3])
+	v2 := uint32(b.b[b.off+2])
+	v1 := uint32(b.b[b.off+1])
+	v0 := uint32(b.b[b.off])
+	return (v3 << 24) | (v2 << 16) | (v1 << 8) | v0
+}
+
+// unread returns the unread portion of the input.
+func (b byteReader) unread() []byte {
+	return b.b[b.off:]
+}
+
+// remain will return the number of bytes remaining.
+func (b byteReader) remain() int {
+	return len(b.b) - b.off
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/compress.go b/vendor/github.com/klauspost/compress/huff0/compress.go
new file mode 100644
index 0000000..0823c92
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/compress.go
@@ -0,0 +1,656 @@
+package huff0
+
+import (
+	"fmt"
+	"runtime"
+	"sync"
+)
+
+// Compress1X will compress the input.
+// The output can be decoded using Decompress1X.
+// Supply a Scratch object. The scratch object contains state about re-use,
+// So when sharing across independent encodes, be sure to set the re-use policy.
+func Compress1X(in []byte, s *Scratch) (out []byte, reUsed bool, err error) {
+	s, err = s.prepare(in)
+	if err != nil {
+		return nil, false, err
+	}
+	return compress(in, s, s.compress1X)
+}
+
+// Compress4X will compress the input. The input is split into 4 independent blocks
+// and compressed similar to Compress1X.
+// The output can be decoded using Decompress4X.
+// Supply a Scratch object. The scratch object contains state about re-use,
+// So when sharing across independent encodes, be sure to set the re-use policy.
+func Compress4X(in []byte, s *Scratch) (out []byte, reUsed bool, err error) {
+	s, err = s.prepare(in)
+	if err != nil {
+		return nil, false, err
+	}
+	if false {
+		// TODO: compress4Xp only slightly faster.
+		const parallelThreshold = 8 << 10
+		if len(in) < parallelThreshold || runtime.GOMAXPROCS(0) == 1 {
+			return compress(in, s, s.compress4X)
+		}
+		return compress(in, s, s.compress4Xp)
+	}
+	return compress(in, s, s.compress4X)
+}
+
+func compress(in []byte, s *Scratch, compressor func(src []byte) ([]byte, error)) (out []byte, reUsed bool, err error) {
+	// Nuke previous table if we cannot reuse anyway.
+	if s.Reuse == ReusePolicyNone {
+		s.prevTable = s.prevTable[:0]
+	}
+
+	// Create histogram, if none was provided.
+	maxCount := s.maxCount
+	var canReuse = false
+	if maxCount == 0 {
+		maxCount, canReuse = s.countSimple(in)
+	} else {
+		canReuse = s.canUseTable(s.prevTable)
+	}
+
+	// We want the output size to be less than this:
+	wantSize := len(in)
+	if s.WantLogLess > 0 {
+		wantSize -= wantSize >> s.WantLogLess
+	}
+
+	// Reset for next run.
+	s.clearCount = true
+	s.maxCount = 0
+	if maxCount >= len(in) {
+		if maxCount > len(in) {
+			return nil, false, fmt.Errorf("maxCount (%d) > length (%d)", maxCount, len(in))
+		}
+		if len(in) == 1 {
+			return nil, false, ErrIncompressible
+		}
+		// One symbol, use RLE
+		return nil, false, ErrUseRLE
+	}
+	if maxCount == 1 || maxCount < (len(in)>>7) {
+		// Each symbol present maximum once or too well distributed.
+		return nil, false, ErrIncompressible
+	}
+	if s.Reuse == ReusePolicyMust && !canReuse {
+		// We must reuse, but we can't.
+		return nil, false, ErrIncompressible
+	}
+	if (s.Reuse == ReusePolicyPrefer || s.Reuse == ReusePolicyMust) && canReuse {
+		keepTable := s.cTable
+		keepTL := s.actualTableLog
+		s.cTable = s.prevTable
+		s.actualTableLog = s.prevTableLog
+		s.Out, err = compressor(in)
+		s.cTable = keepTable
+		s.actualTableLog = keepTL
+		if err == nil && len(s.Out) < wantSize {
+			s.OutData = s.Out
+			return s.Out, true, nil
+		}
+		if s.Reuse == ReusePolicyMust {
+			return nil, false, ErrIncompressible
+		}
+		// Do not attempt to re-use later.
+		s.prevTable = s.prevTable[:0]
+	}
+
+	// Calculate new table.
+	err = s.buildCTable()
+	if err != nil {
+		return nil, false, err
+	}
+
+	if false && !s.canUseTable(s.cTable) {
+		panic("invalid table generated")
+	}
+
+	if s.Reuse == ReusePolicyAllow && canReuse {
+		hSize := len(s.Out)
+		oldSize := s.prevTable.estimateSize(s.count[:s.symbolLen])
+		newSize := s.cTable.estimateSize(s.count[:s.symbolLen])
+		if oldSize <= hSize+newSize || hSize+12 >= wantSize {
+			// Retain cTable even if we re-use.
+			keepTable := s.cTable
+			keepTL := s.actualTableLog
+
+			s.cTable = s.prevTable
+			s.actualTableLog = s.prevTableLog
+			s.Out, err = compressor(in)
+
+			// Restore ctable.
+			s.cTable = keepTable
+			s.actualTableLog = keepTL
+			if err != nil {
+				return nil, false, err
+			}
+			if len(s.Out) >= wantSize {
+				return nil, false, ErrIncompressible
+			}
+			s.OutData = s.Out
+			return s.Out, true, nil
+		}
+	}
+
+	// Use new table
+	err = s.cTable.write(s)
+	if err != nil {
+		s.OutTable = nil
+		return nil, false, err
+	}
+	s.OutTable = s.Out
+
+	// Compress using new table
+	s.Out, err = compressor(in)
+	if err != nil {
+		s.OutTable = nil
+		return nil, false, err
+	}
+	if len(s.Out) >= wantSize {
+		s.OutTable = nil
+		return nil, false, ErrIncompressible
+	}
+	// Move current table into previous.
+	s.prevTable, s.prevTableLog, s.cTable = s.cTable, s.actualTableLog, s.prevTable[:0]
+	s.OutData = s.Out[len(s.OutTable):]
+	return s.Out, false, nil
+}
+
+func (s *Scratch) compress1X(src []byte) ([]byte, error) {
+	return s.compress1xDo(s.Out, src)
+}
+
+func (s *Scratch) compress1xDo(dst, src []byte) ([]byte, error) {
+	var bw = bitWriter{out: dst}
+
+	// N is length divisible by 4.
+	n := len(src)
+	n -= n & 3
+	cTable := s.cTable[:256]
+
+	// Encode last bytes.
+	for i := len(src) & 3; i > 0; i-- {
+		bw.encSymbol(cTable, src[n+i-1])
+	}
+	n -= 4
+	if s.actualTableLog <= 8 {
+		for ; n >= 0; n -= 4 {
+			tmp := src[n : n+4]
+			// tmp should be len 4
+			bw.flush32()
+			bw.encTwoSymbols(cTable, tmp[3], tmp[2])
+			bw.encTwoSymbols(cTable, tmp[1], tmp[0])
+		}
+	} else {
+		for ; n >= 0; n -= 4 {
+			tmp := src[n : n+4]
+			// tmp should be len 4
+			bw.flush32()
+			bw.encTwoSymbols(cTable, tmp[3], tmp[2])
+			bw.flush32()
+			bw.encTwoSymbols(cTable, tmp[1], tmp[0])
+		}
+	}
+	err := bw.close()
+	return bw.out, err
+}
+
+var sixZeros [6]byte
+
+func (s *Scratch) compress4X(src []byte) ([]byte, error) {
+	if len(src) < 12 {
+		return nil, ErrIncompressible
+	}
+	segmentSize := (len(src) + 3) / 4
+
+	// Add placeholder for output length
+	offsetIdx := len(s.Out)
+	s.Out = append(s.Out, sixZeros[:]...)
+
+	for i := 0; i < 4; i++ {
+		toDo := src
+		if len(toDo) > segmentSize {
+			toDo = toDo[:segmentSize]
+		}
+		src = src[len(toDo):]
+
+		var err error
+		idx := len(s.Out)
+		s.Out, err = s.compress1xDo(s.Out, toDo)
+		if err != nil {
+			return nil, err
+		}
+		// Write compressed length as little endian before block.
+		if i < 3 {
+			// Last length is not written.
+			length := len(s.Out) - idx
+			s.Out[i*2+offsetIdx] = byte(length)
+			s.Out[i*2+offsetIdx+1] = byte(length >> 8)
+		}
+	}
+
+	return s.Out, nil
+}
+
+// compress4Xp will compress 4 streams using separate goroutines.
+func (s *Scratch) compress4Xp(src []byte) ([]byte, error) {
+	if len(src) < 12 {
+		return nil, ErrIncompressible
+	}
+	// Add placeholder for output length
+	s.Out = s.Out[:6]
+
+	segmentSize := (len(src) + 3) / 4
+	var wg sync.WaitGroup
+	var errs [4]error
+	wg.Add(4)
+	for i := 0; i < 4; i++ {
+		toDo := src
+		if len(toDo) > segmentSize {
+			toDo = toDo[:segmentSize]
+		}
+		src = src[len(toDo):]
+
+		// Separate goroutine for each block.
+		go func(i int) {
+			s.tmpOut[i], errs[i] = s.compress1xDo(s.tmpOut[i][:0], toDo)
+			wg.Done()
+		}(i)
+	}
+	wg.Wait()
+	for i := 0; i < 4; i++ {
+		if errs[i] != nil {
+			return nil, errs[i]
+		}
+		o := s.tmpOut[i]
+		// Write compressed length as little endian before block.
+		if i < 3 {
+			// Last length is not written.
+			s.Out[i*2] = byte(len(o))
+			s.Out[i*2+1] = byte(len(o) >> 8)
+		}
+
+		// Write output.
+		s.Out = append(s.Out, o...)
+	}
+	return s.Out, nil
+}
+
+// countSimple will create a simple histogram in s.count.
+// Returns the biggest count.
+// Does not update s.clearCount.
+func (s *Scratch) countSimple(in []byte) (max int, reuse bool) {
+	reuse = true
+	for _, v := range in {
+		s.count[v]++
+	}
+	m := uint32(0)
+	if len(s.prevTable) > 0 {
+		for i, v := range s.count[:] {
+			if v > m {
+				m = v
+			}
+			if v > 0 {
+				s.symbolLen = uint16(i) + 1
+				if i >= len(s.prevTable) {
+					reuse = false
+				} else {
+					if s.prevTable[i].nBits == 0 {
+						reuse = false
+					}
+				}
+			}
+		}
+		return int(m), reuse
+	}
+	for i, v := range s.count[:] {
+		if v > m {
+			m = v
+		}
+		if v > 0 {
+			s.symbolLen = uint16(i) + 1
+		}
+	}
+	return int(m), false
+}
+
+func (s *Scratch) canUseTable(c cTable) bool {
+	if len(c) < int(s.symbolLen) {
+		return false
+	}
+	for i, v := range s.count[:s.symbolLen] {
+		if v != 0 && c[i].nBits == 0 {
+			return false
+		}
+	}
+	return true
+}
+
+func (s *Scratch) validateTable(c cTable) bool {
+	if len(c) < int(s.symbolLen) {
+		return false
+	}
+	for i, v := range s.count[:s.symbolLen] {
+		if v != 0 {
+			if c[i].nBits == 0 {
+				return false
+			}
+			if c[i].nBits > s.actualTableLog {
+				return false
+			}
+		}
+	}
+	return true
+}
+
+// minTableLog provides the minimum logSize to safely represent a distribution.
+func (s *Scratch) minTableLog() uint8 {
+	minBitsSrc := highBit32(uint32(s.br.remain())) + 1
+	minBitsSymbols := highBit32(uint32(s.symbolLen-1)) + 2
+	if minBitsSrc < minBitsSymbols {
+		return uint8(minBitsSrc)
+	}
+	return uint8(minBitsSymbols)
+}
+
+// optimalTableLog calculates and sets the optimal tableLog in s.actualTableLog
+func (s *Scratch) optimalTableLog() {
+	tableLog := s.TableLog
+	minBits := s.minTableLog()
+	maxBitsSrc := uint8(highBit32(uint32(s.br.remain()-1))) - 1
+	if maxBitsSrc < tableLog {
+		// Accuracy can be reduced
+		tableLog = maxBitsSrc
+	}
+	if minBits > tableLog {
+		tableLog = minBits
+	}
+	// Need a minimum to safely represent all symbol values
+	if tableLog < minTablelog {
+		tableLog = minTablelog
+	}
+	if tableLog > tableLogMax {
+		tableLog = tableLogMax
+	}
+	s.actualTableLog = tableLog
+}
+
+type cTableEntry struct {
+	val   uint16
+	nBits uint8
+	// We have 8 bits extra
+}
+
+const huffNodesMask = huffNodesLen - 1
+
+func (s *Scratch) buildCTable() error {
+	s.optimalTableLog()
+	s.huffSort()
+	if cap(s.cTable) < maxSymbolValue+1 {
+		s.cTable = make([]cTableEntry, s.symbolLen, maxSymbolValue+1)
+	} else {
+		s.cTable = s.cTable[:s.symbolLen]
+		for i := range s.cTable {
+			s.cTable[i] = cTableEntry{}
+		}
+	}
+
+	var startNode = int16(s.symbolLen)
+	nonNullRank := s.symbolLen - 1
+
+	nodeNb := startNode
+	huffNode := s.nodes[1 : huffNodesLen+1]
+
+	// This overlays the slice above, but allows "-1" index lookups.
+	// Different from reference implementation.
+	huffNode0 := s.nodes[0 : huffNodesLen+1]
+
+	for huffNode[nonNullRank].count == 0 {
+		nonNullRank--
+	}
+
+	lowS := int16(nonNullRank)
+	nodeRoot := nodeNb + lowS - 1
+	lowN := nodeNb
+	huffNode[nodeNb].count = huffNode[lowS].count + huffNode[lowS-1].count
+	huffNode[lowS].parent, huffNode[lowS-1].parent = uint16(nodeNb), uint16(nodeNb)
+	nodeNb++
+	lowS -= 2
+	for n := nodeNb; n <= nodeRoot; n++ {
+		huffNode[n].count = 1 << 30
+	}
+	// fake entry, strong barrier
+	huffNode0[0].count = 1 << 31
+
+	// create parents
+	for nodeNb <= nodeRoot {
+		var n1, n2 int16
+		if huffNode0[lowS+1].count < huffNode0[lowN+1].count {
+			n1 = lowS
+			lowS--
+		} else {
+			n1 = lowN
+			lowN++
+		}
+		if huffNode0[lowS+1].count < huffNode0[lowN+1].count {
+			n2 = lowS
+			lowS--
+		} else {
+			n2 = lowN
+			lowN++
+		}
+
+		huffNode[nodeNb].count = huffNode0[n1+1].count + huffNode0[n2+1].count
+		huffNode0[n1+1].parent, huffNode0[n2+1].parent = uint16(nodeNb), uint16(nodeNb)
+		nodeNb++
+	}
+
+	// distribute weights (unlimited tree height)
+	huffNode[nodeRoot].nbBits = 0
+	for n := nodeRoot - 1; n >= startNode; n-- {
+		huffNode[n].nbBits = huffNode[huffNode[n].parent].nbBits + 1
+	}
+	for n := uint16(0); n <= nonNullRank; n++ {
+		huffNode[n].nbBits = huffNode[huffNode[n].parent].nbBits + 1
+	}
+	s.actualTableLog = s.setMaxHeight(int(nonNullRank))
+	maxNbBits := s.actualTableLog
+
+	// fill result into tree (val, nbBits)
+	if maxNbBits > tableLogMax {
+		return fmt.Errorf("internal error: maxNbBits (%d) > tableLogMax (%d)", maxNbBits, tableLogMax)
+	}
+	var nbPerRank [tableLogMax + 1]uint16
+	var valPerRank [16]uint16
+	for _, v := range huffNode[:nonNullRank+1] {
+		nbPerRank[v.nbBits]++
+	}
+	// determine stating value per rank
+	{
+		min := uint16(0)
+		for n := maxNbBits; n > 0; n-- {
+			// get starting value within each rank
+			valPerRank[n] = min
+			min += nbPerRank[n]
+			min >>= 1
+		}
+	}
+
+	// push nbBits per symbol, symbol order
+	for _, v := range huffNode[:nonNullRank+1] {
+		s.cTable[v.symbol].nBits = v.nbBits
+	}
+
+	// assign value within rank, symbol order
+	t := s.cTable[:s.symbolLen]
+	for n, val := range t {
+		nbits := val.nBits & 15
+		v := valPerRank[nbits]
+		t[n].val = v
+		valPerRank[nbits] = v + 1
+	}
+
+	return nil
+}
+
+// huffSort will sort symbols, decreasing order.
+func (s *Scratch) huffSort() {
+	type rankPos struct {
+		base    uint32
+		current uint32
+	}
+
+	// Clear nodes
+	nodes := s.nodes[:huffNodesLen+1]
+	s.nodes = nodes
+	nodes = nodes[1 : huffNodesLen+1]
+
+	// Sort into buckets based on length of symbol count.
+	var rank [32]rankPos
+	for _, v := range s.count[:s.symbolLen] {
+		r := highBit32(v+1) & 31
+		rank[r].base++
+	}
+	// maxBitLength is log2(BlockSizeMax) + 1
+	const maxBitLength = 18 + 1
+	for n := maxBitLength; n > 0; n-- {
+		rank[n-1].base += rank[n].base
+	}
+	for n := range rank[:maxBitLength] {
+		rank[n].current = rank[n].base
+	}
+	for n, c := range s.count[:s.symbolLen] {
+		r := (highBit32(c+1) + 1) & 31
+		pos := rank[r].current
+		rank[r].current++
+		prev := nodes[(pos-1)&huffNodesMask]
+		for pos > rank[r].base && c > prev.count {
+			nodes[pos&huffNodesMask] = prev
+			pos--
+			prev = nodes[(pos-1)&huffNodesMask]
+		}
+		nodes[pos&huffNodesMask] = nodeElt{count: c, symbol: byte(n)}
+	}
+}
+
+func (s *Scratch) setMaxHeight(lastNonNull int) uint8 {
+	maxNbBits := s.actualTableLog
+	huffNode := s.nodes[1 : huffNodesLen+1]
+	//huffNode = huffNode[: huffNodesLen]
+
+	largestBits := huffNode[lastNonNull].nbBits
+
+	// early exit : no elt > maxNbBits
+	if largestBits <= maxNbBits {
+		return largestBits
+	}
+	totalCost := int(0)
+	baseCost := int(1) << (largestBits - maxNbBits)
+	n := uint32(lastNonNull)
+
+	for huffNode[n].nbBits > maxNbBits {
+		totalCost += baseCost - (1 << (largestBits - huffNode[n].nbBits))
+		huffNode[n].nbBits = maxNbBits
+		n--
+	}
+	// n stops at huffNode[n].nbBits <= maxNbBits
+
+	for huffNode[n].nbBits == maxNbBits {
+		n--
+	}
+	// n end at index of smallest symbol using < maxNbBits
+
+	// renorm totalCost
+	totalCost >>= largestBits - maxNbBits /* note : totalCost is necessarily a multiple of baseCost */
+
+	// repay normalized cost
+	{
+		const noSymbol = 0xF0F0F0F0
+		var rankLast [tableLogMax + 2]uint32
+
+		for i := range rankLast[:] {
+			rankLast[i] = noSymbol
+		}
+
+		// Get pos of last (smallest) symbol per rank
+		{
+			currentNbBits := maxNbBits
+			for pos := int(n); pos >= 0; pos-- {
+				if huffNode[pos].nbBits >= currentNbBits {
+					continue
+				}
+				currentNbBits = huffNode[pos].nbBits // < maxNbBits
+				rankLast[maxNbBits-currentNbBits] = uint32(pos)
+			}
+		}
+
+		for totalCost > 0 {
+			nBitsToDecrease := uint8(highBit32(uint32(totalCost))) + 1
+
+			for ; nBitsToDecrease > 1; nBitsToDecrease-- {
+				highPos := rankLast[nBitsToDecrease]
+				lowPos := rankLast[nBitsToDecrease-1]
+				if highPos == noSymbol {
+					continue
+				}
+				if lowPos == noSymbol {
+					break
+				}
+				highTotal := huffNode[highPos].count
+				lowTotal := 2 * huffNode[lowPos].count
+				if highTotal <= lowTotal {
+					break
+				}
+			}
+			// only triggered when no more rank 1 symbol left => find closest one (note : there is necessarily at least one !)
+			// HUF_MAX_TABLELOG test just to please gcc 5+; but it should not be necessary
+			// FIXME: try to remove
+			for (nBitsToDecrease <= tableLogMax) && (rankLast[nBitsToDecrease] == noSymbol) {
+				nBitsToDecrease++
+			}
+			totalCost -= 1 << (nBitsToDecrease - 1)
+			if rankLast[nBitsToDecrease-1] == noSymbol {
+				// this rank is no longer empty
+				rankLast[nBitsToDecrease-1] = rankLast[nBitsToDecrease]
+			}
+			huffNode[rankLast[nBitsToDecrease]].nbBits++
+			if rankLast[nBitsToDecrease] == 0 {
+				/* special case, reached largest symbol */
+				rankLast[nBitsToDecrease] = noSymbol
+			} else {
+				rankLast[nBitsToDecrease]--
+				if huffNode[rankLast[nBitsToDecrease]].nbBits != maxNbBits-nBitsToDecrease {
+					rankLast[nBitsToDecrease] = noSymbol /* this rank is now empty */
+				}
+			}
+		}
+
+		for totalCost < 0 { /* Sometimes, cost correction overshoot */
+			if rankLast[1] == noSymbol { /* special case : no rank 1 symbol (using maxNbBits-1); let's create one from largest rank 0 (using maxNbBits) */
+				for huffNode[n].nbBits == maxNbBits {
+					n--
+				}
+				huffNode[n+1].nbBits--
+				rankLast[1] = n + 1
+				totalCost++
+				continue
+			}
+			huffNode[rankLast[1]+1].nbBits--
+			rankLast[1]++
+			totalCost++
+		}
+	}
+	return maxNbBits
+}
+
+type nodeElt struct {
+	count  uint32
+	parent uint16
+	symbol byte
+	nbBits uint8
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/decompress.go b/vendor/github.com/klauspost/compress/huff0/decompress.go
new file mode 100644
index 0000000..41703bb
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/decompress.go
@@ -0,0 +1,1164 @@
+package huff0
+
+import (
+	"errors"
+	"fmt"
+	"io"
+
+	"github.com/klauspost/compress/fse"
+)
+
+type dTable struct {
+	single []dEntrySingle
+	double []dEntryDouble
+}
+
+// single-symbols decoding
+type dEntrySingle struct {
+	entry uint16
+}
+
+// double-symbols decoding
+type dEntryDouble struct {
+	seq   uint16
+	nBits uint8
+	len   uint8
+}
+
+// Uses special code for all tables that are < 8 bits.
+const use8BitTables = true
+
+// ReadTable will read a table from the input.
+// The size of the input may be larger than the table definition.
+// Any content remaining after the table definition will be returned.
+// If no Scratch is provided a new one is allocated.
+// The returned Scratch can be used for encoding or decoding input using this table.
+func ReadTable(in []byte, s *Scratch) (s2 *Scratch, remain []byte, err error) {
+	s, err = s.prepare(in)
+	if err != nil {
+		return s, nil, err
+	}
+	if len(in) <= 1 {
+		return s, nil, errors.New("input too small for table")
+	}
+	iSize := in[0]
+	in = in[1:]
+	if iSize >= 128 {
+		// Uncompressed
+		oSize := iSize - 127
+		iSize = (oSize + 1) / 2
+		if int(iSize) > len(in) {
+			return s, nil, errors.New("input too small for table")
+		}
+		for n := uint8(0); n < oSize; n += 2 {
+			v := in[n/2]
+			s.huffWeight[n] = v >> 4
+			s.huffWeight[n+1] = v & 15
+		}
+		s.symbolLen = uint16(oSize)
+		in = in[iSize:]
+	} else {
+		if len(in) < int(iSize) {
+			return s, nil, fmt.Errorf("input too small for table, want %d bytes, have %d", iSize, len(in))
+		}
+		// FSE compressed weights
+		s.fse.DecompressLimit = 255
+		hw := s.huffWeight[:]
+		s.fse.Out = hw
+		b, err := fse.Decompress(in[:iSize], s.fse)
+		s.fse.Out = nil
+		if err != nil {
+			return s, nil, err
+		}
+		if len(b) > 255 {
+			return s, nil, errors.New("corrupt input: output table too large")
+		}
+		s.symbolLen = uint16(len(b))
+		in = in[iSize:]
+	}
+
+	// collect weight stats
+	var rankStats [16]uint32
+	weightTotal := uint32(0)
+	for _, v := range s.huffWeight[:s.symbolLen] {
+		if v > tableLogMax {
+			return s, nil, errors.New("corrupt input: weight too large")
+		}
+		v2 := v & 15
+		rankStats[v2]++
+		// (1 << (v2-1)) is slower since the compiler cannot prove that v2 isn't 0.
+		weightTotal += (1 << v2) >> 1
+	}
+	if weightTotal == 0 {
+		return s, nil, errors.New("corrupt input: weights zero")
+	}
+
+	// get last non-null symbol weight (implied, total must be 2^n)
+	{
+		tableLog := highBit32(weightTotal) + 1
+		if tableLog > tableLogMax {
+			return s, nil, errors.New("corrupt input: tableLog too big")
+		}
+		s.actualTableLog = uint8(tableLog)
+		// determine last weight
+		{
+			total := uint32(1) << tableLog
+			rest := total - weightTotal
+			verif := uint32(1) << highBit32(rest)
+			lastWeight := highBit32(rest) + 1
+			if verif != rest {
+				// last value must be a clean power of 2
+				return s, nil, errors.New("corrupt input: last value not power of two")
+			}
+			s.huffWeight[s.symbolLen] = uint8(lastWeight)
+			s.symbolLen++
+			rankStats[lastWeight]++
+		}
+	}
+
+	if (rankStats[1] < 2) || (rankStats[1]&1 != 0) {
+		// by construction : at least 2 elts of rank 1, must be even
+		return s, nil, errors.New("corrupt input: min elt size, even check failed ")
+	}
+
+	// TODO: Choose between single/double symbol decoding
+
+	// Calculate starting value for each rank
+	{
+		var nextRankStart uint32
+		for n := uint8(1); n < s.actualTableLog+1; n++ {
+			current := nextRankStart
+			nextRankStart += rankStats[n] << (n - 1)
+			rankStats[n] = current
+		}
+	}
+
+	// fill DTable (always full size)
+	tSize := 1 << tableLogMax
+	if len(s.dt.single) != tSize {
+		s.dt.single = make([]dEntrySingle, tSize)
+	}
+	cTable := s.prevTable
+	if cap(cTable) < maxSymbolValue+1 {
+		cTable = make([]cTableEntry, 0, maxSymbolValue+1)
+	}
+	cTable = cTable[:maxSymbolValue+1]
+	s.prevTable = cTable[:s.symbolLen]
+	s.prevTableLog = s.actualTableLog
+
+	for n, w := range s.huffWeight[:s.symbolLen] {
+		if w == 0 {
+			cTable[n] = cTableEntry{
+				val:   0,
+				nBits: 0,
+			}
+			continue
+		}
+		length := (uint32(1) << w) >> 1
+		d := dEntrySingle{
+			entry: uint16(s.actualTableLog+1-w) | (uint16(n) << 8),
+		}
+
+		rank := &rankStats[w]
+		cTable[n] = cTableEntry{
+			val:   uint16(*rank >> (w - 1)),
+			nBits: uint8(d.entry),
+		}
+
+		single := s.dt.single[*rank : *rank+length]
+		for i := range single {
+			single[i] = d
+		}
+		*rank += length
+	}
+
+	return s, in, nil
+}
+
+// Decompress1X will decompress a 1X encoded stream.
+// The length of the supplied input must match the end of a block exactly.
+// Before this is called, the table must be initialized with ReadTable unless
+// the encoder re-used the table.
+// deprecated: Use the stateless Decoder() to get a concurrent version.
+func (s *Scratch) Decompress1X(in []byte) (out []byte, err error) {
+	if cap(s.Out) < s.MaxDecodedSize {
+		s.Out = make([]byte, s.MaxDecodedSize)
+	}
+	s.Out = s.Out[:0:s.MaxDecodedSize]
+	s.Out, err = s.Decoder().Decompress1X(s.Out, in)
+	return s.Out, err
+}
+
+// Decompress4X will decompress a 4X encoded stream.
+// Before this is called, the table must be initialized with ReadTable unless
+// the encoder re-used the table.
+// The length of the supplied input must match the end of a block exactly.
+// The destination size of the uncompressed data must be known and provided.
+// deprecated: Use the stateless Decoder() to get a concurrent version.
+func (s *Scratch) Decompress4X(in []byte, dstSize int) (out []byte, err error) {
+	if dstSize > s.MaxDecodedSize {
+		return nil, ErrMaxDecodedSizeExceeded
+	}
+	if cap(s.Out) < dstSize {
+		s.Out = make([]byte, s.MaxDecodedSize)
+	}
+	s.Out = s.Out[:0:dstSize]
+	s.Out, err = s.Decoder().Decompress4X(s.Out, in)
+	return s.Out, err
+}
+
+// Decoder will return a stateless decoder that can be used by multiple
+// decompressors concurrently.
+// Before this is called, the table must be initialized with ReadTable.
+// The Decoder is still linked to the scratch buffer so that cannot be reused.
+// However, it is safe to discard the scratch.
+func (s *Scratch) Decoder() *Decoder {
+	return &Decoder{
+		dt:             s.dt,
+		actualTableLog: s.actualTableLog,
+	}
+}
+
+// Decoder provides stateless decoding.
+type Decoder struct {
+	dt             dTable
+	actualTableLog uint8
+}
+
+// Decompress1X will decompress a 1X encoded stream.
+// The cap of the output buffer will be the maximum decompressed size.
+// The length of the supplied input must match the end of a block exactly.
+func (d *Decoder) Decompress1X(dst, src []byte) ([]byte, error) {
+	if len(d.dt.single) == 0 {
+		return nil, errors.New("no table loaded")
+	}
+	if use8BitTables && d.actualTableLog <= 8 {
+		return d.decompress1X8Bit(dst, src)
+	}
+	var br bitReaderShifted
+	err := br.init(src)
+	if err != nil {
+		return dst, err
+	}
+	maxDecodedSize := cap(dst)
+	dst = dst[:0]
+
+	// Avoid bounds check by always having full sized table.
+	const tlSize = 1 << tableLogMax
+	const tlMask = tlSize - 1
+	dt := d.dt.single[:tlSize]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+
+	for br.off >= 8 {
+		br.fillFast()
+		v := dt[br.peekBitsFast(d.actualTableLog)&tlMask]
+		br.advance(uint8(v.entry))
+		buf[off+0] = uint8(v.entry >> 8)
+
+		v = dt[br.peekBitsFast(d.actualTableLog)&tlMask]
+		br.advance(uint8(v.entry))
+		buf[off+1] = uint8(v.entry >> 8)
+
+		// Refill
+		br.fillFast()
+
+		v = dt[br.peekBitsFast(d.actualTableLog)&tlMask]
+		br.advance(uint8(v.entry))
+		buf[off+2] = uint8(v.entry >> 8)
+
+		v = dt[br.peekBitsFast(d.actualTableLog)&tlMask]
+		br.advance(uint8(v.entry))
+		buf[off+3] = uint8(v.entry >> 8)
+
+		off += 4
+		if off == 0 {
+			if len(dst)+256 > maxDecodedSize {
+				br.close()
+				return nil, ErrMaxDecodedSizeExceeded
+			}
+			dst = append(dst, buf[:]...)
+		}
+	}
+
+	if len(dst)+int(off) > maxDecodedSize {
+		br.close()
+		return nil, ErrMaxDecodedSizeExceeded
+	}
+	dst = append(dst, buf[:off]...)
+
+	// br < 8, so uint8 is fine
+	bitsLeft := uint8(br.off)*8 + 64 - br.bitsRead
+	for bitsLeft > 0 {
+		br.fill()
+		if false && br.bitsRead >= 32 {
+			if br.off >= 4 {
+				v := br.in[br.off-4:]
+				v = v[:4]
+				low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+				br.value = (br.value << 32) | uint64(low)
+				br.bitsRead -= 32
+				br.off -= 4
+			} else {
+				for br.off > 0 {
+					br.value = (br.value << 8) | uint64(br.in[br.off-1])
+					br.bitsRead -= 8
+					br.off--
+				}
+			}
+		}
+		if len(dst) >= maxDecodedSize {
+			br.close()
+			return nil, ErrMaxDecodedSizeExceeded
+		}
+		v := d.dt.single[br.peekBitsFast(d.actualTableLog)&tlMask]
+		nBits := uint8(v.entry)
+		br.advance(nBits)
+		bitsLeft -= nBits
+		dst = append(dst, uint8(v.entry>>8))
+	}
+	return dst, br.close()
+}
+
+// decompress1X8Bit will decompress a 1X encoded stream with tablelog <= 8.
+// The cap of the output buffer will be the maximum decompressed size.
+// The length of the supplied input must match the end of a block exactly.
+func (d *Decoder) decompress1X8Bit(dst, src []byte) ([]byte, error) {
+	if d.actualTableLog == 8 {
+		return d.decompress1X8BitExactly(dst, src)
+	}
+	var br bitReaderBytes
+	err := br.init(src)
+	if err != nil {
+		return dst, err
+	}
+	maxDecodedSize := cap(dst)
+	dst = dst[:0]
+
+	// Avoid bounds check by always having full sized table.
+	dt := d.dt.single[:256]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+
+	shift := (8 - d.actualTableLog) & 7
+
+	//fmt.Printf("mask: %b, tl:%d\n", mask, d.actualTableLog)
+	for br.off >= 4 {
+		br.fillFast()
+		v := dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+0] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+1] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+2] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+3] = uint8(v.entry >> 8)
+
+		off += 4
+		if off == 0 {
+			if len(dst)+256 > maxDecodedSize {
+				br.close()
+				return nil, ErrMaxDecodedSizeExceeded
+			}
+			dst = append(dst, buf[:]...)
+		}
+	}
+
+	if len(dst)+int(off) > maxDecodedSize {
+		br.close()
+		return nil, ErrMaxDecodedSizeExceeded
+	}
+	dst = append(dst, buf[:off]...)
+
+	// br < 4, so uint8 is fine
+	bitsLeft := int8(uint8(br.off)*8 + (64 - br.bitsRead))
+	for bitsLeft > 0 {
+		if br.bitsRead >= 64-8 {
+			for br.off > 0 {
+				br.value |= uint64(br.in[br.off-1]) << (br.bitsRead - 8)
+				br.bitsRead -= 8
+				br.off--
+			}
+		}
+		if len(dst) >= maxDecodedSize {
+			br.close()
+			return nil, ErrMaxDecodedSizeExceeded
+		}
+		v := dt[br.peekByteFast()>>shift]
+		nBits := uint8(v.entry)
+		br.advance(nBits)
+		bitsLeft -= int8(nBits)
+		dst = append(dst, uint8(v.entry>>8))
+	}
+	return dst, br.close()
+}
+
+// decompress1X8Bit will decompress a 1X encoded stream with tablelog <= 8.
+// The cap of the output buffer will be the maximum decompressed size.
+// The length of the supplied input must match the end of a block exactly.
+func (d *Decoder) decompress1X8BitExactly(dst, src []byte) ([]byte, error) {
+	var br bitReaderBytes
+	err := br.init(src)
+	if err != nil {
+		return dst, err
+	}
+	maxDecodedSize := cap(dst)
+	dst = dst[:0]
+
+	// Avoid bounds check by always having full sized table.
+	dt := d.dt.single[:256]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+
+	const shift = 0
+
+	//fmt.Printf("mask: %b, tl:%d\n", mask, d.actualTableLog)
+	for br.off >= 4 {
+		br.fillFast()
+		v := dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+0] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+1] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+2] = uint8(v.entry >> 8)
+
+		v = dt[br.peekByteFast()>>shift]
+		br.advance(uint8(v.entry))
+		buf[off+3] = uint8(v.entry >> 8)
+
+		off += 4
+		if off == 0 {
+			if len(dst)+256 > maxDecodedSize {
+				br.close()
+				return nil, ErrMaxDecodedSizeExceeded
+			}
+			dst = append(dst, buf[:]...)
+		}
+	}
+
+	if len(dst)+int(off) > maxDecodedSize {
+		br.close()
+		return nil, ErrMaxDecodedSizeExceeded
+	}
+	dst = append(dst, buf[:off]...)
+
+	// br < 4, so uint8 is fine
+	bitsLeft := int8(uint8(br.off)*8 + (64 - br.bitsRead))
+	for bitsLeft > 0 {
+		if br.bitsRead >= 64-8 {
+			for br.off > 0 {
+				br.value |= uint64(br.in[br.off-1]) << (br.bitsRead - 8)
+				br.bitsRead -= 8
+				br.off--
+			}
+		}
+		if len(dst) >= maxDecodedSize {
+			br.close()
+			return nil, ErrMaxDecodedSizeExceeded
+		}
+		v := dt[br.peekByteFast()>>shift]
+		nBits := uint8(v.entry)
+		br.advance(nBits)
+		bitsLeft -= int8(nBits)
+		dst = append(dst, uint8(v.entry>>8))
+	}
+	return dst, br.close()
+}
+
+// Decompress4X will decompress a 4X encoded stream.
+// The length of the supplied input must match the end of a block exactly.
+// The *capacity* of the dst slice must match the destination size of
+// the uncompressed data exactly.
+func (d *Decoder) Decompress4X(dst, src []byte) ([]byte, error) {
+	if len(d.dt.single) == 0 {
+		return nil, errors.New("no table loaded")
+	}
+	if len(src) < 6+(4*1) {
+		return nil, errors.New("input too small")
+	}
+	if use8BitTables && d.actualTableLog <= 8 {
+		return d.decompress4X8bit(dst, src)
+	}
+
+	var br [4]bitReaderShifted
+	start := 6
+	for i := 0; i < 3; i++ {
+		length := int(src[i*2]) | (int(src[i*2+1]) << 8)
+		if start+length >= len(src) {
+			return nil, errors.New("truncated input (or invalid offset)")
+		}
+		err := br[i].init(src[start : start+length])
+		if err != nil {
+			return nil, err
+		}
+		start += length
+	}
+	err := br[3].init(src[start:])
+	if err != nil {
+		return nil, err
+	}
+
+	// destination, offset to match first output
+	dstSize := cap(dst)
+	dst = dst[:dstSize]
+	out := dst
+	dstEvery := (dstSize + 3) / 4
+
+	const tlSize = 1 << tableLogMax
+	const tlMask = tlSize - 1
+	single := d.dt.single[:tlSize]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+	var decoded int
+
+	// Decode 2 values from each decoder/loop.
+	const bufoff = 256 / 4
+	for {
+		if br[0].off < 4 || br[1].off < 4 || br[2].off < 4 || br[3].off < 4 {
+			break
+		}
+
+		{
+			const stream = 0
+			const stream2 = 1
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			val := br[stream].peekBitsFast(d.actualTableLog)
+			v := single[val&tlMask]
+			br[stream].advance(uint8(v.entry))
+			buf[off+bufoff*stream] = uint8(v.entry >> 8)
+
+			val2 := br[stream2].peekBitsFast(d.actualTableLog)
+			v2 := single[val2&tlMask]
+			br[stream2].advance(uint8(v2.entry))
+			buf[off+bufoff*stream2] = uint8(v2.entry >> 8)
+
+			val = br[stream].peekBitsFast(d.actualTableLog)
+			v = single[val&tlMask]
+			br[stream].advance(uint8(v.entry))
+			buf[off+bufoff*stream+1] = uint8(v.entry >> 8)
+
+			val2 = br[stream2].peekBitsFast(d.actualTableLog)
+			v2 = single[val2&tlMask]
+			br[stream2].advance(uint8(v2.entry))
+			buf[off+bufoff*stream2+1] = uint8(v2.entry >> 8)
+		}
+
+		{
+			const stream = 2
+			const stream2 = 3
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			val := br[stream].peekBitsFast(d.actualTableLog)
+			v := single[val&tlMask]
+			br[stream].advance(uint8(v.entry))
+			buf[off+bufoff*stream] = uint8(v.entry >> 8)
+
+			val2 := br[stream2].peekBitsFast(d.actualTableLog)
+			v2 := single[val2&tlMask]
+			br[stream2].advance(uint8(v2.entry))
+			buf[off+bufoff*stream2] = uint8(v2.entry >> 8)
+
+			val = br[stream].peekBitsFast(d.actualTableLog)
+			v = single[val&tlMask]
+			br[stream].advance(uint8(v.entry))
+			buf[off+bufoff*stream+1] = uint8(v.entry >> 8)
+
+			val2 = br[stream2].peekBitsFast(d.actualTableLog)
+			v2 = single[val2&tlMask]
+			br[stream2].advance(uint8(v2.entry))
+			buf[off+bufoff*stream2+1] = uint8(v2.entry >> 8)
+		}
+
+		off += 2
+
+		if off == bufoff {
+			if bufoff > dstEvery {
+				return nil, errors.New("corruption detected: stream overrun 1")
+			}
+			copy(out, buf[:bufoff])
+			copy(out[dstEvery:], buf[bufoff:bufoff*2])
+			copy(out[dstEvery*2:], buf[bufoff*2:bufoff*3])
+			copy(out[dstEvery*3:], buf[bufoff*3:bufoff*4])
+			off = 0
+			out = out[bufoff:]
+			decoded += 256
+			// There must at least be 3 buffers left.
+			if len(out) < dstEvery*3 {
+				return nil, errors.New("corruption detected: stream overrun 2")
+			}
+		}
+	}
+	if off > 0 {
+		ioff := int(off)
+		if len(out) < dstEvery*3+ioff {
+			return nil, errors.New("corruption detected: stream overrun 3")
+		}
+		copy(out, buf[:off])
+		copy(out[dstEvery:dstEvery+ioff], buf[bufoff:bufoff*2])
+		copy(out[dstEvery*2:dstEvery*2+ioff], buf[bufoff*2:bufoff*3])
+		copy(out[dstEvery*3:dstEvery*3+ioff], buf[bufoff*3:bufoff*4])
+		decoded += int(off) * 4
+		out = out[off:]
+	}
+
+	// Decode remaining.
+	for i := range br {
+		offset := dstEvery * i
+		br := &br[i]
+		bitsLeft := br.off*8 + uint(64-br.bitsRead)
+		for bitsLeft > 0 {
+			br.fill()
+			if false && br.bitsRead >= 32 {
+				if br.off >= 4 {
+					v := br.in[br.off-4:]
+					v = v[:4]
+					low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+					br.value = (br.value << 32) | uint64(low)
+					br.bitsRead -= 32
+					br.off -= 4
+				} else {
+					for br.off > 0 {
+						br.value = (br.value << 8) | uint64(br.in[br.off-1])
+						br.bitsRead -= 8
+						br.off--
+					}
+				}
+			}
+			// end inline...
+			if offset >= len(out) {
+				return nil, errors.New("corruption detected: stream overrun 4")
+			}
+
+			// Read value and increment offset.
+			val := br.peekBitsFast(d.actualTableLog)
+			v := single[val&tlMask].entry
+			nBits := uint8(v)
+			br.advance(nBits)
+			bitsLeft -= uint(nBits)
+			out[offset] = uint8(v >> 8)
+			offset++
+		}
+		decoded += offset - dstEvery*i
+		err = br.close()
+		if err != nil {
+			return nil, err
+		}
+	}
+	if dstSize != decoded {
+		return nil, errors.New("corruption detected: short output block")
+	}
+	return dst, nil
+}
+
+// Decompress4X will decompress a 4X encoded stream.
+// The length of the supplied input must match the end of a block exactly.
+// The *capacity* of the dst slice must match the destination size of
+// the uncompressed data exactly.
+func (d *Decoder) decompress4X8bit(dst, src []byte) ([]byte, error) {
+	if d.actualTableLog == 8 {
+		return d.decompress4X8bitExactly(dst, src)
+	}
+
+	var br [4]bitReaderBytes
+	start := 6
+	for i := 0; i < 3; i++ {
+		length := int(src[i*2]) | (int(src[i*2+1]) << 8)
+		if start+length >= len(src) {
+			return nil, errors.New("truncated input (or invalid offset)")
+		}
+		err := br[i].init(src[start : start+length])
+		if err != nil {
+			return nil, err
+		}
+		start += length
+	}
+	err := br[3].init(src[start:])
+	if err != nil {
+		return nil, err
+	}
+
+	// destination, offset to match first output
+	dstSize := cap(dst)
+	dst = dst[:dstSize]
+	out := dst
+	dstEvery := (dstSize + 3) / 4
+
+	shift := (8 - d.actualTableLog) & 7
+
+	const tlSize = 1 << 8
+	const tlMask = tlSize - 1
+	single := d.dt.single[:tlSize]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+	var decoded int
+
+	// Decode 4 values from each decoder/loop.
+	const bufoff = 256 / 4
+	for {
+		if br[0].off < 4 || br[1].off < 4 || br[2].off < 4 || br[3].off < 4 {
+			break
+		}
+
+		{
+			// Interleave 2 decodes.
+			const stream = 0
+			const stream2 = 1
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			v := single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 := single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+1] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+1] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+2] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+3] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+3] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+		}
+
+		{
+			const stream = 2
+			const stream2 = 3
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			v := single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 := single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+1] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+1] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+2] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+3] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+3] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+		}
+
+		off += 4
+
+		if off == bufoff {
+			if bufoff > dstEvery {
+				return nil, errors.New("corruption detected: stream overrun 1")
+			}
+			copy(out, buf[:bufoff])
+			copy(out[dstEvery:], buf[bufoff:bufoff*2])
+			copy(out[dstEvery*2:], buf[bufoff*2:bufoff*3])
+			copy(out[dstEvery*3:], buf[bufoff*3:bufoff*4])
+			off = 0
+			out = out[bufoff:]
+			decoded += 256
+			// There must at least be 3 buffers left.
+			if len(out) < dstEvery*3 {
+				return nil, errors.New("corruption detected: stream overrun 2")
+			}
+		}
+	}
+	if off > 0 {
+		ioff := int(off)
+		if len(out) < dstEvery*3+ioff {
+			return nil, errors.New("corruption detected: stream overrun 3")
+		}
+		copy(out, buf[:off])
+		copy(out[dstEvery:dstEvery+ioff], buf[bufoff:bufoff*2])
+		copy(out[dstEvery*2:dstEvery*2+ioff], buf[bufoff*2:bufoff*3])
+		copy(out[dstEvery*3:dstEvery*3+ioff], buf[bufoff*3:bufoff*4])
+		decoded += int(off) * 4
+		out = out[off:]
+	}
+
+	// Decode remaining.
+	for i := range br {
+		offset := dstEvery * i
+		br := &br[i]
+		bitsLeft := int(br.off*8) + int(64-br.bitsRead)
+		for bitsLeft > 0 {
+			if br.finished() {
+				return nil, io.ErrUnexpectedEOF
+			}
+			if br.bitsRead >= 56 {
+				if br.off >= 4 {
+					v := br.in[br.off-4:]
+					v = v[:4]
+					low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+					br.value |= uint64(low) << (br.bitsRead - 32)
+					br.bitsRead -= 32
+					br.off -= 4
+				} else {
+					for br.off > 0 {
+						br.value |= uint64(br.in[br.off-1]) << (br.bitsRead - 8)
+						br.bitsRead -= 8
+						br.off--
+					}
+				}
+			}
+			// end inline...
+			if offset >= len(out) {
+				return nil, errors.New("corruption detected: stream overrun 4")
+			}
+
+			// Read value and increment offset.
+			v := single[br.peekByteFast()>>shift].entry
+			nBits := uint8(v)
+			br.advance(nBits)
+			bitsLeft -= int(nBits)
+			out[offset] = uint8(v >> 8)
+			offset++
+		}
+		decoded += offset - dstEvery*i
+		err = br.close()
+		if err != nil {
+			return nil, err
+		}
+	}
+	if dstSize != decoded {
+		return nil, errors.New("corruption detected: short output block")
+	}
+	return dst, nil
+}
+
+// Decompress4X will decompress a 4X encoded stream.
+// The length of the supplied input must match the end of a block exactly.
+// The *capacity* of the dst slice must match the destination size of
+// the uncompressed data exactly.
+func (d *Decoder) decompress4X8bitExactly(dst, src []byte) ([]byte, error) {
+	var br [4]bitReaderBytes
+	start := 6
+	for i := 0; i < 3; i++ {
+		length := int(src[i*2]) | (int(src[i*2+1]) << 8)
+		if start+length >= len(src) {
+			return nil, errors.New("truncated input (or invalid offset)")
+		}
+		err := br[i].init(src[start : start+length])
+		if err != nil {
+			return nil, err
+		}
+		start += length
+	}
+	err := br[3].init(src[start:])
+	if err != nil {
+		return nil, err
+	}
+
+	// destination, offset to match first output
+	dstSize := cap(dst)
+	dst = dst[:dstSize]
+	out := dst
+	dstEvery := (dstSize + 3) / 4
+
+	const shift = 0
+	const tlSize = 1 << 8
+	const tlMask = tlSize - 1
+	single := d.dt.single[:tlSize]
+
+	// Use temp table to avoid bound checks/append penalty.
+	var buf [256]byte
+	var off uint8
+	var decoded int
+
+	// Decode 4 values from each decoder/loop.
+	const bufoff = 256 / 4
+	for {
+		if br[0].off < 4 || br[1].off < 4 || br[2].off < 4 || br[3].off < 4 {
+			break
+		}
+
+		{
+			// Interleave 2 decodes.
+			const stream = 0
+			const stream2 = 1
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			v := single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 := single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+1] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+1] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+2] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+3] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+3] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+		}
+
+		{
+			const stream = 2
+			const stream2 = 3
+			br[stream].fillFast()
+			br[stream2].fillFast()
+
+			v := single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 := single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+1] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+1] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+2] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+2] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+
+			v = single[br[stream].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream+3] = uint8(v >> 8)
+			br[stream].advance(uint8(v))
+
+			v2 = single[br[stream2].peekByteFast()>>shift].entry
+			buf[off+bufoff*stream2+3] = uint8(v2 >> 8)
+			br[stream2].advance(uint8(v2))
+		}
+
+		off += 4
+
+		if off == bufoff {
+			if bufoff > dstEvery {
+				return nil, errors.New("corruption detected: stream overrun 1")
+			}
+			copy(out, buf[:bufoff])
+			copy(out[dstEvery:], buf[bufoff:bufoff*2])
+			copy(out[dstEvery*2:], buf[bufoff*2:bufoff*3])
+			copy(out[dstEvery*3:], buf[bufoff*3:bufoff*4])
+			off = 0
+			out = out[bufoff:]
+			decoded += 256
+			// There must at least be 3 buffers left.
+			if len(out) < dstEvery*3 {
+				return nil, errors.New("corruption detected: stream overrun 2")
+			}
+		}
+	}
+	if off > 0 {
+		ioff := int(off)
+		if len(out) < dstEvery*3+ioff {
+			return nil, errors.New("corruption detected: stream overrun 3")
+		}
+		copy(out, buf[:off])
+		copy(out[dstEvery:dstEvery+ioff], buf[bufoff:bufoff*2])
+		copy(out[dstEvery*2:dstEvery*2+ioff], buf[bufoff*2:bufoff*3])
+		copy(out[dstEvery*3:dstEvery*3+ioff], buf[bufoff*3:bufoff*4])
+		decoded += int(off) * 4
+		out = out[off:]
+	}
+
+	// Decode remaining.
+	for i := range br {
+		offset := dstEvery * i
+		br := &br[i]
+		bitsLeft := int(br.off*8) + int(64-br.bitsRead)
+		for bitsLeft > 0 {
+			if br.finished() {
+				return nil, io.ErrUnexpectedEOF
+			}
+			if br.bitsRead >= 56 {
+				if br.off >= 4 {
+					v := br.in[br.off-4:]
+					v = v[:4]
+					low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+					br.value |= uint64(low) << (br.bitsRead - 32)
+					br.bitsRead -= 32
+					br.off -= 4
+				} else {
+					for br.off > 0 {
+						br.value |= uint64(br.in[br.off-1]) << (br.bitsRead - 8)
+						br.bitsRead -= 8
+						br.off--
+					}
+				}
+			}
+			// end inline...
+			if offset >= len(out) {
+				return nil, errors.New("corruption detected: stream overrun 4")
+			}
+
+			// Read value and increment offset.
+			v := single[br.peekByteFast()>>shift].entry
+			nBits := uint8(v)
+			br.advance(nBits)
+			bitsLeft -= int(nBits)
+			out[offset] = uint8(v >> 8)
+			offset++
+		}
+		decoded += offset - dstEvery*i
+		err = br.close()
+		if err != nil {
+			return nil, err
+		}
+	}
+	if dstSize != decoded {
+		return nil, errors.New("corruption detected: short output block")
+	}
+	return dst, nil
+}
+
+// matches will compare a decoding table to a coding table.
+// Errors are written to the writer.
+// Nothing will be written if table is ok.
+func (s *Scratch) matches(ct cTable, w io.Writer) {
+	if s == nil || len(s.dt.single) == 0 {
+		return
+	}
+	dt := s.dt.single[:1<<s.actualTableLog]
+	tablelog := s.actualTableLog
+	ok := 0
+	broken := 0
+	for sym, enc := range ct {
+		errs := 0
+		broken++
+		if enc.nBits == 0 {
+			for _, dec := range dt {
+				if uint8(dec.entry>>8) == byte(sym) {
+					fmt.Fprintf(w, "symbol %x has decoder, but no encoder\n", sym)
+					errs++
+					break
+				}
+			}
+			if errs == 0 {
+				broken--
+			}
+			continue
+		}
+		// Unused bits in input
+		ub := tablelog - enc.nBits
+		top := enc.val << ub
+		// decoder looks at top bits.
+		dec := dt[top]
+		if uint8(dec.entry) != enc.nBits {
+			fmt.Fprintf(w, "symbol 0x%x bit size mismatch (enc: %d, dec:%d).\n", sym, enc.nBits, uint8(dec.entry))
+			errs++
+		}
+		if uint8(dec.entry>>8) != uint8(sym) {
+			fmt.Fprintf(w, "symbol 0x%x decoder output mismatch (enc: %d, dec:%d).\n", sym, sym, uint8(dec.entry>>8))
+			errs++
+		}
+		if errs > 0 {
+			fmt.Fprintf(w, "%d errros in base, stopping\n", errs)
+			continue
+		}
+		// Ensure that all combinations are covered.
+		for i := uint16(0); i < (1 << ub); i++ {
+			vval := top | i
+			dec := dt[vval]
+			if uint8(dec.entry) != enc.nBits {
+				fmt.Fprintf(w, "symbol 0x%x bit size mismatch (enc: %d, dec:%d).\n", vval, enc.nBits, uint8(dec.entry))
+				errs++
+			}
+			if uint8(dec.entry>>8) != uint8(sym) {
+				fmt.Fprintf(w, "symbol 0x%x decoder output mismatch (enc: %d, dec:%d).\n", vval, sym, uint8(dec.entry>>8))
+				errs++
+			}
+			if errs > 20 {
+				fmt.Fprintf(w, "%d errros, stopping\n", errs)
+				break
+			}
+		}
+		if errs == 0 {
+			ok++
+			broken--
+		}
+	}
+	if broken > 0 {
+		fmt.Fprintf(w, "%d broken, %d ok\n", broken, ok)
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/huff0/huff0.go b/vendor/github.com/klauspost/compress/huff0/huff0.go
new file mode 100644
index 0000000..7ec2022
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/huff0/huff0.go
@@ -0,0 +1,273 @@
+// Package huff0 provides fast huffman encoding as used in zstd.
+//
+// See README.md at https://github.com/klauspost/compress/tree/master/huff0 for details.
+package huff0
+
+import (
+	"errors"
+	"fmt"
+	"math"
+	"math/bits"
+
+	"github.com/klauspost/compress/fse"
+)
+
+const (
+	maxSymbolValue = 255
+
+	// zstandard limits tablelog to 11, see:
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#huffman-tree-description
+	tableLogMax     = 11
+	tableLogDefault = 11
+	minTablelog     = 5
+	huffNodesLen    = 512
+
+	// BlockSizeMax is maximum input size for a single block uncompressed.
+	BlockSizeMax = 1<<18 - 1
+)
+
+var (
+	// ErrIncompressible is returned when input is judged to be too hard to compress.
+	ErrIncompressible = errors.New("input is not compressible")
+
+	// ErrUseRLE is returned from the compressor when the input is a single byte value repeated.
+	ErrUseRLE = errors.New("input is single value repeated")
+
+	// ErrTooBig is return if input is too large for a single block.
+	ErrTooBig = errors.New("input too big")
+
+	// ErrMaxDecodedSizeExceeded is return if input is too large for a single block.
+	ErrMaxDecodedSizeExceeded = errors.New("maximum output size exceeded")
+)
+
+type ReusePolicy uint8
+
+const (
+	// ReusePolicyAllow will allow reuse if it produces smaller output.
+	ReusePolicyAllow ReusePolicy = iota
+
+	// ReusePolicyPrefer will re-use aggressively if possible.
+	// This will not check if a new table will produce smaller output,
+	// except if the current table is impossible to use or
+	// compressed output is bigger than input.
+	ReusePolicyPrefer
+
+	// ReusePolicyNone will disable re-use of tables.
+	// This is slightly faster than ReusePolicyAllow but may produce larger output.
+	ReusePolicyNone
+
+	// ReusePolicyMust must allow reuse and produce smaller output.
+	ReusePolicyMust
+)
+
+type Scratch struct {
+	count [maxSymbolValue + 1]uint32
+
+	// Per block parameters.
+	// These can be used to override compression parameters of the block.
+	// Do not touch, unless you know what you are doing.
+
+	// Out is output buffer.
+	// If the scratch is re-used before the caller is done processing the output,
+	// set this field to nil.
+	// Otherwise the output buffer will be re-used for next Compression/Decompression step
+	// and allocation will be avoided.
+	Out []byte
+
+	// OutTable will contain the table data only, if a new table has been generated.
+	// Slice of the returned data.
+	OutTable []byte
+
+	// OutData will contain the compressed data.
+	// Slice of the returned data.
+	OutData []byte
+
+	// MaxDecodedSize will set the maximum allowed output size.
+	// This value will automatically be set to BlockSizeMax if not set.
+	// Decoders will return ErrMaxDecodedSizeExceeded is this limit is exceeded.
+	MaxDecodedSize int
+
+	br byteReader
+
+	// MaxSymbolValue will override the maximum symbol value of the next block.
+	MaxSymbolValue uint8
+
+	// TableLog will attempt to override the tablelog for the next block.
+	// Must be <= 11 and >= 5.
+	TableLog uint8
+
+	// Reuse will specify the reuse policy
+	Reuse ReusePolicy
+
+	// WantLogLess allows to specify a log 2 reduction that should at least be achieved,
+	// otherwise the block will be returned as incompressible.
+	// The reduction should then at least be (input size >> WantLogLess)
+	// If WantLogLess == 0 any improvement will do.
+	WantLogLess uint8
+
+	symbolLen      uint16 // Length of active part of the symbol table.
+	maxCount       int    // count of the most probable symbol
+	clearCount     bool   // clear count
+	actualTableLog uint8  // Selected tablelog.
+	prevTableLog   uint8  // Tablelog for previous table
+	prevTable      cTable // Table used for previous compression.
+	cTable         cTable // compression table
+	dt             dTable // decompression table
+	nodes          []nodeElt
+	tmpOut         [4][]byte
+	fse            *fse.Scratch
+	huffWeight     [maxSymbolValue + 1]byte
+}
+
+// TransferCTable will transfer the previously used compression table.
+func (s *Scratch) TransferCTable(src *Scratch) {
+	if cap(s.prevTable) < len(src.prevTable) {
+		s.prevTable = make(cTable, 0, maxSymbolValue+1)
+	}
+	s.prevTable = s.prevTable[:len(src.prevTable)]
+	copy(s.prevTable, src.prevTable)
+	s.prevTableLog = src.prevTableLog
+}
+
+func (s *Scratch) prepare(in []byte) (*Scratch, error) {
+	if len(in) > BlockSizeMax {
+		return nil, ErrTooBig
+	}
+	if s == nil {
+		s = &Scratch{}
+	}
+	if s.MaxSymbolValue == 0 {
+		s.MaxSymbolValue = maxSymbolValue
+	}
+	if s.TableLog == 0 {
+		s.TableLog = tableLogDefault
+	}
+	if s.TableLog > tableLogMax || s.TableLog < minTablelog {
+		return nil, fmt.Errorf(" invalid tableLog %d (%d -> %d)", s.TableLog, minTablelog, tableLogMax)
+	}
+	if s.MaxDecodedSize <= 0 || s.MaxDecodedSize > BlockSizeMax {
+		s.MaxDecodedSize = BlockSizeMax
+	}
+	if s.clearCount && s.maxCount == 0 {
+		for i := range s.count {
+			s.count[i] = 0
+		}
+		s.clearCount = false
+	}
+	if cap(s.Out) == 0 {
+		s.Out = make([]byte, 0, len(in))
+	}
+	s.Out = s.Out[:0]
+
+	s.OutTable = nil
+	s.OutData = nil
+	if cap(s.nodes) < huffNodesLen+1 {
+		s.nodes = make([]nodeElt, 0, huffNodesLen+1)
+	}
+	s.nodes = s.nodes[:0]
+	if s.fse == nil {
+		s.fse = &fse.Scratch{}
+	}
+	s.br.init(in)
+
+	return s, nil
+}
+
+type cTable []cTableEntry
+
+func (c cTable) write(s *Scratch) error {
+	var (
+		// precomputed conversion table
+		bitsToWeight [tableLogMax + 1]byte
+		huffLog      = s.actualTableLog
+		// last weight is not saved.
+		maxSymbolValue = uint8(s.symbolLen - 1)
+		huffWeight     = s.huffWeight[:256]
+	)
+	const (
+		maxFSETableLog = 6
+	)
+	// convert to weight
+	bitsToWeight[0] = 0
+	for n := uint8(1); n < huffLog+1; n++ {
+		bitsToWeight[n] = huffLog + 1 - n
+	}
+
+	// Acquire histogram for FSE.
+	hist := s.fse.Histogram()
+	hist = hist[:256]
+	for i := range hist[:16] {
+		hist[i] = 0
+	}
+	for n := uint8(0); n < maxSymbolValue; n++ {
+		v := bitsToWeight[c[n].nBits] & 15
+		huffWeight[n] = v
+		hist[v]++
+	}
+
+	// FSE compress if feasible.
+	if maxSymbolValue >= 2 {
+		huffMaxCnt := uint32(0)
+		huffMax := uint8(0)
+		for i, v := range hist[:16] {
+			if v == 0 {
+				continue
+			}
+			huffMax = byte(i)
+			if v > huffMaxCnt {
+				huffMaxCnt = v
+			}
+		}
+		s.fse.HistogramFinished(huffMax, int(huffMaxCnt))
+		s.fse.TableLog = maxFSETableLog
+		b, err := fse.Compress(huffWeight[:maxSymbolValue], s.fse)
+		if err == nil && len(b) < int(s.symbolLen>>1) {
+			s.Out = append(s.Out, uint8(len(b)))
+			s.Out = append(s.Out, b...)
+			return nil
+		}
+		// Unable to compress (RLE/uncompressible)
+	}
+	// write raw values as 4-bits (max : 15)
+	if maxSymbolValue > (256 - 128) {
+		// should not happen : likely means source cannot be compressed
+		return ErrIncompressible
+	}
+	op := s.Out
+	// special case, pack weights 4 bits/weight.
+	op = append(op, 128|(maxSymbolValue-1))
+	// be sure it doesn't cause msan issue in final combination
+	huffWeight[maxSymbolValue] = 0
+	for n := uint16(0); n < uint16(maxSymbolValue); n += 2 {
+		op = append(op, (huffWeight[n]<<4)|huffWeight[n+1])
+	}
+	s.Out = op
+	return nil
+}
+
+// estimateSize returns the estimated size in bytes of the input represented in the
+// histogram supplied.
+func (c cTable) estimateSize(hist []uint32) int {
+	nbBits := uint32(7)
+	for i, v := range c[:len(hist)] {
+		nbBits += uint32(v.nBits) * hist[i]
+	}
+	return int(nbBits >> 3)
+}
+
+// minSize returns the minimum possible size considering the shannon limit.
+func (s *Scratch) minSize(total int) int {
+	nbBits := float64(7)
+	fTotal := float64(total)
+	for _, v := range s.count[:s.symbolLen] {
+		n := float64(v)
+		if n > 0 {
+			nbBits += math.Log2(fTotal/n) * n
+		}
+	}
+	return int(nbBits) >> 3
+}
+
+func highBit32(val uint32) (n uint32) {
+	return uint32(bits.Len32(val) - 1)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/README.md b/vendor/github.com/klauspost/compress/zstd/README.md
new file mode 100644
index 0000000..e7d7eb0
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/README.md
@@ -0,0 +1,417 @@
+# zstd 
+
+[Zstandard](https://facebook.github.io/zstd/) is a real-time compression algorithm, providing high compression ratios. 
+It offers a very wide range of compression / speed trade-off, while being backed by a very fast decoder.
+A high performance compression algorithm is implemented. For now focused on speed. 
+
+This package provides [compression](#Compressor) to and [decompression](#Decompressor) of Zstandard content. 
+
+This package is pure Go and without use of "unsafe". 
+
+The `zstd` package is provided as open source software using a Go standard license.
+
+Currently the package is heavily optimized for 64 bit processors and will be significantly slower on 32 bit processors.
+
+## Installation
+
+Install using `go get -u github.com/klauspost/compress`. The package is located in `github.com/klauspost/compress/zstd`.
+
+Godoc Documentation: https://godoc.org/github.com/klauspost/compress/zstd
+
+
+## Compressor
+
+### Status: 
+
+STABLE - there may always be subtle bugs, a wide variety of content has been tested and the library is actively 
+used by several projects. This library is being [fuzz-tested](https://github.com/klauspost/compress-fuzz) for all updates.
+
+There may still be specific combinations of data types/size/settings that could lead to edge cases, 
+so as always, testing is recommended.  
+
+For now, a high speed (fastest) and medium-fast (default) compressor has been implemented. 
+
+* The "Fastest" compression ratio is roughly equivalent to zstd level 1. 
+* The "Default" compression ratio is roughly equivalent to zstd level 3 (default).
+* The "Better" compression ratio is roughly equivalent to zstd level 7.
+* The "Best" compression ratio is roughly equivalent to zstd level 11.
+
+In terms of speed, it is typically 2x as fast as the stdlib deflate/gzip in its fastest mode. 
+The compression ratio compared to stdlib is around level 3, but usually 3x as fast.
+
+ 
+### Usage
+
+An Encoder can be used for either compressing a stream via the
+`io.WriteCloser` interface supported by the Encoder or as multiple independent
+tasks via the `EncodeAll` function.
+Smaller encodes are encouraged to use the EncodeAll function.
+Use `NewWriter` to create a new instance that can be used for both.
+
+To create a writer with default options, do like this:
+
+```Go
+// Compress input to output.
+func Compress(in io.Reader, out io.Writer) error {
+    enc, err := zstd.NewWriter(out)
+    if err != nil {
+        return err
+    }
+    _, err = io.Copy(enc, in)
+    if err != nil {
+        enc.Close()
+        return err
+    }
+    return enc.Close()
+}
+```
+
+Now you can encode by writing data to `enc`. The output will be finished writing when `Close()` is called.
+Even if your encode fails, you should still call `Close()` to release any resources that may be held up.  
+
+The above is fine for big encodes. However, whenever possible try to *reuse* the writer.
+
+To reuse the encoder, you can use the `Reset(io.Writer)` function to change to another output. 
+This will allow the encoder to reuse all resources and avoid wasteful allocations. 
+
+Currently stream encoding has 'light' concurrency, meaning up to 2 goroutines can be working on part 
+of a stream. This is independent of the `WithEncoderConcurrency(n)`, but that is likely to change 
+in the future. So if you want to limit concurrency for future updates, specify the concurrency
+you would like.
+
+You can specify your desired compression level using `WithEncoderLevel()` option. Currently only pre-defined 
+compression settings can be specified.
+
+#### Future Compatibility Guarantees
+
+This will be an evolving project. When using this package it is important to note that both the compression efficiency and speed may change.
+
+The goal will be to keep the default efficiency at the default zstd (level 3). 
+However the encoding should never be assumed to remain the same, 
+and you should not use hashes of compressed output for similarity checks.
+
+The Encoder can be assumed to produce the same output from the exact same code version.
+However, the may be modes in the future that break this, 
+although they will not be enabled without an explicit option.   
+
+This encoder is not designed to (and will probably never) output the exact same bitstream as the reference encoder.
+
+Also note, that the cgo decompressor currently does not [report all errors on invalid input](https://github.com/DataDog/zstd/issues/59),
+[omits error checks](https://github.com/DataDog/zstd/issues/61), [ignores checksums](https://github.com/DataDog/zstd/issues/43) 
+and seems to ignore concatenated streams, even though [it is part of the spec](https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#frames).
+
+#### Blocks
+
+For compressing small blocks, the returned encoder has a function called `EncodeAll(src, dst []byte) []byte`.
+
+`EncodeAll` will encode all input in src and append it to dst.
+This function can be called concurrently, but each call will only run on a single goroutine.
+
+Encoded blocks can be concatenated and the result will be the combined input stream.
+Data compressed with EncodeAll can be decoded with the Decoder, using either a stream or `DecodeAll`.
+
+Especially when encoding blocks you should take special care to reuse the encoder. 
+This will effectively make it run without allocations after a warmup period. 
+To make it run completely without allocations, supply a destination buffer with space for all content.   
+
+```Go
+import "github.com/klauspost/compress/zstd"
+
+// Create a writer that caches compressors.
+// For this operation type we supply a nil Reader.
+var encoder, _ = zstd.NewWriter(nil)
+
+// Compress a buffer. 
+// If you have a destination buffer, the allocation in the call can also be eliminated.
+func Compress(src []byte) []byte {
+    return encoder.EncodeAll(src, make([]byte, 0, len(src)))
+} 
+```
+
+You can control the maximum number of concurrent encodes using the `WithEncoderConcurrency(n)` 
+option when creating the writer.
+
+Using the Encoder for both a stream and individual blocks concurrently is safe. 
+
+### Performance
+
+I have collected some speed examples to compare speed and compression against other compressors.
+
+* `file` is the input file.
+* `out` is the compressor used. `zskp` is this package. `zstd` is the Datadog cgo library. `gzstd/gzkp` is gzip standard and this library.
+* `level` is the compression level used. For `zskp` level 1 is "fastest", level 2 is "default"; 3 is "better", 4 is "best".
+* `insize`/`outsize` is the input/output size.
+* `millis` is the number of milliseconds used for compression.
+* `mb/s` is megabytes (2^20 bytes) per second.
+
+```
+Silesia Corpus:
+http://sun.aei.polsl.pl/~sdeor/corpus/silesia.zip
+
+This package:
+file    out     level   insize      outsize     millis  mb/s
+silesia.tar zskp    1   211947520   73101992    643     313.87
+silesia.tar zskp    2   211947520   67504318    969     208.38
+silesia.tar zskp    3   211947520   64595893    2007    100.68
+silesia.tar zskp    4   211947520   60995370    7691    26.28
+
+cgo zstd:
+silesia.tar zstd    1   211947520   73605392    543     371.56
+silesia.tar zstd    3   211947520   66793289    864     233.68
+silesia.tar zstd    6   211947520   62916450    1913    105.66
+silesia.tar zstd    9   211947520   60212393    5063    39.92
+
+gzip, stdlib/this package:
+silesia.tar gzstd   1   211947520   80007735    1654    122.21
+silesia.tar gzkp    1   211947520   80369488    1168    173.06
+
+GOB stream of binary data. Highly compressible.
+https://files.klauspost.com/compress/gob-stream.7z
+
+file        out     level   insize  outsize     millis  mb/s
+gob-stream  zskp    1   1911399616  235022249   3088    590.30
+gob-stream  zskp    2   1911399616  205669791   3786    481.34
+gob-stream  zskp    3   1911399616  175034659   9636    189.17
+gob-stream  zskp    4   1911399616  167273881   29337   62.13
+gob-stream  zstd    1   1911399616  249810424   2637    691.26
+gob-stream  zstd    3   1911399616  208192146   3490    522.31
+gob-stream  zstd    6   1911399616  193632038   6687    272.56
+gob-stream  zstd    9   1911399616  177620386   16175   112.70
+gob-stream  gzstd   1   1911399616  357382641   10251   177.82
+gob-stream  gzkp    1   1911399616  362156523   5695    320.08
+
+The test data for the Large Text Compression Benchmark is the first
+10^9 bytes of the English Wikipedia dump on Mar. 3, 2006.
+http://mattmahoney.net/dc/textdata.html
+
+file    out level   insize      outsize     millis  mb/s
+enwik9  zskp    1   1000000000  343848582   3609    264.18
+enwik9  zskp    2   1000000000  317276632   5746    165.97
+enwik9  zskp    3   1000000000  292243069   12162   78.41
+enwik9  zskp    4   1000000000  275241169   36430   26.18
+enwik9  zstd    1   1000000000  358072021   3110    306.65
+enwik9  zstd    3   1000000000  313734672   4784    199.35
+enwik9  zstd    6   1000000000  295138875   10290   92.68
+enwik9  zstd    9   1000000000  278348700   28549   33.40
+enwik9  gzstd   1   1000000000  382578136   9604    99.30
+enwik9  gzkp    1   1000000000  383825945   6544    145.73
+
+Highly compressible JSON file.
+https://files.klauspost.com/compress/github-june-2days-2019.json.zst
+
+file                        out level   insize      outsize     millis  mb/s
+github-june-2days-2019.json zskp    1   6273951764  699045015   10620   563.40
+github-june-2days-2019.json zskp    2   6273951764  617881763   11687   511.96
+github-june-2days-2019.json zskp    3   6273951764  524340691   34043   175.75
+github-june-2days-2019.json zskp    4   6273951764  503314661   93811   63.78
+github-june-2days-2019.json zstd    1   6273951764  766284037   8450    708.00
+github-june-2days-2019.json zstd    3   6273951764  661889476   10927   547.57
+github-june-2days-2019.json zstd    6   6273951764  642756859   22996   260.18
+github-june-2days-2019.json zstd    9   6273951764  601974523   52413   114.16
+github-june-2days-2019.json gzstd   1   6273951764  1164400847  29948   199.79
+github-june-2days-2019.json gzkp    1   6273951764  1128755542  19236   311.03
+
+VM Image, Linux mint with a few installed applications:
+https://files.klauspost.com/compress/rawstudio-mint14.7z
+
+file                    out level   insize      outsize     millis  mb/s
+rawstudio-mint14.tar    zskp    1   8558382592  3667489370  20210   403.84
+rawstudio-mint14.tar    zskp    2   8558382592  3364592300  31873   256.07
+rawstudio-mint14.tar    zskp    3   8558382592  3158085214  77675   105.08
+rawstudio-mint14.tar    zskp    4   8558382592  3020370044  404956  20.16
+rawstudio-mint14.tar    zstd    1   8558382592  3609250104  17136   476.27
+rawstudio-mint14.tar    zstd    3   8558382592  3341679997  29262   278.92
+rawstudio-mint14.tar    zstd    6   8558382592  3235846406  77904   104.77
+rawstudio-mint14.tar    zstd    9   8558382592  3160778861  140946  57.91
+rawstudio-mint14.tar    gzstd   1   8558382592  3926257486  57722   141.40
+rawstudio-mint14.tar    gzkp    1   8558382592  3970463184  41749   195.49
+
+CSV data:
+https://files.klauspost.com/compress/nyc-taxi-data-10M.csv.zst
+
+file                    out level   insize      outsize     millis  mb/s
+nyc-taxi-data-10M.csv   zskp    1   3325605752  641339945   8925    355.35
+nyc-taxi-data-10M.csv   zskp    2   3325605752  591748091   11268   281.44
+nyc-taxi-data-10M.csv   zskp    3   3325605752  530289687   25239   125.66
+nyc-taxi-data-10M.csv   zskp    4   3325605752  490907191   65939   48.10
+nyc-taxi-data-10M.csv   zstd    1   3325605752  687399637   8233    385.18
+nyc-taxi-data-10M.csv   zstd    3   3325605752  598514411   10065   315.07
+nyc-taxi-data-10M.csv   zstd    6   3325605752  570522953   20038   158.27
+nyc-taxi-data-10M.csv   zstd    9   3325605752  517554797   64565   49.12
+nyc-taxi-data-10M.csv   gzstd   1   3325605752  928656485   23876   132.83
+nyc-taxi-data-10M.csv   gzkp    1   3325605752  924718719   16388   193.53
+```
+
+## Decompressor
+
+Staus: STABLE - there may still be subtle bugs, but a wide variety of content has been tested.
+
+This library is being continuously [fuzz-tested](https://github.com/klauspost/compress-fuzz),
+kindly supplied by [fuzzit.dev](https://fuzzit.dev/). 
+The main purpose of the fuzz testing is to ensure that it is not possible to crash the decoder, 
+or run it past its limits with ANY input provided.  
+ 
+### Usage
+
+The package has been designed for two main usages, big streams of data and smaller in-memory buffers. 
+There are two main usages of the package for these. Both of them are accessed by creating a `Decoder`.
+
+For streaming use a simple setup could look like this:
+
+```Go
+import "github.com/klauspost/compress/zstd"
+
+func Decompress(in io.Reader, out io.Writer) error {
+    d, err := zstd.NewReader(in)
+    if err != nil {
+        return err
+    }
+    defer d.Close()
+    
+    // Copy content...
+    _, err = io.Copy(out, d)
+    return err
+}
+```
+
+It is important to use the "Close" function when you no longer need the Reader to stop running goroutines. 
+See "Allocation-less operation" below.
+
+For decoding buffers, it could look something like this:
+
+```Go
+import "github.com/klauspost/compress/zstd"
+
+// Create a reader that caches decompressors.
+// For this operation type we supply a nil Reader.
+var decoder, _ = zstd.NewReader(nil)
+
+// Decompress a buffer. We don't supply a destination buffer,
+// so it will be allocated by the decoder.
+func Decompress(src []byte) ([]byte, error) {
+    return decoder.DecodeAll(src, nil)
+} 
+```
+
+Both of these cases should provide the functionality needed. 
+The decoder can be used for *concurrent* decompression of multiple buffers. 
+It will only allow a certain number of concurrent operations to run. 
+To tweak that yourself use the `WithDecoderConcurrency(n)` option when creating the decoder.   
+
+### Dictionaries
+
+Data compressed with [dictionaries](https://github.com/facebook/zstd#the-case-for-small-data-compression) can be decompressed.
+
+Dictionaries are added individually to Decoders.
+Dictionaries are generated by the `zstd --train` command and contains an initial state for the decoder.
+To add a dictionary use the `WithDecoderDicts(dicts ...[]byte)` option with the dictionary data.
+Several dictionaries can be added at once.
+
+The dictionary will be used automatically for the data that specifies them.
+A re-used Decoder will still contain the dictionaries registered.
+
+When registering multiple dictionaries with the same ID, the last one will be used.
+
+It is possible to use dictionaries when compressing data.
+
+To enable a dictionary use `WithEncoderDict(dict []byte)`. Here only one dictionary will be used 
+and it will likely be used even if it doesn't improve compression. 
+
+The used dictionary must be used to decompress the content.
+
+For any real gains, the dictionary should be built with similar data. 
+If an unsuitable dictionary is used the output may be slightly larger than using no dictionary.
+Use the [zstd commandline tool](https://github.com/facebook/zstd/releases) to build a dictionary from sample data.
+For information see [zstd dictionary information](https://github.com/facebook/zstd#the-case-for-small-data-compression). 
+
+For now there is a fixed startup performance penalty for compressing content with dictionaries. 
+This will likely be improved over time. Just be aware to test performance when implementing.  
+
+### Allocation-less operation
+
+The decoder has been designed to operate without allocations after a warmup. 
+
+This means that you should *store* the decoder for best performance. 
+To re-use a stream decoder, use the `Reset(r io.Reader) error` to switch to another stream.
+A decoder can safely be re-used even if the previous stream failed.
+
+To release the resources, you must call the `Close()` function on a decoder.
+After this it can *no longer be reused*, but all running goroutines will be stopped.
+So you *must* use this if you will no longer need the Reader.
+
+For decompressing smaller buffers a single decoder can be used.
+When decoding buffers, you can supply a destination slice with length 0 and your expected capacity.
+In this case no unneeded allocations should be made. 
+
+### Concurrency
+
+The buffer decoder does everything on the same goroutine and does nothing concurrently.
+It can however decode several buffers concurrently. Use `WithDecoderConcurrency(n)` to limit that.
+
+The stream decoder operates on
+
+* One goroutine reads input and splits the input to several block decoders.
+* A number of decoders will decode blocks.
+* A goroutine coordinates these blocks and sends history from one to the next.
+
+So effectively this also means the decoder will "read ahead" and prepare data to always be available for output.
+
+Since "blocks" are quite dependent on the output of the previous block stream decoding will only have limited concurrency.
+
+In practice this means that concurrency is often limited to utilizing about 2 cores effectively.
+ 
+ 
+### Benchmarks
+
+These are some examples of performance compared to [datadog cgo library](https://github.com/DataDog/zstd).
+
+The first two are streaming decodes and the last are smaller inputs. 
+ 
+```
+BenchmarkDecoderSilesia-8                          3     385000067 ns/op     550.51 MB/s        5498 B/op          8 allocs/op
+BenchmarkDecoderSilesiaCgo-8                       6     197666567 ns/op    1072.25 MB/s      270672 B/op          8 allocs/op
+
+BenchmarkDecoderEnwik9-8                           1    2027001600 ns/op     493.34 MB/s       10496 B/op         18 allocs/op
+BenchmarkDecoderEnwik9Cgo-8                        2     979499200 ns/op    1020.93 MB/s      270672 B/op          8 allocs/op
+
+Concurrent performance:
+
+BenchmarkDecoder_DecodeAllParallel/kppkn.gtb.zst-16                28915         42469 ns/op    4340.07 MB/s         114 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/geo.protodata.zst-16           116505          9965 ns/op    11900.16 MB/s         16 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/plrabn12.txt.zst-16              8952        134272 ns/op    3588.70 MB/s         915 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/lcet10.txt.zst-16               11820        102538 ns/op    4161.90 MB/s         594 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/asyoulik.txt.zst-16             34782         34184 ns/op    3661.88 MB/s          60 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/alice29.txt.zst-16              27712         43447 ns/op    3500.58 MB/s          99 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/html_x_4.zst-16                 62826         18750 ns/op    21845.10 MB/s        104 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/paper-100k.pdf.zst-16          631545          1794 ns/op    57078.74 MB/s          2 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/fireworks.jpeg.zst-16         1690140           712 ns/op    172938.13 MB/s         1 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/urls.10K.zst-16                 10432        113593 ns/op    6180.73 MB/s        1143 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/html.zst-16                    113206         10671 ns/op    9596.27 MB/s          15 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallel/comp-data.bin.zst-16          1530615           779 ns/op    5229.49 MB/s           0 B/op          0 allocs/op
+
+BenchmarkDecoder_DecodeAllParallelCgo/kppkn.gtb.zst-16             65217         16192 ns/op    11383.34 MB/s         46 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/geo.protodata.zst-16        292671          4039 ns/op    29363.19 MB/s          6 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/plrabn12.txt.zst-16          26314         46021 ns/op    10470.43 MB/s        293 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/lcet10.txt.zst-16            33897         34900 ns/op    12227.96 MB/s        205 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/asyoulik.txt.zst-16         104348         11433 ns/op    10949.01 MB/s         20 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/alice29.txt.zst-16           75949         15510 ns/op    9805.60 MB/s          32 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/html_x_4.zst-16             173910          6756 ns/op    60624.29 MB/s         37 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/paper-100k.pdf.zst-16       923076          1339 ns/op    76474.87 MB/s          1 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/fireworks.jpeg.zst-16       922920          1351 ns/op    91102.57 MB/s          2 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/urls.10K.zst-16              27649         43618 ns/op    16096.19 MB/s        407 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/html.zst-16                 279073          4160 ns/op    24614.18 MB/s          6 B/op          0 allocs/op
+BenchmarkDecoder_DecodeAllParallelCgo/comp-data.bin.zst-16        749938          1579 ns/op    2581.71 MB/s           0 B/op          0 allocs/op
+```
+
+This reflects the performance around May 2020, but this may be out of date.
+
+# Contributions
+
+Contributions are always welcome. 
+For new features/fixes, remember to add tests and for performance enhancements include benchmarks.
+
+For sending files for reproducing errors use a service like [goobox](https://goobox.io/#/upload) or similar to share your files.
+
+For general feedback and experience reports, feel free to open an issue or write me on [Twitter](https://twitter.com/sh0dan).
+
+This package includes the excellent [`github.com/cespare/xxhash`](https://github.com/cespare/xxhash) package Copyright (c) 2016 Caleb Spare.
diff --git a/vendor/github.com/klauspost/compress/zstd/bitreader.go b/vendor/github.com/klauspost/compress/zstd/bitreader.go
new file mode 100644
index 0000000..8544585
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/bitreader.go
@@ -0,0 +1,136 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"encoding/binary"
+	"errors"
+	"io"
+	"math/bits"
+)
+
+// bitReader reads a bitstream in reverse.
+// The last set bit indicates the start of the stream and is used
+// for aligning the input.
+type bitReader struct {
+	in       []byte
+	off      uint   // next byte to read is at in[off - 1]
+	value    uint64 // Maybe use [16]byte, but shifting is awkward.
+	bitsRead uint8
+}
+
+// init initializes and resets the bit reader.
+func (b *bitReader) init(in []byte) error {
+	if len(in) < 1 {
+		return errors.New("corrupt stream: too short")
+	}
+	b.in = in
+	b.off = uint(len(in))
+	// The highest bit of the last byte indicates where to start
+	v := in[len(in)-1]
+	if v == 0 {
+		return errors.New("corrupt stream, did not find end of stream")
+	}
+	b.bitsRead = 64
+	b.value = 0
+	if len(in) >= 8 {
+		b.fillFastStart()
+	} else {
+		b.fill()
+		b.fill()
+	}
+	b.bitsRead += 8 - uint8(highBits(uint32(v)))
+	return nil
+}
+
+// getBits will return n bits. n can be 0.
+func (b *bitReader) getBits(n uint8) int {
+	if n == 0 /*|| b.bitsRead >= 64 */ {
+		return 0
+	}
+	return b.getBitsFast(n)
+}
+
+// getBitsFast requires that at least one bit is requested every time.
+// There are no checks if the buffer is filled.
+func (b *bitReader) getBitsFast(n uint8) int {
+	const regMask = 64 - 1
+	v := uint32((b.value << (b.bitsRead & regMask)) >> ((regMask + 1 - n) & regMask))
+	b.bitsRead += n
+	return int(v)
+}
+
+// fillFast() will make sure at least 32 bits are available.
+// There must be at least 4 bytes available.
+func (b *bitReader) fillFast() {
+	if b.bitsRead < 32 {
+		return
+	}
+	// 2 bounds checks.
+	v := b.in[b.off-4:]
+	v = v[:4]
+	low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+	b.value = (b.value << 32) | uint64(low)
+	b.bitsRead -= 32
+	b.off -= 4
+}
+
+// fillFastStart() assumes the bitreader is empty and there is at least 8 bytes to read.
+func (b *bitReader) fillFastStart() {
+	// Do single re-slice to avoid bounds checks.
+	b.value = binary.LittleEndian.Uint64(b.in[b.off-8:])
+	b.bitsRead = 0
+	b.off -= 8
+}
+
+// fill() will make sure at least 32 bits are available.
+func (b *bitReader) fill() {
+	if b.bitsRead < 32 {
+		return
+	}
+	if b.off >= 4 {
+		v := b.in[b.off-4:]
+		v = v[:4]
+		low := (uint32(v[0])) | (uint32(v[1]) << 8) | (uint32(v[2]) << 16) | (uint32(v[3]) << 24)
+		b.value = (b.value << 32) | uint64(low)
+		b.bitsRead -= 32
+		b.off -= 4
+		return
+	}
+	for b.off > 0 {
+		b.value = (b.value << 8) | uint64(b.in[b.off-1])
+		b.bitsRead -= 8
+		b.off--
+	}
+}
+
+// finished returns true if all bits have been read from the bit stream.
+func (b *bitReader) finished() bool {
+	return b.off == 0 && b.bitsRead >= 64
+}
+
+// overread returns true if more bits have been requested than is on the stream.
+func (b *bitReader) overread() bool {
+	return b.bitsRead > 64
+}
+
+// remain returns the number of bits remaining.
+func (b *bitReader) remain() uint {
+	return b.off*8 + 64 - uint(b.bitsRead)
+}
+
+// close the bitstream and returns an error if out-of-buffer reads occurred.
+func (b *bitReader) close() error {
+	// Release reference.
+	b.in = nil
+	if b.bitsRead > 64 {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+
+func highBits(val uint32) (n uint32) {
+	return uint32(bits.Len32(val) - 1)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/bitwriter.go b/vendor/github.com/klauspost/compress/zstd/bitwriter.go
new file mode 100644
index 0000000..303ae90
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/bitwriter.go
@@ -0,0 +1,169 @@
+// Copyright 2018 Klaus Post. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+// Based on work Copyright (c) 2013, Yann Collet, released under BSD License.
+
+package zstd
+
+import "fmt"
+
+// bitWriter will write bits.
+// First bit will be LSB of the first byte of output.
+type bitWriter struct {
+	bitContainer uint64
+	nBits        uint8
+	out          []byte
+}
+
+// bitMask16 is bitmasks. Has extra to avoid bounds check.
+var bitMask16 = [32]uint16{
+	0, 1, 3, 7, 0xF, 0x1F,
+	0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF,
+	0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF, 0xFFFF,
+	0xFFFF, 0xFFFF} /* up to 16 bits */
+
+var bitMask32 = [32]uint32{
+	0, 1, 3, 7, 0xF, 0x1F, 0x3F, 0x7F, 0xFF,
+	0x1FF, 0x3FF, 0x7FF, 0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF,
+	0x1ffff, 0x3ffff, 0x7FFFF, 0xfFFFF, 0x1fFFFF, 0x3fFFFF, 0x7fFFFF, 0xffFFFF,
+	0x1ffFFFF, 0x3ffFFFF, 0x7ffFFFF, 0xfffFFFF, 0x1fffFFFF, 0x3fffFFFF, 0x7fffFFFF,
+} // up to 32 bits
+
+// addBits16NC will add up to 16 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16NC(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value&bitMask16[bits&31]) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// addBits32NC will add up to 32 bits.
+// It will not check if there is space for them,
+// so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits32NC(value uint32, bits uint8) {
+	b.bitContainer |= uint64(value&bitMask32[bits&31]) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// addBits16Clean will add up to 16 bits. value may not contain more set bits than indicated.
+// It will not check if there is space for them, so the caller must ensure that it has flushed recently.
+func (b *bitWriter) addBits16Clean(value uint16, bits uint8) {
+	b.bitContainer |= uint64(value) << (b.nBits & 63)
+	b.nBits += bits
+}
+
+// flush will flush all pending full bytes.
+// There will be at least 56 bits available for writing when this has been called.
+// Using flush32 is faster, but leaves less space for writing.
+func (b *bitWriter) flush() {
+	v := b.nBits >> 3
+	switch v {
+	case 0:
+	case 1:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+		)
+	case 2:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+		)
+	case 3:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+		)
+	case 4:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+		)
+	case 5:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+		)
+	case 6:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+		)
+	case 7:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+		)
+	case 8:
+		b.out = append(b.out,
+			byte(b.bitContainer),
+			byte(b.bitContainer>>8),
+			byte(b.bitContainer>>16),
+			byte(b.bitContainer>>24),
+			byte(b.bitContainer>>32),
+			byte(b.bitContainer>>40),
+			byte(b.bitContainer>>48),
+			byte(b.bitContainer>>56),
+		)
+	default:
+		panic(fmt.Errorf("bits (%d) > 64", b.nBits))
+	}
+	b.bitContainer >>= v << 3
+	b.nBits &= 7
+}
+
+// flush32 will flush out, so there are at least 32 bits available for writing.
+func (b *bitWriter) flush32() {
+	if b.nBits < 32 {
+		return
+	}
+	b.out = append(b.out,
+		byte(b.bitContainer),
+		byte(b.bitContainer>>8),
+		byte(b.bitContainer>>16),
+		byte(b.bitContainer>>24))
+	b.nBits -= 32
+	b.bitContainer >>= 32
+}
+
+// flushAlign will flush remaining full bytes and align to next byte boundary.
+func (b *bitWriter) flushAlign() {
+	nbBytes := (b.nBits + 7) >> 3
+	for i := uint8(0); i < nbBytes; i++ {
+		b.out = append(b.out, byte(b.bitContainer>>(i*8)))
+	}
+	b.nBits = 0
+	b.bitContainer = 0
+}
+
+// close will write the alignment bit and write the final byte(s)
+// to the output.
+func (b *bitWriter) close() error {
+	// End mark
+	b.addBits16Clean(1, 1)
+	// flush until next byte.
+	b.flushAlign()
+	return nil
+}
+
+// reset and continue writing by appending to out.
+func (b *bitWriter) reset(out []byte) {
+	b.bitContainer = 0
+	b.nBits = 0
+	b.out = out
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/blockdec.go b/vendor/github.com/klauspost/compress/zstd/blockdec.go
new file mode 100644
index 0000000..b51d922
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/blockdec.go
@@ -0,0 +1,739 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"fmt"
+	"io"
+	"sync"
+
+	"github.com/klauspost/compress/huff0"
+	"github.com/klauspost/compress/zstd/internal/xxhash"
+)
+
+type blockType uint8
+
+//go:generate stringer -type=blockType,literalsBlockType,seqCompMode,tableIndex
+
+const (
+	blockTypeRaw blockType = iota
+	blockTypeRLE
+	blockTypeCompressed
+	blockTypeReserved
+)
+
+type literalsBlockType uint8
+
+const (
+	literalsBlockRaw literalsBlockType = iota
+	literalsBlockRLE
+	literalsBlockCompressed
+	literalsBlockTreeless
+)
+
+const (
+	// maxCompressedBlockSize is the biggest allowed compressed block size (128KB)
+	maxCompressedBlockSize = 128 << 10
+
+	// Maximum possible block size (all Raw+Uncompressed).
+	maxBlockSize = (1 << 21) - 1
+
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#literals_section_header
+	maxCompressedLiteralSize = 1 << 18
+	maxRLELiteralSize        = 1 << 20
+	maxMatchLen              = 131074
+	maxSequences             = 0x7f00 + 0xffff
+
+	// We support slightly less than the reference decoder to be able to
+	// use ints on 32 bit archs.
+	maxOffsetBits = 30
+)
+
+var (
+	huffDecoderPool = sync.Pool{New: func() interface{} {
+		return &huff0.Scratch{}
+	}}
+
+	fseDecoderPool = sync.Pool{New: func() interface{} {
+		return &fseDecoder{}
+	}}
+)
+
+type blockDec struct {
+	// Raw source data of the block.
+	data        []byte
+	dataStorage []byte
+
+	// Destination of the decoded data.
+	dst []byte
+
+	// Buffer for literals data.
+	literalBuf []byte
+
+	// Window size of the block.
+	WindowSize uint64
+
+	history     chan *history
+	input       chan struct{}
+	result      chan decodeOutput
+	sequenceBuf []seq
+	err         error
+	decWG       sync.WaitGroup
+
+	// Frame to use for singlethreaded decoding.
+	// Should not be used by the decoder itself since parent may be another frame.
+	localFrame *frameDec
+
+	// Block is RLE, this is the size.
+	RLESize uint32
+	tmp     [4]byte
+
+	Type blockType
+
+	// Is this the last block of a frame?
+	Last bool
+
+	// Use less memory
+	lowMem bool
+}
+
+func (b *blockDec) String() string {
+	if b == nil {
+		return "<nil>"
+	}
+	return fmt.Sprintf("Steam Size: %d, Type: %v, Last: %t, Window: %d", len(b.data), b.Type, b.Last, b.WindowSize)
+}
+
+func newBlockDec(lowMem bool) *blockDec {
+	b := blockDec{
+		lowMem:  lowMem,
+		result:  make(chan decodeOutput, 1),
+		input:   make(chan struct{}, 1),
+		history: make(chan *history, 1),
+	}
+	b.decWG.Add(1)
+	go b.startDecoder()
+	return &b
+}
+
+// reset will reset the block.
+// Input must be a start of a block and will be at the end of the block when returned.
+func (b *blockDec) reset(br byteBuffer, windowSize uint64) error {
+	b.WindowSize = windowSize
+	tmp := br.readSmall(3)
+	if tmp == nil {
+		if debug {
+			println("Reading block header:", io.ErrUnexpectedEOF)
+		}
+		return io.ErrUnexpectedEOF
+	}
+	bh := uint32(tmp[0]) | (uint32(tmp[1]) << 8) | (uint32(tmp[2]) << 16)
+	b.Last = bh&1 != 0
+	b.Type = blockType((bh >> 1) & 3)
+	// find size.
+	cSize := int(bh >> 3)
+	maxSize := maxBlockSize
+	switch b.Type {
+	case blockTypeReserved:
+		return ErrReservedBlockType
+	case blockTypeRLE:
+		b.RLESize = uint32(cSize)
+		if b.lowMem {
+			maxSize = cSize
+		}
+		cSize = 1
+	case blockTypeCompressed:
+		if debug {
+			println("Data size on stream:", cSize)
+		}
+		b.RLESize = 0
+		maxSize = maxCompressedBlockSize
+		if windowSize < maxCompressedBlockSize && b.lowMem {
+			maxSize = int(windowSize)
+		}
+		if cSize > maxCompressedBlockSize || uint64(cSize) > b.WindowSize {
+			if debug {
+				printf("compressed block too big: csize:%d block: %+v\n", uint64(cSize), b)
+			}
+			return ErrCompressedSizeTooBig
+		}
+	case blockTypeRaw:
+		b.RLESize = 0
+		// We do not need a destination for raw blocks.
+		maxSize = -1
+	default:
+		panic("Invalid block type")
+	}
+
+	// Read block data.
+	if cap(b.dataStorage) < cSize {
+		if b.lowMem {
+			b.dataStorage = make([]byte, 0, cSize)
+		} else {
+			b.dataStorage = make([]byte, 0, maxBlockSize)
+		}
+	}
+	if cap(b.dst) <= maxSize {
+		b.dst = make([]byte, 0, maxSize+1)
+	}
+	var err error
+	b.data, err = br.readBig(cSize, b.dataStorage)
+	if err != nil {
+		if debug {
+			println("Reading block:", err, "(", cSize, ")", len(b.data))
+			printf("%T", br)
+		}
+		return err
+	}
+	return nil
+}
+
+// sendEOF will make the decoder send EOF on this frame.
+func (b *blockDec) sendErr(err error) {
+	b.Last = true
+	b.Type = blockTypeReserved
+	b.err = err
+	b.input <- struct{}{}
+}
+
+// Close will release resources.
+// Closed blockDec cannot be reset.
+func (b *blockDec) Close() {
+	close(b.input)
+	close(b.history)
+	close(b.result)
+	b.decWG.Wait()
+}
+
+// decodeAsync will prepare decoding the block when it receives input.
+// This will separate output and history.
+func (b *blockDec) startDecoder() {
+	defer b.decWG.Done()
+	for range b.input {
+		//println("blockDec: Got block input")
+		switch b.Type {
+		case blockTypeRLE:
+			if cap(b.dst) < int(b.RLESize) {
+				if b.lowMem {
+					b.dst = make([]byte, b.RLESize)
+				} else {
+					b.dst = make([]byte, maxBlockSize)
+				}
+			}
+			o := decodeOutput{
+				d:   b,
+				b:   b.dst[:b.RLESize],
+				err: nil,
+			}
+			v := b.data[0]
+			for i := range o.b {
+				o.b[i] = v
+			}
+			hist := <-b.history
+			hist.append(o.b)
+			b.result <- o
+		case blockTypeRaw:
+			o := decodeOutput{
+				d:   b,
+				b:   b.data,
+				err: nil,
+			}
+			hist := <-b.history
+			hist.append(o.b)
+			b.result <- o
+		case blockTypeCompressed:
+			b.dst = b.dst[:0]
+			err := b.decodeCompressed(nil)
+			o := decodeOutput{
+				d:   b,
+				b:   b.dst,
+				err: err,
+			}
+			if debug {
+				println("Decompressed to", len(b.dst), "bytes, error:", err)
+			}
+			b.result <- o
+		case blockTypeReserved:
+			// Used for returning errors.
+			<-b.history
+			b.result <- decodeOutput{
+				d:   b,
+				b:   nil,
+				err: b.err,
+			}
+		default:
+			panic("Invalid block type")
+		}
+		if debug {
+			println("blockDec: Finished block")
+		}
+	}
+}
+
+// decodeAsync will prepare decoding the block when it receives the history.
+// If history is provided, it will not fetch it from the channel.
+func (b *blockDec) decodeBuf(hist *history) error {
+	switch b.Type {
+	case blockTypeRLE:
+		if cap(b.dst) < int(b.RLESize) {
+			if b.lowMem {
+				b.dst = make([]byte, b.RLESize)
+			} else {
+				b.dst = make([]byte, maxBlockSize)
+			}
+		}
+		b.dst = b.dst[:b.RLESize]
+		v := b.data[0]
+		for i := range b.dst {
+			b.dst[i] = v
+		}
+		hist.appendKeep(b.dst)
+		return nil
+	case blockTypeRaw:
+		hist.appendKeep(b.data)
+		return nil
+	case blockTypeCompressed:
+		saved := b.dst
+		b.dst = hist.b
+		hist.b = nil
+		err := b.decodeCompressed(hist)
+		if debug {
+			println("Decompressed to total", len(b.dst), "bytes, hash:", xxhash.Sum64(b.dst), "error:", err)
+		}
+		hist.b = b.dst
+		b.dst = saved
+		return err
+	case blockTypeReserved:
+		// Used for returning errors.
+		return b.err
+	default:
+		panic("Invalid block type")
+	}
+}
+
+// decodeCompressed will start decompressing a block.
+// If no history is supplied the decoder will decodeAsync as much as possible
+// before fetching from blockDec.history
+func (b *blockDec) decodeCompressed(hist *history) error {
+	in := b.data
+	delayedHistory := hist == nil
+
+	if delayedHistory {
+		// We must always grab history.
+		defer func() {
+			if hist == nil {
+				<-b.history
+			}
+		}()
+	}
+	// There must be at least one byte for Literals_Block_Type and one for Sequences_Section_Header
+	if len(in) < 2 {
+		return ErrBlockTooSmall
+	}
+	litType := literalsBlockType(in[0] & 3)
+	var litRegenSize int
+	var litCompSize int
+	sizeFormat := (in[0] >> 2) & 3
+	var fourStreams bool
+	switch litType {
+	case literalsBlockRaw, literalsBlockRLE:
+		switch sizeFormat {
+		case 0, 2:
+			// Regenerated_Size uses 5 bits (0-31). Literals_Section_Header uses 1 byte.
+			litRegenSize = int(in[0] >> 3)
+			in = in[1:]
+		case 1:
+			// Regenerated_Size uses 12 bits (0-4095). Literals_Section_Header uses 2 bytes.
+			litRegenSize = int(in[0]>>4) + (int(in[1]) << 4)
+			in = in[2:]
+		case 3:
+			//  Regenerated_Size uses 20 bits (0-1048575). Literals_Section_Header uses 3 bytes.
+			if len(in) < 3 {
+				println("too small: litType:", litType, " sizeFormat", sizeFormat, len(in))
+				return ErrBlockTooSmall
+			}
+			litRegenSize = int(in[0]>>4) + (int(in[1]) << 4) + (int(in[2]) << 12)
+			in = in[3:]
+		}
+	case literalsBlockCompressed, literalsBlockTreeless:
+		switch sizeFormat {
+		case 0, 1:
+			// Both Regenerated_Size and Compressed_Size use 10 bits (0-1023).
+			if len(in) < 3 {
+				println("too small: litType:", litType, " sizeFormat", sizeFormat, len(in))
+				return ErrBlockTooSmall
+			}
+			n := uint64(in[0]>>4) + (uint64(in[1]) << 4) + (uint64(in[2]) << 12)
+			litRegenSize = int(n & 1023)
+			litCompSize = int(n >> 10)
+			fourStreams = sizeFormat == 1
+			in = in[3:]
+		case 2:
+			fourStreams = true
+			if len(in) < 4 {
+				println("too small: litType:", litType, " sizeFormat", sizeFormat, len(in))
+				return ErrBlockTooSmall
+			}
+			n := uint64(in[0]>>4) + (uint64(in[1]) << 4) + (uint64(in[2]) << 12) + (uint64(in[3]) << 20)
+			litRegenSize = int(n & 16383)
+			litCompSize = int(n >> 14)
+			in = in[4:]
+		case 3:
+			fourStreams = true
+			if len(in) < 5 {
+				println("too small: litType:", litType, " sizeFormat", sizeFormat, len(in))
+				return ErrBlockTooSmall
+			}
+			n := uint64(in[0]>>4) + (uint64(in[1]) << 4) + (uint64(in[2]) << 12) + (uint64(in[3]) << 20) + (uint64(in[4]) << 28)
+			litRegenSize = int(n & 262143)
+			litCompSize = int(n >> 18)
+			in = in[5:]
+		}
+	}
+	if debug {
+		println("literals type:", litType, "litRegenSize:", litRegenSize, "litCompSize:", litCompSize, "sizeFormat:", sizeFormat, "4X:", fourStreams)
+	}
+	var literals []byte
+	var huff *huff0.Scratch
+	switch litType {
+	case literalsBlockRaw:
+		if len(in) < litRegenSize {
+			println("too small: litType:", litType, " sizeFormat", sizeFormat, "remain:", len(in), "want:", litRegenSize)
+			return ErrBlockTooSmall
+		}
+		literals = in[:litRegenSize]
+		in = in[litRegenSize:]
+		//printf("Found %d uncompressed literals\n", litRegenSize)
+	case literalsBlockRLE:
+		if len(in) < 1 {
+			println("too small: litType:", litType, " sizeFormat", sizeFormat, "remain:", len(in), "want:", 1)
+			return ErrBlockTooSmall
+		}
+		if cap(b.literalBuf) < litRegenSize {
+			if b.lowMem {
+				b.literalBuf = make([]byte, litRegenSize)
+			} else {
+				if litRegenSize > maxCompressedLiteralSize {
+					// Exceptional
+					b.literalBuf = make([]byte, litRegenSize)
+				} else {
+					b.literalBuf = make([]byte, litRegenSize, maxCompressedLiteralSize)
+
+				}
+			}
+		}
+		literals = b.literalBuf[:litRegenSize]
+		v := in[0]
+		for i := range literals {
+			literals[i] = v
+		}
+		in = in[1:]
+		if debug {
+			printf("Found %d RLE compressed literals\n", litRegenSize)
+		}
+	case literalsBlockTreeless:
+		if len(in) < litCompSize {
+			println("too small: litType:", litType, " sizeFormat", sizeFormat, "remain:", len(in), "want:", litCompSize)
+			return ErrBlockTooSmall
+		}
+		// Store compressed literals, so we defer decoding until we get history.
+		literals = in[:litCompSize]
+		in = in[litCompSize:]
+		if debug {
+			printf("Found %d compressed literals\n", litCompSize)
+		}
+	case literalsBlockCompressed:
+		if len(in) < litCompSize {
+			println("too small: litType:", litType, " sizeFormat", sizeFormat, "remain:", len(in), "want:", litCompSize)
+			return ErrBlockTooSmall
+		}
+		literals = in[:litCompSize]
+		in = in[litCompSize:]
+		huff = huffDecoderPool.Get().(*huff0.Scratch)
+		var err error
+		// Ensure we have space to store it.
+		if cap(b.literalBuf) < litRegenSize {
+			if b.lowMem {
+				b.literalBuf = make([]byte, 0, litRegenSize)
+			} else {
+				b.literalBuf = make([]byte, 0, maxCompressedLiteralSize)
+			}
+		}
+		if huff == nil {
+			huff = &huff0.Scratch{}
+		}
+		huff, literals, err = huff0.ReadTable(literals, huff)
+		if err != nil {
+			println("reading huffman table:", err)
+			return err
+		}
+		// Use our out buffer.
+		if fourStreams {
+			literals, err = huff.Decoder().Decompress4X(b.literalBuf[:0:litRegenSize], literals)
+		} else {
+			literals, err = huff.Decoder().Decompress1X(b.literalBuf[:0:litRegenSize], literals)
+		}
+		if err != nil {
+			println("decoding compressed literals:", err)
+			return err
+		}
+		// Make sure we don't leak our literals buffer
+		if len(literals) != litRegenSize {
+			return fmt.Errorf("literal output size mismatch want %d, got %d", litRegenSize, len(literals))
+		}
+		if debug {
+			printf("Decompressed %d literals into %d bytes\n", litCompSize, litRegenSize)
+		}
+	}
+
+	// Decode Sequences
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#sequences-section
+	if len(in) < 1 {
+		return ErrBlockTooSmall
+	}
+	seqHeader := in[0]
+	nSeqs := 0
+	switch {
+	case seqHeader == 0:
+		in = in[1:]
+	case seqHeader < 128:
+		nSeqs = int(seqHeader)
+		in = in[1:]
+	case seqHeader < 255:
+		if len(in) < 2 {
+			return ErrBlockTooSmall
+		}
+		nSeqs = int(seqHeader-128)<<8 | int(in[1])
+		in = in[2:]
+	case seqHeader == 255:
+		if len(in) < 3 {
+			return ErrBlockTooSmall
+		}
+		nSeqs = 0x7f00 + int(in[1]) + (int(in[2]) << 8)
+		in = in[3:]
+	}
+	// Allocate sequences
+	if cap(b.sequenceBuf) < nSeqs {
+		if b.lowMem {
+			b.sequenceBuf = make([]seq, nSeqs)
+		} else {
+			// Allocate max
+			b.sequenceBuf = make([]seq, nSeqs, maxSequences)
+		}
+	} else {
+		// Reuse buffer
+		b.sequenceBuf = b.sequenceBuf[:nSeqs]
+	}
+	var seqs = &sequenceDecs{}
+	if nSeqs > 0 {
+		if len(in) < 1 {
+			return ErrBlockTooSmall
+		}
+		br := byteReader{b: in, off: 0}
+		compMode := br.Uint8()
+		br.advance(1)
+		if debug {
+			printf("Compression modes: 0b%b", compMode)
+		}
+		for i := uint(0); i < 3; i++ {
+			mode := seqCompMode((compMode >> (6 - i*2)) & 3)
+			if debug {
+				println("Table", tableIndex(i), "is", mode)
+			}
+			var seq *sequenceDec
+			switch tableIndex(i) {
+			case tableLiteralLengths:
+				seq = &seqs.litLengths
+			case tableOffsets:
+				seq = &seqs.offsets
+			case tableMatchLengths:
+				seq = &seqs.matchLengths
+			default:
+				panic("unknown table")
+			}
+			switch mode {
+			case compModePredefined:
+				seq.fse = &fsePredef[i]
+			case compModeRLE:
+				if br.remain() < 1 {
+					return ErrBlockTooSmall
+				}
+				v := br.Uint8()
+				br.advance(1)
+				dec := fseDecoderPool.Get().(*fseDecoder)
+				symb, err := decSymbolValue(v, symbolTableX[i])
+				if err != nil {
+					printf("RLE Transform table (%v) error: %v", tableIndex(i), err)
+					return err
+				}
+				dec.setRLE(symb)
+				seq.fse = dec
+				if debug {
+					printf("RLE set to %+v, code: %v", symb, v)
+				}
+			case compModeFSE:
+				println("Reading table for", tableIndex(i))
+				dec := fseDecoderPool.Get().(*fseDecoder)
+				err := dec.readNCount(&br, uint16(maxTableSymbol[i]))
+				if err != nil {
+					println("Read table error:", err)
+					return err
+				}
+				err = dec.transform(symbolTableX[i])
+				if err != nil {
+					println("Transform table error:", err)
+					return err
+				}
+				if debug {
+					println("Read table ok", "symbolLen:", dec.symbolLen)
+				}
+				seq.fse = dec
+			case compModeRepeat:
+				seq.repeat = true
+			}
+			if br.overread() {
+				return io.ErrUnexpectedEOF
+			}
+		}
+		in = br.unread()
+	}
+
+	// Wait for history.
+	// All time spent after this is critical since it is strictly sequential.
+	if hist == nil {
+		hist = <-b.history
+		if hist.error {
+			return ErrDecoderClosed
+		}
+	}
+
+	// Decode treeless literal block.
+	if litType == literalsBlockTreeless {
+		// TODO: We could send the history early WITHOUT the stream history.
+		//   This would allow decoding treeless literals before the byte history is available.
+		//   Silencia stats: Treeless 4393, with: 32775, total: 37168, 11% treeless.
+		//   So not much obvious gain here.
+
+		if hist.huffTree == nil {
+			return errors.New("literal block was treeless, but no history was defined")
+		}
+		// Ensure we have space to store it.
+		if cap(b.literalBuf) < litRegenSize {
+			if b.lowMem {
+				b.literalBuf = make([]byte, 0, litRegenSize)
+			} else {
+				b.literalBuf = make([]byte, 0, maxCompressedLiteralSize)
+			}
+		}
+		var err error
+		// Use our out buffer.
+		huff = hist.huffTree
+		if fourStreams {
+			literals, err = huff.Decoder().Decompress4X(b.literalBuf[:0:litRegenSize], literals)
+		} else {
+			literals, err = huff.Decoder().Decompress1X(b.literalBuf[:0:litRegenSize], literals)
+		}
+		// Make sure we don't leak our literals buffer
+		if err != nil {
+			println("decompressing literals:", err)
+			return err
+		}
+		if len(literals) != litRegenSize {
+			return fmt.Errorf("literal output size mismatch want %d, got %d", litRegenSize, len(literals))
+		}
+	} else {
+		if hist.huffTree != nil && huff != nil {
+			if hist.dict == nil || hist.dict.litEnc != hist.huffTree {
+				huffDecoderPool.Put(hist.huffTree)
+			}
+			hist.huffTree = nil
+		}
+	}
+	if huff != nil {
+		hist.huffTree = huff
+	}
+	if debug {
+		println("Final literals:", len(literals), "hash:", xxhash.Sum64(literals), "and", nSeqs, "sequences.")
+	}
+
+	if nSeqs == 0 {
+		// Decompressed content is defined entirely as Literals Section content.
+		b.dst = append(b.dst, literals...)
+		if delayedHistory {
+			hist.append(literals)
+		}
+		return nil
+	}
+
+	seqs, err := seqs.mergeHistory(&hist.decoders)
+	if err != nil {
+		return err
+	}
+	if debug {
+		println("History merged ok")
+	}
+	br := &bitReader{}
+	if err := br.init(in); err != nil {
+		return err
+	}
+
+	// TODO: Investigate if sending history without decoders are faster.
+	//   This would allow the sequences to be decoded async and only have to construct stream history.
+	//   If only recent offsets were not transferred, this would be an obvious win.
+	// 	 Also, if first 3 sequences don't reference recent offsets, all sequences can be decoded.
+
+	hbytes := hist.b
+	if len(hbytes) > hist.windowSize {
+		hbytes = hbytes[len(hbytes)-hist.windowSize:]
+		// We do not need history any more.
+		if hist.dict != nil {
+			hist.dict.content = nil
+		}
+	}
+
+	if err := seqs.initialize(br, hist, literals, b.dst); err != nil {
+		println("initializing sequences:", err)
+		return err
+	}
+
+	err = seqs.decode(nSeqs, br, hbytes)
+	if err != nil {
+		return err
+	}
+	if !br.finished() {
+		return fmt.Errorf("%d extra bits on block, should be 0", br.remain())
+	}
+
+	err = br.close()
+	if err != nil {
+		printf("Closing sequences: %v, %+v\n", err, *br)
+	}
+	if len(b.data) > maxCompressedBlockSize {
+		return fmt.Errorf("compressed block size too large (%d)", len(b.data))
+	}
+	// Set output and release references.
+	b.dst = seqs.out
+	seqs.out, seqs.literals, seqs.hist = nil, nil, nil
+
+	if !delayedHistory {
+		// If we don't have delayed history, no need to update.
+		hist.recentOffsets = seqs.prevOffset
+		return nil
+	}
+	if b.Last {
+		// if last block we don't care about history.
+		println("Last block, no history returned")
+		hist.b = hist.b[:0]
+		return nil
+	}
+	hist.append(b.dst)
+	hist.recentOffsets = seqs.prevOffset
+	if debug {
+		println("Finished block with literals:", len(literals), "and", nSeqs, "sequences.")
+	}
+
+	return nil
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/blockenc.go b/vendor/github.com/klauspost/compress/zstd/blockenc.go
new file mode 100644
index 0000000..e1be092
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/blockenc.go
@@ -0,0 +1,871 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"fmt"
+	"math"
+	"math/bits"
+
+	"github.com/klauspost/compress/huff0"
+)
+
+type blockEnc struct {
+	size       int
+	literals   []byte
+	sequences  []seq
+	coders     seqCoders
+	litEnc     *huff0.Scratch
+	dictLitEnc *huff0.Scratch
+	wr         bitWriter
+
+	extraLits         int
+	output            []byte
+	recentOffsets     [3]uint32
+	prevRecentOffsets [3]uint32
+
+	last   bool
+	lowMem bool
+}
+
+// init should be used once the block has been created.
+// If called more than once, the effect is the same as calling reset.
+func (b *blockEnc) init() {
+	if b.lowMem {
+		// 1K literals
+		if cap(b.literals) < 1<<10 {
+			b.literals = make([]byte, 0, 1<<10)
+		}
+		const defSeqs = 20
+		if cap(b.sequences) < defSeqs {
+			b.sequences = make([]seq, 0, defSeqs)
+		}
+		// 1K
+		if cap(b.output) < 1<<10 {
+			b.output = make([]byte, 0, 1<<10)
+		}
+	} else {
+		if cap(b.literals) < maxCompressedBlockSize {
+			b.literals = make([]byte, 0, maxCompressedBlockSize)
+		}
+		const defSeqs = 200
+		if cap(b.sequences) < defSeqs {
+			b.sequences = make([]seq, 0, defSeqs)
+		}
+		if cap(b.output) < maxCompressedBlockSize {
+			b.output = make([]byte, 0, maxCompressedBlockSize)
+		}
+	}
+
+	if b.coders.mlEnc == nil {
+		b.coders.mlEnc = &fseEncoder{}
+		b.coders.mlPrev = &fseEncoder{}
+		b.coders.ofEnc = &fseEncoder{}
+		b.coders.ofPrev = &fseEncoder{}
+		b.coders.llEnc = &fseEncoder{}
+		b.coders.llPrev = &fseEncoder{}
+	}
+	b.litEnc = &huff0.Scratch{WantLogLess: 4}
+	b.reset(nil)
+}
+
+// initNewEncode can be used to reset offsets and encoders to the initial state.
+func (b *blockEnc) initNewEncode() {
+	b.recentOffsets = [3]uint32{1, 4, 8}
+	b.litEnc.Reuse = huff0.ReusePolicyNone
+	b.coders.setPrev(nil, nil, nil)
+}
+
+// reset will reset the block for a new encode, but in the same stream,
+// meaning that state will be carried over, but the block content is reset.
+// If a previous block is provided, the recent offsets are carried over.
+func (b *blockEnc) reset(prev *blockEnc) {
+	b.extraLits = 0
+	b.literals = b.literals[:0]
+	b.size = 0
+	b.sequences = b.sequences[:0]
+	b.output = b.output[:0]
+	b.last = false
+	if prev != nil {
+		b.recentOffsets = prev.prevRecentOffsets
+	}
+	b.dictLitEnc = nil
+}
+
+// reset will reset the block for a new encode, but in the same stream,
+// meaning that state will be carried over, but the block content is reset.
+// If a previous block is provided, the recent offsets are carried over.
+func (b *blockEnc) swapEncoders(prev *blockEnc) {
+	b.coders.swap(&prev.coders)
+	b.litEnc, prev.litEnc = prev.litEnc, b.litEnc
+}
+
+// blockHeader contains the information for a block header.
+type blockHeader uint32
+
+// setLast sets the 'last' indicator on a block.
+func (h *blockHeader) setLast(b bool) {
+	if b {
+		*h = *h | 1
+	} else {
+		const mask = (1 << 24) - 2
+		*h = *h & mask
+	}
+}
+
+// setSize will store the compressed size of a block.
+func (h *blockHeader) setSize(v uint32) {
+	const mask = 7
+	*h = (*h)&mask | blockHeader(v<<3)
+}
+
+// setType sets the block type.
+func (h *blockHeader) setType(t blockType) {
+	const mask = 1 | (((1 << 24) - 1) ^ 7)
+	*h = (*h & mask) | blockHeader(t<<1)
+}
+
+// appendTo will append the block header to a slice.
+func (h blockHeader) appendTo(b []byte) []byte {
+	return append(b, uint8(h), uint8(h>>8), uint8(h>>16))
+}
+
+// String returns a string representation of the block.
+func (h blockHeader) String() string {
+	return fmt.Sprintf("Type: %d, Size: %d, Last:%t", (h>>1)&3, h>>3, h&1 == 1)
+}
+
+// literalsHeader contains literals header information.
+type literalsHeader uint64
+
+// setType can be used to set the type of literal block.
+func (h *literalsHeader) setType(t literalsBlockType) {
+	const mask = math.MaxUint64 - 3
+	*h = (*h & mask) | literalsHeader(t)
+}
+
+// setSize can be used to set a single size, for uncompressed and RLE content.
+func (h *literalsHeader) setSize(regenLen int) {
+	inBits := bits.Len32(uint32(regenLen))
+	// Only retain 2 bits
+	const mask = 3
+	lh := uint64(*h & mask)
+	switch {
+	case inBits < 5:
+		lh |= (uint64(regenLen) << 3) | (1 << 60)
+		if debug {
+			got := int(lh>>3) & 0xff
+			if got != regenLen {
+				panic(fmt.Sprint("litRegenSize = ", regenLen, "(want) != ", got, "(got)"))
+			}
+		}
+	case inBits < 12:
+		lh |= (1 << 2) | (uint64(regenLen) << 4) | (2 << 60)
+	case inBits < 20:
+		lh |= (3 << 2) | (uint64(regenLen) << 4) | (3 << 60)
+	default:
+		panic(fmt.Errorf("internal error: block too big (%d)", regenLen))
+	}
+	*h = literalsHeader(lh)
+}
+
+// setSizes will set the size of a compressed literals section and the input length.
+func (h *literalsHeader) setSizes(compLen, inLen int, single bool) {
+	compBits, inBits := bits.Len32(uint32(compLen)), bits.Len32(uint32(inLen))
+	// Only retain 2 bits
+	const mask = 3
+	lh := uint64(*h & mask)
+	switch {
+	case compBits <= 10 && inBits <= 10:
+		if !single {
+			lh |= 1 << 2
+		}
+		lh |= (uint64(inLen) << 4) | (uint64(compLen) << (10 + 4)) | (3 << 60)
+		if debug {
+			const mmask = (1 << 24) - 1
+			n := (lh >> 4) & mmask
+			if int(n&1023) != inLen {
+				panic(fmt.Sprint("regensize:", int(n&1023), "!=", inLen, inBits))
+			}
+			if int(n>>10) != compLen {
+				panic(fmt.Sprint("compsize:", int(n>>10), "!=", compLen, compBits))
+			}
+		}
+	case compBits <= 14 && inBits <= 14:
+		lh |= (2 << 2) | (uint64(inLen) << 4) | (uint64(compLen) << (14 + 4)) | (4 << 60)
+		if single {
+			panic("single stream used with more than 10 bits length.")
+		}
+	case compBits <= 18 && inBits <= 18:
+		lh |= (3 << 2) | (uint64(inLen) << 4) | (uint64(compLen) << (18 + 4)) | (5 << 60)
+		if single {
+			panic("single stream used with more than 10 bits length.")
+		}
+	default:
+		panic("internal error: block too big")
+	}
+	*h = literalsHeader(lh)
+}
+
+// appendTo will append the literals header to a byte slice.
+func (h literalsHeader) appendTo(b []byte) []byte {
+	size := uint8(h >> 60)
+	switch size {
+	case 1:
+		b = append(b, uint8(h))
+	case 2:
+		b = append(b, uint8(h), uint8(h>>8))
+	case 3:
+		b = append(b, uint8(h), uint8(h>>8), uint8(h>>16))
+	case 4:
+		b = append(b, uint8(h), uint8(h>>8), uint8(h>>16), uint8(h>>24))
+	case 5:
+		b = append(b, uint8(h), uint8(h>>8), uint8(h>>16), uint8(h>>24), uint8(h>>32))
+	default:
+		panic(fmt.Errorf("internal error: literalsHeader has invalid size (%d)", size))
+	}
+	return b
+}
+
+// size returns the output size with currently set values.
+func (h literalsHeader) size() int {
+	return int(h >> 60)
+}
+
+func (h literalsHeader) String() string {
+	return fmt.Sprintf("Type: %d, SizeFormat: %d, Size: 0x%d, Bytes:%d", literalsBlockType(h&3), (h>>2)&3, h&((1<<60)-1)>>4, h>>60)
+}
+
+// pushOffsets will push the recent offsets to the backup store.
+func (b *blockEnc) pushOffsets() {
+	b.prevRecentOffsets = b.recentOffsets
+}
+
+// pushOffsets will push the recent offsets to the backup store.
+func (b *blockEnc) popOffsets() {
+	b.recentOffsets = b.prevRecentOffsets
+}
+
+// matchOffset will adjust recent offsets and return the adjusted one,
+// if it matches a previous offset.
+func (b *blockEnc) matchOffset(offset, lits uint32) uint32 {
+	// Check if offset is one of the recent offsets.
+	// Adjusts the output offset accordingly.
+	// Gives a tiny bit of compression, typically around 1%.
+	if true {
+		if lits > 0 {
+			switch offset {
+			case b.recentOffsets[0]:
+				offset = 1
+			case b.recentOffsets[1]:
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset = 2
+			case b.recentOffsets[2]:
+				b.recentOffsets[2] = b.recentOffsets[1]
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset = 3
+			default:
+				b.recentOffsets[2] = b.recentOffsets[1]
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset += 3
+			}
+		} else {
+			switch offset {
+			case b.recentOffsets[1]:
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset = 1
+			case b.recentOffsets[2]:
+				b.recentOffsets[2] = b.recentOffsets[1]
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset = 2
+			case b.recentOffsets[0] - 1:
+				b.recentOffsets[2] = b.recentOffsets[1]
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset = 3
+			default:
+				b.recentOffsets[2] = b.recentOffsets[1]
+				b.recentOffsets[1] = b.recentOffsets[0]
+				b.recentOffsets[0] = offset
+				offset += 3
+			}
+		}
+	} else {
+		offset += 3
+	}
+	return offset
+}
+
+// encodeRaw can be used to set the output to a raw representation of supplied bytes.
+func (b *blockEnc) encodeRaw(a []byte) {
+	var bh blockHeader
+	bh.setLast(b.last)
+	bh.setSize(uint32(len(a)))
+	bh.setType(blockTypeRaw)
+	b.output = bh.appendTo(b.output[:0])
+	b.output = append(b.output, a...)
+	if debug {
+		println("Adding RAW block, length", len(a), "last:", b.last)
+	}
+}
+
+// encodeRaw can be used to set the output to a raw representation of supplied bytes.
+func (b *blockEnc) encodeRawTo(dst, src []byte) []byte {
+	var bh blockHeader
+	bh.setLast(b.last)
+	bh.setSize(uint32(len(src)))
+	bh.setType(blockTypeRaw)
+	dst = bh.appendTo(dst)
+	dst = append(dst, src...)
+	if debug {
+		println("Adding RAW block, length", len(src), "last:", b.last)
+	}
+	return dst
+}
+
+// encodeLits can be used if the block is only litLen.
+func (b *blockEnc) encodeLits(lits []byte, raw bool) error {
+	var bh blockHeader
+	bh.setLast(b.last)
+	bh.setSize(uint32(len(lits)))
+
+	// Don't compress extremely small blocks
+	if len(lits) < 8 || (len(lits) < 32 && b.dictLitEnc == nil) || raw {
+		if debug {
+			println("Adding RAW block, length", len(lits), "last:", b.last)
+		}
+		bh.setType(blockTypeRaw)
+		b.output = bh.appendTo(b.output)
+		b.output = append(b.output, lits...)
+		return nil
+	}
+
+	var (
+		out            []byte
+		reUsed, single bool
+		err            error
+	)
+	if b.dictLitEnc != nil {
+		b.litEnc.TransferCTable(b.dictLitEnc)
+		b.litEnc.Reuse = huff0.ReusePolicyAllow
+		b.dictLitEnc = nil
+	}
+	if len(lits) >= 1024 {
+		// Use 4 Streams.
+		out, reUsed, err = huff0.Compress4X(lits, b.litEnc)
+	} else if len(lits) > 32 {
+		// Use 1 stream
+		single = true
+		out, reUsed, err = huff0.Compress1X(lits, b.litEnc)
+	} else {
+		err = huff0.ErrIncompressible
+	}
+
+	switch err {
+	case huff0.ErrIncompressible:
+		if debug {
+			println("Adding RAW block, length", len(lits), "last:", b.last)
+		}
+		bh.setType(blockTypeRaw)
+		b.output = bh.appendTo(b.output)
+		b.output = append(b.output, lits...)
+		return nil
+	case huff0.ErrUseRLE:
+		if debug {
+			println("Adding RLE block, length", len(lits))
+		}
+		bh.setType(blockTypeRLE)
+		b.output = bh.appendTo(b.output)
+		b.output = append(b.output, lits[0])
+		return nil
+	case nil:
+	default:
+		return err
+	}
+	// Compressed...
+	// Now, allow reuse
+	b.litEnc.Reuse = huff0.ReusePolicyAllow
+	bh.setType(blockTypeCompressed)
+	var lh literalsHeader
+	if reUsed {
+		if debug {
+			println("Reused tree, compressed to", len(out))
+		}
+		lh.setType(literalsBlockTreeless)
+	} else {
+		if debug {
+			println("New tree, compressed to", len(out), "tree size:", len(b.litEnc.OutTable))
+		}
+		lh.setType(literalsBlockCompressed)
+	}
+	// Set sizes
+	lh.setSizes(len(out), len(lits), single)
+	bh.setSize(uint32(len(out) + lh.size() + 1))
+
+	// Write block headers.
+	b.output = bh.appendTo(b.output)
+	b.output = lh.appendTo(b.output)
+	// Add compressed data.
+	b.output = append(b.output, out...)
+	// No sequences.
+	b.output = append(b.output, 0)
+	return nil
+}
+
+// fuzzFseEncoder can be used to fuzz the FSE encoder.
+func fuzzFseEncoder(data []byte) int {
+	if len(data) > maxSequences || len(data) < 2 {
+		return 0
+	}
+	enc := fseEncoder{}
+	hist := enc.Histogram()[:256]
+	maxSym := uint8(0)
+	for i, v := range data {
+		v = v & 63
+		data[i] = v
+		hist[v]++
+		if v > maxSym {
+			maxSym = v
+		}
+	}
+	if maxSym == 0 {
+		// All 0
+		return 0
+	}
+	maxCount := func(a []uint32) int {
+		var max uint32
+		for _, v := range a {
+			if v > max {
+				max = v
+			}
+		}
+		return int(max)
+	}
+	cnt := maxCount(hist[:maxSym])
+	if cnt == len(data) {
+		// RLE
+		return 0
+	}
+	enc.HistogramFinished(maxSym, cnt)
+	err := enc.normalizeCount(len(data))
+	if err != nil {
+		return 0
+	}
+	_, err = enc.writeCount(nil)
+	if err != nil {
+		panic(err)
+	}
+	return 1
+}
+
+// encode will encode the block and append the output in b.output.
+// Previous offset codes must be pushed if more blocks are expected.
+func (b *blockEnc) encode(org []byte, raw, rawAllLits bool) error {
+	if len(b.sequences) == 0 {
+		return b.encodeLits(b.literals, rawAllLits)
+	}
+	// We want some difference to at least account for the headers.
+	saved := b.size - len(b.literals) - (b.size >> 5)
+	if saved < 16 {
+		if org == nil {
+			return errIncompressible
+		}
+		b.popOffsets()
+		return b.encodeLits(org, rawAllLits)
+	}
+
+	var bh blockHeader
+	var lh literalsHeader
+	bh.setLast(b.last)
+	bh.setType(blockTypeCompressed)
+	// Store offset of the block header. Needed when we know the size.
+	bhOffset := len(b.output)
+	b.output = bh.appendTo(b.output)
+
+	var (
+		out            []byte
+		reUsed, single bool
+		err            error
+	)
+	if b.dictLitEnc != nil {
+		b.litEnc.TransferCTable(b.dictLitEnc)
+		b.litEnc.Reuse = huff0.ReusePolicyAllow
+		b.dictLitEnc = nil
+	}
+	if len(b.literals) >= 1024 && !raw {
+		// Use 4 Streams.
+		out, reUsed, err = huff0.Compress4X(b.literals, b.litEnc)
+	} else if len(b.literals) > 32 && !raw {
+		// Use 1 stream
+		single = true
+		out, reUsed, err = huff0.Compress1X(b.literals, b.litEnc)
+	} else {
+		err = huff0.ErrIncompressible
+	}
+
+	switch err {
+	case huff0.ErrIncompressible:
+		lh.setType(literalsBlockRaw)
+		lh.setSize(len(b.literals))
+		b.output = lh.appendTo(b.output)
+		b.output = append(b.output, b.literals...)
+		if debug {
+			println("Adding literals RAW, length", len(b.literals))
+		}
+	case huff0.ErrUseRLE:
+		lh.setType(literalsBlockRLE)
+		lh.setSize(len(b.literals))
+		b.output = lh.appendTo(b.output)
+		b.output = append(b.output, b.literals[0])
+		if debug {
+			println("Adding literals RLE")
+		}
+	case nil:
+		// Compressed litLen...
+		if reUsed {
+			if debug {
+				println("reused tree")
+			}
+			lh.setType(literalsBlockTreeless)
+		} else {
+			if debug {
+				println("new tree, size:", len(b.litEnc.OutTable))
+			}
+			lh.setType(literalsBlockCompressed)
+			if debug {
+				_, _, err := huff0.ReadTable(out, nil)
+				if err != nil {
+					panic(err)
+				}
+			}
+		}
+		lh.setSizes(len(out), len(b.literals), single)
+		if debug {
+			printf("Compressed %d literals to %d bytes", len(b.literals), len(out))
+			println("Adding literal header:", lh)
+		}
+		b.output = lh.appendTo(b.output)
+		b.output = append(b.output, out...)
+		b.litEnc.Reuse = huff0.ReusePolicyAllow
+		if debug {
+			println("Adding literals compressed")
+		}
+	default:
+		if debug {
+			println("Adding literals ERROR:", err)
+		}
+		return err
+	}
+	// Sequence compression
+
+	// Write the number of sequences
+	switch {
+	case len(b.sequences) < 128:
+		b.output = append(b.output, uint8(len(b.sequences)))
+	case len(b.sequences) < 0x7f00: // TODO: this could be wrong
+		n := len(b.sequences)
+		b.output = append(b.output, 128+uint8(n>>8), uint8(n))
+	default:
+		n := len(b.sequences) - 0x7f00
+		b.output = append(b.output, 255, uint8(n), uint8(n>>8))
+	}
+	if debug {
+		println("Encoding", len(b.sequences), "sequences")
+	}
+	b.genCodes()
+	llEnc := b.coders.llEnc
+	ofEnc := b.coders.ofEnc
+	mlEnc := b.coders.mlEnc
+	err = llEnc.normalizeCount(len(b.sequences))
+	if err != nil {
+		return err
+	}
+	err = ofEnc.normalizeCount(len(b.sequences))
+	if err != nil {
+		return err
+	}
+	err = mlEnc.normalizeCount(len(b.sequences))
+	if err != nil {
+		return err
+	}
+
+	// Choose the best compression mode for each type.
+	// Will evaluate the new vs predefined and previous.
+	chooseComp := func(cur, prev, preDef *fseEncoder) (*fseEncoder, seqCompMode) {
+		// See if predefined/previous is better
+		hist := cur.count[:cur.symbolLen]
+		nSize := cur.approxSize(hist) + cur.maxHeaderSize()
+		predefSize := preDef.approxSize(hist)
+		prevSize := prev.approxSize(hist)
+
+		// Add a small penalty for new encoders.
+		// Don't bother with extremely small (<2 byte gains).
+		nSize = nSize + (nSize+2*8*16)>>4
+		switch {
+		case predefSize <= prevSize && predefSize <= nSize || forcePreDef:
+			if debug {
+				println("Using predefined", predefSize>>3, "<=", nSize>>3)
+			}
+			return preDef, compModePredefined
+		case prevSize <= nSize:
+			if debug {
+				println("Using previous", prevSize>>3, "<=", nSize>>3)
+			}
+			return prev, compModeRepeat
+		default:
+			if debug {
+				println("Using new, predef", predefSize>>3, ". previous:", prevSize>>3, ">", nSize>>3, "header max:", cur.maxHeaderSize()>>3, "bytes")
+				println("tl:", cur.actualTableLog, "symbolLen:", cur.symbolLen, "norm:", cur.norm[:cur.symbolLen], "hist", cur.count[:cur.symbolLen])
+			}
+			return cur, compModeFSE
+		}
+	}
+
+	// Write compression mode
+	var mode uint8
+	if llEnc.useRLE {
+		mode |= uint8(compModeRLE) << 6
+		llEnc.setRLE(b.sequences[0].llCode)
+		if debug {
+			println("llEnc.useRLE")
+		}
+	} else {
+		var m seqCompMode
+		llEnc, m = chooseComp(llEnc, b.coders.llPrev, &fsePredefEnc[tableLiteralLengths])
+		mode |= uint8(m) << 6
+	}
+	if ofEnc.useRLE {
+		mode |= uint8(compModeRLE) << 4
+		ofEnc.setRLE(b.sequences[0].ofCode)
+		if debug {
+			println("ofEnc.useRLE")
+		}
+	} else {
+		var m seqCompMode
+		ofEnc, m = chooseComp(ofEnc, b.coders.ofPrev, &fsePredefEnc[tableOffsets])
+		mode |= uint8(m) << 4
+	}
+
+	if mlEnc.useRLE {
+		mode |= uint8(compModeRLE) << 2
+		mlEnc.setRLE(b.sequences[0].mlCode)
+		if debug {
+			println("mlEnc.useRLE, code: ", b.sequences[0].mlCode, "value", b.sequences[0].matchLen)
+		}
+	} else {
+		var m seqCompMode
+		mlEnc, m = chooseComp(mlEnc, b.coders.mlPrev, &fsePredefEnc[tableMatchLengths])
+		mode |= uint8(m) << 2
+	}
+	b.output = append(b.output, mode)
+	if debug {
+		printf("Compression modes: 0b%b", mode)
+	}
+	b.output, err = llEnc.writeCount(b.output)
+	if err != nil {
+		return err
+	}
+	start := len(b.output)
+	b.output, err = ofEnc.writeCount(b.output)
+	if err != nil {
+		return err
+	}
+	if false {
+		println("block:", b.output[start:], "tablelog", ofEnc.actualTableLog, "maxcount:", ofEnc.maxCount)
+		fmt.Printf("selected TableLog: %d, Symbol length: %d\n", ofEnc.actualTableLog, ofEnc.symbolLen)
+		for i, v := range ofEnc.norm[:ofEnc.symbolLen] {
+			fmt.Printf("%3d: %5d -> %4d \n", i, ofEnc.count[i], v)
+		}
+	}
+	b.output, err = mlEnc.writeCount(b.output)
+	if err != nil {
+		return err
+	}
+
+	// Maybe in block?
+	wr := &b.wr
+	wr.reset(b.output)
+
+	var ll, of, ml cState
+
+	// Current sequence
+	seq := len(b.sequences) - 1
+	s := b.sequences[seq]
+	llEnc.setBits(llBitsTable[:])
+	mlEnc.setBits(mlBitsTable[:])
+	ofEnc.setBits(nil)
+
+	llTT, ofTT, mlTT := llEnc.ct.symbolTT[:256], ofEnc.ct.symbolTT[:256], mlEnc.ct.symbolTT[:256]
+
+	// We have 3 bounds checks here (and in the loop).
+	// Since we are iterating backwards it is kinda hard to avoid.
+	llB, ofB, mlB := llTT[s.llCode], ofTT[s.ofCode], mlTT[s.mlCode]
+	ll.init(wr, &llEnc.ct, llB)
+	of.init(wr, &ofEnc.ct, ofB)
+	wr.flush32()
+	ml.init(wr, &mlEnc.ct, mlB)
+
+	// Each of these lookups also generates a bounds check.
+	wr.addBits32NC(s.litLen, llB.outBits)
+	wr.addBits32NC(s.matchLen, mlB.outBits)
+	wr.flush32()
+	wr.addBits32NC(s.offset, ofB.outBits)
+	if debugSequences {
+		println("Encoded seq", seq, s, "codes:", s.llCode, s.mlCode, s.ofCode, "states:", ll.state, ml.state, of.state, "bits:", llB, mlB, ofB)
+	}
+	seq--
+	if llEnc.maxBits+mlEnc.maxBits+ofEnc.maxBits <= 32 {
+		// No need to flush (common)
+		for seq >= 0 {
+			s = b.sequences[seq]
+			wr.flush32()
+			llB, ofB, mlB := llTT[s.llCode], ofTT[s.ofCode], mlTT[s.mlCode]
+			// tabelog max is 8 for all.
+			of.encode(ofB)
+			ml.encode(mlB)
+			ll.encode(llB)
+			wr.flush32()
+
+			// We checked that all can stay within 32 bits
+			wr.addBits32NC(s.litLen, llB.outBits)
+			wr.addBits32NC(s.matchLen, mlB.outBits)
+			wr.addBits32NC(s.offset, ofB.outBits)
+
+			if debugSequences {
+				println("Encoded seq", seq, s)
+			}
+
+			seq--
+		}
+	} else {
+		for seq >= 0 {
+			s = b.sequences[seq]
+			wr.flush32()
+			llB, ofB, mlB := llTT[s.llCode], ofTT[s.ofCode], mlTT[s.mlCode]
+			// tabelog max is below 8 for each.
+			of.encode(ofB)
+			ml.encode(mlB)
+			ll.encode(llB)
+			wr.flush32()
+
+			// ml+ll = max 32 bits total
+			wr.addBits32NC(s.litLen, llB.outBits)
+			wr.addBits32NC(s.matchLen, mlB.outBits)
+			wr.flush32()
+			wr.addBits32NC(s.offset, ofB.outBits)
+
+			if debugSequences {
+				println("Encoded seq", seq, s)
+			}
+
+			seq--
+		}
+	}
+	ml.flush(mlEnc.actualTableLog)
+	of.flush(ofEnc.actualTableLog)
+	ll.flush(llEnc.actualTableLog)
+	err = wr.close()
+	if err != nil {
+		return err
+	}
+	b.output = wr.out
+
+	if len(b.output)-3-bhOffset >= b.size {
+		// Maybe even add a bigger margin.
+		b.litEnc.Reuse = huff0.ReusePolicyNone
+		return errIncompressible
+	}
+
+	// Size is output minus block header.
+	bh.setSize(uint32(len(b.output)-bhOffset) - 3)
+	if debug {
+		println("Rewriting block header", bh)
+	}
+	_ = bh.appendTo(b.output[bhOffset:bhOffset])
+	b.coders.setPrev(llEnc, mlEnc, ofEnc)
+	return nil
+}
+
+var errIncompressible = errors.New("incompressible")
+
+func (b *blockEnc) genCodes() {
+	if len(b.sequences) == 0 {
+		// nothing to do
+		return
+	}
+
+	if len(b.sequences) > math.MaxUint16 {
+		panic("can only encode up to 64K sequences")
+	}
+	// No bounds checks after here:
+	llH := b.coders.llEnc.Histogram()[:256]
+	ofH := b.coders.ofEnc.Histogram()[:256]
+	mlH := b.coders.mlEnc.Histogram()[:256]
+	for i := range llH {
+		llH[i] = 0
+	}
+	for i := range ofH {
+		ofH[i] = 0
+	}
+	for i := range mlH {
+		mlH[i] = 0
+	}
+
+	var llMax, ofMax, mlMax uint8
+	for i, seq := range b.sequences {
+		v := llCode(seq.litLen)
+		seq.llCode = v
+		llH[v]++
+		if v > llMax {
+			llMax = v
+		}
+
+		v = ofCode(seq.offset)
+		seq.ofCode = v
+		ofH[v]++
+		if v > ofMax {
+			ofMax = v
+		}
+
+		v = mlCode(seq.matchLen)
+		seq.mlCode = v
+		mlH[v]++
+		if v > mlMax {
+			mlMax = v
+			if debugAsserts && mlMax > maxMatchLengthSymbol {
+				panic(fmt.Errorf("mlMax > maxMatchLengthSymbol (%d), matchlen: %d", mlMax, seq.matchLen))
+			}
+		}
+		b.sequences[i] = seq
+	}
+	maxCount := func(a []uint32) int {
+		var max uint32
+		for _, v := range a {
+			if v > max {
+				max = v
+			}
+		}
+		return int(max)
+	}
+	if debugAsserts && mlMax > maxMatchLengthSymbol {
+		panic(fmt.Errorf("mlMax > maxMatchLengthSymbol (%d)", mlMax))
+	}
+	if debugAsserts && ofMax > maxOffsetBits {
+		panic(fmt.Errorf("ofMax > maxOffsetBits (%d)", ofMax))
+	}
+	if debugAsserts && llMax > maxLiteralLengthSymbol {
+		panic(fmt.Errorf("llMax > maxLiteralLengthSymbol (%d)", llMax))
+	}
+
+	b.coders.mlEnc.HistogramFinished(mlMax, maxCount(mlH[:mlMax+1]))
+	b.coders.ofEnc.HistogramFinished(ofMax, maxCount(ofH[:ofMax+1]))
+	b.coders.llEnc.HistogramFinished(llMax, maxCount(llH[:llMax+1]))
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/blocktype_string.go b/vendor/github.com/klauspost/compress/zstd/blocktype_string.go
new file mode 100644
index 0000000..01a01e4
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/blocktype_string.go
@@ -0,0 +1,85 @@
+// Code generated by "stringer -type=blockType,literalsBlockType,seqCompMode,tableIndex"; DO NOT EDIT.
+
+package zstd
+
+import "strconv"
+
+func _() {
+	// An "invalid array index" compiler error signifies that the constant values have changed.
+	// Re-run the stringer command to generate them again.
+	var x [1]struct{}
+	_ = x[blockTypeRaw-0]
+	_ = x[blockTypeRLE-1]
+	_ = x[blockTypeCompressed-2]
+	_ = x[blockTypeReserved-3]
+}
+
+const _blockType_name = "blockTypeRawblockTypeRLEblockTypeCompressedblockTypeReserved"
+
+var _blockType_index = [...]uint8{0, 12, 24, 43, 60}
+
+func (i blockType) String() string {
+	if i >= blockType(len(_blockType_index)-1) {
+		return "blockType(" + strconv.FormatInt(int64(i), 10) + ")"
+	}
+	return _blockType_name[_blockType_index[i]:_blockType_index[i+1]]
+}
+func _() {
+	// An "invalid array index" compiler error signifies that the constant values have changed.
+	// Re-run the stringer command to generate them again.
+	var x [1]struct{}
+	_ = x[literalsBlockRaw-0]
+	_ = x[literalsBlockRLE-1]
+	_ = x[literalsBlockCompressed-2]
+	_ = x[literalsBlockTreeless-3]
+}
+
+const _literalsBlockType_name = "literalsBlockRawliteralsBlockRLEliteralsBlockCompressedliteralsBlockTreeless"
+
+var _literalsBlockType_index = [...]uint8{0, 16, 32, 55, 76}
+
+func (i literalsBlockType) String() string {
+	if i >= literalsBlockType(len(_literalsBlockType_index)-1) {
+		return "literalsBlockType(" + strconv.FormatInt(int64(i), 10) + ")"
+	}
+	return _literalsBlockType_name[_literalsBlockType_index[i]:_literalsBlockType_index[i+1]]
+}
+func _() {
+	// An "invalid array index" compiler error signifies that the constant values have changed.
+	// Re-run the stringer command to generate them again.
+	var x [1]struct{}
+	_ = x[compModePredefined-0]
+	_ = x[compModeRLE-1]
+	_ = x[compModeFSE-2]
+	_ = x[compModeRepeat-3]
+}
+
+const _seqCompMode_name = "compModePredefinedcompModeRLEcompModeFSEcompModeRepeat"
+
+var _seqCompMode_index = [...]uint8{0, 18, 29, 40, 54}
+
+func (i seqCompMode) String() string {
+	if i >= seqCompMode(len(_seqCompMode_index)-1) {
+		return "seqCompMode(" + strconv.FormatInt(int64(i), 10) + ")"
+	}
+	return _seqCompMode_name[_seqCompMode_index[i]:_seqCompMode_index[i+1]]
+}
+func _() {
+	// An "invalid array index" compiler error signifies that the constant values have changed.
+	// Re-run the stringer command to generate them again.
+	var x [1]struct{}
+	_ = x[tableLiteralLengths-0]
+	_ = x[tableOffsets-1]
+	_ = x[tableMatchLengths-2]
+}
+
+const _tableIndex_name = "tableLiteralLengthstableOffsetstableMatchLengths"
+
+var _tableIndex_index = [...]uint8{0, 19, 31, 48}
+
+func (i tableIndex) String() string {
+	if i >= tableIndex(len(_tableIndex_index)-1) {
+		return "tableIndex(" + strconv.FormatInt(int64(i), 10) + ")"
+	}
+	return _tableIndex_name[_tableIndex_index[i]:_tableIndex_index[i+1]]
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/bytebuf.go b/vendor/github.com/klauspost/compress/zstd/bytebuf.go
new file mode 100644
index 0000000..658ef78
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/bytebuf.go
@@ -0,0 +1,127 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"fmt"
+	"io"
+	"io/ioutil"
+)
+
+type byteBuffer interface {
+	// Read up to 8 bytes.
+	// Returns nil if no more input is available.
+	readSmall(n int) []byte
+
+	// Read >8 bytes.
+	// MAY use the destination slice.
+	readBig(n int, dst []byte) ([]byte, error)
+
+	// Read a single byte.
+	readByte() (byte, error)
+
+	// Skip n bytes.
+	skipN(n int) error
+}
+
+// in-memory buffer
+type byteBuf []byte
+
+func (b *byteBuf) readSmall(n int) []byte {
+	if debugAsserts && n > 8 {
+		panic(fmt.Errorf("small read > 8 (%d). use readBig", n))
+	}
+	bb := *b
+	if len(bb) < n {
+		return nil
+	}
+	r := bb[:n]
+	*b = bb[n:]
+	return r
+}
+
+func (b *byteBuf) readBig(n int, dst []byte) ([]byte, error) {
+	bb := *b
+	if len(bb) < n {
+		return nil, io.ErrUnexpectedEOF
+	}
+	r := bb[:n]
+	*b = bb[n:]
+	return r, nil
+}
+
+func (b *byteBuf) remain() []byte {
+	return *b
+}
+
+func (b *byteBuf) readByte() (byte, error) {
+	bb := *b
+	if len(bb) < 1 {
+		return 0, nil
+	}
+	r := bb[0]
+	*b = bb[1:]
+	return r, nil
+}
+
+func (b *byteBuf) skipN(n int) error {
+	bb := *b
+	if len(bb) < n {
+		return io.ErrUnexpectedEOF
+	}
+	*b = bb[n:]
+	return nil
+}
+
+// wrapper around a reader.
+type readerWrapper struct {
+	r   io.Reader
+	tmp [8]byte
+}
+
+func (r *readerWrapper) readSmall(n int) []byte {
+	if debugAsserts && n > 8 {
+		panic(fmt.Errorf("small read > 8 (%d). use readBig", n))
+	}
+	n2, err := io.ReadFull(r.r, r.tmp[:n])
+	// We only really care about the actual bytes read.
+	if n2 != n {
+		if debug {
+			println("readSmall: got", n2, "want", n, "err", err)
+		}
+		return nil
+	}
+	return r.tmp[:n]
+}
+
+func (r *readerWrapper) readBig(n int, dst []byte) ([]byte, error) {
+	if cap(dst) < n {
+		dst = make([]byte, n)
+	}
+	n2, err := io.ReadFull(r.r, dst[:n])
+	if err == io.EOF && n > 0 {
+		err = io.ErrUnexpectedEOF
+	}
+	return dst[:n2], err
+}
+
+func (r *readerWrapper) readByte() (byte, error) {
+	n2, err := r.r.Read(r.tmp[:1])
+	if err != nil {
+		return 0, err
+	}
+	if n2 != 1 {
+		return 0, io.ErrUnexpectedEOF
+	}
+	return r.tmp[0], nil
+}
+
+func (r *readerWrapper) skipN(n int) error {
+	n2, err := io.CopyN(ioutil.Discard, r.r, int64(n))
+	if n2 != int64(n) {
+		err = io.ErrUnexpectedEOF
+	}
+	return err
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/bytereader.go b/vendor/github.com/klauspost/compress/zstd/bytereader.go
new file mode 100644
index 0000000..2c4fca1
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/bytereader.go
@@ -0,0 +1,88 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+// byteReader provides a byte reader that reads
+// little endian values from a byte stream.
+// The input stream is manually advanced.
+// The reader performs no bounds checks.
+type byteReader struct {
+	b   []byte
+	off int
+}
+
+// init will initialize the reader and set the input.
+func (b *byteReader) init(in []byte) {
+	b.b = in
+	b.off = 0
+}
+
+// advance the stream b n bytes.
+func (b *byteReader) advance(n uint) {
+	b.off += int(n)
+}
+
+// overread returns whether we have advanced too far.
+func (b *byteReader) overread() bool {
+	return b.off > len(b.b)
+}
+
+// Int32 returns a little endian int32 starting at current offset.
+func (b byteReader) Int32() int32 {
+	b2 := b.b[b.off:]
+	b2 = b2[:4]
+	v3 := int32(b2[3])
+	v2 := int32(b2[2])
+	v1 := int32(b2[1])
+	v0 := int32(b2[0])
+	return v0 | (v1 << 8) | (v2 << 16) | (v3 << 24)
+}
+
+// Uint8 returns the next byte
+func (b *byteReader) Uint8() uint8 {
+	v := b.b[b.off]
+	return v
+}
+
+// Uint32 returns a little endian uint32 starting at current offset.
+func (b byteReader) Uint32() uint32 {
+	if r := b.remain(); r < 4 {
+		// Very rare
+		v := uint32(0)
+		for i := 1; i <= r; i++ {
+			v = (v << 8) | uint32(b.b[len(b.b)-i])
+		}
+		return v
+	}
+	b2 := b.b[b.off:]
+	b2 = b2[:4]
+	v3 := uint32(b2[3])
+	v2 := uint32(b2[2])
+	v1 := uint32(b2[1])
+	v0 := uint32(b2[0])
+	return v0 | (v1 << 8) | (v2 << 16) | (v3 << 24)
+}
+
+// Uint32NC returns a little endian uint32 starting at current offset.
+// The caller must be sure if there are at least 4 bytes left.
+func (b byteReader) Uint32NC() uint32 {
+	b2 := b.b[b.off:]
+	b2 = b2[:4]
+	v3 := uint32(b2[3])
+	v2 := uint32(b2[2])
+	v1 := uint32(b2[1])
+	v0 := uint32(b2[0])
+	return v0 | (v1 << 8) | (v2 << 16) | (v3 << 24)
+}
+
+// unread returns the unread portion of the input.
+func (b byteReader) unread() []byte {
+	return b.b[b.off:]
+}
+
+// remain will return the number of bytes remaining.
+func (b byteReader) remain() int {
+	return len(b.b) - b.off
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/decodeheader.go b/vendor/github.com/klauspost/compress/zstd/decodeheader.go
new file mode 100644
index 0000000..69736e8
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/decodeheader.go
@@ -0,0 +1,202 @@
+// Copyright 2020+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+
+package zstd
+
+import (
+	"bytes"
+	"errors"
+	"io"
+)
+
+// HeaderMaxSize is the maximum size of a Frame and Block Header.
+// If less is sent to Header.Decode it *may* still contain enough information.
+const HeaderMaxSize = 14 + 3
+
+// Header contains information about the first frame and block within that.
+type Header struct {
+	// Window Size the window of data to keep while decoding.
+	// Will only be set if HasFCS is false.
+	WindowSize uint64
+
+	// Frame content size.
+	// Expected size of the entire frame.
+	FrameContentSize uint64
+
+	// Dictionary ID.
+	// If 0, no dictionary.
+	DictionaryID uint32
+
+	// First block information.
+	FirstBlock struct {
+		// OK will be set if first block could be decoded.
+		OK bool
+
+		// Is this the last block of a frame?
+		Last bool
+
+		// Is the data compressed?
+		// If true CompressedSize will be populated.
+		// Unfortunately DecompressedSize cannot be determined
+		// without decoding the blocks.
+		Compressed bool
+
+		// DecompressedSize is the expected decompressed size of the block.
+		// Will be 0 if it cannot be determined.
+		DecompressedSize int
+
+		// CompressedSize of the data in the block.
+		// Does not include the block header.
+		// Will be equal to DecompressedSize if not Compressed.
+		CompressedSize int
+	}
+
+	// Skippable will be true if the frame is meant to be skipped.
+	// No other information will be populated.
+	Skippable bool
+
+	// If set there is a checksum present for the block content.
+	HasCheckSum bool
+
+	// If this is true FrameContentSize will have a valid value
+	HasFCS bool
+
+	SingleSegment bool
+}
+
+// Decode the header from the beginning of the stream.
+// This will decode the frame header and the first block header if enough bytes are provided.
+// It is recommended to provide at least HeaderMaxSize bytes.
+// If the frame header cannot be read an error will be returned.
+// If there isn't enough input, io.ErrUnexpectedEOF is returned.
+// The FirstBlock.OK will indicate if enough information was available to decode the first block header.
+func (h *Header) Decode(in []byte) error {
+	if len(in) < 4 {
+		return io.ErrUnexpectedEOF
+	}
+	b, in := in[:4], in[4:]
+	if !bytes.Equal(b, frameMagic) {
+		if !bytes.Equal(b[1:4], skippableFrameMagic) || b[0]&0xf0 != 0x50 {
+			return ErrMagicMismatch
+		}
+		*h = Header{Skippable: true}
+		return nil
+	}
+	if len(in) < 1 {
+		return io.ErrUnexpectedEOF
+	}
+
+	// Clear output
+	*h = Header{}
+	fhd, in := in[0], in[1:]
+	h.SingleSegment = fhd&(1<<5) != 0
+	h.HasCheckSum = fhd&(1<<2) != 0
+
+	if fhd&(1<<3) != 0 {
+		return errors.New("reserved bit set on frame header")
+	}
+
+	// Read Window_Descriptor
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#window_descriptor
+	if !h.SingleSegment {
+		if len(in) < 1 {
+			return io.ErrUnexpectedEOF
+		}
+		var wd byte
+		wd, in = in[0], in[1:]
+		windowLog := 10 + (wd >> 3)
+		windowBase := uint64(1) << windowLog
+		windowAdd := (windowBase / 8) * uint64(wd&0x7)
+		h.WindowSize = windowBase + windowAdd
+	}
+
+	// Read Dictionary_ID
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#dictionary_id
+	if size := fhd & 3; size != 0 {
+		if size == 3 {
+			size = 4
+		}
+		if len(in) < int(size) {
+			return io.ErrUnexpectedEOF
+		}
+		b, in = in[:size], in[size:]
+		if b == nil {
+			return io.ErrUnexpectedEOF
+		}
+		switch size {
+		case 1:
+			h.DictionaryID = uint32(b[0])
+		case 2:
+			h.DictionaryID = uint32(b[0]) | (uint32(b[1]) << 8)
+		case 4:
+			h.DictionaryID = uint32(b[0]) | (uint32(b[1]) << 8) | (uint32(b[2]) << 16) | (uint32(b[3]) << 24)
+		}
+	}
+
+	// Read Frame_Content_Size
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#frame_content_size
+	var fcsSize int
+	v := fhd >> 6
+	switch v {
+	case 0:
+		if h.SingleSegment {
+			fcsSize = 1
+		}
+	default:
+		fcsSize = 1 << v
+	}
+
+	if fcsSize > 0 {
+		h.HasFCS = true
+		if len(in) < fcsSize {
+			return io.ErrUnexpectedEOF
+		}
+		b, in = in[:fcsSize], in[fcsSize:]
+		if b == nil {
+			return io.ErrUnexpectedEOF
+		}
+		switch fcsSize {
+		case 1:
+			h.FrameContentSize = uint64(b[0])
+		case 2:
+			// When FCS_Field_Size is 2, the offset of 256 is added.
+			h.FrameContentSize = uint64(b[0]) | (uint64(b[1]) << 8) + 256
+		case 4:
+			h.FrameContentSize = uint64(b[0]) | (uint64(b[1]) << 8) | (uint64(b[2]) << 16) | (uint64(b[3]) << 24)
+		case 8:
+			d1 := uint32(b[0]) | (uint32(b[1]) << 8) | (uint32(b[2]) << 16) | (uint32(b[3]) << 24)
+			d2 := uint32(b[4]) | (uint32(b[5]) << 8) | (uint32(b[6]) << 16) | (uint32(b[7]) << 24)
+			h.FrameContentSize = uint64(d1) | (uint64(d2) << 32)
+		}
+	}
+
+	// Frame Header done, we will not fail from now on.
+	if len(in) < 3 {
+		return nil
+	}
+	tmp := in[:3]
+	bh := uint32(tmp[0]) | (uint32(tmp[1]) << 8) | (uint32(tmp[2]) << 16)
+	h.FirstBlock.Last = bh&1 != 0
+	blockType := blockType((bh >> 1) & 3)
+	// find size.
+	cSize := int(bh >> 3)
+	switch blockType {
+	case blockTypeReserved:
+		return nil
+	case blockTypeRLE:
+		h.FirstBlock.Compressed = true
+		h.FirstBlock.DecompressedSize = cSize
+		h.FirstBlock.CompressedSize = 1
+	case blockTypeCompressed:
+		h.FirstBlock.Compressed = true
+		h.FirstBlock.CompressedSize = cSize
+	case blockTypeRaw:
+		h.FirstBlock.DecompressedSize = cSize
+		h.FirstBlock.CompressedSize = cSize
+	default:
+		panic("Invalid block type")
+	}
+
+	h.FirstBlock.OK = true
+	return nil
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/decoder.go b/vendor/github.com/klauspost/compress/zstd/decoder.go
new file mode 100644
index 0000000..f593e46
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/decoder.go
@@ -0,0 +1,557 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"io"
+	"sync"
+)
+
+// Decoder provides decoding of zstandard streams.
+// The decoder has been designed to operate without allocations after a warmup.
+// This means that you should store the decoder for best performance.
+// To re-use a stream decoder, use the Reset(r io.Reader) error to switch to another stream.
+// A decoder can safely be re-used even if the previous stream failed.
+// To release the resources, you must call the Close() function on a decoder.
+type Decoder struct {
+	o decoderOptions
+
+	// Unreferenced decoders, ready for use.
+	decoders chan *blockDec
+
+	// Streams ready to be decoded.
+	stream chan decodeStream
+
+	// Current read position used for Reader functionality.
+	current decoderState
+
+	// Custom dictionaries.
+	// Always uses copies.
+	dicts map[uint32]dict
+
+	// streamWg is the waitgroup for all streams
+	streamWg sync.WaitGroup
+}
+
+// decoderState is used for maintaining state when the decoder
+// is used for streaming.
+type decoderState struct {
+	// current block being written to stream.
+	decodeOutput
+
+	// output in order to be written to stream.
+	output chan decodeOutput
+
+	// cancel remaining output.
+	cancel chan struct{}
+
+	flushed bool
+}
+
+var (
+	// Check the interfaces we want to support.
+	_ = io.WriterTo(&Decoder{})
+	_ = io.Reader(&Decoder{})
+)
+
+// NewReader creates a new decoder.
+// A nil Reader can be provided in which case Reset can be used to start a decode.
+//
+// A Decoder can be used in two modes:
+//
+// 1) As a stream, or
+// 2) For stateless decoding using DecodeAll.
+//
+// Only a single stream can be decoded concurrently, but the same decoder
+// can run multiple concurrent stateless decodes. It is even possible to
+// use stateless decodes while a stream is being decoded.
+//
+// The Reset function can be used to initiate a new stream, which is will considerably
+// reduce the allocations normally caused by NewReader.
+func NewReader(r io.Reader, opts ...DOption) (*Decoder, error) {
+	initPredefined()
+	var d Decoder
+	d.o.setDefault()
+	for _, o := range opts {
+		err := o(&d.o)
+		if err != nil {
+			return nil, err
+		}
+	}
+	d.current.output = make(chan decodeOutput, d.o.concurrent)
+	d.current.flushed = true
+
+	if r == nil {
+		d.current.err = ErrDecoderNilInput
+	}
+
+	// Transfer option dicts.
+	d.dicts = make(map[uint32]dict, len(d.o.dicts))
+	for _, dc := range d.o.dicts {
+		d.dicts[dc.id] = dc
+	}
+	d.o.dicts = nil
+
+	// Create decoders
+	d.decoders = make(chan *blockDec, d.o.concurrent)
+	for i := 0; i < d.o.concurrent; i++ {
+		dec := newBlockDec(d.o.lowMem)
+		dec.localFrame = newFrameDec(d.o)
+		d.decoders <- dec
+	}
+
+	if r == nil {
+		return &d, nil
+	}
+	return &d, d.Reset(r)
+}
+
+// Read bytes from the decompressed stream into p.
+// Returns the number of bytes written and any error that occurred.
+// When the stream is done, io.EOF will be returned.
+func (d *Decoder) Read(p []byte) (int, error) {
+	if d.stream == nil {
+		return 0, ErrDecoderNilInput
+	}
+	var n int
+	for {
+		if len(d.current.b) > 0 {
+			filled := copy(p, d.current.b)
+			p = p[filled:]
+			d.current.b = d.current.b[filled:]
+			n += filled
+		}
+		if len(p) == 0 {
+			break
+		}
+		if len(d.current.b) == 0 {
+			// We have an error and no more data
+			if d.current.err != nil {
+				break
+			}
+			if !d.nextBlock(n == 0) {
+				return n, nil
+			}
+		}
+	}
+	if len(d.current.b) > 0 {
+		if debug {
+			println("returning", n, "still bytes left:", len(d.current.b))
+		}
+		// Only return error at end of block
+		return n, nil
+	}
+	if d.current.err != nil {
+		d.drainOutput()
+	}
+	if debug {
+		println("returning", n, d.current.err, len(d.decoders))
+	}
+	return n, d.current.err
+}
+
+// Reset will reset the decoder the supplied stream after the current has finished processing.
+// Note that this functionality cannot be used after Close has been called.
+// Reset can be called with a nil reader to release references to the previous reader.
+// After being called with a nil reader, no other operations than Reset or DecodeAll or Close
+// should be used.
+func (d *Decoder) Reset(r io.Reader) error {
+	if d.current.err == ErrDecoderClosed {
+		return d.current.err
+	}
+
+	d.drainOutput()
+
+	if r == nil {
+		d.current.err = ErrDecoderNilInput
+		d.current.flushed = true
+		return nil
+	}
+
+	if d.stream == nil {
+		d.stream = make(chan decodeStream, 1)
+		d.streamWg.Add(1)
+		go d.startStreamDecoder(d.stream)
+	}
+
+	// If bytes buffer and < 1MB, do sync decoding anyway.
+	if bb, ok := r.(byter); ok && bb.Len() < 1<<20 {
+		bb2 := bb
+		if debug {
+			println("*bytes.Buffer detected, doing sync decode, len:", bb.Len())
+		}
+		b := bb2.Bytes()
+		var dst []byte
+		if cap(d.current.b) > 0 {
+			dst = d.current.b
+		}
+
+		dst, err := d.DecodeAll(b, dst[:0])
+		if err == nil {
+			err = io.EOF
+		}
+		d.current.b = dst
+		d.current.err = err
+		d.current.flushed = true
+		if debug {
+			println("sync decode to", len(dst), "bytes, err:", err)
+		}
+		return nil
+	}
+
+	// Remove current block.
+	d.current.decodeOutput = decodeOutput{}
+	d.current.err = nil
+	d.current.cancel = make(chan struct{})
+	d.current.flushed = false
+	d.current.d = nil
+
+	d.stream <- decodeStream{
+		r:      r,
+		output: d.current.output,
+		cancel: d.current.cancel,
+	}
+	return nil
+}
+
+// drainOutput will drain the output until errEndOfStream is sent.
+func (d *Decoder) drainOutput() {
+	if d.current.cancel != nil {
+		println("cancelling current")
+		close(d.current.cancel)
+		d.current.cancel = nil
+	}
+	if d.current.d != nil {
+		if debug {
+			printf("re-adding current decoder %p, decoders: %d", d.current.d, len(d.decoders))
+		}
+		d.decoders <- d.current.d
+		d.current.d = nil
+		d.current.b = nil
+	}
+	if d.current.output == nil || d.current.flushed {
+		println("current already flushed")
+		return
+	}
+	for v := range d.current.output {
+		if v.d != nil {
+			if debug {
+				printf("re-adding decoder %p", v.d)
+			}
+			d.decoders <- v.d
+		}
+		if v.err == errEndOfStream {
+			println("current flushed")
+			d.current.flushed = true
+			return
+		}
+	}
+}
+
+// WriteTo writes data to w until there's no more data to write or when an error occurs.
+// The return value n is the number of bytes written.
+// Any error encountered during the write is also returned.
+func (d *Decoder) WriteTo(w io.Writer) (int64, error) {
+	if d.stream == nil {
+		return 0, ErrDecoderNilInput
+	}
+	var n int64
+	for {
+		if len(d.current.b) > 0 {
+			n2, err2 := w.Write(d.current.b)
+			n += int64(n2)
+			if err2 != nil && d.current.err == nil {
+				d.current.err = err2
+				break
+			}
+		}
+		if d.current.err != nil {
+			break
+		}
+		d.nextBlock(true)
+	}
+	err := d.current.err
+	if err != nil {
+		d.drainOutput()
+	}
+	if err == io.EOF {
+		err = nil
+	}
+	return n, err
+}
+
+// DecodeAll allows stateless decoding of a blob of bytes.
+// Output will be appended to dst, so if the destination size is known
+// you can pre-allocate the destination slice to avoid allocations.
+// DecodeAll can be used concurrently.
+// The Decoder concurrency limits will be respected.
+func (d *Decoder) DecodeAll(input, dst []byte) ([]byte, error) {
+	if d.current.err == ErrDecoderClosed {
+		return dst, ErrDecoderClosed
+	}
+
+	// Grab a block decoder and frame decoder.
+	block := <-d.decoders
+	frame := block.localFrame
+	defer func() {
+		if debug {
+			printf("re-adding decoder: %p", block)
+		}
+		frame.rawInput = nil
+		frame.bBuf = nil
+		d.decoders <- block
+	}()
+	frame.bBuf = input
+
+	for {
+		frame.history.reset()
+		err := frame.reset(&frame.bBuf)
+		if err == io.EOF {
+			if debug {
+				println("frame reset return EOF")
+			}
+			return dst, nil
+		}
+		if frame.DictionaryID != nil {
+			dict, ok := d.dicts[*frame.DictionaryID]
+			if !ok {
+				return nil, ErrUnknownDictionary
+			}
+			frame.history.setDict(&dict)
+		}
+		if err != nil {
+			return dst, err
+		}
+		if frame.FrameContentSize > d.o.maxDecodedSize-uint64(len(dst)) {
+			return dst, ErrDecoderSizeExceeded
+		}
+		if frame.FrameContentSize > 0 && frame.FrameContentSize < 1<<30 {
+			// Never preallocate moe than 1 GB up front.
+			if cap(dst)-len(dst) < int(frame.FrameContentSize) {
+				dst2 := make([]byte, len(dst), len(dst)+int(frame.FrameContentSize))
+				copy(dst2, dst)
+				dst = dst2
+			}
+		}
+		if cap(dst) == 0 {
+			// Allocate len(input) * 2 by default if nothing is provided
+			// and we didn't get frame content size.
+			size := len(input) * 2
+			// Cap to 1 MB.
+			if size > 1<<20 {
+				size = 1 << 20
+			}
+			if uint64(size) > d.o.maxDecodedSize {
+				size = int(d.o.maxDecodedSize)
+			}
+			dst = make([]byte, 0, size)
+		}
+
+		dst, err = frame.runDecoder(dst, block)
+		if err != nil {
+			return dst, err
+		}
+		if len(frame.bBuf) == 0 {
+			if debug {
+				println("frame dbuf empty")
+			}
+			break
+		}
+	}
+	return dst, nil
+}
+
+// nextBlock returns the next block.
+// If an error occurs d.err will be set.
+// Optionally the function can block for new output.
+// If non-blocking mode is used the returned boolean will be false
+// if no data was available without blocking.
+func (d *Decoder) nextBlock(blocking bool) (ok bool) {
+	if d.current.d != nil {
+		if debug {
+			printf("re-adding current decoder %p", d.current.d)
+		}
+		d.decoders <- d.current.d
+		d.current.d = nil
+	}
+	if d.current.err != nil {
+		// Keep error state.
+		return blocking
+	}
+
+	if blocking {
+		d.current.decodeOutput = <-d.current.output
+	} else {
+		select {
+		case d.current.decodeOutput = <-d.current.output:
+		default:
+			return false
+		}
+	}
+	if debug {
+		println("got", len(d.current.b), "bytes, error:", d.current.err)
+	}
+	return true
+}
+
+// Close will release all resources.
+// It is NOT possible to reuse the decoder after this.
+func (d *Decoder) Close() {
+	if d.current.err == ErrDecoderClosed {
+		return
+	}
+	d.drainOutput()
+	if d.stream != nil {
+		close(d.stream)
+		d.streamWg.Wait()
+		d.stream = nil
+	}
+	if d.decoders != nil {
+		close(d.decoders)
+		for dec := range d.decoders {
+			dec.Close()
+		}
+		d.decoders = nil
+	}
+	if d.current.d != nil {
+		d.current.d.Close()
+		d.current.d = nil
+	}
+	d.current.err = ErrDecoderClosed
+}
+
+// IOReadCloser returns the decoder as an io.ReadCloser for convenience.
+// Any changes to the decoder will be reflected, so the returned ReadCloser
+// can be reused along with the decoder.
+// io.WriterTo is also supported by the returned ReadCloser.
+func (d *Decoder) IOReadCloser() io.ReadCloser {
+	return closeWrapper{d: d}
+}
+
+// closeWrapper wraps a function call as a closer.
+type closeWrapper struct {
+	d *Decoder
+}
+
+// WriteTo forwards WriteTo calls to the decoder.
+func (c closeWrapper) WriteTo(w io.Writer) (n int64, err error) {
+	return c.d.WriteTo(w)
+}
+
+// Read forwards read calls to the decoder.
+func (c closeWrapper) Read(p []byte) (n int, err error) {
+	return c.d.Read(p)
+}
+
+// Close closes the decoder.
+func (c closeWrapper) Close() error {
+	c.d.Close()
+	return nil
+}
+
+type decodeOutput struct {
+	d   *blockDec
+	b   []byte
+	err error
+}
+
+type decodeStream struct {
+	r io.Reader
+
+	// Blocks ready to be written to output.
+	output chan decodeOutput
+
+	// cancel reading from the input
+	cancel chan struct{}
+}
+
+// errEndOfStream indicates that everything from the stream was read.
+var errEndOfStream = errors.New("end-of-stream")
+
+// Create Decoder:
+// Spawn n block decoders. These accept tasks to decode a block.
+// Create goroutine that handles stream processing, this will send history to decoders as they are available.
+// Decoders update the history as they decode.
+// When a block is returned:
+// 		a) history is sent to the next decoder,
+// 		b) content written to CRC.
+// 		c) return data to WRITER.
+// 		d) wait for next block to return data.
+// Once WRITTEN, the decoders reused by the writer frame decoder for re-use.
+func (d *Decoder) startStreamDecoder(inStream chan decodeStream) {
+	defer d.streamWg.Done()
+	frame := newFrameDec(d.o)
+	for stream := range inStream {
+		if debug {
+			println("got new stream")
+		}
+		br := readerWrapper{r: stream.r}
+	decodeStream:
+		for {
+			frame.history.reset()
+			err := frame.reset(&br)
+			if debug && err != nil {
+				println("Frame decoder returned", err)
+			}
+			if err == nil && frame.DictionaryID != nil {
+				dict, ok := d.dicts[*frame.DictionaryID]
+				if !ok {
+					err = ErrUnknownDictionary
+				} else {
+					frame.history.setDict(&dict)
+				}
+			}
+			if err != nil {
+				stream.output <- decodeOutput{
+					err: err,
+				}
+				break
+			}
+			if debug {
+				println("starting frame decoder")
+			}
+
+			// This goroutine will forward history between frames.
+			frame.frameDone.Add(1)
+			frame.initAsync()
+
+			go frame.startDecoder(stream.output)
+		decodeFrame:
+			// Go through all blocks of the frame.
+			for {
+				dec := <-d.decoders
+				select {
+				case <-stream.cancel:
+					if !frame.sendErr(dec, io.EOF) {
+						// To not let the decoder dangle, send it back.
+						stream.output <- decodeOutput{d: dec}
+					}
+					break decodeStream
+				default:
+				}
+				err := frame.next(dec)
+				switch err {
+				case io.EOF:
+					// End of current frame, no error
+					println("EOF on next block")
+					break decodeFrame
+				case nil:
+					continue
+				default:
+					println("block decoder returned", err)
+					break decodeStream
+				}
+			}
+			// All blocks have started decoding, check if there are more frames.
+			println("waiting for done")
+			frame.frameDone.Wait()
+			println("done waiting...")
+		}
+		frame.frameDone.Wait()
+		println("Sending EOS")
+		stream.output <- decodeOutput{err: errEndOfStream}
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/decoder_options.go b/vendor/github.com/klauspost/compress/zstd/decoder_options.go
new file mode 100644
index 0000000..c0fd058
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/decoder_options.go
@@ -0,0 +1,83 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"runtime"
+)
+
+// DOption is an option for creating a decoder.
+type DOption func(*decoderOptions) error
+
+// options retains accumulated state of multiple options.
+type decoderOptions struct {
+	lowMem         bool
+	concurrent     int
+	maxDecodedSize uint64
+	dicts          []dict
+}
+
+func (o *decoderOptions) setDefault() {
+	*o = decoderOptions{
+		// use less ram: true for now, but may change.
+		lowMem:     true,
+		concurrent: runtime.GOMAXPROCS(0),
+	}
+	o.maxDecodedSize = 1 << 63
+}
+
+// WithDecoderLowmem will set whether to use a lower amount of memory,
+// but possibly have to allocate more while running.
+func WithDecoderLowmem(b bool) DOption {
+	return func(o *decoderOptions) error { o.lowMem = b; return nil }
+}
+
+// WithDecoderConcurrency will set the concurrency,
+// meaning the maximum number of decoders to run concurrently.
+// The value supplied must be at least 1.
+// By default this will be set to GOMAXPROCS.
+func WithDecoderConcurrency(n int) DOption {
+	return func(o *decoderOptions) error {
+		if n <= 0 {
+			return errors.New("concurrency must be at least 1")
+		}
+		o.concurrent = n
+		return nil
+	}
+}
+
+// WithDecoderMaxMemory allows to set a maximum decoded size for in-memory
+// non-streaming operations or maximum window size for streaming operations.
+// This can be used to control memory usage of potentially hostile content.
+// For streaming operations, the maximum window size is capped at 1<<30 bytes.
+// Maximum and default is 1 << 63 bytes.
+func WithDecoderMaxMemory(n uint64) DOption {
+	return func(o *decoderOptions) error {
+		if n == 0 {
+			return errors.New("WithDecoderMaxMemory must be at least 1")
+		}
+		if n > 1<<63 {
+			return errors.New("WithDecoderMaxmemory must be less than 1 << 63")
+		}
+		o.maxDecodedSize = n
+		return nil
+	}
+}
+
+// WithDecoderDicts allows to register one or more dictionaries for the decoder.
+// If several dictionaries with the same ID is provided the last one will be used.
+func WithDecoderDicts(dicts ...[]byte) DOption {
+	return func(o *decoderOptions) error {
+		for _, b := range dicts {
+			d, err := loadDict(b)
+			if err != nil {
+				return err
+			}
+			o.dicts = append(o.dicts, *d)
+		}
+		return nil
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/dict.go b/vendor/github.com/klauspost/compress/zstd/dict.go
new file mode 100644
index 0000000..fa25a18
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/dict.go
@@ -0,0 +1,122 @@
+package zstd
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"io"
+
+	"github.com/klauspost/compress/huff0"
+)
+
+type dict struct {
+	id uint32
+
+	litEnc              *huff0.Scratch
+	llDec, ofDec, mlDec sequenceDec
+	//llEnc, ofEnc, mlEnc []*fseEncoder
+	offsets [3]int
+	content []byte
+}
+
+var dictMagic = [4]byte{0x37, 0xa4, 0x30, 0xec}
+
+// ID returns the dictionary id or 0 if d is nil.
+func (d *dict) ID() uint32 {
+	if d == nil {
+		return 0
+	}
+	return d.id
+}
+
+// DictContentSize returns the dictionary content size or 0 if d is nil.
+func (d *dict) DictContentSize() int {
+	if d == nil {
+		return 0
+	}
+	return len(d.content)
+}
+
+// Load a dictionary as described in
+// https://github.com/facebook/zstd/blob/master/doc/zstd_compression_format.md#dictionary-format
+func loadDict(b []byte) (*dict, error) {
+	// Check static field size.
+	if len(b) <= 8+(3*4) {
+		return nil, io.ErrUnexpectedEOF
+	}
+	d := dict{
+		llDec: sequenceDec{fse: &fseDecoder{}},
+		ofDec: sequenceDec{fse: &fseDecoder{}},
+		mlDec: sequenceDec{fse: &fseDecoder{}},
+	}
+	if !bytes.Equal(b[:4], dictMagic[:]) {
+		return nil, ErrMagicMismatch
+	}
+	d.id = binary.LittleEndian.Uint32(b[4:8])
+	if d.id == 0 {
+		return nil, errors.New("dictionaries cannot have ID 0")
+	}
+
+	// Read literal table
+	var err error
+	d.litEnc, b, err = huff0.ReadTable(b[8:], nil)
+	if err != nil {
+		return nil, err
+	}
+	d.litEnc.Reuse = huff0.ReusePolicyMust
+
+	br := byteReader{
+		b:   b,
+		off: 0,
+	}
+	readDec := func(i tableIndex, dec *fseDecoder) error {
+		if err := dec.readNCount(&br, uint16(maxTableSymbol[i])); err != nil {
+			return err
+		}
+		if br.overread() {
+			return io.ErrUnexpectedEOF
+		}
+		err = dec.transform(symbolTableX[i])
+		if err != nil {
+			println("Transform table error:", err)
+			return err
+		}
+		if debug {
+			println("Read table ok", "symbolLen:", dec.symbolLen)
+		}
+		// Set decoders as predefined so they aren't reused.
+		dec.preDefined = true
+		return nil
+	}
+
+	if err := readDec(tableOffsets, d.ofDec.fse); err != nil {
+		return nil, err
+	}
+	if err := readDec(tableMatchLengths, d.mlDec.fse); err != nil {
+		return nil, err
+	}
+	if err := readDec(tableLiteralLengths, d.llDec.fse); err != nil {
+		return nil, err
+	}
+	if br.remain() < 12 {
+		return nil, io.ErrUnexpectedEOF
+	}
+
+	d.offsets[0] = int(br.Uint32())
+	br.advance(4)
+	d.offsets[1] = int(br.Uint32())
+	br.advance(4)
+	d.offsets[2] = int(br.Uint32())
+	br.advance(4)
+	if d.offsets[0] <= 0 || d.offsets[1] <= 0 || d.offsets[2] <= 0 {
+		return nil, errors.New("invalid offset in dictionary")
+	}
+	d.content = make([]byte, br.remain())
+	copy(d.content, br.unread())
+	if d.offsets[0] > len(d.content) || d.offsets[1] > len(d.content) || d.offsets[2] > len(d.content) {
+		return nil, fmt.Errorf("initial offset bigger than dictionary content size %d, offsets: %v", len(d.content), d.offsets)
+	}
+
+	return &d, nil
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/enc_base.go b/vendor/github.com/klauspost/compress/zstd/enc_base.go
new file mode 100644
index 0000000..60f2986
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/enc_base.go
@@ -0,0 +1,178 @@
+package zstd
+
+import (
+	"fmt"
+	"math/bits"
+
+	"github.com/klauspost/compress/zstd/internal/xxhash"
+)
+
+const (
+	dictShardBits = 6
+)
+
+type fastBase struct {
+	// cur is the offset at the start of hist
+	cur int32
+	// maximum offset. Should be at least 2x block size.
+	maxMatchOff int32
+	hist        []byte
+	crc         *xxhash.Digest
+	tmp         [8]byte
+	blk         *blockEnc
+	lastDictID  uint32
+	lowMem      bool
+}
+
+// CRC returns the underlying CRC writer.
+func (e *fastBase) CRC() *xxhash.Digest {
+	return e.crc
+}
+
+// AppendCRC will append the CRC to the destination slice and return it.
+func (e *fastBase) AppendCRC(dst []byte) []byte {
+	crc := e.crc.Sum(e.tmp[:0])
+	dst = append(dst, crc[7], crc[6], crc[5], crc[4])
+	return dst
+}
+
+// WindowSize returns the window size of the encoder,
+// or a window size small enough to contain the input size, if > 0.
+func (e *fastBase) WindowSize(size int) int32 {
+	if size > 0 && size < int(e.maxMatchOff) {
+		b := int32(1) << uint(bits.Len(uint(size)))
+		// Keep minimum window.
+		if b < 1024 {
+			b = 1024
+		}
+		return b
+	}
+	return e.maxMatchOff
+}
+
+// Block returns the current block.
+func (e *fastBase) Block() *blockEnc {
+	return e.blk
+}
+
+func (e *fastBase) addBlock(src []byte) int32 {
+	if debugAsserts && e.cur > bufferReset {
+		panic(fmt.Sprintf("ecur (%d) > buffer reset (%d)", e.cur, bufferReset))
+	}
+	// check if we have space already
+	if len(e.hist)+len(src) > cap(e.hist) {
+		if cap(e.hist) == 0 {
+			e.ensureHist(len(src))
+		} else {
+			if cap(e.hist) < int(e.maxMatchOff+maxCompressedBlockSize) {
+				panic(fmt.Errorf("unexpected buffer cap %d, want at least %d with window %d", cap(e.hist), e.maxMatchOff+maxCompressedBlockSize, e.maxMatchOff))
+			}
+			// Move down
+			offset := int32(len(e.hist)) - e.maxMatchOff
+			copy(e.hist[0:e.maxMatchOff], e.hist[offset:])
+			e.cur += offset
+			e.hist = e.hist[:e.maxMatchOff]
+		}
+	}
+	s := int32(len(e.hist))
+	e.hist = append(e.hist, src...)
+	return s
+}
+
+// ensureHist will ensure that history can keep at least this many bytes.
+func (e *fastBase) ensureHist(n int) {
+	if cap(e.hist) >= n {
+		return
+	}
+	l := e.maxMatchOff
+	if (e.lowMem && e.maxMatchOff > maxCompressedBlockSize) || e.maxMatchOff <= maxCompressedBlockSize {
+		l += maxCompressedBlockSize
+	} else {
+		l += e.maxMatchOff
+	}
+	// Make it at least 1MB.
+	if l < 1<<20 && !e.lowMem {
+		l = 1 << 20
+	}
+	// Make it at least the requested size.
+	if l < int32(n) {
+		l = int32(n)
+	}
+	e.hist = make([]byte, 0, l)
+}
+
+// useBlock will replace the block with the provided one,
+// but transfer recent offsets from the previous.
+func (e *fastBase) UseBlock(enc *blockEnc) {
+	enc.reset(e.blk)
+	e.blk = enc
+}
+
+func (e *fastBase) matchlenNoHist(s, t int32, src []byte) int32 {
+	// Extend the match to be as long as possible.
+	return int32(matchLen(src[s:], src[t:]))
+}
+
+func (e *fastBase) matchlen(s, t int32, src []byte) int32 {
+	if debugAsserts {
+		if s < 0 {
+			err := fmt.Sprintf("s (%d) < 0", s)
+			panic(err)
+		}
+		if t < 0 {
+			err := fmt.Sprintf("s (%d) < 0", s)
+			panic(err)
+		}
+		if s-t > e.maxMatchOff {
+			err := fmt.Sprintf("s (%d) - t (%d) > maxMatchOff (%d)", s, t, e.maxMatchOff)
+			panic(err)
+		}
+		if len(src)-int(s) > maxCompressedBlockSize {
+			panic(fmt.Sprintf("len(src)-s (%d) > maxCompressedBlockSize (%d)", len(src)-int(s), maxCompressedBlockSize))
+		}
+	}
+
+	// Extend the match to be as long as possible.
+	return int32(matchLen(src[s:], src[t:]))
+}
+
+// Reset the encoding table.
+func (e *fastBase) resetBase(d *dict, singleBlock bool) {
+	if e.blk == nil {
+		e.blk = &blockEnc{lowMem: e.lowMem}
+		e.blk.init()
+	} else {
+		e.blk.reset(nil)
+	}
+	e.blk.initNewEncode()
+	if e.crc == nil {
+		e.crc = xxhash.New()
+	} else {
+		e.crc.Reset()
+	}
+	if d != nil {
+		low := e.lowMem
+		if singleBlock {
+			e.lowMem = true
+		}
+		e.ensureHist(d.DictContentSize() + maxCompressedBlockSize)
+		e.lowMem = low
+	}
+
+	// We offset current position so everything will be out of reach.
+	// If above reset line, history will be purged.
+	if e.cur < bufferReset {
+		e.cur += e.maxMatchOff + int32(len(e.hist))
+	}
+	e.hist = e.hist[:0]
+	if d != nil {
+		// Set offsets (currently not used)
+		for i, off := range d.offsets {
+			e.blk.recentOffsets[i] = uint32(off)
+			e.blk.prevRecentOffsets[i] = e.blk.recentOffsets[i]
+		}
+		// Transfer litenc.
+		e.blk.dictLitEnc = d.litEnc
+		e.hist = append(e.hist, d.content...)
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/enc_best.go b/vendor/github.com/klauspost/compress/zstd/enc_best.go
new file mode 100644
index 0000000..dc1eed5
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/enc_best.go
@@ -0,0 +1,501 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"fmt"
+	"math/bits"
+)
+
+const (
+	bestLongTableBits = 20                     // Bits used in the long match table
+	bestLongTableSize = 1 << bestLongTableBits // Size of the table
+
+	// Note: Increasing the short table bits or making the hash shorter
+	// can actually lead to compression degradation since it will 'steal' more from the
+	// long match table and match offsets are quite big.
+	// This greatly depends on the type of input.
+	bestShortTableBits = 16                      // Bits used in the short match table
+	bestShortTableSize = 1 << bestShortTableBits // Size of the table
+)
+
+// bestFastEncoder uses 2 tables, one for short matches (5 bytes) and one for long matches.
+// The long match table contains the previous entry with the same hash,
+// effectively making it a "chain" of length 2.
+// When we find a long match we choose between the two values and select the longest.
+// When we find a short match, after checking the long, we check if we can find a long at n+1
+// and that it is longer (lazy matching).
+type bestFastEncoder struct {
+	fastBase
+	table         [bestShortTableSize]prevEntry
+	longTable     [bestLongTableSize]prevEntry
+	dictTable     []prevEntry
+	dictLongTable []prevEntry
+}
+
+// Encode improves compression...
+func (e *bestFastEncoder) Encode(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 4
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = prevEntry{}
+			}
+			for i := range e.longTable[:] {
+				e.longTable[i] = prevEntry{}
+			}
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			v2 := e.table[i].prev
+			if v < minOff {
+				v = 0
+				v2 = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+				if v2 < minOff {
+					v2 = 0
+				} else {
+					v2 = v2 - e.cur + e.maxMatchOff
+				}
+			}
+			e.table[i] = prevEntry{
+				offset: v,
+				prev:   v2,
+			}
+		}
+		for i := range e.longTable[:] {
+			v := e.longTable[i].offset
+			v2 := e.longTable[i].prev
+			if v < minOff {
+				v = 0
+				v2 = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+				if v2 < minOff {
+					v2 = 0
+				} else {
+					v2 = v2 - e.cur + e.maxMatchOff
+				}
+			}
+			e.longTable[i] = prevEntry{
+				offset: v,
+				prev:   v2,
+			}
+		}
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	const kSearchStrength = 10
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+	offset3 := int32(blk.recentOffsets[2])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	_ = addLiterals
+
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		// We allow the encoder to optionally turn off repeat offsets across blocks
+		canRepeat := len(blk.sequences) > 2
+
+		if debugAsserts && canRepeat && offset1 == 0 {
+			panic("offset0 was 0")
+		}
+
+		type match struct {
+			offset int32
+			s      int32
+			length int32
+			rep    int32
+		}
+		matchAt := func(offset int32, s int32, first uint32, rep int32) match {
+			if s-offset >= e.maxMatchOff || load3232(src, offset) != first {
+				return match{offset: offset, s: s}
+			}
+			return match{offset: offset, s: s, length: 4 + e.matchlen(s+4, offset+4, src), rep: rep}
+		}
+
+		bestOf := func(a, b match) match {
+			aScore := b.s - a.s + a.length
+			bScore := a.s - b.s + b.length
+			if a.rep < 0 {
+				aScore = aScore - int32(bits.Len32(uint32(a.offset)))/8
+			}
+			if b.rep < 0 {
+				bScore = bScore - int32(bits.Len32(uint32(b.offset)))/8
+			}
+			if aScore >= bScore {
+				return a
+			}
+			return b
+		}
+		const goodEnough = 100
+
+		nextHashL := hash8(cv, bestLongTableBits)
+		nextHashS := hash4x64(cv, bestShortTableBits)
+		candidateL := e.longTable[nextHashL]
+		candidateS := e.table[nextHashS]
+
+		best := bestOf(matchAt(candidateL.offset-e.cur, s, uint32(cv), -1), matchAt(candidateL.prev-e.cur, s, uint32(cv), -1))
+		best = bestOf(best, matchAt(candidateS.offset-e.cur, s, uint32(cv), -1))
+		best = bestOf(best, matchAt(candidateS.prev-e.cur, s, uint32(cv), -1))
+		if canRepeat && best.length < goodEnough {
+			best = bestOf(best, matchAt(s-offset1+1, s+1, uint32(cv>>8), 1))
+			best = bestOf(best, matchAt(s-offset2+1, s+1, uint32(cv>>8), 2))
+			best = bestOf(best, matchAt(s-offset3+1, s+1, uint32(cv>>8), 3))
+			if best.length > 0 {
+				best = bestOf(best, matchAt(s-offset1+3, s+3, uint32(cv>>24), 1))
+				best = bestOf(best, matchAt(s-offset2+3, s+3, uint32(cv>>24), 2))
+				best = bestOf(best, matchAt(s-offset3+3, s+3, uint32(cv>>24), 3))
+			}
+		}
+		// Load next and check...
+		e.longTable[nextHashL] = prevEntry{offset: s + e.cur, prev: candidateL.offset}
+		e.table[nextHashS] = prevEntry{offset: s + e.cur, prev: candidateS.offset}
+
+		// Look far ahead, unless we have a really long match already...
+		if best.length < goodEnough {
+			// No match found, move forward on input, no need to check forward...
+			if best.length < 4 {
+				s += 1 + (s-nextEmit)>>(kSearchStrength-1)
+				if s >= sLimit {
+					break encodeLoop
+				}
+				cv = load6432(src, s)
+				continue
+			}
+
+			s++
+			candidateS = e.table[hash4x64(cv>>8, bestShortTableBits)]
+			cv = load6432(src, s)
+			cv2 := load6432(src, s+1)
+			candidateL = e.longTable[hash8(cv, bestLongTableBits)]
+			candidateL2 := e.longTable[hash8(cv2, bestLongTableBits)]
+
+			best = bestOf(best, matchAt(candidateS.offset-e.cur, s, uint32(cv), -1))
+			best = bestOf(best, matchAt(candidateL.offset-e.cur, s, uint32(cv), -1))
+			best = bestOf(best, matchAt(candidateL.prev-e.cur, s, uint32(cv), -1))
+			best = bestOf(best, matchAt(candidateL2.offset-e.cur, s+1, uint32(cv2), -1))
+			best = bestOf(best, matchAt(candidateL2.prev-e.cur, s+1, uint32(cv2), -1))
+
+			// See if we can find a better match by checking where the current best ends.
+			// Use that offset to see if we can find a better full match.
+			if sAt := best.s + best.length; sAt < sLimit {
+				nextHashL := hash8(load6432(src, sAt), bestLongTableBits)
+				candidateEnd := e.longTable[nextHashL]
+				if pos := candidateEnd.offset - e.cur - best.length; pos >= 0 {
+					bestEnd := bestOf(best, matchAt(pos, best.s, load3232(src, best.s), -1))
+					if pos := candidateEnd.prev - e.cur - best.length; pos >= 0 {
+						bestEnd = bestOf(bestEnd, matchAt(pos, best.s, load3232(src, best.s), -1))
+					}
+					best = bestEnd
+				}
+			}
+		}
+
+		// We have a match, we can store the forward value
+		if best.rep > 0 {
+			s = best.s
+			var seq seq
+			seq.matchLen = uint32(best.length - zstdMinMatch)
+
+			// We might be able to match backwards.
+			// Extend as long as we can.
+			start := best.s
+			// We end the search early, so we don't risk 0 literals
+			// and have to do special offset treatment.
+			startLimit := nextEmit + 1
+
+			tMin := s - e.maxMatchOff
+			if tMin < 0 {
+				tMin = 0
+			}
+			repIndex := best.offset
+			for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+				repIndex--
+				start--
+				seq.matchLen++
+			}
+			addLiterals(&seq, start)
+
+			// rep 0
+			seq.offset = uint32(best.rep)
+			if debugSequences {
+				println("repeat sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Index match start+1 (long) -> s - 1
+			index0 := s
+			s = best.s + best.length
+
+			nextEmit = s
+			if s >= sLimit {
+				if debug {
+					println("repeat ended", s, best.length)
+
+				}
+				break encodeLoop
+			}
+			// Index skipped...
+			off := index0 + e.cur
+			for index0 < s-1 {
+				cv0 := load6432(src, index0)
+				h0 := hash8(cv0, bestLongTableBits)
+				h1 := hash4x64(cv0, bestShortTableBits)
+				e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+				e.table[h1] = prevEntry{offset: off, prev: e.table[h1].offset}
+				off++
+				index0++
+			}
+			switch best.rep {
+			case 2:
+				offset1, offset2 = offset2, offset1
+			case 3:
+				offset1, offset2, offset3 = offset3, offset1, offset2
+			}
+			cv = load6432(src, s)
+			continue
+		}
+
+		// A 4-byte match has been found. Update recent offsets.
+		// We'll later see if more than 4 bytes.
+		s = best.s
+		t := best.offset
+		offset1, offset2, offset3 = s-t, offset1, offset2
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the n-byte match as long as possible.
+		l := best.length
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) -> s - 1
+		index0 := s - l + 1
+		// every entry
+		for index0 < s-1 {
+			cv0 := load6432(src, index0)
+			h0 := hash8(cv0, bestLongTableBits)
+			h1 := hash4x64(cv0, bestShortTableBits)
+			off := index0 + e.cur
+			e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+			e.table[h1] = prevEntry{offset: off, prev: e.table[h1].offset}
+			index0++
+		}
+
+		cv = load6432(src, s)
+		if !canRepeat {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash4x64(cv, bestShortTableBits)
+			nextHashL := hash8(cv, bestLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			l := 4 + e.matchlen(s+4, o2+4, src)
+
+			e.longTable[nextHashL] = prevEntry{offset: s + e.cur, prev: e.longTable[nextHashL].offset}
+			e.table[nextHashS] = prevEntry{offset: s + e.cur, prev: e.table[nextHashS].offset}
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	blk.recentOffsets[2] = uint32(offset3)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// EncodeNoHist will encode a block with no history and no following blocks.
+// Most notable difference is that src will not be copied for history and
+// we do not need to check for max match length.
+func (e *bestFastEncoder) EncodeNoHist(blk *blockEnc, src []byte) {
+	e.ensureHist(len(src))
+	e.Encode(blk, src)
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *bestFastEncoder) Reset(d *dict, singleBlock bool) {
+	e.resetBase(d, singleBlock)
+	if d == nil {
+		return
+	}
+	// Init or copy dict table
+	if len(e.dictTable) != len(e.table) || d.id != e.lastDictID {
+		if len(e.dictTable) != len(e.table) {
+			e.dictTable = make([]prevEntry, len(e.table))
+		}
+		end := int32(len(d.content)) - 8 + e.maxMatchOff
+		for i := e.maxMatchOff; i < end; i += 4 {
+			const hashLog = bestShortTableBits
+
+			cv := load6432(d.content, i-e.maxMatchOff)
+			nextHash := hash4x64(cv, hashLog)      // 0 -> 4
+			nextHash1 := hash4x64(cv>>8, hashLog)  // 1 -> 5
+			nextHash2 := hash4x64(cv>>16, hashLog) // 2 -> 6
+			nextHash3 := hash4x64(cv>>24, hashLog) // 3 -> 7
+			e.dictTable[nextHash] = prevEntry{
+				prev:   e.dictTable[nextHash].offset,
+				offset: i,
+			}
+			e.dictTable[nextHash1] = prevEntry{
+				prev:   e.dictTable[nextHash1].offset,
+				offset: i + 1,
+			}
+			e.dictTable[nextHash2] = prevEntry{
+				prev:   e.dictTable[nextHash2].offset,
+				offset: i + 2,
+			}
+			e.dictTable[nextHash3] = prevEntry{
+				prev:   e.dictTable[nextHash3].offset,
+				offset: i + 3,
+			}
+		}
+		e.lastDictID = d.id
+	}
+
+	// Init or copy dict table
+	if len(e.dictLongTable) != len(e.longTable) || d.id != e.lastDictID {
+		if len(e.dictLongTable) != len(e.longTable) {
+			e.dictLongTable = make([]prevEntry, len(e.longTable))
+		}
+		if len(d.content) >= 8 {
+			cv := load6432(d.content, 0)
+			h := hash8(cv, bestLongTableBits)
+			e.dictLongTable[h] = prevEntry{
+				offset: e.maxMatchOff,
+				prev:   e.dictLongTable[h].offset,
+			}
+
+			end := int32(len(d.content)) - 8 + e.maxMatchOff
+			off := 8 // First to read
+			for i := e.maxMatchOff + 1; i < end; i++ {
+				cv = cv>>8 | (uint64(d.content[off]) << 56)
+				h := hash8(cv, bestLongTableBits)
+				e.dictLongTable[h] = prevEntry{
+					offset: i,
+					prev:   e.dictLongTable[h].offset,
+				}
+				off++
+			}
+		}
+		e.lastDictID = d.id
+	}
+	// Reset table to initial state
+	copy(e.longTable[:], e.dictLongTable)
+
+	e.cur = e.maxMatchOff
+	// Reset table to initial state
+	copy(e.table[:], e.dictTable)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/enc_better.go b/vendor/github.com/klauspost/compress/zstd/enc_better.go
new file mode 100644
index 0000000..6049542
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/enc_better.go
@@ -0,0 +1,1235 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import "fmt"
+
+const (
+	betterLongTableBits = 19                       // Bits used in the long match table
+	betterLongTableSize = 1 << betterLongTableBits // Size of the table
+
+	// Note: Increasing the short table bits or making the hash shorter
+	// can actually lead to compression degradation since it will 'steal' more from the
+	// long match table and match offsets are quite big.
+	// This greatly depends on the type of input.
+	betterShortTableBits = 13                        // Bits used in the short match table
+	betterShortTableSize = 1 << betterShortTableBits // Size of the table
+
+	betterLongTableShardCnt  = 1 << (betterLongTableBits - dictShardBits)    // Number of shards in the table
+	betterLongTableShardSize = betterLongTableSize / betterLongTableShardCnt // Size of an individual shard
+
+	betterShortTableShardCnt  = 1 << (betterShortTableBits - dictShardBits)     // Number of shards in the table
+	betterShortTableShardSize = betterShortTableSize / betterShortTableShardCnt // Size of an individual shard
+)
+
+type prevEntry struct {
+	offset int32
+	prev   int32
+}
+
+// betterFastEncoder uses 2 tables, one for short matches (5 bytes) and one for long matches.
+// The long match table contains the previous entry with the same hash,
+// effectively making it a "chain" of length 2.
+// When we find a long match we choose between the two values and select the longest.
+// When we find a short match, after checking the long, we check if we can find a long at n+1
+// and that it is longer (lazy matching).
+type betterFastEncoder struct {
+	fastBase
+	table     [betterShortTableSize]tableEntry
+	longTable [betterLongTableSize]prevEntry
+}
+
+type betterFastEncoderDict struct {
+	betterFastEncoder
+	dictTable            []tableEntry
+	dictLongTable        []prevEntry
+	shortTableShardDirty [betterShortTableShardCnt]bool
+	longTableShardDirty  [betterLongTableShardCnt]bool
+	allDirty             bool
+}
+
+// Encode improves compression...
+func (e *betterFastEncoder) Encode(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 2
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			for i := range e.longTable[:] {
+				e.longTable[i] = prevEntry{}
+			}
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		for i := range e.longTable[:] {
+			v := e.longTable[i].offset
+			v2 := e.longTable[i].prev
+			if v < minOff {
+				v = 0
+				v2 = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+				if v2 < minOff {
+					v2 = 0
+				} else {
+					v2 = v2 - e.cur + e.maxMatchOff
+				}
+			}
+			e.longTable[i] = prevEntry{
+				offset: v,
+				prev:   v2,
+			}
+		}
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 1.
+	const stepSize = 1
+
+	const kSearchStrength = 9
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		var t int32
+		// We allow the encoder to optionally turn off repeat offsets across blocks
+		canRepeat := len(blk.sequences) > 2
+		var matched int32
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHashS := hash5(cv, betterShortTableBits)
+			nextHashL := hash8(cv, betterLongTableBits)
+			candidateL := e.longTable[nextHashL]
+			candidateS := e.table[nextHashS]
+
+			const repOff = 1
+			repIndex := s - offset1 + repOff
+			off := s + e.cur
+			e.longTable[nextHashL] = prevEntry{offset: off, prev: candidateL.offset}
+			e.table[nextHashS] = tableEntry{offset: off, val: uint32(cv)}
+
+			if canRepeat {
+				if repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>(repOff*8)) {
+					// Consider history as well.
+					var seq seq
+					lenght := 4 + e.matchlen(s+4+repOff, repIndex+4, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 0
+					seq.offset = 1
+					if debugSequences {
+						println("repeat sequence", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+
+					// Index match start+1 (long) -> s - 1
+					index0 := s + repOff
+					s += lenght + repOff
+
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+					// Index skipped...
+					for index0 < s-1 {
+						cv0 := load6432(src, index0)
+						cv1 := cv0 >> 8
+						h0 := hash8(cv0, betterLongTableBits)
+						off := index0 + e.cur
+						e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+						e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)}
+						index0 += 2
+					}
+					cv = load6432(src, s)
+					continue
+				}
+				const repOff2 = 1
+
+				// We deviate from the reference encoder and also check offset 2.
+				// Still slower and not much better, so disabled.
+				// repIndex = s - offset2 + repOff2
+				if false && repIndex >= 0 && load6432(src, repIndex) == load6432(src, s+repOff) {
+					// Consider history as well.
+					var seq seq
+					lenght := 8 + e.matchlen(s+8+repOff2, repIndex+8, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff2
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 2
+					seq.offset = 2
+					if debugSequences {
+						println("repeat sequence 2", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+
+					index0 := s + repOff2
+					s += lenght + repOff2
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+
+					// Index skipped...
+					for index0 < s-1 {
+						cv0 := load6432(src, index0)
+						cv1 := cv0 >> 8
+						h0 := hash8(cv0, betterLongTableBits)
+						off := index0 + e.cur
+						e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+						e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)}
+						index0 += 2
+					}
+					cv = load6432(src, s)
+					// Swap offsets
+					offset1, offset2 = offset2, offset1
+					continue
+				}
+			}
+			// Find the offsets of our two matches.
+			coffsetL := candidateL.offset - e.cur
+			coffsetLP := candidateL.prev - e.cur
+
+			// Check if we have a long match.
+			if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+				// Found a long match, at least 8 bytes.
+				matched = e.matchlen(s+8, coffsetL+8, src) + 8
+				t = coffsetL
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+
+				if s-coffsetLP < e.maxMatchOff && cv == load6432(src, coffsetLP) {
+					// Found a long match, at least 8 bytes.
+					prevMatch := e.matchlen(s+8, coffsetLP+8, src) + 8
+					if prevMatch > matched {
+						matched = prevMatch
+						t = coffsetLP
+					}
+					if debugAsserts && s <= t {
+						panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+					}
+					if debugAsserts && s-t > e.maxMatchOff {
+						panic("s - t >e.maxMatchOff")
+					}
+					if debugMatches {
+						println("long match")
+					}
+				}
+				break
+			}
+
+			// Check if we have a long match on prev.
+			if s-coffsetLP < e.maxMatchOff && cv == load6432(src, coffsetLP) {
+				// Found a long match, at least 8 bytes.
+				matched = e.matchlen(s+8, coffsetLP+8, src) + 8
+				t = coffsetLP
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+				break
+			}
+
+			coffsetS := candidateS.offset - e.cur
+
+			// Check if we have a short match.
+			if s-coffsetS < e.maxMatchOff && uint32(cv) == candidateS.val {
+				// found a regular match
+				matched = e.matchlen(s+4, coffsetS+4, src) + 4
+
+				// See if we can find a long match at s+1
+				const checkAt = 1
+				cv := load6432(src, s+checkAt)
+				nextHashL = hash8(cv, betterLongTableBits)
+				candidateL = e.longTable[nextHashL]
+				coffsetL = candidateL.offset - e.cur
+
+				// We can store it, since we have at least a 4 byte match.
+				e.longTable[nextHashL] = prevEntry{offset: s + checkAt + e.cur, prev: candidateL.offset}
+				if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+					// Found a long match, at least 8 bytes.
+					matchedNext := e.matchlen(s+8+checkAt, coffsetL+8, src) + 8
+					if matchedNext > matched {
+						t = coffsetL
+						s += checkAt
+						matched = matchedNext
+						if debugMatches {
+							println("long match (after short)")
+						}
+						break
+					}
+				}
+
+				// Check prev long...
+				coffsetL = candidateL.prev - e.cur
+				if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+					// Found a long match, at least 8 bytes.
+					matchedNext := e.matchlen(s+8+checkAt, coffsetL+8, src) + 8
+					if matchedNext > matched {
+						t = coffsetL
+						s += checkAt
+						matched = matchedNext
+						if debugMatches {
+							println("prev long match (after short)")
+						}
+						break
+					}
+				}
+				t = coffsetS
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				if debugMatches {
+					println("short match")
+				}
+				break
+			}
+
+			// No match found, move forward in input.
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+
+		// Try to find a better match by searching for a long match at the end of the current best match
+		if true && s+matched < sLimit {
+			nextHashL := hash8(load6432(src, s+matched), betterLongTableBits)
+			cv := load3232(src, s)
+			candidateL := e.longTable[nextHashL]
+			coffsetL := candidateL.offset - e.cur - matched
+			if coffsetL >= 0 && coffsetL < s && s-coffsetL < e.maxMatchOff && cv == load3232(src, coffsetL) {
+				// Found a long match, at least 4 bytes.
+				matchedNext := e.matchlen(s+4, coffsetL+4, src) + 4
+				if matchedNext > matched {
+					t = coffsetL
+					matched = matchedNext
+					if debugMatches {
+						println("long match at end-of-match")
+					}
+				}
+			}
+
+			// Check prev long...
+			if true {
+				coffsetL = candidateL.prev - e.cur - matched
+				if coffsetL >= 0 && coffsetL < s && s-coffsetL < e.maxMatchOff && cv == load3232(src, coffsetL) {
+					// Found a long match, at least 4 bytes.
+					matchedNext := e.matchlen(s+4, coffsetL+4, src) + 4
+					if matchedNext > matched {
+						t = coffsetL
+						matched = matchedNext
+						if debugMatches {
+							println("prev long match at end-of-match")
+						}
+					}
+				}
+			}
+		}
+		// A match has been found. Update recent offsets.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the n-byte match as long as possible.
+		l := matched
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) -> s - 1
+		index0 := s - l + 1
+		for index0 < s-1 {
+			cv0 := load6432(src, index0)
+			cv1 := cv0 >> 8
+			h0 := hash8(cv0, betterLongTableBits)
+			off := index0 + e.cur
+			e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+			e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)}
+			index0 += 2
+		}
+
+		cv = load6432(src, s)
+		if !canRepeat {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash5(cv, betterShortTableBits)
+			nextHashL := hash8(cv, betterLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			l := 4 + e.matchlen(s+4, o2+4, src)
+
+			e.longTable[nextHashL] = prevEntry{offset: s + e.cur, prev: e.longTable[nextHashL].offset}
+			e.table[nextHashS] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// EncodeNoHist will encode a block with no history and no following blocks.
+// Most notable difference is that src will not be copied for history and
+// we do not need to check for max match length.
+func (e *betterFastEncoder) EncodeNoHist(blk *blockEnc, src []byte) {
+	e.ensureHist(len(src))
+	e.Encode(blk, src)
+}
+
+// Encode improves compression...
+func (e *betterFastEncoderDict) Encode(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 2
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			for i := range e.longTable[:] {
+				e.longTable[i] = prevEntry{}
+			}
+			e.cur = e.maxMatchOff
+			e.allDirty = true
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		for i := range e.longTable[:] {
+			v := e.longTable[i].offset
+			v2 := e.longTable[i].prev
+			if v < minOff {
+				v = 0
+				v2 = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+				if v2 < minOff {
+					v2 = 0
+				} else {
+					v2 = v2 - e.cur + e.maxMatchOff
+				}
+			}
+			e.longTable[i] = prevEntry{
+				offset: v,
+				prev:   v2,
+			}
+		}
+		e.allDirty = true
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 1.
+	const stepSize = 1
+
+	const kSearchStrength = 9
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		var t int32
+		// We allow the encoder to optionally turn off repeat offsets across blocks
+		canRepeat := len(blk.sequences) > 2
+		var matched int32
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHashS := hash5(cv, betterShortTableBits)
+			nextHashL := hash8(cv, betterLongTableBits)
+			candidateL := e.longTable[nextHashL]
+			candidateS := e.table[nextHashS]
+
+			const repOff = 1
+			repIndex := s - offset1 + repOff
+			off := s + e.cur
+			e.longTable[nextHashL] = prevEntry{offset: off, prev: candidateL.offset}
+			e.markLongShardDirty(nextHashL)
+			e.table[nextHashS] = tableEntry{offset: off, val: uint32(cv)}
+			e.markShortShardDirty(nextHashS)
+
+			if canRepeat {
+				if repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>(repOff*8)) {
+					// Consider history as well.
+					var seq seq
+					lenght := 4 + e.matchlen(s+4+repOff, repIndex+4, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 0
+					seq.offset = 1
+					if debugSequences {
+						println("repeat sequence", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+
+					// Index match start+1 (long) -> s - 1
+					index0 := s + repOff
+					s += lenght + repOff
+
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+					// Index skipped...
+					for index0 < s-1 {
+						cv0 := load6432(src, index0)
+						cv1 := cv0 >> 8
+						h0 := hash8(cv0, betterLongTableBits)
+						off := index0 + e.cur
+						e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+						e.markLongShardDirty(h0)
+						h1 := hash5(cv1, betterShortTableBits)
+						e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)}
+						e.markShortShardDirty(h1)
+						index0 += 2
+					}
+					cv = load6432(src, s)
+					continue
+				}
+				const repOff2 = 1
+
+				// We deviate from the reference encoder and also check offset 2.
+				// Still slower and not much better, so disabled.
+				// repIndex = s - offset2 + repOff2
+				if false && repIndex >= 0 && load6432(src, repIndex) == load6432(src, s+repOff) {
+					// Consider history as well.
+					var seq seq
+					lenght := 8 + e.matchlen(s+8+repOff2, repIndex+8, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff2
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 2
+					seq.offset = 2
+					if debugSequences {
+						println("repeat sequence 2", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+
+					index0 := s + repOff2
+					s += lenght + repOff2
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+
+					// Index skipped...
+					for index0 < s-1 {
+						cv0 := load6432(src, index0)
+						cv1 := cv0 >> 8
+						h0 := hash8(cv0, betterLongTableBits)
+						off := index0 + e.cur
+						e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+						e.markLongShardDirty(h0)
+						h1 := hash5(cv1, betterShortTableBits)
+						e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)}
+						e.markShortShardDirty(h1)
+						index0 += 2
+					}
+					cv = load6432(src, s)
+					// Swap offsets
+					offset1, offset2 = offset2, offset1
+					continue
+				}
+			}
+			// Find the offsets of our two matches.
+			coffsetL := candidateL.offset - e.cur
+			coffsetLP := candidateL.prev - e.cur
+
+			// Check if we have a long match.
+			if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+				// Found a long match, at least 8 bytes.
+				matched = e.matchlen(s+8, coffsetL+8, src) + 8
+				t = coffsetL
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+
+				if s-coffsetLP < e.maxMatchOff && cv == load6432(src, coffsetLP) {
+					// Found a long match, at least 8 bytes.
+					prevMatch := e.matchlen(s+8, coffsetLP+8, src) + 8
+					if prevMatch > matched {
+						matched = prevMatch
+						t = coffsetLP
+					}
+					if debugAsserts && s <= t {
+						panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+					}
+					if debugAsserts && s-t > e.maxMatchOff {
+						panic("s - t >e.maxMatchOff")
+					}
+					if debugMatches {
+						println("long match")
+					}
+				}
+				break
+			}
+
+			// Check if we have a long match on prev.
+			if s-coffsetLP < e.maxMatchOff && cv == load6432(src, coffsetLP) {
+				// Found a long match, at least 8 bytes.
+				matched = e.matchlen(s+8, coffsetLP+8, src) + 8
+				t = coffsetLP
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+				break
+			}
+
+			coffsetS := candidateS.offset - e.cur
+
+			// Check if we have a short match.
+			if s-coffsetS < e.maxMatchOff && uint32(cv) == candidateS.val {
+				// found a regular match
+				matched = e.matchlen(s+4, coffsetS+4, src) + 4
+
+				// See if we can find a long match at s+1
+				const checkAt = 1
+				cv := load6432(src, s+checkAt)
+				nextHashL = hash8(cv, betterLongTableBits)
+				candidateL = e.longTable[nextHashL]
+				coffsetL = candidateL.offset - e.cur
+
+				// We can store it, since we have at least a 4 byte match.
+				e.longTable[nextHashL] = prevEntry{offset: s + checkAt + e.cur, prev: candidateL.offset}
+				e.markLongShardDirty(nextHashL)
+				if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+					// Found a long match, at least 8 bytes.
+					matchedNext := e.matchlen(s+8+checkAt, coffsetL+8, src) + 8
+					if matchedNext > matched {
+						t = coffsetL
+						s += checkAt
+						matched = matchedNext
+						if debugMatches {
+							println("long match (after short)")
+						}
+						break
+					}
+				}
+
+				// Check prev long...
+				coffsetL = candidateL.prev - e.cur
+				if s-coffsetL < e.maxMatchOff && cv == load6432(src, coffsetL) {
+					// Found a long match, at least 8 bytes.
+					matchedNext := e.matchlen(s+8+checkAt, coffsetL+8, src) + 8
+					if matchedNext > matched {
+						t = coffsetL
+						s += checkAt
+						matched = matchedNext
+						if debugMatches {
+							println("prev long match (after short)")
+						}
+						break
+					}
+				}
+				t = coffsetS
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				if debugMatches {
+					println("short match")
+				}
+				break
+			}
+
+			// No match found, move forward in input.
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+		// Try to find a better match by searching for a long match at the end of the current best match
+		if s+matched < sLimit {
+			nextHashL := hash8(load6432(src, s+matched), betterLongTableBits)
+			cv := load3232(src, s)
+			candidateL := e.longTable[nextHashL]
+			coffsetL := candidateL.offset - e.cur - matched
+			if coffsetL >= 0 && coffsetL < s && s-coffsetL < e.maxMatchOff && cv == load3232(src, coffsetL) {
+				// Found a long match, at least 4 bytes.
+				matchedNext := e.matchlen(s+4, coffsetL+4, src) + 4
+				if matchedNext > matched {
+					t = coffsetL
+					matched = matchedNext
+					if debugMatches {
+						println("long match at end-of-match")
+					}
+				}
+			}
+
+			// Check prev long...
+			if true {
+				coffsetL = candidateL.prev - e.cur - matched
+				if coffsetL >= 0 && coffsetL < s && s-coffsetL < e.maxMatchOff && cv == load3232(src, coffsetL) {
+					// Found a long match, at least 4 bytes.
+					matchedNext := e.matchlen(s+4, coffsetL+4, src) + 4
+					if matchedNext > matched {
+						t = coffsetL
+						matched = matchedNext
+						if debugMatches {
+							println("prev long match at end-of-match")
+						}
+					}
+				}
+			}
+		}
+		// A match has been found. Update recent offsets.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the n-byte match as long as possible.
+		l := matched
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) -> s - 1
+		index0 := s - l + 1
+		for index0 < s-1 {
+			cv0 := load6432(src, index0)
+			cv1 := cv0 >> 8
+			h0 := hash8(cv0, betterLongTableBits)
+			off := index0 + e.cur
+			e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset}
+			e.markLongShardDirty(h0)
+			h1 := hash5(cv1, betterShortTableBits)
+			e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)}
+			e.markShortShardDirty(h1)
+			index0 += 2
+		}
+
+		cv = load6432(src, s)
+		if !canRepeat {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash5(cv, betterShortTableBits)
+			nextHashL := hash8(cv, betterLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			l := 4 + e.matchlen(s+4, o2+4, src)
+
+			e.longTable[nextHashL] = prevEntry{offset: s + e.cur, prev: e.longTable[nextHashL].offset}
+			e.markLongShardDirty(nextHashL)
+			e.table[nextHashS] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.markShortShardDirty(nextHashS)
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *betterFastEncoder) Reset(d *dict, singleBlock bool) {
+	e.resetBase(d, singleBlock)
+	if d != nil {
+		panic("betterFastEncoder: Reset with dict")
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *betterFastEncoderDict) Reset(d *dict, singleBlock bool) {
+	e.resetBase(d, singleBlock)
+	if d == nil {
+		return
+	}
+	// Init or copy dict table
+	if len(e.dictTable) != len(e.table) || d.id != e.lastDictID {
+		if len(e.dictTable) != len(e.table) {
+			e.dictTable = make([]tableEntry, len(e.table))
+		}
+		end := int32(len(d.content)) - 8 + e.maxMatchOff
+		for i := e.maxMatchOff; i < end; i += 4 {
+			const hashLog = betterShortTableBits
+
+			cv := load6432(d.content, i-e.maxMatchOff)
+			nextHash := hash5(cv, hashLog)      // 0 -> 4
+			nextHash1 := hash5(cv>>8, hashLog)  // 1 -> 5
+			nextHash2 := hash5(cv>>16, hashLog) // 2 -> 6
+			nextHash3 := hash5(cv>>24, hashLog) // 3 -> 7
+			e.dictTable[nextHash] = tableEntry{
+				val:    uint32(cv),
+				offset: i,
+			}
+			e.dictTable[nextHash1] = tableEntry{
+				val:    uint32(cv >> 8),
+				offset: i + 1,
+			}
+			e.dictTable[nextHash2] = tableEntry{
+				val:    uint32(cv >> 16),
+				offset: i + 2,
+			}
+			e.dictTable[nextHash3] = tableEntry{
+				val:    uint32(cv >> 24),
+				offset: i + 3,
+			}
+		}
+		e.lastDictID = d.id
+		e.allDirty = true
+	}
+
+	// Init or copy dict table
+	if len(e.dictLongTable) != len(e.longTable) || d.id != e.lastDictID {
+		if len(e.dictLongTable) != len(e.longTable) {
+			e.dictLongTable = make([]prevEntry, len(e.longTable))
+		}
+		if len(d.content) >= 8 {
+			cv := load6432(d.content, 0)
+			h := hash8(cv, betterLongTableBits)
+			e.dictLongTable[h] = prevEntry{
+				offset: e.maxMatchOff,
+				prev:   e.dictLongTable[h].offset,
+			}
+
+			end := int32(len(d.content)) - 8 + e.maxMatchOff
+			off := 8 // First to read
+			for i := e.maxMatchOff + 1; i < end; i++ {
+				cv = cv>>8 | (uint64(d.content[off]) << 56)
+				h := hash8(cv, betterLongTableBits)
+				e.dictLongTable[h] = prevEntry{
+					offset: i,
+					prev:   e.dictLongTable[h].offset,
+				}
+				off++
+			}
+		}
+		e.lastDictID = d.id
+		e.allDirty = true
+	}
+
+	// Reset table to initial state
+	{
+		dirtyShardCnt := 0
+		if !e.allDirty {
+			for i := range e.shortTableShardDirty {
+				if e.shortTableShardDirty[i] {
+					dirtyShardCnt++
+				}
+			}
+		}
+		const shardCnt = betterShortTableShardCnt
+		const shardSize = betterShortTableShardSize
+		if e.allDirty || dirtyShardCnt > shardCnt*4/6 {
+			copy(e.table[:], e.dictTable)
+			for i := range e.shortTableShardDirty {
+				e.shortTableShardDirty[i] = false
+			}
+		} else {
+			for i := range e.shortTableShardDirty {
+				if !e.shortTableShardDirty[i] {
+					continue
+				}
+
+				copy(e.table[i*shardSize:(i+1)*shardSize], e.dictTable[i*shardSize:(i+1)*shardSize])
+				e.shortTableShardDirty[i] = false
+			}
+		}
+	}
+	{
+		dirtyShardCnt := 0
+		if !e.allDirty {
+			for i := range e.shortTableShardDirty {
+				if e.shortTableShardDirty[i] {
+					dirtyShardCnt++
+				}
+			}
+		}
+		const shardCnt = betterLongTableShardCnt
+		const shardSize = betterLongTableShardSize
+		if e.allDirty || dirtyShardCnt > shardCnt*4/6 {
+			copy(e.longTable[:], e.dictLongTable)
+			for i := range e.longTableShardDirty {
+				e.longTableShardDirty[i] = false
+			}
+		} else {
+			for i := range e.longTableShardDirty {
+				if !e.longTableShardDirty[i] {
+					continue
+				}
+
+				copy(e.longTable[i*shardSize:(i+1)*shardSize], e.dictLongTable[i*shardSize:(i+1)*shardSize])
+				e.longTableShardDirty[i] = false
+			}
+		}
+	}
+	e.cur = e.maxMatchOff
+	e.allDirty = false
+}
+
+func (e *betterFastEncoderDict) markLongShardDirty(entryNum uint32) {
+	e.longTableShardDirty[entryNum/betterLongTableShardSize] = true
+}
+
+func (e *betterFastEncoderDict) markShortShardDirty(entryNum uint32) {
+	e.shortTableShardDirty[entryNum/betterShortTableShardSize] = true
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/enc_dfast.go b/vendor/github.com/klauspost/compress/zstd/enc_dfast.go
new file mode 100644
index 0000000..8629d43
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/enc_dfast.go
@@ -0,0 +1,1121 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import "fmt"
+
+const (
+	dFastLongTableBits = 17                      // Bits used in the long match table
+	dFastLongTableSize = 1 << dFastLongTableBits // Size of the table
+	dFastLongTableMask = dFastLongTableSize - 1  // Mask for table indices. Redundant, but can eliminate bounds checks.
+
+	dLongTableShardCnt  = 1 << (dFastLongTableBits - dictShardBits) // Number of shards in the table
+	dLongTableShardSize = dFastLongTableSize / tableShardCnt        // Size of an individual shard
+
+	dFastShortTableBits = tableBits                // Bits used in the short match table
+	dFastShortTableSize = 1 << dFastShortTableBits // Size of the table
+	dFastShortTableMask = dFastShortTableSize - 1  // Mask for table indices. Redundant, but can eliminate bounds checks.
+)
+
+type doubleFastEncoder struct {
+	fastEncoder
+	longTable [dFastLongTableSize]tableEntry
+}
+
+type doubleFastEncoderDict struct {
+	fastEncoderDict
+	longTable           [dFastLongTableSize]tableEntry
+	dictLongTable       []tableEntry
+	longTableShardDirty [dLongTableShardCnt]bool
+}
+
+// Encode mimmics functionality in zstd_dfast.c
+func (e *doubleFastEncoder) Encode(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 2
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			for i := range e.longTable[:] {
+				e.longTable[i] = tableEntry{}
+			}
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		for i := range e.longTable[:] {
+			v := e.longTable[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.longTable[i].offset = v
+		}
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 1.
+	const stepSize = 1
+
+	const kSearchStrength = 8
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		var t int32
+		// We allow the encoder to optionally turn off repeat offsets across blocks
+		canRepeat := len(blk.sequences) > 2
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHashS := hash5(cv, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+			candidateL := e.longTable[nextHashL]
+			candidateS := e.table[nextHashS]
+
+			const repOff = 1
+			repIndex := s - offset1 + repOff
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.table[nextHashS] = entry
+
+			if canRepeat {
+				if repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>(repOff*8)) {
+					// Consider history as well.
+					var seq seq
+					lenght := 4 + e.matchlen(s+4+repOff, repIndex+4, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 0
+					seq.offset = 1
+					if debugSequences {
+						println("repeat sequence", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+					s += lenght + repOff
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+					cv = load6432(src, s)
+					continue
+				}
+			}
+			// Find the offsets of our two matches.
+			coffsetL := s - (candidateL.offset - e.cur)
+			coffsetS := s - (candidateS.offset - e.cur)
+
+			// Check if we have a long match.
+			if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+				// Found a long match, likely at least 8 bytes.
+				// Reference encoder checks all 8 bytes, we only check 4,
+				// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+				t = candidateL.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+				break
+			}
+
+			// Check if we have a short match.
+			if coffsetS < e.maxMatchOff && uint32(cv) == candidateS.val {
+				// found a regular match
+				// See if we can find a long match at s+1
+				const checkAt = 1
+				cv := load6432(src, s+checkAt)
+				nextHashL = hash8(cv, dFastLongTableBits)
+				candidateL = e.longTable[nextHashL]
+				coffsetL = s - (candidateL.offset - e.cur) + checkAt
+
+				// We can store it, since we have at least a 4 byte match.
+				e.longTable[nextHashL] = tableEntry{offset: s + checkAt + e.cur, val: uint32(cv)}
+				if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+					// Found a long match, likely at least 8 bytes.
+					// Reference encoder checks all 8 bytes, we only check 4,
+					// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+					t = candidateL.offset - e.cur
+					s += checkAt
+					if debugMatches {
+						println("long match (after short)")
+					}
+					break
+				}
+
+				t = candidateS.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				if debugMatches {
+					println("short match")
+				}
+				break
+			}
+
+			// No match found, move forward in input.
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+
+		// A 4-byte match has been found. Update recent offsets.
+		// We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the 4-byte match as long as possible.
+		l := e.matchlen(s+4, t+4, src) + 4
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) and start+2 (short)
+		index0 := s - l + 1
+		// Index match end-2 (long) and end-1 (short)
+		index1 := s - 2
+
+		cv0 := load6432(src, index0)
+		cv1 := load6432(src, index1)
+		te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)}
+		te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)}
+		e.longTable[hash8(cv0, dFastLongTableBits)] = te0
+		e.longTable[hash8(cv1, dFastLongTableBits)] = te1
+		cv0 >>= 8
+		cv1 >>= 8
+		te0.offset++
+		te1.offset++
+		te0.val = uint32(cv0)
+		te1.val = uint32(cv1)
+		e.table[hash5(cv0, dFastShortTableBits)] = te0
+		e.table[hash5(cv1, dFastShortTableBits)] = te1
+
+		cv = load6432(src, s)
+
+		if !canRepeat {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash5(cv, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			l := 4 + e.matchlen(s+4, o2+4, src)
+
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.table[nextHashS] = entry
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// EncodeNoHist will encode a block with no history and no following blocks.
+// Most notable difference is that src will not be copied for history and
+// we do not need to check for max match length.
+func (e *doubleFastEncoder) EncodeNoHist(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 2
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	if e.cur >= bufferReset {
+		for i := range e.table[:] {
+			e.table[i] = tableEntry{}
+		}
+		for i := range e.longTable[:] {
+			e.longTable[i] = tableEntry{}
+		}
+		e.cur = e.maxMatchOff
+	}
+
+	s := int32(0)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 1.
+	const stepSize = 1
+
+	const kSearchStrength = 8
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		var t int32
+		for {
+
+			nextHashS := hash5(cv, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+			candidateL := e.longTable[nextHashL]
+			candidateS := e.table[nextHashS]
+
+			const repOff = 1
+			repIndex := s - offset1 + repOff
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.table[nextHashS] = entry
+
+			if len(blk.sequences) > 2 {
+				if load3232(src, repIndex) == uint32(cv>>(repOff*8)) {
+					// Consider history as well.
+					var seq seq
+					//length := 4 + e.matchlen(s+4+repOff, repIndex+4, src)
+					length := 4 + int32(matchLen(src[s+4+repOff:], src[repIndex+4:]))
+
+					seq.matchLen = uint32(length - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 0
+					seq.offset = 1
+					if debugSequences {
+						println("repeat sequence", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+					s += length + repOff
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, length)
+
+						}
+						break encodeLoop
+					}
+					cv = load6432(src, s)
+					continue
+				}
+			}
+			// Find the offsets of our two matches.
+			coffsetL := s - (candidateL.offset - e.cur)
+			coffsetS := s - (candidateS.offset - e.cur)
+
+			// Check if we have a long match.
+			if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+				// Found a long match, likely at least 8 bytes.
+				// Reference encoder checks all 8 bytes, we only check 4,
+				// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+				t = candidateL.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d). cur: %d", s, t, e.cur))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+				break
+			}
+
+			// Check if we have a short match.
+			if coffsetS < e.maxMatchOff && uint32(cv) == candidateS.val {
+				// found a regular match
+				// See if we can find a long match at s+1
+				const checkAt = 1
+				cv := load6432(src, s+checkAt)
+				nextHashL = hash8(cv, dFastLongTableBits)
+				candidateL = e.longTable[nextHashL]
+				coffsetL = s - (candidateL.offset - e.cur) + checkAt
+
+				// We can store it, since we have at least a 4 byte match.
+				e.longTable[nextHashL] = tableEntry{offset: s + checkAt + e.cur, val: uint32(cv)}
+				if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+					// Found a long match, likely at least 8 bytes.
+					// Reference encoder checks all 8 bytes, we only check 4,
+					// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+					t = candidateL.offset - e.cur
+					s += checkAt
+					if debugMatches {
+						println("long match (after short)")
+					}
+					break
+				}
+
+				t = candidateS.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				if debugMatches {
+					println("short match")
+				}
+				break
+			}
+
+			// No match found, move forward in input.
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+
+		// A 4-byte match has been found. Update recent offsets.
+		// We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		// Extend the 4-byte match as long as possible.
+		//l := e.matchlen(s+4, t+4, src) + 4
+		l := int32(matchLen(src[s+4:], src[t+4:])) + 4
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) and start+2 (short)
+		index0 := s - l + 1
+		// Index match end-2 (long) and end-1 (short)
+		index1 := s - 2
+
+		cv0 := load6432(src, index0)
+		cv1 := load6432(src, index1)
+		te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)}
+		te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)}
+		e.longTable[hash8(cv0, dFastLongTableBits)] = te0
+		e.longTable[hash8(cv1, dFastLongTableBits)] = te1
+		cv0 >>= 8
+		cv1 >>= 8
+		te0.offset++
+		te1.offset++
+		te0.val = uint32(cv0)
+		te1.val = uint32(cv1)
+		e.table[hash5(cv0, dFastShortTableBits)] = te0
+		e.table[hash5(cv1, dFastShortTableBits)] = te1
+
+		cv = load6432(src, s)
+
+		if len(blk.sequences) <= 2 {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash5(cv1>>8, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			//l := 4 + e.matchlen(s+4, o2+4, src)
+			l := 4 + int32(matchLen(src[s+4:], src[o2+4:]))
+
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.table[nextHashS] = entry
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+
+	// We do not store history, so we must offset e.cur to avoid false matches for next user.
+	if e.cur < bufferReset {
+		e.cur += int32(len(src))
+	}
+}
+
+// Encode will encode the content, with a dictionary if initialized for it.
+func (e *doubleFastEncoderDict) Encode(blk *blockEnc, src []byte) {
+	const (
+		// Input margin is the number of bytes we read (8)
+		// and the maximum we will read ahead (2)
+		inputMargin            = 8 + 2
+		minNonLiteralBlockSize = 16
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			for i := range e.longTable[:] {
+				e.longTable[i] = tableEntry{}
+			}
+			e.markAllShardsDirty()
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		for i := range e.longTable[:] {
+			v := e.longTable[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.longTable[i].offset = v
+		}
+		e.markAllShardsDirty()
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 1.
+	const stepSize = 1
+
+	const kSearchStrength = 8
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		var t int32
+		// We allow the encoder to optionally turn off repeat offsets across blocks
+		canRepeat := len(blk.sequences) > 2
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHashS := hash5(cv, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+			candidateL := e.longTable[nextHashL]
+			candidateS := e.table[nextHashS]
+
+			const repOff = 1
+			repIndex := s - offset1 + repOff
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.markLongShardDirty(nextHashL)
+			e.table[nextHashS] = entry
+			e.markShardDirty(nextHashS)
+
+			if canRepeat {
+				if repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>(repOff*8)) {
+					// Consider history as well.
+					var seq seq
+					lenght := 4 + e.matchlen(s+4+repOff, repIndex+4, src)
+
+					seq.matchLen = uint32(lenght - zstdMinMatch)
+
+					// We might be able to match backwards.
+					// Extend as long as we can.
+					start := s + repOff
+					// We end the search early, so we don't risk 0 literals
+					// and have to do special offset treatment.
+					startLimit := nextEmit + 1
+
+					tMin := s - e.maxMatchOff
+					if tMin < 0 {
+						tMin = 0
+					}
+					for repIndex > tMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch-1 {
+						repIndex--
+						start--
+						seq.matchLen++
+					}
+					addLiterals(&seq, start)
+
+					// rep 0
+					seq.offset = 1
+					if debugSequences {
+						println("repeat sequence", seq, "next s:", s)
+					}
+					blk.sequences = append(blk.sequences, seq)
+					s += lenght + repOff
+					nextEmit = s
+					if s >= sLimit {
+						if debug {
+							println("repeat ended", s, lenght)
+
+						}
+						break encodeLoop
+					}
+					cv = load6432(src, s)
+					continue
+				}
+			}
+			// Find the offsets of our two matches.
+			coffsetL := s - (candidateL.offset - e.cur)
+			coffsetS := s - (candidateS.offset - e.cur)
+
+			// Check if we have a long match.
+			if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+				// Found a long match, likely at least 8 bytes.
+				// Reference encoder checks all 8 bytes, we only check 4,
+				// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+				t = candidateL.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugMatches {
+					println("long match")
+				}
+				break
+			}
+
+			// Check if we have a short match.
+			if coffsetS < e.maxMatchOff && uint32(cv) == candidateS.val {
+				// found a regular match
+				// See if we can find a long match at s+1
+				const checkAt = 1
+				cv := load6432(src, s+checkAt)
+				nextHashL = hash8(cv, dFastLongTableBits)
+				candidateL = e.longTable[nextHashL]
+				coffsetL = s - (candidateL.offset - e.cur) + checkAt
+
+				// We can store it, since we have at least a 4 byte match.
+				e.longTable[nextHashL] = tableEntry{offset: s + checkAt + e.cur, val: uint32(cv)}
+				e.markLongShardDirty(nextHashL)
+				if coffsetL < e.maxMatchOff && uint32(cv) == candidateL.val {
+					// Found a long match, likely at least 8 bytes.
+					// Reference encoder checks all 8 bytes, we only check 4,
+					// but the likelihood of both the first 4 bytes and the hash matching should be enough.
+					t = candidateL.offset - e.cur
+					s += checkAt
+					if debugMatches {
+						println("long match (after short)")
+					}
+					break
+				}
+
+				t = candidateS.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				if debugMatches {
+					println("short match")
+				}
+				break
+			}
+
+			// No match found, move forward in input.
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+
+		// A 4-byte match has been found. Update recent offsets.
+		// We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the 4-byte match as long as possible.
+		l := e.matchlen(s+4, t+4, src) + 4
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+
+		// Index match start+1 (long) and start+2 (short)
+		index0 := s - l + 1
+		// Index match end-2 (long) and end-1 (short)
+		index1 := s - 2
+
+		cv0 := load6432(src, index0)
+		cv1 := load6432(src, index1)
+		te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)}
+		te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)}
+		longHash1 := hash8(cv0, dFastLongTableBits)
+		longHash2 := hash8(cv0, dFastLongTableBits)
+		e.longTable[longHash1] = te0
+		e.longTable[longHash2] = te1
+		e.markLongShardDirty(longHash1)
+		e.markLongShardDirty(longHash2)
+		cv0 >>= 8
+		cv1 >>= 8
+		te0.offset++
+		te1.offset++
+		te0.val = uint32(cv0)
+		te1.val = uint32(cv1)
+		hashVal1 := hash5(cv0, dFastShortTableBits)
+		hashVal2 := hash5(cv1, dFastShortTableBits)
+		e.table[hashVal1] = te0
+		e.markShardDirty(hashVal1)
+		e.table[hashVal2] = te1
+		e.markShardDirty(hashVal2)
+
+		cv = load6432(src, s)
+
+		if !canRepeat {
+			continue
+		}
+
+		// Check offset 2
+		for {
+			o2 := s - offset2
+			if load3232(src, o2) != uint32(cv) {
+				// Do regular search
+				break
+			}
+
+			// Store this, since we have it.
+			nextHashS := hash5(cv, dFastShortTableBits)
+			nextHashL := hash8(cv, dFastLongTableBits)
+
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			l := 4 + e.matchlen(s+4, o2+4, src)
+
+			entry := tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.longTable[nextHashL] = entry
+			e.markLongShardDirty(nextHashL)
+			e.table[nextHashS] = entry
+			e.markShardDirty(nextHashS)
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				// Finished
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+	// If we encoded more than 64K mark all dirty.
+	if len(src) > 64<<10 {
+		e.markAllShardsDirty()
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *doubleFastEncoder) Reset(d *dict, singleBlock bool) {
+	e.fastEncoder.Reset(d, singleBlock)
+	if d != nil {
+		panic("doubleFastEncoder: Reset with dict not supported")
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *doubleFastEncoderDict) Reset(d *dict, singleBlock bool) {
+	allDirty := e.allDirty
+	e.fastEncoderDict.Reset(d, singleBlock)
+	if d == nil {
+		return
+	}
+
+	// Init or copy dict table
+	if len(e.dictLongTable) != len(e.longTable) || d.id != e.lastDictID {
+		if len(e.dictLongTable) != len(e.longTable) {
+			e.dictLongTable = make([]tableEntry, len(e.longTable))
+		}
+		if len(d.content) >= 8 {
+			cv := load6432(d.content, 0)
+			e.dictLongTable[hash8(cv, dFastLongTableBits)] = tableEntry{
+				val:    uint32(cv),
+				offset: e.maxMatchOff,
+			}
+			end := int32(len(d.content)) - 8 + e.maxMatchOff
+			for i := e.maxMatchOff + 1; i < end; i++ {
+				cv = cv>>8 | (uint64(d.content[i-e.maxMatchOff+7]) << 56)
+				e.dictLongTable[hash8(cv, dFastLongTableBits)] = tableEntry{
+					val:    uint32(cv),
+					offset: i,
+				}
+			}
+		}
+		e.lastDictID = d.id
+		e.allDirty = true
+	}
+	// Reset table to initial state
+	e.cur = e.maxMatchOff
+
+	dirtyShardCnt := 0
+	if !allDirty {
+		for i := range e.longTableShardDirty {
+			if e.longTableShardDirty[i] {
+				dirtyShardCnt++
+			}
+		}
+	}
+
+	if allDirty || dirtyShardCnt > dLongTableShardCnt/2 {
+		copy(e.longTable[:], e.dictLongTable)
+		for i := range e.longTableShardDirty {
+			e.longTableShardDirty[i] = false
+		}
+		return
+	}
+	for i := range e.longTableShardDirty {
+		if !e.longTableShardDirty[i] {
+			continue
+		}
+
+		copy(e.longTable[i*dLongTableShardSize:(i+1)*dLongTableShardSize], e.dictLongTable[i*dLongTableShardSize:(i+1)*dLongTableShardSize])
+		e.longTableShardDirty[i] = false
+	}
+}
+
+func (e *doubleFastEncoderDict) markLongShardDirty(entryNum uint32) {
+	e.longTableShardDirty[entryNum/dLongTableShardSize] = true
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/enc_fast.go b/vendor/github.com/klauspost/compress/zstd/enc_fast.go
new file mode 100644
index 0000000..ba4a17e
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/enc_fast.go
@@ -0,0 +1,1018 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"fmt"
+	"math"
+	"math/bits"
+)
+
+const (
+	tableBits      = 15                               // Bits used in the table
+	tableSize      = 1 << tableBits                   // Size of the table
+	tableShardCnt  = 1 << (tableBits - dictShardBits) // Number of shards in the table
+	tableShardSize = tableSize / tableShardCnt        // Size of an individual shard
+	tableMask      = tableSize - 1                    // Mask for table indices. Redundant, but can eliminate bounds checks.
+	maxMatchLength = 131074
+)
+
+type tableEntry struct {
+	val    uint32
+	offset int32
+}
+
+type fastEncoder struct {
+	fastBase
+	table [tableSize]tableEntry
+}
+
+type fastEncoderDict struct {
+	fastEncoder
+	dictTable       []tableEntry
+	tableShardDirty [tableShardCnt]bool
+	allDirty        bool
+}
+
+// Encode mimmics functionality in zstd_fast.c
+func (e *fastEncoder) Encode(blk *blockEnc, src []byte) {
+	const (
+		inputMargin            = 8
+		minNonLiteralBlockSize = 1 + 1 + inputMargin
+	)
+
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 2.
+	const stepSize = 2
+
+	// TEMPLATE
+	const hashLog = tableBits
+	// seems global, but would be nice to tweak.
+	const kSearchStrength = 7
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		// t will contain the match offset when we find one.
+		// When existing the search loop, we have already checked 4 bytes.
+		var t int32
+
+		// We will not use repeat offsets across blocks.
+		// By not using them for the first 3 matches
+		canRepeat := len(blk.sequences) > 2
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHash := hash6(cv, hashLog)
+			nextHash2 := hash6(cv>>8, hashLog)
+			candidate := e.table[nextHash]
+			candidate2 := e.table[nextHash2]
+			repIndex := s - offset1 + 2
+
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.table[nextHash2] = tableEntry{offset: s + e.cur + 1, val: uint32(cv >> 8)}
+
+			if canRepeat && repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>16) {
+				// Consider history as well.
+				var seq seq
+				var length int32
+				// length = 4 + e.matchlen(s+6, repIndex+4, src)
+				{
+					a := src[s+6:]
+					b := src[repIndex+4:]
+					endI := len(a) & (math.MaxInt32 - 7)
+					length = int32(endI) + 4
+					for i := 0; i < endI; i += 8 {
+						if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+							length = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+							break
+						}
+					}
+				}
+
+				seq.matchLen = uint32(length - zstdMinMatch)
+
+				// We might be able to match backwards.
+				// Extend as long as we can.
+				start := s + 2
+				// We end the search early, so we don't risk 0 literals
+				// and have to do special offset treatment.
+				startLimit := nextEmit + 1
+
+				sMin := s - e.maxMatchOff
+				if sMin < 0 {
+					sMin = 0
+				}
+				for repIndex > sMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch {
+					repIndex--
+					start--
+					seq.matchLen++
+				}
+				addLiterals(&seq, start)
+
+				// rep 0
+				seq.offset = 1
+				if debugSequences {
+					println("repeat sequence", seq, "next s:", s)
+				}
+				blk.sequences = append(blk.sequences, seq)
+				s += length + 2
+				nextEmit = s
+				if s >= sLimit {
+					if debug {
+						println("repeat ended", s, length)
+
+					}
+					break encodeLoop
+				}
+				cv = load6432(src, s)
+				continue
+			}
+			coffset0 := s - (candidate.offset - e.cur)
+			coffset1 := s - (candidate2.offset - e.cur) + 1
+			if coffset0 < e.maxMatchOff && uint32(cv) == candidate.val {
+				// found a regular match
+				t = candidate.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				break
+			}
+
+			if coffset1 < e.maxMatchOff && uint32(cv>>8) == candidate2.val {
+				// found a regular match
+				t = candidate2.offset - e.cur
+				s++
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				break
+			}
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+		// A 4-byte match has been found. We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the 4-byte match as long as possible.
+		//l := e.matchlen(s+4, t+4, src) + 4
+		var l int32
+		{
+			a := src[s+4:]
+			b := src[t+4:]
+			endI := len(a) & (math.MaxInt32 - 7)
+			l = int32(endI) + 4
+			for i := 0; i < endI; i += 8 {
+				if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+					l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+					break
+				}
+			}
+		}
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence.
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		// Don't use repeat offsets
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+		cv = load6432(src, s)
+
+		// Check offset 2
+		if o2 := s - offset2; canRepeat && load3232(src, o2) == uint32(cv) {
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			//l := 4 + e.matchlen(s+4, o2+4, src)
+			var l int32
+			{
+				a := src[s+4:]
+				b := src[o2+4:]
+				endI := len(a) & (math.MaxInt32 - 7)
+				l = int32(endI) + 4
+				for i := 0; i < endI; i += 8 {
+					if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+						l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+						break
+					}
+				}
+			}
+
+			// Store this, since we have it.
+			nextHash := hash6(cv, hashLog)
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				break encodeLoop
+			}
+			// Prepare next loop.
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// EncodeNoHist will encode a block with no history and no following blocks.
+// Most notable difference is that src will not be copied for history and
+// we do not need to check for max match length.
+func (e *fastEncoder) EncodeNoHist(blk *blockEnc, src []byte) {
+	const (
+		inputMargin            = 8
+		minNonLiteralBlockSize = 1 + 1 + inputMargin
+	)
+	if debug {
+		if len(src) > maxBlockSize {
+			panic("src too big")
+		}
+	}
+
+	// Protect against e.cur wraparound.
+	if e.cur >= bufferReset {
+		for i := range e.table[:] {
+			e.table[i] = tableEntry{}
+		}
+		e.cur = e.maxMatchOff
+	}
+
+	s := int32(0)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 2.
+	const stepSize = 2
+
+	// TEMPLATE
+	const hashLog = tableBits
+	// seems global, but would be nice to tweak.
+	const kSearchStrength = 8
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		// t will contain the match offset when we find one.
+		// When existing the search loop, we have already checked 4 bytes.
+		var t int32
+
+		// We will not use repeat offsets across blocks.
+		// By not using them for the first 3 matches
+
+		for {
+			nextHash := hash6(cv, hashLog)
+			nextHash2 := hash6(cv>>8, hashLog)
+			candidate := e.table[nextHash]
+			candidate2 := e.table[nextHash2]
+			repIndex := s - offset1 + 2
+
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.table[nextHash2] = tableEntry{offset: s + e.cur + 1, val: uint32(cv >> 8)}
+
+			if len(blk.sequences) > 2 && load3232(src, repIndex) == uint32(cv>>16) {
+				// Consider history as well.
+				var seq seq
+				// length := 4 + e.matchlen(s+6, repIndex+4, src)
+				// length := 4 + int32(matchLen(src[s+6:], src[repIndex+4:]))
+				var length int32
+				{
+					a := src[s+6:]
+					b := src[repIndex+4:]
+					endI := len(a) & (math.MaxInt32 - 7)
+					length = int32(endI) + 4
+					for i := 0; i < endI; i += 8 {
+						if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+							length = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+							break
+						}
+					}
+				}
+
+				seq.matchLen = uint32(length - zstdMinMatch)
+
+				// We might be able to match backwards.
+				// Extend as long as we can.
+				start := s + 2
+				// We end the search early, so we don't risk 0 literals
+				// and have to do special offset treatment.
+				startLimit := nextEmit + 1
+
+				sMin := s - e.maxMatchOff
+				if sMin < 0 {
+					sMin = 0
+				}
+				for repIndex > sMin && start > startLimit && src[repIndex-1] == src[start-1] {
+					repIndex--
+					start--
+					seq.matchLen++
+				}
+				addLiterals(&seq, start)
+
+				// rep 0
+				seq.offset = 1
+				if debugSequences {
+					println("repeat sequence", seq, "next s:", s)
+				}
+				blk.sequences = append(blk.sequences, seq)
+				s += length + 2
+				nextEmit = s
+				if s >= sLimit {
+					if debug {
+						println("repeat ended", s, length)
+
+					}
+					break encodeLoop
+				}
+				cv = load6432(src, s)
+				continue
+			}
+			coffset0 := s - (candidate.offset - e.cur)
+			coffset1 := s - (candidate2.offset - e.cur) + 1
+			if coffset0 < e.maxMatchOff && uint32(cv) == candidate.val {
+				// found a regular match
+				t = candidate.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic(fmt.Sprintf("t (%d) < 0, candidate.offset: %d, e.cur: %d, coffset0: %d, e.maxMatchOff: %d", t, candidate.offset, e.cur, coffset0, e.maxMatchOff))
+				}
+				break
+			}
+
+			if coffset1 < e.maxMatchOff && uint32(cv>>8) == candidate2.val {
+				// found a regular match
+				t = candidate2.offset - e.cur
+				s++
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				break
+			}
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+		// A 4-byte match has been found. We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && t < 0 {
+			panic(fmt.Sprintf("t (%d) < 0 ", t))
+		}
+		// Extend the 4-byte match as long as possible.
+		//l := e.matchlenNoHist(s+4, t+4, src) + 4
+		// l := int32(matchLen(src[s+4:], src[t+4:])) + 4
+		var l int32
+		{
+			a := src[s+4:]
+			b := src[t+4:]
+			endI := len(a) & (math.MaxInt32 - 7)
+			l = int32(endI) + 4
+			for i := 0; i < endI; i += 8 {
+				if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+					l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+					break
+				}
+			}
+		}
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence.
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		// Don't use repeat offsets
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+		cv = load6432(src, s)
+
+		// Check offset 2
+		if o2 := s - offset2; len(blk.sequences) > 2 && load3232(src, o2) == uint32(cv) {
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			//l := 4 + e.matchlenNoHist(s+4, o2+4, src)
+			// l := 4 + int32(matchLen(src[s+4:], src[o2+4:]))
+			var l int32
+			{
+				a := src[s+4:]
+				b := src[o2+4:]
+				endI := len(a) & (math.MaxInt32 - 7)
+				l = int32(endI) + 4
+				for i := 0; i < endI; i += 8 {
+					if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+						l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+						break
+					}
+				}
+			}
+
+			// Store this, since we have it.
+			nextHash := hash6(cv, hashLog)
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				break encodeLoop
+			}
+			// Prepare next loop.
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+	// We do not store history, so we must offset e.cur to avoid false matches for next user.
+	if e.cur < bufferReset {
+		e.cur += int32(len(src))
+	}
+}
+
+// Encode will encode the content, with a dictionary if initialized for it.
+func (e *fastEncoderDict) Encode(blk *blockEnc, src []byte) {
+	const (
+		inputMargin            = 8
+		minNonLiteralBlockSize = 1 + 1 + inputMargin
+	)
+	if e.allDirty || len(src) > 32<<10 {
+		e.fastEncoder.Encode(blk, src)
+		e.allDirty = true
+		return
+	}
+	// Protect against e.cur wraparound.
+	for e.cur >= bufferReset {
+		if len(e.hist) == 0 {
+			for i := range e.table[:] {
+				e.table[i] = tableEntry{}
+			}
+			e.cur = e.maxMatchOff
+			break
+		}
+		// Shift down everything in the table that isn't already too far away.
+		minOff := e.cur + int32(len(e.hist)) - e.maxMatchOff
+		for i := range e.table[:] {
+			v := e.table[i].offset
+			if v < minOff {
+				v = 0
+			} else {
+				v = v - e.cur + e.maxMatchOff
+			}
+			e.table[i].offset = v
+		}
+		e.cur = e.maxMatchOff
+		break
+	}
+
+	s := e.addBlock(src)
+	blk.size = len(src)
+	if len(src) < minNonLiteralBlockSize {
+		blk.extraLits = len(src)
+		blk.literals = blk.literals[:len(src)]
+		copy(blk.literals, src)
+		return
+	}
+
+	// Override src
+	src = e.hist
+	sLimit := int32(len(src)) - inputMargin
+	// stepSize is the number of bytes to skip on every main loop iteration.
+	// It should be >= 2.
+	const stepSize = 2
+
+	// TEMPLATE
+	const hashLog = tableBits
+	// seems global, but would be nice to tweak.
+	const kSearchStrength = 7
+
+	// nextEmit is where in src the next emitLiteral should start from.
+	nextEmit := s
+	cv := load6432(src, s)
+
+	// Relative offsets
+	offset1 := int32(blk.recentOffsets[0])
+	offset2 := int32(blk.recentOffsets[1])
+
+	addLiterals := func(s *seq, until int32) {
+		if until == nextEmit {
+			return
+		}
+		blk.literals = append(blk.literals, src[nextEmit:until]...)
+		s.litLen = uint32(until - nextEmit)
+	}
+	if debug {
+		println("recent offsets:", blk.recentOffsets)
+	}
+
+encodeLoop:
+	for {
+		// t will contain the match offset when we find one.
+		// When existing the search loop, we have already checked 4 bytes.
+		var t int32
+
+		// We will not use repeat offsets across blocks.
+		// By not using them for the first 3 matches
+		canRepeat := len(blk.sequences) > 2
+
+		for {
+			if debugAsserts && canRepeat && offset1 == 0 {
+				panic("offset0 was 0")
+			}
+
+			nextHash := hash6(cv, hashLog)
+			nextHash2 := hash6(cv>>8, hashLog)
+			candidate := e.table[nextHash]
+			candidate2 := e.table[nextHash2]
+			repIndex := s - offset1 + 2
+
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.markShardDirty(nextHash)
+			e.table[nextHash2] = tableEntry{offset: s + e.cur + 1, val: uint32(cv >> 8)}
+			e.markShardDirty(nextHash2)
+
+			if canRepeat && repIndex >= 0 && load3232(src, repIndex) == uint32(cv>>16) {
+				// Consider history as well.
+				var seq seq
+				var length int32
+				// length = 4 + e.matchlen(s+6, repIndex+4, src)
+				{
+					a := src[s+6:]
+					b := src[repIndex+4:]
+					endI := len(a) & (math.MaxInt32 - 7)
+					length = int32(endI) + 4
+					for i := 0; i < endI; i += 8 {
+						if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+							length = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+							break
+						}
+					}
+				}
+
+				seq.matchLen = uint32(length - zstdMinMatch)
+
+				// We might be able to match backwards.
+				// Extend as long as we can.
+				start := s + 2
+				// We end the search early, so we don't risk 0 literals
+				// and have to do special offset treatment.
+				startLimit := nextEmit + 1
+
+				sMin := s - e.maxMatchOff
+				if sMin < 0 {
+					sMin = 0
+				}
+				for repIndex > sMin && start > startLimit && src[repIndex-1] == src[start-1] && seq.matchLen < maxMatchLength-zstdMinMatch {
+					repIndex--
+					start--
+					seq.matchLen++
+				}
+				addLiterals(&seq, start)
+
+				// rep 0
+				seq.offset = 1
+				if debugSequences {
+					println("repeat sequence", seq, "next s:", s)
+				}
+				blk.sequences = append(blk.sequences, seq)
+				s += length + 2
+				nextEmit = s
+				if s >= sLimit {
+					if debug {
+						println("repeat ended", s, length)
+
+					}
+					break encodeLoop
+				}
+				cv = load6432(src, s)
+				continue
+			}
+			coffset0 := s - (candidate.offset - e.cur)
+			coffset1 := s - (candidate2.offset - e.cur) + 1
+			if coffset0 < e.maxMatchOff && uint32(cv) == candidate.val {
+				// found a regular match
+				t = candidate.offset - e.cur
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				break
+			}
+
+			if coffset1 < e.maxMatchOff && uint32(cv>>8) == candidate2.val {
+				// found a regular match
+				t = candidate2.offset - e.cur
+				s++
+				if debugAsserts && s <= t {
+					panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+				}
+				if debugAsserts && s-t > e.maxMatchOff {
+					panic("s - t >e.maxMatchOff")
+				}
+				if debugAsserts && t < 0 {
+					panic("t<0")
+				}
+				break
+			}
+			s += stepSize + ((s - nextEmit) >> (kSearchStrength - 1))
+			if s >= sLimit {
+				break encodeLoop
+			}
+			cv = load6432(src, s)
+		}
+		// A 4-byte match has been found. We'll later see if more than 4 bytes.
+		offset2 = offset1
+		offset1 = s - t
+
+		if debugAsserts && s <= t {
+			panic(fmt.Sprintf("s (%d) <= t (%d)", s, t))
+		}
+
+		if debugAsserts && canRepeat && int(offset1) > len(src) {
+			panic("invalid offset")
+		}
+
+		// Extend the 4-byte match as long as possible.
+		//l := e.matchlen(s+4, t+4, src) + 4
+		var l int32
+		{
+			a := src[s+4:]
+			b := src[t+4:]
+			endI := len(a) & (math.MaxInt32 - 7)
+			l = int32(endI) + 4
+			for i := 0; i < endI; i += 8 {
+				if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+					l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+					break
+				}
+			}
+		}
+
+		// Extend backwards
+		tMin := s - e.maxMatchOff
+		if tMin < 0 {
+			tMin = 0
+		}
+		for t > tMin && s > nextEmit && src[t-1] == src[s-1] && l < maxMatchLength {
+			s--
+			t--
+			l++
+		}
+
+		// Write our sequence.
+		var seq seq
+		seq.litLen = uint32(s - nextEmit)
+		seq.matchLen = uint32(l - zstdMinMatch)
+		if seq.litLen > 0 {
+			blk.literals = append(blk.literals, src[nextEmit:s]...)
+		}
+		// Don't use repeat offsets
+		seq.offset = uint32(s-t) + 3
+		s += l
+		if debugSequences {
+			println("sequence", seq, "next s:", s)
+		}
+		blk.sequences = append(blk.sequences, seq)
+		nextEmit = s
+		if s >= sLimit {
+			break encodeLoop
+		}
+		cv = load6432(src, s)
+
+		// Check offset 2
+		if o2 := s - offset2; canRepeat && load3232(src, o2) == uint32(cv) {
+			// We have at least 4 byte match.
+			// No need to check backwards. We come straight from a match
+			//l := 4 + e.matchlen(s+4, o2+4, src)
+			var l int32
+			{
+				a := src[s+4:]
+				b := src[o2+4:]
+				endI := len(a) & (math.MaxInt32 - 7)
+				l = int32(endI) + 4
+				for i := 0; i < endI; i += 8 {
+					if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+						l = int32(i+bits.TrailingZeros64(diff)>>3) + 4
+						break
+					}
+				}
+			}
+
+			// Store this, since we have it.
+			nextHash := hash6(cv, hashLog)
+			e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)}
+			e.markShardDirty(nextHash)
+			seq.matchLen = uint32(l) - zstdMinMatch
+			seq.litLen = 0
+			// Since litlen is always 0, this is offset 1.
+			seq.offset = 1
+			s += l
+			nextEmit = s
+			if debugSequences {
+				println("sequence", seq, "next s:", s)
+			}
+			blk.sequences = append(blk.sequences, seq)
+
+			// Swap offset 1 and 2.
+			offset1, offset2 = offset2, offset1
+			if s >= sLimit {
+				break encodeLoop
+			}
+			// Prepare next loop.
+			cv = load6432(src, s)
+		}
+	}
+
+	if int(nextEmit) < len(src) {
+		blk.literals = append(blk.literals, src[nextEmit:]...)
+		blk.extraLits = len(src) - int(nextEmit)
+	}
+	blk.recentOffsets[0] = uint32(offset1)
+	blk.recentOffsets[1] = uint32(offset2)
+	if debug {
+		println("returning, recent offsets:", blk.recentOffsets, "extra literals:", blk.extraLits)
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *fastEncoder) Reset(d *dict, singleBlock bool) {
+	e.resetBase(d, singleBlock)
+	if d != nil {
+		panic("fastEncoder: Reset with dict")
+	}
+}
+
+// ResetDict will reset and set a dictionary if not nil
+func (e *fastEncoderDict) Reset(d *dict, singleBlock bool) {
+	e.resetBase(d, singleBlock)
+	if d == nil {
+		return
+	}
+
+	// Init or copy dict table
+	if len(e.dictTable) != len(e.table) || d.id != e.lastDictID {
+		if len(e.dictTable) != len(e.table) {
+			e.dictTable = make([]tableEntry, len(e.table))
+		}
+		if true {
+			end := e.maxMatchOff + int32(len(d.content)) - 8
+			for i := e.maxMatchOff; i < end; i += 3 {
+				const hashLog = tableBits
+
+				cv := load6432(d.content, i-e.maxMatchOff)
+				nextHash := hash6(cv, hashLog)      // 0 -> 5
+				nextHash1 := hash6(cv>>8, hashLog)  // 1 -> 6
+				nextHash2 := hash6(cv>>16, hashLog) // 2 -> 7
+				e.dictTable[nextHash] = tableEntry{
+					val:    uint32(cv),
+					offset: i,
+				}
+				e.dictTable[nextHash1] = tableEntry{
+					val:    uint32(cv >> 8),
+					offset: i + 1,
+				}
+				e.dictTable[nextHash2] = tableEntry{
+					val:    uint32(cv >> 16),
+					offset: i + 2,
+				}
+			}
+		}
+		e.lastDictID = d.id
+		e.allDirty = true
+	}
+
+	e.cur = e.maxMatchOff
+	dirtyShardCnt := 0
+	if !e.allDirty {
+		for i := range e.tableShardDirty {
+			if e.tableShardDirty[i] {
+				dirtyShardCnt++
+			}
+		}
+	}
+
+	const shardCnt = tableShardCnt
+	const shardSize = tableShardSize
+	if e.allDirty || dirtyShardCnt > shardCnt*4/6 {
+		copy(e.table[:], e.dictTable)
+		for i := range e.tableShardDirty {
+			e.tableShardDirty[i] = false
+		}
+		e.allDirty = false
+		return
+	}
+	for i := range e.tableShardDirty {
+		if !e.tableShardDirty[i] {
+			continue
+		}
+
+		copy(e.table[i*shardSize:(i+1)*shardSize], e.dictTable[i*shardSize:(i+1)*shardSize])
+		e.tableShardDirty[i] = false
+	}
+	e.allDirty = false
+}
+
+func (e *fastEncoderDict) markAllShardsDirty() {
+	e.allDirty = true
+}
+
+func (e *fastEncoderDict) markShardDirty(entryNum uint32) {
+	e.tableShardDirty[entryNum/tableShardSize] = true
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/encoder.go b/vendor/github.com/klauspost/compress/zstd/encoder.go
new file mode 100644
index 0000000..4871dd0
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/encoder.go
@@ -0,0 +1,576 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"crypto/rand"
+	"fmt"
+	"io"
+	rdebug "runtime/debug"
+	"sync"
+
+	"github.com/klauspost/compress/zstd/internal/xxhash"
+)
+
+// Encoder provides encoding to Zstandard.
+// An Encoder can be used for either compressing a stream via the
+// io.WriteCloser interface supported by the Encoder or as multiple independent
+// tasks via the EncodeAll function.
+// Smaller encodes are encouraged to use the EncodeAll function.
+// Use NewWriter to create a new instance.
+type Encoder struct {
+	o        encoderOptions
+	encoders chan encoder
+	state    encoderState
+	init     sync.Once
+}
+
+type encoder interface {
+	Encode(blk *blockEnc, src []byte)
+	EncodeNoHist(blk *blockEnc, src []byte)
+	Block() *blockEnc
+	CRC() *xxhash.Digest
+	AppendCRC([]byte) []byte
+	WindowSize(size int) int32
+	UseBlock(*blockEnc)
+	Reset(d *dict, singleBlock bool)
+}
+
+type encoderState struct {
+	w                io.Writer
+	filling          []byte
+	current          []byte
+	previous         []byte
+	encoder          encoder
+	writing          *blockEnc
+	err              error
+	writeErr         error
+	nWritten         int64
+	headerWritten    bool
+	eofWritten       bool
+	fullFrameWritten bool
+
+	// This waitgroup indicates an encode is running.
+	wg sync.WaitGroup
+	// This waitgroup indicates we have a block encoding/writing.
+	wWg sync.WaitGroup
+}
+
+// NewWriter will create a new Zstandard encoder.
+// If the encoder will be used for encoding blocks a nil writer can be used.
+func NewWriter(w io.Writer, opts ...EOption) (*Encoder, error) {
+	initPredefined()
+	var e Encoder
+	e.o.setDefault()
+	for _, o := range opts {
+		err := o(&e.o)
+		if err != nil {
+			return nil, err
+		}
+	}
+	if w != nil {
+		e.Reset(w)
+	}
+	return &e, nil
+}
+
+func (e *Encoder) initialize() {
+	if e.o.concurrent == 0 {
+		e.o.setDefault()
+	}
+	e.encoders = make(chan encoder, e.o.concurrent)
+	for i := 0; i < e.o.concurrent; i++ {
+		enc := e.o.encoder()
+		e.encoders <- enc
+	}
+}
+
+// Reset will re-initialize the writer and new writes will encode to the supplied writer
+// as a new, independent stream.
+func (e *Encoder) Reset(w io.Writer) {
+	s := &e.state
+	s.wg.Wait()
+	s.wWg.Wait()
+	if cap(s.filling) == 0 {
+		s.filling = make([]byte, 0, e.o.blockSize)
+	}
+	if cap(s.current) == 0 {
+		s.current = make([]byte, 0, e.o.blockSize)
+	}
+	if cap(s.previous) == 0 {
+		s.previous = make([]byte, 0, e.o.blockSize)
+	}
+	if s.encoder == nil {
+		s.encoder = e.o.encoder()
+	}
+	if s.writing == nil {
+		s.writing = &blockEnc{lowMem: e.o.lowMem}
+		s.writing.init()
+	}
+	s.writing.initNewEncode()
+	s.filling = s.filling[:0]
+	s.current = s.current[:0]
+	s.previous = s.previous[:0]
+	s.encoder.Reset(e.o.dict, false)
+	s.headerWritten = false
+	s.eofWritten = false
+	s.fullFrameWritten = false
+	s.w = w
+	s.err = nil
+	s.nWritten = 0
+	s.writeErr = nil
+}
+
+// Write data to the encoder.
+// Input data will be buffered and as the buffer fills up
+// content will be compressed and written to the output.
+// When done writing, use Close to flush the remaining output
+// and write CRC if requested.
+func (e *Encoder) Write(p []byte) (n int, err error) {
+	s := &e.state
+	for len(p) > 0 {
+		if len(p)+len(s.filling) < e.o.blockSize {
+			if e.o.crc {
+				_, _ = s.encoder.CRC().Write(p)
+			}
+			s.filling = append(s.filling, p...)
+			return n + len(p), nil
+		}
+		add := p
+		if len(p)+len(s.filling) > e.o.blockSize {
+			add = add[:e.o.blockSize-len(s.filling)]
+		}
+		if e.o.crc {
+			_, _ = s.encoder.CRC().Write(add)
+		}
+		s.filling = append(s.filling, add...)
+		p = p[len(add):]
+		n += len(add)
+		if len(s.filling) < e.o.blockSize {
+			return n, nil
+		}
+		err := e.nextBlock(false)
+		if err != nil {
+			return n, err
+		}
+		if debugAsserts && len(s.filling) > 0 {
+			panic(len(s.filling))
+		}
+	}
+	return n, nil
+}
+
+// nextBlock will synchronize and start compressing input in e.state.filling.
+// If an error has occurred during encoding it will be returned.
+func (e *Encoder) nextBlock(final bool) error {
+	s := &e.state
+	// Wait for current block.
+	s.wg.Wait()
+	if s.err != nil {
+		return s.err
+	}
+	if len(s.filling) > e.o.blockSize {
+		return fmt.Errorf("block > maxStoreBlockSize")
+	}
+	if !s.headerWritten {
+		// If we have a single block encode, do a sync compression.
+		if final && len(s.filling) == 0 && !e.o.fullZero {
+			s.headerWritten = true
+			s.fullFrameWritten = true
+			s.eofWritten = true
+			return nil
+		}
+		if final && len(s.filling) > 0 {
+			s.current = e.EncodeAll(s.filling, s.current[:0])
+			var n2 int
+			n2, s.err = s.w.Write(s.current)
+			if s.err != nil {
+				return s.err
+			}
+			s.nWritten += int64(n2)
+			s.current = s.current[:0]
+			s.filling = s.filling[:0]
+			s.headerWritten = true
+			s.fullFrameWritten = true
+			s.eofWritten = true
+			return nil
+		}
+
+		var tmp [maxHeaderSize]byte
+		fh := frameHeader{
+			ContentSize:   0,
+			WindowSize:    uint32(s.encoder.WindowSize(0)),
+			SingleSegment: false,
+			Checksum:      e.o.crc,
+			DictID:        e.o.dict.ID(),
+		}
+
+		dst, err := fh.appendTo(tmp[:0])
+		if err != nil {
+			return err
+		}
+		s.headerWritten = true
+		s.wWg.Wait()
+		var n2 int
+		n2, s.err = s.w.Write(dst)
+		if s.err != nil {
+			return s.err
+		}
+		s.nWritten += int64(n2)
+	}
+	if s.eofWritten {
+		// Ensure we only write it once.
+		final = false
+	}
+
+	if len(s.filling) == 0 {
+		// Final block, but no data.
+		if final {
+			enc := s.encoder
+			blk := enc.Block()
+			blk.reset(nil)
+			blk.last = true
+			blk.encodeRaw(nil)
+			s.wWg.Wait()
+			_, s.err = s.w.Write(blk.output)
+			s.nWritten += int64(len(blk.output))
+			s.eofWritten = true
+		}
+		return s.err
+	}
+
+	// Move blocks forward.
+	s.filling, s.current, s.previous = s.previous[:0], s.filling, s.current
+	s.wg.Add(1)
+	go func(src []byte) {
+		if debug {
+			println("Adding block,", len(src), "bytes, final:", final)
+		}
+		defer func() {
+			if r := recover(); r != nil {
+				s.err = fmt.Errorf("panic while encoding: %v", r)
+				rdebug.PrintStack()
+			}
+			s.wg.Done()
+		}()
+		enc := s.encoder
+		blk := enc.Block()
+		enc.Encode(blk, src)
+		blk.last = final
+		if final {
+			s.eofWritten = true
+		}
+		// Wait for pending writes.
+		s.wWg.Wait()
+		if s.writeErr != nil {
+			s.err = s.writeErr
+			return
+		}
+		// Transfer encoders from previous write block.
+		blk.swapEncoders(s.writing)
+		// Transfer recent offsets to next.
+		enc.UseBlock(s.writing)
+		s.writing = blk
+		s.wWg.Add(1)
+		go func() {
+			defer func() {
+				if r := recover(); r != nil {
+					s.writeErr = fmt.Errorf("panic while encoding/writing: %v", r)
+					rdebug.PrintStack()
+				}
+				s.wWg.Done()
+			}()
+			err := errIncompressible
+			// If we got the exact same number of literals as input,
+			// assume the literals cannot be compressed.
+			if len(src) != len(blk.literals) || len(src) != e.o.blockSize {
+				err = blk.encode(src, e.o.noEntropy, !e.o.allLitEntropy)
+			}
+			switch err {
+			case errIncompressible:
+				if debug {
+					println("Storing incompressible block as raw")
+				}
+				blk.encodeRaw(src)
+				// In fast mode, we do not transfer offsets, so we don't have to deal with changing the.
+			case nil:
+			default:
+				s.writeErr = err
+				return
+			}
+			_, s.writeErr = s.w.Write(blk.output)
+			s.nWritten += int64(len(blk.output))
+		}()
+	}(s.current)
+	return nil
+}
+
+// ReadFrom reads data from r until EOF or error.
+// The return value n is the number of bytes read.
+// Any error except io.EOF encountered during the read is also returned.
+//
+// The Copy function uses ReaderFrom if available.
+func (e *Encoder) ReadFrom(r io.Reader) (n int64, err error) {
+	if debug {
+		println("Using ReadFrom")
+	}
+
+	// Flush any current writes.
+	if len(e.state.filling) > 0 {
+		if err := e.nextBlock(false); err != nil {
+			return 0, err
+		}
+	}
+	e.state.filling = e.state.filling[:e.o.blockSize]
+	src := e.state.filling
+	for {
+		n2, err := r.Read(src)
+		if e.o.crc {
+			_, _ = e.state.encoder.CRC().Write(src[:n2])
+		}
+		// src is now the unfilled part...
+		src = src[n2:]
+		n += int64(n2)
+		switch err {
+		case io.EOF:
+			e.state.filling = e.state.filling[:len(e.state.filling)-len(src)]
+			if debug {
+				println("ReadFrom: got EOF final block:", len(e.state.filling))
+			}
+			return n, nil
+		case nil:
+		default:
+			if debug {
+				println("ReadFrom: got error:", err)
+			}
+			e.state.err = err
+			return n, err
+		}
+		if len(src) > 0 {
+			if debug {
+				println("ReadFrom: got space left in source:", len(src))
+			}
+			continue
+		}
+		err = e.nextBlock(false)
+		if err != nil {
+			return n, err
+		}
+		e.state.filling = e.state.filling[:e.o.blockSize]
+		src = e.state.filling
+	}
+}
+
+// Flush will send the currently written data to output
+// and block until everything has been written.
+// This should only be used on rare occasions where pushing the currently queued data is critical.
+func (e *Encoder) Flush() error {
+	s := &e.state
+	if len(s.filling) > 0 {
+		err := e.nextBlock(false)
+		if err != nil {
+			return err
+		}
+	}
+	s.wg.Wait()
+	s.wWg.Wait()
+	if s.err != nil {
+		return s.err
+	}
+	return s.writeErr
+}
+
+// Close will flush the final output and close the stream.
+// The function will block until everything has been written.
+// The Encoder can still be re-used after calling this.
+func (e *Encoder) Close() error {
+	s := &e.state
+	if s.encoder == nil {
+		return nil
+	}
+	err := e.nextBlock(true)
+	if err != nil {
+		return err
+	}
+	if e.state.fullFrameWritten {
+		return s.err
+	}
+	s.wg.Wait()
+	s.wWg.Wait()
+
+	if s.err != nil {
+		return s.err
+	}
+	if s.writeErr != nil {
+		return s.writeErr
+	}
+
+	// Write CRC
+	if e.o.crc && s.err == nil {
+		// heap alloc.
+		var tmp [4]byte
+		_, s.err = s.w.Write(s.encoder.AppendCRC(tmp[:0]))
+		s.nWritten += 4
+	}
+
+	// Add padding with content from crypto/rand.Reader
+	if s.err == nil && e.o.pad > 0 {
+		add := calcSkippableFrame(s.nWritten, int64(e.o.pad))
+		frame, err := skippableFrame(s.filling[:0], add, rand.Reader)
+		if err != nil {
+			return err
+		}
+		_, s.err = s.w.Write(frame)
+	}
+	return s.err
+}
+
+// EncodeAll will encode all input in src and append it to dst.
+// This function can be called concurrently, but each call will only run on a single goroutine.
+// If empty input is given, nothing is returned, unless WithZeroFrames is specified.
+// Encoded blocks can be concatenated and the result will be the combined input stream.
+// Data compressed with EncodeAll can be decoded with the Decoder,
+// using either a stream or DecodeAll.
+func (e *Encoder) EncodeAll(src, dst []byte) []byte {
+	if len(src) == 0 {
+		if e.o.fullZero {
+			// Add frame header.
+			fh := frameHeader{
+				ContentSize:   0,
+				WindowSize:    MinWindowSize,
+				SingleSegment: true,
+				// Adding a checksum would be a waste of space.
+				Checksum: false,
+				DictID:   0,
+			}
+			dst, _ = fh.appendTo(dst)
+
+			// Write raw block as last one only.
+			var blk blockHeader
+			blk.setSize(0)
+			blk.setType(blockTypeRaw)
+			blk.setLast(true)
+			dst = blk.appendTo(dst)
+		}
+		return dst
+	}
+	e.init.Do(e.initialize)
+	enc := <-e.encoders
+	defer func() {
+		// Release encoder reference to last block.
+		// If a non-single block is needed the encoder will reset again.
+		e.encoders <- enc
+	}()
+	// Use single segments when above minimum window and below 1MB.
+	single := len(src) < 1<<20 && len(src) > MinWindowSize
+	if e.o.single != nil {
+		single = *e.o.single
+	}
+	fh := frameHeader{
+		ContentSize:   uint64(len(src)),
+		WindowSize:    uint32(enc.WindowSize(len(src))),
+		SingleSegment: single,
+		Checksum:      e.o.crc,
+		DictID:        e.o.dict.ID(),
+	}
+
+	// If less than 1MB, allocate a buffer up front.
+	if len(dst) == 0 && cap(dst) == 0 && len(src) < 1<<20 && !e.o.lowMem {
+		dst = make([]byte, 0, len(src))
+	}
+	dst, err := fh.appendTo(dst)
+	if err != nil {
+		panic(err)
+	}
+
+	// If we can do everything in one block, prefer that.
+	if len(src) <= maxCompressedBlockSize {
+		enc.Reset(e.o.dict, true)
+		// Slightly faster with no history and everything in one block.
+		if e.o.crc {
+			_, _ = enc.CRC().Write(src)
+		}
+		blk := enc.Block()
+		blk.last = true
+		if e.o.dict == nil {
+			enc.EncodeNoHist(blk, src)
+		} else {
+			enc.Encode(blk, src)
+		}
+
+		// If we got the exact same number of literals as input,
+		// assume the literals cannot be compressed.
+		err := errIncompressible
+		oldout := blk.output
+		if len(blk.literals) != len(src) || len(src) != e.o.blockSize {
+			// Output directly to dst
+			blk.output = dst
+			err = blk.encode(src, e.o.noEntropy, !e.o.allLitEntropy)
+		}
+
+		switch err {
+		case errIncompressible:
+			if debug {
+				println("Storing incompressible block as raw")
+			}
+			dst = blk.encodeRawTo(dst, src)
+		case nil:
+			dst = blk.output
+		default:
+			panic(err)
+		}
+		blk.output = oldout
+	} else {
+		enc.Reset(e.o.dict, false)
+		blk := enc.Block()
+		for len(src) > 0 {
+			todo := src
+			if len(todo) > e.o.blockSize {
+				todo = todo[:e.o.blockSize]
+			}
+			src = src[len(todo):]
+			if e.o.crc {
+				_, _ = enc.CRC().Write(todo)
+			}
+			blk.pushOffsets()
+			enc.Encode(blk, todo)
+			if len(src) == 0 {
+				blk.last = true
+			}
+			err := errIncompressible
+			// If we got the exact same number of literals as input,
+			// assume the literals cannot be compressed.
+			if len(blk.literals) != len(todo) || len(todo) != e.o.blockSize {
+				err = blk.encode(todo, e.o.noEntropy, !e.o.allLitEntropy)
+			}
+
+			switch err {
+			case errIncompressible:
+				if debug {
+					println("Storing incompressible block as raw")
+				}
+				dst = blk.encodeRawTo(dst, todo)
+				blk.popOffsets()
+			case nil:
+				dst = append(dst, blk.output...)
+			default:
+				panic(err)
+			}
+			blk.reset(nil)
+		}
+	}
+	if e.o.crc {
+		dst = enc.AppendCRC(dst)
+	}
+	// Add padding with content from crypto/rand.Reader
+	if e.o.pad > 0 {
+		add := calcSkippableFrame(int64(len(dst)), int64(e.o.pad))
+		dst, err = skippableFrame(dst, add, rand.Reader)
+		if err != nil {
+			panic(err)
+		}
+	}
+	return dst
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/encoder_options.go b/vendor/github.com/klauspost/compress/zstd/encoder_options.go
new file mode 100644
index 0000000..16d4ab6
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/encoder_options.go
@@ -0,0 +1,312 @@
+package zstd
+
+import (
+	"errors"
+	"fmt"
+	"runtime"
+	"strings"
+)
+
+// EOption is an option for creating a encoder.
+type EOption func(*encoderOptions) error
+
+// options retains accumulated state of multiple options.
+type encoderOptions struct {
+	concurrent      int
+	level           EncoderLevel
+	single          *bool
+	pad             int
+	blockSize       int
+	windowSize      int
+	crc             bool
+	fullZero        bool
+	noEntropy       bool
+	allLitEntropy   bool
+	customWindow    bool
+	customALEntropy bool
+	lowMem          bool
+	dict            *dict
+}
+
+func (o *encoderOptions) setDefault() {
+	*o = encoderOptions{
+		concurrent:    runtime.GOMAXPROCS(0),
+		crc:           true,
+		single:        nil,
+		blockSize:     1 << 16,
+		windowSize:    8 << 20,
+		level:         SpeedDefault,
+		allLitEntropy: true,
+		lowMem:        false,
+	}
+}
+
+// encoder returns an encoder with the selected options.
+func (o encoderOptions) encoder() encoder {
+	switch o.level {
+	case SpeedFastest:
+		if o.dict != nil {
+			return &fastEncoderDict{fastEncoder: fastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}}
+		}
+		return &fastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}
+
+	case SpeedDefault:
+		if o.dict != nil {
+			return &doubleFastEncoderDict{fastEncoderDict: fastEncoderDict{fastEncoder: fastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}}}
+		}
+		return &doubleFastEncoder{fastEncoder: fastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}}
+	case SpeedBetterCompression:
+		if o.dict != nil {
+			return &betterFastEncoderDict{betterFastEncoder: betterFastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}}
+		}
+		return &betterFastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}
+	case SpeedBestCompression:
+		return &bestFastEncoder{fastBase: fastBase{maxMatchOff: int32(o.windowSize), lowMem: o.lowMem}}
+	}
+	panic("unknown compression level")
+}
+
+// WithEncoderCRC will add CRC value to output.
+// Output will be 4 bytes larger.
+func WithEncoderCRC(b bool) EOption {
+	return func(o *encoderOptions) error { o.crc = b; return nil }
+}
+
+// WithEncoderConcurrency will set the concurrency,
+// meaning the maximum number of encoders to run concurrently.
+// The value supplied must be at least 1.
+// By default this will be set to GOMAXPROCS.
+func WithEncoderConcurrency(n int) EOption {
+	return func(o *encoderOptions) error {
+		if n <= 0 {
+			return fmt.Errorf("concurrency must be at least 1")
+		}
+		o.concurrent = n
+		return nil
+	}
+}
+
+// WithWindowSize will set the maximum allowed back-reference distance.
+// The value must be a power of two between MinWindowSize and MaxWindowSize.
+// A larger value will enable better compression but allocate more memory and,
+// for above-default values, take considerably longer.
+// The default value is determined by the compression level.
+func WithWindowSize(n int) EOption {
+	return func(o *encoderOptions) error {
+		switch {
+		case n < MinWindowSize:
+			return fmt.Errorf("window size must be at least %d", MinWindowSize)
+		case n > MaxWindowSize:
+			return fmt.Errorf("window size must be at most %d", MaxWindowSize)
+		case (n & (n - 1)) != 0:
+			return errors.New("window size must be a power of 2")
+		}
+
+		o.windowSize = n
+		o.customWindow = true
+		if o.blockSize > o.windowSize {
+			o.blockSize = o.windowSize
+		}
+		return nil
+	}
+}
+
+// WithEncoderPadding will add padding to all output so the size will be a multiple of n.
+// This can be used to obfuscate the exact output size or make blocks of a certain size.
+// The contents will be a skippable frame, so it will be invisible by the decoder.
+// n must be > 0 and <= 1GB, 1<<30 bytes.
+// The padded area will be filled with data from crypto/rand.Reader.
+// If `EncodeAll` is used with data already in the destination, the total size will be multiple of this.
+func WithEncoderPadding(n int) EOption {
+	return func(o *encoderOptions) error {
+		if n <= 0 {
+			return fmt.Errorf("padding must be at least 1")
+		}
+		// No need to waste our time.
+		if n == 1 {
+			o.pad = 0
+		}
+		if n > 1<<30 {
+			return fmt.Errorf("padding must less than 1GB (1<<30 bytes) ")
+		}
+		o.pad = n
+		return nil
+	}
+}
+
+// EncoderLevel predefines encoder compression levels.
+// Only use the constants made available, since the actual mapping
+// of these values are very likely to change and your compression could change
+// unpredictably when upgrading the library.
+type EncoderLevel int
+
+const (
+	speedNotSet EncoderLevel = iota
+
+	// SpeedFastest will choose the fastest reasonable compression.
+	// This is roughly equivalent to the fastest Zstandard mode.
+	SpeedFastest
+
+	// SpeedDefault is the default "pretty fast" compression option.
+	// This is roughly equivalent to the default Zstandard mode (level 3).
+	SpeedDefault
+
+	// SpeedBetterCompression will yield better compression than the default.
+	// Currently it is about zstd level 7-8 with ~ 2x-3x the default CPU usage.
+	// By using this, notice that CPU usage may go up in the future.
+	SpeedBetterCompression
+
+	// SpeedBestCompression will choose the best available compression option.
+	// This will offer the best compression no matter the CPU cost.
+	SpeedBestCompression
+
+	// speedLast should be kept as the last actual compression option.
+	// The is not for external usage, but is used to keep track of the valid options.
+	speedLast
+)
+
+// EncoderLevelFromString will convert a string representation of an encoding level back
+// to a compression level. The compare is not case sensitive.
+// If the string wasn't recognized, (false, SpeedDefault) will be returned.
+func EncoderLevelFromString(s string) (bool, EncoderLevel) {
+	for l := speedNotSet + 1; l < speedLast; l++ {
+		if strings.EqualFold(s, l.String()) {
+			return true, l
+		}
+	}
+	return false, SpeedDefault
+}
+
+// EncoderLevelFromZstd will return an encoder level that closest matches the compression
+// ratio of a specific zstd compression level.
+// Many input values will provide the same compression level.
+func EncoderLevelFromZstd(level int) EncoderLevel {
+	switch {
+	case level < 3:
+		return SpeedFastest
+	case level >= 3 && level < 6:
+		return SpeedDefault
+	case level >= 6 && level < 10:
+		return SpeedBetterCompression
+	case level >= 10:
+		return SpeedBetterCompression
+	}
+	return SpeedDefault
+}
+
+// String provides a string representation of the compression level.
+func (e EncoderLevel) String() string {
+	switch e {
+	case SpeedFastest:
+		return "fastest"
+	case SpeedDefault:
+		return "default"
+	case SpeedBetterCompression:
+		return "better"
+	case SpeedBestCompression:
+		return "best"
+	default:
+		return "invalid"
+	}
+}
+
+// WithEncoderLevel specifies a predefined compression level.
+func WithEncoderLevel(l EncoderLevel) EOption {
+	return func(o *encoderOptions) error {
+		switch {
+		case l <= speedNotSet || l >= speedLast:
+			return fmt.Errorf("unknown encoder level")
+		}
+		o.level = l
+		if !o.customWindow {
+			switch o.level {
+			case SpeedFastest:
+				o.windowSize = 4 << 20
+			case SpeedDefault:
+				o.windowSize = 8 << 20
+			case SpeedBetterCompression:
+				o.windowSize = 16 << 20
+			case SpeedBestCompression:
+				o.windowSize = 32 << 20
+			}
+		}
+		if !o.customALEntropy {
+			o.allLitEntropy = l > SpeedFastest
+		}
+
+		return nil
+	}
+}
+
+// WithZeroFrames will encode 0 length input as full frames.
+// This can be needed for compatibility with zstandard usage,
+// but is not needed for this package.
+func WithZeroFrames(b bool) EOption {
+	return func(o *encoderOptions) error {
+		o.fullZero = b
+		return nil
+	}
+}
+
+// WithAllLitEntropyCompression will apply entropy compression if no matches are found.
+// Disabling this will skip incompressible data faster, but in cases with no matches but
+// skewed character distribution compression is lost.
+// Default value depends on the compression level selected.
+func WithAllLitEntropyCompression(b bool) EOption {
+	return func(o *encoderOptions) error {
+		o.customALEntropy = true
+		o.allLitEntropy = b
+		return nil
+	}
+}
+
+// WithNoEntropyCompression will always skip entropy compression of literals.
+// This can be useful if content has matches, but unlikely to benefit from entropy
+// compression. Usually the slight speed improvement is not worth enabling this.
+func WithNoEntropyCompression(b bool) EOption {
+	return func(o *encoderOptions) error {
+		o.noEntropy = b
+		return nil
+	}
+}
+
+// WithSingleSegment will set the "single segment" flag when EncodeAll is used.
+// If this flag is set, data must be regenerated within a single continuous memory segment.
+// In this case, Window_Descriptor byte is skipped, but Frame_Content_Size is necessarily present.
+// As a consequence, the decoder must allocate a memory segment of size equal or larger than size of your content.
+// In order to preserve the decoder from unreasonable memory requirements,
+// a decoder is allowed to reject a compressed frame which requests a memory size beyond decoder's authorized range.
+// For broader compatibility, decoders are recommended to support memory sizes of at least 8 MB.
+// This is only a recommendation, each decoder is free to support higher or lower limits, depending on local limitations.
+// If this is not specified, block encodes will automatically choose this based on the input size.
+// This setting has no effect on streamed encodes.
+func WithSingleSegment(b bool) EOption {
+	return func(o *encoderOptions) error {
+		o.single = &b
+		return nil
+	}
+}
+
+// WithLowerEncoderMem will trade in some memory cases trade less memory usage for
+// slower encoding speed.
+// This will not change the window size which is the primary function for reducing
+// memory usage. See WithWindowSize.
+func WithLowerEncoderMem(b bool) EOption {
+	return func(o *encoderOptions) error {
+		o.lowMem = b
+		return nil
+	}
+}
+
+// WithEncoderDict allows to register a dictionary that will be used for the encode.
+// The encoder *may* choose to use no dictionary instead for certain payloads.
+func WithEncoderDict(dict []byte) EOption {
+	return func(o *encoderOptions) error {
+		d, err := loadDict(dict)
+		if err != nil {
+			return err
+		}
+		o.dict = d
+		return nil
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/framedec.go b/vendor/github.com/klauspost/compress/zstd/framedec.go
new file mode 100644
index 0000000..693c5f0
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/framedec.go
@@ -0,0 +1,494 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"bytes"
+	"encoding/hex"
+	"errors"
+	"hash"
+	"io"
+	"sync"
+
+	"github.com/klauspost/compress/zstd/internal/xxhash"
+)
+
+type frameDec struct {
+	o      decoderOptions
+	crc    hash.Hash64
+	offset int64
+
+	WindowSize uint64
+
+	// maxWindowSize is the maximum windows size to support.
+	// should never be bigger than max-int.
+	maxWindowSize uint64
+
+	// In order queue of blocks being decoded.
+	decoding chan *blockDec
+
+	// Frame history passed between blocks
+	history history
+
+	rawInput byteBuffer
+
+	// Byte buffer that can be reused for small input blocks.
+	bBuf byteBuf
+
+	FrameContentSize uint64
+	frameDone        sync.WaitGroup
+
+	DictionaryID  *uint32
+	HasCheckSum   bool
+	SingleSegment bool
+
+	// asyncRunning indicates whether the async routine processes input on 'decoding'.
+	asyncRunningMu sync.Mutex
+	asyncRunning   bool
+}
+
+const (
+	// The minimum Window_Size is 1 KB.
+	MinWindowSize = 1 << 10
+	MaxWindowSize = 1 << 29
+)
+
+var (
+	frameMagic          = []byte{0x28, 0xb5, 0x2f, 0xfd}
+	skippableFrameMagic = []byte{0x2a, 0x4d, 0x18}
+)
+
+func newFrameDec(o decoderOptions) *frameDec {
+	d := frameDec{
+		o:             o,
+		maxWindowSize: MaxWindowSize,
+	}
+	if d.maxWindowSize > o.maxDecodedSize {
+		d.maxWindowSize = o.maxDecodedSize
+	}
+	return &d
+}
+
+// reset will read the frame header and prepare for block decoding.
+// If nothing can be read from the input, io.EOF will be returned.
+// Any other error indicated that the stream contained data, but
+// there was a problem.
+func (d *frameDec) reset(br byteBuffer) error {
+	d.HasCheckSum = false
+	d.WindowSize = 0
+	var b []byte
+	for {
+		b = br.readSmall(4)
+		if b == nil {
+			return io.EOF
+		}
+		if !bytes.Equal(b[1:4], skippableFrameMagic) || b[0]&0xf0 != 0x50 {
+			if debug {
+				println("Not skippable", hex.EncodeToString(b), hex.EncodeToString(skippableFrameMagic))
+			}
+			// Break if not skippable frame.
+			break
+		}
+		// Read size to skip
+		b = br.readSmall(4)
+		if b == nil {
+			println("Reading Frame Size EOF")
+			return io.ErrUnexpectedEOF
+		}
+		n := uint32(b[0]) | (uint32(b[1]) << 8) | (uint32(b[2]) << 16) | (uint32(b[3]) << 24)
+		println("Skipping frame with", n, "bytes.")
+		err := br.skipN(int(n))
+		if err != nil {
+			if debug {
+				println("Reading discarded frame", err)
+			}
+			return err
+		}
+	}
+	if !bytes.Equal(b, frameMagic) {
+		println("Got magic numbers: ", b, "want:", frameMagic)
+		return ErrMagicMismatch
+	}
+
+	// Read Frame_Header_Descriptor
+	fhd, err := br.readByte()
+	if err != nil {
+		println("Reading Frame_Header_Descriptor", err)
+		return err
+	}
+	d.SingleSegment = fhd&(1<<5) != 0
+
+	if fhd&(1<<3) != 0 {
+		return errors.New("reserved bit set on frame header")
+	}
+
+	// Read Window_Descriptor
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#window_descriptor
+	d.WindowSize = 0
+	if !d.SingleSegment {
+		wd, err := br.readByte()
+		if err != nil {
+			println("Reading Window_Descriptor", err)
+			return err
+		}
+		printf("raw: %x, mantissa: %d, exponent: %d\n", wd, wd&7, wd>>3)
+		windowLog := 10 + (wd >> 3)
+		windowBase := uint64(1) << windowLog
+		windowAdd := (windowBase / 8) * uint64(wd&0x7)
+		d.WindowSize = windowBase + windowAdd
+	}
+
+	// Read Dictionary_ID
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#dictionary_id
+	d.DictionaryID = nil
+	if size := fhd & 3; size != 0 {
+		if size == 3 {
+			size = 4
+		}
+		b = br.readSmall(int(size))
+		if b == nil {
+			if debug {
+				println("Reading Dictionary_ID", io.ErrUnexpectedEOF)
+			}
+			return io.ErrUnexpectedEOF
+		}
+		var id uint32
+		switch size {
+		case 1:
+			id = uint32(b[0])
+		case 2:
+			id = uint32(b[0]) | (uint32(b[1]) << 8)
+		case 4:
+			id = uint32(b[0]) | (uint32(b[1]) << 8) | (uint32(b[2]) << 16) | (uint32(b[3]) << 24)
+		}
+		if debug {
+			println("Dict size", size, "ID:", id)
+		}
+		if id > 0 {
+			// ID 0 means "sorry, no dictionary anyway".
+			// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#dictionary-format
+			d.DictionaryID = &id
+		}
+	}
+
+	// Read Frame_Content_Size
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#frame_content_size
+	var fcsSize int
+	v := fhd >> 6
+	switch v {
+	case 0:
+		if d.SingleSegment {
+			fcsSize = 1
+		}
+	default:
+		fcsSize = 1 << v
+	}
+	d.FrameContentSize = 0
+	if fcsSize > 0 {
+		b := br.readSmall(fcsSize)
+		if b == nil {
+			println("Reading Frame content", io.ErrUnexpectedEOF)
+			return io.ErrUnexpectedEOF
+		}
+		switch fcsSize {
+		case 1:
+			d.FrameContentSize = uint64(b[0])
+		case 2:
+			// When FCS_Field_Size is 2, the offset of 256 is added.
+			d.FrameContentSize = uint64(b[0]) | (uint64(b[1]) << 8) + 256
+		case 4:
+			d.FrameContentSize = uint64(b[0]) | (uint64(b[1]) << 8) | (uint64(b[2]) << 16) | (uint64(b[3]) << 24)
+		case 8:
+			d1 := uint32(b[0]) | (uint32(b[1]) << 8) | (uint32(b[2]) << 16) | (uint32(b[3]) << 24)
+			d2 := uint32(b[4]) | (uint32(b[5]) << 8) | (uint32(b[6]) << 16) | (uint32(b[7]) << 24)
+			d.FrameContentSize = uint64(d1) | (uint64(d2) << 32)
+		}
+		if debug {
+			println("field size bits:", v, "fcsSize:", fcsSize, "FrameContentSize:", d.FrameContentSize, hex.EncodeToString(b[:fcsSize]), "singleseg:", d.SingleSegment, "window:", d.WindowSize)
+		}
+	}
+	// Move this to shared.
+	d.HasCheckSum = fhd&(1<<2) != 0
+	if d.HasCheckSum {
+		if d.crc == nil {
+			d.crc = xxhash.New()
+		}
+		d.crc.Reset()
+	}
+
+	if d.WindowSize == 0 && d.SingleSegment {
+		// We may not need window in this case.
+		d.WindowSize = d.FrameContentSize
+		if d.WindowSize < MinWindowSize {
+			d.WindowSize = MinWindowSize
+		}
+	}
+
+	if d.WindowSize > d.maxWindowSize {
+		printf("window size %d > max %d\n", d.WindowSize, d.maxWindowSize)
+		return ErrWindowSizeExceeded
+	}
+	// The minimum Window_Size is 1 KB.
+	if d.WindowSize < MinWindowSize {
+		println("got window size: ", d.WindowSize)
+		return ErrWindowSizeTooSmall
+	}
+	d.history.windowSize = int(d.WindowSize)
+	if d.o.lowMem && d.history.windowSize < maxBlockSize {
+		d.history.maxSize = d.history.windowSize * 2
+	} else {
+		d.history.maxSize = d.history.windowSize + maxBlockSize
+	}
+	// history contains input - maybe we do something
+	d.rawInput = br
+	return nil
+}
+
+// next will start decoding the next block from stream.
+func (d *frameDec) next(block *blockDec) error {
+	if debug {
+		printf("decoding new block %p:%p", block, block.data)
+	}
+	err := block.reset(d.rawInput, d.WindowSize)
+	if err != nil {
+		println("block error:", err)
+		// Signal the frame decoder we have a problem.
+		d.sendErr(block, err)
+		return err
+	}
+	block.input <- struct{}{}
+	if debug {
+		println("next block:", block)
+	}
+	d.asyncRunningMu.Lock()
+	defer d.asyncRunningMu.Unlock()
+	if !d.asyncRunning {
+		return nil
+	}
+	if block.Last {
+		// We indicate the frame is done by sending io.EOF
+		d.decoding <- block
+		return io.EOF
+	}
+	d.decoding <- block
+	return nil
+}
+
+// sendEOF will queue an error block on the frame.
+// This will cause the frame decoder to return when it encounters the block.
+// Returns true if the decoder was added.
+func (d *frameDec) sendErr(block *blockDec, err error) bool {
+	d.asyncRunningMu.Lock()
+	defer d.asyncRunningMu.Unlock()
+	if !d.asyncRunning {
+		return false
+	}
+
+	println("sending error", err.Error())
+	block.sendErr(err)
+	d.decoding <- block
+	return true
+}
+
+// checkCRC will check the checksum if the frame has one.
+// Will return ErrCRCMismatch if crc check failed, otherwise nil.
+func (d *frameDec) checkCRC() error {
+	if !d.HasCheckSum {
+		return nil
+	}
+	var tmp [4]byte
+	got := d.crc.Sum64()
+	// Flip to match file order.
+	tmp[0] = byte(got >> 0)
+	tmp[1] = byte(got >> 8)
+	tmp[2] = byte(got >> 16)
+	tmp[3] = byte(got >> 24)
+
+	// We can overwrite upper tmp now
+	want := d.rawInput.readSmall(4)
+	if want == nil {
+		println("CRC missing?")
+		return io.ErrUnexpectedEOF
+	}
+
+	if !bytes.Equal(tmp[:], want) {
+		if debug {
+			println("CRC Check Failed:", tmp[:], "!=", want)
+		}
+		return ErrCRCMismatch
+	}
+	if debug {
+		println("CRC ok", tmp[:])
+	}
+	return nil
+}
+
+func (d *frameDec) initAsync() {
+	if !d.o.lowMem && !d.SingleSegment {
+		// set max extra size history to 10MB.
+		d.history.maxSize = d.history.windowSize + maxBlockSize*5
+	}
+	// re-alloc if more than one extra block size.
+	if d.o.lowMem && cap(d.history.b) > d.history.maxSize+maxBlockSize {
+		d.history.b = make([]byte, 0, d.history.maxSize)
+	}
+	if cap(d.history.b) < d.history.maxSize {
+		d.history.b = make([]byte, 0, d.history.maxSize)
+	}
+	if cap(d.decoding) < d.o.concurrent {
+		d.decoding = make(chan *blockDec, d.o.concurrent)
+	}
+	if debug {
+		h := d.history
+		printf("history init. len: %d, cap: %d", len(h.b), cap(h.b))
+	}
+	d.asyncRunningMu.Lock()
+	d.asyncRunning = true
+	d.asyncRunningMu.Unlock()
+}
+
+// startDecoder will start decoding blocks and write them to the writer.
+// The decoder will stop as soon as an error occurs or at end of frame.
+// When the frame has finished decoding the *bufio.Reader
+// containing the remaining input will be sent on frameDec.frameDone.
+func (d *frameDec) startDecoder(output chan decodeOutput) {
+	written := int64(0)
+
+	defer func() {
+		d.asyncRunningMu.Lock()
+		d.asyncRunning = false
+		d.asyncRunningMu.Unlock()
+
+		// Drain the currently decoding.
+		d.history.error = true
+	flushdone:
+		for {
+			select {
+			case b := <-d.decoding:
+				b.history <- &d.history
+				output <- <-b.result
+			default:
+				break flushdone
+			}
+		}
+		println("frame decoder done, signalling done")
+		d.frameDone.Done()
+	}()
+	// Get decoder for first block.
+	block := <-d.decoding
+	block.history <- &d.history
+	for {
+		var next *blockDec
+		// Get result
+		r := <-block.result
+		if r.err != nil {
+			println("Result contained error", r.err)
+			output <- r
+			return
+		}
+		if debug {
+			println("got result, from ", d.offset, "to", d.offset+int64(len(r.b)))
+			d.offset += int64(len(r.b))
+		}
+		if !block.Last {
+			// Send history to next block
+			select {
+			case next = <-d.decoding:
+				if debug {
+					println("Sending ", len(d.history.b), "bytes as history")
+				}
+				next.history <- &d.history
+			default:
+				// Wait until we have sent the block, so
+				// other decoders can potentially get the decoder.
+				next = nil
+			}
+		}
+
+		// Add checksum, async to decoding.
+		if d.HasCheckSum {
+			n, err := d.crc.Write(r.b)
+			if err != nil {
+				r.err = err
+				if n != len(r.b) {
+					r.err = io.ErrShortWrite
+				}
+				output <- r
+				return
+			}
+		}
+		written += int64(len(r.b))
+		if d.SingleSegment && uint64(written) > d.FrameContentSize {
+			println("runDecoder: single segment and", uint64(written), ">", d.FrameContentSize)
+			r.err = ErrFrameSizeExceeded
+			output <- r
+			return
+		}
+		if block.Last {
+			r.err = d.checkCRC()
+			output <- r
+			return
+		}
+		output <- r
+		if next == nil {
+			// There was no decoder available, we wait for one now that we have sent to the writer.
+			if debug {
+				println("Sending ", len(d.history.b), " bytes as history")
+			}
+			next = <-d.decoding
+			next.history <- &d.history
+		}
+		block = next
+	}
+}
+
+// runDecoder will create a sync decoder that will decode a block of data.
+func (d *frameDec) runDecoder(dst []byte, dec *blockDec) ([]byte, error) {
+	saved := d.history.b
+
+	// We use the history for output to avoid copying it.
+	d.history.b = dst
+	// Store input length, so we only check new data.
+	crcStart := len(dst)
+	var err error
+	for {
+		err = dec.reset(d.rawInput, d.WindowSize)
+		if err != nil {
+			break
+		}
+		if debug {
+			println("next block:", dec)
+		}
+		err = dec.decodeBuf(&d.history)
+		if err != nil || dec.Last {
+			break
+		}
+		if uint64(len(d.history.b)) > d.o.maxDecodedSize {
+			err = ErrDecoderSizeExceeded
+			break
+		}
+		if d.SingleSegment && uint64(len(d.history.b)) > d.o.maxDecodedSize {
+			println("runDecoder: single segment and", uint64(len(d.history.b)), ">", d.o.maxDecodedSize)
+			err = ErrFrameSizeExceeded
+			break
+		}
+	}
+	dst = d.history.b
+	if err == nil {
+		if d.HasCheckSum {
+			var n int
+			n, err = d.crc.Write(dst[crcStart:])
+			if err == nil {
+				if n != len(dst)-crcStart {
+					err = io.ErrShortWrite
+				} else {
+					err = d.checkCRC()
+				}
+			}
+		}
+	}
+	d.history.b = saved
+	return dst, err
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/frameenc.go b/vendor/github.com/klauspost/compress/zstd/frameenc.go
new file mode 100644
index 0000000..4ef7f5a
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/frameenc.go
@@ -0,0 +1,137 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"encoding/binary"
+	"fmt"
+	"io"
+	"math"
+	"math/bits"
+)
+
+type frameHeader struct {
+	ContentSize   uint64
+	WindowSize    uint32
+	SingleSegment bool
+	Checksum      bool
+	DictID        uint32
+}
+
+const maxHeaderSize = 14
+
+func (f frameHeader) appendTo(dst []byte) ([]byte, error) {
+	dst = append(dst, frameMagic...)
+	var fhd uint8
+	if f.Checksum {
+		fhd |= 1 << 2
+	}
+	if f.SingleSegment {
+		fhd |= 1 << 5
+	}
+
+	var dictIDContent []byte
+	if f.DictID > 0 {
+		var tmp [4]byte
+		if f.DictID < 256 {
+			fhd |= 1
+			tmp[0] = uint8(f.DictID)
+			dictIDContent = tmp[:1]
+		} else if f.DictID < 1<<16 {
+			fhd |= 2
+			binary.LittleEndian.PutUint16(tmp[:2], uint16(f.DictID))
+			dictIDContent = tmp[:2]
+		} else {
+			fhd |= 3
+			binary.LittleEndian.PutUint32(tmp[:4], f.DictID)
+			dictIDContent = tmp[:4]
+		}
+	}
+	var fcs uint8
+	if f.ContentSize >= 256 {
+		fcs++
+	}
+	if f.ContentSize >= 65536+256 {
+		fcs++
+	}
+	if f.ContentSize >= 0xffffffff {
+		fcs++
+	}
+
+	fhd |= fcs << 6
+
+	dst = append(dst, fhd)
+	if !f.SingleSegment {
+		const winLogMin = 10
+		windowLog := (bits.Len32(f.WindowSize-1) - winLogMin) << 3
+		dst = append(dst, uint8(windowLog))
+	}
+	if f.DictID > 0 {
+		dst = append(dst, dictIDContent...)
+	}
+	switch fcs {
+	case 0:
+		if f.SingleSegment {
+			dst = append(dst, uint8(f.ContentSize))
+		}
+		// Unless SingleSegment is set, framessizes < 256 are nto stored.
+	case 1:
+		f.ContentSize -= 256
+		dst = append(dst, uint8(f.ContentSize), uint8(f.ContentSize>>8))
+	case 2:
+		dst = append(dst, uint8(f.ContentSize), uint8(f.ContentSize>>8), uint8(f.ContentSize>>16), uint8(f.ContentSize>>24))
+	case 3:
+		dst = append(dst, uint8(f.ContentSize), uint8(f.ContentSize>>8), uint8(f.ContentSize>>16), uint8(f.ContentSize>>24),
+			uint8(f.ContentSize>>32), uint8(f.ContentSize>>40), uint8(f.ContentSize>>48), uint8(f.ContentSize>>56))
+	default:
+		panic("invalid fcs")
+	}
+	return dst, nil
+}
+
+const skippableFrameHeader = 4 + 4
+
+// calcSkippableFrame will return a total size to be added for written
+// to be divisible by multiple.
+// The value will always be > skippableFrameHeader.
+// The function will panic if written < 0 or wantMultiple <= 0.
+func calcSkippableFrame(written, wantMultiple int64) int {
+	if wantMultiple <= 0 {
+		panic("wantMultiple <= 0")
+	}
+	if written < 0 {
+		panic("written < 0")
+	}
+	leftOver := written % wantMultiple
+	if leftOver == 0 {
+		return 0
+	}
+	toAdd := wantMultiple - leftOver
+	for toAdd < skippableFrameHeader {
+		toAdd += wantMultiple
+	}
+	return int(toAdd)
+}
+
+// skippableFrame will add a skippable frame with a total size of bytes.
+// total should be >= skippableFrameHeader and < math.MaxUint32.
+func skippableFrame(dst []byte, total int, r io.Reader) ([]byte, error) {
+	if total == 0 {
+		return dst, nil
+	}
+	if total < skippableFrameHeader {
+		return dst, fmt.Errorf("requested skippable frame (%d) < 8", total)
+	}
+	if int64(total) > math.MaxUint32 {
+		return dst, fmt.Errorf("requested skippable frame (%d) > max uint32", total)
+	}
+	dst = append(dst, 0x50, 0x2a, 0x4d, 0x18)
+	f := uint32(total - skippableFrameHeader)
+	dst = append(dst, uint8(f), uint8(f>>8), uint8(f>>16), uint8(f>>24))
+	start := len(dst)
+	dst = append(dst, make([]byte, f)...)
+	_, err := io.ReadFull(r, dst[start:])
+	return dst, err
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/fse_decoder.go b/vendor/github.com/klauspost/compress/zstd/fse_decoder.go
new file mode 100644
index 0000000..e6d3d49
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/fse_decoder.go
@@ -0,0 +1,385 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"fmt"
+)
+
+const (
+	tablelogAbsoluteMax = 9
+)
+
+const (
+	/*!MEMORY_USAGE :
+	 *  Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.)
+	 *  Increasing memory usage improves compression ratio
+	 *  Reduced memory usage can improve speed, due to cache effect
+	 *  Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */
+	maxMemoryUsage = tablelogAbsoluteMax + 2
+
+	maxTableLog    = maxMemoryUsage - 2
+	maxTablesize   = 1 << maxTableLog
+	maxTableMask   = (1 << maxTableLog) - 1
+	minTablelog    = 5
+	maxSymbolValue = 255
+)
+
+// fseDecoder provides temporary storage for compression and decompression.
+type fseDecoder struct {
+	dt             [maxTablesize]decSymbol // Decompression table.
+	symbolLen      uint16                  // Length of active part of the symbol table.
+	actualTableLog uint8                   // Selected tablelog.
+	maxBits        uint8                   // Maximum number of additional bits
+
+	// used for table creation to avoid allocations.
+	stateTable [256]uint16
+	norm       [maxSymbolValue + 1]int16
+	preDefined bool
+}
+
+// tableStep returns the next table index.
+func tableStep(tableSize uint32) uint32 {
+	return (tableSize >> 1) + (tableSize >> 3) + 3
+}
+
+// readNCount will read the symbol distribution so decoding tables can be constructed.
+func (s *fseDecoder) readNCount(b *byteReader, maxSymbol uint16) error {
+	var (
+		charnum   uint16
+		previous0 bool
+	)
+	if b.remain() < 4 {
+		return errors.New("input too small")
+	}
+	bitStream := b.Uint32NC()
+	nbBits := uint((bitStream & 0xF) + minTablelog) // extract tableLog
+	if nbBits > tablelogAbsoluteMax {
+		println("Invalid tablelog:", nbBits)
+		return errors.New("tableLog too large")
+	}
+	bitStream >>= 4
+	bitCount := uint(4)
+
+	s.actualTableLog = uint8(nbBits)
+	remaining := int32((1 << nbBits) + 1)
+	threshold := int32(1 << nbBits)
+	gotTotal := int32(0)
+	nbBits++
+
+	for remaining > 1 && charnum <= maxSymbol {
+		if previous0 {
+			//println("prev0")
+			n0 := charnum
+			for (bitStream & 0xFFFF) == 0xFFFF {
+				//println("24 x 0")
+				n0 += 24
+				if r := b.remain(); r > 5 {
+					b.advance(2)
+					// The check above should make sure we can read 32 bits
+					bitStream = b.Uint32NC() >> bitCount
+				} else {
+					// end of bit stream
+					bitStream >>= 16
+					bitCount += 16
+				}
+			}
+			//printf("bitstream: %d, 0b%b", bitStream&3, bitStream)
+			for (bitStream & 3) == 3 {
+				n0 += 3
+				bitStream >>= 2
+				bitCount += 2
+			}
+			n0 += uint16(bitStream & 3)
+			bitCount += 2
+
+			if n0 > maxSymbolValue {
+				return errors.New("maxSymbolValue too small")
+			}
+			//println("inserting ", n0-charnum, "zeroes from idx", charnum, "ending before", n0)
+			for charnum < n0 {
+				s.norm[uint8(charnum)] = 0
+				charnum++
+			}
+
+			if r := b.remain(); r >= 7 || r-int(bitCount>>3) >= 4 {
+				b.advance(bitCount >> 3)
+				bitCount &= 7
+				// The check above should make sure we can read 32 bits
+				bitStream = b.Uint32NC() >> bitCount
+			} else {
+				bitStream >>= 2
+			}
+		}
+
+		max := (2*threshold - 1) - remaining
+		var count int32
+
+		if int32(bitStream)&(threshold-1) < max {
+			count = int32(bitStream) & (threshold - 1)
+			if debugAsserts && nbBits < 1 {
+				panic("nbBits underflow")
+			}
+			bitCount += nbBits - 1
+		} else {
+			count = int32(bitStream) & (2*threshold - 1)
+			if count >= threshold {
+				count -= max
+			}
+			bitCount += nbBits
+		}
+
+		// extra accuracy
+		count--
+		if count < 0 {
+			// -1 means +1
+			remaining += count
+			gotTotal -= count
+		} else {
+			remaining -= count
+			gotTotal += count
+		}
+		s.norm[charnum&0xff] = int16(count)
+		charnum++
+		previous0 = count == 0
+		for remaining < threshold {
+			nbBits--
+			threshold >>= 1
+		}
+
+		if r := b.remain(); r >= 7 || r-int(bitCount>>3) >= 4 {
+			b.advance(bitCount >> 3)
+			bitCount &= 7
+			// The check above should make sure we can read 32 bits
+			bitStream = b.Uint32NC() >> (bitCount & 31)
+		} else {
+			bitCount -= (uint)(8 * (len(b.b) - 4 - b.off))
+			b.off = len(b.b) - 4
+			bitStream = b.Uint32() >> (bitCount & 31)
+		}
+	}
+	s.symbolLen = charnum
+	if s.symbolLen <= 1 {
+		return fmt.Errorf("symbolLen (%d) too small", s.symbolLen)
+	}
+	if s.symbolLen > maxSymbolValue+1 {
+		return fmt.Errorf("symbolLen (%d) too big", s.symbolLen)
+	}
+	if remaining != 1 {
+		return fmt.Errorf("corruption detected (remaining %d != 1)", remaining)
+	}
+	if bitCount > 32 {
+		return fmt.Errorf("corruption detected (bitCount %d > 32)", bitCount)
+	}
+	if gotTotal != 1<<s.actualTableLog {
+		return fmt.Errorf("corruption detected (total %d != %d)", gotTotal, 1<<s.actualTableLog)
+	}
+	b.advance((bitCount + 7) >> 3)
+	// println(s.norm[:s.symbolLen], s.symbolLen)
+	return s.buildDtable()
+}
+
+// decSymbol contains information about a state entry,
+// Including the state offset base, the output symbol and
+// the number of bits to read for the low part of the destination state.
+// Using a composite uint64 is faster than a struct with separate members.
+type decSymbol uint64
+
+func newDecSymbol(nbits, addBits uint8, newState uint16, baseline uint32) decSymbol {
+	return decSymbol(nbits) | (decSymbol(addBits) << 8) | (decSymbol(newState) << 16) | (decSymbol(baseline) << 32)
+}
+
+func (d decSymbol) nbBits() uint8 {
+	return uint8(d)
+}
+
+func (d decSymbol) addBits() uint8 {
+	return uint8(d >> 8)
+}
+
+func (d decSymbol) newState() uint16 {
+	return uint16(d >> 16)
+}
+
+func (d decSymbol) baseline() uint32 {
+	return uint32(d >> 32)
+}
+
+func (d decSymbol) baselineInt() int {
+	return int(d >> 32)
+}
+
+func (d *decSymbol) set(nbits, addBits uint8, newState uint16, baseline uint32) {
+	*d = decSymbol(nbits) | (decSymbol(addBits) << 8) | (decSymbol(newState) << 16) | (decSymbol(baseline) << 32)
+}
+
+func (d *decSymbol) setNBits(nBits uint8) {
+	const mask = 0xffffffffffffff00
+	*d = (*d & mask) | decSymbol(nBits)
+}
+
+func (d *decSymbol) setAddBits(addBits uint8) {
+	const mask = 0xffffffffffff00ff
+	*d = (*d & mask) | (decSymbol(addBits) << 8)
+}
+
+func (d *decSymbol) setNewState(state uint16) {
+	const mask = 0xffffffff0000ffff
+	*d = (*d & mask) | decSymbol(state)<<16
+}
+
+func (d *decSymbol) setBaseline(baseline uint32) {
+	const mask = 0xffffffff
+	*d = (*d & mask) | decSymbol(baseline)<<32
+}
+
+func (d *decSymbol) setExt(addBits uint8, baseline uint32) {
+	const mask = 0xffff00ff
+	*d = (*d & mask) | (decSymbol(addBits) << 8) | (decSymbol(baseline) << 32)
+}
+
+// decSymbolValue returns the transformed decSymbol for the given symbol.
+func decSymbolValue(symb uint8, t []baseOffset) (decSymbol, error) {
+	if int(symb) >= len(t) {
+		return 0, fmt.Errorf("rle symbol %d >= max %d", symb, len(t))
+	}
+	lu := t[symb]
+	return newDecSymbol(0, lu.addBits, 0, lu.baseLine), nil
+}
+
+// setRLE will set the decoder til RLE mode.
+func (s *fseDecoder) setRLE(symbol decSymbol) {
+	s.actualTableLog = 0
+	s.maxBits = symbol.addBits()
+	s.dt[0] = symbol
+}
+
+// buildDtable will build the decoding table.
+func (s *fseDecoder) buildDtable() error {
+	tableSize := uint32(1 << s.actualTableLog)
+	highThreshold := tableSize - 1
+	symbolNext := s.stateTable[:256]
+
+	// Init, lay down lowprob symbols
+	{
+		for i, v := range s.norm[:s.symbolLen] {
+			if v == -1 {
+				s.dt[highThreshold].setAddBits(uint8(i))
+				highThreshold--
+				symbolNext[i] = 1
+			} else {
+				symbolNext[i] = uint16(v)
+			}
+		}
+	}
+	// Spread symbols
+	{
+		tableMask := tableSize - 1
+		step := tableStep(tableSize)
+		position := uint32(0)
+		for ss, v := range s.norm[:s.symbolLen] {
+			for i := 0; i < int(v); i++ {
+				s.dt[position].setAddBits(uint8(ss))
+				position = (position + step) & tableMask
+				for position > highThreshold {
+					// lowprob area
+					position = (position + step) & tableMask
+				}
+			}
+		}
+		if position != 0 {
+			// position must reach all cells once, otherwise normalizedCounter is incorrect
+			return errors.New("corrupted input (position != 0)")
+		}
+	}
+
+	// Build Decoding table
+	{
+		tableSize := uint16(1 << s.actualTableLog)
+		for u, v := range s.dt[:tableSize] {
+			symbol := v.addBits()
+			nextState := symbolNext[symbol]
+			symbolNext[symbol] = nextState + 1
+			nBits := s.actualTableLog - byte(highBits(uint32(nextState)))
+			s.dt[u&maxTableMask].setNBits(nBits)
+			newState := (nextState << nBits) - tableSize
+			if newState > tableSize {
+				return fmt.Errorf("newState (%d) outside table size (%d)", newState, tableSize)
+			}
+			if newState == uint16(u) && nBits == 0 {
+				// Seems weird that this is possible with nbits > 0.
+				return fmt.Errorf("newState (%d) == oldState (%d) and no bits", newState, u)
+			}
+			s.dt[u&maxTableMask].setNewState(newState)
+		}
+	}
+	return nil
+}
+
+// transform will transform the decoder table into a table usable for
+// decoding without having to apply the transformation while decoding.
+// The state will contain the base value and the number of bits to read.
+func (s *fseDecoder) transform(t []baseOffset) error {
+	tableSize := uint16(1 << s.actualTableLog)
+	s.maxBits = 0
+	for i, v := range s.dt[:tableSize] {
+		add := v.addBits()
+		if int(add) >= len(t) {
+			return fmt.Errorf("invalid decoding table entry %d, symbol %d >= max (%d)", i, v.addBits(), len(t))
+		}
+		lu := t[add]
+		if lu.addBits > s.maxBits {
+			s.maxBits = lu.addBits
+		}
+		v.setExt(lu.addBits, lu.baseLine)
+		s.dt[i] = v
+	}
+	return nil
+}
+
+type fseState struct {
+	dt    []decSymbol
+	state decSymbol
+}
+
+// Initialize and decodeAsync first state and symbol.
+func (s *fseState) init(br *bitReader, tableLog uint8, dt []decSymbol) {
+	s.dt = dt
+	br.fill()
+	s.state = dt[br.getBits(tableLog)]
+}
+
+// next returns the current symbol and sets the next state.
+// At least tablelog bits must be available in the bit reader.
+func (s *fseState) next(br *bitReader) {
+	lowBits := uint16(br.getBits(s.state.nbBits()))
+	s.state = s.dt[s.state.newState()+lowBits]
+}
+
+// finished returns true if all bits have been read from the bitstream
+// and the next state would require reading bits from the input.
+func (s *fseState) finished(br *bitReader) bool {
+	return br.finished() && s.state.nbBits() > 0
+}
+
+// final returns the current state symbol without decoding the next.
+func (s *fseState) final() (int, uint8) {
+	return s.state.baselineInt(), s.state.addBits()
+}
+
+// final returns the current state symbol without decoding the next.
+func (s decSymbol) final() (int, uint8) {
+	return s.baselineInt(), s.addBits()
+}
+
+// nextFast returns the next symbol and sets the next state.
+// This can only be used if no symbols are 0 bits.
+// At least tablelog bits must be available in the bit reader.
+func (s *fseState) nextFast(br *bitReader) (uint32, uint8) {
+	lowBits := uint16(br.getBitsFast(s.state.nbBits()))
+	s.state = s.dt[s.state.newState()+lowBits]
+	return s.state.baseline(), s.state.addBits()
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/fse_encoder.go b/vendor/github.com/klauspost/compress/zstd/fse_encoder.go
new file mode 100644
index 0000000..c74681b
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/fse_encoder.go
@@ -0,0 +1,725 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"fmt"
+	"math"
+)
+
+const (
+	// For encoding we only support up to
+	maxEncTableLog    = 8
+	maxEncTablesize   = 1 << maxTableLog
+	maxEncTableMask   = (1 << maxTableLog) - 1
+	minEncTablelog    = 5
+	maxEncSymbolValue = maxMatchLengthSymbol
+)
+
+// Scratch provides temporary storage for compression and decompression.
+type fseEncoder struct {
+	symbolLen      uint16 // Length of active part of the symbol table.
+	actualTableLog uint8  // Selected tablelog.
+	ct             cTable // Compression tables.
+	maxCount       int    // count of the most probable symbol
+	zeroBits       bool   // no bits has prob > 50%.
+	clearCount     bool   // clear count
+	useRLE         bool   // This encoder is for RLE
+	preDefined     bool   // This encoder is predefined.
+	reUsed         bool   // Set to know when the encoder has been reused.
+	rleVal         uint8  // RLE Symbol
+	maxBits        uint8  // Maximum output bits after transform.
+
+	// TODO: Technically zstd should be fine with 64 bytes.
+	count [256]uint32
+	norm  [256]int16
+}
+
+// cTable contains tables used for compression.
+type cTable struct {
+	tableSymbol []byte
+	stateTable  []uint16
+	symbolTT    []symbolTransform
+}
+
+// symbolTransform contains the state transform for a symbol.
+type symbolTransform struct {
+	deltaNbBits    uint32
+	deltaFindState int16
+	outBits        uint8
+}
+
+// String prints values as a human readable string.
+func (s symbolTransform) String() string {
+	return fmt.Sprintf("{deltabits: %08x, findstate:%d outbits:%d}", s.deltaNbBits, s.deltaFindState, s.outBits)
+}
+
+// Histogram allows to populate the histogram and skip that step in the compression,
+// It otherwise allows to inspect the histogram when compression is done.
+// To indicate that you have populated the histogram call HistogramFinished
+// with the value of the highest populated symbol, as well as the number of entries
+// in the most populated entry. These are accepted at face value.
+// The returned slice will always be length 256.
+func (s *fseEncoder) Histogram() []uint32 {
+	return s.count[:]
+}
+
+// HistogramFinished can be called to indicate that the histogram has been populated.
+// maxSymbol is the index of the highest set symbol of the next data segment.
+// maxCount is the number of entries in the most populated entry.
+// These are accepted at face value.
+func (s *fseEncoder) HistogramFinished(maxSymbol uint8, maxCount int) {
+	s.maxCount = maxCount
+	s.symbolLen = uint16(maxSymbol) + 1
+	s.clearCount = maxCount != 0
+}
+
+// prepare will prepare and allocate scratch tables used for both compression and decompression.
+func (s *fseEncoder) prepare() (*fseEncoder, error) {
+	if s == nil {
+		s = &fseEncoder{}
+	}
+	s.useRLE = false
+	if s.clearCount && s.maxCount == 0 {
+		for i := range s.count {
+			s.count[i] = 0
+		}
+		s.clearCount = false
+	}
+	return s, nil
+}
+
+// allocCtable will allocate tables needed for compression.
+// If existing tables a re big enough, they are simply re-used.
+func (s *fseEncoder) allocCtable() {
+	tableSize := 1 << s.actualTableLog
+	// get tableSymbol that is big enough.
+	if cap(s.ct.tableSymbol) < tableSize {
+		s.ct.tableSymbol = make([]byte, tableSize)
+	}
+	s.ct.tableSymbol = s.ct.tableSymbol[:tableSize]
+
+	ctSize := tableSize
+	if cap(s.ct.stateTable) < ctSize {
+		s.ct.stateTable = make([]uint16, ctSize)
+	}
+	s.ct.stateTable = s.ct.stateTable[:ctSize]
+
+	if cap(s.ct.symbolTT) < 256 {
+		s.ct.symbolTT = make([]symbolTransform, 256)
+	}
+	s.ct.symbolTT = s.ct.symbolTT[:256]
+}
+
+// buildCTable will populate the compression table so it is ready to be used.
+func (s *fseEncoder) buildCTable() error {
+	tableSize := uint32(1 << s.actualTableLog)
+	highThreshold := tableSize - 1
+	var cumul [256]int16
+
+	s.allocCtable()
+	tableSymbol := s.ct.tableSymbol[:tableSize]
+	// symbol start positions
+	{
+		cumul[0] = 0
+		for ui, v := range s.norm[:s.symbolLen-1] {
+			u := byte(ui) // one less than reference
+			if v == -1 {
+				// Low proba symbol
+				cumul[u+1] = cumul[u] + 1
+				tableSymbol[highThreshold] = u
+				highThreshold--
+			} else {
+				cumul[u+1] = cumul[u] + v
+			}
+		}
+		// Encode last symbol separately to avoid overflowing u
+		u := int(s.symbolLen - 1)
+		v := s.norm[s.symbolLen-1]
+		if v == -1 {
+			// Low proba symbol
+			cumul[u+1] = cumul[u] + 1
+			tableSymbol[highThreshold] = byte(u)
+			highThreshold--
+		} else {
+			cumul[u+1] = cumul[u] + v
+		}
+		if uint32(cumul[s.symbolLen]) != tableSize {
+			return fmt.Errorf("internal error: expected cumul[s.symbolLen] (%d) == tableSize (%d)", cumul[s.symbolLen], tableSize)
+		}
+		cumul[s.symbolLen] = int16(tableSize) + 1
+	}
+	// Spread symbols
+	s.zeroBits = false
+	{
+		step := tableStep(tableSize)
+		tableMask := tableSize - 1
+		var position uint32
+		// if any symbol > largeLimit, we may have 0 bits output.
+		largeLimit := int16(1 << (s.actualTableLog - 1))
+		for ui, v := range s.norm[:s.symbolLen] {
+			symbol := byte(ui)
+			if v > largeLimit {
+				s.zeroBits = true
+			}
+			for nbOccurrences := int16(0); nbOccurrences < v; nbOccurrences++ {
+				tableSymbol[position] = symbol
+				position = (position + step) & tableMask
+				for position > highThreshold {
+					position = (position + step) & tableMask
+				} /* Low proba area */
+			}
+		}
+
+		// Check if we have gone through all positions
+		if position != 0 {
+			return errors.New("position!=0")
+		}
+	}
+
+	// Build table
+	table := s.ct.stateTable
+	{
+		tsi := int(tableSize)
+		for u, v := range tableSymbol {
+			// TableU16 : sorted by symbol order; gives next state value
+			table[cumul[v]] = uint16(tsi + u)
+			cumul[v]++
+		}
+	}
+
+	// Build Symbol Transformation Table
+	{
+		total := int16(0)
+		symbolTT := s.ct.symbolTT[:s.symbolLen]
+		tableLog := s.actualTableLog
+		tl := (uint32(tableLog) << 16) - (1 << tableLog)
+		for i, v := range s.norm[:s.symbolLen] {
+			switch v {
+			case 0:
+			case -1, 1:
+				symbolTT[i].deltaNbBits = tl
+				symbolTT[i].deltaFindState = total - 1
+				total++
+			default:
+				maxBitsOut := uint32(tableLog) - highBit(uint32(v-1))
+				minStatePlus := uint32(v) << maxBitsOut
+				symbolTT[i].deltaNbBits = (maxBitsOut << 16) - minStatePlus
+				symbolTT[i].deltaFindState = total - v
+				total += v
+			}
+		}
+		if total != int16(tableSize) {
+			return fmt.Errorf("total mismatch %d (got) != %d (want)", total, tableSize)
+		}
+	}
+	return nil
+}
+
+var rtbTable = [...]uint32{0, 473195, 504333, 520860, 550000, 700000, 750000, 830000}
+
+func (s *fseEncoder) setRLE(val byte) {
+	s.allocCtable()
+	s.actualTableLog = 0
+	s.ct.stateTable = s.ct.stateTable[:1]
+	s.ct.symbolTT[val] = symbolTransform{
+		deltaFindState: 0,
+		deltaNbBits:    0,
+	}
+	if debug {
+		println("setRLE: val", val, "symbolTT", s.ct.symbolTT[val])
+	}
+	s.rleVal = val
+	s.useRLE = true
+}
+
+// setBits will set output bits for the transform.
+// if nil is provided, the number of bits is equal to the index.
+func (s *fseEncoder) setBits(transform []byte) {
+	if s.reUsed || s.preDefined {
+		return
+	}
+	if s.useRLE {
+		if transform == nil {
+			s.ct.symbolTT[s.rleVal].outBits = s.rleVal
+			s.maxBits = s.rleVal
+			return
+		}
+		s.maxBits = transform[s.rleVal]
+		s.ct.symbolTT[s.rleVal].outBits = s.maxBits
+		return
+	}
+	if transform == nil {
+		for i := range s.ct.symbolTT[:s.symbolLen] {
+			s.ct.symbolTT[i].outBits = uint8(i)
+		}
+		s.maxBits = uint8(s.symbolLen - 1)
+		return
+	}
+	s.maxBits = 0
+	for i, v := range transform[:s.symbolLen] {
+		s.ct.symbolTT[i].outBits = v
+		if v > s.maxBits {
+			// We could assume bits always going up, but we play safe.
+			s.maxBits = v
+		}
+	}
+}
+
+// normalizeCount will normalize the count of the symbols so
+// the total is equal to the table size.
+// If successful, compression tables will also be made ready.
+func (s *fseEncoder) normalizeCount(length int) error {
+	if s.reUsed {
+		return nil
+	}
+	s.optimalTableLog(length)
+	var (
+		tableLog          = s.actualTableLog
+		scale             = 62 - uint64(tableLog)
+		step              = (1 << 62) / uint64(length)
+		vStep             = uint64(1) << (scale - 20)
+		stillToDistribute = int16(1 << tableLog)
+		largest           int
+		largestP          int16
+		lowThreshold      = (uint32)(length >> tableLog)
+	)
+	if s.maxCount == length {
+		s.useRLE = true
+		return nil
+	}
+	s.useRLE = false
+	for i, cnt := range s.count[:s.symbolLen] {
+		// already handled
+		// if (count[s] == s.length) return 0;   /* rle special case */
+
+		if cnt == 0 {
+			s.norm[i] = 0
+			continue
+		}
+		if cnt <= lowThreshold {
+			s.norm[i] = -1
+			stillToDistribute--
+		} else {
+			proba := (int16)((uint64(cnt) * step) >> scale)
+			if proba < 8 {
+				restToBeat := vStep * uint64(rtbTable[proba])
+				v := uint64(cnt)*step - (uint64(proba) << scale)
+				if v > restToBeat {
+					proba++
+				}
+			}
+			if proba > largestP {
+				largestP = proba
+				largest = i
+			}
+			s.norm[i] = proba
+			stillToDistribute -= proba
+		}
+	}
+
+	if -stillToDistribute >= (s.norm[largest] >> 1) {
+		// corner case, need another normalization method
+		err := s.normalizeCount2(length)
+		if err != nil {
+			return err
+		}
+		if debugAsserts {
+			err = s.validateNorm()
+			if err != nil {
+				return err
+			}
+		}
+		return s.buildCTable()
+	}
+	s.norm[largest] += stillToDistribute
+	if debugAsserts {
+		err := s.validateNorm()
+		if err != nil {
+			return err
+		}
+	}
+	return s.buildCTable()
+}
+
+// Secondary normalization method.
+// To be used when primary method fails.
+func (s *fseEncoder) normalizeCount2(length int) error {
+	const notYetAssigned = -2
+	var (
+		distributed  uint32
+		total        = uint32(length)
+		tableLog     = s.actualTableLog
+		lowThreshold = total >> tableLog
+		lowOne       = (total * 3) >> (tableLog + 1)
+	)
+	for i, cnt := range s.count[:s.symbolLen] {
+		if cnt == 0 {
+			s.norm[i] = 0
+			continue
+		}
+		if cnt <= lowThreshold {
+			s.norm[i] = -1
+			distributed++
+			total -= cnt
+			continue
+		}
+		if cnt <= lowOne {
+			s.norm[i] = 1
+			distributed++
+			total -= cnt
+			continue
+		}
+		s.norm[i] = notYetAssigned
+	}
+	toDistribute := (1 << tableLog) - distributed
+
+	if (total / toDistribute) > lowOne {
+		// risk of rounding to zero
+		lowOne = (total * 3) / (toDistribute * 2)
+		for i, cnt := range s.count[:s.symbolLen] {
+			if (s.norm[i] == notYetAssigned) && (cnt <= lowOne) {
+				s.norm[i] = 1
+				distributed++
+				total -= cnt
+				continue
+			}
+		}
+		toDistribute = (1 << tableLog) - distributed
+	}
+	if distributed == uint32(s.symbolLen)+1 {
+		// all values are pretty poor;
+		//   probably incompressible data (should have already been detected);
+		//   find max, then give all remaining points to max
+		var maxV int
+		var maxC uint32
+		for i, cnt := range s.count[:s.symbolLen] {
+			if cnt > maxC {
+				maxV = i
+				maxC = cnt
+			}
+		}
+		s.norm[maxV] += int16(toDistribute)
+		return nil
+	}
+
+	if total == 0 {
+		// all of the symbols were low enough for the lowOne or lowThreshold
+		for i := uint32(0); toDistribute > 0; i = (i + 1) % (uint32(s.symbolLen)) {
+			if s.norm[i] > 0 {
+				toDistribute--
+				s.norm[i]++
+			}
+		}
+		return nil
+	}
+
+	var (
+		vStepLog = 62 - uint64(tableLog)
+		mid      = uint64((1 << (vStepLog - 1)) - 1)
+		rStep    = (((1 << vStepLog) * uint64(toDistribute)) + mid) / uint64(total) // scale on remaining
+		tmpTotal = mid
+	)
+	for i, cnt := range s.count[:s.symbolLen] {
+		if s.norm[i] == notYetAssigned {
+			var (
+				end    = tmpTotal + uint64(cnt)*rStep
+				sStart = uint32(tmpTotal >> vStepLog)
+				sEnd   = uint32(end >> vStepLog)
+				weight = sEnd - sStart
+			)
+			if weight < 1 {
+				return errors.New("weight < 1")
+			}
+			s.norm[i] = int16(weight)
+			tmpTotal = end
+		}
+	}
+	return nil
+}
+
+// optimalTableLog calculates and sets the optimal tableLog in s.actualTableLog
+func (s *fseEncoder) optimalTableLog(length int) {
+	tableLog := uint8(maxEncTableLog)
+	minBitsSrc := highBit(uint32(length)) + 1
+	minBitsSymbols := highBit(uint32(s.symbolLen-1)) + 2
+	minBits := uint8(minBitsSymbols)
+	if minBitsSrc < minBitsSymbols {
+		minBits = uint8(minBitsSrc)
+	}
+
+	maxBitsSrc := uint8(highBit(uint32(length-1))) - 2
+	if maxBitsSrc < tableLog {
+		// Accuracy can be reduced
+		tableLog = maxBitsSrc
+	}
+	if minBits > tableLog {
+		tableLog = minBits
+	}
+	// Need a minimum to safely represent all symbol values
+	if tableLog < minEncTablelog {
+		tableLog = minEncTablelog
+	}
+	if tableLog > maxEncTableLog {
+		tableLog = maxEncTableLog
+	}
+	s.actualTableLog = tableLog
+}
+
+// validateNorm validates the normalized histogram table.
+func (s *fseEncoder) validateNorm() (err error) {
+	var total int
+	for _, v := range s.norm[:s.symbolLen] {
+		if v >= 0 {
+			total += int(v)
+		} else {
+			total -= int(v)
+		}
+	}
+	defer func() {
+		if err == nil {
+			return
+		}
+		fmt.Printf("selected TableLog: %d, Symbol length: %d\n", s.actualTableLog, s.symbolLen)
+		for i, v := range s.norm[:s.symbolLen] {
+			fmt.Printf("%3d: %5d -> %4d \n", i, s.count[i], v)
+		}
+	}()
+	if total != (1 << s.actualTableLog) {
+		return fmt.Errorf("warning: Total == %d != %d", total, 1<<s.actualTableLog)
+	}
+	for i, v := range s.count[s.symbolLen:] {
+		if v != 0 {
+			return fmt.Errorf("warning: Found symbol out of range, %d after cut", i)
+		}
+	}
+	return nil
+}
+
+// writeCount will write the normalized histogram count to header.
+// This is read back by readNCount.
+func (s *fseEncoder) writeCount(out []byte) ([]byte, error) {
+	if s.useRLE {
+		return append(out, s.rleVal), nil
+	}
+	if s.preDefined || s.reUsed {
+		// Never write predefined.
+		return out, nil
+	}
+
+	var (
+		tableLog  = s.actualTableLog
+		tableSize = 1 << tableLog
+		previous0 bool
+		charnum   uint16
+
+		// maximum header size plus 2 extra bytes for final output if bitCount == 0.
+		maxHeaderSize = ((int(s.symbolLen) * int(tableLog)) >> 3) + 3 + 2
+
+		// Write Table Size
+		bitStream = uint32(tableLog - minEncTablelog)
+		bitCount  = uint(4)
+		remaining = int16(tableSize + 1) /* +1 for extra accuracy */
+		threshold = int16(tableSize)
+		nbBits    = uint(tableLog + 1)
+		outP      = len(out)
+	)
+	if cap(out) < outP+maxHeaderSize {
+		out = append(out, make([]byte, maxHeaderSize*3)...)
+		out = out[:len(out)-maxHeaderSize*3]
+	}
+	out = out[:outP+maxHeaderSize]
+
+	// stops at 1
+	for remaining > 1 {
+		if previous0 {
+			start := charnum
+			for s.norm[charnum] == 0 {
+				charnum++
+			}
+			for charnum >= start+24 {
+				start += 24
+				bitStream += uint32(0xFFFF) << bitCount
+				out[outP] = byte(bitStream)
+				out[outP+1] = byte(bitStream >> 8)
+				outP += 2
+				bitStream >>= 16
+			}
+			for charnum >= start+3 {
+				start += 3
+				bitStream += 3 << bitCount
+				bitCount += 2
+			}
+			bitStream += uint32(charnum-start) << bitCount
+			bitCount += 2
+			if bitCount > 16 {
+				out[outP] = byte(bitStream)
+				out[outP+1] = byte(bitStream >> 8)
+				outP += 2
+				bitStream >>= 16
+				bitCount -= 16
+			}
+		}
+
+		count := s.norm[charnum]
+		charnum++
+		max := (2*threshold - 1) - remaining
+		if count < 0 {
+			remaining += count
+		} else {
+			remaining -= count
+		}
+		count++ // +1 for extra accuracy
+		if count >= threshold {
+			count += max // [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[
+		}
+		bitStream += uint32(count) << bitCount
+		bitCount += nbBits
+		if count < max {
+			bitCount--
+		}
+
+		previous0 = count == 1
+		if remaining < 1 {
+			return nil, errors.New("internal error: remaining < 1")
+		}
+		for remaining < threshold {
+			nbBits--
+			threshold >>= 1
+		}
+
+		if bitCount > 16 {
+			out[outP] = byte(bitStream)
+			out[outP+1] = byte(bitStream >> 8)
+			outP += 2
+			bitStream >>= 16
+			bitCount -= 16
+		}
+	}
+
+	if outP+2 > len(out) {
+		return nil, fmt.Errorf("internal error: %d > %d, maxheader: %d, sl: %d, tl: %d, normcount: %v", outP+2, len(out), maxHeaderSize, s.symbolLen, int(tableLog), s.norm[:s.symbolLen])
+	}
+	out[outP] = byte(bitStream)
+	out[outP+1] = byte(bitStream >> 8)
+	outP += int((bitCount + 7) / 8)
+
+	if charnum > s.symbolLen {
+		return nil, errors.New("internal error: charnum > s.symbolLen")
+	}
+	return out[:outP], nil
+}
+
+// Approximate symbol cost, as fractional value, using fixed-point format (accuracyLog fractional bits)
+// note 1 : assume symbolValue is valid (<= maxSymbolValue)
+// note 2 : if freq[symbolValue]==0, @return a fake cost of tableLog+1 bits *
+func (s *fseEncoder) bitCost(symbolValue uint8, accuracyLog uint32) uint32 {
+	minNbBits := s.ct.symbolTT[symbolValue].deltaNbBits >> 16
+	threshold := (minNbBits + 1) << 16
+	if debugAsserts {
+		if !(s.actualTableLog < 16) {
+			panic("!s.actualTableLog < 16")
+		}
+		// ensure enough room for renormalization double shift
+		if !(uint8(accuracyLog) < 31-s.actualTableLog) {
+			panic("!uint8(accuracyLog) < 31-s.actualTableLog")
+		}
+	}
+	tableSize := uint32(1) << s.actualTableLog
+	deltaFromThreshold := threshold - (s.ct.symbolTT[symbolValue].deltaNbBits + tableSize)
+	// linear interpolation (very approximate)
+	normalizedDeltaFromThreshold := (deltaFromThreshold << accuracyLog) >> s.actualTableLog
+	bitMultiplier := uint32(1) << accuracyLog
+	if debugAsserts {
+		if s.ct.symbolTT[symbolValue].deltaNbBits+tableSize > threshold {
+			panic("s.ct.symbolTT[symbolValue].deltaNbBits+tableSize > threshold")
+		}
+		if normalizedDeltaFromThreshold > bitMultiplier {
+			panic("normalizedDeltaFromThreshold > bitMultiplier")
+		}
+	}
+	return (minNbBits+1)*bitMultiplier - normalizedDeltaFromThreshold
+}
+
+// Returns the cost in bits of encoding the distribution in count using ctable.
+// Histogram should only be up to the last non-zero symbol.
+// Returns an -1 if ctable cannot represent all the symbols in count.
+func (s *fseEncoder) approxSize(hist []uint32) uint32 {
+	if int(s.symbolLen) < len(hist) {
+		// More symbols than we have.
+		return math.MaxUint32
+	}
+	if s.useRLE {
+		// We will never reuse RLE encoders.
+		return math.MaxUint32
+	}
+	const kAccuracyLog = 8
+	badCost := (uint32(s.actualTableLog) + 1) << kAccuracyLog
+	var cost uint32
+	for i, v := range hist {
+		if v == 0 {
+			continue
+		}
+		if s.norm[i] == 0 {
+			return math.MaxUint32
+		}
+		bitCost := s.bitCost(uint8(i), kAccuracyLog)
+		if bitCost > badCost {
+			return math.MaxUint32
+		}
+		cost += v * bitCost
+	}
+	return cost >> kAccuracyLog
+}
+
+// maxHeaderSize returns the maximum header size in bits.
+// This is not exact size, but we want a penalty for new tables anyway.
+func (s *fseEncoder) maxHeaderSize() uint32 {
+	if s.preDefined {
+		return 0
+	}
+	if s.useRLE {
+		return 8
+	}
+	return (((uint32(s.symbolLen) * uint32(s.actualTableLog)) >> 3) + 3) * 8
+}
+
+// cState contains the compression state of a stream.
+type cState struct {
+	bw         *bitWriter
+	stateTable []uint16
+	state      uint16
+}
+
+// init will initialize the compression state to the first symbol of the stream.
+func (c *cState) init(bw *bitWriter, ct *cTable, first symbolTransform) {
+	c.bw = bw
+	c.stateTable = ct.stateTable
+	if len(c.stateTable) == 1 {
+		// RLE
+		c.stateTable[0] = uint16(0)
+		c.state = 0
+		return
+	}
+	nbBitsOut := (first.deltaNbBits + (1 << 15)) >> 16
+	im := int32((nbBitsOut << 16) - first.deltaNbBits)
+	lu := (im >> nbBitsOut) + int32(first.deltaFindState)
+	c.state = c.stateTable[lu]
+}
+
+// encode the output symbol provided and write it to the bitstream.
+func (c *cState) encode(symbolTT symbolTransform) {
+	nbBitsOut := (uint32(c.state) + symbolTT.deltaNbBits) >> 16
+	dstState := int32(c.state>>(nbBitsOut&15)) + int32(symbolTT.deltaFindState)
+	c.bw.addBits16NC(c.state, uint8(nbBitsOut))
+	c.state = c.stateTable[dstState]
+}
+
+// flush will write the tablelog to the output and flush the remaining full bytes.
+func (c *cState) flush(tableLog uint8) {
+	c.bw.flush32()
+	c.bw.addBits16NC(c.state, tableLog)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/fse_predefined.go b/vendor/github.com/klauspost/compress/zstd/fse_predefined.go
new file mode 100644
index 0000000..474cb77
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/fse_predefined.go
@@ -0,0 +1,158 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"fmt"
+	"math"
+	"sync"
+)
+
+var (
+	// fsePredef are the predefined fse tables as defined here:
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#default-distributions
+	// These values are already transformed.
+	fsePredef [3]fseDecoder
+
+	// fsePredefEnc are the predefined encoder based on fse tables as defined here:
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#default-distributions
+	// These values are already transformed.
+	fsePredefEnc [3]fseEncoder
+
+	// symbolTableX contain the transformations needed for each type as defined in
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#the-codes-for-literals-lengths-match-lengths-and-offsets
+	symbolTableX [3][]baseOffset
+
+	// maxTableSymbol is the biggest supported symbol for each table type
+	// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#the-codes-for-literals-lengths-match-lengths-and-offsets
+	maxTableSymbol = [3]uint8{tableLiteralLengths: maxLiteralLengthSymbol, tableOffsets: maxOffsetLengthSymbol, tableMatchLengths: maxMatchLengthSymbol}
+
+	// bitTables is the bits table for each table.
+	bitTables = [3][]byte{tableLiteralLengths: llBitsTable[:], tableOffsets: nil, tableMatchLengths: mlBitsTable[:]}
+)
+
+type tableIndex uint8
+
+const (
+	// indexes for fsePredef and symbolTableX
+	tableLiteralLengths tableIndex = 0
+	tableOffsets        tableIndex = 1
+	tableMatchLengths   tableIndex = 2
+
+	maxLiteralLengthSymbol = 35
+	maxOffsetLengthSymbol  = 30
+	maxMatchLengthSymbol   = 52
+)
+
+// baseOffset is used for calculating transformations.
+type baseOffset struct {
+	baseLine uint32
+	addBits  uint8
+}
+
+// fillBase will precalculate base offsets with the given bit distributions.
+func fillBase(dst []baseOffset, base uint32, bits ...uint8) {
+	if len(bits) != len(dst) {
+		panic(fmt.Sprintf("len(dst) (%d) != len(bits) (%d)", len(dst), len(bits)))
+	}
+	for i, bit := range bits {
+		if base > math.MaxInt32 {
+			panic("invalid decoding table, base overflows int32")
+		}
+
+		dst[i] = baseOffset{
+			baseLine: base,
+			addBits:  bit,
+		}
+		base += 1 << bit
+	}
+}
+
+var predef sync.Once
+
+func initPredefined() {
+	predef.Do(func() {
+		// Literals length codes
+		tmp := make([]baseOffset, 36)
+		for i := range tmp[:16] {
+			tmp[i] = baseOffset{
+				baseLine: uint32(i),
+				addBits:  0,
+			}
+		}
+		fillBase(tmp[16:], 16, 1, 1, 1, 1, 2, 2, 3, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)
+		symbolTableX[tableLiteralLengths] = tmp
+
+		// Match length codes
+		tmp = make([]baseOffset, 53)
+		for i := range tmp[:32] {
+			tmp[i] = baseOffset{
+				// The transformation adds the 3 length.
+				baseLine: uint32(i) + 3,
+				addBits:  0,
+			}
+		}
+		fillBase(tmp[32:], 35, 1, 1, 1, 1, 2, 2, 3, 3, 4, 4, 5, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)
+		symbolTableX[tableMatchLengths] = tmp
+
+		// Offset codes
+		tmp = make([]baseOffset, maxOffsetBits+1)
+		tmp[1] = baseOffset{
+			baseLine: 1,
+			addBits:  1,
+		}
+		fillBase(tmp[2:], 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)
+		symbolTableX[tableOffsets] = tmp
+
+		// Fill predefined tables and transform them.
+		// https://github.com/facebook/zstd/blob/dev/doc/zstd_compression_format.md#default-distributions
+		for i := range fsePredef[:] {
+			f := &fsePredef[i]
+			switch tableIndex(i) {
+			case tableLiteralLengths:
+				// https://github.com/facebook/zstd/blob/ededcfca57366461021c922720878c81a5854a0a/lib/decompress/zstd_decompress_block.c#L243
+				f.actualTableLog = 6
+				copy(f.norm[:], []int16{4, 3, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1,
+					2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 2, 1, 1, 1, 1, 1,
+					-1, -1, -1, -1})
+				f.symbolLen = 36
+			case tableOffsets:
+				// https://github.com/facebook/zstd/blob/ededcfca57366461021c922720878c81a5854a0a/lib/decompress/zstd_decompress_block.c#L281
+				f.actualTableLog = 5
+				copy(f.norm[:], []int16{
+					1, 1, 1, 1, 1, 1, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
+					1, 1, 1, 1, 1, 1, 1, 1, -1, -1, -1, -1, -1})
+				f.symbolLen = 29
+			case tableMatchLengths:
+				//https://github.com/facebook/zstd/blob/ededcfca57366461021c922720878c81a5854a0a/lib/decompress/zstd_decompress_block.c#L304
+				f.actualTableLog = 6
+				copy(f.norm[:], []int16{
+					1, 4, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1,
+					1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
+					1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, -1, -1,
+					-1, -1, -1, -1, -1})
+				f.symbolLen = 53
+			}
+			if err := f.buildDtable(); err != nil {
+				panic(fmt.Errorf("building table %v: %v", tableIndex(i), err))
+			}
+			if err := f.transform(symbolTableX[i]); err != nil {
+				panic(fmt.Errorf("building table %v: %v", tableIndex(i), err))
+			}
+			f.preDefined = true
+
+			// Create encoder as well
+			enc := &fsePredefEnc[i]
+			copy(enc.norm[:], f.norm[:])
+			enc.symbolLen = f.symbolLen
+			enc.actualTableLog = f.actualTableLog
+			if err := enc.buildCTable(); err != nil {
+				panic(fmt.Errorf("building encoding table %v: %v", tableIndex(i), err))
+			}
+			enc.setBits(bitTables[i])
+			enc.preDefined = true
+		}
+	})
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/hash.go b/vendor/github.com/klauspost/compress/zstd/hash.go
new file mode 100644
index 0000000..4a75206
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/hash.go
@@ -0,0 +1,77 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+const (
+	prime3bytes = 506832829
+	prime4bytes = 2654435761
+	prime5bytes = 889523592379
+	prime6bytes = 227718039650203
+	prime7bytes = 58295818150454627
+	prime8bytes = 0xcf1bbcdcb7a56463
+)
+
+// hashLen returns a hash of the lowest l bytes of u for a size size of h bytes.
+// l must be >=4 and <=8. Any other value will return hash for 4 bytes.
+// h should always be <32.
+// Preferably h and l should be a constant.
+// FIXME: This does NOT get resolved, if 'mls' is constant,
+//  so this cannot be used.
+func hashLen(u uint64, hashLog, mls uint8) uint32 {
+	switch mls {
+	case 5:
+		return hash5(u, hashLog)
+	case 6:
+		return hash6(u, hashLog)
+	case 7:
+		return hash7(u, hashLog)
+	case 8:
+		return hash8(u, hashLog)
+	default:
+		return hash4x64(u, hashLog)
+	}
+}
+
+// hash3 returns the hash of the lower 3 bytes of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <32.
+func hash3(u uint32, h uint8) uint32 {
+	return ((u << (32 - 24)) * prime3bytes) >> ((32 - h) & 31)
+}
+
+// hash4 returns the hash of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <32.
+func hash4(u uint32, h uint8) uint32 {
+	return (u * prime4bytes) >> ((32 - h) & 31)
+}
+
+// hash4x64 returns the hash of the lowest 4 bytes of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <32.
+func hash4x64(u uint64, h uint8) uint32 {
+	return (uint32(u) * prime4bytes) >> ((32 - h) & 31)
+}
+
+// hash5 returns the hash of the lowest 5 bytes of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <64.
+func hash5(u uint64, h uint8) uint32 {
+	return uint32(((u << (64 - 40)) * prime5bytes) >> ((64 - h) & 63))
+}
+
+// hash6 returns the hash of the lowest 6 bytes of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <64.
+func hash6(u uint64, h uint8) uint32 {
+	return uint32(((u << (64 - 48)) * prime6bytes) >> ((64 - h) & 63))
+}
+
+// hash7 returns the hash of the lowest 7 bytes of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <64.
+func hash7(u uint64, h uint8) uint32 {
+	return uint32(((u << (64 - 56)) * prime7bytes) >> ((64 - h) & 63))
+}
+
+// hash8 returns the hash of u to fit in a hash table with h bits.
+// Preferably h should be a constant and should always be <64.
+func hash8(u uint64, h uint8) uint32 {
+	return uint32((u * prime8bytes) >> ((64 - h) & 63))
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/history.go b/vendor/github.com/klauspost/compress/zstd/history.go
new file mode 100644
index 0000000..f783e32
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/history.go
@@ -0,0 +1,89 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"github.com/klauspost/compress/huff0"
+)
+
+// history contains the information transferred between blocks.
+type history struct {
+	b             []byte
+	huffTree      *huff0.Scratch
+	recentOffsets [3]int
+	decoders      sequenceDecs
+	windowSize    int
+	maxSize       int
+	error         bool
+	dict          *dict
+}
+
+// reset will reset the history to initial state of a frame.
+// The history must already have been initialized to the desired size.
+func (h *history) reset() {
+	h.b = h.b[:0]
+	h.error = false
+	h.recentOffsets = [3]int{1, 4, 8}
+	if f := h.decoders.litLengths.fse; f != nil && !f.preDefined {
+		fseDecoderPool.Put(f)
+	}
+	if f := h.decoders.offsets.fse; f != nil && !f.preDefined {
+		fseDecoderPool.Put(f)
+	}
+	if f := h.decoders.matchLengths.fse; f != nil && !f.preDefined {
+		fseDecoderPool.Put(f)
+	}
+	h.decoders = sequenceDecs{}
+	if h.huffTree != nil {
+		if h.dict == nil || h.dict.litEnc != h.huffTree {
+			huffDecoderPool.Put(h.huffTree)
+		}
+	}
+	h.huffTree = nil
+	h.dict = nil
+	//printf("history created: %+v (l: %d, c: %d)", *h, len(h.b), cap(h.b))
+}
+
+func (h *history) setDict(dict *dict) {
+	if dict == nil {
+		return
+	}
+	h.dict = dict
+	h.decoders.litLengths = dict.llDec
+	h.decoders.offsets = dict.ofDec
+	h.decoders.matchLengths = dict.mlDec
+	h.recentOffsets = dict.offsets
+	h.huffTree = dict.litEnc
+}
+
+// append bytes to history.
+// This function will make sure there is space for it,
+// if the buffer has been allocated with enough extra space.
+func (h *history) append(b []byte) {
+	if len(b) >= h.windowSize {
+		// Discard all history by simply overwriting
+		h.b = h.b[:h.windowSize]
+		copy(h.b, b[len(b)-h.windowSize:])
+		return
+	}
+
+	// If there is space, append it.
+	if len(b) < cap(h.b)-len(h.b) {
+		h.b = append(h.b, b...)
+		return
+	}
+
+	// Move data down so we only have window size left.
+	// We know we have less than window size in b at this point.
+	discard := len(b) + len(h.b) - h.windowSize
+	copy(h.b, h.b[discard:])
+	h.b = h.b[:h.windowSize]
+	copy(h.b[h.windowSize-len(b):], b)
+}
+
+// append bytes to history without ever discarding anything.
+func (h *history) appendKeep(b []byte) {
+	h.b = append(h.b, b...)
+}
diff --git a/vendor/github.com/cespare/xxhash/LICENSE.txt b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/LICENSE.txt
similarity index 100%
rename from vendor/github.com/cespare/xxhash/LICENSE.txt
rename to vendor/github.com/klauspost/compress/zstd/internal/xxhash/LICENSE.txt
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/README.md b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/README.md
new file mode 100644
index 0000000..69aa3bb
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/README.md
@@ -0,0 +1,58 @@
+# xxhash
+
+VENDORED: Go to [github.com/cespare/xxhash](https://github.com/cespare/xxhash) for original package.
+
+
+[![GoDoc](https://godoc.org/github.com/cespare/xxhash?status.svg)](https://godoc.org/github.com/cespare/xxhash)
+[![Build Status](https://travis-ci.org/cespare/xxhash.svg?branch=master)](https://travis-ci.org/cespare/xxhash)
+
+xxhash is a Go implementation of the 64-bit
+[xxHash](http://cyan4973.github.io/xxHash/) algorithm, XXH64. This is a
+high-quality hashing algorithm that is much faster than anything in the Go
+standard library.
+
+This package provides a straightforward API:
+
+```
+func Sum64(b []byte) uint64
+func Sum64String(s string) uint64
+type Digest struct{ ... }
+    func New() *Digest
+```
+
+The `Digest` type implements hash.Hash64. Its key methods are:
+
+```
+func (*Digest) Write([]byte) (int, error)
+func (*Digest) WriteString(string) (int, error)
+func (*Digest) Sum64() uint64
+```
+
+This implementation provides a fast pure-Go implementation and an even faster
+assembly implementation for amd64.
+
+## Benchmarks
+
+Here are some quick benchmarks comparing the pure-Go and assembly
+implementations of Sum64.
+
+| input size | purego | asm |
+| --- | --- | --- |
+| 5 B   |  979.66 MB/s |  1291.17 MB/s  |
+| 100 B | 7475.26 MB/s | 7973.40 MB/s  |
+| 4 KB  | 17573.46 MB/s | 17602.65 MB/s |
+| 10 MB | 17131.46 MB/s | 17142.16 MB/s |
+
+These numbers were generated on Ubuntu 18.04 with an Intel i7-8700K CPU using
+the following commands under Go 1.11.2:
+
+```
+$ go test -tags purego -benchtime 10s -bench '/xxhash,direct,bytes'
+$ go test -benchtime 10s -bench '/xxhash,direct,bytes'
+```
+
+## Projects using this package
+
+- [InfluxDB](https://github.com/influxdata/influxdb)
+- [Prometheus](https://github.com/prometheus/prometheus)
+- [FreeCache](https://github.com/coocood/freecache)
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go
new file mode 100644
index 0000000..426b9ca
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go
@@ -0,0 +1,238 @@
+// Package xxhash implements the 64-bit variant of xxHash (XXH64) as described
+// at http://cyan4973.github.io/xxHash/.
+// THIS IS VENDORED: Go to github.com/cespare/xxhash for original package.
+
+package xxhash
+
+import (
+	"encoding/binary"
+	"errors"
+	"math/bits"
+)
+
+const (
+	prime1 uint64 = 11400714785074694791
+	prime2 uint64 = 14029467366897019727
+	prime3 uint64 = 1609587929392839161
+	prime4 uint64 = 9650029242287828579
+	prime5 uint64 = 2870177450012600261
+)
+
+// NOTE(caleb): I'm using both consts and vars of the primes. Using consts where
+// possible in the Go code is worth a small (but measurable) performance boost
+// by avoiding some MOVQs. Vars are needed for the asm and also are useful for
+// convenience in the Go code in a few places where we need to intentionally
+// avoid constant arithmetic (e.g., v1 := prime1 + prime2 fails because the
+// result overflows a uint64).
+var (
+	prime1v = prime1
+	prime2v = prime2
+	prime3v = prime3
+	prime4v = prime4
+	prime5v = prime5
+)
+
+// Digest implements hash.Hash64.
+type Digest struct {
+	v1    uint64
+	v2    uint64
+	v3    uint64
+	v4    uint64
+	total uint64
+	mem   [32]byte
+	n     int // how much of mem is used
+}
+
+// New creates a new Digest that computes the 64-bit xxHash algorithm.
+func New() *Digest {
+	var d Digest
+	d.Reset()
+	return &d
+}
+
+// Reset clears the Digest's state so that it can be reused.
+func (d *Digest) Reset() {
+	d.v1 = prime1v + prime2
+	d.v2 = prime2
+	d.v3 = 0
+	d.v4 = -prime1v
+	d.total = 0
+	d.n = 0
+}
+
+// Size always returns 8 bytes.
+func (d *Digest) Size() int { return 8 }
+
+// BlockSize always returns 32 bytes.
+func (d *Digest) BlockSize() int { return 32 }
+
+// Write adds more data to d. It always returns len(b), nil.
+func (d *Digest) Write(b []byte) (n int, err error) {
+	n = len(b)
+	d.total += uint64(n)
+
+	if d.n+n < 32 {
+		// This new data doesn't even fill the current block.
+		copy(d.mem[d.n:], b)
+		d.n += n
+		return
+	}
+
+	if d.n > 0 {
+		// Finish off the partial block.
+		copy(d.mem[d.n:], b)
+		d.v1 = round(d.v1, u64(d.mem[0:8]))
+		d.v2 = round(d.v2, u64(d.mem[8:16]))
+		d.v3 = round(d.v3, u64(d.mem[16:24]))
+		d.v4 = round(d.v4, u64(d.mem[24:32]))
+		b = b[32-d.n:]
+		d.n = 0
+	}
+
+	if len(b) >= 32 {
+		// One or more full blocks left.
+		nw := writeBlocks(d, b)
+		b = b[nw:]
+	}
+
+	// Store any remaining partial block.
+	copy(d.mem[:], b)
+	d.n = len(b)
+
+	return
+}
+
+// Sum appends the current hash to b and returns the resulting slice.
+func (d *Digest) Sum(b []byte) []byte {
+	s := d.Sum64()
+	return append(
+		b,
+		byte(s>>56),
+		byte(s>>48),
+		byte(s>>40),
+		byte(s>>32),
+		byte(s>>24),
+		byte(s>>16),
+		byte(s>>8),
+		byte(s),
+	)
+}
+
+// Sum64 returns the current hash.
+func (d *Digest) Sum64() uint64 {
+	var h uint64
+
+	if d.total >= 32 {
+		v1, v2, v3, v4 := d.v1, d.v2, d.v3, d.v4
+		h = rol1(v1) + rol7(v2) + rol12(v3) + rol18(v4)
+		h = mergeRound(h, v1)
+		h = mergeRound(h, v2)
+		h = mergeRound(h, v3)
+		h = mergeRound(h, v4)
+	} else {
+		h = d.v3 + prime5
+	}
+
+	h += d.total
+
+	i, end := 0, d.n
+	for ; i+8 <= end; i += 8 {
+		k1 := round(0, u64(d.mem[i:i+8]))
+		h ^= k1
+		h = rol27(h)*prime1 + prime4
+	}
+	if i+4 <= end {
+		h ^= uint64(u32(d.mem[i:i+4])) * prime1
+		h = rol23(h)*prime2 + prime3
+		i += 4
+	}
+	for i < end {
+		h ^= uint64(d.mem[i]) * prime5
+		h = rol11(h) * prime1
+		i++
+	}
+
+	h ^= h >> 33
+	h *= prime2
+	h ^= h >> 29
+	h *= prime3
+	h ^= h >> 32
+
+	return h
+}
+
+const (
+	magic         = "xxh\x06"
+	marshaledSize = len(magic) + 8*5 + 32
+)
+
+// MarshalBinary implements the encoding.BinaryMarshaler interface.
+func (d *Digest) MarshalBinary() ([]byte, error) {
+	b := make([]byte, 0, marshaledSize)
+	b = append(b, magic...)
+	b = appendUint64(b, d.v1)
+	b = appendUint64(b, d.v2)
+	b = appendUint64(b, d.v3)
+	b = appendUint64(b, d.v4)
+	b = appendUint64(b, d.total)
+	b = append(b, d.mem[:d.n]...)
+	b = b[:len(b)+len(d.mem)-d.n]
+	return b, nil
+}
+
+// UnmarshalBinary implements the encoding.BinaryUnmarshaler interface.
+func (d *Digest) UnmarshalBinary(b []byte) error {
+	if len(b) < len(magic) || string(b[:len(magic)]) != magic {
+		return errors.New("xxhash: invalid hash state identifier")
+	}
+	if len(b) != marshaledSize {
+		return errors.New("xxhash: invalid hash state size")
+	}
+	b = b[len(magic):]
+	b, d.v1 = consumeUint64(b)
+	b, d.v2 = consumeUint64(b)
+	b, d.v3 = consumeUint64(b)
+	b, d.v4 = consumeUint64(b)
+	b, d.total = consumeUint64(b)
+	copy(d.mem[:], b)
+	b = b[len(d.mem):]
+	d.n = int(d.total % uint64(len(d.mem)))
+	return nil
+}
+
+func appendUint64(b []byte, x uint64) []byte {
+	var a [8]byte
+	binary.LittleEndian.PutUint64(a[:], x)
+	return append(b, a[:]...)
+}
+
+func consumeUint64(b []byte) ([]byte, uint64) {
+	x := u64(b)
+	return b[8:], x
+}
+
+func u64(b []byte) uint64 { return binary.LittleEndian.Uint64(b) }
+func u32(b []byte) uint32 { return binary.LittleEndian.Uint32(b) }
+
+func round(acc, input uint64) uint64 {
+	acc += input * prime2
+	acc = rol31(acc)
+	acc *= prime1
+	return acc
+}
+
+func mergeRound(acc, val uint64) uint64 {
+	val = round(0, val)
+	acc ^= val
+	acc = acc*prime1 + prime4
+	return acc
+}
+
+func rol1(x uint64) uint64  { return bits.RotateLeft64(x, 1) }
+func rol7(x uint64) uint64  { return bits.RotateLeft64(x, 7) }
+func rol11(x uint64) uint64 { return bits.RotateLeft64(x, 11) }
+func rol12(x uint64) uint64 { return bits.RotateLeft64(x, 12) }
+func rol18(x uint64) uint64 { return bits.RotateLeft64(x, 18) }
+func rol23(x uint64) uint64 { return bits.RotateLeft64(x, 23) }
+func rol27(x uint64) uint64 { return bits.RotateLeft64(x, 27) }
+func rol31(x uint64) uint64 { return bits.RotateLeft64(x, 31) }
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go
new file mode 100644
index 0000000..35318d7
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go
@@ -0,0 +1,13 @@
+// +build !appengine
+// +build gc
+// +build !purego
+
+package xxhash
+
+// Sum64 computes the 64-bit xxHash digest of b.
+//
+//go:noescape
+func Sum64(b []byte) uint64
+
+//go:noescape
+func writeBlocks(*Digest, []byte) int
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s
new file mode 100644
index 0000000..2c9c535
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s
@@ -0,0 +1,215 @@
+// +build !appengine
+// +build gc
+// +build !purego
+
+#include "textflag.h"
+
+// Register allocation:
+// AX	h
+// CX	pointer to advance through b
+// DX	n
+// BX	loop end
+// R8	v1, k1
+// R9	v2
+// R10	v3
+// R11	v4
+// R12	tmp
+// R13	prime1v
+// R14	prime2v
+// R15	prime4v
+
+// round reads from and advances the buffer pointer in CX.
+// It assumes that R13 has prime1v and R14 has prime2v.
+#define round(r) \
+	MOVQ  (CX), R12 \
+	ADDQ  $8, CX    \
+	IMULQ R14, R12  \
+	ADDQ  R12, r    \
+	ROLQ  $31, r    \
+	IMULQ R13, r
+
+// mergeRound applies a merge round on the two registers acc and val.
+// It assumes that R13 has prime1v, R14 has prime2v, and R15 has prime4v.
+#define mergeRound(acc, val) \
+	IMULQ R14, val \
+	ROLQ  $31, val \
+	IMULQ R13, val \
+	XORQ  val, acc \
+	IMULQ R13, acc \
+	ADDQ  R15, acc
+
+// func Sum64(b []byte) uint64
+TEXT ·Sum64(SB), NOSPLIT, $0-32
+	// Load fixed primes.
+	MOVQ ·prime1v(SB), R13
+	MOVQ ·prime2v(SB), R14
+	MOVQ ·prime4v(SB), R15
+
+	// Load slice.
+	MOVQ b_base+0(FP), CX
+	MOVQ b_len+8(FP), DX
+	LEAQ (CX)(DX*1), BX
+
+	// The first loop limit will be len(b)-32.
+	SUBQ $32, BX
+
+	// Check whether we have at least one block.
+	CMPQ DX, $32
+	JLT  noBlocks
+
+	// Set up initial state (v1, v2, v3, v4).
+	MOVQ R13, R8
+	ADDQ R14, R8
+	MOVQ R14, R9
+	XORQ R10, R10
+	XORQ R11, R11
+	SUBQ R13, R11
+
+	// Loop until CX > BX.
+blockLoop:
+	round(R8)
+	round(R9)
+	round(R10)
+	round(R11)
+
+	CMPQ CX, BX
+	JLE  blockLoop
+
+	MOVQ R8, AX
+	ROLQ $1, AX
+	MOVQ R9, R12
+	ROLQ $7, R12
+	ADDQ R12, AX
+	MOVQ R10, R12
+	ROLQ $12, R12
+	ADDQ R12, AX
+	MOVQ R11, R12
+	ROLQ $18, R12
+	ADDQ R12, AX
+
+	mergeRound(AX, R8)
+	mergeRound(AX, R9)
+	mergeRound(AX, R10)
+	mergeRound(AX, R11)
+
+	JMP afterBlocks
+
+noBlocks:
+	MOVQ ·prime5v(SB), AX
+
+afterBlocks:
+	ADDQ DX, AX
+
+	// Right now BX has len(b)-32, and we want to loop until CX > len(b)-8.
+	ADDQ $24, BX
+
+	CMPQ CX, BX
+	JG   fourByte
+
+wordLoop:
+	// Calculate k1.
+	MOVQ  (CX), R8
+	ADDQ  $8, CX
+	IMULQ R14, R8
+	ROLQ  $31, R8
+	IMULQ R13, R8
+
+	XORQ  R8, AX
+	ROLQ  $27, AX
+	IMULQ R13, AX
+	ADDQ  R15, AX
+
+	CMPQ CX, BX
+	JLE  wordLoop
+
+fourByte:
+	ADDQ $4, BX
+	CMPQ CX, BX
+	JG   singles
+
+	MOVL  (CX), R8
+	ADDQ  $4, CX
+	IMULQ R13, R8
+	XORQ  R8, AX
+
+	ROLQ  $23, AX
+	IMULQ R14, AX
+	ADDQ  ·prime3v(SB), AX
+
+singles:
+	ADDQ $4, BX
+	CMPQ CX, BX
+	JGE  finalize
+
+singlesLoop:
+	MOVBQZX (CX), R12
+	ADDQ    $1, CX
+	IMULQ   ·prime5v(SB), R12
+	XORQ    R12, AX
+
+	ROLQ  $11, AX
+	IMULQ R13, AX
+
+	CMPQ CX, BX
+	JL   singlesLoop
+
+finalize:
+	MOVQ  AX, R12
+	SHRQ  $33, R12
+	XORQ  R12, AX
+	IMULQ R14, AX
+	MOVQ  AX, R12
+	SHRQ  $29, R12
+	XORQ  R12, AX
+	IMULQ ·prime3v(SB), AX
+	MOVQ  AX, R12
+	SHRQ  $32, R12
+	XORQ  R12, AX
+
+	MOVQ AX, ret+24(FP)
+	RET
+
+// writeBlocks uses the same registers as above except that it uses AX to store
+// the d pointer.
+
+// func writeBlocks(d *Digest, b []byte) int
+TEXT ·writeBlocks(SB), NOSPLIT, $0-40
+	// Load fixed primes needed for round.
+	MOVQ ·prime1v(SB), R13
+	MOVQ ·prime2v(SB), R14
+
+	// Load slice.
+	MOVQ arg1_base+8(FP), CX
+	MOVQ arg1_len+16(FP), DX
+	LEAQ (CX)(DX*1), BX
+	SUBQ $32, BX
+
+	// Load vN from d.
+	MOVQ arg+0(FP), AX
+	MOVQ 0(AX), R8   // v1
+	MOVQ 8(AX), R9   // v2
+	MOVQ 16(AX), R10 // v3
+	MOVQ 24(AX), R11 // v4
+
+	// We don't need to check the loop condition here; this function is
+	// always called with at least one block of data to process.
+blockLoop:
+	round(R8)
+	round(R9)
+	round(R10)
+	round(R11)
+
+	CMPQ CX, BX
+	JLE  blockLoop
+
+	// Copy vN back to d.
+	MOVQ R8, 0(AX)
+	MOVQ R9, 8(AX)
+	MOVQ R10, 16(AX)
+	MOVQ R11, 24(AX)
+
+	// The number of bytes written is CX minus the old base pointer.
+	SUBQ arg1_base+8(FP), CX
+	MOVQ CX, ret+32(FP)
+
+	RET
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go
new file mode 100644
index 0000000..4a5a821
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go
@@ -0,0 +1,76 @@
+// +build !amd64 appengine !gc purego
+
+package xxhash
+
+// Sum64 computes the 64-bit xxHash digest of b.
+func Sum64(b []byte) uint64 {
+	// A simpler version would be
+	//   d := New()
+	//   d.Write(b)
+	//   return d.Sum64()
+	// but this is faster, particularly for small inputs.
+
+	n := len(b)
+	var h uint64
+
+	if n >= 32 {
+		v1 := prime1v + prime2
+		v2 := prime2
+		v3 := uint64(0)
+		v4 := -prime1v
+		for len(b) >= 32 {
+			v1 = round(v1, u64(b[0:8:len(b)]))
+			v2 = round(v2, u64(b[8:16:len(b)]))
+			v3 = round(v3, u64(b[16:24:len(b)]))
+			v4 = round(v4, u64(b[24:32:len(b)]))
+			b = b[32:len(b):len(b)]
+		}
+		h = rol1(v1) + rol7(v2) + rol12(v3) + rol18(v4)
+		h = mergeRound(h, v1)
+		h = mergeRound(h, v2)
+		h = mergeRound(h, v3)
+		h = mergeRound(h, v4)
+	} else {
+		h = prime5
+	}
+
+	h += uint64(n)
+
+	i, end := 0, len(b)
+	for ; i+8 <= end; i += 8 {
+		k1 := round(0, u64(b[i:i+8:len(b)]))
+		h ^= k1
+		h = rol27(h)*prime1 + prime4
+	}
+	if i+4 <= end {
+		h ^= uint64(u32(b[i:i+4:len(b)])) * prime1
+		h = rol23(h)*prime2 + prime3
+		i += 4
+	}
+	for ; i < end; i++ {
+		h ^= uint64(b[i]) * prime5
+		h = rol11(h) * prime1
+	}
+
+	h ^= h >> 33
+	h *= prime2
+	h ^= h >> 29
+	h *= prime3
+	h ^= h >> 32
+
+	return h
+}
+
+func writeBlocks(d *Digest, b []byte) int {
+	v1, v2, v3, v4 := d.v1, d.v2, d.v3, d.v4
+	n := len(b)
+	for len(b) >= 32 {
+		v1 = round(v1, u64(b[0:8:len(b)]))
+		v2 = round(v2, u64(b[8:16:len(b)]))
+		v3 = round(v3, u64(b[16:24:len(b)]))
+		v4 = round(v4, u64(b[24:32:len(b)]))
+		b = b[32:len(b):len(b)]
+	}
+	d.v1, d.v2, d.v3, d.v4 = v1, v2, v3, v4
+	return n - len(b)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_safe.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_safe.go
new file mode 100644
index 0000000..6f3b0cb
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_safe.go
@@ -0,0 +1,11 @@
+package xxhash
+
+// Sum64String computes the 64-bit xxHash digest of s.
+func Sum64String(s string) uint64 {
+	return Sum64([]byte(s))
+}
+
+// WriteString adds more data to d. It always returns len(s), nil.
+func (d *Digest) WriteString(s string) (n int, err error) {
+	return d.Write([]byte(s))
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/seqdec.go b/vendor/github.com/klauspost/compress/zstd/seqdec.go
new file mode 100644
index 0000000..1dd39e6
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/seqdec.go
@@ -0,0 +1,492 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"errors"
+	"fmt"
+	"io"
+)
+
+type seq struct {
+	litLen   uint32
+	matchLen uint32
+	offset   uint32
+
+	// Codes are stored here for the encoder
+	// so they only have to be looked up once.
+	llCode, mlCode, ofCode uint8
+}
+
+func (s seq) String() string {
+	if s.offset <= 3 {
+		if s.offset == 0 {
+			return fmt.Sprint("litLen:", s.litLen, ", matchLen:", s.matchLen+zstdMinMatch, ", offset: INVALID (0)")
+		}
+		return fmt.Sprint("litLen:", s.litLen, ", matchLen:", s.matchLen+zstdMinMatch, ", offset:", s.offset, " (repeat)")
+	}
+	return fmt.Sprint("litLen:", s.litLen, ", matchLen:", s.matchLen+zstdMinMatch, ", offset:", s.offset-3, " (new)")
+}
+
+type seqCompMode uint8
+
+const (
+	compModePredefined seqCompMode = iota
+	compModeRLE
+	compModeFSE
+	compModeRepeat
+)
+
+type sequenceDec struct {
+	// decoder keeps track of the current state and updates it from the bitstream.
+	fse    *fseDecoder
+	state  fseState
+	repeat bool
+}
+
+// init the state of the decoder with input from stream.
+func (s *sequenceDec) init(br *bitReader) error {
+	if s.fse == nil {
+		return errors.New("sequence decoder not defined")
+	}
+	s.state.init(br, s.fse.actualTableLog, s.fse.dt[:1<<s.fse.actualTableLog])
+	return nil
+}
+
+// sequenceDecs contains all 3 sequence decoders and their state.
+type sequenceDecs struct {
+	litLengths   sequenceDec
+	offsets      sequenceDec
+	matchLengths sequenceDec
+	prevOffset   [3]int
+	hist         []byte
+	dict         []byte
+	literals     []byte
+	out          []byte
+	windowSize   int
+	maxBits      uint8
+}
+
+// initialize all 3 decoders from the stream input.
+func (s *sequenceDecs) initialize(br *bitReader, hist *history, literals, out []byte) error {
+	if err := s.litLengths.init(br); err != nil {
+		return errors.New("litLengths:" + err.Error())
+	}
+	if err := s.offsets.init(br); err != nil {
+		return errors.New("offsets:" + err.Error())
+	}
+	if err := s.matchLengths.init(br); err != nil {
+		return errors.New("matchLengths:" + err.Error())
+	}
+	s.literals = literals
+	s.hist = hist.b
+	s.prevOffset = hist.recentOffsets
+	s.maxBits = s.litLengths.fse.maxBits + s.offsets.fse.maxBits + s.matchLengths.fse.maxBits
+	s.windowSize = hist.windowSize
+	s.out = out
+	s.dict = nil
+	if hist.dict != nil {
+		s.dict = hist.dict.content
+	}
+	return nil
+}
+
+// decode sequences from the stream with the provided history.
+func (s *sequenceDecs) decode(seqs int, br *bitReader, hist []byte) error {
+	startSize := len(s.out)
+	// Grab full sizes tables, to avoid bounds checks.
+	llTable, mlTable, ofTable := s.litLengths.fse.dt[:maxTablesize], s.matchLengths.fse.dt[:maxTablesize], s.offsets.fse.dt[:maxTablesize]
+	llState, mlState, ofState := s.litLengths.state.state, s.matchLengths.state.state, s.offsets.state.state
+
+	for i := seqs - 1; i >= 0; i-- {
+		if br.overread() {
+			printf("reading sequence %d, exceeded available data\n", seqs-i)
+			return io.ErrUnexpectedEOF
+		}
+		var ll, mo, ml int
+		if br.off > 4+((maxOffsetBits+16+16)>>3) {
+			// inlined function:
+			// ll, mo, ml = s.nextFast(br, llState, mlState, ofState)
+
+			// Final will not read from stream.
+			var llB, mlB, moB uint8
+			ll, llB = llState.final()
+			ml, mlB = mlState.final()
+			mo, moB = ofState.final()
+
+			// extra bits are stored in reverse order.
+			br.fillFast()
+			mo += br.getBits(moB)
+			if s.maxBits > 32 {
+				br.fillFast()
+			}
+			ml += br.getBits(mlB)
+			ll += br.getBits(llB)
+
+			if moB > 1 {
+				s.prevOffset[2] = s.prevOffset[1]
+				s.prevOffset[1] = s.prevOffset[0]
+				s.prevOffset[0] = mo
+			} else {
+				// mo = s.adjustOffset(mo, ll, moB)
+				// Inlined for rather big speedup
+				if ll == 0 {
+					// There is an exception though, when current sequence's literals_length = 0.
+					// In this case, repeated offsets are shifted by one, so an offset_value of 1 means Repeated_Offset2,
+					// an offset_value of 2 means Repeated_Offset3, and an offset_value of 3 means Repeated_Offset1 - 1_byte.
+					mo++
+				}
+
+				if mo == 0 {
+					mo = s.prevOffset[0]
+				} else {
+					var temp int
+					if mo == 3 {
+						temp = s.prevOffset[0] - 1
+					} else {
+						temp = s.prevOffset[mo]
+					}
+
+					if temp == 0 {
+						// 0 is not valid; input is corrupted; force offset to 1
+						println("temp was 0")
+						temp = 1
+					}
+
+					if mo != 1 {
+						s.prevOffset[2] = s.prevOffset[1]
+					}
+					s.prevOffset[1] = s.prevOffset[0]
+					s.prevOffset[0] = temp
+					mo = temp
+				}
+			}
+			br.fillFast()
+		} else {
+			ll, mo, ml = s.next(br, llState, mlState, ofState)
+			br.fill()
+		}
+
+		if debugSequences {
+			println("Seq", seqs-i-1, "Litlen:", ll, "mo:", mo, "(abs) ml:", ml)
+		}
+
+		if ll > len(s.literals) {
+			return fmt.Errorf("unexpected literal count, want %d bytes, but only %d is available", ll, len(s.literals))
+		}
+		size := ll + ml + len(s.out)
+		if size-startSize > maxBlockSize {
+			return fmt.Errorf("output (%d) bigger than max block size", size)
+		}
+		if size > cap(s.out) {
+			// Not enough size, which can happen under high volume block streaming conditions
+			// but could be if destination slice is too small for sync operations.
+			// over-allocating here can create a large amount of GC pressure so we try to keep
+			// it as contained as possible
+			used := len(s.out) - startSize
+			addBytes := 256 + ll + ml + used>>2
+			// Clamp to max block size.
+			if used+addBytes > maxBlockSize {
+				addBytes = maxBlockSize - used
+			}
+			s.out = append(s.out, make([]byte, addBytes)...)
+			s.out = s.out[:len(s.out)-addBytes]
+		}
+		if ml > maxMatchLen {
+			return fmt.Errorf("match len (%d) bigger than max allowed length", ml)
+		}
+
+		// Add literals
+		s.out = append(s.out, s.literals[:ll]...)
+		s.literals = s.literals[ll:]
+		out := s.out
+
+		if mo == 0 && ml > 0 {
+			return fmt.Errorf("zero matchoff and matchlen (%d) > 0", ml)
+		}
+
+		if mo > len(s.out)+len(hist) || mo > s.windowSize {
+			if len(s.dict) == 0 {
+				return fmt.Errorf("match offset (%d) bigger than current history (%d)", mo, len(s.out)+len(hist))
+			}
+
+			// we may be in dictionary.
+			dictO := len(s.dict) - (mo - (len(s.out) + len(hist)))
+			if dictO < 0 || dictO >= len(s.dict) {
+				return fmt.Errorf("match offset (%d) bigger than current history (%d)", mo, len(s.out)+len(hist))
+			}
+			end := dictO + ml
+			if end > len(s.dict) {
+				out = append(out, s.dict[dictO:]...)
+				mo -= len(s.dict) - dictO
+				ml -= len(s.dict) - dictO
+			} else {
+				out = append(out, s.dict[dictO:end]...)
+				mo = 0
+				ml = 0
+			}
+		}
+
+		// Copy from history.
+		// TODO: Blocks without history could be made to ignore this completely.
+		if v := mo - len(s.out); v > 0 {
+			// v is the start position in history from end.
+			start := len(s.hist) - v
+			if ml > v {
+				// Some goes into current block.
+				// Copy remainder of history
+				out = append(out, s.hist[start:]...)
+				mo -= v
+				ml -= v
+			} else {
+				out = append(out, s.hist[start:start+ml]...)
+				ml = 0
+			}
+		}
+		// We must be in current buffer now
+		if ml > 0 {
+			start := len(s.out) - mo
+			if ml <= len(s.out)-start {
+				// No overlap
+				out = append(out, s.out[start:start+ml]...)
+			} else {
+				// Overlapping copy
+				// Extend destination slice and copy one byte at the time.
+				out = out[:len(out)+ml]
+				src := out[start : start+ml]
+				// Destination is the space we just added.
+				dst := out[len(out)-ml:]
+				dst = dst[:len(src)]
+				for i := range src {
+					dst[i] = src[i]
+				}
+			}
+		}
+		s.out = out
+		if i == 0 {
+			// This is the last sequence, so we shouldn't update state.
+			break
+		}
+
+		// Manually inlined, ~ 5-20% faster
+		// Update all 3 states at once. Approx 20% faster.
+		nBits := llState.nbBits() + mlState.nbBits() + ofState.nbBits()
+		if nBits == 0 {
+			llState = llTable[llState.newState()&maxTableMask]
+			mlState = mlTable[mlState.newState()&maxTableMask]
+			ofState = ofTable[ofState.newState()&maxTableMask]
+		} else {
+			bits := br.getBitsFast(nBits)
+			lowBits := uint16(bits >> ((ofState.nbBits() + mlState.nbBits()) & 31))
+			llState = llTable[(llState.newState()+lowBits)&maxTableMask]
+
+			lowBits = uint16(bits >> (ofState.nbBits() & 31))
+			lowBits &= bitMask[mlState.nbBits()&15]
+			mlState = mlTable[(mlState.newState()+lowBits)&maxTableMask]
+
+			lowBits = uint16(bits) & bitMask[ofState.nbBits()&15]
+			ofState = ofTable[(ofState.newState()+lowBits)&maxTableMask]
+		}
+	}
+
+	// Add final literals
+	s.out = append(s.out, s.literals...)
+	return nil
+}
+
+// update states, at least 27 bits must be available.
+func (s *sequenceDecs) update(br *bitReader) {
+	// Max 8 bits
+	s.litLengths.state.next(br)
+	// Max 9 bits
+	s.matchLengths.state.next(br)
+	// Max 8 bits
+	s.offsets.state.next(br)
+}
+
+var bitMask [16]uint16
+
+func init() {
+	for i := range bitMask[:] {
+		bitMask[i] = uint16((1 << uint(i)) - 1)
+	}
+}
+
+// update states, at least 27 bits must be available.
+func (s *sequenceDecs) updateAlt(br *bitReader) {
+	// Update all 3 states at once. Approx 20% faster.
+	a, b, c := s.litLengths.state.state, s.matchLengths.state.state, s.offsets.state.state
+
+	nBits := a.nbBits() + b.nbBits() + c.nbBits()
+	if nBits == 0 {
+		s.litLengths.state.state = s.litLengths.state.dt[a.newState()]
+		s.matchLengths.state.state = s.matchLengths.state.dt[b.newState()]
+		s.offsets.state.state = s.offsets.state.dt[c.newState()]
+		return
+	}
+	bits := br.getBitsFast(nBits)
+	lowBits := uint16(bits >> ((c.nbBits() + b.nbBits()) & 31))
+	s.litLengths.state.state = s.litLengths.state.dt[a.newState()+lowBits]
+
+	lowBits = uint16(bits >> (c.nbBits() & 31))
+	lowBits &= bitMask[b.nbBits()&15]
+	s.matchLengths.state.state = s.matchLengths.state.dt[b.newState()+lowBits]
+
+	lowBits = uint16(bits) & bitMask[c.nbBits()&15]
+	s.offsets.state.state = s.offsets.state.dt[c.newState()+lowBits]
+}
+
+// nextFast will return new states when there are at least 4 unused bytes left on the stream when done.
+func (s *sequenceDecs) nextFast(br *bitReader, llState, mlState, ofState decSymbol) (ll, mo, ml int) {
+	// Final will not read from stream.
+	ll, llB := llState.final()
+	ml, mlB := mlState.final()
+	mo, moB := ofState.final()
+
+	// extra bits are stored in reverse order.
+	br.fillFast()
+	mo += br.getBits(moB)
+	if s.maxBits > 32 {
+		br.fillFast()
+	}
+	ml += br.getBits(mlB)
+	ll += br.getBits(llB)
+
+	if moB > 1 {
+		s.prevOffset[2] = s.prevOffset[1]
+		s.prevOffset[1] = s.prevOffset[0]
+		s.prevOffset[0] = mo
+		return
+	}
+	// mo = s.adjustOffset(mo, ll, moB)
+	// Inlined for rather big speedup
+	if ll == 0 {
+		// There is an exception though, when current sequence's literals_length = 0.
+		// In this case, repeated offsets are shifted by one, so an offset_value of 1 means Repeated_Offset2,
+		// an offset_value of 2 means Repeated_Offset3, and an offset_value of 3 means Repeated_Offset1 - 1_byte.
+		mo++
+	}
+
+	if mo == 0 {
+		mo = s.prevOffset[0]
+		return
+	}
+	var temp int
+	if mo == 3 {
+		temp = s.prevOffset[0] - 1
+	} else {
+		temp = s.prevOffset[mo]
+	}
+
+	if temp == 0 {
+		// 0 is not valid; input is corrupted; force offset to 1
+		println("temp was 0")
+		temp = 1
+	}
+
+	if mo != 1 {
+		s.prevOffset[2] = s.prevOffset[1]
+	}
+	s.prevOffset[1] = s.prevOffset[0]
+	s.prevOffset[0] = temp
+	mo = temp
+	return
+}
+
+func (s *sequenceDecs) next(br *bitReader, llState, mlState, ofState decSymbol) (ll, mo, ml int) {
+	// Final will not read from stream.
+	ll, llB := llState.final()
+	ml, mlB := mlState.final()
+	mo, moB := ofState.final()
+
+	// extra bits are stored in reverse order.
+	br.fill()
+	if s.maxBits <= 32 {
+		mo += br.getBits(moB)
+		ml += br.getBits(mlB)
+		ll += br.getBits(llB)
+	} else {
+		mo += br.getBits(moB)
+		br.fill()
+		// matchlength+literal length, max 32 bits
+		ml += br.getBits(mlB)
+		ll += br.getBits(llB)
+
+	}
+	mo = s.adjustOffset(mo, ll, moB)
+	return
+}
+
+func (s *sequenceDecs) adjustOffset(offset, litLen int, offsetB uint8) int {
+	if offsetB > 1 {
+		s.prevOffset[2] = s.prevOffset[1]
+		s.prevOffset[1] = s.prevOffset[0]
+		s.prevOffset[0] = offset
+		return offset
+	}
+
+	if litLen == 0 {
+		// There is an exception though, when current sequence's literals_length = 0.
+		// In this case, repeated offsets are shifted by one, so an offset_value of 1 means Repeated_Offset2,
+		// an offset_value of 2 means Repeated_Offset3, and an offset_value of 3 means Repeated_Offset1 - 1_byte.
+		offset++
+	}
+
+	if offset == 0 {
+		return s.prevOffset[0]
+	}
+	var temp int
+	if offset == 3 {
+		temp = s.prevOffset[0] - 1
+	} else {
+		temp = s.prevOffset[offset]
+	}
+
+	if temp == 0 {
+		// 0 is not valid; input is corrupted; force offset to 1
+		println("temp was 0")
+		temp = 1
+	}
+
+	if offset != 1 {
+		s.prevOffset[2] = s.prevOffset[1]
+	}
+	s.prevOffset[1] = s.prevOffset[0]
+	s.prevOffset[0] = temp
+	return temp
+}
+
+// mergeHistory will merge history.
+func (s *sequenceDecs) mergeHistory(hist *sequenceDecs) (*sequenceDecs, error) {
+	for i := uint(0); i < 3; i++ {
+		var sNew, sHist *sequenceDec
+		switch i {
+		default:
+			// same as "case 0":
+			sNew = &s.litLengths
+			sHist = &hist.litLengths
+		case 1:
+			sNew = &s.offsets
+			sHist = &hist.offsets
+		case 2:
+			sNew = &s.matchLengths
+			sHist = &hist.matchLengths
+		}
+		if sNew.repeat {
+			if sHist.fse == nil {
+				return nil, fmt.Errorf("sequence stream %d, repeat requested, but no history", i)
+			}
+			continue
+		}
+		if sNew.fse == nil {
+			return nil, fmt.Errorf("sequence stream %d, no fse found", i)
+		}
+		if sHist.fse != nil && !sHist.fse.preDefined {
+			fseDecoderPool.Put(sHist.fse)
+		}
+		sHist.fse = sNew.fse
+	}
+	return hist, nil
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/seqenc.go b/vendor/github.com/klauspost/compress/zstd/seqenc.go
new file mode 100644
index 0000000..8014174
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/seqenc.go
@@ -0,0 +1,114 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import "math/bits"
+
+type seqCoders struct {
+	llEnc, ofEnc, mlEnc    *fseEncoder
+	llPrev, ofPrev, mlPrev *fseEncoder
+}
+
+// swap coders with another (block).
+func (s *seqCoders) swap(other *seqCoders) {
+	*s, *other = *other, *s
+}
+
+// setPrev will update the previous encoders to the actually used ones
+// and make sure a fresh one is in the main slot.
+func (s *seqCoders) setPrev(ll, ml, of *fseEncoder) {
+	compareSwap := func(used *fseEncoder, current, prev **fseEncoder) {
+		// We used the new one, more current to history and reuse the previous history
+		if *current == used {
+			*prev, *current = *current, *prev
+			c := *current
+			p := *prev
+			c.reUsed = false
+			p.reUsed = true
+			return
+		}
+		if used == *prev {
+			return
+		}
+		// Ensure we cannot reuse by accident
+		prevEnc := *prev
+		prevEnc.symbolLen = 0
+	}
+	compareSwap(ll, &s.llEnc, &s.llPrev)
+	compareSwap(ml, &s.mlEnc, &s.mlPrev)
+	compareSwap(of, &s.ofEnc, &s.ofPrev)
+}
+
+func highBit(val uint32) (n uint32) {
+	return uint32(bits.Len32(val) - 1)
+}
+
+var llCodeTable = [64]byte{0, 1, 2, 3, 4, 5, 6, 7,
+	8, 9, 10, 11, 12, 13, 14, 15,
+	16, 16, 17, 17, 18, 18, 19, 19,
+	20, 20, 20, 20, 21, 21, 21, 21,
+	22, 22, 22, 22, 22, 22, 22, 22,
+	23, 23, 23, 23, 23, 23, 23, 23,
+	24, 24, 24, 24, 24, 24, 24, 24,
+	24, 24, 24, 24, 24, 24, 24, 24}
+
+// Up to 6 bits
+const maxLLCode = 35
+
+// llBitsTable translates from ll code to number of bits.
+var llBitsTable = [maxLLCode + 1]byte{
+	0, 0, 0, 0, 0, 0, 0, 0,
+	0, 0, 0, 0, 0, 0, 0, 0,
+	1, 1, 1, 1, 2, 2, 3, 3,
+	4, 6, 7, 8, 9, 10, 11, 12,
+	13, 14, 15, 16}
+
+// llCode returns the code that represents the literal length requested.
+func llCode(litLength uint32) uint8 {
+	const llDeltaCode = 19
+	if litLength <= 63 {
+		// Compiler insists on bounds check (Go 1.12)
+		return llCodeTable[litLength&63]
+	}
+	return uint8(highBit(litLength)) + llDeltaCode
+}
+
+var mlCodeTable = [128]byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15,
+	16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31,
+	32, 32, 33, 33, 34, 34, 35, 35, 36, 36, 36, 36, 37, 37, 37, 37,
+	38, 38, 38, 38, 38, 38, 38, 38, 39, 39, 39, 39, 39, 39, 39, 39,
+	40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40,
+	41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41,
+	42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42,
+	42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}
+
+// Up to 6 bits
+const maxMLCode = 52
+
+// mlBitsTable translates from ml code to number of bits.
+var mlBitsTable = [maxMLCode + 1]byte{
+	0, 0, 0, 0, 0, 0, 0, 0,
+	0, 0, 0, 0, 0, 0, 0, 0,
+	0, 0, 0, 0, 0, 0, 0, 0,
+	0, 0, 0, 0, 0, 0, 0, 0,
+	1, 1, 1, 1, 2, 2, 3, 3,
+	4, 4, 5, 7, 8, 9, 10, 11,
+	12, 13, 14, 15, 16}
+
+// note : mlBase = matchLength - MINMATCH;
+// because it's the format it's stored in seqStore->sequences
+func mlCode(mlBase uint32) uint8 {
+	const mlDeltaCode = 36
+	if mlBase <= 127 {
+		// Compiler insists on bounds check (Go 1.12)
+		return mlCodeTable[mlBase&127]
+	}
+	return uint8(highBit(mlBase)) + mlDeltaCode
+}
+
+func ofCode(offset uint32) uint8 {
+	// A valid offset will always be > 0.
+	return uint8(bits.Len32(offset) - 1)
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/snappy.go b/vendor/github.com/klauspost/compress/zstd/snappy.go
new file mode 100644
index 0000000..9d9d1d5
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/snappy.go
@@ -0,0 +1,435 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+// Based on work by Yann Collet, released under BSD License.
+
+package zstd
+
+import (
+	"encoding/binary"
+	"errors"
+	"hash/crc32"
+	"io"
+
+	"github.com/golang/snappy"
+	"github.com/klauspost/compress/huff0"
+)
+
+const (
+	snappyTagLiteral = 0x00
+	snappyTagCopy1   = 0x01
+	snappyTagCopy2   = 0x02
+	snappyTagCopy4   = 0x03
+)
+
+const (
+	snappyChecksumSize = 4
+	snappyMagicBody    = "sNaPpY"
+
+	// snappyMaxBlockSize is the maximum size of the input to encodeBlock. It is not
+	// part of the wire format per se, but some parts of the encoder assume
+	// that an offset fits into a uint16.
+	//
+	// Also, for the framing format (Writer type instead of Encode function),
+	// https://github.com/google/snappy/blob/master/framing_format.txt says
+	// that "the uncompressed data in a chunk must be no longer than 65536
+	// bytes".
+	snappyMaxBlockSize = 65536
+
+	// snappyMaxEncodedLenOfMaxBlockSize equals MaxEncodedLen(snappyMaxBlockSize), but is
+	// hard coded to be a const instead of a variable, so that obufLen can also
+	// be a const. Their equivalence is confirmed by
+	// TestMaxEncodedLenOfMaxBlockSize.
+	snappyMaxEncodedLenOfMaxBlockSize = 76490
+)
+
+const (
+	chunkTypeCompressedData   = 0x00
+	chunkTypeUncompressedData = 0x01
+	chunkTypePadding          = 0xfe
+	chunkTypeStreamIdentifier = 0xff
+)
+
+var (
+	// ErrSnappyCorrupt reports that the input is invalid.
+	ErrSnappyCorrupt = errors.New("snappy: corrupt input")
+	// ErrSnappyTooLarge reports that the uncompressed length is too large.
+	ErrSnappyTooLarge = errors.New("snappy: decoded block is too large")
+	// ErrSnappyUnsupported reports that the input isn't supported.
+	ErrSnappyUnsupported = errors.New("snappy: unsupported input")
+
+	errUnsupportedLiteralLength = errors.New("snappy: unsupported literal length")
+)
+
+// SnappyConverter can read SnappyConverter-compressed streams and convert them to zstd.
+// Conversion is done by converting the stream directly from Snappy without intermediate
+// full decoding.
+// Therefore the compression ratio is much less than what can be done by a full decompression
+// and compression, and a faulty Snappy stream may lead to a faulty Zstandard stream without
+// any errors being generated.
+// No CRC value is being generated and not all CRC values of the Snappy stream are checked.
+// However, it provides really fast recompression of Snappy streams.
+// The converter can be reused to avoid allocations, even after errors.
+type SnappyConverter struct {
+	r     io.Reader
+	err   error
+	buf   []byte
+	block *blockEnc
+}
+
+// Convert the Snappy stream supplied in 'in' and write the zStandard stream to 'w'.
+// If any error is detected on the Snappy stream it is returned.
+// The number of bytes written is returned.
+func (r *SnappyConverter) Convert(in io.Reader, w io.Writer) (int64, error) {
+	initPredefined()
+	r.err = nil
+	r.r = in
+	if r.block == nil {
+		r.block = &blockEnc{}
+		r.block.init()
+	}
+	r.block.initNewEncode()
+	if len(r.buf) != snappyMaxEncodedLenOfMaxBlockSize+snappyChecksumSize {
+		r.buf = make([]byte, snappyMaxEncodedLenOfMaxBlockSize+snappyChecksumSize)
+	}
+	r.block.litEnc.Reuse = huff0.ReusePolicyNone
+	var written int64
+	var readHeader bool
+	{
+		var header []byte
+		var n int
+		header, r.err = frameHeader{WindowSize: snappyMaxBlockSize}.appendTo(r.buf[:0])
+
+		n, r.err = w.Write(header)
+		if r.err != nil {
+			return written, r.err
+		}
+		written += int64(n)
+	}
+
+	for {
+		if !r.readFull(r.buf[:4], true) {
+			// Add empty last block
+			r.block.reset(nil)
+			r.block.last = true
+			err := r.block.encodeLits(r.block.literals, false)
+			if err != nil {
+				return written, err
+			}
+			n, err := w.Write(r.block.output)
+			if err != nil {
+				return written, err
+			}
+			written += int64(n)
+
+			return written, r.err
+		}
+		chunkType := r.buf[0]
+		if !readHeader {
+			if chunkType != chunkTypeStreamIdentifier {
+				println("chunkType != chunkTypeStreamIdentifier", chunkType)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			readHeader = true
+		}
+		chunkLen := int(r.buf[1]) | int(r.buf[2])<<8 | int(r.buf[3])<<16
+		if chunkLen > len(r.buf) {
+			println("chunkLen > len(r.buf)", chunkType)
+			r.err = ErrSnappyUnsupported
+			return written, r.err
+		}
+
+		// The chunk types are specified at
+		// https://github.com/google/snappy/blob/master/framing_format.txt
+		switch chunkType {
+		case chunkTypeCompressedData:
+			// Section 4.2. Compressed data (chunk type 0x00).
+			if chunkLen < snappyChecksumSize {
+				println("chunkLen < snappyChecksumSize", chunkLen, snappyChecksumSize)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			buf := r.buf[:chunkLen]
+			if !r.readFull(buf, false) {
+				return written, r.err
+			}
+			//checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24
+			buf = buf[snappyChecksumSize:]
+
+			n, hdr, err := snappyDecodedLen(buf)
+			if err != nil {
+				r.err = err
+				return written, r.err
+			}
+			buf = buf[hdr:]
+			if n > snappyMaxBlockSize {
+				println("n > snappyMaxBlockSize", n, snappyMaxBlockSize)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			r.block.reset(nil)
+			r.block.pushOffsets()
+			if err := decodeSnappy(r.block, buf); err != nil {
+				r.err = err
+				return written, r.err
+			}
+			if r.block.size+r.block.extraLits != n {
+				printf("invalid size, want %d, got %d\n", n, r.block.size+r.block.extraLits)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			err = r.block.encode(nil, false, false)
+			switch err {
+			case errIncompressible:
+				r.block.popOffsets()
+				r.block.reset(nil)
+				r.block.literals, err = snappy.Decode(r.block.literals[:n], r.buf[snappyChecksumSize:chunkLen])
+				if err != nil {
+					return written, err
+				}
+				err = r.block.encodeLits(r.block.literals, false)
+				if err != nil {
+					return written, err
+				}
+			case nil:
+			default:
+				return written, err
+			}
+
+			n, r.err = w.Write(r.block.output)
+			if r.err != nil {
+				return written, err
+			}
+			written += int64(n)
+			continue
+		case chunkTypeUncompressedData:
+			if debug {
+				println("Uncompressed, chunklen", chunkLen)
+			}
+			// Section 4.3. Uncompressed data (chunk type 0x01).
+			if chunkLen < snappyChecksumSize {
+				println("chunkLen < snappyChecksumSize", chunkLen, snappyChecksumSize)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			r.block.reset(nil)
+			buf := r.buf[:snappyChecksumSize]
+			if !r.readFull(buf, false) {
+				return written, r.err
+			}
+			checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24
+			// Read directly into r.decoded instead of via r.buf.
+			n := chunkLen - snappyChecksumSize
+			if n > snappyMaxBlockSize {
+				println("n > snappyMaxBlockSize", n, snappyMaxBlockSize)
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			r.block.literals = r.block.literals[:n]
+			if !r.readFull(r.block.literals, false) {
+				return written, r.err
+			}
+			if snappyCRC(r.block.literals) != checksum {
+				println("literals crc mismatch")
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			err := r.block.encodeLits(r.block.literals, false)
+			if err != nil {
+				return written, err
+			}
+			n, r.err = w.Write(r.block.output)
+			if r.err != nil {
+				return written, err
+			}
+			written += int64(n)
+			continue
+
+		case chunkTypeStreamIdentifier:
+			if debug {
+				println("stream id", chunkLen, len(snappyMagicBody))
+			}
+			// Section 4.1. Stream identifier (chunk type 0xff).
+			if chunkLen != len(snappyMagicBody) {
+				println("chunkLen != len(snappyMagicBody)", chunkLen, len(snappyMagicBody))
+				r.err = ErrSnappyCorrupt
+				return written, r.err
+			}
+			if !r.readFull(r.buf[:len(snappyMagicBody)], false) {
+				return written, r.err
+			}
+			for i := 0; i < len(snappyMagicBody); i++ {
+				if r.buf[i] != snappyMagicBody[i] {
+					println("r.buf[i] != snappyMagicBody[i]", r.buf[i], snappyMagicBody[i], i)
+					r.err = ErrSnappyCorrupt
+					return written, r.err
+				}
+			}
+			continue
+		}
+
+		if chunkType <= 0x7f {
+			// Section 4.5. Reserved unskippable chunks (chunk types 0x02-0x7f).
+			println("chunkType <= 0x7f")
+			r.err = ErrSnappyUnsupported
+			return written, r.err
+		}
+		// Section 4.4 Padding (chunk type 0xfe).
+		// Section 4.6. Reserved skippable chunks (chunk types 0x80-0xfd).
+		if !r.readFull(r.buf[:chunkLen], false) {
+			return written, r.err
+		}
+	}
+}
+
+// decodeSnappy writes the decoding of src to dst. It assumes that the varint-encoded
+// length of the decompressed bytes has already been read.
+func decodeSnappy(blk *blockEnc, src []byte) error {
+	//decodeRef(make([]byte, snappyMaxBlockSize), src)
+	var s, length int
+	lits := blk.extraLits
+	var offset uint32
+	for s < len(src) {
+		switch src[s] & 0x03 {
+		case snappyTagLiteral:
+			x := uint32(src[s] >> 2)
+			switch {
+			case x < 60:
+				s++
+			case x == 60:
+				s += 2
+				if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+					println("uint(s) > uint(len(src)", s, src)
+					return ErrSnappyCorrupt
+				}
+				x = uint32(src[s-1])
+			case x == 61:
+				s += 3
+				if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+					println("uint(s) > uint(len(src)", s, src)
+					return ErrSnappyCorrupt
+				}
+				x = uint32(src[s-2]) | uint32(src[s-1])<<8
+			case x == 62:
+				s += 4
+				if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+					println("uint(s) > uint(len(src)", s, src)
+					return ErrSnappyCorrupt
+				}
+				x = uint32(src[s-3]) | uint32(src[s-2])<<8 | uint32(src[s-1])<<16
+			case x == 63:
+				s += 5
+				if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+					println("uint(s) > uint(len(src)", s, src)
+					return ErrSnappyCorrupt
+				}
+				x = uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24
+			}
+			if x > snappyMaxBlockSize {
+				println("x > snappyMaxBlockSize", x, snappyMaxBlockSize)
+				return ErrSnappyCorrupt
+			}
+			length = int(x) + 1
+			if length <= 0 {
+				println("length <= 0 ", length)
+
+				return errUnsupportedLiteralLength
+			}
+			//if length > snappyMaxBlockSize-d || uint32(length) > len(src)-s {
+			//	return ErrSnappyCorrupt
+			//}
+
+			blk.literals = append(blk.literals, src[s:s+length]...)
+			//println(length, "litLen")
+			lits += length
+			s += length
+			continue
+
+		case snappyTagCopy1:
+			s += 2
+			if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+				println("uint(s) > uint(len(src)", s, len(src))
+				return ErrSnappyCorrupt
+			}
+			length = 4 + int(src[s-2])>>2&0x7
+			offset = uint32(src[s-2])&0xe0<<3 | uint32(src[s-1])
+
+		case snappyTagCopy2:
+			s += 3
+			if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+				println("uint(s) > uint(len(src)", s, len(src))
+				return ErrSnappyCorrupt
+			}
+			length = 1 + int(src[s-3])>>2
+			offset = uint32(src[s-2]) | uint32(src[s-1])<<8
+
+		case snappyTagCopy4:
+			s += 5
+			if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line.
+				println("uint(s) > uint(len(src)", s, len(src))
+				return ErrSnappyCorrupt
+			}
+			length = 1 + int(src[s-5])>>2
+			offset = uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24
+		}
+
+		if offset <= 0 || blk.size+lits < int(offset) /*|| length > len(blk)-d */ {
+			println("offset <= 0 || blk.size+lits < int(offset)", offset, blk.size+lits, int(offset), blk.size, lits)
+
+			return ErrSnappyCorrupt
+		}
+
+		// Check if offset is one of the recent offsets.
+		// Adjusts the output offset accordingly.
+		// Gives a tiny bit of compression, typically around 1%.
+		if false {
+			offset = blk.matchOffset(offset, uint32(lits))
+		} else {
+			offset += 3
+		}
+
+		blk.sequences = append(blk.sequences, seq{
+			litLen:   uint32(lits),
+			offset:   offset,
+			matchLen: uint32(length) - zstdMinMatch,
+		})
+		blk.size += length + lits
+		lits = 0
+	}
+	blk.extraLits = lits
+	return nil
+}
+
+func (r *SnappyConverter) readFull(p []byte, allowEOF bool) (ok bool) {
+	if _, r.err = io.ReadFull(r.r, p); r.err != nil {
+		if r.err == io.ErrUnexpectedEOF || (r.err == io.EOF && !allowEOF) {
+			r.err = ErrSnappyCorrupt
+		}
+		return false
+	}
+	return true
+}
+
+var crcTable = crc32.MakeTable(crc32.Castagnoli)
+
+// crc implements the checksum specified in section 3 of
+// https://github.com/google/snappy/blob/master/framing_format.txt
+func snappyCRC(b []byte) uint32 {
+	c := crc32.Update(0, crcTable, b)
+	return c>>15 | c<<17 + 0xa282ead8
+}
+
+// snappyDecodedLen returns the length of the decoded block and the number of bytes
+// that the length header occupied.
+func snappyDecodedLen(src []byte) (blockLen, headerLen int, err error) {
+	v, n := binary.Uvarint(src)
+	if n <= 0 || v > 0xffffffff {
+		return 0, 0, ErrSnappyCorrupt
+	}
+
+	const wordSize = 32 << (^uint(0) >> 32 & 1)
+	if wordSize == 32 && v > 0x7fffffff {
+		return 0, 0, ErrSnappyTooLarge
+	}
+	return int(v), n, nil
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/zip.go b/vendor/github.com/klauspost/compress/zstd/zip.go
new file mode 100644
index 0000000..e35a0a2
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/zip.go
@@ -0,0 +1,120 @@
+// Copyright 2019+ Klaus Post. All rights reserved.
+// License information can be found in the LICENSE file.
+
+package zstd
+
+import (
+	"errors"
+	"io"
+	"sync"
+)
+
+// ZipMethodWinZip is the method for Zstandard compressed data inside Zip files for WinZip.
+// See https://www.winzip.com/win/en/comp_info.html
+const ZipMethodWinZip = 93
+
+// ZipMethodPKWare is the method number used by PKWARE to indicate Zstandard compression.
+// See https://pkware.cachefly.net/webdocs/APPNOTE/APPNOTE-6.3.7.TXT
+const ZipMethodPKWare = 20
+
+var zipReaderPool sync.Pool
+
+// newZipReader cannot be used since we would leak goroutines...
+func newZipReader(r io.Reader) io.ReadCloser {
+	dec, ok := zipReaderPool.Get().(*Decoder)
+	if ok {
+		dec.Reset(r)
+	} else {
+		d, err := NewReader(r, WithDecoderConcurrency(1), WithDecoderLowmem(true))
+		if err != nil {
+			panic(err)
+		}
+		dec = d
+	}
+	return &pooledZipReader{dec: dec}
+}
+
+type pooledZipReader struct {
+	mu  sync.Mutex // guards Close and Read
+	dec *Decoder
+}
+
+func (r *pooledZipReader) Read(p []byte) (n int, err error) {
+	r.mu.Lock()
+	defer r.mu.Unlock()
+	if r.dec == nil {
+		return 0, errors.New("Read after Close")
+	}
+	dec, err := r.dec.Read(p)
+
+	return dec, err
+}
+
+func (r *pooledZipReader) Close() error {
+	r.mu.Lock()
+	defer r.mu.Unlock()
+	var err error
+	if r.dec != nil {
+		err = r.dec.Reset(nil)
+		zipReaderPool.Put(r.dec)
+		r.dec = nil
+	}
+	return err
+}
+
+type pooledZipWriter struct {
+	mu  sync.Mutex // guards Close and Read
+	enc *Encoder
+}
+
+func (w *pooledZipWriter) Write(p []byte) (n int, err error) {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+	if w.enc == nil {
+		return 0, errors.New("Write after Close")
+	}
+	return w.enc.Write(p)
+}
+
+func (w *pooledZipWriter) Close() error {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+	var err error
+	if w.enc != nil {
+		err = w.enc.Close()
+		zipReaderPool.Put(w.enc)
+		w.enc = nil
+	}
+	return err
+}
+
+// ZipCompressor returns a compressor that can be registered with zip libraries.
+// The provided encoder options will be used on all encodes.
+func ZipCompressor(opts ...EOption) func(w io.Writer) (io.WriteCloser, error) {
+	var pool sync.Pool
+	return func(w io.Writer) (io.WriteCloser, error) {
+		enc, ok := pool.Get().(*Encoder)
+		if ok {
+			enc.Reset(w)
+		} else {
+			var err error
+			enc, err = NewWriter(w, opts...)
+			if err != nil {
+				return nil, err
+			}
+		}
+		return &pooledZipWriter{enc: enc}, nil
+	}
+}
+
+// ZipDecompressor returns a decompressor that can be registered with zip libraries.
+// See ZipCompressor for example.
+func ZipDecompressor() func(r io.Reader) io.ReadCloser {
+	return func(r io.Reader) io.ReadCloser {
+		d, err := NewReader(r, WithDecoderConcurrency(1), WithDecoderLowmem(true))
+		if err != nil {
+			panic(err)
+		}
+		return d.IOReadCloser()
+	}
+}
diff --git a/vendor/github.com/klauspost/compress/zstd/zstd.go b/vendor/github.com/klauspost/compress/zstd/zstd.go
new file mode 100644
index 0000000..1ba308c
--- /dev/null
+++ b/vendor/github.com/klauspost/compress/zstd/zstd.go
@@ -0,0 +1,146 @@
+// Package zstd provides decompression of zstandard files.
+//
+// For advanced usage and examples, go to the README: https://github.com/klauspost/compress/tree/master/zstd#zstd
+package zstd
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"log"
+	"math"
+	"math/bits"
+)
+
+// enable debug printing
+const debug = false
+
+// Enable extra assertions.
+const debugAsserts = debug || false
+
+// print sequence details
+const debugSequences = false
+
+// print detailed matching information
+const debugMatches = false
+
+// force encoder to use predefined tables.
+const forcePreDef = false
+
+// zstdMinMatch is the minimum zstd match length.
+const zstdMinMatch = 3
+
+// Reset the buffer offset when reaching this.
+const bufferReset = math.MaxInt32 - MaxWindowSize
+
+var (
+	// ErrReservedBlockType is returned when a reserved block type is found.
+	// Typically this indicates wrong or corrupted input.
+	ErrReservedBlockType = errors.New("invalid input: reserved block type encountered")
+
+	// ErrCompressedSizeTooBig is returned when a block is bigger than allowed.
+	// Typically this indicates wrong or corrupted input.
+	ErrCompressedSizeTooBig = errors.New("invalid input: compressed size too big")
+
+	// ErrBlockTooSmall is returned when a block is too small to be decoded.
+	// Typically returned on invalid input.
+	ErrBlockTooSmall = errors.New("block too small")
+
+	// ErrMagicMismatch is returned when a "magic" number isn't what is expected.
+	// Typically this indicates wrong or corrupted input.
+	ErrMagicMismatch = errors.New("invalid input: magic number mismatch")
+
+	// ErrWindowSizeExceeded is returned when a reference exceeds the valid window size.
+	// Typically this indicates wrong or corrupted input.
+	ErrWindowSizeExceeded = errors.New("window size exceeded")
+
+	// ErrWindowSizeTooSmall is returned when no window size is specified.
+	// Typically this indicates wrong or corrupted input.
+	ErrWindowSizeTooSmall = errors.New("invalid input: window size was too small")
+
+	// ErrDecoderSizeExceeded is returned if decompressed size exceeds the configured limit.
+	ErrDecoderSizeExceeded = errors.New("decompressed size exceeds configured limit")
+
+	// ErrUnknownDictionary is returned if the dictionary ID is unknown.
+	// For the time being dictionaries are not supported.
+	ErrUnknownDictionary = errors.New("unknown dictionary")
+
+	// ErrFrameSizeExceeded is returned if the stated frame size is exceeded.
+	// This is only returned if SingleSegment is specified on the frame.
+	ErrFrameSizeExceeded = errors.New("frame size exceeded")
+
+	// ErrCRCMismatch is returned if CRC mismatches.
+	ErrCRCMismatch = errors.New("CRC check failed")
+
+	// ErrDecoderClosed will be returned if the Decoder was used after
+	// Close has been called.
+	ErrDecoderClosed = errors.New("decoder used after Close")
+
+	// ErrDecoderNilInput is returned when a nil Reader was provided
+	// and an operation other than Reset/DecodeAll/Close was attempted.
+	ErrDecoderNilInput = errors.New("nil input provided as reader")
+)
+
+func println(a ...interface{}) {
+	if debug {
+		log.Println(a...)
+	}
+}
+
+func printf(format string, a ...interface{}) {
+	if debug {
+		log.Printf(format, a...)
+	}
+}
+
+// matchLenFast does matching, but will not match the last up to 7 bytes.
+func matchLenFast(a, b []byte) int {
+	endI := len(a) & (math.MaxInt32 - 7)
+	for i := 0; i < endI; i += 8 {
+		if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+			return i + bits.TrailingZeros64(diff)>>3
+		}
+	}
+	return endI
+}
+
+// matchLen returns the maximum length.
+// a must be the shortest of the two.
+// The function also returns whether all bytes matched.
+func matchLen(a, b []byte) int {
+	b = b[:len(a)]
+	for i := 0; i < len(a)-7; i += 8 {
+		if diff := load64(a, i) ^ load64(b, i); diff != 0 {
+			return i + (bits.TrailingZeros64(diff) >> 3)
+		}
+	}
+
+	checked := (len(a) >> 3) << 3
+	a = a[checked:]
+	b = b[checked:]
+	for i := range a {
+		if a[i] != b[i] {
+			return i + checked
+		}
+	}
+	return len(a) + checked
+}
+
+func load3232(b []byte, i int32) uint32 {
+	return binary.LittleEndian.Uint32(b[i:])
+}
+
+func load6432(b []byte, i int32) uint64 {
+	return binary.LittleEndian.Uint64(b[i:])
+}
+
+func load64(b []byte, i int) uint64 {
+	return binary.LittleEndian.Uint64(b[i:])
+}
+
+type byter interface {
+	Bytes() []byte
+	Len() int
+}
+
+var _ byter = &bytes.Buffer{}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/adapter_proxy_if.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/adapter_proxy_if.go
deleted file mode 100644
index c514d6d..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/adapter_proxy_if.go
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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 adapterif
-
-import (
-	"context"
-
-	"github.com/golang/protobuf/proto"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-)
-
-// AdapterProxy interface for AdapterProxy implementation.
-type AdapterProxy interface {
-	SendInterAdapterMessage(ctx context.Context,
-		msg proto.Message,
-		msgType ic.InterAdapterMessageType_Types,
-		fromAdapter string,
-		toAdapter string,
-		toDeviceID string,
-		proxyDeviceID string,
-		messageID string) error
-	TechProfileInstanceRequest(ctx context.Context,
-		tpPath string,
-		ponIntfID uint32,
-		onuID uint32,
-		uniID uint32,
-		fromAdapter string,
-		toAdapter string,
-		toDeviceID string,
-		proxyDeviceID string) (*ic.InterAdapterTechProfileDownloadMessage, error)
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/core_proxy_if.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/core_proxy_if.go
deleted file mode 100644
index 36939bd..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif/core_proxy_if.go
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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 adapterif
-
-import (
-	"context"
-
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-// CoreProxy interface for voltha-go coreproxy.
-type CoreProxy interface {
-	UpdateCoreReference(deviceID string, coreReference string)
-	DeleteCoreReference(deviceID string)
-	RegisterAdapter(ctx context.Context, adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) error
-	DeviceUpdate(ctx context.Context, device *voltha.Device) error
-	PortCreated(ctx context.Context, deviceID string, port *voltha.Port) error
-	PortsStateUpdate(ctx context.Context, deviceID string, portTypeFilter uint32, operStatus voltha.OperStatus_Types) error
-	DeleteAllPorts(ctx context.Context, deviceID string) error
-	GetDevicePort(ctx context.Context, deviceID string, portNo uint32) (*voltha.Port, error)
-	ListDevicePorts(ctx context.Context, deviceID string) ([]*voltha.Port, error)
-	DeviceStateUpdate(ctx context.Context, deviceID string,
-		connStatus voltha.ConnectStatus_Types, operStatus voltha.OperStatus_Types) error
-
-	DevicePMConfigUpdate(ctx context.Context, pmConfigs *voltha.PmConfigs) error
-	ChildDeviceDetected(ctx context.Context, parentDeviceID string, parentPortNo int,
-		childDeviceType string, channelID int, vendorID string, serialNumber string, onuID int64) (*voltha.Device, error)
-
-	ChildDevicesLost(ctx context.Context, parentDeviceID string) error
-	ChildDevicesDetected(ctx context.Context, parentDeviceID string) error
-	GetDevice(ctx context.Context, parentDeviceID string, deviceID string) (*voltha.Device, error)
-	GetChildDevice(ctx context.Context, parentDeviceID string, kwargs map[string]interface{}) (*voltha.Device, error)
-	GetChildDevices(ctx context.Context, parentDeviceID string) (*voltha.Devices, error)
-	SendPacketIn(ctx context.Context, deviceID string, port uint32, pktPayload []byte) error
-	DeviceReasonUpdate(ctx context.Context, deviceID string, deviceReason string) error
-	PortStateUpdate(ctx context.Context, deviceID string, pType voltha.Port_PortType, portNo uint32,
-		operStatus voltha.OperStatus_Types) error
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/adapter_proxy.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/adapter_proxy.go
deleted file mode 100644
index fc31041..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/adapter_proxy.go
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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 common
-
-import (
-	"context"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"google.golang.org/grpc/status"
-
-	"github.com/golang/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/golang/protobuf/ptypes/any"
-	"github.com/google/uuid"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-)
-
-type AdapterProxy struct {
-	kafkaICProxy kafka.InterContainerProxy
-	coreTopic    string
-	endpointMgr  kafka.EndpointManager
-}
-
-func NewAdapterProxy(ctx context.Context, kafkaProxy kafka.InterContainerProxy, coreTopic string, backend *db.Backend) *AdapterProxy {
-	proxy := AdapterProxy{
-		kafkaICProxy: kafkaProxy,
-		coreTopic:    coreTopic,
-		endpointMgr:  kafka.NewEndpointManager(backend),
-	}
-	logger.Debugw(ctx, "topics", log.Fields{"core": proxy.coreTopic})
-	return &proxy
-}
-
-func (ap *AdapterProxy) SendInterAdapterMessage(ctx context.Context,
-	msg proto.Message,
-	msgType ic.InterAdapterMessageType_Types,
-	fromAdapter string,
-	toAdapter string,
-	toDeviceId string,
-	proxyDeviceId string,
-	messageId string) error {
-	logger.Debugw(ctx, "sending-inter-adapter-message", log.Fields{"type": msgType, "from": fromAdapter,
-		"to": toAdapter, "toDevice": toDeviceId, "proxyDevice": proxyDeviceId})
-
-	//Marshal the message
-	var marshalledMsg *any.Any
-	var err error
-	if marshalledMsg, err = ptypes.MarshalAny(msg); err != nil {
-		logger.Warnw(ctx, "cannot-marshal-msg", log.Fields{"error": err})
-		return err
-	}
-
-	// Set up the required rpc arguments
-	endpoint, err := ap.endpointMgr.GetEndpoint(ctx, toDeviceId, toAdapter)
-	if err != nil {
-		return err
-	}
-
-	//Build the inter adapter message
-	header := &ic.InterAdapterHeader{
-		Type:          msgType,
-		FromTopic:     fromAdapter,
-		ToTopic:       string(endpoint),
-		ToDeviceId:    toDeviceId,
-		ProxyDeviceId: proxyDeviceId,
-	}
-	if messageId != "" {
-		header.Id = messageId
-	} else {
-		header.Id = uuid.New().String()
-	}
-	header.Timestamp = ptypes.TimestampNow()
-	iaMsg := &ic.InterAdapterMessage{
-		Header: header,
-		Body:   marshalledMsg,
-	}
-	args := make([]*kafka.KVArg, 1)
-	args[0] = &kafka.KVArg{
-		Key:   "msg",
-		Value: iaMsg,
-	}
-
-	topic := kafka.Topic{Name: string(endpoint)}
-	replyToTopic := kafka.Topic{Name: fromAdapter}
-	rpc := "process_inter_adapter_message"
-
-	// Add a indication in context to differentiate this Inter Adapter message during Span processing in Kafka IC proxy
-	ctx = context.WithValue(ctx, "inter-adapter-msg-type", msgType)
-	success, result := ap.kafkaICProxy.InvokeRPC(ctx, rpc, &topic, &replyToTopic, true, proxyDeviceId, args...)
-	logger.Debugw(ctx, "inter-adapter-msg-response", log.Fields{"replyTopic": replyToTopic, "success": success})
-	return unPackResponse(ctx, rpc, "", success, result)
-}
-
-func (ap *AdapterProxy) TechProfileInstanceRequest(ctx context.Context,
-	tpPath string,
-	parentPonPort uint32,
-	onuID uint32,
-	uniID uint32,
-	fromAdapter string,
-	toAdapter string,
-	toDeviceId string,
-	proxyDeviceId string) (*ic.InterAdapterTechProfileDownloadMessage, error) {
-	logger.Debugw(ctx, "sending-tech-profile-instance-request-message", log.Fields{"from": fromAdapter,
-		"to": toAdapter, "toDevice": toDeviceId, "proxyDevice": proxyDeviceId})
-
-	// Set up the required rpc arguments
-	endpoint, err := ap.endpointMgr.GetEndpoint(ctx, toDeviceId, toAdapter)
-	if err != nil {
-		return nil, err
-	}
-
-	//Build the inter adapter message
-	tpReqMsg := &ic.InterAdapterTechProfileInstanceRequestMessage{
-		TpInstancePath: tpPath,
-		ParentDeviceId: toDeviceId,
-		ParentPonPort:  parentPonPort,
-		OnuId:          onuID,
-		UniId:          uniID,
-	}
-
-	args := make([]*kafka.KVArg, 1)
-	args[0] = &kafka.KVArg{
-		Key:   "msg",
-		Value: tpReqMsg,
-	}
-
-	topic := kafka.Topic{Name: string(endpoint)}
-	replyToTopic := kafka.Topic{Name: fromAdapter}
-	rpc := "process_tech_profile_instance_request"
-
-	ctx = context.WithValue(ctx, "inter-adapter-tp-req-msg", tpPath)
-	success, result := ap.kafkaICProxy.InvokeRPC(ctx, rpc, &topic, &replyToTopic, true, proxyDeviceId, args...)
-	logger.Debugw(ctx, "inter-adapter-msg-response", log.Fields{"replyTopic": replyToTopic, "success": success})
-	if success {
-		tpDwnldMsg := &ic.InterAdapterTechProfileDownloadMessage{}
-		if err := ptypes.UnmarshalAny(result, tpDwnldMsg); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, err
-		}
-		return tpDwnldMsg, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "TechProfileInstanceRequest-return", log.Fields{"tpPath": tpPath, "success": success, "error": err})
-
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/common.go
deleted file mode 100644
index 98085bb..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 common
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/core_proxy.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/core_proxy.go
deleted file mode 100644
index 589d951..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/core_proxy.go
+++ /dev/null
@@ -1,689 +0,0 @@
-/*
- * 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 common
-
-import (
-	"context"
-	"sync"
-
-	"github.com/golang/protobuf/ptypes"
-	a "github.com/golang/protobuf/ptypes/any"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-)
-
-type CoreProxy struct {
-	kafkaICProxy        kafka.InterContainerProxy
-	adapterTopic        string
-	coreTopic           string
-	deviceIdCoreMap     map[string]string
-	lockDeviceIdCoreMap sync.RWMutex
-}
-
-func NewCoreProxy(ctx context.Context, kafkaProxy kafka.InterContainerProxy, adapterTopic string, coreTopic string) *CoreProxy {
-	var proxy CoreProxy
-	proxy.kafkaICProxy = kafkaProxy
-	proxy.adapterTopic = adapterTopic
-	proxy.coreTopic = coreTopic
-	proxy.deviceIdCoreMap = make(map[string]string)
-	proxy.lockDeviceIdCoreMap = sync.RWMutex{}
-	logger.Debugw(ctx, "TOPICS", log.Fields{"core": proxy.coreTopic, "adapter": proxy.adapterTopic})
-
-	return &proxy
-}
-
-func unPackResponse(ctx context.Context, rpc string, deviceId string, success bool, response *a.Any) error {
-	if success {
-		return nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(response, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "response", log.Fields{"rpc": rpc, "device-id": deviceId, "success": success, "error": err})
-		// TODO:  Need to get the real error code
-		return status.Errorf(codes.Canceled, "%s", unpackResult.Reason)
-	}
-}
-
-// UpdateCoreReference adds or update a core reference (really the topic name) for a given device Id
-func (ap *CoreProxy) UpdateCoreReference(deviceId string, coreReference string) {
-	ap.lockDeviceIdCoreMap.Lock()
-	defer ap.lockDeviceIdCoreMap.Unlock()
-	ap.deviceIdCoreMap[deviceId] = coreReference
-}
-
-// DeleteCoreReference removes a core reference (really the topic name) for a given device Id
-func (ap *CoreProxy) DeleteCoreReference(deviceId string) {
-	ap.lockDeviceIdCoreMap.Lock()
-	defer ap.lockDeviceIdCoreMap.Unlock()
-	delete(ap.deviceIdCoreMap, deviceId)
-}
-
-func (ap *CoreProxy) getCoreTopic(deviceId string) kafka.Topic {
-	ap.lockDeviceIdCoreMap.Lock()
-	defer ap.lockDeviceIdCoreMap.Unlock()
-
-	if t, exist := ap.deviceIdCoreMap[deviceId]; exist {
-		return kafka.Topic{Name: t}
-	}
-
-	return kafka.Topic{Name: ap.coreTopic}
-}
-
-func (ap *CoreProxy) getAdapterTopic(args ...string) kafka.Topic {
-	return kafka.Topic{Name: ap.adapterTopic}
-}
-
-func (ap *CoreProxy) RegisterAdapter(ctx context.Context, adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) error {
-	logger.Debugw(ctx, "registering-adapter", log.Fields{"coreTopic": ap.coreTopic, "adapterTopic": ap.adapterTopic})
-	rpc := "Register"
-	topic := kafka.Topic{Name: ap.coreTopic}
-	replyToTopic := ap.getAdapterTopic()
-	args := make([]*kafka.KVArg, 2)
-
-	if adapter.TotalReplicas == 0 && adapter.CurrentReplica != 0 {
-		logger.Fatal(ctx, "totalReplicas can't be 0, since you're here you have at least one")
-	}
-
-	if adapter.CurrentReplica == 0 && adapter.TotalReplicas != 0 {
-		logger.Fatal(ctx, "currentReplica can't be 0, it has to start from 1")
-	}
-
-	if adapter.CurrentReplica == 0 && adapter.TotalReplicas == 0 {
-		// if the adapter is not setting these fields they default to 0,
-		// in that case it means the adapter is not ready to be scaled and thus it defaults
-		// to a single instance
-		adapter.CurrentReplica = 1
-		adapter.TotalReplicas = 1
-	}
-
-	if adapter.CurrentReplica > adapter.TotalReplicas {
-		logger.Fatalf(ctx, "CurrentReplica (%d) can't be greater than TotalReplicas (%d)",
-			adapter.CurrentReplica, adapter.TotalReplicas)
-	}
-
-	args[0] = &kafka.KVArg{
-		Key:   "adapter",
-		Value: adapter,
-	}
-	args[1] = &kafka.KVArg{
-		Key:   "deviceTypes",
-		Value: deviceTypes,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(ctx, rpc, &topic, &replyToTopic, true, "", args...)
-	logger.Debugw(ctx, "Register-Adapter-response", log.Fields{"replyTopic": replyToTopic, "success": success})
-	return unPackResponse(ctx, rpc, "", success, result)
-}
-
-func (ap *CoreProxy) DeviceUpdate(ctx context.Context, device *voltha.Device) error {
-	logger.Debugw(ctx, "DeviceUpdate", log.Fields{"device-id": device.Id})
-	rpc := "DeviceUpdate"
-	toTopic := ap.getCoreTopic(device.Id)
-	args := make([]*kafka.KVArg, 1)
-	args[0] = &kafka.KVArg{
-		Key:   "device",
-		Value: device,
-	}
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, device.Id, args...)
-	logger.Debugw(ctx, "DeviceUpdate-response", log.Fields{"device-id": device.Id, "success": success})
-	return unPackResponse(ctx, rpc, device.Id, success, result)
-}
-
-func (ap *CoreProxy) PortCreated(ctx context.Context, deviceId string, port *voltha.Port) error {
-	logger.Debugw(ctx, "PortCreated", log.Fields{"portNo": port.PortNo})
-	rpc := "PortCreated"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	args := make([]*kafka.KVArg, 2)
-	id := &voltha.ID{Id: deviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-	args[1] = &kafka.KVArg{
-		Key:   "port",
-		Value: port,
-	}
-
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "PortCreated-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) PortsStateUpdate(ctx context.Context, deviceId string, portTypeFilter uint32, operStatus voltha.OperStatus_Types) error {
-	logger.Debugw(ctx, "PortsStateUpdate", log.Fields{"device-id": deviceId})
-	rpc := "PortsStateUpdate"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	args := []*kafka.KVArg{{
-		Key:   "device_id",
-		Value: &voltha.ID{Id: deviceId},
-	}, {
-		Key:   "port_type_filter",
-		Value: &ic.IntType{Val: int64(portTypeFilter)},
-	}, {
-		Key:   "oper_status",
-		Value: &ic.IntType{Val: int64(operStatus)},
-	}}
-
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "PortsStateUpdate-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) DeleteAllPorts(ctx context.Context, deviceId string) error {
-	logger.Debugw(ctx, "DeleteAllPorts", log.Fields{"device-id": deviceId})
-	rpc := "DeleteAllPorts"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	args := make([]*kafka.KVArg, 2)
-	id := &voltha.ID{Id: deviceId}
-
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "DeleteAllPorts-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) GetDevicePort(ctx context.Context, deviceID string, portNo uint32) (*voltha.Port, error) {
-	logger.Debugw(ctx, "GetDevicePort", log.Fields{"device-id": deviceID})
-	rpc := "GetDevicePort"
-
-	toTopic := ap.getCoreTopic(deviceID)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := []*kafka.KVArg{{
-		Key:   "device_id",
-		Value: &voltha.ID{Id: deviceID},
-	}, {
-		Key:   "port_no",
-		Value: &ic.IntType{Val: int64(portNo)},
-	}}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceID, args...)
-	logger.Debugw(ctx, "GetDevicePort-response", log.Fields{"device-id": deviceID, "success": success})
-
-	if success {
-		port := &voltha.Port{}
-		if err := ptypes.UnmarshalAny(result, port); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return port, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "GetDevicePort-return", log.Fields{"device-id": deviceID, "success": success, "error": err})
-		// TODO:  Need to get the real error code
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
-
-func (ap *CoreProxy) ListDevicePorts(ctx context.Context, deviceID string) ([]*voltha.Port, error) {
-	logger.Debugw(ctx, "ListDevicePorts", log.Fields{"device-id": deviceID})
-	rpc := "ListDevicePorts"
-
-	toTopic := ap.getCoreTopic(deviceID)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := []*kafka.KVArg{{
-		Key:   "device_id",
-		Value: &voltha.ID{Id: deviceID},
-	}}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceID, args...)
-	logger.Debugw(ctx, "ListDevicePorts-response", log.Fields{"device-id": deviceID, "success": success})
-
-	if success {
-		ports := &voltha.Ports{}
-		if err := ptypes.UnmarshalAny(result, ports); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return ports.Items, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "ListDevicePorts-return", log.Fields{"device-id": deviceID, "success": success, "error": err})
-		// TODO:  Need to get the real error code
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
-
-func (ap *CoreProxy) DeviceStateUpdate(ctx context.Context, deviceId string,
-	connStatus voltha.ConnectStatus_Types, operStatus voltha.OperStatus_Types) error {
-	logger.Debugw(ctx, "DeviceStateUpdate", log.Fields{"device-id": deviceId})
-	rpc := "DeviceStateUpdate"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	args := make([]*kafka.KVArg, 3)
-	id := &voltha.ID{Id: deviceId}
-	oStatus := &ic.IntType{Val: int64(operStatus)}
-	cStatus := &ic.IntType{Val: int64(connStatus)}
-
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-	args[1] = &kafka.KVArg{
-		Key:   "oper_status",
-		Value: oStatus,
-	}
-	args[2] = &kafka.KVArg{
-		Key:   "connect_status",
-		Value: cStatus,
-	}
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "DeviceStateUpdate-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) ChildDeviceDetected(ctx context.Context, parentDeviceId string, parentPortNo int,
-	childDeviceType string, channelId int, vendorId string, serialNumber string, onuId int64) (*voltha.Device, error) {
-	logger.Debugw(ctx, "ChildDeviceDetected", log.Fields{"parent-device-id": parentDeviceId, "channelId": channelId})
-	rpc := "ChildDeviceDetected"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 7)
-	id := &voltha.ID{Id: parentDeviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "parent_device_id",
-		Value: id,
-	}
-	ppn := &ic.IntType{Val: int64(parentPortNo)}
-	args[1] = &kafka.KVArg{
-		Key:   "parent_port_no",
-		Value: ppn,
-	}
-	cdt := &ic.StrType{Val: childDeviceType}
-	args[2] = &kafka.KVArg{
-		Key:   "child_device_type",
-		Value: cdt,
-	}
-	channel := &ic.IntType{Val: int64(channelId)}
-	args[3] = &kafka.KVArg{
-		Key:   "channel_id",
-		Value: channel,
-	}
-	vId := &ic.StrType{Val: vendorId}
-	args[4] = &kafka.KVArg{
-		Key:   "vendor_id",
-		Value: vId,
-	}
-	sNo := &ic.StrType{Val: serialNumber}
-	args[5] = &kafka.KVArg{
-		Key:   "serial_number",
-		Value: sNo,
-	}
-	oId := &ic.IntType{Val: int64(onuId)}
-	args[6] = &kafka.KVArg{
-		Key:   "onu_id",
-		Value: oId,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "ChildDeviceDetected-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-
-	if success {
-		volthaDevice := &voltha.Device{}
-		if err := ptypes.UnmarshalAny(result, volthaDevice); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return volthaDevice, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "ChildDeviceDetected-return", log.Fields{"device-id": parentDeviceId, "success": success, "error": err})
-
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-
-}
-
-func (ap *CoreProxy) ChildDevicesLost(ctx context.Context, parentDeviceId string) error {
-	logger.Debugw(ctx, "ChildDevicesLost", log.Fields{"parent-device-id": parentDeviceId})
-	rpc := "ChildDevicesLost"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 1)
-	id := &voltha.ID{Id: parentDeviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "parent_device_id",
-		Value: id,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "ChildDevicesLost-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-	return unPackResponse(ctx, rpc, parentDeviceId, success, result)
-}
-
-func (ap *CoreProxy) ChildDevicesDetected(ctx context.Context, parentDeviceId string) error {
-	logger.Debugw(ctx, "ChildDevicesDetected", log.Fields{"parent-device-id": parentDeviceId})
-	rpc := "ChildDevicesDetected"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 1)
-	id := &voltha.ID{Id: parentDeviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "parent_device_id",
-		Value: id,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "ChildDevicesDetected-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-	return unPackResponse(ctx, rpc, parentDeviceId, success, result)
-}
-
-func (ap *CoreProxy) GetDevice(ctx context.Context, parentDeviceId string, deviceId string) (*voltha.Device, error) {
-	logger.Debugw(ctx, "GetDevice", log.Fields{"device-id": deviceId})
-	rpc := "GetDevice"
-
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 1)
-	id := &voltha.ID{Id: deviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "GetDevice-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-
-	if success {
-		volthaDevice := &voltha.Device{}
-		if err := ptypes.UnmarshalAny(result, volthaDevice); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return volthaDevice, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "GetDevice-return", log.Fields{"parent-device-id": parentDeviceId, "success": success, "error": err})
-		// TODO:  Need to get the real error code
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
-
-func (ap *CoreProxy) GetChildDevice(ctx context.Context, parentDeviceId string, kwargs map[string]interface{}) (*voltha.Device, error) {
-	logger.Debugw(ctx, "GetChildDevice", log.Fields{"parent-device-id": parentDeviceId, "kwargs": kwargs})
-	rpc := "GetChildDevice"
-
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 4)
-	id := &voltha.ID{Id: parentDeviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-
-	var cnt uint8 = 0
-	for k, v := range kwargs {
-		cnt += 1
-		if k == "serial_number" {
-			val := &ic.StrType{Val: v.(string)}
-			args[cnt] = &kafka.KVArg{
-				Key:   k,
-				Value: val,
-			}
-		} else if k == "onu_id" {
-			val := &ic.IntType{Val: int64(v.(uint32))}
-			args[cnt] = &kafka.KVArg{
-				Key:   k,
-				Value: val,
-			}
-		} else if k == "parent_port_no" {
-			val := &ic.IntType{Val: int64(v.(uint32))}
-			args[cnt] = &kafka.KVArg{
-				Key:   k,
-				Value: val,
-			}
-		}
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "GetChildDevice-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-
-	if success {
-		volthaDevice := &voltha.Device{}
-		if err := ptypes.UnmarshalAny(result, volthaDevice); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return volthaDevice, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "GetChildDevice-return", log.Fields{"parent-device-id": parentDeviceId, "success": success, "error": err})
-
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
-
-func (ap *CoreProxy) GetChildDevices(ctx context.Context, parentDeviceId string) (*voltha.Devices, error) {
-	logger.Debugw(ctx, "GetChildDevices", log.Fields{"parent-device-id": parentDeviceId})
-	rpc := "GetChildDevices"
-
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 1)
-	id := &voltha.ID{Id: parentDeviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "GetChildDevices-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-
-	if success {
-		volthaDevices := &voltha.Devices{}
-		if err := ptypes.UnmarshalAny(result, volthaDevices); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-			return nil, status.Error(codes.InvalidArgument, err.Error())
-		}
-		return volthaDevices, nil
-	} else {
-		unpackResult := &ic.Error{}
-		var err error
-		if err = ptypes.UnmarshalAny(result, unpackResult); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		}
-		logger.Debugw(ctx, "GetChildDevices-return", log.Fields{"parent-device-id": parentDeviceId, "success": success, "error": err})
-
-		return nil, status.Error(ICProxyErrorCodeToGrpcErrorCode(ctx, unpackResult.Code), unpackResult.Reason)
-	}
-}
-
-func (ap *CoreProxy) SendPacketIn(ctx context.Context, deviceId string, port uint32, pktPayload []byte) error {
-	logger.Debugw(ctx, "SendPacketIn", log.Fields{"device-id": deviceId, "port": port, "pktPayload": pktPayload})
-	rpc := "PacketIn"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 3)
-	id := &voltha.ID{Id: deviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-	portNo := &ic.IntType{Val: int64(port)}
-	args[1] = &kafka.KVArg{
-		Key:   "port",
-		Value: portNo,
-	}
-	pkt := &ic.Packet{Payload: pktPayload}
-	args[2] = &kafka.KVArg{
-		Key:   "packet",
-		Value: pkt,
-	}
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "SendPacketIn-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) DeviceReasonUpdate(ctx context.Context, deviceId string, deviceReason string) error {
-	logger.Debugw(ctx, "DeviceReasonUpdate", log.Fields{"device-id": deviceId, "deviceReason": deviceReason})
-	rpc := "DeviceReasonUpdate"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 2)
-	id := &voltha.ID{Id: deviceId}
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: id,
-	}
-	reason := &ic.StrType{Val: deviceReason}
-	args[1] = &kafka.KVArg{
-		Key:   "device_reason",
-		Value: reason,
-	}
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "DeviceReason-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
-
-func (ap *CoreProxy) DevicePMConfigUpdate(ctx context.Context, pmConfigs *voltha.PmConfigs) error {
-	logger.Debugw(ctx, "DevicePMConfigUpdate", log.Fields{"pmConfigs": pmConfigs})
-	rpc := "DevicePMConfigUpdate"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(pmConfigs.Id)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := make([]*kafka.KVArg, 1)
-	args[0] = &kafka.KVArg{
-		Key:   "device_pm_config",
-		Value: pmConfigs,
-	}
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, pmConfigs.Id, args...)
-	logger.Debugw(ctx, "DevicePMConfigUpdate-response", log.Fields{"pmconfig-device-id": pmConfigs.Id, "success": success})
-	return unPackResponse(ctx, rpc, pmConfigs.Id, success, result)
-}
-
-func (ap *CoreProxy) ReconcileChildDevices(ctx context.Context, parentDeviceId string) error {
-	logger.Debugw(ctx, "ReconcileChildDevices", log.Fields{"parent-device-id": parentDeviceId})
-	rpc := "ReconcileChildDevices"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(parentDeviceId)
-	replyToTopic := ap.getAdapterTopic()
-
-	args := []*kafka.KVArg{
-		{Key: "parent_device_id", Value: &voltha.ID{Id: parentDeviceId}},
-	}
-
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, parentDeviceId, args...)
-	logger.Debugw(ctx, "ReconcileChildDevices-response", log.Fields{"parent-device-id": parentDeviceId, "success": success})
-	return unPackResponse(ctx, rpc, parentDeviceId, success, result)
-}
-
-func (ap *CoreProxy) PortStateUpdate(ctx context.Context, deviceId string, pType voltha.Port_PortType, portNum uint32,
-	operStatus voltha.OperStatus_Types) error {
-	logger.Debugw(ctx, "PortStateUpdate", log.Fields{"device-id": deviceId, "portType": pType, "portNo": portNum, "operation_status": operStatus})
-	rpc := "PortStateUpdate"
-	// Use a device specific topic to send the request.  The adapter handling the device creates a device
-	// specific topic
-	toTopic := ap.getCoreTopic(deviceId)
-	args := make([]*kafka.KVArg, 4)
-	deviceID := &voltha.ID{Id: deviceId}
-	portNo := &ic.IntType{Val: int64(portNum)}
-	portType := &ic.IntType{Val: int64(pType)}
-	oStatus := &ic.IntType{Val: int64(operStatus)}
-
-	args[0] = &kafka.KVArg{
-		Key:   "device_id",
-		Value: deviceID,
-	}
-	args[1] = &kafka.KVArg{
-		Key:   "oper_status",
-		Value: oStatus,
-	}
-	args[2] = &kafka.KVArg{
-		Key:   "port_type",
-		Value: portType,
-	}
-	args[3] = &kafka.KVArg{
-		Key:   "port_no",
-		Value: portNo,
-	}
-
-	// Use a device specific topic as we are the only adaptercore handling requests for this device
-	replyToTopic := ap.getAdapterTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(log.WithSpanFromContext(context.Background(), ctx), rpc, &toTopic, &replyToTopic, true, deviceId, args...)
-	logger.Debugw(ctx, "PortStateUpdate-response", log.Fields{"device-id": deviceId, "success": success})
-	return unPackResponse(ctx, rpc, deviceId, success, result)
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/performance_metrics.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/performance_metrics.go
deleted file mode 100644
index 3b6d4f9..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/performance_metrics.go
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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 common
-
-import (
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-type PmMetrics struct {
-	deviceId          string
-	frequency         uint32
-	grouped           bool
-	frequencyOverride bool
-	metrics           map[string]*voltha.PmConfig
-}
-
-type PmMetricsOption func(*PmMetrics)
-
-func Frequency(frequency uint32) PmMetricsOption {
-	return func(args *PmMetrics) {
-		args.frequency = frequency
-	}
-}
-
-func Grouped(grouped bool) PmMetricsOption {
-	return func(args *PmMetrics) {
-		args.grouped = grouped
-	}
-}
-
-func FrequencyOverride(frequencyOverride bool) PmMetricsOption {
-	return func(args *PmMetrics) {
-		args.frequencyOverride = frequencyOverride
-	}
-}
-
-// UpdateFrequency will update the frequency.
-func (pm *PmMetrics) UpdateFrequency(frequency uint32) {
-	pm.frequency = frequency
-}
-
-func Metrics(pmNames []string) PmMetricsOption {
-	return func(args *PmMetrics) {
-		args.metrics = make(map[string]*voltha.PmConfig)
-		for _, name := range pmNames {
-			args.metrics[name] = &voltha.PmConfig{
-				Name:    name,
-				Type:    voltha.PmConfig_COUNTER,
-				Enabled: true,
-			}
-		}
-	}
-}
-
-func NewPmMetrics(deviceId string, opts ...PmMetricsOption) *PmMetrics {
-	pm := &PmMetrics{deviceId: deviceId}
-	for _, option := range opts {
-		option(pm)
-	}
-	return pm
-}
-
-func (pm *PmMetrics) ToPmConfigs() *voltha.PmConfigs {
-	pmConfigs := &voltha.PmConfigs{
-		Id:           pm.deviceId,
-		DefaultFreq:  pm.frequency,
-		Grouped:      pm.grouped,
-		FreqOverride: pm.frequencyOverride,
-	}
-	for _, v := range pm.metrics {
-		pmConfigs.Metrics = append(pmConfigs.Metrics, &voltha.PmConfig{Name: v.Name, Type: v.Type, Enabled: v.Enabled})
-	}
-	return pmConfigs
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/request_handler.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/request_handler.go
deleted file mode 100644
index 90f575b..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/request_handler.go
+++ /dev/null
@@ -1,1011 +0,0 @@
-/*
- * 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 common
-
-import (
-	"context"
-	"errors"
-
-	"github.com/golang/protobuf/ptypes"
-	"github.com/golang/protobuf/ptypes/empty"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters"
-	"github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-)
-
-type RequestHandlerProxy struct {
-	TestMode       bool
-	coreInstanceId string
-	adapter        adapters.IAdapter
-	coreProxy      adapterif.CoreProxy
-}
-
-func NewRequestHandlerProxy(coreInstanceId string, iadapter adapters.IAdapter, cProxy adapterif.CoreProxy) *RequestHandlerProxy {
-	var proxy RequestHandlerProxy
-	proxy.coreInstanceId = coreInstanceId
-	proxy.adapter = iadapter
-	proxy.coreProxy = cProxy
-	return &proxy
-}
-
-func (rhp *RequestHandlerProxy) Adapter_descriptor() (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Device_types() (*voltha.DeviceTypes, error) {
-	return nil, nil
-}
-
-func (rhp *RequestHandlerProxy) Health() (*voltha.HealthStatus, error) {
-	return nil, nil
-}
-
-func (rhp *RequestHandlerProxy) Adopt_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-
-	logger.Debugw(ctx, "Adopt_device", log.Fields{"deviceId": device.Id})
-
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-
-	//Invoke the adopt device on the adapter
-	if err := rhp.adapter.Adopt_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Reconcile_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-
-	//Invoke the reconcile device API on the adapter
-	if err := rhp.adapter.Reconcile_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Abandon_device(args []*ic.Argument) (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Disable_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-	//Invoke the Disable_device API on the adapter
-	if err := rhp.adapter.Disable_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Reenable_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-	//Invoke the Reenable_device API on the adapter
-	if err := rhp.adapter.Reenable_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Reboot_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-	//Invoke the Reboot_device API on the adapter
-	if err := rhp.adapter.Reboot_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-
-}
-
-func (rhp *RequestHandlerProxy) Self_test_device(args []*ic.Argument) (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Delete_device(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(device.Id, fromTopic.Val)
-	//Invoke the delete_device API on the adapter
-	if err := rhp.adapter.Delete_device(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Get_device_details(args []*ic.Argument) (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Update_flows_bulk(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	logger.Debug(ctx, "Update_flows_bulk")
-	if len(args) < 5 {
-		logger.Warn(ctx, "Update_flows_bulk-invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	flows := &voltha.Flows{}
-	flowMetadata := &voltha.FlowMetadata{}
-	groups := &voltha.FlowGroups{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case "flows":
-			if err := ptypes.UnmarshalAny(arg.Value, flows); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-flows", log.Fields{"error": err})
-				return nil, err
-			}
-		case "groups":
-			if err := ptypes.UnmarshalAny(arg.Value, groups); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-groups", log.Fields{"error": err})
-				return nil, err
-			}
-		case "flow_metadata":
-			if err := ptypes.UnmarshalAny(arg.Value, flowMetadata); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-metadata", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "Update_flows_bulk", log.Fields{"flows": flows, "groups": groups})
-	//Invoke the bulk flow update API of the adapter
-	if err := rhp.adapter.Update_flows_bulk(ctx, device, flows, groups, flowMetadata); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Update_flows_incrementally(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	logger.Debug(ctx, "Update_flows_incrementally")
-	if len(args) < 5 {
-		logger.Warn(ctx, "Update_flows_incrementally-invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	flows := &openflow_13.FlowChanges{}
-	flowMetadata := &voltha.FlowMetadata{}
-	groups := &openflow_13.FlowGroupChanges{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case "flow_changes":
-			if err := ptypes.UnmarshalAny(arg.Value, flows); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-flows", log.Fields{"error": err})
-				return nil, err
-			}
-		case "group_changes":
-			if err := ptypes.UnmarshalAny(arg.Value, groups); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-groups", log.Fields{"error": err})
-				return nil, err
-			}
-		case "flow_metadata":
-			if err := ptypes.UnmarshalAny(arg.Value, flowMetadata); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-metadata", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "Update_flows_incrementally", log.Fields{"flows": flows, "groups": groups})
-	//Invoke the incremental flow update API of the adapter
-	if err := rhp.adapter.Update_flows_incrementally(ctx, device, flows, groups, flowMetadata); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Update_pm_config(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	logger.Debug(ctx, "Update_pm_config")
-	if len(args) < 2 {
-		logger.Warn(ctx, "Update_pm_config-invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	pmConfigs := &voltha.PmConfigs{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case "pm_configs":
-			if err := ptypes.UnmarshalAny(arg.Value, pmConfigs); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-pm-configs", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "Update_pm_config", log.Fields{"device-id": device.Id, "pmConfigs": pmConfigs})
-	//Invoke the pm config update API of the adapter
-	if err := rhp.adapter.Update_pm_config(ctx, device, pmConfigs); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Receive_packet_out(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	logger.Debugw(ctx, "Receive_packet_out", log.Fields{"args": args})
-	if len(args) < 3 {
-		logger.Warn(ctx, "Receive_packet_out-invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	deviceId := &ic.StrType{}
-	egressPort := &ic.IntType{}
-	packet := &openflow_13.OfpPacketOut{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "deviceId":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceId); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "outPort":
-			if err := ptypes.UnmarshalAny(arg.Value, egressPort); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-egressPort", log.Fields{"error": err})
-				return nil, err
-			}
-		case "packet":
-			if err := ptypes.UnmarshalAny(arg.Value, packet); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-packet", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "Receive_packet_out", log.Fields{"device-id": deviceId.Val, "outPort": egressPort, "packet": packet})
-	//Invoke the adopt device on the adapter
-	if err := rhp.adapter.Receive_packet_out(ctx, deviceId.Val, int(egressPort.Val), packet); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Suppress_alarm(args []*ic.Argument) (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Unsuppress_alarm(args []*ic.Argument) (*empty.Empty, error) {
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Get_ofp_device_info(ctx context.Context, args []*ic.Argument) (*ic.SwitchCapability, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	device := &voltha.Device{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-
-	logger.Debugw(ctx, "Get_ofp_device_info", log.Fields{"device-id": device.Id})
-
-	var cap *ic.SwitchCapability
-	var err error
-	if cap, err = rhp.adapter.Get_ofp_device_info(ctx, device); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	logger.Debugw(ctx, "Get_ofp_device_info", log.Fields{"cap": cap})
-	return cap, nil
-}
-
-func (rhp *RequestHandlerProxy) Process_inter_adapter_message(ctx context.Context, args []*ic.Argument) (*empty.Empty, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	iaMsg := &ic.InterAdapterMessage{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "msg":
-			if err := ptypes.UnmarshalAny(arg.Value, iaMsg); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-
-	logger.Debugw(ctx, "Process_inter_adapter_message", log.Fields{"msgId": iaMsg.Header.Id})
-
-	//Invoke the inter adapter API on the handler
-	if err := rhp.adapter.Process_inter_adapter_message(ctx, iaMsg); err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-
-	return new(empty.Empty), nil
-}
-
-func (rhp *RequestHandlerProxy) Process_tech_profile_instance_request(ctx context.Context, args []*ic.Argument) (*ic.InterAdapterTechProfileDownloadMessage, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-	iaTpReqMsg := &ic.InterAdapterTechProfileInstanceRequestMessage{}
-	transactionID := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "msg":
-			if err := ptypes.UnmarshalAny(arg.Value, iaTpReqMsg); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-
-	logger.Debugw(ctx, "Process_tech_profile_instance_request", log.Fields{"tpPath": iaTpReqMsg.TpInstancePath})
-
-	//Invoke the tech profile instance request
-	tpInst := rhp.adapter.Process_tech_profile_instance_request(ctx, iaTpReqMsg)
-
-	return tpInst, nil
-}
-
-func (rhp *RequestHandlerProxy) Download_image(ctx context.Context, args []*ic.Argument) (*voltha.ImageDownload, error) {
-	device, image, err := unMarshalImageDowload(args, ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownload, err := rhp.adapter.Download_image(ctx, device, image)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownload, nil
-}
-
-func (rhp *RequestHandlerProxy) Get_image_download_status(ctx context.Context, args []*ic.Argument) (*voltha.ImageDownload, error) {
-	device, image, err := unMarshalImageDowload(args, ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownload, err := rhp.adapter.Get_image_download_status(ctx, device, image)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownload, nil
-}
-
-func (rhp *RequestHandlerProxy) Cancel_image_download(ctx context.Context, args []*ic.Argument) (*voltha.ImageDownload, error) {
-	device, image, err := unMarshalImageDowload(args, ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownload, err := rhp.adapter.Cancel_image_download(ctx, device, image)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownload, nil
-}
-
-func (rhp *RequestHandlerProxy) Activate_image_update(ctx context.Context, args []*ic.Argument) (*voltha.ImageDownload, error) {
-
-	device, image, err := unMarshalImageDowload(args, ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownload, err := rhp.adapter.Activate_image_update(ctx, device, image)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownload, nil
-}
-
-func (rhp *RequestHandlerProxy) Revert_image_update(ctx context.Context, args []*ic.Argument) (*voltha.ImageDownload, error) {
-	device, image, err := unMarshalImageDowload(args, ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownload, err := rhp.adapter.Revert_image_update(ctx, device, image)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownload, nil
-}
-
-func unMarshalImageDowload(args []*ic.Argument, ctx context.Context) (*voltha.Device, *voltha.ImageDownload, error) {
-	if len(args) < 4 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, nil, err
-	}
-	device := &voltha.Device{}
-	image := &voltha.ImageDownload{}
-	transactionID := &ic.StrType{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, nil, err
-			}
-		case "request":
-			if err := ptypes.UnmarshalAny(arg.Value, image); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-image", log.Fields{"error": err})
-				return nil, nil, err
-			}
-		case kafka.TransactionKey:
-			if err := ptypes.UnmarshalAny(arg.Value, transactionID); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-transaction-ID", log.Fields{"error": err})
-				return nil, nil, err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return nil, nil, err
-			}
-		}
-	}
-	return device, image, nil
-}
-
-func (rhp *RequestHandlerProxy) Enable_port(ctx context.Context, args []*ic.Argument) error {
-	logger.Debugw(ctx, "enable_port", log.Fields{"args": args})
-	deviceId, port, err := rhp.getEnableDisableParams(ctx, args)
-	if err != nil {
-		logger.Warnw(ctx, "enable_port", log.Fields{"args": args, "device-id": deviceId, "port": port})
-		return err
-	}
-	return rhp.adapter.Enable_port(ctx, deviceId, port)
-}
-
-func (rhp *RequestHandlerProxy) Disable_port(ctx context.Context, args []*ic.Argument) error {
-	logger.Debugw(ctx, "disable_port", log.Fields{"args": args})
-	deviceId, port, err := rhp.getEnableDisableParams(ctx, args)
-	if err != nil {
-		logger.Warnw(ctx, "disable_port", log.Fields{"args": args, "device-id": deviceId, "port": port})
-		return err
-	}
-	return rhp.adapter.Disable_port(ctx, deviceId, port)
-}
-
-func (rhp *RequestHandlerProxy) getEnableDisableParams(ctx context.Context, args []*ic.Argument) (string, *voltha.Port, error) {
-	logger.Debugw(ctx, "getEnableDisableParams", log.Fields{"args": args})
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return "", nil, errors.New("invalid-number-of-args")
-	}
-	deviceId := &ic.StrType{}
-	port := &voltha.Port{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "deviceId":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceId); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return "", nil, err
-			}
-		case "port":
-			if err := ptypes.UnmarshalAny(arg.Value, port); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-port", log.Fields{"error": err})
-				return "", nil, err
-			}
-		}
-	}
-	return deviceId.Val, port, nil
-}
-
-func (rhp *RequestHandlerProxy) Child_device_lost(ctx context.Context, args []*ic.Argument) error {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return errors.New("invalid-number-of-args")
-	}
-	childDevice := &voltha.Device{}
-	fromTopic := &ic.StrType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "childDevice":
-			if err := ptypes.UnmarshalAny(arg.Value, childDevice); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-child-device", log.Fields{"error": err})
-				return err
-			}
-		case kafka.FromTopic:
-			if err := ptypes.UnmarshalAny(arg.Value, fromTopic); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-from-topic", log.Fields{"error": err})
-				return err
-			}
-		}
-	}
-	//Update the core reference for that device
-	rhp.coreProxy.UpdateCoreReference(childDevice.ParentId, fromTopic.Val)
-	//Invoke the Child_device_lost API on the adapter
-	if err := rhp.adapter.Child_device_lost(ctx, childDevice); err != nil {
-		return status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return nil
-}
-
-func (rhp *RequestHandlerProxy) Start_omci_test(ctx context.Context, args []*ic.Argument) (*ic.TestResponse, error) {
-	if len(args) < 2 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		err := errors.New("invalid-number-of-args")
-		return nil, err
-	}
-
-	// TODO: See related comment in voltha-go:adapter_proxy_go:startOmciTest()
-	//   Second argument should perhaps be uuid instead of omcitestrequest
-
-	device := &voltha.Device{}
-	request := &voltha.OmciTestRequest{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case "omcitestrequest":
-			if err := ptypes.UnmarshalAny(arg.Value, request); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-omcitestrequest", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	logger.Debugw(ctx, "Start_omci_test", log.Fields{"device-id": device.Id, "req": request})
-	result, err := rhp.adapter.Start_omci_test(ctx, device, request)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return result, nil
-}
-func (rhp *RequestHandlerProxy) Get_ext_value(ctx context.Context, args []*ic.Argument) (*voltha.ReturnValues, error) {
-	if len(args) < 3 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return nil, errors.New("invalid-number-of-args")
-	}
-
-	pDeviceId := &ic.StrType{}
-	device := &voltha.Device{}
-	valuetype := &ic.IntType{}
-	for _, arg := range args {
-		switch arg.Key {
-		case "device":
-			if err := ptypes.UnmarshalAny(arg.Value, device); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		case "pDeviceId":
-			if err := ptypes.UnmarshalAny(arg.Value, pDeviceId); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-parent-device-id", log.Fields{"error": err})
-				return nil, err
-			}
-		case "valuetype":
-			if err := ptypes.UnmarshalAny(arg.Value, valuetype); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-valuetype", log.Fields{"error": err})
-				return nil, err
-			}
-		default:
-			logger.Warnw(ctx, "key-not-found", log.Fields{"arg.Key": arg.Key})
-		}
-	}
-
-	//Invoke the Get_value API on the adapter
-	return rhp.adapter.Get_ext_value(ctx, pDeviceId.Val, device, voltha.ValueType_Type(valuetype.Val))
-}
-
-func (rhp *RequestHandlerProxy) Single_get_value_request(ctx context.Context, args []*ic.Argument) (*extension.SingleGetValueResponse, error) {
-	logger.Debugw(ctx, "req handler Single_get_value_request", log.Fields{"no of args": len(args), "args": args})
-
-	if len(args) < 1 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return nil, errors.New("invalid-number-of-args")
-	}
-	singleGetvalueReq := extension.SingleGetValueRequest{}
-	errResp := func(status extension.GetValueResponse_Status, reason extension.GetValueResponse_ErrorReason) *extension.SingleGetValueResponse {
-		return &extension.SingleGetValueResponse{
-			Response: &extension.GetValueResponse{
-				Status:    status,
-				ErrReason: reason,
-			},
-		}
-	}
-	for _, arg := range args {
-		switch arg.Key {
-		case "request":
-			if err := ptypes.UnmarshalAny(arg.Value, &singleGetvalueReq); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-singleGetvalueReq", log.Fields{"error": err})
-				return errResp(extension.GetValueResponse_ERROR, extension.GetValueResponse_REASON_UNDEFINED), nil
-			}
-		default:
-			logger.Warnw(ctx, "key-not-found", log.Fields{"arg.Key": arg.Key})
-		}
-	}
-	logger.Debugw(ctx, "invoke rhp.adapter.Single_get_value_request ", log.Fields{"singleGetvalueReq": singleGetvalueReq})
-	return rhp.adapter.Single_get_value_request(ctx, singleGetvalueReq)
-}
-
-func (rhp *RequestHandlerProxy) getDeviceDownloadImageRequest(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageDownloadRequest, error) {
-	logger.Debugw(ctx, "getDeviceDownloadImageRequest", log.Fields{"args": args})
-	if len(args) < 1 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return nil, errors.New("invalid-number-of-args")
-	}
-
-	deviceDownloadImageReq := ic.DeviceImageDownloadRequest{}
-
-	for _, arg := range args {
-		switch arg.Key {
-		case "deviceImageDownloadReq":
-			if err := ptypes.UnmarshalAny(arg.Value, &deviceDownloadImageReq); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	return &deviceDownloadImageReq, nil
-}
-
-func (rhp *RequestHandlerProxy) getDeviceImageRequest(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageRequest, error) {
-	logger.Debugw(ctx, "getDeviceImageRequest", log.Fields{"args": args})
-	if len(args) < 1 {
-		logger.Warn(ctx, "invalid-number-of-args", log.Fields{"args": args})
-		return nil, errors.New("invalid-number-of-args")
-	}
-
-	deviceImageReq := ic.DeviceImageRequest{}
-
-	for _, arg := range args {
-		switch arg.Key {
-		case "deviceImageReq":
-			if err := ptypes.UnmarshalAny(arg.Value, &deviceImageReq); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return nil, err
-			}
-		}
-	}
-	return &deviceImageReq, nil
-}
-
-func (rhp *RequestHandlerProxy) getDeviceID(ctx context.Context, args []*ic.Argument) (string, error) {
-	logger.Debugw(ctx, "getDeviceID", log.Fields{"args": args})
-
-	deviceId := &ic.StrType{}
-
-	for _, arg := range args {
-		switch arg.Key {
-		case "deviceId":
-			if err := ptypes.UnmarshalAny(arg.Value, deviceId); err != nil {
-				logger.Warnw(ctx, "cannot-unmarshal-device", log.Fields{"error": err})
-				return "", err
-			}
-		}
-	}
-
-	if deviceId.Val == "" {
-		return "", errors.New("invalid argument")
-	}
-
-	return deviceId.Val, nil
-}
-
-func (rhp *RequestHandlerProxy) Download_onu_image(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageResponse, error) {
-	imageDownloadReq, err := rhp.getDeviceDownloadImageRequest(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	imageDownloadRsp, err := rhp.adapter.Download_onu_image(ctx, imageDownloadReq)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageDownloadRsp, nil
-}
-
-func (rhp *RequestHandlerProxy) Get_onu_image_status(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageResponse, error) {
-	imageStatusReq, err := rhp.getDeviceImageRequest(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	imageStatus, err := rhp.adapter.Get_onu_image_status(ctx, imageStatusReq)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageStatus, nil
-}
-
-func (rhp *RequestHandlerProxy) Abort_onu_image_upgrade(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageResponse, error) {
-	imageAbortReq, err := rhp.getDeviceImageRequest(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	imageAbortRsp, err := rhp.adapter.Abort_onu_image_upgrade(ctx, imageAbortReq)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageAbortRsp, nil
-}
-
-func (rhp *RequestHandlerProxy) Activate_onu_image(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageResponse, error) {
-	imageActivateReq, err := rhp.getDeviceImageRequest(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	imageActivateRsp, err := rhp.adapter.Activate_onu_image(ctx, imageActivateReq)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return imageActivateRsp, nil
-}
-
-func (rhp *RequestHandlerProxy) Commit_onu_image(ctx context.Context, args []*ic.Argument) (*voltha.DeviceImageResponse, error) {
-	imageCommitReq, err := rhp.getDeviceImageRequest(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	imageCommitRsp, err := rhp.adapter.Commit_onu_image(ctx, imageCommitReq)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-
-	return imageCommitRsp, nil
-}
-
-func (rhp *RequestHandlerProxy) Get_onu_images(ctx context.Context, args []*ic.Argument) (*voltha.OnuImages, error) {
-	deviceID, err := rhp.getDeviceID(ctx, args)
-	if err != nil {
-		return nil, err
-	}
-
-	onuImages, err := rhp.adapter.Get_onu_images(ctx, deviceID)
-	if err != nil {
-		return nil, status.Errorf(codes.NotFound, "%s", err.Error())
-	}
-	return onuImages, nil
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/utils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/utils.go
deleted file mode 100644
index 35f227e..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/common/utils.go
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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 common
-
-import (
-	"context"
-	"fmt"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"google.golang.org/grpc/codes"
-	"math/rand"
-	"time"
-)
-
-//GetRandomSerialNumber returns a serial number formatted as "HOST:PORT"
-func GetRandomSerialNumber() string {
-	rand.Seed(time.Now().UnixNano())
-	return fmt.Sprintf("%d.%d.%d.%d:%d",
-		rand.Intn(255),
-		rand.Intn(255),
-		rand.Intn(255),
-		rand.Intn(255),
-		rand.Intn(9000)+1000,
-	)
-}
-
-//GetRandomMacAddress returns a random mac address
-func GetRandomMacAddress() string {
-	rand.Seed(time.Now().UnixNano())
-	return fmt.Sprintf("%02x:%02x:%02x:%02x:%02x:%02x",
-		rand.Intn(128),
-		rand.Intn(128),
-		rand.Intn(128),
-		rand.Intn(128),
-		rand.Intn(128),
-		rand.Intn(128),
-	)
-}
-
-const letterBytes = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"
-const (
-	letterIdxBits = 6                    // 6 bits to represent a letter index
-	letterIdxMask = 1<<letterIdxBits - 1 // All 1-bits, as many as letterIdxBits
-	letterIdxMax  = 63 / letterIdxBits   // # of letter indices fitting in 63 bits
-)
-
-var src = rand.NewSource(time.Now().UnixNano())
-
-func GetRandomString(n int) string {
-	b := make([]byte, n)
-	// A src.Int63() generates 63 random bits, enough for letterIdxMax characters!
-	for i, cache, remain := n-1, src.Int63(), letterIdxMax; i >= 0; {
-		if remain == 0 {
-			cache, remain = src.Int63(), letterIdxMax
-		}
-		if idx := int(cache & letterIdxMask); idx < len(letterBytes) {
-			b[i] = letterBytes[idx]
-			i--
-		}
-		cache >>= letterIdxBits
-		remain--
-	}
-	return string(b)
-}
-
-func ICProxyErrorCodeToGrpcErrorCode(ctx context.Context, icErr ic.ErrorCodeCodes) codes.Code {
-	switch icErr {
-	case ic.ErrorCode_INVALID_PARAMETERS:
-		return codes.InvalidArgument
-	case ic.ErrorCode_UNSUPPORTED_REQUEST:
-		return codes.Unavailable
-	case ic.ErrorCode_DEADLINE_EXCEEDED:
-		return codes.DeadlineExceeded
-	default:
-		logger.Warnw(ctx, "cannnot-map-ic-error-code-to-grpc-error-code", log.Fields{"err": icErr})
-		return codes.Internal
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/iAdapter.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/iAdapter.go
deleted file mode 100644
index aca4271..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/adapters/iAdapter.go
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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 adapters
-
-import (
-	"context"
-
-	"github.com/opencord/voltha-protos/v4/go/extension"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opencord/voltha-protos/v4/go/openflow_13"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-//IAdapter represents the set of APIs a voltha adapter has to support.
-type IAdapter interface {
-	Adapter_descriptor(ctx context.Context) error
-	Device_types(ctx context.Context) (*voltha.DeviceTypes, error)
-	Health(ctx context.Context) (*voltha.HealthStatus, error)
-	Adopt_device(ctx context.Context, device *voltha.Device) error
-	Reconcile_device(ctx context.Context, device *voltha.Device) error
-	Abandon_device(ctx context.Context, device *voltha.Device) error
-	Disable_device(ctx context.Context, device *voltha.Device) error
-	Reenable_device(ctx context.Context, device *voltha.Device) error
-	Reboot_device(ctx context.Context, device *voltha.Device) error
-	Self_test_device(ctx context.Context, device *voltha.Device) error
-	Delete_device(ctx context.Context, device *voltha.Device) error
-	Get_device_details(ctx context.Context, device *voltha.Device) error
-	Update_flows_bulk(ctx context.Context, device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) error
-	Update_flows_incrementally(ctx context.Context, device *voltha.Device, flows *openflow_13.FlowChanges, groups *openflow_13.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) error
-	Update_pm_config(ctx context.Context, device *voltha.Device, pm_configs *voltha.PmConfigs) error
-	Receive_packet_out(ctx context.Context, deviceId string, egress_port_no int, msg *openflow_13.OfpPacketOut) error
-	Suppress_event(ctx context.Context, filter *voltha.EventFilter) error
-	Unsuppress_event(ctx context.Context, filter *voltha.EventFilter) error
-	Get_ofp_device_info(ctx context.Context, device *voltha.Device) (*ic.SwitchCapability, error)
-	Process_inter_adapter_message(ctx context.Context, msg *ic.InterAdapterMessage) error
-	Process_tech_profile_instance_request(ctx context.Context, msg *ic.InterAdapterTechProfileInstanceRequestMessage) *ic.InterAdapterTechProfileDownloadMessage
-	Download_image(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error)
-	Get_image_download_status(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error)
-	Cancel_image_download(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error)
-	Activate_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error)
-	Revert_image_update(ctx context.Context, device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error)
-	Enable_port(ctx context.Context, deviceId string, port *voltha.Port) error
-	Disable_port(ctx context.Context, deviceId string, port *voltha.Port) error
-	Child_device_lost(ctx context.Context, childDevice *voltha.Device) error
-	Start_omci_test(ctx context.Context, device *voltha.Device, request *voltha.OmciTestRequest) (*voltha.TestResponse, error)
-	Get_ext_value(ctx context.Context, deviceId string, device *voltha.Device, valueflag voltha.ValueType_Type) (*voltha.ReturnValues, error)
-	Single_get_value_request(ctx context.Context, request extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error)
-	Download_onu_image(ctx context.Context, request *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error)
-	Get_onu_image_status(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
-	Abort_onu_image_upgrade(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
-	Get_onu_images(ctx context.Context, deviceID string) (*voltha.OnuImages, error)
-	Activate_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
-	Commit_onu_image(ctx context.Context, in *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/common.go
deleted file mode 100644
index 606d18c..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 config
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/configmanager.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/configmanager.go
deleted file mode 100644
index f5efa36..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/configmanager.go
+++ /dev/null
@@ -1,284 +0,0 @@
-/*
- * 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 config
-
-import (
-	"context"
-	"fmt"
-	"os"
-	"strings"
-	"time"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-const (
-	defaultkvStoreConfigPath     = "config"
-	defaultkvStoreDataPathPrefix = "service/voltha_voltha"
-	kvStorePathSeparator         = "/"
-)
-
-// ConfigType represents the type for which config is created inside the kvstore
-// For example, loglevel
-type ConfigType int
-
-const (
-	ConfigTypeLogLevel ConfigType = iota
-	ConfigTypeMetadata
-	ConfigTypeKafka
-	ConfigTypeLogFeatures
-)
-
-func (c ConfigType) String() string {
-	return [...]string{"loglevel", "metadata", "kafka", "logfeatures"}[c]
-}
-
-// ChangeEvent represents the event recieved from watch
-// For example, Put Event
-type ChangeEvent int
-
-const (
-	Put ChangeEvent = iota
-	Delete
-)
-
-func (ce ChangeEvent) String() string {
-	return [...]string{"Put", "Delete"}[ce]
-}
-
-// ConfigChangeEvent represents config for the events recieved from watch
-// For example,ChangeType is Put ,ConfigAttribute default
-type ConfigChangeEvent struct {
-	ChangeType      ChangeEvent
-	ConfigAttribute string
-}
-
-// ConfigManager is a wrapper over Backend to maintain Configuration of voltha components
-// in kvstore based persistent storage
-type ConfigManager struct {
-	Backend               *db.Backend
-	KVStoreConfigPrefix   string
-	KVStoreDataPathPrefix string
-}
-
-// ComponentConfig represents a category of configuration for a specific VOLTHA component type
-// stored in a persistent storage pointed to by Config Manager
-// For example, one ComponentConfig instance will be created for loglevel config type for rw-core
-// component while another ComponentConfig instance will refer to connection config type for same
-// rw-core component. So, there can be multiple ComponentConfig instance created per component
-// pointing to different category of configuration.
-//
-// Configuration pointed to be by ComponentConfig is stored in kvstore as a list of key/value pairs
-// under the hierarchical tree with following base path
-// <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/
-//
-// For example, rw-core ComponentConfig for loglevel config entries will be stored under following path
-// /voltha/service/config/rw-core/loglevel/
-type ComponentConfig struct {
-	cManager         *ConfigManager
-	componentLabel   string
-	configType       ConfigType
-	changeEventChan  chan *ConfigChangeEvent
-	kvStoreEventChan chan *kvstore.Event
-}
-
-func NewConfigManager(ctx context.Context, kvClient kvstore.Client, kvStoreType, kvStoreAddress string, kvStoreTimeout time.Duration) *ConfigManager {
-	var kvStorePrefix string
-	if prefix, present := os.LookupEnv("KV_STORE_DATAPATH_PREFIX"); present {
-		kvStorePrefix = prefix
-		logger.Infow(ctx, "KV_STORE_DATAPATH_PREFIX env variable is set, ", log.Fields{"kvStoreDataPathPrefix": kvStorePrefix})
-	} else {
-		kvStorePrefix = defaultkvStoreDataPathPrefix
-		logger.Infow(ctx, "KV_STORE_DATAPATH_PREFIX env variable is not set, using default", log.Fields{"kvStoreDataPathPrefix": defaultkvStoreDataPathPrefix})
-	}
-	return &ConfigManager{
-		KVStoreConfigPrefix:   defaultkvStoreConfigPath,
-		KVStoreDataPathPrefix: kvStorePrefix,
-		Backend: &db.Backend{
-			Client:     kvClient,
-			StoreType:  kvStoreType,
-			Address:    kvStoreAddress,
-			Timeout:    kvStoreTimeout,
-			PathPrefix: kvStorePrefix,
-		},
-	}
-}
-
-// RetrieveComponentList list the component Names for which loglevel is stored in kvstore
-func (c *ConfigManager) RetrieveComponentList(ctx context.Context, configType ConfigType) ([]string, error) {
-	data, err := c.Backend.List(ctx, c.KVStoreConfigPrefix)
-	if err != nil {
-		return nil, err
-	}
-
-	// Looping through the data recieved from the Backend for config
-	// Trimming and Splitting the required key and value from data and  storing as componentName,PackageName and Level
-	// For Example, recieved key would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default and value \"DEBUG\"
-	// Then in default will be stored as PackageName,componentName as <Component Name> and DEBUG will be stored as value in List struct
-	ccPathPrefix := kvStorePathSeparator + configType.String() + kvStorePathSeparator
-	pathPrefix := c.KVStoreDataPathPrefix + kvStorePathSeparator + c.KVStoreConfigPrefix + kvStorePathSeparator
-	var list []string
-	keys := make(map[string]interface{})
-	for attr := range data {
-		cname := strings.TrimPrefix(attr, pathPrefix)
-		cName := strings.SplitN(cname, ccPathPrefix, 2)
-		if len(cName) != 2 {
-			continue
-		}
-		if _, exist := keys[cName[0]]; !exist {
-			keys[cName[0]] = nil
-			list = append(list, cName[0])
-		}
-	}
-	return list, nil
-}
-
-// Initialize the component config
-func (cm *ConfigManager) InitComponentConfig(componentLabel string, configType ConfigType) *ComponentConfig {
-
-	return &ComponentConfig{
-		componentLabel:   componentLabel,
-		configType:       configType,
-		cManager:         cm,
-		changeEventChan:  nil,
-		kvStoreEventChan: nil,
-	}
-
-}
-
-func (c *ComponentConfig) makeConfigPath() string {
-
-	cType := c.configType.String()
-	return c.cManager.KVStoreConfigPrefix + kvStorePathSeparator +
-		c.componentLabel + kvStorePathSeparator + cType
-}
-
-// MonitorForConfigChange watch on the subkeys for the given key
-// Any changes to the subkeys for the given key will return an event channel
-// Then Event channel will be processed and  new event channel with required values will be created and return
-// For example, rw-core will be watching on <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/
-// will return an event channel for PUT,DELETE eventType.
-// Then values from event channel will be processed and  stored in kvStoreEventChan.
-func (c *ComponentConfig) MonitorForConfigChange(ctx context.Context) chan *ConfigChangeEvent {
-	key := c.makeConfigPath()
-
-	logger.Debugw(ctx, "monitoring-for-config-change", log.Fields{"key": key})
-
-	c.changeEventChan = make(chan *ConfigChangeEvent, 1)
-
-	c.kvStoreEventChan = c.cManager.Backend.CreateWatch(ctx, key, true)
-
-	go c.processKVStoreWatchEvents(ctx)
-
-	return c.changeEventChan
-}
-
-// processKVStoreWatchEvents process event channel recieved from the Backend for any ChangeType
-// It checks for the EventType is valid or not.For the valid EventTypes creates ConfigChangeEvent and send it on channel
-func (c *ComponentConfig) processKVStoreWatchEvents(ctx context.Context) {
-
-	ccKeyPrefix := c.makeConfigPath()
-
-	logger.Debugw(ctx, "processing-kvstore-event-change", log.Fields{"key-prefix": ccKeyPrefix})
-
-	ccPathPrefix := c.cManager.Backend.PathPrefix + ccKeyPrefix + kvStorePathSeparator
-
-	for watchResp := range c.kvStoreEventChan {
-
-		if watchResp.EventType == kvstore.CONNECTIONDOWN || watchResp.EventType == kvstore.UNKNOWN {
-			logger.Warnw(ctx, "received-invalid-change-type-in-watch-channel-from-kvstore", log.Fields{"change-type": watchResp.EventType})
-			continue
-		}
-
-		// populating the configAttribute from the received Key
-		// For Example, Key received would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default
-		// Storing default in configAttribute variable
-		ky := fmt.Sprintf("%s", watchResp.Key)
-
-		c.changeEventChan <- &ConfigChangeEvent{
-			ChangeType:      ChangeEvent(watchResp.EventType),
-			ConfigAttribute: strings.TrimPrefix(ky, ccPathPrefix),
-		}
-	}
-}
-
-// Retrieves value of a specific config key. Value of key is returned in String format
-func (c *ComponentConfig) Retrieve(ctx context.Context, configKey string) (string, error) {
-	key := c.makeConfigPath() + "/" + configKey
-
-	logger.Debugw(ctx, "retrieving-config", log.Fields{"key": key})
-
-	if kvpair, err := c.cManager.Backend.Get(ctx, key); err != nil {
-		return "", err
-	} else {
-		if kvpair == nil {
-			return "", fmt.Errorf("config-key-does-not-exist : %s", key)
-		}
-
-		value := strings.Trim(fmt.Sprintf("%s", kvpair.Value), "\"")
-		logger.Debugw(ctx, "retrieved-config", log.Fields{"key": key, "value": value})
-		return value, nil
-	}
-}
-
-func (c *ComponentConfig) RetrieveAll(ctx context.Context) (map[string]string, error) {
-	key := c.makeConfigPath()
-
-	logger.Debugw(ctx, "retreiving-list", log.Fields{"key": key})
-
-	data, err := c.cManager.Backend.List(ctx, key)
-	if err != nil {
-		return nil, err
-	}
-
-	// Looping through the data recieved from the Backend for the given key
-	// Trimming the required key and value from data and  storing as key/value pair
-	// For Example, recieved key would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default and value \"DEBUG\"
-	// Then in default will be stored as key and DEBUG will be stored as value in map[string]string
-	res := make(map[string]string)
-	ccPathPrefix := c.cManager.Backend.PathPrefix + kvStorePathSeparator + key + kvStorePathSeparator
-	for attr, val := range data {
-		res[strings.TrimPrefix(attr, ccPathPrefix)] = strings.Trim(fmt.Sprintf("%s", val.Value), "\"")
-	}
-
-	return res, nil
-}
-
-func (c *ComponentConfig) Save(ctx context.Context, configKey string, configValue string) error {
-	key := c.makeConfigPath() + "/" + configKey
-
-	logger.Debugw(ctx, "saving-config", log.Fields{"key": key, "value": configValue})
-
-	//save the data for update config
-	if err := c.cManager.Backend.Put(ctx, key, configValue); err != nil {
-		return err
-	}
-	return nil
-}
-
-func (c *ComponentConfig) Delete(ctx context.Context, configKey string) error {
-	//construct key using makeConfigPath
-	key := c.makeConfigPath() + "/" + configKey
-
-	logger.Debugw(ctx, "deleting-config", log.Fields{"key": key})
-	//delete the config
-	if err := c.cManager.Backend.Delete(ctx, key); err != nil {
-		return err
-	}
-	return nil
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logcontroller.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logcontroller.go
deleted file mode 100644
index 68bfb32..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logcontroller.go
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * 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 Config provides dynamic logging configuration for specific Voltha component with loglevel lookup
-// from etcd kvstore implemented using Backend.
-// Any Voltha component can start utilizing dynamic logging by starting goroutine of StartLogLevelConfigProcessing after
-// starting kvClient for the component.
-
-package config
-
-import (
-	"context"
-	"crypto/md5"
-	"encoding/json"
-	"errors"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"os"
-	"sort"
-	"strings"
-)
-
-const (
-	defaultLogLevelKey                = "default"          // kvstore key containing default loglevel
-	globalConfigRootNode              = "global"           // Root Node in kvstore containing global config
-	initialGlobalDefaultLogLevelValue = "WARN"             // Hard-coded Global Default loglevel pushed at PoD startup
-	logPackagesListKey                = "log_package_list" // kvstore key containing list of allowed log packages
-)
-
-// ComponentLogController represents a Configuration for Logging Config of specific Voltha component type
-// It stores ComponentConfig and GlobalConfig of loglevel config of specific Voltha component type
-// For example,ComponentLogController instance will be created for rw-core component
-type ComponentLogController struct {
-	ComponentName       string
-	componentNameConfig *ComponentConfig
-	GlobalConfig        *ComponentConfig
-	configManager       *ConfigManager
-	logHash             [16]byte
-	initialLogLevel     string // Initial default log level set by helm chart
-}
-
-func NewComponentLogController(ctx context.Context, cm *ConfigManager) (*ComponentLogController, error) {
-	logger.Debug(ctx, "creating-new-component-log-controller")
-	componentName := os.Getenv("COMPONENT_NAME")
-	if componentName == "" {
-		return nil, errors.New("Unable to retrieve PoD Component Name from Runtime env")
-	}
-
-	var defaultLogLevel string
-	var err error
-	// Retrieve and save default log level; used for fallback if all loglevel config is cleared in etcd
-	if defaultLogLevel, err = log.LogLevelToString(log.GetDefaultLogLevel()); err != nil {
-		defaultLogLevel = "DEBUG"
-	}
-
-	return &ComponentLogController{
-		ComponentName:       componentName,
-		componentNameConfig: nil,
-		GlobalConfig:        nil,
-		configManager:       cm,
-		initialLogLevel:     defaultLogLevel,
-	}, nil
-
-}
-
-// StartLogLevelConfigProcessing initialize component config and global config
-// Then, it persists initial default Loglevels into Config Store before
-// starting the loading and processing of all Log Configuration
-func StartLogLevelConfigProcessing(cm *ConfigManager, ctx context.Context) {
-	cc, err := NewComponentLogController(ctx, cm)
-	if err != nil {
-		logger.Errorw(ctx, "unable-to-construct-component-log-controller-instance-for-log-config-monitoring", log.Fields{"error": err})
-		return
-	}
-
-	cc.GlobalConfig = cm.InitComponentConfig(globalConfigRootNode, ConfigTypeLogLevel)
-	logger.Debugw(ctx, "global-log-config", log.Fields{"cc-global-config": cc.GlobalConfig})
-
-	cc.componentNameConfig = cm.InitComponentConfig(cc.ComponentName, ConfigTypeLogLevel)
-	logger.Debugw(ctx, "component-log-config", log.Fields{"cc-component-name-config": cc.componentNameConfig})
-
-	cc.persistInitialDefaultLogConfigs(ctx)
-
-	cc.persistRegisteredLogPackageList(ctx)
-
-	cc.processLogConfig(ctx)
-}
-
-// Method to persist Global default loglevel into etcd, if not set yet
-// It also checks and set Component default loglevel into etcd with initial loglevel set from command line
-func (c *ComponentLogController) persistInitialDefaultLogConfigs(ctx context.Context) {
-
-	_, err := c.GlobalConfig.Retrieve(ctx, defaultLogLevelKey)
-	if err != nil {
-		logger.Debugw(ctx, "failed-to-retrieve-global-default-log-config-at-startup", log.Fields{"error": err})
-
-		err = c.GlobalConfig.Save(ctx, defaultLogLevelKey, initialGlobalDefaultLogLevelValue)
-		if err != nil {
-			logger.Errorw(ctx, "failed-to-persist-global-default-log-config-at-startup", log.Fields{"error": err, "loglevel": initialGlobalDefaultLogLevelValue})
-		}
-	}
-
-	_, err = c.componentNameConfig.Retrieve(ctx, defaultLogLevelKey)
-	if err != nil {
-		logger.Debugw(ctx, "failed-to-retrieve-component-default-log-config-at-startup", log.Fields{"error": err})
-
-		err = c.componentNameConfig.Save(ctx, defaultLogLevelKey, c.initialLogLevel)
-		if err != nil {
-			logger.Errorw(ctx, "failed-to-persist-component-default-log-config-at-startup", log.Fields{"error": err, "loglevel": c.initialLogLevel})
-		}
-	}
-}
-
-// Method to save list of all registered packages for component into config kvstore. A single string
-// is constructed with comma-separated package names in sorted order and persisted
-func (c *ComponentLogController) persistRegisteredLogPackageList(ctx context.Context) {
-
-	componentMetadataConfig := c.configManager.InitComponentConfig(c.ComponentName, ConfigTypeMetadata)
-	logger.Debugw(ctx, "component-metadata-config", log.Fields{"component-metadata-config": componentMetadataConfig})
-
-	packageList := log.GetPackageNames()
-	packageList = append(packageList, defaultLogLevelKey)
-	sort.Strings(packageList)
-
-	packageNames, err := json.Marshal(packageList)
-	if err != nil {
-		logger.Errorw(ctx, "failed-to-marshal-log-package-list-for-storage", log.Fields{"error": err, "packageList": packageList})
-		return
-	}
-
-	if err := componentMetadataConfig.Save(ctx, logPackagesListKey, string(packageNames)); err != nil {
-		logger.Errorw(ctx, "failed-to-persist-component-registered-log-package-list-at-startup", log.Fields{"error": err, "packageNames": packageNames})
-	}
-}
-
-// ProcessLogConfig will first load and apply log config and then start waiting on component config and global config
-// channels for any changes. Event channel will be recieved from Backend for valid change type
-// Then data for componentn log config and global log config will be retrieved from Backend and stored in updatedLogConfig in precedence order
-// If any changes in updatedLogConfig will be applied on component
-func (c *ComponentLogController) processLogConfig(ctx context.Context) {
-
-	// Load and apply Log Config for first time
-	initialLogConfig, err := c.buildUpdatedLogConfig(ctx)
-	if err != nil {
-		logger.Warnw(ctx, "unable-to-load-log-config-at-startup", log.Fields{"error": err})
-	} else {
-		if err := c.loadAndApplyLogConfig(ctx, initialLogConfig); err != nil {
-			logger.Warnw(ctx, "unable-to-apply-log-config-at-startup", log.Fields{"error": err})
-		}
-	}
-
-	componentConfigEventChan := c.componentNameConfig.MonitorForConfigChange(ctx)
-
-	globalConfigEventChan := c.GlobalConfig.MonitorForConfigChange(ctx)
-
-	// process the events for componentName and  global config
-	var configEvent *ConfigChangeEvent
-	for {
-		select {
-		case configEvent = <-globalConfigEventChan:
-		case configEvent = <-componentConfigEventChan:
-
-		}
-		logger.Debugw(ctx, "processing-log-config-change", log.Fields{"ChangeType": configEvent.ChangeType, "Package": configEvent.ConfigAttribute})
-
-		updatedLogConfig, err := c.buildUpdatedLogConfig(ctx)
-		if err != nil {
-			logger.Warnw(ctx, "unable-to-fetch-updated-log-config", log.Fields{"error": err})
-			continue
-		}
-
-		logger.Debugw(ctx, "applying-updated-log-config", log.Fields{"updated-log-config": updatedLogConfig})
-
-		if err := c.loadAndApplyLogConfig(ctx, updatedLogConfig); err != nil {
-			logger.Warnw(ctx, "unable-to-load-and-apply-log-config", log.Fields{"error": err})
-		}
-	}
-
-}
-
-// get active loglevel from the zap logger
-func getActiveLogLevels(ctx context.Context) map[string]string {
-	loglevels := make(map[string]string)
-
-	// now do the default log level
-	if level, err := log.LogLevelToString(log.GetDefaultLogLevel()); err == nil {
-		loglevels[defaultLogLevelKey] = level
-	}
-
-	// do the per-package log levels
-	for _, packageName := range log.GetPackageNames() {
-		level, err := log.GetPackageLogLevel(packageName)
-		if err != nil {
-			logger.Warnw(ctx, "unable-to-fetch-current-active-loglevel-for-package-name", log.Fields{"package-name": packageName, "error": err})
-			continue
-		}
-
-		if l, err := log.LogLevelToString(level); err == nil {
-			loglevels[packageName] = l
-		}
-	}
-
-	logger.Debugw(ctx, "retreived-log-levels-from-zap-logger", log.Fields{"loglevels": loglevels})
-
-	return loglevels
-}
-
-func (c *ComponentLogController) getGlobalLogConfig(ctx context.Context) (string, error) {
-
-	globalDefaultLogLevel, err := c.GlobalConfig.Retrieve(ctx, defaultLogLevelKey)
-	if err != nil {
-		return "", err
-	}
-
-	// Handle edge cases when global default loglevel is deleted directly from etcd or set to a invalid value
-	// We should use hard-coded initial default value in such cases
-	if globalDefaultLogLevel == "" {
-		logger.Warn(ctx, "global-default-loglevel-not-found-in-config-store")
-		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
-	}
-
-	if _, err := log.StringToLogLevel(globalDefaultLogLevel); err != nil {
-		logger.Warnw(ctx, "unsupported-loglevel-config-defined-at-global-default", log.Fields{"log-level": globalDefaultLogLevel})
-		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
-	}
-
-	logger.Debugw(ctx, "retrieved-global-default-loglevel", log.Fields{"level": globalDefaultLogLevel})
-
-	return globalDefaultLogLevel, nil
-}
-
-func (c *ComponentLogController) getComponentLogConfig(ctx context.Context, globalDefaultLogLevel string) (map[string]string, error) {
-	componentLogConfig, err := c.componentNameConfig.RetrieveAll(ctx)
-	if err != nil {
-		return nil, err
-	}
-
-	effectiveDefaultLogLevel := ""
-	for logConfigKey, logConfigValue := range componentLogConfig {
-		if _, err := log.StringToLogLevel(logConfigValue); err != nil || logConfigKey == "" {
-			logger.Warnw(ctx, "unsupported-loglevel-config-defined-at-component-context", log.Fields{"package-name": logConfigKey, "log-level": logConfigValue})
-			delete(componentLogConfig, logConfigKey)
-		} else {
-			if logConfigKey == defaultLogLevelKey {
-				effectiveDefaultLogLevel = componentLogConfig[defaultLogLevelKey]
-			}
-		}
-	}
-
-	// if default loglevel is not configured for the component, component should use
-	// default loglevel configured at global level
-	if effectiveDefaultLogLevel == "" {
-		effectiveDefaultLogLevel = globalDefaultLogLevel
-	}
-
-	componentLogConfig[defaultLogLevelKey] = effectiveDefaultLogLevel
-
-	logger.Debugw(ctx, "retrieved-component-log-config", log.Fields{"component-log-level": componentLogConfig})
-
-	return componentLogConfig, nil
-}
-
-// buildUpdatedLogConfig retrieve the global logConfig and component logConfig  from Backend
-// component logConfig stores the log config with precedence order
-// For example, If the global logConfig is set and component logConfig is set only for specific package then
-// component logConfig is stored with global logConfig  and component logConfig of specific package
-// For example, If the global logConfig is set and component logConfig is set for specific package and as well as for default then
-// component logConfig is stored with  component logConfig data only
-func (c *ComponentLogController) buildUpdatedLogConfig(ctx context.Context) (map[string]string, error) {
-	globalLogLevel, err := c.getGlobalLogConfig(ctx)
-	if err != nil {
-		logger.Errorw(ctx, "unable-to-retrieve-global-log-config", log.Fields{"err": err})
-	}
-
-	componentLogConfig, err := c.getComponentLogConfig(ctx, globalLogLevel)
-	if err != nil {
-		return nil, err
-	}
-
-	finalLogConfig := make(map[string]string)
-	for packageName, logLevel := range componentLogConfig {
-		finalLogConfig[strings.ReplaceAll(packageName, "#", "/")] = logLevel
-	}
-
-	return finalLogConfig, nil
-}
-
-// load and apply the current configuration for component name
-// create hash of loaded configuration using GenerateLogConfigHash
-// if there is previous hash stored, compare the hash to stored hash
-// if there is any change will call UpdateLogLevels
-func (c *ComponentLogController) loadAndApplyLogConfig(ctx context.Context, logConfig map[string]string) error {
-	currentLogHash, err := GenerateLogConfigHash(logConfig)
-	if err != nil {
-		return err
-	}
-
-	if c.logHash != currentLogHash {
-		updateLogLevels(ctx, logConfig)
-		c.logHash = currentLogHash
-	} else {
-		logger.Debug(ctx, "effective-loglevel-config-same-as-currently-active")
-	}
-
-	return nil
-}
-
-// createModifiedLogLevels loops through the activeLogLevels recieved from zap logger and updatedLogLevels recieved from buildUpdatedLogConfig
-// to identify and create map of modified Log Levels of 2 types:
-// - Packages for which log level has been changed
-// - Packages for which log level config has been cleared - set to default log level
-func createModifiedLogLevels(ctx context.Context, activeLogLevels, updatedLogLevels map[string]string) map[string]string {
-	defaultLevel := updatedLogLevels[defaultLogLevelKey]
-
-	modifiedLogLevels := make(map[string]string)
-	for activeKey, activeLevel := range activeLogLevels {
-		if _, exist := updatedLogLevels[activeKey]; !exist {
-			if activeLevel != defaultLevel {
-				modifiedLogLevels[activeKey] = defaultLevel
-			}
-		} else if activeLevel != updatedLogLevels[activeKey] {
-			modifiedLogLevels[activeKey] = updatedLogLevels[activeKey]
-		}
-	}
-
-	// Log warnings for all invalid packages for which log config has been set
-	for key, value := range updatedLogLevels {
-		if _, exist := activeLogLevels[key]; !exist {
-			logger.Warnw(ctx, "ignoring-loglevel-set-for-invalid-package", log.Fields{"package": key, "log-level": value})
-		}
-	}
-
-	return modifiedLogLevels
-}
-
-// updateLogLevels update the loglevels for the component
-// retrieve active confguration from logger
-// compare with entries one by one and apply
-func updateLogLevels(ctx context.Context, updatedLogConfig map[string]string) {
-
-	activeLogLevels := getActiveLogLevels(ctx)
-	changedLogLevels := createModifiedLogLevels(ctx, activeLogLevels, updatedLogConfig)
-
-	// If no changed log levels are found, just return. It may happen on configuration of a invalid package
-	if len(changedLogLevels) == 0 {
-		logger.Debug(ctx, "no-change-in-effective-loglevel-config")
-		return
-	}
-
-	logger.Debugw(ctx, "applying-log-level-for-modified-packages", log.Fields{"changed-log-levels": changedLogLevels})
-	for key, level := range changedLogLevels {
-		if key == defaultLogLevelKey {
-			if l, err := log.StringToLogLevel(level); err == nil {
-				log.SetDefaultLogLevel(l)
-			}
-		} else {
-			if l, err := log.StringToLogLevel(level); err == nil {
-				log.SetPackageLogLevel(key, l)
-			}
-		}
-	}
-}
-
-// generate md5 hash of key value pairs appended into a single string
-// in order by key name
-func GenerateLogConfigHash(createHashLog map[string]string) ([16]byte, error) {
-	createHashLogBytes := []byte{}
-	levelData, err := json.Marshal(createHashLog)
-	if err != nil {
-		return [16]byte{}, err
-	}
-	createHashLogBytes = append(createHashLogBytes, levelData...)
-	return md5.Sum(createHashLogBytes), nil
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logfeaturescontroller.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logfeaturescontroller.go
deleted file mode 100644
index 95c5bde..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/config/logfeaturescontroller.go
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * 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 config
-
-import (
-	"context"
-	"errors"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"os"
-	"strings"
-)
-
-const (
-	defaultTracingStatusKey        = "trace_publish"   // kvstore key containing tracing configuration status
-	defaultLogCorrelationStatusKey = "log_correlation" // kvstore key containing log correlation configuration status
-)
-
-// ComponentLogFeatureController represents Configuration for Logging related features of Tracing and Log
-// Correlation of specific Voltha component.
-type ComponentLogFeaturesController struct {
-	ComponentName               string
-	componentNameConfig         *ComponentConfig
-	configManager               *ConfigManager
-	initialTracingStatus        bool // Initial default tracing status set by helm chart
-	initialLogCorrelationStatus bool // Initial default log correlation status set by helm chart
-}
-
-func NewComponentLogFeaturesController(ctx context.Context, cm *ConfigManager) (*ComponentLogFeaturesController, error) {
-	logger.Debug(ctx, "creating-new-component-log-features-controller")
-	componentName := os.Getenv("COMPONENT_NAME")
-	if componentName == "" {
-		return nil, errors.New("Unable to retrieve PoD Component Name from Runtime env")
-	}
-
-	tracingStatus := log.GetGlobalLFM().GetTracePublishingStatus()
-	logCorrelationStatus := log.GetGlobalLFM().GetLogCorrelationStatus()
-
-	return &ComponentLogFeaturesController{
-		ComponentName:               componentName,
-		componentNameConfig:         nil,
-		configManager:               cm,
-		initialTracingStatus:        tracingStatus,
-		initialLogCorrelationStatus: logCorrelationStatus,
-	}, nil
-
-}
-
-// StartLogFeaturesConfigProcessing persists initial config of Log Features into Config Store before
-// starting the loading and processing of Configuration updates
-func StartLogFeaturesConfigProcessing(cm *ConfigManager, ctx context.Context) {
-	cc, err := NewComponentLogFeaturesController(ctx, cm)
-	if err != nil {
-		logger.Errorw(ctx, "unable-to-construct-component-log-features-controller-instance-for-monitoring", log.Fields{"error": err})
-		return
-	}
-
-	cc.componentNameConfig = cm.InitComponentConfig(cc.ComponentName, ConfigTypeLogFeatures)
-	logger.Debugw(ctx, "component-log-features-config", log.Fields{"cc-component-name-config": cc.componentNameConfig})
-
-	cc.persistInitialLogFeaturesConfigs(ctx)
-
-	cc.processLogFeaturesConfig(ctx)
-}
-
-// Method to persist Initial status of Log Correlation and Tracing features (as set from command line)
-// into config store (etcd kvstore), if not set yet
-func (cc *ComponentLogFeaturesController) persistInitialLogFeaturesConfigs(ctx context.Context) {
-
-	_, err := cc.componentNameConfig.Retrieve(ctx, defaultTracingStatusKey)
-	if err != nil {
-		statusString := "DISABLED"
-		if cc.initialTracingStatus {
-			statusString = "ENABLED"
-		}
-		err = cc.componentNameConfig.Save(ctx, defaultTracingStatusKey, statusString)
-		if err != nil {
-			logger.Errorw(ctx, "failed-to-persist-component-initial-tracing-status-at-startup", log.Fields{"error": err, "tracingstatus": statusString})
-		}
-	}
-
-	_, err = cc.componentNameConfig.Retrieve(ctx, defaultLogCorrelationStatusKey)
-	if err != nil {
-		statusString := "DISABLED"
-		if cc.initialLogCorrelationStatus {
-			statusString = "ENABLED"
-		}
-		err = cc.componentNameConfig.Save(ctx, defaultLogCorrelationStatusKey, statusString)
-		if err != nil {
-			logger.Errorw(ctx, "failed-to-persist-component-initial-log-correlation-status-at-startup", log.Fields{"error": err, "logcorrelationstatus": statusString})
-		}
-	}
-}
-
-// processLogFeaturesConfig will first load and apply configuration of log features. Then it will start waiting for any changes
-// made to configuration in config store (etcd) and apply the same
-func (cc *ComponentLogFeaturesController) processLogFeaturesConfig(ctx context.Context) {
-
-	// Load and apply Tracing Status and log correlation status for first time
-	cc.loadAndApplyTracingStatusUpdate(ctx)
-	cc.loadAndApplyLogCorrelationStatusUpdate(ctx)
-
-	componentConfigEventChan := cc.componentNameConfig.MonitorForConfigChange(ctx)
-
-	// process the change events received on the channel
-	var configEvent *ConfigChangeEvent
-	for {
-		select {
-		case <-ctx.Done():
-			return
-
-		case configEvent = <-componentConfigEventChan:
-			logger.Debugw(ctx, "processing-log-features-config-change", log.Fields{"ChangeType": configEvent.ChangeType, "Package": configEvent.ConfigAttribute})
-
-			if strings.HasSuffix(configEvent.ConfigAttribute, defaultTracingStatusKey) {
-				cc.loadAndApplyTracingStatusUpdate(ctx)
-			} else if strings.HasSuffix(configEvent.ConfigAttribute, defaultLogCorrelationStatusKey) {
-				cc.loadAndApplyLogCorrelationStatusUpdate(ctx)
-			}
-		}
-	}
-
-}
-
-func (cc *ComponentLogFeaturesController) loadAndApplyTracingStatusUpdate(ctx context.Context) {
-
-	desiredTracingStatus, err := cc.componentNameConfig.Retrieve(ctx, defaultTracingStatusKey)
-	if err != nil || desiredTracingStatus == "" {
-		logger.Warn(ctx, "unable-to-retrieve-tracing-status-from-config-store")
-		return
-	}
-
-	if desiredTracingStatus != "ENABLED" && desiredTracingStatus != "DISABLED" {
-		logger.Warnw(ctx, "unsupported-tracing-status-configured-in-config-store", log.Fields{"failed-tracing-status": desiredTracingStatus, "tracing-status": log.GetGlobalLFM().GetTracePublishingStatus()})
-		return
-	}
-
-	logger.Debugw(ctx, "retrieved-tracing-status", log.Fields{"tracing-status": desiredTracingStatus})
-
-	log.GetGlobalLFM().SetTracePublishingStatus(desiredTracingStatus == "ENABLED")
-}
-
-func (cc *ComponentLogFeaturesController) loadAndApplyLogCorrelationStatusUpdate(ctx context.Context) {
-
-	desiredLogCorrelationStatus, err := cc.componentNameConfig.Retrieve(ctx, defaultLogCorrelationStatusKey)
-	if err != nil || desiredLogCorrelationStatus == "" {
-		logger.Warn(ctx, "unable-to-retrieve-log-correlation-status-from-config-store")
-		return
-	}
-
-	if desiredLogCorrelationStatus != "ENABLED" && desiredLogCorrelationStatus != "DISABLED" {
-		logger.Warnw(ctx, "unsupported-log-correlation-status-configured-in-config-store", log.Fields{"failed-log-correlation-status": desiredLogCorrelationStatus, "log-correlation-status": log.GetGlobalLFM().GetLogCorrelationStatus()})
-		return
-	}
-
-	logger.Debugw(ctx, "retrieved-log-correlation-status", log.Fields{"log-correlation-status": desiredLogCorrelationStatus})
-
-	log.GetGlobalLFM().SetLogCorrelationStatus(desiredLogCorrelationStatus == "ENABLED")
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/backend.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/backend.go
deleted file mode 100644
index ff0b5b7..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/backend.go
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * 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 db
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"sync"
-	"time"
-
-	"github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-)
-
-const (
-	// Default Minimal Interval for posting alive state of backend kvstore on Liveness Channel
-	DefaultLivenessChannelInterval = time.Second * 30
-)
-
-// Backend structure holds details for accessing the kv store
-type Backend struct {
-	Client                  kvstore.Client
-	StoreType               string
-	Timeout                 time.Duration
-	Address                 string
-	PathPrefix              string
-	alive                   bool // Is this backend connection alive?
-	livenessMutex           sync.Mutex
-	liveness                chan bool     // channel to post alive state
-	LivenessChannelInterval time.Duration // regularly push alive state beyond this interval
-	lastLivenessTime        time.Time     // Instant of last alive state push
-}
-
-// NewBackend creates a new instance of a Backend structure
-func NewBackend(ctx context.Context, storeType string, address string, timeout time.Duration, pathPrefix string) *Backend {
-	var err error
-
-	b := &Backend{
-		StoreType:               storeType,
-		Address:                 address,
-		Timeout:                 timeout,
-		LivenessChannelInterval: DefaultLivenessChannelInterval,
-		PathPrefix:              pathPrefix,
-		alive:                   false, // connection considered down at start
-	}
-
-	if b.Client, err = b.newClient(ctx, address, timeout); err != nil {
-		logger.Errorw(ctx, "failed-to-create-kv-client",
-			log.Fields{
-				"type": storeType, "address": address,
-				"timeout": timeout, "prefix": pathPrefix,
-				"error": err.Error(),
-			})
-	}
-
-	return b
-}
-
-func (b *Backend) newClient(ctx context.Context, address string, timeout time.Duration) (kvstore.Client, error) {
-	switch b.StoreType {
-	case "etcd":
-		return kvstore.NewEtcdClient(ctx, address, timeout, log.WarnLevel)
-	}
-	return nil, errors.New("unsupported-kv-store")
-}
-
-func (b *Backend) makePath(ctx context.Context, key string) string {
-	path := fmt.Sprintf("%s/%s", b.PathPrefix, key)
-	return path
-}
-
-func (b *Backend) updateLiveness(ctx context.Context, alive bool) {
-	// Periodically push stream of liveness data to the channel,
-	// so that in a live state, the core does not timeout and
-	// send a forced liveness message. Push alive state if the
-	// last push to channel was beyond livenessChannelInterval
-	b.livenessMutex.Lock()
-	defer b.livenessMutex.Unlock()
-	if b.liveness != nil {
-		if b.alive != alive {
-			logger.Debug(ctx, "update-liveness-channel-reason-change")
-			b.liveness <- alive
-			b.lastLivenessTime = time.Now()
-		} else if time.Since(b.lastLivenessTime) > b.LivenessChannelInterval {
-			logger.Debug(ctx, "update-liveness-channel-reason-interval")
-			b.liveness <- alive
-			b.lastLivenessTime = time.Now()
-		}
-	}
-
-	// Emit log message only for alive state change
-	if b.alive != alive {
-		logger.Debugw(ctx, "change-kvstore-alive-status", log.Fields{"alive": alive})
-		b.alive = alive
-	}
-}
-
-// Perform a dummy Key Lookup on kvstore to test Connection Liveness and
-// post on Liveness channel
-func (b *Backend) PerformLivenessCheck(ctx context.Context) bool {
-	alive := b.Client.IsConnectionUp(ctx)
-	logger.Debugw(ctx, "kvstore-liveness-check-result", log.Fields{"alive": alive})
-
-	b.updateLiveness(ctx, alive)
-	return alive
-}
-
-// Enable the liveness monitor channel. This channel will report
-// a "true" or "false" on every kvstore operation which indicates whether
-// or not the connection is still Live. This channel is then picked up
-// by the service (i.e. rw_core / ro_core) to update readiness status
-// and/or take other actions.
-func (b *Backend) EnableLivenessChannel(ctx context.Context) chan bool {
-	logger.Debug(ctx, "enable-kvstore-liveness-channel")
-	b.livenessMutex.Lock()
-	defer b.livenessMutex.Unlock()
-	if b.liveness == nil {
-		b.liveness = make(chan bool, 10)
-		b.liveness <- b.alive
-		b.lastLivenessTime = time.Now()
-	}
-
-	return b.liveness
-}
-
-// Extract Alive status of Kvstore based on type of error
-func (b *Backend) isErrorIndicatingAliveKvstore(ctx context.Context, err error) bool {
-	// Alive unless observed an error indicating so
-	alive := true
-
-	if err != nil {
-
-		// timeout indicates kvstore not reachable/alive
-		if err == context.DeadlineExceeded {
-			alive = false
-		}
-
-		// Need to analyze client-specific errors based on backend type
-		if b.StoreType == "etcd" {
-
-			// For etcd backend, consider not-alive only for errors indicating
-			// timedout request or unavailable/corrupted cluster. For all remaining
-			// error codes listed in https://godoc.org/google.golang.org/grpc/codes#Code,
-			// we would not infer a not-alive backend because such a error may also
-			// occur due to bad client requests or sequence of operations
-			switch status.Code(err) {
-			case codes.DeadlineExceeded:
-				fallthrough
-			case codes.Unavailable:
-				fallthrough
-			case codes.DataLoss:
-				alive = false
-			}
-		}
-	}
-
-	return alive
-}
-
-// List retrieves one or more items that match the specified key
-func (b *Backend) List(ctx context.Context, key string) (map[string]*kvstore.KVPair, error) {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-list")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "listing-key", log.Fields{"key": key, "path": formattedPath})
-
-	pair, err := b.Client.List(ctx, formattedPath)
-
-	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
-
-	return pair, err
-}
-
-// Get retrieves an item that matches the specified key
-func (b *Backend) Get(ctx context.Context, key string) (*kvstore.KVPair, error) {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-get")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "getting-key", log.Fields{"key": key, "path": formattedPath})
-
-	pair, err := b.Client.Get(ctx, formattedPath)
-
-	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
-
-	return pair, err
-}
-
-// Put stores an item value under the specifed key
-func (b *Backend) Put(ctx context.Context, key string, value interface{}) error {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-put")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "putting-key", log.Fields{"key": key, "path": formattedPath})
-
-	err := b.Client.Put(ctx, formattedPath, value)
-
-	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
-
-	return err
-}
-
-// Delete removes an item under the specified key
-func (b *Backend) Delete(ctx context.Context, key string) error {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-delete")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "deleting-key", log.Fields{"key": key, "path": formattedPath})
-
-	err := b.Client.Delete(ctx, formattedPath)
-
-	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
-
-	return err
-}
-
-// DeleteWithPrefix removes items having prefix key
-func (b *Backend) DeleteWithPrefix(ctx context.Context, prefixKey string) error {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-delete-with-prefix")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, prefixKey)
-	logger.Debugw(ctx, "deleting-prefix-key", log.Fields{"key": prefixKey, "path": formattedPath})
-
-	err := b.Client.DeleteWithPrefix(ctx, formattedPath)
-
-	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
-
-	return err
-}
-
-// CreateWatch starts watching events for the specified key
-func (b *Backend) CreateWatch(ctx context.Context, key string, withPrefix bool) chan *kvstore.Event {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-create-watch")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "creating-key-watch", log.Fields{"key": key, "path": formattedPath})
-
-	return b.Client.Watch(ctx, formattedPath, withPrefix)
-}
-
-// DeleteWatch stops watching events for the specified key
-func (b *Backend) DeleteWatch(ctx context.Context, key string, ch chan *kvstore.Event) {
-	span, ctx := log.CreateChildSpan(ctx, "etcd-delete-watch")
-	defer span.Finish()
-
-	formattedPath := b.makePath(ctx, key)
-	logger.Debugw(ctx, "deleting-key-watch", log.Fields{"key": key, "path": formattedPath})
-
-	b.Client.CloseWatch(ctx, formattedPath, ch)
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/common.go
deleted file mode 100644
index 4bc92b1..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 db
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/common.go
deleted file mode 100644
index b8509db..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 kvstore
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go
deleted file mode 100644
index 96ffc2f..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdclient.go
+++ /dev/null
@@ -1,473 +0,0 @@
-/*
- * 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 kvstore
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	v3Client "go.etcd.io/etcd/clientv3"
-	v3rpcTypes "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	"os"
-	"strconv"
-	"sync"
-	"time"
-)
-
-const (
-	poolCapacityEnvName = "VOLTHA_ETCD_CLIENT_POOL_CAPACITY"
-	maxUsageEnvName     = "VOLTHA_ETCD_CLIENT_MAX_USAGE"
-)
-
-const (
-	defaultMaxPoolCapacity = 1000 // Default size of an Etcd Client pool
-	defaultMaxPoolUsage    = 100  // Maximum concurrent request an Etcd Client is allowed to process
-)
-
-// EtcdClient represents the Etcd KV store client
-type EtcdClient struct {
-	pool               EtcdClientAllocator
-	watchedChannels    sync.Map
-	watchedClients     map[string]*v3Client.Client
-	watchedClientsLock sync.RWMutex
-}
-
-// NewEtcdCustomClient returns a new client for the Etcd KV store allowing
-// the called to specify etcd client configuration
-func NewEtcdCustomClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
-	// Get the capacity and max usage from the environment
-	capacity := defaultMaxPoolCapacity
-	maxUsage := defaultMaxPoolUsage
-	if capacityStr, present := os.LookupEnv(poolCapacityEnvName); present {
-		if val, err := strconv.Atoi(capacityStr); err == nil {
-			capacity = val
-			logger.Infow(ctx, "env-variable-set", log.Fields{"pool-capacity": capacity})
-		} else {
-			logger.Warnw(ctx, "invalid-capacity-value", log.Fields{"error": err, "capacity": capacityStr})
-		}
-	}
-	if maxUsageStr, present := os.LookupEnv(maxUsageEnvName); present {
-		if val, err := strconv.Atoi(maxUsageStr); err == nil {
-			maxUsage = val
-			logger.Infow(ctx, "env-variable-set", log.Fields{"max-usage": maxUsage})
-		} else {
-			logger.Warnw(ctx, "invalid-max-usage-value", log.Fields{"error": err, "max-usage": maxUsageStr})
-		}
-	}
-
-	var err error
-
-	pool, err := NewRoundRobinEtcdClientAllocator([]string{addr}, timeout, capacity, maxUsage, level)
-	if err != nil {
-		logger.Errorw(ctx, "failed-to-create-rr-client", log.Fields{
-			"error": err,
-		})
-	}
-
-	logger.Infow(ctx, "etcd-pool-created", log.Fields{"capacity": capacity, "max-usage": maxUsage})
-
-	return &EtcdClient{pool: pool,
-		watchedClients: make(map[string]*v3Client.Client),
-	}, nil
-}
-
-// NewEtcdClient returns a new client for the Etcd KV store
-func NewEtcdClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
-	return NewEtcdCustomClient(ctx, addr, timeout, level)
-}
-
-// IsConnectionUp returns whether the connection to the Etcd KV store is up.  If a timeout occurs then
-// it is assumed the connection is down or unreachable.
-func (c *EtcdClient) IsConnectionUp(ctx context.Context) bool {
-	// Let's try to get a non existent key.  If the connection is up then there will be no error returned.
-	if _, err := c.Get(ctx, "non-existent-key"); err != nil {
-		return false
-	}
-	return true
-}
-
-// List returns an array of key-value pairs with key as a prefix.  Timeout defines how long the function will
-// wait for a response
-func (c *EtcdClient) List(ctx context.Context, key string) (map[string]*KVPair, error) {
-	client, err := c.pool.Get(ctx)
-	if err != nil {
-		return nil, err
-	}
-	defer c.pool.Put(client)
-	resp, err := client.Get(ctx, key, v3Client.WithPrefix())
-
-	if err != nil {
-		logger.Error(ctx, err)
-		return nil, err
-	}
-	m := make(map[string]*KVPair)
-	for _, ev := range resp.Kvs {
-		m[string(ev.Key)] = NewKVPair(string(ev.Key), ev.Value, "", ev.Lease, ev.Version)
-	}
-	return m, nil
-}
-
-// Get returns a key-value pair for a given key. Timeout defines how long the function will
-// wait for a response
-func (c *EtcdClient) Get(ctx context.Context, key string) (*KVPair, error) {
-	client, err := c.pool.Get(ctx)
-	if err != nil {
-		return nil, err
-	}
-	defer c.pool.Put(client)
-
-	attempt := 0
-
-startLoop:
-	for {
-		resp, err := client.Get(ctx, key)
-		if err != nil {
-			switch err {
-			case context.Canceled:
-				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
-			case context.DeadlineExceeded:
-				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
-			case v3rpcTypes.ErrEmptyKey:
-				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
-			case v3rpcTypes.ErrLeaderChanged,
-				v3rpcTypes.ErrGRPCNoLeader,
-				v3rpcTypes.ErrTimeout,
-				v3rpcTypes.ErrTimeoutDueToLeaderFail,
-				v3rpcTypes.ErrTimeoutDueToConnectionLost:
-				// Retry for these server errors
-				attempt += 1
-				if er := backoff(ctx, attempt); er != nil {
-					logger.Warnw(ctx, "get-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
-					return nil, err
-				}
-				logger.Warnw(ctx, "retrying-get", log.Fields{"key": key, "error": err, "attempt": attempt})
-				goto startLoop
-			default:
-				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
-			}
-			return nil, err
-		}
-
-		for _, ev := range resp.Kvs {
-			// Only one value is returned
-			return NewKVPair(string(ev.Key), ev.Value, "", ev.Lease, ev.Version), nil
-		}
-		return nil, nil
-	}
-}
-
-// Put writes a key-value pair to the KV store.  Value can only be a string or []byte since the etcd API
-// accepts only a string as a value for a put operation. Timeout defines how long the function will
-// wait for a response
-func (c *EtcdClient) Put(ctx context.Context, key string, value interface{}) error {
-
-	// Validate that we can convert value to a string as etcd API expects a string
-	var val string
-	var err error
-	if val, err = ToString(value); err != nil {
-		return fmt.Errorf("unexpected-type-%T", value)
-	}
-
-	client, err := c.pool.Get(ctx)
-	if err != nil {
-		return err
-	}
-	defer c.pool.Put(client)
-
-	attempt := 0
-startLoop:
-	for {
-		_, err = client.Put(ctx, key, val)
-		if err != nil {
-			switch err {
-			case context.Canceled:
-				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
-			case context.DeadlineExceeded:
-				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
-			case v3rpcTypes.ErrEmptyKey:
-				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
-			case v3rpcTypes.ErrLeaderChanged,
-				v3rpcTypes.ErrGRPCNoLeader,
-				v3rpcTypes.ErrTimeout,
-				v3rpcTypes.ErrTimeoutDueToLeaderFail,
-				v3rpcTypes.ErrTimeoutDueToConnectionLost:
-				// Retry for these server errors
-				attempt += 1
-				if er := backoff(ctx, attempt); er != nil {
-					logger.Warnw(ctx, "put-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
-					return err
-				}
-				logger.Warnw(ctx, "retrying-put", log.Fields{"key": key, "error": err, "attempt": attempt})
-				goto startLoop
-			default:
-				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
-			}
-			return err
-		}
-		return nil
-	}
-}
-
-// Delete removes a key from the KV store. Timeout defines how long the function will
-// wait for a response
-func (c *EtcdClient) Delete(ctx context.Context, key string) error {
-	client, err := c.pool.Get(ctx)
-	if err != nil {
-		return err
-	}
-	defer c.pool.Put(client)
-
-	attempt := 0
-startLoop:
-	for {
-		_, err = client.Delete(ctx, key)
-		if err != nil {
-			switch err {
-			case context.Canceled:
-				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
-			case context.DeadlineExceeded:
-				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
-			case v3rpcTypes.ErrEmptyKey:
-				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
-			case v3rpcTypes.ErrLeaderChanged,
-				v3rpcTypes.ErrGRPCNoLeader,
-				v3rpcTypes.ErrTimeout,
-				v3rpcTypes.ErrTimeoutDueToLeaderFail,
-				v3rpcTypes.ErrTimeoutDueToConnectionLost:
-				// Retry for these server errors
-				attempt += 1
-				if er := backoff(ctx, attempt); er != nil {
-					logger.Warnw(ctx, "delete-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
-					return err
-				}
-				logger.Warnw(ctx, "retrying-delete", log.Fields{"key": key, "error": err, "attempt": attempt})
-				goto startLoop
-			default:
-				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
-			}
-			return err
-		}
-		logger.Debugw(ctx, "key(s)-deleted", log.Fields{"key": key})
-		return nil
-	}
-}
-
-func (c *EtcdClient) DeleteWithPrefix(ctx context.Context, prefixKey string) error {
-
-	client, err := c.pool.Get(ctx)
-	if err != nil {
-		return err
-	}
-	defer c.pool.Put(client)
-
-	//delete the prefix
-	if _, err := client.Delete(ctx, prefixKey, v3Client.WithPrefix()); err != nil {
-		logger.Errorw(ctx, "failed-to-delete-prefix-key", log.Fields{"key": prefixKey, "error": err})
-		return err
-	}
-	logger.Debugw(ctx, "key(s)-deleted", log.Fields{"key": prefixKey})
-	return nil
-}
-
-// Watch provides the watch capability on a given key.  It returns a channel onto which the callee needs to
-// listen to receive Events.
-func (c *EtcdClient) Watch(ctx context.Context, key string, withPrefix bool) chan *Event {
-	var err error
-	// Reuse the Etcd client when multiple callees are watching the same key.
-	c.watchedClientsLock.Lock()
-	client, exist := c.watchedClients[key]
-	if !exist {
-		client, err = c.pool.Get(ctx)
-		if err != nil {
-			logger.Errorw(ctx, "failed-to-an-etcd-client", log.Fields{"key": key, "error": err})
-			c.watchedClientsLock.Unlock()
-			return nil
-		}
-		c.watchedClients[key] = client
-	}
-	c.watchedClientsLock.Unlock()
-
-	w := v3Client.NewWatcher(client)
-	ctx, cancel := context.WithCancel(ctx)
-	var channel v3Client.WatchChan
-	if withPrefix {
-		channel = w.Watch(ctx, key, v3Client.WithPrefix())
-	} else {
-		channel = w.Watch(ctx, key)
-	}
-
-	// Create a new channel
-	ch := make(chan *Event, maxClientChannelBufferSize)
-
-	// Keep track of the created channels so they can be closed when required
-	channelMap := make(map[chan *Event]v3Client.Watcher)
-	channelMap[ch] = w
-	channelMaps := c.addChannelMap(key, channelMap)
-
-	// Changing the log field (from channelMaps) as the underlying logger cannot format the map of channels into a
-	// json format.
-	logger.Debugw(ctx, "watched-channels", log.Fields{"len": len(channelMaps)})
-	// Launch a go routine to listen for updates
-	go c.listenForKeyChange(ctx, channel, ch, cancel)
-
-	return ch
-
-}
-
-func (c *EtcdClient) addChannelMap(key string, channelMap map[chan *Event]v3Client.Watcher) []map[chan *Event]v3Client.Watcher {
-	var channels interface{}
-	var exists bool
-
-	if channels, exists = c.watchedChannels.Load(key); exists {
-		channels = append(channels.([]map[chan *Event]v3Client.Watcher), channelMap)
-	} else {
-		channels = []map[chan *Event]v3Client.Watcher{channelMap}
-	}
-	c.watchedChannels.Store(key, channels)
-
-	return channels.([]map[chan *Event]v3Client.Watcher)
-}
-
-func (c *EtcdClient) removeChannelMap(key string, pos int) []map[chan *Event]v3Client.Watcher {
-	var channels interface{}
-	var exists bool
-
-	if channels, exists = c.watchedChannels.Load(key); exists {
-		channels = append(channels.([]map[chan *Event]v3Client.Watcher)[:pos], channels.([]map[chan *Event]v3Client.Watcher)[pos+1:]...)
-		c.watchedChannels.Store(key, channels)
-	}
-
-	return channels.([]map[chan *Event]v3Client.Watcher)
-}
-
-func (c *EtcdClient) getChannelMaps(key string) ([]map[chan *Event]v3Client.Watcher, bool) {
-	var channels interface{}
-	var exists bool
-
-	channels, exists = c.watchedChannels.Load(key)
-
-	if channels == nil {
-		return nil, exists
-	}
-
-	return channels.([]map[chan *Event]v3Client.Watcher), exists
-}
-
-// CloseWatch closes a specific watch. Both the key and the channel are required when closing a watch as there
-// may be multiple listeners on the same key.  The previously created channel serves as a key
-func (c *EtcdClient) CloseWatch(ctx context.Context, key string, ch chan *Event) {
-	// Get the array of channels mapping
-	var watchedChannels []map[chan *Event]v3Client.Watcher
-	var ok bool
-
-	if watchedChannels, ok = c.getChannelMaps(key); !ok {
-		logger.Warnw(ctx, "key-has-no-watched-channels", log.Fields{"key": key})
-		return
-	}
-	// Look for the channels
-	var pos = -1
-	for i, chMap := range watchedChannels {
-		if t, ok := chMap[ch]; ok {
-			logger.Debug(ctx, "channel-found")
-			// Close the etcd watcher before the client channel.  This should close the etcd channel as well
-			if err := t.Close(); err != nil {
-				logger.Errorw(ctx, "watcher-cannot-be-closed", log.Fields{"key": key, "error": err})
-			}
-			pos = i
-			break
-		}
-	}
-
-	channelMaps, _ := c.getChannelMaps(key)
-	// Remove that entry if present
-	if pos >= 0 {
-		channelMaps = c.removeChannelMap(key, pos)
-	}
-
-	// If we don't have any keys being watched then return the Etcd client to the pool
-	if len(channelMaps) == 0 {
-		c.watchedClientsLock.Lock()
-		// Sanity
-		if client, ok := c.watchedClients[key]; ok {
-			c.pool.Put(client)
-			delete(c.watchedClients, key)
-		}
-		c.watchedClientsLock.Unlock()
-	}
-	logger.Infow(ctx, "watcher-channel-exiting", log.Fields{"key": key, "channel": channelMaps})
-}
-
-func (c *EtcdClient) listenForKeyChange(ctx context.Context, channel v3Client.WatchChan, ch chan<- *Event, cancel context.CancelFunc) {
-	logger.Debug(ctx, "start-listening-on-channel ...")
-	defer cancel()
-	defer close(ch)
-	for resp := range channel {
-		for _, ev := range resp.Events {
-			ch <- NewEvent(getEventType(ev), ev.Kv.Key, ev.Kv.Value, ev.Kv.Version)
-		}
-	}
-	logger.Debug(ctx, "stop-listening-on-channel ...")
-}
-
-func getEventType(event *v3Client.Event) int {
-	switch event.Type {
-	case v3Client.EventTypePut:
-		return PUT
-	case v3Client.EventTypeDelete:
-		return DELETE
-	}
-	return UNKNOWN
-}
-
-// Close closes all the connection in the pool store client
-func (c *EtcdClient) Close(ctx context.Context) {
-	logger.Debug(ctx, "closing-etcd-pool")
-	c.pool.Close(ctx)
-}
-
-// The APIs below are not used
-var errUnimplemented = errors.New("deprecated")
-
-// Reserve is deprecated
-func (c *EtcdClient) Reserve(ctx context.Context, key string, value interface{}, ttl time.Duration) (interface{}, error) {
-	return nil, errUnimplemented
-}
-
-// ReleaseAllReservations is deprecated
-func (c *EtcdClient) ReleaseAllReservations(ctx context.Context) error {
-	return errUnimplemented
-}
-
-// ReleaseReservation is deprecated
-func (c *EtcdClient) ReleaseReservation(ctx context.Context, key string) error {
-	return errUnimplemented
-}
-
-// RenewReservation is deprecated
-func (c *EtcdClient) RenewReservation(ctx context.Context, key string) error {
-	return errUnimplemented
-}
-
-// AcquireLock is deprecated
-func (c *EtcdClient) AcquireLock(ctx context.Context, lockName string, timeout time.Duration) error {
-	return errUnimplemented
-}
-
-// ReleaseLock is deprecated
-func (c *EtcdClient) ReleaseLock(lockName string) error {
-	return errUnimplemented
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go
deleted file mode 100644
index 6af7d3d..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/etcdpool.go
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * Copyright 2021-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 kvstore
-
-import (
-	"container/list"
-	"context"
-	"errors"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"go.etcd.io/etcd/clientv3"
-	"sync"
-	"time"
-)
-
-// EtcdClientAllocator represents a generic interface to allocate an Etcd Client
-type EtcdClientAllocator interface {
-	Get(context.Context) (*clientv3.Client, error)
-	Put(*clientv3.Client)
-	Close(ctx context.Context)
-}
-
-// NewRoundRobinEtcdClientAllocator creates a new ETCD Client Allocator using a Round Robin scheme
-func NewRoundRobinEtcdClientAllocator(endpoints []string, timeout time.Duration, capacity, maxUsage int, level log.LogLevel) (EtcdClientAllocator, error) {
-	return &roundRobin{
-		all:       make(map[*clientv3.Client]*rrEntry),
-		full:      make(map[*clientv3.Client]*rrEntry),
-		waitList:  list.New(),
-		max:       maxUsage,
-		capacity:  capacity,
-		timeout:   timeout,
-		endpoints: endpoints,
-		logLevel:  level,
-		closingCh: make(chan struct{}, capacity*maxUsage),
-		stopCh:    make(chan struct{}),
-	}, nil
-}
-
-type rrEntry struct {
-	client *clientv3.Client
-	count  int
-	age    time.Time
-}
-
-type roundRobin struct {
-	//block chan struct{}
-	sync.Mutex
-	available []*rrEntry
-	all       map[*clientv3.Client]*rrEntry
-	full      map[*clientv3.Client]*rrEntry
-	waitList  *list.List
-	max       int
-	capacity  int
-	timeout   time.Duration
-	//ageOut    time.Duration
-	endpoints []string
-	size      int
-	logLevel  log.LogLevel
-	closing   bool
-	closingCh chan struct{}
-	stopCh    chan struct{}
-}
-
-// Get returns an Etcd client. If not is available, it will create one
-// until the maximum allowed capacity.  If maximum capacity has been
-// reached then it will wait until s used one is freed.
-func (r *roundRobin) Get(ctx context.Context) (*clientv3.Client, error) {
-	r.Lock()
-
-	if r.closing {
-		r.Unlock()
-		return nil, errors.New("pool-is-closing")
-	}
-
-	// first determine if we need to block, which would mean the
-	// available queue is empty and we are at capacity
-	if len(r.available) == 0 && r.size >= r.capacity {
-
-		// create a channel on which to wait and
-		// add it to the list
-		ch := make(chan struct{})
-		element := r.waitList.PushBack(ch)
-		r.Unlock()
-
-		// block until it is our turn or context
-		// expires or is canceled
-		select {
-		case <-r.stopCh:
-			logger.Info(ctx, "stop-waiting-pool-is-closing")
-			r.waitList.Remove(element)
-			return nil, errors.New("stop-waiting-pool-is-closing")
-		case <-ch:
-			r.waitList.Remove(element)
-		case <-ctx.Done():
-			r.waitList.Remove(element)
-			return nil, ctx.Err()
-		}
-		r.Lock()
-	}
-
-	defer r.Unlock()
-	if len(r.available) > 0 {
-		// pull off back end as it is operationally quicker
-		last := len(r.available) - 1
-		entry := r.available[last]
-		entry.count++
-		if entry.count >= r.max {
-			r.available = r.available[:last]
-			r.full[entry.client] = entry
-		}
-		entry.age = time.Now()
-		return entry.client, nil
-	}
-
-	logConfig := log.ConstructZapConfig(log.JSON, r.logLevel, log.Fields{})
-	// increase capacity
-	client, err := clientv3.New(clientv3.Config{
-		Endpoints:   r.endpoints,
-		DialTimeout: r.timeout,
-		LogConfig:   &logConfig,
-	})
-	if err != nil {
-		return nil, err
-	}
-	entry := &rrEntry{
-		client: client,
-		count:  1,
-	}
-	r.all[entry.client] = entry
-
-	if r.max > 1 {
-		r.available = append(r.available, entry)
-	} else {
-		r.full[entry.client] = entry
-	}
-	r.size++
-	return client, nil
-}
-
-// Put returns the Etcd Client back to the pool
-func (r *roundRobin) Put(client *clientv3.Client) {
-	r.Lock()
-
-	entry := r.all[client]
-	entry.count--
-
-	if r.closing {
-		// Close client if count is 0
-		if entry.count == 0 {
-			if err := entry.client.Close(); err != nil {
-				logger.Warnw(context.Background(), "error-closing-client", log.Fields{"error": err})
-			}
-			delete(r.all, entry.client)
-		}
-		// Notify Close function that a client was returned to the pool
-		r.closingCh <- struct{}{}
-		r.Unlock()
-		return
-	}
-
-	// This entry is now available for use, so
-	// if in full map add it to available and
-	// remove from full
-	if _, ok := r.full[client]; ok {
-		r.available = append(r.available, entry)
-		delete(r.full, client)
-	}
-
-	front := r.waitList.Front()
-	if front != nil {
-		ch := r.waitList.Remove(front)
-		r.Unlock()
-		// need to unblock if someone is waiting
-		ch.(chan struct{}) <- struct{}{}
-		return
-	}
-	r.Unlock()
-}
-
-func (r *roundRobin) Close(ctx context.Context) {
-	r.Lock()
-	r.closing = true
-
-	// Notify anyone waiting for a client to stop waiting
-	close(r.stopCh)
-
-	// Clean-up unused clients
-	for i := 0; i < len(r.available); i++ {
-		// Count 0 means no one is using that client
-		if r.available[i].count == 0 {
-			if err := r.available[i].client.Close(); err != nil {
-				logger.Warnw(ctx, "failure-closing-client", log.Fields{"client": r.available[i].client, "error": err})
-			}
-			// Remove client for all list
-			delete(r.all, r.available[i].client)
-		}
-	}
-
-	// Figure out how many clients are in use
-	numberInUse := 0
-	for _, rrEntry := range r.all {
-		numberInUse += rrEntry.count
-	}
-	r.Unlock()
-
-	if numberInUse == 0 {
-		logger.Info(ctx, "no-connection-in-use")
-		return
-	}
-
-	logger.Infow(ctx, "waiting-for-clients-return", log.Fields{"count": numberInUse})
-
-	// Wait for notifications when a client is returned to the pool
-	for {
-		select {
-		case <-r.closingCh:
-			numberInUse--
-			if numberInUse == 0 {
-				logger.Info(ctx, "all-connections-closed")
-				return
-			}
-		case <-ctx.Done():
-			logger.Warnw(ctx, "context-done", log.Fields{"error": ctx.Err()})
-			return
-		}
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/common.go
deleted file mode 100644
index df3e839..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 events
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/eventif/events_proxy_if.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/eventif/events_proxy_if.go
deleted file mode 100644
index 35f821f..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/eventif/events_proxy_if.go
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2020-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 eventif
-
-import (
-	"context"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-// EventProxy interface for eventproxy
-type EventProxy interface {
-	SendDeviceEvent(ctx context.Context, deviceEvent *voltha.DeviceEvent, category EventCategory,
-		subCategory EventSubCategory, raisedTs int64) error
-	SendKpiEvent(ctx context.Context, id string, deviceEvent *voltha.KpiEvent2, category EventCategory,
-		subCategory EventSubCategory, raisedTs int64) error
-	SendRPCEvent(ctx context.Context, id string, deviceEvent *voltha.RPCEvent, category EventCategory,
-		subCategory *EventSubCategory, raisedTs int64) error
-	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
-	SendLiveness(ctx context.Context) error
-}
-
-const (
-	EventTypeVersion = "0.1"
-)
-
-type (
-	EventType        = voltha.EventType_Types
-	EventCategory    = voltha.EventCategory_Types
-	EventSubCategory = voltha.EventSubCategory_Types
-)
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/events_proxy.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/events_proxy.go
deleted file mode 100644
index 19a4f26..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/events_proxy.go
+++ /dev/null
@@ -1,353 +0,0 @@
-/*
- * Copyright 2020-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 events
-
-import (
-	"container/ring"
-	"context"
-	"errors"
-	"fmt"
-	"strconv"
-	"strings"
-	"sync"
-	"time"
-
-	"github.com/golang/protobuf/ptypes"
-	"github.com/opencord/voltha-lib-go/v5/pkg/events/eventif"
-	"github.com/opencord/voltha-lib-go/v5/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-// TODO: Make configurable through helm chart
-const EVENT_THRESHOLD = 1000
-
-type lastEvent struct{}
-
-type EventProxy struct {
-	kafkaClient    kafka.Client
-	eventTopic     kafka.Topic
-	eventQueue     *EventQueue
-	queueCtx       context.Context
-	queueCancelCtx context.CancelFunc
-}
-
-func NewEventProxy(opts ...EventProxyOption) *EventProxy {
-	var proxy EventProxy
-	for _, option := range opts {
-		option(&proxy)
-	}
-	proxy.eventQueue = newEventQueue()
-	proxy.queueCtx, proxy.queueCancelCtx = context.WithCancel(context.Background())
-	return &proxy
-}
-
-type EventProxyOption func(*EventProxy)
-
-func MsgClient(client kafka.Client) EventProxyOption {
-	return func(args *EventProxy) {
-		args.kafkaClient = client
-	}
-}
-
-func MsgTopic(topic kafka.Topic) EventProxyOption {
-	return func(args *EventProxy) {
-		args.eventTopic = topic
-	}
-}
-
-func (ep *EventProxy) formatId(eventName string) string {
-	return fmt.Sprintf("Voltha.openolt.%s.%s", eventName, strconv.FormatInt(time.Now().UnixNano(), 10))
-}
-
-func (ep *EventProxy) getEventHeader(eventName string,
-	category eventif.EventCategory,
-	subCategory *eventif.EventSubCategory,
-	eventType eventif.EventType,
-	raisedTs int64) (*voltha.EventHeader, error) {
-	var header voltha.EventHeader
-	if strings.Contains(eventName, "_") {
-		eventName = strings.Join(strings.Split(eventName, "_")[:len(strings.Split(eventName, "_"))-2], "_")
-	} else {
-		eventName = "UNKNOWN_EVENT"
-	}
-	/* Populating event header */
-	header.Id = ep.formatId(eventName)
-	header.Category = category
-	if subCategory != nil {
-		header.SubCategory = *subCategory
-	} else {
-		header.SubCategory = voltha.EventSubCategory_NONE
-	}
-	header.Type = eventType
-	header.TypeVersion = eventif.EventTypeVersion
-
-	// raisedTs is in seconds
-	timestamp, err := ptypes.TimestampProto(time.Unix(raisedTs, 0))
-	if err != nil {
-		return nil, err
-	}
-	header.RaisedTs = timestamp
-
-	timestamp, err = ptypes.TimestampProto(time.Now())
-	if err != nil {
-		return nil, err
-	}
-	header.ReportedTs = timestamp
-
-	return &header, nil
-}
-
-/* Send out rpc events*/
-func (ep *EventProxy) SendRPCEvent(ctx context.Context, id string, rpcEvent *voltha.RPCEvent, category eventif.EventCategory, subCategory *eventif.EventSubCategory, raisedTs int64) error {
-	if rpcEvent == nil {
-		logger.Error(ctx, "Received empty rpc event")
-		return errors.New("rpc event nil")
-	}
-	var event voltha.Event
-	var err error
-	if event.Header, err = ep.getEventHeader(id, category, subCategory, voltha.EventType_RPC_EVENT, raisedTs); err != nil {
-		return err
-	}
-	event.EventType = &voltha.Event_RpcEvent{RpcEvent: rpcEvent}
-	ep.eventQueue.push(&event)
-	return nil
-
-}
-
-/* Send out device events*/
-func (ep *EventProxy) SendDeviceEvent(ctx context.Context, deviceEvent *voltha.DeviceEvent, category eventif.EventCategory, subCategory eventif.EventSubCategory, raisedTs int64) error {
-	if deviceEvent == nil {
-		logger.Error(ctx, "Recieved empty device event")
-		return errors.New("Device event nil")
-	}
-	var event voltha.Event
-	var de voltha.Event_DeviceEvent
-	var err error
-	de.DeviceEvent = deviceEvent
-	if event.Header, err = ep.getEventHeader(deviceEvent.DeviceEventName, category, &subCategory, voltha.EventType_DEVICE_EVENT, raisedTs); err != nil {
-		return err
-	}
-	event.EventType = &de
-	if err := ep.sendEvent(ctx, &event); err != nil {
-		logger.Errorw(ctx, "Failed to send device event to KAFKA bus", log.Fields{"device-event": deviceEvent})
-		return err
-	}
-	logger.Infow(ctx, "Successfully sent device event KAFKA", log.Fields{"Id": event.Header.Id, "Category": event.Header.Category,
-		"SubCategory": event.Header.SubCategory, "Type": event.Header.Type, "TypeVersion": event.Header.TypeVersion,
-		"ReportedTs": event.Header.ReportedTs, "ResourceId": deviceEvent.ResourceId, "Context": deviceEvent.Context,
-		"DeviceEventName": deviceEvent.DeviceEventName})
-
-	return nil
-
-}
-
-// SendKpiEvent is to send kpi events to voltha.event topic
-func (ep *EventProxy) SendKpiEvent(ctx context.Context, id string, kpiEvent *voltha.KpiEvent2, category eventif.EventCategory, subCategory eventif.EventSubCategory, raisedTs int64) error {
-	if kpiEvent == nil {
-		logger.Error(ctx, "Recieved empty kpi event")
-		return errors.New("KPI event nil")
-	}
-	var event voltha.Event
-	var de voltha.Event_KpiEvent2
-	var err error
-	de.KpiEvent2 = kpiEvent
-	if event.Header, err = ep.getEventHeader(id, category, &subCategory, voltha.EventType_KPI_EVENT2, raisedTs); err != nil {
-		return err
-	}
-	event.EventType = &de
-	if err := ep.sendEvent(ctx, &event); err != nil {
-		logger.Errorw(ctx, "Failed to send kpi event to KAFKA bus", log.Fields{"device-event": kpiEvent})
-		return err
-	}
-	logger.Infow(ctx, "Successfully sent kpi event to KAFKA", log.Fields{"Id": event.Header.Id, "Category": event.Header.Category,
-		"SubCategory": event.Header.SubCategory, "Type": event.Header.Type, "TypeVersion": event.Header.TypeVersion,
-		"ReportedTs": event.Header.ReportedTs, "KpiEventName": "STATS_EVENT"})
-
-	return nil
-
-}
-
-func (ep *EventProxy) sendEvent(ctx context.Context, event *voltha.Event) error {
-	logger.Debugw(ctx, "Send event to kafka", log.Fields{"event": event})
-	if err := ep.kafkaClient.Send(ctx, event, &ep.eventTopic); err != nil {
-		return err
-	}
-	logger.Debugw(ctx, "Sent event to kafka", log.Fields{"event": event})
-
-	return nil
-}
-
-func (ep *EventProxy) EnableLivenessChannel(ctx context.Context, enable bool) chan bool {
-	return ep.kafkaClient.EnableLivenessChannel(ctx, enable)
-}
-
-func (ep *EventProxy) SendLiveness(ctx context.Context) error {
-	return ep.kafkaClient.SendLiveness(ctx)
-}
-
-// Start the event proxy
-func (ep *EventProxy) Start() {
-	eq := ep.eventQueue
-	go eq.start(ep.queueCtx)
-	logger.Debugw(context.Background(), "event-proxy-starting...", log.Fields{"events-threashold": EVENT_THRESHOLD})
-	for {
-		// Notify the queue I am ready
-		eq.readyToSendToKafkaCh <- struct{}{}
-		// Wait for an event
-		elem, ok := <-eq.eventChannel
-		if !ok {
-			logger.Debug(context.Background(), "event-channel-closed-exiting")
-			break
-		}
-		// Check for last event
-		if _, ok := elem.(*lastEvent); ok {
-			// close the queuing loop
-			logger.Info(context.Background(), "received-last-event")
-			ep.queueCancelCtx()
-			break
-		}
-		ctx := context.Background()
-		event, ok := elem.(*voltha.Event)
-		if !ok {
-			logger.Warnw(ctx, "invalid-event", log.Fields{"element": elem})
-			continue
-		}
-		if err := ep.sendEvent(ctx, event); err != nil {
-			logger.Errorw(ctx, "failed-to-send-event-to-kafka-bus", log.Fields{"event": event})
-		} else {
-			logger.Debugw(ctx, "successfully-sent-rpc-event-to-kafka-bus", log.Fields{"id": event.Header.Id, "category": event.Header.Category,
-				"sub-category": event.Header.SubCategory, "type": event.Header.Type, "type-version": event.Header.TypeVersion,
-				"reported-ts": event.Header.ReportedTs, "event-type": event.EventType})
-		}
-	}
-}
-
-func (ep *EventProxy) Stop() {
-	ep.eventQueue.stop()
-}
-
-type EventQueue struct {
-	mutex                sync.RWMutex
-	eventChannel         chan interface{}
-	insertPosition       *ring.Ring
-	popPosition          *ring.Ring
-	dataToSendAvailable  chan struct{}
-	readyToSendToKafkaCh chan struct{}
-	eventQueueStopped    chan struct{}
-}
-
-func newEventQueue() *EventQueue {
-	ev := &EventQueue{
-		eventChannel:         make(chan interface{}),
-		insertPosition:       ring.New(EVENT_THRESHOLD),
-		dataToSendAvailable:  make(chan struct{}),
-		readyToSendToKafkaCh: make(chan struct{}),
-		eventQueueStopped:    make(chan struct{}),
-	}
-	ev.popPosition = ev.insertPosition
-	return ev
-}
-
-// push is invoked to push an event at the back of a queue
-func (eq *EventQueue) push(event interface{}) {
-	eq.mutex.Lock()
-
-	if eq.insertPosition != nil {
-		// Handle Queue is full.
-		// TODO: Current default is to overwrite old data if queue is full. Is there a need to
-		// block caller if max threshold is reached?
-		if eq.insertPosition.Value != nil && eq.insertPosition == eq.popPosition {
-			eq.popPosition = eq.popPosition.Next()
-		}
-
-		// Insert data and move pointer to next empty position
-		eq.insertPosition.Value = event
-		eq.insertPosition = eq.insertPosition.Next()
-
-		// Check for last event
-		if _, ok := event.(*lastEvent); ok {
-			eq.insertPosition = nil
-		}
-		eq.mutex.Unlock()
-		// Notify waiting thread of data availability
-		eq.dataToSendAvailable <- struct{}{}
-
-	} else {
-		logger.Debug(context.Background(), "event-queue-is-closed-as-insert-position-is-cleared")
-		eq.mutex.Unlock()
-	}
-}
-
-// start starts the routine that extracts an element from the event queue and
-// send it to the kafka sending routine to process.
-func (eq *EventQueue) start(ctx context.Context) {
-	logger.Info(ctx, "starting-event-queue")
-loop:
-	for {
-		select {
-		case <-eq.dataToSendAvailable:
-		//	Do nothing - use to prevent caller pushing data to block
-		case <-eq.readyToSendToKafkaCh:
-			{
-				// Kafka sending routine is ready to process an event
-				eq.mutex.Lock()
-				element := eq.popPosition.Value
-				if element == nil {
-					// No events to send. Wait
-					eq.mutex.Unlock()
-					select {
-					case _, ok := <-eq.dataToSendAvailable:
-						if !ok {
-							// channel closed
-							eq.eventQueueStopped <- struct{}{}
-							return
-						}
-					case <-ctx.Done():
-						logger.Info(ctx, "event-queue-context-done")
-						eq.eventQueueStopped <- struct{}{}
-						return
-					}
-					eq.mutex.Lock()
-					element = eq.popPosition.Value
-				}
-				eq.popPosition.Value = nil
-				eq.popPosition = eq.popPosition.Next()
-				eq.mutex.Unlock()
-				eq.eventChannel <- element
-			}
-		case <-ctx.Done():
-			logger.Info(ctx, "event-queue-context-done")
-			eq.eventQueueStopped <- struct{}{}
-			break loop
-		}
-	}
-	logger.Info(ctx, "event-queue-stopped")
-
-}
-
-func (eq *EventQueue) stop() {
-	// Flush all
-	eq.push(&lastEvent{})
-	<-eq.eventQueueStopped
-	eq.mutex.Lock()
-	close(eq.readyToSendToKafkaCh)
-	close(eq.dataToSendAvailable)
-	close(eq.eventChannel)
-	eq.mutex.Unlock()
-
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/utils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/utils.go
deleted file mode 100644
index fe3a017..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/events/utils.go
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2020-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 events
-
-import (
-	"fmt"
-	"strconv"
-
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-)
-
-type ContextType string
-
-const (
-	// ContextAdminState is for the admin state of the Device in the context of the event
-	ContextAdminState ContextType = "admin-state"
-	// ContextConnectState is for the connect state of the Device in the context of the event
-	ContextConnectState ContextType = "connect-state"
-	// ContextOperState is for the operational state of the Device in the context of the event
-	ContextOperState ContextType = "oper-state"
-	// ContextPrevdminState is for the previous admin state of the Device in the context of the event
-	ContextPrevAdminState ContextType = "prev-admin-state"
-	// ContextPrevConnectState is for the previous connect state of the Device in the context of the event
-	ContextPrevConnectState ContextType = "prev-connect-state"
-	// ContextPrevOperState is for the previous operational state of the Device in the context of the event
-	ContextPrevOperState ContextType = "prev-oper-state"
-	// ContextDeviceID is for the previous operational state of the Device in the context of the event
-	ContextDeviceID ContextType = "id"
-	// ContextParentID is for the parent id in the context of the event
-	ContextParentID ContextType = "parent-id"
-	// ContextSerialNumber is for the serial number of the Device in the context of the event
-	ContextSerialNumber ContextType = "serial-number"
-	// ContextIsRoot is for the root flag of Device in the context of the event
-	ContextIsRoot ContextType = "is-root"
-	// ContextParentPort is for the parent interface id of child in the context of the event
-	ContextParentPort ContextType = "parent-port"
-)
-
-type EventName string
-
-const (
-	DeviceStateChangeEvent EventName = "DEVICE_STATE_CHANGE"
-)
-
-type EventAction string
-
-const (
-	Raise EventAction = "RAISE_EVENT"
-	Clear EventAction = "CLEAR_EVENT"
-)
-
-//CreateDeviceStateChangeEvent forms and returns a new DeviceStateChange Event
-func CreateDeviceStateChangeEvent(serialNumber string, deviceID string, parentID string,
-	prevOperStatus voltha.OperStatus_Types, prevConnStatus voltha.ConnectStatus_Types, prevAdminStatus voltha.AdminState_Types,
-	operStatus voltha.OperStatus_Types, connStatus voltha.ConnectStatus_Types, adminStatus voltha.AdminState_Types,
-	parentPort uint32, isRoot bool) *voltha.DeviceEvent {
-
-	context := make(map[string]string)
-	/* Populating event context */
-	context[string(ContextSerialNumber)] = serialNumber
-	context[string(ContextDeviceID)] = deviceID
-	context[string(ContextParentID)] = parentID
-	context[string(ContextPrevOperState)] = prevOperStatus.String()
-	context[string(ContextPrevConnectState)] = prevConnStatus.String()
-	context[string(ContextPrevAdminState)] = prevAdminStatus.String()
-	context[string(ContextOperState)] = operStatus.String()
-	context[string(ContextConnectState)] = connStatus.String()
-	context[string(ContextAdminState)] = adminStatus.String()
-	context[string(ContextIsRoot)] = strconv.FormatBool(isRoot)
-	context[string(ContextParentPort)] = strconv.FormatUint(uint64(parentPort), 10)
-
-	return &voltha.DeviceEvent{
-		Context:         context,
-		ResourceId:      deviceID,
-		DeviceEventName: fmt.Sprintf("%s_%s", string(DeviceStateChangeEvent), string(Raise)),
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/common.go
deleted file mode 100644
index beb0574..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 flows
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/flow_utils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/flow_utils.go
deleted file mode 100644
index ff6aaf0..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/flows/flow_utils.go
+++ /dev/null
@@ -1,1608 +0,0 @@
-/*
- * 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 flows
-
-import (
-	"bytes"
-	"context"
-	"crypto/md5"
-	"encoding/binary"
-	"fmt"
-	"hash"
-	"sort"
-
-	"github.com/cevaris/ordered_map"
-	"github.com/gogo/protobuf/proto"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-)
-
-var (
-	// Instructions shortcut
-	APPLY_ACTIONS  = ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS
-	WRITE_METADATA = ofp.OfpInstructionType_OFPIT_WRITE_METADATA
-	METER_ACTION   = ofp.OfpInstructionType_OFPIT_METER
-
-	//OFPAT_* shortcuts
-	OUTPUT       = ofp.OfpActionType_OFPAT_OUTPUT
-	COPY_TTL_OUT = ofp.OfpActionType_OFPAT_COPY_TTL_OUT
-	COPY_TTL_IN  = ofp.OfpActionType_OFPAT_COPY_TTL_IN
-	SET_MPLS_TTL = ofp.OfpActionType_OFPAT_SET_MPLS_TTL
-	DEC_MPLS_TTL = ofp.OfpActionType_OFPAT_DEC_MPLS_TTL
-	PUSH_VLAN    = ofp.OfpActionType_OFPAT_PUSH_VLAN
-	POP_VLAN     = ofp.OfpActionType_OFPAT_POP_VLAN
-	PUSH_MPLS    = ofp.OfpActionType_OFPAT_PUSH_MPLS
-	POP_MPLS     = ofp.OfpActionType_OFPAT_POP_MPLS
-	SET_QUEUE    = ofp.OfpActionType_OFPAT_SET_QUEUE
-	GROUP        = ofp.OfpActionType_OFPAT_GROUP
-	SET_NW_TTL   = ofp.OfpActionType_OFPAT_SET_NW_TTL
-	NW_TTL       = ofp.OfpActionType_OFPAT_DEC_NW_TTL
-	SET_FIELD    = ofp.OfpActionType_OFPAT_SET_FIELD
-	PUSH_PBB     = ofp.OfpActionType_OFPAT_PUSH_PBB
-	POP_PBB      = ofp.OfpActionType_OFPAT_POP_PBB
-	EXPERIMENTER = ofp.OfpActionType_OFPAT_EXPERIMENTER
-
-	//OFPXMT_OFB_* shortcuts (incomplete)
-	IN_PORT         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT
-	IN_PHY_PORT     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT
-	METADATA        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_METADATA
-	ETH_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST
-	ETH_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC
-	ETH_TYPE        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE
-	VLAN_VID        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
-	VLAN_PCP        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP
-	IP_DSCP         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP
-	IP_ECN          = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN
-	IP_PROTO        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO
-	IPV4_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC
-	IPV4_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST
-	TCP_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC
-	TCP_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST
-	UDP_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC
-	UDP_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST
-	SCTP_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC
-	SCTP_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST
-	ICMPV4_TYPE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE
-	ICMPV4_CODE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE
-	ARP_OP          = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP
-	ARP_SPA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA
-	ARP_TPA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA
-	ARP_SHA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA
-	ARP_THA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA
-	IPV6_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC
-	IPV6_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST
-	IPV6_FLABEL     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL
-	ICMPV6_TYPE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE
-	ICMPV6_CODE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE
-	IPV6_ND_TARGET  = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET
-	OFB_IPV6_ND_SLL = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL
-	IPV6_ND_TLL     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL
-	MPLS_LABEL      = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL
-	MPLS_TC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC
-	MPLS_BOS        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS
-	PBB_ISID        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID
-	TUNNEL_ID       = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID
-	IPV6_EXTHDR     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR
-)
-
-//ofp_action_* shortcuts
-
-func Output(port uint32, maxLen ...ofp.OfpControllerMaxLen) *ofp.OfpAction {
-	maxLength := ofp.OfpControllerMaxLen_OFPCML_MAX
-	if len(maxLen) > 0 {
-		maxLength = maxLen[0]
-	}
-	return &ofp.OfpAction{Type: OUTPUT, Action: &ofp.OfpAction_Output{Output: &ofp.OfpActionOutput{Port: port, MaxLen: uint32(maxLength)}}}
-}
-
-func MplsTtl(ttl uint32) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: SET_MPLS_TTL, Action: &ofp.OfpAction_MplsTtl{MplsTtl: &ofp.OfpActionMplsTtl{MplsTtl: ttl}}}
-}
-
-func PushVlan(ethType uint32) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: PUSH_VLAN, Action: &ofp.OfpAction_Push{Push: &ofp.OfpActionPush{Ethertype: ethType}}}
-}
-
-func PopVlan() *ofp.OfpAction {
-	return &ofp.OfpAction{Type: POP_VLAN}
-}
-
-func PopMpls(ethType uint32) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: POP_MPLS, Action: &ofp.OfpAction_PopMpls{PopMpls: &ofp.OfpActionPopMpls{Ethertype: ethType}}}
-}
-
-func Group(groupId uint32) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: GROUP, Action: &ofp.OfpAction_Group{Group: &ofp.OfpActionGroup{GroupId: groupId}}}
-}
-
-func NwTtl(nwTtl uint32) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: NW_TTL, Action: &ofp.OfpAction_NwTtl{NwTtl: &ofp.OfpActionNwTtl{NwTtl: nwTtl}}}
-}
-
-func SetField(field *ofp.OfpOxmOfbField) *ofp.OfpAction {
-	actionSetField := &ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: &ofp.OfpOxmField_OfbField{OfbField: field}}
-	return &ofp.OfpAction{Type: SET_FIELD, Action: &ofp.OfpAction_SetField{SetField: &ofp.OfpActionSetField{Field: actionSetField}}}
-}
-
-func Experimenter(experimenter uint32, data []byte) *ofp.OfpAction {
-	return &ofp.OfpAction{Type: EXPERIMENTER, Action: &ofp.OfpAction_Experimenter{Experimenter: &ofp.OfpActionExperimenter{Experimenter: experimenter, Data: data}}}
-}
-
-//ofb_field generators (incomplete set)
-
-func InPort(inPort uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IN_PORT, Value: &ofp.OfpOxmOfbField_Port{Port: inPort}}
-}
-
-func InPhyPort(inPhyPort uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IN_PHY_PORT, Value: &ofp.OfpOxmOfbField_Port{Port: inPhyPort}}
-}
-
-func Metadata_ofp(tableMetadata uint64) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: METADATA, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: tableMetadata}}
-}
-
-// should Metadata_ofp used here ?????
-func EthDst(ethDst uint64) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ETH_DST, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: ethDst}}
-}
-
-// should Metadata_ofp used here ?????
-func EthSrc(ethSrc uint64) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ETH_SRC, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: ethSrc}}
-}
-
-func EthType(ethType uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ETH_TYPE, Value: &ofp.OfpOxmOfbField_EthType{EthType: ethType}}
-}
-
-func VlanVid(vlanVid uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: VLAN_VID, Value: &ofp.OfpOxmOfbField_VlanVid{VlanVid: vlanVid}}
-}
-
-func VlanPcp(vlanPcp uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: VLAN_PCP, Value: &ofp.OfpOxmOfbField_VlanPcp{VlanPcp: vlanPcp}}
-}
-
-func IpDscp(ipDscp uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IP_DSCP, Value: &ofp.OfpOxmOfbField_IpDscp{IpDscp: ipDscp}}
-}
-
-func IpEcn(ipEcn uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IP_ECN, Value: &ofp.OfpOxmOfbField_IpEcn{IpEcn: ipEcn}}
-}
-
-func IpProto(ipProto uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IP_PROTO, Value: &ofp.OfpOxmOfbField_IpProto{IpProto: ipProto}}
-}
-
-func Ipv4Src(ipv4Src uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV4_SRC, Value: &ofp.OfpOxmOfbField_Ipv4Src{Ipv4Src: ipv4Src}}
-}
-
-func Ipv4Dst(ipv4Dst uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV4_DST, Value: &ofp.OfpOxmOfbField_Ipv4Dst{Ipv4Dst: ipv4Dst}}
-}
-
-func TcpSrc(tcpSrc uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: TCP_SRC, Value: &ofp.OfpOxmOfbField_TcpSrc{TcpSrc: tcpSrc}}
-}
-
-func TcpDst(tcpDst uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: TCP_DST, Value: &ofp.OfpOxmOfbField_TcpDst{TcpDst: tcpDst}}
-}
-
-func UdpSrc(udpSrc uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: UDP_SRC, Value: &ofp.OfpOxmOfbField_UdpSrc{UdpSrc: udpSrc}}
-}
-
-func UdpDst(udpDst uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: UDP_DST, Value: &ofp.OfpOxmOfbField_UdpDst{UdpDst: udpDst}}
-}
-
-func SctpSrc(sctpSrc uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: SCTP_SRC, Value: &ofp.OfpOxmOfbField_SctpSrc{SctpSrc: sctpSrc}}
-}
-
-func SctpDst(sctpDst uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: SCTP_DST, Value: &ofp.OfpOxmOfbField_SctpDst{SctpDst: sctpDst}}
-}
-
-func Icmpv4Type(icmpv4Type uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ICMPV4_TYPE, Value: &ofp.OfpOxmOfbField_Icmpv4Type{Icmpv4Type: icmpv4Type}}
-}
-
-func Icmpv4Code(icmpv4Code uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ICMPV4_CODE, Value: &ofp.OfpOxmOfbField_Icmpv4Code{Icmpv4Code: icmpv4Code}}
-}
-
-func ArpOp(arpOp uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ARP_OP, Value: &ofp.OfpOxmOfbField_ArpOp{ArpOp: arpOp}}
-}
-
-func ArpSpa(arpSpa uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ARP_SPA, Value: &ofp.OfpOxmOfbField_ArpSpa{ArpSpa: arpSpa}}
-}
-
-func ArpTpa(arpTpa uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ARP_TPA, Value: &ofp.OfpOxmOfbField_ArpTpa{ArpTpa: arpTpa}}
-}
-
-func ArpSha(arpSha []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ARP_SHA, Value: &ofp.OfpOxmOfbField_ArpSha{ArpSha: arpSha}}
-}
-
-func ArpTha(arpTha []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ARP_THA, Value: &ofp.OfpOxmOfbField_ArpTha{ArpTha: arpTha}}
-}
-
-func Ipv6Src(ipv6Src []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_SRC, Value: &ofp.OfpOxmOfbField_Ipv6Src{Ipv6Src: ipv6Src}}
-}
-
-func Ipv6Dst(ipv6Dst []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_DST, Value: &ofp.OfpOxmOfbField_Ipv6Dst{Ipv6Dst: ipv6Dst}}
-}
-
-func Ipv6Flabel(ipv6Flabel uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_FLABEL, Value: &ofp.OfpOxmOfbField_Ipv6Flabel{Ipv6Flabel: ipv6Flabel}}
-}
-
-func Icmpv6Type(icmpv6Type uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ICMPV6_TYPE, Value: &ofp.OfpOxmOfbField_Icmpv6Type{Icmpv6Type: icmpv6Type}}
-}
-
-func Icmpv6Code(icmpv6Code uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: ICMPV6_CODE, Value: &ofp.OfpOxmOfbField_Icmpv6Code{Icmpv6Code: icmpv6Code}}
-}
-
-func Ipv6NdTarget(ipv6NdTarget []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_ND_TARGET, Value: &ofp.OfpOxmOfbField_Ipv6NdTarget{Ipv6NdTarget: ipv6NdTarget}}
-}
-
-func OfbIpv6NdSll(ofbIpv6NdSll []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: OFB_IPV6_ND_SLL, Value: &ofp.OfpOxmOfbField_Ipv6NdSsl{Ipv6NdSsl: ofbIpv6NdSll}}
-}
-
-func Ipv6NdTll(ipv6NdTll []byte) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_ND_TLL, Value: &ofp.OfpOxmOfbField_Ipv6NdTll{Ipv6NdTll: ipv6NdTll}}
-}
-
-func MplsLabel(mplsLabel uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: MPLS_LABEL, Value: &ofp.OfpOxmOfbField_MplsLabel{MplsLabel: mplsLabel}}
-}
-
-func MplsTc(mplsTc uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: MPLS_TC, Value: &ofp.OfpOxmOfbField_MplsTc{MplsTc: mplsTc}}
-}
-
-func MplsBos(mplsBos uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: MPLS_BOS, Value: &ofp.OfpOxmOfbField_MplsBos{MplsBos: mplsBos}}
-}
-
-func PbbIsid(pbbIsid uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: PBB_ISID, Value: &ofp.OfpOxmOfbField_PbbIsid{PbbIsid: pbbIsid}}
-}
-
-func TunnelId(tunnelId uint64) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: TUNNEL_ID, Value: &ofp.OfpOxmOfbField_TunnelId{TunnelId: tunnelId}}
-}
-
-func Ipv6Exthdr(ipv6Exthdr uint32) *ofp.OfpOxmOfbField {
-	return &ofp.OfpOxmOfbField{Type: IPV6_EXTHDR, Value: &ofp.OfpOxmOfbField_Ipv6Exthdr{Ipv6Exthdr: ipv6Exthdr}}
-}
-
-//frequently used extractors
-
-func excludeAction(action *ofp.OfpAction, exclude ...ofp.OfpActionType) bool {
-	for _, actionToExclude := range exclude {
-		if action.Type == actionToExclude {
-			return true
-		}
-	}
-	return false
-}
-
-func GetActions(flow *ofp.OfpFlowStats, exclude ...ofp.OfpActionType) []*ofp.OfpAction {
-	if flow == nil {
-		return nil
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
-			instActions := instruction.GetActions()
-			if instActions == nil {
-				return nil
-			}
-			if len(exclude) == 0 {
-				return instActions.Actions
-			} else {
-				filteredAction := make([]*ofp.OfpAction, 0)
-				for _, action := range instActions.Actions {
-					if !excludeAction(action, exclude...) {
-						filteredAction = append(filteredAction, action)
-					}
-				}
-				return filteredAction
-			}
-		}
-	}
-	return nil
-}
-
-func UpdateOutputPortByActionType(flow *ofp.OfpFlowStats, actionType uint32, toPort uint32) *ofp.OfpFlowStats {
-	if flow == nil {
-		return nil
-	}
-	nFlow := (proto.Clone(flow)).(*ofp.OfpFlowStats)
-	nFlow.Instructions = nil
-	nInsts := make([]*ofp.OfpInstruction, 0)
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == actionType {
-			instActions := instruction.GetActions()
-			if instActions == nil {
-				return nil
-			}
-			nActions := make([]*ofp.OfpAction, 0)
-			for _, action := range instActions.Actions {
-				if action.GetOutput() != nil {
-					nActions = append(nActions, Output(toPort))
-				} else {
-					nActions = append(nActions, action)
-				}
-			}
-			instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: nActions}}
-			nInsts = append(nInsts, &ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction})
-		} else {
-			nInsts = append(nInsts, instruction)
-		}
-	}
-	nFlow.Instructions = nInsts
-	return nFlow
-}
-
-func excludeOxmOfbField(field *ofp.OfpOxmOfbField, exclude ...ofp.OxmOfbFieldTypes) bool {
-	for _, fieldToExclude := range exclude {
-		if field.Type == fieldToExclude {
-			return true
-		}
-	}
-	return false
-}
-
-func GetOfbFields(flow *ofp.OfpFlowStats, exclude ...ofp.OxmOfbFieldTypes) []*ofp.OfpOxmOfbField {
-	if flow == nil || flow.Match == nil || flow.Match.Type != ofp.OfpMatchType_OFPMT_OXM {
-		return nil
-	}
-	ofbFields := make([]*ofp.OfpOxmOfbField, 0)
-	for _, field := range flow.Match.OxmFields {
-		if field.OxmClass == ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC {
-			ofbFields = append(ofbFields, field.GetOfbField())
-		}
-	}
-	if len(exclude) == 0 {
-		return ofbFields
-	} else {
-		filteredFields := make([]*ofp.OfpOxmOfbField, 0)
-		for _, ofbField := range ofbFields {
-			if !excludeOxmOfbField(ofbField, exclude...) {
-				filteredFields = append(filteredFields, ofbField)
-			}
-		}
-		return filteredFields
-	}
-}
-
-func GetPacketOutPort(packet *ofp.OfpPacketOut) uint32 {
-	if packet == nil {
-		return 0
-	}
-	for _, action := range packet.GetActions() {
-		if action.Type == OUTPUT {
-			return action.GetOutput().Port
-		}
-	}
-	return 0
-}
-
-func GetOutPort(flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, action := range GetActions(flow) {
-		if action.Type == OUTPUT {
-			out := action.GetOutput()
-			if out == nil {
-				return 0
-			}
-			return out.GetPort()
-		}
-	}
-	return 0
-}
-
-func GetInPort(flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, field := range GetOfbFields(flow) {
-		if field.Type == IN_PORT {
-			return field.GetPort()
-		}
-	}
-	return 0
-}
-
-func GetGotoTableId(flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_GOTO_TABLE) {
-			gotoTable := instruction.GetGotoTable()
-			if gotoTable == nil {
-				return 0
-			}
-			return gotoTable.GetTableId()
-		}
-	}
-	return 0
-}
-
-func GetMeterId(flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_METER) {
-			MeterInstruction := instruction.GetMeter()
-			if MeterInstruction == nil {
-				return 0
-			}
-			return MeterInstruction.GetMeterId()
-		}
-	}
-	return 0
-}
-
-func GetVlanVid(flow *ofp.OfpFlowStats) *uint32 {
-	if flow == nil {
-		return nil
-	}
-	for _, field := range GetOfbFields(flow) {
-		if field.Type == VLAN_VID {
-			ret := field.GetVlanVid()
-			return &ret
-		}
-	}
-	// Dont return 0 if the field is missing as vlan id value 0 has meaning and cannot be overloaded as "not found"
-	return nil
-}
-
-func GetSetActionField(ctx context.Context, flow *ofp.OfpFlowStats, ofbType ofp.OxmOfbFieldTypes) (uint32, bool) {
-	if flow == nil {
-		return 0, false
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(APPLY_ACTIONS) {
-			actions := instruction.GetActions()
-			for _, action := range actions.GetActions() {
-				if action.Type == SET_FIELD {
-					setField := action.GetSetField()
-					if setField.Field.GetOfbField().Type == ofbType {
-						switch ofbType {
-						case VLAN_PCP:
-							return setField.Field.GetOfbField().GetVlanPcp(), true
-						case VLAN_VID:
-							return setField.Field.GetOfbField().GetVlanVid(), true
-						default:
-							logger.Errorw(ctx, "unsupported-ofb-field-type", log.Fields{"ofbType": ofbType})
-							return 0, false
-						}
-					}
-				}
-			}
-			return 0, false
-		}
-	}
-	return 0, false
-}
-
-func GetTunnelId(flow *ofp.OfpFlowStats) uint64 {
-	if flow == nil {
-		return 0
-	}
-	for _, field := range GetOfbFields(flow) {
-		if field.Type == TUNNEL_ID {
-			return field.GetTunnelId()
-		}
-	}
-	return 0
-}
-
-//GetMetaData - legacy get method (only want lower 32 bits)
-func GetMetaData(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, field := range GetOfbFields(flow) {
-		if field.Type == METADATA {
-			return uint32(field.GetTableMetadata() & 0xFFFFFFFF)
-		}
-	}
-	logger.Debug(ctx, "No-metadata-present")
-	return 0
-}
-
-func GetMetaData64Bit(ctx context.Context, flow *ofp.OfpFlowStats) uint64 {
-	if flow == nil {
-		return 0
-	}
-	for _, field := range GetOfbFields(flow) {
-		if field.Type == METADATA {
-			return field.GetTableMetadata()
-		}
-	}
-	logger.Debug(ctx, "No-metadata-present")
-	return 0
-}
-
-// function returns write metadata value from write_metadata action field
-func GetMetadataFromWriteMetadataAction(ctx context.Context, flow *ofp.OfpFlowStats) uint64 {
-	if flow != nil {
-		for _, instruction := range flow.Instructions {
-			if instruction.Type == uint32(WRITE_METADATA) {
-				if writeMetadata := instruction.GetWriteMetadata(); writeMetadata != nil {
-					return writeMetadata.GetMetadata()
-				}
-			}
-		}
-	}
-	logger.Debugw(ctx, "No-write-metadata-present", log.Fields{"flow": flow})
-	return 0
-}
-
-func GetTechProfileIDFromWriteMetaData(ctx context.Context, metadata uint64) uint16 {
-	/*
-	   Write metadata instruction value (metadata) is 8 bytes:
-	   MS 2 bytes: C Tag
-	   Next 2 bytes: Technology Profile Id
-	   Next 4 bytes: Port number (uni or nni)
-
-	   This is set in the ONOS OltPipeline as a write metadata instruction
-	*/
-	var tpId uint16 = 0
-	logger.Debugw(ctx, "Write metadata value for Techprofile ID", log.Fields{"metadata": metadata})
-	if metadata != 0 {
-		tpId = uint16((metadata >> 32) & 0xFFFF)
-		logger.Debugw(ctx, "Found techprofile ID from write metadata action", log.Fields{"tpid": tpId})
-	}
-	return tpId
-}
-
-func GetEgressPortNumberFromWriteMetadata(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
-	/*
-			  Write metadata instruction value (metadata) is 8 bytes:
-		    	MS 2 bytes: C Tag
-		    	Next 2 bytes: Technology Profile Id
-		    	Next 4 bytes: Port number (uni or nni)
-		    	This is set in the ONOS OltPipeline as a write metadata instruction
-	*/
-	var uniPort uint32 = 0
-	md := GetMetadataFromWriteMetadataAction(ctx, flow)
-	logger.Debugw(ctx, "Metadata found for egress/uni port ", log.Fields{"metadata": md})
-	if md != 0 {
-		uniPort = uint32(md & 0xFFFFFFFF)
-		logger.Debugw(ctx, "Found EgressPort from write metadata action", log.Fields{"egress_port": uniPort})
-	}
-	return uniPort
-
-}
-
-func GetInnerTagFromMetaData(ctx context.Context, flow *ofp.OfpFlowStats) uint16 {
-	/*
-			  Write metadata instruction value (metadata) is 8 bytes:
-		    	MS 2 bytes: C Tag
-		    	Next 2 bytes: Technology Profile Id
-		    	Next 4 bytes: Port number (uni or nni)
-		    	This is set in the ONOS OltPipeline as a write metadata instruction
-	*/
-	var innerTag uint16 = 0
-	md := GetMetadataFromWriteMetadataAction(ctx, flow)
-	if md != 0 {
-		innerTag = uint16((md >> 48) & 0xFFFF)
-		logger.Debugw(ctx, "Found  CVLAN from write metadate action", log.Fields{"c_vlan": innerTag})
-	}
-	return innerTag
-}
-
-//GetInnerTagFromMetaData retrieves the inner tag from the Metadata_ofp. The port number (UNI on ONU) is in the
-// lower 32-bits of Metadata_ofp and the inner_tag is in the upper 32-bits. This is set in the ONOS OltPipeline as
-//// a Metadata_ofp field
-/*func GetInnerTagFromMetaData(flow *ofp.OfpFlowStats) uint64 {
-	md := GetMetaData64Bit(flow)
-	if md == 0 {
-		return 0
-	}
-	if md <= 0xffffffff {
-		logger.Debugw(ctx, "onos-upgrade-suggested", logger.Fields{"Metadata_ofp": md, "message": "Legacy MetaData detected form OltPipeline"})
-		return md
-	}
-	return (md >> 32) & 0xffffffff
-}*/
-
-// Extract the child device port from a flow that contains the parent device peer port.  Typically the UNI port of an
-// ONU child device.  Per TST agreement this will be the lower 32 bits of tunnel id reserving upper 32 bits for later
-// use
-func GetChildPortFromTunnelId(flow *ofp.OfpFlowStats) uint32 {
-	tid := GetTunnelId(flow)
-	if tid == 0 {
-		return 0
-	}
-	// Per TST agreement we are keeping any child port id (uni port id) in the lower 32 bits
-	return uint32(tid & 0xffffffff)
-}
-
-func HasNextTable(flow *ofp.OfpFlowStats) bool {
-	if flow == nil {
-		return false
-	}
-	return GetGotoTableId(flow) != 0
-}
-
-func GetGroup(flow *ofp.OfpFlowStats) uint32 {
-	if flow == nil {
-		return 0
-	}
-	for _, action := range GetActions(flow) {
-		if action.Type == GROUP {
-			grp := action.GetGroup()
-			if grp == nil {
-				return 0
-			}
-			return grp.GetGroupId()
-		}
-	}
-	return 0
-}
-
-func HasGroup(flow *ofp.OfpFlowStats) bool {
-	return GetGroup(flow) != 0
-}
-
-// GetNextTableId returns the next table ID if the "table_id" is present in the map, otherwise return nil
-func GetNextTableId(kw OfpFlowModArgs) *uint32 {
-	if val, exist := kw["table_id"]; exist {
-		ret := uint32(val)
-		return &ret
-	}
-	return nil
-}
-
-// GetMeterIdFlowModArgs returns the meterId if the "meter_id" is present in the map, otherwise return 0
-func GetMeterIdFlowModArgs(kw OfpFlowModArgs) uint32 {
-	if val, exist := kw["meter_id"]; exist {
-		return uint32(val)
-	}
-	return 0
-}
-
-// Function returns the metadata if the "write_metadata" is present in the map, otherwise return nil
-func GetMetadataFlowModArgs(kw OfpFlowModArgs) uint64 {
-	if val, exist := kw["write_metadata"]; exist {
-		ret := uint64(val)
-		return ret
-	}
-	return 0
-}
-
-// HashFlowStats returns a unique 64-bit integer hash of 'table_id', 'priority', and 'match'
-// The OF spec states that:
-// A flow table entry is identified by its match fields and priority: the match fields
-// and priority taken together identify a unique flow entry in the flow table.
-func HashFlowStats(flow *ofp.OfpFlowStats) (uint64, error) {
-	// first we need to make sure the oxm fields are in a predictable order (the specific order doesn't matter)
-	sort.Slice(flow.Match.OxmFields, func(a, b int) bool {
-		fieldsA, fieldsB := flow.Match.OxmFields[a], flow.Match.OxmFields[b]
-		if fieldsA.OxmClass < fieldsB.OxmClass {
-			return true
-		}
-		switch fieldA := fieldsA.Field.(type) {
-		case *ofp.OfpOxmField_OfbField:
-			switch fieldB := fieldsB.Field.(type) {
-			case *ofp.OfpOxmField_ExperimenterField:
-				return true // ofp < experimenter
-			case *ofp.OfpOxmField_OfbField:
-				return fieldA.OfbField.Type < fieldB.OfbField.Type
-			}
-		case *ofp.OfpOxmField_ExperimenterField:
-			switch fieldB := fieldsB.Field.(type) {
-			case *ofp.OfpOxmField_OfbField:
-				return false // ofp < experimenter
-			case *ofp.OfpOxmField_ExperimenterField:
-				eFieldA, eFieldB := fieldA.ExperimenterField, fieldB.ExperimenterField
-				if eFieldA.Experimenter != eFieldB.Experimenter {
-					return eFieldA.Experimenter < eFieldB.Experimenter
-				}
-				return eFieldA.OxmHeader < eFieldB.OxmHeader
-			}
-		}
-		return false
-	})
-
-	md5Hash := md5.New() // note that write errors will never occur with md5 hashing
-	var tmp [12]byte
-
-	binary.BigEndian.PutUint32(tmp[0:4], flow.TableId)             // tableId
-	binary.BigEndian.PutUint32(tmp[4:8], flow.Priority)            // priority
-	binary.BigEndian.PutUint32(tmp[8:12], uint32(flow.Match.Type)) // match type
-	_, _ = md5Hash.Write(tmp[:12])
-
-	for _, field := range flow.Match.OxmFields { // for all match fields
-		binary.BigEndian.PutUint32(tmp[:4], uint32(field.OxmClass)) // match class
-		_, _ = md5Hash.Write(tmp[:4])
-
-		switch oxmField := field.Field.(type) {
-		case *ofp.OfpOxmField_ExperimenterField:
-			binary.BigEndian.PutUint32(tmp[0:4], oxmField.ExperimenterField.Experimenter)
-			binary.BigEndian.PutUint32(tmp[4:8], oxmField.ExperimenterField.OxmHeader)
-			_, _ = md5Hash.Write(tmp[:8])
-
-		case *ofp.OfpOxmField_OfbField:
-			if err := hashWriteOfbField(md5Hash, oxmField.OfbField); err != nil {
-				return 0, err
-			}
-
-		default:
-			return 0, fmt.Errorf("unknown OfpOxmField type: %T", field.Field)
-		}
-	}
-
-	ret := md5Hash.Sum(nil)
-	return binary.BigEndian.Uint64(ret[0:8]), nil
-}
-
-func hashWriteOfbField(md5Hash hash.Hash, field *ofp.OfpOxmOfbField) error {
-	var tmp [8]byte
-	binary.BigEndian.PutUint32(tmp[:4], uint32(field.Type)) // type
-	_, _ = md5Hash.Write(tmp[:4])
-
-	// value
-	valType, val32, val64, valSlice := uint8(0), uint32(0), uint64(0), []byte(nil)
-	switch val := field.Value.(type) {
-	case *ofp.OfpOxmOfbField_Port:
-		valType, val32 = 4, val.Port
-	case *ofp.OfpOxmOfbField_PhysicalPort:
-		valType, val32 = 4, val.PhysicalPort
-	case *ofp.OfpOxmOfbField_TableMetadata:
-		valType, val64 = 8, val.TableMetadata
-	case *ofp.OfpOxmOfbField_EthDst:
-		valType, valSlice = 1, val.EthDst
-	case *ofp.OfpOxmOfbField_EthSrc:
-		valType, valSlice = 1, val.EthSrc
-	case *ofp.OfpOxmOfbField_EthType:
-		valType, val32 = 4, val.EthType
-	case *ofp.OfpOxmOfbField_VlanVid:
-		valType, val32 = 4, val.VlanVid
-	case *ofp.OfpOxmOfbField_VlanPcp:
-		valType, val32 = 4, val.VlanPcp
-	case *ofp.OfpOxmOfbField_IpDscp:
-		valType, val32 = 4, val.IpDscp
-	case *ofp.OfpOxmOfbField_IpEcn:
-		valType, val32 = 4, val.IpEcn
-	case *ofp.OfpOxmOfbField_IpProto:
-		valType, val32 = 4, val.IpProto
-	case *ofp.OfpOxmOfbField_Ipv4Src:
-		valType, val32 = 4, val.Ipv4Src
-	case *ofp.OfpOxmOfbField_Ipv4Dst:
-		valType, val32 = 4, val.Ipv4Dst
-	case *ofp.OfpOxmOfbField_TcpSrc:
-		valType, val32 = 4, val.TcpSrc
-	case *ofp.OfpOxmOfbField_TcpDst:
-		valType, val32 = 4, val.TcpDst
-	case *ofp.OfpOxmOfbField_UdpSrc:
-		valType, val32 = 4, val.UdpSrc
-	case *ofp.OfpOxmOfbField_UdpDst:
-		valType, val32 = 4, val.UdpDst
-	case *ofp.OfpOxmOfbField_SctpSrc:
-		valType, val32 = 4, val.SctpSrc
-	case *ofp.OfpOxmOfbField_SctpDst:
-		valType, val32 = 4, val.SctpDst
-	case *ofp.OfpOxmOfbField_Icmpv4Type:
-		valType, val32 = 4, val.Icmpv4Type
-	case *ofp.OfpOxmOfbField_Icmpv4Code:
-		valType, val32 = 4, val.Icmpv4Code
-	case *ofp.OfpOxmOfbField_ArpOp:
-		valType, val32 = 4, val.ArpOp
-	case *ofp.OfpOxmOfbField_ArpSpa:
-		valType, val32 = 4, val.ArpSpa
-	case *ofp.OfpOxmOfbField_ArpTpa:
-		valType, val32 = 4, val.ArpTpa
-	case *ofp.OfpOxmOfbField_ArpSha:
-		valType, valSlice = 1, val.ArpSha
-	case *ofp.OfpOxmOfbField_ArpTha:
-		valType, valSlice = 1, val.ArpTha
-	case *ofp.OfpOxmOfbField_Ipv6Src:
-		valType, valSlice = 1, val.Ipv6Src
-	case *ofp.OfpOxmOfbField_Ipv6Dst:
-		valType, valSlice = 1, val.Ipv6Dst
-	case *ofp.OfpOxmOfbField_Ipv6Flabel:
-		valType, val32 = 4, val.Ipv6Flabel
-	case *ofp.OfpOxmOfbField_Icmpv6Type:
-		valType, val32 = 4, val.Icmpv6Type
-	case *ofp.OfpOxmOfbField_Icmpv6Code:
-		valType, val32 = 4, val.Icmpv6Code
-	case *ofp.OfpOxmOfbField_Ipv6NdTarget:
-		valType, valSlice = 1, val.Ipv6NdTarget
-	case *ofp.OfpOxmOfbField_Ipv6NdSsl:
-		valType, valSlice = 1, val.Ipv6NdSsl
-	case *ofp.OfpOxmOfbField_Ipv6NdTll:
-		valType, valSlice = 1, val.Ipv6NdTll
-	case *ofp.OfpOxmOfbField_MplsLabel:
-		valType, val32 = 4, val.MplsLabel
-	case *ofp.OfpOxmOfbField_MplsTc:
-		valType, val32 = 4, val.MplsTc
-	case *ofp.OfpOxmOfbField_MplsBos:
-		valType, val32 = 4, val.MplsBos
-	case *ofp.OfpOxmOfbField_PbbIsid:
-		valType, val32 = 4, val.PbbIsid
-	case *ofp.OfpOxmOfbField_TunnelId:
-		valType, val64 = 8, val.TunnelId
-	case *ofp.OfpOxmOfbField_Ipv6Exthdr:
-		valType, val32 = 4, val.Ipv6Exthdr
-	default:
-		return fmt.Errorf("unknown OfpOxmField value type: %T", val)
-	}
-	switch valType {
-	case 1: // slice
-		_, _ = md5Hash.Write(valSlice)
-	case 4: // uint32
-		binary.BigEndian.PutUint32(tmp[:4], val32)
-		_, _ = md5Hash.Write(tmp[:4])
-	case 8: // uint64
-		binary.BigEndian.PutUint64(tmp[:8], val64)
-		_, _ = md5Hash.Write(tmp[:8])
-	}
-
-	// mask
-	if !field.HasMask {
-		tmp[0] = 0x00
-		_, _ = md5Hash.Write(tmp[:1]) // match hasMask = false
-	} else {
-		tmp[0] = 0x01
-		_, _ = md5Hash.Write(tmp[:1]) // match hasMask = true
-
-		maskType, mask32, mask64, maskSlice := uint8(0), uint32(0), uint64(0), []byte(nil)
-		switch mask := field.Mask.(type) {
-		case *ofp.OfpOxmOfbField_TableMetadataMask:
-			maskType, mask64 = 8, mask.TableMetadataMask
-		case *ofp.OfpOxmOfbField_EthDstMask:
-			maskType, maskSlice = 1, mask.EthDstMask
-		case *ofp.OfpOxmOfbField_EthSrcMask:
-			maskType, maskSlice = 1, mask.EthSrcMask
-		case *ofp.OfpOxmOfbField_VlanVidMask:
-			maskType, mask32 = 4, mask.VlanVidMask
-		case *ofp.OfpOxmOfbField_Ipv4SrcMask:
-			maskType, mask32 = 4, mask.Ipv4SrcMask
-		case *ofp.OfpOxmOfbField_Ipv4DstMask:
-			maskType, mask32 = 4, mask.Ipv4DstMask
-		case *ofp.OfpOxmOfbField_ArpSpaMask:
-			maskType, mask32 = 4, mask.ArpSpaMask
-		case *ofp.OfpOxmOfbField_ArpTpaMask:
-			maskType, mask32 = 4, mask.ArpTpaMask
-		case *ofp.OfpOxmOfbField_Ipv6SrcMask:
-			maskType, maskSlice = 1, mask.Ipv6SrcMask
-		case *ofp.OfpOxmOfbField_Ipv6DstMask:
-			maskType, maskSlice = 1, mask.Ipv6DstMask
-		case *ofp.OfpOxmOfbField_Ipv6FlabelMask:
-			maskType, mask32 = 4, mask.Ipv6FlabelMask
-		case *ofp.OfpOxmOfbField_PbbIsidMask:
-			maskType, mask32 = 4, mask.PbbIsidMask
-		case *ofp.OfpOxmOfbField_TunnelIdMask:
-			maskType, mask64 = 8, mask.TunnelIdMask
-		case *ofp.OfpOxmOfbField_Ipv6ExthdrMask:
-			maskType, mask32 = 4, mask.Ipv6ExthdrMask
-		case nil:
-			return fmt.Errorf("hasMask set to true, but no mask present")
-		default:
-			return fmt.Errorf("unknown OfpOxmField mask type: %T", mask)
-		}
-		switch maskType {
-		case 1: // slice
-			_, _ = md5Hash.Write(maskSlice)
-		case 4: // uint32
-			binary.BigEndian.PutUint32(tmp[:4], mask32)
-			_, _ = md5Hash.Write(tmp[:4])
-		case 8: // uint64
-			binary.BigEndian.PutUint64(tmp[:8], mask64)
-			_, _ = md5Hash.Write(tmp[:8])
-		}
-	}
-	return nil
-}
-
-// flowStatsEntryFromFlowModMessage maps an ofp_flow_mod message to an ofp_flow_stats message
-func FlowStatsEntryFromFlowModMessage(mod *ofp.OfpFlowMod) (*ofp.OfpFlowStats, error) {
-	flow := &ofp.OfpFlowStats{}
-	if mod == nil {
-		return flow, nil
-	}
-	flow.TableId = mod.TableId
-	flow.Priority = mod.Priority
-	flow.IdleTimeout = mod.IdleTimeout
-	flow.HardTimeout = mod.HardTimeout
-	flow.Flags = mod.Flags
-	flow.Cookie = mod.Cookie
-	flow.Match = mod.Match
-	flow.Instructions = mod.Instructions
-	var err error
-	if flow.Id, err = HashFlowStats(flow); err != nil {
-		return nil, err
-	}
-
-	return flow, nil
-}
-
-func GroupEntryFromGroupMod(mod *ofp.OfpGroupMod) *ofp.OfpGroupEntry {
-	group := &ofp.OfpGroupEntry{}
-	if mod == nil {
-		return group
-	}
-	group.Desc = &ofp.OfpGroupDesc{Type: mod.Type, GroupId: mod.GroupId, Buckets: mod.Buckets}
-	group.Stats = &ofp.OfpGroupStats{GroupId: mod.GroupId}
-	//TODO do we need to instantiate bucket bins?
-	return group
-}
-
-// flowStatsEntryFromFlowModMessage maps an ofp_flow_mod message to an ofp_flow_stats message
-func MeterEntryFromMeterMod(ctx context.Context, meterMod *ofp.OfpMeterMod) *ofp.OfpMeterEntry {
-	bandStats := make([]*ofp.OfpMeterBandStats, 0)
-	meter := &ofp.OfpMeterEntry{Config: &ofp.OfpMeterConfig{},
-		Stats: &ofp.OfpMeterStats{BandStats: bandStats}}
-	if meterMod == nil {
-		logger.Error(ctx, "Invalid meter mod command")
-		return meter
-	}
-	// config init
-	meter.Config.MeterId = meterMod.MeterId
-	meter.Config.Flags = meterMod.Flags
-	meter.Config.Bands = meterMod.Bands
-	// meter stats init
-	meter.Stats.MeterId = meterMod.MeterId
-	meter.Stats.FlowCount = 0
-	meter.Stats.PacketInCount = 0
-	meter.Stats.ByteInCount = 0
-	meter.Stats.DurationSec = 0
-	meter.Stats.DurationNsec = 0
-	// band stats init
-	for range meterMod.Bands {
-		band := &ofp.OfpMeterBandStats{}
-		band.PacketBandCount = 0
-		band.ByteBandCount = 0
-		bandStats = append(bandStats, band)
-	}
-	meter.Stats.BandStats = bandStats
-	logger.Debugw(ctx, "Allocated meter entry", log.Fields{"meter": *meter})
-	return meter
-
-}
-
-func GetMeterIdFromFlow(flow *ofp.OfpFlowStats) uint32 {
-	if flow != nil {
-		for _, instruction := range flow.Instructions {
-			if instruction.Type == uint32(METER_ACTION) {
-				if meterInst := instruction.GetMeter(); meterInst != nil {
-					return meterInst.GetMeterId()
-				}
-			}
-		}
-	}
-
-	return uint32(0)
-}
-
-func MkOxmFields(matchFields []ofp.OfpOxmField) []*ofp.OfpOxmField {
-	oxmFields := make([]*ofp.OfpOxmField, 0)
-	for _, matchField := range matchFields {
-		oxmField := ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: matchField.Field}
-		oxmFields = append(oxmFields, &oxmField)
-	}
-	return oxmFields
-}
-
-func MkInstructionsFromActions(actions []*ofp.OfpAction) []*ofp.OfpInstruction {
-	instructions := make([]*ofp.OfpInstruction, 0)
-	instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: actions}}
-	instruction := ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction}
-	instructions = append(instructions, &instruction)
-	return instructions
-}
-
-// Convenience function to generare ofp_flow_mod message with OXM BASIC match composed from the match_fields, and
-// single APPLY_ACTIONS instruction with a list if ofp_action objects.
-func MkSimpleFlowMod(matchFields []*ofp.OfpOxmField, actions []*ofp.OfpAction, command *ofp.OfpFlowModCommand, kw OfpFlowModArgs) *ofp.OfpFlowMod {
-
-	// Process actions instructions
-	instructions := make([]*ofp.OfpInstruction, 0)
-	instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: actions}}
-	instruction := ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction}
-	instructions = append(instructions, &instruction)
-
-	// Process next table
-	if tableId := GetNextTableId(kw); tableId != nil {
-		var instGotoTable ofp.OfpInstruction_GotoTable
-		instGotoTable.GotoTable = &ofp.OfpInstructionGotoTable{TableId: *tableId}
-		inst := ofp.OfpInstruction{Type: uint32(ofp.OfpInstructionType_OFPIT_GOTO_TABLE), Data: &instGotoTable}
-		instructions = append(instructions, &inst)
-	}
-	// Process meter action
-	if meterId := GetMeterIdFlowModArgs(kw); meterId != 0 {
-		var instMeter ofp.OfpInstruction_Meter
-		instMeter.Meter = &ofp.OfpInstructionMeter{MeterId: meterId}
-		inst := ofp.OfpInstruction{Type: uint32(METER_ACTION), Data: &instMeter}
-		instructions = append(instructions, &inst)
-	}
-	//process write_metadata action
-	if metadata := GetMetadataFlowModArgs(kw); metadata != 0 {
-		var instWriteMetadata ofp.OfpInstruction_WriteMetadata
-		instWriteMetadata.WriteMetadata = &ofp.OfpInstructionWriteMetadata{Metadata: metadata}
-		inst := ofp.OfpInstruction{Type: uint32(WRITE_METADATA), Data: &instWriteMetadata}
-		instructions = append(instructions, &inst)
-	}
-
-	// Process match fields
-	oxmFields := make([]*ofp.OfpOxmField, 0)
-	for _, matchField := range matchFields {
-		oxmField := ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: matchField.Field}
-		oxmFields = append(oxmFields, &oxmField)
-	}
-	var match ofp.OfpMatch
-	match.Type = ofp.OfpMatchType_OFPMT_OXM
-	match.OxmFields = oxmFields
-
-	// Create ofp_flow_message
-	msg := &ofp.OfpFlowMod{}
-	if command == nil {
-		msg.Command = ofp.OfpFlowModCommand_OFPFC_ADD
-	} else {
-		msg.Command = *command
-	}
-	msg.Instructions = instructions
-	msg.Match = &match
-
-	// Set the variadic argument values
-	msg = setVariadicModAttributes(msg, kw)
-
-	return msg
-}
-
-func MkMulticastGroupMod(groupId uint32, buckets []*ofp.OfpBucket, command *ofp.OfpGroupModCommand) *ofp.OfpGroupMod {
-	group := &ofp.OfpGroupMod{}
-	if command == nil {
-		group.Command = ofp.OfpGroupModCommand_OFPGC_ADD
-	} else {
-		group.Command = *command
-	}
-	group.Type = ofp.OfpGroupType_OFPGT_ALL
-	group.GroupId = groupId
-	group.Buckets = buckets
-	return group
-}
-
-//SetVariadicModAttributes sets only uint64 or uint32 fields of the ofp_flow_mod message
-func setVariadicModAttributes(mod *ofp.OfpFlowMod, args OfpFlowModArgs) *ofp.OfpFlowMod {
-	if args == nil {
-		return mod
-	}
-	for key, val := range args {
-		switch key {
-		case "cookie":
-			mod.Cookie = val
-		case "cookie_mask":
-			mod.CookieMask = val
-		case "table_id":
-			mod.TableId = uint32(val)
-		case "idle_timeout":
-			mod.IdleTimeout = uint32(val)
-		case "hard_timeout":
-			mod.HardTimeout = uint32(val)
-		case "priority":
-			mod.Priority = uint32(val)
-		case "buffer_id":
-			mod.BufferId = uint32(val)
-		case "out_port":
-			mod.OutPort = uint32(val)
-		case "out_group":
-			mod.OutGroup = uint32(val)
-		case "flags":
-			mod.Flags = uint32(val)
-		}
-	}
-	return mod
-}
-
-func MkPacketIn(port uint32, packet []byte) *ofp.OfpPacketIn {
-	packetIn := &ofp.OfpPacketIn{
-		Reason: ofp.OfpPacketInReason_OFPR_ACTION,
-		Match: &ofp.OfpMatch{
-			Type: ofp.OfpMatchType_OFPMT_OXM,
-			OxmFields: []*ofp.OfpOxmField{
-				{
-					OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC,
-					Field: &ofp.OfpOxmField_OfbField{
-						OfbField: InPort(port)},
-				},
-			},
-		},
-		Data: packet,
-	}
-	return packetIn
-}
-
-// MkFlowStat is a helper method to build flows
-func MkFlowStat(fa *FlowArgs) (*ofp.OfpFlowStats, error) {
-	//Build the match-fields
-	matchFields := make([]*ofp.OfpOxmField, 0)
-	for _, val := range fa.MatchFields {
-		matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
-	}
-	return FlowStatsEntryFromFlowModMessage(MkSimpleFlowMod(matchFields, fa.Actions, fa.Command, fa.KV))
-}
-
-func MkGroupStat(ga *GroupArgs) *ofp.OfpGroupEntry {
-	return GroupEntryFromGroupMod(MkMulticastGroupMod(ga.GroupId, ga.Buckets, ga.Command))
-}
-
-type OfpFlowModArgs map[string]uint64
-
-type FlowArgs struct {
-	MatchFields []*ofp.OfpOxmOfbField
-	Actions     []*ofp.OfpAction
-	Command     *ofp.OfpFlowModCommand
-	Priority    uint32
-	KV          OfpFlowModArgs
-}
-
-type GroupArgs struct {
-	GroupId uint32
-	Buckets []*ofp.OfpBucket
-	Command *ofp.OfpGroupModCommand
-}
-
-type FlowsAndGroups struct {
-	Flows  *ordered_map.OrderedMap
-	Groups *ordered_map.OrderedMap
-}
-
-func NewFlowsAndGroups() *FlowsAndGroups {
-	var fg FlowsAndGroups
-	fg.Flows = ordered_map.NewOrderedMap()
-	fg.Groups = ordered_map.NewOrderedMap()
-	return &fg
-}
-
-func (fg *FlowsAndGroups) Copy() *FlowsAndGroups {
-	copyFG := NewFlowsAndGroups()
-	iter := fg.Flows.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
-			copyFG.Flows.Set(kv.Key, proto.Clone(protoMsg))
-		}
-	}
-	iter = fg.Groups.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
-			copyFG.Groups.Set(kv.Key, proto.Clone(protoMsg))
-		}
-	}
-	return copyFG
-}
-
-func (fg *FlowsAndGroups) GetFlow(index int) *ofp.OfpFlowStats {
-	iter := fg.Flows.IterFunc()
-	pos := 0
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if pos == index {
-			if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
-				return protoMsg
-			}
-			return nil
-		}
-		pos += 1
-	}
-	return nil
-}
-
-func (fg *FlowsAndGroups) ListFlows() []*ofp.OfpFlowStats {
-	flows := make([]*ofp.OfpFlowStats, 0)
-	iter := fg.Flows.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
-			flows = append(flows, protoMsg)
-		}
-	}
-	return flows
-}
-
-func (fg *FlowsAndGroups) ListGroups() []*ofp.OfpGroupEntry {
-	groups := make([]*ofp.OfpGroupEntry, 0)
-	iter := fg.Groups.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
-			groups = append(groups, protoMsg)
-		}
-	}
-	return groups
-}
-
-func (fg *FlowsAndGroups) String() string {
-	var buffer bytes.Buffer
-	iter := fg.Flows.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
-			buffer.WriteString("\nFlow:\n")
-			buffer.WriteString(proto.MarshalTextString(protoMsg))
-			buffer.WriteString("\n")
-		}
-	}
-	iter = fg.Groups.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
-			buffer.WriteString("\nGroup:\n")
-			buffer.WriteString(proto.MarshalTextString(protoMsg))
-			buffer.WriteString("\n")
-		}
-	}
-	return buffer.String()
-}
-
-func (fg *FlowsAndGroups) AddFlow(flow *ofp.OfpFlowStats) {
-	if flow == nil {
-		return
-	}
-
-	if fg.Flows == nil {
-		fg.Flows = ordered_map.NewOrderedMap()
-	}
-	if fg.Groups == nil {
-		fg.Groups = ordered_map.NewOrderedMap()
-	}
-	//Add flow only if absent
-	if _, exist := fg.Flows.Get(flow.Id); !exist {
-		fg.Flows.Set(flow.Id, flow)
-	}
-}
-
-func (fg *FlowsAndGroups) AddGroup(group *ofp.OfpGroupEntry) {
-	if group == nil {
-		return
-	}
-
-	if fg.Flows == nil {
-		fg.Flows = ordered_map.NewOrderedMap()
-	}
-	if fg.Groups == nil {
-		fg.Groups = ordered_map.NewOrderedMap()
-	}
-	//Add group only if absent
-	if _, exist := fg.Groups.Get(group.Desc.GroupId); !exist {
-		fg.Groups.Set(group.Desc.GroupId, group)
-	}
-}
-
-//AddFrom add flows and groups from the argument into this structure only if they do not already exist
-func (fg *FlowsAndGroups) AddFrom(from *FlowsAndGroups) {
-	iter := from.Flows.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
-			if _, exist := fg.Flows.Get(protoMsg.Id); !exist {
-				fg.Flows.Set(protoMsg.Id, protoMsg)
-			}
-		}
-	}
-	iter = from.Groups.IterFunc()
-	for kv, ok := iter(); ok; kv, ok = iter() {
-		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
-			if _, exist := fg.Groups.Get(protoMsg.Stats.GroupId); !exist {
-				fg.Groups.Set(protoMsg.Stats.GroupId, protoMsg)
-			}
-		}
-	}
-}
-
-type DeviceRules struct {
-	Rules map[string]*FlowsAndGroups
-}
-
-func NewDeviceRules() *DeviceRules {
-	var dr DeviceRules
-	dr.Rules = make(map[string]*FlowsAndGroups)
-	return &dr
-}
-
-func (dr *DeviceRules) Copy() *DeviceRules {
-	copyDR := NewDeviceRules()
-	if dr != nil {
-		for key, val := range dr.Rules {
-			if val != nil {
-				copyDR.Rules[key] = val.Copy()
-			}
-		}
-	}
-	return copyDR
-}
-
-func (dr *DeviceRules) ClearFlows(deviceId string) {
-	if _, exist := dr.Rules[deviceId]; exist {
-		dr.Rules[deviceId].Flows = ordered_map.NewOrderedMap()
-	}
-}
-
-func (dr *DeviceRules) FilterRules(deviceIds map[string]string) *DeviceRules {
-	filteredDR := NewDeviceRules()
-	for key, val := range dr.Rules {
-		if _, exist := deviceIds[key]; exist {
-			filteredDR.Rules[key] = val.Copy()
-		}
-	}
-	return filteredDR
-}
-
-func (dr *DeviceRules) AddFlow(deviceId string, flow *ofp.OfpFlowStats) {
-	if _, exist := dr.Rules[deviceId]; !exist {
-		dr.Rules[deviceId] = NewFlowsAndGroups()
-	}
-	dr.Rules[deviceId].AddFlow(flow)
-}
-
-func (dr *DeviceRules) GetRules() map[string]*FlowsAndGroups {
-	return dr.Rules
-}
-
-func (dr *DeviceRules) String() string {
-	var buffer bytes.Buffer
-	for key, value := range dr.Rules {
-		buffer.WriteString("DeviceId:")
-		buffer.WriteString(key)
-		buffer.WriteString(value.String())
-		buffer.WriteString("\n\n")
-	}
-	return buffer.String()
-}
-
-func (dr *DeviceRules) AddFlowsAndGroup(deviceId string, fg *FlowsAndGroups) {
-	if _, ok := dr.Rules[deviceId]; !ok {
-		dr.Rules[deviceId] = NewFlowsAndGroups()
-	}
-	dr.Rules[deviceId] = fg
-}
-
-// CreateEntryIfNotExist creates a new deviceId in the Map if it does not exist and assigns an
-// empty FlowsAndGroups to it.  Otherwise, it does nothing.
-func (dr *DeviceRules) CreateEntryIfNotExist(deviceId string) {
-	if _, ok := dr.Rules[deviceId]; !ok {
-		dr.Rules[deviceId] = NewFlowsAndGroups()
-	}
-}
-
-/*
- *  Common flow routines
- */
-
-//FindOverlappingFlows return a list of overlapping flow(s) where mod is the flow request
-func FindOverlappingFlows(flows []*ofp.OfpFlowStats, mod *ofp.OfpFlowMod) []*ofp.OfpFlowStats {
-	return nil //TODO - complete implementation
-}
-
-// FindFlowById returns the index of the flow in the flows array if present. Otherwise, it returns -1
-func FindFlowById(flows []*ofp.OfpFlowStats, flow *ofp.OfpFlowStats) int {
-	for idx, f := range flows {
-		if flow.Id == f.Id {
-			return idx
-		}
-	}
-	return -1
-}
-
-// FindFlows returns the index in flows where flow if present.  Otherwise, it returns -1
-func FindFlows(flows []*ofp.OfpFlowStats, flow *ofp.OfpFlowStats) int {
-	for idx, f := range flows {
-		if f.Id == flow.Id {
-			return idx
-		}
-	}
-	return -1
-}
-
-//FlowMatch returns true if two flows matches on the following flow attributes:
-//TableId, Priority, Flags, Cookie, Match
-func FlowMatch(f1 *ofp.OfpFlowStats, f2 *ofp.OfpFlowStats) bool {
-	return f1 != nil && f2 != nil && f1.Id == f2.Id
-}
-
-//FlowMatchesMod returns True if given flow is "covered" by the wildcard flow_mod, taking into consideration of
-//both exact matches as well as masks-based match fields if any. Otherwise return False
-func FlowMatchesMod(flow *ofp.OfpFlowStats, mod *ofp.OfpFlowMod) bool {
-	if flow == nil || mod == nil {
-		return false
-	}
-	//Check if flow.cookie is covered by mod.cookie and mod.cookie_mask
-	if (flow.Cookie & mod.CookieMask) != (mod.Cookie & mod.CookieMask) {
-		return false
-	}
-
-	//Check if flow.table_id is covered by flow_mod.table_id
-	if mod.TableId != uint32(ofp.OfpTable_OFPTT_ALL) && flow.TableId != mod.TableId {
-		return false
-	}
-
-	//Check out_port
-	if (mod.OutPort&0x7fffffff) != uint32(ofp.OfpPortNo_OFPP_ANY) && !FlowHasOutPort(flow, mod.OutPort) {
-		return false
-	}
-
-	//	Check out_group
-	if (mod.OutGroup&0x7fffffff) != uint32(ofp.OfpGroup_OFPG_ANY) && !FlowHasOutGroup(flow, mod.OutGroup) {
-		return false
-	}
-
-	//Priority is ignored
-
-	//Check match condition
-	//If the flow_mod match field is empty, that is a special case and indicates the flow entry matches
-	if (mod.Match == nil) || (mod.Match.OxmFields == nil) || (len(mod.Match.OxmFields) == 0) {
-		//If we got this far and the match is empty in the flow spec, than the flow matches
-		return true
-	} // TODO : implement the flow match analysis
-	return false
-
-}
-
-//FlowHasOutPort returns True if flow has a output command with the given out_port
-func FlowHasOutPort(flow *ofp.OfpFlowStats, outPort uint32) bool {
-	if flow == nil {
-		return false
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
-			if instruction.GetActions() == nil {
-				return false
-			}
-			for _, action := range instruction.GetActions().Actions {
-				if action.Type == ofp.OfpActionType_OFPAT_OUTPUT {
-					if (action.GetOutput() != nil) && (action.GetOutput().Port == outPort) {
-						return true
-					}
-				}
-
-			}
-		}
-	}
-	return false
-}
-
-//FlowHasOutGroup return True if flow has a output command with the given out_group
-func FlowHasOutGroup(flow *ofp.OfpFlowStats, groupID uint32) bool {
-	if flow == nil {
-		return false
-	}
-	for _, instruction := range flow.Instructions {
-		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
-			if instruction.GetActions() == nil {
-				return false
-			}
-			for _, action := range instruction.GetActions().Actions {
-				if action.Type == ofp.OfpActionType_OFPAT_GROUP {
-					if (action.GetGroup() != nil) && (action.GetGroup().GroupId == groupID) {
-						return true
-					}
-				}
-
-			}
-		}
-	}
-	return false
-}
-
-//FindGroup returns index of group if found, else returns -1
-func FindGroup(groups []*ofp.OfpGroupEntry, groupId uint32) int {
-	for idx, group := range groups {
-		if group.Desc.GroupId == groupId {
-			return idx
-		}
-	}
-	return -1
-}
-
-func FlowsDeleteByGroupId(flows []*ofp.OfpFlowStats, groupId uint32) (bool, []*ofp.OfpFlowStats) {
-	toKeep := make([]*ofp.OfpFlowStats, 0)
-
-	for _, f := range flows {
-		if !FlowHasOutGroup(f, groupId) {
-			toKeep = append(toKeep, f)
-		}
-	}
-	return len(toKeep) < len(flows), toKeep
-}
-
-func ToOfpOxmField(from []*ofp.OfpOxmOfbField) []*ofp.OfpOxmField {
-	matchFields := make([]*ofp.OfpOxmField, 0)
-	for _, val := range from {
-		matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
-	}
-	return matchFields
-}
-
-//IsMulticastIp returns true if the ip starts with the byte sequence of 1110;
-//false otherwise.
-func IsMulticastIp(ip uint32) bool {
-	return ip>>28 == 14
-}
-
-//ConvertToMulticastMacInt returns equivalent mac address of the given multicast ip address
-func ConvertToMulticastMacInt(ip uint32) uint64 {
-	//get last 23 bits of ip address by ip & 00000000011111111111111111111111
-	theLast23BitsOfIp := ip & 8388607
-	// perform OR with 0x1005E000000 to build mcast mac address
-	return 1101088686080 | uint64(theLast23BitsOfIp)
-}
-
-//ConvertToMulticastMacBytes returns equivalent mac address of the given multicast ip address
-func ConvertToMulticastMacBytes(ip uint32) []byte {
-	mac := ConvertToMulticastMacInt(ip)
-	var b bytes.Buffer
-	// catalyze (48 bits) in binary:111111110000000000000000000000000000000000000000
-	catalyze := uint64(280375465082880)
-	//convert each octet to decimal
-	for i := 0; i < 6; i++ {
-		if i != 0 {
-			catalyze = catalyze >> 8
-		}
-		octet := mac & catalyze
-		octetDecimal := octet >> uint8(40-i*8)
-		b.WriteByte(byte(octetDecimal))
-	}
-	return b.Bytes()
-}
-
-func GetMeterIdFromWriteMetadata(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
-	/*
-			  Write metadata instruction value (metadata) is 8 bytes:
-		    	MS 2 bytes: C Tag
-		    	Next 2 bytes: Technology Profile Id
-		    	Next 4 bytes: Port number (uni or nni) or MeterId
-		    	This is set in the ONOS OltPipeline as a write metadata instruction
-	*/
-	var meterID uint32 = 0
-	md := GetMetadataFromWriteMetadataAction(ctx, flow)
-	logger.Debugw(ctx, "found-metadata-for-egress/uni-port", log.Fields{"metadata": md})
-	if md != 0 {
-		meterID = uint32(md & 0xFFFFFFFF)
-		logger.Debugw(ctx, "found-meterID-in-write-metadata-action", log.Fields{"meterID": meterID})
-	}
-	return meterID
-}
-
-func SetMeterIdToFlow(flow *ofp.OfpFlowStats, meterId uint32) {
-	if flow != nil {
-		for _, instruction := range flow.Instructions {
-			if instruction.Type == uint32(METER_ACTION) {
-				if meterInst := instruction.GetMeter(); meterInst != nil {
-					meterInst.MeterId = meterId
-				}
-			}
-		}
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/client.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/client.go
deleted file mode 100644
index eee9631..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/client.go
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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 kafka
-
-import (
-	"context"
-	"time"
-
-	ca "github.com/opencord/voltha-protos/v4/go/inter_container"
-)
-
-const (
-	PartitionConsumer = iota
-	GroupCustomer     = iota
-)
-
-const (
-	OffsetNewest = -1
-	OffsetOldest = -2
-)
-
-const (
-	GroupIdKey = "groupId"
-	Offset     = "offset"
-)
-
-const (
-	DefaultKafkaAddress             = "127.0.0.1:9092"
-	DefaultGroupName                = "voltha"
-	DefaultSleepOnError             = 1
-	DefaultProducerFlushFrequency   = 10
-	DefaultProducerFlushMessages    = 10
-	DefaultProducerFlushMaxmessages = 100
-	DefaultProducerReturnSuccess    = true
-	DefaultProducerReturnErrors     = true
-	DefaultProducerRetryMax         = 3
-	DefaultProducerRetryBackoff     = time.Millisecond * 100
-	DefaultConsumerMaxwait          = 100
-	DefaultMaxProcessingTime        = 100
-	DefaultConsumerType             = PartitionConsumer
-	DefaultNumberPartitions         = 3
-	DefaultNumberReplicas           = 1
-	DefaultAutoCreateTopic          = false
-	DefaultMetadataMaxRetry         = 3
-	DefaultLivenessChannelInterval  = time.Second * 30
-)
-
-// MsgClient represents the set of APIs  a Kafka MsgClient must implement
-type Client interface {
-	Start(ctx context.Context) error
-	Stop(ctx context.Context)
-	CreateTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error
-	DeleteTopic(ctx context.Context, topic *Topic) error
-	Subscribe(ctx context.Context, topic *Topic, kvArgs ...*KVArg) (<-chan *ca.InterContainerMessage, error)
-	UnSubscribe(ctx context.Context, topic *Topic, ch <-chan *ca.InterContainerMessage) error
-	SubscribeForMetadata(context.Context, func(fromTopic string, timestamp time.Time))
-	Send(ctx context.Context, msg interface{}, topic *Topic, keys ...string) error
-	SendLiveness(ctx context.Context) error
-	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
-	EnableHealthinessChannel(ctx context.Context, enable bool) chan bool
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/common.go
deleted file mode 100644
index f4d7661..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 kafka
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/endpoint_manager.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/endpoint_manager.go
deleted file mode 100644
index 962b932..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/endpoint_manager.go
+++ /dev/null
@@ -1,352 +0,0 @@
-/*
- * Copyright 2020-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 kafka
-
-import (
-	"context"
-	"fmt"
-	"github.com/buraksezer/consistent"
-	"github.com/cespare/xxhash"
-	"github.com/golang/protobuf/proto"
-	"github.com/opencord/voltha-lib-go/v5/pkg/db"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"github.com/opencord/voltha-protos/v4/go/voltha"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-	"sync"
-)
-
-const (
-	// All the values below can be tuned to get optimal data distribution.  The numbers below seems to work well when
-	// supporting 1000-10000 devices and 1 - 20 replicas of a service
-
-	// Keys are distributed among partitions. Prime numbers are good to distribute keys uniformly.
-	DefaultPartitionCount = 1117
-
-	// Represents how many times a node is replicated on the consistent ring.
-	DefaultReplicationFactor = 117
-
-	// Load is used to calculate average load.
-	DefaultLoad = 1.1
-)
-
-type Endpoint string // Endpoint of a service instance.  When using kafka, this is the topic name of a service
-type ReplicaID int32 // The replication ID of a service instance
-
-type EndpointManager interface {
-
-	// GetEndpoint is called to get the endpoint to communicate with for a specific device and service type.  For
-	// now this will return the topic name
-	GetEndpoint(ctx context.Context, deviceID string, serviceType string) (Endpoint, error)
-
-	// IsDeviceOwnedByService is invoked when a specific service (service type + replicaNumber) is restarted and
-	// devices owned by that service need to be reconciled
-	IsDeviceOwnedByService(ctx context.Context, deviceID string, serviceType string, replicaNumber int32) (bool, error)
-
-	// GetReplicaAssignment returns the replica number of the service that owns the deviceID.  This is used by the
-	// test only
-	GetReplicaAssignment(ctx context.Context, deviceID string, serviceType string) (ReplicaID, error)
-}
-
-type service struct {
-	id             string // Id of the service.  The same id is used for all replicas
-	totalReplicas  int32
-	replicas       map[ReplicaID]Endpoint
-	consistentRing *consistent.Consistent
-}
-
-type endpointManager struct {
-	partitionCount           int
-	replicationFactor        int
-	load                     float64
-	backend                  *db.Backend
-	services                 map[string]*service
-	servicesLock             sync.RWMutex
-	deviceTypeServiceMap     map[string]string
-	deviceTypeServiceMapLock sync.RWMutex
-}
-
-type EndpointManagerOption func(*endpointManager)
-
-func PartitionCount(count int) EndpointManagerOption {
-	return func(args *endpointManager) {
-		args.partitionCount = count
-	}
-}
-
-func ReplicationFactor(replicas int) EndpointManagerOption {
-	return func(args *endpointManager) {
-		args.replicationFactor = replicas
-	}
-}
-
-func Load(load float64) EndpointManagerOption {
-	return func(args *endpointManager) {
-		args.load = load
-	}
-}
-
-func newEndpointManager(backend *db.Backend, opts ...EndpointManagerOption) EndpointManager {
-	tm := &endpointManager{
-		partitionCount:       DefaultPartitionCount,
-		replicationFactor:    DefaultReplicationFactor,
-		load:                 DefaultLoad,
-		backend:              backend,
-		services:             make(map[string]*service),
-		deviceTypeServiceMap: make(map[string]string),
-	}
-
-	for _, option := range opts {
-		option(tm)
-	}
-	return tm
-}
-
-func NewEndpointManager(backend *db.Backend, opts ...EndpointManagerOption) EndpointManager {
-	return newEndpointManager(backend, opts...)
-}
-
-func (ep *endpointManager) GetEndpoint(ctx context.Context, deviceID string, serviceType string) (Endpoint, error) {
-	logger.Debugw(ctx, "getting-endpoint", log.Fields{"device-id": deviceID, "service": serviceType})
-	owner, err := ep.getOwner(ctx, deviceID, serviceType)
-	if err != nil {
-		return "", err
-	}
-	m, ok := owner.(Member)
-	if !ok {
-		return "", status.Errorf(codes.Aborted, "invalid-member-%v", owner)
-	}
-	endpoint := m.getEndPoint()
-	if endpoint == "" {
-		return "", status.Errorf(codes.Unavailable, "endpoint-not-set-%s", serviceType)
-	}
-	logger.Debugw(ctx, "returning-endpoint", log.Fields{"device-id": deviceID, "service": serviceType, "endpoint": endpoint})
-	return endpoint, nil
-}
-
-func (ep *endpointManager) IsDeviceOwnedByService(ctx context.Context, deviceID string, serviceType string, replicaNumber int32) (bool, error) {
-	logger.Debugw(ctx, "device-ownership", log.Fields{"device-id": deviceID, "service": serviceType, "replica-number": replicaNumber})
-	owner, err := ep.getOwner(ctx, deviceID, serviceType)
-	if err != nil {
-		return false, nil
-	}
-	m, ok := owner.(Member)
-	if !ok {
-		return false, status.Errorf(codes.Aborted, "invalid-member-%v", owner)
-	}
-	return m.getReplica() == ReplicaID(replicaNumber), nil
-}
-
-func (ep *endpointManager) GetReplicaAssignment(ctx context.Context, deviceID string, serviceType string) (ReplicaID, error) {
-	owner, err := ep.getOwner(ctx, deviceID, serviceType)
-	if err != nil {
-		return 0, nil
-	}
-	m, ok := owner.(Member)
-	if !ok {
-		return 0, status.Errorf(codes.Aborted, "invalid-member-%v", owner)
-	}
-	return m.getReplica(), nil
-}
-
-func (ep *endpointManager) getOwner(ctx context.Context, deviceID string, serviceType string) (consistent.Member, error) {
-	serv, dType, err := ep.getServiceAndDeviceType(ctx, serviceType)
-	if err != nil {
-		return nil, err
-	}
-	key := ep.makeKey(deviceID, dType, serviceType)
-	return serv.consistentRing.LocateKey(key), nil
-}
-
-func (ep *endpointManager) getServiceAndDeviceType(ctx context.Context, serviceType string) (*service, string, error) {
-	// Check whether service exist
-	ep.servicesLock.RLock()
-	serv, serviceExist := ep.services[serviceType]
-	ep.servicesLock.RUnlock()
-
-	// Load the service and device types if needed
-	if !serviceExist || serv == nil || int(serv.totalReplicas) != len(serv.consistentRing.GetMembers()) {
-		if err := ep.loadServices(ctx); err != nil {
-			return nil, "", err
-		}
-
-		// Check whether the service exists now
-		ep.servicesLock.RLock()
-		serv, serviceExist = ep.services[serviceType]
-		ep.servicesLock.RUnlock()
-		if !serviceExist || serv == nil || int(serv.totalReplicas) != len(serv.consistentRing.GetMembers()) {
-			return nil, "", status.Errorf(codes.NotFound, "service-%s", serviceType)
-		}
-	}
-
-	ep.deviceTypeServiceMapLock.RLock()
-	defer ep.deviceTypeServiceMapLock.RUnlock()
-	for dType, sType := range ep.deviceTypeServiceMap {
-		if sType == serviceType {
-			return serv, dType, nil
-		}
-	}
-	return nil, "", status.Errorf(codes.NotFound, "service-%s", serviceType)
-}
-
-func (ep *endpointManager) getConsistentConfig() consistent.Config {
-	return consistent.Config{
-		PartitionCount:    ep.partitionCount,
-		ReplicationFactor: ep.replicationFactor,
-		Load:              ep.load,
-		Hasher:            hasher{},
-	}
-}
-
-// loadServices loads the services (adapters) and device types in memory. Because of the small size of the data and
-// the data format in the dB being binary protobuf then it is better to load all the data if inconsistency is detected,
-// instead of watching for updates in the dB and acting on it.
-func (ep *endpointManager) loadServices(ctx context.Context) error {
-	ep.servicesLock.Lock()
-	defer ep.servicesLock.Unlock()
-	ep.deviceTypeServiceMapLock.Lock()
-	defer ep.deviceTypeServiceMapLock.Unlock()
-
-	if ep.backend == nil {
-		return status.Error(codes.Aborted, "backend-not-set")
-	}
-	ep.services = make(map[string]*service)
-	ep.deviceTypeServiceMap = make(map[string]string)
-
-	// Load the adapters
-	blobs, err := ep.backend.List(log.WithSpanFromContext(context.Background(), ctx), "adapters")
-	if err != nil {
-		return err
-	}
-
-	// Data is marshalled as proto bytes in the data store
-	for _, blob := range blobs {
-		data := blob.Value.([]byte)
-		adapter := &voltha.Adapter{}
-		if err := proto.Unmarshal(data, adapter); err != nil {
-			return err
-		}
-		// A valid adapter should have the vendorID set
-		if adapter.Vendor != "" {
-			if _, ok := ep.services[adapter.Type]; !ok {
-				ep.services[adapter.Type] = &service{
-					id:             adapter.Type,
-					totalReplicas:  adapter.TotalReplicas,
-					replicas:       make(map[ReplicaID]Endpoint),
-					consistentRing: consistent.New(nil, ep.getConsistentConfig()),
-				}
-
-			}
-			currentReplica := ReplicaID(adapter.CurrentReplica)
-			endpoint := Endpoint(adapter.Endpoint)
-			ep.services[adapter.Type].replicas[currentReplica] = endpoint
-			ep.services[adapter.Type].consistentRing.Add(newMember(adapter.Id, adapter.Type, adapter.Vendor, endpoint, adapter.Version, currentReplica))
-		}
-	}
-	// Load the device types
-	blobs, err = ep.backend.List(log.WithSpanFromContext(context.Background(), ctx), "device_types")
-	if err != nil {
-		return err
-	}
-	for _, blob := range blobs {
-		data := blob.Value.([]byte)
-		deviceType := &voltha.DeviceType{}
-		if err := proto.Unmarshal(data, deviceType); err != nil {
-			return err
-		}
-		if _, ok := ep.deviceTypeServiceMap[deviceType.Id]; !ok {
-			ep.deviceTypeServiceMap[deviceType.Id] = deviceType.Adapter
-		}
-	}
-
-	// Log the loaded data in debug mode to facilitate trouble shooting
-	if logger.V(log.DebugLevel) {
-		for key, val := range ep.services {
-			members := val.consistentRing.GetMembers()
-			logger.Debugw(ctx, "service", log.Fields{"service": key, "expected-replica": val.totalReplicas, "replicas": len(val.consistentRing.GetMembers())})
-			for _, m := range members {
-				n := m.(Member)
-				logger.Debugw(ctx, "service-loaded", log.Fields{"serviceId": n.getID(), "serviceType": n.getServiceType(), "replica": n.getReplica(), "endpoint": n.getEndPoint()})
-			}
-		}
-		logger.Debugw(ctx, "device-types-loaded", log.Fields{"device-types": ep.deviceTypeServiceMap})
-	}
-	return nil
-}
-
-// makeKey creates the string that the hash function uses to create the hash
-func (ep *endpointManager) makeKey(deviceID string, deviceType string, serviceType string) []byte {
-	return []byte(fmt.Sprintf("%s_%s_%s", serviceType, deviceType, deviceID))
-}
-
-// The consistent package requires a hasher function
-type hasher struct{}
-
-// Sum64 provides the hasher function.  Based upon numerous testing scenarios, the xxhash package seems to provide the
-// best distribution compare to other hash packages
-func (h hasher) Sum64(data []byte) uint64 {
-	return xxhash.Sum64(data)
-}
-
-// Member represents a member on the consistent ring
-type Member interface {
-	String() string
-	getReplica() ReplicaID
-	getEndPoint() Endpoint
-	getID() string
-	getServiceType() string
-}
-
-// member implements the Member interface
-type member struct {
-	id          string
-	serviceType string
-	vendor      string
-	version     string
-	replica     ReplicaID
-	endpoint    Endpoint
-}
-
-func newMember(ID string, serviceType string, vendor string, endPoint Endpoint, version string, replica ReplicaID) Member {
-	return &member{
-		id:          ID,
-		serviceType: serviceType,
-		vendor:      vendor,
-		version:     version,
-		replica:     replica,
-		endpoint:    endPoint,
-	}
-}
-
-func (m *member) String() string {
-	return string(m.endpoint)
-}
-
-func (m *member) getReplica() ReplicaID {
-	return m.replica
-}
-
-func (m *member) getEndPoint() Endpoint {
-	return m.endpoint
-}
-
-func (m *member) getID() string {
-	return m.id
-}
-
-func (m *member) getServiceType() string {
-	return m.serviceType
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/kafka_inter_container_library.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/kafka_inter_container_library.go
deleted file mode 100644
index 120ed45..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/kafka_inter_container_library.go
+++ /dev/null
@@ -1,1097 +0,0 @@
-/*
- * 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 kafka
-
-import (
-	"context"
-	"encoding/json"
-	"errors"
-	"fmt"
-	"reflect"
-	"strings"
-	"sync"
-	"time"
-
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/status"
-
-	"github.com/golang/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/golang/protobuf/ptypes/any"
-	"github.com/google/uuid"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-	"github.com/opentracing/opentracing-go"
-)
-
-const (
-	DefaultMaxRetries     = 3
-	DefaultRequestTimeout = 60000 // 60000 milliseconds - to handle a wider latency range
-)
-
-const (
-	TransactionKey = "transactionID"
-	FromTopic      = "fromTopic"
-)
-
-var ErrorTransactionNotAcquired = errors.New("transaction-not-acquired")
-var ErrorTransactionInvalidId = errors.New("transaction-invalid-id")
-
-// requestHandlerChannel represents an interface associated with a channel.  Whenever, an event is
-// obtained from that channel, this interface is invoked.   This is used to handle
-// async requests into the Core via the kafka messaging bus
-type requestHandlerChannel struct {
-	requesthandlerInterface interface{}
-	ch                      <-chan *ic.InterContainerMessage
-}
-
-// transactionChannel represents a combination of a topic and a channel onto which a response received
-// on the kafka bus will be sent to
-type transactionChannel struct {
-	topic *Topic
-	ch    chan *ic.InterContainerMessage
-}
-
-type InterContainerProxy interface {
-	Start(ctx context.Context) error
-	Stop(ctx context.Context)
-	GetDefaultTopic() *Topic
-	InvokeRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic, waitForResponse bool, key string, kvArgs ...*KVArg) (bool, *any.Any)
-	InvokeAsyncRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic, waitForResponse bool, key string, kvArgs ...*KVArg) chan *RpcResponse
-	SubscribeWithRequestHandlerInterface(ctx context.Context, topic Topic, handler interface{}) error
-	SubscribeWithDefaultRequestHandler(ctx context.Context, topic Topic, initialOffset int64) error
-	UnSubscribeFromRequestHandler(ctx context.Context, topic Topic) error
-	DeleteTopic(ctx context.Context, topic Topic) error
-	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
-	SendLiveness(ctx context.Context) error
-}
-
-// interContainerProxy represents the messaging proxy
-type interContainerProxy struct {
-	kafkaAddress                   string
-	defaultTopic                   *Topic
-	defaultRequestHandlerInterface interface{}
-	kafkaClient                    Client
-	doneCh                         chan struct{}
-	doneOnce                       sync.Once
-
-	// This map is used to map a topic to an interface and channel.   When a request is received
-	// on that channel (registered to the topic) then that interface is invoked.
-	topicToRequestHandlerChannelMap   map[string]*requestHandlerChannel
-	lockTopicRequestHandlerChannelMap sync.RWMutex
-
-	// This map is used to map a channel to a response topic.   This channel handles all responses on that
-	// channel for that topic and forward them to the appropriate consumers channel, using the
-	// transactionIdToChannelMap.
-	topicToResponseChannelMap   map[string]<-chan *ic.InterContainerMessage
-	lockTopicResponseChannelMap sync.RWMutex
-
-	// This map is used to map a transaction to a consumers channel.  This is used whenever a request has been
-	// sent out and we are waiting for a response.
-	transactionIdToChannelMap     map[string]*transactionChannel
-	lockTransactionIdToChannelMap sync.RWMutex
-}
-
-type InterContainerProxyOption func(*interContainerProxy)
-
-func InterContainerAddress(address string) InterContainerProxyOption {
-	return func(args *interContainerProxy) {
-		args.kafkaAddress = address
-	}
-}
-
-func DefaultTopic(topic *Topic) InterContainerProxyOption {
-	return func(args *interContainerProxy) {
-		args.defaultTopic = topic
-	}
-}
-
-func RequestHandlerInterface(handler interface{}) InterContainerProxyOption {
-	return func(args *interContainerProxy) {
-		args.defaultRequestHandlerInterface = handler
-	}
-}
-
-func MsgClient(client Client) InterContainerProxyOption {
-	return func(args *interContainerProxy) {
-		args.kafkaClient = client
-	}
-}
-
-func newInterContainerProxy(opts ...InterContainerProxyOption) *interContainerProxy {
-	proxy := &interContainerProxy{
-		kafkaAddress: DefaultKafkaAddress,
-		doneCh:       make(chan struct{}),
-	}
-
-	for _, option := range opts {
-		option(proxy)
-	}
-
-	return proxy
-}
-
-func NewInterContainerProxy(opts ...InterContainerProxyOption) InterContainerProxy {
-	return newInterContainerProxy(opts...)
-}
-
-func (kp *interContainerProxy) Start(ctx context.Context) error {
-	logger.Info(ctx, "Starting-Proxy")
-
-	// Kafka MsgClient should already have been created.  If not, output fatal error
-	if kp.kafkaClient == nil {
-		logger.Fatal(ctx, "kafka-client-not-set")
-	}
-
-	// Start the kafka client
-	if err := kp.kafkaClient.Start(ctx); err != nil {
-		logger.Errorw(ctx, "Cannot-create-kafka-proxy", log.Fields{"error": err})
-		return err
-	}
-
-	// Create the topic to response channel map
-	kp.topicToResponseChannelMap = make(map[string]<-chan *ic.InterContainerMessage)
-	//
-	// Create the transactionId to Channel Map
-	kp.transactionIdToChannelMap = make(map[string]*transactionChannel)
-
-	// Create the topic to request channel map
-	kp.topicToRequestHandlerChannelMap = make(map[string]*requestHandlerChannel)
-
-	return nil
-}
-
-func (kp *interContainerProxy) Stop(ctx context.Context) {
-	logger.Info(ctx, "stopping-intercontainer-proxy")
-	kp.doneOnce.Do(func() { close(kp.doneCh) })
-	// TODO : Perform cleanup
-	kp.kafkaClient.Stop(ctx)
-	err := kp.deleteAllTopicRequestHandlerChannelMap(ctx)
-	if err != nil {
-		logger.Errorw(ctx, "failed-delete-all-topic-request-handler-channel-map", log.Fields{"error": err})
-	}
-	err = kp.deleteAllTopicResponseChannelMap(ctx)
-	if err != nil {
-		logger.Errorw(ctx, "failed-delete-all-topic-response-channel-map", log.Fields{"error": err})
-	}
-	kp.deleteAllTransactionIdToChannelMap(ctx)
-}
-
-func (kp *interContainerProxy) GetDefaultTopic() *Topic {
-	return kp.defaultTopic
-}
-
-// InvokeAsyncRPC is used to make an RPC request asynchronously
-func (kp *interContainerProxy) InvokeAsyncRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
-	waitForResponse bool, key string, kvArgs ...*KVArg) chan *RpcResponse {
-
-	spanArg, span, ctx := kp.embedSpanAsArg(ctx, rpc, !waitForResponse)
-	if spanArg != nil {
-		kvArgs = append(kvArgs, &spanArg[0])
-	}
-
-	defer span.Finish()
-
-	logger.Debugw(ctx, "InvokeAsyncRPC", log.Fields{"rpc": rpc, "key": key, "kvArgs": kvArgs})
-
-	//	If a replyToTopic is provided then we use it, otherwise just use the  default toTopic.  The replyToTopic is
-	// typically the device ID.
-	responseTopic := replyToTopic
-	if responseTopic == nil {
-		responseTopic = kp.GetDefaultTopic()
-	}
-
-	chnl := make(chan *RpcResponse)
-
-	go func() {
-
-		// once we're done,
-		// close the response channel
-		defer close(chnl)
-
-		var err error
-		var protoRequest *ic.InterContainerMessage
-
-		// Encode the request
-		protoRequest, err = encodeRequest(ctx, rpc, toTopic, responseTopic, key, kvArgs...)
-		if err != nil {
-			logger.Warnw(ctx, "cannot-format-request", log.Fields{"rpc": rpc, "error": err})
-			log.MarkSpanError(ctx, errors.New("cannot-format-request"))
-			chnl <- NewResponse(RpcFormattingError, err, nil)
-			return
-		}
-
-		// Subscribe for response, if needed, before sending request
-		var ch <-chan *ic.InterContainerMessage
-		if ch, err = kp.subscribeForResponse(ctx, *responseTopic, protoRequest.Header.Id); err != nil {
-			logger.Errorw(ctx, "failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
-			log.MarkSpanError(ctx, errors.New("failed-to-subscribe-for-response"))
-			chnl <- NewResponse(RpcTransportError, err, nil)
-			return
-		}
-
-		// Send request - if the topic is formatted with a device Id then we will send the request using a
-		// specific key, hence ensuring a single partition is used to publish the request.  This ensures that the
-		// subscriber on that topic will receive the request in the order it was sent.  The key used is the deviceId.
-		logger.Debugw(ctx, "sending-msg", log.Fields{"rpc": rpc, "toTopic": toTopic, "replyTopic": responseTopic, "key": key, "xId": protoRequest.Header.Id})
-
-		// if the message is not sent on kafka publish an event an close the channel
-		if err = kp.kafkaClient.Send(ctx, protoRequest, toTopic, key); err != nil {
-			chnl <- NewResponse(RpcTransportError, err, nil)
-			return
-		}
-
-		// if the client is not waiting for a response send the ack and close the channel
-		chnl <- NewResponse(RpcSent, nil, nil)
-		if !waitForResponse {
-			return
-		}
-
-		defer func() {
-			// Remove the subscription for a response on return
-			if err := kp.unSubscribeForResponse(ctx, protoRequest.Header.Id); err != nil {
-				logger.Warnw(ctx, "invoke-async-rpc-unsubscriber-for-response-failed", log.Fields{"err": err})
-			}
-		}()
-
-		// Wait for response as well as timeout or cancellation
-		select {
-		case msg, ok := <-ch:
-			if !ok {
-				logger.Warnw(ctx, "channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
-				log.MarkSpanError(ctx, errors.New("channel-closed"))
-				chnl <- NewResponse(RpcTransportError, status.Error(codes.Aborted, "channel closed"), nil)
-			}
-			logger.Debugw(ctx, "received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
-			if responseBody, err := decodeResponse(ctx, msg); err != nil {
-				chnl <- NewResponse(RpcReply, err, nil)
-			} else {
-				if responseBody.Success {
-					chnl <- NewResponse(RpcReply, nil, responseBody.Result)
-				} else {
-					// response body contains an error
-					unpackErr := &ic.Error{}
-					if err := ptypes.UnmarshalAny(responseBody.Result, unpackErr); err != nil {
-						chnl <- NewResponse(RpcReply, err, nil)
-					} else {
-						chnl <- NewResponse(RpcReply, status.Error(codes.Internal, unpackErr.Reason), nil)
-					}
-				}
-			}
-		case <-ctx.Done():
-			logger.Errorw(ctx, "context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
-			log.MarkSpanError(ctx, errors.New("context-cancelled"))
-			err := status.Error(codes.DeadlineExceeded, ctx.Err().Error())
-			chnl <- NewResponse(RpcTimeout, err, nil)
-		case <-kp.doneCh:
-			chnl <- NewResponse(RpcSystemClosing, nil, nil)
-			logger.Warnw(ctx, "received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
-		}
-	}()
-	return chnl
-}
-
-// Method to extract Open-tracing Span from Context and serialize it for transport over Kafka embedded as a additional argument.
-// Additional argument is injected using key as "span" and value as Span marshalled into a byte slice
-//
-// The span name is automatically constructed using the RPC name with following convention (<rpc-name> represents name of invoked method):
-// - RPC invoked in Sync manner (WaitForResponse=true) : kafka-rpc-<rpc-name>
-// - RPC invoked in Async manner (WaitForResponse=false) : kafka-async-rpc-<rpc-name>
-// - Inter Adapter RPC invoked in Sync manner (WaitForResponse=true) : kafka-inter-adapter-rpc-<rpc-name>
-// - Inter Adapter RPC invoked in Async manner (WaitForResponse=false) : kafka-inter-adapter-async-rpc-<rpc-name>
-func (kp *interContainerProxy) embedSpanAsArg(ctx context.Context, rpc string, isAsync bool) ([]KVArg, opentracing.Span, context.Context) {
-	var err error
-	var newCtx context.Context
-	var spanToInject opentracing.Span
-
-	if !log.GetGlobalLFM().GetLogCorrelationStatus() && !log.GetGlobalLFM().GetTracePublishingStatus() {
-		// if both log correlation and trace publishing is disable do not generate the span
-		logger.Debugw(ctx, "not-embedding-span-in-KVArg-", log.Fields{"rpc": rpc,
-			"log-correlation-status": log.GetGlobalLFM().GetLogCorrelationStatus(), "trace-publishing-status": log.GetGlobalLFM().GetTracePublishingStatus()})
-		return nil, opentracing.GlobalTracer().StartSpan(rpc), ctx
-	}
-
-	var spanName strings.Builder
-	spanName.WriteString("kafka-")
-
-	// In case of inter adapter message, use Msg Type for constructing RPC name
-	if rpc == "process_inter_adapter_message" {
-		if msgType, ok := ctx.Value("inter-adapter-msg-type").(ic.InterAdapterMessageType_Types); ok {
-			spanName.WriteString("inter-adapter-")
-			rpc = msgType.String()
-		}
-	}
-
-	if isAsync {
-		spanName.WriteString("async-rpc-")
-	} else {
-		spanName.WriteString("rpc-")
-	}
-	spanName.WriteString(rpc)
-
-	if isAsync {
-		spanToInject, newCtx = log.CreateAsyncSpan(ctx, spanName.String())
-	} else {
-		spanToInject, newCtx = log.CreateChildSpan(ctx, spanName.String())
-	}
-
-	spanToInject.SetBaggageItem("rpc-span-name", spanName.String())
-
-	textMapCarrier := opentracing.TextMapCarrier(make(map[string]string))
-	if err = opentracing.GlobalTracer().Inject(spanToInject.Context(), opentracing.TextMap, textMapCarrier); err != nil {
-		logger.Warnw(ctx, "unable-to-serialize-span-to-textmap", log.Fields{"span": spanToInject, "error": err})
-		return nil, spanToInject, newCtx
-	}
-
-	var textMapJson []byte
-	if textMapJson, err = json.Marshal(textMapCarrier); err != nil {
-		logger.Warnw(ctx, "unable-to-marshal-textmap-to-json-string", log.Fields{"textMap": textMapCarrier, "error": err})
-		return nil, spanToInject, newCtx
-	}
-
-	spanArg := make([]KVArg, 1)
-	spanArg[0] = KVArg{Key: "span", Value: &ic.StrType{Val: string(textMapJson)}}
-	return spanArg, spanToInject, newCtx
-}
-
-// InvokeRPC is used to send a request to a given topic
-func (kp *interContainerProxy) InvokeRPC(ctx context.Context, rpc string, toTopic *Topic, replyToTopic *Topic,
-	waitForResponse bool, key string, kvArgs ...*KVArg) (bool, *any.Any) {
-
-	spanArg, span, ctx := kp.embedSpanAsArg(ctx, rpc, false)
-	if spanArg != nil {
-		kvArgs = append(kvArgs, &spanArg[0])
-	}
-
-	defer span.Finish()
-
-	logger.Debugw(ctx, "InvokeRPC", log.Fields{"rpc": rpc, "key": key, "kvArgs": kvArgs})
-
-	//	If a replyToTopic is provided then we use it, otherwise just use the  default toTopic.  The replyToTopic is
-	// typically the device ID.
-	responseTopic := replyToTopic
-	if responseTopic == nil {
-		responseTopic = kp.defaultTopic
-	}
-
-	// Encode the request
-	protoRequest, err := encodeRequest(ctx, rpc, toTopic, responseTopic, key, kvArgs...)
-	if err != nil {
-		logger.Warnw(ctx, "cannot-format-request", log.Fields{"rpc": rpc, "error": err})
-		log.MarkSpanError(ctx, errors.New("cannot-format-request"))
-		return false, nil
-	}
-
-	// Subscribe for response, if needed, before sending request
-	var ch <-chan *ic.InterContainerMessage
-	if waitForResponse {
-		var err error
-		if ch, err = kp.subscribeForResponse(ctx, *responseTopic, protoRequest.Header.Id); err != nil {
-			logger.Errorw(ctx, "failed-to-subscribe-for-response", log.Fields{"error": err, "toTopic": toTopic.Name})
-		}
-	}
-
-	// Send request - if the topic is formatted with a device Id then we will send the request using a
-	// specific key, hence ensuring a single partition is used to publish the request.  This ensures that the
-	// subscriber on that topic will receive the request in the order it was sent.  The key used is the deviceId.
-	//key := GetDeviceIdFromTopic(*toTopic)
-	logger.Debugw(ctx, "sending-msg", log.Fields{"rpc": rpc, "toTopic": toTopic, "replyTopic": responseTopic, "key": key, "xId": protoRequest.Header.Id})
-	go func() {
-		if err := kp.kafkaClient.Send(ctx, protoRequest, toTopic, key); err != nil {
-			log.MarkSpanError(ctx, errors.New("send-failed"))
-			logger.Errorw(ctx, "send-failed", log.Fields{
-				"topic": toTopic,
-				"key":   key,
-				"error": err})
-		}
-	}()
-
-	if waitForResponse {
-		// Create a child context based on the parent context, if any
-		var cancel context.CancelFunc
-		childCtx := context.Background()
-		if ctx == nil {
-			ctx, cancel = context.WithTimeout(context.Background(), DefaultRequestTimeout*time.Millisecond)
-		} else {
-			childCtx, cancel = context.WithTimeout(ctx, DefaultRequestTimeout*time.Millisecond)
-		}
-		defer cancel()
-
-		// Wait for response as well as timeout or cancellation
-		// Remove the subscription for a response on return
-		defer func() {
-			if err := kp.unSubscribeForResponse(ctx, protoRequest.Header.Id); err != nil {
-				logger.Errorw(ctx, "response-unsubscribe-failed", log.Fields{
-					"id":    protoRequest.Header.Id,
-					"error": err})
-			}
-		}()
-		select {
-		case msg, ok := <-ch:
-			if !ok {
-				logger.Warnw(ctx, "channel-closed", log.Fields{"rpc": rpc, "replyTopic": replyToTopic.Name})
-				log.MarkSpanError(ctx, errors.New("channel-closed"))
-				protoError := &ic.Error{Reason: "channel-closed"}
-				var marshalledArg *any.Any
-				if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
-					return false, nil // Should never happen
-				}
-				return false, marshalledArg
-			}
-			logger.Debugw(ctx, "received-response", log.Fields{"rpc": rpc, "msgHeader": msg.Header})
-			var responseBody *ic.InterContainerResponseBody
-			var err error
-			if responseBody, err = decodeResponse(ctx, msg); err != nil {
-				logger.Errorw(ctx, "decode-response-error", log.Fields{"error": err})
-				// FIXME we should return something
-			}
-			return responseBody.Success, responseBody.Result
-		case <-ctx.Done():
-			logger.Debugw(ctx, "context-cancelled", log.Fields{"rpc": rpc, "ctx": ctx.Err()})
-			log.MarkSpanError(ctx, errors.New("context-cancelled"))
-			//	 pack the error as proto any type
-			protoError := &ic.Error{Reason: ctx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
-
-			var marshalledArg *any.Any
-			if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
-				return false, nil // Should never happen
-			}
-			return false, marshalledArg
-		case <-childCtx.Done():
-			logger.Debugw(ctx, "context-cancelled", log.Fields{"rpc": rpc, "ctx": childCtx.Err()})
-			log.MarkSpanError(ctx, errors.New("context-cancelled"))
-			//	 pack the error as proto any type
-			protoError := &ic.Error{Reason: childCtx.Err().Error(), Code: ic.ErrorCode_DEADLINE_EXCEEDED}
-
-			var marshalledArg *any.Any
-			if marshalledArg, err = ptypes.MarshalAny(protoError); err != nil {
-				return false, nil // Should never happen
-			}
-			return false, marshalledArg
-		case <-kp.doneCh:
-			logger.Infow(ctx, "received-exit-signal", log.Fields{"toTopic": toTopic.Name, "rpc": rpc})
-			return true, nil
-		}
-	}
-	return true, nil
-}
-
-// SubscribeWithRequestHandlerInterface allows a caller to assign a target object to be invoked automatically
-// when a message is received on a given topic
-func (kp *interContainerProxy) SubscribeWithRequestHandlerInterface(ctx context.Context, topic Topic, handler interface{}) error {
-
-	// Subscribe to receive messages for that topic
-	var ch <-chan *ic.InterContainerMessage
-	var err error
-	if ch, err = kp.kafkaClient.Subscribe(ctx, &topic); err != nil {
-		//if ch, err = kp.Subscribe(topic); err != nil {
-		logger.Errorw(ctx, "failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
-		return err
-	}
-
-	kp.defaultRequestHandlerInterface = handler
-	kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: handler, ch: ch})
-	// Launch a go routine to receive and process kafka messages
-	go kp.waitForMessages(ctx, ch, topic, handler)
-
-	return nil
-}
-
-// SubscribeWithDefaultRequestHandler allows a caller to add a topic to an existing target object to be invoked automatically
-// when a message is received on a given topic.  So far there is only 1 target registered per microservice
-func (kp *interContainerProxy) SubscribeWithDefaultRequestHandler(ctx context.Context, topic Topic, initialOffset int64) error {
-	// Subscribe to receive messages for that topic
-	var ch <-chan *ic.InterContainerMessage
-	var err error
-	if ch, err = kp.kafkaClient.Subscribe(ctx, &topic, &KVArg{Key: Offset, Value: initialOffset}); err != nil {
-		logger.Errorw(ctx, "failed-to-subscribe", log.Fields{"error": err, "topic": topic.Name})
-		return err
-	}
-	kp.addToTopicRequestHandlerChannelMap(topic.Name, &requestHandlerChannel{requesthandlerInterface: kp.defaultRequestHandlerInterface, ch: ch})
-
-	// Launch a go routine to receive and process kafka messages
-	go kp.waitForMessages(ctx, ch, topic, kp.defaultRequestHandlerInterface)
-
-	return nil
-}
-
-func (kp *interContainerProxy) UnSubscribeFromRequestHandler(ctx context.Context, topic Topic) error {
-	return kp.deleteFromTopicRequestHandlerChannelMap(ctx, topic.Name)
-}
-
-func (kp *interContainerProxy) deleteFromTopicResponseChannelMap(ctx context.Context, topic string) error {
-	kp.lockTopicResponseChannelMap.Lock()
-	defer kp.lockTopicResponseChannelMap.Unlock()
-	if _, exist := kp.topicToResponseChannelMap[topic]; exist {
-		// Unsubscribe to this topic first - this will close the subscribed channel
-		var err error
-		if err = kp.kafkaClient.UnSubscribe(ctx, &Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
-			logger.Errorw(ctx, "unsubscribing-error", log.Fields{"topic": topic})
-		}
-		delete(kp.topicToResponseChannelMap, topic)
-		return err
-	} else {
-		return fmt.Errorf("%s-Topic-not-found", topic)
-	}
-}
-
-// nolint: unused
-func (kp *interContainerProxy) deleteAllTopicResponseChannelMap(ctx context.Context) error {
-	logger.Debug(ctx, "delete-all-topic-response-channel")
-	kp.lockTopicResponseChannelMap.Lock()
-	defer kp.lockTopicResponseChannelMap.Unlock()
-	var unsubscribeFailTopics []string
-	for topic := range kp.topicToResponseChannelMap {
-		// Unsubscribe to this topic first - this will close the subscribed channel
-		if err := kp.kafkaClient.UnSubscribe(ctx, &Topic{Name: topic}, kp.topicToResponseChannelMap[topic]); err != nil {
-			unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
-			logger.Errorw(ctx, "unsubscribing-error", log.Fields{"topic": topic, "error": err})
-			// Do not return. Continue to try to unsubscribe to other topics.
-		} else {
-			// Only delete from channel map if successfully unsubscribed.
-			delete(kp.topicToResponseChannelMap, topic)
-		}
-	}
-	if len(unsubscribeFailTopics) > 0 {
-		return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
-	}
-	return nil
-}
-
-func (kp *interContainerProxy) addToTopicRequestHandlerChannelMap(topic string, arg *requestHandlerChannel) {
-	kp.lockTopicRequestHandlerChannelMap.Lock()
-	defer kp.lockTopicRequestHandlerChannelMap.Unlock()
-	if _, exist := kp.topicToRequestHandlerChannelMap[topic]; !exist {
-		kp.topicToRequestHandlerChannelMap[topic] = arg
-	}
-}
-
-func (kp *interContainerProxy) deleteFromTopicRequestHandlerChannelMap(ctx context.Context, topic string) error {
-	kp.lockTopicRequestHandlerChannelMap.Lock()
-	defer kp.lockTopicRequestHandlerChannelMap.Unlock()
-	if _, exist := kp.topicToRequestHandlerChannelMap[topic]; exist {
-		// Close the kafka client client first by unsubscribing to this topic
-		if err := kp.kafkaClient.UnSubscribe(ctx, &Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
-			return err
-		}
-		delete(kp.topicToRequestHandlerChannelMap, topic)
-		return nil
-	} else {
-		return fmt.Errorf("%s-Topic-not-found", topic)
-	}
-}
-
-// nolint: unused
-func (kp *interContainerProxy) deleteAllTopicRequestHandlerChannelMap(ctx context.Context) error {
-	logger.Debug(ctx, "delete-all-topic-request-channel")
-	kp.lockTopicRequestHandlerChannelMap.Lock()
-	defer kp.lockTopicRequestHandlerChannelMap.Unlock()
-	var unsubscribeFailTopics []string
-	for topic := range kp.topicToRequestHandlerChannelMap {
-		// Close the kafka client client first by unsubscribing to this topic
-		if err := kp.kafkaClient.UnSubscribe(ctx, &Topic{Name: topic}, kp.topicToRequestHandlerChannelMap[topic].ch); err != nil {
-			unsubscribeFailTopics = append(unsubscribeFailTopics, topic)
-			logger.Errorw(ctx, "unsubscribing-error", log.Fields{"topic": topic, "error": err})
-			// Do not return. Continue to try to unsubscribe to other topics.
-		} else {
-			// Only delete from channel map if successfully unsubscribed.
-			delete(kp.topicToRequestHandlerChannelMap, topic)
-		}
-	}
-	if len(unsubscribeFailTopics) > 0 {
-		return fmt.Errorf("unsubscribe-errors: %v", unsubscribeFailTopics)
-	}
-	return nil
-}
-
-func (kp *interContainerProxy) addToTransactionIdToChannelMap(id string, topic *Topic, arg chan *ic.InterContainerMessage) {
-	kp.lockTransactionIdToChannelMap.Lock()
-	defer kp.lockTransactionIdToChannelMap.Unlock()
-	if _, exist := kp.transactionIdToChannelMap[id]; !exist {
-		kp.transactionIdToChannelMap[id] = &transactionChannel{topic: topic, ch: arg}
-	}
-}
-
-func (kp *interContainerProxy) deleteFromTransactionIdToChannelMap(id string) {
-	kp.lockTransactionIdToChannelMap.Lock()
-	defer kp.lockTransactionIdToChannelMap.Unlock()
-	if transChannel, exist := kp.transactionIdToChannelMap[id]; exist {
-		// Close the channel first
-		close(transChannel.ch)
-		delete(kp.transactionIdToChannelMap, id)
-	}
-}
-
-func (kp *interContainerProxy) deleteTopicTransactionIdToChannelMap(id string) {
-	kp.lockTransactionIdToChannelMap.Lock()
-	defer kp.lockTransactionIdToChannelMap.Unlock()
-	for key, value := range kp.transactionIdToChannelMap {
-		if value.topic.Name == id {
-			close(value.ch)
-			delete(kp.transactionIdToChannelMap, key)
-		}
-	}
-}
-
-// nolint: unused
-func (kp *interContainerProxy) deleteAllTransactionIdToChannelMap(ctx context.Context) {
-	logger.Debug(ctx, "delete-all-transaction-id-channel-map")
-	kp.lockTransactionIdToChannelMap.Lock()
-	defer kp.lockTransactionIdToChannelMap.Unlock()
-	for key, value := range kp.transactionIdToChannelMap {
-		close(value.ch)
-		delete(kp.transactionIdToChannelMap, key)
-	}
-}
-
-func (kp *interContainerProxy) DeleteTopic(ctx context.Context, topic Topic) error {
-	// If we have any consumers on that topic we need to close them
-	if err := kp.deleteFromTopicResponseChannelMap(ctx, topic.Name); err != nil {
-		logger.Errorw(ctx, "delete-from-topic-responsechannelmap-failed", log.Fields{"error": err})
-	}
-	if err := kp.deleteFromTopicRequestHandlerChannelMap(ctx, topic.Name); err != nil {
-		logger.Errorw(ctx, "delete-from-topic-requesthandlerchannelmap-failed", log.Fields{"error": err})
-	}
-	kp.deleteTopicTransactionIdToChannelMap(topic.Name)
-
-	return kp.kafkaClient.DeleteTopic(ctx, &topic)
-}
-
-func encodeReturnedValue(ctx context.Context, returnedVal interface{}) (*any.Any, error) {
-	// Encode the response argument - needs to be a proto message
-	if returnedVal == nil {
-		return nil, nil
-	}
-	protoValue, ok := returnedVal.(proto.Message)
-	if !ok {
-		logger.Warnw(ctx, "response-value-not-proto-message", log.Fields{"error": ok, "returnVal": returnedVal})
-		err := errors.New("response-value-not-proto-message")
-		return nil, err
-	}
-
-	// Marshal the returned value, if any
-	var marshalledReturnedVal *any.Any
-	var err error
-	if marshalledReturnedVal, err = ptypes.MarshalAny(protoValue); err != nil {
-		logger.Warnw(ctx, "cannot-marshal-returned-val", log.Fields{"error": err})
-		return nil, err
-	}
-	return marshalledReturnedVal, nil
-}
-
-func encodeDefaultFailedResponse(ctx context.Context, request *ic.InterContainerMessage) *ic.InterContainerMessage {
-	responseHeader := &ic.Header{
-		Id:        request.Header.Id,
-		Type:      ic.MessageType_RESPONSE,
-		FromTopic: request.Header.ToTopic,
-		ToTopic:   request.Header.FromTopic,
-		Timestamp: ptypes.TimestampNow(),
-	}
-	responseBody := &ic.InterContainerResponseBody{
-		Success: false,
-		Result:  nil,
-	}
-	var marshalledResponseBody *any.Any
-	var err error
-	// Error should never happen here
-	if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
-		logger.Warnw(ctx, "cannot-marshal-failed-response-body", log.Fields{"error": err})
-	}
-
-	return &ic.InterContainerMessage{
-		Header: responseHeader,
-		Body:   marshalledResponseBody,
-	}
-
-}
-
-//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
-//or an error on failure
-func encodeResponse(ctx context.Context, request *ic.InterContainerMessage, success bool, returnedValues ...interface{}) (*ic.InterContainerMessage, error) {
-	//logger.Debugw(ctx, "encodeResponse", log.Fields{"success": success, "returnedValues": returnedValues})
-	responseHeader := &ic.Header{
-		Id:        request.Header.Id,
-		Type:      ic.MessageType_RESPONSE,
-		FromTopic: request.Header.ToTopic,
-		ToTopic:   request.Header.FromTopic,
-		KeyTopic:  request.Header.KeyTopic,
-		Timestamp: ptypes.TimestampNow(),
-	}
-
-	// Go over all returned values
-	var marshalledReturnedVal *any.Any
-	var err error
-
-	// for now we support only 1 returned value - (excluding the error)
-	if len(returnedValues) > 0 {
-		if marshalledReturnedVal, err = encodeReturnedValue(ctx, returnedValues[0]); err != nil {
-			logger.Warnw(ctx, "cannot-marshal-response-body", log.Fields{"error": err})
-		}
-	}
-
-	responseBody := &ic.InterContainerResponseBody{
-		Success: success,
-		Result:  marshalledReturnedVal,
-	}
-
-	// Marshal the response body
-	var marshalledResponseBody *any.Any
-	if marshalledResponseBody, err = ptypes.MarshalAny(responseBody); err != nil {
-		logger.Warnw(ctx, "cannot-marshal-response-body", log.Fields{"error": err})
-		return nil, err
-	}
-
-	return &ic.InterContainerMessage{
-		Header: responseHeader,
-		Body:   marshalledResponseBody,
-	}, nil
-}
-
-func CallFuncByName(ctx context.Context, myClass interface{}, funcName string, params ...interface{}) (out []reflect.Value, err error) {
-	myClassValue := reflect.ValueOf(myClass)
-	// Capitalize the first letter in the funcName to workaround the first capital letters required to
-	// invoke a function from a different package
-	funcName = strings.Title(funcName)
-	m := myClassValue.MethodByName(funcName)
-	if !m.IsValid() {
-		return make([]reflect.Value, 0), fmt.Errorf("method-not-found \"%s\"", funcName)
-	}
-	in := make([]reflect.Value, len(params)+1)
-	in[0] = reflect.ValueOf(ctx)
-	for i, param := range params {
-		in[i+1] = reflect.ValueOf(param)
-	}
-	out = m.Call(in)
-	return
-}
-
-func (kp *interContainerProxy) addTransactionId(ctx context.Context, transactionId string, currentArgs []*ic.Argument) []*ic.Argument {
-	arg := &KVArg{
-		Key:   TransactionKey,
-		Value: &ic.StrType{Val: transactionId},
-	}
-
-	var marshalledArg *any.Any
-	var err error
-	if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: transactionId}); err != nil {
-		logger.Warnw(ctx, "cannot-add-transactionId", log.Fields{"error": err})
-		return currentArgs
-	}
-	protoArg := &ic.Argument{
-		Key:   arg.Key,
-		Value: marshalledArg,
-	}
-	return append(currentArgs, protoArg)
-}
-
-func (kp *interContainerProxy) addFromTopic(ctx context.Context, fromTopic string, currentArgs []*ic.Argument) []*ic.Argument {
-	var marshalledArg *any.Any
-	var err error
-	if marshalledArg, err = ptypes.MarshalAny(&ic.StrType{Val: fromTopic}); err != nil {
-		logger.Warnw(ctx, "cannot-add-transactionId", log.Fields{"error": err})
-		return currentArgs
-	}
-	protoArg := &ic.Argument{
-		Key:   FromTopic,
-		Value: marshalledArg,
-	}
-	return append(currentArgs, protoArg)
-}
-
-// Method to extract the Span embedded in Kafka RPC request on the receiver side. If span is found embedded in the KV args (with key as "span"),
-// it is de-serialized and injected into the Context to be carried forward by the RPC request processor thread.
-// If no span is found embedded, even then a span is created with name as "kafka-rpc-<rpc-name>" to enrich the Context for RPC calls coming
-// from components currently not sending the span (e.g. openonu adapter)
-func (kp *interContainerProxy) enrichContextWithSpan(ctx context.Context, rpcName string, args []*ic.Argument) (opentracing.Span, context.Context) {
-
-	for _, arg := range args {
-		if arg.Key == "span" {
-			var err error
-			var textMapString ic.StrType
-			if err = ptypes.UnmarshalAny(arg.Value, &textMapString); err != nil {
-				logger.Debug(ctx, "unable-to-unmarshal-kvarg-to-textmap-string", log.Fields{"value": arg.Value})
-				break
-			}
-
-			spanTextMap := make(map[string]string)
-			if err = json.Unmarshal([]byte(textMapString.Val), &spanTextMap); err != nil {
-				logger.Debug(ctx, "unable-to-unmarshal-textmap-from-json-string", log.Fields{"textMapString": textMapString, "error": err})
-				break
-			}
-
-			var spanContext opentracing.SpanContext
-			if spanContext, err = opentracing.GlobalTracer().Extract(opentracing.TextMap, opentracing.TextMapCarrier(spanTextMap)); err != nil {
-				logger.Debug(ctx, "unable-to-deserialize-textmap-to-span", log.Fields{"textMap": spanTextMap, "error": err})
-				break
-			}
-
-			var receivedRpcName string
-			extractBaggage := func(k, v string) bool {
-				if k == "rpc-span-name" {
-					receivedRpcName = v
-					return false
-				}
-				return true
-			}
-
-			spanContext.ForeachBaggageItem(extractBaggage)
-
-			return opentracing.StartSpanFromContext(ctx, receivedRpcName, opentracing.FollowsFrom(spanContext))
-		}
-	}
-
-	// Create new Child Span with rpc as name if no span details were received in kafka arguments
-	var spanName strings.Builder
-	spanName.WriteString("kafka-")
-
-	// In case of inter adapter message, use Msg Type for constructing RPC name
-	if rpcName == "process_inter_adapter_message" {
-		for _, arg := range args {
-			if arg.Key == "msg" {
-				iamsg := ic.InterAdapterMessage{}
-				if err := ptypes.UnmarshalAny(arg.Value, &iamsg); err == nil {
-					spanName.WriteString("inter-adapter-")
-					rpcName = iamsg.Header.Type.String()
-				}
-			}
-		}
-	}
-
-	spanName.WriteString("rpc-")
-	spanName.WriteString(rpcName)
-
-	return opentracing.StartSpanFromContext(ctx, spanName.String())
-}
-
-func (kp *interContainerProxy) handleMessage(ctx context.Context, msg *ic.InterContainerMessage, targetInterface interface{}) {
-
-	// First extract the header to know whether this is a request - responses are handled by a different handler
-	if msg.Header.Type == ic.MessageType_REQUEST {
-		var out []reflect.Value
-		var err error
-
-		// Get the request body
-		requestBody := &ic.InterContainerRequestBody{}
-		if err = ptypes.UnmarshalAny(msg.Body, requestBody); err != nil {
-			logger.Warnw(ctx, "cannot-unmarshal-request", log.Fields{"error": err})
-		} else {
-			logger.Debugw(ctx, "received-request", log.Fields{"rpc": requestBody.Rpc, "header": msg.Header, "args": requestBody.Args})
-			span, ctx := kp.enrichContextWithSpan(ctx, requestBody.Rpc, requestBody.Args)
-			defer span.Finish()
-
-			// let the callee unpack the arguments as its the only one that knows the real proto type
-			// Augment the requestBody with the message Id as it will be used in scenarios where cores
-			// are set in pairs and competing
-			requestBody.Args = kp.addTransactionId(ctx, msg.Header.Id, requestBody.Args)
-
-			// Augment the requestBody with the From topic name as it will be used in scenarios where a container
-			// needs to send an unsollicited message to the currently requested container
-			requestBody.Args = kp.addFromTopic(ctx, msg.Header.FromTopic, requestBody.Args)
-
-			out, err = CallFuncByName(ctx, targetInterface, requestBody.Rpc, requestBody.Args)
-			if err != nil {
-				logger.Warn(ctx, err)
-			}
-		}
-		// Response required?
-		if requestBody.ResponseRequired {
-			// If we already have an error before then just return that
-			var returnError *ic.Error
-			var returnedValues []interface{}
-			var success bool
-			if err != nil {
-				returnError = &ic.Error{Reason: err.Error()}
-				returnedValues = make([]interface{}, 1)
-				returnedValues[0] = returnError
-			} else {
-				returnedValues = make([]interface{}, 0)
-				// Check for errors first
-				lastIndex := len(out) - 1
-				if out[lastIndex].Interface() != nil { // Error
-					if retError, ok := out[lastIndex].Interface().(error); ok {
-						if retError.Error() == ErrorTransactionNotAcquired.Error() {
-							logger.Debugw(ctx, "Ignoring request", log.Fields{"error": retError, "txId": msg.Header.Id})
-							return // Ignore - process is in competing mode and ignored transaction
-						}
-						returnError = &ic.Error{Reason: retError.Error()}
-						returnedValues = append(returnedValues, returnError)
-					} else { // Should never happen
-						returnError = &ic.Error{Reason: "incorrect-error-returns"}
-						returnedValues = append(returnedValues, returnError)
-					}
-				} else if len(out) == 2 && reflect.ValueOf(out[0].Interface()).IsValid() && reflect.ValueOf(out[0].Interface()).IsNil() {
-					logger.Warnw(ctx, "Unexpected response of (nil,nil)", log.Fields{"txId": msg.Header.Id})
-					return // Ignore - should not happen
-				} else { // Non-error case
-					success = true
-					for idx, val := range out {
-						//logger.Debugw(ctx, "returned-api-response-loop", log.Fields{"idx": idx, "val": val.Interface()})
-						if idx != lastIndex {
-							returnedValues = append(returnedValues, val.Interface())
-						}
-					}
-				}
-			}
-
-			var icm *ic.InterContainerMessage
-			if icm, err = encodeResponse(ctx, msg, success, returnedValues...); err != nil {
-				logger.Warnw(ctx, "error-encoding-response-returning-failure-result", log.Fields{"error": err})
-				icm = encodeDefaultFailedResponse(ctx, msg)
-			}
-			// To preserve ordering of messages, all messages to a given topic are sent to the same partition
-			// by providing a message key.   The key is encoded in the topic name.  If the deviceId is not
-			// present then the key will be empty, hence all messages for a given topic will be sent to all
-			// partitions.
-			replyTopic := &Topic{Name: msg.Header.FromTopic}
-			key := msg.Header.KeyTopic
-			logger.Debugw(ctx, "sending-response-to-kafka", log.Fields{"rpc": requestBody.Rpc, "header": icm.Header, "key": key})
-			// TODO: handle error response.
-			go func() {
-				if err := kp.kafkaClient.Send(ctx, icm, replyTopic, key); err != nil {
-					logger.Errorw(ctx, "send-reply-failed", log.Fields{
-						"topic": replyTopic,
-						"key":   key,
-						"error": err})
-				}
-			}()
-		}
-	} else if msg.Header.Type == ic.MessageType_RESPONSE {
-		logger.Debugw(ctx, "response-received", log.Fields{"msg-header": msg.Header})
-		go kp.dispatchResponse(ctx, msg)
-	} else {
-		logger.Warnw(ctx, "unsupported-message-received", log.Fields{"msg-header": msg.Header})
-	}
-}
-
-func (kp *interContainerProxy) waitForMessages(ctx context.Context, ch <-chan *ic.InterContainerMessage, topic Topic, targetInterface interface{}) {
-	//	Wait for messages
-	for msg := range ch {
-		//logger.Debugw(ctx, "request-received", log.Fields{"msg": msg, "topic": topic.Name, "target": targetInterface})
-		go kp.handleMessage(context.Background(), msg, targetInterface)
-	}
-}
-
-func (kp *interContainerProxy) dispatchResponse(ctx context.Context, msg *ic.InterContainerMessage) {
-	kp.lockTransactionIdToChannelMap.RLock()
-	defer kp.lockTransactionIdToChannelMap.RUnlock()
-	if _, exist := kp.transactionIdToChannelMap[msg.Header.Id]; !exist {
-		logger.Debugw(ctx, "no-waiting-channel", log.Fields{"transaction": msg.Header.Id})
-		return
-	}
-	kp.transactionIdToChannelMap[msg.Header.Id].ch <- msg
-}
-
-// subscribeForResponse allows a caller to subscribe to a given topic when waiting for a response.
-// This method is built to prevent all subscribers to receive all messages as is the case of the Subscribe
-// API. There is one response channel waiting for kafka messages before dispatching the message to the
-// corresponding waiting channel
-func (kp *interContainerProxy) subscribeForResponse(ctx context.Context, topic Topic, trnsId string) (chan *ic.InterContainerMessage, error) {
-	logger.Debugw(ctx, "subscribeForResponse", log.Fields{"topic": topic.Name, "trnsid": trnsId})
-
-	// Create a specific channel for this consumers.  We cannot use the channel from the kafkaclient as it will
-	// broadcast any message for this topic to all channels waiting on it.
-	// Set channel size to 1 to prevent deadlock, see VOL-2708
-	ch := make(chan *ic.InterContainerMessage, 1)
-	kp.addToTransactionIdToChannelMap(trnsId, &topic, ch)
-
-	return ch, nil
-}
-
-func (kp *interContainerProxy) unSubscribeForResponse(ctx context.Context, trnsId string) error {
-	logger.Debugw(ctx, "unsubscribe-for-response", log.Fields{"trnsId": trnsId})
-	kp.deleteFromTransactionIdToChannelMap(trnsId)
-	return nil
-}
-
-func (kp *interContainerProxy) EnableLivenessChannel(ctx context.Context, enable bool) chan bool {
-	return kp.kafkaClient.EnableLivenessChannel(ctx, enable)
-}
-
-func (kp *interContainerProxy) EnableHealthinessChannel(ctx context.Context, enable bool) chan bool {
-	return kp.kafkaClient.EnableHealthinessChannel(ctx, enable)
-}
-
-func (kp *interContainerProxy) SendLiveness(ctx context.Context) error {
-	return kp.kafkaClient.SendLiveness(ctx)
-}
-
-//formatRequest formats a request to send over kafka and returns an InterContainerMessage message on success
-//or an error on failure
-func encodeRequest(ctx context.Context, rpc string, toTopic *Topic, replyTopic *Topic, key string, kvArgs ...*KVArg) (*ic.InterContainerMessage, error) {
-	requestHeader := &ic.Header{
-		Id:        uuid.New().String(),
-		Type:      ic.MessageType_REQUEST,
-		FromTopic: replyTopic.Name,
-		ToTopic:   toTopic.Name,
-		KeyTopic:  key,
-		Timestamp: ptypes.TimestampNow(),
-	}
-	requestBody := &ic.InterContainerRequestBody{
-		Rpc:              rpc,
-		ResponseRequired: true,
-		ReplyToTopic:     replyTopic.Name,
-	}
-
-	for _, arg := range kvArgs {
-		if arg == nil {
-			// In case the caller sends an array with empty args
-			continue
-		}
-		var marshalledArg *any.Any
-		var err error
-		// ascertain the value interface type is a proto.Message
-		protoValue, ok := arg.Value.(proto.Message)
-		if !ok {
-			logger.Warnw(ctx, "argument-value-not-proto-message", log.Fields{"error": ok, "Value": arg.Value})
-			err := errors.New("argument-value-not-proto-message")
-			return nil, err
-		}
-		if marshalledArg, err = ptypes.MarshalAny(protoValue); err != nil {
-			logger.Warnw(ctx, "cannot-marshal-request", log.Fields{"error": err})
-			return nil, err
-		}
-		protoArg := &ic.Argument{
-			Key:   arg.Key,
-			Value: marshalledArg,
-		}
-		requestBody.Args = append(requestBody.Args, protoArg)
-	}
-
-	var marshalledData *any.Any
-	var err error
-	if marshalledData, err = ptypes.MarshalAny(requestBody); err != nil {
-		logger.Warnw(ctx, "cannot-marshal-request", log.Fields{"error": err})
-		return nil, err
-	}
-	request := &ic.InterContainerMessage{
-		Header: requestHeader,
-		Body:   marshalledData,
-	}
-	return request, nil
-}
-
-func decodeResponse(ctx context.Context, response *ic.InterContainerMessage) (*ic.InterContainerResponseBody, error) {
-	//	Extract the message body
-	responseBody := ic.InterContainerResponseBody{}
-	if err := ptypes.UnmarshalAny(response.Body, &responseBody); err != nil {
-		logger.Warnw(ctx, "cannot-unmarshal-response", log.Fields{"error": err})
-		return nil, err
-	}
-	//logger.Debugw(ctx, "response-decoded-successfully", log.Fields{"response-status": &responseBody.Success})
-
-	return &responseBody, nil
-
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/sarama_client.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/sarama_client.go
deleted file mode 100644
index 3273470..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/sarama_client.go
+++ /dev/null
@@ -1,1157 +0,0 @@
-/*
- * 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 kafka
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"strings"
-	"sync"
-	"time"
-
-	"github.com/Shopify/sarama"
-	scc "github.com/bsm/sarama-cluster"
-	"github.com/eapache/go-resiliency/breaker"
-	"github.com/golang/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/google/uuid"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ic "github.com/opencord/voltha-protos/v4/go/inter_container"
-)
-
-// consumerChannels represents one or more consumers listening on a kafka topic.  Once a message is received on that
-// topic, the consumer(s) broadcasts the message to all the listening channels.   The consumer can be a partition
-//consumer or a group consumer
-type consumerChannels struct {
-	consumers []interface{}
-	channels  []chan *ic.InterContainerMessage
-}
-
-// static check to ensure SaramaClient implements Client
-var _ Client = &SaramaClient{}
-
-// SaramaClient represents the messaging proxy
-type SaramaClient struct {
-	cAdmin                        sarama.ClusterAdmin
-	KafkaAddress                  string
-	producer                      sarama.AsyncProducer
-	consumer                      sarama.Consumer
-	groupConsumers                map[string]*scc.Consumer
-	lockOfGroupConsumers          sync.RWMutex
-	consumerGroupPrefix           string
-	consumerType                  int
-	consumerGroupName             string
-	producerFlushFrequency        int
-	producerFlushMessages         int
-	producerFlushMaxmessages      int
-	producerRetryMax              int
-	producerRetryBackOff          time.Duration
-	producerReturnSuccess         bool
-	producerReturnErrors          bool
-	consumerMaxwait               int
-	maxProcessingTime             int
-	numPartitions                 int
-	numReplicas                   int
-	autoCreateTopic               bool
-	doneCh                        chan int
-	metadataCallback              func(fromTopic string, timestamp time.Time)
-	topicToConsumerChannelMap     map[string]*consumerChannels
-	lockTopicToConsumerChannelMap sync.RWMutex
-	topicLockMap                  map[string]*sync.RWMutex
-	lockOfTopicLockMap            sync.RWMutex
-	metadataMaxRetry              int
-	alive                         bool
-	livenessMutex                 sync.Mutex
-	liveness                      chan bool
-	livenessChannelInterval       time.Duration
-	lastLivenessTime              time.Time
-	started                       bool
-	healthinessMutex              sync.Mutex
-	healthy                       bool
-	healthiness                   chan bool
-}
-
-type SaramaClientOption func(*SaramaClient)
-
-func Address(address string) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.KafkaAddress = address
-	}
-}
-
-func ConsumerGroupPrefix(prefix string) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.consumerGroupPrefix = prefix
-	}
-}
-
-func ConsumerGroupName(name string) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.consumerGroupName = name
-	}
-}
-
-func ConsumerType(consumer int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.consumerType = consumer
-	}
-}
-
-func ProducerFlushFrequency(frequency int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerFlushFrequency = frequency
-	}
-}
-
-func ProducerFlushMessages(num int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerFlushMessages = num
-	}
-}
-
-func ProducerFlushMaxMessages(num int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerFlushMaxmessages = num
-	}
-}
-
-func ProducerMaxRetries(num int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerRetryMax = num
-	}
-}
-
-func ProducerRetryBackoff(duration time.Duration) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerRetryBackOff = duration
-	}
-}
-
-func ProducerReturnOnErrors(opt bool) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerReturnErrors = opt
-	}
-}
-
-func ProducerReturnOnSuccess(opt bool) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.producerReturnSuccess = opt
-	}
-}
-
-func ConsumerMaxWait(wait int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.consumerMaxwait = wait
-	}
-}
-
-func MaxProcessingTime(pTime int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.maxProcessingTime = pTime
-	}
-}
-
-func NumPartitions(number int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.numPartitions = number
-	}
-}
-
-func NumReplicas(number int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.numReplicas = number
-	}
-}
-
-func AutoCreateTopic(opt bool) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.autoCreateTopic = opt
-	}
-}
-
-func MetadatMaxRetries(retry int) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.metadataMaxRetry = retry
-	}
-}
-
-func LivenessChannelInterval(opt time.Duration) SaramaClientOption {
-	return func(args *SaramaClient) {
-		args.livenessChannelInterval = opt
-	}
-}
-
-func NewSaramaClient(opts ...SaramaClientOption) *SaramaClient {
-	client := &SaramaClient{
-		KafkaAddress: DefaultKafkaAddress,
-	}
-	client.consumerType = DefaultConsumerType
-	client.producerFlushFrequency = DefaultProducerFlushFrequency
-	client.producerFlushMessages = DefaultProducerFlushMessages
-	client.producerFlushMaxmessages = DefaultProducerFlushMaxmessages
-	client.producerReturnErrors = DefaultProducerReturnErrors
-	client.producerReturnSuccess = DefaultProducerReturnSuccess
-	client.producerRetryMax = DefaultProducerRetryMax
-	client.producerRetryBackOff = DefaultProducerRetryBackoff
-	client.consumerMaxwait = DefaultConsumerMaxwait
-	client.maxProcessingTime = DefaultMaxProcessingTime
-	client.numPartitions = DefaultNumberPartitions
-	client.numReplicas = DefaultNumberReplicas
-	client.autoCreateTopic = DefaultAutoCreateTopic
-	client.metadataMaxRetry = DefaultMetadataMaxRetry
-	client.livenessChannelInterval = DefaultLivenessChannelInterval
-
-	for _, option := range opts {
-		option(client)
-	}
-
-	client.groupConsumers = make(map[string]*scc.Consumer)
-
-	client.lockTopicToConsumerChannelMap = sync.RWMutex{}
-	client.topicLockMap = make(map[string]*sync.RWMutex)
-	client.lockOfTopicLockMap = sync.RWMutex{}
-	client.lockOfGroupConsumers = sync.RWMutex{}
-
-	// healthy and alive until proven otherwise
-	client.alive = true
-	client.healthy = true
-
-	return client
-}
-
-func (sc *SaramaClient) Start(ctx context.Context) error {
-	logger.Info(ctx, "Starting-kafka-sarama-client")
-
-	// Create the Done channel
-	sc.doneCh = make(chan int, 1)
-
-	var err error
-
-	// Add a cleanup in case of failure to startup
-	defer func() {
-		if err != nil {
-			sc.Stop(ctx)
-		}
-	}()
-
-	// Create the Cluster Admin
-	if err = sc.createClusterAdmin(ctx); err != nil {
-		logger.Errorw(ctx, "Cannot-create-cluster-admin", log.Fields{"error": err})
-		return err
-	}
-
-	// Create the Publisher
-	if err := sc.createPublisher(ctx); err != nil {
-		logger.Errorw(ctx, "Cannot-create-kafka-publisher", log.Fields{"error": err})
-		return err
-	}
-
-	if sc.consumerType == DefaultConsumerType {
-		// Create the master consumers
-		if err := sc.createConsumer(ctx); err != nil {
-			logger.Errorw(ctx, "Cannot-create-kafka-consumers", log.Fields{"error": err})
-			return err
-		}
-	}
-
-	// Create the topic to consumers/channel map
-	sc.topicToConsumerChannelMap = make(map[string]*consumerChannels)
-
-	logger.Info(ctx, "kafka-sarama-client-started")
-
-	sc.started = true
-
-	return nil
-}
-
-func (sc *SaramaClient) Stop(ctx context.Context) {
-	logger.Info(ctx, "stopping-sarama-client")
-
-	sc.started = false
-
-	//Send a message over the done channel to close all long running routines
-	sc.doneCh <- 1
-
-	if sc.producer != nil {
-		if err := sc.producer.Close(); err != nil {
-			logger.Errorw(ctx, "closing-producer-failed", log.Fields{"error": err})
-		}
-	}
-
-	if sc.consumer != nil {
-		if err := sc.consumer.Close(); err != nil {
-			logger.Errorw(ctx, "closing-partition-consumer-failed", log.Fields{"error": err})
-		}
-	}
-
-	for key, val := range sc.groupConsumers {
-		logger.Debugw(ctx, "closing-group-consumer", log.Fields{"topic": key})
-		if err := val.Close(); err != nil {
-			logger.Errorw(ctx, "closing-group-consumer-failed", log.Fields{"error": err, "topic": key})
-		}
-	}
-
-	if sc.cAdmin != nil {
-		if err := sc.cAdmin.Close(); err != nil {
-			logger.Errorw(ctx, "closing-cluster-admin-failed", log.Fields{"error": err})
-		}
-	}
-
-	//TODO: Clear the consumers map
-	//sc.clearConsumerChannelMap()
-
-	logger.Info(ctx, "sarama-client-stopped")
-}
-
-//createTopic is an internal function to create a topic on the Kafka Broker. No locking is required as
-// the invoking function must hold the lock
-func (sc *SaramaClient) createTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error {
-	// Set the topic details
-	topicDetail := &sarama.TopicDetail{}
-	topicDetail.NumPartitions = int32(numPartition)
-	topicDetail.ReplicationFactor = int16(repFactor)
-	topicDetail.ConfigEntries = make(map[string]*string)
-	topicDetails := make(map[string]*sarama.TopicDetail)
-	topicDetails[topic.Name] = topicDetail
-
-	if err := sc.cAdmin.CreateTopic(topic.Name, topicDetail, false); err != nil {
-		if err == sarama.ErrTopicAlreadyExists {
-			//	Not an error
-			logger.Debugw(ctx, "topic-already-exist", log.Fields{"topic": topic.Name})
-			return nil
-		}
-		logger.Errorw(ctx, "create-topic-failure", log.Fields{"error": err})
-		return err
-	}
-	// TODO: Wait until the topic has been created.  No API is available in the Sarama clusterAdmin to
-	// do so.
-	logger.Debugw(ctx, "topic-created", log.Fields{"topic": topic, "numPartition": numPartition, "replicationFactor": repFactor})
-	return nil
-}
-
-//CreateTopic is a public API to create a topic on the Kafka Broker.  It uses a lock on a specific topic to
-// ensure no two go routines are performing operations on the same topic
-func (sc *SaramaClient) CreateTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error {
-	sc.lockTopic(topic)
-	defer sc.unLockTopic(topic)
-
-	return sc.createTopic(ctx, topic, numPartition, repFactor)
-}
-
-//DeleteTopic removes a topic from the kafka Broker
-func (sc *SaramaClient) DeleteTopic(ctx context.Context, topic *Topic) error {
-	sc.lockTopic(topic)
-	defer sc.unLockTopic(topic)
-
-	// Remove the topic from the broker
-	if err := sc.cAdmin.DeleteTopic(topic.Name); err != nil {
-		if err == sarama.ErrUnknownTopicOrPartition {
-			//	Not an error as does not exist
-			logger.Debugw(ctx, "topic-not-exist", log.Fields{"topic": topic.Name})
-			return nil
-		}
-		logger.Errorw(ctx, "delete-topic-failed", log.Fields{"topic": topic, "error": err})
-		return err
-	}
-
-	// Clear the topic from the consumer channel.  This will also close any consumers listening on that topic.
-	if err := sc.clearTopicFromConsumerChannelMap(ctx, *topic); err != nil {
-		logger.Errorw(ctx, "failure-clearing-channels", log.Fields{"topic": topic, "error": err})
-		return err
-	}
-	return nil
-}
-
-// Subscribe registers a caller to a topic. It returns a channel that the caller can use to receive
-// messages from that topic
-func (sc *SaramaClient) Subscribe(ctx context.Context, topic *Topic, kvArgs ...*KVArg) (<-chan *ic.InterContainerMessage, error) {
-	sc.lockTopic(topic)
-	defer sc.unLockTopic(topic)
-
-	logger.Debugw(ctx, "subscribe", log.Fields{"topic": topic.Name})
-
-	// If a consumers already exist for that topic then resuse it
-	if consumerCh := sc.getConsumerChannel(topic); consumerCh != nil {
-		logger.Debugw(ctx, "topic-already-subscribed", log.Fields{"topic": topic.Name})
-		// Create a channel specific for that consumers and add it to the consumers channel map
-		ch := make(chan *ic.InterContainerMessage)
-		sc.addChannelToConsumerChannelMap(ctx, topic, ch)
-		return ch, nil
-	}
-
-	// Register for the topic and set it up
-	var consumerListeningChannel chan *ic.InterContainerMessage
-	var err error
-
-	// Use the consumerType option to figure out the type of consumer to launch
-	if sc.consumerType == PartitionConsumer {
-		if sc.autoCreateTopic {
-			if err = sc.createTopic(ctx, topic, sc.numPartitions, sc.numReplicas); err != nil {
-				logger.Errorw(ctx, "create-topic-failure", log.Fields{"error": err, "topic": topic.Name})
-				return nil, err
-			}
-		}
-		if consumerListeningChannel, err = sc.setupPartitionConsumerChannel(ctx, topic, getOffset(kvArgs...)); err != nil {
-			logger.Warnw(ctx, "create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name})
-			return nil, err
-		}
-	} else if sc.consumerType == GroupCustomer {
-		// TODO: create topic if auto create is on.  There is an issue with the sarama cluster library that
-		// does not consume from a precreated topic in some scenarios
-		//if sc.autoCreateTopic {
-		//	if err = sc.createTopic(topic, sc.numPartitions, sc.numReplicas); err != nil {
-		//		logger.Errorw(ctx, "create-topic-failure", logger.Fields{"error": err, "topic": topic.Name})
-		//		return nil, err
-		//	}
-		//}
-		//groupId := sc.consumerGroupName
-		groupId := getGroupId(kvArgs...)
-		// Include the group prefix
-		if groupId != "" {
-			groupId = sc.consumerGroupPrefix + groupId
-		} else {
-			// Need to use a unique group Id per topic
-			groupId = sc.consumerGroupPrefix + topic.Name
-		}
-		if consumerListeningChannel, err = sc.setupGroupConsumerChannel(ctx, topic, groupId, getOffset(kvArgs...)); err != nil {
-			logger.Warnw(ctx, "create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
-			return nil, err
-		}
-
-	} else {
-		logger.Warnw(ctx, "unknown-consumer-type", log.Fields{"consumer-type": sc.consumerType})
-		return nil, errors.New("unknown-consumer-type")
-	}
-
-	return consumerListeningChannel, nil
-}
-
-//UnSubscribe unsubscribe a consumer from a given topic
-func (sc *SaramaClient) UnSubscribe(ctx context.Context, topic *Topic, ch <-chan *ic.InterContainerMessage) error {
-	sc.lockTopic(topic)
-	defer sc.unLockTopic(topic)
-
-	logger.Debugw(ctx, "unsubscribing-channel-from-topic", log.Fields{"topic": topic.Name})
-	var err error
-	if err = sc.removeChannelFromConsumerChannelMap(ctx, *topic, ch); err != nil {
-		logger.Errorw(ctx, "failed-removing-channel", log.Fields{"error": err})
-	}
-	if err = sc.deleteFromGroupConsumers(ctx, topic.Name); err != nil {
-		logger.Errorw(ctx, "failed-deleting-group-consumer", log.Fields{"error": err})
-	}
-	return err
-}
-
-func (sc *SaramaClient) SubscribeForMetadata(ctx context.Context, callback func(fromTopic string, timestamp time.Time)) {
-	sc.metadataCallback = callback
-}
-
-func (sc *SaramaClient) updateLiveness(ctx context.Context, alive bool) {
-	// Post a consistent stream of liveness data to the channel,
-	// so that in a live state, the core does not timeout and
-	// send a forced liveness message. Production of liveness
-	// events to the channel is rate-limited by livenessChannelInterval.
-	sc.livenessMutex.Lock()
-	defer sc.livenessMutex.Unlock()
-	if sc.liveness != nil {
-		if sc.alive != alive {
-			logger.Info(ctx, "update-liveness-channel-because-change")
-			sc.liveness <- alive
-			sc.lastLivenessTime = time.Now()
-		} else if time.Since(sc.lastLivenessTime) > sc.livenessChannelInterval {
-			logger.Info(ctx, "update-liveness-channel-because-interval")
-			sc.liveness <- alive
-			sc.lastLivenessTime = time.Now()
-		}
-	}
-
-	// Only emit a log message when the state changes
-	if sc.alive != alive {
-		logger.Info(ctx, "set-client-alive", log.Fields{"alive": alive})
-		sc.alive = alive
-	}
-}
-
-// Once unhealthy, we never go back
-func (sc *SaramaClient) setUnhealthy(ctx context.Context) {
-	sc.healthy = false
-	sc.healthinessMutex.Lock()
-	defer sc.healthinessMutex.Unlock()
-	if sc.healthiness != nil {
-		logger.Infow(ctx, "set-client-unhealthy", log.Fields{"healthy": sc.healthy})
-		sc.healthiness <- sc.healthy
-	}
-}
-
-func (sc *SaramaClient) isLivenessError(ctx context.Context, err error) bool {
-	// Sarama producers and consumers encapsulate the error inside
-	// a ProducerError or ConsumerError struct.
-	if prodError, ok := err.(*sarama.ProducerError); ok {
-		err = prodError.Err
-	} else if consumerError, ok := err.(*sarama.ConsumerError); ok {
-		err = consumerError.Err
-	}
-
-	// Sarama-Cluster will compose the error into a ClusterError struct,
-	// which we can't do a compare by reference. To handle that, we the
-	// best we can do is compare the error strings.
-
-	switch err.Error() {
-	case context.DeadlineExceeded.Error():
-		logger.Info(ctx, "is-liveness-error-timeout")
-		return true
-	case sarama.ErrOutOfBrokers.Error(): // "Kafka: client has run out of available brokers"
-		logger.Info(ctx, "is-liveness-error-no-brokers")
-		return true
-	case sarama.ErrShuttingDown.Error(): // "Kafka: message received by producer in process of shutting down"
-		logger.Info(ctx, "is-liveness-error-shutting-down")
-		return true
-	case sarama.ErrControllerNotAvailable.Error(): // "Kafka: controller is not available"
-		logger.Info(ctx, "is-liveness-error-not-available")
-		return true
-	case breaker.ErrBreakerOpen.Error(): // "circuit breaker is open"
-		logger.Info(ctx, "is-liveness-error-circuit-breaker-open")
-		return true
-	}
-
-	if strings.HasSuffix(err.Error(), "connection refused") { // "dial tcp 10.244.1.176:9092: connect: connection refused"
-		logger.Info(ctx, "is-liveness-error-connection-refused")
-		return true
-	}
-
-	if strings.HasSuffix(err.Error(), "i/o timeout") { // "dial tcp 10.244.1.176:9092: i/o timeout"
-		logger.Info(ctx, "is-liveness-error-io-timeout")
-		return true
-	}
-
-	// Other errors shouldn't trigger a loss of liveness
-
-	logger.Infow(ctx, "is-liveness-error-ignored", log.Fields{"err": err})
-
-	return false
-}
-
-// send formats and sends the request onto the kafka messaging bus.
-func (sc *SaramaClient) Send(ctx context.Context, msg interface{}, topic *Topic, keys ...string) error {
-
-	// Assert message is a proto message
-	var protoMsg proto.Message
-	var ok bool
-	// ascertain the value interface type is a proto.Message
-	if protoMsg, ok = msg.(proto.Message); !ok {
-		logger.Warnw(ctx, "message-not-proto-message", log.Fields{"msg": msg})
-		return fmt.Errorf("not-a-proto-msg-%s", msg)
-	}
-
-	var marshalled []byte
-	var err error
-	//	Create the Sarama producer message
-	if marshalled, err = proto.Marshal(protoMsg); err != nil {
-		logger.Errorw(ctx, "marshalling-failed", log.Fields{"msg": protoMsg, "error": err})
-		return err
-	}
-	key := ""
-	if len(keys) > 0 {
-		key = keys[0] // Only the first key is relevant
-	}
-	kafkaMsg := &sarama.ProducerMessage{
-		Topic: topic.Name,
-		Key:   sarama.StringEncoder(key),
-		Value: sarama.ByteEncoder(marshalled),
-	}
-
-	// Send message to kafka
-	sc.producer.Input() <- kafkaMsg
-	// Wait for result
-	// TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
-	select {
-	case ok := <-sc.producer.Successes():
-		logger.Debugw(ctx, "message-sent", log.Fields{"status": ok.Topic})
-		sc.updateLiveness(ctx, true)
-	case notOk := <-sc.producer.Errors():
-		logger.Debugw(ctx, "error-sending", log.Fields{"status": notOk})
-		if sc.isLivenessError(ctx, notOk) {
-			sc.updateLiveness(ctx, false)
-		}
-		return notOk
-	}
-	return nil
-}
-
-// Enable the liveness monitor channel. This channel will report
-// a "true" or "false" on every publish, which indicates whether
-// or not the channel is still live. This channel is then picked up
-// by the service (i.e. rw_core / ro_core) to update readiness status
-// and/or take other actions.
-func (sc *SaramaClient) EnableLivenessChannel(ctx context.Context, enable bool) chan bool {
-	logger.Infow(ctx, "kafka-enable-liveness-channel", log.Fields{"enable": enable})
-	if enable {
-		sc.livenessMutex.Lock()
-		defer sc.livenessMutex.Unlock()
-		if sc.liveness == nil {
-			logger.Info(ctx, "kafka-create-liveness-channel")
-			// At least 1, so we can immediately post to it without blocking
-			// Setting a bigger number (10) allows the monitor to fall behind
-			// without blocking others. The monitor shouldn't really fall
-			// behind...
-			sc.liveness = make(chan bool, 10)
-			// post intial state to the channel
-			sc.liveness <- sc.alive
-		}
-	} else {
-		// TODO: Think about whether we need the ability to turn off
-		// liveness monitoring
-		panic("Turning off liveness reporting is not supported")
-	}
-	return sc.liveness
-}
-
-// Enable the Healthiness monitor channel. This channel will report "false"
-// if the kafka consumers die, or some other problem occurs which is
-// catastrophic that would require re-creating the client.
-func (sc *SaramaClient) EnableHealthinessChannel(ctx context.Context, enable bool) chan bool {
-	logger.Infow(ctx, "kafka-enable-healthiness-channel", log.Fields{"enable": enable})
-	if enable {
-		sc.healthinessMutex.Lock()
-		defer sc.healthinessMutex.Unlock()
-		if sc.healthiness == nil {
-			logger.Info(ctx, "kafka-create-healthiness-channel")
-			// At least 1, so we can immediately post to it without blocking
-			// Setting a bigger number (10) allows the monitor to fall behind
-			// without blocking others. The monitor shouldn't really fall
-			// behind...
-			sc.healthiness = make(chan bool, 10)
-			// post intial state to the channel
-			sc.healthiness <- sc.healthy
-		}
-	} else {
-		// TODO: Think about whether we need the ability to turn off
-		// liveness monitoring
-		panic("Turning off healthiness reporting is not supported")
-	}
-	return sc.healthiness
-}
-
-// send an empty message on the liveness channel to check whether connectivity has
-// been restored.
-func (sc *SaramaClient) SendLiveness(ctx context.Context) error {
-	if !sc.started {
-		return fmt.Errorf("SendLiveness() called while not started")
-	}
-
-	kafkaMsg := &sarama.ProducerMessage{
-		Topic: "_liveness_test",
-		Value: sarama.StringEncoder(time.Now().Format(time.RFC3339)), // for debugging / informative use
-	}
-
-	// Send message to kafka
-	sc.producer.Input() <- kafkaMsg
-	// Wait for result
-	// TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
-	select {
-	case ok := <-sc.producer.Successes():
-		logger.Debugw(ctx, "liveness-message-sent", log.Fields{"status": ok.Topic})
-		sc.updateLiveness(ctx, true)
-	case notOk := <-sc.producer.Errors():
-		logger.Debugw(ctx, "liveness-error-sending", log.Fields{"status": notOk})
-		if sc.isLivenessError(ctx, notOk) {
-			sc.updateLiveness(ctx, false)
-		}
-		return notOk
-	}
-	return nil
-}
-
-// getGroupId returns the group id from the key-value args.
-func getGroupId(kvArgs ...*KVArg) string {
-	for _, arg := range kvArgs {
-		if arg.Key == GroupIdKey {
-			return arg.Value.(string)
-		}
-	}
-	return ""
-}
-
-// getOffset returns the offset from the key-value args.
-func getOffset(kvArgs ...*KVArg) int64 {
-	for _, arg := range kvArgs {
-		if arg.Key == Offset {
-			return arg.Value.(int64)
-		}
-	}
-	return sarama.OffsetNewest
-}
-
-func (sc *SaramaClient) createClusterAdmin(ctx context.Context) error {
-	config := sarama.NewConfig()
-	config.Version = sarama.V1_0_0_0
-
-	// Create a cluster Admin
-	var cAdmin sarama.ClusterAdmin
-	var err error
-	if cAdmin, err = sarama.NewClusterAdmin([]string{sc.KafkaAddress}, config); err != nil {
-		logger.Errorw(ctx, "cluster-admin-failure", log.Fields{"error": err, "broker-address": sc.KafkaAddress})
-		return err
-	}
-	sc.cAdmin = cAdmin
-	return nil
-}
-
-func (sc *SaramaClient) lockTopic(topic *Topic) {
-	sc.lockOfTopicLockMap.Lock()
-	if _, exist := sc.topicLockMap[topic.Name]; exist {
-		sc.lockOfTopicLockMap.Unlock()
-		sc.topicLockMap[topic.Name].Lock()
-	} else {
-		sc.topicLockMap[topic.Name] = &sync.RWMutex{}
-		sc.lockOfTopicLockMap.Unlock()
-		sc.topicLockMap[topic.Name].Lock()
-	}
-}
-
-func (sc *SaramaClient) unLockTopic(topic *Topic) {
-	sc.lockOfTopicLockMap.Lock()
-	defer sc.lockOfTopicLockMap.Unlock()
-	if _, exist := sc.topicLockMap[topic.Name]; exist {
-		sc.topicLockMap[topic.Name].Unlock()
-	}
-}
-
-func (sc *SaramaClient) addTopicToConsumerChannelMap(id string, arg *consumerChannels) {
-	sc.lockTopicToConsumerChannelMap.Lock()
-	defer sc.lockTopicToConsumerChannelMap.Unlock()
-	if _, exist := sc.topicToConsumerChannelMap[id]; !exist {
-		sc.topicToConsumerChannelMap[id] = arg
-	}
-}
-
-func (sc *SaramaClient) getConsumerChannel(topic *Topic) *consumerChannels {
-	sc.lockTopicToConsumerChannelMap.RLock()
-	defer sc.lockTopicToConsumerChannelMap.RUnlock()
-
-	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
-		return consumerCh
-	}
-	return nil
-}
-
-func (sc *SaramaClient) addChannelToConsumerChannelMap(ctx context.Context, topic *Topic, ch chan *ic.InterContainerMessage) {
-	sc.lockTopicToConsumerChannelMap.Lock()
-	defer sc.lockTopicToConsumerChannelMap.Unlock()
-	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
-		consumerCh.channels = append(consumerCh.channels, ch)
-		return
-	}
-	logger.Warnw(ctx, "consumers-channel-not-exist", log.Fields{"topic": topic.Name})
-}
-
-//closeConsumers closes a list of sarama consumers.  The consumers can either be a partition consumers or a group consumers
-func closeConsumers(ctx context.Context, consumers []interface{}) error {
-	var err error
-	for _, consumer := range consumers {
-		//	Is it a partition consumers?
-		if partionConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
-			if errTemp := partionConsumer.Close(); errTemp != nil {
-				logger.Debugw(ctx, "partition!!!", log.Fields{"err": errTemp})
-				if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
-					// This can occur on race condition
-					err = nil
-				} else {
-					err = errTemp
-				}
-			}
-		} else if groupConsumer, ok := consumer.(*scc.Consumer); ok {
-			if errTemp := groupConsumer.Close(); errTemp != nil {
-				if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
-					// This can occur on race condition
-					err = nil
-				} else {
-					err = errTemp
-				}
-			}
-		}
-	}
-	return err
-}
-
-func (sc *SaramaClient) removeChannelFromConsumerChannelMap(ctx context.Context, topic Topic, ch <-chan *ic.InterContainerMessage) error {
-	sc.lockTopicToConsumerChannelMap.Lock()
-	defer sc.lockTopicToConsumerChannelMap.Unlock()
-	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
-		// Channel will be closed in the removeChannel method
-		consumerCh.channels = removeChannel(ctx, consumerCh.channels, ch)
-		// If there are no more channels then we can close the consumers itself
-		if len(consumerCh.channels) == 0 {
-			logger.Debugw(ctx, "closing-consumers", log.Fields{"topic": topic})
-			err := closeConsumers(ctx, consumerCh.consumers)
-			//err := consumerCh.consumers.Close()
-			delete(sc.topicToConsumerChannelMap, topic.Name)
-			return err
-		}
-		return nil
-	}
-	logger.Warnw(ctx, "topic-does-not-exist", log.Fields{"topic": topic.Name})
-	return errors.New("topic-does-not-exist")
-}
-
-func (sc *SaramaClient) clearTopicFromConsumerChannelMap(ctx context.Context, topic Topic) error {
-	sc.lockTopicToConsumerChannelMap.Lock()
-	defer sc.lockTopicToConsumerChannelMap.Unlock()
-	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
-		for _, ch := range consumerCh.channels {
-			// Channel will be closed in the removeChannel method
-			removeChannel(ctx, consumerCh.channels, ch)
-		}
-		err := closeConsumers(ctx, consumerCh.consumers)
-		//if err == sarama.ErrUnknownTopicOrPartition {
-		//	// Not an error
-		//	err = nil
-		//}
-		//err := consumerCh.consumers.Close()
-		delete(sc.topicToConsumerChannelMap, topic.Name)
-		return err
-	}
-	logger.Debugw(ctx, "topic-does-not-exist", log.Fields{"topic": topic.Name})
-	return nil
-}
-
-//createPublisher creates the publisher which is used to send a message onto kafka
-func (sc *SaramaClient) createPublisher(ctx context.Context) error {
-	// This Creates the publisher
-	config := sarama.NewConfig()
-	config.Version = sarama.V1_0_0_0
-	config.Producer.Partitioner = sarama.NewRandomPartitioner
-	config.Producer.Flush.Frequency = time.Duration(sc.producerFlushFrequency)
-	config.Producer.Flush.Messages = sc.producerFlushMessages
-	config.Producer.Flush.MaxMessages = sc.producerFlushMaxmessages
-	config.Producer.Return.Errors = sc.producerReturnErrors
-	config.Producer.Return.Successes = sc.producerReturnSuccess
-	//config.Producer.RequiredAcks = sarama.WaitForAll
-	config.Producer.RequiredAcks = sarama.WaitForLocal
-
-	brokers := []string{sc.KafkaAddress}
-
-	if producer, err := sarama.NewAsyncProducer(brokers, config); err != nil {
-		logger.Errorw(ctx, "error-starting-publisher", log.Fields{"error": err})
-		return err
-	} else {
-		sc.producer = producer
-	}
-	logger.Info(ctx, "Kafka-publisher-created")
-	return nil
-}
-
-func (sc *SaramaClient) createConsumer(ctx context.Context) error {
-	config := sarama.NewConfig()
-	config.Version = sarama.V1_0_0_0
-	config.Consumer.Return.Errors = true
-	config.Consumer.Fetch.Min = 1
-	config.Consumer.MaxWaitTime = time.Duration(sc.consumerMaxwait) * time.Millisecond
-	config.Consumer.MaxProcessingTime = time.Duration(sc.maxProcessingTime) * time.Millisecond
-	config.Consumer.Offsets.Initial = sarama.OffsetNewest
-	config.Metadata.Retry.Max = sc.metadataMaxRetry
-	brokers := []string{sc.KafkaAddress}
-
-	if consumer, err := sarama.NewConsumer(brokers, config); err != nil {
-		logger.Errorw(ctx, "error-starting-consumers", log.Fields{"error": err})
-		return err
-	} else {
-		sc.consumer = consumer
-	}
-	logger.Info(ctx, "Kafka-consumers-created")
-	return nil
-}
-
-// createGroupConsumer creates a consumers group
-func (sc *SaramaClient) createGroupConsumer(ctx context.Context, topic *Topic, groupId string, initialOffset int64, retries int) (*scc.Consumer, error) {
-	config := scc.NewConfig()
-	config.Version = sarama.V1_0_0_0
-	config.ClientID = uuid.New().String()
-	config.Group.Mode = scc.ConsumerModeMultiplex
-	config.Consumer.Group.Heartbeat.Interval, _ = time.ParseDuration("1s")
-	config.Consumer.Return.Errors = true
-	//config.Group.Return.Notifications = false
-	//config.Consumer.MaxWaitTime = time.Duration(DefaultConsumerMaxwait) * time.Millisecond
-	//config.Consumer.MaxProcessingTime = time.Duration(DefaultMaxProcessingTime) * time.Millisecond
-	config.Consumer.Offsets.Initial = initialOffset
-	//config.Consumer.Offsets.Initial = sarama.OffsetOldest
-	brokers := []string{sc.KafkaAddress}
-
-	topics := []string{topic.Name}
-	var consumer *scc.Consumer
-	var err error
-
-	if consumer, err = scc.NewConsumer(brokers, groupId, topics, config); err != nil {
-		logger.Errorw(ctx, "create-group-consumers-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
-		return nil, err
-	}
-	logger.Debugw(ctx, "create-group-consumers-success", log.Fields{"topic": topic.Name, "groupId": groupId})
-
-	//sc.groupConsumers[topic.Name] = consumer
-	sc.addToGroupConsumers(topic.Name, consumer)
-	return consumer, nil
-}
-
-// dispatchToConsumers sends the intercontainermessage received on a given topic to all subscribers for that
-// topic via the unique channel each subscriber received during subscription
-func (sc *SaramaClient) dispatchToConsumers(consumerCh *consumerChannels, protoMessage *ic.InterContainerMessage) {
-	// Need to go over all channels and publish messages to them - do we need to copy msg?
-	sc.lockTopicToConsumerChannelMap.RLock()
-	for _, ch := range consumerCh.channels {
-		go func(c chan *ic.InterContainerMessage) {
-			c <- protoMessage
-		}(ch)
-	}
-	sc.lockTopicToConsumerChannelMap.RUnlock()
-
-	if callback := sc.metadataCallback; callback != nil {
-		ts, _ := ptypes.Timestamp(protoMessage.Header.Timestamp)
-		callback(protoMessage.Header.FromTopic, ts)
-	}
-}
-
-func (sc *SaramaClient) consumeFromAPartition(ctx context.Context, topic *Topic, consumer sarama.PartitionConsumer, consumerChnls *consumerChannels) {
-	logger.Debugw(ctx, "starting-partition-consumption-loop", log.Fields{"topic": topic.Name})
-startloop:
-	for {
-		select {
-		case err, ok := <-consumer.Errors():
-			if ok {
-				if sc.isLivenessError(ctx, err) {
-					sc.updateLiveness(ctx, false)
-					logger.Warnw(ctx, "partition-consumers-error", log.Fields{"error": err})
-				}
-			} else {
-				// Channel is closed
-				break startloop
-			}
-		case msg, ok := <-consumer.Messages():
-			//logger.Debugw(ctx, "message-received", logger.Fields{"msg": msg, "receivedTopic": msg.Topic})
-			if !ok {
-				// channel is closed
-				break startloop
-			}
-			msgBody := msg.Value
-			sc.updateLiveness(ctx, true)
-			logger.Debugw(ctx, "message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
-			icm := &ic.InterContainerMessage{}
-			if err := proto.Unmarshal(msgBody, icm); err != nil {
-				logger.Warnw(ctx, "partition-invalid-message", log.Fields{"error": err})
-				continue
-			}
-			go sc.dispatchToConsumers(consumerChnls, icm)
-		case <-sc.doneCh:
-			logger.Infow(ctx, "partition-received-exit-signal", log.Fields{"topic": topic.Name})
-			break startloop
-		}
-	}
-	logger.Infow(ctx, "partition-consumer-stopped", log.Fields{"topic": topic.Name})
-	sc.setUnhealthy(ctx)
-}
-
-func (sc *SaramaClient) consumeGroupMessages(ctx context.Context, topic *Topic, consumer *scc.Consumer, consumerChnls *consumerChannels) {
-	logger.Debugw(ctx, "starting-group-consumption-loop", log.Fields{"topic": topic.Name})
-
-startloop:
-	for {
-		select {
-		case err, ok := <-consumer.Errors():
-			if ok {
-				if sc.isLivenessError(ctx, err) {
-					sc.updateLiveness(ctx, false)
-				}
-				logger.Warnw(ctx, "group-consumers-error", log.Fields{"topic": topic.Name, "error": err})
-			} else {
-				logger.Warnw(ctx, "group-consumers-closed-err", log.Fields{"topic": topic.Name})
-				// channel is closed
-				break startloop
-			}
-		case msg, ok := <-consumer.Messages():
-			if !ok {
-				logger.Warnw(ctx, "group-consumers-closed-msg", log.Fields{"topic": topic.Name})
-				// Channel closed
-				break startloop
-			}
-			sc.updateLiveness(ctx, true)
-			logger.Debugw(ctx, "message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
-			msgBody := msg.Value
-			icm := &ic.InterContainerMessage{}
-			if err := proto.Unmarshal(msgBody, icm); err != nil {
-				logger.Warnw(ctx, "invalid-message", log.Fields{"error": err})
-				continue
-			}
-			go sc.dispatchToConsumers(consumerChnls, icm)
-			consumer.MarkOffset(msg, "")
-		case ntf := <-consumer.Notifications():
-			logger.Debugw(ctx, "group-received-notification", log.Fields{"notification": ntf})
-		case <-sc.doneCh:
-			logger.Infow(ctx, "group-received-exit-signal", log.Fields{"topic": topic.Name})
-			break startloop
-		}
-	}
-	logger.Infow(ctx, "group-consumer-stopped", log.Fields{"topic": topic.Name})
-	sc.setUnhealthy(ctx)
-}
-
-func (sc *SaramaClient) startConsumers(ctx context.Context, topic *Topic) error {
-	logger.Debugw(ctx, "starting-consumers", log.Fields{"topic": topic.Name})
-	var consumerCh *consumerChannels
-	if consumerCh = sc.getConsumerChannel(topic); consumerCh == nil {
-		logger.Errorw(ctx, "consumers-not-exist", log.Fields{"topic": topic.Name})
-		return errors.New("consumers-not-exist")
-	}
-	// For each consumer listening for that topic, start a consumption loop
-	for _, consumer := range consumerCh.consumers {
-		if pConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
-			go sc.consumeFromAPartition(ctx, topic, pConsumer, consumerCh)
-		} else if gConsumer, ok := consumer.(*scc.Consumer); ok {
-			go sc.consumeGroupMessages(ctx, topic, gConsumer, consumerCh)
-		} else {
-			logger.Errorw(ctx, "invalid-consumer", log.Fields{"topic": topic})
-			return errors.New("invalid-consumer")
-		}
-	}
-	return nil
-}
-
-//// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
-//// for that topic.  It also starts the routine that listens for messages on that topic.
-func (sc *SaramaClient) setupPartitionConsumerChannel(ctx context.Context, topic *Topic, initialOffset int64) (chan *ic.InterContainerMessage, error) {
-	var pConsumers []sarama.PartitionConsumer
-	var err error
-
-	if pConsumers, err = sc.createPartitionConsumers(ctx, topic, initialOffset); err != nil {
-		logger.Errorw(ctx, "creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
-		return nil, err
-	}
-
-	consumersIf := make([]interface{}, 0)
-	for _, pConsumer := range pConsumers {
-		consumersIf = append(consumersIf, pConsumer)
-	}
-
-	// Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
-	// unbuffered to verify race conditions.
-	consumerListeningChannel := make(chan *ic.InterContainerMessage)
-	cc := &consumerChannels{
-		consumers: consumersIf,
-		channels:  []chan *ic.InterContainerMessage{consumerListeningChannel},
-	}
-
-	// Add the consumers channel to the map
-	sc.addTopicToConsumerChannelMap(topic.Name, cc)
-
-	//Start a consumers to listen on that specific topic
-	go func() {
-		if err := sc.startConsumers(ctx, topic); err != nil {
-			logger.Errorw(ctx, "start-consumers-failed", log.Fields{
-				"topic": topic,
-				"error": err})
-		}
-	}()
-
-	return consumerListeningChannel, nil
-}
-
-// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
-// for that topic.  It also starts the routine that listens for messages on that topic.
-func (sc *SaramaClient) setupGroupConsumerChannel(ctx context.Context, topic *Topic, groupId string, initialOffset int64) (chan *ic.InterContainerMessage, error) {
-	// TODO:  Replace this development partition consumers with a group consumers
-	var pConsumer *scc.Consumer
-	var err error
-	if pConsumer, err = sc.createGroupConsumer(ctx, topic, groupId, initialOffset, DefaultMaxRetries); err != nil {
-		logger.Errorw(ctx, "creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
-		return nil, err
-	}
-	// Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
-	// unbuffered to verify race conditions.
-	consumerListeningChannel := make(chan *ic.InterContainerMessage)
-	cc := &consumerChannels{
-		consumers: []interface{}{pConsumer},
-		channels:  []chan *ic.InterContainerMessage{consumerListeningChannel},
-	}
-
-	// Add the consumers channel to the map
-	sc.addTopicToConsumerChannelMap(topic.Name, cc)
-
-	//Start a consumers to listen on that specific topic
-	go func() {
-		if err := sc.startConsumers(ctx, topic); err != nil {
-			logger.Errorw(ctx, "start-consumers-failed", log.Fields{
-				"topic": topic,
-				"error": err})
-		}
-	}()
-
-	return consumerListeningChannel, nil
-}
-
-func (sc *SaramaClient) createPartitionConsumers(ctx context.Context, topic *Topic, initialOffset int64) ([]sarama.PartitionConsumer, error) {
-	logger.Debugw(ctx, "creating-partition-consumers", log.Fields{"topic": topic.Name})
-	partitionList, err := sc.consumer.Partitions(topic.Name)
-	if err != nil {
-		logger.Warnw(ctx, "get-partition-failure", log.Fields{"error": err, "topic": topic.Name})
-		return nil, err
-	}
-
-	pConsumers := make([]sarama.PartitionConsumer, 0)
-	for _, partition := range partitionList {
-		var pConsumer sarama.PartitionConsumer
-		if pConsumer, err = sc.consumer.ConsumePartition(topic.Name, partition, initialOffset); err != nil {
-			logger.Warnw(ctx, "consumers-partition-failure", log.Fields{"error": err, "topic": topic.Name})
-			return nil, err
-		}
-		pConsumers = append(pConsumers, pConsumer)
-	}
-	return pConsumers, nil
-}
-
-func removeChannel(ctx context.Context, channels []chan *ic.InterContainerMessage, ch <-chan *ic.InterContainerMessage) []chan *ic.InterContainerMessage {
-	var i int
-	var channel chan *ic.InterContainerMessage
-	for i, channel = range channels {
-		if channel == ch {
-			channels[len(channels)-1], channels[i] = channels[i], channels[len(channels)-1]
-			close(channel)
-			logger.Debug(ctx, "channel-closed")
-			return channels[:len(channels)-1]
-		}
-	}
-	return channels
-}
-
-func (sc *SaramaClient) addToGroupConsumers(topic string, consumer *scc.Consumer) {
-	sc.lockOfGroupConsumers.Lock()
-	defer sc.lockOfGroupConsumers.Unlock()
-	if _, exist := sc.groupConsumers[topic]; !exist {
-		sc.groupConsumers[topic] = consumer
-	}
-}
-
-func (sc *SaramaClient) deleteFromGroupConsumers(ctx context.Context, topic string) error {
-	sc.lockOfGroupConsumers.Lock()
-	defer sc.lockOfGroupConsumers.Unlock()
-	if _, exist := sc.groupConsumers[topic]; exist {
-		consumer := sc.groupConsumers[topic]
-		delete(sc.groupConsumers, topic)
-		if err := consumer.Close(); err != nil {
-			logger.Errorw(ctx, "failure-closing-consumer", log.Fields{"error": err})
-			return err
-		}
-	}
-	return nil
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/utils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/utils.go
deleted file mode 100644
index bdc615f..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/kafka/utils.go
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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 kafka
-
-import (
-	"github.com/golang/protobuf/ptypes/any"
-	"strings"
-)
-
-const (
-	TopicSeparator = "_"
-	DeviceIdLength = 24
-)
-
-// A Topic definition - may be augmented with additional attributes eventually
-type Topic struct {
-	// The name of the topic. It must start with a letter,
-	// and contain only letters (`[A-Za-z]`), numbers (`[0-9]`), dashes (`-`),
-	// underscores (`_`), periods (`.`), tildes (`~`), plus (`+`) or percent
-	// signs (`%`).
-	Name string
-}
-
-type KVArg struct {
-	Key   string
-	Value interface{}
-}
-
-type RpcMType int
-
-const (
-	RpcFormattingError RpcMType = iota
-	RpcSent
-	RpcReply
-	RpcTimeout
-	RpcTransportError
-	RpcSystemClosing
-)
-
-type RpcResponse struct {
-	MType RpcMType
-	Err   error
-	Reply *any.Any
-}
-
-func NewResponse(messageType RpcMType, err error, body *any.Any) *RpcResponse {
-	return &RpcResponse{
-		MType: messageType,
-		Err:   err,
-		Reply: body,
-	}
-}
-
-// TODO:  Remove and provide better may to get the device id
-// GetDeviceIdFromTopic extract the deviceId from the topic name.  The topic name is formatted either as:
-//			<any string> or <any string>_<deviceId>.  The device Id is 24 characters long.
-func GetDeviceIdFromTopic(topic Topic) string {
-	pos := strings.LastIndex(topic.Name, TopicSeparator)
-	if pos == -1 {
-		return ""
-	}
-	adjustedPos := pos + len(TopicSeparator)
-	if adjustedPos >= len(topic.Name) {
-		return ""
-	}
-	deviceId := topic.Name[adjustedPos:len(topic.Name)]
-	if len(deviceId) != DeviceIdLength {
-		return ""
-	}
-	return deviceId
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/common.go
deleted file mode 100644
index e058e48..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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 meters
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/meter_utils.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/meter_utils.go
deleted file mode 100644
index d220c0b..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/meters/meter_utils.go
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 meters
-
-import (
-	"context"
-	"fmt"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	ofp "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	tp_pb "github.com/opencord/voltha-protos/v4/go/tech_profile"
-)
-
-// GetTrafficShapingInfo returns CIR,PIR and GIR values
-func GetTrafficShapingInfo(ctx context.Context, meterConfig *ofp.OfpMeterConfig) (*tp_pb.TrafficShapingInfo, error) {
-	switch meterBandSize := len(meterConfig.Bands); {
-	case meterBandSize == 1:
-		band := meterConfig.Bands[0]
-		if band.BurstSize == 0 { // GIR, tcont type 1
-			return &tp_pb.TrafficShapingInfo{Gir: band.Rate}, nil
-		}
-		return &tp_pb.TrafficShapingInfo{Pir: band.Rate, Pbs: band.BurstSize}, nil // PIR, tcont type 4
-	case meterBandSize == 2:
-		firstBand, secondBand := meterConfig.Bands[0], meterConfig.Bands[1]
-		if firstBand.BurstSize == 0 && secondBand.BurstSize == 0 &&
-			firstBand.Rate == secondBand.Rate { // PIR == GIR, tcont type 1
-			return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Gir: secondBand.Rate}, nil
-		}
-		if firstBand.BurstSize > 0 && secondBand.BurstSize > 0 { // PIR, CIR, tcont type 2 or 3
-			if firstBand.Rate > secondBand.Rate { // always PIR >= CIR
-				return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Pbs: firstBand.BurstSize, Cir: secondBand.Rate, Cbs: secondBand.BurstSize}, nil
-			}
-			return &tp_pb.TrafficShapingInfo{Pir: secondBand.Rate, Pbs: secondBand.BurstSize, Cir: firstBand.Rate, Cbs: firstBand.BurstSize}, nil
-		}
-	case meterBandSize == 3: // PIR,CIR,GIR, tcont type 5
-		var count, girIndex int
-		for i, band := range meterConfig.Bands {
-			if band.BurstSize == 0 { // find GIR
-				count = count + 1
-				girIndex = i
-			}
-		}
-		if count == 1 {
-			bands := make([]*ofp.OfpMeterBandHeader, len(meterConfig.Bands))
-			copy(bands, meterConfig.Bands)
-			pirCirBands := append(bands[:girIndex], bands[girIndex+1:]...)
-			firstBand, secondBand := pirCirBands[0], pirCirBands[1]
-			if firstBand.Rate > secondBand.Rate {
-				return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Pbs: firstBand.BurstSize, Cir: secondBand.Rate, Cbs: secondBand.BurstSize, Gir: meterConfig.Bands[girIndex].Rate}, nil
-			}
-			return &tp_pb.TrafficShapingInfo{Pir: secondBand.Rate, Pbs: secondBand.BurstSize, Cir: firstBand.Rate, Cbs: firstBand.BurstSize, Gir: meterConfig.Bands[girIndex].Rate}, nil
-		}
-	default:
-		logger.Errorw(ctx, "invalid-meter-config", log.Fields{"meter-config": meterConfig})
-		return nil, fmt.Errorf("invalid-meter-config: %v", meterConfig)
-	}
-	return nil, fmt.Errorf("invalid-meter-config: %v", meterConfig)
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/common.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/common.go
deleted file mode 100644
index 119d78e..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2020-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 probe
-
-import (
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-)
-
-var logger log.CLogger
-
-func init() {
-	// Setup this package so that it's log level can be modified at run time
-	var err error
-	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
-	if err != nil {
-		panic(err)
-	}
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/probe.go b/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/probe.go
deleted file mode 100644
index b66f398..0000000
--- a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/probe/probe.go
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * 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 probe
-
-import (
-	"context"
-	"fmt"
-	"github.com/opencord/voltha-lib-go/v5/pkg/log"
-	"net/http"
-	"sync"
-)
-
-// ProbeContextKey used to fetch the Probe instance from a context
-type ProbeContextKeyType string
-
-// ServiceStatus typed values for service status
-type ServiceStatus int
-
-const (
-	// ServiceStatusUnknown initial state of services
-	ServiceStatusUnknown ServiceStatus = iota
-
-	// ServiceStatusPreparing to optionally be used for prep, such as connecting
-	ServiceStatusPreparing
-
-	// ServiceStatusPrepared to optionally be used when prep is complete, but before run
-	ServiceStatusPrepared
-
-	// ServiceStatusRunning service is functional
-	ServiceStatusRunning
-
-	// ServiceStatusStopped service has stopped, but not because of error
-	ServiceStatusStopped
-
-	// ServiceStatusFailed service has stopped because of an error
-	ServiceStatusFailed
-
-	// ServiceStatusNotReady service has started but is unable to accept requests
-	ServiceStatusNotReady
-)
-
-const (
-	// ProbeContextKey value of context key to fetch probe
-	ProbeContextKey = ProbeContextKeyType("status-update-probe")
-)
-
-// String convert ServiceStatus values to strings
-func (s ServiceStatus) String() string {
-	switch s {
-	default:
-		fallthrough
-	case ServiceStatusUnknown:
-		return "Unknown"
-	case ServiceStatusPreparing:
-		return "Preparing"
-	case ServiceStatusPrepared:
-		return "Prepared"
-	case ServiceStatusRunning:
-		return "Running"
-	case ServiceStatusStopped:
-		return "Stopped"
-	case ServiceStatusFailed:
-		return "Failed"
-	case ServiceStatusNotReady:
-		return "NotReady"
-	}
-}
-
-// ServiceStatusUpdate status update event
-type ServiceStatusUpdate struct {
-	Name   string
-	Status ServiceStatus
-}
-
-// Probe reciever on which to implement probe capabilities
-type Probe struct {
-	readyFunc  func(map[string]ServiceStatus) bool
-	healthFunc func(map[string]ServiceStatus) bool
-
-	mutex     sync.RWMutex
-	status    map[string]ServiceStatus
-	isReady   bool
-	isHealthy bool
-}
-
-// WithReadyFunc override the default ready calculation function
-func (p *Probe) WithReadyFunc(readyFunc func(map[string]ServiceStatus) bool) *Probe {
-	p.readyFunc = readyFunc
-	return p
-}
-
-// WithHealthFunc override the default health calculation function
-func (p *Probe) WithHealthFunc(healthFunc func(map[string]ServiceStatus) bool) *Probe {
-	p.healthFunc = healthFunc
-	return p
-}
-
-// RegisterService register one or more service names with the probe, status will be track against service name
-func (p *Probe) RegisterService(ctx context.Context, names ...string) {
-	p.mutex.Lock()
-	defer p.mutex.Unlock()
-	if p.status == nil {
-		p.status = make(map[string]ServiceStatus)
-	}
-	for _, name := range names {
-		if _, ok := p.status[name]; !ok {
-			p.status[name] = ServiceStatusUnknown
-			logger.Debugw(ctx, "probe-service-registered", log.Fields{"service-name": name})
-		}
-	}
-
-	if p.readyFunc != nil {
-		p.isReady = p.readyFunc(p.status)
-	} else {
-		p.isReady = defaultReadyFunc(p.status)
-	}
-
-	if p.healthFunc != nil {
-		p.isHealthy = p.healthFunc(p.status)
-	} else {
-		p.isHealthy = defaultHealthFunc(p.status)
-	}
-}
-
-// UpdateStatus utility function to send a service update to the probe
-func (p *Probe) UpdateStatus(ctx context.Context, name string, status ServiceStatus) {
-	p.mutex.Lock()
-	defer p.mutex.Unlock()
-	if p.status == nil {
-		p.status = make(map[string]ServiceStatus)
-	}
-
-	// if status hasn't changed, avoid doing useless work
-	existingStatus, ok := p.status[name]
-	if ok && (existingStatus == status) {
-		return
-	}
-
-	p.status[name] = status
-	if p.readyFunc != nil {
-		p.isReady = p.readyFunc(p.status)
-	} else {
-		p.isReady = defaultReadyFunc(p.status)
-	}
-
-	if p.healthFunc != nil {
-		p.isHealthy = p.healthFunc(p.status)
-	} else {
-		p.isHealthy = defaultHealthFunc(p.status)
-	}
-	logger.Debugw(ctx, "probe-service-status-updated",
-		log.Fields{
-			"service-name": name,
-			"status":       status.String(),
-			"ready":        p.isReady,
-			"health":       p.isHealthy,
-		})
-}
-
-func (p *Probe) GetStatus(name string) ServiceStatus {
-	p.mutex.Lock()
-	defer p.mutex.Unlock()
-
-	if p.status == nil {
-		p.status = make(map[string]ServiceStatus)
-	}
-
-	currentStatus, ok := p.status[name]
-	if ok {
-		return currentStatus
-	}
-
-	return ServiceStatusUnknown
-}
-
-func GetProbeFromContext(ctx context.Context) *Probe {
-	if ctx != nil {
-		if value := ctx.Value(ProbeContextKey); value != nil {
-			if p, ok := value.(*Probe); ok {
-				return p
-			}
-		}
-	}
-	return nil
-}
-
-// UpdateStatusFromContext a convenience function to pull the Probe reference from the
-// Context, if it exists, and then calling UpdateStatus on that Probe reference. If Context
-// is nil or if a Probe reference is not associated with the ProbeContextKey then nothing
-// happens
-func UpdateStatusFromContext(ctx context.Context, name string, status ServiceStatus) {
-	p := GetProbeFromContext(ctx)
-	if p != nil {
-		p.UpdateStatus(ctx, name, status)
-	}
-}
-
-// pulled out to a function to help better enable unit testing
-func (p *Probe) readzFunc(w http.ResponseWriter, req *http.Request) {
-	p.mutex.RLock()
-	defer p.mutex.RUnlock()
-	if p.isReady {
-		w.WriteHeader(http.StatusOK)
-	} else {
-		w.WriteHeader(http.StatusTeapot)
-	}
-}
-func (p *Probe) healthzFunc(w http.ResponseWriter, req *http.Request) {
-	p.mutex.RLock()
-	defer p.mutex.RUnlock()
-	if p.isHealthy {
-		w.WriteHeader(http.StatusOK)
-	} else {
-		w.WriteHeader(http.StatusTeapot)
-	}
-}
-func (p *Probe) detailzFunc(w http.ResponseWriter, req *http.Request) {
-	ctx := context.Background()
-	p.mutex.RLock()
-	defer p.mutex.RUnlock()
-	w.Header().Set("Content-Type", "application/json")
-	if _, err := w.Write([]byte("{")); err != nil {
-		logger.Errorw(ctx, "write-response", log.Fields{"error": err})
-		w.WriteHeader(http.StatusInternalServerError)
-		return
-	}
-	comma := ""
-	for c, s := range p.status {
-		if _, err := w.Write([]byte(fmt.Sprintf("%s\"%s\": \"%s\"", comma, c, s.String()))); err != nil {
-			logger.Errorw(ctx, "write-response", log.Fields{"error": err})
-			w.WriteHeader(http.StatusInternalServerError)
-			return
-		}
-		comma = ", "
-	}
-	if _, err := w.Write([]byte("}")); err != nil {
-		logger.Errorw(ctx, "write-response", log.Fields{"error": err})
-		w.WriteHeader(http.StatusInternalServerError)
-		return
-	}
-	w.WriteHeader(http.StatusOK)
-}
-
-// ListenAndServe implements 3 HTTP endpoints on the given port for healthz, readz, and detailz. Returns only on error
-func (p *Probe) ListenAndServe(ctx context.Context, address string) {
-	mux := http.NewServeMux()
-
-	// Returns the result of the readyFunc calculation
-	mux.HandleFunc("/readz", p.readzFunc)
-
-	// Returns the result of the healthFunc calculation
-	mux.HandleFunc("/healthz", p.healthzFunc)
-
-	// Returns the details of the services and their status as JSON
-	mux.HandleFunc("/detailz", p.detailzFunc)
-	s := &http.Server{
-		Addr:    address,
-		Handler: mux,
-	}
-	logger.Fatal(ctx, s.ListenAndServe())
-}
-
-func (p *Probe) IsReady() bool {
-	return p.isReady
-}
-
-// defaultReadyFunc if all services are running then ready, else not
-func defaultReadyFunc(services map[string]ServiceStatus) bool {
-	if len(services) == 0 {
-		return false
-	}
-	for _, status := range services {
-		if status != ServiceStatusRunning {
-			return false
-		}
-	}
-	return true
-}
-
-// defaultHealthFunc if no service is stopped or failed, then healthy, else not.
-// service is start as unknown, so they are considered healthy
-func defaultHealthFunc(services map[string]ServiceStatus) bool {
-	if len(services) == 0 {
-		return false
-	}
-	for _, status := range services {
-		if status == ServiceStatusStopped || status == ServiceStatusFailed {
-			return false
-		}
-	}
-	return true
-}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/common.go
new file mode 100644
index 0000000..4813ba1
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 config
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/configmanager.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/configmanager.go
new file mode 100644
index 0000000..c489407
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/configmanager.go
@@ -0,0 +1,284 @@
+/*
+ * 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 config
+
+import (
+	"context"
+	"fmt"
+	"os"
+	"strings"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/db"
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	defaultkvStoreConfigPath     = "config"
+	defaultkvStoreDataPathPrefix = "service/voltha/voltha1_voltha1"
+	kvStorePathSeparator         = "/"
+)
+
+// ConfigType represents the type for which config is created inside the kvstore
+// For example, loglevel
+type ConfigType int
+
+const (
+	ConfigTypeLogLevel ConfigType = iota
+	ConfigTypeMetadata
+	ConfigTypeKafka
+	ConfigTypeLogFeatures
+)
+
+func (c ConfigType) String() string {
+	return [...]string{"loglevel", "metadata", "kafka", "logfeatures"}[c]
+}
+
+// ChangeEvent represents the event recieved from watch
+// For example, Put Event
+type ChangeEvent int
+
+const (
+	Put ChangeEvent = iota
+	Delete
+)
+
+func (ce ChangeEvent) String() string {
+	return [...]string{"Put", "Delete"}[ce]
+}
+
+// ConfigChangeEvent represents config for the events recieved from watch
+// For example,ChangeType is Put ,ConfigAttribute default
+type ConfigChangeEvent struct {
+	ChangeType      ChangeEvent
+	ConfigAttribute string
+}
+
+// ConfigManager is a wrapper over Backend to maintain Configuration of voltha components
+// in kvstore based persistent storage
+type ConfigManager struct {
+	Backend               *db.Backend
+	KVStoreConfigPrefix   string
+	KVStoreDataPathPrefix string
+}
+
+// ComponentConfig represents a category of configuration for a specific VOLTHA component type
+// stored in a persistent storage pointed to by Config Manager
+// For example, one ComponentConfig instance will be created for loglevel config type for rw-core
+// component while another ComponentConfig instance will refer to connection config type for same
+// rw-core component. So, there can be multiple ComponentConfig instance created per component
+// pointing to different category of configuration.
+//
+// Configuration pointed to be by ComponentConfig is stored in kvstore as a list of key/value pairs
+// under the hierarchical tree with following base path
+// <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/
+//
+// For example, rw-core ComponentConfig for loglevel config entries will be stored under following path
+// /voltha/service/config/rw-core/loglevel/
+type ComponentConfig struct {
+	cManager         *ConfigManager
+	componentLabel   string
+	configType       ConfigType
+	changeEventChan  chan *ConfigChangeEvent
+	kvStoreEventChan chan *kvstore.Event
+}
+
+func NewConfigManager(ctx context.Context, kvClient kvstore.Client, kvStoreType, kvStoreAddress string, kvStoreTimeout time.Duration) *ConfigManager {
+	var kvStorePrefix string
+	if prefix, present := os.LookupEnv("KV_STORE_DATAPATH_PREFIX"); present {
+		kvStorePrefix = prefix
+		logger.Infow(ctx, "KV_STORE_DATAPATH_PREFIX env variable is set, ", log.Fields{"kvStoreDataPathPrefix": kvStorePrefix})
+	} else {
+		kvStorePrefix = defaultkvStoreDataPathPrefix
+		logger.Infow(ctx, "KV_STORE_DATAPATH_PREFIX env variable is not set, using default", log.Fields{"kvStoreDataPathPrefix": defaultkvStoreDataPathPrefix})
+	}
+	return &ConfigManager{
+		KVStoreConfigPrefix:   defaultkvStoreConfigPath,
+		KVStoreDataPathPrefix: kvStorePrefix,
+		Backend: &db.Backend{
+			Client:     kvClient,
+			StoreType:  kvStoreType,
+			Address:    kvStoreAddress,
+			Timeout:    kvStoreTimeout,
+			PathPrefix: kvStorePrefix,
+		},
+	}
+}
+
+// RetrieveComponentList list the component Names for which loglevel is stored in kvstore
+func (c *ConfigManager) RetrieveComponentList(ctx context.Context, configType ConfigType) ([]string, error) {
+	data, err := c.Backend.List(ctx, c.KVStoreConfigPrefix)
+	if err != nil {
+		return nil, err
+	}
+
+	// Looping through the data recieved from the Backend for config
+	// Trimming and Splitting the required key and value from data and  storing as componentName,PackageName and Level
+	// For Example, recieved key would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default and value \"DEBUG\"
+	// Then in default will be stored as PackageName,componentName as <Component Name> and DEBUG will be stored as value in List struct
+	ccPathPrefix := kvStorePathSeparator + configType.String() + kvStorePathSeparator
+	pathPrefix := c.KVStoreDataPathPrefix + kvStorePathSeparator + c.KVStoreConfigPrefix + kvStorePathSeparator
+	var list []string
+	keys := make(map[string]interface{})
+	for attr := range data {
+		cname := strings.TrimPrefix(attr, pathPrefix)
+		cName := strings.SplitN(cname, ccPathPrefix, 2)
+		if len(cName) != 2 {
+			continue
+		}
+		if _, exist := keys[cName[0]]; !exist {
+			keys[cName[0]] = nil
+			list = append(list, cName[0])
+		}
+	}
+	return list, nil
+}
+
+// Initialize the component config
+func (cm *ConfigManager) InitComponentConfig(componentLabel string, configType ConfigType) *ComponentConfig {
+
+	return &ComponentConfig{
+		componentLabel:   componentLabel,
+		configType:       configType,
+		cManager:         cm,
+		changeEventChan:  nil,
+		kvStoreEventChan: nil,
+	}
+
+}
+
+func (c *ComponentConfig) makeConfigPath() string {
+
+	cType := c.configType.String()
+	return c.cManager.KVStoreConfigPrefix + kvStorePathSeparator +
+		c.componentLabel + kvStorePathSeparator + cType
+}
+
+// MonitorForConfigChange watch on the subkeys for the given key
+// Any changes to the subkeys for the given key will return an event channel
+// Then Event channel will be processed and  new event channel with required values will be created and return
+// For example, rw-core will be watching on <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/
+// will return an event channel for PUT,DELETE eventType.
+// Then values from event channel will be processed and  stored in kvStoreEventChan.
+func (c *ComponentConfig) MonitorForConfigChange(ctx context.Context) chan *ConfigChangeEvent {
+	key := c.makeConfigPath()
+
+	logger.Debugw(ctx, "monitoring-for-config-change", log.Fields{"key": key})
+
+	c.changeEventChan = make(chan *ConfigChangeEvent, 1)
+
+	c.kvStoreEventChan = c.cManager.Backend.CreateWatch(ctx, key, true)
+
+	go c.processKVStoreWatchEvents(ctx)
+
+	return c.changeEventChan
+}
+
+// processKVStoreWatchEvents process event channel recieved from the Backend for any ChangeType
+// It checks for the EventType is valid or not.For the valid EventTypes creates ConfigChangeEvent and send it on channel
+func (c *ComponentConfig) processKVStoreWatchEvents(ctx context.Context) {
+
+	ccKeyPrefix := c.makeConfigPath()
+
+	logger.Debugw(ctx, "processing-kvstore-event-change", log.Fields{"key-prefix": ccKeyPrefix})
+
+	ccPathPrefix := c.cManager.Backend.PathPrefix + ccKeyPrefix + kvStorePathSeparator
+
+	for watchResp := range c.kvStoreEventChan {
+
+		if watchResp.EventType == kvstore.CONNECTIONDOWN || watchResp.EventType == kvstore.UNKNOWN {
+			logger.Warnw(ctx, "received-invalid-change-type-in-watch-channel-from-kvstore", log.Fields{"change-type": watchResp.EventType})
+			continue
+		}
+
+		// populating the configAttribute from the received Key
+		// For Example, Key received would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default
+		// Storing default in configAttribute variable
+		ky := fmt.Sprintf("%s", watchResp.Key)
+
+		c.changeEventChan <- &ConfigChangeEvent{
+			ChangeType:      ChangeEvent(watchResp.EventType),
+			ConfigAttribute: strings.TrimPrefix(ky, ccPathPrefix),
+		}
+	}
+}
+
+// Retrieves value of a specific config key. Value of key is returned in String format
+func (c *ComponentConfig) Retrieve(ctx context.Context, configKey string) (string, error) {
+	key := c.makeConfigPath() + "/" + configKey
+
+	logger.Debugw(ctx, "retrieving-config", log.Fields{"key": key})
+
+	if kvpair, err := c.cManager.Backend.Get(ctx, key); err != nil {
+		return "", err
+	} else {
+		if kvpair == nil {
+			return "", fmt.Errorf("config-key-does-not-exist : %s", key)
+		}
+
+		value := strings.Trim(fmt.Sprintf("%s", kvpair.Value), "\"")
+		logger.Debugw(ctx, "retrieved-config", log.Fields{"key": key, "value": value})
+		return value, nil
+	}
+}
+
+func (c *ComponentConfig) RetrieveAll(ctx context.Context) (map[string]string, error) {
+	key := c.makeConfigPath()
+
+	logger.Debugw(ctx, "retreiving-list", log.Fields{"key": key})
+
+	data, err := c.cManager.Backend.List(ctx, key)
+	if err != nil {
+		return nil, err
+	}
+
+	// Looping through the data recieved from the Backend for the given key
+	// Trimming the required key and value from data and  storing as key/value pair
+	// For Example, recieved key would be <Backend Prefix Path>/<Config Prefix>/<Component Name>/<Config Type>/default and value \"DEBUG\"
+	// Then in default will be stored as key and DEBUG will be stored as value in map[string]string
+	res := make(map[string]string)
+	ccPathPrefix := c.cManager.Backend.PathPrefix + kvStorePathSeparator + key + kvStorePathSeparator
+	for attr, val := range data {
+		res[strings.TrimPrefix(attr, ccPathPrefix)] = strings.Trim(fmt.Sprintf("%s", val.Value), "\"")
+	}
+
+	return res, nil
+}
+
+func (c *ComponentConfig) Save(ctx context.Context, configKey string, configValue string) error {
+	key := c.makeConfigPath() + "/" + configKey
+
+	logger.Debugw(ctx, "saving-config", log.Fields{"key": key, "value": configValue})
+
+	//save the data for update config
+	if err := c.cManager.Backend.Put(ctx, key, configValue); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (c *ComponentConfig) Delete(ctx context.Context, configKey string) error {
+	//construct key using makeConfigPath
+	key := c.makeConfigPath() + "/" + configKey
+
+	logger.Debugw(ctx, "deleting-config", log.Fields{"key": key})
+	//delete the config
+	if err := c.cManager.Backend.Delete(ctx, key); err != nil {
+		return err
+	}
+	return nil
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logcontroller.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logcontroller.go
new file mode 100644
index 0000000..b58f999
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logcontroller.go
@@ -0,0 +1,387 @@
+/*
+ * 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 Config provides dynamic logging configuration for specific Voltha component with loglevel lookup
+// from etcd kvstore implemented using Backend.
+// Any Voltha component can start utilizing dynamic logging by starting goroutine of StartLogLevelConfigProcessing after
+// starting kvClient for the component.
+
+package config
+
+import (
+	"context"
+	"crypto/md5"
+	"encoding/json"
+	"errors"
+	"os"
+	"sort"
+	"strings"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	defaultLogLevelKey                = "default"          // kvstore key containing default loglevel
+	globalConfigRootNode              = "global"           // Root Node in kvstore containing global config
+	initialGlobalDefaultLogLevelValue = "WARN"             // Hard-coded Global Default loglevel pushed at PoD startup
+	logPackagesListKey                = "log_package_list" // kvstore key containing list of allowed log packages
+)
+
+// ComponentLogController represents a Configuration for Logging Config of specific Voltha component type
+// It stores ComponentConfig and GlobalConfig of loglevel config of specific Voltha component type
+// For example,ComponentLogController instance will be created for rw-core component
+type ComponentLogController struct {
+	ComponentName       string
+	componentNameConfig *ComponentConfig
+	GlobalConfig        *ComponentConfig
+	configManager       *ConfigManager
+	logHash             [16]byte
+	initialLogLevel     string // Initial default log level set by helm chart
+}
+
+func NewComponentLogController(ctx context.Context, cm *ConfigManager) (*ComponentLogController, error) {
+	logger.Debug(ctx, "creating-new-component-log-controller")
+	componentName := os.Getenv("COMPONENT_NAME")
+	if componentName == "" {
+		return nil, errors.New("Unable to retrieve PoD Component Name from Runtime env")
+	}
+
+	var defaultLogLevel string
+	var err error
+	// Retrieve and save default log level; used for fallback if all loglevel config is cleared in etcd
+	if defaultLogLevel, err = log.LogLevelToString(log.GetDefaultLogLevel()); err != nil {
+		defaultLogLevel = "DEBUG"
+	}
+
+	return &ComponentLogController{
+		ComponentName:       componentName,
+		componentNameConfig: nil,
+		GlobalConfig:        nil,
+		configManager:       cm,
+		initialLogLevel:     defaultLogLevel,
+	}, nil
+
+}
+
+// StartLogLevelConfigProcessing initialize component config and global config
+// Then, it persists initial default Loglevels into Config Store before
+// starting the loading and processing of all Log Configuration
+func StartLogLevelConfigProcessing(cm *ConfigManager, ctx context.Context) {
+	cc, err := NewComponentLogController(ctx, cm)
+	if err != nil {
+		logger.Errorw(ctx, "unable-to-construct-component-log-controller-instance-for-log-config-monitoring", log.Fields{"error": err})
+		return
+	}
+
+	cc.GlobalConfig = cm.InitComponentConfig(globalConfigRootNode, ConfigTypeLogLevel)
+	logger.Debugw(ctx, "global-log-config", log.Fields{"cc-global-config": cc.GlobalConfig})
+
+	cc.componentNameConfig = cm.InitComponentConfig(cc.ComponentName, ConfigTypeLogLevel)
+	logger.Debugw(ctx, "component-log-config", log.Fields{"cc-component-name-config": cc.componentNameConfig})
+
+	cc.persistInitialDefaultLogConfigs(ctx)
+
+	cc.persistRegisteredLogPackageList(ctx)
+
+	cc.processLogConfig(ctx)
+}
+
+// Method to persist Global default loglevel into etcd, if not set yet
+// It also checks and set Component default loglevel into etcd with initial loglevel set from command line
+func (c *ComponentLogController) persistInitialDefaultLogConfigs(ctx context.Context) {
+
+	_, err := c.GlobalConfig.Retrieve(ctx, defaultLogLevelKey)
+	if err != nil {
+		logger.Debugw(ctx, "failed-to-retrieve-global-default-log-config-at-startup", log.Fields{"error": err})
+
+		err = c.GlobalConfig.Save(ctx, defaultLogLevelKey, initialGlobalDefaultLogLevelValue)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-persist-global-default-log-config-at-startup", log.Fields{"error": err, "loglevel": initialGlobalDefaultLogLevelValue})
+		}
+	}
+
+	_, err = c.componentNameConfig.Retrieve(ctx, defaultLogLevelKey)
+	if err != nil {
+		logger.Debugw(ctx, "failed-to-retrieve-component-default-log-config-at-startup", log.Fields{"error": err})
+
+		err = c.componentNameConfig.Save(ctx, defaultLogLevelKey, c.initialLogLevel)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-persist-component-default-log-config-at-startup", log.Fields{"error": err, "loglevel": c.initialLogLevel})
+		}
+	}
+}
+
+// Method to save list of all registered packages for component into config kvstore. A single string
+// is constructed with comma-separated package names in sorted order and persisted
+func (c *ComponentLogController) persistRegisteredLogPackageList(ctx context.Context) {
+
+	componentMetadataConfig := c.configManager.InitComponentConfig(c.ComponentName, ConfigTypeMetadata)
+	logger.Debugw(ctx, "component-metadata-config", log.Fields{"component-metadata-config": componentMetadataConfig})
+
+	packageList := log.GetPackageNames()
+	packageList = append(packageList, defaultLogLevelKey)
+	sort.Strings(packageList)
+
+	packageNames, err := json.Marshal(packageList)
+	if err != nil {
+		logger.Errorw(ctx, "failed-to-marshal-log-package-list-for-storage", log.Fields{"error": err, "packageList": packageList})
+		return
+	}
+
+	if err := componentMetadataConfig.Save(ctx, logPackagesListKey, string(packageNames)); err != nil {
+		logger.Errorw(ctx, "failed-to-persist-component-registered-log-package-list-at-startup", log.Fields{"error": err, "packageNames": packageNames})
+	}
+}
+
+// ProcessLogConfig will first load and apply log config and then start waiting on component config and global config
+// channels for any changes. Event channel will be recieved from Backend for valid change type
+// Then data for componentn log config and global log config will be retrieved from Backend and stored in updatedLogConfig in precedence order
+// If any changes in updatedLogConfig will be applied on component
+func (c *ComponentLogController) processLogConfig(ctx context.Context) {
+
+	// Load and apply Log Config for first time
+	initialLogConfig, err := c.buildUpdatedLogConfig(ctx)
+	if err != nil {
+		logger.Warnw(ctx, "unable-to-load-log-config-at-startup", log.Fields{"error": err})
+	} else {
+		if err := c.loadAndApplyLogConfig(ctx, initialLogConfig); err != nil {
+			logger.Warnw(ctx, "unable-to-apply-log-config-at-startup", log.Fields{"error": err})
+		}
+	}
+
+	componentConfigEventChan := c.componentNameConfig.MonitorForConfigChange(ctx)
+
+	globalConfigEventChan := c.GlobalConfig.MonitorForConfigChange(ctx)
+
+	// process the events for componentName and  global config
+	var configEvent *ConfigChangeEvent
+	for {
+		select {
+		case configEvent = <-globalConfigEventChan:
+		case configEvent = <-componentConfigEventChan:
+
+		}
+		logger.Debugw(ctx, "processing-log-config-change", log.Fields{"ChangeType": configEvent.ChangeType, "Package": configEvent.ConfigAttribute})
+
+		updatedLogConfig, err := c.buildUpdatedLogConfig(ctx)
+		if err != nil {
+			logger.Warnw(ctx, "unable-to-fetch-updated-log-config", log.Fields{"error": err})
+			continue
+		}
+
+		logger.Debugw(ctx, "applying-updated-log-config", log.Fields{"updated-log-config": updatedLogConfig})
+
+		if err := c.loadAndApplyLogConfig(ctx, updatedLogConfig); err != nil {
+			logger.Warnw(ctx, "unable-to-load-and-apply-log-config", log.Fields{"error": err})
+		}
+	}
+
+}
+
+// get active loglevel from the zap logger
+func getActiveLogLevels(ctx context.Context) map[string]string {
+	loglevels := make(map[string]string)
+
+	// now do the default log level
+	if level, err := log.LogLevelToString(log.GetDefaultLogLevel()); err == nil {
+		loglevels[defaultLogLevelKey] = level
+	}
+
+	// do the per-package log levels
+	for _, packageName := range log.GetPackageNames() {
+		level, err := log.GetPackageLogLevel(packageName)
+		if err != nil {
+			logger.Warnw(ctx, "unable-to-fetch-current-active-loglevel-for-package-name", log.Fields{"package-name": packageName, "error": err})
+			continue
+		}
+
+		if l, err := log.LogLevelToString(level); err == nil {
+			loglevels[packageName] = l
+		}
+	}
+
+	logger.Debugw(ctx, "retreived-log-levels-from-zap-logger", log.Fields{"loglevels": loglevels})
+
+	return loglevels
+}
+
+func (c *ComponentLogController) getGlobalLogConfig(ctx context.Context) (string, error) {
+
+	globalDefaultLogLevel, err := c.GlobalConfig.Retrieve(ctx, defaultLogLevelKey)
+	if err != nil {
+		return "", err
+	}
+
+	// Handle edge cases when global default loglevel is deleted directly from etcd or set to a invalid value
+	// We should use hard-coded initial default value in such cases
+	if globalDefaultLogLevel == "" {
+		logger.Warn(ctx, "global-default-loglevel-not-found-in-config-store")
+		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
+	}
+
+	if _, err := log.StringToLogLevel(globalDefaultLogLevel); err != nil {
+		logger.Warnw(ctx, "unsupported-loglevel-config-defined-at-global-default", log.Fields{"log-level": globalDefaultLogLevel})
+		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
+	}
+
+	logger.Debugw(ctx, "retrieved-global-default-loglevel", log.Fields{"level": globalDefaultLogLevel})
+
+	return globalDefaultLogLevel, nil
+}
+
+func (c *ComponentLogController) getComponentLogConfig(ctx context.Context, globalDefaultLogLevel string) (map[string]string, error) {
+	componentLogConfig, err := c.componentNameConfig.RetrieveAll(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	effectiveDefaultLogLevel := ""
+	for logConfigKey, logConfigValue := range componentLogConfig {
+		if _, err := log.StringToLogLevel(logConfigValue); err != nil || logConfigKey == "" {
+			logger.Warnw(ctx, "unsupported-loglevel-config-defined-at-component-context", log.Fields{"package-name": logConfigKey, "log-level": logConfigValue})
+			delete(componentLogConfig, logConfigKey)
+		} else {
+			if logConfigKey == defaultLogLevelKey {
+				effectiveDefaultLogLevel = componentLogConfig[defaultLogLevelKey]
+			}
+		}
+	}
+
+	// if default loglevel is not configured for the component, component should use
+	// default loglevel configured at global level
+	if effectiveDefaultLogLevel == "" {
+		effectiveDefaultLogLevel = globalDefaultLogLevel
+	}
+
+	componentLogConfig[defaultLogLevelKey] = effectiveDefaultLogLevel
+
+	logger.Debugw(ctx, "retrieved-component-log-config", log.Fields{"component-log-level": componentLogConfig})
+
+	return componentLogConfig, nil
+}
+
+// buildUpdatedLogConfig retrieve the global logConfig and component logConfig  from Backend
+// component logConfig stores the log config with precedence order
+// For example, If the global logConfig is set and component logConfig is set only for specific package then
+// component logConfig is stored with global logConfig  and component logConfig of specific package
+// For example, If the global logConfig is set and component logConfig is set for specific package and as well as for default then
+// component logConfig is stored with  component logConfig data only
+func (c *ComponentLogController) buildUpdatedLogConfig(ctx context.Context) (map[string]string, error) {
+	globalLogLevel, err := c.getGlobalLogConfig(ctx)
+	if err != nil {
+		logger.Errorw(ctx, "unable-to-retrieve-global-log-config", log.Fields{"err": err})
+	}
+
+	componentLogConfig, err := c.getComponentLogConfig(ctx, globalLogLevel)
+	if err != nil {
+		return nil, err
+	}
+
+	finalLogConfig := make(map[string]string)
+	for packageName, logLevel := range componentLogConfig {
+		finalLogConfig[strings.ReplaceAll(packageName, "#", "/")] = logLevel
+	}
+
+	return finalLogConfig, nil
+}
+
+// load and apply the current configuration for component name
+// create hash of loaded configuration using GenerateLogConfigHash
+// if there is previous hash stored, compare the hash to stored hash
+// if there is any change will call UpdateLogLevels
+func (c *ComponentLogController) loadAndApplyLogConfig(ctx context.Context, logConfig map[string]string) error {
+	currentLogHash, err := GenerateLogConfigHash(logConfig)
+	if err != nil {
+		return err
+	}
+
+	if c.logHash != currentLogHash {
+		updateLogLevels(ctx, logConfig)
+		c.logHash = currentLogHash
+	} else {
+		logger.Debug(ctx, "effective-loglevel-config-same-as-currently-active")
+	}
+
+	return nil
+}
+
+// createModifiedLogLevels loops through the activeLogLevels recieved from zap logger and updatedLogLevels recieved from buildUpdatedLogConfig
+// to identify and create map of modified Log Levels of 2 types:
+// - Packages for which log level has been changed
+// - Packages for which log level config has been cleared - set to default log level
+func createModifiedLogLevels(ctx context.Context, activeLogLevels, updatedLogLevels map[string]string) map[string]string {
+	defaultLevel := updatedLogLevels[defaultLogLevelKey]
+
+	modifiedLogLevels := make(map[string]string)
+	for activeKey, activeLevel := range activeLogLevels {
+		if _, exist := updatedLogLevels[activeKey]; !exist {
+			if activeLevel != defaultLevel {
+				modifiedLogLevels[activeKey] = defaultLevel
+			}
+		} else if activeLevel != updatedLogLevels[activeKey] {
+			modifiedLogLevels[activeKey] = updatedLogLevels[activeKey]
+		}
+	}
+
+	// Log warnings for all invalid packages for which log config has been set
+	for key, value := range updatedLogLevels {
+		if _, exist := activeLogLevels[key]; !exist {
+			logger.Warnw(ctx, "ignoring-loglevel-set-for-invalid-package", log.Fields{"package": key, "log-level": value})
+		}
+	}
+
+	return modifiedLogLevels
+}
+
+// updateLogLevels update the loglevels for the component
+// retrieve active confguration from logger
+// compare with entries one by one and apply
+func updateLogLevels(ctx context.Context, updatedLogConfig map[string]string) {
+
+	activeLogLevels := getActiveLogLevels(ctx)
+	changedLogLevels := createModifiedLogLevels(ctx, activeLogLevels, updatedLogConfig)
+
+	// If no changed log levels are found, just return. It may happen on configuration of a invalid package
+	if len(changedLogLevels) == 0 {
+		logger.Debug(ctx, "no-change-in-effective-loglevel-config")
+		return
+	}
+
+	logger.Debugw(ctx, "applying-log-level-for-modified-packages", log.Fields{"changed-log-levels": changedLogLevels})
+	for key, level := range changedLogLevels {
+		if key == defaultLogLevelKey {
+			if l, err := log.StringToLogLevel(level); err == nil {
+				log.SetDefaultLogLevel(l)
+			}
+		} else {
+			if l, err := log.StringToLogLevel(level); err == nil {
+				log.SetPackageLogLevel(key, l)
+			}
+		}
+	}
+}
+
+// generate md5 hash of key value pairs appended into a single string
+// in order by key name
+func GenerateLogConfigHash(createHashLog map[string]string) ([16]byte, error) {
+	createHashLogBytes := []byte{}
+	levelData, err := json.Marshal(createHashLog)
+	if err != nil {
+		return [16]byte{}, err
+	}
+	createHashLogBytes = append(createHashLogBytes, levelData...)
+	return md5.Sum(createHashLogBytes), nil
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logfeaturescontroller.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logfeaturescontroller.go
new file mode 100644
index 0000000..579c1de
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/config/logfeaturescontroller.go
@@ -0,0 +1,173 @@
+/*
+ * 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 config
+
+import (
+	"context"
+	"errors"
+	"os"
+	"strings"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+const (
+	defaultTracingStatusKey        = "trace_publish"   // kvstore key containing tracing configuration status
+	defaultLogCorrelationStatusKey = "log_correlation" // kvstore key containing log correlation configuration status
+)
+
+// ComponentLogFeatureController represents Configuration for Logging related features of Tracing and Log
+// Correlation of specific Voltha component.
+type ComponentLogFeaturesController struct {
+	ComponentName               string
+	componentNameConfig         *ComponentConfig
+	configManager               *ConfigManager
+	initialTracingStatus        bool // Initial default tracing status set by helm chart
+	initialLogCorrelationStatus bool // Initial default log correlation status set by helm chart
+}
+
+func NewComponentLogFeaturesController(ctx context.Context, cm *ConfigManager) (*ComponentLogFeaturesController, error) {
+	logger.Debug(ctx, "creating-new-component-log-features-controller")
+	componentName := os.Getenv("COMPONENT_NAME")
+	if componentName == "" {
+		return nil, errors.New("Unable to retrieve PoD Component Name from Runtime env")
+	}
+
+	tracingStatus := log.GetGlobalLFM().GetTracePublishingStatus()
+	logCorrelationStatus := log.GetGlobalLFM().GetLogCorrelationStatus()
+
+	return &ComponentLogFeaturesController{
+		ComponentName:               componentName,
+		componentNameConfig:         nil,
+		configManager:               cm,
+		initialTracingStatus:        tracingStatus,
+		initialLogCorrelationStatus: logCorrelationStatus,
+	}, nil
+
+}
+
+// StartLogFeaturesConfigProcessing persists initial config of Log Features into Config Store before
+// starting the loading and processing of Configuration updates
+func StartLogFeaturesConfigProcessing(cm *ConfigManager, ctx context.Context) {
+	cc, err := NewComponentLogFeaturesController(ctx, cm)
+	if err != nil {
+		logger.Errorw(ctx, "unable-to-construct-component-log-features-controller-instance-for-monitoring", log.Fields{"error": err})
+		return
+	}
+
+	cc.componentNameConfig = cm.InitComponentConfig(cc.ComponentName, ConfigTypeLogFeatures)
+	logger.Debugw(ctx, "component-log-features-config", log.Fields{"cc-component-name-config": cc.componentNameConfig})
+
+	cc.persistInitialLogFeaturesConfigs(ctx)
+
+	cc.processLogFeaturesConfig(ctx)
+}
+
+// Method to persist Initial status of Log Correlation and Tracing features (as set from command line)
+// into config store (etcd kvstore), if not set yet
+func (cc *ComponentLogFeaturesController) persistInitialLogFeaturesConfigs(ctx context.Context) {
+
+	_, err := cc.componentNameConfig.Retrieve(ctx, defaultTracingStatusKey)
+	if err != nil {
+		statusString := "DISABLED"
+		if cc.initialTracingStatus {
+			statusString = "ENABLED"
+		}
+		err = cc.componentNameConfig.Save(ctx, defaultTracingStatusKey, statusString)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-persist-component-initial-tracing-status-at-startup", log.Fields{"error": err, "tracingstatus": statusString})
+		}
+	}
+
+	_, err = cc.componentNameConfig.Retrieve(ctx, defaultLogCorrelationStatusKey)
+	if err != nil {
+		statusString := "DISABLED"
+		if cc.initialLogCorrelationStatus {
+			statusString = "ENABLED"
+		}
+		err = cc.componentNameConfig.Save(ctx, defaultLogCorrelationStatusKey, statusString)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-persist-component-initial-log-correlation-status-at-startup", log.Fields{"error": err, "logcorrelationstatus": statusString})
+		}
+	}
+}
+
+// processLogFeaturesConfig will first load and apply configuration of log features. Then it will start waiting for any changes
+// made to configuration in config store (etcd) and apply the same
+func (cc *ComponentLogFeaturesController) processLogFeaturesConfig(ctx context.Context) {
+
+	// Load and apply Tracing Status and log correlation status for first time
+	cc.loadAndApplyTracingStatusUpdate(ctx)
+	cc.loadAndApplyLogCorrelationStatusUpdate(ctx)
+
+	componentConfigEventChan := cc.componentNameConfig.MonitorForConfigChange(ctx)
+
+	// process the change events received on the channel
+	var configEvent *ConfigChangeEvent
+	for {
+		select {
+		case <-ctx.Done():
+			return
+
+		case configEvent = <-componentConfigEventChan:
+			logger.Debugw(ctx, "processing-log-features-config-change", log.Fields{"ChangeType": configEvent.ChangeType, "Package": configEvent.ConfigAttribute})
+
+			if strings.HasSuffix(configEvent.ConfigAttribute, defaultTracingStatusKey) {
+				cc.loadAndApplyTracingStatusUpdate(ctx)
+			} else if strings.HasSuffix(configEvent.ConfigAttribute, defaultLogCorrelationStatusKey) {
+				cc.loadAndApplyLogCorrelationStatusUpdate(ctx)
+			}
+		}
+	}
+
+}
+
+func (cc *ComponentLogFeaturesController) loadAndApplyTracingStatusUpdate(ctx context.Context) {
+
+	desiredTracingStatus, err := cc.componentNameConfig.Retrieve(ctx, defaultTracingStatusKey)
+	if err != nil || desiredTracingStatus == "" {
+		logger.Warn(ctx, "unable-to-retrieve-tracing-status-from-config-store")
+		return
+	}
+
+	if desiredTracingStatus != "ENABLED" && desiredTracingStatus != "DISABLED" {
+		logger.Warnw(ctx, "unsupported-tracing-status-configured-in-config-store", log.Fields{"failed-tracing-status": desiredTracingStatus, "tracing-status": log.GetGlobalLFM().GetTracePublishingStatus()})
+		return
+	}
+
+	logger.Debugw(ctx, "retrieved-tracing-status", log.Fields{"tracing-status": desiredTracingStatus})
+
+	log.GetGlobalLFM().SetTracePublishingStatus(desiredTracingStatus == "ENABLED")
+}
+
+func (cc *ComponentLogFeaturesController) loadAndApplyLogCorrelationStatusUpdate(ctx context.Context) {
+
+	desiredLogCorrelationStatus, err := cc.componentNameConfig.Retrieve(ctx, defaultLogCorrelationStatusKey)
+	if err != nil || desiredLogCorrelationStatus == "" {
+		logger.Warn(ctx, "unable-to-retrieve-log-correlation-status-from-config-store")
+		return
+	}
+
+	if desiredLogCorrelationStatus != "ENABLED" && desiredLogCorrelationStatus != "DISABLED" {
+		logger.Warnw(ctx, "unsupported-log-correlation-status-configured-in-config-store", log.Fields{"failed-log-correlation-status": desiredLogCorrelationStatus, "log-correlation-status": log.GetGlobalLFM().GetLogCorrelationStatus()})
+		return
+	}
+
+	logger.Debugw(ctx, "retrieved-log-correlation-status", log.Fields{"log-correlation-status": desiredLogCorrelationStatus})
+
+	log.GetGlobalLFM().SetLogCorrelationStatus(desiredLogCorrelationStatus == "ENABLED")
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/backend.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/backend.go
new file mode 100644
index 0000000..2e57a27
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/backend.go
@@ -0,0 +1,272 @@
+/*
+ * 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 db
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+const (
+	// Default Minimal Interval for posting alive state of backend kvstore on Liveness Channel
+	DefaultLivenessChannelInterval = time.Second * 30
+)
+
+// Backend structure holds details for accessing the kv store
+type Backend struct {
+	Client                  kvstore.Client
+	StoreType               string
+	Timeout                 time.Duration
+	Address                 string
+	PathPrefix              string
+	alive                   bool // Is this backend connection alive?
+	livenessMutex           sync.Mutex
+	liveness                chan bool     // channel to post alive state
+	LivenessChannelInterval time.Duration // regularly push alive state beyond this interval
+	lastLivenessTime        time.Time     // Instant of last alive state push
+}
+
+// NewBackend creates a new instance of a Backend structure
+func NewBackend(ctx context.Context, storeType string, address string, timeout time.Duration, pathPrefix string) *Backend {
+	var err error
+
+	b := &Backend{
+		StoreType:               storeType,
+		Address:                 address,
+		Timeout:                 timeout,
+		LivenessChannelInterval: DefaultLivenessChannelInterval,
+		PathPrefix:              pathPrefix,
+		alive:                   false, // connection considered down at start
+	}
+
+	if b.Client, err = b.newClient(ctx, address, timeout); err != nil {
+		logger.Errorw(ctx, "failed-to-create-kv-client",
+			log.Fields{
+				"type": storeType, "address": address,
+				"timeout": timeout, "prefix": pathPrefix,
+				"error": err.Error(),
+			})
+	}
+
+	return b
+}
+
+func (b *Backend) newClient(ctx context.Context, address string, timeout time.Duration) (kvstore.Client, error) {
+	switch b.StoreType {
+	case "etcd":
+		return kvstore.NewEtcdClient(ctx, address, timeout, log.WarnLevel)
+	}
+	return nil, errors.New("unsupported-kv-store")
+}
+
+func (b *Backend) makePath(ctx context.Context, key string) string {
+	path := fmt.Sprintf("%s/%s", b.PathPrefix, key)
+	return path
+}
+
+func (b *Backend) updateLiveness(ctx context.Context, alive bool) {
+	// Periodically push stream of liveness data to the channel,
+	// so that in a live state, the core does not timeout and
+	// send a forced liveness message. Push alive state if the
+	// last push to channel was beyond livenessChannelInterval
+	b.livenessMutex.Lock()
+	defer b.livenessMutex.Unlock()
+	if b.liveness != nil {
+		if b.alive != alive {
+			logger.Debug(ctx, "update-liveness-channel-reason-change")
+			b.liveness <- alive
+			b.lastLivenessTime = time.Now()
+		} else if time.Since(b.lastLivenessTime) > b.LivenessChannelInterval {
+			logger.Debug(ctx, "update-liveness-channel-reason-interval")
+			b.liveness <- alive
+			b.lastLivenessTime = time.Now()
+		}
+	}
+
+	// Emit log message only for alive state change
+	if b.alive != alive {
+		logger.Debugw(ctx, "change-kvstore-alive-status", log.Fields{"alive": alive})
+		b.alive = alive
+	}
+}
+
+// Perform a dummy Key Lookup on kvstore to test Connection Liveness and
+// post on Liveness channel
+func (b *Backend) PerformLivenessCheck(ctx context.Context) bool {
+	alive := b.Client.IsConnectionUp(ctx)
+	logger.Debugw(ctx, "kvstore-liveness-check-result", log.Fields{"alive": alive})
+
+	b.updateLiveness(ctx, alive)
+	return alive
+}
+
+// Enable the liveness monitor channel. This channel will report
+// a "true" or "false" on every kvstore operation which indicates whether
+// or not the connection is still Live. This channel is then picked up
+// by the service (i.e. rw_core / ro_core) to update readiness status
+// and/or take other actions.
+func (b *Backend) EnableLivenessChannel(ctx context.Context) chan bool {
+	logger.Debug(ctx, "enable-kvstore-liveness-channel")
+	b.livenessMutex.Lock()
+	defer b.livenessMutex.Unlock()
+	if b.liveness == nil {
+		b.liveness = make(chan bool, 10)
+		b.liveness <- b.alive
+		b.lastLivenessTime = time.Now()
+	}
+
+	return b.liveness
+}
+
+// Extract Alive status of Kvstore based on type of error
+func (b *Backend) isErrorIndicatingAliveKvstore(ctx context.Context, err error) bool {
+	// Alive unless observed an error indicating so
+	alive := true
+
+	if err != nil {
+
+		// timeout indicates kvstore not reachable/alive
+		if err == context.DeadlineExceeded {
+			alive = false
+		}
+
+		// Need to analyze client-specific errors based on backend type
+		if b.StoreType == "etcd" {
+
+			// For etcd backend, consider not-alive only for errors indicating
+			// timedout request or unavailable/corrupted cluster. For all remaining
+			// error codes listed in https://godoc.org/google.golang.org/grpc/codes#Code,
+			// we would not infer a not-alive backend because such a error may also
+			// occur due to bad client requests or sequence of operations
+			switch status.Code(err) {
+			case codes.DeadlineExceeded:
+				fallthrough
+			case codes.Unavailable:
+				fallthrough
+			case codes.DataLoss:
+				alive = false
+			}
+		}
+	}
+
+	return alive
+}
+
+// List retrieves one or more items that match the specified key
+func (b *Backend) List(ctx context.Context, key string) (map[string]*kvstore.KVPair, error) {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-list")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "listing-key", log.Fields{"key": key, "path": formattedPath})
+
+	pair, err := b.Client.List(ctx, formattedPath)
+
+	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
+
+	return pair, err
+}
+
+// Get retrieves an item that matches the specified key
+func (b *Backend) Get(ctx context.Context, key string) (*kvstore.KVPair, error) {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-get")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "getting-key", log.Fields{"key": key, "path": formattedPath})
+
+	pair, err := b.Client.Get(ctx, formattedPath)
+
+	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
+
+	return pair, err
+}
+
+// Put stores an item value under the specifed key
+func (b *Backend) Put(ctx context.Context, key string, value interface{}) error {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-put")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "putting-key", log.Fields{"key": key, "path": formattedPath})
+
+	err := b.Client.Put(ctx, formattedPath, value)
+
+	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
+
+	return err
+}
+
+// Delete removes an item under the specified key
+func (b *Backend) Delete(ctx context.Context, key string) error {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-delete")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "deleting-key", log.Fields{"key": key, "path": formattedPath})
+
+	err := b.Client.Delete(ctx, formattedPath)
+
+	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
+
+	return err
+}
+
+// DeleteWithPrefix removes items having prefix key
+func (b *Backend) DeleteWithPrefix(ctx context.Context, prefixKey string) error {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-delete-with-prefix")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, prefixKey)
+	logger.Debugw(ctx, "deleting-prefix-key", log.Fields{"key": prefixKey, "path": formattedPath})
+
+	err := b.Client.DeleteWithPrefix(ctx, formattedPath)
+
+	b.updateLiveness(ctx, b.isErrorIndicatingAliveKvstore(ctx, err))
+
+	return err
+}
+
+// CreateWatch starts watching events for the specified key
+func (b *Backend) CreateWatch(ctx context.Context, key string, withPrefix bool) chan *kvstore.Event {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-create-watch")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "creating-key-watch", log.Fields{"key": key, "path": formattedPath})
+
+	return b.Client.Watch(ctx, formattedPath, withPrefix)
+}
+
+// DeleteWatch stops watching events for the specified key
+func (b *Backend) DeleteWatch(ctx context.Context, key string, ch chan *kvstore.Event) {
+	span, ctx := log.CreateChildSpan(ctx, "etcd-delete-watch")
+	defer span.Finish()
+
+	formattedPath := b.makePath(ctx, key)
+	logger.Debugw(ctx, "deleting-key-watch", log.Fields{"key": key, "path": formattedPath})
+
+	b.Client.CloseWatch(ctx, formattedPath, ch)
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/common.go
new file mode 100644
index 0000000..d8a0571
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 db
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/client.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/client.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/client.go
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/common.go
new file mode 100644
index 0000000..8ac2a4a
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 kvstore
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdclient.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdclient.go
new file mode 100644
index 0000000..6ca5329
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdclient.go
@@ -0,0 +1,474 @@
+/*
+ * 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 kvstore
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"os"
+	"strconv"
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	v3Client "go.etcd.io/etcd/clientv3"
+	v3rpcTypes "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+)
+
+const (
+	poolCapacityEnvName = "VOLTHA_ETCD_CLIENT_POOL_CAPACITY"
+	maxUsageEnvName     = "VOLTHA_ETCD_CLIENT_MAX_USAGE"
+)
+
+const (
+	defaultMaxPoolCapacity = 1000 // Default size of an Etcd Client pool
+	defaultMaxPoolUsage    = 100  // Maximum concurrent request an Etcd Client is allowed to process
+)
+
+// EtcdClient represents the Etcd KV store client
+type EtcdClient struct {
+	pool               EtcdClientAllocator
+	watchedChannels    sync.Map
+	watchedClients     map[string]*v3Client.Client
+	watchedClientsLock sync.RWMutex
+}
+
+// NewEtcdCustomClient returns a new client for the Etcd KV store allowing
+// the called to specify etcd client configuration
+func NewEtcdCustomClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
+	// Get the capacity and max usage from the environment
+	capacity := defaultMaxPoolCapacity
+	maxUsage := defaultMaxPoolUsage
+	if capacityStr, present := os.LookupEnv(poolCapacityEnvName); present {
+		if val, err := strconv.Atoi(capacityStr); err == nil {
+			capacity = val
+			logger.Infow(ctx, "env-variable-set", log.Fields{"pool-capacity": capacity})
+		} else {
+			logger.Warnw(ctx, "invalid-capacity-value", log.Fields{"error": err, "capacity": capacityStr})
+		}
+	}
+	if maxUsageStr, present := os.LookupEnv(maxUsageEnvName); present {
+		if val, err := strconv.Atoi(maxUsageStr); err == nil {
+			maxUsage = val
+			logger.Infow(ctx, "env-variable-set", log.Fields{"max-usage": maxUsage})
+		} else {
+			logger.Warnw(ctx, "invalid-max-usage-value", log.Fields{"error": err, "max-usage": maxUsageStr})
+		}
+	}
+
+	var err error
+
+	pool, err := NewRoundRobinEtcdClientAllocator([]string{addr}, timeout, capacity, maxUsage, level)
+	if err != nil {
+		logger.Errorw(ctx, "failed-to-create-rr-client", log.Fields{
+			"error": err,
+		})
+	}
+
+	logger.Infow(ctx, "etcd-pool-created", log.Fields{"capacity": capacity, "max-usage": maxUsage})
+
+	return &EtcdClient{pool: pool,
+		watchedClients: make(map[string]*v3Client.Client),
+	}, nil
+}
+
+// NewEtcdClient returns a new client for the Etcd KV store
+func NewEtcdClient(ctx context.Context, addr string, timeout time.Duration, level log.LogLevel) (*EtcdClient, error) {
+	return NewEtcdCustomClient(ctx, addr, timeout, level)
+}
+
+// IsConnectionUp returns whether the connection to the Etcd KV store is up.  If a timeout occurs then
+// it is assumed the connection is down or unreachable.
+func (c *EtcdClient) IsConnectionUp(ctx context.Context) bool {
+	// Let's try to get a non existent key.  If the connection is up then there will be no error returned.
+	if _, err := c.Get(ctx, "non-existent-key"); err != nil {
+		return false
+	}
+	return true
+}
+
+// List returns an array of key-value pairs with key as a prefix.  Timeout defines how long the function will
+// wait for a response
+func (c *EtcdClient) List(ctx context.Context, key string) (map[string]*KVPair, error) {
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return nil, err
+	}
+	defer c.pool.Put(client)
+	resp, err := client.Get(ctx, key, v3Client.WithPrefix())
+
+	if err != nil {
+		logger.Error(ctx, err)
+		return nil, err
+	}
+	m := make(map[string]*KVPair)
+	for _, ev := range resp.Kvs {
+		m[string(ev.Key)] = NewKVPair(string(ev.Key), ev.Value, "", ev.Lease, ev.Version)
+	}
+	return m, nil
+}
+
+// Get returns a key-value pair for a given key. Timeout defines how long the function will
+// wait for a response
+func (c *EtcdClient) Get(ctx context.Context, key string) (*KVPair, error) {
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return nil, err
+	}
+	defer c.pool.Put(client)
+
+	attempt := 0
+
+startLoop:
+	for {
+		resp, err := client.Get(ctx, key)
+		if err != nil {
+			switch err {
+			case context.Canceled:
+				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
+			case context.DeadlineExceeded:
+				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
+			case v3rpcTypes.ErrEmptyKey:
+				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
+			case v3rpcTypes.ErrLeaderChanged,
+				v3rpcTypes.ErrGRPCNoLeader,
+				v3rpcTypes.ErrTimeout,
+				v3rpcTypes.ErrTimeoutDueToLeaderFail,
+				v3rpcTypes.ErrTimeoutDueToConnectionLost:
+				// Retry for these server errors
+				attempt += 1
+				if er := backoff(ctx, attempt); er != nil {
+					logger.Warnw(ctx, "get-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
+					return nil, err
+				}
+				logger.Warnw(ctx, "retrying-get", log.Fields{"key": key, "error": err, "attempt": attempt})
+				goto startLoop
+			default:
+				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
+			}
+			return nil, err
+		}
+
+		for _, ev := range resp.Kvs {
+			// Only one value is returned
+			return NewKVPair(string(ev.Key), ev.Value, "", ev.Lease, ev.Version), nil
+		}
+		return nil, nil
+	}
+}
+
+// Put writes a key-value pair to the KV store.  Value can only be a string or []byte since the etcd API
+// accepts only a string as a value for a put operation. Timeout defines how long the function will
+// wait for a response
+func (c *EtcdClient) Put(ctx context.Context, key string, value interface{}) error {
+
+	// Validate that we can convert value to a string as etcd API expects a string
+	var val string
+	var err error
+	if val, err = ToString(value); err != nil {
+		return fmt.Errorf("unexpected-type-%T", value)
+	}
+
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
+
+	attempt := 0
+startLoop:
+	for {
+		_, err = client.Put(ctx, key, val)
+		if err != nil {
+			switch err {
+			case context.Canceled:
+				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
+			case context.DeadlineExceeded:
+				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
+			case v3rpcTypes.ErrEmptyKey:
+				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
+			case v3rpcTypes.ErrLeaderChanged,
+				v3rpcTypes.ErrGRPCNoLeader,
+				v3rpcTypes.ErrTimeout,
+				v3rpcTypes.ErrTimeoutDueToLeaderFail,
+				v3rpcTypes.ErrTimeoutDueToConnectionLost:
+				// Retry for these server errors
+				attempt += 1
+				if er := backoff(ctx, attempt); er != nil {
+					logger.Warnw(ctx, "put-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
+					return err
+				}
+				logger.Warnw(ctx, "retrying-put", log.Fields{"key": key, "error": err, "attempt": attempt})
+				goto startLoop
+			default:
+				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
+			}
+			return err
+		}
+		return nil
+	}
+}
+
+// Delete removes a key from the KV store. Timeout defines how long the function will
+// wait for a response
+func (c *EtcdClient) Delete(ctx context.Context, key string) error {
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
+
+	attempt := 0
+startLoop:
+	for {
+		_, err = client.Delete(ctx, key)
+		if err != nil {
+			switch err {
+			case context.Canceled:
+				logger.Warnw(ctx, "context-cancelled", log.Fields{"error": err})
+			case context.DeadlineExceeded:
+				logger.Warnw(ctx, "context-deadline-exceeded", log.Fields{"error": err, "context": ctx})
+			case v3rpcTypes.ErrEmptyKey:
+				logger.Warnw(ctx, "etcd-client-error", log.Fields{"error": err})
+			case v3rpcTypes.ErrLeaderChanged,
+				v3rpcTypes.ErrGRPCNoLeader,
+				v3rpcTypes.ErrTimeout,
+				v3rpcTypes.ErrTimeoutDueToLeaderFail,
+				v3rpcTypes.ErrTimeoutDueToConnectionLost:
+				// Retry for these server errors
+				attempt += 1
+				if er := backoff(ctx, attempt); er != nil {
+					logger.Warnw(ctx, "delete-retries-failed", log.Fields{"key": key, "error": er, "attempt": attempt})
+					return err
+				}
+				logger.Warnw(ctx, "retrying-delete", log.Fields{"key": key, "error": err, "attempt": attempt})
+				goto startLoop
+			default:
+				logger.Warnw(ctx, "etcd-server-error", log.Fields{"error": err})
+			}
+			return err
+		}
+		logger.Debugw(ctx, "key(s)-deleted", log.Fields{"key": key})
+		return nil
+	}
+}
+
+func (c *EtcdClient) DeleteWithPrefix(ctx context.Context, prefixKey string) error {
+
+	client, err := c.pool.Get(ctx)
+	if err != nil {
+		return err
+	}
+	defer c.pool.Put(client)
+
+	//delete the prefix
+	if _, err := client.Delete(ctx, prefixKey, v3Client.WithPrefix()); err != nil {
+		logger.Errorw(ctx, "failed-to-delete-prefix-key", log.Fields{"key": prefixKey, "error": err})
+		return err
+	}
+	logger.Debugw(ctx, "key(s)-deleted", log.Fields{"key": prefixKey})
+	return nil
+}
+
+// Watch provides the watch capability on a given key.  It returns a channel onto which the callee needs to
+// listen to receive Events.
+func (c *EtcdClient) Watch(ctx context.Context, key string, withPrefix bool) chan *Event {
+	var err error
+	// Reuse the Etcd client when multiple callees are watching the same key.
+	c.watchedClientsLock.Lock()
+	client, exist := c.watchedClients[key]
+	if !exist {
+		client, err = c.pool.Get(ctx)
+		if err != nil {
+			logger.Errorw(ctx, "failed-to-an-etcd-client", log.Fields{"key": key, "error": err})
+			c.watchedClientsLock.Unlock()
+			return nil
+		}
+		c.watchedClients[key] = client
+	}
+	c.watchedClientsLock.Unlock()
+
+	w := v3Client.NewWatcher(client)
+	ctx, cancel := context.WithCancel(ctx)
+	var channel v3Client.WatchChan
+	if withPrefix {
+		channel = w.Watch(ctx, key, v3Client.WithPrefix())
+	} else {
+		channel = w.Watch(ctx, key)
+	}
+
+	// Create a new channel
+	ch := make(chan *Event, maxClientChannelBufferSize)
+
+	// Keep track of the created channels so they can be closed when required
+	channelMap := make(map[chan *Event]v3Client.Watcher)
+	channelMap[ch] = w
+	channelMaps := c.addChannelMap(key, channelMap)
+
+	// Changing the log field (from channelMaps) as the underlying logger cannot format the map of channels into a
+	// json format.
+	logger.Debugw(ctx, "watched-channels", log.Fields{"len": len(channelMaps)})
+	// Launch a go routine to listen for updates
+	go c.listenForKeyChange(ctx, channel, ch, cancel)
+
+	return ch
+
+}
+
+func (c *EtcdClient) addChannelMap(key string, channelMap map[chan *Event]v3Client.Watcher) []map[chan *Event]v3Client.Watcher {
+	var channels interface{}
+	var exists bool
+
+	if channels, exists = c.watchedChannels.Load(key); exists {
+		channels = append(channels.([]map[chan *Event]v3Client.Watcher), channelMap)
+	} else {
+		channels = []map[chan *Event]v3Client.Watcher{channelMap}
+	}
+	c.watchedChannels.Store(key, channels)
+
+	return channels.([]map[chan *Event]v3Client.Watcher)
+}
+
+func (c *EtcdClient) removeChannelMap(key string, pos int) []map[chan *Event]v3Client.Watcher {
+	var channels interface{}
+	var exists bool
+
+	if channels, exists = c.watchedChannels.Load(key); exists {
+		channels = append(channels.([]map[chan *Event]v3Client.Watcher)[:pos], channels.([]map[chan *Event]v3Client.Watcher)[pos+1:]...)
+		c.watchedChannels.Store(key, channels)
+	}
+
+	return channels.([]map[chan *Event]v3Client.Watcher)
+}
+
+func (c *EtcdClient) getChannelMaps(key string) ([]map[chan *Event]v3Client.Watcher, bool) {
+	var channels interface{}
+	var exists bool
+
+	channels, exists = c.watchedChannels.Load(key)
+
+	if channels == nil {
+		return nil, exists
+	}
+
+	return channels.([]map[chan *Event]v3Client.Watcher), exists
+}
+
+// CloseWatch closes a specific watch. Both the key and the channel are required when closing a watch as there
+// may be multiple listeners on the same key.  The previously created channel serves as a key
+func (c *EtcdClient) CloseWatch(ctx context.Context, key string, ch chan *Event) {
+	// Get the array of channels mapping
+	var watchedChannels []map[chan *Event]v3Client.Watcher
+	var ok bool
+
+	if watchedChannels, ok = c.getChannelMaps(key); !ok {
+		logger.Warnw(ctx, "key-has-no-watched-channels", log.Fields{"key": key})
+		return
+	}
+	// Look for the channels
+	var pos = -1
+	for i, chMap := range watchedChannels {
+		if t, ok := chMap[ch]; ok {
+			logger.Debug(ctx, "channel-found")
+			// Close the etcd watcher before the client channel.  This should close the etcd channel as well
+			if err := t.Close(); err != nil {
+				logger.Errorw(ctx, "watcher-cannot-be-closed", log.Fields{"key": key, "error": err})
+			}
+			pos = i
+			break
+		}
+	}
+
+	channelMaps, _ := c.getChannelMaps(key)
+	// Remove that entry if present
+	if pos >= 0 {
+		channelMaps = c.removeChannelMap(key, pos)
+	}
+
+	// If we don't have any keys being watched then return the Etcd client to the pool
+	if len(channelMaps) == 0 {
+		c.watchedClientsLock.Lock()
+		// Sanity
+		if client, ok := c.watchedClients[key]; ok {
+			c.pool.Put(client)
+			delete(c.watchedClients, key)
+		}
+		c.watchedClientsLock.Unlock()
+	}
+	logger.Infow(ctx, "watcher-channel-exiting", log.Fields{"key": key, "channel": channelMaps})
+}
+
+func (c *EtcdClient) listenForKeyChange(ctx context.Context, channel v3Client.WatchChan, ch chan<- *Event, cancel context.CancelFunc) {
+	logger.Debug(ctx, "start-listening-on-channel ...")
+	defer cancel()
+	defer close(ch)
+	for resp := range channel {
+		for _, ev := range resp.Events {
+			ch <- NewEvent(getEventType(ev), ev.Kv.Key, ev.Kv.Value, ev.Kv.Version)
+		}
+	}
+	logger.Debug(ctx, "stop-listening-on-channel ...")
+}
+
+func getEventType(event *v3Client.Event) int {
+	switch event.Type {
+	case v3Client.EventTypePut:
+		return PUT
+	case v3Client.EventTypeDelete:
+		return DELETE
+	}
+	return UNKNOWN
+}
+
+// Close closes all the connection in the pool store client
+func (c *EtcdClient) Close(ctx context.Context) {
+	logger.Debug(ctx, "closing-etcd-pool")
+	c.pool.Close(ctx)
+}
+
+// The APIs below are not used
+var errUnimplemented = errors.New("deprecated")
+
+// Reserve is deprecated
+func (c *EtcdClient) Reserve(ctx context.Context, key string, value interface{}, ttl time.Duration) (interface{}, error) {
+	return nil, errUnimplemented
+}
+
+// ReleaseAllReservations is deprecated
+func (c *EtcdClient) ReleaseAllReservations(ctx context.Context) error {
+	return errUnimplemented
+}
+
+// ReleaseReservation is deprecated
+func (c *EtcdClient) ReleaseReservation(ctx context.Context, key string) error {
+	return errUnimplemented
+}
+
+// RenewReservation is deprecated
+func (c *EtcdClient) RenewReservation(ctx context.Context, key string) error {
+	return errUnimplemented
+}
+
+// AcquireLock is deprecated
+func (c *EtcdClient) AcquireLock(ctx context.Context, lockName string, timeout time.Duration) error {
+	return errUnimplemented
+}
+
+// ReleaseLock is deprecated
+func (c *EtcdClient) ReleaseLock(lockName string) error {
+	return errUnimplemented
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdpool.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdpool.go
new file mode 100644
index 0000000..4d33c27
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/etcdpool.go
@@ -0,0 +1,240 @@
+/*
+ * Copyright 2021-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 kvstore
+
+import (
+	"container/list"
+	"context"
+	"errors"
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"go.etcd.io/etcd/clientv3"
+)
+
+// EtcdClientAllocator represents a generic interface to allocate an Etcd Client
+type EtcdClientAllocator interface {
+	Get(context.Context) (*clientv3.Client, error)
+	Put(*clientv3.Client)
+	Close(ctx context.Context)
+}
+
+// NewRoundRobinEtcdClientAllocator creates a new ETCD Client Allocator using a Round Robin scheme
+func NewRoundRobinEtcdClientAllocator(endpoints []string, timeout time.Duration, capacity, maxUsage int, level log.LogLevel) (EtcdClientAllocator, error) {
+	return &roundRobin{
+		all:       make(map[*clientv3.Client]*rrEntry),
+		full:      make(map[*clientv3.Client]*rrEntry),
+		waitList:  list.New(),
+		max:       maxUsage,
+		capacity:  capacity,
+		timeout:   timeout,
+		endpoints: endpoints,
+		logLevel:  level,
+		closingCh: make(chan struct{}, capacity*maxUsage),
+		stopCh:    make(chan struct{}),
+	}, nil
+}
+
+type rrEntry struct {
+	client *clientv3.Client
+	count  int
+	age    time.Time
+}
+
+type roundRobin struct {
+	//block chan struct{}
+	sync.Mutex
+	available []*rrEntry
+	all       map[*clientv3.Client]*rrEntry
+	full      map[*clientv3.Client]*rrEntry
+	waitList  *list.List
+	max       int
+	capacity  int
+	timeout   time.Duration
+	//ageOut    time.Duration
+	endpoints []string
+	size      int
+	logLevel  log.LogLevel
+	closing   bool
+	closingCh chan struct{}
+	stopCh    chan struct{}
+}
+
+// Get returns an Etcd client. If not is available, it will create one
+// until the maximum allowed capacity.  If maximum capacity has been
+// reached then it will wait until s used one is freed.
+func (r *roundRobin) Get(ctx context.Context) (*clientv3.Client, error) {
+	r.Lock()
+
+	if r.closing {
+		r.Unlock()
+		return nil, errors.New("pool-is-closing")
+	}
+
+	// first determine if we need to block, which would mean the
+	// available queue is empty and we are at capacity
+	if len(r.available) == 0 && r.size >= r.capacity {
+
+		// create a channel on which to wait and
+		// add it to the list
+		ch := make(chan struct{})
+		element := r.waitList.PushBack(ch)
+		r.Unlock()
+
+		// block until it is our turn or context
+		// expires or is canceled
+		select {
+		case <-r.stopCh:
+			logger.Info(ctx, "stop-waiting-pool-is-closing")
+			r.waitList.Remove(element)
+			return nil, errors.New("stop-waiting-pool-is-closing")
+		case <-ch:
+			r.waitList.Remove(element)
+		case <-ctx.Done():
+			r.waitList.Remove(element)
+			return nil, ctx.Err()
+		}
+		r.Lock()
+	}
+
+	defer r.Unlock()
+	if len(r.available) > 0 {
+		// pull off back end as it is operationally quicker
+		last := len(r.available) - 1
+		entry := r.available[last]
+		entry.count++
+		if entry.count >= r.max {
+			r.available = r.available[:last]
+			r.full[entry.client] = entry
+		}
+		entry.age = time.Now()
+		return entry.client, nil
+	}
+
+	logConfig := log.ConstructZapConfig(log.JSON, r.logLevel, log.Fields{})
+	// increase capacity
+	client, err := clientv3.New(clientv3.Config{
+		Endpoints:   r.endpoints,
+		DialTimeout: r.timeout,
+		LogConfig:   &logConfig,
+	})
+	if err != nil {
+		return nil, err
+	}
+	entry := &rrEntry{
+		client: client,
+		count:  1,
+	}
+	r.all[entry.client] = entry
+
+	if r.max > 1 {
+		r.available = append(r.available, entry)
+	} else {
+		r.full[entry.client] = entry
+	}
+	r.size++
+	return client, nil
+}
+
+// Put returns the Etcd Client back to the pool
+func (r *roundRobin) Put(client *clientv3.Client) {
+	r.Lock()
+
+	entry := r.all[client]
+	entry.count--
+
+	if r.closing {
+		// Close client if count is 0
+		if entry.count == 0 {
+			if err := entry.client.Close(); err != nil {
+				logger.Warnw(context.Background(), "error-closing-client", log.Fields{"error": err})
+			}
+			delete(r.all, entry.client)
+		}
+		// Notify Close function that a client was returned to the pool
+		r.closingCh <- struct{}{}
+		r.Unlock()
+		return
+	}
+
+	// This entry is now available for use, so
+	// if in full map add it to available and
+	// remove from full
+	if _, ok := r.full[client]; ok {
+		r.available = append(r.available, entry)
+		delete(r.full, client)
+	}
+
+	front := r.waitList.Front()
+	if front != nil {
+		ch := r.waitList.Remove(front)
+		r.Unlock()
+		// need to unblock if someone is waiting
+		ch.(chan struct{}) <- struct{}{}
+		return
+	}
+	r.Unlock()
+}
+
+func (r *roundRobin) Close(ctx context.Context) {
+	r.Lock()
+	r.closing = true
+
+	// Notify anyone waiting for a client to stop waiting
+	close(r.stopCh)
+
+	// Clean-up unused clients
+	for i := 0; i < len(r.available); i++ {
+		// Count 0 means no one is using that client
+		if r.available[i].count == 0 {
+			if err := r.available[i].client.Close(); err != nil {
+				logger.Warnw(ctx, "failure-closing-client", log.Fields{"client": r.available[i].client, "error": err})
+			}
+			// Remove client for all list
+			delete(r.all, r.available[i].client)
+		}
+	}
+
+	// Figure out how many clients are in use
+	numberInUse := 0
+	for _, rrEntry := range r.all {
+		numberInUse += rrEntry.count
+	}
+	r.Unlock()
+
+	if numberInUse == 0 {
+		logger.Info(ctx, "no-connection-in-use")
+		return
+	}
+
+	logger.Infow(ctx, "waiting-for-clients-return", log.Fields{"count": numberInUse})
+
+	// Wait for notifications when a client is returned to the pool
+	for {
+		select {
+		case <-r.closingCh:
+			numberInUse--
+			if numberInUse == 0 {
+				logger.Info(ctx, "all-connections-closed")
+				return
+			}
+		case <-ctx.Done():
+			logger.Warnw(ctx, "context-done", log.Fields{"error": ctx.Err()})
+			return
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/kvutils.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore/kvutils.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore/kvutils.go
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/common.go
new file mode 100644
index 0000000..0f0468e
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 events
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/eventif/events_proxy_if.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/eventif/events_proxy_if.go
new file mode 100644
index 0000000..e4ebc36
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/eventif/events_proxy_if.go
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2020-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 eventif
+
+import (
+	"context"
+
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+// EventProxy interface for eventproxy
+type EventProxy interface {
+	SendDeviceEvent(ctx context.Context, deviceEvent *voltha.DeviceEvent, category EventCategory,
+		subCategory EventSubCategory, raisedTs int64) error
+	SendKpiEvent(ctx context.Context, id string, deviceEvent *voltha.KpiEvent2, category EventCategory,
+		subCategory EventSubCategory, raisedTs int64) error
+	SendRPCEvent(ctx context.Context, id string, deviceEvent *voltha.RPCEvent, category EventCategory,
+		subCategory *EventSubCategory, raisedTs int64) error
+	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
+	SendLiveness(ctx context.Context) error
+	Start() error
+	Stop()
+}
+
+const (
+	EventTypeVersion = "0.1"
+)
+
+type (
+	EventType        = voltha.EventType_Types
+	EventCategory    = voltha.EventCategory_Types
+	EventSubCategory = voltha.EventSubCategory_Types
+)
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/events_proxy.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/events_proxy.go
new file mode 100644
index 0000000..e4493f9
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/events_proxy.go
@@ -0,0 +1,347 @@
+/*
+ * Copyright 2020-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 events
+
+import (
+	"container/ring"
+	"context"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/events/eventif"
+	"github.com/opencord/voltha-lib-go/v7/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+	"google.golang.org/protobuf/types/known/timestamppb"
+)
+
+// TODO: Make configurable through helm chart
+const EVENT_THRESHOLD = 1000
+
+type lastEvent struct{}
+
+type EventProxy struct {
+	kafkaClient    kafka.Client
+	eventTopic     kafka.Topic
+	eventQueue     *EventQueue
+	queueCtx       context.Context
+	queueCancelCtx context.CancelFunc
+}
+
+func NewEventProxy(opts ...EventProxyOption) *EventProxy {
+	var proxy EventProxy
+	for _, option := range opts {
+		option(&proxy)
+	}
+	proxy.eventQueue = newEventQueue()
+	proxy.queueCtx, proxy.queueCancelCtx = context.WithCancel(context.Background())
+	return &proxy
+}
+
+type EventProxyOption func(*EventProxy)
+
+func MsgClient(client kafka.Client) EventProxyOption {
+	return func(args *EventProxy) {
+		args.kafkaClient = client
+	}
+}
+
+func MsgTopic(topic kafka.Topic) EventProxyOption {
+	return func(args *EventProxy) {
+		args.eventTopic = topic
+	}
+}
+
+func (ep *EventProxy) formatId(eventName string) string {
+	return fmt.Sprintf("Voltha.openolt.%s.%s", eventName, strconv.FormatInt(time.Now().UnixNano(), 10))
+}
+
+func (ep *EventProxy) getEventHeader(eventName string,
+	category eventif.EventCategory,
+	subCategory *eventif.EventSubCategory,
+	eventType eventif.EventType,
+	raisedTs int64) (*voltha.EventHeader, error) {
+	var header voltha.EventHeader
+	if strings.Contains(eventName, "_") {
+		eventName = strings.Join(strings.Split(eventName, "_")[:len(strings.Split(eventName, "_"))-2], "_")
+	} else {
+		eventName = "UNKNOWN_EVENT"
+	}
+	/* Populating event header */
+	header.Id = ep.formatId(eventName)
+	header.Category = category
+	if subCategory != nil {
+		header.SubCategory = *subCategory
+	} else {
+		header.SubCategory = voltha.EventSubCategory_NONE
+	}
+	header.Type = eventType
+	header.TypeVersion = eventif.EventTypeVersion
+
+	// raisedTs is in seconds
+	header.RaisedTs = timestamppb.New(time.Unix(raisedTs, 0))
+	header.ReportedTs = timestamppb.New(time.Now())
+
+	return &header, nil
+}
+
+/* Send out rpc events*/
+func (ep *EventProxy) SendRPCEvent(ctx context.Context, id string, rpcEvent *voltha.RPCEvent, category eventif.EventCategory, subCategory *eventif.EventSubCategory, raisedTs int64) error {
+	if rpcEvent == nil {
+		logger.Error(ctx, "Received empty rpc event")
+		return errors.New("rpc event nil")
+	}
+	var event voltha.Event
+	var err error
+	if event.Header, err = ep.getEventHeader(id, category, subCategory, voltha.EventType_RPC_EVENT, raisedTs); err != nil {
+		return err
+	}
+	event.EventType = &voltha.Event_RpcEvent{RpcEvent: rpcEvent}
+	ep.eventQueue.push(&event)
+	return nil
+
+}
+
+/* Send out device events*/
+func (ep *EventProxy) SendDeviceEvent(ctx context.Context, deviceEvent *voltha.DeviceEvent, category eventif.EventCategory, subCategory eventif.EventSubCategory, raisedTs int64) error {
+	if deviceEvent == nil {
+		logger.Error(ctx, "Recieved empty device event")
+		return errors.New("Device event nil")
+	}
+	var event voltha.Event
+	var de voltha.Event_DeviceEvent
+	var err error
+	de.DeviceEvent = deviceEvent
+	if event.Header, err = ep.getEventHeader(deviceEvent.DeviceEventName, category, &subCategory, voltha.EventType_DEVICE_EVENT, raisedTs); err != nil {
+		return err
+	}
+	event.EventType = &de
+	if err := ep.sendEvent(ctx, &event); err != nil {
+		logger.Errorw(ctx, "Failed to send device event to KAFKA bus", log.Fields{"device-event": deviceEvent})
+		return err
+	}
+	logger.Infow(ctx, "Successfully sent device event KAFKA", log.Fields{"Id": event.Header.Id, "Category": event.Header.Category,
+		"SubCategory": event.Header.SubCategory, "Type": event.Header.Type, "TypeVersion": event.Header.TypeVersion,
+		"ReportedTs": event.Header.ReportedTs, "ResourceId": deviceEvent.ResourceId, "Context": deviceEvent.Context,
+		"DeviceEventName": deviceEvent.DeviceEventName})
+
+	return nil
+
+}
+
+// SendKpiEvent is to send kpi events to voltha.event topic
+func (ep *EventProxy) SendKpiEvent(ctx context.Context, id string, kpiEvent *voltha.KpiEvent2, category eventif.EventCategory, subCategory eventif.EventSubCategory, raisedTs int64) error {
+	if kpiEvent == nil {
+		logger.Error(ctx, "Recieved empty kpi event")
+		return errors.New("KPI event nil")
+	}
+	var event voltha.Event
+	var de voltha.Event_KpiEvent2
+	var err error
+	de.KpiEvent2 = kpiEvent
+	if event.Header, err = ep.getEventHeader(id, category, &subCategory, voltha.EventType_KPI_EVENT2, raisedTs); err != nil {
+		return err
+	}
+	event.EventType = &de
+	if err := ep.sendEvent(ctx, &event); err != nil {
+		logger.Errorw(ctx, "Failed to send kpi event to KAFKA bus", log.Fields{"device-event": kpiEvent})
+		return err
+	}
+	logger.Infow(ctx, "Successfully sent kpi event to KAFKA", log.Fields{"Id": event.Header.Id, "Category": event.Header.Category,
+		"SubCategory": event.Header.SubCategory, "Type": event.Header.Type, "TypeVersion": event.Header.TypeVersion,
+		"ReportedTs": event.Header.ReportedTs, "KpiEventName": "STATS_EVENT"})
+
+	return nil
+
+}
+
+func (ep *EventProxy) sendEvent(ctx context.Context, event *voltha.Event) error {
+	logger.Debugw(ctx, "Send event to kafka", log.Fields{"event": event})
+	if err := ep.kafkaClient.Send(ctx, event, &ep.eventTopic); err != nil {
+		return err
+	}
+	logger.Debugw(ctx, "Sent event to kafka", log.Fields{"event": event})
+
+	return nil
+}
+
+func (ep *EventProxy) EnableLivenessChannel(ctx context.Context, enable bool) chan bool {
+	return ep.kafkaClient.EnableLivenessChannel(ctx, enable)
+}
+
+func (ep *EventProxy) SendLiveness(ctx context.Context) error {
+	return ep.kafkaClient.SendLiveness(ctx)
+}
+
+// Start the event proxy
+func (ep *EventProxy) Start() error {
+	eq := ep.eventQueue
+	go eq.start(ep.queueCtx)
+	logger.Debugw(context.Background(), "event-proxy-starting...", log.Fields{"events-threashold": EVENT_THRESHOLD})
+	for {
+		// Notify the queue I am ready
+		eq.readyToSendToKafkaCh <- struct{}{}
+		// Wait for an event
+		elem, ok := <-eq.eventChannel
+		if !ok {
+			logger.Debug(context.Background(), "event-channel-closed-exiting")
+			break
+		}
+		// Check for last event
+		if _, ok := elem.(*lastEvent); ok {
+			// close the queuing loop
+			logger.Info(context.Background(), "received-last-event")
+			ep.queueCancelCtx()
+			break
+		}
+		ctx := context.Background()
+		event, ok := elem.(*voltha.Event)
+		if !ok {
+			logger.Warnw(ctx, "invalid-event", log.Fields{"element": elem})
+			continue
+		}
+		if err := ep.sendEvent(ctx, event); err != nil {
+			logger.Errorw(ctx, "failed-to-send-event-to-kafka-bus", log.Fields{"event": event})
+		} else {
+			logger.Debugw(ctx, "successfully-sent-rpc-event-to-kafka-bus", log.Fields{"id": event.Header.Id, "category": event.Header.Category,
+				"sub-category": event.Header.SubCategory, "type": event.Header.Type, "type-version": event.Header.TypeVersion,
+				"reported-ts": event.Header.ReportedTs, "event-type": event.EventType})
+		}
+	}
+	return nil
+}
+
+func (ep *EventProxy) Stop() {
+	if ep.eventQueue != nil {
+		ep.eventQueue.stop()
+	}
+}
+
+type EventQueue struct {
+	mutex                sync.RWMutex
+	eventChannel         chan interface{}
+	insertPosition       *ring.Ring
+	popPosition          *ring.Ring
+	dataToSendAvailable  chan struct{}
+	readyToSendToKafkaCh chan struct{}
+	eventQueueStopped    chan struct{}
+}
+
+func newEventQueue() *EventQueue {
+	ev := &EventQueue{
+		eventChannel:         make(chan interface{}),
+		insertPosition:       ring.New(EVENT_THRESHOLD),
+		dataToSendAvailable:  make(chan struct{}),
+		readyToSendToKafkaCh: make(chan struct{}),
+		eventQueueStopped:    make(chan struct{}),
+	}
+	ev.popPosition = ev.insertPosition
+	return ev
+}
+
+// push is invoked to push an event at the back of a queue
+func (eq *EventQueue) push(event interface{}) {
+	eq.mutex.Lock()
+
+	if eq.insertPosition != nil {
+		// Handle Queue is full.
+		// TODO: Current default is to overwrite old data if queue is full. Is there a need to
+		// block caller if max threshold is reached?
+		if eq.insertPosition.Value != nil && eq.insertPosition == eq.popPosition {
+			eq.popPosition = eq.popPosition.Next()
+		}
+
+		// Insert data and move pointer to next empty position
+		eq.insertPosition.Value = event
+		eq.insertPosition = eq.insertPosition.Next()
+
+		// Check for last event
+		if _, ok := event.(*lastEvent); ok {
+			eq.insertPosition = nil
+		}
+		eq.mutex.Unlock()
+		// Notify waiting thread of data availability
+		eq.dataToSendAvailable <- struct{}{}
+
+	} else {
+		logger.Debug(context.Background(), "event-queue-is-closed-as-insert-position-is-cleared")
+		eq.mutex.Unlock()
+	}
+}
+
+// start starts the routine that extracts an element from the event queue and
+// send it to the kafka sending routine to process.
+func (eq *EventQueue) start(ctx context.Context) {
+	logger.Info(ctx, "starting-event-queue")
+loop:
+	for {
+		select {
+		case <-eq.dataToSendAvailable:
+		//	Do nothing - use to prevent caller pushing data to block
+		case <-eq.readyToSendToKafkaCh:
+			{
+				// Kafka sending routine is ready to process an event
+				eq.mutex.Lock()
+				element := eq.popPosition.Value
+				if element == nil {
+					// No events to send. Wait
+					eq.mutex.Unlock()
+					select {
+					case _, ok := <-eq.dataToSendAvailable:
+						if !ok {
+							// channel closed
+							eq.eventQueueStopped <- struct{}{}
+							return
+						}
+					case <-ctx.Done():
+						logger.Info(ctx, "event-queue-context-done")
+						eq.eventQueueStopped <- struct{}{}
+						return
+					}
+					eq.mutex.Lock()
+					element = eq.popPosition.Value
+				}
+				eq.popPosition.Value = nil
+				eq.popPosition = eq.popPosition.Next()
+				eq.mutex.Unlock()
+				eq.eventChannel <- element
+			}
+		case <-ctx.Done():
+			logger.Info(ctx, "event-queue-context-done")
+			eq.eventQueueStopped <- struct{}{}
+			break loop
+		}
+	}
+	logger.Info(ctx, "event-queue-stopped")
+
+}
+
+func (eq *EventQueue) stop() {
+	// Flush all
+	eq.push(&lastEvent{})
+	<-eq.eventQueueStopped
+	eq.mutex.Lock()
+	close(eq.readyToSendToKafkaCh)
+	close(eq.dataToSendAvailable)
+	close(eq.eventChannel)
+	eq.mutex.Unlock()
+
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/utils.go
new file mode 100644
index 0000000..4598161
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/events/utils.go
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2020-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 events
+
+import (
+	"fmt"
+	"strconv"
+
+	"github.com/opencord/voltha-protos/v5/go/common"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+type ContextType string
+
+const (
+	// ContextAdminState is for the admin state of the Device in the context of the event
+	ContextAdminState ContextType = "admin-state"
+	// ContextConnectState is for the connect state of the Device in the context of the event
+	ContextConnectState ContextType = "connect-state"
+	// ContextOperState is for the operational state of the Device in the context of the event
+	ContextOperState ContextType = "oper-state"
+	// ContextPrevdminState is for the previous admin state of the Device in the context of the event
+	ContextPrevAdminState ContextType = "prev-admin-state"
+	// ContextPrevConnectState is for the previous connect state of the Device in the context of the event
+	ContextPrevConnectState ContextType = "prev-connect-state"
+	// ContextPrevOperState is for the previous operational state of the Device in the context of the event
+	ContextPrevOperState ContextType = "prev-oper-state"
+	// ContextDeviceID is for the previous operational state of the Device in the context of the event
+	ContextDeviceID ContextType = "id"
+	// ContextParentID is for the parent id in the context of the event
+	ContextParentID ContextType = "parent-id"
+	// ContextSerialNumber is for the serial number of the Device in the context of the event
+	ContextSerialNumber ContextType = "serial-number"
+	// ContextIsRoot is for the root flag of Device in the context of the event
+	ContextIsRoot ContextType = "is-root"
+	// ContextParentPort is for the parent interface id of child in the context of the event
+	ContextParentPort ContextType = "parent-port"
+)
+
+type EventName string
+
+const (
+	DeviceStateChangeEvent EventName = "DEVICE_STATE_CHANGE"
+)
+
+type EventAction string
+
+const (
+	Raise EventAction = "RAISE_EVENT"
+	Clear EventAction = "CLEAR_EVENT"
+)
+
+//CreateDeviceStateChangeEvent forms and returns a new DeviceStateChange Event
+func CreateDeviceStateChangeEvent(serialNumber string, deviceID string, parentID string,
+	prevOperStatus common.OperStatus_Types, prevConnStatus common.ConnectStatus_Types, prevAdminStatus common.AdminState_Types,
+	operStatus common.OperStatus_Types, connStatus common.ConnectStatus_Types, adminStatus common.AdminState_Types,
+	parentPort uint32, isRoot bool) *voltha.DeviceEvent {
+
+	context := make(map[string]string)
+	/* Populating event context */
+	context[string(ContextSerialNumber)] = serialNumber
+	context[string(ContextDeviceID)] = deviceID
+	context[string(ContextParentID)] = parentID
+	context[string(ContextPrevOperState)] = prevOperStatus.String()
+	context[string(ContextPrevConnectState)] = prevConnStatus.String()
+	context[string(ContextPrevAdminState)] = prevAdminStatus.String()
+	context[string(ContextOperState)] = operStatus.String()
+	context[string(ContextConnectState)] = connStatus.String()
+	context[string(ContextAdminState)] = adminStatus.String()
+	context[string(ContextIsRoot)] = strconv.FormatBool(isRoot)
+	context[string(ContextParentPort)] = strconv.FormatUint(uint64(parentPort), 10)
+
+	return &voltha.DeviceEvent{
+		Context:         context,
+		ResourceId:      deviceID,
+		DeviceEventName: fmt.Sprintf("%s_%s", string(DeviceStateChangeEvent), string(Raise)),
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/common.go
new file mode 100644
index 0000000..2f5ff92
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 flows
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/flow_utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/flow_utils.go
new file mode 100644
index 0000000..41b615a
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/flows/flow_utils.go
@@ -0,0 +1,1636 @@
+/*
+ * 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 flows
+
+import (
+	"bytes"
+	"context"
+	"crypto/md5"
+	"encoding/binary"
+	"fmt"
+	"hash"
+	"sort"
+	"sync"
+
+	"github.com/cevaris/ordered_map"
+	"github.com/golang/protobuf/proto"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+)
+
+var (
+	// Instructions shortcut
+	APPLY_ACTIONS  = ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS
+	WRITE_METADATA = ofp.OfpInstructionType_OFPIT_WRITE_METADATA
+	METER_ACTION   = ofp.OfpInstructionType_OFPIT_METER
+
+	//OFPAT_* shortcuts
+	OUTPUT       = ofp.OfpActionType_OFPAT_OUTPUT
+	COPY_TTL_OUT = ofp.OfpActionType_OFPAT_COPY_TTL_OUT
+	COPY_TTL_IN  = ofp.OfpActionType_OFPAT_COPY_TTL_IN
+	SET_MPLS_TTL = ofp.OfpActionType_OFPAT_SET_MPLS_TTL
+	DEC_MPLS_TTL = ofp.OfpActionType_OFPAT_DEC_MPLS_TTL
+	PUSH_VLAN    = ofp.OfpActionType_OFPAT_PUSH_VLAN
+	POP_VLAN     = ofp.OfpActionType_OFPAT_POP_VLAN
+	PUSH_MPLS    = ofp.OfpActionType_OFPAT_PUSH_MPLS
+	POP_MPLS     = ofp.OfpActionType_OFPAT_POP_MPLS
+	SET_QUEUE    = ofp.OfpActionType_OFPAT_SET_QUEUE
+	GROUP        = ofp.OfpActionType_OFPAT_GROUP
+	SET_NW_TTL   = ofp.OfpActionType_OFPAT_SET_NW_TTL
+	NW_TTL       = ofp.OfpActionType_OFPAT_DEC_NW_TTL
+	SET_FIELD    = ofp.OfpActionType_OFPAT_SET_FIELD
+	PUSH_PBB     = ofp.OfpActionType_OFPAT_PUSH_PBB
+	POP_PBB      = ofp.OfpActionType_OFPAT_POP_PBB
+	EXPERIMENTER = ofp.OfpActionType_OFPAT_EXPERIMENTER
+
+	//OFPXMT_OFB_* shortcuts (incomplete)
+	IN_PORT         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT
+	IN_PHY_PORT     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT
+	METADATA        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_METADATA
+	ETH_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST
+	ETH_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC
+	ETH_TYPE        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE
+	VLAN_VID        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID
+	VLAN_PCP        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP
+	IP_DSCP         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP
+	IP_ECN          = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN
+	IP_PROTO        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO
+	IPV4_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC
+	IPV4_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST
+	TCP_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC
+	TCP_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST
+	UDP_SRC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC
+	UDP_DST         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST
+	SCTP_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC
+	SCTP_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST
+	ICMPV4_TYPE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE
+	ICMPV4_CODE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE
+	ARP_OP          = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP
+	ARP_SPA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA
+	ARP_TPA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA
+	ARP_SHA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA
+	ARP_THA         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA
+	IPV6_SRC        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC
+	IPV6_DST        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST
+	IPV6_FLABEL     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL
+	ICMPV6_TYPE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE
+	ICMPV6_CODE     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE
+	IPV6_ND_TARGET  = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET
+	OFB_IPV6_ND_SLL = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL
+	IPV6_ND_TLL     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL
+	MPLS_LABEL      = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL
+	MPLS_TC         = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC
+	MPLS_BOS        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS
+	PBB_ISID        = ofp.OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID
+	TUNNEL_ID       = ofp.OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID
+	IPV6_EXTHDR     = ofp.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR
+)
+
+//ofp_action_* shortcuts
+
+func Output(port uint32, maxLen ...ofp.OfpControllerMaxLen) *ofp.OfpAction {
+	maxLength := ofp.OfpControllerMaxLen_OFPCML_MAX
+	if len(maxLen) > 0 {
+		maxLength = maxLen[0]
+	}
+	return &ofp.OfpAction{Type: OUTPUT, Action: &ofp.OfpAction_Output{Output: &ofp.OfpActionOutput{Port: port, MaxLen: uint32(maxLength)}}}
+}
+
+func MplsTtl(ttl uint32) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: SET_MPLS_TTL, Action: &ofp.OfpAction_MplsTtl{MplsTtl: &ofp.OfpActionMplsTtl{MplsTtl: ttl}}}
+}
+
+func PushVlan(ethType uint32) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: PUSH_VLAN, Action: &ofp.OfpAction_Push{Push: &ofp.OfpActionPush{Ethertype: ethType}}}
+}
+
+func PopVlan() *ofp.OfpAction {
+	return &ofp.OfpAction{Type: POP_VLAN}
+}
+
+func PopMpls(ethType uint32) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: POP_MPLS, Action: &ofp.OfpAction_PopMpls{PopMpls: &ofp.OfpActionPopMpls{Ethertype: ethType}}}
+}
+
+func Group(groupId uint32) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: GROUP, Action: &ofp.OfpAction_Group{Group: &ofp.OfpActionGroup{GroupId: groupId}}}
+}
+
+func NwTtl(nwTtl uint32) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: NW_TTL, Action: &ofp.OfpAction_NwTtl{NwTtl: &ofp.OfpActionNwTtl{NwTtl: nwTtl}}}
+}
+
+func SetField(field *ofp.OfpOxmOfbField) *ofp.OfpAction {
+	actionSetField := &ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: &ofp.OfpOxmField_OfbField{OfbField: field}}
+	return &ofp.OfpAction{Type: SET_FIELD, Action: &ofp.OfpAction_SetField{SetField: &ofp.OfpActionSetField{Field: actionSetField}}}
+}
+
+func Experimenter(experimenter uint32, data []byte) *ofp.OfpAction {
+	return &ofp.OfpAction{Type: EXPERIMENTER, Action: &ofp.OfpAction_Experimenter{Experimenter: &ofp.OfpActionExperimenter{Experimenter: experimenter, Data: data}}}
+}
+
+//ofb_field generators (incomplete set)
+
+func InPort(inPort uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IN_PORT, Value: &ofp.OfpOxmOfbField_Port{Port: inPort}}
+}
+
+func InPhyPort(inPhyPort uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IN_PHY_PORT, Value: &ofp.OfpOxmOfbField_Port{Port: inPhyPort}}
+}
+
+func Metadata_ofp(tableMetadata uint64) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: METADATA, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: tableMetadata}}
+}
+
+// should Metadata_ofp used here ?????
+func EthDst(ethDst uint64) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ETH_DST, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: ethDst}}
+}
+
+// should Metadata_ofp used here ?????
+func EthSrc(ethSrc uint64) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ETH_SRC, Value: &ofp.OfpOxmOfbField_TableMetadata{TableMetadata: ethSrc}}
+}
+
+func EthType(ethType uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ETH_TYPE, Value: &ofp.OfpOxmOfbField_EthType{EthType: ethType}}
+}
+
+func VlanVid(vlanVid uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: VLAN_VID, Value: &ofp.OfpOxmOfbField_VlanVid{VlanVid: vlanVid}}
+}
+
+func VlanPcp(vlanPcp uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: VLAN_PCP, Value: &ofp.OfpOxmOfbField_VlanPcp{VlanPcp: vlanPcp}}
+}
+
+func IpDscp(ipDscp uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IP_DSCP, Value: &ofp.OfpOxmOfbField_IpDscp{IpDscp: ipDscp}}
+}
+
+func IpEcn(ipEcn uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IP_ECN, Value: &ofp.OfpOxmOfbField_IpEcn{IpEcn: ipEcn}}
+}
+
+func IpProto(ipProto uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IP_PROTO, Value: &ofp.OfpOxmOfbField_IpProto{IpProto: ipProto}}
+}
+
+func Ipv4Src(ipv4Src uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV4_SRC, Value: &ofp.OfpOxmOfbField_Ipv4Src{Ipv4Src: ipv4Src}}
+}
+
+func Ipv4Dst(ipv4Dst uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV4_DST, Value: &ofp.OfpOxmOfbField_Ipv4Dst{Ipv4Dst: ipv4Dst}}
+}
+
+func TcpSrc(tcpSrc uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: TCP_SRC, Value: &ofp.OfpOxmOfbField_TcpSrc{TcpSrc: tcpSrc}}
+}
+
+func TcpDst(tcpDst uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: TCP_DST, Value: &ofp.OfpOxmOfbField_TcpDst{TcpDst: tcpDst}}
+}
+
+func UdpSrc(udpSrc uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: UDP_SRC, Value: &ofp.OfpOxmOfbField_UdpSrc{UdpSrc: udpSrc}}
+}
+
+func UdpDst(udpDst uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: UDP_DST, Value: &ofp.OfpOxmOfbField_UdpDst{UdpDst: udpDst}}
+}
+
+func SctpSrc(sctpSrc uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: SCTP_SRC, Value: &ofp.OfpOxmOfbField_SctpSrc{SctpSrc: sctpSrc}}
+}
+
+func SctpDst(sctpDst uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: SCTP_DST, Value: &ofp.OfpOxmOfbField_SctpDst{SctpDst: sctpDst}}
+}
+
+func Icmpv4Type(icmpv4Type uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ICMPV4_TYPE, Value: &ofp.OfpOxmOfbField_Icmpv4Type{Icmpv4Type: icmpv4Type}}
+}
+
+func Icmpv4Code(icmpv4Code uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ICMPV4_CODE, Value: &ofp.OfpOxmOfbField_Icmpv4Code{Icmpv4Code: icmpv4Code}}
+}
+
+func ArpOp(arpOp uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ARP_OP, Value: &ofp.OfpOxmOfbField_ArpOp{ArpOp: arpOp}}
+}
+
+func ArpSpa(arpSpa uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ARP_SPA, Value: &ofp.OfpOxmOfbField_ArpSpa{ArpSpa: arpSpa}}
+}
+
+func ArpTpa(arpTpa uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ARP_TPA, Value: &ofp.OfpOxmOfbField_ArpTpa{ArpTpa: arpTpa}}
+}
+
+func ArpSha(arpSha []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ARP_SHA, Value: &ofp.OfpOxmOfbField_ArpSha{ArpSha: arpSha}}
+}
+
+func ArpTha(arpTha []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ARP_THA, Value: &ofp.OfpOxmOfbField_ArpTha{ArpTha: arpTha}}
+}
+
+func Ipv6Src(ipv6Src []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_SRC, Value: &ofp.OfpOxmOfbField_Ipv6Src{Ipv6Src: ipv6Src}}
+}
+
+func Ipv6Dst(ipv6Dst []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_DST, Value: &ofp.OfpOxmOfbField_Ipv6Dst{Ipv6Dst: ipv6Dst}}
+}
+
+func Ipv6Flabel(ipv6Flabel uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_FLABEL, Value: &ofp.OfpOxmOfbField_Ipv6Flabel{Ipv6Flabel: ipv6Flabel}}
+}
+
+func Icmpv6Type(icmpv6Type uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ICMPV6_TYPE, Value: &ofp.OfpOxmOfbField_Icmpv6Type{Icmpv6Type: icmpv6Type}}
+}
+
+func Icmpv6Code(icmpv6Code uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: ICMPV6_CODE, Value: &ofp.OfpOxmOfbField_Icmpv6Code{Icmpv6Code: icmpv6Code}}
+}
+
+func Ipv6NdTarget(ipv6NdTarget []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_ND_TARGET, Value: &ofp.OfpOxmOfbField_Ipv6NdTarget{Ipv6NdTarget: ipv6NdTarget}}
+}
+
+func OfbIpv6NdSll(ofbIpv6NdSll []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: OFB_IPV6_ND_SLL, Value: &ofp.OfpOxmOfbField_Ipv6NdSsl{Ipv6NdSsl: ofbIpv6NdSll}}
+}
+
+func Ipv6NdTll(ipv6NdTll []byte) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_ND_TLL, Value: &ofp.OfpOxmOfbField_Ipv6NdTll{Ipv6NdTll: ipv6NdTll}}
+}
+
+func MplsLabel(mplsLabel uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: MPLS_LABEL, Value: &ofp.OfpOxmOfbField_MplsLabel{MplsLabel: mplsLabel}}
+}
+
+func MplsTc(mplsTc uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: MPLS_TC, Value: &ofp.OfpOxmOfbField_MplsTc{MplsTc: mplsTc}}
+}
+
+func MplsBos(mplsBos uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: MPLS_BOS, Value: &ofp.OfpOxmOfbField_MplsBos{MplsBos: mplsBos}}
+}
+
+func PbbIsid(pbbIsid uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: PBB_ISID, Value: &ofp.OfpOxmOfbField_PbbIsid{PbbIsid: pbbIsid}}
+}
+
+func TunnelId(tunnelId uint64) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: TUNNEL_ID, Value: &ofp.OfpOxmOfbField_TunnelId{TunnelId: tunnelId}}
+}
+
+func Ipv6Exthdr(ipv6Exthdr uint32) *ofp.OfpOxmOfbField {
+	return &ofp.OfpOxmOfbField{Type: IPV6_EXTHDR, Value: &ofp.OfpOxmOfbField_Ipv6Exthdr{Ipv6Exthdr: ipv6Exthdr}}
+}
+
+//frequently used extractors
+
+func excludeAction(action *ofp.OfpAction, exclude ...ofp.OfpActionType) bool {
+	for _, actionToExclude := range exclude {
+		if action.Type == actionToExclude {
+			return true
+		}
+	}
+	return false
+}
+
+func GetActions(flow *ofp.OfpFlowStats, exclude ...ofp.OfpActionType) []*ofp.OfpAction {
+	if flow == nil {
+		return nil
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
+			instActions := instruction.GetActions()
+			if instActions == nil {
+				return nil
+			}
+			if len(exclude) == 0 {
+				return instActions.Actions
+			} else {
+				filteredAction := make([]*ofp.OfpAction, 0)
+				for _, action := range instActions.Actions {
+					if !excludeAction(action, exclude...) {
+						filteredAction = append(filteredAction, action)
+					}
+				}
+				return filteredAction
+			}
+		}
+	}
+	return nil
+}
+
+func UpdateOutputPortByActionType(flow *ofp.OfpFlowStats, actionType uint32, toPort uint32) *ofp.OfpFlowStats {
+	if flow == nil {
+		return nil
+	}
+	nFlow := (proto.Clone(flow)).(*ofp.OfpFlowStats)
+	nFlow.Instructions = nil
+	nInsts := make([]*ofp.OfpInstruction, 0)
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == actionType {
+			instActions := instruction.GetActions()
+			if instActions == nil {
+				return nil
+			}
+			nActions := make([]*ofp.OfpAction, 0)
+			for _, action := range instActions.Actions {
+				if action.GetOutput() != nil {
+					nActions = append(nActions, Output(toPort))
+				} else {
+					nActions = append(nActions, action)
+				}
+			}
+			instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: nActions}}
+			nInsts = append(nInsts, &ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction})
+		} else {
+			nInsts = append(nInsts, instruction)
+		}
+	}
+	nFlow.Instructions = nInsts
+	return nFlow
+}
+
+func excludeOxmOfbField(field *ofp.OfpOxmOfbField, exclude ...ofp.OxmOfbFieldTypes) bool {
+	for _, fieldToExclude := range exclude {
+		if field.Type == fieldToExclude {
+			return true
+		}
+	}
+	return false
+}
+
+func GetOfbFields(flow *ofp.OfpFlowStats, exclude ...ofp.OxmOfbFieldTypes) []*ofp.OfpOxmOfbField {
+	if flow == nil || flow.Match == nil || flow.Match.Type != ofp.OfpMatchType_OFPMT_OXM {
+		return nil
+	}
+	ofbFields := make([]*ofp.OfpOxmOfbField, 0)
+	for _, field := range flow.Match.OxmFields {
+		if field.OxmClass == ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC {
+			ofbFields = append(ofbFields, field.GetOfbField())
+		}
+	}
+	if len(exclude) == 0 {
+		return ofbFields
+	} else {
+		filteredFields := make([]*ofp.OfpOxmOfbField, 0)
+		for _, ofbField := range ofbFields {
+			if !excludeOxmOfbField(ofbField, exclude...) {
+				filteredFields = append(filteredFields, ofbField)
+			}
+		}
+		return filteredFields
+	}
+}
+
+func GetPacketOutPort(packet *ofp.OfpPacketOut) uint32 {
+	if packet == nil {
+		return 0
+	}
+	for _, action := range packet.GetActions() {
+		if action.Type == OUTPUT {
+			return action.GetOutput().Port
+		}
+	}
+	return 0
+}
+
+func GetOutPort(flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, action := range GetActions(flow) {
+		if action.Type == OUTPUT {
+			out := action.GetOutput()
+			if out == nil {
+				return 0
+			}
+			return out.GetPort()
+		}
+	}
+	return 0
+}
+
+func GetInPort(flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == IN_PORT {
+			return field.GetPort()
+		}
+	}
+	return 0
+}
+
+func GetGotoTableId(flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_GOTO_TABLE) {
+			gotoTable := instruction.GetGotoTable()
+			if gotoTable == nil {
+				return 0
+			}
+			return gotoTable.GetTableId()
+		}
+	}
+	return 0
+}
+
+func GetMeterId(flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_METER) {
+			MeterInstruction := instruction.GetMeter()
+			if MeterInstruction == nil {
+				return 0
+			}
+			return MeterInstruction.GetMeterId()
+		}
+	}
+	return 0
+}
+
+func GetVlanVid(flow *ofp.OfpFlowStats) *uint32 {
+	if flow == nil {
+		return nil
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == VLAN_VID {
+			ret := field.GetVlanVid()
+			return &ret
+		}
+	}
+	// Dont return 0 if the field is missing as vlan id value 0 has meaning and cannot be overloaded as "not found"
+	return nil
+}
+
+func GetSetActionField(ctx context.Context, flow *ofp.OfpFlowStats, ofbType ofp.OxmOfbFieldTypes) (uint32, bool) {
+	if flow == nil {
+		return 0, false
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(APPLY_ACTIONS) {
+			actions := instruction.GetActions()
+			for _, action := range actions.GetActions() {
+				if action.Type == SET_FIELD {
+					setField := action.GetSetField()
+					if setField.Field.GetOfbField().Type == ofbType {
+						switch ofbType {
+						case VLAN_PCP:
+							return setField.Field.GetOfbField().GetVlanPcp(), true
+						case VLAN_VID:
+							return setField.Field.GetOfbField().GetVlanVid(), true
+						default:
+							logger.Errorw(ctx, "unsupported-ofb-field-type", log.Fields{"ofbType": ofbType})
+							return 0, false
+						}
+					}
+				}
+			}
+			return 0, false
+		}
+	}
+	return 0, false
+}
+
+func GetTunnelId(flow *ofp.OfpFlowStats) uint64 {
+	if flow == nil {
+		return 0
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == TUNNEL_ID {
+			return field.GetTunnelId()
+		}
+	}
+	return 0
+}
+
+//GetMetaData - legacy get method (only want lower 32 bits)
+func GetMetaData(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == METADATA {
+			return uint32(field.GetTableMetadata() & 0xFFFFFFFF)
+		}
+	}
+	logger.Debug(ctx, "No-metadata-present")
+	return 0
+}
+
+func GetMetaData64Bit(ctx context.Context, flow *ofp.OfpFlowStats) uint64 {
+	if flow == nil {
+		return 0
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == METADATA {
+			return field.GetTableMetadata()
+		}
+	}
+	logger.Debug(ctx, "No-metadata-present")
+	return 0
+}
+
+// function returns write metadata value from write_metadata action field
+func GetMetadataFromWriteMetadataAction(ctx context.Context, flow *ofp.OfpFlowStats) uint64 {
+	if flow != nil {
+		for _, instruction := range flow.Instructions {
+			if instruction.Type == uint32(WRITE_METADATA) {
+				if writeMetadata := instruction.GetWriteMetadata(); writeMetadata != nil {
+					return writeMetadata.GetMetadata()
+				}
+			}
+		}
+	}
+	logger.Debugw(ctx, "No-write-metadata-present", log.Fields{"flow": flow})
+	return 0
+}
+
+func GetTechProfileIDFromWriteMetaData(ctx context.Context, metadata uint64) uint16 {
+	/*
+	   Write metadata instruction value (metadata) is 8 bytes:
+	   MS 2 bytes: C Tag
+	   Next 2 bytes: Technology Profile Id
+	   Next 4 bytes: Port number (uni or nni)
+
+	   This is set in the ONOS OltPipeline as a write metadata instruction
+	*/
+	var tpId uint16 = 0
+	logger.Debugw(ctx, "Write metadata value for Techprofile ID", log.Fields{"metadata": metadata})
+	if metadata != 0 {
+		tpId = uint16((metadata >> 32) & 0xFFFF)
+		logger.Debugw(ctx, "Found techprofile ID from write metadata action", log.Fields{"tpid": tpId})
+	}
+	return tpId
+}
+
+func GetEgressPortNumberFromWriteMetadata(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
+	/*
+			  Write metadata instruction value (metadata) is 8 bytes:
+		    	MS 2 bytes: C Tag
+		    	Next 2 bytes: Technology Profile Id
+		    	Next 4 bytes: Port number (uni or nni)
+		    	This is set in the ONOS OltPipeline as a write metadata instruction
+	*/
+	var uniPort uint32 = 0
+	md := GetMetadataFromWriteMetadataAction(ctx, flow)
+	logger.Debugw(ctx, "Metadata found for egress/uni port ", log.Fields{"metadata": md})
+	if md != 0 {
+		uniPort = uint32(md & 0xFFFFFFFF)
+		logger.Debugw(ctx, "Found EgressPort from write metadata action", log.Fields{"egress_port": uniPort})
+	}
+	return uniPort
+
+}
+
+func GetInnerTagFromMetaData(ctx context.Context, flow *ofp.OfpFlowStats) uint16 {
+	/*
+			  Write metadata instruction value (metadata) is 8 bytes:
+		    	MS 2 bytes: C Tag
+		    	Next 2 bytes: Technology Profile Id
+		    	Next 4 bytes: Port number (uni or nni)
+		    	This is set in the ONOS OltPipeline as a write metadata instruction
+	*/
+	var innerTag uint16 = 0
+	md := GetMetadataFromWriteMetadataAction(ctx, flow)
+	if md != 0 {
+		innerTag = uint16((md >> 48) & 0xFFFF)
+		logger.Debugw(ctx, "Found  CVLAN from write metadate action", log.Fields{"c_vlan": innerTag})
+	}
+	return innerTag
+}
+
+//GetInnerTagFromMetaData retrieves the inner tag from the Metadata_ofp. The port number (UNI on ONU) is in the
+// lower 32-bits of Metadata_ofp and the inner_tag is in the upper 32-bits. This is set in the ONOS OltPipeline as
+//// a Metadata_ofp field
+/*func GetInnerTagFromMetaData(flow *ofp.OfpFlowStats) uint64 {
+	md := GetMetaData64Bit(flow)
+	if md == 0 {
+		return 0
+	}
+	if md <= 0xffffffff {
+		logger.Debugw(ctx, "onos-upgrade-suggested", logger.Fields{"Metadata_ofp": md, "message": "Legacy MetaData detected form OltPipeline"})
+		return md
+	}
+	return (md >> 32) & 0xffffffff
+}*/
+
+// Extract the child device port from a flow that contains the parent device peer port.  Typically the UNI port of an
+// ONU child device.  Per TST agreement this will be the lower 32 bits of tunnel id reserving upper 32 bits for later
+// use
+func GetChildPortFromTunnelId(flow *ofp.OfpFlowStats) uint32 {
+	tid := GetTunnelId(flow)
+	if tid == 0 {
+		return 0
+	}
+	// Per TST agreement we are keeping any child port id (uni port id) in the lower 32 bits
+	return uint32(tid & 0xffffffff)
+}
+
+func HasNextTable(flow *ofp.OfpFlowStats) bool {
+	if flow == nil {
+		return false
+	}
+	return GetGotoTableId(flow) != 0
+}
+
+func GetGroup(flow *ofp.OfpFlowStats) uint32 {
+	if flow == nil {
+		return 0
+	}
+	for _, action := range GetActions(flow) {
+		if action.Type == GROUP {
+			grp := action.GetGroup()
+			if grp == nil {
+				return 0
+			}
+			return grp.GetGroupId()
+		}
+	}
+	return 0
+}
+
+func HasGroup(flow *ofp.OfpFlowStats) bool {
+	return GetGroup(flow) != 0
+}
+
+// GetNextTableId returns the next table ID if the "table_id" is present in the map, otherwise return nil
+func GetNextTableId(kw OfpFlowModArgs) *uint32 {
+	if val, exist := kw["table_id"]; exist {
+		ret := uint32(val)
+		return &ret
+	}
+	return nil
+}
+
+// GetMeterIdFlowModArgs returns the meterId if the "meter_id" is present in the map, otherwise return 0
+func GetMeterIdFlowModArgs(kw OfpFlowModArgs) uint32 {
+	if val, exist := kw["meter_id"]; exist {
+		return uint32(val)
+	}
+	return 0
+}
+
+// Function returns the metadata if the "write_metadata" is present in the map, otherwise return nil
+func GetMetadataFlowModArgs(kw OfpFlowModArgs) uint64 {
+	if val, exist := kw["write_metadata"]; exist {
+		ret := uint64(val)
+		return ret
+	}
+	return 0
+}
+
+// HashFlowStats returns a unique 64-bit integer hash of 'table_id', 'priority', and 'match'
+// The OF spec states that:
+// A flow table entry is identified by its match fields and priority: the match fields
+// and priority taken together identify a unique flow entry in the flow table.
+func HashFlowStats(flow *ofp.OfpFlowStats) (uint64, error) {
+	// first we need to make sure the oxm fields are in a predictable order (the specific order doesn't matter)
+	sort.Slice(flow.Match.OxmFields, func(a, b int) bool {
+		fieldsA, fieldsB := flow.Match.OxmFields[a], flow.Match.OxmFields[b]
+		if fieldsA.OxmClass < fieldsB.OxmClass {
+			return true
+		}
+		switch fieldA := fieldsA.Field.(type) {
+		case *ofp.OfpOxmField_OfbField:
+			switch fieldB := fieldsB.Field.(type) {
+			case *ofp.OfpOxmField_ExperimenterField:
+				return true // ofp < experimenter
+			case *ofp.OfpOxmField_OfbField:
+				return fieldA.OfbField.Type < fieldB.OfbField.Type
+			}
+		case *ofp.OfpOxmField_ExperimenterField:
+			switch fieldB := fieldsB.Field.(type) {
+			case *ofp.OfpOxmField_OfbField:
+				return false // ofp < experimenter
+			case *ofp.OfpOxmField_ExperimenterField:
+				eFieldA, eFieldB := fieldA.ExperimenterField, fieldB.ExperimenterField
+				if eFieldA.Experimenter != eFieldB.Experimenter {
+					return eFieldA.Experimenter < eFieldB.Experimenter
+				}
+				return eFieldA.OxmHeader < eFieldB.OxmHeader
+			}
+		}
+		return false
+	})
+
+	md5Hash := md5.New() // note that write errors will never occur with md5 hashing
+	var tmp [12]byte
+
+	binary.BigEndian.PutUint32(tmp[0:4], flow.TableId)             // tableId
+	binary.BigEndian.PutUint32(tmp[4:8], flow.Priority)            // priority
+	binary.BigEndian.PutUint32(tmp[8:12], uint32(flow.Match.Type)) // match type
+	_, _ = md5Hash.Write(tmp[:12])
+
+	for _, field := range flow.Match.OxmFields { // for all match fields
+		binary.BigEndian.PutUint32(tmp[:4], uint32(field.OxmClass)) // match class
+		_, _ = md5Hash.Write(tmp[:4])
+
+		switch oxmField := field.Field.(type) {
+		case *ofp.OfpOxmField_ExperimenterField:
+			binary.BigEndian.PutUint32(tmp[0:4], oxmField.ExperimenterField.Experimenter)
+			binary.BigEndian.PutUint32(tmp[4:8], oxmField.ExperimenterField.OxmHeader)
+			_, _ = md5Hash.Write(tmp[:8])
+
+		case *ofp.OfpOxmField_OfbField:
+			if err := hashWriteOfbField(md5Hash, oxmField.OfbField); err != nil {
+				return 0, err
+			}
+
+		default:
+			return 0, fmt.Errorf("unknown OfpOxmField type: %T", field.Field)
+		}
+	}
+
+	ret := md5Hash.Sum(nil)
+	return binary.BigEndian.Uint64(ret[0:8]), nil
+}
+
+func hashWriteOfbField(md5Hash hash.Hash, field *ofp.OfpOxmOfbField) error {
+	var tmp [8]byte
+	binary.BigEndian.PutUint32(tmp[:4], uint32(field.Type)) // type
+	_, _ = md5Hash.Write(tmp[:4])
+
+	// value
+	valType, val32, val64, valSlice := uint8(0), uint32(0), uint64(0), []byte(nil)
+	switch val := field.Value.(type) {
+	case *ofp.OfpOxmOfbField_Port:
+		valType, val32 = 4, val.Port
+	case *ofp.OfpOxmOfbField_PhysicalPort:
+		valType, val32 = 4, val.PhysicalPort
+	case *ofp.OfpOxmOfbField_TableMetadata:
+		valType, val64 = 8, val.TableMetadata
+	case *ofp.OfpOxmOfbField_EthDst:
+		valType, valSlice = 1, val.EthDst
+	case *ofp.OfpOxmOfbField_EthSrc:
+		valType, valSlice = 1, val.EthSrc
+	case *ofp.OfpOxmOfbField_EthType:
+		valType, val32 = 4, val.EthType
+	case *ofp.OfpOxmOfbField_VlanVid:
+		valType, val32 = 4, val.VlanVid
+	case *ofp.OfpOxmOfbField_VlanPcp:
+		valType, val32 = 4, val.VlanPcp
+	case *ofp.OfpOxmOfbField_IpDscp:
+		valType, val32 = 4, val.IpDscp
+	case *ofp.OfpOxmOfbField_IpEcn:
+		valType, val32 = 4, val.IpEcn
+	case *ofp.OfpOxmOfbField_IpProto:
+		valType, val32 = 4, val.IpProto
+	case *ofp.OfpOxmOfbField_Ipv4Src:
+		valType, val32 = 4, val.Ipv4Src
+	case *ofp.OfpOxmOfbField_Ipv4Dst:
+		valType, val32 = 4, val.Ipv4Dst
+	case *ofp.OfpOxmOfbField_TcpSrc:
+		valType, val32 = 4, val.TcpSrc
+	case *ofp.OfpOxmOfbField_TcpDst:
+		valType, val32 = 4, val.TcpDst
+	case *ofp.OfpOxmOfbField_UdpSrc:
+		valType, val32 = 4, val.UdpSrc
+	case *ofp.OfpOxmOfbField_UdpDst:
+		valType, val32 = 4, val.UdpDst
+	case *ofp.OfpOxmOfbField_SctpSrc:
+		valType, val32 = 4, val.SctpSrc
+	case *ofp.OfpOxmOfbField_SctpDst:
+		valType, val32 = 4, val.SctpDst
+	case *ofp.OfpOxmOfbField_Icmpv4Type:
+		valType, val32 = 4, val.Icmpv4Type
+	case *ofp.OfpOxmOfbField_Icmpv4Code:
+		valType, val32 = 4, val.Icmpv4Code
+	case *ofp.OfpOxmOfbField_ArpOp:
+		valType, val32 = 4, val.ArpOp
+	case *ofp.OfpOxmOfbField_ArpSpa:
+		valType, val32 = 4, val.ArpSpa
+	case *ofp.OfpOxmOfbField_ArpTpa:
+		valType, val32 = 4, val.ArpTpa
+	case *ofp.OfpOxmOfbField_ArpSha:
+		valType, valSlice = 1, val.ArpSha
+	case *ofp.OfpOxmOfbField_ArpTha:
+		valType, valSlice = 1, val.ArpTha
+	case *ofp.OfpOxmOfbField_Ipv6Src:
+		valType, valSlice = 1, val.Ipv6Src
+	case *ofp.OfpOxmOfbField_Ipv6Dst:
+		valType, valSlice = 1, val.Ipv6Dst
+	case *ofp.OfpOxmOfbField_Ipv6Flabel:
+		valType, val32 = 4, val.Ipv6Flabel
+	case *ofp.OfpOxmOfbField_Icmpv6Type:
+		valType, val32 = 4, val.Icmpv6Type
+	case *ofp.OfpOxmOfbField_Icmpv6Code:
+		valType, val32 = 4, val.Icmpv6Code
+	case *ofp.OfpOxmOfbField_Ipv6NdTarget:
+		valType, valSlice = 1, val.Ipv6NdTarget
+	case *ofp.OfpOxmOfbField_Ipv6NdSsl:
+		valType, valSlice = 1, val.Ipv6NdSsl
+	case *ofp.OfpOxmOfbField_Ipv6NdTll:
+		valType, valSlice = 1, val.Ipv6NdTll
+	case *ofp.OfpOxmOfbField_MplsLabel:
+		valType, val32 = 4, val.MplsLabel
+	case *ofp.OfpOxmOfbField_MplsTc:
+		valType, val32 = 4, val.MplsTc
+	case *ofp.OfpOxmOfbField_MplsBos:
+		valType, val32 = 4, val.MplsBos
+	case *ofp.OfpOxmOfbField_PbbIsid:
+		valType, val32 = 4, val.PbbIsid
+	case *ofp.OfpOxmOfbField_TunnelId:
+		valType, val64 = 8, val.TunnelId
+	case *ofp.OfpOxmOfbField_Ipv6Exthdr:
+		valType, val32 = 4, val.Ipv6Exthdr
+	default:
+		return fmt.Errorf("unknown OfpOxmField value type: %T", val)
+	}
+	switch valType {
+	case 1: // slice
+		_, _ = md5Hash.Write(valSlice)
+	case 4: // uint32
+		binary.BigEndian.PutUint32(tmp[:4], val32)
+		_, _ = md5Hash.Write(tmp[:4])
+	case 8: // uint64
+		binary.BigEndian.PutUint64(tmp[:8], val64)
+		_, _ = md5Hash.Write(tmp[:8])
+	}
+
+	// mask
+	if !field.HasMask {
+		tmp[0] = 0x00
+		_, _ = md5Hash.Write(tmp[:1]) // match hasMask = false
+	} else {
+		tmp[0] = 0x01
+		_, _ = md5Hash.Write(tmp[:1]) // match hasMask = true
+
+		maskType, mask32, mask64, maskSlice := uint8(0), uint32(0), uint64(0), []byte(nil)
+		switch mask := field.Mask.(type) {
+		case *ofp.OfpOxmOfbField_TableMetadataMask:
+			maskType, mask64 = 8, mask.TableMetadataMask
+		case *ofp.OfpOxmOfbField_EthDstMask:
+			maskType, maskSlice = 1, mask.EthDstMask
+		case *ofp.OfpOxmOfbField_EthSrcMask:
+			maskType, maskSlice = 1, mask.EthSrcMask
+		case *ofp.OfpOxmOfbField_VlanVidMask:
+			maskType, mask32 = 4, mask.VlanVidMask
+		case *ofp.OfpOxmOfbField_Ipv4SrcMask:
+			maskType, mask32 = 4, mask.Ipv4SrcMask
+		case *ofp.OfpOxmOfbField_Ipv4DstMask:
+			maskType, mask32 = 4, mask.Ipv4DstMask
+		case *ofp.OfpOxmOfbField_ArpSpaMask:
+			maskType, mask32 = 4, mask.ArpSpaMask
+		case *ofp.OfpOxmOfbField_ArpTpaMask:
+			maskType, mask32 = 4, mask.ArpTpaMask
+		case *ofp.OfpOxmOfbField_Ipv6SrcMask:
+			maskType, maskSlice = 1, mask.Ipv6SrcMask
+		case *ofp.OfpOxmOfbField_Ipv6DstMask:
+			maskType, maskSlice = 1, mask.Ipv6DstMask
+		case *ofp.OfpOxmOfbField_Ipv6FlabelMask:
+			maskType, mask32 = 4, mask.Ipv6FlabelMask
+		case *ofp.OfpOxmOfbField_PbbIsidMask:
+			maskType, mask32 = 4, mask.PbbIsidMask
+		case *ofp.OfpOxmOfbField_TunnelIdMask:
+			maskType, mask64 = 8, mask.TunnelIdMask
+		case *ofp.OfpOxmOfbField_Ipv6ExthdrMask:
+			maskType, mask32 = 4, mask.Ipv6ExthdrMask
+		case nil:
+			return fmt.Errorf("hasMask set to true, but no mask present")
+		default:
+			return fmt.Errorf("unknown OfpOxmField mask type: %T", mask)
+		}
+		switch maskType {
+		case 1: // slice
+			_, _ = md5Hash.Write(maskSlice)
+		case 4: // uint32
+			binary.BigEndian.PutUint32(tmp[:4], mask32)
+			_, _ = md5Hash.Write(tmp[:4])
+		case 8: // uint64
+			binary.BigEndian.PutUint64(tmp[:8], mask64)
+			_, _ = md5Hash.Write(tmp[:8])
+		}
+	}
+	return nil
+}
+
+// flowStatsEntryFromFlowModMessage maps an ofp_flow_mod message to an ofp_flow_stats message
+func FlowStatsEntryFromFlowModMessage(mod *ofp.OfpFlowMod) (*ofp.OfpFlowStats, error) {
+	flow := &ofp.OfpFlowStats{}
+	if mod == nil {
+		return flow, nil
+	}
+	flow.TableId = mod.TableId
+	flow.Priority = mod.Priority
+	flow.IdleTimeout = mod.IdleTimeout
+	flow.HardTimeout = mod.HardTimeout
+	flow.Flags = mod.Flags
+	flow.Cookie = mod.Cookie
+	flow.Match = mod.Match
+	flow.Instructions = mod.Instructions
+	var err error
+	if flow.Id, err = HashFlowStats(flow); err != nil {
+		return nil, err
+	}
+
+	return flow, nil
+}
+
+func GroupEntryFromGroupMod(mod *ofp.OfpGroupMod) *ofp.OfpGroupEntry {
+	group := &ofp.OfpGroupEntry{}
+	if mod == nil {
+		return group
+	}
+	group.Desc = &ofp.OfpGroupDesc{Type: mod.Type, GroupId: mod.GroupId, Buckets: mod.Buckets}
+	group.Stats = &ofp.OfpGroupStats{GroupId: mod.GroupId}
+	//TODO do we need to instantiate bucket bins?
+	return group
+}
+
+// flowStatsEntryFromFlowModMessage maps an ofp_flow_mod message to an ofp_flow_stats message
+func MeterEntryFromMeterMod(ctx context.Context, meterMod *ofp.OfpMeterMod) *ofp.OfpMeterEntry {
+	bandStats := make([]*ofp.OfpMeterBandStats, 0)
+	meter := &ofp.OfpMeterEntry{Config: &ofp.OfpMeterConfig{},
+		Stats: &ofp.OfpMeterStats{BandStats: bandStats}}
+	if meterMod == nil {
+		logger.Error(ctx, "Invalid meter mod command")
+		return meter
+	}
+	// config init
+	meter.Config.MeterId = meterMod.MeterId
+	meter.Config.Flags = meterMod.Flags
+	meter.Config.Bands = meterMod.Bands
+	// meter stats init
+	meter.Stats.MeterId = meterMod.MeterId
+	meter.Stats.FlowCount = 0
+	meter.Stats.PacketInCount = 0
+	meter.Stats.ByteInCount = 0
+	meter.Stats.DurationSec = 0
+	meter.Stats.DurationNsec = 0
+	// band stats init
+	for range meterMod.Bands {
+		band := &ofp.OfpMeterBandStats{}
+		band.PacketBandCount = 0
+		band.ByteBandCount = 0
+		bandStats = append(bandStats, band)
+	}
+	meter.Stats.BandStats = bandStats
+	logger.Debugw(ctx, "Allocated meter entry", log.Fields{"meter": *meter})
+	return meter
+
+}
+
+func GetMeterIdFromFlow(flow *ofp.OfpFlowStats) uint32 {
+	if flow != nil {
+		for _, instruction := range flow.Instructions {
+			if instruction.Type == uint32(METER_ACTION) {
+				if meterInst := instruction.GetMeter(); meterInst != nil {
+					return meterInst.GetMeterId()
+				}
+			}
+		}
+	}
+
+	return uint32(0)
+}
+
+func MkOxmFields(matchFields []ofp.OfpOxmField) []*ofp.OfpOxmField {
+	oxmFields := make([]*ofp.OfpOxmField, 0)
+	for _, matchField := range matchFields {
+		oxmField := ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: matchField.Field}
+		oxmFields = append(oxmFields, &oxmField)
+	}
+	return oxmFields
+}
+
+func MkInstructionsFromActions(actions []*ofp.OfpAction) []*ofp.OfpInstruction {
+	instructions := make([]*ofp.OfpInstruction, 0)
+	instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: actions}}
+	instruction := ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction}
+	instructions = append(instructions, &instruction)
+	return instructions
+}
+
+// Convenience function to generare ofp_flow_mod message with OXM BASIC match composed from the match_fields, and
+// single APPLY_ACTIONS instruction with a list if ofp_action objects.
+func MkSimpleFlowMod(matchFields []*ofp.OfpOxmField, actions []*ofp.OfpAction, command *ofp.OfpFlowModCommand, kw OfpFlowModArgs) *ofp.OfpFlowMod {
+
+	// Process actions instructions
+	instructions := make([]*ofp.OfpInstruction, 0)
+	instructionAction := ofp.OfpInstruction_Actions{Actions: &ofp.OfpInstructionActions{Actions: actions}}
+	instruction := ofp.OfpInstruction{Type: uint32(APPLY_ACTIONS), Data: &instructionAction}
+	instructions = append(instructions, &instruction)
+
+	// Process next table
+	if tableId := GetNextTableId(kw); tableId != nil {
+		var instGotoTable ofp.OfpInstruction_GotoTable
+		instGotoTable.GotoTable = &ofp.OfpInstructionGotoTable{TableId: *tableId}
+		inst := ofp.OfpInstruction{Type: uint32(ofp.OfpInstructionType_OFPIT_GOTO_TABLE), Data: &instGotoTable}
+		instructions = append(instructions, &inst)
+	}
+	// Process meter action
+	if meterId := GetMeterIdFlowModArgs(kw); meterId != 0 {
+		var instMeter ofp.OfpInstruction_Meter
+		instMeter.Meter = &ofp.OfpInstructionMeter{MeterId: meterId}
+		inst := ofp.OfpInstruction{Type: uint32(METER_ACTION), Data: &instMeter}
+		instructions = append(instructions, &inst)
+	}
+	//process write_metadata action
+	if metadata := GetMetadataFlowModArgs(kw); metadata != 0 {
+		var instWriteMetadata ofp.OfpInstruction_WriteMetadata
+		instWriteMetadata.WriteMetadata = &ofp.OfpInstructionWriteMetadata{Metadata: metadata}
+		inst := ofp.OfpInstruction{Type: uint32(WRITE_METADATA), Data: &instWriteMetadata}
+		instructions = append(instructions, &inst)
+	}
+
+	// Process match fields
+	oxmFields := make([]*ofp.OfpOxmField, 0)
+	for _, matchField := range matchFields {
+		oxmField := ofp.OfpOxmField{OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC, Field: matchField.Field}
+		oxmFields = append(oxmFields, &oxmField)
+	}
+	var match ofp.OfpMatch
+	match.Type = ofp.OfpMatchType_OFPMT_OXM
+	match.OxmFields = oxmFields
+
+	// Create ofp_flow_message
+	msg := &ofp.OfpFlowMod{}
+	if command == nil {
+		msg.Command = ofp.OfpFlowModCommand_OFPFC_ADD
+	} else {
+		msg.Command = *command
+	}
+	msg.Instructions = instructions
+	msg.Match = &match
+
+	// Set the variadic argument values
+	msg = setVariadicModAttributes(msg, kw)
+
+	return msg
+}
+
+func MkMulticastGroupMod(groupId uint32, buckets []*ofp.OfpBucket, command *ofp.OfpGroupModCommand) *ofp.OfpGroupMod {
+	group := &ofp.OfpGroupMod{}
+	if command == nil {
+		group.Command = ofp.OfpGroupModCommand_OFPGC_ADD
+	} else {
+		group.Command = *command
+	}
+	group.Type = ofp.OfpGroupType_OFPGT_ALL
+	group.GroupId = groupId
+	group.Buckets = buckets
+	return group
+}
+
+//SetVariadicModAttributes sets only uint64 or uint32 fields of the ofp_flow_mod message
+func setVariadicModAttributes(mod *ofp.OfpFlowMod, args OfpFlowModArgs) *ofp.OfpFlowMod {
+	if args == nil {
+		return mod
+	}
+	for key, val := range args {
+		switch key {
+		case "cookie":
+			mod.Cookie = val
+		case "cookie_mask":
+			mod.CookieMask = val
+		case "table_id":
+			mod.TableId = uint32(val)
+		case "idle_timeout":
+			mod.IdleTimeout = uint32(val)
+		case "hard_timeout":
+			mod.HardTimeout = uint32(val)
+		case "priority":
+			mod.Priority = uint32(val)
+		case "buffer_id":
+			mod.BufferId = uint32(val)
+		case "out_port":
+			mod.OutPort = uint32(val)
+		case "out_group":
+			mod.OutGroup = uint32(val)
+		case "flags":
+			mod.Flags = uint32(val)
+		}
+	}
+	return mod
+}
+
+func MkPacketIn(port uint32, packet []byte) *ofp.OfpPacketIn {
+	packetIn := &ofp.OfpPacketIn{
+		Reason: ofp.OfpPacketInReason_OFPR_ACTION,
+		Match: &ofp.OfpMatch{
+			Type: ofp.OfpMatchType_OFPMT_OXM,
+			OxmFields: []*ofp.OfpOxmField{
+				{
+					OxmClass: ofp.OfpOxmClass_OFPXMC_OPENFLOW_BASIC,
+					Field: &ofp.OfpOxmField_OfbField{
+						OfbField: InPort(port)},
+				},
+			},
+		},
+		Data: packet,
+	}
+	return packetIn
+}
+
+// MkFlowStat is a helper method to build flows
+func MkFlowStat(fa *FlowArgs) (*ofp.OfpFlowStats, error) {
+	//Build the match-fields
+	matchFields := make([]*ofp.OfpOxmField, 0)
+	for _, val := range fa.MatchFields {
+		matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
+	}
+	return FlowStatsEntryFromFlowModMessage(MkSimpleFlowMod(matchFields, fa.Actions, fa.Command, fa.KV))
+}
+
+func MkGroupStat(ga *GroupArgs) *ofp.OfpGroupEntry {
+	return GroupEntryFromGroupMod(MkMulticastGroupMod(ga.GroupId, ga.Buckets, ga.Command))
+}
+
+type OfpFlowModArgs map[string]uint64
+
+type FlowArgs struct {
+	MatchFields []*ofp.OfpOxmOfbField
+	Actions     []*ofp.OfpAction
+	Command     *ofp.OfpFlowModCommand
+	Priority    uint32
+	KV          OfpFlowModArgs
+}
+
+type GroupArgs struct {
+	GroupId uint32
+	Buckets []*ofp.OfpBucket
+	Command *ofp.OfpGroupModCommand
+}
+
+type FlowsAndGroups struct {
+	Flows  *ordered_map.OrderedMap
+	Groups *ordered_map.OrderedMap
+}
+
+func NewFlowsAndGroups() *FlowsAndGroups {
+	var fg FlowsAndGroups
+	fg.Flows = ordered_map.NewOrderedMap()
+	fg.Groups = ordered_map.NewOrderedMap()
+	return &fg
+}
+
+func (fg *FlowsAndGroups) Copy() *FlowsAndGroups {
+	copyFG := NewFlowsAndGroups()
+	iter := fg.Flows.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
+			copyFG.Flows.Set(kv.Key, proto.Clone(protoMsg))
+		}
+	}
+	iter = fg.Groups.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
+			copyFG.Groups.Set(kv.Key, proto.Clone(protoMsg))
+		}
+	}
+	return copyFG
+}
+
+func (fg *FlowsAndGroups) GetFlow(index int) *ofp.OfpFlowStats {
+	iter := fg.Flows.IterFunc()
+	pos := 0
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if pos == index {
+			if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
+				return protoMsg
+			}
+			return nil
+		}
+		pos += 1
+	}
+	return nil
+}
+
+func (fg *FlowsAndGroups) ListFlows() []*ofp.OfpFlowStats {
+	flows := make([]*ofp.OfpFlowStats, 0)
+	iter := fg.Flows.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
+			flows = append(flows, protoMsg)
+		}
+	}
+	return flows
+}
+
+func (fg *FlowsAndGroups) ListGroups() []*ofp.OfpGroupEntry {
+	groups := make([]*ofp.OfpGroupEntry, 0)
+	iter := fg.Groups.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
+			groups = append(groups, protoMsg)
+		}
+	}
+	return groups
+}
+
+func (fg *FlowsAndGroups) String() string {
+	var buffer bytes.Buffer
+	iter := fg.Flows.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
+			buffer.WriteString("\nFlow:\n")
+			buffer.WriteString(proto.MarshalTextString(protoMsg))
+			buffer.WriteString("\n")
+		}
+	}
+	iter = fg.Groups.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
+			buffer.WriteString("\nGroup:\n")
+			buffer.WriteString(proto.MarshalTextString(protoMsg))
+			buffer.WriteString("\n")
+		}
+	}
+	return buffer.String()
+}
+
+func (fg *FlowsAndGroups) AddFlow(flow *ofp.OfpFlowStats) {
+	if flow == nil {
+		return
+	}
+
+	if fg.Flows == nil {
+		fg.Flows = ordered_map.NewOrderedMap()
+	}
+	if fg.Groups == nil {
+		fg.Groups = ordered_map.NewOrderedMap()
+	}
+	//Add flow only if absent
+	if _, exist := fg.Flows.Get(flow.Id); !exist {
+		fg.Flows.Set(flow.Id, flow)
+	}
+}
+
+func (fg *FlowsAndGroups) AddGroup(group *ofp.OfpGroupEntry) {
+	if group == nil {
+		return
+	}
+
+	if fg.Flows == nil {
+		fg.Flows = ordered_map.NewOrderedMap()
+	}
+	if fg.Groups == nil {
+		fg.Groups = ordered_map.NewOrderedMap()
+	}
+	//Add group only if absent
+	if _, exist := fg.Groups.Get(group.Desc.GroupId); !exist {
+		fg.Groups.Set(group.Desc.GroupId, group)
+	}
+}
+
+//AddFrom add flows and groups from the argument into this structure only if they do not already exist
+func (fg *FlowsAndGroups) AddFrom(from *FlowsAndGroups) {
+	iter := from.Flows.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpFlowStats); isMsg {
+			if _, exist := fg.Flows.Get(protoMsg.Id); !exist {
+				fg.Flows.Set(protoMsg.Id, protoMsg)
+			}
+		}
+	}
+	iter = from.Groups.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		if protoMsg, isMsg := kv.Value.(*ofp.OfpGroupEntry); isMsg {
+			if _, exist := fg.Groups.Get(protoMsg.Stats.GroupId); !exist {
+				fg.Groups.Set(protoMsg.Stats.GroupId, protoMsg)
+			}
+		}
+	}
+}
+
+type DeviceRules struct {
+	Rules     map[string]*FlowsAndGroups
+	rulesLock sync.RWMutex
+}
+
+func NewDeviceRules() *DeviceRules {
+	var dr DeviceRules
+	dr.Rules = make(map[string]*FlowsAndGroups)
+	return &dr
+}
+
+func (dr *DeviceRules) Copy() *DeviceRules {
+	copyDR := NewDeviceRules()
+	if dr != nil {
+		dr.rulesLock.RLock()
+		defer dr.rulesLock.RUnlock()
+		for key, val := range dr.Rules {
+			if val != nil {
+				copyDR.Rules[key] = val.Copy()
+			}
+		}
+	}
+	return copyDR
+}
+
+func (dr *DeviceRules) Keys() []string {
+	dr.rulesLock.RLock()
+	defer dr.rulesLock.RUnlock()
+	keys := make([]string, 0, len(dr.Rules))
+	for k := range dr.Rules {
+		keys = append(keys, k)
+	}
+	return keys
+}
+
+func (dr *DeviceRules) ClearFlows(deviceId string) {
+	dr.rulesLock.Lock()
+	defer dr.rulesLock.Unlock()
+	if _, exist := dr.Rules[deviceId]; exist {
+		dr.Rules[deviceId].Flows = ordered_map.NewOrderedMap()
+	}
+}
+
+func (dr *DeviceRules) FilterRules(deviceIds map[string]string) *DeviceRules {
+	filteredDR := NewDeviceRules()
+	dr.rulesLock.RLock()
+	defer dr.rulesLock.RUnlock()
+	for key, val := range dr.Rules {
+		if _, exist := deviceIds[key]; exist {
+			filteredDR.Rules[key] = val.Copy()
+		}
+	}
+	return filteredDR
+}
+
+func (dr *DeviceRules) AddFlow(deviceId string, flow *ofp.OfpFlowStats) {
+	dr.rulesLock.Lock()
+	defer dr.rulesLock.Unlock()
+	if _, exist := dr.Rules[deviceId]; !exist {
+		dr.Rules[deviceId] = NewFlowsAndGroups()
+	}
+	dr.Rules[deviceId].AddFlow(flow)
+}
+
+func (dr *DeviceRules) GetRules() map[string]*FlowsAndGroups {
+	dr.rulesLock.RLock()
+	defer dr.rulesLock.RUnlock()
+	return dr.Rules
+}
+
+func (dr *DeviceRules) String() string {
+	var buffer bytes.Buffer
+	dr.rulesLock.RLock()
+	defer dr.rulesLock.RUnlock()
+	for key, value := range dr.Rules {
+		buffer.WriteString("DeviceId:")
+		buffer.WriteString(key)
+		buffer.WriteString(value.String())
+		buffer.WriteString("\n\n")
+	}
+	return buffer.String()
+}
+
+func (dr *DeviceRules) AddFlowsAndGroup(deviceId string, fg *FlowsAndGroups) {
+	dr.rulesLock.Lock()
+	defer dr.rulesLock.Unlock()
+	if _, ok := dr.Rules[deviceId]; !ok {
+		dr.Rules[deviceId] = NewFlowsAndGroups()
+	}
+	dr.Rules[deviceId] = fg
+}
+
+// CreateEntryIfNotExist creates a new deviceId in the Map if it does not exist and assigns an
+// empty FlowsAndGroups to it.  Otherwise, it does nothing.
+func (dr *DeviceRules) CreateEntryIfNotExist(deviceId string) {
+	dr.rulesLock.Lock()
+	defer dr.rulesLock.Unlock()
+	if _, ok := dr.Rules[deviceId]; !ok {
+		dr.Rules[deviceId] = NewFlowsAndGroups()
+	}
+}
+
+/*
+ *  Common flow routines
+ */
+
+//FindOverlappingFlows return a list of overlapping flow(s) where mod is the flow request
+func FindOverlappingFlows(flows []*ofp.OfpFlowStats, mod *ofp.OfpFlowMod) []*ofp.OfpFlowStats {
+	return nil //TODO - complete implementation
+}
+
+// FindFlowById returns the index of the flow in the flows array if present. Otherwise, it returns -1
+func FindFlowById(flows []*ofp.OfpFlowStats, flow *ofp.OfpFlowStats) int {
+	for idx, f := range flows {
+		if flow.Id == f.Id {
+			return idx
+		}
+	}
+	return -1
+}
+
+// FindFlows returns the index in flows where flow if present.  Otherwise, it returns -1
+func FindFlows(flows []*ofp.OfpFlowStats, flow *ofp.OfpFlowStats) int {
+	for idx, f := range flows {
+		if f.Id == flow.Id {
+			return idx
+		}
+	}
+	return -1
+}
+
+//FlowMatch returns true if two flows matches on the following flow attributes:
+//TableId, Priority, Flags, Cookie, Match
+func FlowMatch(f1 *ofp.OfpFlowStats, f2 *ofp.OfpFlowStats) bool {
+	return f1 != nil && f2 != nil && f1.Id == f2.Id
+}
+
+//FlowMatchesMod returns True if given flow is "covered" by the wildcard flow_mod, taking into consideration of
+//both exact matches as well as masks-based match fields if any. Otherwise return False
+func FlowMatchesMod(flow *ofp.OfpFlowStats, mod *ofp.OfpFlowMod) bool {
+	if flow == nil || mod == nil {
+		return false
+	}
+	//Check if flow.cookie is covered by mod.cookie and mod.cookie_mask
+	if (flow.Cookie & mod.CookieMask) != (mod.Cookie & mod.CookieMask) {
+		return false
+	}
+
+	//Check if flow.table_id is covered by flow_mod.table_id
+	if mod.TableId != uint32(ofp.OfpTable_OFPTT_ALL) && flow.TableId != mod.TableId {
+		return false
+	}
+
+	//Check out_port
+	if (mod.OutPort&0x7fffffff) != uint32(ofp.OfpPortNo_OFPP_ANY) && !FlowHasOutPort(flow, mod.OutPort) {
+		return false
+	}
+
+	//	Check out_group
+	if (mod.OutGroup&0x7fffffff) != uint32(ofp.OfpGroup_OFPG_ANY) && !FlowHasOutGroup(flow, mod.OutGroup) {
+		return false
+	}
+
+	//Priority is ignored
+
+	//Check match condition
+	//If the flow_mod match field is empty, that is a special case and indicates the flow entry matches
+	if (mod.Match == nil) || (mod.Match.OxmFields == nil) || (len(mod.Match.OxmFields) == 0) {
+		//If we got this far and the match is empty in the flow spec, than the flow matches
+		return true
+	} // TODO : implement the flow match analysis
+	return false
+
+}
+
+//FlowHasOutPort returns True if flow has a output command with the given out_port
+func FlowHasOutPort(flow *ofp.OfpFlowStats, outPort uint32) bool {
+	if flow == nil {
+		return false
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
+			if instruction.GetActions() == nil {
+				return false
+			}
+			for _, action := range instruction.GetActions().Actions {
+				if action.Type == ofp.OfpActionType_OFPAT_OUTPUT {
+					if (action.GetOutput() != nil) && (action.GetOutput().Port == outPort) {
+						return true
+					}
+				}
+
+			}
+		}
+	}
+	return false
+}
+
+//FlowHasOutGroup return True if flow has a output command with the given out_group
+func FlowHasOutGroup(flow *ofp.OfpFlowStats, groupID uint32) bool {
+	if flow == nil {
+		return false
+	}
+	for _, instruction := range flow.Instructions {
+		if instruction.Type == uint32(ofp.OfpInstructionType_OFPIT_APPLY_ACTIONS) {
+			if instruction.GetActions() == nil {
+				return false
+			}
+			for _, action := range instruction.GetActions().Actions {
+				if action.Type == ofp.OfpActionType_OFPAT_GROUP {
+					if (action.GetGroup() != nil) && (action.GetGroup().GroupId == groupID) {
+						return true
+					}
+				}
+
+			}
+		}
+	}
+	return false
+}
+
+//FindGroup returns index of group if found, else returns -1
+func FindGroup(groups []*ofp.OfpGroupEntry, groupId uint32) int {
+	for idx, group := range groups {
+		if group.Desc.GroupId == groupId {
+			return idx
+		}
+	}
+	return -1
+}
+
+func FlowsDeleteByGroupId(flows []*ofp.OfpFlowStats, groupId uint32) (bool, []*ofp.OfpFlowStats) {
+	toKeep := make([]*ofp.OfpFlowStats, 0)
+
+	for _, f := range flows {
+		if !FlowHasOutGroup(f, groupId) {
+			toKeep = append(toKeep, f)
+		}
+	}
+	return len(toKeep) < len(flows), toKeep
+}
+
+func ToOfpOxmField(from []*ofp.OfpOxmOfbField) []*ofp.OfpOxmField {
+	matchFields := make([]*ofp.OfpOxmField, 0)
+	for _, val := range from {
+		matchFields = append(matchFields, &ofp.OfpOxmField{Field: &ofp.OfpOxmField_OfbField{OfbField: val}})
+	}
+	return matchFields
+}
+
+//IsMulticastIp returns true if the ip starts with the byte sequence of 1110;
+//false otherwise.
+func IsMulticastIp(ip uint32) bool {
+	return ip>>28 == 14
+}
+
+//ConvertToMulticastMacInt returns equivalent mac address of the given multicast ip address
+func ConvertToMulticastMacInt(ip uint32) uint64 {
+	//get last 23 bits of ip address by ip & 00000000011111111111111111111111
+	theLast23BitsOfIp := ip & 8388607
+	// perform OR with 0x1005E000000 to build mcast mac address
+	return 1101088686080 | uint64(theLast23BitsOfIp)
+}
+
+//ConvertToMulticastMacBytes returns equivalent mac address of the given multicast ip address
+func ConvertToMulticastMacBytes(ip uint32) []byte {
+	mac := ConvertToMulticastMacInt(ip)
+	var b bytes.Buffer
+	// catalyze (48 bits) in binary:111111110000000000000000000000000000000000000000
+	catalyze := uint64(280375465082880)
+	//convert each octet to decimal
+	for i := 0; i < 6; i++ {
+		if i != 0 {
+			catalyze >>= 8
+		}
+		octet := mac & catalyze
+		octetDecimal := octet >> uint8(40-i*8)
+		b.WriteByte(byte(octetDecimal))
+	}
+	return b.Bytes()
+}
+
+func GetMeterIdFromWriteMetadata(ctx context.Context, flow *ofp.OfpFlowStats) uint32 {
+	/*
+			  Write metadata instruction value (metadata) is 8 bytes:
+		    	MS 2 bytes: C Tag
+		    	Next 2 bytes: Technology Profile Id
+		    	Next 4 bytes: Port number (uni or nni) or MeterId
+		    	This is set in the ONOS OltPipeline as a write metadata instruction
+	*/
+	var meterID uint32 = 0
+	md := GetMetadataFromWriteMetadataAction(ctx, flow)
+	logger.Debugw(ctx, "found-metadata-for-egress/uni-port", log.Fields{"metadata": md})
+	if md != 0 {
+		meterID = uint32(md & 0xFFFFFFFF)
+		logger.Debugw(ctx, "found-meterID-in-write-metadata-action", log.Fields{"meterID": meterID})
+	}
+	return meterID
+}
+
+func SetMeterIdToFlow(flow *ofp.OfpFlowStats, meterId uint32) {
+	if flow != nil {
+		for _, instruction := range flow.Instructions {
+			if instruction.Type == uint32(METER_ACTION) {
+				if meterInst := instruction.GetMeter(); meterInst != nil {
+					meterInst.MeterId = meterId
+				}
+			}
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/client.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/client.go
new file mode 100644
index 0000000..de649d6
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/client.go
@@ -0,0 +1,541 @@
+/*
+ * Copyright 2021-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 grpc
+
+import (
+	"context"
+	"fmt"
+	"reflect"
+	"strings"
+	"sync"
+	"time"
+
+	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
+	grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+	"github.com/opencord/voltha-protos/v5/go/adapter_services"
+	"github.com/opencord/voltha-protos/v5/go/core"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/keepalive"
+)
+
+type event byte
+type state byte
+type SetAndTestServiceHandler func(context.Context, *grpc.ClientConn) interface{}
+type RestartedHandler func(ctx context.Context, endPoint string) error
+
+type contextKey string
+
+func (c contextKey) String() string {
+	return string(c)
+}
+
+var (
+	grpcMonitorContextKey = contextKey("grpc-monitor")
+)
+
+const (
+	grpcBackoffInitialInterval = "GRPC_BACKOFF_INITIAL_INTERVAL"
+	grpcBackoffMaxInterval     = "GRPC_BACKOFF_MAX_INTERVAL"
+	grpcBackoffMaxElapsedTime  = "GRPC_BACKOFF_MAX_ELAPSED_TIME"
+	grpcMonitorInterval        = "GRPC_MONITOR_INTERVAL"
+)
+
+const (
+	DefaultBackoffInitialInterval = 100 * time.Millisecond
+	DefaultBackoffMaxInterval     = 5 * time.Second
+	DefaultBackoffMaxElapsedTime  = 0 * time.Second // No time limit
+	DefaultGRPCMonitorInterval    = 5 * time.Second
+)
+
+const (
+	connectionErrorSubString  = "SubConns are in TransientFailure"
+	connectionClosedSubstring = "client connection is closing"
+	connectionError           = "connection error"
+	connectionSystemNotReady  = "system is not ready"
+)
+
+const (
+	eventConnecting = event(iota)
+	eventConnected
+	eventDisconnected
+	eventStopped
+	eventError
+
+	stateConnected = state(iota)
+	stateConnecting
+	stateDisconnected
+)
+
+type Client struct {
+	apiEndPoint            string
+	connection             *grpc.ClientConn
+	connectionLock         sync.RWMutex
+	stateLock              sync.RWMutex
+	state                  state
+	service                interface{}
+	events                 chan event
+	onRestart              RestartedHandler
+	backoffInitialInterval time.Duration
+	backoffMaxInterval     time.Duration
+	backoffMaxElapsedTime  time.Duration
+	activityCheck          bool
+	monitorInterval        time.Duration
+	activeCh               chan struct{}
+	activeChMutex          sync.RWMutex
+	done                   bool
+	livenessCallback       func(timestamp time.Time)
+}
+
+type ClientOption func(*Client)
+
+func ActivityCheck(enable bool) ClientOption {
+	return func(args *Client) {
+		args.activityCheck = enable
+	}
+}
+
+func NewClient(endpoint string, onRestart RestartedHandler, opts ...ClientOption) (*Client, error) {
+	c := &Client{
+		apiEndPoint:            endpoint,
+		onRestart:              onRestart,
+		events:                 make(chan event, 1),
+		state:                  stateDisconnected,
+		backoffInitialInterval: DefaultBackoffInitialInterval,
+		backoffMaxInterval:     DefaultBackoffMaxInterval,
+		backoffMaxElapsedTime:  DefaultBackoffMaxElapsedTime,
+		monitorInterval:        DefaultGRPCMonitorInterval,
+	}
+	for _, option := range opts {
+		option(c)
+	}
+
+	// Check for environment variables
+	if err := SetFromEnvVariable(grpcBackoffInitialInterval, &c.backoffInitialInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffInitialInterval})
+	}
+
+	if err := SetFromEnvVariable(grpcBackoffMaxInterval, &c.backoffMaxInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffMaxInterval})
+	}
+
+	if err := SetFromEnvVariable(grpcBackoffMaxElapsedTime, &c.backoffMaxElapsedTime); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcBackoffMaxElapsedTime})
+	}
+
+	if err := SetFromEnvVariable(grpcMonitorInterval, &c.monitorInterval); err != nil {
+		logger.Warnw(context.Background(), "failure-reading-env-variable", log.Fields{"error": err, "variable": grpcMonitorInterval})
+	}
+
+	logger.Infow(context.Background(), "initialized-client", log.Fields{"client": c})
+
+	// Sanity check
+	if c.backoffInitialInterval > c.backoffMaxInterval {
+		return nil, fmt.Errorf("initial retry delay %v is greater than maximum retry delay %v", c.backoffInitialInterval, c.backoffMaxInterval)
+	}
+
+	return c, nil
+}
+
+func (c *Client) GetClient() (interface{}, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no connection to %s", c.apiEndPoint)
+	}
+	return c.service, nil
+}
+
+// GetCoreServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetCoreServiceClient() (core.CoreServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no core connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(core.CoreServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+// GetOnuAdapterServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetOnuInterAdapterServiceClient() (adapter_services.OnuInterAdapterServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no child adapter connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(adapter_services.OnuInterAdapterServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+// GetOltAdapterServiceClient is a helper function that returns a concrete service instead of the GetClient() API
+// which returns an interface
+func (c *Client) GetOltInterAdapterServiceClient() (adapter_services.OltInterAdapterServiceClient, error) {
+	c.connectionLock.RLock()
+	defer c.connectionLock.RUnlock()
+	if c.service == nil {
+		return nil, fmt.Errorf("no parent adapter connection to %s", c.apiEndPoint)
+	}
+	client, ok := c.service.(adapter_services.OltInterAdapterServiceClient)
+	if ok {
+		return client, nil
+	}
+	return nil, fmt.Errorf("invalid-service-%s", reflect.TypeOf(c.service))
+}
+
+func (c *Client) Reset(ctx context.Context) {
+	logger.Debugw(ctx, "resetting-client-connection", log.Fields{"endpoint": c.apiEndPoint})
+	c.stateLock.Lock()
+	defer c.stateLock.Unlock()
+	if c.state == stateConnected {
+		c.state = stateDisconnected
+		c.events <- eventDisconnected
+	}
+}
+
+func (c *Client) clientInterceptor(ctx context.Context, method string, req interface{}, reply interface{},
+	cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+	// Nothing to do before intercepting the call
+	err := invoker(ctx, method, req, reply, cc, opts...)
+	// On connection failure, start the reconnect process depending on the error response
+	if err != nil {
+		logger.Errorw(ctx, "received-error", log.Fields{"error": err, "context": ctx, "endpoint": c.apiEndPoint})
+		if strings.Contains(err.Error(), connectionErrorSubString) ||
+			strings.Contains(err.Error(), connectionError) ||
+			strings.Contains(err.Error(), connectionSystemNotReady) ||
+			isGrpcMonitorKeyPresentInContext(ctx) {
+			c.stateLock.Lock()
+			if c.state == stateConnected {
+				logger.Warnw(context.Background(), "sending-disconnect-event", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+				c.state = stateDisconnected
+				c.events <- eventDisconnected
+			}
+			c.stateLock.Unlock()
+		} else if strings.Contains(err.Error(), connectionClosedSubstring) {
+			logger.Errorw(context.Background(), "invalid-client-connection-closed", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+		}
+		return err
+	}
+	// Update activity on success only
+	c.updateActivity(ctx)
+	return nil
+}
+
+// updateActivity pushes an activity indication on the channel so that the monitoring routine does not validate
+// the gRPC connection when the connection is being used. Note that this update is done both when the connection
+// is alive or a connection error is returned. A separate routine takes care of doing the re-connect.
+func (c *Client) updateActivity(ctx context.Context) {
+	c.activeChMutex.RLock()
+	defer c.activeChMutex.RUnlock()
+	if c.activeCh != nil {
+		logger.Debugw(ctx, "update-activity", log.Fields{"api-endpoint": c.apiEndPoint})
+		c.activeCh <- struct{}{}
+
+		// Update liveness only in connected state
+		if c.livenessCallback != nil {
+			c.stateLock.RLock()
+			if c.state == stateConnected {
+				c.livenessCallback(time.Now())
+			}
+			c.stateLock.RUnlock()
+		}
+	}
+}
+
+func WithGrpcMonitorContext(ctx context.Context, name string) context.Context {
+	ctx = context.WithValue(ctx, grpcMonitorContextKey, name)
+	return ctx
+}
+
+func isGrpcMonitorKeyPresentInContext(ctx context.Context) bool {
+	if ctx != nil {
+		_, present := ctx.Value(grpcMonitorContextKey).(string)
+		return present
+	}
+	return false
+}
+
+// monitorActivity monitors the activity on the gRPC connection.   If there are no activity after a specified
+// timeout, it will send a default API request on that connection.   If the connection is good then nothing
+// happens.  If it's bad this will trigger reconnection attempts.
+func (c *Client) monitorActivity(ctx context.Context, handler SetAndTestServiceHandler) {
+	logger.Infow(ctx, "start-activity-monitor", log.Fields{"endpoint": c.apiEndPoint})
+
+	// Create an activity monitor channel.  Unbuffered channel works well.  However, we use a buffered
+	// channel here as a safeguard of having the grpc interceptor publishing too many events that can be
+	// consumed by this monitoring thread
+	c.activeChMutex.Lock()
+	c.activeCh = make(chan struct{}, 10)
+	c.activeChMutex.Unlock()
+
+	// Interval to wait for no activity before probing the connection
+	timeout := c.monitorInterval
+loop:
+	for {
+		timeoutTimer := time.NewTimer(timeout)
+		select {
+
+		case <-c.activeCh:
+			logger.Debugw(ctx, "received-active-notification", log.Fields{"endpoint": c.apiEndPoint})
+
+			// Reset timer
+			if !timeoutTimer.Stop() {
+				<-timeoutTimer.C
+			}
+
+		case <-ctx.Done():
+			break loop
+
+		case <-timeoutTimer.C:
+			// Trigger an activity check if the state is connected.  If the state is not connected then there is already
+			// a backoff retry mechanism in place to retry establishing connection.
+			c.stateLock.RLock()
+			runCheck := c.state == stateConnected
+			c.stateLock.RUnlock()
+			if runCheck {
+				go func() {
+					logger.Debugw(ctx, "connection-check-start", log.Fields{"api-endpoint": c.apiEndPoint})
+					subCtx, cancel := context.WithTimeout(ctx, c.backoffMaxInterval)
+					defer cancel()
+					subCtx = WithGrpcMonitorContext(subCtx, "grpc-monitor")
+					c.connectionLock.RLock()
+					defer c.connectionLock.RUnlock()
+					if c.connection != nil {
+						response := handler(subCtx, c.connection)
+						logger.Debugw(ctx, "connection-check-response", log.Fields{"api-endpoint": c.apiEndPoint, "up": response != nil})
+					}
+				}()
+			}
+		}
+	}
+	logger.Infow(ctx, "activity-monitor-stopping", log.Fields{"endpoint": c.apiEndPoint})
+}
+
+// Start kicks off the adapter agent by trying to connect to the adapter
+func (c *Client) Start(ctx context.Context, handler SetAndTestServiceHandler) {
+	logger.Debugw(ctx, "Starting GRPC - Client", log.Fields{"api-endpoint": c.apiEndPoint})
+
+	// If the context contains a k8s probe then register services
+	p := probe.GetProbeFromContext(ctx)
+	if p != nil {
+		p.RegisterService(ctx, c.apiEndPoint)
+	}
+
+	// Enable activity check, if required
+	if c.activityCheck {
+		go c.monitorActivity(ctx, handler)
+	}
+
+	initialConnection := true
+	c.events <- eventConnecting
+	backoff := NewBackoff(c.backoffInitialInterval, c.backoffMaxInterval, c.backoffMaxElapsedTime)
+	attempt := 1
+loop:
+	for {
+		select {
+		case <-ctx.Done():
+			logger.Debugw(ctx, "context-closing", log.Fields{"endpoint": c.apiEndPoint})
+			return
+		case event := <-c.events:
+			logger.Debugw(ctx, "received-event", log.Fields{"event": event, "endpoint": c.apiEndPoint})
+			switch event {
+			case eventConnecting:
+				logger.Debugw(ctx, "connection-start", log.Fields{"endpoint": c.apiEndPoint, "attempts": attempt})
+
+				c.stateLock.Lock()
+				if c.state == stateConnected {
+					c.state = stateDisconnected
+				}
+				if c.state != stateConnecting {
+					c.state = stateConnecting
+					go func() {
+						if err := c.connectToEndpoint(ctx, handler, p); err != nil {
+							c.stateLock.Lock()
+							c.state = stateDisconnected
+							c.stateLock.Unlock()
+							logger.Errorw(ctx, "connection-failed", log.Fields{"endpoint": c.apiEndPoint, "attempt": attempt, "error": err})
+
+							// Retry connection after a delay
+							if err = backoff.Backoff(ctx); err != nil {
+								// Context has closed or reached maximum elapsed time, if set
+								logger.Errorw(ctx, "retry-aborted", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+								return
+							}
+							attempt += 1
+							c.events <- eventConnecting
+						} else {
+							backoff.Reset()
+						}
+					}()
+				}
+				c.stateLock.Unlock()
+
+			case eventConnected:
+				logger.Debugw(ctx, "endpoint-connected", log.Fields{"endpoint": c.apiEndPoint})
+				attempt = 1
+				c.stateLock.Lock()
+				if c.state != stateConnected {
+					c.state = stateConnected
+					if initialConnection {
+						logger.Debugw(ctx, "initial-endpoint-connection", log.Fields{"endpoint": c.apiEndPoint})
+						initialConnection = false
+					} else {
+						logger.Debugw(ctx, "endpoint-reconnection", log.Fields{"endpoint": c.apiEndPoint})
+						// Trigger any callback on a restart
+						go func() {
+							err := c.onRestart(log.WithSpanFromContext(context.Background(), ctx), c.apiEndPoint)
+							if err != nil {
+								logger.Errorw(ctx, "unable-to-restart-endpoint", log.Fields{"error": err, "endpoint": c.apiEndPoint})
+							}
+						}()
+					}
+				}
+				c.stateLock.Unlock()
+
+			case eventDisconnected:
+				if p != nil {
+					p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusNotReady)
+				}
+				logger.Debugw(ctx, "endpoint-disconnected", log.Fields{"endpoint": c.apiEndPoint, "status": c.state})
+
+				// Try to connect again
+				c.events <- eventConnecting
+
+			case eventStopped:
+				logger.Debugw(ctx, "endPoint-stopped", log.Fields{"adapter": c.apiEndPoint})
+				go func() {
+					if err := c.closeConnection(ctx, p); err != nil {
+						logger.Errorw(ctx, "endpoint-closing-connection-failed", log.Fields{"endpoint": c.apiEndPoint, "error": err})
+					}
+				}()
+				break loop
+			case eventError:
+				logger.Errorw(ctx, "endpoint-error-event", log.Fields{"endpoint": c.apiEndPoint})
+			default:
+				logger.Errorw(ctx, "endpoint-unknown-event", log.Fields{"endpoint": c.apiEndPoint, "error": event})
+			}
+		}
+	}
+	logger.Infow(ctx, "endpoint-stopped", log.Fields{"endpoint": c.apiEndPoint})
+}
+
+func (c *Client) connectToEndpoint(ctx context.Context, handler SetAndTestServiceHandler, p *probe.Probe) error {
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusPreparing)
+	}
+
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+
+	if c.connection != nil {
+		_ = c.connection.Close()
+		c.connection = nil
+	}
+
+	c.service = nil
+
+	// Use Interceptors to:
+	// 1. automatically inject
+	// 2. publish Open Tracing Spans by this GRPC Client
+	// 3. detect connection failure on client calls such that the reconnection process can begin
+	conn, err := grpc.Dial(c.apiEndPoint,
+		grpc.WithInsecure(),
+		grpc.WithStreamInterceptor(grpc_middleware.ChainStreamClient(
+			grpc_opentracing.StreamClientInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+		)),
+		grpc.WithUnaryInterceptor(grpc_middleware.ChainUnaryClient(
+			grpc_opentracing.UnaryClientInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+		)),
+		grpc.WithUnaryInterceptor(c.clientInterceptor),
+		// Set keealive parameter - use default grpc values
+		grpc.WithKeepaliveParams(keepalive.ClientParameters{
+			Time:                c.monitorInterval,
+			Timeout:             c.backoffMaxInterval,
+			PermitWithoutStream: true,
+		}),
+	)
+
+	if err == nil {
+		subCtx, cancel := context.WithTimeout(ctx, c.backoffMaxInterval)
+		defer cancel()
+		svc := handler(subCtx, conn)
+		if svc != nil {
+			c.connection = conn
+			c.service = svc
+			if p != nil {
+				p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusRunning)
+			}
+			logger.Infow(ctx, "connected-to-endpoint", log.Fields{"endpoint": c.apiEndPoint})
+			c.events <- eventConnected
+			return nil
+		}
+	}
+	logger.Warnw(ctx, "Failed to connect to endpoint",
+		log.Fields{
+			"endpoint": c.apiEndPoint,
+			"error":    err,
+		})
+
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusFailed)
+	}
+	return fmt.Errorf("no connection to endpoint %s", c.apiEndPoint)
+}
+
+func (c *Client) closeConnection(ctx context.Context, p *probe.Probe) error {
+	if p != nil {
+		p.UpdateStatus(ctx, c.apiEndPoint, probe.ServiceStatusStopped)
+	}
+
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+
+	if c.connection != nil {
+		err := c.connection.Close()
+		c.connection = nil
+		return err
+	}
+
+	return nil
+}
+
+func (c *Client) Stop(ctx context.Context) {
+	if !c.done {
+		c.events <- eventStopped
+		close(c.events)
+		c.done = true
+	}
+}
+
+// SetService is used for testing only
+func (c *Client) SetService(srv interface{}) {
+	c.connectionLock.Lock()
+	defer c.connectionLock.Unlock()
+	c.service = srv
+}
+
+func (c *Client) SubscribeForLiveness(callback func(timestamp time.Time)) {
+	c.livenessCallback = callback
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/common.go
new file mode 100644
index 0000000..77aad4f
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 grpc
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/mock_core_service.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/mock_core_service.go
new file mode 100644
index 0000000..745753c
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/mock_core_service.go
@@ -0,0 +1,135 @@
+/*
+ * Copyright 2021-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 grpc
+
+import (
+	"context"
+	"strconv"
+	"time"
+
+	"github.com/golang/protobuf/ptypes/empty"
+	"github.com/opencord/voltha-protos/v5/go/common"
+	ic "github.com/opencord/voltha-protos/v5/go/inter_container"
+	"github.com/opencord/voltha-protos/v5/go/voltha"
+)
+
+//MockCoreServiceHandler implements the methods in the core service
+type MockCoreServiceHandler struct{}
+
+func (handler *MockCoreServiceHandler) RegisterAdapter(ctx context.Context, reg *ic.AdapterRegistration) (*empty.Empty, error) {
+	//logger.Debugw(ctx, "registration-received", log.Fields{"input": reg})
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceUpdate(context.Context, *voltha.Device) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortCreated(context.Context, *voltha.Port) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortsStateUpdate(context.Context, *ic.PortStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeleteAllPorts(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetDevicePort(context.Context, *ic.PortFilter) (*voltha.Port, error) {
+	return &voltha.Port{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ListDevicePorts(context.Context, *common.ID) (*voltha.Ports, error) {
+	return &voltha.Ports{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceStateUpdate(context.Context, *ic.DeviceStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DevicePMConfigUpdate(context.Context, *voltha.PmConfigs) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDeviceDetected(context.Context, *ic.DeviceDiscovery) (*voltha.Device, error) {
+	return &voltha.Device{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDevicesLost(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildDevicesDetected(context.Context, *common.ID) (*empty.Empty, error) {
+	time.Sleep(50 * time.Millisecond)
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetDevice(ctx context.Context, id *common.ID) (*voltha.Device, error) {
+	time.Sleep(50 * time.Millisecond)
+	vlan, _ := strconv.Atoi(id.Id)
+	return &voltha.Device{
+		Id:   id.Id,
+		Type: "test-1234",
+		Vlan: uint32(vlan),
+	}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDevice(context.Context, *ic.ChildDeviceFilter) (*voltha.Device, error) {
+	return nil, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDevices(context.Context, *common.ID) (*voltha.Devices, error) {
+	return &voltha.Devices{}, nil
+}
+
+func (handler *MockCoreServiceHandler) SendPacketIn(context.Context, *ic.PacketIn) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) DeviceReasonUpdate(context.Context, *ic.DeviceReason) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) PortStateUpdate(context.Context, *ic.PortState) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+// Additional API found in the Core - unused?
+func (handler *MockCoreServiceHandler) ReconcileChildDevices(context.Context, *common.ID) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetChildDeviceWithProxyAddress(context.Context, *voltha.Device_ProxyAddress) (*voltha.Device, error) {
+	return &voltha.Device{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetPorts(context.Context, *ic.PortFilter) (*voltha.Ports, error) {
+	return &voltha.Ports{}, nil
+}
+
+func (handler *MockCoreServiceHandler) ChildrenStateUpdate(context.Context, *ic.DeviceStateFilter) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) UpdateImageDownload(context.Context, *voltha.ImageDownload) (*empty.Empty, error) {
+	return &empty.Empty{}, nil
+}
+
+func (handler *MockCoreServiceHandler) GetHealthStatus(ctx context.Context, empty *empty.Empty) (*voltha.HealthStatus, error) {
+	return &voltha.HealthStatus{State: voltha.HealthStatus_HEALTHY}, nil
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/security.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/security.go
new file mode 100644
index 0000000..930d2c8
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/security.go
@@ -0,0 +1,22 @@
+/*
+ * 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 grpc
+
+type GrpcSecurity struct {
+	KeyFile  string
+	CertFile string
+	CaFile   string
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/server.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/server.go
new file mode 100644
index 0000000..bee418d
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/server.go
@@ -0,0 +1,180 @@
+/*
+ * 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 grpc
+
+import (
+	"context"
+	"net"
+
+	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
+	grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/reflection"
+	"google.golang.org/grpc/status"
+)
+
+/*
+To add a GRPC server to your existing component simply follow these steps:
+
+1. Create a server instance by passing the host and port where it should run and optionally add certificate information
+
+	e.g.
+	s.server = server.NewGrpcServer(s.config.GrpcHost, s.config.GrpcPort, nil, false)
+
+2. Create a function that will register your service with the GRPC server
+
+    e.g.
+	f := func(gs *grpc.Server) {
+		voltha.RegisterVolthaReadOnlyServiceServer(
+			gs,
+			core.NewReadOnlyServiceHandler(s.root),
+		)
+	}
+
+3. Add the service to the server
+
+    e.g.
+	s.server.AddService(f)
+
+4. Start the server
+
+	s.server.Start(ctx)
+*/
+
+// Interface allows probes to be attached to server
+// A probe must support the IsReady() method
+type ReadyProbe interface {
+	IsReady() bool
+}
+
+type GrpcServer struct {
+	gs       *grpc.Server
+	address  string
+	secure   bool
+	services []func(*grpc.Server)
+	probe    ReadyProbe // optional
+
+	*GrpcSecurity
+}
+
+/*
+Instantiate a GRPC server data structure
+*/
+func NewGrpcServer(
+	address string,
+	certs *GrpcSecurity,
+	secure bool,
+	probe ReadyProbe,
+) *GrpcServer {
+	server := &GrpcServer{
+		address:      address,
+		secure:       secure,
+		GrpcSecurity: certs,
+		probe:        probe,
+	}
+	return server
+}
+
+/*
+Start prepares the GRPC server and starts servicing requests
+*/
+func (s *GrpcServer) Start(ctx context.Context) {
+
+	lis, err := net.Listen("tcp", s.address)
+	if err != nil {
+		logger.Fatalf(ctx, "failed to listen: %v", err)
+	}
+
+	// Use Intercepters to automatically inject and publish Open Tracing Spans by this GRPC server
+	serverOptions := []grpc.ServerOption{
+		grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
+			grpc_opentracing.StreamServerInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+		)),
+		grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
+			grpc_opentracing.UnaryServerInterceptor(grpc_opentracing.WithTracer(log.ActiveTracerProxy{})),
+			mkServerInterceptor(s),
+		))}
+
+	if s.secure && s.GrpcSecurity != nil {
+		creds, err := credentials.NewServerTLSFromFile(s.CertFile, s.KeyFile)
+		if err != nil {
+			logger.Fatalf(ctx, "could not load TLS keys: %s", err)
+		}
+
+		serverOptions = append(serverOptions, grpc.Creds(creds))
+		s.gs = grpc.NewServer(serverOptions...)
+	} else {
+		logger.Info(ctx, "starting-insecure-grpc-server")
+		s.gs = grpc.NewServer(serverOptions...)
+	}
+
+	// Register all required services
+	for _, service := range s.services {
+		service(s.gs)
+	}
+	reflection.Register(s.gs)
+
+	if err := s.gs.Serve(lis); err != nil {
+		logger.Fatalf(ctx, "failed to serve: %v\n", err)
+	}
+}
+
+// Make a serverInterceptor for the given GrpcServer
+// This interceptor will check whether there is an attached probe,
+// and if that probe indicates NotReady, then an UNAVAILABLE
+// response will be returned.
+func mkServerInterceptor(s *GrpcServer) 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) {
+
+		if (s.probe != nil) && (!s.probe.IsReady()) {
+			logger.Warnf(ctx, "Grpc request received while not ready %v", req)
+			return nil, status.Error(codes.Unavailable, "system is not ready")
+		}
+
+		// Calls the handler
+		h, err := handler(ctx, req)
+
+		return h, err
+	}
+}
+
+/*
+Stop servicing GRPC requests
+*/
+func (s *GrpcServer) Stop() {
+	if s.gs != nil {
+		s.gs.Stop()
+	}
+}
+
+/*
+AddService appends a generic service request function
+*/
+func (s *GrpcServer) AddService(
+	registerFunction func(*grpc.Server),
+) {
+	s.services = append(s.services, registerFunction)
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/utils.go
new file mode 100644
index 0000000..85686de
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/grpc/utils.go
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2021-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 grpc
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"math"
+	"os"
+	"reflect"
+	"sync"
+	"time"
+)
+
+const (
+	incrementalFactor = 1.5
+	minBackOff        = 10 * time.Millisecond
+)
+
+type Backoff struct {
+	attempt          int
+	initialInterval  time.Duration
+	maxElapsedTime   time.Duration
+	maxInterval      time.Duration
+	totalElapsedTime time.Duration
+	mutex            sync.RWMutex
+}
+
+func NewBackoff(initialInterval, maxInterval, maxElapsedTime time.Duration) *Backoff {
+	bo := &Backoff{}
+	bo.initialInterval = initialInterval
+	bo.maxInterval = maxInterval
+	bo.maxElapsedTime = maxElapsedTime
+	return bo
+}
+
+func (bo *Backoff) Backoff(ctx context.Context) error {
+	duration, err := bo.getBackOffDuration()
+	if err != nil {
+		return err
+	}
+
+	ticker := time.NewTicker(duration)
+	defer ticker.Stop()
+	select {
+	case <-ctx.Done():
+		return ctx.Err()
+	case <-ticker.C:
+	}
+	return nil
+}
+
+func (bo *Backoff) getBackOffDuration() (duration time.Duration, err error) {
+	err = nil
+	defer func() {
+		bo.mutex.Lock()
+		defer bo.mutex.Unlock()
+		bo.attempt += 1
+		bo.totalElapsedTime += duration
+		if bo.maxElapsedTime > 0 && bo.totalElapsedTime > bo.maxElapsedTime {
+			err = errors.New("max elapsed backoff time reached")
+		}
+	}()
+
+	if bo.initialInterval <= minBackOff {
+		bo.initialInterval = minBackOff
+	}
+	if bo.initialInterval > bo.maxInterval {
+		duration = bo.initialInterval
+		return
+	}
+
+	// Calculate incremental duration
+	minf := float64(bo.initialInterval)
+	durf := minf * math.Pow(incrementalFactor, float64(bo.attempt))
+
+	if durf > math.MaxInt64 {
+		duration = bo.maxInterval
+		return
+	}
+	duration = time.Duration(durf)
+
+	//Keep within bounds
+	if duration < bo.initialInterval {
+		duration = bo.initialInterval
+	}
+	if duration > bo.maxInterval {
+		duration = bo.maxInterval
+	}
+	return
+}
+
+func (bo *Backoff) Reset() {
+	bo.mutex.Lock()
+	defer bo.mutex.Unlock()
+	bo.attempt = 0
+	bo.totalElapsedTime = 0
+}
+
+func SetFromEnvVariable(key string, variableToSet interface{}) error {
+	if _, ok := variableToSet.(*time.Duration); !ok {
+		return fmt.Errorf("unsupported type %T", variableToSet)
+	}
+	if valStr, present := os.LookupEnv(key); present {
+		val, err := time.ParseDuration(valStr)
+		if err != nil {
+			return err
+		}
+		reflect.ValueOf(variableToSet).Elem().Set(reflect.ValueOf(val))
+	}
+	return nil
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/client.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/client.go
new file mode 100644
index 0000000..fdc05bc
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/client.go
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ *
+ *
+ * NOTE:  The kafka client is used to publish events on Kafka in voltha
+ * release 2.9.  It is no longer used for inter voltha container
+ * communication.
+ */
+package kafka
+
+import (
+	"context"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+)
+
+const (
+	PartitionConsumer = iota
+	GroupCustomer     = iota
+)
+
+const (
+	OffsetNewest = -1
+	OffsetOldest = -2
+)
+
+const (
+	GroupIdKey = "groupId"
+	Offset     = "offset"
+)
+
+const (
+	DefaultKafkaAddress             = "127.0.0.1:9092"
+	DefaultGroupName                = "voltha"
+	DefaultSleepOnError             = 1
+	DefaultProducerFlushFrequency   = 10
+	DefaultProducerFlushMessages    = 10
+	DefaultProducerFlushMaxmessages = 100
+	DefaultProducerReturnSuccess    = true
+	DefaultProducerReturnErrors     = true
+	DefaultProducerRetryMax         = 3
+	DefaultProducerRetryBackoff     = time.Millisecond * 100
+	DefaultConsumerMaxwait          = 100
+	DefaultMaxProcessingTime        = 100
+	DefaultConsumerType             = PartitionConsumer
+	DefaultNumberPartitions         = 3
+	DefaultNumberReplicas           = 1
+	DefaultAutoCreateTopic          = false
+	DefaultMetadataMaxRetry         = 3
+	DefaultMaxRetries               = 3
+	DefaultLivenessChannelInterval  = time.Second * 30
+)
+
+// MsgClient represents the set of APIs  a Kafka MsgClient must implement
+type Client interface {
+	Start(ctx context.Context) error
+	Stop(ctx context.Context)
+	CreateTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error
+	DeleteTopic(ctx context.Context, topic *Topic) error
+	Subscribe(ctx context.Context, topic *Topic, kvArgs ...*KVArg) (<-chan proto.Message, error)
+	UnSubscribe(ctx context.Context, topic *Topic, ch <-chan proto.Message) error
+	SubscribeForMetadata(context.Context, func(fromTopic string, timestamp time.Time))
+	Send(ctx context.Context, msg interface{}, topic *Topic, keys ...string) error
+	SendLiveness(ctx context.Context) error
+	EnableLivenessChannel(ctx context.Context, enable bool) chan bool
+	EnableHealthinessChannel(ctx context.Context, enable bool) chan bool
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/common.go
new file mode 100644
index 0000000..c0d169a
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 kafka
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/sarama_client.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/sarama_client.go
new file mode 100644
index 0000000..185f6ec
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/sarama_client.go
@@ -0,0 +1,1154 @@
+/*
+ * 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 kafka
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/Shopify/sarama"
+	scc "github.com/bsm/sarama-cluster"
+	"github.com/eapache/go-resiliency/breaker"
+	"github.com/golang/protobuf/proto"
+	"github.com/google/uuid"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// consumerChannels represents one or more consumers listening on a kafka topic.  Once a message is received on that
+// topic, the consumer(s) broadcasts the message to all the listening channels.   The consumer can be a partition
+//consumer or a group consumer
+type consumerChannels struct {
+	consumers []interface{}
+	channels  []chan proto.Message
+}
+
+// static check to ensure SaramaClient implements Client
+var _ Client = &SaramaClient{}
+
+// SaramaClient represents the messaging proxy
+type SaramaClient struct {
+	cAdmin                        sarama.ClusterAdmin
+	KafkaAddress                  string
+	producer                      sarama.AsyncProducer
+	consumer                      sarama.Consumer
+	groupConsumers                map[string]*scc.Consumer
+	lockOfGroupConsumers          sync.RWMutex
+	consumerGroupPrefix           string
+	consumerType                  int
+	consumerGroupName             string
+	producerFlushFrequency        int
+	producerFlushMessages         int
+	producerFlushMaxmessages      int
+	producerRetryMax              int
+	producerRetryBackOff          time.Duration
+	producerReturnSuccess         bool
+	producerReturnErrors          bool
+	consumerMaxwait               int
+	maxProcessingTime             int
+	numPartitions                 int
+	numReplicas                   int
+	autoCreateTopic               bool
+	doneCh                        chan int
+	metadataCallback              func(fromTopic string, timestamp time.Time)
+	topicToConsumerChannelMap     map[string]*consumerChannels
+	lockTopicToConsumerChannelMap sync.RWMutex
+	topicLockMap                  map[string]*sync.RWMutex
+	lockOfTopicLockMap            sync.RWMutex
+	metadataMaxRetry              int
+	alive                         bool
+	livenessMutex                 sync.Mutex
+	liveness                      chan bool
+	livenessChannelInterval       time.Duration
+	lastLivenessTime              time.Time
+	started                       bool
+	healthinessMutex              sync.Mutex
+	healthy                       bool
+	healthiness                   chan bool
+}
+
+type SaramaClientOption func(*SaramaClient)
+
+func Address(address string) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.KafkaAddress = address
+	}
+}
+
+func ConsumerGroupPrefix(prefix string) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.consumerGroupPrefix = prefix
+	}
+}
+
+func ConsumerGroupName(name string) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.consumerGroupName = name
+	}
+}
+
+func ConsumerType(consumer int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.consumerType = consumer
+	}
+}
+
+func ProducerFlushFrequency(frequency int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerFlushFrequency = frequency
+	}
+}
+
+func ProducerFlushMessages(num int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerFlushMessages = num
+	}
+}
+
+func ProducerFlushMaxMessages(num int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerFlushMaxmessages = num
+	}
+}
+
+func ProducerMaxRetries(num int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerRetryMax = num
+	}
+}
+
+func ProducerRetryBackoff(duration time.Duration) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerRetryBackOff = duration
+	}
+}
+
+func ProducerReturnOnErrors(opt bool) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerReturnErrors = opt
+	}
+}
+
+func ProducerReturnOnSuccess(opt bool) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.producerReturnSuccess = opt
+	}
+}
+
+func ConsumerMaxWait(wait int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.consumerMaxwait = wait
+	}
+}
+
+func MaxProcessingTime(pTime int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.maxProcessingTime = pTime
+	}
+}
+
+func NumPartitions(number int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.numPartitions = number
+	}
+}
+
+func NumReplicas(number int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.numReplicas = number
+	}
+}
+
+func AutoCreateTopic(opt bool) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.autoCreateTopic = opt
+	}
+}
+
+func MetadatMaxRetries(retry int) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.metadataMaxRetry = retry
+	}
+}
+
+func LivenessChannelInterval(opt time.Duration) SaramaClientOption {
+	return func(args *SaramaClient) {
+		args.livenessChannelInterval = opt
+	}
+}
+
+func NewSaramaClient(opts ...SaramaClientOption) *SaramaClient {
+	client := &SaramaClient{
+		KafkaAddress: DefaultKafkaAddress,
+	}
+	client.consumerType = DefaultConsumerType
+	client.producerFlushFrequency = DefaultProducerFlushFrequency
+	client.producerFlushMessages = DefaultProducerFlushMessages
+	client.producerFlushMaxmessages = DefaultProducerFlushMaxmessages
+	client.producerReturnErrors = DefaultProducerReturnErrors
+	client.producerReturnSuccess = DefaultProducerReturnSuccess
+	client.producerRetryMax = DefaultProducerRetryMax
+	client.producerRetryBackOff = DefaultProducerRetryBackoff
+	client.consumerMaxwait = DefaultConsumerMaxwait
+	client.maxProcessingTime = DefaultMaxProcessingTime
+	client.numPartitions = DefaultNumberPartitions
+	client.numReplicas = DefaultNumberReplicas
+	client.autoCreateTopic = DefaultAutoCreateTopic
+	client.metadataMaxRetry = DefaultMetadataMaxRetry
+	client.livenessChannelInterval = DefaultLivenessChannelInterval
+
+	for _, option := range opts {
+		option(client)
+	}
+
+	client.groupConsumers = make(map[string]*scc.Consumer)
+
+	client.lockTopicToConsumerChannelMap = sync.RWMutex{}
+	client.topicLockMap = make(map[string]*sync.RWMutex)
+	client.lockOfTopicLockMap = sync.RWMutex{}
+	client.lockOfGroupConsumers = sync.RWMutex{}
+
+	// healthy and alive until proven otherwise
+	client.alive = true
+	client.healthy = true
+
+	return client
+}
+
+func (sc *SaramaClient) Start(ctx context.Context) error {
+	logger.Info(ctx, "Starting-kafka-sarama-client")
+
+	// Create the Done channel
+	sc.doneCh = make(chan int, 1)
+
+	var err error
+
+	// Add a cleanup in case of failure to startup
+	defer func() {
+		if err != nil {
+			sc.Stop(ctx)
+		}
+	}()
+
+	// Create the Cluster Admin
+	if err = sc.createClusterAdmin(ctx); err != nil {
+		logger.Errorw(ctx, "Cannot-create-cluster-admin", log.Fields{"error": err})
+		return err
+	}
+
+	// Create the Publisher
+	if err := sc.createPublisher(ctx); err != nil {
+		logger.Errorw(ctx, "Cannot-create-kafka-publisher", log.Fields{"error": err})
+		return err
+	}
+
+	if sc.consumerType == DefaultConsumerType {
+		// Create the master consumers
+		if err := sc.createConsumer(ctx); err != nil {
+			logger.Errorw(ctx, "Cannot-create-kafka-consumers", log.Fields{"error": err})
+			return err
+		}
+	}
+
+	// Create the topic to consumers/channel map
+	sc.topicToConsumerChannelMap = make(map[string]*consumerChannels)
+
+	logger.Info(ctx, "kafka-sarama-client-started")
+
+	sc.started = true
+
+	return nil
+}
+
+func (sc *SaramaClient) Stop(ctx context.Context) {
+	logger.Info(ctx, "stopping-sarama-client")
+
+	sc.started = false
+
+	//Send a message over the done channel to close all long running routines
+	sc.doneCh <- 1
+
+	if sc.producer != nil {
+		if err := sc.producer.Close(); err != nil {
+			logger.Errorw(ctx, "closing-producer-failed", log.Fields{"error": err})
+		}
+	}
+
+	if sc.consumer != nil {
+		if err := sc.consumer.Close(); err != nil {
+			logger.Errorw(ctx, "closing-partition-consumer-failed", log.Fields{"error": err})
+		}
+	}
+
+	for key, val := range sc.groupConsumers {
+		logger.Debugw(ctx, "closing-group-consumer", log.Fields{"topic": key})
+		if err := val.Close(); err != nil {
+			logger.Errorw(ctx, "closing-group-consumer-failed", log.Fields{"error": err, "topic": key})
+		}
+	}
+
+	if sc.cAdmin != nil {
+		if err := sc.cAdmin.Close(); err != nil {
+			logger.Errorw(ctx, "closing-cluster-admin-failed", log.Fields{"error": err})
+		}
+	}
+
+	//TODO: Clear the consumers map
+	//sc.clearConsumerChannelMap()
+
+	logger.Info(ctx, "sarama-client-stopped")
+}
+
+//createTopic is an internal function to create a topic on the Kafka Broker. No locking is required as
+// the invoking function must hold the lock
+func (sc *SaramaClient) createTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error {
+	// Set the topic details
+	topicDetail := &sarama.TopicDetail{}
+	topicDetail.NumPartitions = int32(numPartition)
+	topicDetail.ReplicationFactor = int16(repFactor)
+	topicDetail.ConfigEntries = make(map[string]*string)
+	topicDetails := make(map[string]*sarama.TopicDetail)
+	topicDetails[topic.Name] = topicDetail
+
+	if err := sc.cAdmin.CreateTopic(topic.Name, topicDetail, false); err != nil {
+		if err == sarama.ErrTopicAlreadyExists {
+			//	Not an error
+			logger.Debugw(ctx, "topic-already-exist", log.Fields{"topic": topic.Name})
+			return nil
+		}
+		logger.Errorw(ctx, "create-topic-failure", log.Fields{"error": err})
+		return err
+	}
+	// TODO: Wait until the topic has been created.  No API is available in the Sarama clusterAdmin to
+	// do so.
+	logger.Debugw(ctx, "topic-created", log.Fields{"topic": topic, "numPartition": numPartition, "replicationFactor": repFactor})
+	return nil
+}
+
+//CreateTopic is a public API to create a topic on the Kafka Broker.  It uses a lock on a specific topic to
+// ensure no two go routines are performing operations on the same topic
+func (sc *SaramaClient) CreateTopic(ctx context.Context, topic *Topic, numPartition int, repFactor int) error {
+	sc.lockTopic(topic)
+	defer sc.unLockTopic(topic)
+
+	return sc.createTopic(ctx, topic, numPartition, repFactor)
+}
+
+//DeleteTopic removes a topic from the kafka Broker
+func (sc *SaramaClient) DeleteTopic(ctx context.Context, topic *Topic) error {
+	sc.lockTopic(topic)
+	defer sc.unLockTopic(topic)
+
+	// Remove the topic from the broker
+	if err := sc.cAdmin.DeleteTopic(topic.Name); err != nil {
+		if err == sarama.ErrUnknownTopicOrPartition {
+			//	Not an error as does not exist
+			logger.Debugw(ctx, "topic-not-exist", log.Fields{"topic": topic.Name})
+			return nil
+		}
+		logger.Errorw(ctx, "delete-topic-failed", log.Fields{"topic": topic, "error": err})
+		return err
+	}
+
+	// Clear the topic from the consumer channel.  This will also close any consumers listening on that topic.
+	if err := sc.clearTopicFromConsumerChannelMap(ctx, *topic); err != nil {
+		logger.Errorw(ctx, "failure-clearing-channels", log.Fields{"topic": topic, "error": err})
+		return err
+	}
+	return nil
+}
+
+// Subscribe registers a caller to a topic. It returns a channel that the caller can use to receive
+// messages from that topic
+func (sc *SaramaClient) Subscribe(ctx context.Context, topic *Topic, kvArgs ...*KVArg) (<-chan proto.Message, error) {
+	sc.lockTopic(topic)
+	defer sc.unLockTopic(topic)
+
+	logger.Debugw(ctx, "subscribe", log.Fields{"topic": topic.Name})
+
+	// If a consumers already exist for that topic then resuse it
+	if consumerCh := sc.getConsumerChannel(topic); consumerCh != nil {
+		logger.Debugw(ctx, "topic-already-subscribed", log.Fields{"topic": topic.Name})
+		// Create a channel specific for that consumers and add it to the consumers channel map
+		ch := make(chan proto.Message)
+		sc.addChannelToConsumerChannelMap(ctx, topic, ch)
+		return ch, nil
+	}
+
+	// Register for the topic and set it up
+	var consumerListeningChannel chan proto.Message
+	var err error
+
+	// Use the consumerType option to figure out the type of consumer to launch
+	if sc.consumerType == PartitionConsumer {
+		if sc.autoCreateTopic {
+			if err = sc.createTopic(ctx, topic, sc.numPartitions, sc.numReplicas); err != nil {
+				logger.Errorw(ctx, "create-topic-failure", log.Fields{"error": err, "topic": topic.Name})
+				return nil, err
+			}
+		}
+		if consumerListeningChannel, err = sc.setupPartitionConsumerChannel(ctx, topic, getOffset(kvArgs...)); err != nil {
+			logger.Warnw(ctx, "create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name})
+			return nil, err
+		}
+	} else if sc.consumerType == GroupCustomer {
+		// TODO: create topic if auto create is on.  There is an issue with the sarama cluster library that
+		// does not consume from a precreated topic in some scenarios
+		//if sc.autoCreateTopic {
+		//	if err = sc.createTopic(topic, sc.numPartitions, sc.numReplicas); err != nil {
+		//		logger.Errorw(ctx, "create-topic-failure", logger.Fields{"error": err, "topic": topic.Name})
+		//		return nil, err
+		//	}
+		//}
+		//groupId := sc.consumerGroupName
+		groupId := getGroupId(kvArgs...)
+		// Include the group prefix
+		if groupId != "" {
+			groupId = sc.consumerGroupPrefix + groupId
+		} else {
+			// Need to use a unique group Id per topic
+			groupId = sc.consumerGroupPrefix + topic.Name
+		}
+		if consumerListeningChannel, err = sc.setupGroupConsumerChannel(ctx, topic, groupId, getOffset(kvArgs...)); err != nil {
+			logger.Warnw(ctx, "create-consumers-channel-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
+			return nil, err
+		}
+
+	} else {
+		logger.Warnw(ctx, "unknown-consumer-type", log.Fields{"consumer-type": sc.consumerType})
+		return nil, errors.New("unknown-consumer-type")
+	}
+
+	return consumerListeningChannel, nil
+}
+
+//UnSubscribe unsubscribe a consumer from a given topic
+func (sc *SaramaClient) UnSubscribe(ctx context.Context, topic *Topic, ch <-chan proto.Message) error {
+	sc.lockTopic(topic)
+	defer sc.unLockTopic(topic)
+
+	logger.Debugw(ctx, "unsubscribing-channel-from-topic", log.Fields{"topic": topic.Name})
+	var err error
+	if err = sc.removeChannelFromConsumerChannelMap(ctx, *topic, ch); err != nil {
+		logger.Errorw(ctx, "failed-removing-channel", log.Fields{"error": err})
+	}
+	if err = sc.deleteFromGroupConsumers(ctx, topic.Name); err != nil {
+		logger.Errorw(ctx, "failed-deleting-group-consumer", log.Fields{"error": err})
+	}
+	return err
+}
+
+func (sc *SaramaClient) SubscribeForMetadata(ctx context.Context, callback func(fromTopic string, timestamp time.Time)) {
+	sc.metadataCallback = callback
+}
+
+func (sc *SaramaClient) updateLiveness(ctx context.Context, alive bool) {
+	// Post a consistent stream of liveness data to the channel,
+	// so that in a live state, the core does not timeout and
+	// send a forced liveness message. Production of liveness
+	// events to the channel is rate-limited by livenessChannelInterval.
+	sc.livenessMutex.Lock()
+	defer sc.livenessMutex.Unlock()
+	if sc.liveness != nil {
+		if sc.alive != alive {
+			logger.Info(ctx, "update-liveness-channel-because-change")
+			sc.liveness <- alive
+			sc.lastLivenessTime = time.Now()
+		} else if time.Since(sc.lastLivenessTime) > sc.livenessChannelInterval {
+			logger.Info(ctx, "update-liveness-channel-because-interval")
+			sc.liveness <- alive
+			sc.lastLivenessTime = time.Now()
+		}
+	}
+
+	// Only emit a log message when the state changes
+	if sc.alive != alive {
+		logger.Info(ctx, "set-client-alive", log.Fields{"alive": alive})
+		sc.alive = alive
+	}
+}
+
+// Once unhealthy, we never go back
+func (sc *SaramaClient) setUnhealthy(ctx context.Context) {
+	sc.healthy = false
+	sc.healthinessMutex.Lock()
+	defer sc.healthinessMutex.Unlock()
+	if sc.healthiness != nil {
+		logger.Infow(ctx, "set-client-unhealthy", log.Fields{"healthy": sc.healthy})
+		sc.healthiness <- sc.healthy
+	}
+}
+
+func (sc *SaramaClient) isLivenessError(ctx context.Context, err error) bool {
+	// Sarama producers and consumers encapsulate the error inside
+	// a ProducerError or ConsumerError struct.
+	if prodError, ok := err.(*sarama.ProducerError); ok {
+		err = prodError.Err
+	} else if consumerError, ok := err.(*sarama.ConsumerError); ok {
+		err = consumerError.Err
+	}
+
+	// Sarama-Cluster will compose the error into a ClusterError struct,
+	// which we can't do a compare by reference. To handle that, we the
+	// best we can do is compare the error strings.
+
+	switch err.Error() {
+	case context.DeadlineExceeded.Error():
+		logger.Info(ctx, "is-liveness-error-timeout")
+		return true
+	case sarama.ErrOutOfBrokers.Error(): // "Kafka: client has run out of available brokers"
+		logger.Info(ctx, "is-liveness-error-no-brokers")
+		return true
+	case sarama.ErrShuttingDown.Error(): // "Kafka: message received by producer in process of shutting down"
+		logger.Info(ctx, "is-liveness-error-shutting-down")
+		return true
+	case sarama.ErrControllerNotAvailable.Error(): // "Kafka: controller is not available"
+		logger.Info(ctx, "is-liveness-error-not-available")
+		return true
+	case breaker.ErrBreakerOpen.Error(): // "circuit breaker is open"
+		logger.Info(ctx, "is-liveness-error-circuit-breaker-open")
+		return true
+	}
+
+	if strings.HasSuffix(err.Error(), "connection refused") { // "dial tcp 10.244.1.176:9092: connect: connection refused"
+		logger.Info(ctx, "is-liveness-error-connection-refused")
+		return true
+	}
+
+	if strings.HasSuffix(err.Error(), "i/o timeout") { // "dial tcp 10.244.1.176:9092: i/o timeout"
+		logger.Info(ctx, "is-liveness-error-io-timeout")
+		return true
+	}
+
+	// Other errors shouldn't trigger a loss of liveness
+
+	logger.Infow(ctx, "is-liveness-error-ignored", log.Fields{"err": err})
+
+	return false
+}
+
+// send formats and sends the request onto the kafka messaging bus.
+func (sc *SaramaClient) Send(ctx context.Context, msg interface{}, topic *Topic, keys ...string) error {
+
+	// Assert message is a proto message
+	var protoMsg proto.Message
+	var ok bool
+	// ascertain the value interface type is a proto.Message
+	if protoMsg, ok = msg.(proto.Message); !ok {
+		logger.Warnw(ctx, "message-not-proto-message", log.Fields{"msg": msg})
+		return fmt.Errorf("not-a-proto-msg-%s", msg)
+	}
+
+	var marshalled []byte
+	var err error
+	//	Create the Sarama producer message
+	if marshalled, err = proto.Marshal(protoMsg); err != nil {
+		logger.Errorw(ctx, "marshalling-failed", log.Fields{"msg": protoMsg, "error": err})
+		return err
+	}
+	key := ""
+	if len(keys) > 0 {
+		key = keys[0] // Only the first key is relevant
+	}
+	kafkaMsg := &sarama.ProducerMessage{
+		Topic: topic.Name,
+		Key:   sarama.StringEncoder(key),
+		Value: sarama.ByteEncoder(marshalled),
+	}
+
+	// Send message to kafka
+	sc.producer.Input() <- kafkaMsg
+	// Wait for result
+	// TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
+	select {
+	case ok := <-sc.producer.Successes():
+		logger.Debugw(ctx, "message-sent", log.Fields{"status": ok.Topic})
+		sc.updateLiveness(ctx, true)
+	case notOk := <-sc.producer.Errors():
+		logger.Debugw(ctx, "error-sending", log.Fields{"status": notOk})
+		if sc.isLivenessError(ctx, notOk) {
+			sc.updateLiveness(ctx, false)
+		}
+		return notOk
+	}
+	return nil
+}
+
+// Enable the liveness monitor channel. This channel will report
+// a "true" or "false" on every publish, which indicates whether
+// or not the channel is still live. This channel is then picked up
+// by the service (i.e. rw_core / ro_core) to update readiness status
+// and/or take other actions.
+func (sc *SaramaClient) EnableLivenessChannel(ctx context.Context, enable bool) chan bool {
+	logger.Infow(ctx, "kafka-enable-liveness-channel", log.Fields{"enable": enable})
+	if enable {
+		sc.livenessMutex.Lock()
+		defer sc.livenessMutex.Unlock()
+		if sc.liveness == nil {
+			logger.Info(ctx, "kafka-create-liveness-channel")
+			// At least 1, so we can immediately post to it without blocking
+			// Setting a bigger number (10) allows the monitor to fall behind
+			// without blocking others. The monitor shouldn't really fall
+			// behind...
+			sc.liveness = make(chan bool, 10)
+			// post initial state to the channel
+			sc.liveness <- sc.alive
+		}
+	} else {
+		// TODO: Think about whether we need the ability to turn off
+		// liveness monitoring
+		panic("Turning off liveness reporting is not supported")
+	}
+	return sc.liveness
+}
+
+// Enable the Healthiness monitor channel. This channel will report "false"
+// if the kafka consumers die, or some other problem occurs which is
+// catastrophic that would require re-creating the client.
+func (sc *SaramaClient) EnableHealthinessChannel(ctx context.Context, enable bool) chan bool {
+	logger.Infow(ctx, "kafka-enable-healthiness-channel", log.Fields{"enable": enable})
+	if enable {
+		sc.healthinessMutex.Lock()
+		defer sc.healthinessMutex.Unlock()
+		if sc.healthiness == nil {
+			logger.Info(ctx, "kafka-create-healthiness-channel")
+			// At least 1, so we can immediately post to it without blocking
+			// Setting a bigger number (10) allows the monitor to fall behind
+			// without blocking others. The monitor shouldn't really fall
+			// behind...
+			sc.healthiness = make(chan bool, 10)
+			// post initial state to the channel
+			sc.healthiness <- sc.healthy
+		}
+	} else {
+		// TODO: Think about whether we need the ability to turn off
+		// liveness monitoring
+		panic("Turning off healthiness reporting is not supported")
+	}
+	return sc.healthiness
+}
+
+// send an empty message on the liveness channel to check whether connectivity has
+// been restored.
+func (sc *SaramaClient) SendLiveness(ctx context.Context) error {
+	if !sc.started {
+		return fmt.Errorf("SendLiveness() called while not started")
+	}
+
+	kafkaMsg := &sarama.ProducerMessage{
+		Topic: "_liveness_test",
+		Value: sarama.StringEncoder(time.Now().Format(time.RFC3339)), // for debugging / informative use
+	}
+
+	// Send message to kafka
+	sc.producer.Input() <- kafkaMsg
+	// Wait for result
+	// TODO: Use a lock or a different mechanism to ensure the response received corresponds to the message sent.
+	select {
+	case ok := <-sc.producer.Successes():
+		logger.Debugw(ctx, "liveness-message-sent", log.Fields{"status": ok.Topic})
+		sc.updateLiveness(ctx, true)
+	case notOk := <-sc.producer.Errors():
+		logger.Debugw(ctx, "liveness-error-sending", log.Fields{"status": notOk})
+		if sc.isLivenessError(ctx, notOk) {
+			sc.updateLiveness(ctx, false)
+		}
+		return notOk
+	}
+	return nil
+}
+
+// getGroupId returns the group id from the key-value args.
+func getGroupId(kvArgs ...*KVArg) string {
+	for _, arg := range kvArgs {
+		if arg.Key == GroupIdKey {
+			return arg.Value.(string)
+		}
+	}
+	return ""
+}
+
+// getOffset returns the offset from the key-value args.
+func getOffset(kvArgs ...*KVArg) int64 {
+	for _, arg := range kvArgs {
+		if arg.Key == Offset {
+			return arg.Value.(int64)
+		}
+	}
+	return sarama.OffsetNewest
+}
+
+func (sc *SaramaClient) createClusterAdmin(ctx context.Context) error {
+	config := sarama.NewConfig()
+	config.Version = sarama.V1_0_0_0
+
+	// Create a cluster Admin
+	var cAdmin sarama.ClusterAdmin
+	var err error
+	if cAdmin, err = sarama.NewClusterAdmin([]string{sc.KafkaAddress}, config); err != nil {
+		logger.Errorw(ctx, "cluster-admin-failure", log.Fields{"error": err, "broker-address": sc.KafkaAddress})
+		return err
+	}
+	sc.cAdmin = cAdmin
+	return nil
+}
+
+func (sc *SaramaClient) lockTopic(topic *Topic) {
+	sc.lockOfTopicLockMap.Lock()
+	if _, exist := sc.topicLockMap[topic.Name]; exist {
+		sc.lockOfTopicLockMap.Unlock()
+		sc.topicLockMap[topic.Name].Lock()
+	} else {
+		sc.topicLockMap[topic.Name] = &sync.RWMutex{}
+		sc.lockOfTopicLockMap.Unlock()
+		sc.topicLockMap[topic.Name].Lock()
+	}
+}
+
+func (sc *SaramaClient) unLockTopic(topic *Topic) {
+	sc.lockOfTopicLockMap.Lock()
+	defer sc.lockOfTopicLockMap.Unlock()
+	if _, exist := sc.topicLockMap[topic.Name]; exist {
+		sc.topicLockMap[topic.Name].Unlock()
+	}
+}
+
+func (sc *SaramaClient) addTopicToConsumerChannelMap(id string, arg *consumerChannels) {
+	sc.lockTopicToConsumerChannelMap.Lock()
+	defer sc.lockTopicToConsumerChannelMap.Unlock()
+	if _, exist := sc.topicToConsumerChannelMap[id]; !exist {
+		sc.topicToConsumerChannelMap[id] = arg
+	}
+}
+
+func (sc *SaramaClient) getConsumerChannel(topic *Topic) *consumerChannels {
+	sc.lockTopicToConsumerChannelMap.RLock()
+	defer sc.lockTopicToConsumerChannelMap.RUnlock()
+
+	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
+		return consumerCh
+	}
+	return nil
+}
+
+func (sc *SaramaClient) addChannelToConsumerChannelMap(ctx context.Context, topic *Topic, ch chan proto.Message) {
+	sc.lockTopicToConsumerChannelMap.Lock()
+	defer sc.lockTopicToConsumerChannelMap.Unlock()
+	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
+		consumerCh.channels = append(consumerCh.channels, ch)
+		return
+	}
+	logger.Warnw(ctx, "consumers-channel-not-exist", log.Fields{"topic": topic.Name})
+}
+
+//closeConsumers closes a list of sarama consumers.  The consumers can either be a partition consumers or a group consumers
+func closeConsumers(ctx context.Context, consumers []interface{}) error {
+	var err error
+	for _, consumer := range consumers {
+		//	Is it a partition consumers?
+		if partionConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
+			if errTemp := partionConsumer.Close(); errTemp != nil {
+				logger.Debugw(ctx, "partition!!!", log.Fields{"err": errTemp})
+				if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
+					// This can occur on race condition
+					err = nil
+				} else {
+					err = errTemp
+				}
+			}
+		} else if groupConsumer, ok := consumer.(*scc.Consumer); ok {
+			if errTemp := groupConsumer.Close(); errTemp != nil {
+				if strings.Compare(errTemp.Error(), sarama.ErrUnknownTopicOrPartition.Error()) == 0 {
+					// This can occur on race condition
+					err = nil
+				} else {
+					err = errTemp
+				}
+			}
+		}
+	}
+	return err
+}
+
+func (sc *SaramaClient) removeChannelFromConsumerChannelMap(ctx context.Context, topic Topic, ch <-chan proto.Message) error {
+	sc.lockTopicToConsumerChannelMap.Lock()
+	defer sc.lockTopicToConsumerChannelMap.Unlock()
+	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
+		// Channel will be closed in the removeChannel method
+		consumerCh.channels = removeChannel(ctx, consumerCh.channels, ch)
+		// If there are no more channels then we can close the consumers itself
+		if len(consumerCh.channels) == 0 {
+			logger.Debugw(ctx, "closing-consumers", log.Fields{"topic": topic})
+			err := closeConsumers(ctx, consumerCh.consumers)
+			//err := consumerCh.consumers.Close()
+			delete(sc.topicToConsumerChannelMap, topic.Name)
+			return err
+		}
+		return nil
+	}
+	logger.Warnw(ctx, "topic-does-not-exist", log.Fields{"topic": topic.Name})
+	return errors.New("topic-does-not-exist")
+}
+
+func (sc *SaramaClient) clearTopicFromConsumerChannelMap(ctx context.Context, topic Topic) error {
+	sc.lockTopicToConsumerChannelMap.Lock()
+	defer sc.lockTopicToConsumerChannelMap.Unlock()
+	if consumerCh, exist := sc.topicToConsumerChannelMap[topic.Name]; exist {
+		for _, ch := range consumerCh.channels {
+			// Channel will be closed in the removeChannel method
+			removeChannel(ctx, consumerCh.channels, ch)
+		}
+		err := closeConsumers(ctx, consumerCh.consumers)
+		//if err == sarama.ErrUnknownTopicOrPartition {
+		//	// Not an error
+		//	err = nil
+		//}
+		//err := consumerCh.consumers.Close()
+		delete(sc.topicToConsumerChannelMap, topic.Name)
+		return err
+	}
+	logger.Debugw(ctx, "topic-does-not-exist", log.Fields{"topic": topic.Name})
+	return nil
+}
+
+//createPublisher creates the publisher which is used to send a message onto kafka
+func (sc *SaramaClient) createPublisher(ctx context.Context) error {
+	// This Creates the publisher
+	config := sarama.NewConfig()
+	config.Version = sarama.V1_0_0_0
+	config.Producer.Partitioner = sarama.NewRandomPartitioner
+	config.Producer.Flush.Frequency = time.Duration(sc.producerFlushFrequency)
+	config.Producer.Flush.Messages = sc.producerFlushMessages
+	config.Producer.Flush.MaxMessages = sc.producerFlushMaxmessages
+	config.Producer.Return.Errors = sc.producerReturnErrors
+	config.Producer.Return.Successes = sc.producerReturnSuccess
+	//config.Producer.RequiredAcks = sarama.WaitForAll
+	config.Producer.RequiredAcks = sarama.WaitForLocal
+
+	brokers := []string{sc.KafkaAddress}
+
+	if producer, err := sarama.NewAsyncProducer(brokers, config); err != nil {
+		logger.Errorw(ctx, "error-starting-publisher", log.Fields{"error": err})
+		return err
+	} else {
+		sc.producer = producer
+	}
+	logger.Info(ctx, "Kafka-publisher-created")
+	return nil
+}
+
+func (sc *SaramaClient) createConsumer(ctx context.Context) error {
+	config := sarama.NewConfig()
+	config.Version = sarama.V1_0_0_0
+	config.Consumer.Return.Errors = true
+	config.Consumer.Fetch.Min = 1
+	config.Consumer.MaxWaitTime = time.Duration(sc.consumerMaxwait) * time.Millisecond
+	config.Consumer.MaxProcessingTime = time.Duration(sc.maxProcessingTime) * time.Millisecond
+	config.Consumer.Offsets.Initial = sarama.OffsetNewest
+	config.Metadata.Retry.Max = sc.metadataMaxRetry
+	brokers := []string{sc.KafkaAddress}
+
+	if consumer, err := sarama.NewConsumer(brokers, config); err != nil {
+		logger.Errorw(ctx, "error-starting-consumers", log.Fields{"error": err})
+		return err
+	} else {
+		sc.consumer = consumer
+	}
+	logger.Info(ctx, "Kafka-consumers-created")
+	return nil
+}
+
+// createGroupConsumer creates a consumers group
+func (sc *SaramaClient) createGroupConsumer(ctx context.Context, topic *Topic, groupId string, initialOffset int64, retries int) (*scc.Consumer, error) {
+	config := scc.NewConfig()
+	config.Version = sarama.V1_0_0_0
+	config.ClientID = uuid.New().String()
+	config.Group.Mode = scc.ConsumerModeMultiplex
+	config.Consumer.Group.Heartbeat.Interval, _ = time.ParseDuration("1s")
+	config.Consumer.Return.Errors = true
+	//config.Group.Return.Notifications = false
+	//config.Consumer.MaxWaitTime = time.Duration(DefaultConsumerMaxwait) * time.Millisecond
+	//config.Consumer.MaxProcessingTime = time.Duration(DefaultMaxProcessingTime) * time.Millisecond
+	config.Consumer.Offsets.Initial = initialOffset
+	//config.Consumer.Offsets.Initial = sarama.OffsetOldest
+	brokers := []string{sc.KafkaAddress}
+
+	topics := []string{topic.Name}
+	var consumer *scc.Consumer
+	var err error
+
+	if consumer, err = scc.NewConsumer(brokers, groupId, topics, config); err != nil {
+		logger.Errorw(ctx, "create-group-consumers-failure", log.Fields{"error": err, "topic": topic.Name, "groupId": groupId})
+		return nil, err
+	}
+	logger.Debugw(ctx, "create-group-consumers-success", log.Fields{"topic": topic.Name, "groupId": groupId})
+
+	//sc.groupConsumers[topic.Name] = consumer
+	sc.addToGroupConsumers(topic.Name, consumer)
+	return consumer, nil
+}
+
+// dispatchToConsumers sends the intercontainermessage received on a given topic to all subscribers for that
+// topic via the unique channel each subscriber received during subscription
+func (sc *SaramaClient) dispatchToConsumers(consumerCh *consumerChannels, protoMessage proto.Message, fromTopic string, ts time.Time) {
+	// Need to go over all channels and publish messages to them - do we need to copy msg?
+	sc.lockTopicToConsumerChannelMap.RLock()
+	for _, ch := range consumerCh.channels {
+		go func(c chan proto.Message) {
+			c <- protoMessage
+		}(ch)
+	}
+	sc.lockTopicToConsumerChannelMap.RUnlock()
+
+	if callback := sc.metadataCallback; callback != nil {
+		callback(fromTopic, ts)
+	}
+}
+
+func (sc *SaramaClient) consumeFromAPartition(ctx context.Context, topic *Topic, consumer sarama.PartitionConsumer, consumerChnls *consumerChannels) {
+	logger.Debugw(ctx, "starting-partition-consumption-loop", log.Fields{"topic": topic.Name})
+startloop:
+	for {
+		select {
+		case err, ok := <-consumer.Errors():
+			if ok {
+				if sc.isLivenessError(ctx, err) {
+					sc.updateLiveness(ctx, false)
+					logger.Warnw(ctx, "partition-consumers-error", log.Fields{"error": err})
+				}
+			} else {
+				// Channel is closed
+				break startloop
+			}
+		case msg, ok := <-consumer.Messages():
+			//logger.Debugw(ctx, "message-received", logger.Fields{"msg": msg, "receivedTopic": msg.Topic})
+			if !ok {
+				// channel is closed
+				break startloop
+			}
+			msgBody := msg.Value
+			sc.updateLiveness(ctx, true)
+			logger.Debugw(ctx, "message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
+			var protoMsg proto.Message
+			if err := proto.Unmarshal(msgBody, protoMsg); err != nil {
+				logger.Warnw(ctx, "partition-invalid-message", log.Fields{"error": err})
+				continue
+			}
+			go sc.dispatchToConsumers(consumerChnls, protoMsg, msg.Topic, msg.Timestamp)
+		case <-sc.doneCh:
+			logger.Infow(ctx, "partition-received-exit-signal", log.Fields{"topic": topic.Name})
+			break startloop
+		}
+	}
+	logger.Infow(ctx, "partition-consumer-stopped", log.Fields{"topic": topic.Name})
+	sc.setUnhealthy(ctx)
+}
+
+func (sc *SaramaClient) consumeGroupMessages(ctx context.Context, topic *Topic, consumer *scc.Consumer, consumerChnls *consumerChannels) {
+	logger.Debugw(ctx, "starting-group-consumption-loop", log.Fields{"topic": topic.Name})
+
+startloop:
+	for {
+		select {
+		case err, ok := <-consumer.Errors():
+			if ok {
+				if sc.isLivenessError(ctx, err) {
+					sc.updateLiveness(ctx, false)
+				}
+				logger.Warnw(ctx, "group-consumers-error", log.Fields{"topic": topic.Name, "error": err})
+			} else {
+				logger.Warnw(ctx, "group-consumers-closed-err", log.Fields{"topic": topic.Name})
+				// channel is closed
+				break startloop
+			}
+		case msg, ok := <-consumer.Messages():
+			if !ok {
+				logger.Warnw(ctx, "group-consumers-closed-msg", log.Fields{"topic": topic.Name})
+				// Channel closed
+				break startloop
+			}
+			sc.updateLiveness(ctx, true)
+			logger.Debugw(ctx, "message-received", log.Fields{"timestamp": msg.Timestamp, "receivedTopic": msg.Topic})
+			msgBody := msg.Value
+			var protoMsg proto.Message
+			if err := proto.Unmarshal(msgBody, protoMsg); err != nil {
+				logger.Warnw(ctx, "invalid-message", log.Fields{"error": err})
+				continue
+			}
+			go sc.dispatchToConsumers(consumerChnls, protoMsg, msg.Topic, msg.Timestamp)
+			consumer.MarkOffset(msg, "")
+		case ntf := <-consumer.Notifications():
+			logger.Debugw(ctx, "group-received-notification", log.Fields{"notification": ntf})
+		case <-sc.doneCh:
+			logger.Infow(ctx, "group-received-exit-signal", log.Fields{"topic": topic.Name})
+			break startloop
+		}
+	}
+	logger.Infow(ctx, "group-consumer-stopped", log.Fields{"topic": topic.Name})
+	sc.setUnhealthy(ctx)
+}
+
+func (sc *SaramaClient) startConsumers(ctx context.Context, topic *Topic) error {
+	logger.Debugw(ctx, "starting-consumers", log.Fields{"topic": topic.Name})
+	var consumerCh *consumerChannels
+	if consumerCh = sc.getConsumerChannel(topic); consumerCh == nil {
+		logger.Errorw(ctx, "consumers-not-exist", log.Fields{"topic": topic.Name})
+		return errors.New("consumers-not-exist")
+	}
+	// For each consumer listening for that topic, start a consumption loop
+	for _, consumer := range consumerCh.consumers {
+		if pConsumer, ok := consumer.(sarama.PartitionConsumer); ok {
+			go sc.consumeFromAPartition(ctx, topic, pConsumer, consumerCh)
+		} else if gConsumer, ok := consumer.(*scc.Consumer); ok {
+			go sc.consumeGroupMessages(ctx, topic, gConsumer, consumerCh)
+		} else {
+			logger.Errorw(ctx, "invalid-consumer", log.Fields{"topic": topic})
+			return errors.New("invalid-consumer")
+		}
+	}
+	return nil
+}
+
+//// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
+//// for that topic.  It also starts the routine that listens for messages on that topic.
+func (sc *SaramaClient) setupPartitionConsumerChannel(ctx context.Context, topic *Topic, initialOffset int64) (chan proto.Message, error) {
+	var pConsumers []sarama.PartitionConsumer
+	var err error
+
+	if pConsumers, err = sc.createPartitionConsumers(ctx, topic, initialOffset); err != nil {
+		logger.Errorw(ctx, "creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
+		return nil, err
+	}
+
+	consumersIf := make([]interface{}, 0)
+	for _, pConsumer := range pConsumers {
+		consumersIf = append(consumersIf, pConsumer)
+	}
+
+	// Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
+	// unbuffered to verify race conditions.
+	consumerListeningChannel := make(chan proto.Message)
+	cc := &consumerChannels{
+		consumers: consumersIf,
+		channels:  []chan proto.Message{consumerListeningChannel},
+	}
+
+	// Add the consumers channel to the map
+	sc.addTopicToConsumerChannelMap(topic.Name, cc)
+
+	//Start a consumers to listen on that specific topic
+	go func() {
+		if err := sc.startConsumers(ctx, topic); err != nil {
+			logger.Errorw(ctx, "start-consumers-failed", log.Fields{
+				"topic": topic,
+				"error": err})
+		}
+	}()
+
+	return consumerListeningChannel, nil
+}
+
+// setupConsumerChannel creates a consumerChannels object for that topic and add it to the consumerChannels map
+// for that topic.  It also starts the routine that listens for messages on that topic.
+func (sc *SaramaClient) setupGroupConsumerChannel(ctx context.Context, topic *Topic, groupId string, initialOffset int64) (chan proto.Message, error) {
+	// TODO:  Replace this development partition consumers with a group consumers
+	var pConsumer *scc.Consumer
+	var err error
+	if pConsumer, err = sc.createGroupConsumer(ctx, topic, groupId, initialOffset, DefaultMaxRetries); err != nil {
+		logger.Errorw(ctx, "creating-partition-consumers-failure", log.Fields{"error": err, "topic": topic.Name})
+		return nil, err
+	}
+	// Create the consumers/channel structure and set the consumers and create a channel on that topic - for now
+	// unbuffered to verify race conditions.
+	consumerListeningChannel := make(chan proto.Message)
+	cc := &consumerChannels{
+		consumers: []interface{}{pConsumer},
+		channels:  []chan proto.Message{consumerListeningChannel},
+	}
+
+	// Add the consumers channel to the map
+	sc.addTopicToConsumerChannelMap(topic.Name, cc)
+
+	//Start a consumers to listen on that specific topic
+	go func() {
+		if err := sc.startConsumers(ctx, topic); err != nil {
+			logger.Errorw(ctx, "start-consumers-failed", log.Fields{
+				"topic": topic,
+				"error": err})
+		}
+	}()
+
+	return consumerListeningChannel, nil
+}
+
+func (sc *SaramaClient) createPartitionConsumers(ctx context.Context, topic *Topic, initialOffset int64) ([]sarama.PartitionConsumer, error) {
+	logger.Debugw(ctx, "creating-partition-consumers", log.Fields{"topic": topic.Name})
+	partitionList, err := sc.consumer.Partitions(topic.Name)
+	if err != nil {
+		logger.Warnw(ctx, "get-partition-failure", log.Fields{"error": err, "topic": topic.Name})
+		return nil, err
+	}
+
+	pConsumers := make([]sarama.PartitionConsumer, 0)
+	for _, partition := range partitionList {
+		var pConsumer sarama.PartitionConsumer
+		if pConsumer, err = sc.consumer.ConsumePartition(topic.Name, partition, initialOffset); err != nil {
+			logger.Warnw(ctx, "consumers-partition-failure", log.Fields{"error": err, "topic": topic.Name})
+			return nil, err
+		}
+		pConsumers = append(pConsumers, pConsumer)
+	}
+	return pConsumers, nil
+}
+
+func removeChannel(ctx context.Context, channels []chan proto.Message, ch <-chan proto.Message) []chan proto.Message {
+	var i int
+	var channel chan proto.Message
+	for i, channel = range channels {
+		if channel == ch {
+			channels[len(channels)-1], channels[i] = channels[i], channels[len(channels)-1]
+			close(channel)
+			logger.Debug(ctx, "channel-closed")
+			return channels[:len(channels)-1]
+		}
+	}
+	return channels
+}
+
+func (sc *SaramaClient) addToGroupConsumers(topic string, consumer *scc.Consumer) {
+	sc.lockOfGroupConsumers.Lock()
+	defer sc.lockOfGroupConsumers.Unlock()
+	if _, exist := sc.groupConsumers[topic]; !exist {
+		sc.groupConsumers[topic] = consumer
+	}
+}
+
+func (sc *SaramaClient) deleteFromGroupConsumers(ctx context.Context, topic string) error {
+	sc.lockOfGroupConsumers.Lock()
+	defer sc.lockOfGroupConsumers.Unlock()
+	if _, exist := sc.groupConsumers[topic]; exist {
+		consumer := sc.groupConsumers[topic]
+		delete(sc.groupConsumers, topic)
+		if err := consumer.Close(); err != nil {
+			logger.Errorw(ctx, "failure-closing-consumer", log.Fields{"error": err})
+			return err
+		}
+	}
+	return nil
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/utils.go
new file mode 100644
index 0000000..608361b
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/kafka/utils.go
@@ -0,0 +1,185 @@
+/*
+ * 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 kafka
+
+import (
+	"context"
+	"errors"
+	"strings"
+	"time"
+
+	"github.com/golang/protobuf/ptypes/any"
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	"github.com/opencord/voltha-lib-go/v7/pkg/probe"
+)
+
+const (
+	TopicSeparator = "_"
+	DeviceIdLength = 24
+)
+
+// A Topic definition - may be augmented with additional attributes eventually
+type Topic struct {
+	// The name of the topic. It must start with a letter,
+	// and contain only letters (`[A-Za-z]`), numbers (`[0-9]`), dashes (`-`),
+	// underscores (`_`), periods (`.`), tildes (`~`), plus (`+`) or percent
+	// signs (`%`).
+	Name string
+}
+
+type KVArg struct {
+	Key   string
+	Value interface{}
+}
+
+type RpcMType int
+
+const (
+	RpcFormattingError RpcMType = iota
+	RpcSent
+	RpcReply
+	RpcTimeout
+	RpcTransportError
+	RpcSystemClosing
+)
+
+type RpcResponse struct {
+	MType RpcMType
+	Err   error
+	Reply *any.Any
+}
+
+func NewResponse(messageType RpcMType, err error, body *any.Any) *RpcResponse {
+	return &RpcResponse{
+		MType: messageType,
+		Err:   err,
+		Reply: body,
+	}
+}
+
+// TODO:  Remove and provide better may to get the device id
+// GetDeviceIdFromTopic extract the deviceId from the topic name.  The topic name is formatted either as:
+//			<any string> or <any string>_<deviceId>.  The device Id is 24 characters long.
+func GetDeviceIdFromTopic(topic Topic) string {
+	pos := strings.LastIndex(topic.Name, TopicSeparator)
+	if pos == -1 {
+		return ""
+	}
+	adjustedPos := pos + len(TopicSeparator)
+	if adjustedPos >= len(topic.Name) {
+		return ""
+	}
+	deviceId := topic.Name[adjustedPos:len(topic.Name)]
+	if len(deviceId) != DeviceIdLength {
+		return ""
+	}
+	return deviceId
+}
+
+// WaitUntilKafkaConnectionIsUp waits until the kafka client can establish a connection to the kafka broker or until the
+// context times out.
+func StartAndWaitUntilKafkaConnectionIsUp(ctx context.Context, kClient Client, connectionRetryInterval time.Duration, serviceName string) error {
+	if kClient == nil {
+		return errors.New("kafka-client-is-nil")
+	}
+	for {
+		if err := kClient.Start(ctx); err != nil {
+			probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
+			logger.Warnw(ctx, "kafka-connection-down", log.Fields{"error": err})
+			select {
+			case <-time.After(connectionRetryInterval):
+				continue
+			case <-ctx.Done():
+				return ctx.Err()
+			}
+		}
+		probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+		logger.Info(ctx, "kafka-connection-up")
+		break
+	}
+	return nil
+}
+
+/**
+MonitorKafkaReadiness checks the liveliness and readiness of the kafka service
+and update the status in the probe.
+*/
+func MonitorKafkaReadiness(ctx context.Context,
+	kClient Client,
+	liveProbeInterval, notLiveProbeInterval time.Duration,
+	serviceName string) {
+
+	if kClient == nil {
+		logger.Fatal(ctx, "kafka-client-is-nil")
+	}
+
+	logger.Infow(ctx, "monitor-kafka-readiness", log.Fields{"service": serviceName})
+
+	livelinessChannel := kClient.EnableLivenessChannel(ctx, true)
+	healthinessChannel := kClient.EnableHealthinessChannel(ctx, true)
+	timeout := liveProbeInterval
+	failed := false
+	for {
+		timeoutTimer := time.NewTimer(timeout)
+
+		select {
+		case healthiness := <-healthinessChannel:
+			if !healthiness {
+				// This will eventually cause K8s to restart the container, and will do
+				// so in a way that allows cleanup to continue, rather than an immediate
+				// panic and exit here.
+				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusFailed)
+				logger.Infow(ctx, "kafka-not-healthy", log.Fields{"service": serviceName})
+				failed = true
+			}
+			// Check if the timer has expired or not
+			if !timeoutTimer.Stop() {
+				<-timeoutTimer.C
+			}
+		case liveliness := <-livelinessChannel:
+			if failed {
+				// Failures of the message bus are permanent and can't ever be recovered from,
+				// so make sure we never inadvertently reset a failed state back to unready.
+			} else if !liveliness {
+				// kafka not reachable or down, updating the status to not ready state
+				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusNotReady)
+				logger.Infow(ctx, "kafka-not-live", log.Fields{"service": serviceName})
+				timeout = notLiveProbeInterval
+			} else {
+				// kafka is reachable , updating the status to running state
+				probe.UpdateStatusFromContext(ctx, serviceName, probe.ServiceStatusRunning)
+				timeout = liveProbeInterval
+			}
+			// Check if the timer has expired or not
+			if !timeoutTimer.Stop() {
+				<-timeoutTimer.C
+			}
+		case <-timeoutTimer.C:
+			logger.Infow(ctx, "kafka-proxy-liveness-recheck", log.Fields{"service": serviceName})
+			// send the liveness probe in a goroutine; we don't want to deadlock ourselves as
+			// the liveness probe may wait (and block) writing to our channel.
+			go func() {
+				err := kClient.SendLiveness(ctx)
+				if err != nil {
+					// Catch possible error case if sending liveness after Sarama has been stopped.
+					logger.Warnw(ctx, "error-kafka-send-liveness", log.Fields{"error": err, "service": serviceName})
+				}
+			}()
+		case <-ctx.Done():
+			return // just exit
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/common.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/common.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/common.go
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/log.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/log.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/log.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/log.go
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/utils.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/log/utils.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/log/utils.go
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/common.go
new file mode 100644
index 0000000..3aef492
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/common.go
@@ -0,0 +1,31 @@
+/*
+ * 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 meters
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/meter_utils.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/meter_utils.go
new file mode 100644
index 0000000..56e8ecc
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/meters/meter_utils.go
@@ -0,0 +1,71 @@
+/*
+ * 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 meters
+
+import (
+	"context"
+	"fmt"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	tp_pb "github.com/opencord/voltha-protos/v5/go/tech_profile"
+)
+
+// GetTrafficShapingInfo returns CIR,PIR and GIR values
+func GetTrafficShapingInfo(ctx context.Context, meterConfig *ofp.OfpMeterConfig) (*tp_pb.TrafficShapingInfo, error) {
+	switch meterBandSize := len(meterConfig.Bands); {
+	case meterBandSize == 1:
+		band := meterConfig.Bands[0]
+		if band.BurstSize == 0 { // GIR, tcont type 1
+			return &tp_pb.TrafficShapingInfo{Gir: band.Rate}, nil
+		}
+		return &tp_pb.TrafficShapingInfo{Pir: band.Rate, Pbs: band.BurstSize}, nil // PIR, tcont type 4
+	case meterBandSize == 2:
+		firstBand, secondBand := meterConfig.Bands[0], meterConfig.Bands[1]
+		if firstBand.BurstSize == 0 && secondBand.BurstSize == 0 &&
+			firstBand.Rate == secondBand.Rate { // PIR == GIR, tcont type 1
+			return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Gir: secondBand.Rate}, nil
+		}
+		if firstBand.BurstSize > 0 && secondBand.BurstSize > 0 { // PIR, CIR, tcont type 2 or 3
+			if firstBand.Rate > secondBand.Rate { // always PIR >= CIR
+				return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Pbs: firstBand.BurstSize, Cir: secondBand.Rate, Cbs: secondBand.BurstSize}, nil
+			}
+			return &tp_pb.TrafficShapingInfo{Pir: secondBand.Rate, Pbs: secondBand.BurstSize, Cir: firstBand.Rate, Cbs: firstBand.BurstSize}, nil
+		}
+	case meterBandSize == 3: // PIR,CIR,GIR, tcont type 5
+		var count, girIndex int
+		for i, band := range meterConfig.Bands {
+			if band.BurstSize == 0 { // find GIR
+				count = count + 1
+				girIndex = i
+			}
+		}
+		if count == 1 {
+			bands := make([]*ofp.OfpMeterBandHeader, len(meterConfig.Bands))
+			copy(bands, meterConfig.Bands)
+			pirCirBands := append(bands[:girIndex], bands[girIndex+1:]...)
+			firstBand, secondBand := pirCirBands[0], pirCirBands[1]
+			if firstBand.Rate > secondBand.Rate {
+				return &tp_pb.TrafficShapingInfo{Pir: firstBand.Rate, Pbs: firstBand.BurstSize, Cir: secondBand.Rate, Cbs: secondBand.BurstSize, Gir: meterConfig.Bands[girIndex].Rate}, nil
+			}
+			return &tp_pb.TrafficShapingInfo{Pir: secondBand.Rate, Pbs: secondBand.BurstSize, Cir: firstBand.Rate, Cbs: firstBand.BurstSize, Gir: meterConfig.Bands[girIndex].Rate}, nil
+		}
+	default:
+		logger.Errorw(ctx, "invalid-meter-config", log.Fields{"meter-config": meterConfig})
+		return nil, fmt.Errorf("invalid-meter-config: %v", meterConfig)
+	}
+	return nil, fmt.Errorf("invalid-meter-config: %v", meterConfig)
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/common.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/common.go
new file mode 100644
index 0000000..6508fd4
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/common.go
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2020-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 probe
+
+import (
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+var logger log.CLogger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.RegisterPackage(log.JSON, log.ErrorLevel, log.Fields{})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/probe.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/probe.go
new file mode 100644
index 0000000..84a2d5f
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/probe/probe.go
@@ -0,0 +1,306 @@
+/*
+ * 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 probe
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+	"sync"
+
+	"github.com/opencord/voltha-lib-go/v7/pkg/log"
+)
+
+// ProbeContextKey used to fetch the Probe instance from a context
+type ProbeContextKeyType string
+
+// ServiceStatus typed values for service status
+type ServiceStatus int
+
+const (
+	// ServiceStatusUnknown initial state of services
+	ServiceStatusUnknown ServiceStatus = iota
+
+	// ServiceStatusPreparing to optionally be used for prep, such as connecting
+	ServiceStatusPreparing
+
+	// ServiceStatusPrepared to optionally be used when prep is complete, but before run
+	ServiceStatusPrepared
+
+	// ServiceStatusRunning service is functional
+	ServiceStatusRunning
+
+	// ServiceStatusStopped service has stopped, but not because of error
+	ServiceStatusStopped
+
+	// ServiceStatusFailed service has stopped because of an error
+	ServiceStatusFailed
+
+	// ServiceStatusNotReady service has started but is unable to accept requests
+	ServiceStatusNotReady
+)
+
+const (
+	// ProbeContextKey value of context key to fetch probe
+	ProbeContextKey = ProbeContextKeyType("status-update-probe")
+)
+
+// String convert ServiceStatus values to strings
+func (s ServiceStatus) String() string {
+	switch s {
+	default:
+		fallthrough
+	case ServiceStatusUnknown:
+		return "Unknown"
+	case ServiceStatusPreparing:
+		return "Preparing"
+	case ServiceStatusPrepared:
+		return "Prepared"
+	case ServiceStatusRunning:
+		return "Running"
+	case ServiceStatusStopped:
+		return "Stopped"
+	case ServiceStatusFailed:
+		return "Failed"
+	case ServiceStatusNotReady:
+		return "NotReady"
+	}
+}
+
+// ServiceStatusUpdate status update event
+type ServiceStatusUpdate struct {
+	Name   string
+	Status ServiceStatus
+}
+
+// Probe reciever on which to implement probe capabilities
+type Probe struct {
+	readyFunc  func(map[string]ServiceStatus) bool
+	healthFunc func(map[string]ServiceStatus) bool
+
+	mutex     sync.RWMutex
+	status    map[string]ServiceStatus
+	isReady   bool
+	isHealthy bool
+}
+
+// WithReadyFunc override the default ready calculation function
+func (p *Probe) WithReadyFunc(readyFunc func(map[string]ServiceStatus) bool) *Probe {
+	p.readyFunc = readyFunc
+	return p
+}
+
+// WithHealthFunc override the default health calculation function
+func (p *Probe) WithHealthFunc(healthFunc func(map[string]ServiceStatus) bool) *Probe {
+	p.healthFunc = healthFunc
+	return p
+}
+
+// RegisterService register one or more service names with the probe, status will be track against service name
+func (p *Probe) RegisterService(ctx context.Context, names ...string) {
+	p.mutex.Lock()
+	defer p.mutex.Unlock()
+	if p.status == nil {
+		p.status = make(map[string]ServiceStatus)
+	}
+	for _, name := range names {
+		if _, ok := p.status[name]; !ok {
+			p.status[name] = ServiceStatusUnknown
+			logger.Debugw(ctx, "probe-service-registered", log.Fields{"service-name": name})
+		}
+	}
+
+	if p.readyFunc != nil {
+		p.isReady = p.readyFunc(p.status)
+	} else {
+		p.isReady = defaultReadyFunc(p.status)
+	}
+
+	if p.healthFunc != nil {
+		p.isHealthy = p.healthFunc(p.status)
+	} else {
+		p.isHealthy = defaultHealthFunc(p.status)
+	}
+}
+
+// UpdateStatus utility function to send a service update to the probe
+func (p *Probe) UpdateStatus(ctx context.Context, name string, status ServiceStatus) {
+	p.mutex.Lock()
+	defer p.mutex.Unlock()
+	if p.status == nil {
+		p.status = make(map[string]ServiceStatus)
+	}
+
+	// if status hasn't changed, avoid doing useless work
+	existingStatus, ok := p.status[name]
+	if ok && (existingStatus == status) {
+		return
+	}
+
+	p.status[name] = status
+	if p.readyFunc != nil {
+		p.isReady = p.readyFunc(p.status)
+	} else {
+		p.isReady = defaultReadyFunc(p.status)
+	}
+
+	if p.healthFunc != nil {
+		p.isHealthy = p.healthFunc(p.status)
+	} else {
+		p.isHealthy = defaultHealthFunc(p.status)
+	}
+	logger.Debugw(ctx, "probe-service-status-updated",
+		log.Fields{
+			"service-name": name,
+			"status":       status.String(),
+			"ready":        p.isReady,
+			"health":       p.isHealthy,
+		})
+}
+
+func (p *Probe) GetStatus(name string) ServiceStatus {
+	p.mutex.Lock()
+	defer p.mutex.Unlock()
+
+	if p.status == nil {
+		p.status = make(map[string]ServiceStatus)
+	}
+
+	currentStatus, ok := p.status[name]
+	if ok {
+		return currentStatus
+	}
+
+	return ServiceStatusUnknown
+}
+
+func GetProbeFromContext(ctx context.Context) *Probe {
+	if ctx != nil {
+		if value := ctx.Value(ProbeContextKey); value != nil {
+			if p, ok := value.(*Probe); ok {
+				return p
+			}
+		}
+	}
+	return nil
+}
+
+// UpdateStatusFromContext a convenience function to pull the Probe reference from the
+// Context, if it exists, and then calling UpdateStatus on that Probe reference. If Context
+// is nil or if a Probe reference is not associated with the ProbeContextKey then nothing
+// happens
+func UpdateStatusFromContext(ctx context.Context, name string, status ServiceStatus) {
+	p := GetProbeFromContext(ctx)
+	if p != nil {
+		p.UpdateStatus(ctx, name, status)
+	}
+}
+
+// pulled out to a function to help better enable unit testing
+func (p *Probe) readzFunc(w http.ResponseWriter, req *http.Request) {
+	p.mutex.RLock()
+	defer p.mutex.RUnlock()
+	if p.isReady {
+		w.WriteHeader(http.StatusOK)
+	} else {
+		w.WriteHeader(http.StatusTeapot)
+	}
+}
+func (p *Probe) healthzFunc(w http.ResponseWriter, req *http.Request) {
+	p.mutex.RLock()
+	defer p.mutex.RUnlock()
+	if p.isHealthy {
+		w.WriteHeader(http.StatusOK)
+	} else {
+		w.WriteHeader(http.StatusTeapot)
+	}
+}
+func (p *Probe) detailzFunc(w http.ResponseWriter, req *http.Request) {
+	ctx := context.Background()
+	p.mutex.RLock()
+	defer p.mutex.RUnlock()
+	w.Header().Set("Content-Type", "application/json")
+	if _, err := w.Write([]byte("{")); err != nil {
+		logger.Errorw(ctx, "write-response", log.Fields{"error": err})
+		w.WriteHeader(http.StatusInternalServerError)
+		return
+	}
+	comma := ""
+	for c, s := range p.status {
+		if _, err := w.Write([]byte(fmt.Sprintf("%s\"%s\": \"%s\"", comma, c, s.String()))); err != nil {
+			logger.Errorw(ctx, "write-response", log.Fields{"error": err})
+			w.WriteHeader(http.StatusInternalServerError)
+			return
+		}
+		comma = ", "
+	}
+	if _, err := w.Write([]byte("}")); err != nil {
+		logger.Errorw(ctx, "write-response", log.Fields{"error": err})
+		w.WriteHeader(http.StatusInternalServerError)
+		return
+	}
+	w.WriteHeader(http.StatusOK)
+}
+
+// ListenAndServe implements 3 HTTP endpoints on the given port for healthz, readz, and detailz. Returns only on error
+func (p *Probe) ListenAndServe(ctx context.Context, address string) {
+	mux := http.NewServeMux()
+
+	// Returns the result of the readyFunc calculation
+	mux.HandleFunc("/readz", p.readzFunc)
+
+	// Returns the result of the healthFunc calculation
+	mux.HandleFunc("/healthz", p.healthzFunc)
+
+	// Returns the details of the services and their status as JSON
+	mux.HandleFunc("/detailz", p.detailzFunc)
+	s := &http.Server{
+		Addr:    address,
+		Handler: mux,
+	}
+	logger.Fatal(ctx, s.ListenAndServe())
+}
+
+func (p *Probe) IsReady() bool {
+	return p.isReady
+}
+
+// defaultReadyFunc if all services are running then ready, else not
+func defaultReadyFunc(services map[string]ServiceStatus) bool {
+	if len(services) == 0 {
+		return false
+	}
+	for _, status := range services {
+		if status != ServiceStatusRunning {
+			return false
+		}
+	}
+	return true
+}
+
+// defaultHealthFunc if no service is stopped or failed, then healthy, else not.
+// service is start as unknown, so they are considered healthy
+func defaultHealthFunc(services map[string]ServiceStatus) bool {
+	if len(services) == 0 {
+		return false
+	}
+	for _, status := range services {
+		if status == ServiceStatusStopped || status == ServiceStatusFailed {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v5/pkg/version/version.go b/vendor/github.com/opencord/voltha-lib-go/v7/pkg/version/version.go
similarity index 100%
rename from vendor/github.com/opencord/voltha-lib-go/v5/pkg/version/version.go
rename to vendor/github.com/opencord/voltha-lib-go/v7/pkg/version/version.go
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/common/common.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/common/common.pb.go
deleted file mode 100644
index b57e775..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/common/common.pb.go
+++ /dev/null
@@ -1,651 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/common.proto
-
-package common
-
-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
-
-type TestModeKeys int32
-
-const (
-	TestModeKeys_api_test TestModeKeys = 0
-)
-
-var TestModeKeys_name = map[int32]string{
-	0: "api_test",
-}
-
-var TestModeKeys_value = map[string]int32{
-	"api_test": 0,
-}
-
-func (x TestModeKeys) String() string {
-	return proto.EnumName(TestModeKeys_name, int32(x))
-}
-
-func (TestModeKeys) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{0}
-}
-
-// Administrative State
-type AdminState_Types int32
-
-const (
-	// The administrative state of the device is unknown
-	AdminState_UNKNOWN AdminState_Types = 0
-	// The device is pre-provisioned into Voltha, but not contacted by it
-	AdminState_PREPROVISIONED AdminState_Types = 1
-	// The device is enabled for activation and operation
-	AdminState_ENABLED AdminState_Types = 2
-	// The device is disabled and shall not perform its intended forwarding
-	// functions other than being available for re-activation.
-	AdminState_DISABLED AdminState_Types = 3
-	// The device is in the state of image download
-	AdminState_DOWNLOADING_IMAGE AdminState_Types = 4
-)
-
-var AdminState_Types_name = map[int32]string{
-	0: "UNKNOWN",
-	1: "PREPROVISIONED",
-	2: "ENABLED",
-	3: "DISABLED",
-	4: "DOWNLOADING_IMAGE",
-}
-
-var AdminState_Types_value = map[string]int32{
-	"UNKNOWN":           0,
-	"PREPROVISIONED":    1,
-	"ENABLED":           2,
-	"DISABLED":          3,
-	"DOWNLOADING_IMAGE": 4,
-}
-
-func (x AdminState_Types) String() string {
-	return proto.EnumName(AdminState_Types_name, int32(x))
-}
-
-func (AdminState_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{2, 0}
-}
-
-// Operational Status
-type OperStatus_Types int32
-
-const (
-	// The status of the device is unknown at this point
-	OperStatus_UNKNOWN OperStatus_Types = 0
-	// The device has been discovered, but not yet activated
-	OperStatus_DISCOVERED OperStatus_Types = 1
-	// The device is being activated (booted, rebooted, upgraded, etc.)
-	OperStatus_ACTIVATING OperStatus_Types = 2
-	// Service impacting tests are being conducted
-	OperStatus_TESTING OperStatus_Types = 3
-	// The device is up and active
-	OperStatus_ACTIVE OperStatus_Types = 4
-	// The device has failed and cannot fulfill its intended role
-	OperStatus_FAILED OperStatus_Types = 5
-	// The device is reconciling
-	OperStatus_RECONCILING OperStatus_Types = 6
-	// The device is in reconciling failed
-	OperStatus_RECONCILING_FAILED OperStatus_Types = 7
-)
-
-var OperStatus_Types_name = map[int32]string{
-	0: "UNKNOWN",
-	1: "DISCOVERED",
-	2: "ACTIVATING",
-	3: "TESTING",
-	4: "ACTIVE",
-	5: "FAILED",
-	6: "RECONCILING",
-	7: "RECONCILING_FAILED",
-}
-
-var OperStatus_Types_value = map[string]int32{
-	"UNKNOWN":            0,
-	"DISCOVERED":         1,
-	"ACTIVATING":         2,
-	"TESTING":            3,
-	"ACTIVE":             4,
-	"FAILED":             5,
-	"RECONCILING":        6,
-	"RECONCILING_FAILED": 7,
-}
-
-func (x OperStatus_Types) String() string {
-	return proto.EnumName(OperStatus_Types_name, int32(x))
-}
-
-func (OperStatus_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{3, 0}
-}
-
-// Connectivity Status
-type ConnectStatus_Types int32
-
-const (
-	// The device connectivity status is unknown
-	ConnectStatus_UNKNOWN ConnectStatus_Types = 0
-	// The device cannot be reached by Voltha
-	ConnectStatus_UNREACHABLE ConnectStatus_Types = 1
-	// There is live communication between device and Voltha
-	ConnectStatus_REACHABLE ConnectStatus_Types = 2
-)
-
-var ConnectStatus_Types_name = map[int32]string{
-	0: "UNKNOWN",
-	1: "UNREACHABLE",
-	2: "REACHABLE",
-}
-
-var ConnectStatus_Types_value = map[string]int32{
-	"UNKNOWN":     0,
-	"UNREACHABLE": 1,
-	"REACHABLE":   2,
-}
-
-func (x ConnectStatus_Types) String() string {
-	return proto.EnumName(ConnectStatus_Types_name, int32(x))
-}
-
-func (ConnectStatus_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{4, 0}
-}
-
-type OperationResp_OperationReturnCode int32
-
-const (
-	OperationResp_OPERATION_SUCCESS     OperationResp_OperationReturnCode = 0
-	OperationResp_OPERATION_FAILURE     OperationResp_OperationReturnCode = 1
-	OperationResp_OPERATION_UNSUPPORTED OperationResp_OperationReturnCode = 2
-	OperationResp_OPERATION_IN_PROGRESS OperationResp_OperationReturnCode = 3
-)
-
-var OperationResp_OperationReturnCode_name = map[int32]string{
-	0: "OPERATION_SUCCESS",
-	1: "OPERATION_FAILURE",
-	2: "OPERATION_UNSUPPORTED",
-	3: "OPERATION_IN_PROGRESS",
-}
-
-var OperationResp_OperationReturnCode_value = map[string]int32{
-	"OPERATION_SUCCESS":     0,
-	"OPERATION_FAILURE":     1,
-	"OPERATION_UNSUPPORTED": 2,
-	"OPERATION_IN_PROGRESS": 3,
-}
-
-func (x OperationResp_OperationReturnCode) String() string {
-	return proto.EnumName(OperationResp_OperationReturnCode_name, int32(x))
-}
-
-func (OperationResp_OperationReturnCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{5, 0}
-}
-
-type ValueType_Type int32
-
-const (
-	ValueType_EMPTY    ValueType_Type = 0
-	ValueType_DISTANCE ValueType_Type = 1
-)
-
-var ValueType_Type_name = map[int32]string{
-	0: "EMPTY",
-	1: "DISTANCE",
-}
-
-var ValueType_Type_value = map[string]int32{
-	"EMPTY":    0,
-	"DISTANCE": 1,
-}
-
-func (x ValueType_Type) String() string {
-	return proto.EnumName(ValueType_Type_name, int32(x))
-}
-
-func (ValueType_Type) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{6, 0}
-}
-
-// Convey a resource identifier
-type ID struct {
-	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ID) Reset()         { *m = ID{} }
-func (m *ID) String() string { return proto.CompactTextString(m) }
-func (*ID) ProtoMessage()    {}
-func (*ID) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{0}
-}
-
-func (m *ID) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ID.Unmarshal(m, b)
-}
-func (m *ID) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ID.Marshal(b, m, deterministic)
-}
-func (m *ID) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ID.Merge(m, src)
-}
-func (m *ID) XXX_Size() int {
-	return xxx_messageInfo_ID.Size(m)
-}
-func (m *ID) XXX_DiscardUnknown() {
-	xxx_messageInfo_ID.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ID proto.InternalMessageInfo
-
-func (m *ID) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-// Represents a list of IDs
-type IDs struct {
-	Items                []*ID    `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *IDs) Reset()         { *m = IDs{} }
-func (m *IDs) String() string { return proto.CompactTextString(m) }
-func (*IDs) ProtoMessage()    {}
-func (*IDs) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{1}
-}
-
-func (m *IDs) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IDs.Unmarshal(m, b)
-}
-func (m *IDs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IDs.Marshal(b, m, deterministic)
-}
-func (m *IDs) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IDs.Merge(m, src)
-}
-func (m *IDs) XXX_Size() int {
-	return xxx_messageInfo_IDs.Size(m)
-}
-func (m *IDs) XXX_DiscardUnknown() {
-	xxx_messageInfo_IDs.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IDs proto.InternalMessageInfo
-
-func (m *IDs) GetItems() []*ID {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type AdminState struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AdminState) Reset()         { *m = AdminState{} }
-func (m *AdminState) String() string { return proto.CompactTextString(m) }
-func (*AdminState) ProtoMessage()    {}
-func (*AdminState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{2}
-}
-
-func (m *AdminState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AdminState.Unmarshal(m, b)
-}
-func (m *AdminState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AdminState.Marshal(b, m, deterministic)
-}
-func (m *AdminState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AdminState.Merge(m, src)
-}
-func (m *AdminState) XXX_Size() int {
-	return xxx_messageInfo_AdminState.Size(m)
-}
-func (m *AdminState) XXX_DiscardUnknown() {
-	xxx_messageInfo_AdminState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AdminState proto.InternalMessageInfo
-
-type OperStatus struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OperStatus) Reset()         { *m = OperStatus{} }
-func (m *OperStatus) String() string { return proto.CompactTextString(m) }
-func (*OperStatus) ProtoMessage()    {}
-func (*OperStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{3}
-}
-
-func (m *OperStatus) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OperStatus.Unmarshal(m, b)
-}
-func (m *OperStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OperStatus.Marshal(b, m, deterministic)
-}
-func (m *OperStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OperStatus.Merge(m, src)
-}
-func (m *OperStatus) XXX_Size() int {
-	return xxx_messageInfo_OperStatus.Size(m)
-}
-func (m *OperStatus) XXX_DiscardUnknown() {
-	xxx_messageInfo_OperStatus.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OperStatus proto.InternalMessageInfo
-
-type ConnectStatus struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ConnectStatus) Reset()         { *m = ConnectStatus{} }
-func (m *ConnectStatus) String() string { return proto.CompactTextString(m) }
-func (*ConnectStatus) ProtoMessage()    {}
-func (*ConnectStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{4}
-}
-
-func (m *ConnectStatus) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ConnectStatus.Unmarshal(m, b)
-}
-func (m *ConnectStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ConnectStatus.Marshal(b, m, deterministic)
-}
-func (m *ConnectStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ConnectStatus.Merge(m, src)
-}
-func (m *ConnectStatus) XXX_Size() int {
-	return xxx_messageInfo_ConnectStatus.Size(m)
-}
-func (m *ConnectStatus) XXX_DiscardUnknown() {
-	xxx_messageInfo_ConnectStatus.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ConnectStatus proto.InternalMessageInfo
-
-type OperationResp struct {
-	// Return code
-	Code OperationResp_OperationReturnCode `protobuf:"varint,1,opt,name=code,proto3,enum=common.OperationResp_OperationReturnCode" json:"code,omitempty"`
-	// Additional Info
-	AdditionalInfo       string   `protobuf:"bytes,2,opt,name=additional_info,json=additionalInfo,proto3" json:"additional_info,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OperationResp) Reset()         { *m = OperationResp{} }
-func (m *OperationResp) String() string { return proto.CompactTextString(m) }
-func (*OperationResp) ProtoMessage()    {}
-func (*OperationResp) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{5}
-}
-
-func (m *OperationResp) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OperationResp.Unmarshal(m, b)
-}
-func (m *OperationResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OperationResp.Marshal(b, m, deterministic)
-}
-func (m *OperationResp) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OperationResp.Merge(m, src)
-}
-func (m *OperationResp) XXX_Size() int {
-	return xxx_messageInfo_OperationResp.Size(m)
-}
-func (m *OperationResp) XXX_DiscardUnknown() {
-	xxx_messageInfo_OperationResp.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OperationResp proto.InternalMessageInfo
-
-func (m *OperationResp) GetCode() OperationResp_OperationReturnCode {
-	if m != nil {
-		return m.Code
-	}
-	return OperationResp_OPERATION_SUCCESS
-}
-
-func (m *OperationResp) GetAdditionalInfo() string {
-	if m != nil {
-		return m.AdditionalInfo
-	}
-	return ""
-}
-
-type ValueType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ValueType) Reset()         { *m = ValueType{} }
-func (m *ValueType) String() string { return proto.CompactTextString(m) }
-func (*ValueType) ProtoMessage()    {}
-func (*ValueType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{6}
-}
-
-func (m *ValueType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ValueType.Unmarshal(m, b)
-}
-func (m *ValueType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ValueType.Marshal(b, m, deterministic)
-}
-func (m *ValueType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueType.Merge(m, src)
-}
-func (m *ValueType) XXX_Size() int {
-	return xxx_messageInfo_ValueType.Size(m)
-}
-func (m *ValueType) XXX_DiscardUnknown() {
-	xxx_messageInfo_ValueType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ValueType proto.InternalMessageInfo
-
-type ValueSpecifier struct {
-	Id                   string         `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Value                ValueType_Type `protobuf:"varint,2,opt,name=value,proto3,enum=common.ValueType_Type" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *ValueSpecifier) Reset()         { *m = ValueSpecifier{} }
-func (m *ValueSpecifier) String() string { return proto.CompactTextString(m) }
-func (*ValueSpecifier) ProtoMessage()    {}
-func (*ValueSpecifier) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{7}
-}
-
-func (m *ValueSpecifier) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ValueSpecifier.Unmarshal(m, b)
-}
-func (m *ValueSpecifier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ValueSpecifier.Marshal(b, m, deterministic)
-}
-func (m *ValueSpecifier) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueSpecifier.Merge(m, src)
-}
-func (m *ValueSpecifier) XXX_Size() int {
-	return xxx_messageInfo_ValueSpecifier.Size(m)
-}
-func (m *ValueSpecifier) XXX_DiscardUnknown() {
-	xxx_messageInfo_ValueSpecifier.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ValueSpecifier proto.InternalMessageInfo
-
-func (m *ValueSpecifier) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *ValueSpecifier) GetValue() ValueType_Type {
-	if m != nil {
-		return m.Value
-	}
-	return ValueType_EMPTY
-}
-
-type ReturnValues struct {
-	Set                  uint32   `protobuf:"varint,1,opt,name=Set,proto3" json:"Set,omitempty"`
-	Unsupported          uint32   `protobuf:"varint,2,opt,name=Unsupported,proto3" json:"Unsupported,omitempty"`
-	Error                uint32   `protobuf:"varint,3,opt,name=Error,proto3" json:"Error,omitempty"`
-	Distance             uint32   `protobuf:"varint,4,opt,name=Distance,proto3" json:"Distance,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ReturnValues) Reset()         { *m = ReturnValues{} }
-func (m *ReturnValues) String() string { return proto.CompactTextString(m) }
-func (*ReturnValues) ProtoMessage()    {}
-func (*ReturnValues) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c2e3fd231961e826, []int{8}
-}
-
-func (m *ReturnValues) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReturnValues.Unmarshal(m, b)
-}
-func (m *ReturnValues) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReturnValues.Marshal(b, m, deterministic)
-}
-func (m *ReturnValues) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReturnValues.Merge(m, src)
-}
-func (m *ReturnValues) XXX_Size() int {
-	return xxx_messageInfo_ReturnValues.Size(m)
-}
-func (m *ReturnValues) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReturnValues.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReturnValues proto.InternalMessageInfo
-
-func (m *ReturnValues) GetSet() uint32 {
-	if m != nil {
-		return m.Set
-	}
-	return 0
-}
-
-func (m *ReturnValues) GetUnsupported() uint32 {
-	if m != nil {
-		return m.Unsupported
-	}
-	return 0
-}
-
-func (m *ReturnValues) GetError() uint32 {
-	if m != nil {
-		return m.Error
-	}
-	return 0
-}
-
-func (m *ReturnValues) GetDistance() uint32 {
-	if m != nil {
-		return m.Distance
-	}
-	return 0
-}
-
-func init() {
-	proto.RegisterEnum("common.TestModeKeys", TestModeKeys_name, TestModeKeys_value)
-	proto.RegisterEnum("common.AdminState_Types", AdminState_Types_name, AdminState_Types_value)
-	proto.RegisterEnum("common.OperStatus_Types", OperStatus_Types_name, OperStatus_Types_value)
-	proto.RegisterEnum("common.ConnectStatus_Types", ConnectStatus_Types_name, ConnectStatus_Types_value)
-	proto.RegisterEnum("common.OperationResp_OperationReturnCode", OperationResp_OperationReturnCode_name, OperationResp_OperationReturnCode_value)
-	proto.RegisterEnum("common.ValueType_Type", ValueType_Type_name, ValueType_Type_value)
-	proto.RegisterType((*ID)(nil), "common.ID")
-	proto.RegisterType((*IDs)(nil), "common.IDs")
-	proto.RegisterType((*AdminState)(nil), "common.AdminState")
-	proto.RegisterType((*OperStatus)(nil), "common.OperStatus")
-	proto.RegisterType((*ConnectStatus)(nil), "common.ConnectStatus")
-	proto.RegisterType((*OperationResp)(nil), "common.OperationResp")
-	proto.RegisterType((*ValueType)(nil), "common.ValueType")
-	proto.RegisterType((*ValueSpecifier)(nil), "common.ValueSpecifier")
-	proto.RegisterType((*ReturnValues)(nil), "common.ReturnValues")
-}
-
-func init() { proto.RegisterFile("voltha_protos/common.proto", fileDescriptor_c2e3fd231961e826) }
-
-var fileDescriptor_c2e3fd231961e826 = []byte{
-	// 634 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0x5d, 0x4f, 0xe3, 0x3a,
-	0x10, 0x6d, 0xfa, 0x05, 0x9d, 0xd2, 0x90, 0x6b, 0x3e, 0xd4, 0x8b, 0xae, 0x74, 0xab, 0xbc, 0xc0,
-	0xbd, 0x62, 0x5b, 0x89, 0xe5, 0x75, 0x1f, 0x42, 0xe2, 0xed, 0x5a, 0x80, 0x53, 0x39, 0x49, 0xd1,
-	0xf2, 0x12, 0x85, 0xc6, 0x40, 0x24, 0x1a, 0x47, 0x89, 0x8b, 0xc4, 0xeb, 0xfe, 0x83, 0xfd, 0xab,
-	0xfb, 0x0b, 0x56, 0x76, 0xca, 0xd7, 0x8a, 0x97, 0xd6, 0x67, 0xce, 0xc9, 0x8c, 0xcf, 0x8c, 0x07,
-	0x0e, 0x1e, 0xc5, 0x83, 0xbc, 0x4f, 0xe2, 0xa2, 0x14, 0x52, 0x54, 0x93, 0x85, 0x58, 0x2e, 0x45,
-	0x3e, 0xd6, 0x08, 0x75, 0x6b, 0x64, 0xef, 0x42, 0x93, 0x78, 0xc8, 0x84, 0x66, 0x96, 0x0e, 0x8d,
-	0x91, 0x71, 0xd4, 0x63, 0xcd, 0x2c, 0xb5, 0x0f, 0xa1, 0x45, 0xbc, 0x0a, 0x8d, 0xa0, 0x93, 0x49,
-	0xbe, 0xac, 0x86, 0xc6, 0xa8, 0x75, 0xd4, 0x3f, 0x81, 0xf1, 0x3a, 0x05, 0xf1, 0x58, 0x4d, 0xd8,
-	0xf7, 0x00, 0x4e, 0xba, 0xcc, 0xf2, 0x40, 0x26, 0x92, 0xdb, 0xd7, 0xd0, 0x09, 0x9f, 0x0a, 0x5e,
-	0xa1, 0x3e, 0x6c, 0x44, 0xf4, 0x9c, 0xfa, 0x57, 0xd4, 0x6a, 0x20, 0x04, 0xe6, 0x8c, 0xe1, 0x19,
-	0xf3, 0xe7, 0x24, 0x20, 0x3e, 0xc5, 0x9e, 0x65, 0x28, 0x01, 0xa6, 0xce, 0xd9, 0x05, 0xf6, 0xac,
-	0x26, 0xda, 0x82, 0x4d, 0x8f, 0x04, 0x35, 0x6a, 0xa1, 0x3d, 0xf8, 0xcb, 0xf3, 0xaf, 0xe8, 0x85,
-	0xef, 0x78, 0x84, 0x4e, 0x63, 0x72, 0xe9, 0x4c, 0xb1, 0xd5, 0xb6, 0x7f, 0x1a, 0x00, 0x7e, 0xc1,
-	0x4b, 0x55, 0x69, 0x55, 0xd9, 0x3f, 0x8c, 0x0f, 0x6b, 0x99, 0x00, 0x1e, 0x09, 0x5c, 0x7f, 0x8e,
-	0x99, 0xae, 0x63, 0x02, 0x38, 0x6e, 0x48, 0xe6, 0x4e, 0x48, 0xe8, 0xd4, 0x6a, 0x2a, 0x71, 0x88,
-	0x03, 0x0d, 0x5a, 0x08, 0xa0, 0xab, 0x49, 0x6c, 0xb5, 0xd5, 0xf9, 0xab, 0x43, 0xd4, 0x0d, 0x3a,
-	0x68, 0x1b, 0xfa, 0x0c, 0xbb, 0x3e, 0x75, 0xc9, 0x85, 0x12, 0x76, 0xd1, 0x3e, 0xa0, 0x37, 0x81,
-	0x78, 0x2d, 0xdc, 0xb0, 0x31, 0x0c, 0x5c, 0x91, 0xe7, 0x7c, 0x21, 0xd7, 0xb7, 0x3a, 0xfd, 0xf0,
-	0x52, 0xdb, 0xd0, 0x8f, 0x28, 0xc3, 0x8e, 0xfb, 0x4d, 0x79, 0xb4, 0x0c, 0x34, 0x80, 0xde, 0x2b,
-	0x6c, 0xda, 0xbf, 0x0c, 0x18, 0x28, 0x6b, 0x89, 0xcc, 0x44, 0xce, 0x78, 0x55, 0xa0, 0x2f, 0xd0,
-	0x5e, 0x88, 0x94, 0xeb, 0x89, 0x98, 0x27, 0xff, 0x3d, 0xf7, 0xfd, 0x9d, 0xe8, 0x2d, 0x92, 0xab,
-	0x32, 0x77, 0x45, 0xca, 0x99, 0xfe, 0x0c, 0x1d, 0xc2, 0x76, 0x92, 0xa6, 0x99, 0xe2, 0x92, 0x87,
-	0x38, 0xcb, 0x6f, 0xc5, 0xb0, 0xa9, 0x67, 0x6b, 0xbe, 0x86, 0x49, 0x7e, 0x2b, 0xec, 0x27, 0xd8,
-	0xf9, 0x20, 0x8b, 0x1a, 0x81, 0x3f, 0xc3, 0xcc, 0x09, 0x89, 0x4f, 0xe3, 0x20, 0x72, 0x5d, 0x1c,
-	0x04, 0x56, 0xe3, 0x7d, 0x58, 0x35, 0x21, 0x62, 0xca, 0xcd, 0xdf, 0xb0, 0xf7, 0x1a, 0x8e, 0x68,
-	0x10, 0xcd, 0x66, 0x3e, 0x0b, 0xf5, 0x64, 0xdf, 0x51, 0x84, 0xc6, 0x33, 0xe6, 0x4f, 0x99, 0x4a,
-	0xd6, 0xb2, 0x8f, 0xa1, 0x37, 0x4f, 0x1e, 0x56, 0x5c, 0xf5, 0xcb, 0xfe, 0x17, 0xda, 0xea, 0x1f,
-	0xf5, 0xa0, 0x83, 0x2f, 0x67, 0xe1, 0x77, 0xab, 0xb1, 0x7e, 0x14, 0xa1, 0x43, 0x5d, 0x6c, 0x19,
-	0x36, 0x05, 0x53, 0xab, 0x83, 0x82, 0x2f, 0xb2, 0xdb, 0x8c, 0x97, 0x7f, 0x3e, 0x59, 0x74, 0x0c,
-	0x9d, 0x47, 0xa5, 0xd0, 0x4e, 0xcd, 0x93, 0xfd, 0xe7, 0x9e, 0xbd, 0x14, 0x19, 0xab, 0x1f, 0x56,
-	0x8b, 0x6c, 0x09, 0x5b, 0xb5, 0x5f, 0x4d, 0x57, 0xc8, 0x82, 0x56, 0xc0, 0xa5, 0x4e, 0x37, 0x60,
-	0xea, 0x88, 0x46, 0xd0, 0x8f, 0xf2, 0x6a, 0x55, 0x14, 0xa2, 0x94, 0x3c, 0xd5, 0x59, 0x07, 0xec,
-	0x6d, 0x08, 0xed, 0x42, 0x07, 0x97, 0xa5, 0x28, 0x87, 0x2d, 0xcd, 0xd5, 0x00, 0x1d, 0xc0, 0xa6,
-	0x97, 0x55, 0x32, 0xc9, 0x17, 0x7c, 0xd8, 0xd6, 0xc4, 0x0b, 0xfe, 0xff, 0x1f, 0xd8, 0x0a, 0x79,
-	0x25, 0x2f, 0x45, 0xca, 0xcf, 0xf9, 0x53, 0xa5, 0x3c, 0x26, 0x45, 0x16, 0x4b, 0x5e, 0x49, 0xab,
-	0x71, 0x86, 0x61, 0x47, 0x94, 0x77, 0x63, 0x51, 0xf0, 0x7c, 0x21, 0xca, 0x74, 0x5c, 0x6f, 0xef,
-	0xf5, 0xf8, 0x2e, 0x93, 0xf7, 0xab, 0x1b, 0xe5, 0x67, 0xf2, 0xcc, 0x4d, 0x6a, 0xee, 0xd3, 0x7a,
-	0xb3, 0x1f, 0x4f, 0x27, 0x77, 0x62, 0xbd, 0xdf, 0x37, 0x5d, 0x1d, 0xfc, 0xfc, 0x3b, 0x00, 0x00,
-	0xff, 0xff, 0x29, 0xd3, 0x39, 0x3c, 0xfe, 0x03, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/common/meta.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/common/meta.pb.go
deleted file mode 100644
index 0af225d..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/common/meta.pb.go
+++ /dev/null
@@ -1,142 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/meta.proto
-
-package common
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
-	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 Access int32
-
-const (
-	// read-write, stored attribute
-	Access_CONFIG Access = 0
-	// read-only field, stored with the model, covered by its hash
-	Access_READ_ONLY Access = 1
-	// A read-only attribute that is not stored in the model, not covered
-	// by its hash, its value is filled real-time upon each request.
-	Access_REAL_TIME Access = 2
-)
-
-var Access_name = map[int32]string{
-	0: "CONFIG",
-	1: "READ_ONLY",
-	2: "REAL_TIME",
-}
-
-var Access_value = map[string]int32{
-	"CONFIG":    0,
-	"READ_ONLY": 1,
-	"REAL_TIME": 2,
-}
-
-func (x Access) String() string {
-	return proto.EnumName(Access_name, int32(x))
-}
-
-func (Access) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_96b320e8a67781f3, []int{0}
-}
-
-type ChildNode struct {
-	Key                  string   `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ChildNode) Reset()         { *m = ChildNode{} }
-func (m *ChildNode) String() string { return proto.CompactTextString(m) }
-func (*ChildNode) ProtoMessage()    {}
-func (*ChildNode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_96b320e8a67781f3, []int{0}
-}
-
-func (m *ChildNode) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ChildNode.Unmarshal(m, b)
-}
-func (m *ChildNode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ChildNode.Marshal(b, m, deterministic)
-}
-func (m *ChildNode) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ChildNode.Merge(m, src)
-}
-func (m *ChildNode) XXX_Size() int {
-	return xxx_messageInfo_ChildNode.Size(m)
-}
-func (m *ChildNode) XXX_DiscardUnknown() {
-	xxx_messageInfo_ChildNode.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ChildNode proto.InternalMessageInfo
-
-func (m *ChildNode) GetKey() string {
-	if m != nil {
-		return m.Key
-	}
-	return ""
-}
-
-var E_ChildNode = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.FieldOptions)(nil),
-	ExtensionType: (*ChildNode)(nil),
-	Field:         7761772,
-	Name:          "voltha.child_node",
-	Tag:           "bytes,7761772,opt,name=child_node",
-	Filename:      "voltha_protos/meta.proto",
-}
-
-var E_Access = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.FieldOptions)(nil),
-	ExtensionType: (*Access)(nil),
-	Field:         7761773,
-	Name:          "voltha.access",
-	Tag:           "varint,7761773,opt,name=access,enum=voltha.Access",
-	Filename:      "voltha_protos/meta.proto",
-}
-
-func init() {
-	proto.RegisterEnum("voltha.Access", Access_name, Access_value)
-	proto.RegisterType((*ChildNode)(nil), "voltha.ChildNode")
-	proto.RegisterExtension(E_ChildNode)
-	proto.RegisterExtension(E_Access)
-}
-
-func init() { proto.RegisterFile("voltha_protos/meta.proto", fileDescriptor_96b320e8a67781f3) }
-
-var fileDescriptor_96b320e8a67781f3 = []byte{
-	// 281 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0x41, 0x4b, 0x84, 0x40,
-	0x14, 0xc7, 0xb3, 0x05, 0xc1, 0x17, 0x2d, 0xe6, 0x69, 0x59, 0x58, 0x90, 0x4e, 0x4b, 0xd0, 0x0c,
-	0x58, 0xa7, 0xbd, 0x6d, 0xdb, 0x6e, 0x2d, 0x98, 0x82, 0x74, 0xa9, 0x8b, 0xe8, 0x38, 0xa9, 0xa4,
-	0x3e, 0x71, 0x66, 0x17, 0xfa, 0xa8, 0x5d, 0xfa, 0x04, 0xf5, 0x1d, 0x42, 0xc7, 0xe9, 0xda, 0xed,
-	0xff, 0xde, 0xfc, 0xe7, 0xc7, 0x8f, 0x07, 0xb3, 0x23, 0x56, 0xb2, 0x48, 0xe2, 0xb6, 0x43, 0x89,
-	0x82, 0xd6, 0x5c, 0x26, 0x64, 0xc8, 0x8e, 0xa9, 0x5e, 0xe6, 0x6e, 0x8e, 0x98, 0x57, 0x9c, 0x0e,
-	0xdb, 0xf4, 0xf0, 0x46, 0x33, 0x2e, 0x58, 0x57, 0xb6, 0x12, 0x3b, 0xd5, 0xbc, 0x5c, 0x80, 0xb5,
-	0x29, 0xca, 0x2a, 0x0b, 0x30, 0xe3, 0x8e, 0x0d, 0x93, 0x77, 0xfe, 0x31, 0x33, 0x5c, 0x63, 0x69,
-	0x45, 0x7d, 0xbc, 0xf2, 0xc0, 0x5c, 0x33, 0xc6, 0x85, 0x70, 0x00, 0xcc, 0x4d, 0x18, 0xec, 0xf6,
-	0x0f, 0xf6, 0x89, 0x73, 0x0e, 0x56, 0xb4, 0x5d, 0xdf, 0xc7, 0x61, 0xe0, 0xbf, 0xd8, 0xc6, 0x38,
-	0xfa, 0xf1, 0xf3, 0xfe, 0x69, 0x6b, 0x9f, 0xae, 0x22, 0x00, 0xd6, 0x23, 0xe3, 0xa6, 0x67, 0x2e,
-	0x88, 0x72, 0x20, 0xda, 0x81, 0xec, 0x4a, 0x5e, 0x65, 0x61, 0x2b, 0x4b, 0x6c, 0xc4, 0xec, 0xfb,
-	0xeb, 0x73, 0xe2, 0x1a, 0xcb, 0x33, 0xef, 0x82, 0x28, 0x67, 0xf2, 0xa7, 0x13, 0x59, 0x4c, 0xc7,
-	0xd5, 0x23, 0x98, 0x89, 0xf2, 0xf8, 0x87, 0xf7, 0xa3, 0x78, 0x53, 0x6f, 0xaa, 0x79, 0xca, 0x3f,
-	0x1a, 0xff, 0xdf, 0xf9, 0x30, 0xc7, 0x2e, 0x27, 0xd8, 0xf2, 0x86, 0x61, 0x97, 0xe9, 0x16, 0xc3,
-	0xba, 0xc6, 0xe6, 0x95, 0xe4, 0xa5, 0x2c, 0x0e, 0x69, 0x3f, 0x52, 0x5d, 0xa1, 0xaa, 0x72, 0x3d,
-	0x9e, 0xf9, 0x78, 0x4b, 0x73, 0xa4, 0xaa, 0x9f, 0x9a, 0xc3, 0xf2, 0xe6, 0x37, 0x00, 0x00, 0xff,
-	0xff, 0x0e, 0xea, 0x69, 0xf5, 0x8b, 0x01, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/ext/config/ext_config.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/ext/config/ext_config.pb.go
deleted file mode 100644
index b68a7e4..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/ext/config/ext_config.pb.go
+++ /dev/null
@@ -1,523 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/ext_config.proto
-
-package config
-
-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
-
-type OnuItuPonAlarm_AlarmID int32
-
-const (
-	OnuItuPonAlarm_RDI_ERRORS OnuItuPonAlarm_AlarmID = 0
-)
-
-var OnuItuPonAlarm_AlarmID_name = map[int32]string{
-	0: "RDI_ERRORS",
-}
-
-var OnuItuPonAlarm_AlarmID_value = map[string]int32{
-	"RDI_ERRORS": 0,
-}
-
-func (x OnuItuPonAlarm_AlarmID) String() string {
-	return proto.EnumName(OnuItuPonAlarm_AlarmID_name, int32(x))
-}
-
-func (OnuItuPonAlarm_AlarmID) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 0}
-}
-
-type OnuItuPonAlarm_AlarmReportingCondition int32
-
-const (
-	OnuItuPonAlarm_RATE_THRESHOLD  OnuItuPonAlarm_AlarmReportingCondition = 0
-	OnuItuPonAlarm_RATE_RANGE      OnuItuPonAlarm_AlarmReportingCondition = 1
-	OnuItuPonAlarm_VALUE_THRESHOLD OnuItuPonAlarm_AlarmReportingCondition = 2
-)
-
-var OnuItuPonAlarm_AlarmReportingCondition_name = map[int32]string{
-	0: "RATE_THRESHOLD",
-	1: "RATE_RANGE",
-	2: "VALUE_THRESHOLD",
-}
-
-var OnuItuPonAlarm_AlarmReportingCondition_value = map[string]int32{
-	"RATE_THRESHOLD":  0,
-	"RATE_RANGE":      1,
-	"VALUE_THRESHOLD": 2,
-}
-
-func (x OnuItuPonAlarm_AlarmReportingCondition) String() string {
-	return proto.EnumName(OnuItuPonAlarm_AlarmReportingCondition_name, int32(x))
-}
-
-func (OnuItuPonAlarm_AlarmReportingCondition) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 1}
-}
-
-type AlarmConfig struct {
-	// Types that are valid to be assigned to Config:
-	//	*AlarmConfig_OnuItuPonAlarmConfig
-	Config               isAlarmConfig_Config `protobuf_oneof:"config"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *AlarmConfig) Reset()         { *m = AlarmConfig{} }
-func (m *AlarmConfig) String() string { return proto.CompactTextString(m) }
-func (*AlarmConfig) ProtoMessage()    {}
-func (*AlarmConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{0}
-}
-
-func (m *AlarmConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmConfig.Unmarshal(m, b)
-}
-func (m *AlarmConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmConfig.Marshal(b, m, deterministic)
-}
-func (m *AlarmConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmConfig.Merge(m, src)
-}
-func (m *AlarmConfig) XXX_Size() int {
-	return xxx_messageInfo_AlarmConfig.Size(m)
-}
-func (m *AlarmConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmConfig proto.InternalMessageInfo
-
-type isAlarmConfig_Config interface {
-	isAlarmConfig_Config()
-}
-
-type AlarmConfig_OnuItuPonAlarmConfig struct {
-	OnuItuPonAlarmConfig *OnuItuPonAlarm `protobuf:"bytes,1,opt,name=onu_itu_pon_alarm_config,json=onuItuPonAlarmConfig,proto3,oneof"`
-}
-
-func (*AlarmConfig_OnuItuPonAlarmConfig) isAlarmConfig_Config() {}
-
-func (m *AlarmConfig) GetConfig() isAlarmConfig_Config {
-	if m != nil {
-		return m.Config
-	}
-	return nil
-}
-
-func (m *AlarmConfig) GetOnuItuPonAlarmConfig() *OnuItuPonAlarm {
-	if x, ok := m.GetConfig().(*AlarmConfig_OnuItuPonAlarmConfig); ok {
-		return x.OnuItuPonAlarmConfig
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*AlarmConfig) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*AlarmConfig_OnuItuPonAlarmConfig)(nil),
-	}
-}
-
-type OnuItuPonAlarm struct {
-	PonNi                   uint32                                 `protobuf:"fixed32,1,opt,name=pon_ni,json=ponNi,proto3" json:"pon_ni,omitempty"`
-	OnuId                   uint32                                 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	AlarmId                 OnuItuPonAlarm_AlarmID                 `protobuf:"varint,3,opt,name=alarm_id,json=alarmId,proto3,enum=config.OnuItuPonAlarm_AlarmID" json:"alarm_id,omitempty"`
-	AlarmReportingCondition OnuItuPonAlarm_AlarmReportingCondition `protobuf:"varint,4,opt,name=alarm_reporting_condition,json=alarmReportingCondition,proto3,enum=config.OnuItuPonAlarm_AlarmReportingCondition" json:"alarm_reporting_condition,omitempty"`
-	// Types that are valid to be assigned to Config:
-	//	*OnuItuPonAlarm_RateThresholdConfig_
-	//	*OnuItuPonAlarm_RateRangeConfig_
-	//	*OnuItuPonAlarm_ValueThresholdConfig_
-	Config               isOnuItuPonAlarm_Config `protobuf_oneof:"config"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
-}
-
-func (m *OnuItuPonAlarm) Reset()         { *m = OnuItuPonAlarm{} }
-func (m *OnuItuPonAlarm) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonAlarm) ProtoMessage()    {}
-func (*OnuItuPonAlarm) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1}
-}
-
-func (m *OnuItuPonAlarm) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonAlarm.Unmarshal(m, b)
-}
-func (m *OnuItuPonAlarm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonAlarm.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonAlarm) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonAlarm.Merge(m, src)
-}
-func (m *OnuItuPonAlarm) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonAlarm.Size(m)
-}
-func (m *OnuItuPonAlarm) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonAlarm.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonAlarm proto.InternalMessageInfo
-
-func (m *OnuItuPonAlarm) GetPonNi() uint32 {
-	if m != nil {
-		return m.PonNi
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm) GetAlarmId() OnuItuPonAlarm_AlarmID {
-	if m != nil {
-		return m.AlarmId
-	}
-	return OnuItuPonAlarm_RDI_ERRORS
-}
-
-func (m *OnuItuPonAlarm) GetAlarmReportingCondition() OnuItuPonAlarm_AlarmReportingCondition {
-	if m != nil {
-		return m.AlarmReportingCondition
-	}
-	return OnuItuPonAlarm_RATE_THRESHOLD
-}
-
-type isOnuItuPonAlarm_Config interface {
-	isOnuItuPonAlarm_Config()
-}
-
-type OnuItuPonAlarm_RateThresholdConfig_ struct {
-	RateThresholdConfig *OnuItuPonAlarm_RateThresholdConfig `protobuf:"bytes,5,opt,name=rate_threshold_config,json=rateThresholdConfig,proto3,oneof"`
-}
-
-type OnuItuPonAlarm_RateRangeConfig_ struct {
-	RateRangeConfig *OnuItuPonAlarm_RateRangeConfig `protobuf:"bytes,6,opt,name=rate_range_config,json=rateRangeConfig,proto3,oneof"`
-}
-
-type OnuItuPonAlarm_ValueThresholdConfig_ struct {
-	ValueThresholdConfig *OnuItuPonAlarm_ValueThresholdConfig `protobuf:"bytes,7,opt,name=value_threshold_config,json=valueThresholdConfig,proto3,oneof"`
-}
-
-func (*OnuItuPonAlarm_RateThresholdConfig_) isOnuItuPonAlarm_Config() {}
-
-func (*OnuItuPonAlarm_RateRangeConfig_) isOnuItuPonAlarm_Config() {}
-
-func (*OnuItuPonAlarm_ValueThresholdConfig_) isOnuItuPonAlarm_Config() {}
-
-func (m *OnuItuPonAlarm) GetConfig() isOnuItuPonAlarm_Config {
-	if m != nil {
-		return m.Config
-	}
-	return nil
-}
-
-func (m *OnuItuPonAlarm) GetRateThresholdConfig() *OnuItuPonAlarm_RateThresholdConfig {
-	if x, ok := m.GetConfig().(*OnuItuPonAlarm_RateThresholdConfig_); ok {
-		return x.RateThresholdConfig
-	}
-	return nil
-}
-
-func (m *OnuItuPonAlarm) GetRateRangeConfig() *OnuItuPonAlarm_RateRangeConfig {
-	if x, ok := m.GetConfig().(*OnuItuPonAlarm_RateRangeConfig_); ok {
-		return x.RateRangeConfig
-	}
-	return nil
-}
-
-func (m *OnuItuPonAlarm) GetValueThresholdConfig() *OnuItuPonAlarm_ValueThresholdConfig {
-	if x, ok := m.GetConfig().(*OnuItuPonAlarm_ValueThresholdConfig_); ok {
-		return x.ValueThresholdConfig
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OnuItuPonAlarm) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OnuItuPonAlarm_RateThresholdConfig_)(nil),
-		(*OnuItuPonAlarm_RateRangeConfig_)(nil),
-		(*OnuItuPonAlarm_ValueThresholdConfig_)(nil),
-	}
-}
-
-type OnuItuPonAlarm_SoakTime struct {
-	ActiveSoakTime       uint32   `protobuf:"fixed32,1,opt,name=active_soak_time,json=activeSoakTime,proto3" json:"active_soak_time,omitempty"`
-	ClearSoakTime        uint32   `protobuf:"fixed32,2,opt,name=clear_soak_time,json=clearSoakTime,proto3" json:"clear_soak_time,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuItuPonAlarm_SoakTime) Reset()         { *m = OnuItuPonAlarm_SoakTime{} }
-func (m *OnuItuPonAlarm_SoakTime) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonAlarm_SoakTime) ProtoMessage()    {}
-func (*OnuItuPonAlarm_SoakTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 0}
-}
-
-func (m *OnuItuPonAlarm_SoakTime) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Unmarshal(m, b)
-}
-func (m *OnuItuPonAlarm_SoakTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonAlarm_SoakTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonAlarm_SoakTime.Merge(m, src)
-}
-func (m *OnuItuPonAlarm_SoakTime) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Size(m)
-}
-func (m *OnuItuPonAlarm_SoakTime) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonAlarm_SoakTime.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonAlarm_SoakTime proto.InternalMessageInfo
-
-func (m *OnuItuPonAlarm_SoakTime) GetActiveSoakTime() uint32 {
-	if m != nil {
-		return m.ActiveSoakTime
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_SoakTime) GetClearSoakTime() uint32 {
-	if m != nil {
-		return m.ClearSoakTime
-	}
-	return 0
-}
-
-type OnuItuPonAlarm_RateThresholdConfig struct {
-	RateThresholdRising  uint64                   `protobuf:"fixed64,1,opt,name=rate_threshold_rising,json=rateThresholdRising,proto3" json:"rate_threshold_rising,omitempty"`
-	RateThresholdFalling uint64                   `protobuf:"fixed64,2,opt,name=rate_threshold_falling,json=rateThresholdFalling,proto3" json:"rate_threshold_falling,omitempty"`
-	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,3,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
-}
-
-func (m *OnuItuPonAlarm_RateThresholdConfig) Reset()         { *m = OnuItuPonAlarm_RateThresholdConfig{} }
-func (m *OnuItuPonAlarm_RateThresholdConfig) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonAlarm_RateThresholdConfig) ProtoMessage()    {}
-func (*OnuItuPonAlarm_RateThresholdConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 1}
-}
-
-func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Unmarshal(m, b)
-}
-func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Merge(m, src)
-}
-func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Size(m)
-}
-func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig proto.InternalMessageInfo
-
-func (m *OnuItuPonAlarm_RateThresholdConfig) GetRateThresholdRising() uint64 {
-	if m != nil {
-		return m.RateThresholdRising
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_RateThresholdConfig) GetRateThresholdFalling() uint64 {
-	if m != nil {
-		return m.RateThresholdFalling
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_RateThresholdConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
-	if m != nil {
-		return m.SoakTime
-	}
-	return nil
-}
-
-type OnuItuPonAlarm_RateRangeConfig struct {
-	RateRangeLower       uint64                   `protobuf:"fixed64,1,opt,name=rate_range_lower,json=rateRangeLower,proto3" json:"rate_range_lower,omitempty"`
-	RateRangeUpper       uint64                   `protobuf:"fixed64,2,opt,name=rate_range_upper,json=rateRangeUpper,proto3" json:"rate_range_upper,omitempty"`
-	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,3,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
-}
-
-func (m *OnuItuPonAlarm_RateRangeConfig) Reset()         { *m = OnuItuPonAlarm_RateRangeConfig{} }
-func (m *OnuItuPonAlarm_RateRangeConfig) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonAlarm_RateRangeConfig) ProtoMessage()    {}
-func (*OnuItuPonAlarm_RateRangeConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 2}
-}
-
-func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Unmarshal(m, b)
-}
-func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Merge(m, src)
-}
-func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Size(m)
-}
-func (m *OnuItuPonAlarm_RateRangeConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig proto.InternalMessageInfo
-
-func (m *OnuItuPonAlarm_RateRangeConfig) GetRateRangeLower() uint64 {
-	if m != nil {
-		return m.RateRangeLower
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_RateRangeConfig) GetRateRangeUpper() uint64 {
-	if m != nil {
-		return m.RateRangeUpper
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_RateRangeConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
-	if m != nil {
-		return m.SoakTime
-	}
-	return nil
-}
-
-type OnuItuPonAlarm_ValueThresholdConfig struct {
-	ThresholdLimit       uint64                   `protobuf:"fixed64,1,opt,name=threshold_limit,json=thresholdLimit,proto3" json:"threshold_limit,omitempty"`
-	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,2,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
-}
-
-func (m *OnuItuPonAlarm_ValueThresholdConfig) Reset()         { *m = OnuItuPonAlarm_ValueThresholdConfig{} }
-func (m *OnuItuPonAlarm_ValueThresholdConfig) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonAlarm_ValueThresholdConfig) ProtoMessage()    {}
-func (*OnuItuPonAlarm_ValueThresholdConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 3}
-}
-
-func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Unmarshal(m, b)
-}
-func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Merge(m, src)
-}
-func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Size(m)
-}
-func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig proto.InternalMessageInfo
-
-func (m *OnuItuPonAlarm_ValueThresholdConfig) GetThresholdLimit() uint64 {
-	if m != nil {
-		return m.ThresholdLimit
-	}
-	return 0
-}
-
-func (m *OnuItuPonAlarm_ValueThresholdConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
-	if m != nil {
-		return m.SoakTime
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("config.OnuItuPonAlarm_AlarmID", OnuItuPonAlarm_AlarmID_name, OnuItuPonAlarm_AlarmID_value)
-	proto.RegisterEnum("config.OnuItuPonAlarm_AlarmReportingCondition", OnuItuPonAlarm_AlarmReportingCondition_name, OnuItuPonAlarm_AlarmReportingCondition_value)
-	proto.RegisterType((*AlarmConfig)(nil), "config.AlarmConfig")
-	proto.RegisterType((*OnuItuPonAlarm)(nil), "config.OnuItuPonAlarm")
-	proto.RegisterType((*OnuItuPonAlarm_SoakTime)(nil), "config.OnuItuPonAlarm.SoakTime")
-	proto.RegisterType((*OnuItuPonAlarm_RateThresholdConfig)(nil), "config.OnuItuPonAlarm.RateThresholdConfig")
-	proto.RegisterType((*OnuItuPonAlarm_RateRangeConfig)(nil), "config.OnuItuPonAlarm.RateRangeConfig")
-	proto.RegisterType((*OnuItuPonAlarm_ValueThresholdConfig)(nil), "config.OnuItuPonAlarm.ValueThresholdConfig")
-}
-
-func init() { proto.RegisterFile("voltha_protos/ext_config.proto", fileDescriptor_fb43b44b7fa3aba9) }
-
-var fileDescriptor_fb43b44b7fa3aba9 = []byte{
-	// 609 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xd1, 0x4e, 0x1a, 0x41,
-	0x14, 0x05, 0xac, 0x80, 0xd7, 0x14, 0xe8, 0x80, 0x8a, 0x3e, 0x58, 0xe3, 0x83, 0x35, 0x6d, 0xba,
-	0x24, 0xd4, 0x97, 0x26, 0x7d, 0x41, 0xa5, 0x85, 0x84, 0xa8, 0x19, 0xd1, 0x87, 0xa6, 0xc9, 0x76,
-	0x64, 0xc7, 0x65, 0xea, 0x32, 0xb3, 0x19, 0x66, 0x69, 0x5f, 0xfa, 0x35, 0xfd, 0x8e, 0xf6, 0xdb,
-	0x9a, 0x99, 0xd9, 0x05, 0x81, 0xc5, 0xa4, 0xe9, 0x0b, 0xc9, 0x9e, 0x7b, 0xcf, 0xb9, 0x87, 0x73,
-	0x67, 0x06, 0xf6, 0x27, 0x22, 0x50, 0x43, 0xe2, 0x86, 0x52, 0x28, 0x31, 0x6e, 0xd0, 0x1f, 0xca,
-	0x1d, 0x08, 0x7e, 0xcf, 0x7c, 0xc7, 0x20, 0x28, 0x6f, 0xbf, 0x0e, 0x19, 0x6c, 0xb6, 0x02, 0x22,
-	0x47, 0x67, 0xe6, 0x13, 0x5d, 0x41, 0x5d, 0xf0, 0xc8, 0x65, 0x2a, 0x72, 0x43, 0xc1, 0x5d, 0xa2,
-	0x4b, 0x31, 0xb1, 0x9e, 0x3d, 0xc8, 0x1e, 0x6f, 0x36, 0xb7, 0x9d, 0x58, 0xe7, 0x92, 0x47, 0x5d,
-	0x15, 0x5d, 0x09, 0x6e, 0xf8, 0x9d, 0x0c, 0xae, 0x89, 0x39, 0xc4, 0x2a, 0x9e, 0x16, 0x21, 0x19,
-	0xf5, 0x67, 0x03, 0x4a, 0xf3, 0x24, 0xb4, 0x05, 0x79, 0x3d, 0x86, 0x33, 0x23, 0x5e, 0xc0, 0xeb,
-	0xa1, 0xe0, 0x17, 0x4c, 0xc3, 0xc6, 0x85, 0x57, 0xcf, 0x59, 0x58, 0x2b, 0x7b, 0xe8, 0x3d, 0x14,
-	0xad, 0x21, 0xe6, 0xd5, 0xd7, 0x0e, 0xb2, 0xc7, 0xa5, 0xe6, 0x7e, 0xba, 0x19, 0xc7, 0xfc, 0x76,
-	0xcf, 0x71, 0xc1, 0xf4, 0x77, 0x3d, 0xf4, 0x0d, 0x76, 0x2d, 0x55, 0xd2, 0x50, 0x48, 0xc5, 0xb8,
-	0xaf, 0xff, 0x95, 0xc7, 0x14, 0x13, 0xbc, 0xfe, 0xcc, 0x68, 0x39, 0x4f, 0x69, 0xe1, 0x84, 0x76,
-	0x96, 0xb0, 0xf0, 0x0e, 0x49, 0x2f, 0xa0, 0xaf, 0xb0, 0x25, 0x89, 0xa2, 0xae, 0x1a, 0x4a, 0x3a,
-	0x1e, 0x8a, 0xc0, 0x4b, 0x02, 0x5c, 0x37, 0x01, 0xbe, 0x5e, 0x31, 0x07, 0x13, 0x45, 0xfb, 0x09,
-	0xc5, 0x86, 0xd7, 0xc9, 0xe0, 0xaa, 0x5c, 0x86, 0x51, 0x1f, 0x5e, 0x98, 0x09, 0x92, 0x70, 0x9f,
-	0x26, 0xea, 0x79, 0xa3, 0x7e, 0xf4, 0x84, 0x3a, 0xd6, 0xed, 0x53, 0xe5, 0xb2, 0x9c, 0x87, 0xd0,
-	0x00, 0xb6, 0x27, 0x24, 0x88, 0x52, 0x8c, 0x17, 0x8c, 0xf4, 0x9b, 0x15, 0xd2, 0xb7, 0x9a, 0xb4,
-	0xec, 0xbc, 0x36, 0x49, 0xc1, 0xf7, 0xbe, 0x40, 0xf1, 0x5a, 0x90, 0x87, 0x3e, 0x1b, 0x51, 0x74,
-	0x0c, 0x15, 0x32, 0x50, 0x6c, 0x42, 0xdd, 0xb1, 0x20, 0x0f, 0xae, 0x62, 0x23, 0x1a, 0x9f, 0x83,
-	0x92, 0xc5, 0xa7, 0x9d, 0x47, 0x50, 0x1e, 0x04, 0x94, 0xc8, 0x47, 0x8d, 0xf6, 0x64, 0x3c, 0x37,
-	0x70, 0xd2, 0xb7, 0xf7, 0x3b, 0x0b, 0xd5, 0x94, 0x1c, 0x51, 0x73, 0x69, 0x25, 0x92, 0x8d, 0x19,
-	0xb7, 0x67, 0x3a, 0xbf, 0x10, 0x32, 0x36, 0x25, 0x74, 0x02, 0xdb, 0x0b, 0x9c, 0x7b, 0x12, 0x04,
-	0x9a, 0x94, 0x33, 0xa4, 0xda, 0x1c, 0xe9, 0xa3, 0xad, 0xa1, 0x0f, 0xb0, 0x31, 0xf3, 0xb8, 0x66,
-	0x72, 0x7b, 0xb9, 0x22, 0xb7, 0xc4, 0x35, 0x2e, 0x8e, 0x13, 0xff, 0xbf, 0xb2, 0x50, 0x5e, 0xd8,
-	0x94, 0x4e, 0xe9, 0xd1, 0xb2, 0x03, 0xf1, 0x9d, 0xca, 0xd8, 0x76, 0x69, 0xba, 0xc1, 0x9e, 0x46,
-	0x17, 0x3a, 0xa3, 0x30, 0xa4, 0x32, 0xf6, 0x3a, 0xeb, 0xbc, 0xd1, 0xe8, 0x7f, 0xba, 0xfc, 0x09,
-	0xb5, 0xb4, 0x9d, 0xa3, 0x57, 0x50, 0x9e, 0x85, 0x15, 0xb0, 0x11, 0x53, 0x89, 0xd1, 0x29, 0xdc,
-	0xd3, 0xe8, 0xfc, 0xf8, 0xdc, 0x3f, 0x8e, 0x3f, 0xdc, 0x85, 0x42, 0x7c, 0xbf, 0x51, 0x09, 0x00,
-	0x9f, 0x77, 0xdd, 0x36, 0xc6, 0x97, 0xf8, 0xba, 0x92, 0x39, 0xc4, 0xb0, 0xb3, 0xe2, 0xba, 0x22,
-	0x04, 0x25, 0xdc, 0xea, 0xb7, 0xdd, 0x7e, 0x07, 0xb7, 0xaf, 0x3b, 0x97, 0xbd, 0xf3, 0x4a, 0xc6,
-	0xd0, 0x35, 0x86, 0x5b, 0x17, 0x9f, 0xda, 0x95, 0x2c, 0xaa, 0x42, 0xf9, 0xb6, 0xd5, 0xbb, 0x79,
-	0xdc, 0x94, 0x9b, 0x3d, 0x60, 0xa7, 0x5d, 0xa8, 0x0a, 0xe9, 0x3b, 0x22, 0xa4, 0x7c, 0x20, 0xa4,
-	0xe7, 0xd8, 0x27, 0xf6, 0x73, 0xd3, 0x67, 0x6a, 0x18, 0xdd, 0x39, 0x03, 0x31, 0x6a, 0x24, 0xb5,
-	0x86, 0xad, 0xbd, 0x8d, 0x9f, 0xdf, 0xc9, 0x49, 0xc3, 0x17, 0xfa, 0x11, 0x6e, 0x58, 0xa9, 0xbb,
-	0xbc, 0x29, 0xbc, 0xfb, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xe2, 0x77, 0x17, 0x08, 0xa7, 0x05, 0x00,
-	0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/extension/extensions.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/extension/extensions.pb.go
deleted file mode 100644
index 92bc924..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/extension/extensions.pb.go
+++ /dev/null
@@ -1,3313 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/extensions.proto
-
-package extension
-
-import (
-	context "context"
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	empty "github.com/golang/protobuf/ptypes/empty"
-	config "github.com/opencord/voltha-protos/v4/go/ext/config"
-	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
-
-// AlarmConfig from public import voltha_protos/ext_config.proto
-type AlarmConfig = config.AlarmConfig
-type AlarmConfig_OnuItuPonAlarmConfig = config.AlarmConfig_OnuItuPonAlarmConfig
-
-// OnuItuPonAlarm from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm = config.OnuItuPonAlarm
-type OnuItuPonAlarm_RateThresholdConfig_ = config.OnuItuPonAlarm_RateThresholdConfig_
-type OnuItuPonAlarm_RateRangeConfig_ = config.OnuItuPonAlarm_RateRangeConfig_
-type OnuItuPonAlarm_ValueThresholdConfig_ = config.OnuItuPonAlarm_ValueThresholdConfig_
-
-// OnuItuPonAlarm_SoakTime from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_SoakTime = config.OnuItuPonAlarm_SoakTime
-
-// OnuItuPonAlarm_RateThresholdConfig from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_RateThresholdConfig = config.OnuItuPonAlarm_RateThresholdConfig
-
-// OnuItuPonAlarm_RateRangeConfig from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_RateRangeConfig = config.OnuItuPonAlarm_RateRangeConfig
-
-// OnuItuPonAlarm_ValueThresholdConfig from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_ValueThresholdConfig = config.OnuItuPonAlarm_ValueThresholdConfig
-
-// OnuItuPonAlarm_AlarmID from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_AlarmID = config.OnuItuPonAlarm_AlarmID
-
-var OnuItuPonAlarm_AlarmID_name = config.OnuItuPonAlarm_AlarmID_name
-var OnuItuPonAlarm_AlarmID_value = config.OnuItuPonAlarm_AlarmID_value
-
-const OnuItuPonAlarm_RDI_ERRORS = OnuItuPonAlarm_AlarmID(config.OnuItuPonAlarm_RDI_ERRORS)
-
-// OnuItuPonAlarm_AlarmReportingCondition from public import voltha_protos/ext_config.proto
-type OnuItuPonAlarm_AlarmReportingCondition = config.OnuItuPonAlarm_AlarmReportingCondition
-
-var OnuItuPonAlarm_AlarmReportingCondition_name = config.OnuItuPonAlarm_AlarmReportingCondition_name
-var OnuItuPonAlarm_AlarmReportingCondition_value = config.OnuItuPonAlarm_AlarmReportingCondition_value
-
-const OnuItuPonAlarm_RATE_THRESHOLD = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_RATE_THRESHOLD)
-const OnuItuPonAlarm_RATE_RANGE = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_RATE_RANGE)
-const OnuItuPonAlarm_VALUE_THRESHOLD = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_VALUE_THRESHOLD)
-
-type GetOnuUniInfoResponse_ConfigurationInd int32
-
-const (
-	GetOnuUniInfoResponse_UNKOWN               GetOnuUniInfoResponse_ConfigurationInd = 0
-	GetOnuUniInfoResponse_TEN_BASE_T_FDX       GetOnuUniInfoResponse_ConfigurationInd = 1
-	GetOnuUniInfoResponse_HUNDRED_BASE_T_FDX   GetOnuUniInfoResponse_ConfigurationInd = 2
-	GetOnuUniInfoResponse_GIGABIT_ETHERNET_FDX GetOnuUniInfoResponse_ConfigurationInd = 3
-	GetOnuUniInfoResponse_TEN_G_ETHERNET_FDX   GetOnuUniInfoResponse_ConfigurationInd = 4
-	GetOnuUniInfoResponse_TEN_BASE_T_HDX       GetOnuUniInfoResponse_ConfigurationInd = 5
-	GetOnuUniInfoResponse_HUNDRED_BASE_T_HDX   GetOnuUniInfoResponse_ConfigurationInd = 6
-	GetOnuUniInfoResponse_GIGABIT_ETHERNET_HDX GetOnuUniInfoResponse_ConfigurationInd = 7
-)
-
-var GetOnuUniInfoResponse_ConfigurationInd_name = map[int32]string{
-	0: "UNKOWN",
-	1: "TEN_BASE_T_FDX",
-	2: "HUNDRED_BASE_T_FDX",
-	3: "GIGABIT_ETHERNET_FDX",
-	4: "TEN_G_ETHERNET_FDX",
-	5: "TEN_BASE_T_HDX",
-	6: "HUNDRED_BASE_T_HDX",
-	7: "GIGABIT_ETHERNET_HDX",
-}
-
-var GetOnuUniInfoResponse_ConfigurationInd_value = map[string]int32{
-	"UNKOWN":               0,
-	"TEN_BASE_T_FDX":       1,
-	"HUNDRED_BASE_T_FDX":   2,
-	"GIGABIT_ETHERNET_FDX": 3,
-	"TEN_G_ETHERNET_FDX":   4,
-	"TEN_BASE_T_HDX":       5,
-	"HUNDRED_BASE_T_HDX":   6,
-	"GIGABIT_ETHERNET_HDX": 7,
-}
-
-func (x GetOnuUniInfoResponse_ConfigurationInd) String() string {
-	return proto.EnumName(GetOnuUniInfoResponse_ConfigurationInd_name, int32(x))
-}
-
-func (GetOnuUniInfoResponse_ConfigurationInd) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{3, 0}
-}
-
-type GetOnuUniInfoResponse_AdministrativeState int32
-
-const (
-	GetOnuUniInfoResponse_ADMSTATE_UNDEFINED GetOnuUniInfoResponse_AdministrativeState = 0
-	GetOnuUniInfoResponse_LOCKED             GetOnuUniInfoResponse_AdministrativeState = 1
-	GetOnuUniInfoResponse_UNLOCKED           GetOnuUniInfoResponse_AdministrativeState = 2
-)
-
-var GetOnuUniInfoResponse_AdministrativeState_name = map[int32]string{
-	0: "ADMSTATE_UNDEFINED",
-	1: "LOCKED",
-	2: "UNLOCKED",
-}
-
-var GetOnuUniInfoResponse_AdministrativeState_value = map[string]int32{
-	"ADMSTATE_UNDEFINED": 0,
-	"LOCKED":             1,
-	"UNLOCKED":           2,
-}
-
-func (x GetOnuUniInfoResponse_AdministrativeState) String() string {
-	return proto.EnumName(GetOnuUniInfoResponse_AdministrativeState_name, int32(x))
-}
-
-func (GetOnuUniInfoResponse_AdministrativeState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{3, 1}
-}
-
-type GetOnuUniInfoResponse_OperationalState int32
-
-const (
-	GetOnuUniInfoResponse_OPERSTATE_UNDEFINED GetOnuUniInfoResponse_OperationalState = 0
-	GetOnuUniInfoResponse_ENABLED             GetOnuUniInfoResponse_OperationalState = 1
-	GetOnuUniInfoResponse_DISABLED            GetOnuUniInfoResponse_OperationalState = 2
-)
-
-var GetOnuUniInfoResponse_OperationalState_name = map[int32]string{
-	0: "OPERSTATE_UNDEFINED",
-	1: "ENABLED",
-	2: "DISABLED",
-}
-
-var GetOnuUniInfoResponse_OperationalState_value = map[string]int32{
-	"OPERSTATE_UNDEFINED": 0,
-	"ENABLED":             1,
-	"DISABLED":            2,
-}
-
-func (x GetOnuUniInfoResponse_OperationalState) String() string {
-	return proto.EnumName(GetOnuUniInfoResponse_OperationalState_name, int32(x))
-}
-
-func (GetOnuUniInfoResponse_OperationalState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{3, 2}
-}
-
-type GetOltPortCounters_PortType int32
-
-const (
-	GetOltPortCounters_Port_UNKNOWN      GetOltPortCounters_PortType = 0
-	GetOltPortCounters_Port_ETHERNET_NNI GetOltPortCounters_PortType = 1
-	GetOltPortCounters_Port_PON_OLT      GetOltPortCounters_PortType = 2
-)
-
-var GetOltPortCounters_PortType_name = map[int32]string{
-	0: "Port_UNKNOWN",
-	1: "Port_ETHERNET_NNI",
-	2: "Port_PON_OLT",
-}
-
-var GetOltPortCounters_PortType_value = map[string]int32{
-	"Port_UNKNOWN":      0,
-	"Port_ETHERNET_NNI": 1,
-	"Port_PON_OLT":      2,
-}
-
-func (x GetOltPortCounters_PortType) String() string {
-	return proto.EnumName(GetOltPortCounters_PortType_name, int32(x))
-}
-
-func (GetOltPortCounters_PortType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{4, 0}
-}
-
-type GetOnuEthernetBridgePortHistory_Direction int32
-
-const (
-	GetOnuEthernetBridgePortHistory_UNDEFINED  GetOnuEthernetBridgePortHistory_Direction = 0
-	GetOnuEthernetBridgePortHistory_UPSTREAM   GetOnuEthernetBridgePortHistory_Direction = 1
-	GetOnuEthernetBridgePortHistory_DOWNSTREAM GetOnuEthernetBridgePortHistory_Direction = 2
-)
-
-var GetOnuEthernetBridgePortHistory_Direction_name = map[int32]string{
-	0: "UNDEFINED",
-	1: "UPSTREAM",
-	2: "DOWNSTREAM",
-}
-
-var GetOnuEthernetBridgePortHistory_Direction_value = map[string]int32{
-	"UNDEFINED":  0,
-	"UPSTREAM":   1,
-	"DOWNSTREAM": 2,
-}
-
-func (x GetOnuEthernetBridgePortHistory_Direction) String() string {
-	return proto.EnumName(GetOnuEthernetBridgePortHistory_Direction_name, int32(x))
-}
-
-func (GetOnuEthernetBridgePortHistory_Direction) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{8, 0}
-}
-
-type GetValueResponse_Status int32
-
-const (
-	GetValueResponse_STATUS_UNDEFINED GetValueResponse_Status = 0
-	GetValueResponse_OK               GetValueResponse_Status = 1
-	GetValueResponse_ERROR            GetValueResponse_Status = 2
-)
-
-var GetValueResponse_Status_name = map[int32]string{
-	0: "STATUS_UNDEFINED",
-	1: "OK",
-	2: "ERROR",
-}
-
-var GetValueResponse_Status_value = map[string]int32{
-	"STATUS_UNDEFINED": 0,
-	"OK":               1,
-	"ERROR":            2,
-}
-
-func (x GetValueResponse_Status) String() string {
-	return proto.EnumName(GetValueResponse_Status_name, int32(x))
-}
-
-func (GetValueResponse_Status) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{20, 0}
-}
-
-type GetValueResponse_ErrorReason int32
-
-const (
-	GetValueResponse_REASON_UNDEFINED  GetValueResponse_ErrorReason = 0
-	GetValueResponse_UNSUPPORTED       GetValueResponse_ErrorReason = 1
-	GetValueResponse_INVALID_DEVICE_ID GetValueResponse_ErrorReason = 2
-	GetValueResponse_INVALID_PORT_TYPE GetValueResponse_ErrorReason = 3
-	GetValueResponse_TIMEOUT           GetValueResponse_ErrorReason = 4
-	GetValueResponse_INVALID_REQ_TYPE  GetValueResponse_ErrorReason = 5
-	GetValueResponse_INTERNAL_ERROR    GetValueResponse_ErrorReason = 6
-	GetValueResponse_INVALID_DEVICE    GetValueResponse_ErrorReason = 7
-)
-
-var GetValueResponse_ErrorReason_name = map[int32]string{
-	0: "REASON_UNDEFINED",
-	1: "UNSUPPORTED",
-	2: "INVALID_DEVICE_ID",
-	3: "INVALID_PORT_TYPE",
-	4: "TIMEOUT",
-	5: "INVALID_REQ_TYPE",
-	6: "INTERNAL_ERROR",
-	7: "INVALID_DEVICE",
-}
-
-var GetValueResponse_ErrorReason_value = map[string]int32{
-	"REASON_UNDEFINED":  0,
-	"UNSUPPORTED":       1,
-	"INVALID_DEVICE_ID": 2,
-	"INVALID_PORT_TYPE": 3,
-	"TIMEOUT":           4,
-	"INVALID_REQ_TYPE":  5,
-	"INTERNAL_ERROR":    6,
-	"INVALID_DEVICE":    7,
-}
-
-func (x GetValueResponse_ErrorReason) String() string {
-	return proto.EnumName(GetValueResponse_ErrorReason_name, int32(x))
-}
-
-func (GetValueResponse_ErrorReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{20, 1}
-}
-
-type SetValueResponse_Status int32
-
-const (
-	SetValueResponse_STATUS_UNDEFINED SetValueResponse_Status = 0
-	SetValueResponse_OK               SetValueResponse_Status = 1
-	SetValueResponse_ERROR            SetValueResponse_Status = 2
-)
-
-var SetValueResponse_Status_name = map[int32]string{
-	0: "STATUS_UNDEFINED",
-	1: "OK",
-	2: "ERROR",
-}
-
-var SetValueResponse_Status_value = map[string]int32{
-	"STATUS_UNDEFINED": 0,
-	"OK":               1,
-	"ERROR":            2,
-}
-
-func (x SetValueResponse_Status) String() string {
-	return proto.EnumName(SetValueResponse_Status_name, int32(x))
-}
-
-func (SetValueResponse_Status) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{22, 0}
-}
-
-type SetValueResponse_ErrorReason int32
-
-const (
-	SetValueResponse_REASON_UNDEFINED SetValueResponse_ErrorReason = 0
-	SetValueResponse_UNSUPPORTED      SetValueResponse_ErrorReason = 1
-)
-
-var SetValueResponse_ErrorReason_name = map[int32]string{
-	0: "REASON_UNDEFINED",
-	1: "UNSUPPORTED",
-}
-
-var SetValueResponse_ErrorReason_value = map[string]int32{
-	"REASON_UNDEFINED": 0,
-	"UNSUPPORTED":      1,
-}
-
-func (x SetValueResponse_ErrorReason) String() string {
-	return proto.EnumName(SetValueResponse_ErrorReason_name, int32(x))
-}
-
-func (SetValueResponse_ErrorReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{22, 1}
-}
-
-type GetDistanceRequest struct {
-	OnuDeviceId          string   `protobuf:"bytes,1,opt,name=onuDeviceId,proto3" json:"onuDeviceId,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetDistanceRequest) Reset()         { *m = GetDistanceRequest{} }
-func (m *GetDistanceRequest) String() string { return proto.CompactTextString(m) }
-func (*GetDistanceRequest) ProtoMessage()    {}
-func (*GetDistanceRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{0}
-}
-
-func (m *GetDistanceRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetDistanceRequest.Unmarshal(m, b)
-}
-func (m *GetDistanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetDistanceRequest.Marshal(b, m, deterministic)
-}
-func (m *GetDistanceRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetDistanceRequest.Merge(m, src)
-}
-func (m *GetDistanceRequest) XXX_Size() int {
-	return xxx_messageInfo_GetDistanceRequest.Size(m)
-}
-func (m *GetDistanceRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetDistanceRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetDistanceRequest proto.InternalMessageInfo
-
-func (m *GetDistanceRequest) GetOnuDeviceId() string {
-	if m != nil {
-		return m.OnuDeviceId
-	}
-	return ""
-}
-
-type GetDistanceResponse struct {
-	Distance             uint32   `protobuf:"varint,1,opt,name=distance,proto3" json:"distance,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetDistanceResponse) Reset()         { *m = GetDistanceResponse{} }
-func (m *GetDistanceResponse) String() string { return proto.CompactTextString(m) }
-func (*GetDistanceResponse) ProtoMessage()    {}
-func (*GetDistanceResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{1}
-}
-
-func (m *GetDistanceResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetDistanceResponse.Unmarshal(m, b)
-}
-func (m *GetDistanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetDistanceResponse.Marshal(b, m, deterministic)
-}
-func (m *GetDistanceResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetDistanceResponse.Merge(m, src)
-}
-func (m *GetDistanceResponse) XXX_Size() int {
-	return xxx_messageInfo_GetDistanceResponse.Size(m)
-}
-func (m *GetDistanceResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetDistanceResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetDistanceResponse proto.InternalMessageInfo
-
-func (m *GetDistanceResponse) GetDistance() uint32 {
-	if m != nil {
-		return m.Distance
-	}
-	return 0
-}
-
-type GetOnuUniInfoRequest struct {
-	UniIndex             uint32   `protobuf:"varint,1,opt,name=uniIndex,proto3" json:"uniIndex,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetOnuUniInfoRequest) Reset()         { *m = GetOnuUniInfoRequest{} }
-func (m *GetOnuUniInfoRequest) String() string { return proto.CompactTextString(m) }
-func (*GetOnuUniInfoRequest) ProtoMessage()    {}
-func (*GetOnuUniInfoRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{2}
-}
-
-func (m *GetOnuUniInfoRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuUniInfoRequest.Unmarshal(m, b)
-}
-func (m *GetOnuUniInfoRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuUniInfoRequest.Marshal(b, m, deterministic)
-}
-func (m *GetOnuUniInfoRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuUniInfoRequest.Merge(m, src)
-}
-func (m *GetOnuUniInfoRequest) XXX_Size() int {
-	return xxx_messageInfo_GetOnuUniInfoRequest.Size(m)
-}
-func (m *GetOnuUniInfoRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuUniInfoRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuUniInfoRequest proto.InternalMessageInfo
-
-func (m *GetOnuUniInfoRequest) GetUniIndex() uint32 {
-	if m != nil {
-		return m.UniIndex
-	}
-	return 0
-}
-
-type GetOnuUniInfoResponse struct {
-	AdmState             GetOnuUniInfoResponse_AdministrativeState `protobuf:"varint,1,opt,name=admState,proto3,enum=extension.GetOnuUniInfoResponse_AdministrativeState" json:"admState,omitempty"`
-	OperState            GetOnuUniInfoResponse_OperationalState    `protobuf:"varint,2,opt,name=operState,proto3,enum=extension.GetOnuUniInfoResponse_OperationalState" json:"operState,omitempty"`
-	ConfigInd            GetOnuUniInfoResponse_ConfigurationInd    `protobuf:"varint,3,opt,name=configInd,proto3,enum=extension.GetOnuUniInfoResponse_ConfigurationInd" json:"configInd,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                  `json:"-"`
-	XXX_unrecognized     []byte                                    `json:"-"`
-	XXX_sizecache        int32                                     `json:"-"`
-}
-
-func (m *GetOnuUniInfoResponse) Reset()         { *m = GetOnuUniInfoResponse{} }
-func (m *GetOnuUniInfoResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOnuUniInfoResponse) ProtoMessage()    {}
-func (*GetOnuUniInfoResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{3}
-}
-
-func (m *GetOnuUniInfoResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuUniInfoResponse.Unmarshal(m, b)
-}
-func (m *GetOnuUniInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuUniInfoResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOnuUniInfoResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuUniInfoResponse.Merge(m, src)
-}
-func (m *GetOnuUniInfoResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOnuUniInfoResponse.Size(m)
-}
-func (m *GetOnuUniInfoResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuUniInfoResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuUniInfoResponse proto.InternalMessageInfo
-
-func (m *GetOnuUniInfoResponse) GetAdmState() GetOnuUniInfoResponse_AdministrativeState {
-	if m != nil {
-		return m.AdmState
-	}
-	return GetOnuUniInfoResponse_ADMSTATE_UNDEFINED
-}
-
-func (m *GetOnuUniInfoResponse) GetOperState() GetOnuUniInfoResponse_OperationalState {
-	if m != nil {
-		return m.OperState
-	}
-	return GetOnuUniInfoResponse_OPERSTATE_UNDEFINED
-}
-
-func (m *GetOnuUniInfoResponse) GetConfigInd() GetOnuUniInfoResponse_ConfigurationInd {
-	if m != nil {
-		return m.ConfigInd
-	}
-	return GetOnuUniInfoResponse_UNKOWN
-}
-
-type GetOltPortCounters struct {
-	PortNo               uint32                      `protobuf:"varint,1,opt,name=portNo,proto3" json:"portNo,omitempty"`
-	PortType             GetOltPortCounters_PortType `protobuf:"varint,2,opt,name=portType,proto3,enum=extension.GetOltPortCounters_PortType" json:"portType,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
-}
-
-func (m *GetOltPortCounters) Reset()         { *m = GetOltPortCounters{} }
-func (m *GetOltPortCounters) String() string { return proto.CompactTextString(m) }
-func (*GetOltPortCounters) ProtoMessage()    {}
-func (*GetOltPortCounters) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{4}
-}
-
-func (m *GetOltPortCounters) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOltPortCounters.Unmarshal(m, b)
-}
-func (m *GetOltPortCounters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOltPortCounters.Marshal(b, m, deterministic)
-}
-func (m *GetOltPortCounters) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOltPortCounters.Merge(m, src)
-}
-func (m *GetOltPortCounters) XXX_Size() int {
-	return xxx_messageInfo_GetOltPortCounters.Size(m)
-}
-func (m *GetOltPortCounters) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOltPortCounters.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOltPortCounters proto.InternalMessageInfo
-
-func (m *GetOltPortCounters) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *GetOltPortCounters) GetPortType() GetOltPortCounters_PortType {
-	if m != nil {
-		return m.PortType
-	}
-	return GetOltPortCounters_Port_UNKNOWN
-}
-
-type GetOltPortCountersResponse struct {
-	TxBytes              uint64   `protobuf:"varint,1,opt,name=txBytes,proto3" json:"txBytes,omitempty"`
-	RxBytes              uint64   `protobuf:"varint,2,opt,name=rxBytes,proto3" json:"rxBytes,omitempty"`
-	TxPackets            uint64   `protobuf:"varint,3,opt,name=txPackets,proto3" json:"txPackets,omitempty"`
-	RxPackets            uint64   `protobuf:"varint,4,opt,name=rxPackets,proto3" json:"rxPackets,omitempty"`
-	TxErrorPackets       uint64   `protobuf:"varint,5,opt,name=txErrorPackets,proto3" json:"txErrorPackets,omitempty"`
-	RxErrorPackets       uint64   `protobuf:"varint,6,opt,name=rxErrorPackets,proto3" json:"rxErrorPackets,omitempty"`
-	TxBcastPackets       uint64   `protobuf:"varint,7,opt,name=txBcastPackets,proto3" json:"txBcastPackets,omitempty"`
-	RxBcastPackets       uint64   `protobuf:"varint,8,opt,name=rxBcastPackets,proto3" json:"rxBcastPackets,omitempty"`
-	TxUcastPackets       uint64   `protobuf:"varint,9,opt,name=txUcastPackets,proto3" json:"txUcastPackets,omitempty"`
-	RxUcastPackets       uint64   `protobuf:"varint,10,opt,name=rxUcastPackets,proto3" json:"rxUcastPackets,omitempty"`
-	TxMcastPackets       uint64   `protobuf:"varint,11,opt,name=txMcastPackets,proto3" json:"txMcastPackets,omitempty"`
-	RxMcastPackets       uint64   `protobuf:"varint,12,opt,name=rxMcastPackets,proto3" json:"rxMcastPackets,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetOltPortCountersResponse) Reset()         { *m = GetOltPortCountersResponse{} }
-func (m *GetOltPortCountersResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOltPortCountersResponse) ProtoMessage()    {}
-func (*GetOltPortCountersResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{5}
-}
-
-func (m *GetOltPortCountersResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOltPortCountersResponse.Unmarshal(m, b)
-}
-func (m *GetOltPortCountersResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOltPortCountersResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOltPortCountersResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOltPortCountersResponse.Merge(m, src)
-}
-func (m *GetOltPortCountersResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOltPortCountersResponse.Size(m)
-}
-func (m *GetOltPortCountersResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOltPortCountersResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOltPortCountersResponse proto.InternalMessageInfo
-
-func (m *GetOltPortCountersResponse) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetTxErrorPackets() uint64 {
-	if m != nil {
-		return m.TxErrorPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxErrorPackets() uint64 {
-	if m != nil {
-		return m.RxErrorPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetTxBcastPackets() uint64 {
-	if m != nil {
-		return m.TxBcastPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxBcastPackets() uint64 {
-	if m != nil {
-		return m.RxBcastPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetTxUcastPackets() uint64 {
-	if m != nil {
-		return m.TxUcastPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxUcastPackets() uint64 {
-	if m != nil {
-		return m.RxUcastPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetTxMcastPackets() uint64 {
-	if m != nil {
-		return m.TxMcastPackets
-	}
-	return 0
-}
-
-func (m *GetOltPortCountersResponse) GetRxMcastPackets() uint64 {
-	if m != nil {
-		return m.RxMcastPackets
-	}
-	return 0
-}
-
-type GetOnuPonOpticalInfo struct {
-	Empty                *empty.Empty `protobuf:"bytes,1,opt,name=empty,proto3" json:"empty,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *GetOnuPonOpticalInfo) Reset()         { *m = GetOnuPonOpticalInfo{} }
-func (m *GetOnuPonOpticalInfo) String() string { return proto.CompactTextString(m) }
-func (*GetOnuPonOpticalInfo) ProtoMessage()    {}
-func (*GetOnuPonOpticalInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{6}
-}
-
-func (m *GetOnuPonOpticalInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuPonOpticalInfo.Unmarshal(m, b)
-}
-func (m *GetOnuPonOpticalInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuPonOpticalInfo.Marshal(b, m, deterministic)
-}
-func (m *GetOnuPonOpticalInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuPonOpticalInfo.Merge(m, src)
-}
-func (m *GetOnuPonOpticalInfo) XXX_Size() int {
-	return xxx_messageInfo_GetOnuPonOpticalInfo.Size(m)
-}
-func (m *GetOnuPonOpticalInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuPonOpticalInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuPonOpticalInfo proto.InternalMessageInfo
-
-func (m *GetOnuPonOpticalInfo) GetEmpty() *empty.Empty {
-	if m != nil {
-		return m.Empty
-	}
-	return nil
-}
-
-// These values correspond to the Optical Line Supervision Test results
-// described in section A3.39.5 of ITU-T G.988 (11/2017) specification.
-type GetOnuPonOpticalInfoResponse struct {
-	PowerFeedVoltage       float32  `protobuf:"fixed32,1,opt,name=powerFeedVoltage,proto3" json:"powerFeedVoltage,omitempty"`
-	ReceivedOpticalPower   float32  `protobuf:"fixed32,2,opt,name=receivedOpticalPower,proto3" json:"receivedOpticalPower,omitempty"`
-	MeanOpticalLaunchPower float32  `protobuf:"fixed32,3,opt,name=meanOpticalLaunchPower,proto3" json:"meanOpticalLaunchPower,omitempty"`
-	LaserBiasCurrent       float32  `protobuf:"fixed32,4,opt,name=laserBiasCurrent,proto3" json:"laserBiasCurrent,omitempty"`
-	Temperature            float32  `protobuf:"fixed32,5,opt,name=temperature,proto3" json:"temperature,omitempty"`
-	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
-	XXX_unrecognized       []byte   `json:"-"`
-	XXX_sizecache          int32    `json:"-"`
-}
-
-func (m *GetOnuPonOpticalInfoResponse) Reset()         { *m = GetOnuPonOpticalInfoResponse{} }
-func (m *GetOnuPonOpticalInfoResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOnuPonOpticalInfoResponse) ProtoMessage()    {}
-func (*GetOnuPonOpticalInfoResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{7}
-}
-
-func (m *GetOnuPonOpticalInfoResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Unmarshal(m, b)
-}
-func (m *GetOnuPonOpticalInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOnuPonOpticalInfoResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuPonOpticalInfoResponse.Merge(m, src)
-}
-func (m *GetOnuPonOpticalInfoResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Size(m)
-}
-func (m *GetOnuPonOpticalInfoResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuPonOpticalInfoResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuPonOpticalInfoResponse proto.InternalMessageInfo
-
-func (m *GetOnuPonOpticalInfoResponse) GetPowerFeedVoltage() float32 {
-	if m != nil {
-		return m.PowerFeedVoltage
-	}
-	return 0
-}
-
-func (m *GetOnuPonOpticalInfoResponse) GetReceivedOpticalPower() float32 {
-	if m != nil {
-		return m.ReceivedOpticalPower
-	}
-	return 0
-}
-
-func (m *GetOnuPonOpticalInfoResponse) GetMeanOpticalLaunchPower() float32 {
-	if m != nil {
-		return m.MeanOpticalLaunchPower
-	}
-	return 0
-}
-
-func (m *GetOnuPonOpticalInfoResponse) GetLaserBiasCurrent() float32 {
-	if m != nil {
-		return m.LaserBiasCurrent
-	}
-	return 0
-}
-
-func (m *GetOnuPonOpticalInfoResponse) GetTemperature() float32 {
-	if m != nil {
-		return m.Temperature
-	}
-	return 0
-}
-
-type GetOnuEthernetBridgePortHistory struct {
-	Direction            GetOnuEthernetBridgePortHistory_Direction `protobuf:"varint,1,opt,name=direction,proto3,enum=extension.GetOnuEthernetBridgePortHistory_Direction" json:"direction,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                  `json:"-"`
-	XXX_unrecognized     []byte                                    `json:"-"`
-	XXX_sizecache        int32                                     `json:"-"`
-}
-
-func (m *GetOnuEthernetBridgePortHistory) Reset()         { *m = GetOnuEthernetBridgePortHistory{} }
-func (m *GetOnuEthernetBridgePortHistory) String() string { return proto.CompactTextString(m) }
-func (*GetOnuEthernetBridgePortHistory) ProtoMessage()    {}
-func (*GetOnuEthernetBridgePortHistory) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{8}
-}
-
-func (m *GetOnuEthernetBridgePortHistory) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Unmarshal(m, b)
-}
-func (m *GetOnuEthernetBridgePortHistory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Marshal(b, m, deterministic)
-}
-func (m *GetOnuEthernetBridgePortHistory) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuEthernetBridgePortHistory.Merge(m, src)
-}
-func (m *GetOnuEthernetBridgePortHistory) XXX_Size() int {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Size(m)
-}
-func (m *GetOnuEthernetBridgePortHistory) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuEthernetBridgePortHistory.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuEthernetBridgePortHistory proto.InternalMessageInfo
-
-func (m *GetOnuEthernetBridgePortHistory) GetDirection() GetOnuEthernetBridgePortHistory_Direction {
-	if m != nil {
-		return m.Direction
-	}
-	return GetOnuEthernetBridgePortHistory_UNDEFINED
-}
-
-type GetOnuEthernetBridgePortHistoryResponse struct {
-	DropEvents              uint32   `protobuf:"varint,1,opt,name=dropEvents,proto3" json:"dropEvents,omitempty"`
-	Octets                  uint32   `protobuf:"varint,2,opt,name=octets,proto3" json:"octets,omitempty"`
-	Packets                 uint32   `protobuf:"varint,3,opt,name=packets,proto3" json:"packets,omitempty"`
-	BroadcastPackets        uint32   `protobuf:"varint,4,opt,name=broadcastPackets,proto3" json:"broadcastPackets,omitempty"`
-	MulticastPackets        uint32   `protobuf:"varint,5,opt,name=multicastPackets,proto3" json:"multicastPackets,omitempty"`
-	CrcErroredPackets       uint32   `protobuf:"varint,6,opt,name=crcErroredPackets,proto3" json:"crcErroredPackets,omitempty"`
-	UndersizePackets        uint32   `protobuf:"varint,7,opt,name=undersizePackets,proto3" json:"undersizePackets,omitempty"`
-	OversizePackets         uint32   `protobuf:"varint,8,opt,name=oversizePackets,proto3" json:"oversizePackets,omitempty"`
-	Packets64Octets         uint32   `protobuf:"varint,9,opt,name=packets64octets,proto3" json:"packets64octets,omitempty"`
-	Packets65To127Octets    uint32   `protobuf:"varint,10,opt,name=packets65To127octets,proto3" json:"packets65To127octets,omitempty"`
-	Packets128To255Octets   uint32   `protobuf:"varint,11,opt,name=packets128To255Octets,proto3" json:"packets128To255Octets,omitempty"`
-	Packets256To511Octets   uint32   `protobuf:"varint,12,opt,name=packets256To511octets,proto3" json:"packets256To511octets,omitempty"`
-	Packets512To1023Octets  uint32   `protobuf:"varint,13,opt,name=packets512To1023octets,proto3" json:"packets512To1023octets,omitempty"`
-	Packets1024To1518Octets uint32   `protobuf:"varint,14,opt,name=packets1024To1518octets,proto3" json:"packets1024To1518octets,omitempty"`
-	XXX_NoUnkeyedLiteral    struct{} `json:"-"`
-	XXX_unrecognized        []byte   `json:"-"`
-	XXX_sizecache           int32    `json:"-"`
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) Reset() {
-	*m = GetOnuEthernetBridgePortHistoryResponse{}
-}
-func (m *GetOnuEthernetBridgePortHistoryResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOnuEthernetBridgePortHistoryResponse) ProtoMessage()    {}
-func (*GetOnuEthernetBridgePortHistoryResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{9}
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Unmarshal(m, b)
-}
-func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Merge(m, src)
-}
-func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Size(m)
-}
-func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse proto.InternalMessageInfo
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetDropEvents() uint32 {
-	if m != nil {
-		return m.DropEvents
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetOctets() uint32 {
-	if m != nil {
-		return m.Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets() uint32 {
-	if m != nil {
-		return m.Packets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetBroadcastPackets() uint32 {
-	if m != nil {
-		return m.BroadcastPackets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetMulticastPackets() uint32 {
-	if m != nil {
-		return m.MulticastPackets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetCrcErroredPackets() uint32 {
-	if m != nil {
-		return m.CrcErroredPackets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetUndersizePackets() uint32 {
-	if m != nil {
-		return m.UndersizePackets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetOversizePackets() uint32 {
-	if m != nil {
-		return m.OversizePackets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets64Octets() uint32 {
-	if m != nil {
-		return m.Packets64Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets65To127Octets() uint32 {
-	if m != nil {
-		return m.Packets65To127Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets128To255Octets() uint32 {
-	if m != nil {
-		return m.Packets128To255Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets256To511Octets() uint32 {
-	if m != nil {
-		return m.Packets256To511Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets512To1023Octets() uint32 {
-	if m != nil {
-		return m.Packets512To1023Octets
-	}
-	return 0
-}
-
-func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets1024To1518Octets() uint32 {
-	if m != nil {
-		return m.Packets1024To1518Octets
-	}
-	return 0
-}
-
-type GetOnuFecHistory struct {
-	Empty                *empty.Empty `protobuf:"bytes,1,opt,name=empty,proto3" json:"empty,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *GetOnuFecHistory) Reset()         { *m = GetOnuFecHistory{} }
-func (m *GetOnuFecHistory) String() string { return proto.CompactTextString(m) }
-func (*GetOnuFecHistory) ProtoMessage()    {}
-func (*GetOnuFecHistory) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{10}
-}
-
-func (m *GetOnuFecHistory) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuFecHistory.Unmarshal(m, b)
-}
-func (m *GetOnuFecHistory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuFecHistory.Marshal(b, m, deterministic)
-}
-func (m *GetOnuFecHistory) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuFecHistory.Merge(m, src)
-}
-func (m *GetOnuFecHistory) XXX_Size() int {
-	return xxx_messageInfo_GetOnuFecHistory.Size(m)
-}
-func (m *GetOnuFecHistory) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuFecHistory.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuFecHistory proto.InternalMessageInfo
-
-func (m *GetOnuFecHistory) GetEmpty() *empty.Empty {
-	if m != nil {
-		return m.Empty
-	}
-	return nil
-}
-
-type GetOnuFecHistoryResponse struct {
-	CorrectedBytes         uint32   `protobuf:"varint,1,opt,name=correctedBytes,proto3" json:"correctedBytes,omitempty"`
-	CorrectedCodeWords     uint32   `protobuf:"varint,2,opt,name=correctedCodeWords,proto3" json:"correctedCodeWords,omitempty"`
-	FecSeconds             uint32   `protobuf:"varint,3,opt,name=fecSeconds,proto3" json:"fecSeconds,omitempty"`
-	TotalCodeWords         uint32   `protobuf:"varint,4,opt,name=totalCodeWords,proto3" json:"totalCodeWords,omitempty"`
-	UncorrectableCodeWords uint32   `protobuf:"varint,5,opt,name=uncorrectableCodeWords,proto3" json:"uncorrectableCodeWords,omitempty"`
-	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
-	XXX_unrecognized       []byte   `json:"-"`
-	XXX_sizecache          int32    `json:"-"`
-}
-
-func (m *GetOnuFecHistoryResponse) Reset()         { *m = GetOnuFecHistoryResponse{} }
-func (m *GetOnuFecHistoryResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOnuFecHistoryResponse) ProtoMessage()    {}
-func (*GetOnuFecHistoryResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{11}
-}
-
-func (m *GetOnuFecHistoryResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuFecHistoryResponse.Unmarshal(m, b)
-}
-func (m *GetOnuFecHistoryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuFecHistoryResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOnuFecHistoryResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuFecHistoryResponse.Merge(m, src)
-}
-func (m *GetOnuFecHistoryResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOnuFecHistoryResponse.Size(m)
-}
-func (m *GetOnuFecHistoryResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuFecHistoryResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuFecHistoryResponse proto.InternalMessageInfo
-
-func (m *GetOnuFecHistoryResponse) GetCorrectedBytes() uint32 {
-	if m != nil {
-		return m.CorrectedBytes
-	}
-	return 0
-}
-
-func (m *GetOnuFecHistoryResponse) GetCorrectedCodeWords() uint32 {
-	if m != nil {
-		return m.CorrectedCodeWords
-	}
-	return 0
-}
-
-func (m *GetOnuFecHistoryResponse) GetFecSeconds() uint32 {
-	if m != nil {
-		return m.FecSeconds
-	}
-	return 0
-}
-
-func (m *GetOnuFecHistoryResponse) GetTotalCodeWords() uint32 {
-	if m != nil {
-		return m.TotalCodeWords
-	}
-	return 0
-}
-
-func (m *GetOnuFecHistoryResponse) GetUncorrectableCodeWords() uint32 {
-	if m != nil {
-		return m.UncorrectableCodeWords
-	}
-	return 0
-}
-
-type GetOnuCountersRequest struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetOnuCountersRequest) Reset()         { *m = GetOnuCountersRequest{} }
-func (m *GetOnuCountersRequest) String() string { return proto.CompactTextString(m) }
-func (*GetOnuCountersRequest) ProtoMessage()    {}
-func (*GetOnuCountersRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{12}
-}
-
-func (m *GetOnuCountersRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuCountersRequest.Unmarshal(m, b)
-}
-func (m *GetOnuCountersRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuCountersRequest.Marshal(b, m, deterministic)
-}
-func (m *GetOnuCountersRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuCountersRequest.Merge(m, src)
-}
-func (m *GetOnuCountersRequest) XXX_Size() int {
-	return xxx_messageInfo_GetOnuCountersRequest.Size(m)
-}
-func (m *GetOnuCountersRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuCountersRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuCountersRequest proto.InternalMessageInfo
-
-func (m *GetOnuCountersRequest) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *GetOnuCountersRequest) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-type GetOmciEthernetFrameExtendedPmRequest struct {
-	OnuDeviceId          string   `protobuf:"bytes,1,opt,name=onuDeviceId,proto3" json:"onuDeviceId,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetOmciEthernetFrameExtendedPmRequest) Reset()         { *m = GetOmciEthernetFrameExtendedPmRequest{} }
-func (m *GetOmciEthernetFrameExtendedPmRequest) String() string { return proto.CompactTextString(m) }
-func (*GetOmciEthernetFrameExtendedPmRequest) ProtoMessage()    {}
-func (*GetOmciEthernetFrameExtendedPmRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{13}
-}
-
-func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Unmarshal(m, b)
-}
-func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Marshal(b, m, deterministic)
-}
-func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Merge(m, src)
-}
-func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Size() int {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Size(m)
-}
-func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest proto.InternalMessageInfo
-
-func (m *GetOmciEthernetFrameExtendedPmRequest) GetOnuDeviceId() string {
-	if m != nil {
-		return m.OnuDeviceId
-	}
-	return ""
-}
-
-type GetRxPowerRequest struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetRxPowerRequest) Reset()         { *m = GetRxPowerRequest{} }
-func (m *GetRxPowerRequest) String() string { return proto.CompactTextString(m) }
-func (*GetRxPowerRequest) ProtoMessage()    {}
-func (*GetRxPowerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{14}
-}
-
-func (m *GetRxPowerRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetRxPowerRequest.Unmarshal(m, b)
-}
-func (m *GetRxPowerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetRxPowerRequest.Marshal(b, m, deterministic)
-}
-func (m *GetRxPowerRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetRxPowerRequest.Merge(m, src)
-}
-func (m *GetRxPowerRequest) XXX_Size() int {
-	return xxx_messageInfo_GetRxPowerRequest.Size(m)
-}
-func (m *GetRxPowerRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetRxPowerRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetRxPowerRequest proto.InternalMessageInfo
-
-func (m *GetRxPowerRequest) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *GetRxPowerRequest) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-type GetOnuCountersResponse struct {
-	// Types that are valid to be assigned to IsIntfId:
-	//	*GetOnuCountersResponse_IntfId
-	IsIntfId isGetOnuCountersResponse_IsIntfId `protobuf_oneof:"is_intf_id"`
-	// Types that are valid to be assigned to IsOnuId:
-	//	*GetOnuCountersResponse_OnuId
-	IsOnuId isGetOnuCountersResponse_IsOnuId `protobuf_oneof:"is_onu_id"`
-	// Types that are valid to be assigned to IsPositiveDrift:
-	//	*GetOnuCountersResponse_PositiveDrift
-	IsPositiveDrift isGetOnuCountersResponse_IsPositiveDrift `protobuf_oneof:"is_positive_drift"`
-	// Types that are valid to be assigned to IsNegativeDrift:
-	//	*GetOnuCountersResponse_NegativeDrift
-	IsNegativeDrift isGetOnuCountersResponse_IsNegativeDrift `protobuf_oneof:"is_negative_drift"`
-	// Types that are valid to be assigned to IsDelimiterMissDetection:
-	//	*GetOnuCountersResponse_DelimiterMissDetection
-	IsDelimiterMissDetection isGetOnuCountersResponse_IsDelimiterMissDetection `protobuf_oneof:"is_delimiter_miss_detection"`
-	// Types that are valid to be assigned to IsBipErrors:
-	//	*GetOnuCountersResponse_BipErrors
-	IsBipErrors isGetOnuCountersResponse_IsBipErrors `protobuf_oneof:"is_bip_errors"`
-	// Types that are valid to be assigned to IsBipUnits:
-	//	*GetOnuCountersResponse_BipUnits
-	IsBipUnits isGetOnuCountersResponse_IsBipUnits `protobuf_oneof:"is_bip_units"`
-	// Types that are valid to be assigned to IsFecCorrectedSymbols:
-	//	*GetOnuCountersResponse_FecCorrectedSymbols
-	IsFecCorrectedSymbols isGetOnuCountersResponse_IsFecCorrectedSymbols `protobuf_oneof:"is_fec_corrected_symbols"`
-	// Types that are valid to be assigned to IsFecCodewordsCorrected:
-	//	*GetOnuCountersResponse_FecCodewordsCorrected
-	IsFecCodewordsCorrected isGetOnuCountersResponse_IsFecCodewordsCorrected `protobuf_oneof:"is_fec_codewords_corrected"`
-	// Types that are valid to be assigned to IsFecCodewordsUncorrectable:
-	//	*GetOnuCountersResponse_FecCodewordsUncorrectable
-	IsFecCodewordsUncorrectable isGetOnuCountersResponse_IsFecCodewordsUncorrectable `protobuf_oneof:"is_fec_codewords_uncorrectable"`
-	// Types that are valid to be assigned to IsFecCodewords:
-	//	*GetOnuCountersResponse_FecCodewords
-	IsFecCodewords isGetOnuCountersResponse_IsFecCodewords `protobuf_oneof:"is_fec_codewords"`
-	// Types that are valid to be assigned to IsFecCorrectedUnits:
-	//	*GetOnuCountersResponse_FecCorrectedUnits
-	IsFecCorrectedUnits isGetOnuCountersResponse_IsFecCorrectedUnits `protobuf_oneof:"is_fec_corrected_units"`
-	// Types that are valid to be assigned to IsXgemKeyErrors:
-	//	*GetOnuCountersResponse_XgemKeyErrors
-	IsXgemKeyErrors isGetOnuCountersResponse_IsXgemKeyErrors `protobuf_oneof:"is_xgem_key_errors"`
-	// Types that are valid to be assigned to IsXgemLoss:
-	//	*GetOnuCountersResponse_XgemLoss
-	IsXgemLoss isGetOnuCountersResponse_IsXgemLoss `protobuf_oneof:"is_xgem_loss"`
-	// Types that are valid to be assigned to IsRxPloamsError:
-	//	*GetOnuCountersResponse_RxPloamsError
-	IsRxPloamsError isGetOnuCountersResponse_IsRxPloamsError `protobuf_oneof:"is_rx_ploams_error"`
-	// Types that are valid to be assigned to IsRxPloamsNonIdle:
-	//	*GetOnuCountersResponse_RxPloamsNonIdle
-	IsRxPloamsNonIdle isGetOnuCountersResponse_IsRxPloamsNonIdle `protobuf_oneof:"is_rx_ploams_non_idle"`
-	// Types that are valid to be assigned to IsRxOmci:
-	//	*GetOnuCountersResponse_RxOmci
-	IsRxOmci isGetOnuCountersResponse_IsRxOmci `protobuf_oneof:"is_rx_omci"`
-	// Types that are valid to be assigned to IsTxOmci:
-	//	*GetOnuCountersResponse_TxOmci
-	IsTxOmci isGetOnuCountersResponse_IsTxOmci `protobuf_oneof:"is_tx_omci"`
-	// Types that are valid to be assigned to IsRxOmciPacketsCrcError:
-	//	*GetOnuCountersResponse_RxOmciPacketsCrcError
-	IsRxOmciPacketsCrcError isGetOnuCountersResponse_IsRxOmciPacketsCrcError `protobuf_oneof:"is_rx_omci_packets_crc_error"`
-	// Types that are valid to be assigned to IsRxBytes:
-	//	*GetOnuCountersResponse_RxBytes
-	IsRxBytes isGetOnuCountersResponse_IsRxBytes `protobuf_oneof:"is_rx_bytes"`
-	// Types that are valid to be assigned to IsRxPackets:
-	//	*GetOnuCountersResponse_RxPackets
-	IsRxPackets isGetOnuCountersResponse_IsRxPackets `protobuf_oneof:"is_rx_packets"`
-	// Types that are valid to be assigned to IsTxBytes:
-	//	*GetOnuCountersResponse_TxBytes
-	IsTxBytes isGetOnuCountersResponse_IsTxBytes `protobuf_oneof:"is_tx_bytes"`
-	// Types that are valid to be assigned to IsTxPackets:
-	//	*GetOnuCountersResponse_TxPackets
-	IsTxPackets isGetOnuCountersResponse_IsTxPackets `protobuf_oneof:"is_tx_packets"`
-	// Types that are valid to be assigned to IsBerReported:
-	//	*GetOnuCountersResponse_BerReported
-	IsBerReported isGetOnuCountersResponse_IsBerReported `protobuf_oneof:"is_ber_reported"`
-	// Types that are valid to be assigned to IsLcdgErrors:
-	//	*GetOnuCountersResponse_LcdgErrors
-	IsLcdgErrors isGetOnuCountersResponse_IsLcdgErrors `protobuf_oneof:"is_lcdg_errors"`
-	// Types that are valid to be assigned to IsRdiErrors:
-	//	*GetOnuCountersResponse_RdiErrors
-	IsRdiErrors isGetOnuCountersResponse_IsRdiErrors `protobuf_oneof:"is_rdi_errors"`
-	// Types that are valid to be assigned to IsTimestamp:
-	//	*GetOnuCountersResponse_Timestamp
-	IsTimestamp          isGetOnuCountersResponse_IsTimestamp `protobuf_oneof:"is_timestamp"`
-	XXX_NoUnkeyedLiteral struct{}                             `json:"-"`
-	XXX_unrecognized     []byte                               `json:"-"`
-	XXX_sizecache        int32                                `json:"-"`
-}
-
-func (m *GetOnuCountersResponse) Reset()         { *m = GetOnuCountersResponse{} }
-func (m *GetOnuCountersResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOnuCountersResponse) ProtoMessage()    {}
-func (*GetOnuCountersResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{15}
-}
-
-func (m *GetOnuCountersResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOnuCountersResponse.Unmarshal(m, b)
-}
-func (m *GetOnuCountersResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOnuCountersResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOnuCountersResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOnuCountersResponse.Merge(m, src)
-}
-func (m *GetOnuCountersResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOnuCountersResponse.Size(m)
-}
-func (m *GetOnuCountersResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOnuCountersResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOnuCountersResponse proto.InternalMessageInfo
-
-type isGetOnuCountersResponse_IsIntfId interface {
-	isGetOnuCountersResponse_IsIntfId()
-}
-
-type GetOnuCountersResponse_IntfId struct {
-	IntfId uint32 `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_IntfId) isGetOnuCountersResponse_IsIntfId() {}
-
-func (m *GetOnuCountersResponse) GetIsIntfId() isGetOnuCountersResponse_IsIntfId {
-	if m != nil {
-		return m.IsIntfId
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetIntfId() uint32 {
-	if x, ok := m.GetIsIntfId().(*GetOnuCountersResponse_IntfId); ok {
-		return x.IntfId
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsOnuId interface {
-	isGetOnuCountersResponse_IsOnuId()
-}
-
-type GetOnuCountersResponse_OnuId struct {
-	OnuId uint32 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_OnuId) isGetOnuCountersResponse_IsOnuId() {}
-
-func (m *GetOnuCountersResponse) GetIsOnuId() isGetOnuCountersResponse_IsOnuId {
-	if m != nil {
-		return m.IsOnuId
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetOnuId() uint32 {
-	if x, ok := m.GetIsOnuId().(*GetOnuCountersResponse_OnuId); ok {
-		return x.OnuId
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsPositiveDrift interface {
-	isGetOnuCountersResponse_IsPositiveDrift()
-}
-
-type GetOnuCountersResponse_PositiveDrift struct {
-	PositiveDrift uint64 `protobuf:"fixed64,3,opt,name=positive_drift,json=positiveDrift,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_PositiveDrift) isGetOnuCountersResponse_IsPositiveDrift() {}
-
-func (m *GetOnuCountersResponse) GetIsPositiveDrift() isGetOnuCountersResponse_IsPositiveDrift {
-	if m != nil {
-		return m.IsPositiveDrift
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetPositiveDrift() uint64 {
-	if x, ok := m.GetIsPositiveDrift().(*GetOnuCountersResponse_PositiveDrift); ok {
-		return x.PositiveDrift
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsNegativeDrift interface {
-	isGetOnuCountersResponse_IsNegativeDrift()
-}
-
-type GetOnuCountersResponse_NegativeDrift struct {
-	NegativeDrift uint64 `protobuf:"fixed64,4,opt,name=negative_drift,json=negativeDrift,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_NegativeDrift) isGetOnuCountersResponse_IsNegativeDrift() {}
-
-func (m *GetOnuCountersResponse) GetIsNegativeDrift() isGetOnuCountersResponse_IsNegativeDrift {
-	if m != nil {
-		return m.IsNegativeDrift
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetNegativeDrift() uint64 {
-	if x, ok := m.GetIsNegativeDrift().(*GetOnuCountersResponse_NegativeDrift); ok {
-		return x.NegativeDrift
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsDelimiterMissDetection interface {
-	isGetOnuCountersResponse_IsDelimiterMissDetection()
-}
-
-type GetOnuCountersResponse_DelimiterMissDetection struct {
-	DelimiterMissDetection uint64 `protobuf:"fixed64,5,opt,name=delimiter_miss_detection,json=delimiterMissDetection,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_DelimiterMissDetection) isGetOnuCountersResponse_IsDelimiterMissDetection() {
-}
-
-func (m *GetOnuCountersResponse) GetIsDelimiterMissDetection() isGetOnuCountersResponse_IsDelimiterMissDetection {
-	if m != nil {
-		return m.IsDelimiterMissDetection
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetDelimiterMissDetection() uint64 {
-	if x, ok := m.GetIsDelimiterMissDetection().(*GetOnuCountersResponse_DelimiterMissDetection); ok {
-		return x.DelimiterMissDetection
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsBipErrors interface {
-	isGetOnuCountersResponse_IsBipErrors()
-}
-
-type GetOnuCountersResponse_BipErrors struct {
-	BipErrors uint64 `protobuf:"fixed64,6,opt,name=bip_errors,json=bipErrors,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_BipErrors) isGetOnuCountersResponse_IsBipErrors() {}
-
-func (m *GetOnuCountersResponse) GetIsBipErrors() isGetOnuCountersResponse_IsBipErrors {
-	if m != nil {
-		return m.IsBipErrors
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetBipErrors() uint64 {
-	if x, ok := m.GetIsBipErrors().(*GetOnuCountersResponse_BipErrors); ok {
-		return x.BipErrors
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsBipUnits interface {
-	isGetOnuCountersResponse_IsBipUnits()
-}
-
-type GetOnuCountersResponse_BipUnits struct {
-	BipUnits uint64 `protobuf:"fixed64,7,opt,name=bip_units,json=bipUnits,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_BipUnits) isGetOnuCountersResponse_IsBipUnits() {}
-
-func (m *GetOnuCountersResponse) GetIsBipUnits() isGetOnuCountersResponse_IsBipUnits {
-	if m != nil {
-		return m.IsBipUnits
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetBipUnits() uint64 {
-	if x, ok := m.GetIsBipUnits().(*GetOnuCountersResponse_BipUnits); ok {
-		return x.BipUnits
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsFecCorrectedSymbols interface {
-	isGetOnuCountersResponse_IsFecCorrectedSymbols()
-}
-
-type GetOnuCountersResponse_FecCorrectedSymbols struct {
-	FecCorrectedSymbols uint64 `protobuf:"fixed64,8,opt,name=fec_corrected_symbols,json=fecCorrectedSymbols,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_FecCorrectedSymbols) isGetOnuCountersResponse_IsFecCorrectedSymbols() {}
-
-func (m *GetOnuCountersResponse) GetIsFecCorrectedSymbols() isGetOnuCountersResponse_IsFecCorrectedSymbols {
-	if m != nil {
-		return m.IsFecCorrectedSymbols
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetFecCorrectedSymbols() uint64 {
-	if x, ok := m.GetIsFecCorrectedSymbols().(*GetOnuCountersResponse_FecCorrectedSymbols); ok {
-		return x.FecCorrectedSymbols
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsFecCodewordsCorrected interface {
-	isGetOnuCountersResponse_IsFecCodewordsCorrected()
-}
-
-type GetOnuCountersResponse_FecCodewordsCorrected struct {
-	FecCodewordsCorrected uint64 `protobuf:"fixed64,9,opt,name=fec_codewords_corrected,json=fecCodewordsCorrected,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_FecCodewordsCorrected) isGetOnuCountersResponse_IsFecCodewordsCorrected() {
-}
-
-func (m *GetOnuCountersResponse) GetIsFecCodewordsCorrected() isGetOnuCountersResponse_IsFecCodewordsCorrected {
-	if m != nil {
-		return m.IsFecCodewordsCorrected
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetFecCodewordsCorrected() uint64 {
-	if x, ok := m.GetIsFecCodewordsCorrected().(*GetOnuCountersResponse_FecCodewordsCorrected); ok {
-		return x.FecCodewordsCorrected
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsFecCodewordsUncorrectable interface {
-	isGetOnuCountersResponse_IsFecCodewordsUncorrectable()
-}
-
-type GetOnuCountersResponse_FecCodewordsUncorrectable struct {
-	FecCodewordsUncorrectable uint64 `protobuf:"fixed64,10,opt,name=fec_codewords_uncorrectable,json=fecCodewordsUncorrectable,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_FecCodewordsUncorrectable) isGetOnuCountersResponse_IsFecCodewordsUncorrectable() {
-}
-
-func (m *GetOnuCountersResponse) GetIsFecCodewordsUncorrectable() isGetOnuCountersResponse_IsFecCodewordsUncorrectable {
-	if m != nil {
-		return m.IsFecCodewordsUncorrectable
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetFecCodewordsUncorrectable() uint64 {
-	if x, ok := m.GetIsFecCodewordsUncorrectable().(*GetOnuCountersResponse_FecCodewordsUncorrectable); ok {
-		return x.FecCodewordsUncorrectable
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsFecCodewords interface {
-	isGetOnuCountersResponse_IsFecCodewords()
-}
-
-type GetOnuCountersResponse_FecCodewords struct {
-	FecCodewords uint64 `protobuf:"fixed64,11,opt,name=fec_codewords,json=fecCodewords,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_FecCodewords) isGetOnuCountersResponse_IsFecCodewords() {}
-
-func (m *GetOnuCountersResponse) GetIsFecCodewords() isGetOnuCountersResponse_IsFecCodewords {
-	if m != nil {
-		return m.IsFecCodewords
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetFecCodewords() uint64 {
-	if x, ok := m.GetIsFecCodewords().(*GetOnuCountersResponse_FecCodewords); ok {
-		return x.FecCodewords
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsFecCorrectedUnits interface {
-	isGetOnuCountersResponse_IsFecCorrectedUnits()
-}
-
-type GetOnuCountersResponse_FecCorrectedUnits struct {
-	FecCorrectedUnits uint64 `protobuf:"fixed64,12,opt,name=fec_corrected_units,json=fecCorrectedUnits,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_FecCorrectedUnits) isGetOnuCountersResponse_IsFecCorrectedUnits() {}
-
-func (m *GetOnuCountersResponse) GetIsFecCorrectedUnits() isGetOnuCountersResponse_IsFecCorrectedUnits {
-	if m != nil {
-		return m.IsFecCorrectedUnits
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetFecCorrectedUnits() uint64 {
-	if x, ok := m.GetIsFecCorrectedUnits().(*GetOnuCountersResponse_FecCorrectedUnits); ok {
-		return x.FecCorrectedUnits
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsXgemKeyErrors interface {
-	isGetOnuCountersResponse_IsXgemKeyErrors()
-}
-
-type GetOnuCountersResponse_XgemKeyErrors struct {
-	XgemKeyErrors uint64 `protobuf:"fixed64,13,opt,name=xgem_key_errors,json=xgemKeyErrors,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_XgemKeyErrors) isGetOnuCountersResponse_IsXgemKeyErrors() {}
-
-func (m *GetOnuCountersResponse) GetIsXgemKeyErrors() isGetOnuCountersResponse_IsXgemKeyErrors {
-	if m != nil {
-		return m.IsXgemKeyErrors
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetXgemKeyErrors() uint64 {
-	if x, ok := m.GetIsXgemKeyErrors().(*GetOnuCountersResponse_XgemKeyErrors); ok {
-		return x.XgemKeyErrors
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsXgemLoss interface {
-	isGetOnuCountersResponse_IsXgemLoss()
-}
-
-type GetOnuCountersResponse_XgemLoss struct {
-	XgemLoss uint64 `protobuf:"fixed64,14,opt,name=xgem_loss,json=xgemLoss,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_XgemLoss) isGetOnuCountersResponse_IsXgemLoss() {}
-
-func (m *GetOnuCountersResponse) GetIsXgemLoss() isGetOnuCountersResponse_IsXgemLoss {
-	if m != nil {
-		return m.IsXgemLoss
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetXgemLoss() uint64 {
-	if x, ok := m.GetIsXgemLoss().(*GetOnuCountersResponse_XgemLoss); ok {
-		return x.XgemLoss
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxPloamsError interface {
-	isGetOnuCountersResponse_IsRxPloamsError()
-}
-
-type GetOnuCountersResponse_RxPloamsError struct {
-	RxPloamsError uint64 `protobuf:"fixed64,15,opt,name=rx_ploams_error,json=rxPloamsError,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxPloamsError) isGetOnuCountersResponse_IsRxPloamsError() {}
-
-func (m *GetOnuCountersResponse) GetIsRxPloamsError() isGetOnuCountersResponse_IsRxPloamsError {
-	if m != nil {
-		return m.IsRxPloamsError
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxPloamsError() uint64 {
-	if x, ok := m.GetIsRxPloamsError().(*GetOnuCountersResponse_RxPloamsError); ok {
-		return x.RxPloamsError
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxPloamsNonIdle interface {
-	isGetOnuCountersResponse_IsRxPloamsNonIdle()
-}
-
-type GetOnuCountersResponse_RxPloamsNonIdle struct {
-	RxPloamsNonIdle uint64 `protobuf:"fixed64,16,opt,name=rx_ploams_non_idle,json=rxPloamsNonIdle,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxPloamsNonIdle) isGetOnuCountersResponse_IsRxPloamsNonIdle() {}
-
-func (m *GetOnuCountersResponse) GetIsRxPloamsNonIdle() isGetOnuCountersResponse_IsRxPloamsNonIdle {
-	if m != nil {
-		return m.IsRxPloamsNonIdle
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxPloamsNonIdle() uint64 {
-	if x, ok := m.GetIsRxPloamsNonIdle().(*GetOnuCountersResponse_RxPloamsNonIdle); ok {
-		return x.RxPloamsNonIdle
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxOmci interface {
-	isGetOnuCountersResponse_IsRxOmci()
-}
-
-type GetOnuCountersResponse_RxOmci struct {
-	RxOmci uint64 `protobuf:"fixed64,17,opt,name=rx_omci,json=rxOmci,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxOmci) isGetOnuCountersResponse_IsRxOmci() {}
-
-func (m *GetOnuCountersResponse) GetIsRxOmci() isGetOnuCountersResponse_IsRxOmci {
-	if m != nil {
-		return m.IsRxOmci
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxOmci() uint64 {
-	if x, ok := m.GetIsRxOmci().(*GetOnuCountersResponse_RxOmci); ok {
-		return x.RxOmci
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsTxOmci interface {
-	isGetOnuCountersResponse_IsTxOmci()
-}
-
-type GetOnuCountersResponse_TxOmci struct {
-	TxOmci uint64 `protobuf:"fixed64,18,opt,name=tx_omci,json=txOmci,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_TxOmci) isGetOnuCountersResponse_IsTxOmci() {}
-
-func (m *GetOnuCountersResponse) GetIsTxOmci() isGetOnuCountersResponse_IsTxOmci {
-	if m != nil {
-		return m.IsTxOmci
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetTxOmci() uint64 {
-	if x, ok := m.GetIsTxOmci().(*GetOnuCountersResponse_TxOmci); ok {
-		return x.TxOmci
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxOmciPacketsCrcError interface {
-	isGetOnuCountersResponse_IsRxOmciPacketsCrcError()
-}
-
-type GetOnuCountersResponse_RxOmciPacketsCrcError struct {
-	RxOmciPacketsCrcError uint64 `protobuf:"fixed64,19,opt,name=rx_omci_packets_crc_error,json=rxOmciPacketsCrcError,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxOmciPacketsCrcError) isGetOnuCountersResponse_IsRxOmciPacketsCrcError() {
-}
-
-func (m *GetOnuCountersResponse) GetIsRxOmciPacketsCrcError() isGetOnuCountersResponse_IsRxOmciPacketsCrcError {
-	if m != nil {
-		return m.IsRxOmciPacketsCrcError
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxOmciPacketsCrcError() uint64 {
-	if x, ok := m.GetIsRxOmciPacketsCrcError().(*GetOnuCountersResponse_RxOmciPacketsCrcError); ok {
-		return x.RxOmciPacketsCrcError
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxBytes interface {
-	isGetOnuCountersResponse_IsRxBytes()
-}
-
-type GetOnuCountersResponse_RxBytes struct {
-	RxBytes uint64 `protobuf:"fixed64,20,opt,name=rx_bytes,json=rxBytes,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxBytes) isGetOnuCountersResponse_IsRxBytes() {}
-
-func (m *GetOnuCountersResponse) GetIsRxBytes() isGetOnuCountersResponse_IsRxBytes {
-	if m != nil {
-		return m.IsRxBytes
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxBytes() uint64 {
-	if x, ok := m.GetIsRxBytes().(*GetOnuCountersResponse_RxBytes); ok {
-		return x.RxBytes
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRxPackets interface {
-	isGetOnuCountersResponse_IsRxPackets()
-}
-
-type GetOnuCountersResponse_RxPackets struct {
-	RxPackets uint64 `protobuf:"fixed64,21,opt,name=rx_packets,json=rxPackets,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RxPackets) isGetOnuCountersResponse_IsRxPackets() {}
-
-func (m *GetOnuCountersResponse) GetIsRxPackets() isGetOnuCountersResponse_IsRxPackets {
-	if m != nil {
-		return m.IsRxPackets
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRxPackets() uint64 {
-	if x, ok := m.GetIsRxPackets().(*GetOnuCountersResponse_RxPackets); ok {
-		return x.RxPackets
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsTxBytes interface {
-	isGetOnuCountersResponse_IsTxBytes()
-}
-
-type GetOnuCountersResponse_TxBytes struct {
-	TxBytes uint64 `protobuf:"fixed64,22,opt,name=tx_bytes,json=txBytes,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_TxBytes) isGetOnuCountersResponse_IsTxBytes() {}
-
-func (m *GetOnuCountersResponse) GetIsTxBytes() isGetOnuCountersResponse_IsTxBytes {
-	if m != nil {
-		return m.IsTxBytes
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetTxBytes() uint64 {
-	if x, ok := m.GetIsTxBytes().(*GetOnuCountersResponse_TxBytes); ok {
-		return x.TxBytes
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsTxPackets interface {
-	isGetOnuCountersResponse_IsTxPackets()
-}
-
-type GetOnuCountersResponse_TxPackets struct {
-	TxPackets uint64 `protobuf:"fixed64,23,opt,name=tx_packets,json=txPackets,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_TxPackets) isGetOnuCountersResponse_IsTxPackets() {}
-
-func (m *GetOnuCountersResponse) GetIsTxPackets() isGetOnuCountersResponse_IsTxPackets {
-	if m != nil {
-		return m.IsTxPackets
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetTxPackets() uint64 {
-	if x, ok := m.GetIsTxPackets().(*GetOnuCountersResponse_TxPackets); ok {
-		return x.TxPackets
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsBerReported interface {
-	isGetOnuCountersResponse_IsBerReported()
-}
-
-type GetOnuCountersResponse_BerReported struct {
-	BerReported uint64 `protobuf:"fixed64,24,opt,name=ber_reported,json=berReported,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_BerReported) isGetOnuCountersResponse_IsBerReported() {}
-
-func (m *GetOnuCountersResponse) GetIsBerReported() isGetOnuCountersResponse_IsBerReported {
-	if m != nil {
-		return m.IsBerReported
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetBerReported() uint64 {
-	if x, ok := m.GetIsBerReported().(*GetOnuCountersResponse_BerReported); ok {
-		return x.BerReported
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsLcdgErrors interface {
-	isGetOnuCountersResponse_IsLcdgErrors()
-}
-
-type GetOnuCountersResponse_LcdgErrors struct {
-	LcdgErrors uint64 `protobuf:"fixed64,25,opt,name=lcdg_errors,json=lcdgErrors,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_LcdgErrors) isGetOnuCountersResponse_IsLcdgErrors() {}
-
-func (m *GetOnuCountersResponse) GetIsLcdgErrors() isGetOnuCountersResponse_IsLcdgErrors {
-	if m != nil {
-		return m.IsLcdgErrors
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetLcdgErrors() uint64 {
-	if x, ok := m.GetIsLcdgErrors().(*GetOnuCountersResponse_LcdgErrors); ok {
-		return x.LcdgErrors
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsRdiErrors interface {
-	isGetOnuCountersResponse_IsRdiErrors()
-}
-
-type GetOnuCountersResponse_RdiErrors struct {
-	RdiErrors uint64 `protobuf:"fixed64,26,opt,name=rdi_errors,json=rdiErrors,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_RdiErrors) isGetOnuCountersResponse_IsRdiErrors() {}
-
-func (m *GetOnuCountersResponse) GetIsRdiErrors() isGetOnuCountersResponse_IsRdiErrors {
-	if m != nil {
-		return m.IsRdiErrors
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetRdiErrors() uint64 {
-	if x, ok := m.GetIsRdiErrors().(*GetOnuCountersResponse_RdiErrors); ok {
-		return x.RdiErrors
-	}
-	return 0
-}
-
-type isGetOnuCountersResponse_IsTimestamp interface {
-	isGetOnuCountersResponse_IsTimestamp()
-}
-
-type GetOnuCountersResponse_Timestamp struct {
-	Timestamp uint32 `protobuf:"fixed32,27,opt,name=timestamp,proto3,oneof"`
-}
-
-func (*GetOnuCountersResponse_Timestamp) isGetOnuCountersResponse_IsTimestamp() {}
-
-func (m *GetOnuCountersResponse) GetIsTimestamp() isGetOnuCountersResponse_IsTimestamp {
-	if m != nil {
-		return m.IsTimestamp
-	}
-	return nil
-}
-
-func (m *GetOnuCountersResponse) GetTimestamp() uint32 {
-	if x, ok := m.GetIsTimestamp().(*GetOnuCountersResponse_Timestamp); ok {
-		return x.Timestamp
-	}
-	return 0
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*GetOnuCountersResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*GetOnuCountersResponse_IntfId)(nil),
-		(*GetOnuCountersResponse_OnuId)(nil),
-		(*GetOnuCountersResponse_PositiveDrift)(nil),
-		(*GetOnuCountersResponse_NegativeDrift)(nil),
-		(*GetOnuCountersResponse_DelimiterMissDetection)(nil),
-		(*GetOnuCountersResponse_BipErrors)(nil),
-		(*GetOnuCountersResponse_BipUnits)(nil),
-		(*GetOnuCountersResponse_FecCorrectedSymbols)(nil),
-		(*GetOnuCountersResponse_FecCodewordsCorrected)(nil),
-		(*GetOnuCountersResponse_FecCodewordsUncorrectable)(nil),
-		(*GetOnuCountersResponse_FecCodewords)(nil),
-		(*GetOnuCountersResponse_FecCorrectedUnits)(nil),
-		(*GetOnuCountersResponse_XgemKeyErrors)(nil),
-		(*GetOnuCountersResponse_XgemLoss)(nil),
-		(*GetOnuCountersResponse_RxPloamsError)(nil),
-		(*GetOnuCountersResponse_RxPloamsNonIdle)(nil),
-		(*GetOnuCountersResponse_RxOmci)(nil),
-		(*GetOnuCountersResponse_TxOmci)(nil),
-		(*GetOnuCountersResponse_RxOmciPacketsCrcError)(nil),
-		(*GetOnuCountersResponse_RxBytes)(nil),
-		(*GetOnuCountersResponse_RxPackets)(nil),
-		(*GetOnuCountersResponse_TxBytes)(nil),
-		(*GetOnuCountersResponse_TxPackets)(nil),
-		(*GetOnuCountersResponse_BerReported)(nil),
-		(*GetOnuCountersResponse_LcdgErrors)(nil),
-		(*GetOnuCountersResponse_RdiErrors)(nil),
-		(*GetOnuCountersResponse_Timestamp)(nil),
-	}
-}
-
-type OmciEthernetFrameExtendedPm struct {
-	DropEvents               uint64   `protobuf:"fixed64,1,opt,name=drop_events,json=dropEvents,proto3" json:"drop_events,omitempty"`
-	Octets                   uint64   `protobuf:"fixed64,2,opt,name=octets,proto3" json:"octets,omitempty"`
-	Frames                   uint64   `protobuf:"fixed64,3,opt,name=frames,proto3" json:"frames,omitempty"`
-	BroadcastFrames          uint64   `protobuf:"fixed64,4,opt,name=broadcast_frames,json=broadcastFrames,proto3" json:"broadcast_frames,omitempty"`
-	MulticastFrames          uint64   `protobuf:"fixed64,5,opt,name=multicast_frames,json=multicastFrames,proto3" json:"multicast_frames,omitempty"`
-	CrcErroredFrames         uint64   `protobuf:"fixed64,6,opt,name=crc_errored_frames,json=crcErroredFrames,proto3" json:"crc_errored_frames,omitempty"`
-	UndersizeFrames          uint64   `protobuf:"fixed64,7,opt,name=undersize_frames,json=undersizeFrames,proto3" json:"undersize_frames,omitempty"`
-	OversizeFrames           uint64   `protobuf:"fixed64,8,opt,name=oversize_frames,json=oversizeFrames,proto3" json:"oversize_frames,omitempty"`
-	Frames_64Octets          uint64   `protobuf:"fixed64,9,opt,name=frames_64_octets,json=frames64Octets,proto3" json:"frames_64_octets,omitempty"`
-	Frames_65To_127Octets    uint64   `protobuf:"fixed64,10,opt,name=frames_65_to_127_octets,json=frames65To127Octets,proto3" json:"frames_65_to_127_octets,omitempty"`
-	Frames_128To_255Octets   uint64   `protobuf:"fixed64,11,opt,name=frames_128_to_255_octets,json=frames128To255Octets,proto3" json:"frames_128_to_255_octets,omitempty"`
-	Frames_256To_511Octets   uint64   `protobuf:"fixed64,12,opt,name=frames_256_to_511_octets,json=frames256To511Octets,proto3" json:"frames_256_to_511_octets,omitempty"`
-	Frames_512To_1023Octets  uint64   `protobuf:"fixed64,13,opt,name=frames_512_to_1023_octets,json=frames512To1023Octets,proto3" json:"frames_512_to_1023_octets,omitempty"`
-	Frames_1024To_1518Octets uint64   `protobuf:"fixed64,14,opt,name=frames_1024_to_1518_octets,json=frames1024To1518Octets,proto3" json:"frames_1024_to_1518_octets,omitempty"`
-	XXX_NoUnkeyedLiteral     struct{} `json:"-"`
-	XXX_unrecognized         []byte   `json:"-"`
-	XXX_sizecache            int32    `json:"-"`
-}
-
-func (m *OmciEthernetFrameExtendedPm) Reset()         { *m = OmciEthernetFrameExtendedPm{} }
-func (m *OmciEthernetFrameExtendedPm) String() string { return proto.CompactTextString(m) }
-func (*OmciEthernetFrameExtendedPm) ProtoMessage()    {}
-func (*OmciEthernetFrameExtendedPm) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{16}
-}
-
-func (m *OmciEthernetFrameExtendedPm) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Unmarshal(m, b)
-}
-func (m *OmciEthernetFrameExtendedPm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Marshal(b, m, deterministic)
-}
-func (m *OmciEthernetFrameExtendedPm) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OmciEthernetFrameExtendedPm.Merge(m, src)
-}
-func (m *OmciEthernetFrameExtendedPm) XXX_Size() int {
-	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Size(m)
-}
-func (m *OmciEthernetFrameExtendedPm) XXX_DiscardUnknown() {
-	xxx_messageInfo_OmciEthernetFrameExtendedPm.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OmciEthernetFrameExtendedPm proto.InternalMessageInfo
-
-func (m *OmciEthernetFrameExtendedPm) GetDropEvents() uint64 {
-	if m != nil {
-		return m.DropEvents
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetOctets() uint64 {
-	if m != nil {
-		return m.Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames() uint64 {
-	if m != nil {
-		return m.Frames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetBroadcastFrames() uint64 {
-	if m != nil {
-		return m.BroadcastFrames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetMulticastFrames() uint64 {
-	if m != nil {
-		return m.MulticastFrames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetCrcErroredFrames() uint64 {
-	if m != nil {
-		return m.CrcErroredFrames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetUndersizeFrames() uint64 {
-	if m != nil {
-		return m.UndersizeFrames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetOversizeFrames() uint64 {
-	if m != nil {
-		return m.OversizeFrames
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_64Octets() uint64 {
-	if m != nil {
-		return m.Frames_64Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_65To_127Octets() uint64 {
-	if m != nil {
-		return m.Frames_65To_127Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_128To_255Octets() uint64 {
-	if m != nil {
-		return m.Frames_128To_255Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_256To_511Octets() uint64 {
-	if m != nil {
-		return m.Frames_256To_511Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_512To_1023Octets() uint64 {
-	if m != nil {
-		return m.Frames_512To_1023Octets
-	}
-	return 0
-}
-
-func (m *OmciEthernetFrameExtendedPm) GetFrames_1024To_1518Octets() uint64 {
-	if m != nil {
-		return m.Frames_1024To_1518Octets
-	}
-	return 0
-}
-
-type GetOmciEthernetFrameExtendedPmResponse struct {
-	Upstream             *OmciEthernetFrameExtendedPm `protobuf:"bytes,1,opt,name=upstream,proto3" json:"upstream,omitempty"`
-	Downstream           *OmciEthernetFrameExtendedPm `protobuf:"bytes,2,opt,name=downstream,proto3" json:"downstream,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
-}
-
-func (m *GetOmciEthernetFrameExtendedPmResponse) Reset() {
-	*m = GetOmciEthernetFrameExtendedPmResponse{}
-}
-func (m *GetOmciEthernetFrameExtendedPmResponse) String() string { return proto.CompactTextString(m) }
-func (*GetOmciEthernetFrameExtendedPmResponse) ProtoMessage()    {}
-func (*GetOmciEthernetFrameExtendedPmResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{17}
-}
-
-func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Unmarshal(m, b)
-}
-func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Marshal(b, m, deterministic)
-}
-func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Merge(m, src)
-}
-func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Size() int {
-	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Size(m)
-}
-func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse proto.InternalMessageInfo
-
-func (m *GetOmciEthernetFrameExtendedPmResponse) GetUpstream() *OmciEthernetFrameExtendedPm {
-	if m != nil {
-		return m.Upstream
-	}
-	return nil
-}
-
-func (m *GetOmciEthernetFrameExtendedPmResponse) GetDownstream() *OmciEthernetFrameExtendedPm {
-	if m != nil {
-		return m.Downstream
-	}
-	return nil
-}
-
-type GetRxPowerResponse struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	FailReason           string   `protobuf:"bytes,4,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
-	RxPower              float64  `protobuf:"fixed64,5,opt,name=rx_power,json=rxPower,proto3" json:"rx_power,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GetRxPowerResponse) Reset()         { *m = GetRxPowerResponse{} }
-func (m *GetRxPowerResponse) String() string { return proto.CompactTextString(m) }
-func (*GetRxPowerResponse) ProtoMessage()    {}
-func (*GetRxPowerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{18}
-}
-
-func (m *GetRxPowerResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetRxPowerResponse.Unmarshal(m, b)
-}
-func (m *GetRxPowerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetRxPowerResponse.Marshal(b, m, deterministic)
-}
-func (m *GetRxPowerResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetRxPowerResponse.Merge(m, src)
-}
-func (m *GetRxPowerResponse) XXX_Size() int {
-	return xxx_messageInfo_GetRxPowerResponse.Size(m)
-}
-func (m *GetRxPowerResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetRxPowerResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetRxPowerResponse proto.InternalMessageInfo
-
-func (m *GetRxPowerResponse) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *GetRxPowerResponse) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *GetRxPowerResponse) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *GetRxPowerResponse) GetFailReason() string {
-	if m != nil {
-		return m.FailReason
-	}
-	return ""
-}
-
-func (m *GetRxPowerResponse) GetRxPower() float64 {
-	if m != nil {
-		return m.RxPower
-	}
-	return 0
-}
-
-type GetValueRequest struct {
-	// Types that are valid to be assigned to Request:
-	//	*GetValueRequest_Distance
-	//	*GetValueRequest_UniInfo
-	//	*GetValueRequest_OltPortInfo
-	//	*GetValueRequest_OnuOpticalInfo
-	//	*GetValueRequest_EthBridgePort
-	//	*GetValueRequest_FecHistory
-	//	*GetValueRequest_OnuPonInfo
-	//	*GetValueRequest_OnuInfo
-	//	*GetValueRequest_RxPower
-	Request              isGetValueRequest_Request `protobuf_oneof:"request"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *GetValueRequest) Reset()         { *m = GetValueRequest{} }
-func (m *GetValueRequest) String() string { return proto.CompactTextString(m) }
-func (*GetValueRequest) ProtoMessage()    {}
-func (*GetValueRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{19}
-}
-
-func (m *GetValueRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetValueRequest.Unmarshal(m, b)
-}
-func (m *GetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetValueRequest.Marshal(b, m, deterministic)
-}
-func (m *GetValueRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetValueRequest.Merge(m, src)
-}
-func (m *GetValueRequest) XXX_Size() int {
-	return xxx_messageInfo_GetValueRequest.Size(m)
-}
-func (m *GetValueRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetValueRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetValueRequest proto.InternalMessageInfo
-
-type isGetValueRequest_Request interface {
-	isGetValueRequest_Request()
-}
-
-type GetValueRequest_Distance struct {
-	Distance *GetDistanceRequest `protobuf:"bytes,1,opt,name=distance,proto3,oneof"`
-}
-
-type GetValueRequest_UniInfo struct {
-	UniInfo *GetOnuUniInfoRequest `protobuf:"bytes,2,opt,name=uniInfo,proto3,oneof"`
-}
-
-type GetValueRequest_OltPortInfo struct {
-	OltPortInfo *GetOltPortCounters `protobuf:"bytes,3,opt,name=oltPortInfo,proto3,oneof"`
-}
-
-type GetValueRequest_OnuOpticalInfo struct {
-	OnuOpticalInfo *GetOnuPonOpticalInfo `protobuf:"bytes,4,opt,name=onuOpticalInfo,proto3,oneof"`
-}
-
-type GetValueRequest_EthBridgePort struct {
-	EthBridgePort *GetOnuEthernetBridgePortHistory `protobuf:"bytes,5,opt,name=ethBridgePort,proto3,oneof"`
-}
-
-type GetValueRequest_FecHistory struct {
-	FecHistory *GetOnuFecHistory `protobuf:"bytes,6,opt,name=fecHistory,proto3,oneof"`
-}
-
-type GetValueRequest_OnuPonInfo struct {
-	OnuPonInfo *GetOnuCountersRequest `protobuf:"bytes,7,opt,name=onuPonInfo,proto3,oneof"`
-}
-
-type GetValueRequest_OnuInfo struct {
-	OnuInfo *GetOmciEthernetFrameExtendedPmRequest `protobuf:"bytes,8,opt,name=onuInfo,proto3,oneof"`
-}
-
-type GetValueRequest_RxPower struct {
-	RxPower *GetRxPowerRequest `protobuf:"bytes,9,opt,name=rxPower,proto3,oneof"`
-}
-
-func (*GetValueRequest_Distance) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_UniInfo) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_OltPortInfo) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_OnuOpticalInfo) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_EthBridgePort) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_FecHistory) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_OnuPonInfo) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_OnuInfo) isGetValueRequest_Request() {}
-
-func (*GetValueRequest_RxPower) isGetValueRequest_Request() {}
-
-func (m *GetValueRequest) GetRequest() isGetValueRequest_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetDistance() *GetDistanceRequest {
-	if x, ok := m.GetRequest().(*GetValueRequest_Distance); ok {
-		return x.Distance
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetUniInfo() *GetOnuUniInfoRequest {
-	if x, ok := m.GetRequest().(*GetValueRequest_UniInfo); ok {
-		return x.UniInfo
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetOltPortInfo() *GetOltPortCounters {
-	if x, ok := m.GetRequest().(*GetValueRequest_OltPortInfo); ok {
-		return x.OltPortInfo
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetOnuOpticalInfo() *GetOnuPonOpticalInfo {
-	if x, ok := m.GetRequest().(*GetValueRequest_OnuOpticalInfo); ok {
-		return x.OnuOpticalInfo
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetEthBridgePort() *GetOnuEthernetBridgePortHistory {
-	if x, ok := m.GetRequest().(*GetValueRequest_EthBridgePort); ok {
-		return x.EthBridgePort
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetFecHistory() *GetOnuFecHistory {
-	if x, ok := m.GetRequest().(*GetValueRequest_FecHistory); ok {
-		return x.FecHistory
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetOnuPonInfo() *GetOnuCountersRequest {
-	if x, ok := m.GetRequest().(*GetValueRequest_OnuPonInfo); ok {
-		return x.OnuPonInfo
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetOnuInfo() *GetOmciEthernetFrameExtendedPmRequest {
-	if x, ok := m.GetRequest().(*GetValueRequest_OnuInfo); ok {
-		return x.OnuInfo
-	}
-	return nil
-}
-
-func (m *GetValueRequest) GetRxPower() *GetRxPowerRequest {
-	if x, ok := m.GetRequest().(*GetValueRequest_RxPower); ok {
-		return x.RxPower
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*GetValueRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*GetValueRequest_Distance)(nil),
-		(*GetValueRequest_UniInfo)(nil),
-		(*GetValueRequest_OltPortInfo)(nil),
-		(*GetValueRequest_OnuOpticalInfo)(nil),
-		(*GetValueRequest_EthBridgePort)(nil),
-		(*GetValueRequest_FecHistory)(nil),
-		(*GetValueRequest_OnuPonInfo)(nil),
-		(*GetValueRequest_OnuInfo)(nil),
-		(*GetValueRequest_RxPower)(nil),
-	}
-}
-
-type GetValueResponse struct {
-	Status    GetValueResponse_Status      `protobuf:"varint,1,opt,name=status,proto3,enum=extension.GetValueResponse_Status" json:"status,omitempty"`
-	ErrReason GetValueResponse_ErrorReason `protobuf:"varint,2,opt,name=errReason,proto3,enum=extension.GetValueResponse_ErrorReason" json:"errReason,omitempty"`
-	// Types that are valid to be assigned to Response:
-	//	*GetValueResponse_Distance
-	//	*GetValueResponse_UniInfo
-	//	*GetValueResponse_PortCoutners
-	//	*GetValueResponse_OnuOpticalInfo
-	//	*GetValueResponse_EthBridgePortInfo
-	//	*GetValueResponse_FecHistory
-	//	*GetValueResponse_OnuPonCounters
-	//	*GetValueResponse_OnuCounters
-	//	*GetValueResponse_RxPower
-	Response             isGetValueResponse_Response `protobuf_oneof:"response"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
-}
-
-func (m *GetValueResponse) Reset()         { *m = GetValueResponse{} }
-func (m *GetValueResponse) String() string { return proto.CompactTextString(m) }
-func (*GetValueResponse) ProtoMessage()    {}
-func (*GetValueResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{20}
-}
-
-func (m *GetValueResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetValueResponse.Unmarshal(m, b)
-}
-func (m *GetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetValueResponse.Marshal(b, m, deterministic)
-}
-func (m *GetValueResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetValueResponse.Merge(m, src)
-}
-func (m *GetValueResponse) XXX_Size() int {
-	return xxx_messageInfo_GetValueResponse.Size(m)
-}
-func (m *GetValueResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetValueResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetValueResponse proto.InternalMessageInfo
-
-func (m *GetValueResponse) GetStatus() GetValueResponse_Status {
-	if m != nil {
-		return m.Status
-	}
-	return GetValueResponse_STATUS_UNDEFINED
-}
-
-func (m *GetValueResponse) GetErrReason() GetValueResponse_ErrorReason {
-	if m != nil {
-		return m.ErrReason
-	}
-	return GetValueResponse_REASON_UNDEFINED
-}
-
-type isGetValueResponse_Response interface {
-	isGetValueResponse_Response()
-}
-
-type GetValueResponse_Distance struct {
-	Distance *GetDistanceResponse `protobuf:"bytes,3,opt,name=distance,proto3,oneof"`
-}
-
-type GetValueResponse_UniInfo struct {
-	UniInfo *GetOnuUniInfoResponse `protobuf:"bytes,4,opt,name=uniInfo,proto3,oneof"`
-}
-
-type GetValueResponse_PortCoutners struct {
-	PortCoutners *GetOltPortCountersResponse `protobuf:"bytes,5,opt,name=portCoutners,proto3,oneof"`
-}
-
-type GetValueResponse_OnuOpticalInfo struct {
-	OnuOpticalInfo *GetOnuPonOpticalInfoResponse `protobuf:"bytes,6,opt,name=onuOpticalInfo,proto3,oneof"`
-}
-
-type GetValueResponse_EthBridgePortInfo struct {
-	EthBridgePortInfo *GetOnuEthernetBridgePortHistoryResponse `protobuf:"bytes,7,opt,name=ethBridgePortInfo,proto3,oneof"`
-}
-
-type GetValueResponse_FecHistory struct {
-	FecHistory *GetOnuFecHistoryResponse `protobuf:"bytes,8,opt,name=fecHistory,proto3,oneof"`
-}
-
-type GetValueResponse_OnuPonCounters struct {
-	OnuPonCounters *GetOnuCountersResponse `protobuf:"bytes,9,opt,name=onuPonCounters,proto3,oneof"`
-}
-
-type GetValueResponse_OnuCounters struct {
-	OnuCounters *GetOmciEthernetFrameExtendedPmResponse `protobuf:"bytes,10,opt,name=onuCounters,proto3,oneof"`
-}
-
-type GetValueResponse_RxPower struct {
-	RxPower *GetRxPowerResponse `protobuf:"bytes,11,opt,name=rxPower,proto3,oneof"`
-}
-
-func (*GetValueResponse_Distance) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_UniInfo) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_PortCoutners) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_OnuOpticalInfo) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_EthBridgePortInfo) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_FecHistory) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_OnuPonCounters) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_OnuCounters) isGetValueResponse_Response() {}
-
-func (*GetValueResponse_RxPower) isGetValueResponse_Response() {}
-
-func (m *GetValueResponse) GetResponse() isGetValueResponse_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetDistance() *GetDistanceResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_Distance); ok {
-		return x.Distance
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetUniInfo() *GetOnuUniInfoResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_UniInfo); ok {
-		return x.UniInfo
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetPortCoutners() *GetOltPortCountersResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_PortCoutners); ok {
-		return x.PortCoutners
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetOnuOpticalInfo() *GetOnuPonOpticalInfoResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_OnuOpticalInfo); ok {
-		return x.OnuOpticalInfo
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetEthBridgePortInfo() *GetOnuEthernetBridgePortHistoryResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_EthBridgePortInfo); ok {
-		return x.EthBridgePortInfo
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetFecHistory() *GetOnuFecHistoryResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_FecHistory); ok {
-		return x.FecHistory
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetOnuPonCounters() *GetOnuCountersResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_OnuPonCounters); ok {
-		return x.OnuPonCounters
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetOnuCounters() *GetOmciEthernetFrameExtendedPmResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_OnuCounters); ok {
-		return x.OnuCounters
-	}
-	return nil
-}
-
-func (m *GetValueResponse) GetRxPower() *GetRxPowerResponse {
-	if x, ok := m.GetResponse().(*GetValueResponse_RxPower); ok {
-		return x.RxPower
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*GetValueResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*GetValueResponse_Distance)(nil),
-		(*GetValueResponse_UniInfo)(nil),
-		(*GetValueResponse_PortCoutners)(nil),
-		(*GetValueResponse_OnuOpticalInfo)(nil),
-		(*GetValueResponse_EthBridgePortInfo)(nil),
-		(*GetValueResponse_FecHistory)(nil),
-		(*GetValueResponse_OnuPonCounters)(nil),
-		(*GetValueResponse_OnuCounters)(nil),
-		(*GetValueResponse_RxPower)(nil),
-	}
-}
-
-type SetValueRequest struct {
-	// Types that are valid to be assigned to Request:
-	//	*SetValueRequest_AlarmConfig
-	Request              isSetValueRequest_Request `protobuf_oneof:"request"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *SetValueRequest) Reset()         { *m = SetValueRequest{} }
-func (m *SetValueRequest) String() string { return proto.CompactTextString(m) }
-func (*SetValueRequest) ProtoMessage()    {}
-func (*SetValueRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{21}
-}
-
-func (m *SetValueRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SetValueRequest.Unmarshal(m, b)
-}
-func (m *SetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SetValueRequest.Marshal(b, m, deterministic)
-}
-func (m *SetValueRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SetValueRequest.Merge(m, src)
-}
-func (m *SetValueRequest) XXX_Size() int {
-	return xxx_messageInfo_SetValueRequest.Size(m)
-}
-func (m *SetValueRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_SetValueRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SetValueRequest proto.InternalMessageInfo
-
-type isSetValueRequest_Request interface {
-	isSetValueRequest_Request()
-}
-
-type SetValueRequest_AlarmConfig struct {
-	AlarmConfig *config.AlarmConfig `protobuf:"bytes,1,opt,name=alarm_config,json=alarmConfig,proto3,oneof"`
-}
-
-func (*SetValueRequest_AlarmConfig) isSetValueRequest_Request() {}
-
-func (m *SetValueRequest) GetRequest() isSetValueRequest_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *SetValueRequest) GetAlarmConfig() *config.AlarmConfig {
-	if x, ok := m.GetRequest().(*SetValueRequest_AlarmConfig); ok {
-		return x.AlarmConfig
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*SetValueRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*SetValueRequest_AlarmConfig)(nil),
-	}
-}
-
-type SetValueResponse struct {
-	Status               SetValueResponse_Status      `protobuf:"varint,1,opt,name=status,proto3,enum=extension.SetValueResponse_Status" json:"status,omitempty"`
-	ErrReason            SetValueResponse_ErrorReason `protobuf:"varint,2,opt,name=errReason,proto3,enum=extension.SetValueResponse_ErrorReason" json:"errReason,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
-}
-
-func (m *SetValueResponse) Reset()         { *m = SetValueResponse{} }
-func (m *SetValueResponse) String() string { return proto.CompactTextString(m) }
-func (*SetValueResponse) ProtoMessage()    {}
-func (*SetValueResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{22}
-}
-
-func (m *SetValueResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SetValueResponse.Unmarshal(m, b)
-}
-func (m *SetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SetValueResponse.Marshal(b, m, deterministic)
-}
-func (m *SetValueResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SetValueResponse.Merge(m, src)
-}
-func (m *SetValueResponse) XXX_Size() int {
-	return xxx_messageInfo_SetValueResponse.Size(m)
-}
-func (m *SetValueResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_SetValueResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SetValueResponse proto.InternalMessageInfo
-
-func (m *SetValueResponse) GetStatus() SetValueResponse_Status {
-	if m != nil {
-		return m.Status
-	}
-	return SetValueResponse_STATUS_UNDEFINED
-}
-
-func (m *SetValueResponse) GetErrReason() SetValueResponse_ErrorReason {
-	if m != nil {
-		return m.ErrReason
-	}
-	return SetValueResponse_REASON_UNDEFINED
-}
-
-type SingleGetValueRequest struct {
-	TargetId             string           `protobuf:"bytes,1,opt,name=targetId,proto3" json:"targetId,omitempty"`
-	Request              *GetValueRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *SingleGetValueRequest) Reset()         { *m = SingleGetValueRequest{} }
-func (m *SingleGetValueRequest) String() string { return proto.CompactTextString(m) }
-func (*SingleGetValueRequest) ProtoMessage()    {}
-func (*SingleGetValueRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{23}
-}
-
-func (m *SingleGetValueRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SingleGetValueRequest.Unmarshal(m, b)
-}
-func (m *SingleGetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SingleGetValueRequest.Marshal(b, m, deterministic)
-}
-func (m *SingleGetValueRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SingleGetValueRequest.Merge(m, src)
-}
-func (m *SingleGetValueRequest) XXX_Size() int {
-	return xxx_messageInfo_SingleGetValueRequest.Size(m)
-}
-func (m *SingleGetValueRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_SingleGetValueRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SingleGetValueRequest proto.InternalMessageInfo
-
-func (m *SingleGetValueRequest) GetTargetId() string {
-	if m != nil {
-		return m.TargetId
-	}
-	return ""
-}
-
-func (m *SingleGetValueRequest) GetRequest() *GetValueRequest {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-type SingleGetValueResponse struct {
-	Response             *GetValueResponse `protobuf:"bytes,1,opt,name=response,proto3" json:"response,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *SingleGetValueResponse) Reset()         { *m = SingleGetValueResponse{} }
-func (m *SingleGetValueResponse) String() string { return proto.CompactTextString(m) }
-func (*SingleGetValueResponse) ProtoMessage()    {}
-func (*SingleGetValueResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{24}
-}
-
-func (m *SingleGetValueResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SingleGetValueResponse.Unmarshal(m, b)
-}
-func (m *SingleGetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SingleGetValueResponse.Marshal(b, m, deterministic)
-}
-func (m *SingleGetValueResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SingleGetValueResponse.Merge(m, src)
-}
-func (m *SingleGetValueResponse) XXX_Size() int {
-	return xxx_messageInfo_SingleGetValueResponse.Size(m)
-}
-func (m *SingleGetValueResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_SingleGetValueResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SingleGetValueResponse proto.InternalMessageInfo
-
-func (m *SingleGetValueResponse) GetResponse() *GetValueResponse {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-type SingleSetValueRequest struct {
-	TargetId             string           `protobuf:"bytes,1,opt,name=targetId,proto3" json:"targetId,omitempty"`
-	Request              *SetValueRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *SingleSetValueRequest) Reset()         { *m = SingleSetValueRequest{} }
-func (m *SingleSetValueRequest) String() string { return proto.CompactTextString(m) }
-func (*SingleSetValueRequest) ProtoMessage()    {}
-func (*SingleSetValueRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{25}
-}
-
-func (m *SingleSetValueRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SingleSetValueRequest.Unmarshal(m, b)
-}
-func (m *SingleSetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SingleSetValueRequest.Marshal(b, m, deterministic)
-}
-func (m *SingleSetValueRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SingleSetValueRequest.Merge(m, src)
-}
-func (m *SingleSetValueRequest) XXX_Size() int {
-	return xxx_messageInfo_SingleSetValueRequest.Size(m)
-}
-func (m *SingleSetValueRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_SingleSetValueRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SingleSetValueRequest proto.InternalMessageInfo
-
-func (m *SingleSetValueRequest) GetTargetId() string {
-	if m != nil {
-		return m.TargetId
-	}
-	return ""
-}
-
-func (m *SingleSetValueRequest) GetRequest() *SetValueRequest {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-type SingleSetValueResponse struct {
-	Response             *SetValueResponse `protobuf:"bytes,1,opt,name=response,proto3" json:"response,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *SingleSetValueResponse) Reset()         { *m = SingleSetValueResponse{} }
-func (m *SingleSetValueResponse) String() string { return proto.CompactTextString(m) }
-func (*SingleSetValueResponse) ProtoMessage()    {}
-func (*SingleSetValueResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ecf6e9799a9202d, []int{26}
-}
-
-func (m *SingleSetValueResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SingleSetValueResponse.Unmarshal(m, b)
-}
-func (m *SingleSetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SingleSetValueResponse.Marshal(b, m, deterministic)
-}
-func (m *SingleSetValueResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SingleSetValueResponse.Merge(m, src)
-}
-func (m *SingleSetValueResponse) XXX_Size() int {
-	return xxx_messageInfo_SingleSetValueResponse.Size(m)
-}
-func (m *SingleSetValueResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_SingleSetValueResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SingleSetValueResponse proto.InternalMessageInfo
-
-func (m *SingleSetValueResponse) GetResponse() *SetValueResponse {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("extension.GetOnuUniInfoResponse_ConfigurationInd", GetOnuUniInfoResponse_ConfigurationInd_name, GetOnuUniInfoResponse_ConfigurationInd_value)
-	proto.RegisterEnum("extension.GetOnuUniInfoResponse_AdministrativeState", GetOnuUniInfoResponse_AdministrativeState_name, GetOnuUniInfoResponse_AdministrativeState_value)
-	proto.RegisterEnum("extension.GetOnuUniInfoResponse_OperationalState", GetOnuUniInfoResponse_OperationalState_name, GetOnuUniInfoResponse_OperationalState_value)
-	proto.RegisterEnum("extension.GetOltPortCounters_PortType", GetOltPortCounters_PortType_name, GetOltPortCounters_PortType_value)
-	proto.RegisterEnum("extension.GetOnuEthernetBridgePortHistory_Direction", GetOnuEthernetBridgePortHistory_Direction_name, GetOnuEthernetBridgePortHistory_Direction_value)
-	proto.RegisterEnum("extension.GetValueResponse_Status", GetValueResponse_Status_name, GetValueResponse_Status_value)
-	proto.RegisterEnum("extension.GetValueResponse_ErrorReason", GetValueResponse_ErrorReason_name, GetValueResponse_ErrorReason_value)
-	proto.RegisterEnum("extension.SetValueResponse_Status", SetValueResponse_Status_name, SetValueResponse_Status_value)
-	proto.RegisterEnum("extension.SetValueResponse_ErrorReason", SetValueResponse_ErrorReason_name, SetValueResponse_ErrorReason_value)
-	proto.RegisterType((*GetDistanceRequest)(nil), "extension.GetDistanceRequest")
-	proto.RegisterType((*GetDistanceResponse)(nil), "extension.GetDistanceResponse")
-	proto.RegisterType((*GetOnuUniInfoRequest)(nil), "extension.GetOnuUniInfoRequest")
-	proto.RegisterType((*GetOnuUniInfoResponse)(nil), "extension.GetOnuUniInfoResponse")
-	proto.RegisterType((*GetOltPortCounters)(nil), "extension.GetOltPortCounters")
-	proto.RegisterType((*GetOltPortCountersResponse)(nil), "extension.GetOltPortCountersResponse")
-	proto.RegisterType((*GetOnuPonOpticalInfo)(nil), "extension.GetOnuPonOpticalInfo")
-	proto.RegisterType((*GetOnuPonOpticalInfoResponse)(nil), "extension.GetOnuPonOpticalInfoResponse")
-	proto.RegisterType((*GetOnuEthernetBridgePortHistory)(nil), "extension.GetOnuEthernetBridgePortHistory")
-	proto.RegisterType((*GetOnuEthernetBridgePortHistoryResponse)(nil), "extension.GetOnuEthernetBridgePortHistoryResponse")
-	proto.RegisterType((*GetOnuFecHistory)(nil), "extension.GetOnuFecHistory")
-	proto.RegisterType((*GetOnuFecHistoryResponse)(nil), "extension.GetOnuFecHistoryResponse")
-	proto.RegisterType((*GetOnuCountersRequest)(nil), "extension.GetOnuCountersRequest")
-	proto.RegisterType((*GetOmciEthernetFrameExtendedPmRequest)(nil), "extension.GetOmciEthernetFrameExtendedPmRequest")
-	proto.RegisterType((*GetRxPowerRequest)(nil), "extension.GetRxPowerRequest")
-	proto.RegisterType((*GetOnuCountersResponse)(nil), "extension.GetOnuCountersResponse")
-	proto.RegisterType((*OmciEthernetFrameExtendedPm)(nil), "extension.OmciEthernetFrameExtendedPm")
-	proto.RegisterType((*GetOmciEthernetFrameExtendedPmResponse)(nil), "extension.GetOmciEthernetFrameExtendedPmResponse")
-	proto.RegisterType((*GetRxPowerResponse)(nil), "extension.GetRxPowerResponse")
-	proto.RegisterType((*GetValueRequest)(nil), "extension.GetValueRequest")
-	proto.RegisterType((*GetValueResponse)(nil), "extension.GetValueResponse")
-	proto.RegisterType((*SetValueRequest)(nil), "extension.SetValueRequest")
-	proto.RegisterType((*SetValueResponse)(nil), "extension.SetValueResponse")
-	proto.RegisterType((*SingleGetValueRequest)(nil), "extension.SingleGetValueRequest")
-	proto.RegisterType((*SingleGetValueResponse)(nil), "extension.SingleGetValueResponse")
-	proto.RegisterType((*SingleSetValueRequest)(nil), "extension.SingleSetValueRequest")
-	proto.RegisterType((*SingleSetValueResponse)(nil), "extension.SingleSetValueResponse")
-}
-
-func init() { proto.RegisterFile("voltha_protos/extensions.proto", fileDescriptor_7ecf6e9799a9202d) }
-
-var fileDescriptor_7ecf6e9799a9202d = []byte{
-	// 2978 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x5a, 0xcb, 0x72, 0x1b, 0xc7,
-	0xd5, 0x26, 0x20, 0x72, 0x08, 0x1c, 0x10, 0x24, 0xd8, 0xbc, 0x41, 0xa4, 0x6e, 0x1e, 0x97, 0x25,
-	0xfd, 0x2e, 0x1b, 0x12, 0x60, 0x42, 0xe6, 0x2f, 0x3b, 0x55, 0xe6, 0x10, 0x43, 0x02, 0x25, 0x0a,
-	0x80, 0x1b, 0x80, 0xec, 0x64, 0x33, 0x35, 0xc4, 0x34, 0xa9, 0x29, 0x03, 0x33, 0xc8, 0x4c, 0x43,
-	0x86, 0xb2, 0x4d, 0x96, 0xd9, 0x65, 0x93, 0x67, 0x48, 0x55, 0x2a, 0x8b, 0xa4, 0x2a, 0xfb, 0xac,
-	0xf3, 0x12, 0xa9, 0xca, 0x13, 0xe4, 0x01, 0x52, 0xa9, 0xbe, 0xcc, 0x15, 0x20, 0x25, 0x39, 0xd9,
-	0xb1, 0xcf, 0xf9, 0xbe, 0x6f, 0x7a, 0xba, 0xcf, 0x39, 0x7d, 0xa6, 0x41, 0xb8, 0xf7, 0xc6, 0x1d,
-	0xd1, 0xd7, 0xa6, 0x31, 0xf1, 0x5c, 0xea, 0xfa, 0x4f, 0xc8, 0x8c, 0x12, 0xc7, 0xb7, 0x5d, 0xc7,
-	0xaf, 0x70, 0x0b, 0xca, 0x87, 0x96, 0xfd, 0x79, 0xa8, 0x31, 0x74, 0x9d, 0x4b, 0xfb, 0x4a, 0x40,
-	0xf7, 0x0f, 0xae, 0x5c, 0xf7, 0x6a, 0x44, 0x9e, 0xf0, 0xd1, 0xc5, 0xf4, 0xf2, 0x09, 0x19, 0x4f,
-	0xe8, 0x5b, 0xe1, 0x54, 0x9f, 0x01, 0x3a, 0x23, 0xb4, 0x61, 0xfb, 0xd4, 0x74, 0x86, 0x04, 0x93,
-	0x5f, 0x4e, 0x89, 0x4f, 0xd1, 0x03, 0x28, 0xb8, 0xce, 0xb4, 0x41, 0xde, 0xd8, 0x43, 0xd2, 0xb2,
-	0xca, 0x99, 0x07, 0x99, 0xc7, 0x79, 0x1c, 0x37, 0xa9, 0x55, 0xd8, 0x4a, 0xf0, 0xfc, 0x89, 0xeb,
-	0xf8, 0x04, 0xed, 0x43, 0xce, 0x92, 0x36, 0xce, 0x2a, 0xe2, 0x70, 0xac, 0xd6, 0x60, 0xfb, 0x8c,
-	0xd0, 0x8e, 0x33, 0x1d, 0x38, 0x76, 0xcb, 0xb9, 0x74, 0x83, 0x87, 0xed, 0x43, 0x6e, 0xca, 0x2c,
-	0x16, 0x99, 0x05, 0x9c, 0x60, 0xac, 0xfe, 0x63, 0x19, 0x76, 0x52, 0x24, 0xf9, 0xa4, 0x2e, 0xe4,
-	0x4c, 0x6b, 0xdc, 0xa3, 0x26, 0x15, 0x4f, 0x5a, 0xaf, 0x1d, 0x56, 0xc2, 0x35, 0xa9, 0x2c, 0xe4,
-	0x54, 0x8e, 0xad, 0xb1, 0xed, 0xd8, 0x3e, 0xf5, 0x4c, 0x6a, 0xbf, 0x21, 0x9c, 0x8b, 0x43, 0x15,
-	0xd4, 0x81, 0xbc, 0x3b, 0x21, 0x9e, 0x90, 0xcc, 0x72, 0xc9, 0xea, 0x3b, 0x25, 0x3b, 0x13, 0xc2,
-	0xd4, 0x5c, 0xc7, 0x1c, 0x09, 0xbd, 0x48, 0x83, 0x09, 0x8a, 0x8d, 0x68, 0x39, 0x56, 0xf9, 0xd6,
-	0x7b, 0x0a, 0x9e, 0x70, 0xc6, 0x54, 0x88, 0xb6, 0x1c, 0x0b, 0x47, 0x1a, 0xea, 0xdf, 0x32, 0x50,
-	0x4a, 0xfb, 0x11, 0x80, 0x32, 0x68, 0xbf, 0xe8, 0x7c, 0xd7, 0x2e, 0x2d, 0x21, 0x04, 0xeb, 0x7d,
-	0xbd, 0x6d, 0x68, 0xc7, 0x3d, 0xdd, 0xe8, 0x1b, 0xa7, 0x8d, 0xef, 0x4b, 0x19, 0xb4, 0x0b, 0xa8,
-	0x39, 0x68, 0x37, 0xb0, 0xde, 0x88, 0xdb, 0xb3, 0xa8, 0x0c, 0xdb, 0x67, 0xad, 0xb3, 0x63, 0xad,
-	0xd5, 0x37, 0xf4, 0x7e, 0x53, 0xc7, 0x6d, 0x5d, 0x78, 0x6e, 0x31, 0x06, 0x53, 0x39, 0x4b, 0xda,
-	0x97, 0x53, 0xea, 0xcd, 0xc6, 0xf7, 0xa5, 0x95, 0x05, 0xea, 0xcc, 0xae, 0x2c, 0x54, 0x67, 0x9e,
-	0x55, 0xf5, 0x0c, 0xb6, 0x16, 0xec, 0x03, 0x13, 0x3a, 0x6e, 0xbc, 0xec, 0xf5, 0x8f, 0xfb, 0xba,
-	0x31, 0x68, 0x37, 0xf4, 0xd3, 0x56, 0x5b, 0x6f, 0x94, 0x96, 0xd8, 0xeb, 0x9d, 0x77, 0x4e, 0x5e,
-	0xe8, 0x8d, 0x52, 0x06, 0xad, 0x41, 0x6e, 0xd0, 0x96, 0xa3, 0xac, 0x7a, 0x0a, 0xa5, 0xf4, 0xea,
-	0xa3, 0x3d, 0xd8, 0xea, 0x74, 0x75, 0x3c, 0x2f, 0x53, 0x80, 0x55, 0xbd, 0x7d, 0xac, 0x9d, 0x07,
-	0x3a, 0x8d, 0x56, 0x4f, 0x8c, 0xb2, 0xea, 0x5f, 0x33, 0x3c, 0x07, 0x3a, 0x23, 0xda, 0x75, 0x3d,
-	0x7a, 0xe2, 0x4e, 0x1d, 0x4a, 0x3c, 0x1f, 0xed, 0x82, 0x32, 0x71, 0x3d, 0xda, 0x76, 0x65, 0x50,
-	0xca, 0x11, 0xd2, 0x20, 0xc7, 0xfe, 0xea, 0xbf, 0x9d, 0x04, 0x51, 0xf2, 0x30, 0xb5, 0xa9, 0x49,
-	0xa1, 0x4a, 0x57, 0xa2, 0x71, 0xc8, 0x53, 0x75, 0xc8, 0x05, 0x56, 0x54, 0x82, 0x35, 0xf6, 0xb7,
-	0x31, 0x68, 0xbf, 0x68, 0x8b, 0x5d, 0xdc, 0x81, 0x4d, 0x6e, 0x09, 0x17, 0xae, 0xdd, 0x6e, 0x95,
-	0x32, 0x21, 0xb0, 0xdb, 0x69, 0x1b, 0x9d, 0xf3, 0x7e, 0x29, 0xab, 0xfe, 0xfd, 0x16, 0xec, 0xcf,
-	0x3f, 0x30, 0x4c, 0x91, 0x32, 0xac, 0xd2, 0x99, 0xf6, 0x96, 0x12, 0x9f, 0xbf, 0xc2, 0x32, 0x0e,
-	0x86, 0xcc, 0xe3, 0x49, 0x4f, 0x56, 0x78, 0xe4, 0x10, 0xdd, 0x81, 0x3c, 0x9d, 0x75, 0xcd, 0xe1,
-	0x0f, 0x84, 0xfa, 0x3c, 0x66, 0x97, 0x71, 0x64, 0x60, 0x5e, 0x2f, 0xf4, 0x2e, 0x0b, 0x6f, 0x68,
-	0x40, 0x0f, 0x61, 0x9d, 0xce, 0x74, 0xcf, 0x73, 0xbd, 0x00, 0xb2, 0xc2, 0x21, 0x29, 0x2b, 0xc3,
-	0x79, 0x49, 0x9c, 0x22, 0x70, 0xde, 0x1c, 0x8e, 0xce, 0xb4, 0xa1, 0xe9, 0xd3, 0x00, 0xb7, 0x1a,
-	0xe8, 0xc5, 0xad, 0x42, 0x2f, 0x81, 0xcb, 0x05, 0x7a, 0x69, 0x1c, 0x9d, 0x0d, 0xe2, 0xb8, 0x7c,
-	0xa0, 0x37, 0x98, 0xd3, 0x4b, 0xe0, 0x20, 0xd0, 0x1b, 0xcc, 0xe9, 0xbd, 0x8c, 0xe3, 0x0a, 0x81,
-	0xde, 0xcb, 0x39, 0xbd, 0x04, 0x6e, 0x2d, 0xd0, 0x8b, 0x5b, 0xd5, 0x46, 0x50, 0x20, 0xbb, 0xae,
-	0xd3, 0x99, 0x50, 0x7b, 0x68, 0x8e, 0x58, 0x69, 0x40, 0x9f, 0xc1, 0x0a, 0x2f, 0xd9, 0x7c, 0x17,
-	0x0b, 0xb5, 0xdd, 0x8a, 0x28, 0xe8, 0x95, 0xa0, 0xa0, 0x57, 0x74, 0xe6, 0xc5, 0x02, 0xa4, 0xfe,
-	0x26, 0x0b, 0x77, 0x16, 0xc9, 0x84, 0x61, 0xf1, 0x29, 0x94, 0x26, 0xee, 0x8f, 0xc4, 0x3b, 0x25,
-	0xc4, 0x7a, 0xe5, 0x8e, 0xa8, 0x79, 0x25, 0x2a, 0x68, 0x16, 0xcf, 0xd9, 0x51, 0x0d, 0xb6, 0x3d,
-	0x32, 0x24, 0xf6, 0x1b, 0x62, 0x49, 0xa9, 0x2e, 0x83, 0xf0, 0xa8, 0xc9, 0xe2, 0x85, 0x3e, 0xf4,
-	0x0c, 0x76, 0xc7, 0xc4, 0x0c, 0x1e, 0x7d, 0x6e, 0x4e, 0x9d, 0xe1, 0x6b, 0xc1, 0xba, 0xc5, 0x59,
-	0xd7, 0x78, 0xd9, 0xbc, 0x46, 0xa6, 0x4f, 0x3c, 0xcd, 0x36, 0xfd, 0x93, 0xa9, 0xe7, 0x11, 0x87,
-	0xf2, 0x18, 0xcb, 0xe2, 0x39, 0x3b, 0x3b, 0xa0, 0x28, 0x19, 0xf3, 0xec, 0x9f, 0x7a, 0x84, 0xc7,
-	0x59, 0x16, 0xc7, 0x4d, 0xea, 0x9f, 0x32, 0x70, 0x5f, 0x2c, 0x83, 0x4e, 0x5f, 0x13, 0xcf, 0x21,
-	0x54, 0xf3, 0x6c, 0xeb, 0x8a, 0xb0, 0x4c, 0x69, 0xda, 0x3e, 0x75, 0xbd, 0xb7, 0x08, 0x43, 0xde,
-	0xb2, 0x3d, 0x32, 0x64, 0x15, 0xe4, 0xda, 0x43, 0xe4, 0x5a, 0x7a, 0xa5, 0x11, 0x70, 0x71, 0x24,
-	0xa3, 0x1e, 0x41, 0x3e, 0xb4, 0xa3, 0x22, 0xe4, 0xe3, 0x45, 0x88, 0xd5, 0xaf, 0x6e, 0xaf, 0x8f,
-	0xf5, 0xe3, 0x97, 0xa5, 0x0c, 0x5a, 0x07, 0x68, 0x74, 0xbe, 0x6b, 0xcb, 0x71, 0x56, 0xfd, 0xdd,
-	0x0a, 0x3c, 0x7a, 0xc7, 0x23, 0xc3, 0x3d, 0xbc, 0x07, 0x60, 0x79, 0xee, 0x44, 0x7f, 0x43, 0x1c,
-	0xea, 0xcb, 0x02, 0x15, 0xb3, 0xb0, 0xe2, 0xe5, 0x0e, 0x29, 0x0b, 0xb5, 0xac, 0x28, 0x5e, 0x62,
-	0xc4, 0x12, 0x7f, 0x12, 0x4b, 0xee, 0x22, 0x0e, 0x86, 0x6c, 0xf5, 0x2f, 0x3c, 0xd7, 0xb4, 0xe2,
-	0x61, 0xba, 0xcc, 0x21, 0x73, 0x76, 0x86, 0x1d, 0x4f, 0x47, 0x6c, 0x03, 0x23, 0xec, 0x8a, 0xc0,
-	0xa6, 0xed, 0xe8, 0x33, 0xd8, 0x1c, 0x7a, 0x43, 0x9e, 0xd7, 0xc4, 0x8a, 0xe7, 0x7b, 0x11, 0xcf,
-	0x3b, 0x98, 0xf2, 0xd4, 0xb1, 0x88, 0xe7, 0xdb, 0xbf, 0x22, 0xf1, 0xa4, 0x2f, 0xe2, 0x39, 0x3b,
-	0x7a, 0x0c, 0x1b, 0xee, 0x9b, 0x24, 0x34, 0xc7, 0xa1, 0x69, 0x33, 0x43, 0xca, 0xd7, 0x7c, 0x76,
-	0x28, 0x97, 0x25, 0x2f, 0x90, 0x29, 0x33, 0x8b, 0xf7, 0xc0, 0x54, 0xef, 0xbb, 0xd5, 0xda, 0x97,
-	0x12, 0x0e, 0x1c, 0xbe, 0xd0, 0x87, 0x0e, 0x61, 0x47, 0xda, 0xab, 0xb5, 0xa3, 0xbe, 0x5b, 0xab,
-	0xd7, 0x3b, 0x82, 0x54, 0xe0, 0xa4, 0xc5, 0xce, 0x18, 0xab, 0x56, 0x7f, 0xd6, 0x77, 0xeb, 0xd5,
-	0xaa, 0x7c, 0xd4, 0x5a, 0x82, 0x95, 0x74, 0xb2, 0xdc, 0x92, 0x8e, 0x7a, 0xb5, 0xd6, 0x77, 0xab,
-	0x4f, 0x6b, 0x5f, 0x48, 0x5a, 0x91, 0xd3, 0xae, 0xf1, 0xa2, 0x23, 0xd8, 0x0b, 0xa6, 0xf1, 0xb4,
-	0x76, 0xd8, 0x77, 0xab, 0xf5, 0xea, 0x91, 0x24, 0xae, 0x73, 0xe2, 0x75, 0x6e, 0xf5, 0x1b, 0x28,
-	0x89, 0xa0, 0x3c, 0x25, 0xc3, 0x20, 0x6f, 0x3e, 0xac, 0x20, 0xfd, 0x2b, 0x03, 0xe5, 0xb4, 0x44,
-	0x18, 0xc8, 0x0f, 0x61, 0x7d, 0xe8, 0x7a, 0x2c, 0x5f, 0x88, 0x15, 0x1d, 0x55, 0x45, 0x9c, 0xb2,
-	0xa2, 0x0a, 0xa0, 0xd0, 0x72, 0xe2, 0x5a, 0xe4, 0x3b, 0xd7, 0xb3, 0x82, 0xe0, 0x5e, 0xe0, 0x61,
-	0x09, 0x72, 0x49, 0x86, 0x3d, 0x32, 0x74, 0x1d, 0x2b, 0x88, 0xf5, 0x98, 0x85, 0xd7, 0x6e, 0x97,
-	0x9a, 0xa3, 0x48, 0x4b, 0x04, 0x7b, 0xca, 0xca, 0x16, 0x7c, 0xea, 0x48, 0x7d, 0xf3, 0x62, 0x44,
-	0x22, 0xbc, 0x08, 0xf8, 0x6b, 0xbc, 0xea, 0x59, 0xd0, 0xb7, 0x46, 0xa7, 0xb2, 0xe8, 0x76, 0xf7,
-	0x60, 0xd5, 0x76, 0xe8, 0xa5, 0x61, 0x8b, 0xb6, 0x7a, 0x15, 0x2b, 0x6c, 0xd8, 0xb2, 0xd0, 0x0e,
-	0x28, 0xae, 0x33, 0x65, 0xf6, 0x2c, 0xb7, 0xaf, 0xb8, 0xce, 0xb4, 0x65, 0xa9, 0x2d, 0xf8, 0x84,
-	0x09, 0x8d, 0x87, 0x76, 0x50, 0x15, 0x4e, 0x3d, 0x73, 0x4c, 0x74, 0x56, 0xa5, 0x2c, 0x62, 0x75,
-	0xc7, 0xef, 0xdf, 0xb3, 0x9f, 0xc0, 0xe6, 0x19, 0xa1, 0x78, 0xc6, 0xcb, 0xed, 0x4f, 0x9d, 0xcf,
-	0x3f, 0xd7, 0x60, 0x37, 0xfd, 0x66, 0x72, 0x2f, 0x6f, 0xa7, 0xa4, 0x9a, 0x4b, 0xa1, 0xd8, 0x5e,
-	0x52, 0xac, 0x99, 0x91, 0x72, 0xe8, 0x11, 0xac, 0x4f, 0x5c, 0xdf, 0x66, 0x7d, 0xa0, 0x61, 0x79,
-	0xf6, 0x25, 0xe5, 0x7b, 0xa5, 0x34, 0xb3, 0xb8, 0x18, 0xd8, 0x1b, 0xcc, 0xcc, 0x80, 0x0e, 0xb9,
-	0x32, 0x63, 0xc0, 0x65, 0x0e, 0xbc, 0x85, 0x8b, 0x81, 0x5d, 0x00, 0x9f, 0x43, 0xd9, 0x22, 0x23,
-	0x7b, 0x6c, 0x53, 0xe2, 0x19, 0x63, 0xdb, 0xf7, 0x0d, 0x8b, 0x50, 0x59, 0xe3, 0x57, 0x38, 0x65,
-	0x19, 0xef, 0x86, 0x88, 0x97, 0xb6, 0xef, 0x37, 0x02, 0x3f, 0xba, 0x0f, 0x70, 0x61, 0x4f, 0x0c,
-	0xc2, 0x8a, 0x92, 0xa8, 0x52, 0x4a, 0x73, 0x05, 0xe7, 0x2f, 0xec, 0x09, 0xaf, 0x53, 0x3e, 0xba,
-	0x0b, 0x6c, 0x60, 0x4c, 0x1d, 0x5b, 0x16, 0x26, 0xa5, 0xa9, 0xe0, 0xdc, 0x85, 0x3d, 0x19, 0x30,
-	0x0b, 0x4b, 0xea, 0x4b, 0x32, 0x34, 0xc2, 0x78, 0x34, 0xfc, 0xb7, 0xe3, 0x0b, 0x77, 0x24, 0x0a,
-	0x93, 0xd2, 0x5c, 0xc5, 0x5b, 0x97, 0x64, 0x78, 0x12, 0x78, 0x7b, 0xc2, 0xc9, 0x92, 0x53, 0xb0,
-	0x2c, 0xf2, 0x23, 0x0b, 0x9e, 0x88, 0xcf, 0xcb, 0x94, 0xd2, 0xcc, 0xe1, 0x1d, 0xce, 0x93, 0xfe,
-	0x50, 0x00, 0x7d, 0x03, 0x07, 0x49, 0x66, 0x22, 0x1a, 0x79, 0xd5, 0x52, 0x9a, 0x79, 0x7c, 0x3b,
-	0xce, 0x1e, 0xc4, 0x21, 0xe8, 0x13, 0x28, 0x26, 0x14, 0x78, 0xd1, 0x52, 0x9a, 0x80, 0xd7, 0xe2,
-	0x1c, 0xf4, 0x14, 0xb6, 0x92, 0x2f, 0x26, 0x56, 0x60, 0x8d, 0x83, 0x0b, 0x78, 0x33, 0xfe, 0x5a,
-	0x62, 0x29, 0x1e, 0xc3, 0xc6, 0xec, 0x8a, 0x8c, 0x8d, 0x1f, 0xc8, 0xdb, 0x60, 0x3d, 0x8b, 0x1c,
-	0xbd, 0x86, 0x8b, 0xcc, 0xf1, 0x82, 0xbc, 0x8d, 0xd6, 0x94, 0x23, 0x47, 0xae, 0x2f, 0xaa, 0x91,
-	0xd2, 0x2c, 0xe2, 0x1c, 0x33, 0x9d, 0xbb, 0x3e, 0x17, 0xf2, 0x66, 0xc6, 0x64, 0xe4, 0x9a, 0x63,
-	0x5f, 0x28, 0x95, 0x37, 0x38, 0x68, 0x1d, 0x17, 0xbd, 0x59, 0x97, 0xdb, 0xb9, 0x12, 0xfa, 0x1c,
-	0x50, 0x84, 0x74, 0x5c, 0xc7, 0xb0, 0xad, 0x11, 0x29, 0x97, 0x38, 0x78, 0x03, 0x6f, 0x04, 0xe0,
-	0xb6, 0xeb, 0xb4, 0xac, 0x11, 0x0f, 0x57, 0x6f, 0x66, 0xb8, 0xe3, 0xa1, 0x5d, 0xde, 0xe4, 0x98,
-	0x12, 0x56, 0xbc, 0x19, 0xcb, 0x34, 0xe6, 0xa2, 0xd2, 0x85, 0xb8, 0x6b, 0x13, 0x2b, 0x54, 0xb8,
-	0x9e, 0xc3, 0x6d, 0xc9, 0x32, 0x64, 0xc9, 0x34, 0x86, 0xde, 0x50, 0x4e, 0x6c, 0x8b, 0x83, 0x11,
-	0xde, 0x11, 0x3a, 0xf2, 0xfc, 0x39, 0x91, 0xc7, 0x1c, 0x3a, 0x80, 0x9c, 0x37, 0x33, 0x2e, 0x78,
-	0x99, 0xdb, 0xe6, 0xd0, 0xad, 0xa8, 0xf3, 0xbe, 0x0f, 0xc0, 0x66, 0x2f, 0x4f, 0xb2, 0x1d, 0xee,
-	0xde, 0x8e, 0xb7, 0xd7, 0x07, 0x90, 0xa3, 0x01, 0x7b, 0x97, 0xbb, 0x77, 0xa2, 0x8e, 0xfe, 0x3e,
-	0x00, 0x8d, 0xd8, 0x7b, 0xdc, 0xbd, 0x1b, 0x6f, 0xdd, 0x3f, 0x86, 0xb5, 0x0b, 0xe2, 0x19, 0x1e,
-	0x61, 0x1f, 0x21, 0xc4, 0x2a, 0x97, 0x39, 0x64, 0x0f, 0x17, 0x2e, 0x58, 0x2d, 0x10, 0x46, 0xf4,
-	0x11, 0x14, 0x46, 0x43, 0xeb, 0x2a, 0xd8, 0xb0, 0xdb, 0x1c, 0x53, 0xc6, 0xc0, 0x8c, 0x72, 0xb7,
-	0xd8, 0x34, 0x2d, 0x3b, 0x40, 0xec, 0x73, 0xc4, 0x6d, 0x9c, 0xf7, 0x2c, 0x5b, 0x02, 0xee, 0x41,
-	0x9e, 0xda, 0x63, 0xe2, 0x53, 0x73, 0x3c, 0x29, 0x1f, 0xf0, 0x6c, 0xdf, 0xc7, 0x91, 0x49, 0x5b,
-	0x03, 0xb0, 0x7d, 0x43, 0x16, 0x0a, 0xad, 0x00, 0x79, 0xdb, 0x37, 0x44, 0x6d, 0xd0, 0xb6, 0x60,
-	0xd3, 0xf6, 0x8d, 0x64, 0x3d, 0x90, 0xc6, 0x64, 0xee, 0x6b, 0x77, 0xe1, 0xc0, 0x66, 0x89, 0xbd,
-	0x38, 0xcf, 0xb5, 0x0d, 0x28, 0xda, 0xbe, 0x11, 0xa5, 0xb2, 0xb6, 0x0e, 0x6b, 0xd2, 0xc0, 0x03,
-	0x57, 0xdb, 0x87, 0xb2, 0xed, 0x1b, 0x0b, 0x73, 0x55, 0xbb, 0x03, 0xfb, 0xa1, 0x6f, 0x2e, 0x23,
-	0xb5, 0x07, 0x70, 0x6f, 0xce, 0x9b, 0xc8, 0x3a, 0x0d, 0x41, 0x29, 0x8d, 0xd0, 0xca, 0xb0, 0x3b,
-	0xf7, 0x3c, 0x31, 0x93, 0x6d, 0x40, 0xb6, 0x6f, 0xa4, 0x52, 0x45, 0xce, 0x37, 0x4c, 0x0b, 0x89,
-	0x4a, 0xe5, 0x81, 0xb6, 0x07, 0x3b, 0x09, 0x6b, 0x10, 0xf3, 0x72, 0x8d, 0x65, 0x9c, 0xca, 0x91,
-	0x0c, 0x68, 0xed, 0x1e, 0xdc, 0x89, 0x7c, 0xf3, 0x31, 0xac, 0x15, 0xa1, 0x20, 0xfc, 0x3c, 0xd2,
-	0xe4, 0x52, 0x46, 0x91, 0x29, 0xfd, 0x34, 0xe9, 0x8f, 0x62, 0x4f, 0xdb, 0x84, 0x0d, 0xb6, 0xd4,
-	0xb1, 0x58, 0xd3, 0x4a, 0xb0, 0x6e, 0xfb, 0x46, 0x2c, 0xb2, 0x02, 0xd5, 0x30, 0x90, 0xe4, 0x0b,
-	0x87, 0x51, 0xa2, 0xfe, 0x76, 0x05, 0x0e, 0x6e, 0x38, 0xf4, 0xd0, 0x7d, 0x28, 0xb0, 0x76, 0xd7,
-	0x20, 0x51, 0x07, 0xac, 0xdc, 0xd0, 0x01, 0x2b, 0x61, 0x07, 0xbc, 0x0b, 0xca, 0x25, 0xd3, 0x12,
-	0x4d, 0x81, 0x82, 0xe5, 0x08, 0xfd, 0x5f, 0xac, 0xff, 0x35, 0x24, 0x82, 0x9f, 0x30, 0x78, 0x23,
-	0xb4, 0x9f, 0x86, 0xd0, 0xb0, 0xcd, 0x0d, 0xa0, 0x2b, 0x02, 0x1a, 0xda, 0x25, 0xf4, 0x33, 0x40,
-	0xe1, 0xca, 0x12, 0x2b, 0x00, 0xf3, 0x83, 0x05, 0x97, 0xa2, 0xf6, 0x37, 0x12, 0x0e, 0xbb, 0xdc,
-	0x00, 0xbb, 0x2a, 0x84, 0x43, 0xbb, 0x84, 0x3e, 0x8a, 0x9a, 0xdf, 0x00, 0xc9, 0xcf, 0x18, 0xbc,
-	0x1e, 0x98, 0x25, 0xf0, 0x31, 0x94, 0x84, 0xdf, 0x78, 0x76, 0x68, 0xc4, 0x9a, 0x5f, 0x05, 0xaf,
-	0x0b, 0xfb, 0xb3, 0xc3, 0xb0, 0x23, 0xdd, 0x0b, 0x90, 0x75, 0x83, 0xba, 0x46, 0xb5, 0xf6, 0xa5,
-	0x11, 0x6b, 0x7f, 0x15, 0xbc, 0x25, 0x09, 0xa2, 0xfb, 0xed, 0x04, 0x1d, 0x69, 0x59, 0xb2, 0xaa,
-	0xb5, 0x23, 0x46, 0xab, 0xd5, 0xeb, 0x01, 0x8d, 0x9f, 0x25, 0x78, 0x5b, 0xf8, 0x53, 0xfd, 0x6f,
-	0xc4, 0xab, 0xd5, 0x9f, 0x31, 0x5e, 0xbd, 0x5a, 0x35, 0x62, 0x2d, 0x70, 0xc8, 0x0b, 0x3a, 0xe0,
-	0x4e, 0xd0, 0xc9, 0xde, 0x96, 0xbc, 0x7a, 0xb5, 0xc6, 0xa7, 0xf9, 0xb4, 0xf6, 0x85, 0x11, 0x6b,
-	0x82, 0x15, 0xbc, 0x23, 0x00, 0x61, 0x0f, 0x2c, 0x99, 0xcf, 0x61, 0x3f, 0x98, 0xe9, 0xd3, 0xda,
-	0x21, 0xa7, 0xd6, 0xab, 0x47, 0x46, 0xac, 0x0d, 0x56, 0xf0, 0xae, 0x9c, 0x6b, 0xd8, 0x05, 0x0b,
-	0xae, 0xfa, 0x97, 0x0c, 0x3c, 0x7c, 0x57, 0x1b, 0x26, 0xbb, 0x20, 0x0d, 0x72, 0xd3, 0x89, 0x4f,
-	0x3d, 0x62, 0x8e, 0x65, 0x7f, 0x1c, 0xbf, 0x1f, 0xba, 0x49, 0x21, 0xe4, 0xa1, 0x53, 0x00, 0xcb,
-	0xfd, 0xd1, 0x91, 0x2a, 0xd9, 0x0f, 0x52, 0x89, 0x31, 0xd5, 0xdf, 0x8b, 0xab, 0xad, 0xb0, 0xe5,
-	0x93, 0x53, 0xfc, 0xc0, 0x9e, 0x8f, 0xe5, 0x8c, 0x4f, 0x4d, 0x3a, 0x15, 0x39, 0x93, 0xc7, 0x72,
-	0xc4, 0x92, 0xf0, 0xd2, 0xb4, 0x47, 0x86, 0x47, 0x4c, 0xdf, 0x75, 0x78, 0xba, 0xe4, 0x31, 0x30,
-	0x13, 0xe6, 0x16, 0x74, 0x9b, 0x1f, 0x78, 0xfc, 0x56, 0x81, 0x67, 0x48, 0x86, 0x1d, 0x77, 0x7c,
-	0x2e, 0xea, 0xaf, 0x57, 0x60, 0xe3, 0x8c, 0xd0, 0x57, 0xe6, 0x68, 0x1a, 0x5e, 0x3b, 0x7f, 0x95,
-	0xba, 0x3d, 0x2e, 0xd4, 0xee, 0x26, 0x3f, 0xc7, 0x53, 0xf7, 0xd4, 0xcd, 0xa5, 0xe8, 0x7a, 0x19,
-	0x7d, 0x05, 0xab, 0x53, 0x71, 0x97, 0x2a, 0x17, 0xec, 0xfe, 0xf5, 0x77, 0xad, 0x01, 0x3b, 0x60,
-	0xa0, 0x63, 0x28, 0xb8, 0xe2, 0x16, 0x8d, 0x0b, 0xdc, 0x5a, 0xf4, 0xf0, 0xd4, 0x35, 0x5b, 0x73,
-	0x09, 0xc7, 0x39, 0xa8, 0x05, 0xeb, 0xae, 0x33, 0x8d, 0x5d, 0xb8, 0xf0, 0xf5, 0x58, 0x34, 0x8d,
-	0xe4, 0xbd, 0x4c, 0x73, 0x09, 0xa7, 0x88, 0x08, 0x43, 0x91, 0xd0, 0xd7, 0xd1, 0xd7, 0x3f, 0x5f,
-	0xbb, 0x42, 0xed, 0xd3, 0xf7, 0xbf, 0x9b, 0x68, 0x2e, 0xe1, 0xa4, 0x04, 0xfa, 0x19, 0xff, 0x20,
-	0x92, 0x6e, 0x5e, 0x81, 0x0a, 0xb5, 0x83, 0x39, 0xc1, 0xe8, 0x0b, 0xad, 0xb9, 0x84, 0x63, 0x04,
-	0xa4, 0x01, 0xb8, 0x7c, 0xe6, 0xfc, 0xcd, 0x56, 0x39, 0xfd, 0xc1, 0x1c, 0x3d, 0xf5, 0xb1, 0xc3,
-	0x34, 0x22, 0x16, 0x3a, 0x87, 0x55, 0x16, 0x4f, 0x4c, 0x20, 0xc7, 0x05, 0x9e, 0xa6, 0x04, 0xde,
-	0xf9, 0x91, 0xc3, 0xb6, 0x4c, 0x4a, 0xa0, 0x23, 0x08, 0x62, 0x89, 0xd7, 0xb3, 0x42, 0xed, 0x4e,
-	0x52, 0x2d, 0xf9, 0x9d, 0xc3, 0x98, 0x12, 0xae, 0xe5, 0x61, 0xd5, 0x13, 0x56, 0xf5, 0x0f, 0x39,
-	0xfe, 0x79, 0x2b, 0xa3, 0x50, 0xa6, 0xc7, 0xf3, 0x30, 0xdc, 0xc5, 0x9d, 0x90, 0x9a, 0x14, 0x4e,
-	0x80, 0x2b, 0x3d, 0x8e, 0x0c, 0x53, 0x42, 0x87, 0x3c, 0xf1, 0x3c, 0x11, 0xfe, 0xf2, 0x7a, 0xf8,
-	0xd1, 0x4d, 0x74, 0x7e, 0x00, 0x08, 0x38, 0x8e, 0x98, 0xe8, 0xeb, 0x58, 0x26, 0x88, 0x60, 0xbc,
-	0x77, 0x5d, 0x26, 0x08, 0xa1, 0x44, 0x2a, 0x7c, 0x1d, 0xa5, 0xc2, 0xf2, 0x35, 0x3b, 0x95, 0xfa,
-	0xd9, 0x21, 0x9e, 0x0b, 0x2f, 0x60, 0x6d, 0x22, 0xe2, 0x9c, 0x3a, 0xc4, 0xf3, 0x65, 0xf0, 0x7d,
-	0x72, 0x63, 0x32, 0xc4, 0x74, 0x12, 0x64, 0xf4, 0xed, 0x5c, 0x56, 0x88, 0xd0, 0x7b, 0xf4, 0x8e,
-	0xac, 0x88, 0x09, 0xa6, 0xb3, 0xe3, 0x02, 0x36, 0x13, 0xa1, 0x1d, 0x8b, 0xc8, 0xda, 0xfb, 0x67,
-	0x48, 0xec, 0x01, 0xf3, 0x72, 0x48, 0x4f, 0x64, 0x8b, 0x88, 0xd6, 0x8f, 0x6f, 0xc8, 0x96, 0x98,
-	0x5a, 0x3c, 0x6b, 0x5e, 0xf0, 0xb7, 0xef, 0xba, 0x4e, 0xb0, 0x4e, 0x32, 0x54, 0x3f, 0xba, 0x21,
-	0x73, 0x12, 0xef, 0x1d, 0xa3, 0xa2, 0x01, 0xff, 0xc0, 0x0f, 0x95, 0x80, 0x2b, 0x55, 0x3f, 0x20,
-	0x85, 0x42, 0xe5, 0xb8, 0x0e, 0xfa, 0xff, 0x28, 0x8f, 0x0a, 0x8b, 0xca, 0x5e, 0xea, 0xf0, 0x88,
-	0x25, 0x92, 0x5a, 0x05, 0x45, 0x84, 0x3f, 0xda, 0x86, 0x52, 0xaf, 0x7f, 0xdc, 0x1f, 0xf4, 0x12,
-	0x3f, 0xba, 0x28, 0x90, 0xed, 0xbc, 0x28, 0x65, 0x50, 0x1e, 0x56, 0x74, 0x8c, 0x3b, 0xb8, 0x94,
-	0x55, 0xff, 0x98, 0x81, 0x42, 0x2c, 0xe6, 0x19, 0x11, 0xeb, 0xc7, 0xbd, 0x4e, 0x3b, 0x41, 0xdc,
-	0x80, 0xc2, 0xa0, 0xdd, 0x1b, 0x74, 0xbb, 0x1d, 0xdc, 0xe7, 0xbf, 0xd8, 0xec, 0xc0, 0x66, 0xab,
-	0xfd, 0xea, 0xf8, 0xbc, 0xd5, 0x30, 0x1a, 0xfa, 0xab, 0xd6, 0x89, 0x6e, 0xb4, 0x1a, 0xa5, 0x6c,
-	0xdc, 0xcc, 0xa0, 0x46, 0xff, 0xe7, 0x5d, 0xbd, 0x74, 0x0b, 0x15, 0x60, 0xb5, 0xdf, 0x7a, 0xa9,
-	0x77, 0x06, 0xfd, 0xd2, 0x32, 0x7b, 0x42, 0x80, 0xc1, 0xfa, 0xb7, 0x02, 0xb2, 0x82, 0x10, 0xac,
-	0xb7, 0xda, 0x7d, 0x1d, 0xb7, 0x8f, 0xcf, 0x0d, 0x31, 0x37, 0x45, 0xd8, 0xe2, 0x0f, 0x29, 0xad,
-	0x6a, 0x00, 0x39, 0x4f, 0xbe, 0xb9, 0xfa, 0x0a, 0x36, 0x7a, 0xa9, 0x13, 0xeb, 0x08, 0xd6, 0xcc,
-	0x91, 0xe9, 0x8d, 0xe5, 0x2f, 0xae, 0xf2, 0xd4, 0xda, 0xaa, 0xc8, 0x1f, 0x60, 0x8f, 0x99, 0x4f,
-	0xfc, 0x62, 0xc7, 0x96, 0xdd, 0x8c, 0x86, 0xf1, 0x22, 0xf4, 0xef, 0x0c, 0x94, 0x7a, 0x1f, 0x52,
-	0x84, 0x7a, 0xff, 0x5d, 0x11, 0xea, 0xbd, 0x5f, 0x11, 0xfa, 0x29, 0xdb, 0x7b, 0xf8, 0x53, 0x76,
-	0x57, 0xb5, 0x61, 0xa7, 0x67, 0x3b, 0x57, 0x23, 0x92, 0x6e, 0x08, 0xf6, 0x21, 0x47, 0x4d, 0xef,
-	0x8a, 0xd0, 0xf0, 0x42, 0x2b, 0x1c, 0xa3, 0xc3, 0x70, 0x01, 0xe5, 0x79, 0xbf, 0xbf, 0xb0, 0xce,
-	0x72, 0x04, 0x0e, 0xd7, 0xfa, 0x5b, 0xd8, 0x4d, 0x3f, 0x4a, 0x2e, 0xf8, 0x97, 0xd1, 0x4e, 0xcb,
-	0x6d, 0x3c, 0xb8, 0xa1, 0x70, 0xe3, 0x28, 0x2c, 0xc2, 0xd9, 0xf7, 0xfe, 0x57, 0xb3, 0xef, 0xbd,
-	0x73, 0xf6, 0xbd, 0x0f, 0x9b, 0x7d, 0xef, 0xda, 0xd9, 0xd7, 0xfe, 0x9c, 0x81, 0xbc, 0x1e, 0x00,
-	0x11, 0x86, 0xc2, 0x19, 0xa1, 0xfa, 0x4c, 0xc0, 0x51, 0xfc, 0xdc, 0x58, 0xb8, 0x43, 0xfb, 0x1f,
-	0xdd, 0x80, 0x90, 0x53, 0xc3, 0x50, 0xe8, 0xdd, 0xa8, 0xd9, 0x7b, 0xa7, 0x66, 0x7a, 0xfe, 0x1a,
-	0x86, 0xbb, 0xae, 0x77, 0x55, 0x71, 0x27, 0x84, 0x7d, 0x7a, 0x5b, 0x15, 0xf1, 0x2f, 0x10, 0x11,
-	0xef, 0x17, 0xd5, 0x2b, 0x9b, 0xbe, 0x9e, 0x5e, 0x54, 0x86, 0xee, 0xf8, 0x49, 0x80, 0x7a, 0x22,
-	0x50, 0x9f, 0xcb, 0x7f, 0x94, 0x78, 0x73, 0xf8, 0xe4, 0xca, 0x8d, 0xfe, 0xb3, 0xa2, 0xbb, 0x74,
-	0xa1, 0x70, 0xcf, 0x17, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0x71, 0x00, 0x5d, 0x55, 0x7d, 0x21,
-	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
-
-// ExtensionClient is the client API for Extension service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type ExtensionClient interface {
-	// Get a single attribute
-	GetExtValue(ctx context.Context, in *SingleGetValueRequest, opts ...grpc.CallOption) (*SingleGetValueResponse, error)
-	// Set a single attribute
-	SetExtValue(ctx context.Context, in *SingleSetValueRequest, opts ...grpc.CallOption) (*SingleSetValueResponse, error)
-}
-
-type extensionClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewExtensionClient(cc *grpc.ClientConn) ExtensionClient {
-	return &extensionClient{cc}
-}
-
-func (c *extensionClient) GetExtValue(ctx context.Context, in *SingleGetValueRequest, opts ...grpc.CallOption) (*SingleGetValueResponse, error) {
-	out := new(SingleGetValueResponse)
-	err := c.cc.Invoke(ctx, "/extension.Extension/GetExtValue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *extensionClient) SetExtValue(ctx context.Context, in *SingleSetValueRequest, opts ...grpc.CallOption) (*SingleSetValueResponse, error) {
-	out := new(SingleSetValueResponse)
-	err := c.cc.Invoke(ctx, "/extension.Extension/SetExtValue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// ExtensionServer is the server API for Extension service.
-type ExtensionServer interface {
-	// Get a single attribute
-	GetExtValue(context.Context, *SingleGetValueRequest) (*SingleGetValueResponse, error)
-	// Set a single attribute
-	SetExtValue(context.Context, *SingleSetValueRequest) (*SingleSetValueResponse, error)
-}
-
-// UnimplementedExtensionServer can be embedded to have forward compatible implementations.
-type UnimplementedExtensionServer struct {
-}
-
-func (*UnimplementedExtensionServer) GetExtValue(ctx context.Context, req *SingleGetValueRequest) (*SingleGetValueResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
-}
-func (*UnimplementedExtensionServer) SetExtValue(ctx context.Context, req *SingleSetValueRequest) (*SingleSetValueResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method SetExtValue not implemented")
-}
-
-func RegisterExtensionServer(s *grpc.Server, srv ExtensionServer) {
-	s.RegisterService(&_Extension_serviceDesc, srv)
-}
-
-func _Extension_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(SingleGetValueRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ExtensionServer).GetExtValue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/extension.Extension/GetExtValue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ExtensionServer).GetExtValue(ctx, req.(*SingleGetValueRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Extension_SetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(SingleSetValueRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ExtensionServer).SetExtValue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/extension.Extension/SetExtValue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ExtensionServer).SetExtValue(ctx, req.(*SingleSetValueRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Extension_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "extension.Extension",
-	HandlerType: (*ExtensionServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "GetExtValue",
-			Handler:    _Extension_GetExtValue_Handler,
-		},
-		{
-			MethodName: "SetExtValue",
-			Handler:    _Extension_SetExtValue_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "voltha_protos/extensions.proto",
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/inter_container/inter_container.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/inter_container/inter_container.pb.go
deleted file mode 100644
index 403d9e6..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/inter_container/inter_container.pb.go
+++ /dev/null
@@ -1,1681 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/inter_container.proto
-
-package inter_container
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	any "github.com/golang/protobuf/ptypes/any"
-	timestamp "github.com/golang/protobuf/ptypes/timestamp"
-	common "github.com/opencord/voltha-protos/v4/go/common"
-	openflow_13 "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	tech_profile "github.com/opencord/voltha-protos/v4/go/tech_profile"
-	voltha "github.com/opencord/voltha-protos/v4/go/voltha"
-	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
-
-// ID from public import voltha_protos/common.proto
-type ID = common.ID
-
-// IDs from public import voltha_protos/common.proto
-type IDs = common.IDs
-
-// AdminState from public import voltha_protos/common.proto
-type AdminState = common.AdminState
-
-// OperStatus from public import voltha_protos/common.proto
-type OperStatus = common.OperStatus
-
-// ConnectStatus from public import voltha_protos/common.proto
-type ConnectStatus = common.ConnectStatus
-
-// OperationResp from public import voltha_protos/common.proto
-type OperationResp = common.OperationResp
-
-// ValueType from public import voltha_protos/common.proto
-type ValueType = common.ValueType
-
-// ValueSpecifier from public import voltha_protos/common.proto
-type ValueSpecifier = common.ValueSpecifier
-
-// ReturnValues from public import voltha_protos/common.proto
-type ReturnValues = common.ReturnValues
-
-// TestModeKeys from public import voltha_protos/common.proto
-type TestModeKeys = common.TestModeKeys
-
-var TestModeKeys_name = common.TestModeKeys_name
-var TestModeKeys_value = common.TestModeKeys_value
-
-const TestModeKeys_api_test = TestModeKeys(common.TestModeKeys_api_test)
-
-// AdminState_Types from public import voltha_protos/common.proto
-type AdminState_Types = common.AdminState_Types
-
-var AdminState_Types_name = common.AdminState_Types_name
-var AdminState_Types_value = common.AdminState_Types_value
-
-const AdminState_UNKNOWN = AdminState_Types(common.AdminState_UNKNOWN)
-const AdminState_PREPROVISIONED = AdminState_Types(common.AdminState_PREPROVISIONED)
-const AdminState_ENABLED = AdminState_Types(common.AdminState_ENABLED)
-const AdminState_DISABLED = AdminState_Types(common.AdminState_DISABLED)
-const AdminState_DOWNLOADING_IMAGE = AdminState_Types(common.AdminState_DOWNLOADING_IMAGE)
-
-// OperStatus_Types from public import voltha_protos/common.proto
-type OperStatus_Types = common.OperStatus_Types
-
-var OperStatus_Types_name = common.OperStatus_Types_name
-var OperStatus_Types_value = common.OperStatus_Types_value
-
-const OperStatus_UNKNOWN = OperStatus_Types(common.OperStatus_UNKNOWN)
-const OperStatus_DISCOVERED = OperStatus_Types(common.OperStatus_DISCOVERED)
-const OperStatus_ACTIVATING = OperStatus_Types(common.OperStatus_ACTIVATING)
-const OperStatus_TESTING = OperStatus_Types(common.OperStatus_TESTING)
-const OperStatus_ACTIVE = OperStatus_Types(common.OperStatus_ACTIVE)
-const OperStatus_FAILED = OperStatus_Types(common.OperStatus_FAILED)
-const OperStatus_RECONCILING = OperStatus_Types(common.OperStatus_RECONCILING)
-const OperStatus_RECONCILING_FAILED = OperStatus_Types(common.OperStatus_RECONCILING_FAILED)
-
-// ConnectStatus_Types from public import voltha_protos/common.proto
-type ConnectStatus_Types = common.ConnectStatus_Types
-
-var ConnectStatus_Types_name = common.ConnectStatus_Types_name
-var ConnectStatus_Types_value = common.ConnectStatus_Types_value
-
-const ConnectStatus_UNKNOWN = ConnectStatus_Types(common.ConnectStatus_UNKNOWN)
-const ConnectStatus_UNREACHABLE = ConnectStatus_Types(common.ConnectStatus_UNREACHABLE)
-const ConnectStatus_REACHABLE = ConnectStatus_Types(common.ConnectStatus_REACHABLE)
-
-// OperationResp_OperationReturnCode from public import voltha_protos/common.proto
-type OperationResp_OperationReturnCode = common.OperationResp_OperationReturnCode
-
-var OperationResp_OperationReturnCode_name = common.OperationResp_OperationReturnCode_name
-var OperationResp_OperationReturnCode_value = common.OperationResp_OperationReturnCode_value
-
-const OperationResp_OPERATION_SUCCESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_SUCCESS)
-const OperationResp_OPERATION_FAILURE = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_FAILURE)
-const OperationResp_OPERATION_UNSUPPORTED = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_UNSUPPORTED)
-const OperationResp_OPERATION_IN_PROGRESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_IN_PROGRESS)
-
-// ValueType_Type from public import voltha_protos/common.proto
-type ValueType_Type = common.ValueType_Type
-
-var ValueType_Type_name = common.ValueType_Type_name
-var ValueType_Type_value = common.ValueType_Type_value
-
-const ValueType_EMPTY = ValueType_Type(common.ValueType_EMPTY)
-const ValueType_DISTANCE = ValueType_Type(common.ValueType_DISTANCE)
-
-// DeviceGroup from public import voltha_protos/voltha.proto
-type DeviceGroup = voltha.DeviceGroup
-
-// DeviceGroups from public import voltha_protos/voltha.proto
-type DeviceGroups = voltha.DeviceGroups
-
-// EventFilterRuleKey from public import voltha_protos/voltha.proto
-type EventFilterRuleKey = voltha.EventFilterRuleKey
-
-// EventFilterRule from public import voltha_protos/voltha.proto
-type EventFilterRule = voltha.EventFilterRule
-
-// EventFilter from public import voltha_protos/voltha.proto
-type EventFilter = voltha.EventFilter
-
-// EventFilters from public import voltha_protos/voltha.proto
-type EventFilters = voltha.EventFilters
-
-// CoreInstance from public import voltha_protos/voltha.proto
-type CoreInstance = voltha.CoreInstance
-
-// CoreInstances from public import voltha_protos/voltha.proto
-type CoreInstances = voltha.CoreInstances
-
-// OmciTestRequest from public import voltha_protos/voltha.proto
-type OmciTestRequest = voltha.OmciTestRequest
-
-// TestResponse from public import voltha_protos/voltha.proto
-type TestResponse = voltha.TestResponse
-
-// ValueSet from public import voltha_protos/voltha.proto
-type ValueSet = voltha.ValueSet
-type ValueSet_AlarmConfig = voltha.ValueSet_AlarmConfig
-
-// Voltha from public import voltha_protos/voltha.proto
-type Voltha = voltha.Voltha
-
-// SelfTestResponse from public import voltha_protos/voltha.proto
-type SelfTestResponse = voltha.SelfTestResponse
-
-// OfAgentSubscriber from public import voltha_protos/voltha.proto
-type OfAgentSubscriber = voltha.OfAgentSubscriber
-
-// Membership from public import voltha_protos/voltha.proto
-type Membership = voltha.Membership
-
-// DeviceImageDownloadRequest from public import voltha_protos/voltha.proto
-type DeviceImageDownloadRequest = voltha.DeviceImageDownloadRequest
-
-// DeviceImageRequest from public import voltha_protos/voltha.proto
-type DeviceImageRequest = voltha.DeviceImageRequest
-
-// DeviceImageResponse from public import voltha_protos/voltha.proto
-type DeviceImageResponse = voltha.DeviceImageResponse
-
-// FlowMetadata from public import voltha_protos/voltha.proto
-type FlowMetadata = voltha.FlowMetadata
-
-// EventFilterRuleKey_EventFilterRuleType from public import voltha_protos/voltha.proto
-type EventFilterRuleKey_EventFilterRuleType = voltha.EventFilterRuleKey_EventFilterRuleType
-
-var EventFilterRuleKey_EventFilterRuleType_name = voltha.EventFilterRuleKey_EventFilterRuleType_name
-var EventFilterRuleKey_EventFilterRuleType_value = voltha.EventFilterRuleKey_EventFilterRuleType_value
-
-const EventFilterRuleKey_filter_all = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_filter_all)
-const EventFilterRuleKey_category = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_category)
-const EventFilterRuleKey_sub_category = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_sub_category)
-const EventFilterRuleKey_kpi_event_type = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_kpi_event_type)
-const EventFilterRuleKey_config_event_type = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_config_event_type)
-const EventFilterRuleKey_device_event_type = EventFilterRuleKey_EventFilterRuleType(voltha.EventFilterRuleKey_device_event_type)
-
-// TestResponse_TestResponseResult from public import voltha_protos/voltha.proto
-type TestResponse_TestResponseResult = voltha.TestResponse_TestResponseResult
-
-var TestResponse_TestResponseResult_name = voltha.TestResponse_TestResponseResult_name
-var TestResponse_TestResponseResult_value = voltha.TestResponse_TestResponseResult_value
-
-const TestResponse_SUCCESS = TestResponse_TestResponseResult(voltha.TestResponse_SUCCESS)
-const TestResponse_FAILURE = TestResponse_TestResponseResult(voltha.TestResponse_FAILURE)
-
-// SelfTestResponse_SelfTestResult from public import voltha_protos/voltha.proto
-type SelfTestResponse_SelfTestResult = voltha.SelfTestResponse_SelfTestResult
-
-var SelfTestResponse_SelfTestResult_name = voltha.SelfTestResponse_SelfTestResult_name
-var SelfTestResponse_SelfTestResult_value = voltha.SelfTestResponse_SelfTestResult_value
-
-const SelfTestResponse_SUCCESS = SelfTestResponse_SelfTestResult(voltha.SelfTestResponse_SUCCESS)
-const SelfTestResponse_FAILURE = SelfTestResponse_SelfTestResult(voltha.SelfTestResponse_FAILURE)
-const SelfTestResponse_NOT_SUPPORTED = SelfTestResponse_SelfTestResult(voltha.SelfTestResponse_NOT_SUPPORTED)
-const SelfTestResponse_UNKNOWN_ERROR = SelfTestResponse_SelfTestResult(voltha.SelfTestResponse_UNKNOWN_ERROR)
-
-// LogicalPortId from public import voltha_protos/logical_device.proto
-type LogicalPortId = voltha.LogicalPortId
-
-// LogicalPort from public import voltha_protos/logical_device.proto
-type LogicalPort = voltha.LogicalPort
-
-// LogicalPorts from public import voltha_protos/logical_device.proto
-type LogicalPorts = voltha.LogicalPorts
-
-// LogicalDevice from public import voltha_protos/logical_device.proto
-type LogicalDevice = voltha.LogicalDevice
-
-// LogicalDevices from public import voltha_protos/logical_device.proto
-type LogicalDevices = voltha.LogicalDevices
-
-type MessageType int32
-
-const (
-	MessageType_REQUEST           MessageType = 0
-	MessageType_RESPONSE          MessageType = 1
-	MessageType_DEVICE_DISCOVERED MessageType = 2
-)
-
-var MessageType_name = map[int32]string{
-	0: "REQUEST",
-	1: "RESPONSE",
-	2: "DEVICE_DISCOVERED",
-}
-
-var MessageType_value = map[string]int32{
-	"REQUEST":           0,
-	"RESPONSE":          1,
-	"DEVICE_DISCOVERED": 2,
-}
-
-func (x MessageType) String() string {
-	return proto.EnumName(MessageType_name, int32(x))
-}
-
-func (MessageType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{0}
-}
-
-type ErrorCodeCodes int32
-
-const (
-	ErrorCode_UNSUPPORTED_REQUEST ErrorCodeCodes = 0
-	ErrorCode_INVALID_PARAMETERS  ErrorCodeCodes = 1
-	ErrorCode_DEADLINE_EXCEEDED   ErrorCodeCodes = 2
-)
-
-var ErrorCodeCodes_name = map[int32]string{
-	0: "UNSUPPORTED_REQUEST",
-	1: "INVALID_PARAMETERS",
-	2: "DEADLINE_EXCEEDED",
-}
-
-var ErrorCodeCodes_value = map[string]int32{
-	"UNSUPPORTED_REQUEST": 0,
-	"INVALID_PARAMETERS":  1,
-	"DEADLINE_EXCEEDED":   2,
-}
-
-func (x ErrorCodeCodes) String() string {
-	return proto.EnumName(ErrorCodeCodes_name, int32(x))
-}
-
-func (ErrorCodeCodes) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{4, 0}
-}
-
-type InterAdapterMessageType_Types int32
-
-const (
-	InterAdapterMessageType_FLOW_REQUEST                  InterAdapterMessageType_Types = 0
-	InterAdapterMessageType_FLOW_RESPONSE                 InterAdapterMessageType_Types = 1
-	InterAdapterMessageType_OMCI_REQUEST                  InterAdapterMessageType_Types = 2
-	InterAdapterMessageType_OMCI_RESPONSE                 InterAdapterMessageType_Types = 3
-	InterAdapterMessageType_METRICS_REQUEST               InterAdapterMessageType_Types = 4
-	InterAdapterMessageType_METRICS_RESPONSE              InterAdapterMessageType_Types = 5
-	InterAdapterMessageType_ONU_IND_REQUEST               InterAdapterMessageType_Types = 6
-	InterAdapterMessageType_ONU_IND_RESPONSE              InterAdapterMessageType_Types = 7
-	InterAdapterMessageType_TECH_PROFILE_DOWNLOAD_REQUEST InterAdapterMessageType_Types = 8
-	InterAdapterMessageType_DELETE_GEM_PORT_REQUEST       InterAdapterMessageType_Types = 9
-	InterAdapterMessageType_DELETE_TCONT_REQUEST          InterAdapterMessageType_Types = 10
-)
-
-var InterAdapterMessageType_Types_name = map[int32]string{
-	0:  "FLOW_REQUEST",
-	1:  "FLOW_RESPONSE",
-	2:  "OMCI_REQUEST",
-	3:  "OMCI_RESPONSE",
-	4:  "METRICS_REQUEST",
-	5:  "METRICS_RESPONSE",
-	6:  "ONU_IND_REQUEST",
-	7:  "ONU_IND_RESPONSE",
-	8:  "TECH_PROFILE_DOWNLOAD_REQUEST",
-	9:  "DELETE_GEM_PORT_REQUEST",
-	10: "DELETE_TCONT_REQUEST",
-}
-
-var InterAdapterMessageType_Types_value = map[string]int32{
-	"FLOW_REQUEST":                  0,
-	"FLOW_RESPONSE":                 1,
-	"OMCI_REQUEST":                  2,
-	"OMCI_RESPONSE":                 3,
-	"METRICS_REQUEST":               4,
-	"METRICS_RESPONSE":              5,
-	"ONU_IND_REQUEST":               6,
-	"ONU_IND_RESPONSE":              7,
-	"TECH_PROFILE_DOWNLOAD_REQUEST": 8,
-	"DELETE_GEM_PORT_REQUEST":       9,
-	"DELETE_TCONT_REQUEST":          10,
-}
-
-func (x InterAdapterMessageType_Types) String() string {
-	return proto.EnumName(InterAdapterMessageType_Types_name, int32(x))
-}
-
-func (InterAdapterMessageType_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{13, 0}
-}
-
-type StrType struct {
-	Val                  string   `protobuf:"bytes,1,opt,name=val,proto3" json:"val,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StrType) Reset()         { *m = StrType{} }
-func (m *StrType) String() string { return proto.CompactTextString(m) }
-func (*StrType) ProtoMessage()    {}
-func (*StrType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{0}
-}
-
-func (m *StrType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StrType.Unmarshal(m, b)
-}
-func (m *StrType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StrType.Marshal(b, m, deterministic)
-}
-func (m *StrType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StrType.Merge(m, src)
-}
-func (m *StrType) XXX_Size() int {
-	return xxx_messageInfo_StrType.Size(m)
-}
-func (m *StrType) XXX_DiscardUnknown() {
-	xxx_messageInfo_StrType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StrType proto.InternalMessageInfo
-
-func (m *StrType) GetVal() string {
-	if m != nil {
-		return m.Val
-	}
-	return ""
-}
-
-type IntType struct {
-	Val                  int64    `protobuf:"varint,1,opt,name=val,proto3" json:"val,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *IntType) Reset()         { *m = IntType{} }
-func (m *IntType) String() string { return proto.CompactTextString(m) }
-func (*IntType) ProtoMessage()    {}
-func (*IntType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{1}
-}
-
-func (m *IntType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IntType.Unmarshal(m, b)
-}
-func (m *IntType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IntType.Marshal(b, m, deterministic)
-}
-func (m *IntType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IntType.Merge(m, src)
-}
-func (m *IntType) XXX_Size() int {
-	return xxx_messageInfo_IntType.Size(m)
-}
-func (m *IntType) XXX_DiscardUnknown() {
-	xxx_messageInfo_IntType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IntType proto.InternalMessageInfo
-
-func (m *IntType) GetVal() int64 {
-	if m != nil {
-		return m.Val
-	}
-	return 0
-}
-
-type BoolType struct {
-	Val                  bool     `protobuf:"varint,1,opt,name=val,proto3" json:"val,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *BoolType) Reset()         { *m = BoolType{} }
-func (m *BoolType) String() string { return proto.CompactTextString(m) }
-func (*BoolType) ProtoMessage()    {}
-func (*BoolType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{2}
-}
-
-func (m *BoolType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_BoolType.Unmarshal(m, b)
-}
-func (m *BoolType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_BoolType.Marshal(b, m, deterministic)
-}
-func (m *BoolType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BoolType.Merge(m, src)
-}
-func (m *BoolType) XXX_Size() int {
-	return xxx_messageInfo_BoolType.Size(m)
-}
-func (m *BoolType) XXX_DiscardUnknown() {
-	xxx_messageInfo_BoolType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_BoolType proto.InternalMessageInfo
-
-func (m *BoolType) GetVal() bool {
-	if m != nil {
-		return m.Val
-	}
-	return false
-}
-
-type Packet struct {
-	Payload              []byte   `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Packet) Reset()         { *m = Packet{} }
-func (m *Packet) String() string { return proto.CompactTextString(m) }
-func (*Packet) ProtoMessage()    {}
-func (*Packet) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{3}
-}
-
-func (m *Packet) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Packet.Unmarshal(m, b)
-}
-func (m *Packet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Packet.Marshal(b, m, deterministic)
-}
-func (m *Packet) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Packet.Merge(m, src)
-}
-func (m *Packet) XXX_Size() int {
-	return xxx_messageInfo_Packet.Size(m)
-}
-func (m *Packet) XXX_DiscardUnknown() {
-	xxx_messageInfo_Packet.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Packet proto.InternalMessageInfo
-
-func (m *Packet) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
-	}
-	return nil
-}
-
-type ErrorCode struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ErrorCode) Reset()         { *m = ErrorCode{} }
-func (m *ErrorCode) String() string { return proto.CompactTextString(m) }
-func (*ErrorCode) ProtoMessage()    {}
-func (*ErrorCode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{4}
-}
-
-func (m *ErrorCode) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ErrorCode.Unmarshal(m, b)
-}
-func (m *ErrorCode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ErrorCode.Marshal(b, m, deterministic)
-}
-func (m *ErrorCode) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ErrorCode.Merge(m, src)
-}
-func (m *ErrorCode) XXX_Size() int {
-	return xxx_messageInfo_ErrorCode.Size(m)
-}
-func (m *ErrorCode) XXX_DiscardUnknown() {
-	xxx_messageInfo_ErrorCode.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ErrorCode proto.InternalMessageInfo
-
-type Error struct {
-	Code                 ErrorCodeCodes `protobuf:"varint,1,opt,name=code,proto3,enum=voltha.ErrorCodeCodes" json:"code,omitempty"`
-	Reason               string         `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *Error) Reset()         { *m = Error{} }
-func (m *Error) String() string { return proto.CompactTextString(m) }
-func (*Error) ProtoMessage()    {}
-func (*Error) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{5}
-}
-
-func (m *Error) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Error.Unmarshal(m, b)
-}
-func (m *Error) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Error.Marshal(b, m, deterministic)
-}
-func (m *Error) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Error.Merge(m, src)
-}
-func (m *Error) XXX_Size() int {
-	return xxx_messageInfo_Error.Size(m)
-}
-func (m *Error) XXX_DiscardUnknown() {
-	xxx_messageInfo_Error.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Error proto.InternalMessageInfo
-
-func (m *Error) GetCode() ErrorCodeCodes {
-	if m != nil {
-		return m.Code
-	}
-	return ErrorCode_UNSUPPORTED_REQUEST
-}
-
-func (m *Error) GetReason() string {
-	if m != nil {
-		return m.Reason
-	}
-	return ""
-}
-
-type Header struct {
-	Id                   string               `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Type                 MessageType          `protobuf:"varint,2,opt,name=type,proto3,enum=voltha.MessageType" json:"type,omitempty"`
-	FromTopic            string               `protobuf:"bytes,3,opt,name=from_topic,json=fromTopic,proto3" json:"from_topic,omitempty"`
-	ToTopic              string               `protobuf:"bytes,4,opt,name=to_topic,json=toTopic,proto3" json:"to_topic,omitempty"`
-	KeyTopic             string               `protobuf:"bytes,5,opt,name=key_topic,json=keyTopic,proto3" json:"key_topic,omitempty"`
-	Timestamp            *timestamp.Timestamp `protobuf:"bytes,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *Header) Reset()         { *m = Header{} }
-func (m *Header) String() string { return proto.CompactTextString(m) }
-func (*Header) ProtoMessage()    {}
-func (*Header) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{6}
-}
-
-func (m *Header) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Header.Unmarshal(m, b)
-}
-func (m *Header) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Header.Marshal(b, m, deterministic)
-}
-func (m *Header) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Header.Merge(m, src)
-}
-func (m *Header) XXX_Size() int {
-	return xxx_messageInfo_Header.Size(m)
-}
-func (m *Header) XXX_DiscardUnknown() {
-	xxx_messageInfo_Header.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Header proto.InternalMessageInfo
-
-func (m *Header) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *Header) GetType() MessageType {
-	if m != nil {
-		return m.Type
-	}
-	return MessageType_REQUEST
-}
-
-func (m *Header) GetFromTopic() string {
-	if m != nil {
-		return m.FromTopic
-	}
-	return ""
-}
-
-func (m *Header) GetToTopic() string {
-	if m != nil {
-		return m.ToTopic
-	}
-	return ""
-}
-
-func (m *Header) GetKeyTopic() string {
-	if m != nil {
-		return m.KeyTopic
-	}
-	return ""
-}
-
-func (m *Header) GetTimestamp() *timestamp.Timestamp {
-	if m != nil {
-		return m.Timestamp
-	}
-	return nil
-}
-
-type Argument struct {
-	Key                  string   `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	Value                *any.Any `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Argument) Reset()         { *m = Argument{} }
-func (m *Argument) String() string { return proto.CompactTextString(m) }
-func (*Argument) ProtoMessage()    {}
-func (*Argument) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{7}
-}
-
-func (m *Argument) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Argument.Unmarshal(m, b)
-}
-func (m *Argument) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Argument.Marshal(b, m, deterministic)
-}
-func (m *Argument) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Argument.Merge(m, src)
-}
-func (m *Argument) XXX_Size() int {
-	return xxx_messageInfo_Argument.Size(m)
-}
-func (m *Argument) XXX_DiscardUnknown() {
-	xxx_messageInfo_Argument.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Argument proto.InternalMessageInfo
-
-func (m *Argument) GetKey() string {
-	if m != nil {
-		return m.Key
-	}
-	return ""
-}
-
-func (m *Argument) GetValue() *any.Any {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-type InterContainerMessage struct {
-	Header               *Header  `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
-	Body                 *any.Any `protobuf:"bytes,2,opt,name=body,proto3" json:"body,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterContainerMessage) Reset()         { *m = InterContainerMessage{} }
-func (m *InterContainerMessage) String() string { return proto.CompactTextString(m) }
-func (*InterContainerMessage) ProtoMessage()    {}
-func (*InterContainerMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{8}
-}
-
-func (m *InterContainerMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterContainerMessage.Unmarshal(m, b)
-}
-func (m *InterContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterContainerMessage.Marshal(b, m, deterministic)
-}
-func (m *InterContainerMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterContainerMessage.Merge(m, src)
-}
-func (m *InterContainerMessage) XXX_Size() int {
-	return xxx_messageInfo_InterContainerMessage.Size(m)
-}
-func (m *InterContainerMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterContainerMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterContainerMessage proto.InternalMessageInfo
-
-func (m *InterContainerMessage) GetHeader() *Header {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *InterContainerMessage) GetBody() *any.Any {
-	if m != nil {
-		return m.Body
-	}
-	return nil
-}
-
-type InterContainerRequestBody struct {
-	Rpc                  string      `protobuf:"bytes,2,opt,name=rpc,proto3" json:"rpc,omitempty"`
-	Args                 []*Argument `protobuf:"bytes,3,rep,name=args,proto3" json:"args,omitempty"`
-	ResponseRequired     bool        `protobuf:"varint,4,opt,name=response_required,json=responseRequired,proto3" json:"response_required,omitempty"`
-	ReplyToTopic         string      `protobuf:"bytes,5,opt,name=reply_to_topic,json=replyToTopic,proto3" json:"reply_to_topic,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *InterContainerRequestBody) Reset()         { *m = InterContainerRequestBody{} }
-func (m *InterContainerRequestBody) String() string { return proto.CompactTextString(m) }
-func (*InterContainerRequestBody) ProtoMessage()    {}
-func (*InterContainerRequestBody) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{9}
-}
-
-func (m *InterContainerRequestBody) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterContainerRequestBody.Unmarshal(m, b)
-}
-func (m *InterContainerRequestBody) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterContainerRequestBody.Marshal(b, m, deterministic)
-}
-func (m *InterContainerRequestBody) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterContainerRequestBody.Merge(m, src)
-}
-func (m *InterContainerRequestBody) XXX_Size() int {
-	return xxx_messageInfo_InterContainerRequestBody.Size(m)
-}
-func (m *InterContainerRequestBody) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterContainerRequestBody.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterContainerRequestBody proto.InternalMessageInfo
-
-func (m *InterContainerRequestBody) GetRpc() string {
-	if m != nil {
-		return m.Rpc
-	}
-	return ""
-}
-
-func (m *InterContainerRequestBody) GetArgs() []*Argument {
-	if m != nil {
-		return m.Args
-	}
-	return nil
-}
-
-func (m *InterContainerRequestBody) GetResponseRequired() bool {
-	if m != nil {
-		return m.ResponseRequired
-	}
-	return false
-}
-
-func (m *InterContainerRequestBody) GetReplyToTopic() string {
-	if m != nil {
-		return m.ReplyToTopic
-	}
-	return ""
-}
-
-type InterContainerResponseBody struct {
-	Success              bool     `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
-	Result               *any.Any `protobuf:"bytes,3,opt,name=result,proto3" json:"result,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterContainerResponseBody) Reset()         { *m = InterContainerResponseBody{} }
-func (m *InterContainerResponseBody) String() string { return proto.CompactTextString(m) }
-func (*InterContainerResponseBody) ProtoMessage()    {}
-func (*InterContainerResponseBody) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{10}
-}
-
-func (m *InterContainerResponseBody) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterContainerResponseBody.Unmarshal(m, b)
-}
-func (m *InterContainerResponseBody) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterContainerResponseBody.Marshal(b, m, deterministic)
-}
-func (m *InterContainerResponseBody) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterContainerResponseBody.Merge(m, src)
-}
-func (m *InterContainerResponseBody) XXX_Size() int {
-	return xxx_messageInfo_InterContainerResponseBody.Size(m)
-}
-func (m *InterContainerResponseBody) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterContainerResponseBody.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterContainerResponseBody proto.InternalMessageInfo
-
-func (m *InterContainerResponseBody) GetSuccess() bool {
-	if m != nil {
-		return m.Success
-	}
-	return false
-}
-
-func (m *InterContainerResponseBody) GetResult() *any.Any {
-	if m != nil {
-		return m.Result
-	}
-	return nil
-}
-
-type SwitchCapability struct {
-	Desc                 *openflow_13.OfpDesc           `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"`
-	SwitchFeatures       *openflow_13.OfpSwitchFeatures `protobuf:"bytes,2,opt,name=switch_features,json=switchFeatures,proto3" json:"switch_features,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                       `json:"-"`
-	XXX_unrecognized     []byte                         `json:"-"`
-	XXX_sizecache        int32                          `json:"-"`
-}
-
-func (m *SwitchCapability) Reset()         { *m = SwitchCapability{} }
-func (m *SwitchCapability) String() string { return proto.CompactTextString(m) }
-func (*SwitchCapability) ProtoMessage()    {}
-func (*SwitchCapability) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{11}
-}
-
-func (m *SwitchCapability) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SwitchCapability.Unmarshal(m, b)
-}
-func (m *SwitchCapability) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SwitchCapability.Marshal(b, m, deterministic)
-}
-func (m *SwitchCapability) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SwitchCapability.Merge(m, src)
-}
-func (m *SwitchCapability) XXX_Size() int {
-	return xxx_messageInfo_SwitchCapability.Size(m)
-}
-func (m *SwitchCapability) XXX_DiscardUnknown() {
-	xxx_messageInfo_SwitchCapability.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SwitchCapability proto.InternalMessageInfo
-
-func (m *SwitchCapability) GetDesc() *openflow_13.OfpDesc {
-	if m != nil {
-		return m.Desc
-	}
-	return nil
-}
-
-func (m *SwitchCapability) GetSwitchFeatures() *openflow_13.OfpSwitchFeatures {
-	if m != nil {
-		return m.SwitchFeatures
-	}
-	return nil
-}
-
-type DeviceDiscovered struct {
-	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	ParentId             string   `protobuf:"bytes,2,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
-	DeviceType           string   `protobuf:"bytes,3,opt,name=device_type,json=deviceType,proto3" json:"device_type,omitempty"`
-	Publisher            string   `protobuf:"bytes,4,opt,name=publisher,proto3" json:"publisher,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DeviceDiscovered) Reset()         { *m = DeviceDiscovered{} }
-func (m *DeviceDiscovered) String() string { return proto.CompactTextString(m) }
-func (*DeviceDiscovered) ProtoMessage()    {}
-func (*DeviceDiscovered) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{12}
-}
-
-func (m *DeviceDiscovered) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceDiscovered.Unmarshal(m, b)
-}
-func (m *DeviceDiscovered) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceDiscovered.Marshal(b, m, deterministic)
-}
-func (m *DeviceDiscovered) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceDiscovered.Merge(m, src)
-}
-func (m *DeviceDiscovered) XXX_Size() int {
-	return xxx_messageInfo_DeviceDiscovered.Size(m)
-}
-func (m *DeviceDiscovered) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceDiscovered.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceDiscovered proto.InternalMessageInfo
-
-func (m *DeviceDiscovered) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *DeviceDiscovered) GetParentId() string {
-	if m != nil {
-		return m.ParentId
-	}
-	return ""
-}
-
-func (m *DeviceDiscovered) GetDeviceType() string {
-	if m != nil {
-		return m.DeviceType
-	}
-	return ""
-}
-
-func (m *DeviceDiscovered) GetPublisher() string {
-	if m != nil {
-		return m.Publisher
-	}
-	return ""
-}
-
-type InterAdapterMessageType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterAdapterMessageType) Reset()         { *m = InterAdapterMessageType{} }
-func (m *InterAdapterMessageType) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterMessageType) ProtoMessage()    {}
-func (*InterAdapterMessageType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{13}
-}
-
-func (m *InterAdapterMessageType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterMessageType.Unmarshal(m, b)
-}
-func (m *InterAdapterMessageType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterMessageType.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterMessageType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterMessageType.Merge(m, src)
-}
-func (m *InterAdapterMessageType) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterMessageType.Size(m)
-}
-func (m *InterAdapterMessageType) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterMessageType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterMessageType proto.InternalMessageInfo
-
-type InterAdapterHeader struct {
-	Id                   string                        `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Type                 InterAdapterMessageType_Types `protobuf:"varint,2,opt,name=type,proto3,enum=voltha.InterAdapterMessageType_Types" json:"type,omitempty"`
-	FromTopic            string                        `protobuf:"bytes,3,opt,name=from_topic,json=fromTopic,proto3" json:"from_topic,omitempty"`
-	ToTopic              string                        `protobuf:"bytes,4,opt,name=to_topic,json=toTopic,proto3" json:"to_topic,omitempty"`
-	ToDeviceId           string                        `protobuf:"bytes,5,opt,name=to_device_id,json=toDeviceId,proto3" json:"to_device_id,omitempty"`
-	ProxyDeviceId        string                        `protobuf:"bytes,6,opt,name=proxy_device_id,json=proxyDeviceId,proto3" json:"proxy_device_id,omitempty"`
-	Timestamp            *timestamp.Timestamp          `protobuf:"bytes,7,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
-	XXX_unrecognized     []byte                        `json:"-"`
-	XXX_sizecache        int32                         `json:"-"`
-}
-
-func (m *InterAdapterHeader) Reset()         { *m = InterAdapterHeader{} }
-func (m *InterAdapterHeader) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterHeader) ProtoMessage()    {}
-func (*InterAdapterHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{14}
-}
-
-func (m *InterAdapterHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterHeader.Unmarshal(m, b)
-}
-func (m *InterAdapterHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterHeader.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterHeader.Merge(m, src)
-}
-func (m *InterAdapterHeader) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterHeader.Size(m)
-}
-func (m *InterAdapterHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterHeader proto.InternalMessageInfo
-
-func (m *InterAdapterHeader) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *InterAdapterHeader) GetType() InterAdapterMessageType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return InterAdapterMessageType_FLOW_REQUEST
-}
-
-func (m *InterAdapterHeader) GetFromTopic() string {
-	if m != nil {
-		return m.FromTopic
-	}
-	return ""
-}
-
-func (m *InterAdapterHeader) GetToTopic() string {
-	if m != nil {
-		return m.ToTopic
-	}
-	return ""
-}
-
-func (m *InterAdapterHeader) GetToDeviceId() string {
-	if m != nil {
-		return m.ToDeviceId
-	}
-	return ""
-}
-
-func (m *InterAdapterHeader) GetProxyDeviceId() string {
-	if m != nil {
-		return m.ProxyDeviceId
-	}
-	return ""
-}
-
-func (m *InterAdapterHeader) GetTimestamp() *timestamp.Timestamp {
-	if m != nil {
-		return m.Timestamp
-	}
-	return nil
-}
-
-type InterAdapterOmciMessage struct {
-	Message              []byte                      `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"`
-	ConnectStatus        common.ConnectStatus_Types  `protobuf:"varint,2,opt,name=connect_status,json=connectStatus,proto3,enum=common.ConnectStatus_Types" json:"connect_status,omitempty"`
-	ProxyAddress         *voltha.Device_ProxyAddress `protobuf:"bytes,3,opt,name=proxy_address,json=proxyAddress,proto3" json:"proxy_address,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
-}
-
-func (m *InterAdapterOmciMessage) Reset()         { *m = InterAdapterOmciMessage{} }
-func (m *InterAdapterOmciMessage) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterOmciMessage) ProtoMessage()    {}
-func (*InterAdapterOmciMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{15}
-}
-
-func (m *InterAdapterOmciMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterOmciMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterOmciMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterOmciMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterOmciMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterOmciMessage.Merge(m, src)
-}
-func (m *InterAdapterOmciMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterOmciMessage.Size(m)
-}
-func (m *InterAdapterOmciMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterOmciMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterOmciMessage proto.InternalMessageInfo
-
-func (m *InterAdapterOmciMessage) GetMessage() []byte {
-	if m != nil {
-		return m.Message
-	}
-	return nil
-}
-
-func (m *InterAdapterOmciMessage) GetConnectStatus() common.ConnectStatus_Types {
-	if m != nil {
-		return m.ConnectStatus
-	}
-	return common.ConnectStatus_UNKNOWN
-}
-
-func (m *InterAdapterOmciMessage) GetProxyAddress() *voltha.Device_ProxyAddress {
-	if m != nil {
-		return m.ProxyAddress
-	}
-	return nil
-}
-
-type InterAdapterTechProfileInstanceRequestMessage struct {
-	TpInstancePath       string   `protobuf:"bytes,1,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
-	ParentDeviceId       string   `protobuf:"bytes,2,opt,name=parent_device_id,json=parentDeviceId,proto3" json:"parent_device_id,omitempty"`
-	ParentPonPort        uint32   `protobuf:"varint,3,opt,name=parent_pon_port,json=parentPonPort,proto3" json:"parent_pon_port,omitempty"`
-	OnuId                uint32   `protobuf:"varint,4,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	UniId                uint32   `protobuf:"varint,5,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) Reset() {
-	*m = InterAdapterTechProfileInstanceRequestMessage{}
-}
-func (m *InterAdapterTechProfileInstanceRequestMessage) String() string {
-	return proto.CompactTextString(m)
-}
-func (*InterAdapterTechProfileInstanceRequestMessage) ProtoMessage() {}
-func (*InterAdapterTechProfileInstanceRequestMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{16}
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterTechProfileInstanceRequestMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterTechProfileInstanceRequestMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage.Merge(m, src)
-}
-func (m *InterAdapterTechProfileInstanceRequestMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage.Size(m)
-}
-func (m *InterAdapterTechProfileInstanceRequestMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterTechProfileInstanceRequestMessage proto.InternalMessageInfo
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) GetTpInstancePath() string {
-	if m != nil {
-		return m.TpInstancePath
-	}
-	return ""
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) GetParentDeviceId() string {
-	if m != nil {
-		return m.ParentDeviceId
-	}
-	return ""
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) GetParentPonPort() uint32 {
-	if m != nil {
-		return m.ParentPonPort
-	}
-	return 0
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *InterAdapterTechProfileInstanceRequestMessage) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-type InterAdapterTechProfileDownloadMessage struct {
-	UniId          uint32 `protobuf:"varint,1,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	TpInstancePath string `protobuf:"bytes,2,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
-	// Types that are valid to be assigned to TechTpInstance:
-	//	*InterAdapterTechProfileDownloadMessage_TpInstance
-	//	*InterAdapterTechProfileDownloadMessage_EponTpInstance
-	TechTpInstance       isInterAdapterTechProfileDownloadMessage_TechTpInstance `protobuf_oneof:"tech_tp_instance"`
-	XXX_NoUnkeyedLiteral struct{}                                                `json:"-"`
-	XXX_unrecognized     []byte                                                  `json:"-"`
-	XXX_sizecache        int32                                                   `json:"-"`
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) Reset() {
-	*m = InterAdapterTechProfileDownloadMessage{}
-}
-func (m *InterAdapterTechProfileDownloadMessage) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterTechProfileDownloadMessage) ProtoMessage()    {}
-func (*InterAdapterTechProfileDownloadMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{17}
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterTechProfileDownloadMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterTechProfileDownloadMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterTechProfileDownloadMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterTechProfileDownloadMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterTechProfileDownloadMessage.Merge(m, src)
-}
-func (m *InterAdapterTechProfileDownloadMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterTechProfileDownloadMessage.Size(m)
-}
-func (m *InterAdapterTechProfileDownloadMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterTechProfileDownloadMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterTechProfileDownloadMessage proto.InternalMessageInfo
-
-func (m *InterAdapterTechProfileDownloadMessage) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) GetTpInstancePath() string {
-	if m != nil {
-		return m.TpInstancePath
-	}
-	return ""
-}
-
-type isInterAdapterTechProfileDownloadMessage_TechTpInstance interface {
-	isInterAdapterTechProfileDownloadMessage_TechTpInstance()
-}
-
-type InterAdapterTechProfileDownloadMessage_TpInstance struct {
-	TpInstance *tech_profile.TechProfileInstance `protobuf:"bytes,3,opt,name=tp_instance,json=tpInstance,proto3,oneof"`
-}
-
-type InterAdapterTechProfileDownloadMessage_EponTpInstance struct {
-	EponTpInstance *tech_profile.EponTechProfileInstance `protobuf:"bytes,4,opt,name=epon_tp_instance,json=eponTpInstance,proto3,oneof"`
-}
-
-func (*InterAdapterTechProfileDownloadMessage_TpInstance) isInterAdapterTechProfileDownloadMessage_TechTpInstance() {
-}
-
-func (*InterAdapterTechProfileDownloadMessage_EponTpInstance) isInterAdapterTechProfileDownloadMessage_TechTpInstance() {
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) GetTechTpInstance() isInterAdapterTechProfileDownloadMessage_TechTpInstance {
-	if m != nil {
-		return m.TechTpInstance
-	}
-	return nil
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) GetTpInstance() *tech_profile.TechProfileInstance {
-	if x, ok := m.GetTechTpInstance().(*InterAdapterTechProfileDownloadMessage_TpInstance); ok {
-		return x.TpInstance
-	}
-	return nil
-}
-
-func (m *InterAdapterTechProfileDownloadMessage) GetEponTpInstance() *tech_profile.EponTechProfileInstance {
-	if x, ok := m.GetTechTpInstance().(*InterAdapterTechProfileDownloadMessage_EponTpInstance); ok {
-		return x.EponTpInstance
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*InterAdapterTechProfileDownloadMessage) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*InterAdapterTechProfileDownloadMessage_TpInstance)(nil),
-		(*InterAdapterTechProfileDownloadMessage_EponTpInstance)(nil),
-	}
-}
-
-type InterAdapterDeleteGemPortMessage struct {
-	UniId                uint32   `protobuf:"varint,1,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	TpInstancePath       string   `protobuf:"bytes,2,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
-	GemPortId            uint32   `protobuf:"varint,3,opt,name=gem_port_id,json=gemPortId,proto3" json:"gem_port_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterAdapterDeleteGemPortMessage) Reset()         { *m = InterAdapterDeleteGemPortMessage{} }
-func (m *InterAdapterDeleteGemPortMessage) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterDeleteGemPortMessage) ProtoMessage()    {}
-func (*InterAdapterDeleteGemPortMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{18}
-}
-
-func (m *InterAdapterDeleteGemPortMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterDeleteGemPortMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterDeleteGemPortMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterDeleteGemPortMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterDeleteGemPortMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterDeleteGemPortMessage.Merge(m, src)
-}
-func (m *InterAdapterDeleteGemPortMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterDeleteGemPortMessage.Size(m)
-}
-func (m *InterAdapterDeleteGemPortMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterDeleteGemPortMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterDeleteGemPortMessage proto.InternalMessageInfo
-
-func (m *InterAdapterDeleteGemPortMessage) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *InterAdapterDeleteGemPortMessage) GetTpInstancePath() string {
-	if m != nil {
-		return m.TpInstancePath
-	}
-	return ""
-}
-
-func (m *InterAdapterDeleteGemPortMessage) GetGemPortId() uint32 {
-	if m != nil {
-		return m.GemPortId
-	}
-	return 0
-}
-
-type InterAdapterDeleteTcontMessage struct {
-	UniId                uint32   `protobuf:"varint,1,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	TpInstancePath       string   `protobuf:"bytes,2,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
-	AllocId              uint32   `protobuf:"varint,3,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InterAdapterDeleteTcontMessage) Reset()         { *m = InterAdapterDeleteTcontMessage{} }
-func (m *InterAdapterDeleteTcontMessage) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterDeleteTcontMessage) ProtoMessage()    {}
-func (*InterAdapterDeleteTcontMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{19}
-}
-
-func (m *InterAdapterDeleteTcontMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterDeleteTcontMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterDeleteTcontMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterDeleteTcontMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterDeleteTcontMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterDeleteTcontMessage.Merge(m, src)
-}
-func (m *InterAdapterDeleteTcontMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterDeleteTcontMessage.Size(m)
-}
-func (m *InterAdapterDeleteTcontMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterDeleteTcontMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterDeleteTcontMessage proto.InternalMessageInfo
-
-func (m *InterAdapterDeleteTcontMessage) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *InterAdapterDeleteTcontMessage) GetTpInstancePath() string {
-	if m != nil {
-		return m.TpInstancePath
-	}
-	return ""
-}
-
-func (m *InterAdapterDeleteTcontMessage) GetAllocId() uint32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-type InterAdapterResponseBody struct {
-	Status bool `protobuf:"varint,1,opt,name=status,proto3" json:"status,omitempty"`
-	// Types that are valid to be assigned to Payload:
-	//	*InterAdapterResponseBody_Body
-	//	*InterAdapterResponseBody_Omci
-	Payload              isInterAdapterResponseBody_Payload `protobuf_oneof:"payload"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
-}
-
-func (m *InterAdapterResponseBody) Reset()         { *m = InterAdapterResponseBody{} }
-func (m *InterAdapterResponseBody) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterResponseBody) ProtoMessage()    {}
-func (*InterAdapterResponseBody) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{20}
-}
-
-func (m *InterAdapterResponseBody) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterResponseBody.Unmarshal(m, b)
-}
-func (m *InterAdapterResponseBody) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterResponseBody.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterResponseBody) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterResponseBody.Merge(m, src)
-}
-func (m *InterAdapterResponseBody) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterResponseBody.Size(m)
-}
-func (m *InterAdapterResponseBody) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterResponseBody.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterResponseBody proto.InternalMessageInfo
-
-func (m *InterAdapterResponseBody) GetStatus() bool {
-	if m != nil {
-		return m.Status
-	}
-	return false
-}
-
-type isInterAdapterResponseBody_Payload interface {
-	isInterAdapterResponseBody_Payload()
-}
-
-type InterAdapterResponseBody_Body struct {
-	Body *any.Any `protobuf:"bytes,2,opt,name=body,proto3,oneof"`
-}
-
-type InterAdapterResponseBody_Omci struct {
-	Omci *InterAdapterOmciMessage `protobuf:"bytes,3,opt,name=omci,proto3,oneof"`
-}
-
-func (*InterAdapterResponseBody_Body) isInterAdapterResponseBody_Payload() {}
-
-func (*InterAdapterResponseBody_Omci) isInterAdapterResponseBody_Payload() {}
-
-func (m *InterAdapterResponseBody) GetPayload() isInterAdapterResponseBody_Payload {
-	if m != nil {
-		return m.Payload
-	}
-	return nil
-}
-
-func (m *InterAdapterResponseBody) GetBody() *any.Any {
-	if x, ok := m.GetPayload().(*InterAdapterResponseBody_Body); ok {
-		return x.Body
-	}
-	return nil
-}
-
-func (m *InterAdapterResponseBody) GetOmci() *InterAdapterOmciMessage {
-	if x, ok := m.GetPayload().(*InterAdapterResponseBody_Omci); ok {
-		return x.Omci
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*InterAdapterResponseBody) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*InterAdapterResponseBody_Body)(nil),
-		(*InterAdapterResponseBody_Omci)(nil),
-	}
-}
-
-type InterAdapterMessage struct {
-	Header               *InterAdapterHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
-	Body                 *any.Any            `protobuf:"bytes,2,opt,name=body,proto3" json:"body,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *InterAdapterMessage) Reset()         { *m = InterAdapterMessage{} }
-func (m *InterAdapterMessage) String() string { return proto.CompactTextString(m) }
-func (*InterAdapterMessage) ProtoMessage()    {}
-func (*InterAdapterMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_941f0031a549667f, []int{21}
-}
-
-func (m *InterAdapterMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InterAdapterMessage.Unmarshal(m, b)
-}
-func (m *InterAdapterMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InterAdapterMessage.Marshal(b, m, deterministic)
-}
-func (m *InterAdapterMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InterAdapterMessage.Merge(m, src)
-}
-func (m *InterAdapterMessage) XXX_Size() int {
-	return xxx_messageInfo_InterAdapterMessage.Size(m)
-}
-func (m *InterAdapterMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_InterAdapterMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InterAdapterMessage proto.InternalMessageInfo
-
-func (m *InterAdapterMessage) GetHeader() *InterAdapterHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *InterAdapterMessage) GetBody() *any.Any {
-	if m != nil {
-		return m.Body
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("voltha.MessageType", MessageType_name, MessageType_value)
-	proto.RegisterEnum("voltha.ErrorCodeCodes", ErrorCodeCodes_name, ErrorCodeCodes_value)
-	proto.RegisterEnum("voltha.InterAdapterMessageType_Types", InterAdapterMessageType_Types_name, InterAdapterMessageType_Types_value)
-	proto.RegisterType((*StrType)(nil), "voltha.StrType")
-	proto.RegisterType((*IntType)(nil), "voltha.IntType")
-	proto.RegisterType((*BoolType)(nil), "voltha.BoolType")
-	proto.RegisterType((*Packet)(nil), "voltha.Packet")
-	proto.RegisterType((*ErrorCode)(nil), "voltha.ErrorCode")
-	proto.RegisterType((*Error)(nil), "voltha.Error")
-	proto.RegisterType((*Header)(nil), "voltha.Header")
-	proto.RegisterType((*Argument)(nil), "voltha.Argument")
-	proto.RegisterType((*InterContainerMessage)(nil), "voltha.InterContainerMessage")
-	proto.RegisterType((*InterContainerRequestBody)(nil), "voltha.InterContainerRequestBody")
-	proto.RegisterType((*InterContainerResponseBody)(nil), "voltha.InterContainerResponseBody")
-	proto.RegisterType((*SwitchCapability)(nil), "voltha.SwitchCapability")
-	proto.RegisterType((*DeviceDiscovered)(nil), "voltha.DeviceDiscovered")
-	proto.RegisterType((*InterAdapterMessageType)(nil), "voltha.InterAdapterMessageType")
-	proto.RegisterType((*InterAdapterHeader)(nil), "voltha.InterAdapterHeader")
-	proto.RegisterType((*InterAdapterOmciMessage)(nil), "voltha.InterAdapterOmciMessage")
-	proto.RegisterType((*InterAdapterTechProfileInstanceRequestMessage)(nil), "voltha.InterAdapterTechProfileInstanceRequestMessage")
-	proto.RegisterType((*InterAdapterTechProfileDownloadMessage)(nil), "voltha.InterAdapterTechProfileDownloadMessage")
-	proto.RegisterType((*InterAdapterDeleteGemPortMessage)(nil), "voltha.InterAdapterDeleteGemPortMessage")
-	proto.RegisterType((*InterAdapterDeleteTcontMessage)(nil), "voltha.InterAdapterDeleteTcontMessage")
-	proto.RegisterType((*InterAdapterResponseBody)(nil), "voltha.InterAdapterResponseBody")
-	proto.RegisterType((*InterAdapterMessage)(nil), "voltha.InterAdapterMessage")
-}
-
-func init() {
-	proto.RegisterFile("voltha_protos/inter_container.proto", fileDescriptor_941f0031a549667f)
-}
-
-var fileDescriptor_941f0031a549667f = []byte{
-	// 1468 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xdd, 0x72, 0xdb, 0x44,
-	0x14, 0x8e, 0x1d, 0xff, 0x1e, 0x27, 0xae, 0xbb, 0x69, 0x1a, 0x27, 0xe9, 0x4f, 0x2a, 0xda, 0x12,
-	0x5a, 0xea, 0x0c, 0x29, 0x0c, 0x70, 0x05, 0x8e, 0xad, 0x36, 0x9a, 0x49, 0x6c, 0x55, 0x56, 0x5a,
-	0x86, 0x61, 0x46, 0xa3, 0x48, 0x1b, 0x5b, 0x13, 0x59, 0xab, 0x4a, 0xab, 0x14, 0xcf, 0x30, 0xcc,
-	0x30, 0xdc, 0xf0, 0x04, 0xdc, 0x31, 0xc3, 0x15, 0xef, 0xc0, 0x6b, 0x70, 0xc7, 0xdb, 0x30, 0xfb,
-	0x23, 0x5b, 0x76, 0x13, 0x18, 0xa0, 0x77, 0xda, 0xf3, 0x7d, 0xfb, 0xed, 0xee, 0x39, 0x7b, 0xce,
-	0x1e, 0xc1, 0x7b, 0x17, 0xc4, 0xa7, 0x23, 0xdb, 0x0a, 0x23, 0x42, 0x49, 0xbc, 0xe7, 0x05, 0x14,
-	0x47, 0x96, 0x43, 0x02, 0x6a, 0x7b, 0x01, 0x8e, 0x5a, 0xdc, 0x8c, 0x4a, 0x82, 0xb4, 0xb5, 0x35,
-	0x4f, 0x76, 0xc8, 0x78, 0x4c, 0x02, 0xc1, 0x59, 0xc4, 0xc4, 0x48, 0x62, 0x9b, 0x43, 0x42, 0x86,
-	0x3e, 0xde, 0xe3, 0xa3, 0xd3, 0xe4, 0x6c, 0xcf, 0x0e, 0x26, 0x12, 0xba, 0x3b, 0x3f, 0x8d, 0x84,
-	0x38, 0x38, 0xf3, 0xc9, 0x1b, 0xeb, 0xa3, 0xa7, 0x92, 0xa0, 0xcc, 0x13, 0x7c, 0x32, 0xf4, 0x1c,
-	0xdb, 0xb7, 0x5c, 0x7c, 0xe1, 0x39, 0x38, 0x15, 0x59, 0xd4, 0xa7, 0xde, 0x18, 0xc7, 0xd4, 0x1e,
-	0x87, 0x92, 0xb0, 0x33, 0x2f, 0x42, 0xb1, 0x33, 0x62, 0xdf, 0x67, 0x9e, 0x2f, 0x25, 0x94, 0x6d,
-	0x28, 0x0f, 0x68, 0x64, 0x4e, 0x42, 0x8c, 0x1a, 0xb0, 0x7c, 0x61, 0xfb, 0xcd, 0xdc, 0x4e, 0x6e,
-	0xb7, 0x6a, 0xb0, 0x4f, 0x06, 0x6a, 0x01, 0x5d, 0x04, 0x97, 0x05, 0x78, 0x0b, 0x2a, 0x07, 0x84,
-	0xf8, 0x8b, 0x68, 0x45, 0xa0, 0x0a, 0x94, 0x74, 0xdb, 0x39, 0xc7, 0x14, 0x35, 0xa1, 0x1c, 0xda,
-	0x13, 0x9f, 0xd8, 0x2e, 0xc7, 0x57, 0x8c, 0x74, 0xa8, 0x7c, 0x03, 0x55, 0x35, 0x8a, 0x48, 0xd4,
-	0x21, 0x2e, 0x56, 0xfa, 0x50, 0x74, 0x88, 0x8b, 0x63, 0xb4, 0x01, 0x6b, 0x27, 0xbd, 0xc1, 0x89,
-	0xae, 0xf7, 0x0d, 0x53, 0xed, 0x5a, 0x86, 0xfa, 0xe2, 0x44, 0x1d, 0x98, 0x8d, 0x25, 0x74, 0x13,
-	0x90, 0xd6, 0x7b, 0xd9, 0x3e, 0xd2, 0xba, 0x96, 0xde, 0x36, 0xda, 0xc7, 0xaa, 0xa9, 0x1a, 0x83,
-	0x46, 0x0e, 0xad, 0xc3, 0xf5, 0xae, 0xda, 0xee, 0x1e, 0x69, 0x3d, 0xd5, 0x52, 0xbf, 0xea, 0xa8,
-	0x6a, 0x57, 0xed, 0x36, 0xf2, 0xca, 0x11, 0x14, 0xb9, 0x3a, 0x7a, 0x0c, 0x05, 0xa6, 0xcc, 0x57,
-	0xaf, 0xef, 0x6f, 0xb4, 0x64, 0x88, 0xa6, 0x4b, 0xb7, 0xf8, 0xba, 0x06, 0x27, 0xa1, 0x9b, 0x50,
-	0x8a, 0xb0, 0x1d, 0x93, 0xa0, 0x99, 0xe7, 0x7e, 0x90, 0x23, 0xe5, 0x8f, 0x1c, 0x94, 0x0e, 0xb1,
-	0xed, 0xe2, 0x08, 0xd5, 0x21, 0xef, 0xb9, 0xd2, 0x4d, 0x79, 0xcf, 0x45, 0xef, 0x43, 0x81, 0x4e,
-	0x42, 0xcc, 0x27, 0xd4, 0xf7, 0xd7, 0x52, 0xfd, 0x63, 0x1c, 0xc7, 0xf6, 0x10, 0x33, 0xff, 0x18,
-	0x9c, 0x80, 0x6e, 0x03, 0x9c, 0x45, 0x64, 0x6c, 0x51, 0x12, 0x7a, 0x4e, 0x73, 0x99, 0x0b, 0x54,
-	0x99, 0xc5, 0x64, 0x06, 0xb4, 0x09, 0x15, 0x4a, 0x24, 0x58, 0xe0, 0x60, 0x99, 0x12, 0x01, 0x6d,
-	0x43, 0xf5, 0x1c, 0x4f, 0x24, 0x56, 0xe4, 0x58, 0xe5, 0x1c, 0x4f, 0x04, 0xf8, 0x19, 0x54, 0xa7,
-	0x71, 0x6f, 0x96, 0x76, 0x72, 0xbb, 0xb5, 0xfd, 0xad, 0x96, 0xb8, 0x19, 0xad, 0xf4, 0x66, 0xb4,
-	0xcc, 0x94, 0x61, 0xcc, 0xc8, 0xca, 0x21, 0x54, 0xda, 0xd1, 0x30, 0x19, 0xe3, 0x80, 0xb2, 0x10,
-	0x9e, 0xe3, 0x49, 0x1a, 0xfd, 0x73, 0x3c, 0x41, 0x8f, 0xa0, 0x78, 0x61, 0xfb, 0x89, 0x38, 0x58,
-	0x6d, 0xff, 0xc6, 0x5b, 0x9a, 0xed, 0x60, 0x62, 0x08, 0x8a, 0xe2, 0xc1, 0xba, 0xc6, 0x52, 0xa8,
-	0x93, 0x66, 0x90, 0x3c, 0x3d, 0x7a, 0x08, 0xa5, 0x11, 0x77, 0x1b, 0x57, 0xae, 0xed, 0xd7, 0x53,
-	0xf7, 0x08, 0x67, 0x1a, 0x12, 0x45, 0xbb, 0x50, 0x38, 0x25, 0xee, 0xe4, 0x6f, 0xd7, 0xe2, 0x0c,
-	0xe5, 0xb7, 0x1c, 0x6c, 0xce, 0xaf, 0x65, 0xe0, 0xd7, 0x09, 0x8e, 0xe9, 0x01, 0x71, 0x27, 0xec,
-	0x18, 0x51, 0xe8, 0xc8, 0xe0, 0xb1, 0x4f, 0x74, 0x1f, 0x0a, 0x76, 0x34, 0x8c, 0x9b, 0xcb, 0x3b,
-	0xcb, 0xbb, 0xb5, 0xfd, 0x46, 0xba, 0x7e, 0x7a, 0x70, 0x83, 0xa3, 0xe8, 0x31, 0x5c, 0x8f, 0x70,
-	0x1c, 0x92, 0x20, 0xc6, 0x56, 0x84, 0x5f, 0x27, 0x5e, 0x84, 0x5d, 0x1e, 0x85, 0x8a, 0xd1, 0x48,
-	0x01, 0x43, 0xda, 0xd1, 0x7d, 0xa8, 0x47, 0x38, 0xf4, 0x59, 0x40, 0xe6, 0x62, 0xb2, 0xc2, 0xad,
-	0xa6, 0x08, 0x9a, 0xe2, 0xc2, 0xd6, 0xe2, 0x3e, 0x85, 0x0e, 0xdf, 0x68, 0x13, 0xca, 0x71, 0xe2,
-	0x38, 0x38, 0x8e, 0x65, 0xda, 0xa4, 0x43, 0xf4, 0x21, 0xbb, 0x82, 0x71, 0xe2, 0x53, 0x7e, 0x45,
-	0xae, 0x72, 0x86, 0xe4, 0x28, 0x3f, 0xe5, 0xa0, 0x31, 0x78, 0xe3, 0x51, 0x67, 0xd4, 0xb1, 0x43,
-	0xfb, 0xd4, 0xf3, 0x3d, 0x3a, 0x41, 0x1f, 0x40, 0xc1, 0xc5, 0xb1, 0x23, 0x7d, 0xbe, 0xde, 0xca,
-	0x96, 0x17, 0x72, 0x16, 0x5a, 0x0c, 0x34, 0x38, 0x05, 0x69, 0x70, 0x2d, 0xe6, 0xd3, 0xad, 0x33,
-	0x6c, 0xd3, 0x24, 0xc2, 0xb1, 0x8c, 0xc1, 0xce, 0x5b, 0xb3, 0x16, 0x78, 0x46, 0x5d, 0x18, 0x9e,
-	0xc9, 0xb1, 0xf2, 0x3d, 0x34, 0xba, 0xbc, 0x3c, 0x75, 0xbd, 0xd8, 0x21, 0x17, 0x98, 0xb9, 0x6a,
-	0x31, 0x59, 0xb6, 0xa1, 0x1a, 0xda, 0x11, 0x0e, 0xa8, 0xe5, 0xb9, 0x32, 0x4a, 0x15, 0x61, 0xd0,
-	0x5c, 0x74, 0x17, 0x6a, 0xa2, 0xbe, 0x59, 0x3c, 0xa1, 0x44, 0x86, 0x80, 0x30, 0xf1, 0x3a, 0x73,
-	0x0b, 0xaa, 0x61, 0x72, 0xea, 0x7b, 0xf1, 0x08, 0x47, 0x32, 0x47, 0x66, 0x06, 0xe5, 0x97, 0x3c,
-	0x6c, 0x70, 0x8f, 0xb7, 0x5d, 0x3b, 0xa4, 0xd3, 0x3b, 0xc8, 0x66, 0x2a, 0x3f, 0xe4, 0xa1, 0xc8,
-	0x3e, 0x62, 0xd4, 0x80, 0x95, 0x67, 0x47, 0xfd, 0x57, 0x99, 0xc2, 0x72, 0x1d, 0x56, 0xa5, 0x65,
-	0xa0, 0xf7, 0x7b, 0x03, 0xb5, 0x91, 0x63, 0xa4, 0xfe, 0x71, 0x47, 0x9b, 0x92, 0xf2, 0x8c, 0x24,
-	0x2d, 0x92, 0xb4, 0x8c, 0xd6, 0xe0, 0xda, 0xb1, 0x6a, 0x1a, 0x5a, 0x67, 0x30, 0xe5, 0x15, 0xd0,
-	0x0d, 0x68, 0xcc, 0x8c, 0x92, 0x5a, 0x64, 0xd4, 0x7e, 0xef, 0xc4, 0xd2, 0x7a, 0xb3, 0x82, 0x56,
-	0x62, 0xd4, 0x99, 0x51, 0x52, 0xcb, 0xe8, 0x1e, 0xdc, 0x36, 0xd5, 0xce, 0xa1, 0xa5, 0x1b, 0xfd,
-	0x67, 0xda, 0x91, 0x6a, 0x75, 0xfb, 0xaf, 0x7a, 0x47, 0xfd, 0xf6, 0x6c, 0x62, 0x05, 0x6d, 0xc3,
-	0x46, 0x57, 0x3d, 0x52, 0x4d, 0xd5, 0x7a, 0xae, 0x1e, 0x5b, 0xac, 0x50, 0x4e, 0xc1, 0x2a, 0x6a,
-	0xc2, 0x0d, 0x09, 0x9a, 0x9d, 0x7e, 0x6f, 0x86, 0x00, 0xf3, 0x0f, 0xca, 0xfa, 0xe7, 0x8a, 0x7a,
-	0xf6, 0xf9, 0x5c, 0x3d, 0x7b, 0x90, 0x26, 0xcc, 0x15, 0x9e, 0x6d, 0x71, 0xaf, 0xfe, 0xef, 0x0a,
-	0xb7, 0x03, 0x2b, 0x94, 0xc8, 0xd7, 0x8d, 0x5d, 0x0d, 0x91, 0x50, 0x40, 0x89, 0xb8, 0x51, 0x9a,
-	0x8b, 0x1e, 0xc2, 0xb5, 0x30, 0x22, 0xdf, 0x4e, 0x32, 0xa4, 0x12, 0x27, 0xad, 0x72, 0xf3, 0x94,
-	0x37, 0x57, 0x0e, 0xcb, 0xff, 0xa6, 0x1c, 0xfe, 0x9e, 0x9b, 0xbf, 0x3f, 0xfd, 0xb1, 0xe3, 0xa5,
-	0x75, 0xac, 0x09, 0xe5, 0xb1, 0xf8, 0x4c, 0x5f, 0x31, 0x39, 0x44, 0x07, 0x50, 0x77, 0x48, 0x10,
-	0x60, 0x87, 0x5a, 0x31, 0xb5, 0x69, 0x12, 0x4b, 0xc7, 0x6d, 0xb7, 0x64, 0x9f, 0xd0, 0x11, 0xe8,
-	0x80, 0x83, 0xd2, 0x5d, 0xab, 0x4e, 0xd6, 0x88, 0xbe, 0x04, 0x71, 0x08, 0xcb, 0x76, 0xdd, 0x88,
-	0x95, 0x04, 0x91, 0xf9, 0xdb, 0xa9, 0xef, 0xc5, 0xe1, 0x5a, 0x3a, 0xe3, 0xb4, 0x05, 0xc5, 0x58,
-	0x09, 0x33, 0x23, 0xe5, 0xcf, 0x1c, 0x3c, 0xc9, 0xee, 0xdd, 0xc4, 0xce, 0x48, 0x17, 0x2f, 0xbd,
-	0x16, 0xc4, 0xd4, 0x0e, 0x1c, 0x2c, 0xcb, 0x64, 0x7a, 0xa2, 0x5d, 0x68, 0xd0, 0xd0, 0xf2, 0x24,
-	0x68, 0x85, 0x36, 0x1d, 0xc9, 0x4b, 0x50, 0xa7, 0x61, 0x3a, 0x47, 0xb7, 0xe9, 0x88, 0x31, 0x65,
-	0xce, 0xce, 0x5c, 0x2f, 0x52, 0xb7, 0x2e, 0xec, 0x73, 0x31, 0x12, 0xcc, 0x90, 0x04, 0x56, 0x48,
-	0x22, 0x51, 0xc3, 0x56, 0x8d, 0x55, 0x61, 0xd6, 0x49, 0xa0, 0x93, 0x88, 0xa2, 0x75, 0x28, 0x91,
-	0x20, 0x61, 0x3a, 0x05, 0x0e, 0x17, 0x49, 0x90, 0x68, 0x2e, 0x33, 0x27, 0x81, 0x97, 0x86, 0x7f,
-	0xd5, 0x28, 0x26, 0x81, 0xa7, 0xb9, 0xca, 0xcf, 0x79, 0x78, 0x78, 0xc5, 0xd9, 0xba, 0xe4, 0x4d,
-	0xc0, 0x7a, 0x89, 0xf4, 0x50, 0x33, 0x85, 0x5c, 0x46, 0xe1, 0xd2, 0xb3, 0xe6, 0x2f, 0x3d, 0x6b,
-	0x17, 0x6a, 0x19, 0xa6, 0x8c, 0xc3, 0xbd, 0xd6, 0x5c, 0xe7, 0x74, 0x89, 0x6f, 0x0f, 0x97, 0x0c,
-	0x98, 0x29, 0xa1, 0x17, 0xd0, 0xc0, 0xcc, 0x03, 0x59, 0xa9, 0x02, 0x97, 0x7a, 0x30, 0x2f, 0xa5,
-	0x86, 0x24, 0xb8, 0x5c, 0xae, 0xce, 0x04, 0xcc, 0xa9, 0xe4, 0x01, 0x82, 0x06, 0x9f, 0x99, 0x91,
-	0x54, 0x7e, 0xcc, 0xc1, 0x4e, 0xd6, 0x31, 0x5d, 0xec, 0x63, 0x8a, 0x9f, 0xe3, 0x31, 0x73, 0xf2,
-	0x3b, 0x73, 0xc9, 0x1d, 0xa8, 0x0d, 0xf1, 0x98, 0x47, 0x93, 0xa9, 0x88, 0x80, 0x56, 0x87, 0x62,
-	0x15, 0xcd, 0x55, 0xbe, 0x83, 0x3b, 0x6f, 0x6f, 0xc2, 0x64, 0xbd, 0xf4, 0x3b, 0xdb, 0xc2, 0x26,
-	0x54, 0x6c, 0xdf, 0x27, 0xce, 0x6c, 0xfd, 0x32, 0x1f, 0x6b, 0xae, 0xf2, 0x6b, 0x0e, 0x9a, 0xd9,
-	0xe5, 0xe7, 0x1e, 0xd9, 0x9b, 0x50, 0x92, 0x39, 0x29, 0xde, 0x58, 0x39, 0x42, 0x8f, 0xfe, 0xb9,
-	0xdb, 0x38, 0x5c, 0x12, 0xfd, 0x06, 0xfa, 0x04, 0x0a, 0x64, 0xec, 0x78, 0xf2, 0x2a, 0xdc, 0xbd,
-	0xac, 0x1c, 0x66, 0x0a, 0x05, 0x9b, 0xc6, 0xe8, 0x07, 0xd5, 0x69, 0xdb, 0xab, 0xc4, 0xb0, 0x76,
-	0x49, 0xf1, 0x44, 0xfb, 0x0b, 0xad, 0xd1, 0xd6, 0x65, 0xd2, 0xff, 0xb5, 0x4d, 0x7a, 0xf4, 0x05,
-	0xd4, 0x32, 0x55, 0x1a, 0xd5, 0xa0, 0x3c, 0x7b, 0xf0, 0x56, 0xa0, 0x92, 0x79, 0xeb, 0x78, 0xff,
-	0xfc, 0x52, 0xeb, 0xa8, 0x56, 0x57, 0x1b, 0x74, 0xfa, 0x2f, 0x55, 0x83, 0xf5, 0xcf, 0x07, 0x3d,
-	0x58, 0x23, 0xd1, 0x90, 0x37, 0x01, 0x0e, 0x89, 0x5c, 0xb9, 0xb9, 0xaf, 0x3f, 0x1d, 0x7a, 0x74,
-	0x94, 0x9c, 0xb2, 0xe2, 0xb6, 0x97, 0x62, 0xf2, 0xaf, 0xe7, 0x49, 0xfa, 0x0f, 0xf4, 0xf1, 0xde,
-	0x90, 0x2c, 0xfe, 0x52, 0xe9, 0x4b, 0x7a, 0x4e, 0x2f, 0x9c, 0x96, 0x38, 0xe7, 0xe9, 0x5f, 0x01,
-	0x00, 0x00, 0xff, 0xff, 0xb3, 0x9d, 0x9e, 0x3e, 0x80, 0x0d, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_alarm_db.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_alarm_db.pb.go
deleted file mode 100644
index 58824c0..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_alarm_db.pb.go
+++ /dev/null
@@ -1,516 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/omci_alarm_db.proto
-
-package omci
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	_ "github.com/opencord/voltha-protos/v4/go/common"
-	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 AlarmOpenOmciEventType_OpenOmciEventType int32
-
-const (
-	AlarmOpenOmciEventType_state_change AlarmOpenOmciEventType_OpenOmciEventType = 0
-)
-
-var AlarmOpenOmciEventType_OpenOmciEventType_name = map[int32]string{
-	0: "state_change",
-}
-
-var AlarmOpenOmciEventType_OpenOmciEventType_value = map[string]int32{
-	"state_change": 0,
-}
-
-func (x AlarmOpenOmciEventType_OpenOmciEventType) String() string {
-	return proto.EnumName(AlarmOpenOmciEventType_OpenOmciEventType_name, int32(x))
-}
-
-func (AlarmOpenOmciEventType_OpenOmciEventType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{6, 0}
-}
-
-type AlarmAttributeData struct {
-	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AlarmAttributeData) Reset()         { *m = AlarmAttributeData{} }
-func (m *AlarmAttributeData) String() string { return proto.CompactTextString(m) }
-func (*AlarmAttributeData) ProtoMessage()    {}
-func (*AlarmAttributeData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{0}
-}
-
-func (m *AlarmAttributeData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmAttributeData.Unmarshal(m, b)
-}
-func (m *AlarmAttributeData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmAttributeData.Marshal(b, m, deterministic)
-}
-func (m *AlarmAttributeData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmAttributeData.Merge(m, src)
-}
-func (m *AlarmAttributeData) XXX_Size() int {
-	return xxx_messageInfo_AlarmAttributeData.Size(m)
-}
-func (m *AlarmAttributeData) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmAttributeData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmAttributeData proto.InternalMessageInfo
-
-func (m *AlarmAttributeData) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AlarmAttributeData) GetValue() string {
-	if m != nil {
-		return m.Value
-	}
-	return ""
-}
-
-type AlarmInstanceData struct {
-	InstanceId           uint32                `protobuf:"varint,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
-	Created              string                `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
-	Modified             string                `protobuf:"bytes,3,opt,name=modified,proto3" json:"modified,omitempty"`
-	Attributes           []*AlarmAttributeData `protobuf:"bytes,4,rep,name=attributes,proto3" json:"attributes,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *AlarmInstanceData) Reset()         { *m = AlarmInstanceData{} }
-func (m *AlarmInstanceData) String() string { return proto.CompactTextString(m) }
-func (*AlarmInstanceData) ProtoMessage()    {}
-func (*AlarmInstanceData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{1}
-}
-
-func (m *AlarmInstanceData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmInstanceData.Unmarshal(m, b)
-}
-func (m *AlarmInstanceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmInstanceData.Marshal(b, m, deterministic)
-}
-func (m *AlarmInstanceData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmInstanceData.Merge(m, src)
-}
-func (m *AlarmInstanceData) XXX_Size() int {
-	return xxx_messageInfo_AlarmInstanceData.Size(m)
-}
-func (m *AlarmInstanceData) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmInstanceData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmInstanceData proto.InternalMessageInfo
-
-func (m *AlarmInstanceData) GetInstanceId() uint32 {
-	if m != nil {
-		return m.InstanceId
-	}
-	return 0
-}
-
-func (m *AlarmInstanceData) GetCreated() string {
-	if m != nil {
-		return m.Created
-	}
-	return ""
-}
-
-func (m *AlarmInstanceData) GetModified() string {
-	if m != nil {
-		return m.Modified
-	}
-	return ""
-}
-
-func (m *AlarmInstanceData) GetAttributes() []*AlarmAttributeData {
-	if m != nil {
-		return m.Attributes
-	}
-	return nil
-}
-
-type AlarmClassData struct {
-	ClassId              uint32               `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
-	Instances            []*AlarmInstanceData `protobuf:"bytes,2,rep,name=instances,proto3" json:"instances,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *AlarmClassData) Reset()         { *m = AlarmClassData{} }
-func (m *AlarmClassData) String() string { return proto.CompactTextString(m) }
-func (*AlarmClassData) ProtoMessage()    {}
-func (*AlarmClassData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{2}
-}
-
-func (m *AlarmClassData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmClassData.Unmarshal(m, b)
-}
-func (m *AlarmClassData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmClassData.Marshal(b, m, deterministic)
-}
-func (m *AlarmClassData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmClassData.Merge(m, src)
-}
-func (m *AlarmClassData) XXX_Size() int {
-	return xxx_messageInfo_AlarmClassData.Size(m)
-}
-func (m *AlarmClassData) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmClassData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmClassData proto.InternalMessageInfo
-
-func (m *AlarmClassData) GetClassId() uint32 {
-	if m != nil {
-		return m.ClassId
-	}
-	return 0
-}
-
-func (m *AlarmClassData) GetInstances() []*AlarmInstanceData {
-	if m != nil {
-		return m.Instances
-	}
-	return nil
-}
-
-type AlarmManagedEntity struct {
-	ClassId              uint32   `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
-	Name                 string   `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AlarmManagedEntity) Reset()         { *m = AlarmManagedEntity{} }
-func (m *AlarmManagedEntity) String() string { return proto.CompactTextString(m) }
-func (*AlarmManagedEntity) ProtoMessage()    {}
-func (*AlarmManagedEntity) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{3}
-}
-
-func (m *AlarmManagedEntity) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmManagedEntity.Unmarshal(m, b)
-}
-func (m *AlarmManagedEntity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmManagedEntity.Marshal(b, m, deterministic)
-}
-func (m *AlarmManagedEntity) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmManagedEntity.Merge(m, src)
-}
-func (m *AlarmManagedEntity) XXX_Size() int {
-	return xxx_messageInfo_AlarmManagedEntity.Size(m)
-}
-func (m *AlarmManagedEntity) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmManagedEntity.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmManagedEntity proto.InternalMessageInfo
-
-func (m *AlarmManagedEntity) GetClassId() uint32 {
-	if m != nil {
-		return m.ClassId
-	}
-	return 0
-}
-
-func (m *AlarmManagedEntity) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type AlarmMessageType struct {
-	MessageType          uint32   `protobuf:"varint,1,opt,name=message_type,json=messageType,proto3" json:"message_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AlarmMessageType) Reset()         { *m = AlarmMessageType{} }
-func (m *AlarmMessageType) String() string { return proto.CompactTextString(m) }
-func (*AlarmMessageType) ProtoMessage()    {}
-func (*AlarmMessageType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{4}
-}
-
-func (m *AlarmMessageType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmMessageType.Unmarshal(m, b)
-}
-func (m *AlarmMessageType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmMessageType.Marshal(b, m, deterministic)
-}
-func (m *AlarmMessageType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmMessageType.Merge(m, src)
-}
-func (m *AlarmMessageType) XXX_Size() int {
-	return xxx_messageInfo_AlarmMessageType.Size(m)
-}
-func (m *AlarmMessageType) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmMessageType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmMessageType proto.InternalMessageInfo
-
-func (m *AlarmMessageType) GetMessageType() uint32 {
-	if m != nil {
-		return m.MessageType
-	}
-	return 0
-}
-
-type AlarmDeviceData struct {
-	DeviceId             string                `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	Created              string                `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
-	LastAlarmSequence    uint32                `protobuf:"varint,3,opt,name=last_alarm_sequence,json=lastAlarmSequence,proto3" json:"last_alarm_sequence,omitempty"`
-	LastSyncTime         string                `protobuf:"bytes,4,opt,name=last_sync_time,json=lastSyncTime,proto3" json:"last_sync_time,omitempty"`
-	Version              uint32                `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"`
-	Classes              []*AlarmClassData     `protobuf:"bytes,6,rep,name=classes,proto3" json:"classes,omitempty"`
-	ManagedEntities      []*AlarmManagedEntity `protobuf:"bytes,7,rep,name=managed_entities,json=managedEntities,proto3" json:"managed_entities,omitempty"`
-	MessageTypes         []*AlarmMessageType   `protobuf:"bytes,8,rep,name=message_types,json=messageTypes,proto3" json:"message_types,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *AlarmDeviceData) Reset()         { *m = AlarmDeviceData{} }
-func (m *AlarmDeviceData) String() string { return proto.CompactTextString(m) }
-func (*AlarmDeviceData) ProtoMessage()    {}
-func (*AlarmDeviceData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{5}
-}
-
-func (m *AlarmDeviceData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmDeviceData.Unmarshal(m, b)
-}
-func (m *AlarmDeviceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmDeviceData.Marshal(b, m, deterministic)
-}
-func (m *AlarmDeviceData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmDeviceData.Merge(m, src)
-}
-func (m *AlarmDeviceData) XXX_Size() int {
-	return xxx_messageInfo_AlarmDeviceData.Size(m)
-}
-func (m *AlarmDeviceData) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmDeviceData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmDeviceData proto.InternalMessageInfo
-
-func (m *AlarmDeviceData) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *AlarmDeviceData) GetCreated() string {
-	if m != nil {
-		return m.Created
-	}
-	return ""
-}
-
-func (m *AlarmDeviceData) GetLastAlarmSequence() uint32 {
-	if m != nil {
-		return m.LastAlarmSequence
-	}
-	return 0
-}
-
-func (m *AlarmDeviceData) GetLastSyncTime() string {
-	if m != nil {
-		return m.LastSyncTime
-	}
-	return ""
-}
-
-func (m *AlarmDeviceData) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *AlarmDeviceData) GetClasses() []*AlarmClassData {
-	if m != nil {
-		return m.Classes
-	}
-	return nil
-}
-
-func (m *AlarmDeviceData) GetManagedEntities() []*AlarmManagedEntity {
-	if m != nil {
-		return m.ManagedEntities
-	}
-	return nil
-}
-
-func (m *AlarmDeviceData) GetMessageTypes() []*AlarmMessageType {
-	if m != nil {
-		return m.MessageTypes
-	}
-	return nil
-}
-
-type AlarmOpenOmciEventType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AlarmOpenOmciEventType) Reset()         { *m = AlarmOpenOmciEventType{} }
-func (m *AlarmOpenOmciEventType) String() string { return proto.CompactTextString(m) }
-func (*AlarmOpenOmciEventType) ProtoMessage()    {}
-func (*AlarmOpenOmciEventType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{6}
-}
-
-func (m *AlarmOpenOmciEventType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmOpenOmciEventType.Unmarshal(m, b)
-}
-func (m *AlarmOpenOmciEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmOpenOmciEventType.Marshal(b, m, deterministic)
-}
-func (m *AlarmOpenOmciEventType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmOpenOmciEventType.Merge(m, src)
-}
-func (m *AlarmOpenOmciEventType) XXX_Size() int {
-	return xxx_messageInfo_AlarmOpenOmciEventType.Size(m)
-}
-func (m *AlarmOpenOmciEventType) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmOpenOmciEventType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmOpenOmciEventType proto.InternalMessageInfo
-
-type AlarmOpenOmciEvent struct {
-	Type                 AlarmOpenOmciEventType_OpenOmciEventType `protobuf:"varint,1,opt,name=type,proto3,enum=omci.AlarmOpenOmciEventType_OpenOmciEventType" json:"type,omitempty"`
-	Data                 string                                   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                 `json:"-"`
-	XXX_unrecognized     []byte                                   `json:"-"`
-	XXX_sizecache        int32                                    `json:"-"`
-}
-
-func (m *AlarmOpenOmciEvent) Reset()         { *m = AlarmOpenOmciEvent{} }
-func (m *AlarmOpenOmciEvent) String() string { return proto.CompactTextString(m) }
-func (*AlarmOpenOmciEvent) ProtoMessage()    {}
-func (*AlarmOpenOmciEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8d41f1e38aadb08d, []int{7}
-}
-
-func (m *AlarmOpenOmciEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmOpenOmciEvent.Unmarshal(m, b)
-}
-func (m *AlarmOpenOmciEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmOpenOmciEvent.Marshal(b, m, deterministic)
-}
-func (m *AlarmOpenOmciEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmOpenOmciEvent.Merge(m, src)
-}
-func (m *AlarmOpenOmciEvent) XXX_Size() int {
-	return xxx_messageInfo_AlarmOpenOmciEvent.Size(m)
-}
-func (m *AlarmOpenOmciEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmOpenOmciEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmOpenOmciEvent proto.InternalMessageInfo
-
-func (m *AlarmOpenOmciEvent) GetType() AlarmOpenOmciEventType_OpenOmciEventType {
-	if m != nil {
-		return m.Type
-	}
-	return AlarmOpenOmciEventType_state_change
-}
-
-func (m *AlarmOpenOmciEvent) GetData() string {
-	if m != nil {
-		return m.Data
-	}
-	return ""
-}
-
-func init() {
-	proto.RegisterEnum("omci.AlarmOpenOmciEventType_OpenOmciEventType", AlarmOpenOmciEventType_OpenOmciEventType_name, AlarmOpenOmciEventType_OpenOmciEventType_value)
-	proto.RegisterType((*AlarmAttributeData)(nil), "omci.AlarmAttributeData")
-	proto.RegisterType((*AlarmInstanceData)(nil), "omci.AlarmInstanceData")
-	proto.RegisterType((*AlarmClassData)(nil), "omci.AlarmClassData")
-	proto.RegisterType((*AlarmManagedEntity)(nil), "omci.AlarmManagedEntity")
-	proto.RegisterType((*AlarmMessageType)(nil), "omci.AlarmMessageType")
-	proto.RegisterType((*AlarmDeviceData)(nil), "omci.AlarmDeviceData")
-	proto.RegisterType((*AlarmOpenOmciEventType)(nil), "omci.AlarmOpenOmciEventType")
-	proto.RegisterType((*AlarmOpenOmciEvent)(nil), "omci.AlarmOpenOmciEvent")
-}
-
-func init() { proto.RegisterFile("voltha_protos/omci_alarm_db.proto", fileDescriptor_8d41f1e38aadb08d) }
-
-var fileDescriptor_8d41f1e38aadb08d = []byte{
-	// 606 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xc1, 0x6e, 0xd3, 0x40,
-	0x10, 0x25, 0x6d, 0xda, 0xa6, 0x93, 0xa4, 0x4d, 0x97, 0xaa, 0x2c, 0x95, 0x2a, 0x15, 0x0b, 0x50,
-	0x0f, 0xe0, 0x48, 0x85, 0x1b, 0x48, 0x55, 0xd3, 0x46, 0x22, 0x07, 0x54, 0xe1, 0xf6, 0xc4, 0xc5,
-	0xda, 0xd8, 0x83, 0xbb, 0x92, 0x77, 0x1d, 0xbc, 0x1b, 0x4b, 0x39, 0x70, 0xe1, 0xab, 0xf8, 0x89,
-	0xfe, 0x04, 0x27, 0xbe, 0xa0, 0x67, 0xe4, 0xb1, 0x9d, 0x38, 0xb2, 0x84, 0xb8, 0xed, 0x7b, 0x33,
-	0xf3, 0x66, 0x76, 0xde, 0x6a, 0xe1, 0x45, 0x96, 0xc4, 0xf6, 0x5e, 0xf8, 0xb3, 0x34, 0xb1, 0x89,
-	0x19, 0x26, 0x2a, 0x90, 0xbe, 0x88, 0x45, 0xaa, 0xfc, 0x70, 0xea, 0x12, 0xc9, 0xda, 0x39, 0x79,
-	0xcc, 0xd7, 0x13, 0x15, 0x5a, 0x51, 0xc4, 0x9d, 0x31, 0xb0, 0xcb, 0xbc, 0xe2, 0xd2, 0xda, 0x54,
-	0x4e, 0xe7, 0x16, 0xaf, 0x85, 0x15, 0xec, 0x39, 0xb4, 0xb5, 0x50, 0xc8, 0x5b, 0xa7, 0xad, 0xb3,
-	0xdd, 0xd1, 0xd6, 0x9f, 0xc7, 0x87, 0x93, 0x96, 0x47, 0x14, 0x3b, 0x84, 0xad, 0x4c, 0xc4, 0x73,
-	0xe4, 0x1b, 0x79, 0xcc, 0x2b, 0x80, 0xf3, 0xab, 0x05, 0x07, 0xa4, 0x33, 0xd1, 0xc6, 0x0a, 0x1d,
-	0x14, 0x32, 0xaf, 0xa1, 0x2b, 0x4b, 0xec, 0xcb, 0x90, 0xd4, 0xfa, 0x95, 0x1a, 0x54, 0x91, 0x49,
-	0xc8, 0x38, 0xec, 0x04, 0x29, 0x0a, 0x8b, 0x61, 0xa9, 0x5a, 0x41, 0x76, 0x0c, 0x1d, 0x95, 0x84,
-	0xf2, 0x9b, 0xc4, 0x90, 0x6f, 0x52, 0x68, 0x89, 0xd9, 0x18, 0x40, 0x54, 0x53, 0x1b, 0xde, 0x3e,
-	0xdd, 0x3c, 0xeb, 0x9e, 0x73, 0x37, 0xbf, 0xaf, 0xdb, 0xbc, 0xd2, 0xa8, 0xfb, 0xfb, 0xf1, 0xe1,
-	0x64, 0xbb, 0xb8, 0x97, 0x57, 0x2b, 0x74, 0x7e, 0xc0, 0x1e, 0xa5, 0x5f, 0xc5, 0xc2, 0x18, 0x1a,
-	0xfb, 0x14, 0x3a, 0x41, 0x0e, 0x1a, 0x33, 0xef, 0x10, 0x3d, 0x09, 0xd9, 0x04, 0x76, 0xab, 0xf1,
-	0x0d, 0xdf, 0xa0, 0xce, 0xcf, 0x6a, 0x9d, 0xeb, 0x4b, 0x18, 0xb1, 0xbc, 0x71, 0x7f, 0x6d, 0x13,
-	0xde, 0xaa, 0xda, 0xf9, 0x52, 0x1a, 0xf0, 0x59, 0x68, 0x11, 0x61, 0x38, 0xd6, 0x56, 0xda, 0xc5,
-	0x7f, 0x8c, 0x50, 0x59, 0xb4, 0xd1, 0xb0, 0xc8, 0xf9, 0x08, 0x83, 0x42, 0x12, 0x8d, 0x11, 0x11,
-	0xde, 0x2d, 0x66, 0xc8, 0xce, 0xa0, 0xa7, 0x0a, 0xe8, 0xdb, 0xc5, 0x0c, 0xd7, 0x45, 0xbb, 0x6a,
-	0x95, 0xe9, 0xfc, 0xdc, 0x84, 0x7d, 0x2a, 0xbf, 0xc6, 0x4c, 0x96, 0x46, 0x3a, 0xb0, 0x1b, 0x12,
-	0xaa, 0xe6, 0x59, 0x76, 0xec, 0x14, 0xfc, 0x3f, 0x4d, 0x74, 0xe1, 0x69, 0x2c, 0x8c, 0x2d, 0x9f,
-	0xa6, 0xc1, 0xef, 0x73, 0xd4, 0x01, 0x92, 0x9f, 0x7d, 0xef, 0x20, 0x0f, 0x51, 0xbf, 0xdb, 0x32,
-	0xc0, 0x5e, 0xc2, 0x1e, 0xe5, 0x9b, 0x85, 0x0e, 0x7c, 0x2b, 0x15, 0xf2, 0x36, 0x09, 0xf6, 0x72,
-	0xf6, 0x76, 0xa1, 0x83, 0x3b, 0xa9, 0x30, 0xef, 0x97, 0x61, 0x6a, 0x64, 0xa2, 0xf9, 0x16, 0x29,
-	0x55, 0x90, 0x5d, 0x40, 0xb1, 0x25, 0x34, 0x7c, 0x9b, 0xbc, 0x39, 0xac, 0x79, 0xb3, 0xb4, 0x79,
-	0xb4, 0x9f, 0x1b, 0x03, 0xab, 0x45, 0x7b, 0x55, 0x15, 0xbb, 0x82, 0x81, 0x2a, 0xec, 0xf0, 0x31,
-	0xf7, 0x43, 0xa2, 0xe1, 0x3b, 0x8d, 0xf7, 0xb5, 0xe6, 0x98, 0xb7, 0xaf, 0x6a, 0x50, 0xa2, 0x61,
-	0x1f, 0xa0, 0x5f, 0xdf, 0xb8, 0xe1, 0x1d, 0x52, 0x38, 0xaa, 0x2b, 0xac, 0xd6, 0xee, 0xf5, 0x6a,
-	0x1e, 0x18, 0xe7, 0x02, 0x8e, 0x28, 0xe3, 0x66, 0x86, 0xfa, 0x46, 0x05, 0x72, 0x9c, 0xa1, 0xb6,
-	0x64, 0xcf, 0x2b, 0x38, 0x68, 0x90, 0x6c, 0x00, 0x3d, 0x63, 0x85, 0x45, 0x3f, 0xb8, 0x17, 0x3a,
-	0xc2, 0xc1, 0x13, 0x27, 0x2e, 0x9f, 0xd5, 0x5a, 0x2e, 0x1b, 0x41, 0x7b, 0xe9, 0xfe, 0xde, 0xb9,
-	0x5b, 0x1b, 0xa5, 0xa1, 0xe9, 0x36, 0x18, 0x8f, 0x6a, 0x19, 0x83, 0x76, 0x28, 0xac, 0x28, 0x4d,
-	0xa6, 0xf3, 0xe8, 0x13, 0xf0, 0x24, 0x8d, 0xdc, 0x64, 0x86, 0x3a, 0x48, 0xd2, 0xd0, 0x2d, 0xbe,
-	0x1b, 0x92, 0xff, 0xfa, 0x26, 0x92, 0xf6, 0x7e, 0x3e, 0x75, 0x83, 0x44, 0x0d, 0xab, 0x84, 0x61,
-	0x91, 0xf0, 0xb6, 0xfc, 0x8f, 0xb2, 0xf7, 0xc3, 0x28, 0xa1, 0xef, 0x6b, 0xba, 0x4d, 0xd4, 0xbb,
-	0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x32, 0x98, 0x5c, 0x3a, 0xdb, 0x04, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_mib_db.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_mib_db.pb.go
deleted file mode 100644
index d644bf8..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/omci/omci_mib_db.pb.go
+++ /dev/null
@@ -1,516 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/omci_mib_db.proto
-
-package omci
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	_ "github.com/opencord/voltha-protos/v4/go/common"
-	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 OpenOmciEventType_Types int32
-
-const (
-	OpenOmciEventType_state_change OpenOmciEventType_Types = 0
-)
-
-var OpenOmciEventType_Types_name = map[int32]string{
-	0: "state_change",
-}
-
-var OpenOmciEventType_Types_value = map[string]int32{
-	"state_change": 0,
-}
-
-func (x OpenOmciEventType_Types) String() string {
-	return proto.EnumName(OpenOmciEventType_Types_name, int32(x))
-}
-
-func (OpenOmciEventType_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{6, 0}
-}
-
-type MibAttributeData struct {
-	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *MibAttributeData) Reset()         { *m = MibAttributeData{} }
-func (m *MibAttributeData) String() string { return proto.CompactTextString(m) }
-func (*MibAttributeData) ProtoMessage()    {}
-func (*MibAttributeData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{0}
-}
-
-func (m *MibAttributeData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MibAttributeData.Unmarshal(m, b)
-}
-func (m *MibAttributeData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MibAttributeData.Marshal(b, m, deterministic)
-}
-func (m *MibAttributeData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MibAttributeData.Merge(m, src)
-}
-func (m *MibAttributeData) XXX_Size() int {
-	return xxx_messageInfo_MibAttributeData.Size(m)
-}
-func (m *MibAttributeData) XXX_DiscardUnknown() {
-	xxx_messageInfo_MibAttributeData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MibAttributeData proto.InternalMessageInfo
-
-func (m *MibAttributeData) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *MibAttributeData) GetValue() string {
-	if m != nil {
-		return m.Value
-	}
-	return ""
-}
-
-type MibInstanceData struct {
-	InstanceId           uint32              `protobuf:"varint,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
-	Created              string              `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
-	Modified             string              `protobuf:"bytes,3,opt,name=modified,proto3" json:"modified,omitempty"`
-	Attributes           []*MibAttributeData `protobuf:"bytes,4,rep,name=attributes,proto3" json:"attributes,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *MibInstanceData) Reset()         { *m = MibInstanceData{} }
-func (m *MibInstanceData) String() string { return proto.CompactTextString(m) }
-func (*MibInstanceData) ProtoMessage()    {}
-func (*MibInstanceData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{1}
-}
-
-func (m *MibInstanceData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MibInstanceData.Unmarshal(m, b)
-}
-func (m *MibInstanceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MibInstanceData.Marshal(b, m, deterministic)
-}
-func (m *MibInstanceData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MibInstanceData.Merge(m, src)
-}
-func (m *MibInstanceData) XXX_Size() int {
-	return xxx_messageInfo_MibInstanceData.Size(m)
-}
-func (m *MibInstanceData) XXX_DiscardUnknown() {
-	xxx_messageInfo_MibInstanceData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MibInstanceData proto.InternalMessageInfo
-
-func (m *MibInstanceData) GetInstanceId() uint32 {
-	if m != nil {
-		return m.InstanceId
-	}
-	return 0
-}
-
-func (m *MibInstanceData) GetCreated() string {
-	if m != nil {
-		return m.Created
-	}
-	return ""
-}
-
-func (m *MibInstanceData) GetModified() string {
-	if m != nil {
-		return m.Modified
-	}
-	return ""
-}
-
-func (m *MibInstanceData) GetAttributes() []*MibAttributeData {
-	if m != nil {
-		return m.Attributes
-	}
-	return nil
-}
-
-type MibClassData struct {
-	ClassId              uint32             `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
-	Instances            []*MibInstanceData `protobuf:"bytes,2,rep,name=instances,proto3" json:"instances,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *MibClassData) Reset()         { *m = MibClassData{} }
-func (m *MibClassData) String() string { return proto.CompactTextString(m) }
-func (*MibClassData) ProtoMessage()    {}
-func (*MibClassData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{2}
-}
-
-func (m *MibClassData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MibClassData.Unmarshal(m, b)
-}
-func (m *MibClassData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MibClassData.Marshal(b, m, deterministic)
-}
-func (m *MibClassData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MibClassData.Merge(m, src)
-}
-func (m *MibClassData) XXX_Size() int {
-	return xxx_messageInfo_MibClassData.Size(m)
-}
-func (m *MibClassData) XXX_DiscardUnknown() {
-	xxx_messageInfo_MibClassData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MibClassData proto.InternalMessageInfo
-
-func (m *MibClassData) GetClassId() uint32 {
-	if m != nil {
-		return m.ClassId
-	}
-	return 0
-}
-
-func (m *MibClassData) GetInstances() []*MibInstanceData {
-	if m != nil {
-		return m.Instances
-	}
-	return nil
-}
-
-type ManagedEntity struct {
-	ClassId              uint32   `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
-	Name                 string   `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ManagedEntity) Reset()         { *m = ManagedEntity{} }
-func (m *ManagedEntity) String() string { return proto.CompactTextString(m) }
-func (*ManagedEntity) ProtoMessage()    {}
-func (*ManagedEntity) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{3}
-}
-
-func (m *ManagedEntity) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ManagedEntity.Unmarshal(m, b)
-}
-func (m *ManagedEntity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ManagedEntity.Marshal(b, m, deterministic)
-}
-func (m *ManagedEntity) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ManagedEntity.Merge(m, src)
-}
-func (m *ManagedEntity) XXX_Size() int {
-	return xxx_messageInfo_ManagedEntity.Size(m)
-}
-func (m *ManagedEntity) XXX_DiscardUnknown() {
-	xxx_messageInfo_ManagedEntity.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ManagedEntity proto.InternalMessageInfo
-
-func (m *ManagedEntity) GetClassId() uint32 {
-	if m != nil {
-		return m.ClassId
-	}
-	return 0
-}
-
-func (m *ManagedEntity) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type MessageType struct {
-	MessageType          uint32   `protobuf:"varint,1,opt,name=message_type,json=messageType,proto3" json:"message_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *MessageType) Reset()         { *m = MessageType{} }
-func (m *MessageType) String() string { return proto.CompactTextString(m) }
-func (*MessageType) ProtoMessage()    {}
-func (*MessageType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{4}
-}
-
-func (m *MessageType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MessageType.Unmarshal(m, b)
-}
-func (m *MessageType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MessageType.Marshal(b, m, deterministic)
-}
-func (m *MessageType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MessageType.Merge(m, src)
-}
-func (m *MessageType) XXX_Size() int {
-	return xxx_messageInfo_MessageType.Size(m)
-}
-func (m *MessageType) XXX_DiscardUnknown() {
-	xxx_messageInfo_MessageType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MessageType proto.InternalMessageInfo
-
-func (m *MessageType) GetMessageType() uint32 {
-	if m != nil {
-		return m.MessageType
-	}
-	return 0
-}
-
-type MibDeviceData struct {
-	DeviceId             string           `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	Created              string           `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
-	LastSyncTime         string           `protobuf:"bytes,3,opt,name=last_sync_time,json=lastSyncTime,proto3" json:"last_sync_time,omitempty"`
-	MibDataSync          uint32           `protobuf:"varint,4,opt,name=mib_data_sync,json=mibDataSync,proto3" json:"mib_data_sync,omitempty"`
-	Version              uint32           `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"`
-	Classes              []*MibClassData  `protobuf:"bytes,6,rep,name=classes,proto3" json:"classes,omitempty"`
-	ManagedEntities      []*ManagedEntity `protobuf:"bytes,7,rep,name=managed_entities,json=managedEntities,proto3" json:"managed_entities,omitempty"`
-	MessageTypes         []*MessageType   `protobuf:"bytes,8,rep,name=message_types,json=messageTypes,proto3" json:"message_types,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *MibDeviceData) Reset()         { *m = MibDeviceData{} }
-func (m *MibDeviceData) String() string { return proto.CompactTextString(m) }
-func (*MibDeviceData) ProtoMessage()    {}
-func (*MibDeviceData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{5}
-}
-
-func (m *MibDeviceData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MibDeviceData.Unmarshal(m, b)
-}
-func (m *MibDeviceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MibDeviceData.Marshal(b, m, deterministic)
-}
-func (m *MibDeviceData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MibDeviceData.Merge(m, src)
-}
-func (m *MibDeviceData) XXX_Size() int {
-	return xxx_messageInfo_MibDeviceData.Size(m)
-}
-func (m *MibDeviceData) XXX_DiscardUnknown() {
-	xxx_messageInfo_MibDeviceData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MibDeviceData proto.InternalMessageInfo
-
-func (m *MibDeviceData) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *MibDeviceData) GetCreated() string {
-	if m != nil {
-		return m.Created
-	}
-	return ""
-}
-
-func (m *MibDeviceData) GetLastSyncTime() string {
-	if m != nil {
-		return m.LastSyncTime
-	}
-	return ""
-}
-
-func (m *MibDeviceData) GetMibDataSync() uint32 {
-	if m != nil {
-		return m.MibDataSync
-	}
-	return 0
-}
-
-func (m *MibDeviceData) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *MibDeviceData) GetClasses() []*MibClassData {
-	if m != nil {
-		return m.Classes
-	}
-	return nil
-}
-
-func (m *MibDeviceData) GetManagedEntities() []*ManagedEntity {
-	if m != nil {
-		return m.ManagedEntities
-	}
-	return nil
-}
-
-func (m *MibDeviceData) GetMessageTypes() []*MessageType {
-	if m != nil {
-		return m.MessageTypes
-	}
-	return nil
-}
-
-type OpenOmciEventType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OpenOmciEventType) Reset()         { *m = OpenOmciEventType{} }
-func (m *OpenOmciEventType) String() string { return proto.CompactTextString(m) }
-func (*OpenOmciEventType) ProtoMessage()    {}
-func (*OpenOmciEventType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{6}
-}
-
-func (m *OpenOmciEventType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OpenOmciEventType.Unmarshal(m, b)
-}
-func (m *OpenOmciEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OpenOmciEventType.Marshal(b, m, deterministic)
-}
-func (m *OpenOmciEventType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OpenOmciEventType.Merge(m, src)
-}
-func (m *OpenOmciEventType) XXX_Size() int {
-	return xxx_messageInfo_OpenOmciEventType.Size(m)
-}
-func (m *OpenOmciEventType) XXX_DiscardUnknown() {
-	xxx_messageInfo_OpenOmciEventType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OpenOmciEventType proto.InternalMessageInfo
-
-type OpenOmciEvent struct {
-	Type                 OpenOmciEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=omci.OpenOmciEventType_Types" json:"type,omitempty"`
-	Data                 string                  `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
-}
-
-func (m *OpenOmciEvent) Reset()         { *m = OpenOmciEvent{} }
-func (m *OpenOmciEvent) String() string { return proto.CompactTextString(m) }
-func (*OpenOmciEvent) ProtoMessage()    {}
-func (*OpenOmciEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_4fa402a2df36dcc1, []int{7}
-}
-
-func (m *OpenOmciEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OpenOmciEvent.Unmarshal(m, b)
-}
-func (m *OpenOmciEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OpenOmciEvent.Marshal(b, m, deterministic)
-}
-func (m *OpenOmciEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OpenOmciEvent.Merge(m, src)
-}
-func (m *OpenOmciEvent) XXX_Size() int {
-	return xxx_messageInfo_OpenOmciEvent.Size(m)
-}
-func (m *OpenOmciEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_OpenOmciEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OpenOmciEvent proto.InternalMessageInfo
-
-func (m *OpenOmciEvent) GetType() OpenOmciEventType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return OpenOmciEventType_state_change
-}
-
-func (m *OpenOmciEvent) GetData() string {
-	if m != nil {
-		return m.Data
-	}
-	return ""
-}
-
-func init() {
-	proto.RegisterEnum("omci.OpenOmciEventType_Types", OpenOmciEventType_Types_name, OpenOmciEventType_Types_value)
-	proto.RegisterType((*MibAttributeData)(nil), "omci.MibAttributeData")
-	proto.RegisterType((*MibInstanceData)(nil), "omci.MibInstanceData")
-	proto.RegisterType((*MibClassData)(nil), "omci.MibClassData")
-	proto.RegisterType((*ManagedEntity)(nil), "omci.ManagedEntity")
-	proto.RegisterType((*MessageType)(nil), "omci.MessageType")
-	proto.RegisterType((*MibDeviceData)(nil), "omci.MibDeviceData")
-	proto.RegisterType((*OpenOmciEventType)(nil), "omci.OpenOmciEventType")
-	proto.RegisterType((*OpenOmciEvent)(nil), "omci.OpenOmciEvent")
-}
-
-func init() { proto.RegisterFile("voltha_protos/omci_mib_db.proto", fileDescriptor_4fa402a2df36dcc1) }
-
-var fileDescriptor_4fa402a2df36dcc1 = []byte{
-	// 598 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdf, 0x6a, 0xd4, 0x4e,
-	0x14, 0xfe, 0xed, 0x76, 0xb7, 0xdd, 0x9e, 0xdd, 0xb4, 0xdb, 0xf9, 0xa9, 0x4c, 0x0b, 0xc5, 0x12,
-	0x44, 0x7a, 0xa1, 0x59, 0xac, 0xa2, 0x57, 0x0a, 0x6e, 0x5b, 0xb4, 0x60, 0x28, 0xc4, 0xe2, 0x85,
-	0x37, 0x61, 0x92, 0x1c, 0xd3, 0x81, 0x9d, 0xc9, 0x92, 0x99, 0x06, 0xf2, 0x5e, 0xbe, 0x46, 0x5f,
-	0xc2, 0xab, 0x3e, 0x41, 0xaf, 0x65, 0x26, 0x7f, 0x36, 0x6b, 0x41, 0xbc, 0xcb, 0x77, 0xfe, 0x7c,
-	0xe7, 0x9c, 0xef, 0x0b, 0x03, 0x4f, 0x8b, 0x6c, 0xa1, 0xaf, 0x59, 0xb8, 0xcc, 0x33, 0x9d, 0xa9,
-	0x59, 0x26, 0x62, 0x1e, 0x0a, 0x1e, 0x85, 0x49, 0xe4, 0xd9, 0x10, 0x19, 0x98, 0xd0, 0x01, 0x5d,
-	0x2f, 0x13, 0xa8, 0x59, 0x95, 0x77, 0x4f, 0x61, 0xea, 0xf3, 0xe8, 0xa3, 0xd6, 0x39, 0x8f, 0x6e,
-	0x34, 0x9e, 0x31, 0xcd, 0xc8, 0x3e, 0x0c, 0x24, 0x13, 0x48, 0x7b, 0x47, 0xbd, 0xe3, 0xed, 0xf9,
-	0xf0, 0xee, 0xfe, 0xf6, 0xb0, 0x17, 0xd8, 0x10, 0x79, 0x04, 0xc3, 0x82, 0x2d, 0x6e, 0x90, 0xf6,
-	0x4d, 0x2e, 0xa8, 0x80, 0xfb, 0xb3, 0x07, 0xbb, 0x3e, 0x8f, 0x2e, 0xa4, 0xd2, 0x4c, 0xc6, 0x15,
-	0xc9, 0x73, 0x18, 0xf3, 0x1a, 0x87, 0x3c, 0xb1, 0x5c, 0x4e, 0xc3, 0x05, 0x4d, 0xe6, 0x22, 0x21,
-	0x14, 0xb6, 0xe2, 0x1c, 0x99, 0xc6, 0xa4, 0xe6, 0x6c, 0x20, 0x39, 0x80, 0x91, 0xc8, 0x12, 0xfe,
-	0x83, 0x63, 0x42, 0x37, 0x6c, 0xaa, 0xc5, 0xe4, 0x14, 0x80, 0x35, 0x3b, 0x2b, 0x3a, 0x38, 0xda,
-	0x38, 0x1e, 0x9f, 0x3c, 0xf1, 0xcc, 0xad, 0xde, 0x9f, 0xe7, 0xcc, 0xc7, 0xbf, 0xee, 0x6f, 0x0f,
-	0x37, 0xab, 0x9b, 0x82, 0x4e, 0x9b, 0x5b, 0xc2, 0xc4, 0xe7, 0xd1, 0xe9, 0x82, 0x29, 0x65, 0x57,
-	0x3e, 0x82, 0x51, 0x6c, 0xc0, 0x83, 0x7d, 0xb7, 0x6c, 0xf8, 0x22, 0x21, 0x9f, 0x60, 0xbb, 0x59,
-	0x5d, 0xd1, 0xbe, 0x9d, 0xfa, 0xb8, 0x9d, 0xda, 0x3d, 0x7f, 0x4e, 0xcc, 0x50, 0x67, 0x4d, 0x83,
-	0x60, 0xd5, 0xeb, 0x7e, 0x01, 0xc7, 0x67, 0x92, 0xa5, 0x98, 0x9c, 0x4b, 0xcd, 0x75, 0xf9, 0x0f,
-	0xb3, 0x1b, 0x57, 0xfa, 0x0f, 0x5c, 0x71, 0xdf, 0xc1, 0xd8, 0x47, 0xa5, 0x58, 0x8a, 0x57, 0xe5,
-	0x12, 0xc9, 0x31, 0x4c, 0x44, 0x05, 0x43, 0x5d, 0x2e, 0x71, 0x9d, 0x6f, 0x2c, 0x56, 0x95, 0xee,
-	0x5d, 0x1f, 0x1c, 0x9f, 0x47, 0x67, 0x58, 0xf0, 0xda, 0x36, 0x17, 0xb6, 0x13, 0x8b, 0x9a, 0x45,
-	0xda, 0x51, 0xa3, 0x2a, 0xfe, 0x57, 0xcb, 0x9e, 0xc1, 0xce, 0x82, 0x29, 0x1d, 0xaa, 0x52, 0xc6,
-	0xa1, 0xe6, 0x02, 0x6b, 0xe3, 0x26, 0x26, 0xfa, 0xb5, 0x94, 0xf1, 0x15, 0x17, 0x48, 0x5c, 0x70,
-	0xec, 0x3f, 0xca, 0x34, 0xb3, 0x95, 0x74, 0x60, 0x16, 0x0c, 0xc6, 0x82, 0x47, 0x66, 0x07, 0x53,
-	0x67, 0x66, 0x14, 0x98, 0x2b, 0x9e, 0x49, 0x3a, 0xb4, 0xd9, 0x06, 0x92, 0xf7, 0x50, 0x49, 0x82,
-	0x8a, 0x6e, 0x5a, 0x07, 0x48, 0xeb, 0x40, 0x6b, 0xe5, 0x7c, 0xd7, 0xc8, 0x0f, 0x2b, 0x4d, 0x83,
-	0xa6, 0x87, 0x7c, 0x80, 0xa9, 0xa8, 0x94, 0x0f, 0xd1, 0x48, 0xcf, 0x51, 0xd1, 0x2d, 0xcb, 0xf3,
-	0x7f, 0xcd, 0xd3, 0xf5, 0x25, 0xd8, 0x15, 0x1d, 0xc8, 0x51, 0x91, 0xb7, 0xe0, 0x74, 0xc5, 0x55,
-	0x74, 0x64, 0x9b, 0xf7, 0xea, 0xe6, 0x95, 0xb8, 0xc1, 0xa4, 0xa3, 0xb4, 0x72, 0x3d, 0xd8, 0xbb,
-	0x5c, 0xa2, 0xbc, 0x14, 0x31, 0x3f, 0x2f, 0x50, 0x6a, 0xab, 0xff, 0x3e, 0x0c, 0x6d, 0x96, 0x4c,
-	0x61, 0xa2, 0x34, 0xd3, 0x18, 0xc6, 0xd7, 0x4c, 0xa6, 0x38, 0xfd, 0xcf, 0xfd, 0x06, 0xce, 0x5a,
-	0x3d, 0x79, 0x05, 0x83, 0xd6, 0xcd, 0x9d, 0x93, 0xc3, 0x6a, 0xde, 0x03, 0x4a, 0xcf, 0xf2, 0x05,
-	0xb6, 0x94, 0x10, 0x18, 0x18, 0x91, 0x6b, 0x97, 0xec, 0xf7, 0xfc, 0x33, 0xd0, 0x2c, 0x4f, 0xbd,
-	0x6c, 0x89, 0x32, 0xce, 0xf2, 0xc4, 0xab, 0x5e, 0x06, 0xcb, 0xf6, 0xfd, 0x45, 0xca, 0xf5, 0xf5,
-	0x4d, 0xe4, 0xc5, 0x99, 0x98, 0x35, 0x05, 0xb3, 0xaa, 0xe0, 0x65, 0xfd, 0x74, 0x14, 0x6f, 0x66,
-	0x69, 0x66, 0xdf, 0x99, 0x68, 0xd3, 0x86, 0x5e, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x6d, 0xd7,
-	0xc0, 0x7b, 0x84, 0x04, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/openflow_13/openflow_13.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/openflow_13/openflow_13.pb.go
deleted file mode 100644
index a1889d1..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/openflow_13/openflow_13.pb.go
+++ /dev/null
@@ -1,9863 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/openflow_13.proto
-
-package openflow_13
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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
-
-// Port numbering. Ports are numbered starting from 1.
-type OfpPortNo int32
-
-const (
-	OfpPortNo_OFPP_INVALID OfpPortNo = 0
-	// Maximum number of physical and logical switch ports.
-	OfpPortNo_OFPP_MAX OfpPortNo = 2147483392
-	// Reserved OpenFlow Port (fake output "ports").
-	OfpPortNo_OFPP_IN_PORT    OfpPortNo = 2147483640
-	OfpPortNo_OFPP_TABLE      OfpPortNo = 2147483641
-	OfpPortNo_OFPP_NORMAL     OfpPortNo = 2147483642
-	OfpPortNo_OFPP_FLOOD      OfpPortNo = 2147483643
-	OfpPortNo_OFPP_ALL        OfpPortNo = 2147483644
-	OfpPortNo_OFPP_CONTROLLER OfpPortNo = 2147483645
-	OfpPortNo_OFPP_LOCAL      OfpPortNo = 2147483646
-	OfpPortNo_OFPP_ANY        OfpPortNo = 2147483647
-)
-
-var OfpPortNo_name = map[int32]string{
-	0:          "OFPP_INVALID",
-	2147483392: "OFPP_MAX",
-	2147483640: "OFPP_IN_PORT",
-	2147483641: "OFPP_TABLE",
-	2147483642: "OFPP_NORMAL",
-	2147483643: "OFPP_FLOOD",
-	2147483644: "OFPP_ALL",
-	2147483645: "OFPP_CONTROLLER",
-	2147483646: "OFPP_LOCAL",
-	2147483647: "OFPP_ANY",
-}
-
-var OfpPortNo_value = map[string]int32{
-	"OFPP_INVALID":    0,
-	"OFPP_MAX":        2147483392,
-	"OFPP_IN_PORT":    2147483640,
-	"OFPP_TABLE":      2147483641,
-	"OFPP_NORMAL":     2147483642,
-	"OFPP_FLOOD":      2147483643,
-	"OFPP_ALL":        2147483644,
-	"OFPP_CONTROLLER": 2147483645,
-	"OFPP_LOCAL":      2147483646,
-	"OFPP_ANY":        2147483647,
-}
-
-func (x OfpPortNo) String() string {
-	return proto.EnumName(OfpPortNo_name, int32(x))
-}
-
-func (OfpPortNo) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{0}
-}
-
-type OfpType int32
-
-const (
-	// Immutable messages.
-	OfpType_OFPT_HELLO        OfpType = 0
-	OfpType_OFPT_ERROR        OfpType = 1
-	OfpType_OFPT_ECHO_REQUEST OfpType = 2
-	OfpType_OFPT_ECHO_REPLY   OfpType = 3
-	OfpType_OFPT_EXPERIMENTER OfpType = 4
-	// Switch configuration messages.
-	OfpType_OFPT_FEATURES_REQUEST   OfpType = 5
-	OfpType_OFPT_FEATURES_REPLY     OfpType = 6
-	OfpType_OFPT_GET_CONFIG_REQUEST OfpType = 7
-	OfpType_OFPT_GET_CONFIG_REPLY   OfpType = 8
-	OfpType_OFPT_SET_CONFIG         OfpType = 9
-	// Asynchronous messages.
-	OfpType_OFPT_PACKET_IN    OfpType = 10
-	OfpType_OFPT_FLOW_REMOVED OfpType = 11
-	OfpType_OFPT_PORT_STATUS  OfpType = 12
-	// Controller command messages.
-	OfpType_OFPT_PACKET_OUT OfpType = 13
-	OfpType_OFPT_FLOW_MOD   OfpType = 14
-	OfpType_OFPT_GROUP_MOD  OfpType = 15
-	OfpType_OFPT_PORT_MOD   OfpType = 16
-	OfpType_OFPT_TABLE_MOD  OfpType = 17
-	// Multipart messages.
-	OfpType_OFPT_MULTIPART_REQUEST OfpType = 18
-	OfpType_OFPT_MULTIPART_REPLY   OfpType = 19
-	// Barrier messages.
-	OfpType_OFPT_BARRIER_REQUEST OfpType = 20
-	OfpType_OFPT_BARRIER_REPLY   OfpType = 21
-	// Queue Configuration messages.
-	OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST OfpType = 22
-	OfpType_OFPT_QUEUE_GET_CONFIG_REPLY   OfpType = 23
-	// Controller role change request messages.
-	OfpType_OFPT_ROLE_REQUEST OfpType = 24
-	OfpType_OFPT_ROLE_REPLY   OfpType = 25
-	// Asynchronous message configuration.
-	OfpType_OFPT_GET_ASYNC_REQUEST OfpType = 26
-	OfpType_OFPT_GET_ASYNC_REPLY   OfpType = 27
-	OfpType_OFPT_SET_ASYNC         OfpType = 28
-	// Meters and rate limiters configuration messages.
-	OfpType_OFPT_METER_MOD OfpType = 29
-)
-
-var OfpType_name = map[int32]string{
-	0:  "OFPT_HELLO",
-	1:  "OFPT_ERROR",
-	2:  "OFPT_ECHO_REQUEST",
-	3:  "OFPT_ECHO_REPLY",
-	4:  "OFPT_EXPERIMENTER",
-	5:  "OFPT_FEATURES_REQUEST",
-	6:  "OFPT_FEATURES_REPLY",
-	7:  "OFPT_GET_CONFIG_REQUEST",
-	8:  "OFPT_GET_CONFIG_REPLY",
-	9:  "OFPT_SET_CONFIG",
-	10: "OFPT_PACKET_IN",
-	11: "OFPT_FLOW_REMOVED",
-	12: "OFPT_PORT_STATUS",
-	13: "OFPT_PACKET_OUT",
-	14: "OFPT_FLOW_MOD",
-	15: "OFPT_GROUP_MOD",
-	16: "OFPT_PORT_MOD",
-	17: "OFPT_TABLE_MOD",
-	18: "OFPT_MULTIPART_REQUEST",
-	19: "OFPT_MULTIPART_REPLY",
-	20: "OFPT_BARRIER_REQUEST",
-	21: "OFPT_BARRIER_REPLY",
-	22: "OFPT_QUEUE_GET_CONFIG_REQUEST",
-	23: "OFPT_QUEUE_GET_CONFIG_REPLY",
-	24: "OFPT_ROLE_REQUEST",
-	25: "OFPT_ROLE_REPLY",
-	26: "OFPT_GET_ASYNC_REQUEST",
-	27: "OFPT_GET_ASYNC_REPLY",
-	28: "OFPT_SET_ASYNC",
-	29: "OFPT_METER_MOD",
-}
-
-var OfpType_value = map[string]int32{
-	"OFPT_HELLO":                    0,
-	"OFPT_ERROR":                    1,
-	"OFPT_ECHO_REQUEST":             2,
-	"OFPT_ECHO_REPLY":               3,
-	"OFPT_EXPERIMENTER":             4,
-	"OFPT_FEATURES_REQUEST":         5,
-	"OFPT_FEATURES_REPLY":           6,
-	"OFPT_GET_CONFIG_REQUEST":       7,
-	"OFPT_GET_CONFIG_REPLY":         8,
-	"OFPT_SET_CONFIG":               9,
-	"OFPT_PACKET_IN":                10,
-	"OFPT_FLOW_REMOVED":             11,
-	"OFPT_PORT_STATUS":              12,
-	"OFPT_PACKET_OUT":               13,
-	"OFPT_FLOW_MOD":                 14,
-	"OFPT_GROUP_MOD":                15,
-	"OFPT_PORT_MOD":                 16,
-	"OFPT_TABLE_MOD":                17,
-	"OFPT_MULTIPART_REQUEST":        18,
-	"OFPT_MULTIPART_REPLY":          19,
-	"OFPT_BARRIER_REQUEST":          20,
-	"OFPT_BARRIER_REPLY":            21,
-	"OFPT_QUEUE_GET_CONFIG_REQUEST": 22,
-	"OFPT_QUEUE_GET_CONFIG_REPLY":   23,
-	"OFPT_ROLE_REQUEST":             24,
-	"OFPT_ROLE_REPLY":               25,
-	"OFPT_GET_ASYNC_REQUEST":        26,
-	"OFPT_GET_ASYNC_REPLY":          27,
-	"OFPT_SET_ASYNC":                28,
-	"OFPT_METER_MOD":                29,
-}
-
-func (x OfpType) String() string {
-	return proto.EnumName(OfpType_name, int32(x))
-}
-
-func (OfpType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{1}
-}
-
-// Hello elements types.
-type OfpHelloElemType int32
-
-const (
-	OfpHelloElemType_OFPHET_INVALID       OfpHelloElemType = 0
-	OfpHelloElemType_OFPHET_VERSIONBITMAP OfpHelloElemType = 1
-)
-
-var OfpHelloElemType_name = map[int32]string{
-	0: "OFPHET_INVALID",
-	1: "OFPHET_VERSIONBITMAP",
-}
-
-var OfpHelloElemType_value = map[string]int32{
-	"OFPHET_INVALID":       0,
-	"OFPHET_VERSIONBITMAP": 1,
-}
-
-func (x OfpHelloElemType) String() string {
-	return proto.EnumName(OfpHelloElemType_name, int32(x))
-}
-
-func (OfpHelloElemType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{2}
-}
-
-type OfpConfigFlags int32
-
-const (
-	// Handling of IP fragments.
-	OfpConfigFlags_OFPC_FRAG_NORMAL OfpConfigFlags = 0
-	OfpConfigFlags_OFPC_FRAG_DROP   OfpConfigFlags = 1
-	OfpConfigFlags_OFPC_FRAG_REASM  OfpConfigFlags = 2
-	OfpConfigFlags_OFPC_FRAG_MASK   OfpConfigFlags = 3
-)
-
-var OfpConfigFlags_name = map[int32]string{
-	0: "OFPC_FRAG_NORMAL",
-	1: "OFPC_FRAG_DROP",
-	2: "OFPC_FRAG_REASM",
-	3: "OFPC_FRAG_MASK",
-}
-
-var OfpConfigFlags_value = map[string]int32{
-	"OFPC_FRAG_NORMAL": 0,
-	"OFPC_FRAG_DROP":   1,
-	"OFPC_FRAG_REASM":  2,
-	"OFPC_FRAG_MASK":   3,
-}
-
-func (x OfpConfigFlags) String() string {
-	return proto.EnumName(OfpConfigFlags_name, int32(x))
-}
-
-func (OfpConfigFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{3}
-}
-
-// Flags to configure the table. Reserved for future use.
-type OfpTableConfig int32
-
-const (
-	OfpTableConfig_OFPTC_INVALID         OfpTableConfig = 0
-	OfpTableConfig_OFPTC_DEPRECATED_MASK OfpTableConfig = 3
-)
-
-var OfpTableConfig_name = map[int32]string{
-	0: "OFPTC_INVALID",
-	3: "OFPTC_DEPRECATED_MASK",
-}
-
-var OfpTableConfig_value = map[string]int32{
-	"OFPTC_INVALID":         0,
-	"OFPTC_DEPRECATED_MASK": 3,
-}
-
-func (x OfpTableConfig) String() string {
-	return proto.EnumName(OfpTableConfig_name, int32(x))
-}
-
-func (OfpTableConfig) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{4}
-}
-
-// Table numbering. Tables can use any number up to OFPT_MAX.
-type OfpTable int32
-
-const (
-	OfpTable_OFPTT_INVALID OfpTable = 0
-	// Last usable table number.
-	OfpTable_OFPTT_MAX OfpTable = 254
-	// Fake tables.
-	OfpTable_OFPTT_ALL OfpTable = 255
-)
-
-var OfpTable_name = map[int32]string{
-	0:   "OFPTT_INVALID",
-	254: "OFPTT_MAX",
-	255: "OFPTT_ALL",
-}
-
-var OfpTable_value = map[string]int32{
-	"OFPTT_INVALID": 0,
-	"OFPTT_MAX":     254,
-	"OFPTT_ALL":     255,
-}
-
-func (x OfpTable) String() string {
-	return proto.EnumName(OfpTable_name, int32(x))
-}
-
-func (OfpTable) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{5}
-}
-
-// Capabilities supported by the datapath.
-type OfpCapabilities int32
-
-const (
-	OfpCapabilities_OFPC_INVALID      OfpCapabilities = 0
-	OfpCapabilities_OFPC_FLOW_STATS   OfpCapabilities = 1
-	OfpCapabilities_OFPC_TABLE_STATS  OfpCapabilities = 2
-	OfpCapabilities_OFPC_PORT_STATS   OfpCapabilities = 4
-	OfpCapabilities_OFPC_GROUP_STATS  OfpCapabilities = 8
-	OfpCapabilities_OFPC_IP_REASM     OfpCapabilities = 32
-	OfpCapabilities_OFPC_QUEUE_STATS  OfpCapabilities = 64
-	OfpCapabilities_OFPC_PORT_BLOCKED OfpCapabilities = 256
-)
-
-var OfpCapabilities_name = map[int32]string{
-	0:   "OFPC_INVALID",
-	1:   "OFPC_FLOW_STATS",
-	2:   "OFPC_TABLE_STATS",
-	4:   "OFPC_PORT_STATS",
-	8:   "OFPC_GROUP_STATS",
-	32:  "OFPC_IP_REASM",
-	64:  "OFPC_QUEUE_STATS",
-	256: "OFPC_PORT_BLOCKED",
-}
-
-var OfpCapabilities_value = map[string]int32{
-	"OFPC_INVALID":      0,
-	"OFPC_FLOW_STATS":   1,
-	"OFPC_TABLE_STATS":  2,
-	"OFPC_PORT_STATS":   4,
-	"OFPC_GROUP_STATS":  8,
-	"OFPC_IP_REASM":     32,
-	"OFPC_QUEUE_STATS":  64,
-	"OFPC_PORT_BLOCKED": 256,
-}
-
-func (x OfpCapabilities) String() string {
-	return proto.EnumName(OfpCapabilities_name, int32(x))
-}
-
-func (OfpCapabilities) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{6}
-}
-
-// Flags to indicate behavior of the physical port.  These flags are
-// used in ofp_port to describe the current configuration.  They are
-// used in the ofp_port_mod message to configure the port's behavior.
-type OfpPortConfig int32
-
-const (
-	OfpPortConfig_OFPPC_INVALID      OfpPortConfig = 0
-	OfpPortConfig_OFPPC_PORT_DOWN    OfpPortConfig = 1
-	OfpPortConfig_OFPPC_NO_RECV      OfpPortConfig = 4
-	OfpPortConfig_OFPPC_NO_FWD       OfpPortConfig = 32
-	OfpPortConfig_OFPPC_NO_PACKET_IN OfpPortConfig = 64
-)
-
-var OfpPortConfig_name = map[int32]string{
-	0:  "OFPPC_INVALID",
-	1:  "OFPPC_PORT_DOWN",
-	4:  "OFPPC_NO_RECV",
-	32: "OFPPC_NO_FWD",
-	64: "OFPPC_NO_PACKET_IN",
-}
-
-var OfpPortConfig_value = map[string]int32{
-	"OFPPC_INVALID":      0,
-	"OFPPC_PORT_DOWN":    1,
-	"OFPPC_NO_RECV":      4,
-	"OFPPC_NO_FWD":       32,
-	"OFPPC_NO_PACKET_IN": 64,
-}
-
-func (x OfpPortConfig) String() string {
-	return proto.EnumName(OfpPortConfig_name, int32(x))
-}
-
-func (OfpPortConfig) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{7}
-}
-
-// Current state of the physical port.  These are not configurable from
-// the controller.
-type OfpPortState int32
-
-const (
-	OfpPortState_OFPPS_INVALID   OfpPortState = 0
-	OfpPortState_OFPPS_LINK_DOWN OfpPortState = 1
-	OfpPortState_OFPPS_BLOCKED   OfpPortState = 2
-	OfpPortState_OFPPS_LIVE      OfpPortState = 4
-)
-
-var OfpPortState_name = map[int32]string{
-	0: "OFPPS_INVALID",
-	1: "OFPPS_LINK_DOWN",
-	2: "OFPPS_BLOCKED",
-	4: "OFPPS_LIVE",
-}
-
-var OfpPortState_value = map[string]int32{
-	"OFPPS_INVALID":   0,
-	"OFPPS_LINK_DOWN": 1,
-	"OFPPS_BLOCKED":   2,
-	"OFPPS_LIVE":      4,
-}
-
-func (x OfpPortState) String() string {
-	return proto.EnumName(OfpPortState_name, int32(x))
-}
-
-func (OfpPortState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{8}
-}
-
-// Features of ports available in a datapath.
-type OfpPortFeatures int32
-
-const (
-	OfpPortFeatures_OFPPF_INVALID    OfpPortFeatures = 0
-	OfpPortFeatures_OFPPF_10MB_HD    OfpPortFeatures = 1
-	OfpPortFeatures_OFPPF_10MB_FD    OfpPortFeatures = 2
-	OfpPortFeatures_OFPPF_100MB_HD   OfpPortFeatures = 4
-	OfpPortFeatures_OFPPF_100MB_FD   OfpPortFeatures = 8
-	OfpPortFeatures_OFPPF_1GB_HD     OfpPortFeatures = 16
-	OfpPortFeatures_OFPPF_1GB_FD     OfpPortFeatures = 32
-	OfpPortFeatures_OFPPF_10GB_FD    OfpPortFeatures = 64
-	OfpPortFeatures_OFPPF_40GB_FD    OfpPortFeatures = 128
-	OfpPortFeatures_OFPPF_100GB_FD   OfpPortFeatures = 256
-	OfpPortFeatures_OFPPF_1TB_FD     OfpPortFeatures = 512
-	OfpPortFeatures_OFPPF_OTHER      OfpPortFeatures = 1024
-	OfpPortFeatures_OFPPF_COPPER     OfpPortFeatures = 2048
-	OfpPortFeatures_OFPPF_FIBER      OfpPortFeatures = 4096
-	OfpPortFeatures_OFPPF_AUTONEG    OfpPortFeatures = 8192
-	OfpPortFeatures_OFPPF_PAUSE      OfpPortFeatures = 16384
-	OfpPortFeatures_OFPPF_PAUSE_ASYM OfpPortFeatures = 32768
-)
-
-var OfpPortFeatures_name = map[int32]string{
-	0:     "OFPPF_INVALID",
-	1:     "OFPPF_10MB_HD",
-	2:     "OFPPF_10MB_FD",
-	4:     "OFPPF_100MB_HD",
-	8:     "OFPPF_100MB_FD",
-	16:    "OFPPF_1GB_HD",
-	32:    "OFPPF_1GB_FD",
-	64:    "OFPPF_10GB_FD",
-	128:   "OFPPF_40GB_FD",
-	256:   "OFPPF_100GB_FD",
-	512:   "OFPPF_1TB_FD",
-	1024:  "OFPPF_OTHER",
-	2048:  "OFPPF_COPPER",
-	4096:  "OFPPF_FIBER",
-	8192:  "OFPPF_AUTONEG",
-	16384: "OFPPF_PAUSE",
-	32768: "OFPPF_PAUSE_ASYM",
-}
-
-var OfpPortFeatures_value = map[string]int32{
-	"OFPPF_INVALID":    0,
-	"OFPPF_10MB_HD":    1,
-	"OFPPF_10MB_FD":    2,
-	"OFPPF_100MB_HD":   4,
-	"OFPPF_100MB_FD":   8,
-	"OFPPF_1GB_HD":     16,
-	"OFPPF_1GB_FD":     32,
-	"OFPPF_10GB_FD":    64,
-	"OFPPF_40GB_FD":    128,
-	"OFPPF_100GB_FD":   256,
-	"OFPPF_1TB_FD":     512,
-	"OFPPF_OTHER":      1024,
-	"OFPPF_COPPER":     2048,
-	"OFPPF_FIBER":      4096,
-	"OFPPF_AUTONEG":    8192,
-	"OFPPF_PAUSE":      16384,
-	"OFPPF_PAUSE_ASYM": 32768,
-}
-
-func (x OfpPortFeatures) String() string {
-	return proto.EnumName(OfpPortFeatures_name, int32(x))
-}
-
-func (OfpPortFeatures) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{9}
-}
-
-// What changed about the physical port
-type OfpPortReason int32
-
-const (
-	OfpPortReason_OFPPR_ADD    OfpPortReason = 0
-	OfpPortReason_OFPPR_DELETE OfpPortReason = 1
-	OfpPortReason_OFPPR_MODIFY OfpPortReason = 2
-)
-
-var OfpPortReason_name = map[int32]string{
-	0: "OFPPR_ADD",
-	1: "OFPPR_DELETE",
-	2: "OFPPR_MODIFY",
-}
-
-var OfpPortReason_value = map[string]int32{
-	"OFPPR_ADD":    0,
-	"OFPPR_DELETE": 1,
-	"OFPPR_MODIFY": 2,
-}
-
-func (x OfpPortReason) String() string {
-	return proto.EnumName(OfpPortReason_name, int32(x))
-}
-
-func (OfpPortReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{10}
-}
-
-// The match type indicates the match structure (set of fields that compose the
-// match) in use. The match type is placed in the type field at the beginning
-// of all match structures. The "OpenFlow Extensible Match" type corresponds
-// to OXM TLV format described below and must be supported by all OpenFlow
-// switches. Extensions that define other match types may be published on the
-// ONF wiki. Support for extensions is optional.
-type OfpMatchType int32
-
-const (
-	OfpMatchType_OFPMT_STANDARD OfpMatchType = 0
-	OfpMatchType_OFPMT_OXM      OfpMatchType = 1
-)
-
-var OfpMatchType_name = map[int32]string{
-	0: "OFPMT_STANDARD",
-	1: "OFPMT_OXM",
-}
-
-var OfpMatchType_value = map[string]int32{
-	"OFPMT_STANDARD": 0,
-	"OFPMT_OXM":      1,
-}
-
-func (x OfpMatchType) String() string {
-	return proto.EnumName(OfpMatchType_name, int32(x))
-}
-
-func (OfpMatchType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{11}
-}
-
-// OXM Class IDs.
-// The high order bit differentiate reserved classes from member classes.
-// Classes 0x0000 to 0x7FFF are member classes, allocated by ONF.
-// Classes 0x8000 to 0xFFFE are reserved classes, reserved for standardisation.
-type OfpOxmClass int32
-
-const (
-	OfpOxmClass_OFPXMC_NXM_0          OfpOxmClass = 0
-	OfpOxmClass_OFPXMC_NXM_1          OfpOxmClass = 1
-	OfpOxmClass_OFPXMC_OPENFLOW_BASIC OfpOxmClass = 32768
-	OfpOxmClass_OFPXMC_EXPERIMENTER   OfpOxmClass = 65535
-)
-
-var OfpOxmClass_name = map[int32]string{
-	0:     "OFPXMC_NXM_0",
-	1:     "OFPXMC_NXM_1",
-	32768: "OFPXMC_OPENFLOW_BASIC",
-	65535: "OFPXMC_EXPERIMENTER",
-}
-
-var OfpOxmClass_value = map[string]int32{
-	"OFPXMC_NXM_0":          0,
-	"OFPXMC_NXM_1":          1,
-	"OFPXMC_OPENFLOW_BASIC": 32768,
-	"OFPXMC_EXPERIMENTER":   65535,
-}
-
-func (x OfpOxmClass) String() string {
-	return proto.EnumName(OfpOxmClass_name, int32(x))
-}
-
-func (OfpOxmClass) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{12}
-}
-
-// OXM Flow field types for OpenFlow basic class.
-type OxmOfbFieldTypes int32
-
-const (
-	OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT        OxmOfbFieldTypes = 0
-	OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT    OxmOfbFieldTypes = 1
-	OxmOfbFieldTypes_OFPXMT_OFB_METADATA       OxmOfbFieldTypes = 2
-	OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST        OxmOfbFieldTypes = 3
-	OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC        OxmOfbFieldTypes = 4
-	OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE       OxmOfbFieldTypes = 5
-	OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID       OxmOfbFieldTypes = 6
-	OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP       OxmOfbFieldTypes = 7
-	OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP        OxmOfbFieldTypes = 8
-	OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN         OxmOfbFieldTypes = 9
-	OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO       OxmOfbFieldTypes = 10
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC       OxmOfbFieldTypes = 11
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST       OxmOfbFieldTypes = 12
-	OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC        OxmOfbFieldTypes = 13
-	OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST        OxmOfbFieldTypes = 14
-	OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC        OxmOfbFieldTypes = 15
-	OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST        OxmOfbFieldTypes = 16
-	OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC       OxmOfbFieldTypes = 17
-	OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST       OxmOfbFieldTypes = 18
-	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE    OxmOfbFieldTypes = 19
-	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE    OxmOfbFieldTypes = 20
-	OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP         OxmOfbFieldTypes = 21
-	OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA        OxmOfbFieldTypes = 22
-	OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA        OxmOfbFieldTypes = 23
-	OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA        OxmOfbFieldTypes = 24
-	OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA        OxmOfbFieldTypes = 25
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC       OxmOfbFieldTypes = 26
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST       OxmOfbFieldTypes = 27
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL    OxmOfbFieldTypes = 28
-	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE    OxmOfbFieldTypes = 29
-	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE    OxmOfbFieldTypes = 30
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET OxmOfbFieldTypes = 31
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL    OxmOfbFieldTypes = 32
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL    OxmOfbFieldTypes = 33
-	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL     OxmOfbFieldTypes = 34
-	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC        OxmOfbFieldTypes = 35
-	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS       OxmOfbFieldTypes = 36
-	OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID       OxmOfbFieldTypes = 37
-	OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID      OxmOfbFieldTypes = 38
-	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR    OxmOfbFieldTypes = 39
-)
-
-var OxmOfbFieldTypes_name = map[int32]string{
-	0:  "OFPXMT_OFB_IN_PORT",
-	1:  "OFPXMT_OFB_IN_PHY_PORT",
-	2:  "OFPXMT_OFB_METADATA",
-	3:  "OFPXMT_OFB_ETH_DST",
-	4:  "OFPXMT_OFB_ETH_SRC",
-	5:  "OFPXMT_OFB_ETH_TYPE",
-	6:  "OFPXMT_OFB_VLAN_VID",
-	7:  "OFPXMT_OFB_VLAN_PCP",
-	8:  "OFPXMT_OFB_IP_DSCP",
-	9:  "OFPXMT_OFB_IP_ECN",
-	10: "OFPXMT_OFB_IP_PROTO",
-	11: "OFPXMT_OFB_IPV4_SRC",
-	12: "OFPXMT_OFB_IPV4_DST",
-	13: "OFPXMT_OFB_TCP_SRC",
-	14: "OFPXMT_OFB_TCP_DST",
-	15: "OFPXMT_OFB_UDP_SRC",
-	16: "OFPXMT_OFB_UDP_DST",
-	17: "OFPXMT_OFB_SCTP_SRC",
-	18: "OFPXMT_OFB_SCTP_DST",
-	19: "OFPXMT_OFB_ICMPV4_TYPE",
-	20: "OFPXMT_OFB_ICMPV4_CODE",
-	21: "OFPXMT_OFB_ARP_OP",
-	22: "OFPXMT_OFB_ARP_SPA",
-	23: "OFPXMT_OFB_ARP_TPA",
-	24: "OFPXMT_OFB_ARP_SHA",
-	25: "OFPXMT_OFB_ARP_THA",
-	26: "OFPXMT_OFB_IPV6_SRC",
-	27: "OFPXMT_OFB_IPV6_DST",
-	28: "OFPXMT_OFB_IPV6_FLABEL",
-	29: "OFPXMT_OFB_ICMPV6_TYPE",
-	30: "OFPXMT_OFB_ICMPV6_CODE",
-	31: "OFPXMT_OFB_IPV6_ND_TARGET",
-	32: "OFPXMT_OFB_IPV6_ND_SLL",
-	33: "OFPXMT_OFB_IPV6_ND_TLL",
-	34: "OFPXMT_OFB_MPLS_LABEL",
-	35: "OFPXMT_OFB_MPLS_TC",
-	36: "OFPXMT_OFB_MPLS_BOS",
-	37: "OFPXMT_OFB_PBB_ISID",
-	38: "OFPXMT_OFB_TUNNEL_ID",
-	39: "OFPXMT_OFB_IPV6_EXTHDR",
-}
-
-var OxmOfbFieldTypes_value = map[string]int32{
-	"OFPXMT_OFB_IN_PORT":        0,
-	"OFPXMT_OFB_IN_PHY_PORT":    1,
-	"OFPXMT_OFB_METADATA":       2,
-	"OFPXMT_OFB_ETH_DST":        3,
-	"OFPXMT_OFB_ETH_SRC":        4,
-	"OFPXMT_OFB_ETH_TYPE":       5,
-	"OFPXMT_OFB_VLAN_VID":       6,
-	"OFPXMT_OFB_VLAN_PCP":       7,
-	"OFPXMT_OFB_IP_DSCP":        8,
-	"OFPXMT_OFB_IP_ECN":         9,
-	"OFPXMT_OFB_IP_PROTO":       10,
-	"OFPXMT_OFB_IPV4_SRC":       11,
-	"OFPXMT_OFB_IPV4_DST":       12,
-	"OFPXMT_OFB_TCP_SRC":        13,
-	"OFPXMT_OFB_TCP_DST":        14,
-	"OFPXMT_OFB_UDP_SRC":        15,
-	"OFPXMT_OFB_UDP_DST":        16,
-	"OFPXMT_OFB_SCTP_SRC":       17,
-	"OFPXMT_OFB_SCTP_DST":       18,
-	"OFPXMT_OFB_ICMPV4_TYPE":    19,
-	"OFPXMT_OFB_ICMPV4_CODE":    20,
-	"OFPXMT_OFB_ARP_OP":         21,
-	"OFPXMT_OFB_ARP_SPA":        22,
-	"OFPXMT_OFB_ARP_TPA":        23,
-	"OFPXMT_OFB_ARP_SHA":        24,
-	"OFPXMT_OFB_ARP_THA":        25,
-	"OFPXMT_OFB_IPV6_SRC":       26,
-	"OFPXMT_OFB_IPV6_DST":       27,
-	"OFPXMT_OFB_IPV6_FLABEL":    28,
-	"OFPXMT_OFB_ICMPV6_TYPE":    29,
-	"OFPXMT_OFB_ICMPV6_CODE":    30,
-	"OFPXMT_OFB_IPV6_ND_TARGET": 31,
-	"OFPXMT_OFB_IPV6_ND_SLL":    32,
-	"OFPXMT_OFB_IPV6_ND_TLL":    33,
-	"OFPXMT_OFB_MPLS_LABEL":     34,
-	"OFPXMT_OFB_MPLS_TC":        35,
-	"OFPXMT_OFB_MPLS_BOS":       36,
-	"OFPXMT_OFB_PBB_ISID":       37,
-	"OFPXMT_OFB_TUNNEL_ID":      38,
-	"OFPXMT_OFB_IPV6_EXTHDR":    39,
-}
-
-func (x OxmOfbFieldTypes) String() string {
-	return proto.EnumName(OxmOfbFieldTypes_name, int32(x))
-}
-
-func (OxmOfbFieldTypes) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{13}
-}
-
-// The VLAN id is 12-bits, so we can use the entire 16 bits to indicate
-// special conditions.
-type OfpVlanId int32
-
-const (
-	OfpVlanId_OFPVID_NONE    OfpVlanId = 0
-	OfpVlanId_OFPVID_PRESENT OfpVlanId = 4096
-)
-
-var OfpVlanId_name = map[int32]string{
-	0:    "OFPVID_NONE",
-	4096: "OFPVID_PRESENT",
-}
-
-var OfpVlanId_value = map[string]int32{
-	"OFPVID_NONE":    0,
-	"OFPVID_PRESENT": 4096,
-}
-
-func (x OfpVlanId) String() string {
-	return proto.EnumName(OfpVlanId_name, int32(x))
-}
-
-func (OfpVlanId) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{14}
-}
-
-// Bit definitions for IPv6 Extension Header pseudo-field.
-type OfpIpv6ExthdrFlags int32
-
-const (
-	OfpIpv6ExthdrFlags_OFPIEH_INVALID OfpIpv6ExthdrFlags = 0
-	OfpIpv6ExthdrFlags_OFPIEH_NONEXT  OfpIpv6ExthdrFlags = 1
-	OfpIpv6ExthdrFlags_OFPIEH_ESP     OfpIpv6ExthdrFlags = 2
-	OfpIpv6ExthdrFlags_OFPIEH_AUTH    OfpIpv6ExthdrFlags = 4
-	OfpIpv6ExthdrFlags_OFPIEH_DEST    OfpIpv6ExthdrFlags = 8
-	OfpIpv6ExthdrFlags_OFPIEH_FRAG    OfpIpv6ExthdrFlags = 16
-	OfpIpv6ExthdrFlags_OFPIEH_ROUTER  OfpIpv6ExthdrFlags = 32
-	OfpIpv6ExthdrFlags_OFPIEH_HOP     OfpIpv6ExthdrFlags = 64
-	OfpIpv6ExthdrFlags_OFPIEH_UNREP   OfpIpv6ExthdrFlags = 128
-	OfpIpv6ExthdrFlags_OFPIEH_UNSEQ   OfpIpv6ExthdrFlags = 256
-)
-
-var OfpIpv6ExthdrFlags_name = map[int32]string{
-	0:   "OFPIEH_INVALID",
-	1:   "OFPIEH_NONEXT",
-	2:   "OFPIEH_ESP",
-	4:   "OFPIEH_AUTH",
-	8:   "OFPIEH_DEST",
-	16:  "OFPIEH_FRAG",
-	32:  "OFPIEH_ROUTER",
-	64:  "OFPIEH_HOP",
-	128: "OFPIEH_UNREP",
-	256: "OFPIEH_UNSEQ",
-}
-
-var OfpIpv6ExthdrFlags_value = map[string]int32{
-	"OFPIEH_INVALID": 0,
-	"OFPIEH_NONEXT":  1,
-	"OFPIEH_ESP":     2,
-	"OFPIEH_AUTH":    4,
-	"OFPIEH_DEST":    8,
-	"OFPIEH_FRAG":    16,
-	"OFPIEH_ROUTER":  32,
-	"OFPIEH_HOP":     64,
-	"OFPIEH_UNREP":   128,
-	"OFPIEH_UNSEQ":   256,
-}
-
-func (x OfpIpv6ExthdrFlags) String() string {
-	return proto.EnumName(OfpIpv6ExthdrFlags_name, int32(x))
-}
-
-func (OfpIpv6ExthdrFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{15}
-}
-
-type OfpActionType int32
-
-const (
-	OfpActionType_OFPAT_OUTPUT       OfpActionType = 0
-	OfpActionType_OFPAT_COPY_TTL_OUT OfpActionType = 11
-	OfpActionType_OFPAT_COPY_TTL_IN  OfpActionType = 12
-	OfpActionType_OFPAT_SET_MPLS_TTL OfpActionType = 15
-	OfpActionType_OFPAT_DEC_MPLS_TTL OfpActionType = 16
-	OfpActionType_OFPAT_PUSH_VLAN    OfpActionType = 17
-	OfpActionType_OFPAT_POP_VLAN     OfpActionType = 18
-	OfpActionType_OFPAT_PUSH_MPLS    OfpActionType = 19
-	OfpActionType_OFPAT_POP_MPLS     OfpActionType = 20
-	OfpActionType_OFPAT_SET_QUEUE    OfpActionType = 21
-	OfpActionType_OFPAT_GROUP        OfpActionType = 22
-	OfpActionType_OFPAT_SET_NW_TTL   OfpActionType = 23
-	OfpActionType_OFPAT_DEC_NW_TTL   OfpActionType = 24
-	OfpActionType_OFPAT_SET_FIELD    OfpActionType = 25
-	OfpActionType_OFPAT_PUSH_PBB     OfpActionType = 26
-	OfpActionType_OFPAT_POP_PBB      OfpActionType = 27
-	OfpActionType_OFPAT_EXPERIMENTER OfpActionType = 65535
-)
-
-var OfpActionType_name = map[int32]string{
-	0:     "OFPAT_OUTPUT",
-	11:    "OFPAT_COPY_TTL_OUT",
-	12:    "OFPAT_COPY_TTL_IN",
-	15:    "OFPAT_SET_MPLS_TTL",
-	16:    "OFPAT_DEC_MPLS_TTL",
-	17:    "OFPAT_PUSH_VLAN",
-	18:    "OFPAT_POP_VLAN",
-	19:    "OFPAT_PUSH_MPLS",
-	20:    "OFPAT_POP_MPLS",
-	21:    "OFPAT_SET_QUEUE",
-	22:    "OFPAT_GROUP",
-	23:    "OFPAT_SET_NW_TTL",
-	24:    "OFPAT_DEC_NW_TTL",
-	25:    "OFPAT_SET_FIELD",
-	26:    "OFPAT_PUSH_PBB",
-	27:    "OFPAT_POP_PBB",
-	65535: "OFPAT_EXPERIMENTER",
-}
-
-var OfpActionType_value = map[string]int32{
-	"OFPAT_OUTPUT":       0,
-	"OFPAT_COPY_TTL_OUT": 11,
-	"OFPAT_COPY_TTL_IN":  12,
-	"OFPAT_SET_MPLS_TTL": 15,
-	"OFPAT_DEC_MPLS_TTL": 16,
-	"OFPAT_PUSH_VLAN":    17,
-	"OFPAT_POP_VLAN":     18,
-	"OFPAT_PUSH_MPLS":    19,
-	"OFPAT_POP_MPLS":     20,
-	"OFPAT_SET_QUEUE":    21,
-	"OFPAT_GROUP":        22,
-	"OFPAT_SET_NW_TTL":   23,
-	"OFPAT_DEC_NW_TTL":   24,
-	"OFPAT_SET_FIELD":    25,
-	"OFPAT_PUSH_PBB":     26,
-	"OFPAT_POP_PBB":      27,
-	"OFPAT_EXPERIMENTER": 65535,
-}
-
-func (x OfpActionType) String() string {
-	return proto.EnumName(OfpActionType_name, int32(x))
-}
-
-func (OfpActionType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{16}
-}
-
-type OfpControllerMaxLen int32
-
-const (
-	OfpControllerMaxLen_OFPCML_INVALID   OfpControllerMaxLen = 0
-	OfpControllerMaxLen_OFPCML_MAX       OfpControllerMaxLen = 65509
-	OfpControllerMaxLen_OFPCML_NO_BUFFER OfpControllerMaxLen = 65535
-)
-
-var OfpControllerMaxLen_name = map[int32]string{
-	0:     "OFPCML_INVALID",
-	65509: "OFPCML_MAX",
-	65535: "OFPCML_NO_BUFFER",
-}
-
-var OfpControllerMaxLen_value = map[string]int32{
-	"OFPCML_INVALID":   0,
-	"OFPCML_MAX":       65509,
-	"OFPCML_NO_BUFFER": 65535,
-}
-
-func (x OfpControllerMaxLen) String() string {
-	return proto.EnumName(OfpControllerMaxLen_name, int32(x))
-}
-
-func (OfpControllerMaxLen) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{17}
-}
-
-type OfpInstructionType int32
-
-const (
-	OfpInstructionType_OFPIT_INVALID        OfpInstructionType = 0
-	OfpInstructionType_OFPIT_GOTO_TABLE     OfpInstructionType = 1
-	OfpInstructionType_OFPIT_WRITE_METADATA OfpInstructionType = 2
-	OfpInstructionType_OFPIT_WRITE_ACTIONS  OfpInstructionType = 3
-	OfpInstructionType_OFPIT_APPLY_ACTIONS  OfpInstructionType = 4
-	OfpInstructionType_OFPIT_CLEAR_ACTIONS  OfpInstructionType = 5
-	OfpInstructionType_OFPIT_METER          OfpInstructionType = 6
-	OfpInstructionType_OFPIT_EXPERIMENTER   OfpInstructionType = 65535
-)
-
-var OfpInstructionType_name = map[int32]string{
-	0:     "OFPIT_INVALID",
-	1:     "OFPIT_GOTO_TABLE",
-	2:     "OFPIT_WRITE_METADATA",
-	3:     "OFPIT_WRITE_ACTIONS",
-	4:     "OFPIT_APPLY_ACTIONS",
-	5:     "OFPIT_CLEAR_ACTIONS",
-	6:     "OFPIT_METER",
-	65535: "OFPIT_EXPERIMENTER",
-}
-
-var OfpInstructionType_value = map[string]int32{
-	"OFPIT_INVALID":        0,
-	"OFPIT_GOTO_TABLE":     1,
-	"OFPIT_WRITE_METADATA": 2,
-	"OFPIT_WRITE_ACTIONS":  3,
-	"OFPIT_APPLY_ACTIONS":  4,
-	"OFPIT_CLEAR_ACTIONS":  5,
-	"OFPIT_METER":          6,
-	"OFPIT_EXPERIMENTER":   65535,
-}
-
-func (x OfpInstructionType) String() string {
-	return proto.EnumName(OfpInstructionType_name, int32(x))
-}
-
-func (OfpInstructionType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{18}
-}
-
-type OfpFlowModCommand int32
-
-const (
-	OfpFlowModCommand_OFPFC_ADD           OfpFlowModCommand = 0
-	OfpFlowModCommand_OFPFC_MODIFY        OfpFlowModCommand = 1
-	OfpFlowModCommand_OFPFC_MODIFY_STRICT OfpFlowModCommand = 2
-	OfpFlowModCommand_OFPFC_DELETE        OfpFlowModCommand = 3
-	OfpFlowModCommand_OFPFC_DELETE_STRICT OfpFlowModCommand = 4
-)
-
-var OfpFlowModCommand_name = map[int32]string{
-	0: "OFPFC_ADD",
-	1: "OFPFC_MODIFY",
-	2: "OFPFC_MODIFY_STRICT",
-	3: "OFPFC_DELETE",
-	4: "OFPFC_DELETE_STRICT",
-}
-
-var OfpFlowModCommand_value = map[string]int32{
-	"OFPFC_ADD":           0,
-	"OFPFC_MODIFY":        1,
-	"OFPFC_MODIFY_STRICT": 2,
-	"OFPFC_DELETE":        3,
-	"OFPFC_DELETE_STRICT": 4,
-}
-
-func (x OfpFlowModCommand) String() string {
-	return proto.EnumName(OfpFlowModCommand_name, int32(x))
-}
-
-func (OfpFlowModCommand) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{19}
-}
-
-type OfpFlowModFlags int32
-
-const (
-	OfpFlowModFlags_OFPFF_INVALID       OfpFlowModFlags = 0
-	OfpFlowModFlags_OFPFF_SEND_FLOW_REM OfpFlowModFlags = 1
-	OfpFlowModFlags_OFPFF_CHECK_OVERLAP OfpFlowModFlags = 2
-	OfpFlowModFlags_OFPFF_RESET_COUNTS  OfpFlowModFlags = 4
-	OfpFlowModFlags_OFPFF_NO_PKT_COUNTS OfpFlowModFlags = 8
-	OfpFlowModFlags_OFPFF_NO_BYT_COUNTS OfpFlowModFlags = 16
-)
-
-var OfpFlowModFlags_name = map[int32]string{
-	0:  "OFPFF_INVALID",
-	1:  "OFPFF_SEND_FLOW_REM",
-	2:  "OFPFF_CHECK_OVERLAP",
-	4:  "OFPFF_RESET_COUNTS",
-	8:  "OFPFF_NO_PKT_COUNTS",
-	16: "OFPFF_NO_BYT_COUNTS",
-}
-
-var OfpFlowModFlags_value = map[string]int32{
-	"OFPFF_INVALID":       0,
-	"OFPFF_SEND_FLOW_REM": 1,
-	"OFPFF_CHECK_OVERLAP": 2,
-	"OFPFF_RESET_COUNTS":  4,
-	"OFPFF_NO_PKT_COUNTS": 8,
-	"OFPFF_NO_BYT_COUNTS": 16,
-}
-
-func (x OfpFlowModFlags) String() string {
-	return proto.EnumName(OfpFlowModFlags_name, int32(x))
-}
-
-func (OfpFlowModFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{20}
-}
-
-// Group numbering. Groups can use any number up to OFPG_MAX.
-type OfpGroup int32
-
-const (
-	OfpGroup_OFPG_INVALID OfpGroup = 0
-	// Last usable group number.
-	OfpGroup_OFPG_MAX OfpGroup = 2147483392
-	// Fake groups.
-	OfpGroup_OFPG_ALL OfpGroup = 2147483644
-	OfpGroup_OFPG_ANY OfpGroup = 2147483647
-)
-
-var OfpGroup_name = map[int32]string{
-	0:          "OFPG_INVALID",
-	2147483392: "OFPG_MAX",
-	2147483644: "OFPG_ALL",
-	2147483647: "OFPG_ANY",
-}
-
-var OfpGroup_value = map[string]int32{
-	"OFPG_INVALID": 0,
-	"OFPG_MAX":     2147483392,
-	"OFPG_ALL":     2147483644,
-	"OFPG_ANY":     2147483647,
-}
-
-func (x OfpGroup) String() string {
-	return proto.EnumName(OfpGroup_name, int32(x))
-}
-
-func (OfpGroup) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{21}
-}
-
-// Group commands
-type OfpGroupModCommand int32
-
-const (
-	OfpGroupModCommand_OFPGC_ADD    OfpGroupModCommand = 0
-	OfpGroupModCommand_OFPGC_MODIFY OfpGroupModCommand = 1
-	OfpGroupModCommand_OFPGC_DELETE OfpGroupModCommand = 2
-)
-
-var OfpGroupModCommand_name = map[int32]string{
-	0: "OFPGC_ADD",
-	1: "OFPGC_MODIFY",
-	2: "OFPGC_DELETE",
-}
-
-var OfpGroupModCommand_value = map[string]int32{
-	"OFPGC_ADD":    0,
-	"OFPGC_MODIFY": 1,
-	"OFPGC_DELETE": 2,
-}
-
-func (x OfpGroupModCommand) String() string {
-	return proto.EnumName(OfpGroupModCommand_name, int32(x))
-}
-
-func (OfpGroupModCommand) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{22}
-}
-
-// Group types.  Values in the range [128; 255] are reserved for experimental
-// use.
-type OfpGroupType int32
-
-const (
-	OfpGroupType_OFPGT_ALL      OfpGroupType = 0
-	OfpGroupType_OFPGT_SELECT   OfpGroupType = 1
-	OfpGroupType_OFPGT_INDIRECT OfpGroupType = 2
-	OfpGroupType_OFPGT_FF       OfpGroupType = 3
-)
-
-var OfpGroupType_name = map[int32]string{
-	0: "OFPGT_ALL",
-	1: "OFPGT_SELECT",
-	2: "OFPGT_INDIRECT",
-	3: "OFPGT_FF",
-}
-
-var OfpGroupType_value = map[string]int32{
-	"OFPGT_ALL":      0,
-	"OFPGT_SELECT":   1,
-	"OFPGT_INDIRECT": 2,
-	"OFPGT_FF":       3,
-}
-
-func (x OfpGroupType) String() string {
-	return proto.EnumName(OfpGroupType_name, int32(x))
-}
-
-func (OfpGroupType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{23}
-}
-
-// Why is this packet being sent to the controller?
-type OfpPacketInReason int32
-
-const (
-	OfpPacketInReason_OFPR_NO_MATCH    OfpPacketInReason = 0
-	OfpPacketInReason_OFPR_ACTION      OfpPacketInReason = 1
-	OfpPacketInReason_OFPR_INVALID_TTL OfpPacketInReason = 2
-)
-
-var OfpPacketInReason_name = map[int32]string{
-	0: "OFPR_NO_MATCH",
-	1: "OFPR_ACTION",
-	2: "OFPR_INVALID_TTL",
-}
-
-var OfpPacketInReason_value = map[string]int32{
-	"OFPR_NO_MATCH":    0,
-	"OFPR_ACTION":      1,
-	"OFPR_INVALID_TTL": 2,
-}
-
-func (x OfpPacketInReason) String() string {
-	return proto.EnumName(OfpPacketInReason_name, int32(x))
-}
-
-func (OfpPacketInReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{24}
-}
-
-// Why was this flow removed?
-type OfpFlowRemovedReason int32
-
-const (
-	OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT OfpFlowRemovedReason = 0
-	OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT OfpFlowRemovedReason = 1
-	OfpFlowRemovedReason_OFPRR_DELETE       OfpFlowRemovedReason = 2
-	OfpFlowRemovedReason_OFPRR_GROUP_DELETE OfpFlowRemovedReason = 3
-	OfpFlowRemovedReason_OFPRR_METER_DELETE OfpFlowRemovedReason = 4
-)
-
-var OfpFlowRemovedReason_name = map[int32]string{
-	0: "OFPRR_IDLE_TIMEOUT",
-	1: "OFPRR_HARD_TIMEOUT",
-	2: "OFPRR_DELETE",
-	3: "OFPRR_GROUP_DELETE",
-	4: "OFPRR_METER_DELETE",
-}
-
-var OfpFlowRemovedReason_value = map[string]int32{
-	"OFPRR_IDLE_TIMEOUT": 0,
-	"OFPRR_HARD_TIMEOUT": 1,
-	"OFPRR_DELETE":       2,
-	"OFPRR_GROUP_DELETE": 3,
-	"OFPRR_METER_DELETE": 4,
-}
-
-func (x OfpFlowRemovedReason) String() string {
-	return proto.EnumName(OfpFlowRemovedReason_name, int32(x))
-}
-
-func (OfpFlowRemovedReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{25}
-}
-
-// Meter numbering. Flow meters can use any number up to OFPM_MAX.
-type OfpMeter int32
-
-const (
-	OfpMeter_OFPM_ZERO OfpMeter = 0
-	// Last usable meter.
-	OfpMeter_OFPM_MAX OfpMeter = 2147418112
-	// Virtual meters.
-	OfpMeter_OFPM_SLOWPATH   OfpMeter = 2147483645
-	OfpMeter_OFPM_CONTROLLER OfpMeter = 2147483646
-	OfpMeter_OFPM_ALL        OfpMeter = 2147483647
-)
-
-var OfpMeter_name = map[int32]string{
-	0:          "OFPM_ZERO",
-	2147418112: "OFPM_MAX",
-	2147483645: "OFPM_SLOWPATH",
-	2147483646: "OFPM_CONTROLLER",
-	2147483647: "OFPM_ALL",
-}
-
-var OfpMeter_value = map[string]int32{
-	"OFPM_ZERO":       0,
-	"OFPM_MAX":        2147418112,
-	"OFPM_SLOWPATH":   2147483645,
-	"OFPM_CONTROLLER": 2147483646,
-	"OFPM_ALL":        2147483647,
-}
-
-func (x OfpMeter) String() string {
-	return proto.EnumName(OfpMeter_name, int32(x))
-}
-
-func (OfpMeter) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{26}
-}
-
-// Meter band types
-type OfpMeterBandType int32
-
-const (
-	OfpMeterBandType_OFPMBT_INVALID      OfpMeterBandType = 0
-	OfpMeterBandType_OFPMBT_DROP         OfpMeterBandType = 1
-	OfpMeterBandType_OFPMBT_DSCP_REMARK  OfpMeterBandType = 2
-	OfpMeterBandType_OFPMBT_EXPERIMENTER OfpMeterBandType = 65535
-)
-
-var OfpMeterBandType_name = map[int32]string{
-	0:     "OFPMBT_INVALID",
-	1:     "OFPMBT_DROP",
-	2:     "OFPMBT_DSCP_REMARK",
-	65535: "OFPMBT_EXPERIMENTER",
-}
-
-var OfpMeterBandType_value = map[string]int32{
-	"OFPMBT_INVALID":      0,
-	"OFPMBT_DROP":         1,
-	"OFPMBT_DSCP_REMARK":  2,
-	"OFPMBT_EXPERIMENTER": 65535,
-}
-
-func (x OfpMeterBandType) String() string {
-	return proto.EnumName(OfpMeterBandType_name, int32(x))
-}
-
-func (OfpMeterBandType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{27}
-}
-
-// Meter commands
-type OfpMeterModCommand int32
-
-const (
-	OfpMeterModCommand_OFPMC_ADD    OfpMeterModCommand = 0
-	OfpMeterModCommand_OFPMC_MODIFY OfpMeterModCommand = 1
-	OfpMeterModCommand_OFPMC_DELETE OfpMeterModCommand = 2
-)
-
-var OfpMeterModCommand_name = map[int32]string{
-	0: "OFPMC_ADD",
-	1: "OFPMC_MODIFY",
-	2: "OFPMC_DELETE",
-}
-
-var OfpMeterModCommand_value = map[string]int32{
-	"OFPMC_ADD":    0,
-	"OFPMC_MODIFY": 1,
-	"OFPMC_DELETE": 2,
-}
-
-func (x OfpMeterModCommand) String() string {
-	return proto.EnumName(OfpMeterModCommand_name, int32(x))
-}
-
-func (OfpMeterModCommand) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{28}
-}
-
-// Meter configuration flags
-type OfpMeterFlags int32
-
-const (
-	OfpMeterFlags_OFPMF_INVALID OfpMeterFlags = 0
-	OfpMeterFlags_OFPMF_KBPS    OfpMeterFlags = 1
-	OfpMeterFlags_OFPMF_PKTPS   OfpMeterFlags = 2
-	OfpMeterFlags_OFPMF_BURST   OfpMeterFlags = 4
-	OfpMeterFlags_OFPMF_STATS   OfpMeterFlags = 8
-)
-
-var OfpMeterFlags_name = map[int32]string{
-	0: "OFPMF_INVALID",
-	1: "OFPMF_KBPS",
-	2: "OFPMF_PKTPS",
-	4: "OFPMF_BURST",
-	8: "OFPMF_STATS",
-}
-
-var OfpMeterFlags_value = map[string]int32{
-	"OFPMF_INVALID": 0,
-	"OFPMF_KBPS":    1,
-	"OFPMF_PKTPS":   2,
-	"OFPMF_BURST":   4,
-	"OFPMF_STATS":   8,
-}
-
-func (x OfpMeterFlags) String() string {
-	return proto.EnumName(OfpMeterFlags_name, int32(x))
-}
-
-func (OfpMeterFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{29}
-}
-
-// Values for 'type' in ofp_error_message.  These values are immutable: they
-// will not change in future versions of the protocol (although new values may
-// be added).
-type OfpErrorType int32
-
-const (
-	OfpErrorType_OFPET_HELLO_FAILED          OfpErrorType = 0
-	OfpErrorType_OFPET_BAD_REQUEST           OfpErrorType = 1
-	OfpErrorType_OFPET_BAD_ACTION            OfpErrorType = 2
-	OfpErrorType_OFPET_BAD_INSTRUCTION       OfpErrorType = 3
-	OfpErrorType_OFPET_BAD_MATCH             OfpErrorType = 4
-	OfpErrorType_OFPET_FLOW_MOD_FAILED       OfpErrorType = 5
-	OfpErrorType_OFPET_GROUP_MOD_FAILED      OfpErrorType = 6
-	OfpErrorType_OFPET_PORT_MOD_FAILED       OfpErrorType = 7
-	OfpErrorType_OFPET_TABLE_MOD_FAILED      OfpErrorType = 8
-	OfpErrorType_OFPET_QUEUE_OP_FAILED       OfpErrorType = 9
-	OfpErrorType_OFPET_SWITCH_CONFIG_FAILED  OfpErrorType = 10
-	OfpErrorType_OFPET_ROLE_REQUEST_FAILED   OfpErrorType = 11
-	OfpErrorType_OFPET_METER_MOD_FAILED      OfpErrorType = 12
-	OfpErrorType_OFPET_TABLE_FEATURES_FAILED OfpErrorType = 13
-	OfpErrorType_OFPET_EXPERIMENTER          OfpErrorType = 65535
-)
-
-var OfpErrorType_name = map[int32]string{
-	0:     "OFPET_HELLO_FAILED",
-	1:     "OFPET_BAD_REQUEST",
-	2:     "OFPET_BAD_ACTION",
-	3:     "OFPET_BAD_INSTRUCTION",
-	4:     "OFPET_BAD_MATCH",
-	5:     "OFPET_FLOW_MOD_FAILED",
-	6:     "OFPET_GROUP_MOD_FAILED",
-	7:     "OFPET_PORT_MOD_FAILED",
-	8:     "OFPET_TABLE_MOD_FAILED",
-	9:     "OFPET_QUEUE_OP_FAILED",
-	10:    "OFPET_SWITCH_CONFIG_FAILED",
-	11:    "OFPET_ROLE_REQUEST_FAILED",
-	12:    "OFPET_METER_MOD_FAILED",
-	13:    "OFPET_TABLE_FEATURES_FAILED",
-	65535: "OFPET_EXPERIMENTER",
-}
-
-var OfpErrorType_value = map[string]int32{
-	"OFPET_HELLO_FAILED":          0,
-	"OFPET_BAD_REQUEST":           1,
-	"OFPET_BAD_ACTION":            2,
-	"OFPET_BAD_INSTRUCTION":       3,
-	"OFPET_BAD_MATCH":             4,
-	"OFPET_FLOW_MOD_FAILED":       5,
-	"OFPET_GROUP_MOD_FAILED":      6,
-	"OFPET_PORT_MOD_FAILED":       7,
-	"OFPET_TABLE_MOD_FAILED":      8,
-	"OFPET_QUEUE_OP_FAILED":       9,
-	"OFPET_SWITCH_CONFIG_FAILED":  10,
-	"OFPET_ROLE_REQUEST_FAILED":   11,
-	"OFPET_METER_MOD_FAILED":      12,
-	"OFPET_TABLE_FEATURES_FAILED": 13,
-	"OFPET_EXPERIMENTER":          65535,
-}
-
-func (x OfpErrorType) String() string {
-	return proto.EnumName(OfpErrorType_name, int32(x))
-}
-
-func (OfpErrorType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{30}
-}
-
-// ofp_error_msg 'code' values for OFPET_HELLO_FAILED.  'data' contains an
-// ASCII text string that may give failure details.
-type OfpHelloFailedCode int32
-
-const (
-	OfpHelloFailedCode_OFPHFC_INCOMPATIBLE OfpHelloFailedCode = 0
-	OfpHelloFailedCode_OFPHFC_EPERM        OfpHelloFailedCode = 1
-)
-
-var OfpHelloFailedCode_name = map[int32]string{
-	0: "OFPHFC_INCOMPATIBLE",
-	1: "OFPHFC_EPERM",
-}
-
-var OfpHelloFailedCode_value = map[string]int32{
-	"OFPHFC_INCOMPATIBLE": 0,
-	"OFPHFC_EPERM":        1,
-}
-
-func (x OfpHelloFailedCode) String() string {
-	return proto.EnumName(OfpHelloFailedCode_name, int32(x))
-}
-
-func (OfpHelloFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{31}
-}
-
-// ofp_error_msg 'code' values for OFPET_BAD_REQUEST.  'data' contains at least
-// the first 64 bytes of the failed request.
-type OfpBadRequestCode int32
-
-const (
-	OfpBadRequestCode_OFPBRC_BAD_VERSION               OfpBadRequestCode = 0
-	OfpBadRequestCode_OFPBRC_BAD_TYPE                  OfpBadRequestCode = 1
-	OfpBadRequestCode_OFPBRC_BAD_MULTIPART             OfpBadRequestCode = 2
-	OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER          OfpBadRequestCode = 3
-	OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE              OfpBadRequestCode = 4
-	OfpBadRequestCode_OFPBRC_EPERM                     OfpBadRequestCode = 5
-	OfpBadRequestCode_OFPBRC_BAD_LEN                   OfpBadRequestCode = 6
-	OfpBadRequestCode_OFPBRC_BUFFER_EMPTY              OfpBadRequestCode = 7
-	OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN            OfpBadRequestCode = 8
-	OfpBadRequestCode_OFPBRC_BAD_TABLE_ID              OfpBadRequestCode = 9
-	OfpBadRequestCode_OFPBRC_IS_SLAVE                  OfpBadRequestCode = 10
-	OfpBadRequestCode_OFPBRC_BAD_PORT                  OfpBadRequestCode = 11
-	OfpBadRequestCode_OFPBRC_BAD_PACKET                OfpBadRequestCode = 12
-	OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW OfpBadRequestCode = 13
-)
-
-var OfpBadRequestCode_name = map[int32]string{
-	0:  "OFPBRC_BAD_VERSION",
-	1:  "OFPBRC_BAD_TYPE",
-	2:  "OFPBRC_BAD_MULTIPART",
-	3:  "OFPBRC_BAD_EXPERIMENTER",
-	4:  "OFPBRC_BAD_EXP_TYPE",
-	5:  "OFPBRC_EPERM",
-	6:  "OFPBRC_BAD_LEN",
-	7:  "OFPBRC_BUFFER_EMPTY",
-	8:  "OFPBRC_BUFFER_UNKNOWN",
-	9:  "OFPBRC_BAD_TABLE_ID",
-	10: "OFPBRC_IS_SLAVE",
-	11: "OFPBRC_BAD_PORT",
-	12: "OFPBRC_BAD_PACKET",
-	13: "OFPBRC_MULTIPART_BUFFER_OVERFLOW",
-}
-
-var OfpBadRequestCode_value = map[string]int32{
-	"OFPBRC_BAD_VERSION":               0,
-	"OFPBRC_BAD_TYPE":                  1,
-	"OFPBRC_BAD_MULTIPART":             2,
-	"OFPBRC_BAD_EXPERIMENTER":          3,
-	"OFPBRC_BAD_EXP_TYPE":              4,
-	"OFPBRC_EPERM":                     5,
-	"OFPBRC_BAD_LEN":                   6,
-	"OFPBRC_BUFFER_EMPTY":              7,
-	"OFPBRC_BUFFER_UNKNOWN":            8,
-	"OFPBRC_BAD_TABLE_ID":              9,
-	"OFPBRC_IS_SLAVE":                  10,
-	"OFPBRC_BAD_PORT":                  11,
-	"OFPBRC_BAD_PACKET":                12,
-	"OFPBRC_MULTIPART_BUFFER_OVERFLOW": 13,
-}
-
-func (x OfpBadRequestCode) String() string {
-	return proto.EnumName(OfpBadRequestCode_name, int32(x))
-}
-
-func (OfpBadRequestCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{32}
-}
-
-// ofp_error_msg 'code' values for OFPET_BAD_ACTION.  'data' contains at least
-// the first 64 bytes of the failed request.
-type OfpBadActionCode int32
-
-const (
-	OfpBadActionCode_OFPBAC_BAD_TYPE           OfpBadActionCode = 0
-	OfpBadActionCode_OFPBAC_BAD_LEN            OfpBadActionCode = 1
-	OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER   OfpBadActionCode = 2
-	OfpBadActionCode_OFPBAC_BAD_EXP_TYPE       OfpBadActionCode = 3
-	OfpBadActionCode_OFPBAC_BAD_OUT_PORT       OfpBadActionCode = 4
-	OfpBadActionCode_OFPBAC_BAD_ARGUMENT       OfpBadActionCode = 5
-	OfpBadActionCode_OFPBAC_EPERM              OfpBadActionCode = 6
-	OfpBadActionCode_OFPBAC_TOO_MANY           OfpBadActionCode = 7
-	OfpBadActionCode_OFPBAC_BAD_QUEUE          OfpBadActionCode = 8
-	OfpBadActionCode_OFPBAC_BAD_OUT_GROUP      OfpBadActionCode = 9
-	OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT OfpBadActionCode = 10
-	OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER  OfpBadActionCode = 11
-	OfpBadActionCode_OFPBAC_BAD_TAG            OfpBadActionCode = 12
-	OfpBadActionCode_OFPBAC_BAD_SET_TYPE       OfpBadActionCode = 13
-	OfpBadActionCode_OFPBAC_BAD_SET_LEN        OfpBadActionCode = 14
-	OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT   OfpBadActionCode = 15
-)
-
-var OfpBadActionCode_name = map[int32]string{
-	0:  "OFPBAC_BAD_TYPE",
-	1:  "OFPBAC_BAD_LEN",
-	2:  "OFPBAC_BAD_EXPERIMENTER",
-	3:  "OFPBAC_BAD_EXP_TYPE",
-	4:  "OFPBAC_BAD_OUT_PORT",
-	5:  "OFPBAC_BAD_ARGUMENT",
-	6:  "OFPBAC_EPERM",
-	7:  "OFPBAC_TOO_MANY",
-	8:  "OFPBAC_BAD_QUEUE",
-	9:  "OFPBAC_BAD_OUT_GROUP",
-	10: "OFPBAC_MATCH_INCONSISTENT",
-	11: "OFPBAC_UNSUPPORTED_ORDER",
-	12: "OFPBAC_BAD_TAG",
-	13: "OFPBAC_BAD_SET_TYPE",
-	14: "OFPBAC_BAD_SET_LEN",
-	15: "OFPBAC_BAD_SET_ARGUMENT",
-}
-
-var OfpBadActionCode_value = map[string]int32{
-	"OFPBAC_BAD_TYPE":           0,
-	"OFPBAC_BAD_LEN":            1,
-	"OFPBAC_BAD_EXPERIMENTER":   2,
-	"OFPBAC_BAD_EXP_TYPE":       3,
-	"OFPBAC_BAD_OUT_PORT":       4,
-	"OFPBAC_BAD_ARGUMENT":       5,
-	"OFPBAC_EPERM":              6,
-	"OFPBAC_TOO_MANY":           7,
-	"OFPBAC_BAD_QUEUE":          8,
-	"OFPBAC_BAD_OUT_GROUP":      9,
-	"OFPBAC_MATCH_INCONSISTENT": 10,
-	"OFPBAC_UNSUPPORTED_ORDER":  11,
-	"OFPBAC_BAD_TAG":            12,
-	"OFPBAC_BAD_SET_TYPE":       13,
-	"OFPBAC_BAD_SET_LEN":        14,
-	"OFPBAC_BAD_SET_ARGUMENT":   15,
-}
-
-func (x OfpBadActionCode) String() string {
-	return proto.EnumName(OfpBadActionCode_name, int32(x))
-}
-
-func (OfpBadActionCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{33}
-}
-
-// ofp_error_msg 'code' values for OFPET_BAD_INSTRUCTION.  'data' contains at
-// least the first 64 bytes of the failed request.
-type OfpBadInstructionCode int32
-
-const (
-	OfpBadInstructionCode_OFPBIC_UNKNOWN_INST        OfpBadInstructionCode = 0
-	OfpBadInstructionCode_OFPBIC_UNSUP_INST          OfpBadInstructionCode = 1
-	OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID        OfpBadInstructionCode = 2
-	OfpBadInstructionCode_OFPBIC_UNSUP_METADATA      OfpBadInstructionCode = 3
-	OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK OfpBadInstructionCode = 4
-	OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER    OfpBadInstructionCode = 5
-	OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE        OfpBadInstructionCode = 6
-	OfpBadInstructionCode_OFPBIC_BAD_LEN             OfpBadInstructionCode = 7
-	OfpBadInstructionCode_OFPBIC_EPERM               OfpBadInstructionCode = 8
-)
-
-var OfpBadInstructionCode_name = map[int32]string{
-	0: "OFPBIC_UNKNOWN_INST",
-	1: "OFPBIC_UNSUP_INST",
-	2: "OFPBIC_BAD_TABLE_ID",
-	3: "OFPBIC_UNSUP_METADATA",
-	4: "OFPBIC_UNSUP_METADATA_MASK",
-	5: "OFPBIC_BAD_EXPERIMENTER",
-	6: "OFPBIC_BAD_EXP_TYPE",
-	7: "OFPBIC_BAD_LEN",
-	8: "OFPBIC_EPERM",
-}
-
-var OfpBadInstructionCode_value = map[string]int32{
-	"OFPBIC_UNKNOWN_INST":        0,
-	"OFPBIC_UNSUP_INST":          1,
-	"OFPBIC_BAD_TABLE_ID":        2,
-	"OFPBIC_UNSUP_METADATA":      3,
-	"OFPBIC_UNSUP_METADATA_MASK": 4,
-	"OFPBIC_BAD_EXPERIMENTER":    5,
-	"OFPBIC_BAD_EXP_TYPE":        6,
-	"OFPBIC_BAD_LEN":             7,
-	"OFPBIC_EPERM":               8,
-}
-
-func (x OfpBadInstructionCode) String() string {
-	return proto.EnumName(OfpBadInstructionCode_name, int32(x))
-}
-
-func (OfpBadInstructionCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{34}
-}
-
-// ofp_error_msg 'code' values for OFPET_BAD_MATCH.  'data' contains at least
-// the first 64 bytes of the failed request.
-type OfpBadMatchCode int32
-
-const (
-	OfpBadMatchCode_OFPBMC_BAD_TYPE         OfpBadMatchCode = 0
-	OfpBadMatchCode_OFPBMC_BAD_LEN          OfpBadMatchCode = 1
-	OfpBadMatchCode_OFPBMC_BAD_TAG          OfpBadMatchCode = 2
-	OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK OfpBadMatchCode = 3
-	OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK OfpBadMatchCode = 4
-	OfpBadMatchCode_OFPBMC_BAD_WILDCARDS    OfpBadMatchCode = 5
-	OfpBadMatchCode_OFPBMC_BAD_FIELD        OfpBadMatchCode = 6
-	OfpBadMatchCode_OFPBMC_BAD_VALUE        OfpBadMatchCode = 7
-	OfpBadMatchCode_OFPBMC_BAD_MASK         OfpBadMatchCode = 8
-	OfpBadMatchCode_OFPBMC_BAD_PREREQ       OfpBadMatchCode = 9
-	OfpBadMatchCode_OFPBMC_DUP_FIELD        OfpBadMatchCode = 10
-	OfpBadMatchCode_OFPBMC_EPERM            OfpBadMatchCode = 11
-)
-
-var OfpBadMatchCode_name = map[int32]string{
-	0:  "OFPBMC_BAD_TYPE",
-	1:  "OFPBMC_BAD_LEN",
-	2:  "OFPBMC_BAD_TAG",
-	3:  "OFPBMC_BAD_DL_ADDR_MASK",
-	4:  "OFPBMC_BAD_NW_ADDR_MASK",
-	5:  "OFPBMC_BAD_WILDCARDS",
-	6:  "OFPBMC_BAD_FIELD",
-	7:  "OFPBMC_BAD_VALUE",
-	8:  "OFPBMC_BAD_MASK",
-	9:  "OFPBMC_BAD_PREREQ",
-	10: "OFPBMC_DUP_FIELD",
-	11: "OFPBMC_EPERM",
-}
-
-var OfpBadMatchCode_value = map[string]int32{
-	"OFPBMC_BAD_TYPE":         0,
-	"OFPBMC_BAD_LEN":          1,
-	"OFPBMC_BAD_TAG":          2,
-	"OFPBMC_BAD_DL_ADDR_MASK": 3,
-	"OFPBMC_BAD_NW_ADDR_MASK": 4,
-	"OFPBMC_BAD_WILDCARDS":    5,
-	"OFPBMC_BAD_FIELD":        6,
-	"OFPBMC_BAD_VALUE":        7,
-	"OFPBMC_BAD_MASK":         8,
-	"OFPBMC_BAD_PREREQ":       9,
-	"OFPBMC_DUP_FIELD":        10,
-	"OFPBMC_EPERM":            11,
-}
-
-func (x OfpBadMatchCode) String() string {
-	return proto.EnumName(OfpBadMatchCode_name, int32(x))
-}
-
-func (OfpBadMatchCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{35}
-}
-
-// ofp_error_msg 'code' values for OFPET_FLOW_MOD_FAILED.  'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpFlowModFailedCode int32
-
-const (
-	OfpFlowModFailedCode_OFPFMFC_UNKNOWN      OfpFlowModFailedCode = 0
-	OfpFlowModFailedCode_OFPFMFC_TABLE_FULL   OfpFlowModFailedCode = 1
-	OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID OfpFlowModFailedCode = 2
-	OfpFlowModFailedCode_OFPFMFC_OVERLAP      OfpFlowModFailedCode = 3
-	OfpFlowModFailedCode_OFPFMFC_EPERM        OfpFlowModFailedCode = 4
-	OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT  OfpFlowModFailedCode = 5
-	OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND  OfpFlowModFailedCode = 6
-	OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS    OfpFlowModFailedCode = 7
-)
-
-var OfpFlowModFailedCode_name = map[int32]string{
-	0: "OFPFMFC_UNKNOWN",
-	1: "OFPFMFC_TABLE_FULL",
-	2: "OFPFMFC_BAD_TABLE_ID",
-	3: "OFPFMFC_OVERLAP",
-	4: "OFPFMFC_EPERM",
-	5: "OFPFMFC_BAD_TIMEOUT",
-	6: "OFPFMFC_BAD_COMMAND",
-	7: "OFPFMFC_BAD_FLAGS",
-}
-
-var OfpFlowModFailedCode_value = map[string]int32{
-	"OFPFMFC_UNKNOWN":      0,
-	"OFPFMFC_TABLE_FULL":   1,
-	"OFPFMFC_BAD_TABLE_ID": 2,
-	"OFPFMFC_OVERLAP":      3,
-	"OFPFMFC_EPERM":        4,
-	"OFPFMFC_BAD_TIMEOUT":  5,
-	"OFPFMFC_BAD_COMMAND":  6,
-	"OFPFMFC_BAD_FLAGS":    7,
-}
-
-func (x OfpFlowModFailedCode) String() string {
-	return proto.EnumName(OfpFlowModFailedCode_name, int32(x))
-}
-
-func (OfpFlowModFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{36}
-}
-
-// ofp_error_msg 'code' values for OFPET_GROUP_MOD_FAILED.  'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpGroupModFailedCode int32
-
-const (
-	OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS         OfpGroupModFailedCode = 0
-	OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP        OfpGroupModFailedCode = 1
-	OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED   OfpGroupModFailedCode = 2
-	OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS        OfpGroupModFailedCode = 3
-	OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS       OfpGroupModFailedCode = 4
-	OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED OfpGroupModFailedCode = 5
-	OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED    OfpGroupModFailedCode = 6
-	OfpGroupModFailedCode_OFPGMFC_LOOP                 OfpGroupModFailedCode = 7
-	OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP        OfpGroupModFailedCode = 8
-	OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP        OfpGroupModFailedCode = 9
-	OfpGroupModFailedCode_OFPGMFC_BAD_TYPE             OfpGroupModFailedCode = 10
-	OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND          OfpGroupModFailedCode = 11
-	OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET           OfpGroupModFailedCode = 12
-	OfpGroupModFailedCode_OFPGMFC_BAD_WATCH            OfpGroupModFailedCode = 13
-	OfpGroupModFailedCode_OFPGMFC_EPERM                OfpGroupModFailedCode = 14
-)
-
-var OfpGroupModFailedCode_name = map[int32]string{
-	0:  "OFPGMFC_GROUP_EXISTS",
-	1:  "OFPGMFC_INVALID_GROUP",
-	2:  "OFPGMFC_WEIGHT_UNSUPPORTED",
-	3:  "OFPGMFC_OUT_OF_GROUPS",
-	4:  "OFPGMFC_OUT_OF_BUCKETS",
-	5:  "OFPGMFC_CHAINING_UNSUPPORTED",
-	6:  "OFPGMFC_WATCH_UNSUPPORTED",
-	7:  "OFPGMFC_LOOP",
-	8:  "OFPGMFC_UNKNOWN_GROUP",
-	9:  "OFPGMFC_CHAINED_GROUP",
-	10: "OFPGMFC_BAD_TYPE",
-	11: "OFPGMFC_BAD_COMMAND",
-	12: "OFPGMFC_BAD_BUCKET",
-	13: "OFPGMFC_BAD_WATCH",
-	14: "OFPGMFC_EPERM",
-}
-
-var OfpGroupModFailedCode_value = map[string]int32{
-	"OFPGMFC_GROUP_EXISTS":         0,
-	"OFPGMFC_INVALID_GROUP":        1,
-	"OFPGMFC_WEIGHT_UNSUPPORTED":   2,
-	"OFPGMFC_OUT_OF_GROUPS":        3,
-	"OFPGMFC_OUT_OF_BUCKETS":       4,
-	"OFPGMFC_CHAINING_UNSUPPORTED": 5,
-	"OFPGMFC_WATCH_UNSUPPORTED":    6,
-	"OFPGMFC_LOOP":                 7,
-	"OFPGMFC_UNKNOWN_GROUP":        8,
-	"OFPGMFC_CHAINED_GROUP":        9,
-	"OFPGMFC_BAD_TYPE":             10,
-	"OFPGMFC_BAD_COMMAND":          11,
-	"OFPGMFC_BAD_BUCKET":           12,
-	"OFPGMFC_BAD_WATCH":            13,
-	"OFPGMFC_EPERM":                14,
-}
-
-func (x OfpGroupModFailedCode) String() string {
-	return proto.EnumName(OfpGroupModFailedCode_name, int32(x))
-}
-
-func (OfpGroupModFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{37}
-}
-
-// ofp_error_msg 'code' values for OFPET_PORT_MOD_FAILED.  'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpPortModFailedCode int32
-
-const (
-	OfpPortModFailedCode_OFPPMFC_BAD_PORT      OfpPortModFailedCode = 0
-	OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR   OfpPortModFailedCode = 1
-	OfpPortModFailedCode_OFPPMFC_BAD_CONFIG    OfpPortModFailedCode = 2
-	OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE OfpPortModFailedCode = 3
-	OfpPortModFailedCode_OFPPMFC_EPERM         OfpPortModFailedCode = 4
-)
-
-var OfpPortModFailedCode_name = map[int32]string{
-	0: "OFPPMFC_BAD_PORT",
-	1: "OFPPMFC_BAD_HW_ADDR",
-	2: "OFPPMFC_BAD_CONFIG",
-	3: "OFPPMFC_BAD_ADVERTISE",
-	4: "OFPPMFC_EPERM",
-}
-
-var OfpPortModFailedCode_value = map[string]int32{
-	"OFPPMFC_BAD_PORT":      0,
-	"OFPPMFC_BAD_HW_ADDR":   1,
-	"OFPPMFC_BAD_CONFIG":    2,
-	"OFPPMFC_BAD_ADVERTISE": 3,
-	"OFPPMFC_EPERM":         4,
-}
-
-func (x OfpPortModFailedCode) String() string {
-	return proto.EnumName(OfpPortModFailedCode_name, int32(x))
-}
-
-func (OfpPortModFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{38}
-}
-
-// ofp_error_msg 'code' values for OFPET_TABLE_MOD_FAILED.  'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpTableModFailedCode int32
-
-const (
-	OfpTableModFailedCode_OFPTMFC_BAD_TABLE  OfpTableModFailedCode = 0
-	OfpTableModFailedCode_OFPTMFC_BAD_CONFIG OfpTableModFailedCode = 1
-	OfpTableModFailedCode_OFPTMFC_EPERM      OfpTableModFailedCode = 2
-)
-
-var OfpTableModFailedCode_name = map[int32]string{
-	0: "OFPTMFC_BAD_TABLE",
-	1: "OFPTMFC_BAD_CONFIG",
-	2: "OFPTMFC_EPERM",
-}
-
-var OfpTableModFailedCode_value = map[string]int32{
-	"OFPTMFC_BAD_TABLE":  0,
-	"OFPTMFC_BAD_CONFIG": 1,
-	"OFPTMFC_EPERM":      2,
-}
-
-func (x OfpTableModFailedCode) String() string {
-	return proto.EnumName(OfpTableModFailedCode_name, int32(x))
-}
-
-func (OfpTableModFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{39}
-}
-
-// ofp_error msg 'code' values for OFPET_QUEUE_OP_FAILED. 'data' contains
-// at least the first 64 bytes of the failed request
-type OfpQueueOpFailedCode int32
-
-const (
-	OfpQueueOpFailedCode_OFPQOFC_BAD_PORT  OfpQueueOpFailedCode = 0
-	OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE OfpQueueOpFailedCode = 1
-	OfpQueueOpFailedCode_OFPQOFC_EPERM     OfpQueueOpFailedCode = 2
-)
-
-var OfpQueueOpFailedCode_name = map[int32]string{
-	0: "OFPQOFC_BAD_PORT",
-	1: "OFPQOFC_BAD_QUEUE",
-	2: "OFPQOFC_EPERM",
-}
-
-var OfpQueueOpFailedCode_value = map[string]int32{
-	"OFPQOFC_BAD_PORT":  0,
-	"OFPQOFC_BAD_QUEUE": 1,
-	"OFPQOFC_EPERM":     2,
-}
-
-func (x OfpQueueOpFailedCode) String() string {
-	return proto.EnumName(OfpQueueOpFailedCode_name, int32(x))
-}
-
-func (OfpQueueOpFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{40}
-}
-
-// ofp_error_msg 'code' values for OFPET_SWITCH_CONFIG_FAILED. 'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpSwitchConfigFailedCode int32
-
-const (
-	OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS OfpSwitchConfigFailedCode = 0
-	OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN   OfpSwitchConfigFailedCode = 1
-	OfpSwitchConfigFailedCode_OFPSCFC_EPERM     OfpSwitchConfigFailedCode = 2
-)
-
-var OfpSwitchConfigFailedCode_name = map[int32]string{
-	0: "OFPSCFC_BAD_FLAGS",
-	1: "OFPSCFC_BAD_LEN",
-	2: "OFPSCFC_EPERM",
-}
-
-var OfpSwitchConfigFailedCode_value = map[string]int32{
-	"OFPSCFC_BAD_FLAGS": 0,
-	"OFPSCFC_BAD_LEN":   1,
-	"OFPSCFC_EPERM":     2,
-}
-
-func (x OfpSwitchConfigFailedCode) String() string {
-	return proto.EnumName(OfpSwitchConfigFailedCode_name, int32(x))
-}
-
-func (OfpSwitchConfigFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{41}
-}
-
-// ofp_error_msg 'code' values for OFPET_ROLE_REQUEST_FAILED. 'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpRoleRequestFailedCode int32
-
-const (
-	OfpRoleRequestFailedCode_OFPRRFC_STALE    OfpRoleRequestFailedCode = 0
-	OfpRoleRequestFailedCode_OFPRRFC_UNSUP    OfpRoleRequestFailedCode = 1
-	OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE OfpRoleRequestFailedCode = 2
-)
-
-var OfpRoleRequestFailedCode_name = map[int32]string{
-	0: "OFPRRFC_STALE",
-	1: "OFPRRFC_UNSUP",
-	2: "OFPRRFC_BAD_ROLE",
-}
-
-var OfpRoleRequestFailedCode_value = map[string]int32{
-	"OFPRRFC_STALE":    0,
-	"OFPRRFC_UNSUP":    1,
-	"OFPRRFC_BAD_ROLE": 2,
-}
-
-func (x OfpRoleRequestFailedCode) String() string {
-	return proto.EnumName(OfpRoleRequestFailedCode_name, int32(x))
-}
-
-func (OfpRoleRequestFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{42}
-}
-
-// ofp_error_msg 'code' values for OFPET_METER_MOD_FAILED.  'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpMeterModFailedCode int32
-
-const (
-	OfpMeterModFailedCode_OFPMMFC_UNKNOWN         OfpMeterModFailedCode = 0
-	OfpMeterModFailedCode_OFPMMFC_METER_EXISTS    OfpMeterModFailedCode = 1
-	OfpMeterModFailedCode_OFPMMFC_INVALID_METER   OfpMeterModFailedCode = 2
-	OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER   OfpMeterModFailedCode = 3
-	OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND     OfpMeterModFailedCode = 4
-	OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS       OfpMeterModFailedCode = 5
-	OfpMeterModFailedCode_OFPMMFC_BAD_RATE        OfpMeterModFailedCode = 6
-	OfpMeterModFailedCode_OFPMMFC_BAD_BURST       OfpMeterModFailedCode = 7
-	OfpMeterModFailedCode_OFPMMFC_BAD_BAND        OfpMeterModFailedCode = 8
-	OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL OfpMeterModFailedCode = 9
-	OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS   OfpMeterModFailedCode = 10
-	OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS    OfpMeterModFailedCode = 11
-)
-
-var OfpMeterModFailedCode_name = map[int32]string{
-	0:  "OFPMMFC_UNKNOWN",
-	1:  "OFPMMFC_METER_EXISTS",
-	2:  "OFPMMFC_INVALID_METER",
-	3:  "OFPMMFC_UNKNOWN_METER",
-	4:  "OFPMMFC_BAD_COMMAND",
-	5:  "OFPMMFC_BAD_FLAGS",
-	6:  "OFPMMFC_BAD_RATE",
-	7:  "OFPMMFC_BAD_BURST",
-	8:  "OFPMMFC_BAD_BAND",
-	9:  "OFPMMFC_BAD_BAND_DETAIL",
-	10: "OFPMMFC_OUT_OF_METERS",
-	11: "OFPMMFC_OUT_OF_BANDS",
-}
-
-var OfpMeterModFailedCode_value = map[string]int32{
-	"OFPMMFC_UNKNOWN":         0,
-	"OFPMMFC_METER_EXISTS":    1,
-	"OFPMMFC_INVALID_METER":   2,
-	"OFPMMFC_UNKNOWN_METER":   3,
-	"OFPMMFC_BAD_COMMAND":     4,
-	"OFPMMFC_BAD_FLAGS":       5,
-	"OFPMMFC_BAD_RATE":        6,
-	"OFPMMFC_BAD_BURST":       7,
-	"OFPMMFC_BAD_BAND":        8,
-	"OFPMMFC_BAD_BAND_DETAIL": 9,
-	"OFPMMFC_OUT_OF_METERS":   10,
-	"OFPMMFC_OUT_OF_BANDS":    11,
-}
-
-func (x OfpMeterModFailedCode) String() string {
-	return proto.EnumName(OfpMeterModFailedCode_name, int32(x))
-}
-
-func (OfpMeterModFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{43}
-}
-
-// ofp_error_msg 'code' values for OFPET_TABLE_FEATURES_FAILED. 'data' contains
-// at least the first 64 bytes of the failed request.
-type OfpTableFeaturesFailedCode int32
-
-const (
-	OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE    OfpTableFeaturesFailedCode = 0
-	OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA OfpTableFeaturesFailedCode = 1
-	OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE     OfpTableFeaturesFailedCode = 2
-	OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN      OfpTableFeaturesFailedCode = 3
-	OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT OfpTableFeaturesFailedCode = 4
-	OfpTableFeaturesFailedCode_OFPTFFC_EPERM        OfpTableFeaturesFailedCode = 5
-)
-
-var OfpTableFeaturesFailedCode_name = map[int32]string{
-	0: "OFPTFFC_BAD_TABLE",
-	1: "OFPTFFC_BAD_METADATA",
-	2: "OFPTFFC_BAD_TYPE",
-	3: "OFPTFFC_BAD_LEN",
-	4: "OFPTFFC_BAD_ARGUMENT",
-	5: "OFPTFFC_EPERM",
-}
-
-var OfpTableFeaturesFailedCode_value = map[string]int32{
-	"OFPTFFC_BAD_TABLE":    0,
-	"OFPTFFC_BAD_METADATA": 1,
-	"OFPTFFC_BAD_TYPE":     2,
-	"OFPTFFC_BAD_LEN":      3,
-	"OFPTFFC_BAD_ARGUMENT": 4,
-	"OFPTFFC_EPERM":        5,
-}
-
-func (x OfpTableFeaturesFailedCode) String() string {
-	return proto.EnumName(OfpTableFeaturesFailedCode_name, int32(x))
-}
-
-func (OfpTableFeaturesFailedCode) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{44}
-}
-
-type OfpMultipartType int32
-
-const (
-	// Description of this OpenFlow switch.
-	// The request body is empty.
-	// The reply body is struct ofp_desc.
-	OfpMultipartType_OFPMP_DESC OfpMultipartType = 0
-	// Individual flow statistics.
-	// The request body is struct ofp_flow_stats_request.
-	// The reply body is an array of struct ofp_flow_stats.
-	OfpMultipartType_OFPMP_FLOW OfpMultipartType = 1
-	// Aggregate flow statistics.
-	// The request body is struct ofp_aggregate_stats_request.
-	// The reply body is struct ofp_aggregate_stats_reply.
-	OfpMultipartType_OFPMP_AGGREGATE OfpMultipartType = 2
-	// Flow table statistics.
-	// The request body is empty.
-	// The reply body is an array of struct ofp_table_stats.
-	OfpMultipartType_OFPMP_TABLE OfpMultipartType = 3
-	// Port statistics.
-	// The request body is struct ofp_port_stats_request.
-	// The reply body is an array of struct ofp_port_stats.
-	OfpMultipartType_OFPMP_PORT_STATS OfpMultipartType = 4
-	// Queue statistics for a port
-	// The request body is struct ofp_queue_stats_request.
-	// The reply body is an array of struct ofp_queue_stats
-	OfpMultipartType_OFPMP_QUEUE OfpMultipartType = 5
-	// Group counter statistics.
-	// The request body is struct ofp_group_stats_request.
-	// The reply is an array of struct ofp_group_stats.
-	OfpMultipartType_OFPMP_GROUP OfpMultipartType = 6
-	// Group description.
-	// The request body is empty.
-	// The reply body is an array of struct ofp_group_desc.
-	OfpMultipartType_OFPMP_GROUP_DESC OfpMultipartType = 7
-	// Group features.
-	// The request body is empty.
-	// The reply body is struct ofp_group_features.
-	OfpMultipartType_OFPMP_GROUP_FEATURES OfpMultipartType = 8
-	// Meter statistics.
-	// The request body is struct ofp_meter_multipart_requests.
-	// The reply body is an array of struct ofp_meter_stats.
-	OfpMultipartType_OFPMP_METER OfpMultipartType = 9
-	// Meter configuration.
-	// The request body is struct ofp_meter_multipart_requests.
-	// The reply body is an array of struct ofp_meter_config.
-	OfpMultipartType_OFPMP_METER_CONFIG OfpMultipartType = 10
-	// Meter features.
-	// The request body is empty.
-	// The reply body is struct ofp_meter_features.
-	OfpMultipartType_OFPMP_METER_FEATURES OfpMultipartType = 11
-	// Table features.
-	// The request body is either empty or contains an array of
-	// struct ofp_table_features containing the controller's
-	// desired view of the switch. If the switch is unable to
-	// set the specified view an error is returned.
-	// The reply body is an array of struct ofp_table_features.
-	OfpMultipartType_OFPMP_TABLE_FEATURES OfpMultipartType = 12
-	// Port description.
-	// The request body is empty.
-	// The reply body is an array of struct ofp_port.
-	OfpMultipartType_OFPMP_PORT_DESC OfpMultipartType = 13
-	// Experimenter extension.
-	// The request and reply bodies begin with
-	// struct ofp_experimenter_multipart_header.
-	// The request and reply bodies are otherwise experimenter-defined.
-	OfpMultipartType_OFPMP_EXPERIMENTER OfpMultipartType = 65535
-)
-
-var OfpMultipartType_name = map[int32]string{
-	0:     "OFPMP_DESC",
-	1:     "OFPMP_FLOW",
-	2:     "OFPMP_AGGREGATE",
-	3:     "OFPMP_TABLE",
-	4:     "OFPMP_PORT_STATS",
-	5:     "OFPMP_QUEUE",
-	6:     "OFPMP_GROUP",
-	7:     "OFPMP_GROUP_DESC",
-	8:     "OFPMP_GROUP_FEATURES",
-	9:     "OFPMP_METER",
-	10:    "OFPMP_METER_CONFIG",
-	11:    "OFPMP_METER_FEATURES",
-	12:    "OFPMP_TABLE_FEATURES",
-	13:    "OFPMP_PORT_DESC",
-	65535: "OFPMP_EXPERIMENTER",
-}
-
-var OfpMultipartType_value = map[string]int32{
-	"OFPMP_DESC":           0,
-	"OFPMP_FLOW":           1,
-	"OFPMP_AGGREGATE":      2,
-	"OFPMP_TABLE":          3,
-	"OFPMP_PORT_STATS":     4,
-	"OFPMP_QUEUE":          5,
-	"OFPMP_GROUP":          6,
-	"OFPMP_GROUP_DESC":     7,
-	"OFPMP_GROUP_FEATURES": 8,
-	"OFPMP_METER":          9,
-	"OFPMP_METER_CONFIG":   10,
-	"OFPMP_METER_FEATURES": 11,
-	"OFPMP_TABLE_FEATURES": 12,
-	"OFPMP_PORT_DESC":      13,
-	"OFPMP_EXPERIMENTER":   65535,
-}
-
-func (x OfpMultipartType) String() string {
-	return proto.EnumName(OfpMultipartType_name, int32(x))
-}
-
-func (OfpMultipartType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{45}
-}
-
-type OfpMultipartRequestFlags int32
-
-const (
-	OfpMultipartRequestFlags_OFPMPF_REQ_INVALID OfpMultipartRequestFlags = 0
-	OfpMultipartRequestFlags_OFPMPF_REQ_MORE    OfpMultipartRequestFlags = 1
-)
-
-var OfpMultipartRequestFlags_name = map[int32]string{
-	0: "OFPMPF_REQ_INVALID",
-	1: "OFPMPF_REQ_MORE",
-}
-
-var OfpMultipartRequestFlags_value = map[string]int32{
-	"OFPMPF_REQ_INVALID": 0,
-	"OFPMPF_REQ_MORE":    1,
-}
-
-func (x OfpMultipartRequestFlags) String() string {
-	return proto.EnumName(OfpMultipartRequestFlags_name, int32(x))
-}
-
-func (OfpMultipartRequestFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{46}
-}
-
-type OfpMultipartReplyFlags int32
-
-const (
-	OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID OfpMultipartReplyFlags = 0
-	OfpMultipartReplyFlags_OFPMPF_REPLY_MORE    OfpMultipartReplyFlags = 1
-)
-
-var OfpMultipartReplyFlags_name = map[int32]string{
-	0: "OFPMPF_REPLY_INVALID",
-	1: "OFPMPF_REPLY_MORE",
-}
-
-var OfpMultipartReplyFlags_value = map[string]int32{
-	"OFPMPF_REPLY_INVALID": 0,
-	"OFPMPF_REPLY_MORE":    1,
-}
-
-func (x OfpMultipartReplyFlags) String() string {
-	return proto.EnumName(OfpMultipartReplyFlags_name, int32(x))
-}
-
-func (OfpMultipartReplyFlags) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{47}
-}
-
-// Table Feature property types.
-// Low order bit cleared indicates a property for a regular Flow Entry.
-// Low order bit set indicates a property for the Table-Miss Flow Entry.
-type OfpTableFeaturePropType int32
-
-const (
-	OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS        OfpTableFeaturePropType = 0
-	OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS   OfpTableFeaturePropType = 1
-	OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES         OfpTableFeaturePropType = 2
-	OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS    OfpTableFeaturePropType = 3
-	OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS       OfpTableFeaturePropType = 4
-	OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS  OfpTableFeaturePropType = 5
-	OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS       OfpTableFeaturePropType = 6
-	OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS  OfpTableFeaturePropType = 7
-	OfpTableFeaturePropType_OFPTFPT_MATCH               OfpTableFeaturePropType = 8
-	OfpTableFeaturePropType_OFPTFPT_WILDCARDS           OfpTableFeaturePropType = 10
-	OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD      OfpTableFeaturePropType = 12
-	OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS OfpTableFeaturePropType = 13
-	OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD      OfpTableFeaturePropType = 14
-	OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS OfpTableFeaturePropType = 15
-	OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER        OfpTableFeaturePropType = 65534
-	OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS   OfpTableFeaturePropType = 65535
-)
-
-var OfpTableFeaturePropType_name = map[int32]string{
-	0:     "OFPTFPT_INSTRUCTIONS",
-	1:     "OFPTFPT_INSTRUCTIONS_MISS",
-	2:     "OFPTFPT_NEXT_TABLES",
-	3:     "OFPTFPT_NEXT_TABLES_MISS",
-	4:     "OFPTFPT_WRITE_ACTIONS",
-	5:     "OFPTFPT_WRITE_ACTIONS_MISS",
-	6:     "OFPTFPT_APPLY_ACTIONS",
-	7:     "OFPTFPT_APPLY_ACTIONS_MISS",
-	8:     "OFPTFPT_MATCH",
-	10:    "OFPTFPT_WILDCARDS",
-	12:    "OFPTFPT_WRITE_SETFIELD",
-	13:    "OFPTFPT_WRITE_SETFIELD_MISS",
-	14:    "OFPTFPT_APPLY_SETFIELD",
-	15:    "OFPTFPT_APPLY_SETFIELD_MISS",
-	65534: "OFPTFPT_EXPERIMENTER",
-	65535: "OFPTFPT_EXPERIMENTER_MISS",
-}
-
-var OfpTableFeaturePropType_value = map[string]int32{
-	"OFPTFPT_INSTRUCTIONS":        0,
-	"OFPTFPT_INSTRUCTIONS_MISS":   1,
-	"OFPTFPT_NEXT_TABLES":         2,
-	"OFPTFPT_NEXT_TABLES_MISS":    3,
-	"OFPTFPT_WRITE_ACTIONS":       4,
-	"OFPTFPT_WRITE_ACTIONS_MISS":  5,
-	"OFPTFPT_APPLY_ACTIONS":       6,
-	"OFPTFPT_APPLY_ACTIONS_MISS":  7,
-	"OFPTFPT_MATCH":               8,
-	"OFPTFPT_WILDCARDS":           10,
-	"OFPTFPT_WRITE_SETFIELD":      12,
-	"OFPTFPT_WRITE_SETFIELD_MISS": 13,
-	"OFPTFPT_APPLY_SETFIELD":      14,
-	"OFPTFPT_APPLY_SETFIELD_MISS": 15,
-	"OFPTFPT_EXPERIMENTER":        65534,
-	"OFPTFPT_EXPERIMENTER_MISS":   65535,
-}
-
-func (x OfpTableFeaturePropType) String() string {
-	return proto.EnumName(OfpTableFeaturePropType_name, int32(x))
-}
-
-func (OfpTableFeaturePropType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{48}
-}
-
-// Group configuration flags
-type OfpGroupCapabilities int32
-
-const (
-	OfpGroupCapabilities_OFPGFC_INVALID         OfpGroupCapabilities = 0
-	OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT   OfpGroupCapabilities = 1
-	OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS OfpGroupCapabilities = 2
-	OfpGroupCapabilities_OFPGFC_CHAINING        OfpGroupCapabilities = 4
-	OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS OfpGroupCapabilities = 8
-)
-
-var OfpGroupCapabilities_name = map[int32]string{
-	0: "OFPGFC_INVALID",
-	1: "OFPGFC_SELECT_WEIGHT",
-	2: "OFPGFC_SELECT_LIVENESS",
-	4: "OFPGFC_CHAINING",
-	8: "OFPGFC_CHAINING_CHECKS",
-}
-
-var OfpGroupCapabilities_value = map[string]int32{
-	"OFPGFC_INVALID":         0,
-	"OFPGFC_SELECT_WEIGHT":   1,
-	"OFPGFC_SELECT_LIVENESS": 2,
-	"OFPGFC_CHAINING":        4,
-	"OFPGFC_CHAINING_CHECKS": 8,
-}
-
-func (x OfpGroupCapabilities) String() string {
-	return proto.EnumName(OfpGroupCapabilities_name, int32(x))
-}
-
-func (OfpGroupCapabilities) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{49}
-}
-
-type OfpQueueProperties int32
-
-const (
-	OfpQueueProperties_OFPQT_INVALID      OfpQueueProperties = 0
-	OfpQueueProperties_OFPQT_MIN_RATE     OfpQueueProperties = 1
-	OfpQueueProperties_OFPQT_MAX_RATE     OfpQueueProperties = 2
-	OfpQueueProperties_OFPQT_EXPERIMENTER OfpQueueProperties = 65535
-)
-
-var OfpQueueProperties_name = map[int32]string{
-	0:     "OFPQT_INVALID",
-	1:     "OFPQT_MIN_RATE",
-	2:     "OFPQT_MAX_RATE",
-	65535: "OFPQT_EXPERIMENTER",
-}
-
-var OfpQueueProperties_value = map[string]int32{
-	"OFPQT_INVALID":      0,
-	"OFPQT_MIN_RATE":     1,
-	"OFPQT_MAX_RATE":     2,
-	"OFPQT_EXPERIMENTER": 65535,
-}
-
-func (x OfpQueueProperties) String() string {
-	return proto.EnumName(OfpQueueProperties_name, int32(x))
-}
-
-func (OfpQueueProperties) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{50}
-}
-
-// Controller roles.
-type OfpControllerRole int32
-
-const (
-	OfpControllerRole_OFPCR_ROLE_NOCHANGE OfpControllerRole = 0
-	OfpControllerRole_OFPCR_ROLE_EQUAL    OfpControllerRole = 1
-	OfpControllerRole_OFPCR_ROLE_MASTER   OfpControllerRole = 2
-	OfpControllerRole_OFPCR_ROLE_SLAVE    OfpControllerRole = 3
-)
-
-var OfpControllerRole_name = map[int32]string{
-	0: "OFPCR_ROLE_NOCHANGE",
-	1: "OFPCR_ROLE_EQUAL",
-	2: "OFPCR_ROLE_MASTER",
-	3: "OFPCR_ROLE_SLAVE",
-}
-
-var OfpControllerRole_value = map[string]int32{
-	"OFPCR_ROLE_NOCHANGE": 0,
-	"OFPCR_ROLE_EQUAL":    1,
-	"OFPCR_ROLE_MASTER":   2,
-	"OFPCR_ROLE_SLAVE":    3,
-}
-
-func (x OfpControllerRole) String() string {
-	return proto.EnumName(OfpControllerRole_name, int32(x))
-}
-
-func (OfpControllerRole) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{51}
-}
-
-// Header on all OpenFlow packets.
-type OfpHeader struct {
-	Version              uint32   `protobuf:"varint,1,opt,name=version,proto3" json:"version,omitempty"`
-	Type                 OfpType  `protobuf:"varint,2,opt,name=type,proto3,enum=openflow_13.OfpType" json:"type,omitempty"`
-	Xid                  uint32   `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpHeader) Reset()         { *m = OfpHeader{} }
-func (m *OfpHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpHeader) ProtoMessage()    {}
-func (*OfpHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{0}
-}
-
-func (m *OfpHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpHeader.Unmarshal(m, b)
-}
-func (m *OfpHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpHeader.Merge(m, src)
-}
-func (m *OfpHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpHeader.Size(m)
-}
-func (m *OfpHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpHeader proto.InternalMessageInfo
-
-func (m *OfpHeader) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *OfpHeader) GetType() OfpType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpType_OFPT_HELLO
-}
-
-func (m *OfpHeader) GetXid() uint32 {
-	if m != nil {
-		return m.Xid
-	}
-	return 0
-}
-
-// Common header for all Hello Elements
-type OfpHelloElemHeader struct {
-	Type OfpHelloElemType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpHelloElemType" json:"type,omitempty"`
-	// Types that are valid to be assigned to Element:
-	//	*OfpHelloElemHeader_Versionbitmap
-	Element              isOfpHelloElemHeader_Element `protobuf_oneof:"element"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
-}
-
-func (m *OfpHelloElemHeader) Reset()         { *m = OfpHelloElemHeader{} }
-func (m *OfpHelloElemHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpHelloElemHeader) ProtoMessage()    {}
-func (*OfpHelloElemHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{1}
-}
-
-func (m *OfpHelloElemHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpHelloElemHeader.Unmarshal(m, b)
-}
-func (m *OfpHelloElemHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpHelloElemHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpHelloElemHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpHelloElemHeader.Merge(m, src)
-}
-func (m *OfpHelloElemHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpHelloElemHeader.Size(m)
-}
-func (m *OfpHelloElemHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpHelloElemHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpHelloElemHeader proto.InternalMessageInfo
-
-func (m *OfpHelloElemHeader) GetType() OfpHelloElemType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpHelloElemType_OFPHET_INVALID
-}
-
-type isOfpHelloElemHeader_Element interface {
-	isOfpHelloElemHeader_Element()
-}
-
-type OfpHelloElemHeader_Versionbitmap struct {
-	Versionbitmap *OfpHelloElemVersionbitmap `protobuf:"bytes,2,opt,name=versionbitmap,proto3,oneof"`
-}
-
-func (*OfpHelloElemHeader_Versionbitmap) isOfpHelloElemHeader_Element() {}
-
-func (m *OfpHelloElemHeader) GetElement() isOfpHelloElemHeader_Element {
-	if m != nil {
-		return m.Element
-	}
-	return nil
-}
-
-func (m *OfpHelloElemHeader) GetVersionbitmap() *OfpHelloElemVersionbitmap {
-	if x, ok := m.GetElement().(*OfpHelloElemHeader_Versionbitmap); ok {
-		return x.Versionbitmap
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpHelloElemHeader) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpHelloElemHeader_Versionbitmap)(nil),
-	}
-}
-
-// Version bitmap Hello Element
-type OfpHelloElemVersionbitmap struct {
-	Bitmaps              []uint32 `protobuf:"varint,2,rep,packed,name=bitmaps,proto3" json:"bitmaps,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpHelloElemVersionbitmap) Reset()         { *m = OfpHelloElemVersionbitmap{} }
-func (m *OfpHelloElemVersionbitmap) String() string { return proto.CompactTextString(m) }
-func (*OfpHelloElemVersionbitmap) ProtoMessage()    {}
-func (*OfpHelloElemVersionbitmap) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{2}
-}
-
-func (m *OfpHelloElemVersionbitmap) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpHelloElemVersionbitmap.Unmarshal(m, b)
-}
-func (m *OfpHelloElemVersionbitmap) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpHelloElemVersionbitmap.Marshal(b, m, deterministic)
-}
-func (m *OfpHelloElemVersionbitmap) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpHelloElemVersionbitmap.Merge(m, src)
-}
-func (m *OfpHelloElemVersionbitmap) XXX_Size() int {
-	return xxx_messageInfo_OfpHelloElemVersionbitmap.Size(m)
-}
-func (m *OfpHelloElemVersionbitmap) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpHelloElemVersionbitmap.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpHelloElemVersionbitmap proto.InternalMessageInfo
-
-func (m *OfpHelloElemVersionbitmap) GetBitmaps() []uint32 {
-	if m != nil {
-		return m.Bitmaps
-	}
-	return nil
-}
-
-// OFPT_HELLO.  This message includes zero or more hello elements having
-// variable size. Unknown elements types must be ignored/skipped, to allow
-// for future extensions.
-type OfpHello struct {
-	// Hello element list
-	Elements             []*OfpHelloElemHeader `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpHello) Reset()         { *m = OfpHello{} }
-func (m *OfpHello) String() string { return proto.CompactTextString(m) }
-func (*OfpHello) ProtoMessage()    {}
-func (*OfpHello) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{3}
-}
-
-func (m *OfpHello) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpHello.Unmarshal(m, b)
-}
-func (m *OfpHello) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpHello.Marshal(b, m, deterministic)
-}
-func (m *OfpHello) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpHello.Merge(m, src)
-}
-func (m *OfpHello) XXX_Size() int {
-	return xxx_messageInfo_OfpHello.Size(m)
-}
-func (m *OfpHello) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpHello.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpHello proto.InternalMessageInfo
-
-func (m *OfpHello) GetElements() []*OfpHelloElemHeader {
-	if m != nil {
-		return m.Elements
-	}
-	return nil
-}
-
-// Switch configuration.
-type OfpSwitchConfig struct {
-	//ofp_header header;
-	Flags                uint32   `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"`
-	MissSendLen          uint32   `protobuf:"varint,2,opt,name=miss_send_len,json=missSendLen,proto3" json:"miss_send_len,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpSwitchConfig) Reset()         { *m = OfpSwitchConfig{} }
-func (m *OfpSwitchConfig) String() string { return proto.CompactTextString(m) }
-func (*OfpSwitchConfig) ProtoMessage()    {}
-func (*OfpSwitchConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{4}
-}
-
-func (m *OfpSwitchConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpSwitchConfig.Unmarshal(m, b)
-}
-func (m *OfpSwitchConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpSwitchConfig.Marshal(b, m, deterministic)
-}
-func (m *OfpSwitchConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpSwitchConfig.Merge(m, src)
-}
-func (m *OfpSwitchConfig) XXX_Size() int {
-	return xxx_messageInfo_OfpSwitchConfig.Size(m)
-}
-func (m *OfpSwitchConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpSwitchConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpSwitchConfig proto.InternalMessageInfo
-
-func (m *OfpSwitchConfig) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpSwitchConfig) GetMissSendLen() uint32 {
-	if m != nil {
-		return m.MissSendLen
-	}
-	return 0
-}
-
-// Configure/Modify behavior of a flow table
-type OfpTableMod struct {
-	//ofp_header header;
-	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	Config               uint32   `protobuf:"varint,2,opt,name=config,proto3" json:"config,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpTableMod) Reset()         { *m = OfpTableMod{} }
-func (m *OfpTableMod) String() string { return proto.CompactTextString(m) }
-func (*OfpTableMod) ProtoMessage()    {}
-func (*OfpTableMod) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{5}
-}
-
-func (m *OfpTableMod) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableMod.Unmarshal(m, b)
-}
-func (m *OfpTableMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableMod.Marshal(b, m, deterministic)
-}
-func (m *OfpTableMod) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableMod.Merge(m, src)
-}
-func (m *OfpTableMod) XXX_Size() int {
-	return xxx_messageInfo_OfpTableMod.Size(m)
-}
-func (m *OfpTableMod) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableMod.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableMod proto.InternalMessageInfo
-
-func (m *OfpTableMod) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpTableMod) GetConfig() uint32 {
-	if m != nil {
-		return m.Config
-	}
-	return 0
-}
-
-// Description of a port
-type OfpPort struct {
-	PortNo uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	HwAddr []uint32 `protobuf:"varint,2,rep,packed,name=hw_addr,json=hwAddr,proto3" json:"hw_addr,omitempty"`
-	Name   string   `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"`
-	Config uint32   `protobuf:"varint,4,opt,name=config,proto3" json:"config,omitempty"`
-	State  uint32   `protobuf:"varint,5,opt,name=state,proto3" json:"state,omitempty"`
-	// Bitmaps of OFPPF_* that describe features.  All bits zeroed if
-	// unsupported or unavailable.
-	Curr                 uint32   `protobuf:"varint,6,opt,name=curr,proto3" json:"curr,omitempty"`
-	Advertised           uint32   `protobuf:"varint,7,opt,name=advertised,proto3" json:"advertised,omitempty"`
-	Supported            uint32   `protobuf:"varint,8,opt,name=supported,proto3" json:"supported,omitempty"`
-	Peer                 uint32   `protobuf:"varint,9,opt,name=peer,proto3" json:"peer,omitempty"`
-	CurrSpeed            uint32   `protobuf:"varint,10,opt,name=curr_speed,json=currSpeed,proto3" json:"curr_speed,omitempty"`
-	MaxSpeed             uint32   `protobuf:"varint,11,opt,name=max_speed,json=maxSpeed,proto3" json:"max_speed,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpPort) Reset()         { *m = OfpPort{} }
-func (m *OfpPort) String() string { return proto.CompactTextString(m) }
-func (*OfpPort) ProtoMessage()    {}
-func (*OfpPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{6}
-}
-
-func (m *OfpPort) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPort.Unmarshal(m, b)
-}
-func (m *OfpPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPort.Marshal(b, m, deterministic)
-}
-func (m *OfpPort) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPort.Merge(m, src)
-}
-func (m *OfpPort) XXX_Size() int {
-	return xxx_messageInfo_OfpPort.Size(m)
-}
-func (m *OfpPort) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPort.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPort proto.InternalMessageInfo
-
-func (m *OfpPort) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OfpPort) GetHwAddr() []uint32 {
-	if m != nil {
-		return m.HwAddr
-	}
-	return nil
-}
-
-func (m *OfpPort) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *OfpPort) GetConfig() uint32 {
-	if m != nil {
-		return m.Config
-	}
-	return 0
-}
-
-func (m *OfpPort) GetState() uint32 {
-	if m != nil {
-		return m.State
-	}
-	return 0
-}
-
-func (m *OfpPort) GetCurr() uint32 {
-	if m != nil {
-		return m.Curr
-	}
-	return 0
-}
-
-func (m *OfpPort) GetAdvertised() uint32 {
-	if m != nil {
-		return m.Advertised
-	}
-	return 0
-}
-
-func (m *OfpPort) GetSupported() uint32 {
-	if m != nil {
-		return m.Supported
-	}
-	return 0
-}
-
-func (m *OfpPort) GetPeer() uint32 {
-	if m != nil {
-		return m.Peer
-	}
-	return 0
-}
-
-func (m *OfpPort) GetCurrSpeed() uint32 {
-	if m != nil {
-		return m.CurrSpeed
-	}
-	return 0
-}
-
-func (m *OfpPort) GetMaxSpeed() uint32 {
-	if m != nil {
-		return m.MaxSpeed
-	}
-	return 0
-}
-
-// Switch features.
-type OfpSwitchFeatures struct {
-	//ofp_header header;
-	DatapathId  uint64 `protobuf:"varint,1,opt,name=datapath_id,json=datapathId,proto3" json:"datapath_id,omitempty"`
-	NBuffers    uint32 `protobuf:"varint,2,opt,name=n_buffers,json=nBuffers,proto3" json:"n_buffers,omitempty"`
-	NTables     uint32 `protobuf:"varint,3,opt,name=n_tables,json=nTables,proto3" json:"n_tables,omitempty"`
-	AuxiliaryId uint32 `protobuf:"varint,4,opt,name=auxiliary_id,json=auxiliaryId,proto3" json:"auxiliary_id,omitempty"`
-	// Features.
-	Capabilities         uint32   `protobuf:"varint,5,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpSwitchFeatures) Reset()         { *m = OfpSwitchFeatures{} }
-func (m *OfpSwitchFeatures) String() string { return proto.CompactTextString(m) }
-func (*OfpSwitchFeatures) ProtoMessage()    {}
-func (*OfpSwitchFeatures) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{7}
-}
-
-func (m *OfpSwitchFeatures) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpSwitchFeatures.Unmarshal(m, b)
-}
-func (m *OfpSwitchFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpSwitchFeatures.Marshal(b, m, deterministic)
-}
-func (m *OfpSwitchFeatures) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpSwitchFeatures.Merge(m, src)
-}
-func (m *OfpSwitchFeatures) XXX_Size() int {
-	return xxx_messageInfo_OfpSwitchFeatures.Size(m)
-}
-func (m *OfpSwitchFeatures) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpSwitchFeatures.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpSwitchFeatures proto.InternalMessageInfo
-
-func (m *OfpSwitchFeatures) GetDatapathId() uint64 {
-	if m != nil {
-		return m.DatapathId
-	}
-	return 0
-}
-
-func (m *OfpSwitchFeatures) GetNBuffers() uint32 {
-	if m != nil {
-		return m.NBuffers
-	}
-	return 0
-}
-
-func (m *OfpSwitchFeatures) GetNTables() uint32 {
-	if m != nil {
-		return m.NTables
-	}
-	return 0
-}
-
-func (m *OfpSwitchFeatures) GetAuxiliaryId() uint32 {
-	if m != nil {
-		return m.AuxiliaryId
-	}
-	return 0
-}
-
-func (m *OfpSwitchFeatures) GetCapabilities() uint32 {
-	if m != nil {
-		return m.Capabilities
-	}
-	return 0
-}
-
-// A physical port has changed in the datapath
-type OfpPortStatus struct {
-	//ofp_header header;
-	Reason               OfpPortReason `protobuf:"varint,1,opt,name=reason,proto3,enum=openflow_13.OfpPortReason" json:"reason,omitempty"`
-	Desc                 *OfpPort      `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *OfpPortStatus) Reset()         { *m = OfpPortStatus{} }
-func (m *OfpPortStatus) String() string { return proto.CompactTextString(m) }
-func (*OfpPortStatus) ProtoMessage()    {}
-func (*OfpPortStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{8}
-}
-
-func (m *OfpPortStatus) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPortStatus.Unmarshal(m, b)
-}
-func (m *OfpPortStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPortStatus.Marshal(b, m, deterministic)
-}
-func (m *OfpPortStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPortStatus.Merge(m, src)
-}
-func (m *OfpPortStatus) XXX_Size() int {
-	return xxx_messageInfo_OfpPortStatus.Size(m)
-}
-func (m *OfpPortStatus) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPortStatus.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPortStatus proto.InternalMessageInfo
-
-func (m *OfpPortStatus) GetReason() OfpPortReason {
-	if m != nil {
-		return m.Reason
-	}
-	return OfpPortReason_OFPPR_ADD
-}
-
-func (m *OfpPortStatus) GetDesc() *OfpPort {
-	if m != nil {
-		return m.Desc
-	}
-	return nil
-}
-
-// Modify behavior of the physical port
-type OfpPortMod struct {
-	//ofp_header header;
-	PortNo uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	HwAddr []uint32 `protobuf:"varint,2,rep,packed,name=hw_addr,json=hwAddr,proto3" json:"hw_addr,omitempty"`
-	// The hardware address is not
-	//configurable.  This is used to
-	//sanity-check the request, so it must
-	//be the same as returned in an
-	//ofp_port struct.
-	Config               uint32   `protobuf:"varint,3,opt,name=config,proto3" json:"config,omitempty"`
-	Mask                 uint32   `protobuf:"varint,4,opt,name=mask,proto3" json:"mask,omitempty"`
-	Advertise            uint32   `protobuf:"varint,5,opt,name=advertise,proto3" json:"advertise,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpPortMod) Reset()         { *m = OfpPortMod{} }
-func (m *OfpPortMod) String() string { return proto.CompactTextString(m) }
-func (*OfpPortMod) ProtoMessage()    {}
-func (*OfpPortMod) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{9}
-}
-
-func (m *OfpPortMod) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPortMod.Unmarshal(m, b)
-}
-func (m *OfpPortMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPortMod.Marshal(b, m, deterministic)
-}
-func (m *OfpPortMod) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPortMod.Merge(m, src)
-}
-func (m *OfpPortMod) XXX_Size() int {
-	return xxx_messageInfo_OfpPortMod.Size(m)
-}
-func (m *OfpPortMod) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPortMod.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPortMod proto.InternalMessageInfo
-
-func (m *OfpPortMod) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OfpPortMod) GetHwAddr() []uint32 {
-	if m != nil {
-		return m.HwAddr
-	}
-	return nil
-}
-
-func (m *OfpPortMod) GetConfig() uint32 {
-	if m != nil {
-		return m.Config
-	}
-	return 0
-}
-
-func (m *OfpPortMod) GetMask() uint32 {
-	if m != nil {
-		return m.Mask
-	}
-	return 0
-}
-
-func (m *OfpPortMod) GetAdvertise() uint32 {
-	if m != nil {
-		return m.Advertise
-	}
-	return 0
-}
-
-// Fields to match against flows
-type OfpMatch struct {
-	Type                 OfpMatchType   `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMatchType" json:"type,omitempty"`
-	OxmFields            []*OfpOxmField `protobuf:"bytes,2,rep,name=oxm_fields,json=oxmFields,proto3" json:"oxm_fields,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *OfpMatch) Reset()         { *m = OfpMatch{} }
-func (m *OfpMatch) String() string { return proto.CompactTextString(m) }
-func (*OfpMatch) ProtoMessage()    {}
-func (*OfpMatch) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{10}
-}
-
-func (m *OfpMatch) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMatch.Unmarshal(m, b)
-}
-func (m *OfpMatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMatch.Marshal(b, m, deterministic)
-}
-func (m *OfpMatch) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMatch.Merge(m, src)
-}
-func (m *OfpMatch) XXX_Size() int {
-	return xxx_messageInfo_OfpMatch.Size(m)
-}
-func (m *OfpMatch) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMatch.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMatch proto.InternalMessageInfo
-
-func (m *OfpMatch) GetType() OfpMatchType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpMatchType_OFPMT_STANDARD
-}
-
-func (m *OfpMatch) GetOxmFields() []*OfpOxmField {
-	if m != nil {
-		return m.OxmFields
-	}
-	return nil
-}
-
-// OXM Flow match fields
-type OfpOxmField struct {
-	OxmClass OfpOxmClass `protobuf:"varint,1,opt,name=oxm_class,json=oxmClass,proto3,enum=openflow_13.OfpOxmClass" json:"oxm_class,omitempty"`
-	// Types that are valid to be assigned to Field:
-	//	*OfpOxmField_OfbField
-	//	*OfpOxmField_ExperimenterField
-	Field                isOfpOxmField_Field `protobuf_oneof:"field"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *OfpOxmField) Reset()         { *m = OfpOxmField{} }
-func (m *OfpOxmField) String() string { return proto.CompactTextString(m) }
-func (*OfpOxmField) ProtoMessage()    {}
-func (*OfpOxmField) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{11}
-}
-
-func (m *OfpOxmField) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpOxmField.Unmarshal(m, b)
-}
-func (m *OfpOxmField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpOxmField.Marshal(b, m, deterministic)
-}
-func (m *OfpOxmField) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpOxmField.Merge(m, src)
-}
-func (m *OfpOxmField) XXX_Size() int {
-	return xxx_messageInfo_OfpOxmField.Size(m)
-}
-func (m *OfpOxmField) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpOxmField.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpOxmField proto.InternalMessageInfo
-
-func (m *OfpOxmField) GetOxmClass() OfpOxmClass {
-	if m != nil {
-		return m.OxmClass
-	}
-	return OfpOxmClass_OFPXMC_NXM_0
-}
-
-type isOfpOxmField_Field interface {
-	isOfpOxmField_Field()
-}
-
-type OfpOxmField_OfbField struct {
-	OfbField *OfpOxmOfbField `protobuf:"bytes,4,opt,name=ofb_field,json=ofbField,proto3,oneof"`
-}
-
-type OfpOxmField_ExperimenterField struct {
-	ExperimenterField *OfpOxmExperimenterField `protobuf:"bytes,5,opt,name=experimenter_field,json=experimenterField,proto3,oneof"`
-}
-
-func (*OfpOxmField_OfbField) isOfpOxmField_Field() {}
-
-func (*OfpOxmField_ExperimenterField) isOfpOxmField_Field() {}
-
-func (m *OfpOxmField) GetField() isOfpOxmField_Field {
-	if m != nil {
-		return m.Field
-	}
-	return nil
-}
-
-func (m *OfpOxmField) GetOfbField() *OfpOxmOfbField {
-	if x, ok := m.GetField().(*OfpOxmField_OfbField); ok {
-		return x.OfbField
-	}
-	return nil
-}
-
-func (m *OfpOxmField) GetExperimenterField() *OfpOxmExperimenterField {
-	if x, ok := m.GetField().(*OfpOxmField_ExperimenterField); ok {
-		return x.ExperimenterField
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpOxmField) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpOxmField_OfbField)(nil),
-		(*OfpOxmField_ExperimenterField)(nil),
-	}
-}
-
-// OXM OpenFlow Basic Match Field
-type OfpOxmOfbField struct {
-	Type    OxmOfbFieldTypes `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OxmOfbFieldTypes" json:"type,omitempty"`
-	HasMask bool             `protobuf:"varint,2,opt,name=has_mask,json=hasMask,proto3" json:"has_mask,omitempty"`
-	// Types that are valid to be assigned to Value:
-	//	*OfpOxmOfbField_Port
-	//	*OfpOxmOfbField_PhysicalPort
-	//	*OfpOxmOfbField_TableMetadata
-	//	*OfpOxmOfbField_EthDst
-	//	*OfpOxmOfbField_EthSrc
-	//	*OfpOxmOfbField_EthType
-	//	*OfpOxmOfbField_VlanVid
-	//	*OfpOxmOfbField_VlanPcp
-	//	*OfpOxmOfbField_IpDscp
-	//	*OfpOxmOfbField_IpEcn
-	//	*OfpOxmOfbField_IpProto
-	//	*OfpOxmOfbField_Ipv4Src
-	//	*OfpOxmOfbField_Ipv4Dst
-	//	*OfpOxmOfbField_TcpSrc
-	//	*OfpOxmOfbField_TcpDst
-	//	*OfpOxmOfbField_UdpSrc
-	//	*OfpOxmOfbField_UdpDst
-	//	*OfpOxmOfbField_SctpSrc
-	//	*OfpOxmOfbField_SctpDst
-	//	*OfpOxmOfbField_Icmpv4Type
-	//	*OfpOxmOfbField_Icmpv4Code
-	//	*OfpOxmOfbField_ArpOp
-	//	*OfpOxmOfbField_ArpSpa
-	//	*OfpOxmOfbField_ArpTpa
-	//	*OfpOxmOfbField_ArpSha
-	//	*OfpOxmOfbField_ArpTha
-	//	*OfpOxmOfbField_Ipv6Src
-	//	*OfpOxmOfbField_Ipv6Dst
-	//	*OfpOxmOfbField_Ipv6Flabel
-	//	*OfpOxmOfbField_Icmpv6Type
-	//	*OfpOxmOfbField_Icmpv6Code
-	//	*OfpOxmOfbField_Ipv6NdTarget
-	//	*OfpOxmOfbField_Ipv6NdSsl
-	//	*OfpOxmOfbField_Ipv6NdTll
-	//	*OfpOxmOfbField_MplsLabel
-	//	*OfpOxmOfbField_MplsTc
-	//	*OfpOxmOfbField_MplsBos
-	//	*OfpOxmOfbField_PbbIsid
-	//	*OfpOxmOfbField_TunnelId
-	//	*OfpOxmOfbField_Ipv6Exthdr
-	Value isOfpOxmOfbField_Value `protobuf_oneof:"value"`
-	// Optional mask values (must be present when has_mask is true
-	//
-	// Types that are valid to be assigned to Mask:
-	//	*OfpOxmOfbField_TableMetadataMask
-	//	*OfpOxmOfbField_EthDstMask
-	//	*OfpOxmOfbField_EthSrcMask
-	//	*OfpOxmOfbField_VlanVidMask
-	//	*OfpOxmOfbField_Ipv4SrcMask
-	//	*OfpOxmOfbField_Ipv4DstMask
-	//	*OfpOxmOfbField_ArpSpaMask
-	//	*OfpOxmOfbField_ArpTpaMask
-	//	*OfpOxmOfbField_Ipv6SrcMask
-	//	*OfpOxmOfbField_Ipv6DstMask
-	//	*OfpOxmOfbField_Ipv6FlabelMask
-	//	*OfpOxmOfbField_PbbIsidMask
-	//	*OfpOxmOfbField_TunnelIdMask
-	//	*OfpOxmOfbField_Ipv6ExthdrMask
-	Mask                 isOfpOxmOfbField_Mask `protobuf_oneof:"mask"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpOxmOfbField) Reset()         { *m = OfpOxmOfbField{} }
-func (m *OfpOxmOfbField) String() string { return proto.CompactTextString(m) }
-func (*OfpOxmOfbField) ProtoMessage()    {}
-func (*OfpOxmOfbField) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{12}
-}
-
-func (m *OfpOxmOfbField) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpOxmOfbField.Unmarshal(m, b)
-}
-func (m *OfpOxmOfbField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpOxmOfbField.Marshal(b, m, deterministic)
-}
-func (m *OfpOxmOfbField) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpOxmOfbField.Merge(m, src)
-}
-func (m *OfpOxmOfbField) XXX_Size() int {
-	return xxx_messageInfo_OfpOxmOfbField.Size(m)
-}
-func (m *OfpOxmOfbField) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpOxmOfbField.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpOxmOfbField proto.InternalMessageInfo
-
-func (m *OfpOxmOfbField) GetType() OxmOfbFieldTypes {
-	if m != nil {
-		return m.Type
-	}
-	return OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT
-}
-
-func (m *OfpOxmOfbField) GetHasMask() bool {
-	if m != nil {
-		return m.HasMask
-	}
-	return false
-}
-
-type isOfpOxmOfbField_Value interface {
-	isOfpOxmOfbField_Value()
-}
-
-type OfpOxmOfbField_Port struct {
-	Port uint32 `protobuf:"varint,3,opt,name=port,proto3,oneof"`
-}
-
-type OfpOxmOfbField_PhysicalPort struct {
-	PhysicalPort uint32 `protobuf:"varint,4,opt,name=physical_port,json=physicalPort,proto3,oneof"`
-}
-
-type OfpOxmOfbField_TableMetadata struct {
-	TableMetadata uint64 `protobuf:"varint,5,opt,name=table_metadata,json=tableMetadata,proto3,oneof"`
-}
-
-type OfpOxmOfbField_EthDst struct {
-	EthDst []byte `protobuf:"bytes,6,opt,name=eth_dst,json=ethDst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_EthSrc struct {
-	EthSrc []byte `protobuf:"bytes,7,opt,name=eth_src,json=ethSrc,proto3,oneof"`
-}
-
-type OfpOxmOfbField_EthType struct {
-	EthType uint32 `protobuf:"varint,8,opt,name=eth_type,json=ethType,proto3,oneof"`
-}
-
-type OfpOxmOfbField_VlanVid struct {
-	VlanVid uint32 `protobuf:"varint,9,opt,name=vlan_vid,json=vlanVid,proto3,oneof"`
-}
-
-type OfpOxmOfbField_VlanPcp struct {
-	VlanPcp uint32 `protobuf:"varint,10,opt,name=vlan_pcp,json=vlanPcp,proto3,oneof"`
-}
-
-type OfpOxmOfbField_IpDscp struct {
-	IpDscp uint32 `protobuf:"varint,11,opt,name=ip_dscp,json=ipDscp,proto3,oneof"`
-}
-
-type OfpOxmOfbField_IpEcn struct {
-	IpEcn uint32 `protobuf:"varint,12,opt,name=ip_ecn,json=ipEcn,proto3,oneof"`
-}
-
-type OfpOxmOfbField_IpProto struct {
-	IpProto uint32 `protobuf:"varint,13,opt,name=ip_proto,json=ipProto,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv4Src struct {
-	Ipv4Src uint32 `protobuf:"varint,14,opt,name=ipv4_src,json=ipv4Src,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv4Dst struct {
-	Ipv4Dst uint32 `protobuf:"varint,15,opt,name=ipv4_dst,json=ipv4Dst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_TcpSrc struct {
-	TcpSrc uint32 `protobuf:"varint,16,opt,name=tcp_src,json=tcpSrc,proto3,oneof"`
-}
-
-type OfpOxmOfbField_TcpDst struct {
-	TcpDst uint32 `protobuf:"varint,17,opt,name=tcp_dst,json=tcpDst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_UdpSrc struct {
-	UdpSrc uint32 `protobuf:"varint,18,opt,name=udp_src,json=udpSrc,proto3,oneof"`
-}
-
-type OfpOxmOfbField_UdpDst struct {
-	UdpDst uint32 `protobuf:"varint,19,opt,name=udp_dst,json=udpDst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_SctpSrc struct {
-	SctpSrc uint32 `protobuf:"varint,20,opt,name=sctp_src,json=sctpSrc,proto3,oneof"`
-}
-
-type OfpOxmOfbField_SctpDst struct {
-	SctpDst uint32 `protobuf:"varint,21,opt,name=sctp_dst,json=sctpDst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Icmpv4Type struct {
-	Icmpv4Type uint32 `protobuf:"varint,22,opt,name=icmpv4_type,json=icmpv4Type,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Icmpv4Code struct {
-	Icmpv4Code uint32 `protobuf:"varint,23,opt,name=icmpv4_code,json=icmpv4Code,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpOp struct {
-	ArpOp uint32 `protobuf:"varint,24,opt,name=arp_op,json=arpOp,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpSpa struct {
-	ArpSpa uint32 `protobuf:"varint,25,opt,name=arp_spa,json=arpSpa,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpTpa struct {
-	ArpTpa uint32 `protobuf:"varint,26,opt,name=arp_tpa,json=arpTpa,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpSha struct {
-	ArpSha []byte `protobuf:"bytes,27,opt,name=arp_sha,json=arpSha,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpTha struct {
-	ArpTha []byte `protobuf:"bytes,28,opt,name=arp_tha,json=arpTha,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6Src struct {
-	Ipv6Src []byte `protobuf:"bytes,29,opt,name=ipv6_src,json=ipv6Src,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6Dst struct {
-	Ipv6Dst []byte `protobuf:"bytes,30,opt,name=ipv6_dst,json=ipv6Dst,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6Flabel struct {
-	Ipv6Flabel uint32 `protobuf:"varint,31,opt,name=ipv6_flabel,json=ipv6Flabel,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Icmpv6Type struct {
-	Icmpv6Type uint32 `protobuf:"varint,32,opt,name=icmpv6_type,json=icmpv6Type,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Icmpv6Code struct {
-	Icmpv6Code uint32 `protobuf:"varint,33,opt,name=icmpv6_code,json=icmpv6Code,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6NdTarget struct {
-	Ipv6NdTarget []byte `protobuf:"bytes,34,opt,name=ipv6_nd_target,json=ipv6NdTarget,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6NdSsl struct {
-	Ipv6NdSsl []byte `protobuf:"bytes,35,opt,name=ipv6_nd_ssl,json=ipv6NdSsl,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6NdTll struct {
-	Ipv6NdTll []byte `protobuf:"bytes,36,opt,name=ipv6_nd_tll,json=ipv6NdTll,proto3,oneof"`
-}
-
-type OfpOxmOfbField_MplsLabel struct {
-	MplsLabel uint32 `protobuf:"varint,37,opt,name=mpls_label,json=mplsLabel,proto3,oneof"`
-}
-
-type OfpOxmOfbField_MplsTc struct {
-	MplsTc uint32 `protobuf:"varint,38,opt,name=mpls_tc,json=mplsTc,proto3,oneof"`
-}
-
-type OfpOxmOfbField_MplsBos struct {
-	MplsBos uint32 `protobuf:"varint,39,opt,name=mpls_bos,json=mplsBos,proto3,oneof"`
-}
-
-type OfpOxmOfbField_PbbIsid struct {
-	PbbIsid uint32 `protobuf:"varint,40,opt,name=pbb_isid,json=pbbIsid,proto3,oneof"`
-}
-
-type OfpOxmOfbField_TunnelId struct {
-	TunnelId uint64 `protobuf:"varint,41,opt,name=tunnel_id,json=tunnelId,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6Exthdr struct {
-	Ipv6Exthdr uint32 `protobuf:"varint,42,opt,name=ipv6_exthdr,json=ipv6Exthdr,proto3,oneof"`
-}
-
-func (*OfpOxmOfbField_Port) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_PhysicalPort) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_TableMetadata) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_EthDst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_EthSrc) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_EthType) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_VlanVid) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_VlanPcp) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_IpDscp) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_IpEcn) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_IpProto) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv4Src) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv4Dst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_TcpSrc) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_TcpDst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_UdpSrc) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_UdpDst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_SctpSrc) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_SctpDst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Icmpv4Type) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Icmpv4Code) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_ArpOp) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_ArpSpa) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_ArpTpa) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_ArpSha) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_ArpTha) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6Src) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6Dst) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6Flabel) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Icmpv6Type) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Icmpv6Code) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6NdTarget) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6NdSsl) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6NdTll) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_MplsLabel) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_MplsTc) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_MplsBos) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_PbbIsid) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_TunnelId) isOfpOxmOfbField_Value() {}
-
-func (*OfpOxmOfbField_Ipv6Exthdr) isOfpOxmOfbField_Value() {}
-
-func (m *OfpOxmOfbField) GetValue() isOfpOxmOfbField_Value {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetPort() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Port); ok {
-		return x.Port
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetPhysicalPort() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_PhysicalPort); ok {
-		return x.PhysicalPort
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetTableMetadata() uint64 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_TableMetadata); ok {
-		return x.TableMetadata
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetEthDst() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_EthDst); ok {
-		return x.EthDst
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetEthSrc() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_EthSrc); ok {
-		return x.EthSrc
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetEthType() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_EthType); ok {
-		return x.EthType
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetVlanVid() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_VlanVid); ok {
-		return x.VlanVid
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetVlanPcp() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_VlanPcp); ok {
-		return x.VlanPcp
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpDscp() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_IpDscp); ok {
-		return x.IpDscp
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpEcn() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_IpEcn); ok {
-		return x.IpEcn
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpProto() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_IpProto); ok {
-		return x.IpProto
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv4Src() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv4Src); ok {
-		return x.Ipv4Src
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv4Dst() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv4Dst); ok {
-		return x.Ipv4Dst
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetTcpSrc() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_TcpSrc); ok {
-		return x.TcpSrc
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetTcpDst() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_TcpDst); ok {
-		return x.TcpDst
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetUdpSrc() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_UdpSrc); ok {
-		return x.UdpSrc
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetUdpDst() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_UdpDst); ok {
-		return x.UdpDst
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetSctpSrc() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_SctpSrc); ok {
-		return x.SctpSrc
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetSctpDst() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_SctpDst); ok {
-		return x.SctpDst
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIcmpv4Type() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv4Type); ok {
-		return x.Icmpv4Type
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIcmpv4Code() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv4Code); ok {
-		return x.Icmpv4Code
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpOp() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpOp); ok {
-		return x.ArpOp
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpSpa() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpSpa); ok {
-		return x.ArpSpa
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpTpa() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpTpa); ok {
-		return x.ArpTpa
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpSha() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpSha); ok {
-		return x.ArpSha
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetArpTha() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpTha); ok {
-		return x.ArpTha
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6Src() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Src); ok {
-		return x.Ipv6Src
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6Dst() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Dst); ok {
-		return x.Ipv6Dst
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6Flabel() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Flabel); ok {
-		return x.Ipv6Flabel
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIcmpv6Type() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv6Type); ok {
-		return x.Icmpv6Type
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIcmpv6Code() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv6Code); ok {
-		return x.Icmpv6Code
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv6NdTarget() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdTarget); ok {
-		return x.Ipv6NdTarget
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6NdSsl() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdSsl); ok {
-		return x.Ipv6NdSsl
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6NdTll() []byte {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdTll); ok {
-		return x.Ipv6NdTll
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetMplsLabel() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsLabel); ok {
-		return x.MplsLabel
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetMplsTc() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsTc); ok {
-		return x.MplsTc
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetMplsBos() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsBos); ok {
-		return x.MplsBos
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetPbbIsid() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_PbbIsid); ok {
-		return x.PbbIsid
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetTunnelId() uint64 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_TunnelId); ok {
-		return x.TunnelId
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv6Exthdr() uint32 {
-	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Exthdr); ok {
-		return x.Ipv6Exthdr
-	}
-	return 0
-}
-
-type isOfpOxmOfbField_Mask interface {
-	isOfpOxmOfbField_Mask()
-}
-
-type OfpOxmOfbField_TableMetadataMask struct {
-	TableMetadataMask uint64 `protobuf:"varint,105,opt,name=table_metadata_mask,json=tableMetadataMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_EthDstMask struct {
-	EthDstMask []byte `protobuf:"bytes,106,opt,name=eth_dst_mask,json=ethDstMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_EthSrcMask struct {
-	EthSrcMask []byte `protobuf:"bytes,107,opt,name=eth_src_mask,json=ethSrcMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_VlanVidMask struct {
-	VlanVidMask uint32 `protobuf:"varint,109,opt,name=vlan_vid_mask,json=vlanVidMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv4SrcMask struct {
-	Ipv4SrcMask uint32 `protobuf:"varint,114,opt,name=ipv4_src_mask,json=ipv4SrcMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv4DstMask struct {
-	Ipv4DstMask uint32 `protobuf:"varint,115,opt,name=ipv4_dst_mask,json=ipv4DstMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpSpaMask struct {
-	ArpSpaMask uint32 `protobuf:"varint,125,opt,name=arp_spa_mask,json=arpSpaMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_ArpTpaMask struct {
-	ArpTpaMask uint32 `protobuf:"varint,126,opt,name=arp_tpa_mask,json=arpTpaMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6SrcMask struct {
-	Ipv6SrcMask []byte `protobuf:"bytes,129,opt,name=ipv6_src_mask,json=ipv6SrcMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6DstMask struct {
-	Ipv6DstMask []byte `protobuf:"bytes,130,opt,name=ipv6_dst_mask,json=ipv6DstMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6FlabelMask struct {
-	Ipv6FlabelMask uint32 `protobuf:"varint,131,opt,name=ipv6_flabel_mask,json=ipv6FlabelMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_PbbIsidMask struct {
-	PbbIsidMask uint32 `protobuf:"varint,140,opt,name=pbb_isid_mask,json=pbbIsidMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_TunnelIdMask struct {
-	TunnelIdMask uint64 `protobuf:"varint,141,opt,name=tunnel_id_mask,json=tunnelIdMask,proto3,oneof"`
-}
-
-type OfpOxmOfbField_Ipv6ExthdrMask struct {
-	Ipv6ExthdrMask uint32 `protobuf:"varint,142,opt,name=ipv6_exthdr_mask,json=ipv6ExthdrMask,proto3,oneof"`
-}
-
-func (*OfpOxmOfbField_TableMetadataMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_EthDstMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_EthSrcMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_VlanVidMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv4SrcMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv4DstMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_ArpSpaMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_ArpTpaMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv6SrcMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv6DstMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv6FlabelMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_PbbIsidMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_TunnelIdMask) isOfpOxmOfbField_Mask() {}
-
-func (*OfpOxmOfbField_Ipv6ExthdrMask) isOfpOxmOfbField_Mask() {}
-
-func (m *OfpOxmOfbField) GetMask() isOfpOxmOfbField_Mask {
-	if m != nil {
-		return m.Mask
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetTableMetadataMask() uint64 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_TableMetadataMask); ok {
-		return x.TableMetadataMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetEthDstMask() []byte {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_EthDstMask); ok {
-		return x.EthDstMask
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetEthSrcMask() []byte {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_EthSrcMask); ok {
-		return x.EthSrcMask
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetVlanVidMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_VlanVidMask); ok {
-		return x.VlanVidMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv4SrcMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv4SrcMask); ok {
-		return x.Ipv4SrcMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv4DstMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv4DstMask); ok {
-		return x.Ipv4DstMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpSpaMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_ArpSpaMask); ok {
-		return x.ArpSpaMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetArpTpaMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_ArpTpaMask); ok {
-		return x.ArpTpaMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv6SrcMask() []byte {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6SrcMask); ok {
-		return x.Ipv6SrcMask
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6DstMask() []byte {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6DstMask); ok {
-		return x.Ipv6DstMask
-	}
-	return nil
-}
-
-func (m *OfpOxmOfbField) GetIpv6FlabelMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6FlabelMask); ok {
-		return x.Ipv6FlabelMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetPbbIsidMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_PbbIsidMask); ok {
-		return x.PbbIsidMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetTunnelIdMask() uint64 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_TunnelIdMask); ok {
-		return x.TunnelIdMask
-	}
-	return 0
-}
-
-func (m *OfpOxmOfbField) GetIpv6ExthdrMask() uint32 {
-	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6ExthdrMask); ok {
-		return x.Ipv6ExthdrMask
-	}
-	return 0
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpOxmOfbField) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpOxmOfbField_Port)(nil),
-		(*OfpOxmOfbField_PhysicalPort)(nil),
-		(*OfpOxmOfbField_TableMetadata)(nil),
-		(*OfpOxmOfbField_EthDst)(nil),
-		(*OfpOxmOfbField_EthSrc)(nil),
-		(*OfpOxmOfbField_EthType)(nil),
-		(*OfpOxmOfbField_VlanVid)(nil),
-		(*OfpOxmOfbField_VlanPcp)(nil),
-		(*OfpOxmOfbField_IpDscp)(nil),
-		(*OfpOxmOfbField_IpEcn)(nil),
-		(*OfpOxmOfbField_IpProto)(nil),
-		(*OfpOxmOfbField_Ipv4Src)(nil),
-		(*OfpOxmOfbField_Ipv4Dst)(nil),
-		(*OfpOxmOfbField_TcpSrc)(nil),
-		(*OfpOxmOfbField_TcpDst)(nil),
-		(*OfpOxmOfbField_UdpSrc)(nil),
-		(*OfpOxmOfbField_UdpDst)(nil),
-		(*OfpOxmOfbField_SctpSrc)(nil),
-		(*OfpOxmOfbField_SctpDst)(nil),
-		(*OfpOxmOfbField_Icmpv4Type)(nil),
-		(*OfpOxmOfbField_Icmpv4Code)(nil),
-		(*OfpOxmOfbField_ArpOp)(nil),
-		(*OfpOxmOfbField_ArpSpa)(nil),
-		(*OfpOxmOfbField_ArpTpa)(nil),
-		(*OfpOxmOfbField_ArpSha)(nil),
-		(*OfpOxmOfbField_ArpTha)(nil),
-		(*OfpOxmOfbField_Ipv6Src)(nil),
-		(*OfpOxmOfbField_Ipv6Dst)(nil),
-		(*OfpOxmOfbField_Ipv6Flabel)(nil),
-		(*OfpOxmOfbField_Icmpv6Type)(nil),
-		(*OfpOxmOfbField_Icmpv6Code)(nil),
-		(*OfpOxmOfbField_Ipv6NdTarget)(nil),
-		(*OfpOxmOfbField_Ipv6NdSsl)(nil),
-		(*OfpOxmOfbField_Ipv6NdTll)(nil),
-		(*OfpOxmOfbField_MplsLabel)(nil),
-		(*OfpOxmOfbField_MplsTc)(nil),
-		(*OfpOxmOfbField_MplsBos)(nil),
-		(*OfpOxmOfbField_PbbIsid)(nil),
-		(*OfpOxmOfbField_TunnelId)(nil),
-		(*OfpOxmOfbField_Ipv6Exthdr)(nil),
-		(*OfpOxmOfbField_TableMetadataMask)(nil),
-		(*OfpOxmOfbField_EthDstMask)(nil),
-		(*OfpOxmOfbField_EthSrcMask)(nil),
-		(*OfpOxmOfbField_VlanVidMask)(nil),
-		(*OfpOxmOfbField_Ipv4SrcMask)(nil),
-		(*OfpOxmOfbField_Ipv4DstMask)(nil),
-		(*OfpOxmOfbField_ArpSpaMask)(nil),
-		(*OfpOxmOfbField_ArpTpaMask)(nil),
-		(*OfpOxmOfbField_Ipv6SrcMask)(nil),
-		(*OfpOxmOfbField_Ipv6DstMask)(nil),
-		(*OfpOxmOfbField_Ipv6FlabelMask)(nil),
-		(*OfpOxmOfbField_PbbIsidMask)(nil),
-		(*OfpOxmOfbField_TunnelIdMask)(nil),
-		(*OfpOxmOfbField_Ipv6ExthdrMask)(nil),
-	}
-}
-
-// Header for OXM experimenter match fields.
-// The experimenter class should not use OXM_HEADER() macros for defining
-// fields due to this extra header.
-type OfpOxmExperimenterField struct {
-	OxmHeader            uint32   `protobuf:"varint,1,opt,name=oxm_header,json=oxmHeader,proto3" json:"oxm_header,omitempty"`
-	Experimenter         uint32   `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpOxmExperimenterField) Reset()         { *m = OfpOxmExperimenterField{} }
-func (m *OfpOxmExperimenterField) String() string { return proto.CompactTextString(m) }
-func (*OfpOxmExperimenterField) ProtoMessage()    {}
-func (*OfpOxmExperimenterField) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{13}
-}
-
-func (m *OfpOxmExperimenterField) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpOxmExperimenterField.Unmarshal(m, b)
-}
-func (m *OfpOxmExperimenterField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpOxmExperimenterField.Marshal(b, m, deterministic)
-}
-func (m *OfpOxmExperimenterField) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpOxmExperimenterField.Merge(m, src)
-}
-func (m *OfpOxmExperimenterField) XXX_Size() int {
-	return xxx_messageInfo_OfpOxmExperimenterField.Size(m)
-}
-func (m *OfpOxmExperimenterField) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpOxmExperimenterField.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpOxmExperimenterField proto.InternalMessageInfo
-
-func (m *OfpOxmExperimenterField) GetOxmHeader() uint32 {
-	if m != nil {
-		return m.OxmHeader
-	}
-	return 0
-}
-
-func (m *OfpOxmExperimenterField) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-// Action header that is common to all actions.  The length includes the
-// header and any padding used to make the action 64-bit aligned.
-// NB: The length of an action *must* always be a multiple of eight.
-type OfpAction struct {
-	Type OfpActionType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpActionType" json:"type,omitempty"`
-	// Types that are valid to be assigned to Action:
-	//	*OfpAction_Output
-	//	*OfpAction_MplsTtl
-	//	*OfpAction_Push
-	//	*OfpAction_PopMpls
-	//	*OfpAction_Group
-	//	*OfpAction_NwTtl
-	//	*OfpAction_SetField
-	//	*OfpAction_Experimenter
-	Action               isOfpAction_Action `protobuf_oneof:"action"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *OfpAction) Reset()         { *m = OfpAction{} }
-func (m *OfpAction) String() string { return proto.CompactTextString(m) }
-func (*OfpAction) ProtoMessage()    {}
-func (*OfpAction) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{14}
-}
-
-func (m *OfpAction) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpAction.Unmarshal(m, b)
-}
-func (m *OfpAction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpAction.Marshal(b, m, deterministic)
-}
-func (m *OfpAction) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpAction.Merge(m, src)
-}
-func (m *OfpAction) XXX_Size() int {
-	return xxx_messageInfo_OfpAction.Size(m)
-}
-func (m *OfpAction) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpAction.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpAction proto.InternalMessageInfo
-
-func (m *OfpAction) GetType() OfpActionType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpActionType_OFPAT_OUTPUT
-}
-
-type isOfpAction_Action interface {
-	isOfpAction_Action()
-}
-
-type OfpAction_Output struct {
-	Output *OfpActionOutput `protobuf:"bytes,2,opt,name=output,proto3,oneof"`
-}
-
-type OfpAction_MplsTtl struct {
-	MplsTtl *OfpActionMplsTtl `protobuf:"bytes,3,opt,name=mpls_ttl,json=mplsTtl,proto3,oneof"`
-}
-
-type OfpAction_Push struct {
-	Push *OfpActionPush `protobuf:"bytes,4,opt,name=push,proto3,oneof"`
-}
-
-type OfpAction_PopMpls struct {
-	PopMpls *OfpActionPopMpls `protobuf:"bytes,5,opt,name=pop_mpls,json=popMpls,proto3,oneof"`
-}
-
-type OfpAction_Group struct {
-	Group *OfpActionGroup `protobuf:"bytes,6,opt,name=group,proto3,oneof"`
-}
-
-type OfpAction_NwTtl struct {
-	NwTtl *OfpActionNwTtl `protobuf:"bytes,7,opt,name=nw_ttl,json=nwTtl,proto3,oneof"`
-}
-
-type OfpAction_SetField struct {
-	SetField *OfpActionSetField `protobuf:"bytes,8,opt,name=set_field,json=setField,proto3,oneof"`
-}
-
-type OfpAction_Experimenter struct {
-	Experimenter *OfpActionExperimenter `protobuf:"bytes,9,opt,name=experimenter,proto3,oneof"`
-}
-
-func (*OfpAction_Output) isOfpAction_Action() {}
-
-func (*OfpAction_MplsTtl) isOfpAction_Action() {}
-
-func (*OfpAction_Push) isOfpAction_Action() {}
-
-func (*OfpAction_PopMpls) isOfpAction_Action() {}
-
-func (*OfpAction_Group) isOfpAction_Action() {}
-
-func (*OfpAction_NwTtl) isOfpAction_Action() {}
-
-func (*OfpAction_SetField) isOfpAction_Action() {}
-
-func (*OfpAction_Experimenter) isOfpAction_Action() {}
-
-func (m *OfpAction) GetAction() isOfpAction_Action {
-	if m != nil {
-		return m.Action
-	}
-	return nil
-}
-
-func (m *OfpAction) GetOutput() *OfpActionOutput {
-	if x, ok := m.GetAction().(*OfpAction_Output); ok {
-		return x.Output
-	}
-	return nil
-}
-
-func (m *OfpAction) GetMplsTtl() *OfpActionMplsTtl {
-	if x, ok := m.GetAction().(*OfpAction_MplsTtl); ok {
-		return x.MplsTtl
-	}
-	return nil
-}
-
-func (m *OfpAction) GetPush() *OfpActionPush {
-	if x, ok := m.GetAction().(*OfpAction_Push); ok {
-		return x.Push
-	}
-	return nil
-}
-
-func (m *OfpAction) GetPopMpls() *OfpActionPopMpls {
-	if x, ok := m.GetAction().(*OfpAction_PopMpls); ok {
-		return x.PopMpls
-	}
-	return nil
-}
-
-func (m *OfpAction) GetGroup() *OfpActionGroup {
-	if x, ok := m.GetAction().(*OfpAction_Group); ok {
-		return x.Group
-	}
-	return nil
-}
-
-func (m *OfpAction) GetNwTtl() *OfpActionNwTtl {
-	if x, ok := m.GetAction().(*OfpAction_NwTtl); ok {
-		return x.NwTtl
-	}
-	return nil
-}
-
-func (m *OfpAction) GetSetField() *OfpActionSetField {
-	if x, ok := m.GetAction().(*OfpAction_SetField); ok {
-		return x.SetField
-	}
-	return nil
-}
-
-func (m *OfpAction) GetExperimenter() *OfpActionExperimenter {
-	if x, ok := m.GetAction().(*OfpAction_Experimenter); ok {
-		return x.Experimenter
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpAction) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpAction_Output)(nil),
-		(*OfpAction_MplsTtl)(nil),
-		(*OfpAction_Push)(nil),
-		(*OfpAction_PopMpls)(nil),
-		(*OfpAction_Group)(nil),
-		(*OfpAction_NwTtl)(nil),
-		(*OfpAction_SetField)(nil),
-		(*OfpAction_Experimenter)(nil),
-	}
-}
-
-// Action structure for OFPAT_OUTPUT, which sends packets out 'port'.
-// When the 'port' is the OFPP_CONTROLLER, 'max_len' indicates the max
-// number of bytes to send.  A 'max_len' of zero means no bytes of the
-// packet should be sent. A 'max_len' of OFPCML_NO_BUFFER means that
-// the packet is not buffered and the complete packet is to be sent to
-// the controller.
-type OfpActionOutput struct {
-	Port                 uint32   `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
-	MaxLen               uint32   `protobuf:"varint,2,opt,name=max_len,json=maxLen,proto3" json:"max_len,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionOutput) Reset()         { *m = OfpActionOutput{} }
-func (m *OfpActionOutput) String() string { return proto.CompactTextString(m) }
-func (*OfpActionOutput) ProtoMessage()    {}
-func (*OfpActionOutput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{15}
-}
-
-func (m *OfpActionOutput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionOutput.Unmarshal(m, b)
-}
-func (m *OfpActionOutput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionOutput.Marshal(b, m, deterministic)
-}
-func (m *OfpActionOutput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionOutput.Merge(m, src)
-}
-func (m *OfpActionOutput) XXX_Size() int {
-	return xxx_messageInfo_OfpActionOutput.Size(m)
-}
-func (m *OfpActionOutput) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionOutput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionOutput proto.InternalMessageInfo
-
-func (m *OfpActionOutput) GetPort() uint32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-func (m *OfpActionOutput) GetMaxLen() uint32 {
-	if m != nil {
-		return m.MaxLen
-	}
-	return 0
-}
-
-// Action structure for OFPAT_SET_MPLS_TTL.
-type OfpActionMplsTtl struct {
-	MplsTtl              uint32   `protobuf:"varint,1,opt,name=mpls_ttl,json=mplsTtl,proto3" json:"mpls_ttl,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionMplsTtl) Reset()         { *m = OfpActionMplsTtl{} }
-func (m *OfpActionMplsTtl) String() string { return proto.CompactTextString(m) }
-func (*OfpActionMplsTtl) ProtoMessage()    {}
-func (*OfpActionMplsTtl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{16}
-}
-
-func (m *OfpActionMplsTtl) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionMplsTtl.Unmarshal(m, b)
-}
-func (m *OfpActionMplsTtl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionMplsTtl.Marshal(b, m, deterministic)
-}
-func (m *OfpActionMplsTtl) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionMplsTtl.Merge(m, src)
-}
-func (m *OfpActionMplsTtl) XXX_Size() int {
-	return xxx_messageInfo_OfpActionMplsTtl.Size(m)
-}
-func (m *OfpActionMplsTtl) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionMplsTtl.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionMplsTtl proto.InternalMessageInfo
-
-func (m *OfpActionMplsTtl) GetMplsTtl() uint32 {
-	if m != nil {
-		return m.MplsTtl
-	}
-	return 0
-}
-
-// Action structure for OFPAT_PUSH_VLAN/MPLS/PBB.
-type OfpActionPush struct {
-	Ethertype            uint32   `protobuf:"varint,1,opt,name=ethertype,proto3" json:"ethertype,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionPush) Reset()         { *m = OfpActionPush{} }
-func (m *OfpActionPush) String() string { return proto.CompactTextString(m) }
-func (*OfpActionPush) ProtoMessage()    {}
-func (*OfpActionPush) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{17}
-}
-
-func (m *OfpActionPush) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionPush.Unmarshal(m, b)
-}
-func (m *OfpActionPush) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionPush.Marshal(b, m, deterministic)
-}
-func (m *OfpActionPush) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionPush.Merge(m, src)
-}
-func (m *OfpActionPush) XXX_Size() int {
-	return xxx_messageInfo_OfpActionPush.Size(m)
-}
-func (m *OfpActionPush) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionPush.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionPush proto.InternalMessageInfo
-
-func (m *OfpActionPush) GetEthertype() uint32 {
-	if m != nil {
-		return m.Ethertype
-	}
-	return 0
-}
-
-// Action structure for OFPAT_POP_MPLS.
-type OfpActionPopMpls struct {
-	Ethertype            uint32   `protobuf:"varint,1,opt,name=ethertype,proto3" json:"ethertype,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionPopMpls) Reset()         { *m = OfpActionPopMpls{} }
-func (m *OfpActionPopMpls) String() string { return proto.CompactTextString(m) }
-func (*OfpActionPopMpls) ProtoMessage()    {}
-func (*OfpActionPopMpls) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{18}
-}
-
-func (m *OfpActionPopMpls) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionPopMpls.Unmarshal(m, b)
-}
-func (m *OfpActionPopMpls) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionPopMpls.Marshal(b, m, deterministic)
-}
-func (m *OfpActionPopMpls) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionPopMpls.Merge(m, src)
-}
-func (m *OfpActionPopMpls) XXX_Size() int {
-	return xxx_messageInfo_OfpActionPopMpls.Size(m)
-}
-func (m *OfpActionPopMpls) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionPopMpls.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionPopMpls proto.InternalMessageInfo
-
-func (m *OfpActionPopMpls) GetEthertype() uint32 {
-	if m != nil {
-		return m.Ethertype
-	}
-	return 0
-}
-
-// Action structure for OFPAT_GROUP.
-type OfpActionGroup struct {
-	GroupId              uint32   `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionGroup) Reset()         { *m = OfpActionGroup{} }
-func (m *OfpActionGroup) String() string { return proto.CompactTextString(m) }
-func (*OfpActionGroup) ProtoMessage()    {}
-func (*OfpActionGroup) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{19}
-}
-
-func (m *OfpActionGroup) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionGroup.Unmarshal(m, b)
-}
-func (m *OfpActionGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionGroup.Marshal(b, m, deterministic)
-}
-func (m *OfpActionGroup) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionGroup.Merge(m, src)
-}
-func (m *OfpActionGroup) XXX_Size() int {
-	return xxx_messageInfo_OfpActionGroup.Size(m)
-}
-func (m *OfpActionGroup) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionGroup.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionGroup proto.InternalMessageInfo
-
-func (m *OfpActionGroup) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-// Action structure for OFPAT_SET_NW_TTL.
-type OfpActionNwTtl struct {
-	NwTtl                uint32   `protobuf:"varint,1,opt,name=nw_ttl,json=nwTtl,proto3" json:"nw_ttl,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionNwTtl) Reset()         { *m = OfpActionNwTtl{} }
-func (m *OfpActionNwTtl) String() string { return proto.CompactTextString(m) }
-func (*OfpActionNwTtl) ProtoMessage()    {}
-func (*OfpActionNwTtl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{20}
-}
-
-func (m *OfpActionNwTtl) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionNwTtl.Unmarshal(m, b)
-}
-func (m *OfpActionNwTtl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionNwTtl.Marshal(b, m, deterministic)
-}
-func (m *OfpActionNwTtl) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionNwTtl.Merge(m, src)
-}
-func (m *OfpActionNwTtl) XXX_Size() int {
-	return xxx_messageInfo_OfpActionNwTtl.Size(m)
-}
-func (m *OfpActionNwTtl) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionNwTtl.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionNwTtl proto.InternalMessageInfo
-
-func (m *OfpActionNwTtl) GetNwTtl() uint32 {
-	if m != nil {
-		return m.NwTtl
-	}
-	return 0
-}
-
-// Action structure for OFPAT_SET_FIELD.
-type OfpActionSetField struct {
-	Field                *OfpOxmField `protobuf:"bytes,1,opt,name=field,proto3" json:"field,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *OfpActionSetField) Reset()         { *m = OfpActionSetField{} }
-func (m *OfpActionSetField) String() string { return proto.CompactTextString(m) }
-func (*OfpActionSetField) ProtoMessage()    {}
-func (*OfpActionSetField) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{21}
-}
-
-func (m *OfpActionSetField) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionSetField.Unmarshal(m, b)
-}
-func (m *OfpActionSetField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionSetField.Marshal(b, m, deterministic)
-}
-func (m *OfpActionSetField) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionSetField.Merge(m, src)
-}
-func (m *OfpActionSetField) XXX_Size() int {
-	return xxx_messageInfo_OfpActionSetField.Size(m)
-}
-func (m *OfpActionSetField) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionSetField.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionSetField proto.InternalMessageInfo
-
-func (m *OfpActionSetField) GetField() *OfpOxmField {
-	if m != nil {
-		return m.Field
-	}
-	return nil
-}
-
-// Action header for OFPAT_EXPERIMENTER.
-// The rest of the body is experimenter-defined.
-type OfpActionExperimenter struct {
-	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionExperimenter) Reset()         { *m = OfpActionExperimenter{} }
-func (m *OfpActionExperimenter) String() string { return proto.CompactTextString(m) }
-func (*OfpActionExperimenter) ProtoMessage()    {}
-func (*OfpActionExperimenter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{22}
-}
-
-func (m *OfpActionExperimenter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionExperimenter.Unmarshal(m, b)
-}
-func (m *OfpActionExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionExperimenter.Marshal(b, m, deterministic)
-}
-func (m *OfpActionExperimenter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionExperimenter.Merge(m, src)
-}
-func (m *OfpActionExperimenter) XXX_Size() int {
-	return xxx_messageInfo_OfpActionExperimenter.Size(m)
-}
-func (m *OfpActionExperimenter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionExperimenter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionExperimenter proto.InternalMessageInfo
-
-func (m *OfpActionExperimenter) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpActionExperimenter) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Instruction header that is common to all instructions.  The length includes
-// the header and any padding used to make the instruction 64-bit aligned.
-// NB: The length of an instruction *must* always be a multiple of eight.
-type OfpInstruction struct {
-	Type uint32 `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
-	// Types that are valid to be assigned to Data:
-	//	*OfpInstruction_GotoTable
-	//	*OfpInstruction_WriteMetadata
-	//	*OfpInstruction_Actions
-	//	*OfpInstruction_Meter
-	//	*OfpInstruction_Experimenter
-	Data                 isOfpInstruction_Data `protobuf_oneof:"data"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpInstruction) Reset()         { *m = OfpInstruction{} }
-func (m *OfpInstruction) String() string { return proto.CompactTextString(m) }
-func (*OfpInstruction) ProtoMessage()    {}
-func (*OfpInstruction) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{23}
-}
-
-func (m *OfpInstruction) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstruction.Unmarshal(m, b)
-}
-func (m *OfpInstruction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstruction.Marshal(b, m, deterministic)
-}
-func (m *OfpInstruction) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstruction.Merge(m, src)
-}
-func (m *OfpInstruction) XXX_Size() int {
-	return xxx_messageInfo_OfpInstruction.Size(m)
-}
-func (m *OfpInstruction) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstruction.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstruction proto.InternalMessageInfo
-
-func (m *OfpInstruction) GetType() uint32 {
-	if m != nil {
-		return m.Type
-	}
-	return 0
-}
-
-type isOfpInstruction_Data interface {
-	isOfpInstruction_Data()
-}
-
-type OfpInstruction_GotoTable struct {
-	GotoTable *OfpInstructionGotoTable `protobuf:"bytes,2,opt,name=goto_table,json=gotoTable,proto3,oneof"`
-}
-
-type OfpInstruction_WriteMetadata struct {
-	WriteMetadata *OfpInstructionWriteMetadata `protobuf:"bytes,3,opt,name=write_metadata,json=writeMetadata,proto3,oneof"`
-}
-
-type OfpInstruction_Actions struct {
-	Actions *OfpInstructionActions `protobuf:"bytes,4,opt,name=actions,proto3,oneof"`
-}
-
-type OfpInstruction_Meter struct {
-	Meter *OfpInstructionMeter `protobuf:"bytes,5,opt,name=meter,proto3,oneof"`
-}
-
-type OfpInstruction_Experimenter struct {
-	Experimenter *OfpInstructionExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
-}
-
-func (*OfpInstruction_GotoTable) isOfpInstruction_Data() {}
-
-func (*OfpInstruction_WriteMetadata) isOfpInstruction_Data() {}
-
-func (*OfpInstruction_Actions) isOfpInstruction_Data() {}
-
-func (*OfpInstruction_Meter) isOfpInstruction_Data() {}
-
-func (*OfpInstruction_Experimenter) isOfpInstruction_Data() {}
-
-func (m *OfpInstruction) GetData() isOfpInstruction_Data {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-func (m *OfpInstruction) GetGotoTable() *OfpInstructionGotoTable {
-	if x, ok := m.GetData().(*OfpInstruction_GotoTable); ok {
-		return x.GotoTable
-	}
-	return nil
-}
-
-func (m *OfpInstruction) GetWriteMetadata() *OfpInstructionWriteMetadata {
-	if x, ok := m.GetData().(*OfpInstruction_WriteMetadata); ok {
-		return x.WriteMetadata
-	}
-	return nil
-}
-
-func (m *OfpInstruction) GetActions() *OfpInstructionActions {
-	if x, ok := m.GetData().(*OfpInstruction_Actions); ok {
-		return x.Actions
-	}
-	return nil
-}
-
-func (m *OfpInstruction) GetMeter() *OfpInstructionMeter {
-	if x, ok := m.GetData().(*OfpInstruction_Meter); ok {
-		return x.Meter
-	}
-	return nil
-}
-
-func (m *OfpInstruction) GetExperimenter() *OfpInstructionExperimenter {
-	if x, ok := m.GetData().(*OfpInstruction_Experimenter); ok {
-		return x.Experimenter
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpInstruction) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpInstruction_GotoTable)(nil),
-		(*OfpInstruction_WriteMetadata)(nil),
-		(*OfpInstruction_Actions)(nil),
-		(*OfpInstruction_Meter)(nil),
-		(*OfpInstruction_Experimenter)(nil),
-	}
-}
-
-// Instruction structure for OFPIT_GOTO_TABLE
-type OfpInstructionGotoTable struct {
-	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpInstructionGotoTable) Reset()         { *m = OfpInstructionGotoTable{} }
-func (m *OfpInstructionGotoTable) String() string { return proto.CompactTextString(m) }
-func (*OfpInstructionGotoTable) ProtoMessage()    {}
-func (*OfpInstructionGotoTable) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{24}
-}
-
-func (m *OfpInstructionGotoTable) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstructionGotoTable.Unmarshal(m, b)
-}
-func (m *OfpInstructionGotoTable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstructionGotoTable.Marshal(b, m, deterministic)
-}
-func (m *OfpInstructionGotoTable) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstructionGotoTable.Merge(m, src)
-}
-func (m *OfpInstructionGotoTable) XXX_Size() int {
-	return xxx_messageInfo_OfpInstructionGotoTable.Size(m)
-}
-func (m *OfpInstructionGotoTable) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstructionGotoTable.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstructionGotoTable proto.InternalMessageInfo
-
-func (m *OfpInstructionGotoTable) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-// Instruction structure for OFPIT_WRITE_METADATA
-type OfpInstructionWriteMetadata struct {
-	Metadata             uint64   `protobuf:"varint,1,opt,name=metadata,proto3" json:"metadata,omitempty"`
-	MetadataMask         uint64   `protobuf:"varint,2,opt,name=metadata_mask,json=metadataMask,proto3" json:"metadata_mask,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpInstructionWriteMetadata) Reset()         { *m = OfpInstructionWriteMetadata{} }
-func (m *OfpInstructionWriteMetadata) String() string { return proto.CompactTextString(m) }
-func (*OfpInstructionWriteMetadata) ProtoMessage()    {}
-func (*OfpInstructionWriteMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{25}
-}
-
-func (m *OfpInstructionWriteMetadata) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstructionWriteMetadata.Unmarshal(m, b)
-}
-func (m *OfpInstructionWriteMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstructionWriteMetadata.Marshal(b, m, deterministic)
-}
-func (m *OfpInstructionWriteMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstructionWriteMetadata.Merge(m, src)
-}
-func (m *OfpInstructionWriteMetadata) XXX_Size() int {
-	return xxx_messageInfo_OfpInstructionWriteMetadata.Size(m)
-}
-func (m *OfpInstructionWriteMetadata) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstructionWriteMetadata.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstructionWriteMetadata proto.InternalMessageInfo
-
-func (m *OfpInstructionWriteMetadata) GetMetadata() uint64 {
-	if m != nil {
-		return m.Metadata
-	}
-	return 0
-}
-
-func (m *OfpInstructionWriteMetadata) GetMetadataMask() uint64 {
-	if m != nil {
-		return m.MetadataMask
-	}
-	return 0
-}
-
-// Instruction structure for OFPIT_WRITE/APPLY/CLEAR_ACTIONS
-type OfpInstructionActions struct {
-	Actions              []*OfpAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *OfpInstructionActions) Reset()         { *m = OfpInstructionActions{} }
-func (m *OfpInstructionActions) String() string { return proto.CompactTextString(m) }
-func (*OfpInstructionActions) ProtoMessage()    {}
-func (*OfpInstructionActions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{26}
-}
-
-func (m *OfpInstructionActions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstructionActions.Unmarshal(m, b)
-}
-func (m *OfpInstructionActions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstructionActions.Marshal(b, m, deterministic)
-}
-func (m *OfpInstructionActions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstructionActions.Merge(m, src)
-}
-func (m *OfpInstructionActions) XXX_Size() int {
-	return xxx_messageInfo_OfpInstructionActions.Size(m)
-}
-func (m *OfpInstructionActions) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstructionActions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstructionActions proto.InternalMessageInfo
-
-func (m *OfpInstructionActions) GetActions() []*OfpAction {
-	if m != nil {
-		return m.Actions
-	}
-	return nil
-}
-
-// Instruction structure for OFPIT_METER
-type OfpInstructionMeter struct {
-	MeterId              uint32   `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpInstructionMeter) Reset()         { *m = OfpInstructionMeter{} }
-func (m *OfpInstructionMeter) String() string { return proto.CompactTextString(m) }
-func (*OfpInstructionMeter) ProtoMessage()    {}
-func (*OfpInstructionMeter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{27}
-}
-
-func (m *OfpInstructionMeter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstructionMeter.Unmarshal(m, b)
-}
-func (m *OfpInstructionMeter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstructionMeter.Marshal(b, m, deterministic)
-}
-func (m *OfpInstructionMeter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstructionMeter.Merge(m, src)
-}
-func (m *OfpInstructionMeter) XXX_Size() int {
-	return xxx_messageInfo_OfpInstructionMeter.Size(m)
-}
-func (m *OfpInstructionMeter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstructionMeter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstructionMeter proto.InternalMessageInfo
-
-func (m *OfpInstructionMeter) GetMeterId() uint32 {
-	if m != nil {
-		return m.MeterId
-	}
-	return 0
-}
-
-// Instruction structure for experimental instructions
-type OfpInstructionExperimenter struct {
-	Experimenter uint32 `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	// Experimenter-defined arbitrary additional data.
-	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpInstructionExperimenter) Reset()         { *m = OfpInstructionExperimenter{} }
-func (m *OfpInstructionExperimenter) String() string { return proto.CompactTextString(m) }
-func (*OfpInstructionExperimenter) ProtoMessage()    {}
-func (*OfpInstructionExperimenter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{28}
-}
-
-func (m *OfpInstructionExperimenter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpInstructionExperimenter.Unmarshal(m, b)
-}
-func (m *OfpInstructionExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpInstructionExperimenter.Marshal(b, m, deterministic)
-}
-func (m *OfpInstructionExperimenter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpInstructionExperimenter.Merge(m, src)
-}
-func (m *OfpInstructionExperimenter) XXX_Size() int {
-	return xxx_messageInfo_OfpInstructionExperimenter.Size(m)
-}
-func (m *OfpInstructionExperimenter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpInstructionExperimenter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpInstructionExperimenter proto.InternalMessageInfo
-
-func (m *OfpInstructionExperimenter) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpInstructionExperimenter) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Flow setup and teardown (controller -> datapath).
-type OfpFlowMod struct {
-	//ofp_header header;
-	Cookie               uint64            `protobuf:"varint,1,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	CookieMask           uint64            `protobuf:"varint,2,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
-	TableId              uint32            `protobuf:"varint,3,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	Command              OfpFlowModCommand `protobuf:"varint,4,opt,name=command,proto3,enum=openflow_13.OfpFlowModCommand" json:"command,omitempty"`
-	IdleTimeout          uint32            `protobuf:"varint,5,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
-	HardTimeout          uint32            `protobuf:"varint,6,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
-	Priority             uint32            `protobuf:"varint,7,opt,name=priority,proto3" json:"priority,omitempty"`
-	BufferId             uint32            `protobuf:"varint,8,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
-	OutPort              uint32            `protobuf:"varint,9,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
-	OutGroup             uint32            `protobuf:"varint,10,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
-	Flags                uint32            `protobuf:"varint,11,opt,name=flags,proto3" json:"flags,omitempty"`
-	Match                *OfpMatch         `protobuf:"bytes,12,opt,name=match,proto3" json:"match,omitempty"`
-	Instructions         []*OfpInstruction `protobuf:"bytes,13,rep,name=instructions,proto3" json:"instructions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpFlowMod) Reset()         { *m = OfpFlowMod{} }
-func (m *OfpFlowMod) String() string { return proto.CompactTextString(m) }
-func (*OfpFlowMod) ProtoMessage()    {}
-func (*OfpFlowMod) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{29}
-}
-
-func (m *OfpFlowMod) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpFlowMod.Unmarshal(m, b)
-}
-func (m *OfpFlowMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpFlowMod.Marshal(b, m, deterministic)
-}
-func (m *OfpFlowMod) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpFlowMod.Merge(m, src)
-}
-func (m *OfpFlowMod) XXX_Size() int {
-	return xxx_messageInfo_OfpFlowMod.Size(m)
-}
-func (m *OfpFlowMod) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpFlowMod.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpFlowMod proto.InternalMessageInfo
-
-func (m *OfpFlowMod) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetCookieMask() uint64 {
-	if m != nil {
-		return m.CookieMask
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetCommand() OfpFlowModCommand {
-	if m != nil {
-		return m.Command
-	}
-	return OfpFlowModCommand_OFPFC_ADD
-}
-
-func (m *OfpFlowMod) GetIdleTimeout() uint32 {
-	if m != nil {
-		return m.IdleTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetHardTimeout() uint32 {
-	if m != nil {
-		return m.HardTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetBufferId() uint32 {
-	if m != nil {
-		return m.BufferId
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetOutPort() uint32 {
-	if m != nil {
-		return m.OutPort
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetOutGroup() uint32 {
-	if m != nil {
-		return m.OutGroup
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpFlowMod) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-func (m *OfpFlowMod) GetInstructions() []*OfpInstruction {
-	if m != nil {
-		return m.Instructions
-	}
-	return nil
-}
-
-// Bucket for use in groups.
-type OfpBucket struct {
-	Weight               uint32       `protobuf:"varint,1,opt,name=weight,proto3" json:"weight,omitempty"`
-	WatchPort            uint32       `protobuf:"varint,2,opt,name=watch_port,json=watchPort,proto3" json:"watch_port,omitempty"`
-	WatchGroup           uint32       `protobuf:"varint,3,opt,name=watch_group,json=watchGroup,proto3" json:"watch_group,omitempty"`
-	Actions              []*OfpAction `protobuf:"bytes,4,rep,name=actions,proto3" json:"actions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *OfpBucket) Reset()         { *m = OfpBucket{} }
-func (m *OfpBucket) String() string { return proto.CompactTextString(m) }
-func (*OfpBucket) ProtoMessage()    {}
-func (*OfpBucket) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{30}
-}
-
-func (m *OfpBucket) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpBucket.Unmarshal(m, b)
-}
-func (m *OfpBucket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpBucket.Marshal(b, m, deterministic)
-}
-func (m *OfpBucket) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpBucket.Merge(m, src)
-}
-func (m *OfpBucket) XXX_Size() int {
-	return xxx_messageInfo_OfpBucket.Size(m)
-}
-func (m *OfpBucket) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpBucket.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpBucket proto.InternalMessageInfo
-
-func (m *OfpBucket) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *OfpBucket) GetWatchPort() uint32 {
-	if m != nil {
-		return m.WatchPort
-	}
-	return 0
-}
-
-func (m *OfpBucket) GetWatchGroup() uint32 {
-	if m != nil {
-		return m.WatchGroup
-	}
-	return 0
-}
-
-func (m *OfpBucket) GetActions() []*OfpAction {
-	if m != nil {
-		return m.Actions
-	}
-	return nil
-}
-
-// Group setup and teardown (controller -> datapath).
-type OfpGroupMod struct {
-	//ofp_header header;
-	Command              OfpGroupModCommand `protobuf:"varint,1,opt,name=command,proto3,enum=openflow_13.OfpGroupModCommand" json:"command,omitempty"`
-	Type                 OfpGroupType       `protobuf:"varint,2,opt,name=type,proto3,enum=openflow_13.OfpGroupType" json:"type,omitempty"`
-	GroupId              uint32             `protobuf:"varint,3,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	Buckets              []*OfpBucket       `protobuf:"bytes,4,rep,name=buckets,proto3" json:"buckets,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *OfpGroupMod) Reset()         { *m = OfpGroupMod{} }
-func (m *OfpGroupMod) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupMod) ProtoMessage()    {}
-func (*OfpGroupMod) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{31}
-}
-
-func (m *OfpGroupMod) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupMod.Unmarshal(m, b)
-}
-func (m *OfpGroupMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupMod.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupMod) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupMod.Merge(m, src)
-}
-func (m *OfpGroupMod) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupMod.Size(m)
-}
-func (m *OfpGroupMod) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupMod.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupMod proto.InternalMessageInfo
-
-func (m *OfpGroupMod) GetCommand() OfpGroupModCommand {
-	if m != nil {
-		return m.Command
-	}
-	return OfpGroupModCommand_OFPGC_ADD
-}
-
-func (m *OfpGroupMod) GetType() OfpGroupType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpGroupType_OFPGT_ALL
-}
-
-func (m *OfpGroupMod) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-func (m *OfpGroupMod) GetBuckets() []*OfpBucket {
-	if m != nil {
-		return m.Buckets
-	}
-	return nil
-}
-
-// Send packet (controller -> datapath).
-type OfpPacketOut struct {
-	//ofp_header header;
-	BufferId uint32       `protobuf:"varint,1,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
-	InPort   uint32       `protobuf:"varint,2,opt,name=in_port,json=inPort,proto3" json:"in_port,omitempty"`
-	Actions  []*OfpAction `protobuf:"bytes,3,rep,name=actions,proto3" json:"actions,omitempty"`
-	// The variable size action list is optionally followed by packet data.
-	// This data is only present and meaningful if buffer_id == -1.
-	Data                 []byte   `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpPacketOut) Reset()         { *m = OfpPacketOut{} }
-func (m *OfpPacketOut) String() string { return proto.CompactTextString(m) }
-func (*OfpPacketOut) ProtoMessage()    {}
-func (*OfpPacketOut) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{32}
-}
-
-func (m *OfpPacketOut) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPacketOut.Unmarshal(m, b)
-}
-func (m *OfpPacketOut) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPacketOut.Marshal(b, m, deterministic)
-}
-func (m *OfpPacketOut) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPacketOut.Merge(m, src)
-}
-func (m *OfpPacketOut) XXX_Size() int {
-	return xxx_messageInfo_OfpPacketOut.Size(m)
-}
-func (m *OfpPacketOut) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPacketOut.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPacketOut proto.InternalMessageInfo
-
-func (m *OfpPacketOut) GetBufferId() uint32 {
-	if m != nil {
-		return m.BufferId
-	}
-	return 0
-}
-
-func (m *OfpPacketOut) GetInPort() uint32 {
-	if m != nil {
-		return m.InPort
-	}
-	return 0
-}
-
-func (m *OfpPacketOut) GetActions() []*OfpAction {
-	if m != nil {
-		return m.Actions
-	}
-	return nil
-}
-
-func (m *OfpPacketOut) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Packet received on port (datapath -> controller).
-type OfpPacketIn struct {
-	//ofp_header header;
-	BufferId             uint32            `protobuf:"varint,1,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
-	Reason               OfpPacketInReason `protobuf:"varint,2,opt,name=reason,proto3,enum=openflow_13.OfpPacketInReason" json:"reason,omitempty"`
-	TableId              uint32            `protobuf:"varint,3,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	Cookie               uint64            `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	Match                *OfpMatch         `protobuf:"bytes,5,opt,name=match,proto3" json:"match,omitempty"`
-	Data                 []byte            `protobuf:"bytes,6,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpPacketIn) Reset()         { *m = OfpPacketIn{} }
-func (m *OfpPacketIn) String() string { return proto.CompactTextString(m) }
-func (*OfpPacketIn) ProtoMessage()    {}
-func (*OfpPacketIn) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{33}
-}
-
-func (m *OfpPacketIn) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPacketIn.Unmarshal(m, b)
-}
-func (m *OfpPacketIn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPacketIn.Marshal(b, m, deterministic)
-}
-func (m *OfpPacketIn) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPacketIn.Merge(m, src)
-}
-func (m *OfpPacketIn) XXX_Size() int {
-	return xxx_messageInfo_OfpPacketIn.Size(m)
-}
-func (m *OfpPacketIn) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPacketIn.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPacketIn proto.InternalMessageInfo
-
-func (m *OfpPacketIn) GetBufferId() uint32 {
-	if m != nil {
-		return m.BufferId
-	}
-	return 0
-}
-
-func (m *OfpPacketIn) GetReason() OfpPacketInReason {
-	if m != nil {
-		return m.Reason
-	}
-	return OfpPacketInReason_OFPR_NO_MATCH
-}
-
-func (m *OfpPacketIn) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpPacketIn) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpPacketIn) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-func (m *OfpPacketIn) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Flow removed (datapath -> controller).
-type OfpFlowRemoved struct {
-	//ofp_header header;
-	Cookie               uint64               `protobuf:"varint,1,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	Priority             uint32               `protobuf:"varint,2,opt,name=priority,proto3" json:"priority,omitempty"`
-	Reason               OfpFlowRemovedReason `protobuf:"varint,3,opt,name=reason,proto3,enum=openflow_13.OfpFlowRemovedReason" json:"reason,omitempty"`
-	TableId              uint32               `protobuf:"varint,4,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	DurationSec          uint32               `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32               `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	IdleTimeout          uint32               `protobuf:"varint,7,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
-	HardTimeout          uint32               `protobuf:"varint,8,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
-	PacketCount          uint64               `protobuf:"varint,9,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
-	ByteCount            uint64               `protobuf:"varint,10,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
-	Match                *OfpMatch            `protobuf:"bytes,121,opt,name=match,proto3" json:"match,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *OfpFlowRemoved) Reset()         { *m = OfpFlowRemoved{} }
-func (m *OfpFlowRemoved) String() string { return proto.CompactTextString(m) }
-func (*OfpFlowRemoved) ProtoMessage()    {}
-func (*OfpFlowRemoved) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{34}
-}
-
-func (m *OfpFlowRemoved) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpFlowRemoved.Unmarshal(m, b)
-}
-func (m *OfpFlowRemoved) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpFlowRemoved.Marshal(b, m, deterministic)
-}
-func (m *OfpFlowRemoved) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpFlowRemoved.Merge(m, src)
-}
-func (m *OfpFlowRemoved) XXX_Size() int {
-	return xxx_messageInfo_OfpFlowRemoved.Size(m)
-}
-func (m *OfpFlowRemoved) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpFlowRemoved.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpFlowRemoved proto.InternalMessageInfo
-
-func (m *OfpFlowRemoved) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetReason() OfpFlowRemovedReason {
-	if m != nil {
-		return m.Reason
-	}
-	return OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT
-}
-
-func (m *OfpFlowRemoved) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetIdleTimeout() uint32 {
-	if m != nil {
-		return m.IdleTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetHardTimeout() uint32 {
-	if m != nil {
-		return m.HardTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetPacketCount() uint64 {
-	if m != nil {
-		return m.PacketCount
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetByteCount() uint64 {
-	if m != nil {
-		return m.ByteCount
-	}
-	return 0
-}
-
-func (m *OfpFlowRemoved) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-// Common header for all meter bands
-type OfpMeterBandHeader struct {
-	Type      OfpMeterBandType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMeterBandType" json:"type,omitempty"`
-	Rate      uint32           `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
-	BurstSize uint32           `protobuf:"varint,3,opt,name=burst_size,json=burstSize,proto3" json:"burst_size,omitempty"`
-	// Types that are valid to be assigned to Data:
-	//	*OfpMeterBandHeader_Drop
-	//	*OfpMeterBandHeader_DscpRemark
-	//	*OfpMeterBandHeader_Experimenter
-	Data                 isOfpMeterBandHeader_Data `protobuf_oneof:"data"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *OfpMeterBandHeader) Reset()         { *m = OfpMeterBandHeader{} }
-func (m *OfpMeterBandHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterBandHeader) ProtoMessage()    {}
-func (*OfpMeterBandHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{35}
-}
-
-func (m *OfpMeterBandHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterBandHeader.Unmarshal(m, b)
-}
-func (m *OfpMeterBandHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterBandHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterBandHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterBandHeader.Merge(m, src)
-}
-func (m *OfpMeterBandHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterBandHeader.Size(m)
-}
-func (m *OfpMeterBandHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterBandHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterBandHeader proto.InternalMessageInfo
-
-func (m *OfpMeterBandHeader) GetType() OfpMeterBandType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpMeterBandType_OFPMBT_INVALID
-}
-
-func (m *OfpMeterBandHeader) GetRate() uint32 {
-	if m != nil {
-		return m.Rate
-	}
-	return 0
-}
-
-func (m *OfpMeterBandHeader) GetBurstSize() uint32 {
-	if m != nil {
-		return m.BurstSize
-	}
-	return 0
-}
-
-type isOfpMeterBandHeader_Data interface {
-	isOfpMeterBandHeader_Data()
-}
-
-type OfpMeterBandHeader_Drop struct {
-	Drop *OfpMeterBandDrop `protobuf:"bytes,4,opt,name=drop,proto3,oneof"`
-}
-
-type OfpMeterBandHeader_DscpRemark struct {
-	DscpRemark *OfpMeterBandDscpRemark `protobuf:"bytes,5,opt,name=dscp_remark,json=dscpRemark,proto3,oneof"`
-}
-
-type OfpMeterBandHeader_Experimenter struct {
-	Experimenter *OfpMeterBandExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
-}
-
-func (*OfpMeterBandHeader_Drop) isOfpMeterBandHeader_Data() {}
-
-func (*OfpMeterBandHeader_DscpRemark) isOfpMeterBandHeader_Data() {}
-
-func (*OfpMeterBandHeader_Experimenter) isOfpMeterBandHeader_Data() {}
-
-func (m *OfpMeterBandHeader) GetData() isOfpMeterBandHeader_Data {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-func (m *OfpMeterBandHeader) GetDrop() *OfpMeterBandDrop {
-	if x, ok := m.GetData().(*OfpMeterBandHeader_Drop); ok {
-		return x.Drop
-	}
-	return nil
-}
-
-func (m *OfpMeterBandHeader) GetDscpRemark() *OfpMeterBandDscpRemark {
-	if x, ok := m.GetData().(*OfpMeterBandHeader_DscpRemark); ok {
-		return x.DscpRemark
-	}
-	return nil
-}
-
-func (m *OfpMeterBandHeader) GetExperimenter() *OfpMeterBandExperimenter {
-	if x, ok := m.GetData().(*OfpMeterBandHeader_Experimenter); ok {
-		return x.Experimenter
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpMeterBandHeader) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpMeterBandHeader_Drop)(nil),
-		(*OfpMeterBandHeader_DscpRemark)(nil),
-		(*OfpMeterBandHeader_Experimenter)(nil),
-	}
-}
-
-// OFPMBT_DROP band - drop packets
-type OfpMeterBandDrop struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterBandDrop) Reset()         { *m = OfpMeterBandDrop{} }
-func (m *OfpMeterBandDrop) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterBandDrop) ProtoMessage()    {}
-func (*OfpMeterBandDrop) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{36}
-}
-
-func (m *OfpMeterBandDrop) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterBandDrop.Unmarshal(m, b)
-}
-func (m *OfpMeterBandDrop) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterBandDrop.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterBandDrop) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterBandDrop.Merge(m, src)
-}
-func (m *OfpMeterBandDrop) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterBandDrop.Size(m)
-}
-func (m *OfpMeterBandDrop) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterBandDrop.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterBandDrop proto.InternalMessageInfo
-
-// OFPMBT_DSCP_REMARK band - Remark DSCP in the IP header
-type OfpMeterBandDscpRemark struct {
-	PrecLevel            uint32   `protobuf:"varint,1,opt,name=prec_level,json=precLevel,proto3" json:"prec_level,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterBandDscpRemark) Reset()         { *m = OfpMeterBandDscpRemark{} }
-func (m *OfpMeterBandDscpRemark) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterBandDscpRemark) ProtoMessage()    {}
-func (*OfpMeterBandDscpRemark) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{37}
-}
-
-func (m *OfpMeterBandDscpRemark) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterBandDscpRemark.Unmarshal(m, b)
-}
-func (m *OfpMeterBandDscpRemark) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterBandDscpRemark.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterBandDscpRemark) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterBandDscpRemark.Merge(m, src)
-}
-func (m *OfpMeterBandDscpRemark) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterBandDscpRemark.Size(m)
-}
-func (m *OfpMeterBandDscpRemark) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterBandDscpRemark.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterBandDscpRemark proto.InternalMessageInfo
-
-func (m *OfpMeterBandDscpRemark) GetPrecLevel() uint32 {
-	if m != nil {
-		return m.PrecLevel
-	}
-	return 0
-}
-
-// OFPMBT_EXPERIMENTER band - Experimenter type.
-// The rest of the band is experimenter-defined.
-type OfpMeterBandExperimenter struct {
-	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterBandExperimenter) Reset()         { *m = OfpMeterBandExperimenter{} }
-func (m *OfpMeterBandExperimenter) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterBandExperimenter) ProtoMessage()    {}
-func (*OfpMeterBandExperimenter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{38}
-}
-
-func (m *OfpMeterBandExperimenter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterBandExperimenter.Unmarshal(m, b)
-}
-func (m *OfpMeterBandExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterBandExperimenter.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterBandExperimenter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterBandExperimenter.Merge(m, src)
-}
-func (m *OfpMeterBandExperimenter) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterBandExperimenter.Size(m)
-}
-func (m *OfpMeterBandExperimenter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterBandExperimenter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterBandExperimenter proto.InternalMessageInfo
-
-func (m *OfpMeterBandExperimenter) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-// Meter configuration. OFPT_METER_MOD.
-type OfpMeterMod struct {
-	Command              OfpMeterModCommand    `protobuf:"varint,1,opt,name=command,proto3,enum=openflow_13.OfpMeterModCommand" json:"command,omitempty"`
-	Flags                uint32                `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
-	MeterId              uint32                `protobuf:"varint,3,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
-	Bands                []*OfpMeterBandHeader `protobuf:"bytes,4,rep,name=bands,proto3" json:"bands,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpMeterMod) Reset()         { *m = OfpMeterMod{} }
-func (m *OfpMeterMod) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterMod) ProtoMessage()    {}
-func (*OfpMeterMod) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{39}
-}
-
-func (m *OfpMeterMod) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterMod.Unmarshal(m, b)
-}
-func (m *OfpMeterMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterMod.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterMod) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterMod.Merge(m, src)
-}
-func (m *OfpMeterMod) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterMod.Size(m)
-}
-func (m *OfpMeterMod) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterMod.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterMod proto.InternalMessageInfo
-
-func (m *OfpMeterMod) GetCommand() OfpMeterModCommand {
-	if m != nil {
-		return m.Command
-	}
-	return OfpMeterModCommand_OFPMC_ADD
-}
-
-func (m *OfpMeterMod) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpMeterMod) GetMeterId() uint32 {
-	if m != nil {
-		return m.MeterId
-	}
-	return 0
-}
-
-func (m *OfpMeterMod) GetBands() []*OfpMeterBandHeader {
-	if m != nil {
-		return m.Bands
-	}
-	return nil
-}
-
-// OFPT_ERROR: Error message (datapath -> controller).
-type OfpErrorMsg struct {
-	Header               *OfpHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
-	Type                 uint32     `protobuf:"varint,2,opt,name=type,proto3" json:"type,omitempty"`
-	Code                 uint32     `protobuf:"varint,3,opt,name=code,proto3" json:"code,omitempty"`
-	Data                 []byte     `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *OfpErrorMsg) Reset()         { *m = OfpErrorMsg{} }
-func (m *OfpErrorMsg) String() string { return proto.CompactTextString(m) }
-func (*OfpErrorMsg) ProtoMessage()    {}
-func (*OfpErrorMsg) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{40}
-}
-
-func (m *OfpErrorMsg) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpErrorMsg.Unmarshal(m, b)
-}
-func (m *OfpErrorMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpErrorMsg.Marshal(b, m, deterministic)
-}
-func (m *OfpErrorMsg) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpErrorMsg.Merge(m, src)
-}
-func (m *OfpErrorMsg) XXX_Size() int {
-	return xxx_messageInfo_OfpErrorMsg.Size(m)
-}
-func (m *OfpErrorMsg) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpErrorMsg.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpErrorMsg proto.InternalMessageInfo
-
-func (m *OfpErrorMsg) GetHeader() *OfpHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *OfpErrorMsg) GetType() uint32 {
-	if m != nil {
-		return m.Type
-	}
-	return 0
-}
-
-func (m *OfpErrorMsg) GetCode() uint32 {
-	if m != nil {
-		return m.Code
-	}
-	return 0
-}
-
-func (m *OfpErrorMsg) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// OFPET_EXPERIMENTER: Error message (datapath -> controller).
-type OfpErrorExperimenterMsg struct {
-	Type                 uint32   `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
-	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
-	Experimenter         uint32   `protobuf:"varint,3,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	Data                 []byte   `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpErrorExperimenterMsg) Reset()         { *m = OfpErrorExperimenterMsg{} }
-func (m *OfpErrorExperimenterMsg) String() string { return proto.CompactTextString(m) }
-func (*OfpErrorExperimenterMsg) ProtoMessage()    {}
-func (*OfpErrorExperimenterMsg) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{41}
-}
-
-func (m *OfpErrorExperimenterMsg) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpErrorExperimenterMsg.Unmarshal(m, b)
-}
-func (m *OfpErrorExperimenterMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpErrorExperimenterMsg.Marshal(b, m, deterministic)
-}
-func (m *OfpErrorExperimenterMsg) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpErrorExperimenterMsg.Merge(m, src)
-}
-func (m *OfpErrorExperimenterMsg) XXX_Size() int {
-	return xxx_messageInfo_OfpErrorExperimenterMsg.Size(m)
-}
-func (m *OfpErrorExperimenterMsg) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpErrorExperimenterMsg.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpErrorExperimenterMsg proto.InternalMessageInfo
-
-func (m *OfpErrorExperimenterMsg) GetType() uint32 {
-	if m != nil {
-		return m.Type
-	}
-	return 0
-}
-
-func (m *OfpErrorExperimenterMsg) GetExpType() uint32 {
-	if m != nil {
-		return m.ExpType
-	}
-	return 0
-}
-
-func (m *OfpErrorExperimenterMsg) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpErrorExperimenterMsg) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-type OfpMultipartRequest struct {
-	//ofp_header header;
-	Type                 OfpMultipartType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMultipartType" json:"type,omitempty"`
-	Flags                uint32           `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
-	Body                 []byte           `protobuf:"bytes,3,opt,name=body,proto3" json:"body,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *OfpMultipartRequest) Reset()         { *m = OfpMultipartRequest{} }
-func (m *OfpMultipartRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpMultipartRequest) ProtoMessage()    {}
-func (*OfpMultipartRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{42}
-}
-
-func (m *OfpMultipartRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMultipartRequest.Unmarshal(m, b)
-}
-func (m *OfpMultipartRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMultipartRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpMultipartRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMultipartRequest.Merge(m, src)
-}
-func (m *OfpMultipartRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpMultipartRequest.Size(m)
-}
-func (m *OfpMultipartRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMultipartRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMultipartRequest proto.InternalMessageInfo
-
-func (m *OfpMultipartRequest) GetType() OfpMultipartType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpMultipartType_OFPMP_DESC
-}
-
-func (m *OfpMultipartRequest) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpMultipartRequest) GetBody() []byte {
-	if m != nil {
-		return m.Body
-	}
-	return nil
-}
-
-type OfpMultipartReply struct {
-	//ofp_header header;
-	Type                 OfpMultipartType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMultipartType" json:"type,omitempty"`
-	Flags                uint32           `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
-	Body                 []byte           `protobuf:"bytes,3,opt,name=body,proto3" json:"body,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *OfpMultipartReply) Reset()         { *m = OfpMultipartReply{} }
-func (m *OfpMultipartReply) String() string { return proto.CompactTextString(m) }
-func (*OfpMultipartReply) ProtoMessage()    {}
-func (*OfpMultipartReply) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{43}
-}
-
-func (m *OfpMultipartReply) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMultipartReply.Unmarshal(m, b)
-}
-func (m *OfpMultipartReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMultipartReply.Marshal(b, m, deterministic)
-}
-func (m *OfpMultipartReply) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMultipartReply.Merge(m, src)
-}
-func (m *OfpMultipartReply) XXX_Size() int {
-	return xxx_messageInfo_OfpMultipartReply.Size(m)
-}
-func (m *OfpMultipartReply) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMultipartReply.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMultipartReply proto.InternalMessageInfo
-
-func (m *OfpMultipartReply) GetType() OfpMultipartType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpMultipartType_OFPMP_DESC
-}
-
-func (m *OfpMultipartReply) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpMultipartReply) GetBody() []byte {
-	if m != nil {
-		return m.Body
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_DESC request.  Each entry is a NULL-terminated
-// ASCII string.
-type OfpDesc struct {
-	MfrDesc              string   `protobuf:"bytes,1,opt,name=mfr_desc,json=mfrDesc,proto3" json:"mfr_desc,omitempty"`
-	HwDesc               string   `protobuf:"bytes,2,opt,name=hw_desc,json=hwDesc,proto3" json:"hw_desc,omitempty"`
-	SwDesc               string   `protobuf:"bytes,3,opt,name=sw_desc,json=swDesc,proto3" json:"sw_desc,omitempty"`
-	SerialNum            string   `protobuf:"bytes,4,opt,name=serial_num,json=serialNum,proto3" json:"serial_num,omitempty"`
-	DpDesc               string   `protobuf:"bytes,5,opt,name=dp_desc,json=dpDesc,proto3" json:"dp_desc,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpDesc) Reset()         { *m = OfpDesc{} }
-func (m *OfpDesc) String() string { return proto.CompactTextString(m) }
-func (*OfpDesc) ProtoMessage()    {}
-func (*OfpDesc) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{44}
-}
-
-func (m *OfpDesc) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpDesc.Unmarshal(m, b)
-}
-func (m *OfpDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpDesc.Marshal(b, m, deterministic)
-}
-func (m *OfpDesc) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpDesc.Merge(m, src)
-}
-func (m *OfpDesc) XXX_Size() int {
-	return xxx_messageInfo_OfpDesc.Size(m)
-}
-func (m *OfpDesc) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpDesc.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpDesc proto.InternalMessageInfo
-
-func (m *OfpDesc) GetMfrDesc() string {
-	if m != nil {
-		return m.MfrDesc
-	}
-	return ""
-}
-
-func (m *OfpDesc) GetHwDesc() string {
-	if m != nil {
-		return m.HwDesc
-	}
-	return ""
-}
-
-func (m *OfpDesc) GetSwDesc() string {
-	if m != nil {
-		return m.SwDesc
-	}
-	return ""
-}
-
-func (m *OfpDesc) GetSerialNum() string {
-	if m != nil {
-		return m.SerialNum
-	}
-	return ""
-}
-
-func (m *OfpDesc) GetDpDesc() string {
-	if m != nil {
-		return m.DpDesc
-	}
-	return ""
-}
-
-// Body for ofp_multipart_request of type OFPMP_FLOW.
-type OfpFlowStatsRequest struct {
-	TableId              uint32    `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	OutPort              uint32    `protobuf:"varint,2,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
-	OutGroup             uint32    `protobuf:"varint,3,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
-	Cookie               uint64    `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	CookieMask           uint64    `protobuf:"varint,5,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
-	Match                *OfpMatch `protobuf:"bytes,6,opt,name=match,proto3" json:"match,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *OfpFlowStatsRequest) Reset()         { *m = OfpFlowStatsRequest{} }
-func (m *OfpFlowStatsRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpFlowStatsRequest) ProtoMessage()    {}
-func (*OfpFlowStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{45}
-}
-
-func (m *OfpFlowStatsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpFlowStatsRequest.Unmarshal(m, b)
-}
-func (m *OfpFlowStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpFlowStatsRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpFlowStatsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpFlowStatsRequest.Merge(m, src)
-}
-func (m *OfpFlowStatsRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpFlowStatsRequest.Size(m)
-}
-func (m *OfpFlowStatsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpFlowStatsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpFlowStatsRequest proto.InternalMessageInfo
-
-func (m *OfpFlowStatsRequest) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpFlowStatsRequest) GetOutPort() uint32 {
-	if m != nil {
-		return m.OutPort
-	}
-	return 0
-}
-
-func (m *OfpFlowStatsRequest) GetOutGroup() uint32 {
-	if m != nil {
-		return m.OutGroup
-	}
-	return 0
-}
-
-func (m *OfpFlowStatsRequest) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpFlowStatsRequest) GetCookieMask() uint64 {
-	if m != nil {
-		return m.CookieMask
-	}
-	return 0
-}
-
-func (m *OfpFlowStatsRequest) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_FLOW request.
-type OfpFlowStats struct {
-	Id                   uint64            `protobuf:"varint,14,opt,name=id,proto3" json:"id,omitempty"`
-	TableId              uint32            `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	DurationSec          uint32            `protobuf:"varint,2,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32            `protobuf:"varint,3,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	Priority             uint32            `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
-	IdleTimeout          uint32            `protobuf:"varint,5,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
-	HardTimeout          uint32            `protobuf:"varint,6,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
-	Flags                uint32            `protobuf:"varint,7,opt,name=flags,proto3" json:"flags,omitempty"`
-	Cookie               uint64            `protobuf:"varint,8,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	PacketCount          uint64            `protobuf:"varint,9,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
-	ByteCount            uint64            `protobuf:"varint,10,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
-	Match                *OfpMatch         `protobuf:"bytes,12,opt,name=match,proto3" json:"match,omitempty"`
-	Instructions         []*OfpInstruction `protobuf:"bytes,13,rep,name=instructions,proto3" json:"instructions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpFlowStats) Reset()         { *m = OfpFlowStats{} }
-func (m *OfpFlowStats) String() string { return proto.CompactTextString(m) }
-func (*OfpFlowStats) ProtoMessage()    {}
-func (*OfpFlowStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{46}
-}
-
-func (m *OfpFlowStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpFlowStats.Unmarshal(m, b)
-}
-func (m *OfpFlowStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpFlowStats.Marshal(b, m, deterministic)
-}
-func (m *OfpFlowStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpFlowStats.Merge(m, src)
-}
-func (m *OfpFlowStats) XXX_Size() int {
-	return xxx_messageInfo_OfpFlowStats.Size(m)
-}
-func (m *OfpFlowStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpFlowStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpFlowStats proto.InternalMessageInfo
-
-func (m *OfpFlowStats) GetId() uint64 {
-	if m != nil {
-		return m.Id
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetIdleTimeout() uint32 {
-	if m != nil {
-		return m.IdleTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetHardTimeout() uint32 {
-	if m != nil {
-		return m.HardTimeout
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetPacketCount() uint64 {
-	if m != nil {
-		return m.PacketCount
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetByteCount() uint64 {
-	if m != nil {
-		return m.ByteCount
-	}
-	return 0
-}
-
-func (m *OfpFlowStats) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-func (m *OfpFlowStats) GetInstructions() []*OfpInstruction {
-	if m != nil {
-		return m.Instructions
-	}
-	return nil
-}
-
-// Body for ofp_multipart_request of type OFPMP_AGGREGATE.
-type OfpAggregateStatsRequest struct {
-	TableId              uint32    `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	OutPort              uint32    `protobuf:"varint,2,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
-	OutGroup             uint32    `protobuf:"varint,3,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
-	Cookie               uint64    `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	CookieMask           uint64    `protobuf:"varint,5,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
-	Match                *OfpMatch `protobuf:"bytes,6,opt,name=match,proto3" json:"match,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *OfpAggregateStatsRequest) Reset()         { *m = OfpAggregateStatsRequest{} }
-func (m *OfpAggregateStatsRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpAggregateStatsRequest) ProtoMessage()    {}
-func (*OfpAggregateStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{47}
-}
-
-func (m *OfpAggregateStatsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpAggregateStatsRequest.Unmarshal(m, b)
-}
-func (m *OfpAggregateStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpAggregateStatsRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpAggregateStatsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpAggregateStatsRequest.Merge(m, src)
-}
-func (m *OfpAggregateStatsRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpAggregateStatsRequest.Size(m)
-}
-func (m *OfpAggregateStatsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpAggregateStatsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpAggregateStatsRequest proto.InternalMessageInfo
-
-func (m *OfpAggregateStatsRequest) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsRequest) GetOutPort() uint32 {
-	if m != nil {
-		return m.OutPort
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsRequest) GetOutGroup() uint32 {
-	if m != nil {
-		return m.OutGroup
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsRequest) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsRequest) GetCookieMask() uint64 {
-	if m != nil {
-		return m.CookieMask
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsRequest) GetMatch() *OfpMatch {
-	if m != nil {
-		return m.Match
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_AGGREGATE request.
-type OfpAggregateStatsReply struct {
-	PacketCount          uint64   `protobuf:"varint,1,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
-	ByteCount            uint64   `protobuf:"varint,2,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
-	FlowCount            uint32   `protobuf:"varint,3,opt,name=flow_count,json=flowCount,proto3" json:"flow_count,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpAggregateStatsReply) Reset()         { *m = OfpAggregateStatsReply{} }
-func (m *OfpAggregateStatsReply) String() string { return proto.CompactTextString(m) }
-func (*OfpAggregateStatsReply) ProtoMessage()    {}
-func (*OfpAggregateStatsReply) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{48}
-}
-
-func (m *OfpAggregateStatsReply) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpAggregateStatsReply.Unmarshal(m, b)
-}
-func (m *OfpAggregateStatsReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpAggregateStatsReply.Marshal(b, m, deterministic)
-}
-func (m *OfpAggregateStatsReply) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpAggregateStatsReply.Merge(m, src)
-}
-func (m *OfpAggregateStatsReply) XXX_Size() int {
-	return xxx_messageInfo_OfpAggregateStatsReply.Size(m)
-}
-func (m *OfpAggregateStatsReply) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpAggregateStatsReply.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpAggregateStatsReply proto.InternalMessageInfo
-
-func (m *OfpAggregateStatsReply) GetPacketCount() uint64 {
-	if m != nil {
-		return m.PacketCount
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsReply) GetByteCount() uint64 {
-	if m != nil {
-		return m.ByteCount
-	}
-	return 0
-}
-
-func (m *OfpAggregateStatsReply) GetFlowCount() uint32 {
-	if m != nil {
-		return m.FlowCount
-	}
-	return 0
-}
-
-// Common header for all Table Feature Properties
-type OfpTableFeatureProperty struct {
-	Type OfpTableFeaturePropType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpTableFeaturePropType" json:"type,omitempty"`
-	// Types that are valid to be assigned to Value:
-	//	*OfpTableFeatureProperty_Instructions
-	//	*OfpTableFeatureProperty_NextTables
-	//	*OfpTableFeatureProperty_Actions
-	//	*OfpTableFeatureProperty_Oxm
-	//	*OfpTableFeatureProperty_Experimenter
-	Value                isOfpTableFeatureProperty_Value `protobuf_oneof:"value"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *OfpTableFeatureProperty) Reset()         { *m = OfpTableFeatureProperty{} }
-func (m *OfpTableFeatureProperty) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeatureProperty) ProtoMessage()    {}
-func (*OfpTableFeatureProperty) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{49}
-}
-
-func (m *OfpTableFeatureProperty) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeatureProperty.Unmarshal(m, b)
-}
-func (m *OfpTableFeatureProperty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeatureProperty.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeatureProperty) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeatureProperty.Merge(m, src)
-}
-func (m *OfpTableFeatureProperty) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeatureProperty.Size(m)
-}
-func (m *OfpTableFeatureProperty) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeatureProperty.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeatureProperty proto.InternalMessageInfo
-
-func (m *OfpTableFeatureProperty) GetType() OfpTableFeaturePropType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS
-}
-
-type isOfpTableFeatureProperty_Value interface {
-	isOfpTableFeatureProperty_Value()
-}
-
-type OfpTableFeatureProperty_Instructions struct {
-	Instructions *OfpTableFeaturePropInstructions `protobuf:"bytes,2,opt,name=instructions,proto3,oneof"`
-}
-
-type OfpTableFeatureProperty_NextTables struct {
-	NextTables *OfpTableFeaturePropNextTables `protobuf:"bytes,3,opt,name=next_tables,json=nextTables,proto3,oneof"`
-}
-
-type OfpTableFeatureProperty_Actions struct {
-	Actions *OfpTableFeaturePropActions `protobuf:"bytes,4,opt,name=actions,proto3,oneof"`
-}
-
-type OfpTableFeatureProperty_Oxm struct {
-	Oxm *OfpTableFeaturePropOxm `protobuf:"bytes,5,opt,name=oxm,proto3,oneof"`
-}
-
-type OfpTableFeatureProperty_Experimenter struct {
-	Experimenter *OfpTableFeaturePropExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
-}
-
-func (*OfpTableFeatureProperty_Instructions) isOfpTableFeatureProperty_Value() {}
-
-func (*OfpTableFeatureProperty_NextTables) isOfpTableFeatureProperty_Value() {}
-
-func (*OfpTableFeatureProperty_Actions) isOfpTableFeatureProperty_Value() {}
-
-func (*OfpTableFeatureProperty_Oxm) isOfpTableFeatureProperty_Value() {}
-
-func (*OfpTableFeatureProperty_Experimenter) isOfpTableFeatureProperty_Value() {}
-
-func (m *OfpTableFeatureProperty) GetValue() isOfpTableFeatureProperty_Value {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func (m *OfpTableFeatureProperty) GetInstructions() *OfpTableFeaturePropInstructions {
-	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Instructions); ok {
-		return x.Instructions
-	}
-	return nil
-}
-
-func (m *OfpTableFeatureProperty) GetNextTables() *OfpTableFeaturePropNextTables {
-	if x, ok := m.GetValue().(*OfpTableFeatureProperty_NextTables); ok {
-		return x.NextTables
-	}
-	return nil
-}
-
-func (m *OfpTableFeatureProperty) GetActions() *OfpTableFeaturePropActions {
-	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Actions); ok {
-		return x.Actions
-	}
-	return nil
-}
-
-func (m *OfpTableFeatureProperty) GetOxm() *OfpTableFeaturePropOxm {
-	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Oxm); ok {
-		return x.Oxm
-	}
-	return nil
-}
-
-func (m *OfpTableFeatureProperty) GetExperimenter() *OfpTableFeaturePropExperimenter {
-	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Experimenter); ok {
-		return x.Experimenter
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OfpTableFeatureProperty) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OfpTableFeatureProperty_Instructions)(nil),
-		(*OfpTableFeatureProperty_NextTables)(nil),
-		(*OfpTableFeatureProperty_Actions)(nil),
-		(*OfpTableFeatureProperty_Oxm)(nil),
-		(*OfpTableFeatureProperty_Experimenter)(nil),
-	}
-}
-
-// Instructions property
-type OfpTableFeaturePropInstructions struct {
-	// One of OFPTFPT_INSTRUCTIONS,
-	//OFPTFPT_INSTRUCTIONS_MISS.
-	Instructions         []*OfpInstruction `protobuf:"bytes,1,rep,name=instructions,proto3" json:"instructions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpTableFeaturePropInstructions) Reset()         { *m = OfpTableFeaturePropInstructions{} }
-func (m *OfpTableFeaturePropInstructions) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeaturePropInstructions) ProtoMessage()    {}
-func (*OfpTableFeaturePropInstructions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{50}
-}
-
-func (m *OfpTableFeaturePropInstructions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeaturePropInstructions.Unmarshal(m, b)
-}
-func (m *OfpTableFeaturePropInstructions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeaturePropInstructions.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeaturePropInstructions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeaturePropInstructions.Merge(m, src)
-}
-func (m *OfpTableFeaturePropInstructions) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeaturePropInstructions.Size(m)
-}
-func (m *OfpTableFeaturePropInstructions) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeaturePropInstructions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeaturePropInstructions proto.InternalMessageInfo
-
-func (m *OfpTableFeaturePropInstructions) GetInstructions() []*OfpInstruction {
-	if m != nil {
-		return m.Instructions
-	}
-	return nil
-}
-
-// Next Tables property
-type OfpTableFeaturePropNextTables struct {
-	// One of OFPTFPT_NEXT_TABLES,
-	//OFPTFPT_NEXT_TABLES_MISS.
-	NextTableIds         []uint32 `protobuf:"varint,1,rep,packed,name=next_table_ids,json=nextTableIds,proto3" json:"next_table_ids,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpTableFeaturePropNextTables) Reset()         { *m = OfpTableFeaturePropNextTables{} }
-func (m *OfpTableFeaturePropNextTables) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeaturePropNextTables) ProtoMessage()    {}
-func (*OfpTableFeaturePropNextTables) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{51}
-}
-
-func (m *OfpTableFeaturePropNextTables) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeaturePropNextTables.Unmarshal(m, b)
-}
-func (m *OfpTableFeaturePropNextTables) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeaturePropNextTables.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeaturePropNextTables) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeaturePropNextTables.Merge(m, src)
-}
-func (m *OfpTableFeaturePropNextTables) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeaturePropNextTables.Size(m)
-}
-func (m *OfpTableFeaturePropNextTables) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeaturePropNextTables.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeaturePropNextTables proto.InternalMessageInfo
-
-func (m *OfpTableFeaturePropNextTables) GetNextTableIds() []uint32 {
-	if m != nil {
-		return m.NextTableIds
-	}
-	return nil
-}
-
-// Actions property
-type OfpTableFeaturePropActions struct {
-	// One of OFPTFPT_WRITE_ACTIONS,
-	//OFPTFPT_WRITE_ACTIONS_MISS,
-	//OFPTFPT_APPLY_ACTIONS,
-	//OFPTFPT_APPLY_ACTIONS_MISS.
-	Actions              []*OfpAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *OfpTableFeaturePropActions) Reset()         { *m = OfpTableFeaturePropActions{} }
-func (m *OfpTableFeaturePropActions) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeaturePropActions) ProtoMessage()    {}
-func (*OfpTableFeaturePropActions) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{52}
-}
-
-func (m *OfpTableFeaturePropActions) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeaturePropActions.Unmarshal(m, b)
-}
-func (m *OfpTableFeaturePropActions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeaturePropActions.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeaturePropActions) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeaturePropActions.Merge(m, src)
-}
-func (m *OfpTableFeaturePropActions) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeaturePropActions.Size(m)
-}
-func (m *OfpTableFeaturePropActions) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeaturePropActions.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeaturePropActions proto.InternalMessageInfo
-
-func (m *OfpTableFeaturePropActions) GetActions() []*OfpAction {
-	if m != nil {
-		return m.Actions
-	}
-	return nil
-}
-
-// Match, Wildcard or Set-Field property
-type OfpTableFeaturePropOxm struct {
-	// TODO is this a uint32???
-	OxmIds               []uint32 `protobuf:"varint,3,rep,packed,name=oxm_ids,json=oxmIds,proto3" json:"oxm_ids,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpTableFeaturePropOxm) Reset()         { *m = OfpTableFeaturePropOxm{} }
-func (m *OfpTableFeaturePropOxm) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeaturePropOxm) ProtoMessage()    {}
-func (*OfpTableFeaturePropOxm) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{53}
-}
-
-func (m *OfpTableFeaturePropOxm) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeaturePropOxm.Unmarshal(m, b)
-}
-func (m *OfpTableFeaturePropOxm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeaturePropOxm.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeaturePropOxm) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeaturePropOxm.Merge(m, src)
-}
-func (m *OfpTableFeaturePropOxm) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeaturePropOxm.Size(m)
-}
-func (m *OfpTableFeaturePropOxm) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeaturePropOxm.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeaturePropOxm proto.InternalMessageInfo
-
-func (m *OfpTableFeaturePropOxm) GetOxmIds() []uint32 {
-	if m != nil {
-		return m.OxmIds
-	}
-	return nil
-}
-
-// Experimenter table feature property
-type OfpTableFeaturePropExperimenter struct {
-	// One of OFPTFPT_EXPERIMENTER,
-	//OFPTFPT_EXPERIMENTER_MISS.
-	Experimenter         uint32   `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	ExpType              uint32   `protobuf:"varint,3,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
-	ExperimenterData     []uint32 `protobuf:"varint,4,rep,packed,name=experimenter_data,json=experimenterData,proto3" json:"experimenter_data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpTableFeaturePropExperimenter) Reset()         { *m = OfpTableFeaturePropExperimenter{} }
-func (m *OfpTableFeaturePropExperimenter) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeaturePropExperimenter) ProtoMessage()    {}
-func (*OfpTableFeaturePropExperimenter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{54}
-}
-
-func (m *OfpTableFeaturePropExperimenter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Unmarshal(m, b)
-}
-func (m *OfpTableFeaturePropExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeaturePropExperimenter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeaturePropExperimenter.Merge(m, src)
-}
-func (m *OfpTableFeaturePropExperimenter) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Size(m)
-}
-func (m *OfpTableFeaturePropExperimenter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeaturePropExperimenter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeaturePropExperimenter proto.InternalMessageInfo
-
-func (m *OfpTableFeaturePropExperimenter) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpTableFeaturePropExperimenter) GetExpType() uint32 {
-	if m != nil {
-		return m.ExpType
-	}
-	return 0
-}
-
-func (m *OfpTableFeaturePropExperimenter) GetExperimenterData() []uint32 {
-	if m != nil {
-		return m.ExperimenterData
-	}
-	return nil
-}
-
-// Body for ofp_multipart_request of type OFPMP_TABLE_FEATURES./
-// Body of reply to OFPMP_TABLE_FEATURES request.
-type OfpTableFeatures struct {
-	TableId       uint32 `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	Name          string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	MetadataMatch uint64 `protobuf:"varint,3,opt,name=metadata_match,json=metadataMatch,proto3" json:"metadata_match,omitempty"`
-	MetadataWrite uint64 `protobuf:"varint,4,opt,name=metadata_write,json=metadataWrite,proto3" json:"metadata_write,omitempty"`
-	Config        uint32 `protobuf:"varint,5,opt,name=config,proto3" json:"config,omitempty"`
-	MaxEntries    uint32 `protobuf:"varint,6,opt,name=max_entries,json=maxEntries,proto3" json:"max_entries,omitempty"`
-	// Table Feature Property list
-	Properties           []*OfpTableFeatureProperty `protobuf:"bytes,7,rep,name=properties,proto3" json:"properties,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *OfpTableFeatures) Reset()         { *m = OfpTableFeatures{} }
-func (m *OfpTableFeatures) String() string { return proto.CompactTextString(m) }
-func (*OfpTableFeatures) ProtoMessage()    {}
-func (*OfpTableFeatures) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{55}
-}
-
-func (m *OfpTableFeatures) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableFeatures.Unmarshal(m, b)
-}
-func (m *OfpTableFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableFeatures.Marshal(b, m, deterministic)
-}
-func (m *OfpTableFeatures) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableFeatures.Merge(m, src)
-}
-func (m *OfpTableFeatures) XXX_Size() int {
-	return xxx_messageInfo_OfpTableFeatures.Size(m)
-}
-func (m *OfpTableFeatures) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableFeatures.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableFeatures proto.InternalMessageInfo
-
-func (m *OfpTableFeatures) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpTableFeatures) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *OfpTableFeatures) GetMetadataMatch() uint64 {
-	if m != nil {
-		return m.MetadataMatch
-	}
-	return 0
-}
-
-func (m *OfpTableFeatures) GetMetadataWrite() uint64 {
-	if m != nil {
-		return m.MetadataWrite
-	}
-	return 0
-}
-
-func (m *OfpTableFeatures) GetConfig() uint32 {
-	if m != nil {
-		return m.Config
-	}
-	return 0
-}
-
-func (m *OfpTableFeatures) GetMaxEntries() uint32 {
-	if m != nil {
-		return m.MaxEntries
-	}
-	return 0
-}
-
-func (m *OfpTableFeatures) GetProperties() []*OfpTableFeatureProperty {
-	if m != nil {
-		return m.Properties
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_TABLE request.
-type OfpTableStats struct {
-	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
-	ActiveCount          uint32   `protobuf:"varint,2,opt,name=active_count,json=activeCount,proto3" json:"active_count,omitempty"`
-	LookupCount          uint64   `protobuf:"varint,3,opt,name=lookup_count,json=lookupCount,proto3" json:"lookup_count,omitempty"`
-	MatchedCount         uint64   `protobuf:"varint,4,opt,name=matched_count,json=matchedCount,proto3" json:"matched_count,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpTableStats) Reset()         { *m = OfpTableStats{} }
-func (m *OfpTableStats) String() string { return proto.CompactTextString(m) }
-func (*OfpTableStats) ProtoMessage()    {}
-func (*OfpTableStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{56}
-}
-
-func (m *OfpTableStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpTableStats.Unmarshal(m, b)
-}
-func (m *OfpTableStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpTableStats.Marshal(b, m, deterministic)
-}
-func (m *OfpTableStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpTableStats.Merge(m, src)
-}
-func (m *OfpTableStats) XXX_Size() int {
-	return xxx_messageInfo_OfpTableStats.Size(m)
-}
-func (m *OfpTableStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpTableStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpTableStats proto.InternalMessageInfo
-
-func (m *OfpTableStats) GetTableId() uint32 {
-	if m != nil {
-		return m.TableId
-	}
-	return 0
-}
-
-func (m *OfpTableStats) GetActiveCount() uint32 {
-	if m != nil {
-		return m.ActiveCount
-	}
-	return 0
-}
-
-func (m *OfpTableStats) GetLookupCount() uint64 {
-	if m != nil {
-		return m.LookupCount
-	}
-	return 0
-}
-
-func (m *OfpTableStats) GetMatchedCount() uint64 {
-	if m != nil {
-		return m.MatchedCount
-	}
-	return 0
-}
-
-// Body for ofp_multipart_request of type OFPMP_PORT.
-type OfpPortStatsRequest struct {
-	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpPortStatsRequest) Reset()         { *m = OfpPortStatsRequest{} }
-func (m *OfpPortStatsRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpPortStatsRequest) ProtoMessage()    {}
-func (*OfpPortStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{57}
-}
-
-func (m *OfpPortStatsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPortStatsRequest.Unmarshal(m, b)
-}
-func (m *OfpPortStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPortStatsRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpPortStatsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPortStatsRequest.Merge(m, src)
-}
-func (m *OfpPortStatsRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpPortStatsRequest.Size(m)
-}
-func (m *OfpPortStatsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPortStatsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPortStatsRequest proto.InternalMessageInfo
-
-func (m *OfpPortStatsRequest) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-// Body of reply to OFPMP_PORT request. If a counter is unsupported, set
-// the field to all ones.
-type OfpPortStats struct {
-	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	RxPackets            uint64   `protobuf:"varint,2,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	TxPackets            uint64   `protobuf:"varint,3,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	RxBytes              uint64   `protobuf:"varint,4,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	TxBytes              uint64   `protobuf:"varint,5,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	RxDropped            uint64   `protobuf:"varint,6,opt,name=rx_dropped,json=rxDropped,proto3" json:"rx_dropped,omitempty"`
-	TxDropped            uint64   `protobuf:"varint,7,opt,name=tx_dropped,json=txDropped,proto3" json:"tx_dropped,omitempty"`
-	RxErrors             uint64   `protobuf:"varint,8,opt,name=rx_errors,json=rxErrors,proto3" json:"rx_errors,omitempty"`
-	TxErrors             uint64   `protobuf:"varint,9,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
-	RxFrameErr           uint64   `protobuf:"varint,10,opt,name=rx_frame_err,json=rxFrameErr,proto3" json:"rx_frame_err,omitempty"`
-	RxOverErr            uint64   `protobuf:"varint,11,opt,name=rx_over_err,json=rxOverErr,proto3" json:"rx_over_err,omitempty"`
-	RxCrcErr             uint64   `protobuf:"varint,12,opt,name=rx_crc_err,json=rxCrcErr,proto3" json:"rx_crc_err,omitempty"`
-	Collisions           uint64   `protobuf:"varint,13,opt,name=collisions,proto3" json:"collisions,omitempty"`
-	DurationSec          uint32   `protobuf:"varint,14,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32   `protobuf:"varint,15,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpPortStats) Reset()         { *m = OfpPortStats{} }
-func (m *OfpPortStats) String() string { return proto.CompactTextString(m) }
-func (*OfpPortStats) ProtoMessage()    {}
-func (*OfpPortStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{58}
-}
-
-func (m *OfpPortStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPortStats.Unmarshal(m, b)
-}
-func (m *OfpPortStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPortStats.Marshal(b, m, deterministic)
-}
-func (m *OfpPortStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPortStats.Merge(m, src)
-}
-func (m *OfpPortStats) XXX_Size() int {
-	return xxx_messageInfo_OfpPortStats.Size(m)
-}
-func (m *OfpPortStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPortStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPortStats proto.InternalMessageInfo
-
-func (m *OfpPortStats) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxDropped() uint64 {
-	if m != nil {
-		return m.RxDropped
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetTxDropped() uint64 {
-	if m != nil {
-		return m.TxDropped
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxErrors() uint64 {
-	if m != nil {
-		return m.RxErrors
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetTxErrors() uint64 {
-	if m != nil {
-		return m.TxErrors
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxFrameErr() uint64 {
-	if m != nil {
-		return m.RxFrameErr
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxOverErr() uint64 {
-	if m != nil {
-		return m.RxOverErr
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetRxCrcErr() uint64 {
-	if m != nil {
-		return m.RxCrcErr
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetCollisions() uint64 {
-	if m != nil {
-		return m.Collisions
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpPortStats) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-// Body of OFPMP_GROUP request.
-type OfpGroupStatsRequest struct {
-	GroupId              uint32   `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpGroupStatsRequest) Reset()         { *m = OfpGroupStatsRequest{} }
-func (m *OfpGroupStatsRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupStatsRequest) ProtoMessage()    {}
-func (*OfpGroupStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{59}
-}
-
-func (m *OfpGroupStatsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupStatsRequest.Unmarshal(m, b)
-}
-func (m *OfpGroupStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupStatsRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupStatsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupStatsRequest.Merge(m, src)
-}
-func (m *OfpGroupStatsRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupStatsRequest.Size(m)
-}
-func (m *OfpGroupStatsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupStatsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupStatsRequest proto.InternalMessageInfo
-
-func (m *OfpGroupStatsRequest) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-// Used in group stats replies.
-type OfpBucketCounter struct {
-	PacketCount          uint64   `protobuf:"varint,1,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
-	ByteCount            uint64   `protobuf:"varint,2,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpBucketCounter) Reset()         { *m = OfpBucketCounter{} }
-func (m *OfpBucketCounter) String() string { return proto.CompactTextString(m) }
-func (*OfpBucketCounter) ProtoMessage()    {}
-func (*OfpBucketCounter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{60}
-}
-
-func (m *OfpBucketCounter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpBucketCounter.Unmarshal(m, b)
-}
-func (m *OfpBucketCounter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpBucketCounter.Marshal(b, m, deterministic)
-}
-func (m *OfpBucketCounter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpBucketCounter.Merge(m, src)
-}
-func (m *OfpBucketCounter) XXX_Size() int {
-	return xxx_messageInfo_OfpBucketCounter.Size(m)
-}
-func (m *OfpBucketCounter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpBucketCounter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpBucketCounter proto.InternalMessageInfo
-
-func (m *OfpBucketCounter) GetPacketCount() uint64 {
-	if m != nil {
-		return m.PacketCount
-	}
-	return 0
-}
-
-func (m *OfpBucketCounter) GetByteCount() uint64 {
-	if m != nil {
-		return m.ByteCount
-	}
-	return 0
-}
-
-// Body of reply to OFPMP_GROUP request.
-type OfpGroupStats struct {
-	GroupId              uint32              `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	RefCount             uint32              `protobuf:"varint,2,opt,name=ref_count,json=refCount,proto3" json:"ref_count,omitempty"`
-	PacketCount          uint64              `protobuf:"varint,3,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
-	ByteCount            uint64              `protobuf:"varint,4,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
-	DurationSec          uint32              `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32              `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	BucketStats          []*OfpBucketCounter `protobuf:"bytes,7,rep,name=bucket_stats,json=bucketStats,proto3" json:"bucket_stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *OfpGroupStats) Reset()         { *m = OfpGroupStats{} }
-func (m *OfpGroupStats) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupStats) ProtoMessage()    {}
-func (*OfpGroupStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{61}
-}
-
-func (m *OfpGroupStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupStats.Unmarshal(m, b)
-}
-func (m *OfpGroupStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupStats.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupStats.Merge(m, src)
-}
-func (m *OfpGroupStats) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupStats.Size(m)
-}
-func (m *OfpGroupStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupStats proto.InternalMessageInfo
-
-func (m *OfpGroupStats) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetRefCount() uint32 {
-	if m != nil {
-		return m.RefCount
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetPacketCount() uint64 {
-	if m != nil {
-		return m.PacketCount
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetByteCount() uint64 {
-	if m != nil {
-		return m.ByteCount
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-func (m *OfpGroupStats) GetBucketStats() []*OfpBucketCounter {
-	if m != nil {
-		return m.BucketStats
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_GROUP_DESC request.
-type OfpGroupDesc struct {
-	Type                 OfpGroupType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpGroupType" json:"type,omitempty"`
-	GroupId              uint32       `protobuf:"varint,2,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	Buckets              []*OfpBucket `protobuf:"bytes,3,rep,name=buckets,proto3" json:"buckets,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *OfpGroupDesc) Reset()         { *m = OfpGroupDesc{} }
-func (m *OfpGroupDesc) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupDesc) ProtoMessage()    {}
-func (*OfpGroupDesc) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{62}
-}
-
-func (m *OfpGroupDesc) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupDesc.Unmarshal(m, b)
-}
-func (m *OfpGroupDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupDesc.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupDesc) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupDesc.Merge(m, src)
-}
-func (m *OfpGroupDesc) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupDesc.Size(m)
-}
-func (m *OfpGroupDesc) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupDesc.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupDesc proto.InternalMessageInfo
-
-func (m *OfpGroupDesc) GetType() OfpGroupType {
-	if m != nil {
-		return m.Type
-	}
-	return OfpGroupType_OFPGT_ALL
-}
-
-func (m *OfpGroupDesc) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-func (m *OfpGroupDesc) GetBuckets() []*OfpBucket {
-	if m != nil {
-		return m.Buckets
-	}
-	return nil
-}
-
-type OfpGroupEntry struct {
-	Desc                 *OfpGroupDesc  `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"`
-	Stats                *OfpGroupStats `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *OfpGroupEntry) Reset()         { *m = OfpGroupEntry{} }
-func (m *OfpGroupEntry) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupEntry) ProtoMessage()    {}
-func (*OfpGroupEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{63}
-}
-
-func (m *OfpGroupEntry) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupEntry.Unmarshal(m, b)
-}
-func (m *OfpGroupEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupEntry.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupEntry.Merge(m, src)
-}
-func (m *OfpGroupEntry) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupEntry.Size(m)
-}
-func (m *OfpGroupEntry) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupEntry.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupEntry proto.InternalMessageInfo
-
-func (m *OfpGroupEntry) GetDesc() *OfpGroupDesc {
-	if m != nil {
-		return m.Desc
-	}
-	return nil
-}
-
-func (m *OfpGroupEntry) GetStats() *OfpGroupStats {
-	if m != nil {
-		return m.Stats
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_GROUP_FEATURES request. Group features.
-type OfpGroupFeatures struct {
-	Types                uint32   `protobuf:"varint,1,opt,name=types,proto3" json:"types,omitempty"`
-	Capabilities         uint32   `protobuf:"varint,2,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
-	MaxGroups            []uint32 `protobuf:"varint,3,rep,packed,name=max_groups,json=maxGroups,proto3" json:"max_groups,omitempty"`
-	Actions              []uint32 `protobuf:"varint,4,rep,packed,name=actions,proto3" json:"actions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpGroupFeatures) Reset()         { *m = OfpGroupFeatures{} }
-func (m *OfpGroupFeatures) String() string { return proto.CompactTextString(m) }
-func (*OfpGroupFeatures) ProtoMessage()    {}
-func (*OfpGroupFeatures) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{64}
-}
-
-func (m *OfpGroupFeatures) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpGroupFeatures.Unmarshal(m, b)
-}
-func (m *OfpGroupFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpGroupFeatures.Marshal(b, m, deterministic)
-}
-func (m *OfpGroupFeatures) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpGroupFeatures.Merge(m, src)
-}
-func (m *OfpGroupFeatures) XXX_Size() int {
-	return xxx_messageInfo_OfpGroupFeatures.Size(m)
-}
-func (m *OfpGroupFeatures) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpGroupFeatures.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpGroupFeatures proto.InternalMessageInfo
-
-func (m *OfpGroupFeatures) GetTypes() uint32 {
-	if m != nil {
-		return m.Types
-	}
-	return 0
-}
-
-func (m *OfpGroupFeatures) GetCapabilities() uint32 {
-	if m != nil {
-		return m.Capabilities
-	}
-	return 0
-}
-
-func (m *OfpGroupFeatures) GetMaxGroups() []uint32 {
-	if m != nil {
-		return m.MaxGroups
-	}
-	return nil
-}
-
-func (m *OfpGroupFeatures) GetActions() []uint32 {
-	if m != nil {
-		return m.Actions
-	}
-	return nil
-}
-
-// Body of OFPMP_METER and OFPMP_METER_CONFIG requests.
-type OfpMeterMultipartRequest struct {
-	MeterId              uint32   `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterMultipartRequest) Reset()         { *m = OfpMeterMultipartRequest{} }
-func (m *OfpMeterMultipartRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterMultipartRequest) ProtoMessage()    {}
-func (*OfpMeterMultipartRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{65}
-}
-
-func (m *OfpMeterMultipartRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterMultipartRequest.Unmarshal(m, b)
-}
-func (m *OfpMeterMultipartRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterMultipartRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterMultipartRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterMultipartRequest.Merge(m, src)
-}
-func (m *OfpMeterMultipartRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterMultipartRequest.Size(m)
-}
-func (m *OfpMeterMultipartRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterMultipartRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterMultipartRequest proto.InternalMessageInfo
-
-func (m *OfpMeterMultipartRequest) GetMeterId() uint32 {
-	if m != nil {
-		return m.MeterId
-	}
-	return 0
-}
-
-// Statistics for each meter band
-type OfpMeterBandStats struct {
-	PacketBandCount      uint64   `protobuf:"varint,1,opt,name=packet_band_count,json=packetBandCount,proto3" json:"packet_band_count,omitempty"`
-	ByteBandCount        uint64   `protobuf:"varint,2,opt,name=byte_band_count,json=byteBandCount,proto3" json:"byte_band_count,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterBandStats) Reset()         { *m = OfpMeterBandStats{} }
-func (m *OfpMeterBandStats) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterBandStats) ProtoMessage()    {}
-func (*OfpMeterBandStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{66}
-}
-
-func (m *OfpMeterBandStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterBandStats.Unmarshal(m, b)
-}
-func (m *OfpMeterBandStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterBandStats.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterBandStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterBandStats.Merge(m, src)
-}
-func (m *OfpMeterBandStats) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterBandStats.Size(m)
-}
-func (m *OfpMeterBandStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterBandStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterBandStats proto.InternalMessageInfo
-
-func (m *OfpMeterBandStats) GetPacketBandCount() uint64 {
-	if m != nil {
-		return m.PacketBandCount
-	}
-	return 0
-}
-
-func (m *OfpMeterBandStats) GetByteBandCount() uint64 {
-	if m != nil {
-		return m.ByteBandCount
-	}
-	return 0
-}
-
-// Body of reply to OFPMP_METER request. Meter statistics.
-type OfpMeterStats struct {
-	MeterId              uint32               `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
-	FlowCount            uint32               `protobuf:"varint,2,opt,name=flow_count,json=flowCount,proto3" json:"flow_count,omitempty"`
-	PacketInCount        uint64               `protobuf:"varint,3,opt,name=packet_in_count,json=packetInCount,proto3" json:"packet_in_count,omitempty"`
-	ByteInCount          uint64               `protobuf:"varint,4,opt,name=byte_in_count,json=byteInCount,proto3" json:"byte_in_count,omitempty"`
-	DurationSec          uint32               `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32               `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	BandStats            []*OfpMeterBandStats `protobuf:"bytes,7,rep,name=band_stats,json=bandStats,proto3" json:"band_stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *OfpMeterStats) Reset()         { *m = OfpMeterStats{} }
-func (m *OfpMeterStats) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterStats) ProtoMessage()    {}
-func (*OfpMeterStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{67}
-}
-
-func (m *OfpMeterStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterStats.Unmarshal(m, b)
-}
-func (m *OfpMeterStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterStats.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterStats.Merge(m, src)
-}
-func (m *OfpMeterStats) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterStats.Size(m)
-}
-func (m *OfpMeterStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterStats proto.InternalMessageInfo
-
-func (m *OfpMeterStats) GetMeterId() uint32 {
-	if m != nil {
-		return m.MeterId
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetFlowCount() uint32 {
-	if m != nil {
-		return m.FlowCount
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetPacketInCount() uint64 {
-	if m != nil {
-		return m.PacketInCount
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetByteInCount() uint64 {
-	if m != nil {
-		return m.ByteInCount
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-func (m *OfpMeterStats) GetBandStats() []*OfpMeterBandStats {
-	if m != nil {
-		return m.BandStats
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_METER_CONFIG request. Meter configuration.
-type OfpMeterConfig struct {
-	Flags                uint32                `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"`
-	MeterId              uint32                `protobuf:"varint,2,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
-	Bands                []*OfpMeterBandHeader `protobuf:"bytes,3,rep,name=bands,proto3" json:"bands,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpMeterConfig) Reset()         { *m = OfpMeterConfig{} }
-func (m *OfpMeterConfig) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterConfig) ProtoMessage()    {}
-func (*OfpMeterConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{68}
-}
-
-func (m *OfpMeterConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterConfig.Unmarshal(m, b)
-}
-func (m *OfpMeterConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterConfig.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterConfig.Merge(m, src)
-}
-func (m *OfpMeterConfig) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterConfig.Size(m)
-}
-func (m *OfpMeterConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterConfig proto.InternalMessageInfo
-
-func (m *OfpMeterConfig) GetFlags() uint32 {
-	if m != nil {
-		return m.Flags
-	}
-	return 0
-}
-
-func (m *OfpMeterConfig) GetMeterId() uint32 {
-	if m != nil {
-		return m.MeterId
-	}
-	return 0
-}
-
-func (m *OfpMeterConfig) GetBands() []*OfpMeterBandHeader {
-	if m != nil {
-		return m.Bands
-	}
-	return nil
-}
-
-// Body of reply to OFPMP_METER_FEATURES request. Meter features.
-type OfpMeterFeatures struct {
-	MaxMeter             uint32   `protobuf:"varint,1,opt,name=max_meter,json=maxMeter,proto3" json:"max_meter,omitempty"`
-	BandTypes            uint32   `protobuf:"varint,2,opt,name=band_types,json=bandTypes,proto3" json:"band_types,omitempty"`
-	Capabilities         uint32   `protobuf:"varint,3,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
-	MaxBands             uint32   `protobuf:"varint,4,opt,name=max_bands,json=maxBands,proto3" json:"max_bands,omitempty"`
-	MaxColor             uint32   `protobuf:"varint,5,opt,name=max_color,json=maxColor,proto3" json:"max_color,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpMeterFeatures) Reset()         { *m = OfpMeterFeatures{} }
-func (m *OfpMeterFeatures) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterFeatures) ProtoMessage()    {}
-func (*OfpMeterFeatures) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{69}
-}
-
-func (m *OfpMeterFeatures) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterFeatures.Unmarshal(m, b)
-}
-func (m *OfpMeterFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterFeatures.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterFeatures) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterFeatures.Merge(m, src)
-}
-func (m *OfpMeterFeatures) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterFeatures.Size(m)
-}
-func (m *OfpMeterFeatures) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterFeatures.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterFeatures proto.InternalMessageInfo
-
-func (m *OfpMeterFeatures) GetMaxMeter() uint32 {
-	if m != nil {
-		return m.MaxMeter
-	}
-	return 0
-}
-
-func (m *OfpMeterFeatures) GetBandTypes() uint32 {
-	if m != nil {
-		return m.BandTypes
-	}
-	return 0
-}
-
-func (m *OfpMeterFeatures) GetCapabilities() uint32 {
-	if m != nil {
-		return m.Capabilities
-	}
-	return 0
-}
-
-func (m *OfpMeterFeatures) GetMaxBands() uint32 {
-	if m != nil {
-		return m.MaxBands
-	}
-	return 0
-}
-
-func (m *OfpMeterFeatures) GetMaxColor() uint32 {
-	if m != nil {
-		return m.MaxColor
-	}
-	return 0
-}
-
-type OfpMeterEntry struct {
-	Config               *OfpMeterConfig `protobuf:"bytes,1,opt,name=config,proto3" json:"config,omitempty"`
-	Stats                *OfpMeterStats  `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *OfpMeterEntry) Reset()         { *m = OfpMeterEntry{} }
-func (m *OfpMeterEntry) String() string { return proto.CompactTextString(m) }
-func (*OfpMeterEntry) ProtoMessage()    {}
-func (*OfpMeterEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{70}
-}
-
-func (m *OfpMeterEntry) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpMeterEntry.Unmarshal(m, b)
-}
-func (m *OfpMeterEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpMeterEntry.Marshal(b, m, deterministic)
-}
-func (m *OfpMeterEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpMeterEntry.Merge(m, src)
-}
-func (m *OfpMeterEntry) XXX_Size() int {
-	return xxx_messageInfo_OfpMeterEntry.Size(m)
-}
-func (m *OfpMeterEntry) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpMeterEntry.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpMeterEntry proto.InternalMessageInfo
-
-func (m *OfpMeterEntry) GetConfig() *OfpMeterConfig {
-	if m != nil {
-		return m.Config
-	}
-	return nil
-}
-
-func (m *OfpMeterEntry) GetStats() *OfpMeterStats {
-	if m != nil {
-		return m.Stats
-	}
-	return nil
-}
-
-// Body for ofp_multipart_request/reply of type OFPMP_EXPERIMENTER.
-type OfpExperimenterMultipartHeader struct {
-	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
-	Data                 []byte   `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpExperimenterMultipartHeader) Reset()         { *m = OfpExperimenterMultipartHeader{} }
-func (m *OfpExperimenterMultipartHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpExperimenterMultipartHeader) ProtoMessage()    {}
-func (*OfpExperimenterMultipartHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{71}
-}
-
-func (m *OfpExperimenterMultipartHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpExperimenterMultipartHeader.Unmarshal(m, b)
-}
-func (m *OfpExperimenterMultipartHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpExperimenterMultipartHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpExperimenterMultipartHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpExperimenterMultipartHeader.Merge(m, src)
-}
-func (m *OfpExperimenterMultipartHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpExperimenterMultipartHeader.Size(m)
-}
-func (m *OfpExperimenterMultipartHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpExperimenterMultipartHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpExperimenterMultipartHeader proto.InternalMessageInfo
-
-func (m *OfpExperimenterMultipartHeader) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpExperimenterMultipartHeader) GetExpType() uint32 {
-	if m != nil {
-		return m.ExpType
-	}
-	return 0
-}
-
-func (m *OfpExperimenterMultipartHeader) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Experimenter extension.
-type OfpExperimenterHeader struct {
-	//ofp_header header;  /* Type OFPT_EXPERIMENTER. */
-	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
-	Data                 []byte   `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpExperimenterHeader) Reset()         { *m = OfpExperimenterHeader{} }
-func (m *OfpExperimenterHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpExperimenterHeader) ProtoMessage()    {}
-func (*OfpExperimenterHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{72}
-}
-
-func (m *OfpExperimenterHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpExperimenterHeader.Unmarshal(m, b)
-}
-func (m *OfpExperimenterHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpExperimenterHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpExperimenterHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpExperimenterHeader.Merge(m, src)
-}
-func (m *OfpExperimenterHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpExperimenterHeader.Size(m)
-}
-func (m *OfpExperimenterHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpExperimenterHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpExperimenterHeader proto.InternalMessageInfo
-
-func (m *OfpExperimenterHeader) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpExperimenterHeader) GetExpType() uint32 {
-	if m != nil {
-		return m.ExpType
-	}
-	return 0
-}
-
-func (m *OfpExperimenterHeader) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Common description for a queue.
-type OfpQueuePropHeader struct {
-	Property             uint32   `protobuf:"varint,1,opt,name=property,proto3" json:"property,omitempty"`
-	Len                  uint32   `protobuf:"varint,2,opt,name=len,proto3" json:"len,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpQueuePropHeader) Reset()         { *m = OfpQueuePropHeader{} }
-func (m *OfpQueuePropHeader) String() string { return proto.CompactTextString(m) }
-func (*OfpQueuePropHeader) ProtoMessage()    {}
-func (*OfpQueuePropHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{73}
-}
-
-func (m *OfpQueuePropHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueuePropHeader.Unmarshal(m, b)
-}
-func (m *OfpQueuePropHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueuePropHeader.Marshal(b, m, deterministic)
-}
-func (m *OfpQueuePropHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueuePropHeader.Merge(m, src)
-}
-func (m *OfpQueuePropHeader) XXX_Size() int {
-	return xxx_messageInfo_OfpQueuePropHeader.Size(m)
-}
-func (m *OfpQueuePropHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueuePropHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueuePropHeader proto.InternalMessageInfo
-
-func (m *OfpQueuePropHeader) GetProperty() uint32 {
-	if m != nil {
-		return m.Property
-	}
-	return 0
-}
-
-func (m *OfpQueuePropHeader) GetLen() uint32 {
-	if m != nil {
-		return m.Len
-	}
-	return 0
-}
-
-// Min-Rate queue property description.
-type OfpQueuePropMinRate struct {
-	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
-	Rate                 uint32              `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *OfpQueuePropMinRate) Reset()         { *m = OfpQueuePropMinRate{} }
-func (m *OfpQueuePropMinRate) String() string { return proto.CompactTextString(m) }
-func (*OfpQueuePropMinRate) ProtoMessage()    {}
-func (*OfpQueuePropMinRate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{74}
-}
-
-func (m *OfpQueuePropMinRate) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueuePropMinRate.Unmarshal(m, b)
-}
-func (m *OfpQueuePropMinRate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueuePropMinRate.Marshal(b, m, deterministic)
-}
-func (m *OfpQueuePropMinRate) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueuePropMinRate.Merge(m, src)
-}
-func (m *OfpQueuePropMinRate) XXX_Size() int {
-	return xxx_messageInfo_OfpQueuePropMinRate.Size(m)
-}
-func (m *OfpQueuePropMinRate) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueuePropMinRate.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueuePropMinRate proto.InternalMessageInfo
-
-func (m *OfpQueuePropMinRate) GetPropHeader() *OfpQueuePropHeader {
-	if m != nil {
-		return m.PropHeader
-	}
-	return nil
-}
-
-func (m *OfpQueuePropMinRate) GetRate() uint32 {
-	if m != nil {
-		return m.Rate
-	}
-	return 0
-}
-
-// Max-Rate queue property description.
-type OfpQueuePropMaxRate struct {
-	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
-	Rate                 uint32              `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *OfpQueuePropMaxRate) Reset()         { *m = OfpQueuePropMaxRate{} }
-func (m *OfpQueuePropMaxRate) String() string { return proto.CompactTextString(m) }
-func (*OfpQueuePropMaxRate) ProtoMessage()    {}
-func (*OfpQueuePropMaxRate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{75}
-}
-
-func (m *OfpQueuePropMaxRate) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueuePropMaxRate.Unmarshal(m, b)
-}
-func (m *OfpQueuePropMaxRate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueuePropMaxRate.Marshal(b, m, deterministic)
-}
-func (m *OfpQueuePropMaxRate) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueuePropMaxRate.Merge(m, src)
-}
-func (m *OfpQueuePropMaxRate) XXX_Size() int {
-	return xxx_messageInfo_OfpQueuePropMaxRate.Size(m)
-}
-func (m *OfpQueuePropMaxRate) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueuePropMaxRate.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueuePropMaxRate proto.InternalMessageInfo
-
-func (m *OfpQueuePropMaxRate) GetPropHeader() *OfpQueuePropHeader {
-	if m != nil {
-		return m.PropHeader
-	}
-	return nil
-}
-
-func (m *OfpQueuePropMaxRate) GetRate() uint32 {
-	if m != nil {
-		return m.Rate
-	}
-	return 0
-}
-
-// Experimenter queue property description.
-type OfpQueuePropExperimenter struct {
-	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
-	Experimenter         uint32              `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
-	Data                 []byte              `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *OfpQueuePropExperimenter) Reset()         { *m = OfpQueuePropExperimenter{} }
-func (m *OfpQueuePropExperimenter) String() string { return proto.CompactTextString(m) }
-func (*OfpQueuePropExperimenter) ProtoMessage()    {}
-func (*OfpQueuePropExperimenter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{76}
-}
-
-func (m *OfpQueuePropExperimenter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueuePropExperimenter.Unmarshal(m, b)
-}
-func (m *OfpQueuePropExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueuePropExperimenter.Marshal(b, m, deterministic)
-}
-func (m *OfpQueuePropExperimenter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueuePropExperimenter.Merge(m, src)
-}
-func (m *OfpQueuePropExperimenter) XXX_Size() int {
-	return xxx_messageInfo_OfpQueuePropExperimenter.Size(m)
-}
-func (m *OfpQueuePropExperimenter) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueuePropExperimenter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueuePropExperimenter proto.InternalMessageInfo
-
-func (m *OfpQueuePropExperimenter) GetPropHeader() *OfpQueuePropHeader {
-	if m != nil {
-		return m.PropHeader
-	}
-	return nil
-}
-
-func (m *OfpQueuePropExperimenter) GetExperimenter() uint32 {
-	if m != nil {
-		return m.Experimenter
-	}
-	return 0
-}
-
-func (m *OfpQueuePropExperimenter) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-// Full description for a queue.
-type OfpPacketQueue struct {
-	QueueId              uint32                `protobuf:"varint,1,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
-	Port                 uint32                `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
-	Properties           []*OfpQueuePropHeader `protobuf:"bytes,4,rep,name=properties,proto3" json:"properties,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *OfpPacketQueue) Reset()         { *m = OfpPacketQueue{} }
-func (m *OfpPacketQueue) String() string { return proto.CompactTextString(m) }
-func (*OfpPacketQueue) ProtoMessage()    {}
-func (*OfpPacketQueue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{77}
-}
-
-func (m *OfpPacketQueue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpPacketQueue.Unmarshal(m, b)
-}
-func (m *OfpPacketQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpPacketQueue.Marshal(b, m, deterministic)
-}
-func (m *OfpPacketQueue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpPacketQueue.Merge(m, src)
-}
-func (m *OfpPacketQueue) XXX_Size() int {
-	return xxx_messageInfo_OfpPacketQueue.Size(m)
-}
-func (m *OfpPacketQueue) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpPacketQueue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpPacketQueue proto.InternalMessageInfo
-
-func (m *OfpPacketQueue) GetQueueId() uint32 {
-	if m != nil {
-		return m.QueueId
-	}
-	return 0
-}
-
-func (m *OfpPacketQueue) GetPort() uint32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-func (m *OfpPacketQueue) GetProperties() []*OfpQueuePropHeader {
-	if m != nil {
-		return m.Properties
-	}
-	return nil
-}
-
-// Query for port queue configuration.
-type OfpQueueGetConfigRequest struct {
-	//ofp_header header;
-	Port                 uint32   `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpQueueGetConfigRequest) Reset()         { *m = OfpQueueGetConfigRequest{} }
-func (m *OfpQueueGetConfigRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpQueueGetConfigRequest) ProtoMessage()    {}
-func (*OfpQueueGetConfigRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{78}
-}
-
-func (m *OfpQueueGetConfigRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueueGetConfigRequest.Unmarshal(m, b)
-}
-func (m *OfpQueueGetConfigRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueueGetConfigRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpQueueGetConfigRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueueGetConfigRequest.Merge(m, src)
-}
-func (m *OfpQueueGetConfigRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpQueueGetConfigRequest.Size(m)
-}
-func (m *OfpQueueGetConfigRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueueGetConfigRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueueGetConfigRequest proto.InternalMessageInfo
-
-func (m *OfpQueueGetConfigRequest) GetPort() uint32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-// Queue configuration for a given port.
-type OfpQueueGetConfigReply struct {
-	//ofp_header header;
-	Port                 uint32            `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
-	Queues               []*OfpPacketQueue `protobuf:"bytes,2,rep,name=queues,proto3" json:"queues,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpQueueGetConfigReply) Reset()         { *m = OfpQueueGetConfigReply{} }
-func (m *OfpQueueGetConfigReply) String() string { return proto.CompactTextString(m) }
-func (*OfpQueueGetConfigReply) ProtoMessage()    {}
-func (*OfpQueueGetConfigReply) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{79}
-}
-
-func (m *OfpQueueGetConfigReply) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueueGetConfigReply.Unmarshal(m, b)
-}
-func (m *OfpQueueGetConfigReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueueGetConfigReply.Marshal(b, m, deterministic)
-}
-func (m *OfpQueueGetConfigReply) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueueGetConfigReply.Merge(m, src)
-}
-func (m *OfpQueueGetConfigReply) XXX_Size() int {
-	return xxx_messageInfo_OfpQueueGetConfigReply.Size(m)
-}
-func (m *OfpQueueGetConfigReply) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueueGetConfigReply.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueueGetConfigReply proto.InternalMessageInfo
-
-func (m *OfpQueueGetConfigReply) GetPort() uint32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-func (m *OfpQueueGetConfigReply) GetQueues() []*OfpPacketQueue {
-	if m != nil {
-		return m.Queues
-	}
-	return nil
-}
-
-// OFPAT_SET_QUEUE action struct: send packets to given queue on port.
-type OfpActionSetQueue struct {
-	Type                 uint32   `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
-	QueueId              uint32   `protobuf:"varint,3,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpActionSetQueue) Reset()         { *m = OfpActionSetQueue{} }
-func (m *OfpActionSetQueue) String() string { return proto.CompactTextString(m) }
-func (*OfpActionSetQueue) ProtoMessage()    {}
-func (*OfpActionSetQueue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{80}
-}
-
-func (m *OfpActionSetQueue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpActionSetQueue.Unmarshal(m, b)
-}
-func (m *OfpActionSetQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpActionSetQueue.Marshal(b, m, deterministic)
-}
-func (m *OfpActionSetQueue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpActionSetQueue.Merge(m, src)
-}
-func (m *OfpActionSetQueue) XXX_Size() int {
-	return xxx_messageInfo_OfpActionSetQueue.Size(m)
-}
-func (m *OfpActionSetQueue) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpActionSetQueue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpActionSetQueue proto.InternalMessageInfo
-
-func (m *OfpActionSetQueue) GetType() uint32 {
-	if m != nil {
-		return m.Type
-	}
-	return 0
-}
-
-func (m *OfpActionSetQueue) GetQueueId() uint32 {
-	if m != nil {
-		return m.QueueId
-	}
-	return 0
-}
-
-type OfpQueueStatsRequest struct {
-	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	QueueId              uint32   `protobuf:"varint,2,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpQueueStatsRequest) Reset()         { *m = OfpQueueStatsRequest{} }
-func (m *OfpQueueStatsRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpQueueStatsRequest) ProtoMessage()    {}
-func (*OfpQueueStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{81}
-}
-
-func (m *OfpQueueStatsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueueStatsRequest.Unmarshal(m, b)
-}
-func (m *OfpQueueStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueueStatsRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpQueueStatsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueueStatsRequest.Merge(m, src)
-}
-func (m *OfpQueueStatsRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpQueueStatsRequest.Size(m)
-}
-func (m *OfpQueueStatsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueueStatsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueueStatsRequest proto.InternalMessageInfo
-
-func (m *OfpQueueStatsRequest) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OfpQueueStatsRequest) GetQueueId() uint32 {
-	if m != nil {
-		return m.QueueId
-	}
-	return 0
-}
-
-type OfpQueueStats struct {
-	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	QueueId              uint32   `protobuf:"varint,2,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
-	TxBytes              uint64   `protobuf:"varint,3,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	TxPackets            uint64   `protobuf:"varint,4,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	TxErrors             uint64   `protobuf:"varint,5,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
-	DurationSec          uint32   `protobuf:"varint,6,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
-	DurationNsec         uint32   `protobuf:"varint,7,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpQueueStats) Reset()         { *m = OfpQueueStats{} }
-func (m *OfpQueueStats) String() string { return proto.CompactTextString(m) }
-func (*OfpQueueStats) ProtoMessage()    {}
-func (*OfpQueueStats) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{82}
-}
-
-func (m *OfpQueueStats) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpQueueStats.Unmarshal(m, b)
-}
-func (m *OfpQueueStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpQueueStats.Marshal(b, m, deterministic)
-}
-func (m *OfpQueueStats) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpQueueStats.Merge(m, src)
-}
-func (m *OfpQueueStats) XXX_Size() int {
-	return xxx_messageInfo_OfpQueueStats.Size(m)
-}
-func (m *OfpQueueStats) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpQueueStats.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpQueueStats proto.InternalMessageInfo
-
-func (m *OfpQueueStats) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetQueueId() uint32 {
-	if m != nil {
-		return m.QueueId
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetTxErrors() uint64 {
-	if m != nil {
-		return m.TxErrors
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetDurationSec() uint32 {
-	if m != nil {
-		return m.DurationSec
-	}
-	return 0
-}
-
-func (m *OfpQueueStats) GetDurationNsec() uint32 {
-	if m != nil {
-		return m.DurationNsec
-	}
-	return 0
-}
-
-// Role request and reply message.
-type OfpRoleRequest struct {
-	//ofp_header header;        /* Type OFPT_ROLE_REQUEST/OFPT_ROLE_REPLY. */
-	Role                 OfpControllerRole `protobuf:"varint,1,opt,name=role,proto3,enum=openflow_13.OfpControllerRole" json:"role,omitempty"`
-	GenerationId         uint64            `protobuf:"varint,2,opt,name=generation_id,json=generationId,proto3" json:"generation_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *OfpRoleRequest) Reset()         { *m = OfpRoleRequest{} }
-func (m *OfpRoleRequest) String() string { return proto.CompactTextString(m) }
-func (*OfpRoleRequest) ProtoMessage()    {}
-func (*OfpRoleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{83}
-}
-
-func (m *OfpRoleRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpRoleRequest.Unmarshal(m, b)
-}
-func (m *OfpRoleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpRoleRequest.Marshal(b, m, deterministic)
-}
-func (m *OfpRoleRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpRoleRequest.Merge(m, src)
-}
-func (m *OfpRoleRequest) XXX_Size() int {
-	return xxx_messageInfo_OfpRoleRequest.Size(m)
-}
-func (m *OfpRoleRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpRoleRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpRoleRequest proto.InternalMessageInfo
-
-func (m *OfpRoleRequest) GetRole() OfpControllerRole {
-	if m != nil {
-		return m.Role
-	}
-	return OfpControllerRole_OFPCR_ROLE_NOCHANGE
-}
-
-func (m *OfpRoleRequest) GetGenerationId() uint64 {
-	if m != nil {
-		return m.GenerationId
-	}
-	return 0
-}
-
-// Asynchronous message configuration.
-type OfpAsyncConfig struct {
-	//ofp_header header;    /* OFPT_GET_ASYNC_REPLY or OFPT_SET_ASYNC. */
-	PacketInMask         []uint32 `protobuf:"varint,1,rep,packed,name=packet_in_mask,json=packetInMask,proto3" json:"packet_in_mask,omitempty"`
-	PortStatusMask       []uint32 `protobuf:"varint,2,rep,packed,name=port_status_mask,json=portStatusMask,proto3" json:"port_status_mask,omitempty"`
-	FlowRemovedMask      []uint32 `protobuf:"varint,3,rep,packed,name=flow_removed_mask,json=flowRemovedMask,proto3" json:"flow_removed_mask,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfpAsyncConfig) Reset()         { *m = OfpAsyncConfig{} }
-func (m *OfpAsyncConfig) String() string { return proto.CompactTextString(m) }
-func (*OfpAsyncConfig) ProtoMessage()    {}
-func (*OfpAsyncConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{84}
-}
-
-func (m *OfpAsyncConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfpAsyncConfig.Unmarshal(m, b)
-}
-func (m *OfpAsyncConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfpAsyncConfig.Marshal(b, m, deterministic)
-}
-func (m *OfpAsyncConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfpAsyncConfig.Merge(m, src)
-}
-func (m *OfpAsyncConfig) XXX_Size() int {
-	return xxx_messageInfo_OfpAsyncConfig.Size(m)
-}
-func (m *OfpAsyncConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfpAsyncConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfpAsyncConfig proto.InternalMessageInfo
-
-func (m *OfpAsyncConfig) GetPacketInMask() []uint32 {
-	if m != nil {
-		return m.PacketInMask
-	}
-	return nil
-}
-
-func (m *OfpAsyncConfig) GetPortStatusMask() []uint32 {
-	if m != nil {
-		return m.PortStatusMask
-	}
-	return nil
-}
-
-func (m *OfpAsyncConfig) GetFlowRemovedMask() []uint32 {
-	if m != nil {
-		return m.FlowRemovedMask
-	}
-	return nil
-}
-
-type MeterModUpdate struct {
-	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	MeterMod             *OfpMeterMod `protobuf:"bytes,2,opt,name=meter_mod,json=meterMod,proto3" json:"meter_mod,omitempty"`
-	Xid                  uint32       `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *MeterModUpdate) Reset()         { *m = MeterModUpdate{} }
-func (m *MeterModUpdate) String() string { return proto.CompactTextString(m) }
-func (*MeterModUpdate) ProtoMessage()    {}
-func (*MeterModUpdate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{85}
-}
-
-func (m *MeterModUpdate) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MeterModUpdate.Unmarshal(m, b)
-}
-func (m *MeterModUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MeterModUpdate.Marshal(b, m, deterministic)
-}
-func (m *MeterModUpdate) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MeterModUpdate.Merge(m, src)
-}
-func (m *MeterModUpdate) XXX_Size() int {
-	return xxx_messageInfo_MeterModUpdate.Size(m)
-}
-func (m *MeterModUpdate) XXX_DiscardUnknown() {
-	xxx_messageInfo_MeterModUpdate.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MeterModUpdate proto.InternalMessageInfo
-
-func (m *MeterModUpdate) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *MeterModUpdate) GetMeterMod() *OfpMeterMod {
-	if m != nil {
-		return m.MeterMod
-	}
-	return nil
-}
-
-func (m *MeterModUpdate) GetXid() uint32 {
-	if m != nil {
-		return m.Xid
-	}
-	return 0
-}
-
-type MeterStatsReply struct {
-	MeterStats           []*OfpMeterStats `protobuf:"bytes,1,rep,name=meter_stats,json=meterStats,proto3" json:"meter_stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *MeterStatsReply) Reset()         { *m = MeterStatsReply{} }
-func (m *MeterStatsReply) String() string { return proto.CompactTextString(m) }
-func (*MeterStatsReply) ProtoMessage()    {}
-func (*MeterStatsReply) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{86}
-}
-
-func (m *MeterStatsReply) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MeterStatsReply.Unmarshal(m, b)
-}
-func (m *MeterStatsReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MeterStatsReply.Marshal(b, m, deterministic)
-}
-func (m *MeterStatsReply) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MeterStatsReply.Merge(m, src)
-}
-func (m *MeterStatsReply) XXX_Size() int {
-	return xxx_messageInfo_MeterStatsReply.Size(m)
-}
-func (m *MeterStatsReply) XXX_DiscardUnknown() {
-	xxx_messageInfo_MeterStatsReply.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MeterStatsReply proto.InternalMessageInfo
-
-func (m *MeterStatsReply) GetMeterStats() []*OfpMeterStats {
-	if m != nil {
-		return m.MeterStats
-	}
-	return nil
-}
-
-type FlowTableUpdate struct {
-	Id                   string      `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	FlowMod              *OfpFlowMod `protobuf:"bytes,2,opt,name=flow_mod,json=flowMod,proto3" json:"flow_mod,omitempty"`
-	Xid                  uint32      `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *FlowTableUpdate) Reset()         { *m = FlowTableUpdate{} }
-func (m *FlowTableUpdate) String() string { return proto.CompactTextString(m) }
-func (*FlowTableUpdate) ProtoMessage()    {}
-func (*FlowTableUpdate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{87}
-}
-
-func (m *FlowTableUpdate) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowTableUpdate.Unmarshal(m, b)
-}
-func (m *FlowTableUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowTableUpdate.Marshal(b, m, deterministic)
-}
-func (m *FlowTableUpdate) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowTableUpdate.Merge(m, src)
-}
-func (m *FlowTableUpdate) XXX_Size() int {
-	return xxx_messageInfo_FlowTableUpdate.Size(m)
-}
-func (m *FlowTableUpdate) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowTableUpdate.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowTableUpdate proto.InternalMessageInfo
-
-func (m *FlowTableUpdate) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *FlowTableUpdate) GetFlowMod() *OfpFlowMod {
-	if m != nil {
-		return m.FlowMod
-	}
-	return nil
-}
-
-func (m *FlowTableUpdate) GetXid() uint32 {
-	if m != nil {
-		return m.Xid
-	}
-	return 0
-}
-
-type FlowGroupTableUpdate struct {
-	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	GroupMod             *OfpGroupMod `protobuf:"bytes,2,opt,name=group_mod,json=groupMod,proto3" json:"group_mod,omitempty"`
-	Xid                  uint32       `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *FlowGroupTableUpdate) Reset()         { *m = FlowGroupTableUpdate{} }
-func (m *FlowGroupTableUpdate) String() string { return proto.CompactTextString(m) }
-func (*FlowGroupTableUpdate) ProtoMessage()    {}
-func (*FlowGroupTableUpdate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{88}
-}
-
-func (m *FlowGroupTableUpdate) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowGroupTableUpdate.Unmarshal(m, b)
-}
-func (m *FlowGroupTableUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowGroupTableUpdate.Marshal(b, m, deterministic)
-}
-func (m *FlowGroupTableUpdate) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowGroupTableUpdate.Merge(m, src)
-}
-func (m *FlowGroupTableUpdate) XXX_Size() int {
-	return xxx_messageInfo_FlowGroupTableUpdate.Size(m)
-}
-func (m *FlowGroupTableUpdate) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowGroupTableUpdate.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowGroupTableUpdate proto.InternalMessageInfo
-
-func (m *FlowGroupTableUpdate) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *FlowGroupTableUpdate) GetGroupMod() *OfpGroupMod {
-	if m != nil {
-		return m.GroupMod
-	}
-	return nil
-}
-
-func (m *FlowGroupTableUpdate) GetXid() uint32 {
-	if m != nil {
-		return m.Xid
-	}
-	return 0
-}
-
-type Flows struct {
-	Items                []*OfpFlowStats `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *Flows) Reset()         { *m = Flows{} }
-func (m *Flows) String() string { return proto.CompactTextString(m) }
-func (*Flows) ProtoMessage()    {}
-func (*Flows) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{89}
-}
-
-func (m *Flows) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Flows.Unmarshal(m, b)
-}
-func (m *Flows) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Flows.Marshal(b, m, deterministic)
-}
-func (m *Flows) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Flows.Merge(m, src)
-}
-func (m *Flows) XXX_Size() int {
-	return xxx_messageInfo_Flows.Size(m)
-}
-func (m *Flows) XXX_DiscardUnknown() {
-	xxx_messageInfo_Flows.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Flows proto.InternalMessageInfo
-
-func (m *Flows) GetItems() []*OfpFlowStats {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type Meters struct {
-	Items                []*OfpMeterEntry `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *Meters) Reset()         { *m = Meters{} }
-func (m *Meters) String() string { return proto.CompactTextString(m) }
-func (*Meters) ProtoMessage()    {}
-func (*Meters) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{90}
-}
-
-func (m *Meters) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Meters.Unmarshal(m, b)
-}
-func (m *Meters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Meters.Marshal(b, m, deterministic)
-}
-func (m *Meters) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Meters.Merge(m, src)
-}
-func (m *Meters) XXX_Size() int {
-	return xxx_messageInfo_Meters.Size(m)
-}
-func (m *Meters) XXX_DiscardUnknown() {
-	xxx_messageInfo_Meters.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Meters proto.InternalMessageInfo
-
-func (m *Meters) GetItems() []*OfpMeterEntry {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type FlowGroups struct {
-	Items                []*OfpGroupEntry `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *FlowGroups) Reset()         { *m = FlowGroups{} }
-func (m *FlowGroups) String() string { return proto.CompactTextString(m) }
-func (*FlowGroups) ProtoMessage()    {}
-func (*FlowGroups) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{91}
-}
-
-func (m *FlowGroups) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowGroups.Unmarshal(m, b)
-}
-func (m *FlowGroups) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowGroups.Marshal(b, m, deterministic)
-}
-func (m *FlowGroups) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowGroups.Merge(m, src)
-}
-func (m *FlowGroups) XXX_Size() int {
-	return xxx_messageInfo_FlowGroups.Size(m)
-}
-func (m *FlowGroups) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowGroups.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowGroups proto.InternalMessageInfo
-
-func (m *FlowGroups) GetItems() []*OfpGroupEntry {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type FlowChanges struct {
-	ToAdd                *Flows   `protobuf:"bytes,1,opt,name=to_add,json=toAdd,proto3" json:"to_add,omitempty"`
-	ToRemove             *Flows   `protobuf:"bytes,2,opt,name=to_remove,json=toRemove,proto3" json:"to_remove,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *FlowChanges) Reset()         { *m = FlowChanges{} }
-func (m *FlowChanges) String() string { return proto.CompactTextString(m) }
-func (*FlowChanges) ProtoMessage()    {}
-func (*FlowChanges) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{92}
-}
-
-func (m *FlowChanges) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowChanges.Unmarshal(m, b)
-}
-func (m *FlowChanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowChanges.Marshal(b, m, deterministic)
-}
-func (m *FlowChanges) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowChanges.Merge(m, src)
-}
-func (m *FlowChanges) XXX_Size() int {
-	return xxx_messageInfo_FlowChanges.Size(m)
-}
-func (m *FlowChanges) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowChanges.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowChanges proto.InternalMessageInfo
-
-func (m *FlowChanges) GetToAdd() *Flows {
-	if m != nil {
-		return m.ToAdd
-	}
-	return nil
-}
-
-func (m *FlowChanges) GetToRemove() *Flows {
-	if m != nil {
-		return m.ToRemove
-	}
-	return nil
-}
-
-type FlowGroupChanges struct {
-	ToAdd                *FlowGroups `protobuf:"bytes,1,opt,name=to_add,json=toAdd,proto3" json:"to_add,omitempty"`
-	ToRemove             *FlowGroups `protobuf:"bytes,2,opt,name=to_remove,json=toRemove,proto3" json:"to_remove,omitempty"`
-	ToUpdate             *FlowGroups `protobuf:"bytes,3,opt,name=to_update,json=toUpdate,proto3" json:"to_update,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *FlowGroupChanges) Reset()         { *m = FlowGroupChanges{} }
-func (m *FlowGroupChanges) String() string { return proto.CompactTextString(m) }
-func (*FlowGroupChanges) ProtoMessage()    {}
-func (*FlowGroupChanges) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{93}
-}
-
-func (m *FlowGroupChanges) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowGroupChanges.Unmarshal(m, b)
-}
-func (m *FlowGroupChanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowGroupChanges.Marshal(b, m, deterministic)
-}
-func (m *FlowGroupChanges) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowGroupChanges.Merge(m, src)
-}
-func (m *FlowGroupChanges) XXX_Size() int {
-	return xxx_messageInfo_FlowGroupChanges.Size(m)
-}
-func (m *FlowGroupChanges) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowGroupChanges.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowGroupChanges proto.InternalMessageInfo
-
-func (m *FlowGroupChanges) GetToAdd() *FlowGroups {
-	if m != nil {
-		return m.ToAdd
-	}
-	return nil
-}
-
-func (m *FlowGroupChanges) GetToRemove() *FlowGroups {
-	if m != nil {
-		return m.ToRemove
-	}
-	return nil
-}
-
-func (m *FlowGroupChanges) GetToUpdate() *FlowGroups {
-	if m != nil {
-		return m.ToUpdate
-	}
-	return nil
-}
-
-type PacketIn struct {
-	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	PacketIn             *OfpPacketIn `protobuf:"bytes,2,opt,name=packet_in,json=packetIn,proto3" json:"packet_in,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
-	XXX_unrecognized     []byte       `json:"-"`
-	XXX_sizecache        int32        `json:"-"`
-}
-
-func (m *PacketIn) Reset()         { *m = PacketIn{} }
-func (m *PacketIn) String() string { return proto.CompactTextString(m) }
-func (*PacketIn) ProtoMessage()    {}
-func (*PacketIn) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{94}
-}
-
-func (m *PacketIn) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PacketIn.Unmarshal(m, b)
-}
-func (m *PacketIn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PacketIn.Marshal(b, m, deterministic)
-}
-func (m *PacketIn) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PacketIn.Merge(m, src)
-}
-func (m *PacketIn) XXX_Size() int {
-	return xxx_messageInfo_PacketIn.Size(m)
-}
-func (m *PacketIn) XXX_DiscardUnknown() {
-	xxx_messageInfo_PacketIn.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PacketIn proto.InternalMessageInfo
-
-func (m *PacketIn) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *PacketIn) GetPacketIn() *OfpPacketIn {
-	if m != nil {
-		return m.PacketIn
-	}
-	return nil
-}
-
-type PacketOut struct {
-	Id                   string        `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	PacketOut            *OfpPacketOut `protobuf:"bytes,2,opt,name=packet_out,json=packetOut,proto3" json:"packet_out,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *PacketOut) Reset()         { *m = PacketOut{} }
-func (m *PacketOut) String() string { return proto.CompactTextString(m) }
-func (*PacketOut) ProtoMessage()    {}
-func (*PacketOut) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{95}
-}
-
-func (m *PacketOut) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PacketOut.Unmarshal(m, b)
-}
-func (m *PacketOut) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PacketOut.Marshal(b, m, deterministic)
-}
-func (m *PacketOut) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PacketOut.Merge(m, src)
-}
-func (m *PacketOut) XXX_Size() int {
-	return xxx_messageInfo_PacketOut.Size(m)
-}
-func (m *PacketOut) XXX_DiscardUnknown() {
-	xxx_messageInfo_PacketOut.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PacketOut proto.InternalMessageInfo
-
-func (m *PacketOut) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *PacketOut) GetPacketOut() *OfpPacketOut {
-	if m != nil {
-		return m.PacketOut
-	}
-	return nil
-}
-
-type ChangeEvent struct {
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Types that are valid to be assigned to Event:
-	//	*ChangeEvent_PortStatus
-	//	*ChangeEvent_Error
-	Event                isChangeEvent_Event `protobuf_oneof:"event"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *ChangeEvent) Reset()         { *m = ChangeEvent{} }
-func (m *ChangeEvent) String() string { return proto.CompactTextString(m) }
-func (*ChangeEvent) ProtoMessage()    {}
-func (*ChangeEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_08e3a4e375aeddc7, []int{96}
-}
-
-func (m *ChangeEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ChangeEvent.Unmarshal(m, b)
-}
-func (m *ChangeEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ChangeEvent.Marshal(b, m, deterministic)
-}
-func (m *ChangeEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ChangeEvent.Merge(m, src)
-}
-func (m *ChangeEvent) XXX_Size() int {
-	return xxx_messageInfo_ChangeEvent.Size(m)
-}
-func (m *ChangeEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_ChangeEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ChangeEvent proto.InternalMessageInfo
-
-func (m *ChangeEvent) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-type isChangeEvent_Event interface {
-	isChangeEvent_Event()
-}
-
-type ChangeEvent_PortStatus struct {
-	PortStatus *OfpPortStatus `protobuf:"bytes,2,opt,name=port_status,json=portStatus,proto3,oneof"`
-}
-
-type ChangeEvent_Error struct {
-	Error *OfpErrorMsg `protobuf:"bytes,3,opt,name=error,proto3,oneof"`
-}
-
-func (*ChangeEvent_PortStatus) isChangeEvent_Event() {}
-
-func (*ChangeEvent_Error) isChangeEvent_Event() {}
-
-func (m *ChangeEvent) GetEvent() isChangeEvent_Event {
-	if m != nil {
-		return m.Event
-	}
-	return nil
-}
-
-func (m *ChangeEvent) GetPortStatus() *OfpPortStatus {
-	if x, ok := m.GetEvent().(*ChangeEvent_PortStatus); ok {
-		return x.PortStatus
-	}
-	return nil
-}
-
-func (m *ChangeEvent) GetError() *OfpErrorMsg {
-	if x, ok := m.GetEvent().(*ChangeEvent_Error); ok {
-		return x.Error
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ChangeEvent) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*ChangeEvent_PortStatus)(nil),
-		(*ChangeEvent_Error)(nil),
-	}
-}
-
-func init() {
-	proto.RegisterEnum("openflow_13.OfpPortNo", OfpPortNo_name, OfpPortNo_value)
-	proto.RegisterEnum("openflow_13.OfpType", OfpType_name, OfpType_value)
-	proto.RegisterEnum("openflow_13.OfpHelloElemType", OfpHelloElemType_name, OfpHelloElemType_value)
-	proto.RegisterEnum("openflow_13.OfpConfigFlags", OfpConfigFlags_name, OfpConfigFlags_value)
-	proto.RegisterEnum("openflow_13.OfpTableConfig", OfpTableConfig_name, OfpTableConfig_value)
-	proto.RegisterEnum("openflow_13.OfpTable", OfpTable_name, OfpTable_value)
-	proto.RegisterEnum("openflow_13.OfpCapabilities", OfpCapabilities_name, OfpCapabilities_value)
-	proto.RegisterEnum("openflow_13.OfpPortConfig", OfpPortConfig_name, OfpPortConfig_value)
-	proto.RegisterEnum("openflow_13.OfpPortState", OfpPortState_name, OfpPortState_value)
-	proto.RegisterEnum("openflow_13.OfpPortFeatures", OfpPortFeatures_name, OfpPortFeatures_value)
-	proto.RegisterEnum("openflow_13.OfpPortReason", OfpPortReason_name, OfpPortReason_value)
-	proto.RegisterEnum("openflow_13.OfpMatchType", OfpMatchType_name, OfpMatchType_value)
-	proto.RegisterEnum("openflow_13.OfpOxmClass", OfpOxmClass_name, OfpOxmClass_value)
-	proto.RegisterEnum("openflow_13.OxmOfbFieldTypes", OxmOfbFieldTypes_name, OxmOfbFieldTypes_value)
-	proto.RegisterEnum("openflow_13.OfpVlanId", OfpVlanId_name, OfpVlanId_value)
-	proto.RegisterEnum("openflow_13.OfpIpv6ExthdrFlags", OfpIpv6ExthdrFlags_name, OfpIpv6ExthdrFlags_value)
-	proto.RegisterEnum("openflow_13.OfpActionType", OfpActionType_name, OfpActionType_value)
-	proto.RegisterEnum("openflow_13.OfpControllerMaxLen", OfpControllerMaxLen_name, OfpControllerMaxLen_value)
-	proto.RegisterEnum("openflow_13.OfpInstructionType", OfpInstructionType_name, OfpInstructionType_value)
-	proto.RegisterEnum("openflow_13.OfpFlowModCommand", OfpFlowModCommand_name, OfpFlowModCommand_value)
-	proto.RegisterEnum("openflow_13.OfpFlowModFlags", OfpFlowModFlags_name, OfpFlowModFlags_value)
-	proto.RegisterEnum("openflow_13.OfpGroup", OfpGroup_name, OfpGroup_value)
-	proto.RegisterEnum("openflow_13.OfpGroupModCommand", OfpGroupModCommand_name, OfpGroupModCommand_value)
-	proto.RegisterEnum("openflow_13.OfpGroupType", OfpGroupType_name, OfpGroupType_value)
-	proto.RegisterEnum("openflow_13.OfpPacketInReason", OfpPacketInReason_name, OfpPacketInReason_value)
-	proto.RegisterEnum("openflow_13.OfpFlowRemovedReason", OfpFlowRemovedReason_name, OfpFlowRemovedReason_value)
-	proto.RegisterEnum("openflow_13.OfpMeter", OfpMeter_name, OfpMeter_value)
-	proto.RegisterEnum("openflow_13.OfpMeterBandType", OfpMeterBandType_name, OfpMeterBandType_value)
-	proto.RegisterEnum("openflow_13.OfpMeterModCommand", OfpMeterModCommand_name, OfpMeterModCommand_value)
-	proto.RegisterEnum("openflow_13.OfpMeterFlags", OfpMeterFlags_name, OfpMeterFlags_value)
-	proto.RegisterEnum("openflow_13.OfpErrorType", OfpErrorType_name, OfpErrorType_value)
-	proto.RegisterEnum("openflow_13.OfpHelloFailedCode", OfpHelloFailedCode_name, OfpHelloFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpBadRequestCode", OfpBadRequestCode_name, OfpBadRequestCode_value)
-	proto.RegisterEnum("openflow_13.OfpBadActionCode", OfpBadActionCode_name, OfpBadActionCode_value)
-	proto.RegisterEnum("openflow_13.OfpBadInstructionCode", OfpBadInstructionCode_name, OfpBadInstructionCode_value)
-	proto.RegisterEnum("openflow_13.OfpBadMatchCode", OfpBadMatchCode_name, OfpBadMatchCode_value)
-	proto.RegisterEnum("openflow_13.OfpFlowModFailedCode", OfpFlowModFailedCode_name, OfpFlowModFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpGroupModFailedCode", OfpGroupModFailedCode_name, OfpGroupModFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpPortModFailedCode", OfpPortModFailedCode_name, OfpPortModFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpTableModFailedCode", OfpTableModFailedCode_name, OfpTableModFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpQueueOpFailedCode", OfpQueueOpFailedCode_name, OfpQueueOpFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpSwitchConfigFailedCode", OfpSwitchConfigFailedCode_name, OfpSwitchConfigFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpRoleRequestFailedCode", OfpRoleRequestFailedCode_name, OfpRoleRequestFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpMeterModFailedCode", OfpMeterModFailedCode_name, OfpMeterModFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpTableFeaturesFailedCode", OfpTableFeaturesFailedCode_name, OfpTableFeaturesFailedCode_value)
-	proto.RegisterEnum("openflow_13.OfpMultipartType", OfpMultipartType_name, OfpMultipartType_value)
-	proto.RegisterEnum("openflow_13.OfpMultipartRequestFlags", OfpMultipartRequestFlags_name, OfpMultipartRequestFlags_value)
-	proto.RegisterEnum("openflow_13.OfpMultipartReplyFlags", OfpMultipartReplyFlags_name, OfpMultipartReplyFlags_value)
-	proto.RegisterEnum("openflow_13.OfpTableFeaturePropType", OfpTableFeaturePropType_name, OfpTableFeaturePropType_value)
-	proto.RegisterEnum("openflow_13.OfpGroupCapabilities", OfpGroupCapabilities_name, OfpGroupCapabilities_value)
-	proto.RegisterEnum("openflow_13.OfpQueueProperties", OfpQueueProperties_name, OfpQueueProperties_value)
-	proto.RegisterEnum("openflow_13.OfpControllerRole", OfpControllerRole_name, OfpControllerRole_value)
-	proto.RegisterType((*OfpHeader)(nil), "openflow_13.ofp_header")
-	proto.RegisterType((*OfpHelloElemHeader)(nil), "openflow_13.ofp_hello_elem_header")
-	proto.RegisterType((*OfpHelloElemVersionbitmap)(nil), "openflow_13.ofp_hello_elem_versionbitmap")
-	proto.RegisterType((*OfpHello)(nil), "openflow_13.ofp_hello")
-	proto.RegisterType((*OfpSwitchConfig)(nil), "openflow_13.ofp_switch_config")
-	proto.RegisterType((*OfpTableMod)(nil), "openflow_13.ofp_table_mod")
-	proto.RegisterType((*OfpPort)(nil), "openflow_13.ofp_port")
-	proto.RegisterType((*OfpSwitchFeatures)(nil), "openflow_13.ofp_switch_features")
-	proto.RegisterType((*OfpPortStatus)(nil), "openflow_13.ofp_port_status")
-	proto.RegisterType((*OfpPortMod)(nil), "openflow_13.ofp_port_mod")
-	proto.RegisterType((*OfpMatch)(nil), "openflow_13.ofp_match")
-	proto.RegisterType((*OfpOxmField)(nil), "openflow_13.ofp_oxm_field")
-	proto.RegisterType((*OfpOxmOfbField)(nil), "openflow_13.ofp_oxm_ofb_field")
-	proto.RegisterType((*OfpOxmExperimenterField)(nil), "openflow_13.ofp_oxm_experimenter_field")
-	proto.RegisterType((*OfpAction)(nil), "openflow_13.ofp_action")
-	proto.RegisterType((*OfpActionOutput)(nil), "openflow_13.ofp_action_output")
-	proto.RegisterType((*OfpActionMplsTtl)(nil), "openflow_13.ofp_action_mpls_ttl")
-	proto.RegisterType((*OfpActionPush)(nil), "openflow_13.ofp_action_push")
-	proto.RegisterType((*OfpActionPopMpls)(nil), "openflow_13.ofp_action_pop_mpls")
-	proto.RegisterType((*OfpActionGroup)(nil), "openflow_13.ofp_action_group")
-	proto.RegisterType((*OfpActionNwTtl)(nil), "openflow_13.ofp_action_nw_ttl")
-	proto.RegisterType((*OfpActionSetField)(nil), "openflow_13.ofp_action_set_field")
-	proto.RegisterType((*OfpActionExperimenter)(nil), "openflow_13.ofp_action_experimenter")
-	proto.RegisterType((*OfpInstruction)(nil), "openflow_13.ofp_instruction")
-	proto.RegisterType((*OfpInstructionGotoTable)(nil), "openflow_13.ofp_instruction_goto_table")
-	proto.RegisterType((*OfpInstructionWriteMetadata)(nil), "openflow_13.ofp_instruction_write_metadata")
-	proto.RegisterType((*OfpInstructionActions)(nil), "openflow_13.ofp_instruction_actions")
-	proto.RegisterType((*OfpInstructionMeter)(nil), "openflow_13.ofp_instruction_meter")
-	proto.RegisterType((*OfpInstructionExperimenter)(nil), "openflow_13.ofp_instruction_experimenter")
-	proto.RegisterType((*OfpFlowMod)(nil), "openflow_13.ofp_flow_mod")
-	proto.RegisterType((*OfpBucket)(nil), "openflow_13.ofp_bucket")
-	proto.RegisterType((*OfpGroupMod)(nil), "openflow_13.ofp_group_mod")
-	proto.RegisterType((*OfpPacketOut)(nil), "openflow_13.ofp_packet_out")
-	proto.RegisterType((*OfpPacketIn)(nil), "openflow_13.ofp_packet_in")
-	proto.RegisterType((*OfpFlowRemoved)(nil), "openflow_13.ofp_flow_removed")
-	proto.RegisterType((*OfpMeterBandHeader)(nil), "openflow_13.ofp_meter_band_header")
-	proto.RegisterType((*OfpMeterBandDrop)(nil), "openflow_13.ofp_meter_band_drop")
-	proto.RegisterType((*OfpMeterBandDscpRemark)(nil), "openflow_13.ofp_meter_band_dscp_remark")
-	proto.RegisterType((*OfpMeterBandExperimenter)(nil), "openflow_13.ofp_meter_band_experimenter")
-	proto.RegisterType((*OfpMeterMod)(nil), "openflow_13.ofp_meter_mod")
-	proto.RegisterType((*OfpErrorMsg)(nil), "openflow_13.ofp_error_msg")
-	proto.RegisterType((*OfpErrorExperimenterMsg)(nil), "openflow_13.ofp_error_experimenter_msg")
-	proto.RegisterType((*OfpMultipartRequest)(nil), "openflow_13.ofp_multipart_request")
-	proto.RegisterType((*OfpMultipartReply)(nil), "openflow_13.ofp_multipart_reply")
-	proto.RegisterType((*OfpDesc)(nil), "openflow_13.ofp_desc")
-	proto.RegisterType((*OfpFlowStatsRequest)(nil), "openflow_13.ofp_flow_stats_request")
-	proto.RegisterType((*OfpFlowStats)(nil), "openflow_13.ofp_flow_stats")
-	proto.RegisterType((*OfpAggregateStatsRequest)(nil), "openflow_13.ofp_aggregate_stats_request")
-	proto.RegisterType((*OfpAggregateStatsReply)(nil), "openflow_13.ofp_aggregate_stats_reply")
-	proto.RegisterType((*OfpTableFeatureProperty)(nil), "openflow_13.ofp_table_feature_property")
-	proto.RegisterType((*OfpTableFeaturePropInstructions)(nil), "openflow_13.ofp_table_feature_prop_instructions")
-	proto.RegisterType((*OfpTableFeaturePropNextTables)(nil), "openflow_13.ofp_table_feature_prop_next_tables")
-	proto.RegisterType((*OfpTableFeaturePropActions)(nil), "openflow_13.ofp_table_feature_prop_actions")
-	proto.RegisterType((*OfpTableFeaturePropOxm)(nil), "openflow_13.ofp_table_feature_prop_oxm")
-	proto.RegisterType((*OfpTableFeaturePropExperimenter)(nil), "openflow_13.ofp_table_feature_prop_experimenter")
-	proto.RegisterType((*OfpTableFeatures)(nil), "openflow_13.ofp_table_features")
-	proto.RegisterType((*OfpTableStats)(nil), "openflow_13.ofp_table_stats")
-	proto.RegisterType((*OfpPortStatsRequest)(nil), "openflow_13.ofp_port_stats_request")
-	proto.RegisterType((*OfpPortStats)(nil), "openflow_13.ofp_port_stats")
-	proto.RegisterType((*OfpGroupStatsRequest)(nil), "openflow_13.ofp_group_stats_request")
-	proto.RegisterType((*OfpBucketCounter)(nil), "openflow_13.ofp_bucket_counter")
-	proto.RegisterType((*OfpGroupStats)(nil), "openflow_13.ofp_group_stats")
-	proto.RegisterType((*OfpGroupDesc)(nil), "openflow_13.ofp_group_desc")
-	proto.RegisterType((*OfpGroupEntry)(nil), "openflow_13.ofp_group_entry")
-	proto.RegisterType((*OfpGroupFeatures)(nil), "openflow_13.ofp_group_features")
-	proto.RegisterType((*OfpMeterMultipartRequest)(nil), "openflow_13.ofp_meter_multipart_request")
-	proto.RegisterType((*OfpMeterBandStats)(nil), "openflow_13.ofp_meter_band_stats")
-	proto.RegisterType((*OfpMeterStats)(nil), "openflow_13.ofp_meter_stats")
-	proto.RegisterType((*OfpMeterConfig)(nil), "openflow_13.ofp_meter_config")
-	proto.RegisterType((*OfpMeterFeatures)(nil), "openflow_13.ofp_meter_features")
-	proto.RegisterType((*OfpMeterEntry)(nil), "openflow_13.ofp_meter_entry")
-	proto.RegisterType((*OfpExperimenterMultipartHeader)(nil), "openflow_13.ofp_experimenter_multipart_header")
-	proto.RegisterType((*OfpExperimenterHeader)(nil), "openflow_13.ofp_experimenter_header")
-	proto.RegisterType((*OfpQueuePropHeader)(nil), "openflow_13.ofp_queue_prop_header")
-	proto.RegisterType((*OfpQueuePropMinRate)(nil), "openflow_13.ofp_queue_prop_min_rate")
-	proto.RegisterType((*OfpQueuePropMaxRate)(nil), "openflow_13.ofp_queue_prop_max_rate")
-	proto.RegisterType((*OfpQueuePropExperimenter)(nil), "openflow_13.ofp_queue_prop_experimenter")
-	proto.RegisterType((*OfpPacketQueue)(nil), "openflow_13.ofp_packet_queue")
-	proto.RegisterType((*OfpQueueGetConfigRequest)(nil), "openflow_13.ofp_queue_get_config_request")
-	proto.RegisterType((*OfpQueueGetConfigReply)(nil), "openflow_13.ofp_queue_get_config_reply")
-	proto.RegisterType((*OfpActionSetQueue)(nil), "openflow_13.ofp_action_set_queue")
-	proto.RegisterType((*OfpQueueStatsRequest)(nil), "openflow_13.ofp_queue_stats_request")
-	proto.RegisterType((*OfpQueueStats)(nil), "openflow_13.ofp_queue_stats")
-	proto.RegisterType((*OfpRoleRequest)(nil), "openflow_13.ofp_role_request")
-	proto.RegisterType((*OfpAsyncConfig)(nil), "openflow_13.ofp_async_config")
-	proto.RegisterType((*MeterModUpdate)(nil), "openflow_13.MeterModUpdate")
-	proto.RegisterType((*MeterStatsReply)(nil), "openflow_13.MeterStatsReply")
-	proto.RegisterType((*FlowTableUpdate)(nil), "openflow_13.FlowTableUpdate")
-	proto.RegisterType((*FlowGroupTableUpdate)(nil), "openflow_13.FlowGroupTableUpdate")
-	proto.RegisterType((*Flows)(nil), "openflow_13.Flows")
-	proto.RegisterType((*Meters)(nil), "openflow_13.Meters")
-	proto.RegisterType((*FlowGroups)(nil), "openflow_13.FlowGroups")
-	proto.RegisterType((*FlowChanges)(nil), "openflow_13.FlowChanges")
-	proto.RegisterType((*FlowGroupChanges)(nil), "openflow_13.FlowGroupChanges")
-	proto.RegisterType((*PacketIn)(nil), "openflow_13.PacketIn")
-	proto.RegisterType((*PacketOut)(nil), "openflow_13.PacketOut")
-	proto.RegisterType((*ChangeEvent)(nil), "openflow_13.ChangeEvent")
-}
-
-func init() { proto.RegisterFile("voltha_protos/openflow_13.proto", fileDescriptor_08e3a4e375aeddc7) }
-
-var fileDescriptor_08e3a4e375aeddc7 = []byte{
-	// 8470 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x7d, 0x5b, 0x8c, 0x1c, 0x59,
-	0x96, 0x90, 0xf3, 0x51, 0x55, 0x99, 0x37, 0xab, 0xca, 0xe1, 0xf0, 0x2b, 0xed, 0xb2, 0xdb, 0x76,
-	0x76, 0xf7, 0x4c, 0x4f, 0x0e, 0xdb, 0x6e, 0xbb, 0x3d, 0x9e, 0xd9, 0x79, 0x2c, 0x8e, 0xcc, 0x8c,
-	0xac, 0xcc, 0x76, 0xbe, 0x1c, 0x11, 0x55, 0xb6, 0x07, 0x41, 0x28, 0x2b, 0x33, 0x5c, 0x95, 0xeb,
-	0xcc, 0x8c, 0x9c, 0x88, 0xa8, 0x72, 0x79, 0x99, 0x45, 0x86, 0x15, 0x42, 0x02, 0x76, 0x17, 0xb4,
-	0x1f, 0x23, 0xc1, 0x22, 0xb1, 0x02, 0x3e, 0x10, 0x12, 0x1f, 0x48, 0x48, 0x48, 0x7c, 0xaf, 0x04,
-	0x12, 0x02, 0x69, 0x25, 0xb4, 0x3f, 0xbb, 0x7f, 0xcb, 0x0f, 0xd2, 0xfe, 0xc3, 0x32, 0x8c, 0xd1,
-	0xb9, 0xe7, 0xdc, 0x1b, 0x37, 0xf2, 0x51, 0x5d, 0x33, 0xf4, 0xf0, 0xc1, 0x97, 0x33, 0xce, 0xeb,
-	0x9e, 0x7b, 0xee, 0x39, 0xe7, 0x9e, 0x7b, 0xe2, 0x46, 0x99, 0xdd, 0x39, 0xf1, 0xc7, 0xd1, 0x51,
-	0xdf, 0x9d, 0x05, 0x7e, 0xe4, 0x87, 0xf7, 0xfd, 0x99, 0x37, 0x7d, 0x35, 0xf6, 0xdf, 0xb8, 0x0f,
-	0x3e, 0xff, 0x94, 0x83, 0xf4, 0x82, 0x02, 0xba, 0x79, 0xeb, 0xd0, 0xf7, 0x0f, 0xc7, 0xde, 0xfd,
-	0xfe, 0x6c, 0x74, 0xbf, 0x3f, 0x9d, 0xfa, 0x51, 0x3f, 0x1a, 0xf9, 0xd3, 0x10, 0x49, 0x4b, 0x03,
-	0xc6, 0xfc, 0x57, 0x33, 0xf7, 0xc8, 0xeb, 0x0f, 0xbd, 0x40, 0x2f, 0xb2, 0x8d, 0x13, 0x2f, 0x08,
-	0x47, 0xfe, 0xb4, 0x98, 0xba, 0x9b, 0xfa, 0x64, 0xcb, 0x12, 0x8f, 0xfa, 0x37, 0x58, 0x36, 0x7a,
-	0x3b, 0xf3, 0x8a, 0xe9, 0xbb, 0xa9, 0x4f, 0xb6, 0x1f, 0x5e, 0xfd, 0x54, 0x1d, 0x14, 0x04, 0x00,
-	0xd2, 0xe2, 0x24, 0xba, 0xc6, 0x32, 0xa7, 0xa3, 0x61, 0x31, 0xc3, 0x05, 0xc0, 0xcf, 0xd2, 0xbf,
-	0x4a, 0xb1, 0xab, 0x38, 0xca, 0x78, 0xec, 0xbb, 0xde, 0xd8, 0x9b, 0x88, 0x01, 0x1f, 0x91, 0xd8,
-	0x14, 0x17, 0x7b, 0x77, 0x41, 0xac, 0xc2, 0xa1, 0x8c, 0xf0, 0x8c, 0x6d, 0x91, 0x5e, 0x07, 0xa3,
-	0x68, 0xd2, 0x9f, 0x71, 0xad, 0x0a, 0x0f, 0xbf, 0x71, 0x16, 0x7b, 0x82, 0xa1, 0x71, 0xc1, 0x4a,
-	0x4a, 0xa8, 0xe4, 0xd9, 0x06, 0x90, 0x79, 0xd3, 0xa8, 0xf4, 0x1d, 0x76, 0xeb, 0x2c, 0x5e, 0x30,
-	0x12, 0xfe, 0x0a, 0x8b, 0xe9, 0xbb, 0x19, 0x30, 0x12, 0x3d, 0x96, 0x9e, 0xb2, 0xbc, 0xe4, 0xd4,
-	0x7f, 0x8d, 0xe5, 0x48, 0x62, 0x58, 0x4c, 0xdd, 0xcd, 0x7c, 0x52, 0x78, 0x58, 0x3a, 0x4b, 0x3f,
-	0x34, 0x88, 0x25, 0x79, 0x4a, 0x6d, 0x76, 0x09, 0x48, 0xc2, 0x37, 0xa3, 0x68, 0x70, 0xe4, 0x0e,
-	0xfc, 0xe9, 0xab, 0xd1, 0xa1, 0x7e, 0x85, 0xad, 0xbd, 0x1a, 0xf7, 0x0f, 0x43, 0x5a, 0x1e, 0x7c,
-	0xd0, 0x4b, 0x6c, 0x6b, 0x32, 0x0a, 0x43, 0x37, 0xf4, 0xa6, 0x43, 0x77, 0xec, 0x4d, 0xb9, 0x3d,
-	0xb6, 0xac, 0x02, 0x00, 0x6d, 0x6f, 0x3a, 0x6c, 0x79, 0xd3, 0x52, 0x85, 0x6d, 0xf1, 0x75, 0xea,
-	0x1f, 0x8c, 0x3d, 0x77, 0xe2, 0x0f, 0xf5, 0x1b, 0x2c, 0x87, 0x0f, 0xa3, 0xa1, 0x58, 0x6c, 0xfe,
-	0xdc, 0x1c, 0xea, 0xd7, 0xd8, 0x3a, 0x8e, 0x47, 0x82, 0xe8, 0xa9, 0xf4, 0x4f, 0xd3, 0x2c, 0x07,
-	0x42, 0x66, 0x7e, 0x10, 0xe9, 0xd7, 0xd9, 0x06, 0xfc, 0xeb, 0x4e, 0x7d, 0x62, 0x5f, 0x87, 0xc7,
-	0x8e, 0x0f, 0x88, 0xa3, 0x37, 0x6e, 0x7f, 0x38, 0x0c, 0xc8, 0x3e, 0xeb, 0x47, 0x6f, 0x8c, 0xe1,
-	0x30, 0xd0, 0x75, 0x96, 0x9d, 0xf6, 0x27, 0x1e, 0xf7, 0x8c, 0xbc, 0xc5, 0x7f, 0x2b, 0x43, 0x65,
-	0xd5, 0xa1, 0x60, 0xa2, 0x61, 0xd4, 0x8f, 0xbc, 0xe2, 0x1a, 0x4e, 0x94, 0x3f, 0x80, 0x84, 0xc1,
-	0x71, 0x10, 0x14, 0xd7, 0x39, 0x90, 0xff, 0xd6, 0x3f, 0x60, 0xac, 0x3f, 0x3c, 0xf1, 0x82, 0x68,
-	0x14, 0x7a, 0xc3, 0xe2, 0x06, 0xc7, 0x28, 0x10, 0xfd, 0x16, 0xcb, 0x87, 0xc7, 0x33, 0xd0, 0xcd,
-	0x1b, 0x16, 0x73, 0x1c, 0x1d, 0x03, 0x40, 0xe2, 0xcc, 0xf3, 0x82, 0x62, 0x1e, 0x25, 0xc2, 0x6f,
-	0xfd, 0x36, 0x63, 0x20, 0xd9, 0x0d, 0x67, 0x9e, 0x37, 0x2c, 0x32, 0x64, 0x01, 0x88, 0x0d, 0x00,
-	0x7d, 0x87, 0xe5, 0x27, 0xfd, 0x53, 0xc2, 0x16, 0x38, 0x36, 0x37, 0xe9, 0x9f, 0x72, 0x64, 0xe9,
-	0xdf, 0xa6, 0xd8, 0x65, 0x65, 0xd9, 0x5e, 0x79, 0xfd, 0xe8, 0x38, 0xf0, 0x42, 0xfd, 0x0e, 0x2b,
-	0x0c, 0xfb, 0x51, 0x7f, 0xd6, 0x8f, 0x8e, 0x84, 0xc1, 0xb3, 0x16, 0x13, 0xa0, 0x26, 0x97, 0x3a,
-	0x75, 0x0f, 0x8e, 0x5f, 0xbd, 0xf2, 0x82, 0x90, 0xcc, 0x9e, 0x9b, 0x56, 0xf0, 0x19, 0xd6, 0x6a,
-	0x8a, 0x4b, 0x17, 0x52, 0x5c, 0x6d, 0x4c, 0x1d, 0xfe, 0xa8, 0xdf, 0x63, 0x9b, 0xfd, 0xe3, 0xd3,
-	0xd1, 0x78, 0xd4, 0x0f, 0xde, 0x82, 0x64, 0x34, 0x63, 0x41, 0xc2, 0x9a, 0x43, 0xbd, 0xc4, 0x36,
-	0x07, 0xfd, 0x59, 0xff, 0x60, 0x34, 0x1e, 0x45, 0x23, 0x2f, 0x24, 0x93, 0x26, 0x60, 0xa5, 0x80,
-	0x5d, 0x14, 0x2b, 0xeb, 0x82, 0xad, 0x8f, 0x43, 0xfd, 0x11, 0x5b, 0x0f, 0xbc, 0x7e, 0x48, 0xb9,
-	0x60, 0xfb, 0xe1, 0xad, 0x05, 0xf7, 0xe5, 0xd4, 0x48, 0x63, 0x11, 0x2d, 0x24, 0x8a, 0xa1, 0x17,
-	0x0e, 0x28, 0x24, 0xaf, 0x2e, 0xe5, 0xb1, 0x38, 0x49, 0xe9, 0xef, 0xa6, 0xd8, 0xa6, 0x14, 0x03,
-	0x2e, 0xf9, 0xf3, 0xbb, 0x54, 0xec, 0x3e, 0x99, 0x84, 0xfb, 0xe8, 0x2c, 0x3b, 0xe9, 0x87, 0xaf,
-	0xc9, 0x1a, 0xfc, 0x37, 0x38, 0x82, 0x74, 0x0b, 0xb2, 0x41, 0x0c, 0x28, 0xbd, 0xc1, 0xd8, 0x9d,
-	0xf4, 0xa3, 0xc1, 0x91, 0x7e, 0x3f, 0x91, 0x96, 0x76, 0x16, 0x26, 0xc1, 0xa9, 0xd4, 0x8c, 0xf4,
-	0xab, 0x8c, 0xf9, 0xa7, 0x13, 0xf7, 0xd5, 0xc8, 0x1b, 0x0f, 0x31, 0x2d, 0x14, 0x1e, 0xde, 0x5c,
-	0x60, 0x93, 0x24, 0x56, 0xde, 0x3f, 0x9d, 0xd4, 0x39, 0x71, 0xe9, 0xbf, 0xa7, 0x30, 0x32, 0x25,
-	0x52, 0xff, 0x36, 0x03, 0xb4, 0x3b, 0x18, 0xf7, 0xc3, 0x90, 0x54, 0x58, 0x2e, 0x8b, 0x53, 0x58,
-	0x39, 0xff, 0x74, 0x52, 0x85, 0x5f, 0xfa, 0x0f, 0x60, 0x0e, 0x07, 0x28, 0x85, 0x4f, 0xbd, 0xf0,
-	0xf0, 0x83, 0xa5, 0x8c, 0x92, 0xaa, 0x71, 0xc1, 0xca, 0xf9, 0xaf, 0x0e, 0xb8, 0x2a, 0xfa, 0x0b,
-	0xa6, 0x7b, 0xa7, 0x33, 0x2f, 0x18, 0x41, 0x02, 0xf2, 0x02, 0x92, 0xb3, 0xc6, 0xe5, 0x7c, 0x7d,
-	0xa9, 0x9c, 0x45, 0xf2, 0xc6, 0x05, 0xeb, 0x92, 0x0a, 0xe5, 0x92, 0x2b, 0x1b, 0x6c, 0x8d, 0x63,
-	0x4b, 0x7f, 0xbc, 0x8d, 0x59, 0x2d, 0xa1, 0xc4, 0xd9, 0xbb, 0x80, 0x4a, 0xc9, 0x4d, 0x1e, 0x92,
-	0xcd, 0x6f, 0xb0, 0xdc, 0x51, 0x3f, 0x74, 0xf9, 0x3a, 0x83, 0xb7, 0xe5, 0xac, 0x8d, 0xa3, 0x7e,
-	0xd8, 0x86, 0xa5, 0xbe, 0xc2, 0xb2, 0xe0, 0x39, 0xe8, 0x14, 0x8d, 0x0b, 0x16, 0x7f, 0xd2, 0x3f,
-	0x66, 0x5b, 0xb3, 0xa3, 0xb7, 0xe1, 0x68, 0xd0, 0x1f, 0x73, 0x9f, 0x43, 0xef, 0x68, 0x5c, 0xb0,
-	0x36, 0x05, 0xb8, 0x07, 0x64, 0x5f, 0x67, 0xdb, 0x94, 0x25, 0xbd, 0xa8, 0x0f, 0x11, 0xca, 0x4d,
-	0x90, 0x85, 0x3d, 0x83, 0xc3, 0xdb, 0x04, 0xd6, 0x6f, 0xb0, 0x0d, 0x2f, 0x3a, 0x72, 0x87, 0x61,
-	0xc4, 0x13, 0xd2, 0x66, 0xe3, 0x82, 0xb5, 0xee, 0x45, 0x47, 0xb5, 0x30, 0x12, 0xa8, 0x30, 0x18,
-	0xf0, 0x8c, 0x24, 0x50, 0x76, 0x30, 0xd0, 0x77, 0x58, 0x0e, 0x50, 0x7c, 0xc2, 0x39, 0x52, 0x00,
-	0x88, 0x1d, 0x98, 0xd3, 0x0e, 0xcb, 0x9d, 0x8c, 0xfb, 0x53, 0xf7, 0x64, 0x34, 0xc4, 0x94, 0x04,
-	0x48, 0x80, 0xec, 0x8f, 0x86, 0x12, 0x39, 0x1b, 0xcc, 0x30, 0x2b, 0x09, 0x64, 0x6f, 0x30, 0x83,
-	0x11, 0x47, 0x33, 0x77, 0x18, 0x0e, 0x66, 0x98, 0x93, 0x60, 0xc4, 0xd1, 0xac, 0x16, 0x0e, 0x66,
-	0xfa, 0x75, 0xb6, 0x3e, 0x9a, 0xb9, 0xde, 0x60, 0x5a, 0xdc, 0x24, 0xcc, 0xda, 0x68, 0x66, 0x0e,
-	0xa6, 0x20, 0x70, 0x34, 0xc3, 0x32, 0xa2, 0xb8, 0x25, 0x04, 0x8e, 0x66, 0x3d, 0x5e, 0x44, 0x70,
-	0xe4, 0xc9, 0x23, 0x3e, 0x87, 0xed, 0x18, 0x79, 0xf2, 0x88, 0x26, 0xc1, 0x91, 0x30, 0xf7, 0x8b,
-	0x2a, 0x92, 0x26, 0x1f, 0x0d, 0x66, 0x9c, 0x51, 0x13, 0xaa, 0x44, 0x83, 0x19, 0xf0, 0x11, 0x0a,
-	0xd8, 0x2e, 0x29, 0x28, 0xe2, 0x3a, 0x1e, 0x22, 0x97, 0x2e, 0x50, 0xc7, 0x43, 0xc1, 0x05, 0x28,
-	0xe0, 0xba, 0xac, 0xa0, 0x80, 0x6b, 0x87, 0xe5, 0xc2, 0x41, 0x84, 0x6c, 0x57, 0x84, 0x22, 0x00,
-	0x21, 0x2d, 0x39, 0x12, 0x18, 0xaf, 0xaa, 0x48, 0xe0, 0xbc, 0xc7, 0x0a, 0xa3, 0xc1, 0x04, 0x26,
-	0xc1, 0x97, 0xe2, 0x1a, 0xe1, 0x19, 0x02, 0xf9, 0x6a, 0xc4, 0x24, 0x03, 0x7f, 0xe8, 0x15, 0xaf,
-	0x27, 0x49, 0xaa, 0xfe, 0xd0, 0x03, 0xdb, 0xf6, 0x83, 0x99, 0xeb, 0xcf, 0x8a, 0x45, 0x61, 0xdb,
-	0x7e, 0x30, 0xeb, 0xf2, 0xf5, 0x00, 0x44, 0x38, 0xeb, 0x17, 0x6f, 0x08, 0x9d, 0xfb, 0xc1, 0xcc,
-	0x9e, 0xf5, 0x05, 0x2a, 0x9a, 0xf5, 0x8b, 0x37, 0x15, 0x94, 0x13, 0xa3, 0xc2, 0xa3, 0x7e, 0x71,
-	0x47, 0xf8, 0x0d, 0x70, 0x1d, 0xc5, 0x5c, 0x47, 0xfd, 0xe2, 0x2d, 0x05, 0xe5, 0x1c, 0xf5, 0x69,
-	0x35, 0x1e, 0x73, 0x23, 0xdc, 0x26, 0x1c, 0xac, 0xc6, 0xe3, 0x78, 0xa9, 0x1e, 0x73, 0x23, 0x7c,
-	0xa0, 0x22, 0x85, 0x11, 0x00, 0xf9, 0x6a, 0xdc, 0x3f, 0xf0, 0xc6, 0xc5, 0x3b, 0x72, 0x86, 0xb3,
-	0x93, 0xc7, 0x75, 0x0e, 0x93, 0x46, 0x78, 0x8c, 0x76, 0xba, 0x9b, 0x30, 0xc2, 0xe3, 0x84, 0x9d,
-	0x1e, 0xa3, 0x9d, 0xee, 0x25, 0x49, 0xb8, 0x9d, 0xbe, 0xc6, 0xb6, 0xf9, 0x40, 0xd3, 0xa1, 0x1b,
-	0xf5, 0x83, 0x43, 0x2f, 0x2a, 0x96, 0x48, 0x97, 0x4d, 0x80, 0x77, 0x86, 0x0e, 0x87, 0xea, 0x77,
-	0x49, 0xa1, 0xe9, 0xd0, 0x0d, 0xc3, 0x71, 0xf1, 0x43, 0x22, 0xca, 0x23, 0x91, 0x1d, 0x8e, 0x55,
-	0x8a, 0x68, 0x3c, 0x2e, 0x7e, 0x94, 0xa4, 0x70, 0xc6, 0x63, 0xfd, 0x0e, 0x63, 0x93, 0xd9, 0x38,
-	0x74, 0x71, 0x4e, 0x1f, 0x93, 0x36, 0x79, 0x80, 0xb5, 0xf8, 0x94, 0x6e, 0xb0, 0x0d, 0x4e, 0x10,
-	0x0d, 0x8a, 0x5f, 0x13, 0x0b, 0x00, 0x00, 0x87, 0x5b, 0x8b, 0xa3, 0x0e, 0xfc, 0xb0, 0xf8, 0x75,
-	0xe1, 0x32, 0x00, 0xa9, 0xf8, 0x21, 0x20, 0x67, 0x07, 0x07, 0xee, 0x28, 0x1c, 0x0d, 0x8b, 0x9f,
-	0x08, 0xe4, 0xec, 0xe0, 0xa0, 0x19, 0x8e, 0x86, 0xfa, 0x6d, 0x96, 0x8f, 0x8e, 0xa7, 0x53, 0x6f,
-	0x0c, 0xbb, 0xf0, 0x37, 0x28, 0x63, 0xe4, 0x10, 0xd4, 0x1c, 0x4a, 0x4b, 0x7b, 0xa7, 0xd1, 0xd1,
-	0x30, 0x28, 0x96, 0x55, 0x4b, 0x9b, 0x1c, 0xa6, 0x7f, 0xc6, 0x2e, 0x27, 0x13, 0x0f, 0xe6, 0xb6,
-	0x11, 0x97, 0x95, 0xb2, 0x2e, 0x25, 0xb2, 0x0f, 0xcf, 0x73, 0x25, 0xb6, 0x49, 0x19, 0x08, 0x49,
-	0x7f, 0x9d, 0x1b, 0x23, 0x65, 0x31, 0x4c, 0x43, 0x2a, 0x4d, 0x18, 0x0c, 0x90, 0xe6, 0xb5, 0x42,
-	0x63, 0x07, 0x03, 0x4e, 0xf3, 0x11, 0xdb, 0x12, 0x69, 0x07, 0x89, 0x26, 0x5c, 0xbd, 0x94, 0x55,
-	0xa0, 0xdc, 0x23, 0xa8, 0x44, 0x46, 0x40, 0xaa, 0x40, 0x50, 0x51, 0x5a, 0x48, 0x50, 0x49, 0xa5,
-	0x42, 0x95, 0x4a, 0xd1, 0x8a, 0xc2, 0x03, 0x89, 0x7e, 0x93, 0x88, 0x18, 0xc6, 0x88, 0x4a, 0x13,
-	0x09, 0x9a, 0xbf, 0xa1, 0xd0, 0x38, 0x44, 0xf3, 0x31, 0x1f, 0xed, 0x71, 0xac, 0xd3, 0xdf, 0x4c,
-	0xd1, 0xfc, 0x0a, 0x14, 0x00, 0x09, 0x32, 0xa9, 0xd4, 0xdf, 0x4a, 0x90, 0x09, 0xad, 0xbe, 0xc9,
-	0x34, 0x25, 0x1c, 0x90, 0xf2, 0xb7, 0x52, 0x34, 0xec, 0x76, 0x1c, 0x14, 0x42, 0xa6, 0xf0, 0x06,
-	0xa4, 0xfc, 0xfb, 0x82, 0xb2, 0x40, 0x3e, 0xc1, 0xc9, 0x60, 0x3b, 0x11, 0x7e, 0x81, 0x74, 0xbf,
-	0x9d, 0xa2, 0x15, 0xdd, 0x14, 0xde, 0x91, 0x18, 0x1c, 0x3d, 0x04, 0x49, 0x7f, 0x27, 0x31, 0x38,
-	0xfa, 0x09, 0x10, 0xc3, 0x8e, 0x7a, 0xd2, 0x1f, 0x1f, 0x7b, 0x95, 0x75, 0xac, 0x74, 0x4a, 0x2e,
-	0xbb, 0xb9, 0x7a, 0x57, 0x86, 0x92, 0x16, 0x30, 0x78, 0xc8, 0xa0, 0xe2, 0x0a, 0x8a, 0x8c, 0x06,
-	0x1e, 0xc3, 0xc0, 0x47, 0x14, 0x26, 0xaa, 0x3f, 0x13, 0xb0, 0xd2, 0xbf, 0xc9, 0xe2, 0x51, 0xb1,
-	0x3f, 0x80, 0xf3, 0xa3, 0xfe, 0x59, 0x62, 0xcf, 0x5e, 0xac, 0x0d, 0x91, 0x4c, 0xad, 0x91, 0xbe,
-	0xc3, 0xd6, 0xfd, 0xe3, 0x68, 0x76, 0x1c, 0x51, 0x6d, 0xf8, 0xc1, 0x2a, 0x1e, 0xa4, 0x82, 0xa0,
-	0xc4, 0x5f, 0xfa, 0x0f, 0x28, 0x28, 0xa3, 0x68, 0xcc, 0xb7, 0xf4, 0xc2, 0x92, 0x93, 0x22, 0xf1,
-	0x0a, 0x3a, 0x11, 0xb6, 0x4e, 0x34, 0xd6, 0x1f, 0xb2, 0xec, 0xec, 0x38, 0x3c, 0xa2, 0x8a, 0x68,
-	0xa5, 0xaa, 0x40, 0xc3, 0x6b, 0x85, 0xe3, 0xf0, 0x08, 0x86, 0x9c, 0xf9, 0x33, 0x2e, 0x8e, 0x2a,
-	0xa0, 0x95, 0x43, 0x0a, 0x3a, 0x9e, 0x0c, 0xfc, 0x59, 0x7b, 0x36, 0x0e, 0xf5, 0x6f, 0xb1, 0xb5,
-	0xc3, 0xc0, 0x3f, 0x9e, 0xf1, 0xc2, 0xa0, 0xf0, 0xf0, 0xf6, 0x2a, 0x5e, 0x4e, 0x04, 0x9b, 0x06,
-	0xff, 0xa1, 0x7f, 0x9b, 0xad, 0x4f, 0xdf, 0xf0, 0x69, 0x6e, 0x9c, 0x6d, 0x22, 0xa4, 0x02, 0xc6,
-	0xe9, 0x1b, 0x98, 0xe2, 0x13, 0x96, 0x0f, 0xbd, 0x88, 0x2a, 0xb6, 0x1c, 0xe7, 0xbd, 0xb7, 0x8a,
-	0x57, 0x12, 0x42, 0x7e, 0x0a, 0xbd, 0x08, 0x8b, 0xbf, 0x2f, 0xe6, 0x5c, 0x20, 0xcf, 0x85, 0x7c,
-	0xb4, 0x4a, 0x88, 0x4a, 0x0b, 0x49, 0x5c, 0x7d, 0xae, 0xe4, 0xd8, 0x3a, 0x92, 0x95, 0x9e, 0x60,
-	0xb9, 0x97, 0x58, 0x58, 0x7e, 0xe6, 0x82, 0xf2, 0x2b, 0x45, 0x67, 0x2e, 0x3a, 0x4d, 0xc2, 0xa1,
-	0x2a, 0x3e, 0xbc, 0xae, 0x4f, 0xfa, 0xa7, 0x70, 0x6e, 0xfd, 0x0c, 0xcf, 0x53, 0x73, 0xcb, 0x0b,
-	0xc5, 0x9f, 0x74, 0x09, 0x3a, 0xbd, 0xd2, 0x72, 0x97, 0xee, 0xe3, 0x51, 0x46, 0x59, 0x55, 0x28,
-	0xfd, 0xbd, 0xe8, 0xc8, 0x0b, 0xa4, 0xc7, 0x6e, 0x59, 0x31, 0xa0, 0xf4, 0x79, 0x62, 0x08, 0xb1,
-	0x9c, 0x5f, 0xc2, 0xf4, 0x2b, 0x4c, 0x9b, 0x5f, 0x47, 0x50, 0x8a, 0xff, 0x50, 0x8e, 0xd4, 0xfc,
-	0xb9, 0x39, 0x2c, 0x95, 0x13, 0x86, 0xc0, 0xe5, 0xd3, 0xaf, 0xca, 0xe5, 0xa6, 0xe3, 0x3c, 0x5f,
-	0xcc, 0x52, 0x83, 0x5d, 0x59, 0xb6, 0x5c, 0xfa, 0x67, 0x54, 0x45, 0x73, 0xea, 0xb3, 0xcf, 0x17,
-	0x54, 0x6e, 0x3f, 0x63, 0xd7, 0x57, 0xac, 0xd9, 0x42, 0xc8, 0xa7, 0x16, 0x43, 0x1e, 0x16, 0x8a,
-	0xd7, 0xbf, 0xb0, 0x22, 0x9b, 0x16, 0xff, 0x5d, 0xfa, 0xfd, 0x0c, 0x9a, 0x77, 0x34, 0x0d, 0xa3,
-	0xe0, 0x18, 0x73, 0x81, 0xae, 0xe4, 0x82, 0x2d, 0x8a, 0xf6, 0x06, 0x63, 0x87, 0x7e, 0xe4, 0xe3,
-	0xa9, 0x95, 0x22, 0x7e, 0xf1, 0x10, 0xa1, 0x48, 0x71, 0x63, 0x72, 0xd8, 0xad, 0xe1, 0x89, 0x1f,
-	0x71, 0x75, 0x87, 0x6d, 0xbf, 0x09, 0x46, 0x91, 0x52, 0x8f, 0x63, 0x0e, 0xf8, 0xe6, 0x99, 0xd2,
-	0x92, 0x2c, 0x50, 0xbc, 0x73, 0x88, 0x2c, 0xde, 0x9f, 0xb0, 0x0d, 0x34, 0x4b, 0x48, 0x79, 0xe1,
-	0xa3, 0x33, 0xc5, 0x11, 0x2d, 0xc4, 0x38, 0xfd, 0xd4, 0xbf, 0xcb, 0xd6, 0x26, 0x1e, 0x98, 0x0e,
-	0xf3, 0x43, 0xe9, 0x4c, 0x7e, 0x4e, 0x09, 0xf1, 0xca, 0x7f, 0xe8, 0xdd, 0x39, 0xeb, 0xaf, 0xaf,
-	0x68, 0x60, 0xa9, 0x22, 0xce, 0x0c, 0xb9, 0x75, 0x5c, 0xaa, 0xd2, 0xb7, 0x71, 0x1b, 0x58, 0x6e,
-	0xd7, 0x33, 0x7a, 0x3e, 0xa5, 0x3e, 0xfb, 0xe0, 0x6c, 0x13, 0xea, 0x37, 0x59, 0x4e, 0xae, 0x00,
-	0xf6, 0x2f, 0xe4, 0xb3, 0xfe, 0x21, 0xdb, 0x4a, 0x16, 0x2d, 0x69, 0x4e, 0xb0, 0x39, 0x51, 0xaa,
-	0x95, 0x52, 0x0b, 0xbd, 0x71, 0x89, 0x59, 0xf5, 0x07, 0xf1, 0x6a, 0x60, 0xaf, 0xec, 0xfa, 0x8a,
-	0xc4, 0x23, 0xcd, 0x5f, 0x7a, 0x88, 0x3d, 0xc5, 0x05, 0x23, 0xf3, 0xd4, 0x00, 0x3f, 0x94, 0x49,
-	0xf2, 0xe7, 0xe6, 0xb0, 0xb4, 0x8f, 0xad, 0xbd, 0x55, 0x56, 0xfd, 0x85, 0x83, 0xe2, 0x4f, 0x32,
-	0xd8, 0xc9, 0xe0, 0xfa, 0x4e, 0x7c, 0xea, 0xa0, 0xf9, 0xaf, 0x47, 0x1e, 0x59, 0x8a, 0x9e, 0xf4,
-	0x3b, 0xac, 0x80, 0xbf, 0x54, 0x2b, 0x31, 0x04, 0xf1, 0x22, 0x40, 0x5d, 0xa1, 0x4c, 0xb2, 0x2b,
-	0xf7, 0x3d, 0xb6, 0x31, 0xf0, 0x27, 0x93, 0xfe, 0x14, 0xcf, 0xf6, 0xdb, 0x4b, 0x32, 0xbc, 0x18,
-	0xdf, 0x25, 0x42, 0x4b, 0x70, 0xe8, 0xf7, 0xd8, 0xe6, 0x68, 0x38, 0xf6, 0xdc, 0x68, 0x34, 0xf1,
-	0xfc, 0xe3, 0x88, 0xfa, 0x1f, 0x05, 0x80, 0x39, 0x08, 0x02, 0x92, 0xa3, 0x7e, 0x30, 0x94, 0x24,
-	0xd8, 0x64, 0x2b, 0x00, 0x4c, 0x90, 0xdc, 0x64, 0xb9, 0x59, 0x30, 0xf2, 0x83, 0x51, 0xf4, 0x96,
-	0x3a, 0x6d, 0xf2, 0x59, 0xdf, 0x61, 0x79, 0x6c, 0x5f, 0x81, 0xea, 0xd8, 0x67, 0xcb, 0x21, 0xa0,
-	0xc9, 0x9b, 0x8d, 0xfe, 0x71, 0x84, 0xa7, 0x6e, 0x6c, 0xb5, 0x6d, 0xf8, 0xc7, 0x11, 0x3f, 0x6e,
-	0xef, 0xb0, 0x3c, 0xa0, 0x70, 0xbb, 0xc4, 0x66, 0x1b, 0xd0, 0xee, 0xf2, 0x8c, 0x2a, 0xfb, 0x9d,
-	0x05, 0xb5, 0xdf, 0xf9, 0x97, 0xd8, 0x1a, 0xef, 0xc0, 0xf0, 0xf3, 0x6c, 0xe1, 0xe1, 0xb5, 0xe5,
-	0xfd, 0x19, 0x0b, 0x89, 0xf4, 0x27, 0x6c, 0x53, 0x59, 0xf0, 0xb0, 0xb8, 0xc5, 0x1d, 0xec, 0xd6,
-	0x59, 0xb1, 0x66, 0x25, 0x38, 0x4a, 0x3f, 0x49, 0x61, 0xe9, 0x73, 0x70, 0x3c, 0x78, 0xed, 0x45,
-	0xb0, 0xb8, 0x6f, 0xbc, 0xd1, 0xe1, 0x91, 0xd8, 0xc1, 0xe8, 0x09, 0x8a, 0xac, 0x37, 0xbc, 0x31,
-	0xc4, 0xa7, 0x89, 0xdb, 0x58, 0x9e, 0x43, 0xf8, 0x44, 0xef, 0xb0, 0x02, 0xa2, 0x71, 0xaa, 0xb8,
-	0xba, 0xc8, 0x81, 0x93, 0x7d, 0xa0, 0xa6, 0xa4, 0xf3, 0x05, 0xc1, 0x7f, 0xa2, 0xe6, 0x11, 0x6e,
-	0x3b, 0xe0, 0x79, 0xdf, 0x8f, 0xbd, 0x04, 0x4b, 0xb3, 0xc5, 0xbc, 0x24, 0x89, 0x17, 0xdd, 0xe4,
-	0x7e, 0xa2, 0xcd, 0xbf, 0xb3, 0x82, 0x55, 0x29, 0xea, 0xd4, 0x2d, 0x2f, 0x93, 0xd8, 0xf2, 0x60,
-	0x3a, 0x68, 0xb0, 0xd5, 0xd3, 0x41, 0xbc, 0x25, 0xe8, 0x4a, 0xbf, 0x9d, 0x62, 0xdb, 0xbc, 0x23,
-	0xd8, 0x87, 0x67, 0xa8, 0x17, 0x92, 0x6e, 0x95, 0x9a, 0x73, 0xab, 0xeb, 0x6c, 0x63, 0x34, 0x55,
-	0xcd, 0xbd, 0x3e, 0x9a, 0x72, 0x5b, 0x2b, 0xa6, 0xcc, 0x9c, 0xcf, 0x94, 0x32, 0xae, 0xb3, 0x6a,
-	0x5c, 0x93, 0x79, 0x49, 0x9f, 0xd1, 0xf4, 0x6c, 0x75, 0x7e, 0x55, 0x76, 0x4c, 0xd3, 0x2b, 0x02,
-	0x54, 0x0a, 0x9a, 0x6f, 0x9b, 0x9e, 0x11, 0xf7, 0x71, 0x2e, 0xc9, 0x26, 0x72, 0x89, 0x8c, 0x82,
-	0xb5, 0xf3, 0x44, 0x81, 0x98, 0xde, 0xba, 0x32, 0xbd, 0x7f, 0x92, 0xc1, 0x22, 0x86, 0x33, 0x05,
-	0xde, 0xc4, 0x3f, 0xf1, 0x56, 0xa7, 0x2e, 0x35, 0xf6, 0xd3, 0x73, 0xb1, 0xff, 0x7d, 0x39, 0xf1,
-	0x0c, 0x9f, 0xf8, 0x47, 0xcb, 0x33, 0x13, 0x0d, 0x71, 0xd6, 0xdc, 0xb3, 0xc9, 0xb9, 0xdf, 0x63,
-	0x9b, 0xc3, 0xe3, 0xa0, 0x4f, 0x85, 0xd0, 0x40, 0xa4, 0x2d, 0x01, 0xb3, 0xbd, 0x01, 0x6c, 0x3d,
-	0x92, 0x64, 0x0a, 0x34, 0x98, 0xb7, 0x24, 0x5f, 0x27, 0xf4, 0x06, 0x0b, 0xe9, 0x6f, 0xe3, 0xcb,
-	0xd3, 0x5f, 0x6e, 0x31, 0xfd, 0xdd, 0x63, 0x9b, 0xb4, 0x80, 0x03, 0xff, 0x78, 0x8a, 0x99, 0x2c,
-	0x6b, 0x15, 0x10, 0x56, 0x05, 0x10, 0xe4, 0x80, 0x83, 0xb7, 0x91, 0x47, 0x04, 0x8c, 0x13, 0xe4,
-	0x01, 0x82, 0x68, 0xb9, 0x66, 0x6f, 0xcf, 0xb1, 0x66, 0xa5, 0x3f, 0x49, 0xe3, 0x1e, 0x87, 0xdb,
-	0xd9, 0x41, 0x7f, 0x3a, 0x3c, 0xef, 0x7b, 0x33, 0x85, 0x43, 0x09, 0x56, 0x9d, 0x65, 0x83, 0x7e,
-	0xe4, 0xd1, 0xf2, 0xf1, 0xdf, 0x5c, 0xe1, 0xe3, 0x20, 0x8c, 0xdc, 0x70, 0xf4, 0x1b, 0x1e, 0xb9,
-	0x5e, 0x9e, 0x43, 0xec, 0xd1, 0x6f, 0x78, 0xfa, 0x63, 0x96, 0x1d, 0x06, 0xfe, 0x8c, 0x6a, 0xa4,
-	0x33, 0x07, 0x02, 0x3a, 0x38, 0x3f, 0xc1, 0xbf, 0xfa, 0x17, 0xac, 0x30, 0x0c, 0x07, 0x33, 0x58,
-	0xf2, 0x7e, 0xf0, 0x7a, 0x65, 0x13, 0x59, 0x65, 0x8f, 0xc9, 0x1b, 0x17, 0x2c, 0x06, 0x8f, 0x16,
-	0x7f, 0xd2, 0x3b, 0x4b, 0x8b, 0xa5, 0x4f, 0xce, 0x12, 0x76, 0xae, 0x5a, 0xe9, 0x2a, 0xd6, 0xfd,
-	0x73, 0x53, 0x28, 0x7d, 0x0f, 0x4b, 0xa8, 0xe5, 0xaa, 0x81, 0xbd, 0x66, 0x81, 0x37, 0x70, 0xc7,
-	0xde, 0x89, 0x27, 0xea, 0xf6, 0x3c, 0x40, 0x5a, 0x00, 0x28, 0x19, 0x6c, 0xe7, 0x0c, 0x55, 0xce,
-	0x53, 0x60, 0x94, 0xfe, 0x1d, 0x25, 0x1d, 0x94, 0x71, 0xce, 0x9c, 0x2e, 0x89, 0x17, 0x73, 0xba,
-	0xdc, 0x43, 0xd3, 0xea, 0x1e, 0xaa, 0x56, 0x49, 0x99, 0x44, 0x95, 0xa4, 0x7f, 0x87, 0xad, 0x81,
-	0xe6, 0x22, 0x6d, 0x97, 0xce, 0x32, 0x34, 0xbd, 0xb6, 0x44, 0x86, 0xd2, 0x8f, 0x51, 0x73, 0x2f,
-	0x08, 0xfc, 0xc0, 0x9d, 0x84, 0x87, 0xfa, 0x7d, 0xb6, 0xae, 0xf4, 0x1c, 0x96, 0xa5, 0x61, 0x12,
-	0x40, 0x64, 0xf2, 0x28, 0x91, 0x56, 0x8e, 0x12, 0x3a, 0xcb, 0xf2, 0xbe, 0x62, 0x86, 0xde, 0xfa,
-	0xf9, 0x43, 0x6f, 0x69, 0xb6, 0xfe, 0xad, 0x14, 0xae, 0x1c, 0x0e, 0x9f, 0xe8, 0x82, 0x80, 0x2e,
-	0xcb, 0x4e, 0x29, 0x37, 0x58, 0xce, 0x3b, 0xc5, 0x0d, 0x8d, 0x86, 0xdc, 0xf0, 0x4e, 0x67, 0xbc,
-	0xa9, 0x39, 0xbf, 0x54, 0x99, 0x33, 0x6a, 0x41, 0x55, 0x8b, 0x13, 0x8a, 0xd9, 0xe3, 0x71, 0x34,
-	0x9a, 0xf5, 0xf9, 0x0b, 0xb2, 0x1f, 0x1d, 0x7b, 0x61, 0xa4, 0x7f, 0x9e, 0x88, 0xd9, 0x3b, 0x8b,
-	0x56, 0x95, 0x1c, 0x4a, 0xc8, 0x2e, 0x5f, 0x3c, 0x9d, 0x65, 0x0f, 0xfc, 0xe1, 0x5b, 0xae, 0xd3,
-	0xa6, 0xc5, 0x7f, 0x97, 0x22, 0xf2, 0x66, 0x65, 0xdc, 0xd9, 0xf8, 0xed, 0x2f, 0x7b, 0xd4, 0xdf,
-	0x4d, 0xe1, 0x2b, 0xe1, 0xa1, 0x17, 0x0e, 0xb8, 0x4f, 0xbd, 0x0a, 0xf8, 0x6f, 0x3e, 0x5e, 0xde,
-	0xda, 0x98, 0xbc, 0x0a, 0x6a, 0x80, 0xc2, 0x37, 0x78, 0xf2, 0xcd, 0x60, 0xde, 0x5a, 0x3f, 0x7a,
-	0x23, 0x10, 0x21, 0x21, 0xf0, 0xbd, 0xf0, 0x7a, 0x88, 0x88, 0xdb, 0x8c, 0x85, 0x5e, 0x30, 0xea,
-	0x8f, 0xdd, 0xe9, 0xf1, 0x84, 0x5b, 0x38, 0x6f, 0xe5, 0x11, 0xd2, 0x39, 0x9e, 0x00, 0xdf, 0x10,
-	0x87, 0xe5, 0xc9, 0x25, 0x6f, 0xad, 0x0f, 0x67, 0xc0, 0x57, 0xfa, 0xa3, 0x14, 0xbb, 0x26, 0x77,
-	0x9c, 0x30, 0xea, 0x47, 0xa1, 0x5c, 0x81, 0x33, 0x5e, 0x79, 0xab, 0x05, 0x6a, 0xfa, 0x8c, 0x02,
-	0x35, 0x33, 0x57, 0xa0, 0xae, 0xda, 0x9c, 0xe7, 0x0a, 0xfd, 0xb5, 0x85, 0x42, 0x5f, 0xee, 0x04,
-	0xeb, 0xe7, 0xd9, 0x09, 0xfe, 0x30, 0x83, 0x85, 0x51, 0x3c, 0x29, 0x7d, 0x9b, 0xa5, 0x47, 0x43,
-	0xfe, 0x66, 0x26, 0x6b, 0xa5, 0x47, 0x67, 0xbe, 0xcf, 0x9f, 0xdf, 0x45, 0xd3, 0xe7, 0xd8, 0x45,
-	0x33, 0x4b, 0x76, 0x51, 0xb5, 0x04, 0xc8, 0xce, 0x95, 0x00, 0x5f, 0xcd, 0x01, 0x43, 0x3a, 0xde,
-	0x86, 0xea, 0x78, 0xb1, 0x91, 0x73, 0x09, 0x23, 0x7f, 0x85, 0xfb, 0xf1, 0xff, 0xa3, 0x93, 0xc4,
-	0x1f, 0xa7, 0x70, 0x7f, 0xe8, 0x1f, 0x1e, 0x06, 0xde, 0x61, 0x3f, 0xf2, 0xfe, 0xbf, 0xf1, 0xd0,
-	0x1f, 0xb3, 0x1b, 0xcb, 0x27, 0x06, 0x49, 0x68, 0x7e, 0xa1, 0x52, 0x5f, 0xb6, 0x50, 0xe9, 0xf9,
-	0x85, 0xba, 0xcd, 0x18, 0x1f, 0x1a, 0xd1, 0x54, 0xa6, 0x00, 0x84, 0xa3, 0x4b, 0x7f, 0x9e, 0xc1,
-	0xd4, 0x8f, 0xc6, 0xa3, 0x5b, 0x17, 0xee, 0x2c, 0xf0, 0x67, 0x5e, 0xc0, 0xeb, 0x53, 0x35, 0x09,
-	0x2e, 0x56, 0x0e, 0x8b, 0x6c, 0x6a, 0x36, 0xdc, 0x9f, 0x5b, 0x76, 0x6c, 0x66, 0x7d, 0x76, 0x1e,
-	0x29, 0x2a, 0x1f, 0x7f, 0xd7, 0xa5, 0x3c, 0xeb, 0x16, 0x2b, 0x4c, 0xbd, 0xd3, 0x48, 0xbd, 0xd8,
-	0x51, 0x78, 0x78, 0xff, 0x3c, 0x62, 0x15, 0x36, 0xa8, 0x95, 0xe0, 0x91, 0xae, 0x83, 0xec, 0xce,
-	0xb7, 0xb5, 0xbe, 0x79, 0x1e, 0x79, 0x4b, 0xba, 0x5b, 0xdf, 0x63, 0x19, 0xff, 0x74, 0xb2, 0xb2,
-	0x70, 0x5b, 0x22, 0xc4, 0x3f, 0x9d, 0x34, 0x2e, 0x58, 0xc0, 0x05, 0x16, 0x5b, 0x52, 0xb1, 0x9d,
-	0xcb, 0x62, 0x67, 0x56, 0x6e, 0xe2, 0xad, 0x47, 0xe9, 0x90, 0x7d, 0x78, 0x0e, 0x8b, 0x2f, 0x04,
-	0x6c, 0xea, 0xe7, 0x0e, 0xd8, 0x2f, 0x58, 0xe9, 0xcb, 0xd7, 0x40, 0xff, 0x88, 0x6d, 0xc7, 0x8f,
-	0xee, 0x68, 0x88, 0x23, 0x6d, 0x59, 0x9b, 0x72, 0x65, 0x9a, 0xc3, 0xb0, 0x64, 0x63, 0x8b, 0x6d,
-	0xb5, 0xfd, 0x7f, 0x91, 0x36, 0xd8, 0xb7, 0x56, 0x39, 0x3e, 0xac, 0x07, 0xec, 0x92, 0xfe, 0xe9,
-	0x84, 0x6b, 0x94, 0xc1, 0x8b, 0x33, 0xfe, 0xe9, 0x04, 0x74, 0xf9, 0x87, 0xa9, 0x95, 0x16, 0x3c,
-	0xb3, 0x60, 0x5d, 0xf2, 0x66, 0x28, 0x51, 0x44, 0x65, 0x92, 0x45, 0xd4, 0x37, 0x59, 0xe2, 0x36,
-	0x88, 0x4b, 0xd5, 0x12, 0x68, 0xa2, 0xa9, 0x88, 0x1a, 0x54, 0x4e, 0xbf, 0x97, 0x66, 0xfa, 0x82,
-	0x4e, 0xe1, 0x59, 0x39, 0x51, 0xdc, 0x28, 0x4b, 0x2b, 0x37, 0xca, 0x3e, 0x66, 0xdb, 0x4a, 0x2b,
-	0x12, 0xf2, 0x57, 0x86, 0x27, 0x93, 0xad, 0xb8, 0x17, 0x09, 0xb9, 0x5c, 0x25, 0xe3, 0x8d, 0x4e,
-	0x4a, 0x8f, 0x92, 0xec, 0x39, 0x00, 0x95, 0x0b, 0x46, 0x6b, 0x89, 0x0b, 0x46, 0x77, 0x58, 0x61,
-	0xd2, 0x3f, 0x75, 0xbd, 0x69, 0x14, 0x8c, 0xbc, 0x90, 0xb6, 0x32, 0x36, 0xe9, 0x9f, 0x9a, 0x08,
-	0xd1, 0x77, 0xe1, 0x9c, 0xc0, 0xd3, 0x0f, 0xe0, 0x37, 0xf8, 0x6a, 0x9e, 0x27, 0x8c, 0x20, 0x5f,
-	0x59, 0x0a, 0x6b, 0xe9, 0x27, 0x29, 0x6c, 0xb8, 0x23, 0x29, 0xee, 0xfd, 0x67, 0xef, 0xf5, 0xe0,
-	0x1a, 0x27, 0x6a, 0x26, 0xdd, 0xb2, 0x0a, 0x08, 0xc3, 0x5c, 0x7a, 0x8f, 0x6d, 0x8e, 0x7d, 0xff,
-	0xf5, 0xf1, 0x4c, 0xc9, 0xa6, 0x59, 0xab, 0x80, 0x30, 0x24, 0xf9, 0x90, 0x6d, 0x71, 0xdb, 0x79,
-	0x43, 0xa2, 0xc9, 0x52, 0x3f, 0x17, 0x81, 0x98, 0x74, 0x1f, 0x60, 0xa1, 0x25, 0xef, 0x8c, 0xc5,
-	0xdb, 0xd8, 0xaa, 0x8b, 0x5c, 0xa5, 0x3f, 0xa5, 0x3a, 0x26, 0xe6, 0x59, 0x7d, 0xe9, 0xeb, 0x36,
-	0x63, 0xc1, 0x29, 0x75, 0x4c, 0x42, 0xb1, 0x23, 0x04, 0xa7, 0x3d, 0x04, 0x00, 0x3a, 0x8a, 0xd1,
-	0x38, 0x87, 0x7c, 0x24, 0xd1, 0x37, 0x58, 0x2e, 0x38, 0x75, 0x61, 0x03, 0x09, 0x49, 0xf9, 0x8d,
-	0xe0, 0xb4, 0x02, 0x8f, 0xdc, 0x7a, 0x02, 0x85, 0xdb, 0xde, 0x46, 0x44, 0x28, 0x1c, 0x13, 0x8e,
-	0x81, 0x33, 0x6f, 0xc8, 0x57, 0x95, 0x8f, 0x59, 0x43, 0x00, 0x8d, 0x29, 0xd0, 0x1b, 0x62, 0x4c,
-	0x81, 0xde, 0x61, 0xf9, 0xe0, 0x14, 0x8f, 0x1f, 0x21, 0x95, 0x2a, 0xb9, 0xe0, 0xd4, 0xe4, 0xcf,
-	0x80, 0x8c, 0x24, 0x12, 0x2b, 0x95, 0x5c, 0x24, 0x90, 0x77, 0xd9, 0x66, 0x70, 0xea, 0xbe, 0x0a,
-	0xfa, 0x13, 0x0f, 0x48, 0xa8, 0x50, 0x61, 0xc1, 0x69, 0x1d, 0x40, 0x26, 0xbf, 0xe6, 0x58, 0x08,
-	0x4e, 0x5d, 0xff, 0xc4, 0x0b, 0x38, 0x41, 0x41, 0xa8, 0xd6, 0x3d, 0xf1, 0x02, 0xc0, 0xdf, 0xe2,
-	0x9a, 0x0f, 0x82, 0x01, 0x47, 0x6f, 0x8a, 0xc1, 0xab, 0xc1, 0x00, 0xb9, 0xd9, 0xc0, 0x1f, 0x8f,
-	0x47, 0x21, 0xd5, 0x2d, 0xb4, 0xd7, 0x0b, 0xc8, 0x42, 0x85, 0xb8, 0x7d, 0x8e, 0x0a, 0xf1, 0xe2,
-	0x62, 0x85, 0x58, 0x7a, 0x84, 0x2d, 0x7e, 0x6c, 0x09, 0x2e, 0x94, 0x36, 0xab, 0x5e, 0x8e, 0xed,
-	0x63, 0xdc, 0x63, 0x17, 0x10, 0x1d, 0xce, 0x0b, 0xfe, 0xef, 0x8b, 0x86, 0xd2, 0x4f, 0xd2, 0x18,
-	0x3a, 0x8a, 0x3a, 0x67, 0xa8, 0xc1, 0x97, 0xcf, 0x7b, 0x95, 0x88, 0x9b, 0x5c, 0xe0, 0xbd, 0x92,
-	0x41, 0x93, 0xd0, 0x26, 0xf3, 0x65, 0xda, 0x64, 0xe7, 0x4b, 0x98, 0xaf, 0xaa, 0x97, 0x55, 0x61,
-	0x9b, 0x64, 0x29, 0x3e, 0x23, 0xca, 0x2d, 0x77, 0x56, 0x34, 0x57, 0x85, 0x39, 0xad, 0x02, 0x3e,
-	0xdb, 0xc0, 0x03, 0xc7, 0xb6, 0xed, 0xd8, 0x32, 0xfc, 0xf0, 0xf6, 0x65, 0x77, 0x1e, 0xcf, 0x6c,
-	0xfd, 0xa6, 0x57, 0xb6, 0x7e, 0x33, 0xe7, 0x6c, 0xfd, 0x9e, 0xa8, 0x4b, 0x05, 0x69, 0xf5, 0x2d,
-	0x68, 0x24, 0x8f, 0x92, 0x85, 0x95, 0x1a, 0x01, 0x09, 0x5e, 0x28, 0xd5, 0x1f, 0xe2, 0xa5, 0x61,
-	0x51, 0xa1, 0xdd, 0x5a, 0xc1, 0xc1, 0x69, 0xf0, 0x4a, 0x71, 0x58, 0xfa, 0x3b, 0x29, 0x74, 0x3e,
-	0x44, 0xc9, 0x4d, 0xe7, 0x0a, 0x5b, 0xe3, 0x77, 0x0d, 0xc5, 0x9b, 0x59, 0xfe, 0xb0, 0x70, 0x93,
-	0x36, 0xbd, 0x78, 0x93, 0x16, 0xbc, 0x00, 0x76, 0x06, 0x2e, 0x4f, 0xec, 0xba, 0xf9, 0x49, 0xff,
-	0x94, 0x57, 0xe3, 0xa1, 0x5e, 0x4c, 0x36, 0xf9, 0xb7, 0xe2, 0x9d, 0xfc, 0x3b, 0x6a, 0xeb, 0x68,
-	0xb1, 0x7d, 0x70, 0xc6, 0x6b, 0xad, 0x5f, 0xc7, 0x17, 0xc6, 0x4a, 0x5b, 0x06, 0x7d, 0xbd, 0xcc,
-	0x2e, 0x91, 0xcf, 0x72, 0xa0, 0x1a, 0x46, 0x17, 0x11, 0x51, 0xe9, 0x4f, 0x31, 0x99, 0xeb, 0x5f,
-	0x63, 0x17, 0xb9, 0xf3, 0x2a, 0x94, 0x18, 0x4f, 0x5b, 0x00, 0x96, 0x74, 0xa5, 0x3f, 0xa0, 0x98,
-	0xc2, 0xc1, 0x64, 0x4c, 0xad, 0x50, 0x6d, 0xae, 0x6e, 0x4f, 0xcf, 0xd5, 0xed, 0x30, 0x6a, 0xdc,
-	0x12, 0x57, 0x03, 0x6b, 0x0b, 0xc1, 0xcd, 0x29, 0xd2, 0x95, 0x18, 0x57, 0x23, 0xa6, 0xc2, 0xe8,
-	0x2a, 0x00, 0x50, 0xd0, 0x7c, 0x55, 0xf1, 0xf5, 0x84, 0xb1, 0xd8, 0x86, 0x14, 0x5d, 0xf7, 0xce,
-	0xea, 0x81, 0xa1, 0x3f, 0xe5, 0xe1, 0x37, 0x46, 0xd7, 0x6f, 0x62, 0x5b, 0x1d, 0x49, 0xce, 0xbc,
-	0xb9, 0xaf, 0x5a, 0x2e, 0xbd, 0xa2, 0x0b, 0x97, 0xf9, 0x79, 0xbb, 0x70, 0xff, 0x9a, 0x5c, 0x1a,
-	0x09, 0xa4, 0x4b, 0xd3, 0xbd, 0x75, 0x7c, 0x67, 0x9d, 0x92, 0xf7, 0xd6, 0xdb, 0xfc, 0xa5, 0xe9,
-	0x6d, 0x9a, 0x34, 0x3a, 0x3d, 0xad, 0x13, 0x40, 0x9c, 0xa5, 0x8e, 0x9f, 0x59, 0xe2, 0xf8, 0x24,
-	0x5f, 0xb4, 0x0e, 0x85, 0x7c, 0x70, 0x1d, 0x89, 0x1c, 0xf8, 0x63, 0x3f, 0xa0, 0x95, 0x01, 0x64,
-	0x15, 0x9e, 0x4b, 0x3f, 0x56, 0x5d, 0x0a, 0x63, 0xff, 0x5b, 0xb2, 0xee, 0x4a, 0xad, 0xb8, 0x41,
-	0xa3, 0x5a, 0x57, 0x96, 0x65, 0x5f, 0x9a, 0x01, 0x14, 0xb7, 0x15, 0x19, 0xe0, 0x84, 0xdd, 0xe3,
-	0x5d, 0xc3, 0x44, 0xbf, 0x50, 0x86, 0xdf, 0xd1, 0xf2, 0x1b, 0x52, 0xa9, 0x2f, 0xa9, 0x83, 0xe7,
-	0x9a, 0x89, 0xa2, 0x51, 0x98, 0x51, 0x1a, 0x85, 0x63, 0xdc, 0x2b, 0x13, 0xe3, 0xfe, 0xf2, 0x46,
-	0x33, 0xb1, 0x2d, 0xf9, 0xa3, 0x63, 0xef, 0x98, 0xea, 0x7c, 0x1a, 0x8b, 0x37, 0x75, 0xb0, 0xee,
-	0x14, 0x5e, 0x21, 0xcf, 0xcd, 0x1a, 0xcb, 0xc4, 0x37, 0x72, 0xe0, 0x67, 0x29, 0x40, 0xa5, 0x15,
-	0x31, 0x93, 0xd1, 0xd4, 0xe5, 0x6f, 0x12, 0xaa, 0xac, 0xa0, 0xc8, 0xa5, 0x75, 0x5b, 0x74, 0xdb,
-	0x05, 0x0d, 0xb0, 0xda, 0x6d, 0xc8, 0xfe, 0xef, 0xfc, 0x2b, 0x8a, 0x65, 0x63, 0xf6, 0x4f, 0x7f,
-	0xc9, 0x63, 0xfe, 0x23, 0x6a, 0xd4, 0x28, 0x9c, 0x09, 0xeb, 0x7f, 0x25, 0x03, 0x9f, 0xe7, 0x70,
-	0xb5, 0x6c, 0x2d, 0xff, 0x76, 0x0a, 0x13, 0x0c, 0xa5, 0x4e, 0x3e, 0x08, 0xf8, 0x03, 0x8e, 0x16,
-	0x27, 0x61, 0xfe, 0x8c, 0xc7, 0x24, 0xa5, 0x6d, 0x84, 0x17, 0xae, 0x2a, 0x89, 0xf3, 0xc9, 0xaa,
-	0x4e, 0xff, 0x0a, 0xfd, 0xe9, 0x68, 0xf2, 0x10, 0xaf, 0x53, 0x20, 0xd1, 0x21, 0xaf, 0x35, 0x20,
-	0x0a, 0xe5, 0x96, 0xb5, 0xe4, 0xa2, 0x57, 0xe9, 0x10, 0xcf, 0xab, 0x4b, 0x78, 0x66, 0xe3, 0xb7,
-	0x4b, 0xaf, 0x86, 0x7d, 0x8b, 0xad, 0x73, 0x6a, 0xf1, 0x5d, 0xc5, 0xed, 0x55, 0x6f, 0x55, 0x39,
-	0x95, 0x45, 0xc4, 0x25, 0x73, 0xe1, 0x16, 0x15, 0xda, 0x69, 0xc5, 0x6b, 0x00, 0x69, 0xbb, 0x4c,
-	0xc2, 0x76, 0xa5, 0xb6, 0xea, 0x7c, 0xe7, 0x3b, 0xe5, 0x24, 0xc4, 0xa5, 0x93, 0xe2, 0xfe, 0x8c,
-	0x4e, 0x73, 0x8a, 0xbc, 0x5f, 0x44, 0x4e, 0xe2, 0x0c, 0x93, 0x59, 0x38, 0xc3, 0x28, 0x07, 0xa3,
-	0xec, 0xfc, 0xc1, 0x28, 0x71, 0x0e, 0x59, 0x9b, 0x3b, 0x87, 0xcc, 0xef, 0xa1, 0xeb, 0xe7, 0xd8,
-	0x43, 0x37, 0x96, 0x9c, 0x03, 0x26, 0xe8, 0xa0, 0x81, 0x3f, 0xf6, 0xa4, 0xb9, 0x1e, 0xb1, 0x2c,
-	0x3c, 0xaf, 0x7c, 0x67, 0x39, 0xf0, 0xa7, 0x51, 0xe0, 0x8f, 0xc7, 0x5e, 0xc0, 0xf9, 0x2c, 0x4e,
-	0x0d, 0xc3, 0x1d, 0x7a, 0x53, 0x8f, 0x06, 0x24, 0x43, 0x64, 0xad, 0xcd, 0x18, 0xd8, 0x1c, 0x96,
-	0x7e, 0x87, 0x02, 0xa2, 0x1f, 0xbe, 0x9d, 0x0e, 0xc4, 0x8e, 0xfb, 0x11, 0xdb, 0x8e, 0x6b, 0x0b,
-	0xde, 0xe3, 0xa4, 0xa6, 0x8c, 0x28, 0x2d, 0x78, 0x97, 0xf3, 0x13, 0xa6, 0x29, 0x1f, 0x3d, 0x89,
-	0x6b, 0x39, 0x40, 0xb7, 0x0d, 0x70, 0x9b, 0x83, 0x39, 0x65, 0x99, 0x5d, 0x4a, 0xbc, 0xc5, 0xe6,
-	0xa4, 0x58, 0xdf, 0x5d, 0x04, 0x84, 0x85, 0x70, 0x7e, 0xd5, 0xe9, 0x35, 0xdb, 0xe6, 0xfb, 0x6a,
-	0xdb, 0x1f, 0xee, 0xcd, 0x86, 0x90, 0xa9, 0xb0, 0x5d, 0x8f, 0x6f, 0x45, 0xd2, 0x23, 0xfe, 0x91,
-	0x8f, 0x7c, 0x67, 0x47, 0xbb, 0xd5, 0xcd, 0xd5, 0x6f, 0xf5, 0x2c, 0x2c, 0x13, 0xda, 0xfe, 0x70,
-	0xc9, 0xe7, 0x95, 0x3d, 0x76, 0x91, 0x0f, 0xc6, 0x8b, 0x0f, 0x8b, 0xc7, 0xd1, 0x0f, 0x58, 0x41,
-	0xd9, 0xe9, 0x56, 0xf6, 0xbd, 0xd4, 0xdd, 0x90, 0x4d, 0xa4, 0x8c, 0xd2, 0x88, 0x5d, 0xac, 0x8f,
-	0xfd, 0x37, 0xbc, 0x73, 0xb5, 0x42, 0xff, 0x47, 0x2c, 0x27, 0x6e, 0x1b, 0x91, 0xfa, 0x37, 0x56,
-	0x5e, 0x47, 0xb2, 0x36, 0xe0, 0xd7, 0x72, 0xe5, 0x7f, 0xc4, 0xae, 0xc0, 0x50, 0xbc, 0x3a, 0x3e,
-	0x6b, 0xbc, 0x6f, 0xb3, 0xbc, 0xbc, 0xb7, 0xb2, 0xd2, 0x5e, 0x92, 0xc2, 0xc2, 0xa3, 0xc9, 0xf2,
-	0x21, 0xbf, 0xcb, 0xd6, 0x60, 0xc8, 0x50, 0x7f, 0xc0, 0xd6, 0x46, 0x91, 0x37, 0x11, 0xf6, 0xd9,
-	0x59, 0x3e, 0x01, 0x2a, 0x16, 0x38, 0x65, 0xe9, 0xfb, 0x6c, 0x9d, 0xdb, 0x3a, 0x84, 0x52, 0x43,
-	0x65, 0x5e, 0x65, 0x5c, 0x5e, 0xce, 0x08, 0xee, 0x27, 0x8c, 0xc9, 0xc9, 0x9e, 0x43, 0x82, 0x72,
-	0x18, 0x12, 0x12, 0x46, 0xac, 0x00, 0x12, 0xaa, 0x47, 0xfd, 0xe9, 0xa1, 0x17, 0xea, 0xdf, 0x60,
-	0xeb, 0x91, 0xef, 0xf6, 0x87, 0xe2, 0x4e, 0xa8, 0x9e, 0x90, 0xc1, 0x67, 0x69, 0xad, 0x45, 0xbe,
-	0x31, 0x1c, 0xea, 0xf7, 0x59, 0x3e, 0xf2, 0xc9, 0x79, 0xc9, 0x80, 0xcb, 0xa8, 0x73, 0x91, 0x8f,
-	0x8e, 0x0c, 0x65, 0xa4, 0x26, 0xb5, 0x15, 0x03, 0x7e, 0x3a, 0x37, 0xe0, 0xf5, 0x05, 0x11, 0x38,
-	0x39, 0x31, 0xea, 0xa3, 0xc5, 0x51, 0x57, 0xb2, 0xc8, 0xa1, 0x89, 0xeb, 0x98, 0x7b, 0x02, 0xf5,
-	0xc5, 0xcf, 0xe2, 0x42, 0x97, 0x29, 0xd9, 0x2c, 0xd7, 0xa3, 0xd0, 0x5e, 0xe6, 0x3e, 0x32, 0x19,
-	0xac, 0x74, 0x1f, 0x49, 0x61, 0xe5, 0x44, 0x8e, 0x28, 0x3d, 0x67, 0x79, 0x14, 0xda, 0x3d, 0x8e,
-	0x16, 0xa4, 0x7e, 0x97, 0xb1, 0xf8, 0xaa, 0x12, 0x89, 0xdd, 0x59, 0x25, 0xd6, 0x3f, 0x8e, 0x2c,
-	0x52, 0xa2, 0x7b, 0x1c, 0x95, 0xfe, 0x71, 0x8a, 0x15, 0xd0, 0xaa, 0xe6, 0x89, 0x37, 0x5d, 0x94,
-	0xfd, 0x97, 0x59, 0x41, 0x49, 0x4c, 0x2b, 0x0b, 0x5a, 0x85, 0xa6, 0x71, 0xc1, 0x62, 0x71, 0xce,
-	0x02, 0xf7, 0xe2, 0x59, 0x9e, 0x0c, 0xb8, 0x38, 0x5d, 0xf9, 0x9a, 0xbe, 0x71, 0xc1, 0x42, 0xd2,
-	0xca, 0x06, 0x5b, 0xf3, 0x40, 0x9b, 0xf2, 0x7f, 0x4d, 0xb1, 0x82, 0x14, 0x3f, 0xf5, 0x75, 0x8d,
-	0x6d, 0x76, 0xeb, 0xbd, 0x9e, 0xdb, 0xec, 0xec, 0x1b, 0xad, 0x66, 0x4d, 0xbb, 0xa0, 0x6b, 0x2c,
-	0xc7, 0x21, 0x6d, 0xe3, 0x85, 0xf6, 0xee, 0x67, 0xef, 0xdf, 0x6f, 0xe8, 0x57, 0x24, 0x8d, 0xdb,
-	0xeb, 0x5a, 0x8e, 0xf6, 0x3f, 0xde, 0x03, 0x54, 0x67, 0x8c, 0x43, 0x1d, 0xa3, 0xd2, 0x32, 0xb5,
-	0xff, 0xc9, 0x61, 0x97, 0x59, 0x81, 0xc3, 0x3a, 0x5d, 0xab, 0x6d, 0xb4, 0xb4, 0xbf, 0x48, 0x10,
-	0xd6, 0x5b, 0xdd, 0x6e, 0x4d, 0xfb, 0x5f, 0x1c, 0x26, 0x06, 0x31, 0x5a, 0x2d, 0xed, 0xa7, 0x1c,
-	0x72, 0x9d, 0x5d, 0xe4, 0x90, 0x6a, 0xb7, 0xe3, 0x58, 0xdd, 0x56, 0xcb, 0xb4, 0xb4, 0xff, 0x9d,
-	0x60, 0x6f, 0x75, 0xab, 0x46, 0x4b, 0xfb, 0x59, 0x92, 0xbd, 0xf3, 0x52, 0x7b, 0x0f, 0x90, 0xf2,
-	0x7f, 0x58, 0xc3, 0xf7, 0xd5, 0x7c, 0xdb, 0xdf, 0xe6, 0x2c, 0x8e, 0xdb, 0x30, 0x5b, 0xad, 0xae,
-	0x76, 0x41, 0x3e, 0x9b, 0x96, 0xd5, 0xb5, 0xb4, 0x94, 0x7e, 0x95, 0x5d, 0xc2, 0xe7, 0x6a, 0xa3,
-	0xeb, 0x5a, 0xe6, 0xb3, 0x3d, 0xd3, 0x76, 0xb4, 0xb4, 0x7e, 0x99, 0xab, 0x20, 0xc1, 0xbd, 0xd6,
-	0x4b, 0x2d, 0x13, 0xd3, 0xbe, 0xe8, 0x99, 0x56, 0xb3, 0x6d, 0x76, 0x1c, 0xd3, 0xd2, 0xb2, 0xfa,
-	0x0d, 0x76, 0x95, 0x83, 0xeb, 0xa6, 0xe1, 0xec, 0x59, 0xa6, 0x2d, 0xc5, 0xac, 0xe9, 0xd7, 0xd9,
-	0xe5, 0x79, 0x14, 0x88, 0x5a, 0xd7, 0x77, 0xd8, 0x75, 0x8e, 0xd8, 0x35, 0x1d, 0x98, 0x66, 0xbd,
-	0xb9, 0x2b, 0xb9, 0x36, 0xa4, 0xc0, 0x04, 0x12, 0xf8, 0x72, 0x52, 0x2f, 0x5b, 0xa2, 0xb4, 0xbc,
-	0xae, 0xb3, 0x6d, 0x0e, 0xec, 0x19, 0xd5, 0xa7, 0xa6, 0xe3, 0x36, 0x3b, 0x1a, 0x93, 0xba, 0xd6,
-	0x5b, 0xdd, 0xe7, 0xae, 0x65, 0xb6, 0xbb, 0xfb, 0x66, 0x4d, 0x2b, 0xe8, 0x57, 0x98, 0x86, 0xa4,
-	0x5d, 0xcb, 0x71, 0x6d, 0xc7, 0x70, 0xf6, 0x6c, 0x6d, 0x53, 0x4a, 0x25, 0x01, 0xdd, 0x3d, 0x47,
-	0xdb, 0xd2, 0x2f, 0xb1, 0xad, 0x58, 0x42, 0xbb, 0x5b, 0xd3, 0xb6, 0xe5, 0x40, 0xbb, 0x56, 0x77,
-	0xaf, 0xc7, 0x61, 0x17, 0x25, 0x19, 0x97, 0x08, 0x20, 0x4d, 0x92, 0x71, 0x77, 0xe0, 0xb0, 0x4b,
-	0xfa, 0x4d, 0x76, 0x8d, 0xc3, 0xda, 0x7b, 0x2d, 0xa7, 0xd9, 0x33, 0x2c, 0x47, 0xce, 0x57, 0xd7,
-	0x8b, 0xec, 0xca, 0x02, 0x0e, 0xa6, 0x7b, 0x59, 0x62, 0x2a, 0x86, 0x65, 0x35, 0x4d, 0x4b, 0xf2,
-	0x5c, 0xd1, 0xaf, 0x31, 0x7d, 0x0e, 0x03, 0x1c, 0x57, 0xf5, 0x7b, 0xec, 0x36, 0x87, 0x3f, 0xdb,
-	0x33, 0xf7, 0xcc, 0x65, 0xe6, 0xbd, 0xa6, 0xdf, 0x61, 0x3b, 0xab, 0x48, 0x40, 0xc6, 0x75, 0x69,
-	0x3b, 0xab, 0xdb, 0x32, 0x25, 0x5f, 0x51, 0x5a, 0x89, 0xc0, 0x40, 0x7b, 0x43, 0xce, 0x0b, 0xc4,
-	0x18, 0xf6, 0xcb, 0x4e, 0x55, 0x32, 0xdc, 0x94, 0xda, 0xab, 0x38, 0xe0, 0xda, 0x91, 0x16, 0xb2,
-	0x05, 0x46, 0xbb, 0x25, 0x61, 0x6d, 0xd3, 0x31, 0x2d, 0x6e, 0xb5, 0xdb, 0xe5, 0x2a, 0x5e, 0xf8,
-	0x98, 0xfb, 0x13, 0x09, 0x44, 0xda, 0xe0, 0x6b, 0x2d, 0x62, 0x15, 0x07, 0x03, 0xd8, 0xbe, 0x69,
-	0xd9, 0xcd, 0x6e, 0xa7, 0xd2, 0x74, 0xda, 0x46, 0x4f, 0x4b, 0x95, 0x3d, 0x2c, 0x9c, 0xa8, 0x08,
-	0xc7, 0xa6, 0x04, 0xfa, 0x41, 0xd5, 0xad, 0x5b, 0xc6, 0xae, 0x08, 0xd1, 0x0b, 0x24, 0x97, 0xa0,
-	0x35, 0xab, 0xdb, 0xd3, 0x52, 0x34, 0x6b, 0x82, 0x59, 0xa6, 0x61, 0xb7, 0xb5, 0x74, 0x92, 0xb0,
-	0x6d, 0xd8, 0x4f, 0xb5, 0x4c, 0xf9, 0x09, 0x0e, 0x83, 0x2f, 0x2d, 0xa8, 0x3e, 0x23, 0xe7, 0xa8,
-	0x2a, 0x7a, 0x92, 0x73, 0x57, 0xdd, 0x9a, 0xd9, 0xb3, 0xcc, 0xaa, 0xe1, 0x98, 0x35, 0x21, 0xe1,
-	0xd7, 0xf0, 0xfb, 0x6c, 0xbc, 0xc7, 0x4e, 0xac, 0xea, 0x14, 0xb7, 0x59, 0x1e, 0x41, 0x90, 0x8f,
-	0x7e, 0x96, 0x8a, 0x9f, 0x21, 0x75, 0xbc, 0x4f, 0x95, 0xff, 0x3d, 0x95, 0x88, 0x89, 0x96, 0x05,
-	0x66, 0x35, 0x55, 0x03, 0x39, 0x23, 0x70, 0x6c, 0x88, 0x01, 0x5b, 0x4b, 0x49, 0x83, 0xa0, 0xcf,
-	0x22, 0x34, 0x2d, 0x49, 0x65, 0xb8, 0xd8, 0x5a, 0x56, 0x92, 0x62, 0x14, 0x20, 0x34, 0x47, 0xfa,
-	0x56, 0xdd, 0x66, 0x8f, 0xac, 0x74, 0x57, 0x12, 0xa2, 0xa3, 0x21, 0xe1, 0x13, 0xfd, 0x1a, 0xf7,
-	0x2e, 0x92, 0x59, 0x69, 0x75, 0xab, 0x4f, 0xcd, 0x9a, 0xf6, 0x2e, 0x5d, 0x3e, 0x51, 0x3e, 0xcf,
-	0x4f, 0x98, 0x6f, 0x89, 0xf2, 0x82, 0xbd, 0xd6, 0x7d, 0xde, 0xd1, 0x52, 0x31, 0x5d, 0x07, 0x92,
-	0x55, 0x75, 0x5f, 0xcb, 0x8a, 0x64, 0xce, 0x41, 0xf5, 0xe7, 0x35, 0xed, 0x2e, 0x45, 0x0c, 0x42,
-	0xe2, 0x4c, 0xf1, 0xa4, 0xfc, 0x57, 0xe6, 0x5e, 0xd7, 0x08, 0xd3, 0xf7, 0xec, 0xc5, 0x61, 0x6d,
-	0xb7, 0xd5, 0xec, 0x3c, 0x9d, 0x1b, 0xd6, 0x96, 0xb3, 0x48, 0x53, 0x7a, 0xe5, 0x74, 0xfb, 0xa6,
-	0x96, 0x2d, 0xff, 0x69, 0x1a, 0x3f, 0x8a, 0xe1, 0xd2, 0x65, 0x9b, 0x8a, 0x18, 0xeb, 0xca, 0x00,
-	0x12, 0xf4, 0xe0, 0xb3, 0x76, 0xc5, 0x6d, 0xd4, 0x62, 0xf1, 0x04, 0xaa, 0xd7, 0xa4, 0xdf, 0x71,
-	0x10, 0x91, 0x65, 0xe7, 0x61, 0xf5, 0x9a, 0x96, 0x13, 0xb3, 0xaf, 0xbb, 0x0f, 0x76, 0x39, 0x95,
-	0x96, 0x84, 0xd4, 0xc1, 0x1e, 0x8a, 0x78, 0x04, 0x3d, 0xd1, 0x75, 0x01, 0x7a, 0x44, 0xa0, 0x77,
-	0xe0, 0xff, 0xb1, 0x78, 0x02, 0xa6, 0xf5, 0x4b, 0x52, 0x9a, 0x83, 0x20, 0x30, 0x78, 0x01, 0x41,
-	0x5d, 0xa7, 0x61, 0x5a, 0xda, 0xbb, 0x5c, 0x4c, 0x54, 0xed, 0xf6, 0x7a, 0x00, 0xd2, 0x62, 0xa2,
-	0x7a, 0xb3, 0x02, 0x90, 0xbb, 0xf1, 0x90, 0xc6, 0x9e, 0xd3, 0xed, 0x98, 0xbb, 0xda, 0xbb, 0x27,
-	0xfa, 0x25, 0x41, 0xd5, 0x33, 0xf6, 0x6c, 0x53, 0x7b, 0xf7, 0x2e, 0xa5, 0x5f, 0xe3, 0xae, 0x24,
-	0x40, 0x90, 0x33, 0xda, 0xda, 0xbb, 0x77, 0xe9, 0x72, 0x4d, 0x71, 0x1a, 0xba, 0x6a, 0xbb, 0xc5,
-	0xa3, 0xa2, 0x67, 0xb9, 0x46, 0x0d, 0xf7, 0xf0, 0x4d, 0x7c, 0xac, 0x99, 0x2d, 0xd3, 0x31, 0xb5,
-	0x54, 0x0c, 0x69, 0x77, 0x6b, 0xcd, 0xfa, 0x4b, 0x2d, 0x5d, 0xfe, 0x1c, 0x5d, 0x20, 0xfe, 0x93,
-	0x07, 0x64, 0xd4, 0x36, 0x77, 0xfa, 0x4e, 0xcd, 0xb0, 0x40, 0x12, 0x0a, 0x6e, 0x3b, 0x6e, 0xf7,
-	0x45, 0x5b, 0x4b, 0x95, 0x5f, 0xc7, 0x7f, 0xd3, 0x80, 0xff, 0x91, 0x02, 0x92, 0xfb, 0xa2, 0x5d,
-	0x75, 0x3b, 0x2f, 0xda, 0xee, 0x67, 0x72, 0x6c, 0x01, 0x79, 0xa0, 0xa5, 0xf4, 0x1d, 0x1e, 0xfd,
-	0x00, 0xe9, 0xf6, 0xcc, 0x0e, 0x8f, 0xc0, 0x8a, 0x61, 0x37, 0xab, 0x30, 0x19, 0xfd, 0x06, 0xdf,
-	0x2d, 0x01, 0x99, 0xd8, 0x61, 0xdf, 0xbf, 0xcf, 0x94, 0xff, 0x41, 0x8e, 0x5d, 0x5e, 0xf2, 0x67,
-	0x02, 0xc8, 0xa9, 0x5f, 0x80, 0x52, 0xf5, 0x8a, 0xac, 0x4a, 0x2e, 0x50, 0x5a, 0x56, 0xe1, 0x8d,
-	0x97, 0x88, 0x4b, 0xd1, 0xa6, 0x2c, 0x70, 0x6d, 0xd3, 0x31, 0x6a, 0x86, 0x63, 0x68, 0xe9, 0x39,
-	0x61, 0xa6, 0xd3, 0x70, 0x6b, 0xb6, 0xa3, 0x65, 0x96, 0xc0, 0x6d, 0xab, 0xaa, 0x65, 0xe7, 0x04,
-	0x01, 0xdc, 0x79, 0xd9, 0x33, 0xe5, 0xb6, 0x2f, 0x10, 0xfb, 0x2d, 0xa3, 0xe3, 0xee, 0x37, 0x6b,
-	0xda, 0xfa, 0x32, 0x44, 0xaf, 0xda, 0xd3, 0x36, 0xe6, 0xe7, 0xd1, 0x73, 0x6b, 0x76, 0xb5, 0xa7,
-	0xe5, 0x68, 0x2b, 0x52, 0xe0, 0x66, 0xb5, 0xa3, 0xe5, 0xe7, 0xe4, 0x34, 0x7b, 0x6e, 0xcf, 0xea,
-	0x3a, 0x5d, 0x8d, 0x2d, 0x20, 0xf6, 0x1f, 0x71, 0x5d, 0x0b, 0xcb, 0x10, 0x30, 0xb9, 0xcd, 0xb9,
-	0x91, 0x9d, 0x6a, 0x8f, 0x33, 0x6c, 0x2d, 0x81, 0x03, 0xfd, 0xf6, 0x1c, 0x7c, 0xaf, 0x86, 0xf4,
-	0x17, 0x97, 0xc0, 0x81, 0x5e, 0x9b, 0x1b, 0xd8, 0xae, 0x3a, 0xc8, 0x70, 0x69, 0x19, 0xa2, 0xc6,
-	0xcb, 0x81, 0xb9, 0xb5, 0xab, 0xb6, 0x41, 0x59, 0x6e, 0xd9, 0xcb, 0xcb, 0x71, 0xd5, 0x6e, 0xcd,
-	0xd4, 0xae, 0xcc, 0xd9, 0xca, 0xb0, 0x7a, 0x6e, 0xb7, 0xa7, 0x5d, 0x9d, 0x53, 0x0c, 0xc0, 0x76,
-	0xcf, 0xd0, 0xae, 0x2d, 0x81, 0x3b, 0x3d, 0x43, 0xbb, 0xbe, 0x8c, 0xbe, 0x61, 0x68, 0xc5, 0x65,
-	0xf4, 0x0d, 0x43, 0xbb, 0xb1, 0x68, 0xd9, 0xc7, 0x7c, 0x82, 0x37, 0x97, 0x21, 0x60, 0x82, 0x3b,
-	0xf3, 0x93, 0x00, 0x44, 0xbd, 0x65, 0x54, 0xcc, 0x96, 0x76, 0x6b, 0xd9, 0x04, 0x1f, 0xe3, 0xe4,
-	0x6f, 0x2f, 0xc7, 0xf1, 0xc9, 0x7f, 0xa0, 0xdf, 0x66, 0x37, 0xe6, 0x65, 0x76, 0x6a, 0xae, 0x63,
-	0x58, 0xbb, 0xa6, 0xa3, 0xdd, 0x59, 0x36, 0x64, 0xa7, 0xe6, 0xda, 0xad, 0x96, 0x76, 0x77, 0x05,
-	0xce, 0x69, 0xb5, 0xb4, 0x7b, 0xb4, 0x5b, 0xcb, 0x58, 0xe9, 0xb5, 0x6c, 0x17, 0x35, 0x2d, 0xcd,
-	0xd9, 0x83, 0xa3, 0x9c, 0xaa, 0xf6, 0xe1, 0x7c, 0x78, 0x01, 0xbc, 0xd2, 0xb5, 0xb5, 0x8f, 0xe6,
-	0x10, 0xbd, 0x4a, 0xc5, 0x6d, 0xda, 0xcd, 0x9a, 0xf6, 0x31, 0x95, 0x2e, 0xd2, 0xd5, 0xf6, 0x3a,
-	0x1d, 0xb3, 0xe5, 0x36, 0x6b, 0xda, 0xd7, 0x96, 0xa9, 0x66, 0xbe, 0x70, 0x1a, 0x35, 0x4b, 0xfb,
-	0x7a, 0xf9, 0x73, 0x3c, 0xbd, 0xf0, 0x8f, 0xda, 0x47, 0x43, 0xfd, 0x22, 0x4f, 0x9a, 0xfb, 0xcd,
-	0x9a, 0xdb, 0xe9, 0x76, 0x4c, 0xbe, 0x65, 0x6d, 0x13, 0xa0, 0x67, 0x99, 0xb6, 0xd9, 0x71, 0xb4,
-	0x77, 0x77, 0xcb, 0xff, 0x31, 0x85, 0x2d, 0xc3, 0xd1, 0xec, 0xe4, 0x31, 0x7d, 0x84, 0x2d, 0x2e,
-	0xbe, 0x02, 0x75, 0xd3, 0x6c, 0x2c, 0xec, 0x49, 0x00, 0x03, 0x91, 0x2f, 0x20, 0x77, 0xe0, 0xfe,
-	0x06, 0x20, 0xd3, 0xee, 0x69, 0x69, 0x1a, 0x15, 0x9e, 0x8d, 0x3d, 0xa7, 0xa1, 0x65, 0x15, 0x40,
-	0x0d, 0x8a, 0xc0, 0x9c, 0x02, 0x80, 0x62, 0x49, 0xd3, 0x14, 0xa9, 0x56, 0x77, 0x0f, 0xf2, 0xdb,
-	0x5d, 0x45, 0x6a, 0xa3, 0xdb, 0xd3, 0x9e, 0xd0, 0xce, 0x01, 0xcf, 0x7b, 0x1d, 0xcb, 0xec, 0xc1,
-	0x36, 0xa4, 0x82, 0x6c, 0xf3, 0x19, 0x14, 0x0c, 0x3f, 0x4d, 0x27, 0xbe, 0x82, 0xa5, 0x3f, 0xcc,
-	0x05, 0x64, 0x06, 0xaf, 0xe1, 0x7b, 0x7b, 0x90, 0x09, 0x71, 0x99, 0x0c, 0x28, 0x72, 0x7b, 0x2f,
-	0x5d, 0xc7, 0x69, 0xf1, 0xf2, 0xbe, 0x40, 0xd1, 0xa2, 0xc2, 0x9b, 0x1d, 0x99, 0x0e, 0x0c, 0x2c,
-	0x4d, 0x71, 0x51, 0x9d, 0x96, 0x0c, 0x6f, 0xc3, 0x71, 0x6b, 0x66, 0x35, 0x86, 0x6b, 0x54, 0x18,
-	0x18, 0x8e, 0xdb, 0xdb, 0xb3, 0x1b, 0x3c, 0xa3, 0x69, 0x97, 0xc8, 0x98, 0x00, 0xec, 0xf6, 0x10,
-	0xa6, 0xcf, 0x11, 0x82, 0x04, 0xed, 0x72, 0x92, 0x90, 0xc3, 0xae, 0xc4, 0x84, 0xa0, 0x01, 0x2f,
-	0x9d, 0xb4, 0xab, 0x64, 0x45, 0x83, 0x8e, 0x1e, 0xda, 0x35, 0xaa, 0xad, 0x88, 0xaa, 0xf3, 0x9c,
-	0x6b, 0x73, 0x3d, 0x86, 0x82, 0x96, 0x04, 0x2d, 0x26, 0x25, 0xd6, 0x9b, 0x66, 0xab, 0xa6, 0xdd,
-	0x50, 0x86, 0x06, 0x7d, 0x7a, 0x95, 0x8a, 0x76, 0x93, 0x96, 0x86, 0xd4, 0x01, 0xd0, 0x8e, 0x5e,
-	0x14, 0xf3, 0x5e, 0xd8, 0x92, 0xf6, 0xf1, 0x6a, 0x8c, 0xd2, 0xd2, 0xa4, 0xaf, 0x9b, 0x45, 0x75,
-	0xdc, 0x6e, 0x25, 0x8e, 0xd2, 0x8c, 0x60, 0x50, 0xbc, 0xfe, 0xb7, 0xf7, 0x19, 0xda, 0xd2, 0x01,
-	0xd2, 0xe9, 0xba, 0x95, 0xbd, 0x7a, 0x9d, 0xe4, 0xfe, 0x17, 0xe1, 0xa2, 0xca, 0x17, 0x8c, 0x7c,
-	0x6d, 0xc9, 0x71, 0xd4, 0x8a, 0x18, 0xe7, 0xdb, 0x74, 0xdc, 0xdd, 0xae, 0xd3, 0xa5, 0xe3, 0x77,
-	0x8a, 0xe2, 0xa9, 0xe9, 0xb8, 0xcf, 0xad, 0xa6, 0x63, 0xaa, 0x3b, 0x1c, 0x86, 0xa0, 0xc4, 0x18,
-	0x55, 0xa7, 0xd9, 0xed, 0xd8, 0x5a, 0x26, 0x46, 0x18, 0xbd, 0x5e, 0xeb, 0xa5, 0x44, 0x64, 0x63,
-	0x44, 0xb5, 0x65, 0x1a, 0x96, 0x44, 0xac, 0x09, 0xbf, 0xa6, 0xf3, 0x8a, 0xb6, 0x4e, 0x96, 0x6a,
-	0x2e, 0xb1, 0xd4, 0x5f, 0xc7, 0x09, 0xcd, 0x7f, 0xb9, 0x48, 0x05, 0x45, 0xbd, 0x9a, 0xa8, 0x54,
-	0xea, 0x55, 0x51, 0x97, 0x88, 0x9d, 0x5a, 0x42, 0x5c, 0xdb, 0xb1, 0x9a, 0x55, 0x38, 0x9e, 0x4b,
-	0x52, 0x2a, 0x6a, 0x32, 0x31, 0x29, 0x42, 0x04, 0x69, 0xb6, 0xfc, 0xcf, 0xe8, 0x4d, 0xa9, 0x1c,
-	0x1d, 0xe3, 0x1d, 0x8d, 0x59, 0x57, 0x4b, 0x50, 0x12, 0x51, 0x77, 0x6d, 0xb3, 0x53, 0x93, 0x07,
-	0xe7, 0x58, 0x8d, 0xba, 0x5b, 0x6d, 0x98, 0xd5, 0xa7, 0x6e, 0x77, 0xdf, 0xb4, 0x5a, 0x46, 0x4f,
-	0x16, 0x0c, 0xf5, 0xba, 0x0b, 0x09, 0x06, 0x22, 0x69, 0xaf, 0xe3, 0xc4, 0x46, 0xab, 0xd7, 0x79,
-	0xa9, 0xfd, 0x54, 0x22, 0x72, 0x09, 0x44, 0xe5, 0xa5, 0x44, 0x68, 0x65, 0x1b, 0x8f, 0x3e, 0xf8,
-	0x8d, 0x39, 0xce, 0x6e, 0x77, 0xa1, 0x11, 0xb3, 0xab, 0x34, 0x62, 0x04, 0x24, 0xee, 0x9a, 0x48,
-	0x88, 0x6c, 0x84, 0x7c, 0x81, 0xef, 0x03, 0x17, 0xbe, 0x05, 0x24, 0xc3, 0xef, 0x26, 0x0d, 0xbf,
-	0xab, 0x18, 0x5e, 0x42, 0xc8, 0xbe, 0xe9, 0xb2, 0xad, 0x5e, 0x26, 0xe1, 0xee, 0x48, 0x42, 0xf0,
-	0xf4, 0x25, 0x85, 0x40, 0x90, 0xb5, 0xcc, 0x2a, 0xe4, 0x4a, 0x0c, 0x83, 0x5d, 0xf0, 0xd7, 0x5a,
-	0xd3, 0x32, 0xf9, 0xc2, 0x6d, 0xa2, 0x92, 0x8e, 0x5b, 0xaf, 0x6b, 0x99, 0x72, 0x0f, 0x1d, 0x63,
-	0xfe, 0x8b, 0x39, 0x5a, 0x1c, 0x0b, 0xac, 0xd4, 0x36, 0x9c, 0x6a, 0x43, 0xbb, 0x40, 0xee, 0x26,
-	0x1c, 0x50, 0x1e, 0xd8, 0x2c, 0x61, 0x24, 0x1e, 0xea, 0xe9, 0xf2, 0xdf, 0x4b, 0xe1, 0xbb, 0x9c,
-	0x25, 0xdf, 0xa2, 0xd1, 0x6a, 0x59, 0x96, 0xdb, 0xac, 0xb5, 0x4c, 0xd7, 0x69, 0xb6, 0xcd, 0xae,
-	0x92, 0x21, 0x2d, 0xcb, 0x6d, 0x18, 0x56, 0x4d, 0xc2, 0x85, 0x11, 0x2c, 0x59, 0x39, 0xa7, 0x63,
-	0x4a, 0x3c, 0xfa, 0x49, 0xe7, 0x93, 0x70, 0x3c, 0xbb, 0x13, 0x3c, 0x5b, 0x9e, 0xd2, 0x1f, 0x1c,
-	0xe3, 0xaf, 0xdf, 0xa9, 0x7c, 0x76, 0x7f, 0x68, 0x5a, 0x5d, 0xb9, 0xa4, 0x6d, 0x5c, 0xd2, 0x77,
-	0x3f, 0x7d, 0xbf, 0xa1, 0x5f, 0xe5, 0xb3, 0x6e, 0xbb, 0x76, 0xab, 0xfb, 0xbc, 0x67, 0x38, 0x0d,
-	0x6a, 0x7a, 0x61, 0x37, 0xac, 0xad, 0x76, 0xc3, 0xd4, 0xce, 0x57, 0x1b, 0x4f, 0xbf, 0x7c, 0xc1,
-	0x27, 0x0b, 0x5f, 0x3b, 0xa9, 0xc5, 0x7c, 0x45, 0xcd, 0x1c, 0x68, 0x4f, 0x80, 0xd1, 0x39, 0x1f,
-	0xe7, 0xc0, 0x01, 0x76, 0x15, 0xce, 0xb0, 0x6d, 0xc3, 0x7a, 0xaa, 0x89, 0xa2, 0x1c, 0xe0, 0x0b,
-	0x71, 0xfd, 0x85, 0xfa, 0xe9, 0xda, 0xa2, 0x7f, 0xb5, 0x93, 0xfe, 0xd5, 0x5e, 0xf0, 0xaf, 0xb6,
-	0xe2, 0x5f, 0x87, 0xea, 0xfd, 0x00, 0x35, 0x44, 0xdb, 0xf5, 0x44, 0x07, 0x80, 0x21, 0xe8, 0x69,
-	0xa5, 0x07, 0xa7, 0x76, 0x9a, 0x45, 0x1d, 0xa2, 0xac, 0x67, 0xcb, 0xfd, 0xb8, 0x5d, 0x77, 0x2b,
-	0x7b, 0x96, 0xed, 0xc8, 0xfd, 0xb8, 0x5d, 0x17, 0xe7, 0xf4, 0xf2, 0x3f, 0xa7, 0xeb, 0x89, 0xd8,
-	0x19, 0xe5, 0xf6, 0xc1, 0xa9, 0x9b, 0xd4, 0x24, 0x74, 0xeb, 0x46, 0xb3, 0x65, 0xc2, 0x68, 0xb8,
-	0x45, 0x9a, 0x8e, 0x5b, 0x31, 0x6a, 0xb2, 0xad, 0x23, 0x3c, 0x8f, 0xc0, 0xe4, 0x8f, 0x69, 0xaa,
-	0x94, 0x08, 0xda, 0xec, 0xd8, 0x8e, 0xb5, 0x87, 0xa8, 0x0c, 0xed, 0x3f, 0x84, 0x42, 0x87, 0xce,
-	0xc6, 0xf4, 0xa2, 0xbf, 0x26, 0xc6, 0x5d, 0xa3, 0xaa, 0xc7, 0x54, 0xfa, 0x6c, 0x02, 0xb7, 0x1e,
-	0xb3, 0x89, 0x7e, 0x9b, 0x40, 0x6d, 0xc4, 0x6c, 0xb2, 0xef, 0x26, 0x70, 0xb9, 0x98, 0x0d, 0x7b,
-	0x11, 0xdd, 0x9e, 0x40, 0xe5, 0xf5, 0x0f, 0xd8, 0x4d, 0x44, 0xd9, 0xcf, 0x9b, 0x4e, 0xb5, 0x21,
-	0x9a, 0x61, 0x84, 0x67, 0x54, 0x59, 0x9a, 0xc9, 0x76, 0x98, 0x40, 0x17, 0xe2, 0x51, 0x65, 0xdf,
-	0x4a, 0xe0, 0x36, 0xa9, 0xd3, 0x26, 0x35, 0x92, 0x5d, 0x50, 0x22, 0xd8, 0xa2, 0x3d, 0xc3, 0x5c,
-	0xe2, 0x5b, 0x15, 0xf5, 0xcf, 0x89, 0xbe, 0xea, 0x8f, 0xc6, 0xfc, 0x9a, 0x2a, 0xff, 0x6b, 0x5c,
-	0xe0, 0x8f, 0x8d, 0x7a, 0xd5, 0x6d, 0x76, 0xaa, 0xdd, 0x76, 0xcf, 0x70, 0x9a, 0xb0, 0xeb, 0x09,
-	0x2f, 0x03, 0x84, 0xd9, 0x33, 0x2d, 0x38, 0xa1, 0xfe, 0x79, 0x1a, 0xf3, 0xcb, 0x41, 0x7f, 0x28,
-	0xde, 0x50, 0xa2, 0x0c, 0x5c, 0xf0, 0x8a, 0x55, 0xe5, 0x2b, 0x42, 0xfd, 0x32, 0xd9, 0xe5, 0x10,
-	0x70, 0x5e, 0x75, 0x8b, 0xdd, 0x54, 0x00, 0x65, 0x8f, 0x52, 0x4b, 0x53, 0x13, 0x57, 0x60, 0x12,
-	0x53, 0x10, 0x1b, 0x92, 0x82, 0x44, 0x79, 0xa2, 0x33, 0x03, 0x08, 0xd4, 0x73, 0x8d, 0xe2, 0x53,
-	0x90, 0xb6, 0xcc, 0x8e, 0x3c, 0x29, 0x72, 0x18, 0x2f, 0x0d, 0x5c, 0xb3, 0xdd, 0x73, 0x5e, 0xca,
-	0xe6, 0xb0, 0x82, 0xd8, 0xeb, 0x3c, 0xed, 0x74, 0x9f, 0x77, 0xe4, 0xee, 0x22, 0xd5, 0xe7, 0x36,
-	0x6f, 0xc2, 0x12, 0xc7, 0xf3, 0x6a, 0xda, 0xae, 0xdd, 0x32, 0xf6, 0x4d, 0x8d, 0xcd, 0x4d, 0x96,
-	0x9f, 0x8d, 0x45, 0x55, 0x28, 0x81, 0xbc, 0x4d, 0xa4, 0x6d, 0xea, 0x1f, 0xb1, 0xbb, 0x04, 0x8e,
-	0x7b, 0xb4, 0x34, 0x3c, 0xec, 0x86, 0xe0, 0xc2, 0xda, 0x56, 0xf9, 0xf7, 0x33, 0x98, 0x7f, 0xc0,
-	0xde, 0x54, 0x94, 0x72, 0x73, 0xd3, 0x48, 0x86, 0x62, 0x56, 0xd1, 0x6b, 0x14, 0x40, 0x98, 0x74,
-	0x4a, 0x18, 0xd4, 0x58, 0x62, 0x50, 0x51, 0xbb, 0x28, 0x48, 0x94, 0x94, 0x99, 0x43, 0x74, 0xf7,
-	0x30, 0x36, 0xe4, 0x36, 0x2c, 0x10, 0x86, 0xb5, 0xbb, 0x07, 0xc2, 0xb4, 0x35, 0xb1, 0x04, 0x86,
-	0x58, 0x82, 0x75, 0x45, 0x45, 0xa7, 0x0b, 0x9b, 0x4e, 0x07, 0x4c, 0x8d, 0x81, 0x2e, 0xf8, 0xb1,
-	0x14, 0xcd, 0x09, 0x7f, 0x50, 0x86, 0xc3, 0x9a, 0x34, 0x4f, 0x91, 0x02, 0x18, 0x1e, 0xe4, 0xdc,
-	0x41, 0x3b, 0x76, 0xd3, 0x76, 0x60, 0x54, 0xa6, 0xdf, 0x62, 0x45, 0x42, 0xef, 0x75, 0xec, 0xbd,
-	0x1e, 0x28, 0x69, 0xd6, 0xdc, 0xae, 0x55, 0x33, 0x2d, 0xad, 0x30, 0x67, 0x0f, 0xc7, 0xd8, 0xd5,
-	0x36, 0xe7, 0x26, 0x00, 0x25, 0x06, 0x9f, 0xb2, 0x38, 0x9c, 0xab, 0x08, 0x30, 0xe0, 0xf6, 0x9c,
-	0x01, 0x79, 0x77, 0x59, 0xcc, 0xfa, 0x62, 0xf9, 0x2f, 0x52, 0xac, 0x28, 0x96, 0x47, 0x2d, 0x2e,
-	0x95, 0xb0, 0xaa, 0x34, 0xab, 0xc2, 0x9f, 0x78, 0x0e, 0x93, 0x49, 0x10, 0x11, 0xf6, 0x5e, 0x0f,
-	0xc1, 0x29, 0x85, 0x3e, 0xe1, 0x6b, 0x22, 0x0f, 0xc6, 0xf4, 0xb2, 0xfa, 0xcc, 0x50, 0xa6, 0x59,
-	0x44, 0x61, 0xff, 0x37, 0x2b, 0xb4, 0x6f, 0x2e, 0x59, 0xfe, 0xb5, 0xb9, 0x01, 0xe5, 0xf2, 0xaf,
-	0x0b, 0xc3, 0x35, 0x63, 0x47, 0xda, 0x10, 0x0b, 0xdc, 0x14, 0x0b, 0x9c, 0x2b, 0xff, 0x0b, 0xfa,
-	0xf0, 0x00, 0x26, 0x8f, 0x7d, 0x2e, 0xd5, 0x35, 0xdb, 0xcb, 0x5c, 0xb3, 0xad, 0xba, 0x66, 0x12,
-	0x06, 0xcb, 0x23, 0xe3, 0x9f, 0x60, 0xb5, 0x16, 0x6c, 0x77, 0x16, 0x35, 0xb3, 0xe7, 0x90, 0x9d,
-	0xe7, 0x0a, 0x32, 0x2b, 0x7c, 0x88, 0x90, 0xcf, 0x9b, 0xad, 0x5a, 0xd5, 0xb0, 0x6a, 0x50, 0x56,
-	0x93, 0xcf, 0x11, 0x06, 0x0f, 0x2b, 0xeb, 0x73, 0xd0, 0x7d, 0xa3, 0xb5, 0x67, 0x6a, 0x1b, 0x73,
-	0xca, 0x73, 0xd1, 0xa2, 0x63, 0x24, 0x80, 0x3d, 0xcb, 0xb4, 0xcc, 0x67, 0x5a, 0x5e, 0x91, 0x50,
-	0xdb, 0xeb, 0x91, 0x5c, 0x26, 0xec, 0xd4, 0x16, 0x76, 0x2a, 0x94, 0xff, 0x88, 0x9c, 0x24, 0x2e,
-	0x97, 0x95, 0xdc, 0x8b, 0x03, 0xd6, 0xdb, 0x75, 0xe9, 0x25, 0xb2, 0x7c, 0xe2, 0x40, 0x4a, 0xf3,
-	0x7b, 0xad, 0x96, 0xcc, 0x9b, 0x1c, 0x3e, 0xe7, 0x22, 0x8a, 0x18, 0x51, 0x4b, 0x67, 0x44, 0x41,
-	0xde, 0x96, 0xf9, 0x5b, 0x96, 0xd1, 0x52, 0x02, 0x55, 0x66, 0x6b, 0xf3, 0x88, 0x6a, 0xb7, 0xdd,
-	0x36, 0x3a, 0x60, 0x27, 0x9c, 0xbc, 0x44, 0xd4, 0x5b, 0xc6, 0xae, 0xad, 0x6d, 0x94, 0xff, 0x20,
-	0x83, 0x5f, 0xae, 0xc5, 0x95, 0xb0, 0x3a, 0x2b, 0x54, 0x74, 0x17, 0x98, 0x70, 0xc3, 0x35, 0x5f,
-	0x34, 0x6d, 0xc7, 0x96, 0xef, 0x2a, 0x38, 0x46, 0x94, 0x99, 0x18, 0xeb, 0x29, 0xf2, 0x65, 0x8e,
-	0x7a, 0x6e, 0x36, 0x77, 0x1b, 0x8e, 0x1a, 0xd4, 0x32, 0x0c, 0x38, 0x1e, 0x52, 0x44, 0xb7, 0x8e,
-	0x9c, 0x70, 0xd6, 0xc2, 0x1d, 0x53, 0x45, 0x55, 0xf6, 0x20, 0xcf, 0xc2, 0xc9, 0xe1, 0x2e, 0xbb,
-	0x25, 0x70, 0xd5, 0x86, 0xd1, 0xec, 0x34, 0x3b, 0xbb, 0x09, 0xc1, 0x6b, 0x94, 0x64, 0x70, 0x60,
-	0x9e, 0x65, 0x54, 0xf4, 0xba, 0x28, 0xc3, 0x01, 0xdd, 0xea, 0x76, 0x7b, 0x72, 0xc3, 0xd8, 0x55,
-	0x16, 0x8d, 0x26, 0x91, 0x53, 0x51, 0x7c, 0x34, 0xb3, 0x26, 0x73, 0x19, 0xfa, 0xcb, 0xae, 0xb4,
-	0x3d, 0x44, 0x86, 0x68, 0x2f, 0xee, 0xce, 0x1b, 0xbe, 0x40, 0x4e, 0x20, 0x11, 0x38, 0x21, 0x6d,
-	0x93, 0x16, 0x44, 0xc2, 0xb9, 0xc6, 0xf2, 0xdd, 0xe2, 0x6e, 0xbc, 0xd8, 0xdb, 0xe5, 0xdf, 0x25,
-	0xc7, 0x13, 0x7f, 0x29, 0x38, 0xb1, 0x44, 0xa8, 0x4d, 0x4f, 0x88, 0xa1, 0x26, 0x2f, 0x6a, 0x23,
-	0xa1, 0x0d, 0x8c, 0x31, 0x59, 0xcb, 0xf6, 0x62, 0x35, 0xf9, 0x8b, 0x52, 0xb1, 0x28, 0x12, 0x6e,
-	0xd4, 0xf6, 0x4d, 0xcb, 0x69, 0xda, 0xa6, 0x74, 0xbf, 0x9e, 0xe2, 0x7e, 0xe5, 0xbf, 0x8a, 0x4e,
-	0x23, 0xff, 0x9c, 0x76, 0x42, 0x23, 0x7a, 0x47, 0x98, 0xf0, 0x6e, 0x19, 0x0c, 0xce, 0xdc, 0xc8,
-	0xe2, 0x5d, 0x86, 0x13, 0x8b, 0x4f, 0x97, 0x7f, 0x88, 0xf3, 0xc5, 0xdb, 0x5f, 0xfe, 0x6c, 0xc9,
-	0x7c, 0x9f, 0x75, 0x93, 0xf3, 0xc5, 0x31, 0x25, 0x14, 0x37, 0x24, 0x21, 0x9b, 0x83, 0x85, 0xec,
-	0xbf, 0xc6, 0x6e, 0x2f, 0xfc, 0x61, 0xf1, 0x25, 0xea, 0xdb, 0xd5, 0x44, 0xa0, 0x88, 0x02, 0x48,
-	0x82, 0x31, 0xf5, 0xa1, 0x7c, 0x0e, 0x8c, 0x75, 0xbf, 0x35, 0x7f, 0xf7, 0x2b, 0x21, 0x9e, 0x0e,
-	0x70, 0x56, 0xbd, 0x0a, 0x75, 0x37, 0xb7, 0x8c, 0x02, 0xe2, 0x1e, 0x1b, 0x1f, 0xe1, 0x2c, 0x1a,
-	0x0d, 0xea, 0x4b, 0x2d, 0x5d, 0xfe, 0xc3, 0x34, 0xda, 0x3d, 0x3e, 0x56, 0x2c, 0xa6, 0xa0, 0x76,
-	0x32, 0x05, 0x61, 0x04, 0x73, 0x20, 0x56, 0xa1, 0x14, 0xc1, 0x29, 0x5a, 0xf1, 0xb6, 0x1a, 0xc1,
-	0xd8, 0xaf, 0x48, 0xab, 0x28, 0x11, 0x17, 0x88, 0x12, 0x15, 0x45, 0x7b, 0xde, 0xcd, 0xb3, 0x64,
-	0xb6, 0x76, 0x32, 0xbf, 0x88, 0xa4, 0x2d, 0xc1, 0x96, 0xe1, 0x98, 0x32, 0x19, 0xb5, 0xe3, 0x98,
-	0xb0, 0xf8, 0xdb, 0xfd, 0x39, 0xe2, 0x0a, 0x48, 0xce, 0xd1, 0x76, 0x91, 0x80, 0xba, 0x35, 0xd3,
-	0x31, 0x9a, 0x2d, 0x2d, 0xaf, 0xaa, 0x4a, 0x19, 0x83, 0x6b, 0x6a, 0x6b, 0x4c, 0x9d, 0xba, 0x48,
-	0x26, 0x46, 0xa7, 0x66, 0x6b, 0x85, 0xf2, 0xbf, 0x4c, 0x2d, 0xf9, 0x16, 0x31, 0x5c, 0xe6, 0xc4,
-	0xf5, 0x39, 0x27, 0xa6, 0xf7, 0xd6, 0x02, 0x2c, 0x77, 0x70, 0xb1, 0x62, 0x31, 0x03, 0x64, 0x05,
-	0x79, 0x59, 0xa2, 0xae, 0x78, 0x4d, 0x66, 0x5e, 0x88, 0xac, 0x43, 0xb2, 0x22, 0x16, 0xea, 0xd2,
-	0x9f, 0xd6, 0xca, 0xff, 0x99, 0x76, 0xe7, 0xe4, 0x5f, 0x2a, 0x10, 0xc7, 0x3d, 0x38, 0x69, 0xdb,
-	0xd5, 0xf8, 0xf8, 0xc7, 0xef, 0x8f, 0x3c, 0x97, 0xef, 0xa6, 0xdb, 0x3d, 0xd7, 0xd8, 0xdd, 0xb5,
-	0xcc, 0x5d, 0x83, 0x1f, 0xd2, 0xe9, 0xc4, 0x27, 0x6e, 0xa3, 0x64, 0x84, 0xc1, 0x7b, 0xc9, 0xb7,
-	0xb8, 0x92, 0x0c, 0xc3, 0x68, 0x2d, 0x06, 0x60, 0x0a, 0x5c, 0x8f, 0xf9, 0xc4, 0x69, 0xdf, 0xae,
-	0x6a, 0x1b, 0xc2, 0xe0, 0x02, 0x2a, 0xce, 0x34, 0xb2, 0xd3, 0xdb, 0xee, 0x91, 0x1b, 0xe5, 0xc5,
-	0x91, 0x9a, 0x00, 0x22, 0x19, 0xb0, 0x58, 0x04, 0xc2, 0xa5, 0x88, 0x42, 0x8c, 0x49, 0x1e, 0x98,
-	0xe4, 0x15, 0x0d, 0x31, 0x09, 0xae, 0x8b, 0x38, 0x3e, 0xb5, 0x7b, 0xcb, 0x8e, 0xe6, 0x3b, 0x4b,
-	0xff, 0x42, 0x85, 0x2b, 0xbe, 0xb6, 0x47, 0xc6, 0x3a, 0x9c, 0xe7, 0x16, 0x5e, 0xf3, 0x0a, 0x78,
-	0xbb, 0x6b, 0x99, 0x5a, 0xaa, 0xdc, 0xa2, 0x78, 0x4c, 0xfe, 0xd5, 0x09, 0x92, 0x24, 0x34, 0xae,
-	0xe3, 0xdd, 0x06, 0x45, 0x16, 0xb9, 0xbf, 0xc4, 0x90, 0xb4, 0x3f, 0xcb, 0xa0, 0x6a, 0x2b, 0xbe,
-	0xc7, 0x96, 0x7e, 0xd3, 0x73, 0xd4, 0x53, 0x34, 0x24, 0x27, 0xdc, 0xf9, 0x16, 0x30, 0x6e, 0xbb,
-	0x69, 0xdb, 0xb2, 0x22, 0xe5, 0xe8, 0x8e, 0xf9, 0x82, 0xce, 0x9c, 0xb6, 0x96, 0xa6, 0xba, 0x7b,
-	0x1e, 0x81, 0x6c, 0x19, 0x71, 0x1f, 0x01, 0xb0, 0xc9, 0xa6, 0x68, 0x96, 0xf6, 0xf8, 0x45, 0x14,
-	0xb2, 0xae, 0xa9, 0xac, 0xc9, 0xb6, 0xe9, 0xba, 0xca, 0x9a, 0x40, 0x21, 0xeb, 0x86, 0x8c, 0x81,
-	0x9e, 0x43, 0x0d, 0x81, 0x9c, 0x0c, 0x46, 0x18, 0x4d, 0x16, 0x84, 0x4c, 0x5c, 0x30, 0x89, 0x95,
-	0xb0, 0x4d, 0x07, 0xcb, 0x37, 0x71, 0xbe, 0x5e, 0x82, 0xc3, 0x61, 0xb6, 0x54, 0x66, 0x54, 0x43,
-	0x32, 0x6f, 0xab, 0xcc, 0x49, 0x1c, 0x32, 0x5f, 0xd4, 0x6f, 0xc6, 0x2b, 0x91, 0xf0, 0xaf, 0x9f,
-	0xbd, 0xcf, 0xe8, 0x77, 0xe2, 0xb5, 0x50, 0x71, 0xc8, 0x0a, 0x0e, 0xf8, 0x7b, 0xf4, 0x27, 0x3a,
-	0xb0, 0xe4, 0x4a, 0xdc, 0xc8, 0xa0, 0xbe, 0x60, 0xbd, 0xba, 0x70, 0x7b, 0x05, 0x60, 0xd8, 0x3e,
-	0xa4, 0xa2, 0x4a, 0x4b, 0x89, 0x6a, 0x29, 0xc6, 0xb4, 0x9a, 0xfb, 0x66, 0xc7, 0xb4, 0xe3, 0xeb,
-	0x19, 0xbb, 0x4a, 0xb1, 0xa4, 0x65, 0x15, 0x06, 0x59, 0x41, 0xf1, 0xbe, 0xad, 0xad, 0xe5, 0xca,
-	0xaf, 0xb1, 0x21, 0x10, 0x5f, 0x79, 0xc7, 0x5b, 0xee, 0x62, 0x0b, 0x55, 0x1b, 0x64, 0xa8, 0xe5,
-	0x33, 0xc7, 0x6d, 0x37, 0x3b, 0x98, 0xd1, 0x53, 0x0a, 0xcc, 0x78, 0x81, 0xb0, 0x34, 0xc5, 0xe0,
-	0xb3, 0x25, 0x2d, 0x8c, 0x1f, 0xe1, 0x69, 0x78, 0xee, 0xce, 0x33, 0xf9, 0x69, 0xd5, 0xc2, 0x7e,
-	0x4a, 0xa7, 0x5b, 0x6d, 0x18, 0x9d, 0x5d, 0x53, 0x36, 0xf3, 0x05, 0xc2, 0x7c, 0xb6, 0x67, 0xb4,
-	0xe4, 0x05, 0x35, 0x01, 0x6d, 0x1b, 0x36, 0xee, 0x5e, 0x49, 0x62, 0x3c, 0xd3, 0x67, 0x2a, 0x7b,
-	0xec, 0x03, 0x3f, 0x38, 0xe4, 0xb7, 0x00, 0x07, 0x7e, 0x30, 0xfc, 0x14, 0xff, 0x17, 0x19, 0x79,
-	0x2b, 0xf0, 0xc1, 0xe7, 0x3f, 0xfc, 0xfc, 0x70, 0x14, 0x1d, 0x1d, 0x1f, 0x7c, 0x3a, 0xf0, 0x27,
-	0xf7, 0x05, 0xd9, 0x7d, 0x24, 0xfb, 0x15, 0xfa, 0xcf, 0x66, 0x4e, 0x1e, 0xdd, 0x3f, 0xf4, 0xd5,
-	0xff, 0x72, 0xe6, 0x60, 0x9d, 0x63, 0x3e, 0xff, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x5b, 0x23,
-	0xc1, 0x5b, 0x96, 0x66, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/openolt/openolt.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/openolt/openolt.pb.go
deleted file mode 100644
index 2c8af68..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/openolt/openolt.pb.go
+++ /dev/null
@@ -1,6405 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/openolt.proto
-
-package openolt
-
-import (
-	context "context"
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	common "github.com/opencord/voltha-protos/v4/go/common"
-	config "github.com/opencord/voltha-protos/v4/go/ext/config"
-	tech_profile "github.com/opencord/voltha-protos/v4/go/tech_profile"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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
-
-// SchedulerConfig from public import voltha_protos/tech_profile.proto
-type SchedulerConfig = tech_profile.SchedulerConfig
-
-// TrafficShapingInfo from public import voltha_protos/tech_profile.proto
-type TrafficShapingInfo = tech_profile.TrafficShapingInfo
-
-// TrafficScheduler from public import voltha_protos/tech_profile.proto
-type TrafficScheduler = tech_profile.TrafficScheduler
-
-// TrafficSchedulers from public import voltha_protos/tech_profile.proto
-type TrafficSchedulers = tech_profile.TrafficSchedulers
-
-// TailDropDiscardConfig from public import voltha_protos/tech_profile.proto
-type TailDropDiscardConfig = tech_profile.TailDropDiscardConfig
-
-// RedDiscardConfig from public import voltha_protos/tech_profile.proto
-type RedDiscardConfig = tech_profile.RedDiscardConfig
-
-// WRedDiscardConfig from public import voltha_protos/tech_profile.proto
-type WRedDiscardConfig = tech_profile.WRedDiscardConfig
-
-// DiscardConfig from public import voltha_protos/tech_profile.proto
-type DiscardConfig = tech_profile.DiscardConfig
-type DiscardConfig_TailDropDiscardConfig = tech_profile.DiscardConfig_TailDropDiscardConfig
-type DiscardConfig_RedDiscardConfig = tech_profile.DiscardConfig_RedDiscardConfig
-type DiscardConfig_WredDiscardConfig = tech_profile.DiscardConfig_WredDiscardConfig
-
-// TrafficQueue from public import voltha_protos/tech_profile.proto
-type TrafficQueue = tech_profile.TrafficQueue
-
-// TrafficQueues from public import voltha_protos/tech_profile.proto
-type TrafficQueues = tech_profile.TrafficQueues
-
-// InstanceControl from public import voltha_protos/tech_profile.proto
-type InstanceControl = tech_profile.InstanceControl
-
-// QThresholds from public import voltha_protos/tech_profile.proto
-type QThresholds = tech_profile.QThresholds
-
-// GemPortAttributes from public import voltha_protos/tech_profile.proto
-type GemPortAttributes = tech_profile.GemPortAttributes
-
-// SchedulerAttributes from public import voltha_protos/tech_profile.proto
-type SchedulerAttributes = tech_profile.SchedulerAttributes
-
-// EPONQueueAttributes from public import voltha_protos/tech_profile.proto
-type EPONQueueAttributes = tech_profile.EPONQueueAttributes
-
-// TechProfile from public import voltha_protos/tech_profile.proto
-type TechProfile = tech_profile.TechProfile
-
-// EponTechProfile from public import voltha_protos/tech_profile.proto
-type EponTechProfile = tech_profile.EponTechProfile
-
-// TechProfileInstance from public import voltha_protos/tech_profile.proto
-type TechProfileInstance = tech_profile.TechProfileInstance
-
-// EponTechProfileInstance from public import voltha_protos/tech_profile.proto
-type EponTechProfileInstance = tech_profile.EponTechProfileInstance
-
-// ResourceInstance from public import voltha_protos/tech_profile.proto
-type ResourceInstance = tech_profile.ResourceInstance
-
-// Direction from public import voltha_protos/tech_profile.proto
-type Direction = tech_profile.Direction
-
-var Direction_name = tech_profile.Direction_name
-var Direction_value = tech_profile.Direction_value
-
-const Direction_UPSTREAM = Direction(tech_profile.Direction_UPSTREAM)
-const Direction_DOWNSTREAM = Direction(tech_profile.Direction_DOWNSTREAM)
-const Direction_BIDIRECTIONAL = Direction(tech_profile.Direction_BIDIRECTIONAL)
-
-// SchedulingPolicy from public import voltha_protos/tech_profile.proto
-type SchedulingPolicy = tech_profile.SchedulingPolicy
-
-var SchedulingPolicy_name = tech_profile.SchedulingPolicy_name
-var SchedulingPolicy_value = tech_profile.SchedulingPolicy_value
-
-const SchedulingPolicy_WRR = SchedulingPolicy(tech_profile.SchedulingPolicy_WRR)
-const SchedulingPolicy_StrictPriority = SchedulingPolicy(tech_profile.SchedulingPolicy_StrictPriority)
-const SchedulingPolicy_Hybrid = SchedulingPolicy(tech_profile.SchedulingPolicy_Hybrid)
-
-// AdditionalBW from public import voltha_protos/tech_profile.proto
-type AdditionalBW = tech_profile.AdditionalBW
-
-var AdditionalBW_name = tech_profile.AdditionalBW_name
-var AdditionalBW_value = tech_profile.AdditionalBW_value
-
-const AdditionalBW_AdditionalBW_None = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_None)
-const AdditionalBW_AdditionalBW_NA = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_NA)
-const AdditionalBW_AdditionalBW_BestEffort = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_BestEffort)
-const AdditionalBW_AdditionalBW_Auto = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_Auto)
-
-// DiscardPolicy from public import voltha_protos/tech_profile.proto
-type DiscardPolicy = tech_profile.DiscardPolicy
-
-var DiscardPolicy_name = tech_profile.DiscardPolicy_name
-var DiscardPolicy_value = tech_profile.DiscardPolicy_value
-
-const DiscardPolicy_TailDrop = DiscardPolicy(tech_profile.DiscardPolicy_TailDrop)
-const DiscardPolicy_WTailDrop = DiscardPolicy(tech_profile.DiscardPolicy_WTailDrop)
-const DiscardPolicy_Red = DiscardPolicy(tech_profile.DiscardPolicy_Red)
-const DiscardPolicy_WRed = DiscardPolicy(tech_profile.DiscardPolicy_WRed)
-
-// InferredAdditionBWIndication from public import voltha_protos/tech_profile.proto
-type InferredAdditionBWIndication = tech_profile.InferredAdditionBWIndication
-
-var InferredAdditionBWIndication_name = tech_profile.InferredAdditionBWIndication_name
-var InferredAdditionBWIndication_value = tech_profile.InferredAdditionBWIndication_value
-
-const InferredAdditionBWIndication_InferredAdditionBWIndication_None = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_None)
-const InferredAdditionBWIndication_InferredAdditionBWIndication_Assured = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_Assured)
-const InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort)
-
-// ID from public import voltha_protos/common.proto
-type ID = common.ID
-
-// IDs from public import voltha_protos/common.proto
-type IDs = common.IDs
-
-// AdminState from public import voltha_protos/common.proto
-type AdminState = common.AdminState
-
-// OperStatus from public import voltha_protos/common.proto
-type OperStatus = common.OperStatus
-
-// ConnectStatus from public import voltha_protos/common.proto
-type ConnectStatus = common.ConnectStatus
-
-// OperationResp from public import voltha_protos/common.proto
-type OperationResp = common.OperationResp
-
-// ValueType from public import voltha_protos/common.proto
-type ValueType = common.ValueType
-
-// ValueSpecifier from public import voltha_protos/common.proto
-type ValueSpecifier = common.ValueSpecifier
-
-// ReturnValues from public import voltha_protos/common.proto
-type ReturnValues = common.ReturnValues
-
-// TestModeKeys from public import voltha_protos/common.proto
-type TestModeKeys = common.TestModeKeys
-
-var TestModeKeys_name = common.TestModeKeys_name
-var TestModeKeys_value = common.TestModeKeys_value
-
-const TestModeKeys_api_test = TestModeKeys(common.TestModeKeys_api_test)
-
-// AdminState_Types from public import voltha_protos/common.proto
-type AdminState_Types = common.AdminState_Types
-
-var AdminState_Types_name = common.AdminState_Types_name
-var AdminState_Types_value = common.AdminState_Types_value
-
-const AdminState_UNKNOWN = AdminState_Types(common.AdminState_UNKNOWN)
-const AdminState_PREPROVISIONED = AdminState_Types(common.AdminState_PREPROVISIONED)
-const AdminState_ENABLED = AdminState_Types(common.AdminState_ENABLED)
-const AdminState_DISABLED = AdminState_Types(common.AdminState_DISABLED)
-const AdminState_DOWNLOADING_IMAGE = AdminState_Types(common.AdminState_DOWNLOADING_IMAGE)
-
-// OperStatus_Types from public import voltha_protos/common.proto
-type OperStatus_Types = common.OperStatus_Types
-
-var OperStatus_Types_name = common.OperStatus_Types_name
-var OperStatus_Types_value = common.OperStatus_Types_value
-
-const OperStatus_UNKNOWN = OperStatus_Types(common.OperStatus_UNKNOWN)
-const OperStatus_DISCOVERED = OperStatus_Types(common.OperStatus_DISCOVERED)
-const OperStatus_ACTIVATING = OperStatus_Types(common.OperStatus_ACTIVATING)
-const OperStatus_TESTING = OperStatus_Types(common.OperStatus_TESTING)
-const OperStatus_ACTIVE = OperStatus_Types(common.OperStatus_ACTIVE)
-const OperStatus_FAILED = OperStatus_Types(common.OperStatus_FAILED)
-const OperStatus_RECONCILING = OperStatus_Types(common.OperStatus_RECONCILING)
-const OperStatus_RECONCILING_FAILED = OperStatus_Types(common.OperStatus_RECONCILING_FAILED)
-
-// ConnectStatus_Types from public import voltha_protos/common.proto
-type ConnectStatus_Types = common.ConnectStatus_Types
-
-var ConnectStatus_Types_name = common.ConnectStatus_Types_name
-var ConnectStatus_Types_value = common.ConnectStatus_Types_value
-
-const ConnectStatus_UNKNOWN = ConnectStatus_Types(common.ConnectStatus_UNKNOWN)
-const ConnectStatus_UNREACHABLE = ConnectStatus_Types(common.ConnectStatus_UNREACHABLE)
-const ConnectStatus_REACHABLE = ConnectStatus_Types(common.ConnectStatus_REACHABLE)
-
-// OperationResp_OperationReturnCode from public import voltha_protos/common.proto
-type OperationResp_OperationReturnCode = common.OperationResp_OperationReturnCode
-
-var OperationResp_OperationReturnCode_name = common.OperationResp_OperationReturnCode_name
-var OperationResp_OperationReturnCode_value = common.OperationResp_OperationReturnCode_value
-
-const OperationResp_OPERATION_SUCCESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_SUCCESS)
-const OperationResp_OPERATION_FAILURE = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_FAILURE)
-const OperationResp_OPERATION_UNSUPPORTED = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_UNSUPPORTED)
-const OperationResp_OPERATION_IN_PROGRESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_IN_PROGRESS)
-
-// ValueType_Type from public import voltha_protos/common.proto
-type ValueType_Type = common.ValueType_Type
-
-var ValueType_Type_name = common.ValueType_Type_name
-var ValueType_Type_value = common.ValueType_Type_value
-
-const ValueType_EMPTY = ValueType_Type(common.ValueType_EMPTY)
-const ValueType_DISTANCE = ValueType_Type(common.ValueType_DISTANCE)
-
-//* activation fail reason.
-type OnuIndication_ActivationFailReason int32
-
-const (
-	OnuIndication_ONU_ACTIVATION_FAIL_REASON_NONE                    OnuIndication_ActivationFailReason = 0
-	OnuIndication_ONU_ACTIVATION_FAIL_REASON_RANGING                 OnuIndication_ActivationFailReason = 1
-	OnuIndication_ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION OnuIndication_ActivationFailReason = 2
-	OnuIndication_ONU_ACTIVATION_FAIL_REASON_LOS                     OnuIndication_ActivationFailReason = 3
-	OnuIndication_ONU_ACTIVATION_FAIL_ONU_ALARM                      OnuIndication_ActivationFailReason = 4
-	OnuIndication_ONU_ACTIVATION_FAIL_SWITCH_OVER                    OnuIndication_ActivationFailReason = 5
-)
-
-var OnuIndication_ActivationFailReason_name = map[int32]string{
-	0: "ONU_ACTIVATION_FAIL_REASON_NONE",
-	1: "ONU_ACTIVATION_FAIL_REASON_RANGING",
-	2: "ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION",
-	3: "ONU_ACTIVATION_FAIL_REASON_LOS",
-	4: "ONU_ACTIVATION_FAIL_ONU_ALARM",
-	5: "ONU_ACTIVATION_FAIL_SWITCH_OVER",
-}
-
-var OnuIndication_ActivationFailReason_value = map[string]int32{
-	"ONU_ACTIVATION_FAIL_REASON_NONE":                    0,
-	"ONU_ACTIVATION_FAIL_REASON_RANGING":                 1,
-	"ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION": 2,
-	"ONU_ACTIVATION_FAIL_REASON_LOS":                     3,
-	"ONU_ACTIVATION_FAIL_ONU_ALARM":                      4,
-	"ONU_ACTIVATION_FAIL_SWITCH_OVER":                    5,
-}
-
-func (x OnuIndication_ActivationFailReason) String() string {
-	return proto.EnumName(OnuIndication_ActivationFailReason_name, int32(x))
-}
-
-func (OnuIndication_ActivationFailReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{5, 0}
-}
-
-type DeviceInfo_DeviceResourceRanges_Pool_PoolType int32
-
-const (
-	DeviceInfo_DeviceResourceRanges_Pool_ONU_ID     DeviceInfo_DeviceResourceRanges_Pool_PoolType = 0
-	DeviceInfo_DeviceResourceRanges_Pool_ALLOC_ID   DeviceInfo_DeviceResourceRanges_Pool_PoolType = 1
-	DeviceInfo_DeviceResourceRanges_Pool_GEMPORT_ID DeviceInfo_DeviceResourceRanges_Pool_PoolType = 2
-	DeviceInfo_DeviceResourceRanges_Pool_FLOW_ID    DeviceInfo_DeviceResourceRanges_Pool_PoolType = 3
-)
-
-var DeviceInfo_DeviceResourceRanges_Pool_PoolType_name = map[int32]string{
-	0: "ONU_ID",
-	1: "ALLOC_ID",
-	2: "GEMPORT_ID",
-	3: "FLOW_ID",
-}
-
-var DeviceInfo_DeviceResourceRanges_Pool_PoolType_value = map[string]int32{
-	"ONU_ID":     0,
-	"ALLOC_ID":   1,
-	"GEMPORT_ID": 2,
-	"FLOW_ID":    3,
-}
-
-func (x DeviceInfo_DeviceResourceRanges_Pool_PoolType) String() string {
-	return proto.EnumName(DeviceInfo_DeviceResourceRanges_Pool_PoolType_name, int32(x))
-}
-
-func (DeviceInfo_DeviceResourceRanges_Pool_PoolType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0, 0}
-}
-
-type DeviceInfo_DeviceResourceRanges_Pool_SharingType int32
-
-const (
-	DeviceInfo_DeviceResourceRanges_Pool_DEDICATED_PER_INTF           DeviceInfo_DeviceResourceRanges_Pool_SharingType = 0
-	DeviceInfo_DeviceResourceRanges_Pool_SHARED_BY_ALL_INTF_ALL_TECH  DeviceInfo_DeviceResourceRanges_Pool_SharingType = 1
-	DeviceInfo_DeviceResourceRanges_Pool_SHARED_BY_ALL_INTF_SAME_TECH DeviceInfo_DeviceResourceRanges_Pool_SharingType = 2
-)
-
-var DeviceInfo_DeviceResourceRanges_Pool_SharingType_name = map[int32]string{
-	0: "DEDICATED_PER_INTF",
-	1: "SHARED_BY_ALL_INTF_ALL_TECH",
-	2: "SHARED_BY_ALL_INTF_SAME_TECH",
-}
-
-var DeviceInfo_DeviceResourceRanges_Pool_SharingType_value = map[string]int32{
-	"DEDICATED_PER_INTF":           0,
-	"SHARED_BY_ALL_INTF_ALL_TECH":  1,
-	"SHARED_BY_ALL_INTF_SAME_TECH": 2,
-}
-
-func (x DeviceInfo_DeviceResourceRanges_Pool_SharingType) String() string {
-	return proto.EnumName(DeviceInfo_DeviceResourceRanges_Pool_SharingType_name, int32(x))
-}
-
-func (DeviceInfo_DeviceResourceRanges_Pool_SharingType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0, 1}
-}
-
-type GroupMember_InterfaceType int32
-
-const (
-	GroupMember_PON           GroupMember_InterfaceType = 0
-	GroupMember_EPON_1G_PATH  GroupMember_InterfaceType = 1
-	GroupMember_EPON_10G_PATH GroupMember_InterfaceType = 2
-)
-
-var GroupMember_InterfaceType_name = map[int32]string{
-	0: "PON",
-	1: "EPON_1G_PATH",
-	2: "EPON_10G_PATH",
-}
-
-var GroupMember_InterfaceType_value = map[string]int32{
-	"PON":           0,
-	"EPON_1G_PATH":  1,
-	"EPON_10G_PATH": 2,
-}
-
-func (x GroupMember_InterfaceType) String() string {
-	return proto.EnumName(GroupMember_InterfaceType_name, int32(x))
-}
-
-func (GroupMember_InterfaceType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{46, 0}
-}
-
-type Group_GroupMembersCommand int32
-
-const (
-	Group_ADD_MEMBERS    Group_GroupMembersCommand = 0
-	Group_REMOVE_MEMBERS Group_GroupMembersCommand = 1
-	Group_SET_MEMBERS    Group_GroupMembersCommand = 2
-)
-
-var Group_GroupMembersCommand_name = map[int32]string{
-	0: "ADD_MEMBERS",
-	1: "REMOVE_MEMBERS",
-	2: "SET_MEMBERS",
-}
-
-var Group_GroupMembersCommand_value = map[string]int32{
-	"ADD_MEMBERS":    0,
-	"REMOVE_MEMBERS": 1,
-	"SET_MEMBERS":    2,
-}
-
-func (x Group_GroupMembersCommand) String() string {
-	return proto.EnumName(Group_GroupMembersCommand_name, int32(x))
-}
-
-func (Group_GroupMembersCommand) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{47, 0}
-}
-
-type PonRxPowerData_RssiMeasurementFailReason int32
-
-const (
-	PonRxPowerData_FAIL_REASON_NONE         PonRxPowerData_RssiMeasurementFailReason = 0
-	PonRxPowerData_FAIL_REASON_NO_DELIMITER PonRxPowerData_RssiMeasurementFailReason = 1
-	PonRxPowerData_FAIL_REASON_NO_ACCESS    PonRxPowerData_RssiMeasurementFailReason = 2
-)
-
-var PonRxPowerData_RssiMeasurementFailReason_name = map[int32]string{
-	0: "FAIL_REASON_NONE",
-	1: "FAIL_REASON_NO_DELIMITER",
-	2: "FAIL_REASON_NO_ACCESS",
-}
-
-var PonRxPowerData_RssiMeasurementFailReason_value = map[string]int32{
-	"FAIL_REASON_NONE":         0,
-	"FAIL_REASON_NO_DELIMITER": 1,
-	"FAIL_REASON_NO_ACCESS":    2,
-}
-
-func (x PonRxPowerData_RssiMeasurementFailReason) String() string {
-	return proto.EnumName(PonRxPowerData_RssiMeasurementFailReason_name, int32(x))
-}
-
-func (PonRxPowerData_RssiMeasurementFailReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{49, 0}
-}
-
-type Indication struct {
-	// Types that are valid to be assigned to Data:
-	//	*Indication_OltInd
-	//	*Indication_IntfInd
-	//	*Indication_IntfOperInd
-	//	*Indication_OnuDiscInd
-	//	*Indication_OnuInd
-	//	*Indication_OmciInd
-	//	*Indication_PktInd
-	//	*Indication_PortStats
-	//	*Indication_FlowStats
-	//	*Indication_AlarmInd
-	Data                 isIndication_Data `protobuf_oneof:"data"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *Indication) Reset()         { *m = Indication{} }
-func (m *Indication) String() string { return proto.CompactTextString(m) }
-func (*Indication) ProtoMessage()    {}
-func (*Indication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{0}
-}
-
-func (m *Indication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Indication.Unmarshal(m, b)
-}
-func (m *Indication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Indication.Marshal(b, m, deterministic)
-}
-func (m *Indication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Indication.Merge(m, src)
-}
-func (m *Indication) XXX_Size() int {
-	return xxx_messageInfo_Indication.Size(m)
-}
-func (m *Indication) XXX_DiscardUnknown() {
-	xxx_messageInfo_Indication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Indication proto.InternalMessageInfo
-
-type isIndication_Data interface {
-	isIndication_Data()
-}
-
-type Indication_OltInd struct {
-	OltInd *OltIndication `protobuf:"bytes,1,opt,name=olt_ind,json=oltInd,proto3,oneof"`
-}
-
-type Indication_IntfInd struct {
-	IntfInd *IntfIndication `protobuf:"bytes,2,opt,name=intf_ind,json=intfInd,proto3,oneof"`
-}
-
-type Indication_IntfOperInd struct {
-	IntfOperInd *IntfOperIndication `protobuf:"bytes,3,opt,name=intf_oper_ind,json=intfOperInd,proto3,oneof"`
-}
-
-type Indication_OnuDiscInd struct {
-	OnuDiscInd *OnuDiscIndication `protobuf:"bytes,4,opt,name=onu_disc_ind,json=onuDiscInd,proto3,oneof"`
-}
-
-type Indication_OnuInd struct {
-	OnuInd *OnuIndication `protobuf:"bytes,5,opt,name=onu_ind,json=onuInd,proto3,oneof"`
-}
-
-type Indication_OmciInd struct {
-	OmciInd *OmciIndication `protobuf:"bytes,6,opt,name=omci_ind,json=omciInd,proto3,oneof"`
-}
-
-type Indication_PktInd struct {
-	PktInd *PacketIndication `protobuf:"bytes,7,opt,name=pkt_ind,json=pktInd,proto3,oneof"`
-}
-
-type Indication_PortStats struct {
-	PortStats *PortStatistics `protobuf:"bytes,8,opt,name=port_stats,json=portStats,proto3,oneof"`
-}
-
-type Indication_FlowStats struct {
-	FlowStats *FlowStatistics `protobuf:"bytes,9,opt,name=flow_stats,json=flowStats,proto3,oneof"`
-}
-
-type Indication_AlarmInd struct {
-	AlarmInd *AlarmIndication `protobuf:"bytes,10,opt,name=alarm_ind,json=alarmInd,proto3,oneof"`
-}
-
-func (*Indication_OltInd) isIndication_Data() {}
-
-func (*Indication_IntfInd) isIndication_Data() {}
-
-func (*Indication_IntfOperInd) isIndication_Data() {}
-
-func (*Indication_OnuDiscInd) isIndication_Data() {}
-
-func (*Indication_OnuInd) isIndication_Data() {}
-
-func (*Indication_OmciInd) isIndication_Data() {}
-
-func (*Indication_PktInd) isIndication_Data() {}
-
-func (*Indication_PortStats) isIndication_Data() {}
-
-func (*Indication_FlowStats) isIndication_Data() {}
-
-func (*Indication_AlarmInd) isIndication_Data() {}
-
-func (m *Indication) GetData() isIndication_Data {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-func (m *Indication) GetOltInd() *OltIndication {
-	if x, ok := m.GetData().(*Indication_OltInd); ok {
-		return x.OltInd
-	}
-	return nil
-}
-
-func (m *Indication) GetIntfInd() *IntfIndication {
-	if x, ok := m.GetData().(*Indication_IntfInd); ok {
-		return x.IntfInd
-	}
-	return nil
-}
-
-func (m *Indication) GetIntfOperInd() *IntfOperIndication {
-	if x, ok := m.GetData().(*Indication_IntfOperInd); ok {
-		return x.IntfOperInd
-	}
-	return nil
-}
-
-func (m *Indication) GetOnuDiscInd() *OnuDiscIndication {
-	if x, ok := m.GetData().(*Indication_OnuDiscInd); ok {
-		return x.OnuDiscInd
-	}
-	return nil
-}
-
-func (m *Indication) GetOnuInd() *OnuIndication {
-	if x, ok := m.GetData().(*Indication_OnuInd); ok {
-		return x.OnuInd
-	}
-	return nil
-}
-
-func (m *Indication) GetOmciInd() *OmciIndication {
-	if x, ok := m.GetData().(*Indication_OmciInd); ok {
-		return x.OmciInd
-	}
-	return nil
-}
-
-func (m *Indication) GetPktInd() *PacketIndication {
-	if x, ok := m.GetData().(*Indication_PktInd); ok {
-		return x.PktInd
-	}
-	return nil
-}
-
-func (m *Indication) GetPortStats() *PortStatistics {
-	if x, ok := m.GetData().(*Indication_PortStats); ok {
-		return x.PortStats
-	}
-	return nil
-}
-
-func (m *Indication) GetFlowStats() *FlowStatistics {
-	if x, ok := m.GetData().(*Indication_FlowStats); ok {
-		return x.FlowStats
-	}
-	return nil
-}
-
-func (m *Indication) GetAlarmInd() *AlarmIndication {
-	if x, ok := m.GetData().(*Indication_AlarmInd); ok {
-		return x.AlarmInd
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*Indication) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*Indication_OltInd)(nil),
-		(*Indication_IntfInd)(nil),
-		(*Indication_IntfOperInd)(nil),
-		(*Indication_OnuDiscInd)(nil),
-		(*Indication_OnuInd)(nil),
-		(*Indication_OmciInd)(nil),
-		(*Indication_PktInd)(nil),
-		(*Indication_PortStats)(nil),
-		(*Indication_FlowStats)(nil),
-		(*Indication_AlarmInd)(nil),
-	}
-}
-
-type AlarmIndication struct {
-	// Types that are valid to be assigned to Data:
-	//	*AlarmIndication_LosInd
-	//	*AlarmIndication_DyingGaspInd
-	//	*AlarmIndication_OnuAlarmInd
-	//	*AlarmIndication_OnuStartupFailInd
-	//	*AlarmIndication_OnuSignalDegradeInd
-	//	*AlarmIndication_OnuDriftOfWindowInd
-	//	*AlarmIndication_OnuLossOmciInd
-	//	*AlarmIndication_OnuSignalsFailInd
-	//	*AlarmIndication_OnuTiwiInd
-	//	*AlarmIndication_OnuActivationFailInd
-	//	*AlarmIndication_OnuProcessingErrorInd
-	//	*AlarmIndication_OnuLossOfSyncFailInd
-	//	*AlarmIndication_OnuItuPonStatsInd
-	//	*AlarmIndication_OnuDeactivationFailureInd
-	//	*AlarmIndication_OnuRemoteDefectInd
-	//	*AlarmIndication_OnuLossGemDelineationInd
-	//	*AlarmIndication_OnuPhysicalEquipmentErrorInd
-	//	*AlarmIndication_OnuLossOfAckInd
-	//	*AlarmIndication_OnuDiffReachExceededInd
-	Data                 isAlarmIndication_Data `protobuf_oneof:"data"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *AlarmIndication) Reset()         { *m = AlarmIndication{} }
-func (m *AlarmIndication) String() string { return proto.CompactTextString(m) }
-func (*AlarmIndication) ProtoMessage()    {}
-func (*AlarmIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{1}
-}
-
-func (m *AlarmIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AlarmIndication.Unmarshal(m, b)
-}
-func (m *AlarmIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AlarmIndication.Marshal(b, m, deterministic)
-}
-func (m *AlarmIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AlarmIndication.Merge(m, src)
-}
-func (m *AlarmIndication) XXX_Size() int {
-	return xxx_messageInfo_AlarmIndication.Size(m)
-}
-func (m *AlarmIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_AlarmIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AlarmIndication proto.InternalMessageInfo
-
-type isAlarmIndication_Data interface {
-	isAlarmIndication_Data()
-}
-
-type AlarmIndication_LosInd struct {
-	LosInd *LosIndication `protobuf:"bytes,1,opt,name=los_ind,json=losInd,proto3,oneof"`
-}
-
-type AlarmIndication_DyingGaspInd struct {
-	DyingGaspInd *DyingGaspIndication `protobuf:"bytes,2,opt,name=dying_gasp_ind,json=dyingGaspInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuAlarmInd struct {
-	OnuAlarmInd *OnuAlarmIndication `protobuf:"bytes,3,opt,name=onu_alarm_ind,json=onuAlarmInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuStartupFailInd struct {
-	OnuStartupFailInd *OnuStartupFailureIndication `protobuf:"bytes,4,opt,name=onu_startup_fail_ind,json=onuStartupFailInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuSignalDegradeInd struct {
-	OnuSignalDegradeInd *OnuSignalDegradeIndication `protobuf:"bytes,5,opt,name=onu_signal_degrade_ind,json=onuSignalDegradeInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuDriftOfWindowInd struct {
-	OnuDriftOfWindowInd *OnuDriftOfWindowIndication `protobuf:"bytes,6,opt,name=onu_drift_of_window_ind,json=onuDriftOfWindowInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuLossOmciInd struct {
-	OnuLossOmciInd *OnuLossOfOmciChannelIndication `protobuf:"bytes,7,opt,name=onu_loss_omci_ind,json=onuLossOmciInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuSignalsFailInd struct {
-	OnuSignalsFailInd *OnuSignalsFailureIndication `protobuf:"bytes,8,opt,name=onu_signals_fail_ind,json=onuSignalsFailInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuTiwiInd struct {
-	OnuTiwiInd *OnuTransmissionInterferenceWarning `protobuf:"bytes,9,opt,name=onu_tiwi_ind,json=onuTiwiInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuActivationFailInd struct {
-	OnuActivationFailInd *OnuActivationFailureIndication `protobuf:"bytes,10,opt,name=onu_activation_fail_ind,json=onuActivationFailInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuProcessingErrorInd struct {
-	OnuProcessingErrorInd *OnuProcessingErrorIndication `protobuf:"bytes,11,opt,name=onu_processing_error_ind,json=onuProcessingErrorInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuLossOfSyncFailInd struct {
-	OnuLossOfSyncFailInd *OnuLossOfKeySyncFailureIndication `protobuf:"bytes,12,opt,name=onu_loss_of_sync_fail_ind,json=onuLossOfSyncFailInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuItuPonStatsInd struct {
-	OnuItuPonStatsInd *OnuItuPonStatsIndication `protobuf:"bytes,13,opt,name=onu_itu_pon_stats_ind,json=onuItuPonStatsInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuDeactivationFailureInd struct {
-	OnuDeactivationFailureInd *OnuDeactivationFailureIndication `protobuf:"bytes,14,opt,name=onu_deactivation_failure_ind,json=onuDeactivationFailureInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuRemoteDefectInd struct {
-	OnuRemoteDefectInd *OnuRemoteDefectIndication `protobuf:"bytes,15,opt,name=onu_remote_defect_ind,json=onuRemoteDefectInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuLossGemDelineationInd struct {
-	OnuLossGemDelineationInd *OnuLossOfGEMChannelDelineationIndication `protobuf:"bytes,16,opt,name=onu_loss_gem_delineation_ind,json=onuLossGemDelineationInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuPhysicalEquipmentErrorInd struct {
-	OnuPhysicalEquipmentErrorInd *OnuPhysicalEquipmentErrorIndication `protobuf:"bytes,17,opt,name=onu_physical_equipment_error_ind,json=onuPhysicalEquipmentErrorInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuLossOfAckInd struct {
-	OnuLossOfAckInd *OnuLossOfAcknowledgementIndication `protobuf:"bytes,18,opt,name=onu_loss_of_ack_ind,json=onuLossOfAckInd,proto3,oneof"`
-}
-
-type AlarmIndication_OnuDiffReachExceededInd struct {
-	OnuDiffReachExceededInd *OnuDifferentialReachExceededIndication `protobuf:"bytes,19,opt,name=onu_diff_reach_exceeded_ind,json=onuDiffReachExceededInd,proto3,oneof"`
-}
-
-func (*AlarmIndication_LosInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_DyingGaspInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuAlarmInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuStartupFailInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuSignalDegradeInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuDriftOfWindowInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuLossOmciInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuSignalsFailInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuTiwiInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuActivationFailInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuProcessingErrorInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuLossOfSyncFailInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuItuPonStatsInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuDeactivationFailureInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuRemoteDefectInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuLossGemDelineationInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuPhysicalEquipmentErrorInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuLossOfAckInd) isAlarmIndication_Data() {}
-
-func (*AlarmIndication_OnuDiffReachExceededInd) isAlarmIndication_Data() {}
-
-func (m *AlarmIndication) GetData() isAlarmIndication_Data {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetLosInd() *LosIndication {
-	if x, ok := m.GetData().(*AlarmIndication_LosInd); ok {
-		return x.LosInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetDyingGaspInd() *DyingGaspIndication {
-	if x, ok := m.GetData().(*AlarmIndication_DyingGaspInd); ok {
-		return x.DyingGaspInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuAlarmInd() *OnuAlarmIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuAlarmInd); ok {
-		return x.OnuAlarmInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuStartupFailInd() *OnuStartupFailureIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuStartupFailInd); ok {
-		return x.OnuStartupFailInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuSignalDegradeInd() *OnuSignalDegradeIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuSignalDegradeInd); ok {
-		return x.OnuSignalDegradeInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuDriftOfWindowInd() *OnuDriftOfWindowIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuDriftOfWindowInd); ok {
-		return x.OnuDriftOfWindowInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuLossOmciInd() *OnuLossOfOmciChannelIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuLossOmciInd); ok {
-		return x.OnuLossOmciInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuSignalsFailInd() *OnuSignalsFailureIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuSignalsFailInd); ok {
-		return x.OnuSignalsFailInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuTiwiInd() *OnuTransmissionInterferenceWarning {
-	if x, ok := m.GetData().(*AlarmIndication_OnuTiwiInd); ok {
-		return x.OnuTiwiInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuActivationFailInd() *OnuActivationFailureIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuActivationFailInd); ok {
-		return x.OnuActivationFailInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuProcessingErrorInd() *OnuProcessingErrorIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuProcessingErrorInd); ok {
-		return x.OnuProcessingErrorInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuLossOfSyncFailInd() *OnuLossOfKeySyncFailureIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuLossOfSyncFailInd); ok {
-		return x.OnuLossOfSyncFailInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuItuPonStatsInd() *OnuItuPonStatsIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuItuPonStatsInd); ok {
-		return x.OnuItuPonStatsInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuDeactivationFailureInd() *OnuDeactivationFailureIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuDeactivationFailureInd); ok {
-		return x.OnuDeactivationFailureInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuRemoteDefectInd() *OnuRemoteDefectIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuRemoteDefectInd); ok {
-		return x.OnuRemoteDefectInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuLossGemDelineationInd() *OnuLossOfGEMChannelDelineationIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuLossGemDelineationInd); ok {
-		return x.OnuLossGemDelineationInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuPhysicalEquipmentErrorInd() *OnuPhysicalEquipmentErrorIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuPhysicalEquipmentErrorInd); ok {
-		return x.OnuPhysicalEquipmentErrorInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuLossOfAckInd() *OnuLossOfAcknowledgementIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuLossOfAckInd); ok {
-		return x.OnuLossOfAckInd
-	}
-	return nil
-}
-
-func (m *AlarmIndication) GetOnuDiffReachExceededInd() *OnuDifferentialReachExceededIndication {
-	if x, ok := m.GetData().(*AlarmIndication_OnuDiffReachExceededInd); ok {
-		return x.OnuDiffReachExceededInd
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*AlarmIndication) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*AlarmIndication_LosInd)(nil),
-		(*AlarmIndication_DyingGaspInd)(nil),
-		(*AlarmIndication_OnuAlarmInd)(nil),
-		(*AlarmIndication_OnuStartupFailInd)(nil),
-		(*AlarmIndication_OnuSignalDegradeInd)(nil),
-		(*AlarmIndication_OnuDriftOfWindowInd)(nil),
-		(*AlarmIndication_OnuLossOmciInd)(nil),
-		(*AlarmIndication_OnuSignalsFailInd)(nil),
-		(*AlarmIndication_OnuTiwiInd)(nil),
-		(*AlarmIndication_OnuActivationFailInd)(nil),
-		(*AlarmIndication_OnuProcessingErrorInd)(nil),
-		(*AlarmIndication_OnuLossOfSyncFailInd)(nil),
-		(*AlarmIndication_OnuItuPonStatsInd)(nil),
-		(*AlarmIndication_OnuDeactivationFailureInd)(nil),
-		(*AlarmIndication_OnuRemoteDefectInd)(nil),
-		(*AlarmIndication_OnuLossGemDelineationInd)(nil),
-		(*AlarmIndication_OnuPhysicalEquipmentErrorInd)(nil),
-		(*AlarmIndication_OnuLossOfAckInd)(nil),
-		(*AlarmIndication_OnuDiffReachExceededInd)(nil),
-	}
-}
-
-type OltIndication struct {
-	OperState            string   `protobuf:"bytes,1,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OltIndication) Reset()         { *m = OltIndication{} }
-func (m *OltIndication) String() string { return proto.CompactTextString(m) }
-func (*OltIndication) ProtoMessage()    {}
-func (*OltIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{2}
-}
-
-func (m *OltIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OltIndication.Unmarshal(m, b)
-}
-func (m *OltIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OltIndication.Marshal(b, m, deterministic)
-}
-func (m *OltIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OltIndication.Merge(m, src)
-}
-func (m *OltIndication) XXX_Size() int {
-	return xxx_messageInfo_OltIndication.Size(m)
-}
-func (m *OltIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OltIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OltIndication proto.InternalMessageInfo
-
-func (m *OltIndication) GetOperState() string {
-	if m != nil {
-		return m.OperState
-	}
-	return ""
-}
-
-type IntfIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OperState            string   `protobuf:"bytes,2,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *IntfIndication) Reset()         { *m = IntfIndication{} }
-func (m *IntfIndication) String() string { return proto.CompactTextString(m) }
-func (*IntfIndication) ProtoMessage()    {}
-func (*IntfIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{3}
-}
-
-func (m *IntfIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IntfIndication.Unmarshal(m, b)
-}
-func (m *IntfIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IntfIndication.Marshal(b, m, deterministic)
-}
-func (m *IntfIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IntfIndication.Merge(m, src)
-}
-func (m *IntfIndication) XXX_Size() int {
-	return xxx_messageInfo_IntfIndication.Size(m)
-}
-func (m *IntfIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_IntfIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IntfIndication proto.InternalMessageInfo
-
-func (m *IntfIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *IntfIndication) GetOperState() string {
-	if m != nil {
-		return m.OperState
-	}
-	return ""
-}
-
-type OnuDiscIndication struct {
-	IntfId               uint32        `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	SerialNumber         *SerialNumber `protobuf:"bytes,2,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *OnuDiscIndication) Reset()         { *m = OnuDiscIndication{} }
-func (m *OnuDiscIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuDiscIndication) ProtoMessage()    {}
-func (*OnuDiscIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{4}
-}
-
-func (m *OnuDiscIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuDiscIndication.Unmarshal(m, b)
-}
-func (m *OnuDiscIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuDiscIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuDiscIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuDiscIndication.Merge(m, src)
-}
-func (m *OnuDiscIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuDiscIndication.Size(m)
-}
-func (m *OnuDiscIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuDiscIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuDiscIndication proto.InternalMessageInfo
-
-func (m *OnuDiscIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuDiscIndication) GetSerialNumber() *SerialNumber {
-	if m != nil {
-		return m.SerialNumber
-	}
-	return nil
-}
-
-type OnuIndication struct {
-	IntfId               uint32                             `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32                             `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	OperState            string                             `protobuf:"bytes,3,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
-	AdminState           string                             `protobuf:"bytes,5,opt,name=admin_state,json=adminState,proto3" json:"admin_state,omitempty"`
-	SerialNumber         *SerialNumber                      `protobuf:"bytes,4,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
-	FailReason           OnuIndication_ActivationFailReason `protobuf:"varint,6,opt,name=fail_reason,json=failReason,proto3,enum=openolt.OnuIndication_ActivationFailReason" json:"fail_reason,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
-}
-
-func (m *OnuIndication) Reset()         { *m = OnuIndication{} }
-func (m *OnuIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuIndication) ProtoMessage()    {}
-func (*OnuIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{5}
-}
-
-func (m *OnuIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuIndication.Unmarshal(m, b)
-}
-func (m *OnuIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuIndication.Merge(m, src)
-}
-func (m *OnuIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuIndication.Size(m)
-}
-func (m *OnuIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuIndication proto.InternalMessageInfo
-
-func (m *OnuIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuIndication) GetOperState() string {
-	if m != nil {
-		return m.OperState
-	}
-	return ""
-}
-
-func (m *OnuIndication) GetAdminState() string {
-	if m != nil {
-		return m.AdminState
-	}
-	return ""
-}
-
-func (m *OnuIndication) GetSerialNumber() *SerialNumber {
-	if m != nil {
-		return m.SerialNumber
-	}
-	return nil
-}
-
-func (m *OnuIndication) GetFailReason() OnuIndication_ActivationFailReason {
-	if m != nil {
-		return m.FailReason
-	}
-	return OnuIndication_ONU_ACTIVATION_FAIL_REASON_NONE
-}
-
-type IntfOperIndication struct {
-	Type                 string   `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
-	IntfId               uint32   `protobuf:"fixed32,2,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OperState            string   `protobuf:"bytes,3,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *IntfOperIndication) Reset()         { *m = IntfOperIndication{} }
-func (m *IntfOperIndication) String() string { return proto.CompactTextString(m) }
-func (*IntfOperIndication) ProtoMessage()    {}
-func (*IntfOperIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{6}
-}
-
-func (m *IntfOperIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IntfOperIndication.Unmarshal(m, b)
-}
-func (m *IntfOperIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IntfOperIndication.Marshal(b, m, deterministic)
-}
-func (m *IntfOperIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IntfOperIndication.Merge(m, src)
-}
-func (m *IntfOperIndication) XXX_Size() int {
-	return xxx_messageInfo_IntfOperIndication.Size(m)
-}
-func (m *IntfOperIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_IntfOperIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IntfOperIndication proto.InternalMessageInfo
-
-func (m *IntfOperIndication) GetType() string {
-	if m != nil {
-		return m.Type
-	}
-	return ""
-}
-
-func (m *IntfOperIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *IntfOperIndication) GetOperState() string {
-	if m != nil {
-		return m.OperState
-	}
-	return ""
-}
-
-type OmciIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OmciIndication) Reset()         { *m = OmciIndication{} }
-func (m *OmciIndication) String() string { return proto.CompactTextString(m) }
-func (*OmciIndication) ProtoMessage()    {}
-func (*OmciIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{7}
-}
-
-func (m *OmciIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OmciIndication.Unmarshal(m, b)
-}
-func (m *OmciIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OmciIndication.Marshal(b, m, deterministic)
-}
-func (m *OmciIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OmciIndication.Merge(m, src)
-}
-func (m *OmciIndication) XXX_Size() int {
-	return xxx_messageInfo_OmciIndication.Size(m)
-}
-func (m *OmciIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OmciIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OmciIndication proto.InternalMessageInfo
-
-func (m *OmciIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OmciIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OmciIndication) GetPkt() []byte {
-	if m != nil {
-		return m.Pkt
-	}
-	return nil
-}
-
-type PacketIndication struct {
-	IntfType             string   `protobuf:"bytes,5,opt,name=intf_type,json=intfType,proto3" json:"intf_type,omitempty"`
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,8,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	UniId                uint32   `protobuf:"fixed32,9,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	GemportId            uint32   `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	FlowId               uint32   `protobuf:"fixed32,3,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
-	PortNo               uint32   `protobuf:"fixed32,6,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	Cookie               uint64   `protobuf:"fixed64,7,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	Pkt                  []byte   `protobuf:"bytes,4,opt,name=pkt,proto3" json:"pkt,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PacketIndication) Reset()         { *m = PacketIndication{} }
-func (m *PacketIndication) String() string { return proto.CompactTextString(m) }
-func (*PacketIndication) ProtoMessage()    {}
-func (*PacketIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{8}
-}
-
-func (m *PacketIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PacketIndication.Unmarshal(m, b)
-}
-func (m *PacketIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PacketIndication.Marshal(b, m, deterministic)
-}
-func (m *PacketIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PacketIndication.Merge(m, src)
-}
-func (m *PacketIndication) XXX_Size() int {
-	return xxx_messageInfo_PacketIndication.Size(m)
-}
-func (m *PacketIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_PacketIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PacketIndication proto.InternalMessageInfo
-
-func (m *PacketIndication) GetIntfType() string {
-	if m != nil {
-		return m.IntfType
-	}
-	return ""
-}
-
-func (m *PacketIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetFlowId() uint32 {
-	if m != nil {
-		return m.FlowId
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *PacketIndication) GetPkt() []byte {
-	if m != nil {
-		return m.Pkt
-	}
-	return nil
-}
-
-type Interface struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Interface) Reset()         { *m = Interface{} }
-func (m *Interface) String() string { return proto.CompactTextString(m) }
-func (*Interface) ProtoMessage()    {}
-func (*Interface) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{9}
-}
-
-func (m *Interface) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Interface.Unmarshal(m, b)
-}
-func (m *Interface) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Interface.Marshal(b, m, deterministic)
-}
-func (m *Interface) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Interface.Merge(m, src)
-}
-func (m *Interface) XXX_Size() int {
-	return xxx_messageInfo_Interface.Size(m)
-}
-func (m *Interface) XXX_DiscardUnknown() {
-	xxx_messageInfo_Interface.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Interface proto.InternalMessageInfo
-
-func (m *Interface) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-type Heartbeat struct {
-	HeartbeatSignature   uint32   `protobuf:"fixed32,1,opt,name=heartbeat_signature,json=heartbeatSignature,proto3" json:"heartbeat_signature,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Heartbeat) Reset()         { *m = Heartbeat{} }
-func (m *Heartbeat) String() string { return proto.CompactTextString(m) }
-func (*Heartbeat) ProtoMessage()    {}
-func (*Heartbeat) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{10}
-}
-
-func (m *Heartbeat) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Heartbeat.Unmarshal(m, b)
-}
-func (m *Heartbeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Heartbeat.Marshal(b, m, deterministic)
-}
-func (m *Heartbeat) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Heartbeat.Merge(m, src)
-}
-func (m *Heartbeat) XXX_Size() int {
-	return xxx_messageInfo_Heartbeat.Size(m)
-}
-func (m *Heartbeat) XXX_DiscardUnknown() {
-	xxx_messageInfo_Heartbeat.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Heartbeat proto.InternalMessageInfo
-
-func (m *Heartbeat) GetHeartbeatSignature() uint32 {
-	if m != nil {
-		return m.HeartbeatSignature
-	}
-	return 0
-}
-
-type Onu struct {
-	IntfId               uint32        `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32        `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	SerialNumber         *SerialNumber `protobuf:"bytes,3,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
-	Pir                  uint32        `protobuf:"fixed32,4,opt,name=pir,proto3" json:"pir,omitempty"`
-	OmccEncryption       bool          `protobuf:"varint,5,opt,name=omcc_encryption,json=omccEncryption,proto3" json:"omcc_encryption,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *Onu) Reset()         { *m = Onu{} }
-func (m *Onu) String() string { return proto.CompactTextString(m) }
-func (*Onu) ProtoMessage()    {}
-func (*Onu) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{11}
-}
-
-func (m *Onu) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Onu.Unmarshal(m, b)
-}
-func (m *Onu) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Onu.Marshal(b, m, deterministic)
-}
-func (m *Onu) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Onu.Merge(m, src)
-}
-func (m *Onu) XXX_Size() int {
-	return xxx_messageInfo_Onu.Size(m)
-}
-func (m *Onu) XXX_DiscardUnknown() {
-	xxx_messageInfo_Onu.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Onu proto.InternalMessageInfo
-
-func (m *Onu) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *Onu) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *Onu) GetSerialNumber() *SerialNumber {
-	if m != nil {
-		return m.SerialNumber
-	}
-	return nil
-}
-
-func (m *Onu) GetPir() uint32 {
-	if m != nil {
-		return m.Pir
-	}
-	return 0
-}
-
-func (m *Onu) GetOmccEncryption() bool {
-	if m != nil {
-		return m.OmccEncryption
-	}
-	return false
-}
-
-type OnuLogicalDistance struct {
-	IntfId                 uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                  uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	LogicalOnuDistanceZero uint32   `protobuf:"fixed32,3,opt,name=logical_onu_distance_zero,json=logicalOnuDistanceZero,proto3" json:"logical_onu_distance_zero,omitempty"`
-	LogicalOnuDistance     uint32   `protobuf:"fixed32,4,opt,name=logical_onu_distance,json=logicalOnuDistance,proto3" json:"logical_onu_distance,omitempty"`
-	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
-	XXX_unrecognized       []byte   `json:"-"`
-	XXX_sizecache          int32    `json:"-"`
-}
-
-func (m *OnuLogicalDistance) Reset()         { *m = OnuLogicalDistance{} }
-func (m *OnuLogicalDistance) String() string { return proto.CompactTextString(m) }
-func (*OnuLogicalDistance) ProtoMessage()    {}
-func (*OnuLogicalDistance) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{12}
-}
-
-func (m *OnuLogicalDistance) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuLogicalDistance.Unmarshal(m, b)
-}
-func (m *OnuLogicalDistance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuLogicalDistance.Marshal(b, m, deterministic)
-}
-func (m *OnuLogicalDistance) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuLogicalDistance.Merge(m, src)
-}
-func (m *OnuLogicalDistance) XXX_Size() int {
-	return xxx_messageInfo_OnuLogicalDistance.Size(m)
-}
-func (m *OnuLogicalDistance) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuLogicalDistance.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuLogicalDistance proto.InternalMessageInfo
-
-func (m *OnuLogicalDistance) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuLogicalDistance) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuLogicalDistance) GetLogicalOnuDistanceZero() uint32 {
-	if m != nil {
-		return m.LogicalOnuDistanceZero
-	}
-	return 0
-}
-
-func (m *OnuLogicalDistance) GetLogicalOnuDistance() uint32 {
-	if m != nil {
-		return m.LogicalOnuDistance
-	}
-	return 0
-}
-
-type OmciMsg struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OmciMsg) Reset()         { *m = OmciMsg{} }
-func (m *OmciMsg) String() string { return proto.CompactTextString(m) }
-func (*OmciMsg) ProtoMessage()    {}
-func (*OmciMsg) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{13}
-}
-
-func (m *OmciMsg) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OmciMsg.Unmarshal(m, b)
-}
-func (m *OmciMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OmciMsg.Marshal(b, m, deterministic)
-}
-func (m *OmciMsg) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OmciMsg.Merge(m, src)
-}
-func (m *OmciMsg) XXX_Size() int {
-	return xxx_messageInfo_OmciMsg.Size(m)
-}
-func (m *OmciMsg) XXX_DiscardUnknown() {
-	xxx_messageInfo_OmciMsg.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OmciMsg proto.InternalMessageInfo
-
-func (m *OmciMsg) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OmciMsg) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OmciMsg) GetPkt() []byte {
-	if m != nil {
-		return m.Pkt
-	}
-	return nil
-}
-
-type OnuPacket struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	PortNo               uint32   `protobuf:"fixed32,4,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	GemportId            uint32   `protobuf:"fixed32,5,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuPacket) Reset()         { *m = OnuPacket{} }
-func (m *OnuPacket) String() string { return proto.CompactTextString(m) }
-func (*OnuPacket) ProtoMessage()    {}
-func (*OnuPacket) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{14}
-}
-
-func (m *OnuPacket) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuPacket.Unmarshal(m, b)
-}
-func (m *OnuPacket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuPacket.Marshal(b, m, deterministic)
-}
-func (m *OnuPacket) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuPacket.Merge(m, src)
-}
-func (m *OnuPacket) XXX_Size() int {
-	return xxx_messageInfo_OnuPacket.Size(m)
-}
-func (m *OnuPacket) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuPacket.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuPacket proto.InternalMessageInfo
-
-func (m *OnuPacket) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuPacket) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuPacket) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *OnuPacket) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *OnuPacket) GetPkt() []byte {
-	if m != nil {
-		return m.Pkt
-	}
-	return nil
-}
-
-type UplinkPacket struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	Pkt                  []byte   `protobuf:"bytes,2,opt,name=pkt,proto3" json:"pkt,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *UplinkPacket) Reset()         { *m = UplinkPacket{} }
-func (m *UplinkPacket) String() string { return proto.CompactTextString(m) }
-func (*UplinkPacket) ProtoMessage()    {}
-func (*UplinkPacket) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{15}
-}
-
-func (m *UplinkPacket) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_UplinkPacket.Unmarshal(m, b)
-}
-func (m *UplinkPacket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_UplinkPacket.Marshal(b, m, deterministic)
-}
-func (m *UplinkPacket) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_UplinkPacket.Merge(m, src)
-}
-func (m *UplinkPacket) XXX_Size() int {
-	return xxx_messageInfo_UplinkPacket.Size(m)
-}
-func (m *UplinkPacket) XXX_DiscardUnknown() {
-	xxx_messageInfo_UplinkPacket.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_UplinkPacket proto.InternalMessageInfo
-
-func (m *UplinkPacket) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *UplinkPacket) GetPkt() []byte {
-	if m != nil {
-		return m.Pkt
-	}
-	return nil
-}
-
-type DeviceInfo struct {
-	Vendor              string `protobuf:"bytes,1,opt,name=vendor,proto3" json:"vendor,omitempty"`
-	Model               string `protobuf:"bytes,2,opt,name=model,proto3" json:"model,omitempty"`
-	HardwareVersion     string `protobuf:"bytes,3,opt,name=hardware_version,json=hardwareVersion,proto3" json:"hardware_version,omitempty"`
-	FirmwareVersion     string `protobuf:"bytes,4,opt,name=firmware_version,json=firmwareVersion,proto3" json:"firmware_version,omitempty"`
-	DeviceId            string `protobuf:"bytes,16,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	DeviceSerialNumber  string `protobuf:"bytes,17,opt,name=device_serial_number,json=deviceSerialNumber,proto3" json:"device_serial_number,omitempty"`
-	PreviouslyConnected bool   `protobuf:"varint,19,opt,name=previously_connected,json=previouslyConnected,proto3" json:"previously_connected,omitempty"`
-	// Total number of pon intf ports on the device
-	PonPorts uint32 `protobuf:"fixed32,12,opt,name=pon_ports,json=ponPorts,proto3" json:"pon_ports,omitempty"`
-	// If using global per-device technology profile. To be deprecated
-	Technology           string                             `protobuf:"bytes,5,opt,name=technology,proto3" json:"technology,omitempty"`
-	OnuIdStart           uint32                             `protobuf:"fixed32,6,opt,name=onu_id_start,json=onuIdStart,proto3" json:"onu_id_start,omitempty"`
-	OnuIdEnd             uint32                             `protobuf:"fixed32,7,opt,name=onu_id_end,json=onuIdEnd,proto3" json:"onu_id_end,omitempty"`
-	AllocIdStart         uint32                             `protobuf:"fixed32,8,opt,name=alloc_id_start,json=allocIdStart,proto3" json:"alloc_id_start,omitempty"`
-	AllocIdEnd           uint32                             `protobuf:"fixed32,9,opt,name=alloc_id_end,json=allocIdEnd,proto3" json:"alloc_id_end,omitempty"`
-	GemportIdStart       uint32                             `protobuf:"fixed32,10,opt,name=gemport_id_start,json=gemportIdStart,proto3" json:"gemport_id_start,omitempty"`
-	GemportIdEnd         uint32                             `protobuf:"fixed32,11,opt,name=gemport_id_end,json=gemportIdEnd,proto3" json:"gemport_id_end,omitempty"`
-	FlowIdStart          uint32                             `protobuf:"fixed32,13,opt,name=flow_id_start,json=flowIdStart,proto3" json:"flow_id_start,omitempty"`
-	FlowIdEnd            uint32                             `protobuf:"fixed32,14,opt,name=flow_id_end,json=flowIdEnd,proto3" json:"flow_id_end,omitempty"`
-	Ranges               []*DeviceInfo_DeviceResourceRanges `protobuf:"bytes,15,rep,name=ranges,proto3" json:"ranges,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
-}
-
-func (m *DeviceInfo) Reset()         { *m = DeviceInfo{} }
-func (m *DeviceInfo) String() string { return proto.CompactTextString(m) }
-func (*DeviceInfo) ProtoMessage()    {}
-func (*DeviceInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{16}
-}
-
-func (m *DeviceInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceInfo.Unmarshal(m, b)
-}
-func (m *DeviceInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceInfo.Marshal(b, m, deterministic)
-}
-func (m *DeviceInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceInfo.Merge(m, src)
-}
-func (m *DeviceInfo) XXX_Size() int {
-	return xxx_messageInfo_DeviceInfo.Size(m)
-}
-func (m *DeviceInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceInfo proto.InternalMessageInfo
-
-func (m *DeviceInfo) GetVendor() string {
-	if m != nil {
-		return m.Vendor
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetModel() string {
-	if m != nil {
-		return m.Model
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetHardwareVersion() string {
-	if m != nil {
-		return m.HardwareVersion
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetFirmwareVersion() string {
-	if m != nil {
-		return m.FirmwareVersion
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetDeviceSerialNumber() string {
-	if m != nil {
-		return m.DeviceSerialNumber
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetPreviouslyConnected() bool {
-	if m != nil {
-		return m.PreviouslyConnected
-	}
-	return false
-}
-
-func (m *DeviceInfo) GetPonPorts() uint32 {
-	if m != nil {
-		return m.PonPorts
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetTechnology() string {
-	if m != nil {
-		return m.Technology
-	}
-	return ""
-}
-
-func (m *DeviceInfo) GetOnuIdStart() uint32 {
-	if m != nil {
-		return m.OnuIdStart
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetOnuIdEnd() uint32 {
-	if m != nil {
-		return m.OnuIdEnd
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetAllocIdStart() uint32 {
-	if m != nil {
-		return m.AllocIdStart
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetAllocIdEnd() uint32 {
-	if m != nil {
-		return m.AllocIdEnd
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetGemportIdStart() uint32 {
-	if m != nil {
-		return m.GemportIdStart
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetGemportIdEnd() uint32 {
-	if m != nil {
-		return m.GemportIdEnd
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetFlowIdStart() uint32 {
-	if m != nil {
-		return m.FlowIdStart
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetFlowIdEnd() uint32 {
-	if m != nil {
-		return m.FlowIdEnd
-	}
-	return 0
-}
-
-func (m *DeviceInfo) GetRanges() []*DeviceInfo_DeviceResourceRanges {
-	if m != nil {
-		return m.Ranges
-	}
-	return nil
-}
-
-type DeviceInfo_DeviceResourceRanges struct {
-	// List of 0 or more intf_ids that use the same technology and pools.
-	// If 0 intf_ids supplied, it implies ALL interfaces
-	IntfIds []uint32 `protobuf:"fixed32,1,rep,packed,name=intf_ids,json=intfIds,proto3" json:"intf_ids,omitempty"`
-	// Technology profile for this pool
-	Technology           string                                  `protobuf:"bytes,2,opt,name=technology,proto3" json:"technology,omitempty"`
-	Pools                []*DeviceInfo_DeviceResourceRanges_Pool `protobuf:"bytes,3,rep,name=pools,proto3" json:"pools,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                `json:"-"`
-	XXX_unrecognized     []byte                                  `json:"-"`
-	XXX_sizecache        int32                                   `json:"-"`
-}
-
-func (m *DeviceInfo_DeviceResourceRanges) Reset()         { *m = DeviceInfo_DeviceResourceRanges{} }
-func (m *DeviceInfo_DeviceResourceRanges) String() string { return proto.CompactTextString(m) }
-func (*DeviceInfo_DeviceResourceRanges) ProtoMessage()    {}
-func (*DeviceInfo_DeviceResourceRanges) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0}
-}
-
-func (m *DeviceInfo_DeviceResourceRanges) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Unmarshal(m, b)
-}
-func (m *DeviceInfo_DeviceResourceRanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Marshal(b, m, deterministic)
-}
-func (m *DeviceInfo_DeviceResourceRanges) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Merge(m, src)
-}
-func (m *DeviceInfo_DeviceResourceRanges) XXX_Size() int {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Size(m)
-}
-func (m *DeviceInfo_DeviceResourceRanges) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceInfo_DeviceResourceRanges.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceInfo_DeviceResourceRanges proto.InternalMessageInfo
-
-func (m *DeviceInfo_DeviceResourceRanges) GetIntfIds() []uint32 {
-	if m != nil {
-		return m.IntfIds
-	}
-	return nil
-}
-
-func (m *DeviceInfo_DeviceResourceRanges) GetTechnology() string {
-	if m != nil {
-		return m.Technology
-	}
-	return ""
-}
-
-func (m *DeviceInfo_DeviceResourceRanges) GetPools() []*DeviceInfo_DeviceResourceRanges_Pool {
-	if m != nil {
-		return m.Pools
-	}
-	return nil
-}
-
-type DeviceInfo_DeviceResourceRanges_Pool struct {
-	Type                 DeviceInfo_DeviceResourceRanges_Pool_PoolType    `protobuf:"varint,1,opt,name=type,proto3,enum=openolt.DeviceInfo_DeviceResourceRanges_Pool_PoolType" json:"type,omitempty"`
-	Sharing              DeviceInfo_DeviceResourceRanges_Pool_SharingType `protobuf:"varint,2,opt,name=sharing,proto3,enum=openolt.DeviceInfo_DeviceResourceRanges_Pool_SharingType" json:"sharing,omitempty"`
-	Start                uint32                                           `protobuf:"fixed32,3,opt,name=start,proto3" json:"start,omitempty"`
-	End                  uint32                                           `protobuf:"fixed32,4,opt,name=end,proto3" json:"end,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                         `json:"-"`
-	XXX_unrecognized     []byte                                           `json:"-"`
-	XXX_sizecache        int32                                            `json:"-"`
-}
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) Reset()         { *m = DeviceInfo_DeviceResourceRanges_Pool{} }
-func (m *DeviceInfo_DeviceResourceRanges_Pool) String() string { return proto.CompactTextString(m) }
-func (*DeviceInfo_DeviceResourceRanges_Pool) ProtoMessage()    {}
-func (*DeviceInfo_DeviceResourceRanges_Pool) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0}
-}
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Unmarshal(m, b)
-}
-func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Marshal(b, m, deterministic)
-}
-func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Merge(m, src)
-}
-func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Size() int {
-	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Size(m)
-}
-func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool proto.InternalMessageInfo
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) GetType() DeviceInfo_DeviceResourceRanges_Pool_PoolType {
-	if m != nil {
-		return m.Type
-	}
-	return DeviceInfo_DeviceResourceRanges_Pool_ONU_ID
-}
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) GetSharing() DeviceInfo_DeviceResourceRanges_Pool_SharingType {
-	if m != nil {
-		return m.Sharing
-	}
-	return DeviceInfo_DeviceResourceRanges_Pool_DEDICATED_PER_INTF
-}
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) GetStart() uint32 {
-	if m != nil {
-		return m.Start
-	}
-	return 0
-}
-
-func (m *DeviceInfo_DeviceResourceRanges_Pool) GetEnd() uint32 {
-	if m != nil {
-		return m.End
-	}
-	return 0
-}
-
-type Classifier struct {
-	OTpid                uint32   `protobuf:"fixed32,1,opt,name=o_tpid,json=oTpid,proto3" json:"o_tpid,omitempty"`
-	OVid                 uint32   `protobuf:"fixed32,2,opt,name=o_vid,json=oVid,proto3" json:"o_vid,omitempty"`
-	ITpid                uint32   `protobuf:"fixed32,3,opt,name=i_tpid,json=iTpid,proto3" json:"i_tpid,omitempty"`
-	IVid                 uint32   `protobuf:"fixed32,4,opt,name=i_vid,json=iVid,proto3" json:"i_vid,omitempty"`
-	OPbits               uint32   `protobuf:"fixed32,5,opt,name=o_pbits,json=oPbits,proto3" json:"o_pbits,omitempty"`
-	IPbits               uint32   `protobuf:"fixed32,6,opt,name=i_pbits,json=iPbits,proto3" json:"i_pbits,omitempty"`
-	EthType              uint32   `protobuf:"fixed32,7,opt,name=eth_type,json=ethType,proto3" json:"eth_type,omitempty"`
-	DstMac               []byte   `protobuf:"bytes,8,opt,name=dst_mac,json=dstMac,proto3" json:"dst_mac,omitempty"`
-	SrcMac               []byte   `protobuf:"bytes,9,opt,name=src_mac,json=srcMac,proto3" json:"src_mac,omitempty"`
-	IpProto              uint32   `protobuf:"fixed32,10,opt,name=ip_proto,json=ipProto,proto3" json:"ip_proto,omitempty"`
-	DstIp                uint32   `protobuf:"fixed32,11,opt,name=dst_ip,json=dstIp,proto3" json:"dst_ip,omitempty"`
-	SrcIp                uint32   `protobuf:"fixed32,12,opt,name=src_ip,json=srcIp,proto3" json:"src_ip,omitempty"`
-	SrcPort              uint32   `protobuf:"fixed32,13,opt,name=src_port,json=srcPort,proto3" json:"src_port,omitempty"`
-	DstPort              uint32   `protobuf:"fixed32,14,opt,name=dst_port,json=dstPort,proto3" json:"dst_port,omitempty"`
-	PktTagType           string   `protobuf:"bytes,15,opt,name=pkt_tag_type,json=pktTagType,proto3" json:"pkt_tag_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Classifier) Reset()         { *m = Classifier{} }
-func (m *Classifier) String() string { return proto.CompactTextString(m) }
-func (*Classifier) ProtoMessage()    {}
-func (*Classifier) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{17}
-}
-
-func (m *Classifier) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Classifier.Unmarshal(m, b)
-}
-func (m *Classifier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Classifier.Marshal(b, m, deterministic)
-}
-func (m *Classifier) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Classifier.Merge(m, src)
-}
-func (m *Classifier) XXX_Size() int {
-	return xxx_messageInfo_Classifier.Size(m)
-}
-func (m *Classifier) XXX_DiscardUnknown() {
-	xxx_messageInfo_Classifier.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Classifier proto.InternalMessageInfo
-
-func (m *Classifier) GetOTpid() uint32 {
-	if m != nil {
-		return m.OTpid
-	}
-	return 0
-}
-
-func (m *Classifier) GetOVid() uint32 {
-	if m != nil {
-		return m.OVid
-	}
-	return 0
-}
-
-func (m *Classifier) GetITpid() uint32 {
-	if m != nil {
-		return m.ITpid
-	}
-	return 0
-}
-
-func (m *Classifier) GetIVid() uint32 {
-	if m != nil {
-		return m.IVid
-	}
-	return 0
-}
-
-func (m *Classifier) GetOPbits() uint32 {
-	if m != nil {
-		return m.OPbits
-	}
-	return 0
-}
-
-func (m *Classifier) GetIPbits() uint32 {
-	if m != nil {
-		return m.IPbits
-	}
-	return 0
-}
-
-func (m *Classifier) GetEthType() uint32 {
-	if m != nil {
-		return m.EthType
-	}
-	return 0
-}
-
-func (m *Classifier) GetDstMac() []byte {
-	if m != nil {
-		return m.DstMac
-	}
-	return nil
-}
-
-func (m *Classifier) GetSrcMac() []byte {
-	if m != nil {
-		return m.SrcMac
-	}
-	return nil
-}
-
-func (m *Classifier) GetIpProto() uint32 {
-	if m != nil {
-		return m.IpProto
-	}
-	return 0
-}
-
-func (m *Classifier) GetDstIp() uint32 {
-	if m != nil {
-		return m.DstIp
-	}
-	return 0
-}
-
-func (m *Classifier) GetSrcIp() uint32 {
-	if m != nil {
-		return m.SrcIp
-	}
-	return 0
-}
-
-func (m *Classifier) GetSrcPort() uint32 {
-	if m != nil {
-		return m.SrcPort
-	}
-	return 0
-}
-
-func (m *Classifier) GetDstPort() uint32 {
-	if m != nil {
-		return m.DstPort
-	}
-	return 0
-}
-
-func (m *Classifier) GetPktTagType() string {
-	if m != nil {
-		return m.PktTagType
-	}
-	return ""
-}
-
-type ActionCmd struct {
-	AddOuterTag          bool     `protobuf:"varint,1,opt,name=add_outer_tag,json=addOuterTag,proto3" json:"add_outer_tag,omitempty"`
-	RemoveOuterTag       bool     `protobuf:"varint,2,opt,name=remove_outer_tag,json=removeOuterTag,proto3" json:"remove_outer_tag,omitempty"`
-	TrapToHost           bool     `protobuf:"varint,3,opt,name=trap_to_host,json=trapToHost,proto3" json:"trap_to_host,omitempty"`
-	RemarkOuterPbits     bool     `protobuf:"varint,4,opt,name=remark_outer_pbits,json=remarkOuterPbits,proto3" json:"remark_outer_pbits,omitempty"`
-	RemarkInnerPbits     bool     `protobuf:"varint,5,opt,name=remark_inner_pbits,json=remarkInnerPbits,proto3" json:"remark_inner_pbits,omitempty"`
-	AddInnerTag          bool     `protobuf:"varint,6,opt,name=add_inner_tag,json=addInnerTag,proto3" json:"add_inner_tag,omitempty"`
-	RemoveInnerTag       bool     `protobuf:"varint,7,opt,name=remove_inner_tag,json=removeInnerTag,proto3" json:"remove_inner_tag,omitempty"`
-	TranslateInnerTag    bool     `protobuf:"varint,8,opt,name=translate_inner_tag,json=translateInnerTag,proto3" json:"translate_inner_tag,omitempty"`
-	TranslateOuterTag    bool     `protobuf:"varint,9,opt,name=translate_outer_tag,json=translateOuterTag,proto3" json:"translate_outer_tag,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ActionCmd) Reset()         { *m = ActionCmd{} }
-func (m *ActionCmd) String() string { return proto.CompactTextString(m) }
-func (*ActionCmd) ProtoMessage()    {}
-func (*ActionCmd) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{18}
-}
-
-func (m *ActionCmd) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ActionCmd.Unmarshal(m, b)
-}
-func (m *ActionCmd) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ActionCmd.Marshal(b, m, deterministic)
-}
-func (m *ActionCmd) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ActionCmd.Merge(m, src)
-}
-func (m *ActionCmd) XXX_Size() int {
-	return xxx_messageInfo_ActionCmd.Size(m)
-}
-func (m *ActionCmd) XXX_DiscardUnknown() {
-	xxx_messageInfo_ActionCmd.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ActionCmd proto.InternalMessageInfo
-
-func (m *ActionCmd) GetAddOuterTag() bool {
-	if m != nil {
-		return m.AddOuterTag
-	}
-	return false
-}
-
-func (m *ActionCmd) GetRemoveOuterTag() bool {
-	if m != nil {
-		return m.RemoveOuterTag
-	}
-	return false
-}
-
-func (m *ActionCmd) GetTrapToHost() bool {
-	if m != nil {
-		return m.TrapToHost
-	}
-	return false
-}
-
-func (m *ActionCmd) GetRemarkOuterPbits() bool {
-	if m != nil {
-		return m.RemarkOuterPbits
-	}
-	return false
-}
-
-func (m *ActionCmd) GetRemarkInnerPbits() bool {
-	if m != nil {
-		return m.RemarkInnerPbits
-	}
-	return false
-}
-
-func (m *ActionCmd) GetAddInnerTag() bool {
-	if m != nil {
-		return m.AddInnerTag
-	}
-	return false
-}
-
-func (m *ActionCmd) GetRemoveInnerTag() bool {
-	if m != nil {
-		return m.RemoveInnerTag
-	}
-	return false
-}
-
-func (m *ActionCmd) GetTranslateInnerTag() bool {
-	if m != nil {
-		return m.TranslateInnerTag
-	}
-	return false
-}
-
-func (m *ActionCmd) GetTranslateOuterTag() bool {
-	if m != nil {
-		return m.TranslateOuterTag
-	}
-	return false
-}
-
-type Action struct {
-	Cmd                  *ActionCmd `protobuf:"bytes,1,opt,name=cmd,proto3" json:"cmd,omitempty"`
-	OVid                 uint32     `protobuf:"fixed32,2,opt,name=o_vid,json=oVid,proto3" json:"o_vid,omitempty"`
-	OPbits               uint32     `protobuf:"fixed32,3,opt,name=o_pbits,json=oPbits,proto3" json:"o_pbits,omitempty"`
-	OTpid                uint32     `protobuf:"fixed32,4,opt,name=o_tpid,json=oTpid,proto3" json:"o_tpid,omitempty"`
-	IVid                 uint32     `protobuf:"fixed32,5,opt,name=i_vid,json=iVid,proto3" json:"i_vid,omitempty"`
-	IPbits               uint32     `protobuf:"fixed32,6,opt,name=i_pbits,json=iPbits,proto3" json:"i_pbits,omitempty"`
-	ITpid                uint32     `protobuf:"fixed32,7,opt,name=i_tpid,json=iTpid,proto3" json:"i_tpid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Action) Reset()         { *m = Action{} }
-func (m *Action) String() string { return proto.CompactTextString(m) }
-func (*Action) ProtoMessage()    {}
-func (*Action) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{19}
-}
-
-func (m *Action) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Action.Unmarshal(m, b)
-}
-func (m *Action) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Action.Marshal(b, m, deterministic)
-}
-func (m *Action) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Action.Merge(m, src)
-}
-func (m *Action) XXX_Size() int {
-	return xxx_messageInfo_Action.Size(m)
-}
-func (m *Action) XXX_DiscardUnknown() {
-	xxx_messageInfo_Action.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Action proto.InternalMessageInfo
-
-func (m *Action) GetCmd() *ActionCmd {
-	if m != nil {
-		return m.Cmd
-	}
-	return nil
-}
-
-func (m *Action) GetOVid() uint32 {
-	if m != nil {
-		return m.OVid
-	}
-	return 0
-}
-
-func (m *Action) GetOPbits() uint32 {
-	if m != nil {
-		return m.OPbits
-	}
-	return 0
-}
-
-func (m *Action) GetOTpid() uint32 {
-	if m != nil {
-		return m.OTpid
-	}
-	return 0
-}
-
-func (m *Action) GetIVid() uint32 {
-	if m != nil {
-		return m.IVid
-	}
-	return 0
-}
-
-func (m *Action) GetIPbits() uint32 {
-	if m != nil {
-		return m.IPbits
-	}
-	return 0
-}
-
-func (m *Action) GetITpid() uint32 {
-	if m != nil {
-		return m.ITpid
-	}
-	return 0
-}
-
-type Flow struct {
-	AccessIntfId         int32             `protobuf:"fixed32,1,opt,name=access_intf_id,json=accessIntfId,proto3" json:"access_intf_id,omitempty"`
-	OnuId                int32             `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	UniId                int32             `protobuf:"fixed32,11,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	FlowId               uint64            `protobuf:"fixed64,3,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
-	SymmetricFlowId      uint64            `protobuf:"fixed64,18,opt,name=symmetric_flow_id,json=symmetricFlowId,proto3" json:"symmetric_flow_id,omitempty"`
-	FlowType             string            `protobuf:"bytes,4,opt,name=flow_type,json=flowType,proto3" json:"flow_type,omitempty"`
-	AllocId              int32             `protobuf:"fixed32,10,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	NetworkIntfId        int32             `protobuf:"fixed32,5,opt,name=network_intf_id,json=networkIntfId,proto3" json:"network_intf_id,omitempty"`
-	GemportId            int32             `protobuf:"fixed32,6,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	Classifier           *Classifier       `protobuf:"bytes,7,opt,name=classifier,proto3" json:"classifier,omitempty"`
-	Action               *Action           `protobuf:"bytes,8,opt,name=action,proto3" json:"action,omitempty"`
-	Priority             int32             `protobuf:"fixed32,9,opt,name=priority,proto3" json:"priority,omitempty"`
-	Cookie               uint64            `protobuf:"fixed64,12,opt,name=cookie,proto3" json:"cookie,omitempty"`
-	PortNo               uint32            `protobuf:"fixed32,13,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	GroupId              uint32            `protobuf:"fixed32,14,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	TechProfileId        uint32            `protobuf:"fixed32,15,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
-	ReplicateFlow        bool              `protobuf:"varint,16,opt,name=replicate_flow,json=replicateFlow,proto3" json:"replicate_flow,omitempty"`
-	PbitToGemport        map[uint32]uint32 `protobuf:"bytes,17,rep,name=pbit_to_gemport,json=pbitToGemport,proto3" json:"pbit_to_gemport,omitempty" protobuf_key:"fixed32,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
-	GemportToAes         map[uint32]bool   `protobuf:"bytes,19,rep,name=gemport_to_aes,json=gemportToAes,proto3" json:"gemport_to_aes,omitempty" protobuf_key:"fixed32,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *Flow) Reset()         { *m = Flow{} }
-func (m *Flow) String() string { return proto.CompactTextString(m) }
-func (*Flow) ProtoMessage()    {}
-func (*Flow) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{20}
-}
-
-func (m *Flow) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Flow.Unmarshal(m, b)
-}
-func (m *Flow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Flow.Marshal(b, m, deterministic)
-}
-func (m *Flow) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Flow.Merge(m, src)
-}
-func (m *Flow) XXX_Size() int {
-	return xxx_messageInfo_Flow.Size(m)
-}
-func (m *Flow) XXX_DiscardUnknown() {
-	xxx_messageInfo_Flow.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Flow proto.InternalMessageInfo
-
-func (m *Flow) GetAccessIntfId() int32 {
-	if m != nil {
-		return m.AccessIntfId
-	}
-	return 0
-}
-
-func (m *Flow) GetOnuId() int32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *Flow) GetUniId() int32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *Flow) GetFlowId() uint64 {
-	if m != nil {
-		return m.FlowId
-	}
-	return 0
-}
-
-func (m *Flow) GetSymmetricFlowId() uint64 {
-	if m != nil {
-		return m.SymmetricFlowId
-	}
-	return 0
-}
-
-func (m *Flow) GetFlowType() string {
-	if m != nil {
-		return m.FlowType
-	}
-	return ""
-}
-
-func (m *Flow) GetAllocId() int32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-func (m *Flow) GetNetworkIntfId() int32 {
-	if m != nil {
-		return m.NetworkIntfId
-	}
-	return 0
-}
-
-func (m *Flow) GetGemportId() int32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *Flow) GetClassifier() *Classifier {
-	if m != nil {
-		return m.Classifier
-	}
-	return nil
-}
-
-func (m *Flow) GetAction() *Action {
-	if m != nil {
-		return m.Action
-	}
-	return nil
-}
-
-func (m *Flow) GetPriority() int32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *Flow) GetCookie() uint64 {
-	if m != nil {
-		return m.Cookie
-	}
-	return 0
-}
-
-func (m *Flow) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *Flow) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-func (m *Flow) GetTechProfileId() uint32 {
-	if m != nil {
-		return m.TechProfileId
-	}
-	return 0
-}
-
-func (m *Flow) GetReplicateFlow() bool {
-	if m != nil {
-		return m.ReplicateFlow
-	}
-	return false
-}
-
-func (m *Flow) GetPbitToGemport() map[uint32]uint32 {
-	if m != nil {
-		return m.PbitToGemport
-	}
-	return nil
-}
-
-func (m *Flow) GetGemportToAes() map[uint32]bool {
-	if m != nil {
-		return m.GemportToAes
-	}
-	return nil
-}
-
-type SerialNumber struct {
-	VendorId             []byte   `protobuf:"bytes,1,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
-	VendorSpecific       []byte   `protobuf:"bytes,2,opt,name=vendor_specific,json=vendorSpecific,proto3" json:"vendor_specific,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *SerialNumber) Reset()         { *m = SerialNumber{} }
-func (m *SerialNumber) String() string { return proto.CompactTextString(m) }
-func (*SerialNumber) ProtoMessage()    {}
-func (*SerialNumber) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{21}
-}
-
-func (m *SerialNumber) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SerialNumber.Unmarshal(m, b)
-}
-func (m *SerialNumber) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SerialNumber.Marshal(b, m, deterministic)
-}
-func (m *SerialNumber) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SerialNumber.Merge(m, src)
-}
-func (m *SerialNumber) XXX_Size() int {
-	return xxx_messageInfo_SerialNumber.Size(m)
-}
-func (m *SerialNumber) XXX_DiscardUnknown() {
-	xxx_messageInfo_SerialNumber.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SerialNumber proto.InternalMessageInfo
-
-func (m *SerialNumber) GetVendorId() []byte {
-	if m != nil {
-		return m.VendorId
-	}
-	return nil
-}
-
-func (m *SerialNumber) GetVendorSpecific() []byte {
-	if m != nil {
-		return m.VendorSpecific
-	}
-	return nil
-}
-
-type PortStatistics struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	RxBytes              uint64   `protobuf:"fixed64,2,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	RxUcastPackets       uint64   `protobuf:"fixed64,4,opt,name=rx_ucast_packets,json=rxUcastPackets,proto3" json:"rx_ucast_packets,omitempty"`
-	RxMcastPackets       uint64   `protobuf:"fixed64,5,opt,name=rx_mcast_packets,json=rxMcastPackets,proto3" json:"rx_mcast_packets,omitempty"`
-	RxBcastPackets       uint64   `protobuf:"fixed64,6,opt,name=rx_bcast_packets,json=rxBcastPackets,proto3" json:"rx_bcast_packets,omitempty"`
-	RxErrorPackets       uint64   `protobuf:"fixed64,7,opt,name=rx_error_packets,json=rxErrorPackets,proto3" json:"rx_error_packets,omitempty"`
-	RxFrames             uint64   `protobuf:"fixed64,17,opt,name=rx_frames,json=rxFrames,proto3" json:"rx_frames,omitempty"`
-	RxFrames_64          uint64   `protobuf:"fixed64,18,opt,name=rx_frames_64,json=rxFrames64,proto3" json:"rx_frames_64,omitempty"`
-	RxFrames_65_127      uint64   `protobuf:"fixed64,19,opt,name=rx_frames_65_127,json=rxFrames65127,proto3" json:"rx_frames_65_127,omitempty"`
-	RxFrames_128_255     uint64   `protobuf:"fixed64,20,opt,name=rx_frames_128_255,json=rxFrames128255,proto3" json:"rx_frames_128_255,omitempty"`
-	RxFrames_256_511     uint64   `protobuf:"fixed64,21,opt,name=rx_frames_256_511,json=rxFrames256511,proto3" json:"rx_frames_256_511,omitempty"`
-	RxFrames_512_1023    uint64   `protobuf:"fixed64,22,opt,name=rx_frames_512_1023,json=rxFrames5121023,proto3" json:"rx_frames_512_1023,omitempty"`
-	RxFrames_1024_1518   uint64   `protobuf:"fixed64,23,opt,name=rx_frames_1024_1518,json=rxFrames10241518,proto3" json:"rx_frames_1024_1518,omitempty"`
-	RxFrames_1519_2047   uint64   `protobuf:"fixed64,24,opt,name=rx_frames_1519_2047,json=rxFrames15192047,proto3" json:"rx_frames_1519_2047,omitempty"`
-	RxFrames_2048_4095   uint64   `protobuf:"fixed64,25,opt,name=rx_frames_2048_4095,json=rxFrames20484095,proto3" json:"rx_frames_2048_4095,omitempty"`
-	RxFrames_4096_9216   uint64   `protobuf:"fixed64,26,opt,name=rx_frames_4096_9216,json=rxFrames40969216,proto3" json:"rx_frames_4096_9216,omitempty"`
-	RxFrames_9217_16383  uint64   `protobuf:"fixed64,27,opt,name=rx_frames_9217_16383,json=rxFrames921716383,proto3" json:"rx_frames_9217_16383,omitempty"`
-	RxCrcErrors          uint64   `protobuf:"fixed64,14,opt,name=rx_crc_errors,json=rxCrcErrors,proto3" json:"rx_crc_errors,omitempty"`
-	TxBytes              uint64   `protobuf:"fixed64,8,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	TxPackets            uint64   `protobuf:"fixed64,9,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	TxUcastPackets       uint64   `protobuf:"fixed64,10,opt,name=tx_ucast_packets,json=txUcastPackets,proto3" json:"tx_ucast_packets,omitempty"`
-	TxMcastPackets       uint64   `protobuf:"fixed64,11,opt,name=tx_mcast_packets,json=txMcastPackets,proto3" json:"tx_mcast_packets,omitempty"`
-	TxBcastPackets       uint64   `protobuf:"fixed64,12,opt,name=tx_bcast_packets,json=txBcastPackets,proto3" json:"tx_bcast_packets,omitempty"`
-	TxErrorPackets       uint64   `protobuf:"fixed64,13,opt,name=tx_error_packets,json=txErrorPackets,proto3" json:"tx_error_packets,omitempty"`
-	TxFrames             uint64   `protobuf:"fixed64,28,opt,name=tx_frames,json=txFrames,proto3" json:"tx_frames,omitempty"`
-	TxFrames_64          uint64   `protobuf:"fixed64,29,opt,name=tx_frames_64,json=txFrames64,proto3" json:"tx_frames_64,omitempty"`
-	TxFrames_65_127      uint64   `protobuf:"fixed64,30,opt,name=tx_frames_65_127,json=txFrames65127,proto3" json:"tx_frames_65_127,omitempty"`
-	TxFrames_128_255     uint64   `protobuf:"fixed64,31,opt,name=tx_frames_128_255,json=txFrames128255,proto3" json:"tx_frames_128_255,omitempty"`
-	TxFrames_256_511     uint64   `protobuf:"fixed64,32,opt,name=tx_frames_256_511,json=txFrames256511,proto3" json:"tx_frames_256_511,omitempty"`
-	TxFrames_512_1023    uint64   `protobuf:"fixed64,33,opt,name=tx_frames_512_1023,json=txFrames5121023,proto3" json:"tx_frames_512_1023,omitempty"`
-	TxFrames_1024_1518   uint64   `protobuf:"fixed64,34,opt,name=tx_frames_1024_1518,json=txFrames10241518,proto3" json:"tx_frames_1024_1518,omitempty"`
-	TxFrames_1519_2047   uint64   `protobuf:"fixed64,35,opt,name=tx_frames_1519_2047,json=txFrames15192047,proto3" json:"tx_frames_1519_2047,omitempty"`
-	TxFrames_2048_4095   uint64   `protobuf:"fixed64,36,opt,name=tx_frames_2048_4095,json=txFrames20484095,proto3" json:"tx_frames_2048_4095,omitempty"`
-	TxFrames_4096_9216   uint64   `protobuf:"fixed64,37,opt,name=tx_frames_4096_9216,json=txFrames40969216,proto3" json:"tx_frames_4096_9216,omitempty"`
-	TxFrames_9217_16383  uint64   `protobuf:"fixed64,38,opt,name=tx_frames_9217_16383,json=txFrames921716383,proto3" json:"tx_frames_9217_16383,omitempty"`
-	BipErrors            uint64   `protobuf:"fixed64,15,opt,name=bip_errors,json=bipErrors,proto3" json:"bip_errors,omitempty"`
-	Timestamp            uint32   `protobuf:"fixed32,16,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PortStatistics) Reset()         { *m = PortStatistics{} }
-func (m *PortStatistics) String() string { return proto.CompactTextString(m) }
-func (*PortStatistics) ProtoMessage()    {}
-func (*PortStatistics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{22}
-}
-
-func (m *PortStatistics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PortStatistics.Unmarshal(m, b)
-}
-func (m *PortStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PortStatistics.Marshal(b, m, deterministic)
-}
-func (m *PortStatistics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PortStatistics.Merge(m, src)
-}
-func (m *PortStatistics) XXX_Size() int {
-	return xxx_messageInfo_PortStatistics.Size(m)
-}
-func (m *PortStatistics) XXX_DiscardUnknown() {
-	xxx_messageInfo_PortStatistics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PortStatistics proto.InternalMessageInfo
-
-func (m *PortStatistics) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxUcastPackets() uint64 {
-	if m != nil {
-		return m.RxUcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxMcastPackets() uint64 {
-	if m != nil {
-		return m.RxMcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxBcastPackets() uint64 {
-	if m != nil {
-		return m.RxBcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxErrorPackets() uint64 {
-	if m != nil {
-		return m.RxErrorPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames() uint64 {
-	if m != nil {
-		return m.RxFrames
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_64() uint64 {
-	if m != nil {
-		return m.RxFrames_64
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_65_127() uint64 {
-	if m != nil {
-		return m.RxFrames_65_127
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_128_255() uint64 {
-	if m != nil {
-		return m.RxFrames_128_255
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_256_511() uint64 {
-	if m != nil {
-		return m.RxFrames_256_511
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_512_1023() uint64 {
-	if m != nil {
-		return m.RxFrames_512_1023
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_1024_1518() uint64 {
-	if m != nil {
-		return m.RxFrames_1024_1518
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_1519_2047() uint64 {
-	if m != nil {
-		return m.RxFrames_1519_2047
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_2048_4095() uint64 {
-	if m != nil {
-		return m.RxFrames_2048_4095
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_4096_9216() uint64 {
-	if m != nil {
-		return m.RxFrames_4096_9216
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxFrames_9217_16383() uint64 {
-	if m != nil {
-		return m.RxFrames_9217_16383
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetRxCrcErrors() uint64 {
-	if m != nil {
-		return m.RxCrcErrors
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxUcastPackets() uint64 {
-	if m != nil {
-		return m.TxUcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxMcastPackets() uint64 {
-	if m != nil {
-		return m.TxMcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxBcastPackets() uint64 {
-	if m != nil {
-		return m.TxBcastPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxErrorPackets() uint64 {
-	if m != nil {
-		return m.TxErrorPackets
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames() uint64 {
-	if m != nil {
-		return m.TxFrames
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_64() uint64 {
-	if m != nil {
-		return m.TxFrames_64
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_65_127() uint64 {
-	if m != nil {
-		return m.TxFrames_65_127
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_128_255() uint64 {
-	if m != nil {
-		return m.TxFrames_128_255
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_256_511() uint64 {
-	if m != nil {
-		return m.TxFrames_256_511
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_512_1023() uint64 {
-	if m != nil {
-		return m.TxFrames_512_1023
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_1024_1518() uint64 {
-	if m != nil {
-		return m.TxFrames_1024_1518
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_1519_2047() uint64 {
-	if m != nil {
-		return m.TxFrames_1519_2047
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_2048_4095() uint64 {
-	if m != nil {
-		return m.TxFrames_2048_4095
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_4096_9216() uint64 {
-	if m != nil {
-		return m.TxFrames_4096_9216
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTxFrames_9217_16383() uint64 {
-	if m != nil {
-		return m.TxFrames_9217_16383
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetBipErrors() uint64 {
-	if m != nil {
-		return m.BipErrors
-	}
-	return 0
-}
-
-func (m *PortStatistics) GetTimestamp() uint32 {
-	if m != nil {
-		return m.Timestamp
-	}
-	return 0
-}
-
-type OnuStatistics struct {
-	IntfId                    uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                     uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	PositiveDrift             uint64   `protobuf:"fixed64,3,opt,name=positive_drift,json=positiveDrift,proto3" json:"positive_drift,omitempty"`
-	NegativeDrift             uint64   `protobuf:"fixed64,4,opt,name=negative_drift,json=negativeDrift,proto3" json:"negative_drift,omitempty"`
-	DelimiterMissDetection    uint64   `protobuf:"fixed64,5,opt,name=delimiter_miss_detection,json=delimiterMissDetection,proto3" json:"delimiter_miss_detection,omitempty"`
-	BipErrors                 uint64   `protobuf:"fixed64,6,opt,name=bip_errors,json=bipErrors,proto3" json:"bip_errors,omitempty"`
-	BipUnits                  uint64   `protobuf:"fixed64,7,opt,name=bip_units,json=bipUnits,proto3" json:"bip_units,omitempty"`
-	FecCorrectedSymbols       uint64   `protobuf:"fixed64,8,opt,name=fec_corrected_symbols,json=fecCorrectedSymbols,proto3" json:"fec_corrected_symbols,omitempty"`
-	FecCodewordsCorrected     uint64   `protobuf:"fixed64,9,opt,name=fec_codewords_corrected,json=fecCodewordsCorrected,proto3" json:"fec_codewords_corrected,omitempty"`
-	FecCodewordsUncorrectable uint64   `protobuf:"fixed64,10,opt,name=fec_codewords_uncorrectable,json=fecCodewordsUncorrectable,proto3" json:"fec_codewords_uncorrectable,omitempty"`
-	FecCodewords              uint64   `protobuf:"fixed64,11,opt,name=fec_codewords,json=fecCodewords,proto3" json:"fec_codewords,omitempty"`
-	FecCorrectedUnits         uint64   `protobuf:"fixed64,12,opt,name=fec_corrected_units,json=fecCorrectedUnits,proto3" json:"fec_corrected_units,omitempty"`
-	XgemKeyErrors             uint64   `protobuf:"fixed64,13,opt,name=xgem_key_errors,json=xgemKeyErrors,proto3" json:"xgem_key_errors,omitempty"`
-	XgemLoss                  uint64   `protobuf:"fixed64,14,opt,name=xgem_loss,json=xgemLoss,proto3" json:"xgem_loss,omitempty"`
-	RxPloamsError             uint64   `protobuf:"fixed64,15,opt,name=rx_ploams_error,json=rxPloamsError,proto3" json:"rx_ploams_error,omitempty"`
-	RxPloamsNonIdle           uint64   `protobuf:"fixed64,16,opt,name=rx_ploams_non_idle,json=rxPloamsNonIdle,proto3" json:"rx_ploams_non_idle,omitempty"`
-	RxOmci                    uint64   `protobuf:"fixed64,17,opt,name=rx_omci,json=rxOmci,proto3" json:"rx_omci,omitempty"`
-	RxOmciPacketsCrcError     uint64   `protobuf:"fixed64,18,opt,name=rx_omci_packets_crc_error,json=rxOmciPacketsCrcError,proto3" json:"rx_omci_packets_crc_error,omitempty"`
-	RxBytes                   uint64   `protobuf:"fixed64,19,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	RxPackets                 uint64   `protobuf:"fixed64,20,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	TxBytes                   uint64   `protobuf:"fixed64,21,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	TxPackets                 uint64   `protobuf:"fixed64,22,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	BerReported               uint64   `protobuf:"fixed64,23,opt,name=ber_reported,json=berReported,proto3" json:"ber_reported,omitempty"`
-	LcdgErrors                uint64   `protobuf:"fixed64,24,opt,name=lcdg_errors,json=lcdgErrors,proto3" json:"lcdg_errors,omitempty"`
-	RdiErrors                 uint64   `protobuf:"fixed64,25,opt,name=rdi_errors,json=rdiErrors,proto3" json:"rdi_errors,omitempty"`
-	Timestamp                 uint32   `protobuf:"fixed32,26,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral      struct{} `json:"-"`
-	XXX_unrecognized          []byte   `json:"-"`
-	XXX_sizecache             int32    `json:"-"`
-}
-
-func (m *OnuStatistics) Reset()         { *m = OnuStatistics{} }
-func (m *OnuStatistics) String() string { return proto.CompactTextString(m) }
-func (*OnuStatistics) ProtoMessage()    {}
-func (*OnuStatistics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{23}
-}
-
-func (m *OnuStatistics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuStatistics.Unmarshal(m, b)
-}
-func (m *OnuStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuStatistics.Marshal(b, m, deterministic)
-}
-func (m *OnuStatistics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuStatistics.Merge(m, src)
-}
-func (m *OnuStatistics) XXX_Size() int {
-	return xxx_messageInfo_OnuStatistics.Size(m)
-}
-func (m *OnuStatistics) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuStatistics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuStatistics proto.InternalMessageInfo
-
-func (m *OnuStatistics) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetPositiveDrift() uint64 {
-	if m != nil {
-		return m.PositiveDrift
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetNegativeDrift() uint64 {
-	if m != nil {
-		return m.NegativeDrift
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetDelimiterMissDetection() uint64 {
-	if m != nil {
-		return m.DelimiterMissDetection
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetBipErrors() uint64 {
-	if m != nil {
-		return m.BipErrors
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetBipUnits() uint64 {
-	if m != nil {
-		return m.BipUnits
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetFecCorrectedSymbols() uint64 {
-	if m != nil {
-		return m.FecCorrectedSymbols
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetFecCodewordsCorrected() uint64 {
-	if m != nil {
-		return m.FecCodewordsCorrected
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetFecCodewordsUncorrectable() uint64 {
-	if m != nil {
-		return m.FecCodewordsUncorrectable
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetFecCodewords() uint64 {
-	if m != nil {
-		return m.FecCodewords
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetFecCorrectedUnits() uint64 {
-	if m != nil {
-		return m.FecCorrectedUnits
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetXgemKeyErrors() uint64 {
-	if m != nil {
-		return m.XgemKeyErrors
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetXgemLoss() uint64 {
-	if m != nil {
-		return m.XgemLoss
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxPloamsError() uint64 {
-	if m != nil {
-		return m.RxPloamsError
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxPloamsNonIdle() uint64 {
-	if m != nil {
-		return m.RxPloamsNonIdle
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxOmci() uint64 {
-	if m != nil {
-		return m.RxOmci
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxOmciPacketsCrcError() uint64 {
-	if m != nil {
-		return m.RxOmciPacketsCrcError
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetBerReported() uint64 {
-	if m != nil {
-		return m.BerReported
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetLcdgErrors() uint64 {
-	if m != nil {
-		return m.LcdgErrors
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetRdiErrors() uint64 {
-	if m != nil {
-		return m.RdiErrors
-	}
-	return 0
-}
-
-func (m *OnuStatistics) GetTimestamp() uint32 {
-	if m != nil {
-		return m.Timestamp
-	}
-	return 0
-}
-
-type GemPortStatistics struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	GemportId            uint32   `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	RxBytes              uint64   `protobuf:"fixed64,4,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	TxPackets            uint64   `protobuf:"fixed64,5,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	TxBytes              uint64   `protobuf:"fixed64,6,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	Timestamp            uint32   `protobuf:"fixed32,26,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *GemPortStatistics) Reset()         { *m = GemPortStatistics{} }
-func (m *GemPortStatistics) String() string { return proto.CompactTextString(m) }
-func (*GemPortStatistics) ProtoMessage()    {}
-func (*GemPortStatistics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{24}
-}
-
-func (m *GemPortStatistics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GemPortStatistics.Unmarshal(m, b)
-}
-func (m *GemPortStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GemPortStatistics.Marshal(b, m, deterministic)
-}
-func (m *GemPortStatistics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GemPortStatistics.Merge(m, src)
-}
-func (m *GemPortStatistics) XXX_Size() int {
-	return xxx_messageInfo_GemPortStatistics.Size(m)
-}
-func (m *GemPortStatistics) XXX_DiscardUnknown() {
-	xxx_messageInfo_GemPortStatistics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GemPortStatistics proto.InternalMessageInfo
-
-func (m *GemPortStatistics) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *GemPortStatistics) GetTimestamp() uint32 {
-	if m != nil {
-		return m.Timestamp
-	}
-	return 0
-}
-
-type FlowStatistics struct {
-	FlowId               uint32   `protobuf:"fixed32,1,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
-	RxBytes              uint64   `protobuf:"fixed64,2,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	TxBytes              uint64   `protobuf:"fixed64,8,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	TxPackets            uint64   `protobuf:"fixed64,9,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	Timestamp            uint32   `protobuf:"fixed32,16,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *FlowStatistics) Reset()         { *m = FlowStatistics{} }
-func (m *FlowStatistics) String() string { return proto.CompactTextString(m) }
-func (*FlowStatistics) ProtoMessage()    {}
-func (*FlowStatistics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{25}
-}
-
-func (m *FlowStatistics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowStatistics.Unmarshal(m, b)
-}
-func (m *FlowStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowStatistics.Marshal(b, m, deterministic)
-}
-func (m *FlowStatistics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowStatistics.Merge(m, src)
-}
-func (m *FlowStatistics) XXX_Size() int {
-	return xxx_messageInfo_FlowStatistics.Size(m)
-}
-func (m *FlowStatistics) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowStatistics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowStatistics proto.InternalMessageInfo
-
-func (m *FlowStatistics) GetFlowId() uint32 {
-	if m != nil {
-		return m.FlowId
-	}
-	return 0
-}
-
-func (m *FlowStatistics) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *FlowStatistics) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *FlowStatistics) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *FlowStatistics) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *FlowStatistics) GetTimestamp() uint32 {
-	if m != nil {
-		return m.Timestamp
-	}
-	return 0
-}
-
-type LosIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	Status               string   `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *LosIndication) Reset()         { *m = LosIndication{} }
-func (m *LosIndication) String() string { return proto.CompactTextString(m) }
-func (*LosIndication) ProtoMessage()    {}
-func (*LosIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{26}
-}
-
-func (m *LosIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LosIndication.Unmarshal(m, b)
-}
-func (m *LosIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LosIndication.Marshal(b, m, deterministic)
-}
-func (m *LosIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LosIndication.Merge(m, src)
-}
-func (m *LosIndication) XXX_Size() int {
-	return xxx_messageInfo_LosIndication.Size(m)
-}
-func (m *LosIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_LosIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LosIndication proto.InternalMessageInfo
-
-func (m *LosIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *LosIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type DyingGaspIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DyingGaspIndication) Reset()         { *m = DyingGaspIndication{} }
-func (m *DyingGaspIndication) String() string { return proto.CompactTextString(m) }
-func (*DyingGaspIndication) ProtoMessage()    {}
-func (*DyingGaspIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{27}
-}
-
-func (m *DyingGaspIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DyingGaspIndication.Unmarshal(m, b)
-}
-func (m *DyingGaspIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DyingGaspIndication.Marshal(b, m, deterministic)
-}
-func (m *DyingGaspIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DyingGaspIndication.Merge(m, src)
-}
-func (m *DyingGaspIndication) XXX_Size() int {
-	return xxx_messageInfo_DyingGaspIndication.Size(m)
-}
-func (m *DyingGaspIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_DyingGaspIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DyingGaspIndication proto.InternalMessageInfo
-
-func (m *DyingGaspIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *DyingGaspIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *DyingGaspIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuAlarmIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	LosStatus            string   `protobuf:"bytes,3,opt,name=los_status,json=losStatus,proto3" json:"los_status,omitempty"`
-	LobStatus            string   `protobuf:"bytes,4,opt,name=lob_status,json=lobStatus,proto3" json:"lob_status,omitempty"`
-	LopcMissStatus       string   `protobuf:"bytes,5,opt,name=lopc_miss_status,json=lopcMissStatus,proto3" json:"lopc_miss_status,omitempty"`
-	LopcMicErrorStatus   string   `protobuf:"bytes,6,opt,name=lopc_mic_error_status,json=lopcMicErrorStatus,proto3" json:"lopc_mic_error_status,omitempty"`
-	LofiStatus           string   `protobuf:"bytes,7,opt,name=lofi_status,json=lofiStatus,proto3" json:"lofi_status,omitempty"`
-	LoamiStatus          string   `protobuf:"bytes,8,opt,name=loami_status,json=loamiStatus,proto3" json:"loami_status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuAlarmIndication) Reset()         { *m = OnuAlarmIndication{} }
-func (m *OnuAlarmIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuAlarmIndication) ProtoMessage()    {}
-func (*OnuAlarmIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{28}
-}
-
-func (m *OnuAlarmIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuAlarmIndication.Unmarshal(m, b)
-}
-func (m *OnuAlarmIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuAlarmIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuAlarmIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuAlarmIndication.Merge(m, src)
-}
-func (m *OnuAlarmIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuAlarmIndication.Size(m)
-}
-func (m *OnuAlarmIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuAlarmIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuAlarmIndication proto.InternalMessageInfo
-
-func (m *OnuAlarmIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuAlarmIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuAlarmIndication) GetLosStatus() string {
-	if m != nil {
-		return m.LosStatus
-	}
-	return ""
-}
-
-func (m *OnuAlarmIndication) GetLobStatus() string {
-	if m != nil {
-		return m.LobStatus
-	}
-	return ""
-}
-
-func (m *OnuAlarmIndication) GetLopcMissStatus() string {
-	if m != nil {
-		return m.LopcMissStatus
-	}
-	return ""
-}
-
-func (m *OnuAlarmIndication) GetLopcMicErrorStatus() string {
-	if m != nil {
-		return m.LopcMicErrorStatus
-	}
-	return ""
-}
-
-func (m *OnuAlarmIndication) GetLofiStatus() string {
-	if m != nil {
-		return m.LofiStatus
-	}
-	return ""
-}
-
-func (m *OnuAlarmIndication) GetLoamiStatus() string {
-	if m != nil {
-		return m.LoamiStatus
-	}
-	return ""
-}
-
-type OnuStartupFailureIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuStartupFailureIndication) Reset()         { *m = OnuStartupFailureIndication{} }
-func (m *OnuStartupFailureIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuStartupFailureIndication) ProtoMessage()    {}
-func (*OnuStartupFailureIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{29}
-}
-
-func (m *OnuStartupFailureIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuStartupFailureIndication.Unmarshal(m, b)
-}
-func (m *OnuStartupFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuStartupFailureIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuStartupFailureIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuStartupFailureIndication.Merge(m, src)
-}
-func (m *OnuStartupFailureIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuStartupFailureIndication.Size(m)
-}
-func (m *OnuStartupFailureIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuStartupFailureIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuStartupFailureIndication proto.InternalMessageInfo
-
-func (m *OnuStartupFailureIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuStartupFailureIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuStartupFailureIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuSignalDegradeIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	InverseBitErrorRate  uint32   `protobuf:"fixed32,4,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuSignalDegradeIndication) Reset()         { *m = OnuSignalDegradeIndication{} }
-func (m *OnuSignalDegradeIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuSignalDegradeIndication) ProtoMessage()    {}
-func (*OnuSignalDegradeIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{30}
-}
-
-func (m *OnuSignalDegradeIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuSignalDegradeIndication.Unmarshal(m, b)
-}
-func (m *OnuSignalDegradeIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuSignalDegradeIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuSignalDegradeIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuSignalDegradeIndication.Merge(m, src)
-}
-func (m *OnuSignalDegradeIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuSignalDegradeIndication.Size(m)
-}
-func (m *OnuSignalDegradeIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuSignalDegradeIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuSignalDegradeIndication proto.InternalMessageInfo
-
-func (m *OnuSignalDegradeIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuSignalDegradeIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuSignalDegradeIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuSignalDegradeIndication) GetInverseBitErrorRate() uint32 {
-	if m != nil {
-		return m.InverseBitErrorRate
-	}
-	return 0
-}
-
-type OnuDriftOfWindowIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	Drift                uint32   `protobuf:"fixed32,4,opt,name=drift,proto3" json:"drift,omitempty"`
-	NewEqd               uint32   `protobuf:"fixed32,5,opt,name=new_eqd,json=newEqd,proto3" json:"new_eqd,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuDriftOfWindowIndication) Reset()         { *m = OnuDriftOfWindowIndication{} }
-func (m *OnuDriftOfWindowIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuDriftOfWindowIndication) ProtoMessage()    {}
-func (*OnuDriftOfWindowIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{31}
-}
-
-func (m *OnuDriftOfWindowIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuDriftOfWindowIndication.Unmarshal(m, b)
-}
-func (m *OnuDriftOfWindowIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuDriftOfWindowIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuDriftOfWindowIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuDriftOfWindowIndication.Merge(m, src)
-}
-func (m *OnuDriftOfWindowIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuDriftOfWindowIndication.Size(m)
-}
-func (m *OnuDriftOfWindowIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuDriftOfWindowIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuDriftOfWindowIndication proto.InternalMessageInfo
-
-func (m *OnuDriftOfWindowIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuDriftOfWindowIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuDriftOfWindowIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuDriftOfWindowIndication) GetDrift() uint32 {
-	if m != nil {
-		return m.Drift
-	}
-	return 0
-}
-
-func (m *OnuDriftOfWindowIndication) GetNewEqd() uint32 {
-	if m != nil {
-		return m.NewEqd
-	}
-	return 0
-}
-
-type OnuLossOfOmciChannelIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuLossOfOmciChannelIndication) Reset()         { *m = OnuLossOfOmciChannelIndication{} }
-func (m *OnuLossOfOmciChannelIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuLossOfOmciChannelIndication) ProtoMessage()    {}
-func (*OnuLossOfOmciChannelIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{32}
-}
-
-func (m *OnuLossOfOmciChannelIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Unmarshal(m, b)
-}
-func (m *OnuLossOfOmciChannelIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuLossOfOmciChannelIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuLossOfOmciChannelIndication.Merge(m, src)
-}
-func (m *OnuLossOfOmciChannelIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Size(m)
-}
-func (m *OnuLossOfOmciChannelIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuLossOfOmciChannelIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuLossOfOmciChannelIndication proto.InternalMessageInfo
-
-func (m *OnuLossOfOmciChannelIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuLossOfOmciChannelIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuLossOfOmciChannelIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuSignalsFailureIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	InverseBitErrorRate  uint32   `protobuf:"fixed32,4,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuSignalsFailureIndication) Reset()         { *m = OnuSignalsFailureIndication{} }
-func (m *OnuSignalsFailureIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuSignalsFailureIndication) ProtoMessage()    {}
-func (*OnuSignalsFailureIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{33}
-}
-
-func (m *OnuSignalsFailureIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuSignalsFailureIndication.Unmarshal(m, b)
-}
-func (m *OnuSignalsFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuSignalsFailureIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuSignalsFailureIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuSignalsFailureIndication.Merge(m, src)
-}
-func (m *OnuSignalsFailureIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuSignalsFailureIndication.Size(m)
-}
-func (m *OnuSignalsFailureIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuSignalsFailureIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuSignalsFailureIndication proto.InternalMessageInfo
-
-func (m *OnuSignalsFailureIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuSignalsFailureIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuSignalsFailureIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuSignalsFailureIndication) GetInverseBitErrorRate() uint32 {
-	if m != nil {
-		return m.InverseBitErrorRate
-	}
-	return 0
-}
-
-type OnuTransmissionInterferenceWarning struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	Drift                uint32   `protobuf:"fixed32,4,opt,name=drift,proto3" json:"drift,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuTransmissionInterferenceWarning) Reset()         { *m = OnuTransmissionInterferenceWarning{} }
-func (m *OnuTransmissionInterferenceWarning) String() string { return proto.CompactTextString(m) }
-func (*OnuTransmissionInterferenceWarning) ProtoMessage()    {}
-func (*OnuTransmissionInterferenceWarning) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{34}
-}
-
-func (m *OnuTransmissionInterferenceWarning) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Unmarshal(m, b)
-}
-func (m *OnuTransmissionInterferenceWarning) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Marshal(b, m, deterministic)
-}
-func (m *OnuTransmissionInterferenceWarning) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuTransmissionInterferenceWarning.Merge(m, src)
-}
-func (m *OnuTransmissionInterferenceWarning) XXX_Size() int {
-	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Size(m)
-}
-func (m *OnuTransmissionInterferenceWarning) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuTransmissionInterferenceWarning.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuTransmissionInterferenceWarning proto.InternalMessageInfo
-
-func (m *OnuTransmissionInterferenceWarning) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuTransmissionInterferenceWarning) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuTransmissionInterferenceWarning) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuTransmissionInterferenceWarning) GetDrift() uint32 {
-	if m != nil {
-		return m.Drift
-	}
-	return 0
-}
-
-type OnuActivationFailureIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	FailReason           uint32   `protobuf:"fixed32,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuActivationFailureIndication) Reset()         { *m = OnuActivationFailureIndication{} }
-func (m *OnuActivationFailureIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuActivationFailureIndication) ProtoMessage()    {}
-func (*OnuActivationFailureIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{35}
-}
-
-func (m *OnuActivationFailureIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuActivationFailureIndication.Unmarshal(m, b)
-}
-func (m *OnuActivationFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuActivationFailureIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuActivationFailureIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuActivationFailureIndication.Merge(m, src)
-}
-func (m *OnuActivationFailureIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuActivationFailureIndication.Size(m)
-}
-func (m *OnuActivationFailureIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuActivationFailureIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuActivationFailureIndication proto.InternalMessageInfo
-
-func (m *OnuActivationFailureIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuActivationFailureIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuActivationFailureIndication) GetFailReason() uint32 {
-	if m != nil {
-		return m.FailReason
-	}
-	return 0
-}
-
-type OnuLossOfKeySyncFailureIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuLossOfKeySyncFailureIndication) Reset()         { *m = OnuLossOfKeySyncFailureIndication{} }
-func (m *OnuLossOfKeySyncFailureIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuLossOfKeySyncFailureIndication) ProtoMessage()    {}
-func (*OnuLossOfKeySyncFailureIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{36}
-}
-
-func (m *OnuLossOfKeySyncFailureIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Unmarshal(m, b)
-}
-func (m *OnuLossOfKeySyncFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuLossOfKeySyncFailureIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Merge(m, src)
-}
-func (m *OnuLossOfKeySyncFailureIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Size(m)
-}
-func (m *OnuLossOfKeySyncFailureIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuLossOfKeySyncFailureIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuLossOfKeySyncFailureIndication proto.InternalMessageInfo
-
-func (m *OnuLossOfKeySyncFailureIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuLossOfKeySyncFailureIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuLossOfKeySyncFailureIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type RdiErrorIndication struct {
-	RdiErrorCount        uint64   `protobuf:"fixed64,1,opt,name=rdi_error_count,json=rdiErrorCount,proto3" json:"rdi_error_count,omitempty"`
-	Status               string   `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *RdiErrorIndication) Reset()         { *m = RdiErrorIndication{} }
-func (m *RdiErrorIndication) String() string { return proto.CompactTextString(m) }
-func (*RdiErrorIndication) ProtoMessage()    {}
-func (*RdiErrorIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{37}
-}
-
-func (m *RdiErrorIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RdiErrorIndication.Unmarshal(m, b)
-}
-func (m *RdiErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RdiErrorIndication.Marshal(b, m, deterministic)
-}
-func (m *RdiErrorIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RdiErrorIndication.Merge(m, src)
-}
-func (m *RdiErrorIndication) XXX_Size() int {
-	return xxx_messageInfo_RdiErrorIndication.Size(m)
-}
-func (m *RdiErrorIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_RdiErrorIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RdiErrorIndication proto.InternalMessageInfo
-
-func (m *RdiErrorIndication) GetRdiErrorCount() uint64 {
-	if m != nil {
-		return m.RdiErrorCount
-	}
-	return 0
-}
-
-func (m *RdiErrorIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuItuPonStatsIndication struct {
-	IntfId uint32 `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId  uint32 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	// Types that are valid to be assigned to Stats:
-	//	*OnuItuPonStatsIndication_RdiErrorInd
-	Stats                isOnuItuPonStatsIndication_Stats `protobuf_oneof:"stats"`
-	XXX_NoUnkeyedLiteral struct{}                         `json:"-"`
-	XXX_unrecognized     []byte                           `json:"-"`
-	XXX_sizecache        int32                            `json:"-"`
-}
-
-func (m *OnuItuPonStatsIndication) Reset()         { *m = OnuItuPonStatsIndication{} }
-func (m *OnuItuPonStatsIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuItuPonStatsIndication) ProtoMessage()    {}
-func (*OnuItuPonStatsIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{38}
-}
-
-func (m *OnuItuPonStatsIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuItuPonStatsIndication.Unmarshal(m, b)
-}
-func (m *OnuItuPonStatsIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuItuPonStatsIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuItuPonStatsIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuItuPonStatsIndication.Merge(m, src)
-}
-func (m *OnuItuPonStatsIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuItuPonStatsIndication.Size(m)
-}
-func (m *OnuItuPonStatsIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuItuPonStatsIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuItuPonStatsIndication proto.InternalMessageInfo
-
-func (m *OnuItuPonStatsIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuItuPonStatsIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-type isOnuItuPonStatsIndication_Stats interface {
-	isOnuItuPonStatsIndication_Stats()
-}
-
-type OnuItuPonStatsIndication_RdiErrorInd struct {
-	RdiErrorInd *RdiErrorIndication `protobuf:"bytes,3,opt,name=rdi_error_ind,json=rdiErrorInd,proto3,oneof"`
-}
-
-func (*OnuItuPonStatsIndication_RdiErrorInd) isOnuItuPonStatsIndication_Stats() {}
-
-func (m *OnuItuPonStatsIndication) GetStats() isOnuItuPonStatsIndication_Stats {
-	if m != nil {
-		return m.Stats
-	}
-	return nil
-}
-
-func (m *OnuItuPonStatsIndication) GetRdiErrorInd() *RdiErrorIndication {
-	if x, ok := m.GetStats().(*OnuItuPonStatsIndication_RdiErrorInd); ok {
-		return x.RdiErrorInd
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*OnuItuPonStatsIndication) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*OnuItuPonStatsIndication_RdiErrorInd)(nil),
-	}
-}
-
-type OnuProcessingErrorIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuProcessingErrorIndication) Reset()         { *m = OnuProcessingErrorIndication{} }
-func (m *OnuProcessingErrorIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuProcessingErrorIndication) ProtoMessage()    {}
-func (*OnuProcessingErrorIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{39}
-}
-
-func (m *OnuProcessingErrorIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuProcessingErrorIndication.Unmarshal(m, b)
-}
-func (m *OnuProcessingErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuProcessingErrorIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuProcessingErrorIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuProcessingErrorIndication.Merge(m, src)
-}
-func (m *OnuProcessingErrorIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuProcessingErrorIndication.Size(m)
-}
-func (m *OnuProcessingErrorIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuProcessingErrorIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuProcessingErrorIndication proto.InternalMessageInfo
-
-func (m *OnuProcessingErrorIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuProcessingErrorIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-type OnuDeactivationFailureIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuDeactivationFailureIndication) Reset()         { *m = OnuDeactivationFailureIndication{} }
-func (m *OnuDeactivationFailureIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuDeactivationFailureIndication) ProtoMessage()    {}
-func (*OnuDeactivationFailureIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{40}
-}
-
-func (m *OnuDeactivationFailureIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuDeactivationFailureIndication.Unmarshal(m, b)
-}
-func (m *OnuDeactivationFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuDeactivationFailureIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuDeactivationFailureIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuDeactivationFailureIndication.Merge(m, src)
-}
-func (m *OnuDeactivationFailureIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuDeactivationFailureIndication.Size(m)
-}
-func (m *OnuDeactivationFailureIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuDeactivationFailureIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuDeactivationFailureIndication proto.InternalMessageInfo
-
-func (m *OnuDeactivationFailureIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuDeactivationFailureIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuDeactivationFailureIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuRemoteDefectIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	RdiErrors            uint64   `protobuf:"fixed64,3,opt,name=rdi_errors,json=rdiErrors,proto3" json:"rdi_errors,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuRemoteDefectIndication) Reset()         { *m = OnuRemoteDefectIndication{} }
-func (m *OnuRemoteDefectIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuRemoteDefectIndication) ProtoMessage()    {}
-func (*OnuRemoteDefectIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{41}
-}
-
-func (m *OnuRemoteDefectIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuRemoteDefectIndication.Unmarshal(m, b)
-}
-func (m *OnuRemoteDefectIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuRemoteDefectIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuRemoteDefectIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuRemoteDefectIndication.Merge(m, src)
-}
-func (m *OnuRemoteDefectIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuRemoteDefectIndication.Size(m)
-}
-func (m *OnuRemoteDefectIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuRemoteDefectIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuRemoteDefectIndication proto.InternalMessageInfo
-
-func (m *OnuRemoteDefectIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuRemoteDefectIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuRemoteDefectIndication) GetRdiErrors() uint64 {
-	if m != nil {
-		return m.RdiErrors
-	}
-	return 0
-}
-
-type OnuLossOfGEMChannelDelineationIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	DelineationErrors    uint32   `protobuf:"fixed32,4,opt,name=delineation_errors,json=delineationErrors,proto3" json:"delineation_errors,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuLossOfGEMChannelDelineationIndication) Reset() {
-	*m = OnuLossOfGEMChannelDelineationIndication{}
-}
-func (m *OnuLossOfGEMChannelDelineationIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuLossOfGEMChannelDelineationIndication) ProtoMessage()    {}
-func (*OnuLossOfGEMChannelDelineationIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{42}
-}
-
-func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Unmarshal(m, b)
-}
-func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Merge(m, src)
-}
-func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Size(m)
-}
-func (m *OnuLossOfGEMChannelDelineationIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication proto.InternalMessageInfo
-
-func (m *OnuLossOfGEMChannelDelineationIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuLossOfGEMChannelDelineationIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuLossOfGEMChannelDelineationIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuLossOfGEMChannelDelineationIndication) GetDelineationErrors() uint32 {
-	if m != nil {
-		return m.DelineationErrors
-	}
-	return 0
-}
-
-type OnuPhysicalEquipmentErrorIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuPhysicalEquipmentErrorIndication) Reset()         { *m = OnuPhysicalEquipmentErrorIndication{} }
-func (m *OnuPhysicalEquipmentErrorIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuPhysicalEquipmentErrorIndication) ProtoMessage()    {}
-func (*OnuPhysicalEquipmentErrorIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{43}
-}
-
-func (m *OnuPhysicalEquipmentErrorIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Unmarshal(m, b)
-}
-func (m *OnuPhysicalEquipmentErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuPhysicalEquipmentErrorIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Merge(m, src)
-}
-func (m *OnuPhysicalEquipmentErrorIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Size(m)
-}
-func (m *OnuPhysicalEquipmentErrorIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuPhysicalEquipmentErrorIndication proto.InternalMessageInfo
-
-func (m *OnuPhysicalEquipmentErrorIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuPhysicalEquipmentErrorIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuPhysicalEquipmentErrorIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuLossOfAcknowledgementIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuLossOfAcknowledgementIndication) Reset()         { *m = OnuLossOfAcknowledgementIndication{} }
-func (m *OnuLossOfAcknowledgementIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuLossOfAcknowledgementIndication) ProtoMessage()    {}
-func (*OnuLossOfAcknowledgementIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{44}
-}
-
-func (m *OnuLossOfAcknowledgementIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Unmarshal(m, b)
-}
-func (m *OnuLossOfAcknowledgementIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuLossOfAcknowledgementIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuLossOfAcknowledgementIndication.Merge(m, src)
-}
-func (m *OnuLossOfAcknowledgementIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Size(m)
-}
-func (m *OnuLossOfAcknowledgementIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuLossOfAcknowledgementIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuLossOfAcknowledgementIndication proto.InternalMessageInfo
-
-func (m *OnuLossOfAcknowledgementIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuLossOfAcknowledgementIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuLossOfAcknowledgementIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-type OnuDifferentialReachExceededIndication struct {
-	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	Distance             uint32   `protobuf:"fixed32,4,opt,name=distance,proto3" json:"distance,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuDifferentialReachExceededIndication) Reset() {
-	*m = OnuDifferentialReachExceededIndication{}
-}
-func (m *OnuDifferentialReachExceededIndication) String() string { return proto.CompactTextString(m) }
-func (*OnuDifferentialReachExceededIndication) ProtoMessage()    {}
-func (*OnuDifferentialReachExceededIndication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{45}
-}
-
-func (m *OnuDifferentialReachExceededIndication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Unmarshal(m, b)
-}
-func (m *OnuDifferentialReachExceededIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Marshal(b, m, deterministic)
-}
-func (m *OnuDifferentialReachExceededIndication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuDifferentialReachExceededIndication.Merge(m, src)
-}
-func (m *OnuDifferentialReachExceededIndication) XXX_Size() int {
-	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Size(m)
-}
-func (m *OnuDifferentialReachExceededIndication) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuDifferentialReachExceededIndication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuDifferentialReachExceededIndication proto.InternalMessageInfo
-
-func (m *OnuDifferentialReachExceededIndication) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *OnuDifferentialReachExceededIndication) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *OnuDifferentialReachExceededIndication) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *OnuDifferentialReachExceededIndication) GetDistance() uint32 {
-	if m != nil {
-		return m.Distance
-	}
-	return 0
-}
-
-type GroupMember struct {
-	InterfaceId          uint32                    `protobuf:"varint,1,opt,name=interface_id,json=interfaceId,proto3" json:"interface_id,omitempty"`
-	InterfaceType        GroupMember_InterfaceType `protobuf:"varint,2,opt,name=interface_type,json=interfaceType,proto3,enum=openolt.GroupMember_InterfaceType" json:"interface_type,omitempty"`
-	GemPortId            uint32                    `protobuf:"varint,3,opt,name=gem_port_id,json=gemPortId,proto3" json:"gem_port_id,omitempty"`
-	Priority             uint32                    `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *GroupMember) Reset()         { *m = GroupMember{} }
-func (m *GroupMember) String() string { return proto.CompactTextString(m) }
-func (*GroupMember) ProtoMessage()    {}
-func (*GroupMember) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{46}
-}
-
-func (m *GroupMember) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GroupMember.Unmarshal(m, b)
-}
-func (m *GroupMember) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GroupMember.Marshal(b, m, deterministic)
-}
-func (m *GroupMember) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GroupMember.Merge(m, src)
-}
-func (m *GroupMember) XXX_Size() int {
-	return xxx_messageInfo_GroupMember.Size(m)
-}
-func (m *GroupMember) XXX_DiscardUnknown() {
-	xxx_messageInfo_GroupMember.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GroupMember proto.InternalMessageInfo
-
-func (m *GroupMember) GetInterfaceId() uint32 {
-	if m != nil {
-		return m.InterfaceId
-	}
-	return 0
-}
-
-func (m *GroupMember) GetInterfaceType() GroupMember_InterfaceType {
-	if m != nil {
-		return m.InterfaceType
-	}
-	return GroupMember_PON
-}
-
-func (m *GroupMember) GetGemPortId() uint32 {
-	if m != nil {
-		return m.GemPortId
-	}
-	return 0
-}
-
-func (m *GroupMember) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-type Group struct {
-	GroupId              uint32                    `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
-	Command              Group_GroupMembersCommand `protobuf:"varint,2,opt,name=command,proto3,enum=openolt.Group_GroupMembersCommand" json:"command,omitempty"`
-	Members              []*GroupMember            `protobuf:"bytes,3,rep,name=members,proto3" json:"members,omitempty"`
-	Action               *Action                   `protobuf:"bytes,4,opt,name=action,proto3" json:"action,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *Group) Reset()         { *m = Group{} }
-func (m *Group) String() string { return proto.CompactTextString(m) }
-func (*Group) ProtoMessage()    {}
-func (*Group) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{47}
-}
-
-func (m *Group) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Group.Unmarshal(m, b)
-}
-func (m *Group) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Group.Marshal(b, m, deterministic)
-}
-func (m *Group) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Group.Merge(m, src)
-}
-func (m *Group) XXX_Size() int {
-	return xxx_messageInfo_Group.Size(m)
-}
-func (m *Group) XXX_DiscardUnknown() {
-	xxx_messageInfo_Group.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Group proto.InternalMessageInfo
-
-func (m *Group) GetGroupId() uint32 {
-	if m != nil {
-		return m.GroupId
-	}
-	return 0
-}
-
-func (m *Group) GetCommand() Group_GroupMembersCommand {
-	if m != nil {
-		return m.Command
-	}
-	return Group_ADD_MEMBERS
-}
-
-func (m *Group) GetMembers() []*GroupMember {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-func (m *Group) GetAction() *Action {
-	if m != nil {
-		return m.Action
-	}
-	return nil
-}
-
-type ValueParam struct {
-	Onu                  *Onu                  `protobuf:"bytes,1,opt,name=onu,proto3" json:"onu,omitempty"`
-	Value                common.ValueType_Type `protobuf:"varint,2,opt,name=value,proto3,enum=common.ValueType_Type" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *ValueParam) Reset()         { *m = ValueParam{} }
-func (m *ValueParam) String() string { return proto.CompactTextString(m) }
-func (*ValueParam) ProtoMessage()    {}
-func (*ValueParam) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{48}
-}
-
-func (m *ValueParam) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ValueParam.Unmarshal(m, b)
-}
-func (m *ValueParam) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ValueParam.Marshal(b, m, deterministic)
-}
-func (m *ValueParam) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueParam.Merge(m, src)
-}
-func (m *ValueParam) XXX_Size() int {
-	return xxx_messageInfo_ValueParam.Size(m)
-}
-func (m *ValueParam) XXX_DiscardUnknown() {
-	xxx_messageInfo_ValueParam.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ValueParam proto.InternalMessageInfo
-
-func (m *ValueParam) GetOnu() *Onu {
-	if m != nil {
-		return m.Onu
-	}
-	return nil
-}
-
-func (m *ValueParam) GetValue() common.ValueType_Type {
-	if m != nil {
-		return m.Value
-	}
-	return common.ValueType_EMPTY
-}
-
-type PonRxPowerData struct {
-	IntfId               uint32                                   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32                                   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	Status               string                                   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	FailReason           PonRxPowerData_RssiMeasurementFailReason `protobuf:"varint,4,opt,name=fail_reason,json=failReason,proto3,enum=openolt.PonRxPowerData_RssiMeasurementFailReason" json:"fail_reason,omitempty"`
-	RxPowerMeanDbm       float64                                  `protobuf:"fixed64,5,opt,name=rx_power_mean_dbm,json=rxPowerMeanDbm,proto3" json:"rx_power_mean_dbm,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                 `json:"-"`
-	XXX_unrecognized     []byte                                   `json:"-"`
-	XXX_sizecache        int32                                    `json:"-"`
-}
-
-func (m *PonRxPowerData) Reset()         { *m = PonRxPowerData{} }
-func (m *PonRxPowerData) String() string { return proto.CompactTextString(m) }
-func (*PonRxPowerData) ProtoMessage()    {}
-func (*PonRxPowerData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{49}
-}
-
-func (m *PonRxPowerData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonRxPowerData.Unmarshal(m, b)
-}
-func (m *PonRxPowerData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonRxPowerData.Marshal(b, m, deterministic)
-}
-func (m *PonRxPowerData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonRxPowerData.Merge(m, src)
-}
-func (m *PonRxPowerData) XXX_Size() int {
-	return xxx_messageInfo_PonRxPowerData.Size(m)
-}
-func (m *PonRxPowerData) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonRxPowerData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonRxPowerData proto.InternalMessageInfo
-
-func (m *PonRxPowerData) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *PonRxPowerData) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *PonRxPowerData) GetStatus() string {
-	if m != nil {
-		return m.Status
-	}
-	return ""
-}
-
-func (m *PonRxPowerData) GetFailReason() PonRxPowerData_RssiMeasurementFailReason {
-	if m != nil {
-		return m.FailReason
-	}
-	return PonRxPowerData_FAIL_REASON_NONE
-}
-
-func (m *PonRxPowerData) GetRxPowerMeanDbm() float64 {
-	if m != nil {
-		return m.RxPowerMeanDbm
-	}
-	return 0
-}
-
-type Empty struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Empty) Reset()         { *m = Empty{} }
-func (m *Empty) String() string { return proto.CompactTextString(m) }
-func (*Empty) ProtoMessage()    {}
-func (*Empty) Descriptor() ([]byte, []int) {
-	return fileDescriptor_c072e7aa0dfd74d5, []int{50}
-}
-
-func (m *Empty) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Empty.Unmarshal(m, b)
-}
-func (m *Empty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Empty.Marshal(b, m, deterministic)
-}
-func (m *Empty) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Empty.Merge(m, src)
-}
-func (m *Empty) XXX_Size() int {
-	return xxx_messageInfo_Empty.Size(m)
-}
-func (m *Empty) XXX_DiscardUnknown() {
-	xxx_messageInfo_Empty.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Empty proto.InternalMessageInfo
-
-func init() {
-	proto.RegisterEnum("openolt.OnuIndication_ActivationFailReason", OnuIndication_ActivationFailReason_name, OnuIndication_ActivationFailReason_value)
-	proto.RegisterEnum("openolt.DeviceInfo_DeviceResourceRanges_Pool_PoolType", DeviceInfo_DeviceResourceRanges_Pool_PoolType_name, DeviceInfo_DeviceResourceRanges_Pool_PoolType_value)
-	proto.RegisterEnum("openolt.DeviceInfo_DeviceResourceRanges_Pool_SharingType", DeviceInfo_DeviceResourceRanges_Pool_SharingType_name, DeviceInfo_DeviceResourceRanges_Pool_SharingType_value)
-	proto.RegisterEnum("openolt.GroupMember_InterfaceType", GroupMember_InterfaceType_name, GroupMember_InterfaceType_value)
-	proto.RegisterEnum("openolt.Group_GroupMembersCommand", Group_GroupMembersCommand_name, Group_GroupMembersCommand_value)
-	proto.RegisterEnum("openolt.PonRxPowerData_RssiMeasurementFailReason", PonRxPowerData_RssiMeasurementFailReason_name, PonRxPowerData_RssiMeasurementFailReason_value)
-	proto.RegisterType((*Indication)(nil), "openolt.Indication")
-	proto.RegisterType((*AlarmIndication)(nil), "openolt.AlarmIndication")
-	proto.RegisterType((*OltIndication)(nil), "openolt.OltIndication")
-	proto.RegisterType((*IntfIndication)(nil), "openolt.IntfIndication")
-	proto.RegisterType((*OnuDiscIndication)(nil), "openolt.OnuDiscIndication")
-	proto.RegisterType((*OnuIndication)(nil), "openolt.OnuIndication")
-	proto.RegisterType((*IntfOperIndication)(nil), "openolt.IntfOperIndication")
-	proto.RegisterType((*OmciIndication)(nil), "openolt.OmciIndication")
-	proto.RegisterType((*PacketIndication)(nil), "openolt.PacketIndication")
-	proto.RegisterType((*Interface)(nil), "openolt.Interface")
-	proto.RegisterType((*Heartbeat)(nil), "openolt.Heartbeat")
-	proto.RegisterType((*Onu)(nil), "openolt.Onu")
-	proto.RegisterType((*OnuLogicalDistance)(nil), "openolt.OnuLogicalDistance")
-	proto.RegisterType((*OmciMsg)(nil), "openolt.OmciMsg")
-	proto.RegisterType((*OnuPacket)(nil), "openolt.OnuPacket")
-	proto.RegisterType((*UplinkPacket)(nil), "openolt.UplinkPacket")
-	proto.RegisterType((*DeviceInfo)(nil), "openolt.DeviceInfo")
-	proto.RegisterType((*DeviceInfo_DeviceResourceRanges)(nil), "openolt.DeviceInfo.DeviceResourceRanges")
-	proto.RegisterType((*DeviceInfo_DeviceResourceRanges_Pool)(nil), "openolt.DeviceInfo.DeviceResourceRanges.Pool")
-	proto.RegisterType((*Classifier)(nil), "openolt.Classifier")
-	proto.RegisterType((*ActionCmd)(nil), "openolt.ActionCmd")
-	proto.RegisterType((*Action)(nil), "openolt.Action")
-	proto.RegisterType((*Flow)(nil), "openolt.Flow")
-	proto.RegisterMapType((map[uint32]bool)(nil), "openolt.Flow.GemportToAesEntry")
-	proto.RegisterMapType((map[uint32]uint32)(nil), "openolt.Flow.PbitToGemportEntry")
-	proto.RegisterType((*SerialNumber)(nil), "openolt.SerialNumber")
-	proto.RegisterType((*PortStatistics)(nil), "openolt.PortStatistics")
-	proto.RegisterType((*OnuStatistics)(nil), "openolt.OnuStatistics")
-	proto.RegisterType((*GemPortStatistics)(nil), "openolt.GemPortStatistics")
-	proto.RegisterType((*FlowStatistics)(nil), "openolt.FlowStatistics")
-	proto.RegisterType((*LosIndication)(nil), "openolt.LosIndication")
-	proto.RegisterType((*DyingGaspIndication)(nil), "openolt.DyingGaspIndication")
-	proto.RegisterType((*OnuAlarmIndication)(nil), "openolt.OnuAlarmIndication")
-	proto.RegisterType((*OnuStartupFailureIndication)(nil), "openolt.OnuStartupFailureIndication")
-	proto.RegisterType((*OnuSignalDegradeIndication)(nil), "openolt.OnuSignalDegradeIndication")
-	proto.RegisterType((*OnuDriftOfWindowIndication)(nil), "openolt.OnuDriftOfWindowIndication")
-	proto.RegisterType((*OnuLossOfOmciChannelIndication)(nil), "openolt.OnuLossOfOmciChannelIndication")
-	proto.RegisterType((*OnuSignalsFailureIndication)(nil), "openolt.OnuSignalsFailureIndication")
-	proto.RegisterType((*OnuTransmissionInterferenceWarning)(nil), "openolt.OnuTransmissionInterferenceWarning")
-	proto.RegisterType((*OnuActivationFailureIndication)(nil), "openolt.OnuActivationFailureIndication")
-	proto.RegisterType((*OnuLossOfKeySyncFailureIndication)(nil), "openolt.OnuLossOfKeySyncFailureIndication")
-	proto.RegisterType((*RdiErrorIndication)(nil), "openolt.RdiErrorIndication")
-	proto.RegisterType((*OnuItuPonStatsIndication)(nil), "openolt.OnuItuPonStatsIndication")
-	proto.RegisterType((*OnuProcessingErrorIndication)(nil), "openolt.OnuProcessingErrorIndication")
-	proto.RegisterType((*OnuDeactivationFailureIndication)(nil), "openolt.OnuDeactivationFailureIndication")
-	proto.RegisterType((*OnuRemoteDefectIndication)(nil), "openolt.OnuRemoteDefectIndication")
-	proto.RegisterType((*OnuLossOfGEMChannelDelineationIndication)(nil), "openolt.OnuLossOfGEMChannelDelineationIndication")
-	proto.RegisterType((*OnuPhysicalEquipmentErrorIndication)(nil), "openolt.OnuPhysicalEquipmentErrorIndication")
-	proto.RegisterType((*OnuLossOfAcknowledgementIndication)(nil), "openolt.OnuLossOfAcknowledgementIndication")
-	proto.RegisterType((*OnuDifferentialReachExceededIndication)(nil), "openolt.OnuDifferentialReachExceededIndication")
-	proto.RegisterType((*GroupMember)(nil), "openolt.GroupMember")
-	proto.RegisterType((*Group)(nil), "openolt.Group")
-	proto.RegisterType((*ValueParam)(nil), "openolt.ValueParam")
-	proto.RegisterType((*PonRxPowerData)(nil), "openolt.PonRxPowerData")
-	proto.RegisterType((*Empty)(nil), "openolt.Empty")
-}
-
-func init() { proto.RegisterFile("voltha_protos/openolt.proto", fileDescriptor_c072e7aa0dfd74d5) }
-
-var fileDescriptor_c072e7aa0dfd74d5 = []byte{
-	// 5347 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5c, 0x4d, 0x70, 0x1c, 0x49,
-	0x56, 0x76, 0xeb, 0xa7, 0xbb, 0xf5, 0xfa, 0x47, 0xad, 0xd4, 0x8f, 0xf5, 0xe3, 0xb1, 0xe5, 0x1a,
-	0xcf, 0x8c, 0x77, 0x76, 0x47, 0xb6, 0x7a, 0x2c, 0x8f, 0x3d, 0x2c, 0xbb, 0x23, 0x4b, 0x6d, 0xa9,
-	0x19, 0x49, 0x2d, 0x4a, 0x6d, 0x9b, 0x9d, 0x8d, 0x89, 0xda, 0x52, 0x55, 0x76, 0xab, 0x56, 0xd5,
-	0x95, 0x35, 0x55, 0xd9, 0xfa, 0xe1, 0xb8, 0xc1, 0xc2, 0x85, 0xdb, 0x06, 0x44, 0x40, 0x70, 0x21,
-	0xb8, 0x72, 0xe1, 0x46, 0x04, 0x47, 0x82, 0xe0, 0xc2, 0x8d, 0x33, 0x37, 0x82, 0x0b, 0x27, 0x2e,
-	0x9c, 0x08, 0x82, 0xc8, 0x97, 0x59, 0x7f, 0xdd, 0x2d, 0xd9, 0x1e, 0x44, 0x70, 0x71, 0x28, 0xdf,
-	0xfb, 0xde, 0xf7, 0xf2, 0xe7, 0xe5, 0xcb, 0x57, 0x55, 0xd9, 0x86, 0x95, 0x33, 0xe6, 0xf2, 0x13,
-	0xd3, 0xf0, 0x03, 0xc6, 0x59, 0xf8, 0x88, 0xf9, 0xd4, 0x63, 0x2e, 0x5f, 0xc3, 0x26, 0x29, 0xa8,
-	0xe6, 0xf2, 0x9d, 0x2e, 0x63, 0x5d, 0x97, 0x3e, 0x32, 0x7d, 0xe7, 0x91, 0xe9, 0x79, 0x8c, 0x9b,
-	0xdc, 0x61, 0x5e, 0x28, 0x61, 0xcb, 0xab, 0x59, 0x0e, 0x4e, 0xad, 0x13, 0xf1, 0x77, 0xc7, 0x71,
-	0xa9, 0x42, 0x2c, 0x67, 0x11, 0x16, 0xeb, 0xf5, 0x98, 0xa7, 0x74, 0x77, 0xb3, 0x3a, 0x7a, 0xc1,
-	0x0d, 0x8b, 0x79, 0x1d, 0xa7, 0x2b, 0xf5, 0xda, 0x3f, 0x4d, 0x00, 0x34, 0x3d, 0xdb, 0xb1, 0xd0,
-	0x27, 0x59, 0x87, 0x02, 0x73, 0xb9, 0xe1, 0x78, 0xf6, 0x62, 0x6e, 0x35, 0xf7, 0xb0, 0x54, 0x5f,
-	0x58, 0x8b, 0x3a, 0xdd, 0x72, 0x79, 0x02, 0xdc, 0xbd, 0xa5, 0xe7, 0x19, 0x0a, 0xc8, 0x13, 0x28,
-	0x3a, 0x1e, 0xef, 0xa0, 0xcd, 0x18, 0xda, 0xdc, 0x8e, 0x6d, 0x9a, 0x1e, 0xef, 0x64, 0x8c, 0x0a,
-	0x8e, 0x94, 0x90, 0x4d, 0xa8, 0xa0, 0x15, 0xf3, 0x69, 0x80, 0xa6, 0xe3, 0x68, 0xba, 0x92, 0x31,
-	0x6d, 0xf9, 0x34, 0xc8, 0x98, 0x97, 0x9c, 0x44, 0x4a, 0x7e, 0x02, 0x65, 0xe6, 0xf5, 0x0d, 0xdb,
-	0x09, 0x2d, 0x64, 0x98, 0x40, 0x86, 0xe5, 0xa4, 0xc3, 0x5e, 0x7f, 0xdb, 0x09, 0xad, 0x0c, 0x01,
-	0xb0, 0x58, 0x88, 0x63, 0xf5, 0xfa, 0x68, 0x3a, 0x39, 0x38, 0x56, 0xaf, 0x3f, 0x30, 0x56, 0x14,
-	0x88, 0xb1, 0xb2, 0x9e, 0xe5, 0xa0, 0x4d, 0x7e, 0x60, 0xac, 0xad, 0x9e, 0xe5, 0x64, 0xc7, 0xca,
-	0xa4, 0x84, 0x3c, 0x81, 0x82, 0x7f, 0x2a, 0x27, 0xb5, 0x80, 0x46, 0x4b, 0xb1, 0xd1, 0xa1, 0x69,
-	0x9d, 0xd2, 0x81, 0x79, 0xf5, 0x4f, 0x71, 0x5e, 0x9f, 0x01, 0xf8, 0x2c, 0xe0, 0x46, 0xc8, 0x4d,
-	0x1e, 0x2e, 0x16, 0x07, 0xbc, 0x1d, 0xb2, 0x80, 0x1f, 0x89, 0x40, 0x09, 0xb9, 0x63, 0x85, 0xbb,
-	0xb7, 0xf4, 0x29, 0x5f, 0x49, 0x42, 0x61, 0xd9, 0x71, 0xd9, 0xb9, 0xb2, 0x9c, 0x1a, 0xb0, 0x7c,
-	0xe9, 0xb2, 0xf3, 0xac, 0x65, 0x47, 0x49, 0x42, 0xf2, 0x05, 0x4c, 0x99, 0xae, 0x19, 0xf4, 0xb0,
-	0xaf, 0x80, 0x86, 0x8b, 0xb1, 0xe1, 0xa6, 0xd0, 0x64, 0xba, 0x5a, 0x34, 0x95, 0xe8, 0x45, 0x1e,
-	0x26, 0x6c, 0x93, 0x9b, 0xda, 0xbf, 0x57, 0x60, 0x7a, 0x00, 0x27, 0xe6, 0xd9, 0x65, 0xe1, 0xc8,
-	0x98, 0xda, 0x63, 0x61, 0x76, 0xec, 0x2e, 0x0a, 0xc8, 0x36, 0x54, 0xed, 0x4b, 0xc7, 0xeb, 0x1a,
-	0x5d, 0x33, 0xf4, 0x53, 0x91, 0x75, 0x27, 0xb6, 0xdc, 0x16, 0xea, 0x1d, 0x33, 0xf4, 0x33, 0xf6,
-	0x65, 0x3b, 0x25, 0x16, 0x31, 0x26, 0x16, 0x38, 0x19, 0xd1, 0x60, 0x8c, 0xb5, 0xbc, 0xfe, 0xf0,
-	0xa0, 0x4a, 0x2c, 0x91, 0x92, 0x37, 0x30, 0x27, 0x28, 0x42, 0x6e, 0x06, 0xbc, 0xef, 0x1b, 0x1d,
-	0xd3, 0x71, 0x53, 0xb1, 0xf6, 0x20, 0xcd, 0x74, 0x24, 0x31, 0x2f, 0x4d, 0xc7, 0xed, 0x07, 0x34,
-	0x43, 0x39, 0xc3, 0x32, 0x6a, 0x41, 0xfc, 0x0d, 0x2c, 0x20, 0xb1, 0xd3, 0xf5, 0x4c, 0xd7, 0xb0,
-	0x69, 0x37, 0x30, 0x6d, 0x9a, 0x8a, 0xc5, 0x0f, 0x33, 0xd4, 0x88, 0xda, 0x96, 0xa0, 0x0c, 0xf3,
-	0x2c, 0x1b, 0xd6, 0x92, 0x9f, 0xc3, 0x6d, 0xdc, 0x18, 0x81, 0xd3, 0xe1, 0x06, 0xeb, 0x18, 0xe7,
-	0x8e, 0x67, 0xb3, 0xf3, 0x54, 0xd0, 0x66, 0xc8, 0xb7, 0x05, 0xac, 0xd5, 0x79, 0x83, 0xa0, 0x21,
-	0xf2, 0x41, 0x2d, 0x69, 0x83, 0x18, 0x8d, 0xe1, 0xb2, 0x30, 0x34, 0xe2, 0xbd, 0x20, 0xc3, 0xfa,
-	0x93, 0x34, 0xed, 0x1e, 0x0b, 0xc3, 0x56, 0x47, 0x6c, 0x8a, 0xad, 0x13, 0xd3, 0xf3, 0xa8, 0x9b,
-	0xa1, 0xae, 0x32, 0x85, 0x50, 0x5b, 0x24, 0x9a, 0x67, 0x1c, 0x4a, 0x98, 0xcc, 0x73, 0x71, 0xc4,
-	0x3c, 0x4b, 0xcc, 0x95, 0xf3, 0x9c, 0xa8, 0x05, 0x71, 0x4b, 0x26, 0x09, 0xee, 0x9c, 0xcb, 0x9e,
-	0xca, 0xdd, 0xf0, 0xc3, 0x34, 0x61, 0x3b, 0x30, 0xbd, 0xb0, 0xe7, 0x84, 0xa1, 0xc3, 0xbc, 0xa6,
-	0xc7, 0x69, 0xd0, 0xa1, 0x01, 0xf5, 0x2c, 0xfa, 0xc6, 0x0c, 0x3c, 0xc7, 0xeb, 0xaa, 0xac, 0xd1,
-	0x76, 0xce, 0xb1, 0xa7, 0xbf, 0x90, 0x93, 0x6b, 0x5a, 0xdc, 0x39, 0x43, 0xbf, 0x49, 0x67, 0x61,
-	0x78, 0x16, 0x36, 0x63, 0xd8, 0xa8, 0xfe, 0x8a, 0x31, 0x67, 0x11, 0xd2, 0xc3, 0xa2, 0xf0, 0xe0,
-	0x07, 0xcc, 0xa2, 0x61, 0x28, 0x76, 0x01, 0x0d, 0x02, 0x26, 0xb3, 0x64, 0x09, 0x5d, 0x7c, 0x94,
-	0x76, 0x71, 0x18, 0xe3, 0x1a, 0x02, 0x96, 0x71, 0x30, 0xcf, 0x46, 0xe9, 0x09, 0x85, 0xa5, 0x64,
-	0x0d, 0x3b, 0x46, 0x78, 0xe9, 0x59, 0xc9, 0x28, 0xca, 0xe8, 0xe2, 0xd3, 0xe1, 0xb5, 0xfc, 0x9a,
-	0x5e, 0x1e, 0x5d, 0x7a, 0xd6, 0x55, 0x03, 0x91, 0xa0, 0x08, 0x21, 0xdc, 0xbc, 0x82, 0x79, 0x4c,
-	0xb0, 0xbc, 0x6f, 0xf8, 0xcc, 0x93, 0xe9, 0x08, 0x5d, 0x54, 0xd0, 0xc5, 0xfd, 0x4c, 0xba, 0xe5,
-	0xfd, 0x43, 0xe6, 0x61, 0x16, 0x1a, 0x5a, 0xd2, 0xac, 0x8e, 0xb8, 0x70, 0x07, 0xc3, 0x9b, 0x0e,
-	0xac, 0x41, 0x3f, 0x90, 0x1b, 0xa8, 0x8a, 0xec, 0x3f, 0xc8, 0xc4, 0x78, 0x0a, 0x3b, 0xaa, 0xff,
-	0x62, 0x3a, 0x46, 0x63, 0xc8, 0x1b, 0x39, 0x88, 0x80, 0xf6, 0x18, 0xa7, 0x86, 0x4d, 0x3b, 0xd4,
-	0x92, 0xa9, 0x7c, 0x1a, 0xdd, 0x68, 0x69, 0x37, 0x3a, 0x82, 0xb6, 0x11, 0x93, 0xe1, 0x27, 0x6c,
-	0x48, 0x49, 0x42, 0x39, 0x0c, 0x5c, 0x84, 0x2e, 0xed, 0x19, 0x36, 0x75, 0x1d, 0x8f, 0xca, 0xe1,
-	0x08, 0xfe, 0x1a, 0xf2, 0xaf, 0x0f, 0xaf, 0xc3, 0x4e, 0x63, 0x5f, 0x6d, 0xa9, 0xed, 0xc4, 0x24,
-	0xe3, 0x6e, 0x51, 0x2d, 0xc7, 0x0e, 0xed, 0x65, 0x21, 0xe4, 0x0c, 0x56, 0x31, 0xb6, 0x4e, 0x2e,
-	0x43, 0xc7, 0x32, 0x5d, 0x83, 0x7e, 0xd7, 0x77, 0xfc, 0x1e, 0xf5, 0x78, 0x2a, 0xc6, 0x66, 0xd0,
-	0xf1, 0x8f, 0x32, 0x31, 0xa6, 0xf0, 0x8d, 0x08, 0x3e, 0x1c, 0x6a, 0x62, 0x30, 0x57, 0xc2, 0xc8,
-	0xcf, 0x61, 0x36, 0x1d, 0x71, 0xa6, 0x75, 0x8a, 0xae, 0xc8, 0xf0, 0x6e, 0x94, 0x63, 0xdc, 0xb4,
-	0x4e, 0x3d, 0x76, 0xee, 0x52, 0xbb, 0x4b, 0x05, 0x4f, 0xc6, 0xd3, 0x34, 0x4b, 0xa1, 0x04, 0x39,
-	0x83, 0x15, 0x59, 0x08, 0x74, 0x3a, 0x46, 0x40, 0x4d, 0xeb, 0xc4, 0xa0, 0x17, 0x16, 0xa5, 0x36,
-	0xb5, 0xd1, 0xc9, 0x2c, 0x3a, 0x79, 0x94, 0xad, 0x0b, 0x3a, 0xb8, 0xc9, 0xb9, 0x63, 0xba, 0xba,
-	0xb0, 0x68, 0x28, 0x83, 0x8c, 0xa3, 0xdb, 0x4c, 0x22, 0x07, 0x11, 0xf1, 0x69, 0xb7, 0x06, 0x95,
-	0x4c, 0x55, 0x44, 0x3e, 0x00, 0xc0, 0x82, 0x46, 0x84, 0x3a, 0xc5, 0xd3, 0x6e, 0x4a, 0x9f, 0x12,
-	0x12, 0x11, 0xbc, 0x54, 0xdb, 0x85, 0x6a, 0xb6, 0x22, 0x22, 0xb7, 0xa1, 0x20, 0x8b, 0x27, 0x79,
-	0x36, 0x16, 0xf4, 0x3c, 0x16, 0x48, 0xf6, 0x00, 0xd3, 0xd8, 0x20, 0xd3, 0x09, 0xcc, 0x0c, 0x95,
-	0x37, 0x57, 0x93, 0x7d, 0x09, 0x95, 0x90, 0x06, 0x8e, 0xe9, 0x1a, 0x5e, 0xbf, 0x77, 0x4c, 0x03,
-	0x75, 0x9a, 0xce, 0xc7, 0x53, 0x72, 0x84, 0xda, 0x03, 0x54, 0xea, 0xe5, 0x30, 0xd5, 0xd2, 0x7e,
-	0x33, 0x01, 0x95, 0x4c, 0x39, 0x74, 0xb5, 0x9b, 0x79, 0xc8, 0xe3, 0x7e, 0x97, 0xa7, 0x75, 0x41,
-	0x9f, 0x14, 0x7b, 0x77, 0x70, 0x28, 0xe3, 0x03, 0x43, 0x21, 0xf7, 0xa0, 0x64, 0xda, 0x3d, 0xc7,
-	0x53, 0xfa, 0x49, 0xd4, 0x03, 0x8a, 0x24, 0x60, 0xa8, 0xf7, 0x13, 0xef, 0xdc, 0x7b, 0xb2, 0x07,
-	0x25, 0x4c, 0x6c, 0x01, 0x35, 0x43, 0xe6, 0xe1, 0xf1, 0x57, 0xcd, 0xc6, 0x5b, 0x32, 0xb0, 0xb5,
-	0x6c, 0x2a, 0xd6, 0xd1, 0x44, 0x87, 0x4e, 0xfc, 0xb7, 0xf6, 0x47, 0x63, 0x30, 0x37, 0x0a, 0x44,
-	0x3e, 0x84, 0x7b, 0xad, 0x83, 0x57, 0xc6, 0xe6, 0x56, 0xbb, 0xf9, 0x7a, 0xb3, 0xdd, 0x6c, 0x1d,
-	0x18, 0x2f, 0x37, 0x9b, 0x7b, 0x86, 0xde, 0xd8, 0x3c, 0x6a, 0x1d, 0x18, 0x07, 0xad, 0x83, 0x46,
-	0xed, 0x16, 0xf9, 0x18, 0xb4, 0x6b, 0x40, 0xfa, 0xe6, 0xc1, 0x4e, 0xf3, 0x60, 0xa7, 0x96, 0x23,
-	0x4f, 0xa1, 0x7e, 0x0d, 0xee, 0x70, 0xf3, 0xe8, 0xe8, 0x4d, 0x4b, 0xdf, 0x36, 0x36, 0x5f, 0xb5,
-	0x77, 0x1b, 0x07, 0xed, 0xe6, 0x16, 0x62, 0x6a, 0x63, 0x44, 0x83, 0xbb, 0xd7, 0xd8, 0xed, 0xb5,
-	0x8e, 0x6a, 0xe3, 0xe4, 0x3e, 0x7c, 0x30, 0x0a, 0x83, 0xb2, 0xbd, 0x4d, 0x7d, 0xbf, 0x36, 0x71,
-	0xd5, 0x58, 0x8e, 0xde, 0x34, 0xdb, 0x5b, 0xbb, 0x46, 0xeb, 0x75, 0x43, 0xaf, 0x4d, 0x6a, 0xbf,
-	0x00, 0x32, 0x5c, 0xa0, 0x13, 0x02, 0x13, 0xfc, 0xd2, 0x8f, 0x02, 0x1f, 0xff, 0x4e, 0x47, 0xcb,
-	0xd8, 0x35, 0x11, 0x3e, 0x18, 0x16, 0x9a, 0x0e, 0xd5, 0x6c, 0x45, 0xfd, 0xde, 0x71, 0x57, 0x83,
-	0x71, 0xff, 0x94, 0x23, 0x73, 0x59, 0x17, 0x7f, 0x6a, 0xff, 0x91, 0x83, 0xda, 0x60, 0xc5, 0x4d,
-	0x56, 0x60, 0x0a, 0x69, 0xb1, 0xe7, 0x32, 0xfa, 0xf0, 0x81, 0xa6, 0x3d, 0xd0, 0xfb, 0xab, 0x7c,
-	0x16, 0xd3, 0x3e, 0xe7, 0x21, 0xdf, 0xf7, 0x1c, 0x21, 0x9e, 0x92, 0xe2, 0xbe, 0xe7, 0xc8, 0xb1,
-	0x76, 0x69, 0x0f, 0xcb, 0xf9, 0xb8, 0x97, 0x53, 0x4a, 0xd2, 0xb4, 0x85, 0x17, 0x2c, 0xd8, 0x1d,
-	0x59, 0xa2, 0x16, 0xf4, 0xbc, 0x68, 0x4a, 0x05, 0x1a, 0x79, 0x0c, 0x43, 0xb7, 0xa0, 0xe7, 0x45,
-	0xf3, 0x80, 0x91, 0x05, 0xc8, 0x5b, 0x8c, 0x9d, 0x3a, 0x14, 0x4b, 0xaf, 0xbc, 0xae, 0x5a, 0xd1,
-	0x98, 0x27, 0x92, 0x31, 0x3f, 0x80, 0x29, 0x59, 0xd4, 0x98, 0xd6, 0xd5, 0xc3, 0xd1, 0x7e, 0x0c,
-	0x53, 0xbb, 0xd4, 0x0c, 0xf8, 0x31, 0x35, 0x39, 0x79, 0x04, 0xb3, 0x27, 0x51, 0x43, 0x96, 0x64,
-	0xbc, 0x1f, 0x50, 0x65, 0x41, 0x62, 0xd5, 0x51, 0xa4, 0xd1, 0xfe, 0x3a, 0x07, 0xe3, 0x2d, 0xaf,
-	0xff, 0xde, 0x2b, 0x34, 0xb4, 0xb3, 0xc7, 0xdf, 0x7d, 0x67, 0x8b, 0x91, 0x3a, 0x32, 0x17, 0x14,
-	0x74, 0xf1, 0x27, 0xf9, 0x04, 0xa6, 0x59, 0xcf, 0xb2, 0x0c, 0xea, 0x59, 0xc1, 0xa5, 0x2f, 0xd6,
-	0x16, 0x97, 0xb3, 0xa8, 0x57, 0x85, 0xb8, 0x11, 0x4b, 0xb5, 0xbf, 0xc9, 0x01, 0xc1, 0x93, 0xa6,
-	0x2b, 0x0e, 0xab, 0x6d, 0x27, 0xe4, 0xa6, 0x67, 0xd1, 0xf7, 0xee, 0xfd, 0x73, 0x58, 0x72, 0x25,
-	0x85, 0xa1, 0x9e, 0x43, 0x91, 0xc7, 0xf8, 0x7d, 0x1a, 0x30, 0xb5, 0x8e, 0x0b, 0x0a, 0x20, 0x73,
-	0x35, 0xaa, 0xbf, 0xa1, 0x01, 0x23, 0x8f, 0x61, 0x6e, 0x94, 0xa9, 0x1a, 0x0d, 0x19, 0xb6, 0xd2,
-	0xbe, 0x86, 0x82, 0xd8, 0x0e, 0xfb, 0x61, 0xf7, 0x06, 0xf6, 0xc1, 0xaf, 0x73, 0x30, 0x25, 0x4e,
-	0x75, 0xdc, 0x0a, 0xef, 0xcd, 0x97, 0x0a, 0xca, 0x89, 0x4c, 0x50, 0x66, 0xa3, 0x7c, 0x72, 0x30,
-	0xca, 0x87, 0xfb, 0xf1, 0x1c, 0xca, 0xaf, 0x7c, 0xd7, 0xf1, 0x4e, 0xdf, 0xd6, 0x13, 0x65, 0x3a,
-	0x96, 0x98, 0xfe, 0xc3, 0x14, 0xc0, 0x36, 0x3d, 0x73, 0x2c, 0xda, 0xf4, 0x3a, 0xb8, 0x1f, 0xce,
-	0xa8, 0x67, 0xb3, 0x40, 0xe5, 0x1e, 0xd5, 0x22, 0x73, 0x30, 0xd9, 0x63, 0x36, 0x75, 0xd5, 0x09,
-	0x2a, 0x1b, 0xe4, 0x07, 0x50, 0x3b, 0x31, 0x03, 0xfb, 0xdc, 0x0c, 0xa8, 0x71, 0x46, 0x03, 0x51,
-	0xf8, 0xab, 0x04, 0x34, 0x1d, 0xc9, 0x5f, 0x4b, 0xb1, 0x80, 0x76, 0x9c, 0xa0, 0x97, 0x81, 0x4e,
-	0x48, 0x68, 0x24, 0x8f, 0xa0, 0x2b, 0x30, 0x65, 0x63, 0x8f, 0x44, 0xff, 0x6b, 0x32, 0x91, 0x48,
-	0x41, 0xd3, 0x16, 0x2b, 0xae, 0x94, 0xd9, 0x88, 0x9f, 0x41, 0x1c, 0x91, 0xba, 0x74, 0xb8, 0x93,
-	0x75, 0x98, 0xf3, 0x03, 0x7a, 0xe6, 0xb0, 0x7e, 0xe8, 0x5e, 0x1a, 0x16, 0xf3, 0x3c, 0x6a, 0x71,
-	0x2a, 0xcb, 0x99, 0xa2, 0x3e, 0x9b, 0xe8, 0xb6, 0x22, 0x95, 0xe8, 0x81, 0x28, 0xb4, 0xc5, 0x7c,
-	0x87, 0x58, 0xc7, 0x17, 0xf4, 0xa2, 0xcf, 0xbc, 0x43, 0xd1, 0x26, 0x77, 0x01, 0x38, 0xb5, 0x4e,
-	0x3c, 0xe6, 0xb2, 0xee, 0x65, 0x74, 0xcc, 0x26, 0x12, 0xb2, 0x2a, 0x9f, 0x94, 0x1c, 0x5b, 0x3e,
-	0xed, 0xaa, 0x84, 0x03, 0xb8, 0xe6, 0xf8, 0xf0, 0x4a, 0xee, 0x00, 0x28, 0x04, 0x55, 0xcf, 0x7c,
-	0x05, 0xbd, 0x88, 0xfa, 0x86, 0x67, 0x93, 0x07, 0x50, 0x35, 0x5d, 0x97, 0x59, 0x09, 0x83, 0xcc,
-	0x8c, 0x65, 0x94, 0x46, 0x1c, 0xab, 0x50, 0x8e, 0x51, 0xd4, 0x8b, 0xd2, 0x24, 0x28, 0x8c, 0xe0,
-	0x79, 0x08, 0xb5, 0x24, 0x8a, 0x14, 0x13, 0x20, 0xaa, 0x1a, 0xc7, 0x92, 0xe4, 0x7a, 0x00, 0xd5,
-	0x14, 0x92, 0xaa, 0xc7, 0xa3, 0x82, 0x5e, 0x8e, 0x71, 0x82, 0x4f, 0x83, 0x8a, 0x4a, 0xae, 0x8a,
-	0xac, 0x82, 0xa0, 0x92, 0x4c, 0xb1, 0x92, 0xe9, 0x2e, 0x94, 0x22, 0x0c, 0x55, 0x4f, 0x10, 0x05,
-	0xf9, 0x5e, 0x44, 0x72, 0x7c, 0x05, 0xf9, 0xc0, 0xf4, 0xba, 0x34, 0x5c, 0x9c, 0x5e, 0x1d, 0x7f,
-	0x58, 0xaa, 0x3f, 0x4c, 0xde, 0x43, 0xc4, 0x31, 0xa8, 0xfe, 0xd4, 0x69, 0xc8, 0xfa, 0x81, 0x45,
-	0x75, 0xc4, 0xeb, 0xca, 0x6e, 0xf9, 0x4f, 0x26, 0x60, 0x6e, 0x14, 0x80, 0x2c, 0x45, 0xaf, 0xcf,
-	0xec, 0x70, 0x31, 0xb7, 0x3a, 0xfe, 0xb0, 0xa0, 0xde, 0x91, 0xd9, 0x83, 0x2b, 0x36, 0x36, 0xb4,
-	0x62, 0x5b, 0x30, 0xe9, 0x33, 0xe6, 0x86, 0x8b, 0xe3, 0xd8, 0xa9, 0xcf, 0xde, 0xb5, 0x53, 0x6b,
-	0x87, 0x8c, 0xb9, 0xba, 0xb4, 0x5d, 0xfe, 0xaf, 0x31, 0x98, 0x10, 0x6d, 0xf2, 0x3b, 0xa9, 0xc3,
-	0xbb, 0x5a, 0x7f, 0xfa, 0x5e, 0x64, 0xf8, 0x8f, 0x38, 0x30, 0xd5, 0xa1, 0x7f, 0x04, 0x85, 0xf0,
-	0xc4, 0x0c, 0x1c, 0xaf, 0x8b, 0xdd, 0xae, 0xd6, 0x9f, 0xbf, 0x1f, 0xdd, 0x91, 0x34, 0x46, 0xc6,
-	0x88, 0x49, 0xec, 0x65, 0xb9, 0x80, 0x32, 0xb7, 0xca, 0x86, 0x48, 0x0d, 0x54, 0xbd, 0x90, 0x29,
-	0xe8, 0xe2, 0x4f, 0x6d, 0x13, 0x8a, 0x51, 0x77, 0x08, 0x40, 0x5e, 0x14, 0x33, 0xcd, 0xed, 0xda,
-	0x2d, 0x52, 0x86, 0xe2, 0xe6, 0xde, 0x5e, 0x6b, 0x4b, 0xb4, 0x72, 0xa4, 0x0a, 0xb0, 0xd3, 0xd8,
-	0x3f, 0x6c, 0xe9, 0x6d, 0xd1, 0x1e, 0x23, 0x25, 0x28, 0xbc, 0xdc, 0x6b, 0xbd, 0x11, 0x8d, 0x71,
-	0xed, 0x04, 0x4a, 0xa9, 0x2e, 0x90, 0x05, 0x20, 0xdb, 0x8d, 0x6d, 0x51, 0x69, 0x35, 0xb6, 0x8d,
-	0xc3, 0x86, 0x6e, 0x34, 0x0f, 0xda, 0x2f, 0x6b, 0xb7, 0xc8, 0x3d, 0x58, 0x39, 0xda, 0xdd, 0xd4,
-	0x1b, 0xdb, 0xc6, 0x8b, 0x9f, 0x19, 0x9b, 0x7b, 0x7b, 0x28, 0xc7, 0x3f, 0xda, 0x8d, 0xad, 0xdd,
-	0x5a, 0x8e, 0xac, 0xc2, 0x9d, 0x11, 0x80, 0xa3, 0xcd, 0xfd, 0x86, 0x44, 0x8c, 0x69, 0x7f, 0x30,
-	0x0e, 0xb0, 0xe5, 0x9a, 0x61, 0xe8, 0x74, 0x1c, 0x1a, 0x60, 0xca, 0x35, 0xb8, 0x1f, 0x27, 0xc0,
-	0x49, 0xd6, 0xf6, 0x1d, 0x9b, 0xcc, 0xc2, 0x24, 0x33, 0xce, 0xe2, 0x44, 0x3c, 0xc1, 0x5e, 0x3b,
-	0x98, 0x9e, 0x1d, 0x89, 0x55, 0x13, 0xe2, 0x44, 0x58, 0x07, 0xb1, 0x72, 0x4a, 0x26, 0x1c, 0x81,
-	0xbd, 0x0d, 0x05, 0x66, 0xf8, 0xc7, 0x0e, 0x0f, 0x55, 0x5e, 0xce, 0xb3, 0x43, 0xd1, 0xc2, 0x94,
-	0xab, 0x14, 0xaa, 0xc2, 0x70, 0xa4, 0x62, 0x09, 0x8a, 0x94, 0x9f, 0xc8, 0xaa, 0x48, 0x6e, 0xf5,
-	0x02, 0xe5, 0x27, 0x51, 0x51, 0x64, 0x87, 0xdc, 0xe8, 0x99, 0x16, 0x6e, 0xf1, 0xb2, 0x9e, 0xb7,
-	0x43, 0xbe, 0x6f, 0x5a, 0x42, 0x11, 0x06, 0x16, 0x2a, 0xa6, 0xa4, 0x22, 0x0c, 0x2c, 0xa1, 0x10,
-	0x41, 0xee, 0xcb, 0x77, 0xd0, 0x6a, 0x2f, 0x17, 0x1c, 0xff, 0x10, 0xdf, 0x82, 0xcf, 0x83, 0xb0,
-	0x36, 0x1c, 0x5f, 0x6d, 0xde, 0x49, 0x3b, 0xe4, 0x4d, 0x5f, 0x88, 0x05, 0x95, 0xe3, 0xab, 0x3c,
-	0x36, 0x19, 0x06, 0x56, 0xd3, 0x17, 0x44, 0x42, 0x2c, 0x76, 0xb7, 0xda, 0xc7, 0xc2, 0xa3, 0x48,
-	0x70, 0x42, 0x25, 0x88, 0x50, 0x25, 0x37, 0xb0, 0xe8, 0x25, 0xaa, 0x56, 0xa1, 0xec, 0x9f, 0x72,
-	0x83, 0x9b, 0x5d, 0x39, 0x9e, 0x69, 0xb9, 0x95, 0xfc, 0x53, 0xde, 0x36, 0x71, 0x85, 0xb5, 0x5f,
-	0x8f, 0xc3, 0x94, 0xa8, 0xec, 0x99, 0xb7, 0xd5, 0xc3, 0x94, 0x61, 0xda, 0xb6, 0xc1, 0xfa, 0x9c,
-	0x06, 0xc2, 0x0a, 0x17, 0xa3, 0xa8, 0x97, 0x4c, 0xdb, 0x6e, 0x09, 0x59, 0xdb, 0xec, 0x8a, 0x34,
-	0x15, 0xd0, 0x1e, 0x3b, 0xa3, 0x29, 0xd8, 0x98, 0x2c, 0x37, 0xa4, 0x3c, 0x46, 0xae, 0x42, 0x99,
-	0x07, 0xa6, 0x6f, 0x70, 0x66, 0x9c, 0xb0, 0x50, 0x86, 0x6f, 0x51, 0x07, 0x21, 0x6b, 0xb3, 0x5d,
-	0x16, 0x72, 0xf2, 0x23, 0x20, 0x01, 0xed, 0x99, 0xc1, 0xa9, 0xe2, 0x92, 0xeb, 0x31, 0x81, 0xb8,
-	0x9a, 0xd4, 0x20, 0x9b, 0x5c, 0x99, 0x04, 0xed, 0x78, 0x5e, 0x8c, 0x9e, 0x4c, 0xa3, 0x9b, 0x42,
-	0x21, 0xd1, 0x6a, 0x2c, 0x12, 0x2a, 0x3a, 0x99, 0x8f, 0xc7, 0x82, 0xa8, 0xec, 0x58, 0x12, 0x58,
-	0x21, 0x3d, 0x96, 0x18, 0xb9, 0x06, 0xb3, 0x3c, 0x30, 0xbd, 0xd0, 0x35, 0x79, 0x1a, 0x5c, 0x44,
-	0xf0, 0x4c, 0xac, 0x1a, 0x8d, 0x4f, 0x26, 0x6a, 0x6a, 0x00, 0x1f, 0xcd, 0x95, 0xf6, 0xb7, 0x39,
-	0xc8, 0xcb, 0x75, 0x20, 0x0f, 0x60, 0xdc, 0xea, 0x45, 0xaf, 0x8c, 0x49, 0xf2, 0x16, 0x3a, 0x5a,
-	0x25, 0x5d, 0xa8, 0x47, 0xef, 0x8c, 0x54, 0xb4, 0x8f, 0x67, 0xa2, 0x3d, 0xd9, 0x5e, 0x13, 0x03,
-	0xdb, 0x4b, 0x6e, 0x99, 0xc9, 0xec, 0x96, 0x19, 0xbd, 0x33, 0x92, 0x7d, 0x57, 0x48, 0xed, 0x3b,
-	0xed, 0x1f, 0xf3, 0x30, 0xf1, 0xd2, 0x65, 0xe7, 0x78, 0x10, 0x5a, 0x16, 0x0d, 0x43, 0x23, 0x5d,
-	0xcc, 0x4c, 0xeb, 0x65, 0x29, 0x6d, 0x8e, 0x2a, 0xae, 0xa6, 0x87, 0x1f, 0x20, 0x4a, 0x52, 0x2c,
-	0x1f, 0x20, 0x06, 0x9e, 0x10, 0xf2, 0xf1, 0x13, 0xc2, 0xa7, 0x30, 0x13, 0x5e, 0xf6, 0x7a, 0x94,
-	0x07, 0x8e, 0x65, 0x44, 0x10, 0x82, 0x90, 0xe9, 0x58, 0xf1, 0x52, 0x62, 0x57, 0x00, 0x8f, 0x34,
-	0xb9, 0x07, 0x64, 0x11, 0x53, 0x14, 0x02, 0xdc, 0xd4, 0x4b, 0x50, 0x8c, 0x0e, 0x66, 0xdc, 0xa2,
-	0xd3, 0x7a, 0x41, 0x1d, 0xca, 0xe4, 0x63, 0x98, 0xf6, 0x28, 0x3f, 0x67, 0x18, 0x71, 0x72, 0x44,
-	0x93, 0x88, 0xa8, 0x28, 0x71, 0x33, 0x7e, 0xa2, 0x4b, 0xd5, 0x7f, 0x79, 0x84, 0xa4, 0xea, 0xbf,
-	0xcf, 0x01, 0xac, 0x38, 0xd3, 0xa9, 0x57, 0xc6, 0xb3, 0xf1, 0xba, 0x26, 0x49, 0x50, 0x4f, 0xc1,
-	0xc8, 0x27, 0x90, 0x37, 0x71, 0xc5, 0xd5, 0xab, 0xe0, 0xe9, 0x81, 0x40, 0xd0, 0x95, 0x9a, 0x2c,
-	0x43, 0xd1, 0x0f, 0x1c, 0x16, 0x38, 0xfc, 0x12, 0xc3, 0x6b, 0x5a, 0x8f, 0xdb, 0xa9, 0xa7, 0xa5,
-	0x72, 0xe6, 0x69, 0x29, 0x55, 0xc9, 0x56, 0x32, 0x95, 0xec, 0x12, 0x14, 0xbb, 0x01, 0xeb, 0xfb,
-	0x62, 0x1c, 0x2a, 0x97, 0x60, 0x5b, 0x4e, 0x46, 0xfa, 0x13, 0x9c, 0x40, 0x4c, 0x23, 0xa2, 0x22,
-	0xc4, 0x87, 0x52, 0xda, 0xb4, 0xc9, 0x47, 0x50, 0x0d, 0xa8, 0xef, 0x8a, 0xa7, 0x4c, 0x8a, 0x0b,
-	0x83, 0x25, 0x61, 0x51, 0xaf, 0xc4, 0x52, 0x0c, 0x96, 0x5d, 0x98, 0x16, 0x31, 0x26, 0x92, 0x83,
-	0x9a, 0xa9, 0xc5, 0x19, 0x3c, 0xcd, 0x57, 0x33, 0x1f, 0x6c, 0xd6, 0x44, 0xe8, 0xb5, 0xd9, 0x8e,
-	0x84, 0x34, 0x3c, 0x1e, 0x5c, 0xea, 0x15, 0x3f, 0x2d, 0x23, 0x8d, 0xa4, 0x1a, 0xe2, 0xcc, 0x30,
-	0x69, 0xb8, 0x38, 0x8b, 0x44, 0xf7, 0xb2, 0x44, 0x0a, 0xde, 0x66, 0x9b, 0x34, 0x94, 0x3c, 0x51,
-	0xb9, 0x84, 0xa2, 0xe5, 0xaf, 0x80, 0x0c, 0xfb, 0x12, 0xa7, 0xec, 0x29, 0xbd, 0x54, 0x87, 0x92,
-	0xf8, 0x53, 0x9c, 0xc6, 0x67, 0xa6, 0xdb, 0xa7, 0xd1, 0xb3, 0x01, 0x36, 0xbe, 0x1c, 0x7b, 0x96,
-	0x5b, 0xfe, 0x29, 0xcc, 0x0c, 0x39, 0x79, 0x1b, 0x41, 0x31, 0x45, 0xa0, 0xb5, 0xa1, 0x9c, 0xa9,
-	0x84, 0x57, 0x60, 0x4a, 0x96, 0xf3, 0xd1, 0x5e, 0x2a, 0xeb, 0x45, 0x29, 0x68, 0xda, 0xe2, 0xa9,
-	0x4f, 0x29, 0x43, 0x9f, 0x5a, 0x4e, 0xc7, 0xb1, 0xd4, 0x63, 0x42, 0x55, 0x8a, 0x8f, 0x94, 0x54,
-	0xfb, 0xef, 0x12, 0x54, 0xb3, 0x5f, 0xcd, 0xae, 0x7e, 0xde, 0x58, 0x82, 0x62, 0x70, 0x61, 0x1c,
-	0x5f, 0x72, 0x1a, 0x22, 0x5b, 0x5e, 0x2f, 0x04, 0x17, 0x2f, 0x44, 0x53, 0x04, 0x79, 0x70, 0x61,
-	0xf8, 0xf8, 0xc0, 0x12, 0xaa, 0xcd, 0x38, 0x15, 0x5c, 0xc8, 0x27, 0x98, 0x10, 0x53, 0xe9, 0x85,
-	0xd1, 0xb7, 0x4c, 0x71, 0x14, 0x29, 0xd0, 0x04, 0x82, 0xaa, 0xc1, 0xc5, 0x2b, 0x21, 0xce, 0x22,
-	0x7b, 0x19, 0xe4, 0x64, 0x84, 0xdc, 0x1f, 0x46, 0x1e, 0x67, 0x90, 0xf9, 0x08, 0xf9, 0x62, 0x18,
-	0x29, 0x5f, 0xe5, 0x46, 0xc8, 0x42, 0x84, 0xc4, 0x97, 0xb1, 0x11, 0x72, 0x05, 0xa6, 0x82, 0x0b,
-	0xa3, 0x13, 0x98, 0x3d, 0x1a, 0xe2, 0x43, 0x48, 0x5e, 0x2f, 0x06, 0x17, 0x2f, 0xb1, 0x2d, 0x4e,
-	0xac, 0x58, 0x69, 0x3c, 0x7d, 0xa2, 0xf2, 0x09, 0x44, 0xfa, 0xa7, 0x4f, 0xc8, 0x27, 0xe8, 0x28,
-	0x42, 0x6c, 0x18, 0xeb, 0xf5, 0x2f, 0xf0, 0xc1, 0x24, 0xaf, 0x57, 0x62, 0xd4, 0xc6, 0x7a, 0xfd,
-	0x0b, 0xf2, 0x03, 0x98, 0x49, 0x80, 0xeb, 0xf5, 0x67, 0x46, 0x7d, 0x63, 0x63, 0x71, 0x2e, 0xea,
-	0x92, 0x44, 0xae, 0xd7, 0x9f, 0xd5, 0x37, 0x36, 0xb2, 0xd0, 0xfa, 0xc6, 0x53, 0x63, 0x63, 0x7d,
-	0x7d, 0x71, 0x3e, 0x0b, 0xad, 0x6f, 0x3c, 0xdd, 0x58, 0x5f, 0x27, 0x3f, 0x04, 0x92, 0x40, 0x37,
-	0xd6, 0xeb, 0xc6, 0xfa, 0xe3, 0xfa, 0xe7, 0x8b, 0x0b, 0x32, 0xed, 0x45, 0xd8, 0x8d, 0xf5, 0xba,
-	0x10, 0x93, 0xcf, 0x60, 0x36, 0xd5, 0x85, 0xc7, 0xf5, 0x27, 0xc6, 0xfa, 0xc6, 0xfa, 0xb3, 0xc5,
-	0xdb, 0x88, 0xae, 0xc5, 0x9d, 0x78, 0x5c, 0x7f, 0x22, 0xe4, 0x03, 0xf0, 0x8d, 0xf5, 0xe7, 0x46,
-	0xfd, 0xf1, 0x93, 0x2f, 0x16, 0x17, 0x07, 0xe0, 0x1b, 0xeb, 0xcf, 0x85, 0x3c, 0x0b, 0xaf, 0x3f,
-	0x7e, 0xf2, 0xcc, 0x78, 0xf2, 0xf8, 0xf9, 0xc6, 0xe2, 0x52, 0x16, 0x2e, 0x14, 0x42, 0x9e, 0x85,
-	0x3f, 0x79, 0xfc, 0xfc, 0xa9, 0xf1, 0xbc, 0xbe, 0xfe, 0x74, 0x71, 0x39, 0x0b, 0x17, 0x0a, 0x21,
-	0x27, 0x8f, 0x60, 0x2e, 0x81, 0x3f, 0xaf, 0xaf, 0x7f, 0x61, 0xac, 0x3f, 0xfd, 0xfc, 0xd9, 0xe7,
-	0x8b, 0x2b, 0x88, 0x9f, 0x89, 0xf0, 0x42, 0x83, 0x0a, 0x71, 0xdc, 0x07, 0x17, 0x86, 0x15, 0x58,
-	0x32, 0x0a, 0x42, 0x4c, 0x5f, 0x79, 0xbd, 0x14, 0x5c, 0x6c, 0x05, 0x16, 0x46, 0x00, 0x96, 0x76,
-	0x3c, 0x8a, 0xee, 0xa2, 0x8c, 0x6e, 0x9e, 0x44, 0x37, 0x4f, 0xa2, 0x7b, 0x4a, 0x46, 0x37, 0x4f,
-	0x47, 0x37, 0x1f, 0x8c, 0x6e, 0x90, 0x2b, 0xc4, 0x87, 0xa2, 0x9b, 0x0f, 0x46, 0x77, 0x29, 0x42,
-	0xee, 0x0f, 0x23, 0xb3, 0xd1, 0x5d, 0x8e, 0x90, 0x2f, 0x86, 0x91, 0xd9, 0xe8, 0xae, 0x44, 0xc8,
-	0xc1, 0xe8, 0xe6, 0x71, 0x74, 0xdf, 0x91, 0xd1, 0xcd, 0x53, 0xd1, 0xcd, 0xd3, 0xd1, 0xfd, 0x81,
-	0x8c, 0x6e, 0x9e, 0x89, 0x6e, 0x3e, 0x18, 0xdd, 0x77, 0x65, 0x74, 0xf3, 0xc1, 0xe8, 0xe6, 0x43,
-	0xd1, 0x7d, 0x2f, 0xea, 0xd2, 0x60, 0x74, 0xf3, 0xa1, 0xe8, 0x5e, 0xcd, 0x42, 0x93, 0xe8, 0xe6,
-	0xc3, 0xd1, 0x7d, 0x5f, 0x46, 0x37, 0x1f, 0x8e, 0x6e, 0x3e, 0x22, 0xba, 0x35, 0x19, 0x50, 0x7c,
-	0x44, 0x74, 0xf3, 0x11, 0xd1, 0xfd, 0xe1, 0x00, 0x3c, 0x15, 0xdd, 0x7c, 0x44, 0x74, 0x3f, 0xc8,
-	0xc2, 0xd3, 0xd1, 0xcd, 0x47, 0x44, 0xf7, 0x47, 0x59, 0x78, 0x3a, 0xba, 0xf9, 0xa8, 0xe8, 0xfe,
-	0x58, 0x46, 0x37, 0x1f, 0x8a, 0xee, 0x0f, 0x00, 0x8e, 0x1d, 0x3f, 0x0a, 0xed, 0x69, 0x19, 0x9e,
-	0xc7, 0x8e, 0xaf, 0x02, 0xfb, 0x0e, 0x4c, 0x71, 0xa7, 0x47, 0x43, 0x6e, 0xf6, 0x7c, 0x3c, 0x6e,
-	0x0b, 0x7a, 0x22, 0xd0, 0xfe, 0xad, 0x80, 0x5f, 0x32, 0xde, 0x25, 0xff, 0x5f, 0xf1, 0xe6, 0xeb,
-	0x23, 0xa8, 0xfa, 0x2c, 0x74, 0xb8, 0x73, 0x46, 0xe5, 0xd7, 0x75, 0x95, 0xff, 0x2b, 0x91, 0x14,
-	0xbf, 0x96, 0x0b, 0x98, 0x47, 0xbb, 0x66, 0x0a, 0x26, 0x4f, 0x80, 0x4a, 0x24, 0x95, 0xb0, 0x67,
-	0xb0, 0x68, 0x53, 0xd7, 0xe9, 0x39, 0xa2, 0x2a, 0xee, 0x39, 0x61, 0x68, 0xd8, 0x94, 0x53, 0x2b,
-	0x7e, 0x71, 0x99, 0xd7, 0x17, 0x62, 0xfd, 0xbe, 0x13, 0x86, 0xdb, 0x91, 0x76, 0x60, 0x1a, 0xf2,
-	0x83, 0xd3, 0xb0, 0x02, 0xa2, 0x61, 0xf4, 0x3d, 0x27, 0x4e, 0xff, 0xc5, 0x63, 0xc7, 0x7f, 0x25,
-	0xda, 0xa4, 0x0e, 0xf3, 0x1d, 0x6a, 0x19, 0x16, 0x0b, 0x02, 0x7c, 0x69, 0x64, 0x84, 0x97, 0xbd,
-	0x63, 0xe6, 0x46, 0x99, 0x60, 0xb6, 0x43, 0xad, 0xad, 0x48, 0x77, 0x24, 0x55, 0xe4, 0x29, 0xdc,
-	0x96, 0x36, 0x36, 0x3d, 0x67, 0x81, 0x1d, 0x26, 0xd6, 0x2a, 0x45, 0xcc, 0xa3, 0x95, 0xd2, 0xc6,
-	0xe6, 0xe4, 0x27, 0xb0, 0x92, 0xb5, 0xeb, 0x7b, 0xca, 0xd2, 0x3c, 0x76, 0xa9, 0xca, 0x1c, 0x4b,
-	0x69, 0xdb, 0x57, 0x69, 0x00, 0xf9, 0x10, 0x2a, 0x19, 0x7b, 0x95, 0x41, 0xca, 0x69, 0x0b, 0xf1,
-	0x88, 0x91, 0x1d, 0x90, 0x1c, 0xb7, 0x4c, 0x21, 0x33, 0xe9, 0xe1, 0xc8, 0x09, 0xf8, 0x18, 0xa6,
-	0x2f, 0xba, 0xb4, 0x67, 0x9c, 0xd2, 0xcb, 0x68, 0x06, 0x65, 0x12, 0xa9, 0x08, 0xf1, 0xd7, 0xf4,
-	0x32, 0x99, 0x45, 0xc4, 0xb9, 0x2c, 0x8c, 0xb2, 0x68, 0x51, 0x08, 0xf6, 0x58, 0x88, 0x24, 0xa2,
-	0x0a, 0x70, 0x99, 0xd9, 0x0b, 0x25, 0x8b, 0x8a, 0xc6, 0x4a, 0x70, 0x71, 0x88, 0x52, 0x64, 0x51,
-	0x07, 0x95, 0xc2, 0x79, 0xcc, 0x33, 0x1c, 0xdb, 0xa5, 0x18, 0x9a, 0x78, 0x50, 0x49, 0xe8, 0x01,
-	0xf3, 0x9a, 0xb6, 0x8b, 0xe5, 0x68, 0x70, 0x81, 0x97, 0x2a, 0xd4, 0x89, 0x9c, 0x0f, 0x2e, 0x5a,
-	0x3d, 0xcb, 0x21, 0xcf, 0x60, 0x49, 0x29, 0xa2, 0xbc, 0x97, 0x64, 0x78, 0x75, 0x38, 0xcf, 0x4b,
-	0xa8, 0x4a, 0x80, 0x51, 0xae, 0xcf, 0x14, 0x32, 0xb3, 0xd7, 0x15, 0x32, 0x73, 0x83, 0x85, 0x4c,
-	0xfa, 0x90, 0x98, 0xbf, 0xee, 0x90, 0x58, 0x18, 0x3c, 0x24, 0xee, 0x43, 0xf9, 0x98, 0x06, 0x46,
-	0x40, 0x45, 0x09, 0x48, 0x6d, 0x75, 0xd0, 0x96, 0x8e, 0x69, 0xa0, 0x2b, 0x11, 0xb9, 0x07, 0x25,
-	0xd7, 0xb2, 0xbb, 0xd1, 0xfc, 0xcb, 0xb3, 0x15, 0x84, 0x48, 0x4d, 0xbe, 0xe8, 0x9c, 0xed, 0x44,
-	0xfa, 0x25, 0xd5, 0x39, 0xdb, 0x19, 0xb5, 0xd1, 0x97, 0x07, 0x37, 0xfa, 0xbf, 0xe4, 0xb0, 0x02,
-	0x7d, 0xd7, 0x62, 0xef, 0x2d, 0x1f, 0x67, 0xde, 0x52, 0xf0, 0xa5, 0x67, 0x78, 0x62, 0x68, 0x86,
-	0x53, 0xf3, 0x34, 0x39, 0x38, 0x4f, 0xe9, 0x19, 0xce, 0x67, 0x67, 0xf8, 0xfa, 0xf1, 0xfd, 0x5d,
-	0x0e, 0xaa, 0xd9, 0x5b, 0x5c, 0xe9, 0xe7, 0xc3, 0x5c, 0xe6, 0x0b, 0xd2, 0xf7, 0xaf, 0x64, 0xbf,
-	0x7f, 0x95, 0x70, 0x7d, 0x1a, 0xfe, 0x0a, 0x2a, 0x99, 0x6b, 0x5f, 0x57, 0x2f, 0xcc, 0x02, 0xe4,
-	0x43, 0x6e, 0xf2, 0x7e, 0xa8, 0xde, 0x7d, 0xaa, 0x96, 0xf6, 0x2d, 0xcc, 0x8e, 0xb8, 0xfe, 0xf5,
-	0xde, 0xd9, 0x3c, 0xa1, 0x1f, 0xcf, 0xd0, 0xff, 0xd5, 0x18, 0x7e, 0x1e, 0x1a, 0xbc, 0xc6, 0xf6,
-	0x3d, 0x3e, 0x7b, 0xbb, 0x2c, 0x34, 0x32, 0x2e, 0xa6, 0x5c, 0x16, 0x1e, 0xa1, 0x40, 0xaa, 0x8f,
-	0x23, 0xf5, 0x44, 0xa4, 0x3e, 0x56, 0xea, 0x87, 0x50, 0x73, 0x99, 0x6f, 0xc9, 0x73, 0x41, 0x81,
-	0xe4, 0x3b, 0xfb, 0xaa, 0x90, 0x8b, 0xf3, 0x40, 0x21, 0xd7, 0x61, 0x5e, 0x21, 0x55, 0x46, 0x88,
-	0xe0, 0x79, 0xf9, 0x69, 0x41, 0xc2, 0x65, 0x3e, 0x50, 0x26, 0x62, 0xfb, 0xb1, 0x8e, 0x13, 0x01,
-	0x0b, 0xf2, 0x75, 0x98, 0x10, 0x29, 0xc0, 0x7d, 0x28, 0x8b, 0xcc, 0x14, 0x23, 0x8a, 0x88, 0x28,
-	0xa1, 0x4c, 0x42, 0x34, 0x0a, 0x2b, 0xd7, 0x5c, 0x7a, 0xbb, 0xb1, 0xc5, 0xf8, 0xb3, 0x1c, 0x2c,
-	0x5f, 0x7d, 0x03, 0xee, 0xa6, 0xdc, 0x90, 0xcf, 0x61, 0xc1, 0xf1, 0xce, 0x68, 0x10, 0x52, 0x43,
-	0x3c, 0x8d, 0xcb, 0x79, 0x0c, 0x4c, 0x1e, 0x7d, 0x92, 0x9b, 0x55, 0xda, 0x17, 0x8e, 0xbc, 0xd3,
-	0xa2, 0x9b, 0x9c, 0x6a, 0xbf, 0x91, 0x7d, 0xbb, 0xe2, 0x02, 0xdd, 0x8d, 0xf5, 0x6d, 0x0e, 0x26,
-	0x93, 0x2a, 0xa2, 0xa0, 0xcb, 0x86, 0x60, 0xf7, 0xe8, 0xb9, 0x41, 0xbf, 0x8b, 0xde, 0x5a, 0xe5,
-	0x3d, 0x7a, 0xde, 0xf8, 0xce, 0xd6, 0x4e, 0xe0, 0xee, 0xf5, 0xd7, 0xef, 0x6e, 0x6c, 0x6d, 0xfe,
-	0x3c, 0x27, 0x63, 0xe0, 0x8a, 0x0b, 0x79, 0xff, 0xbf, 0x8b, 0xf3, 0xab, 0x1c, 0x68, 0x6f, 0xbf,
-	0xdc, 0xf7, 0x7f, 0xbb, 0x48, 0xda, 0x77, 0xb8, 0x16, 0xd7, 0x5c, 0x02, 0x7c, 0x6f, 0xff, 0xf7,
-	0xb2, 0x17, 0x5a, 0xe4, 0xeb, 0xcd, 0xf4, 0x1d, 0x95, 0x53, 0xb8, 0xff, 0xd6, 0x1b, 0x7b, 0x37,
-	0x16, 0x01, 0x6d, 0x20, 0xba, 0x3a, 0x94, 0x53, 0xec, 0xa2, 0x38, 0x8a, 0x0e, 0x6f, 0xc3, 0x62,
-	0x7d, 0x8f, 0xa3, 0x17, 0x51, 0x1c, 0x29, 0xf0, 0x96, 0x10, 0x5e, 0x99, 0xdf, 0xff, 0x34, 0x07,
-	0x8b, 0x57, 0x5d, 0x09, 0x7c, 0xef, 0xae, 0x6f, 0x42, 0x25, 0xe9, 0xcc, 0xa8, 0x4b, 0xc0, 0xc3,
-	0x03, 0xd8, 0xbd, 0xa5, 0x97, 0x82, 0x44, 0xfa, 0xa2, 0x80, 0x1f, 0x9e, 0x78, 0xa8, 0x1d, 0xc0,
-	0x9d, 0xeb, 0x2e, 0x5c, 0xbe, 0x6f, 0xdf, 0xb4, 0x5f, 0xc2, 0xea, 0xdb, 0x2e, 0x27, 0xde, 0xd8,
-	0x52, 0xfd, 0x12, 0x96, 0xae, 0xbc, 0xa1, 0xf8, 0x7d, 0xce, 0xb6, 0x54, 0x79, 0x36, 0x3e, 0x50,
-	0x9e, 0x69, 0x7f, 0x99, 0x83, 0x87, 0xef, 0x7a, 0x5d, 0xf1, 0xc6, 0x76, 0xe0, 0x67, 0x40, 0xd2,
-	0x57, 0x28, 0x55, 0xdf, 0xe4, 0x76, 0x9c, 0x49, 0x69, 0x54, 0x1f, 0x7b, 0xf0, 0xe1, 0x3b, 0x5c,
-	0x6c, 0xbc, 0xb1, 0xe9, 0x77, 0x31, 0x1b, 0xbd, 0xe5, 0x72, 0xe3, 0x8d, 0x79, 0xfb, 0xe3, 0x1c,
-	0x7c, 0xfc, 0x6e, 0xd7, 0x1c, 0x6f, 0x6c, 0xfa, 0x97, 0xa1, 0x38, 0x70, 0x8d, 0x25, 0x6e, 0x6b,
-	0xff, 0x99, 0x83, 0xd2, 0x4e, 0xc0, 0xfa, 0xfe, 0x3e, 0xc5, 0x17, 0xba, 0xf7, 0xa1, 0xec, 0x44,
-	0x77, 0x92, 0x22, 0xc7, 0x15, 0xfc, 0x71, 0x87, 0x94, 0x35, 0x6d, 0xd2, 0x84, 0x6a, 0x02, 0xc1,
-	0x0f, 0x16, 0xf2, 0x43, 0x72, 0x72, 0xdf, 0x36, 0x45, 0xb8, 0x16, 0xdf, 0x70, 0xc2, 0x2f, 0xc6,
-	0x15, 0x27, 0xdd, 0x24, 0x77, 0xa1, 0x24, 0x9e, 0xe3, 0xa2, 0x02, 0x7f, 0x1c, 0x9d, 0x89, 0x02,
-	0xff, 0x50, 0x16, 0xf8, 0xe9, 0x2f, 0x07, 0x13, 0xa8, 0x8c, 0xdb, 0xda, 0x6f, 0x43, 0x25, 0xc3,
-	0x4d, 0x0a, 0x30, 0x7e, 0xd8, 0x3a, 0xa8, 0xdd, 0x22, 0x35, 0x28, 0x37, 0x0e, 0x5b, 0x07, 0xc6,
-	0xfa, 0x8e, 0x71, 0xb8, 0xd9, 0xde, 0xad, 0xe5, 0xc8, 0x0c, 0x54, 0xa4, 0xe4, 0xb1, 0x12, 0x8d,
-	0x69, 0x7f, 0x38, 0x06, 0x93, 0xd8, 0xcf, 0xcc, 0x17, 0x05, 0x39, 0xdc, 0xf8, 0x8b, 0xc2, 0x8f,
-	0xa1, 0x60, 0xb1, 0x5e, 0xcf, 0x54, 0xbf, 0x72, 0x18, 0x1a, 0x63, 0x7a, 0xa4, 0xe1, 0x96, 0x44,
-	0xea, 0x91, 0x09, 0x59, 0x83, 0x42, 0x4f, 0xaa, 0xd4, 0x35, 0x80, 0xb9, 0x51, 0x33, 0xa4, 0x47,
-	0xa0, 0xd4, 0x07, 0x95, 0x89, 0x6b, 0x3f, 0xa8, 0x68, 0x5f, 0xc3, 0xec, 0x08, 0xc7, 0x64, 0x1a,
-	0x4a, 0x9b, 0xdb, 0xdb, 0xc6, 0x7e, 0x63, 0xff, 0x45, 0x43, 0x3f, 0xaa, 0xdd, 0x22, 0x04, 0xaa,
-	0x7a, 0x63, 0xbf, 0xf5, 0xba, 0x11, 0xcb, 0x72, 0x02, 0x74, 0xd4, 0x68, 0xc7, 0x82, 0x31, 0xed,
-	0x1b, 0x80, 0xd7, 0xa6, 0xdb, 0xa7, 0x87, 0x66, 0x60, 0xf6, 0xc8, 0x5d, 0x18, 0x67, 0x5e, 0x5f,
-	0x7d, 0xda, 0x2b, 0x67, 0x6e, 0x50, 0x0b, 0x05, 0xf9, 0x51, 0xfa, 0xd3, 0x40, 0xb5, 0xbe, 0xb0,
-	0xa6, 0x7e, 0xd2, 0x84, 0x14, 0x62, 0x19, 0xd6, 0x70, 0x9d, 0x25, 0x48, 0xfb, 0xfb, 0x31, 0xa8,
-	0x1e, 0x32, 0x4f, 0xbf, 0x38, 0x64, 0xe7, 0x34, 0xd8, 0x36, 0xb9, 0x79, 0x63, 0x41, 0xad, 0x67,
-	0x4f, 0xdb, 0x09, 0xec, 0xce, 0x7a, 0xea, 0x47, 0x38, 0x69, 0xaf, 0x6b, 0x7a, 0x18, 0x3a, 0xfb,
-	0xd4, 0x0c, 0xfb, 0x01, 0xee, 0xeb, 0xd1, 0x97, 0x48, 0xd5, 0x6b, 0x6e, 0x5f, 0x58, 0x19, 0x3d,
-	0x6a, 0x7a, 0x86, 0x7d, 0xdc, 0xc3, 0x12, 0x2e, 0xa7, 0x57, 0x03, 0xc9, 0xb6, 0x4f, 0x4d, 0x6f,
-	0xfb, 0xb8, 0x27, 0x72, 0xf6, 0x95, 0x9c, 0x64, 0x0e, 0x6a, 0x23, 0x2e, 0x99, 0xde, 0x81, 0xc5,
-	0xac, 0xd4, 0xd8, 0x6e, 0xec, 0x35, 0xf7, 0x9b, 0xed, 0x86, 0x5e, 0xcb, 0x91, 0x25, 0x98, 0x1f,
-	0xd0, 0x6e, 0x6e, 0x6d, 0x35, 0x8e, 0xc4, 0x0a, 0x15, 0x60, 0xb2, 0xd1, 0xf3, 0xf9, 0x65, 0xfd,
-	0x2f, 0xe6, 0xa0, 0xd0, 0x92, 0x03, 0x24, 0xdb, 0x00, 0xdb, 0x4e, 0x68, 0x1e, 0xbb, 0xb4, 0xe5,
-	0x72, 0x52, 0x8d, 0x07, 0x8e, 0xc8, 0xe5, 0x81, 0xb6, 0xb6, 0xf0, 0xab, 0x7f, 0xfe, 0xd7, 0xdf,
-	0x8c, 0xd5, 0xb4, 0xd2, 0xa3, 0xb3, 0xf5, 0x47, 0xca, 0xee, 0xcb, 0xdc, 0xa7, 0xe4, 0x25, 0x94,
-	0x74, 0x4a, 0xbd, 0x77, 0xa5, 0xb9, 0x8d, 0x34, 0x33, 0x5a, 0x59, 0xd0, 0x44, 0x86, 0x82, 0xa7,
-	0x01, 0x25, 0x55, 0x4a, 0xd1, 0x96, 0xd7, 0x27, 0x99, 0xc0, 0x19, 0x62, 0x59, 0x44, 0x16, 0xa2,
-	0x55, 0x04, 0x4b, 0x43, 0x3a, 0xf7, 0xfa, 0x82, 0x66, 0x17, 0x2a, 0xf1, 0x91, 0xfb, 0x0e, 0x44,
-	0x4b, 0x48, 0x34, 0xab, 0x55, 0x53, 0xa3, 0x52, 0x4c, 0x5b, 0x30, 0xb5, 0x4d, 0x5d, 0xfa, 0xde,
-	0xdd, 0x89, 0x8d, 0x04, 0x49, 0x13, 0x40, 0xdd, 0xec, 0x6b, 0xf5, 0x39, 0xa9, 0x65, 0x7e, 0x4f,
-	0xb6, 0x1f, 0x76, 0xaf, 0xef, 0x4f, 0x62, 0x29, 0xa8, 0x5a, 0x50, 0x8e, 0xaf, 0xf5, 0x09, 0x32,
-	0x92, 0xb9, 0xc3, 0x8f, 0xe2, 0x21, 0xba, 0x15, 0xa4, 0x9b, 0xd7, 0x6a, 0x48, 0x97, 0xb2, 0x16,
-	0x84, 0xbf, 0x07, 0xd3, 0xe9, 0x0b, 0x7a, 0x82, 0x33, 0xb9, 0x9c, 0x99, 0xd6, 0x0c, 0xd1, 0xde,
-	0x45, 0xda, 0x45, 0x6d, 0x56, 0xd0, 0x0e, 0x70, 0x08, 0xe6, 0xaf, 0xa0, 0xf0, 0xd2, 0x65, 0xe7,
-	0x9b, 0xb6, 0x4d, 0x2a, 0x99, 0x0f, 0x94, 0xd7, 0x47, 0x95, 0xb2, 0x91, 0x51, 0x05, 0xa2, 0xa5,
-	0xe3, 0x05, 0x85, 0xb7, 0x91, 0x64, 0x26, 0x2d, 0x31, 0x13, 0x3c, 0x47, 0x50, 0x8d, 0xaf, 0xbe,
-	0x6e, 0x9d, 0x50, 0xeb, 0x74, 0x28, 0x40, 0x93, 0x69, 0x8c, 0x81, 0xda, 0x07, 0x48, 0x78, 0x5b,
-	0x23, 0x82, 0x30, 0x6b, 0x2f, 0x48, 0xf7, 0xa1, 0x24, 0x63, 0xee, 0x90, 0x79, 0xcd, 0x4e, 0x6a,
-	0x21, 0xe2, 0xd3, 0x64, 0xa8, 0x8b, 0xcb, 0xc8, 0x38, 0xa7, 0x4d, 0x27, 0x01, 0x8b, 0xc6, 0x6a,
-	0x61, 0x55, 0xe4, 0xbd, 0x3b, 0x5f, 0x66, 0x61, 0xd3, 0xd6, 0x82, 0x50, 0x87, 0xca, 0x0e, 0xe5,
-	0xa9, 0x0b, 0x94, 0x83, 0x63, 0x9e, 0x1d, 0x71, 0x61, 0x4b, 0xbb, 0x83, 0x94, 0x0b, 0xda, 0x8c,
-	0xa0, 0xcc, 0xd8, 0x0b, 0xce, 0x9f, 0x42, 0x5e, 0xa7, 0xc7, 0x8c, 0xbd, 0x7d, 0x87, 0xcf, 0x23,
-	0xcf, 0xb4, 0x06, 0x72, 0x87, 0x0b, 0x1b, 0x41, 0xf0, 0x0a, 0x66, 0xb6, 0x98, 0xeb, 0x52, 0x2b,
-	0xfd, 0xda, 0xee, 0x6d, 0x5c, 0xab, 0xc8, 0xb5, 0xac, 0xcd, 0x0b, 0xae, 0x21, 0x73, 0x41, 0xfb,
-	0x33, 0xa8, 0xed, 0x50, 0x9e, 0x7d, 0xf3, 0x9f, 0xdd, 0xac, 0x0b, 0x03, 0xbf, 0xe3, 0x53, 0x28,
-	0xed, 0x1e, 0x72, 0x2f, 0x69, 0x73, 0x6a, 0xbc, 0x19, 0xad, 0xa0, 0x3e, 0x85, 0xb9, 0x1d, 0xca,
-	0x87, 0xdf, 0x35, 0x8e, 0xda, 0x78, 0xc9, 0x0f, 0x53, 0x87, 0xf0, 0xda, 0x87, 0xe8, 0xe8, 0x03,
-	0x6d, 0x51, 0x39, 0x1a, 0x42, 0x08, 0x67, 0x01, 0xdc, 0xde, 0x0a, 0xa8, 0xc9, 0x69, 0x3b, 0x30,
-	0x3b, 0x1d, 0xc7, 0x3a, 0xb2, 0x4e, 0xa8, 0xdd, 0x77, 0xc5, 0xa1, 0x7e, 0x6f, 0x2d, 0xf3, 0xb3,
-	0xe0, 0x21, 0xc0, 0xd0, 0xac, 0x7d, 0x8c, 0x0e, 0x57, 0xb5, 0x15, 0x9c, 0xb5, 0xd1, 0xac, 0xca,
-	0xa7, 0xdc, 0x29, 0x37, 0xed, 0xf3, 0x0a, 0x56, 0xe1, 0xb3, 0x03, 0xb3, 0x99, 0x1e, 0xfd, 0x6e,
-	0x9f, 0xf6, 0x69, 0x48, 0x56, 0x46, 0xfa, 0x93, 0xca, 0x21, 0x5f, 0x1a, 0xfa, 0xba, 0xa3, 0xdd,
-	0x1e, 0x1a, 0x9f, 0x34, 0x50, 0x7e, 0x32, 0xbd, 0xf8, 0x5f, 0xfb, 0x19, 0xc1, 0x26, 0xfc, 0xfc,
-	0x16, 0xd4, 0xe4, 0x76, 0x4e, 0x55, 0xdd, 0x57, 0x6f, 0xb7, 0x04, 0xa4, 0xdd, 0x7a, 0x9c, 0x23,
-	0xdf, 0xc2, 0xfc, 0x21, 0x0d, 0x3a, 0x2c, 0xe8, 0x61, 0x31, 0xd6, 0xf2, 0x69, 0x30, 0xc8, 0x80,
-	0x8a, 0xa1, 0x9e, 0x3d, 0xc0, 0x9e, 0xdd, 0xd5, 0x96, 0x44, 0xcf, 0x46, 0x52, 0xc8, 0xc3, 0xa7,
-	0x24, 0x0f, 0x23, 0x59, 0xa5, 0xbe, 0x8d, 0x34, 0x93, 0xa3, 0x52, 0x86, 0x82, 0xaa, 0x0d, 0xa5,
-	0x1d, 0xca, 0x1b, 0x17, 0x1c, 0xab, 0x34, 0x92, 0x8c, 0x28, 0x29, 0xfc, 0x96, 0xe7, 0xa2, 0x4a,
-	0x4e, 0xa7, 0xbc, 0x1f, 0x78, 0xa8, 0x09, 0xb3, 0xac, 0x29, 0x0e, 0xc1, 0xfa, 0x2d, 0xfe, 0xd0,
-	0x49, 0xbe, 0x0a, 0xc0, 0x37, 0xb2, 0x47, 0x94, 0x13, 0x51, 0x10, 0xe2, 0x6f, 0xd8, 0xb3, 0xaa,
-	0xeb, 0x73, 0xc3, 0x10, 0x8d, 0xa0, 0xf7, 0x61, 0x69, 0x87, 0xf2, 0xbd, 0xd1, 0xb7, 0xf4, 0xb3,
-	0x49, 0x62, 0x25, 0xfb, 0x2b, 0xb5, 0xcc, 0x6f, 0x07, 0xb4, 0x87, 0xe8, 0x49, 0xd3, 0x3e, 0x50,
-	0x43, 0x18, 0xcd, 0x28, 0x3c, 0x9e, 0xc0, 0xfc, 0x48, 0xfd, 0xfb, 0x78, 0xcb, 0xac, 0xed, 0x48,
-	0x36, 0xb9, 0x20, 0x22, 0xc7, 0x27, 0x35, 0xea, 0x80, 0x87, 0xdb, 0x57, 0x94, 0xb1, 0x43, 0x59,
-	0x3e, 0x51, 0x7f, 0x99, 0xfb, 0xf4, 0xc5, 0xb7, 0xb0, 0xc2, 0x82, 0x2e, 0xda, 0x5a, 0x2c, 0xb0,
-	0xd7, 0xe4, 0xff, 0x2f, 0x10, 0x71, 0xbd, 0xa8, 0xbc, 0xc6, 0xb6, 0x28, 0x20, 0x5b, 0x7b, 0xed,
-	0x6f, 0x1e, 0x75, 0x1d, 0x7e, 0xd2, 0x3f, 0x16, 0xcb, 0xfe, 0x28, 0x32, 0x79, 0x24, 0x4d, 0x3e,
-	0x53, 0xff, 0x25, 0xc1, 0xd9, 0x93, 0x47, 0x5d, 0x16, 0xfd, 0xd7, 0x08, 0x87, 0xb9, 0xc3, 0xb1,
-	0xe3, 0x3c, 0x6a, 0x3e, 0xff, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x20, 0x58, 0x9a, 0x83, 0x3e,
-	0x41, 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
-
-// OpenoltClient is the client API for Openolt service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type OpenoltClient interface {
-	DisableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
-	ReenableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
-	ActivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
-	DeactivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
-	DeleteOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
-	OmciMsgOut(ctx context.Context, in *OmciMsg, opts ...grpc.CallOption) (*Empty, error)
-	OnuPacketOut(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*Empty, error)
-	UplinkPacketOut(ctx context.Context, in *UplinkPacket, opts ...grpc.CallOption) (*Empty, error)
-	FlowAdd(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error)
-	FlowRemove(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error)
-	HeartbeatCheck(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Heartbeat, error)
-	EnablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error)
-	DisablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error)
-	GetDeviceInfo(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*DeviceInfo, error)
-	Reboot(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
-	CollectStatistics(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
-	GetOnuStatistics(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuStatistics, error)
-	GetGemPortStatistics(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*GemPortStatistics, error)
-	CreateTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error)
-	RemoveTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error)
-	CreateTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error)
-	RemoveTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error)
-	EnableIndication(ctx context.Context, in *Empty, opts ...grpc.CallOption) (Openolt_EnableIndicationClient, error)
-	PerformGroupOperation(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error)
-	DeleteGroup(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error)
-	GetExtValue(ctx context.Context, in *ValueParam, opts ...grpc.CallOption) (*common.ReturnValues, error)
-	OnuItuPonAlarmSet(ctx context.Context, in *config.OnuItuPonAlarm, opts ...grpc.CallOption) (*Empty, error)
-	GetLogicalOnuDistanceZero(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error)
-	GetLogicalOnuDistance(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error)
-	GetPonRxPower(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*PonRxPowerData, error)
-}
-
-type openoltClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewOpenoltClient(cc *grpc.ClientConn) OpenoltClient {
-	return &openoltClient{cc}
-}
-
-func (c *openoltClient) DisableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/DisableOlt", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) ReenableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/ReenableOlt", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) ActivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/ActivateOnu", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) DeactivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeactivateOnu", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) DeleteOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeleteOnu", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) OmciMsgOut(ctx context.Context, in *OmciMsg, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/OmciMsgOut", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) OnuPacketOut(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/OnuPacketOut", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) UplinkPacketOut(ctx context.Context, in *UplinkPacket, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/UplinkPacketOut", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) FlowAdd(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/FlowAdd", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) FlowRemove(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/FlowRemove", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) HeartbeatCheck(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Heartbeat, error) {
-	out := new(Heartbeat)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/HeartbeatCheck", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) EnablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/EnablePonIf", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) DisablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/DisablePonIf", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetDeviceInfo(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*DeviceInfo, error) {
-	out := new(DeviceInfo)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetDeviceInfo", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) Reboot(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/Reboot", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) CollectStatistics(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/CollectStatistics", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetOnuStatistics(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuStatistics, error) {
-	out := new(OnuStatistics)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetOnuStatistics", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetGemPortStatistics(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*GemPortStatistics, error) {
-	out := new(GemPortStatistics)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetGemPortStatistics", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) CreateTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/CreateTrafficSchedulers", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) RemoveTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/RemoveTrafficSchedulers", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) CreateTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/CreateTrafficQueues", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) RemoveTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/RemoveTrafficQueues", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) EnableIndication(ctx context.Context, in *Empty, opts ...grpc.CallOption) (Openolt_EnableIndicationClient, error) {
-	stream, err := c.cc.NewStream(ctx, &_Openolt_serviceDesc.Streams[0], "/openolt.Openolt/EnableIndication", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &openoltEnableIndicationClient{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 Openolt_EnableIndicationClient interface {
-	Recv() (*Indication, error)
-	grpc.ClientStream
-}
-
-type openoltEnableIndicationClient struct {
-	grpc.ClientStream
-}
-
-func (x *openoltEnableIndicationClient) Recv() (*Indication, error) {
-	m := new(Indication)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *openoltClient) PerformGroupOperation(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/PerformGroupOperation", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) DeleteGroup(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeleteGroup", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetExtValue(ctx context.Context, in *ValueParam, opts ...grpc.CallOption) (*common.ReturnValues, error) {
-	out := new(common.ReturnValues)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetExtValue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) OnuItuPonAlarmSet(ctx context.Context, in *config.OnuItuPonAlarm, opts ...grpc.CallOption) (*Empty, error) {
-	out := new(Empty)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/OnuItuPonAlarmSet", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetLogicalOnuDistanceZero(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error) {
-	out := new(OnuLogicalDistance)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetLogicalOnuDistanceZero", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetLogicalOnuDistance(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error) {
-	out := new(OnuLogicalDistance)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetLogicalOnuDistance", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *openoltClient) GetPonRxPower(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*PonRxPowerData, error) {
-	out := new(PonRxPowerData)
-	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetPonRxPower", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// OpenoltServer is the server API for Openolt service.
-type OpenoltServer interface {
-	DisableOlt(context.Context, *Empty) (*Empty, error)
-	ReenableOlt(context.Context, *Empty) (*Empty, error)
-	ActivateOnu(context.Context, *Onu) (*Empty, error)
-	DeactivateOnu(context.Context, *Onu) (*Empty, error)
-	DeleteOnu(context.Context, *Onu) (*Empty, error)
-	OmciMsgOut(context.Context, *OmciMsg) (*Empty, error)
-	OnuPacketOut(context.Context, *OnuPacket) (*Empty, error)
-	UplinkPacketOut(context.Context, *UplinkPacket) (*Empty, error)
-	FlowAdd(context.Context, *Flow) (*Empty, error)
-	FlowRemove(context.Context, *Flow) (*Empty, error)
-	HeartbeatCheck(context.Context, *Empty) (*Heartbeat, error)
-	EnablePonIf(context.Context, *Interface) (*Empty, error)
-	DisablePonIf(context.Context, *Interface) (*Empty, error)
-	GetDeviceInfo(context.Context, *Empty) (*DeviceInfo, error)
-	Reboot(context.Context, *Empty) (*Empty, error)
-	CollectStatistics(context.Context, *Empty) (*Empty, error)
-	GetOnuStatistics(context.Context, *Onu) (*OnuStatistics, error)
-	GetGemPortStatistics(context.Context, *OnuPacket) (*GemPortStatistics, error)
-	CreateTrafficSchedulers(context.Context, *tech_profile.TrafficSchedulers) (*Empty, error)
-	RemoveTrafficSchedulers(context.Context, *tech_profile.TrafficSchedulers) (*Empty, error)
-	CreateTrafficQueues(context.Context, *tech_profile.TrafficQueues) (*Empty, error)
-	RemoveTrafficQueues(context.Context, *tech_profile.TrafficQueues) (*Empty, error)
-	EnableIndication(*Empty, Openolt_EnableIndicationServer) error
-	PerformGroupOperation(context.Context, *Group) (*Empty, error)
-	DeleteGroup(context.Context, *Group) (*Empty, error)
-	GetExtValue(context.Context, *ValueParam) (*common.ReturnValues, error)
-	OnuItuPonAlarmSet(context.Context, *config.OnuItuPonAlarm) (*Empty, error)
-	GetLogicalOnuDistanceZero(context.Context, *Onu) (*OnuLogicalDistance, error)
-	GetLogicalOnuDistance(context.Context, *Onu) (*OnuLogicalDistance, error)
-	GetPonRxPower(context.Context, *Onu) (*PonRxPowerData, error)
-}
-
-// UnimplementedOpenoltServer can be embedded to have forward compatible implementations.
-type UnimplementedOpenoltServer struct {
-}
-
-func (*UnimplementedOpenoltServer) DisableOlt(ctx context.Context, req *Empty) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DisableOlt not implemented")
-}
-func (*UnimplementedOpenoltServer) ReenableOlt(ctx context.Context, req *Empty) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ReenableOlt not implemented")
-}
-func (*UnimplementedOpenoltServer) ActivateOnu(ctx context.Context, req *Onu) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ActivateOnu not implemented")
-}
-func (*UnimplementedOpenoltServer) DeactivateOnu(ctx context.Context, req *Onu) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DeactivateOnu not implemented")
-}
-func (*UnimplementedOpenoltServer) DeleteOnu(ctx context.Context, req *Onu) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DeleteOnu not implemented")
-}
-func (*UnimplementedOpenoltServer) OmciMsgOut(ctx context.Context, req *OmciMsg) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method OmciMsgOut not implemented")
-}
-func (*UnimplementedOpenoltServer) OnuPacketOut(ctx context.Context, req *OnuPacket) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method OnuPacketOut not implemented")
-}
-func (*UnimplementedOpenoltServer) UplinkPacketOut(ctx context.Context, req *UplinkPacket) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UplinkPacketOut not implemented")
-}
-func (*UnimplementedOpenoltServer) FlowAdd(ctx context.Context, req *Flow) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method FlowAdd not implemented")
-}
-func (*UnimplementedOpenoltServer) FlowRemove(ctx context.Context, req *Flow) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method FlowRemove not implemented")
-}
-func (*UnimplementedOpenoltServer) HeartbeatCheck(ctx context.Context, req *Empty) (*Heartbeat, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method HeartbeatCheck not implemented")
-}
-func (*UnimplementedOpenoltServer) EnablePonIf(ctx context.Context, req *Interface) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method EnablePonIf not implemented")
-}
-func (*UnimplementedOpenoltServer) DisablePonIf(ctx context.Context, req *Interface) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DisablePonIf not implemented")
-}
-func (*UnimplementedOpenoltServer) GetDeviceInfo(ctx context.Context, req *Empty) (*DeviceInfo, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceInfo not implemented")
-}
-func (*UnimplementedOpenoltServer) Reboot(ctx context.Context, req *Empty) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method Reboot not implemented")
-}
-func (*UnimplementedOpenoltServer) CollectStatistics(ctx context.Context, req *Empty) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CollectStatistics not implemented")
-}
-func (*UnimplementedOpenoltServer) GetOnuStatistics(ctx context.Context, req *Onu) (*OnuStatistics, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetOnuStatistics not implemented")
-}
-func (*UnimplementedOpenoltServer) GetGemPortStatistics(ctx context.Context, req *OnuPacket) (*GemPortStatistics, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetGemPortStatistics not implemented")
-}
-func (*UnimplementedOpenoltServer) CreateTrafficSchedulers(ctx context.Context, req *tech_profile.TrafficSchedulers) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateTrafficSchedulers not implemented")
-}
-func (*UnimplementedOpenoltServer) RemoveTrafficSchedulers(ctx context.Context, req *tech_profile.TrafficSchedulers) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method RemoveTrafficSchedulers not implemented")
-}
-func (*UnimplementedOpenoltServer) CreateTrafficQueues(ctx context.Context, req *tech_profile.TrafficQueues) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateTrafficQueues not implemented")
-}
-func (*UnimplementedOpenoltServer) RemoveTrafficQueues(ctx context.Context, req *tech_profile.TrafficQueues) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method RemoveTrafficQueues not implemented")
-}
-func (*UnimplementedOpenoltServer) EnableIndication(req *Empty, srv Openolt_EnableIndicationServer) error {
-	return status.Errorf(codes.Unimplemented, "method EnableIndication not implemented")
-}
-func (*UnimplementedOpenoltServer) PerformGroupOperation(ctx context.Context, req *Group) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method PerformGroupOperation not implemented")
-}
-func (*UnimplementedOpenoltServer) DeleteGroup(ctx context.Context, req *Group) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DeleteGroup not implemented")
-}
-func (*UnimplementedOpenoltServer) GetExtValue(ctx context.Context, req *ValueParam) (*common.ReturnValues, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
-}
-func (*UnimplementedOpenoltServer) OnuItuPonAlarmSet(ctx context.Context, req *config.OnuItuPonAlarm) (*Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method OnuItuPonAlarmSet not implemented")
-}
-func (*UnimplementedOpenoltServer) GetLogicalOnuDistanceZero(ctx context.Context, req *Onu) (*OnuLogicalDistance, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalOnuDistanceZero not implemented")
-}
-func (*UnimplementedOpenoltServer) GetLogicalOnuDistance(ctx context.Context, req *Onu) (*OnuLogicalDistance, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalOnuDistance not implemented")
-}
-func (*UnimplementedOpenoltServer) GetPonRxPower(ctx context.Context, req *Onu) (*PonRxPowerData, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetPonRxPower not implemented")
-}
-
-func RegisterOpenoltServer(s *grpc.Server, srv OpenoltServer) {
-	s.RegisterService(&_Openolt_serviceDesc, srv)
-}
-
-func _Openolt_DisableOlt_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).DisableOlt(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/DisableOlt",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).DisableOlt(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_ReenableOlt_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).ReenableOlt(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/ReenableOlt",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).ReenableOlt(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_ActivateOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).ActivateOnu(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/ActivateOnu",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).ActivateOnu(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_DeactivateOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).DeactivateOnu(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/DeactivateOnu",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).DeactivateOnu(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_DeleteOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).DeleteOnu(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/DeleteOnu",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).DeleteOnu(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_OmciMsgOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(OmciMsg)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).OmciMsgOut(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/OmciMsgOut",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).OmciMsgOut(ctx, req.(*OmciMsg))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_OnuPacketOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(OnuPacket)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).OnuPacketOut(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/OnuPacketOut",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).OnuPacketOut(ctx, req.(*OnuPacket))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_UplinkPacketOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(UplinkPacket)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).UplinkPacketOut(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/UplinkPacketOut",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).UplinkPacketOut(ctx, req.(*UplinkPacket))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_FlowAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Flow)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).FlowAdd(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/FlowAdd",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).FlowAdd(ctx, req.(*Flow))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_FlowRemove_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Flow)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).FlowRemove(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/FlowRemove",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).FlowRemove(ctx, req.(*Flow))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_HeartbeatCheck_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).HeartbeatCheck(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/HeartbeatCheck",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).HeartbeatCheck(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_EnablePonIf_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Interface)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).EnablePonIf(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/EnablePonIf",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).EnablePonIf(ctx, req.(*Interface))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_DisablePonIf_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Interface)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).DisablePonIf(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/DisablePonIf",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).DisablePonIf(ctx, req.(*Interface))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetDeviceInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetDeviceInfo(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetDeviceInfo",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetDeviceInfo(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_Reboot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).Reboot(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/Reboot",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).Reboot(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_CollectStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).CollectStatistics(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/CollectStatistics",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).CollectStatistics(ctx, req.(*Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetOnuStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetOnuStatistics(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetOnuStatistics",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetOnuStatistics(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetGemPortStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(OnuPacket)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetGemPortStatistics(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetGemPortStatistics",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetGemPortStatistics(ctx, req.(*OnuPacket))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_CreateTrafficSchedulers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(tech_profile.TrafficSchedulers)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).CreateTrafficSchedulers(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/CreateTrafficSchedulers",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).CreateTrafficSchedulers(ctx, req.(*tech_profile.TrafficSchedulers))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_RemoveTrafficSchedulers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(tech_profile.TrafficSchedulers)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).RemoveTrafficSchedulers(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/RemoveTrafficSchedulers",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).RemoveTrafficSchedulers(ctx, req.(*tech_profile.TrafficSchedulers))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_CreateTrafficQueues_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(tech_profile.TrafficQueues)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).CreateTrafficQueues(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/CreateTrafficQueues",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).CreateTrafficQueues(ctx, req.(*tech_profile.TrafficQueues))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_RemoveTrafficQueues_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(tech_profile.TrafficQueues)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).RemoveTrafficQueues(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/RemoveTrafficQueues",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).RemoveTrafficQueues(ctx, req.(*tech_profile.TrafficQueues))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_EnableIndication_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(Empty)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(OpenoltServer).EnableIndication(m, &openoltEnableIndicationServer{stream})
-}
-
-type Openolt_EnableIndicationServer interface {
-	Send(*Indication) error
-	grpc.ServerStream
-}
-
-type openoltEnableIndicationServer struct {
-	grpc.ServerStream
-}
-
-func (x *openoltEnableIndicationServer) Send(m *Indication) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func _Openolt_PerformGroupOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Group)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).PerformGroupOperation(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/PerformGroupOperation",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).PerformGroupOperation(ctx, req.(*Group))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_DeleteGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Group)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).DeleteGroup(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/DeleteGroup",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).DeleteGroup(ctx, req.(*Group))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ValueParam)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetExtValue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetExtValue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetExtValue(ctx, req.(*ValueParam))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_OnuItuPonAlarmSet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(config.OnuItuPonAlarm)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).OnuItuPonAlarmSet(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/OnuItuPonAlarmSet",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).OnuItuPonAlarmSet(ctx, req.(*config.OnuItuPonAlarm))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetLogicalOnuDistanceZero_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetLogicalOnuDistanceZero(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetLogicalOnuDistanceZero",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetLogicalOnuDistanceZero(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetLogicalOnuDistance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetLogicalOnuDistance(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetLogicalOnuDistance",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetLogicalOnuDistance(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Openolt_GetPonRxPower_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Onu)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(OpenoltServer).GetPonRxPower(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/openolt.Openolt/GetPonRxPower",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(OpenoltServer).GetPonRxPower(ctx, req.(*Onu))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Openolt_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "openolt.Openolt",
-	HandlerType: (*OpenoltServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "DisableOlt",
-			Handler:    _Openolt_DisableOlt_Handler,
-		},
-		{
-			MethodName: "ReenableOlt",
-			Handler:    _Openolt_ReenableOlt_Handler,
-		},
-		{
-			MethodName: "ActivateOnu",
-			Handler:    _Openolt_ActivateOnu_Handler,
-		},
-		{
-			MethodName: "DeactivateOnu",
-			Handler:    _Openolt_DeactivateOnu_Handler,
-		},
-		{
-			MethodName: "DeleteOnu",
-			Handler:    _Openolt_DeleteOnu_Handler,
-		},
-		{
-			MethodName: "OmciMsgOut",
-			Handler:    _Openolt_OmciMsgOut_Handler,
-		},
-		{
-			MethodName: "OnuPacketOut",
-			Handler:    _Openolt_OnuPacketOut_Handler,
-		},
-		{
-			MethodName: "UplinkPacketOut",
-			Handler:    _Openolt_UplinkPacketOut_Handler,
-		},
-		{
-			MethodName: "FlowAdd",
-			Handler:    _Openolt_FlowAdd_Handler,
-		},
-		{
-			MethodName: "FlowRemove",
-			Handler:    _Openolt_FlowRemove_Handler,
-		},
-		{
-			MethodName: "HeartbeatCheck",
-			Handler:    _Openolt_HeartbeatCheck_Handler,
-		},
-		{
-			MethodName: "EnablePonIf",
-			Handler:    _Openolt_EnablePonIf_Handler,
-		},
-		{
-			MethodName: "DisablePonIf",
-			Handler:    _Openolt_DisablePonIf_Handler,
-		},
-		{
-			MethodName: "GetDeviceInfo",
-			Handler:    _Openolt_GetDeviceInfo_Handler,
-		},
-		{
-			MethodName: "Reboot",
-			Handler:    _Openolt_Reboot_Handler,
-		},
-		{
-			MethodName: "CollectStatistics",
-			Handler:    _Openolt_CollectStatistics_Handler,
-		},
-		{
-			MethodName: "GetOnuStatistics",
-			Handler:    _Openolt_GetOnuStatistics_Handler,
-		},
-		{
-			MethodName: "GetGemPortStatistics",
-			Handler:    _Openolt_GetGemPortStatistics_Handler,
-		},
-		{
-			MethodName: "CreateTrafficSchedulers",
-			Handler:    _Openolt_CreateTrafficSchedulers_Handler,
-		},
-		{
-			MethodName: "RemoveTrafficSchedulers",
-			Handler:    _Openolt_RemoveTrafficSchedulers_Handler,
-		},
-		{
-			MethodName: "CreateTrafficQueues",
-			Handler:    _Openolt_CreateTrafficQueues_Handler,
-		},
-		{
-			MethodName: "RemoveTrafficQueues",
-			Handler:    _Openolt_RemoveTrafficQueues_Handler,
-		},
-		{
-			MethodName: "PerformGroupOperation",
-			Handler:    _Openolt_PerformGroupOperation_Handler,
-		},
-		{
-			MethodName: "DeleteGroup",
-			Handler:    _Openolt_DeleteGroup_Handler,
-		},
-		{
-			MethodName: "GetExtValue",
-			Handler:    _Openolt_GetExtValue_Handler,
-		},
-		{
-			MethodName: "OnuItuPonAlarmSet",
-			Handler:    _Openolt_OnuItuPonAlarmSet_Handler,
-		},
-		{
-			MethodName: "GetLogicalOnuDistanceZero",
-			Handler:    _Openolt_GetLogicalOnuDistanceZero_Handler,
-		},
-		{
-			MethodName: "GetLogicalOnuDistance",
-			Handler:    _Openolt_GetLogicalOnuDistance_Handler,
-		},
-		{
-			MethodName: "GetPonRxPower",
-			Handler:    _Openolt_GetPonRxPower_Handler,
-		},
-	},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "EnableIndication",
-			Handler:       _Openolt_EnableIndication_Handler,
-			ServerStreams: true,
-		},
-	},
-	Metadata: "voltha_protos/openolt.proto",
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/tech_profile/tech_profile.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/tech_profile/tech_profile.pb.go
deleted file mode 100644
index fc70ea8..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/tech_profile/tech_profile.pb.go
+++ /dev/null
@@ -1,2089 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/tech_profile.proto
-
-package tech_profile
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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 Direction int32
-
-const (
-	Direction_UPSTREAM      Direction = 0
-	Direction_DOWNSTREAM    Direction = 1
-	Direction_BIDIRECTIONAL Direction = 2
-)
-
-var Direction_name = map[int32]string{
-	0: "UPSTREAM",
-	1: "DOWNSTREAM",
-	2: "BIDIRECTIONAL",
-}
-
-var Direction_value = map[string]int32{
-	"UPSTREAM":      0,
-	"DOWNSTREAM":    1,
-	"BIDIRECTIONAL": 2,
-}
-
-func (x Direction) String() string {
-	return proto.EnumName(Direction_name, int32(x))
-}
-
-func (Direction) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{0}
-}
-
-type SchedulingPolicy int32
-
-const (
-	SchedulingPolicy_WRR            SchedulingPolicy = 0
-	SchedulingPolicy_StrictPriority SchedulingPolicy = 1
-	SchedulingPolicy_Hybrid         SchedulingPolicy = 2
-)
-
-var SchedulingPolicy_name = map[int32]string{
-	0: "WRR",
-	1: "StrictPriority",
-	2: "Hybrid",
-}
-
-var SchedulingPolicy_value = map[string]int32{
-	"WRR":            0,
-	"StrictPriority": 1,
-	"Hybrid":         2,
-}
-
-func (x SchedulingPolicy) String() string {
-	return proto.EnumName(SchedulingPolicy_name, int32(x))
-}
-
-func (SchedulingPolicy) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{1}
-}
-
-type AdditionalBW int32
-
-const (
-	AdditionalBW_AdditionalBW_None       AdditionalBW = 0
-	AdditionalBW_AdditionalBW_NA         AdditionalBW = 1
-	AdditionalBW_AdditionalBW_BestEffort AdditionalBW = 2
-	AdditionalBW_AdditionalBW_Auto       AdditionalBW = 3
-)
-
-var AdditionalBW_name = map[int32]string{
-	0: "AdditionalBW_None",
-	1: "AdditionalBW_NA",
-	2: "AdditionalBW_BestEffort",
-	3: "AdditionalBW_Auto",
-}
-
-var AdditionalBW_value = map[string]int32{
-	"AdditionalBW_None":       0,
-	"AdditionalBW_NA":         1,
-	"AdditionalBW_BestEffort": 2,
-	"AdditionalBW_Auto":       3,
-}
-
-func (x AdditionalBW) String() string {
-	return proto.EnumName(AdditionalBW_name, int32(x))
-}
-
-func (AdditionalBW) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{2}
-}
-
-type DiscardPolicy int32
-
-const (
-	DiscardPolicy_TailDrop  DiscardPolicy = 0
-	DiscardPolicy_WTailDrop DiscardPolicy = 1
-	DiscardPolicy_Red       DiscardPolicy = 2
-	DiscardPolicy_WRed      DiscardPolicy = 3
-)
-
-var DiscardPolicy_name = map[int32]string{
-	0: "TailDrop",
-	1: "WTailDrop",
-	2: "Red",
-	3: "WRed",
-}
-
-var DiscardPolicy_value = map[string]int32{
-	"TailDrop":  0,
-	"WTailDrop": 1,
-	"Red":       2,
-	"WRed":      3,
-}
-
-func (x DiscardPolicy) String() string {
-	return proto.EnumName(DiscardPolicy_name, int32(x))
-}
-
-func (DiscardPolicy) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{3}
-}
-
-type InferredAdditionBWIndication int32
-
-const (
-	InferredAdditionBWIndication_InferredAdditionBWIndication_None       InferredAdditionBWIndication = 0
-	InferredAdditionBWIndication_InferredAdditionBWIndication_Assured    InferredAdditionBWIndication = 1
-	InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort InferredAdditionBWIndication = 2
-)
-
-var InferredAdditionBWIndication_name = map[int32]string{
-	0: "InferredAdditionBWIndication_None",
-	1: "InferredAdditionBWIndication_Assured",
-	2: "InferredAdditionBWIndication_BestEffort",
-}
-
-var InferredAdditionBWIndication_value = map[string]int32{
-	"InferredAdditionBWIndication_None":       0,
-	"InferredAdditionBWIndication_Assured":    1,
-	"InferredAdditionBWIndication_BestEffort": 2,
-}
-
-func (x InferredAdditionBWIndication) String() string {
-	return proto.EnumName(InferredAdditionBWIndication_name, int32(x))
-}
-
-func (InferredAdditionBWIndication) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{4}
-}
-
-type SchedulerConfig struct {
-	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
-	AdditionalBw         AdditionalBW     `protobuf:"varint,2,opt,name=additional_bw,json=additionalBw,proto3,enum=tech_profile.AdditionalBW" json:"additional_bw,omitempty"`
-	Priority             uint32           `protobuf:"fixed32,3,opt,name=priority,proto3" json:"priority,omitempty"`
-	Weight               uint32           `protobuf:"fixed32,4,opt,name=weight,proto3" json:"weight,omitempty"`
-	SchedPolicy          SchedulingPolicy `protobuf:"varint,5,opt,name=sched_policy,json=schedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"sched_policy,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *SchedulerConfig) Reset()         { *m = SchedulerConfig{} }
-func (m *SchedulerConfig) String() string { return proto.CompactTextString(m) }
-func (*SchedulerConfig) ProtoMessage()    {}
-func (*SchedulerConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{0}
-}
-
-func (m *SchedulerConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SchedulerConfig.Unmarshal(m, b)
-}
-func (m *SchedulerConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SchedulerConfig.Marshal(b, m, deterministic)
-}
-func (m *SchedulerConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SchedulerConfig.Merge(m, src)
-}
-func (m *SchedulerConfig) XXX_Size() int {
-	return xxx_messageInfo_SchedulerConfig.Size(m)
-}
-func (m *SchedulerConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_SchedulerConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SchedulerConfig proto.InternalMessageInfo
-
-func (m *SchedulerConfig) GetDirection() Direction {
-	if m != nil {
-		return m.Direction
-	}
-	return Direction_UPSTREAM
-}
-
-func (m *SchedulerConfig) GetAdditionalBw() AdditionalBW {
-	if m != nil {
-		return m.AdditionalBw
-	}
-	return AdditionalBW_AdditionalBW_None
-}
-
-func (m *SchedulerConfig) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *SchedulerConfig) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *SchedulerConfig) GetSchedPolicy() SchedulingPolicy {
-	if m != nil {
-		return m.SchedPolicy
-	}
-	return SchedulingPolicy_WRR
-}
-
-type TrafficShapingInfo struct {
-	Cir                  uint32                       `protobuf:"fixed32,1,opt,name=cir,proto3" json:"cir,omitempty"`
-	Cbs                  uint32                       `protobuf:"fixed32,2,opt,name=cbs,proto3" json:"cbs,omitempty"`
-	Pir                  uint32                       `protobuf:"fixed32,3,opt,name=pir,proto3" json:"pir,omitempty"`
-	Pbs                  uint32                       `protobuf:"fixed32,4,opt,name=pbs,proto3" json:"pbs,omitempty"`
-	Gir                  uint32                       `protobuf:"fixed32,5,opt,name=gir,proto3" json:"gir,omitempty"`
-	AddBwInd             InferredAdditionBWIndication `protobuf:"varint,6,opt,name=add_bw_ind,json=addBwInd,proto3,enum=tech_profile.InferredAdditionBWIndication" json:"add_bw_ind,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
-}
-
-func (m *TrafficShapingInfo) Reset()         { *m = TrafficShapingInfo{} }
-func (m *TrafficShapingInfo) String() string { return proto.CompactTextString(m) }
-func (*TrafficShapingInfo) ProtoMessage()    {}
-func (*TrafficShapingInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{1}
-}
-
-func (m *TrafficShapingInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TrafficShapingInfo.Unmarshal(m, b)
-}
-func (m *TrafficShapingInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TrafficShapingInfo.Marshal(b, m, deterministic)
-}
-func (m *TrafficShapingInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TrafficShapingInfo.Merge(m, src)
-}
-func (m *TrafficShapingInfo) XXX_Size() int {
-	return xxx_messageInfo_TrafficShapingInfo.Size(m)
-}
-func (m *TrafficShapingInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_TrafficShapingInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TrafficShapingInfo proto.InternalMessageInfo
-
-func (m *TrafficShapingInfo) GetCir() uint32 {
-	if m != nil {
-		return m.Cir
-	}
-	return 0
-}
-
-func (m *TrafficShapingInfo) GetCbs() uint32 {
-	if m != nil {
-		return m.Cbs
-	}
-	return 0
-}
-
-func (m *TrafficShapingInfo) GetPir() uint32 {
-	if m != nil {
-		return m.Pir
-	}
-	return 0
-}
-
-func (m *TrafficShapingInfo) GetPbs() uint32 {
-	if m != nil {
-		return m.Pbs
-	}
-	return 0
-}
-
-func (m *TrafficShapingInfo) GetGir() uint32 {
-	if m != nil {
-		return m.Gir
-	}
-	return 0
-}
-
-func (m *TrafficShapingInfo) GetAddBwInd() InferredAdditionBWIndication {
-	if m != nil {
-		return m.AddBwInd
-	}
-	return InferredAdditionBWIndication_InferredAdditionBWIndication_None
-}
-
-type TrafficScheduler struct {
-	Direction            Direction           `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
-	AllocId              uint32              `protobuf:"fixed32,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	Scheduler            *SchedulerConfig    `protobuf:"bytes,3,opt,name=scheduler,proto3" json:"scheduler,omitempty"`
-	TrafficShapingInfo   *TrafficShapingInfo `protobuf:"bytes,4,opt,name=traffic_shaping_info,json=trafficShapingInfo,proto3" json:"traffic_shaping_info,omitempty"`
-	TechProfileId        uint32              `protobuf:"fixed32,5,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *TrafficScheduler) Reset()         { *m = TrafficScheduler{} }
-func (m *TrafficScheduler) String() string { return proto.CompactTextString(m) }
-func (*TrafficScheduler) ProtoMessage()    {}
-func (*TrafficScheduler) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{2}
-}
-
-func (m *TrafficScheduler) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TrafficScheduler.Unmarshal(m, b)
-}
-func (m *TrafficScheduler) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TrafficScheduler.Marshal(b, m, deterministic)
-}
-func (m *TrafficScheduler) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TrafficScheduler.Merge(m, src)
-}
-func (m *TrafficScheduler) XXX_Size() int {
-	return xxx_messageInfo_TrafficScheduler.Size(m)
-}
-func (m *TrafficScheduler) XXX_DiscardUnknown() {
-	xxx_messageInfo_TrafficScheduler.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TrafficScheduler proto.InternalMessageInfo
-
-func (m *TrafficScheduler) GetDirection() Direction {
-	if m != nil {
-		return m.Direction
-	}
-	return Direction_UPSTREAM
-}
-
-func (m *TrafficScheduler) GetAllocId() uint32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-func (m *TrafficScheduler) GetScheduler() *SchedulerConfig {
-	if m != nil {
-		return m.Scheduler
-	}
-	return nil
-}
-
-func (m *TrafficScheduler) GetTrafficShapingInfo() *TrafficShapingInfo {
-	if m != nil {
-		return m.TrafficShapingInfo
-	}
-	return nil
-}
-
-func (m *TrafficScheduler) GetTechProfileId() uint32 {
-	if m != nil {
-		return m.TechProfileId
-	}
-	return 0
-}
-
-type TrafficSchedulers struct {
-	IntfId               uint32              `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32              `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	UniId                uint32              `protobuf:"fixed32,4,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	PortNo               uint32              `protobuf:"fixed32,5,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	TrafficScheds        []*TrafficScheduler `protobuf:"bytes,3,rep,name=traffic_scheds,json=trafficScheds,proto3" json:"traffic_scheds,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *TrafficSchedulers) Reset()         { *m = TrafficSchedulers{} }
-func (m *TrafficSchedulers) String() string { return proto.CompactTextString(m) }
-func (*TrafficSchedulers) ProtoMessage()    {}
-func (*TrafficSchedulers) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{3}
-}
-
-func (m *TrafficSchedulers) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TrafficSchedulers.Unmarshal(m, b)
-}
-func (m *TrafficSchedulers) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TrafficSchedulers.Marshal(b, m, deterministic)
-}
-func (m *TrafficSchedulers) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TrafficSchedulers.Merge(m, src)
-}
-func (m *TrafficSchedulers) XXX_Size() int {
-	return xxx_messageInfo_TrafficSchedulers.Size(m)
-}
-func (m *TrafficSchedulers) XXX_DiscardUnknown() {
-	xxx_messageInfo_TrafficSchedulers.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TrafficSchedulers proto.InternalMessageInfo
-
-func (m *TrafficSchedulers) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *TrafficSchedulers) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *TrafficSchedulers) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *TrafficSchedulers) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *TrafficSchedulers) GetTrafficScheds() []*TrafficScheduler {
-	if m != nil {
-		return m.TrafficScheds
-	}
-	return nil
-}
-
-type TailDropDiscardConfig struct {
-	QueueSize            uint32   `protobuf:"fixed32,1,opt,name=queue_size,json=queueSize,proto3" json:"queue_size,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *TailDropDiscardConfig) Reset()         { *m = TailDropDiscardConfig{} }
-func (m *TailDropDiscardConfig) String() string { return proto.CompactTextString(m) }
-func (*TailDropDiscardConfig) ProtoMessage()    {}
-func (*TailDropDiscardConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{4}
-}
-
-func (m *TailDropDiscardConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TailDropDiscardConfig.Unmarshal(m, b)
-}
-func (m *TailDropDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TailDropDiscardConfig.Marshal(b, m, deterministic)
-}
-func (m *TailDropDiscardConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TailDropDiscardConfig.Merge(m, src)
-}
-func (m *TailDropDiscardConfig) XXX_Size() int {
-	return xxx_messageInfo_TailDropDiscardConfig.Size(m)
-}
-func (m *TailDropDiscardConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_TailDropDiscardConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TailDropDiscardConfig proto.InternalMessageInfo
-
-func (m *TailDropDiscardConfig) GetQueueSize() uint32 {
-	if m != nil {
-		return m.QueueSize
-	}
-	return 0
-}
-
-type RedDiscardConfig struct {
-	MinThreshold         uint32   `protobuf:"fixed32,1,opt,name=min_threshold,json=minThreshold,proto3" json:"min_threshold,omitempty"`
-	MaxThreshold         uint32   `protobuf:"fixed32,2,opt,name=max_threshold,json=maxThreshold,proto3" json:"max_threshold,omitempty"`
-	MaxProbability       uint32   `protobuf:"fixed32,3,opt,name=max_probability,json=maxProbability,proto3" json:"max_probability,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *RedDiscardConfig) Reset()         { *m = RedDiscardConfig{} }
-func (m *RedDiscardConfig) String() string { return proto.CompactTextString(m) }
-func (*RedDiscardConfig) ProtoMessage()    {}
-func (*RedDiscardConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{5}
-}
-
-func (m *RedDiscardConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RedDiscardConfig.Unmarshal(m, b)
-}
-func (m *RedDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RedDiscardConfig.Marshal(b, m, deterministic)
-}
-func (m *RedDiscardConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RedDiscardConfig.Merge(m, src)
-}
-func (m *RedDiscardConfig) XXX_Size() int {
-	return xxx_messageInfo_RedDiscardConfig.Size(m)
-}
-func (m *RedDiscardConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_RedDiscardConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RedDiscardConfig proto.InternalMessageInfo
-
-func (m *RedDiscardConfig) GetMinThreshold() uint32 {
-	if m != nil {
-		return m.MinThreshold
-	}
-	return 0
-}
-
-func (m *RedDiscardConfig) GetMaxThreshold() uint32 {
-	if m != nil {
-		return m.MaxThreshold
-	}
-	return 0
-}
-
-func (m *RedDiscardConfig) GetMaxProbability() uint32 {
-	if m != nil {
-		return m.MaxProbability
-	}
-	return 0
-}
-
-type WRedDiscardConfig struct {
-	Green                *RedDiscardConfig `protobuf:"bytes,1,opt,name=green,proto3" json:"green,omitempty"`
-	Yellow               *RedDiscardConfig `protobuf:"bytes,2,opt,name=yellow,proto3" json:"yellow,omitempty"`
-	Red                  *RedDiscardConfig `protobuf:"bytes,3,opt,name=red,proto3" json:"red,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *WRedDiscardConfig) Reset()         { *m = WRedDiscardConfig{} }
-func (m *WRedDiscardConfig) String() string { return proto.CompactTextString(m) }
-func (*WRedDiscardConfig) ProtoMessage()    {}
-func (*WRedDiscardConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{6}
-}
-
-func (m *WRedDiscardConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WRedDiscardConfig.Unmarshal(m, b)
-}
-func (m *WRedDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WRedDiscardConfig.Marshal(b, m, deterministic)
-}
-func (m *WRedDiscardConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WRedDiscardConfig.Merge(m, src)
-}
-func (m *WRedDiscardConfig) XXX_Size() int {
-	return xxx_messageInfo_WRedDiscardConfig.Size(m)
-}
-func (m *WRedDiscardConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_WRedDiscardConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WRedDiscardConfig proto.InternalMessageInfo
-
-func (m *WRedDiscardConfig) GetGreen() *RedDiscardConfig {
-	if m != nil {
-		return m.Green
-	}
-	return nil
-}
-
-func (m *WRedDiscardConfig) GetYellow() *RedDiscardConfig {
-	if m != nil {
-		return m.Yellow
-	}
-	return nil
-}
-
-func (m *WRedDiscardConfig) GetRed() *RedDiscardConfig {
-	if m != nil {
-		return m.Red
-	}
-	return nil
-}
-
-type DiscardConfig struct {
-	DiscardPolicy DiscardPolicy `protobuf:"varint,1,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
-	// Types that are valid to be assigned to DiscardConfig:
-	//	*DiscardConfig_TailDropDiscardConfig
-	//	*DiscardConfig_RedDiscardConfig
-	//	*DiscardConfig_WredDiscardConfig
-	DiscardConfig        isDiscardConfig_DiscardConfig `protobuf_oneof:"discard_config"`
-	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
-	XXX_unrecognized     []byte                        `json:"-"`
-	XXX_sizecache        int32                         `json:"-"`
-}
-
-func (m *DiscardConfig) Reset()         { *m = DiscardConfig{} }
-func (m *DiscardConfig) String() string { return proto.CompactTextString(m) }
-func (*DiscardConfig) ProtoMessage()    {}
-func (*DiscardConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{7}
-}
-
-func (m *DiscardConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DiscardConfig.Unmarshal(m, b)
-}
-func (m *DiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DiscardConfig.Marshal(b, m, deterministic)
-}
-func (m *DiscardConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DiscardConfig.Merge(m, src)
-}
-func (m *DiscardConfig) XXX_Size() int {
-	return xxx_messageInfo_DiscardConfig.Size(m)
-}
-func (m *DiscardConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_DiscardConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DiscardConfig proto.InternalMessageInfo
-
-func (m *DiscardConfig) GetDiscardPolicy() DiscardPolicy {
-	if m != nil {
-		return m.DiscardPolicy
-	}
-	return DiscardPolicy_TailDrop
-}
-
-type isDiscardConfig_DiscardConfig interface {
-	isDiscardConfig_DiscardConfig()
-}
-
-type DiscardConfig_TailDropDiscardConfig struct {
-	TailDropDiscardConfig *TailDropDiscardConfig `protobuf:"bytes,2,opt,name=tail_drop_discard_config,json=tailDropDiscardConfig,proto3,oneof"`
-}
-
-type DiscardConfig_RedDiscardConfig struct {
-	RedDiscardConfig *RedDiscardConfig `protobuf:"bytes,3,opt,name=red_discard_config,json=redDiscardConfig,proto3,oneof"`
-}
-
-type DiscardConfig_WredDiscardConfig struct {
-	WredDiscardConfig *WRedDiscardConfig `protobuf:"bytes,4,opt,name=wred_discard_config,json=wredDiscardConfig,proto3,oneof"`
-}
-
-func (*DiscardConfig_TailDropDiscardConfig) isDiscardConfig_DiscardConfig() {}
-
-func (*DiscardConfig_RedDiscardConfig) isDiscardConfig_DiscardConfig() {}
-
-func (*DiscardConfig_WredDiscardConfig) isDiscardConfig_DiscardConfig() {}
-
-func (m *DiscardConfig) GetDiscardConfig() isDiscardConfig_DiscardConfig {
-	if m != nil {
-		return m.DiscardConfig
-	}
-	return nil
-}
-
-func (m *DiscardConfig) GetTailDropDiscardConfig() *TailDropDiscardConfig {
-	if x, ok := m.GetDiscardConfig().(*DiscardConfig_TailDropDiscardConfig); ok {
-		return x.TailDropDiscardConfig
-	}
-	return nil
-}
-
-func (m *DiscardConfig) GetRedDiscardConfig() *RedDiscardConfig {
-	if x, ok := m.GetDiscardConfig().(*DiscardConfig_RedDiscardConfig); ok {
-		return x.RedDiscardConfig
-	}
-	return nil
-}
-
-func (m *DiscardConfig) GetWredDiscardConfig() *WRedDiscardConfig {
-	if x, ok := m.GetDiscardConfig().(*DiscardConfig_WredDiscardConfig); ok {
-		return x.WredDiscardConfig
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*DiscardConfig) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*DiscardConfig_TailDropDiscardConfig)(nil),
-		(*DiscardConfig_RedDiscardConfig)(nil),
-		(*DiscardConfig_WredDiscardConfig)(nil),
-	}
-}
-
-type TrafficQueue struct {
-	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
-	GemportId            uint32           `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	PbitMap              string           `protobuf:"bytes,3,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
-	AesEncryption        bool             `protobuf:"varint,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
-	SchedPolicy          SchedulingPolicy `protobuf:"varint,5,opt,name=sched_policy,json=schedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"sched_policy,omitempty"`
-	Priority             uint32           `protobuf:"fixed32,6,opt,name=priority,proto3" json:"priority,omitempty"`
-	Weight               uint32           `protobuf:"fixed32,7,opt,name=weight,proto3" json:"weight,omitempty"`
-	DiscardPolicy        DiscardPolicy    `protobuf:"varint,8,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
-	DiscardConfig        *DiscardConfig   `protobuf:"bytes,9,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *TrafficQueue) Reset()         { *m = TrafficQueue{} }
-func (m *TrafficQueue) String() string { return proto.CompactTextString(m) }
-func (*TrafficQueue) ProtoMessage()    {}
-func (*TrafficQueue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{8}
-}
-
-func (m *TrafficQueue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TrafficQueue.Unmarshal(m, b)
-}
-func (m *TrafficQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TrafficQueue.Marshal(b, m, deterministic)
-}
-func (m *TrafficQueue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TrafficQueue.Merge(m, src)
-}
-func (m *TrafficQueue) XXX_Size() int {
-	return xxx_messageInfo_TrafficQueue.Size(m)
-}
-func (m *TrafficQueue) XXX_DiscardUnknown() {
-	xxx_messageInfo_TrafficQueue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TrafficQueue proto.InternalMessageInfo
-
-func (m *TrafficQueue) GetDirection() Direction {
-	if m != nil {
-		return m.Direction
-	}
-	return Direction_UPSTREAM
-}
-
-func (m *TrafficQueue) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *TrafficQueue) GetPbitMap() string {
-	if m != nil {
-		return m.PbitMap
-	}
-	return ""
-}
-
-func (m *TrafficQueue) GetAesEncryption() bool {
-	if m != nil {
-		return m.AesEncryption
-	}
-	return false
-}
-
-func (m *TrafficQueue) GetSchedPolicy() SchedulingPolicy {
-	if m != nil {
-		return m.SchedPolicy
-	}
-	return SchedulingPolicy_WRR
-}
-
-func (m *TrafficQueue) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *TrafficQueue) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *TrafficQueue) GetDiscardPolicy() DiscardPolicy {
-	if m != nil {
-		return m.DiscardPolicy
-	}
-	return DiscardPolicy_TailDrop
-}
-
-func (m *TrafficQueue) GetDiscardConfig() *DiscardConfig {
-	if m != nil {
-		return m.DiscardConfig
-	}
-	return nil
-}
-
-type TrafficQueues struct {
-	IntfId               uint32          `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	OnuId                uint32          `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	UniId                uint32          `protobuf:"fixed32,4,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
-	PortNo               uint32          `protobuf:"fixed32,5,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	TrafficQueues        []*TrafficQueue `protobuf:"bytes,6,rep,name=traffic_queues,json=trafficQueues,proto3" json:"traffic_queues,omitempty"`
-	TechProfileId        uint32          `protobuf:"fixed32,7,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *TrafficQueues) Reset()         { *m = TrafficQueues{} }
-func (m *TrafficQueues) String() string { return proto.CompactTextString(m) }
-func (*TrafficQueues) ProtoMessage()    {}
-func (*TrafficQueues) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{9}
-}
-
-func (m *TrafficQueues) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TrafficQueues.Unmarshal(m, b)
-}
-func (m *TrafficQueues) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TrafficQueues.Marshal(b, m, deterministic)
-}
-func (m *TrafficQueues) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TrafficQueues.Merge(m, src)
-}
-func (m *TrafficQueues) XXX_Size() int {
-	return xxx_messageInfo_TrafficQueues.Size(m)
-}
-func (m *TrafficQueues) XXX_DiscardUnknown() {
-	xxx_messageInfo_TrafficQueues.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TrafficQueues proto.InternalMessageInfo
-
-func (m *TrafficQueues) GetIntfId() uint32 {
-	if m != nil {
-		return m.IntfId
-	}
-	return 0
-}
-
-func (m *TrafficQueues) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *TrafficQueues) GetUniId() uint32 {
-	if m != nil {
-		return m.UniId
-	}
-	return 0
-}
-
-func (m *TrafficQueues) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *TrafficQueues) GetTrafficQueues() []*TrafficQueue {
-	if m != nil {
-		return m.TrafficQueues
-	}
-	return nil
-}
-
-func (m *TrafficQueues) GetTechProfileId() uint32 {
-	if m != nil {
-		return m.TechProfileId
-	}
-	return 0
-}
-
-type InstanceControl struct {
-	Onu                  string   `protobuf:"bytes,1,opt,name=onu,proto3" json:"onu,omitempty"`
-	Uni                  string   `protobuf:"bytes,2,opt,name=uni,proto3" json:"uni,omitempty"`
-	MaxGemPayloadSize    string   `protobuf:"bytes,3,opt,name=max_gem_payload_size,json=maxGemPayloadSize,proto3" json:"max_gem_payload_size,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *InstanceControl) Reset()         { *m = InstanceControl{} }
-func (m *InstanceControl) String() string { return proto.CompactTextString(m) }
-func (*InstanceControl) ProtoMessage()    {}
-func (*InstanceControl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{10}
-}
-
-func (m *InstanceControl) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InstanceControl.Unmarshal(m, b)
-}
-func (m *InstanceControl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InstanceControl.Marshal(b, m, deterministic)
-}
-func (m *InstanceControl) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InstanceControl.Merge(m, src)
-}
-func (m *InstanceControl) XXX_Size() int {
-	return xxx_messageInfo_InstanceControl.Size(m)
-}
-func (m *InstanceControl) XXX_DiscardUnknown() {
-	xxx_messageInfo_InstanceControl.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InstanceControl proto.InternalMessageInfo
-
-func (m *InstanceControl) GetOnu() string {
-	if m != nil {
-		return m.Onu
-	}
-	return ""
-}
-
-func (m *InstanceControl) GetUni() string {
-	if m != nil {
-		return m.Uni
-	}
-	return ""
-}
-
-func (m *InstanceControl) GetMaxGemPayloadSize() string {
-	if m != nil {
-		return m.MaxGemPayloadSize
-	}
-	return ""
-}
-
-type QThresholds struct {
-	QThreshold1          uint32   `protobuf:"varint,1,opt,name=q_threshold1,json=qThreshold1,proto3" json:"q_threshold1,omitempty"`
-	QThreshold2          uint32   `protobuf:"varint,2,opt,name=q_threshold2,json=qThreshold2,proto3" json:"q_threshold2,omitempty"`
-	QThreshold3          uint32   `protobuf:"varint,3,opt,name=q_threshold3,json=qThreshold3,proto3" json:"q_threshold3,omitempty"`
-	QThreshold4          uint32   `protobuf:"varint,4,opt,name=q_threshold4,json=qThreshold4,proto3" json:"q_threshold4,omitempty"`
-	QThreshold5          uint32   `protobuf:"varint,5,opt,name=q_threshold5,json=qThreshold5,proto3" json:"q_threshold5,omitempty"`
-	QThreshold6          uint32   `protobuf:"varint,6,opt,name=q_threshold6,json=qThreshold6,proto3" json:"q_threshold6,omitempty"`
-	QThreshold7          uint32   `protobuf:"varint,7,opt,name=q_threshold7,json=qThreshold7,proto3" json:"q_threshold7,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *QThresholds) Reset()         { *m = QThresholds{} }
-func (m *QThresholds) String() string { return proto.CompactTextString(m) }
-func (*QThresholds) ProtoMessage()    {}
-func (*QThresholds) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{11}
-}
-
-func (m *QThresholds) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QThresholds.Unmarshal(m, b)
-}
-func (m *QThresholds) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QThresholds.Marshal(b, m, deterministic)
-}
-func (m *QThresholds) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QThresholds.Merge(m, src)
-}
-func (m *QThresholds) XXX_Size() int {
-	return xxx_messageInfo_QThresholds.Size(m)
-}
-func (m *QThresholds) XXX_DiscardUnknown() {
-	xxx_messageInfo_QThresholds.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QThresholds proto.InternalMessageInfo
-
-func (m *QThresholds) GetQThreshold1() uint32 {
-	if m != nil {
-		return m.QThreshold1
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold2() uint32 {
-	if m != nil {
-		return m.QThreshold2
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold3() uint32 {
-	if m != nil {
-		return m.QThreshold3
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold4() uint32 {
-	if m != nil {
-		return m.QThreshold4
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold5() uint32 {
-	if m != nil {
-		return m.QThreshold5
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold6() uint32 {
-	if m != nil {
-		return m.QThreshold6
-	}
-	return 0
-}
-
-func (m *QThresholds) GetQThreshold7() uint32 {
-	if m != nil {
-		return m.QThreshold7
-	}
-	return 0
-}
-
-type GemPortAttributes struct {
-	GemportId                uint32            `protobuf:"fixed32,1,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	MaxQSize                 string            `protobuf:"bytes,2,opt,name=max_q_size,json=maxQSize,proto3" json:"max_q_size,omitempty"`
-	PbitMap                  string            `protobuf:"bytes,3,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
-	AesEncryption            string            `protobuf:"bytes,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
-	SchedulingPolicy         SchedulingPolicy  `protobuf:"varint,5,opt,name=scheduling_policy,json=schedulingPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"scheduling_policy,omitempty"`
-	PriorityQ                uint32            `protobuf:"fixed32,6,opt,name=priority_q,json=priorityQ,proto3" json:"priority_q,omitempty"`
-	Weight                   uint32            `protobuf:"fixed32,7,opt,name=weight,proto3" json:"weight,omitempty"`
-	DiscardPolicy            DiscardPolicy     `protobuf:"varint,8,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
-	DiscardConfig            *RedDiscardConfig `protobuf:"bytes,9,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
-	DiscardConfigV2          *DiscardConfig    `protobuf:"bytes,14,opt,name=discard_config_v2,json=discardConfigV2,proto3" json:"discard_config_v2,omitempty"`
-	IsMulticast              string            `protobuf:"bytes,10,opt,name=is_multicast,json=isMulticast,proto3" json:"is_multicast,omitempty"`
-	MulticastGemId           uint32            `protobuf:"fixed32,11,opt,name=multicast_gem_id,json=multicastGemId,proto3" json:"multicast_gem_id,omitempty"`
-	DynamicAccessControlList string            `protobuf:"bytes,12,opt,name=dynamic_access_control_list,json=dynamicAccessControlList,proto3" json:"dynamic_access_control_list,omitempty"`
-	StaticAccessControlList  string            `protobuf:"bytes,13,opt,name=static_access_control_list,json=staticAccessControlList,proto3" json:"static_access_control_list,omitempty"`
-	XXX_NoUnkeyedLiteral     struct{}          `json:"-"`
-	XXX_unrecognized         []byte            `json:"-"`
-	XXX_sizecache            int32             `json:"-"`
-}
-
-func (m *GemPortAttributes) Reset()         { *m = GemPortAttributes{} }
-func (m *GemPortAttributes) String() string { return proto.CompactTextString(m) }
-func (*GemPortAttributes) ProtoMessage()    {}
-func (*GemPortAttributes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{12}
-}
-
-func (m *GemPortAttributes) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GemPortAttributes.Unmarshal(m, b)
-}
-func (m *GemPortAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GemPortAttributes.Marshal(b, m, deterministic)
-}
-func (m *GemPortAttributes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GemPortAttributes.Merge(m, src)
-}
-func (m *GemPortAttributes) XXX_Size() int {
-	return xxx_messageInfo_GemPortAttributes.Size(m)
-}
-func (m *GemPortAttributes) XXX_DiscardUnknown() {
-	xxx_messageInfo_GemPortAttributes.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GemPortAttributes proto.InternalMessageInfo
-
-func (m *GemPortAttributes) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *GemPortAttributes) GetMaxQSize() string {
-	if m != nil {
-		return m.MaxQSize
-	}
-	return ""
-}
-
-func (m *GemPortAttributes) GetPbitMap() string {
-	if m != nil {
-		return m.PbitMap
-	}
-	return ""
-}
-
-func (m *GemPortAttributes) GetAesEncryption() string {
-	if m != nil {
-		return m.AesEncryption
-	}
-	return ""
-}
-
-func (m *GemPortAttributes) GetSchedulingPolicy() SchedulingPolicy {
-	if m != nil {
-		return m.SchedulingPolicy
-	}
-	return SchedulingPolicy_WRR
-}
-
-func (m *GemPortAttributes) GetPriorityQ() uint32 {
-	if m != nil {
-		return m.PriorityQ
-	}
-	return 0
-}
-
-func (m *GemPortAttributes) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *GemPortAttributes) GetDiscardPolicy() DiscardPolicy {
-	if m != nil {
-		return m.DiscardPolicy
-	}
-	return DiscardPolicy_TailDrop
-}
-
-func (m *GemPortAttributes) GetDiscardConfig() *RedDiscardConfig {
-	if m != nil {
-		return m.DiscardConfig
-	}
-	return nil
-}
-
-func (m *GemPortAttributes) GetDiscardConfigV2() *DiscardConfig {
-	if m != nil {
-		return m.DiscardConfigV2
-	}
-	return nil
-}
-
-func (m *GemPortAttributes) GetIsMulticast() string {
-	if m != nil {
-		return m.IsMulticast
-	}
-	return ""
-}
-
-func (m *GemPortAttributes) GetMulticastGemId() uint32 {
-	if m != nil {
-		return m.MulticastGemId
-	}
-	return 0
-}
-
-func (m *GemPortAttributes) GetDynamicAccessControlList() string {
-	if m != nil {
-		return m.DynamicAccessControlList
-	}
-	return ""
-}
-
-func (m *GemPortAttributes) GetStaticAccessControlList() string {
-	if m != nil {
-		return m.StaticAccessControlList
-	}
-	return ""
-}
-
-type SchedulerAttributes struct {
-	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
-	AllocId              uint32           `protobuf:"varint,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	AdditionalBw         AdditionalBW     `protobuf:"varint,3,opt,name=additional_bw,json=additionalBw,proto3,enum=tech_profile.AdditionalBW" json:"additional_bw,omitempty"`
-	Priority             uint32           `protobuf:"fixed32,4,opt,name=priority,proto3" json:"priority,omitempty"`
-	Weight               uint32           `protobuf:"fixed32,5,opt,name=weight,proto3" json:"weight,omitempty"`
-	QSchedPolicy         SchedulingPolicy `protobuf:"varint,6,opt,name=q_sched_policy,json=qSchedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"q_sched_policy,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *SchedulerAttributes) Reset()         { *m = SchedulerAttributes{} }
-func (m *SchedulerAttributes) String() string { return proto.CompactTextString(m) }
-func (*SchedulerAttributes) ProtoMessage()    {}
-func (*SchedulerAttributes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{13}
-}
-
-func (m *SchedulerAttributes) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SchedulerAttributes.Unmarshal(m, b)
-}
-func (m *SchedulerAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SchedulerAttributes.Marshal(b, m, deterministic)
-}
-func (m *SchedulerAttributes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SchedulerAttributes.Merge(m, src)
-}
-func (m *SchedulerAttributes) XXX_Size() int {
-	return xxx_messageInfo_SchedulerAttributes.Size(m)
-}
-func (m *SchedulerAttributes) XXX_DiscardUnknown() {
-	xxx_messageInfo_SchedulerAttributes.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SchedulerAttributes proto.InternalMessageInfo
-
-func (m *SchedulerAttributes) GetDirection() Direction {
-	if m != nil {
-		return m.Direction
-	}
-	return Direction_UPSTREAM
-}
-
-func (m *SchedulerAttributes) GetAllocId() uint32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-func (m *SchedulerAttributes) GetAdditionalBw() AdditionalBW {
-	if m != nil {
-		return m.AdditionalBw
-	}
-	return AdditionalBW_AdditionalBW_None
-}
-
-func (m *SchedulerAttributes) GetPriority() uint32 {
-	if m != nil {
-		return m.Priority
-	}
-	return 0
-}
-
-func (m *SchedulerAttributes) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *SchedulerAttributes) GetQSchedPolicy() SchedulingPolicy {
-	if m != nil {
-		return m.QSchedPolicy
-	}
-	return SchedulingPolicy_WRR
-}
-
-type EPONQueueAttributes struct {
-	MaxQSize                  string            `protobuf:"bytes,1,opt,name=max_q_size,json=maxQSize,proto3" json:"max_q_size,omitempty"`
-	PbitMap                   string            `protobuf:"bytes,2,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
-	GemportId                 uint32            `protobuf:"varint,3,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
-	AesEncryption             string            `protobuf:"bytes,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
-	TrafficType               string            `protobuf:"bytes,5,opt,name=traffic_type,json=trafficType,proto3" json:"traffic_type,omitempty"`
-	UnsolicitedGrantSize      uint32            `protobuf:"varint,6,opt,name=unsolicited_grant_size,json=unsolicitedGrantSize,proto3" json:"unsolicited_grant_size,omitempty"`
-	NominalInterval           uint32            `protobuf:"varint,7,opt,name=nominal_interval,json=nominalInterval,proto3" json:"nominal_interval,omitempty"`
-	ToleratedPollJitter       uint32            `protobuf:"varint,8,opt,name=tolerated_poll_jitter,json=toleratedPollJitter,proto3" json:"tolerated_poll_jitter,omitempty"`
-	RequestTransmissionPolicy uint32            `protobuf:"varint,9,opt,name=request_transmission_policy,json=requestTransmissionPolicy,proto3" json:"request_transmission_policy,omitempty"`
-	NumQSets                  uint32            `protobuf:"varint,10,opt,name=num_q_sets,json=numQSets,proto3" json:"num_q_sets,omitempty"`
-	QThresholds               *QThresholds      `protobuf:"bytes,11,opt,name=q_thresholds,json=qThresholds,proto3" json:"q_thresholds,omitempty"`
-	SchedulingPolicy          SchedulingPolicy  `protobuf:"varint,12,opt,name=scheduling_policy,json=schedulingPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"scheduling_policy,omitempty"`
-	PriorityQ                 uint32            `protobuf:"varint,13,opt,name=priority_q,json=priorityQ,proto3" json:"priority_q,omitempty"`
-	Weight                    uint32            `protobuf:"varint,14,opt,name=weight,proto3" json:"weight,omitempty"`
-	DiscardPolicy             DiscardPolicy     `protobuf:"varint,15,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
-	DiscardConfig             *RedDiscardConfig `protobuf:"bytes,16,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
-	DiscardConfigV2           *DiscardConfig    `protobuf:"bytes,17,opt,name=discard_config_v2,json=discardConfigV2,proto3" json:"discard_config_v2,omitempty"`
-	XXX_NoUnkeyedLiteral      struct{}          `json:"-"`
-	XXX_unrecognized          []byte            `json:"-"`
-	XXX_sizecache             int32             `json:"-"`
-}
-
-func (m *EPONQueueAttributes) Reset()         { *m = EPONQueueAttributes{} }
-func (m *EPONQueueAttributes) String() string { return proto.CompactTextString(m) }
-func (*EPONQueueAttributes) ProtoMessage()    {}
-func (*EPONQueueAttributes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{14}
-}
-
-func (m *EPONQueueAttributes) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EPONQueueAttributes.Unmarshal(m, b)
-}
-func (m *EPONQueueAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EPONQueueAttributes.Marshal(b, m, deterministic)
-}
-func (m *EPONQueueAttributes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EPONQueueAttributes.Merge(m, src)
-}
-func (m *EPONQueueAttributes) XXX_Size() int {
-	return xxx_messageInfo_EPONQueueAttributes.Size(m)
-}
-func (m *EPONQueueAttributes) XXX_DiscardUnknown() {
-	xxx_messageInfo_EPONQueueAttributes.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EPONQueueAttributes proto.InternalMessageInfo
-
-func (m *EPONQueueAttributes) GetMaxQSize() string {
-	if m != nil {
-		return m.MaxQSize
-	}
-	return ""
-}
-
-func (m *EPONQueueAttributes) GetPbitMap() string {
-	if m != nil {
-		return m.PbitMap
-	}
-	return ""
-}
-
-func (m *EPONQueueAttributes) GetGemportId() uint32 {
-	if m != nil {
-		return m.GemportId
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetAesEncryption() string {
-	if m != nil {
-		return m.AesEncryption
-	}
-	return ""
-}
-
-func (m *EPONQueueAttributes) GetTrafficType() string {
-	if m != nil {
-		return m.TrafficType
-	}
-	return ""
-}
-
-func (m *EPONQueueAttributes) GetUnsolicitedGrantSize() uint32 {
-	if m != nil {
-		return m.UnsolicitedGrantSize
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetNominalInterval() uint32 {
-	if m != nil {
-		return m.NominalInterval
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetToleratedPollJitter() uint32 {
-	if m != nil {
-		return m.ToleratedPollJitter
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetRequestTransmissionPolicy() uint32 {
-	if m != nil {
-		return m.RequestTransmissionPolicy
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetNumQSets() uint32 {
-	if m != nil {
-		return m.NumQSets
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetQThresholds() *QThresholds {
-	if m != nil {
-		return m.QThresholds
-	}
-	return nil
-}
-
-func (m *EPONQueueAttributes) GetSchedulingPolicy() SchedulingPolicy {
-	if m != nil {
-		return m.SchedulingPolicy
-	}
-	return SchedulingPolicy_WRR
-}
-
-func (m *EPONQueueAttributes) GetPriorityQ() uint32 {
-	if m != nil {
-		return m.PriorityQ
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetWeight() uint32 {
-	if m != nil {
-		return m.Weight
-	}
-	return 0
-}
-
-func (m *EPONQueueAttributes) GetDiscardPolicy() DiscardPolicy {
-	if m != nil {
-		return m.DiscardPolicy
-	}
-	return DiscardPolicy_TailDrop
-}
-
-func (m *EPONQueueAttributes) GetDiscardConfig() *RedDiscardConfig {
-	if m != nil {
-		return m.DiscardConfig
-	}
-	return nil
-}
-
-func (m *EPONQueueAttributes) GetDiscardConfigV2() *DiscardConfig {
-	if m != nil {
-		return m.DiscardConfigV2
-	}
-	return nil
-}
-
-// TechProfile definition (relevant for GPON, XGPON and XGS-PON technologies)
-type TechProfile struct {
-	Name                           string               `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Version                        uint32               `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
-	ProfileType                    string               `protobuf:"bytes,3,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
-	NumGemPorts                    uint32               `protobuf:"varint,4,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
-	InstanceControl                *InstanceControl     `protobuf:"bytes,5,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
-	UsScheduler                    *SchedulerAttributes `protobuf:"bytes,6,opt,name=us_scheduler,json=usScheduler,proto3" json:"us_scheduler,omitempty"`
-	DsScheduler                    *SchedulerAttributes `protobuf:"bytes,7,opt,name=ds_scheduler,json=dsScheduler,proto3" json:"ds_scheduler,omitempty"`
-	UpstreamGemPortAttributeList   []*GemPortAttributes `protobuf:"bytes,8,rep,name=upstream_gem_port_attribute_list,json=upstreamGemPortAttributeList,proto3" json:"upstream_gem_port_attribute_list,omitempty"`
-	DownstreamGemPortAttributeList []*GemPortAttributes `protobuf:"bytes,9,rep,name=downstream_gem_port_attribute_list,json=downstreamGemPortAttributeList,proto3" json:"downstream_gem_port_attribute_list,omitempty"`
-	XXX_NoUnkeyedLiteral           struct{}             `json:"-"`
-	XXX_unrecognized               []byte               `json:"-"`
-	XXX_sizecache                  int32                `json:"-"`
-}
-
-func (m *TechProfile) Reset()         { *m = TechProfile{} }
-func (m *TechProfile) String() string { return proto.CompactTextString(m) }
-func (*TechProfile) ProtoMessage()    {}
-func (*TechProfile) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{15}
-}
-
-func (m *TechProfile) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TechProfile.Unmarshal(m, b)
-}
-func (m *TechProfile) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TechProfile.Marshal(b, m, deterministic)
-}
-func (m *TechProfile) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TechProfile.Merge(m, src)
-}
-func (m *TechProfile) XXX_Size() int {
-	return xxx_messageInfo_TechProfile.Size(m)
-}
-func (m *TechProfile) XXX_DiscardUnknown() {
-	xxx_messageInfo_TechProfile.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TechProfile proto.InternalMessageInfo
-
-func (m *TechProfile) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *TechProfile) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *TechProfile) GetProfileType() string {
-	if m != nil {
-		return m.ProfileType
-	}
-	return ""
-}
-
-func (m *TechProfile) GetNumGemPorts() uint32 {
-	if m != nil {
-		return m.NumGemPorts
-	}
-	return 0
-}
-
-func (m *TechProfile) GetInstanceControl() *InstanceControl {
-	if m != nil {
-		return m.InstanceControl
-	}
-	return nil
-}
-
-func (m *TechProfile) GetUsScheduler() *SchedulerAttributes {
-	if m != nil {
-		return m.UsScheduler
-	}
-	return nil
-}
-
-func (m *TechProfile) GetDsScheduler() *SchedulerAttributes {
-	if m != nil {
-		return m.DsScheduler
-	}
-	return nil
-}
-
-func (m *TechProfile) GetUpstreamGemPortAttributeList() []*GemPortAttributes {
-	if m != nil {
-		return m.UpstreamGemPortAttributeList
-	}
-	return nil
-}
-
-func (m *TechProfile) GetDownstreamGemPortAttributeList() []*GemPortAttributes {
-	if m != nil {
-		return m.DownstreamGemPortAttributeList
-	}
-	return nil
-}
-
-// EPON TechProfile definition
-type EponTechProfile struct {
-	Name                         string                 `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Version                      uint32                 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
-	ProfileType                  string                 `protobuf:"bytes,3,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
-	NumGemPorts                  uint32                 `protobuf:"varint,4,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
-	InstanceControl              *InstanceControl       `protobuf:"bytes,5,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
-	PackageType                  string                 `protobuf:"bytes,6,opt,name=package_type,json=packageType,proto3" json:"package_type,omitempty"`
-	UpstreamQueueAttributeList   []*EPONQueueAttributes `protobuf:"bytes,7,rep,name=upstream_queue_attribute_list,json=upstreamQueueAttributeList,proto3" json:"upstream_queue_attribute_list,omitempty"`
-	DownstreamQueueAttributeList []*EPONQueueAttributes `protobuf:"bytes,8,rep,name=downstream_queue_attribute_list,json=downstreamQueueAttributeList,proto3" json:"downstream_queue_attribute_list,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	XXX_unrecognized             []byte                 `json:"-"`
-	XXX_sizecache                int32                  `json:"-"`
-}
-
-func (m *EponTechProfile) Reset()         { *m = EponTechProfile{} }
-func (m *EponTechProfile) String() string { return proto.CompactTextString(m) }
-func (*EponTechProfile) ProtoMessage()    {}
-func (*EponTechProfile) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{16}
-}
-
-func (m *EponTechProfile) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EponTechProfile.Unmarshal(m, b)
-}
-func (m *EponTechProfile) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EponTechProfile.Marshal(b, m, deterministic)
-}
-func (m *EponTechProfile) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EponTechProfile.Merge(m, src)
-}
-func (m *EponTechProfile) XXX_Size() int {
-	return xxx_messageInfo_EponTechProfile.Size(m)
-}
-func (m *EponTechProfile) XXX_DiscardUnknown() {
-	xxx_messageInfo_EponTechProfile.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EponTechProfile proto.InternalMessageInfo
-
-func (m *EponTechProfile) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *EponTechProfile) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *EponTechProfile) GetProfileType() string {
-	if m != nil {
-		return m.ProfileType
-	}
-	return ""
-}
-
-func (m *EponTechProfile) GetNumGemPorts() uint32 {
-	if m != nil {
-		return m.NumGemPorts
-	}
-	return 0
-}
-
-func (m *EponTechProfile) GetInstanceControl() *InstanceControl {
-	if m != nil {
-		return m.InstanceControl
-	}
-	return nil
-}
-
-func (m *EponTechProfile) GetPackageType() string {
-	if m != nil {
-		return m.PackageType
-	}
-	return ""
-}
-
-func (m *EponTechProfile) GetUpstreamQueueAttributeList() []*EPONQueueAttributes {
-	if m != nil {
-		return m.UpstreamQueueAttributeList
-	}
-	return nil
-}
-
-func (m *EponTechProfile) GetDownstreamQueueAttributeList() []*EPONQueueAttributes {
-	if m != nil {
-		return m.DownstreamQueueAttributeList
-	}
-	return nil
-}
-
-// TechProfile Instance definition (relevant for GPON, XGPON and XGS-PON technologies)
-type TechProfileInstance struct {
-	Name                           string               `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Version                        uint32               `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
-	SubscriberIdentifier           string               `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
-	ProfileType                    string               `protobuf:"bytes,4,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
-	NumGemPorts                    uint32               `protobuf:"varint,5,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
-	InstanceControl                *InstanceControl     `protobuf:"bytes,6,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
-	UsScheduler                    *SchedulerAttributes `protobuf:"bytes,7,opt,name=us_scheduler,json=usScheduler,proto3" json:"us_scheduler,omitempty"`
-	DsScheduler                    *SchedulerAttributes `protobuf:"bytes,8,opt,name=ds_scheduler,json=dsScheduler,proto3" json:"ds_scheduler,omitempty"`
-	UpstreamGemPortAttributeList   []*GemPortAttributes `protobuf:"bytes,9,rep,name=upstream_gem_port_attribute_list,json=upstreamGemPortAttributeList,proto3" json:"upstream_gem_port_attribute_list,omitempty"`
-	DownstreamGemPortAttributeList []*GemPortAttributes `protobuf:"bytes,10,rep,name=downstream_gem_port_attribute_list,json=downstreamGemPortAttributeList,proto3" json:"downstream_gem_port_attribute_list,omitempty"`
-	XXX_NoUnkeyedLiteral           struct{}             `json:"-"`
-	XXX_unrecognized               []byte               `json:"-"`
-	XXX_sizecache                  int32                `json:"-"`
-}
-
-func (m *TechProfileInstance) Reset()         { *m = TechProfileInstance{} }
-func (m *TechProfileInstance) String() string { return proto.CompactTextString(m) }
-func (*TechProfileInstance) ProtoMessage()    {}
-func (*TechProfileInstance) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{17}
-}
-
-func (m *TechProfileInstance) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TechProfileInstance.Unmarshal(m, b)
-}
-func (m *TechProfileInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TechProfileInstance.Marshal(b, m, deterministic)
-}
-func (m *TechProfileInstance) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TechProfileInstance.Merge(m, src)
-}
-func (m *TechProfileInstance) XXX_Size() int {
-	return xxx_messageInfo_TechProfileInstance.Size(m)
-}
-func (m *TechProfileInstance) XXX_DiscardUnknown() {
-	xxx_messageInfo_TechProfileInstance.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TechProfileInstance proto.InternalMessageInfo
-
-func (m *TechProfileInstance) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *TechProfileInstance) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *TechProfileInstance) GetSubscriberIdentifier() string {
-	if m != nil {
-		return m.SubscriberIdentifier
-	}
-	return ""
-}
-
-func (m *TechProfileInstance) GetProfileType() string {
-	if m != nil {
-		return m.ProfileType
-	}
-	return ""
-}
-
-func (m *TechProfileInstance) GetNumGemPorts() uint32 {
-	if m != nil {
-		return m.NumGemPorts
-	}
-	return 0
-}
-
-func (m *TechProfileInstance) GetInstanceControl() *InstanceControl {
-	if m != nil {
-		return m.InstanceControl
-	}
-	return nil
-}
-
-func (m *TechProfileInstance) GetUsScheduler() *SchedulerAttributes {
-	if m != nil {
-		return m.UsScheduler
-	}
-	return nil
-}
-
-func (m *TechProfileInstance) GetDsScheduler() *SchedulerAttributes {
-	if m != nil {
-		return m.DsScheduler
-	}
-	return nil
-}
-
-func (m *TechProfileInstance) GetUpstreamGemPortAttributeList() []*GemPortAttributes {
-	if m != nil {
-		return m.UpstreamGemPortAttributeList
-	}
-	return nil
-}
-
-func (m *TechProfileInstance) GetDownstreamGemPortAttributeList() []*GemPortAttributes {
-	if m != nil {
-		return m.DownstreamGemPortAttributeList
-	}
-	return nil
-}
-
-// EPON TechProfile Instance definition.
-type EponTechProfileInstance struct {
-	Name                         string                 `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Version                      uint32                 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
-	SubscriberIdentifier         string                 `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
-	ProfileType                  string                 `protobuf:"bytes,4,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
-	NumGemPorts                  uint32                 `protobuf:"varint,5,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
-	AllocId                      uint32                 `protobuf:"varint,6,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	InstanceControl              *InstanceControl       `protobuf:"bytes,7,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
-	PackageType                  string                 `protobuf:"bytes,8,opt,name=package_type,json=packageType,proto3" json:"package_type,omitempty"`
-	UpstreamQueueAttributeList   []*EPONQueueAttributes `protobuf:"bytes,9,rep,name=upstream_queue_attribute_list,json=upstreamQueueAttributeList,proto3" json:"upstream_queue_attribute_list,omitempty"`
-	DownstreamQueueAttributeList []*EPONQueueAttributes `protobuf:"bytes,10,rep,name=downstream_queue_attribute_list,json=downstreamQueueAttributeList,proto3" json:"downstream_queue_attribute_list,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
-	XXX_unrecognized             []byte                 `json:"-"`
-	XXX_sizecache                int32                  `json:"-"`
-}
-
-func (m *EponTechProfileInstance) Reset()         { *m = EponTechProfileInstance{} }
-func (m *EponTechProfileInstance) String() string { return proto.CompactTextString(m) }
-func (*EponTechProfileInstance) ProtoMessage()    {}
-func (*EponTechProfileInstance) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{18}
-}
-
-func (m *EponTechProfileInstance) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EponTechProfileInstance.Unmarshal(m, b)
-}
-func (m *EponTechProfileInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EponTechProfileInstance.Marshal(b, m, deterministic)
-}
-func (m *EponTechProfileInstance) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EponTechProfileInstance.Merge(m, src)
-}
-func (m *EponTechProfileInstance) XXX_Size() int {
-	return xxx_messageInfo_EponTechProfileInstance.Size(m)
-}
-func (m *EponTechProfileInstance) XXX_DiscardUnknown() {
-	xxx_messageInfo_EponTechProfileInstance.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EponTechProfileInstance proto.InternalMessageInfo
-
-func (m *EponTechProfileInstance) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *EponTechProfileInstance) GetVersion() uint32 {
-	if m != nil {
-		return m.Version
-	}
-	return 0
-}
-
-func (m *EponTechProfileInstance) GetSubscriberIdentifier() string {
-	if m != nil {
-		return m.SubscriberIdentifier
-	}
-	return ""
-}
-
-func (m *EponTechProfileInstance) GetProfileType() string {
-	if m != nil {
-		return m.ProfileType
-	}
-	return ""
-}
-
-func (m *EponTechProfileInstance) GetNumGemPorts() uint32 {
-	if m != nil {
-		return m.NumGemPorts
-	}
-	return 0
-}
-
-func (m *EponTechProfileInstance) GetAllocId() uint32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-func (m *EponTechProfileInstance) GetInstanceControl() *InstanceControl {
-	if m != nil {
-		return m.InstanceControl
-	}
-	return nil
-}
-
-func (m *EponTechProfileInstance) GetPackageType() string {
-	if m != nil {
-		return m.PackageType
-	}
-	return ""
-}
-
-func (m *EponTechProfileInstance) GetUpstreamQueueAttributeList() []*EPONQueueAttributes {
-	if m != nil {
-		return m.UpstreamQueueAttributeList
-	}
-	return nil
-}
-
-func (m *EponTechProfileInstance) GetDownstreamQueueAttributeList() []*EPONQueueAttributes {
-	if m != nil {
-		return m.DownstreamQueueAttributeList
-	}
-	return nil
-}
-
-// Resource Instance definition
-type ResourceInstance struct {
-	TpId                 uint32   `protobuf:"varint,1,opt,name=tp_id,json=tpId,proto3" json:"tp_id,omitempty"`
-	ProfileType          string   `protobuf:"bytes,2,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
-	SubscriberIdentifier string   `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
-	AllocId              uint32   `protobuf:"varint,4,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
-	GemportIds           []uint32 `protobuf:"varint,5,rep,packed,name=gemport_ids,json=gemportIds,proto3" json:"gemport_ids,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ResourceInstance) Reset()         { *m = ResourceInstance{} }
-func (m *ResourceInstance) String() string { return proto.CompactTextString(m) }
-func (*ResourceInstance) ProtoMessage()    {}
-func (*ResourceInstance) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d019a68bffe14cae, []int{19}
-}
-
-func (m *ResourceInstance) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResourceInstance.Unmarshal(m, b)
-}
-func (m *ResourceInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResourceInstance.Marshal(b, m, deterministic)
-}
-func (m *ResourceInstance) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResourceInstance.Merge(m, src)
-}
-func (m *ResourceInstance) XXX_Size() int {
-	return xxx_messageInfo_ResourceInstance.Size(m)
-}
-func (m *ResourceInstance) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResourceInstance.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResourceInstance proto.InternalMessageInfo
-
-func (m *ResourceInstance) GetTpId() uint32 {
-	if m != nil {
-		return m.TpId
-	}
-	return 0
-}
-
-func (m *ResourceInstance) GetProfileType() string {
-	if m != nil {
-		return m.ProfileType
-	}
-	return ""
-}
-
-func (m *ResourceInstance) GetSubscriberIdentifier() string {
-	if m != nil {
-		return m.SubscriberIdentifier
-	}
-	return ""
-}
-
-func (m *ResourceInstance) GetAllocId() uint32 {
-	if m != nil {
-		return m.AllocId
-	}
-	return 0
-}
-
-func (m *ResourceInstance) GetGemportIds() []uint32 {
-	if m != nil {
-		return m.GemportIds
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("tech_profile.Direction", Direction_name, Direction_value)
-	proto.RegisterEnum("tech_profile.SchedulingPolicy", SchedulingPolicy_name, SchedulingPolicy_value)
-	proto.RegisterEnum("tech_profile.AdditionalBW", AdditionalBW_name, AdditionalBW_value)
-	proto.RegisterEnum("tech_profile.DiscardPolicy", DiscardPolicy_name, DiscardPolicy_value)
-	proto.RegisterEnum("tech_profile.InferredAdditionBWIndication", InferredAdditionBWIndication_name, InferredAdditionBWIndication_value)
-	proto.RegisterType((*SchedulerConfig)(nil), "tech_profile.SchedulerConfig")
-	proto.RegisterType((*TrafficShapingInfo)(nil), "tech_profile.TrafficShapingInfo")
-	proto.RegisterType((*TrafficScheduler)(nil), "tech_profile.TrafficScheduler")
-	proto.RegisterType((*TrafficSchedulers)(nil), "tech_profile.TrafficSchedulers")
-	proto.RegisterType((*TailDropDiscardConfig)(nil), "tech_profile.TailDropDiscardConfig")
-	proto.RegisterType((*RedDiscardConfig)(nil), "tech_profile.RedDiscardConfig")
-	proto.RegisterType((*WRedDiscardConfig)(nil), "tech_profile.WRedDiscardConfig")
-	proto.RegisterType((*DiscardConfig)(nil), "tech_profile.DiscardConfig")
-	proto.RegisterType((*TrafficQueue)(nil), "tech_profile.TrafficQueue")
-	proto.RegisterType((*TrafficQueues)(nil), "tech_profile.TrafficQueues")
-	proto.RegisterType((*InstanceControl)(nil), "tech_profile.InstanceControl")
-	proto.RegisterType((*QThresholds)(nil), "tech_profile.QThresholds")
-	proto.RegisterType((*GemPortAttributes)(nil), "tech_profile.GemPortAttributes")
-	proto.RegisterType((*SchedulerAttributes)(nil), "tech_profile.SchedulerAttributes")
-	proto.RegisterType((*EPONQueueAttributes)(nil), "tech_profile.EPONQueueAttributes")
-	proto.RegisterType((*TechProfile)(nil), "tech_profile.TechProfile")
-	proto.RegisterType((*EponTechProfile)(nil), "tech_profile.EponTechProfile")
-	proto.RegisterType((*TechProfileInstance)(nil), "tech_profile.TechProfileInstance")
-	proto.RegisterType((*EponTechProfileInstance)(nil), "tech_profile.EponTechProfileInstance")
-	proto.RegisterType((*ResourceInstance)(nil), "tech_profile.ResourceInstance")
-}
-
-func init() { proto.RegisterFile("voltha_protos/tech_profile.proto", fileDescriptor_d019a68bffe14cae) }
-
-var fileDescriptor_d019a68bffe14cae = []byte{
-	// 2138 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0x4d, 0x6f, 0x1b, 0xb9,
-	0xf9, 0xb7, 0x24, 0x5b, 0x2f, 0x8f, 0x24, 0x7b, 0x4c, 0xc7, 0x1b, 0xc5, 0x71, 0x36, 0x8e, 0xf6,
-	0xbf, 0xff, 0x75, 0x5d, 0x34, 0xee, 0x3a, 0x4e, 0xf6, 0x90, 0x6d, 0x17, 0x52, 0x6c, 0x24, 0x6a,
-	0x37, 0x8e, 0x3d, 0x76, 0xeb, 0xa2, 0x87, 0x0e, 0x46, 0x33, 0x94, 0xcc, 0xee, 0x88, 0x1c, 0x93,
-	0x9c, 0x38, 0xde, 0x53, 0x51, 0xa0, 0x9f, 0xa2, 0xed, 0xa5, 0x40, 0xaf, 0xbd, 0xec, 0xa5, 0x68,
-	0x2f, 0x05, 0xfa, 0x2d, 0xfa, 0x09, 0x0a, 0xf4, 0x4b, 0x14, 0xe4, 0xcc, 0x68, 0x5e, 0xa4, 0x24,
-	0x76, 0xaa, 0x2c, 0xd0, 0xde, 0xc8, 0x87, 0x3f, 0x3e, 0x7c, 0xf8, 0xbc, 0xfc, 0x48, 0xce, 0xc0,
-	0xc6, 0x4b, 0xe6, 0xc9, 0x33, 0xdb, 0xf2, 0x39, 0x93, 0x4c, 0x6c, 0x4b, 0xec, 0x9c, 0xa9, 0xf6,
-	0x80, 0x78, 0xf8, 0xbe, 0x96, 0xa1, 0x46, 0x5a, 0xb6, 0xb6, 0x3e, 0x64, 0x6c, 0xe8, 0xe1, 0x6d,
-	0xdb, 0x27, 0xdb, 0x36, 0xa5, 0x4c, 0xda, 0x92, 0x30, 0x2a, 0x42, 0x6c, 0xfb, 0x57, 0x45, 0x58,
-	0x3a, 0x76, 0xce, 0xb0, 0x1b, 0x78, 0x98, 0x3f, 0x61, 0x74, 0x40, 0x86, 0xe8, 0x21, 0xd4, 0x5c,
-	0xc2, 0xb1, 0xa3, 0x70, 0xad, 0xc2, 0x46, 0x61, 0x73, 0x71, 0xe7, 0xe6, 0xfd, 0xcc, 0x3a, 0x7b,
-	0xf1, 0xb0, 0x99, 0x20, 0xd1, 0x17, 0xd0, 0xb4, 0x5d, 0x97, 0xa8, 0xb6, 0xed, 0x59, 0xfd, 0x8b,
-	0x56, 0x51, 0x4f, 0x5d, 0xcb, 0x4e, 0xed, 0x8c, 0x21, 0xdd, 0x53, 0xb3, 0x91, 0x4c, 0xe8, 0x5e,
-	0xa0, 0x35, 0xa8, 0xfa, 0x9c, 0x30, 0x4e, 0xe4, 0x65, 0xab, 0xb4, 0x51, 0xd8, 0xac, 0x98, 0xe3,
-	0x3e, 0xfa, 0x00, 0xca, 0x17, 0x98, 0x0c, 0xcf, 0x64, 0x6b, 0x5e, 0x8f, 0x44, 0x3d, 0xd4, 0x81,
-	0x86, 0x50, 0xe6, 0x5b, 0x3e, 0xf3, 0x88, 0x73, 0xd9, 0x5a, 0xd0, 0x6b, 0x7e, 0x98, 0x5d, 0x33,
-	0xda, 0x20, 0xa1, 0xc3, 0x43, 0x8d, 0x32, 0xeb, 0x7a, 0x4e, 0xd8, 0x69, 0xff, 0xb9, 0x00, 0xe8,
-	0x84, 0xdb, 0x83, 0x01, 0x71, 0x8e, 0xcf, 0x6c, 0x9f, 0xd0, 0x61, 0x8f, 0x0e, 0x18, 0x32, 0xa0,
-	0xe4, 0x10, 0xae, 0xf7, 0x5f, 0x31, 0x55, 0x53, 0x4b, 0xfa, 0x42, 0x6f, 0x4b, 0x49, 0xfa, 0x42,
-	0x49, 0x7c, 0xc2, 0x23, 0x63, 0x55, 0x53, 0x4b, 0xfa, 0x22, 0x32, 0x52, 0x35, 0x95, 0x64, 0x48,
-	0xb8, 0x36, 0xac, 0x62, 0xaa, 0x26, 0x7a, 0x06, 0x60, 0xbb, 0xae, 0xd5, 0xbf, 0xb0, 0x08, 0x75,
-	0x5b, 0x65, 0x6d, 0xf1, 0x56, 0xd6, 0xe2, 0x1e, 0x1d, 0x60, 0xce, 0xb1, 0x1b, 0x7b, 0xab, 0x7b,
-	0xda, 0xa3, 0x2e, 0x71, 0x74, 0xe8, 0xcc, 0xaa, 0xed, 0xba, 0xdd, 0x8b, 0x1e, 0x75, 0xdb, 0xbf,
-	0x2f, 0x82, 0x11, 0x9b, 0x1e, 0x07, 0xf1, 0x5d, 0xc3, 0x77, 0x0b, 0xaa, 0xb6, 0xe7, 0x31, 0xc7,
-	0x22, 0x6e, 0xb4, 0xc5, 0x8a, 0xee, 0xf7, 0x5c, 0xf4, 0x18, 0x6a, 0x22, 0x56, 0xaf, 0x37, 0x5b,
-	0xdf, 0xb9, 0x33, 0xd5, 0xc3, 0x71, 0x0a, 0x99, 0x09, 0x1e, 0x99, 0x70, 0x43, 0x86, 0x26, 0x5a,
-	0x22, 0x74, 0xaf, 0x45, 0xe8, 0x80, 0x69, 0x17, 0xd5, 0x77, 0x36, 0xb2, 0x7a, 0x26, 0xe3, 0x60,
-	0x22, 0x39, 0x19, 0x9b, 0xff, 0x87, 0xa5, 0xf4, 0x34, 0x65, 0x72, 0xe8, 0xdf, 0xa6, 0x12, 0x1f,
-	0x86, 0xd2, 0x9e, 0xdb, 0xfe, 0x4b, 0x01, 0x96, 0xf3, 0xfe, 0x11, 0xe8, 0x26, 0x54, 0x08, 0x95,
-	0x03, 0x35, 0x2b, 0x8c, 0x6e, 0x59, 0x75, 0x7b, 0x2e, 0x5a, 0x85, 0x32, 0xa3, 0x41, 0xe2, 0x80,
-	0x05, 0x46, 0x83, 0x50, 0x1c, 0x50, 0xa2, 0xc4, 0x61, 0x58, 0x17, 0x02, 0x4a, 0x7a, 0xae, 0x52,
-	0xe3, 0x33, 0x2e, 0x2d, 0xca, 0xa2, 0xc5, 0xcb, 0xaa, 0x7b, 0xc0, 0xd0, 0x3e, 0x2c, 0x8e, 0x77,
-	0xac, 0x56, 0x15, 0xad, 0xd2, 0x46, 0x69, 0xb3, 0x9e, 0xcf, 0xca, 0xbc, 0x61, 0x66, 0x53, 0xa6,
-	0x24, 0xa2, 0xfd, 0x08, 0x56, 0x4f, 0x6c, 0xe2, 0xed, 0x71, 0xe6, 0xef, 0x11, 0xe1, 0xd8, 0xdc,
-	0x8d, 0xea, 0xf3, 0x0e, 0xc0, 0x79, 0x80, 0x03, 0x6c, 0x09, 0xf2, 0x35, 0x8e, 0xb6, 0x50, 0xd3,
-	0x92, 0x63, 0xf2, 0x35, 0x6e, 0xff, 0xa6, 0x00, 0x86, 0x89, 0xdd, 0xec, 0x9c, 0x8f, 0xa0, 0x39,
-	0x22, 0xd4, 0x92, 0x67, 0x1c, 0x8b, 0x33, 0xe6, 0xc5, 0x3b, 0x6f, 0x8c, 0x08, 0x3d, 0x89, 0x65,
-	0x1a, 0x64, 0xbf, 0x4a, 0x81, 0x8a, 0x11, 0xc8, 0x7e, 0x95, 0x80, 0x3e, 0x81, 0x25, 0x05, 0xf2,
-	0x39, 0xeb, 0xdb, 0x7d, 0xe2, 0x25, 0xc5, 0xba, 0x38, 0xb2, 0x5f, 0x1d, 0x26, 0xd2, 0xf6, 0x37,
-	0x05, 0x58, 0x3e, 0x9d, 0x30, 0x64, 0x17, 0x16, 0x86, 0x1c, 0xe3, 0x30, 0x33, 0x27, 0x7c, 0x92,
-	0x87, 0x9b, 0x21, 0x18, 0x3d, 0x82, 0xf2, 0x25, 0xf6, 0x3c, 0x16, 0x92, 0xca, 0xdb, 0xa7, 0x45,
-	0x68, 0xf4, 0x7d, 0x28, 0x71, 0xec, 0x46, 0x39, 0xfb, 0xb6, 0x49, 0x0a, 0xda, 0xfe, 0x67, 0x11,
-	0x9a, 0x59, 0x8b, 0xbb, 0xb0, 0xe8, 0x86, 0x82, 0x98, 0x64, 0xc2, 0xa2, 0xba, 0x9d, 0x2f, 0x2a,
-	0x8d, 0x89, 0x18, 0xa6, 0xe9, 0xa6, 0xbb, 0xe8, 0x17, 0xd0, 0x92, 0x36, 0xf1, 0x2c, 0x97, 0x33,
-	0xdf, 0x8a, 0xb5, 0x39, 0x5a, 0x7f, 0xb4, 0xa3, 0x8f, 0x72, 0xc9, 0x31, 0x2d, 0xf2, 0xcf, 0xe6,
-	0xcc, 0x55, 0x39, 0x35, 0x25, 0x0e, 0x00, 0x71, 0xec, 0xe6, 0x35, 0x5f, 0x69, 0xdb, 0xcf, 0xe6,
-	0x4c, 0x83, 0xe7, 0xa3, 0x74, 0x04, 0x2b, 0x17, 0x53, 0x14, 0x86, 0x35, 0x7b, 0x37, 0xab, 0xf0,
-	0x74, 0x8a, 0xc6, 0xe5, 0x8b, 0xbc, 0xca, 0xae, 0x91, 0xb8, 0x31, 0xd4, 0xd6, 0xfe, 0x63, 0x09,
-	0x1a, 0x51, 0x11, 0x1c, 0xa9, 0xec, 0x7d, 0x57, 0xe6, 0xba, 0x03, 0x30, 0xc4, 0x23, 0x5d, 0x8b,
-	0xe3, 0xd2, 0xad, 0x45, 0x92, 0x9e, 0xab, 0x88, 0xcd, 0xef, 0x13, 0x69, 0x8d, 0x6c, 0x5f, 0x7b,
-	0xa4, 0x66, 0x56, 0x54, 0xff, 0xb9, 0xed, 0xa3, 0x8f, 0x61, 0xd1, 0xc6, 0xc2, 0xc2, 0xd4, 0xe1,
-	0x97, 0xbe, 0x5e, 0x55, 0xed, 0xb0, 0x6a, 0x36, 0x6d, 0x2c, 0xf6, 0xc7, 0xc2, 0x19, 0x1c, 0x32,
-	0x99, 0xb3, 0xad, 0xfc, 0xda, 0xb3, 0xad, 0x92, 0x39, 0xdb, 0x26, 0x13, 0xaf, 0x7a, 0xed, 0xc4,
-	0xeb, 0xe6, 0xbd, 0xde, 0xaa, 0xe9, 0x18, 0x4e, 0xd7, 0x11, 0x15, 0x42, 0xac, 0x23, 0xec, 0xb6,
-	0xff, 0x51, 0x80, 0x66, 0x3a, 0x4e, 0xef, 0x9f, 0x41, 0x3b, 0x09, 0x83, 0x6a, 0x5e, 0x13, 0xad,
-	0xb2, 0x66, 0xd0, 0xb5, 0xa9, 0x0c, 0xaa, 0x8d, 0x1a, 0xb3, 0x67, 0x64, 0xe2, 0x94, 0x23, 0xa2,
-	0x32, 0xed, 0x88, 0x18, 0xc0, 0x52, 0x8f, 0x0a, 0x69, 0x53, 0x07, 0x3f, 0x61, 0x54, 0x72, 0xe6,
-	0xa9, 0x13, 0x9b, 0xd1, 0x40, 0xef, 0xac, 0x66, 0xaa, 0xa6, 0x92, 0x04, 0x94, 0xe8, 0x3d, 0xd5,
-	0x4c, 0xd5, 0x44, 0xdb, 0x70, 0x43, 0xb1, 0xe0, 0x10, 0x8f, 0x2c, 0xdf, 0xbe, 0xf4, 0x98, 0xed,
-	0x86, 0x6c, 0x1c, 0x26, 0xd8, 0xf2, 0xc8, 0x7e, 0xf5, 0x14, 0x8f, 0x0e, 0xc3, 0x11, 0xcd, 0xca,
-	0xbf, 0x2e, 0x42, 0xfd, 0x68, 0xcc, 0xa2, 0x02, 0xdd, 0x83, 0xc6, 0x79, 0xc2, 0xb4, 0x9f, 0xea,
-	0xd5, 0x9a, 0x66, 0xfd, 0x7c, 0x0c, 0xf9, 0x34, 0x07, 0xd9, 0xd1, 0xcb, 0x67, 0x20, 0x3b, 0x39,
-	0xc8, 0x03, 0xbd, 0x7c, 0x06, 0xf2, 0x20, 0x07, 0xd9, 0xd5, 0x11, 0xc8, 0x40, 0x76, 0x73, 0x90,
-	0x87, 0x3a, 0x18, 0x19, 0xc8, 0xc3, 0x1c, 0xe4, 0x91, 0xce, 0xe1, 0x0c, 0xe4, 0x51, 0x0e, 0xf2,
-	0x99, 0x76, 0x77, 0x06, 0xf2, 0x59, 0xfb, 0x9b, 0x05, 0x58, 0x56, 0x7e, 0x61, 0x5c, 0x76, 0xa4,
-	0xe4, 0xa4, 0x1f, 0x48, 0x2c, 0x72, 0xf5, 0x5b, 0xc8, 0xd7, 0xef, 0x3a, 0x80, 0x72, 0xf5, 0x79,
-	0xe8, 0xe0, 0x30, 0x06, 0xd5, 0x91, 0xfd, 0xea, 0x48, 0xf9, 0xf5, 0xfa, 0xd5, 0x5d, 0xcb, 0x57,
-	0xf7, 0x8f, 0x61, 0x59, 0x8c, 0x6b, 0xf7, 0x7a, 0x25, 0x6e, 0x88, 0x9c, 0x44, 0xed, 0x25, 0xae,
-	0x6b, 0xeb, 0x3c, 0xaa, 0xf4, 0x5a, 0x2c, 0x39, 0x7a, 0xaf, 0xa5, 0xbe, 0xff, 0x9a, 0x52, 0x7f,
-	0xdb, 0xb1, 0x97, 0xad, 0x76, 0xf4, 0x14, 0x96, 0xb3, 0x6a, 0xac, 0x97, 0x3b, 0xad, 0xc5, 0xb7,
-	0x93, 0xc6, 0x52, 0x46, 0xcd, 0x4f, 0x75, 0x6e, 0x12, 0x61, 0x8d, 0x02, 0x4f, 0x12, 0xc7, 0x16,
-	0xb2, 0x05, 0xda, 0xf9, 0x75, 0x22, 0x9e, 0xc7, 0x22, 0xb4, 0x09, 0xc6, 0x78, 0x5c, 0xd7, 0x12,
-	0x71, 0x5b, 0xf5, 0xe8, 0x32, 0x11, 0xcb, 0x9f, 0xe2, 0x51, 0xcf, 0x45, 0x3f, 0x80, 0xdb, 0xee,
-	0x25, 0xb5, 0x47, 0xc4, 0xb1, 0x6c, 0xc7, 0xc1, 0x42, 0x28, 0xe3, 0x54, 0xb5, 0x5a, 0x1e, 0x11,
-	0xb2, 0xd5, 0xd0, 0xba, 0x5b, 0x11, 0xa4, 0xa3, 0x11, 0x51, 0x39, 0x7f, 0x49, 0x84, 0x44, 0x8f,
-	0x61, 0x4d, 0xa8, 0x87, 0xcf, 0xf4, 0xd9, 0x4d, 0x3d, 0xfb, 0x66, 0x88, 0x98, 0x98, 0xdc, 0xfe,
-	0x43, 0x11, 0x56, 0xc6, 0xb7, 0xb4, 0x54, 0xde, 0xce, 0xe8, 0xa2, 0xdd, 0x4c, 0x2e, 0xda, 0x13,
-	0x4f, 0xa8, 0xd2, 0x7f, 0xf0, 0x84, 0x9a, 0x7f, 0xed, 0x31, 0xb3, 0x90, 0xc9, 0xbd, 0x3d, 0x58,
-	0x3c, 0xb7, 0x32, 0xe7, 0x5b, 0xf9, 0x4a, 0xc9, 0xdf, 0x38, 0x3f, 0x4e, 0xbd, 0xa2, 0xfe, 0x56,
-	0x86, 0x95, 0xfd, 0xc3, 0x17, 0x07, 0x9a, 0x7e, 0x53, 0x4e, 0xca, 0x56, 0x6f, 0xe1, 0x0d, 0xd5,
-	0x5b, 0xcc, 0x56, 0x6f, 0x96, 0x15, 0x42, 0x62, 0x4b, 0xb1, 0xc2, 0x15, 0x8b, 0xfb, 0x1e, 0x34,
-	0xe2, 0x93, 0x44, 0x5e, 0xfa, 0x58, 0x6f, 0xbd, 0x66, 0xd6, 0x23, 0xd9, 0xc9, 0xa5, 0x8f, 0xd1,
-	0x2e, 0x7c, 0x10, 0x50, 0xa1, 0x76, 0x41, 0x24, 0x76, 0xad, 0x21, 0xb7, 0xa9, 0x0c, 0xad, 0x0d,
-	0x49, 0xee, 0x46, 0x6a, 0xf4, 0xa9, 0x1a, 0xd4, 0x96, 0x7f, 0x07, 0x0c, 0xca, 0x46, 0x44, 0xc5,
-	0x89, 0x50, 0x89, 0xf9, 0x4b, 0xdb, 0x8b, 0x18, 0x6f, 0x29, 0x92, 0xf7, 0x22, 0x31, 0xda, 0x81,
-	0x55, 0xc9, 0x3c, 0xcc, 0x6d, 0x19, 0xba, 0xd8, 0xb3, 0x7e, 0x49, 0xa4, 0xc4, 0x5c, 0xd7, 0x78,
-	0xd3, 0x5c, 0x19, 0x0f, 0x1e, 0x32, 0xcf, 0xfb, 0x91, 0x1e, 0x42, 0x3f, 0x84, 0xdb, 0x1c, 0x9f,
-	0x07, 0x58, 0x48, 0x4b, 0x72, 0x9b, 0x8a, 0x11, 0x11, 0x82, 0x30, 0x1a, 0x47, 0xa8, 0xa6, 0x67,
-	0xde, 0x8a, 0x20, 0x27, 0x29, 0x44, 0x44, 0x06, 0xeb, 0x00, 0x34, 0x18, 0x29, 0xb7, 0x63, 0x29,
-	0x74, 0xe9, 0x35, 0xcd, 0x2a, 0x0d, 0x46, 0x47, 0xc7, 0x58, 0x0a, 0xf4, 0x79, 0x86, 0xaa, 0x85,
-	0xae, 0xb9, 0xfa, 0xce, 0xad, 0x6c, 0xc0, 0x53, 0xa7, 0x55, 0x9a, 0xc5, 0xc5, 0x74, 0xc2, 0x6c,
-	0xcc, 0x84, 0x30, 0x9b, 0x61, 0x98, 0xa7, 0x11, 0xe6, 0xa2, 0x1e, 0x7a, 0x3d, 0x61, 0x2e, 0xcd,
-	0x80, 0x30, 0x8d, 0x99, 0x11, 0xe6, 0xf2, 0xf5, 0x09, 0xb3, 0xfd, 0xa7, 0x79, 0xa8, 0x9f, 0x24,
-	0x97, 0x13, 0x84, 0x60, 0x9e, 0xda, 0xa3, 0xb8, 0x68, 0x74, 0x1b, 0xb5, 0xa0, 0xf2, 0x12, 0x73,
-	0x15, 0xe8, 0x98, 0x3b, 0xa2, 0xae, 0xca, 0xf4, 0xf8, 0xae, 0xa3, 0x33, 0x3d, 0x3c, 0x0c, 0xeb,
-	0x91, 0x4c, 0x67, 0x7a, 0x1b, 0x9a, 0x2a, 0x29, 0xf4, 0xa5, 0x85, 0x71, 0x29, 0xe2, 0xbb, 0x00,
-	0x0d, 0x46, 0xd1, 0xa9, 0x2c, 0xd0, 0x33, 0x30, 0x48, 0x74, 0x1f, 0x8a, 0x49, 0x52, 0x17, 0xcd,
-	0xc4, 0x93, 0x3f, 0x77, 0x6b, 0x32, 0x97, 0x48, 0xee, 0x1a, 0xb5, 0x07, 0x8d, 0x40, 0x58, 0xc9,
-	0x87, 0x83, 0xb2, 0xd6, 0x72, 0xef, 0x35, 0x1f, 0x0e, 0x12, 0xca, 0x30, 0xeb, 0x81, 0x48, 0xbe,
-	0x66, 0xec, 0x41, 0xc3, 0x4d, 0x6b, 0xa9, 0x5c, 0x59, 0x8b, 0x9b, 0xd2, 0x32, 0x84, 0x8d, 0xc0,
-	0x17, 0x92, 0x63, 0x3b, 0xd9, 0xbe, 0x65, 0xc7, 0xe0, 0xf0, 0x14, 0xa8, 0xea, 0x2b, 0x66, 0xee,
-	0x71, 0x33, 0x71, 0x5b, 0x31, 0xd7, 0x63, 0x45, 0xf9, 0x21, 0x7d, 0xd0, 0x7c, 0x05, 0x6d, 0x97,
-	0x5d, 0xd0, 0xb7, 0x2c, 0x55, 0xbb, 0xda, 0x52, 0x1f, 0x26, 0xaa, 0xa6, 0x2d, 0xd6, 0xfe, 0x7b,
-	0x09, 0x96, 0xf6, 0x7d, 0x46, 0xff, 0x87, 0x92, 0x46, 0x19, 0x64, 0x3b, 0x5f, 0xd9, 0xc3, 0xc8,
-	0xa0, 0x72, 0x64, 0x50, 0x28, 0xd3, 0x06, 0xb9, 0x70, 0x67, 0x1c, 0xcb, 0xf0, 0x3b, 0x48, 0xce,
-	0xbb, 0x15, 0xed, 0xdd, 0x5c, 0x8a, 0x4c, 0x39, 0x9b, 0xcc, 0xb5, 0x58, 0x4f, 0x76, 0x40, 0x07,
-	0xf2, 0x0c, 0xee, 0xa6, 0x02, 0x39, 0x75, 0x9d, 0xea, 0x55, 0xd7, 0x59, 0x4f, 0x34, 0x4d, 0xae,
-	0xd4, 0xfe, 0xd7, 0x3c, 0xac, 0xa4, 0x22, 0x18, 0xbb, 0xe8, 0x9a, 0x91, 0x7c, 0x00, 0xab, 0x22,
-	0xe8, 0x0b, 0x87, 0x93, 0x3e, 0xe6, 0x16, 0x71, 0x31, 0x95, 0x64, 0x40, 0xa2, 0xef, 0x75, 0x35,
-	0xf3, 0x46, 0x32, 0xd8, 0x1b, 0x8f, 0x4d, 0x84, 0x7f, 0xfe, 0x0a, 0xe1, 0x5f, 0xb8, 0x5a, 0xf8,
-	0xcb, 0x33, 0xe1, 0x8c, 0xca, 0x4c, 0x38, 0xa3, 0xfa, 0xde, 0x38, 0xa3, 0xf6, 0xed, 0x71, 0x06,
-	0xcc, 0x86, 0x33, 0x7e, 0x3b, 0x0f, 0x37, 0x73, 0x9c, 0xf1, 0x5f, 0x98, 0x71, 0xe9, 0x3b, 0x74,
-	0x39, 0x7b, 0x87, 0x9e, 0x96, 0x8c, 0x95, 0x99, 0x70, 0x51, 0xf5, 0x1d, 0xb8, 0xa8, 0xf6, 0x2d,
-	0x71, 0x11, 0xcc, 0x86, 0x8b, 0xfe, 0xaa, 0xbf, 0x1d, 0x0b, 0x16, 0x70, 0x27, 0x49, 0x8b, 0x15,
-	0x58, 0x90, 0x7e, 0xfc, 0x34, 0x6f, 0x9a, 0xf3, 0xd2, 0xef, 0xb9, 0x13, 0x81, 0x2c, 0x4e, 0x06,
-	0xf2, 0x9d, 0x12, 0x24, 0x1d, 0xd9, 0xf9, 0x6c, 0x64, 0xef, 0x42, 0x3d, 0x79, 0x11, 0xa8, 0xb4,
-	0x28, 0x6d, 0x36, 0x4d, 0x18, 0x3f, 0x09, 0xc4, 0xd6, 0xe7, 0x50, 0x1b, 0xbf, 0xb8, 0x50, 0x03,
-	0xaa, 0x3f, 0x39, 0x3c, 0x3e, 0x31, 0xf7, 0x3b, 0xcf, 0x8d, 0x39, 0xb4, 0x08, 0xb0, 0xf7, 0xe2,
-	0xf4, 0x20, 0xea, 0x17, 0xd0, 0x32, 0x34, 0xbb, 0xbd, 0xbd, 0x9e, 0xb9, 0xff, 0xe4, 0xa4, 0xf7,
-	0xe2, 0xa0, 0xf3, 0xa5, 0x51, 0xdc, 0x7a, 0x0c, 0x46, 0xfe, 0xbe, 0x8a, 0x2a, 0x50, 0x3a, 0x35,
-	0x4d, 0x63, 0x0e, 0x21, 0x58, 0x3c, 0x96, 0x9c, 0x38, 0xf2, 0x30, 0xba, 0x9a, 0x1a, 0x05, 0x04,
-	0x50, 0x7e, 0x76, 0xd9, 0xe7, 0xc4, 0x35, 0x8a, 0x5b, 0x14, 0x1a, 0xe9, 0x67, 0x19, 0x5a, 0x85,
-	0xe5, 0x74, 0xdf, 0x3a, 0x60, 0x14, 0x1b, 0x73, 0x68, 0x05, 0x96, 0xb2, 0xe2, 0x8e, 0x51, 0x40,
-	0xb7, 0xe1, 0x66, 0x46, 0xd8, 0xc5, 0x42, 0xee, 0x0f, 0x06, 0x8c, 0x4b, 0xa3, 0x38, 0xa1, 0xa8,
-	0x13, 0x48, 0x66, 0x94, 0xb6, 0xbe, 0x18, 0x7f, 0xa5, 0x8e, 0x2c, 0x6d, 0x40, 0x35, 0xfe, 0x66,
-	0x6c, 0xcc, 0xa1, 0x26, 0xd4, 0x4e, 0xc7, 0xdd, 0x82, 0xda, 0x86, 0x89, 0x5d, 0xa3, 0x88, 0xaa,
-	0x30, 0x7f, 0xaa, 0x5a, 0xa5, 0xad, 0xdf, 0x15, 0x60, 0xfd, 0x4d, 0x7f, 0x99, 0xd0, 0xc7, 0x70,
-	0xef, 0x4d, 0xe3, 0xf1, 0x8e, 0x36, 0xe1, 0xff, 0xde, 0x08, 0xeb, 0x08, 0x11, 0x70, 0xec, 0x1a,
-	0x05, 0xf4, 0x5d, 0xf8, 0xe4, 0x8d, 0xc8, 0xf4, 0xb6, 0xbb, 0x3f, 0x83, 0x0d, 0xc6, 0x87, 0xf7,
-	0x99, 0x8f, 0xa9, 0xc3, 0xb8, 0x7b, 0x3f, 0xfc, 0xe9, 0x99, 0x49, 0xef, 0x9f, 0xef, 0x0e, 0x89,
-	0x3c, 0x0b, 0xfa, 0xf7, 0x1d, 0x36, 0xda, 0x8e, 0x81, 0xdb, 0x21, 0xf0, 0x7b, 0xd1, 0xdf, 0xd1,
-	0x97, 0xbb, 0xdb, 0x43, 0x96, 0xf9, 0x47, 0xda, 0x2f, 0xeb, 0xa1, 0x07, 0xff, 0x0e, 0x00, 0x00,
-	0xff, 0xff, 0xc7, 0x6b, 0x83, 0x2e, 0x48, 0x1d, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/adapter.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/adapter.pb.go
deleted file mode 100644
index bb868a4..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/adapter.pb.go
+++ /dev/null
@@ -1,271 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/adapter.proto
-
-package voltha
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	any "github.com/golang/protobuf/ptypes/any"
-	timestamp "github.com/golang/protobuf/ptypes/timestamp"
-	_ "github.com/opencord/voltha-protos/v4/go/common"
-	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 AdapterConfig struct {
-	// Custom (vendor-specific) configuration attributes
-	AdditionalConfig     *any.Any `protobuf:"bytes,64,opt,name=additional_config,json=additionalConfig,proto3" json:"additional_config,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *AdapterConfig) Reset()         { *m = AdapterConfig{} }
-func (m *AdapterConfig) String() string { return proto.CompactTextString(m) }
-func (*AdapterConfig) ProtoMessage()    {}
-func (*AdapterConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7e998ce153307274, []int{0}
-}
-
-func (m *AdapterConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AdapterConfig.Unmarshal(m, b)
-}
-func (m *AdapterConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AdapterConfig.Marshal(b, m, deterministic)
-}
-func (m *AdapterConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AdapterConfig.Merge(m, src)
-}
-func (m *AdapterConfig) XXX_Size() int {
-	return xxx_messageInfo_AdapterConfig.Size(m)
-}
-func (m *AdapterConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_AdapterConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AdapterConfig proto.InternalMessageInfo
-
-func (m *AdapterConfig) GetAdditionalConfig() *any.Any {
-	if m != nil {
-		return m.AdditionalConfig
-	}
-	return nil
-}
-
-// Adapter (software plugin)
-type Adapter struct {
-	// the adapter ID has to be unique,
-	// it will be generated as Type + CurrentReplica
-	Id      string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Vendor  string `protobuf:"bytes,2,opt,name=vendor,proto3" json:"vendor,omitempty"`
-	Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"`
-	// Adapter configuration
-	Config *AdapterConfig `protobuf:"bytes,16,opt,name=config,proto3" json:"config,omitempty"`
-	// Custom descriptors and custom configuration
-	AdditionalDescription *any.Any `protobuf:"bytes,64,opt,name=additional_description,json=additionalDescription,proto3" json:"additional_description,omitempty"`
-	LogicalDeviceIds      []string `protobuf:"bytes,4,rep,name=logical_device_ids,json=logicalDeviceIds,proto3" json:"logical_device_ids,omitempty"`
-	// timestamp when the adapter last sent a message to the core
-	LastCommunication *timestamp.Timestamp `protobuf:"bytes,5,opt,name=last_communication,json=lastCommunication,proto3" json:"last_communication,omitempty"`
-	CurrentReplica    int32                `protobuf:"varint,6,opt,name=currentReplica,proto3" json:"currentReplica,omitempty"`
-	TotalReplicas     int32                `protobuf:"varint,7,opt,name=totalReplicas,proto3" json:"totalReplicas,omitempty"`
-	Endpoint          string               `protobuf:"bytes,8,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
-	// all replicas of the same adapter will have the same type
-	// it is used to associate a device to an adapter
-	Type                 string   `protobuf:"bytes,9,opt,name=type,proto3" json:"type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Adapter) Reset()         { *m = Adapter{} }
-func (m *Adapter) String() string { return proto.CompactTextString(m) }
-func (*Adapter) ProtoMessage()    {}
-func (*Adapter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7e998ce153307274, []int{1}
-}
-
-func (m *Adapter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Adapter.Unmarshal(m, b)
-}
-func (m *Adapter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Adapter.Marshal(b, m, deterministic)
-}
-func (m *Adapter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Adapter.Merge(m, src)
-}
-func (m *Adapter) XXX_Size() int {
-	return xxx_messageInfo_Adapter.Size(m)
-}
-func (m *Adapter) XXX_DiscardUnknown() {
-	xxx_messageInfo_Adapter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Adapter proto.InternalMessageInfo
-
-func (m *Adapter) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *Adapter) GetVendor() string {
-	if m != nil {
-		return m.Vendor
-	}
-	return ""
-}
-
-func (m *Adapter) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *Adapter) GetConfig() *AdapterConfig {
-	if m != nil {
-		return m.Config
-	}
-	return nil
-}
-
-func (m *Adapter) GetAdditionalDescription() *any.Any {
-	if m != nil {
-		return m.AdditionalDescription
-	}
-	return nil
-}
-
-func (m *Adapter) GetLogicalDeviceIds() []string {
-	if m != nil {
-		return m.LogicalDeviceIds
-	}
-	return nil
-}
-
-func (m *Adapter) GetLastCommunication() *timestamp.Timestamp {
-	if m != nil {
-		return m.LastCommunication
-	}
-	return nil
-}
-
-func (m *Adapter) GetCurrentReplica() int32 {
-	if m != nil {
-		return m.CurrentReplica
-	}
-	return 0
-}
-
-func (m *Adapter) GetTotalReplicas() int32 {
-	if m != nil {
-		return m.TotalReplicas
-	}
-	return 0
-}
-
-func (m *Adapter) GetEndpoint() string {
-	if m != nil {
-		return m.Endpoint
-	}
-	return ""
-}
-
-func (m *Adapter) GetType() string {
-	if m != nil {
-		return m.Type
-	}
-	return ""
-}
-
-type Adapters struct {
-	Items                []*Adapter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Adapters) Reset()         { *m = Adapters{} }
-func (m *Adapters) String() string { return proto.CompactTextString(m) }
-func (*Adapters) ProtoMessage()    {}
-func (*Adapters) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7e998ce153307274, []int{2}
-}
-
-func (m *Adapters) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Adapters.Unmarshal(m, b)
-}
-func (m *Adapters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Adapters.Marshal(b, m, deterministic)
-}
-func (m *Adapters) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Adapters.Merge(m, src)
-}
-func (m *Adapters) XXX_Size() int {
-	return xxx_messageInfo_Adapters.Size(m)
-}
-func (m *Adapters) XXX_DiscardUnknown() {
-	xxx_messageInfo_Adapters.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Adapters proto.InternalMessageInfo
-
-func (m *Adapters) GetItems() []*Adapter {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterType((*AdapterConfig)(nil), "voltha.AdapterConfig")
-	proto.RegisterType((*Adapter)(nil), "voltha.Adapter")
-	proto.RegisterType((*Adapters)(nil), "voltha.Adapters")
-}
-
-func init() { proto.RegisterFile("voltha_protos/adapter.proto", fileDescriptor_7e998ce153307274) }
-
-var fileDescriptor_7e998ce153307274 = []byte{
-	// 455 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x52, 0xdd, 0x6e, 0xd3, 0x30,
-	0x14, 0x56, 0xda, 0xf5, 0xcf, 0x53, 0xa1, 0x33, 0x14, 0x99, 0xa2, 0x69, 0x51, 0x05, 0x28, 0x17,
-	0x2c, 0x11, 0x83, 0x07, 0xa0, 0xdd, 0x6e, 0x76, 0x6b, 0x4d, 0x5c, 0x70, 0x53, 0xb9, 0xb6, 0x97,
-	0x59, 0x4a, 0x7c, 0xa2, 0xd8, 0x8d, 0xd4, 0x87, 0xe4, 0x05, 0x78, 0x02, 0x9e, 0x80, 0x6b, 0x54,
-	0xdb, 0xa1, 0x3f, 0x48, 0xbb, 0x4a, 0xce, 0xf7, 0x73, 0x3e, 0x9f, 0x63, 0xa3, 0x77, 0x0d, 0x14,
-	0xf6, 0x89, 0xad, 0xaa, 0x1a, 0x2c, 0x98, 0x8c, 0x09, 0x56, 0x59, 0x59, 0xa7, 0xae, 0xc4, 0x7d,
-	0x4f, 0xce, 0xde, 0xe6, 0x00, 0x79, 0x21, 0x33, 0x87, 0xae, 0x37, 0x8f, 0x19, 0xd3, 0x5b, 0x2f,
-	0x99, 0x91, 0x63, 0x7f, 0x29, 0x2d, 0x0b, 0xcc, 0xd5, 0xa9, 0xc9, 0xaa, 0x52, 0x1a, 0xcb, 0xca,
-	0xca, 0x0b, 0xe6, 0x14, 0x8d, 0x17, 0x3e, 0xee, 0x16, 0xf4, 0xa3, 0xca, 0xf1, 0x02, 0x5d, 0x30,
-	0x21, 0x94, 0x55, 0xa0, 0x59, 0xb1, 0xe2, 0x0e, 0x24, 0xdf, 0xe2, 0x28, 0x39, 0xbf, 0x79, 0x9d,
-	0xfa, 0x6e, 0x69, 0xdb, 0x2d, 0x5d, 0xe8, 0x2d, 0x9d, 0xec, 0xe5, 0xbe, 0xc5, 0xfc, 0x57, 0x17,
-	0x0d, 0x42, 0x53, 0x3c, 0x45, 0x1d, 0x25, 0x48, 0x14, 0x47, 0xc9, 0x68, 0xd9, 0xfb, 0xfd, 0xe7,
-	0xe7, 0x65, 0x44, 0x3b, 0x4a, 0xe0, 0x4b, 0xd4, 0x6f, 0xa4, 0x16, 0x50, 0x93, 0xce, 0x21, 0x15,
-	0x40, 0x7c, 0x85, 0x06, 0x8d, 0xac, 0x8d, 0x02, 0x4d, 0xba, 0x87, 0x7c, 0x8b, 0xe2, 0x6b, 0xd4,
-	0x0f, 0x47, 0x9b, 0xb8, 0xa3, 0x4d, 0x53, 0xbf, 0x82, 0xf4, 0x68, 0x18, 0x1a, 0x44, 0x98, 0xa2,
-	0x37, 0x07, 0x43, 0x09, 0x69, 0x78, 0xad, 0xaa, 0x5d, 0xf5, 0xdc, 0x64, 0x6d, 0xe8, 0x74, 0x6f,
-	0xbd, 0xdb, 0x3b, 0xf1, 0x27, 0x84, 0x0b, 0xc8, 0x15, 0x77, 0x0d, 0x1b, 0xc5, 0xe5, 0x4a, 0x09,
-	0x43, 0xce, 0xe2, 0x6e, 0x32, 0xa2, 0x93, 0xc0, 0xdc, 0x39, 0xe2, 0x5e, 0x18, 0x7c, 0x8f, 0x70,
-	0xc1, 0x8c, 0x5d, 0x71, 0x28, 0xcb, 0x8d, 0x56, 0x9c, 0xb9, 0xf4, 0x9e, 0x4b, 0x9f, 0xfd, 0x97,
-	0xfe, 0xd0, 0xde, 0x12, 0xbd, 0xd8, 0xb9, 0x6e, 0x0f, 0x4d, 0xf8, 0x23, 0x7a, 0xc1, 0x37, 0x75,
-	0x2d, 0xb5, 0xa5, 0xb2, 0x2a, 0x14, 0x67, 0xa4, 0x1f, 0x47, 0x49, 0x8f, 0x9e, 0xa0, 0xf8, 0x3d,
-	0x1a, 0x5b, 0xb0, 0xac, 0x08, 0xb5, 0x21, 0x03, 0x27, 0x3b, 0x06, 0xf1, 0x0c, 0x0d, 0xa5, 0x16,
-	0x15, 0x28, 0x6d, 0xc9, 0x70, 0xb7, 0x6b, 0xfa, 0xaf, 0xc6, 0x18, 0x9d, 0xd9, 0x6d, 0x25, 0xc9,
-	0xc8, 0xe1, 0xee, 0x7f, 0xfe, 0x19, 0x0d, 0xc3, 0x8e, 0x0d, 0xfe, 0x80, 0x7a, 0xca, 0xca, 0xd2,
-	0x90, 0x28, 0xee, 0x26, 0xe7, 0x37, 0x2f, 0x4f, 0x2e, 0x81, 0x7a, 0x76, 0xf9, 0x80, 0x5e, 0x41,
-	0x9d, 0xa7, 0x50, 0x49, 0xcd, 0xa1, 0x16, 0x41, 0xb5, 0x1c, 0x7f, 0x77, 0xdf, 0x20, 0xfe, 0x91,
-	0xe6, 0xca, 0x3e, 0x6d, 0xd6, 0x29, 0x87, 0x32, 0x6b, 0xa5, 0x99, 0x97, 0x5e, 0x87, 0x87, 0xdd,
-	0x7c, 0xcd, 0x72, 0x08, 0xd8, 0xba, 0xef, 0xc0, 0x2f, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x07,
-	0xa4, 0x0d, 0x2b, 0x3d, 0x03, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/core.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/core.pb.go
deleted file mode 100644
index 64d42d0..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/core.pb.go
+++ /dev/null
@@ -1,138 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/core.proto
-
-package voltha
-
-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
-
-// Transient State for devices
-type DeviceTransientState_Types int32
-
-const (
-	// The transient state of the device is not set
-	DeviceTransientState_NONE DeviceTransientState_Types = 0
-	// The state of the device in core is any state, i.e DELETING, DELETED, DELETE_FAILED, NONE.
-	// This state is only used for transitions.
-	DeviceTransientState_ANY DeviceTransientState_Types = 1
-	// The device is in FORCE_DELETING state
-	DeviceTransientState_FORCE_DELETING DeviceTransientState_Types = 2
-	// The device is getting deleted from adapter state
-	DeviceTransientState_DELETING_FROM_ADAPTER DeviceTransientState_Types = 3
-	// The device is deleted from adapter and is getting deleted in core.
-	DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE DeviceTransientState_Types = 4
-	// State to represent that the device deletion is failed
-	DeviceTransientState_DELETE_FAILED DeviceTransientState_Types = 5
-	// State to represent that reconcile is in progress
-	DeviceTransientState_RECONCILE_IN_PROGRESS DeviceTransientState_Types = 6
-)
-
-var DeviceTransientState_Types_name = map[int32]string{
-	0: "NONE",
-	1: "ANY",
-	2: "FORCE_DELETING",
-	3: "DELETING_FROM_ADAPTER",
-	4: "DELETING_POST_ADAPTER_RESPONSE",
-	5: "DELETE_FAILED",
-	6: "RECONCILE_IN_PROGRESS",
-}
-
-var DeviceTransientState_Types_value = map[string]int32{
-	"NONE":                           0,
-	"ANY":                            1,
-	"FORCE_DELETING":                 2,
-	"DELETING_FROM_ADAPTER":          3,
-	"DELETING_POST_ADAPTER_RESPONSE": 4,
-	"DELETE_FAILED":                  5,
-	"RECONCILE_IN_PROGRESS":          6,
-}
-
-func (x DeviceTransientState_Types) String() string {
-	return proto.EnumName(DeviceTransientState_Types_name, int32(x))
-}
-
-func (DeviceTransientState_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_39634f15fb8a505e, []int{0, 0}
-}
-
-type DeviceTransientState struct {
-	TransientState       DeviceTransientState_Types `protobuf:"varint,1,opt,name=transient_state,json=transientState,proto3,enum=voltha.DeviceTransientState_Types" json:"transient_state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *DeviceTransientState) Reset()         { *m = DeviceTransientState{} }
-func (m *DeviceTransientState) String() string { return proto.CompactTextString(m) }
-func (*DeviceTransientState) ProtoMessage()    {}
-func (*DeviceTransientState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_39634f15fb8a505e, []int{0}
-}
-
-func (m *DeviceTransientState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceTransientState.Unmarshal(m, b)
-}
-func (m *DeviceTransientState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceTransientState.Marshal(b, m, deterministic)
-}
-func (m *DeviceTransientState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceTransientState.Merge(m, src)
-}
-func (m *DeviceTransientState) XXX_Size() int {
-	return xxx_messageInfo_DeviceTransientState.Size(m)
-}
-func (m *DeviceTransientState) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceTransientState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceTransientState proto.InternalMessageInfo
-
-func (m *DeviceTransientState) GetTransientState() DeviceTransientState_Types {
-	if m != nil {
-		return m.TransientState
-	}
-	return DeviceTransientState_NONE
-}
-
-func init() {
-	proto.RegisterEnum("voltha.DeviceTransientState_Types", DeviceTransientState_Types_name, DeviceTransientState_Types_value)
-	proto.RegisterType((*DeviceTransientState)(nil), "voltha.DeviceTransientState")
-}
-
-func init() { proto.RegisterFile("voltha_protos/core.proto", fileDescriptor_39634f15fb8a505e) }
-
-var fileDescriptor_39634f15fb8a505e = []byte{
-	// 284 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0x4d, 0x4b, 0xf4, 0x30,
-	0x10, 0xc7, 0x9f, 0xee, 0xdb, 0x23, 0x01, 0xd7, 0x18, 0x15, 0xd6, 0x8b, 0x48, 0x4f, 0x5e, 0x4c,
-	0x41, 0xfd, 0x02, 0x75, 0x3b, 0xbb, 0x14, 0xd7, 0xa4, 0x24, 0xbd, 0xe8, 0x25, 0x74, 0x6b, 0xe8,
-	0x16, 0xb4, 0x29, 0x6d, 0x2c, 0x78, 0xf4, 0x73, 0xf8, 0x65, 0x65, 0xfb, 0x22, 0x08, 0xde, 0x66,
-	0x7e, 0xbf, 0xf9, 0x0f, 0xcc, 0xa0, 0x45, 0x63, 0x5e, 0xed, 0x2e, 0x51, 0x65, 0x65, 0xac, 0xa9,
-	0xbd, 0xd4, 0x54, 0x9a, 0xb6, 0x35, 0x99, 0x75, 0xc6, 0xfd, 0x1c, 0xa1, 0xd3, 0x40, 0x37, 0x79,
-	0xaa, 0xe3, 0x2a, 0x29, 0xea, 0x5c, 0x17, 0x56, 0xda, 0xc4, 0x6a, 0xf2, 0x80, 0x8e, 0xec, 0x40,
-	0x54, 0xbd, 0x47, 0x0b, 0xe7, 0xd2, 0xb9, 0x9a, 0xdf, 0xb8, 0xb4, 0x8b, 0xd2, 0xbf, 0x62, 0x34,
-	0xfe, 0x28, 0x75, 0x2d, 0xe6, 0xf6, 0x17, 0x75, 0xbf, 0x1c, 0x34, 0x6d, 0x0d, 0x39, 0x40, 0x13,
-	0xc6, 0x19, 0xe0, 0x7f, 0xe4, 0x3f, 0x1a, 0xfb, 0xec, 0x09, 0x3b, 0x84, 0xa0, 0xf9, 0x8a, 0x8b,
-	0x25, 0xa8, 0x00, 0x36, 0x10, 0x87, 0x6c, 0x8d, 0x47, 0xe4, 0x1c, 0x9d, 0x0d, 0x9d, 0x5a, 0x09,
-	0xfe, 0xa8, 0xfc, 0xc0, 0x8f, 0x62, 0x10, 0x78, 0x4c, 0x5c, 0x74, 0xf1, 0xa3, 0x22, 0x2e, 0xe3,
-	0x41, 0x29, 0x01, 0x32, 0xe2, 0x4c, 0x02, 0x9e, 0x90, 0x63, 0x74, 0xd8, 0xce, 0x80, 0x5a, 0xf9,
-	0xe1, 0x06, 0x02, 0x3c, 0xdd, 0x6f, 0x14, 0xb0, 0xe4, 0x6c, 0x19, 0x6e, 0x40, 0x85, 0x4c, 0x45,
-	0x82, 0xaf, 0x05, 0x48, 0x89, 0x67, 0xf7, 0x80, 0x4e, 0x4c, 0x95, 0x51, 0x53, 0xea, 0x22, 0x35,
-	0xd5, 0x4b, 0x7f, 0xdf, 0x33, 0xcd, 0x72, 0xbb, 0x7b, 0xdf, 0xd2, 0xd4, 0xbc, 0x79, 0x83, 0xf3,
-	0x3a, 0x77, 0xdd, 0x3f, 0xb4, 0xb9, 0xf3, 0x32, 0xd3, 0xb3, 0xed, 0xac, 0x85, 0xb7, 0xdf, 0x01,
-	0x00, 0x00, 0xff, 0xff, 0x1a, 0x0c, 0x99, 0x87, 0x75, 0x01, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/device.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/device.pb.go
deleted file mode 100644
index 957af8f..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/device.pb.go
+++ /dev/null
@@ -1,2313 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/device.proto
-
-package voltha
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	any "github.com/golang/protobuf/ptypes/any"
-	common "github.com/opencord/voltha-protos/v4/go/common"
-	openflow_13 "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	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 PmConfig_PmType int32
-
-const (
-	PmConfig_COUNTER PmConfig_PmType = 0
-	PmConfig_GAUGE   PmConfig_PmType = 1
-	PmConfig_STATE   PmConfig_PmType = 2
-	PmConfig_CONTEXT PmConfig_PmType = 3
-)
-
-var PmConfig_PmType_name = map[int32]string{
-	0: "COUNTER",
-	1: "GAUGE",
-	2: "STATE",
-	3: "CONTEXT",
-}
-
-var PmConfig_PmType_value = map[string]int32{
-	"COUNTER": 0,
-	"GAUGE":   1,
-	"STATE":   2,
-	"CONTEXT": 3,
-}
-
-func (x PmConfig_PmType) String() string {
-	return proto.EnumName(PmConfig_PmType_name, int32(x))
-}
-
-func (PmConfig_PmType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{2, 0}
-}
-
-type ImageDownload_ImageDownloadState int32
-
-const (
-	ImageDownload_DOWNLOAD_UNKNOWN     ImageDownload_ImageDownloadState = 0
-	ImageDownload_DOWNLOAD_SUCCEEDED   ImageDownload_ImageDownloadState = 1
-	ImageDownload_DOWNLOAD_REQUESTED   ImageDownload_ImageDownloadState = 2
-	ImageDownload_DOWNLOAD_STARTED     ImageDownload_ImageDownloadState = 3
-	ImageDownload_DOWNLOAD_FAILED      ImageDownload_ImageDownloadState = 4
-	ImageDownload_DOWNLOAD_UNSUPPORTED ImageDownload_ImageDownloadState = 5
-	ImageDownload_DOWNLOAD_CANCELLED   ImageDownload_ImageDownloadState = 6
-)
-
-var ImageDownload_ImageDownloadState_name = map[int32]string{
-	0: "DOWNLOAD_UNKNOWN",
-	1: "DOWNLOAD_SUCCEEDED",
-	2: "DOWNLOAD_REQUESTED",
-	3: "DOWNLOAD_STARTED",
-	4: "DOWNLOAD_FAILED",
-	5: "DOWNLOAD_UNSUPPORTED",
-	6: "DOWNLOAD_CANCELLED",
-}
-
-var ImageDownload_ImageDownloadState_value = map[string]int32{
-	"DOWNLOAD_UNKNOWN":     0,
-	"DOWNLOAD_SUCCEEDED":   1,
-	"DOWNLOAD_REQUESTED":   2,
-	"DOWNLOAD_STARTED":     3,
-	"DOWNLOAD_FAILED":      4,
-	"DOWNLOAD_UNSUPPORTED": 5,
-	"DOWNLOAD_CANCELLED":   6,
-}
-
-func (x ImageDownload_ImageDownloadState) String() string {
-	return proto.EnumName(ImageDownload_ImageDownloadState_name, int32(x))
-}
-
-func (ImageDownload_ImageDownloadState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{6, 0}
-}
-
-type ImageDownload_ImageDownloadFailureReason int32
-
-const (
-	ImageDownload_NO_ERROR           ImageDownload_ImageDownloadFailureReason = 0
-	ImageDownload_INVALID_URL        ImageDownload_ImageDownloadFailureReason = 1
-	ImageDownload_DEVICE_BUSY        ImageDownload_ImageDownloadFailureReason = 2
-	ImageDownload_INSUFFICIENT_SPACE ImageDownload_ImageDownloadFailureReason = 3
-	ImageDownload_UNKNOWN_ERROR      ImageDownload_ImageDownloadFailureReason = 4
-	ImageDownload_CANCELLED          ImageDownload_ImageDownloadFailureReason = 5
-)
-
-var ImageDownload_ImageDownloadFailureReason_name = map[int32]string{
-	0: "NO_ERROR",
-	1: "INVALID_URL",
-	2: "DEVICE_BUSY",
-	3: "INSUFFICIENT_SPACE",
-	4: "UNKNOWN_ERROR",
-	5: "CANCELLED",
-}
-
-var ImageDownload_ImageDownloadFailureReason_value = map[string]int32{
-	"NO_ERROR":           0,
-	"INVALID_URL":        1,
-	"DEVICE_BUSY":        2,
-	"INSUFFICIENT_SPACE": 3,
-	"UNKNOWN_ERROR":      4,
-	"CANCELLED":          5,
-}
-
-func (x ImageDownload_ImageDownloadFailureReason) String() string {
-	return proto.EnumName(ImageDownload_ImageDownloadFailureReason_name, int32(x))
-}
-
-func (ImageDownload_ImageDownloadFailureReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{6, 1}
-}
-
-type ImageDownload_ImageActivateState int32
-
-const (
-	ImageDownload_IMAGE_UNKNOWN    ImageDownload_ImageActivateState = 0
-	ImageDownload_IMAGE_INACTIVE   ImageDownload_ImageActivateState = 1
-	ImageDownload_IMAGE_ACTIVATING ImageDownload_ImageActivateState = 2
-	ImageDownload_IMAGE_ACTIVE     ImageDownload_ImageActivateState = 3
-	ImageDownload_IMAGE_REVERTING  ImageDownload_ImageActivateState = 4
-	ImageDownload_IMAGE_REVERTED   ImageDownload_ImageActivateState = 5
-)
-
-var ImageDownload_ImageActivateState_name = map[int32]string{
-	0: "IMAGE_UNKNOWN",
-	1: "IMAGE_INACTIVE",
-	2: "IMAGE_ACTIVATING",
-	3: "IMAGE_ACTIVE",
-	4: "IMAGE_REVERTING",
-	5: "IMAGE_REVERTED",
-}
-
-var ImageDownload_ImageActivateState_value = map[string]int32{
-	"IMAGE_UNKNOWN":    0,
-	"IMAGE_INACTIVE":   1,
-	"IMAGE_ACTIVATING": 2,
-	"IMAGE_ACTIVE":     3,
-	"IMAGE_REVERTING":  4,
-	"IMAGE_REVERTED":   5,
-}
-
-func (x ImageDownload_ImageActivateState) String() string {
-	return proto.EnumName(ImageDownload_ImageActivateState_name, int32(x))
-}
-
-func (ImageDownload_ImageActivateState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{6, 2}
-}
-
-type ImageState_ImageDownloadState int32
-
-const (
-	ImageState_DOWNLOAD_UNKNOWN     ImageState_ImageDownloadState = 0
-	ImageState_DOWNLOAD_SUCCEEDED   ImageState_ImageDownloadState = 1
-	ImageState_DOWNLOAD_REQUESTED   ImageState_ImageDownloadState = 2
-	ImageState_DOWNLOAD_STARTED     ImageState_ImageDownloadState = 3
-	ImageState_DOWNLOAD_FAILED      ImageState_ImageDownloadState = 4
-	ImageState_DOWNLOAD_UNSUPPORTED ImageState_ImageDownloadState = 5
-	ImageState_DOWNLOAD_CANCELLING  ImageState_ImageDownloadState = 6
-	ImageState_DOWNLOAD_CANCELLED   ImageState_ImageDownloadState = 7
-)
-
-var ImageState_ImageDownloadState_name = map[int32]string{
-	0: "DOWNLOAD_UNKNOWN",
-	1: "DOWNLOAD_SUCCEEDED",
-	2: "DOWNLOAD_REQUESTED",
-	3: "DOWNLOAD_STARTED",
-	4: "DOWNLOAD_FAILED",
-	5: "DOWNLOAD_UNSUPPORTED",
-	6: "DOWNLOAD_CANCELLING",
-	7: "DOWNLOAD_CANCELLED",
-}
-
-var ImageState_ImageDownloadState_value = map[string]int32{
-	"DOWNLOAD_UNKNOWN":     0,
-	"DOWNLOAD_SUCCEEDED":   1,
-	"DOWNLOAD_REQUESTED":   2,
-	"DOWNLOAD_STARTED":     3,
-	"DOWNLOAD_FAILED":      4,
-	"DOWNLOAD_UNSUPPORTED": 5,
-	"DOWNLOAD_CANCELLING":  6,
-	"DOWNLOAD_CANCELLED":   7,
-}
-
-func (x ImageState_ImageDownloadState) String() string {
-	return proto.EnumName(ImageState_ImageDownloadState_name, int32(x))
-}
-
-func (ImageState_ImageDownloadState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{12, 0}
-}
-
-type ImageState_ImageFailureReason int32
-
-const (
-	ImageState_NO_ERROR               ImageState_ImageFailureReason = 0
-	ImageState_INVALID_URL            ImageState_ImageFailureReason = 1
-	ImageState_DEVICE_BUSY            ImageState_ImageFailureReason = 2
-	ImageState_INSUFFICIENT_SPACE     ImageState_ImageFailureReason = 3
-	ImageState_UNKNOWN_ERROR          ImageState_ImageFailureReason = 4
-	ImageState_CANCELLED_ON_REQUEST   ImageState_ImageFailureReason = 5
-	ImageState_CANCELLED_ON_ONU_STATE ImageState_ImageFailureReason = 6
-	ImageState_VENDOR_DEVICE_MISMATCH ImageState_ImageFailureReason = 7
-	ImageState_OMCI_TRANSFER_ERROR    ImageState_ImageFailureReason = 8
-	ImageState_IMAGE_REFUSED_BY_ONU   ImageState_ImageFailureReason = 9
-)
-
-var ImageState_ImageFailureReason_name = map[int32]string{
-	0: "NO_ERROR",
-	1: "INVALID_URL",
-	2: "DEVICE_BUSY",
-	3: "INSUFFICIENT_SPACE",
-	4: "UNKNOWN_ERROR",
-	5: "CANCELLED_ON_REQUEST",
-	6: "CANCELLED_ON_ONU_STATE",
-	7: "VENDOR_DEVICE_MISMATCH",
-	8: "OMCI_TRANSFER_ERROR",
-	9: "IMAGE_REFUSED_BY_ONU",
-}
-
-var ImageState_ImageFailureReason_value = map[string]int32{
-	"NO_ERROR":               0,
-	"INVALID_URL":            1,
-	"DEVICE_BUSY":            2,
-	"INSUFFICIENT_SPACE":     3,
-	"UNKNOWN_ERROR":          4,
-	"CANCELLED_ON_REQUEST":   5,
-	"CANCELLED_ON_ONU_STATE": 6,
-	"VENDOR_DEVICE_MISMATCH": 7,
-	"OMCI_TRANSFER_ERROR":    8,
-	"IMAGE_REFUSED_BY_ONU":   9,
-}
-
-func (x ImageState_ImageFailureReason) String() string {
-	return proto.EnumName(ImageState_ImageFailureReason_name, int32(x))
-}
-
-func (ImageState_ImageFailureReason) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{12, 1}
-}
-
-type ImageState_ImageActivationState int32
-
-const (
-	ImageState_IMAGE_UNKNOWN             ImageState_ImageActivationState = 0
-	ImageState_IMAGE_INACTIVE            ImageState_ImageActivationState = 1
-	ImageState_IMAGE_ACTIVATING          ImageState_ImageActivationState = 2
-	ImageState_IMAGE_ACTIVE              ImageState_ImageActivationState = 3
-	ImageState_IMAGE_COMMITTING          ImageState_ImageActivationState = 4
-	ImageState_IMAGE_COMMITTED           ImageState_ImageActivationState = 5
-	ImageState_IMAGE_ACTIVATION_ABORTING ImageState_ImageActivationState = 6
-	ImageState_IMAGE_ACTIVATION_ABORTED  ImageState_ImageActivationState = 7
-	ImageState_IMAGE_COMMIT_ABORTING     ImageState_ImageActivationState = 8
-	ImageState_IMAGE_COMMIT_ABORTED      ImageState_ImageActivationState = 9
-	ImageState_IMAGE_DOWNLOADING         ImageState_ImageActivationState = 10
-)
-
-var ImageState_ImageActivationState_name = map[int32]string{
-	0:  "IMAGE_UNKNOWN",
-	1:  "IMAGE_INACTIVE",
-	2:  "IMAGE_ACTIVATING",
-	3:  "IMAGE_ACTIVE",
-	4:  "IMAGE_COMMITTING",
-	5:  "IMAGE_COMMITTED",
-	6:  "IMAGE_ACTIVATION_ABORTING",
-	7:  "IMAGE_ACTIVATION_ABORTED",
-	8:  "IMAGE_COMMIT_ABORTING",
-	9:  "IMAGE_COMMIT_ABORTED",
-	10: "IMAGE_DOWNLOADING",
-}
-
-var ImageState_ImageActivationState_value = map[string]int32{
-	"IMAGE_UNKNOWN":             0,
-	"IMAGE_INACTIVE":            1,
-	"IMAGE_ACTIVATING":          2,
-	"IMAGE_ACTIVE":              3,
-	"IMAGE_COMMITTING":          4,
-	"IMAGE_COMMITTED":           5,
-	"IMAGE_ACTIVATION_ABORTING": 6,
-	"IMAGE_ACTIVATION_ABORTED":  7,
-	"IMAGE_COMMIT_ABORTING":     8,
-	"IMAGE_COMMIT_ABORTED":      9,
-	"IMAGE_DOWNLOADING":         10,
-}
-
-func (x ImageState_ImageActivationState) String() string {
-	return proto.EnumName(ImageState_ImageActivationState_name, int32(x))
-}
-
-func (ImageState_ImageActivationState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{12, 2}
-}
-
-type Port_PortType int32
-
-const (
-	Port_UNKNOWN      Port_PortType = 0
-	Port_ETHERNET_NNI Port_PortType = 1
-	Port_ETHERNET_UNI Port_PortType = 2
-	Port_PON_OLT      Port_PortType = 3
-	Port_PON_ONU      Port_PortType = 4
-	Port_VENET_OLT    Port_PortType = 5
-	Port_VENET_ONU    Port_PortType = 6
-)
-
-var Port_PortType_name = map[int32]string{
-	0: "UNKNOWN",
-	1: "ETHERNET_NNI",
-	2: "ETHERNET_UNI",
-	3: "PON_OLT",
-	4: "PON_ONU",
-	5: "VENET_OLT",
-	6: "VENET_ONU",
-}
-
-var Port_PortType_value = map[string]int32{
-	"UNKNOWN":      0,
-	"ETHERNET_NNI": 1,
-	"ETHERNET_UNI": 2,
-	"PON_OLT":      3,
-	"PON_ONU":      4,
-	"VENET_OLT":    5,
-	"VENET_ONU":    6,
-}
-
-func (x Port_PortType) String() string {
-	return proto.EnumName(Port_PortType_name, int32(x))
-}
-
-func (Port_PortType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{13, 0}
-}
-
-type SimulateAlarmRequest_OperationType int32
-
-const (
-	SimulateAlarmRequest_RAISE SimulateAlarmRequest_OperationType = 0
-	SimulateAlarmRequest_CLEAR SimulateAlarmRequest_OperationType = 1
-)
-
-var SimulateAlarmRequest_OperationType_name = map[int32]string{
-	0: "RAISE",
-	1: "CLEAR",
-}
-
-var SimulateAlarmRequest_OperationType_value = map[string]int32{
-	"RAISE": 0,
-	"CLEAR": 1,
-}
-
-func (x SimulateAlarmRequest_OperationType) String() string {
-	return proto.EnumName(SimulateAlarmRequest_OperationType_name, int32(x))
-}
-
-func (SimulateAlarmRequest_OperationType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{17, 0}
-}
-
-// A Device Type
-type DeviceType struct {
-	// Unique name for the device type
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Unique vendor id for the device type applicable to ONU
-	// 4 bytes of vendor id from ONU serial number
-	VendorId  string   `protobuf:"bytes,5,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
-	VendorIds []string `protobuf:"bytes,6,rep,name=vendor_ids,json=vendorIds,proto3" json:"vendor_ids,omitempty"`
-	// Name of the adapter that handles device type
-	Adapter string `protobuf:"bytes,2,opt,name=adapter,proto3" json:"adapter,omitempty"`
-	// Capabilities
-	AcceptsBulkFlowUpdate           bool     `protobuf:"varint,3,opt,name=accepts_bulk_flow_update,json=acceptsBulkFlowUpdate,proto3" json:"accepts_bulk_flow_update,omitempty"`
-	AcceptsAddRemoveFlowUpdates     bool     `protobuf:"varint,4,opt,name=accepts_add_remove_flow_updates,json=acceptsAddRemoveFlowUpdates,proto3" json:"accepts_add_remove_flow_updates,omitempty"`
-	AcceptsDirectLogicalFlowsUpdate bool     `protobuf:"varint,7,opt,name=accepts_direct_logical_flows_update,json=acceptsDirectLogicalFlowsUpdate,proto3" json:"accepts_direct_logical_flows_update,omitempty"`
-	XXX_NoUnkeyedLiteral            struct{} `json:"-"`
-	XXX_unrecognized                []byte   `json:"-"`
-	XXX_sizecache                   int32    `json:"-"`
-}
-
-func (m *DeviceType) Reset()         { *m = DeviceType{} }
-func (m *DeviceType) String() string { return proto.CompactTextString(m) }
-func (*DeviceType) ProtoMessage()    {}
-func (*DeviceType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{0}
-}
-
-func (m *DeviceType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceType.Unmarshal(m, b)
-}
-func (m *DeviceType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceType.Marshal(b, m, deterministic)
-}
-func (m *DeviceType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceType.Merge(m, src)
-}
-func (m *DeviceType) XXX_Size() int {
-	return xxx_messageInfo_DeviceType.Size(m)
-}
-func (m *DeviceType) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceType proto.InternalMessageInfo
-
-func (m *DeviceType) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *DeviceType) GetVendorId() string {
-	if m != nil {
-		return m.VendorId
-	}
-	return ""
-}
-
-func (m *DeviceType) GetVendorIds() []string {
-	if m != nil {
-		return m.VendorIds
-	}
-	return nil
-}
-
-func (m *DeviceType) GetAdapter() string {
-	if m != nil {
-		return m.Adapter
-	}
-	return ""
-}
-
-func (m *DeviceType) GetAcceptsBulkFlowUpdate() bool {
-	if m != nil {
-		return m.AcceptsBulkFlowUpdate
-	}
-	return false
-}
-
-func (m *DeviceType) GetAcceptsAddRemoveFlowUpdates() bool {
-	if m != nil {
-		return m.AcceptsAddRemoveFlowUpdates
-	}
-	return false
-}
-
-func (m *DeviceType) GetAcceptsDirectLogicalFlowsUpdate() bool {
-	if m != nil {
-		return m.AcceptsDirectLogicalFlowsUpdate
-	}
-	return false
-}
-
-// A plurality of device types
-type DeviceTypes struct {
-	Items                []*DeviceType `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *DeviceTypes) Reset()         { *m = DeviceTypes{} }
-func (m *DeviceTypes) String() string { return proto.CompactTextString(m) }
-func (*DeviceTypes) ProtoMessage()    {}
-func (*DeviceTypes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{1}
-}
-
-func (m *DeviceTypes) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceTypes.Unmarshal(m, b)
-}
-func (m *DeviceTypes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceTypes.Marshal(b, m, deterministic)
-}
-func (m *DeviceTypes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceTypes.Merge(m, src)
-}
-func (m *DeviceTypes) XXX_Size() int {
-	return xxx_messageInfo_DeviceTypes.Size(m)
-}
-func (m *DeviceTypes) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceTypes.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceTypes proto.InternalMessageInfo
-
-func (m *DeviceTypes) GetItems() []*DeviceType {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type PmConfig struct {
-	Name                 string          `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Type                 PmConfig_PmType `protobuf:"varint,2,opt,name=type,proto3,enum=voltha.PmConfig_PmType" json:"type,omitempty"`
-	Enabled              bool            `protobuf:"varint,3,opt,name=enabled,proto3" json:"enabled,omitempty"`
-	SampleFreq           uint32          `protobuf:"varint,4,opt,name=sample_freq,json=sampleFreq,proto3" json:"sample_freq,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *PmConfig) Reset()         { *m = PmConfig{} }
-func (m *PmConfig) String() string { return proto.CompactTextString(m) }
-func (*PmConfig) ProtoMessage()    {}
-func (*PmConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{2}
-}
-
-func (m *PmConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PmConfig.Unmarshal(m, b)
-}
-func (m *PmConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PmConfig.Marshal(b, m, deterministic)
-}
-func (m *PmConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PmConfig.Merge(m, src)
-}
-func (m *PmConfig) XXX_Size() int {
-	return xxx_messageInfo_PmConfig.Size(m)
-}
-func (m *PmConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_PmConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PmConfig proto.InternalMessageInfo
-
-func (m *PmConfig) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *PmConfig) GetType() PmConfig_PmType {
-	if m != nil {
-		return m.Type
-	}
-	return PmConfig_COUNTER
-}
-
-func (m *PmConfig) GetEnabled() bool {
-	if m != nil {
-		return m.Enabled
-	}
-	return false
-}
-
-func (m *PmConfig) GetSampleFreq() uint32 {
-	if m != nil {
-		return m.SampleFreq
-	}
-	return 0
-}
-
-type PmGroupConfig struct {
-	GroupName            string      `protobuf:"bytes,1,opt,name=group_name,json=groupName,proto3" json:"group_name,omitempty"`
-	GroupFreq            uint32      `protobuf:"varint,2,opt,name=group_freq,json=groupFreq,proto3" json:"group_freq,omitempty"`
-	Enabled              bool        `protobuf:"varint,3,opt,name=enabled,proto3" json:"enabled,omitempty"`
-	Metrics              []*PmConfig `protobuf:"bytes,4,rep,name=metrics,proto3" json:"metrics,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *PmGroupConfig) Reset()         { *m = PmGroupConfig{} }
-func (m *PmGroupConfig) String() string { return proto.CompactTextString(m) }
-func (*PmGroupConfig) ProtoMessage()    {}
-func (*PmGroupConfig) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{3}
-}
-
-func (m *PmGroupConfig) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PmGroupConfig.Unmarshal(m, b)
-}
-func (m *PmGroupConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PmGroupConfig.Marshal(b, m, deterministic)
-}
-func (m *PmGroupConfig) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PmGroupConfig.Merge(m, src)
-}
-func (m *PmGroupConfig) XXX_Size() int {
-	return xxx_messageInfo_PmGroupConfig.Size(m)
-}
-func (m *PmGroupConfig) XXX_DiscardUnknown() {
-	xxx_messageInfo_PmGroupConfig.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PmGroupConfig proto.InternalMessageInfo
-
-func (m *PmGroupConfig) GetGroupName() string {
-	if m != nil {
-		return m.GroupName
-	}
-	return ""
-}
-
-func (m *PmGroupConfig) GetGroupFreq() uint32 {
-	if m != nil {
-		return m.GroupFreq
-	}
-	return 0
-}
-
-func (m *PmGroupConfig) GetEnabled() bool {
-	if m != nil {
-		return m.Enabled
-	}
-	return false
-}
-
-func (m *PmGroupConfig) GetMetrics() []*PmConfig {
-	if m != nil {
-		return m.Metrics
-	}
-	return nil
-}
-
-type PmConfigs struct {
-	Id          string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	DefaultFreq uint32 `protobuf:"varint,2,opt,name=default_freq,json=defaultFreq,proto3" json:"default_freq,omitempty"`
-	// Forces group names and group semantics
-	Grouped bool `protobuf:"varint,3,opt,name=grouped,proto3" json:"grouped,omitempty"`
-	// Allows Pm to set an individual sample frequency
-	FreqOverride         bool             `protobuf:"varint,4,opt,name=freq_override,json=freqOverride,proto3" json:"freq_override,omitempty"`
-	Groups               []*PmGroupConfig `protobuf:"bytes,5,rep,name=groups,proto3" json:"groups,omitempty"`
-	Metrics              []*PmConfig      `protobuf:"bytes,6,rep,name=metrics,proto3" json:"metrics,omitempty"`
-	MaxSkew              uint32           `protobuf:"varint,7,opt,name=max_skew,json=maxSkew,proto3" json:"max_skew,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *PmConfigs) Reset()         { *m = PmConfigs{} }
-func (m *PmConfigs) String() string { return proto.CompactTextString(m) }
-func (*PmConfigs) ProtoMessage()    {}
-func (*PmConfigs) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{4}
-}
-
-func (m *PmConfigs) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PmConfigs.Unmarshal(m, b)
-}
-func (m *PmConfigs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PmConfigs.Marshal(b, m, deterministic)
-}
-func (m *PmConfigs) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PmConfigs.Merge(m, src)
-}
-func (m *PmConfigs) XXX_Size() int {
-	return xxx_messageInfo_PmConfigs.Size(m)
-}
-func (m *PmConfigs) XXX_DiscardUnknown() {
-	xxx_messageInfo_PmConfigs.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PmConfigs proto.InternalMessageInfo
-
-func (m *PmConfigs) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *PmConfigs) GetDefaultFreq() uint32 {
-	if m != nil {
-		return m.DefaultFreq
-	}
-	return 0
-}
-
-func (m *PmConfigs) GetGrouped() bool {
-	if m != nil {
-		return m.Grouped
-	}
-	return false
-}
-
-func (m *PmConfigs) GetFreqOverride() bool {
-	if m != nil {
-		return m.FreqOverride
-	}
-	return false
-}
-
-func (m *PmConfigs) GetGroups() []*PmGroupConfig {
-	if m != nil {
-		return m.Groups
-	}
-	return nil
-}
-
-func (m *PmConfigs) GetMetrics() []*PmConfig {
-	if m != nil {
-		return m.Metrics
-	}
-	return nil
-}
-
-func (m *PmConfigs) GetMaxSkew() uint32 {
-	if m != nil {
-		return m.MaxSkew
-	}
-	return 0
-}
-
-//Object representing an image
-type Image struct {
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	// Version, this is the sole identifier of the image. it's the vendor specified OMCI version
-	// must be known at the time of initiating a download, activate, commit image on an onu.
-	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
-	// hash of the image to be verified against
-	// Deprecated in voltha 2.8, will be removed
-	Hash uint32 `protobuf:"varint,3,opt,name=hash,proto3" json:"hash,omitempty"`
-	// Deprecated in voltha 2.8, will be removed
-	InstallDatetime string `protobuf:"bytes,4,opt,name=install_datetime,json=installDatetime,proto3" json:"install_datetime,omitempty"`
-	// The active software image is one that is currently loaded and executing
-	// in the ONU or circuit pack. Under normal operation, one software image
-	// is always active while the other is inactive. Under no circumstances are
-	// both software images allowed to be active at the same time
-	// Deprecated in voltha 2.8, will be removed
-	IsActive bool `protobuf:"varint,5,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"`
-	// The committed software image is loaded and executed upon reboot of the
-	// ONU and/or circuit pack. During normal operation, one software image is
-	// always committed, while the other is uncommitted.
-	// Deprecated in voltha 2.8, will be removed
-	IsCommitted bool `protobuf:"varint,6,opt,name=is_committed,json=isCommitted,proto3" json:"is_committed,omitempty"`
-	// A software image is valid if it has been verified to be an executable
-	// code image. The verification mechanism is not subject to standardization;
-	// however, it should include at least a data integrity (e.g., CRC) check of
-	// the entire code image.
-	// Deprecated in voltha 2.8, will be removed
-	IsValid bool `protobuf:"varint,7,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"`
-	// URL where the image is available
-	// URL MUST be fully qualified,
-	// including filename, username and password
-	Url string `protobuf:"bytes,8,opt,name=url,proto3" json:"url,omitempty"`
-	// Represents the vendor/device mfr
-	// Needs to match the vendor of the device the image will be installed on
-	// optional, if not matched no check will be performed
-	Vendor string `protobuf:"bytes,9,opt,name=vendor,proto3" json:"vendor,omitempty"`
-	// Represents the ONU Image CRC value.
-	// Default to value 0 if not specified.
-	// If different then 0 it's used to verify the image retrieved from outside before sending it to the ONU.
-	// Calculation of this needs to be done according to ITU-T I.363.5 as per OMCI spec (section A.2.27)
-	Crc32                uint32   `protobuf:"varint,10,opt,name=crc32,proto3" json:"crc32,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Image) Reset()         { *m = Image{} }
-func (m *Image) String() string { return proto.CompactTextString(m) }
-func (*Image) ProtoMessage()    {}
-func (*Image) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{5}
-}
-
-func (m *Image) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Image.Unmarshal(m, b)
-}
-func (m *Image) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Image.Marshal(b, m, deterministic)
-}
-func (m *Image) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Image.Merge(m, src)
-}
-func (m *Image) XXX_Size() int {
-	return xxx_messageInfo_Image.Size(m)
-}
-func (m *Image) XXX_DiscardUnknown() {
-	xxx_messageInfo_Image.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Image proto.InternalMessageInfo
-
-func (m *Image) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *Image) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *Image) GetHash() uint32 {
-	if m != nil {
-		return m.Hash
-	}
-	return 0
-}
-
-func (m *Image) GetInstallDatetime() string {
-	if m != nil {
-		return m.InstallDatetime
-	}
-	return ""
-}
-
-func (m *Image) GetIsActive() bool {
-	if m != nil {
-		return m.IsActive
-	}
-	return false
-}
-
-func (m *Image) GetIsCommitted() bool {
-	if m != nil {
-		return m.IsCommitted
-	}
-	return false
-}
-
-func (m *Image) GetIsValid() bool {
-	if m != nil {
-		return m.IsValid
-	}
-	return false
-}
-
-func (m *Image) GetUrl() string {
-	if m != nil {
-		return m.Url
-	}
-	return ""
-}
-
-func (m *Image) GetVendor() string {
-	if m != nil {
-		return m.Vendor
-	}
-	return ""
-}
-
-func (m *Image) GetCrc32() uint32 {
-	if m != nil {
-		return m.Crc32
-	}
-	return 0
-}
-
-// Older version of the API please see DeviceImageDownloadRequest
-// Deprecated in voltha 2.8, will be removed
-type ImageDownload struct {
-	// Device Identifier
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Image unique identifier
-	Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	// URL where the image is available
-	// should include username password
-	Url string `protobuf:"bytes,3,opt,name=url,proto3" json:"url,omitempty"`
-	// CRC of the image to be verified aginst
-	Crc uint32 `protobuf:"varint,4,opt,name=crc,proto3" json:"crc,omitempty"`
-	// Download state
-	DownloadState ImageDownload_ImageDownloadState `protobuf:"varint,5,opt,name=download_state,json=downloadState,proto3,enum=voltha.ImageDownload_ImageDownloadState" json:"download_state,omitempty"`
-	// Downloaded version
-	ImageVersion string `protobuf:"bytes,6,opt,name=image_version,json=imageVersion,proto3" json:"image_version,omitempty"`
-	// Bytes downloaded
-	DownloadedBytes uint32 `protobuf:"varint,7,opt,name=downloaded_bytes,json=downloadedBytes,proto3" json:"downloaded_bytes,omitempty"`
-	// Download failure reason
-	Reason ImageDownload_ImageDownloadFailureReason `protobuf:"varint,8,opt,name=reason,proto3,enum=voltha.ImageDownload_ImageDownloadFailureReason" json:"reason,omitempty"`
-	// Additional info
-	AdditionalInfo string `protobuf:"bytes,9,opt,name=additional_info,json=additionalInfo,proto3" json:"additional_info,omitempty"`
-	// Save current configuration
-	SaveConfig bool `protobuf:"varint,10,opt,name=save_config,json=saveConfig,proto3" json:"save_config,omitempty"`
-	// Image local location
-	LocalDir string `protobuf:"bytes,11,opt,name=local_dir,json=localDir,proto3" json:"local_dir,omitempty"`
-	// Image activation state
-	ImageState ImageDownload_ImageActivateState `protobuf:"varint,12,opt,name=image_state,json=imageState,proto3,enum=voltha.ImageDownload_ImageActivateState" json:"image_state,omitempty"`
-	// Image file size
-	FileSize             uint32   `protobuf:"varint,13,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ImageDownload) Reset()         { *m = ImageDownload{} }
-func (m *ImageDownload) String() string { return proto.CompactTextString(m) }
-func (*ImageDownload) ProtoMessage()    {}
-func (*ImageDownload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{6}
-}
-
-func (m *ImageDownload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ImageDownload.Unmarshal(m, b)
-}
-func (m *ImageDownload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ImageDownload.Marshal(b, m, deterministic)
-}
-func (m *ImageDownload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ImageDownload.Merge(m, src)
-}
-func (m *ImageDownload) XXX_Size() int {
-	return xxx_messageInfo_ImageDownload.Size(m)
-}
-func (m *ImageDownload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ImageDownload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ImageDownload proto.InternalMessageInfo
-
-func (m *ImageDownload) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetUrl() string {
-	if m != nil {
-		return m.Url
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetCrc() uint32 {
-	if m != nil {
-		return m.Crc
-	}
-	return 0
-}
-
-func (m *ImageDownload) GetDownloadState() ImageDownload_ImageDownloadState {
-	if m != nil {
-		return m.DownloadState
-	}
-	return ImageDownload_DOWNLOAD_UNKNOWN
-}
-
-func (m *ImageDownload) GetImageVersion() string {
-	if m != nil {
-		return m.ImageVersion
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetDownloadedBytes() uint32 {
-	if m != nil {
-		return m.DownloadedBytes
-	}
-	return 0
-}
-
-func (m *ImageDownload) GetReason() ImageDownload_ImageDownloadFailureReason {
-	if m != nil {
-		return m.Reason
-	}
-	return ImageDownload_NO_ERROR
-}
-
-func (m *ImageDownload) GetAdditionalInfo() string {
-	if m != nil {
-		return m.AdditionalInfo
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetSaveConfig() bool {
-	if m != nil {
-		return m.SaveConfig
-	}
-	return false
-}
-
-func (m *ImageDownload) GetLocalDir() string {
-	if m != nil {
-		return m.LocalDir
-	}
-	return ""
-}
-
-func (m *ImageDownload) GetImageState() ImageDownload_ImageActivateState {
-	if m != nil {
-		return m.ImageState
-	}
-	return ImageDownload_IMAGE_UNKNOWN
-}
-
-func (m *ImageDownload) GetFileSize() uint32 {
-	if m != nil {
-		return m.FileSize
-	}
-	return 0
-}
-
-// Deprecated in voltha 2.8, will be removed
-type ImageDownloads struct {
-	Items                []*ImageDownload `protobuf:"bytes,2,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *ImageDownloads) Reset()         { *m = ImageDownloads{} }
-func (m *ImageDownloads) String() string { return proto.CompactTextString(m) }
-func (*ImageDownloads) ProtoMessage()    {}
-func (*ImageDownloads) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{7}
-}
-
-func (m *ImageDownloads) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ImageDownloads.Unmarshal(m, b)
-}
-func (m *ImageDownloads) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ImageDownloads.Marshal(b, m, deterministic)
-}
-func (m *ImageDownloads) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ImageDownloads.Merge(m, src)
-}
-func (m *ImageDownloads) XXX_Size() int {
-	return xxx_messageInfo_ImageDownloads.Size(m)
-}
-func (m *ImageDownloads) XXX_DiscardUnknown() {
-	xxx_messageInfo_ImageDownloads.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ImageDownloads proto.InternalMessageInfo
-
-func (m *ImageDownloads) GetItems() []*ImageDownload {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type Images struct {
-	Image                []*Image `protobuf:"bytes,1,rep,name=image,proto3" json:"image,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Images) Reset()         { *m = Images{} }
-func (m *Images) String() string { return proto.CompactTextString(m) }
-func (*Images) ProtoMessage()    {}
-func (*Images) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{8}
-}
-
-func (m *Images) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Images.Unmarshal(m, b)
-}
-func (m *Images) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Images.Marshal(b, m, deterministic)
-}
-func (m *Images) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Images.Merge(m, src)
-}
-func (m *Images) XXX_Size() int {
-	return xxx_messageInfo_Images.Size(m)
-}
-func (m *Images) XXX_DiscardUnknown() {
-	xxx_messageInfo_Images.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Images proto.InternalMessageInfo
-
-func (m *Images) GetImage() []*Image {
-	if m != nil {
-		return m.Image
-	}
-	return nil
-}
-
-// OnuImage represents the OMCI information as per OMCI spec
-// the information will be populates exactly as extracted from the device.
-type OnuImage struct {
-	//image version
-	Version     string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
-	IsCommited  bool   `protobuf:"varint,2,opt,name=isCommited,proto3" json:"isCommited,omitempty"`
-	IsActive    bool   `protobuf:"varint,3,opt,name=isActive,proto3" json:"isActive,omitempty"`
-	IsValid     bool   `protobuf:"varint,4,opt,name=isValid,proto3" json:"isValid,omitempty"`
-	ProductCode string `protobuf:"bytes,5,opt,name=productCode,proto3" json:"productCode,omitempty"`
-	// Hash is computed by the ONU and is optional as per OMCI spec (paragraph 9.1.4)
-	// No assumption should be made on the existence of this attribute at any time.
-	Hash                 string   `protobuf:"bytes,6,opt,name=hash,proto3" json:"hash,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OnuImage) Reset()         { *m = OnuImage{} }
-func (m *OnuImage) String() string { return proto.CompactTextString(m) }
-func (*OnuImage) ProtoMessage()    {}
-func (*OnuImage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{9}
-}
-
-func (m *OnuImage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuImage.Unmarshal(m, b)
-}
-func (m *OnuImage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuImage.Marshal(b, m, deterministic)
-}
-func (m *OnuImage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuImage.Merge(m, src)
-}
-func (m *OnuImage) XXX_Size() int {
-	return xxx_messageInfo_OnuImage.Size(m)
-}
-func (m *OnuImage) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuImage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuImage proto.InternalMessageInfo
-
-func (m *OnuImage) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *OnuImage) GetIsCommited() bool {
-	if m != nil {
-		return m.IsCommited
-	}
-	return false
-}
-
-func (m *OnuImage) GetIsActive() bool {
-	if m != nil {
-		return m.IsActive
-	}
-	return false
-}
-
-func (m *OnuImage) GetIsValid() bool {
-	if m != nil {
-		return m.IsValid
-	}
-	return false
-}
-
-func (m *OnuImage) GetProductCode() string {
-	if m != nil {
-		return m.ProductCode
-	}
-	return ""
-}
-
-func (m *OnuImage) GetHash() string {
-	if m != nil {
-		return m.Hash
-	}
-	return ""
-}
-
-type OnuImages struct {
-	Items                []*OnuImage `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *OnuImages) Reset()         { *m = OnuImages{} }
-func (m *OnuImages) String() string { return proto.CompactTextString(m) }
-func (*OnuImages) ProtoMessage()    {}
-func (*OnuImages) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{10}
-}
-
-func (m *OnuImages) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnuImages.Unmarshal(m, b)
-}
-func (m *OnuImages) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnuImages.Marshal(b, m, deterministic)
-}
-func (m *OnuImages) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnuImages.Merge(m, src)
-}
-func (m *OnuImages) XXX_Size() int {
-	return xxx_messageInfo_OnuImages.Size(m)
-}
-func (m *OnuImages) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnuImages.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnuImages proto.InternalMessageInfo
-
-func (m *OnuImages) GetItems() []*OnuImage {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type DeviceImageState struct {
-	DeviceId             string      `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	ImageState           *ImageState `protobuf:"bytes,2,opt,name=imageState,proto3" json:"imageState,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *DeviceImageState) Reset()         { *m = DeviceImageState{} }
-func (m *DeviceImageState) String() string { return proto.CompactTextString(m) }
-func (*DeviceImageState) ProtoMessage()    {}
-func (*DeviceImageState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{11}
-}
-
-func (m *DeviceImageState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceImageState.Unmarshal(m, b)
-}
-func (m *DeviceImageState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceImageState.Marshal(b, m, deterministic)
-}
-func (m *DeviceImageState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceImageState.Merge(m, src)
-}
-func (m *DeviceImageState) XXX_Size() int {
-	return xxx_messageInfo_DeviceImageState.Size(m)
-}
-func (m *DeviceImageState) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceImageState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceImageState proto.InternalMessageInfo
-
-func (m *DeviceImageState) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *DeviceImageState) GetImageState() *ImageState {
-	if m != nil {
-		return m.ImageState
-	}
-	return nil
-}
-
-type ImageState struct {
-	// image version
-	Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
-	// Download state
-	DownloadState ImageState_ImageDownloadState `protobuf:"varint,2,opt,name=download_state,json=downloadState,proto3,enum=voltha.ImageState_ImageDownloadState" json:"download_state,omitempty"`
-	// Image Operation Failure reason (use for both Download and Activate)
-	Reason ImageState_ImageFailureReason `protobuf:"varint,3,opt,name=reason,proto3,enum=voltha.ImageState_ImageFailureReason" json:"reason,omitempty"`
-	// Image activation state
-	ImageState           ImageState_ImageActivationState `protobuf:"varint,4,opt,name=image_state,json=imageState,proto3,enum=voltha.ImageState_ImageActivationState" json:"image_state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *ImageState) Reset()         { *m = ImageState{} }
-func (m *ImageState) String() string { return proto.CompactTextString(m) }
-func (*ImageState) ProtoMessage()    {}
-func (*ImageState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{12}
-}
-
-func (m *ImageState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ImageState.Unmarshal(m, b)
-}
-func (m *ImageState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ImageState.Marshal(b, m, deterministic)
-}
-func (m *ImageState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ImageState.Merge(m, src)
-}
-func (m *ImageState) XXX_Size() int {
-	return xxx_messageInfo_ImageState.Size(m)
-}
-func (m *ImageState) XXX_DiscardUnknown() {
-	xxx_messageInfo_ImageState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ImageState proto.InternalMessageInfo
-
-func (m *ImageState) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *ImageState) GetDownloadState() ImageState_ImageDownloadState {
-	if m != nil {
-		return m.DownloadState
-	}
-	return ImageState_DOWNLOAD_UNKNOWN
-}
-
-func (m *ImageState) GetReason() ImageState_ImageFailureReason {
-	if m != nil {
-		return m.Reason
-	}
-	return ImageState_NO_ERROR
-}
-
-func (m *ImageState) GetImageState() ImageState_ImageActivationState {
-	if m != nil {
-		return m.ImageState
-	}
-	return ImageState_IMAGE_UNKNOWN
-}
-
-type Port struct {
-	PortNo     uint32                  `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	Label      string                  `protobuf:"bytes,2,opt,name=label,proto3" json:"label,omitempty"`
-	Type       Port_PortType           `protobuf:"varint,3,opt,name=type,proto3,enum=voltha.Port_PortType" json:"type,omitempty"`
-	AdminState common.AdminState_Types `protobuf:"varint,5,opt,name=admin_state,json=adminState,proto3,enum=common.AdminState_Types" json:"admin_state,omitempty"`
-	OperStatus common.OperStatus_Types `protobuf:"varint,6,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
-	DeviceId   string                  `protobuf:"bytes,7,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	Peers      []*Port_PeerPort        `protobuf:"bytes,8,rep,name=peers,proto3" json:"peers,omitempty"`
-	RxPackets  uint64                  `protobuf:"fixed64,9,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
-	RxBytes    uint64                  `protobuf:"fixed64,10,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
-	RxErrors   uint64                  `protobuf:"fixed64,11,opt,name=rx_errors,json=rxErrors,proto3" json:"rx_errors,omitempty"`
-	TxPackets  uint64                  `protobuf:"fixed64,12,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
-	TxBytes    uint64                  `protobuf:"fixed64,13,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
-	TxErrors   uint64                  `protobuf:"fixed64,14,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
-	// ofp_port represents the characteristics of a port, e.g. hardware
-	// address and supported features.  This field is relevant only for
-	// UNI and NNI ports.   For PON ports, it can be left empty.
-	OfpPort              *openflow_13.OfpPort `protobuf:"bytes,15,opt,name=ofp_port,json=ofpPort,proto3" json:"ofp_port,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *Port) Reset()         { *m = Port{} }
-func (m *Port) String() string { return proto.CompactTextString(m) }
-func (*Port) ProtoMessage()    {}
-func (*Port) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{13}
-}
-
-func (m *Port) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Port.Unmarshal(m, b)
-}
-func (m *Port) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Port.Marshal(b, m, deterministic)
-}
-func (m *Port) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Port.Merge(m, src)
-}
-func (m *Port) XXX_Size() int {
-	return xxx_messageInfo_Port.Size(m)
-}
-func (m *Port) XXX_DiscardUnknown() {
-	xxx_messageInfo_Port.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Port proto.InternalMessageInfo
-
-func (m *Port) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-func (m *Port) GetLabel() string {
-	if m != nil {
-		return m.Label
-	}
-	return ""
-}
-
-func (m *Port) GetType() Port_PortType {
-	if m != nil {
-		return m.Type
-	}
-	return Port_UNKNOWN
-}
-
-func (m *Port) GetAdminState() common.AdminState_Types {
-	if m != nil {
-		return m.AdminState
-	}
-	return common.AdminState_UNKNOWN
-}
-
-func (m *Port) GetOperStatus() common.OperStatus_Types {
-	if m != nil {
-		return m.OperStatus
-	}
-	return common.OperStatus_UNKNOWN
-}
-
-func (m *Port) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *Port) GetPeers() []*Port_PeerPort {
-	if m != nil {
-		return m.Peers
-	}
-	return nil
-}
-
-func (m *Port) GetRxPackets() uint64 {
-	if m != nil {
-		return m.RxPackets
-	}
-	return 0
-}
-
-func (m *Port) GetRxBytes() uint64 {
-	if m != nil {
-		return m.RxBytes
-	}
-	return 0
-}
-
-func (m *Port) GetRxErrors() uint64 {
-	if m != nil {
-		return m.RxErrors
-	}
-	return 0
-}
-
-func (m *Port) GetTxPackets() uint64 {
-	if m != nil {
-		return m.TxPackets
-	}
-	return 0
-}
-
-func (m *Port) GetTxBytes() uint64 {
-	if m != nil {
-		return m.TxBytes
-	}
-	return 0
-}
-
-func (m *Port) GetTxErrors() uint64 {
-	if m != nil {
-		return m.TxErrors
-	}
-	return 0
-}
-
-func (m *Port) GetOfpPort() *openflow_13.OfpPort {
-	if m != nil {
-		return m.OfpPort
-	}
-	return nil
-}
-
-type Port_PeerPort struct {
-	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	PortNo               uint32   `protobuf:"varint,2,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Port_PeerPort) Reset()         { *m = Port_PeerPort{} }
-func (m *Port_PeerPort) String() string { return proto.CompactTextString(m) }
-func (*Port_PeerPort) ProtoMessage()    {}
-func (*Port_PeerPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{13, 0}
-}
-
-func (m *Port_PeerPort) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Port_PeerPort.Unmarshal(m, b)
-}
-func (m *Port_PeerPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Port_PeerPort.Marshal(b, m, deterministic)
-}
-func (m *Port_PeerPort) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Port_PeerPort.Merge(m, src)
-}
-func (m *Port_PeerPort) XXX_Size() int {
-	return xxx_messageInfo_Port_PeerPort.Size(m)
-}
-func (m *Port_PeerPort) XXX_DiscardUnknown() {
-	xxx_messageInfo_Port_PeerPort.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Port_PeerPort proto.InternalMessageInfo
-
-func (m *Port_PeerPort) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *Port_PeerPort) GetPortNo() uint32 {
-	if m != nil {
-		return m.PortNo
-	}
-	return 0
-}
-
-type Ports struct {
-	Items                []*Port  `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Ports) Reset()         { *m = Ports{} }
-func (m *Ports) String() string { return proto.CompactTextString(m) }
-func (*Ports) ProtoMessage()    {}
-func (*Ports) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{14}
-}
-
-func (m *Ports) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Ports.Unmarshal(m, b)
-}
-func (m *Ports) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Ports.Marshal(b, m, deterministic)
-}
-func (m *Ports) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Ports.Merge(m, src)
-}
-func (m *Ports) XXX_Size() int {
-	return xxx_messageInfo_Ports.Size(m)
-}
-func (m *Ports) XXX_DiscardUnknown() {
-	xxx_messageInfo_Ports.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Ports proto.InternalMessageInfo
-
-func (m *Ports) GetItems() []*Port {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-// A Physical Device instance
-type Device struct {
-	// Voltha's device identifier
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Device type, refers to one of the registered device types
-	Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"`
-	// Is this device a root device. Each logical switch has one root
-	// device that is associated with the logical flow switch.
-	Root bool `protobuf:"varint,3,opt,name=root,proto3" json:"root,omitempty"`
-	// Parent device id, in the device tree (for a root device, the parent_id
-	// is the logical_device.id)
-	ParentId     string `protobuf:"bytes,4,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
-	ParentPortNo uint32 `protobuf:"varint,20,opt,name=parent_port_no,json=parentPortNo,proto3" json:"parent_port_no,omitempty"`
-	// Vendor, version, serial number, etc.
-	Vendor          string `protobuf:"bytes,5,opt,name=vendor,proto3" json:"vendor,omitempty"`
-	Model           string `protobuf:"bytes,6,opt,name=model,proto3" json:"model,omitempty"`
-	HardwareVersion string `protobuf:"bytes,7,opt,name=hardware_version,json=hardwareVersion,proto3" json:"hardware_version,omitempty"`
-	FirmwareVersion string `protobuf:"bytes,8,opt,name=firmware_version,json=firmwareVersion,proto3" json:"firmware_version,omitempty"`
-	// List of software on the device
-	Images       *Images `protobuf:"bytes,9,opt,name=images,proto3" json:"images,omitempty"`
-	SerialNumber string  `protobuf:"bytes,10,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
-	VendorId     string  `protobuf:"bytes,24,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
-	// Adapter that takes care of device
-	Adapter string `protobuf:"bytes,11,opt,name=adapter,proto3" json:"adapter,omitempty"`
-	// Device contact on vlan (if 0, no vlan)
-	Vlan uint32 `protobuf:"varint,12,opt,name=vlan,proto3" json:"vlan,omitempty"`
-	// Device contact MAC address (format: "xx:xx:xx:xx:xx:xx")
-	MacAddress string `protobuf:"bytes,13,opt,name=mac_address,json=macAddress,proto3" json:"mac_address,omitempty"`
-	// Types that are valid to be assigned to Address:
-	//	*Device_Ipv4Address
-	//	*Device_Ipv6Address
-	//	*Device_HostAndPort
-	Address       isDevice_Address           `protobuf_oneof:"address"`
-	ExtraArgs     string                     `protobuf:"bytes,23,opt,name=extra_args,json=extraArgs,proto3" json:"extra_args,omitempty"`
-	ProxyAddress  *Device_ProxyAddress       `protobuf:"bytes,19,opt,name=proxy_address,json=proxyAddress,proto3" json:"proxy_address,omitempty"`
-	AdminState    common.AdminState_Types    `protobuf:"varint,16,opt,name=admin_state,json=adminState,proto3,enum=common.AdminState_Types" json:"admin_state,omitempty"`
-	OperStatus    common.OperStatus_Types    `protobuf:"varint,17,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
-	Reason        string                     `protobuf:"bytes,22,opt,name=reason,proto3" json:"reason,omitempty"`
-	ConnectStatus common.ConnectStatus_Types `protobuf:"varint,18,opt,name=connect_status,json=connectStatus,proto3,enum=common.ConnectStatus_Types" json:"connect_status,omitempty"`
-	// Device type specific attributes
-	Custom *any.Any `protobuf:"bytes,64,opt,name=custom,proto3" json:"custom,omitempty"`
-	// PmConfigs will eventually converted to a child node of the
-	// device to falicitata callbacks and to simplify manipulation.
-	PmConfigs            *PmConfigs       `protobuf:"bytes,131,opt,name=pm_configs,json=pmConfigs,proto3" json:"pm_configs,omitempty"`
-	ImageDownloads       []*ImageDownload `protobuf:"bytes,133,rep,name=image_downloads,json=imageDownloads,proto3" json:"image_downloads,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *Device) Reset()         { *m = Device{} }
-func (m *Device) String() string { return proto.CompactTextString(m) }
-func (*Device) ProtoMessage()    {}
-func (*Device) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{15}
-}
-
-func (m *Device) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Device.Unmarshal(m, b)
-}
-func (m *Device) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Device.Marshal(b, m, deterministic)
-}
-func (m *Device) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Device.Merge(m, src)
-}
-func (m *Device) XXX_Size() int {
-	return xxx_messageInfo_Device.Size(m)
-}
-func (m *Device) XXX_DiscardUnknown() {
-	xxx_messageInfo_Device.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Device proto.InternalMessageInfo
-
-func (m *Device) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *Device) GetType() string {
-	if m != nil {
-		return m.Type
-	}
-	return ""
-}
-
-func (m *Device) GetRoot() bool {
-	if m != nil {
-		return m.Root
-	}
-	return false
-}
-
-func (m *Device) GetParentId() string {
-	if m != nil {
-		return m.ParentId
-	}
-	return ""
-}
-
-func (m *Device) GetParentPortNo() uint32 {
-	if m != nil {
-		return m.ParentPortNo
-	}
-	return 0
-}
-
-func (m *Device) GetVendor() string {
-	if m != nil {
-		return m.Vendor
-	}
-	return ""
-}
-
-func (m *Device) GetModel() string {
-	if m != nil {
-		return m.Model
-	}
-	return ""
-}
-
-func (m *Device) GetHardwareVersion() string {
-	if m != nil {
-		return m.HardwareVersion
-	}
-	return ""
-}
-
-func (m *Device) GetFirmwareVersion() string {
-	if m != nil {
-		return m.FirmwareVersion
-	}
-	return ""
-}
-
-func (m *Device) GetImages() *Images {
-	if m != nil {
-		return m.Images
-	}
-	return nil
-}
-
-func (m *Device) GetSerialNumber() string {
-	if m != nil {
-		return m.SerialNumber
-	}
-	return ""
-}
-
-func (m *Device) GetVendorId() string {
-	if m != nil {
-		return m.VendorId
-	}
-	return ""
-}
-
-func (m *Device) GetAdapter() string {
-	if m != nil {
-		return m.Adapter
-	}
-	return ""
-}
-
-func (m *Device) GetVlan() uint32 {
-	if m != nil {
-		return m.Vlan
-	}
-	return 0
-}
-
-func (m *Device) GetMacAddress() string {
-	if m != nil {
-		return m.MacAddress
-	}
-	return ""
-}
-
-type isDevice_Address interface {
-	isDevice_Address()
-}
-
-type Device_Ipv4Address struct {
-	Ipv4Address string `protobuf:"bytes,14,opt,name=ipv4_address,json=ipv4Address,proto3,oneof"`
-}
-
-type Device_Ipv6Address struct {
-	Ipv6Address string `protobuf:"bytes,15,opt,name=ipv6_address,json=ipv6Address,proto3,oneof"`
-}
-
-type Device_HostAndPort struct {
-	HostAndPort string `protobuf:"bytes,21,opt,name=host_and_port,json=hostAndPort,proto3,oneof"`
-}
-
-func (*Device_Ipv4Address) isDevice_Address() {}
-
-func (*Device_Ipv6Address) isDevice_Address() {}
-
-func (*Device_HostAndPort) isDevice_Address() {}
-
-func (m *Device) GetAddress() isDevice_Address {
-	if m != nil {
-		return m.Address
-	}
-	return nil
-}
-
-func (m *Device) GetIpv4Address() string {
-	if x, ok := m.GetAddress().(*Device_Ipv4Address); ok {
-		return x.Ipv4Address
-	}
-	return ""
-}
-
-func (m *Device) GetIpv6Address() string {
-	if x, ok := m.GetAddress().(*Device_Ipv6Address); ok {
-		return x.Ipv6Address
-	}
-	return ""
-}
-
-func (m *Device) GetHostAndPort() string {
-	if x, ok := m.GetAddress().(*Device_HostAndPort); ok {
-		return x.HostAndPort
-	}
-	return ""
-}
-
-func (m *Device) GetExtraArgs() string {
-	if m != nil {
-		return m.ExtraArgs
-	}
-	return ""
-}
-
-func (m *Device) GetProxyAddress() *Device_ProxyAddress {
-	if m != nil {
-		return m.ProxyAddress
-	}
-	return nil
-}
-
-func (m *Device) GetAdminState() common.AdminState_Types {
-	if m != nil {
-		return m.AdminState
-	}
-	return common.AdminState_UNKNOWN
-}
-
-func (m *Device) GetOperStatus() common.OperStatus_Types {
-	if m != nil {
-		return m.OperStatus
-	}
-	return common.OperStatus_UNKNOWN
-}
-
-func (m *Device) GetReason() string {
-	if m != nil {
-		return m.Reason
-	}
-	return ""
-}
-
-func (m *Device) GetConnectStatus() common.ConnectStatus_Types {
-	if m != nil {
-		return m.ConnectStatus
-	}
-	return common.ConnectStatus_UNKNOWN
-}
-
-func (m *Device) GetCustom() *any.Any {
-	if m != nil {
-		return m.Custom
-	}
-	return nil
-}
-
-func (m *Device) GetPmConfigs() *PmConfigs {
-	if m != nil {
-		return m.PmConfigs
-	}
-	return nil
-}
-
-func (m *Device) GetImageDownloads() []*ImageDownload {
-	if m != nil {
-		return m.ImageDownloads
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*Device) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*Device_Ipv4Address)(nil),
-		(*Device_Ipv6Address)(nil),
-		(*Device_HostAndPort)(nil),
-	}
-}
-
-type Device_ProxyAddress struct {
-	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	DeviceType           string   `protobuf:"bytes,2,opt,name=device_type,json=deviceType,proto3" json:"device_type,omitempty"`
-	ChannelId            uint32   `protobuf:"varint,3,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"`
-	ChannelGroupId       uint32   `protobuf:"varint,4,opt,name=channel_group_id,json=channelGroupId,proto3" json:"channel_group_id,omitempty"`
-	ChannelTermination   string   `protobuf:"bytes,5,opt,name=channel_termination,json=channelTermination,proto3" json:"channel_termination,omitempty"`
-	OnuId                uint32   `protobuf:"varint,6,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
-	OnuSessionId         uint32   `protobuf:"varint,7,opt,name=onu_session_id,json=onuSessionId,proto3" json:"onu_session_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Device_ProxyAddress) Reset()         { *m = Device_ProxyAddress{} }
-func (m *Device_ProxyAddress) String() string { return proto.CompactTextString(m) }
-func (*Device_ProxyAddress) ProtoMessage()    {}
-func (*Device_ProxyAddress) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{15, 0}
-}
-
-func (m *Device_ProxyAddress) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Device_ProxyAddress.Unmarshal(m, b)
-}
-func (m *Device_ProxyAddress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Device_ProxyAddress.Marshal(b, m, deterministic)
-}
-func (m *Device_ProxyAddress) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Device_ProxyAddress.Merge(m, src)
-}
-func (m *Device_ProxyAddress) XXX_Size() int {
-	return xxx_messageInfo_Device_ProxyAddress.Size(m)
-}
-func (m *Device_ProxyAddress) XXX_DiscardUnknown() {
-	xxx_messageInfo_Device_ProxyAddress.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Device_ProxyAddress proto.InternalMessageInfo
-
-func (m *Device_ProxyAddress) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *Device_ProxyAddress) GetDeviceType() string {
-	if m != nil {
-		return m.DeviceType
-	}
-	return ""
-}
-
-func (m *Device_ProxyAddress) GetChannelId() uint32 {
-	if m != nil {
-		return m.ChannelId
-	}
-	return 0
-}
-
-func (m *Device_ProxyAddress) GetChannelGroupId() uint32 {
-	if m != nil {
-		return m.ChannelGroupId
-	}
-	return 0
-}
-
-func (m *Device_ProxyAddress) GetChannelTermination() string {
-	if m != nil {
-		return m.ChannelTermination
-	}
-	return ""
-}
-
-func (m *Device_ProxyAddress) GetOnuId() uint32 {
-	if m != nil {
-		return m.OnuId
-	}
-	return 0
-}
-
-func (m *Device_ProxyAddress) GetOnuSessionId() uint32 {
-	if m != nil {
-		return m.OnuSessionId
-	}
-	return 0
-}
-
-type Devices struct {
-	Items                []*Device `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *Devices) Reset()         { *m = Devices{} }
-func (m *Devices) String() string { return proto.CompactTextString(m) }
-func (*Devices) ProtoMessage()    {}
-func (*Devices) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{16}
-}
-
-func (m *Devices) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Devices.Unmarshal(m, b)
-}
-func (m *Devices) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Devices.Marshal(b, m, deterministic)
-}
-func (m *Devices) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Devices.Merge(m, src)
-}
-func (m *Devices) XXX_Size() int {
-	return xxx_messageInfo_Devices.Size(m)
-}
-func (m *Devices) XXX_DiscardUnknown() {
-	xxx_messageInfo_Devices.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Devices proto.InternalMessageInfo
-
-func (m *Devices) GetItems() []*Device {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type SimulateAlarmRequest struct {
-	// Device Identifier
-	Id                   string                             `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Indicator            string                             `protobuf:"bytes,2,opt,name=indicator,proto3" json:"indicator,omitempty"`
-	IntfId               string                             `protobuf:"bytes,3,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
-	PortTypeName         string                             `protobuf:"bytes,4,opt,name=port_type_name,json=portTypeName,proto3" json:"port_type_name,omitempty"`
-	OnuDeviceId          string                             `protobuf:"bytes,5,opt,name=onu_device_id,json=onuDeviceId,proto3" json:"onu_device_id,omitempty"`
-	InverseBitErrorRate  int32                              `protobuf:"varint,6,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
-	Drift                int32                              `protobuf:"varint,7,opt,name=drift,proto3" json:"drift,omitempty"`
-	NewEqd               int32                              `protobuf:"varint,8,opt,name=new_eqd,json=newEqd,proto3" json:"new_eqd,omitempty"`
-	OnuSerialNumber      string                             `protobuf:"bytes,9,opt,name=onu_serial_number,json=onuSerialNumber,proto3" json:"onu_serial_number,omitempty"`
-	Operation            SimulateAlarmRequest_OperationType `protobuf:"varint,10,opt,name=operation,proto3,enum=voltha.SimulateAlarmRequest_OperationType" json:"operation,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
-}
-
-func (m *SimulateAlarmRequest) Reset()         { *m = SimulateAlarmRequest{} }
-func (m *SimulateAlarmRequest) String() string { return proto.CompactTextString(m) }
-func (*SimulateAlarmRequest) ProtoMessage()    {}
-func (*SimulateAlarmRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_200940f73d155856, []int{17}
-}
-
-func (m *SimulateAlarmRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SimulateAlarmRequest.Unmarshal(m, b)
-}
-func (m *SimulateAlarmRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SimulateAlarmRequest.Marshal(b, m, deterministic)
-}
-func (m *SimulateAlarmRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SimulateAlarmRequest.Merge(m, src)
-}
-func (m *SimulateAlarmRequest) XXX_Size() int {
-	return xxx_messageInfo_SimulateAlarmRequest.Size(m)
-}
-func (m *SimulateAlarmRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_SimulateAlarmRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SimulateAlarmRequest proto.InternalMessageInfo
-
-func (m *SimulateAlarmRequest) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetIndicator() string {
-	if m != nil {
-		return m.Indicator
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetIntfId() string {
-	if m != nil {
-		return m.IntfId
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetPortTypeName() string {
-	if m != nil {
-		return m.PortTypeName
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetOnuDeviceId() string {
-	if m != nil {
-		return m.OnuDeviceId
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetInverseBitErrorRate() int32 {
-	if m != nil {
-		return m.InverseBitErrorRate
-	}
-	return 0
-}
-
-func (m *SimulateAlarmRequest) GetDrift() int32 {
-	if m != nil {
-		return m.Drift
-	}
-	return 0
-}
-
-func (m *SimulateAlarmRequest) GetNewEqd() int32 {
-	if m != nil {
-		return m.NewEqd
-	}
-	return 0
-}
-
-func (m *SimulateAlarmRequest) GetOnuSerialNumber() string {
-	if m != nil {
-		return m.OnuSerialNumber
-	}
-	return ""
-}
-
-func (m *SimulateAlarmRequest) GetOperation() SimulateAlarmRequest_OperationType {
-	if m != nil {
-		return m.Operation
-	}
-	return SimulateAlarmRequest_RAISE
-}
-
-func init() {
-	proto.RegisterEnum("voltha.PmConfig_PmType", PmConfig_PmType_name, PmConfig_PmType_value)
-	proto.RegisterEnum("voltha.ImageDownload_ImageDownloadState", ImageDownload_ImageDownloadState_name, ImageDownload_ImageDownloadState_value)
-	proto.RegisterEnum("voltha.ImageDownload_ImageDownloadFailureReason", ImageDownload_ImageDownloadFailureReason_name, ImageDownload_ImageDownloadFailureReason_value)
-	proto.RegisterEnum("voltha.ImageDownload_ImageActivateState", ImageDownload_ImageActivateState_name, ImageDownload_ImageActivateState_value)
-	proto.RegisterEnum("voltha.ImageState_ImageDownloadState", ImageState_ImageDownloadState_name, ImageState_ImageDownloadState_value)
-	proto.RegisterEnum("voltha.ImageState_ImageFailureReason", ImageState_ImageFailureReason_name, ImageState_ImageFailureReason_value)
-	proto.RegisterEnum("voltha.ImageState_ImageActivationState", ImageState_ImageActivationState_name, ImageState_ImageActivationState_value)
-	proto.RegisterEnum("voltha.Port_PortType", Port_PortType_name, Port_PortType_value)
-	proto.RegisterEnum("voltha.SimulateAlarmRequest_OperationType", SimulateAlarmRequest_OperationType_name, SimulateAlarmRequest_OperationType_value)
-	proto.RegisterType((*DeviceType)(nil), "voltha.DeviceType")
-	proto.RegisterType((*DeviceTypes)(nil), "voltha.DeviceTypes")
-	proto.RegisterType((*PmConfig)(nil), "voltha.PmConfig")
-	proto.RegisterType((*PmGroupConfig)(nil), "voltha.PmGroupConfig")
-	proto.RegisterType((*PmConfigs)(nil), "voltha.PmConfigs")
-	proto.RegisterType((*Image)(nil), "voltha.Image")
-	proto.RegisterType((*ImageDownload)(nil), "voltha.ImageDownload")
-	proto.RegisterType((*ImageDownloads)(nil), "voltha.ImageDownloads")
-	proto.RegisterType((*Images)(nil), "voltha.Images")
-	proto.RegisterType((*OnuImage)(nil), "voltha.OnuImage")
-	proto.RegisterType((*OnuImages)(nil), "voltha.OnuImages")
-	proto.RegisterType((*DeviceImageState)(nil), "voltha.DeviceImageState")
-	proto.RegisterType((*ImageState)(nil), "voltha.ImageState")
-	proto.RegisterType((*Port)(nil), "voltha.Port")
-	proto.RegisterType((*Port_PeerPort)(nil), "voltha.Port.PeerPort")
-	proto.RegisterType((*Ports)(nil), "voltha.Ports")
-	proto.RegisterType((*Device)(nil), "voltha.Device")
-	proto.RegisterType((*Device_ProxyAddress)(nil), "voltha.Device.ProxyAddress")
-	proto.RegisterType((*Devices)(nil), "voltha.Devices")
-	proto.RegisterType((*SimulateAlarmRequest)(nil), "voltha.SimulateAlarmRequest")
-}
-
-func init() { proto.RegisterFile("voltha_protos/device.proto", fileDescriptor_200940f73d155856) }
-
-var fileDescriptor_200940f73d155856 = []byte{
-	// 2701 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x59, 0xcb, 0x72, 0x1b, 0xc7,
-	0xd5, 0x16, 0x40, 0x62, 0x80, 0x39, 0xb8, 0x70, 0xd4, 0x22, 0xa5, 0x11, 0x69, 0xfe, 0xd2, 0x0f,
-	0xd9, 0xbf, 0x69, 0xe9, 0x17, 0xa9, 0x48, 0x2e, 0xdb, 0x59, 0xb8, 0x4a, 0x20, 0x30, 0x94, 0xa6,
-	0x42, 0x0d, 0x98, 0x06, 0x40, 0xc7, 0xd9, 0x4c, 0x0d, 0x31, 0x4d, 0x72, 0x4a, 0x83, 0x19, 0xa8,
-	0x67, 0x40, 0x52, 0xde, 0xa5, 0x5c, 0xc9, 0x2a, 0x55, 0x59, 0x64, 0x95, 0x3c, 0x41, 0x96, 0xd9,
-	0x79, 0x99, 0xbc, 0x80, 0xab, 0xf2, 0x0c, 0xc9, 0x26, 0x0f, 0x90, 0xf2, 0x3a, 0xd5, 0xb7, 0xb9,
-	0x90, 0x92, 0xec, 0x54, 0xa5, 0x52, 0xd9, 0x90, 0xd3, 0xdf, 0xb9, 0x74, 0xf7, 0xe9, 0x3e, 0x5f,
-	0x9f, 0x6e, 0xc0, 0xfa, 0x59, 0x1c, 0xa6, 0xa7, 0x9e, 0x3b, 0xa7, 0x71, 0x1a, 0x27, 0x3b, 0x3e,
-	0x39, 0x0b, 0xa6, 0x64, 0x9b, 0xb7, 0x90, 0x26, 0x64, 0xeb, 0xb7, 0x4f, 0xe2, 0xf8, 0x24, 0x24,
-	0x3b, 0x1c, 0x3d, 0x5a, 0x1c, 0xef, 0x78, 0xd1, 0x6b, 0xa1, 0xb2, 0x7e, 0xc9, 0x7c, 0x1a, 0xcf,
-	0x66, 0x71, 0x24, 0x65, 0x66, 0x59, 0x36, 0x23, 0xa9, 0x27, 0x25, 0x77, 0xca, 0x92, 0x78, 0x4e,
-	0xa2, 0xe3, 0x30, 0x3e, 0x77, 0x7f, 0xf4, 0x44, 0x28, 0x74, 0xff, 0x54, 0x05, 0x18, 0xf0, 0xa1,
-	0x8c, 0x5f, 0xcf, 0x09, 0xea, 0x40, 0x35, 0xf0, 0xcd, 0xca, 0xdd, 0xca, 0x96, 0x8e, 0xab, 0x81,
-	0x8f, 0x36, 0x40, 0x3f, 0x23, 0x91, 0x1f, 0x53, 0x37, 0xf0, 0xcd, 0x1a, 0x87, 0x1b, 0x02, 0xb0,
-	0x7d, 0xb4, 0x09, 0x90, 0x09, 0x13, 0x53, 0xbb, 0xbb, 0xb4, 0xa5, 0x63, 0x5d, 0x49, 0x13, 0x64,
-	0x42, 0xdd, 0xf3, 0xbd, 0x79, 0x4a, 0xa8, 0x59, 0xe5, 0x96, 0xaa, 0x89, 0x3e, 0x05, 0xd3, 0x9b,
-	0x4e, 0xc9, 0x3c, 0x4d, 0xdc, 0xa3, 0x45, 0xf8, 0xd2, 0xe5, 0x43, 0x5a, 0xcc, 0x7d, 0x2f, 0x25,
-	0xe6, 0xd2, 0xdd, 0xca, 0x56, 0x03, 0xaf, 0x49, 0xf9, 0xee, 0x22, 0x7c, 0xb9, 0x17, 0xc6, 0xe7,
-	0x13, 0x2e, 0x44, 0x03, 0xb8, 0xa3, 0x0c, 0x3d, 0xdf, 0x77, 0x29, 0x99, 0xc5, 0x67, 0xa4, 0x68,
-	0x9e, 0x98, 0xcb, 0xdc, 0x7e, 0x43, 0xaa, 0xf5, 0x7c, 0x1f, 0x73, 0xa5, 0xdc, 0x49, 0x82, 0xf6,
-	0xe1, 0x9e, 0xf2, 0xe2, 0x07, 0x94, 0x4c, 0x53, 0x37, 0x8c, 0x4f, 0x82, 0xa9, 0x17, 0x72, 0x4f,
-	0x89, 0x1a, 0x49, 0x9d, 0x7b, 0x52, 0x1d, 0x0e, 0xb8, 0xe6, 0xbe, 0x50, 0x64, 0xde, 0x12, 0xe1,
-	0xae, 0xfb, 0x29, 0x34, 0xf3, 0x00, 0x26, 0x68, 0x0b, 0x6a, 0x41, 0x4a, 0x66, 0x89, 0x59, 0xb9,
-	0xbb, 0xb4, 0xd5, 0x7c, 0x8c, 0xb6, 0xc5, 0x0a, 0x6c, 0xe7, 0x3a, 0x58, 0x28, 0x74, 0xff, 0x5c,
-	0x81, 0xc6, 0xc1, 0xac, 0x1f, 0x47, 0xc7, 0xc1, 0x09, 0x42, 0xb0, 0x1c, 0x79, 0x33, 0x22, 0x43,
-	0xcf, 0xbf, 0xd1, 0x03, 0x58, 0x4e, 0x5f, 0xcf, 0x09, 0x8f, 0x5e, 0xe7, 0xf1, 0x2d, 0xe5, 0x49,
-	0xd9, 0x6c, 0x1f, 0xcc, 0xb8, 0x3b, 0xae, 0xc4, 0xa2, 0x4d, 0x22, 0xef, 0x28, 0x24, 0xbe, 0x0c,
-	0xa1, 0x6a, 0xa2, 0x3b, 0xd0, 0x4c, 0xbc, 0xd9, 0x3c, 0x24, 0xee, 0x31, 0x25, 0xaf, 0x78, 0x80,
-	0xda, 0x18, 0x04, 0xb4, 0x47, 0xc9, 0xab, 0xee, 0x67, 0xa0, 0x09, 0x57, 0xa8, 0x09, 0xf5, 0xfe,
-	0x70, 0xe2, 0x8c, 0x2d, 0x6c, 0x5c, 0x43, 0x3a, 0xd4, 0x9e, 0xf5, 0x26, 0xcf, 0x2c, 0xa3, 0xc2,
-	0x3e, 0x47, 0xe3, 0xde, 0xd8, 0x32, 0xaa, 0x42, 0xc5, 0x19, 0x5b, 0x3f, 0x1b, 0x1b, 0x4b, 0xdd,
-	0xdf, 0x56, 0xa0, 0x7d, 0x30, 0x7b, 0x46, 0xe3, 0xc5, 0x5c, 0xce, 0x63, 0x13, 0xe0, 0x84, 0x35,
-	0xdd, 0xc2, 0x6c, 0x74, 0x8e, 0x38, 0x6c, 0x4a, 0x99, 0x98, 0x0f, 0xa5, 0xca, 0x87, 0x22, 0xc4,
-	0x6c, 0x24, 0xef, 0x98, 0xc4, 0x7d, 0xa8, 0xcf, 0x48, 0x4a, 0x83, 0x29, 0x5b, 0x61, 0x16, 0x58,
-	0xe3, 0x72, 0x38, 0xb0, 0x52, 0xe8, 0xfe, 0xa2, 0x0a, 0xba, 0x42, 0x93, 0x2b, 0x5b, 0xfa, 0x7f,
-	0xa1, 0xe5, 0x93, 0x63, 0x6f, 0x11, 0xa6, 0xc5, 0x41, 0x34, 0x25, 0xc6, 0x87, 0x71, 0x07, 0xea,
-	0x7c, 0x4c, 0x6a, 0x18, 0xbb, 0xb5, 0xbf, 0x7f, 0xf7, 0xed, 0x66, 0x05, 0x2b, 0x14, 0xdd, 0x87,
-	0x36, 0xb3, 0x75, 0xe3, 0x33, 0x42, 0x69, 0xe0, 0x13, 0xb1, 0xeb, 0x94, 0x5a, 0x8b, 0xc9, 0x86,
-	0x52, 0x84, 0x1e, 0x82, 0xc6, 0xcd, 0x12, 0xb3, 0xc6, 0x07, 0xbe, 0x96, 0x0f, 0xbc, 0x10, 0x38,
-	0x2c, 0x95, 0x8a, 0x13, 0xd5, 0xbe, 0x67, 0xa2, 0xe8, 0x36, 0x34, 0x66, 0xde, 0x85, 0x9b, 0xbc,
-	0x24, 0xe7, 0x7c, 0xb7, 0xb6, 0x71, 0x7d, 0xe6, 0x5d, 0x8c, 0x5e, 0x92, 0xf3, 0xee, 0x6f, 0xaa,
-	0x50, 0xb3, 0x67, 0xde, 0x09, 0x79, 0xe3, 0xce, 0x32, 0xa1, 0x7e, 0x46, 0x68, 0x12, 0xc4, 0x91,
-	0x4a, 0x4d, 0xd9, 0x64, 0xda, 0xa7, 0x5e, 0x72, 0xca, 0xe7, 0xdd, 0xc6, 0xfc, 0x1b, 0x7d, 0x04,
-	0x46, 0x10, 0x25, 0xa9, 0x17, 0x86, 0x2e, 0xdb, 0xf1, 0x69, 0x30, 0x13, 0x13, 0xd6, 0xf1, 0x8a,
-	0xc4, 0x07, 0x12, 0x66, 0x7c, 0x11, 0x24, 0xae, 0x37, 0x4d, 0x83, 0x33, 0xc2, 0xf9, 0xa2, 0x81,
-	0x1b, 0x41, 0xd2, 0xe3, 0x6d, 0x16, 0xf9, 0x20, 0x71, 0x19, 0x73, 0x05, 0x69, 0x4a, 0x7c, 0x53,
-	0xe3, 0xf2, 0x66, 0x90, 0xf4, 0x15, 0xc4, 0x66, 0x14, 0x24, 0xee, 0x99, 0x17, 0x06, 0xbe, 0xcc,
-	0xbf, 0x7a, 0x90, 0x1c, 0xb2, 0x26, 0x32, 0x60, 0x69, 0x41, 0x43, 0xb3, 0xc1, 0x3b, 0x66, 0x9f,
-	0xe8, 0x26, 0x68, 0x82, 0x6d, 0x4c, 0x9d, 0x83, 0xb2, 0x85, 0x56, 0xa1, 0x36, 0xa5, 0xd3, 0x27,
-	0x8f, 0x4d, 0xe0, 0x93, 0x10, 0x8d, 0xee, 0xdf, 0xea, 0xd0, 0xe6, 0x11, 0x19, 0xc4, 0xe7, 0x51,
-	0x18, 0x7b, 0xfe, 0x95, 0x9d, 0xa1, 0x22, 0x55, 0x2d, 0x44, 0x4a, 0xf6, 0xba, 0x94, 0xf7, 0x6a,
-	0xc0, 0xd2, 0x94, 0x4e, 0x65, 0x1a, 0xb1, 0x4f, 0x34, 0x84, 0x8e, 0x2f, 0x7d, 0xba, 0x49, 0xca,
-	0xa8, 0xa3, 0xc6, 0x33, 0x76, 0x4b, 0xad, 0x5c, 0xa9, 0xdb, 0x72, 0x6b, 0xc4, 0xf4, 0x71, 0xdb,
-	0x2f, 0x36, 0xd1, 0x3d, 0x68, 0x07, 0x4c, 0xc9, 0x55, 0x8b, 0xa4, 0xf1, 0xee, 0x5b, 0x1c, 0x3c,
-	0x94, 0x2b, 0xf5, 0x11, 0x18, 0xca, 0x8a, 0xf8, 0xee, 0xd1, 0x6b, 0x46, 0x7e, 0x62, 0x13, 0xac,
-	0xe4, 0xf8, 0x2e, 0x83, 0xd1, 0x73, 0xd0, 0x28, 0xf1, 0x92, 0x38, 0xe2, 0xd1, 0xeb, 0x3c, 0x7e,
-	0xf4, 0x03, 0x06, 0xb6, 0xe7, 0x05, 0xe1, 0x82, 0x12, 0xcc, 0xed, 0xb0, 0xb4, 0x47, 0x1f, 0xc2,
-	0x8a, 0xe7, 0xfb, 0x41, 0x1a, 0xc4, 0x91, 0x17, 0xba, 0x41, 0x74, 0x1c, 0xcb, 0xd8, 0x77, 0x72,
-	0xd8, 0x8e, 0x8e, 0x63, 0x41, 0x3a, 0x67, 0xc4, 0x9d, 0xf2, 0x2d, 0xcb, 0x57, 0xa2, 0xc1, 0x48,
-	0xe7, 0x8c, 0x48, 0xa2, 0xd8, 0x00, 0x3d, 0x8c, 0x19, 0xe7, 0xfa, 0x01, 0x35, 0x9b, 0xe2, 0x64,
-	0xe1, 0xc0, 0x20, 0xa0, 0xc8, 0x86, 0xa6, 0x08, 0x80, 0x08, 0x67, 0xeb, 0x7b, 0xc3, 0xc9, 0x77,
-	0x98, 0x97, 0x12, 0x11, 0x4e, 0xe0, 0xc6, 0x22, 0x96, 0x1b, 0xa0, 0x1f, 0x07, 0x21, 0x71, 0x93,
-	0xe0, 0x2b, 0x62, 0xb6, 0x79, 0x7c, 0x1a, 0x0c, 0x18, 0x05, 0x5f, 0x91, 0xee, 0x37, 0x15, 0x40,
-	0x57, 0x97, 0x03, 0xad, 0x82, 0x31, 0x18, 0x7e, 0xe1, 0xec, 0x0f, 0x7b, 0x03, 0x77, 0xe2, 0xfc,
-	0xc4, 0x19, 0x7e, 0xe1, 0x18, 0xd7, 0xd0, 0x4d, 0x40, 0x19, 0x3a, 0x9a, 0xf4, 0xfb, 0x96, 0x35,
-	0xb0, 0x06, 0x46, 0xa5, 0x84, 0x63, 0xeb, 0xa7, 0x13, 0x6b, 0x34, 0xb6, 0x06, 0x46, 0xb5, 0xe4,
-	0x65, 0x34, 0xee, 0x61, 0x86, 0x2e, 0xa1, 0x1b, 0xb0, 0x92, 0xa1, 0x7b, 0x3d, 0x7b, 0xdf, 0x1a,
-	0x18, 0xcb, 0xc8, 0x84, 0xd5, 0x42, 0x87, 0xa3, 0xc9, 0xc1, 0xc1, 0x90, 0xab, 0xd7, 0x4a, 0xce,
-	0xfb, 0x3d, 0xa7, 0x6f, 0xed, 0x33, 0x0b, 0xad, 0xfb, 0xab, 0x0a, 0xac, 0xbf, 0x7d, 0xbd, 0x50,
-	0x0b, 0x1a, 0xce, 0xd0, 0xb5, 0x30, 0x1e, 0x32, 0x26, 0x5f, 0x81, 0xa6, 0xed, 0x1c, 0xf6, 0xf6,
-	0xed, 0x81, 0x3b, 0xc1, 0xfb, 0x46, 0x85, 0x01, 0x03, 0xeb, 0xd0, 0xee, 0x5b, 0xee, 0xee, 0x64,
-	0xf4, 0xa5, 0x51, 0x65, 0xdd, 0xd8, 0xce, 0x68, 0xb2, 0xb7, 0x67, 0xf7, 0x6d, 0xcb, 0x19, 0xbb,
-	0xa3, 0x83, 0x5e, 0xdf, 0x32, 0x96, 0xd0, 0x75, 0x68, 0xcb, 0x00, 0x48, 0x67, 0xcb, 0xa8, 0x0d,
-	0x7a, 0x3e, 0x90, 0x5a, 0xf7, 0xd7, 0x2a, 0x84, 0xa5, 0x25, 0x60, 0x86, 0xf6, 0x8b, 0xde, 0x33,
-	0xab, 0x10, 0x3f, 0x04, 0x1d, 0x01, 0xd9, 0x4e, 0xaf, 0x3f, 0xb6, 0x0f, 0xd9, 0xc1, 0xb2, 0x0a,
-	0x86, 0xc0, 0x38, 0xd2, 0x1b, 0xdb, 0xce, 0x33, 0xa3, 0x8a, 0x0c, 0x68, 0x15, 0x50, 0x4b, 0x44,
-	0x4d, 0x20, 0xd8, 0x3a, 0xb4, 0x30, 0x57, 0x5b, 0xce, 0x1d, 0x0a, 0x90, 0x0f, 0xe7, 0x73, 0xe8,
-	0x94, 0xc2, 0x92, 0xa0, 0x07, 0xea, 0x40, 0xae, 0x96, 0xe9, 0xb7, 0xa4, 0xa6, 0xce, 0xe4, 0x87,
-	0xa0, 0x71, 0x3c, 0x41, 0xf7, 0xa0, 0xc6, 0x77, 0x91, 0x3c, 0xc7, 0xdb, 0x25, 0x33, 0x2c, 0x64,
-	0xdd, 0x3f, 0x56, 0xa0, 0x31, 0x8c, 0x16, 0x82, 0x68, 0x0b, 0xa4, 0x5a, 0x29, 0x93, 0xea, 0xff,
-	0x00, 0x28, 0x92, 0x23, 0x3e, 0xa7, 0x97, 0x06, 0x2e, 0x20, 0x68, 0x1d, 0x32, 0x92, 0x94, 0xe7,
-	0x5e, 0x4e, 0x9a, 0x26, 0x28, 0x06, 0x94, 0xa5, 0x4d, 0x46, 0x88, 0x77, 0xa1, 0x39, 0xa7, 0xb1,
-	0xbf, 0x98, 0xa6, 0xfd, 0xd8, 0x27, 0xb2, 0x3a, 0x2b, 0x42, 0x19, 0x99, 0x0b, 0xfa, 0xe0, 0xdf,
-	0xdd, 0x27, 0xa0, 0xab, 0x11, 0x27, 0xe8, 0xff, 0xca, 0xc5, 0x4a, 0x76, 0xd4, 0x28, 0x0d, 0x15,
-	0x96, 0x29, 0x18, 0xa2, 0x7e, 0xb1, 0x4b, 0x89, 0x25, 0x6a, 0x58, 0x37, 0x23, 0xd1, 0x86, 0x00,
-	0x6c, 0x1f, 0x3d, 0x86, 0x42, 0x0e, 0xf2, 0x19, 0x17, 0x4a, 0xa1, 0xdc, 0x49, 0x31, 0x53, 0x19,
-	0x41, 0x43, 0xc1, 0xff, 0xdb, 0xc3, 0xb9, 0x7f, 0x85, 0x6f, 0x45, 0x85, 0xf4, 0xc1, 0xd5, 0x0e,
-	0x7e, 0x00, 0xd9, 0x7e, 0x9e, 0x91, 0xe3, 0xd2, 0xbb, 0xbd, 0xbc, 0x99, 0x11, 0x9f, 0x97, 0xa9,
-	0x6a, 0x99, 0xfb, 0xf8, 0xf0, 0x6d, 0x3e, 0x64, 0x92, 0x04, 0x71, 0x74, 0x75, 0xfe, 0x7f, 0xf9,
-	0xaf, 0x27, 0xa3, 0x5b, 0x70, 0xe3, 0x32, 0x19, 0xb1, 0x4c, 0xd4, 0xde, 0xc2, 0x52, 0xf5, 0xee,
-	0x3f, 0xd4, 0x94, 0xfe, 0x63, 0xec, 0x64, 0xc2, 0x6a, 0x36, 0x00, 0x77, 0xe8, 0xa8, 0x18, 0x18,
-	0x35, 0xb4, 0x0e, 0x37, 0x4b, 0x92, 0xa1, 0x33, 0x71, 0x45, 0x51, 0xab, 0x31, 0xd9, 0xa1, 0xe5,
-	0x0c, 0x86, 0xd8, 0x95, 0x1d, 0xbf, 0xb0, 0x47, 0x2f, 0x7a, 0xe3, 0xfe, 0x73, 0xa3, 0xce, 0x26,
-	0x3d, 0x7c, 0xd1, 0xb7, 0xdd, 0x31, 0xee, 0x39, 0xa3, 0x3d, 0x0b, 0xcb, 0xae, 0x1a, 0xac, 0x2b,
-	0x45, 0x3f, 0x7b, 0x93, 0x91, 0x35, 0x70, 0x77, 0xbf, 0x64, 0x4e, 0x0d, 0xbd, 0xfb, 0xbb, 0x2a,
-	0xac, 0xbe, 0x69, 0xb9, 0xff, 0xdd, 0xac, 0x98, 0xe9, 0xf5, 0x87, 0x2f, 0x5e, 0xd8, 0x63, 0x49,
-	0x8b, 0x19, 0x57, 0x4a, 0x94, 0x2f, 0xdd, 0x26, 0xdc, 0x2e, 0xbb, 0x1c, 0x3a, 0x6e, 0x6f, 0x77,
-	0x28, 0xa8, 0x54, 0x43, 0xef, 0x81, 0xf9, 0x66, 0x31, 0x5b, 0x46, 0x74, 0x1b, 0xd6, 0x8a, 0x1e,
-	0x73, 0xc3, 0x42, 0x10, 0x8a, 0x22, 0x6b, 0x60, 0xe8, 0x68, 0x0d, 0xae, 0x0b, 0x89, 0xda, 0x19,
-	0xcc, 0x00, 0xba, 0xdf, 0xd4, 0x60, 0xf9, 0x20, 0xa6, 0x29, 0xba, 0x05, 0xf5, 0x79, 0x4c, 0x53,
-	0x37, 0x8a, 0x79, 0x7e, 0xb7, 0xb1, 0xc6, 0x9a, 0x4e, 0xcc, 0xca, 0xb7, 0xd0, 0x3b, 0x22, 0xa1,
-	0xac, 0xc3, 0x44, 0x03, 0x7d, 0x24, 0x2f, 0x43, 0x22, 0x49, 0xf3, 0x22, 0x3a, 0xa6, 0x29, 0xff,
-	0x53, 0xb8, 0x0a, 0xfd, 0x18, 0x9a, 0x9e, 0x3f, 0x0b, 0xa2, 0x52, 0x31, 0x66, 0x6e, 0xcb, 0x2b,
-	0x73, 0x8f, 0x89, 0x44, 0x4a, 0xf2, 0x1b, 0x1b, 0x06, 0x2f, 0x43, 0x98, 0x69, 0x3c, 0x27, 0x94,
-	0x5b, 0x2e, 0x12, 0x4e, 0x9c, 0x05, 0xd3, 0xe1, 0x9c, 0xd0, 0x11, 0x97, 0x28, 0xd3, 0x38, 0x43,
-	0xca, 0x7c, 0x58, 0xbf, 0xc4, 0x87, 0x0f, 0xa0, 0x36, 0x27, 0x84, 0x26, 0x66, 0xe3, 0xd2, 0x1d,
-	0x80, 0x0f, 0x9f, 0x10, 0xca, 0x3e, 0xb0, 0xd0, 0x61, 0x97, 0x24, 0x7a, 0xe1, 0xce, 0xbd, 0xe9,
-	0x4b, 0x92, 0x26, 0xbc, 0xbe, 0xd2, 0xb0, 0x4e, 0x2f, 0x0e, 0x04, 0xc0, 0x6a, 0x64, 0x7a, 0x21,
-	0x0b, 0x3e, 0xe0, 0xc2, 0x3a, 0xbd, 0x10, 0x85, 0xde, 0x06, 0xe8, 0xf4, 0xc2, 0x25, 0x94, 0xc6,
-	0x34, 0xe1, 0x45, 0x95, 0x86, 0x1b, 0xf4, 0xc2, 0xe2, 0x6d, 0xe6, 0x36, 0xcd, 0xdd, 0xb6, 0x84,
-	0xdb, 0xb4, 0xe8, 0x36, 0x55, 0x6e, 0xdb, 0xc2, 0x6d, 0x9a, 0xbb, 0x4d, 0x33, 0xb7, 0x1d, 0xe1,
-	0x36, 0x55, 0x6e, 0x1f, 0x41, 0x23, 0x3e, 0x9e, 0xbb, 0x6c, 0xf1, 0xcc, 0x15, 0x4e, 0xf4, 0x6b,
-	0xdb, 0xc5, 0x77, 0x06, 0x25, 0xc4, 0xf5, 0xf8, 0x78, 0xce, 0xa6, 0xb9, 0xfe, 0x14, 0x1a, 0x6a,
-	0xca, 0xef, 0x3e, 0x45, 0x0a, 0x5b, 0xa4, 0x5a, 0xdc, 0x22, 0xdd, 0x04, 0x1a, 0x6a, 0xcd, 0xd9,
-	0x85, 0x34, 0xcf, 0x26, 0x03, 0x5a, 0xd6, 0xf8, 0xb9, 0x85, 0x1d, 0x6b, 0xec, 0x3a, 0x8e, 0x6d,
-	0x54, 0x4a, 0xc8, 0xc4, 0xb1, 0xc5, 0x0d, 0xf6, 0x80, 0xe5, 0xff, 0xfe, 0xd8, 0x58, 0xca, 0x1a,
-	0xce, 0x44, 0x94, 0x36, 0x87, 0x16, 0x53, 0x64, 0xb2, 0x5a, 0xa1, 0xe9, 0x4c, 0x0c, 0xad, 0xfb,
-	0x00, 0x6a, 0xac, 0xd3, 0x04, 0x75, 0xcb, 0xa7, 0x66, 0xab, 0xb8, 0x98, 0xea, 0xc4, 0xfc, 0x7d,
-	0x13, 0x34, 0x71, 0x64, 0xa2, 0xb5, 0xfc, 0x9a, 0xa1, 0x6e, 0x88, 0xec, 0xb6, 0x71, 0xbb, 0x70,
-	0xbb, 0xcf, 0x04, 0x62, 0x03, 0xdf, 0x86, 0x65, 0x1a, 0xc7, 0x69, 0xf9, 0xf2, 0xc9, 0x21, 0xd4,
-	0x05, 0x7d, 0xee, 0x51, 0x12, 0xa5, 0xae, 0x2c, 0x08, 0x32, 0xd3, 0x86, 0xc0, 0xf9, 0x66, 0xeb,
-	0x48, 0x1d, 0x15, 0xbd, 0x55, 0x16, 0xbd, 0xec, 0x7a, 0x2a, 0x84, 0x07, 0x22, 0xdb, 0x36, 0xb3,
-	0x4b, 0x54, 0xad, 0xe8, 0x4d, 0xdd, 0xa5, 0x36, 0xa0, 0x36, 0x8b, 0x7d, 0x12, 0x8a, 0x1a, 0x42,
-	0x49, 0x05, 0x86, 0x1e, 0x81, 0x71, 0xea, 0x51, 0xff, 0xdc, 0xa3, 0xf9, 0x55, 0xa5, 0x5e, 0xd4,
-	0x5b, 0x51, 0x62, 0x75, 0x69, 0x79, 0x04, 0xc6, 0x71, 0x40, 0x67, 0x25, 0x8b, 0x46, 0xc9, 0x42,
-	0x89, 0x95, 0xc5, 0x43, 0xd0, 0xf8, 0x19, 0x29, 0x12, 0xa1, 0xf9, 0xb8, 0x53, 0x3a, 0x5a, 0x93,
-	0x6c, 0xbc, 0x42, 0x89, 0xdd, 0xcc, 0x13, 0x42, 0x03, 0x2f, 0x74, 0xa3, 0xc5, 0xec, 0x88, 0x50,
-	0x9e, 0x21, 0x99, 0xf7, 0x96, 0x90, 0x39, 0x5c, 0xc4, 0x62, 0x99, 0x3f, 0x6e, 0x99, 0xa5, 0x58,
-	0x66, 0x6f, 0x5c, 0x77, 0xf2, 0x47, 0xac, 0x66, 0x51, 0x23, 0x7b, 0xcb, 0x42, 0xb0, 0x7c, 0x16,
-	0x7a, 0x11, 0xcf, 0xa7, 0x36, 0xe6, 0xdf, 0xec, 0xf2, 0x33, 0xf3, 0xa6, 0xae, 0xe7, 0xfb, 0x94,
-	0x24, 0x22, 0x9b, 0x74, 0x0c, 0x33, 0x6f, 0xda, 0x13, 0x08, 0xba, 0x07, 0xad, 0x60, 0x7e, 0xf6,
-	0x71, 0xa6, 0xc1, 0x72, 0x4a, 0x7f, 0x7e, 0x0d, 0x37, 0x19, 0x5a, 0x56, 0xfa, 0x24, 0x53, 0x5a,
-	0x29, 0x28, 0x7d, 0xa2, 0x94, 0xde, 0x87, 0xf6, 0x69, 0x9c, 0xa4, 0xae, 0x17, 0xf9, 0x22, 0x05,
-	0xd7, 0x94, 0x16, 0x83, 0x7b, 0x91, 0xcf, 0xb3, 0x6c, 0x13, 0x80, 0x5c, 0xa4, 0xd4, 0x73, 0x3d,
-	0x7a, 0x92, 0x98, 0xb7, 0xc4, 0xab, 0x0c, 0x47, 0x7a, 0xf4, 0x24, 0x41, 0x4f, 0xa1, 0x3d, 0xa7,
-	0xf1, 0xc5, 0xeb, 0xac, 0xab, 0x1b, 0x3c, 0xd4, 0x1b, 0xe5, 0xb7, 0xab, 0xed, 0x03, 0xa6, 0x23,
-	0x3b, 0xc6, 0xad, 0x79, 0xa1, 0x75, 0x99, 0x72, 0x8d, 0x7f, 0x81, 0x72, 0x9f, 0x96, 0x29, 0xf7,
-	0xfa, 0xbb, 0x29, 0x57, 0xc5, 0xbf, 0xc8, 0xbc, 0x9b, 0x59, 0x05, 0x77, 0xb3, 0xb4, 0x85, 0x65,
-	0x85, 0x66, 0x43, 0x67, 0x1a, 0x47, 0x11, 0x99, 0xa6, 0xaa, 0x0f, 0xc4, 0xfb, 0xd8, 0x50, 0x7d,
-	0xf4, 0x85, 0xf4, 0x4d, 0xdd, 0xb4, 0xa7, 0x45, 0x19, 0xfa, 0x7f, 0xd0, 0xa6, 0x8b, 0x24, 0x8d,
-	0x67, 0xe6, 0x53, 0x1e, 0xa1, 0xd5, 0x6d, 0xf1, 0x60, 0xbb, 0xad, 0x1e, 0x6c, 0xb7, 0x7b, 0xd1,
-	0x6b, 0x2c, 0x75, 0xd0, 0x67, 0x00, 0xf3, 0x99, 0xbc, 0x01, 0x27, 0xe6, 0xd7, 0x15, 0x6e, 0x72,
-	0xfd, 0xf2, 0x73, 0x4e, 0xb2, 0x5b, 0xfb, 0xeb, 0x77, 0xdf, 0x6e, 0x5e, 0xc3, 0xfa, 0x3c, 0x7b,
-	0xb3, 0xda, 0x87, 0x15, 0x51, 0x54, 0xaa, 0x52, 0x35, 0x31, 0x7f, 0x59, 0x79, 0xc7, 0xf5, 0x65,
-	0xb7, 0xc9, 0x5c, 0x68, 0xe2, 0xfd, 0x02, 0x77, 0x82, 0xd2, 0x0d, 0x68, 0xfd, 0xeb, 0x2a, 0xb4,
-	0x8a, 0x6b, 0xf7, 0x6e, 0xd2, 0xbd, 0x03, 0x4d, 0x29, 0xcc, 0xe9, 0x09, 0x83, 0x9f, 0xbf, 0x11,
-	0x6f, 0x02, 0x4c, 0x4f, 0xbd, 0x28, 0x22, 0x21, 0x33, 0x17, 0x0f, 0x45, 0xba, 0x44, 0x6c, 0x1f,
-	0x6d, 0x81, 0xa1, 0xc4, 0xe2, 0xa9, 0x4f, 0x12, 0x55, 0x1b, 0x77, 0x24, 0xce, 0x9f, 0xbd, 0x6c,
-	0x1f, 0xed, 0xc0, 0x0d, 0xa5, 0x99, 0x12, 0x3a, 0x0b, 0x22, 0x5e, 0x29, 0xc9, 0x8b, 0x0c, 0x92,
-	0xa2, 0x71, 0x2e, 0x41, 0x6b, 0xa0, 0xc5, 0xd1, 0x82, 0x39, 0xd4, 0xc4, 0xcb, 0x4e, 0x1c, 0x2d,
-	0x6c, 0x1f, 0xbd, 0x0f, 0x1d, 0x06, 0x27, 0x24, 0x61, 0x8c, 0xa1, 0x8e, 0xdf, 0x36, 0x6e, 0xc5,
-	0xd1, 0x62, 0x24, 0x40, 0xdb, 0xdf, 0xd5, 0x59, 0x26, 0xf3, 0xf9, 0x77, 0x77, 0xa0, 0x2e, 0xb6,
-	0x34, 0xcb, 0x9f, 0x12, 0x97, 0x77, 0xca, 0x5b, 0x5e, 0xb1, 0xf9, 0x1f, 0x96, 0x60, 0x75, 0x14,
-	0xcc, 0x16, 0xa1, 0x97, 0x92, 0x5e, 0xe8, 0xd1, 0x19, 0x26, 0xaf, 0x16, 0x24, 0x49, 0xaf, 0x3c,
-	0x21, 0xbd, 0x07, 0x7a, 0x10, 0xf9, 0xc1, 0xd4, 0x4b, 0x63, 0xf5, 0xea, 0x9d, 0x03, 0xec, 0x3c,
-	0x0b, 0xa2, 0xf4, 0x58, 0x85, 0x4d, 0xc7, 0x1a, 0x6b, 0x8a, 0x19, 0x70, 0xaa, 0x66, 0x11, 0x17,
-	0x2f, 0xa7, 0xe2, 0x7d, 0xad, 0x35, 0x97, 0xa7, 0x1c, 0x7f, 0x3c, 0xed, 0x42, 0x9b, 0xcd, 0x33,
-	0x5f, 0x3a, 0x79, 0xe5, 0x8b, 0xa3, 0xc5, 0x40, 0xad, 0xde, 0x13, 0xb8, 0x19, 0x44, 0x8c, 0x59,
-	0x89, 0x7b, 0x14, 0xa4, 0xe2, 0xcc, 0x76, 0x29, 0xcb, 0x49, 0x16, 0xb2, 0x1a, 0xbe, 0x21, 0xa5,
-	0xbb, 0x41, 0xca, 0xcf, 0x6f, 0x2c, 0xae, 0x18, 0x35, 0x9f, 0x06, 0xc7, 0x29, 0x8f, 0x5b, 0x0d,
-	0x8b, 0x06, 0x1b, 0x6d, 0x44, 0xce, 0x5d, 0xf2, 0xca, 0xe7, 0x14, 0x5d, 0xc3, 0x5a, 0x44, 0xce,
-	0xad, 0x57, 0x3e, 0xba, 0x0f, 0xd7, 0x45, 0xbc, 0x8b, 0x3c, 0x2b, 0x9e, 0x81, 0x56, 0x78, 0xc8,
-	0x0b, 0x1c, 0xfb, 0x1c, 0x74, 0x96, 0xa9, 0x62, 0x65, 0x81, 0xe7, 0xdd, 0x7d, 0x15, 0xe3, 0x37,
-	0x45, 0x94, 0x27, 0x3c, 0xd7, 0xe6, 0x05, 0x5d, 0x6e, 0xdc, 0xfd, 0x00, 0xda, 0x25, 0x19, 0xd2,
-	0xa1, 0x86, 0x7b, 0xf6, 0xc8, 0x12, 0x4f, 0xd5, 0xfd, 0x7d, 0xab, 0x87, 0x8d, 0xca, 0xee, 0x08,
-	0x6e, 0xc4, 0xf4, 0x84, 0x57, 0x20, 0xd3, 0x98, 0xfa, 0xb2, 0xaf, 0xdd, 0xd6, 0x21, 0xff, 0x2f,
-	0xe2, 0xf4, 0xf3, 0xed, 0x93, 0x20, 0x3d, 0x5d, 0x1c, 0x31, 0x02, 0xd8, 0x51, 0x9a, 0x3b, 0x42,
-	0xf3, 0xa1, 0xfc, 0xa9, 0xe4, 0xec, 0xe3, 0x9d, 0x93, 0x58, 0x62, 0x47, 0x1a, 0x07, 0x9f, 0xfc,
-	0x33, 0x00, 0x00, 0xff, 0xff, 0x07, 0x84, 0x3f, 0x04, 0xc4, 0x19, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/events.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/events.pb.go
deleted file mode 100644
index 2dc1826..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/events.pb.go
+++ /dev/null
@@ -1,1237 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/events.proto
-
-package voltha
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	timestamp "github.com/golang/protobuf/ptypes/timestamp"
-	common "github.com/opencord/voltha-protos/v4/go/common"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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 ConfigEventType_Types int32
-
-const (
-	ConfigEventType_add    ConfigEventType_Types = 0
-	ConfigEventType_remove ConfigEventType_Types = 1
-	ConfigEventType_update ConfigEventType_Types = 2
-)
-
-var ConfigEventType_Types_name = map[int32]string{
-	0: "add",
-	1: "remove",
-	2: "update",
-}
-
-var ConfigEventType_Types_value = map[string]int32{
-	"add":    0,
-	"remove": 1,
-	"update": 2,
-}
-
-func (x ConfigEventType_Types) String() string {
-	return proto.EnumName(ConfigEventType_Types_name, int32(x))
-}
-
-func (ConfigEventType_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{0, 0}
-}
-
-type KpiEventType_Types int32
-
-const (
-	KpiEventType_slice KpiEventType_Types = 0
-	KpiEventType_ts    KpiEventType_Types = 1
-)
-
-var KpiEventType_Types_name = map[int32]string{
-	0: "slice",
-	1: "ts",
-}
-
-var KpiEventType_Types_value = map[string]int32{
-	"slice": 0,
-	"ts":    1,
-}
-
-func (x KpiEventType_Types) String() string {
-	return proto.EnumName(KpiEventType_Types_name, int32(x))
-}
-
-func (KpiEventType_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{2, 0}
-}
-
-type EventCategory_Types int32
-
-const (
-	EventCategory_COMMUNICATION EventCategory_Types = 0
-	EventCategory_ENVIRONMENT   EventCategory_Types = 1
-	EventCategory_EQUIPMENT     EventCategory_Types = 2
-	EventCategory_SERVICE       EventCategory_Types = 3
-	EventCategory_PROCESSING    EventCategory_Types = 4
-	EventCategory_SECURITY      EventCategory_Types = 5
-)
-
-var EventCategory_Types_name = map[int32]string{
-	0: "COMMUNICATION",
-	1: "ENVIRONMENT",
-	2: "EQUIPMENT",
-	3: "SERVICE",
-	4: "PROCESSING",
-	5: "SECURITY",
-}
-
-var EventCategory_Types_value = map[string]int32{
-	"COMMUNICATION": 0,
-	"ENVIRONMENT":   1,
-	"EQUIPMENT":     2,
-	"SERVICE":       3,
-	"PROCESSING":    4,
-	"SECURITY":      5,
-}
-
-func (x EventCategory_Types) String() string {
-	return proto.EnumName(EventCategory_Types_name, int32(x))
-}
-
-func (EventCategory_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{10, 0}
-}
-
-type EventSubCategory_Types int32
-
-const (
-	EventSubCategory_PON  EventSubCategory_Types = 0
-	EventSubCategory_OLT  EventSubCategory_Types = 1
-	EventSubCategory_ONT  EventSubCategory_Types = 2
-	EventSubCategory_ONU  EventSubCategory_Types = 3
-	EventSubCategory_NNI  EventSubCategory_Types = 4
-	EventSubCategory_NONE EventSubCategory_Types = 5
-)
-
-var EventSubCategory_Types_name = map[int32]string{
-	0: "PON",
-	1: "OLT",
-	2: "ONT",
-	3: "ONU",
-	4: "NNI",
-	5: "NONE",
-}
-
-var EventSubCategory_Types_value = map[string]int32{
-	"PON":  0,
-	"OLT":  1,
-	"ONT":  2,
-	"ONU":  3,
-	"NNI":  4,
-	"NONE": 5,
-}
-
-func (x EventSubCategory_Types) String() string {
-	return proto.EnumName(EventSubCategory_Types_name, int32(x))
-}
-
-func (EventSubCategory_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{11, 0}
-}
-
-type EventType_Types int32
-
-const (
-	EventType_CONFIG_EVENT EventType_Types = 0
-	EventType_KPI_EVENT    EventType_Types = 1
-	EventType_KPI_EVENT2   EventType_Types = 2
-	EventType_DEVICE_EVENT EventType_Types = 3
-	EventType_RPC_EVENT    EventType_Types = 4
-)
-
-var EventType_Types_name = map[int32]string{
-	0: "CONFIG_EVENT",
-	1: "KPI_EVENT",
-	2: "KPI_EVENT2",
-	3: "DEVICE_EVENT",
-	4: "RPC_EVENT",
-}
-
-var EventType_Types_value = map[string]int32{
-	"CONFIG_EVENT": 0,
-	"KPI_EVENT":    1,
-	"KPI_EVENT2":   2,
-	"DEVICE_EVENT": 3,
-	"RPC_EVENT":    4,
-}
-
-func (x EventType_Types) String() string {
-	return proto.EnumName(EventType_Types_name, int32(x))
-}
-
-func (EventType_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{12, 0}
-}
-
-type ConfigEventType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ConfigEventType) Reset()         { *m = ConfigEventType{} }
-func (m *ConfigEventType) String() string { return proto.CompactTextString(m) }
-func (*ConfigEventType) ProtoMessage()    {}
-func (*ConfigEventType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{0}
-}
-
-func (m *ConfigEventType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ConfigEventType.Unmarshal(m, b)
-}
-func (m *ConfigEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ConfigEventType.Marshal(b, m, deterministic)
-}
-func (m *ConfigEventType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ConfigEventType.Merge(m, src)
-}
-func (m *ConfigEventType) XXX_Size() int {
-	return xxx_messageInfo_ConfigEventType.Size(m)
-}
-func (m *ConfigEventType) XXX_DiscardUnknown() {
-	xxx_messageInfo_ConfigEventType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ConfigEventType proto.InternalMessageInfo
-
-type ConfigEvent struct {
-	Type                 ConfigEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.ConfigEventType_Types" json:"type,omitempty"`
-	Hash                 string                `protobuf:"bytes,2,opt,name=hash,proto3" json:"hash,omitempty"`
-	Data                 string                `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *ConfigEvent) Reset()         { *m = ConfigEvent{} }
-func (m *ConfigEvent) String() string { return proto.CompactTextString(m) }
-func (*ConfigEvent) ProtoMessage()    {}
-func (*ConfigEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{1}
-}
-
-func (m *ConfigEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ConfigEvent.Unmarshal(m, b)
-}
-func (m *ConfigEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ConfigEvent.Marshal(b, m, deterministic)
-}
-func (m *ConfigEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ConfigEvent.Merge(m, src)
-}
-func (m *ConfigEvent) XXX_Size() int {
-	return xxx_messageInfo_ConfigEvent.Size(m)
-}
-func (m *ConfigEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_ConfigEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ConfigEvent proto.InternalMessageInfo
-
-func (m *ConfigEvent) GetType() ConfigEventType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return ConfigEventType_add
-}
-
-func (m *ConfigEvent) GetHash() string {
-	if m != nil {
-		return m.Hash
-	}
-	return ""
-}
-
-func (m *ConfigEvent) GetData() string {
-	if m != nil {
-		return m.Data
-	}
-	return ""
-}
-
-type KpiEventType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *KpiEventType) Reset()         { *m = KpiEventType{} }
-func (m *KpiEventType) String() string { return proto.CompactTextString(m) }
-func (*KpiEventType) ProtoMessage()    {}
-func (*KpiEventType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{2}
-}
-
-func (m *KpiEventType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_KpiEventType.Unmarshal(m, b)
-}
-func (m *KpiEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_KpiEventType.Marshal(b, m, deterministic)
-}
-func (m *KpiEventType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_KpiEventType.Merge(m, src)
-}
-func (m *KpiEventType) XXX_Size() int {
-	return xxx_messageInfo_KpiEventType.Size(m)
-}
-func (m *KpiEventType) XXX_DiscardUnknown() {
-	xxx_messageInfo_KpiEventType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_KpiEventType proto.InternalMessageInfo
-
-//
-// Struct to convey a dictionary of metric metadata.
-type MetricMetaData struct {
-	Title           string  `protobuf:"bytes,1,opt,name=title,proto3" json:"title,omitempty"`
-	Ts              float64 `protobuf:"fixed64,2,opt,name=ts,proto3" json:"ts,omitempty"`
-	LogicalDeviceId string  `protobuf:"bytes,3,opt,name=logical_device_id,json=logicalDeviceId,proto3" json:"logical_device_id,omitempty"`
-	// (equivalent to the DPID that ONOS has
-	// for the VOLTHA device without the
-	//  'of:' prefix
-	SerialNo             string            `protobuf:"bytes,4,opt,name=serial_no,json=serialNo,proto3" json:"serial_no,omitempty"`
-	DeviceId             string            `protobuf:"bytes,5,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	Context              map[string]string `protobuf:"bytes,6,rep,name=context,proto3" json:"context,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	Uuid                 string            `protobuf:"bytes,7,opt,name=uuid,proto3" json:"uuid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *MetricMetaData) Reset()         { *m = MetricMetaData{} }
-func (m *MetricMetaData) String() string { return proto.CompactTextString(m) }
-func (*MetricMetaData) ProtoMessage()    {}
-func (*MetricMetaData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{3}
-}
-
-func (m *MetricMetaData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MetricMetaData.Unmarshal(m, b)
-}
-func (m *MetricMetaData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MetricMetaData.Marshal(b, m, deterministic)
-}
-func (m *MetricMetaData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MetricMetaData.Merge(m, src)
-}
-func (m *MetricMetaData) XXX_Size() int {
-	return xxx_messageInfo_MetricMetaData.Size(m)
-}
-func (m *MetricMetaData) XXX_DiscardUnknown() {
-	xxx_messageInfo_MetricMetaData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MetricMetaData proto.InternalMessageInfo
-
-func (m *MetricMetaData) GetTitle() string {
-	if m != nil {
-		return m.Title
-	}
-	return ""
-}
-
-func (m *MetricMetaData) GetTs() float64 {
-	if m != nil {
-		return m.Ts
-	}
-	return 0
-}
-
-func (m *MetricMetaData) GetLogicalDeviceId() string {
-	if m != nil {
-		return m.LogicalDeviceId
-	}
-	return ""
-}
-
-func (m *MetricMetaData) GetSerialNo() string {
-	if m != nil {
-		return m.SerialNo
-	}
-	return ""
-}
-
-func (m *MetricMetaData) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *MetricMetaData) GetContext() map[string]string {
-	if m != nil {
-		return m.Context
-	}
-	return nil
-}
-
-func (m *MetricMetaData) GetUuid() string {
-	if m != nil {
-		return m.Uuid
-	}
-	return ""
-}
-
-//
-// Struct to convey a dictionary of metric->value pairs. Typically used in
-// pure shared-timestamp or shared-timestamp + shared object prefix situations.
-type MetricValuePairs struct {
-	// Metric / value pairs.
-	Metrics              map[string]float32 `protobuf:"bytes,1,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *MetricValuePairs) Reset()         { *m = MetricValuePairs{} }
-func (m *MetricValuePairs) String() string { return proto.CompactTextString(m) }
-func (*MetricValuePairs) ProtoMessage()    {}
-func (*MetricValuePairs) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{4}
-}
-
-func (m *MetricValuePairs) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MetricValuePairs.Unmarshal(m, b)
-}
-func (m *MetricValuePairs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MetricValuePairs.Marshal(b, m, deterministic)
-}
-func (m *MetricValuePairs) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MetricValuePairs.Merge(m, src)
-}
-func (m *MetricValuePairs) XXX_Size() int {
-	return xxx_messageInfo_MetricValuePairs.Size(m)
-}
-func (m *MetricValuePairs) XXX_DiscardUnknown() {
-	xxx_messageInfo_MetricValuePairs.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MetricValuePairs proto.InternalMessageInfo
-
-func (m *MetricValuePairs) GetMetrics() map[string]float32 {
-	if m != nil {
-		return m.Metrics
-	}
-	return nil
-}
-
-//
-// Struct to group metadata for a metric (or group of metrics) with the key-value
-// pairs of collected metrics
-type MetricInformation struct {
-	Metadata             *MetricMetaData    `protobuf:"bytes,1,opt,name=metadata,proto3" json:"metadata,omitempty"`
-	Metrics              map[string]float32 `protobuf:"bytes,2,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *MetricInformation) Reset()         { *m = MetricInformation{} }
-func (m *MetricInformation) String() string { return proto.CompactTextString(m) }
-func (*MetricInformation) ProtoMessage()    {}
-func (*MetricInformation) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{5}
-}
-
-func (m *MetricInformation) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MetricInformation.Unmarshal(m, b)
-}
-func (m *MetricInformation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MetricInformation.Marshal(b, m, deterministic)
-}
-func (m *MetricInformation) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MetricInformation.Merge(m, src)
-}
-func (m *MetricInformation) XXX_Size() int {
-	return xxx_messageInfo_MetricInformation.Size(m)
-}
-func (m *MetricInformation) XXX_DiscardUnknown() {
-	xxx_messageInfo_MetricInformation.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MetricInformation proto.InternalMessageInfo
-
-func (m *MetricInformation) GetMetadata() *MetricMetaData {
-	if m != nil {
-		return m.Metadata
-	}
-	return nil
-}
-
-func (m *MetricInformation) GetMetrics() map[string]float32 {
-	if m != nil {
-		return m.Metrics
-	}
-	return nil
-}
-
-//
-// Legacy KPI Event structured.  In mid-August, the KPI event format was updated
-//                               to a more easily parsable format. See VOL-1140
-//                               for more information.
-type KpiEvent struct {
-	Type                 KpiEventType_Types           `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.KpiEventType_Types" json:"type,omitempty"`
-	Ts                   float32                      `protobuf:"fixed32,2,opt,name=ts,proto3" json:"ts,omitempty"`
-	Prefixes             map[string]*MetricValuePairs `protobuf:"bytes,3,rep,name=prefixes,proto3" json:"prefixes,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 *KpiEvent) Reset()         { *m = KpiEvent{} }
-func (m *KpiEvent) String() string { return proto.CompactTextString(m) }
-func (*KpiEvent) ProtoMessage()    {}
-func (*KpiEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{6}
-}
-
-func (m *KpiEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_KpiEvent.Unmarshal(m, b)
-}
-func (m *KpiEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_KpiEvent.Marshal(b, m, deterministic)
-}
-func (m *KpiEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_KpiEvent.Merge(m, src)
-}
-func (m *KpiEvent) XXX_Size() int {
-	return xxx_messageInfo_KpiEvent.Size(m)
-}
-func (m *KpiEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_KpiEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_KpiEvent proto.InternalMessageInfo
-
-func (m *KpiEvent) GetType() KpiEventType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return KpiEventType_slice
-}
-
-func (m *KpiEvent) GetTs() float32 {
-	if m != nil {
-		return m.Ts
-	}
-	return 0
-}
-
-func (m *KpiEvent) GetPrefixes() map[string]*MetricValuePairs {
-	if m != nil {
-		return m.Prefixes
-	}
-	return nil
-}
-
-type KpiEvent2 struct {
-	// Type of KPI Event
-	Type KpiEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.KpiEventType_Types" json:"type,omitempty"`
-	// Fields used when for slice:
-	Ts                   float64              `protobuf:"fixed64,2,opt,name=ts,proto3" json:"ts,omitempty"`
-	SliceData            []*MetricInformation `protobuf:"bytes,3,rep,name=slice_data,json=sliceData,proto3" json:"slice_data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *KpiEvent2) Reset()         { *m = KpiEvent2{} }
-func (m *KpiEvent2) String() string { return proto.CompactTextString(m) }
-func (*KpiEvent2) ProtoMessage()    {}
-func (*KpiEvent2) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{7}
-}
-
-func (m *KpiEvent2) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_KpiEvent2.Unmarshal(m, b)
-}
-func (m *KpiEvent2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_KpiEvent2.Marshal(b, m, deterministic)
-}
-func (m *KpiEvent2) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_KpiEvent2.Merge(m, src)
-}
-func (m *KpiEvent2) XXX_Size() int {
-	return xxx_messageInfo_KpiEvent2.Size(m)
-}
-func (m *KpiEvent2) XXX_DiscardUnknown() {
-	xxx_messageInfo_KpiEvent2.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_KpiEvent2 proto.InternalMessageInfo
-
-func (m *KpiEvent2) GetType() KpiEventType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return KpiEventType_slice
-}
-
-func (m *KpiEvent2) GetTs() float64 {
-	if m != nil {
-		return m.Ts
-	}
-	return 0
-}
-
-func (m *KpiEvent2) GetSliceData() []*MetricInformation {
-	if m != nil {
-		return m.SliceData
-	}
-	return nil
-}
-
-//
-// Describes the events specific to device
-type DeviceEvent struct {
-	// Identifier of the originating resource of the event, for ex: device_id
-	ResourceId string `protobuf:"bytes,1,opt,name=resource_id,json=resourceId,proto3" json:"resource_id,omitempty"`
-	// device_event_name indicates clearly the name of the device event
-	DeviceEventName string `protobuf:"bytes,2,opt,name=device_event_name,json=deviceEventName,proto3" json:"device_event_name,omitempty"`
-	// Textual explanation of the device event
-	Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"`
-	// Key/Value storage for extra information that may give context to the event
-	Context              map[string]string `protobuf:"bytes,4,rep,name=context,proto3" json:"context,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 *DeviceEvent) Reset()         { *m = DeviceEvent{} }
-func (m *DeviceEvent) String() string { return proto.CompactTextString(m) }
-func (*DeviceEvent) ProtoMessage()    {}
-func (*DeviceEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{8}
-}
-
-func (m *DeviceEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceEvent.Unmarshal(m, b)
-}
-func (m *DeviceEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceEvent.Marshal(b, m, deterministic)
-}
-func (m *DeviceEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceEvent.Merge(m, src)
-}
-func (m *DeviceEvent) XXX_Size() int {
-	return xxx_messageInfo_DeviceEvent.Size(m)
-}
-func (m *DeviceEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceEvent proto.InternalMessageInfo
-
-func (m *DeviceEvent) GetResourceId() string {
-	if m != nil {
-		return m.ResourceId
-	}
-	return ""
-}
-
-func (m *DeviceEvent) GetDeviceEventName() string {
-	if m != nil {
-		return m.DeviceEventName
-	}
-	return ""
-}
-
-func (m *DeviceEvent) GetDescription() string {
-	if m != nil {
-		return m.Description
-	}
-	return ""
-}
-
-func (m *DeviceEvent) GetContext() map[string]string {
-	if m != nil {
-		return m.Context
-	}
-	return nil
-}
-
-//
-// Describes the events specific to an RPC request
-type RPCEvent struct {
-	// RPC name
-	Rpc string `protobuf:"bytes,1,opt,name=rpc,proto3" json:"rpc,omitempty"`
-	// The operation id of that request.  Can be a log correlation ID
-	OperationId string `protobuf:"bytes,2,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"`
-	// Identifies the service name originating the event
-	Service string `protobuf:"bytes,3,opt,name=service,proto3" json:"service,omitempty"`
-	// Identifies the stack originating the event
-	StackId string `protobuf:"bytes,4,opt,name=stack_id,json=stackId,proto3" json:"stack_id,omitempty"`
-	// Identifies the resource upon which the action is taken, e.g. device_id
-	ResourceId string `protobuf:"bytes,5,opt,name=resource_id,json=resourceId,proto3" json:"resource_id,omitempty"`
-	// Textual explanation of the event
-	Description string `protobuf:"bytes,6,opt,name=description,proto3" json:"description,omitempty"`
-	// Key/Value storage for extra information that may give context to the event
-	Context map[string]string `protobuf:"bytes,7,rep,name=context,proto3" json:"context,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	// Status of the RPC Event
-	Status               *common.OperationResp `protobuf:"bytes,8,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *RPCEvent) Reset()         { *m = RPCEvent{} }
-func (m *RPCEvent) String() string { return proto.CompactTextString(m) }
-func (*RPCEvent) ProtoMessage()    {}
-func (*RPCEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{9}
-}
-
-func (m *RPCEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RPCEvent.Unmarshal(m, b)
-}
-func (m *RPCEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RPCEvent.Marshal(b, m, deterministic)
-}
-func (m *RPCEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RPCEvent.Merge(m, src)
-}
-func (m *RPCEvent) XXX_Size() int {
-	return xxx_messageInfo_RPCEvent.Size(m)
-}
-func (m *RPCEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_RPCEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RPCEvent proto.InternalMessageInfo
-
-func (m *RPCEvent) GetRpc() string {
-	if m != nil {
-		return m.Rpc
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetOperationId() string {
-	if m != nil {
-		return m.OperationId
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetService() string {
-	if m != nil {
-		return m.Service
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetStackId() string {
-	if m != nil {
-		return m.StackId
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetResourceId() string {
-	if m != nil {
-		return m.ResourceId
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetDescription() string {
-	if m != nil {
-		return m.Description
-	}
-	return ""
-}
-
-func (m *RPCEvent) GetContext() map[string]string {
-	if m != nil {
-		return m.Context
-	}
-	return nil
-}
-
-func (m *RPCEvent) GetStatus() *common.OperationResp {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-//
-// Identify the area of the system impacted by the event.
-type EventCategory struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EventCategory) Reset()         { *m = EventCategory{} }
-func (m *EventCategory) String() string { return proto.CompactTextString(m) }
-func (*EventCategory) ProtoMessage()    {}
-func (*EventCategory) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{10}
-}
-
-func (m *EventCategory) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventCategory.Unmarshal(m, b)
-}
-func (m *EventCategory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventCategory.Marshal(b, m, deterministic)
-}
-func (m *EventCategory) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventCategory.Merge(m, src)
-}
-func (m *EventCategory) XXX_Size() int {
-	return xxx_messageInfo_EventCategory.Size(m)
-}
-func (m *EventCategory) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventCategory.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventCategory proto.InternalMessageInfo
-
-//
-// Identify the functional category originating the event
-type EventSubCategory struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EventSubCategory) Reset()         { *m = EventSubCategory{} }
-func (m *EventSubCategory) String() string { return proto.CompactTextString(m) }
-func (*EventSubCategory) ProtoMessage()    {}
-func (*EventSubCategory) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{11}
-}
-
-func (m *EventSubCategory) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventSubCategory.Unmarshal(m, b)
-}
-func (m *EventSubCategory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventSubCategory.Marshal(b, m, deterministic)
-}
-func (m *EventSubCategory) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventSubCategory.Merge(m, src)
-}
-func (m *EventSubCategory) XXX_Size() int {
-	return xxx_messageInfo_EventSubCategory.Size(m)
-}
-func (m *EventSubCategory) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventSubCategory.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventSubCategory proto.InternalMessageInfo
-
-//
-// Identify the type of event
-type EventType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EventType) Reset()         { *m = EventType{} }
-func (m *EventType) String() string { return proto.CompactTextString(m) }
-func (*EventType) ProtoMessage()    {}
-func (*EventType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{12}
-}
-
-func (m *EventType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventType.Unmarshal(m, b)
-}
-func (m *EventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventType.Marshal(b, m, deterministic)
-}
-func (m *EventType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventType.Merge(m, src)
-}
-func (m *EventType) XXX_Size() int {
-	return xxx_messageInfo_EventType.Size(m)
-}
-func (m *EventType) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventType proto.InternalMessageInfo
-
-//
-// Identify the functional category originating the event
-type EventHeader struct {
-	// Unique ID for this event.  e.g. voltha.some_olt.1234
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Refers to the functional area affect by the event
-	Category EventCategory_Types `protobuf:"varint,2,opt,name=category,proto3,enum=voltha.EventCategory_Types" json:"category,omitempty"`
-	// Refers to functional category of the event
-	SubCategory EventSubCategory_Types `protobuf:"varint,3,opt,name=sub_category,json=subCategory,proto3,enum=voltha.EventSubCategory_Types" json:"sub_category,omitempty"`
-	// Refers to the type of the event
-	Type EventType_Types `protobuf:"varint,4,opt,name=type,proto3,enum=voltha.EventType_Types" json:"type,omitempty"`
-	// The version identifier for this event type, thus allowing each
-	// event type to evolve independently. The version should be in the
-	// format “MAJOR.MINOR” format and minor changes must only be additive
-	// and non-breaking.
-	TypeVersion string `protobuf:"bytes,5,opt,name=type_version,json=typeVersion,proto3" json:"type_version,omitempty"`
-	// Timestamp at which the event was first raised.
-	// This represents the UTC time stamp since epoch (in seconds) when the
-	// the event was first raised from the source entity.
-	// If the source entity doesn't send the raised_ts, this shall be set
-	// to timestamp when the event was received.
-	RaisedTs *timestamp.Timestamp `protobuf:"bytes,6,opt,name=raised_ts,json=raisedTs,proto3" json:"raised_ts,omitempty"`
-	// Timestamp at which the event was reported.
-	// This represents the UTC time stamp since epoch (in seconds) when the
-	// the event was reported (this time stamp is >= raised_ts).
-	// If the source entity that reported this event doesn't send the
-	// reported_ts, this shall be set to the same value as raised_ts.
-	ReportedTs           *timestamp.Timestamp `protobuf:"bytes,7,opt,name=reported_ts,json=reportedTs,proto3" json:"reported_ts,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *EventHeader) Reset()         { *m = EventHeader{} }
-func (m *EventHeader) String() string { return proto.CompactTextString(m) }
-func (*EventHeader) ProtoMessage()    {}
-func (*EventHeader) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{13}
-}
-
-func (m *EventHeader) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventHeader.Unmarshal(m, b)
-}
-func (m *EventHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventHeader.Marshal(b, m, deterministic)
-}
-func (m *EventHeader) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventHeader.Merge(m, src)
-}
-func (m *EventHeader) XXX_Size() int {
-	return xxx_messageInfo_EventHeader.Size(m)
-}
-func (m *EventHeader) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventHeader.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventHeader proto.InternalMessageInfo
-
-func (m *EventHeader) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *EventHeader) GetCategory() EventCategory_Types {
-	if m != nil {
-		return m.Category
-	}
-	return EventCategory_COMMUNICATION
-}
-
-func (m *EventHeader) GetSubCategory() EventSubCategory_Types {
-	if m != nil {
-		return m.SubCategory
-	}
-	return EventSubCategory_PON
-}
-
-func (m *EventHeader) GetType() EventType_Types {
-	if m != nil {
-		return m.Type
-	}
-	return EventType_CONFIG_EVENT
-}
-
-func (m *EventHeader) GetTypeVersion() string {
-	if m != nil {
-		return m.TypeVersion
-	}
-	return ""
-}
-
-func (m *EventHeader) GetRaisedTs() *timestamp.Timestamp {
-	if m != nil {
-		return m.RaisedTs
-	}
-	return nil
-}
-
-func (m *EventHeader) GetReportedTs() *timestamp.Timestamp {
-	if m != nil {
-		return m.ReportedTs
-	}
-	return nil
-}
-
-//
-// Event Structure
-type Event struct {
-	// event header
-	Header *EventHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
-	// oneof event types referred by EventType.
-	//
-	// Types that are valid to be assigned to EventType:
-	//	*Event_ConfigEvent
-	//	*Event_KpiEvent
-	//	*Event_KpiEvent2
-	//	*Event_DeviceEvent
-	//	*Event_RpcEvent
-	EventType            isEvent_EventType `protobuf_oneof:"event_type"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *Event) Reset()         { *m = Event{} }
-func (m *Event) String() string { return proto.CompactTextString(m) }
-func (*Event) ProtoMessage()    {}
-func (*Event) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e63e6c07044fd2c4, []int{14}
-}
-
-func (m *Event) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Event.Unmarshal(m, b)
-}
-func (m *Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Event.Marshal(b, m, deterministic)
-}
-func (m *Event) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Event.Merge(m, src)
-}
-func (m *Event) XXX_Size() int {
-	return xxx_messageInfo_Event.Size(m)
-}
-func (m *Event) XXX_DiscardUnknown() {
-	xxx_messageInfo_Event.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Event proto.InternalMessageInfo
-
-func (m *Event) GetHeader() *EventHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type isEvent_EventType interface {
-	isEvent_EventType()
-}
-
-type Event_ConfigEvent struct {
-	ConfigEvent *ConfigEvent `protobuf:"bytes,2,opt,name=config_event,json=configEvent,proto3,oneof"`
-}
-
-type Event_KpiEvent struct {
-	KpiEvent *KpiEvent `protobuf:"bytes,3,opt,name=kpi_event,json=kpiEvent,proto3,oneof"`
-}
-
-type Event_KpiEvent2 struct {
-	KpiEvent2 *KpiEvent2 `protobuf:"bytes,4,opt,name=kpi_event2,json=kpiEvent2,proto3,oneof"`
-}
-
-type Event_DeviceEvent struct {
-	DeviceEvent *DeviceEvent `protobuf:"bytes,5,opt,name=device_event,json=deviceEvent,proto3,oneof"`
-}
-
-type Event_RpcEvent struct {
-	RpcEvent *RPCEvent `protobuf:"bytes,6,opt,name=rpc_event,json=rpcEvent,proto3,oneof"`
-}
-
-func (*Event_ConfigEvent) isEvent_EventType() {}
-
-func (*Event_KpiEvent) isEvent_EventType() {}
-
-func (*Event_KpiEvent2) isEvent_EventType() {}
-
-func (*Event_DeviceEvent) isEvent_EventType() {}
-
-func (*Event_RpcEvent) isEvent_EventType() {}
-
-func (m *Event) GetEventType() isEvent_EventType {
-	if m != nil {
-		return m.EventType
-	}
-	return nil
-}
-
-func (m *Event) GetConfigEvent() *ConfigEvent {
-	if x, ok := m.GetEventType().(*Event_ConfigEvent); ok {
-		return x.ConfigEvent
-	}
-	return nil
-}
-
-func (m *Event) GetKpiEvent() *KpiEvent {
-	if x, ok := m.GetEventType().(*Event_KpiEvent); ok {
-		return x.KpiEvent
-	}
-	return nil
-}
-
-func (m *Event) GetKpiEvent2() *KpiEvent2 {
-	if x, ok := m.GetEventType().(*Event_KpiEvent2); ok {
-		return x.KpiEvent2
-	}
-	return nil
-}
-
-func (m *Event) GetDeviceEvent() *DeviceEvent {
-	if x, ok := m.GetEventType().(*Event_DeviceEvent); ok {
-		return x.DeviceEvent
-	}
-	return nil
-}
-
-func (m *Event) GetRpcEvent() *RPCEvent {
-	if x, ok := m.GetEventType().(*Event_RpcEvent); ok {
-		return x.RpcEvent
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*Event) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*Event_ConfigEvent)(nil),
-		(*Event_KpiEvent)(nil),
-		(*Event_KpiEvent2)(nil),
-		(*Event_DeviceEvent)(nil),
-		(*Event_RpcEvent)(nil),
-	}
-}
-
-func init() {
-	proto.RegisterEnum("voltha.ConfigEventType_Types", ConfigEventType_Types_name, ConfigEventType_Types_value)
-	proto.RegisterEnum("voltha.KpiEventType_Types", KpiEventType_Types_name, KpiEventType_Types_value)
-	proto.RegisterEnum("voltha.EventCategory_Types", EventCategory_Types_name, EventCategory_Types_value)
-	proto.RegisterEnum("voltha.EventSubCategory_Types", EventSubCategory_Types_name, EventSubCategory_Types_value)
-	proto.RegisterEnum("voltha.EventType_Types", EventType_Types_name, EventType_Types_value)
-	proto.RegisterType((*ConfigEventType)(nil), "voltha.ConfigEventType")
-	proto.RegisterType((*ConfigEvent)(nil), "voltha.ConfigEvent")
-	proto.RegisterType((*KpiEventType)(nil), "voltha.KpiEventType")
-	proto.RegisterType((*MetricMetaData)(nil), "voltha.MetricMetaData")
-	proto.RegisterMapType((map[string]string)(nil), "voltha.MetricMetaData.ContextEntry")
-	proto.RegisterType((*MetricValuePairs)(nil), "voltha.MetricValuePairs")
-	proto.RegisterMapType((map[string]float32)(nil), "voltha.MetricValuePairs.MetricsEntry")
-	proto.RegisterType((*MetricInformation)(nil), "voltha.MetricInformation")
-	proto.RegisterMapType((map[string]float32)(nil), "voltha.MetricInformation.MetricsEntry")
-	proto.RegisterType((*KpiEvent)(nil), "voltha.KpiEvent")
-	proto.RegisterMapType((map[string]*MetricValuePairs)(nil), "voltha.KpiEvent.PrefixesEntry")
-	proto.RegisterType((*KpiEvent2)(nil), "voltha.KpiEvent2")
-	proto.RegisterType((*DeviceEvent)(nil), "voltha.DeviceEvent")
-	proto.RegisterMapType((map[string]string)(nil), "voltha.DeviceEvent.ContextEntry")
-	proto.RegisterType((*RPCEvent)(nil), "voltha.RPCEvent")
-	proto.RegisterMapType((map[string]string)(nil), "voltha.RPCEvent.ContextEntry")
-	proto.RegisterType((*EventCategory)(nil), "voltha.EventCategory")
-	proto.RegisterType((*EventSubCategory)(nil), "voltha.EventSubCategory")
-	proto.RegisterType((*EventType)(nil), "voltha.EventType")
-	proto.RegisterType((*EventHeader)(nil), "voltha.EventHeader")
-	proto.RegisterType((*Event)(nil), "voltha.Event")
-}
-
-func init() { proto.RegisterFile("voltha_protos/events.proto", fileDescriptor_e63e6c07044fd2c4) }
-
-var fileDescriptor_e63e6c07044fd2c4 = []byte{
-	// 1282 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xdb, 0x6e, 0xdb, 0x46,
-	0x13, 0x16, 0xa9, 0xf3, 0x50, 0xb6, 0xe9, 0xcd, 0xff, 0xb7, 0x8a, 0xd2, 0x26, 0x8e, 0x8a, 0x16,
-	0x46, 0x82, 0x50, 0x28, 0x5b, 0x20, 0x86, 0x83, 0x1e, 0x12, 0x85, 0x8d, 0x89, 0xd4, 0x94, 0x4a,
-	0xcb, 0x06, 0xd2, 0x1b, 0x61, 0x4d, 0xae, 0x65, 0xc2, 0x92, 0x48, 0x70, 0x57, 0x42, 0xfc, 0x00,
-	0xbd, 0xee, 0x83, 0xf4, 0x39, 0x7a, 0xd7, 0x37, 0x28, 0x8a, 0xbe, 0x44, 0x1f, 0xa0, 0xd8, 0x03,
-	0x25, 0x52, 0x71, 0x90, 0x0b, 0xa3, 0x57, 0xdc, 0x9d, 0x9d, 0x6f, 0xe7, 0x9b, 0x8f, 0x3b, 0xb3,
-	0x0b, 0x9d, 0x65, 0x3c, 0x65, 0x97, 0x78, 0x9c, 0xa4, 0x31, 0x8b, 0x69, 0x8f, 0x2c, 0xc9, 0x9c,
-	0x51, 0x4b, 0xcc, 0x50, 0x4d, 0xae, 0x75, 0xda, 0x45, 0x9f, 0x19, 0x61, 0x58, 0x7a, 0x74, 0x3e,
-	0x99, 0xc4, 0xf1, 0x64, 0x4a, 0x7a, 0x38, 0x89, 0x7a, 0x78, 0x3e, 0x8f, 0x19, 0x66, 0x51, 0x3c,
-	0x57, 0xf8, 0xce, 0x03, 0xb5, 0x2a, 0x66, 0xe7, 0x8b, 0x8b, 0x1e, 0x8b, 0x66, 0x84, 0x32, 0x3c,
-	0x4b, 0x94, 0xc3, 0x46, 0xf0, 0x20, 0x9e, 0xcd, 0xe2, 0xb9, 0x5c, 0xeb, 0x3e, 0x83, 0x9d, 0x7e,
-	0x3c, 0xbf, 0x88, 0x26, 0x0e, 0xa7, 0x34, 0xba, 0x4e, 0x48, 0x77, 0x1f, 0xaa, 0xfc, 0x4b, 0x51,
-	0x1d, 0xca, 0x38, 0x0c, 0xcd, 0x12, 0x02, 0xa8, 0xa5, 0x64, 0x16, 0x2f, 0x89, 0xa9, 0xf1, 0xf1,
-	0x22, 0x09, 0x31, 0x23, 0xa6, 0xde, 0xbd, 0x04, 0x23, 0x07, 0x46, 0x5f, 0x42, 0x85, 0x5d, 0x27,
-	0xa4, 0xad, 0xed, 0x69, 0xfb, 0xdb, 0xf6, 0xa7, 0x96, 0x0c, 0x6b, 0x6d, 0xec, 0x6f, 0x89, 0xcd,
-	0x7d, 0xe1, 0x8a, 0x10, 0x54, 0x2e, 0x31, 0xbd, 0x6c, 0xeb, 0x7b, 0xda, 0x7e, 0xd3, 0x17, 0x63,
-	0x6e, 0x0b, 0x31, 0xc3, 0xed, 0xb2, 0xb4, 0xf1, 0x71, 0xf7, 0x11, 0xb4, 0x5e, 0x27, 0xd1, 0x9a,
-	0x63, 0x27, 0xe3, 0xd8, 0x84, 0x2a, 0x9d, 0x46, 0x01, 0x31, 0x4b, 0xa8, 0x06, 0x3a, 0xa3, 0xa6,
-	0xd6, 0xfd, 0x4d, 0x87, 0xed, 0x63, 0xc2, 0xd2, 0x28, 0x38, 0x26, 0x0c, 0xbf, 0xc4, 0x0c, 0xa3,
-	0xff, 0x41, 0x95, 0x45, 0x6c, 0x2a, 0xa9, 0x35, 0x7d, 0x39, 0x41, 0xdb, 0x1c, 0x20, 0x42, 0x6b,
-	0xbe, 0xce, 0x28, 0x7a, 0x04, 0xbb, 0xd3, 0x78, 0x12, 0x05, 0x78, 0x3a, 0x0e, 0xc9, 0x32, 0x0a,
-	0xc8, 0x38, 0x0a, 0x15, 0x8b, 0x1d, 0xb5, 0xf0, 0x52, 0xd8, 0xdd, 0x10, 0xdd, 0x83, 0x26, 0x25,
-	0x69, 0x84, 0xa7, 0xe3, 0x79, 0xdc, 0xae, 0x08, 0x9f, 0x86, 0x34, 0x78, 0x31, 0x5f, 0x5c, 0x6f,
-	0x50, 0x95, 0x8b, 0x61, 0x86, 0xfc, 0x06, 0xea, 0x41, 0x3c, 0x67, 0xe4, 0x2d, 0x6b, 0xd7, 0xf6,
-	0xca, 0xfb, 0x86, 0xfd, 0x59, 0x26, 0x54, 0x91, 0x34, 0xd7, 0x8d, 0x7b, 0x39, 0x73, 0x96, 0x5e,
-	0xfb, 0x19, 0x86, 0xab, 0xb3, 0x58, 0x44, 0x61, 0xbb, 0x2e, 0xd5, 0xe1, 0xe3, 0xce, 0x21, 0xb4,
-	0xf2, 0xce, 0xc8, 0x84, 0xf2, 0x15, 0xb9, 0x56, 0xc9, 0xf2, 0x21, 0x17, 0x60, 0x89, 0xa7, 0x0b,
-	0xa2, 0x84, 0x96, 0x93, 0x43, 0xfd, 0x40, 0xeb, 0xfe, 0xaa, 0x81, 0x29, 0x03, 0x9f, 0x71, 0xdb,
-	0x10, 0x47, 0x29, 0x45, 0xdf, 0x41, 0x7d, 0x26, 0x6c, 0xb4, 0xad, 0x09, 0x8e, 0x9f, 0x17, 0x39,
-	0xae, 0x5d, 0x95, 0x81, 0x2a, 0x96, 0x0a, 0xc5, 0x19, 0xe5, 0x17, 0x3e, 0xc4, 0x48, 0xcf, 0x33,
-	0xfa, 0x5d, 0x83, 0x5d, 0x09, 0x76, 0xe7, 0x17, 0x71, 0x3a, 0x13, 0x87, 0x1d, 0xd9, 0xd0, 0xe0,
-	0x15, 0x21, 0x4e, 0x06, 0xdf, 0xc6, 0xb0, 0x3f, 0xba, 0x59, 0x37, 0x7f, 0xe5, 0x87, 0xbe, 0x5f,
-	0xa7, 0xa1, 0x8b, 0x34, 0xbe, 0x28, 0x42, 0x72, 0xfb, 0xff, 0x07, 0x79, 0xfc, 0xa5, 0x41, 0x23,
-	0x3b, 0xb4, 0xc8, 0x2a, 0xd4, 0x46, 0x27, 0xe3, 0x91, 0x3f, 0xd4, 0x85, 0xc2, 0x58, 0x9f, 0x4d,
-	0x5d, 0x9c, 0xcd, 0x43, 0x68, 0x24, 0x29, 0xb9, 0x88, 0xde, 0x12, 0xda, 0x2e, 0x8b, 0x5c, 0xee,
-	0x6f, 0xee, 0x61, 0x0d, 0x95, 0x83, 0xcc, 0x61, 0xe5, 0xdf, 0x39, 0x85, 0xad, 0xc2, 0xd2, 0x0d,
-	0x59, 0x58, 0xf9, 0x2c, 0x0c, 0xbb, 0xfd, 0xbe, 0xdf, 0x9d, 0xcf, 0xef, 0x17, 0x0d, 0x9a, 0x59,
-	0x6c, 0xfb, 0x16, 0x09, 0xca, 0xe2, 0x3b, 0x00, 0x10, 0x85, 0x3c, 0x56, 0xb5, 0xcf, 0x53, 0xbc,
-	0xfb, 0xde, 0xdf, 0xe5, 0x37, 0x85, 0x33, 0xff, 0xdf, 0xdd, 0x7f, 0x34, 0x30, 0x64, 0x5d, 0x4a,
-	0xa9, 0x1f, 0x80, 0x91, 0x12, 0x1a, 0x2f, 0x52, 0x59, 0x7f, 0x32, 0x4b, 0xc8, 0x4c, 0x6e, 0xc8,
-	0xeb, 0x5c, 0x95, 0xa7, 0xe8, 0xc3, 0xe3, 0x39, 0x9e, 0x65, 0x85, 0xb1, 0x13, 0xae, 0x37, 0xf2,
-	0xf0, 0x8c, 0xa0, 0x3d, 0x30, 0x42, 0x42, 0x83, 0x34, 0x4a, 0x78, 0x58, 0xd5, 0x0d, 0xf2, 0x26,
-	0x74, 0xb8, 0xae, 0xe7, 0x8a, 0x60, 0xbd, 0x97, 0xb1, 0xce, 0x91, 0xba, 0xb9, 0x98, 0x6f, 0x55,
-	0xb8, 0x7f, 0xea, 0xd0, 0xf0, 0x87, 0x7d, 0x99, 0xb3, 0x09, 0xe5, 0x34, 0x09, 0x32, 0x60, 0x9a,
-	0x04, 0xe8, 0x21, 0xb4, 0xe2, 0x84, 0xa4, 0x42, 0x2d, 0x2e, 0x83, 0xc4, 0x1b, 0x2b, 0x9b, 0x1b,
-	0xa2, 0x36, 0xd4, 0x29, 0x49, 0x39, 0x47, 0x95, 0x57, 0x36, 0x45, 0x77, 0xa1, 0x41, 0x19, 0x0e,
-	0xae, 0x38, 0xb0, 0xa2, 0x96, 0xf8, 0xdc, 0x0d, 0x37, 0xd5, 0xad, 0xbe, 0xa3, 0xee, 0x86, 0x62,
-	0xb5, 0x77, 0x15, 0x7b, 0xba, 0x56, 0xac, 0x2e, 0x14, 0x5b, 0x5d, 0x15, 0x59, 0x3e, 0xef, 0xe9,
-	0x7d, 0x4f, 0xa0, 0x46, 0x19, 0x66, 0x0b, 0xda, 0x6e, 0x88, 0x63, 0xfa, 0x7f, 0x4b, 0xdd, 0x65,
-	0x83, 0x2c, 0x2b, 0x9f, 0xd0, 0xc4, 0x57, 0x4e, 0xb7, 0x52, 0x77, 0x09, 0x5b, 0x82, 0x49, 0x1f,
-	0x33, 0x32, 0x89, 0xd3, 0xeb, 0x2e, 0xc9, 0x6e, 0x9c, 0x5d, 0xd8, 0xea, 0x0f, 0x8e, 0x8f, 0x4f,
-	0x3d, 0xb7, 0xff, 0x7c, 0xe4, 0x0e, 0x3c, 0xb3, 0x84, 0x76, 0xc0, 0x70, 0xbc, 0x33, 0xd7, 0x1f,
-	0x78, 0xc7, 0x8e, 0x37, 0x32, 0x35, 0xb4, 0x05, 0x4d, 0xe7, 0xa7, 0x53, 0x77, 0x28, 0xa6, 0x3a,
-	0x32, 0xa0, 0x7e, 0xe2, 0xf8, 0x67, 0x6e, 0xdf, 0x31, 0xcb, 0x68, 0x1b, 0x60, 0xe8, 0x0f, 0xfa,
-	0xce, 0xc9, 0x89, 0xeb, 0xbd, 0x32, 0x2b, 0xa8, 0x05, 0x8d, 0x13, 0xa7, 0x7f, 0xea, 0xbb, 0xa3,
-	0x37, 0x66, 0xb5, 0xeb, 0x83, 0x29, 0xe2, 0x9e, 0x2c, 0xce, 0x57, 0xa1, 0xbf, 0xcd, 0x5d, 0xc8,
-	0x43, 0x11, 0xb0, 0x0e, 0xe5, 0xc1, 0x8f, 0x3c, 0x10, 0x1f, 0x88, 0x10, 0x62, 0x70, 0x6a, 0x96,
-	0xf9, 0xc0, 0xf3, 0x5c, 0xb3, 0x82, 0x1a, 0x50, 0xf1, 0x06, 0x9e, 0x63, 0x56, 0xbb, 0x17, 0xd0,
-	0x5c, 0xdf, 0x9c, 0x6f, 0xb2, 0xcd, 0x4c, 0x68, 0xf5, 0x07, 0xde, 0x0f, 0xee, 0xab, 0xb1, 0x73,
-	0xc6, 0x69, 0x96, 0x38, 0xeb, 0xd7, 0x43, 0x57, 0x4d, 0x35, 0x4e, 0x74, 0x35, 0xb5, 0x4d, 0x9d,
-	0x03, 0x5e, 0x3a, 0x3c, 0x09, 0xe5, 0x51, 0xe6, 0x00, 0x7f, 0xd8, 0x57, 0xd3, 0x4a, 0xf7, 0x6f,
-	0x1d, 0x0c, 0x11, 0xe8, 0x88, 0xe0, 0x90, 0xa4, 0xbc, 0xc4, 0x57, 0xf5, 0xa7, 0x47, 0x21, 0x7a,
-	0x0a, 0x8d, 0x40, 0xe5, 0x24, 0x04, 0xdf, 0xb6, 0xef, 0x65, 0x3f, 0xbe, 0xa0, 0xb5, 0xea, 0x13,
-	0x2b, 0x67, 0xf4, 0x1c, 0x5a, 0x74, 0x71, 0x3e, 0x5e, 0x81, 0xcb, 0x02, 0x7c, 0xbf, 0x00, 0xce,
-	0x09, 0xa6, 0xf0, 0x06, 0x5d, 0x9b, 0xd0, 0x63, 0xd5, 0x9e, 0x2a, 0x02, 0xfa, 0x71, 0x01, 0xfa,
-	0x4e, 0x6f, 0x7a, 0x08, 0x2d, 0xfe, 0x1d, 0x2f, 0x49, 0x4a, 0xf9, 0x19, 0x96, 0x87, 0xdc, 0xe0,
-	0xb6, 0x33, 0x69, 0x42, 0x4f, 0xa1, 0x99, 0xe2, 0x88, 0x92, 0x70, 0xcc, 0xa8, 0x38, 0xe3, 0x86,
-	0xdd, 0xb1, 0xe4, 0x43, 0xcc, 0xca, 0x1e, 0x62, 0xd6, 0x28, 0x7b, 0x88, 0xf9, 0x0d, 0xe9, 0x3c,
-	0xa2, 0xe8, 0x19, 0xaf, 0x9f, 0x24, 0x4e, 0x99, 0x84, 0xd6, 0x3f, 0x08, 0x85, 0xcc, 0x7d, 0x44,
-	0xbb, 0x7f, 0xe8, 0x50, 0x95, 0x05, 0xff, 0x18, 0x6a, 0x97, 0x42, 0x65, 0x75, 0x19, 0xde, 0x29,
-	0x64, 0x24, 0x7f, 0x80, 0xaf, 0x5c, 0xd0, 0x01, 0xb4, 0x02, 0xf1, 0x08, 0x93, 0x0d, 0x4f, 0x35,
-	0xf9, 0x3b, 0x37, 0x3c, 0xd0, 0x8e, 0x4a, 0xbe, 0x11, 0xe4, 0x9e, 0x74, 0x3d, 0x68, 0x5e, 0x25,
-	0x91, 0x82, 0x95, 0x05, 0xcc, 0xdc, 0x6c, 0xed, 0x47, 0x25, 0xbf, 0x71, 0x95, 0xdd, 0x73, 0x36,
-	0xc0, 0x0a, 0x60, 0x0b, 0xb5, 0x0d, 0x7b, 0x77, 0x13, 0x61, 0x1f, 0x95, 0xfc, 0xe6, 0xd5, 0xea,
-	0xea, 0x38, 0x80, 0x56, 0xbe, 0x1f, 0x0b, 0xb9, 0x73, 0xf4, 0x72, 0x6d, 0x94, 0xd3, 0xcb, 0x75,
-	0x68, 0x4e, 0x2f, 0x4d, 0x02, 0x05, 0xab, 0x15, 0xe9, 0x65, 0xbd, 0x84, 0xd3, 0x4b, 0x93, 0x40,
-	0x8c, 0x5f, 0xb4, 0x00, 0x64, 0xcf, 0xe7, 0xff, 0xf2, 0x85, 0x03, 0x77, 0xe2, 0x74, 0x62, 0xc5,
-	0x09, 0x99, 0x07, 0x71, 0x1a, 0x2a, 0xe4, 0xcf, 0xd6, 0x24, 0x62, 0x97, 0x8b, 0x73, 0xde, 0x5c,
-	0x7a, 0xd9, 0x5a, 0x4f, 0xae, 0x3d, 0x51, 0x6f, 0xe8, 0xe5, 0xd7, 0xbd, 0x49, 0xac, 0x6c, 0xe7,
-	0x35, 0x61, 0xfc, 0xea, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4e, 0xd5, 0xe2, 0x3b, 0xe5, 0x0b,
-	0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/health.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/health.pb.go
deleted file mode 100644
index 4bde873..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/health.pb.go
+++ /dev/null
@@ -1,210 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/health.proto
-
-package voltha
-
-import (
-	context "context"
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	empty "github.com/golang/protobuf/ptypes/empty"
-	_ "github.com/opencord/voltha-protos/v4/go/common"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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
-
-// Health states
-type HealthStatus_HealthState int32
-
-const (
-	HealthStatus_HEALTHY    HealthStatus_HealthState = 0
-	HealthStatus_OVERLOADED HealthStatus_HealthState = 1
-	HealthStatus_DYING      HealthStatus_HealthState = 2
-)
-
-var HealthStatus_HealthState_name = map[int32]string{
-	0: "HEALTHY",
-	1: "OVERLOADED",
-	2: "DYING",
-}
-
-var HealthStatus_HealthState_value = map[string]int32{
-	"HEALTHY":    0,
-	"OVERLOADED": 1,
-	"DYING":      2,
-}
-
-func (x HealthStatus_HealthState) String() string {
-	return proto.EnumName(HealthStatus_HealthState_name, int32(x))
-}
-
-func (HealthStatus_HealthState) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_dd1fc2b2d96d69b8, []int{0, 0}
-}
-
-// Encode health status of a Voltha instance
-type HealthStatus struct {
-	// Current state of health of this Voltha instance
-	State                HealthStatus_HealthState `protobuf:"varint,1,opt,name=state,proto3,enum=voltha.HealthStatus_HealthState" json:"state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
-}
-
-func (m *HealthStatus) Reset()         { *m = HealthStatus{} }
-func (m *HealthStatus) String() string { return proto.CompactTextString(m) }
-func (*HealthStatus) ProtoMessage()    {}
-func (*HealthStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_dd1fc2b2d96d69b8, []int{0}
-}
-
-func (m *HealthStatus) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HealthStatus.Unmarshal(m, b)
-}
-func (m *HealthStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HealthStatus.Marshal(b, m, deterministic)
-}
-func (m *HealthStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HealthStatus.Merge(m, src)
-}
-func (m *HealthStatus) XXX_Size() int {
-	return xxx_messageInfo_HealthStatus.Size(m)
-}
-func (m *HealthStatus) XXX_DiscardUnknown() {
-	xxx_messageInfo_HealthStatus.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HealthStatus proto.InternalMessageInfo
-
-func (m *HealthStatus) GetState() HealthStatus_HealthState {
-	if m != nil {
-		return m.State
-	}
-	return HealthStatus_HEALTHY
-}
-
-func init() {
-	proto.RegisterEnum("voltha.HealthStatus_HealthState", HealthStatus_HealthState_name, HealthStatus_HealthState_value)
-	proto.RegisterType((*HealthStatus)(nil), "voltha.HealthStatus")
-}
-
-func init() { proto.RegisterFile("voltha_protos/health.proto", fileDescriptor_dd1fc2b2d96d69b8) }
-
-var fileDescriptor_dd1fc2b2d96d69b8 = []byte{
-	// 302 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2a, 0xcb, 0xcf, 0x29,
-	0xc9, 0x48, 0x8c, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x2f, 0xd6, 0xcf, 0x48, 0x4d, 0xcc, 0x29, 0xc9,
-	0xd0, 0x03, 0xf3, 0x84, 0xd8, 0x20, 0x72, 0x52, 0x32, 0xe9, 0xf9, 0xf9, 0xe9, 0x39, 0xa9, 0xfa,
-	0x89, 0x05, 0x99, 0xfa, 0x89, 0x79, 0x79, 0xf9, 0x25, 0x89, 0x25, 0x99, 0xf9, 0x79, 0xc5, 0x10,
-	0x55, 0x52, 0xd2, 0x50, 0x59, 0x30, 0x2f, 0xa9, 0x34, 0x4d, 0x3f, 0x35, 0xb7, 0xa0, 0xa4, 0x12,
-	0x2a, 0x29, 0x81, 0x6a, 0x7c, 0x6e, 0x6a, 0x49, 0x22, 0x44, 0x46, 0xa9, 0x85, 0x91, 0x8b, 0xc7,
-	0x03, 0x6c, 0x5b, 0x70, 0x49, 0x62, 0x49, 0x69, 0xb1, 0x90, 0x2d, 0x17, 0x6b, 0x71, 0x49, 0x62,
-	0x49, 0xaa, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x9f, 0x91, 0x82, 0x1e, 0x44, 0xab, 0x1e, 0xb2, 0x22,
-	0x24, 0x4e, 0xaa, 0x13, 0xeb, 0x8b, 0x6f, 0x67, 0x65, 0x19, 0x83, 0x20, 0xba, 0x94, 0x4c, 0xb9,
-	0xb8, 0x91, 0x24, 0x85, 0xb8, 0xb9, 0xd8, 0x3d, 0x5c, 0x1d, 0x7d, 0x42, 0x3c, 0x22, 0x05, 0x18,
-	0x84, 0xf8, 0xb8, 0xb8, 0xfc, 0xc3, 0x5c, 0x83, 0x7c, 0xfc, 0x1d, 0x5d, 0x5c, 0x5d, 0x04, 0x18,
-	0x85, 0x38, 0xb9, 0x58, 0x5d, 0x22, 0x3d, 0xfd, 0xdc, 0x05, 0x98, 0x8c, 0x12, 0xb9, 0x78, 0xa1,
-	0xda, 0x52, 0x8b, 0xca, 0x32, 0x93, 0x53, 0x85, 0x02, 0xb8, 0xf8, 0xdd, 0x53, 0x4b, 0x50, 0x5c,
-	0x26, 0xa6, 0x07, 0xf1, 0xa2, 0x1e, 0xcc, 0x8b, 0x7a, 0xae, 0x20, 0x2f, 0x4a, 0x89, 0x60, 0x73,
-	0xa2, 0x12, 0x7f, 0xd3, 0xe5, 0x27, 0x93, 0x99, 0x38, 0x85, 0xd8, 0xa1, 0x81, 0xe9, 0xe4, 0xca,
-	0x25, 0x9c, 0x5f, 0x94, 0xae, 0x97, 0x5f, 0x90, 0x9a, 0x97, 0x9c, 0x5f, 0x94, 0x02, 0xd5, 0x14,
-	0xa5, 0x97, 0x9e, 0x59, 0x92, 0x51, 0x9a, 0xa4, 0x97, 0x9c, 0x9f, 0xab, 0x0f, 0x93, 0xd3, 0x87,
-	0xc8, 0xe9, 0x42, 0x83, 0xab, 0xcc, 0x44, 0x3f, 0x3d, 0x1f, 0x2a, 0x96, 0xc4, 0x06, 0x16, 0x34,
-	0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0x72, 0x0c, 0x5a, 0x9d, 0xb2, 0x01, 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
-
-// HealthServiceClient is the client API for HealthService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type HealthServiceClient interface {
-	// Return current health status of a Voltha instance
-	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*HealthStatus, error)
-}
-
-type healthServiceClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewHealthServiceClient(cc *grpc.ClientConn) HealthServiceClient {
-	return &healthServiceClient{cc}
-}
-
-func (c *healthServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*HealthStatus, error) {
-	out := new(HealthStatus)
-	err := c.cc.Invoke(ctx, "/voltha.HealthService/GetHealthStatus", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// HealthServiceServer is the server API for HealthService service.
-type HealthServiceServer interface {
-	// Return current health status of a Voltha instance
-	GetHealthStatus(context.Context, *empty.Empty) (*HealthStatus, error)
-}
-
-// UnimplementedHealthServiceServer can be embedded to have forward compatible implementations.
-type UnimplementedHealthServiceServer struct {
-}
-
-func (*UnimplementedHealthServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*HealthStatus, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
-}
-
-func RegisterHealthServiceServer(s *grpc.Server, srv HealthServiceServer) {
-	s.RegisterService(&_HealthService_serviceDesc, srv)
-}
-
-func _HealthService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(HealthServiceServer).GetHealthStatus(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.HealthService/GetHealthStatus",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(HealthServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _HealthService_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "voltha.HealthService",
-	HandlerType: (*HealthServiceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "GetHealthStatus",
-			Handler:    _HealthService_GetHealthStatus_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "voltha_protos/health.proto",
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/logical_device.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/logical_device.pb.go
deleted file mode 100644
index 5428ecc..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/logical_device.pb.go
+++ /dev/null
@@ -1,351 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/logical_device.proto
-
-package voltha
-
-import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	_ "github.com/opencord/voltha-protos/v4/go/common"
-	openflow_13 "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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 LogicalPortId struct {
-	// unique id of logical device
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// id of the port on the logical device
-	PortId               string   `protobuf:"bytes,2,opt,name=port_id,json=portId,proto3" json:"port_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *LogicalPortId) Reset()         { *m = LogicalPortId{} }
-func (m *LogicalPortId) String() string { return proto.CompactTextString(m) }
-func (*LogicalPortId) ProtoMessage()    {}
-func (*LogicalPortId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_caf139ab3abc8240, []int{0}
-}
-
-func (m *LogicalPortId) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalPortId.Unmarshal(m, b)
-}
-func (m *LogicalPortId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalPortId.Marshal(b, m, deterministic)
-}
-func (m *LogicalPortId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalPortId.Merge(m, src)
-}
-func (m *LogicalPortId) XXX_Size() int {
-	return xxx_messageInfo_LogicalPortId.Size(m)
-}
-func (m *LogicalPortId) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalPortId.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalPortId proto.InternalMessageInfo
-
-func (m *LogicalPortId) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *LogicalPortId) GetPortId() string {
-	if m != nil {
-		return m.PortId
-	}
-	return ""
-}
-
-type LogicalPort struct {
-	Id                   string                    `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	OfpPort              *openflow_13.OfpPort      `protobuf:"bytes,2,opt,name=ofp_port,json=ofpPort,proto3" json:"ofp_port,omitempty"`
-	DeviceId             string                    `protobuf:"bytes,3,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	DevicePortNo         uint32                    `protobuf:"varint,4,opt,name=device_port_no,json=devicePortNo,proto3" json:"device_port_no,omitempty"`
-	RootPort             bool                      `protobuf:"varint,5,opt,name=root_port,json=rootPort,proto3" json:"root_port,omitempty"`
-	OfpPortStats         *openflow_13.OfpPortStats `protobuf:"bytes,6,opt,name=ofp_port_stats,json=ofpPortStats,proto3" json:"ofp_port_stats,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
-}
-
-func (m *LogicalPort) Reset()         { *m = LogicalPort{} }
-func (m *LogicalPort) String() string { return proto.CompactTextString(m) }
-func (*LogicalPort) ProtoMessage()    {}
-func (*LogicalPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_caf139ab3abc8240, []int{1}
-}
-
-func (m *LogicalPort) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalPort.Unmarshal(m, b)
-}
-func (m *LogicalPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalPort.Marshal(b, m, deterministic)
-}
-func (m *LogicalPort) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalPort.Merge(m, src)
-}
-func (m *LogicalPort) XXX_Size() int {
-	return xxx_messageInfo_LogicalPort.Size(m)
-}
-func (m *LogicalPort) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalPort.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalPort proto.InternalMessageInfo
-
-func (m *LogicalPort) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *LogicalPort) GetOfpPort() *openflow_13.OfpPort {
-	if m != nil {
-		return m.OfpPort
-	}
-	return nil
-}
-
-func (m *LogicalPort) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *LogicalPort) GetDevicePortNo() uint32 {
-	if m != nil {
-		return m.DevicePortNo
-	}
-	return 0
-}
-
-func (m *LogicalPort) GetRootPort() bool {
-	if m != nil {
-		return m.RootPort
-	}
-	return false
-}
-
-func (m *LogicalPort) GetOfpPortStats() *openflow_13.OfpPortStats {
-	if m != nil {
-		return m.OfpPortStats
-	}
-	return nil
-}
-
-type LogicalPorts struct {
-	Items                []*LogicalPort `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *LogicalPorts) Reset()         { *m = LogicalPorts{} }
-func (m *LogicalPorts) String() string { return proto.CompactTextString(m) }
-func (*LogicalPorts) ProtoMessage()    {}
-func (*LogicalPorts) Descriptor() ([]byte, []int) {
-	return fileDescriptor_caf139ab3abc8240, []int{2}
-}
-
-func (m *LogicalPorts) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalPorts.Unmarshal(m, b)
-}
-func (m *LogicalPorts) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalPorts.Marshal(b, m, deterministic)
-}
-func (m *LogicalPorts) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalPorts.Merge(m, src)
-}
-func (m *LogicalPorts) XXX_Size() int {
-	return xxx_messageInfo_LogicalPorts.Size(m)
-}
-func (m *LogicalPorts) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalPorts.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalPorts proto.InternalMessageInfo
-
-func (m *LogicalPorts) GetItems() []*LogicalPort {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type LogicalDevice struct {
-	// unique id of logical device
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// unique datapath id for the logical device (used by the SDN controller)
-	DatapathId uint64 `protobuf:"varint,2,opt,name=datapath_id,json=datapathId,proto3" json:"datapath_id,omitempty"`
-	// device description
-	Desc *openflow_13.OfpDesc `protobuf:"bytes,3,opt,name=desc,proto3" json:"desc,omitempty"`
-	// device features
-	SwitchFeatures *openflow_13.OfpSwitchFeatures `protobuf:"bytes,4,opt,name=switch_features,json=switchFeatures,proto3" json:"switch_features,omitempty"`
-	// name of the root device anchoring logical device
-	RootDeviceId         string   `protobuf:"bytes,5,opt,name=root_device_id,json=rootDeviceId,proto3" json:"root_device_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *LogicalDevice) Reset()         { *m = LogicalDevice{} }
-func (m *LogicalDevice) String() string { return proto.CompactTextString(m) }
-func (*LogicalDevice) ProtoMessage()    {}
-func (*LogicalDevice) Descriptor() ([]byte, []int) {
-	return fileDescriptor_caf139ab3abc8240, []int{3}
-}
-
-func (m *LogicalDevice) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalDevice.Unmarshal(m, b)
-}
-func (m *LogicalDevice) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalDevice.Marshal(b, m, deterministic)
-}
-func (m *LogicalDevice) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalDevice.Merge(m, src)
-}
-func (m *LogicalDevice) XXX_Size() int {
-	return xxx_messageInfo_LogicalDevice.Size(m)
-}
-func (m *LogicalDevice) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalDevice.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalDevice proto.InternalMessageInfo
-
-func (m *LogicalDevice) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *LogicalDevice) GetDatapathId() uint64 {
-	if m != nil {
-		return m.DatapathId
-	}
-	return 0
-}
-
-func (m *LogicalDevice) GetDesc() *openflow_13.OfpDesc {
-	if m != nil {
-		return m.Desc
-	}
-	return nil
-}
-
-func (m *LogicalDevice) GetSwitchFeatures() *openflow_13.OfpSwitchFeatures {
-	if m != nil {
-		return m.SwitchFeatures
-	}
-	return nil
-}
-
-func (m *LogicalDevice) GetRootDeviceId() string {
-	if m != nil {
-		return m.RootDeviceId
-	}
-	return ""
-}
-
-type LogicalDevices struct {
-	Items                []*LogicalDevice `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *LogicalDevices) Reset()         { *m = LogicalDevices{} }
-func (m *LogicalDevices) String() string { return proto.CompactTextString(m) }
-func (*LogicalDevices) ProtoMessage()    {}
-func (*LogicalDevices) Descriptor() ([]byte, []int) {
-	return fileDescriptor_caf139ab3abc8240, []int{4}
-}
-
-func (m *LogicalDevices) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalDevices.Unmarshal(m, b)
-}
-func (m *LogicalDevices) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalDevices.Marshal(b, m, deterministic)
-}
-func (m *LogicalDevices) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalDevices.Merge(m, src)
-}
-func (m *LogicalDevices) XXX_Size() int {
-	return xxx_messageInfo_LogicalDevices.Size(m)
-}
-func (m *LogicalDevices) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalDevices.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalDevices proto.InternalMessageInfo
-
-func (m *LogicalDevices) GetItems() []*LogicalDevice {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterType((*LogicalPortId)(nil), "voltha.LogicalPortId")
-	proto.RegisterType((*LogicalPort)(nil), "voltha.LogicalPort")
-	proto.RegisterType((*LogicalPorts)(nil), "voltha.LogicalPorts")
-	proto.RegisterType((*LogicalDevice)(nil), "voltha.LogicalDevice")
-	proto.RegisterType((*LogicalDevices)(nil), "voltha.LogicalDevices")
-}
-
-func init() {
-	proto.RegisterFile("voltha_protos/logical_device.proto", fileDescriptor_caf139ab3abc8240)
-}
-
-var fileDescriptor_caf139ab3abc8240 = []byte{
-	// 456 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xc1, 0x6a, 0xdb, 0x40,
-	0x10, 0x86, 0x91, 0x63, 0x3b, 0xce, 0xca, 0x51, 0x40, 0x21, 0x54, 0x24, 0x85, 0x08, 0xd1, 0x83,
-	0x43, 0xa9, 0xd4, 0x3a, 0x3d, 0xb4, 0x87, 0x1e, 0x1a, 0x42, 0xc0, 0x50, 0xda, 0xb2, 0x85, 0x1e,
-	0x7a, 0x11, 0x1b, 0xed, 0x5a, 0x5e, 0x90, 0x35, 0x42, 0xbb, 0x71, 0x5e, 0xb6, 0x4f, 0xd1, 0x27,
-	0x28, 0x3b, 0xbb, 0x6a, 0xa5, 0x38, 0x47, 0xfd, 0xf3, 0xcd, 0xcc, 0x3f, 0xff, 0x8a, 0x24, 0x3b,
-	0xa8, 0xf4, 0x86, 0xe5, 0x4d, 0x0b, 0x1a, 0x54, 0x56, 0x41, 0x29, 0x0b, 0x56, 0xe5, 0x5c, 0xec,
-	0x64, 0x21, 0x52, 0x54, 0xc3, 0xa9, 0x65, 0xce, 0x5f, 0x96, 0x00, 0x65, 0x25, 0x32, 0xd6, 0xc8,
-	0x8c, 0xd5, 0x35, 0x68, 0xa6, 0x25, 0xd4, 0xca, 0x52, 0xe7, 0xd1, 0x70, 0xd2, 0x56, 0x68, 0xe6,
-	0x2a, 0x97, 0xc3, 0x0a, 0x34, 0xa2, 0x5e, 0x57, 0xf0, 0x98, 0xbf, 0xbb, 0xb6, 0x40, 0xf2, 0x81,
-	0x1c, 0x7f, 0xb1, 0x8b, 0xbf, 0x43, 0xab, 0x57, 0x3c, 0x0c, 0xc8, 0x48, 0xf2, 0xc8, 0x8b, 0xbd,
-	0xc5, 0x11, 0x1d, 0x49, 0x1e, 0xbe, 0x20, 0x87, 0x0d, 0xb4, 0x3a, 0x97, 0x3c, 0x1a, 0xa1, 0x38,
-	0x6d, 0x10, 0x4c, 0xfe, 0x78, 0xc4, 0xef, 0xb5, 0xee, 0x35, 0xbe, 0x25, 0x33, 0x58, 0x37, 0xb9,
-	0xa1, 0xb1, 0xd3, 0x5f, 0x9e, 0xa5, 0xfd, 0xfd, 0x5d, 0x91, 0x1e, 0xc2, 0xba, 0xc1, 0x09, 0x17,
-	0xe4, 0xc8, 0x1e, 0x6f, 0x96, 0x1d, 0xe0, 0xa0, 0x99, 0x15, 0x56, 0x3c, 0x7c, 0x45, 0x02, 0x57,
-	0x44, 0x3b, 0x35, 0x44, 0xe3, 0xd8, 0x5b, 0x1c, 0xd3, 0xb9, 0x55, 0xcd, 0x80, 0xaf, 0x60, 0x46,
-	0xb4, 0x00, 0xda, 0x6e, 0x9d, 0xc4, 0xde, 0x62, 0x46, 0x67, 0x46, 0xc0, 0xf9, 0x9f, 0x49, 0xd0,
-	0x2d, 0xcd, 0x95, 0x66, 0x5a, 0x45, 0x53, 0xf4, 0x75, 0xf1, 0xac, 0x2f, 0x8b, 0xd0, 0xb9, 0x73,
-	0xf7, 0xc3, 0x7c, 0x25, 0x1f, 0xc9, 0xbc, 0x77, 0xb3, 0x0a, 0xaf, 0xc8, 0x44, 0x6a, 0xb1, 0x55,
-	0x91, 0x17, 0x1f, 0x2c, 0xfc, 0xe5, 0x69, 0x6a, 0xf3, 0x4e, 0x7b, 0x10, 0xb5, 0x44, 0xf2, 0xdb,
-	0xfb, 0x17, 0xf5, 0x2d, 0x5a, 0xde, 0x4b, 0xec, 0x92, 0xf8, 0x9c, 0x69, 0xd6, 0x30, 0xbd, 0xe9,
-	0xe2, 0x1e, 0x53, 0xd2, 0x49, 0x2b, 0x1e, 0x5e, 0x91, 0x31, 0x17, 0xaa, 0xc0, 0x6c, 0x9e, 0x8b,
-	0xd3, 0x14, 0x29, 0x22, 0xe1, 0x8a, 0x9c, 0xa8, 0x47, 0xa9, 0x8b, 0x4d, 0xbe, 0x16, 0x4c, 0x3f,
-	0xb4, 0x42, 0x61, 0x5e, 0xfe, 0x32, 0xde, 0xeb, 0x7a, 0xc2, 0xd1, 0xc0, 0x0a, 0x77, 0xee, 0xdb,
-	0x24, 0x8f, 0x99, 0xfe, 0x7f, 0x9b, 0x09, 0x5a, 0x9e, 0x1b, 0xf5, 0xd6, 0xbd, 0x4f, 0xf2, 0x89,
-	0x04, 0x83, 0xeb, 0x54, 0xf8, 0x7a, 0x98, 0xcd, 0xd9, 0x93, 0x6c, 0x2c, 0xe6, 0xd2, 0xb9, 0xf9,
-	0x49, 0x4e, 0xa1, 0x2d, 0xd1, 0x5b, 0x01, 0x2d, 0x77, 0xec, 0xcd, 0xc9, 0xb7, 0xbb, 0x01, 0xfe,
-	0x2b, 0x2d, 0xa5, 0xde, 0x3c, 0xdc, 0xa7, 0x05, 0x6c, 0xb3, 0x0e, 0xce, 0x2c, 0xfc, 0xc6, 0xfd,
-	0xe4, 0xbb, 0xf7, 0x59, 0x09, 0x4e, 0xbb, 0x9f, 0xa2, 0x78, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff,
-	0xda, 0x87, 0xca, 0xb8, 0x6d, 0x03, 0x00, 0x00,
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/ponsim.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/ponsim.pb.go
deleted file mode 100644
index e2abe7f..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/ponsim.pb.go
+++ /dev/null
@@ -1,708 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/ponsim.proto
-
-package voltha
-
-import (
-	context "context"
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	empty "github.com/golang/protobuf/ptypes/empty"
-	openflow_13 "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	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 PonSimOnuDeviceInfo struct {
-	UniPort              int32    `protobuf:"varint,1,opt,name=uni_port,json=uniPort,proto3" json:"uni_port,omitempty"`
-	SerialNumber         string   `protobuf:"bytes,2,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PonSimOnuDeviceInfo) Reset()         { *m = PonSimOnuDeviceInfo{} }
-func (m *PonSimOnuDeviceInfo) String() string { return proto.CompactTextString(m) }
-func (*PonSimOnuDeviceInfo) ProtoMessage()    {}
-func (*PonSimOnuDeviceInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{0}
-}
-
-func (m *PonSimOnuDeviceInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimOnuDeviceInfo.Unmarshal(m, b)
-}
-func (m *PonSimOnuDeviceInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimOnuDeviceInfo.Marshal(b, m, deterministic)
-}
-func (m *PonSimOnuDeviceInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimOnuDeviceInfo.Merge(m, src)
-}
-func (m *PonSimOnuDeviceInfo) XXX_Size() int {
-	return xxx_messageInfo_PonSimOnuDeviceInfo.Size(m)
-}
-func (m *PonSimOnuDeviceInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimOnuDeviceInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimOnuDeviceInfo proto.InternalMessageInfo
-
-func (m *PonSimOnuDeviceInfo) GetUniPort() int32 {
-	if m != nil {
-		return m.UniPort
-	}
-	return 0
-}
-
-func (m *PonSimOnuDeviceInfo) GetSerialNumber() string {
-	if m != nil {
-		return m.SerialNumber
-	}
-	return ""
-}
-
-type PonSimDeviceInfo struct {
-	NniPort              int32                  `protobuf:"varint,1,opt,name=nni_port,json=nniPort,proto3" json:"nni_port,omitempty"`
-	Onus                 []*PonSimOnuDeviceInfo `protobuf:"bytes,2,rep,name=onus,proto3" json:"onus,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *PonSimDeviceInfo) Reset()         { *m = PonSimDeviceInfo{} }
-func (m *PonSimDeviceInfo) String() string { return proto.CompactTextString(m) }
-func (*PonSimDeviceInfo) ProtoMessage()    {}
-func (*PonSimDeviceInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{1}
-}
-
-func (m *PonSimDeviceInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimDeviceInfo.Unmarshal(m, b)
-}
-func (m *PonSimDeviceInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimDeviceInfo.Marshal(b, m, deterministic)
-}
-func (m *PonSimDeviceInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimDeviceInfo.Merge(m, src)
-}
-func (m *PonSimDeviceInfo) XXX_Size() int {
-	return xxx_messageInfo_PonSimDeviceInfo.Size(m)
-}
-func (m *PonSimDeviceInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimDeviceInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimDeviceInfo proto.InternalMessageInfo
-
-func (m *PonSimDeviceInfo) GetNniPort() int32 {
-	if m != nil {
-		return m.NniPort
-	}
-	return 0
-}
-
-func (m *PonSimDeviceInfo) GetOnus() []*PonSimOnuDeviceInfo {
-	if m != nil {
-		return m.Onus
-	}
-	return nil
-}
-
-type FlowTable struct {
-	Port                 int32                       `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
-	Flows                []*openflow_13.OfpFlowStats `protobuf:"bytes,2,rep,name=flows,proto3" json:"flows,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
-}
-
-func (m *FlowTable) Reset()         { *m = FlowTable{} }
-func (m *FlowTable) String() string { return proto.CompactTextString(m) }
-func (*FlowTable) ProtoMessage()    {}
-func (*FlowTable) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{2}
-}
-
-func (m *FlowTable) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowTable.Unmarshal(m, b)
-}
-func (m *FlowTable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowTable.Marshal(b, m, deterministic)
-}
-func (m *FlowTable) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowTable.Merge(m, src)
-}
-func (m *FlowTable) XXX_Size() int {
-	return xxx_messageInfo_FlowTable.Size(m)
-}
-func (m *FlowTable) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowTable.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowTable proto.InternalMessageInfo
-
-func (m *FlowTable) GetPort() int32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-func (m *FlowTable) GetFlows() []*openflow_13.OfpFlowStats {
-	if m != nil {
-		return m.Flows
-	}
-	return nil
-}
-
-type PonSimFrame struct {
-	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Payload              []byte   `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
-	OutPort              int32    `protobuf:"varint,3,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PonSimFrame) Reset()         { *m = PonSimFrame{} }
-func (m *PonSimFrame) String() string { return proto.CompactTextString(m) }
-func (*PonSimFrame) ProtoMessage()    {}
-func (*PonSimFrame) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{3}
-}
-
-func (m *PonSimFrame) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimFrame.Unmarshal(m, b)
-}
-func (m *PonSimFrame) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimFrame.Marshal(b, m, deterministic)
-}
-func (m *PonSimFrame) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimFrame.Merge(m, src)
-}
-func (m *PonSimFrame) XXX_Size() int {
-	return xxx_messageInfo_PonSimFrame.Size(m)
-}
-func (m *PonSimFrame) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimFrame.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimFrame proto.InternalMessageInfo
-
-func (m *PonSimFrame) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *PonSimFrame) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
-	}
-	return nil
-}
-
-func (m *PonSimFrame) GetOutPort() int32 {
-	if m != nil {
-		return m.OutPort
-	}
-	return 0
-}
-
-type PonSimPacketCounter struct {
-	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Value                int64    `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PonSimPacketCounter) Reset()         { *m = PonSimPacketCounter{} }
-func (m *PonSimPacketCounter) String() string { return proto.CompactTextString(m) }
-func (*PonSimPacketCounter) ProtoMessage()    {}
-func (*PonSimPacketCounter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{4}
-}
-
-func (m *PonSimPacketCounter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimPacketCounter.Unmarshal(m, b)
-}
-func (m *PonSimPacketCounter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimPacketCounter.Marshal(b, m, deterministic)
-}
-func (m *PonSimPacketCounter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimPacketCounter.Merge(m, src)
-}
-func (m *PonSimPacketCounter) XXX_Size() int {
-	return xxx_messageInfo_PonSimPacketCounter.Size(m)
-}
-func (m *PonSimPacketCounter) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimPacketCounter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimPacketCounter proto.InternalMessageInfo
-
-func (m *PonSimPacketCounter) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *PonSimPacketCounter) GetValue() int64 {
-	if m != nil {
-		return m.Value
-	}
-	return 0
-}
-
-type PonSimPortMetrics struct {
-	PortName             string                 `protobuf:"bytes,1,opt,name=port_name,json=portName,proto3" json:"port_name,omitempty"`
-	Packets              []*PonSimPacketCounter `protobuf:"bytes,2,rep,name=packets,proto3" json:"packets,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *PonSimPortMetrics) Reset()         { *m = PonSimPortMetrics{} }
-func (m *PonSimPortMetrics) String() string { return proto.CompactTextString(m) }
-func (*PonSimPortMetrics) ProtoMessage()    {}
-func (*PonSimPortMetrics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{5}
-}
-
-func (m *PonSimPortMetrics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimPortMetrics.Unmarshal(m, b)
-}
-func (m *PonSimPortMetrics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimPortMetrics.Marshal(b, m, deterministic)
-}
-func (m *PonSimPortMetrics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimPortMetrics.Merge(m, src)
-}
-func (m *PonSimPortMetrics) XXX_Size() int {
-	return xxx_messageInfo_PonSimPortMetrics.Size(m)
-}
-func (m *PonSimPortMetrics) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimPortMetrics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimPortMetrics proto.InternalMessageInfo
-
-func (m *PonSimPortMetrics) GetPortName() string {
-	if m != nil {
-		return m.PortName
-	}
-	return ""
-}
-
-func (m *PonSimPortMetrics) GetPackets() []*PonSimPacketCounter {
-	if m != nil {
-		return m.Packets
-	}
-	return nil
-}
-
-type PonSimMetrics struct {
-	Device               string               `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
-	Metrics              []*PonSimPortMetrics `protobuf:"bytes,2,rep,name=metrics,proto3" json:"metrics,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *PonSimMetrics) Reset()         { *m = PonSimMetrics{} }
-func (m *PonSimMetrics) String() string { return proto.CompactTextString(m) }
-func (*PonSimMetrics) ProtoMessage()    {}
-func (*PonSimMetrics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{6}
-}
-
-func (m *PonSimMetrics) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimMetrics.Unmarshal(m, b)
-}
-func (m *PonSimMetrics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimMetrics.Marshal(b, m, deterministic)
-}
-func (m *PonSimMetrics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimMetrics.Merge(m, src)
-}
-func (m *PonSimMetrics) XXX_Size() int {
-	return xxx_messageInfo_PonSimMetrics.Size(m)
-}
-func (m *PonSimMetrics) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimMetrics.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimMetrics proto.InternalMessageInfo
-
-func (m *PonSimMetrics) GetDevice() string {
-	if m != nil {
-		return m.Device
-	}
-	return ""
-}
-
-func (m *PonSimMetrics) GetMetrics() []*PonSimPortMetrics {
-	if m != nil {
-		return m.Metrics
-	}
-	return nil
-}
-
-type PonSimMetricsRequest struct {
-	Port                 int32    `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PonSimMetricsRequest) Reset()         { *m = PonSimMetricsRequest{} }
-func (m *PonSimMetricsRequest) String() string { return proto.CompactTextString(m) }
-func (*PonSimMetricsRequest) ProtoMessage()    {}
-func (*PonSimMetricsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_352253851b8ea7c0, []int{7}
-}
-
-func (m *PonSimMetricsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PonSimMetricsRequest.Unmarshal(m, b)
-}
-func (m *PonSimMetricsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PonSimMetricsRequest.Marshal(b, m, deterministic)
-}
-func (m *PonSimMetricsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PonSimMetricsRequest.Merge(m, src)
-}
-func (m *PonSimMetricsRequest) XXX_Size() int {
-	return xxx_messageInfo_PonSimMetricsRequest.Size(m)
-}
-func (m *PonSimMetricsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_PonSimMetricsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PonSimMetricsRequest proto.InternalMessageInfo
-
-func (m *PonSimMetricsRequest) GetPort() int32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-func init() {
-	proto.RegisterType((*PonSimOnuDeviceInfo)(nil), "voltha.PonSimOnuDeviceInfo")
-	proto.RegisterType((*PonSimDeviceInfo)(nil), "voltha.PonSimDeviceInfo")
-	proto.RegisterType((*FlowTable)(nil), "voltha.FlowTable")
-	proto.RegisterType((*PonSimFrame)(nil), "voltha.PonSimFrame")
-	proto.RegisterType((*PonSimPacketCounter)(nil), "voltha.PonSimPacketCounter")
-	proto.RegisterType((*PonSimPortMetrics)(nil), "voltha.PonSimPortMetrics")
-	proto.RegisterType((*PonSimMetrics)(nil), "voltha.PonSimMetrics")
-	proto.RegisterType((*PonSimMetricsRequest)(nil), "voltha.PonSimMetricsRequest")
-}
-
-func init() { proto.RegisterFile("voltha_protos/ponsim.proto", fileDescriptor_352253851b8ea7c0) }
-
-var fileDescriptor_352253851b8ea7c0 = []byte{
-	// 565 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x94, 0x5f, 0x6f, 0xd3, 0x3c,
-	0x14, 0xc6, 0xdb, 0x6e, 0xeb, 0x9f, 0xb3, 0xf5, 0x7d, 0x99, 0x3b, 0xa6, 0xae, 0xbd, 0xa0, 0x32,
-	0x37, 0x15, 0x12, 0x09, 0x5b, 0xe1, 0x06, 0x24, 0x40, 0x8c, 0x6d, 0xe2, 0x82, 0x51, 0xb9, 0xec,
-	0x06, 0x21, 0xa2, 0x34, 0x71, 0xb3, 0x88, 0xc4, 0x27, 0x24, 0x76, 0xa7, 0x7d, 0x43, 0x3e, 0x16,
-	0x8a, 0x9d, 0xd0, 0x66, 0x6a, 0xb9, 0xb3, 0x8f, 0x9f, 0xfe, 0x9e, 0xf3, 0x9c, 0x1c, 0x15, 0x06,
-	0x4b, 0x8c, 0xe4, 0xad, 0xeb, 0x24, 0x29, 0x4a, 0xcc, 0xec, 0x04, 0x45, 0x16, 0xc6, 0x96, 0xbe,
-	0x91, 0xa6, 0x79, 0x1b, 0x0c, 0x03, 0xc4, 0x20, 0xe2, 0xb6, 0xae, 0xce, 0xd5, 0xc2, 0xe6, 0x71,
-	0x22, 0xef, 0x8d, 0x68, 0xf0, 0xa4, 0x0a, 0xc0, 0x84, 0x8b, 0x45, 0x84, 0x77, 0xce, 0xe9, 0xc4,
-	0x08, 0xe8, 0x0d, 0xf4, 0xa6, 0x28, 0x66, 0x61, 0xfc, 0x45, 0xa8, 0x8f, 0x7c, 0x19, 0x7a, 0xfc,
-	0x93, 0x58, 0x20, 0x39, 0x81, 0xb6, 0x12, 0xa1, 0x93, 0x60, 0x2a, 0xfb, 0xf5, 0x51, 0x7d, 0xbc,
-	0xc7, 0x5a, 0x4a, 0x84, 0x53, 0x4c, 0x25, 0x79, 0x0a, 0xdd, 0x8c, 0xa7, 0xa1, 0x1b, 0x39, 0x42,
-	0xc5, 0x73, 0x9e, 0xf6, 0x1b, 0xa3, 0xfa, 0xb8, 0xc3, 0x0e, 0x4c, 0xf1, 0x5a, 0xd7, 0xe8, 0x0f,
-	0x78, 0x64, 0xb0, 0x55, 0xa6, 0x78, 0xc0, 0x14, 0x05, 0xd3, 0x86, 0x5d, 0x14, 0x2a, 0xeb, 0x37,
-	0x46, 0x3b, 0xe3, 0xfd, 0xb3, 0xa1, 0x65, 0xba, 0xb6, 0x36, 0x74, 0xc6, 0xb4, 0x90, 0x32, 0xe8,
-	0x5c, 0x46, 0x78, 0xf7, 0xd5, 0x9d, 0x47, 0x9c, 0x10, 0xd8, 0x5d, 0x83, 0xea, 0x33, 0x39, 0x85,
-	0xbd, 0x3c, 0xe8, 0x0a, 0xb9, 0x1e, 0x1d, 0x17, 0x89, 0xa3, 0xcf, 0x99, 0x74, 0x65, 0xc6, 0x8c,
-	0x92, 0x32, 0xd8, 0x37, 0x86, 0x97, 0xa9, 0x1b, 0x73, 0xf2, 0x1f, 0x34, 0x42, 0x5f, 0x33, 0x3b,
-	0xac, 0x11, 0xfa, 0xa4, 0x0f, 0xad, 0xc4, 0xbd, 0x8f, 0xd0, 0xf5, 0x75, 0xe2, 0x03, 0x56, 0x5e,
-	0xf3, 0x60, 0xa8, 0xa4, 0x09, 0xb6, 0x63, 0x82, 0xa1, 0x92, 0x79, 0x30, 0xfa, 0xae, 0x1c, 0xef,
-	0xd4, 0xf5, 0x7e, 0x72, 0x79, 0x8e, 0x4a, 0x48, 0x9e, 0xe6, 0x1d, 0x0b, 0x37, 0xe6, 0x05, 0x5d,
-	0x9f, 0xc9, 0x11, 0xec, 0x2d, 0xdd, 0x48, 0x71, 0x4d, 0xdf, 0x61, 0xe6, 0x42, 0x03, 0x38, 0x2c,
-	0x00, 0x98, 0xca, 0xcf, 0x5c, 0xa6, 0xa1, 0x97, 0x91, 0x21, 0x74, 0x72, 0x33, 0x67, 0x8d, 0xd1,
-	0xce, 0x0b, 0xd7, 0x39, 0xe7, 0x55, 0xde, 0x67, 0x6e, 0xb6, 0x65, 0x9c, 0x95, 0x4e, 0x58, 0xa9,
-	0xa5, 0xdf, 0xa1, 0x6b, 0xde, 0x4b, 0x93, 0x63, 0x68, 0xfa, 0x7a, 0xec, 0x85, 0x43, 0x71, 0x23,
-	0x13, 0x68, 0xc5, 0x46, 0x52, 0xf0, 0x4f, 0x1e, 0xf0, 0x57, 0x8d, 0xb2, 0x52, 0x49, 0x9f, 0xc1,
-	0x51, 0x85, 0xce, 0xf8, 0x2f, 0xc5, 0x33, 0xb9, 0xe9, 0xd3, 0x9d, 0xfd, 0x6e, 0x40, 0xd3, 0x88,
-	0xc9, 0x6b, 0xe8, 0xcc, 0xb8, 0xf0, 0xcd, 0x07, 0xe9, 0x55, 0x7d, 0x74, 0x71, 0x70, 0x6c, 0x99,
-	0xf5, 0xb7, 0xca, 0xf5, 0xb7, 0x2e, 0xf2, 0xf5, 0xa7, 0x35, 0xf2, 0x1e, 0xba, 0x8c, 0x7b, 0x3c,
-	0x5c, 0x72, 0xad, 0xcc, 0xc8, 0x16, 0xe9, 0x60, 0x13, 0x97, 0xd6, 0x5e, 0xd4, 0xc9, 0x39, 0x74,
-	0xaf, 0xb8, 0x5c, 0xdb, 0xe0, 0x6d, 0x84, 0x7e, 0x95, 0xb0, 0xfa, 0x05, 0xad, 0x91, 0xb7, 0xf0,
-	0xff, 0x4d, 0xe2, 0xbb, 0x92, 0xaf, 0xf6, 0xf5, 0xb0, 0x94, 0xff, 0x2d, 0xfd, 0x23, 0xc6, 0x1b,
-	0x68, 0x5f, 0x71, 0x39, 0xcb, 0x17, 0x75, 0xab, 0xff, 0xe3, 0xaa, 0x7f, 0x31, 0x63, 0x5a, 0xfb,
-	0x70, 0x01, 0x3d, 0x4c, 0x03, 0xbd, 0xfb, 0x1e, 0xa6, 0x7e, 0x21, 0xfb, 0x66, 0x05, 0xa1, 0xbc,
-	0x55, 0x73, 0xcb, 0xc3, 0xd8, 0x2e, 0xdf, 0x6c, 0xf3, 0xf6, 0xbc, 0xf8, 0xa7, 0x58, 0xbe, 0xb4,
-	0x03, 0x2c, 0x6a, 0xf3, 0xa6, 0x2e, 0x4e, 0xfe, 0x04, 0x00, 0x00, 0xff, 0xff, 0xf9, 0xf9, 0xa0,
-	0x81, 0x8f, 0x04, 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
-
-// PonSimClient is the client API for PonSim service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type PonSimClient interface {
-	SendFrame(ctx context.Context, in *PonSimFrame, opts ...grpc.CallOption) (*empty.Empty, error)
-	ReceiveFrames(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (PonSim_ReceiveFramesClient, error)
-	GetDeviceInfo(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*PonSimDeviceInfo, error)
-	UpdateFlowTable(ctx context.Context, in *FlowTable, opts ...grpc.CallOption) (*empty.Empty, error)
-	GetStats(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*PonSimMetrics, error)
-}
-
-type ponSimClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewPonSimClient(cc *grpc.ClientConn) PonSimClient {
-	return &ponSimClient{cc}
-}
-
-func (c *ponSimClient) SendFrame(ctx context.Context, in *PonSimFrame, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.PonSim/SendFrame", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *ponSimClient) ReceiveFrames(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (PonSim_ReceiveFramesClient, error) {
-	stream, err := c.cc.NewStream(ctx, &_PonSim_serviceDesc.Streams[0], "/voltha.PonSim/ReceiveFrames", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &ponSimReceiveFramesClient{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 PonSim_ReceiveFramesClient interface {
-	Recv() (*PonSimFrame, error)
-	grpc.ClientStream
-}
-
-type ponSimReceiveFramesClient struct {
-	grpc.ClientStream
-}
-
-func (x *ponSimReceiveFramesClient) Recv() (*PonSimFrame, error) {
-	m := new(PonSimFrame)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *ponSimClient) GetDeviceInfo(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*PonSimDeviceInfo, error) {
-	out := new(PonSimDeviceInfo)
-	err := c.cc.Invoke(ctx, "/voltha.PonSim/GetDeviceInfo", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *ponSimClient) UpdateFlowTable(ctx context.Context, in *FlowTable, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.PonSim/UpdateFlowTable", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *ponSimClient) GetStats(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*PonSimMetrics, error) {
-	out := new(PonSimMetrics)
-	err := c.cc.Invoke(ctx, "/voltha.PonSim/GetStats", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// PonSimServer is the server API for PonSim service.
-type PonSimServer interface {
-	SendFrame(context.Context, *PonSimFrame) (*empty.Empty, error)
-	ReceiveFrames(*empty.Empty, PonSim_ReceiveFramesServer) error
-	GetDeviceInfo(context.Context, *empty.Empty) (*PonSimDeviceInfo, error)
-	UpdateFlowTable(context.Context, *FlowTable) (*empty.Empty, error)
-	GetStats(context.Context, *empty.Empty) (*PonSimMetrics, error)
-}
-
-// UnimplementedPonSimServer can be embedded to have forward compatible implementations.
-type UnimplementedPonSimServer struct {
-}
-
-func (*UnimplementedPonSimServer) SendFrame(ctx context.Context, req *PonSimFrame) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method SendFrame not implemented")
-}
-func (*UnimplementedPonSimServer) ReceiveFrames(req *empty.Empty, srv PonSim_ReceiveFramesServer) error {
-	return status.Errorf(codes.Unimplemented, "method ReceiveFrames not implemented")
-}
-func (*UnimplementedPonSimServer) GetDeviceInfo(ctx context.Context, req *empty.Empty) (*PonSimDeviceInfo, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceInfo not implemented")
-}
-func (*UnimplementedPonSimServer) UpdateFlowTable(ctx context.Context, req *FlowTable) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateFlowTable not implemented")
-}
-func (*UnimplementedPonSimServer) GetStats(ctx context.Context, req *empty.Empty) (*PonSimMetrics, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetStats not implemented")
-}
-
-func RegisterPonSimServer(s *grpc.Server, srv PonSimServer) {
-	s.RegisterService(&_PonSim_serviceDesc, srv)
-}
-
-func _PonSim_SendFrame_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PonSimFrame)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PonSimServer).SendFrame(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.PonSim/SendFrame",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PonSimServer).SendFrame(ctx, req.(*PonSimFrame))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PonSim_ReceiveFrames_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(empty.Empty)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(PonSimServer).ReceiveFrames(m, &ponSimReceiveFramesServer{stream})
-}
-
-type PonSim_ReceiveFramesServer interface {
-	Send(*PonSimFrame) error
-	grpc.ServerStream
-}
-
-type ponSimReceiveFramesServer struct {
-	grpc.ServerStream
-}
-
-func (x *ponSimReceiveFramesServer) Send(m *PonSimFrame) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func _PonSim_GetDeviceInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PonSimServer).GetDeviceInfo(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.PonSim/GetDeviceInfo",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PonSimServer).GetDeviceInfo(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PonSim_UpdateFlowTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(FlowTable)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PonSimServer).UpdateFlowTable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.PonSim/UpdateFlowTable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PonSimServer).UpdateFlowTable(ctx, req.(*FlowTable))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _PonSim_GetStats_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(PonSimServer).GetStats(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.PonSim/GetStats",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(PonSimServer).GetStats(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _PonSim_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "voltha.PonSim",
-	HandlerType: (*PonSimServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "SendFrame",
-			Handler:    _PonSim_SendFrame_Handler,
-		},
-		{
-			MethodName: "GetDeviceInfo",
-			Handler:    _PonSim_GetDeviceInfo_Handler,
-		},
-		{
-			MethodName: "UpdateFlowTable",
-			Handler:    _PonSim_UpdateFlowTable_Handler,
-		},
-		{
-			MethodName: "GetStats",
-			Handler:    _PonSim_GetStats_Handler,
-		},
-	},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "ReceiveFrames",
-			Handler:       _PonSim_ReceiveFrames_Handler,
-			ServerStreams: true,
-		},
-	},
-	Metadata: "voltha_protos/ponsim.proto",
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/voltha.pb.go b/vendor/github.com/opencord/voltha-protos/v4/go/voltha/voltha.pb.go
deleted file mode 100644
index cc7717b..0000000
--- a/vendor/github.com/opencord/voltha-protos/v4/go/voltha/voltha.pb.go
+++ /dev/null
@@ -1,5370 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: voltha_protos/voltha.proto
-
-package voltha
-
-import (
-	context "context"
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	empty "github.com/golang/protobuf/ptypes/empty"
-	common "github.com/opencord/voltha-protos/v4/go/common"
-	config "github.com/opencord/voltha-protos/v4/go/ext/config"
-	omci "github.com/opencord/voltha-protos/v4/go/omci"
-	openflow_13 "github.com/opencord/voltha-protos/v4/go/openflow_13"
-	_ "google.golang.org/genproto/googleapis/api/annotations"
-	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
-
-// ChildNode from public import voltha_protos/meta.proto
-type ChildNode = common.ChildNode
-
-// Access from public import voltha_protos/meta.proto
-type Access = common.Access
-
-var Access_name = common.Access_name
-var Access_value = common.Access_value
-
-const Access_CONFIG = Access(common.Access_CONFIG)
-const Access_READ_ONLY = Access(common.Access_READ_ONLY)
-const Access_REAL_TIME = Access(common.Access_REAL_TIME)
-
-var E_ChildNode = common.E_ChildNode
-
-var E_Access = common.E_Access
-
-// ID from public import voltha_protos/common.proto
-type ID = common.ID
-
-// IDs from public import voltha_protos/common.proto
-type IDs = common.IDs
-
-// AdminState from public import voltha_protos/common.proto
-type AdminState = common.AdminState
-
-// OperStatus from public import voltha_protos/common.proto
-type OperStatus = common.OperStatus
-
-// ConnectStatus from public import voltha_protos/common.proto
-type ConnectStatus = common.ConnectStatus
-
-// OperationResp from public import voltha_protos/common.proto
-type OperationResp = common.OperationResp
-
-// ValueType from public import voltha_protos/common.proto
-type ValueType = common.ValueType
-
-// ValueSpecifier from public import voltha_protos/common.proto
-type ValueSpecifier = common.ValueSpecifier
-
-// ReturnValues from public import voltha_protos/common.proto
-type ReturnValues = common.ReturnValues
-
-// TestModeKeys from public import voltha_protos/common.proto
-type TestModeKeys = common.TestModeKeys
-
-var TestModeKeys_name = common.TestModeKeys_name
-var TestModeKeys_value = common.TestModeKeys_value
-
-const TestModeKeys_api_test = TestModeKeys(common.TestModeKeys_api_test)
-
-// AdminState_Types from public import voltha_protos/common.proto
-type AdminState_Types = common.AdminState_Types
-
-var AdminState_Types_name = common.AdminState_Types_name
-var AdminState_Types_value = common.AdminState_Types_value
-
-const AdminState_UNKNOWN = AdminState_Types(common.AdminState_UNKNOWN)
-const AdminState_PREPROVISIONED = AdminState_Types(common.AdminState_PREPROVISIONED)
-const AdminState_ENABLED = AdminState_Types(common.AdminState_ENABLED)
-const AdminState_DISABLED = AdminState_Types(common.AdminState_DISABLED)
-const AdminState_DOWNLOADING_IMAGE = AdminState_Types(common.AdminState_DOWNLOADING_IMAGE)
-
-// OperStatus_Types from public import voltha_protos/common.proto
-type OperStatus_Types = common.OperStatus_Types
-
-var OperStatus_Types_name = common.OperStatus_Types_name
-var OperStatus_Types_value = common.OperStatus_Types_value
-
-const OperStatus_UNKNOWN = OperStatus_Types(common.OperStatus_UNKNOWN)
-const OperStatus_DISCOVERED = OperStatus_Types(common.OperStatus_DISCOVERED)
-const OperStatus_ACTIVATING = OperStatus_Types(common.OperStatus_ACTIVATING)
-const OperStatus_TESTING = OperStatus_Types(common.OperStatus_TESTING)
-const OperStatus_ACTIVE = OperStatus_Types(common.OperStatus_ACTIVE)
-const OperStatus_FAILED = OperStatus_Types(common.OperStatus_FAILED)
-const OperStatus_RECONCILING = OperStatus_Types(common.OperStatus_RECONCILING)
-const OperStatus_RECONCILING_FAILED = OperStatus_Types(common.OperStatus_RECONCILING_FAILED)
-
-// ConnectStatus_Types from public import voltha_protos/common.proto
-type ConnectStatus_Types = common.ConnectStatus_Types
-
-var ConnectStatus_Types_name = common.ConnectStatus_Types_name
-var ConnectStatus_Types_value = common.ConnectStatus_Types_value
-
-const ConnectStatus_UNKNOWN = ConnectStatus_Types(common.ConnectStatus_UNKNOWN)
-const ConnectStatus_UNREACHABLE = ConnectStatus_Types(common.ConnectStatus_UNREACHABLE)
-const ConnectStatus_REACHABLE = ConnectStatus_Types(common.ConnectStatus_REACHABLE)
-
-// OperationResp_OperationReturnCode from public import voltha_protos/common.proto
-type OperationResp_OperationReturnCode = common.OperationResp_OperationReturnCode
-
-var OperationResp_OperationReturnCode_name = common.OperationResp_OperationReturnCode_name
-var OperationResp_OperationReturnCode_value = common.OperationResp_OperationReturnCode_value
-
-const OperationResp_OPERATION_SUCCESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_SUCCESS)
-const OperationResp_OPERATION_FAILURE = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_FAILURE)
-const OperationResp_OPERATION_UNSUPPORTED = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_UNSUPPORTED)
-const OperationResp_OPERATION_IN_PROGRESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_IN_PROGRESS)
-
-// ValueType_Type from public import voltha_protos/common.proto
-type ValueType_Type = common.ValueType_Type
-
-var ValueType_Type_name = common.ValueType_Type_name
-var ValueType_Type_value = common.ValueType_Type_value
-
-const ValueType_EMPTY = ValueType_Type(common.ValueType_EMPTY)
-const ValueType_DISTANCE = ValueType_Type(common.ValueType_DISTANCE)
-
-// OfpHeader from public import voltha_protos/openflow_13.proto
-type OfpHeader = openflow_13.OfpHeader
-
-// OfpHelloElemHeader from public import voltha_protos/openflow_13.proto
-type OfpHelloElemHeader = openflow_13.OfpHelloElemHeader
-type OfpHelloElemHeader_Versionbitmap = openflow_13.OfpHelloElemHeader_Versionbitmap
-
-// OfpHelloElemVersionbitmap from public import voltha_protos/openflow_13.proto
-type OfpHelloElemVersionbitmap = openflow_13.OfpHelloElemVersionbitmap
-
-// OfpHello from public import voltha_protos/openflow_13.proto
-type OfpHello = openflow_13.OfpHello
-
-// OfpSwitchConfig from public import voltha_protos/openflow_13.proto
-type OfpSwitchConfig = openflow_13.OfpSwitchConfig
-
-// OfpTableMod from public import voltha_protos/openflow_13.proto
-type OfpTableMod = openflow_13.OfpTableMod
-
-// OfpPort from public import voltha_protos/openflow_13.proto
-type OfpPort = openflow_13.OfpPort
-
-// OfpSwitchFeatures from public import voltha_protos/openflow_13.proto
-type OfpSwitchFeatures = openflow_13.OfpSwitchFeatures
-
-// OfpPortStatus from public import voltha_protos/openflow_13.proto
-type OfpPortStatus = openflow_13.OfpPortStatus
-
-// OfpPortMod from public import voltha_protos/openflow_13.proto
-type OfpPortMod = openflow_13.OfpPortMod
-
-// OfpMatch from public import voltha_protos/openflow_13.proto
-type OfpMatch = openflow_13.OfpMatch
-
-// OfpOxmField from public import voltha_protos/openflow_13.proto
-type OfpOxmField = openflow_13.OfpOxmField
-type OfpOxmField_OfbField = openflow_13.OfpOxmField_OfbField
-type OfpOxmField_ExperimenterField = openflow_13.OfpOxmField_ExperimenterField
-
-// OfpOxmOfbField from public import voltha_protos/openflow_13.proto
-type OfpOxmOfbField = openflow_13.OfpOxmOfbField
-type OfpOxmOfbField_Port = openflow_13.OfpOxmOfbField_Port
-type OfpOxmOfbField_PhysicalPort = openflow_13.OfpOxmOfbField_PhysicalPort
-type OfpOxmOfbField_TableMetadata = openflow_13.OfpOxmOfbField_TableMetadata
-type OfpOxmOfbField_EthDst = openflow_13.OfpOxmOfbField_EthDst
-type OfpOxmOfbField_EthSrc = openflow_13.OfpOxmOfbField_EthSrc
-type OfpOxmOfbField_EthType = openflow_13.OfpOxmOfbField_EthType
-type OfpOxmOfbField_VlanVid = openflow_13.OfpOxmOfbField_VlanVid
-type OfpOxmOfbField_VlanPcp = openflow_13.OfpOxmOfbField_VlanPcp
-type OfpOxmOfbField_IpDscp = openflow_13.OfpOxmOfbField_IpDscp
-type OfpOxmOfbField_IpEcn = openflow_13.OfpOxmOfbField_IpEcn
-type OfpOxmOfbField_IpProto = openflow_13.OfpOxmOfbField_IpProto
-type OfpOxmOfbField_Ipv4Src = openflow_13.OfpOxmOfbField_Ipv4Src
-type OfpOxmOfbField_Ipv4Dst = openflow_13.OfpOxmOfbField_Ipv4Dst
-type OfpOxmOfbField_TcpSrc = openflow_13.OfpOxmOfbField_TcpSrc
-type OfpOxmOfbField_TcpDst = openflow_13.OfpOxmOfbField_TcpDst
-type OfpOxmOfbField_UdpSrc = openflow_13.OfpOxmOfbField_UdpSrc
-type OfpOxmOfbField_UdpDst = openflow_13.OfpOxmOfbField_UdpDst
-type OfpOxmOfbField_SctpSrc = openflow_13.OfpOxmOfbField_SctpSrc
-type OfpOxmOfbField_SctpDst = openflow_13.OfpOxmOfbField_SctpDst
-type OfpOxmOfbField_Icmpv4Type = openflow_13.OfpOxmOfbField_Icmpv4Type
-type OfpOxmOfbField_Icmpv4Code = openflow_13.OfpOxmOfbField_Icmpv4Code
-type OfpOxmOfbField_ArpOp = openflow_13.OfpOxmOfbField_ArpOp
-type OfpOxmOfbField_ArpSpa = openflow_13.OfpOxmOfbField_ArpSpa
-type OfpOxmOfbField_ArpTpa = openflow_13.OfpOxmOfbField_ArpTpa
-type OfpOxmOfbField_ArpSha = openflow_13.OfpOxmOfbField_ArpSha
-type OfpOxmOfbField_ArpTha = openflow_13.OfpOxmOfbField_ArpTha
-type OfpOxmOfbField_Ipv6Src = openflow_13.OfpOxmOfbField_Ipv6Src
-type OfpOxmOfbField_Ipv6Dst = openflow_13.OfpOxmOfbField_Ipv6Dst
-type OfpOxmOfbField_Ipv6Flabel = openflow_13.OfpOxmOfbField_Ipv6Flabel
-type OfpOxmOfbField_Icmpv6Type = openflow_13.OfpOxmOfbField_Icmpv6Type
-type OfpOxmOfbField_Icmpv6Code = openflow_13.OfpOxmOfbField_Icmpv6Code
-type OfpOxmOfbField_Ipv6NdTarget = openflow_13.OfpOxmOfbField_Ipv6NdTarget
-type OfpOxmOfbField_Ipv6NdSsl = openflow_13.OfpOxmOfbField_Ipv6NdSsl
-type OfpOxmOfbField_Ipv6NdTll = openflow_13.OfpOxmOfbField_Ipv6NdTll
-type OfpOxmOfbField_MplsLabel = openflow_13.OfpOxmOfbField_MplsLabel
-type OfpOxmOfbField_MplsTc = openflow_13.OfpOxmOfbField_MplsTc
-type OfpOxmOfbField_MplsBos = openflow_13.OfpOxmOfbField_MplsBos
-type OfpOxmOfbField_PbbIsid = openflow_13.OfpOxmOfbField_PbbIsid
-type OfpOxmOfbField_TunnelId = openflow_13.OfpOxmOfbField_TunnelId
-type OfpOxmOfbField_Ipv6Exthdr = openflow_13.OfpOxmOfbField_Ipv6Exthdr
-type OfpOxmOfbField_TableMetadataMask = openflow_13.OfpOxmOfbField_TableMetadataMask
-type OfpOxmOfbField_EthDstMask = openflow_13.OfpOxmOfbField_EthDstMask
-type OfpOxmOfbField_EthSrcMask = openflow_13.OfpOxmOfbField_EthSrcMask
-type OfpOxmOfbField_VlanVidMask = openflow_13.OfpOxmOfbField_VlanVidMask
-type OfpOxmOfbField_Ipv4SrcMask = openflow_13.OfpOxmOfbField_Ipv4SrcMask
-type OfpOxmOfbField_Ipv4DstMask = openflow_13.OfpOxmOfbField_Ipv4DstMask
-type OfpOxmOfbField_ArpSpaMask = openflow_13.OfpOxmOfbField_ArpSpaMask
-type OfpOxmOfbField_ArpTpaMask = openflow_13.OfpOxmOfbField_ArpTpaMask
-type OfpOxmOfbField_Ipv6SrcMask = openflow_13.OfpOxmOfbField_Ipv6SrcMask
-type OfpOxmOfbField_Ipv6DstMask = openflow_13.OfpOxmOfbField_Ipv6DstMask
-type OfpOxmOfbField_Ipv6FlabelMask = openflow_13.OfpOxmOfbField_Ipv6FlabelMask
-type OfpOxmOfbField_PbbIsidMask = openflow_13.OfpOxmOfbField_PbbIsidMask
-type OfpOxmOfbField_TunnelIdMask = openflow_13.OfpOxmOfbField_TunnelIdMask
-type OfpOxmOfbField_Ipv6ExthdrMask = openflow_13.OfpOxmOfbField_Ipv6ExthdrMask
-
-// OfpOxmExperimenterField from public import voltha_protos/openflow_13.proto
-type OfpOxmExperimenterField = openflow_13.OfpOxmExperimenterField
-
-// OfpAction from public import voltha_protos/openflow_13.proto
-type OfpAction = openflow_13.OfpAction
-type OfpAction_Output = openflow_13.OfpAction_Output
-type OfpAction_MplsTtl = openflow_13.OfpAction_MplsTtl
-type OfpAction_Push = openflow_13.OfpAction_Push
-type OfpAction_PopMpls = openflow_13.OfpAction_PopMpls
-type OfpAction_Group = openflow_13.OfpAction_Group
-type OfpAction_NwTtl = openflow_13.OfpAction_NwTtl
-type OfpAction_SetField = openflow_13.OfpAction_SetField
-type OfpAction_Experimenter = openflow_13.OfpAction_Experimenter
-
-// OfpActionOutput from public import voltha_protos/openflow_13.proto
-type OfpActionOutput = openflow_13.OfpActionOutput
-
-// OfpActionMplsTtl from public import voltha_protos/openflow_13.proto
-type OfpActionMplsTtl = openflow_13.OfpActionMplsTtl
-
-// OfpActionPush from public import voltha_protos/openflow_13.proto
-type OfpActionPush = openflow_13.OfpActionPush
-
-// OfpActionPopMpls from public import voltha_protos/openflow_13.proto
-type OfpActionPopMpls = openflow_13.OfpActionPopMpls
-
-// OfpActionGroup from public import voltha_protos/openflow_13.proto
-type OfpActionGroup = openflow_13.OfpActionGroup
-
-// OfpActionNwTtl from public import voltha_protos/openflow_13.proto
-type OfpActionNwTtl = openflow_13.OfpActionNwTtl
-
-// OfpActionSetField from public import voltha_protos/openflow_13.proto
-type OfpActionSetField = openflow_13.OfpActionSetField
-
-// OfpActionExperimenter from public import voltha_protos/openflow_13.proto
-type OfpActionExperimenter = openflow_13.OfpActionExperimenter
-
-// OfpInstruction from public import voltha_protos/openflow_13.proto
-type OfpInstruction = openflow_13.OfpInstruction
-type OfpInstruction_GotoTable = openflow_13.OfpInstruction_GotoTable
-type OfpInstruction_WriteMetadata = openflow_13.OfpInstruction_WriteMetadata
-type OfpInstruction_Actions = openflow_13.OfpInstruction_Actions
-type OfpInstruction_Meter = openflow_13.OfpInstruction_Meter
-type OfpInstruction_Experimenter = openflow_13.OfpInstruction_Experimenter
-
-// OfpInstructionGotoTable from public import voltha_protos/openflow_13.proto
-type OfpInstructionGotoTable = openflow_13.OfpInstructionGotoTable
-
-// OfpInstructionWriteMetadata from public import voltha_protos/openflow_13.proto
-type OfpInstructionWriteMetadata = openflow_13.OfpInstructionWriteMetadata
-
-// OfpInstructionActions from public import voltha_protos/openflow_13.proto
-type OfpInstructionActions = openflow_13.OfpInstructionActions
-
-// OfpInstructionMeter from public import voltha_protos/openflow_13.proto
-type OfpInstructionMeter = openflow_13.OfpInstructionMeter
-
-// OfpInstructionExperimenter from public import voltha_protos/openflow_13.proto
-type OfpInstructionExperimenter = openflow_13.OfpInstructionExperimenter
-
-// OfpFlowMod from public import voltha_protos/openflow_13.proto
-type OfpFlowMod = openflow_13.OfpFlowMod
-
-// OfpBucket from public import voltha_protos/openflow_13.proto
-type OfpBucket = openflow_13.OfpBucket
-
-// OfpGroupMod from public import voltha_protos/openflow_13.proto
-type OfpGroupMod = openflow_13.OfpGroupMod
-
-// OfpPacketOut from public import voltha_protos/openflow_13.proto
-type OfpPacketOut = openflow_13.OfpPacketOut
-
-// OfpPacketIn from public import voltha_protos/openflow_13.proto
-type OfpPacketIn = openflow_13.OfpPacketIn
-
-// OfpFlowRemoved from public import voltha_protos/openflow_13.proto
-type OfpFlowRemoved = openflow_13.OfpFlowRemoved
-
-// OfpMeterBandHeader from public import voltha_protos/openflow_13.proto
-type OfpMeterBandHeader = openflow_13.OfpMeterBandHeader
-type OfpMeterBandHeader_Drop = openflow_13.OfpMeterBandHeader_Drop
-type OfpMeterBandHeader_DscpRemark = openflow_13.OfpMeterBandHeader_DscpRemark
-type OfpMeterBandHeader_Experimenter = openflow_13.OfpMeterBandHeader_Experimenter
-
-// OfpMeterBandDrop from public import voltha_protos/openflow_13.proto
-type OfpMeterBandDrop = openflow_13.OfpMeterBandDrop
-
-// OfpMeterBandDscpRemark from public import voltha_protos/openflow_13.proto
-type OfpMeterBandDscpRemark = openflow_13.OfpMeterBandDscpRemark
-
-// OfpMeterBandExperimenter from public import voltha_protos/openflow_13.proto
-type OfpMeterBandExperimenter = openflow_13.OfpMeterBandExperimenter
-
-// OfpMeterMod from public import voltha_protos/openflow_13.proto
-type OfpMeterMod = openflow_13.OfpMeterMod
-
-// OfpErrorMsg from public import voltha_protos/openflow_13.proto
-type OfpErrorMsg = openflow_13.OfpErrorMsg
-
-// OfpErrorExperimenterMsg from public import voltha_protos/openflow_13.proto
-type OfpErrorExperimenterMsg = openflow_13.OfpErrorExperimenterMsg
-
-// OfpMultipartRequest from public import voltha_protos/openflow_13.proto
-type OfpMultipartRequest = openflow_13.OfpMultipartRequest
-
-// OfpMultipartReply from public import voltha_protos/openflow_13.proto
-type OfpMultipartReply = openflow_13.OfpMultipartReply
-
-// OfpDesc from public import voltha_protos/openflow_13.proto
-type OfpDesc = openflow_13.OfpDesc
-
-// OfpFlowStatsRequest from public import voltha_protos/openflow_13.proto
-type OfpFlowStatsRequest = openflow_13.OfpFlowStatsRequest
-
-// OfpFlowStats from public import voltha_protos/openflow_13.proto
-type OfpFlowStats = openflow_13.OfpFlowStats
-
-// OfpAggregateStatsRequest from public import voltha_protos/openflow_13.proto
-type OfpAggregateStatsRequest = openflow_13.OfpAggregateStatsRequest
-
-// OfpAggregateStatsReply from public import voltha_protos/openflow_13.proto
-type OfpAggregateStatsReply = openflow_13.OfpAggregateStatsReply
-
-// OfpTableFeatureProperty from public import voltha_protos/openflow_13.proto
-type OfpTableFeatureProperty = openflow_13.OfpTableFeatureProperty
-type OfpTableFeatureProperty_Instructions = openflow_13.OfpTableFeatureProperty_Instructions
-type OfpTableFeatureProperty_NextTables = openflow_13.OfpTableFeatureProperty_NextTables
-type OfpTableFeatureProperty_Actions = openflow_13.OfpTableFeatureProperty_Actions
-type OfpTableFeatureProperty_Oxm = openflow_13.OfpTableFeatureProperty_Oxm
-type OfpTableFeatureProperty_Experimenter = openflow_13.OfpTableFeatureProperty_Experimenter
-
-// OfpTableFeaturePropInstructions from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropInstructions = openflow_13.OfpTableFeaturePropInstructions
-
-// OfpTableFeaturePropNextTables from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropNextTables = openflow_13.OfpTableFeaturePropNextTables
-
-// OfpTableFeaturePropActions from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropActions = openflow_13.OfpTableFeaturePropActions
-
-// OfpTableFeaturePropOxm from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropOxm = openflow_13.OfpTableFeaturePropOxm
-
-// OfpTableFeaturePropExperimenter from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropExperimenter = openflow_13.OfpTableFeaturePropExperimenter
-
-// OfpTableFeatures from public import voltha_protos/openflow_13.proto
-type OfpTableFeatures = openflow_13.OfpTableFeatures
-
-// OfpTableStats from public import voltha_protos/openflow_13.proto
-type OfpTableStats = openflow_13.OfpTableStats
-
-// OfpPortStatsRequest from public import voltha_protos/openflow_13.proto
-type OfpPortStatsRequest = openflow_13.OfpPortStatsRequest
-
-// OfpPortStats from public import voltha_protos/openflow_13.proto
-type OfpPortStats = openflow_13.OfpPortStats
-
-// OfpGroupStatsRequest from public import voltha_protos/openflow_13.proto
-type OfpGroupStatsRequest = openflow_13.OfpGroupStatsRequest
-
-// OfpBucketCounter from public import voltha_protos/openflow_13.proto
-type OfpBucketCounter = openflow_13.OfpBucketCounter
-
-// OfpGroupStats from public import voltha_protos/openflow_13.proto
-type OfpGroupStats = openflow_13.OfpGroupStats
-
-// OfpGroupDesc from public import voltha_protos/openflow_13.proto
-type OfpGroupDesc = openflow_13.OfpGroupDesc
-
-// OfpGroupEntry from public import voltha_protos/openflow_13.proto
-type OfpGroupEntry = openflow_13.OfpGroupEntry
-
-// OfpGroupFeatures from public import voltha_protos/openflow_13.proto
-type OfpGroupFeatures = openflow_13.OfpGroupFeatures
-
-// OfpMeterMultipartRequest from public import voltha_protos/openflow_13.proto
-type OfpMeterMultipartRequest = openflow_13.OfpMeterMultipartRequest
-
-// OfpMeterBandStats from public import voltha_protos/openflow_13.proto
-type OfpMeterBandStats = openflow_13.OfpMeterBandStats
-
-// OfpMeterStats from public import voltha_protos/openflow_13.proto
-type OfpMeterStats = openflow_13.OfpMeterStats
-
-// OfpMeterConfig from public import voltha_protos/openflow_13.proto
-type OfpMeterConfig = openflow_13.OfpMeterConfig
-
-// OfpMeterFeatures from public import voltha_protos/openflow_13.proto
-type OfpMeterFeatures = openflow_13.OfpMeterFeatures
-
-// OfpMeterEntry from public import voltha_protos/openflow_13.proto
-type OfpMeterEntry = openflow_13.OfpMeterEntry
-
-// OfpExperimenterMultipartHeader from public import voltha_protos/openflow_13.proto
-type OfpExperimenterMultipartHeader = openflow_13.OfpExperimenterMultipartHeader
-
-// OfpExperimenterHeader from public import voltha_protos/openflow_13.proto
-type OfpExperimenterHeader = openflow_13.OfpExperimenterHeader
-
-// OfpQueuePropHeader from public import voltha_protos/openflow_13.proto
-type OfpQueuePropHeader = openflow_13.OfpQueuePropHeader
-
-// OfpQueuePropMinRate from public import voltha_protos/openflow_13.proto
-type OfpQueuePropMinRate = openflow_13.OfpQueuePropMinRate
-
-// OfpQueuePropMaxRate from public import voltha_protos/openflow_13.proto
-type OfpQueuePropMaxRate = openflow_13.OfpQueuePropMaxRate
-
-// OfpQueuePropExperimenter from public import voltha_protos/openflow_13.proto
-type OfpQueuePropExperimenter = openflow_13.OfpQueuePropExperimenter
-
-// OfpPacketQueue from public import voltha_protos/openflow_13.proto
-type OfpPacketQueue = openflow_13.OfpPacketQueue
-
-// OfpQueueGetConfigRequest from public import voltha_protos/openflow_13.proto
-type OfpQueueGetConfigRequest = openflow_13.OfpQueueGetConfigRequest
-
-// OfpQueueGetConfigReply from public import voltha_protos/openflow_13.proto
-type OfpQueueGetConfigReply = openflow_13.OfpQueueGetConfigReply
-
-// OfpActionSetQueue from public import voltha_protos/openflow_13.proto
-type OfpActionSetQueue = openflow_13.OfpActionSetQueue
-
-// OfpQueueStatsRequest from public import voltha_protos/openflow_13.proto
-type OfpQueueStatsRequest = openflow_13.OfpQueueStatsRequest
-
-// OfpQueueStats from public import voltha_protos/openflow_13.proto
-type OfpQueueStats = openflow_13.OfpQueueStats
-
-// OfpRoleRequest from public import voltha_protos/openflow_13.proto
-type OfpRoleRequest = openflow_13.OfpRoleRequest
-
-// OfpAsyncConfig from public import voltha_protos/openflow_13.proto
-type OfpAsyncConfig = openflow_13.OfpAsyncConfig
-
-// MeterModUpdate from public import voltha_protos/openflow_13.proto
-type MeterModUpdate = openflow_13.MeterModUpdate
-
-// MeterStatsReply from public import voltha_protos/openflow_13.proto
-type MeterStatsReply = openflow_13.MeterStatsReply
-
-// FlowTableUpdate from public import voltha_protos/openflow_13.proto
-type FlowTableUpdate = openflow_13.FlowTableUpdate
-
-// FlowGroupTableUpdate from public import voltha_protos/openflow_13.proto
-type FlowGroupTableUpdate = openflow_13.FlowGroupTableUpdate
-
-// Flows from public import voltha_protos/openflow_13.proto
-type Flows = openflow_13.Flows
-
-// Meters from public import voltha_protos/openflow_13.proto
-type Meters = openflow_13.Meters
-
-// FlowGroups from public import voltha_protos/openflow_13.proto
-type FlowGroups = openflow_13.FlowGroups
-
-// FlowChanges from public import voltha_protos/openflow_13.proto
-type FlowChanges = openflow_13.FlowChanges
-
-// FlowGroupChanges from public import voltha_protos/openflow_13.proto
-type FlowGroupChanges = openflow_13.FlowGroupChanges
-
-// PacketIn from public import voltha_protos/openflow_13.proto
-type PacketIn = openflow_13.PacketIn
-
-// PacketOut from public import voltha_protos/openflow_13.proto
-type PacketOut = openflow_13.PacketOut
-
-// ChangeEvent from public import voltha_protos/openflow_13.proto
-type ChangeEvent = openflow_13.ChangeEvent
-type ChangeEvent_PortStatus = openflow_13.ChangeEvent_PortStatus
-type ChangeEvent_Error = openflow_13.ChangeEvent_Error
-
-// OfpPortNo from public import voltha_protos/openflow_13.proto
-type OfpPortNo = openflow_13.OfpPortNo
-
-var OfpPortNo_name = openflow_13.OfpPortNo_name
-var OfpPortNo_value = openflow_13.OfpPortNo_value
-
-const OfpPortNo_OFPP_INVALID = OfpPortNo(openflow_13.OfpPortNo_OFPP_INVALID)
-const OfpPortNo_OFPP_MAX = OfpPortNo(openflow_13.OfpPortNo_OFPP_MAX)
-const OfpPortNo_OFPP_IN_PORT = OfpPortNo(openflow_13.OfpPortNo_OFPP_IN_PORT)
-const OfpPortNo_OFPP_TABLE = OfpPortNo(openflow_13.OfpPortNo_OFPP_TABLE)
-const OfpPortNo_OFPP_NORMAL = OfpPortNo(openflow_13.OfpPortNo_OFPP_NORMAL)
-const OfpPortNo_OFPP_FLOOD = OfpPortNo(openflow_13.OfpPortNo_OFPP_FLOOD)
-const OfpPortNo_OFPP_ALL = OfpPortNo(openflow_13.OfpPortNo_OFPP_ALL)
-const OfpPortNo_OFPP_CONTROLLER = OfpPortNo(openflow_13.OfpPortNo_OFPP_CONTROLLER)
-const OfpPortNo_OFPP_LOCAL = OfpPortNo(openflow_13.OfpPortNo_OFPP_LOCAL)
-const OfpPortNo_OFPP_ANY = OfpPortNo(openflow_13.OfpPortNo_OFPP_ANY)
-
-// OfpType from public import voltha_protos/openflow_13.proto
-type OfpType = openflow_13.OfpType
-
-var OfpType_name = openflow_13.OfpType_name
-var OfpType_value = openflow_13.OfpType_value
-
-const OfpType_OFPT_HELLO = OfpType(openflow_13.OfpType_OFPT_HELLO)
-const OfpType_OFPT_ERROR = OfpType(openflow_13.OfpType_OFPT_ERROR)
-const OfpType_OFPT_ECHO_REQUEST = OfpType(openflow_13.OfpType_OFPT_ECHO_REQUEST)
-const OfpType_OFPT_ECHO_REPLY = OfpType(openflow_13.OfpType_OFPT_ECHO_REPLY)
-const OfpType_OFPT_EXPERIMENTER = OfpType(openflow_13.OfpType_OFPT_EXPERIMENTER)
-const OfpType_OFPT_FEATURES_REQUEST = OfpType(openflow_13.OfpType_OFPT_FEATURES_REQUEST)
-const OfpType_OFPT_FEATURES_REPLY = OfpType(openflow_13.OfpType_OFPT_FEATURES_REPLY)
-const OfpType_OFPT_GET_CONFIG_REQUEST = OfpType(openflow_13.OfpType_OFPT_GET_CONFIG_REQUEST)
-const OfpType_OFPT_GET_CONFIG_REPLY = OfpType(openflow_13.OfpType_OFPT_GET_CONFIG_REPLY)
-const OfpType_OFPT_SET_CONFIG = OfpType(openflow_13.OfpType_OFPT_SET_CONFIG)
-const OfpType_OFPT_PACKET_IN = OfpType(openflow_13.OfpType_OFPT_PACKET_IN)
-const OfpType_OFPT_FLOW_REMOVED = OfpType(openflow_13.OfpType_OFPT_FLOW_REMOVED)
-const OfpType_OFPT_PORT_STATUS = OfpType(openflow_13.OfpType_OFPT_PORT_STATUS)
-const OfpType_OFPT_PACKET_OUT = OfpType(openflow_13.OfpType_OFPT_PACKET_OUT)
-const OfpType_OFPT_FLOW_MOD = OfpType(openflow_13.OfpType_OFPT_FLOW_MOD)
-const OfpType_OFPT_GROUP_MOD = OfpType(openflow_13.OfpType_OFPT_GROUP_MOD)
-const OfpType_OFPT_PORT_MOD = OfpType(openflow_13.OfpType_OFPT_PORT_MOD)
-const OfpType_OFPT_TABLE_MOD = OfpType(openflow_13.OfpType_OFPT_TABLE_MOD)
-const OfpType_OFPT_MULTIPART_REQUEST = OfpType(openflow_13.OfpType_OFPT_MULTIPART_REQUEST)
-const OfpType_OFPT_MULTIPART_REPLY = OfpType(openflow_13.OfpType_OFPT_MULTIPART_REPLY)
-const OfpType_OFPT_BARRIER_REQUEST = OfpType(openflow_13.OfpType_OFPT_BARRIER_REQUEST)
-const OfpType_OFPT_BARRIER_REPLY = OfpType(openflow_13.OfpType_OFPT_BARRIER_REPLY)
-const OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST = OfpType(openflow_13.OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST)
-const OfpType_OFPT_QUEUE_GET_CONFIG_REPLY = OfpType(openflow_13.OfpType_OFPT_QUEUE_GET_CONFIG_REPLY)
-const OfpType_OFPT_ROLE_REQUEST = OfpType(openflow_13.OfpType_OFPT_ROLE_REQUEST)
-const OfpType_OFPT_ROLE_REPLY = OfpType(openflow_13.OfpType_OFPT_ROLE_REPLY)
-const OfpType_OFPT_GET_ASYNC_REQUEST = OfpType(openflow_13.OfpType_OFPT_GET_ASYNC_REQUEST)
-const OfpType_OFPT_GET_ASYNC_REPLY = OfpType(openflow_13.OfpType_OFPT_GET_ASYNC_REPLY)
-const OfpType_OFPT_SET_ASYNC = OfpType(openflow_13.OfpType_OFPT_SET_ASYNC)
-const OfpType_OFPT_METER_MOD = OfpType(openflow_13.OfpType_OFPT_METER_MOD)
-
-// OfpHelloElemType from public import voltha_protos/openflow_13.proto
-type OfpHelloElemType = openflow_13.OfpHelloElemType
-
-var OfpHelloElemType_name = openflow_13.OfpHelloElemType_name
-var OfpHelloElemType_value = openflow_13.OfpHelloElemType_value
-
-const OfpHelloElemType_OFPHET_INVALID = OfpHelloElemType(openflow_13.OfpHelloElemType_OFPHET_INVALID)
-const OfpHelloElemType_OFPHET_VERSIONBITMAP = OfpHelloElemType(openflow_13.OfpHelloElemType_OFPHET_VERSIONBITMAP)
-
-// OfpConfigFlags from public import voltha_protos/openflow_13.proto
-type OfpConfigFlags = openflow_13.OfpConfigFlags
-
-var OfpConfigFlags_name = openflow_13.OfpConfigFlags_name
-var OfpConfigFlags_value = openflow_13.OfpConfigFlags_value
-
-const OfpConfigFlags_OFPC_FRAG_NORMAL = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_NORMAL)
-const OfpConfigFlags_OFPC_FRAG_DROP = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_DROP)
-const OfpConfigFlags_OFPC_FRAG_REASM = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_REASM)
-const OfpConfigFlags_OFPC_FRAG_MASK = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_MASK)
-
-// OfpTableConfig from public import voltha_protos/openflow_13.proto
-type OfpTableConfig = openflow_13.OfpTableConfig
-
-var OfpTableConfig_name = openflow_13.OfpTableConfig_name
-var OfpTableConfig_value = openflow_13.OfpTableConfig_value
-
-const OfpTableConfig_OFPTC_INVALID = OfpTableConfig(openflow_13.OfpTableConfig_OFPTC_INVALID)
-const OfpTableConfig_OFPTC_DEPRECATED_MASK = OfpTableConfig(openflow_13.OfpTableConfig_OFPTC_DEPRECATED_MASK)
-
-// OfpTable from public import voltha_protos/openflow_13.proto
-type OfpTable = openflow_13.OfpTable
-
-var OfpTable_name = openflow_13.OfpTable_name
-var OfpTable_value = openflow_13.OfpTable_value
-
-const OfpTable_OFPTT_INVALID = OfpTable(openflow_13.OfpTable_OFPTT_INVALID)
-const OfpTable_OFPTT_MAX = OfpTable(openflow_13.OfpTable_OFPTT_MAX)
-const OfpTable_OFPTT_ALL = OfpTable(openflow_13.OfpTable_OFPTT_ALL)
-
-// OfpCapabilities from public import voltha_protos/openflow_13.proto
-type OfpCapabilities = openflow_13.OfpCapabilities
-
-var OfpCapabilities_name = openflow_13.OfpCapabilities_name
-var OfpCapabilities_value = openflow_13.OfpCapabilities_value
-
-const OfpCapabilities_OFPC_INVALID = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_INVALID)
-const OfpCapabilities_OFPC_FLOW_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_FLOW_STATS)
-const OfpCapabilities_OFPC_TABLE_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_TABLE_STATS)
-const OfpCapabilities_OFPC_PORT_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_PORT_STATS)
-const OfpCapabilities_OFPC_GROUP_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_GROUP_STATS)
-const OfpCapabilities_OFPC_IP_REASM = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_IP_REASM)
-const OfpCapabilities_OFPC_QUEUE_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_QUEUE_STATS)
-const OfpCapabilities_OFPC_PORT_BLOCKED = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_PORT_BLOCKED)
-
-// OfpPortConfig from public import voltha_protos/openflow_13.proto
-type OfpPortConfig = openflow_13.OfpPortConfig
-
-var OfpPortConfig_name = openflow_13.OfpPortConfig_name
-var OfpPortConfig_value = openflow_13.OfpPortConfig_value
-
-const OfpPortConfig_OFPPC_INVALID = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_INVALID)
-const OfpPortConfig_OFPPC_PORT_DOWN = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_PORT_DOWN)
-const OfpPortConfig_OFPPC_NO_RECV = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_RECV)
-const OfpPortConfig_OFPPC_NO_FWD = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_FWD)
-const OfpPortConfig_OFPPC_NO_PACKET_IN = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_PACKET_IN)
-
-// OfpPortState from public import voltha_protos/openflow_13.proto
-type OfpPortState = openflow_13.OfpPortState
-
-var OfpPortState_name = openflow_13.OfpPortState_name
-var OfpPortState_value = openflow_13.OfpPortState_value
-
-const OfpPortState_OFPPS_INVALID = OfpPortState(openflow_13.OfpPortState_OFPPS_INVALID)
-const OfpPortState_OFPPS_LINK_DOWN = OfpPortState(openflow_13.OfpPortState_OFPPS_LINK_DOWN)
-const OfpPortState_OFPPS_BLOCKED = OfpPortState(openflow_13.OfpPortState_OFPPS_BLOCKED)
-const OfpPortState_OFPPS_LIVE = OfpPortState(openflow_13.OfpPortState_OFPPS_LIVE)
-
-// OfpPortFeatures from public import voltha_protos/openflow_13.proto
-type OfpPortFeatures = openflow_13.OfpPortFeatures
-
-var OfpPortFeatures_name = openflow_13.OfpPortFeatures_name
-var OfpPortFeatures_value = openflow_13.OfpPortFeatures_value
-
-const OfpPortFeatures_OFPPF_INVALID = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_INVALID)
-const OfpPortFeatures_OFPPF_10MB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10MB_HD)
-const OfpPortFeatures_OFPPF_10MB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10MB_FD)
-const OfpPortFeatures_OFPPF_100MB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100MB_HD)
-const OfpPortFeatures_OFPPF_100MB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100MB_FD)
-const OfpPortFeatures_OFPPF_1GB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1GB_HD)
-const OfpPortFeatures_OFPPF_1GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1GB_FD)
-const OfpPortFeatures_OFPPF_10GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10GB_FD)
-const OfpPortFeatures_OFPPF_40GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_40GB_FD)
-const OfpPortFeatures_OFPPF_100GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100GB_FD)
-const OfpPortFeatures_OFPPF_1TB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1TB_FD)
-const OfpPortFeatures_OFPPF_OTHER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_OTHER)
-const OfpPortFeatures_OFPPF_COPPER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_COPPER)
-const OfpPortFeatures_OFPPF_FIBER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_FIBER)
-const OfpPortFeatures_OFPPF_AUTONEG = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_AUTONEG)
-const OfpPortFeatures_OFPPF_PAUSE = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_PAUSE)
-const OfpPortFeatures_OFPPF_PAUSE_ASYM = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_PAUSE_ASYM)
-
-// OfpPortReason from public import voltha_protos/openflow_13.proto
-type OfpPortReason = openflow_13.OfpPortReason
-
-var OfpPortReason_name = openflow_13.OfpPortReason_name
-var OfpPortReason_value = openflow_13.OfpPortReason_value
-
-const OfpPortReason_OFPPR_ADD = OfpPortReason(openflow_13.OfpPortReason_OFPPR_ADD)
-const OfpPortReason_OFPPR_DELETE = OfpPortReason(openflow_13.OfpPortReason_OFPPR_DELETE)
-const OfpPortReason_OFPPR_MODIFY = OfpPortReason(openflow_13.OfpPortReason_OFPPR_MODIFY)
-
-// OfpMatchType from public import voltha_protos/openflow_13.proto
-type OfpMatchType = openflow_13.OfpMatchType
-
-var OfpMatchType_name = openflow_13.OfpMatchType_name
-var OfpMatchType_value = openflow_13.OfpMatchType_value
-
-const OfpMatchType_OFPMT_STANDARD = OfpMatchType(openflow_13.OfpMatchType_OFPMT_STANDARD)
-const OfpMatchType_OFPMT_OXM = OfpMatchType(openflow_13.OfpMatchType_OFPMT_OXM)
-
-// OfpOxmClass from public import voltha_protos/openflow_13.proto
-type OfpOxmClass = openflow_13.OfpOxmClass
-
-var OfpOxmClass_name = openflow_13.OfpOxmClass_name
-var OfpOxmClass_value = openflow_13.OfpOxmClass_value
-
-const OfpOxmClass_OFPXMC_NXM_0 = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_NXM_0)
-const OfpOxmClass_OFPXMC_NXM_1 = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_NXM_1)
-const OfpOxmClass_OFPXMC_OPENFLOW_BASIC = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_OPENFLOW_BASIC)
-const OfpOxmClass_OFPXMC_EXPERIMENTER = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_EXPERIMENTER)
-
-// OxmOfbFieldTypes from public import voltha_protos/openflow_13.proto
-type OxmOfbFieldTypes = openflow_13.OxmOfbFieldTypes
-
-var OxmOfbFieldTypes_name = openflow_13.OxmOfbFieldTypes_name
-var OxmOfbFieldTypes_value = openflow_13.OxmOfbFieldTypes_value
-
-const OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT)
-const OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT)
-const OxmOfbFieldTypes_OFPXMT_OFB_METADATA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_METADATA)
-const OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE)
-const OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID)
-const OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP)
-const OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP)
-const OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN)
-const OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE)
-const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE)
-const OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP)
-const OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA)
-const OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA)
-const OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA)
-const OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL)
-const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE)
-const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL)
-const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL)
-const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC)
-const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS)
-const OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID)
-const OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID)
-const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR)
-
-// OfpVlanId from public import voltha_protos/openflow_13.proto
-type OfpVlanId = openflow_13.OfpVlanId
-
-var OfpVlanId_name = openflow_13.OfpVlanId_name
-var OfpVlanId_value = openflow_13.OfpVlanId_value
-
-const OfpVlanId_OFPVID_NONE = OfpVlanId(openflow_13.OfpVlanId_OFPVID_NONE)
-const OfpVlanId_OFPVID_PRESENT = OfpVlanId(openflow_13.OfpVlanId_OFPVID_PRESENT)
-
-// OfpIpv6ExthdrFlags from public import voltha_protos/openflow_13.proto
-type OfpIpv6ExthdrFlags = openflow_13.OfpIpv6ExthdrFlags
-
-var OfpIpv6ExthdrFlags_name = openflow_13.OfpIpv6ExthdrFlags_name
-var OfpIpv6ExthdrFlags_value = openflow_13.OfpIpv6ExthdrFlags_value
-
-const OfpIpv6ExthdrFlags_OFPIEH_INVALID = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_INVALID)
-const OfpIpv6ExthdrFlags_OFPIEH_NONEXT = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_NONEXT)
-const OfpIpv6ExthdrFlags_OFPIEH_ESP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_ESP)
-const OfpIpv6ExthdrFlags_OFPIEH_AUTH = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_AUTH)
-const OfpIpv6ExthdrFlags_OFPIEH_DEST = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_DEST)
-const OfpIpv6ExthdrFlags_OFPIEH_FRAG = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_FRAG)
-const OfpIpv6ExthdrFlags_OFPIEH_ROUTER = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_ROUTER)
-const OfpIpv6ExthdrFlags_OFPIEH_HOP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_HOP)
-const OfpIpv6ExthdrFlags_OFPIEH_UNREP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_UNREP)
-const OfpIpv6ExthdrFlags_OFPIEH_UNSEQ = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_UNSEQ)
-
-// OfpActionType from public import voltha_protos/openflow_13.proto
-type OfpActionType = openflow_13.OfpActionType
-
-var OfpActionType_name = openflow_13.OfpActionType_name
-var OfpActionType_value = openflow_13.OfpActionType_value
-
-const OfpActionType_OFPAT_OUTPUT = OfpActionType(openflow_13.OfpActionType_OFPAT_OUTPUT)
-const OfpActionType_OFPAT_COPY_TTL_OUT = OfpActionType(openflow_13.OfpActionType_OFPAT_COPY_TTL_OUT)
-const OfpActionType_OFPAT_COPY_TTL_IN = OfpActionType(openflow_13.OfpActionType_OFPAT_COPY_TTL_IN)
-const OfpActionType_OFPAT_SET_MPLS_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_MPLS_TTL)
-const OfpActionType_OFPAT_DEC_MPLS_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_DEC_MPLS_TTL)
-const OfpActionType_OFPAT_PUSH_VLAN = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_VLAN)
-const OfpActionType_OFPAT_POP_VLAN = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_VLAN)
-const OfpActionType_OFPAT_PUSH_MPLS = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_MPLS)
-const OfpActionType_OFPAT_POP_MPLS = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_MPLS)
-const OfpActionType_OFPAT_SET_QUEUE = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_QUEUE)
-const OfpActionType_OFPAT_GROUP = OfpActionType(openflow_13.OfpActionType_OFPAT_GROUP)
-const OfpActionType_OFPAT_SET_NW_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_NW_TTL)
-const OfpActionType_OFPAT_DEC_NW_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_DEC_NW_TTL)
-const OfpActionType_OFPAT_SET_FIELD = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_FIELD)
-const OfpActionType_OFPAT_PUSH_PBB = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_PBB)
-const OfpActionType_OFPAT_POP_PBB = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_PBB)
-const OfpActionType_OFPAT_EXPERIMENTER = OfpActionType(openflow_13.OfpActionType_OFPAT_EXPERIMENTER)
-
-// OfpControllerMaxLen from public import voltha_protos/openflow_13.proto
-type OfpControllerMaxLen = openflow_13.OfpControllerMaxLen
-
-var OfpControllerMaxLen_name = openflow_13.OfpControllerMaxLen_name
-var OfpControllerMaxLen_value = openflow_13.OfpControllerMaxLen_value
-
-const OfpControllerMaxLen_OFPCML_INVALID = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_INVALID)
-const OfpControllerMaxLen_OFPCML_MAX = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_MAX)
-const OfpControllerMaxLen_OFPCML_NO_BUFFER = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_NO_BUFFER)
-
-// OfpInstructionType from public import voltha_protos/openflow_13.proto
-type OfpInstructionType = openflow_13.OfpInstructionType
-
-var OfpInstructionType_name = openflow_13.OfpInstructionType_name
-var OfpInstructionType_value = openflow_13.OfpInstructionType_value
-
-const OfpInstructionType_OFPIT_INVALID = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_INVALID)
-const OfpInstructionType_OFPIT_GOTO_TABLE = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_GOTO_TABLE)
-const OfpInstructionType_OFPIT_WRITE_METADATA = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_WRITE_METADATA)
-const OfpInstructionType_OFPIT_WRITE_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_WRITE_ACTIONS)
-const OfpInstructionType_OFPIT_APPLY_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_APPLY_ACTIONS)
-const OfpInstructionType_OFPIT_CLEAR_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_CLEAR_ACTIONS)
-const OfpInstructionType_OFPIT_METER = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_METER)
-const OfpInstructionType_OFPIT_EXPERIMENTER = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_EXPERIMENTER)
-
-// OfpFlowModCommand from public import voltha_protos/openflow_13.proto
-type OfpFlowModCommand = openflow_13.OfpFlowModCommand
-
-var OfpFlowModCommand_name = openflow_13.OfpFlowModCommand_name
-var OfpFlowModCommand_value = openflow_13.OfpFlowModCommand_value
-
-const OfpFlowModCommand_OFPFC_ADD = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_ADD)
-const OfpFlowModCommand_OFPFC_MODIFY = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_MODIFY)
-const OfpFlowModCommand_OFPFC_MODIFY_STRICT = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_MODIFY_STRICT)
-const OfpFlowModCommand_OFPFC_DELETE = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_DELETE)
-const OfpFlowModCommand_OFPFC_DELETE_STRICT = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_DELETE_STRICT)
-
-// OfpFlowModFlags from public import voltha_protos/openflow_13.proto
-type OfpFlowModFlags = openflow_13.OfpFlowModFlags
-
-var OfpFlowModFlags_name = openflow_13.OfpFlowModFlags_name
-var OfpFlowModFlags_value = openflow_13.OfpFlowModFlags_value
-
-const OfpFlowModFlags_OFPFF_INVALID = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_INVALID)
-const OfpFlowModFlags_OFPFF_SEND_FLOW_REM = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_SEND_FLOW_REM)
-const OfpFlowModFlags_OFPFF_CHECK_OVERLAP = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_CHECK_OVERLAP)
-const OfpFlowModFlags_OFPFF_RESET_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_RESET_COUNTS)
-const OfpFlowModFlags_OFPFF_NO_PKT_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_NO_PKT_COUNTS)
-const OfpFlowModFlags_OFPFF_NO_BYT_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_NO_BYT_COUNTS)
-
-// OfpGroup from public import voltha_protos/openflow_13.proto
-type OfpGroup = openflow_13.OfpGroup
-
-var OfpGroup_name = openflow_13.OfpGroup_name
-var OfpGroup_value = openflow_13.OfpGroup_value
-
-const OfpGroup_OFPG_INVALID = OfpGroup(openflow_13.OfpGroup_OFPG_INVALID)
-const OfpGroup_OFPG_MAX = OfpGroup(openflow_13.OfpGroup_OFPG_MAX)
-const OfpGroup_OFPG_ALL = OfpGroup(openflow_13.OfpGroup_OFPG_ALL)
-const OfpGroup_OFPG_ANY = OfpGroup(openflow_13.OfpGroup_OFPG_ANY)
-
-// OfpGroupModCommand from public import voltha_protos/openflow_13.proto
-type OfpGroupModCommand = openflow_13.OfpGroupModCommand
-
-var OfpGroupModCommand_name = openflow_13.OfpGroupModCommand_name
-var OfpGroupModCommand_value = openflow_13.OfpGroupModCommand_value
-
-const OfpGroupModCommand_OFPGC_ADD = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_ADD)
-const OfpGroupModCommand_OFPGC_MODIFY = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_MODIFY)
-const OfpGroupModCommand_OFPGC_DELETE = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_DELETE)
-
-// OfpGroupType from public import voltha_protos/openflow_13.proto
-type OfpGroupType = openflow_13.OfpGroupType
-
-var OfpGroupType_name = openflow_13.OfpGroupType_name
-var OfpGroupType_value = openflow_13.OfpGroupType_value
-
-const OfpGroupType_OFPGT_ALL = OfpGroupType(openflow_13.OfpGroupType_OFPGT_ALL)
-const OfpGroupType_OFPGT_SELECT = OfpGroupType(openflow_13.OfpGroupType_OFPGT_SELECT)
-const OfpGroupType_OFPGT_INDIRECT = OfpGroupType(openflow_13.OfpGroupType_OFPGT_INDIRECT)
-const OfpGroupType_OFPGT_FF = OfpGroupType(openflow_13.OfpGroupType_OFPGT_FF)
-
-// OfpPacketInReason from public import voltha_protos/openflow_13.proto
-type OfpPacketInReason = openflow_13.OfpPacketInReason
-
-var OfpPacketInReason_name = openflow_13.OfpPacketInReason_name
-var OfpPacketInReason_value = openflow_13.OfpPacketInReason_value
-
-const OfpPacketInReason_OFPR_NO_MATCH = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_NO_MATCH)
-const OfpPacketInReason_OFPR_ACTION = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_ACTION)
-const OfpPacketInReason_OFPR_INVALID_TTL = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_INVALID_TTL)
-
-// OfpFlowRemovedReason from public import voltha_protos/openflow_13.proto
-type OfpFlowRemovedReason = openflow_13.OfpFlowRemovedReason
-
-var OfpFlowRemovedReason_name = openflow_13.OfpFlowRemovedReason_name
-var OfpFlowRemovedReason_value = openflow_13.OfpFlowRemovedReason_value
-
-const OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT)
-const OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT)
-const OfpFlowRemovedReason_OFPRR_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_DELETE)
-const OfpFlowRemovedReason_OFPRR_GROUP_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_GROUP_DELETE)
-const OfpFlowRemovedReason_OFPRR_METER_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_METER_DELETE)
-
-// OfpMeter from public import voltha_protos/openflow_13.proto
-type OfpMeter = openflow_13.OfpMeter
-
-var OfpMeter_name = openflow_13.OfpMeter_name
-var OfpMeter_value = openflow_13.OfpMeter_value
-
-const OfpMeter_OFPM_ZERO = OfpMeter(openflow_13.OfpMeter_OFPM_ZERO)
-const OfpMeter_OFPM_MAX = OfpMeter(openflow_13.OfpMeter_OFPM_MAX)
-const OfpMeter_OFPM_SLOWPATH = OfpMeter(openflow_13.OfpMeter_OFPM_SLOWPATH)
-const OfpMeter_OFPM_CONTROLLER = OfpMeter(openflow_13.OfpMeter_OFPM_CONTROLLER)
-const OfpMeter_OFPM_ALL = OfpMeter(openflow_13.OfpMeter_OFPM_ALL)
-
-// OfpMeterBandType from public import voltha_protos/openflow_13.proto
-type OfpMeterBandType = openflow_13.OfpMeterBandType
-
-var OfpMeterBandType_name = openflow_13.OfpMeterBandType_name
-var OfpMeterBandType_value = openflow_13.OfpMeterBandType_value
-
-const OfpMeterBandType_OFPMBT_INVALID = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_INVALID)
-const OfpMeterBandType_OFPMBT_DROP = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_DROP)
-const OfpMeterBandType_OFPMBT_DSCP_REMARK = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_DSCP_REMARK)
-const OfpMeterBandType_OFPMBT_EXPERIMENTER = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_EXPERIMENTER)
-
-// OfpMeterModCommand from public import voltha_protos/openflow_13.proto
-type OfpMeterModCommand = openflow_13.OfpMeterModCommand
-
-var OfpMeterModCommand_name = openflow_13.OfpMeterModCommand_name
-var OfpMeterModCommand_value = openflow_13.OfpMeterModCommand_value
-
-const OfpMeterModCommand_OFPMC_ADD = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_ADD)
-const OfpMeterModCommand_OFPMC_MODIFY = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_MODIFY)
-const OfpMeterModCommand_OFPMC_DELETE = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_DELETE)
-
-// OfpMeterFlags from public import voltha_protos/openflow_13.proto
-type OfpMeterFlags = openflow_13.OfpMeterFlags
-
-var OfpMeterFlags_name = openflow_13.OfpMeterFlags_name
-var OfpMeterFlags_value = openflow_13.OfpMeterFlags_value
-
-const OfpMeterFlags_OFPMF_INVALID = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_INVALID)
-const OfpMeterFlags_OFPMF_KBPS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_KBPS)
-const OfpMeterFlags_OFPMF_PKTPS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_PKTPS)
-const OfpMeterFlags_OFPMF_BURST = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_BURST)
-const OfpMeterFlags_OFPMF_STATS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_STATS)
-
-// OfpErrorType from public import voltha_protos/openflow_13.proto
-type OfpErrorType = openflow_13.OfpErrorType
-
-var OfpErrorType_name = openflow_13.OfpErrorType_name
-var OfpErrorType_value = openflow_13.OfpErrorType_value
-
-const OfpErrorType_OFPET_HELLO_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_HELLO_FAILED)
-const OfpErrorType_OFPET_BAD_REQUEST = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_REQUEST)
-const OfpErrorType_OFPET_BAD_ACTION = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_ACTION)
-const OfpErrorType_OFPET_BAD_INSTRUCTION = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_INSTRUCTION)
-const OfpErrorType_OFPET_BAD_MATCH = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_MATCH)
-const OfpErrorType_OFPET_FLOW_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_FLOW_MOD_FAILED)
-const OfpErrorType_OFPET_GROUP_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_GROUP_MOD_FAILED)
-const OfpErrorType_OFPET_PORT_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_PORT_MOD_FAILED)
-const OfpErrorType_OFPET_TABLE_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_TABLE_MOD_FAILED)
-const OfpErrorType_OFPET_QUEUE_OP_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_QUEUE_OP_FAILED)
-const OfpErrorType_OFPET_SWITCH_CONFIG_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_SWITCH_CONFIG_FAILED)
-const OfpErrorType_OFPET_ROLE_REQUEST_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_ROLE_REQUEST_FAILED)
-const OfpErrorType_OFPET_METER_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_METER_MOD_FAILED)
-const OfpErrorType_OFPET_TABLE_FEATURES_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_TABLE_FEATURES_FAILED)
-const OfpErrorType_OFPET_EXPERIMENTER = OfpErrorType(openflow_13.OfpErrorType_OFPET_EXPERIMENTER)
-
-// OfpHelloFailedCode from public import voltha_protos/openflow_13.proto
-type OfpHelloFailedCode = openflow_13.OfpHelloFailedCode
-
-var OfpHelloFailedCode_name = openflow_13.OfpHelloFailedCode_name
-var OfpHelloFailedCode_value = openflow_13.OfpHelloFailedCode_value
-
-const OfpHelloFailedCode_OFPHFC_INCOMPATIBLE = OfpHelloFailedCode(openflow_13.OfpHelloFailedCode_OFPHFC_INCOMPATIBLE)
-const OfpHelloFailedCode_OFPHFC_EPERM = OfpHelloFailedCode(openflow_13.OfpHelloFailedCode_OFPHFC_EPERM)
-
-// OfpBadRequestCode from public import voltha_protos/openflow_13.proto
-type OfpBadRequestCode = openflow_13.OfpBadRequestCode
-
-var OfpBadRequestCode_name = openflow_13.OfpBadRequestCode_name
-var OfpBadRequestCode_value = openflow_13.OfpBadRequestCode_value
-
-const OfpBadRequestCode_OFPBRC_BAD_VERSION = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_VERSION)
-const OfpBadRequestCode_OFPBRC_BAD_TYPE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_TYPE)
-const OfpBadRequestCode_OFPBRC_BAD_MULTIPART = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_MULTIPART)
-const OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER)
-const OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE)
-const OfpBadRequestCode_OFPBRC_EPERM = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_EPERM)
-const OfpBadRequestCode_OFPBRC_BAD_LEN = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_LEN)
-const OfpBadRequestCode_OFPBRC_BUFFER_EMPTY = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BUFFER_EMPTY)
-const OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN)
-const OfpBadRequestCode_OFPBRC_BAD_TABLE_ID = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_TABLE_ID)
-const OfpBadRequestCode_OFPBRC_IS_SLAVE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_IS_SLAVE)
-const OfpBadRequestCode_OFPBRC_BAD_PORT = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_PORT)
-const OfpBadRequestCode_OFPBRC_BAD_PACKET = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_PACKET)
-const OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW)
-
-// OfpBadActionCode from public import voltha_protos/openflow_13.proto
-type OfpBadActionCode = openflow_13.OfpBadActionCode
-
-var OfpBadActionCode_name = openflow_13.OfpBadActionCode_name
-var OfpBadActionCode_value = openflow_13.OfpBadActionCode_value
-
-const OfpBadActionCode_OFPBAC_BAD_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_TYPE)
-const OfpBadActionCode_OFPBAC_BAD_LEN = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_LEN)
-const OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER)
-const OfpBadActionCode_OFPBAC_BAD_EXP_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_EXP_TYPE)
-const OfpBadActionCode_OFPBAC_BAD_OUT_PORT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_OUT_PORT)
-const OfpBadActionCode_OFPBAC_BAD_ARGUMENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_ARGUMENT)
-const OfpBadActionCode_OFPBAC_EPERM = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_EPERM)
-const OfpBadActionCode_OFPBAC_TOO_MANY = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_TOO_MANY)
-const OfpBadActionCode_OFPBAC_BAD_QUEUE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_QUEUE)
-const OfpBadActionCode_OFPBAC_BAD_OUT_GROUP = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_OUT_GROUP)
-const OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT)
-const OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER)
-const OfpBadActionCode_OFPBAC_BAD_TAG = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_TAG)
-const OfpBadActionCode_OFPBAC_BAD_SET_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_TYPE)
-const OfpBadActionCode_OFPBAC_BAD_SET_LEN = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_LEN)
-const OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT)
-
-// OfpBadInstructionCode from public import voltha_protos/openflow_13.proto
-type OfpBadInstructionCode = openflow_13.OfpBadInstructionCode
-
-var OfpBadInstructionCode_name = openflow_13.OfpBadInstructionCode_name
-var OfpBadInstructionCode_value = openflow_13.OfpBadInstructionCode_value
-
-const OfpBadInstructionCode_OFPBIC_UNKNOWN_INST = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNKNOWN_INST)
-const OfpBadInstructionCode_OFPBIC_UNSUP_INST = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_INST)
-const OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID)
-const OfpBadInstructionCode_OFPBIC_UNSUP_METADATA = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_METADATA)
-const OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK)
-const OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER)
-const OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE)
-const OfpBadInstructionCode_OFPBIC_BAD_LEN = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_LEN)
-const OfpBadInstructionCode_OFPBIC_EPERM = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_EPERM)
-
-// OfpBadMatchCode from public import voltha_protos/openflow_13.proto
-type OfpBadMatchCode = openflow_13.OfpBadMatchCode
-
-var OfpBadMatchCode_name = openflow_13.OfpBadMatchCode_name
-var OfpBadMatchCode_value = openflow_13.OfpBadMatchCode_value
-
-const OfpBadMatchCode_OFPBMC_BAD_TYPE = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_TYPE)
-const OfpBadMatchCode_OFPBMC_BAD_LEN = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_LEN)
-const OfpBadMatchCode_OFPBMC_BAD_TAG = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_TAG)
-const OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK)
-const OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK)
-const OfpBadMatchCode_OFPBMC_BAD_WILDCARDS = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_WILDCARDS)
-const OfpBadMatchCode_OFPBMC_BAD_FIELD = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_FIELD)
-const OfpBadMatchCode_OFPBMC_BAD_VALUE = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_VALUE)
-const OfpBadMatchCode_OFPBMC_BAD_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_MASK)
-const OfpBadMatchCode_OFPBMC_BAD_PREREQ = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_PREREQ)
-const OfpBadMatchCode_OFPBMC_DUP_FIELD = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_DUP_FIELD)
-const OfpBadMatchCode_OFPBMC_EPERM = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_EPERM)
-
-// OfpFlowModFailedCode from public import voltha_protos/openflow_13.proto
-type OfpFlowModFailedCode = openflow_13.OfpFlowModFailedCode
-
-var OfpFlowModFailedCode_name = openflow_13.OfpFlowModFailedCode_name
-var OfpFlowModFailedCode_value = openflow_13.OfpFlowModFailedCode_value
-
-const OfpFlowModFailedCode_OFPFMFC_UNKNOWN = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_UNKNOWN)
-const OfpFlowModFailedCode_OFPFMFC_TABLE_FULL = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_TABLE_FULL)
-const OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID)
-const OfpFlowModFailedCode_OFPFMFC_OVERLAP = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_OVERLAP)
-const OfpFlowModFailedCode_OFPFMFC_EPERM = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_EPERM)
-const OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT)
-const OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND)
-const OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS)
-
-// OfpGroupModFailedCode from public import voltha_protos/openflow_13.proto
-type OfpGroupModFailedCode = openflow_13.OfpGroupModFailedCode
-
-var OfpGroupModFailedCode_name = openflow_13.OfpGroupModFailedCode_name
-var OfpGroupModFailedCode_value = openflow_13.OfpGroupModFailedCode_value
-
-const OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS)
-const OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP)
-const OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED)
-const OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS)
-const OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS)
-const OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED)
-const OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED)
-const OfpGroupModFailedCode_OFPGMFC_LOOP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_LOOP)
-const OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP)
-const OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP)
-const OfpGroupModFailedCode_OFPGMFC_BAD_TYPE = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_TYPE)
-const OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND)
-const OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET)
-const OfpGroupModFailedCode_OFPGMFC_BAD_WATCH = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_WATCH)
-const OfpGroupModFailedCode_OFPGMFC_EPERM = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_EPERM)
-
-// OfpPortModFailedCode from public import voltha_protos/openflow_13.proto
-type OfpPortModFailedCode = openflow_13.OfpPortModFailedCode
-
-var OfpPortModFailedCode_name = openflow_13.OfpPortModFailedCode_name
-var OfpPortModFailedCode_value = openflow_13.OfpPortModFailedCode_value
-
-const OfpPortModFailedCode_OFPPMFC_BAD_PORT = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_PORT)
-const OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR)
-const OfpPortModFailedCode_OFPPMFC_BAD_CONFIG = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_CONFIG)
-const OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE)
-const OfpPortModFailedCode_OFPPMFC_EPERM = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_EPERM)
-
-// OfpTableModFailedCode from public import voltha_protos/openflow_13.proto
-type OfpTableModFailedCode = openflow_13.OfpTableModFailedCode
-
-var OfpTableModFailedCode_name = openflow_13.OfpTableModFailedCode_name
-var OfpTableModFailedCode_value = openflow_13.OfpTableModFailedCode_value
-
-const OfpTableModFailedCode_OFPTMFC_BAD_TABLE = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_BAD_TABLE)
-const OfpTableModFailedCode_OFPTMFC_BAD_CONFIG = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_BAD_CONFIG)
-const OfpTableModFailedCode_OFPTMFC_EPERM = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_EPERM)
-
-// OfpQueueOpFailedCode from public import voltha_protos/openflow_13.proto
-type OfpQueueOpFailedCode = openflow_13.OfpQueueOpFailedCode
-
-var OfpQueueOpFailedCode_name = openflow_13.OfpQueueOpFailedCode_name
-var OfpQueueOpFailedCode_value = openflow_13.OfpQueueOpFailedCode_value
-
-const OfpQueueOpFailedCode_OFPQOFC_BAD_PORT = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_BAD_PORT)
-const OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE)
-const OfpQueueOpFailedCode_OFPQOFC_EPERM = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_EPERM)
-
-// OfpSwitchConfigFailedCode from public import voltha_protos/openflow_13.proto
-type OfpSwitchConfigFailedCode = openflow_13.OfpSwitchConfigFailedCode
-
-var OfpSwitchConfigFailedCode_name = openflow_13.OfpSwitchConfigFailedCode_name
-var OfpSwitchConfigFailedCode_value = openflow_13.OfpSwitchConfigFailedCode_value
-
-const OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS)
-const OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN)
-const OfpSwitchConfigFailedCode_OFPSCFC_EPERM = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_EPERM)
-
-// OfpRoleRequestFailedCode from public import voltha_protos/openflow_13.proto
-type OfpRoleRequestFailedCode = openflow_13.OfpRoleRequestFailedCode
-
-var OfpRoleRequestFailedCode_name = openflow_13.OfpRoleRequestFailedCode_name
-var OfpRoleRequestFailedCode_value = openflow_13.OfpRoleRequestFailedCode_value
-
-const OfpRoleRequestFailedCode_OFPRRFC_STALE = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_STALE)
-const OfpRoleRequestFailedCode_OFPRRFC_UNSUP = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_UNSUP)
-const OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE)
-
-// OfpMeterModFailedCode from public import voltha_protos/openflow_13.proto
-type OfpMeterModFailedCode = openflow_13.OfpMeterModFailedCode
-
-var OfpMeterModFailedCode_name = openflow_13.OfpMeterModFailedCode_name
-var OfpMeterModFailedCode_value = openflow_13.OfpMeterModFailedCode_value
-
-const OfpMeterModFailedCode_OFPMMFC_UNKNOWN = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_UNKNOWN)
-const OfpMeterModFailedCode_OFPMMFC_METER_EXISTS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_METER_EXISTS)
-const OfpMeterModFailedCode_OFPMMFC_INVALID_METER = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_INVALID_METER)
-const OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER)
-const OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND)
-const OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS)
-const OfpMeterModFailedCode_OFPMMFC_BAD_RATE = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_RATE)
-const OfpMeterModFailedCode_OFPMMFC_BAD_BURST = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BURST)
-const OfpMeterModFailedCode_OFPMMFC_BAD_BAND = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BAND)
-const OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL)
-const OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS)
-const OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS)
-
-// OfpTableFeaturesFailedCode from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturesFailedCode = openflow_13.OfpTableFeaturesFailedCode
-
-var OfpTableFeaturesFailedCode_name = openflow_13.OfpTableFeaturesFailedCode_name
-var OfpTableFeaturesFailedCode_value = openflow_13.OfpTableFeaturesFailedCode_value
-
-const OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE)
-const OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA)
-const OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE)
-const OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN)
-const OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT)
-const OfpTableFeaturesFailedCode_OFPTFFC_EPERM = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_EPERM)
-
-// OfpMultipartType from public import voltha_protos/openflow_13.proto
-type OfpMultipartType = openflow_13.OfpMultipartType
-
-var OfpMultipartType_name = openflow_13.OfpMultipartType_name
-var OfpMultipartType_value = openflow_13.OfpMultipartType_value
-
-const OfpMultipartType_OFPMP_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_DESC)
-const OfpMultipartType_OFPMP_FLOW = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_FLOW)
-const OfpMultipartType_OFPMP_AGGREGATE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_AGGREGATE)
-const OfpMultipartType_OFPMP_TABLE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_TABLE)
-const OfpMultipartType_OFPMP_PORT_STATS = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_PORT_STATS)
-const OfpMultipartType_OFPMP_QUEUE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_QUEUE)
-const OfpMultipartType_OFPMP_GROUP = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP)
-const OfpMultipartType_OFPMP_GROUP_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP_DESC)
-const OfpMultipartType_OFPMP_GROUP_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP_FEATURES)
-const OfpMultipartType_OFPMP_METER = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER)
-const OfpMultipartType_OFPMP_METER_CONFIG = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER_CONFIG)
-const OfpMultipartType_OFPMP_METER_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER_FEATURES)
-const OfpMultipartType_OFPMP_TABLE_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_TABLE_FEATURES)
-const OfpMultipartType_OFPMP_PORT_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_PORT_DESC)
-const OfpMultipartType_OFPMP_EXPERIMENTER = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_EXPERIMENTER)
-
-// OfpMultipartRequestFlags from public import voltha_protos/openflow_13.proto
-type OfpMultipartRequestFlags = openflow_13.OfpMultipartRequestFlags
-
-var OfpMultipartRequestFlags_name = openflow_13.OfpMultipartRequestFlags_name
-var OfpMultipartRequestFlags_value = openflow_13.OfpMultipartRequestFlags_value
-
-const OfpMultipartRequestFlags_OFPMPF_REQ_INVALID = OfpMultipartRequestFlags(openflow_13.OfpMultipartRequestFlags_OFPMPF_REQ_INVALID)
-const OfpMultipartRequestFlags_OFPMPF_REQ_MORE = OfpMultipartRequestFlags(openflow_13.OfpMultipartRequestFlags_OFPMPF_REQ_MORE)
-
-// OfpMultipartReplyFlags from public import voltha_protos/openflow_13.proto
-type OfpMultipartReplyFlags = openflow_13.OfpMultipartReplyFlags
-
-var OfpMultipartReplyFlags_name = openflow_13.OfpMultipartReplyFlags_name
-var OfpMultipartReplyFlags_value = openflow_13.OfpMultipartReplyFlags_value
-
-const OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID = OfpMultipartReplyFlags(openflow_13.OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID)
-const OfpMultipartReplyFlags_OFPMPF_REPLY_MORE = OfpMultipartReplyFlags(openflow_13.OfpMultipartReplyFlags_OFPMPF_REPLY_MORE)
-
-// OfpTableFeaturePropType from public import voltha_protos/openflow_13.proto
-type OfpTableFeaturePropType = openflow_13.OfpTableFeaturePropType
-
-var OfpTableFeaturePropType_name = openflow_13.OfpTableFeaturePropType_name
-var OfpTableFeaturePropType_value = openflow_13.OfpTableFeaturePropType_value
-
-const OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS)
-const OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS)
-const OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES)
-const OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS)
-const OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS)
-const OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS)
-const OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS)
-const OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS)
-const OfpTableFeaturePropType_OFPTFPT_MATCH = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_MATCH)
-const OfpTableFeaturePropType_OFPTFPT_WILDCARDS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WILDCARDS)
-const OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD)
-const OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS)
-const OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD)
-const OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS)
-const OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER)
-const OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS)
-
-// OfpGroupCapabilities from public import voltha_protos/openflow_13.proto
-type OfpGroupCapabilities = openflow_13.OfpGroupCapabilities
-
-var OfpGroupCapabilities_name = openflow_13.OfpGroupCapabilities_name
-var OfpGroupCapabilities_value = openflow_13.OfpGroupCapabilities_value
-
-const OfpGroupCapabilities_OFPGFC_INVALID = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_INVALID)
-const OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT)
-const OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS)
-const OfpGroupCapabilities_OFPGFC_CHAINING = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_CHAINING)
-const OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS)
-
-// OfpQueueProperties from public import voltha_protos/openflow_13.proto
-type OfpQueueProperties = openflow_13.OfpQueueProperties
-
-var OfpQueueProperties_name = openflow_13.OfpQueueProperties_name
-var OfpQueueProperties_value = openflow_13.OfpQueueProperties_value
-
-const OfpQueueProperties_OFPQT_INVALID = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_INVALID)
-const OfpQueueProperties_OFPQT_MIN_RATE = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_MIN_RATE)
-const OfpQueueProperties_OFPQT_MAX_RATE = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_MAX_RATE)
-const OfpQueueProperties_OFPQT_EXPERIMENTER = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_EXPERIMENTER)
-
-// OfpControllerRole from public import voltha_protos/openflow_13.proto
-type OfpControllerRole = openflow_13.OfpControllerRole
-
-var OfpControllerRole_name = openflow_13.OfpControllerRole_name
-var OfpControllerRole_value = openflow_13.OfpControllerRole_value
-
-const OfpControllerRole_OFPCR_ROLE_NOCHANGE = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_NOCHANGE)
-const OfpControllerRole_OFPCR_ROLE_EQUAL = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_EQUAL)
-const OfpControllerRole_OFPCR_ROLE_MASTER = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_MASTER)
-const OfpControllerRole_OFPCR_ROLE_SLAVE = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_SLAVE)
-
-type EventFilterRuleKey_EventFilterRuleType int32
-
-const (
-	EventFilterRuleKey_filter_all        EventFilterRuleKey_EventFilterRuleType = 0
-	EventFilterRuleKey_category          EventFilterRuleKey_EventFilterRuleType = 1
-	EventFilterRuleKey_sub_category      EventFilterRuleKey_EventFilterRuleType = 2
-	EventFilterRuleKey_kpi_event_type    EventFilterRuleKey_EventFilterRuleType = 3
-	EventFilterRuleKey_config_event_type EventFilterRuleKey_EventFilterRuleType = 4
-	EventFilterRuleKey_device_event_type EventFilterRuleKey_EventFilterRuleType = 5
-)
-
-var EventFilterRuleKey_EventFilterRuleType_name = map[int32]string{
-	0: "filter_all",
-	1: "category",
-	2: "sub_category",
-	3: "kpi_event_type",
-	4: "config_event_type",
-	5: "device_event_type",
-}
-
-var EventFilterRuleKey_EventFilterRuleType_value = map[string]int32{
-	"filter_all":        0,
-	"category":          1,
-	"sub_category":      2,
-	"kpi_event_type":    3,
-	"config_event_type": 4,
-	"device_event_type": 5,
-}
-
-func (x EventFilterRuleKey_EventFilterRuleType) String() string {
-	return proto.EnumName(EventFilterRuleKey_EventFilterRuleType_name, int32(x))
-}
-
-func (EventFilterRuleKey_EventFilterRuleType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{2, 0}
-}
-
-type TestResponse_TestResponseResult int32
-
-const (
-	TestResponse_SUCCESS TestResponse_TestResponseResult = 0
-	TestResponse_FAILURE TestResponse_TestResponseResult = 1
-)
-
-var TestResponse_TestResponseResult_name = map[int32]string{
-	0: "SUCCESS",
-	1: "FAILURE",
-}
-
-var TestResponse_TestResponseResult_value = map[string]int32{
-	"SUCCESS": 0,
-	"FAILURE": 1,
-}
-
-func (x TestResponse_TestResponseResult) String() string {
-	return proto.EnumName(TestResponse_TestResponseResult_name, int32(x))
-}
-
-func (TestResponse_TestResponseResult) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{9, 0}
-}
-
-type SelfTestResponse_SelfTestResult int32
-
-const (
-	SelfTestResponse_SUCCESS       SelfTestResponse_SelfTestResult = 0
-	SelfTestResponse_FAILURE       SelfTestResponse_SelfTestResult = 1
-	SelfTestResponse_NOT_SUPPORTED SelfTestResponse_SelfTestResult = 2
-	SelfTestResponse_UNKNOWN_ERROR SelfTestResponse_SelfTestResult = 3
-)
-
-var SelfTestResponse_SelfTestResult_name = map[int32]string{
-	0: "SUCCESS",
-	1: "FAILURE",
-	2: "NOT_SUPPORTED",
-	3: "UNKNOWN_ERROR",
-}
-
-var SelfTestResponse_SelfTestResult_value = map[string]int32{
-	"SUCCESS":       0,
-	"FAILURE":       1,
-	"NOT_SUPPORTED": 2,
-	"UNKNOWN_ERROR": 3,
-}
-
-func (x SelfTestResponse_SelfTestResult) String() string {
-	return proto.EnumName(SelfTestResponse_SelfTestResult_name, int32(x))
-}
-
-func (SelfTestResponse_SelfTestResult) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{12, 0}
-}
-
-type DeviceGroup struct {
-	Id                   string           `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	LogicalDevices       []*LogicalDevice `protobuf:"bytes,2,rep,name=logical_devices,json=logicalDevices,proto3" json:"logical_devices,omitempty"`
-	Devices              []*Device        `protobuf:"bytes,3,rep,name=devices,proto3" json:"devices,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *DeviceGroup) Reset()         { *m = DeviceGroup{} }
-func (m *DeviceGroup) String() string { return proto.CompactTextString(m) }
-func (*DeviceGroup) ProtoMessage()    {}
-func (*DeviceGroup) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{0}
-}
-
-func (m *DeviceGroup) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceGroup.Unmarshal(m, b)
-}
-func (m *DeviceGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceGroup.Marshal(b, m, deterministic)
-}
-func (m *DeviceGroup) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceGroup.Merge(m, src)
-}
-func (m *DeviceGroup) XXX_Size() int {
-	return xxx_messageInfo_DeviceGroup.Size(m)
-}
-func (m *DeviceGroup) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceGroup.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceGroup proto.InternalMessageInfo
-
-func (m *DeviceGroup) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *DeviceGroup) GetLogicalDevices() []*LogicalDevice {
-	if m != nil {
-		return m.LogicalDevices
-	}
-	return nil
-}
-
-func (m *DeviceGroup) GetDevices() []*Device {
-	if m != nil {
-		return m.Devices
-	}
-	return nil
-}
-
-type DeviceGroups struct {
-	Items                []*DeviceGroup `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *DeviceGroups) Reset()         { *m = DeviceGroups{} }
-func (m *DeviceGroups) String() string { return proto.CompactTextString(m) }
-func (*DeviceGroups) ProtoMessage()    {}
-func (*DeviceGroups) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{1}
-}
-
-func (m *DeviceGroups) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceGroups.Unmarshal(m, b)
-}
-func (m *DeviceGroups) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceGroups.Marshal(b, m, deterministic)
-}
-func (m *DeviceGroups) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceGroups.Merge(m, src)
-}
-func (m *DeviceGroups) XXX_Size() int {
-	return xxx_messageInfo_DeviceGroups.Size(m)
-}
-func (m *DeviceGroups) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceGroups.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceGroups proto.InternalMessageInfo
-
-func (m *DeviceGroups) GetItems() []*DeviceGroup {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type EventFilterRuleKey struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *EventFilterRuleKey) Reset()         { *m = EventFilterRuleKey{} }
-func (m *EventFilterRuleKey) String() string { return proto.CompactTextString(m) }
-func (*EventFilterRuleKey) ProtoMessage()    {}
-func (*EventFilterRuleKey) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{2}
-}
-
-func (m *EventFilterRuleKey) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventFilterRuleKey.Unmarshal(m, b)
-}
-func (m *EventFilterRuleKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventFilterRuleKey.Marshal(b, m, deterministic)
-}
-func (m *EventFilterRuleKey) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventFilterRuleKey.Merge(m, src)
-}
-func (m *EventFilterRuleKey) XXX_Size() int {
-	return xxx_messageInfo_EventFilterRuleKey.Size(m)
-}
-func (m *EventFilterRuleKey) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventFilterRuleKey.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventFilterRuleKey proto.InternalMessageInfo
-
-type EventFilterRule struct {
-	Key                  EventFilterRuleKey_EventFilterRuleType `protobuf:"varint,1,opt,name=key,proto3,enum=voltha.EventFilterRuleKey_EventFilterRuleType" json:"key,omitempty"`
-	Value                string                                 `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                               `json:"-"`
-	XXX_unrecognized     []byte                                 `json:"-"`
-	XXX_sizecache        int32                                  `json:"-"`
-}
-
-func (m *EventFilterRule) Reset()         { *m = EventFilterRule{} }
-func (m *EventFilterRule) String() string { return proto.CompactTextString(m) }
-func (*EventFilterRule) ProtoMessage()    {}
-func (*EventFilterRule) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{3}
-}
-
-func (m *EventFilterRule) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventFilterRule.Unmarshal(m, b)
-}
-func (m *EventFilterRule) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventFilterRule.Marshal(b, m, deterministic)
-}
-func (m *EventFilterRule) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventFilterRule.Merge(m, src)
-}
-func (m *EventFilterRule) XXX_Size() int {
-	return xxx_messageInfo_EventFilterRule.Size(m)
-}
-func (m *EventFilterRule) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventFilterRule.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventFilterRule proto.InternalMessageInfo
-
-func (m *EventFilterRule) GetKey() EventFilterRuleKey_EventFilterRuleType {
-	if m != nil {
-		return m.Key
-	}
-	return EventFilterRuleKey_filter_all
-}
-
-func (m *EventFilterRule) GetValue() string {
-	if m != nil {
-		return m.Value
-	}
-	return ""
-}
-
-type EventFilter struct {
-	Id                   string             `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Enable               bool               `protobuf:"varint,2,opt,name=enable,proto3" json:"enable,omitempty"`
-	DeviceId             string             `protobuf:"bytes,3,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	EventType            string             `protobuf:"bytes,4,opt,name=event_type,json=eventType,proto3" json:"event_type,omitempty"`
-	Rules                []*EventFilterRule `protobuf:"bytes,5,rep,name=rules,proto3" json:"rules,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
-}
-
-func (m *EventFilter) Reset()         { *m = EventFilter{} }
-func (m *EventFilter) String() string { return proto.CompactTextString(m) }
-func (*EventFilter) ProtoMessage()    {}
-func (*EventFilter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{4}
-}
-
-func (m *EventFilter) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventFilter.Unmarshal(m, b)
-}
-func (m *EventFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventFilter.Marshal(b, m, deterministic)
-}
-func (m *EventFilter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventFilter.Merge(m, src)
-}
-func (m *EventFilter) XXX_Size() int {
-	return xxx_messageInfo_EventFilter.Size(m)
-}
-func (m *EventFilter) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventFilter.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventFilter proto.InternalMessageInfo
-
-func (m *EventFilter) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *EventFilter) GetEnable() bool {
-	if m != nil {
-		return m.Enable
-	}
-	return false
-}
-
-func (m *EventFilter) GetDeviceId() string {
-	if m != nil {
-		return m.DeviceId
-	}
-	return ""
-}
-
-func (m *EventFilter) GetEventType() string {
-	if m != nil {
-		return m.EventType
-	}
-	return ""
-}
-
-func (m *EventFilter) GetRules() []*EventFilterRule {
-	if m != nil {
-		return m.Rules
-	}
-	return nil
-}
-
-type EventFilters struct {
-	Filters              []*EventFilter `protobuf:"bytes,1,rep,name=filters,proto3" json:"filters,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *EventFilters) Reset()         { *m = EventFilters{} }
-func (m *EventFilters) String() string { return proto.CompactTextString(m) }
-func (*EventFilters) ProtoMessage()    {}
-func (*EventFilters) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{5}
-}
-
-func (m *EventFilters) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EventFilters.Unmarshal(m, b)
-}
-func (m *EventFilters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EventFilters.Marshal(b, m, deterministic)
-}
-func (m *EventFilters) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventFilters.Merge(m, src)
-}
-func (m *EventFilters) XXX_Size() int {
-	return xxx_messageInfo_EventFilters.Size(m)
-}
-func (m *EventFilters) XXX_DiscardUnknown() {
-	xxx_messageInfo_EventFilters.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EventFilters proto.InternalMessageInfo
-
-func (m *EventFilters) GetFilters() []*EventFilter {
-	if m != nil {
-		return m.Filters
-	}
-	return nil
-}
-
-// CoreInstance represents a core instance.  It is data held in memory when a core
-// is running.  This data is not persistent.
-type CoreInstance struct {
-	InstanceId           string        `protobuf:"bytes,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
-	Health               *HealthStatus `protobuf:"bytes,2,opt,name=health,proto3" json:"health,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *CoreInstance) Reset()         { *m = CoreInstance{} }
-func (m *CoreInstance) String() string { return proto.CompactTextString(m) }
-func (*CoreInstance) ProtoMessage()    {}
-func (*CoreInstance) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{6}
-}
-
-func (m *CoreInstance) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CoreInstance.Unmarshal(m, b)
-}
-func (m *CoreInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CoreInstance.Marshal(b, m, deterministic)
-}
-func (m *CoreInstance) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CoreInstance.Merge(m, src)
-}
-func (m *CoreInstance) XXX_Size() int {
-	return xxx_messageInfo_CoreInstance.Size(m)
-}
-func (m *CoreInstance) XXX_DiscardUnknown() {
-	xxx_messageInfo_CoreInstance.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CoreInstance proto.InternalMessageInfo
-
-func (m *CoreInstance) GetInstanceId() string {
-	if m != nil {
-		return m.InstanceId
-	}
-	return ""
-}
-
-func (m *CoreInstance) GetHealth() *HealthStatus {
-	if m != nil {
-		return m.Health
-	}
-	return nil
-}
-
-type CoreInstances struct {
-	Items                []*CoreInstance `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *CoreInstances) Reset()         { *m = CoreInstances{} }
-func (m *CoreInstances) String() string { return proto.CompactTextString(m) }
-func (*CoreInstances) ProtoMessage()    {}
-func (*CoreInstances) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{7}
-}
-
-func (m *CoreInstances) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CoreInstances.Unmarshal(m, b)
-}
-func (m *CoreInstances) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CoreInstances.Marshal(b, m, deterministic)
-}
-func (m *CoreInstances) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CoreInstances.Merge(m, src)
-}
-func (m *CoreInstances) XXX_Size() int {
-	return xxx_messageInfo_CoreInstances.Size(m)
-}
-func (m *CoreInstances) XXX_DiscardUnknown() {
-	xxx_messageInfo_CoreInstances.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CoreInstances proto.InternalMessageInfo
-
-func (m *CoreInstances) GetItems() []*CoreInstance {
-	if m != nil {
-		return m.Items
-	}
-	return nil
-}
-
-type OmciTestRequest struct {
-	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	Uuid                 string   `protobuf:"bytes,2,opt,name=uuid,proto3" json:"uuid,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OmciTestRequest) Reset()         { *m = OmciTestRequest{} }
-func (m *OmciTestRequest) String() string { return proto.CompactTextString(m) }
-func (*OmciTestRequest) ProtoMessage()    {}
-func (*OmciTestRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{8}
-}
-
-func (m *OmciTestRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OmciTestRequest.Unmarshal(m, b)
-}
-func (m *OmciTestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OmciTestRequest.Marshal(b, m, deterministic)
-}
-func (m *OmciTestRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OmciTestRequest.Merge(m, src)
-}
-func (m *OmciTestRequest) XXX_Size() int {
-	return xxx_messageInfo_OmciTestRequest.Size(m)
-}
-func (m *OmciTestRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_OmciTestRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OmciTestRequest proto.InternalMessageInfo
-
-func (m *OmciTestRequest) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *OmciTestRequest) GetUuid() string {
-	if m != nil {
-		return m.Uuid
-	}
-	return ""
-}
-
-type TestResponse struct {
-	Result               TestResponse_TestResponseResult `protobuf:"varint,1,opt,name=result,proto3,enum=voltha.TestResponse_TestResponseResult" json:"result,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *TestResponse) Reset()         { *m = TestResponse{} }
-func (m *TestResponse) String() string { return proto.CompactTextString(m) }
-func (*TestResponse) ProtoMessage()    {}
-func (*TestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{9}
-}
-
-func (m *TestResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestResponse.Unmarshal(m, b)
-}
-func (m *TestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestResponse.Marshal(b, m, deterministic)
-}
-func (m *TestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestResponse.Merge(m, src)
-}
-func (m *TestResponse) XXX_Size() int {
-	return xxx_messageInfo_TestResponse.Size(m)
-}
-func (m *TestResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestResponse proto.InternalMessageInfo
-
-func (m *TestResponse) GetResult() TestResponse_TestResponseResult {
-	if m != nil {
-		return m.Result
-	}
-	return TestResponse_SUCCESS
-}
-
-type ValueSet struct {
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// Types that are valid to be assigned to Value:
-	//	*ValueSet_AlarmConfig
-	Value                isValueSet_Value `protobuf_oneof:"value"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *ValueSet) Reset()         { *m = ValueSet{} }
-func (m *ValueSet) String() string { return proto.CompactTextString(m) }
-func (*ValueSet) ProtoMessage()    {}
-func (*ValueSet) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{10}
-}
-
-func (m *ValueSet) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ValueSet.Unmarshal(m, b)
-}
-func (m *ValueSet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ValueSet.Marshal(b, m, deterministic)
-}
-func (m *ValueSet) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueSet.Merge(m, src)
-}
-func (m *ValueSet) XXX_Size() int {
-	return xxx_messageInfo_ValueSet.Size(m)
-}
-func (m *ValueSet) XXX_DiscardUnknown() {
-	xxx_messageInfo_ValueSet.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ValueSet proto.InternalMessageInfo
-
-func (m *ValueSet) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-type isValueSet_Value interface {
-	isValueSet_Value()
-}
-
-type ValueSet_AlarmConfig struct {
-	AlarmConfig *config.AlarmConfig `protobuf:"bytes,2,opt,name=alarm_config,json=alarmConfig,proto3,oneof"`
-}
-
-func (*ValueSet_AlarmConfig) isValueSet_Value() {}
-
-func (m *ValueSet) GetValue() isValueSet_Value {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func (m *ValueSet) GetAlarmConfig() *config.AlarmConfig {
-	if x, ok := m.GetValue().(*ValueSet_AlarmConfig); ok {
-		return x.AlarmConfig
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ValueSet) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*ValueSet_AlarmConfig)(nil),
-	}
-}
-
-// Voltha represents the Voltha cluster data.  Each Core instance will hold a subset of
-// the entire cluster. However, some items (e.g. adapters) will be held by all cores
-// for better performance
-type Voltha struct {
-	Version              string                  `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
-	Adapters             []*Adapter              `protobuf:"bytes,2,rep,name=adapters,proto3" json:"adapters,omitempty"`
-	LogicalDevices       []*LogicalDevice        `protobuf:"bytes,3,rep,name=logical_devices,json=logicalDevices,proto3" json:"logical_devices,omitempty"`
-	Devices              []*Device               `protobuf:"bytes,4,rep,name=devices,proto3" json:"devices,omitempty"`
-	DeviceTypes          []*DeviceType           `protobuf:"bytes,5,rep,name=device_types,json=deviceTypes,proto3" json:"device_types,omitempty"`
-	DeviceGroups         []*DeviceGroup          `protobuf:"bytes,6,rep,name=device_groups,json=deviceGroups,proto3" json:"device_groups,omitempty"`
-	EventFilters         []*EventFilter          `protobuf:"bytes,7,rep,name=event_filters,json=eventFilters,proto3" json:"event_filters,omitempty"`
-	OmciMibDatabase      []*omci.MibDeviceData   `protobuf:"bytes,28,rep,name=omci_mib_database,json=omciMibDatabase,proto3" json:"omci_mib_database,omitempty"`
-	OmciAlarmDatabase    []*omci.AlarmDeviceData `protobuf:"bytes,29,rep,name=omci_alarm_database,json=omciAlarmDatabase,proto3" json:"omci_alarm_database,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
-}
-
-func (m *Voltha) Reset()         { *m = Voltha{} }
-func (m *Voltha) String() string { return proto.CompactTextString(m) }
-func (*Voltha) ProtoMessage()    {}
-func (*Voltha) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{11}
-}
-
-func (m *Voltha) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Voltha.Unmarshal(m, b)
-}
-func (m *Voltha) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Voltha.Marshal(b, m, deterministic)
-}
-func (m *Voltha) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Voltha.Merge(m, src)
-}
-func (m *Voltha) XXX_Size() int {
-	return xxx_messageInfo_Voltha.Size(m)
-}
-func (m *Voltha) XXX_DiscardUnknown() {
-	xxx_messageInfo_Voltha.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Voltha proto.InternalMessageInfo
-
-func (m *Voltha) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *Voltha) GetAdapters() []*Adapter {
-	if m != nil {
-		return m.Adapters
-	}
-	return nil
-}
-
-func (m *Voltha) GetLogicalDevices() []*LogicalDevice {
-	if m != nil {
-		return m.LogicalDevices
-	}
-	return nil
-}
-
-func (m *Voltha) GetDevices() []*Device {
-	if m != nil {
-		return m.Devices
-	}
-	return nil
-}
-
-func (m *Voltha) GetDeviceTypes() []*DeviceType {
-	if m != nil {
-		return m.DeviceTypes
-	}
-	return nil
-}
-
-func (m *Voltha) GetDeviceGroups() []*DeviceGroup {
-	if m != nil {
-		return m.DeviceGroups
-	}
-	return nil
-}
-
-func (m *Voltha) GetEventFilters() []*EventFilter {
-	if m != nil {
-		return m.EventFilters
-	}
-	return nil
-}
-
-func (m *Voltha) GetOmciMibDatabase() []*omci.MibDeviceData {
-	if m != nil {
-		return m.OmciMibDatabase
-	}
-	return nil
-}
-
-func (m *Voltha) GetOmciAlarmDatabase() []*omci.AlarmDeviceData {
-	if m != nil {
-		return m.OmciAlarmDatabase
-	}
-	return nil
-}
-
-// Device Self Test Response
-type SelfTestResponse struct {
-	Result               SelfTestResponse_SelfTestResult `protobuf:"varint,1,opt,name=result,proto3,enum=voltha.SelfTestResponse_SelfTestResult" json:"result,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *SelfTestResponse) Reset()         { *m = SelfTestResponse{} }
-func (m *SelfTestResponse) String() string { return proto.CompactTextString(m) }
-func (*SelfTestResponse) ProtoMessage()    {}
-func (*SelfTestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{12}
-}
-
-func (m *SelfTestResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SelfTestResponse.Unmarshal(m, b)
-}
-func (m *SelfTestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SelfTestResponse.Marshal(b, m, deterministic)
-}
-func (m *SelfTestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SelfTestResponse.Merge(m, src)
-}
-func (m *SelfTestResponse) XXX_Size() int {
-	return xxx_messageInfo_SelfTestResponse.Size(m)
-}
-func (m *SelfTestResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_SelfTestResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SelfTestResponse proto.InternalMessageInfo
-
-func (m *SelfTestResponse) GetResult() SelfTestResponse_SelfTestResult {
-	if m != nil {
-		return m.Result
-	}
-	return SelfTestResponse_SUCCESS
-}
-
-type OfAgentSubscriber struct {
-	// ID of ofagent instance
-	OfagentId string `protobuf:"bytes,1,opt,name=ofagent_id,json=ofagentId,proto3" json:"ofagent_id,omitempty"`
-	// ID of voltha instance to which the ofagent is subscribed
-	VolthaId             string   `protobuf:"bytes,2,opt,name=voltha_id,json=volthaId,proto3" json:"voltha_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *OfAgentSubscriber) Reset()         { *m = OfAgentSubscriber{} }
-func (m *OfAgentSubscriber) String() string { return proto.CompactTextString(m) }
-func (*OfAgentSubscriber) ProtoMessage()    {}
-func (*OfAgentSubscriber) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{13}
-}
-
-func (m *OfAgentSubscriber) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OfAgentSubscriber.Unmarshal(m, b)
-}
-func (m *OfAgentSubscriber) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OfAgentSubscriber.Marshal(b, m, deterministic)
-}
-func (m *OfAgentSubscriber) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OfAgentSubscriber.Merge(m, src)
-}
-func (m *OfAgentSubscriber) XXX_Size() int {
-	return xxx_messageInfo_OfAgentSubscriber.Size(m)
-}
-func (m *OfAgentSubscriber) XXX_DiscardUnknown() {
-	xxx_messageInfo_OfAgentSubscriber.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OfAgentSubscriber proto.InternalMessageInfo
-
-func (m *OfAgentSubscriber) GetOfagentId() string {
-	if m != nil {
-		return m.OfagentId
-	}
-	return ""
-}
-
-func (m *OfAgentSubscriber) GetVolthaId() string {
-	if m != nil {
-		return m.VolthaId
-	}
-	return ""
-}
-
-// Identifies a membership group a Core belongs to
-type Membership struct {
-	//  Group name
-	GroupName string `protobuf:"bytes,1,opt,name=group_name,json=groupName,proto3" json:"group_name,omitempty"`
-	// Unique ID of a container within that group
-	Id                   string   `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Membership) Reset()         { *m = Membership{} }
-func (m *Membership) String() string { return proto.CompactTextString(m) }
-func (*Membership) ProtoMessage()    {}
-func (*Membership) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{14}
-}
-
-func (m *Membership) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Membership.Unmarshal(m, b)
-}
-func (m *Membership) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Membership.Marshal(b, m, deterministic)
-}
-func (m *Membership) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Membership.Merge(m, src)
-}
-func (m *Membership) XXX_Size() int {
-	return xxx_messageInfo_Membership.Size(m)
-}
-func (m *Membership) XXX_DiscardUnknown() {
-	xxx_messageInfo_Membership.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Membership proto.InternalMessageInfo
-
-func (m *Membership) GetGroupName() string {
-	if m != nil {
-		return m.GroupName
-	}
-	return ""
-}
-
-func (m *Membership) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-type DeviceImageDownloadRequest struct {
-	// Device Id
-	// allows for operations on multiple devices.
-	DeviceId []*common.ID `protobuf:"bytes,1,rep,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	//The image for the device containing all the information
-	Image *Image `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"`
-	//Activate the image if the download to the device is successful
-	ActivateOnSuccess bool `protobuf:"varint,3,opt,name=activateOnSuccess,proto3" json:"activateOnSuccess,omitempty"`
-	//Automatically commit the image if the activation on the device is successful
-	CommitOnSuccess      bool     `protobuf:"varint,4,opt,name=commitOnSuccess,proto3" json:"commitOnSuccess,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DeviceImageDownloadRequest) Reset()         { *m = DeviceImageDownloadRequest{} }
-func (m *DeviceImageDownloadRequest) String() string { return proto.CompactTextString(m) }
-func (*DeviceImageDownloadRequest) ProtoMessage()    {}
-func (*DeviceImageDownloadRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{15}
-}
-
-func (m *DeviceImageDownloadRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceImageDownloadRequest.Unmarshal(m, b)
-}
-func (m *DeviceImageDownloadRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceImageDownloadRequest.Marshal(b, m, deterministic)
-}
-func (m *DeviceImageDownloadRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceImageDownloadRequest.Merge(m, src)
-}
-func (m *DeviceImageDownloadRequest) XXX_Size() int {
-	return xxx_messageInfo_DeviceImageDownloadRequest.Size(m)
-}
-func (m *DeviceImageDownloadRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceImageDownloadRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceImageDownloadRequest proto.InternalMessageInfo
-
-func (m *DeviceImageDownloadRequest) GetDeviceId() []*common.ID {
-	if m != nil {
-		return m.DeviceId
-	}
-	return nil
-}
-
-func (m *DeviceImageDownloadRequest) GetImage() *Image {
-	if m != nil {
-		return m.Image
-	}
-	return nil
-}
-
-func (m *DeviceImageDownloadRequest) GetActivateOnSuccess() bool {
-	if m != nil {
-		return m.ActivateOnSuccess
-	}
-	return false
-}
-
-func (m *DeviceImageDownloadRequest) GetCommitOnSuccess() bool {
-	if m != nil {
-		return m.CommitOnSuccess
-	}
-	return false
-}
-
-type DeviceImageRequest struct {
-	//Device Id
-	//allows for operations on multiple adapters.
-	DeviceId []*common.ID `protobuf:"bytes,1,rep,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
-	// Image Version, this is the sole identifier of the image. it's the vendor specified OMCI version
-	// must be known at the time of initiating a download, activate, commit image on an onu.
-	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
-	//Automatically commit the image if the activation on the device is successful
-	CommitOnSuccess      bool     `protobuf:"varint,3,opt,name=commitOnSuccess,proto3" json:"commitOnSuccess,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DeviceImageRequest) Reset()         { *m = DeviceImageRequest{} }
-func (m *DeviceImageRequest) String() string { return proto.CompactTextString(m) }
-func (*DeviceImageRequest) ProtoMessage()    {}
-func (*DeviceImageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{16}
-}
-
-func (m *DeviceImageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceImageRequest.Unmarshal(m, b)
-}
-func (m *DeviceImageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceImageRequest.Marshal(b, m, deterministic)
-}
-func (m *DeviceImageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceImageRequest.Merge(m, src)
-}
-func (m *DeviceImageRequest) XXX_Size() int {
-	return xxx_messageInfo_DeviceImageRequest.Size(m)
-}
-func (m *DeviceImageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceImageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceImageRequest proto.InternalMessageInfo
-
-func (m *DeviceImageRequest) GetDeviceId() []*common.ID {
-	if m != nil {
-		return m.DeviceId
-	}
-	return nil
-}
-
-func (m *DeviceImageRequest) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *DeviceImageRequest) GetCommitOnSuccess() bool {
-	if m != nil {
-		return m.CommitOnSuccess
-	}
-	return false
-}
-
-type DeviceImageResponse struct {
-	//Image state for the different devices
-	DeviceImageStates    []*DeviceImageState `protobuf:"bytes,1,rep,name=device_image_states,json=deviceImageStates,proto3" json:"device_image_states,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *DeviceImageResponse) Reset()         { *m = DeviceImageResponse{} }
-func (m *DeviceImageResponse) String() string { return proto.CompactTextString(m) }
-func (*DeviceImageResponse) ProtoMessage()    {}
-func (*DeviceImageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{17}
-}
-
-func (m *DeviceImageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeviceImageResponse.Unmarshal(m, b)
-}
-func (m *DeviceImageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeviceImageResponse.Marshal(b, m, deterministic)
-}
-func (m *DeviceImageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeviceImageResponse.Merge(m, src)
-}
-func (m *DeviceImageResponse) XXX_Size() int {
-	return xxx_messageInfo_DeviceImageResponse.Size(m)
-}
-func (m *DeviceImageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeviceImageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeviceImageResponse proto.InternalMessageInfo
-
-func (m *DeviceImageResponse) GetDeviceImageStates() []*DeviceImageState {
-	if m != nil {
-		return m.DeviceImageStates
-	}
-	return nil
-}
-
-// Additional information required to process flow at device adapters
-type FlowMetadata struct {
-	// Meters associated with flow-update to adapter
-	Meters               []*openflow_13.OfpMeterConfig `protobuf:"bytes,1,rep,name=meters,proto3" json:"meters,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
-	XXX_unrecognized     []byte                        `json:"-"`
-	XXX_sizecache        int32                         `json:"-"`
-}
-
-func (m *FlowMetadata) Reset()         { *m = FlowMetadata{} }
-func (m *FlowMetadata) String() string { return proto.CompactTextString(m) }
-func (*FlowMetadata) ProtoMessage()    {}
-func (*FlowMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e084f1a60ce7016c, []int{18}
-}
-
-func (m *FlowMetadata) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FlowMetadata.Unmarshal(m, b)
-}
-func (m *FlowMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FlowMetadata.Marshal(b, m, deterministic)
-}
-func (m *FlowMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FlowMetadata.Merge(m, src)
-}
-func (m *FlowMetadata) XXX_Size() int {
-	return xxx_messageInfo_FlowMetadata.Size(m)
-}
-func (m *FlowMetadata) XXX_DiscardUnknown() {
-	xxx_messageInfo_FlowMetadata.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FlowMetadata proto.InternalMessageInfo
-
-func (m *FlowMetadata) GetMeters() []*openflow_13.OfpMeterConfig {
-	if m != nil {
-		return m.Meters
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("voltha.EventFilterRuleKey_EventFilterRuleType", EventFilterRuleKey_EventFilterRuleType_name, EventFilterRuleKey_EventFilterRuleType_value)
-	proto.RegisterEnum("voltha.TestResponse_TestResponseResult", TestResponse_TestResponseResult_name, TestResponse_TestResponseResult_value)
-	proto.RegisterEnum("voltha.SelfTestResponse_SelfTestResult", SelfTestResponse_SelfTestResult_name, SelfTestResponse_SelfTestResult_value)
-	proto.RegisterType((*DeviceGroup)(nil), "voltha.DeviceGroup")
-	proto.RegisterType((*DeviceGroups)(nil), "voltha.DeviceGroups")
-	proto.RegisterType((*EventFilterRuleKey)(nil), "voltha.EventFilterRuleKey")
-	proto.RegisterType((*EventFilterRule)(nil), "voltha.EventFilterRule")
-	proto.RegisterType((*EventFilter)(nil), "voltha.EventFilter")
-	proto.RegisterType((*EventFilters)(nil), "voltha.EventFilters")
-	proto.RegisterType((*CoreInstance)(nil), "voltha.CoreInstance")
-	proto.RegisterType((*CoreInstances)(nil), "voltha.CoreInstances")
-	proto.RegisterType((*OmciTestRequest)(nil), "voltha.OmciTestRequest")
-	proto.RegisterType((*TestResponse)(nil), "voltha.TestResponse")
-	proto.RegisterType((*ValueSet)(nil), "voltha.ValueSet")
-	proto.RegisterType((*Voltha)(nil), "voltha.Voltha")
-	proto.RegisterType((*SelfTestResponse)(nil), "voltha.SelfTestResponse")
-	proto.RegisterType((*OfAgentSubscriber)(nil), "voltha.OfAgentSubscriber")
-	proto.RegisterType((*Membership)(nil), "voltha.Membership")
-	proto.RegisterType((*DeviceImageDownloadRequest)(nil), "voltha.DeviceImageDownloadRequest")
-	proto.RegisterType((*DeviceImageRequest)(nil), "voltha.DeviceImageRequest")
-	proto.RegisterType((*DeviceImageResponse)(nil), "voltha.DeviceImageResponse")
-	proto.RegisterType((*FlowMetadata)(nil), "voltha.FlowMetadata")
-}
-
-func init() { proto.RegisterFile("voltha_protos/voltha.proto", fileDescriptor_e084f1a60ce7016c) }
-
-var fileDescriptor_e084f1a60ce7016c = []byte{
-	// 2900 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x49, 0x73, 0x1b, 0xc7,
-	0x15, 0x16, 0xb8, 0xf3, 0x01, 0x24, 0x81, 0x06, 0x17, 0x08, 0x24, 0xb5, 0xb4, 0x6c, 0x89, 0xa6,
-	0x2d, 0xc0, 0x5a, 0xac, 0x72, 0xec, 0xb8, 0x62, 0x6e, 0xa2, 0x11, 0x4b, 0x02, 0x32, 0x20, 0x25,
-	0x27, 0xb1, 0x0a, 0x35, 0xc0, 0x34, 0xc0, 0x29, 0x01, 0x18, 0x64, 0xa6, 0x41, 0x49, 0xa5, 0x72,
-	0xa5, 0xe2, 0x2c, 0x76, 0xce, 0xbe, 0xe7, 0x94, 0x54, 0xaa, 0x72, 0xc9, 0x7f, 0xc8, 0xdd, 0xa7,
-	0x9c, 0x72, 0x4d, 0xe5, 0x90, 0x5f, 0xe0, 0x73, 0xaa, 0x5f, 0x77, 0x03, 0x33, 0x98, 0x19, 0x2e,
-	0xb2, 0xab, 0x72, 0x22, 0xa7, 0xfb, 0xf5, 0xf7, 0xbd, 0x7e, 0xdd, 0x6f, 0xe9, 0x6e, 0x40, 0xfe,
-	0xd8, 0x69, 0xf3, 0x23, 0xb3, 0xd6, 0x73, 0x1d, 0xee, 0x78, 0x45, 0xf9, 0x55, 0xc0, 0x2f, 0x32,
-	0x25, 0xbf, 0xf2, 0x6b, 0x2d, 0xc7, 0x69, 0xb5, 0x59, 0xd1, 0xec, 0xd9, 0x45, 0xb3, 0xdb, 0x75,
-	0xb8, 0xc9, 0x6d, 0xa7, 0xeb, 0x49, 0xa9, 0xfc, 0xaa, 0xea, 0xc5, 0xaf, 0x7a, 0xbf, 0x59, 0x64,
-	0x9d, 0x1e, 0x7f, 0xa9, 0x3a, 0x73, 0x41, 0xf8, 0x0e, 0xe3, 0x0a, 0x3c, 0x3f, 0x42, 0xdc, 0x70,
-	0x3a, 0x1d, 0xa7, 0x1b, 0xdd, 0x77, 0xc4, 0xcc, 0x36, 0x3f, 0x52, 0x7d, 0x34, 0xd8, 0xd7, 0x76,
-	0x5a, 0x76, 0xc3, 0x6c, 0xd7, 0x2c, 0x76, 0x6c, 0x37, 0x58, 0xf4, 0xf8, 0x40, 0xdf, 0x6a, 0xb0,
-	0xcf, 0xb4, 0xcc, 0x1e, 0x67, 0xae, 0xea, 0xbc, 0x1c, 0xec, 0x74, 0x7a, 0xac, 0xdb, 0x6c, 0x3b,
-	0xcf, 0x6b, 0xb7, 0xee, 0xc4, 0x08, 0x74, 0x1a, 0x76, 0xad, 0x63, 0xd7, 0x6b, 0x56, 0x5d, 0x09,
-	0x5c, 0x8d, 0x10, 0x30, 0xdb, 0xa6, 0xdb, 0x19, 0x8a, 0x5c, 0x0a, 0x8a, 0xb0, 0x17, 0xbc, 0xd6,
-	0x70, 0xba, 0x4d, 0xbb, 0x25, 0xfb, 0xe9, 0x9f, 0x13, 0x90, 0xdc, 0x45, 0x95, 0xf7, 0x5d, 0xa7,
-	0xdf, 0x23, 0x4b, 0x30, 0x66, 0x5b, 0xb9, 0xc4, 0x95, 0xc4, 0xc6, 0xec, 0xf6, 0xe4, 0x7f, 0xbf,
-	0xfb, 0x76, 0x3d, 0x61, 0x8c, 0xd9, 0x16, 0x29, 0xc1, 0x42, 0x70, 0xf2, 0x5e, 0x6e, 0xec, 0xca,
-	0xf8, 0x46, 0xf2, 0xf6, 0x52, 0x41, 0xad, 0xe2, 0x03, 0xd9, 0x2d, 0xb1, 0xb6, 0x67, 0xff, 0xfd,
-	0xdd, 0xb7, 0xeb, 0x13, 0x02, 0xcb, 0x98, 0x6f, 0xfb, 0x7b, 0x3c, 0x72, 0x07, 0xa6, 0x35, 0xc4,
-	0x38, 0x42, 0xcc, 0x6b, 0x88, 0xf0, 0x58, 0x2d, 0x49, 0x7f, 0x04, 0x29, 0x9f, 0x96, 0x1e, 0x79,
-	0x0b, 0x26, 0x6d, 0xce, 0x3a, 0x5e, 0x2e, 0x81, 0x10, 0xd9, 0x20, 0x04, 0x0a, 0x19, 0x52, 0x82,
-	0xfe, 0x29, 0x01, 0x64, 0xef, 0x98, 0x75, 0xf9, 0x7d, 0xbb, 0xcd, 0x99, 0x6b, 0xf4, 0xdb, 0xec,
-	0x53, 0xf6, 0x92, 0x7e, 0x95, 0x80, 0xec, 0x48, 0xf3, 0xc1, 0xcb, 0x1e, 0x23, 0xf3, 0x00, 0x4d,
-	0x6c, 0xa9, 0x99, 0xed, 0x76, 0xfa, 0x02, 0x49, 0xc1, 0x4c, 0xc3, 0xe4, 0xac, 0xe5, 0xb8, 0x2f,
-	0xd3, 0x09, 0x92, 0x86, 0x94, 0xd7, 0xaf, 0xd7, 0x06, 0x2d, 0x63, 0x84, 0xc0, 0xfc, 0xb3, 0x9e,
-	0x5d, 0x63, 0x02, 0xaa, 0xc6, 0x5f, 0xf6, 0x58, 0x7a, 0x9c, 0x2c, 0x41, 0x46, 0x1a, 0xd9, 0xdf,
-	0x3c, 0x21, 0x9a, 0xe5, 0x7c, 0xfc, 0xcd, 0x93, 0xd4, 0x86, 0x85, 0x11, 0x45, 0xc8, 0xc7, 0x30,
-	0xfe, 0x8c, 0xbd, 0xc4, 0x65, 0x98, 0xbf, 0x5d, 0xd0, 0x93, 0x0b, 0xcf, 0xa2, 0x10, 0x31, 0x03,
-	0x43, 0x0c, 0x25, 0x8b, 0x30, 0x79, 0x6c, 0xb6, 0xfb, 0x2c, 0x37, 0x26, 0x96, 0xd2, 0x90, 0x1f,
-	0xf4, 0xaf, 0x09, 0x48, 0xfa, 0x86, 0xc4, 0xad, 0xf6, 0x32, 0x4c, 0xb1, 0xae, 0x59, 0x6f, 0xcb,
-	0xd1, 0x33, 0x86, 0xfa, 0x22, 0xab, 0x30, 0xab, 0x26, 0x60, 0x5b, 0xb9, 0x71, 0x04, 0x9e, 0x91,
-	0x0d, 0x25, 0x8b, 0xac, 0x03, 0x0c, 0xa7, 0x95, 0x9b, 0xc0, 0xde, 0x59, 0x6c, 0x41, 0xbb, 0xde,
-	0x84, 0x49, 0xb7, 0xdf, 0x66, 0x5e, 0x6e, 0x12, 0x57, 0x6c, 0x25, 0x66, 0x52, 0x86, 0x94, 0xa2,
-	0x1f, 0x41, 0xca, 0xd7, 0xe3, 0x91, 0x9b, 0x30, 0x2d, 0x97, 0x25, 0xb4, 0xe4, 0x7e, 0x00, 0x2d,
-	0x43, 0x9f, 0x41, 0x6a, 0xc7, 0x71, 0x59, 0xa9, 0xeb, 0x71, 0xb3, 0xdb, 0x60, 0xe4, 0x3a, 0x24,
-	0x6d, 0xf5, 0x7f, 0x6d, 0x74, 0xc6, 0xa0, 0x7b, 0x4a, 0x16, 0xb9, 0x03, 0x53, 0x32, 0x00, 0xe0,
-	0xcc, 0x93, 0xb7, 0x17, 0x35, 0xcb, 0x27, 0xd8, 0x5a, 0xe5, 0x26, 0xef, 0x7b, 0xdb, 0x93, 0x62,
-	0x87, 0x5e, 0x30, 0x94, 0x28, 0xfd, 0x10, 0xe6, 0xfc, 0x64, 0x1e, 0xd9, 0x0c, 0xee, 0xce, 0x01,
-	0x88, 0x5f, 0x4a, 0x6f, 0xcf, 0xf7, 0x60, 0xa1, 0xdc, 0x69, 0xd8, 0x07, 0xcc, 0xe3, 0x06, 0xfb,
-	0x55, 0x9f, 0x79, 0x9c, 0xcc, 0x0f, 0x57, 0x05, 0x97, 0x83, 0xc0, 0x44, 0xbf, 0x6f, 0x5b, 0x6a,
-	0x29, 0xf1, 0x7f, 0xfa, 0x6b, 0x48, 0xc9, 0x21, 0x5e, 0xcf, 0xe9, 0x7a, 0x8c, 0xfc, 0x04, 0xa6,
-	0x5c, 0xe6, 0xf5, 0xdb, 0x5c, 0x6d, 0x9a, 0x1b, 0x9a, 0xd3, 0x2f, 0x15, 0xf8, 0x30, 0x50, 0xdc,
-	0x50, 0xc3, 0x68, 0x01, 0x48, 0xb8, 0x97, 0x24, 0x61, 0xba, 0x7a, 0xb8, 0xb3, 0xb3, 0x57, 0xad,
-	0xa6, 0x2f, 0x88, 0x8f, 0xfb, 0x5b, 0xa5, 0x07, 0x87, 0xc6, 0x5e, 0x3a, 0x41, 0x9f, 0xc2, 0xcc,
-	0x63, 0xb1, 0xa7, 0xaa, 0x2c, 0xac, 0xf0, 0xfb, 0x90, 0x92, 0x61, 0x48, 0x7a, 0x81, 0xb2, 0x65,
-	0xb6, 0xa0, 0x22, 0xcf, 0x96, 0xe8, 0xdb, 0xc1, 0xff, 0x3f, 0xb9, 0x60, 0x24, 0xcd, 0xe1, 0xe7,
-	0xf6, 0xb4, 0xda, 0xb6, 0xf4, 0x5f, 0x13, 0x30, 0xf5, 0x18, 0x67, 0x40, 0x2e, 0xc3, 0xf4, 0x31,
-	0x73, 0x3d, 0xdb, 0xe9, 0x06, 0xd7, 0x4d, 0xb7, 0x92, 0x7b, 0x30, 0xa3, 0x22, 0xab, 0x8e, 0x4a,
-	0x0b, 0x7a, 0xf6, 0x5b, 0xb2, 0xdd, 0x1f, 0x53, 0x06, 0xb2, 0x51, 0x41, 0x6d, 0xfc, 0xfb, 0x07,
-	0xb5, 0x89, 0xb3, 0x06, 0x35, 0xf2, 0x31, 0xa4, 0x94, 0x3b, 0x09, 0x97, 0xd1, 0x9e, 0x41, 0x82,
-	0x23, 0x85, 0xf3, 0xf8, 0x47, 0x27, 0xad, 0x41, 0xb3, 0x47, 0x76, 0x60, 0x4e, 0x21, 0xb4, 0x30,
-	0x2e, 0xe6, 0xa6, 0x62, 0xc3, 0xa1, 0x1f, 0x43, 0xd1, 0xaa, 0x58, 0xba, 0x03, 0x73, 0xd2, 0x71,
-	0xb5, 0x83, 0x4d, 0xc7, 0x3a, 0x58, 0x00, 0x84, 0xf9, 0xfd, 0xf3, 0x67, 0x90, 0x19, 0xe6, 0x27,
-	0x93, 0x9b, 0x75, 0xd3, 0x63, 0xb9, 0x35, 0x05, 0x24, 0x7a, 0x0a, 0x0f, 0xed, 0xba, 0x54, 0x67,
-	0xd7, 0xe4, 0xe6, 0x76, 0x5a, 0x00, 0x25, 0x7d, 0xf1, 0xc4, 0x58, 0x10, 0x52, 0x42, 0x48, 0x8d,
-	0x26, 0x4f, 0x20, 0xeb, 0xcf, 0x68, 0x1a, 0x74, 0x5d, 0x2d, 0x11, 0x82, 0xe2, 0x56, 0x3a, 0x11,
-	0x16, 0xd5, 0x92, 0x62, 0x0a, 0x81, 0xfe, 0x25, 0x01, 0xe9, 0x2a, 0x6b, 0x37, 0xcf, 0xe6, 0x40,
-	0xa3, 0x92, 0xfe, 0x06, 0xbf, 0x03, 0x55, 0x60, 0x3e, 0xd8, 0x13, 0xef, 0x3c, 0x24, 0x03, 0x73,
-	0x8f, 0xca, 0x07, 0xb5, 0xea, 0x61, 0xa5, 0x52, 0x36, 0x0e, 0xf6, 0x76, 0xd3, 0x63, 0xa2, 0xe9,
-	0xf0, 0xd1, 0xa7, 0x8f, 0xca, 0x4f, 0x1e, 0xd5, 0xf6, 0x0c, 0xa3, 0x6c, 0xa4, 0xc7, 0x69, 0x19,
-	0x32, 0xe5, 0xe6, 0x56, 0x8b, 0x75, 0x79, 0xb5, 0x5f, 0xf7, 0x1a, 0xae, 0x5d, 0x67, 0xae, 0x08,
-	0xb3, 0x4e, 0xd3, 0x14, 0x8d, 0x83, 0x40, 0x66, 0xcc, 0xaa, 0x96, 0x92, 0x25, 0x42, 0xb4, 0xca,
-	0xf8, 0x83, 0x80, 0x31, 0x23, 0x1b, 0x4a, 0x16, 0xfd, 0x10, 0xe0, 0x21, 0xeb, 0xd4, 0x99, 0xeb,
-	0x1d, 0xd9, 0x3d, 0x81, 0x84, 0xbb, 0xa6, 0xd6, 0x35, 0x3b, 0x4c, 0x23, 0x61, 0xcb, 0x23, 0xb3,
-	0xc3, 0x94, 0x53, 0x8f, 0x69, 0xa7, 0xa6, 0xff, 0x48, 0x40, 0x5e, 0x5a, 0xba, 0xd4, 0x31, 0x5b,
-	0x6c, 0xd7, 0x79, 0xde, 0x6d, 0x3b, 0xa6, 0xa5, 0x83, 0xd6, 0x0d, 0x7f, 0x6e, 0x90, 0x71, 0x0f,
-	0x0a, 0xaa, 0xd0, 0x2a, 0xed, 0xfa, 0xf2, 0xc4, 0x35, 0x98, 0xb4, 0x05, 0x80, 0x8a, 0x0a, 0x73,
-	0xda, 0xce, 0x88, 0x6a, 0xc8, 0x3e, 0xf2, 0x0e, 0x64, 0xcc, 0x06, 0xb7, 0x8f, 0x4d, 0xce, 0xca,
-	0xdd, 0x6a, 0xbf, 0xd1, 0x60, 0x9e, 0x87, 0x19, 0x67, 0xc6, 0x08, 0x77, 0x90, 0x0d, 0x58, 0x10,
-	0x4c, 0x36, 0x1f, 0xca, 0x4e, 0xa0, 0xec, 0x68, 0x33, 0xfd, 0x4d, 0x02, 0x88, 0x6f, 0x12, 0xe7,
-	0x56, 0x3e, 0x37, 0x8c, 0x45, 0xd2, 0x32, 0x83, 0x20, 0x14, 0xa1, 0xc3, 0x78, 0xb4, 0x0e, 0x35,
-	0xc8, 0x06, 0x54, 0x50, 0x1b, 0xf0, 0x13, 0xc8, 0x6a, 0x1d, 0x44, 0x7b, 0xcd, 0xe3, 0x26, 0x67,
-	0x3a, 0x85, 0xe4, 0x82, 0x1e, 0x8d, 0x23, 0x45, 0x32, 0x62, 0x86, 0x2a, 0x29, 0x86, 0x2d, 0x1e,
-	0xdd, 0x83, 0xd4, 0xfd, 0xb6, 0xf3, 0xfc, 0x21, 0xe3, 0xa6, 0xf0, 0x1a, 0xf2, 0x1e, 0x4c, 0x75,
-	0x98, 0x2f, 0x75, 0xae, 0x17, 0xfc, 0xb5, 0xa6, 0xd3, 0xec, 0xd5, 0xb0, 0x5b, 0x45, 0x6b, 0x43,
-	0x09, 0xdf, 0xfe, 0xfb, 0x3d, 0x98, 0x93, 0x21, 0xb8, 0xca, 0x5c, 0xc1, 0x41, 0x9e, 0xc0, 0xdc,
-	0x3e, 0xe3, 0xbe, 0x2d, 0xb4, 0x5c, 0x90, 0xf5, 0x78, 0x41, 0xd7, 0xe3, 0x85, 0x3d, 0x51, 0x8f,
-	0xe7, 0x07, 0x31, 0x6c, 0x28, 0x4b, 0xf3, 0x5f, 0xfe, 0xf3, 0x3f, 0xdf, 0x8c, 0x2d, 0x12, 0x82,
-	0xa5, 0xfd, 0xf1, 0xad, 0x62, 0x67, 0x88, 0xf3, 0x14, 0xd2, 0x87, 0x3d, 0xcb, 0xe4, 0xcc, 0x87,
-	0x1d, 0x81, 0x91, 0x8f, 0xe1, 0xa3, 0xeb, 0x88, 0xbd, 0x42, 0x23, 0xb0, 0x3f, 0x48, 0x6c, 0x92,
-	0x5d, 0x98, 0xdd, 0x67, 0x5c, 0xa5, 0x93, 0x38, 0x9d, 0x07, 0x11, 0x5b, 0xca, 0xd1, 0x05, 0xc4,
-	0x9c, 0x25, 0xd3, 0x0a, 0x93, 0x3c, 0x85, 0xcc, 0x03, 0xdb, 0xe3, 0xc1, 0x54, 0x1f, 0x87, 0xb6,
-	0x14, 0x95, 0xf3, 0x3d, 0x7a, 0x11, 0x41, 0xb3, 0x24, 0xa3, 0x15, 0xb5, 0x07, 0x48, 0x55, 0x58,
-	0xd8, 0x67, 0x01, 0x74, 0xe2, 0xdb, 0x83, 0xf9, 0xc8, 0x22, 0x82, 0x5e, 0x42, 0xbc, 0x1c, 0x59,
-	0x0e, 0xe1, 0x15, 0x5f, 0xd9, 0xd6, 0x17, 0xc4, 0x80, 0x94, 0xd0, 0x79, 0x4b, 0xa7, 0xbc, 0x38,
-	0x75, 0xd3, 0x23, 0x09, 0xd3, 0xa3, 0x39, 0x44, 0x26, 0x24, 0xad, 0x91, 0x07, 0x69, 0x93, 0x01,
-	0x11, 0x98, 0x0f, 0x82, 0x19, 0x30, 0x0e, 0x79, 0x39, 0x32, 0x97, 0x7a, 0xf4, 0x32, 0xe2, 0x5f,
-	0x24, 0x2b, 0x1a, 0x7f, 0x24, 0x15, 0x93, 0x5f, 0x42, 0x7a, 0x9f, 0x05, 0x59, 0x02, 0x06, 0x89,
-	0x4e, 0xd2, 0xf4, 0x0d, 0xc4, 0xbd, 0x44, 0xd6, 0x62, 0x70, 0xa5, 0x5d, 0x9a, 0xb0, 0x1c, 0x9a,
-	0x43, 0xc5, 0x71, 0xb9, 0x17, 0x6d, 0x73, 0x25, 0x87, 0x12, 0x74, 0x13, 0x19, 0xde, 0x20, 0xf4,
-	0x24, 0x86, 0x62, 0x0f, 0xd1, 0x5e, 0xc0, 0xe2, 0xe8, 0x24, 0x04, 0x08, 0x59, 0x8a, 0x40, 0x2e,
-	0x59, 0xf9, 0x6c, 0x44, 0x33, 0xbd, 0x8b, 0x7c, 0x05, 0xf2, 0xce, 0xe9, 0x7c, 0xc5, 0x57, 0xe2,
-	0x4f, 0x4d, 0xcc, 0xf0, 0xf7, 0x09, 0x58, 0xd9, 0xc3, 0xb2, 0xfd, 0xcc, 0xec, 0x71, 0xde, 0xf5,
-	0x21, 0x2a, 0xf0, 0x1e, 0xbd, 0x73, 0x1e, 0x05, 0x8a, 0xea, 0xcc, 0xf0, 0x55, 0x02, 0x72, 0xbb,
-	0xb6, 0xf7, 0x83, 0x28, 0xf2, 0x63, 0x54, 0xe4, 0x1e, 0xbd, 0x7b, 0x2e, 0x45, 0x2c, 0xc9, 0x4e,
-	0xac, 0x88, 0x35, 0x17, 0x71, 0x32, 0xb8, 0xe6, 0x24, 0x10, 0x1c, 0xb1, 0xff, 0x8c, 0x2b, 0xde,
-	0x44, 0xac, 0xdf, 0x26, 0x60, 0x4d, 0xc6, 0xb2, 0x10, 0xd1, 0x01, 0xaa, 0xb1, 0x16, 0x22, 0xc0,
-	0x76, 0x39, 0x26, 0x76, 0xea, 0x37, 0x51, 0x85, 0x1b, 0xf4, 0x0c, 0x2a, 0x88, 0x88, 0xf7, 0xbb,
-	0x04, 0xac, 0x47, 0x68, 0xf1, 0x50, 0x44, 0x76, 0xa9, 0xc6, 0x6a, 0x40, 0x0d, 0xec, 0x78, 0xe8,
-	0x58, 0xa7, 0x68, 0x51, 0x40, 0x2d, 0x36, 0xe8, 0xb5, 0x13, 0xb5, 0x90, 0xf9, 0x43, 0xa8, 0xd1,
-	0x82, 0x95, 0x90, 0xc9, 0x91, 0x2a, 0x68, 0xf3, 0x6c, 0x58, 0x17, 0x8f, 0xbe, 0x8d, 0x5c, 0x6f,
-	0x92, 0xb3, 0x70, 0x11, 0x0e, 0xab, 0x91, 0x6b, 0xab, 0x4a, 0x5c, 0x3f, 0xd9, 0x4a, 0xc8, 0xfe,
-	0x52, 0x88, 0xbe, 0x8b, 0x84, 0x9b, 0x64, 0xe3, 0x54, 0x13, 0xab, 0x6a, 0x9b, 0x7c, 0x93, 0x80,
-	0xab, 0x31, 0x6b, 0x8d, 0x98, 0xd2, 0xd2, 0x57, 0xa3, 0x09, 0xcf, 0xb2, 0xea, 0x77, 0x50, 0xa5,
-	0x9b, 0xf4, 0xcc, 0x2a, 0x09, 0xa3, 0x97, 0x21, 0x29, 0x6c, 0x71, 0x5a, 0x60, 0x5e, 0x08, 0x96,
-	0x14, 0x1e, 0x5d, 0x41, 0xb2, 0x0c, 0x59, 0xd0, 0x64, 0x3a, 0x12, 0x97, 0x61, 0x6e, 0x08, 0x58,
-	0xb2, 0xe2, 0x21, 0x93, 0x43, 0x33, 0x47, 0xa4, 0x3a, 0x09, 0x67, 0x5b, 0x1e, 0x39, 0x84, 0xb4,
-	0xc1, 0x1a, 0x4e, 0xb7, 0x61, 0xb7, 0x99, 0x56, 0xd3, 0x3f, 0x36, 0xd6, 0x1e, 0x6b, 0x88, 0xb9,
-	0x4c, 0xc3, 0x98, 0x62, 0xe2, 0x7b, 0x98, 0xe6, 0x23, 0x52, 0xc5, 0xc8, 0x61, 0x4c, 0xc3, 0x90,
-	0xc5, 0x91, 0x99, 0xca, 0xdc, 0xf0, 0x53, 0x48, 0xed, 0xb8, 0xcc, 0xe4, 0x4a, 0x35, 0x32, 0x32,
-	0x3a, 0x84, 0xa6, 0x0a, 0x1b, 0x3a, 0x6a, 0x37, 0xa1, 0xd2, 0x13, 0x48, 0xc9, 0x20, 0x1c, 0xa1,
-	0x55, 0xdc, 0x24, 0xaf, 0x21, 0xde, 0x3a, 0x5d, 0x8d, 0xd2, 0x4e, 0x87, 0xd5, 0x9f, 0xc3, 0x9c,
-	0x8a, 0xaa, 0xe7, 0x40, 0x56, 0xb9, 0x91, 0xae, 0x45, 0x22, 0xeb, 0x38, 0xf9, 0x04, 0x52, 0x06,
-	0xab, 0x3b, 0x0e, 0xff, 0xc1, 0x74, 0x76, 0x11, 0x4e, 0x00, 0xef, 0xb2, 0x36, 0xe3, 0xaf, 0x61,
-	0x8c, 0xcd, 0x68, 0x60, 0x0b, 0xe1, 0x48, 0x1d, 0x32, 0xf7, 0x1d, 0xb7, 0xc1, 0xce, 0x8d, 0xfe,
-	0x16, 0xa2, 0x5f, 0xdb, 0xbc, 0x1a, 0x89, 0xde, 0x14, 0x98, 0x35, 0xc5, 0xd1, 0x87, 0x39, 0x7d,
-	0xe4, 0xc1, 0x5a, 0x7b, 0x98, 0xbb, 0x02, 0xc7, 0xa1, 0xfc, 0x92, 0xa6, 0x2d, 0xf7, 0x98, 0x8b,
-	0x77, 0xd7, 0xa2, 0xc2, 0xa7, 0xf7, 0x90, 0xe9, 0x5d, 0xfa, 0x76, 0x24, 0x93, 0xac, 0xf8, 0x2d,
-	0x85, 0xe1, 0x15, 0x5f, 0x89, 0x23, 0xd9, 0x17, 0x62, 0x03, 0x7d, 0x99, 0x80, 0xe5, 0x7d, 0xc6,
-	0x03, 0x1c, 0xf2, 0x16, 0x2a, 0x5e, 0x81, 0xa8, 0x66, 0xfa, 0x01, 0x2a, 0x70, 0x97, 0xdc, 0x3e,
-	0x87, 0x02, 0x45, 0x4f, 0x32, 0xf5, 0xb1, 0x14, 0x0b, 0xe0, 0x9d, 0x93, 0x5d, 0x05, 0x32, 0x72,
-	0x9e, 0xe9, 0x93, 0xa6, 0x2c, 0x34, 0x03, 0x48, 0xde, 0xc8, 0xba, 0x46, 0xb1, 0x79, 0xf4, 0x1d,
-	0xa4, 0xbb, 0x4e, 0xde, 0x38, 0x0b, 0x1d, 0x79, 0x01, 0xd9, 0x1d, 0x51, 0x33, 0xb7, 0xcf, 0x38,
-	0xc3, 0xc8, 0x05, 0x56, 0x33, 0xdc, 0x3c, 0xd7, 0x0c, 0xbf, 0x4e, 0x40, 0x76, 0x4b, 0x1d, 0x67,
-	0x91, 0x45, 0xe6, 0x83, 0x73, 0x52, 0xef, 0x20, 0xf5, 0x47, 0xf4, 0xfd, 0xf3, 0x2c, 0xad, 0x6c,
-	0xee, 0x23, 0x9f, 0xd8, 0x68, 0x7f, 0x48, 0x40, 0xc6, 0x60, 0xc7, 0xcc, 0xe5, 0xff, 0x17, 0x45,
-	0x5c, 0xa4, 0x16, 0x8a, 0x7c, 0x9d, 0x80, 0xa5, 0x80, 0xa7, 0x1d, 0x38, 0xca, 0xa3, 0x69, 0xc4,
-	0x21, 0x78, 0xe4, 0x1a, 0x22, 0xbf, 0x1a, 0x21, 0xa3, 0x8f, 0xd8, 0xba, 0x7c, 0x21, 0xd7, 0x47,
-	0xf5, 0x43, 0x1d, 0xbc, 0xa2, 0xd6, 0x4d, 0x9e, 0xc0, 0x3d, 0xf2, 0x1c, 0xe6, 0xf5, 0xbe, 0x57,
-	0x3e, 0x97, 0x8f, 0x84, 0x3f, 0x03, 0x75, 0xec, 0x8e, 0x54, 0xd4, 0xf2, 0x4f, 0x4d, 0x39, 0xdc,
-	0x1f, 0x13, 0x70, 0x71, 0xab, 0xee, 0x0c, 0xd6, 0xa2, 0xe5, 0x9a, 0xd6, 0xd0, 0x0e, 0xaf, 0xad,
-	0x44, 0xac, 0x17, 0x2a, 0x25, 0x4c, 0x41, 0x59, 0xeb, 0x4b, 0x3a, 0x6d, 0x84, 0xc7, 0x90, 0xda,
-	0x67, 0xbc, 0xdc, 0xed, 0x97, 0xe4, 0xb7, 0xdf, 0xff, 0x32, 0x9a, 0x6d, 0xd0, 0x4d, 0x6f, 0x20,
-	0xc7, 0x55, 0x72, 0x39, 0x72, 0x0f, 0x38, 0xdd, 0xbe, 0xc6, 0x7d, 0x05, 0x73, 0x81, 0xad, 0xff,
-	0xfa, 0xd3, 0xba, 0x85, 0x94, 0x6f, 0xd3, 0xb8, 0x65, 0xd5, 0x17, 0x46, 0x8a, 0x59, 0x6c, 0xb2,
-	0xe7, 0x90, 0xdc, 0xc1, 0x6b, 0x99, 0xef, 0x49, 0x5d, 0x44, 0xea, 0xb7, 0x68, 0xdc, 0xb2, 0xca,
-	0xbb, 0x1f, 0x1f, 0x71, 0x05, 0x16, 0x86, 0xb5, 0x54, 0xf8, 0xc4, 0x39, 0xb8, 0x0d, 0x93, 0x47,
-	0x4d, 0x8a, 0xf0, 0x6b, 0x24, 0x1f, 0x69, 0x4c, 0x79, 0xc4, 0x7c, 0x0a, 0x59, 0x1f, 0xa2, 0xba,
-	0x48, 0x8f, 0x59, 0xa6, 0x41, 0xf7, 0x29, 0xcb, 0xd4, 0xd3, 0x57, 0xf7, 0x1e, 0xe9, 0xc2, 0x92,
-	0x8c, 0x05, 0xa3, 0x04, 0x61, 0xd0, 0xd8, 0x34, 0xab, 0xce, 0x4f, 0xf4, 0x34, 0x32, 0x61, 0xa0,
-	0x43, 0xbf, 0x81, 0xce, 0x76, 0x3c, 0x3b, 0xd9, 0x4a, 0xf2, 0x58, 0xc6, 0x60, 0x31, 0x08, 0x7b,
-	0x9e, 0x93, 0xc1, 0x06, 0x12, 0x50, 0x72, 0x25, 0x96, 0x40, 0x9f, 0x08, 0x3e, 0xf7, 0x6b, 0x2f,
-	0xef, 0xe8, 0xe3, 0x8a, 0xe5, 0x6c, 0xf8, 0x9e, 0xdf, 0x8b, 0xab, 0x4c, 0xe5, 0x03, 0x01, 0x31,
-	0xf0, 0xfe, 0x6d, 0x28, 0x3f, 0x62, 0x99, 0x10, 0x1e, 0xbd, 0x8a, 0x70, 0xab, 0xe4, 0x62, 0x14,
-	0x9c, 0xac, 0x76, 0x6b, 0x90, 0x1e, 0x6a, 0xac, 0x8c, 0x12, 0xa7, 0xf2, 0x62, 0xc4, 0xbb, 0x82,
-	0xa7, 0x2f, 0xdf, 0xc8, 0xd2, 0x08, 0x89, 0x32, 0xc9, 0x7d, 0x48, 0x57, 0xb9, 0xcb, 0xcc, 0x4e,
-	0xc5, 0x6c, 0x3c, 0x63, 0xdc, 0x2b, 0xf7, 0x39, 0x59, 0x0e, 0x58, 0x5a, 0x76, 0x94, 0xfb, 0x3c,
-	0x76, 0x03, 0x5d, 0xd8, 0x48, 0x90, 0x3d, 0x3c, 0x34, 0x30, 0xfb, 0x98, 0x29, 0xa0, 0x52, 0xf7,
-	0x84, 0xdb, 0xb7, 0x30, 0x7e, 0xa9, 0x4b, 0x2f, 0xbc, 0x9b, 0x20, 0x9f, 0x42, 0x56, 0xc1, 0xec,
-	0x1c, 0x99, 0xdd, 0x16, 0xc3, 0xd7, 0x8d, 0xf8, 0x29, 0xe7, 0x02, 0x48, 0xbe, 0x21, 0x08, 0x76,
-	0x88, 0x09, 0xc2, 0xff, 0x7e, 0x1e, 0x3c, 0xd6, 0x86, 0xcd, 0x15, 0xb7, 0x59, 0x95, 0xb5, 0xf4,
-	0x9a, 0x64, 0xe4, 0x09, 0xc4, 0xff, 0x56, 0x1b, 0xf5, 0x1e, 0x93, 0x8f, 0x6a, 0xa4, 0x57, 0x90,
-	0x22, 0x4f, 0x07, 0x0b, 0x12, 0x78, 0xde, 0x11, 0x4e, 0xf6, 0x18, 0xf5, 0xf6, 0xa3, 0x47, 0x5e,
-	0x7b, 0xf9, 0x5f, 0x60, 0xc3, 0x8a, 0x07, 0x50, 0xa5, 0xe2, 0x16, 0x64, 0x64, 0xb0, 0x78, 0x3d,
-	0xc5, 0xdf, 0x44, 0x8a, 0xcb, 0xf9, 0x13, 0x28, 0x84, 0xf6, 0x16, 0x64, 0x64, 0xa5, 0x7f, 0x2a,
-	0x4b, 0xdc, 0x7e, 0x52, 0x73, 0xd9, 0x3c, 0x69, 0x2e, 0xca, 0x31, 0x02, 0xaf, 0xd0, 0xa7, 0x3a,
-	0x46, 0xc0, 0x62, 0x21, 0xc7, 0x08, 0xb0, 0x90, 0x07, 0x78, 0x5c, 0x8d, 0xc8, 0xaa, 0xf3, 0x81,
-	0xa2, 0xcb, 0xd3, 0x67, 0x20, 0xb2, 0x1a, 0x5f, 0x56, 0x79, 0xe4, 0x33, 0x98, 0xd1, 0xcf, 0x4f,
-	0x01, 0xb0, 0x5c, 0xdc, 0x3b, 0x16, 0xbd, 0x8e, 0xb0, 0x57, 0xe8, 0xa5, 0x48, 0x58, 0x8f, 0xb5,
-	0x9b, 0x35, 0x2e, 0xd0, 0x1e, 0x63, 0xf5, 0x1f, 0x78, 0xbe, 0x1b, 0xbd, 0xbd, 0x09, 0xbd, 0xef,
-	0x85, 0x23, 0x8f, 0x70, 0x23, 0x21, 0xa7, 0xae, 0x6d, 0xec, 0x3a, 0xf9, 0x1c, 0xc8, 0x3e, 0xe3,
-	0x23, 0x2f, 0x78, 0x23, 0x57, 0xbc, 0x51, 0x8f, 0x7c, 0x61, 0x7b, 0x04, 0xb1, 0xf1, 0xbd, 0x90,
-	0x78, 0x30, 0x57, 0xb5, 0x3b, 0xfd, 0xb6, 0xc9, 0x19, 0x8e, 0x27, 0x6b, 0x03, 0x43, 0xf8, 0x9b,
-	0x75, 0x96, 0x8f, 0xa9, 0x68, 0x43, 0xd7, 0x6e, 0x41, 0x1b, 0x29, 0xa4, 0x9a, 0x40, 0x12, 0x3b,
-	0x73, 0x07, 0x66, 0x07, 0x4f, 0x75, 0xe4, 0xe2, 0xa0, 0x38, 0x1a, 0x7d, 0xc4, 0xcb, 0xc7, 0x77,
-	0xd1, 0x0b, 0xe4, 0x21, 0x80, 0xbc, 0x33, 0xc0, 0x2b, 0xd2, 0x94, 0xbf, 0x22, 0x88, 0xdd, 0xd0,
-	0xea, 0xb2, 0x85, 0xce, 0x0b, 0x1d, 0x87, 0xa3, 0xd5, 0x75, 0x90, 0xba, 0x29, 0x38, 0x07, 0xde,
-	0xf0, 0x4e, 0xe3, 0xf8, 0x56, 0xd1, 0x37, 0x5c, 0x00, 0x7e, 0x06, 0x49, 0x11, 0x3c, 0x5e, 0x70,
-	0x7c, 0xff, 0x27, 0xcb, 0xda, 0x72, 0xf2, 0xe7, 0x00, 0x3d, 0xd6, 0xb0, 0x9b, 0x36, 0x73, 0xf3,
-	0x8b, 0xba, 0xdd, 0x60, 0xbc, 0xef, 0x76, 0xb1, 0xd7, 0xa3, 0xab, 0x08, 0xbc, 0x44, 0xb2, 0xda,
-	0xa0, 0x7e, 0xa8, 0x03, 0x48, 0x56, 0x7d, 0x9f, 0x83, 0x47, 0x09, 0xfd, 0x43, 0x83, 0x58, 0x75,
-	0x43, 0xa8, 0x7e, 0x98, 0x23, 0xc8, 0x56, 0xb9, 0xe9, 0x72, 0xfd, 0x33, 0x0b, 0x51, 0x75, 0x3a,
-	0x5d, 0x32, 0xf8, 0x05, 0xca, 0xc8, 0xcf, 0x2f, 0x86, 0xce, 0x1c, 0xf0, 0x16, 0x15, 0x32, 0xe8,
-	0xe0, 0xbd, 0xc2, 0x13, 0x98, 0x35, 0x7c, 0xa1, 0x16, 0x6e, 0xf2, 0x41, 0x62, 0x73, 0xbb, 0x0d,
-	0x59, 0xc7, 0x6d, 0x61, 0xc6, 0x68, 0x38, 0xae, 0xa5, 0x70, 0xb6, 0x53, 0xf2, 0x45, 0xa9, 0x82,
-	0x3f, 0xc1, 0xfa, 0x45, 0xa1, 0x65, 0xf3, 0xa3, 0x7e, 0x5d, 0x58, 0xa7, 0xa8, 0x25, 0xd5, 0x4f,
-	0xe1, 0x6e, 0xea, 0x1f, 0xc6, 0xdd, 0x2d, 0xb6, 0x1c, 0xd5, 0xf6, 0xb7, 0xb1, 0xe5, 0xb2, 0xc6,
-	0x7b, 0xec, 0x7f, 0xa0, 0xaa, 0x8c, 0x55, 0xc6, 0x2b, 0x13, 0x95, 0xc9, 0xca, 0x54, 0x65, 0xba,
-	0x32, 0x53, 0x9f, 0xc2, 0xb1, 0x77, 0xfe, 0x17, 0x00, 0x00, 0xff, 0xff, 0x87, 0x9b, 0x36, 0xf4,
-	0x64, 0x27, 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
-
-// VolthaServiceClient is the client API for VolthaService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type VolthaServiceClient interface {
-	// Get the membership group of a Voltha Core
-	GetMembership(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Membership, error)
-	// Set the membership group of a Voltha Core
-	UpdateMembership(ctx context.Context, in *Membership, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Get high level information on the Voltha cluster
-	GetVoltha(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Voltha, error)
-	// List all Voltha cluster core instances
-	ListCoreInstances(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*CoreInstances, error)
-	// Get details on a Voltha cluster instance
-	GetCoreInstance(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*CoreInstance, error)
-	// List all active adapters (plugins) in the Voltha cluster
-	ListAdapters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Adapters, error)
-	// List all logical devices managed by the Voltha cluster
-	ListLogicalDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*LogicalDevices, error)
-	// Get additional information on a given logical device
-	GetLogicalDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalDevice, error)
-	// List ports of a logical device
-	ListLogicalDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalPorts, error)
-	// Gets a logical device port
-	GetLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*LogicalPort, error)
-	// Enables a logical device port
-	EnableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Disables a logical device port
-	DisableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error)
-	// List all flows of a logical device
-	ListLogicalDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error)
-	// Update flow table for logical device
-	UpdateLogicalDeviceFlowTable(ctx context.Context, in *openflow_13.FlowTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Update meter table for logical device
-	UpdateLogicalDeviceMeterTable(ctx context.Context, in *openflow_13.MeterModUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
-	// List all meters of a logical device
-	ListLogicalDeviceMeters(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Meters, error)
-	// List all flow groups of a logical device
-	ListLogicalDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error)
-	// Update group table for device
-	UpdateLogicalDeviceFlowGroupTable(ctx context.Context, in *openflow_13.FlowGroupTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
-	// List all physical devices controlled by the Voltha cluster
-	ListDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Devices, error)
-	// List all physical devices IDs controlled by the Voltha cluster
-	ListDeviceIds(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*common.IDs, error)
-	// Request to a voltha Core to reconcile a set of devices based on their IDs
-	ReconcileDevices(ctx context.Context, in *common.IDs, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Get more information on a given physical device
-	GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Device, error)
-	// Pre-provision a new physical device
-	CreateDevice(ctx context.Context, in *Device, opts ...grpc.CallOption) (*Device, error)
-	// Enable a device.  If the device was in pre-provisioned state then it
-	// will transition to ENABLED state.  If it was is DISABLED state then it
-	// will transition to ENABLED state as well.
-	EnableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Disable a device
-	DisableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Reboot a device
-	RebootDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Delete a device
-	DeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Forcefully delete a device
-	ForceDeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Request an image download to the standby partition
-	// of a device.
-	// Note that the call is expected to be non-blocking.
-	DownloadImage(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
-	// Get image download status on a device
-	// The request retrieves progress on device and updates db record
-	// Deprecated in voltha 2.8, will be removed
-	GetImageDownloadStatus(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error)
-	// Get image download db record
-	// Deprecated in voltha 2.8, will be removed
-	GetImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error)
-	// List image download db records for a given device
-	// Deprecated in voltha 2.8, will be removed
-	ListImageDownloads(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*ImageDownloads, error)
-	// Cancel an existing image download process on a device
-	// Deprecated in voltha 2.8, will be removed
-	CancelImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
-	// Activate the specified image at a standby partition
-	// to active partition.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device
-	// Note that the call is expected to be non-blocking.
-	// Deprecated in voltha 2.8, will be removed
-	ActivateImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
-	// Revert the specified image at standby partition
-	// to active partition, and revert to previous image
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// previous image running on device
-	// Note that the call is expected to be non-blocking.
-	// Deprecated in voltha 2.8, will be removed
-	RevertImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
-	// Downloads a certain image to the standby partition of the devices
-	// Note that the call is expected to be non-blocking.
-	DownloadImageToDevice(ctx context.Context, in *DeviceImageDownloadRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
-	// Get image status on a number of devices devices
-	// Polled from northbound systems to get state of download/activate/commit
-	GetImageStatus(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
-	// Aborts the upgrade of an image on a device
-	// To be used carefully, stops any further operations for the Image on the given devices
-	// Might also stop if possible existing work, but no guarantees are given,
-	// depends on implementation and procedure status.
-	AbortImageUpgradeToDevice(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
-	// Get Both Active and Standby image for a given device
-	GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*OnuImages, error)
-	// Activate the specified image from a standby partition
-	// to active partition.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device
-	// Note that the call is expected to be non-blocking.
-	ActivateImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
-	// Commit the specified image to be default.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device upon next reboot
-	// Note that the call is expected to be non-blocking.
-	CommitImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
-	// List ports of a device
-	ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Ports, error)
-	// List pm config of a device
-	ListDevicePmConfigs(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*PmConfigs, error)
-	// Update the pm config of a device
-	UpdateDevicePmConfigs(ctx context.Context, in *PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error)
-	// List all flows of a device
-	ListDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error)
-	// List all flow groups of a device
-	ListDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error)
-	// List device types known to Voltha
-	ListDeviceTypes(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceTypes, error)
-	// Get additional information on a device type
-	GetDeviceType(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceType, error)
-	// List all device sharding groups
-	ListDeviceGroups(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceGroups, error)
-	// Stream control packets to the dataplane
-	StreamPacketsOut(ctx context.Context, opts ...grpc.CallOption) (VolthaService_StreamPacketsOutClient, error)
-	// Receive control packet stream
-	ReceivePacketsIn(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceivePacketsInClient, error)
-	ReceiveChangeEvents(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceiveChangeEventsClient, error)
-	// Get additional information on a device group
-	GetDeviceGroup(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceGroup, error)
-	CreateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error)
-	// Get all filters present for a device
-	GetEventFilter(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*EventFilters, error)
-	UpdateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error)
-	DeleteEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*empty.Empty, error)
-	// Get all the filters present
-	ListEventFilters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*EventFilters, error)
-	GetImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Images, error)
-	SelfTest(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*SelfTestResponse, error)
-	// OpenOMCI MIB information
-	GetMibDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.MibDeviceData, error)
-	// OpenOMCI ALARM information
-	GetAlarmDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.AlarmDeviceData, error)
-	// Simulate an Alarm
-	SimulateAlarm(ctx context.Context, in *SimulateAlarmRequest, opts ...grpc.CallOption) (*common.OperationResp, error)
-	Subscribe(ctx context.Context, in *OfAgentSubscriber, opts ...grpc.CallOption) (*OfAgentSubscriber, error)
-	EnablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error)
-	DisablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error)
-	GetExtValue(ctx context.Context, in *common.ValueSpecifier, opts ...grpc.CallOption) (*common.ReturnValues, error)
-	SetExtValue(ctx context.Context, in *ValueSet, opts ...grpc.CallOption) (*empty.Empty, error)
-	// omci start and stop cli implementation
-	StartOmciTestAction(ctx context.Context, in *OmciTestRequest, opts ...grpc.CallOption) (*TestResponse, error)
-}
-
-type volthaServiceClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewVolthaServiceClient(cc *grpc.ClientConn) VolthaServiceClient {
-	return &volthaServiceClient{cc}
-}
-
-func (c *volthaServiceClient) GetMembership(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Membership, error) {
-	out := new(Membership)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetMembership", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateMembership(ctx context.Context, in *Membership, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateMembership", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetVoltha(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Voltha, error) {
-	out := new(Voltha)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetVoltha", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListCoreInstances(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*CoreInstances, error) {
-	out := new(CoreInstances)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListCoreInstances", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetCoreInstance(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*CoreInstance, error) {
-	out := new(CoreInstance)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetCoreInstance", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListAdapters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Adapters, error) {
-	out := new(Adapters)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListAdapters", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListLogicalDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*LogicalDevices, error) {
-	out := new(LogicalDevices)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDevices", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetLogicalDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalDevice, error) {
-	out := new(LogicalDevice)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetLogicalDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListLogicalDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalPorts, error) {
-	out := new(LogicalPorts)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDevicePorts", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*LogicalPort, error) {
-	out := new(LogicalPort)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetLogicalDevicePort", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) EnableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnableLogicalDevicePort", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DisableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisableLogicalDevicePort", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListLogicalDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error) {
-	out := new(openflow_13.Flows)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceFlows", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateLogicalDeviceFlowTable(ctx context.Context, in *openflow_13.FlowTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceFlowTable", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateLogicalDeviceMeterTable(ctx context.Context, in *openflow_13.MeterModUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceMeterTable", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListLogicalDeviceMeters(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Meters, error) {
-	out := new(openflow_13.Meters)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceMeters", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListLogicalDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error) {
-	out := new(openflow_13.FlowGroups)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceFlowGroups", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateLogicalDeviceFlowGroupTable(ctx context.Context, in *openflow_13.FlowGroupTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceFlowGroupTable", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Devices, error) {
-	out := new(Devices)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevices", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDeviceIds(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*common.IDs, error) {
-	out := new(common.IDs)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceIds", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ReconcileDevices(ctx context.Context, in *common.IDs, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ReconcileDevices", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Device, error) {
-	out := new(Device)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) CreateDevice(ctx context.Context, in *Device, opts ...grpc.CallOption) (*Device, error) {
-	out := new(Device)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CreateDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) EnableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnableDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DisableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisableDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) RebootDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/RebootDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DeleteDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ForceDeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ForceDeleteDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DownloadImage(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
-	out := new(common.OperationResp)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DownloadImage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetImageDownloadStatus(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error) {
-	out := new(ImageDownload)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageDownloadStatus", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error) {
-	out := new(ImageDownload)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageDownload", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListImageDownloads(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*ImageDownloads, error) {
-	out := new(ImageDownloads)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListImageDownloads", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) CancelImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
-	out := new(common.OperationResp)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CancelImageDownload", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ActivateImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
-	out := new(common.OperationResp)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ActivateImageUpdate", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) RevertImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
-	out := new(common.OperationResp)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/RevertImageUpdate", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DownloadImageToDevice(ctx context.Context, in *DeviceImageDownloadRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
-	out := new(DeviceImageResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DownloadImageToDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetImageStatus(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
-	out := new(DeviceImageResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageStatus", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) AbortImageUpgradeToDevice(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
-	out := new(DeviceImageResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/AbortImageUpgradeToDevice", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*OnuImages, error) {
-	out := new(OnuImages)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetOnuImages", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ActivateImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
-	out := new(DeviceImageResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ActivateImage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) CommitImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
-	out := new(DeviceImageResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CommitImage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Ports, error) {
-	out := new(Ports)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevicePorts", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDevicePmConfigs(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*PmConfigs, error) {
-	out := new(PmConfigs)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevicePmConfigs", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateDevicePmConfigs(ctx context.Context, in *PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateDevicePmConfigs", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error) {
-	out := new(openflow_13.Flows)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceFlows", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error) {
-	out := new(openflow_13.FlowGroups)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceFlowGroups", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDeviceTypes(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceTypes, error) {
-	out := new(DeviceTypes)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceTypes", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetDeviceType(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceType, error) {
-	out := new(DeviceType)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDeviceType", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListDeviceGroups(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceGroups, error) {
-	out := new(DeviceGroups)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceGroups", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) StreamPacketsOut(ctx context.Context, opts ...grpc.CallOption) (VolthaService_StreamPacketsOutClient, error) {
-	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[0], "/voltha.VolthaService/StreamPacketsOut", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &volthaServiceStreamPacketsOutClient{stream}
-	return x, nil
-}
-
-type VolthaService_StreamPacketsOutClient interface {
-	Send(*openflow_13.PacketOut) error
-	CloseAndRecv() (*empty.Empty, error)
-	grpc.ClientStream
-}
-
-type volthaServiceStreamPacketsOutClient struct {
-	grpc.ClientStream
-}
-
-func (x *volthaServiceStreamPacketsOutClient) Send(m *openflow_13.PacketOut) error {
-	return x.ClientStream.SendMsg(m)
-}
-
-func (x *volthaServiceStreamPacketsOutClient) CloseAndRecv() (*empty.Empty, error) {
-	if err := x.ClientStream.CloseSend(); err != nil {
-		return nil, err
-	}
-	m := new(empty.Empty)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *volthaServiceClient) ReceivePacketsIn(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceivePacketsInClient, error) {
-	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[1], "/voltha.VolthaService/ReceivePacketsIn", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &volthaServiceReceivePacketsInClient{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 VolthaService_ReceivePacketsInClient interface {
-	Recv() (*openflow_13.PacketIn, error)
-	grpc.ClientStream
-}
-
-type volthaServiceReceivePacketsInClient struct {
-	grpc.ClientStream
-}
-
-func (x *volthaServiceReceivePacketsInClient) Recv() (*openflow_13.PacketIn, error) {
-	m := new(openflow_13.PacketIn)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *volthaServiceClient) ReceiveChangeEvents(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceiveChangeEventsClient, error) {
-	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[2], "/voltha.VolthaService/ReceiveChangeEvents", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &volthaServiceReceiveChangeEventsClient{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 VolthaService_ReceiveChangeEventsClient interface {
-	Recv() (*openflow_13.ChangeEvent, error)
-	grpc.ClientStream
-}
-
-type volthaServiceReceiveChangeEventsClient struct {
-	grpc.ClientStream
-}
-
-func (x *volthaServiceReceiveChangeEventsClient) Recv() (*openflow_13.ChangeEvent, error) {
-	m := new(openflow_13.ChangeEvent)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *volthaServiceClient) GetDeviceGroup(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceGroup, error) {
-	out := new(DeviceGroup)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDeviceGroup", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) CreateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error) {
-	out := new(EventFilter)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CreateEventFilter", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetEventFilter(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*EventFilters, error) {
-	out := new(EventFilters)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetEventFilter", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) UpdateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error) {
-	out := new(EventFilter)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateEventFilter", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DeleteEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DeleteEventFilter", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) ListEventFilters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*EventFilters, error) {
-	out := new(EventFilters)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListEventFilters", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Images, error) {
-	out := new(Images)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImages", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) SelfTest(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*SelfTestResponse, error) {
-	out := new(SelfTestResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SelfTest", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetMibDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.MibDeviceData, error) {
-	out := new(omci.MibDeviceData)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetMibDeviceData", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetAlarmDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.AlarmDeviceData, error) {
-	out := new(omci.AlarmDeviceData)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetAlarmDeviceData", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) SimulateAlarm(ctx context.Context, in *SimulateAlarmRequest, opts ...grpc.CallOption) (*common.OperationResp, error) {
-	out := new(common.OperationResp)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SimulateAlarm", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) Subscribe(ctx context.Context, in *OfAgentSubscriber, opts ...grpc.CallOption) (*OfAgentSubscriber, error) {
-	out := new(OfAgentSubscriber)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/Subscribe", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) EnablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnablePort", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) DisablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisablePort", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) GetExtValue(ctx context.Context, in *common.ValueSpecifier, opts ...grpc.CallOption) (*common.ReturnValues, error) {
-	out := new(common.ReturnValues)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetExtValue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) SetExtValue(ctx context.Context, in *ValueSet, opts ...grpc.CallOption) (*empty.Empty, error) {
-	out := new(empty.Empty)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SetExtValue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *volthaServiceClient) StartOmciTestAction(ctx context.Context, in *OmciTestRequest, opts ...grpc.CallOption) (*TestResponse, error) {
-	out := new(TestResponse)
-	err := c.cc.Invoke(ctx, "/voltha.VolthaService/StartOmciTestAction", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// VolthaServiceServer is the server API for VolthaService service.
-type VolthaServiceServer interface {
-	// Get the membership group of a Voltha Core
-	GetMembership(context.Context, *empty.Empty) (*Membership, error)
-	// Set the membership group of a Voltha Core
-	UpdateMembership(context.Context, *Membership) (*empty.Empty, error)
-	// Get high level information on the Voltha cluster
-	GetVoltha(context.Context, *empty.Empty) (*Voltha, error)
-	// List all Voltha cluster core instances
-	ListCoreInstances(context.Context, *empty.Empty) (*CoreInstances, error)
-	// Get details on a Voltha cluster instance
-	GetCoreInstance(context.Context, *common.ID) (*CoreInstance, error)
-	// List all active adapters (plugins) in the Voltha cluster
-	ListAdapters(context.Context, *empty.Empty) (*Adapters, error)
-	// List all logical devices managed by the Voltha cluster
-	ListLogicalDevices(context.Context, *empty.Empty) (*LogicalDevices, error)
-	// Get additional information on a given logical device
-	GetLogicalDevice(context.Context, *common.ID) (*LogicalDevice, error)
-	// List ports of a logical device
-	ListLogicalDevicePorts(context.Context, *common.ID) (*LogicalPorts, error)
-	// Gets a logical device port
-	GetLogicalDevicePort(context.Context, *LogicalPortId) (*LogicalPort, error)
-	// Enables a logical device port
-	EnableLogicalDevicePort(context.Context, *LogicalPortId) (*empty.Empty, error)
-	// Disables a logical device port
-	DisableLogicalDevicePort(context.Context, *LogicalPortId) (*empty.Empty, error)
-	// List all flows of a logical device
-	ListLogicalDeviceFlows(context.Context, *common.ID) (*openflow_13.Flows, error)
-	// Update flow table for logical device
-	UpdateLogicalDeviceFlowTable(context.Context, *openflow_13.FlowTableUpdate) (*empty.Empty, error)
-	// Update meter table for logical device
-	UpdateLogicalDeviceMeterTable(context.Context, *openflow_13.MeterModUpdate) (*empty.Empty, error)
-	// List all meters of a logical device
-	ListLogicalDeviceMeters(context.Context, *common.ID) (*openflow_13.Meters, error)
-	// List all flow groups of a logical device
-	ListLogicalDeviceFlowGroups(context.Context, *common.ID) (*openflow_13.FlowGroups, error)
-	// Update group table for device
-	UpdateLogicalDeviceFlowGroupTable(context.Context, *openflow_13.FlowGroupTableUpdate) (*empty.Empty, error)
-	// List all physical devices controlled by the Voltha cluster
-	ListDevices(context.Context, *empty.Empty) (*Devices, error)
-	// List all physical devices IDs controlled by the Voltha cluster
-	ListDeviceIds(context.Context, *empty.Empty) (*common.IDs, error)
-	// Request to a voltha Core to reconcile a set of devices based on their IDs
-	ReconcileDevices(context.Context, *common.IDs) (*empty.Empty, error)
-	// Get more information on a given physical device
-	GetDevice(context.Context, *common.ID) (*Device, error)
-	// Pre-provision a new physical device
-	CreateDevice(context.Context, *Device) (*Device, error)
-	// Enable a device.  If the device was in pre-provisioned state then it
-	// will transition to ENABLED state.  If it was is DISABLED state then it
-	// will transition to ENABLED state as well.
-	EnableDevice(context.Context, *common.ID) (*empty.Empty, error)
-	// Disable a device
-	DisableDevice(context.Context, *common.ID) (*empty.Empty, error)
-	// Reboot a device
-	RebootDevice(context.Context, *common.ID) (*empty.Empty, error)
-	// Delete a device
-	DeleteDevice(context.Context, *common.ID) (*empty.Empty, error)
-	// Forcefully delete a device
-	ForceDeleteDevice(context.Context, *common.ID) (*empty.Empty, error)
-	// Request an image download to the standby partition
-	// of a device.
-	// Note that the call is expected to be non-blocking.
-	DownloadImage(context.Context, *ImageDownload) (*common.OperationResp, error)
-	// Get image download status on a device
-	// The request retrieves progress on device and updates db record
-	// Deprecated in voltha 2.8, will be removed
-	GetImageDownloadStatus(context.Context, *ImageDownload) (*ImageDownload, error)
-	// Get image download db record
-	// Deprecated in voltha 2.8, will be removed
-	GetImageDownload(context.Context, *ImageDownload) (*ImageDownload, error)
-	// List image download db records for a given device
-	// Deprecated in voltha 2.8, will be removed
-	ListImageDownloads(context.Context, *common.ID) (*ImageDownloads, error)
-	// Cancel an existing image download process on a device
-	// Deprecated in voltha 2.8, will be removed
-	CancelImageDownload(context.Context, *ImageDownload) (*common.OperationResp, error)
-	// Activate the specified image at a standby partition
-	// to active partition.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device
-	// Note that the call is expected to be non-blocking.
-	// Deprecated in voltha 2.8, will be removed
-	ActivateImageUpdate(context.Context, *ImageDownload) (*common.OperationResp, error)
-	// Revert the specified image at standby partition
-	// to active partition, and revert to previous image
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// previous image running on device
-	// Note that the call is expected to be non-blocking.
-	// Deprecated in voltha 2.8, will be removed
-	RevertImageUpdate(context.Context, *ImageDownload) (*common.OperationResp, error)
-	// Downloads a certain image to the standby partition of the devices
-	// Note that the call is expected to be non-blocking.
-	DownloadImageToDevice(context.Context, *DeviceImageDownloadRequest) (*DeviceImageResponse, error)
-	// Get image status on a number of devices devices
-	// Polled from northbound systems to get state of download/activate/commit
-	GetImageStatus(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
-	// Aborts the upgrade of an image on a device
-	// To be used carefully, stops any further operations for the Image on the given devices
-	// Might also stop if possible existing work, but no guarantees are given,
-	// depends on implementation and procedure status.
-	AbortImageUpgradeToDevice(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
-	// Get Both Active and Standby image for a given device
-	GetOnuImages(context.Context, *common.ID) (*OnuImages, error)
-	// Activate the specified image from a standby partition
-	// to active partition.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device
-	// Note that the call is expected to be non-blocking.
-	ActivateImage(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
-	// Commit the specified image to be default.
-	// Depending on the device implementation, this call
-	// may or may not cause device reboot.
-	// If no reboot, then a reboot is required to make the
-	// activated image running on device upon next reboot
-	// Note that the call is expected to be non-blocking.
-	CommitImage(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
-	// List ports of a device
-	ListDevicePorts(context.Context, *common.ID) (*Ports, error)
-	// List pm config of a device
-	ListDevicePmConfigs(context.Context, *common.ID) (*PmConfigs, error)
-	// Update the pm config of a device
-	UpdateDevicePmConfigs(context.Context, *PmConfigs) (*empty.Empty, error)
-	// List all flows of a device
-	ListDeviceFlows(context.Context, *common.ID) (*openflow_13.Flows, error)
-	// List all flow groups of a device
-	ListDeviceFlowGroups(context.Context, *common.ID) (*openflow_13.FlowGroups, error)
-	// List device types known to Voltha
-	ListDeviceTypes(context.Context, *empty.Empty) (*DeviceTypes, error)
-	// Get additional information on a device type
-	GetDeviceType(context.Context, *common.ID) (*DeviceType, error)
-	// List all device sharding groups
-	ListDeviceGroups(context.Context, *empty.Empty) (*DeviceGroups, error)
-	// Stream control packets to the dataplane
-	StreamPacketsOut(VolthaService_StreamPacketsOutServer) error
-	// Receive control packet stream
-	ReceivePacketsIn(*empty.Empty, VolthaService_ReceivePacketsInServer) error
-	ReceiveChangeEvents(*empty.Empty, VolthaService_ReceiveChangeEventsServer) error
-	// Get additional information on a device group
-	GetDeviceGroup(context.Context, *common.ID) (*DeviceGroup, error)
-	CreateEventFilter(context.Context, *EventFilter) (*EventFilter, error)
-	// Get all filters present for a device
-	GetEventFilter(context.Context, *common.ID) (*EventFilters, error)
-	UpdateEventFilter(context.Context, *EventFilter) (*EventFilter, error)
-	DeleteEventFilter(context.Context, *EventFilter) (*empty.Empty, error)
-	// Get all the filters present
-	ListEventFilters(context.Context, *empty.Empty) (*EventFilters, error)
-	GetImages(context.Context, *common.ID) (*Images, error)
-	SelfTest(context.Context, *common.ID) (*SelfTestResponse, error)
-	// OpenOMCI MIB information
-	GetMibDeviceData(context.Context, *common.ID) (*omci.MibDeviceData, error)
-	// OpenOMCI ALARM information
-	GetAlarmDeviceData(context.Context, *common.ID) (*omci.AlarmDeviceData, error)
-	// Simulate an Alarm
-	SimulateAlarm(context.Context, *SimulateAlarmRequest) (*common.OperationResp, error)
-	Subscribe(context.Context, *OfAgentSubscriber) (*OfAgentSubscriber, error)
-	EnablePort(context.Context, *Port) (*empty.Empty, error)
-	DisablePort(context.Context, *Port) (*empty.Empty, error)
-	GetExtValue(context.Context, *common.ValueSpecifier) (*common.ReturnValues, error)
-	SetExtValue(context.Context, *ValueSet) (*empty.Empty, error)
-	// omci start and stop cli implementation
-	StartOmciTestAction(context.Context, *OmciTestRequest) (*TestResponse, error)
-}
-
-// UnimplementedVolthaServiceServer can be embedded to have forward compatible implementations.
-type UnimplementedVolthaServiceServer struct {
-}
-
-func (*UnimplementedVolthaServiceServer) GetMembership(ctx context.Context, req *empty.Empty) (*Membership, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetMembership not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateMembership(ctx context.Context, req *Membership) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateMembership not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetVoltha(ctx context.Context, req *empty.Empty) (*Voltha, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetVoltha not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListCoreInstances(ctx context.Context, req *empty.Empty) (*CoreInstances, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListCoreInstances not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetCoreInstance(ctx context.Context, req *common.ID) (*CoreInstance, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetCoreInstance not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListAdapters(ctx context.Context, req *empty.Empty) (*Adapters, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListAdapters not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListLogicalDevices(ctx context.Context, req *empty.Empty) (*LogicalDevices, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDevices not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetLogicalDevice(ctx context.Context, req *common.ID) (*LogicalDevice, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListLogicalDevicePorts(ctx context.Context, req *common.ID) (*LogicalPorts, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDevicePorts not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*LogicalPort, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalDevicePort not implemented")
-}
-func (*UnimplementedVolthaServiceServer) EnableLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method EnableLogicalDevicePort not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DisableLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DisableLogicalDevicePort not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListLogicalDeviceFlows(ctx context.Context, req *common.ID) (*openflow_13.Flows, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceFlows not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceFlowTable(ctx context.Context, req *openflow_13.FlowTableUpdate) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceFlowTable not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceMeterTable(ctx context.Context, req *openflow_13.MeterModUpdate) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceMeterTable not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListLogicalDeviceMeters(ctx context.Context, req *common.ID) (*openflow_13.Meters, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceMeters not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListLogicalDeviceFlowGroups(ctx context.Context, req *common.ID) (*openflow_13.FlowGroups, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceFlowGroups not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceFlowGroupTable(ctx context.Context, req *openflow_13.FlowGroupTableUpdate) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceFlowGroupTable not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDevices(ctx context.Context, req *empty.Empty) (*Devices, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDevices not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDeviceIds(ctx context.Context, req *empty.Empty) (*common.IDs, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceIds not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ReconcileDevices(ctx context.Context, req *common.IDs) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ReconcileDevices not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetDevice(ctx context.Context, req *common.ID) (*Device, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) CreateDevice(ctx context.Context, req *Device) (*Device, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) EnableDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method EnableDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DisableDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DisableDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) RebootDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method RebootDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DeleteDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DeleteDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ForceDeleteDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ForceDeleteDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DownloadImage(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DownloadImage not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetImageDownloadStatus(ctx context.Context, req *ImageDownload) (*ImageDownload, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetImageDownloadStatus not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetImageDownload(ctx context.Context, req *ImageDownload) (*ImageDownload, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetImageDownload not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListImageDownloads(ctx context.Context, req *common.ID) (*ImageDownloads, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListImageDownloads not implemented")
-}
-func (*UnimplementedVolthaServiceServer) CancelImageDownload(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CancelImageDownload not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ActivateImageUpdate(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ActivateImageUpdate not implemented")
-}
-func (*UnimplementedVolthaServiceServer) RevertImageUpdate(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method RevertImageUpdate not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DownloadImageToDevice(ctx context.Context, req *DeviceImageDownloadRequest) (*DeviceImageResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DownloadImageToDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetImageStatus(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetImageStatus not implemented")
-}
-func (*UnimplementedVolthaServiceServer) AbortImageUpgradeToDevice(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method AbortImageUpgradeToDevice not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetOnuImages(ctx context.Context, req *common.ID) (*OnuImages, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetOnuImages not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ActivateImage(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ActivateImage not implemented")
-}
-func (*UnimplementedVolthaServiceServer) CommitImage(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CommitImage not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDevicePorts(ctx context.Context, req *common.ID) (*Ports, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDevicePorts not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDevicePmConfigs(ctx context.Context, req *common.ID) (*PmConfigs, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDevicePmConfigs not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateDevicePmConfigs(ctx context.Context, req *PmConfigs) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateDevicePmConfigs not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDeviceFlows(ctx context.Context, req *common.ID) (*openflow_13.Flows, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceFlows not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDeviceFlowGroups(ctx context.Context, req *common.ID) (*openflow_13.FlowGroups, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceFlowGroups not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDeviceTypes(ctx context.Context, req *empty.Empty) (*DeviceTypes, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceTypes not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetDeviceType(ctx context.Context, req *common.ID) (*DeviceType, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceType not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListDeviceGroups(ctx context.Context, req *empty.Empty) (*DeviceGroups, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceGroups not implemented")
-}
-func (*UnimplementedVolthaServiceServer) StreamPacketsOut(srv VolthaService_StreamPacketsOutServer) error {
-	return status.Errorf(codes.Unimplemented, "method StreamPacketsOut not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ReceivePacketsIn(req *empty.Empty, srv VolthaService_ReceivePacketsInServer) error {
-	return status.Errorf(codes.Unimplemented, "method ReceivePacketsIn not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ReceiveChangeEvents(req *empty.Empty, srv VolthaService_ReceiveChangeEventsServer) error {
-	return status.Errorf(codes.Unimplemented, "method ReceiveChangeEvents not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetDeviceGroup(ctx context.Context, req *common.ID) (*DeviceGroup, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceGroup not implemented")
-}
-func (*UnimplementedVolthaServiceServer) CreateEventFilter(ctx context.Context, req *EventFilter) (*EventFilter, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateEventFilter not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetEventFilter(ctx context.Context, req *common.ID) (*EventFilters, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetEventFilter not implemented")
-}
-func (*UnimplementedVolthaServiceServer) UpdateEventFilter(ctx context.Context, req *EventFilter) (*EventFilter, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method UpdateEventFilter not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DeleteEventFilter(ctx context.Context, req *EventFilter) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DeleteEventFilter not implemented")
-}
-func (*UnimplementedVolthaServiceServer) ListEventFilters(ctx context.Context, req *empty.Empty) (*EventFilters, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ListEventFilters not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetImages(ctx context.Context, req *common.ID) (*Images, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetImages not implemented")
-}
-func (*UnimplementedVolthaServiceServer) SelfTest(ctx context.Context, req *common.ID) (*SelfTestResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method SelfTest not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetMibDeviceData(ctx context.Context, req *common.ID) (*omci.MibDeviceData, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetMibDeviceData not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetAlarmDeviceData(ctx context.Context, req *common.ID) (*omci.AlarmDeviceData, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetAlarmDeviceData not implemented")
-}
-func (*UnimplementedVolthaServiceServer) SimulateAlarm(ctx context.Context, req *SimulateAlarmRequest) (*common.OperationResp, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method SimulateAlarm not implemented")
-}
-func (*UnimplementedVolthaServiceServer) Subscribe(ctx context.Context, req *OfAgentSubscriber) (*OfAgentSubscriber, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
-}
-func (*UnimplementedVolthaServiceServer) EnablePort(ctx context.Context, req *Port) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method EnablePort not implemented")
-}
-func (*UnimplementedVolthaServiceServer) DisablePort(ctx context.Context, req *Port) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DisablePort not implemented")
-}
-func (*UnimplementedVolthaServiceServer) GetExtValue(ctx context.Context, req *common.ValueSpecifier) (*common.ReturnValues, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
-}
-func (*UnimplementedVolthaServiceServer) SetExtValue(ctx context.Context, req *ValueSet) (*empty.Empty, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method SetExtValue not implemented")
-}
-func (*UnimplementedVolthaServiceServer) StartOmciTestAction(ctx context.Context, req *OmciTestRequest) (*TestResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method StartOmciTestAction not implemented")
-}
-
-func RegisterVolthaServiceServer(s *grpc.Server, srv VolthaServiceServer) {
-	s.RegisterService(&_VolthaService_serviceDesc, srv)
-}
-
-func _VolthaService_GetMembership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetMembership(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetMembership",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetMembership(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateMembership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Membership)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateMembership(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateMembership",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateMembership(ctx, req.(*Membership))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetVoltha_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetVoltha(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetVoltha",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetVoltha(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListCoreInstances_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListCoreInstances(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListCoreInstances",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListCoreInstances(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetCoreInstance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetCoreInstance(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetCoreInstance",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetCoreInstance(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListAdapters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListAdapters(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListAdapters",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListAdapters(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListLogicalDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListLogicalDevices(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListLogicalDevices",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListLogicalDevices(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetLogicalDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetLogicalDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetLogicalDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetLogicalDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListLogicalDevicePorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListLogicalDevicePorts(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListLogicalDevicePorts",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListLogicalDevicePorts(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LogicalPortId)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetLogicalDevicePort(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetLogicalDevicePort",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetLogicalDevicePort(ctx, req.(*LogicalPortId))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_EnableLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LogicalPortId)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).EnableLogicalDevicePort(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/EnableLogicalDevicePort",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).EnableLogicalDevicePort(ctx, req.(*LogicalPortId))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DisableLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LogicalPortId)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DisableLogicalDevicePort(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DisableLogicalDevicePort",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DisableLogicalDevicePort(ctx, req.(*LogicalPortId))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListLogicalDeviceFlows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListLogicalDeviceFlows(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListLogicalDeviceFlows",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListLogicalDeviceFlows(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateLogicalDeviceFlowTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(openflow_13.FlowTableUpdate)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowTable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceFlowTable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowTable(ctx, req.(*openflow_13.FlowTableUpdate))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateLogicalDeviceMeterTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(openflow_13.MeterModUpdate)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceMeterTable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceMeterTable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceMeterTable(ctx, req.(*openflow_13.MeterModUpdate))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListLogicalDeviceMeters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListLogicalDeviceMeters(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListLogicalDeviceMeters",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListLogicalDeviceMeters(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListLogicalDeviceFlowGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListLogicalDeviceFlowGroups(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListLogicalDeviceFlowGroups",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListLogicalDeviceFlowGroups(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateLogicalDeviceFlowGroupTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(openflow_13.FlowGroupTableUpdate)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowGroupTable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceFlowGroupTable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowGroupTable(ctx, req.(*openflow_13.FlowGroupTableUpdate))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDevices(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDevices",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDevices(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDeviceIds_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDeviceIds(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDeviceIds",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDeviceIds(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ReconcileDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.IDs)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ReconcileDevices(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ReconcileDevices",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ReconcileDevices(ctx, req.(*common.IDs))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_CreateDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Device)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).CreateDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/CreateDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).CreateDevice(ctx, req.(*Device))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_EnableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).EnableDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/EnableDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).EnableDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DisableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DisableDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DisableDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DisableDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_RebootDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).RebootDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/RebootDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).RebootDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DeleteDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DeleteDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DeleteDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DeleteDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ForceDeleteDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ForceDeleteDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ForceDeleteDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ForceDeleteDevice(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DownloadImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DownloadImage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DownloadImage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DownloadImage(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetImageDownloadStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetImageDownloadStatus(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetImageDownloadStatus",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetImageDownloadStatus(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetImageDownload(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetImageDownload",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetImageDownload(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListImageDownloads_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListImageDownloads(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListImageDownloads",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListImageDownloads(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_CancelImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).CancelImageDownload(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/CancelImageDownload",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).CancelImageDownload(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ActivateImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ActivateImageUpdate(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ActivateImageUpdate",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ActivateImageUpdate(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_RevertImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ImageDownload)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).RevertImageUpdate(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/RevertImageUpdate",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).RevertImageUpdate(ctx, req.(*ImageDownload))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DownloadImageToDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeviceImageDownloadRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DownloadImageToDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DownloadImageToDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DownloadImageToDevice(ctx, req.(*DeviceImageDownloadRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetImageStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeviceImageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetImageStatus(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetImageStatus",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetImageStatus(ctx, req.(*DeviceImageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_AbortImageUpgradeToDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeviceImageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).AbortImageUpgradeToDevice(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/AbortImageUpgradeToDevice",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).AbortImageUpgradeToDevice(ctx, req.(*DeviceImageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetOnuImages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetOnuImages(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetOnuImages",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetOnuImages(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ActivateImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeviceImageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ActivateImage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ActivateImage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ActivateImage(ctx, req.(*DeviceImageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_CommitImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeviceImageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).CommitImage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/CommitImage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).CommitImage(ctx, req.(*DeviceImageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDevicePorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDevicePorts(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDevicePorts",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDevicePorts(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDevicePmConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDevicePmConfigs(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDevicePmConfigs",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDevicePmConfigs(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateDevicePmConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PmConfigs)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateDevicePmConfigs(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateDevicePmConfigs",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateDevicePmConfigs(ctx, req.(*PmConfigs))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDeviceFlows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDeviceFlows(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDeviceFlows",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDeviceFlows(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDeviceFlowGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDeviceFlowGroups(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDeviceFlowGroups",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDeviceFlowGroups(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDeviceTypes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDeviceTypes(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDeviceTypes",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDeviceTypes(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetDeviceType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetDeviceType(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetDeviceType",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetDeviceType(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListDeviceGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListDeviceGroups(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListDeviceGroups",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListDeviceGroups(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_StreamPacketsOut_Handler(srv interface{}, stream grpc.ServerStream) error {
-	return srv.(VolthaServiceServer).StreamPacketsOut(&volthaServiceStreamPacketsOutServer{stream})
-}
-
-type VolthaService_StreamPacketsOutServer interface {
-	SendAndClose(*empty.Empty) error
-	Recv() (*openflow_13.PacketOut, error)
-	grpc.ServerStream
-}
-
-type volthaServiceStreamPacketsOutServer struct {
-	grpc.ServerStream
-}
-
-func (x *volthaServiceStreamPacketsOutServer) SendAndClose(m *empty.Empty) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func (x *volthaServiceStreamPacketsOutServer) Recv() (*openflow_13.PacketOut, error) {
-	m := new(openflow_13.PacketOut)
-	if err := x.ServerStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func _VolthaService_ReceivePacketsIn_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(empty.Empty)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(VolthaServiceServer).ReceivePacketsIn(m, &volthaServiceReceivePacketsInServer{stream})
-}
-
-type VolthaService_ReceivePacketsInServer interface {
-	Send(*openflow_13.PacketIn) error
-	grpc.ServerStream
-}
-
-type volthaServiceReceivePacketsInServer struct {
-	grpc.ServerStream
-}
-
-func (x *volthaServiceReceivePacketsInServer) Send(m *openflow_13.PacketIn) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func _VolthaService_ReceiveChangeEvents_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(empty.Empty)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(VolthaServiceServer).ReceiveChangeEvents(m, &volthaServiceReceiveChangeEventsServer{stream})
-}
-
-type VolthaService_ReceiveChangeEventsServer interface {
-	Send(*openflow_13.ChangeEvent) error
-	grpc.ServerStream
-}
-
-type volthaServiceReceiveChangeEventsServer struct {
-	grpc.ServerStream
-}
-
-func (x *volthaServiceReceiveChangeEventsServer) Send(m *openflow_13.ChangeEvent) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func _VolthaService_GetDeviceGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetDeviceGroup(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetDeviceGroup",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetDeviceGroup(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_CreateEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(EventFilter)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).CreateEventFilter(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/CreateEventFilter",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).CreateEventFilter(ctx, req.(*EventFilter))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetEventFilter(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetEventFilter",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetEventFilter(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_UpdateEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(EventFilter)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).UpdateEventFilter(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/UpdateEventFilter",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).UpdateEventFilter(ctx, req.(*EventFilter))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DeleteEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(EventFilter)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DeleteEventFilter(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DeleteEventFilter",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DeleteEventFilter(ctx, req.(*EventFilter))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_ListEventFilters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(empty.Empty)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).ListEventFilters(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/ListEventFilters",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).ListEventFilters(ctx, req.(*empty.Empty))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetImages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetImages(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetImages",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetImages(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_SelfTest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).SelfTest(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/SelfTest",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).SelfTest(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetMibDeviceData_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetMibDeviceData(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetMibDeviceData",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetMibDeviceData(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetAlarmDeviceData_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ID)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetAlarmDeviceData(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetAlarmDeviceData",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetAlarmDeviceData(ctx, req.(*common.ID))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_SimulateAlarm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(SimulateAlarmRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).SimulateAlarm(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/SimulateAlarm",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).SimulateAlarm(ctx, req.(*SimulateAlarmRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_Subscribe_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(OfAgentSubscriber)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).Subscribe(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/Subscribe",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).Subscribe(ctx, req.(*OfAgentSubscriber))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_EnablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Port)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).EnablePort(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/EnablePort",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).EnablePort(ctx, req.(*Port))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_DisablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(Port)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).DisablePort(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/DisablePort",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).DisablePort(ctx, req.(*Port))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(common.ValueSpecifier)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).GetExtValue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/GetExtValue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).GetExtValue(ctx, req.(*common.ValueSpecifier))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_SetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ValueSet)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).SetExtValue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/SetExtValue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).SetExtValue(ctx, req.(*ValueSet))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _VolthaService_StartOmciTestAction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(OmciTestRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(VolthaServiceServer).StartOmciTestAction(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/voltha.VolthaService/StartOmciTestAction",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(VolthaServiceServer).StartOmciTestAction(ctx, req.(*OmciTestRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _VolthaService_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "voltha.VolthaService",
-	HandlerType: (*VolthaServiceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "GetMembership",
-			Handler:    _VolthaService_GetMembership_Handler,
-		},
-		{
-			MethodName: "UpdateMembership",
-			Handler:    _VolthaService_UpdateMembership_Handler,
-		},
-		{
-			MethodName: "GetVoltha",
-			Handler:    _VolthaService_GetVoltha_Handler,
-		},
-		{
-			MethodName: "ListCoreInstances",
-			Handler:    _VolthaService_ListCoreInstances_Handler,
-		},
-		{
-			MethodName: "GetCoreInstance",
-			Handler:    _VolthaService_GetCoreInstance_Handler,
-		},
-		{
-			MethodName: "ListAdapters",
-			Handler:    _VolthaService_ListAdapters_Handler,
-		},
-		{
-			MethodName: "ListLogicalDevices",
-			Handler:    _VolthaService_ListLogicalDevices_Handler,
-		},
-		{
-			MethodName: "GetLogicalDevice",
-			Handler:    _VolthaService_GetLogicalDevice_Handler,
-		},
-		{
-			MethodName: "ListLogicalDevicePorts",
-			Handler:    _VolthaService_ListLogicalDevicePorts_Handler,
-		},
-		{
-			MethodName: "GetLogicalDevicePort",
-			Handler:    _VolthaService_GetLogicalDevicePort_Handler,
-		},
-		{
-			MethodName: "EnableLogicalDevicePort",
-			Handler:    _VolthaService_EnableLogicalDevicePort_Handler,
-		},
-		{
-			MethodName: "DisableLogicalDevicePort",
-			Handler:    _VolthaService_DisableLogicalDevicePort_Handler,
-		},
-		{
-			MethodName: "ListLogicalDeviceFlows",
-			Handler:    _VolthaService_ListLogicalDeviceFlows_Handler,
-		},
-		{
-			MethodName: "UpdateLogicalDeviceFlowTable",
-			Handler:    _VolthaService_UpdateLogicalDeviceFlowTable_Handler,
-		},
-		{
-			MethodName: "UpdateLogicalDeviceMeterTable",
-			Handler:    _VolthaService_UpdateLogicalDeviceMeterTable_Handler,
-		},
-		{
-			MethodName: "ListLogicalDeviceMeters",
-			Handler:    _VolthaService_ListLogicalDeviceMeters_Handler,
-		},
-		{
-			MethodName: "ListLogicalDeviceFlowGroups",
-			Handler:    _VolthaService_ListLogicalDeviceFlowGroups_Handler,
-		},
-		{
-			MethodName: "UpdateLogicalDeviceFlowGroupTable",
-			Handler:    _VolthaService_UpdateLogicalDeviceFlowGroupTable_Handler,
-		},
-		{
-			MethodName: "ListDevices",
-			Handler:    _VolthaService_ListDevices_Handler,
-		},
-		{
-			MethodName: "ListDeviceIds",
-			Handler:    _VolthaService_ListDeviceIds_Handler,
-		},
-		{
-			MethodName: "ReconcileDevices",
-			Handler:    _VolthaService_ReconcileDevices_Handler,
-		},
-		{
-			MethodName: "GetDevice",
-			Handler:    _VolthaService_GetDevice_Handler,
-		},
-		{
-			MethodName: "CreateDevice",
-			Handler:    _VolthaService_CreateDevice_Handler,
-		},
-		{
-			MethodName: "EnableDevice",
-			Handler:    _VolthaService_EnableDevice_Handler,
-		},
-		{
-			MethodName: "DisableDevice",
-			Handler:    _VolthaService_DisableDevice_Handler,
-		},
-		{
-			MethodName: "RebootDevice",
-			Handler:    _VolthaService_RebootDevice_Handler,
-		},
-		{
-			MethodName: "DeleteDevice",
-			Handler:    _VolthaService_DeleteDevice_Handler,
-		},
-		{
-			MethodName: "ForceDeleteDevice",
-			Handler:    _VolthaService_ForceDeleteDevice_Handler,
-		},
-		{
-			MethodName: "DownloadImage",
-			Handler:    _VolthaService_DownloadImage_Handler,
-		},
-		{
-			MethodName: "GetImageDownloadStatus",
-			Handler:    _VolthaService_GetImageDownloadStatus_Handler,
-		},
-		{
-			MethodName: "GetImageDownload",
-			Handler:    _VolthaService_GetImageDownload_Handler,
-		},
-		{
-			MethodName: "ListImageDownloads",
-			Handler:    _VolthaService_ListImageDownloads_Handler,
-		},
-		{
-			MethodName: "CancelImageDownload",
-			Handler:    _VolthaService_CancelImageDownload_Handler,
-		},
-		{
-			MethodName: "ActivateImageUpdate",
-			Handler:    _VolthaService_ActivateImageUpdate_Handler,
-		},
-		{
-			MethodName: "RevertImageUpdate",
-			Handler:    _VolthaService_RevertImageUpdate_Handler,
-		},
-		{
-			MethodName: "DownloadImageToDevice",
-			Handler:    _VolthaService_DownloadImageToDevice_Handler,
-		},
-		{
-			MethodName: "GetImageStatus",
-			Handler:    _VolthaService_GetImageStatus_Handler,
-		},
-		{
-			MethodName: "AbortImageUpgradeToDevice",
-			Handler:    _VolthaService_AbortImageUpgradeToDevice_Handler,
-		},
-		{
-			MethodName: "GetOnuImages",
-			Handler:    _VolthaService_GetOnuImages_Handler,
-		},
-		{
-			MethodName: "ActivateImage",
-			Handler:    _VolthaService_ActivateImage_Handler,
-		},
-		{
-			MethodName: "CommitImage",
-			Handler:    _VolthaService_CommitImage_Handler,
-		},
-		{
-			MethodName: "ListDevicePorts",
-			Handler:    _VolthaService_ListDevicePorts_Handler,
-		},
-		{
-			MethodName: "ListDevicePmConfigs",
-			Handler:    _VolthaService_ListDevicePmConfigs_Handler,
-		},
-		{
-			MethodName: "UpdateDevicePmConfigs",
-			Handler:    _VolthaService_UpdateDevicePmConfigs_Handler,
-		},
-		{
-			MethodName: "ListDeviceFlows",
-			Handler:    _VolthaService_ListDeviceFlows_Handler,
-		},
-		{
-			MethodName: "ListDeviceFlowGroups",
-			Handler:    _VolthaService_ListDeviceFlowGroups_Handler,
-		},
-		{
-			MethodName: "ListDeviceTypes",
-			Handler:    _VolthaService_ListDeviceTypes_Handler,
-		},
-		{
-			MethodName: "GetDeviceType",
-			Handler:    _VolthaService_GetDeviceType_Handler,
-		},
-		{
-			MethodName: "ListDeviceGroups",
-			Handler:    _VolthaService_ListDeviceGroups_Handler,
-		},
-		{
-			MethodName: "GetDeviceGroup",
-			Handler:    _VolthaService_GetDeviceGroup_Handler,
-		},
-		{
-			MethodName: "CreateEventFilter",
-			Handler:    _VolthaService_CreateEventFilter_Handler,
-		},
-		{
-			MethodName: "GetEventFilter",
-			Handler:    _VolthaService_GetEventFilter_Handler,
-		},
-		{
-			MethodName: "UpdateEventFilter",
-			Handler:    _VolthaService_UpdateEventFilter_Handler,
-		},
-		{
-			MethodName: "DeleteEventFilter",
-			Handler:    _VolthaService_DeleteEventFilter_Handler,
-		},
-		{
-			MethodName: "ListEventFilters",
-			Handler:    _VolthaService_ListEventFilters_Handler,
-		},
-		{
-			MethodName: "GetImages",
-			Handler:    _VolthaService_GetImages_Handler,
-		},
-		{
-			MethodName: "SelfTest",
-			Handler:    _VolthaService_SelfTest_Handler,
-		},
-		{
-			MethodName: "GetMibDeviceData",
-			Handler:    _VolthaService_GetMibDeviceData_Handler,
-		},
-		{
-			MethodName: "GetAlarmDeviceData",
-			Handler:    _VolthaService_GetAlarmDeviceData_Handler,
-		},
-		{
-			MethodName: "SimulateAlarm",
-			Handler:    _VolthaService_SimulateAlarm_Handler,
-		},
-		{
-			MethodName: "Subscribe",
-			Handler:    _VolthaService_Subscribe_Handler,
-		},
-		{
-			MethodName: "EnablePort",
-			Handler:    _VolthaService_EnablePort_Handler,
-		},
-		{
-			MethodName: "DisablePort",
-			Handler:    _VolthaService_DisablePort_Handler,
-		},
-		{
-			MethodName: "GetExtValue",
-			Handler:    _VolthaService_GetExtValue_Handler,
-		},
-		{
-			MethodName: "SetExtValue",
-			Handler:    _VolthaService_SetExtValue_Handler,
-		},
-		{
-			MethodName: "StartOmciTestAction",
-			Handler:    _VolthaService_StartOmciTestAction_Handler,
-		},
-	},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "StreamPacketsOut",
-			Handler:       _VolthaService_StreamPacketsOut_Handler,
-			ClientStreams: true,
-		},
-		{
-			StreamName:    "ReceivePacketsIn",
-			Handler:       _VolthaService_ReceivePacketsIn_Handler,
-			ServerStreams: true,
-		},
-		{
-			StreamName:    "ReceiveChangeEvents",
-			Handler:       _VolthaService_ReceiveChangeEvents_Handler,
-			ServerStreams: true,
-		},
-	},
-	Metadata: "voltha_protos/voltha.proto",
-}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/adapter_services/adapter_services.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/adapter_services/adapter_services.pb.go
new file mode 100644
index 0000000..3f2ca0c
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/adapter_services/adapter_services.pb.go
@@ -0,0 +1,1832 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/adapter_services.proto
+
+package adapter_services
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	empty "github.com/golang/protobuf/ptypes/empty"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	extension "github.com/opencord/voltha-protos/v5/go/extension"
+	inter_container "github.com/opencord/voltha-protos/v5/go/inter_container"
+	_ "github.com/opencord/voltha-protos/v5/go/openolt"
+	voltha "github.com/opencord/voltha-protos/v5/go/voltha"
+	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
+
+func init() {
+	proto.RegisterFile("voltha_protos/adapter_services.proto", fileDescriptor_5826bd5d7bb77df1)
+}
+
+var fileDescriptor_5826bd5d7bb77df1 = []byte{
+	// 965 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x97, 0x4b, 0x6f, 0xe3, 0x36,
+	0x10, 0xc7, 0x81, 0x16, 0xd8, 0xc3, 0x24, 0x76, 0x12, 0xe5, 0x81, 0x85, 0xb2, 0x2d, 0xfa, 0xba,
+	0xf4, 0xb0, 0x0a, 0x90, 0xc5, 0x3e, 0x5a, 0xa4, 0x0d, 0x1c, 0xdb, 0xeb, 0x75, 0x9b, 0xc0, 0xa9,
+	0x95, 0xec, 0xa1, 0x97, 0x05, 0x2d, 0x8d, 0x65, 0x62, 0x29, 0x52, 0x15, 0x47, 0x79, 0x9c, 0xfb,
+	0x75, 0xfa, 0x11, 0xfa, 0xe1, 0x0a, 0xea, 0x65, 0x29, 0xb1, 0xd7, 0xb1, 0xd3, 0x3d, 0xf2, 0x3f,
+	0x33, 0x3f, 0x52, 0x33, 0x43, 0x8a, 0x84, 0x1f, 0xae, 0x94, 0xa0, 0x09, 0xfb, 0x10, 0xc5, 0x8a,
+	0x94, 0x3e, 0x60, 0x3e, 0x8b, 0x08, 0xe3, 0x0f, 0x1a, 0xe3, 0x2b, 0xee, 0xa1, 0x76, 0x52, 0xdd,
+	0x7a, 0x92, 0x79, 0xd9, 0xfb, 0x81, 0x52, 0x81, 0xc0, 0x83, 0x54, 0x1d, 0x25, 0xe3, 0x03, 0x0c,
+	0x23, 0xba, 0xcd, 0x9c, 0x6c, 0xbb, 0x8e, 0xf2, 0x54, 0x18, 0x2a, 0x99, 0xdb, 0xbe, 0xaf, 0xdb,
+	0xb8, 0x34, 0x93, 0x78, 0x4a, 0x12, 0xe3, 0x12, 0xe3, 0xd9, 0x00, 0x1f, 0xcd, 0x12, 0x72, 0xdb,
+	0xd7, 0x75, 0x1b, 0xde, 0x10, 0x4a, 0xcd, 0x95, 0xcc, 0x57, 0x68, 0xef, 0xd7, 0xed, 0x2a, 0x42,
+	0xa9, 0x04, 0xcd, 0x06, 0x4f, 0x90, 0x09, 0x9a, 0xcc, 0xb6, 0x65, 0xa3, 0xcc, 0x76, 0xf8, 0x8f,
+	0x05, 0xcd, 0x56, 0x96, 0x11, 0x37, 0x4b, 0x88, 0x75, 0x0c, 0x1b, 0x3d, 0xa4, 0x77, 0x29, 0xc1,
+	0x25, 0x46, 0x89, 0xb6, 0xf6, 0x9c, 0x2c, 0x2b, 0x4e, 0x91, 0x15, 0xa7, 0x6b, 0xb2, 0x62, 0xef,
+	0x38, 0x39, 0xac, 0xe6, 0xfd, 0x12, 0xd6, 0x5a, 0xbe, 0x8a, 0xa8, 0x93, 0x7e, 0x9d, 0xd5, 0x2c,
+	0x9c, 0xb2, 0xb1, 0x3d, 0x07, 0x66, 0xfd, 0x04, 0x1b, 0x43, 0xf4, 0x94, 0xf4, 0xb8, 0xc0, 0x25,
+	0x43, 0x5f, 0xc1, 0x7a, 0x07, 0x05, 0xd2, 0xb2, 0x71, 0xaf, 0xa1, 0xd1, 0xe1, 0x9a, 0x8d, 0x96,
+	0x9e, 0xf0, 0x0d, 0x34, 0x87, 0xd8, 0x95, 0x2b, 0x44, 0xbe, 0x82, 0xf5, 0x21, 0x8e, 0x94, 0xa2,
+	0xe5, 0x67, 0x74, 0x51, 0x8c, 0x2f, 0x50, 0x2f, 0x1b, 0x79, 0x04, 0x9b, 0x3d, 0xa4, 0xc1, 0x38,
+	0xca, 0xfc, 0xfa, 0x72, 0xac, 0xee, 0xc5, 0x3e, 0x2d, 0xc6, 0xee, 0x35, 0x27, 0x6f, 0xd2, 0x66,
+	0x11, 0x1b, 0x71, 0xc1, 0xb3, 0xaa, 0xb4, 0x27, 0x5c, 0xf8, 0x99, 0xe3, 0xa9, 0xd2, 0xf4, 0xe0,
+	0x89, 0x0f, 0x01, 0xb2, 0x14, 0x9d, 0xab, 0x98, 0xac, 0xf5, 0x22, 0xca, 0x8c, 0xe6, 0xc6, 0xbc,
+	0x80, 0xb5, 0xbc, 0x22, 0x4b, 0x04, 0x1d, 0xc1, 0xc6, 0x65, 0xe4, 0x33, 0xc2, 0xb7, 0x42, 0x5d,
+	0xeb, 0x93, 0x44, 0x7c, 0xb4, 0xb6, 0x8a, 0x40, 0x33, 0x4a, 0xe5, 0xb9, 0xd1, 0xa7, 0xf0, 0xb4,
+	0x12, 0xdd, 0x97, 0x5e, 0x8c, 0x21, 0x4a, 0x62, 0x42, 0xdc, 0x5a, 0x65, 0x5e, 0x2a, 0xf2, 0xa7,
+	0x69, 0x3f, 0x43, 0xc3, 0x45, 0xe9, 0x9f, 0x33, 0xef, 0x23, 0xd2, 0x20, 0xa1, 0xe9, 0x4a, 0x4a,
+	0x69, 0x6e, 0xec, 0x31, 0x34, 0xb3, 0x95, 0x9c, 0x87, 0x6d, 0x25, 0xc7, 0x3c, 0xb0, 0x76, 0xcb,
+	0xe0, 0x5c, 0xd1, 0xa6, 0x7c, 0x73, 0x01, 0x2e, 0x6c, 0x76, 0xd4, 0xb5, 0x14, 0x8a, 0xf9, 0x03,
+	0x99, 0xf4, 0x43, 0x16, 0xa0, 0xf5, 0x5d, 0xbd, 0x5a, 0xa9, 0x58, 0x38, 0x0d, 0xf1, 0xaf, 0x04,
+	0x35, 0xd9, 0xfb, 0x33, 0x7c, 0x86, 0xa8, 0x23, 0x25, 0x35, 0x5a, 0xa7, 0xb0, 0x65, 0xfa, 0x27,
+	0xe7, 0xe5, 0x7b, 0xdc, 0x9e, 0x19, 0xf1, 0x00, 0xda, 0x00, 0x76, 0x5a, 0x23, 0x15, 0x97, 0xbc,
+	0xcb, 0x28, 0x88, 0x99, 0x8f, 0xab, 0x03, 0x9f, 0xc3, 0x7a, 0x65, 0x79, 0xda, 0x02, 0x27, 0x3f,
+	0x96, 0xfb, 0x1d, 0xbb, 0xcc, 0xfd, 0xd4, 0xfc, 0x3b, 0x6c, 0xb6, 0x3c, 0xe2, 0x57, 0x8c, 0xb0,
+	0x4c, 0xd1, 0xca, 0x73, 0xf7, 0xa1, 0xd9, 0x56, 0x61, 0xc8, 0xe9, 0xf1, 0xa8, 0x0e, 0x34, 0x8a,
+	0xaa, 0x64, 0xa4, 0x67, 0x65, 0xeb, 0x55, 0x2b, 0x76, 0x86, 0x5a, 0xb3, 0x00, 0xed, 0xdd, 0x99,
+	0x56, 0xeb, 0x0c, 0xf6, 0x7a, 0x48, 0x35, 0x2d, 0x2f, 0xd8, 0x4a, 0xb8, 0xdf, 0x60, 0xbb, 0xcd,
+	0xa4, 0x87, 0xa2, 0x2e, 0xaf, 0xca, 0x2a, 0x12, 0x9f, 0x17, 0xde, 0x74, 0xfa, 0x6a, 0xac, 0x77,
+	0xb0, 0x35, 0xc4, 0x2b, 0x8c, 0xe9, 0xd1, 0xa4, 0xd7, 0xd0, 0x70, 0x89, 0xc5, 0x34, 0x08, 0x3d,
+	0x6e, 0xce, 0x56, 0x6b, 0xb3, 0x6c, 0x99, 0xb3, 0x76, 0xdf, 0x28, 0xd3, 0x9f, 0x9c, 0x19, 0x55,
+	0xeb, 0xe5, 0xf2, 0x30, 0x11, 0x8c, 0xb0, 0x25, 0x58, 0x1c, 0x4e, 0xa7, 0xaf, 0xc9, 0xd3, 0xe9,
+	0xf3, 0xae, 0x1c, 0x44, 0x18, 0x33, 0xe2, 0x4a, 0x1a, 0x92, 0x75, 0x04, 0x0d, 0x37, 0x89, 0xa2,
+	0x18, 0xb5, 0xee, 0x5e, 0xa1, 0x24, 0x6b, 0xbb, 0xa0, 0xa4, 0xc3, 0xb7, 0x5c, 0x10, 0xc6, 0x73,
+	0xb7, 0xfb, 0xaf, 0xb0, 0x71, 0x29, 0x1f, 0x11, 0x7f, 0x0c, 0x6b, 0x3d, 0xa4, 0xee, 0x0d, 0xbd,
+	0x67, 0x22, 0xa9, 0xf4, 0x6e, 0x45, 0x2c, 0xd6, 0xbf, 0x53, 0xac, 0x7f, 0x88, 0x94, 0xc4, 0x32,
+	0xb5, 0x69, 0xab, 0x05, 0x6b, 0xee, 0x2c, 0x80, 0x7b, 0x1f, 0x30, 0x6f, 0x0d, 0x97, 0xd0, 0xec,
+	0x21, 0xb9, 0x5c, 0x06, 0x02, 0x33, 0xca, 0x37, 0x4e, 0x79, 0xf5, 0x71, 0x32, 0xbd, 0x87, 0x19,
+	0xab, 0xd8, 0x48, 0xdf, 0x7e, 0xc2, 0x23, 0x2f, 0xcf, 0xa5, 0xf9, 0x5d, 0x2e, 0xc0, 0xba, 0x0b,
+	0xb1, 0xee, 0x1d, 0xec, 0xe1, 0xbf, 0x5f, 0xc2, 0x9e, 0xd9, 0xeb, 0xe6, 0x72, 0xf7, 0x7f, 0x5f,
+	0x9b, 0xba, 0xd0, 0x48, 0xd1, 0x3e, 0xf7, 0xd2, 0x06, 0x99, 0x76, 0x54, 0x4d, 0x5e, 0x94, 0xd0,
+	0x5f, 0xa0, 0x69, 0x9a, 0xb9, 0xc2, 0x29, 0x7b, 0xc2, 0xe8, 0x8b, 0xc2, 0xff, 0x80, 0xed, 0x62,
+	0x73, 0x5c, 0xa0, 0x37, 0x39, 0x8f, 0xd5, 0x98, 0x8b, 0xca, 0x5f, 0xa4, 0x22, 0xde, 0xdd, 0x62,
+	0xf3, 0x90, 0x5d, 0x68, 0x64, 0xb7, 0xb3, 0x1e, 0x86, 0xe9, 0x5f, 0xfd, 0xd9, 0xf4, 0x20, 0xac,
+	0xc8, 0x8b, 0x30, 0x2d, 0x58, 0xcb, 0xfc, 0x2f, 0xda, 0x4a, 0x52, 0xf5, 0xa4, 0x4d, 0x45, 0x73,
+	0xd5, 0x5e, 0x80, 0x38, 0xfc, 0xfb, 0x0b, 0xd8, 0x1b, 0x08, 0xfa, 0x2c, 0xe5, 0x3b, 0x86, 0xcd,
+	0xf3, 0x58, 0xdd, 0xdc, 0x9a, 0x24, 0xe7, 0x1d, 0xb5, 0x5c, 0xe6, 0x83, 0xf4, 0xec, 0xae, 0xe4,
+	0xb7, 0x2f, 0x35, 0x99, 0xd3, 0xd7, 0xfa, 0x71, 0x46, 0xf2, 0x0b, 0x63, 0x3e, 0x55, 0x01, 0x7f,
+	0x40, 0x9d, 0x4e, 0x22, 0xf8, 0x4a, 0xc5, 0x81, 0x63, 0x1e, 0x10, 0x9e, 0x8a, 0x7d, 0xe7, 0xee,
+	0x8b, 0xe8, 0x64, 0xf7, 0x7d, 0xca, 0xa8, 0x67, 0x48, 0xff, 0xf9, 0x26, 0xe0, 0x34, 0x49, 0x46,
+	0xe6, 0x24, 0x38, 0x28, 0x82, 0xf3, 0xd7, 0xc4, 0xf3, 0xe2, 0x6d, 0xf1, 0xf2, 0x20, 0x50, 0xf7,
+	0x9e, 0x58, 0xa3, 0x27, 0xa9, 0xf9, 0xc5, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x62, 0xe3, 0x4e,
+	0xd4, 0x8b, 0x0d, 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
+
+// AdapterServiceClient is the client API for AdapterService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type AdapterServiceClient interface {
+	// GetHealthStatus is used by an AdapterService client to verify connectivity
+	// to the gRPC server hosting the AdapterService service
+	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error)
+	// Device
+	AdoptDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	ReconcileDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeleteDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	DisableDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	ReEnableDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	RebootDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	SelfTestDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetOfpDeviceInfo(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*inter_container.SwitchCapability, error)
+	ChildDeviceLost(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Ports
+	EnablePort(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error)
+	DisablePort(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Flows
+	UpdateFlowsBulk(ctx context.Context, in *inter_container.BulkFlows, opts ...grpc.CallOption) (*empty.Empty, error)
+	UpdateFlowsIncrementally(ctx context.Context, in *inter_container.IncrementalFlows, opts ...grpc.CallOption) (*empty.Empty, error)
+	//Packets
+	SendPacketOut(ctx context.Context, in *inter_container.PacketOut, opts ...grpc.CallOption) (*empty.Empty, error)
+	// PM
+	UpdatePmConfig(ctx context.Context, in *inter_container.PmConfigsInfo, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Image
+	DownloadOnuImage(ctx context.Context, in *voltha.DeviceImageDownloadRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error)
+	GetOnuImageStatus(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error)
+	AbortOnuImageUpgrade(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error)
+	GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.OnuImages, error)
+	ActivateOnuImage(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error)
+	CommitOnuImage(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error)
+	// Deprecated Image APIs
+	DownloadImage(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error)
+	GetImageDownloadStatus(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error)
+	CancelImageDownload(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error)
+	ActivateImageUpdate(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error)
+	RevertImageUpdate(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error)
+	// Tests
+	StartOmciTest(ctx context.Context, in *inter_container.OMCITest, opts ...grpc.CallOption) (*voltha.TestResponse, error)
+	SimulateAlarm(ctx context.Context, in *inter_container.SimulateAlarmMessage, opts ...grpc.CallOption) (*common.OperationResp, error)
+	// Events
+	SuppressEvent(ctx context.Context, in *voltha.EventFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	UnSuppressEvent(ctx context.Context, in *voltha.EventFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Get/Set
+	GetExtValue(ctx context.Context, in *inter_container.GetExtValueMessage, opts ...grpc.CallOption) (*common.ReturnValues, error)
+	SetExtValue(ctx context.Context, in *inter_container.SetExtValueMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetSingleValue(ctx context.Context, in *extension.SingleGetValueRequest, opts ...grpc.CallOption) (*extension.SingleGetValueResponse, error)
+	SetSingleValue(ctx context.Context, in *extension.SingleSetValueRequest, opts ...grpc.CallOption) (*extension.SingleSetValueResponse, error)
+}
+
+type adapterServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewAdapterServiceClient(cc *grpc.ClientConn) AdapterServiceClient {
+	return &adapterServiceClient{cc}
+}
+
+func (c *adapterServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error) {
+	out := new(voltha.HealthStatus)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetHealthStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) AdoptDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/AdoptDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) ReconcileDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/ReconcileDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) DeleteDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/DeleteDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) DisableDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/DisableDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) ReEnableDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/ReEnableDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) RebootDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/RebootDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SelfTestDevice(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SelfTestDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetOfpDeviceInfo(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*inter_container.SwitchCapability, error) {
+	out := new(inter_container.SwitchCapability)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetOfpDeviceInfo", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) ChildDeviceLost(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/ChildDeviceLost", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) EnablePort(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/EnablePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) DisablePort(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/DisablePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) UpdateFlowsBulk(ctx context.Context, in *inter_container.BulkFlows, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/UpdateFlowsBulk", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) UpdateFlowsIncrementally(ctx context.Context, in *inter_container.IncrementalFlows, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/UpdateFlowsIncrementally", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SendPacketOut(ctx context.Context, in *inter_container.PacketOut, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SendPacketOut", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) UpdatePmConfig(ctx context.Context, in *inter_container.PmConfigsInfo, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/UpdatePmConfig", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) DownloadOnuImage(ctx context.Context, in *voltha.DeviceImageDownloadRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error) {
+	out := new(voltha.DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/DownloadOnuImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetOnuImageStatus(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error) {
+	out := new(voltha.DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetOnuImageStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) AbortOnuImageUpgrade(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error) {
+	out := new(voltha.DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/AbortOnuImageUpgrade", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.OnuImages, error) {
+	out := new(voltha.OnuImages)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetOnuImages", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) ActivateOnuImage(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error) {
+	out := new(voltha.DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/ActivateOnuImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) CommitOnuImage(ctx context.Context, in *voltha.DeviceImageRequest, opts ...grpc.CallOption) (*voltha.DeviceImageResponse, error) {
+	out := new(voltha.DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/CommitOnuImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) DownloadImage(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error) {
+	out := new(voltha.ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/DownloadImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetImageDownloadStatus(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error) {
+	out := new(voltha.ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetImageDownloadStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) CancelImageDownload(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error) {
+	out := new(voltha.ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/CancelImageDownload", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) ActivateImageUpdate(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error) {
+	out := new(voltha.ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/ActivateImageUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) RevertImageUpdate(ctx context.Context, in *inter_container.ImageDownloadMessage, opts ...grpc.CallOption) (*voltha.ImageDownload, error) {
+	out := new(voltha.ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/RevertImageUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) StartOmciTest(ctx context.Context, in *inter_container.OMCITest, opts ...grpc.CallOption) (*voltha.TestResponse, error) {
+	out := new(voltha.TestResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/StartOmciTest", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SimulateAlarm(ctx context.Context, in *inter_container.SimulateAlarmMessage, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SimulateAlarm", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SuppressEvent(ctx context.Context, in *voltha.EventFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SuppressEvent", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) UnSuppressEvent(ctx context.Context, in *voltha.EventFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/UnSuppressEvent", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetExtValue(ctx context.Context, in *inter_container.GetExtValueMessage, opts ...grpc.CallOption) (*common.ReturnValues, error) {
+	out := new(common.ReturnValues)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SetExtValue(ctx context.Context, in *inter_container.SetExtValueMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) GetSingleValue(ctx context.Context, in *extension.SingleGetValueRequest, opts ...grpc.CallOption) (*extension.SingleGetValueResponse, error) {
+	out := new(extension.SingleGetValueResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/GetSingleValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *adapterServiceClient) SetSingleValue(ctx context.Context, in *extension.SingleSetValueRequest, opts ...grpc.CallOption) (*extension.SingleSetValueResponse, error) {
+	out := new(extension.SingleSetValueResponse)
+	err := c.cc.Invoke(ctx, "/voltha.AdapterService/SetSingleValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// AdapterServiceServer is the server API for AdapterService service.
+type AdapterServiceServer interface {
+	// GetHealthStatus is used by an AdapterService client to verify connectivity
+	// to the gRPC server hosting the AdapterService service
+	GetHealthStatus(context.Context, *empty.Empty) (*voltha.HealthStatus, error)
+	// Device
+	AdoptDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	ReconcileDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	DeleteDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	DisableDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	ReEnableDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	RebootDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	SelfTestDevice(context.Context, *voltha.Device) (*empty.Empty, error)
+	GetOfpDeviceInfo(context.Context, *voltha.Device) (*inter_container.SwitchCapability, error)
+	ChildDeviceLost(context.Context, *voltha.Device) (*empty.Empty, error)
+	// Ports
+	EnablePort(context.Context, *voltha.Port) (*empty.Empty, error)
+	DisablePort(context.Context, *voltha.Port) (*empty.Empty, error)
+	// Flows
+	UpdateFlowsBulk(context.Context, *inter_container.BulkFlows) (*empty.Empty, error)
+	UpdateFlowsIncrementally(context.Context, *inter_container.IncrementalFlows) (*empty.Empty, error)
+	//Packets
+	SendPacketOut(context.Context, *inter_container.PacketOut) (*empty.Empty, error)
+	// PM
+	UpdatePmConfig(context.Context, *inter_container.PmConfigsInfo) (*empty.Empty, error)
+	// Image
+	DownloadOnuImage(context.Context, *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error)
+	GetOnuImageStatus(context.Context, *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
+	AbortOnuImageUpgrade(context.Context, *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
+	GetOnuImages(context.Context, *common.ID) (*voltha.OnuImages, error)
+	ActivateOnuImage(context.Context, *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
+	CommitOnuImage(context.Context, *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error)
+	// Deprecated Image APIs
+	DownloadImage(context.Context, *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error)
+	GetImageDownloadStatus(context.Context, *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error)
+	CancelImageDownload(context.Context, *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error)
+	ActivateImageUpdate(context.Context, *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error)
+	RevertImageUpdate(context.Context, *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error)
+	// Tests
+	StartOmciTest(context.Context, *inter_container.OMCITest) (*voltha.TestResponse, error)
+	SimulateAlarm(context.Context, *inter_container.SimulateAlarmMessage) (*common.OperationResp, error)
+	// Events
+	SuppressEvent(context.Context, *voltha.EventFilter) (*empty.Empty, error)
+	UnSuppressEvent(context.Context, *voltha.EventFilter) (*empty.Empty, error)
+	// Get/Set
+	GetExtValue(context.Context, *inter_container.GetExtValueMessage) (*common.ReturnValues, error)
+	SetExtValue(context.Context, *inter_container.SetExtValueMessage) (*empty.Empty, error)
+	GetSingleValue(context.Context, *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error)
+	SetSingleValue(context.Context, *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error)
+}
+
+// UnimplementedAdapterServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedAdapterServiceServer struct {
+}
+
+func (*UnimplementedAdapterServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*voltha.HealthStatus, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
+}
+func (*UnimplementedAdapterServiceServer) AdoptDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method AdoptDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) ReconcileDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ReconcileDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) DeleteDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) DisableDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisableDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) ReEnableDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ReEnableDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) RebootDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RebootDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SelfTestDevice(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SelfTestDevice not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetOfpDeviceInfo(ctx context.Context, req *voltha.Device) (*inter_container.SwitchCapability, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetOfpDeviceInfo not implemented")
+}
+func (*UnimplementedAdapterServiceServer) ChildDeviceLost(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChildDeviceLost not implemented")
+}
+func (*UnimplementedAdapterServiceServer) EnablePort(ctx context.Context, req *voltha.Port) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method EnablePort not implemented")
+}
+func (*UnimplementedAdapterServiceServer) DisablePort(ctx context.Context, req *voltha.Port) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisablePort not implemented")
+}
+func (*UnimplementedAdapterServiceServer) UpdateFlowsBulk(ctx context.Context, req *inter_container.BulkFlows) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateFlowsBulk not implemented")
+}
+func (*UnimplementedAdapterServiceServer) UpdateFlowsIncrementally(ctx context.Context, req *inter_container.IncrementalFlows) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateFlowsIncrementally not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SendPacketOut(ctx context.Context, req *inter_container.PacketOut) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SendPacketOut not implemented")
+}
+func (*UnimplementedAdapterServiceServer) UpdatePmConfig(ctx context.Context, req *inter_container.PmConfigsInfo) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdatePmConfig not implemented")
+}
+func (*UnimplementedAdapterServiceServer) DownloadOnuImage(ctx context.Context, req *voltha.DeviceImageDownloadRequest) (*voltha.DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DownloadOnuImage not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetOnuImageStatus(ctx context.Context, req *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetOnuImageStatus not implemented")
+}
+func (*UnimplementedAdapterServiceServer) AbortOnuImageUpgrade(ctx context.Context, req *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method AbortOnuImageUpgrade not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetOnuImages(ctx context.Context, req *common.ID) (*voltha.OnuImages, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetOnuImages not implemented")
+}
+func (*UnimplementedAdapterServiceServer) ActivateOnuImage(ctx context.Context, req *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ActivateOnuImage not implemented")
+}
+func (*UnimplementedAdapterServiceServer) CommitOnuImage(ctx context.Context, req *voltha.DeviceImageRequest) (*voltha.DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CommitOnuImage not implemented")
+}
+func (*UnimplementedAdapterServiceServer) DownloadImage(ctx context.Context, req *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DownloadImage not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetImageDownloadStatus(ctx context.Context, req *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetImageDownloadStatus not implemented")
+}
+func (*UnimplementedAdapterServiceServer) CancelImageDownload(ctx context.Context, req *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CancelImageDownload not implemented")
+}
+func (*UnimplementedAdapterServiceServer) ActivateImageUpdate(ctx context.Context, req *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ActivateImageUpdate not implemented")
+}
+func (*UnimplementedAdapterServiceServer) RevertImageUpdate(ctx context.Context, req *inter_container.ImageDownloadMessage) (*voltha.ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RevertImageUpdate not implemented")
+}
+func (*UnimplementedAdapterServiceServer) StartOmciTest(ctx context.Context, req *inter_container.OMCITest) (*voltha.TestResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method StartOmciTest not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SimulateAlarm(ctx context.Context, req *inter_container.SimulateAlarmMessage) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SimulateAlarm not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SuppressEvent(ctx context.Context, req *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SuppressEvent not implemented")
+}
+func (*UnimplementedAdapterServiceServer) UnSuppressEvent(ctx context.Context, req *voltha.EventFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UnSuppressEvent not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetExtValue(ctx context.Context, req *inter_container.GetExtValueMessage) (*common.ReturnValues, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SetExtValue(ctx context.Context, req *inter_container.SetExtValueMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SetExtValue not implemented")
+}
+func (*UnimplementedAdapterServiceServer) GetSingleValue(ctx context.Context, req *extension.SingleGetValueRequest) (*extension.SingleGetValueResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetSingleValue not implemented")
+}
+func (*UnimplementedAdapterServiceServer) SetSingleValue(ctx context.Context, req *extension.SingleSetValueRequest) (*extension.SingleSetValueResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SetSingleValue not implemented")
+}
+
+func RegisterAdapterServiceServer(s *grpc.Server, srv AdapterServiceServer) {
+	s.RegisterService(&_AdapterService_serviceDesc, srv)
+}
+
+func _AdapterService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetHealthStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetHealthStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_AdoptDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).AdoptDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/AdoptDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).AdoptDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_ReconcileDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).ReconcileDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/ReconcileDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).ReconcileDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_DeleteDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).DeleteDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/DeleteDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).DeleteDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_DisableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).DisableDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/DisableDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).DisableDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_ReEnableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).ReEnableDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/ReEnableDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).ReEnableDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_RebootDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).RebootDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/RebootDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).RebootDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SelfTestDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SelfTestDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SelfTestDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SelfTestDevice(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetOfpDeviceInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetOfpDeviceInfo(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetOfpDeviceInfo",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetOfpDeviceInfo(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_ChildDeviceLost_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).ChildDeviceLost(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/ChildDeviceLost",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).ChildDeviceLost(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_EnablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Port)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).EnablePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/EnablePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).EnablePort(ctx, req.(*voltha.Port))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_DisablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Port)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).DisablePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/DisablePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).DisablePort(ctx, req.(*voltha.Port))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_UpdateFlowsBulk_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.BulkFlows)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).UpdateFlowsBulk(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/UpdateFlowsBulk",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).UpdateFlowsBulk(ctx, req.(*inter_container.BulkFlows))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_UpdateFlowsIncrementally_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.IncrementalFlows)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).UpdateFlowsIncrementally(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/UpdateFlowsIncrementally",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).UpdateFlowsIncrementally(ctx, req.(*inter_container.IncrementalFlows))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SendPacketOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PacketOut)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SendPacketOut(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SendPacketOut",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SendPacketOut(ctx, req.(*inter_container.PacketOut))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_UpdatePmConfig_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PmConfigsInfo)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).UpdatePmConfig(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/UpdatePmConfig",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).UpdatePmConfig(ctx, req.(*inter_container.PmConfigsInfo))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_DownloadOnuImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.DeviceImageDownloadRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).DownloadOnuImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/DownloadOnuImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).DownloadOnuImage(ctx, req.(*voltha.DeviceImageDownloadRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetOnuImageStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetOnuImageStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetOnuImageStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetOnuImageStatus(ctx, req.(*voltha.DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_AbortOnuImageUpgrade_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).AbortOnuImageUpgrade(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/AbortOnuImageUpgrade",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).AbortOnuImageUpgrade(ctx, req.(*voltha.DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetOnuImages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetOnuImages(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetOnuImages",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetOnuImages(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_ActivateOnuImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).ActivateOnuImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/ActivateOnuImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).ActivateOnuImage(ctx, req.(*voltha.DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_CommitOnuImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).CommitOnuImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/CommitOnuImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).CommitOnuImage(ctx, req.(*voltha.DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_DownloadImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ImageDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).DownloadImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/DownloadImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).DownloadImage(ctx, req.(*inter_container.ImageDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetImageDownloadStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ImageDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetImageDownloadStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetImageDownloadStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetImageDownloadStatus(ctx, req.(*inter_container.ImageDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_CancelImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ImageDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).CancelImageDownload(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/CancelImageDownload",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).CancelImageDownload(ctx, req.(*inter_container.ImageDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_ActivateImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ImageDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).ActivateImageUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/ActivateImageUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).ActivateImageUpdate(ctx, req.(*inter_container.ImageDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_RevertImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ImageDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).RevertImageUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/RevertImageUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).RevertImageUpdate(ctx, req.(*inter_container.ImageDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_StartOmciTest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.OMCITest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).StartOmciTest(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/StartOmciTest",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).StartOmciTest(ctx, req.(*inter_container.OMCITest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SimulateAlarm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.SimulateAlarmMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SimulateAlarm(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SimulateAlarm",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SimulateAlarm(ctx, req.(*inter_container.SimulateAlarmMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SuppressEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.EventFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SuppressEvent(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SuppressEvent",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SuppressEvent(ctx, req.(*voltha.EventFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_UnSuppressEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.EventFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).UnSuppressEvent(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/UnSuppressEvent",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).UnSuppressEvent(ctx, req.(*voltha.EventFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.GetExtValueMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetExtValue(ctx, req.(*inter_container.GetExtValueMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.SetExtValueMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SetExtValue(ctx, req.(*inter_container.SetExtValueMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_GetSingleValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(extension.SingleGetValueRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).GetSingleValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/GetSingleValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).GetSingleValue(ctx, req.(*extension.SingleGetValueRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _AdapterService_SetSingleValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(extension.SingleSetValueRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdapterServiceServer).SetSingleValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.AdapterService/SetSingleValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdapterServiceServer).SetSingleValue(ctx, req.(*extension.SingleSetValueRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _AdapterService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.AdapterService",
+	HandlerType: (*AdapterServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetHealthStatus",
+			Handler:    _AdapterService_GetHealthStatus_Handler,
+		},
+		{
+			MethodName: "AdoptDevice",
+			Handler:    _AdapterService_AdoptDevice_Handler,
+		},
+		{
+			MethodName: "ReconcileDevice",
+			Handler:    _AdapterService_ReconcileDevice_Handler,
+		},
+		{
+			MethodName: "DeleteDevice",
+			Handler:    _AdapterService_DeleteDevice_Handler,
+		},
+		{
+			MethodName: "DisableDevice",
+			Handler:    _AdapterService_DisableDevice_Handler,
+		},
+		{
+			MethodName: "ReEnableDevice",
+			Handler:    _AdapterService_ReEnableDevice_Handler,
+		},
+		{
+			MethodName: "RebootDevice",
+			Handler:    _AdapterService_RebootDevice_Handler,
+		},
+		{
+			MethodName: "SelfTestDevice",
+			Handler:    _AdapterService_SelfTestDevice_Handler,
+		},
+		{
+			MethodName: "GetOfpDeviceInfo",
+			Handler:    _AdapterService_GetOfpDeviceInfo_Handler,
+		},
+		{
+			MethodName: "ChildDeviceLost",
+			Handler:    _AdapterService_ChildDeviceLost_Handler,
+		},
+		{
+			MethodName: "EnablePort",
+			Handler:    _AdapterService_EnablePort_Handler,
+		},
+		{
+			MethodName: "DisablePort",
+			Handler:    _AdapterService_DisablePort_Handler,
+		},
+		{
+			MethodName: "UpdateFlowsBulk",
+			Handler:    _AdapterService_UpdateFlowsBulk_Handler,
+		},
+		{
+			MethodName: "UpdateFlowsIncrementally",
+			Handler:    _AdapterService_UpdateFlowsIncrementally_Handler,
+		},
+		{
+			MethodName: "SendPacketOut",
+			Handler:    _AdapterService_SendPacketOut_Handler,
+		},
+		{
+			MethodName: "UpdatePmConfig",
+			Handler:    _AdapterService_UpdatePmConfig_Handler,
+		},
+		{
+			MethodName: "DownloadOnuImage",
+			Handler:    _AdapterService_DownloadOnuImage_Handler,
+		},
+		{
+			MethodName: "GetOnuImageStatus",
+			Handler:    _AdapterService_GetOnuImageStatus_Handler,
+		},
+		{
+			MethodName: "AbortOnuImageUpgrade",
+			Handler:    _AdapterService_AbortOnuImageUpgrade_Handler,
+		},
+		{
+			MethodName: "GetOnuImages",
+			Handler:    _AdapterService_GetOnuImages_Handler,
+		},
+		{
+			MethodName: "ActivateOnuImage",
+			Handler:    _AdapterService_ActivateOnuImage_Handler,
+		},
+		{
+			MethodName: "CommitOnuImage",
+			Handler:    _AdapterService_CommitOnuImage_Handler,
+		},
+		{
+			MethodName: "DownloadImage",
+			Handler:    _AdapterService_DownloadImage_Handler,
+		},
+		{
+			MethodName: "GetImageDownloadStatus",
+			Handler:    _AdapterService_GetImageDownloadStatus_Handler,
+		},
+		{
+			MethodName: "CancelImageDownload",
+			Handler:    _AdapterService_CancelImageDownload_Handler,
+		},
+		{
+			MethodName: "ActivateImageUpdate",
+			Handler:    _AdapterService_ActivateImageUpdate_Handler,
+		},
+		{
+			MethodName: "RevertImageUpdate",
+			Handler:    _AdapterService_RevertImageUpdate_Handler,
+		},
+		{
+			MethodName: "StartOmciTest",
+			Handler:    _AdapterService_StartOmciTest_Handler,
+		},
+		{
+			MethodName: "SimulateAlarm",
+			Handler:    _AdapterService_SimulateAlarm_Handler,
+		},
+		{
+			MethodName: "SuppressEvent",
+			Handler:    _AdapterService_SuppressEvent_Handler,
+		},
+		{
+			MethodName: "UnSuppressEvent",
+			Handler:    _AdapterService_UnSuppressEvent_Handler,
+		},
+		{
+			MethodName: "GetExtValue",
+			Handler:    _AdapterService_GetExtValue_Handler,
+		},
+		{
+			MethodName: "SetExtValue",
+			Handler:    _AdapterService_SetExtValue_Handler,
+		},
+		{
+			MethodName: "GetSingleValue",
+			Handler:    _AdapterService_GetSingleValue_Handler,
+		},
+		{
+			MethodName: "SetSingleValue",
+			Handler:    _AdapterService_SetSingleValue_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/adapter_services.proto",
+}
+
+// OnuInterAdapterServiceClient is the client API for OnuInterAdapterService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type OnuInterAdapterServiceClient interface {
+	// GetHealthStatus is used by an OnuInterAdapterService client to verify connectivity
+	// to the gRPC server hosting the OnuInterAdapterService service
+	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error)
+	OnuIndication(ctx context.Context, in *inter_container.OnuIndicationMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	OmciIndication(ctx context.Context, in *inter_container.OmciMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	DownloadTechProfile(ctx context.Context, in *inter_container.TechProfileDownloadMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeleteGemPort(ctx context.Context, in *inter_container.DeleteGemPortMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeleteTCont(ctx context.Context, in *inter_container.DeleteTcontMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+}
+
+type onuInterAdapterServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewOnuInterAdapterServiceClient(cc *grpc.ClientConn) OnuInterAdapterServiceClient {
+	return &onuInterAdapterServiceClient{cc}
+}
+
+func (c *onuInterAdapterServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error) {
+	out := new(voltha.HealthStatus)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/GetHealthStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *onuInterAdapterServiceClient) OnuIndication(ctx context.Context, in *inter_container.OnuIndicationMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/OnuIndication", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *onuInterAdapterServiceClient) OmciIndication(ctx context.Context, in *inter_container.OmciMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/OmciIndication", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *onuInterAdapterServiceClient) DownloadTechProfile(ctx context.Context, in *inter_container.TechProfileDownloadMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/DownloadTechProfile", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *onuInterAdapterServiceClient) DeleteGemPort(ctx context.Context, in *inter_container.DeleteGemPortMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/DeleteGemPort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *onuInterAdapterServiceClient) DeleteTCont(ctx context.Context, in *inter_container.DeleteTcontMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OnuInterAdapterService/DeleteTCont", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// OnuInterAdapterServiceServer is the server API for OnuInterAdapterService service.
+type OnuInterAdapterServiceServer interface {
+	// GetHealthStatus is used by an OnuInterAdapterService client to verify connectivity
+	// to the gRPC server hosting the OnuInterAdapterService service
+	GetHealthStatus(context.Context, *empty.Empty) (*voltha.HealthStatus, error)
+	OnuIndication(context.Context, *inter_container.OnuIndicationMessage) (*empty.Empty, error)
+	OmciIndication(context.Context, *inter_container.OmciMessage) (*empty.Empty, error)
+	DownloadTechProfile(context.Context, *inter_container.TechProfileDownloadMessage) (*empty.Empty, error)
+	DeleteGemPort(context.Context, *inter_container.DeleteGemPortMessage) (*empty.Empty, error)
+	DeleteTCont(context.Context, *inter_container.DeleteTcontMessage) (*empty.Empty, error)
+}
+
+// UnimplementedOnuInterAdapterServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedOnuInterAdapterServiceServer struct {
+}
+
+func (*UnimplementedOnuInterAdapterServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*voltha.HealthStatus, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
+}
+func (*UnimplementedOnuInterAdapterServiceServer) OnuIndication(ctx context.Context, req *inter_container.OnuIndicationMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method OnuIndication not implemented")
+}
+func (*UnimplementedOnuInterAdapterServiceServer) OmciIndication(ctx context.Context, req *inter_container.OmciMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method OmciIndication not implemented")
+}
+func (*UnimplementedOnuInterAdapterServiceServer) DownloadTechProfile(ctx context.Context, req *inter_container.TechProfileDownloadMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DownloadTechProfile not implemented")
+}
+func (*UnimplementedOnuInterAdapterServiceServer) DeleteGemPort(ctx context.Context, req *inter_container.DeleteGemPortMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteGemPort not implemented")
+}
+func (*UnimplementedOnuInterAdapterServiceServer) DeleteTCont(ctx context.Context, req *inter_container.DeleteTcontMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteTCont not implemented")
+}
+
+func RegisterOnuInterAdapterServiceServer(s *grpc.Server, srv OnuInterAdapterServiceServer) {
+	s.RegisterService(&_OnuInterAdapterService_serviceDesc, srv)
+}
+
+func _OnuInterAdapterService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).GetHealthStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/GetHealthStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OnuInterAdapterService_OnuIndication_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.OnuIndicationMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).OnuIndication(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/OnuIndication",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).OnuIndication(ctx, req.(*inter_container.OnuIndicationMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OnuInterAdapterService_OmciIndication_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.OmciMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).OmciIndication(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/OmciIndication",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).OmciIndication(ctx, req.(*inter_container.OmciMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OnuInterAdapterService_DownloadTechProfile_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.TechProfileDownloadMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).DownloadTechProfile(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/DownloadTechProfile",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).DownloadTechProfile(ctx, req.(*inter_container.TechProfileDownloadMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OnuInterAdapterService_DeleteGemPort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeleteGemPortMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).DeleteGemPort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/DeleteGemPort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).DeleteGemPort(ctx, req.(*inter_container.DeleteGemPortMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OnuInterAdapterService_DeleteTCont_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeleteTcontMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OnuInterAdapterServiceServer).DeleteTCont(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OnuInterAdapterService/DeleteTCont",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OnuInterAdapterServiceServer).DeleteTCont(ctx, req.(*inter_container.DeleteTcontMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _OnuInterAdapterService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.OnuInterAdapterService",
+	HandlerType: (*OnuInterAdapterServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetHealthStatus",
+			Handler:    _OnuInterAdapterService_GetHealthStatus_Handler,
+		},
+		{
+			MethodName: "OnuIndication",
+			Handler:    _OnuInterAdapterService_OnuIndication_Handler,
+		},
+		{
+			MethodName: "OmciIndication",
+			Handler:    _OnuInterAdapterService_OmciIndication_Handler,
+		},
+		{
+			MethodName: "DownloadTechProfile",
+			Handler:    _OnuInterAdapterService_DownloadTechProfile_Handler,
+		},
+		{
+			MethodName: "DeleteGemPort",
+			Handler:    _OnuInterAdapterService_DeleteGemPort_Handler,
+		},
+		{
+			MethodName: "DeleteTCont",
+			Handler:    _OnuInterAdapterService_DeleteTCont_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/adapter_services.proto",
+}
+
+// OltInterAdapterServiceClient is the client API for OltInterAdapterService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type OltInterAdapterServiceClient interface {
+	// GetHealthStatus is used by an OltInterAdapterService client to verify connectivity
+	// to the gRPC server hosting the OltInterAdapterService service
+	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error)
+	ProxyOmciRequest(ctx context.Context, in *inter_container.OmciMessage, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetTechProfileInstance(ctx context.Context, in *inter_container.TechProfileInstanceRequestMessage, opts ...grpc.CallOption) (*inter_container.TechProfileDownloadMessage, error)
+}
+
+type oltInterAdapterServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewOltInterAdapterServiceClient(cc *grpc.ClientConn) OltInterAdapterServiceClient {
+	return &oltInterAdapterServiceClient{cc}
+}
+
+func (c *oltInterAdapterServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error) {
+	out := new(voltha.HealthStatus)
+	err := c.cc.Invoke(ctx, "/voltha.OltInterAdapterService/GetHealthStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *oltInterAdapterServiceClient) ProxyOmciRequest(ctx context.Context, in *inter_container.OmciMessage, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.OltInterAdapterService/ProxyOmciRequest", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *oltInterAdapterServiceClient) GetTechProfileInstance(ctx context.Context, in *inter_container.TechProfileInstanceRequestMessage, opts ...grpc.CallOption) (*inter_container.TechProfileDownloadMessage, error) {
+	out := new(inter_container.TechProfileDownloadMessage)
+	err := c.cc.Invoke(ctx, "/voltha.OltInterAdapterService/GetTechProfileInstance", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// OltInterAdapterServiceServer is the server API for OltInterAdapterService service.
+type OltInterAdapterServiceServer interface {
+	// GetHealthStatus is used by an OltInterAdapterService client to verify connectivity
+	// to the gRPC server hosting the OltInterAdapterService service
+	GetHealthStatus(context.Context, *empty.Empty) (*voltha.HealthStatus, error)
+	ProxyOmciRequest(context.Context, *inter_container.OmciMessage) (*empty.Empty, error)
+	GetTechProfileInstance(context.Context, *inter_container.TechProfileInstanceRequestMessage) (*inter_container.TechProfileDownloadMessage, error)
+}
+
+// UnimplementedOltInterAdapterServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedOltInterAdapterServiceServer struct {
+}
+
+func (*UnimplementedOltInterAdapterServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*voltha.HealthStatus, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
+}
+func (*UnimplementedOltInterAdapterServiceServer) ProxyOmciRequest(ctx context.Context, req *inter_container.OmciMessage) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ProxyOmciRequest not implemented")
+}
+func (*UnimplementedOltInterAdapterServiceServer) GetTechProfileInstance(ctx context.Context, req *inter_container.TechProfileInstanceRequestMessage) (*inter_container.TechProfileDownloadMessage, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetTechProfileInstance not implemented")
+}
+
+func RegisterOltInterAdapterServiceServer(s *grpc.Server, srv OltInterAdapterServiceServer) {
+	s.RegisterService(&_OltInterAdapterService_serviceDesc, srv)
+}
+
+func _OltInterAdapterService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OltInterAdapterServiceServer).GetHealthStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OltInterAdapterService/GetHealthStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OltInterAdapterServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OltInterAdapterService_ProxyOmciRequest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.OmciMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OltInterAdapterServiceServer).ProxyOmciRequest(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OltInterAdapterService/ProxyOmciRequest",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OltInterAdapterServiceServer).ProxyOmciRequest(ctx, req.(*inter_container.OmciMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _OltInterAdapterService_GetTechProfileInstance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.TechProfileInstanceRequestMessage)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OltInterAdapterServiceServer).GetTechProfileInstance(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.OltInterAdapterService/GetTechProfileInstance",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OltInterAdapterServiceServer).GetTechProfileInstance(ctx, req.(*inter_container.TechProfileInstanceRequestMessage))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _OltInterAdapterService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.OltInterAdapterService",
+	HandlerType: (*OltInterAdapterServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetHealthStatus",
+			Handler:    _OltInterAdapterService_GetHealthStatus_Handler,
+		},
+		{
+			MethodName: "ProxyOmciRequest",
+			Handler:    _OltInterAdapterService_ProxyOmciRequest_Handler,
+		},
+		{
+			MethodName: "GetTechProfileInstance",
+			Handler:    _OltInterAdapterService_GetTechProfileInstance_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/adapter_services.proto",
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/common/common.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/common/common.pb.go
new file mode 100644
index 0000000..a397778
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/common/common.pb.go
@@ -0,0 +1,651 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/common.proto
+
+package common
+
+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
+
+type TestModeKeys int32
+
+const (
+	TestModeKeys_api_test TestModeKeys = 0
+)
+
+var TestModeKeys_name = map[int32]string{
+	0: "api_test",
+}
+
+var TestModeKeys_value = map[string]int32{
+	"api_test": 0,
+}
+
+func (x TestModeKeys) String() string {
+	return proto.EnumName(TestModeKeys_name, int32(x))
+}
+
+func (TestModeKeys) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{0}
+}
+
+// Administrative State
+type AdminState_Types int32
+
+const (
+	// The administrative state of the device is unknown
+	AdminState_UNKNOWN AdminState_Types = 0
+	// The device is pre-provisioned into Voltha, but not contacted by it
+	AdminState_PREPROVISIONED AdminState_Types = 1
+	// The device is enabled for activation and operation
+	AdminState_ENABLED AdminState_Types = 2
+	// The device is disabled and shall not perform its intended forwarding
+	// functions other than being available for re-activation.
+	AdminState_DISABLED AdminState_Types = 3
+	// The device is in the state of image download
+	AdminState_DOWNLOADING_IMAGE AdminState_Types = 4
+)
+
+var AdminState_Types_name = map[int32]string{
+	0: "UNKNOWN",
+	1: "PREPROVISIONED",
+	2: "ENABLED",
+	3: "DISABLED",
+	4: "DOWNLOADING_IMAGE",
+}
+
+var AdminState_Types_value = map[string]int32{
+	"UNKNOWN":           0,
+	"PREPROVISIONED":    1,
+	"ENABLED":           2,
+	"DISABLED":          3,
+	"DOWNLOADING_IMAGE": 4,
+}
+
+func (x AdminState_Types) String() string {
+	return proto.EnumName(AdminState_Types_name, int32(x))
+}
+
+func (AdminState_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{2, 0}
+}
+
+// Operational Status
+type OperStatus_Types int32
+
+const (
+	// The status of the device is unknown at this point
+	OperStatus_UNKNOWN OperStatus_Types = 0
+	// The device has been discovered, but not yet activated
+	OperStatus_DISCOVERED OperStatus_Types = 1
+	// The device is being activated (booted, rebooted, upgraded, etc.)
+	OperStatus_ACTIVATING OperStatus_Types = 2
+	// Service impacting tests are being conducted
+	OperStatus_TESTING OperStatus_Types = 3
+	// The device is up and active
+	OperStatus_ACTIVE OperStatus_Types = 4
+	// The device has failed and cannot fulfill its intended role
+	OperStatus_FAILED OperStatus_Types = 5
+	// The device is reconciling
+	OperStatus_RECONCILING OperStatus_Types = 6
+	// The device is in reconciling failed
+	OperStatus_RECONCILING_FAILED OperStatus_Types = 7
+)
+
+var OperStatus_Types_name = map[int32]string{
+	0: "UNKNOWN",
+	1: "DISCOVERED",
+	2: "ACTIVATING",
+	3: "TESTING",
+	4: "ACTIVE",
+	5: "FAILED",
+	6: "RECONCILING",
+	7: "RECONCILING_FAILED",
+}
+
+var OperStatus_Types_value = map[string]int32{
+	"UNKNOWN":            0,
+	"DISCOVERED":         1,
+	"ACTIVATING":         2,
+	"TESTING":            3,
+	"ACTIVE":             4,
+	"FAILED":             5,
+	"RECONCILING":        6,
+	"RECONCILING_FAILED": 7,
+}
+
+func (x OperStatus_Types) String() string {
+	return proto.EnumName(OperStatus_Types_name, int32(x))
+}
+
+func (OperStatus_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{3, 0}
+}
+
+// Connectivity Status
+type ConnectStatus_Types int32
+
+const (
+	// The device connectivity status is unknown
+	ConnectStatus_UNKNOWN ConnectStatus_Types = 0
+	// The device cannot be reached by Voltha
+	ConnectStatus_UNREACHABLE ConnectStatus_Types = 1
+	// There is live communication between device and Voltha
+	ConnectStatus_REACHABLE ConnectStatus_Types = 2
+)
+
+var ConnectStatus_Types_name = map[int32]string{
+	0: "UNKNOWN",
+	1: "UNREACHABLE",
+	2: "REACHABLE",
+}
+
+var ConnectStatus_Types_value = map[string]int32{
+	"UNKNOWN":     0,
+	"UNREACHABLE": 1,
+	"REACHABLE":   2,
+}
+
+func (x ConnectStatus_Types) String() string {
+	return proto.EnumName(ConnectStatus_Types_name, int32(x))
+}
+
+func (ConnectStatus_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{4, 0}
+}
+
+type OperationResp_OperationReturnCode int32
+
+const (
+	OperationResp_OPERATION_SUCCESS     OperationResp_OperationReturnCode = 0
+	OperationResp_OPERATION_FAILURE     OperationResp_OperationReturnCode = 1
+	OperationResp_OPERATION_UNSUPPORTED OperationResp_OperationReturnCode = 2
+	OperationResp_OPERATION_IN_PROGRESS OperationResp_OperationReturnCode = 3
+)
+
+var OperationResp_OperationReturnCode_name = map[int32]string{
+	0: "OPERATION_SUCCESS",
+	1: "OPERATION_FAILURE",
+	2: "OPERATION_UNSUPPORTED",
+	3: "OPERATION_IN_PROGRESS",
+}
+
+var OperationResp_OperationReturnCode_value = map[string]int32{
+	"OPERATION_SUCCESS":     0,
+	"OPERATION_FAILURE":     1,
+	"OPERATION_UNSUPPORTED": 2,
+	"OPERATION_IN_PROGRESS": 3,
+}
+
+func (x OperationResp_OperationReturnCode) String() string {
+	return proto.EnumName(OperationResp_OperationReturnCode_name, int32(x))
+}
+
+func (OperationResp_OperationReturnCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{5, 0}
+}
+
+type ValueType_Type int32
+
+const (
+	ValueType_EMPTY    ValueType_Type = 0
+	ValueType_DISTANCE ValueType_Type = 1
+)
+
+var ValueType_Type_name = map[int32]string{
+	0: "EMPTY",
+	1: "DISTANCE",
+}
+
+var ValueType_Type_value = map[string]int32{
+	"EMPTY":    0,
+	"DISTANCE": 1,
+}
+
+func (x ValueType_Type) String() string {
+	return proto.EnumName(ValueType_Type_name, int32(x))
+}
+
+func (ValueType_Type) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{6, 0}
+}
+
+// Convey a resource identifier
+type ID struct {
+	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ID) Reset()         { *m = ID{} }
+func (m *ID) String() string { return proto.CompactTextString(m) }
+func (*ID) ProtoMessage()    {}
+func (*ID) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{0}
+}
+
+func (m *ID) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ID.Unmarshal(m, b)
+}
+func (m *ID) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ID.Marshal(b, m, deterministic)
+}
+func (m *ID) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ID.Merge(m, src)
+}
+func (m *ID) XXX_Size() int {
+	return xxx_messageInfo_ID.Size(m)
+}
+func (m *ID) XXX_DiscardUnknown() {
+	xxx_messageInfo_ID.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ID proto.InternalMessageInfo
+
+func (m *ID) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+// Represents a list of IDs
+type IDs struct {
+	Items                []*ID    `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *IDs) Reset()         { *m = IDs{} }
+func (m *IDs) String() string { return proto.CompactTextString(m) }
+func (*IDs) ProtoMessage()    {}
+func (*IDs) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{1}
+}
+
+func (m *IDs) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_IDs.Unmarshal(m, b)
+}
+func (m *IDs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_IDs.Marshal(b, m, deterministic)
+}
+func (m *IDs) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IDs.Merge(m, src)
+}
+func (m *IDs) XXX_Size() int {
+	return xxx_messageInfo_IDs.Size(m)
+}
+func (m *IDs) XXX_DiscardUnknown() {
+	xxx_messageInfo_IDs.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_IDs proto.InternalMessageInfo
+
+func (m *IDs) GetItems() []*ID {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type AdminState struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AdminState) Reset()         { *m = AdminState{} }
+func (m *AdminState) String() string { return proto.CompactTextString(m) }
+func (*AdminState) ProtoMessage()    {}
+func (*AdminState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{2}
+}
+
+func (m *AdminState) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AdminState.Unmarshal(m, b)
+}
+func (m *AdminState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AdminState.Marshal(b, m, deterministic)
+}
+func (m *AdminState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AdminState.Merge(m, src)
+}
+func (m *AdminState) XXX_Size() int {
+	return xxx_messageInfo_AdminState.Size(m)
+}
+func (m *AdminState) XXX_DiscardUnknown() {
+	xxx_messageInfo_AdminState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AdminState proto.InternalMessageInfo
+
+type OperStatus struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OperStatus) Reset()         { *m = OperStatus{} }
+func (m *OperStatus) String() string { return proto.CompactTextString(m) }
+func (*OperStatus) ProtoMessage()    {}
+func (*OperStatus) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{3}
+}
+
+func (m *OperStatus) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OperStatus.Unmarshal(m, b)
+}
+func (m *OperStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OperStatus.Marshal(b, m, deterministic)
+}
+func (m *OperStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OperStatus.Merge(m, src)
+}
+func (m *OperStatus) XXX_Size() int {
+	return xxx_messageInfo_OperStatus.Size(m)
+}
+func (m *OperStatus) XXX_DiscardUnknown() {
+	xxx_messageInfo_OperStatus.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OperStatus proto.InternalMessageInfo
+
+type ConnectStatus struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ConnectStatus) Reset()         { *m = ConnectStatus{} }
+func (m *ConnectStatus) String() string { return proto.CompactTextString(m) }
+func (*ConnectStatus) ProtoMessage()    {}
+func (*ConnectStatus) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{4}
+}
+
+func (m *ConnectStatus) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ConnectStatus.Unmarshal(m, b)
+}
+func (m *ConnectStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ConnectStatus.Marshal(b, m, deterministic)
+}
+func (m *ConnectStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ConnectStatus.Merge(m, src)
+}
+func (m *ConnectStatus) XXX_Size() int {
+	return xxx_messageInfo_ConnectStatus.Size(m)
+}
+func (m *ConnectStatus) XXX_DiscardUnknown() {
+	xxx_messageInfo_ConnectStatus.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ConnectStatus proto.InternalMessageInfo
+
+type OperationResp struct {
+	// Return code
+	Code OperationResp_OperationReturnCode `protobuf:"varint,1,opt,name=code,proto3,enum=common.OperationResp_OperationReturnCode" json:"code,omitempty"`
+	// Additional Info
+	AdditionalInfo       string   `protobuf:"bytes,2,opt,name=additional_info,json=additionalInfo,proto3" json:"additional_info,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OperationResp) Reset()         { *m = OperationResp{} }
+func (m *OperationResp) String() string { return proto.CompactTextString(m) }
+func (*OperationResp) ProtoMessage()    {}
+func (*OperationResp) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{5}
+}
+
+func (m *OperationResp) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OperationResp.Unmarshal(m, b)
+}
+func (m *OperationResp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OperationResp.Marshal(b, m, deterministic)
+}
+func (m *OperationResp) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OperationResp.Merge(m, src)
+}
+func (m *OperationResp) XXX_Size() int {
+	return xxx_messageInfo_OperationResp.Size(m)
+}
+func (m *OperationResp) XXX_DiscardUnknown() {
+	xxx_messageInfo_OperationResp.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OperationResp proto.InternalMessageInfo
+
+func (m *OperationResp) GetCode() OperationResp_OperationReturnCode {
+	if m != nil {
+		return m.Code
+	}
+	return OperationResp_OPERATION_SUCCESS
+}
+
+func (m *OperationResp) GetAdditionalInfo() string {
+	if m != nil {
+		return m.AdditionalInfo
+	}
+	return ""
+}
+
+type ValueType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ValueType) Reset()         { *m = ValueType{} }
+func (m *ValueType) String() string { return proto.CompactTextString(m) }
+func (*ValueType) ProtoMessage()    {}
+func (*ValueType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{6}
+}
+
+func (m *ValueType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ValueType.Unmarshal(m, b)
+}
+func (m *ValueType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ValueType.Marshal(b, m, deterministic)
+}
+func (m *ValueType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ValueType.Merge(m, src)
+}
+func (m *ValueType) XXX_Size() int {
+	return xxx_messageInfo_ValueType.Size(m)
+}
+func (m *ValueType) XXX_DiscardUnknown() {
+	xxx_messageInfo_ValueType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ValueType proto.InternalMessageInfo
+
+type ValueSpecifier struct {
+	Id                   string         `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	Value                ValueType_Type `protobuf:"varint,2,opt,name=value,proto3,enum=common.ValueType_Type" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *ValueSpecifier) Reset()         { *m = ValueSpecifier{} }
+func (m *ValueSpecifier) String() string { return proto.CompactTextString(m) }
+func (*ValueSpecifier) ProtoMessage()    {}
+func (*ValueSpecifier) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{7}
+}
+
+func (m *ValueSpecifier) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ValueSpecifier.Unmarshal(m, b)
+}
+func (m *ValueSpecifier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ValueSpecifier.Marshal(b, m, deterministic)
+}
+func (m *ValueSpecifier) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ValueSpecifier.Merge(m, src)
+}
+func (m *ValueSpecifier) XXX_Size() int {
+	return xxx_messageInfo_ValueSpecifier.Size(m)
+}
+func (m *ValueSpecifier) XXX_DiscardUnknown() {
+	xxx_messageInfo_ValueSpecifier.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ValueSpecifier proto.InternalMessageInfo
+
+func (m *ValueSpecifier) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *ValueSpecifier) GetValue() ValueType_Type {
+	if m != nil {
+		return m.Value
+	}
+	return ValueType_EMPTY
+}
+
+type ReturnValues struct {
+	Set                  uint32   `protobuf:"varint,1,opt,name=Set,proto3" json:"Set,omitempty"`
+	Unsupported          uint32   `protobuf:"varint,2,opt,name=Unsupported,proto3" json:"Unsupported,omitempty"`
+	Error                uint32   `protobuf:"varint,3,opt,name=Error,proto3" json:"Error,omitempty"`
+	Distance             uint32   `protobuf:"varint,4,opt,name=Distance,proto3" json:"Distance,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ReturnValues) Reset()         { *m = ReturnValues{} }
+func (m *ReturnValues) String() string { return proto.CompactTextString(m) }
+func (*ReturnValues) ProtoMessage()    {}
+func (*ReturnValues) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c2e3fd231961e826, []int{8}
+}
+
+func (m *ReturnValues) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ReturnValues.Unmarshal(m, b)
+}
+func (m *ReturnValues) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ReturnValues.Marshal(b, m, deterministic)
+}
+func (m *ReturnValues) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ReturnValues.Merge(m, src)
+}
+func (m *ReturnValues) XXX_Size() int {
+	return xxx_messageInfo_ReturnValues.Size(m)
+}
+func (m *ReturnValues) XXX_DiscardUnknown() {
+	xxx_messageInfo_ReturnValues.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ReturnValues proto.InternalMessageInfo
+
+func (m *ReturnValues) GetSet() uint32 {
+	if m != nil {
+		return m.Set
+	}
+	return 0
+}
+
+func (m *ReturnValues) GetUnsupported() uint32 {
+	if m != nil {
+		return m.Unsupported
+	}
+	return 0
+}
+
+func (m *ReturnValues) GetError() uint32 {
+	if m != nil {
+		return m.Error
+	}
+	return 0
+}
+
+func (m *ReturnValues) GetDistance() uint32 {
+	if m != nil {
+		return m.Distance
+	}
+	return 0
+}
+
+func init() {
+	proto.RegisterEnum("common.TestModeKeys", TestModeKeys_name, TestModeKeys_value)
+	proto.RegisterEnum("common.AdminState_Types", AdminState_Types_name, AdminState_Types_value)
+	proto.RegisterEnum("common.OperStatus_Types", OperStatus_Types_name, OperStatus_Types_value)
+	proto.RegisterEnum("common.ConnectStatus_Types", ConnectStatus_Types_name, ConnectStatus_Types_value)
+	proto.RegisterEnum("common.OperationResp_OperationReturnCode", OperationResp_OperationReturnCode_name, OperationResp_OperationReturnCode_value)
+	proto.RegisterEnum("common.ValueType_Type", ValueType_Type_name, ValueType_Type_value)
+	proto.RegisterType((*ID)(nil), "common.ID")
+	proto.RegisterType((*IDs)(nil), "common.IDs")
+	proto.RegisterType((*AdminState)(nil), "common.AdminState")
+	proto.RegisterType((*OperStatus)(nil), "common.OperStatus")
+	proto.RegisterType((*ConnectStatus)(nil), "common.ConnectStatus")
+	proto.RegisterType((*OperationResp)(nil), "common.OperationResp")
+	proto.RegisterType((*ValueType)(nil), "common.ValueType")
+	proto.RegisterType((*ValueSpecifier)(nil), "common.ValueSpecifier")
+	proto.RegisterType((*ReturnValues)(nil), "common.ReturnValues")
+}
+
+func init() { proto.RegisterFile("voltha_protos/common.proto", fileDescriptor_c2e3fd231961e826) }
+
+var fileDescriptor_c2e3fd231961e826 = []byte{
+	// 635 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0x5d, 0x4f, 0xdb, 0x4a,
+	0x10, 0x8d, 0xf3, 0x05, 0x99, 0x10, 0xe3, 0xbb, 0x7c, 0x28, 0x17, 0x5d, 0xe9, 0x46, 0x7e, 0x81,
+	0x7b, 0x45, 0x13, 0x89, 0xb6, 0x8f, 0x7d, 0x30, 0xf6, 0x36, 0x5d, 0x01, 0xeb, 0x68, 0x6d, 0x07,
+	0x95, 0x17, 0xcb, 0xc4, 0x0b, 0x58, 0x22, 0x5e, 0xcb, 0xde, 0x20, 0xf1, 0xda, 0x7f, 0xd0, 0xbf,
+	0xda, 0x5f, 0x50, 0xed, 0x3a, 0x7c, 0x55, 0xbc, 0x24, 0x7b, 0xe6, 0x1c, 0xcf, 0xec, 0x99, 0xd9,
+	0x81, 0x83, 0x07, 0x71, 0x2f, 0xef, 0x92, 0xb8, 0x28, 0x85, 0x14, 0xd5, 0x64, 0x21, 0x96, 0x4b,
+	0x91, 0x8f, 0x35, 0x42, 0xdd, 0x1a, 0xd9, 0xbb, 0xd0, 0x24, 0x1e, 0x32, 0xa1, 0x99, 0xa5, 0x43,
+	0x63, 0x64, 0x1c, 0xf5, 0x58, 0x33, 0x4b, 0xed, 0x43, 0x68, 0x11, 0xaf, 0x42, 0x23, 0xe8, 0x64,
+	0x92, 0x2f, 0xab, 0xa1, 0x31, 0x6a, 0x1d, 0xf5, 0x4f, 0x60, 0xbc, 0x4e, 0x41, 0x3c, 0x56, 0x13,
+	0xf6, 0x1d, 0x80, 0x93, 0x2e, 0xb3, 0x3c, 0x90, 0x89, 0xe4, 0xf6, 0x15, 0x74, 0xc2, 0xc7, 0x82,
+	0x57, 0xa8, 0x0f, 0x1b, 0x11, 0x3d, 0xa3, 0xfe, 0x25, 0xb5, 0x1a, 0x08, 0x81, 0x39, 0x63, 0x78,
+	0xc6, 0xfc, 0x39, 0x09, 0x88, 0x4f, 0xb1, 0x67, 0x19, 0x4a, 0x80, 0xa9, 0x73, 0x7a, 0x8e, 0x3d,
+	0xab, 0x89, 0xb6, 0x60, 0xd3, 0x23, 0x41, 0x8d, 0x5a, 0x68, 0x0f, 0xfe, 0xf2, 0xfc, 0x4b, 0x7a,
+	0xee, 0x3b, 0x1e, 0xa1, 0xd3, 0x98, 0x5c, 0x38, 0x53, 0x6c, 0xb5, 0xed, 0x9f, 0x06, 0x80, 0x5f,
+	0xf0, 0x52, 0x55, 0x5a, 0x55, 0xf6, 0x0f, 0xe3, 0xdd, 0x5a, 0x26, 0x80, 0x47, 0x02, 0xd7, 0x9f,
+	0x63, 0xa6, 0xeb, 0x98, 0x00, 0x8e, 0x1b, 0x92, 0xb9, 0x13, 0x12, 0x3a, 0xb5, 0x9a, 0x4a, 0x1c,
+	0xe2, 0x40, 0x83, 0x16, 0x02, 0xe8, 0x6a, 0x12, 0x5b, 0x6d, 0x75, 0xfe, 0xea, 0x10, 0x75, 0x83,
+	0x0e, 0xda, 0x86, 0x3e, 0xc3, 0xae, 0x4f, 0x5d, 0x72, 0xae, 0x84, 0x5d, 0xb4, 0x0f, 0xe8, 0x55,
+	0x20, 0x5e, 0x0b, 0x37, 0x6c, 0x0c, 0x03, 0x57, 0xe4, 0x39, 0x5f, 0xc8, 0xf5, 0xad, 0x3e, 0xbd,
+	0x7b, 0xa9, 0x6d, 0xe8, 0x47, 0x94, 0x61, 0xc7, 0xfd, 0xa6, 0x3c, 0x5a, 0x06, 0x1a, 0x40, 0xef,
+	0x05, 0x36, 0xed, 0x5f, 0x06, 0x0c, 0x94, 0xb5, 0x44, 0x66, 0x22, 0x67, 0xbc, 0x2a, 0xd0, 0x17,
+	0x68, 0x2f, 0x44, 0xca, 0xf5, 0x44, 0xcc, 0x93, 0xff, 0x9e, 0xfa, 0xfe, 0x46, 0xf4, 0x1a, 0xc9,
+	0x55, 0x99, 0xbb, 0x22, 0xe5, 0x4c, 0x7f, 0x86, 0x0e, 0x61, 0x3b, 0x49, 0xd3, 0x4c, 0x71, 0xc9,
+	0x7d, 0x9c, 0xe5, 0x37, 0x62, 0xd8, 0xd4, 0xb3, 0x35, 0x5f, 0xc2, 0x24, 0xbf, 0x11, 0xf6, 0x23,
+	0xec, 0xbc, 0x93, 0x45, 0x8d, 0xc0, 0x9f, 0x61, 0xe6, 0x84, 0xc4, 0xa7, 0x71, 0x10, 0xb9, 0x2e,
+	0x0e, 0x02, 0xab, 0xf1, 0x36, 0xac, 0x9a, 0x10, 0x31, 0xe5, 0xe6, 0x6f, 0xd8, 0x7b, 0x09, 0x47,
+	0x34, 0x88, 0x66, 0x33, 0x9f, 0x85, 0x7a, 0xb2, 0x6f, 0x28, 0x42, 0xe3, 0x19, 0xf3, 0xa7, 0x4c,
+	0x25, 0x6b, 0xd9, 0xc7, 0xd0, 0x9b, 0x27, 0xf7, 0x2b, 0xae, 0xfa, 0x65, 0xff, 0x0b, 0x6d, 0xf5,
+	0x8f, 0x7a, 0xd0, 0xc1, 0x17, 0xb3, 0xf0, 0xbb, 0xd5, 0x58, 0x3f, 0x8a, 0xd0, 0xa1, 0x2e, 0xb6,
+	0x0c, 0x9b, 0x82, 0xa9, 0xd5, 0x41, 0xc1, 0x17, 0xd9, 0x4d, 0xc6, 0xcb, 0x3f, 0x9f, 0x2c, 0x3a,
+	0x86, 0xce, 0x83, 0x52, 0x68, 0xa7, 0xe6, 0xc9, 0xfe, 0x53, 0xcf, 0x9e, 0x8b, 0x8c, 0xd5, 0x0f,
+	0xab, 0x45, 0xb6, 0x84, 0xad, 0xda, 0xaf, 0xa6, 0x2b, 0x64, 0x41, 0x2b, 0xe0, 0x52, 0xa7, 0x1b,
+	0x30, 0x75, 0x44, 0x23, 0xe8, 0x47, 0x79, 0xb5, 0x2a, 0x0a, 0x51, 0x4a, 0x9e, 0xea, 0xac, 0x03,
+	0xf6, 0x3a, 0x84, 0x76, 0xa1, 0x83, 0xcb, 0x52, 0x94, 0xc3, 0x96, 0xe6, 0x6a, 0x80, 0x0e, 0x60,
+	0xd3, 0xcb, 0x2a, 0x99, 0xe4, 0x0b, 0x3e, 0x6c, 0x6b, 0xe2, 0x19, 0xff, 0xff, 0x0f, 0x6c, 0x85,
+	0xbc, 0x92, 0x17, 0x22, 0xe5, 0x67, 0xfc, 0xb1, 0x52, 0x1e, 0x93, 0x22, 0x8b, 0x25, 0xaf, 0xa4,
+	0xd5, 0x38, 0xc5, 0xb0, 0x23, 0xca, 0xdb, 0xb1, 0x28, 0x78, 0xbe, 0x10, 0x65, 0x3a, 0xae, 0xb7,
+	0xf7, 0x6a, 0x7c, 0x9b, 0xc9, 0xbb, 0xd5, 0xb5, 0xf2, 0x33, 0x79, 0xe2, 0x26, 0x35, 0xf7, 0x61,
+	0xbd, 0xd9, 0x0f, 0x9f, 0x27, 0xb7, 0x62, 0xbd, 0xdf, 0xd7, 0x5d, 0x1d, 0xfc, 0xf8, 0x3b, 0x00,
+	0x00, 0xff, 0xff, 0x71, 0x53, 0xdb, 0xeb, 0xfe, 0x03, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/common/meta.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/common/meta.pb.go
new file mode 100644
index 0000000..26dfeef
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/common/meta.pb.go
@@ -0,0 +1,142 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/meta.proto
+
+package common
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	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 Access int32
+
+const (
+	// read-write, stored attribute
+	Access_CONFIG Access = 0
+	// read-only field, stored with the model, covered by its hash
+	Access_READ_ONLY Access = 1
+	// A read-only attribute that is not stored in the model, not covered
+	// by its hash, its value is filled real-time upon each request.
+	Access_REAL_TIME Access = 2
+)
+
+var Access_name = map[int32]string{
+	0: "CONFIG",
+	1: "READ_ONLY",
+	2: "REAL_TIME",
+}
+
+var Access_value = map[string]int32{
+	"CONFIG":    0,
+	"READ_ONLY": 1,
+	"REAL_TIME": 2,
+}
+
+func (x Access) String() string {
+	return proto.EnumName(Access_name, int32(x))
+}
+
+func (Access) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_96b320e8a67781f3, []int{0}
+}
+
+type ChildNode struct {
+	Key                  string   `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ChildNode) Reset()         { *m = ChildNode{} }
+func (m *ChildNode) String() string { return proto.CompactTextString(m) }
+func (*ChildNode) ProtoMessage()    {}
+func (*ChildNode) Descriptor() ([]byte, []int) {
+	return fileDescriptor_96b320e8a67781f3, []int{0}
+}
+
+func (m *ChildNode) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ChildNode.Unmarshal(m, b)
+}
+func (m *ChildNode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ChildNode.Marshal(b, m, deterministic)
+}
+func (m *ChildNode) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ChildNode.Merge(m, src)
+}
+func (m *ChildNode) XXX_Size() int {
+	return xxx_messageInfo_ChildNode.Size(m)
+}
+func (m *ChildNode) XXX_DiscardUnknown() {
+	xxx_messageInfo_ChildNode.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ChildNode proto.InternalMessageInfo
+
+func (m *ChildNode) GetKey() string {
+	if m != nil {
+		return m.Key
+	}
+	return ""
+}
+
+var E_ChildNode = &proto.ExtensionDesc{
+	ExtendedType:  (*descriptor.FieldOptions)(nil),
+	ExtensionType: (*ChildNode)(nil),
+	Field:         7761772,
+	Name:          "voltha.child_node",
+	Tag:           "bytes,7761772,opt,name=child_node",
+	Filename:      "voltha_protos/meta.proto",
+}
+
+var E_Access = &proto.ExtensionDesc{
+	ExtendedType:  (*descriptor.FieldOptions)(nil),
+	ExtensionType: (*Access)(nil),
+	Field:         7761773,
+	Name:          "voltha.access",
+	Tag:           "varint,7761773,opt,name=access,enum=voltha.Access",
+	Filename:      "voltha_protos/meta.proto",
+}
+
+func init() {
+	proto.RegisterEnum("voltha.Access", Access_name, Access_value)
+	proto.RegisterType((*ChildNode)(nil), "voltha.ChildNode")
+	proto.RegisterExtension(E_ChildNode)
+	proto.RegisterExtension(E_Access)
+}
+
+func init() { proto.RegisterFile("voltha_protos/meta.proto", fileDescriptor_96b320e8a67781f3) }
+
+var fileDescriptor_96b320e8a67781f3 = []byte{
+	// 282 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0x41, 0x4b, 0x84, 0x40,
+	0x14, 0xc7, 0xb3, 0x05, 0xc1, 0x17, 0x2d, 0xe6, 0x69, 0x59, 0x58, 0x90, 0x4e, 0x4b, 0xd0, 0x0c,
+	0x18, 0x5d, 0xf6, 0xb6, 0x6d, 0xbb, 0xb5, 0x60, 0x0a, 0xd2, 0xa5, 0x2e, 0xa2, 0xe3, 0xa4, 0x92,
+	0xfa, 0xc4, 0x99, 0x5d, 0xe8, 0xa3, 0x76, 0xe9, 0x13, 0xd4, 0x77, 0x08, 0x1d, 0xa7, 0x6b, 0xb7,
+	0xff, 0x7b, 0xf3, 0x9f, 0x1f, 0x3f, 0x1e, 0xcc, 0x8e, 0x58, 0xc9, 0x22, 0x89, 0xdb, 0x0e, 0x25,
+	0x0a, 0x5a, 0x73, 0x99, 0x90, 0x21, 0x3b, 0xa6, 0x7a, 0x99, 0xbb, 0x39, 0x62, 0x5e, 0x71, 0x3a,
+	0x6c, 0xd3, 0xc3, 0x1b, 0xcd, 0xb8, 0x60, 0x5d, 0xd9, 0x4a, 0xec, 0x54, 0xf3, 0x72, 0x01, 0xd6,
+	0xa6, 0x28, 0xab, 0x2c, 0xc0, 0x8c, 0x3b, 0x36, 0x4c, 0xde, 0xf9, 0xc7, 0xcc, 0x70, 0x8d, 0xa5,
+	0x15, 0xf5, 0xf1, 0xca, 0x03, 0x73, 0xcd, 0x18, 0x17, 0xc2, 0x01, 0x30, 0x37, 0x61, 0xb0, 0xdb,
+	0x3f, 0xd8, 0x27, 0xce, 0x39, 0x58, 0xd1, 0x76, 0x7d, 0x1f, 0x87, 0x81, 0xff, 0x62, 0x1b, 0xe3,
+	0xe8, 0xc7, 0xcf, 0xfb, 0xa7, 0xad, 0x7d, 0xba, 0x8a, 0x00, 0x58, 0x8f, 0x8c, 0x9b, 0x9e, 0xb9,
+	0x20, 0xca, 0x81, 0x68, 0x07, 0xb2, 0x2b, 0x79, 0x95, 0x85, 0xad, 0x2c, 0xb1, 0x11, 0xb3, 0xef,
+	0xaf, 0xcf, 0x89, 0x6b, 0x2c, 0xcf, 0xbc, 0x0b, 0xa2, 0x9c, 0xc9, 0x9f, 0x4e, 0x64, 0x31, 0x1d,
+	0x57, 0x8f, 0x60, 0x26, 0xca, 0xe3, 0x1f, 0xde, 0x8f, 0xe2, 0x4d, 0xbd, 0xa9, 0xe6, 0x29, 0xff,
+	0x68, 0xfc, 0x7f, 0xe7, 0xc3, 0x1c, 0xbb, 0x9c, 0x60, 0xcb, 0x1b, 0x86, 0x5d, 0xa6, 0x5b, 0x0c,
+	0xeb, 0x1a, 0x9b, 0x57, 0x92, 0x97, 0xb2, 0x38, 0xa4, 0xfd, 0x48, 0x75, 0x85, 0xaa, 0xca, 0xf5,
+	0x78, 0xe6, 0xe3, 0x2d, 0xcd, 0x91, 0xaa, 0x7e, 0x6a, 0x0e, 0xcb, 0x9b, 0xdf, 0x00, 0x00, 0x00,
+	0xff, 0xff, 0x56, 0x6a, 0x8b, 0x22, 0x8b, 0x01, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/core/core.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/core/core.pb.go
new file mode 100644
index 0000000..f586730
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/core/core.pb.go
@@ -0,0 +1,1089 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/core.proto
+
+package core
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	empty "github.com/golang/protobuf/ptypes/empty"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	inter_container "github.com/opencord/voltha-protos/v5/go/inter_container"
+	voltha "github.com/opencord/voltha-protos/v5/go/voltha"
+	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
+
+// Transient State for devices
+type DeviceTransientState_Types int32
+
+const (
+	// The transient state of the device is not set
+	DeviceTransientState_NONE DeviceTransientState_Types = 0
+	// The state of the device in core is any state, i.e DELETING, DELETED, DELETE_FAILED, NONE.
+	// This state is only used for transitions.
+	DeviceTransientState_ANY DeviceTransientState_Types = 1
+	// The device is in FORCE_DELETING state
+	DeviceTransientState_FORCE_DELETING DeviceTransientState_Types = 2
+	// The device is getting deleted from adapter state
+	DeviceTransientState_DELETING_FROM_ADAPTER DeviceTransientState_Types = 3
+	// The device is deleted from adapter and is getting deleted in core.
+	DeviceTransientState_DELETING_POST_ADAPTER_RESPONSE DeviceTransientState_Types = 4
+	// State to represent that the device deletion is failed
+	DeviceTransientState_DELETE_FAILED DeviceTransientState_Types = 5
+	// State to represent that reconcile is in progress
+	DeviceTransientState_RECONCILE_IN_PROGRESS DeviceTransientState_Types = 6
+)
+
+var DeviceTransientState_Types_name = map[int32]string{
+	0: "NONE",
+	1: "ANY",
+	2: "FORCE_DELETING",
+	3: "DELETING_FROM_ADAPTER",
+	4: "DELETING_POST_ADAPTER_RESPONSE",
+	5: "DELETE_FAILED",
+	6: "RECONCILE_IN_PROGRESS",
+}
+
+var DeviceTransientState_Types_value = map[string]int32{
+	"NONE":                           0,
+	"ANY":                            1,
+	"FORCE_DELETING":                 2,
+	"DELETING_FROM_ADAPTER":          3,
+	"DELETING_POST_ADAPTER_RESPONSE": 4,
+	"DELETE_FAILED":                  5,
+	"RECONCILE_IN_PROGRESS":          6,
+}
+
+func (x DeviceTransientState_Types) String() string {
+	return proto.EnumName(DeviceTransientState_Types_name, int32(x))
+}
+
+func (DeviceTransientState_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_39634f15fb8a505e, []int{0, 0}
+}
+
+type DeviceTransientState struct {
+	TransientState       DeviceTransientState_Types `protobuf:"varint,1,opt,name=transient_state,json=transientState,proto3,enum=voltha.DeviceTransientState_Types" json:"transient_state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
+	XXX_unrecognized     []byte                     `json:"-"`
+	XXX_sizecache        int32                      `json:"-"`
+}
+
+func (m *DeviceTransientState) Reset()         { *m = DeviceTransientState{} }
+func (m *DeviceTransientState) String() string { return proto.CompactTextString(m) }
+func (*DeviceTransientState) ProtoMessage()    {}
+func (*DeviceTransientState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_39634f15fb8a505e, []int{0}
+}
+
+func (m *DeviceTransientState) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceTransientState.Unmarshal(m, b)
+}
+func (m *DeviceTransientState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceTransientState.Marshal(b, m, deterministic)
+}
+func (m *DeviceTransientState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceTransientState.Merge(m, src)
+}
+func (m *DeviceTransientState) XXX_Size() int {
+	return xxx_messageInfo_DeviceTransientState.Size(m)
+}
+func (m *DeviceTransientState) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceTransientState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceTransientState proto.InternalMessageInfo
+
+func (m *DeviceTransientState) GetTransientState() DeviceTransientState_Types {
+	if m != nil {
+		return m.TransientState
+	}
+	return DeviceTransientState_NONE
+}
+
+func init() {
+	proto.RegisterEnum("voltha.DeviceTransientState_Types", DeviceTransientState_Types_name, DeviceTransientState_Types_value)
+	proto.RegisterType((*DeviceTransientState)(nil), "voltha.DeviceTransientState")
+}
+
+func init() { proto.RegisterFile("voltha_protos/core.proto", fileDescriptor_39634f15fb8a505e) }
+
+var fileDescriptor_39634f15fb8a505e = []byte{
+	// 771 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x5d, 0x6f, 0xeb, 0x44,
+	0x10, 0x25, 0xf7, 0xa3, 0x5c, 0xa6, 0x6d, 0xe2, 0xee, 0x4d, 0xb9, 0x6d, 0x2a, 0x55, 0x28, 0x3c,
+	0xc0, 0x43, 0xeb, 0x88, 0x16, 0x2a, 0x44, 0x11, 0xc8, 0xb5, 0x9d, 0x34, 0x22, 0x4d, 0x2c, 0x3b,
+	0x08, 0xc1, 0x8b, 0xe5, 0xda, 0x53, 0xc7, 0xc2, 0xf1, 0x46, 0xbb, 0xdb, 0x40, 0x1e, 0xf9, 0x1d,
+	0xfc, 0x14, 0xfe, 0x1c, 0xb2, 0xd7, 0x4e, 0xed, 0x34, 0x96, 0x2a, 0xdd, 0xb7, 0x64, 0xce, 0x9c,
+	0x33, 0x3b, 0x73, 0x76, 0xbc, 0x70, 0xb4, 0xa4, 0xb1, 0x98, 0x79, 0xee, 0x82, 0x51, 0x41, 0x79,
+	0xcf, 0xa7, 0x0c, 0xd5, 0xec, 0x37, 0xd9, 0x91, 0x48, 0xe7, 0x24, 0xa4, 0x34, 0x8c, 0xb1, 0x97,
+	0x45, 0xef, 0x1f, 0x1f, 0x7a, 0x38, 0x5f, 0x88, 0x95, 0x4c, 0xea, 0x7c, 0x59, 0xa5, 0x47, 0x89,
+	0x40, 0xe6, 0xfa, 0x34, 0x11, 0x5e, 0x94, 0x20, 0xcb, 0x93, 0x3a, 0x9b, 0x35, 0xe6, 0x73, 0x9a,
+	0x6c, 0xc7, 0x02, 0x5c, 0x46, 0x3e, 0x6e, 0xc7, 0x66, 0xe8, 0xc5, 0x62, 0x26, 0xb1, 0xee, 0x3f,
+	0xaf, 0xa0, 0x6d, 0x64, 0xc9, 0x53, 0xe6, 0x25, 0x3c, 0xc2, 0x44, 0x38, 0xc2, 0x13, 0x48, 0x7e,
+	0x81, 0x96, 0x28, 0x22, 0x2e, 0x4f, 0x43, 0x47, 0x8d, 0x2f, 0x1a, 0x5f, 0x37, 0x2f, 0xba, 0xaa,
+	0x94, 0x53, 0xb7, 0xd1, 0xd4, 0xe9, 0x6a, 0x81, 0xdc, 0x6e, 0x8a, 0x4a, 0xb4, 0xfb, 0x6f, 0x03,
+	0xde, 0x66, 0x08, 0x79, 0x07, 0x6f, 0xc6, 0x93, 0xb1, 0xa9, 0x7c, 0x42, 0x3e, 0x85, 0xd7, 0xda,
+	0xf8, 0x77, 0xa5, 0x41, 0x08, 0x34, 0xfb, 0x13, 0x5b, 0x37, 0x5d, 0xc3, 0x1c, 0x99, 0xd3, 0xe1,
+	0x78, 0xa0, 0xbc, 0x22, 0xc7, 0x70, 0x58, 0xfc, 0x73, 0xfb, 0xf6, 0xe4, 0xce, 0xd5, 0x0c, 0xcd,
+	0x9a, 0x9a, 0xb6, 0xf2, 0x9a, 0x74, 0xe1, 0x74, 0x0d, 0x59, 0x13, 0x67, 0x5a, 0x40, 0xae, 0x6d,
+	0x3a, 0xd6, 0x64, 0xec, 0x98, 0xca, 0x1b, 0x72, 0x00, 0xfb, 0x59, 0x8e, 0xe9, 0xf6, 0xb5, 0xe1,
+	0xc8, 0x34, 0x94, 0xb7, 0xa9, 0xa2, 0x6d, 0xea, 0x93, 0xb1, 0x3e, 0x1c, 0x99, 0xee, 0x70, 0xec,
+	0x5a, 0xf6, 0x64, 0x60, 0x9b, 0x8e, 0xa3, 0xec, 0x5c, 0xfc, 0xb7, 0x0b, 0xbb, 0x3a, 0x65, 0xe8,
+	0x20, 0x4b, 0x3b, 0x22, 0x3f, 0x43, 0x6b, 0x80, 0xe2, 0x36, 0x1b, 0x53, 0x7a, 0xfe, 0x47, 0x4e,
+	0x3e, 0x57, 0xa5, 0x7b, 0x6a, 0xe1, 0x9e, 0x6a, 0xa6, 0xee, 0x75, 0xda, 0xc5, 0x30, 0x2a, 0xd9,
+	0x7d, 0x68, 0xd9, 0x18, 0x46, 0x5c, 0x20, 0xd3, 0x02, 0x6f, 0x21, 0x90, 0x91, 0x93, 0x22, 0x31,
+	0x0f, 0x48, 0x9c, 0x79, 0x22, 0xa2, 0x49, 0xa7, 0x46, 0x9d, 0x5c, 0xc1, 0x9e, 0x1c, 0xf2, 0xaf,
+	0x8b, 0x20, 0xf5, 0xa4, 0x59, 0x1d, 0x7d, 0x2d, 0xef, 0x12, 0x76, 0x2d, 0xca, 0x84, 0xce, 0xd0,
+	0x13, 0x18, 0x90, 0xbd, 0x82, 0x96, 0x06, 0x6b, 0x49, 0x3a, 0x28, 0x29, 0xce, 0x33, 0xc7, 0xf2,
+	0x82, 0x1f, 0xca, 0xcc, 0x0c, 0xe8, 0x47, 0xb1, 0x40, 0x56, 0x2b, 0xf2, 0x2d, 0x34, 0x0d, 0x8c,
+	0x51, 0xa0, 0x16, 0xc7, 0x99, 0x1a, 0x01, 0x35, 0xbf, 0xa7, 0x43, 0xa3, 0x96, 0xf5, 0x0d, 0xec,
+	0x0f, 0x50, 0xc8, 0xa6, 0x52, 0x16, 0x21, 0xe5, 0xba, 0x79, 0xc9, 0x4a, 0x17, 0xe4, 0x0c, 0x5a,
+	0xa3, 0x88, 0x97, 0x38, 0xd5, 0x4a, 0xfb, 0xe5, 0xe4, 0xd4, 0x90, 0x03, 0x99, 0x59, 0x6e, 0xee,
+	0xb8, 0x3a, 0xcd, 0x97, 0xb4, 0xa7, 0x15, 0xcb, 0x62, 0xdd, 0xe9, 0x34, 0x79, 0x88, 0xc2, 0x5c,
+	0xea, 0x60, 0x5d, 0x6e, 0x2e, 0xe3, 0xbc, 0x56, 0xe2, 0x27, 0x78, 0xaf, 0xcf, 0xa2, 0x38, 0x90,
+	0x3a, 0x06, 0x0a, 0xf4, 0x53, 0x8f, 0x3e, 0x54, 0x0f, 0x63, 0x44, 0xdc, 0xa7, 0x4b, 0x64, 0xab,
+	0xce, 0x86, 0xe7, 0xe4, 0x0a, 0x94, 0x12, 0x9f, 0x8f, 0x28, 0x17, 0x2f, 0x9a, 0xf1, 0x0f, 0xd0,
+	0x2e, 0xf3, 0xd6, 0x85, 0x5f, 0xc2, 0xfd, 0x0a, 0x3e, 0x5b, 0xfb, 0x53, 0x21, 0x6c, 0x1e, 0xee,
+	0x1a, 0x9a, 0x03, 0x14, 0xa5, 0x3a, 0x4f, 0x43, 0x2e, 0x05, 0xf3, 0x21, 0x6f, 0x92, 0xd5, 0x6c,
+	0xed, 0xca, 0x87, 0xac, 0xd4, 0x6a, 0x55, 0xd3, 0x39, 0xf9, 0x1e, 0xf6, 0x1c, 0x4c, 0x02, 0xcb,
+	0xf3, 0xff, 0x44, 0x31, 0x4c, 0x88, 0xb2, 0x36, 0x21, 0x8f, 0xd4, 0xf6, 0x73, 0x03, 0x44, 0x8a,
+	0xd8, 0xe8, 0x71, 0x9a, 0xe4, 0x26, 0xb6, 0xab, 0x05, 0x24, 0x56, 0xab, 0xf1, 0x23, 0xb4, 0xd6,
+	0x4b, 0xf1, 0xec, 0x16, 0x14, 0x40, 0x2d, 0xfb, 0x1a, 0x0e, 0x6d, 0xf4, 0x69, 0xe2, 0x47, 0x31,
+	0xd6, 0x76, 0x5c, 0x47, 0xbe, 0x83, 0xd3, 0xea, 0xa0, 0x7e, 0x8b, 0xc4, 0xcc, 0x62, 0xf4, 0xef,
+	0x95, 0x16, 0x04, 0x0c, 0x39, 0x7f, 0xfa, 0xda, 0x48, 0x5c, 0x2d, 0x83, 0xcf, 0xe6, 0x7e, 0x0e,
+	0xef, 0x06, 0x28, 0xe4, 0xa2, 0x6c, 0x5b, 0xbc, 0x8d, 0x5d, 0xba, 0xcd, 0x2f, 0x30, 0xc3, 0xe4,
+	0x23, 0xb7, 0xc9, 0x80, 0xf7, 0x92, 0x3c, 0x9c, 0x7b, 0x21, 0x1a, 0xf4, 0xaf, 0x24, 0xa6, 0x5e,
+	0x40, 0x0e, 0x0b, 0xa5, 0x4a, 0xb8, 0x4e, 0xe5, 0xe6, 0x16, 0x8e, 0x28, 0x0b, 0x55, 0xba, 0xc0,
+	0xc4, 0xa7, 0x2c, 0x28, 0xc8, 0xe9, 0x0b, 0xfc, 0xc7, 0x59, 0x18, 0x89, 0xd9, 0xe3, 0x7d, 0x3a,
+	0xd3, 0x5e, 0x91, 0xd0, 0x93, 0x09, 0xe7, 0xf9, 0x6b, 0xb8, 0xfc, 0xae, 0x17, 0xd2, 0xec, 0xbd,
+	0xbe, 0xdf, 0xc9, 0x42, 0x97, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x60, 0x7e, 0xd5, 0xfd, 0xcc,
+	0x07, 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
+
+// CoreServiceClient is the client API for CoreService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type CoreServiceClient interface {
+	//	 in coreProxy interface
+	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error)
+	RegisterAdapter(ctx context.Context, in *inter_container.AdapterRegistration, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeviceUpdate(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error)
+	PortCreated(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error)
+	PortsStateUpdate(ctx context.Context, in *inter_container.PortStateFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeleteAllPorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetDevicePort(ctx context.Context, in *inter_container.PortFilter, opts ...grpc.CallOption) (*voltha.Port, error)
+	ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Ports, error)
+	DeviceStateUpdate(ctx context.Context, in *inter_container.DeviceStateFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	DevicePMConfigUpdate(ctx context.Context, in *voltha.PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error)
+	ChildDeviceDetected(ctx context.Context, in *inter_container.DeviceDiscovery, opts ...grpc.CallOption) (*voltha.Device, error)
+	ChildDevicesLost(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	ChildDevicesDetected(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Device, error)
+	GetChildDevice(ctx context.Context, in *inter_container.ChildDeviceFilter, opts ...grpc.CallOption) (*voltha.Device, error)
+	GetChildDevices(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Devices, error)
+	SendPacketIn(ctx context.Context, in *inter_container.PacketIn, opts ...grpc.CallOption) (*empty.Empty, error)
+	DeviceReasonUpdate(ctx context.Context, in *inter_container.DeviceReason, opts ...grpc.CallOption) (*empty.Empty, error)
+	PortStateUpdate(ctx context.Context, in *inter_container.PortState, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Additional API found in the Core - unused?
+	ReconcileChildDevices(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetChildDeviceWithProxyAddress(ctx context.Context, in *voltha.Device_ProxyAddress, opts ...grpc.CallOption) (*voltha.Device, error)
+	GetPorts(ctx context.Context, in *inter_container.PortFilter, opts ...grpc.CallOption) (*voltha.Ports, error)
+	ChildrenStateUpdate(ctx context.Context, in *inter_container.DeviceStateFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	UpdateImageDownload(ctx context.Context, in *voltha.ImageDownload, opts ...grpc.CallOption) (*empty.Empty, error)
+}
+
+type coreServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewCoreServiceClient(cc *grpc.ClientConn) CoreServiceClient {
+	return &coreServiceClient{cc}
+}
+
+func (c *coreServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*voltha.HealthStatus, error) {
+	out := new(voltha.HealthStatus)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetHealthStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) RegisterAdapter(ctx context.Context, in *inter_container.AdapterRegistration, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/RegisterAdapter", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) DeviceUpdate(ctx context.Context, in *voltha.Device, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/DeviceUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) PortCreated(ctx context.Context, in *voltha.Port, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/PortCreated", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) PortsStateUpdate(ctx context.Context, in *inter_container.PortStateFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/PortsStateUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) DeleteAllPorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/DeleteAllPorts", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetDevicePort(ctx context.Context, in *inter_container.PortFilter, opts ...grpc.CallOption) (*voltha.Port, error) {
+	out := new(voltha.Port)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetDevicePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Ports, error) {
+	out := new(voltha.Ports)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ListDevicePorts", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) DeviceStateUpdate(ctx context.Context, in *inter_container.DeviceStateFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/DeviceStateUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) DevicePMConfigUpdate(ctx context.Context, in *voltha.PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/DevicePMConfigUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ChildDeviceDetected(ctx context.Context, in *inter_container.DeviceDiscovery, opts ...grpc.CallOption) (*voltha.Device, error) {
+	out := new(voltha.Device)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ChildDeviceDetected", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ChildDevicesLost(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ChildDevicesLost", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ChildDevicesDetected(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ChildDevicesDetected", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Device, error) {
+	out := new(voltha.Device)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetChildDevice(ctx context.Context, in *inter_container.ChildDeviceFilter, opts ...grpc.CallOption) (*voltha.Device, error) {
+	out := new(voltha.Device)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetChildDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetChildDevices(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*voltha.Devices, error) {
+	out := new(voltha.Devices)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetChildDevices", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) SendPacketIn(ctx context.Context, in *inter_container.PacketIn, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/SendPacketIn", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) DeviceReasonUpdate(ctx context.Context, in *inter_container.DeviceReason, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/DeviceReasonUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) PortStateUpdate(ctx context.Context, in *inter_container.PortState, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/PortStateUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ReconcileChildDevices(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ReconcileChildDevices", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetChildDeviceWithProxyAddress(ctx context.Context, in *voltha.Device_ProxyAddress, opts ...grpc.CallOption) (*voltha.Device, error) {
+	out := new(voltha.Device)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetChildDeviceWithProxyAddress", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) GetPorts(ctx context.Context, in *inter_container.PortFilter, opts ...grpc.CallOption) (*voltha.Ports, error) {
+	out := new(voltha.Ports)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/GetPorts", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) ChildrenStateUpdate(ctx context.Context, in *inter_container.DeviceStateFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/ChildrenStateUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *coreServiceClient) UpdateImageDownload(ctx context.Context, in *voltha.ImageDownload, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.CoreService/UpdateImageDownload", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// CoreServiceServer is the server API for CoreService service.
+type CoreServiceServer interface {
+	//	 in coreProxy interface
+	GetHealthStatus(context.Context, *empty.Empty) (*voltha.HealthStatus, error)
+	RegisterAdapter(context.Context, *inter_container.AdapterRegistration) (*empty.Empty, error)
+	DeviceUpdate(context.Context, *voltha.Device) (*empty.Empty, error)
+	PortCreated(context.Context, *voltha.Port) (*empty.Empty, error)
+	PortsStateUpdate(context.Context, *inter_container.PortStateFilter) (*empty.Empty, error)
+	DeleteAllPorts(context.Context, *common.ID) (*empty.Empty, error)
+	GetDevicePort(context.Context, *inter_container.PortFilter) (*voltha.Port, error)
+	ListDevicePorts(context.Context, *common.ID) (*voltha.Ports, error)
+	DeviceStateUpdate(context.Context, *inter_container.DeviceStateFilter) (*empty.Empty, error)
+	DevicePMConfigUpdate(context.Context, *voltha.PmConfigs) (*empty.Empty, error)
+	ChildDeviceDetected(context.Context, *inter_container.DeviceDiscovery) (*voltha.Device, error)
+	ChildDevicesLost(context.Context, *common.ID) (*empty.Empty, error)
+	ChildDevicesDetected(context.Context, *common.ID) (*empty.Empty, error)
+	GetDevice(context.Context, *common.ID) (*voltha.Device, error)
+	GetChildDevice(context.Context, *inter_container.ChildDeviceFilter) (*voltha.Device, error)
+	GetChildDevices(context.Context, *common.ID) (*voltha.Devices, error)
+	SendPacketIn(context.Context, *inter_container.PacketIn) (*empty.Empty, error)
+	DeviceReasonUpdate(context.Context, *inter_container.DeviceReason) (*empty.Empty, error)
+	PortStateUpdate(context.Context, *inter_container.PortState) (*empty.Empty, error)
+	// Additional API found in the Core - unused?
+	ReconcileChildDevices(context.Context, *common.ID) (*empty.Empty, error)
+	GetChildDeviceWithProxyAddress(context.Context, *voltha.Device_ProxyAddress) (*voltha.Device, error)
+	GetPorts(context.Context, *inter_container.PortFilter) (*voltha.Ports, error)
+	ChildrenStateUpdate(context.Context, *inter_container.DeviceStateFilter) (*empty.Empty, error)
+	UpdateImageDownload(context.Context, *voltha.ImageDownload) (*empty.Empty, error)
+}
+
+// UnimplementedCoreServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedCoreServiceServer struct {
+}
+
+func (*UnimplementedCoreServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*voltha.HealthStatus, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
+}
+func (*UnimplementedCoreServiceServer) RegisterAdapter(ctx context.Context, req *inter_container.AdapterRegistration) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RegisterAdapter not implemented")
+}
+func (*UnimplementedCoreServiceServer) DeviceUpdate(ctx context.Context, req *voltha.Device) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeviceUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) PortCreated(ctx context.Context, req *voltha.Port) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method PortCreated not implemented")
+}
+func (*UnimplementedCoreServiceServer) PortsStateUpdate(ctx context.Context, req *inter_container.PortStateFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method PortsStateUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) DeleteAllPorts(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteAllPorts not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetDevicePort(ctx context.Context, req *inter_container.PortFilter) (*voltha.Port, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDevicePort not implemented")
+}
+func (*UnimplementedCoreServiceServer) ListDevicePorts(ctx context.Context, req *common.ID) (*voltha.Ports, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDevicePorts not implemented")
+}
+func (*UnimplementedCoreServiceServer) DeviceStateUpdate(ctx context.Context, req *inter_container.DeviceStateFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeviceStateUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) DevicePMConfigUpdate(ctx context.Context, req *voltha.PmConfigs) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DevicePMConfigUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) ChildDeviceDetected(ctx context.Context, req *inter_container.DeviceDiscovery) (*voltha.Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChildDeviceDetected not implemented")
+}
+func (*UnimplementedCoreServiceServer) ChildDevicesLost(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChildDevicesLost not implemented")
+}
+func (*UnimplementedCoreServiceServer) ChildDevicesDetected(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChildDevicesDetected not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetDevice(ctx context.Context, req *common.ID) (*voltha.Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDevice not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetChildDevice(ctx context.Context, req *inter_container.ChildDeviceFilter) (*voltha.Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetChildDevice not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetChildDevices(ctx context.Context, req *common.ID) (*voltha.Devices, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetChildDevices not implemented")
+}
+func (*UnimplementedCoreServiceServer) SendPacketIn(ctx context.Context, req *inter_container.PacketIn) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SendPacketIn not implemented")
+}
+func (*UnimplementedCoreServiceServer) DeviceReasonUpdate(ctx context.Context, req *inter_container.DeviceReason) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeviceReasonUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) PortStateUpdate(ctx context.Context, req *inter_container.PortState) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method PortStateUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) ReconcileChildDevices(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ReconcileChildDevices not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetChildDeviceWithProxyAddress(ctx context.Context, req *voltha.Device_ProxyAddress) (*voltha.Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetChildDeviceWithProxyAddress not implemented")
+}
+func (*UnimplementedCoreServiceServer) GetPorts(ctx context.Context, req *inter_container.PortFilter) (*voltha.Ports, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetPorts not implemented")
+}
+func (*UnimplementedCoreServiceServer) ChildrenStateUpdate(ctx context.Context, req *inter_container.DeviceStateFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChildrenStateUpdate not implemented")
+}
+func (*UnimplementedCoreServiceServer) UpdateImageDownload(ctx context.Context, req *voltha.ImageDownload) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateImageDownload not implemented")
+}
+
+func RegisterCoreServiceServer(s *grpc.Server, srv CoreServiceServer) {
+	s.RegisterService(&_CoreService_serviceDesc, srv)
+}
+
+func _CoreService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetHealthStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetHealthStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_RegisterAdapter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.AdapterRegistration)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).RegisterAdapter(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/RegisterAdapter",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).RegisterAdapter(ctx, req.(*inter_container.AdapterRegistration))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_DeviceUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).DeviceUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/DeviceUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).DeviceUpdate(ctx, req.(*voltha.Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_PortCreated_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Port)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).PortCreated(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/PortCreated",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).PortCreated(ctx, req.(*voltha.Port))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_PortsStateUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PortStateFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).PortsStateUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/PortsStateUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).PortsStateUpdate(ctx, req.(*inter_container.PortStateFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_DeleteAllPorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).DeleteAllPorts(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/DeleteAllPorts",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).DeleteAllPorts(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PortFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetDevicePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetDevicePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetDevicePort(ctx, req.(*inter_container.PortFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ListDevicePorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ListDevicePorts(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ListDevicePorts",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ListDevicePorts(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_DeviceStateUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeviceStateFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).DeviceStateUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/DeviceStateUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).DeviceStateUpdate(ctx, req.(*inter_container.DeviceStateFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_DevicePMConfigUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.PmConfigs)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).DevicePMConfigUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/DevicePMConfigUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).DevicePMConfigUpdate(ctx, req.(*voltha.PmConfigs))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ChildDeviceDetected_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeviceDiscovery)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ChildDeviceDetected(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ChildDeviceDetected",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ChildDeviceDetected(ctx, req.(*inter_container.DeviceDiscovery))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ChildDevicesLost_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ChildDevicesLost(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ChildDevicesLost",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ChildDevicesLost(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ChildDevicesDetected_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ChildDevicesDetected(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ChildDevicesDetected",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ChildDevicesDetected(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetChildDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.ChildDeviceFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetChildDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetChildDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetChildDevice(ctx, req.(*inter_container.ChildDeviceFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetChildDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetChildDevices(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetChildDevices",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetChildDevices(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_SendPacketIn_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PacketIn)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).SendPacketIn(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/SendPacketIn",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).SendPacketIn(ctx, req.(*inter_container.PacketIn))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_DeviceReasonUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeviceReason)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).DeviceReasonUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/DeviceReasonUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).DeviceReasonUpdate(ctx, req.(*inter_container.DeviceReason))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_PortStateUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PortState)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).PortStateUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/PortStateUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).PortStateUpdate(ctx, req.(*inter_container.PortState))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ReconcileChildDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ReconcileChildDevices(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ReconcileChildDevices",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ReconcileChildDevices(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetChildDeviceWithProxyAddress_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.Device_ProxyAddress)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetChildDeviceWithProxyAddress(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetChildDeviceWithProxyAddress",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetChildDeviceWithProxyAddress(ctx, req.(*voltha.Device_ProxyAddress))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_GetPorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.PortFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).GetPorts(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/GetPorts",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).GetPorts(ctx, req.(*inter_container.PortFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_ChildrenStateUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(inter_container.DeviceStateFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).ChildrenStateUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/ChildrenStateUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).ChildrenStateUpdate(ctx, req.(*inter_container.DeviceStateFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _CoreService_UpdateImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(voltha.ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(CoreServiceServer).UpdateImageDownload(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.CoreService/UpdateImageDownload",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(CoreServiceServer).UpdateImageDownload(ctx, req.(*voltha.ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _CoreService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.CoreService",
+	HandlerType: (*CoreServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetHealthStatus",
+			Handler:    _CoreService_GetHealthStatus_Handler,
+		},
+		{
+			MethodName: "RegisterAdapter",
+			Handler:    _CoreService_RegisterAdapter_Handler,
+		},
+		{
+			MethodName: "DeviceUpdate",
+			Handler:    _CoreService_DeviceUpdate_Handler,
+		},
+		{
+			MethodName: "PortCreated",
+			Handler:    _CoreService_PortCreated_Handler,
+		},
+		{
+			MethodName: "PortsStateUpdate",
+			Handler:    _CoreService_PortsStateUpdate_Handler,
+		},
+		{
+			MethodName: "DeleteAllPorts",
+			Handler:    _CoreService_DeleteAllPorts_Handler,
+		},
+		{
+			MethodName: "GetDevicePort",
+			Handler:    _CoreService_GetDevicePort_Handler,
+		},
+		{
+			MethodName: "ListDevicePorts",
+			Handler:    _CoreService_ListDevicePorts_Handler,
+		},
+		{
+			MethodName: "DeviceStateUpdate",
+			Handler:    _CoreService_DeviceStateUpdate_Handler,
+		},
+		{
+			MethodName: "DevicePMConfigUpdate",
+			Handler:    _CoreService_DevicePMConfigUpdate_Handler,
+		},
+		{
+			MethodName: "ChildDeviceDetected",
+			Handler:    _CoreService_ChildDeviceDetected_Handler,
+		},
+		{
+			MethodName: "ChildDevicesLost",
+			Handler:    _CoreService_ChildDevicesLost_Handler,
+		},
+		{
+			MethodName: "ChildDevicesDetected",
+			Handler:    _CoreService_ChildDevicesDetected_Handler,
+		},
+		{
+			MethodName: "GetDevice",
+			Handler:    _CoreService_GetDevice_Handler,
+		},
+		{
+			MethodName: "GetChildDevice",
+			Handler:    _CoreService_GetChildDevice_Handler,
+		},
+		{
+			MethodName: "GetChildDevices",
+			Handler:    _CoreService_GetChildDevices_Handler,
+		},
+		{
+			MethodName: "SendPacketIn",
+			Handler:    _CoreService_SendPacketIn_Handler,
+		},
+		{
+			MethodName: "DeviceReasonUpdate",
+			Handler:    _CoreService_DeviceReasonUpdate_Handler,
+		},
+		{
+			MethodName: "PortStateUpdate",
+			Handler:    _CoreService_PortStateUpdate_Handler,
+		},
+		{
+			MethodName: "ReconcileChildDevices",
+			Handler:    _CoreService_ReconcileChildDevices_Handler,
+		},
+		{
+			MethodName: "GetChildDeviceWithProxyAddress",
+			Handler:    _CoreService_GetChildDeviceWithProxyAddress_Handler,
+		},
+		{
+			MethodName: "GetPorts",
+			Handler:    _CoreService_GetPorts_Handler,
+		},
+		{
+			MethodName: "ChildrenStateUpdate",
+			Handler:    _CoreService_ChildrenStateUpdate_Handler,
+		},
+		{
+			MethodName: "UpdateImageDownload",
+			Handler:    _CoreService_UpdateImageDownload_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/core.proto",
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/ext/config/ext_config.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/ext/config/ext_config.pb.go
new file mode 100644
index 0000000..2db2ed1
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/ext/config/ext_config.pb.go
@@ -0,0 +1,523 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/ext_config.proto
+
+package config
+
+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
+
+type OnuItuPonAlarm_AlarmID int32
+
+const (
+	OnuItuPonAlarm_RDI_ERRORS OnuItuPonAlarm_AlarmID = 0
+)
+
+var OnuItuPonAlarm_AlarmID_name = map[int32]string{
+	0: "RDI_ERRORS",
+}
+
+var OnuItuPonAlarm_AlarmID_value = map[string]int32{
+	"RDI_ERRORS": 0,
+}
+
+func (x OnuItuPonAlarm_AlarmID) String() string {
+	return proto.EnumName(OnuItuPonAlarm_AlarmID_name, int32(x))
+}
+
+func (OnuItuPonAlarm_AlarmID) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 0}
+}
+
+type OnuItuPonAlarm_AlarmReportingCondition int32
+
+const (
+	OnuItuPonAlarm_RATE_THRESHOLD  OnuItuPonAlarm_AlarmReportingCondition = 0
+	OnuItuPonAlarm_RATE_RANGE      OnuItuPonAlarm_AlarmReportingCondition = 1
+	OnuItuPonAlarm_VALUE_THRESHOLD OnuItuPonAlarm_AlarmReportingCondition = 2
+)
+
+var OnuItuPonAlarm_AlarmReportingCondition_name = map[int32]string{
+	0: "RATE_THRESHOLD",
+	1: "RATE_RANGE",
+	2: "VALUE_THRESHOLD",
+}
+
+var OnuItuPonAlarm_AlarmReportingCondition_value = map[string]int32{
+	"RATE_THRESHOLD":  0,
+	"RATE_RANGE":      1,
+	"VALUE_THRESHOLD": 2,
+}
+
+func (x OnuItuPonAlarm_AlarmReportingCondition) String() string {
+	return proto.EnumName(OnuItuPonAlarm_AlarmReportingCondition_name, int32(x))
+}
+
+func (OnuItuPonAlarm_AlarmReportingCondition) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 1}
+}
+
+type AlarmConfig struct {
+	// Types that are valid to be assigned to Config:
+	//	*AlarmConfig_OnuItuPonAlarmConfig
+	Config               isAlarmConfig_Config `protobuf_oneof:"config"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *AlarmConfig) Reset()         { *m = AlarmConfig{} }
+func (m *AlarmConfig) String() string { return proto.CompactTextString(m) }
+func (*AlarmConfig) ProtoMessage()    {}
+func (*AlarmConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{0}
+}
+
+func (m *AlarmConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmConfig.Unmarshal(m, b)
+}
+func (m *AlarmConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmConfig.Marshal(b, m, deterministic)
+}
+func (m *AlarmConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmConfig.Merge(m, src)
+}
+func (m *AlarmConfig) XXX_Size() int {
+	return xxx_messageInfo_AlarmConfig.Size(m)
+}
+func (m *AlarmConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmConfig proto.InternalMessageInfo
+
+type isAlarmConfig_Config interface {
+	isAlarmConfig_Config()
+}
+
+type AlarmConfig_OnuItuPonAlarmConfig struct {
+	OnuItuPonAlarmConfig *OnuItuPonAlarm `protobuf:"bytes,1,opt,name=onu_itu_pon_alarm_config,json=onuItuPonAlarmConfig,proto3,oneof"`
+}
+
+func (*AlarmConfig_OnuItuPonAlarmConfig) isAlarmConfig_Config() {}
+
+func (m *AlarmConfig) GetConfig() isAlarmConfig_Config {
+	if m != nil {
+		return m.Config
+	}
+	return nil
+}
+
+func (m *AlarmConfig) GetOnuItuPonAlarmConfig() *OnuItuPonAlarm {
+	if x, ok := m.GetConfig().(*AlarmConfig_OnuItuPonAlarmConfig); ok {
+		return x.OnuItuPonAlarmConfig
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*AlarmConfig) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*AlarmConfig_OnuItuPonAlarmConfig)(nil),
+	}
+}
+
+type OnuItuPonAlarm struct {
+	PonNi                   uint32                                 `protobuf:"fixed32,1,opt,name=pon_ni,json=ponNi,proto3" json:"pon_ni,omitempty"`
+	OnuId                   uint32                                 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	AlarmId                 OnuItuPonAlarm_AlarmID                 `protobuf:"varint,3,opt,name=alarm_id,json=alarmId,proto3,enum=config.OnuItuPonAlarm_AlarmID" json:"alarm_id,omitempty"`
+	AlarmReportingCondition OnuItuPonAlarm_AlarmReportingCondition `protobuf:"varint,4,opt,name=alarm_reporting_condition,json=alarmReportingCondition,proto3,enum=config.OnuItuPonAlarm_AlarmReportingCondition" json:"alarm_reporting_condition,omitempty"`
+	// Types that are valid to be assigned to Config:
+	//	*OnuItuPonAlarm_RateThresholdConfig_
+	//	*OnuItuPonAlarm_RateRangeConfig_
+	//	*OnuItuPonAlarm_ValueThresholdConfig_
+	Config               isOnuItuPonAlarm_Config `protobuf_oneof:"config"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *OnuItuPonAlarm) Reset()         { *m = OnuItuPonAlarm{} }
+func (m *OnuItuPonAlarm) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonAlarm) ProtoMessage()    {}
+func (*OnuItuPonAlarm) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1}
+}
+
+func (m *OnuItuPonAlarm) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonAlarm.Unmarshal(m, b)
+}
+func (m *OnuItuPonAlarm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonAlarm.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonAlarm) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonAlarm.Merge(m, src)
+}
+func (m *OnuItuPonAlarm) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonAlarm.Size(m)
+}
+func (m *OnuItuPonAlarm) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonAlarm.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonAlarm proto.InternalMessageInfo
+
+func (m *OnuItuPonAlarm) GetPonNi() uint32 {
+	if m != nil {
+		return m.PonNi
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm) GetAlarmId() OnuItuPonAlarm_AlarmID {
+	if m != nil {
+		return m.AlarmId
+	}
+	return OnuItuPonAlarm_RDI_ERRORS
+}
+
+func (m *OnuItuPonAlarm) GetAlarmReportingCondition() OnuItuPonAlarm_AlarmReportingCondition {
+	if m != nil {
+		return m.AlarmReportingCondition
+	}
+	return OnuItuPonAlarm_RATE_THRESHOLD
+}
+
+type isOnuItuPonAlarm_Config interface {
+	isOnuItuPonAlarm_Config()
+}
+
+type OnuItuPonAlarm_RateThresholdConfig_ struct {
+	RateThresholdConfig *OnuItuPonAlarm_RateThresholdConfig `protobuf:"bytes,5,opt,name=rate_threshold_config,json=rateThresholdConfig,proto3,oneof"`
+}
+
+type OnuItuPonAlarm_RateRangeConfig_ struct {
+	RateRangeConfig *OnuItuPonAlarm_RateRangeConfig `protobuf:"bytes,6,opt,name=rate_range_config,json=rateRangeConfig,proto3,oneof"`
+}
+
+type OnuItuPonAlarm_ValueThresholdConfig_ struct {
+	ValueThresholdConfig *OnuItuPonAlarm_ValueThresholdConfig `protobuf:"bytes,7,opt,name=value_threshold_config,json=valueThresholdConfig,proto3,oneof"`
+}
+
+func (*OnuItuPonAlarm_RateThresholdConfig_) isOnuItuPonAlarm_Config() {}
+
+func (*OnuItuPonAlarm_RateRangeConfig_) isOnuItuPonAlarm_Config() {}
+
+func (*OnuItuPonAlarm_ValueThresholdConfig_) isOnuItuPonAlarm_Config() {}
+
+func (m *OnuItuPonAlarm) GetConfig() isOnuItuPonAlarm_Config {
+	if m != nil {
+		return m.Config
+	}
+	return nil
+}
+
+func (m *OnuItuPonAlarm) GetRateThresholdConfig() *OnuItuPonAlarm_RateThresholdConfig {
+	if x, ok := m.GetConfig().(*OnuItuPonAlarm_RateThresholdConfig_); ok {
+		return x.RateThresholdConfig
+	}
+	return nil
+}
+
+func (m *OnuItuPonAlarm) GetRateRangeConfig() *OnuItuPonAlarm_RateRangeConfig {
+	if x, ok := m.GetConfig().(*OnuItuPonAlarm_RateRangeConfig_); ok {
+		return x.RateRangeConfig
+	}
+	return nil
+}
+
+func (m *OnuItuPonAlarm) GetValueThresholdConfig() *OnuItuPonAlarm_ValueThresholdConfig {
+	if x, ok := m.GetConfig().(*OnuItuPonAlarm_ValueThresholdConfig_); ok {
+		return x.ValueThresholdConfig
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OnuItuPonAlarm) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OnuItuPonAlarm_RateThresholdConfig_)(nil),
+		(*OnuItuPonAlarm_RateRangeConfig_)(nil),
+		(*OnuItuPonAlarm_ValueThresholdConfig_)(nil),
+	}
+}
+
+type OnuItuPonAlarm_SoakTime struct {
+	ActiveSoakTime       uint32   `protobuf:"fixed32,1,opt,name=active_soak_time,json=activeSoakTime,proto3" json:"active_soak_time,omitempty"`
+	ClearSoakTime        uint32   `protobuf:"fixed32,2,opt,name=clear_soak_time,json=clearSoakTime,proto3" json:"clear_soak_time,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuItuPonAlarm_SoakTime) Reset()         { *m = OnuItuPonAlarm_SoakTime{} }
+func (m *OnuItuPonAlarm_SoakTime) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonAlarm_SoakTime) ProtoMessage()    {}
+func (*OnuItuPonAlarm_SoakTime) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 0}
+}
+
+func (m *OnuItuPonAlarm_SoakTime) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Unmarshal(m, b)
+}
+func (m *OnuItuPonAlarm_SoakTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonAlarm_SoakTime) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonAlarm_SoakTime.Merge(m, src)
+}
+func (m *OnuItuPonAlarm_SoakTime) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonAlarm_SoakTime.Size(m)
+}
+func (m *OnuItuPonAlarm_SoakTime) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonAlarm_SoakTime.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonAlarm_SoakTime proto.InternalMessageInfo
+
+func (m *OnuItuPonAlarm_SoakTime) GetActiveSoakTime() uint32 {
+	if m != nil {
+		return m.ActiveSoakTime
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_SoakTime) GetClearSoakTime() uint32 {
+	if m != nil {
+		return m.ClearSoakTime
+	}
+	return 0
+}
+
+type OnuItuPonAlarm_RateThresholdConfig struct {
+	RateThresholdRising  uint64                   `protobuf:"fixed64,1,opt,name=rate_threshold_rising,json=rateThresholdRising,proto3" json:"rate_threshold_rising,omitempty"`
+	RateThresholdFalling uint64                   `protobuf:"fixed64,2,opt,name=rate_threshold_falling,json=rateThresholdFalling,proto3" json:"rate_threshold_falling,omitempty"`
+	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,3,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
+	XXX_unrecognized     []byte                   `json:"-"`
+	XXX_sizecache        int32                    `json:"-"`
+}
+
+func (m *OnuItuPonAlarm_RateThresholdConfig) Reset()         { *m = OnuItuPonAlarm_RateThresholdConfig{} }
+func (m *OnuItuPonAlarm_RateThresholdConfig) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonAlarm_RateThresholdConfig) ProtoMessage()    {}
+func (*OnuItuPonAlarm_RateThresholdConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 1}
+}
+
+func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Unmarshal(m, b)
+}
+func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Merge(m, src)
+}
+func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.Size(m)
+}
+func (m *OnuItuPonAlarm_RateThresholdConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonAlarm_RateThresholdConfig proto.InternalMessageInfo
+
+func (m *OnuItuPonAlarm_RateThresholdConfig) GetRateThresholdRising() uint64 {
+	if m != nil {
+		return m.RateThresholdRising
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_RateThresholdConfig) GetRateThresholdFalling() uint64 {
+	if m != nil {
+		return m.RateThresholdFalling
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_RateThresholdConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
+	if m != nil {
+		return m.SoakTime
+	}
+	return nil
+}
+
+type OnuItuPonAlarm_RateRangeConfig struct {
+	RateRangeLower       uint64                   `protobuf:"fixed64,1,opt,name=rate_range_lower,json=rateRangeLower,proto3" json:"rate_range_lower,omitempty"`
+	RateRangeUpper       uint64                   `protobuf:"fixed64,2,opt,name=rate_range_upper,json=rateRangeUpper,proto3" json:"rate_range_upper,omitempty"`
+	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,3,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
+	XXX_unrecognized     []byte                   `json:"-"`
+	XXX_sizecache        int32                    `json:"-"`
+}
+
+func (m *OnuItuPonAlarm_RateRangeConfig) Reset()         { *m = OnuItuPonAlarm_RateRangeConfig{} }
+func (m *OnuItuPonAlarm_RateRangeConfig) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonAlarm_RateRangeConfig) ProtoMessage()    {}
+func (*OnuItuPonAlarm_RateRangeConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 2}
+}
+
+func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Unmarshal(m, b)
+}
+func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Merge(m, src)
+}
+func (m *OnuItuPonAlarm_RateRangeConfig) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.Size(m)
+}
+func (m *OnuItuPonAlarm_RateRangeConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonAlarm_RateRangeConfig proto.InternalMessageInfo
+
+func (m *OnuItuPonAlarm_RateRangeConfig) GetRateRangeLower() uint64 {
+	if m != nil {
+		return m.RateRangeLower
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_RateRangeConfig) GetRateRangeUpper() uint64 {
+	if m != nil {
+		return m.RateRangeUpper
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_RateRangeConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
+	if m != nil {
+		return m.SoakTime
+	}
+	return nil
+}
+
+type OnuItuPonAlarm_ValueThresholdConfig struct {
+	ThresholdLimit       uint64                   `protobuf:"fixed64,1,opt,name=threshold_limit,json=thresholdLimit,proto3" json:"threshold_limit,omitempty"`
+	SoakTime             *OnuItuPonAlarm_SoakTime `protobuf:"bytes,2,opt,name=soak_time,json=soakTime,proto3" json:"soak_time,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
+	XXX_unrecognized     []byte                   `json:"-"`
+	XXX_sizecache        int32                    `json:"-"`
+}
+
+func (m *OnuItuPonAlarm_ValueThresholdConfig) Reset()         { *m = OnuItuPonAlarm_ValueThresholdConfig{} }
+func (m *OnuItuPonAlarm_ValueThresholdConfig) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonAlarm_ValueThresholdConfig) ProtoMessage()    {}
+func (*OnuItuPonAlarm_ValueThresholdConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_fb43b44b7fa3aba9, []int{1, 3}
+}
+
+func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Unmarshal(m, b)
+}
+func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Merge(m, src)
+}
+func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.Size(m)
+}
+func (m *OnuItuPonAlarm_ValueThresholdConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonAlarm_ValueThresholdConfig proto.InternalMessageInfo
+
+func (m *OnuItuPonAlarm_ValueThresholdConfig) GetThresholdLimit() uint64 {
+	if m != nil {
+		return m.ThresholdLimit
+	}
+	return 0
+}
+
+func (m *OnuItuPonAlarm_ValueThresholdConfig) GetSoakTime() *OnuItuPonAlarm_SoakTime {
+	if m != nil {
+		return m.SoakTime
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("config.OnuItuPonAlarm_AlarmID", OnuItuPonAlarm_AlarmID_name, OnuItuPonAlarm_AlarmID_value)
+	proto.RegisterEnum("config.OnuItuPonAlarm_AlarmReportingCondition", OnuItuPonAlarm_AlarmReportingCondition_name, OnuItuPonAlarm_AlarmReportingCondition_value)
+	proto.RegisterType((*AlarmConfig)(nil), "config.AlarmConfig")
+	proto.RegisterType((*OnuItuPonAlarm)(nil), "config.OnuItuPonAlarm")
+	proto.RegisterType((*OnuItuPonAlarm_SoakTime)(nil), "config.OnuItuPonAlarm.SoakTime")
+	proto.RegisterType((*OnuItuPonAlarm_RateThresholdConfig)(nil), "config.OnuItuPonAlarm.RateThresholdConfig")
+	proto.RegisterType((*OnuItuPonAlarm_RateRangeConfig)(nil), "config.OnuItuPonAlarm.RateRangeConfig")
+	proto.RegisterType((*OnuItuPonAlarm_ValueThresholdConfig)(nil), "config.OnuItuPonAlarm.ValueThresholdConfig")
+}
+
+func init() { proto.RegisterFile("voltha_protos/ext_config.proto", fileDescriptor_fb43b44b7fa3aba9) }
+
+var fileDescriptor_fb43b44b7fa3aba9 = []byte{
+	// 610 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xd1, 0x6e, 0x12, 0x41,
+	0x14, 0x05, 0x6a, 0x81, 0xde, 0x46, 0xc0, 0x81, 0xb6, 0xb4, 0x0f, 0xb5, 0xe9, 0x43, 0x6d, 0x34,
+	0x2e, 0x09, 0xea, 0x83, 0x89, 0x2f, 0xb4, 0x45, 0x21, 0x21, 0x6d, 0x33, 0xa5, 0x7d, 0x30, 0x26,
+	0xeb, 0x94, 0x9d, 0x2e, 0x63, 0x97, 0x99, 0xcd, 0x30, 0x8b, 0xbe, 0xf8, 0x35, 0x7e, 0x87, 0x7e,
+	0x9b, 0x99, 0x99, 0x5d, 0x28, 0xb0, 0x34, 0x31, 0xbe, 0x90, 0xec, 0xb9, 0xf7, 0x9c, 0x7b, 0x38,
+	0x77, 0x66, 0x60, 0x7f, 0x22, 0x02, 0x35, 0x24, 0x6e, 0x28, 0x85, 0x12, 0xe3, 0x06, 0xfd, 0xa1,
+	0xdc, 0x81, 0xe0, 0x77, 0xcc, 0x77, 0x0c, 0x82, 0xf2, 0xf6, 0xeb, 0x90, 0xc1, 0x66, 0x2b, 0x20,
+	0x72, 0x74, 0x6a, 0x3e, 0xd1, 0x25, 0xd4, 0x05, 0x8f, 0x5c, 0xa6, 0x22, 0x37, 0x14, 0xdc, 0x25,
+	0xba, 0x14, 0x13, 0xeb, 0xd9, 0x83, 0xec, 0xf1, 0x66, 0x73, 0xdb, 0x89, 0x75, 0x2e, 0x78, 0xd4,
+	0x55, 0xd1, 0xa5, 0xe0, 0x86, 0xdf, 0xc9, 0xe0, 0x9a, 0x98, 0x43, 0xac, 0xe2, 0x49, 0x11, 0x92,
+	0x51, 0x7f, 0x36, 0xa0, 0x34, 0x4f, 0x42, 0x5b, 0x90, 0xd7, 0x63, 0x38, 0x33, 0xe2, 0x05, 0xbc,
+	0x1e, 0x0a, 0x7e, 0xce, 0x34, 0x6c, 0x5c, 0x78, 0xf5, 0x9c, 0x85, 0xb5, 0xb2, 0x87, 0xde, 0x43,
+	0xd1, 0x1a, 0x62, 0x5e, 0x7d, 0xed, 0x20, 0x7b, 0x5c, 0x6a, 0xee, 0xa7, 0x9b, 0x71, 0xcc, 0x6f,
+	0xf7, 0x0c, 0x17, 0x4c, 0x7f, 0xd7, 0x43, 0xdf, 0x60, 0xd7, 0x52, 0x25, 0x0d, 0x85, 0x54, 0x8c,
+	0xfb, 0xfa, 0x5f, 0x79, 0x4c, 0x31, 0xc1, 0xeb, 0x4f, 0x8c, 0x96, 0xf3, 0x98, 0x16, 0x4e, 0x68,
+	0xa7, 0x09, 0x0b, 0xef, 0x90, 0xf4, 0x02, 0xfa, 0x0a, 0x5b, 0x92, 0x28, 0xea, 0xaa, 0xa1, 0xa4,
+	0xe3, 0xa1, 0x08, 0xbc, 0x24, 0xc0, 0x75, 0x13, 0xe0, 0xcb, 0x15, 0x73, 0x30, 0x51, 0xb4, 0x9f,
+	0x50, 0x6c, 0x78, 0x9d, 0x0c, 0xae, 0xca, 0x65, 0x18, 0xf5, 0xe1, 0x99, 0x99, 0x20, 0x09, 0xf7,
+	0x69, 0xa2, 0x9e, 0x37, 0xea, 0x47, 0x8f, 0xa8, 0x63, 0xdd, 0x3e, 0x55, 0x2e, 0xcb, 0x79, 0x08,
+	0x0d, 0x60, 0x7b, 0x42, 0x82, 0x28, 0xc5, 0x78, 0xc1, 0x48, 0xbf, 0x5a, 0x21, 0x7d, 0xa3, 0x49,
+	0xcb, 0xce, 0x6b, 0x93, 0x14, 0x7c, 0xef, 0x0b, 0x14, 0xaf, 0x04, 0xb9, 0xef, 0xb3, 0x11, 0x45,
+	0xc7, 0x50, 0x21, 0x03, 0xc5, 0x26, 0xd4, 0x1d, 0x0b, 0x72, 0xef, 0x2a, 0x36, 0xa2, 0xf1, 0x39,
+	0x28, 0x59, 0x7c, 0xda, 0x79, 0x04, 0xe5, 0x41, 0x40, 0x89, 0x7c, 0xd0, 0x68, 0x4f, 0xc6, 0x53,
+	0x03, 0x27, 0x7d, 0x7b, 0xbf, 0xb3, 0x50, 0x4d, 0xc9, 0x11, 0x35, 0x97, 0x56, 0x22, 0xd9, 0x98,
+	0x71, 0x7b, 0xa6, 0xf3, 0x0b, 0x21, 0x63, 0x53, 0x42, 0x6f, 0x61, 0x7b, 0x81, 0x73, 0x47, 0x82,
+	0x40, 0x93, 0x72, 0x86, 0x54, 0x9b, 0x23, 0x7d, 0xb4, 0x35, 0xf4, 0x01, 0x36, 0x66, 0x1e, 0xd7,
+	0x4c, 0x6e, 0xcf, 0x57, 0xe4, 0x96, 0xb8, 0xc6, 0xc5, 0x71, 0xe2, 0xff, 0x57, 0x16, 0xca, 0x0b,
+	0x9b, 0xd2, 0x29, 0x3d, 0x58, 0x76, 0x20, 0xbe, 0x53, 0x19, 0xdb, 0x2e, 0x4d, 0x37, 0xd8, 0xd3,
+	0xe8, 0x42, 0x67, 0x14, 0x86, 0x54, 0xc6, 0x5e, 0x67, 0x9d, 0xd7, 0x1a, 0xfd, 0x4f, 0x97, 0x3f,
+	0xa1, 0x96, 0xb6, 0x73, 0xf4, 0x02, 0xca, 0xb3, 0xb0, 0x02, 0x36, 0x62, 0x2a, 0x31, 0x3a, 0x85,
+	0x7b, 0x1a, 0x9d, 0x1f, 0x9f, 0xfb, 0xc7, 0xf1, 0x87, 0xbb, 0x50, 0x88, 0xef, 0x37, 0x2a, 0x01,
+	0xe0, 0xb3, 0xae, 0xdb, 0xc6, 0xf8, 0x02, 0x5f, 0x55, 0x32, 0x87, 0x18, 0x76, 0x56, 0x5c, 0x57,
+	0x84, 0xa0, 0x84, 0x5b, 0xfd, 0xb6, 0xdb, 0xef, 0xe0, 0xf6, 0x55, 0xe7, 0xa2, 0x77, 0x56, 0xc9,
+	0x18, 0xba, 0xc6, 0x70, 0xeb, 0xfc, 0x53, 0xbb, 0x92, 0x45, 0x55, 0x28, 0xdf, 0xb4, 0x7a, 0xd7,
+	0x0f, 0x9b, 0x72, 0xb3, 0x07, 0xec, 0xa4, 0x0b, 0x55, 0x21, 0x7d, 0x47, 0x84, 0x94, 0x0f, 0x84,
+	0xf4, 0x1c, 0xfb, 0xc4, 0x7e, 0x6e, 0xfa, 0x4c, 0x0d, 0xa3, 0x5b, 0x67, 0x20, 0x46, 0x8d, 0xa4,
+	0xd6, 0xb0, 0xb5, 0xd7, 0xf1, 0xf3, 0x3b, 0x79, 0xd7, 0xf0, 0x85, 0x7e, 0x84, 0x1b, 0x56, 0xea,
+	0x36, 0x6f, 0x0a, 0x6f, 0xfe, 0x06, 0x00, 0x00, 0xff, 0xff, 0x36, 0x9d, 0x64, 0x93, 0xa7, 0x05,
+	0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/extension/extensions.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/extension/extensions.pb.go
new file mode 100644
index 0000000..16fe305
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/extension/extensions.pb.go
@@ -0,0 +1,3313 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/extensions.proto
+
+package extension
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	empty "github.com/golang/protobuf/ptypes/empty"
+	config "github.com/opencord/voltha-protos/v5/go/ext/config"
+	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
+
+// AlarmConfig from public import voltha_protos/ext_config.proto
+type AlarmConfig = config.AlarmConfig
+type AlarmConfig_OnuItuPonAlarmConfig = config.AlarmConfig_OnuItuPonAlarmConfig
+
+// OnuItuPonAlarm from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm = config.OnuItuPonAlarm
+type OnuItuPonAlarm_RateThresholdConfig_ = config.OnuItuPonAlarm_RateThresholdConfig_
+type OnuItuPonAlarm_RateRangeConfig_ = config.OnuItuPonAlarm_RateRangeConfig_
+type OnuItuPonAlarm_ValueThresholdConfig_ = config.OnuItuPonAlarm_ValueThresholdConfig_
+
+// OnuItuPonAlarm_SoakTime from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_SoakTime = config.OnuItuPonAlarm_SoakTime
+
+// OnuItuPonAlarm_RateThresholdConfig from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_RateThresholdConfig = config.OnuItuPonAlarm_RateThresholdConfig
+
+// OnuItuPonAlarm_RateRangeConfig from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_RateRangeConfig = config.OnuItuPonAlarm_RateRangeConfig
+
+// OnuItuPonAlarm_ValueThresholdConfig from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_ValueThresholdConfig = config.OnuItuPonAlarm_ValueThresholdConfig
+
+// OnuItuPonAlarm_AlarmID from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_AlarmID = config.OnuItuPonAlarm_AlarmID
+
+var OnuItuPonAlarm_AlarmID_name = config.OnuItuPonAlarm_AlarmID_name
+var OnuItuPonAlarm_AlarmID_value = config.OnuItuPonAlarm_AlarmID_value
+
+const OnuItuPonAlarm_RDI_ERRORS = OnuItuPonAlarm_AlarmID(config.OnuItuPonAlarm_RDI_ERRORS)
+
+// OnuItuPonAlarm_AlarmReportingCondition from public import voltha_protos/ext_config.proto
+type OnuItuPonAlarm_AlarmReportingCondition = config.OnuItuPonAlarm_AlarmReportingCondition
+
+var OnuItuPonAlarm_AlarmReportingCondition_name = config.OnuItuPonAlarm_AlarmReportingCondition_name
+var OnuItuPonAlarm_AlarmReportingCondition_value = config.OnuItuPonAlarm_AlarmReportingCondition_value
+
+const OnuItuPonAlarm_RATE_THRESHOLD = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_RATE_THRESHOLD)
+const OnuItuPonAlarm_RATE_RANGE = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_RATE_RANGE)
+const OnuItuPonAlarm_VALUE_THRESHOLD = OnuItuPonAlarm_AlarmReportingCondition(config.OnuItuPonAlarm_VALUE_THRESHOLD)
+
+type GetOnuUniInfoResponse_ConfigurationInd int32
+
+const (
+	GetOnuUniInfoResponse_UNKOWN               GetOnuUniInfoResponse_ConfigurationInd = 0
+	GetOnuUniInfoResponse_TEN_BASE_T_FDX       GetOnuUniInfoResponse_ConfigurationInd = 1
+	GetOnuUniInfoResponse_HUNDRED_BASE_T_FDX   GetOnuUniInfoResponse_ConfigurationInd = 2
+	GetOnuUniInfoResponse_GIGABIT_ETHERNET_FDX GetOnuUniInfoResponse_ConfigurationInd = 3
+	GetOnuUniInfoResponse_TEN_G_ETHERNET_FDX   GetOnuUniInfoResponse_ConfigurationInd = 4
+	GetOnuUniInfoResponse_TEN_BASE_T_HDX       GetOnuUniInfoResponse_ConfigurationInd = 5
+	GetOnuUniInfoResponse_HUNDRED_BASE_T_HDX   GetOnuUniInfoResponse_ConfigurationInd = 6
+	GetOnuUniInfoResponse_GIGABIT_ETHERNET_HDX GetOnuUniInfoResponse_ConfigurationInd = 7
+)
+
+var GetOnuUniInfoResponse_ConfigurationInd_name = map[int32]string{
+	0: "UNKOWN",
+	1: "TEN_BASE_T_FDX",
+	2: "HUNDRED_BASE_T_FDX",
+	3: "GIGABIT_ETHERNET_FDX",
+	4: "TEN_G_ETHERNET_FDX",
+	5: "TEN_BASE_T_HDX",
+	6: "HUNDRED_BASE_T_HDX",
+	7: "GIGABIT_ETHERNET_HDX",
+}
+
+var GetOnuUniInfoResponse_ConfigurationInd_value = map[string]int32{
+	"UNKOWN":               0,
+	"TEN_BASE_T_FDX":       1,
+	"HUNDRED_BASE_T_FDX":   2,
+	"GIGABIT_ETHERNET_FDX": 3,
+	"TEN_G_ETHERNET_FDX":   4,
+	"TEN_BASE_T_HDX":       5,
+	"HUNDRED_BASE_T_HDX":   6,
+	"GIGABIT_ETHERNET_HDX": 7,
+}
+
+func (x GetOnuUniInfoResponse_ConfigurationInd) String() string {
+	return proto.EnumName(GetOnuUniInfoResponse_ConfigurationInd_name, int32(x))
+}
+
+func (GetOnuUniInfoResponse_ConfigurationInd) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{3, 0}
+}
+
+type GetOnuUniInfoResponse_AdministrativeState int32
+
+const (
+	GetOnuUniInfoResponse_ADMSTATE_UNDEFINED GetOnuUniInfoResponse_AdministrativeState = 0
+	GetOnuUniInfoResponse_LOCKED             GetOnuUniInfoResponse_AdministrativeState = 1
+	GetOnuUniInfoResponse_UNLOCKED           GetOnuUniInfoResponse_AdministrativeState = 2
+)
+
+var GetOnuUniInfoResponse_AdministrativeState_name = map[int32]string{
+	0: "ADMSTATE_UNDEFINED",
+	1: "LOCKED",
+	2: "UNLOCKED",
+}
+
+var GetOnuUniInfoResponse_AdministrativeState_value = map[string]int32{
+	"ADMSTATE_UNDEFINED": 0,
+	"LOCKED":             1,
+	"UNLOCKED":           2,
+}
+
+func (x GetOnuUniInfoResponse_AdministrativeState) String() string {
+	return proto.EnumName(GetOnuUniInfoResponse_AdministrativeState_name, int32(x))
+}
+
+func (GetOnuUniInfoResponse_AdministrativeState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{3, 1}
+}
+
+type GetOnuUniInfoResponse_OperationalState int32
+
+const (
+	GetOnuUniInfoResponse_OPERSTATE_UNDEFINED GetOnuUniInfoResponse_OperationalState = 0
+	GetOnuUniInfoResponse_ENABLED             GetOnuUniInfoResponse_OperationalState = 1
+	GetOnuUniInfoResponse_DISABLED            GetOnuUniInfoResponse_OperationalState = 2
+)
+
+var GetOnuUniInfoResponse_OperationalState_name = map[int32]string{
+	0: "OPERSTATE_UNDEFINED",
+	1: "ENABLED",
+	2: "DISABLED",
+}
+
+var GetOnuUniInfoResponse_OperationalState_value = map[string]int32{
+	"OPERSTATE_UNDEFINED": 0,
+	"ENABLED":             1,
+	"DISABLED":            2,
+}
+
+func (x GetOnuUniInfoResponse_OperationalState) String() string {
+	return proto.EnumName(GetOnuUniInfoResponse_OperationalState_name, int32(x))
+}
+
+func (GetOnuUniInfoResponse_OperationalState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{3, 2}
+}
+
+type GetOltPortCounters_PortType int32
+
+const (
+	GetOltPortCounters_Port_UNKNOWN      GetOltPortCounters_PortType = 0
+	GetOltPortCounters_Port_ETHERNET_NNI GetOltPortCounters_PortType = 1
+	GetOltPortCounters_Port_PON_OLT      GetOltPortCounters_PortType = 2
+)
+
+var GetOltPortCounters_PortType_name = map[int32]string{
+	0: "Port_UNKNOWN",
+	1: "Port_ETHERNET_NNI",
+	2: "Port_PON_OLT",
+}
+
+var GetOltPortCounters_PortType_value = map[string]int32{
+	"Port_UNKNOWN":      0,
+	"Port_ETHERNET_NNI": 1,
+	"Port_PON_OLT":      2,
+}
+
+func (x GetOltPortCounters_PortType) String() string {
+	return proto.EnumName(GetOltPortCounters_PortType_name, int32(x))
+}
+
+func (GetOltPortCounters_PortType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{4, 0}
+}
+
+type GetOnuEthernetBridgePortHistory_Direction int32
+
+const (
+	GetOnuEthernetBridgePortHistory_UNDEFINED  GetOnuEthernetBridgePortHistory_Direction = 0
+	GetOnuEthernetBridgePortHistory_UPSTREAM   GetOnuEthernetBridgePortHistory_Direction = 1
+	GetOnuEthernetBridgePortHistory_DOWNSTREAM GetOnuEthernetBridgePortHistory_Direction = 2
+)
+
+var GetOnuEthernetBridgePortHistory_Direction_name = map[int32]string{
+	0: "UNDEFINED",
+	1: "UPSTREAM",
+	2: "DOWNSTREAM",
+}
+
+var GetOnuEthernetBridgePortHistory_Direction_value = map[string]int32{
+	"UNDEFINED":  0,
+	"UPSTREAM":   1,
+	"DOWNSTREAM": 2,
+}
+
+func (x GetOnuEthernetBridgePortHistory_Direction) String() string {
+	return proto.EnumName(GetOnuEthernetBridgePortHistory_Direction_name, int32(x))
+}
+
+func (GetOnuEthernetBridgePortHistory_Direction) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{8, 0}
+}
+
+type GetValueResponse_Status int32
+
+const (
+	GetValueResponse_STATUS_UNDEFINED GetValueResponse_Status = 0
+	GetValueResponse_OK               GetValueResponse_Status = 1
+	GetValueResponse_ERROR            GetValueResponse_Status = 2
+)
+
+var GetValueResponse_Status_name = map[int32]string{
+	0: "STATUS_UNDEFINED",
+	1: "OK",
+	2: "ERROR",
+}
+
+var GetValueResponse_Status_value = map[string]int32{
+	"STATUS_UNDEFINED": 0,
+	"OK":               1,
+	"ERROR":            2,
+}
+
+func (x GetValueResponse_Status) String() string {
+	return proto.EnumName(GetValueResponse_Status_name, int32(x))
+}
+
+func (GetValueResponse_Status) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{20, 0}
+}
+
+type GetValueResponse_ErrorReason int32
+
+const (
+	GetValueResponse_REASON_UNDEFINED  GetValueResponse_ErrorReason = 0
+	GetValueResponse_UNSUPPORTED       GetValueResponse_ErrorReason = 1
+	GetValueResponse_INVALID_DEVICE_ID GetValueResponse_ErrorReason = 2
+	GetValueResponse_INVALID_PORT_TYPE GetValueResponse_ErrorReason = 3
+	GetValueResponse_TIMEOUT           GetValueResponse_ErrorReason = 4
+	GetValueResponse_INVALID_REQ_TYPE  GetValueResponse_ErrorReason = 5
+	GetValueResponse_INTERNAL_ERROR    GetValueResponse_ErrorReason = 6
+	GetValueResponse_INVALID_DEVICE    GetValueResponse_ErrorReason = 7
+)
+
+var GetValueResponse_ErrorReason_name = map[int32]string{
+	0: "REASON_UNDEFINED",
+	1: "UNSUPPORTED",
+	2: "INVALID_DEVICE_ID",
+	3: "INVALID_PORT_TYPE",
+	4: "TIMEOUT",
+	5: "INVALID_REQ_TYPE",
+	6: "INTERNAL_ERROR",
+	7: "INVALID_DEVICE",
+}
+
+var GetValueResponse_ErrorReason_value = map[string]int32{
+	"REASON_UNDEFINED":  0,
+	"UNSUPPORTED":       1,
+	"INVALID_DEVICE_ID": 2,
+	"INVALID_PORT_TYPE": 3,
+	"TIMEOUT":           4,
+	"INVALID_REQ_TYPE":  5,
+	"INTERNAL_ERROR":    6,
+	"INVALID_DEVICE":    7,
+}
+
+func (x GetValueResponse_ErrorReason) String() string {
+	return proto.EnumName(GetValueResponse_ErrorReason_name, int32(x))
+}
+
+func (GetValueResponse_ErrorReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{20, 1}
+}
+
+type SetValueResponse_Status int32
+
+const (
+	SetValueResponse_STATUS_UNDEFINED SetValueResponse_Status = 0
+	SetValueResponse_OK               SetValueResponse_Status = 1
+	SetValueResponse_ERROR            SetValueResponse_Status = 2
+)
+
+var SetValueResponse_Status_name = map[int32]string{
+	0: "STATUS_UNDEFINED",
+	1: "OK",
+	2: "ERROR",
+}
+
+var SetValueResponse_Status_value = map[string]int32{
+	"STATUS_UNDEFINED": 0,
+	"OK":               1,
+	"ERROR":            2,
+}
+
+func (x SetValueResponse_Status) String() string {
+	return proto.EnumName(SetValueResponse_Status_name, int32(x))
+}
+
+func (SetValueResponse_Status) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{22, 0}
+}
+
+type SetValueResponse_ErrorReason int32
+
+const (
+	SetValueResponse_REASON_UNDEFINED SetValueResponse_ErrorReason = 0
+	SetValueResponse_UNSUPPORTED      SetValueResponse_ErrorReason = 1
+)
+
+var SetValueResponse_ErrorReason_name = map[int32]string{
+	0: "REASON_UNDEFINED",
+	1: "UNSUPPORTED",
+}
+
+var SetValueResponse_ErrorReason_value = map[string]int32{
+	"REASON_UNDEFINED": 0,
+	"UNSUPPORTED":      1,
+}
+
+func (x SetValueResponse_ErrorReason) String() string {
+	return proto.EnumName(SetValueResponse_ErrorReason_name, int32(x))
+}
+
+func (SetValueResponse_ErrorReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{22, 1}
+}
+
+type GetDistanceRequest struct {
+	OnuDeviceId          string   `protobuf:"bytes,1,opt,name=onuDeviceId,proto3" json:"onuDeviceId,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetDistanceRequest) Reset()         { *m = GetDistanceRequest{} }
+func (m *GetDistanceRequest) String() string { return proto.CompactTextString(m) }
+func (*GetDistanceRequest) ProtoMessage()    {}
+func (*GetDistanceRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{0}
+}
+
+func (m *GetDistanceRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetDistanceRequest.Unmarshal(m, b)
+}
+func (m *GetDistanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetDistanceRequest.Marshal(b, m, deterministic)
+}
+func (m *GetDistanceRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetDistanceRequest.Merge(m, src)
+}
+func (m *GetDistanceRequest) XXX_Size() int {
+	return xxx_messageInfo_GetDistanceRequest.Size(m)
+}
+func (m *GetDistanceRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetDistanceRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetDistanceRequest proto.InternalMessageInfo
+
+func (m *GetDistanceRequest) GetOnuDeviceId() string {
+	if m != nil {
+		return m.OnuDeviceId
+	}
+	return ""
+}
+
+type GetDistanceResponse struct {
+	Distance             uint32   `protobuf:"varint,1,opt,name=distance,proto3" json:"distance,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetDistanceResponse) Reset()         { *m = GetDistanceResponse{} }
+func (m *GetDistanceResponse) String() string { return proto.CompactTextString(m) }
+func (*GetDistanceResponse) ProtoMessage()    {}
+func (*GetDistanceResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{1}
+}
+
+func (m *GetDistanceResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetDistanceResponse.Unmarshal(m, b)
+}
+func (m *GetDistanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetDistanceResponse.Marshal(b, m, deterministic)
+}
+func (m *GetDistanceResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetDistanceResponse.Merge(m, src)
+}
+func (m *GetDistanceResponse) XXX_Size() int {
+	return xxx_messageInfo_GetDistanceResponse.Size(m)
+}
+func (m *GetDistanceResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetDistanceResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetDistanceResponse proto.InternalMessageInfo
+
+func (m *GetDistanceResponse) GetDistance() uint32 {
+	if m != nil {
+		return m.Distance
+	}
+	return 0
+}
+
+type GetOnuUniInfoRequest struct {
+	UniIndex             uint32   `protobuf:"varint,1,opt,name=uniIndex,proto3" json:"uniIndex,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetOnuUniInfoRequest) Reset()         { *m = GetOnuUniInfoRequest{} }
+func (m *GetOnuUniInfoRequest) String() string { return proto.CompactTextString(m) }
+func (*GetOnuUniInfoRequest) ProtoMessage()    {}
+func (*GetOnuUniInfoRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{2}
+}
+
+func (m *GetOnuUniInfoRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuUniInfoRequest.Unmarshal(m, b)
+}
+func (m *GetOnuUniInfoRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuUniInfoRequest.Marshal(b, m, deterministic)
+}
+func (m *GetOnuUniInfoRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuUniInfoRequest.Merge(m, src)
+}
+func (m *GetOnuUniInfoRequest) XXX_Size() int {
+	return xxx_messageInfo_GetOnuUniInfoRequest.Size(m)
+}
+func (m *GetOnuUniInfoRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuUniInfoRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuUniInfoRequest proto.InternalMessageInfo
+
+func (m *GetOnuUniInfoRequest) GetUniIndex() uint32 {
+	if m != nil {
+		return m.UniIndex
+	}
+	return 0
+}
+
+type GetOnuUniInfoResponse struct {
+	AdmState             GetOnuUniInfoResponse_AdministrativeState `protobuf:"varint,1,opt,name=admState,proto3,enum=extension.GetOnuUniInfoResponse_AdministrativeState" json:"admState,omitempty"`
+	OperState            GetOnuUniInfoResponse_OperationalState    `protobuf:"varint,2,opt,name=operState,proto3,enum=extension.GetOnuUniInfoResponse_OperationalState" json:"operState,omitempty"`
+	ConfigInd            GetOnuUniInfoResponse_ConfigurationInd    `protobuf:"varint,3,opt,name=configInd,proto3,enum=extension.GetOnuUniInfoResponse_ConfigurationInd" json:"configInd,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                  `json:"-"`
+	XXX_unrecognized     []byte                                    `json:"-"`
+	XXX_sizecache        int32                                     `json:"-"`
+}
+
+func (m *GetOnuUniInfoResponse) Reset()         { *m = GetOnuUniInfoResponse{} }
+func (m *GetOnuUniInfoResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOnuUniInfoResponse) ProtoMessage()    {}
+func (*GetOnuUniInfoResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{3}
+}
+
+func (m *GetOnuUniInfoResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuUniInfoResponse.Unmarshal(m, b)
+}
+func (m *GetOnuUniInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuUniInfoResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOnuUniInfoResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuUniInfoResponse.Merge(m, src)
+}
+func (m *GetOnuUniInfoResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOnuUniInfoResponse.Size(m)
+}
+func (m *GetOnuUniInfoResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuUniInfoResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuUniInfoResponse proto.InternalMessageInfo
+
+func (m *GetOnuUniInfoResponse) GetAdmState() GetOnuUniInfoResponse_AdministrativeState {
+	if m != nil {
+		return m.AdmState
+	}
+	return GetOnuUniInfoResponse_ADMSTATE_UNDEFINED
+}
+
+func (m *GetOnuUniInfoResponse) GetOperState() GetOnuUniInfoResponse_OperationalState {
+	if m != nil {
+		return m.OperState
+	}
+	return GetOnuUniInfoResponse_OPERSTATE_UNDEFINED
+}
+
+func (m *GetOnuUniInfoResponse) GetConfigInd() GetOnuUniInfoResponse_ConfigurationInd {
+	if m != nil {
+		return m.ConfigInd
+	}
+	return GetOnuUniInfoResponse_UNKOWN
+}
+
+type GetOltPortCounters struct {
+	PortNo               uint32                      `protobuf:"varint,1,opt,name=portNo,proto3" json:"portNo,omitempty"`
+	PortType             GetOltPortCounters_PortType `protobuf:"varint,2,opt,name=portType,proto3,enum=extension.GetOltPortCounters_PortType" json:"portType,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
+	XXX_unrecognized     []byte                      `json:"-"`
+	XXX_sizecache        int32                       `json:"-"`
+}
+
+func (m *GetOltPortCounters) Reset()         { *m = GetOltPortCounters{} }
+func (m *GetOltPortCounters) String() string { return proto.CompactTextString(m) }
+func (*GetOltPortCounters) ProtoMessage()    {}
+func (*GetOltPortCounters) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{4}
+}
+
+func (m *GetOltPortCounters) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOltPortCounters.Unmarshal(m, b)
+}
+func (m *GetOltPortCounters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOltPortCounters.Marshal(b, m, deterministic)
+}
+func (m *GetOltPortCounters) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOltPortCounters.Merge(m, src)
+}
+func (m *GetOltPortCounters) XXX_Size() int {
+	return xxx_messageInfo_GetOltPortCounters.Size(m)
+}
+func (m *GetOltPortCounters) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOltPortCounters.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOltPortCounters proto.InternalMessageInfo
+
+func (m *GetOltPortCounters) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *GetOltPortCounters) GetPortType() GetOltPortCounters_PortType {
+	if m != nil {
+		return m.PortType
+	}
+	return GetOltPortCounters_Port_UNKNOWN
+}
+
+type GetOltPortCountersResponse struct {
+	TxBytes              uint64   `protobuf:"varint,1,opt,name=txBytes,proto3" json:"txBytes,omitempty"`
+	RxBytes              uint64   `protobuf:"varint,2,opt,name=rxBytes,proto3" json:"rxBytes,omitempty"`
+	TxPackets            uint64   `protobuf:"varint,3,opt,name=txPackets,proto3" json:"txPackets,omitempty"`
+	RxPackets            uint64   `protobuf:"varint,4,opt,name=rxPackets,proto3" json:"rxPackets,omitempty"`
+	TxErrorPackets       uint64   `protobuf:"varint,5,opt,name=txErrorPackets,proto3" json:"txErrorPackets,omitempty"`
+	RxErrorPackets       uint64   `protobuf:"varint,6,opt,name=rxErrorPackets,proto3" json:"rxErrorPackets,omitempty"`
+	TxBcastPackets       uint64   `protobuf:"varint,7,opt,name=txBcastPackets,proto3" json:"txBcastPackets,omitempty"`
+	RxBcastPackets       uint64   `protobuf:"varint,8,opt,name=rxBcastPackets,proto3" json:"rxBcastPackets,omitempty"`
+	TxUcastPackets       uint64   `protobuf:"varint,9,opt,name=txUcastPackets,proto3" json:"txUcastPackets,omitempty"`
+	RxUcastPackets       uint64   `protobuf:"varint,10,opt,name=rxUcastPackets,proto3" json:"rxUcastPackets,omitempty"`
+	TxMcastPackets       uint64   `protobuf:"varint,11,opt,name=txMcastPackets,proto3" json:"txMcastPackets,omitempty"`
+	RxMcastPackets       uint64   `protobuf:"varint,12,opt,name=rxMcastPackets,proto3" json:"rxMcastPackets,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetOltPortCountersResponse) Reset()         { *m = GetOltPortCountersResponse{} }
+func (m *GetOltPortCountersResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOltPortCountersResponse) ProtoMessage()    {}
+func (*GetOltPortCountersResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{5}
+}
+
+func (m *GetOltPortCountersResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOltPortCountersResponse.Unmarshal(m, b)
+}
+func (m *GetOltPortCountersResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOltPortCountersResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOltPortCountersResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOltPortCountersResponse.Merge(m, src)
+}
+func (m *GetOltPortCountersResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOltPortCountersResponse.Size(m)
+}
+func (m *GetOltPortCountersResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOltPortCountersResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOltPortCountersResponse proto.InternalMessageInfo
+
+func (m *GetOltPortCountersResponse) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetTxErrorPackets() uint64 {
+	if m != nil {
+		return m.TxErrorPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxErrorPackets() uint64 {
+	if m != nil {
+		return m.RxErrorPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetTxBcastPackets() uint64 {
+	if m != nil {
+		return m.TxBcastPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxBcastPackets() uint64 {
+	if m != nil {
+		return m.RxBcastPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetTxUcastPackets() uint64 {
+	if m != nil {
+		return m.TxUcastPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxUcastPackets() uint64 {
+	if m != nil {
+		return m.RxUcastPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetTxMcastPackets() uint64 {
+	if m != nil {
+		return m.TxMcastPackets
+	}
+	return 0
+}
+
+func (m *GetOltPortCountersResponse) GetRxMcastPackets() uint64 {
+	if m != nil {
+		return m.RxMcastPackets
+	}
+	return 0
+}
+
+type GetOnuPonOpticalInfo struct {
+	Empty                *empty.Empty `protobuf:"bytes,1,opt,name=empty,proto3" json:"empty,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *GetOnuPonOpticalInfo) Reset()         { *m = GetOnuPonOpticalInfo{} }
+func (m *GetOnuPonOpticalInfo) String() string { return proto.CompactTextString(m) }
+func (*GetOnuPonOpticalInfo) ProtoMessage()    {}
+func (*GetOnuPonOpticalInfo) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{6}
+}
+
+func (m *GetOnuPonOpticalInfo) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuPonOpticalInfo.Unmarshal(m, b)
+}
+func (m *GetOnuPonOpticalInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuPonOpticalInfo.Marshal(b, m, deterministic)
+}
+func (m *GetOnuPonOpticalInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuPonOpticalInfo.Merge(m, src)
+}
+func (m *GetOnuPonOpticalInfo) XXX_Size() int {
+	return xxx_messageInfo_GetOnuPonOpticalInfo.Size(m)
+}
+func (m *GetOnuPonOpticalInfo) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuPonOpticalInfo.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuPonOpticalInfo proto.InternalMessageInfo
+
+func (m *GetOnuPonOpticalInfo) GetEmpty() *empty.Empty {
+	if m != nil {
+		return m.Empty
+	}
+	return nil
+}
+
+// These values correspond to the Optical Line Supervision Test results
+// described in section A3.39.5 of ITU-T G.988 (11/2017) specification.
+type GetOnuPonOpticalInfoResponse struct {
+	PowerFeedVoltage       float32  `protobuf:"fixed32,1,opt,name=powerFeedVoltage,proto3" json:"powerFeedVoltage,omitempty"`
+	ReceivedOpticalPower   float32  `protobuf:"fixed32,2,opt,name=receivedOpticalPower,proto3" json:"receivedOpticalPower,omitempty"`
+	MeanOpticalLaunchPower float32  `protobuf:"fixed32,3,opt,name=meanOpticalLaunchPower,proto3" json:"meanOpticalLaunchPower,omitempty"`
+	LaserBiasCurrent       float32  `protobuf:"fixed32,4,opt,name=laserBiasCurrent,proto3" json:"laserBiasCurrent,omitempty"`
+	Temperature            float32  `protobuf:"fixed32,5,opt,name=temperature,proto3" json:"temperature,omitempty"`
+	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
+	XXX_unrecognized       []byte   `json:"-"`
+	XXX_sizecache          int32    `json:"-"`
+}
+
+func (m *GetOnuPonOpticalInfoResponse) Reset()         { *m = GetOnuPonOpticalInfoResponse{} }
+func (m *GetOnuPonOpticalInfoResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOnuPonOpticalInfoResponse) ProtoMessage()    {}
+func (*GetOnuPonOpticalInfoResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{7}
+}
+
+func (m *GetOnuPonOpticalInfoResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Unmarshal(m, b)
+}
+func (m *GetOnuPonOpticalInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOnuPonOpticalInfoResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuPonOpticalInfoResponse.Merge(m, src)
+}
+func (m *GetOnuPonOpticalInfoResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOnuPonOpticalInfoResponse.Size(m)
+}
+func (m *GetOnuPonOpticalInfoResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuPonOpticalInfoResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuPonOpticalInfoResponse proto.InternalMessageInfo
+
+func (m *GetOnuPonOpticalInfoResponse) GetPowerFeedVoltage() float32 {
+	if m != nil {
+		return m.PowerFeedVoltage
+	}
+	return 0
+}
+
+func (m *GetOnuPonOpticalInfoResponse) GetReceivedOpticalPower() float32 {
+	if m != nil {
+		return m.ReceivedOpticalPower
+	}
+	return 0
+}
+
+func (m *GetOnuPonOpticalInfoResponse) GetMeanOpticalLaunchPower() float32 {
+	if m != nil {
+		return m.MeanOpticalLaunchPower
+	}
+	return 0
+}
+
+func (m *GetOnuPonOpticalInfoResponse) GetLaserBiasCurrent() float32 {
+	if m != nil {
+		return m.LaserBiasCurrent
+	}
+	return 0
+}
+
+func (m *GetOnuPonOpticalInfoResponse) GetTemperature() float32 {
+	if m != nil {
+		return m.Temperature
+	}
+	return 0
+}
+
+type GetOnuEthernetBridgePortHistory struct {
+	Direction            GetOnuEthernetBridgePortHistory_Direction `protobuf:"varint,1,opt,name=direction,proto3,enum=extension.GetOnuEthernetBridgePortHistory_Direction" json:"direction,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                  `json:"-"`
+	XXX_unrecognized     []byte                                    `json:"-"`
+	XXX_sizecache        int32                                     `json:"-"`
+}
+
+func (m *GetOnuEthernetBridgePortHistory) Reset()         { *m = GetOnuEthernetBridgePortHistory{} }
+func (m *GetOnuEthernetBridgePortHistory) String() string { return proto.CompactTextString(m) }
+func (*GetOnuEthernetBridgePortHistory) ProtoMessage()    {}
+func (*GetOnuEthernetBridgePortHistory) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{8}
+}
+
+func (m *GetOnuEthernetBridgePortHistory) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Unmarshal(m, b)
+}
+func (m *GetOnuEthernetBridgePortHistory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Marshal(b, m, deterministic)
+}
+func (m *GetOnuEthernetBridgePortHistory) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuEthernetBridgePortHistory.Merge(m, src)
+}
+func (m *GetOnuEthernetBridgePortHistory) XXX_Size() int {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistory.Size(m)
+}
+func (m *GetOnuEthernetBridgePortHistory) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuEthernetBridgePortHistory.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuEthernetBridgePortHistory proto.InternalMessageInfo
+
+func (m *GetOnuEthernetBridgePortHistory) GetDirection() GetOnuEthernetBridgePortHistory_Direction {
+	if m != nil {
+		return m.Direction
+	}
+	return GetOnuEthernetBridgePortHistory_UNDEFINED
+}
+
+type GetOnuEthernetBridgePortHistoryResponse struct {
+	DropEvents              uint32   `protobuf:"varint,1,opt,name=dropEvents,proto3" json:"dropEvents,omitempty"`
+	Octets                  uint32   `protobuf:"varint,2,opt,name=octets,proto3" json:"octets,omitempty"`
+	Packets                 uint32   `protobuf:"varint,3,opt,name=packets,proto3" json:"packets,omitempty"`
+	BroadcastPackets        uint32   `protobuf:"varint,4,opt,name=broadcastPackets,proto3" json:"broadcastPackets,omitempty"`
+	MulticastPackets        uint32   `protobuf:"varint,5,opt,name=multicastPackets,proto3" json:"multicastPackets,omitempty"`
+	CrcErroredPackets       uint32   `protobuf:"varint,6,opt,name=crcErroredPackets,proto3" json:"crcErroredPackets,omitempty"`
+	UndersizePackets        uint32   `protobuf:"varint,7,opt,name=undersizePackets,proto3" json:"undersizePackets,omitempty"`
+	OversizePackets         uint32   `protobuf:"varint,8,opt,name=oversizePackets,proto3" json:"oversizePackets,omitempty"`
+	Packets64Octets         uint32   `protobuf:"varint,9,opt,name=packets64octets,proto3" json:"packets64octets,omitempty"`
+	Packets65To127Octets    uint32   `protobuf:"varint,10,opt,name=packets65To127octets,proto3" json:"packets65To127octets,omitempty"`
+	Packets128To255Octets   uint32   `protobuf:"varint,11,opt,name=packets128To255Octets,proto3" json:"packets128To255Octets,omitempty"`
+	Packets256To511Octets   uint32   `protobuf:"varint,12,opt,name=packets256To511octets,proto3" json:"packets256To511octets,omitempty"`
+	Packets512To1023Octets  uint32   `protobuf:"varint,13,opt,name=packets512To1023octets,proto3" json:"packets512To1023octets,omitempty"`
+	Packets1024To1518Octets uint32   `protobuf:"varint,14,opt,name=packets1024To1518octets,proto3" json:"packets1024To1518octets,omitempty"`
+	XXX_NoUnkeyedLiteral    struct{} `json:"-"`
+	XXX_unrecognized        []byte   `json:"-"`
+	XXX_sizecache           int32    `json:"-"`
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) Reset() {
+	*m = GetOnuEthernetBridgePortHistoryResponse{}
+}
+func (m *GetOnuEthernetBridgePortHistoryResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOnuEthernetBridgePortHistoryResponse) ProtoMessage()    {}
+func (*GetOnuEthernetBridgePortHistoryResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{9}
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Unmarshal(m, b)
+}
+func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Merge(m, src)
+}
+func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.Size(m)
+}
+func (m *GetOnuEthernetBridgePortHistoryResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuEthernetBridgePortHistoryResponse proto.InternalMessageInfo
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetDropEvents() uint32 {
+	if m != nil {
+		return m.DropEvents
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetOctets() uint32 {
+	if m != nil {
+		return m.Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets() uint32 {
+	if m != nil {
+		return m.Packets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetBroadcastPackets() uint32 {
+	if m != nil {
+		return m.BroadcastPackets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetMulticastPackets() uint32 {
+	if m != nil {
+		return m.MulticastPackets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetCrcErroredPackets() uint32 {
+	if m != nil {
+		return m.CrcErroredPackets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetUndersizePackets() uint32 {
+	if m != nil {
+		return m.UndersizePackets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetOversizePackets() uint32 {
+	if m != nil {
+		return m.OversizePackets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets64Octets() uint32 {
+	if m != nil {
+		return m.Packets64Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets65To127Octets() uint32 {
+	if m != nil {
+		return m.Packets65To127Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets128To255Octets() uint32 {
+	if m != nil {
+		return m.Packets128To255Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets256To511Octets() uint32 {
+	if m != nil {
+		return m.Packets256To511Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets512To1023Octets() uint32 {
+	if m != nil {
+		return m.Packets512To1023Octets
+	}
+	return 0
+}
+
+func (m *GetOnuEthernetBridgePortHistoryResponse) GetPackets1024To1518Octets() uint32 {
+	if m != nil {
+		return m.Packets1024To1518Octets
+	}
+	return 0
+}
+
+type GetOnuFecHistory struct {
+	Empty                *empty.Empty `protobuf:"bytes,1,opt,name=empty,proto3" json:"empty,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *GetOnuFecHistory) Reset()         { *m = GetOnuFecHistory{} }
+func (m *GetOnuFecHistory) String() string { return proto.CompactTextString(m) }
+func (*GetOnuFecHistory) ProtoMessage()    {}
+func (*GetOnuFecHistory) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{10}
+}
+
+func (m *GetOnuFecHistory) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuFecHistory.Unmarshal(m, b)
+}
+func (m *GetOnuFecHistory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuFecHistory.Marshal(b, m, deterministic)
+}
+func (m *GetOnuFecHistory) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuFecHistory.Merge(m, src)
+}
+func (m *GetOnuFecHistory) XXX_Size() int {
+	return xxx_messageInfo_GetOnuFecHistory.Size(m)
+}
+func (m *GetOnuFecHistory) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuFecHistory.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuFecHistory proto.InternalMessageInfo
+
+func (m *GetOnuFecHistory) GetEmpty() *empty.Empty {
+	if m != nil {
+		return m.Empty
+	}
+	return nil
+}
+
+type GetOnuFecHistoryResponse struct {
+	CorrectedBytes         uint32   `protobuf:"varint,1,opt,name=correctedBytes,proto3" json:"correctedBytes,omitempty"`
+	CorrectedCodeWords     uint32   `protobuf:"varint,2,opt,name=correctedCodeWords,proto3" json:"correctedCodeWords,omitempty"`
+	FecSeconds             uint32   `protobuf:"varint,3,opt,name=fecSeconds,proto3" json:"fecSeconds,omitempty"`
+	TotalCodeWords         uint32   `protobuf:"varint,4,opt,name=totalCodeWords,proto3" json:"totalCodeWords,omitempty"`
+	UncorrectableCodeWords uint32   `protobuf:"varint,5,opt,name=uncorrectableCodeWords,proto3" json:"uncorrectableCodeWords,omitempty"`
+	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
+	XXX_unrecognized       []byte   `json:"-"`
+	XXX_sizecache          int32    `json:"-"`
+}
+
+func (m *GetOnuFecHistoryResponse) Reset()         { *m = GetOnuFecHistoryResponse{} }
+func (m *GetOnuFecHistoryResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOnuFecHistoryResponse) ProtoMessage()    {}
+func (*GetOnuFecHistoryResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{11}
+}
+
+func (m *GetOnuFecHistoryResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuFecHistoryResponse.Unmarshal(m, b)
+}
+func (m *GetOnuFecHistoryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuFecHistoryResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOnuFecHistoryResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuFecHistoryResponse.Merge(m, src)
+}
+func (m *GetOnuFecHistoryResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOnuFecHistoryResponse.Size(m)
+}
+func (m *GetOnuFecHistoryResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuFecHistoryResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuFecHistoryResponse proto.InternalMessageInfo
+
+func (m *GetOnuFecHistoryResponse) GetCorrectedBytes() uint32 {
+	if m != nil {
+		return m.CorrectedBytes
+	}
+	return 0
+}
+
+func (m *GetOnuFecHistoryResponse) GetCorrectedCodeWords() uint32 {
+	if m != nil {
+		return m.CorrectedCodeWords
+	}
+	return 0
+}
+
+func (m *GetOnuFecHistoryResponse) GetFecSeconds() uint32 {
+	if m != nil {
+		return m.FecSeconds
+	}
+	return 0
+}
+
+func (m *GetOnuFecHistoryResponse) GetTotalCodeWords() uint32 {
+	if m != nil {
+		return m.TotalCodeWords
+	}
+	return 0
+}
+
+func (m *GetOnuFecHistoryResponse) GetUncorrectableCodeWords() uint32 {
+	if m != nil {
+		return m.UncorrectableCodeWords
+	}
+	return 0
+}
+
+type GetOnuCountersRequest struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetOnuCountersRequest) Reset()         { *m = GetOnuCountersRequest{} }
+func (m *GetOnuCountersRequest) String() string { return proto.CompactTextString(m) }
+func (*GetOnuCountersRequest) ProtoMessage()    {}
+func (*GetOnuCountersRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{12}
+}
+
+func (m *GetOnuCountersRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuCountersRequest.Unmarshal(m, b)
+}
+func (m *GetOnuCountersRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuCountersRequest.Marshal(b, m, deterministic)
+}
+func (m *GetOnuCountersRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuCountersRequest.Merge(m, src)
+}
+func (m *GetOnuCountersRequest) XXX_Size() int {
+	return xxx_messageInfo_GetOnuCountersRequest.Size(m)
+}
+func (m *GetOnuCountersRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuCountersRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuCountersRequest proto.InternalMessageInfo
+
+func (m *GetOnuCountersRequest) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *GetOnuCountersRequest) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+type GetOmciEthernetFrameExtendedPmRequest struct {
+	OnuDeviceId          string   `protobuf:"bytes,1,opt,name=onuDeviceId,proto3" json:"onuDeviceId,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetOmciEthernetFrameExtendedPmRequest) Reset()         { *m = GetOmciEthernetFrameExtendedPmRequest{} }
+func (m *GetOmciEthernetFrameExtendedPmRequest) String() string { return proto.CompactTextString(m) }
+func (*GetOmciEthernetFrameExtendedPmRequest) ProtoMessage()    {}
+func (*GetOmciEthernetFrameExtendedPmRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{13}
+}
+
+func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Unmarshal(m, b)
+}
+func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Marshal(b, m, deterministic)
+}
+func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Merge(m, src)
+}
+func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_Size() int {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.Size(m)
+}
+func (m *GetOmciEthernetFrameExtendedPmRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOmciEthernetFrameExtendedPmRequest proto.InternalMessageInfo
+
+func (m *GetOmciEthernetFrameExtendedPmRequest) GetOnuDeviceId() string {
+	if m != nil {
+		return m.OnuDeviceId
+	}
+	return ""
+}
+
+type GetRxPowerRequest struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetRxPowerRequest) Reset()         { *m = GetRxPowerRequest{} }
+func (m *GetRxPowerRequest) String() string { return proto.CompactTextString(m) }
+func (*GetRxPowerRequest) ProtoMessage()    {}
+func (*GetRxPowerRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{14}
+}
+
+func (m *GetRxPowerRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetRxPowerRequest.Unmarshal(m, b)
+}
+func (m *GetRxPowerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetRxPowerRequest.Marshal(b, m, deterministic)
+}
+func (m *GetRxPowerRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetRxPowerRequest.Merge(m, src)
+}
+func (m *GetRxPowerRequest) XXX_Size() int {
+	return xxx_messageInfo_GetRxPowerRequest.Size(m)
+}
+func (m *GetRxPowerRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetRxPowerRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetRxPowerRequest proto.InternalMessageInfo
+
+func (m *GetRxPowerRequest) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *GetRxPowerRequest) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+type GetOnuCountersResponse struct {
+	// Types that are valid to be assigned to IsIntfId:
+	//	*GetOnuCountersResponse_IntfId
+	IsIntfId isGetOnuCountersResponse_IsIntfId `protobuf_oneof:"is_intf_id"`
+	// Types that are valid to be assigned to IsOnuId:
+	//	*GetOnuCountersResponse_OnuId
+	IsOnuId isGetOnuCountersResponse_IsOnuId `protobuf_oneof:"is_onu_id"`
+	// Types that are valid to be assigned to IsPositiveDrift:
+	//	*GetOnuCountersResponse_PositiveDrift
+	IsPositiveDrift isGetOnuCountersResponse_IsPositiveDrift `protobuf_oneof:"is_positive_drift"`
+	// Types that are valid to be assigned to IsNegativeDrift:
+	//	*GetOnuCountersResponse_NegativeDrift
+	IsNegativeDrift isGetOnuCountersResponse_IsNegativeDrift `protobuf_oneof:"is_negative_drift"`
+	// Types that are valid to be assigned to IsDelimiterMissDetection:
+	//	*GetOnuCountersResponse_DelimiterMissDetection
+	IsDelimiterMissDetection isGetOnuCountersResponse_IsDelimiterMissDetection `protobuf_oneof:"is_delimiter_miss_detection"`
+	// Types that are valid to be assigned to IsBipErrors:
+	//	*GetOnuCountersResponse_BipErrors
+	IsBipErrors isGetOnuCountersResponse_IsBipErrors `protobuf_oneof:"is_bip_errors"`
+	// Types that are valid to be assigned to IsBipUnits:
+	//	*GetOnuCountersResponse_BipUnits
+	IsBipUnits isGetOnuCountersResponse_IsBipUnits `protobuf_oneof:"is_bip_units"`
+	// Types that are valid to be assigned to IsFecCorrectedSymbols:
+	//	*GetOnuCountersResponse_FecCorrectedSymbols
+	IsFecCorrectedSymbols isGetOnuCountersResponse_IsFecCorrectedSymbols `protobuf_oneof:"is_fec_corrected_symbols"`
+	// Types that are valid to be assigned to IsFecCodewordsCorrected:
+	//	*GetOnuCountersResponse_FecCodewordsCorrected
+	IsFecCodewordsCorrected isGetOnuCountersResponse_IsFecCodewordsCorrected `protobuf_oneof:"is_fec_codewords_corrected"`
+	// Types that are valid to be assigned to IsFecCodewordsUncorrectable:
+	//	*GetOnuCountersResponse_FecCodewordsUncorrectable
+	IsFecCodewordsUncorrectable isGetOnuCountersResponse_IsFecCodewordsUncorrectable `protobuf_oneof:"is_fec_codewords_uncorrectable"`
+	// Types that are valid to be assigned to IsFecCodewords:
+	//	*GetOnuCountersResponse_FecCodewords
+	IsFecCodewords isGetOnuCountersResponse_IsFecCodewords `protobuf_oneof:"is_fec_codewords"`
+	// Types that are valid to be assigned to IsFecCorrectedUnits:
+	//	*GetOnuCountersResponse_FecCorrectedUnits
+	IsFecCorrectedUnits isGetOnuCountersResponse_IsFecCorrectedUnits `protobuf_oneof:"is_fec_corrected_units"`
+	// Types that are valid to be assigned to IsXgemKeyErrors:
+	//	*GetOnuCountersResponse_XgemKeyErrors
+	IsXgemKeyErrors isGetOnuCountersResponse_IsXgemKeyErrors `protobuf_oneof:"is_xgem_key_errors"`
+	// Types that are valid to be assigned to IsXgemLoss:
+	//	*GetOnuCountersResponse_XgemLoss
+	IsXgemLoss isGetOnuCountersResponse_IsXgemLoss `protobuf_oneof:"is_xgem_loss"`
+	// Types that are valid to be assigned to IsRxPloamsError:
+	//	*GetOnuCountersResponse_RxPloamsError
+	IsRxPloamsError isGetOnuCountersResponse_IsRxPloamsError `protobuf_oneof:"is_rx_ploams_error"`
+	// Types that are valid to be assigned to IsRxPloamsNonIdle:
+	//	*GetOnuCountersResponse_RxPloamsNonIdle
+	IsRxPloamsNonIdle isGetOnuCountersResponse_IsRxPloamsNonIdle `protobuf_oneof:"is_rx_ploams_non_idle"`
+	// Types that are valid to be assigned to IsRxOmci:
+	//	*GetOnuCountersResponse_RxOmci
+	IsRxOmci isGetOnuCountersResponse_IsRxOmci `protobuf_oneof:"is_rx_omci"`
+	// Types that are valid to be assigned to IsTxOmci:
+	//	*GetOnuCountersResponse_TxOmci
+	IsTxOmci isGetOnuCountersResponse_IsTxOmci `protobuf_oneof:"is_tx_omci"`
+	// Types that are valid to be assigned to IsRxOmciPacketsCrcError:
+	//	*GetOnuCountersResponse_RxOmciPacketsCrcError
+	IsRxOmciPacketsCrcError isGetOnuCountersResponse_IsRxOmciPacketsCrcError `protobuf_oneof:"is_rx_omci_packets_crc_error"`
+	// Types that are valid to be assigned to IsRxBytes:
+	//	*GetOnuCountersResponse_RxBytes
+	IsRxBytes isGetOnuCountersResponse_IsRxBytes `protobuf_oneof:"is_rx_bytes"`
+	// Types that are valid to be assigned to IsRxPackets:
+	//	*GetOnuCountersResponse_RxPackets
+	IsRxPackets isGetOnuCountersResponse_IsRxPackets `protobuf_oneof:"is_rx_packets"`
+	// Types that are valid to be assigned to IsTxBytes:
+	//	*GetOnuCountersResponse_TxBytes
+	IsTxBytes isGetOnuCountersResponse_IsTxBytes `protobuf_oneof:"is_tx_bytes"`
+	// Types that are valid to be assigned to IsTxPackets:
+	//	*GetOnuCountersResponse_TxPackets
+	IsTxPackets isGetOnuCountersResponse_IsTxPackets `protobuf_oneof:"is_tx_packets"`
+	// Types that are valid to be assigned to IsBerReported:
+	//	*GetOnuCountersResponse_BerReported
+	IsBerReported isGetOnuCountersResponse_IsBerReported `protobuf_oneof:"is_ber_reported"`
+	// Types that are valid to be assigned to IsLcdgErrors:
+	//	*GetOnuCountersResponse_LcdgErrors
+	IsLcdgErrors isGetOnuCountersResponse_IsLcdgErrors `protobuf_oneof:"is_lcdg_errors"`
+	// Types that are valid to be assigned to IsRdiErrors:
+	//	*GetOnuCountersResponse_RdiErrors
+	IsRdiErrors isGetOnuCountersResponse_IsRdiErrors `protobuf_oneof:"is_rdi_errors"`
+	// Types that are valid to be assigned to IsTimestamp:
+	//	*GetOnuCountersResponse_Timestamp
+	IsTimestamp          isGetOnuCountersResponse_IsTimestamp `protobuf_oneof:"is_timestamp"`
+	XXX_NoUnkeyedLiteral struct{}                             `json:"-"`
+	XXX_unrecognized     []byte                               `json:"-"`
+	XXX_sizecache        int32                                `json:"-"`
+}
+
+func (m *GetOnuCountersResponse) Reset()         { *m = GetOnuCountersResponse{} }
+func (m *GetOnuCountersResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOnuCountersResponse) ProtoMessage()    {}
+func (*GetOnuCountersResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{15}
+}
+
+func (m *GetOnuCountersResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOnuCountersResponse.Unmarshal(m, b)
+}
+func (m *GetOnuCountersResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOnuCountersResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOnuCountersResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOnuCountersResponse.Merge(m, src)
+}
+func (m *GetOnuCountersResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOnuCountersResponse.Size(m)
+}
+func (m *GetOnuCountersResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOnuCountersResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOnuCountersResponse proto.InternalMessageInfo
+
+type isGetOnuCountersResponse_IsIntfId interface {
+	isGetOnuCountersResponse_IsIntfId()
+}
+
+type GetOnuCountersResponse_IntfId struct {
+	IntfId uint32 `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_IntfId) isGetOnuCountersResponse_IsIntfId() {}
+
+func (m *GetOnuCountersResponse) GetIsIntfId() isGetOnuCountersResponse_IsIntfId {
+	if m != nil {
+		return m.IsIntfId
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetIntfId() uint32 {
+	if x, ok := m.GetIsIntfId().(*GetOnuCountersResponse_IntfId); ok {
+		return x.IntfId
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsOnuId interface {
+	isGetOnuCountersResponse_IsOnuId()
+}
+
+type GetOnuCountersResponse_OnuId struct {
+	OnuId uint32 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_OnuId) isGetOnuCountersResponse_IsOnuId() {}
+
+func (m *GetOnuCountersResponse) GetIsOnuId() isGetOnuCountersResponse_IsOnuId {
+	if m != nil {
+		return m.IsOnuId
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetOnuId() uint32 {
+	if x, ok := m.GetIsOnuId().(*GetOnuCountersResponse_OnuId); ok {
+		return x.OnuId
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsPositiveDrift interface {
+	isGetOnuCountersResponse_IsPositiveDrift()
+}
+
+type GetOnuCountersResponse_PositiveDrift struct {
+	PositiveDrift uint64 `protobuf:"fixed64,3,opt,name=positive_drift,json=positiveDrift,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_PositiveDrift) isGetOnuCountersResponse_IsPositiveDrift() {}
+
+func (m *GetOnuCountersResponse) GetIsPositiveDrift() isGetOnuCountersResponse_IsPositiveDrift {
+	if m != nil {
+		return m.IsPositiveDrift
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetPositiveDrift() uint64 {
+	if x, ok := m.GetIsPositiveDrift().(*GetOnuCountersResponse_PositiveDrift); ok {
+		return x.PositiveDrift
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsNegativeDrift interface {
+	isGetOnuCountersResponse_IsNegativeDrift()
+}
+
+type GetOnuCountersResponse_NegativeDrift struct {
+	NegativeDrift uint64 `protobuf:"fixed64,4,opt,name=negative_drift,json=negativeDrift,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_NegativeDrift) isGetOnuCountersResponse_IsNegativeDrift() {}
+
+func (m *GetOnuCountersResponse) GetIsNegativeDrift() isGetOnuCountersResponse_IsNegativeDrift {
+	if m != nil {
+		return m.IsNegativeDrift
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetNegativeDrift() uint64 {
+	if x, ok := m.GetIsNegativeDrift().(*GetOnuCountersResponse_NegativeDrift); ok {
+		return x.NegativeDrift
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsDelimiterMissDetection interface {
+	isGetOnuCountersResponse_IsDelimiterMissDetection()
+}
+
+type GetOnuCountersResponse_DelimiterMissDetection struct {
+	DelimiterMissDetection uint64 `protobuf:"fixed64,5,opt,name=delimiter_miss_detection,json=delimiterMissDetection,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_DelimiterMissDetection) isGetOnuCountersResponse_IsDelimiterMissDetection() {
+}
+
+func (m *GetOnuCountersResponse) GetIsDelimiterMissDetection() isGetOnuCountersResponse_IsDelimiterMissDetection {
+	if m != nil {
+		return m.IsDelimiterMissDetection
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetDelimiterMissDetection() uint64 {
+	if x, ok := m.GetIsDelimiterMissDetection().(*GetOnuCountersResponse_DelimiterMissDetection); ok {
+		return x.DelimiterMissDetection
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsBipErrors interface {
+	isGetOnuCountersResponse_IsBipErrors()
+}
+
+type GetOnuCountersResponse_BipErrors struct {
+	BipErrors uint64 `protobuf:"fixed64,6,opt,name=bip_errors,json=bipErrors,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_BipErrors) isGetOnuCountersResponse_IsBipErrors() {}
+
+func (m *GetOnuCountersResponse) GetIsBipErrors() isGetOnuCountersResponse_IsBipErrors {
+	if m != nil {
+		return m.IsBipErrors
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetBipErrors() uint64 {
+	if x, ok := m.GetIsBipErrors().(*GetOnuCountersResponse_BipErrors); ok {
+		return x.BipErrors
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsBipUnits interface {
+	isGetOnuCountersResponse_IsBipUnits()
+}
+
+type GetOnuCountersResponse_BipUnits struct {
+	BipUnits uint64 `protobuf:"fixed64,7,opt,name=bip_units,json=bipUnits,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_BipUnits) isGetOnuCountersResponse_IsBipUnits() {}
+
+func (m *GetOnuCountersResponse) GetIsBipUnits() isGetOnuCountersResponse_IsBipUnits {
+	if m != nil {
+		return m.IsBipUnits
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetBipUnits() uint64 {
+	if x, ok := m.GetIsBipUnits().(*GetOnuCountersResponse_BipUnits); ok {
+		return x.BipUnits
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsFecCorrectedSymbols interface {
+	isGetOnuCountersResponse_IsFecCorrectedSymbols()
+}
+
+type GetOnuCountersResponse_FecCorrectedSymbols struct {
+	FecCorrectedSymbols uint64 `protobuf:"fixed64,8,opt,name=fec_corrected_symbols,json=fecCorrectedSymbols,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_FecCorrectedSymbols) isGetOnuCountersResponse_IsFecCorrectedSymbols() {}
+
+func (m *GetOnuCountersResponse) GetIsFecCorrectedSymbols() isGetOnuCountersResponse_IsFecCorrectedSymbols {
+	if m != nil {
+		return m.IsFecCorrectedSymbols
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetFecCorrectedSymbols() uint64 {
+	if x, ok := m.GetIsFecCorrectedSymbols().(*GetOnuCountersResponse_FecCorrectedSymbols); ok {
+		return x.FecCorrectedSymbols
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsFecCodewordsCorrected interface {
+	isGetOnuCountersResponse_IsFecCodewordsCorrected()
+}
+
+type GetOnuCountersResponse_FecCodewordsCorrected struct {
+	FecCodewordsCorrected uint64 `protobuf:"fixed64,9,opt,name=fec_codewords_corrected,json=fecCodewordsCorrected,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_FecCodewordsCorrected) isGetOnuCountersResponse_IsFecCodewordsCorrected() {
+}
+
+func (m *GetOnuCountersResponse) GetIsFecCodewordsCorrected() isGetOnuCountersResponse_IsFecCodewordsCorrected {
+	if m != nil {
+		return m.IsFecCodewordsCorrected
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetFecCodewordsCorrected() uint64 {
+	if x, ok := m.GetIsFecCodewordsCorrected().(*GetOnuCountersResponse_FecCodewordsCorrected); ok {
+		return x.FecCodewordsCorrected
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsFecCodewordsUncorrectable interface {
+	isGetOnuCountersResponse_IsFecCodewordsUncorrectable()
+}
+
+type GetOnuCountersResponse_FecCodewordsUncorrectable struct {
+	FecCodewordsUncorrectable uint64 `protobuf:"fixed64,10,opt,name=fec_codewords_uncorrectable,json=fecCodewordsUncorrectable,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_FecCodewordsUncorrectable) isGetOnuCountersResponse_IsFecCodewordsUncorrectable() {
+}
+
+func (m *GetOnuCountersResponse) GetIsFecCodewordsUncorrectable() isGetOnuCountersResponse_IsFecCodewordsUncorrectable {
+	if m != nil {
+		return m.IsFecCodewordsUncorrectable
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetFecCodewordsUncorrectable() uint64 {
+	if x, ok := m.GetIsFecCodewordsUncorrectable().(*GetOnuCountersResponse_FecCodewordsUncorrectable); ok {
+		return x.FecCodewordsUncorrectable
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsFecCodewords interface {
+	isGetOnuCountersResponse_IsFecCodewords()
+}
+
+type GetOnuCountersResponse_FecCodewords struct {
+	FecCodewords uint64 `protobuf:"fixed64,11,opt,name=fec_codewords,json=fecCodewords,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_FecCodewords) isGetOnuCountersResponse_IsFecCodewords() {}
+
+func (m *GetOnuCountersResponse) GetIsFecCodewords() isGetOnuCountersResponse_IsFecCodewords {
+	if m != nil {
+		return m.IsFecCodewords
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetFecCodewords() uint64 {
+	if x, ok := m.GetIsFecCodewords().(*GetOnuCountersResponse_FecCodewords); ok {
+		return x.FecCodewords
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsFecCorrectedUnits interface {
+	isGetOnuCountersResponse_IsFecCorrectedUnits()
+}
+
+type GetOnuCountersResponse_FecCorrectedUnits struct {
+	FecCorrectedUnits uint64 `protobuf:"fixed64,12,opt,name=fec_corrected_units,json=fecCorrectedUnits,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_FecCorrectedUnits) isGetOnuCountersResponse_IsFecCorrectedUnits() {}
+
+func (m *GetOnuCountersResponse) GetIsFecCorrectedUnits() isGetOnuCountersResponse_IsFecCorrectedUnits {
+	if m != nil {
+		return m.IsFecCorrectedUnits
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetFecCorrectedUnits() uint64 {
+	if x, ok := m.GetIsFecCorrectedUnits().(*GetOnuCountersResponse_FecCorrectedUnits); ok {
+		return x.FecCorrectedUnits
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsXgemKeyErrors interface {
+	isGetOnuCountersResponse_IsXgemKeyErrors()
+}
+
+type GetOnuCountersResponse_XgemKeyErrors struct {
+	XgemKeyErrors uint64 `protobuf:"fixed64,13,opt,name=xgem_key_errors,json=xgemKeyErrors,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_XgemKeyErrors) isGetOnuCountersResponse_IsXgemKeyErrors() {}
+
+func (m *GetOnuCountersResponse) GetIsXgemKeyErrors() isGetOnuCountersResponse_IsXgemKeyErrors {
+	if m != nil {
+		return m.IsXgemKeyErrors
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetXgemKeyErrors() uint64 {
+	if x, ok := m.GetIsXgemKeyErrors().(*GetOnuCountersResponse_XgemKeyErrors); ok {
+		return x.XgemKeyErrors
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsXgemLoss interface {
+	isGetOnuCountersResponse_IsXgemLoss()
+}
+
+type GetOnuCountersResponse_XgemLoss struct {
+	XgemLoss uint64 `protobuf:"fixed64,14,opt,name=xgem_loss,json=xgemLoss,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_XgemLoss) isGetOnuCountersResponse_IsXgemLoss() {}
+
+func (m *GetOnuCountersResponse) GetIsXgemLoss() isGetOnuCountersResponse_IsXgemLoss {
+	if m != nil {
+		return m.IsXgemLoss
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetXgemLoss() uint64 {
+	if x, ok := m.GetIsXgemLoss().(*GetOnuCountersResponse_XgemLoss); ok {
+		return x.XgemLoss
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxPloamsError interface {
+	isGetOnuCountersResponse_IsRxPloamsError()
+}
+
+type GetOnuCountersResponse_RxPloamsError struct {
+	RxPloamsError uint64 `protobuf:"fixed64,15,opt,name=rx_ploams_error,json=rxPloamsError,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxPloamsError) isGetOnuCountersResponse_IsRxPloamsError() {}
+
+func (m *GetOnuCountersResponse) GetIsRxPloamsError() isGetOnuCountersResponse_IsRxPloamsError {
+	if m != nil {
+		return m.IsRxPloamsError
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxPloamsError() uint64 {
+	if x, ok := m.GetIsRxPloamsError().(*GetOnuCountersResponse_RxPloamsError); ok {
+		return x.RxPloamsError
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxPloamsNonIdle interface {
+	isGetOnuCountersResponse_IsRxPloamsNonIdle()
+}
+
+type GetOnuCountersResponse_RxPloamsNonIdle struct {
+	RxPloamsNonIdle uint64 `protobuf:"fixed64,16,opt,name=rx_ploams_non_idle,json=rxPloamsNonIdle,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxPloamsNonIdle) isGetOnuCountersResponse_IsRxPloamsNonIdle() {}
+
+func (m *GetOnuCountersResponse) GetIsRxPloamsNonIdle() isGetOnuCountersResponse_IsRxPloamsNonIdle {
+	if m != nil {
+		return m.IsRxPloamsNonIdle
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxPloamsNonIdle() uint64 {
+	if x, ok := m.GetIsRxPloamsNonIdle().(*GetOnuCountersResponse_RxPloamsNonIdle); ok {
+		return x.RxPloamsNonIdle
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxOmci interface {
+	isGetOnuCountersResponse_IsRxOmci()
+}
+
+type GetOnuCountersResponse_RxOmci struct {
+	RxOmci uint64 `protobuf:"fixed64,17,opt,name=rx_omci,json=rxOmci,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxOmci) isGetOnuCountersResponse_IsRxOmci() {}
+
+func (m *GetOnuCountersResponse) GetIsRxOmci() isGetOnuCountersResponse_IsRxOmci {
+	if m != nil {
+		return m.IsRxOmci
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxOmci() uint64 {
+	if x, ok := m.GetIsRxOmci().(*GetOnuCountersResponse_RxOmci); ok {
+		return x.RxOmci
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsTxOmci interface {
+	isGetOnuCountersResponse_IsTxOmci()
+}
+
+type GetOnuCountersResponse_TxOmci struct {
+	TxOmci uint64 `protobuf:"fixed64,18,opt,name=tx_omci,json=txOmci,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_TxOmci) isGetOnuCountersResponse_IsTxOmci() {}
+
+func (m *GetOnuCountersResponse) GetIsTxOmci() isGetOnuCountersResponse_IsTxOmci {
+	if m != nil {
+		return m.IsTxOmci
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetTxOmci() uint64 {
+	if x, ok := m.GetIsTxOmci().(*GetOnuCountersResponse_TxOmci); ok {
+		return x.TxOmci
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxOmciPacketsCrcError interface {
+	isGetOnuCountersResponse_IsRxOmciPacketsCrcError()
+}
+
+type GetOnuCountersResponse_RxOmciPacketsCrcError struct {
+	RxOmciPacketsCrcError uint64 `protobuf:"fixed64,19,opt,name=rx_omci_packets_crc_error,json=rxOmciPacketsCrcError,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxOmciPacketsCrcError) isGetOnuCountersResponse_IsRxOmciPacketsCrcError() {
+}
+
+func (m *GetOnuCountersResponse) GetIsRxOmciPacketsCrcError() isGetOnuCountersResponse_IsRxOmciPacketsCrcError {
+	if m != nil {
+		return m.IsRxOmciPacketsCrcError
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxOmciPacketsCrcError() uint64 {
+	if x, ok := m.GetIsRxOmciPacketsCrcError().(*GetOnuCountersResponse_RxOmciPacketsCrcError); ok {
+		return x.RxOmciPacketsCrcError
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxBytes interface {
+	isGetOnuCountersResponse_IsRxBytes()
+}
+
+type GetOnuCountersResponse_RxBytes struct {
+	RxBytes uint64 `protobuf:"fixed64,20,opt,name=rx_bytes,json=rxBytes,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxBytes) isGetOnuCountersResponse_IsRxBytes() {}
+
+func (m *GetOnuCountersResponse) GetIsRxBytes() isGetOnuCountersResponse_IsRxBytes {
+	if m != nil {
+		return m.IsRxBytes
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxBytes() uint64 {
+	if x, ok := m.GetIsRxBytes().(*GetOnuCountersResponse_RxBytes); ok {
+		return x.RxBytes
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRxPackets interface {
+	isGetOnuCountersResponse_IsRxPackets()
+}
+
+type GetOnuCountersResponse_RxPackets struct {
+	RxPackets uint64 `protobuf:"fixed64,21,opt,name=rx_packets,json=rxPackets,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RxPackets) isGetOnuCountersResponse_IsRxPackets() {}
+
+func (m *GetOnuCountersResponse) GetIsRxPackets() isGetOnuCountersResponse_IsRxPackets {
+	if m != nil {
+		return m.IsRxPackets
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRxPackets() uint64 {
+	if x, ok := m.GetIsRxPackets().(*GetOnuCountersResponse_RxPackets); ok {
+		return x.RxPackets
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsTxBytes interface {
+	isGetOnuCountersResponse_IsTxBytes()
+}
+
+type GetOnuCountersResponse_TxBytes struct {
+	TxBytes uint64 `protobuf:"fixed64,22,opt,name=tx_bytes,json=txBytes,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_TxBytes) isGetOnuCountersResponse_IsTxBytes() {}
+
+func (m *GetOnuCountersResponse) GetIsTxBytes() isGetOnuCountersResponse_IsTxBytes {
+	if m != nil {
+		return m.IsTxBytes
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetTxBytes() uint64 {
+	if x, ok := m.GetIsTxBytes().(*GetOnuCountersResponse_TxBytes); ok {
+		return x.TxBytes
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsTxPackets interface {
+	isGetOnuCountersResponse_IsTxPackets()
+}
+
+type GetOnuCountersResponse_TxPackets struct {
+	TxPackets uint64 `protobuf:"fixed64,23,opt,name=tx_packets,json=txPackets,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_TxPackets) isGetOnuCountersResponse_IsTxPackets() {}
+
+func (m *GetOnuCountersResponse) GetIsTxPackets() isGetOnuCountersResponse_IsTxPackets {
+	if m != nil {
+		return m.IsTxPackets
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetTxPackets() uint64 {
+	if x, ok := m.GetIsTxPackets().(*GetOnuCountersResponse_TxPackets); ok {
+		return x.TxPackets
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsBerReported interface {
+	isGetOnuCountersResponse_IsBerReported()
+}
+
+type GetOnuCountersResponse_BerReported struct {
+	BerReported uint64 `protobuf:"fixed64,24,opt,name=ber_reported,json=berReported,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_BerReported) isGetOnuCountersResponse_IsBerReported() {}
+
+func (m *GetOnuCountersResponse) GetIsBerReported() isGetOnuCountersResponse_IsBerReported {
+	if m != nil {
+		return m.IsBerReported
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetBerReported() uint64 {
+	if x, ok := m.GetIsBerReported().(*GetOnuCountersResponse_BerReported); ok {
+		return x.BerReported
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsLcdgErrors interface {
+	isGetOnuCountersResponse_IsLcdgErrors()
+}
+
+type GetOnuCountersResponse_LcdgErrors struct {
+	LcdgErrors uint64 `protobuf:"fixed64,25,opt,name=lcdg_errors,json=lcdgErrors,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_LcdgErrors) isGetOnuCountersResponse_IsLcdgErrors() {}
+
+func (m *GetOnuCountersResponse) GetIsLcdgErrors() isGetOnuCountersResponse_IsLcdgErrors {
+	if m != nil {
+		return m.IsLcdgErrors
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetLcdgErrors() uint64 {
+	if x, ok := m.GetIsLcdgErrors().(*GetOnuCountersResponse_LcdgErrors); ok {
+		return x.LcdgErrors
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsRdiErrors interface {
+	isGetOnuCountersResponse_IsRdiErrors()
+}
+
+type GetOnuCountersResponse_RdiErrors struct {
+	RdiErrors uint64 `protobuf:"fixed64,26,opt,name=rdi_errors,json=rdiErrors,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_RdiErrors) isGetOnuCountersResponse_IsRdiErrors() {}
+
+func (m *GetOnuCountersResponse) GetIsRdiErrors() isGetOnuCountersResponse_IsRdiErrors {
+	if m != nil {
+		return m.IsRdiErrors
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetRdiErrors() uint64 {
+	if x, ok := m.GetIsRdiErrors().(*GetOnuCountersResponse_RdiErrors); ok {
+		return x.RdiErrors
+	}
+	return 0
+}
+
+type isGetOnuCountersResponse_IsTimestamp interface {
+	isGetOnuCountersResponse_IsTimestamp()
+}
+
+type GetOnuCountersResponse_Timestamp struct {
+	Timestamp uint32 `protobuf:"fixed32,27,opt,name=timestamp,proto3,oneof"`
+}
+
+func (*GetOnuCountersResponse_Timestamp) isGetOnuCountersResponse_IsTimestamp() {}
+
+func (m *GetOnuCountersResponse) GetIsTimestamp() isGetOnuCountersResponse_IsTimestamp {
+	if m != nil {
+		return m.IsTimestamp
+	}
+	return nil
+}
+
+func (m *GetOnuCountersResponse) GetTimestamp() uint32 {
+	if x, ok := m.GetIsTimestamp().(*GetOnuCountersResponse_Timestamp); ok {
+		return x.Timestamp
+	}
+	return 0
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*GetOnuCountersResponse) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*GetOnuCountersResponse_IntfId)(nil),
+		(*GetOnuCountersResponse_OnuId)(nil),
+		(*GetOnuCountersResponse_PositiveDrift)(nil),
+		(*GetOnuCountersResponse_NegativeDrift)(nil),
+		(*GetOnuCountersResponse_DelimiterMissDetection)(nil),
+		(*GetOnuCountersResponse_BipErrors)(nil),
+		(*GetOnuCountersResponse_BipUnits)(nil),
+		(*GetOnuCountersResponse_FecCorrectedSymbols)(nil),
+		(*GetOnuCountersResponse_FecCodewordsCorrected)(nil),
+		(*GetOnuCountersResponse_FecCodewordsUncorrectable)(nil),
+		(*GetOnuCountersResponse_FecCodewords)(nil),
+		(*GetOnuCountersResponse_FecCorrectedUnits)(nil),
+		(*GetOnuCountersResponse_XgemKeyErrors)(nil),
+		(*GetOnuCountersResponse_XgemLoss)(nil),
+		(*GetOnuCountersResponse_RxPloamsError)(nil),
+		(*GetOnuCountersResponse_RxPloamsNonIdle)(nil),
+		(*GetOnuCountersResponse_RxOmci)(nil),
+		(*GetOnuCountersResponse_TxOmci)(nil),
+		(*GetOnuCountersResponse_RxOmciPacketsCrcError)(nil),
+		(*GetOnuCountersResponse_RxBytes)(nil),
+		(*GetOnuCountersResponse_RxPackets)(nil),
+		(*GetOnuCountersResponse_TxBytes)(nil),
+		(*GetOnuCountersResponse_TxPackets)(nil),
+		(*GetOnuCountersResponse_BerReported)(nil),
+		(*GetOnuCountersResponse_LcdgErrors)(nil),
+		(*GetOnuCountersResponse_RdiErrors)(nil),
+		(*GetOnuCountersResponse_Timestamp)(nil),
+	}
+}
+
+type OmciEthernetFrameExtendedPm struct {
+	DropEvents               uint64   `protobuf:"fixed64,1,opt,name=drop_events,json=dropEvents,proto3" json:"drop_events,omitempty"`
+	Octets                   uint64   `protobuf:"fixed64,2,opt,name=octets,proto3" json:"octets,omitempty"`
+	Frames                   uint64   `protobuf:"fixed64,3,opt,name=frames,proto3" json:"frames,omitempty"`
+	BroadcastFrames          uint64   `protobuf:"fixed64,4,opt,name=broadcast_frames,json=broadcastFrames,proto3" json:"broadcast_frames,omitempty"`
+	MulticastFrames          uint64   `protobuf:"fixed64,5,opt,name=multicast_frames,json=multicastFrames,proto3" json:"multicast_frames,omitempty"`
+	CrcErroredFrames         uint64   `protobuf:"fixed64,6,opt,name=crc_errored_frames,json=crcErroredFrames,proto3" json:"crc_errored_frames,omitempty"`
+	UndersizeFrames          uint64   `protobuf:"fixed64,7,opt,name=undersize_frames,json=undersizeFrames,proto3" json:"undersize_frames,omitempty"`
+	OversizeFrames           uint64   `protobuf:"fixed64,8,opt,name=oversize_frames,json=oversizeFrames,proto3" json:"oversize_frames,omitempty"`
+	Frames_64Octets          uint64   `protobuf:"fixed64,9,opt,name=frames_64_octets,json=frames64Octets,proto3" json:"frames_64_octets,omitempty"`
+	Frames_65To_127Octets    uint64   `protobuf:"fixed64,10,opt,name=frames_65_to_127_octets,json=frames65To127Octets,proto3" json:"frames_65_to_127_octets,omitempty"`
+	Frames_128To_255Octets   uint64   `protobuf:"fixed64,11,opt,name=frames_128_to_255_octets,json=frames128To255Octets,proto3" json:"frames_128_to_255_octets,omitempty"`
+	Frames_256To_511Octets   uint64   `protobuf:"fixed64,12,opt,name=frames_256_to_511_octets,json=frames256To511Octets,proto3" json:"frames_256_to_511_octets,omitempty"`
+	Frames_512To_1023Octets  uint64   `protobuf:"fixed64,13,opt,name=frames_512_to_1023_octets,json=frames512To1023Octets,proto3" json:"frames_512_to_1023_octets,omitempty"`
+	Frames_1024To_1518Octets uint64   `protobuf:"fixed64,14,opt,name=frames_1024_to_1518_octets,json=frames1024To1518Octets,proto3" json:"frames_1024_to_1518_octets,omitempty"`
+	XXX_NoUnkeyedLiteral     struct{} `json:"-"`
+	XXX_unrecognized         []byte   `json:"-"`
+	XXX_sizecache            int32    `json:"-"`
+}
+
+func (m *OmciEthernetFrameExtendedPm) Reset()         { *m = OmciEthernetFrameExtendedPm{} }
+func (m *OmciEthernetFrameExtendedPm) String() string { return proto.CompactTextString(m) }
+func (*OmciEthernetFrameExtendedPm) ProtoMessage()    {}
+func (*OmciEthernetFrameExtendedPm) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{16}
+}
+
+func (m *OmciEthernetFrameExtendedPm) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Unmarshal(m, b)
+}
+func (m *OmciEthernetFrameExtendedPm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Marshal(b, m, deterministic)
+}
+func (m *OmciEthernetFrameExtendedPm) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OmciEthernetFrameExtendedPm.Merge(m, src)
+}
+func (m *OmciEthernetFrameExtendedPm) XXX_Size() int {
+	return xxx_messageInfo_OmciEthernetFrameExtendedPm.Size(m)
+}
+func (m *OmciEthernetFrameExtendedPm) XXX_DiscardUnknown() {
+	xxx_messageInfo_OmciEthernetFrameExtendedPm.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OmciEthernetFrameExtendedPm proto.InternalMessageInfo
+
+func (m *OmciEthernetFrameExtendedPm) GetDropEvents() uint64 {
+	if m != nil {
+		return m.DropEvents
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetOctets() uint64 {
+	if m != nil {
+		return m.Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames() uint64 {
+	if m != nil {
+		return m.Frames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetBroadcastFrames() uint64 {
+	if m != nil {
+		return m.BroadcastFrames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetMulticastFrames() uint64 {
+	if m != nil {
+		return m.MulticastFrames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetCrcErroredFrames() uint64 {
+	if m != nil {
+		return m.CrcErroredFrames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetUndersizeFrames() uint64 {
+	if m != nil {
+		return m.UndersizeFrames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetOversizeFrames() uint64 {
+	if m != nil {
+		return m.OversizeFrames
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_64Octets() uint64 {
+	if m != nil {
+		return m.Frames_64Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_65To_127Octets() uint64 {
+	if m != nil {
+		return m.Frames_65To_127Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_128To_255Octets() uint64 {
+	if m != nil {
+		return m.Frames_128To_255Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_256To_511Octets() uint64 {
+	if m != nil {
+		return m.Frames_256To_511Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_512To_1023Octets() uint64 {
+	if m != nil {
+		return m.Frames_512To_1023Octets
+	}
+	return 0
+}
+
+func (m *OmciEthernetFrameExtendedPm) GetFrames_1024To_1518Octets() uint64 {
+	if m != nil {
+		return m.Frames_1024To_1518Octets
+	}
+	return 0
+}
+
+type GetOmciEthernetFrameExtendedPmResponse struct {
+	Upstream             *OmciEthernetFrameExtendedPm `protobuf:"bytes,1,opt,name=upstream,proto3" json:"upstream,omitempty"`
+	Downstream           *OmciEthernetFrameExtendedPm `protobuf:"bytes,2,opt,name=downstream,proto3" json:"downstream,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
+	XXX_unrecognized     []byte                       `json:"-"`
+	XXX_sizecache        int32                        `json:"-"`
+}
+
+func (m *GetOmciEthernetFrameExtendedPmResponse) Reset() {
+	*m = GetOmciEthernetFrameExtendedPmResponse{}
+}
+func (m *GetOmciEthernetFrameExtendedPmResponse) String() string { return proto.CompactTextString(m) }
+func (*GetOmciEthernetFrameExtendedPmResponse) ProtoMessage()    {}
+func (*GetOmciEthernetFrameExtendedPmResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{17}
+}
+
+func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Unmarshal(m, b)
+}
+func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Marshal(b, m, deterministic)
+}
+func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Merge(m, src)
+}
+func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_Size() int {
+	return xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.Size(m)
+}
+func (m *GetOmciEthernetFrameExtendedPmResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetOmciEthernetFrameExtendedPmResponse proto.InternalMessageInfo
+
+func (m *GetOmciEthernetFrameExtendedPmResponse) GetUpstream() *OmciEthernetFrameExtendedPm {
+	if m != nil {
+		return m.Upstream
+	}
+	return nil
+}
+
+func (m *GetOmciEthernetFrameExtendedPmResponse) GetDownstream() *OmciEthernetFrameExtendedPm {
+	if m != nil {
+		return m.Downstream
+	}
+	return nil
+}
+
+type GetRxPowerResponse struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	FailReason           string   `protobuf:"bytes,4,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
+	RxPower              float64  `protobuf:"fixed64,5,opt,name=rx_power,json=rxPower,proto3" json:"rx_power,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GetRxPowerResponse) Reset()         { *m = GetRxPowerResponse{} }
+func (m *GetRxPowerResponse) String() string { return proto.CompactTextString(m) }
+func (*GetRxPowerResponse) ProtoMessage()    {}
+func (*GetRxPowerResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{18}
+}
+
+func (m *GetRxPowerResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetRxPowerResponse.Unmarshal(m, b)
+}
+func (m *GetRxPowerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetRxPowerResponse.Marshal(b, m, deterministic)
+}
+func (m *GetRxPowerResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetRxPowerResponse.Merge(m, src)
+}
+func (m *GetRxPowerResponse) XXX_Size() int {
+	return xxx_messageInfo_GetRxPowerResponse.Size(m)
+}
+func (m *GetRxPowerResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetRxPowerResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetRxPowerResponse proto.InternalMessageInfo
+
+func (m *GetRxPowerResponse) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *GetRxPowerResponse) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *GetRxPowerResponse) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *GetRxPowerResponse) GetFailReason() string {
+	if m != nil {
+		return m.FailReason
+	}
+	return ""
+}
+
+func (m *GetRxPowerResponse) GetRxPower() float64 {
+	if m != nil {
+		return m.RxPower
+	}
+	return 0
+}
+
+type GetValueRequest struct {
+	// Types that are valid to be assigned to Request:
+	//	*GetValueRequest_Distance
+	//	*GetValueRequest_UniInfo
+	//	*GetValueRequest_OltPortInfo
+	//	*GetValueRequest_OnuOpticalInfo
+	//	*GetValueRequest_EthBridgePort
+	//	*GetValueRequest_FecHistory
+	//	*GetValueRequest_OnuPonInfo
+	//	*GetValueRequest_OnuInfo
+	//	*GetValueRequest_RxPower
+	Request              isGetValueRequest_Request `protobuf_oneof:"request"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *GetValueRequest) Reset()         { *m = GetValueRequest{} }
+func (m *GetValueRequest) String() string { return proto.CompactTextString(m) }
+func (*GetValueRequest) ProtoMessage()    {}
+func (*GetValueRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{19}
+}
+
+func (m *GetValueRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetValueRequest.Unmarshal(m, b)
+}
+func (m *GetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetValueRequest.Marshal(b, m, deterministic)
+}
+func (m *GetValueRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetValueRequest.Merge(m, src)
+}
+func (m *GetValueRequest) XXX_Size() int {
+	return xxx_messageInfo_GetValueRequest.Size(m)
+}
+func (m *GetValueRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetValueRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetValueRequest proto.InternalMessageInfo
+
+type isGetValueRequest_Request interface {
+	isGetValueRequest_Request()
+}
+
+type GetValueRequest_Distance struct {
+	Distance *GetDistanceRequest `protobuf:"bytes,1,opt,name=distance,proto3,oneof"`
+}
+
+type GetValueRequest_UniInfo struct {
+	UniInfo *GetOnuUniInfoRequest `protobuf:"bytes,2,opt,name=uniInfo,proto3,oneof"`
+}
+
+type GetValueRequest_OltPortInfo struct {
+	OltPortInfo *GetOltPortCounters `protobuf:"bytes,3,opt,name=oltPortInfo,proto3,oneof"`
+}
+
+type GetValueRequest_OnuOpticalInfo struct {
+	OnuOpticalInfo *GetOnuPonOpticalInfo `protobuf:"bytes,4,opt,name=onuOpticalInfo,proto3,oneof"`
+}
+
+type GetValueRequest_EthBridgePort struct {
+	EthBridgePort *GetOnuEthernetBridgePortHistory `protobuf:"bytes,5,opt,name=ethBridgePort,proto3,oneof"`
+}
+
+type GetValueRequest_FecHistory struct {
+	FecHistory *GetOnuFecHistory `protobuf:"bytes,6,opt,name=fecHistory,proto3,oneof"`
+}
+
+type GetValueRequest_OnuPonInfo struct {
+	OnuPonInfo *GetOnuCountersRequest `protobuf:"bytes,7,opt,name=onuPonInfo,proto3,oneof"`
+}
+
+type GetValueRequest_OnuInfo struct {
+	OnuInfo *GetOmciEthernetFrameExtendedPmRequest `protobuf:"bytes,8,opt,name=onuInfo,proto3,oneof"`
+}
+
+type GetValueRequest_RxPower struct {
+	RxPower *GetRxPowerRequest `protobuf:"bytes,9,opt,name=rxPower,proto3,oneof"`
+}
+
+func (*GetValueRequest_Distance) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_UniInfo) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_OltPortInfo) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_OnuOpticalInfo) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_EthBridgePort) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_FecHistory) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_OnuPonInfo) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_OnuInfo) isGetValueRequest_Request() {}
+
+func (*GetValueRequest_RxPower) isGetValueRequest_Request() {}
+
+func (m *GetValueRequest) GetRequest() isGetValueRequest_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetDistance() *GetDistanceRequest {
+	if x, ok := m.GetRequest().(*GetValueRequest_Distance); ok {
+		return x.Distance
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetUniInfo() *GetOnuUniInfoRequest {
+	if x, ok := m.GetRequest().(*GetValueRequest_UniInfo); ok {
+		return x.UniInfo
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetOltPortInfo() *GetOltPortCounters {
+	if x, ok := m.GetRequest().(*GetValueRequest_OltPortInfo); ok {
+		return x.OltPortInfo
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetOnuOpticalInfo() *GetOnuPonOpticalInfo {
+	if x, ok := m.GetRequest().(*GetValueRequest_OnuOpticalInfo); ok {
+		return x.OnuOpticalInfo
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetEthBridgePort() *GetOnuEthernetBridgePortHistory {
+	if x, ok := m.GetRequest().(*GetValueRequest_EthBridgePort); ok {
+		return x.EthBridgePort
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetFecHistory() *GetOnuFecHistory {
+	if x, ok := m.GetRequest().(*GetValueRequest_FecHistory); ok {
+		return x.FecHistory
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetOnuPonInfo() *GetOnuCountersRequest {
+	if x, ok := m.GetRequest().(*GetValueRequest_OnuPonInfo); ok {
+		return x.OnuPonInfo
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetOnuInfo() *GetOmciEthernetFrameExtendedPmRequest {
+	if x, ok := m.GetRequest().(*GetValueRequest_OnuInfo); ok {
+		return x.OnuInfo
+	}
+	return nil
+}
+
+func (m *GetValueRequest) GetRxPower() *GetRxPowerRequest {
+	if x, ok := m.GetRequest().(*GetValueRequest_RxPower); ok {
+		return x.RxPower
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*GetValueRequest) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*GetValueRequest_Distance)(nil),
+		(*GetValueRequest_UniInfo)(nil),
+		(*GetValueRequest_OltPortInfo)(nil),
+		(*GetValueRequest_OnuOpticalInfo)(nil),
+		(*GetValueRequest_EthBridgePort)(nil),
+		(*GetValueRequest_FecHistory)(nil),
+		(*GetValueRequest_OnuPonInfo)(nil),
+		(*GetValueRequest_OnuInfo)(nil),
+		(*GetValueRequest_RxPower)(nil),
+	}
+}
+
+type GetValueResponse struct {
+	Status    GetValueResponse_Status      `protobuf:"varint,1,opt,name=status,proto3,enum=extension.GetValueResponse_Status" json:"status,omitempty"`
+	ErrReason GetValueResponse_ErrorReason `protobuf:"varint,2,opt,name=errReason,proto3,enum=extension.GetValueResponse_ErrorReason" json:"errReason,omitempty"`
+	// Types that are valid to be assigned to Response:
+	//	*GetValueResponse_Distance
+	//	*GetValueResponse_UniInfo
+	//	*GetValueResponse_PortCoutners
+	//	*GetValueResponse_OnuOpticalInfo
+	//	*GetValueResponse_EthBridgePortInfo
+	//	*GetValueResponse_FecHistory
+	//	*GetValueResponse_OnuPonCounters
+	//	*GetValueResponse_OnuCounters
+	//	*GetValueResponse_RxPower
+	Response             isGetValueResponse_Response `protobuf_oneof:"response"`
+	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
+	XXX_unrecognized     []byte                      `json:"-"`
+	XXX_sizecache        int32                       `json:"-"`
+}
+
+func (m *GetValueResponse) Reset()         { *m = GetValueResponse{} }
+func (m *GetValueResponse) String() string { return proto.CompactTextString(m) }
+func (*GetValueResponse) ProtoMessage()    {}
+func (*GetValueResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{20}
+}
+
+func (m *GetValueResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetValueResponse.Unmarshal(m, b)
+}
+func (m *GetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetValueResponse.Marshal(b, m, deterministic)
+}
+func (m *GetValueResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetValueResponse.Merge(m, src)
+}
+func (m *GetValueResponse) XXX_Size() int {
+	return xxx_messageInfo_GetValueResponse.Size(m)
+}
+func (m *GetValueResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetValueResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetValueResponse proto.InternalMessageInfo
+
+func (m *GetValueResponse) GetStatus() GetValueResponse_Status {
+	if m != nil {
+		return m.Status
+	}
+	return GetValueResponse_STATUS_UNDEFINED
+}
+
+func (m *GetValueResponse) GetErrReason() GetValueResponse_ErrorReason {
+	if m != nil {
+		return m.ErrReason
+	}
+	return GetValueResponse_REASON_UNDEFINED
+}
+
+type isGetValueResponse_Response interface {
+	isGetValueResponse_Response()
+}
+
+type GetValueResponse_Distance struct {
+	Distance *GetDistanceResponse `protobuf:"bytes,3,opt,name=distance,proto3,oneof"`
+}
+
+type GetValueResponse_UniInfo struct {
+	UniInfo *GetOnuUniInfoResponse `protobuf:"bytes,4,opt,name=uniInfo,proto3,oneof"`
+}
+
+type GetValueResponse_PortCoutners struct {
+	PortCoutners *GetOltPortCountersResponse `protobuf:"bytes,5,opt,name=portCoutners,proto3,oneof"`
+}
+
+type GetValueResponse_OnuOpticalInfo struct {
+	OnuOpticalInfo *GetOnuPonOpticalInfoResponse `protobuf:"bytes,6,opt,name=onuOpticalInfo,proto3,oneof"`
+}
+
+type GetValueResponse_EthBridgePortInfo struct {
+	EthBridgePortInfo *GetOnuEthernetBridgePortHistoryResponse `protobuf:"bytes,7,opt,name=ethBridgePortInfo,proto3,oneof"`
+}
+
+type GetValueResponse_FecHistory struct {
+	FecHistory *GetOnuFecHistoryResponse `protobuf:"bytes,8,opt,name=fecHistory,proto3,oneof"`
+}
+
+type GetValueResponse_OnuPonCounters struct {
+	OnuPonCounters *GetOnuCountersResponse `protobuf:"bytes,9,opt,name=onuPonCounters,proto3,oneof"`
+}
+
+type GetValueResponse_OnuCounters struct {
+	OnuCounters *GetOmciEthernetFrameExtendedPmResponse `protobuf:"bytes,10,opt,name=onuCounters,proto3,oneof"`
+}
+
+type GetValueResponse_RxPower struct {
+	RxPower *GetRxPowerResponse `protobuf:"bytes,11,opt,name=rxPower,proto3,oneof"`
+}
+
+func (*GetValueResponse_Distance) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_UniInfo) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_PortCoutners) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_OnuOpticalInfo) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_EthBridgePortInfo) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_FecHistory) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_OnuPonCounters) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_OnuCounters) isGetValueResponse_Response() {}
+
+func (*GetValueResponse_RxPower) isGetValueResponse_Response() {}
+
+func (m *GetValueResponse) GetResponse() isGetValueResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetDistance() *GetDistanceResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_Distance); ok {
+		return x.Distance
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetUniInfo() *GetOnuUniInfoResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_UniInfo); ok {
+		return x.UniInfo
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetPortCoutners() *GetOltPortCountersResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_PortCoutners); ok {
+		return x.PortCoutners
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetOnuOpticalInfo() *GetOnuPonOpticalInfoResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_OnuOpticalInfo); ok {
+		return x.OnuOpticalInfo
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetEthBridgePortInfo() *GetOnuEthernetBridgePortHistoryResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_EthBridgePortInfo); ok {
+		return x.EthBridgePortInfo
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetFecHistory() *GetOnuFecHistoryResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_FecHistory); ok {
+		return x.FecHistory
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetOnuPonCounters() *GetOnuCountersResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_OnuPonCounters); ok {
+		return x.OnuPonCounters
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetOnuCounters() *GetOmciEthernetFrameExtendedPmResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_OnuCounters); ok {
+		return x.OnuCounters
+	}
+	return nil
+}
+
+func (m *GetValueResponse) GetRxPower() *GetRxPowerResponse {
+	if x, ok := m.GetResponse().(*GetValueResponse_RxPower); ok {
+		return x.RxPower
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*GetValueResponse) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*GetValueResponse_Distance)(nil),
+		(*GetValueResponse_UniInfo)(nil),
+		(*GetValueResponse_PortCoutners)(nil),
+		(*GetValueResponse_OnuOpticalInfo)(nil),
+		(*GetValueResponse_EthBridgePortInfo)(nil),
+		(*GetValueResponse_FecHistory)(nil),
+		(*GetValueResponse_OnuPonCounters)(nil),
+		(*GetValueResponse_OnuCounters)(nil),
+		(*GetValueResponse_RxPower)(nil),
+	}
+}
+
+type SetValueRequest struct {
+	// Types that are valid to be assigned to Request:
+	//	*SetValueRequest_AlarmConfig
+	Request              isSetValueRequest_Request `protobuf_oneof:"request"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *SetValueRequest) Reset()         { *m = SetValueRequest{} }
+func (m *SetValueRequest) String() string { return proto.CompactTextString(m) }
+func (*SetValueRequest) ProtoMessage()    {}
+func (*SetValueRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{21}
+}
+
+func (m *SetValueRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SetValueRequest.Unmarshal(m, b)
+}
+func (m *SetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SetValueRequest.Marshal(b, m, deterministic)
+}
+func (m *SetValueRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SetValueRequest.Merge(m, src)
+}
+func (m *SetValueRequest) XXX_Size() int {
+	return xxx_messageInfo_SetValueRequest.Size(m)
+}
+func (m *SetValueRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_SetValueRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SetValueRequest proto.InternalMessageInfo
+
+type isSetValueRequest_Request interface {
+	isSetValueRequest_Request()
+}
+
+type SetValueRequest_AlarmConfig struct {
+	AlarmConfig *config.AlarmConfig `protobuf:"bytes,1,opt,name=alarm_config,json=alarmConfig,proto3,oneof"`
+}
+
+func (*SetValueRequest_AlarmConfig) isSetValueRequest_Request() {}
+
+func (m *SetValueRequest) GetRequest() isSetValueRequest_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (m *SetValueRequest) GetAlarmConfig() *config.AlarmConfig {
+	if x, ok := m.GetRequest().(*SetValueRequest_AlarmConfig); ok {
+		return x.AlarmConfig
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*SetValueRequest) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*SetValueRequest_AlarmConfig)(nil),
+	}
+}
+
+type SetValueResponse struct {
+	Status               SetValueResponse_Status      `protobuf:"varint,1,opt,name=status,proto3,enum=extension.SetValueResponse_Status" json:"status,omitempty"`
+	ErrReason            SetValueResponse_ErrorReason `protobuf:"varint,2,opt,name=errReason,proto3,enum=extension.SetValueResponse_ErrorReason" json:"errReason,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
+	XXX_unrecognized     []byte                       `json:"-"`
+	XXX_sizecache        int32                        `json:"-"`
+}
+
+func (m *SetValueResponse) Reset()         { *m = SetValueResponse{} }
+func (m *SetValueResponse) String() string { return proto.CompactTextString(m) }
+func (*SetValueResponse) ProtoMessage()    {}
+func (*SetValueResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{22}
+}
+
+func (m *SetValueResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SetValueResponse.Unmarshal(m, b)
+}
+func (m *SetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SetValueResponse.Marshal(b, m, deterministic)
+}
+func (m *SetValueResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SetValueResponse.Merge(m, src)
+}
+func (m *SetValueResponse) XXX_Size() int {
+	return xxx_messageInfo_SetValueResponse.Size(m)
+}
+func (m *SetValueResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_SetValueResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SetValueResponse proto.InternalMessageInfo
+
+func (m *SetValueResponse) GetStatus() SetValueResponse_Status {
+	if m != nil {
+		return m.Status
+	}
+	return SetValueResponse_STATUS_UNDEFINED
+}
+
+func (m *SetValueResponse) GetErrReason() SetValueResponse_ErrorReason {
+	if m != nil {
+		return m.ErrReason
+	}
+	return SetValueResponse_REASON_UNDEFINED
+}
+
+type SingleGetValueRequest struct {
+	TargetId             string           `protobuf:"bytes,1,opt,name=targetId,proto3" json:"targetId,omitempty"`
+	Request              *GetValueRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *SingleGetValueRequest) Reset()         { *m = SingleGetValueRequest{} }
+func (m *SingleGetValueRequest) String() string { return proto.CompactTextString(m) }
+func (*SingleGetValueRequest) ProtoMessage()    {}
+func (*SingleGetValueRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{23}
+}
+
+func (m *SingleGetValueRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SingleGetValueRequest.Unmarshal(m, b)
+}
+func (m *SingleGetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SingleGetValueRequest.Marshal(b, m, deterministic)
+}
+func (m *SingleGetValueRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SingleGetValueRequest.Merge(m, src)
+}
+func (m *SingleGetValueRequest) XXX_Size() int {
+	return xxx_messageInfo_SingleGetValueRequest.Size(m)
+}
+func (m *SingleGetValueRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_SingleGetValueRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SingleGetValueRequest proto.InternalMessageInfo
+
+func (m *SingleGetValueRequest) GetTargetId() string {
+	if m != nil {
+		return m.TargetId
+	}
+	return ""
+}
+
+func (m *SingleGetValueRequest) GetRequest() *GetValueRequest {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+type SingleGetValueResponse struct {
+	Response             *GetValueResponse `protobuf:"bytes,1,opt,name=response,proto3" json:"response,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *SingleGetValueResponse) Reset()         { *m = SingleGetValueResponse{} }
+func (m *SingleGetValueResponse) String() string { return proto.CompactTextString(m) }
+func (*SingleGetValueResponse) ProtoMessage()    {}
+func (*SingleGetValueResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{24}
+}
+
+func (m *SingleGetValueResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SingleGetValueResponse.Unmarshal(m, b)
+}
+func (m *SingleGetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SingleGetValueResponse.Marshal(b, m, deterministic)
+}
+func (m *SingleGetValueResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SingleGetValueResponse.Merge(m, src)
+}
+func (m *SingleGetValueResponse) XXX_Size() int {
+	return xxx_messageInfo_SingleGetValueResponse.Size(m)
+}
+func (m *SingleGetValueResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_SingleGetValueResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SingleGetValueResponse proto.InternalMessageInfo
+
+func (m *SingleGetValueResponse) GetResponse() *GetValueResponse {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+type SingleSetValueRequest struct {
+	TargetId             string           `protobuf:"bytes,1,opt,name=targetId,proto3" json:"targetId,omitempty"`
+	Request              *SetValueRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *SingleSetValueRequest) Reset()         { *m = SingleSetValueRequest{} }
+func (m *SingleSetValueRequest) String() string { return proto.CompactTextString(m) }
+func (*SingleSetValueRequest) ProtoMessage()    {}
+func (*SingleSetValueRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{25}
+}
+
+func (m *SingleSetValueRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SingleSetValueRequest.Unmarshal(m, b)
+}
+func (m *SingleSetValueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SingleSetValueRequest.Marshal(b, m, deterministic)
+}
+func (m *SingleSetValueRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SingleSetValueRequest.Merge(m, src)
+}
+func (m *SingleSetValueRequest) XXX_Size() int {
+	return xxx_messageInfo_SingleSetValueRequest.Size(m)
+}
+func (m *SingleSetValueRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_SingleSetValueRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SingleSetValueRequest proto.InternalMessageInfo
+
+func (m *SingleSetValueRequest) GetTargetId() string {
+	if m != nil {
+		return m.TargetId
+	}
+	return ""
+}
+
+func (m *SingleSetValueRequest) GetRequest() *SetValueRequest {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+type SingleSetValueResponse struct {
+	Response             *SetValueResponse `protobuf:"bytes,1,opt,name=response,proto3" json:"response,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *SingleSetValueResponse) Reset()         { *m = SingleSetValueResponse{} }
+func (m *SingleSetValueResponse) String() string { return proto.CompactTextString(m) }
+func (*SingleSetValueResponse) ProtoMessage()    {}
+func (*SingleSetValueResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ecf6e9799a9202d, []int{26}
+}
+
+func (m *SingleSetValueResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SingleSetValueResponse.Unmarshal(m, b)
+}
+func (m *SingleSetValueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SingleSetValueResponse.Marshal(b, m, deterministic)
+}
+func (m *SingleSetValueResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SingleSetValueResponse.Merge(m, src)
+}
+func (m *SingleSetValueResponse) XXX_Size() int {
+	return xxx_messageInfo_SingleSetValueResponse.Size(m)
+}
+func (m *SingleSetValueResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_SingleSetValueResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SingleSetValueResponse proto.InternalMessageInfo
+
+func (m *SingleSetValueResponse) GetResponse() *SetValueResponse {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("extension.GetOnuUniInfoResponse_ConfigurationInd", GetOnuUniInfoResponse_ConfigurationInd_name, GetOnuUniInfoResponse_ConfigurationInd_value)
+	proto.RegisterEnum("extension.GetOnuUniInfoResponse_AdministrativeState", GetOnuUniInfoResponse_AdministrativeState_name, GetOnuUniInfoResponse_AdministrativeState_value)
+	proto.RegisterEnum("extension.GetOnuUniInfoResponse_OperationalState", GetOnuUniInfoResponse_OperationalState_name, GetOnuUniInfoResponse_OperationalState_value)
+	proto.RegisterEnum("extension.GetOltPortCounters_PortType", GetOltPortCounters_PortType_name, GetOltPortCounters_PortType_value)
+	proto.RegisterEnum("extension.GetOnuEthernetBridgePortHistory_Direction", GetOnuEthernetBridgePortHistory_Direction_name, GetOnuEthernetBridgePortHistory_Direction_value)
+	proto.RegisterEnum("extension.GetValueResponse_Status", GetValueResponse_Status_name, GetValueResponse_Status_value)
+	proto.RegisterEnum("extension.GetValueResponse_ErrorReason", GetValueResponse_ErrorReason_name, GetValueResponse_ErrorReason_value)
+	proto.RegisterEnum("extension.SetValueResponse_Status", SetValueResponse_Status_name, SetValueResponse_Status_value)
+	proto.RegisterEnum("extension.SetValueResponse_ErrorReason", SetValueResponse_ErrorReason_name, SetValueResponse_ErrorReason_value)
+	proto.RegisterType((*GetDistanceRequest)(nil), "extension.GetDistanceRequest")
+	proto.RegisterType((*GetDistanceResponse)(nil), "extension.GetDistanceResponse")
+	proto.RegisterType((*GetOnuUniInfoRequest)(nil), "extension.GetOnuUniInfoRequest")
+	proto.RegisterType((*GetOnuUniInfoResponse)(nil), "extension.GetOnuUniInfoResponse")
+	proto.RegisterType((*GetOltPortCounters)(nil), "extension.GetOltPortCounters")
+	proto.RegisterType((*GetOltPortCountersResponse)(nil), "extension.GetOltPortCountersResponse")
+	proto.RegisterType((*GetOnuPonOpticalInfo)(nil), "extension.GetOnuPonOpticalInfo")
+	proto.RegisterType((*GetOnuPonOpticalInfoResponse)(nil), "extension.GetOnuPonOpticalInfoResponse")
+	proto.RegisterType((*GetOnuEthernetBridgePortHistory)(nil), "extension.GetOnuEthernetBridgePortHistory")
+	proto.RegisterType((*GetOnuEthernetBridgePortHistoryResponse)(nil), "extension.GetOnuEthernetBridgePortHistoryResponse")
+	proto.RegisterType((*GetOnuFecHistory)(nil), "extension.GetOnuFecHistory")
+	proto.RegisterType((*GetOnuFecHistoryResponse)(nil), "extension.GetOnuFecHistoryResponse")
+	proto.RegisterType((*GetOnuCountersRequest)(nil), "extension.GetOnuCountersRequest")
+	proto.RegisterType((*GetOmciEthernetFrameExtendedPmRequest)(nil), "extension.GetOmciEthernetFrameExtendedPmRequest")
+	proto.RegisterType((*GetRxPowerRequest)(nil), "extension.GetRxPowerRequest")
+	proto.RegisterType((*GetOnuCountersResponse)(nil), "extension.GetOnuCountersResponse")
+	proto.RegisterType((*OmciEthernetFrameExtendedPm)(nil), "extension.OmciEthernetFrameExtendedPm")
+	proto.RegisterType((*GetOmciEthernetFrameExtendedPmResponse)(nil), "extension.GetOmciEthernetFrameExtendedPmResponse")
+	proto.RegisterType((*GetRxPowerResponse)(nil), "extension.GetRxPowerResponse")
+	proto.RegisterType((*GetValueRequest)(nil), "extension.GetValueRequest")
+	proto.RegisterType((*GetValueResponse)(nil), "extension.GetValueResponse")
+	proto.RegisterType((*SetValueRequest)(nil), "extension.SetValueRequest")
+	proto.RegisterType((*SetValueResponse)(nil), "extension.SetValueResponse")
+	proto.RegisterType((*SingleGetValueRequest)(nil), "extension.SingleGetValueRequest")
+	proto.RegisterType((*SingleGetValueResponse)(nil), "extension.SingleGetValueResponse")
+	proto.RegisterType((*SingleSetValueRequest)(nil), "extension.SingleSetValueRequest")
+	proto.RegisterType((*SingleSetValueResponse)(nil), "extension.SingleSetValueResponse")
+}
+
+func init() { proto.RegisterFile("voltha_protos/extensions.proto", fileDescriptor_7ecf6e9799a9202d) }
+
+var fileDescriptor_7ecf6e9799a9202d = []byte{
+	// 2978 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x5a, 0xcb, 0x72, 0x1b, 0xc7,
+	0xd5, 0x26, 0x20, 0x72, 0x08, 0x1c, 0x10, 0x24, 0xd8, 0xbc, 0x41, 0xa4, 0x6e, 0x1e, 0x97, 0x25,
+	0xfd, 0x2e, 0x1b, 0x12, 0x60, 0x42, 0xe6, 0x2f, 0x3b, 0x55, 0xe6, 0x10, 0x43, 0x02, 0x25, 0x0a,
+	0x80, 0x1b, 0x80, 0xec, 0x64, 0x33, 0x35, 0xc4, 0x34, 0xa9, 0x29, 0x03, 0x33, 0xc8, 0x4c, 0x43,
+	0x86, 0xb2, 0x4d, 0x96, 0xd9, 0x65, 0x93, 0x67, 0x48, 0x55, 0x2a, 0x8b, 0xa4, 0x2a, 0xfb, 0xac,
+	0xf3, 0x12, 0xa9, 0xca, 0x13, 0xe4, 0x01, 0x52, 0xa9, 0xbe, 0xcc, 0x15, 0x20, 0x25, 0x39, 0xd9,
+	0xb1, 0xcf, 0xf9, 0xbe, 0x6f, 0x7a, 0xba, 0xcf, 0x39, 0x7d, 0xa6, 0x41, 0xb8, 0xf7, 0xc6, 0x1d,
+	0xd1, 0xd7, 0xa6, 0x31, 0xf1, 0x5c, 0xea, 0xfa, 0x4f, 0xc8, 0x8c, 0x12, 0xc7, 0xb7, 0x5d, 0xc7,
+	0xaf, 0x70, 0x0b, 0xca, 0x87, 0x96, 0xfd, 0x79, 0xa8, 0x31, 0x74, 0x9d, 0x4b, 0xfb, 0x4a, 0x40,
+	0xf7, 0x0f, 0xae, 0x5c, 0xf7, 0x6a, 0x44, 0x9e, 0xf0, 0xd1, 0xc5, 0xf4, 0xf2, 0x09, 0x19, 0x4f,
+	0xe8, 0x5b, 0xe1, 0x54, 0x9f, 0x01, 0x3a, 0x23, 0xb4, 0x61, 0xfb, 0xd4, 0x74, 0x86, 0x04, 0x93,
+	0x5f, 0x4e, 0x89, 0x4f, 0xd1, 0x03, 0x28, 0xb8, 0xce, 0xb4, 0x41, 0xde, 0xd8, 0x43, 0xd2, 0xb2,
+	0xca, 0x99, 0x07, 0x99, 0xc7, 0x79, 0x1c, 0x37, 0xa9, 0x55, 0xd8, 0x4a, 0xf0, 0xfc, 0x89, 0xeb,
+	0xf8, 0x04, 0xed, 0x43, 0xce, 0x92, 0x36, 0xce, 0x2a, 0xe2, 0x70, 0xac, 0xd6, 0x60, 0xfb, 0x8c,
+	0xd0, 0x8e, 0x33, 0x1d, 0x38, 0x76, 0xcb, 0xb9, 0x74, 0x83, 0x87, 0xed, 0x43, 0x6e, 0xca, 0x2c,
+	0x16, 0x99, 0x05, 0x9c, 0x60, 0xac, 0xfe, 0x63, 0x19, 0x76, 0x52, 0x24, 0xf9, 0xa4, 0x2e, 0xe4,
+	0x4c, 0x6b, 0xdc, 0xa3, 0x26, 0x15, 0x4f, 0x5a, 0xaf, 0x1d, 0x56, 0xc2, 0x35, 0xa9, 0x2c, 0xe4,
+	0x54, 0x8e, 0xad, 0xb1, 0xed, 0xd8, 0x3e, 0xf5, 0x4c, 0x6a, 0xbf, 0x21, 0x9c, 0x8b, 0x43, 0x15,
+	0xd4, 0x81, 0xbc, 0x3b, 0x21, 0x9e, 0x90, 0xcc, 0x72, 0xc9, 0xea, 0x3b, 0x25, 0x3b, 0x13, 0xc2,
+	0xd4, 0x5c, 0xc7, 0x1c, 0x09, 0xbd, 0x48, 0x83, 0x09, 0x8a, 0x8d, 0x68, 0x39, 0x56, 0xf9, 0xd6,
+	0x7b, 0x0a, 0x9e, 0x70, 0xc6, 0x54, 0x88, 0xb6, 0x1c, 0x0b, 0x47, 0x1a, 0xea, 0xdf, 0x32, 0x50,
+	0x4a, 0xfb, 0x11, 0x80, 0x32, 0x68, 0xbf, 0xe8, 0x7c, 0xd7, 0x2e, 0x2d, 0x21, 0x04, 0xeb, 0x7d,
+	0xbd, 0x6d, 0x68, 0xc7, 0x3d, 0xdd, 0xe8, 0x1b, 0xa7, 0x8d, 0xef, 0x4b, 0x19, 0xb4, 0x0b, 0xa8,
+	0x39, 0x68, 0x37, 0xb0, 0xde, 0x88, 0xdb, 0xb3, 0xa8, 0x0c, 0xdb, 0x67, 0xad, 0xb3, 0x63, 0xad,
+	0xd5, 0x37, 0xf4, 0x7e, 0x53, 0xc7, 0x6d, 0x5d, 0x78, 0x6e, 0x31, 0x06, 0x53, 0x39, 0x4b, 0xda,
+	0x97, 0x53, 0xea, 0xcd, 0xc6, 0xf7, 0xa5, 0x95, 0x05, 0xea, 0xcc, 0xae, 0x2c, 0x54, 0x67, 0x9e,
+	0x55, 0xf5, 0x0c, 0xb6, 0x16, 0xec, 0x03, 0x13, 0x3a, 0x6e, 0xbc, 0xec, 0xf5, 0x8f, 0xfb, 0xba,
+	0x31, 0x68, 0x37, 0xf4, 0xd3, 0x56, 0x5b, 0x6f, 0x94, 0x96, 0xd8, 0xeb, 0x9d, 0x77, 0x4e, 0x5e,
+	0xe8, 0x8d, 0x52, 0x06, 0xad, 0x41, 0x6e, 0xd0, 0x96, 0xa3, 0xac, 0x7a, 0x0a, 0xa5, 0xf4, 0xea,
+	0xa3, 0x3d, 0xd8, 0xea, 0x74, 0x75, 0x3c, 0x2f, 0x53, 0x80, 0x55, 0xbd, 0x7d, 0xac, 0x9d, 0x07,
+	0x3a, 0x8d, 0x56, 0x4f, 0x8c, 0xb2, 0xea, 0x5f, 0x33, 0x3c, 0x07, 0x3a, 0x23, 0xda, 0x75, 0x3d,
+	0x7a, 0xe2, 0x4e, 0x1d, 0x4a, 0x3c, 0x1f, 0xed, 0x82, 0x32, 0x71, 0x3d, 0xda, 0x76, 0x65, 0x50,
+	0xca, 0x11, 0xd2, 0x20, 0xc7, 0xfe, 0xea, 0xbf, 0x9d, 0x04, 0x51, 0xf2, 0x30, 0xb5, 0xa9, 0x49,
+	0xa1, 0x4a, 0x57, 0xa2, 0x71, 0xc8, 0x53, 0x75, 0xc8, 0x05, 0x56, 0x54, 0x82, 0x35, 0xf6, 0xb7,
+	0x31, 0x68, 0xbf, 0x68, 0x8b, 0x5d, 0xdc, 0x81, 0x4d, 0x6e, 0x09, 0x17, 0xae, 0xdd, 0x6e, 0x95,
+	0x32, 0x21, 0xb0, 0xdb, 0x69, 0x1b, 0x9d, 0xf3, 0x7e, 0x29, 0xab, 0xfe, 0xfd, 0x16, 0xec, 0xcf,
+	0x3f, 0x30, 0x4c, 0x91, 0x32, 0xac, 0xd2, 0x99, 0xf6, 0x96, 0x12, 0x9f, 0xbf, 0xc2, 0x32, 0x0e,
+	0x86, 0xcc, 0xe3, 0x49, 0x4f, 0x56, 0x78, 0xe4, 0x10, 0xdd, 0x81, 0x3c, 0x9d, 0x75, 0xcd, 0xe1,
+	0x0f, 0x84, 0xfa, 0x3c, 0x66, 0x97, 0x71, 0x64, 0x60, 0x5e, 0x2f, 0xf4, 0x2e, 0x0b, 0x6f, 0x68,
+	0x40, 0x0f, 0x61, 0x9d, 0xce, 0x74, 0xcf, 0x73, 0xbd, 0x00, 0xb2, 0xc2, 0x21, 0x29, 0x2b, 0xc3,
+	0x79, 0x49, 0x9c, 0x22, 0x70, 0xde, 0x1c, 0x8e, 0xce, 0xb4, 0xa1, 0xe9, 0xd3, 0x00, 0xb7, 0x1a,
+	0xe8, 0xc5, 0xad, 0x42, 0x2f, 0x81, 0xcb, 0x05, 0x7a, 0x69, 0x1c, 0x9d, 0x0d, 0xe2, 0xb8, 0x7c,
+	0xa0, 0x37, 0x98, 0xd3, 0x4b, 0xe0, 0x20, 0xd0, 0x1b, 0xcc, 0xe9, 0xbd, 0x8c, 0xe3, 0x0a, 0x81,
+	0xde, 0xcb, 0x39, 0xbd, 0x04, 0x6e, 0x2d, 0xd0, 0x8b, 0x5b, 0xd5, 0x46, 0x50, 0x20, 0xbb, 0xae,
+	0xd3, 0x99, 0x50, 0x7b, 0x68, 0x8e, 0x58, 0x69, 0x40, 0x9f, 0xc1, 0x0a, 0x2f, 0xd9, 0x7c, 0x17,
+	0x0b, 0xb5, 0xdd, 0x8a, 0x28, 0xe8, 0x95, 0xa0, 0xa0, 0x57, 0x74, 0xe6, 0xc5, 0x02, 0xa4, 0xfe,
+	0x26, 0x0b, 0x77, 0x16, 0xc9, 0x84, 0x61, 0xf1, 0x29, 0x94, 0x26, 0xee, 0x8f, 0xc4, 0x3b, 0x25,
+	0xc4, 0x7a, 0xe5, 0x8e, 0xa8, 0x79, 0x25, 0x2a, 0x68, 0x16, 0xcf, 0xd9, 0x51, 0x0d, 0xb6, 0x3d,
+	0x32, 0x24, 0xf6, 0x1b, 0x62, 0x49, 0xa9, 0x2e, 0x83, 0xf0, 0xa8, 0xc9, 0xe2, 0x85, 0x3e, 0xf4,
+	0x0c, 0x76, 0xc7, 0xc4, 0x0c, 0x1e, 0x7d, 0x6e, 0x4e, 0x9d, 0xe1, 0x6b, 0xc1, 0xba, 0xc5, 0x59,
+	0xd7, 0x78, 0xd9, 0xbc, 0x46, 0xa6, 0x4f, 0x3c, 0xcd, 0x36, 0xfd, 0x93, 0xa9, 0xe7, 0x11, 0x87,
+	0xf2, 0x18, 0xcb, 0xe2, 0x39, 0x3b, 0x3b, 0xa0, 0x28, 0x19, 0xf3, 0xec, 0x9f, 0x7a, 0x84, 0xc7,
+	0x59, 0x16, 0xc7, 0x4d, 0xea, 0x9f, 0x32, 0x70, 0x5f, 0x2c, 0x83, 0x4e, 0x5f, 0x13, 0xcf, 0x21,
+	0x54, 0xf3, 0x6c, 0xeb, 0x8a, 0xb0, 0x4c, 0x69, 0xda, 0x3e, 0x75, 0xbd, 0xb7, 0x08, 0x43, 0xde,
+	0xb2, 0x3d, 0x32, 0x64, 0x15, 0xe4, 0xda, 0x43, 0xe4, 0x5a, 0x7a, 0xa5, 0x11, 0x70, 0x71, 0x24,
+	0xa3, 0x1e, 0x41, 0x3e, 0xb4, 0xa3, 0x22, 0xe4, 0xe3, 0x45, 0x88, 0xd5, 0xaf, 0x6e, 0xaf, 0x8f,
+	0xf5, 0xe3, 0x97, 0xa5, 0x0c, 0x5a, 0x07, 0x68, 0x74, 0xbe, 0x6b, 0xcb, 0x71, 0x56, 0xfd, 0xdd,
+	0x0a, 0x3c, 0x7a, 0xc7, 0x23, 0xc3, 0x3d, 0xbc, 0x07, 0x60, 0x79, 0xee, 0x44, 0x7f, 0x43, 0x1c,
+	0xea, 0xcb, 0x02, 0x15, 0xb3, 0xb0, 0xe2, 0xe5, 0x0e, 0x29, 0x0b, 0xb5, 0xac, 0x28, 0x5e, 0x62,
+	0xc4, 0x12, 0x7f, 0x12, 0x4b, 0xee, 0x22, 0x0e, 0x86, 0x6c, 0xf5, 0x2f, 0x3c, 0xd7, 0xb4, 0xe2,
+	0x61, 0xba, 0xcc, 0x21, 0x73, 0x76, 0x86, 0x1d, 0x4f, 0x47, 0x6c, 0x03, 0x23, 0xec, 0x8a, 0xc0,
+	0xa6, 0xed, 0xe8, 0x33, 0xd8, 0x1c, 0x7a, 0x43, 0x9e, 0xd7, 0xc4, 0x8a, 0xe7, 0x7b, 0x11, 0xcf,
+	0x3b, 0x98, 0xf2, 0xd4, 0xb1, 0x88, 0xe7, 0xdb, 0xbf, 0x22, 0xf1, 0xa4, 0x2f, 0xe2, 0x39, 0x3b,
+	0x7a, 0x0c, 0x1b, 0xee, 0x9b, 0x24, 0x34, 0xc7, 0xa1, 0x69, 0x33, 0x43, 0xca, 0xd7, 0x7c, 0x76,
+	0x28, 0x97, 0x25, 0x2f, 0x90, 0x29, 0x33, 0x8b, 0xf7, 0xc0, 0x54, 0xef, 0xbb, 0xd5, 0xda, 0x97,
+	0x12, 0x0e, 0x1c, 0xbe, 0xd0, 0x87, 0x0e, 0x61, 0x47, 0xda, 0xab, 0xb5, 0xa3, 0xbe, 0x5b, 0xab,
+	0xd7, 0x3b, 0x82, 0x54, 0xe0, 0xa4, 0xc5, 0xce, 0x18, 0xab, 0x56, 0x7f, 0xd6, 0x77, 0xeb, 0xd5,
+	0xaa, 0x7c, 0xd4, 0x5a, 0x82, 0x95, 0x74, 0xb2, 0xdc, 0x92, 0x8e, 0x7a, 0xb5, 0xd6, 0x77, 0xab,
+	0x4f, 0x6b, 0x5f, 0x48, 0x5a, 0x91, 0xd3, 0xae, 0xf1, 0xa2, 0x23, 0xd8, 0x0b, 0xa6, 0xf1, 0xb4,
+	0x76, 0xd8, 0x77, 0xab, 0xf5, 0xea, 0x91, 0x24, 0xae, 0x73, 0xe2, 0x75, 0x6e, 0xf5, 0x1b, 0x28,
+	0x89, 0xa0, 0x3c, 0x25, 0xc3, 0x20, 0x6f, 0x3e, 0xac, 0x20, 0xfd, 0x2b, 0x03, 0xe5, 0xb4, 0x44,
+	0x18, 0xc8, 0x0f, 0x61, 0x7d, 0xe8, 0x7a, 0x2c, 0x5f, 0x88, 0x15, 0x1d, 0x55, 0x45, 0x9c, 0xb2,
+	0xa2, 0x0a, 0xa0, 0xd0, 0x72, 0xe2, 0x5a, 0xe4, 0x3b, 0xd7, 0xb3, 0x82, 0xe0, 0x5e, 0xe0, 0x61,
+	0x09, 0x72, 0x49, 0x86, 0x3d, 0x32, 0x74, 0x1d, 0x2b, 0x88, 0xf5, 0x98, 0x85, 0xd7, 0x6e, 0x97,
+	0x9a, 0xa3, 0x48, 0x4b, 0x04, 0x7b, 0xca, 0xca, 0x16, 0x7c, 0xea, 0x48, 0x7d, 0xf3, 0x62, 0x44,
+	0x22, 0xbc, 0x08, 0xf8, 0x6b, 0xbc, 0xea, 0x59, 0xd0, 0xb7, 0x46, 0xa7, 0xb2, 0xe8, 0x76, 0xf7,
+	0x60, 0xd5, 0x76, 0xe8, 0xa5, 0x61, 0x8b, 0xb6, 0x7a, 0x15, 0x2b, 0x6c, 0xd8, 0xb2, 0xd0, 0x0e,
+	0x28, 0xae, 0x33, 0x65, 0xf6, 0x2c, 0xb7, 0xaf, 0xb8, 0xce, 0xb4, 0x65, 0xa9, 0x2d, 0xf8, 0x84,
+	0x09, 0x8d, 0x87, 0x76, 0x50, 0x15, 0x4e, 0x3d, 0x73, 0x4c, 0x74, 0x56, 0xa5, 0x2c, 0x62, 0x75,
+	0xc7, 0xef, 0xdf, 0xb3, 0x9f, 0xc0, 0xe6, 0x19, 0xa1, 0x78, 0xc6, 0xcb, 0xed, 0x4f, 0x9d, 0xcf,
+	0x3f, 0xd7, 0x60, 0x37, 0xfd, 0x66, 0x72, 0x2f, 0x6f, 0xa7, 0xa4, 0x9a, 0x4b, 0xa1, 0xd8, 0x5e,
+	0x52, 0xac, 0x99, 0x91, 0x72, 0xe8, 0x11, 0xac, 0x4f, 0x5c, 0xdf, 0x66, 0x7d, 0xa0, 0x61, 0x79,
+	0xf6, 0x25, 0xe5, 0x7b, 0xa5, 0x34, 0xb3, 0xb8, 0x18, 0xd8, 0x1b, 0xcc, 0xcc, 0x80, 0x0e, 0xb9,
+	0x32, 0x63, 0xc0, 0x65, 0x0e, 0xbc, 0x85, 0x8b, 0x81, 0x5d, 0x00, 0x9f, 0x43, 0xd9, 0x22, 0x23,
+	0x7b, 0x6c, 0x53, 0xe2, 0x19, 0x63, 0xdb, 0xf7, 0x0d, 0x8b, 0x50, 0x59, 0xe3, 0x57, 0x38, 0x65,
+	0x19, 0xef, 0x86, 0x88, 0x97, 0xb6, 0xef, 0x37, 0x02, 0x3f, 0xba, 0x0f, 0x70, 0x61, 0x4f, 0x0c,
+	0xc2, 0x8a, 0x92, 0xa8, 0x52, 0x4a, 0x73, 0x05, 0xe7, 0x2f, 0xec, 0x09, 0xaf, 0x53, 0x3e, 0xba,
+	0x0b, 0x6c, 0x60, 0x4c, 0x1d, 0x5b, 0x16, 0x26, 0xa5, 0xa9, 0xe0, 0xdc, 0x85, 0x3d, 0x19, 0x30,
+	0x0b, 0x4b, 0xea, 0x4b, 0x32, 0x34, 0xc2, 0x78, 0x34, 0xfc, 0xb7, 0xe3, 0x0b, 0x77, 0x24, 0x0a,
+	0x93, 0xd2, 0x5c, 0xc5, 0x5b, 0x97, 0x64, 0x78, 0x12, 0x78, 0x7b, 0xc2, 0xc9, 0x92, 0x53, 0xb0,
+	0x2c, 0xf2, 0x23, 0x0b, 0x9e, 0x88, 0xcf, 0xcb, 0x94, 0xd2, 0xcc, 0xe1, 0x1d, 0xce, 0x93, 0xfe,
+	0x50, 0x00, 0x7d, 0x03, 0x07, 0x49, 0x66, 0x22, 0x1a, 0x79, 0xd5, 0x52, 0x9a, 0x79, 0x7c, 0x3b,
+	0xce, 0x1e, 0xc4, 0x21, 0xe8, 0x13, 0x28, 0x26, 0x14, 0x78, 0xd1, 0x52, 0x9a, 0x80, 0xd7, 0xe2,
+	0x1c, 0xf4, 0x14, 0xb6, 0x92, 0x2f, 0x26, 0x56, 0x60, 0x8d, 0x83, 0x0b, 0x78, 0x33, 0xfe, 0x5a,
+	0x62, 0x29, 0x1e, 0xc3, 0xc6, 0xec, 0x8a, 0x8c, 0x8d, 0x1f, 0xc8, 0xdb, 0x60, 0x3d, 0x8b, 0x1c,
+	0xbd, 0x86, 0x8b, 0xcc, 0xf1, 0x82, 0xbc, 0x8d, 0xd6, 0x94, 0x23, 0x47, 0xae, 0x2f, 0xaa, 0x91,
+	0xd2, 0x2c, 0xe2, 0x1c, 0x33, 0x9d, 0xbb, 0x3e, 0x17, 0xf2, 0x66, 0xc6, 0x64, 0xe4, 0x9a, 0x63,
+	0x5f, 0x28, 0x95, 0x37, 0x38, 0x68, 0x1d, 0x17, 0xbd, 0x59, 0x97, 0xdb, 0xb9, 0x12, 0xfa, 0x1c,
+	0x50, 0x84, 0x74, 0x5c, 0xc7, 0xb0, 0xad, 0x11, 0x29, 0x97, 0x38, 0x78, 0x03, 0x6f, 0x04, 0xe0,
+	0xb6, 0xeb, 0xb4, 0xac, 0x11, 0x0f, 0x57, 0x6f, 0x66, 0xb8, 0xe3, 0xa1, 0x5d, 0xde, 0xe4, 0x98,
+	0x12, 0x56, 0xbc, 0x19, 0xcb, 0x34, 0xe6, 0xa2, 0xd2, 0x85, 0xb8, 0x6b, 0x13, 0x2b, 0x54, 0xb8,
+	0x9e, 0xc3, 0x6d, 0xc9, 0x32, 0x64, 0xc9, 0x34, 0x86, 0xde, 0x50, 0x4e, 0x6c, 0x8b, 0x83, 0x11,
+	0xde, 0x11, 0x3a, 0xf2, 0xfc, 0x39, 0x91, 0xc7, 0x1c, 0x3a, 0x80, 0x9c, 0x37, 0x33, 0x2e, 0x78,
+	0x99, 0xdb, 0xe6, 0xd0, 0xad, 0xa8, 0xf3, 0xbe, 0x0f, 0xc0, 0x66, 0x2f, 0x4f, 0xb2, 0x1d, 0xee,
+	0xde, 0x8e, 0xb7, 0xd7, 0x07, 0x90, 0xa3, 0x01, 0x7b, 0x97, 0xbb, 0x77, 0xa2, 0x8e, 0xfe, 0x3e,
+	0x00, 0x8d, 0xd8, 0x7b, 0xdc, 0xbd, 0x1b, 0x6f, 0xdd, 0x3f, 0x86, 0xb5, 0x0b, 0xe2, 0x19, 0x1e,
+	0x61, 0x1f, 0x21, 0xc4, 0x2a, 0x97, 0x39, 0x64, 0x0f, 0x17, 0x2e, 0x58, 0x2d, 0x10, 0x46, 0xf4,
+	0x11, 0x14, 0x46, 0x43, 0xeb, 0x2a, 0xd8, 0xb0, 0xdb, 0x1c, 0x53, 0xc6, 0xc0, 0x8c, 0x72, 0xb7,
+	0xd8, 0x34, 0x2d, 0x3b, 0x40, 0xec, 0x73, 0xc4, 0x6d, 0x9c, 0xf7, 0x2c, 0x5b, 0x02, 0xee, 0x41,
+	0x9e, 0xda, 0x63, 0xe2, 0x53, 0x73, 0x3c, 0x29, 0x1f, 0xf0, 0x6c, 0xdf, 0xc7, 0x91, 0x49, 0x5b,
+	0x03, 0xb0, 0x7d, 0x43, 0x16, 0x0a, 0xad, 0x00, 0x79, 0xdb, 0x37, 0x44, 0x6d, 0xd0, 0xb6, 0x60,
+	0xd3, 0xf6, 0x8d, 0x64, 0x3d, 0x90, 0xc6, 0x64, 0xee, 0x6b, 0x77, 0xe1, 0xc0, 0x66, 0x89, 0xbd,
+	0x38, 0xcf, 0xb5, 0x0d, 0x28, 0xda, 0xbe, 0x11, 0xa5, 0xb2, 0xb6, 0x0e, 0x6b, 0xd2, 0xc0, 0x03,
+	0x57, 0xdb, 0x87, 0xb2, 0xed, 0x1b, 0x0b, 0x73, 0x55, 0xbb, 0x03, 0xfb, 0xa1, 0x6f, 0x2e, 0x23,
+	0xb5, 0x07, 0x70, 0x6f, 0xce, 0x9b, 0xc8, 0x3a, 0x0d, 0x41, 0x29, 0x8d, 0xd0, 0xca, 0xb0, 0x3b,
+	0xf7, 0x3c, 0x31, 0x93, 0x6d, 0x40, 0xb6, 0x6f, 0xa4, 0x52, 0x45, 0xce, 0x37, 0x4c, 0x0b, 0x89,
+	0x4a, 0xe5, 0x81, 0xb6, 0x07, 0x3b, 0x09, 0x6b, 0x10, 0xf3, 0x72, 0x8d, 0x65, 0x9c, 0xca, 0x91,
+	0x0c, 0x68, 0xed, 0x1e, 0xdc, 0x89, 0x7c, 0xf3, 0x31, 0xac, 0x15, 0xa1, 0x20, 0xfc, 0x3c, 0xd2,
+	0xe4, 0x52, 0x46, 0x91, 0x29, 0xfd, 0x34, 0xe9, 0x8f, 0x62, 0x4f, 0xdb, 0x84, 0x0d, 0xb6, 0xd4,
+	0xb1, 0x58, 0xd3, 0x4a, 0xb0, 0x6e, 0xfb, 0x46, 0x2c, 0xb2, 0x02, 0xd5, 0x30, 0x90, 0xe4, 0x0b,
+	0x87, 0x51, 0xa2, 0xfe, 0x76, 0x05, 0x0e, 0x6e, 0x38, 0xf4, 0xd0, 0x7d, 0x28, 0xb0, 0x76, 0xd7,
+	0x20, 0x51, 0x07, 0xac, 0xdc, 0xd0, 0x01, 0x2b, 0x61, 0x07, 0xbc, 0x0b, 0xca, 0x25, 0xd3, 0x12,
+	0x4d, 0x81, 0x82, 0xe5, 0x08, 0xfd, 0x5f, 0xac, 0xff, 0x35, 0x24, 0x82, 0x9f, 0x30, 0x78, 0x23,
+	0xb4, 0x9f, 0x86, 0xd0, 0xb0, 0xcd, 0x0d, 0xa0, 0x2b, 0x02, 0x1a, 0xda, 0x25, 0xf4, 0x33, 0x40,
+	0xe1, 0xca, 0x12, 0x2b, 0x00, 0xf3, 0x83, 0x05, 0x97, 0xa2, 0xf6, 0x37, 0x12, 0x0e, 0xbb, 0xdc,
+	0x00, 0xbb, 0x2a, 0x84, 0x43, 0xbb, 0x84, 0x3e, 0x8a, 0x9a, 0xdf, 0x00, 0xc9, 0xcf, 0x18, 0xbc,
+	0x1e, 0x98, 0x25, 0xf0, 0x31, 0x94, 0x84, 0xdf, 0x78, 0x76, 0x68, 0xc4, 0x9a, 0x5f, 0x05, 0xaf,
+	0x0b, 0xfb, 0xb3, 0xc3, 0xb0, 0x23, 0xdd, 0x0b, 0x90, 0x75, 0x83, 0xba, 0x46, 0xb5, 0xf6, 0xa5,
+	0x11, 0x6b, 0x7f, 0x15, 0xbc, 0x25, 0x09, 0xa2, 0xfb, 0xed, 0x04, 0x1d, 0x69, 0x59, 0xb2, 0xaa,
+	0xb5, 0x23, 0x46, 0xab, 0xd5, 0xeb, 0x01, 0x8d, 0x9f, 0x25, 0x78, 0x5b, 0xf8, 0x53, 0xfd, 0x6f,
+	0xc4, 0xab, 0xd5, 0x9f, 0x31, 0x5e, 0xbd, 0x5a, 0x35, 0x62, 0x2d, 0x70, 0xc8, 0x0b, 0x3a, 0xe0,
+	0x4e, 0xd0, 0xc9, 0xde, 0x96, 0xbc, 0x7a, 0xb5, 0xc6, 0xa7, 0xf9, 0xb4, 0xf6, 0x85, 0x11, 0x6b,
+	0x82, 0x15, 0xbc, 0x23, 0x00, 0x61, 0x0f, 0x2c, 0x99, 0xcf, 0x61, 0x3f, 0x98, 0xe9, 0xd3, 0xda,
+	0x21, 0xa7, 0xd6, 0xab, 0x47, 0x46, 0xac, 0x0d, 0x56, 0xf0, 0xae, 0x9c, 0x6b, 0xd8, 0x05, 0x0b,
+	0xae, 0xfa, 0x97, 0x0c, 0x3c, 0x7c, 0x57, 0x1b, 0x26, 0xbb, 0x20, 0x0d, 0x72, 0xd3, 0x89, 0x4f,
+	0x3d, 0x62, 0x8e, 0x65, 0x7f, 0x1c, 0xbf, 0x1f, 0xba, 0x49, 0x21, 0xe4, 0xa1, 0x53, 0x00, 0xcb,
+	0xfd, 0xd1, 0x91, 0x2a, 0xd9, 0x0f, 0x52, 0x89, 0x31, 0xd5, 0xdf, 0x8b, 0xab, 0xad, 0xb0, 0xe5,
+	0x93, 0x53, 0xfc, 0xc0, 0x9e, 0x8f, 0xe5, 0x8c, 0x4f, 0x4d, 0x3a, 0x15, 0x39, 0x93, 0xc7, 0x72,
+	0xc4, 0x92, 0xf0, 0xd2, 0xb4, 0x47, 0x86, 0x47, 0x4c, 0xdf, 0x75, 0x78, 0xba, 0xe4, 0x31, 0x30,
+	0x13, 0xe6, 0x16, 0x74, 0x9b, 0x1f, 0x78, 0xfc, 0x56, 0x81, 0x67, 0x48, 0x86, 0x1d, 0x77, 0x7c,
+	0x2e, 0xea, 0xaf, 0x57, 0x60, 0xe3, 0x8c, 0xd0, 0x57, 0xe6, 0x68, 0x1a, 0x5e, 0x3b, 0x7f, 0x95,
+	0xba, 0x3d, 0x2e, 0xd4, 0xee, 0x26, 0x3f, 0xc7, 0x53, 0xf7, 0xd4, 0xcd, 0xa5, 0xe8, 0x7a, 0x19,
+	0x7d, 0x05, 0xab, 0x53, 0x71, 0x97, 0x2a, 0x17, 0xec, 0xfe, 0xf5, 0x77, 0xad, 0x01, 0x3b, 0x60,
+	0xa0, 0x63, 0x28, 0xb8, 0xe2, 0x16, 0x8d, 0x0b, 0xdc, 0x5a, 0xf4, 0xf0, 0xd4, 0x35, 0x5b, 0x73,
+	0x09, 0xc7, 0x39, 0xa8, 0x05, 0xeb, 0xae, 0x33, 0x8d, 0x5d, 0xb8, 0xf0, 0xf5, 0x58, 0x34, 0x8d,
+	0xe4, 0xbd, 0x4c, 0x73, 0x09, 0xa7, 0x88, 0x08, 0x43, 0x91, 0xd0, 0xd7, 0xd1, 0xd7, 0x3f, 0x5f,
+	0xbb, 0x42, 0xed, 0xd3, 0xf7, 0xbf, 0x9b, 0x68, 0x2e, 0xe1, 0xa4, 0x04, 0xfa, 0x19, 0xff, 0x20,
+	0x92, 0x6e, 0x5e, 0x81, 0x0a, 0xb5, 0x83, 0x39, 0xc1, 0xe8, 0x0b, 0xad, 0xb9, 0x84, 0x63, 0x04,
+	0xa4, 0x01, 0xb8, 0x7c, 0xe6, 0xfc, 0xcd, 0x56, 0x39, 0xfd, 0xc1, 0x1c, 0x3d, 0xf5, 0xb1, 0xc3,
+	0x34, 0x22, 0x16, 0x3a, 0x87, 0x55, 0x16, 0x4f, 0x4c, 0x20, 0xc7, 0x05, 0x9e, 0xa6, 0x04, 0xde,
+	0xf9, 0x91, 0xc3, 0xb6, 0x4c, 0x4a, 0xa0, 0x23, 0x08, 0x62, 0x89, 0xd7, 0xb3, 0x42, 0xed, 0x4e,
+	0x52, 0x2d, 0xf9, 0x9d, 0xc3, 0x98, 0x12, 0xae, 0xe5, 0x61, 0xd5, 0x13, 0x56, 0xf5, 0x0f, 0x39,
+	0xfe, 0x79, 0x2b, 0xa3, 0x50, 0xa6, 0xc7, 0xf3, 0x30, 0xdc, 0xc5, 0x9d, 0x90, 0x9a, 0x14, 0x4e,
+	0x80, 0x2b, 0x3d, 0x8e, 0x0c, 0x53, 0x42, 0x87, 0x3c, 0xf1, 0x3c, 0x11, 0xfe, 0xf2, 0x7a, 0xf8,
+	0xd1, 0x4d, 0x74, 0x7e, 0x00, 0x08, 0x38, 0x8e, 0x98, 0xe8, 0xeb, 0x58, 0x26, 0x88, 0x60, 0xbc,
+	0x77, 0x5d, 0x26, 0x08, 0xa1, 0x44, 0x2a, 0x7c, 0x1d, 0xa5, 0xc2, 0xf2, 0x35, 0x3b, 0x95, 0xfa,
+	0xd9, 0x21, 0x9e, 0x0b, 0x2f, 0x60, 0x6d, 0x22, 0xe2, 0x9c, 0x3a, 0xc4, 0xf3, 0x65, 0xf0, 0x7d,
+	0x72, 0x63, 0x32, 0xc4, 0x74, 0x12, 0x64, 0xf4, 0xed, 0x5c, 0x56, 0x88, 0xd0, 0x7b, 0xf4, 0x8e,
+	0xac, 0x88, 0x09, 0xa6, 0xb3, 0xe3, 0x02, 0x36, 0x13, 0xa1, 0x1d, 0x8b, 0xc8, 0xda, 0xfb, 0x67,
+	0x48, 0xec, 0x01, 0xf3, 0x72, 0x48, 0x4f, 0x64, 0x8b, 0x88, 0xd6, 0x8f, 0x6f, 0xc8, 0x96, 0x98,
+	0x5a, 0x3c, 0x6b, 0x5e, 0xf0, 0xb7, 0xef, 0xba, 0x4e, 0xb0, 0x4e, 0x32, 0x54, 0x3f, 0xba, 0x21,
+	0x73, 0x12, 0xef, 0x1d, 0xa3, 0xa2, 0x01, 0xff, 0xc0, 0x0f, 0x95, 0x80, 0x2b, 0x55, 0x3f, 0x20,
+	0x85, 0x42, 0xe5, 0xb8, 0x0e, 0xfa, 0xff, 0x28, 0x8f, 0x0a, 0x8b, 0xca, 0x5e, 0xea, 0xf0, 0x88,
+	0x25, 0x92, 0x5a, 0x05, 0x45, 0x84, 0x3f, 0xda, 0x86, 0x52, 0xaf, 0x7f, 0xdc, 0x1f, 0xf4, 0x12,
+	0x3f, 0xba, 0x28, 0x90, 0xed, 0xbc, 0x28, 0x65, 0x50, 0x1e, 0x56, 0x74, 0x8c, 0x3b, 0xb8, 0x94,
+	0x55, 0xff, 0x98, 0x81, 0x42, 0x2c, 0xe6, 0x19, 0x11, 0xeb, 0xc7, 0xbd, 0x4e, 0x3b, 0x41, 0xdc,
+	0x80, 0xc2, 0xa0, 0xdd, 0x1b, 0x74, 0xbb, 0x1d, 0xdc, 0xe7, 0xbf, 0xd8, 0xec, 0xc0, 0x66, 0xab,
+	0xfd, 0xea, 0xf8, 0xbc, 0xd5, 0x30, 0x1a, 0xfa, 0xab, 0xd6, 0x89, 0x6e, 0xb4, 0x1a, 0xa5, 0x6c,
+	0xdc, 0xcc, 0xa0, 0x46, 0xff, 0xe7, 0x5d, 0xbd, 0x74, 0x0b, 0x15, 0x60, 0xb5, 0xdf, 0x7a, 0xa9,
+	0x77, 0x06, 0xfd, 0xd2, 0x32, 0x7b, 0x42, 0x80, 0xc1, 0xfa, 0xb7, 0x02, 0xb2, 0x82, 0x10, 0xac,
+	0xb7, 0xda, 0x7d, 0x1d, 0xb7, 0x8f, 0xcf, 0x0d, 0x31, 0x37, 0x45, 0xd8, 0xe2, 0x0f, 0x29, 0xad,
+	0x6a, 0x00, 0x39, 0x4f, 0xbe, 0xb9, 0xfa, 0x0a, 0x36, 0x7a, 0xa9, 0x13, 0xeb, 0x08, 0xd6, 0xcc,
+	0x91, 0xe9, 0x8d, 0xe5, 0x2f, 0xae, 0xf2, 0xd4, 0xda, 0xaa, 0xc8, 0x1f, 0x60, 0x8f, 0x99, 0x4f,
+	0xfc, 0x62, 0xc7, 0x96, 0xdd, 0x8c, 0x86, 0xf1, 0x22, 0xf4, 0xef, 0x0c, 0x94, 0x7a, 0x1f, 0x52,
+	0x84, 0x7a, 0xff, 0x5d, 0x11, 0xea, 0xbd, 0x5f, 0x11, 0xfa, 0x29, 0xdb, 0x7b, 0xf8, 0x53, 0x76,
+	0x57, 0xb5, 0x61, 0xa7, 0x67, 0x3b, 0x57, 0x23, 0x92, 0x6e, 0x08, 0xf6, 0x21, 0x47, 0x4d, 0xef,
+	0x8a, 0xd0, 0xf0, 0x42, 0x2b, 0x1c, 0xa3, 0xc3, 0x70, 0x01, 0xe5, 0x79, 0xbf, 0xbf, 0xb0, 0xce,
+	0x72, 0x04, 0x0e, 0xd7, 0xfa, 0x5b, 0xd8, 0x4d, 0x3f, 0x4a, 0x2e, 0xf8, 0x97, 0xd1, 0x4e, 0xcb,
+	0x6d, 0x3c, 0xb8, 0xa1, 0x70, 0xe3, 0x28, 0x2c, 0xc2, 0xd9, 0xf7, 0xfe, 0x57, 0xb3, 0xef, 0xbd,
+	0x73, 0xf6, 0xbd, 0x0f, 0x9b, 0x7d, 0xef, 0xda, 0xd9, 0xd7, 0xfe, 0x9c, 0x81, 0xbc, 0x1e, 0x00,
+	0x11, 0x86, 0xc2, 0x19, 0xa1, 0xfa, 0x4c, 0xc0, 0x51, 0xfc, 0xdc, 0x58, 0xb8, 0x43, 0xfb, 0x1f,
+	0xdd, 0x80, 0x90, 0x53, 0xc3, 0x50, 0xe8, 0xdd, 0xa8, 0xd9, 0x7b, 0xa7, 0x66, 0x7a, 0xfe, 0x1a,
+	0x86, 0xbb, 0xae, 0x77, 0x55, 0x71, 0x27, 0x84, 0x7d, 0x7a, 0x5b, 0x15, 0xf1, 0x2f, 0x10, 0x11,
+	0xef, 0x17, 0xd5, 0x2b, 0x9b, 0xbe, 0x9e, 0x5e, 0x54, 0x86, 0xee, 0xf8, 0x49, 0x80, 0x7a, 0x22,
+	0x50, 0x9f, 0xcb, 0x7f, 0x94, 0x78, 0x53, 0x7f, 0x72, 0xe5, 0x46, 0xff, 0x59, 0xd1, 0x5d, 0xba,
+	0x50, 0xb8, 0xe7, 0x8b, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x56, 0x65, 0x78, 0xd4, 0x7d, 0x21,
+	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
+
+// ExtensionClient is the client API for Extension service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type ExtensionClient interface {
+	// Get a single attribute
+	GetExtValue(ctx context.Context, in *SingleGetValueRequest, opts ...grpc.CallOption) (*SingleGetValueResponse, error)
+	// Set a single attribute
+	SetExtValue(ctx context.Context, in *SingleSetValueRequest, opts ...grpc.CallOption) (*SingleSetValueResponse, error)
+}
+
+type extensionClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewExtensionClient(cc *grpc.ClientConn) ExtensionClient {
+	return &extensionClient{cc}
+}
+
+func (c *extensionClient) GetExtValue(ctx context.Context, in *SingleGetValueRequest, opts ...grpc.CallOption) (*SingleGetValueResponse, error) {
+	out := new(SingleGetValueResponse)
+	err := c.cc.Invoke(ctx, "/extension.Extension/GetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *extensionClient) SetExtValue(ctx context.Context, in *SingleSetValueRequest, opts ...grpc.CallOption) (*SingleSetValueResponse, error) {
+	out := new(SingleSetValueResponse)
+	err := c.cc.Invoke(ctx, "/extension.Extension/SetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// ExtensionServer is the server API for Extension service.
+type ExtensionServer interface {
+	// Get a single attribute
+	GetExtValue(context.Context, *SingleGetValueRequest) (*SingleGetValueResponse, error)
+	// Set a single attribute
+	SetExtValue(context.Context, *SingleSetValueRequest) (*SingleSetValueResponse, error)
+}
+
+// UnimplementedExtensionServer can be embedded to have forward compatible implementations.
+type UnimplementedExtensionServer struct {
+}
+
+func (*UnimplementedExtensionServer) GetExtValue(ctx context.Context, req *SingleGetValueRequest) (*SingleGetValueResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
+}
+func (*UnimplementedExtensionServer) SetExtValue(ctx context.Context, req *SingleSetValueRequest) (*SingleSetValueResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SetExtValue not implemented")
+}
+
+func RegisterExtensionServer(s *grpc.Server, srv ExtensionServer) {
+	s.RegisterService(&_Extension_serviceDesc, srv)
+}
+
+func _Extension_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(SingleGetValueRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ExtensionServer).GetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/extension.Extension/GetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ExtensionServer).GetExtValue(ctx, req.(*SingleGetValueRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Extension_SetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(SingleSetValueRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ExtensionServer).SetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/extension.Extension/SetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ExtensionServer).SetExtValue(ctx, req.(*SingleSetValueRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Extension_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "extension.Extension",
+	HandlerType: (*ExtensionServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetExtValue",
+			Handler:    _Extension_GetExtValue_Handler,
+		},
+		{
+			MethodName: "SetExtValue",
+			Handler:    _Extension_SetExtValue_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/extensions.proto",
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/inter_container/inter_container.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/inter_container/inter_container.pb.go
new file mode 100644
index 0000000..fbffef2
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/inter_container/inter_container.pb.go
@@ -0,0 +1,1637 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/inter_container.proto
+
+package inter_container
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	openflow_13 "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	openolt "github.com/opencord/voltha-protos/v5/go/openolt"
+	tech_profile "github.com/opencord/voltha-protos/v5/go/tech_profile"
+	voltha "github.com/opencord/voltha-protos/v5/go/voltha"
+	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
+
+//
+// Core-Adapter messages
+// ****
+// ****
+// ****
+type AdapterRegistration struct {
+	Adapter              *voltha.Adapter     `protobuf:"bytes,1,opt,name=adapter,proto3" json:"adapter,omitempty"`
+	DTypes               *voltha.DeviceTypes `protobuf:"bytes,2,opt,name=dTypes,proto3" json:"dTypes,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *AdapterRegistration) Reset()         { *m = AdapterRegistration{} }
+func (m *AdapterRegistration) String() string { return proto.CompactTextString(m) }
+func (*AdapterRegistration) ProtoMessage()    {}
+func (*AdapterRegistration) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{0}
+}
+
+func (m *AdapterRegistration) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AdapterRegistration.Unmarshal(m, b)
+}
+func (m *AdapterRegistration) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AdapterRegistration.Marshal(b, m, deterministic)
+}
+func (m *AdapterRegistration) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AdapterRegistration.Merge(m, src)
+}
+func (m *AdapterRegistration) XXX_Size() int {
+	return xxx_messageInfo_AdapterRegistration.Size(m)
+}
+func (m *AdapterRegistration) XXX_DiscardUnknown() {
+	xxx_messageInfo_AdapterRegistration.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AdapterRegistration proto.InternalMessageInfo
+
+func (m *AdapterRegistration) GetAdapter() *voltha.Adapter {
+	if m != nil {
+		return m.Adapter
+	}
+	return nil
+}
+
+func (m *AdapterRegistration) GetDTypes() *voltha.DeviceTypes {
+	if m != nil {
+		return m.DTypes
+	}
+	return nil
+}
+
+type ChildDeviceFilter struct {
+	ParentId             string   `protobuf:"bytes,1,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
+	SerialNumber         string   `protobuf:"bytes,2,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	OnuId                uint32   `protobuf:"varint,3,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	ParentPortNo         uint32   `protobuf:"varint,4,opt,name=parent_port_no,json=parentPortNo,proto3" json:"parent_port_no,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ChildDeviceFilter) Reset()         { *m = ChildDeviceFilter{} }
+func (m *ChildDeviceFilter) String() string { return proto.CompactTextString(m) }
+func (*ChildDeviceFilter) ProtoMessage()    {}
+func (*ChildDeviceFilter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{1}
+}
+
+func (m *ChildDeviceFilter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ChildDeviceFilter.Unmarshal(m, b)
+}
+func (m *ChildDeviceFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ChildDeviceFilter.Marshal(b, m, deterministic)
+}
+func (m *ChildDeviceFilter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ChildDeviceFilter.Merge(m, src)
+}
+func (m *ChildDeviceFilter) XXX_Size() int {
+	return xxx_messageInfo_ChildDeviceFilter.Size(m)
+}
+func (m *ChildDeviceFilter) XXX_DiscardUnknown() {
+	xxx_messageInfo_ChildDeviceFilter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ChildDeviceFilter proto.InternalMessageInfo
+
+func (m *ChildDeviceFilter) GetParentId() string {
+	if m != nil {
+		return m.ParentId
+	}
+	return ""
+}
+
+func (m *ChildDeviceFilter) GetSerialNumber() string {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return ""
+}
+
+func (m *ChildDeviceFilter) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *ChildDeviceFilter) GetParentPortNo() uint32 {
+	if m != nil {
+		return m.ParentPortNo
+	}
+	return 0
+}
+
+type PortFilter struct {
+	DeviceId             string               `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Port                 uint32               `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
+	PortType             voltha.Port_PortType `protobuf:"varint,3,opt,name=port_type,json=portType,proto3,enum=voltha.Port_PortType" json:"port_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *PortFilter) Reset()         { *m = PortFilter{} }
+func (m *PortFilter) String() string { return proto.CompactTextString(m) }
+func (*PortFilter) ProtoMessage()    {}
+func (*PortFilter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{2}
+}
+
+func (m *PortFilter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PortFilter.Unmarshal(m, b)
+}
+func (m *PortFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PortFilter.Marshal(b, m, deterministic)
+}
+func (m *PortFilter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PortFilter.Merge(m, src)
+}
+func (m *PortFilter) XXX_Size() int {
+	return xxx_messageInfo_PortFilter.Size(m)
+}
+func (m *PortFilter) XXX_DiscardUnknown() {
+	xxx_messageInfo_PortFilter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PortFilter proto.InternalMessageInfo
+
+func (m *PortFilter) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PortFilter) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+func (m *PortFilter) GetPortType() voltha.Port_PortType {
+	if m != nil {
+		return m.PortType
+	}
+	return voltha.Port_UNKNOWN
+}
+
+type DeviceDiscovery struct {
+	ParentId             string   `protobuf:"bytes,1,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
+	ParentPortNo         uint32   `protobuf:"varint,2,opt,name=parent_port_no,json=parentPortNo,proto3" json:"parent_port_no,omitempty"`
+	ChildDeviceType      string   `protobuf:"bytes,3,opt,name=child_device_type,json=childDeviceType,proto3" json:"child_device_type,omitempty"`
+	ChannelId            uint32   `protobuf:"varint,4,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"`
+	VendorId             string   `protobuf:"bytes,5,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
+	SerialNumber         string   `protobuf:"bytes,6,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	OnuId                uint32   `protobuf:"varint,7,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeviceDiscovery) Reset()         { *m = DeviceDiscovery{} }
+func (m *DeviceDiscovery) String() string { return proto.CompactTextString(m) }
+func (*DeviceDiscovery) ProtoMessage()    {}
+func (*DeviceDiscovery) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{3}
+}
+
+func (m *DeviceDiscovery) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceDiscovery.Unmarshal(m, b)
+}
+func (m *DeviceDiscovery) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceDiscovery.Marshal(b, m, deterministic)
+}
+func (m *DeviceDiscovery) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceDiscovery.Merge(m, src)
+}
+func (m *DeviceDiscovery) XXX_Size() int {
+	return xxx_messageInfo_DeviceDiscovery.Size(m)
+}
+func (m *DeviceDiscovery) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceDiscovery.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceDiscovery proto.InternalMessageInfo
+
+func (m *DeviceDiscovery) GetParentId() string {
+	if m != nil {
+		return m.ParentId
+	}
+	return ""
+}
+
+func (m *DeviceDiscovery) GetParentPortNo() uint32 {
+	if m != nil {
+		return m.ParentPortNo
+	}
+	return 0
+}
+
+func (m *DeviceDiscovery) GetChildDeviceType() string {
+	if m != nil {
+		return m.ChildDeviceType
+	}
+	return ""
+}
+
+func (m *DeviceDiscovery) GetChannelId() uint32 {
+	if m != nil {
+		return m.ChannelId
+	}
+	return 0
+}
+
+func (m *DeviceDiscovery) GetVendorId() string {
+	if m != nil {
+		return m.VendorId
+	}
+	return ""
+}
+
+func (m *DeviceDiscovery) GetSerialNumber() string {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return ""
+}
+
+func (m *DeviceDiscovery) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+type DeviceStateFilter struct {
+	DeviceId             string                     `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	ParentDeviceId       string                     `protobuf:"bytes,2,opt,name=parent_device_id,json=parentDeviceId,proto3" json:"parent_device_id,omitempty"`
+	OperStatus           common.OperStatus_Types    `protobuf:"varint,3,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
+	ConnStatus           common.ConnectStatus_Types `protobuf:"varint,4,opt,name=conn_status,json=connStatus,proto3,enum=common.ConnectStatus_Types" json:"conn_status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
+	XXX_unrecognized     []byte                     `json:"-"`
+	XXX_sizecache        int32                      `json:"-"`
+}
+
+func (m *DeviceStateFilter) Reset()         { *m = DeviceStateFilter{} }
+func (m *DeviceStateFilter) String() string { return proto.CompactTextString(m) }
+func (*DeviceStateFilter) ProtoMessage()    {}
+func (*DeviceStateFilter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{4}
+}
+
+func (m *DeviceStateFilter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceStateFilter.Unmarshal(m, b)
+}
+func (m *DeviceStateFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceStateFilter.Marshal(b, m, deterministic)
+}
+func (m *DeviceStateFilter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceStateFilter.Merge(m, src)
+}
+func (m *DeviceStateFilter) XXX_Size() int {
+	return xxx_messageInfo_DeviceStateFilter.Size(m)
+}
+func (m *DeviceStateFilter) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceStateFilter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceStateFilter proto.InternalMessageInfo
+
+func (m *DeviceStateFilter) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeviceStateFilter) GetParentDeviceId() string {
+	if m != nil {
+		return m.ParentDeviceId
+	}
+	return ""
+}
+
+func (m *DeviceStateFilter) GetOperStatus() common.OperStatus_Types {
+	if m != nil {
+		return m.OperStatus
+	}
+	return common.OperStatus_UNKNOWN
+}
+
+func (m *DeviceStateFilter) GetConnStatus() common.ConnectStatus_Types {
+	if m != nil {
+		return m.ConnStatus
+	}
+	return common.ConnectStatus_UNKNOWN
+}
+
+type PortState struct {
+	DeviceId             string                  `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	PortType             voltha.Port_PortType    `protobuf:"varint,2,opt,name=port_type,json=portType,proto3,enum=voltha.Port_PortType" json:"port_type,omitempty"`
+	PortNo               uint32                  `protobuf:"varint,3,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	OperStatus           common.OperStatus_Types `protobuf:"varint,4,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *PortState) Reset()         { *m = PortState{} }
+func (m *PortState) String() string { return proto.CompactTextString(m) }
+func (*PortState) ProtoMessage()    {}
+func (*PortState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{5}
+}
+
+func (m *PortState) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PortState.Unmarshal(m, b)
+}
+func (m *PortState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PortState.Marshal(b, m, deterministic)
+}
+func (m *PortState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PortState.Merge(m, src)
+}
+func (m *PortState) XXX_Size() int {
+	return xxx_messageInfo_PortState.Size(m)
+}
+func (m *PortState) XXX_DiscardUnknown() {
+	xxx_messageInfo_PortState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PortState proto.InternalMessageInfo
+
+func (m *PortState) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PortState) GetPortType() voltha.Port_PortType {
+	if m != nil {
+		return m.PortType
+	}
+	return voltha.Port_UNKNOWN
+}
+
+func (m *PortState) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *PortState) GetOperStatus() common.OperStatus_Types {
+	if m != nil {
+		return m.OperStatus
+	}
+	return common.OperStatus_UNKNOWN
+}
+
+type PortStateFilter struct {
+	DeviceId             string                  `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	PortTypeFilter       uint32                  `protobuf:"varint,2,opt,name=port_type_filter,json=portTypeFilter,proto3" json:"port_type_filter,omitempty"`
+	OperStatus           common.OperStatus_Types `protobuf:"varint,4,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *PortStateFilter) Reset()         { *m = PortStateFilter{} }
+func (m *PortStateFilter) String() string { return proto.CompactTextString(m) }
+func (*PortStateFilter) ProtoMessage()    {}
+func (*PortStateFilter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{6}
+}
+
+func (m *PortStateFilter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PortStateFilter.Unmarshal(m, b)
+}
+func (m *PortStateFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PortStateFilter.Marshal(b, m, deterministic)
+}
+func (m *PortStateFilter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PortStateFilter.Merge(m, src)
+}
+func (m *PortStateFilter) XXX_Size() int {
+	return xxx_messageInfo_PortStateFilter.Size(m)
+}
+func (m *PortStateFilter) XXX_DiscardUnknown() {
+	xxx_messageInfo_PortStateFilter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PortStateFilter proto.InternalMessageInfo
+
+func (m *PortStateFilter) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PortStateFilter) GetPortTypeFilter() uint32 {
+	if m != nil {
+		return m.PortTypeFilter
+	}
+	return 0
+}
+
+func (m *PortStateFilter) GetOperStatus() common.OperStatus_Types {
+	if m != nil {
+		return m.OperStatus
+	}
+	return common.OperStatus_UNKNOWN
+}
+
+type PacketIn struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Port                 uint32   `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
+	Packet               []byte   `protobuf:"bytes,3,opt,name=packet,proto3" json:"packet,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *PacketIn) Reset()         { *m = PacketIn{} }
+func (m *PacketIn) String() string { return proto.CompactTextString(m) }
+func (*PacketIn) ProtoMessage()    {}
+func (*PacketIn) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{7}
+}
+
+func (m *PacketIn) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PacketIn.Unmarshal(m, b)
+}
+func (m *PacketIn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PacketIn.Marshal(b, m, deterministic)
+}
+func (m *PacketIn) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PacketIn.Merge(m, src)
+}
+func (m *PacketIn) XXX_Size() int {
+	return xxx_messageInfo_PacketIn.Size(m)
+}
+func (m *PacketIn) XXX_DiscardUnknown() {
+	xxx_messageInfo_PacketIn.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PacketIn proto.InternalMessageInfo
+
+func (m *PacketIn) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PacketIn) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+func (m *PacketIn) GetPacket() []byte {
+	if m != nil {
+		return m.Packet
+	}
+	return nil
+}
+
+type PacketOut struct {
+	DeviceId             string                    `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	EgressPortNo         uint32                    `protobuf:"varint,2,opt,name=egress_port_no,json=egressPortNo,proto3" json:"egress_port_no,omitempty"`
+	Packet               *openflow_13.OfpPacketOut `protobuf:"bytes,3,opt,name=packet,proto3" json:"packet,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *PacketOut) Reset()         { *m = PacketOut{} }
+func (m *PacketOut) String() string { return proto.CompactTextString(m) }
+func (*PacketOut) ProtoMessage()    {}
+func (*PacketOut) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{8}
+}
+
+func (m *PacketOut) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PacketOut.Unmarshal(m, b)
+}
+func (m *PacketOut) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PacketOut.Marshal(b, m, deterministic)
+}
+func (m *PacketOut) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PacketOut.Merge(m, src)
+}
+func (m *PacketOut) XXX_Size() int {
+	return xxx_messageInfo_PacketOut.Size(m)
+}
+func (m *PacketOut) XXX_DiscardUnknown() {
+	xxx_messageInfo_PacketOut.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PacketOut proto.InternalMessageInfo
+
+func (m *PacketOut) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PacketOut) GetEgressPortNo() uint32 {
+	if m != nil {
+		return m.EgressPortNo
+	}
+	return 0
+}
+
+func (m *PacketOut) GetPacket() *openflow_13.OfpPacketOut {
+	if m != nil {
+		return m.Packet
+	}
+	return nil
+}
+
+type DeviceReason struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Reason               string   `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeviceReason) Reset()         { *m = DeviceReason{} }
+func (m *DeviceReason) String() string { return proto.CompactTextString(m) }
+func (*DeviceReason) ProtoMessage()    {}
+func (*DeviceReason) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{9}
+}
+
+func (m *DeviceReason) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceReason.Unmarshal(m, b)
+}
+func (m *DeviceReason) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceReason.Marshal(b, m, deterministic)
+}
+func (m *DeviceReason) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceReason.Merge(m, src)
+}
+func (m *DeviceReason) XXX_Size() int {
+	return xxx_messageInfo_DeviceReason.Size(m)
+}
+func (m *DeviceReason) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceReason.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceReason proto.InternalMessageInfo
+
+func (m *DeviceReason) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeviceReason) GetReason() string {
+	if m != nil {
+		return m.Reason
+	}
+	return ""
+}
+
+type BulkFlows struct {
+	Device               *voltha.Device          `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Flows                *openflow_13.Flows      `protobuf:"bytes,2,opt,name=flows,proto3" json:"flows,omitempty"`
+	Groups               *openflow_13.FlowGroups `protobuf:"bytes,3,opt,name=groups,proto3" json:"groups,omitempty"`
+	FlowMetadata         *voltha.FlowMetadata    `protobuf:"bytes,4,opt,name=flow_metadata,json=flowMetadata,proto3" json:"flow_metadata,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *BulkFlows) Reset()         { *m = BulkFlows{} }
+func (m *BulkFlows) String() string { return proto.CompactTextString(m) }
+func (*BulkFlows) ProtoMessage()    {}
+func (*BulkFlows) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{10}
+}
+
+func (m *BulkFlows) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_BulkFlows.Unmarshal(m, b)
+}
+func (m *BulkFlows) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_BulkFlows.Marshal(b, m, deterministic)
+}
+func (m *BulkFlows) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BulkFlows.Merge(m, src)
+}
+func (m *BulkFlows) XXX_Size() int {
+	return xxx_messageInfo_BulkFlows.Size(m)
+}
+func (m *BulkFlows) XXX_DiscardUnknown() {
+	xxx_messageInfo_BulkFlows.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_BulkFlows proto.InternalMessageInfo
+
+func (m *BulkFlows) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *BulkFlows) GetFlows() *openflow_13.Flows {
+	if m != nil {
+		return m.Flows
+	}
+	return nil
+}
+
+func (m *BulkFlows) GetGroups() *openflow_13.FlowGroups {
+	if m != nil {
+		return m.Groups
+	}
+	return nil
+}
+
+func (m *BulkFlows) GetFlowMetadata() *voltha.FlowMetadata {
+	if m != nil {
+		return m.FlowMetadata
+	}
+	return nil
+}
+
+type IncrementalFlows struct {
+	Device               *voltha.Device                `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Flows                *openflow_13.FlowChanges      `protobuf:"bytes,2,opt,name=flows,proto3" json:"flows,omitempty"`
+	Groups               *openflow_13.FlowGroupChanges `protobuf:"bytes,3,opt,name=groups,proto3" json:"groups,omitempty"`
+	FlowMetadata         *voltha.FlowMetadata          `protobuf:"bytes,4,opt,name=flow_metadata,json=flowMetadata,proto3" json:"flow_metadata,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
+	XXX_unrecognized     []byte                        `json:"-"`
+	XXX_sizecache        int32                         `json:"-"`
+}
+
+func (m *IncrementalFlows) Reset()         { *m = IncrementalFlows{} }
+func (m *IncrementalFlows) String() string { return proto.CompactTextString(m) }
+func (*IncrementalFlows) ProtoMessage()    {}
+func (*IncrementalFlows) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{11}
+}
+
+func (m *IncrementalFlows) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_IncrementalFlows.Unmarshal(m, b)
+}
+func (m *IncrementalFlows) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_IncrementalFlows.Marshal(b, m, deterministic)
+}
+func (m *IncrementalFlows) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IncrementalFlows.Merge(m, src)
+}
+func (m *IncrementalFlows) XXX_Size() int {
+	return xxx_messageInfo_IncrementalFlows.Size(m)
+}
+func (m *IncrementalFlows) XXX_DiscardUnknown() {
+	xxx_messageInfo_IncrementalFlows.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_IncrementalFlows proto.InternalMessageInfo
+
+func (m *IncrementalFlows) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *IncrementalFlows) GetFlows() *openflow_13.FlowChanges {
+	if m != nil {
+		return m.Flows
+	}
+	return nil
+}
+
+func (m *IncrementalFlows) GetGroups() *openflow_13.FlowGroupChanges {
+	if m != nil {
+		return m.Groups
+	}
+	return nil
+}
+
+func (m *IncrementalFlows) GetFlowMetadata() *voltha.FlowMetadata {
+	if m != nil {
+		return m.FlowMetadata
+	}
+	return nil
+}
+
+type PmConfigsInfo struct {
+	DeviceId             string            `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	PmConfigs            *voltha.PmConfigs `protobuf:"bytes,2,opt,name=pm_configs,json=pmConfigs,proto3" json:"pm_configs,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *PmConfigsInfo) Reset()         { *m = PmConfigsInfo{} }
+func (m *PmConfigsInfo) String() string { return proto.CompactTextString(m) }
+func (*PmConfigsInfo) ProtoMessage()    {}
+func (*PmConfigsInfo) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{12}
+}
+
+func (m *PmConfigsInfo) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PmConfigsInfo.Unmarshal(m, b)
+}
+func (m *PmConfigsInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PmConfigsInfo.Marshal(b, m, deterministic)
+}
+func (m *PmConfigsInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PmConfigsInfo.Merge(m, src)
+}
+func (m *PmConfigsInfo) XXX_Size() int {
+	return xxx_messageInfo_PmConfigsInfo.Size(m)
+}
+func (m *PmConfigsInfo) XXX_DiscardUnknown() {
+	xxx_messageInfo_PmConfigsInfo.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PmConfigsInfo proto.InternalMessageInfo
+
+func (m *PmConfigsInfo) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *PmConfigsInfo) GetPmConfigs() *voltha.PmConfigs {
+	if m != nil {
+		return m.PmConfigs
+	}
+	return nil
+}
+
+type SwitchCapability struct {
+	Desc                 *openflow_13.OfpDesc           `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"`
+	SwitchFeatures       *openflow_13.OfpSwitchFeatures `protobuf:"bytes,2,opt,name=switch_features,json=switchFeatures,proto3" json:"switch_features,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                       `json:"-"`
+	XXX_unrecognized     []byte                         `json:"-"`
+	XXX_sizecache        int32                          `json:"-"`
+}
+
+func (m *SwitchCapability) Reset()         { *m = SwitchCapability{} }
+func (m *SwitchCapability) String() string { return proto.CompactTextString(m) }
+func (*SwitchCapability) ProtoMessage()    {}
+func (*SwitchCapability) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{13}
+}
+
+func (m *SwitchCapability) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SwitchCapability.Unmarshal(m, b)
+}
+func (m *SwitchCapability) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SwitchCapability.Marshal(b, m, deterministic)
+}
+func (m *SwitchCapability) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SwitchCapability.Merge(m, src)
+}
+func (m *SwitchCapability) XXX_Size() int {
+	return xxx_messageInfo_SwitchCapability.Size(m)
+}
+func (m *SwitchCapability) XXX_DiscardUnknown() {
+	xxx_messageInfo_SwitchCapability.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SwitchCapability proto.InternalMessageInfo
+
+func (m *SwitchCapability) GetDesc() *openflow_13.OfpDesc {
+	if m != nil {
+		return m.Desc
+	}
+	return nil
+}
+
+func (m *SwitchCapability) GetSwitchFeatures() *openflow_13.OfpSwitchFeatures {
+	if m != nil {
+		return m.SwitchFeatures
+	}
+	return nil
+}
+
+type ImageDownloadMessage struct {
+	Device               *voltha.Device        `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Image                *voltha.ImageDownload `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *ImageDownloadMessage) Reset()         { *m = ImageDownloadMessage{} }
+func (m *ImageDownloadMessage) String() string { return proto.CompactTextString(m) }
+func (*ImageDownloadMessage) ProtoMessage()    {}
+func (*ImageDownloadMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{14}
+}
+
+func (m *ImageDownloadMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ImageDownloadMessage.Unmarshal(m, b)
+}
+func (m *ImageDownloadMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ImageDownloadMessage.Marshal(b, m, deterministic)
+}
+func (m *ImageDownloadMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ImageDownloadMessage.Merge(m, src)
+}
+func (m *ImageDownloadMessage) XXX_Size() int {
+	return xxx_messageInfo_ImageDownloadMessage.Size(m)
+}
+func (m *ImageDownloadMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_ImageDownloadMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ImageDownloadMessage proto.InternalMessageInfo
+
+func (m *ImageDownloadMessage) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *ImageDownloadMessage) GetImage() *voltha.ImageDownload {
+	if m != nil {
+		return m.Image
+	}
+	return nil
+}
+
+type OMCITest struct {
+	Device               *voltha.Device          `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Request              *voltha.OmciTestRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *OMCITest) Reset()         { *m = OMCITest{} }
+func (m *OMCITest) String() string { return proto.CompactTextString(m) }
+func (*OMCITest) ProtoMessage()    {}
+func (*OMCITest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{15}
+}
+
+func (m *OMCITest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OMCITest.Unmarshal(m, b)
+}
+func (m *OMCITest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OMCITest.Marshal(b, m, deterministic)
+}
+func (m *OMCITest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OMCITest.Merge(m, src)
+}
+func (m *OMCITest) XXX_Size() int {
+	return xxx_messageInfo_OMCITest.Size(m)
+}
+func (m *OMCITest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OMCITest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OMCITest proto.InternalMessageInfo
+
+func (m *OMCITest) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *OMCITest) GetRequest() *voltha.OmciTestRequest {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+type SimulateAlarmMessage struct {
+	Device               *voltha.Device               `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Request              *voltha.SimulateAlarmRequest `protobuf:"bytes,2,opt,name=request,proto3" json:"request,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
+	XXX_unrecognized     []byte                       `json:"-"`
+	XXX_sizecache        int32                        `json:"-"`
+}
+
+func (m *SimulateAlarmMessage) Reset()         { *m = SimulateAlarmMessage{} }
+func (m *SimulateAlarmMessage) String() string { return proto.CompactTextString(m) }
+func (*SimulateAlarmMessage) ProtoMessage()    {}
+func (*SimulateAlarmMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{16}
+}
+
+func (m *SimulateAlarmMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SimulateAlarmMessage.Unmarshal(m, b)
+}
+func (m *SimulateAlarmMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SimulateAlarmMessage.Marshal(b, m, deterministic)
+}
+func (m *SimulateAlarmMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SimulateAlarmMessage.Merge(m, src)
+}
+func (m *SimulateAlarmMessage) XXX_Size() int {
+	return xxx_messageInfo_SimulateAlarmMessage.Size(m)
+}
+func (m *SimulateAlarmMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_SimulateAlarmMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SimulateAlarmMessage proto.InternalMessageInfo
+
+func (m *SimulateAlarmMessage) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *SimulateAlarmMessage) GetRequest() *voltha.SimulateAlarmRequest {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+type GetExtValueMessage struct {
+	ParentDevice         *voltha.Device        `protobuf:"bytes,1,opt,name=parent_device,json=parentDevice,proto3" json:"parent_device,omitempty"`
+	ChildDevice          *voltha.Device        `protobuf:"bytes,2,opt,name=child_device,json=childDevice,proto3" json:"child_device,omitempty"`
+	ValueType            common.ValueType_Type `protobuf:"varint,3,opt,name=value_type,json=valueType,proto3,enum=common.ValueType_Type" json:"value_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *GetExtValueMessage) Reset()         { *m = GetExtValueMessage{} }
+func (m *GetExtValueMessage) String() string { return proto.CompactTextString(m) }
+func (*GetExtValueMessage) ProtoMessage()    {}
+func (*GetExtValueMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{17}
+}
+
+func (m *GetExtValueMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GetExtValueMessage.Unmarshal(m, b)
+}
+func (m *GetExtValueMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GetExtValueMessage.Marshal(b, m, deterministic)
+}
+func (m *GetExtValueMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetExtValueMessage.Merge(m, src)
+}
+func (m *GetExtValueMessage) XXX_Size() int {
+	return xxx_messageInfo_GetExtValueMessage.Size(m)
+}
+func (m *GetExtValueMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_GetExtValueMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GetExtValueMessage proto.InternalMessageInfo
+
+func (m *GetExtValueMessage) GetParentDevice() *voltha.Device {
+	if m != nil {
+		return m.ParentDevice
+	}
+	return nil
+}
+
+func (m *GetExtValueMessage) GetChildDevice() *voltha.Device {
+	if m != nil {
+		return m.ChildDevice
+	}
+	return nil
+}
+
+func (m *GetExtValueMessage) GetValueType() common.ValueType_Type {
+	if m != nil {
+		return m.ValueType
+	}
+	return common.ValueType_EMPTY
+}
+
+type SetExtValueMessage struct {
+	Device               *voltha.Device   `protobuf:"bytes,1,opt,name=device,proto3" json:"device,omitempty"`
+	Value                *voltha.ValueSet `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *SetExtValueMessage) Reset()         { *m = SetExtValueMessage{} }
+func (m *SetExtValueMessage) String() string { return proto.CompactTextString(m) }
+func (*SetExtValueMessage) ProtoMessage()    {}
+func (*SetExtValueMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{18}
+}
+
+func (m *SetExtValueMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SetExtValueMessage.Unmarshal(m, b)
+}
+func (m *SetExtValueMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SetExtValueMessage.Marshal(b, m, deterministic)
+}
+func (m *SetExtValueMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SetExtValueMessage.Merge(m, src)
+}
+func (m *SetExtValueMessage) XXX_Size() int {
+	return xxx_messageInfo_SetExtValueMessage.Size(m)
+}
+func (m *SetExtValueMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_SetExtValueMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SetExtValueMessage proto.InternalMessageInfo
+
+func (m *SetExtValueMessage) GetDevice() *voltha.Device {
+	if m != nil {
+		return m.Device
+	}
+	return nil
+}
+
+func (m *SetExtValueMessage) GetValue() *voltha.ValueSet {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+type OmciMessage struct {
+	Message              []byte                      `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"`
+	ConnectStatus        common.ConnectStatus_Types  `protobuf:"varint,2,opt,name=connect_status,json=connectStatus,proto3,enum=common.ConnectStatus_Types" json:"connect_status,omitempty"`
+	ProxyAddress         *voltha.Device_ProxyAddress `protobuf:"bytes,3,opt,name=proxy_address,json=proxyAddress,proto3" json:"proxy_address,omitempty"`
+	ParentDeviceId       string                      `protobuf:"bytes,4,opt,name=parent_device_id,json=parentDeviceId,proto3" json:"parent_device_id,omitempty"`
+	ChildDeviceId        string                      `protobuf:"bytes,5,opt,name=child_device_id,json=childDeviceId,proto3" json:"child_device_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
+	XXX_unrecognized     []byte                      `json:"-"`
+	XXX_sizecache        int32                       `json:"-"`
+}
+
+func (m *OmciMessage) Reset()         { *m = OmciMessage{} }
+func (m *OmciMessage) String() string { return proto.CompactTextString(m) }
+func (*OmciMessage) ProtoMessage()    {}
+func (*OmciMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{19}
+}
+
+func (m *OmciMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OmciMessage.Unmarshal(m, b)
+}
+func (m *OmciMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OmciMessage.Marshal(b, m, deterministic)
+}
+func (m *OmciMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OmciMessage.Merge(m, src)
+}
+func (m *OmciMessage) XXX_Size() int {
+	return xxx_messageInfo_OmciMessage.Size(m)
+}
+func (m *OmciMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_OmciMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OmciMessage proto.InternalMessageInfo
+
+func (m *OmciMessage) GetMessage() []byte {
+	if m != nil {
+		return m.Message
+	}
+	return nil
+}
+
+func (m *OmciMessage) GetConnectStatus() common.ConnectStatus_Types {
+	if m != nil {
+		return m.ConnectStatus
+	}
+	return common.ConnectStatus_UNKNOWN
+}
+
+func (m *OmciMessage) GetProxyAddress() *voltha.Device_ProxyAddress {
+	if m != nil {
+		return m.ProxyAddress
+	}
+	return nil
+}
+
+func (m *OmciMessage) GetParentDeviceId() string {
+	if m != nil {
+		return m.ParentDeviceId
+	}
+	return ""
+}
+
+func (m *OmciMessage) GetChildDeviceId() string {
+	if m != nil {
+		return m.ChildDeviceId
+	}
+	return ""
+}
+
+type TechProfileDownloadMessage struct {
+	DeviceId       string `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	UniId          uint32 `protobuf:"varint,2,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	TpInstancePath string `protobuf:"bytes,3,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
+	// Types that are valid to be assigned to TechTpInstance:
+	//	*TechProfileDownloadMessage_TpInstance
+	//	*TechProfileDownloadMessage_EponTpInstance
+	TechTpInstance       isTechProfileDownloadMessage_TechTpInstance `protobuf_oneof:"tech_tp_instance"`
+	XXX_NoUnkeyedLiteral struct{}                                    `json:"-"`
+	XXX_unrecognized     []byte                                      `json:"-"`
+	XXX_sizecache        int32                                       `json:"-"`
+}
+
+func (m *TechProfileDownloadMessage) Reset()         { *m = TechProfileDownloadMessage{} }
+func (m *TechProfileDownloadMessage) String() string { return proto.CompactTextString(m) }
+func (*TechProfileDownloadMessage) ProtoMessage()    {}
+func (*TechProfileDownloadMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{20}
+}
+
+func (m *TechProfileDownloadMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TechProfileDownloadMessage.Unmarshal(m, b)
+}
+func (m *TechProfileDownloadMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TechProfileDownloadMessage.Marshal(b, m, deterministic)
+}
+func (m *TechProfileDownloadMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TechProfileDownloadMessage.Merge(m, src)
+}
+func (m *TechProfileDownloadMessage) XXX_Size() int {
+	return xxx_messageInfo_TechProfileDownloadMessage.Size(m)
+}
+func (m *TechProfileDownloadMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_TechProfileDownloadMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TechProfileDownloadMessage proto.InternalMessageInfo
+
+func (m *TechProfileDownloadMessage) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *TechProfileDownloadMessage) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *TechProfileDownloadMessage) GetTpInstancePath() string {
+	if m != nil {
+		return m.TpInstancePath
+	}
+	return ""
+}
+
+type isTechProfileDownloadMessage_TechTpInstance interface {
+	isTechProfileDownloadMessage_TechTpInstance()
+}
+
+type TechProfileDownloadMessage_TpInstance struct {
+	TpInstance *tech_profile.TechProfileInstance `protobuf:"bytes,4,opt,name=tp_instance,json=tpInstance,proto3,oneof"`
+}
+
+type TechProfileDownloadMessage_EponTpInstance struct {
+	EponTpInstance *tech_profile.EponTechProfileInstance `protobuf:"bytes,5,opt,name=epon_tp_instance,json=eponTpInstance,proto3,oneof"`
+}
+
+func (*TechProfileDownloadMessage_TpInstance) isTechProfileDownloadMessage_TechTpInstance() {}
+
+func (*TechProfileDownloadMessage_EponTpInstance) isTechProfileDownloadMessage_TechTpInstance() {}
+
+func (m *TechProfileDownloadMessage) GetTechTpInstance() isTechProfileDownloadMessage_TechTpInstance {
+	if m != nil {
+		return m.TechTpInstance
+	}
+	return nil
+}
+
+func (m *TechProfileDownloadMessage) GetTpInstance() *tech_profile.TechProfileInstance {
+	if x, ok := m.GetTechTpInstance().(*TechProfileDownloadMessage_TpInstance); ok {
+		return x.TpInstance
+	}
+	return nil
+}
+
+func (m *TechProfileDownloadMessage) GetEponTpInstance() *tech_profile.EponTechProfileInstance {
+	if x, ok := m.GetTechTpInstance().(*TechProfileDownloadMessage_EponTpInstance); ok {
+		return x.EponTpInstance
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*TechProfileDownloadMessage) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*TechProfileDownloadMessage_TpInstance)(nil),
+		(*TechProfileDownloadMessage_EponTpInstance)(nil),
+	}
+}
+
+type DeleteGemPortMessage struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	UniId                uint32   `protobuf:"varint,2,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	TpInstancePath       string   `protobuf:"bytes,3,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
+	GemPortId            uint32   `protobuf:"varint,4,opt,name=gem_port_id,json=gemPortId,proto3" json:"gem_port_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeleteGemPortMessage) Reset()         { *m = DeleteGemPortMessage{} }
+func (m *DeleteGemPortMessage) String() string { return proto.CompactTextString(m) }
+func (*DeleteGemPortMessage) ProtoMessage()    {}
+func (*DeleteGemPortMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{21}
+}
+
+func (m *DeleteGemPortMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeleteGemPortMessage.Unmarshal(m, b)
+}
+func (m *DeleteGemPortMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeleteGemPortMessage.Marshal(b, m, deterministic)
+}
+func (m *DeleteGemPortMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeleteGemPortMessage.Merge(m, src)
+}
+func (m *DeleteGemPortMessage) XXX_Size() int {
+	return xxx_messageInfo_DeleteGemPortMessage.Size(m)
+}
+func (m *DeleteGemPortMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeleteGemPortMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeleteGemPortMessage proto.InternalMessageInfo
+
+func (m *DeleteGemPortMessage) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeleteGemPortMessage) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *DeleteGemPortMessage) GetTpInstancePath() string {
+	if m != nil {
+		return m.TpInstancePath
+	}
+	return ""
+}
+
+func (m *DeleteGemPortMessage) GetGemPortId() uint32 {
+	if m != nil {
+		return m.GemPortId
+	}
+	return 0
+}
+
+type DeleteTcontMessage struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	UniId                uint32   `protobuf:"varint,2,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	TpInstancePath       string   `protobuf:"bytes,3,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
+	AllocId              uint32   `protobuf:"varint,4,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeleteTcontMessage) Reset()         { *m = DeleteTcontMessage{} }
+func (m *DeleteTcontMessage) String() string { return proto.CompactTextString(m) }
+func (*DeleteTcontMessage) ProtoMessage()    {}
+func (*DeleteTcontMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{22}
+}
+
+func (m *DeleteTcontMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeleteTcontMessage.Unmarshal(m, b)
+}
+func (m *DeleteTcontMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeleteTcontMessage.Marshal(b, m, deterministic)
+}
+func (m *DeleteTcontMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeleteTcontMessage.Merge(m, src)
+}
+func (m *DeleteTcontMessage) XXX_Size() int {
+	return xxx_messageInfo_DeleteTcontMessage.Size(m)
+}
+func (m *DeleteTcontMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeleteTcontMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeleteTcontMessage proto.InternalMessageInfo
+
+func (m *DeleteTcontMessage) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeleteTcontMessage) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *DeleteTcontMessage) GetTpInstancePath() string {
+	if m != nil {
+		return m.TpInstancePath
+	}
+	return ""
+}
+
+func (m *DeleteTcontMessage) GetAllocId() uint32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+type OnuIndicationMessage struct {
+	DeviceId             string                 `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	OnuIndication        *openolt.OnuIndication `protobuf:"bytes,2,opt,name=onu_indication,json=onuIndication,proto3" json:"onu_indication,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
+	XXX_unrecognized     []byte                 `json:"-"`
+	XXX_sizecache        int32                  `json:"-"`
+}
+
+func (m *OnuIndicationMessage) Reset()         { *m = OnuIndicationMessage{} }
+func (m *OnuIndicationMessage) String() string { return proto.CompactTextString(m) }
+func (*OnuIndicationMessage) ProtoMessage()    {}
+func (*OnuIndicationMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{23}
+}
+
+func (m *OnuIndicationMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuIndicationMessage.Unmarshal(m, b)
+}
+func (m *OnuIndicationMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuIndicationMessage.Marshal(b, m, deterministic)
+}
+func (m *OnuIndicationMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuIndicationMessage.Merge(m, src)
+}
+func (m *OnuIndicationMessage) XXX_Size() int {
+	return xxx_messageInfo_OnuIndicationMessage.Size(m)
+}
+func (m *OnuIndicationMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuIndicationMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuIndicationMessage proto.InternalMessageInfo
+
+func (m *OnuIndicationMessage) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *OnuIndicationMessage) GetOnuIndication() *openolt.OnuIndication {
+	if m != nil {
+		return m.OnuIndication
+	}
+	return nil
+}
+
+type TechProfileInstanceRequestMessage struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	TpInstancePath       string   `protobuf:"bytes,2,opt,name=tp_instance_path,json=tpInstancePath,proto3" json:"tp_instance_path,omitempty"`
+	ParentDeviceId       string   `protobuf:"bytes,3,opt,name=parent_device_id,json=parentDeviceId,proto3" json:"parent_device_id,omitempty"`
+	ParentPonPort        uint32   `protobuf:"varint,4,opt,name=parent_pon_port,json=parentPonPort,proto3" json:"parent_pon_port,omitempty"`
+	OnuId                uint32   `protobuf:"varint,5,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	UniId                uint32   `protobuf:"varint,6,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *TechProfileInstanceRequestMessage) Reset()         { *m = TechProfileInstanceRequestMessage{} }
+func (m *TechProfileInstanceRequestMessage) String() string { return proto.CompactTextString(m) }
+func (*TechProfileInstanceRequestMessage) ProtoMessage()    {}
+func (*TechProfileInstanceRequestMessage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_941f0031a549667f, []int{24}
+}
+
+func (m *TechProfileInstanceRequestMessage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TechProfileInstanceRequestMessage.Unmarshal(m, b)
+}
+func (m *TechProfileInstanceRequestMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TechProfileInstanceRequestMessage.Marshal(b, m, deterministic)
+}
+func (m *TechProfileInstanceRequestMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TechProfileInstanceRequestMessage.Merge(m, src)
+}
+func (m *TechProfileInstanceRequestMessage) XXX_Size() int {
+	return xxx_messageInfo_TechProfileInstanceRequestMessage.Size(m)
+}
+func (m *TechProfileInstanceRequestMessage) XXX_DiscardUnknown() {
+	xxx_messageInfo_TechProfileInstanceRequestMessage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TechProfileInstanceRequestMessage proto.InternalMessageInfo
+
+func (m *TechProfileInstanceRequestMessage) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *TechProfileInstanceRequestMessage) GetTpInstancePath() string {
+	if m != nil {
+		return m.TpInstancePath
+	}
+	return ""
+}
+
+func (m *TechProfileInstanceRequestMessage) GetParentDeviceId() string {
+	if m != nil {
+		return m.ParentDeviceId
+	}
+	return ""
+}
+
+func (m *TechProfileInstanceRequestMessage) GetParentPonPort() uint32 {
+	if m != nil {
+		return m.ParentPonPort
+	}
+	return 0
+}
+
+func (m *TechProfileInstanceRequestMessage) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *TechProfileInstanceRequestMessage) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func init() {
+	proto.RegisterType((*AdapterRegistration)(nil), "voltha.AdapterRegistration")
+	proto.RegisterType((*ChildDeviceFilter)(nil), "voltha.ChildDeviceFilter")
+	proto.RegisterType((*PortFilter)(nil), "voltha.PortFilter")
+	proto.RegisterType((*DeviceDiscovery)(nil), "voltha.DeviceDiscovery")
+	proto.RegisterType((*DeviceStateFilter)(nil), "voltha.DeviceStateFilter")
+	proto.RegisterType((*PortState)(nil), "voltha.PortState")
+	proto.RegisterType((*PortStateFilter)(nil), "voltha.PortStateFilter")
+	proto.RegisterType((*PacketIn)(nil), "voltha.PacketIn")
+	proto.RegisterType((*PacketOut)(nil), "voltha.PacketOut")
+	proto.RegisterType((*DeviceReason)(nil), "voltha.DeviceReason")
+	proto.RegisterType((*BulkFlows)(nil), "voltha.BulkFlows")
+	proto.RegisterType((*IncrementalFlows)(nil), "voltha.IncrementalFlows")
+	proto.RegisterType((*PmConfigsInfo)(nil), "voltha.PmConfigsInfo")
+	proto.RegisterType((*SwitchCapability)(nil), "voltha.SwitchCapability")
+	proto.RegisterType((*ImageDownloadMessage)(nil), "voltha.ImageDownloadMessage")
+	proto.RegisterType((*OMCITest)(nil), "voltha.OMCITest")
+	proto.RegisterType((*SimulateAlarmMessage)(nil), "voltha.SimulateAlarmMessage")
+	proto.RegisterType((*GetExtValueMessage)(nil), "voltha.GetExtValueMessage")
+	proto.RegisterType((*SetExtValueMessage)(nil), "voltha.SetExtValueMessage")
+	proto.RegisterType((*OmciMessage)(nil), "voltha.OmciMessage")
+	proto.RegisterType((*TechProfileDownloadMessage)(nil), "voltha.TechProfileDownloadMessage")
+	proto.RegisterType((*DeleteGemPortMessage)(nil), "voltha.DeleteGemPortMessage")
+	proto.RegisterType((*DeleteTcontMessage)(nil), "voltha.DeleteTcontMessage")
+	proto.RegisterType((*OnuIndicationMessage)(nil), "voltha.OnuIndicationMessage")
+	proto.RegisterType((*TechProfileInstanceRequestMessage)(nil), "voltha.TechProfileInstanceRequestMessage")
+}
+
+func init() {
+	proto.RegisterFile("voltha_protos/inter_container.proto", fileDescriptor_941f0031a549667f)
+}
+
+var fileDescriptor_941f0031a549667f = []byte{
+	// 1440 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4f, 0x6f, 0xdb, 0xc6,
+	0x12, 0x7f, 0x74, 0x2c, 0xd9, 0x1a, 0xfd, 0xb1, 0xbd, 0xb1, 0x1d, 0xbf, 0xf8, 0xe5, 0x3d, 0x87,
+	0xc9, 0x0b, 0x9c, 0x06, 0x95, 0x1b, 0x1b, 0x69, 0x11, 0xa0, 0x05, 0x1a, 0xcb, 0x49, 0xca, 0x43,
+	0x62, 0x97, 0x32, 0x7a, 0xe8, 0xa1, 0xc4, 0x9a, 0x5c, 0x49, 0x44, 0xc8, 0x5d, 0x86, 0x5c, 0xca,
+	0xf1, 0xb9, 0x97, 0x5e, 0x7a, 0x69, 0xfb, 0x1d, 0x7a, 0x2d, 0xd0, 0x4f, 0xd1, 0x73, 0xaf, 0xfd,
+	0x04, 0xfd, 0x04, 0xbd, 0xb5, 0xd8, 0x7f, 0x12, 0x29, 0x29, 0x8e, 0x82, 0x16, 0xb9, 0x08, 0xdc,
+	0x9d, 0xdf, 0xcc, 0xfc, 0x66, 0x66, 0x77, 0x76, 0x57, 0x70, 0x6b, 0xc8, 0x22, 0x3e, 0xc0, 0x5e,
+	0x92, 0x32, 0xce, 0xb2, 0xbd, 0x90, 0x72, 0x92, 0x7a, 0x3e, 0xa3, 0x1c, 0x87, 0x94, 0xa4, 0x6d,
+	0x39, 0x8d, 0xaa, 0x0a, 0x74, 0xfd, 0x7a, 0x19, 0xec, 0xb3, 0x38, 0x66, 0x54, 0x61, 0x26, 0x65,
+	0x6a, 0xa4, 0x65, 0xff, 0x2b, 0xcb, 0x58, 0x42, 0x68, 0x2f, 0x62, 0xe7, 0xde, 0xfd, 0x03, 0x0d,
+	0xd8, 0x29, 0x03, 0x38, 0xf1, 0x07, 0xe2, 0xbb, 0x17, 0x46, 0x44, 0x23, 0xb6, 0xa7, 0x4d, 0xb0,
+	0x88, 0xcf, 0x16, 0xe2, 0x00, 0x27, 0xdc, 0x90, 0x9f, 0x24, 0x16, 0x90, 0x61, 0xe8, 0x6b, 0xab,
+	0x76, 0x0c, 0x57, 0x1f, 0x29, 0xb0, 0x4b, 0xfa, 0x61, 0xc6, 0x53, 0xcc, 0x43, 0x46, 0xd1, 0x5d,
+	0x58, 0xd2, 0x36, 0xb6, 0xac, 0x1d, 0x6b, 0xb7, 0xbe, 0xbf, 0xd2, 0xd6, 0xf1, 0x18, 0xb4, 0x91,
+	0xa3, 0x7b, 0x50, 0x0d, 0x4e, 0x2f, 0x12, 0x92, 0x6d, 0x2d, 0x48, 0xe4, 0x55, 0x83, 0x3c, 0x92,
+	0x7e, 0xa4, 0xc8, 0xd5, 0x10, 0xfb, 0x3b, 0x0b, 0xd6, 0x3a, 0x83, 0x30, 0x0a, 0x94, 0xf0, 0x49,
+	0x18, 0x09, 0x13, 0xdb, 0x50, 0x4b, 0x70, 0x4a, 0x28, 0xf7, 0xc2, 0x40, 0xfa, 0xab, 0xb9, 0xcb,
+	0x6a, 0xc2, 0x09, 0xd0, 0x2d, 0x68, 0x66, 0x24, 0x0d, 0x71, 0xe4, 0xd1, 0x3c, 0x3e, 0x23, 0xa9,
+	0x74, 0x53, 0x73, 0x1b, 0x6a, 0xf2, 0xb9, 0x9c, 0x43, 0x1b, 0x50, 0x65, 0x34, 0x17, 0xea, 0x57,
+	0x76, 0xac, 0xdd, 0xa6, 0x5b, 0x61, 0x34, 0x77, 0x02, 0x74, 0x1b, 0x5a, 0xda, 0x70, 0xc2, 0x52,
+	0xee, 0x51, 0xb6, 0xb5, 0x28, 0xc5, 0x0d, 0x35, 0x7b, 0xc2, 0x52, 0xfe, 0x9c, 0xd9, 0x2f, 0x01,
+	0xc4, 0xd7, 0x98, 0x8c, 0xca, 0x50, 0x81, 0x8c, 0x9a, 0x70, 0x02, 0x84, 0x60, 0x51, 0x58, 0x92,
+	0x1c, 0x9a, 0xae, 0xfc, 0x46, 0xfb, 0x50, 0x93, 0xd6, 0xf9, 0x45, 0x42, 0xa4, 0xfb, 0xd6, 0xfe,
+	0x86, 0xc9, 0x81, 0xb0, 0x2b, 0x7f, 0x44, 0xf8, 0xee, 0x72, 0xa2, 0xbf, 0xec, 0x3f, 0x2c, 0x58,
+	0x51, 0x29, 0x38, 0x0a, 0x33, 0x9f, 0x0d, 0x49, 0x7a, 0x71, 0x79, 0x16, 0xa6, 0x23, 0x59, 0x98,
+	0x8e, 0x04, 0xbd, 0x07, 0x6b, 0xbe, 0xc8, 0xae, 0xa7, 0x23, 0x18, 0x51, 0xaa, 0xb9, 0x2b, 0xfe,
+	0x38, 0xed, 0x82, 0x02, 0xba, 0x01, 0xe0, 0x0f, 0x30, 0xa5, 0x24, 0x12, 0xfe, 0x54, 0x5e, 0x6a,
+	0x7a, 0xc6, 0x09, 0x04, 0x9b, 0x21, 0xa1, 0x01, 0x4b, 0x85, 0xb4, 0xa2, 0xd8, 0xa8, 0x89, 0x59,
+	0x35, 0xa9, 0x5e, 0x5a, 0x93, 0xa5, 0x42, 0x4d, 0xec, 0x5f, 0x2d, 0x58, 0x53, 0x34, 0xba, 0x1c,
+	0x73, 0x32, 0x4f, 0xd6, 0x77, 0x61, 0x55, 0x07, 0x3f, 0xc6, 0xa8, 0x55, 0xa0, 0x93, 0x72, 0x64,
+	0x90, 0x0f, 0xa1, 0xce, 0x12, 0x92, 0x7a, 0x19, 0xc7, 0x3c, 0xcf, 0x74, 0x35, 0xb6, 0xda, 0x7a,
+	0x9f, 0x1e, 0x27, 0x24, 0xed, 0x4a, 0x49, 0x5b, 0x2d, 0x4b, 0x60, 0xa3, 0x19, 0xf4, 0x31, 0xd4,
+	0x7d, 0x46, 0xa9, 0x51, 0x5d, 0x94, 0xaa, 0xdb, 0x46, 0xb5, 0xc3, 0x28, 0x25, 0x3e, 0x2f, 0x6b,
+	0x0b, 0xbc, 0x9a, 0xb1, 0x7f, 0xb2, 0xa0, 0x26, 0x8a, 0x20, 0x63, 0xba, 0x3c, 0x9a, 0xd2, 0x7a,
+	0x59, 0x98, 0x6b, 0xbd, 0xa0, 0x6b, 0xb0, 0x64, 0xea, 0xae, 0x16, 0x78, 0x35, 0x51, 0x15, 0x9f,
+	0x08, 0x78, 0x71, 0xfe, 0x80, 0xed, 0xef, 0x2d, 0x58, 0x19, 0x51, 0x9e, 0xb7, 0x0c, 0x86, 0xb8,
+	0xd7, 0x93, 0x0a, 0x7a, 0x15, 0xb6, 0x0c, 0x51, 0x6d, 0xe6, 0x6f, 0xb0, 0xea, 0xc2, 0xf2, 0x09,
+	0xf6, 0x5f, 0x10, 0xee, 0xd0, 0xb7, 0xdf, 0x8a, 0x9b, 0x50, 0x4d, 0xa4, 0xb2, 0xcc, 0x52, 0xc3,
+	0xd5, 0x23, 0xfb, 0x6b, 0x51, 0x1d, 0xf9, 0x79, 0x9c, 0xf3, 0xcb, 0xcd, 0xde, 0x86, 0x16, 0xe9,
+	0xa7, 0x24, 0xcb, 0x26, 0x37, 0x9a, 0x9a, 0xd5, 0x1b, 0xed, 0xa0, 0xe4, 0xa8, 0xbe, 0xbf, 0xdd,
+	0x2e, 0xb6, 0x74, 0xd6, 0x4b, 0x3c, 0x25, 0xf6, 0x58, 0xce, 0x47, 0x2c, 0x3a, 0xd0, 0x50, 0x0b,
+	0xd5, 0x25, 0x38, 0x63, 0x6f, 0x08, 0x6f, 0x13, 0xaa, 0xa9, 0x84, 0xe9, 0x95, 0xae, 0x47, 0xf6,
+	0x2f, 0x16, 0xd4, 0x0e, 0xf3, 0xe8, 0xc5, 0x93, 0x88, 0x9d, 0x67, 0xe8, 0x0e, 0x54, 0x95, 0x86,
+	0x6e, 0xd3, 0xad, 0x72, 0xf3, 0x75, 0xb5, 0x14, 0xed, 0x42, 0x45, 0x90, 0x33, 0x3d, 0x1a, 0x95,
+	0xe8, 0x4a, 0x53, 0xae, 0x02, 0xa0, 0x3d, 0xa8, 0xf6, 0x53, 0x96, 0x27, 0x99, 0x8e, 0xec, 0xda,
+	0x14, 0xf4, 0xa9, 0x14, 0xbb, 0x1a, 0x86, 0x1e, 0x42, 0x53, 0x4a, 0x63, 0xc2, 0x71, 0x80, 0x39,
+	0x96, 0xd5, 0xae, 0xef, 0xaf, 0x1b, 0x26, 0x42, 0xe5, 0x99, 0x96, 0xb9, 0x8d, 0x5e, 0x61, 0x64,
+	0xff, 0x66, 0xc1, 0xaa, 0x43, 0xfd, 0x94, 0xc4, 0x84, 0x72, 0x1c, 0xbd, 0x5d, 0x48, 0xed, 0x72,
+	0x48, 0x5b, 0x53, 0x3c, 0x3b, 0x03, 0x4c, 0xfb, 0x64, 0x14, 0xd8, 0x83, 0x89, 0xc0, 0x6e, 0xcc,
+	0x0e, 0xcc, 0x68, 0xfd, 0x03, 0xe1, 0x7d, 0x05, 0xcd, 0x93, 0xb8, 0xc3, 0x68, 0x2f, 0xec, 0x67,
+	0x0e, 0xed, 0xb1, 0xcb, 0x0b, 0xfe, 0x01, 0x40, 0x12, 0x8b, 0x8b, 0x87, 0x80, 0xeb, 0xa0, 0xd6,
+	0x46, 0x7d, 0xc1, 0xd8, 0x71, 0x6b, 0x89, 0xf9, 0xb4, 0xbf, 0xb1, 0x60, 0xb5, 0x7b, 0x1e, 0x72,
+	0x7f, 0xd0, 0xc1, 0x09, 0x3e, 0x0b, 0xa3, 0x90, 0x5f, 0xa0, 0xbb, 0xb0, 0x18, 0x90, 0xcc, 0xd7,
+	0xc9, 0xdb, 0x98, 0x5a, 0x97, 0x42, 0xe8, 0x4a, 0x08, 0x72, 0x60, 0x25, 0x93, 0xea, 0x5e, 0x8f,
+	0x60, 0x9e, 0xa7, 0xa3, 0x23, 0x7c, 0x67, 0x4a, 0x6b, 0x02, 0xe7, 0xb6, 0xd4, 0xc4, 0x13, 0x3d,
+	0xb6, 0x5f, 0xc0, 0xba, 0x13, 0xe3, 0x3e, 0x39, 0x62, 0xe7, 0x34, 0x62, 0x38, 0x78, 0x46, 0xb2,
+	0x0c, 0xf7, 0xc9, 0xdc, 0xc5, 0xbc, 0x07, 0x95, 0x50, 0xe8, 0x6b, 0x02, 0xa3, 0x7e, 0x58, 0x32,
+	0xea, 0x2a, 0x8c, 0x4d, 0x60, 0xf9, 0xf8, 0x59, 0xc7, 0x39, 0x25, 0x19, 0x9f, 0xdb, 0xc1, 0x7d,
+	0x58, 0x4a, 0xc9, 0xcb, 0x9c, 0x64, 0x5c, 0xbb, 0xb8, 0x66, 0x80, 0xc7, 0xb1, 0x1f, 0x0a, 0x53,
+	0xae, 0x12, 0xbb, 0x06, 0x67, 0x0f, 0x61, 0xbd, 0x1b, 0xc6, 0x79, 0x84, 0x39, 0x79, 0x14, 0xe1,
+	0x34, 0x7e, 0xdb, 0x98, 0x3e, 0x9c, 0x74, 0xf9, 0x1f, 0x03, 0x2c, 0x99, 0x9d, 0xf2, 0xfb, 0xb3,
+	0x05, 0xe8, 0x29, 0xe1, 0x8f, 0x5f, 0xf1, 0x2f, 0x70, 0x94, 0x13, 0xe3, 0xf6, 0x00, 0x9a, 0xa5,
+	0x43, 0xf0, 0x35, 0xde, 0x1b, 0xc5, 0x13, 0x11, 0xdd, 0x87, 0x46, 0xf1, 0x42, 0xa0, 0x89, 0x4c,
+	0xea, 0xd4, 0x0b, 0x77, 0x03, 0xf4, 0x00, 0x60, 0x28, 0xfc, 0x16, 0xef, 0x33, 0x9b, 0xa6, 0x75,
+	0x4b, 0x46, 0xa2, 0x69, 0xcb, 0xce, 0xed, 0xd6, 0x86, 0x66, 0x6c, 0x07, 0x80, 0xba, 0xd3, 0xa4,
+	0xe7, 0xcd, 0xd5, 0x1d, 0xa8, 0x48, 0x53, 0x7a, 0x6f, 0xae, 0x1a, 0x98, 0x34, 0xd6, 0x25, 0xdc,
+	0x55, 0x62, 0xfb, 0x4f, 0x0b, 0xea, 0xa2, 0x60, 0xc6, 0xfe, 0x16, 0x2c, 0xc5, 0xea, 0x53, 0x3a,
+	0x68, 0xb8, 0x66, 0x88, 0x0e, 0xa1, 0xe5, 0xab, 0x33, 0xdb, 0x9c, 0x42, 0x0b, 0x6f, 0x3e, 0xd1,
+	0x9b, 0x7e, 0x71, 0x12, 0x7d, 0x0a, 0xcd, 0x24, 0x65, 0xaf, 0x2e, 0x3c, 0x1c, 0x04, 0xa2, 0xf9,
+	0x8f, 0x9a, 0x7d, 0x29, 0x88, 0xf6, 0x89, 0xc0, 0x3c, 0x52, 0x10, 0xb7, 0x91, 0x14, 0x46, 0x33,
+	0x6f, 0x2e, 0x8b, 0x33, 0x6f, 0x2e, 0x77, 0x60, 0xa5, 0x74, 0x75, 0x1b, 0xdd, 0xba, 0x9a, 0x85,
+	0xe2, 0x38, 0x81, 0xfd, 0xe3, 0x02, 0x5c, 0x3f, 0x25, 0xfe, 0xe0, 0x44, 0x3d, 0x0e, 0x26, 0x37,
+	0xdc, 0xa5, 0x2d, 0x66, 0x03, 0xaa, 0x39, 0x0d, 0xcd, 0xed, 0xa9, 0xe9, 0x56, 0x72, 0x1a, 0xaa,
+	0x73, 0x9d, 0x27, 0x5e, 0x48, 0x33, 0x8e, 0xa9, 0x4f, 0xbc, 0x04, 0xf3, 0x81, 0xbe, 0x34, 0xb6,
+	0x78, 0xe2, 0xe8, 0xe9, 0x13, 0xcc, 0x07, 0xe8, 0x08, 0xea, 0x05, 0xa4, 0x6e, 0x85, 0x37, 0xdb,
+	0xa5, 0xd7, 0x4a, 0x81, 0x9c, 0xd1, 0xfd, 0xec, 0x5f, 0x2e, 0x8c, 0x2d, 0xa1, 0xcf, 0x61, 0x95,
+	0x24, 0x8c, 0x7a, 0x45, 0x53, 0x15, 0x69, 0xea, 0xff, 0x65, 0x53, 0x8f, 0x13, 0x46, 0x67, 0x9b,
+	0x6b, 0x09, 0x03, 0xa7, 0x23, 0x93, 0x87, 0x08, 0x56, 0xa5, 0x66, 0xc1, 0xa4, 0xfd, 0x83, 0x05,
+	0xeb, 0x47, 0x24, 0x22, 0x9c, 0x3c, 0x25, 0xb1, 0x38, 0xb7, 0xdf, 0x4d, 0x8e, 0xfe, 0x0b, 0xf5,
+	0x3e, 0x89, 0xd5, 0xed, 0x61, 0x7c, 0xb1, 0xee, 0x2b, 0x0a, 0x4e, 0x60, 0x7f, 0x6b, 0x01, 0x52,
+	0xb4, 0x4e, 0xc5, 0x2b, 0xf3, 0xdd, 0x90, 0xfa, 0x37, 0x2c, 0xe3, 0x28, 0x62, 0xfe, 0x98, 0xd1,
+	0x92, 0x1c, 0x3b, 0x81, 0x9d, 0xc2, 0xfa, 0x31, 0xcd, 0x1d, 0x1a, 0x84, 0xbe, 0x7c, 0xfb, 0xcd,
+	0x45, 0xe8, 0x13, 0x68, 0xc9, 0xbb, 0xfd, 0x48, 0x4b, 0x77, 0x96, 0xcd, 0xb6, 0x79, 0x97, 0x96,
+	0x6c, 0xba, 0x4d, 0x56, 0x1c, 0xda, 0xbf, 0x5b, 0x70, 0x73, 0x46, 0x61, 0x75, 0x2b, 0x9c, 0x8b,
+	0xc1, 0xac, 0xd8, 0x17, 0x66, 0xc6, 0x3e, 0x6b, 0x0f, 0x5e, 0x79, 0xdd, 0x1e, 0x1c, 0x3d, 0xb2,
+	0xa8, 0xac, 0xa0, 0x4e, 0x56, 0xd3, 0xbc, 0xb2, 0xa8, 0x28, 0x62, 0xe1, 0x65, 0x53, 0x29, 0xbe,
+	0x36, 0xc7, 0x55, 0xaa, 0x16, 0xaa, 0x74, 0xf8, 0x1c, 0xae, 0xb2, 0xb4, 0x2f, 0x13, 0xe3, 0xb3,
+	0x34, 0xd0, 0xcd, 0xe3, 0xcb, 0x8f, 0xfa, 0x21, 0x1f, 0xe4, 0x67, 0xa2, 0x1d, 0xed, 0x19, 0x99,
+	0xfe, 0xd3, 0xe0, 0x7d, 0xf3, 0x17, 0xc2, 0x83, 0xbd, 0x3e, 0x9b, 0xfc, 0x47, 0xe2, 0xac, 0x2a,
+	0xa5, 0x07, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x9f, 0x82, 0x0a, 0xe6, 0xb9, 0x10, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_alarm_db.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_alarm_db.pb.go
new file mode 100644
index 0000000..7b8e492
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_alarm_db.pb.go
@@ -0,0 +1,516 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/omci_alarm_db.proto
+
+package omci
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_ "github.com/opencord/voltha-protos/v5/go/common"
+	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 AlarmOpenOmciEventType_OpenOmciEventType int32
+
+const (
+	AlarmOpenOmciEventType_state_change AlarmOpenOmciEventType_OpenOmciEventType = 0
+)
+
+var AlarmOpenOmciEventType_OpenOmciEventType_name = map[int32]string{
+	0: "state_change",
+}
+
+var AlarmOpenOmciEventType_OpenOmciEventType_value = map[string]int32{
+	"state_change": 0,
+}
+
+func (x AlarmOpenOmciEventType_OpenOmciEventType) String() string {
+	return proto.EnumName(AlarmOpenOmciEventType_OpenOmciEventType_name, int32(x))
+}
+
+func (AlarmOpenOmciEventType_OpenOmciEventType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{6, 0}
+}
+
+type AlarmAttributeData struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AlarmAttributeData) Reset()         { *m = AlarmAttributeData{} }
+func (m *AlarmAttributeData) String() string { return proto.CompactTextString(m) }
+func (*AlarmAttributeData) ProtoMessage()    {}
+func (*AlarmAttributeData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{0}
+}
+
+func (m *AlarmAttributeData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmAttributeData.Unmarshal(m, b)
+}
+func (m *AlarmAttributeData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmAttributeData.Marshal(b, m, deterministic)
+}
+func (m *AlarmAttributeData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmAttributeData.Merge(m, src)
+}
+func (m *AlarmAttributeData) XXX_Size() int {
+	return xxx_messageInfo_AlarmAttributeData.Size(m)
+}
+func (m *AlarmAttributeData) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmAttributeData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmAttributeData proto.InternalMessageInfo
+
+func (m *AlarmAttributeData) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *AlarmAttributeData) GetValue() string {
+	if m != nil {
+		return m.Value
+	}
+	return ""
+}
+
+type AlarmInstanceData struct {
+	InstanceId           uint32                `protobuf:"varint,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
+	Created              string                `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
+	Modified             string                `protobuf:"bytes,3,opt,name=modified,proto3" json:"modified,omitempty"`
+	Attributes           []*AlarmAttributeData `protobuf:"bytes,4,rep,name=attributes,proto3" json:"attributes,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *AlarmInstanceData) Reset()         { *m = AlarmInstanceData{} }
+func (m *AlarmInstanceData) String() string { return proto.CompactTextString(m) }
+func (*AlarmInstanceData) ProtoMessage()    {}
+func (*AlarmInstanceData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{1}
+}
+
+func (m *AlarmInstanceData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmInstanceData.Unmarshal(m, b)
+}
+func (m *AlarmInstanceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmInstanceData.Marshal(b, m, deterministic)
+}
+func (m *AlarmInstanceData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmInstanceData.Merge(m, src)
+}
+func (m *AlarmInstanceData) XXX_Size() int {
+	return xxx_messageInfo_AlarmInstanceData.Size(m)
+}
+func (m *AlarmInstanceData) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmInstanceData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmInstanceData proto.InternalMessageInfo
+
+func (m *AlarmInstanceData) GetInstanceId() uint32 {
+	if m != nil {
+		return m.InstanceId
+	}
+	return 0
+}
+
+func (m *AlarmInstanceData) GetCreated() string {
+	if m != nil {
+		return m.Created
+	}
+	return ""
+}
+
+func (m *AlarmInstanceData) GetModified() string {
+	if m != nil {
+		return m.Modified
+	}
+	return ""
+}
+
+func (m *AlarmInstanceData) GetAttributes() []*AlarmAttributeData {
+	if m != nil {
+		return m.Attributes
+	}
+	return nil
+}
+
+type AlarmClassData struct {
+	ClassId              uint32               `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
+	Instances            []*AlarmInstanceData `protobuf:"bytes,2,rep,name=instances,proto3" json:"instances,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *AlarmClassData) Reset()         { *m = AlarmClassData{} }
+func (m *AlarmClassData) String() string { return proto.CompactTextString(m) }
+func (*AlarmClassData) ProtoMessage()    {}
+func (*AlarmClassData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{2}
+}
+
+func (m *AlarmClassData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmClassData.Unmarshal(m, b)
+}
+func (m *AlarmClassData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmClassData.Marshal(b, m, deterministic)
+}
+func (m *AlarmClassData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmClassData.Merge(m, src)
+}
+func (m *AlarmClassData) XXX_Size() int {
+	return xxx_messageInfo_AlarmClassData.Size(m)
+}
+func (m *AlarmClassData) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmClassData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmClassData proto.InternalMessageInfo
+
+func (m *AlarmClassData) GetClassId() uint32 {
+	if m != nil {
+		return m.ClassId
+	}
+	return 0
+}
+
+func (m *AlarmClassData) GetInstances() []*AlarmInstanceData {
+	if m != nil {
+		return m.Instances
+	}
+	return nil
+}
+
+type AlarmManagedEntity struct {
+	ClassId              uint32   `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
+	Name                 string   `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AlarmManagedEntity) Reset()         { *m = AlarmManagedEntity{} }
+func (m *AlarmManagedEntity) String() string { return proto.CompactTextString(m) }
+func (*AlarmManagedEntity) ProtoMessage()    {}
+func (*AlarmManagedEntity) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{3}
+}
+
+func (m *AlarmManagedEntity) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmManagedEntity.Unmarshal(m, b)
+}
+func (m *AlarmManagedEntity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmManagedEntity.Marshal(b, m, deterministic)
+}
+func (m *AlarmManagedEntity) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmManagedEntity.Merge(m, src)
+}
+func (m *AlarmManagedEntity) XXX_Size() int {
+	return xxx_messageInfo_AlarmManagedEntity.Size(m)
+}
+func (m *AlarmManagedEntity) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmManagedEntity.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmManagedEntity proto.InternalMessageInfo
+
+func (m *AlarmManagedEntity) GetClassId() uint32 {
+	if m != nil {
+		return m.ClassId
+	}
+	return 0
+}
+
+func (m *AlarmManagedEntity) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+type AlarmMessageType struct {
+	MessageType          uint32   `protobuf:"varint,1,opt,name=message_type,json=messageType,proto3" json:"message_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AlarmMessageType) Reset()         { *m = AlarmMessageType{} }
+func (m *AlarmMessageType) String() string { return proto.CompactTextString(m) }
+func (*AlarmMessageType) ProtoMessage()    {}
+func (*AlarmMessageType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{4}
+}
+
+func (m *AlarmMessageType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmMessageType.Unmarshal(m, b)
+}
+func (m *AlarmMessageType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmMessageType.Marshal(b, m, deterministic)
+}
+func (m *AlarmMessageType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmMessageType.Merge(m, src)
+}
+func (m *AlarmMessageType) XXX_Size() int {
+	return xxx_messageInfo_AlarmMessageType.Size(m)
+}
+func (m *AlarmMessageType) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmMessageType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmMessageType proto.InternalMessageInfo
+
+func (m *AlarmMessageType) GetMessageType() uint32 {
+	if m != nil {
+		return m.MessageType
+	}
+	return 0
+}
+
+type AlarmDeviceData struct {
+	DeviceId             string                `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Created              string                `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
+	LastAlarmSequence    uint32                `protobuf:"varint,3,opt,name=last_alarm_sequence,json=lastAlarmSequence,proto3" json:"last_alarm_sequence,omitempty"`
+	LastSyncTime         string                `protobuf:"bytes,4,opt,name=last_sync_time,json=lastSyncTime,proto3" json:"last_sync_time,omitempty"`
+	Version              uint32                `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"`
+	Classes              []*AlarmClassData     `protobuf:"bytes,6,rep,name=classes,proto3" json:"classes,omitempty"`
+	ManagedEntities      []*AlarmManagedEntity `protobuf:"bytes,7,rep,name=managed_entities,json=managedEntities,proto3" json:"managed_entities,omitempty"`
+	MessageTypes         []*AlarmMessageType   `protobuf:"bytes,8,rep,name=message_types,json=messageTypes,proto3" json:"message_types,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *AlarmDeviceData) Reset()         { *m = AlarmDeviceData{} }
+func (m *AlarmDeviceData) String() string { return proto.CompactTextString(m) }
+func (*AlarmDeviceData) ProtoMessage()    {}
+func (*AlarmDeviceData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{5}
+}
+
+func (m *AlarmDeviceData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmDeviceData.Unmarshal(m, b)
+}
+func (m *AlarmDeviceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmDeviceData.Marshal(b, m, deterministic)
+}
+func (m *AlarmDeviceData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmDeviceData.Merge(m, src)
+}
+func (m *AlarmDeviceData) XXX_Size() int {
+	return xxx_messageInfo_AlarmDeviceData.Size(m)
+}
+func (m *AlarmDeviceData) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmDeviceData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmDeviceData proto.InternalMessageInfo
+
+func (m *AlarmDeviceData) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *AlarmDeviceData) GetCreated() string {
+	if m != nil {
+		return m.Created
+	}
+	return ""
+}
+
+func (m *AlarmDeviceData) GetLastAlarmSequence() uint32 {
+	if m != nil {
+		return m.LastAlarmSequence
+	}
+	return 0
+}
+
+func (m *AlarmDeviceData) GetLastSyncTime() string {
+	if m != nil {
+		return m.LastSyncTime
+	}
+	return ""
+}
+
+func (m *AlarmDeviceData) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *AlarmDeviceData) GetClasses() []*AlarmClassData {
+	if m != nil {
+		return m.Classes
+	}
+	return nil
+}
+
+func (m *AlarmDeviceData) GetManagedEntities() []*AlarmManagedEntity {
+	if m != nil {
+		return m.ManagedEntities
+	}
+	return nil
+}
+
+func (m *AlarmDeviceData) GetMessageTypes() []*AlarmMessageType {
+	if m != nil {
+		return m.MessageTypes
+	}
+	return nil
+}
+
+type AlarmOpenOmciEventType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AlarmOpenOmciEventType) Reset()         { *m = AlarmOpenOmciEventType{} }
+func (m *AlarmOpenOmciEventType) String() string { return proto.CompactTextString(m) }
+func (*AlarmOpenOmciEventType) ProtoMessage()    {}
+func (*AlarmOpenOmciEventType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{6}
+}
+
+func (m *AlarmOpenOmciEventType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmOpenOmciEventType.Unmarshal(m, b)
+}
+func (m *AlarmOpenOmciEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmOpenOmciEventType.Marshal(b, m, deterministic)
+}
+func (m *AlarmOpenOmciEventType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmOpenOmciEventType.Merge(m, src)
+}
+func (m *AlarmOpenOmciEventType) XXX_Size() int {
+	return xxx_messageInfo_AlarmOpenOmciEventType.Size(m)
+}
+func (m *AlarmOpenOmciEventType) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmOpenOmciEventType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmOpenOmciEventType proto.InternalMessageInfo
+
+type AlarmOpenOmciEvent struct {
+	Type                 AlarmOpenOmciEventType_OpenOmciEventType `protobuf:"varint,1,opt,name=type,proto3,enum=omci.AlarmOpenOmciEventType_OpenOmciEventType" json:"type,omitempty"`
+	Data                 string                                   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                 `json:"-"`
+	XXX_unrecognized     []byte                                   `json:"-"`
+	XXX_sizecache        int32                                    `json:"-"`
+}
+
+func (m *AlarmOpenOmciEvent) Reset()         { *m = AlarmOpenOmciEvent{} }
+func (m *AlarmOpenOmciEvent) String() string { return proto.CompactTextString(m) }
+func (*AlarmOpenOmciEvent) ProtoMessage()    {}
+func (*AlarmOpenOmciEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_8d41f1e38aadb08d, []int{7}
+}
+
+func (m *AlarmOpenOmciEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmOpenOmciEvent.Unmarshal(m, b)
+}
+func (m *AlarmOpenOmciEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmOpenOmciEvent.Marshal(b, m, deterministic)
+}
+func (m *AlarmOpenOmciEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmOpenOmciEvent.Merge(m, src)
+}
+func (m *AlarmOpenOmciEvent) XXX_Size() int {
+	return xxx_messageInfo_AlarmOpenOmciEvent.Size(m)
+}
+func (m *AlarmOpenOmciEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmOpenOmciEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmOpenOmciEvent proto.InternalMessageInfo
+
+func (m *AlarmOpenOmciEvent) GetType() AlarmOpenOmciEventType_OpenOmciEventType {
+	if m != nil {
+		return m.Type
+	}
+	return AlarmOpenOmciEventType_state_change
+}
+
+func (m *AlarmOpenOmciEvent) GetData() string {
+	if m != nil {
+		return m.Data
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterEnum("omci.AlarmOpenOmciEventType_OpenOmciEventType", AlarmOpenOmciEventType_OpenOmciEventType_name, AlarmOpenOmciEventType_OpenOmciEventType_value)
+	proto.RegisterType((*AlarmAttributeData)(nil), "omci.AlarmAttributeData")
+	proto.RegisterType((*AlarmInstanceData)(nil), "omci.AlarmInstanceData")
+	proto.RegisterType((*AlarmClassData)(nil), "omci.AlarmClassData")
+	proto.RegisterType((*AlarmManagedEntity)(nil), "omci.AlarmManagedEntity")
+	proto.RegisterType((*AlarmMessageType)(nil), "omci.AlarmMessageType")
+	proto.RegisterType((*AlarmDeviceData)(nil), "omci.AlarmDeviceData")
+	proto.RegisterType((*AlarmOpenOmciEventType)(nil), "omci.AlarmOpenOmciEventType")
+	proto.RegisterType((*AlarmOpenOmciEvent)(nil), "omci.AlarmOpenOmciEvent")
+}
+
+func init() { proto.RegisterFile("voltha_protos/omci_alarm_db.proto", fileDescriptor_8d41f1e38aadb08d) }
+
+var fileDescriptor_8d41f1e38aadb08d = []byte{
+	// 606 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xc1, 0x6e, 0xd3, 0x40,
+	0x10, 0x25, 0x6d, 0xda, 0xa6, 0x93, 0xa4, 0x4d, 0x97, 0xaa, 0x2c, 0x95, 0x2a, 0x15, 0x0b, 0x50,
+	0x0f, 0xe0, 0x48, 0x45, 0x9c, 0x40, 0xaa, 0x9a, 0x36, 0x12, 0x39, 0xa0, 0x0a, 0xb7, 0x27, 0x2e,
+	0xd6, 0xc6, 0x1e, 0xdc, 0x95, 0xbc, 0xeb, 0xe0, 0xdd, 0x58, 0xca, 0x81, 0x0b, 0x5f, 0xc5, 0x4f,
+	0xf4, 0x27, 0x38, 0xf1, 0x05, 0x3d, 0x23, 0x8f, 0xed, 0xc4, 0x91, 0x25, 0xc4, 0x6d, 0xdf, 0x9b,
+	0x99, 0x37, 0xb3, 0xf3, 0x56, 0x0b, 0x2f, 0xb2, 0x24, 0xb6, 0xf7, 0xc2, 0x9f, 0xa5, 0x89, 0x4d,
+	0xcc, 0x30, 0x51, 0x81, 0xf4, 0x45, 0x2c, 0x52, 0xe5, 0x87, 0x53, 0x97, 0x48, 0xd6, 0xce, 0xc9,
+	0x63, 0xbe, 0x9e, 0xa8, 0xd0, 0x8a, 0x22, 0xee, 0x8c, 0x81, 0x5d, 0xe6, 0x15, 0x97, 0xd6, 0xa6,
+	0x72, 0x3a, 0xb7, 0x78, 0x2d, 0xac, 0x60, 0xcf, 0xa1, 0xad, 0x85, 0x42, 0xde, 0x3a, 0x6d, 0x9d,
+	0xed, 0x8e, 0xb6, 0xfe, 0x3c, 0x3e, 0x9c, 0xb4, 0x3c, 0xa2, 0xd8, 0x21, 0x6c, 0x65, 0x22, 0x9e,
+	0x23, 0xdf, 0xc8, 0x63, 0x5e, 0x01, 0x9c, 0x5f, 0x2d, 0x38, 0x20, 0x9d, 0x89, 0x36, 0x56, 0xe8,
+	0xa0, 0x90, 0x79, 0x0d, 0x5d, 0x59, 0x62, 0x5f, 0x86, 0xa4, 0xd6, 0xaf, 0xd4, 0xa0, 0x8a, 0x4c,
+	0x42, 0xc6, 0x61, 0x27, 0x48, 0x51, 0x58, 0x0c, 0x4b, 0xd5, 0x0a, 0xb2, 0x63, 0xe8, 0xa8, 0x24,
+	0x94, 0xdf, 0x24, 0x86, 0x7c, 0x93, 0x42, 0x4b, 0xcc, 0xc6, 0x00, 0xa2, 0x9a, 0xda, 0xf0, 0xf6,
+	0xe9, 0xe6, 0x59, 0xf7, 0x9c, 0xbb, 0xf9, 0x7d, 0xdd, 0xe6, 0x95, 0x46, 0xdd, 0xdf, 0x8f, 0x0f,
+	0x27, 0xdb, 0xc5, 0xbd, 0xbc, 0x5a, 0xa1, 0xf3, 0x03, 0xf6, 0x28, 0xfd, 0x2a, 0x16, 0xc6, 0xd0,
+	0xd8, 0xa7, 0xd0, 0x09, 0x72, 0xd0, 0x98, 0x79, 0x87, 0xe8, 0x49, 0xc8, 0x26, 0xb0, 0x5b, 0x8d,
+	0x6f, 0xf8, 0x06, 0x75, 0x7e, 0x56, 0xeb, 0x5c, 0x5f, 0xc2, 0x88, 0xe5, 0x8d, 0xfb, 0x6b, 0x9b,
+	0xf0, 0x56, 0xd5, 0xce, 0x97, 0xd2, 0x80, 0xcf, 0x42, 0x8b, 0x08, 0xc3, 0xb1, 0xb6, 0xd2, 0x2e,
+	0xfe, 0x63, 0x84, 0xca, 0xa2, 0x8d, 0x86, 0x45, 0xce, 0x47, 0x18, 0x14, 0x92, 0x68, 0x8c, 0x88,
+	0xf0, 0x6e, 0x31, 0x43, 0x76, 0x06, 0x3d, 0x55, 0x40, 0xdf, 0x2e, 0x66, 0xb8, 0x2e, 0xda, 0x55,
+	0xab, 0x4c, 0xe7, 0xe7, 0x26, 0xec, 0x53, 0xf9, 0x35, 0x66, 0xb2, 0x34, 0xd2, 0x81, 0xdd, 0x90,
+	0x50, 0x35, 0xcf, 0xb2, 0x63, 0xa7, 0xe0, 0xff, 0x69, 0xa2, 0x0b, 0x4f, 0x63, 0x61, 0x6c, 0xf9,
+	0x34, 0x0d, 0x7e, 0x9f, 0xa3, 0x0e, 0x90, 0xfc, 0xec, 0x7b, 0x07, 0x79, 0x88, 0xfa, 0xdd, 0x96,
+	0x01, 0xf6, 0x12, 0xf6, 0x28, 0xdf, 0x2c, 0x74, 0xe0, 0x5b, 0xa9, 0x90, 0xb7, 0x49, 0xb0, 0x97,
+	0xb3, 0xb7, 0x0b, 0x1d, 0xdc, 0x49, 0x85, 0x79, 0xbf, 0x0c, 0x53, 0x23, 0x13, 0xcd, 0xb7, 0x48,
+	0xa9, 0x82, 0xec, 0x02, 0x8a, 0x2d, 0xa1, 0xe1, 0xdb, 0xe4, 0xcd, 0x61, 0xcd, 0x9b, 0xa5, 0xcd,
+	0xa3, 0xfd, 0xdc, 0x18, 0x58, 0x2d, 0xda, 0xab, 0xaa, 0xd8, 0x15, 0x0c, 0x54, 0x61, 0x87, 0x8f,
+	0xb9, 0x1f, 0x12, 0x0d, 0xdf, 0x69, 0xbc, 0xaf, 0x35, 0xc7, 0xbc, 0x7d, 0x55, 0x83, 0x12, 0x0d,
+	0xfb, 0x00, 0xfd, 0xfa, 0xc6, 0x0d, 0xef, 0x90, 0xc2, 0x51, 0x5d, 0x61, 0xb5, 0x76, 0xaf, 0x57,
+	0xf3, 0xc0, 0x38, 0x17, 0x70, 0x44, 0x19, 0x37, 0x33, 0xd4, 0x37, 0x2a, 0x90, 0xe3, 0x0c, 0xb5,
+	0x25, 0x7b, 0x5e, 0xc1, 0x41, 0x83, 0x64, 0x03, 0xe8, 0x19, 0x2b, 0x2c, 0xfa, 0xc1, 0xbd, 0xd0,
+	0x11, 0x0e, 0x9e, 0x38, 0x71, 0xf9, 0xac, 0xd6, 0x72, 0xd9, 0x08, 0xda, 0x4b, 0xf7, 0xf7, 0xce,
+	0xdd, 0xda, 0x28, 0x0d, 0x4d, 0xb7, 0xc1, 0x78, 0x54, 0xcb, 0x18, 0xb4, 0x43, 0x61, 0x45, 0x69,
+	0x32, 0x9d, 0x47, 0x9f, 0x80, 0x27, 0x69, 0xe4, 0x26, 0x33, 0xd4, 0x41, 0x92, 0x86, 0x6e, 0xf1,
+	0xdd, 0x90, 0xfc, 0xd7, 0x37, 0x91, 0xb4, 0xf7, 0xf3, 0xa9, 0x1b, 0x24, 0x6a, 0x58, 0x25, 0x0c,
+	0x8b, 0x84, 0xb7, 0xe5, 0x7f, 0x94, 0xbd, 0x1f, 0x46, 0x09, 0x7d, 0x5f, 0xd3, 0x6d, 0xa2, 0xde,
+	0xfd, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x71, 0x53, 0xfa, 0xbd, 0xdb, 0x04, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_mib_db.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_mib_db.pb.go
new file mode 100644
index 0000000..faa0975
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/omci/omci_mib_db.pb.go
@@ -0,0 +1,516 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/omci_mib_db.proto
+
+package omci
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_ "github.com/opencord/voltha-protos/v5/go/common"
+	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 OpenOmciEventType_Types int32
+
+const (
+	OpenOmciEventType_state_change OpenOmciEventType_Types = 0
+)
+
+var OpenOmciEventType_Types_name = map[int32]string{
+	0: "state_change",
+}
+
+var OpenOmciEventType_Types_value = map[string]int32{
+	"state_change": 0,
+}
+
+func (x OpenOmciEventType_Types) String() string {
+	return proto.EnumName(OpenOmciEventType_Types_name, int32(x))
+}
+
+func (OpenOmciEventType_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{6, 0}
+}
+
+type MibAttributeData struct {
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MibAttributeData) Reset()         { *m = MibAttributeData{} }
+func (m *MibAttributeData) String() string { return proto.CompactTextString(m) }
+func (*MibAttributeData) ProtoMessage()    {}
+func (*MibAttributeData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{0}
+}
+
+func (m *MibAttributeData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MibAttributeData.Unmarshal(m, b)
+}
+func (m *MibAttributeData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MibAttributeData.Marshal(b, m, deterministic)
+}
+func (m *MibAttributeData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MibAttributeData.Merge(m, src)
+}
+func (m *MibAttributeData) XXX_Size() int {
+	return xxx_messageInfo_MibAttributeData.Size(m)
+}
+func (m *MibAttributeData) XXX_DiscardUnknown() {
+	xxx_messageInfo_MibAttributeData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MibAttributeData proto.InternalMessageInfo
+
+func (m *MibAttributeData) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *MibAttributeData) GetValue() string {
+	if m != nil {
+		return m.Value
+	}
+	return ""
+}
+
+type MibInstanceData struct {
+	InstanceId           uint32              `protobuf:"varint,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
+	Created              string              `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
+	Modified             string              `protobuf:"bytes,3,opt,name=modified,proto3" json:"modified,omitempty"`
+	Attributes           []*MibAttributeData `protobuf:"bytes,4,rep,name=attributes,proto3" json:"attributes,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *MibInstanceData) Reset()         { *m = MibInstanceData{} }
+func (m *MibInstanceData) String() string { return proto.CompactTextString(m) }
+func (*MibInstanceData) ProtoMessage()    {}
+func (*MibInstanceData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{1}
+}
+
+func (m *MibInstanceData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MibInstanceData.Unmarshal(m, b)
+}
+func (m *MibInstanceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MibInstanceData.Marshal(b, m, deterministic)
+}
+func (m *MibInstanceData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MibInstanceData.Merge(m, src)
+}
+func (m *MibInstanceData) XXX_Size() int {
+	return xxx_messageInfo_MibInstanceData.Size(m)
+}
+func (m *MibInstanceData) XXX_DiscardUnknown() {
+	xxx_messageInfo_MibInstanceData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MibInstanceData proto.InternalMessageInfo
+
+func (m *MibInstanceData) GetInstanceId() uint32 {
+	if m != nil {
+		return m.InstanceId
+	}
+	return 0
+}
+
+func (m *MibInstanceData) GetCreated() string {
+	if m != nil {
+		return m.Created
+	}
+	return ""
+}
+
+func (m *MibInstanceData) GetModified() string {
+	if m != nil {
+		return m.Modified
+	}
+	return ""
+}
+
+func (m *MibInstanceData) GetAttributes() []*MibAttributeData {
+	if m != nil {
+		return m.Attributes
+	}
+	return nil
+}
+
+type MibClassData struct {
+	ClassId              uint32             `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
+	Instances            []*MibInstanceData `protobuf:"bytes,2,rep,name=instances,proto3" json:"instances,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *MibClassData) Reset()         { *m = MibClassData{} }
+func (m *MibClassData) String() string { return proto.CompactTextString(m) }
+func (*MibClassData) ProtoMessage()    {}
+func (*MibClassData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{2}
+}
+
+func (m *MibClassData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MibClassData.Unmarshal(m, b)
+}
+func (m *MibClassData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MibClassData.Marshal(b, m, deterministic)
+}
+func (m *MibClassData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MibClassData.Merge(m, src)
+}
+func (m *MibClassData) XXX_Size() int {
+	return xxx_messageInfo_MibClassData.Size(m)
+}
+func (m *MibClassData) XXX_DiscardUnknown() {
+	xxx_messageInfo_MibClassData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MibClassData proto.InternalMessageInfo
+
+func (m *MibClassData) GetClassId() uint32 {
+	if m != nil {
+		return m.ClassId
+	}
+	return 0
+}
+
+func (m *MibClassData) GetInstances() []*MibInstanceData {
+	if m != nil {
+		return m.Instances
+	}
+	return nil
+}
+
+type ManagedEntity struct {
+	ClassId              uint32   `protobuf:"varint,1,opt,name=class_id,json=classId,proto3" json:"class_id,omitempty"`
+	Name                 string   `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ManagedEntity) Reset()         { *m = ManagedEntity{} }
+func (m *ManagedEntity) String() string { return proto.CompactTextString(m) }
+func (*ManagedEntity) ProtoMessage()    {}
+func (*ManagedEntity) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{3}
+}
+
+func (m *ManagedEntity) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ManagedEntity.Unmarshal(m, b)
+}
+func (m *ManagedEntity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ManagedEntity.Marshal(b, m, deterministic)
+}
+func (m *ManagedEntity) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ManagedEntity.Merge(m, src)
+}
+func (m *ManagedEntity) XXX_Size() int {
+	return xxx_messageInfo_ManagedEntity.Size(m)
+}
+func (m *ManagedEntity) XXX_DiscardUnknown() {
+	xxx_messageInfo_ManagedEntity.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ManagedEntity proto.InternalMessageInfo
+
+func (m *ManagedEntity) GetClassId() uint32 {
+	if m != nil {
+		return m.ClassId
+	}
+	return 0
+}
+
+func (m *ManagedEntity) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+type MessageType struct {
+	MessageType          uint32   `protobuf:"varint,1,opt,name=message_type,json=messageType,proto3" json:"message_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *MessageType) Reset()         { *m = MessageType{} }
+func (m *MessageType) String() string { return proto.CompactTextString(m) }
+func (*MessageType) ProtoMessage()    {}
+func (*MessageType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{4}
+}
+
+func (m *MessageType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MessageType.Unmarshal(m, b)
+}
+func (m *MessageType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MessageType.Marshal(b, m, deterministic)
+}
+func (m *MessageType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MessageType.Merge(m, src)
+}
+func (m *MessageType) XXX_Size() int {
+	return xxx_messageInfo_MessageType.Size(m)
+}
+func (m *MessageType) XXX_DiscardUnknown() {
+	xxx_messageInfo_MessageType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MessageType proto.InternalMessageInfo
+
+func (m *MessageType) GetMessageType() uint32 {
+	if m != nil {
+		return m.MessageType
+	}
+	return 0
+}
+
+type MibDeviceData struct {
+	DeviceId             string           `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Created              string           `protobuf:"bytes,2,opt,name=created,proto3" json:"created,omitempty"`
+	LastSyncTime         string           `protobuf:"bytes,3,opt,name=last_sync_time,json=lastSyncTime,proto3" json:"last_sync_time,omitempty"`
+	MibDataSync          uint32           `protobuf:"varint,4,opt,name=mib_data_sync,json=mibDataSync,proto3" json:"mib_data_sync,omitempty"`
+	Version              uint32           `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"`
+	Classes              []*MibClassData  `protobuf:"bytes,6,rep,name=classes,proto3" json:"classes,omitempty"`
+	ManagedEntities      []*ManagedEntity `protobuf:"bytes,7,rep,name=managed_entities,json=managedEntities,proto3" json:"managed_entities,omitempty"`
+	MessageTypes         []*MessageType   `protobuf:"bytes,8,rep,name=message_types,json=messageTypes,proto3" json:"message_types,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *MibDeviceData) Reset()         { *m = MibDeviceData{} }
+func (m *MibDeviceData) String() string { return proto.CompactTextString(m) }
+func (*MibDeviceData) ProtoMessage()    {}
+func (*MibDeviceData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{5}
+}
+
+func (m *MibDeviceData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MibDeviceData.Unmarshal(m, b)
+}
+func (m *MibDeviceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MibDeviceData.Marshal(b, m, deterministic)
+}
+func (m *MibDeviceData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MibDeviceData.Merge(m, src)
+}
+func (m *MibDeviceData) XXX_Size() int {
+	return xxx_messageInfo_MibDeviceData.Size(m)
+}
+func (m *MibDeviceData) XXX_DiscardUnknown() {
+	xxx_messageInfo_MibDeviceData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MibDeviceData proto.InternalMessageInfo
+
+func (m *MibDeviceData) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *MibDeviceData) GetCreated() string {
+	if m != nil {
+		return m.Created
+	}
+	return ""
+}
+
+func (m *MibDeviceData) GetLastSyncTime() string {
+	if m != nil {
+		return m.LastSyncTime
+	}
+	return ""
+}
+
+func (m *MibDeviceData) GetMibDataSync() uint32 {
+	if m != nil {
+		return m.MibDataSync
+	}
+	return 0
+}
+
+func (m *MibDeviceData) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *MibDeviceData) GetClasses() []*MibClassData {
+	if m != nil {
+		return m.Classes
+	}
+	return nil
+}
+
+func (m *MibDeviceData) GetManagedEntities() []*ManagedEntity {
+	if m != nil {
+		return m.ManagedEntities
+	}
+	return nil
+}
+
+func (m *MibDeviceData) GetMessageTypes() []*MessageType {
+	if m != nil {
+		return m.MessageTypes
+	}
+	return nil
+}
+
+type OpenOmciEventType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OpenOmciEventType) Reset()         { *m = OpenOmciEventType{} }
+func (m *OpenOmciEventType) String() string { return proto.CompactTextString(m) }
+func (*OpenOmciEventType) ProtoMessage()    {}
+func (*OpenOmciEventType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{6}
+}
+
+func (m *OpenOmciEventType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OpenOmciEventType.Unmarshal(m, b)
+}
+func (m *OpenOmciEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OpenOmciEventType.Marshal(b, m, deterministic)
+}
+func (m *OpenOmciEventType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OpenOmciEventType.Merge(m, src)
+}
+func (m *OpenOmciEventType) XXX_Size() int {
+	return xxx_messageInfo_OpenOmciEventType.Size(m)
+}
+func (m *OpenOmciEventType) XXX_DiscardUnknown() {
+	xxx_messageInfo_OpenOmciEventType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OpenOmciEventType proto.InternalMessageInfo
+
+type OpenOmciEvent struct {
+	Type                 OpenOmciEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=omci.OpenOmciEventType_Types" json:"type,omitempty"`
+	Data                 string                  `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *OpenOmciEvent) Reset()         { *m = OpenOmciEvent{} }
+func (m *OpenOmciEvent) String() string { return proto.CompactTextString(m) }
+func (*OpenOmciEvent) ProtoMessage()    {}
+func (*OpenOmciEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_4fa402a2df36dcc1, []int{7}
+}
+
+func (m *OpenOmciEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OpenOmciEvent.Unmarshal(m, b)
+}
+func (m *OpenOmciEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OpenOmciEvent.Marshal(b, m, deterministic)
+}
+func (m *OpenOmciEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OpenOmciEvent.Merge(m, src)
+}
+func (m *OpenOmciEvent) XXX_Size() int {
+	return xxx_messageInfo_OpenOmciEvent.Size(m)
+}
+func (m *OpenOmciEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_OpenOmciEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OpenOmciEvent proto.InternalMessageInfo
+
+func (m *OpenOmciEvent) GetType() OpenOmciEventType_Types {
+	if m != nil {
+		return m.Type
+	}
+	return OpenOmciEventType_state_change
+}
+
+func (m *OpenOmciEvent) GetData() string {
+	if m != nil {
+		return m.Data
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterEnum("omci.OpenOmciEventType_Types", OpenOmciEventType_Types_name, OpenOmciEventType_Types_value)
+	proto.RegisterType((*MibAttributeData)(nil), "omci.MibAttributeData")
+	proto.RegisterType((*MibInstanceData)(nil), "omci.MibInstanceData")
+	proto.RegisterType((*MibClassData)(nil), "omci.MibClassData")
+	proto.RegisterType((*ManagedEntity)(nil), "omci.ManagedEntity")
+	proto.RegisterType((*MessageType)(nil), "omci.MessageType")
+	proto.RegisterType((*MibDeviceData)(nil), "omci.MibDeviceData")
+	proto.RegisterType((*OpenOmciEventType)(nil), "omci.OpenOmciEventType")
+	proto.RegisterType((*OpenOmciEvent)(nil), "omci.OpenOmciEvent")
+}
+
+func init() { proto.RegisterFile("voltha_protos/omci_mib_db.proto", fileDescriptor_4fa402a2df36dcc1) }
+
+var fileDescriptor_4fa402a2df36dcc1 = []byte{
+	// 598 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdf, 0x6a, 0xd4, 0x4e,
+	0x14, 0xfe, 0xed, 0x76, 0xb7, 0xdd, 0x9e, 0xdd, 0xb4, 0xdb, 0xf9, 0xa9, 0x4c, 0x0b, 0xc5, 0x12,
+	0x44, 0x7a, 0xa1, 0x59, 0xac, 0xa8, 0x57, 0x0a, 0x6e, 0x5b, 0xb4, 0x60, 0x28, 0xc4, 0xe2, 0x85,
+	0x37, 0x61, 0x92, 0x1c, 0xd3, 0x81, 0x9d, 0xc9, 0x92, 0x99, 0x06, 0xf2, 0x5e, 0xbe, 0x46, 0x5f,
+	0xc2, 0xab, 0x3e, 0x41, 0xaf, 0x65, 0x26, 0x7f, 0x36, 0x6b, 0x41, 0xbc, 0xcb, 0x77, 0xfe, 0x7c,
+	0xe7, 0x9c, 0xef, 0x0b, 0x03, 0x4f, 0x8b, 0x6c, 0xa1, 0xaf, 0x59, 0xb8, 0xcc, 0x33, 0x9d, 0xa9,
+	0x59, 0x26, 0x62, 0x1e, 0x0a, 0x1e, 0x85, 0x49, 0xe4, 0xd9, 0x10, 0x19, 0x98, 0xd0, 0x01, 0x5d,
+	0x2f, 0x13, 0xa8, 0x59, 0x95, 0x77, 0x4f, 0x61, 0xea, 0xf3, 0xe8, 0xa3, 0xd6, 0x39, 0x8f, 0x6e,
+	0x34, 0x9e, 0x31, 0xcd, 0xc8, 0x3e, 0x0c, 0x24, 0x13, 0x48, 0x7b, 0x47, 0xbd, 0xe3, 0xed, 0xf9,
+	0xf0, 0xee, 0xfe, 0xf6, 0xb0, 0x17, 0xd8, 0x10, 0x79, 0x04, 0xc3, 0x82, 0x2d, 0x6e, 0x90, 0xf6,
+	0x4d, 0x2e, 0xa8, 0x80, 0xfb, 0xb3, 0x07, 0xbb, 0x3e, 0x8f, 0x2e, 0xa4, 0xd2, 0x4c, 0xc6, 0x15,
+	0xc9, 0x73, 0x18, 0xf3, 0x1a, 0x87, 0x3c, 0xb1, 0x5c, 0x4e, 0xc3, 0x05, 0x4d, 0xe6, 0x22, 0x21,
+	0x14, 0xb6, 0xe2, 0x1c, 0x99, 0xc6, 0xa4, 0xe6, 0x6c, 0x20, 0x39, 0x80, 0x91, 0xc8, 0x12, 0xfe,
+	0x83, 0x63, 0x42, 0x37, 0x6c, 0xaa, 0xc5, 0xe4, 0x14, 0x80, 0x35, 0x3b, 0x2b, 0x3a, 0x38, 0xda,
+	0x38, 0x1e, 0x9f, 0x3c, 0xf1, 0xcc, 0xad, 0xde, 0x9f, 0xe7, 0xcc, 0xc7, 0xbf, 0xee, 0x6f, 0x0f,
+	0x37, 0xab, 0x9b, 0x82, 0x4e, 0x9b, 0x5b, 0xc2, 0xc4, 0xe7, 0xd1, 0xe9, 0x82, 0x29, 0x65, 0x57,
+	0x3e, 0x82, 0x51, 0x6c, 0xc0, 0x83, 0x7d, 0xb7, 0x6c, 0xf8, 0x22, 0x21, 0x9f, 0x60, 0xbb, 0x59,
+	0x5d, 0xd1, 0xbe, 0x9d, 0xfa, 0xb8, 0x9d, 0xda, 0x3d, 0x7f, 0x4e, 0xcc, 0x50, 0x67, 0x4d, 0x83,
+	0x60, 0xd5, 0xeb, 0x7e, 0x01, 0xc7, 0x67, 0x92, 0xa5, 0x98, 0x9c, 0x4b, 0xcd, 0x75, 0xf9, 0x0f,
+	0xb3, 0x1b, 0x57, 0xfa, 0x0f, 0x5c, 0x71, 0xdf, 0xc1, 0xd8, 0x47, 0xa5, 0x58, 0x8a, 0x57, 0xe5,
+	0x12, 0xc9, 0x31, 0x4c, 0x44, 0x05, 0x43, 0x5d, 0x2e, 0x71, 0x9d, 0x6f, 0x2c, 0x56, 0x95, 0xee,
+	0x5d, 0x1f, 0x1c, 0x9f, 0x47, 0x67, 0x58, 0xf0, 0xda, 0x36, 0x17, 0xb6, 0x13, 0x8b, 0x9a, 0x45,
+	0xda, 0x51, 0xa3, 0x2a, 0xfe, 0x57, 0xcb, 0x9e, 0xc1, 0xce, 0x82, 0x29, 0x1d, 0xaa, 0x52, 0xc6,
+	0xa1, 0xe6, 0x02, 0x6b, 0xe3, 0x26, 0x26, 0xfa, 0xb5, 0x94, 0xf1, 0x15, 0x17, 0x48, 0x5c, 0x70,
+	0xec, 0x3f, 0xca, 0x34, 0xb3, 0x95, 0x74, 0x60, 0x16, 0x0c, 0xc6, 0x82, 0x47, 0x66, 0x07, 0x53,
+	0x67, 0x66, 0x14, 0x98, 0x2b, 0x9e, 0x49, 0x3a, 0xb4, 0xd9, 0x06, 0x92, 0xf7, 0x50, 0x49, 0x82,
+	0x8a, 0x6e, 0x5a, 0x07, 0x48, 0xeb, 0x40, 0x6b, 0xe5, 0x7c, 0xd7, 0xc8, 0x0f, 0x2b, 0x4d, 0x83,
+	0xa6, 0x87, 0x7c, 0x80, 0xa9, 0xa8, 0x94, 0x0f, 0xd1, 0x48, 0xcf, 0x51, 0xd1, 0x2d, 0xcb, 0xf3,
+	0x7f, 0xcd, 0xd3, 0xf5, 0x25, 0xd8, 0x15, 0x1d, 0xc8, 0x51, 0x91, 0xb7, 0xe0, 0x74, 0xc5, 0x55,
+	0x74, 0x64, 0x9b, 0xf7, 0xea, 0xe6, 0x95, 0xb8, 0xc1, 0xa4, 0xa3, 0xb4, 0x72, 0x3d, 0xd8, 0xbb,
+	0x5c, 0xa2, 0xbc, 0x14, 0x31, 0x3f, 0x2f, 0x50, 0x6a, 0xab, 0xff, 0x3e, 0x0c, 0x6d, 0x96, 0x4c,
+	0x61, 0xa2, 0x34, 0xd3, 0x18, 0xc6, 0xd7, 0x4c, 0xa6, 0x38, 0xfd, 0xcf, 0xfd, 0x06, 0xce, 0x5a,
+	0x3d, 0x79, 0x05, 0x83, 0xd6, 0xcd, 0x9d, 0x93, 0xc3, 0x6a, 0xde, 0x03, 0x4a, 0xcf, 0xf2, 0x05,
+	0xb6, 0x94, 0x10, 0x18, 0x18, 0x91, 0x6b, 0x97, 0xec, 0xf7, 0xfc, 0x33, 0xd0, 0x2c, 0x4f, 0xbd,
+	0x6c, 0x89, 0x32, 0xce, 0xf2, 0xc4, 0xab, 0x5e, 0x06, 0xcb, 0xf6, 0xfd, 0x45, 0xca, 0xf5, 0xf5,
+	0x4d, 0xe4, 0xc5, 0x99, 0x98, 0x35, 0x05, 0xb3, 0xaa, 0xe0, 0x65, 0xfd, 0x74, 0x14, 0x6f, 0x66,
+	0x69, 0x66, 0xdf, 0x99, 0x68, 0xd3, 0x86, 0x5e, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x2e, 0x1c,
+	0x66, 0xfc, 0x84, 0x04, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/openflow_13/openflow_13.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/openflow_13/openflow_13.pb.go
new file mode 100644
index 0000000..fdf9a77
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/openflow_13/openflow_13.pb.go
@@ -0,0 +1,9863 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/openflow_13.proto
+
+package openflow_13
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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
+
+// Port numbering. Ports are numbered starting from 1.
+type OfpPortNo int32
+
+const (
+	OfpPortNo_OFPP_INVALID OfpPortNo = 0
+	// Maximum number of physical and logical switch ports.
+	OfpPortNo_OFPP_MAX OfpPortNo = 2147483392
+	// Reserved OpenFlow Port (fake output "ports").
+	OfpPortNo_OFPP_IN_PORT    OfpPortNo = 2147483640
+	OfpPortNo_OFPP_TABLE      OfpPortNo = 2147483641
+	OfpPortNo_OFPP_NORMAL     OfpPortNo = 2147483642
+	OfpPortNo_OFPP_FLOOD      OfpPortNo = 2147483643
+	OfpPortNo_OFPP_ALL        OfpPortNo = 2147483644
+	OfpPortNo_OFPP_CONTROLLER OfpPortNo = 2147483645
+	OfpPortNo_OFPP_LOCAL      OfpPortNo = 2147483646
+	OfpPortNo_OFPP_ANY        OfpPortNo = 2147483647
+)
+
+var OfpPortNo_name = map[int32]string{
+	0:          "OFPP_INVALID",
+	2147483392: "OFPP_MAX",
+	2147483640: "OFPP_IN_PORT",
+	2147483641: "OFPP_TABLE",
+	2147483642: "OFPP_NORMAL",
+	2147483643: "OFPP_FLOOD",
+	2147483644: "OFPP_ALL",
+	2147483645: "OFPP_CONTROLLER",
+	2147483646: "OFPP_LOCAL",
+	2147483647: "OFPP_ANY",
+}
+
+var OfpPortNo_value = map[string]int32{
+	"OFPP_INVALID":    0,
+	"OFPP_MAX":        2147483392,
+	"OFPP_IN_PORT":    2147483640,
+	"OFPP_TABLE":      2147483641,
+	"OFPP_NORMAL":     2147483642,
+	"OFPP_FLOOD":      2147483643,
+	"OFPP_ALL":        2147483644,
+	"OFPP_CONTROLLER": 2147483645,
+	"OFPP_LOCAL":      2147483646,
+	"OFPP_ANY":        2147483647,
+}
+
+func (x OfpPortNo) String() string {
+	return proto.EnumName(OfpPortNo_name, int32(x))
+}
+
+func (OfpPortNo) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{0}
+}
+
+type OfpType int32
+
+const (
+	// Immutable messages.
+	OfpType_OFPT_HELLO        OfpType = 0
+	OfpType_OFPT_ERROR        OfpType = 1
+	OfpType_OFPT_ECHO_REQUEST OfpType = 2
+	OfpType_OFPT_ECHO_REPLY   OfpType = 3
+	OfpType_OFPT_EXPERIMENTER OfpType = 4
+	// Switch configuration messages.
+	OfpType_OFPT_FEATURES_REQUEST   OfpType = 5
+	OfpType_OFPT_FEATURES_REPLY     OfpType = 6
+	OfpType_OFPT_GET_CONFIG_REQUEST OfpType = 7
+	OfpType_OFPT_GET_CONFIG_REPLY   OfpType = 8
+	OfpType_OFPT_SET_CONFIG         OfpType = 9
+	// Asynchronous messages.
+	OfpType_OFPT_PACKET_IN    OfpType = 10
+	OfpType_OFPT_FLOW_REMOVED OfpType = 11
+	OfpType_OFPT_PORT_STATUS  OfpType = 12
+	// Controller command messages.
+	OfpType_OFPT_PACKET_OUT OfpType = 13
+	OfpType_OFPT_FLOW_MOD   OfpType = 14
+	OfpType_OFPT_GROUP_MOD  OfpType = 15
+	OfpType_OFPT_PORT_MOD   OfpType = 16
+	OfpType_OFPT_TABLE_MOD  OfpType = 17
+	// Multipart messages.
+	OfpType_OFPT_MULTIPART_REQUEST OfpType = 18
+	OfpType_OFPT_MULTIPART_REPLY   OfpType = 19
+	// Barrier messages.
+	OfpType_OFPT_BARRIER_REQUEST OfpType = 20
+	OfpType_OFPT_BARRIER_REPLY   OfpType = 21
+	// Queue Configuration messages.
+	OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST OfpType = 22
+	OfpType_OFPT_QUEUE_GET_CONFIG_REPLY   OfpType = 23
+	// Controller role change request messages.
+	OfpType_OFPT_ROLE_REQUEST OfpType = 24
+	OfpType_OFPT_ROLE_REPLY   OfpType = 25
+	// Asynchronous message configuration.
+	OfpType_OFPT_GET_ASYNC_REQUEST OfpType = 26
+	OfpType_OFPT_GET_ASYNC_REPLY   OfpType = 27
+	OfpType_OFPT_SET_ASYNC         OfpType = 28
+	// Meters and rate limiters configuration messages.
+	OfpType_OFPT_METER_MOD OfpType = 29
+)
+
+var OfpType_name = map[int32]string{
+	0:  "OFPT_HELLO",
+	1:  "OFPT_ERROR",
+	2:  "OFPT_ECHO_REQUEST",
+	3:  "OFPT_ECHO_REPLY",
+	4:  "OFPT_EXPERIMENTER",
+	5:  "OFPT_FEATURES_REQUEST",
+	6:  "OFPT_FEATURES_REPLY",
+	7:  "OFPT_GET_CONFIG_REQUEST",
+	8:  "OFPT_GET_CONFIG_REPLY",
+	9:  "OFPT_SET_CONFIG",
+	10: "OFPT_PACKET_IN",
+	11: "OFPT_FLOW_REMOVED",
+	12: "OFPT_PORT_STATUS",
+	13: "OFPT_PACKET_OUT",
+	14: "OFPT_FLOW_MOD",
+	15: "OFPT_GROUP_MOD",
+	16: "OFPT_PORT_MOD",
+	17: "OFPT_TABLE_MOD",
+	18: "OFPT_MULTIPART_REQUEST",
+	19: "OFPT_MULTIPART_REPLY",
+	20: "OFPT_BARRIER_REQUEST",
+	21: "OFPT_BARRIER_REPLY",
+	22: "OFPT_QUEUE_GET_CONFIG_REQUEST",
+	23: "OFPT_QUEUE_GET_CONFIG_REPLY",
+	24: "OFPT_ROLE_REQUEST",
+	25: "OFPT_ROLE_REPLY",
+	26: "OFPT_GET_ASYNC_REQUEST",
+	27: "OFPT_GET_ASYNC_REPLY",
+	28: "OFPT_SET_ASYNC",
+	29: "OFPT_METER_MOD",
+}
+
+var OfpType_value = map[string]int32{
+	"OFPT_HELLO":                    0,
+	"OFPT_ERROR":                    1,
+	"OFPT_ECHO_REQUEST":             2,
+	"OFPT_ECHO_REPLY":               3,
+	"OFPT_EXPERIMENTER":             4,
+	"OFPT_FEATURES_REQUEST":         5,
+	"OFPT_FEATURES_REPLY":           6,
+	"OFPT_GET_CONFIG_REQUEST":       7,
+	"OFPT_GET_CONFIG_REPLY":         8,
+	"OFPT_SET_CONFIG":               9,
+	"OFPT_PACKET_IN":                10,
+	"OFPT_FLOW_REMOVED":             11,
+	"OFPT_PORT_STATUS":              12,
+	"OFPT_PACKET_OUT":               13,
+	"OFPT_FLOW_MOD":                 14,
+	"OFPT_GROUP_MOD":                15,
+	"OFPT_PORT_MOD":                 16,
+	"OFPT_TABLE_MOD":                17,
+	"OFPT_MULTIPART_REQUEST":        18,
+	"OFPT_MULTIPART_REPLY":          19,
+	"OFPT_BARRIER_REQUEST":          20,
+	"OFPT_BARRIER_REPLY":            21,
+	"OFPT_QUEUE_GET_CONFIG_REQUEST": 22,
+	"OFPT_QUEUE_GET_CONFIG_REPLY":   23,
+	"OFPT_ROLE_REQUEST":             24,
+	"OFPT_ROLE_REPLY":               25,
+	"OFPT_GET_ASYNC_REQUEST":        26,
+	"OFPT_GET_ASYNC_REPLY":          27,
+	"OFPT_SET_ASYNC":                28,
+	"OFPT_METER_MOD":                29,
+}
+
+func (x OfpType) String() string {
+	return proto.EnumName(OfpType_name, int32(x))
+}
+
+func (OfpType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{1}
+}
+
+// Hello elements types.
+type OfpHelloElemType int32
+
+const (
+	OfpHelloElemType_OFPHET_INVALID       OfpHelloElemType = 0
+	OfpHelloElemType_OFPHET_VERSIONBITMAP OfpHelloElemType = 1
+)
+
+var OfpHelloElemType_name = map[int32]string{
+	0: "OFPHET_INVALID",
+	1: "OFPHET_VERSIONBITMAP",
+}
+
+var OfpHelloElemType_value = map[string]int32{
+	"OFPHET_INVALID":       0,
+	"OFPHET_VERSIONBITMAP": 1,
+}
+
+func (x OfpHelloElemType) String() string {
+	return proto.EnumName(OfpHelloElemType_name, int32(x))
+}
+
+func (OfpHelloElemType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{2}
+}
+
+type OfpConfigFlags int32
+
+const (
+	// Handling of IP fragments.
+	OfpConfigFlags_OFPC_FRAG_NORMAL OfpConfigFlags = 0
+	OfpConfigFlags_OFPC_FRAG_DROP   OfpConfigFlags = 1
+	OfpConfigFlags_OFPC_FRAG_REASM  OfpConfigFlags = 2
+	OfpConfigFlags_OFPC_FRAG_MASK   OfpConfigFlags = 3
+)
+
+var OfpConfigFlags_name = map[int32]string{
+	0: "OFPC_FRAG_NORMAL",
+	1: "OFPC_FRAG_DROP",
+	2: "OFPC_FRAG_REASM",
+	3: "OFPC_FRAG_MASK",
+}
+
+var OfpConfigFlags_value = map[string]int32{
+	"OFPC_FRAG_NORMAL": 0,
+	"OFPC_FRAG_DROP":   1,
+	"OFPC_FRAG_REASM":  2,
+	"OFPC_FRAG_MASK":   3,
+}
+
+func (x OfpConfigFlags) String() string {
+	return proto.EnumName(OfpConfigFlags_name, int32(x))
+}
+
+func (OfpConfigFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{3}
+}
+
+// Flags to configure the table. Reserved for future use.
+type OfpTableConfig int32
+
+const (
+	OfpTableConfig_OFPTC_INVALID         OfpTableConfig = 0
+	OfpTableConfig_OFPTC_DEPRECATED_MASK OfpTableConfig = 3
+)
+
+var OfpTableConfig_name = map[int32]string{
+	0: "OFPTC_INVALID",
+	3: "OFPTC_DEPRECATED_MASK",
+}
+
+var OfpTableConfig_value = map[string]int32{
+	"OFPTC_INVALID":         0,
+	"OFPTC_DEPRECATED_MASK": 3,
+}
+
+func (x OfpTableConfig) String() string {
+	return proto.EnumName(OfpTableConfig_name, int32(x))
+}
+
+func (OfpTableConfig) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{4}
+}
+
+// Table numbering. Tables can use any number up to OFPT_MAX.
+type OfpTable int32
+
+const (
+	OfpTable_OFPTT_INVALID OfpTable = 0
+	// Last usable table number.
+	OfpTable_OFPTT_MAX OfpTable = 254
+	// Fake tables.
+	OfpTable_OFPTT_ALL OfpTable = 255
+)
+
+var OfpTable_name = map[int32]string{
+	0:   "OFPTT_INVALID",
+	254: "OFPTT_MAX",
+	255: "OFPTT_ALL",
+}
+
+var OfpTable_value = map[string]int32{
+	"OFPTT_INVALID": 0,
+	"OFPTT_MAX":     254,
+	"OFPTT_ALL":     255,
+}
+
+func (x OfpTable) String() string {
+	return proto.EnumName(OfpTable_name, int32(x))
+}
+
+func (OfpTable) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{5}
+}
+
+// Capabilities supported by the datapath.
+type OfpCapabilities int32
+
+const (
+	OfpCapabilities_OFPC_INVALID      OfpCapabilities = 0
+	OfpCapabilities_OFPC_FLOW_STATS   OfpCapabilities = 1
+	OfpCapabilities_OFPC_TABLE_STATS  OfpCapabilities = 2
+	OfpCapabilities_OFPC_PORT_STATS   OfpCapabilities = 4
+	OfpCapabilities_OFPC_GROUP_STATS  OfpCapabilities = 8
+	OfpCapabilities_OFPC_IP_REASM     OfpCapabilities = 32
+	OfpCapabilities_OFPC_QUEUE_STATS  OfpCapabilities = 64
+	OfpCapabilities_OFPC_PORT_BLOCKED OfpCapabilities = 256
+)
+
+var OfpCapabilities_name = map[int32]string{
+	0:   "OFPC_INVALID",
+	1:   "OFPC_FLOW_STATS",
+	2:   "OFPC_TABLE_STATS",
+	4:   "OFPC_PORT_STATS",
+	8:   "OFPC_GROUP_STATS",
+	32:  "OFPC_IP_REASM",
+	64:  "OFPC_QUEUE_STATS",
+	256: "OFPC_PORT_BLOCKED",
+}
+
+var OfpCapabilities_value = map[string]int32{
+	"OFPC_INVALID":      0,
+	"OFPC_FLOW_STATS":   1,
+	"OFPC_TABLE_STATS":  2,
+	"OFPC_PORT_STATS":   4,
+	"OFPC_GROUP_STATS":  8,
+	"OFPC_IP_REASM":     32,
+	"OFPC_QUEUE_STATS":  64,
+	"OFPC_PORT_BLOCKED": 256,
+}
+
+func (x OfpCapabilities) String() string {
+	return proto.EnumName(OfpCapabilities_name, int32(x))
+}
+
+func (OfpCapabilities) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{6}
+}
+
+// Flags to indicate behavior of the physical port.  These flags are
+// used in ofp_port to describe the current configuration.  They are
+// used in the ofp_port_mod message to configure the port's behavior.
+type OfpPortConfig int32
+
+const (
+	OfpPortConfig_OFPPC_INVALID      OfpPortConfig = 0
+	OfpPortConfig_OFPPC_PORT_DOWN    OfpPortConfig = 1
+	OfpPortConfig_OFPPC_NO_RECV      OfpPortConfig = 4
+	OfpPortConfig_OFPPC_NO_FWD       OfpPortConfig = 32
+	OfpPortConfig_OFPPC_NO_PACKET_IN OfpPortConfig = 64
+)
+
+var OfpPortConfig_name = map[int32]string{
+	0:  "OFPPC_INVALID",
+	1:  "OFPPC_PORT_DOWN",
+	4:  "OFPPC_NO_RECV",
+	32: "OFPPC_NO_FWD",
+	64: "OFPPC_NO_PACKET_IN",
+}
+
+var OfpPortConfig_value = map[string]int32{
+	"OFPPC_INVALID":      0,
+	"OFPPC_PORT_DOWN":    1,
+	"OFPPC_NO_RECV":      4,
+	"OFPPC_NO_FWD":       32,
+	"OFPPC_NO_PACKET_IN": 64,
+}
+
+func (x OfpPortConfig) String() string {
+	return proto.EnumName(OfpPortConfig_name, int32(x))
+}
+
+func (OfpPortConfig) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{7}
+}
+
+// Current state of the physical port.  These are not configurable from
+// the controller.
+type OfpPortState int32
+
+const (
+	OfpPortState_OFPPS_INVALID   OfpPortState = 0
+	OfpPortState_OFPPS_LINK_DOWN OfpPortState = 1
+	OfpPortState_OFPPS_BLOCKED   OfpPortState = 2
+	OfpPortState_OFPPS_LIVE      OfpPortState = 4
+)
+
+var OfpPortState_name = map[int32]string{
+	0: "OFPPS_INVALID",
+	1: "OFPPS_LINK_DOWN",
+	2: "OFPPS_BLOCKED",
+	4: "OFPPS_LIVE",
+}
+
+var OfpPortState_value = map[string]int32{
+	"OFPPS_INVALID":   0,
+	"OFPPS_LINK_DOWN": 1,
+	"OFPPS_BLOCKED":   2,
+	"OFPPS_LIVE":      4,
+}
+
+func (x OfpPortState) String() string {
+	return proto.EnumName(OfpPortState_name, int32(x))
+}
+
+func (OfpPortState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{8}
+}
+
+// Features of ports available in a datapath.
+type OfpPortFeatures int32
+
+const (
+	OfpPortFeatures_OFPPF_INVALID    OfpPortFeatures = 0
+	OfpPortFeatures_OFPPF_10MB_HD    OfpPortFeatures = 1
+	OfpPortFeatures_OFPPF_10MB_FD    OfpPortFeatures = 2
+	OfpPortFeatures_OFPPF_100MB_HD   OfpPortFeatures = 4
+	OfpPortFeatures_OFPPF_100MB_FD   OfpPortFeatures = 8
+	OfpPortFeatures_OFPPF_1GB_HD     OfpPortFeatures = 16
+	OfpPortFeatures_OFPPF_1GB_FD     OfpPortFeatures = 32
+	OfpPortFeatures_OFPPF_10GB_FD    OfpPortFeatures = 64
+	OfpPortFeatures_OFPPF_40GB_FD    OfpPortFeatures = 128
+	OfpPortFeatures_OFPPF_100GB_FD   OfpPortFeatures = 256
+	OfpPortFeatures_OFPPF_1TB_FD     OfpPortFeatures = 512
+	OfpPortFeatures_OFPPF_OTHER      OfpPortFeatures = 1024
+	OfpPortFeatures_OFPPF_COPPER     OfpPortFeatures = 2048
+	OfpPortFeatures_OFPPF_FIBER      OfpPortFeatures = 4096
+	OfpPortFeatures_OFPPF_AUTONEG    OfpPortFeatures = 8192
+	OfpPortFeatures_OFPPF_PAUSE      OfpPortFeatures = 16384
+	OfpPortFeatures_OFPPF_PAUSE_ASYM OfpPortFeatures = 32768
+)
+
+var OfpPortFeatures_name = map[int32]string{
+	0:     "OFPPF_INVALID",
+	1:     "OFPPF_10MB_HD",
+	2:     "OFPPF_10MB_FD",
+	4:     "OFPPF_100MB_HD",
+	8:     "OFPPF_100MB_FD",
+	16:    "OFPPF_1GB_HD",
+	32:    "OFPPF_1GB_FD",
+	64:    "OFPPF_10GB_FD",
+	128:   "OFPPF_40GB_FD",
+	256:   "OFPPF_100GB_FD",
+	512:   "OFPPF_1TB_FD",
+	1024:  "OFPPF_OTHER",
+	2048:  "OFPPF_COPPER",
+	4096:  "OFPPF_FIBER",
+	8192:  "OFPPF_AUTONEG",
+	16384: "OFPPF_PAUSE",
+	32768: "OFPPF_PAUSE_ASYM",
+}
+
+var OfpPortFeatures_value = map[string]int32{
+	"OFPPF_INVALID":    0,
+	"OFPPF_10MB_HD":    1,
+	"OFPPF_10MB_FD":    2,
+	"OFPPF_100MB_HD":   4,
+	"OFPPF_100MB_FD":   8,
+	"OFPPF_1GB_HD":     16,
+	"OFPPF_1GB_FD":     32,
+	"OFPPF_10GB_FD":    64,
+	"OFPPF_40GB_FD":    128,
+	"OFPPF_100GB_FD":   256,
+	"OFPPF_1TB_FD":     512,
+	"OFPPF_OTHER":      1024,
+	"OFPPF_COPPER":     2048,
+	"OFPPF_FIBER":      4096,
+	"OFPPF_AUTONEG":    8192,
+	"OFPPF_PAUSE":      16384,
+	"OFPPF_PAUSE_ASYM": 32768,
+}
+
+func (x OfpPortFeatures) String() string {
+	return proto.EnumName(OfpPortFeatures_name, int32(x))
+}
+
+func (OfpPortFeatures) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{9}
+}
+
+// What changed about the physical port
+type OfpPortReason int32
+
+const (
+	OfpPortReason_OFPPR_ADD    OfpPortReason = 0
+	OfpPortReason_OFPPR_DELETE OfpPortReason = 1
+	OfpPortReason_OFPPR_MODIFY OfpPortReason = 2
+)
+
+var OfpPortReason_name = map[int32]string{
+	0: "OFPPR_ADD",
+	1: "OFPPR_DELETE",
+	2: "OFPPR_MODIFY",
+}
+
+var OfpPortReason_value = map[string]int32{
+	"OFPPR_ADD":    0,
+	"OFPPR_DELETE": 1,
+	"OFPPR_MODIFY": 2,
+}
+
+func (x OfpPortReason) String() string {
+	return proto.EnumName(OfpPortReason_name, int32(x))
+}
+
+func (OfpPortReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{10}
+}
+
+// The match type indicates the match structure (set of fields that compose the
+// match) in use. The match type is placed in the type field at the beginning
+// of all match structures. The "OpenFlow Extensible Match" type corresponds
+// to OXM TLV format described below and must be supported by all OpenFlow
+// switches. Extensions that define other match types may be published on the
+// ONF wiki. Support for extensions is optional.
+type OfpMatchType int32
+
+const (
+	OfpMatchType_OFPMT_STANDARD OfpMatchType = 0
+	OfpMatchType_OFPMT_OXM      OfpMatchType = 1
+)
+
+var OfpMatchType_name = map[int32]string{
+	0: "OFPMT_STANDARD",
+	1: "OFPMT_OXM",
+}
+
+var OfpMatchType_value = map[string]int32{
+	"OFPMT_STANDARD": 0,
+	"OFPMT_OXM":      1,
+}
+
+func (x OfpMatchType) String() string {
+	return proto.EnumName(OfpMatchType_name, int32(x))
+}
+
+func (OfpMatchType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{11}
+}
+
+// OXM Class IDs.
+// The high order bit differentiate reserved classes from member classes.
+// Classes 0x0000 to 0x7FFF are member classes, allocated by ONF.
+// Classes 0x8000 to 0xFFFE are reserved classes, reserved for standardisation.
+type OfpOxmClass int32
+
+const (
+	OfpOxmClass_OFPXMC_NXM_0          OfpOxmClass = 0
+	OfpOxmClass_OFPXMC_NXM_1          OfpOxmClass = 1
+	OfpOxmClass_OFPXMC_OPENFLOW_BASIC OfpOxmClass = 32768
+	OfpOxmClass_OFPXMC_EXPERIMENTER   OfpOxmClass = 65535
+)
+
+var OfpOxmClass_name = map[int32]string{
+	0:     "OFPXMC_NXM_0",
+	1:     "OFPXMC_NXM_1",
+	32768: "OFPXMC_OPENFLOW_BASIC",
+	65535: "OFPXMC_EXPERIMENTER",
+}
+
+var OfpOxmClass_value = map[string]int32{
+	"OFPXMC_NXM_0":          0,
+	"OFPXMC_NXM_1":          1,
+	"OFPXMC_OPENFLOW_BASIC": 32768,
+	"OFPXMC_EXPERIMENTER":   65535,
+}
+
+func (x OfpOxmClass) String() string {
+	return proto.EnumName(OfpOxmClass_name, int32(x))
+}
+
+func (OfpOxmClass) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{12}
+}
+
+// OXM Flow field types for OpenFlow basic class.
+type OxmOfbFieldTypes int32
+
+const (
+	OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT        OxmOfbFieldTypes = 0
+	OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT    OxmOfbFieldTypes = 1
+	OxmOfbFieldTypes_OFPXMT_OFB_METADATA       OxmOfbFieldTypes = 2
+	OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST        OxmOfbFieldTypes = 3
+	OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC        OxmOfbFieldTypes = 4
+	OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE       OxmOfbFieldTypes = 5
+	OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID       OxmOfbFieldTypes = 6
+	OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP       OxmOfbFieldTypes = 7
+	OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP        OxmOfbFieldTypes = 8
+	OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN         OxmOfbFieldTypes = 9
+	OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO       OxmOfbFieldTypes = 10
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC       OxmOfbFieldTypes = 11
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST       OxmOfbFieldTypes = 12
+	OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC        OxmOfbFieldTypes = 13
+	OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST        OxmOfbFieldTypes = 14
+	OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC        OxmOfbFieldTypes = 15
+	OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST        OxmOfbFieldTypes = 16
+	OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC       OxmOfbFieldTypes = 17
+	OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST       OxmOfbFieldTypes = 18
+	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE    OxmOfbFieldTypes = 19
+	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE    OxmOfbFieldTypes = 20
+	OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP         OxmOfbFieldTypes = 21
+	OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA        OxmOfbFieldTypes = 22
+	OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA        OxmOfbFieldTypes = 23
+	OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA        OxmOfbFieldTypes = 24
+	OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA        OxmOfbFieldTypes = 25
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC       OxmOfbFieldTypes = 26
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST       OxmOfbFieldTypes = 27
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL    OxmOfbFieldTypes = 28
+	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE    OxmOfbFieldTypes = 29
+	OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE    OxmOfbFieldTypes = 30
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET OxmOfbFieldTypes = 31
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL    OxmOfbFieldTypes = 32
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL    OxmOfbFieldTypes = 33
+	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL     OxmOfbFieldTypes = 34
+	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC        OxmOfbFieldTypes = 35
+	OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS       OxmOfbFieldTypes = 36
+	OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID       OxmOfbFieldTypes = 37
+	OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID      OxmOfbFieldTypes = 38
+	OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR    OxmOfbFieldTypes = 39
+)
+
+var OxmOfbFieldTypes_name = map[int32]string{
+	0:  "OFPXMT_OFB_IN_PORT",
+	1:  "OFPXMT_OFB_IN_PHY_PORT",
+	2:  "OFPXMT_OFB_METADATA",
+	3:  "OFPXMT_OFB_ETH_DST",
+	4:  "OFPXMT_OFB_ETH_SRC",
+	5:  "OFPXMT_OFB_ETH_TYPE",
+	6:  "OFPXMT_OFB_VLAN_VID",
+	7:  "OFPXMT_OFB_VLAN_PCP",
+	8:  "OFPXMT_OFB_IP_DSCP",
+	9:  "OFPXMT_OFB_IP_ECN",
+	10: "OFPXMT_OFB_IP_PROTO",
+	11: "OFPXMT_OFB_IPV4_SRC",
+	12: "OFPXMT_OFB_IPV4_DST",
+	13: "OFPXMT_OFB_TCP_SRC",
+	14: "OFPXMT_OFB_TCP_DST",
+	15: "OFPXMT_OFB_UDP_SRC",
+	16: "OFPXMT_OFB_UDP_DST",
+	17: "OFPXMT_OFB_SCTP_SRC",
+	18: "OFPXMT_OFB_SCTP_DST",
+	19: "OFPXMT_OFB_ICMPV4_TYPE",
+	20: "OFPXMT_OFB_ICMPV4_CODE",
+	21: "OFPXMT_OFB_ARP_OP",
+	22: "OFPXMT_OFB_ARP_SPA",
+	23: "OFPXMT_OFB_ARP_TPA",
+	24: "OFPXMT_OFB_ARP_SHA",
+	25: "OFPXMT_OFB_ARP_THA",
+	26: "OFPXMT_OFB_IPV6_SRC",
+	27: "OFPXMT_OFB_IPV6_DST",
+	28: "OFPXMT_OFB_IPV6_FLABEL",
+	29: "OFPXMT_OFB_ICMPV6_TYPE",
+	30: "OFPXMT_OFB_ICMPV6_CODE",
+	31: "OFPXMT_OFB_IPV6_ND_TARGET",
+	32: "OFPXMT_OFB_IPV6_ND_SLL",
+	33: "OFPXMT_OFB_IPV6_ND_TLL",
+	34: "OFPXMT_OFB_MPLS_LABEL",
+	35: "OFPXMT_OFB_MPLS_TC",
+	36: "OFPXMT_OFB_MPLS_BOS",
+	37: "OFPXMT_OFB_PBB_ISID",
+	38: "OFPXMT_OFB_TUNNEL_ID",
+	39: "OFPXMT_OFB_IPV6_EXTHDR",
+}
+
+var OxmOfbFieldTypes_value = map[string]int32{
+	"OFPXMT_OFB_IN_PORT":        0,
+	"OFPXMT_OFB_IN_PHY_PORT":    1,
+	"OFPXMT_OFB_METADATA":       2,
+	"OFPXMT_OFB_ETH_DST":        3,
+	"OFPXMT_OFB_ETH_SRC":        4,
+	"OFPXMT_OFB_ETH_TYPE":       5,
+	"OFPXMT_OFB_VLAN_VID":       6,
+	"OFPXMT_OFB_VLAN_PCP":       7,
+	"OFPXMT_OFB_IP_DSCP":        8,
+	"OFPXMT_OFB_IP_ECN":         9,
+	"OFPXMT_OFB_IP_PROTO":       10,
+	"OFPXMT_OFB_IPV4_SRC":       11,
+	"OFPXMT_OFB_IPV4_DST":       12,
+	"OFPXMT_OFB_TCP_SRC":        13,
+	"OFPXMT_OFB_TCP_DST":        14,
+	"OFPXMT_OFB_UDP_SRC":        15,
+	"OFPXMT_OFB_UDP_DST":        16,
+	"OFPXMT_OFB_SCTP_SRC":       17,
+	"OFPXMT_OFB_SCTP_DST":       18,
+	"OFPXMT_OFB_ICMPV4_TYPE":    19,
+	"OFPXMT_OFB_ICMPV4_CODE":    20,
+	"OFPXMT_OFB_ARP_OP":         21,
+	"OFPXMT_OFB_ARP_SPA":        22,
+	"OFPXMT_OFB_ARP_TPA":        23,
+	"OFPXMT_OFB_ARP_SHA":        24,
+	"OFPXMT_OFB_ARP_THA":        25,
+	"OFPXMT_OFB_IPV6_SRC":       26,
+	"OFPXMT_OFB_IPV6_DST":       27,
+	"OFPXMT_OFB_IPV6_FLABEL":    28,
+	"OFPXMT_OFB_ICMPV6_TYPE":    29,
+	"OFPXMT_OFB_ICMPV6_CODE":    30,
+	"OFPXMT_OFB_IPV6_ND_TARGET": 31,
+	"OFPXMT_OFB_IPV6_ND_SLL":    32,
+	"OFPXMT_OFB_IPV6_ND_TLL":    33,
+	"OFPXMT_OFB_MPLS_LABEL":     34,
+	"OFPXMT_OFB_MPLS_TC":        35,
+	"OFPXMT_OFB_MPLS_BOS":       36,
+	"OFPXMT_OFB_PBB_ISID":       37,
+	"OFPXMT_OFB_TUNNEL_ID":      38,
+	"OFPXMT_OFB_IPV6_EXTHDR":    39,
+}
+
+func (x OxmOfbFieldTypes) String() string {
+	return proto.EnumName(OxmOfbFieldTypes_name, int32(x))
+}
+
+func (OxmOfbFieldTypes) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{13}
+}
+
+// The VLAN id is 12-bits, so we can use the entire 16 bits to indicate
+// special conditions.
+type OfpVlanId int32
+
+const (
+	OfpVlanId_OFPVID_NONE    OfpVlanId = 0
+	OfpVlanId_OFPVID_PRESENT OfpVlanId = 4096
+)
+
+var OfpVlanId_name = map[int32]string{
+	0:    "OFPVID_NONE",
+	4096: "OFPVID_PRESENT",
+}
+
+var OfpVlanId_value = map[string]int32{
+	"OFPVID_NONE":    0,
+	"OFPVID_PRESENT": 4096,
+}
+
+func (x OfpVlanId) String() string {
+	return proto.EnumName(OfpVlanId_name, int32(x))
+}
+
+func (OfpVlanId) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{14}
+}
+
+// Bit definitions for IPv6 Extension Header pseudo-field.
+type OfpIpv6ExthdrFlags int32
+
+const (
+	OfpIpv6ExthdrFlags_OFPIEH_INVALID OfpIpv6ExthdrFlags = 0
+	OfpIpv6ExthdrFlags_OFPIEH_NONEXT  OfpIpv6ExthdrFlags = 1
+	OfpIpv6ExthdrFlags_OFPIEH_ESP     OfpIpv6ExthdrFlags = 2
+	OfpIpv6ExthdrFlags_OFPIEH_AUTH    OfpIpv6ExthdrFlags = 4
+	OfpIpv6ExthdrFlags_OFPIEH_DEST    OfpIpv6ExthdrFlags = 8
+	OfpIpv6ExthdrFlags_OFPIEH_FRAG    OfpIpv6ExthdrFlags = 16
+	OfpIpv6ExthdrFlags_OFPIEH_ROUTER  OfpIpv6ExthdrFlags = 32
+	OfpIpv6ExthdrFlags_OFPIEH_HOP     OfpIpv6ExthdrFlags = 64
+	OfpIpv6ExthdrFlags_OFPIEH_UNREP   OfpIpv6ExthdrFlags = 128
+	OfpIpv6ExthdrFlags_OFPIEH_UNSEQ   OfpIpv6ExthdrFlags = 256
+)
+
+var OfpIpv6ExthdrFlags_name = map[int32]string{
+	0:   "OFPIEH_INVALID",
+	1:   "OFPIEH_NONEXT",
+	2:   "OFPIEH_ESP",
+	4:   "OFPIEH_AUTH",
+	8:   "OFPIEH_DEST",
+	16:  "OFPIEH_FRAG",
+	32:  "OFPIEH_ROUTER",
+	64:  "OFPIEH_HOP",
+	128: "OFPIEH_UNREP",
+	256: "OFPIEH_UNSEQ",
+}
+
+var OfpIpv6ExthdrFlags_value = map[string]int32{
+	"OFPIEH_INVALID": 0,
+	"OFPIEH_NONEXT":  1,
+	"OFPIEH_ESP":     2,
+	"OFPIEH_AUTH":    4,
+	"OFPIEH_DEST":    8,
+	"OFPIEH_FRAG":    16,
+	"OFPIEH_ROUTER":  32,
+	"OFPIEH_HOP":     64,
+	"OFPIEH_UNREP":   128,
+	"OFPIEH_UNSEQ":   256,
+}
+
+func (x OfpIpv6ExthdrFlags) String() string {
+	return proto.EnumName(OfpIpv6ExthdrFlags_name, int32(x))
+}
+
+func (OfpIpv6ExthdrFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{15}
+}
+
+type OfpActionType int32
+
+const (
+	OfpActionType_OFPAT_OUTPUT       OfpActionType = 0
+	OfpActionType_OFPAT_COPY_TTL_OUT OfpActionType = 11
+	OfpActionType_OFPAT_COPY_TTL_IN  OfpActionType = 12
+	OfpActionType_OFPAT_SET_MPLS_TTL OfpActionType = 15
+	OfpActionType_OFPAT_DEC_MPLS_TTL OfpActionType = 16
+	OfpActionType_OFPAT_PUSH_VLAN    OfpActionType = 17
+	OfpActionType_OFPAT_POP_VLAN     OfpActionType = 18
+	OfpActionType_OFPAT_PUSH_MPLS    OfpActionType = 19
+	OfpActionType_OFPAT_POP_MPLS     OfpActionType = 20
+	OfpActionType_OFPAT_SET_QUEUE    OfpActionType = 21
+	OfpActionType_OFPAT_GROUP        OfpActionType = 22
+	OfpActionType_OFPAT_SET_NW_TTL   OfpActionType = 23
+	OfpActionType_OFPAT_DEC_NW_TTL   OfpActionType = 24
+	OfpActionType_OFPAT_SET_FIELD    OfpActionType = 25
+	OfpActionType_OFPAT_PUSH_PBB     OfpActionType = 26
+	OfpActionType_OFPAT_POP_PBB      OfpActionType = 27
+	OfpActionType_OFPAT_EXPERIMENTER OfpActionType = 65535
+)
+
+var OfpActionType_name = map[int32]string{
+	0:     "OFPAT_OUTPUT",
+	11:    "OFPAT_COPY_TTL_OUT",
+	12:    "OFPAT_COPY_TTL_IN",
+	15:    "OFPAT_SET_MPLS_TTL",
+	16:    "OFPAT_DEC_MPLS_TTL",
+	17:    "OFPAT_PUSH_VLAN",
+	18:    "OFPAT_POP_VLAN",
+	19:    "OFPAT_PUSH_MPLS",
+	20:    "OFPAT_POP_MPLS",
+	21:    "OFPAT_SET_QUEUE",
+	22:    "OFPAT_GROUP",
+	23:    "OFPAT_SET_NW_TTL",
+	24:    "OFPAT_DEC_NW_TTL",
+	25:    "OFPAT_SET_FIELD",
+	26:    "OFPAT_PUSH_PBB",
+	27:    "OFPAT_POP_PBB",
+	65535: "OFPAT_EXPERIMENTER",
+}
+
+var OfpActionType_value = map[string]int32{
+	"OFPAT_OUTPUT":       0,
+	"OFPAT_COPY_TTL_OUT": 11,
+	"OFPAT_COPY_TTL_IN":  12,
+	"OFPAT_SET_MPLS_TTL": 15,
+	"OFPAT_DEC_MPLS_TTL": 16,
+	"OFPAT_PUSH_VLAN":    17,
+	"OFPAT_POP_VLAN":     18,
+	"OFPAT_PUSH_MPLS":    19,
+	"OFPAT_POP_MPLS":     20,
+	"OFPAT_SET_QUEUE":    21,
+	"OFPAT_GROUP":        22,
+	"OFPAT_SET_NW_TTL":   23,
+	"OFPAT_DEC_NW_TTL":   24,
+	"OFPAT_SET_FIELD":    25,
+	"OFPAT_PUSH_PBB":     26,
+	"OFPAT_POP_PBB":      27,
+	"OFPAT_EXPERIMENTER": 65535,
+}
+
+func (x OfpActionType) String() string {
+	return proto.EnumName(OfpActionType_name, int32(x))
+}
+
+func (OfpActionType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{16}
+}
+
+type OfpControllerMaxLen int32
+
+const (
+	OfpControllerMaxLen_OFPCML_INVALID   OfpControllerMaxLen = 0
+	OfpControllerMaxLen_OFPCML_MAX       OfpControllerMaxLen = 65509
+	OfpControllerMaxLen_OFPCML_NO_BUFFER OfpControllerMaxLen = 65535
+)
+
+var OfpControllerMaxLen_name = map[int32]string{
+	0:     "OFPCML_INVALID",
+	65509: "OFPCML_MAX",
+	65535: "OFPCML_NO_BUFFER",
+}
+
+var OfpControllerMaxLen_value = map[string]int32{
+	"OFPCML_INVALID":   0,
+	"OFPCML_MAX":       65509,
+	"OFPCML_NO_BUFFER": 65535,
+}
+
+func (x OfpControllerMaxLen) String() string {
+	return proto.EnumName(OfpControllerMaxLen_name, int32(x))
+}
+
+func (OfpControllerMaxLen) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{17}
+}
+
+type OfpInstructionType int32
+
+const (
+	OfpInstructionType_OFPIT_INVALID        OfpInstructionType = 0
+	OfpInstructionType_OFPIT_GOTO_TABLE     OfpInstructionType = 1
+	OfpInstructionType_OFPIT_WRITE_METADATA OfpInstructionType = 2
+	OfpInstructionType_OFPIT_WRITE_ACTIONS  OfpInstructionType = 3
+	OfpInstructionType_OFPIT_APPLY_ACTIONS  OfpInstructionType = 4
+	OfpInstructionType_OFPIT_CLEAR_ACTIONS  OfpInstructionType = 5
+	OfpInstructionType_OFPIT_METER          OfpInstructionType = 6
+	OfpInstructionType_OFPIT_EXPERIMENTER   OfpInstructionType = 65535
+)
+
+var OfpInstructionType_name = map[int32]string{
+	0:     "OFPIT_INVALID",
+	1:     "OFPIT_GOTO_TABLE",
+	2:     "OFPIT_WRITE_METADATA",
+	3:     "OFPIT_WRITE_ACTIONS",
+	4:     "OFPIT_APPLY_ACTIONS",
+	5:     "OFPIT_CLEAR_ACTIONS",
+	6:     "OFPIT_METER",
+	65535: "OFPIT_EXPERIMENTER",
+}
+
+var OfpInstructionType_value = map[string]int32{
+	"OFPIT_INVALID":        0,
+	"OFPIT_GOTO_TABLE":     1,
+	"OFPIT_WRITE_METADATA": 2,
+	"OFPIT_WRITE_ACTIONS":  3,
+	"OFPIT_APPLY_ACTIONS":  4,
+	"OFPIT_CLEAR_ACTIONS":  5,
+	"OFPIT_METER":          6,
+	"OFPIT_EXPERIMENTER":   65535,
+}
+
+func (x OfpInstructionType) String() string {
+	return proto.EnumName(OfpInstructionType_name, int32(x))
+}
+
+func (OfpInstructionType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{18}
+}
+
+type OfpFlowModCommand int32
+
+const (
+	OfpFlowModCommand_OFPFC_ADD           OfpFlowModCommand = 0
+	OfpFlowModCommand_OFPFC_MODIFY        OfpFlowModCommand = 1
+	OfpFlowModCommand_OFPFC_MODIFY_STRICT OfpFlowModCommand = 2
+	OfpFlowModCommand_OFPFC_DELETE        OfpFlowModCommand = 3
+	OfpFlowModCommand_OFPFC_DELETE_STRICT OfpFlowModCommand = 4
+)
+
+var OfpFlowModCommand_name = map[int32]string{
+	0: "OFPFC_ADD",
+	1: "OFPFC_MODIFY",
+	2: "OFPFC_MODIFY_STRICT",
+	3: "OFPFC_DELETE",
+	4: "OFPFC_DELETE_STRICT",
+}
+
+var OfpFlowModCommand_value = map[string]int32{
+	"OFPFC_ADD":           0,
+	"OFPFC_MODIFY":        1,
+	"OFPFC_MODIFY_STRICT": 2,
+	"OFPFC_DELETE":        3,
+	"OFPFC_DELETE_STRICT": 4,
+}
+
+func (x OfpFlowModCommand) String() string {
+	return proto.EnumName(OfpFlowModCommand_name, int32(x))
+}
+
+func (OfpFlowModCommand) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{19}
+}
+
+type OfpFlowModFlags int32
+
+const (
+	OfpFlowModFlags_OFPFF_INVALID       OfpFlowModFlags = 0
+	OfpFlowModFlags_OFPFF_SEND_FLOW_REM OfpFlowModFlags = 1
+	OfpFlowModFlags_OFPFF_CHECK_OVERLAP OfpFlowModFlags = 2
+	OfpFlowModFlags_OFPFF_RESET_COUNTS  OfpFlowModFlags = 4
+	OfpFlowModFlags_OFPFF_NO_PKT_COUNTS OfpFlowModFlags = 8
+	OfpFlowModFlags_OFPFF_NO_BYT_COUNTS OfpFlowModFlags = 16
+)
+
+var OfpFlowModFlags_name = map[int32]string{
+	0:  "OFPFF_INVALID",
+	1:  "OFPFF_SEND_FLOW_REM",
+	2:  "OFPFF_CHECK_OVERLAP",
+	4:  "OFPFF_RESET_COUNTS",
+	8:  "OFPFF_NO_PKT_COUNTS",
+	16: "OFPFF_NO_BYT_COUNTS",
+}
+
+var OfpFlowModFlags_value = map[string]int32{
+	"OFPFF_INVALID":       0,
+	"OFPFF_SEND_FLOW_REM": 1,
+	"OFPFF_CHECK_OVERLAP": 2,
+	"OFPFF_RESET_COUNTS":  4,
+	"OFPFF_NO_PKT_COUNTS": 8,
+	"OFPFF_NO_BYT_COUNTS": 16,
+}
+
+func (x OfpFlowModFlags) String() string {
+	return proto.EnumName(OfpFlowModFlags_name, int32(x))
+}
+
+func (OfpFlowModFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{20}
+}
+
+// Group numbering. Groups can use any number up to OFPG_MAX.
+type OfpGroup int32
+
+const (
+	OfpGroup_OFPG_INVALID OfpGroup = 0
+	// Last usable group number.
+	OfpGroup_OFPG_MAX OfpGroup = 2147483392
+	// Fake groups.
+	OfpGroup_OFPG_ALL OfpGroup = 2147483644
+	OfpGroup_OFPG_ANY OfpGroup = 2147483647
+)
+
+var OfpGroup_name = map[int32]string{
+	0:          "OFPG_INVALID",
+	2147483392: "OFPG_MAX",
+	2147483644: "OFPG_ALL",
+	2147483647: "OFPG_ANY",
+}
+
+var OfpGroup_value = map[string]int32{
+	"OFPG_INVALID": 0,
+	"OFPG_MAX":     2147483392,
+	"OFPG_ALL":     2147483644,
+	"OFPG_ANY":     2147483647,
+}
+
+func (x OfpGroup) String() string {
+	return proto.EnumName(OfpGroup_name, int32(x))
+}
+
+func (OfpGroup) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{21}
+}
+
+// Group commands
+type OfpGroupModCommand int32
+
+const (
+	OfpGroupModCommand_OFPGC_ADD    OfpGroupModCommand = 0
+	OfpGroupModCommand_OFPGC_MODIFY OfpGroupModCommand = 1
+	OfpGroupModCommand_OFPGC_DELETE OfpGroupModCommand = 2
+)
+
+var OfpGroupModCommand_name = map[int32]string{
+	0: "OFPGC_ADD",
+	1: "OFPGC_MODIFY",
+	2: "OFPGC_DELETE",
+}
+
+var OfpGroupModCommand_value = map[string]int32{
+	"OFPGC_ADD":    0,
+	"OFPGC_MODIFY": 1,
+	"OFPGC_DELETE": 2,
+}
+
+func (x OfpGroupModCommand) String() string {
+	return proto.EnumName(OfpGroupModCommand_name, int32(x))
+}
+
+func (OfpGroupModCommand) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{22}
+}
+
+// Group types.  Values in the range [128; 255] are reserved for experimental
+// use.
+type OfpGroupType int32
+
+const (
+	OfpGroupType_OFPGT_ALL      OfpGroupType = 0
+	OfpGroupType_OFPGT_SELECT   OfpGroupType = 1
+	OfpGroupType_OFPGT_INDIRECT OfpGroupType = 2
+	OfpGroupType_OFPGT_FF       OfpGroupType = 3
+)
+
+var OfpGroupType_name = map[int32]string{
+	0: "OFPGT_ALL",
+	1: "OFPGT_SELECT",
+	2: "OFPGT_INDIRECT",
+	3: "OFPGT_FF",
+}
+
+var OfpGroupType_value = map[string]int32{
+	"OFPGT_ALL":      0,
+	"OFPGT_SELECT":   1,
+	"OFPGT_INDIRECT": 2,
+	"OFPGT_FF":       3,
+}
+
+func (x OfpGroupType) String() string {
+	return proto.EnumName(OfpGroupType_name, int32(x))
+}
+
+func (OfpGroupType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{23}
+}
+
+// Why is this packet being sent to the controller?
+type OfpPacketInReason int32
+
+const (
+	OfpPacketInReason_OFPR_NO_MATCH    OfpPacketInReason = 0
+	OfpPacketInReason_OFPR_ACTION      OfpPacketInReason = 1
+	OfpPacketInReason_OFPR_INVALID_TTL OfpPacketInReason = 2
+)
+
+var OfpPacketInReason_name = map[int32]string{
+	0: "OFPR_NO_MATCH",
+	1: "OFPR_ACTION",
+	2: "OFPR_INVALID_TTL",
+}
+
+var OfpPacketInReason_value = map[string]int32{
+	"OFPR_NO_MATCH":    0,
+	"OFPR_ACTION":      1,
+	"OFPR_INVALID_TTL": 2,
+}
+
+func (x OfpPacketInReason) String() string {
+	return proto.EnumName(OfpPacketInReason_name, int32(x))
+}
+
+func (OfpPacketInReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{24}
+}
+
+// Why was this flow removed?
+type OfpFlowRemovedReason int32
+
+const (
+	OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT OfpFlowRemovedReason = 0
+	OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT OfpFlowRemovedReason = 1
+	OfpFlowRemovedReason_OFPRR_DELETE       OfpFlowRemovedReason = 2
+	OfpFlowRemovedReason_OFPRR_GROUP_DELETE OfpFlowRemovedReason = 3
+	OfpFlowRemovedReason_OFPRR_METER_DELETE OfpFlowRemovedReason = 4
+)
+
+var OfpFlowRemovedReason_name = map[int32]string{
+	0: "OFPRR_IDLE_TIMEOUT",
+	1: "OFPRR_HARD_TIMEOUT",
+	2: "OFPRR_DELETE",
+	3: "OFPRR_GROUP_DELETE",
+	4: "OFPRR_METER_DELETE",
+}
+
+var OfpFlowRemovedReason_value = map[string]int32{
+	"OFPRR_IDLE_TIMEOUT": 0,
+	"OFPRR_HARD_TIMEOUT": 1,
+	"OFPRR_DELETE":       2,
+	"OFPRR_GROUP_DELETE": 3,
+	"OFPRR_METER_DELETE": 4,
+}
+
+func (x OfpFlowRemovedReason) String() string {
+	return proto.EnumName(OfpFlowRemovedReason_name, int32(x))
+}
+
+func (OfpFlowRemovedReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{25}
+}
+
+// Meter numbering. Flow meters can use any number up to OFPM_MAX.
+type OfpMeter int32
+
+const (
+	OfpMeter_OFPM_ZERO OfpMeter = 0
+	// Last usable meter.
+	OfpMeter_OFPM_MAX OfpMeter = 2147418112
+	// Virtual meters.
+	OfpMeter_OFPM_SLOWPATH   OfpMeter = 2147483645
+	OfpMeter_OFPM_CONTROLLER OfpMeter = 2147483646
+	OfpMeter_OFPM_ALL        OfpMeter = 2147483647
+)
+
+var OfpMeter_name = map[int32]string{
+	0:          "OFPM_ZERO",
+	2147418112: "OFPM_MAX",
+	2147483645: "OFPM_SLOWPATH",
+	2147483646: "OFPM_CONTROLLER",
+	2147483647: "OFPM_ALL",
+}
+
+var OfpMeter_value = map[string]int32{
+	"OFPM_ZERO":       0,
+	"OFPM_MAX":        2147418112,
+	"OFPM_SLOWPATH":   2147483645,
+	"OFPM_CONTROLLER": 2147483646,
+	"OFPM_ALL":        2147483647,
+}
+
+func (x OfpMeter) String() string {
+	return proto.EnumName(OfpMeter_name, int32(x))
+}
+
+func (OfpMeter) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{26}
+}
+
+// Meter band types
+type OfpMeterBandType int32
+
+const (
+	OfpMeterBandType_OFPMBT_INVALID      OfpMeterBandType = 0
+	OfpMeterBandType_OFPMBT_DROP         OfpMeterBandType = 1
+	OfpMeterBandType_OFPMBT_DSCP_REMARK  OfpMeterBandType = 2
+	OfpMeterBandType_OFPMBT_EXPERIMENTER OfpMeterBandType = 65535
+)
+
+var OfpMeterBandType_name = map[int32]string{
+	0:     "OFPMBT_INVALID",
+	1:     "OFPMBT_DROP",
+	2:     "OFPMBT_DSCP_REMARK",
+	65535: "OFPMBT_EXPERIMENTER",
+}
+
+var OfpMeterBandType_value = map[string]int32{
+	"OFPMBT_INVALID":      0,
+	"OFPMBT_DROP":         1,
+	"OFPMBT_DSCP_REMARK":  2,
+	"OFPMBT_EXPERIMENTER": 65535,
+}
+
+func (x OfpMeterBandType) String() string {
+	return proto.EnumName(OfpMeterBandType_name, int32(x))
+}
+
+func (OfpMeterBandType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{27}
+}
+
+// Meter commands
+type OfpMeterModCommand int32
+
+const (
+	OfpMeterModCommand_OFPMC_ADD    OfpMeterModCommand = 0
+	OfpMeterModCommand_OFPMC_MODIFY OfpMeterModCommand = 1
+	OfpMeterModCommand_OFPMC_DELETE OfpMeterModCommand = 2
+)
+
+var OfpMeterModCommand_name = map[int32]string{
+	0: "OFPMC_ADD",
+	1: "OFPMC_MODIFY",
+	2: "OFPMC_DELETE",
+}
+
+var OfpMeterModCommand_value = map[string]int32{
+	"OFPMC_ADD":    0,
+	"OFPMC_MODIFY": 1,
+	"OFPMC_DELETE": 2,
+}
+
+func (x OfpMeterModCommand) String() string {
+	return proto.EnumName(OfpMeterModCommand_name, int32(x))
+}
+
+func (OfpMeterModCommand) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{28}
+}
+
+// Meter configuration flags
+type OfpMeterFlags int32
+
+const (
+	OfpMeterFlags_OFPMF_INVALID OfpMeterFlags = 0
+	OfpMeterFlags_OFPMF_KBPS    OfpMeterFlags = 1
+	OfpMeterFlags_OFPMF_PKTPS   OfpMeterFlags = 2
+	OfpMeterFlags_OFPMF_BURST   OfpMeterFlags = 4
+	OfpMeterFlags_OFPMF_STATS   OfpMeterFlags = 8
+)
+
+var OfpMeterFlags_name = map[int32]string{
+	0: "OFPMF_INVALID",
+	1: "OFPMF_KBPS",
+	2: "OFPMF_PKTPS",
+	4: "OFPMF_BURST",
+	8: "OFPMF_STATS",
+}
+
+var OfpMeterFlags_value = map[string]int32{
+	"OFPMF_INVALID": 0,
+	"OFPMF_KBPS":    1,
+	"OFPMF_PKTPS":   2,
+	"OFPMF_BURST":   4,
+	"OFPMF_STATS":   8,
+}
+
+func (x OfpMeterFlags) String() string {
+	return proto.EnumName(OfpMeterFlags_name, int32(x))
+}
+
+func (OfpMeterFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{29}
+}
+
+// Values for 'type' in ofp_error_message.  These values are immutable: they
+// will not change in future versions of the protocol (although new values may
+// be added).
+type OfpErrorType int32
+
+const (
+	OfpErrorType_OFPET_HELLO_FAILED          OfpErrorType = 0
+	OfpErrorType_OFPET_BAD_REQUEST           OfpErrorType = 1
+	OfpErrorType_OFPET_BAD_ACTION            OfpErrorType = 2
+	OfpErrorType_OFPET_BAD_INSTRUCTION       OfpErrorType = 3
+	OfpErrorType_OFPET_BAD_MATCH             OfpErrorType = 4
+	OfpErrorType_OFPET_FLOW_MOD_FAILED       OfpErrorType = 5
+	OfpErrorType_OFPET_GROUP_MOD_FAILED      OfpErrorType = 6
+	OfpErrorType_OFPET_PORT_MOD_FAILED       OfpErrorType = 7
+	OfpErrorType_OFPET_TABLE_MOD_FAILED      OfpErrorType = 8
+	OfpErrorType_OFPET_QUEUE_OP_FAILED       OfpErrorType = 9
+	OfpErrorType_OFPET_SWITCH_CONFIG_FAILED  OfpErrorType = 10
+	OfpErrorType_OFPET_ROLE_REQUEST_FAILED   OfpErrorType = 11
+	OfpErrorType_OFPET_METER_MOD_FAILED      OfpErrorType = 12
+	OfpErrorType_OFPET_TABLE_FEATURES_FAILED OfpErrorType = 13
+	OfpErrorType_OFPET_EXPERIMENTER          OfpErrorType = 65535
+)
+
+var OfpErrorType_name = map[int32]string{
+	0:     "OFPET_HELLO_FAILED",
+	1:     "OFPET_BAD_REQUEST",
+	2:     "OFPET_BAD_ACTION",
+	3:     "OFPET_BAD_INSTRUCTION",
+	4:     "OFPET_BAD_MATCH",
+	5:     "OFPET_FLOW_MOD_FAILED",
+	6:     "OFPET_GROUP_MOD_FAILED",
+	7:     "OFPET_PORT_MOD_FAILED",
+	8:     "OFPET_TABLE_MOD_FAILED",
+	9:     "OFPET_QUEUE_OP_FAILED",
+	10:    "OFPET_SWITCH_CONFIG_FAILED",
+	11:    "OFPET_ROLE_REQUEST_FAILED",
+	12:    "OFPET_METER_MOD_FAILED",
+	13:    "OFPET_TABLE_FEATURES_FAILED",
+	65535: "OFPET_EXPERIMENTER",
+}
+
+var OfpErrorType_value = map[string]int32{
+	"OFPET_HELLO_FAILED":          0,
+	"OFPET_BAD_REQUEST":           1,
+	"OFPET_BAD_ACTION":            2,
+	"OFPET_BAD_INSTRUCTION":       3,
+	"OFPET_BAD_MATCH":             4,
+	"OFPET_FLOW_MOD_FAILED":       5,
+	"OFPET_GROUP_MOD_FAILED":      6,
+	"OFPET_PORT_MOD_FAILED":       7,
+	"OFPET_TABLE_MOD_FAILED":      8,
+	"OFPET_QUEUE_OP_FAILED":       9,
+	"OFPET_SWITCH_CONFIG_FAILED":  10,
+	"OFPET_ROLE_REQUEST_FAILED":   11,
+	"OFPET_METER_MOD_FAILED":      12,
+	"OFPET_TABLE_FEATURES_FAILED": 13,
+	"OFPET_EXPERIMENTER":          65535,
+}
+
+func (x OfpErrorType) String() string {
+	return proto.EnumName(OfpErrorType_name, int32(x))
+}
+
+func (OfpErrorType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{30}
+}
+
+// ofp_error_msg 'code' values for OFPET_HELLO_FAILED.  'data' contains an
+// ASCII text string that may give failure details.
+type OfpHelloFailedCode int32
+
+const (
+	OfpHelloFailedCode_OFPHFC_INCOMPATIBLE OfpHelloFailedCode = 0
+	OfpHelloFailedCode_OFPHFC_EPERM        OfpHelloFailedCode = 1
+)
+
+var OfpHelloFailedCode_name = map[int32]string{
+	0: "OFPHFC_INCOMPATIBLE",
+	1: "OFPHFC_EPERM",
+}
+
+var OfpHelloFailedCode_value = map[string]int32{
+	"OFPHFC_INCOMPATIBLE": 0,
+	"OFPHFC_EPERM":        1,
+}
+
+func (x OfpHelloFailedCode) String() string {
+	return proto.EnumName(OfpHelloFailedCode_name, int32(x))
+}
+
+func (OfpHelloFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{31}
+}
+
+// ofp_error_msg 'code' values for OFPET_BAD_REQUEST.  'data' contains at least
+// the first 64 bytes of the failed request.
+type OfpBadRequestCode int32
+
+const (
+	OfpBadRequestCode_OFPBRC_BAD_VERSION               OfpBadRequestCode = 0
+	OfpBadRequestCode_OFPBRC_BAD_TYPE                  OfpBadRequestCode = 1
+	OfpBadRequestCode_OFPBRC_BAD_MULTIPART             OfpBadRequestCode = 2
+	OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER          OfpBadRequestCode = 3
+	OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE              OfpBadRequestCode = 4
+	OfpBadRequestCode_OFPBRC_EPERM                     OfpBadRequestCode = 5
+	OfpBadRequestCode_OFPBRC_BAD_LEN                   OfpBadRequestCode = 6
+	OfpBadRequestCode_OFPBRC_BUFFER_EMPTY              OfpBadRequestCode = 7
+	OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN            OfpBadRequestCode = 8
+	OfpBadRequestCode_OFPBRC_BAD_TABLE_ID              OfpBadRequestCode = 9
+	OfpBadRequestCode_OFPBRC_IS_SLAVE                  OfpBadRequestCode = 10
+	OfpBadRequestCode_OFPBRC_BAD_PORT                  OfpBadRequestCode = 11
+	OfpBadRequestCode_OFPBRC_BAD_PACKET                OfpBadRequestCode = 12
+	OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW OfpBadRequestCode = 13
+)
+
+var OfpBadRequestCode_name = map[int32]string{
+	0:  "OFPBRC_BAD_VERSION",
+	1:  "OFPBRC_BAD_TYPE",
+	2:  "OFPBRC_BAD_MULTIPART",
+	3:  "OFPBRC_BAD_EXPERIMENTER",
+	4:  "OFPBRC_BAD_EXP_TYPE",
+	5:  "OFPBRC_EPERM",
+	6:  "OFPBRC_BAD_LEN",
+	7:  "OFPBRC_BUFFER_EMPTY",
+	8:  "OFPBRC_BUFFER_UNKNOWN",
+	9:  "OFPBRC_BAD_TABLE_ID",
+	10: "OFPBRC_IS_SLAVE",
+	11: "OFPBRC_BAD_PORT",
+	12: "OFPBRC_BAD_PACKET",
+	13: "OFPBRC_MULTIPART_BUFFER_OVERFLOW",
+}
+
+var OfpBadRequestCode_value = map[string]int32{
+	"OFPBRC_BAD_VERSION":               0,
+	"OFPBRC_BAD_TYPE":                  1,
+	"OFPBRC_BAD_MULTIPART":             2,
+	"OFPBRC_BAD_EXPERIMENTER":          3,
+	"OFPBRC_BAD_EXP_TYPE":              4,
+	"OFPBRC_EPERM":                     5,
+	"OFPBRC_BAD_LEN":                   6,
+	"OFPBRC_BUFFER_EMPTY":              7,
+	"OFPBRC_BUFFER_UNKNOWN":            8,
+	"OFPBRC_BAD_TABLE_ID":              9,
+	"OFPBRC_IS_SLAVE":                  10,
+	"OFPBRC_BAD_PORT":                  11,
+	"OFPBRC_BAD_PACKET":                12,
+	"OFPBRC_MULTIPART_BUFFER_OVERFLOW": 13,
+}
+
+func (x OfpBadRequestCode) String() string {
+	return proto.EnumName(OfpBadRequestCode_name, int32(x))
+}
+
+func (OfpBadRequestCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{32}
+}
+
+// ofp_error_msg 'code' values for OFPET_BAD_ACTION.  'data' contains at least
+// the first 64 bytes of the failed request.
+type OfpBadActionCode int32
+
+const (
+	OfpBadActionCode_OFPBAC_BAD_TYPE           OfpBadActionCode = 0
+	OfpBadActionCode_OFPBAC_BAD_LEN            OfpBadActionCode = 1
+	OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER   OfpBadActionCode = 2
+	OfpBadActionCode_OFPBAC_BAD_EXP_TYPE       OfpBadActionCode = 3
+	OfpBadActionCode_OFPBAC_BAD_OUT_PORT       OfpBadActionCode = 4
+	OfpBadActionCode_OFPBAC_BAD_ARGUMENT       OfpBadActionCode = 5
+	OfpBadActionCode_OFPBAC_EPERM              OfpBadActionCode = 6
+	OfpBadActionCode_OFPBAC_TOO_MANY           OfpBadActionCode = 7
+	OfpBadActionCode_OFPBAC_BAD_QUEUE          OfpBadActionCode = 8
+	OfpBadActionCode_OFPBAC_BAD_OUT_GROUP      OfpBadActionCode = 9
+	OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT OfpBadActionCode = 10
+	OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER  OfpBadActionCode = 11
+	OfpBadActionCode_OFPBAC_BAD_TAG            OfpBadActionCode = 12
+	OfpBadActionCode_OFPBAC_BAD_SET_TYPE       OfpBadActionCode = 13
+	OfpBadActionCode_OFPBAC_BAD_SET_LEN        OfpBadActionCode = 14
+	OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT   OfpBadActionCode = 15
+)
+
+var OfpBadActionCode_name = map[int32]string{
+	0:  "OFPBAC_BAD_TYPE",
+	1:  "OFPBAC_BAD_LEN",
+	2:  "OFPBAC_BAD_EXPERIMENTER",
+	3:  "OFPBAC_BAD_EXP_TYPE",
+	4:  "OFPBAC_BAD_OUT_PORT",
+	5:  "OFPBAC_BAD_ARGUMENT",
+	6:  "OFPBAC_EPERM",
+	7:  "OFPBAC_TOO_MANY",
+	8:  "OFPBAC_BAD_QUEUE",
+	9:  "OFPBAC_BAD_OUT_GROUP",
+	10: "OFPBAC_MATCH_INCONSISTENT",
+	11: "OFPBAC_UNSUPPORTED_ORDER",
+	12: "OFPBAC_BAD_TAG",
+	13: "OFPBAC_BAD_SET_TYPE",
+	14: "OFPBAC_BAD_SET_LEN",
+	15: "OFPBAC_BAD_SET_ARGUMENT",
+}
+
+var OfpBadActionCode_value = map[string]int32{
+	"OFPBAC_BAD_TYPE":           0,
+	"OFPBAC_BAD_LEN":            1,
+	"OFPBAC_BAD_EXPERIMENTER":   2,
+	"OFPBAC_BAD_EXP_TYPE":       3,
+	"OFPBAC_BAD_OUT_PORT":       4,
+	"OFPBAC_BAD_ARGUMENT":       5,
+	"OFPBAC_EPERM":              6,
+	"OFPBAC_TOO_MANY":           7,
+	"OFPBAC_BAD_QUEUE":          8,
+	"OFPBAC_BAD_OUT_GROUP":      9,
+	"OFPBAC_MATCH_INCONSISTENT": 10,
+	"OFPBAC_UNSUPPORTED_ORDER":  11,
+	"OFPBAC_BAD_TAG":            12,
+	"OFPBAC_BAD_SET_TYPE":       13,
+	"OFPBAC_BAD_SET_LEN":        14,
+	"OFPBAC_BAD_SET_ARGUMENT":   15,
+}
+
+func (x OfpBadActionCode) String() string {
+	return proto.EnumName(OfpBadActionCode_name, int32(x))
+}
+
+func (OfpBadActionCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{33}
+}
+
+// ofp_error_msg 'code' values for OFPET_BAD_INSTRUCTION.  'data' contains at
+// least the first 64 bytes of the failed request.
+type OfpBadInstructionCode int32
+
+const (
+	OfpBadInstructionCode_OFPBIC_UNKNOWN_INST        OfpBadInstructionCode = 0
+	OfpBadInstructionCode_OFPBIC_UNSUP_INST          OfpBadInstructionCode = 1
+	OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID        OfpBadInstructionCode = 2
+	OfpBadInstructionCode_OFPBIC_UNSUP_METADATA      OfpBadInstructionCode = 3
+	OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK OfpBadInstructionCode = 4
+	OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER    OfpBadInstructionCode = 5
+	OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE        OfpBadInstructionCode = 6
+	OfpBadInstructionCode_OFPBIC_BAD_LEN             OfpBadInstructionCode = 7
+	OfpBadInstructionCode_OFPBIC_EPERM               OfpBadInstructionCode = 8
+)
+
+var OfpBadInstructionCode_name = map[int32]string{
+	0: "OFPBIC_UNKNOWN_INST",
+	1: "OFPBIC_UNSUP_INST",
+	2: "OFPBIC_BAD_TABLE_ID",
+	3: "OFPBIC_UNSUP_METADATA",
+	4: "OFPBIC_UNSUP_METADATA_MASK",
+	5: "OFPBIC_BAD_EXPERIMENTER",
+	6: "OFPBIC_BAD_EXP_TYPE",
+	7: "OFPBIC_BAD_LEN",
+	8: "OFPBIC_EPERM",
+}
+
+var OfpBadInstructionCode_value = map[string]int32{
+	"OFPBIC_UNKNOWN_INST":        0,
+	"OFPBIC_UNSUP_INST":          1,
+	"OFPBIC_BAD_TABLE_ID":        2,
+	"OFPBIC_UNSUP_METADATA":      3,
+	"OFPBIC_UNSUP_METADATA_MASK": 4,
+	"OFPBIC_BAD_EXPERIMENTER":    5,
+	"OFPBIC_BAD_EXP_TYPE":        6,
+	"OFPBIC_BAD_LEN":             7,
+	"OFPBIC_EPERM":               8,
+}
+
+func (x OfpBadInstructionCode) String() string {
+	return proto.EnumName(OfpBadInstructionCode_name, int32(x))
+}
+
+func (OfpBadInstructionCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{34}
+}
+
+// ofp_error_msg 'code' values for OFPET_BAD_MATCH.  'data' contains at least
+// the first 64 bytes of the failed request.
+type OfpBadMatchCode int32
+
+const (
+	OfpBadMatchCode_OFPBMC_BAD_TYPE         OfpBadMatchCode = 0
+	OfpBadMatchCode_OFPBMC_BAD_LEN          OfpBadMatchCode = 1
+	OfpBadMatchCode_OFPBMC_BAD_TAG          OfpBadMatchCode = 2
+	OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK OfpBadMatchCode = 3
+	OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK OfpBadMatchCode = 4
+	OfpBadMatchCode_OFPBMC_BAD_WILDCARDS    OfpBadMatchCode = 5
+	OfpBadMatchCode_OFPBMC_BAD_FIELD        OfpBadMatchCode = 6
+	OfpBadMatchCode_OFPBMC_BAD_VALUE        OfpBadMatchCode = 7
+	OfpBadMatchCode_OFPBMC_BAD_MASK         OfpBadMatchCode = 8
+	OfpBadMatchCode_OFPBMC_BAD_PREREQ       OfpBadMatchCode = 9
+	OfpBadMatchCode_OFPBMC_DUP_FIELD        OfpBadMatchCode = 10
+	OfpBadMatchCode_OFPBMC_EPERM            OfpBadMatchCode = 11
+)
+
+var OfpBadMatchCode_name = map[int32]string{
+	0:  "OFPBMC_BAD_TYPE",
+	1:  "OFPBMC_BAD_LEN",
+	2:  "OFPBMC_BAD_TAG",
+	3:  "OFPBMC_BAD_DL_ADDR_MASK",
+	4:  "OFPBMC_BAD_NW_ADDR_MASK",
+	5:  "OFPBMC_BAD_WILDCARDS",
+	6:  "OFPBMC_BAD_FIELD",
+	7:  "OFPBMC_BAD_VALUE",
+	8:  "OFPBMC_BAD_MASK",
+	9:  "OFPBMC_BAD_PREREQ",
+	10: "OFPBMC_DUP_FIELD",
+	11: "OFPBMC_EPERM",
+}
+
+var OfpBadMatchCode_value = map[string]int32{
+	"OFPBMC_BAD_TYPE":         0,
+	"OFPBMC_BAD_LEN":          1,
+	"OFPBMC_BAD_TAG":          2,
+	"OFPBMC_BAD_DL_ADDR_MASK": 3,
+	"OFPBMC_BAD_NW_ADDR_MASK": 4,
+	"OFPBMC_BAD_WILDCARDS":    5,
+	"OFPBMC_BAD_FIELD":        6,
+	"OFPBMC_BAD_VALUE":        7,
+	"OFPBMC_BAD_MASK":         8,
+	"OFPBMC_BAD_PREREQ":       9,
+	"OFPBMC_DUP_FIELD":        10,
+	"OFPBMC_EPERM":            11,
+}
+
+func (x OfpBadMatchCode) String() string {
+	return proto.EnumName(OfpBadMatchCode_name, int32(x))
+}
+
+func (OfpBadMatchCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{35}
+}
+
+// ofp_error_msg 'code' values for OFPET_FLOW_MOD_FAILED.  'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpFlowModFailedCode int32
+
+const (
+	OfpFlowModFailedCode_OFPFMFC_UNKNOWN      OfpFlowModFailedCode = 0
+	OfpFlowModFailedCode_OFPFMFC_TABLE_FULL   OfpFlowModFailedCode = 1
+	OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID OfpFlowModFailedCode = 2
+	OfpFlowModFailedCode_OFPFMFC_OVERLAP      OfpFlowModFailedCode = 3
+	OfpFlowModFailedCode_OFPFMFC_EPERM        OfpFlowModFailedCode = 4
+	OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT  OfpFlowModFailedCode = 5
+	OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND  OfpFlowModFailedCode = 6
+	OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS    OfpFlowModFailedCode = 7
+)
+
+var OfpFlowModFailedCode_name = map[int32]string{
+	0: "OFPFMFC_UNKNOWN",
+	1: "OFPFMFC_TABLE_FULL",
+	2: "OFPFMFC_BAD_TABLE_ID",
+	3: "OFPFMFC_OVERLAP",
+	4: "OFPFMFC_EPERM",
+	5: "OFPFMFC_BAD_TIMEOUT",
+	6: "OFPFMFC_BAD_COMMAND",
+	7: "OFPFMFC_BAD_FLAGS",
+}
+
+var OfpFlowModFailedCode_value = map[string]int32{
+	"OFPFMFC_UNKNOWN":      0,
+	"OFPFMFC_TABLE_FULL":   1,
+	"OFPFMFC_BAD_TABLE_ID": 2,
+	"OFPFMFC_OVERLAP":      3,
+	"OFPFMFC_EPERM":        4,
+	"OFPFMFC_BAD_TIMEOUT":  5,
+	"OFPFMFC_BAD_COMMAND":  6,
+	"OFPFMFC_BAD_FLAGS":    7,
+}
+
+func (x OfpFlowModFailedCode) String() string {
+	return proto.EnumName(OfpFlowModFailedCode_name, int32(x))
+}
+
+func (OfpFlowModFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{36}
+}
+
+// ofp_error_msg 'code' values for OFPET_GROUP_MOD_FAILED.  'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpGroupModFailedCode int32
+
+const (
+	OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS         OfpGroupModFailedCode = 0
+	OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP        OfpGroupModFailedCode = 1
+	OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED   OfpGroupModFailedCode = 2
+	OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS        OfpGroupModFailedCode = 3
+	OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS       OfpGroupModFailedCode = 4
+	OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED OfpGroupModFailedCode = 5
+	OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED    OfpGroupModFailedCode = 6
+	OfpGroupModFailedCode_OFPGMFC_LOOP                 OfpGroupModFailedCode = 7
+	OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP        OfpGroupModFailedCode = 8
+	OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP        OfpGroupModFailedCode = 9
+	OfpGroupModFailedCode_OFPGMFC_BAD_TYPE             OfpGroupModFailedCode = 10
+	OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND          OfpGroupModFailedCode = 11
+	OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET           OfpGroupModFailedCode = 12
+	OfpGroupModFailedCode_OFPGMFC_BAD_WATCH            OfpGroupModFailedCode = 13
+	OfpGroupModFailedCode_OFPGMFC_EPERM                OfpGroupModFailedCode = 14
+)
+
+var OfpGroupModFailedCode_name = map[int32]string{
+	0:  "OFPGMFC_GROUP_EXISTS",
+	1:  "OFPGMFC_INVALID_GROUP",
+	2:  "OFPGMFC_WEIGHT_UNSUPPORTED",
+	3:  "OFPGMFC_OUT_OF_GROUPS",
+	4:  "OFPGMFC_OUT_OF_BUCKETS",
+	5:  "OFPGMFC_CHAINING_UNSUPPORTED",
+	6:  "OFPGMFC_WATCH_UNSUPPORTED",
+	7:  "OFPGMFC_LOOP",
+	8:  "OFPGMFC_UNKNOWN_GROUP",
+	9:  "OFPGMFC_CHAINED_GROUP",
+	10: "OFPGMFC_BAD_TYPE",
+	11: "OFPGMFC_BAD_COMMAND",
+	12: "OFPGMFC_BAD_BUCKET",
+	13: "OFPGMFC_BAD_WATCH",
+	14: "OFPGMFC_EPERM",
+}
+
+var OfpGroupModFailedCode_value = map[string]int32{
+	"OFPGMFC_GROUP_EXISTS":         0,
+	"OFPGMFC_INVALID_GROUP":        1,
+	"OFPGMFC_WEIGHT_UNSUPPORTED":   2,
+	"OFPGMFC_OUT_OF_GROUPS":        3,
+	"OFPGMFC_OUT_OF_BUCKETS":       4,
+	"OFPGMFC_CHAINING_UNSUPPORTED": 5,
+	"OFPGMFC_WATCH_UNSUPPORTED":    6,
+	"OFPGMFC_LOOP":                 7,
+	"OFPGMFC_UNKNOWN_GROUP":        8,
+	"OFPGMFC_CHAINED_GROUP":        9,
+	"OFPGMFC_BAD_TYPE":             10,
+	"OFPGMFC_BAD_COMMAND":          11,
+	"OFPGMFC_BAD_BUCKET":           12,
+	"OFPGMFC_BAD_WATCH":            13,
+	"OFPGMFC_EPERM":                14,
+}
+
+func (x OfpGroupModFailedCode) String() string {
+	return proto.EnumName(OfpGroupModFailedCode_name, int32(x))
+}
+
+func (OfpGroupModFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{37}
+}
+
+// ofp_error_msg 'code' values for OFPET_PORT_MOD_FAILED.  'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpPortModFailedCode int32
+
+const (
+	OfpPortModFailedCode_OFPPMFC_BAD_PORT      OfpPortModFailedCode = 0
+	OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR   OfpPortModFailedCode = 1
+	OfpPortModFailedCode_OFPPMFC_BAD_CONFIG    OfpPortModFailedCode = 2
+	OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE OfpPortModFailedCode = 3
+	OfpPortModFailedCode_OFPPMFC_EPERM         OfpPortModFailedCode = 4
+)
+
+var OfpPortModFailedCode_name = map[int32]string{
+	0: "OFPPMFC_BAD_PORT",
+	1: "OFPPMFC_BAD_HW_ADDR",
+	2: "OFPPMFC_BAD_CONFIG",
+	3: "OFPPMFC_BAD_ADVERTISE",
+	4: "OFPPMFC_EPERM",
+}
+
+var OfpPortModFailedCode_value = map[string]int32{
+	"OFPPMFC_BAD_PORT":      0,
+	"OFPPMFC_BAD_HW_ADDR":   1,
+	"OFPPMFC_BAD_CONFIG":    2,
+	"OFPPMFC_BAD_ADVERTISE": 3,
+	"OFPPMFC_EPERM":         4,
+}
+
+func (x OfpPortModFailedCode) String() string {
+	return proto.EnumName(OfpPortModFailedCode_name, int32(x))
+}
+
+func (OfpPortModFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{38}
+}
+
+// ofp_error_msg 'code' values for OFPET_TABLE_MOD_FAILED.  'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpTableModFailedCode int32
+
+const (
+	OfpTableModFailedCode_OFPTMFC_BAD_TABLE  OfpTableModFailedCode = 0
+	OfpTableModFailedCode_OFPTMFC_BAD_CONFIG OfpTableModFailedCode = 1
+	OfpTableModFailedCode_OFPTMFC_EPERM      OfpTableModFailedCode = 2
+)
+
+var OfpTableModFailedCode_name = map[int32]string{
+	0: "OFPTMFC_BAD_TABLE",
+	1: "OFPTMFC_BAD_CONFIG",
+	2: "OFPTMFC_EPERM",
+}
+
+var OfpTableModFailedCode_value = map[string]int32{
+	"OFPTMFC_BAD_TABLE":  0,
+	"OFPTMFC_BAD_CONFIG": 1,
+	"OFPTMFC_EPERM":      2,
+}
+
+func (x OfpTableModFailedCode) String() string {
+	return proto.EnumName(OfpTableModFailedCode_name, int32(x))
+}
+
+func (OfpTableModFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{39}
+}
+
+// ofp_error msg 'code' values for OFPET_QUEUE_OP_FAILED. 'data' contains
+// at least the first 64 bytes of the failed request
+type OfpQueueOpFailedCode int32
+
+const (
+	OfpQueueOpFailedCode_OFPQOFC_BAD_PORT  OfpQueueOpFailedCode = 0
+	OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE OfpQueueOpFailedCode = 1
+	OfpQueueOpFailedCode_OFPQOFC_EPERM     OfpQueueOpFailedCode = 2
+)
+
+var OfpQueueOpFailedCode_name = map[int32]string{
+	0: "OFPQOFC_BAD_PORT",
+	1: "OFPQOFC_BAD_QUEUE",
+	2: "OFPQOFC_EPERM",
+}
+
+var OfpQueueOpFailedCode_value = map[string]int32{
+	"OFPQOFC_BAD_PORT":  0,
+	"OFPQOFC_BAD_QUEUE": 1,
+	"OFPQOFC_EPERM":     2,
+}
+
+func (x OfpQueueOpFailedCode) String() string {
+	return proto.EnumName(OfpQueueOpFailedCode_name, int32(x))
+}
+
+func (OfpQueueOpFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{40}
+}
+
+// ofp_error_msg 'code' values for OFPET_SWITCH_CONFIG_FAILED. 'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpSwitchConfigFailedCode int32
+
+const (
+	OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS OfpSwitchConfigFailedCode = 0
+	OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN   OfpSwitchConfigFailedCode = 1
+	OfpSwitchConfigFailedCode_OFPSCFC_EPERM     OfpSwitchConfigFailedCode = 2
+)
+
+var OfpSwitchConfigFailedCode_name = map[int32]string{
+	0: "OFPSCFC_BAD_FLAGS",
+	1: "OFPSCFC_BAD_LEN",
+	2: "OFPSCFC_EPERM",
+}
+
+var OfpSwitchConfigFailedCode_value = map[string]int32{
+	"OFPSCFC_BAD_FLAGS": 0,
+	"OFPSCFC_BAD_LEN":   1,
+	"OFPSCFC_EPERM":     2,
+}
+
+func (x OfpSwitchConfigFailedCode) String() string {
+	return proto.EnumName(OfpSwitchConfigFailedCode_name, int32(x))
+}
+
+func (OfpSwitchConfigFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{41}
+}
+
+// ofp_error_msg 'code' values for OFPET_ROLE_REQUEST_FAILED. 'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpRoleRequestFailedCode int32
+
+const (
+	OfpRoleRequestFailedCode_OFPRRFC_STALE    OfpRoleRequestFailedCode = 0
+	OfpRoleRequestFailedCode_OFPRRFC_UNSUP    OfpRoleRequestFailedCode = 1
+	OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE OfpRoleRequestFailedCode = 2
+)
+
+var OfpRoleRequestFailedCode_name = map[int32]string{
+	0: "OFPRRFC_STALE",
+	1: "OFPRRFC_UNSUP",
+	2: "OFPRRFC_BAD_ROLE",
+}
+
+var OfpRoleRequestFailedCode_value = map[string]int32{
+	"OFPRRFC_STALE":    0,
+	"OFPRRFC_UNSUP":    1,
+	"OFPRRFC_BAD_ROLE": 2,
+}
+
+func (x OfpRoleRequestFailedCode) String() string {
+	return proto.EnumName(OfpRoleRequestFailedCode_name, int32(x))
+}
+
+func (OfpRoleRequestFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{42}
+}
+
+// ofp_error_msg 'code' values for OFPET_METER_MOD_FAILED.  'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpMeterModFailedCode int32
+
+const (
+	OfpMeterModFailedCode_OFPMMFC_UNKNOWN         OfpMeterModFailedCode = 0
+	OfpMeterModFailedCode_OFPMMFC_METER_EXISTS    OfpMeterModFailedCode = 1
+	OfpMeterModFailedCode_OFPMMFC_INVALID_METER   OfpMeterModFailedCode = 2
+	OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER   OfpMeterModFailedCode = 3
+	OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND     OfpMeterModFailedCode = 4
+	OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS       OfpMeterModFailedCode = 5
+	OfpMeterModFailedCode_OFPMMFC_BAD_RATE        OfpMeterModFailedCode = 6
+	OfpMeterModFailedCode_OFPMMFC_BAD_BURST       OfpMeterModFailedCode = 7
+	OfpMeterModFailedCode_OFPMMFC_BAD_BAND        OfpMeterModFailedCode = 8
+	OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL OfpMeterModFailedCode = 9
+	OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS   OfpMeterModFailedCode = 10
+	OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS    OfpMeterModFailedCode = 11
+)
+
+var OfpMeterModFailedCode_name = map[int32]string{
+	0:  "OFPMMFC_UNKNOWN",
+	1:  "OFPMMFC_METER_EXISTS",
+	2:  "OFPMMFC_INVALID_METER",
+	3:  "OFPMMFC_UNKNOWN_METER",
+	4:  "OFPMMFC_BAD_COMMAND",
+	5:  "OFPMMFC_BAD_FLAGS",
+	6:  "OFPMMFC_BAD_RATE",
+	7:  "OFPMMFC_BAD_BURST",
+	8:  "OFPMMFC_BAD_BAND",
+	9:  "OFPMMFC_BAD_BAND_DETAIL",
+	10: "OFPMMFC_OUT_OF_METERS",
+	11: "OFPMMFC_OUT_OF_BANDS",
+}
+
+var OfpMeterModFailedCode_value = map[string]int32{
+	"OFPMMFC_UNKNOWN":         0,
+	"OFPMMFC_METER_EXISTS":    1,
+	"OFPMMFC_INVALID_METER":   2,
+	"OFPMMFC_UNKNOWN_METER":   3,
+	"OFPMMFC_BAD_COMMAND":     4,
+	"OFPMMFC_BAD_FLAGS":       5,
+	"OFPMMFC_BAD_RATE":        6,
+	"OFPMMFC_BAD_BURST":       7,
+	"OFPMMFC_BAD_BAND":        8,
+	"OFPMMFC_BAD_BAND_DETAIL": 9,
+	"OFPMMFC_OUT_OF_METERS":   10,
+	"OFPMMFC_OUT_OF_BANDS":    11,
+}
+
+func (x OfpMeterModFailedCode) String() string {
+	return proto.EnumName(OfpMeterModFailedCode_name, int32(x))
+}
+
+func (OfpMeterModFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{43}
+}
+
+// ofp_error_msg 'code' values for OFPET_TABLE_FEATURES_FAILED. 'data' contains
+// at least the first 64 bytes of the failed request.
+type OfpTableFeaturesFailedCode int32
+
+const (
+	OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE    OfpTableFeaturesFailedCode = 0
+	OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA OfpTableFeaturesFailedCode = 1
+	OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE     OfpTableFeaturesFailedCode = 2
+	OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN      OfpTableFeaturesFailedCode = 3
+	OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT OfpTableFeaturesFailedCode = 4
+	OfpTableFeaturesFailedCode_OFPTFFC_EPERM        OfpTableFeaturesFailedCode = 5
+)
+
+var OfpTableFeaturesFailedCode_name = map[int32]string{
+	0: "OFPTFFC_BAD_TABLE",
+	1: "OFPTFFC_BAD_METADATA",
+	2: "OFPTFFC_BAD_TYPE",
+	3: "OFPTFFC_BAD_LEN",
+	4: "OFPTFFC_BAD_ARGUMENT",
+	5: "OFPTFFC_EPERM",
+}
+
+var OfpTableFeaturesFailedCode_value = map[string]int32{
+	"OFPTFFC_BAD_TABLE":    0,
+	"OFPTFFC_BAD_METADATA": 1,
+	"OFPTFFC_BAD_TYPE":     2,
+	"OFPTFFC_BAD_LEN":      3,
+	"OFPTFFC_BAD_ARGUMENT": 4,
+	"OFPTFFC_EPERM":        5,
+}
+
+func (x OfpTableFeaturesFailedCode) String() string {
+	return proto.EnumName(OfpTableFeaturesFailedCode_name, int32(x))
+}
+
+func (OfpTableFeaturesFailedCode) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{44}
+}
+
+type OfpMultipartType int32
+
+const (
+	// Description of this OpenFlow switch.
+	// The request body is empty.
+	// The reply body is struct ofp_desc.
+	OfpMultipartType_OFPMP_DESC OfpMultipartType = 0
+	// Individual flow statistics.
+	// The request body is struct ofp_flow_stats_request.
+	// The reply body is an array of struct ofp_flow_stats.
+	OfpMultipartType_OFPMP_FLOW OfpMultipartType = 1
+	// Aggregate flow statistics.
+	// The request body is struct ofp_aggregate_stats_request.
+	// The reply body is struct ofp_aggregate_stats_reply.
+	OfpMultipartType_OFPMP_AGGREGATE OfpMultipartType = 2
+	// Flow table statistics.
+	// The request body is empty.
+	// The reply body is an array of struct ofp_table_stats.
+	OfpMultipartType_OFPMP_TABLE OfpMultipartType = 3
+	// Port statistics.
+	// The request body is struct ofp_port_stats_request.
+	// The reply body is an array of struct ofp_port_stats.
+	OfpMultipartType_OFPMP_PORT_STATS OfpMultipartType = 4
+	// Queue statistics for a port
+	// The request body is struct ofp_queue_stats_request.
+	// The reply body is an array of struct ofp_queue_stats
+	OfpMultipartType_OFPMP_QUEUE OfpMultipartType = 5
+	// Group counter statistics.
+	// The request body is struct ofp_group_stats_request.
+	// The reply is an array of struct ofp_group_stats.
+	OfpMultipartType_OFPMP_GROUP OfpMultipartType = 6
+	// Group description.
+	// The request body is empty.
+	// The reply body is an array of struct ofp_group_desc.
+	OfpMultipartType_OFPMP_GROUP_DESC OfpMultipartType = 7
+	// Group features.
+	// The request body is empty.
+	// The reply body is struct ofp_group_features.
+	OfpMultipartType_OFPMP_GROUP_FEATURES OfpMultipartType = 8
+	// Meter statistics.
+	// The request body is struct ofp_meter_multipart_requests.
+	// The reply body is an array of struct ofp_meter_stats.
+	OfpMultipartType_OFPMP_METER OfpMultipartType = 9
+	// Meter configuration.
+	// The request body is struct ofp_meter_multipart_requests.
+	// The reply body is an array of struct ofp_meter_config.
+	OfpMultipartType_OFPMP_METER_CONFIG OfpMultipartType = 10
+	// Meter features.
+	// The request body is empty.
+	// The reply body is struct ofp_meter_features.
+	OfpMultipartType_OFPMP_METER_FEATURES OfpMultipartType = 11
+	// Table features.
+	// The request body is either empty or contains an array of
+	// struct ofp_table_features containing the controller's
+	// desired view of the switch. If the switch is unable to
+	// set the specified view an error is returned.
+	// The reply body is an array of struct ofp_table_features.
+	OfpMultipartType_OFPMP_TABLE_FEATURES OfpMultipartType = 12
+	// Port description.
+	// The request body is empty.
+	// The reply body is an array of struct ofp_port.
+	OfpMultipartType_OFPMP_PORT_DESC OfpMultipartType = 13
+	// Experimenter extension.
+	// The request and reply bodies begin with
+	// struct ofp_experimenter_multipart_header.
+	// The request and reply bodies are otherwise experimenter-defined.
+	OfpMultipartType_OFPMP_EXPERIMENTER OfpMultipartType = 65535
+)
+
+var OfpMultipartType_name = map[int32]string{
+	0:     "OFPMP_DESC",
+	1:     "OFPMP_FLOW",
+	2:     "OFPMP_AGGREGATE",
+	3:     "OFPMP_TABLE",
+	4:     "OFPMP_PORT_STATS",
+	5:     "OFPMP_QUEUE",
+	6:     "OFPMP_GROUP",
+	7:     "OFPMP_GROUP_DESC",
+	8:     "OFPMP_GROUP_FEATURES",
+	9:     "OFPMP_METER",
+	10:    "OFPMP_METER_CONFIG",
+	11:    "OFPMP_METER_FEATURES",
+	12:    "OFPMP_TABLE_FEATURES",
+	13:    "OFPMP_PORT_DESC",
+	65535: "OFPMP_EXPERIMENTER",
+}
+
+var OfpMultipartType_value = map[string]int32{
+	"OFPMP_DESC":           0,
+	"OFPMP_FLOW":           1,
+	"OFPMP_AGGREGATE":      2,
+	"OFPMP_TABLE":          3,
+	"OFPMP_PORT_STATS":     4,
+	"OFPMP_QUEUE":          5,
+	"OFPMP_GROUP":          6,
+	"OFPMP_GROUP_DESC":     7,
+	"OFPMP_GROUP_FEATURES": 8,
+	"OFPMP_METER":          9,
+	"OFPMP_METER_CONFIG":   10,
+	"OFPMP_METER_FEATURES": 11,
+	"OFPMP_TABLE_FEATURES": 12,
+	"OFPMP_PORT_DESC":      13,
+	"OFPMP_EXPERIMENTER":   65535,
+}
+
+func (x OfpMultipartType) String() string {
+	return proto.EnumName(OfpMultipartType_name, int32(x))
+}
+
+func (OfpMultipartType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{45}
+}
+
+type OfpMultipartRequestFlags int32
+
+const (
+	OfpMultipartRequestFlags_OFPMPF_REQ_INVALID OfpMultipartRequestFlags = 0
+	OfpMultipartRequestFlags_OFPMPF_REQ_MORE    OfpMultipartRequestFlags = 1
+)
+
+var OfpMultipartRequestFlags_name = map[int32]string{
+	0: "OFPMPF_REQ_INVALID",
+	1: "OFPMPF_REQ_MORE",
+}
+
+var OfpMultipartRequestFlags_value = map[string]int32{
+	"OFPMPF_REQ_INVALID": 0,
+	"OFPMPF_REQ_MORE":    1,
+}
+
+func (x OfpMultipartRequestFlags) String() string {
+	return proto.EnumName(OfpMultipartRequestFlags_name, int32(x))
+}
+
+func (OfpMultipartRequestFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{46}
+}
+
+type OfpMultipartReplyFlags int32
+
+const (
+	OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID OfpMultipartReplyFlags = 0
+	OfpMultipartReplyFlags_OFPMPF_REPLY_MORE    OfpMultipartReplyFlags = 1
+)
+
+var OfpMultipartReplyFlags_name = map[int32]string{
+	0: "OFPMPF_REPLY_INVALID",
+	1: "OFPMPF_REPLY_MORE",
+}
+
+var OfpMultipartReplyFlags_value = map[string]int32{
+	"OFPMPF_REPLY_INVALID": 0,
+	"OFPMPF_REPLY_MORE":    1,
+}
+
+func (x OfpMultipartReplyFlags) String() string {
+	return proto.EnumName(OfpMultipartReplyFlags_name, int32(x))
+}
+
+func (OfpMultipartReplyFlags) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{47}
+}
+
+// Table Feature property types.
+// Low order bit cleared indicates a property for a regular Flow Entry.
+// Low order bit set indicates a property for the Table-Miss Flow Entry.
+type OfpTableFeaturePropType int32
+
+const (
+	OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS        OfpTableFeaturePropType = 0
+	OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS   OfpTableFeaturePropType = 1
+	OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES         OfpTableFeaturePropType = 2
+	OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS    OfpTableFeaturePropType = 3
+	OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS       OfpTableFeaturePropType = 4
+	OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS  OfpTableFeaturePropType = 5
+	OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS       OfpTableFeaturePropType = 6
+	OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS  OfpTableFeaturePropType = 7
+	OfpTableFeaturePropType_OFPTFPT_MATCH               OfpTableFeaturePropType = 8
+	OfpTableFeaturePropType_OFPTFPT_WILDCARDS           OfpTableFeaturePropType = 10
+	OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD      OfpTableFeaturePropType = 12
+	OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS OfpTableFeaturePropType = 13
+	OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD      OfpTableFeaturePropType = 14
+	OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS OfpTableFeaturePropType = 15
+	OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER        OfpTableFeaturePropType = 65534
+	OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS   OfpTableFeaturePropType = 65535
+)
+
+var OfpTableFeaturePropType_name = map[int32]string{
+	0:     "OFPTFPT_INSTRUCTIONS",
+	1:     "OFPTFPT_INSTRUCTIONS_MISS",
+	2:     "OFPTFPT_NEXT_TABLES",
+	3:     "OFPTFPT_NEXT_TABLES_MISS",
+	4:     "OFPTFPT_WRITE_ACTIONS",
+	5:     "OFPTFPT_WRITE_ACTIONS_MISS",
+	6:     "OFPTFPT_APPLY_ACTIONS",
+	7:     "OFPTFPT_APPLY_ACTIONS_MISS",
+	8:     "OFPTFPT_MATCH",
+	10:    "OFPTFPT_WILDCARDS",
+	12:    "OFPTFPT_WRITE_SETFIELD",
+	13:    "OFPTFPT_WRITE_SETFIELD_MISS",
+	14:    "OFPTFPT_APPLY_SETFIELD",
+	15:    "OFPTFPT_APPLY_SETFIELD_MISS",
+	65534: "OFPTFPT_EXPERIMENTER",
+	65535: "OFPTFPT_EXPERIMENTER_MISS",
+}
+
+var OfpTableFeaturePropType_value = map[string]int32{
+	"OFPTFPT_INSTRUCTIONS":        0,
+	"OFPTFPT_INSTRUCTIONS_MISS":   1,
+	"OFPTFPT_NEXT_TABLES":         2,
+	"OFPTFPT_NEXT_TABLES_MISS":    3,
+	"OFPTFPT_WRITE_ACTIONS":       4,
+	"OFPTFPT_WRITE_ACTIONS_MISS":  5,
+	"OFPTFPT_APPLY_ACTIONS":       6,
+	"OFPTFPT_APPLY_ACTIONS_MISS":  7,
+	"OFPTFPT_MATCH":               8,
+	"OFPTFPT_WILDCARDS":           10,
+	"OFPTFPT_WRITE_SETFIELD":      12,
+	"OFPTFPT_WRITE_SETFIELD_MISS": 13,
+	"OFPTFPT_APPLY_SETFIELD":      14,
+	"OFPTFPT_APPLY_SETFIELD_MISS": 15,
+	"OFPTFPT_EXPERIMENTER":        65534,
+	"OFPTFPT_EXPERIMENTER_MISS":   65535,
+}
+
+func (x OfpTableFeaturePropType) String() string {
+	return proto.EnumName(OfpTableFeaturePropType_name, int32(x))
+}
+
+func (OfpTableFeaturePropType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{48}
+}
+
+// Group configuration flags
+type OfpGroupCapabilities int32
+
+const (
+	OfpGroupCapabilities_OFPGFC_INVALID         OfpGroupCapabilities = 0
+	OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT   OfpGroupCapabilities = 1
+	OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS OfpGroupCapabilities = 2
+	OfpGroupCapabilities_OFPGFC_CHAINING        OfpGroupCapabilities = 4
+	OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS OfpGroupCapabilities = 8
+)
+
+var OfpGroupCapabilities_name = map[int32]string{
+	0: "OFPGFC_INVALID",
+	1: "OFPGFC_SELECT_WEIGHT",
+	2: "OFPGFC_SELECT_LIVENESS",
+	4: "OFPGFC_CHAINING",
+	8: "OFPGFC_CHAINING_CHECKS",
+}
+
+var OfpGroupCapabilities_value = map[string]int32{
+	"OFPGFC_INVALID":         0,
+	"OFPGFC_SELECT_WEIGHT":   1,
+	"OFPGFC_SELECT_LIVENESS": 2,
+	"OFPGFC_CHAINING":        4,
+	"OFPGFC_CHAINING_CHECKS": 8,
+}
+
+func (x OfpGroupCapabilities) String() string {
+	return proto.EnumName(OfpGroupCapabilities_name, int32(x))
+}
+
+func (OfpGroupCapabilities) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{49}
+}
+
+type OfpQueueProperties int32
+
+const (
+	OfpQueueProperties_OFPQT_INVALID      OfpQueueProperties = 0
+	OfpQueueProperties_OFPQT_MIN_RATE     OfpQueueProperties = 1
+	OfpQueueProperties_OFPQT_MAX_RATE     OfpQueueProperties = 2
+	OfpQueueProperties_OFPQT_EXPERIMENTER OfpQueueProperties = 65535
+)
+
+var OfpQueueProperties_name = map[int32]string{
+	0:     "OFPQT_INVALID",
+	1:     "OFPQT_MIN_RATE",
+	2:     "OFPQT_MAX_RATE",
+	65535: "OFPQT_EXPERIMENTER",
+}
+
+var OfpQueueProperties_value = map[string]int32{
+	"OFPQT_INVALID":      0,
+	"OFPQT_MIN_RATE":     1,
+	"OFPQT_MAX_RATE":     2,
+	"OFPQT_EXPERIMENTER": 65535,
+}
+
+func (x OfpQueueProperties) String() string {
+	return proto.EnumName(OfpQueueProperties_name, int32(x))
+}
+
+func (OfpQueueProperties) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{50}
+}
+
+// Controller roles.
+type OfpControllerRole int32
+
+const (
+	OfpControllerRole_OFPCR_ROLE_NOCHANGE OfpControllerRole = 0
+	OfpControllerRole_OFPCR_ROLE_EQUAL    OfpControllerRole = 1
+	OfpControllerRole_OFPCR_ROLE_MASTER   OfpControllerRole = 2
+	OfpControllerRole_OFPCR_ROLE_SLAVE    OfpControllerRole = 3
+)
+
+var OfpControllerRole_name = map[int32]string{
+	0: "OFPCR_ROLE_NOCHANGE",
+	1: "OFPCR_ROLE_EQUAL",
+	2: "OFPCR_ROLE_MASTER",
+	3: "OFPCR_ROLE_SLAVE",
+}
+
+var OfpControllerRole_value = map[string]int32{
+	"OFPCR_ROLE_NOCHANGE": 0,
+	"OFPCR_ROLE_EQUAL":    1,
+	"OFPCR_ROLE_MASTER":   2,
+	"OFPCR_ROLE_SLAVE":    3,
+}
+
+func (x OfpControllerRole) String() string {
+	return proto.EnumName(OfpControllerRole_name, int32(x))
+}
+
+func (OfpControllerRole) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{51}
+}
+
+// Header on all OpenFlow packets.
+type OfpHeader struct {
+	Version              uint32   `protobuf:"varint,1,opt,name=version,proto3" json:"version,omitempty"`
+	Type                 OfpType  `protobuf:"varint,2,opt,name=type,proto3,enum=openflow_13.OfpType" json:"type,omitempty"`
+	Xid                  uint32   `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpHeader) Reset()         { *m = OfpHeader{} }
+func (m *OfpHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpHeader) ProtoMessage()    {}
+func (*OfpHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{0}
+}
+
+func (m *OfpHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpHeader.Unmarshal(m, b)
+}
+func (m *OfpHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpHeader.Merge(m, src)
+}
+func (m *OfpHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpHeader.Size(m)
+}
+func (m *OfpHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpHeader proto.InternalMessageInfo
+
+func (m *OfpHeader) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *OfpHeader) GetType() OfpType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpType_OFPT_HELLO
+}
+
+func (m *OfpHeader) GetXid() uint32 {
+	if m != nil {
+		return m.Xid
+	}
+	return 0
+}
+
+// Common header for all Hello Elements
+type OfpHelloElemHeader struct {
+	Type OfpHelloElemType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpHelloElemType" json:"type,omitempty"`
+	// Types that are valid to be assigned to Element:
+	//	*OfpHelloElemHeader_Versionbitmap
+	Element              isOfpHelloElemHeader_Element `protobuf_oneof:"element"`
+	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
+	XXX_unrecognized     []byte                       `json:"-"`
+	XXX_sizecache        int32                        `json:"-"`
+}
+
+func (m *OfpHelloElemHeader) Reset()         { *m = OfpHelloElemHeader{} }
+func (m *OfpHelloElemHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpHelloElemHeader) ProtoMessage()    {}
+func (*OfpHelloElemHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{1}
+}
+
+func (m *OfpHelloElemHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpHelloElemHeader.Unmarshal(m, b)
+}
+func (m *OfpHelloElemHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpHelloElemHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpHelloElemHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpHelloElemHeader.Merge(m, src)
+}
+func (m *OfpHelloElemHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpHelloElemHeader.Size(m)
+}
+func (m *OfpHelloElemHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpHelloElemHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpHelloElemHeader proto.InternalMessageInfo
+
+func (m *OfpHelloElemHeader) GetType() OfpHelloElemType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpHelloElemType_OFPHET_INVALID
+}
+
+type isOfpHelloElemHeader_Element interface {
+	isOfpHelloElemHeader_Element()
+}
+
+type OfpHelloElemHeader_Versionbitmap struct {
+	Versionbitmap *OfpHelloElemVersionbitmap `protobuf:"bytes,2,opt,name=versionbitmap,proto3,oneof"`
+}
+
+func (*OfpHelloElemHeader_Versionbitmap) isOfpHelloElemHeader_Element() {}
+
+func (m *OfpHelloElemHeader) GetElement() isOfpHelloElemHeader_Element {
+	if m != nil {
+		return m.Element
+	}
+	return nil
+}
+
+func (m *OfpHelloElemHeader) GetVersionbitmap() *OfpHelloElemVersionbitmap {
+	if x, ok := m.GetElement().(*OfpHelloElemHeader_Versionbitmap); ok {
+		return x.Versionbitmap
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpHelloElemHeader) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpHelloElemHeader_Versionbitmap)(nil),
+	}
+}
+
+// Version bitmap Hello Element
+type OfpHelloElemVersionbitmap struct {
+	Bitmaps              []uint32 `protobuf:"varint,2,rep,packed,name=bitmaps,proto3" json:"bitmaps,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpHelloElemVersionbitmap) Reset()         { *m = OfpHelloElemVersionbitmap{} }
+func (m *OfpHelloElemVersionbitmap) String() string { return proto.CompactTextString(m) }
+func (*OfpHelloElemVersionbitmap) ProtoMessage()    {}
+func (*OfpHelloElemVersionbitmap) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{2}
+}
+
+func (m *OfpHelloElemVersionbitmap) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpHelloElemVersionbitmap.Unmarshal(m, b)
+}
+func (m *OfpHelloElemVersionbitmap) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpHelloElemVersionbitmap.Marshal(b, m, deterministic)
+}
+func (m *OfpHelloElemVersionbitmap) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpHelloElemVersionbitmap.Merge(m, src)
+}
+func (m *OfpHelloElemVersionbitmap) XXX_Size() int {
+	return xxx_messageInfo_OfpHelloElemVersionbitmap.Size(m)
+}
+func (m *OfpHelloElemVersionbitmap) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpHelloElemVersionbitmap.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpHelloElemVersionbitmap proto.InternalMessageInfo
+
+func (m *OfpHelloElemVersionbitmap) GetBitmaps() []uint32 {
+	if m != nil {
+		return m.Bitmaps
+	}
+	return nil
+}
+
+// OFPT_HELLO.  This message includes zero or more hello elements having
+// variable size. Unknown elements types must be ignored/skipped, to allow
+// for future extensions.
+type OfpHello struct {
+	// Hello element list
+	Elements             []*OfpHelloElemHeader `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpHello) Reset()         { *m = OfpHello{} }
+func (m *OfpHello) String() string { return proto.CompactTextString(m) }
+func (*OfpHello) ProtoMessage()    {}
+func (*OfpHello) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{3}
+}
+
+func (m *OfpHello) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpHello.Unmarshal(m, b)
+}
+func (m *OfpHello) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpHello.Marshal(b, m, deterministic)
+}
+func (m *OfpHello) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpHello.Merge(m, src)
+}
+func (m *OfpHello) XXX_Size() int {
+	return xxx_messageInfo_OfpHello.Size(m)
+}
+func (m *OfpHello) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpHello.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpHello proto.InternalMessageInfo
+
+func (m *OfpHello) GetElements() []*OfpHelloElemHeader {
+	if m != nil {
+		return m.Elements
+	}
+	return nil
+}
+
+// Switch configuration.
+type OfpSwitchConfig struct {
+	//ofp_header header;
+	Flags                uint32   `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"`
+	MissSendLen          uint32   `protobuf:"varint,2,opt,name=miss_send_len,json=missSendLen,proto3" json:"miss_send_len,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpSwitchConfig) Reset()         { *m = OfpSwitchConfig{} }
+func (m *OfpSwitchConfig) String() string { return proto.CompactTextString(m) }
+func (*OfpSwitchConfig) ProtoMessage()    {}
+func (*OfpSwitchConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{4}
+}
+
+func (m *OfpSwitchConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpSwitchConfig.Unmarshal(m, b)
+}
+func (m *OfpSwitchConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpSwitchConfig.Marshal(b, m, deterministic)
+}
+func (m *OfpSwitchConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpSwitchConfig.Merge(m, src)
+}
+func (m *OfpSwitchConfig) XXX_Size() int {
+	return xxx_messageInfo_OfpSwitchConfig.Size(m)
+}
+func (m *OfpSwitchConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpSwitchConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpSwitchConfig proto.InternalMessageInfo
+
+func (m *OfpSwitchConfig) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpSwitchConfig) GetMissSendLen() uint32 {
+	if m != nil {
+		return m.MissSendLen
+	}
+	return 0
+}
+
+// Configure/Modify behavior of a flow table
+type OfpTableMod struct {
+	//ofp_header header;
+	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	Config               uint32   `protobuf:"varint,2,opt,name=config,proto3" json:"config,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpTableMod) Reset()         { *m = OfpTableMod{} }
+func (m *OfpTableMod) String() string { return proto.CompactTextString(m) }
+func (*OfpTableMod) ProtoMessage()    {}
+func (*OfpTableMod) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{5}
+}
+
+func (m *OfpTableMod) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableMod.Unmarshal(m, b)
+}
+func (m *OfpTableMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableMod.Marshal(b, m, deterministic)
+}
+func (m *OfpTableMod) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableMod.Merge(m, src)
+}
+func (m *OfpTableMod) XXX_Size() int {
+	return xxx_messageInfo_OfpTableMod.Size(m)
+}
+func (m *OfpTableMod) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableMod.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableMod proto.InternalMessageInfo
+
+func (m *OfpTableMod) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpTableMod) GetConfig() uint32 {
+	if m != nil {
+		return m.Config
+	}
+	return 0
+}
+
+// Description of a port
+type OfpPort struct {
+	PortNo uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	HwAddr []uint32 `protobuf:"varint,2,rep,packed,name=hw_addr,json=hwAddr,proto3" json:"hw_addr,omitempty"`
+	Name   string   `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"`
+	Config uint32   `protobuf:"varint,4,opt,name=config,proto3" json:"config,omitempty"`
+	State  uint32   `protobuf:"varint,5,opt,name=state,proto3" json:"state,omitempty"`
+	// Bitmaps of OFPPF_* that describe features.  All bits zeroed if
+	// unsupported or unavailable.
+	Curr                 uint32   `protobuf:"varint,6,opt,name=curr,proto3" json:"curr,omitempty"`
+	Advertised           uint32   `protobuf:"varint,7,opt,name=advertised,proto3" json:"advertised,omitempty"`
+	Supported            uint32   `protobuf:"varint,8,opt,name=supported,proto3" json:"supported,omitempty"`
+	Peer                 uint32   `protobuf:"varint,9,opt,name=peer,proto3" json:"peer,omitempty"`
+	CurrSpeed            uint32   `protobuf:"varint,10,opt,name=curr_speed,json=currSpeed,proto3" json:"curr_speed,omitempty"`
+	MaxSpeed             uint32   `protobuf:"varint,11,opt,name=max_speed,json=maxSpeed,proto3" json:"max_speed,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpPort) Reset()         { *m = OfpPort{} }
+func (m *OfpPort) String() string { return proto.CompactTextString(m) }
+func (*OfpPort) ProtoMessage()    {}
+func (*OfpPort) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{6}
+}
+
+func (m *OfpPort) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPort.Unmarshal(m, b)
+}
+func (m *OfpPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPort.Marshal(b, m, deterministic)
+}
+func (m *OfpPort) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPort.Merge(m, src)
+}
+func (m *OfpPort) XXX_Size() int {
+	return xxx_messageInfo_OfpPort.Size(m)
+}
+func (m *OfpPort) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPort.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPort proto.InternalMessageInfo
+
+func (m *OfpPort) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OfpPort) GetHwAddr() []uint32 {
+	if m != nil {
+		return m.HwAddr
+	}
+	return nil
+}
+
+func (m *OfpPort) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *OfpPort) GetConfig() uint32 {
+	if m != nil {
+		return m.Config
+	}
+	return 0
+}
+
+func (m *OfpPort) GetState() uint32 {
+	if m != nil {
+		return m.State
+	}
+	return 0
+}
+
+func (m *OfpPort) GetCurr() uint32 {
+	if m != nil {
+		return m.Curr
+	}
+	return 0
+}
+
+func (m *OfpPort) GetAdvertised() uint32 {
+	if m != nil {
+		return m.Advertised
+	}
+	return 0
+}
+
+func (m *OfpPort) GetSupported() uint32 {
+	if m != nil {
+		return m.Supported
+	}
+	return 0
+}
+
+func (m *OfpPort) GetPeer() uint32 {
+	if m != nil {
+		return m.Peer
+	}
+	return 0
+}
+
+func (m *OfpPort) GetCurrSpeed() uint32 {
+	if m != nil {
+		return m.CurrSpeed
+	}
+	return 0
+}
+
+func (m *OfpPort) GetMaxSpeed() uint32 {
+	if m != nil {
+		return m.MaxSpeed
+	}
+	return 0
+}
+
+// Switch features.
+type OfpSwitchFeatures struct {
+	//ofp_header header;
+	DatapathId  uint64 `protobuf:"varint,1,opt,name=datapath_id,json=datapathId,proto3" json:"datapath_id,omitempty"`
+	NBuffers    uint32 `protobuf:"varint,2,opt,name=n_buffers,json=nBuffers,proto3" json:"n_buffers,omitempty"`
+	NTables     uint32 `protobuf:"varint,3,opt,name=n_tables,json=nTables,proto3" json:"n_tables,omitempty"`
+	AuxiliaryId uint32 `protobuf:"varint,4,opt,name=auxiliary_id,json=auxiliaryId,proto3" json:"auxiliary_id,omitempty"`
+	// Features.
+	Capabilities         uint32   `protobuf:"varint,5,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpSwitchFeatures) Reset()         { *m = OfpSwitchFeatures{} }
+func (m *OfpSwitchFeatures) String() string { return proto.CompactTextString(m) }
+func (*OfpSwitchFeatures) ProtoMessage()    {}
+func (*OfpSwitchFeatures) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{7}
+}
+
+func (m *OfpSwitchFeatures) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpSwitchFeatures.Unmarshal(m, b)
+}
+func (m *OfpSwitchFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpSwitchFeatures.Marshal(b, m, deterministic)
+}
+func (m *OfpSwitchFeatures) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpSwitchFeatures.Merge(m, src)
+}
+func (m *OfpSwitchFeatures) XXX_Size() int {
+	return xxx_messageInfo_OfpSwitchFeatures.Size(m)
+}
+func (m *OfpSwitchFeatures) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpSwitchFeatures.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpSwitchFeatures proto.InternalMessageInfo
+
+func (m *OfpSwitchFeatures) GetDatapathId() uint64 {
+	if m != nil {
+		return m.DatapathId
+	}
+	return 0
+}
+
+func (m *OfpSwitchFeatures) GetNBuffers() uint32 {
+	if m != nil {
+		return m.NBuffers
+	}
+	return 0
+}
+
+func (m *OfpSwitchFeatures) GetNTables() uint32 {
+	if m != nil {
+		return m.NTables
+	}
+	return 0
+}
+
+func (m *OfpSwitchFeatures) GetAuxiliaryId() uint32 {
+	if m != nil {
+		return m.AuxiliaryId
+	}
+	return 0
+}
+
+func (m *OfpSwitchFeatures) GetCapabilities() uint32 {
+	if m != nil {
+		return m.Capabilities
+	}
+	return 0
+}
+
+// A physical port has changed in the datapath
+type OfpPortStatus struct {
+	//ofp_header header;
+	Reason               OfpPortReason `protobuf:"varint,1,opt,name=reason,proto3,enum=openflow_13.OfpPortReason" json:"reason,omitempty"`
+	Desc                 *OfpPort      `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *OfpPortStatus) Reset()         { *m = OfpPortStatus{} }
+func (m *OfpPortStatus) String() string { return proto.CompactTextString(m) }
+func (*OfpPortStatus) ProtoMessage()    {}
+func (*OfpPortStatus) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{8}
+}
+
+func (m *OfpPortStatus) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPortStatus.Unmarshal(m, b)
+}
+func (m *OfpPortStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPortStatus.Marshal(b, m, deterministic)
+}
+func (m *OfpPortStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPortStatus.Merge(m, src)
+}
+func (m *OfpPortStatus) XXX_Size() int {
+	return xxx_messageInfo_OfpPortStatus.Size(m)
+}
+func (m *OfpPortStatus) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPortStatus.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPortStatus proto.InternalMessageInfo
+
+func (m *OfpPortStatus) GetReason() OfpPortReason {
+	if m != nil {
+		return m.Reason
+	}
+	return OfpPortReason_OFPPR_ADD
+}
+
+func (m *OfpPortStatus) GetDesc() *OfpPort {
+	if m != nil {
+		return m.Desc
+	}
+	return nil
+}
+
+// Modify behavior of the physical port
+type OfpPortMod struct {
+	//ofp_header header;
+	PortNo uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	HwAddr []uint32 `protobuf:"varint,2,rep,packed,name=hw_addr,json=hwAddr,proto3" json:"hw_addr,omitempty"`
+	// The hardware address is not
+	//configurable.  This is used to
+	//sanity-check the request, so it must
+	//be the same as returned in an
+	//ofp_port struct.
+	Config               uint32   `protobuf:"varint,3,opt,name=config,proto3" json:"config,omitempty"`
+	Mask                 uint32   `protobuf:"varint,4,opt,name=mask,proto3" json:"mask,omitempty"`
+	Advertise            uint32   `protobuf:"varint,5,opt,name=advertise,proto3" json:"advertise,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpPortMod) Reset()         { *m = OfpPortMod{} }
+func (m *OfpPortMod) String() string { return proto.CompactTextString(m) }
+func (*OfpPortMod) ProtoMessage()    {}
+func (*OfpPortMod) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{9}
+}
+
+func (m *OfpPortMod) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPortMod.Unmarshal(m, b)
+}
+func (m *OfpPortMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPortMod.Marshal(b, m, deterministic)
+}
+func (m *OfpPortMod) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPortMod.Merge(m, src)
+}
+func (m *OfpPortMod) XXX_Size() int {
+	return xxx_messageInfo_OfpPortMod.Size(m)
+}
+func (m *OfpPortMod) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPortMod.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPortMod proto.InternalMessageInfo
+
+func (m *OfpPortMod) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OfpPortMod) GetHwAddr() []uint32 {
+	if m != nil {
+		return m.HwAddr
+	}
+	return nil
+}
+
+func (m *OfpPortMod) GetConfig() uint32 {
+	if m != nil {
+		return m.Config
+	}
+	return 0
+}
+
+func (m *OfpPortMod) GetMask() uint32 {
+	if m != nil {
+		return m.Mask
+	}
+	return 0
+}
+
+func (m *OfpPortMod) GetAdvertise() uint32 {
+	if m != nil {
+		return m.Advertise
+	}
+	return 0
+}
+
+// Fields to match against flows
+type OfpMatch struct {
+	Type                 OfpMatchType   `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMatchType" json:"type,omitempty"`
+	OxmFields            []*OfpOxmField `protobuf:"bytes,2,rep,name=oxm_fields,json=oxmFields,proto3" json:"oxm_fields,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *OfpMatch) Reset()         { *m = OfpMatch{} }
+func (m *OfpMatch) String() string { return proto.CompactTextString(m) }
+func (*OfpMatch) ProtoMessage()    {}
+func (*OfpMatch) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{10}
+}
+
+func (m *OfpMatch) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMatch.Unmarshal(m, b)
+}
+func (m *OfpMatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMatch.Marshal(b, m, deterministic)
+}
+func (m *OfpMatch) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMatch.Merge(m, src)
+}
+func (m *OfpMatch) XXX_Size() int {
+	return xxx_messageInfo_OfpMatch.Size(m)
+}
+func (m *OfpMatch) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMatch.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMatch proto.InternalMessageInfo
+
+func (m *OfpMatch) GetType() OfpMatchType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpMatchType_OFPMT_STANDARD
+}
+
+func (m *OfpMatch) GetOxmFields() []*OfpOxmField {
+	if m != nil {
+		return m.OxmFields
+	}
+	return nil
+}
+
+// OXM Flow match fields
+type OfpOxmField struct {
+	OxmClass OfpOxmClass `protobuf:"varint,1,opt,name=oxm_class,json=oxmClass,proto3,enum=openflow_13.OfpOxmClass" json:"oxm_class,omitempty"`
+	// Types that are valid to be assigned to Field:
+	//	*OfpOxmField_OfbField
+	//	*OfpOxmField_ExperimenterField
+	Field                isOfpOxmField_Field `protobuf_oneof:"field"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *OfpOxmField) Reset()         { *m = OfpOxmField{} }
+func (m *OfpOxmField) String() string { return proto.CompactTextString(m) }
+func (*OfpOxmField) ProtoMessage()    {}
+func (*OfpOxmField) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{11}
+}
+
+func (m *OfpOxmField) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpOxmField.Unmarshal(m, b)
+}
+func (m *OfpOxmField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpOxmField.Marshal(b, m, deterministic)
+}
+func (m *OfpOxmField) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpOxmField.Merge(m, src)
+}
+func (m *OfpOxmField) XXX_Size() int {
+	return xxx_messageInfo_OfpOxmField.Size(m)
+}
+func (m *OfpOxmField) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpOxmField.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpOxmField proto.InternalMessageInfo
+
+func (m *OfpOxmField) GetOxmClass() OfpOxmClass {
+	if m != nil {
+		return m.OxmClass
+	}
+	return OfpOxmClass_OFPXMC_NXM_0
+}
+
+type isOfpOxmField_Field interface {
+	isOfpOxmField_Field()
+}
+
+type OfpOxmField_OfbField struct {
+	OfbField *OfpOxmOfbField `protobuf:"bytes,4,opt,name=ofb_field,json=ofbField,proto3,oneof"`
+}
+
+type OfpOxmField_ExperimenterField struct {
+	ExperimenterField *OfpOxmExperimenterField `protobuf:"bytes,5,opt,name=experimenter_field,json=experimenterField,proto3,oneof"`
+}
+
+func (*OfpOxmField_OfbField) isOfpOxmField_Field() {}
+
+func (*OfpOxmField_ExperimenterField) isOfpOxmField_Field() {}
+
+func (m *OfpOxmField) GetField() isOfpOxmField_Field {
+	if m != nil {
+		return m.Field
+	}
+	return nil
+}
+
+func (m *OfpOxmField) GetOfbField() *OfpOxmOfbField {
+	if x, ok := m.GetField().(*OfpOxmField_OfbField); ok {
+		return x.OfbField
+	}
+	return nil
+}
+
+func (m *OfpOxmField) GetExperimenterField() *OfpOxmExperimenterField {
+	if x, ok := m.GetField().(*OfpOxmField_ExperimenterField); ok {
+		return x.ExperimenterField
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpOxmField) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpOxmField_OfbField)(nil),
+		(*OfpOxmField_ExperimenterField)(nil),
+	}
+}
+
+// OXM OpenFlow Basic Match Field
+type OfpOxmOfbField struct {
+	Type    OxmOfbFieldTypes `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OxmOfbFieldTypes" json:"type,omitempty"`
+	HasMask bool             `protobuf:"varint,2,opt,name=has_mask,json=hasMask,proto3" json:"has_mask,omitempty"`
+	// Types that are valid to be assigned to Value:
+	//	*OfpOxmOfbField_Port
+	//	*OfpOxmOfbField_PhysicalPort
+	//	*OfpOxmOfbField_TableMetadata
+	//	*OfpOxmOfbField_EthDst
+	//	*OfpOxmOfbField_EthSrc
+	//	*OfpOxmOfbField_EthType
+	//	*OfpOxmOfbField_VlanVid
+	//	*OfpOxmOfbField_VlanPcp
+	//	*OfpOxmOfbField_IpDscp
+	//	*OfpOxmOfbField_IpEcn
+	//	*OfpOxmOfbField_IpProto
+	//	*OfpOxmOfbField_Ipv4Src
+	//	*OfpOxmOfbField_Ipv4Dst
+	//	*OfpOxmOfbField_TcpSrc
+	//	*OfpOxmOfbField_TcpDst
+	//	*OfpOxmOfbField_UdpSrc
+	//	*OfpOxmOfbField_UdpDst
+	//	*OfpOxmOfbField_SctpSrc
+	//	*OfpOxmOfbField_SctpDst
+	//	*OfpOxmOfbField_Icmpv4Type
+	//	*OfpOxmOfbField_Icmpv4Code
+	//	*OfpOxmOfbField_ArpOp
+	//	*OfpOxmOfbField_ArpSpa
+	//	*OfpOxmOfbField_ArpTpa
+	//	*OfpOxmOfbField_ArpSha
+	//	*OfpOxmOfbField_ArpTha
+	//	*OfpOxmOfbField_Ipv6Src
+	//	*OfpOxmOfbField_Ipv6Dst
+	//	*OfpOxmOfbField_Ipv6Flabel
+	//	*OfpOxmOfbField_Icmpv6Type
+	//	*OfpOxmOfbField_Icmpv6Code
+	//	*OfpOxmOfbField_Ipv6NdTarget
+	//	*OfpOxmOfbField_Ipv6NdSsl
+	//	*OfpOxmOfbField_Ipv6NdTll
+	//	*OfpOxmOfbField_MplsLabel
+	//	*OfpOxmOfbField_MplsTc
+	//	*OfpOxmOfbField_MplsBos
+	//	*OfpOxmOfbField_PbbIsid
+	//	*OfpOxmOfbField_TunnelId
+	//	*OfpOxmOfbField_Ipv6Exthdr
+	Value isOfpOxmOfbField_Value `protobuf_oneof:"value"`
+	// Optional mask values (must be present when has_mask is true
+	//
+	// Types that are valid to be assigned to Mask:
+	//	*OfpOxmOfbField_TableMetadataMask
+	//	*OfpOxmOfbField_EthDstMask
+	//	*OfpOxmOfbField_EthSrcMask
+	//	*OfpOxmOfbField_VlanVidMask
+	//	*OfpOxmOfbField_Ipv4SrcMask
+	//	*OfpOxmOfbField_Ipv4DstMask
+	//	*OfpOxmOfbField_ArpSpaMask
+	//	*OfpOxmOfbField_ArpTpaMask
+	//	*OfpOxmOfbField_Ipv6SrcMask
+	//	*OfpOxmOfbField_Ipv6DstMask
+	//	*OfpOxmOfbField_Ipv6FlabelMask
+	//	*OfpOxmOfbField_PbbIsidMask
+	//	*OfpOxmOfbField_TunnelIdMask
+	//	*OfpOxmOfbField_Ipv6ExthdrMask
+	Mask                 isOfpOxmOfbField_Mask `protobuf_oneof:"mask"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpOxmOfbField) Reset()         { *m = OfpOxmOfbField{} }
+func (m *OfpOxmOfbField) String() string { return proto.CompactTextString(m) }
+func (*OfpOxmOfbField) ProtoMessage()    {}
+func (*OfpOxmOfbField) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{12}
+}
+
+func (m *OfpOxmOfbField) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpOxmOfbField.Unmarshal(m, b)
+}
+func (m *OfpOxmOfbField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpOxmOfbField.Marshal(b, m, deterministic)
+}
+func (m *OfpOxmOfbField) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpOxmOfbField.Merge(m, src)
+}
+func (m *OfpOxmOfbField) XXX_Size() int {
+	return xxx_messageInfo_OfpOxmOfbField.Size(m)
+}
+func (m *OfpOxmOfbField) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpOxmOfbField.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpOxmOfbField proto.InternalMessageInfo
+
+func (m *OfpOxmOfbField) GetType() OxmOfbFieldTypes {
+	if m != nil {
+		return m.Type
+	}
+	return OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT
+}
+
+func (m *OfpOxmOfbField) GetHasMask() bool {
+	if m != nil {
+		return m.HasMask
+	}
+	return false
+}
+
+type isOfpOxmOfbField_Value interface {
+	isOfpOxmOfbField_Value()
+}
+
+type OfpOxmOfbField_Port struct {
+	Port uint32 `protobuf:"varint,3,opt,name=port,proto3,oneof"`
+}
+
+type OfpOxmOfbField_PhysicalPort struct {
+	PhysicalPort uint32 `protobuf:"varint,4,opt,name=physical_port,json=physicalPort,proto3,oneof"`
+}
+
+type OfpOxmOfbField_TableMetadata struct {
+	TableMetadata uint64 `protobuf:"varint,5,opt,name=table_metadata,json=tableMetadata,proto3,oneof"`
+}
+
+type OfpOxmOfbField_EthDst struct {
+	EthDst []byte `protobuf:"bytes,6,opt,name=eth_dst,json=ethDst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_EthSrc struct {
+	EthSrc []byte `protobuf:"bytes,7,opt,name=eth_src,json=ethSrc,proto3,oneof"`
+}
+
+type OfpOxmOfbField_EthType struct {
+	EthType uint32 `protobuf:"varint,8,opt,name=eth_type,json=ethType,proto3,oneof"`
+}
+
+type OfpOxmOfbField_VlanVid struct {
+	VlanVid uint32 `protobuf:"varint,9,opt,name=vlan_vid,json=vlanVid,proto3,oneof"`
+}
+
+type OfpOxmOfbField_VlanPcp struct {
+	VlanPcp uint32 `protobuf:"varint,10,opt,name=vlan_pcp,json=vlanPcp,proto3,oneof"`
+}
+
+type OfpOxmOfbField_IpDscp struct {
+	IpDscp uint32 `protobuf:"varint,11,opt,name=ip_dscp,json=ipDscp,proto3,oneof"`
+}
+
+type OfpOxmOfbField_IpEcn struct {
+	IpEcn uint32 `protobuf:"varint,12,opt,name=ip_ecn,json=ipEcn,proto3,oneof"`
+}
+
+type OfpOxmOfbField_IpProto struct {
+	IpProto uint32 `protobuf:"varint,13,opt,name=ip_proto,json=ipProto,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv4Src struct {
+	Ipv4Src uint32 `protobuf:"varint,14,opt,name=ipv4_src,json=ipv4Src,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv4Dst struct {
+	Ipv4Dst uint32 `protobuf:"varint,15,opt,name=ipv4_dst,json=ipv4Dst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_TcpSrc struct {
+	TcpSrc uint32 `protobuf:"varint,16,opt,name=tcp_src,json=tcpSrc,proto3,oneof"`
+}
+
+type OfpOxmOfbField_TcpDst struct {
+	TcpDst uint32 `protobuf:"varint,17,opt,name=tcp_dst,json=tcpDst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_UdpSrc struct {
+	UdpSrc uint32 `protobuf:"varint,18,opt,name=udp_src,json=udpSrc,proto3,oneof"`
+}
+
+type OfpOxmOfbField_UdpDst struct {
+	UdpDst uint32 `protobuf:"varint,19,opt,name=udp_dst,json=udpDst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_SctpSrc struct {
+	SctpSrc uint32 `protobuf:"varint,20,opt,name=sctp_src,json=sctpSrc,proto3,oneof"`
+}
+
+type OfpOxmOfbField_SctpDst struct {
+	SctpDst uint32 `protobuf:"varint,21,opt,name=sctp_dst,json=sctpDst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Icmpv4Type struct {
+	Icmpv4Type uint32 `protobuf:"varint,22,opt,name=icmpv4_type,json=icmpv4Type,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Icmpv4Code struct {
+	Icmpv4Code uint32 `protobuf:"varint,23,opt,name=icmpv4_code,json=icmpv4Code,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpOp struct {
+	ArpOp uint32 `protobuf:"varint,24,opt,name=arp_op,json=arpOp,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpSpa struct {
+	ArpSpa uint32 `protobuf:"varint,25,opt,name=arp_spa,json=arpSpa,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpTpa struct {
+	ArpTpa uint32 `protobuf:"varint,26,opt,name=arp_tpa,json=arpTpa,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpSha struct {
+	ArpSha []byte `protobuf:"bytes,27,opt,name=arp_sha,json=arpSha,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpTha struct {
+	ArpTha []byte `protobuf:"bytes,28,opt,name=arp_tha,json=arpTha,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6Src struct {
+	Ipv6Src []byte `protobuf:"bytes,29,opt,name=ipv6_src,json=ipv6Src,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6Dst struct {
+	Ipv6Dst []byte `protobuf:"bytes,30,opt,name=ipv6_dst,json=ipv6Dst,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6Flabel struct {
+	Ipv6Flabel uint32 `protobuf:"varint,31,opt,name=ipv6_flabel,json=ipv6Flabel,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Icmpv6Type struct {
+	Icmpv6Type uint32 `protobuf:"varint,32,opt,name=icmpv6_type,json=icmpv6Type,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Icmpv6Code struct {
+	Icmpv6Code uint32 `protobuf:"varint,33,opt,name=icmpv6_code,json=icmpv6Code,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6NdTarget struct {
+	Ipv6NdTarget []byte `protobuf:"bytes,34,opt,name=ipv6_nd_target,json=ipv6NdTarget,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6NdSsl struct {
+	Ipv6NdSsl []byte `protobuf:"bytes,35,opt,name=ipv6_nd_ssl,json=ipv6NdSsl,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6NdTll struct {
+	Ipv6NdTll []byte `protobuf:"bytes,36,opt,name=ipv6_nd_tll,json=ipv6NdTll,proto3,oneof"`
+}
+
+type OfpOxmOfbField_MplsLabel struct {
+	MplsLabel uint32 `protobuf:"varint,37,opt,name=mpls_label,json=mplsLabel,proto3,oneof"`
+}
+
+type OfpOxmOfbField_MplsTc struct {
+	MplsTc uint32 `protobuf:"varint,38,opt,name=mpls_tc,json=mplsTc,proto3,oneof"`
+}
+
+type OfpOxmOfbField_MplsBos struct {
+	MplsBos uint32 `protobuf:"varint,39,opt,name=mpls_bos,json=mplsBos,proto3,oneof"`
+}
+
+type OfpOxmOfbField_PbbIsid struct {
+	PbbIsid uint32 `protobuf:"varint,40,opt,name=pbb_isid,json=pbbIsid,proto3,oneof"`
+}
+
+type OfpOxmOfbField_TunnelId struct {
+	TunnelId uint64 `protobuf:"varint,41,opt,name=tunnel_id,json=tunnelId,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6Exthdr struct {
+	Ipv6Exthdr uint32 `protobuf:"varint,42,opt,name=ipv6_exthdr,json=ipv6Exthdr,proto3,oneof"`
+}
+
+func (*OfpOxmOfbField_Port) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_PhysicalPort) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_TableMetadata) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_EthDst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_EthSrc) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_EthType) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_VlanVid) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_VlanPcp) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_IpDscp) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_IpEcn) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_IpProto) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv4Src) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv4Dst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_TcpSrc) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_TcpDst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_UdpSrc) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_UdpDst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_SctpSrc) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_SctpDst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Icmpv4Type) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Icmpv4Code) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_ArpOp) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_ArpSpa) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_ArpTpa) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_ArpSha) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_ArpTha) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6Src) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6Dst) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6Flabel) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Icmpv6Type) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Icmpv6Code) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6NdTarget) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6NdSsl) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6NdTll) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_MplsLabel) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_MplsTc) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_MplsBos) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_PbbIsid) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_TunnelId) isOfpOxmOfbField_Value() {}
+
+func (*OfpOxmOfbField_Ipv6Exthdr) isOfpOxmOfbField_Value() {}
+
+func (m *OfpOxmOfbField) GetValue() isOfpOxmOfbField_Value {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetPort() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Port); ok {
+		return x.Port
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetPhysicalPort() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_PhysicalPort); ok {
+		return x.PhysicalPort
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetTableMetadata() uint64 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_TableMetadata); ok {
+		return x.TableMetadata
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetEthDst() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_EthDst); ok {
+		return x.EthDst
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetEthSrc() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_EthSrc); ok {
+		return x.EthSrc
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetEthType() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_EthType); ok {
+		return x.EthType
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetVlanVid() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_VlanVid); ok {
+		return x.VlanVid
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetVlanPcp() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_VlanPcp); ok {
+		return x.VlanPcp
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpDscp() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_IpDscp); ok {
+		return x.IpDscp
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpEcn() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_IpEcn); ok {
+		return x.IpEcn
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpProto() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_IpProto); ok {
+		return x.IpProto
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv4Src() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv4Src); ok {
+		return x.Ipv4Src
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv4Dst() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv4Dst); ok {
+		return x.Ipv4Dst
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetTcpSrc() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_TcpSrc); ok {
+		return x.TcpSrc
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetTcpDst() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_TcpDst); ok {
+		return x.TcpDst
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetUdpSrc() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_UdpSrc); ok {
+		return x.UdpSrc
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetUdpDst() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_UdpDst); ok {
+		return x.UdpDst
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetSctpSrc() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_SctpSrc); ok {
+		return x.SctpSrc
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetSctpDst() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_SctpDst); ok {
+		return x.SctpDst
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIcmpv4Type() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv4Type); ok {
+		return x.Icmpv4Type
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIcmpv4Code() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv4Code); ok {
+		return x.Icmpv4Code
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpOp() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpOp); ok {
+		return x.ArpOp
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpSpa() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpSpa); ok {
+		return x.ArpSpa
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpTpa() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpTpa); ok {
+		return x.ArpTpa
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpSha() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpSha); ok {
+		return x.ArpSha
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetArpTha() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_ArpTha); ok {
+		return x.ArpTha
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6Src() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Src); ok {
+		return x.Ipv6Src
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6Dst() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Dst); ok {
+		return x.Ipv6Dst
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6Flabel() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Flabel); ok {
+		return x.Ipv6Flabel
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIcmpv6Type() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv6Type); ok {
+		return x.Icmpv6Type
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIcmpv6Code() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Icmpv6Code); ok {
+		return x.Icmpv6Code
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv6NdTarget() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdTarget); ok {
+		return x.Ipv6NdTarget
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6NdSsl() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdSsl); ok {
+		return x.Ipv6NdSsl
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6NdTll() []byte {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6NdTll); ok {
+		return x.Ipv6NdTll
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetMplsLabel() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsLabel); ok {
+		return x.MplsLabel
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetMplsTc() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsTc); ok {
+		return x.MplsTc
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetMplsBos() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_MplsBos); ok {
+		return x.MplsBos
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetPbbIsid() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_PbbIsid); ok {
+		return x.PbbIsid
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetTunnelId() uint64 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_TunnelId); ok {
+		return x.TunnelId
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv6Exthdr() uint32 {
+	if x, ok := m.GetValue().(*OfpOxmOfbField_Ipv6Exthdr); ok {
+		return x.Ipv6Exthdr
+	}
+	return 0
+}
+
+type isOfpOxmOfbField_Mask interface {
+	isOfpOxmOfbField_Mask()
+}
+
+type OfpOxmOfbField_TableMetadataMask struct {
+	TableMetadataMask uint64 `protobuf:"varint,105,opt,name=table_metadata_mask,json=tableMetadataMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_EthDstMask struct {
+	EthDstMask []byte `protobuf:"bytes,106,opt,name=eth_dst_mask,json=ethDstMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_EthSrcMask struct {
+	EthSrcMask []byte `protobuf:"bytes,107,opt,name=eth_src_mask,json=ethSrcMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_VlanVidMask struct {
+	VlanVidMask uint32 `protobuf:"varint,109,opt,name=vlan_vid_mask,json=vlanVidMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv4SrcMask struct {
+	Ipv4SrcMask uint32 `protobuf:"varint,114,opt,name=ipv4_src_mask,json=ipv4SrcMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv4DstMask struct {
+	Ipv4DstMask uint32 `protobuf:"varint,115,opt,name=ipv4_dst_mask,json=ipv4DstMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpSpaMask struct {
+	ArpSpaMask uint32 `protobuf:"varint,125,opt,name=arp_spa_mask,json=arpSpaMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_ArpTpaMask struct {
+	ArpTpaMask uint32 `protobuf:"varint,126,opt,name=arp_tpa_mask,json=arpTpaMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6SrcMask struct {
+	Ipv6SrcMask []byte `protobuf:"bytes,129,opt,name=ipv6_src_mask,json=ipv6SrcMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6DstMask struct {
+	Ipv6DstMask []byte `protobuf:"bytes,130,opt,name=ipv6_dst_mask,json=ipv6DstMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6FlabelMask struct {
+	Ipv6FlabelMask uint32 `protobuf:"varint,131,opt,name=ipv6_flabel_mask,json=ipv6FlabelMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_PbbIsidMask struct {
+	PbbIsidMask uint32 `protobuf:"varint,140,opt,name=pbb_isid_mask,json=pbbIsidMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_TunnelIdMask struct {
+	TunnelIdMask uint64 `protobuf:"varint,141,opt,name=tunnel_id_mask,json=tunnelIdMask,proto3,oneof"`
+}
+
+type OfpOxmOfbField_Ipv6ExthdrMask struct {
+	Ipv6ExthdrMask uint32 `protobuf:"varint,142,opt,name=ipv6_exthdr_mask,json=ipv6ExthdrMask,proto3,oneof"`
+}
+
+func (*OfpOxmOfbField_TableMetadataMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_EthDstMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_EthSrcMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_VlanVidMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv4SrcMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv4DstMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_ArpSpaMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_ArpTpaMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv6SrcMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv6DstMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv6FlabelMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_PbbIsidMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_TunnelIdMask) isOfpOxmOfbField_Mask() {}
+
+func (*OfpOxmOfbField_Ipv6ExthdrMask) isOfpOxmOfbField_Mask() {}
+
+func (m *OfpOxmOfbField) GetMask() isOfpOxmOfbField_Mask {
+	if m != nil {
+		return m.Mask
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetTableMetadataMask() uint64 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_TableMetadataMask); ok {
+		return x.TableMetadataMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetEthDstMask() []byte {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_EthDstMask); ok {
+		return x.EthDstMask
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetEthSrcMask() []byte {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_EthSrcMask); ok {
+		return x.EthSrcMask
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetVlanVidMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_VlanVidMask); ok {
+		return x.VlanVidMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv4SrcMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv4SrcMask); ok {
+		return x.Ipv4SrcMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv4DstMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv4DstMask); ok {
+		return x.Ipv4DstMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpSpaMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_ArpSpaMask); ok {
+		return x.ArpSpaMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetArpTpaMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_ArpTpaMask); ok {
+		return x.ArpTpaMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv6SrcMask() []byte {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6SrcMask); ok {
+		return x.Ipv6SrcMask
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6DstMask() []byte {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6DstMask); ok {
+		return x.Ipv6DstMask
+	}
+	return nil
+}
+
+func (m *OfpOxmOfbField) GetIpv6FlabelMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6FlabelMask); ok {
+		return x.Ipv6FlabelMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetPbbIsidMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_PbbIsidMask); ok {
+		return x.PbbIsidMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetTunnelIdMask() uint64 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_TunnelIdMask); ok {
+		return x.TunnelIdMask
+	}
+	return 0
+}
+
+func (m *OfpOxmOfbField) GetIpv6ExthdrMask() uint32 {
+	if x, ok := m.GetMask().(*OfpOxmOfbField_Ipv6ExthdrMask); ok {
+		return x.Ipv6ExthdrMask
+	}
+	return 0
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpOxmOfbField) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpOxmOfbField_Port)(nil),
+		(*OfpOxmOfbField_PhysicalPort)(nil),
+		(*OfpOxmOfbField_TableMetadata)(nil),
+		(*OfpOxmOfbField_EthDst)(nil),
+		(*OfpOxmOfbField_EthSrc)(nil),
+		(*OfpOxmOfbField_EthType)(nil),
+		(*OfpOxmOfbField_VlanVid)(nil),
+		(*OfpOxmOfbField_VlanPcp)(nil),
+		(*OfpOxmOfbField_IpDscp)(nil),
+		(*OfpOxmOfbField_IpEcn)(nil),
+		(*OfpOxmOfbField_IpProto)(nil),
+		(*OfpOxmOfbField_Ipv4Src)(nil),
+		(*OfpOxmOfbField_Ipv4Dst)(nil),
+		(*OfpOxmOfbField_TcpSrc)(nil),
+		(*OfpOxmOfbField_TcpDst)(nil),
+		(*OfpOxmOfbField_UdpSrc)(nil),
+		(*OfpOxmOfbField_UdpDst)(nil),
+		(*OfpOxmOfbField_SctpSrc)(nil),
+		(*OfpOxmOfbField_SctpDst)(nil),
+		(*OfpOxmOfbField_Icmpv4Type)(nil),
+		(*OfpOxmOfbField_Icmpv4Code)(nil),
+		(*OfpOxmOfbField_ArpOp)(nil),
+		(*OfpOxmOfbField_ArpSpa)(nil),
+		(*OfpOxmOfbField_ArpTpa)(nil),
+		(*OfpOxmOfbField_ArpSha)(nil),
+		(*OfpOxmOfbField_ArpTha)(nil),
+		(*OfpOxmOfbField_Ipv6Src)(nil),
+		(*OfpOxmOfbField_Ipv6Dst)(nil),
+		(*OfpOxmOfbField_Ipv6Flabel)(nil),
+		(*OfpOxmOfbField_Icmpv6Type)(nil),
+		(*OfpOxmOfbField_Icmpv6Code)(nil),
+		(*OfpOxmOfbField_Ipv6NdTarget)(nil),
+		(*OfpOxmOfbField_Ipv6NdSsl)(nil),
+		(*OfpOxmOfbField_Ipv6NdTll)(nil),
+		(*OfpOxmOfbField_MplsLabel)(nil),
+		(*OfpOxmOfbField_MplsTc)(nil),
+		(*OfpOxmOfbField_MplsBos)(nil),
+		(*OfpOxmOfbField_PbbIsid)(nil),
+		(*OfpOxmOfbField_TunnelId)(nil),
+		(*OfpOxmOfbField_Ipv6Exthdr)(nil),
+		(*OfpOxmOfbField_TableMetadataMask)(nil),
+		(*OfpOxmOfbField_EthDstMask)(nil),
+		(*OfpOxmOfbField_EthSrcMask)(nil),
+		(*OfpOxmOfbField_VlanVidMask)(nil),
+		(*OfpOxmOfbField_Ipv4SrcMask)(nil),
+		(*OfpOxmOfbField_Ipv4DstMask)(nil),
+		(*OfpOxmOfbField_ArpSpaMask)(nil),
+		(*OfpOxmOfbField_ArpTpaMask)(nil),
+		(*OfpOxmOfbField_Ipv6SrcMask)(nil),
+		(*OfpOxmOfbField_Ipv6DstMask)(nil),
+		(*OfpOxmOfbField_Ipv6FlabelMask)(nil),
+		(*OfpOxmOfbField_PbbIsidMask)(nil),
+		(*OfpOxmOfbField_TunnelIdMask)(nil),
+		(*OfpOxmOfbField_Ipv6ExthdrMask)(nil),
+	}
+}
+
+// Header for OXM experimenter match fields.
+// The experimenter class should not use OXM_HEADER() macros for defining
+// fields due to this extra header.
+type OfpOxmExperimenterField struct {
+	OxmHeader            uint32   `protobuf:"varint,1,opt,name=oxm_header,json=oxmHeader,proto3" json:"oxm_header,omitempty"`
+	Experimenter         uint32   `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpOxmExperimenterField) Reset()         { *m = OfpOxmExperimenterField{} }
+func (m *OfpOxmExperimenterField) String() string { return proto.CompactTextString(m) }
+func (*OfpOxmExperimenterField) ProtoMessage()    {}
+func (*OfpOxmExperimenterField) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{13}
+}
+
+func (m *OfpOxmExperimenterField) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpOxmExperimenterField.Unmarshal(m, b)
+}
+func (m *OfpOxmExperimenterField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpOxmExperimenterField.Marshal(b, m, deterministic)
+}
+func (m *OfpOxmExperimenterField) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpOxmExperimenterField.Merge(m, src)
+}
+func (m *OfpOxmExperimenterField) XXX_Size() int {
+	return xxx_messageInfo_OfpOxmExperimenterField.Size(m)
+}
+func (m *OfpOxmExperimenterField) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpOxmExperimenterField.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpOxmExperimenterField proto.InternalMessageInfo
+
+func (m *OfpOxmExperimenterField) GetOxmHeader() uint32 {
+	if m != nil {
+		return m.OxmHeader
+	}
+	return 0
+}
+
+func (m *OfpOxmExperimenterField) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+// Action header that is common to all actions.  The length includes the
+// header and any padding used to make the action 64-bit aligned.
+// NB: The length of an action *must* always be a multiple of eight.
+type OfpAction struct {
+	Type OfpActionType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpActionType" json:"type,omitempty"`
+	// Types that are valid to be assigned to Action:
+	//	*OfpAction_Output
+	//	*OfpAction_MplsTtl
+	//	*OfpAction_Push
+	//	*OfpAction_PopMpls
+	//	*OfpAction_Group
+	//	*OfpAction_NwTtl
+	//	*OfpAction_SetField
+	//	*OfpAction_Experimenter
+	Action               isOfpAction_Action `protobuf_oneof:"action"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *OfpAction) Reset()         { *m = OfpAction{} }
+func (m *OfpAction) String() string { return proto.CompactTextString(m) }
+func (*OfpAction) ProtoMessage()    {}
+func (*OfpAction) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{14}
+}
+
+func (m *OfpAction) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpAction.Unmarshal(m, b)
+}
+func (m *OfpAction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpAction.Marshal(b, m, deterministic)
+}
+func (m *OfpAction) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpAction.Merge(m, src)
+}
+func (m *OfpAction) XXX_Size() int {
+	return xxx_messageInfo_OfpAction.Size(m)
+}
+func (m *OfpAction) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpAction.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpAction proto.InternalMessageInfo
+
+func (m *OfpAction) GetType() OfpActionType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpActionType_OFPAT_OUTPUT
+}
+
+type isOfpAction_Action interface {
+	isOfpAction_Action()
+}
+
+type OfpAction_Output struct {
+	Output *OfpActionOutput `protobuf:"bytes,2,opt,name=output,proto3,oneof"`
+}
+
+type OfpAction_MplsTtl struct {
+	MplsTtl *OfpActionMplsTtl `protobuf:"bytes,3,opt,name=mpls_ttl,json=mplsTtl,proto3,oneof"`
+}
+
+type OfpAction_Push struct {
+	Push *OfpActionPush `protobuf:"bytes,4,opt,name=push,proto3,oneof"`
+}
+
+type OfpAction_PopMpls struct {
+	PopMpls *OfpActionPopMpls `protobuf:"bytes,5,opt,name=pop_mpls,json=popMpls,proto3,oneof"`
+}
+
+type OfpAction_Group struct {
+	Group *OfpActionGroup `protobuf:"bytes,6,opt,name=group,proto3,oneof"`
+}
+
+type OfpAction_NwTtl struct {
+	NwTtl *OfpActionNwTtl `protobuf:"bytes,7,opt,name=nw_ttl,json=nwTtl,proto3,oneof"`
+}
+
+type OfpAction_SetField struct {
+	SetField *OfpActionSetField `protobuf:"bytes,8,opt,name=set_field,json=setField,proto3,oneof"`
+}
+
+type OfpAction_Experimenter struct {
+	Experimenter *OfpActionExperimenter `protobuf:"bytes,9,opt,name=experimenter,proto3,oneof"`
+}
+
+func (*OfpAction_Output) isOfpAction_Action() {}
+
+func (*OfpAction_MplsTtl) isOfpAction_Action() {}
+
+func (*OfpAction_Push) isOfpAction_Action() {}
+
+func (*OfpAction_PopMpls) isOfpAction_Action() {}
+
+func (*OfpAction_Group) isOfpAction_Action() {}
+
+func (*OfpAction_NwTtl) isOfpAction_Action() {}
+
+func (*OfpAction_SetField) isOfpAction_Action() {}
+
+func (*OfpAction_Experimenter) isOfpAction_Action() {}
+
+func (m *OfpAction) GetAction() isOfpAction_Action {
+	if m != nil {
+		return m.Action
+	}
+	return nil
+}
+
+func (m *OfpAction) GetOutput() *OfpActionOutput {
+	if x, ok := m.GetAction().(*OfpAction_Output); ok {
+		return x.Output
+	}
+	return nil
+}
+
+func (m *OfpAction) GetMplsTtl() *OfpActionMplsTtl {
+	if x, ok := m.GetAction().(*OfpAction_MplsTtl); ok {
+		return x.MplsTtl
+	}
+	return nil
+}
+
+func (m *OfpAction) GetPush() *OfpActionPush {
+	if x, ok := m.GetAction().(*OfpAction_Push); ok {
+		return x.Push
+	}
+	return nil
+}
+
+func (m *OfpAction) GetPopMpls() *OfpActionPopMpls {
+	if x, ok := m.GetAction().(*OfpAction_PopMpls); ok {
+		return x.PopMpls
+	}
+	return nil
+}
+
+func (m *OfpAction) GetGroup() *OfpActionGroup {
+	if x, ok := m.GetAction().(*OfpAction_Group); ok {
+		return x.Group
+	}
+	return nil
+}
+
+func (m *OfpAction) GetNwTtl() *OfpActionNwTtl {
+	if x, ok := m.GetAction().(*OfpAction_NwTtl); ok {
+		return x.NwTtl
+	}
+	return nil
+}
+
+func (m *OfpAction) GetSetField() *OfpActionSetField {
+	if x, ok := m.GetAction().(*OfpAction_SetField); ok {
+		return x.SetField
+	}
+	return nil
+}
+
+func (m *OfpAction) GetExperimenter() *OfpActionExperimenter {
+	if x, ok := m.GetAction().(*OfpAction_Experimenter); ok {
+		return x.Experimenter
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpAction) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpAction_Output)(nil),
+		(*OfpAction_MplsTtl)(nil),
+		(*OfpAction_Push)(nil),
+		(*OfpAction_PopMpls)(nil),
+		(*OfpAction_Group)(nil),
+		(*OfpAction_NwTtl)(nil),
+		(*OfpAction_SetField)(nil),
+		(*OfpAction_Experimenter)(nil),
+	}
+}
+
+// Action structure for OFPAT_OUTPUT, which sends packets out 'port'.
+// When the 'port' is the OFPP_CONTROLLER, 'max_len' indicates the max
+// number of bytes to send.  A 'max_len' of zero means no bytes of the
+// packet should be sent. A 'max_len' of OFPCML_NO_BUFFER means that
+// the packet is not buffered and the complete packet is to be sent to
+// the controller.
+type OfpActionOutput struct {
+	Port                 uint32   `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
+	MaxLen               uint32   `protobuf:"varint,2,opt,name=max_len,json=maxLen,proto3" json:"max_len,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionOutput) Reset()         { *m = OfpActionOutput{} }
+func (m *OfpActionOutput) String() string { return proto.CompactTextString(m) }
+func (*OfpActionOutput) ProtoMessage()    {}
+func (*OfpActionOutput) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{15}
+}
+
+func (m *OfpActionOutput) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionOutput.Unmarshal(m, b)
+}
+func (m *OfpActionOutput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionOutput.Marshal(b, m, deterministic)
+}
+func (m *OfpActionOutput) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionOutput.Merge(m, src)
+}
+func (m *OfpActionOutput) XXX_Size() int {
+	return xxx_messageInfo_OfpActionOutput.Size(m)
+}
+func (m *OfpActionOutput) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionOutput.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionOutput proto.InternalMessageInfo
+
+func (m *OfpActionOutput) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+func (m *OfpActionOutput) GetMaxLen() uint32 {
+	if m != nil {
+		return m.MaxLen
+	}
+	return 0
+}
+
+// Action structure for OFPAT_SET_MPLS_TTL.
+type OfpActionMplsTtl struct {
+	MplsTtl              uint32   `protobuf:"varint,1,opt,name=mpls_ttl,json=mplsTtl,proto3" json:"mpls_ttl,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionMplsTtl) Reset()         { *m = OfpActionMplsTtl{} }
+func (m *OfpActionMplsTtl) String() string { return proto.CompactTextString(m) }
+func (*OfpActionMplsTtl) ProtoMessage()    {}
+func (*OfpActionMplsTtl) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{16}
+}
+
+func (m *OfpActionMplsTtl) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionMplsTtl.Unmarshal(m, b)
+}
+func (m *OfpActionMplsTtl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionMplsTtl.Marshal(b, m, deterministic)
+}
+func (m *OfpActionMplsTtl) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionMplsTtl.Merge(m, src)
+}
+func (m *OfpActionMplsTtl) XXX_Size() int {
+	return xxx_messageInfo_OfpActionMplsTtl.Size(m)
+}
+func (m *OfpActionMplsTtl) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionMplsTtl.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionMplsTtl proto.InternalMessageInfo
+
+func (m *OfpActionMplsTtl) GetMplsTtl() uint32 {
+	if m != nil {
+		return m.MplsTtl
+	}
+	return 0
+}
+
+// Action structure for OFPAT_PUSH_VLAN/MPLS/PBB.
+type OfpActionPush struct {
+	Ethertype            uint32   `protobuf:"varint,1,opt,name=ethertype,proto3" json:"ethertype,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionPush) Reset()         { *m = OfpActionPush{} }
+func (m *OfpActionPush) String() string { return proto.CompactTextString(m) }
+func (*OfpActionPush) ProtoMessage()    {}
+func (*OfpActionPush) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{17}
+}
+
+func (m *OfpActionPush) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionPush.Unmarshal(m, b)
+}
+func (m *OfpActionPush) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionPush.Marshal(b, m, deterministic)
+}
+func (m *OfpActionPush) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionPush.Merge(m, src)
+}
+func (m *OfpActionPush) XXX_Size() int {
+	return xxx_messageInfo_OfpActionPush.Size(m)
+}
+func (m *OfpActionPush) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionPush.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionPush proto.InternalMessageInfo
+
+func (m *OfpActionPush) GetEthertype() uint32 {
+	if m != nil {
+		return m.Ethertype
+	}
+	return 0
+}
+
+// Action structure for OFPAT_POP_MPLS.
+type OfpActionPopMpls struct {
+	Ethertype            uint32   `protobuf:"varint,1,opt,name=ethertype,proto3" json:"ethertype,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionPopMpls) Reset()         { *m = OfpActionPopMpls{} }
+func (m *OfpActionPopMpls) String() string { return proto.CompactTextString(m) }
+func (*OfpActionPopMpls) ProtoMessage()    {}
+func (*OfpActionPopMpls) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{18}
+}
+
+func (m *OfpActionPopMpls) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionPopMpls.Unmarshal(m, b)
+}
+func (m *OfpActionPopMpls) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionPopMpls.Marshal(b, m, deterministic)
+}
+func (m *OfpActionPopMpls) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionPopMpls.Merge(m, src)
+}
+func (m *OfpActionPopMpls) XXX_Size() int {
+	return xxx_messageInfo_OfpActionPopMpls.Size(m)
+}
+func (m *OfpActionPopMpls) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionPopMpls.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionPopMpls proto.InternalMessageInfo
+
+func (m *OfpActionPopMpls) GetEthertype() uint32 {
+	if m != nil {
+		return m.Ethertype
+	}
+	return 0
+}
+
+// Action structure for OFPAT_GROUP.
+type OfpActionGroup struct {
+	GroupId              uint32   `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionGroup) Reset()         { *m = OfpActionGroup{} }
+func (m *OfpActionGroup) String() string { return proto.CompactTextString(m) }
+func (*OfpActionGroup) ProtoMessage()    {}
+func (*OfpActionGroup) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{19}
+}
+
+func (m *OfpActionGroup) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionGroup.Unmarshal(m, b)
+}
+func (m *OfpActionGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionGroup.Marshal(b, m, deterministic)
+}
+func (m *OfpActionGroup) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionGroup.Merge(m, src)
+}
+func (m *OfpActionGroup) XXX_Size() int {
+	return xxx_messageInfo_OfpActionGroup.Size(m)
+}
+func (m *OfpActionGroup) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionGroup.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionGroup proto.InternalMessageInfo
+
+func (m *OfpActionGroup) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+// Action structure for OFPAT_SET_NW_TTL.
+type OfpActionNwTtl struct {
+	NwTtl                uint32   `protobuf:"varint,1,opt,name=nw_ttl,json=nwTtl,proto3" json:"nw_ttl,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionNwTtl) Reset()         { *m = OfpActionNwTtl{} }
+func (m *OfpActionNwTtl) String() string { return proto.CompactTextString(m) }
+func (*OfpActionNwTtl) ProtoMessage()    {}
+func (*OfpActionNwTtl) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{20}
+}
+
+func (m *OfpActionNwTtl) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionNwTtl.Unmarshal(m, b)
+}
+func (m *OfpActionNwTtl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionNwTtl.Marshal(b, m, deterministic)
+}
+func (m *OfpActionNwTtl) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionNwTtl.Merge(m, src)
+}
+func (m *OfpActionNwTtl) XXX_Size() int {
+	return xxx_messageInfo_OfpActionNwTtl.Size(m)
+}
+func (m *OfpActionNwTtl) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionNwTtl.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionNwTtl proto.InternalMessageInfo
+
+func (m *OfpActionNwTtl) GetNwTtl() uint32 {
+	if m != nil {
+		return m.NwTtl
+	}
+	return 0
+}
+
+// Action structure for OFPAT_SET_FIELD.
+type OfpActionSetField struct {
+	Field                *OfpOxmField `protobuf:"bytes,1,opt,name=field,proto3" json:"field,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *OfpActionSetField) Reset()         { *m = OfpActionSetField{} }
+func (m *OfpActionSetField) String() string { return proto.CompactTextString(m) }
+func (*OfpActionSetField) ProtoMessage()    {}
+func (*OfpActionSetField) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{21}
+}
+
+func (m *OfpActionSetField) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionSetField.Unmarshal(m, b)
+}
+func (m *OfpActionSetField) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionSetField.Marshal(b, m, deterministic)
+}
+func (m *OfpActionSetField) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionSetField.Merge(m, src)
+}
+func (m *OfpActionSetField) XXX_Size() int {
+	return xxx_messageInfo_OfpActionSetField.Size(m)
+}
+func (m *OfpActionSetField) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionSetField.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionSetField proto.InternalMessageInfo
+
+func (m *OfpActionSetField) GetField() *OfpOxmField {
+	if m != nil {
+		return m.Field
+	}
+	return nil
+}
+
+// Action header for OFPAT_EXPERIMENTER.
+// The rest of the body is experimenter-defined.
+type OfpActionExperimenter struct {
+	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionExperimenter) Reset()         { *m = OfpActionExperimenter{} }
+func (m *OfpActionExperimenter) String() string { return proto.CompactTextString(m) }
+func (*OfpActionExperimenter) ProtoMessage()    {}
+func (*OfpActionExperimenter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{22}
+}
+
+func (m *OfpActionExperimenter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionExperimenter.Unmarshal(m, b)
+}
+func (m *OfpActionExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionExperimenter.Marshal(b, m, deterministic)
+}
+func (m *OfpActionExperimenter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionExperimenter.Merge(m, src)
+}
+func (m *OfpActionExperimenter) XXX_Size() int {
+	return xxx_messageInfo_OfpActionExperimenter.Size(m)
+}
+func (m *OfpActionExperimenter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionExperimenter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionExperimenter proto.InternalMessageInfo
+
+func (m *OfpActionExperimenter) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpActionExperimenter) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Instruction header that is common to all instructions.  The length includes
+// the header and any padding used to make the instruction 64-bit aligned.
+// NB: The length of an instruction *must* always be a multiple of eight.
+type OfpInstruction struct {
+	Type uint32 `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
+	// Types that are valid to be assigned to Data:
+	//	*OfpInstruction_GotoTable
+	//	*OfpInstruction_WriteMetadata
+	//	*OfpInstruction_Actions
+	//	*OfpInstruction_Meter
+	//	*OfpInstruction_Experimenter
+	Data                 isOfpInstruction_Data `protobuf_oneof:"data"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpInstruction) Reset()         { *m = OfpInstruction{} }
+func (m *OfpInstruction) String() string { return proto.CompactTextString(m) }
+func (*OfpInstruction) ProtoMessage()    {}
+func (*OfpInstruction) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{23}
+}
+
+func (m *OfpInstruction) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstruction.Unmarshal(m, b)
+}
+func (m *OfpInstruction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstruction.Marshal(b, m, deterministic)
+}
+func (m *OfpInstruction) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstruction.Merge(m, src)
+}
+func (m *OfpInstruction) XXX_Size() int {
+	return xxx_messageInfo_OfpInstruction.Size(m)
+}
+func (m *OfpInstruction) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstruction.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstruction proto.InternalMessageInfo
+
+func (m *OfpInstruction) GetType() uint32 {
+	if m != nil {
+		return m.Type
+	}
+	return 0
+}
+
+type isOfpInstruction_Data interface {
+	isOfpInstruction_Data()
+}
+
+type OfpInstruction_GotoTable struct {
+	GotoTable *OfpInstructionGotoTable `protobuf:"bytes,2,opt,name=goto_table,json=gotoTable,proto3,oneof"`
+}
+
+type OfpInstruction_WriteMetadata struct {
+	WriteMetadata *OfpInstructionWriteMetadata `protobuf:"bytes,3,opt,name=write_metadata,json=writeMetadata,proto3,oneof"`
+}
+
+type OfpInstruction_Actions struct {
+	Actions *OfpInstructionActions `protobuf:"bytes,4,opt,name=actions,proto3,oneof"`
+}
+
+type OfpInstruction_Meter struct {
+	Meter *OfpInstructionMeter `protobuf:"bytes,5,opt,name=meter,proto3,oneof"`
+}
+
+type OfpInstruction_Experimenter struct {
+	Experimenter *OfpInstructionExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
+}
+
+func (*OfpInstruction_GotoTable) isOfpInstruction_Data() {}
+
+func (*OfpInstruction_WriteMetadata) isOfpInstruction_Data() {}
+
+func (*OfpInstruction_Actions) isOfpInstruction_Data() {}
+
+func (*OfpInstruction_Meter) isOfpInstruction_Data() {}
+
+func (*OfpInstruction_Experimenter) isOfpInstruction_Data() {}
+
+func (m *OfpInstruction) GetData() isOfpInstruction_Data {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+func (m *OfpInstruction) GetGotoTable() *OfpInstructionGotoTable {
+	if x, ok := m.GetData().(*OfpInstruction_GotoTable); ok {
+		return x.GotoTable
+	}
+	return nil
+}
+
+func (m *OfpInstruction) GetWriteMetadata() *OfpInstructionWriteMetadata {
+	if x, ok := m.GetData().(*OfpInstruction_WriteMetadata); ok {
+		return x.WriteMetadata
+	}
+	return nil
+}
+
+func (m *OfpInstruction) GetActions() *OfpInstructionActions {
+	if x, ok := m.GetData().(*OfpInstruction_Actions); ok {
+		return x.Actions
+	}
+	return nil
+}
+
+func (m *OfpInstruction) GetMeter() *OfpInstructionMeter {
+	if x, ok := m.GetData().(*OfpInstruction_Meter); ok {
+		return x.Meter
+	}
+	return nil
+}
+
+func (m *OfpInstruction) GetExperimenter() *OfpInstructionExperimenter {
+	if x, ok := m.GetData().(*OfpInstruction_Experimenter); ok {
+		return x.Experimenter
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpInstruction) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpInstruction_GotoTable)(nil),
+		(*OfpInstruction_WriteMetadata)(nil),
+		(*OfpInstruction_Actions)(nil),
+		(*OfpInstruction_Meter)(nil),
+		(*OfpInstruction_Experimenter)(nil),
+	}
+}
+
+// Instruction structure for OFPIT_GOTO_TABLE
+type OfpInstructionGotoTable struct {
+	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpInstructionGotoTable) Reset()         { *m = OfpInstructionGotoTable{} }
+func (m *OfpInstructionGotoTable) String() string { return proto.CompactTextString(m) }
+func (*OfpInstructionGotoTable) ProtoMessage()    {}
+func (*OfpInstructionGotoTable) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{24}
+}
+
+func (m *OfpInstructionGotoTable) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstructionGotoTable.Unmarshal(m, b)
+}
+func (m *OfpInstructionGotoTable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstructionGotoTable.Marshal(b, m, deterministic)
+}
+func (m *OfpInstructionGotoTable) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstructionGotoTable.Merge(m, src)
+}
+func (m *OfpInstructionGotoTable) XXX_Size() int {
+	return xxx_messageInfo_OfpInstructionGotoTable.Size(m)
+}
+func (m *OfpInstructionGotoTable) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstructionGotoTable.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstructionGotoTable proto.InternalMessageInfo
+
+func (m *OfpInstructionGotoTable) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+// Instruction structure for OFPIT_WRITE_METADATA
+type OfpInstructionWriteMetadata struct {
+	Metadata             uint64   `protobuf:"varint,1,opt,name=metadata,proto3" json:"metadata,omitempty"`
+	MetadataMask         uint64   `protobuf:"varint,2,opt,name=metadata_mask,json=metadataMask,proto3" json:"metadata_mask,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpInstructionWriteMetadata) Reset()         { *m = OfpInstructionWriteMetadata{} }
+func (m *OfpInstructionWriteMetadata) String() string { return proto.CompactTextString(m) }
+func (*OfpInstructionWriteMetadata) ProtoMessage()    {}
+func (*OfpInstructionWriteMetadata) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{25}
+}
+
+func (m *OfpInstructionWriteMetadata) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstructionWriteMetadata.Unmarshal(m, b)
+}
+func (m *OfpInstructionWriteMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstructionWriteMetadata.Marshal(b, m, deterministic)
+}
+func (m *OfpInstructionWriteMetadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstructionWriteMetadata.Merge(m, src)
+}
+func (m *OfpInstructionWriteMetadata) XXX_Size() int {
+	return xxx_messageInfo_OfpInstructionWriteMetadata.Size(m)
+}
+func (m *OfpInstructionWriteMetadata) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstructionWriteMetadata.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstructionWriteMetadata proto.InternalMessageInfo
+
+func (m *OfpInstructionWriteMetadata) GetMetadata() uint64 {
+	if m != nil {
+		return m.Metadata
+	}
+	return 0
+}
+
+func (m *OfpInstructionWriteMetadata) GetMetadataMask() uint64 {
+	if m != nil {
+		return m.MetadataMask
+	}
+	return 0
+}
+
+// Instruction structure for OFPIT_WRITE/APPLY/CLEAR_ACTIONS
+type OfpInstructionActions struct {
+	Actions              []*OfpAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *OfpInstructionActions) Reset()         { *m = OfpInstructionActions{} }
+func (m *OfpInstructionActions) String() string { return proto.CompactTextString(m) }
+func (*OfpInstructionActions) ProtoMessage()    {}
+func (*OfpInstructionActions) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{26}
+}
+
+func (m *OfpInstructionActions) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstructionActions.Unmarshal(m, b)
+}
+func (m *OfpInstructionActions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstructionActions.Marshal(b, m, deterministic)
+}
+func (m *OfpInstructionActions) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstructionActions.Merge(m, src)
+}
+func (m *OfpInstructionActions) XXX_Size() int {
+	return xxx_messageInfo_OfpInstructionActions.Size(m)
+}
+func (m *OfpInstructionActions) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstructionActions.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstructionActions proto.InternalMessageInfo
+
+func (m *OfpInstructionActions) GetActions() []*OfpAction {
+	if m != nil {
+		return m.Actions
+	}
+	return nil
+}
+
+// Instruction structure for OFPIT_METER
+type OfpInstructionMeter struct {
+	MeterId              uint32   `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpInstructionMeter) Reset()         { *m = OfpInstructionMeter{} }
+func (m *OfpInstructionMeter) String() string { return proto.CompactTextString(m) }
+func (*OfpInstructionMeter) ProtoMessage()    {}
+func (*OfpInstructionMeter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{27}
+}
+
+func (m *OfpInstructionMeter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstructionMeter.Unmarshal(m, b)
+}
+func (m *OfpInstructionMeter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstructionMeter.Marshal(b, m, deterministic)
+}
+func (m *OfpInstructionMeter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstructionMeter.Merge(m, src)
+}
+func (m *OfpInstructionMeter) XXX_Size() int {
+	return xxx_messageInfo_OfpInstructionMeter.Size(m)
+}
+func (m *OfpInstructionMeter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstructionMeter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstructionMeter proto.InternalMessageInfo
+
+func (m *OfpInstructionMeter) GetMeterId() uint32 {
+	if m != nil {
+		return m.MeterId
+	}
+	return 0
+}
+
+// Instruction structure for experimental instructions
+type OfpInstructionExperimenter struct {
+	Experimenter uint32 `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	// Experimenter-defined arbitrary additional data.
+	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpInstructionExperimenter) Reset()         { *m = OfpInstructionExperimenter{} }
+func (m *OfpInstructionExperimenter) String() string { return proto.CompactTextString(m) }
+func (*OfpInstructionExperimenter) ProtoMessage()    {}
+func (*OfpInstructionExperimenter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{28}
+}
+
+func (m *OfpInstructionExperimenter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpInstructionExperimenter.Unmarshal(m, b)
+}
+func (m *OfpInstructionExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpInstructionExperimenter.Marshal(b, m, deterministic)
+}
+func (m *OfpInstructionExperimenter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpInstructionExperimenter.Merge(m, src)
+}
+func (m *OfpInstructionExperimenter) XXX_Size() int {
+	return xxx_messageInfo_OfpInstructionExperimenter.Size(m)
+}
+func (m *OfpInstructionExperimenter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpInstructionExperimenter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpInstructionExperimenter proto.InternalMessageInfo
+
+func (m *OfpInstructionExperimenter) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpInstructionExperimenter) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Flow setup and teardown (controller -> datapath).
+type OfpFlowMod struct {
+	//ofp_header header;
+	Cookie               uint64            `protobuf:"varint,1,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	CookieMask           uint64            `protobuf:"varint,2,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
+	TableId              uint32            `protobuf:"varint,3,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	Command              OfpFlowModCommand `protobuf:"varint,4,opt,name=command,proto3,enum=openflow_13.OfpFlowModCommand" json:"command,omitempty"`
+	IdleTimeout          uint32            `protobuf:"varint,5,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
+	HardTimeout          uint32            `protobuf:"varint,6,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
+	Priority             uint32            `protobuf:"varint,7,opt,name=priority,proto3" json:"priority,omitempty"`
+	BufferId             uint32            `protobuf:"varint,8,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
+	OutPort              uint32            `protobuf:"varint,9,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
+	OutGroup             uint32            `protobuf:"varint,10,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
+	Flags                uint32            `protobuf:"varint,11,opt,name=flags,proto3" json:"flags,omitempty"`
+	Match                *OfpMatch         `protobuf:"bytes,12,opt,name=match,proto3" json:"match,omitempty"`
+	Instructions         []*OfpInstruction `protobuf:"bytes,13,rep,name=instructions,proto3" json:"instructions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpFlowMod) Reset()         { *m = OfpFlowMod{} }
+func (m *OfpFlowMod) String() string { return proto.CompactTextString(m) }
+func (*OfpFlowMod) ProtoMessage()    {}
+func (*OfpFlowMod) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{29}
+}
+
+func (m *OfpFlowMod) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpFlowMod.Unmarshal(m, b)
+}
+func (m *OfpFlowMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpFlowMod.Marshal(b, m, deterministic)
+}
+func (m *OfpFlowMod) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpFlowMod.Merge(m, src)
+}
+func (m *OfpFlowMod) XXX_Size() int {
+	return xxx_messageInfo_OfpFlowMod.Size(m)
+}
+func (m *OfpFlowMod) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpFlowMod.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpFlowMod proto.InternalMessageInfo
+
+func (m *OfpFlowMod) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetCookieMask() uint64 {
+	if m != nil {
+		return m.CookieMask
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetCommand() OfpFlowModCommand {
+	if m != nil {
+		return m.Command
+	}
+	return OfpFlowModCommand_OFPFC_ADD
+}
+
+func (m *OfpFlowMod) GetIdleTimeout() uint32 {
+	if m != nil {
+		return m.IdleTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetHardTimeout() uint32 {
+	if m != nil {
+		return m.HardTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetBufferId() uint32 {
+	if m != nil {
+		return m.BufferId
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetOutPort() uint32 {
+	if m != nil {
+		return m.OutPort
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetOutGroup() uint32 {
+	if m != nil {
+		return m.OutGroup
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpFlowMod) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+func (m *OfpFlowMod) GetInstructions() []*OfpInstruction {
+	if m != nil {
+		return m.Instructions
+	}
+	return nil
+}
+
+// Bucket for use in groups.
+type OfpBucket struct {
+	Weight               uint32       `protobuf:"varint,1,opt,name=weight,proto3" json:"weight,omitempty"`
+	WatchPort            uint32       `protobuf:"varint,2,opt,name=watch_port,json=watchPort,proto3" json:"watch_port,omitempty"`
+	WatchGroup           uint32       `protobuf:"varint,3,opt,name=watch_group,json=watchGroup,proto3" json:"watch_group,omitempty"`
+	Actions              []*OfpAction `protobuf:"bytes,4,rep,name=actions,proto3" json:"actions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *OfpBucket) Reset()         { *m = OfpBucket{} }
+func (m *OfpBucket) String() string { return proto.CompactTextString(m) }
+func (*OfpBucket) ProtoMessage()    {}
+func (*OfpBucket) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{30}
+}
+
+func (m *OfpBucket) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpBucket.Unmarshal(m, b)
+}
+func (m *OfpBucket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpBucket.Marshal(b, m, deterministic)
+}
+func (m *OfpBucket) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpBucket.Merge(m, src)
+}
+func (m *OfpBucket) XXX_Size() int {
+	return xxx_messageInfo_OfpBucket.Size(m)
+}
+func (m *OfpBucket) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpBucket.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpBucket proto.InternalMessageInfo
+
+func (m *OfpBucket) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *OfpBucket) GetWatchPort() uint32 {
+	if m != nil {
+		return m.WatchPort
+	}
+	return 0
+}
+
+func (m *OfpBucket) GetWatchGroup() uint32 {
+	if m != nil {
+		return m.WatchGroup
+	}
+	return 0
+}
+
+func (m *OfpBucket) GetActions() []*OfpAction {
+	if m != nil {
+		return m.Actions
+	}
+	return nil
+}
+
+// Group setup and teardown (controller -> datapath).
+type OfpGroupMod struct {
+	//ofp_header header;
+	Command              OfpGroupModCommand `protobuf:"varint,1,opt,name=command,proto3,enum=openflow_13.OfpGroupModCommand" json:"command,omitempty"`
+	Type                 OfpGroupType       `protobuf:"varint,2,opt,name=type,proto3,enum=openflow_13.OfpGroupType" json:"type,omitempty"`
+	GroupId              uint32             `protobuf:"varint,3,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	Buckets              []*OfpBucket       `protobuf:"bytes,4,rep,name=buckets,proto3" json:"buckets,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *OfpGroupMod) Reset()         { *m = OfpGroupMod{} }
+func (m *OfpGroupMod) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupMod) ProtoMessage()    {}
+func (*OfpGroupMod) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{31}
+}
+
+func (m *OfpGroupMod) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupMod.Unmarshal(m, b)
+}
+func (m *OfpGroupMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupMod.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupMod) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupMod.Merge(m, src)
+}
+func (m *OfpGroupMod) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupMod.Size(m)
+}
+func (m *OfpGroupMod) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupMod.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupMod proto.InternalMessageInfo
+
+func (m *OfpGroupMod) GetCommand() OfpGroupModCommand {
+	if m != nil {
+		return m.Command
+	}
+	return OfpGroupModCommand_OFPGC_ADD
+}
+
+func (m *OfpGroupMod) GetType() OfpGroupType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpGroupType_OFPGT_ALL
+}
+
+func (m *OfpGroupMod) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+func (m *OfpGroupMod) GetBuckets() []*OfpBucket {
+	if m != nil {
+		return m.Buckets
+	}
+	return nil
+}
+
+// Send packet (controller -> datapath).
+type OfpPacketOut struct {
+	//ofp_header header;
+	BufferId uint32       `protobuf:"varint,1,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
+	InPort   uint32       `protobuf:"varint,2,opt,name=in_port,json=inPort,proto3" json:"in_port,omitempty"`
+	Actions  []*OfpAction `protobuf:"bytes,3,rep,name=actions,proto3" json:"actions,omitempty"`
+	// The variable size action list is optionally followed by packet data.
+	// This data is only present and meaningful if buffer_id == -1.
+	Data                 []byte   `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpPacketOut) Reset()         { *m = OfpPacketOut{} }
+func (m *OfpPacketOut) String() string { return proto.CompactTextString(m) }
+func (*OfpPacketOut) ProtoMessage()    {}
+func (*OfpPacketOut) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{32}
+}
+
+func (m *OfpPacketOut) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPacketOut.Unmarshal(m, b)
+}
+func (m *OfpPacketOut) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPacketOut.Marshal(b, m, deterministic)
+}
+func (m *OfpPacketOut) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPacketOut.Merge(m, src)
+}
+func (m *OfpPacketOut) XXX_Size() int {
+	return xxx_messageInfo_OfpPacketOut.Size(m)
+}
+func (m *OfpPacketOut) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPacketOut.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPacketOut proto.InternalMessageInfo
+
+func (m *OfpPacketOut) GetBufferId() uint32 {
+	if m != nil {
+		return m.BufferId
+	}
+	return 0
+}
+
+func (m *OfpPacketOut) GetInPort() uint32 {
+	if m != nil {
+		return m.InPort
+	}
+	return 0
+}
+
+func (m *OfpPacketOut) GetActions() []*OfpAction {
+	if m != nil {
+		return m.Actions
+	}
+	return nil
+}
+
+func (m *OfpPacketOut) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Packet received on port (datapath -> controller).
+type OfpPacketIn struct {
+	//ofp_header header;
+	BufferId             uint32            `protobuf:"varint,1,opt,name=buffer_id,json=bufferId,proto3" json:"buffer_id,omitempty"`
+	Reason               OfpPacketInReason `protobuf:"varint,2,opt,name=reason,proto3,enum=openflow_13.OfpPacketInReason" json:"reason,omitempty"`
+	TableId              uint32            `protobuf:"varint,3,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	Cookie               uint64            `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	Match                *OfpMatch         `protobuf:"bytes,5,opt,name=match,proto3" json:"match,omitempty"`
+	Data                 []byte            `protobuf:"bytes,6,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpPacketIn) Reset()         { *m = OfpPacketIn{} }
+func (m *OfpPacketIn) String() string { return proto.CompactTextString(m) }
+func (*OfpPacketIn) ProtoMessage()    {}
+func (*OfpPacketIn) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{33}
+}
+
+func (m *OfpPacketIn) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPacketIn.Unmarshal(m, b)
+}
+func (m *OfpPacketIn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPacketIn.Marshal(b, m, deterministic)
+}
+func (m *OfpPacketIn) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPacketIn.Merge(m, src)
+}
+func (m *OfpPacketIn) XXX_Size() int {
+	return xxx_messageInfo_OfpPacketIn.Size(m)
+}
+func (m *OfpPacketIn) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPacketIn.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPacketIn proto.InternalMessageInfo
+
+func (m *OfpPacketIn) GetBufferId() uint32 {
+	if m != nil {
+		return m.BufferId
+	}
+	return 0
+}
+
+func (m *OfpPacketIn) GetReason() OfpPacketInReason {
+	if m != nil {
+		return m.Reason
+	}
+	return OfpPacketInReason_OFPR_NO_MATCH
+}
+
+func (m *OfpPacketIn) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpPacketIn) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpPacketIn) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+func (m *OfpPacketIn) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Flow removed (datapath -> controller).
+type OfpFlowRemoved struct {
+	//ofp_header header;
+	Cookie               uint64               `protobuf:"varint,1,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	Priority             uint32               `protobuf:"varint,2,opt,name=priority,proto3" json:"priority,omitempty"`
+	Reason               OfpFlowRemovedReason `protobuf:"varint,3,opt,name=reason,proto3,enum=openflow_13.OfpFlowRemovedReason" json:"reason,omitempty"`
+	TableId              uint32               `protobuf:"varint,4,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	DurationSec          uint32               `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32               `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	IdleTimeout          uint32               `protobuf:"varint,7,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
+	HardTimeout          uint32               `protobuf:"varint,8,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
+	PacketCount          uint64               `protobuf:"varint,9,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
+	ByteCount            uint64               `protobuf:"varint,10,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
+	Match                *OfpMatch            `protobuf:"bytes,121,opt,name=match,proto3" json:"match,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *OfpFlowRemoved) Reset()         { *m = OfpFlowRemoved{} }
+func (m *OfpFlowRemoved) String() string { return proto.CompactTextString(m) }
+func (*OfpFlowRemoved) ProtoMessage()    {}
+func (*OfpFlowRemoved) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{34}
+}
+
+func (m *OfpFlowRemoved) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpFlowRemoved.Unmarshal(m, b)
+}
+func (m *OfpFlowRemoved) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpFlowRemoved.Marshal(b, m, deterministic)
+}
+func (m *OfpFlowRemoved) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpFlowRemoved.Merge(m, src)
+}
+func (m *OfpFlowRemoved) XXX_Size() int {
+	return xxx_messageInfo_OfpFlowRemoved.Size(m)
+}
+func (m *OfpFlowRemoved) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpFlowRemoved.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpFlowRemoved proto.InternalMessageInfo
+
+func (m *OfpFlowRemoved) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetReason() OfpFlowRemovedReason {
+	if m != nil {
+		return m.Reason
+	}
+	return OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT
+}
+
+func (m *OfpFlowRemoved) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetIdleTimeout() uint32 {
+	if m != nil {
+		return m.IdleTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetHardTimeout() uint32 {
+	if m != nil {
+		return m.HardTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetPacketCount() uint64 {
+	if m != nil {
+		return m.PacketCount
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetByteCount() uint64 {
+	if m != nil {
+		return m.ByteCount
+	}
+	return 0
+}
+
+func (m *OfpFlowRemoved) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+// Common header for all meter bands
+type OfpMeterBandHeader struct {
+	Type      OfpMeterBandType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMeterBandType" json:"type,omitempty"`
+	Rate      uint32           `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
+	BurstSize uint32           `protobuf:"varint,3,opt,name=burst_size,json=burstSize,proto3" json:"burst_size,omitempty"`
+	// Types that are valid to be assigned to Data:
+	//	*OfpMeterBandHeader_Drop
+	//	*OfpMeterBandHeader_DscpRemark
+	//	*OfpMeterBandHeader_Experimenter
+	Data                 isOfpMeterBandHeader_Data `protobuf_oneof:"data"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *OfpMeterBandHeader) Reset()         { *m = OfpMeterBandHeader{} }
+func (m *OfpMeterBandHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterBandHeader) ProtoMessage()    {}
+func (*OfpMeterBandHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{35}
+}
+
+func (m *OfpMeterBandHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterBandHeader.Unmarshal(m, b)
+}
+func (m *OfpMeterBandHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterBandHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterBandHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterBandHeader.Merge(m, src)
+}
+func (m *OfpMeterBandHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterBandHeader.Size(m)
+}
+func (m *OfpMeterBandHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterBandHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterBandHeader proto.InternalMessageInfo
+
+func (m *OfpMeterBandHeader) GetType() OfpMeterBandType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpMeterBandType_OFPMBT_INVALID
+}
+
+func (m *OfpMeterBandHeader) GetRate() uint32 {
+	if m != nil {
+		return m.Rate
+	}
+	return 0
+}
+
+func (m *OfpMeterBandHeader) GetBurstSize() uint32 {
+	if m != nil {
+		return m.BurstSize
+	}
+	return 0
+}
+
+type isOfpMeterBandHeader_Data interface {
+	isOfpMeterBandHeader_Data()
+}
+
+type OfpMeterBandHeader_Drop struct {
+	Drop *OfpMeterBandDrop `protobuf:"bytes,4,opt,name=drop,proto3,oneof"`
+}
+
+type OfpMeterBandHeader_DscpRemark struct {
+	DscpRemark *OfpMeterBandDscpRemark `protobuf:"bytes,5,opt,name=dscp_remark,json=dscpRemark,proto3,oneof"`
+}
+
+type OfpMeterBandHeader_Experimenter struct {
+	Experimenter *OfpMeterBandExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
+}
+
+func (*OfpMeterBandHeader_Drop) isOfpMeterBandHeader_Data() {}
+
+func (*OfpMeterBandHeader_DscpRemark) isOfpMeterBandHeader_Data() {}
+
+func (*OfpMeterBandHeader_Experimenter) isOfpMeterBandHeader_Data() {}
+
+func (m *OfpMeterBandHeader) GetData() isOfpMeterBandHeader_Data {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+func (m *OfpMeterBandHeader) GetDrop() *OfpMeterBandDrop {
+	if x, ok := m.GetData().(*OfpMeterBandHeader_Drop); ok {
+		return x.Drop
+	}
+	return nil
+}
+
+func (m *OfpMeterBandHeader) GetDscpRemark() *OfpMeterBandDscpRemark {
+	if x, ok := m.GetData().(*OfpMeterBandHeader_DscpRemark); ok {
+		return x.DscpRemark
+	}
+	return nil
+}
+
+func (m *OfpMeterBandHeader) GetExperimenter() *OfpMeterBandExperimenter {
+	if x, ok := m.GetData().(*OfpMeterBandHeader_Experimenter); ok {
+		return x.Experimenter
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpMeterBandHeader) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpMeterBandHeader_Drop)(nil),
+		(*OfpMeterBandHeader_DscpRemark)(nil),
+		(*OfpMeterBandHeader_Experimenter)(nil),
+	}
+}
+
+// OFPMBT_DROP band - drop packets
+type OfpMeterBandDrop struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterBandDrop) Reset()         { *m = OfpMeterBandDrop{} }
+func (m *OfpMeterBandDrop) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterBandDrop) ProtoMessage()    {}
+func (*OfpMeterBandDrop) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{36}
+}
+
+func (m *OfpMeterBandDrop) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterBandDrop.Unmarshal(m, b)
+}
+func (m *OfpMeterBandDrop) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterBandDrop.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterBandDrop) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterBandDrop.Merge(m, src)
+}
+func (m *OfpMeterBandDrop) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterBandDrop.Size(m)
+}
+func (m *OfpMeterBandDrop) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterBandDrop.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterBandDrop proto.InternalMessageInfo
+
+// OFPMBT_DSCP_REMARK band - Remark DSCP in the IP header
+type OfpMeterBandDscpRemark struct {
+	PrecLevel            uint32   `protobuf:"varint,1,opt,name=prec_level,json=precLevel,proto3" json:"prec_level,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterBandDscpRemark) Reset()         { *m = OfpMeterBandDscpRemark{} }
+func (m *OfpMeterBandDscpRemark) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterBandDscpRemark) ProtoMessage()    {}
+func (*OfpMeterBandDscpRemark) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{37}
+}
+
+func (m *OfpMeterBandDscpRemark) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterBandDscpRemark.Unmarshal(m, b)
+}
+func (m *OfpMeterBandDscpRemark) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterBandDscpRemark.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterBandDscpRemark) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterBandDscpRemark.Merge(m, src)
+}
+func (m *OfpMeterBandDscpRemark) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterBandDscpRemark.Size(m)
+}
+func (m *OfpMeterBandDscpRemark) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterBandDscpRemark.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterBandDscpRemark proto.InternalMessageInfo
+
+func (m *OfpMeterBandDscpRemark) GetPrecLevel() uint32 {
+	if m != nil {
+		return m.PrecLevel
+	}
+	return 0
+}
+
+// OFPMBT_EXPERIMENTER band - Experimenter type.
+// The rest of the band is experimenter-defined.
+type OfpMeterBandExperimenter struct {
+	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterBandExperimenter) Reset()         { *m = OfpMeterBandExperimenter{} }
+func (m *OfpMeterBandExperimenter) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterBandExperimenter) ProtoMessage()    {}
+func (*OfpMeterBandExperimenter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{38}
+}
+
+func (m *OfpMeterBandExperimenter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterBandExperimenter.Unmarshal(m, b)
+}
+func (m *OfpMeterBandExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterBandExperimenter.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterBandExperimenter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterBandExperimenter.Merge(m, src)
+}
+func (m *OfpMeterBandExperimenter) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterBandExperimenter.Size(m)
+}
+func (m *OfpMeterBandExperimenter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterBandExperimenter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterBandExperimenter proto.InternalMessageInfo
+
+func (m *OfpMeterBandExperimenter) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+// Meter configuration. OFPT_METER_MOD.
+type OfpMeterMod struct {
+	Command              OfpMeterModCommand    `protobuf:"varint,1,opt,name=command,proto3,enum=openflow_13.OfpMeterModCommand" json:"command,omitempty"`
+	Flags                uint32                `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
+	MeterId              uint32                `protobuf:"varint,3,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
+	Bands                []*OfpMeterBandHeader `protobuf:"bytes,4,rep,name=bands,proto3" json:"bands,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpMeterMod) Reset()         { *m = OfpMeterMod{} }
+func (m *OfpMeterMod) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterMod) ProtoMessage()    {}
+func (*OfpMeterMod) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{39}
+}
+
+func (m *OfpMeterMod) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterMod.Unmarshal(m, b)
+}
+func (m *OfpMeterMod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterMod.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterMod) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterMod.Merge(m, src)
+}
+func (m *OfpMeterMod) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterMod.Size(m)
+}
+func (m *OfpMeterMod) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterMod.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterMod proto.InternalMessageInfo
+
+func (m *OfpMeterMod) GetCommand() OfpMeterModCommand {
+	if m != nil {
+		return m.Command
+	}
+	return OfpMeterModCommand_OFPMC_ADD
+}
+
+func (m *OfpMeterMod) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpMeterMod) GetMeterId() uint32 {
+	if m != nil {
+		return m.MeterId
+	}
+	return 0
+}
+
+func (m *OfpMeterMod) GetBands() []*OfpMeterBandHeader {
+	if m != nil {
+		return m.Bands
+	}
+	return nil
+}
+
+// OFPT_ERROR: Error message (datapath -> controller).
+type OfpErrorMsg struct {
+	Header               *OfpHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	Type                 uint32     `protobuf:"varint,2,opt,name=type,proto3" json:"type,omitempty"`
+	Code                 uint32     `protobuf:"varint,3,opt,name=code,proto3" json:"code,omitempty"`
+	Data                 []byte     `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
+	XXX_unrecognized     []byte     `json:"-"`
+	XXX_sizecache        int32      `json:"-"`
+}
+
+func (m *OfpErrorMsg) Reset()         { *m = OfpErrorMsg{} }
+func (m *OfpErrorMsg) String() string { return proto.CompactTextString(m) }
+func (*OfpErrorMsg) ProtoMessage()    {}
+func (*OfpErrorMsg) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{40}
+}
+
+func (m *OfpErrorMsg) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpErrorMsg.Unmarshal(m, b)
+}
+func (m *OfpErrorMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpErrorMsg.Marshal(b, m, deterministic)
+}
+func (m *OfpErrorMsg) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpErrorMsg.Merge(m, src)
+}
+func (m *OfpErrorMsg) XXX_Size() int {
+	return xxx_messageInfo_OfpErrorMsg.Size(m)
+}
+func (m *OfpErrorMsg) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpErrorMsg.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpErrorMsg proto.InternalMessageInfo
+
+func (m *OfpErrorMsg) GetHeader() *OfpHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *OfpErrorMsg) GetType() uint32 {
+	if m != nil {
+		return m.Type
+	}
+	return 0
+}
+
+func (m *OfpErrorMsg) GetCode() uint32 {
+	if m != nil {
+		return m.Code
+	}
+	return 0
+}
+
+func (m *OfpErrorMsg) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// OFPET_EXPERIMENTER: Error message (datapath -> controller).
+type OfpErrorExperimenterMsg struct {
+	Type                 uint32   `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
+	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
+	Experimenter         uint32   `protobuf:"varint,3,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	Data                 []byte   `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpErrorExperimenterMsg) Reset()         { *m = OfpErrorExperimenterMsg{} }
+func (m *OfpErrorExperimenterMsg) String() string { return proto.CompactTextString(m) }
+func (*OfpErrorExperimenterMsg) ProtoMessage()    {}
+func (*OfpErrorExperimenterMsg) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{41}
+}
+
+func (m *OfpErrorExperimenterMsg) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpErrorExperimenterMsg.Unmarshal(m, b)
+}
+func (m *OfpErrorExperimenterMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpErrorExperimenterMsg.Marshal(b, m, deterministic)
+}
+func (m *OfpErrorExperimenterMsg) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpErrorExperimenterMsg.Merge(m, src)
+}
+func (m *OfpErrorExperimenterMsg) XXX_Size() int {
+	return xxx_messageInfo_OfpErrorExperimenterMsg.Size(m)
+}
+func (m *OfpErrorExperimenterMsg) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpErrorExperimenterMsg.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpErrorExperimenterMsg proto.InternalMessageInfo
+
+func (m *OfpErrorExperimenterMsg) GetType() uint32 {
+	if m != nil {
+		return m.Type
+	}
+	return 0
+}
+
+func (m *OfpErrorExperimenterMsg) GetExpType() uint32 {
+	if m != nil {
+		return m.ExpType
+	}
+	return 0
+}
+
+func (m *OfpErrorExperimenterMsg) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpErrorExperimenterMsg) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+type OfpMultipartRequest struct {
+	//ofp_header header;
+	Type                 OfpMultipartType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMultipartType" json:"type,omitempty"`
+	Flags                uint32           `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
+	Body                 []byte           `protobuf:"bytes,3,opt,name=body,proto3" json:"body,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *OfpMultipartRequest) Reset()         { *m = OfpMultipartRequest{} }
+func (m *OfpMultipartRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpMultipartRequest) ProtoMessage()    {}
+func (*OfpMultipartRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{42}
+}
+
+func (m *OfpMultipartRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMultipartRequest.Unmarshal(m, b)
+}
+func (m *OfpMultipartRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMultipartRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpMultipartRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMultipartRequest.Merge(m, src)
+}
+func (m *OfpMultipartRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpMultipartRequest.Size(m)
+}
+func (m *OfpMultipartRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMultipartRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMultipartRequest proto.InternalMessageInfo
+
+func (m *OfpMultipartRequest) GetType() OfpMultipartType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpMultipartType_OFPMP_DESC
+}
+
+func (m *OfpMultipartRequest) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpMultipartRequest) GetBody() []byte {
+	if m != nil {
+		return m.Body
+	}
+	return nil
+}
+
+type OfpMultipartReply struct {
+	//ofp_header header;
+	Type                 OfpMultipartType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpMultipartType" json:"type,omitempty"`
+	Flags                uint32           `protobuf:"varint,2,opt,name=flags,proto3" json:"flags,omitempty"`
+	Body                 []byte           `protobuf:"bytes,3,opt,name=body,proto3" json:"body,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *OfpMultipartReply) Reset()         { *m = OfpMultipartReply{} }
+func (m *OfpMultipartReply) String() string { return proto.CompactTextString(m) }
+func (*OfpMultipartReply) ProtoMessage()    {}
+func (*OfpMultipartReply) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{43}
+}
+
+func (m *OfpMultipartReply) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMultipartReply.Unmarshal(m, b)
+}
+func (m *OfpMultipartReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMultipartReply.Marshal(b, m, deterministic)
+}
+func (m *OfpMultipartReply) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMultipartReply.Merge(m, src)
+}
+func (m *OfpMultipartReply) XXX_Size() int {
+	return xxx_messageInfo_OfpMultipartReply.Size(m)
+}
+func (m *OfpMultipartReply) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMultipartReply.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMultipartReply proto.InternalMessageInfo
+
+func (m *OfpMultipartReply) GetType() OfpMultipartType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpMultipartType_OFPMP_DESC
+}
+
+func (m *OfpMultipartReply) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpMultipartReply) GetBody() []byte {
+	if m != nil {
+		return m.Body
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_DESC request.  Each entry is a NULL-terminated
+// ASCII string.
+type OfpDesc struct {
+	MfrDesc              string   `protobuf:"bytes,1,opt,name=mfr_desc,json=mfrDesc,proto3" json:"mfr_desc,omitempty"`
+	HwDesc               string   `protobuf:"bytes,2,opt,name=hw_desc,json=hwDesc,proto3" json:"hw_desc,omitempty"`
+	SwDesc               string   `protobuf:"bytes,3,opt,name=sw_desc,json=swDesc,proto3" json:"sw_desc,omitempty"`
+	SerialNum            string   `protobuf:"bytes,4,opt,name=serial_num,json=serialNum,proto3" json:"serial_num,omitempty"`
+	DpDesc               string   `protobuf:"bytes,5,opt,name=dp_desc,json=dpDesc,proto3" json:"dp_desc,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpDesc) Reset()         { *m = OfpDesc{} }
+func (m *OfpDesc) String() string { return proto.CompactTextString(m) }
+func (*OfpDesc) ProtoMessage()    {}
+func (*OfpDesc) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{44}
+}
+
+func (m *OfpDesc) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpDesc.Unmarshal(m, b)
+}
+func (m *OfpDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpDesc.Marshal(b, m, deterministic)
+}
+func (m *OfpDesc) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpDesc.Merge(m, src)
+}
+func (m *OfpDesc) XXX_Size() int {
+	return xxx_messageInfo_OfpDesc.Size(m)
+}
+func (m *OfpDesc) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpDesc.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpDesc proto.InternalMessageInfo
+
+func (m *OfpDesc) GetMfrDesc() string {
+	if m != nil {
+		return m.MfrDesc
+	}
+	return ""
+}
+
+func (m *OfpDesc) GetHwDesc() string {
+	if m != nil {
+		return m.HwDesc
+	}
+	return ""
+}
+
+func (m *OfpDesc) GetSwDesc() string {
+	if m != nil {
+		return m.SwDesc
+	}
+	return ""
+}
+
+func (m *OfpDesc) GetSerialNum() string {
+	if m != nil {
+		return m.SerialNum
+	}
+	return ""
+}
+
+func (m *OfpDesc) GetDpDesc() string {
+	if m != nil {
+		return m.DpDesc
+	}
+	return ""
+}
+
+// Body for ofp_multipart_request of type OFPMP_FLOW.
+type OfpFlowStatsRequest struct {
+	TableId              uint32    `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	OutPort              uint32    `protobuf:"varint,2,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
+	OutGroup             uint32    `protobuf:"varint,3,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
+	Cookie               uint64    `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	CookieMask           uint64    `protobuf:"varint,5,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
+	Match                *OfpMatch `protobuf:"bytes,6,opt,name=match,proto3" json:"match,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *OfpFlowStatsRequest) Reset()         { *m = OfpFlowStatsRequest{} }
+func (m *OfpFlowStatsRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpFlowStatsRequest) ProtoMessage()    {}
+func (*OfpFlowStatsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{45}
+}
+
+func (m *OfpFlowStatsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpFlowStatsRequest.Unmarshal(m, b)
+}
+func (m *OfpFlowStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpFlowStatsRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpFlowStatsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpFlowStatsRequest.Merge(m, src)
+}
+func (m *OfpFlowStatsRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpFlowStatsRequest.Size(m)
+}
+func (m *OfpFlowStatsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpFlowStatsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpFlowStatsRequest proto.InternalMessageInfo
+
+func (m *OfpFlowStatsRequest) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpFlowStatsRequest) GetOutPort() uint32 {
+	if m != nil {
+		return m.OutPort
+	}
+	return 0
+}
+
+func (m *OfpFlowStatsRequest) GetOutGroup() uint32 {
+	if m != nil {
+		return m.OutGroup
+	}
+	return 0
+}
+
+func (m *OfpFlowStatsRequest) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpFlowStatsRequest) GetCookieMask() uint64 {
+	if m != nil {
+		return m.CookieMask
+	}
+	return 0
+}
+
+func (m *OfpFlowStatsRequest) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_FLOW request.
+type OfpFlowStats struct {
+	Id                   uint64            `protobuf:"varint,14,opt,name=id,proto3" json:"id,omitempty"`
+	TableId              uint32            `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	DurationSec          uint32            `protobuf:"varint,2,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32            `protobuf:"varint,3,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	Priority             uint32            `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
+	IdleTimeout          uint32            `protobuf:"varint,5,opt,name=idle_timeout,json=idleTimeout,proto3" json:"idle_timeout,omitempty"`
+	HardTimeout          uint32            `protobuf:"varint,6,opt,name=hard_timeout,json=hardTimeout,proto3" json:"hard_timeout,omitempty"`
+	Flags                uint32            `protobuf:"varint,7,opt,name=flags,proto3" json:"flags,omitempty"`
+	Cookie               uint64            `protobuf:"varint,8,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	PacketCount          uint64            `protobuf:"varint,9,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
+	ByteCount            uint64            `protobuf:"varint,10,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
+	Match                *OfpMatch         `protobuf:"bytes,12,opt,name=match,proto3" json:"match,omitempty"`
+	Instructions         []*OfpInstruction `protobuf:"bytes,13,rep,name=instructions,proto3" json:"instructions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpFlowStats) Reset()         { *m = OfpFlowStats{} }
+func (m *OfpFlowStats) String() string { return proto.CompactTextString(m) }
+func (*OfpFlowStats) ProtoMessage()    {}
+func (*OfpFlowStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{46}
+}
+
+func (m *OfpFlowStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpFlowStats.Unmarshal(m, b)
+}
+func (m *OfpFlowStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpFlowStats.Marshal(b, m, deterministic)
+}
+func (m *OfpFlowStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpFlowStats.Merge(m, src)
+}
+func (m *OfpFlowStats) XXX_Size() int {
+	return xxx_messageInfo_OfpFlowStats.Size(m)
+}
+func (m *OfpFlowStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpFlowStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpFlowStats proto.InternalMessageInfo
+
+func (m *OfpFlowStats) GetId() uint64 {
+	if m != nil {
+		return m.Id
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetIdleTimeout() uint32 {
+	if m != nil {
+		return m.IdleTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetHardTimeout() uint32 {
+	if m != nil {
+		return m.HardTimeout
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetPacketCount() uint64 {
+	if m != nil {
+		return m.PacketCount
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetByteCount() uint64 {
+	if m != nil {
+		return m.ByteCount
+	}
+	return 0
+}
+
+func (m *OfpFlowStats) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+func (m *OfpFlowStats) GetInstructions() []*OfpInstruction {
+	if m != nil {
+		return m.Instructions
+	}
+	return nil
+}
+
+// Body for ofp_multipart_request of type OFPMP_AGGREGATE.
+type OfpAggregateStatsRequest struct {
+	TableId              uint32    `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	OutPort              uint32    `protobuf:"varint,2,opt,name=out_port,json=outPort,proto3" json:"out_port,omitempty"`
+	OutGroup             uint32    `protobuf:"varint,3,opt,name=out_group,json=outGroup,proto3" json:"out_group,omitempty"`
+	Cookie               uint64    `protobuf:"varint,4,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	CookieMask           uint64    `protobuf:"varint,5,opt,name=cookie_mask,json=cookieMask,proto3" json:"cookie_mask,omitempty"`
+	Match                *OfpMatch `protobuf:"bytes,6,opt,name=match,proto3" json:"match,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *OfpAggregateStatsRequest) Reset()         { *m = OfpAggregateStatsRequest{} }
+func (m *OfpAggregateStatsRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpAggregateStatsRequest) ProtoMessage()    {}
+func (*OfpAggregateStatsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{47}
+}
+
+func (m *OfpAggregateStatsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpAggregateStatsRequest.Unmarshal(m, b)
+}
+func (m *OfpAggregateStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpAggregateStatsRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpAggregateStatsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpAggregateStatsRequest.Merge(m, src)
+}
+func (m *OfpAggregateStatsRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpAggregateStatsRequest.Size(m)
+}
+func (m *OfpAggregateStatsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpAggregateStatsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpAggregateStatsRequest proto.InternalMessageInfo
+
+func (m *OfpAggregateStatsRequest) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsRequest) GetOutPort() uint32 {
+	if m != nil {
+		return m.OutPort
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsRequest) GetOutGroup() uint32 {
+	if m != nil {
+		return m.OutGroup
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsRequest) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsRequest) GetCookieMask() uint64 {
+	if m != nil {
+		return m.CookieMask
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsRequest) GetMatch() *OfpMatch {
+	if m != nil {
+		return m.Match
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_AGGREGATE request.
+type OfpAggregateStatsReply struct {
+	PacketCount          uint64   `protobuf:"varint,1,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
+	ByteCount            uint64   `protobuf:"varint,2,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
+	FlowCount            uint32   `protobuf:"varint,3,opt,name=flow_count,json=flowCount,proto3" json:"flow_count,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpAggregateStatsReply) Reset()         { *m = OfpAggregateStatsReply{} }
+func (m *OfpAggregateStatsReply) String() string { return proto.CompactTextString(m) }
+func (*OfpAggregateStatsReply) ProtoMessage()    {}
+func (*OfpAggregateStatsReply) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{48}
+}
+
+func (m *OfpAggregateStatsReply) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpAggregateStatsReply.Unmarshal(m, b)
+}
+func (m *OfpAggregateStatsReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpAggregateStatsReply.Marshal(b, m, deterministic)
+}
+func (m *OfpAggregateStatsReply) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpAggregateStatsReply.Merge(m, src)
+}
+func (m *OfpAggregateStatsReply) XXX_Size() int {
+	return xxx_messageInfo_OfpAggregateStatsReply.Size(m)
+}
+func (m *OfpAggregateStatsReply) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpAggregateStatsReply.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpAggregateStatsReply proto.InternalMessageInfo
+
+func (m *OfpAggregateStatsReply) GetPacketCount() uint64 {
+	if m != nil {
+		return m.PacketCount
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsReply) GetByteCount() uint64 {
+	if m != nil {
+		return m.ByteCount
+	}
+	return 0
+}
+
+func (m *OfpAggregateStatsReply) GetFlowCount() uint32 {
+	if m != nil {
+		return m.FlowCount
+	}
+	return 0
+}
+
+// Common header for all Table Feature Properties
+type OfpTableFeatureProperty struct {
+	Type OfpTableFeaturePropType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpTableFeaturePropType" json:"type,omitempty"`
+	// Types that are valid to be assigned to Value:
+	//	*OfpTableFeatureProperty_Instructions
+	//	*OfpTableFeatureProperty_NextTables
+	//	*OfpTableFeatureProperty_Actions
+	//	*OfpTableFeatureProperty_Oxm
+	//	*OfpTableFeatureProperty_Experimenter
+	Value                isOfpTableFeatureProperty_Value `protobuf_oneof:"value"`
+	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
+	XXX_unrecognized     []byte                          `json:"-"`
+	XXX_sizecache        int32                           `json:"-"`
+}
+
+func (m *OfpTableFeatureProperty) Reset()         { *m = OfpTableFeatureProperty{} }
+func (m *OfpTableFeatureProperty) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeatureProperty) ProtoMessage()    {}
+func (*OfpTableFeatureProperty) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{49}
+}
+
+func (m *OfpTableFeatureProperty) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeatureProperty.Unmarshal(m, b)
+}
+func (m *OfpTableFeatureProperty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeatureProperty.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeatureProperty) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeatureProperty.Merge(m, src)
+}
+func (m *OfpTableFeatureProperty) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeatureProperty.Size(m)
+}
+func (m *OfpTableFeatureProperty) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeatureProperty.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeatureProperty proto.InternalMessageInfo
+
+func (m *OfpTableFeatureProperty) GetType() OfpTableFeaturePropType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS
+}
+
+type isOfpTableFeatureProperty_Value interface {
+	isOfpTableFeatureProperty_Value()
+}
+
+type OfpTableFeatureProperty_Instructions struct {
+	Instructions *OfpTableFeaturePropInstructions `protobuf:"bytes,2,opt,name=instructions,proto3,oneof"`
+}
+
+type OfpTableFeatureProperty_NextTables struct {
+	NextTables *OfpTableFeaturePropNextTables `protobuf:"bytes,3,opt,name=next_tables,json=nextTables,proto3,oneof"`
+}
+
+type OfpTableFeatureProperty_Actions struct {
+	Actions *OfpTableFeaturePropActions `protobuf:"bytes,4,opt,name=actions,proto3,oneof"`
+}
+
+type OfpTableFeatureProperty_Oxm struct {
+	Oxm *OfpTableFeaturePropOxm `protobuf:"bytes,5,opt,name=oxm,proto3,oneof"`
+}
+
+type OfpTableFeatureProperty_Experimenter struct {
+	Experimenter *OfpTableFeaturePropExperimenter `protobuf:"bytes,6,opt,name=experimenter,proto3,oneof"`
+}
+
+func (*OfpTableFeatureProperty_Instructions) isOfpTableFeatureProperty_Value() {}
+
+func (*OfpTableFeatureProperty_NextTables) isOfpTableFeatureProperty_Value() {}
+
+func (*OfpTableFeatureProperty_Actions) isOfpTableFeatureProperty_Value() {}
+
+func (*OfpTableFeatureProperty_Oxm) isOfpTableFeatureProperty_Value() {}
+
+func (*OfpTableFeatureProperty_Experimenter) isOfpTableFeatureProperty_Value() {}
+
+func (m *OfpTableFeatureProperty) GetValue() isOfpTableFeatureProperty_Value {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+func (m *OfpTableFeatureProperty) GetInstructions() *OfpTableFeaturePropInstructions {
+	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Instructions); ok {
+		return x.Instructions
+	}
+	return nil
+}
+
+func (m *OfpTableFeatureProperty) GetNextTables() *OfpTableFeaturePropNextTables {
+	if x, ok := m.GetValue().(*OfpTableFeatureProperty_NextTables); ok {
+		return x.NextTables
+	}
+	return nil
+}
+
+func (m *OfpTableFeatureProperty) GetActions() *OfpTableFeaturePropActions {
+	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Actions); ok {
+		return x.Actions
+	}
+	return nil
+}
+
+func (m *OfpTableFeatureProperty) GetOxm() *OfpTableFeaturePropOxm {
+	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Oxm); ok {
+		return x.Oxm
+	}
+	return nil
+}
+
+func (m *OfpTableFeatureProperty) GetExperimenter() *OfpTableFeaturePropExperimenter {
+	if x, ok := m.GetValue().(*OfpTableFeatureProperty_Experimenter); ok {
+		return x.Experimenter
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OfpTableFeatureProperty) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OfpTableFeatureProperty_Instructions)(nil),
+		(*OfpTableFeatureProperty_NextTables)(nil),
+		(*OfpTableFeatureProperty_Actions)(nil),
+		(*OfpTableFeatureProperty_Oxm)(nil),
+		(*OfpTableFeatureProperty_Experimenter)(nil),
+	}
+}
+
+// Instructions property
+type OfpTableFeaturePropInstructions struct {
+	// One of OFPTFPT_INSTRUCTIONS,
+	//OFPTFPT_INSTRUCTIONS_MISS.
+	Instructions         []*OfpInstruction `protobuf:"bytes,1,rep,name=instructions,proto3" json:"instructions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpTableFeaturePropInstructions) Reset()         { *m = OfpTableFeaturePropInstructions{} }
+func (m *OfpTableFeaturePropInstructions) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeaturePropInstructions) ProtoMessage()    {}
+func (*OfpTableFeaturePropInstructions) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{50}
+}
+
+func (m *OfpTableFeaturePropInstructions) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeaturePropInstructions.Unmarshal(m, b)
+}
+func (m *OfpTableFeaturePropInstructions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeaturePropInstructions.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeaturePropInstructions) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeaturePropInstructions.Merge(m, src)
+}
+func (m *OfpTableFeaturePropInstructions) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeaturePropInstructions.Size(m)
+}
+func (m *OfpTableFeaturePropInstructions) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeaturePropInstructions.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeaturePropInstructions proto.InternalMessageInfo
+
+func (m *OfpTableFeaturePropInstructions) GetInstructions() []*OfpInstruction {
+	if m != nil {
+		return m.Instructions
+	}
+	return nil
+}
+
+// Next Tables property
+type OfpTableFeaturePropNextTables struct {
+	// One of OFPTFPT_NEXT_TABLES,
+	//OFPTFPT_NEXT_TABLES_MISS.
+	NextTableIds         []uint32 `protobuf:"varint,1,rep,packed,name=next_table_ids,json=nextTableIds,proto3" json:"next_table_ids,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpTableFeaturePropNextTables) Reset()         { *m = OfpTableFeaturePropNextTables{} }
+func (m *OfpTableFeaturePropNextTables) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeaturePropNextTables) ProtoMessage()    {}
+func (*OfpTableFeaturePropNextTables) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{51}
+}
+
+func (m *OfpTableFeaturePropNextTables) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeaturePropNextTables.Unmarshal(m, b)
+}
+func (m *OfpTableFeaturePropNextTables) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeaturePropNextTables.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeaturePropNextTables) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeaturePropNextTables.Merge(m, src)
+}
+func (m *OfpTableFeaturePropNextTables) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeaturePropNextTables.Size(m)
+}
+func (m *OfpTableFeaturePropNextTables) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeaturePropNextTables.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeaturePropNextTables proto.InternalMessageInfo
+
+func (m *OfpTableFeaturePropNextTables) GetNextTableIds() []uint32 {
+	if m != nil {
+		return m.NextTableIds
+	}
+	return nil
+}
+
+// Actions property
+type OfpTableFeaturePropActions struct {
+	// One of OFPTFPT_WRITE_ACTIONS,
+	//OFPTFPT_WRITE_ACTIONS_MISS,
+	//OFPTFPT_APPLY_ACTIONS,
+	//OFPTFPT_APPLY_ACTIONS_MISS.
+	Actions              []*OfpAction `protobuf:"bytes,1,rep,name=actions,proto3" json:"actions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *OfpTableFeaturePropActions) Reset()         { *m = OfpTableFeaturePropActions{} }
+func (m *OfpTableFeaturePropActions) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeaturePropActions) ProtoMessage()    {}
+func (*OfpTableFeaturePropActions) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{52}
+}
+
+func (m *OfpTableFeaturePropActions) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeaturePropActions.Unmarshal(m, b)
+}
+func (m *OfpTableFeaturePropActions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeaturePropActions.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeaturePropActions) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeaturePropActions.Merge(m, src)
+}
+func (m *OfpTableFeaturePropActions) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeaturePropActions.Size(m)
+}
+func (m *OfpTableFeaturePropActions) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeaturePropActions.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeaturePropActions proto.InternalMessageInfo
+
+func (m *OfpTableFeaturePropActions) GetActions() []*OfpAction {
+	if m != nil {
+		return m.Actions
+	}
+	return nil
+}
+
+// Match, Wildcard or Set-Field property
+type OfpTableFeaturePropOxm struct {
+	// TODO is this a uint32???
+	OxmIds               []uint32 `protobuf:"varint,3,rep,packed,name=oxm_ids,json=oxmIds,proto3" json:"oxm_ids,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpTableFeaturePropOxm) Reset()         { *m = OfpTableFeaturePropOxm{} }
+func (m *OfpTableFeaturePropOxm) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeaturePropOxm) ProtoMessage()    {}
+func (*OfpTableFeaturePropOxm) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{53}
+}
+
+func (m *OfpTableFeaturePropOxm) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeaturePropOxm.Unmarshal(m, b)
+}
+func (m *OfpTableFeaturePropOxm) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeaturePropOxm.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeaturePropOxm) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeaturePropOxm.Merge(m, src)
+}
+func (m *OfpTableFeaturePropOxm) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeaturePropOxm.Size(m)
+}
+func (m *OfpTableFeaturePropOxm) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeaturePropOxm.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeaturePropOxm proto.InternalMessageInfo
+
+func (m *OfpTableFeaturePropOxm) GetOxmIds() []uint32 {
+	if m != nil {
+		return m.OxmIds
+	}
+	return nil
+}
+
+// Experimenter table feature property
+type OfpTableFeaturePropExperimenter struct {
+	// One of OFPTFPT_EXPERIMENTER,
+	//OFPTFPT_EXPERIMENTER_MISS.
+	Experimenter         uint32   `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	ExpType              uint32   `protobuf:"varint,3,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
+	ExperimenterData     []uint32 `protobuf:"varint,4,rep,packed,name=experimenter_data,json=experimenterData,proto3" json:"experimenter_data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpTableFeaturePropExperimenter) Reset()         { *m = OfpTableFeaturePropExperimenter{} }
+func (m *OfpTableFeaturePropExperimenter) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeaturePropExperimenter) ProtoMessage()    {}
+func (*OfpTableFeaturePropExperimenter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{54}
+}
+
+func (m *OfpTableFeaturePropExperimenter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Unmarshal(m, b)
+}
+func (m *OfpTableFeaturePropExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeaturePropExperimenter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeaturePropExperimenter.Merge(m, src)
+}
+func (m *OfpTableFeaturePropExperimenter) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeaturePropExperimenter.Size(m)
+}
+func (m *OfpTableFeaturePropExperimenter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeaturePropExperimenter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeaturePropExperimenter proto.InternalMessageInfo
+
+func (m *OfpTableFeaturePropExperimenter) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpTableFeaturePropExperimenter) GetExpType() uint32 {
+	if m != nil {
+		return m.ExpType
+	}
+	return 0
+}
+
+func (m *OfpTableFeaturePropExperimenter) GetExperimenterData() []uint32 {
+	if m != nil {
+		return m.ExperimenterData
+	}
+	return nil
+}
+
+// Body for ofp_multipart_request of type OFPMP_TABLE_FEATURES./
+// Body of reply to OFPMP_TABLE_FEATURES request.
+type OfpTableFeatures struct {
+	TableId       uint32 `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	Name          string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+	MetadataMatch uint64 `protobuf:"varint,3,opt,name=metadata_match,json=metadataMatch,proto3" json:"metadata_match,omitempty"`
+	MetadataWrite uint64 `protobuf:"varint,4,opt,name=metadata_write,json=metadataWrite,proto3" json:"metadata_write,omitempty"`
+	Config        uint32 `protobuf:"varint,5,opt,name=config,proto3" json:"config,omitempty"`
+	MaxEntries    uint32 `protobuf:"varint,6,opt,name=max_entries,json=maxEntries,proto3" json:"max_entries,omitempty"`
+	// Table Feature Property list
+	Properties           []*OfpTableFeatureProperty `protobuf:"bytes,7,rep,name=properties,proto3" json:"properties,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
+	XXX_unrecognized     []byte                     `json:"-"`
+	XXX_sizecache        int32                      `json:"-"`
+}
+
+func (m *OfpTableFeatures) Reset()         { *m = OfpTableFeatures{} }
+func (m *OfpTableFeatures) String() string { return proto.CompactTextString(m) }
+func (*OfpTableFeatures) ProtoMessage()    {}
+func (*OfpTableFeatures) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{55}
+}
+
+func (m *OfpTableFeatures) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableFeatures.Unmarshal(m, b)
+}
+func (m *OfpTableFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableFeatures.Marshal(b, m, deterministic)
+}
+func (m *OfpTableFeatures) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableFeatures.Merge(m, src)
+}
+func (m *OfpTableFeatures) XXX_Size() int {
+	return xxx_messageInfo_OfpTableFeatures.Size(m)
+}
+func (m *OfpTableFeatures) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableFeatures.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableFeatures proto.InternalMessageInfo
+
+func (m *OfpTableFeatures) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpTableFeatures) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *OfpTableFeatures) GetMetadataMatch() uint64 {
+	if m != nil {
+		return m.MetadataMatch
+	}
+	return 0
+}
+
+func (m *OfpTableFeatures) GetMetadataWrite() uint64 {
+	if m != nil {
+		return m.MetadataWrite
+	}
+	return 0
+}
+
+func (m *OfpTableFeatures) GetConfig() uint32 {
+	if m != nil {
+		return m.Config
+	}
+	return 0
+}
+
+func (m *OfpTableFeatures) GetMaxEntries() uint32 {
+	if m != nil {
+		return m.MaxEntries
+	}
+	return 0
+}
+
+func (m *OfpTableFeatures) GetProperties() []*OfpTableFeatureProperty {
+	if m != nil {
+		return m.Properties
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_TABLE request.
+type OfpTableStats struct {
+	TableId              uint32   `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"`
+	ActiveCount          uint32   `protobuf:"varint,2,opt,name=active_count,json=activeCount,proto3" json:"active_count,omitempty"`
+	LookupCount          uint64   `protobuf:"varint,3,opt,name=lookup_count,json=lookupCount,proto3" json:"lookup_count,omitempty"`
+	MatchedCount         uint64   `protobuf:"varint,4,opt,name=matched_count,json=matchedCount,proto3" json:"matched_count,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpTableStats) Reset()         { *m = OfpTableStats{} }
+func (m *OfpTableStats) String() string { return proto.CompactTextString(m) }
+func (*OfpTableStats) ProtoMessage()    {}
+func (*OfpTableStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{56}
+}
+
+func (m *OfpTableStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpTableStats.Unmarshal(m, b)
+}
+func (m *OfpTableStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpTableStats.Marshal(b, m, deterministic)
+}
+func (m *OfpTableStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpTableStats.Merge(m, src)
+}
+func (m *OfpTableStats) XXX_Size() int {
+	return xxx_messageInfo_OfpTableStats.Size(m)
+}
+func (m *OfpTableStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpTableStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpTableStats proto.InternalMessageInfo
+
+func (m *OfpTableStats) GetTableId() uint32 {
+	if m != nil {
+		return m.TableId
+	}
+	return 0
+}
+
+func (m *OfpTableStats) GetActiveCount() uint32 {
+	if m != nil {
+		return m.ActiveCount
+	}
+	return 0
+}
+
+func (m *OfpTableStats) GetLookupCount() uint64 {
+	if m != nil {
+		return m.LookupCount
+	}
+	return 0
+}
+
+func (m *OfpTableStats) GetMatchedCount() uint64 {
+	if m != nil {
+		return m.MatchedCount
+	}
+	return 0
+}
+
+// Body for ofp_multipart_request of type OFPMP_PORT.
+type OfpPortStatsRequest struct {
+	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpPortStatsRequest) Reset()         { *m = OfpPortStatsRequest{} }
+func (m *OfpPortStatsRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpPortStatsRequest) ProtoMessage()    {}
+func (*OfpPortStatsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{57}
+}
+
+func (m *OfpPortStatsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPortStatsRequest.Unmarshal(m, b)
+}
+func (m *OfpPortStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPortStatsRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpPortStatsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPortStatsRequest.Merge(m, src)
+}
+func (m *OfpPortStatsRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpPortStatsRequest.Size(m)
+}
+func (m *OfpPortStatsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPortStatsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPortStatsRequest proto.InternalMessageInfo
+
+func (m *OfpPortStatsRequest) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+// Body of reply to OFPMP_PORT request. If a counter is unsupported, set
+// the field to all ones.
+type OfpPortStats struct {
+	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	RxPackets            uint64   `protobuf:"varint,2,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	TxPackets            uint64   `protobuf:"varint,3,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	RxBytes              uint64   `protobuf:"varint,4,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	TxBytes              uint64   `protobuf:"varint,5,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	RxDropped            uint64   `protobuf:"varint,6,opt,name=rx_dropped,json=rxDropped,proto3" json:"rx_dropped,omitempty"`
+	TxDropped            uint64   `protobuf:"varint,7,opt,name=tx_dropped,json=txDropped,proto3" json:"tx_dropped,omitempty"`
+	RxErrors             uint64   `protobuf:"varint,8,opt,name=rx_errors,json=rxErrors,proto3" json:"rx_errors,omitempty"`
+	TxErrors             uint64   `protobuf:"varint,9,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
+	RxFrameErr           uint64   `protobuf:"varint,10,opt,name=rx_frame_err,json=rxFrameErr,proto3" json:"rx_frame_err,omitempty"`
+	RxOverErr            uint64   `protobuf:"varint,11,opt,name=rx_over_err,json=rxOverErr,proto3" json:"rx_over_err,omitempty"`
+	RxCrcErr             uint64   `protobuf:"varint,12,opt,name=rx_crc_err,json=rxCrcErr,proto3" json:"rx_crc_err,omitempty"`
+	Collisions           uint64   `protobuf:"varint,13,opt,name=collisions,proto3" json:"collisions,omitempty"`
+	DurationSec          uint32   `protobuf:"varint,14,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32   `protobuf:"varint,15,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpPortStats) Reset()         { *m = OfpPortStats{} }
+func (m *OfpPortStats) String() string { return proto.CompactTextString(m) }
+func (*OfpPortStats) ProtoMessage()    {}
+func (*OfpPortStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{58}
+}
+
+func (m *OfpPortStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPortStats.Unmarshal(m, b)
+}
+func (m *OfpPortStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPortStats.Marshal(b, m, deterministic)
+}
+func (m *OfpPortStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPortStats.Merge(m, src)
+}
+func (m *OfpPortStats) XXX_Size() int {
+	return xxx_messageInfo_OfpPortStats.Size(m)
+}
+func (m *OfpPortStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPortStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPortStats proto.InternalMessageInfo
+
+func (m *OfpPortStats) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxDropped() uint64 {
+	if m != nil {
+		return m.RxDropped
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetTxDropped() uint64 {
+	if m != nil {
+		return m.TxDropped
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxErrors() uint64 {
+	if m != nil {
+		return m.RxErrors
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetTxErrors() uint64 {
+	if m != nil {
+		return m.TxErrors
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxFrameErr() uint64 {
+	if m != nil {
+		return m.RxFrameErr
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxOverErr() uint64 {
+	if m != nil {
+		return m.RxOverErr
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetRxCrcErr() uint64 {
+	if m != nil {
+		return m.RxCrcErr
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetCollisions() uint64 {
+	if m != nil {
+		return m.Collisions
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpPortStats) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+// Body of OFPMP_GROUP request.
+type OfpGroupStatsRequest struct {
+	GroupId              uint32   `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpGroupStatsRequest) Reset()         { *m = OfpGroupStatsRequest{} }
+func (m *OfpGroupStatsRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupStatsRequest) ProtoMessage()    {}
+func (*OfpGroupStatsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{59}
+}
+
+func (m *OfpGroupStatsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupStatsRequest.Unmarshal(m, b)
+}
+func (m *OfpGroupStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupStatsRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupStatsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupStatsRequest.Merge(m, src)
+}
+func (m *OfpGroupStatsRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupStatsRequest.Size(m)
+}
+func (m *OfpGroupStatsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupStatsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupStatsRequest proto.InternalMessageInfo
+
+func (m *OfpGroupStatsRequest) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+// Used in group stats replies.
+type OfpBucketCounter struct {
+	PacketCount          uint64   `protobuf:"varint,1,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
+	ByteCount            uint64   `protobuf:"varint,2,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpBucketCounter) Reset()         { *m = OfpBucketCounter{} }
+func (m *OfpBucketCounter) String() string { return proto.CompactTextString(m) }
+func (*OfpBucketCounter) ProtoMessage()    {}
+func (*OfpBucketCounter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{60}
+}
+
+func (m *OfpBucketCounter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpBucketCounter.Unmarshal(m, b)
+}
+func (m *OfpBucketCounter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpBucketCounter.Marshal(b, m, deterministic)
+}
+func (m *OfpBucketCounter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpBucketCounter.Merge(m, src)
+}
+func (m *OfpBucketCounter) XXX_Size() int {
+	return xxx_messageInfo_OfpBucketCounter.Size(m)
+}
+func (m *OfpBucketCounter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpBucketCounter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpBucketCounter proto.InternalMessageInfo
+
+func (m *OfpBucketCounter) GetPacketCount() uint64 {
+	if m != nil {
+		return m.PacketCount
+	}
+	return 0
+}
+
+func (m *OfpBucketCounter) GetByteCount() uint64 {
+	if m != nil {
+		return m.ByteCount
+	}
+	return 0
+}
+
+// Body of reply to OFPMP_GROUP request.
+type OfpGroupStats struct {
+	GroupId              uint32              `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	RefCount             uint32              `protobuf:"varint,2,opt,name=ref_count,json=refCount,proto3" json:"ref_count,omitempty"`
+	PacketCount          uint64              `protobuf:"varint,3,opt,name=packet_count,json=packetCount,proto3" json:"packet_count,omitempty"`
+	ByteCount            uint64              `protobuf:"varint,4,opt,name=byte_count,json=byteCount,proto3" json:"byte_count,omitempty"`
+	DurationSec          uint32              `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32              `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	BucketStats          []*OfpBucketCounter `protobuf:"bytes,7,rep,name=bucket_stats,json=bucketStats,proto3" json:"bucket_stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *OfpGroupStats) Reset()         { *m = OfpGroupStats{} }
+func (m *OfpGroupStats) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupStats) ProtoMessage()    {}
+func (*OfpGroupStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{61}
+}
+
+func (m *OfpGroupStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupStats.Unmarshal(m, b)
+}
+func (m *OfpGroupStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupStats.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupStats.Merge(m, src)
+}
+func (m *OfpGroupStats) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupStats.Size(m)
+}
+func (m *OfpGroupStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupStats proto.InternalMessageInfo
+
+func (m *OfpGroupStats) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetRefCount() uint32 {
+	if m != nil {
+		return m.RefCount
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetPacketCount() uint64 {
+	if m != nil {
+		return m.PacketCount
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetByteCount() uint64 {
+	if m != nil {
+		return m.ByteCount
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+func (m *OfpGroupStats) GetBucketStats() []*OfpBucketCounter {
+	if m != nil {
+		return m.BucketStats
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_GROUP_DESC request.
+type OfpGroupDesc struct {
+	Type                 OfpGroupType `protobuf:"varint,1,opt,name=type,proto3,enum=openflow_13.OfpGroupType" json:"type,omitempty"`
+	GroupId              uint32       `protobuf:"varint,2,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	Buckets              []*OfpBucket `protobuf:"bytes,3,rep,name=buckets,proto3" json:"buckets,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *OfpGroupDesc) Reset()         { *m = OfpGroupDesc{} }
+func (m *OfpGroupDesc) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupDesc) ProtoMessage()    {}
+func (*OfpGroupDesc) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{62}
+}
+
+func (m *OfpGroupDesc) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupDesc.Unmarshal(m, b)
+}
+func (m *OfpGroupDesc) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupDesc.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupDesc) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupDesc.Merge(m, src)
+}
+func (m *OfpGroupDesc) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupDesc.Size(m)
+}
+func (m *OfpGroupDesc) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupDesc.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupDesc proto.InternalMessageInfo
+
+func (m *OfpGroupDesc) GetType() OfpGroupType {
+	if m != nil {
+		return m.Type
+	}
+	return OfpGroupType_OFPGT_ALL
+}
+
+func (m *OfpGroupDesc) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+func (m *OfpGroupDesc) GetBuckets() []*OfpBucket {
+	if m != nil {
+		return m.Buckets
+	}
+	return nil
+}
+
+type OfpGroupEntry struct {
+	Desc                 *OfpGroupDesc  `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"`
+	Stats                *OfpGroupStats `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *OfpGroupEntry) Reset()         { *m = OfpGroupEntry{} }
+func (m *OfpGroupEntry) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupEntry) ProtoMessage()    {}
+func (*OfpGroupEntry) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{63}
+}
+
+func (m *OfpGroupEntry) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupEntry.Unmarshal(m, b)
+}
+func (m *OfpGroupEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupEntry.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupEntry) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupEntry.Merge(m, src)
+}
+func (m *OfpGroupEntry) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupEntry.Size(m)
+}
+func (m *OfpGroupEntry) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupEntry.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupEntry proto.InternalMessageInfo
+
+func (m *OfpGroupEntry) GetDesc() *OfpGroupDesc {
+	if m != nil {
+		return m.Desc
+	}
+	return nil
+}
+
+func (m *OfpGroupEntry) GetStats() *OfpGroupStats {
+	if m != nil {
+		return m.Stats
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_GROUP_FEATURES request. Group features.
+type OfpGroupFeatures struct {
+	Types                uint32   `protobuf:"varint,1,opt,name=types,proto3" json:"types,omitempty"`
+	Capabilities         uint32   `protobuf:"varint,2,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
+	MaxGroups            []uint32 `protobuf:"varint,3,rep,packed,name=max_groups,json=maxGroups,proto3" json:"max_groups,omitempty"`
+	Actions              []uint32 `protobuf:"varint,4,rep,packed,name=actions,proto3" json:"actions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpGroupFeatures) Reset()         { *m = OfpGroupFeatures{} }
+func (m *OfpGroupFeatures) String() string { return proto.CompactTextString(m) }
+func (*OfpGroupFeatures) ProtoMessage()    {}
+func (*OfpGroupFeatures) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{64}
+}
+
+func (m *OfpGroupFeatures) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpGroupFeatures.Unmarshal(m, b)
+}
+func (m *OfpGroupFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpGroupFeatures.Marshal(b, m, deterministic)
+}
+func (m *OfpGroupFeatures) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpGroupFeatures.Merge(m, src)
+}
+func (m *OfpGroupFeatures) XXX_Size() int {
+	return xxx_messageInfo_OfpGroupFeatures.Size(m)
+}
+func (m *OfpGroupFeatures) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpGroupFeatures.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpGroupFeatures proto.InternalMessageInfo
+
+func (m *OfpGroupFeatures) GetTypes() uint32 {
+	if m != nil {
+		return m.Types
+	}
+	return 0
+}
+
+func (m *OfpGroupFeatures) GetCapabilities() uint32 {
+	if m != nil {
+		return m.Capabilities
+	}
+	return 0
+}
+
+func (m *OfpGroupFeatures) GetMaxGroups() []uint32 {
+	if m != nil {
+		return m.MaxGroups
+	}
+	return nil
+}
+
+func (m *OfpGroupFeatures) GetActions() []uint32 {
+	if m != nil {
+		return m.Actions
+	}
+	return nil
+}
+
+// Body of OFPMP_METER and OFPMP_METER_CONFIG requests.
+type OfpMeterMultipartRequest struct {
+	MeterId              uint32   `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterMultipartRequest) Reset()         { *m = OfpMeterMultipartRequest{} }
+func (m *OfpMeterMultipartRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterMultipartRequest) ProtoMessage()    {}
+func (*OfpMeterMultipartRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{65}
+}
+
+func (m *OfpMeterMultipartRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterMultipartRequest.Unmarshal(m, b)
+}
+func (m *OfpMeterMultipartRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterMultipartRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterMultipartRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterMultipartRequest.Merge(m, src)
+}
+func (m *OfpMeterMultipartRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterMultipartRequest.Size(m)
+}
+func (m *OfpMeterMultipartRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterMultipartRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterMultipartRequest proto.InternalMessageInfo
+
+func (m *OfpMeterMultipartRequest) GetMeterId() uint32 {
+	if m != nil {
+		return m.MeterId
+	}
+	return 0
+}
+
+// Statistics for each meter band
+type OfpMeterBandStats struct {
+	PacketBandCount      uint64   `protobuf:"varint,1,opt,name=packet_band_count,json=packetBandCount,proto3" json:"packet_band_count,omitempty"`
+	ByteBandCount        uint64   `protobuf:"varint,2,opt,name=byte_band_count,json=byteBandCount,proto3" json:"byte_band_count,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterBandStats) Reset()         { *m = OfpMeterBandStats{} }
+func (m *OfpMeterBandStats) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterBandStats) ProtoMessage()    {}
+func (*OfpMeterBandStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{66}
+}
+
+func (m *OfpMeterBandStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterBandStats.Unmarshal(m, b)
+}
+func (m *OfpMeterBandStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterBandStats.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterBandStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterBandStats.Merge(m, src)
+}
+func (m *OfpMeterBandStats) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterBandStats.Size(m)
+}
+func (m *OfpMeterBandStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterBandStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterBandStats proto.InternalMessageInfo
+
+func (m *OfpMeterBandStats) GetPacketBandCount() uint64 {
+	if m != nil {
+		return m.PacketBandCount
+	}
+	return 0
+}
+
+func (m *OfpMeterBandStats) GetByteBandCount() uint64 {
+	if m != nil {
+		return m.ByteBandCount
+	}
+	return 0
+}
+
+// Body of reply to OFPMP_METER request. Meter statistics.
+type OfpMeterStats struct {
+	MeterId              uint32               `protobuf:"varint,1,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
+	FlowCount            uint32               `protobuf:"varint,2,opt,name=flow_count,json=flowCount,proto3" json:"flow_count,omitempty"`
+	PacketInCount        uint64               `protobuf:"varint,3,opt,name=packet_in_count,json=packetInCount,proto3" json:"packet_in_count,omitempty"`
+	ByteInCount          uint64               `protobuf:"varint,4,opt,name=byte_in_count,json=byteInCount,proto3" json:"byte_in_count,omitempty"`
+	DurationSec          uint32               `protobuf:"varint,5,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32               `protobuf:"varint,6,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	BandStats            []*OfpMeterBandStats `protobuf:"bytes,7,rep,name=band_stats,json=bandStats,proto3" json:"band_stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *OfpMeterStats) Reset()         { *m = OfpMeterStats{} }
+func (m *OfpMeterStats) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterStats) ProtoMessage()    {}
+func (*OfpMeterStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{67}
+}
+
+func (m *OfpMeterStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterStats.Unmarshal(m, b)
+}
+func (m *OfpMeterStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterStats.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterStats.Merge(m, src)
+}
+func (m *OfpMeterStats) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterStats.Size(m)
+}
+func (m *OfpMeterStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterStats proto.InternalMessageInfo
+
+func (m *OfpMeterStats) GetMeterId() uint32 {
+	if m != nil {
+		return m.MeterId
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetFlowCount() uint32 {
+	if m != nil {
+		return m.FlowCount
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetPacketInCount() uint64 {
+	if m != nil {
+		return m.PacketInCount
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetByteInCount() uint64 {
+	if m != nil {
+		return m.ByteInCount
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+func (m *OfpMeterStats) GetBandStats() []*OfpMeterBandStats {
+	if m != nil {
+		return m.BandStats
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_METER_CONFIG request. Meter configuration.
+type OfpMeterConfig struct {
+	Flags                uint32                `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"`
+	MeterId              uint32                `protobuf:"varint,2,opt,name=meter_id,json=meterId,proto3" json:"meter_id,omitempty"`
+	Bands                []*OfpMeterBandHeader `protobuf:"bytes,3,rep,name=bands,proto3" json:"bands,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpMeterConfig) Reset()         { *m = OfpMeterConfig{} }
+func (m *OfpMeterConfig) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterConfig) ProtoMessage()    {}
+func (*OfpMeterConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{68}
+}
+
+func (m *OfpMeterConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterConfig.Unmarshal(m, b)
+}
+func (m *OfpMeterConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterConfig.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterConfig.Merge(m, src)
+}
+func (m *OfpMeterConfig) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterConfig.Size(m)
+}
+func (m *OfpMeterConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterConfig proto.InternalMessageInfo
+
+func (m *OfpMeterConfig) GetFlags() uint32 {
+	if m != nil {
+		return m.Flags
+	}
+	return 0
+}
+
+func (m *OfpMeterConfig) GetMeterId() uint32 {
+	if m != nil {
+		return m.MeterId
+	}
+	return 0
+}
+
+func (m *OfpMeterConfig) GetBands() []*OfpMeterBandHeader {
+	if m != nil {
+		return m.Bands
+	}
+	return nil
+}
+
+// Body of reply to OFPMP_METER_FEATURES request. Meter features.
+type OfpMeterFeatures struct {
+	MaxMeter             uint32   `protobuf:"varint,1,opt,name=max_meter,json=maxMeter,proto3" json:"max_meter,omitempty"`
+	BandTypes            uint32   `protobuf:"varint,2,opt,name=band_types,json=bandTypes,proto3" json:"band_types,omitempty"`
+	Capabilities         uint32   `protobuf:"varint,3,opt,name=capabilities,proto3" json:"capabilities,omitempty"`
+	MaxBands             uint32   `protobuf:"varint,4,opt,name=max_bands,json=maxBands,proto3" json:"max_bands,omitempty"`
+	MaxColor             uint32   `protobuf:"varint,5,opt,name=max_color,json=maxColor,proto3" json:"max_color,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpMeterFeatures) Reset()         { *m = OfpMeterFeatures{} }
+func (m *OfpMeterFeatures) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterFeatures) ProtoMessage()    {}
+func (*OfpMeterFeatures) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{69}
+}
+
+func (m *OfpMeterFeatures) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterFeatures.Unmarshal(m, b)
+}
+func (m *OfpMeterFeatures) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterFeatures.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterFeatures) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterFeatures.Merge(m, src)
+}
+func (m *OfpMeterFeatures) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterFeatures.Size(m)
+}
+func (m *OfpMeterFeatures) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterFeatures.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterFeatures proto.InternalMessageInfo
+
+func (m *OfpMeterFeatures) GetMaxMeter() uint32 {
+	if m != nil {
+		return m.MaxMeter
+	}
+	return 0
+}
+
+func (m *OfpMeterFeatures) GetBandTypes() uint32 {
+	if m != nil {
+		return m.BandTypes
+	}
+	return 0
+}
+
+func (m *OfpMeterFeatures) GetCapabilities() uint32 {
+	if m != nil {
+		return m.Capabilities
+	}
+	return 0
+}
+
+func (m *OfpMeterFeatures) GetMaxBands() uint32 {
+	if m != nil {
+		return m.MaxBands
+	}
+	return 0
+}
+
+func (m *OfpMeterFeatures) GetMaxColor() uint32 {
+	if m != nil {
+		return m.MaxColor
+	}
+	return 0
+}
+
+type OfpMeterEntry struct {
+	Config               *OfpMeterConfig `protobuf:"bytes,1,opt,name=config,proto3" json:"config,omitempty"`
+	Stats                *OfpMeterStats  `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *OfpMeterEntry) Reset()         { *m = OfpMeterEntry{} }
+func (m *OfpMeterEntry) String() string { return proto.CompactTextString(m) }
+func (*OfpMeterEntry) ProtoMessage()    {}
+func (*OfpMeterEntry) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{70}
+}
+
+func (m *OfpMeterEntry) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpMeterEntry.Unmarshal(m, b)
+}
+func (m *OfpMeterEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpMeterEntry.Marshal(b, m, deterministic)
+}
+func (m *OfpMeterEntry) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpMeterEntry.Merge(m, src)
+}
+func (m *OfpMeterEntry) XXX_Size() int {
+	return xxx_messageInfo_OfpMeterEntry.Size(m)
+}
+func (m *OfpMeterEntry) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpMeterEntry.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpMeterEntry proto.InternalMessageInfo
+
+func (m *OfpMeterEntry) GetConfig() *OfpMeterConfig {
+	if m != nil {
+		return m.Config
+	}
+	return nil
+}
+
+func (m *OfpMeterEntry) GetStats() *OfpMeterStats {
+	if m != nil {
+		return m.Stats
+	}
+	return nil
+}
+
+// Body for ofp_multipart_request/reply of type OFPMP_EXPERIMENTER.
+type OfpExperimenterMultipartHeader struct {
+	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
+	Data                 []byte   `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpExperimenterMultipartHeader) Reset()         { *m = OfpExperimenterMultipartHeader{} }
+func (m *OfpExperimenterMultipartHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpExperimenterMultipartHeader) ProtoMessage()    {}
+func (*OfpExperimenterMultipartHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{71}
+}
+
+func (m *OfpExperimenterMultipartHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpExperimenterMultipartHeader.Unmarshal(m, b)
+}
+func (m *OfpExperimenterMultipartHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpExperimenterMultipartHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpExperimenterMultipartHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpExperimenterMultipartHeader.Merge(m, src)
+}
+func (m *OfpExperimenterMultipartHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpExperimenterMultipartHeader.Size(m)
+}
+func (m *OfpExperimenterMultipartHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpExperimenterMultipartHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpExperimenterMultipartHeader proto.InternalMessageInfo
+
+func (m *OfpExperimenterMultipartHeader) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpExperimenterMultipartHeader) GetExpType() uint32 {
+	if m != nil {
+		return m.ExpType
+	}
+	return 0
+}
+
+func (m *OfpExperimenterMultipartHeader) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Experimenter extension.
+type OfpExperimenterHeader struct {
+	//ofp_header header;  /* Type OFPT_EXPERIMENTER. */
+	Experimenter         uint32   `protobuf:"varint,1,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	ExpType              uint32   `protobuf:"varint,2,opt,name=exp_type,json=expType,proto3" json:"exp_type,omitempty"`
+	Data                 []byte   `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpExperimenterHeader) Reset()         { *m = OfpExperimenterHeader{} }
+func (m *OfpExperimenterHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpExperimenterHeader) ProtoMessage()    {}
+func (*OfpExperimenterHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{72}
+}
+
+func (m *OfpExperimenterHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpExperimenterHeader.Unmarshal(m, b)
+}
+func (m *OfpExperimenterHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpExperimenterHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpExperimenterHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpExperimenterHeader.Merge(m, src)
+}
+func (m *OfpExperimenterHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpExperimenterHeader.Size(m)
+}
+func (m *OfpExperimenterHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpExperimenterHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpExperimenterHeader proto.InternalMessageInfo
+
+func (m *OfpExperimenterHeader) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpExperimenterHeader) GetExpType() uint32 {
+	if m != nil {
+		return m.ExpType
+	}
+	return 0
+}
+
+func (m *OfpExperimenterHeader) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Common description for a queue.
+type OfpQueuePropHeader struct {
+	Property             uint32   `protobuf:"varint,1,opt,name=property,proto3" json:"property,omitempty"`
+	Len                  uint32   `protobuf:"varint,2,opt,name=len,proto3" json:"len,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpQueuePropHeader) Reset()         { *m = OfpQueuePropHeader{} }
+func (m *OfpQueuePropHeader) String() string { return proto.CompactTextString(m) }
+func (*OfpQueuePropHeader) ProtoMessage()    {}
+func (*OfpQueuePropHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{73}
+}
+
+func (m *OfpQueuePropHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueuePropHeader.Unmarshal(m, b)
+}
+func (m *OfpQueuePropHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueuePropHeader.Marshal(b, m, deterministic)
+}
+func (m *OfpQueuePropHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueuePropHeader.Merge(m, src)
+}
+func (m *OfpQueuePropHeader) XXX_Size() int {
+	return xxx_messageInfo_OfpQueuePropHeader.Size(m)
+}
+func (m *OfpQueuePropHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueuePropHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueuePropHeader proto.InternalMessageInfo
+
+func (m *OfpQueuePropHeader) GetProperty() uint32 {
+	if m != nil {
+		return m.Property
+	}
+	return 0
+}
+
+func (m *OfpQueuePropHeader) GetLen() uint32 {
+	if m != nil {
+		return m.Len
+	}
+	return 0
+}
+
+// Min-Rate queue property description.
+type OfpQueuePropMinRate struct {
+	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
+	Rate                 uint32              `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *OfpQueuePropMinRate) Reset()         { *m = OfpQueuePropMinRate{} }
+func (m *OfpQueuePropMinRate) String() string { return proto.CompactTextString(m) }
+func (*OfpQueuePropMinRate) ProtoMessage()    {}
+func (*OfpQueuePropMinRate) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{74}
+}
+
+func (m *OfpQueuePropMinRate) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueuePropMinRate.Unmarshal(m, b)
+}
+func (m *OfpQueuePropMinRate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueuePropMinRate.Marshal(b, m, deterministic)
+}
+func (m *OfpQueuePropMinRate) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueuePropMinRate.Merge(m, src)
+}
+func (m *OfpQueuePropMinRate) XXX_Size() int {
+	return xxx_messageInfo_OfpQueuePropMinRate.Size(m)
+}
+func (m *OfpQueuePropMinRate) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueuePropMinRate.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueuePropMinRate proto.InternalMessageInfo
+
+func (m *OfpQueuePropMinRate) GetPropHeader() *OfpQueuePropHeader {
+	if m != nil {
+		return m.PropHeader
+	}
+	return nil
+}
+
+func (m *OfpQueuePropMinRate) GetRate() uint32 {
+	if m != nil {
+		return m.Rate
+	}
+	return 0
+}
+
+// Max-Rate queue property description.
+type OfpQueuePropMaxRate struct {
+	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
+	Rate                 uint32              `protobuf:"varint,2,opt,name=rate,proto3" json:"rate,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *OfpQueuePropMaxRate) Reset()         { *m = OfpQueuePropMaxRate{} }
+func (m *OfpQueuePropMaxRate) String() string { return proto.CompactTextString(m) }
+func (*OfpQueuePropMaxRate) ProtoMessage()    {}
+func (*OfpQueuePropMaxRate) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{75}
+}
+
+func (m *OfpQueuePropMaxRate) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueuePropMaxRate.Unmarshal(m, b)
+}
+func (m *OfpQueuePropMaxRate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueuePropMaxRate.Marshal(b, m, deterministic)
+}
+func (m *OfpQueuePropMaxRate) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueuePropMaxRate.Merge(m, src)
+}
+func (m *OfpQueuePropMaxRate) XXX_Size() int {
+	return xxx_messageInfo_OfpQueuePropMaxRate.Size(m)
+}
+func (m *OfpQueuePropMaxRate) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueuePropMaxRate.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueuePropMaxRate proto.InternalMessageInfo
+
+func (m *OfpQueuePropMaxRate) GetPropHeader() *OfpQueuePropHeader {
+	if m != nil {
+		return m.PropHeader
+	}
+	return nil
+}
+
+func (m *OfpQueuePropMaxRate) GetRate() uint32 {
+	if m != nil {
+		return m.Rate
+	}
+	return 0
+}
+
+// Experimenter queue property description.
+type OfpQueuePropExperimenter struct {
+	PropHeader           *OfpQueuePropHeader `protobuf:"bytes,1,opt,name=prop_header,json=propHeader,proto3" json:"prop_header,omitempty"`
+	Experimenter         uint32              `protobuf:"varint,2,opt,name=experimenter,proto3" json:"experimenter,omitempty"`
+	Data                 []byte              `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *OfpQueuePropExperimenter) Reset()         { *m = OfpQueuePropExperimenter{} }
+func (m *OfpQueuePropExperimenter) String() string { return proto.CompactTextString(m) }
+func (*OfpQueuePropExperimenter) ProtoMessage()    {}
+func (*OfpQueuePropExperimenter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{76}
+}
+
+func (m *OfpQueuePropExperimenter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueuePropExperimenter.Unmarshal(m, b)
+}
+func (m *OfpQueuePropExperimenter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueuePropExperimenter.Marshal(b, m, deterministic)
+}
+func (m *OfpQueuePropExperimenter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueuePropExperimenter.Merge(m, src)
+}
+func (m *OfpQueuePropExperimenter) XXX_Size() int {
+	return xxx_messageInfo_OfpQueuePropExperimenter.Size(m)
+}
+func (m *OfpQueuePropExperimenter) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueuePropExperimenter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueuePropExperimenter proto.InternalMessageInfo
+
+func (m *OfpQueuePropExperimenter) GetPropHeader() *OfpQueuePropHeader {
+	if m != nil {
+		return m.PropHeader
+	}
+	return nil
+}
+
+func (m *OfpQueuePropExperimenter) GetExperimenter() uint32 {
+	if m != nil {
+		return m.Experimenter
+	}
+	return 0
+}
+
+func (m *OfpQueuePropExperimenter) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Full description for a queue.
+type OfpPacketQueue struct {
+	QueueId              uint32                `protobuf:"varint,1,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
+	Port                 uint32                `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
+	Properties           []*OfpQueuePropHeader `protobuf:"bytes,4,rep,name=properties,proto3" json:"properties,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *OfpPacketQueue) Reset()         { *m = OfpPacketQueue{} }
+func (m *OfpPacketQueue) String() string { return proto.CompactTextString(m) }
+func (*OfpPacketQueue) ProtoMessage()    {}
+func (*OfpPacketQueue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{77}
+}
+
+func (m *OfpPacketQueue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpPacketQueue.Unmarshal(m, b)
+}
+func (m *OfpPacketQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpPacketQueue.Marshal(b, m, deterministic)
+}
+func (m *OfpPacketQueue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpPacketQueue.Merge(m, src)
+}
+func (m *OfpPacketQueue) XXX_Size() int {
+	return xxx_messageInfo_OfpPacketQueue.Size(m)
+}
+func (m *OfpPacketQueue) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpPacketQueue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpPacketQueue proto.InternalMessageInfo
+
+func (m *OfpPacketQueue) GetQueueId() uint32 {
+	if m != nil {
+		return m.QueueId
+	}
+	return 0
+}
+
+func (m *OfpPacketQueue) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+func (m *OfpPacketQueue) GetProperties() []*OfpQueuePropHeader {
+	if m != nil {
+		return m.Properties
+	}
+	return nil
+}
+
+// Query for port queue configuration.
+type OfpQueueGetConfigRequest struct {
+	//ofp_header header;
+	Port                 uint32   `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpQueueGetConfigRequest) Reset()         { *m = OfpQueueGetConfigRequest{} }
+func (m *OfpQueueGetConfigRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpQueueGetConfigRequest) ProtoMessage()    {}
+func (*OfpQueueGetConfigRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{78}
+}
+
+func (m *OfpQueueGetConfigRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueueGetConfigRequest.Unmarshal(m, b)
+}
+func (m *OfpQueueGetConfigRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueueGetConfigRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpQueueGetConfigRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueueGetConfigRequest.Merge(m, src)
+}
+func (m *OfpQueueGetConfigRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpQueueGetConfigRequest.Size(m)
+}
+func (m *OfpQueueGetConfigRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueueGetConfigRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueueGetConfigRequest proto.InternalMessageInfo
+
+func (m *OfpQueueGetConfigRequest) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+// Queue configuration for a given port.
+type OfpQueueGetConfigReply struct {
+	//ofp_header header;
+	Port                 uint32            `protobuf:"varint,1,opt,name=port,proto3" json:"port,omitempty"`
+	Queues               []*OfpPacketQueue `protobuf:"bytes,2,rep,name=queues,proto3" json:"queues,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpQueueGetConfigReply) Reset()         { *m = OfpQueueGetConfigReply{} }
+func (m *OfpQueueGetConfigReply) String() string { return proto.CompactTextString(m) }
+func (*OfpQueueGetConfigReply) ProtoMessage()    {}
+func (*OfpQueueGetConfigReply) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{79}
+}
+
+func (m *OfpQueueGetConfigReply) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueueGetConfigReply.Unmarshal(m, b)
+}
+func (m *OfpQueueGetConfigReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueueGetConfigReply.Marshal(b, m, deterministic)
+}
+func (m *OfpQueueGetConfigReply) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueueGetConfigReply.Merge(m, src)
+}
+func (m *OfpQueueGetConfigReply) XXX_Size() int {
+	return xxx_messageInfo_OfpQueueGetConfigReply.Size(m)
+}
+func (m *OfpQueueGetConfigReply) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueueGetConfigReply.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueueGetConfigReply proto.InternalMessageInfo
+
+func (m *OfpQueueGetConfigReply) GetPort() uint32 {
+	if m != nil {
+		return m.Port
+	}
+	return 0
+}
+
+func (m *OfpQueueGetConfigReply) GetQueues() []*OfpPacketQueue {
+	if m != nil {
+		return m.Queues
+	}
+	return nil
+}
+
+// OFPAT_SET_QUEUE action struct: send packets to given queue on port.
+type OfpActionSetQueue struct {
+	Type                 uint32   `protobuf:"varint,1,opt,name=type,proto3" json:"type,omitempty"`
+	QueueId              uint32   `protobuf:"varint,3,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpActionSetQueue) Reset()         { *m = OfpActionSetQueue{} }
+func (m *OfpActionSetQueue) String() string { return proto.CompactTextString(m) }
+func (*OfpActionSetQueue) ProtoMessage()    {}
+func (*OfpActionSetQueue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{80}
+}
+
+func (m *OfpActionSetQueue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpActionSetQueue.Unmarshal(m, b)
+}
+func (m *OfpActionSetQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpActionSetQueue.Marshal(b, m, deterministic)
+}
+func (m *OfpActionSetQueue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpActionSetQueue.Merge(m, src)
+}
+func (m *OfpActionSetQueue) XXX_Size() int {
+	return xxx_messageInfo_OfpActionSetQueue.Size(m)
+}
+func (m *OfpActionSetQueue) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpActionSetQueue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpActionSetQueue proto.InternalMessageInfo
+
+func (m *OfpActionSetQueue) GetType() uint32 {
+	if m != nil {
+		return m.Type
+	}
+	return 0
+}
+
+func (m *OfpActionSetQueue) GetQueueId() uint32 {
+	if m != nil {
+		return m.QueueId
+	}
+	return 0
+}
+
+type OfpQueueStatsRequest struct {
+	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	QueueId              uint32   `protobuf:"varint,2,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpQueueStatsRequest) Reset()         { *m = OfpQueueStatsRequest{} }
+func (m *OfpQueueStatsRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpQueueStatsRequest) ProtoMessage()    {}
+func (*OfpQueueStatsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{81}
+}
+
+func (m *OfpQueueStatsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueueStatsRequest.Unmarshal(m, b)
+}
+func (m *OfpQueueStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueueStatsRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpQueueStatsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueueStatsRequest.Merge(m, src)
+}
+func (m *OfpQueueStatsRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpQueueStatsRequest.Size(m)
+}
+func (m *OfpQueueStatsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueueStatsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueueStatsRequest proto.InternalMessageInfo
+
+func (m *OfpQueueStatsRequest) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OfpQueueStatsRequest) GetQueueId() uint32 {
+	if m != nil {
+		return m.QueueId
+	}
+	return 0
+}
+
+type OfpQueueStats struct {
+	PortNo               uint32   `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	QueueId              uint32   `protobuf:"varint,2,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
+	TxBytes              uint64   `protobuf:"varint,3,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	TxPackets            uint64   `protobuf:"varint,4,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	TxErrors             uint64   `protobuf:"varint,5,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
+	DurationSec          uint32   `protobuf:"varint,6,opt,name=duration_sec,json=durationSec,proto3" json:"duration_sec,omitempty"`
+	DurationNsec         uint32   `protobuf:"varint,7,opt,name=duration_nsec,json=durationNsec,proto3" json:"duration_nsec,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpQueueStats) Reset()         { *m = OfpQueueStats{} }
+func (m *OfpQueueStats) String() string { return proto.CompactTextString(m) }
+func (*OfpQueueStats) ProtoMessage()    {}
+func (*OfpQueueStats) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{82}
+}
+
+func (m *OfpQueueStats) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpQueueStats.Unmarshal(m, b)
+}
+func (m *OfpQueueStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpQueueStats.Marshal(b, m, deterministic)
+}
+func (m *OfpQueueStats) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpQueueStats.Merge(m, src)
+}
+func (m *OfpQueueStats) XXX_Size() int {
+	return xxx_messageInfo_OfpQueueStats.Size(m)
+}
+func (m *OfpQueueStats) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpQueueStats.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpQueueStats proto.InternalMessageInfo
+
+func (m *OfpQueueStats) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetQueueId() uint32 {
+	if m != nil {
+		return m.QueueId
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetTxErrors() uint64 {
+	if m != nil {
+		return m.TxErrors
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetDurationSec() uint32 {
+	if m != nil {
+		return m.DurationSec
+	}
+	return 0
+}
+
+func (m *OfpQueueStats) GetDurationNsec() uint32 {
+	if m != nil {
+		return m.DurationNsec
+	}
+	return 0
+}
+
+// Role request and reply message.
+type OfpRoleRequest struct {
+	//ofp_header header;        /* Type OFPT_ROLE_REQUEST/OFPT_ROLE_REPLY. */
+	Role                 OfpControllerRole `protobuf:"varint,1,opt,name=role,proto3,enum=openflow_13.OfpControllerRole" json:"role,omitempty"`
+	GenerationId         uint64            `protobuf:"varint,2,opt,name=generation_id,json=generationId,proto3" json:"generation_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *OfpRoleRequest) Reset()         { *m = OfpRoleRequest{} }
+func (m *OfpRoleRequest) String() string { return proto.CompactTextString(m) }
+func (*OfpRoleRequest) ProtoMessage()    {}
+func (*OfpRoleRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{83}
+}
+
+func (m *OfpRoleRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpRoleRequest.Unmarshal(m, b)
+}
+func (m *OfpRoleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpRoleRequest.Marshal(b, m, deterministic)
+}
+func (m *OfpRoleRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpRoleRequest.Merge(m, src)
+}
+func (m *OfpRoleRequest) XXX_Size() int {
+	return xxx_messageInfo_OfpRoleRequest.Size(m)
+}
+func (m *OfpRoleRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpRoleRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpRoleRequest proto.InternalMessageInfo
+
+func (m *OfpRoleRequest) GetRole() OfpControllerRole {
+	if m != nil {
+		return m.Role
+	}
+	return OfpControllerRole_OFPCR_ROLE_NOCHANGE
+}
+
+func (m *OfpRoleRequest) GetGenerationId() uint64 {
+	if m != nil {
+		return m.GenerationId
+	}
+	return 0
+}
+
+// Asynchronous message configuration.
+type OfpAsyncConfig struct {
+	//ofp_header header;    /* OFPT_GET_ASYNC_REPLY or OFPT_SET_ASYNC. */
+	PacketInMask         []uint32 `protobuf:"varint,1,rep,packed,name=packet_in_mask,json=packetInMask,proto3" json:"packet_in_mask,omitempty"`
+	PortStatusMask       []uint32 `protobuf:"varint,2,rep,packed,name=port_status_mask,json=portStatusMask,proto3" json:"port_status_mask,omitempty"`
+	FlowRemovedMask      []uint32 `protobuf:"varint,3,rep,packed,name=flow_removed_mask,json=flowRemovedMask,proto3" json:"flow_removed_mask,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfpAsyncConfig) Reset()         { *m = OfpAsyncConfig{} }
+func (m *OfpAsyncConfig) String() string { return proto.CompactTextString(m) }
+func (*OfpAsyncConfig) ProtoMessage()    {}
+func (*OfpAsyncConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{84}
+}
+
+func (m *OfpAsyncConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfpAsyncConfig.Unmarshal(m, b)
+}
+func (m *OfpAsyncConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfpAsyncConfig.Marshal(b, m, deterministic)
+}
+func (m *OfpAsyncConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfpAsyncConfig.Merge(m, src)
+}
+func (m *OfpAsyncConfig) XXX_Size() int {
+	return xxx_messageInfo_OfpAsyncConfig.Size(m)
+}
+func (m *OfpAsyncConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfpAsyncConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfpAsyncConfig proto.InternalMessageInfo
+
+func (m *OfpAsyncConfig) GetPacketInMask() []uint32 {
+	if m != nil {
+		return m.PacketInMask
+	}
+	return nil
+}
+
+func (m *OfpAsyncConfig) GetPortStatusMask() []uint32 {
+	if m != nil {
+		return m.PortStatusMask
+	}
+	return nil
+}
+
+func (m *OfpAsyncConfig) GetFlowRemovedMask() []uint32 {
+	if m != nil {
+		return m.FlowRemovedMask
+	}
+	return nil
+}
+
+type MeterModUpdate struct {
+	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	MeterMod             *OfpMeterMod `protobuf:"bytes,2,opt,name=meter_mod,json=meterMod,proto3" json:"meter_mod,omitempty"`
+	Xid                  uint32       `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *MeterModUpdate) Reset()         { *m = MeterModUpdate{} }
+func (m *MeterModUpdate) String() string { return proto.CompactTextString(m) }
+func (*MeterModUpdate) ProtoMessage()    {}
+func (*MeterModUpdate) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{85}
+}
+
+func (m *MeterModUpdate) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MeterModUpdate.Unmarshal(m, b)
+}
+func (m *MeterModUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MeterModUpdate.Marshal(b, m, deterministic)
+}
+func (m *MeterModUpdate) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MeterModUpdate.Merge(m, src)
+}
+func (m *MeterModUpdate) XXX_Size() int {
+	return xxx_messageInfo_MeterModUpdate.Size(m)
+}
+func (m *MeterModUpdate) XXX_DiscardUnknown() {
+	xxx_messageInfo_MeterModUpdate.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MeterModUpdate proto.InternalMessageInfo
+
+func (m *MeterModUpdate) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *MeterModUpdate) GetMeterMod() *OfpMeterMod {
+	if m != nil {
+		return m.MeterMod
+	}
+	return nil
+}
+
+func (m *MeterModUpdate) GetXid() uint32 {
+	if m != nil {
+		return m.Xid
+	}
+	return 0
+}
+
+type MeterStatsReply struct {
+	MeterStats           []*OfpMeterStats `protobuf:"bytes,1,rep,name=meter_stats,json=meterStats,proto3" json:"meter_stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *MeterStatsReply) Reset()         { *m = MeterStatsReply{} }
+func (m *MeterStatsReply) String() string { return proto.CompactTextString(m) }
+func (*MeterStatsReply) ProtoMessage()    {}
+func (*MeterStatsReply) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{86}
+}
+
+func (m *MeterStatsReply) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MeterStatsReply.Unmarshal(m, b)
+}
+func (m *MeterStatsReply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MeterStatsReply.Marshal(b, m, deterministic)
+}
+func (m *MeterStatsReply) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MeterStatsReply.Merge(m, src)
+}
+func (m *MeterStatsReply) XXX_Size() int {
+	return xxx_messageInfo_MeterStatsReply.Size(m)
+}
+func (m *MeterStatsReply) XXX_DiscardUnknown() {
+	xxx_messageInfo_MeterStatsReply.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MeterStatsReply proto.InternalMessageInfo
+
+func (m *MeterStatsReply) GetMeterStats() []*OfpMeterStats {
+	if m != nil {
+		return m.MeterStats
+	}
+	return nil
+}
+
+type FlowTableUpdate struct {
+	Id                   string      `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	FlowMod              *OfpFlowMod `protobuf:"bytes,2,opt,name=flow_mod,json=flowMod,proto3" json:"flow_mod,omitempty"`
+	Xid                  uint32      `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *FlowTableUpdate) Reset()         { *m = FlowTableUpdate{} }
+func (m *FlowTableUpdate) String() string { return proto.CompactTextString(m) }
+func (*FlowTableUpdate) ProtoMessage()    {}
+func (*FlowTableUpdate) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{87}
+}
+
+func (m *FlowTableUpdate) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowTableUpdate.Unmarshal(m, b)
+}
+func (m *FlowTableUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowTableUpdate.Marshal(b, m, deterministic)
+}
+func (m *FlowTableUpdate) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowTableUpdate.Merge(m, src)
+}
+func (m *FlowTableUpdate) XXX_Size() int {
+	return xxx_messageInfo_FlowTableUpdate.Size(m)
+}
+func (m *FlowTableUpdate) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowTableUpdate.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowTableUpdate proto.InternalMessageInfo
+
+func (m *FlowTableUpdate) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *FlowTableUpdate) GetFlowMod() *OfpFlowMod {
+	if m != nil {
+		return m.FlowMod
+	}
+	return nil
+}
+
+func (m *FlowTableUpdate) GetXid() uint32 {
+	if m != nil {
+		return m.Xid
+	}
+	return 0
+}
+
+type FlowGroupTableUpdate struct {
+	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	GroupMod             *OfpGroupMod `protobuf:"bytes,2,opt,name=group_mod,json=groupMod,proto3" json:"group_mod,omitempty"`
+	Xid                  uint32       `protobuf:"varint,3,opt,name=xid,proto3" json:"xid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *FlowGroupTableUpdate) Reset()         { *m = FlowGroupTableUpdate{} }
+func (m *FlowGroupTableUpdate) String() string { return proto.CompactTextString(m) }
+func (*FlowGroupTableUpdate) ProtoMessage()    {}
+func (*FlowGroupTableUpdate) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{88}
+}
+
+func (m *FlowGroupTableUpdate) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowGroupTableUpdate.Unmarshal(m, b)
+}
+func (m *FlowGroupTableUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowGroupTableUpdate.Marshal(b, m, deterministic)
+}
+func (m *FlowGroupTableUpdate) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowGroupTableUpdate.Merge(m, src)
+}
+func (m *FlowGroupTableUpdate) XXX_Size() int {
+	return xxx_messageInfo_FlowGroupTableUpdate.Size(m)
+}
+func (m *FlowGroupTableUpdate) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowGroupTableUpdate.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowGroupTableUpdate proto.InternalMessageInfo
+
+func (m *FlowGroupTableUpdate) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *FlowGroupTableUpdate) GetGroupMod() *OfpGroupMod {
+	if m != nil {
+		return m.GroupMod
+	}
+	return nil
+}
+
+func (m *FlowGroupTableUpdate) GetXid() uint32 {
+	if m != nil {
+		return m.Xid
+	}
+	return 0
+}
+
+type Flows struct {
+	Items                []*OfpFlowStats `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *Flows) Reset()         { *m = Flows{} }
+func (m *Flows) String() string { return proto.CompactTextString(m) }
+func (*Flows) ProtoMessage()    {}
+func (*Flows) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{89}
+}
+
+func (m *Flows) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Flows.Unmarshal(m, b)
+}
+func (m *Flows) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Flows.Marshal(b, m, deterministic)
+}
+func (m *Flows) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Flows.Merge(m, src)
+}
+func (m *Flows) XXX_Size() int {
+	return xxx_messageInfo_Flows.Size(m)
+}
+func (m *Flows) XXX_DiscardUnknown() {
+	xxx_messageInfo_Flows.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Flows proto.InternalMessageInfo
+
+func (m *Flows) GetItems() []*OfpFlowStats {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type Meters struct {
+	Items                []*OfpMeterEntry `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *Meters) Reset()         { *m = Meters{} }
+func (m *Meters) String() string { return proto.CompactTextString(m) }
+func (*Meters) ProtoMessage()    {}
+func (*Meters) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{90}
+}
+
+func (m *Meters) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Meters.Unmarshal(m, b)
+}
+func (m *Meters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Meters.Marshal(b, m, deterministic)
+}
+func (m *Meters) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Meters.Merge(m, src)
+}
+func (m *Meters) XXX_Size() int {
+	return xxx_messageInfo_Meters.Size(m)
+}
+func (m *Meters) XXX_DiscardUnknown() {
+	xxx_messageInfo_Meters.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Meters proto.InternalMessageInfo
+
+func (m *Meters) GetItems() []*OfpMeterEntry {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type FlowGroups struct {
+	Items                []*OfpGroupEntry `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *FlowGroups) Reset()         { *m = FlowGroups{} }
+func (m *FlowGroups) String() string { return proto.CompactTextString(m) }
+func (*FlowGroups) ProtoMessage()    {}
+func (*FlowGroups) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{91}
+}
+
+func (m *FlowGroups) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowGroups.Unmarshal(m, b)
+}
+func (m *FlowGroups) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowGroups.Marshal(b, m, deterministic)
+}
+func (m *FlowGroups) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowGroups.Merge(m, src)
+}
+func (m *FlowGroups) XXX_Size() int {
+	return xxx_messageInfo_FlowGroups.Size(m)
+}
+func (m *FlowGroups) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowGroups.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowGroups proto.InternalMessageInfo
+
+func (m *FlowGroups) GetItems() []*OfpGroupEntry {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type FlowChanges struct {
+	ToAdd                *Flows   `protobuf:"bytes,1,opt,name=to_add,json=toAdd,proto3" json:"to_add,omitempty"`
+	ToRemove             *Flows   `protobuf:"bytes,2,opt,name=to_remove,json=toRemove,proto3" json:"to_remove,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *FlowChanges) Reset()         { *m = FlowChanges{} }
+func (m *FlowChanges) String() string { return proto.CompactTextString(m) }
+func (*FlowChanges) ProtoMessage()    {}
+func (*FlowChanges) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{92}
+}
+
+func (m *FlowChanges) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowChanges.Unmarshal(m, b)
+}
+func (m *FlowChanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowChanges.Marshal(b, m, deterministic)
+}
+func (m *FlowChanges) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowChanges.Merge(m, src)
+}
+func (m *FlowChanges) XXX_Size() int {
+	return xxx_messageInfo_FlowChanges.Size(m)
+}
+func (m *FlowChanges) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowChanges.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowChanges proto.InternalMessageInfo
+
+func (m *FlowChanges) GetToAdd() *Flows {
+	if m != nil {
+		return m.ToAdd
+	}
+	return nil
+}
+
+func (m *FlowChanges) GetToRemove() *Flows {
+	if m != nil {
+		return m.ToRemove
+	}
+	return nil
+}
+
+type FlowGroupChanges struct {
+	ToAdd                *FlowGroups `protobuf:"bytes,1,opt,name=to_add,json=toAdd,proto3" json:"to_add,omitempty"`
+	ToRemove             *FlowGroups `protobuf:"bytes,2,opt,name=to_remove,json=toRemove,proto3" json:"to_remove,omitempty"`
+	ToUpdate             *FlowGroups `protobuf:"bytes,3,opt,name=to_update,json=toUpdate,proto3" json:"to_update,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *FlowGroupChanges) Reset()         { *m = FlowGroupChanges{} }
+func (m *FlowGroupChanges) String() string { return proto.CompactTextString(m) }
+func (*FlowGroupChanges) ProtoMessage()    {}
+func (*FlowGroupChanges) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{93}
+}
+
+func (m *FlowGroupChanges) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowGroupChanges.Unmarshal(m, b)
+}
+func (m *FlowGroupChanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowGroupChanges.Marshal(b, m, deterministic)
+}
+func (m *FlowGroupChanges) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowGroupChanges.Merge(m, src)
+}
+func (m *FlowGroupChanges) XXX_Size() int {
+	return xxx_messageInfo_FlowGroupChanges.Size(m)
+}
+func (m *FlowGroupChanges) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowGroupChanges.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowGroupChanges proto.InternalMessageInfo
+
+func (m *FlowGroupChanges) GetToAdd() *FlowGroups {
+	if m != nil {
+		return m.ToAdd
+	}
+	return nil
+}
+
+func (m *FlowGroupChanges) GetToRemove() *FlowGroups {
+	if m != nil {
+		return m.ToRemove
+	}
+	return nil
+}
+
+func (m *FlowGroupChanges) GetToUpdate() *FlowGroups {
+	if m != nil {
+		return m.ToUpdate
+	}
+	return nil
+}
+
+type PacketIn struct {
+	Id                   string       `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	PacketIn             *OfpPacketIn `protobuf:"bytes,2,opt,name=packet_in,json=packetIn,proto3" json:"packet_in,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *PacketIn) Reset()         { *m = PacketIn{} }
+func (m *PacketIn) String() string { return proto.CompactTextString(m) }
+func (*PacketIn) ProtoMessage()    {}
+func (*PacketIn) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{94}
+}
+
+func (m *PacketIn) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PacketIn.Unmarshal(m, b)
+}
+func (m *PacketIn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PacketIn.Marshal(b, m, deterministic)
+}
+func (m *PacketIn) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PacketIn.Merge(m, src)
+}
+func (m *PacketIn) XXX_Size() int {
+	return xxx_messageInfo_PacketIn.Size(m)
+}
+func (m *PacketIn) XXX_DiscardUnknown() {
+	xxx_messageInfo_PacketIn.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PacketIn proto.InternalMessageInfo
+
+func (m *PacketIn) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *PacketIn) GetPacketIn() *OfpPacketIn {
+	if m != nil {
+		return m.PacketIn
+	}
+	return nil
+}
+
+type PacketOut struct {
+	Id                   string        `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	PacketOut            *OfpPacketOut `protobuf:"bytes,2,opt,name=packet_out,json=packetOut,proto3" json:"packet_out,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *PacketOut) Reset()         { *m = PacketOut{} }
+func (m *PacketOut) String() string { return proto.CompactTextString(m) }
+func (*PacketOut) ProtoMessage()    {}
+func (*PacketOut) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{95}
+}
+
+func (m *PacketOut) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PacketOut.Unmarshal(m, b)
+}
+func (m *PacketOut) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PacketOut.Marshal(b, m, deterministic)
+}
+func (m *PacketOut) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PacketOut.Merge(m, src)
+}
+func (m *PacketOut) XXX_Size() int {
+	return xxx_messageInfo_PacketOut.Size(m)
+}
+func (m *PacketOut) XXX_DiscardUnknown() {
+	xxx_messageInfo_PacketOut.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PacketOut proto.InternalMessageInfo
+
+func (m *PacketOut) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *PacketOut) GetPacketOut() *OfpPacketOut {
+	if m != nil {
+		return m.PacketOut
+	}
+	return nil
+}
+
+type ChangeEvent struct {
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Types that are valid to be assigned to Event:
+	//	*ChangeEvent_PortStatus
+	//	*ChangeEvent_Error
+	Event                isChangeEvent_Event `protobuf_oneof:"event"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *ChangeEvent) Reset()         { *m = ChangeEvent{} }
+func (m *ChangeEvent) String() string { return proto.CompactTextString(m) }
+func (*ChangeEvent) ProtoMessage()    {}
+func (*ChangeEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_08e3a4e375aeddc7, []int{96}
+}
+
+func (m *ChangeEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ChangeEvent.Unmarshal(m, b)
+}
+func (m *ChangeEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ChangeEvent.Marshal(b, m, deterministic)
+}
+func (m *ChangeEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ChangeEvent.Merge(m, src)
+}
+func (m *ChangeEvent) XXX_Size() int {
+	return xxx_messageInfo_ChangeEvent.Size(m)
+}
+func (m *ChangeEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_ChangeEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ChangeEvent proto.InternalMessageInfo
+
+func (m *ChangeEvent) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+type isChangeEvent_Event interface {
+	isChangeEvent_Event()
+}
+
+type ChangeEvent_PortStatus struct {
+	PortStatus *OfpPortStatus `protobuf:"bytes,2,opt,name=port_status,json=portStatus,proto3,oneof"`
+}
+
+type ChangeEvent_Error struct {
+	Error *OfpErrorMsg `protobuf:"bytes,3,opt,name=error,proto3,oneof"`
+}
+
+func (*ChangeEvent_PortStatus) isChangeEvent_Event() {}
+
+func (*ChangeEvent_Error) isChangeEvent_Event() {}
+
+func (m *ChangeEvent) GetEvent() isChangeEvent_Event {
+	if m != nil {
+		return m.Event
+	}
+	return nil
+}
+
+func (m *ChangeEvent) GetPortStatus() *OfpPortStatus {
+	if x, ok := m.GetEvent().(*ChangeEvent_PortStatus); ok {
+		return x.PortStatus
+	}
+	return nil
+}
+
+func (m *ChangeEvent) GetError() *OfpErrorMsg {
+	if x, ok := m.GetEvent().(*ChangeEvent_Error); ok {
+		return x.Error
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*ChangeEvent) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*ChangeEvent_PortStatus)(nil),
+		(*ChangeEvent_Error)(nil),
+	}
+}
+
+func init() {
+	proto.RegisterEnum("openflow_13.OfpPortNo", OfpPortNo_name, OfpPortNo_value)
+	proto.RegisterEnum("openflow_13.OfpType", OfpType_name, OfpType_value)
+	proto.RegisterEnum("openflow_13.OfpHelloElemType", OfpHelloElemType_name, OfpHelloElemType_value)
+	proto.RegisterEnum("openflow_13.OfpConfigFlags", OfpConfigFlags_name, OfpConfigFlags_value)
+	proto.RegisterEnum("openflow_13.OfpTableConfig", OfpTableConfig_name, OfpTableConfig_value)
+	proto.RegisterEnum("openflow_13.OfpTable", OfpTable_name, OfpTable_value)
+	proto.RegisterEnum("openflow_13.OfpCapabilities", OfpCapabilities_name, OfpCapabilities_value)
+	proto.RegisterEnum("openflow_13.OfpPortConfig", OfpPortConfig_name, OfpPortConfig_value)
+	proto.RegisterEnum("openflow_13.OfpPortState", OfpPortState_name, OfpPortState_value)
+	proto.RegisterEnum("openflow_13.OfpPortFeatures", OfpPortFeatures_name, OfpPortFeatures_value)
+	proto.RegisterEnum("openflow_13.OfpPortReason", OfpPortReason_name, OfpPortReason_value)
+	proto.RegisterEnum("openflow_13.OfpMatchType", OfpMatchType_name, OfpMatchType_value)
+	proto.RegisterEnum("openflow_13.OfpOxmClass", OfpOxmClass_name, OfpOxmClass_value)
+	proto.RegisterEnum("openflow_13.OxmOfbFieldTypes", OxmOfbFieldTypes_name, OxmOfbFieldTypes_value)
+	proto.RegisterEnum("openflow_13.OfpVlanId", OfpVlanId_name, OfpVlanId_value)
+	proto.RegisterEnum("openflow_13.OfpIpv6ExthdrFlags", OfpIpv6ExthdrFlags_name, OfpIpv6ExthdrFlags_value)
+	proto.RegisterEnum("openflow_13.OfpActionType", OfpActionType_name, OfpActionType_value)
+	proto.RegisterEnum("openflow_13.OfpControllerMaxLen", OfpControllerMaxLen_name, OfpControllerMaxLen_value)
+	proto.RegisterEnum("openflow_13.OfpInstructionType", OfpInstructionType_name, OfpInstructionType_value)
+	proto.RegisterEnum("openflow_13.OfpFlowModCommand", OfpFlowModCommand_name, OfpFlowModCommand_value)
+	proto.RegisterEnum("openflow_13.OfpFlowModFlags", OfpFlowModFlags_name, OfpFlowModFlags_value)
+	proto.RegisterEnum("openflow_13.OfpGroup", OfpGroup_name, OfpGroup_value)
+	proto.RegisterEnum("openflow_13.OfpGroupModCommand", OfpGroupModCommand_name, OfpGroupModCommand_value)
+	proto.RegisterEnum("openflow_13.OfpGroupType", OfpGroupType_name, OfpGroupType_value)
+	proto.RegisterEnum("openflow_13.OfpPacketInReason", OfpPacketInReason_name, OfpPacketInReason_value)
+	proto.RegisterEnum("openflow_13.OfpFlowRemovedReason", OfpFlowRemovedReason_name, OfpFlowRemovedReason_value)
+	proto.RegisterEnum("openflow_13.OfpMeter", OfpMeter_name, OfpMeter_value)
+	proto.RegisterEnum("openflow_13.OfpMeterBandType", OfpMeterBandType_name, OfpMeterBandType_value)
+	proto.RegisterEnum("openflow_13.OfpMeterModCommand", OfpMeterModCommand_name, OfpMeterModCommand_value)
+	proto.RegisterEnum("openflow_13.OfpMeterFlags", OfpMeterFlags_name, OfpMeterFlags_value)
+	proto.RegisterEnum("openflow_13.OfpErrorType", OfpErrorType_name, OfpErrorType_value)
+	proto.RegisterEnum("openflow_13.OfpHelloFailedCode", OfpHelloFailedCode_name, OfpHelloFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpBadRequestCode", OfpBadRequestCode_name, OfpBadRequestCode_value)
+	proto.RegisterEnum("openflow_13.OfpBadActionCode", OfpBadActionCode_name, OfpBadActionCode_value)
+	proto.RegisterEnum("openflow_13.OfpBadInstructionCode", OfpBadInstructionCode_name, OfpBadInstructionCode_value)
+	proto.RegisterEnum("openflow_13.OfpBadMatchCode", OfpBadMatchCode_name, OfpBadMatchCode_value)
+	proto.RegisterEnum("openflow_13.OfpFlowModFailedCode", OfpFlowModFailedCode_name, OfpFlowModFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpGroupModFailedCode", OfpGroupModFailedCode_name, OfpGroupModFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpPortModFailedCode", OfpPortModFailedCode_name, OfpPortModFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpTableModFailedCode", OfpTableModFailedCode_name, OfpTableModFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpQueueOpFailedCode", OfpQueueOpFailedCode_name, OfpQueueOpFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpSwitchConfigFailedCode", OfpSwitchConfigFailedCode_name, OfpSwitchConfigFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpRoleRequestFailedCode", OfpRoleRequestFailedCode_name, OfpRoleRequestFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpMeterModFailedCode", OfpMeterModFailedCode_name, OfpMeterModFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpTableFeaturesFailedCode", OfpTableFeaturesFailedCode_name, OfpTableFeaturesFailedCode_value)
+	proto.RegisterEnum("openflow_13.OfpMultipartType", OfpMultipartType_name, OfpMultipartType_value)
+	proto.RegisterEnum("openflow_13.OfpMultipartRequestFlags", OfpMultipartRequestFlags_name, OfpMultipartRequestFlags_value)
+	proto.RegisterEnum("openflow_13.OfpMultipartReplyFlags", OfpMultipartReplyFlags_name, OfpMultipartReplyFlags_value)
+	proto.RegisterEnum("openflow_13.OfpTableFeaturePropType", OfpTableFeaturePropType_name, OfpTableFeaturePropType_value)
+	proto.RegisterEnum("openflow_13.OfpGroupCapabilities", OfpGroupCapabilities_name, OfpGroupCapabilities_value)
+	proto.RegisterEnum("openflow_13.OfpQueueProperties", OfpQueueProperties_name, OfpQueueProperties_value)
+	proto.RegisterEnum("openflow_13.OfpControllerRole", OfpControllerRole_name, OfpControllerRole_value)
+	proto.RegisterType((*OfpHeader)(nil), "openflow_13.ofp_header")
+	proto.RegisterType((*OfpHelloElemHeader)(nil), "openflow_13.ofp_hello_elem_header")
+	proto.RegisterType((*OfpHelloElemVersionbitmap)(nil), "openflow_13.ofp_hello_elem_versionbitmap")
+	proto.RegisterType((*OfpHello)(nil), "openflow_13.ofp_hello")
+	proto.RegisterType((*OfpSwitchConfig)(nil), "openflow_13.ofp_switch_config")
+	proto.RegisterType((*OfpTableMod)(nil), "openflow_13.ofp_table_mod")
+	proto.RegisterType((*OfpPort)(nil), "openflow_13.ofp_port")
+	proto.RegisterType((*OfpSwitchFeatures)(nil), "openflow_13.ofp_switch_features")
+	proto.RegisterType((*OfpPortStatus)(nil), "openflow_13.ofp_port_status")
+	proto.RegisterType((*OfpPortMod)(nil), "openflow_13.ofp_port_mod")
+	proto.RegisterType((*OfpMatch)(nil), "openflow_13.ofp_match")
+	proto.RegisterType((*OfpOxmField)(nil), "openflow_13.ofp_oxm_field")
+	proto.RegisterType((*OfpOxmOfbField)(nil), "openflow_13.ofp_oxm_ofb_field")
+	proto.RegisterType((*OfpOxmExperimenterField)(nil), "openflow_13.ofp_oxm_experimenter_field")
+	proto.RegisterType((*OfpAction)(nil), "openflow_13.ofp_action")
+	proto.RegisterType((*OfpActionOutput)(nil), "openflow_13.ofp_action_output")
+	proto.RegisterType((*OfpActionMplsTtl)(nil), "openflow_13.ofp_action_mpls_ttl")
+	proto.RegisterType((*OfpActionPush)(nil), "openflow_13.ofp_action_push")
+	proto.RegisterType((*OfpActionPopMpls)(nil), "openflow_13.ofp_action_pop_mpls")
+	proto.RegisterType((*OfpActionGroup)(nil), "openflow_13.ofp_action_group")
+	proto.RegisterType((*OfpActionNwTtl)(nil), "openflow_13.ofp_action_nw_ttl")
+	proto.RegisterType((*OfpActionSetField)(nil), "openflow_13.ofp_action_set_field")
+	proto.RegisterType((*OfpActionExperimenter)(nil), "openflow_13.ofp_action_experimenter")
+	proto.RegisterType((*OfpInstruction)(nil), "openflow_13.ofp_instruction")
+	proto.RegisterType((*OfpInstructionGotoTable)(nil), "openflow_13.ofp_instruction_goto_table")
+	proto.RegisterType((*OfpInstructionWriteMetadata)(nil), "openflow_13.ofp_instruction_write_metadata")
+	proto.RegisterType((*OfpInstructionActions)(nil), "openflow_13.ofp_instruction_actions")
+	proto.RegisterType((*OfpInstructionMeter)(nil), "openflow_13.ofp_instruction_meter")
+	proto.RegisterType((*OfpInstructionExperimenter)(nil), "openflow_13.ofp_instruction_experimenter")
+	proto.RegisterType((*OfpFlowMod)(nil), "openflow_13.ofp_flow_mod")
+	proto.RegisterType((*OfpBucket)(nil), "openflow_13.ofp_bucket")
+	proto.RegisterType((*OfpGroupMod)(nil), "openflow_13.ofp_group_mod")
+	proto.RegisterType((*OfpPacketOut)(nil), "openflow_13.ofp_packet_out")
+	proto.RegisterType((*OfpPacketIn)(nil), "openflow_13.ofp_packet_in")
+	proto.RegisterType((*OfpFlowRemoved)(nil), "openflow_13.ofp_flow_removed")
+	proto.RegisterType((*OfpMeterBandHeader)(nil), "openflow_13.ofp_meter_band_header")
+	proto.RegisterType((*OfpMeterBandDrop)(nil), "openflow_13.ofp_meter_band_drop")
+	proto.RegisterType((*OfpMeterBandDscpRemark)(nil), "openflow_13.ofp_meter_band_dscp_remark")
+	proto.RegisterType((*OfpMeterBandExperimenter)(nil), "openflow_13.ofp_meter_band_experimenter")
+	proto.RegisterType((*OfpMeterMod)(nil), "openflow_13.ofp_meter_mod")
+	proto.RegisterType((*OfpErrorMsg)(nil), "openflow_13.ofp_error_msg")
+	proto.RegisterType((*OfpErrorExperimenterMsg)(nil), "openflow_13.ofp_error_experimenter_msg")
+	proto.RegisterType((*OfpMultipartRequest)(nil), "openflow_13.ofp_multipart_request")
+	proto.RegisterType((*OfpMultipartReply)(nil), "openflow_13.ofp_multipart_reply")
+	proto.RegisterType((*OfpDesc)(nil), "openflow_13.ofp_desc")
+	proto.RegisterType((*OfpFlowStatsRequest)(nil), "openflow_13.ofp_flow_stats_request")
+	proto.RegisterType((*OfpFlowStats)(nil), "openflow_13.ofp_flow_stats")
+	proto.RegisterType((*OfpAggregateStatsRequest)(nil), "openflow_13.ofp_aggregate_stats_request")
+	proto.RegisterType((*OfpAggregateStatsReply)(nil), "openflow_13.ofp_aggregate_stats_reply")
+	proto.RegisterType((*OfpTableFeatureProperty)(nil), "openflow_13.ofp_table_feature_property")
+	proto.RegisterType((*OfpTableFeaturePropInstructions)(nil), "openflow_13.ofp_table_feature_prop_instructions")
+	proto.RegisterType((*OfpTableFeaturePropNextTables)(nil), "openflow_13.ofp_table_feature_prop_next_tables")
+	proto.RegisterType((*OfpTableFeaturePropActions)(nil), "openflow_13.ofp_table_feature_prop_actions")
+	proto.RegisterType((*OfpTableFeaturePropOxm)(nil), "openflow_13.ofp_table_feature_prop_oxm")
+	proto.RegisterType((*OfpTableFeaturePropExperimenter)(nil), "openflow_13.ofp_table_feature_prop_experimenter")
+	proto.RegisterType((*OfpTableFeatures)(nil), "openflow_13.ofp_table_features")
+	proto.RegisterType((*OfpTableStats)(nil), "openflow_13.ofp_table_stats")
+	proto.RegisterType((*OfpPortStatsRequest)(nil), "openflow_13.ofp_port_stats_request")
+	proto.RegisterType((*OfpPortStats)(nil), "openflow_13.ofp_port_stats")
+	proto.RegisterType((*OfpGroupStatsRequest)(nil), "openflow_13.ofp_group_stats_request")
+	proto.RegisterType((*OfpBucketCounter)(nil), "openflow_13.ofp_bucket_counter")
+	proto.RegisterType((*OfpGroupStats)(nil), "openflow_13.ofp_group_stats")
+	proto.RegisterType((*OfpGroupDesc)(nil), "openflow_13.ofp_group_desc")
+	proto.RegisterType((*OfpGroupEntry)(nil), "openflow_13.ofp_group_entry")
+	proto.RegisterType((*OfpGroupFeatures)(nil), "openflow_13.ofp_group_features")
+	proto.RegisterType((*OfpMeterMultipartRequest)(nil), "openflow_13.ofp_meter_multipart_request")
+	proto.RegisterType((*OfpMeterBandStats)(nil), "openflow_13.ofp_meter_band_stats")
+	proto.RegisterType((*OfpMeterStats)(nil), "openflow_13.ofp_meter_stats")
+	proto.RegisterType((*OfpMeterConfig)(nil), "openflow_13.ofp_meter_config")
+	proto.RegisterType((*OfpMeterFeatures)(nil), "openflow_13.ofp_meter_features")
+	proto.RegisterType((*OfpMeterEntry)(nil), "openflow_13.ofp_meter_entry")
+	proto.RegisterType((*OfpExperimenterMultipartHeader)(nil), "openflow_13.ofp_experimenter_multipart_header")
+	proto.RegisterType((*OfpExperimenterHeader)(nil), "openflow_13.ofp_experimenter_header")
+	proto.RegisterType((*OfpQueuePropHeader)(nil), "openflow_13.ofp_queue_prop_header")
+	proto.RegisterType((*OfpQueuePropMinRate)(nil), "openflow_13.ofp_queue_prop_min_rate")
+	proto.RegisterType((*OfpQueuePropMaxRate)(nil), "openflow_13.ofp_queue_prop_max_rate")
+	proto.RegisterType((*OfpQueuePropExperimenter)(nil), "openflow_13.ofp_queue_prop_experimenter")
+	proto.RegisterType((*OfpPacketQueue)(nil), "openflow_13.ofp_packet_queue")
+	proto.RegisterType((*OfpQueueGetConfigRequest)(nil), "openflow_13.ofp_queue_get_config_request")
+	proto.RegisterType((*OfpQueueGetConfigReply)(nil), "openflow_13.ofp_queue_get_config_reply")
+	proto.RegisterType((*OfpActionSetQueue)(nil), "openflow_13.ofp_action_set_queue")
+	proto.RegisterType((*OfpQueueStatsRequest)(nil), "openflow_13.ofp_queue_stats_request")
+	proto.RegisterType((*OfpQueueStats)(nil), "openflow_13.ofp_queue_stats")
+	proto.RegisterType((*OfpRoleRequest)(nil), "openflow_13.ofp_role_request")
+	proto.RegisterType((*OfpAsyncConfig)(nil), "openflow_13.ofp_async_config")
+	proto.RegisterType((*MeterModUpdate)(nil), "openflow_13.MeterModUpdate")
+	proto.RegisterType((*MeterStatsReply)(nil), "openflow_13.MeterStatsReply")
+	proto.RegisterType((*FlowTableUpdate)(nil), "openflow_13.FlowTableUpdate")
+	proto.RegisterType((*FlowGroupTableUpdate)(nil), "openflow_13.FlowGroupTableUpdate")
+	proto.RegisterType((*Flows)(nil), "openflow_13.Flows")
+	proto.RegisterType((*Meters)(nil), "openflow_13.Meters")
+	proto.RegisterType((*FlowGroups)(nil), "openflow_13.FlowGroups")
+	proto.RegisterType((*FlowChanges)(nil), "openflow_13.FlowChanges")
+	proto.RegisterType((*FlowGroupChanges)(nil), "openflow_13.FlowGroupChanges")
+	proto.RegisterType((*PacketIn)(nil), "openflow_13.PacketIn")
+	proto.RegisterType((*PacketOut)(nil), "openflow_13.PacketOut")
+	proto.RegisterType((*ChangeEvent)(nil), "openflow_13.ChangeEvent")
+}
+
+func init() { proto.RegisterFile("voltha_protos/openflow_13.proto", fileDescriptor_08e3a4e375aeddc7) }
+
+var fileDescriptor_08e3a4e375aeddc7 = []byte{
+	// 8470 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x7d, 0x5b, 0x8c, 0x1c, 0x59,
+	0x96, 0x90, 0xf3, 0x51, 0x55, 0x99, 0x37, 0xab, 0xca, 0xe1, 0xf0, 0x2b, 0xed, 0xb2, 0xdb, 0x76,
+	0x4e, 0xf7, 0x4c, 0x4f, 0x0e, 0xdb, 0x6e, 0xbb, 0xdd, 0x9e, 0xd9, 0x79, 0x2c, 0x8e, 0xcc, 0x8c,
+	0xac, 0xcc, 0x76, 0xbe, 0x1c, 0x11, 0x55, 0xb6, 0x07, 0x41, 0x28, 0x2b, 0x33, 0x5c, 0x95, 0xeb,
+	0xcc, 0x8c, 0x9c, 0x88, 0xa8, 0x72, 0x79, 0x99, 0x45, 0x86, 0x15, 0x42, 0x02, 0x76, 0x17, 0xb4,
+	0x1f, 0x23, 0xc1, 0x22, 0xb1, 0x02, 0x3e, 0x10, 0x12, 0x1f, 0x48, 0x48, 0x48, 0x7c, 0xaf, 0x04,
+	0x12, 0x02, 0x69, 0x25, 0xb4, 0x3f, 0xbb, 0x7f, 0xcb, 0x0f, 0xd2, 0xfe, 0xc3, 0x32, 0x8c, 0xd1,
+	0xb9, 0xe7, 0xdc, 0x1b, 0x37, 0xf2, 0x51, 0x5d, 0x33, 0xf4, 0xf0, 0xc1, 0x97, 0x33, 0xce, 0xeb,
+	0x9e, 0x7b, 0xee, 0x39, 0xe7, 0x9e, 0x7b, 0xe2, 0x46, 0x99, 0xdd, 0x39, 0xf1, 0xc7, 0xd1, 0x51,
+	0xdf, 0x9d, 0x05, 0x7e, 0xe4, 0x87, 0xf7, 0xfd, 0x99, 0x37, 0x7d, 0x35, 0xf6, 0xdf, 0xb8, 0x0f,
+	0x3e, 0xfb, 0x84, 0x83, 0xf4, 0x82, 0x02, 0xba, 0x79, 0xeb, 0xd0, 0xf7, 0x0f, 0xc7, 0xde, 0xfd,
+	0xfe, 0x6c, 0x74, 0xbf, 0x3f, 0x9d, 0xfa, 0x51, 0x3f, 0x1a, 0xf9, 0xd3, 0x10, 0x49, 0x4b, 0x03,
+	0xc6, 0xfc, 0x57, 0x33, 0xf7, 0xc8, 0xeb, 0x0f, 0xbd, 0x40, 0x2f, 0xb2, 0x8d, 0x13, 0x2f, 0x08,
+	0x47, 0xfe, 0xb4, 0x98, 0xba, 0x9b, 0xfa, 0x78, 0xcb, 0x12, 0x8f, 0xfa, 0x37, 0x59, 0x36, 0x7a,
+	0x3b, 0xf3, 0x8a, 0xe9, 0xbb, 0xa9, 0x8f, 0xb7, 0x1f, 0x5e, 0xfd, 0x44, 0x1d, 0x14, 0x04, 0x00,
+	0xd2, 0xe2, 0x24, 0xba, 0xc6, 0x32, 0xa7, 0xa3, 0x61, 0x31, 0xc3, 0x05, 0xc0, 0xcf, 0xd2, 0xbf,
+	0x4a, 0xb1, 0xab, 0x38, 0xca, 0x78, 0xec, 0xbb, 0xde, 0xd8, 0x9b, 0x88, 0x01, 0x1f, 0x91, 0xd8,
+	0x14, 0x17, 0x7b, 0x77, 0x41, 0xac, 0xc2, 0xa1, 0x8c, 0xf0, 0x8c, 0x6d, 0x91, 0x5e, 0x07, 0xa3,
+	0x68, 0xd2, 0x9f, 0x71, 0xad, 0x0a, 0x0f, 0xbf, 0x79, 0x16, 0x7b, 0x82, 0xa1, 0x71, 0xc1, 0x4a,
+	0x4a, 0xa8, 0xe4, 0xd9, 0x06, 0x90, 0x79, 0xd3, 0xa8, 0xf4, 0x1d, 0x76, 0xeb, 0x2c, 0x5e, 0x30,
+	0x12, 0xfe, 0x0a, 0x8b, 0xe9, 0xbb, 0x19, 0x30, 0x12, 0x3d, 0x96, 0x9e, 0xb2, 0xbc, 0xe4, 0xd4,
+	0x7f, 0x8d, 0xe5, 0x48, 0x62, 0x58, 0x4c, 0xdd, 0xcd, 0x7c, 0x5c, 0x78, 0x58, 0x3a, 0x4b, 0x3f,
+	0x34, 0x88, 0x25, 0x79, 0x4a, 0x6d, 0x76, 0x09, 0x48, 0xc2, 0x37, 0xa3, 0x68, 0x70, 0xe4, 0x0e,
+	0xfc, 0xe9, 0xab, 0xd1, 0xa1, 0x7e, 0x85, 0xad, 0xbd, 0x1a, 0xf7, 0x0f, 0x43, 0x5a, 0x1e, 0x7c,
+	0xd0, 0x4b, 0x6c, 0x6b, 0x32, 0x0a, 0x43, 0x37, 0xf4, 0xa6, 0x43, 0x77, 0xec, 0x4d, 0xb9, 0x3d,
+	0xb6, 0xac, 0x02, 0x00, 0x6d, 0x6f, 0x3a, 0x6c, 0x79, 0xd3, 0x52, 0x85, 0x6d, 0xf1, 0x75, 0xea,
+	0x1f, 0x8c, 0x3d, 0x77, 0xe2, 0x0f, 0xf5, 0x1b, 0x2c, 0x87, 0x0f, 0xa3, 0xa1, 0x58, 0x6c, 0xfe,
+	0xdc, 0x1c, 0xea, 0xd7, 0xd8, 0x3a, 0x8e, 0x47, 0x82, 0xe8, 0xa9, 0xf4, 0x4f, 0xd3, 0x2c, 0x07,
+	0x42, 0x66, 0x7e, 0x10, 0xe9, 0xd7, 0xd9, 0x06, 0xfc, 0xeb, 0x4e, 0x7d, 0x62, 0x5f, 0x87, 0xc7,
+	0x8e, 0x0f, 0x88, 0xa3, 0x37, 0x6e, 0x7f, 0x38, 0x0c, 0xc8, 0x3e, 0xeb, 0x47, 0x6f, 0x8c, 0xe1,
+	0x30, 0xd0, 0x75, 0x96, 0x9d, 0xf6, 0x27, 0x1e, 0xf7, 0x8c, 0xbc, 0xc5, 0x7f, 0x2b, 0x43, 0x65,
+	0xd5, 0xa1, 0x60, 0xa2, 0x61, 0xd4, 0x8f, 0xbc, 0xe2, 0x1a, 0x4e, 0x94, 0x3f, 0x80, 0x84, 0xc1,
+	0x71, 0x10, 0x14, 0xd7, 0x39, 0x90, 0xff, 0xd6, 0x3f, 0x60, 0xac, 0x3f, 0x3c, 0xf1, 0x82, 0x68,
+	0x14, 0x7a, 0xc3, 0xe2, 0x06, 0xc7, 0x28, 0x10, 0xfd, 0x16, 0xcb, 0x87, 0xc7, 0x33, 0xd0, 0xcd,
+	0x1b, 0x16, 0x73, 0x1c, 0x1d, 0x03, 0x40, 0xe2, 0xcc, 0xf3, 0x82, 0x62, 0x1e, 0x25, 0xc2, 0x6f,
+	0xfd, 0x36, 0x63, 0x20, 0xd9, 0x0d, 0x67, 0x9e, 0x37, 0x2c, 0x32, 0x64, 0x01, 0x88, 0x0d, 0x00,
+	0x7d, 0x87, 0xe5, 0x27, 0xfd, 0x53, 0xc2, 0x16, 0x38, 0x36, 0x37, 0xe9, 0x9f, 0x72, 0x64, 0xe9,
+	0xdf, 0xa6, 0xd8, 0x65, 0x65, 0xd9, 0x5e, 0x79, 0xfd, 0xe8, 0x38, 0xf0, 0x42, 0xfd, 0x0e, 0x2b,
+	0x0c, 0xfb, 0x51, 0x7f, 0xd6, 0x8f, 0x8e, 0x84, 0xc1, 0xb3, 0x16, 0x13, 0xa0, 0x26, 0x97, 0x3a,
+	0x75, 0x0f, 0x8e, 0x5f, 0xbd, 0xf2, 0x82, 0x90, 0xcc, 0x9e, 0x9b, 0x56, 0xf0, 0x19, 0xd6, 0x6a,
+	0x8a, 0x4b, 0x17, 0x52, 0x5c, 0x6d, 0x4c, 0x1d, 0xfe, 0xa8, 0xdf, 0x63, 0x9b, 0xfd, 0xe3, 0xd3,
+	0xd1, 0x78, 0xd4, 0x0f, 0xde, 0x82, 0x64, 0x34, 0x63, 0x41, 0xc2, 0x9a, 0x43, 0xbd, 0xc4, 0x36,
+	0x07, 0xfd, 0x59, 0xff, 0x60, 0x34, 0x1e, 0x45, 0x23, 0x2f, 0x24, 0x93, 0x26, 0x60, 0xa5, 0x80,
+	0x5d, 0x14, 0x2b, 0xeb, 0x82, 0xad, 0x8f, 0x43, 0xfd, 0x11, 0x5b, 0x0f, 0xbc, 0x7e, 0x48, 0xb9,
+	0x60, 0xfb, 0xe1, 0xad, 0x05, 0xf7, 0xe5, 0xd4, 0x48, 0x63, 0x11, 0x2d, 0x24, 0x8a, 0xa1, 0x17,
+	0x0e, 0x28, 0x24, 0xaf, 0x2e, 0xe5, 0xb1, 0x38, 0x49, 0xe9, 0xef, 0xa6, 0xd8, 0xa6, 0x14, 0x03,
+	0x2e, 0xf9, 0xf3, 0xbb, 0x54, 0xec, 0x3e, 0x99, 0x84, 0xfb, 0xe8, 0x2c, 0x3b, 0xe9, 0x87, 0xaf,
+	0xc9, 0x1a, 0xfc, 0x37, 0x38, 0x82, 0x74, 0x0b, 0xb2, 0x41, 0x0c, 0x28, 0xbd, 0xc1, 0xd8, 0x9d,
+	0xf4, 0xa3, 0xc1, 0x91, 0x7e, 0x3f, 0x91, 0x96, 0x76, 0x16, 0x26, 0xc1, 0xa9, 0xd4, 0x8c, 0xf4,
+	0xab, 0x8c, 0xf9, 0xa7, 0x13, 0xf7, 0xd5, 0xc8, 0x1b, 0x0f, 0x31, 0x2d, 0x14, 0x1e, 0xde, 0x5c,
+	0x60, 0x93, 0x24, 0x56, 0xde, 0x3f, 0x9d, 0xd4, 0x39, 0x71, 0xe9, 0xbf, 0xa7, 0x30, 0x32, 0x25,
+	0x52, 0xff, 0x36, 0x03, 0xb4, 0x3b, 0x18, 0xf7, 0xc3, 0x90, 0x54, 0x58, 0x2e, 0x8b, 0x53, 0x58,
+	0x39, 0xff, 0x74, 0x52, 0x85, 0x5f, 0xfa, 0x0f, 0x60, 0x0e, 0x07, 0x28, 0x85, 0x4f, 0xbd, 0xf0,
+	0xf0, 0x83, 0xa5, 0x8c, 0x92, 0xaa, 0x71, 0xc1, 0xca, 0xf9, 0xaf, 0x0e, 0xb8, 0x2a, 0xfa, 0x0b,
+	0xa6, 0x7b, 0xa7, 0x33, 0x2f, 0x18, 0x41, 0x02, 0xf2, 0x02, 0x92, 0xb3, 0xc6, 0xe5, 0x7c, 0x63,
+	0xa9, 0x9c, 0x45, 0xf2, 0xc6, 0x05, 0xeb, 0x92, 0x0a, 0xe5, 0x92, 0x2b, 0x1b, 0x6c, 0x8d, 0x63,
+	0x4b, 0x7f, 0xbc, 0x8d, 0x59, 0x2d, 0xa1, 0xc4, 0xd9, 0xbb, 0x80, 0x4a, 0xc9, 0x4d, 0x1e, 0x92,
+	0xcd, 0x6f, 0xb0, 0xdc, 0x51, 0x3f, 0x74, 0xf9, 0x3a, 0x83, 0xb7, 0xe5, 0xac, 0x8d, 0xa3, 0x7e,
+	0xd8, 0x86, 0xa5, 0xbe, 0xc2, 0xb2, 0xe0, 0x39, 0xe8, 0x14, 0x8d, 0x0b, 0x16, 0x7f, 0xd2, 0x3f,
+	0x62, 0x5b, 0xb3, 0xa3, 0xb7, 0xe1, 0x68, 0xd0, 0x1f, 0x73, 0x9f, 0x43, 0xef, 0x68, 0x5c, 0xb0,
+	0x36, 0x05, 0xb8, 0x07, 0x64, 0xdf, 0x60, 0xdb, 0x94, 0x25, 0xbd, 0xa8, 0x0f, 0x11, 0xca, 0x4d,
+	0x90, 0x85, 0x3d, 0x83, 0xc3, 0xdb, 0x04, 0xd6, 0x6f, 0xb0, 0x0d, 0x2f, 0x3a, 0x72, 0x87, 0x61,
+	0xc4, 0x13, 0xd2, 0x66, 0xe3, 0x82, 0xb5, 0xee, 0x45, 0x47, 0xb5, 0x30, 0x12, 0xa8, 0x30, 0x18,
+	0xf0, 0x8c, 0x24, 0x50, 0x76, 0x30, 0xd0, 0x77, 0x58, 0x0e, 0x50, 0x7c, 0xc2, 0x39, 0x52, 0x00,
+	0x88, 0x1d, 0x98, 0xd3, 0x0e, 0xcb, 0x9d, 0x8c, 0xfb, 0x53, 0xf7, 0x64, 0x34, 0xc4, 0x94, 0x04,
+	0x48, 0x80, 0xec, 0x8f, 0x86, 0x12, 0x39, 0x1b, 0xcc, 0x30, 0x2b, 0x09, 0x64, 0x6f, 0x30, 0x83,
+	0x11, 0x47, 0x33, 0x77, 0x18, 0x0e, 0x66, 0x98, 0x93, 0x60, 0xc4, 0xd1, 0xac, 0x16, 0x0e, 0x66,
+	0xfa, 0x75, 0xb6, 0x3e, 0x9a, 0xb9, 0xde, 0x60, 0x5a, 0xdc, 0x24, 0xcc, 0xda, 0x68, 0x66, 0x0e,
+	0xa6, 0x20, 0x70, 0x34, 0xc3, 0x32, 0xa2, 0xb8, 0x25, 0x04, 0x8e, 0x66, 0x3d, 0x5e, 0x44, 0x70,
+	0xe4, 0xc9, 0x23, 0x3e, 0x87, 0xed, 0x18, 0x79, 0xf2, 0x88, 0x26, 0xc1, 0x91, 0x30, 0xf7, 0x8b,
+	0x2a, 0x92, 0x26, 0x1f, 0x0d, 0x66, 0x9c, 0x51, 0x13, 0xaa, 0x44, 0x83, 0x19, 0xf0, 0x11, 0x0a,
+	0xd8, 0x2e, 0x29, 0x28, 0xe2, 0x3a, 0x1e, 0x22, 0x97, 0x2e, 0x50, 0xc7, 0x43, 0xc1, 0x05, 0x28,
+	0xe0, 0xba, 0xac, 0xa0, 0x80, 0x6b, 0x87, 0xe5, 0xc2, 0x41, 0x84, 0x6c, 0x57, 0x84, 0x22, 0x00,
+	0x21, 0x2d, 0x39, 0x12, 0x18, 0xaf, 0xaa, 0x48, 0xe0, 0xbc, 0xc7, 0x0a, 0xa3, 0xc1, 0x04, 0x26,
+	0xc1, 0x97, 0xe2, 0x1a, 0xe1, 0x19, 0x02, 0xf9, 0x6a, 0xc4, 0x24, 0x03, 0x7f, 0xe8, 0x15, 0xaf,
+	0x27, 0x49, 0xaa, 0xfe, 0xd0, 0x03, 0xdb, 0xf6, 0x83, 0x99, 0xeb, 0xcf, 0x8a, 0x45, 0x61, 0xdb,
+	0x7e, 0x30, 0xeb, 0xf2, 0xf5, 0x00, 0x44, 0x38, 0xeb, 0x17, 0x6f, 0x08, 0x9d, 0xfb, 0xc1, 0xcc,
+	0x9e, 0xf5, 0x05, 0x2a, 0x9a, 0xf5, 0x8b, 0x37, 0x15, 0x94, 0x13, 0xa3, 0xc2, 0xa3, 0x7e, 0x71,
+	0x47, 0xf8, 0x0d, 0x70, 0x1d, 0xc5, 0x5c, 0x47, 0xfd, 0xe2, 0x2d, 0x05, 0xe5, 0x1c, 0xf5, 0x69,
+	0x35, 0x1e, 0x73, 0x23, 0xdc, 0x26, 0x1c, 0xac, 0xc6, 0xe3, 0x78, 0xa9, 0x1e, 0x73, 0x23, 0x7c,
+	0xa0, 0x22, 0x85, 0x11, 0x00, 0xf9, 0x6a, 0xdc, 0x3f, 0xf0, 0xc6, 0xc5, 0x3b, 0x72, 0x86, 0xb3,
+	0x93, 0xc7, 0x75, 0x0e, 0x93, 0x46, 0x78, 0x8c, 0x76, 0xba, 0x9b, 0x30, 0xc2, 0xe3, 0x84, 0x9d,
+	0x1e, 0xa3, 0x9d, 0xee, 0x25, 0x49, 0xb8, 0x9d, 0xbe, 0xce, 0xb6, 0xf9, 0x40, 0xd3, 0xa1, 0x1b,
+	0xf5, 0x83, 0x43, 0x2f, 0x2a, 0x96, 0x48, 0x97, 0x4d, 0x80, 0x77, 0x86, 0x0e, 0x87, 0xea, 0x77,
+	0x49, 0xa1, 0xe9, 0xd0, 0x0d, 0xc3, 0x71, 0xf1, 0x6b, 0x44, 0x94, 0x47, 0x22, 0x3b, 0x1c, 0xab,
+	0x14, 0xd1, 0x78, 0x5c, 0xfc, 0x30, 0x49, 0xe1, 0x8c, 0xc7, 0xfa, 0x1d, 0xc6, 0x26, 0xb3, 0x71,
+	0xe8, 0xe2, 0x9c, 0x3e, 0x22, 0x6d, 0xf2, 0x00, 0x6b, 0xf1, 0x29, 0xdd, 0x60, 0x1b, 0x9c, 0x20,
+	0x1a, 0x14, 0xbf, 0x2e, 0x16, 0x00, 0x00, 0x0e, 0xb7, 0x16, 0x47, 0x1d, 0xf8, 0x61, 0xf1, 0x1b,
+	0xc2, 0x65, 0x00, 0x52, 0xf1, 0x43, 0x40, 0xce, 0x0e, 0x0e, 0xdc, 0x51, 0x38, 0x1a, 0x16, 0x3f,
+	0x16, 0xc8, 0xd9, 0xc1, 0x41, 0x33, 0x1c, 0x0d, 0xf5, 0xdb, 0x2c, 0x1f, 0x1d, 0x4f, 0xa7, 0xde,
+	0x18, 0x76, 0xe1, 0x6f, 0x52, 0xc6, 0xc8, 0x21, 0xa8, 0x39, 0x94, 0x96, 0xf6, 0x4e, 0xa3, 0xa3,
+	0x61, 0x50, 0x2c, 0xab, 0x96, 0x36, 0x39, 0x4c, 0xff, 0x94, 0x5d, 0x4e, 0x26, 0x1e, 0xcc, 0x6d,
+	0x23, 0x2e, 0x2b, 0x65, 0x5d, 0x4a, 0x64, 0x1f, 0x9e, 0xe7, 0x4a, 0x6c, 0x93, 0x32, 0x10, 0x92,
+	0xfe, 0x3a, 0x37, 0x46, 0xca, 0x62, 0x98, 0x86, 0x54, 0x9a, 0x30, 0x18, 0x20, 0xcd, 0x6b, 0x85,
+	0xc6, 0x0e, 0x06, 0x9c, 0xe6, 0x43, 0xb6, 0x25, 0xd2, 0x0e, 0x12, 0x4d, 0xb8, 0x7a, 0x29, 0xab,
+	0x40, 0xb9, 0x47, 0x50, 0x89, 0x8c, 0x80, 0x54, 0x81, 0xa0, 0xa2, 0xb4, 0x90, 0xa0, 0x92, 0x4a,
+	0x85, 0x2a, 0x95, 0xa2, 0x15, 0x85, 0x07, 0x12, 0xfd, 0x26, 0x11, 0x31, 0x8c, 0x11, 0x95, 0x26,
+	0x12, 0x34, 0x7f, 0x43, 0xa1, 0x71, 0x88, 0xe6, 0x23, 0x3e, 0xda, 0xe3, 0x58, 0xa7, 0xbf, 0x99,
+	0xa2, 0xf9, 0x15, 0x28, 0x00, 0x12, 0x64, 0x52, 0xa9, 0xbf, 0x95, 0x20, 0x13, 0x5a, 0x7d, 0x8b,
+	0x69, 0x4a, 0x38, 0x20, 0xe5, 0x6f, 0xa5, 0x68, 0xd8, 0xed, 0x38, 0x28, 0x84, 0x4c, 0xe1, 0x0d,
+	0x48, 0xf9, 0xf7, 0x05, 0x65, 0x81, 0x7c, 0x82, 0x93, 0xc1, 0x76, 0x22, 0xfc, 0x02, 0xe9, 0x7e,
+	0x3b, 0x45, 0x2b, 0xba, 0x29, 0xbc, 0x23, 0x31, 0x38, 0x7a, 0x08, 0x92, 0xfe, 0x4e, 0x62, 0x70,
+	0xf4, 0x13, 0x20, 0x86, 0x1d, 0xf5, 0xa4, 0x3f, 0x3e, 0xf6, 0x2a, 0xeb, 0x58, 0xe9, 0x94, 0x5c,
+	0x76, 0x73, 0xf5, 0xae, 0x0c, 0x25, 0x2d, 0x60, 0xf0, 0x90, 0x41, 0xc5, 0x15, 0x14, 0x19, 0x0d,
+	0x3c, 0x86, 0x81, 0x8f, 0x28, 0x4c, 0x54, 0x7f, 0x26, 0x60, 0xa5, 0x7f, 0x93, 0xc5, 0xa3, 0x62,
+	0x7f, 0x00, 0xe7, 0x47, 0xfd, 0xd3, 0xc4, 0x9e, 0xbd, 0x58, 0x1b, 0x22, 0x99, 0x5a, 0x23, 0x7d,
+	0x87, 0xad, 0xfb, 0xc7, 0xd1, 0xec, 0x38, 0xa2, 0xda, 0xf0, 0x83, 0x55, 0x3c, 0x48, 0x05, 0x41,
+	0x89, 0xbf, 0xf4, 0x1f, 0x50, 0x50, 0x46, 0xd1, 0x98, 0x6f, 0xe9, 0x85, 0x25, 0x27, 0x45, 0xe2,
+	0x15, 0x74, 0x22, 0x6c, 0x9d, 0x68, 0xac, 0x3f, 0x64, 0xd9, 0xd9, 0x71, 0x78, 0x44, 0x15, 0xd1,
+	0x4a, 0x55, 0x81, 0x86, 0xd7, 0x0a, 0xc7, 0xe1, 0x11, 0x0c, 0x39, 0xf3, 0x67, 0x5c, 0x1c, 0x55,
+	0x40, 0x2b, 0x87, 0x14, 0x74, 0x3c, 0x19, 0xf8, 0xb3, 0xf6, 0x6c, 0x1c, 0xea, 0x9f, 0xb3, 0xb5,
+	0xc3, 0xc0, 0x3f, 0x9e, 0xf1, 0xc2, 0xa0, 0xf0, 0xf0, 0xf6, 0x2a, 0x5e, 0x4e, 0x04, 0x9b, 0x06,
+	0xff, 0xa1, 0x7f, 0x9b, 0xad, 0x4f, 0xdf, 0xf0, 0x69, 0x6e, 0x9c, 0x6d, 0x22, 0xa4, 0x02, 0xc6,
+	0xe9, 0x1b, 0x98, 0xe2, 0x13, 0x96, 0x0f, 0xbd, 0x88, 0x2a, 0xb6, 0x1c, 0xe7, 0xbd, 0xb7, 0x8a,
+	0x57, 0x12, 0x42, 0x7e, 0x0a, 0xbd, 0x08, 0x8b, 0xbf, 0x2f, 0xe6, 0x5c, 0x20, 0xcf, 0x85, 0x7c,
+	0xb8, 0x4a, 0x88, 0x4a, 0x0b, 0x49, 0x5c, 0x7d, 0xae, 0xe4, 0xd8, 0x3a, 0x92, 0x95, 0x9e, 0x60,
+	0xb9, 0x97, 0x58, 0x58, 0x7e, 0xe6, 0x82, 0xf2, 0x2b, 0x45, 0x67, 0x2e, 0x3a, 0x4d, 0xc2, 0xa1,
+	0x2a, 0x3e, 0xbc, 0xae, 0x4f, 0xfa, 0xa7, 0x70, 0x6e, 0xfd, 0x14, 0xcf, 0x53, 0x73, 0xcb, 0x0b,
+	0xc5, 0x9f, 0x74, 0x09, 0x3a, 0xbd, 0xd2, 0x72, 0x97, 0xee, 0xe3, 0x51, 0x46, 0x59, 0x55, 0x28,
+	0xfd, 0xbd, 0xe8, 0xc8, 0x0b, 0xa4, 0xc7, 0x6e, 0x59, 0x31, 0xa0, 0xf4, 0x59, 0x62, 0x08, 0xb1,
+	0x9c, 0x5f, 0xc2, 0xf4, 0x2b, 0x4c, 0x9b, 0x5f, 0x47, 0x50, 0x8a, 0xff, 0x50, 0x8e, 0xd4, 0xfc,
+	0xb9, 0x39, 0x2c, 0x95, 0x13, 0x86, 0xc0, 0xe5, 0xd3, 0xaf, 0xca, 0xe5, 0xa6, 0xe3, 0x3c, 0x5f,
+	0xcc, 0x52, 0x83, 0x5d, 0x59, 0xb6, 0x5c, 0xfa, 0xa7, 0x54, 0x45, 0x73, 0xea, 0xb3, 0xcf, 0x17,
+	0x54, 0x6e, 0x3f, 0x63, 0xd7, 0x57, 0xac, 0xd9, 0x42, 0xc8, 0xa7, 0x16, 0x43, 0x1e, 0x16, 0x8a,
+	0xd7, 0xbf, 0xb0, 0x22, 0x9b, 0x16, 0xff, 0x5d, 0xfa, 0xfd, 0x0c, 0x9a, 0x77, 0x34, 0x0d, 0xa3,
+	0xe0, 0x18, 0x73, 0x81, 0xae, 0xe4, 0x82, 0x2d, 0x8a, 0xf6, 0x06, 0x63, 0x87, 0x7e, 0xe4, 0xe3,
+	0xa9, 0x95, 0x22, 0x7e, 0xf1, 0x10, 0xa1, 0x48, 0x71, 0x63, 0x72, 0xd8, 0xad, 0xe1, 0x89, 0x1f,
+	0x71, 0x75, 0x87, 0x6d, 0xbf, 0x09, 0x46, 0x91, 0x52, 0x8f, 0x63, 0x0e, 0xf8, 0xd6, 0x99, 0xd2,
+	0x92, 0x2c, 0x50, 0xbc, 0x73, 0x88, 0x2c, 0xde, 0x9f, 0xb0, 0x0d, 0x34, 0x4b, 0x48, 0x79, 0xe1,
+	0xc3, 0x33, 0xc5, 0x11, 0x2d, 0xc4, 0x38, 0xfd, 0xd4, 0xbf, 0xcb, 0xd6, 0x26, 0x1e, 0x98, 0x0e,
+	0xf3, 0x43, 0xe9, 0x4c, 0x7e, 0x4e, 0x09, 0xf1, 0xca, 0x7f, 0xe8, 0xdd, 0x39, 0xeb, 0xaf, 0xaf,
+	0x68, 0x60, 0xa9, 0x22, 0xce, 0x0c, 0xb9, 0x75, 0x5c, 0xaa, 0xd2, 0xb7, 0x71, 0x1b, 0x58, 0x6e,
+	0xd7, 0x33, 0x7a, 0x3e, 0xa5, 0x3e, 0xfb, 0xe0, 0x6c, 0x13, 0xea, 0x37, 0x59, 0x4e, 0xae, 0x00,
+	0xf6, 0x2f, 0xe4, 0xb3, 0xfe, 0x35, 0xb6, 0x95, 0x2c, 0x5a, 0xd2, 0x9c, 0x60, 0x73, 0xa2, 0x54,
+	0x2b, 0xa5, 0x16, 0x7a, 0xe3, 0x12, 0xb3, 0xea, 0x0f, 0xe2, 0xd5, 0xc0, 0x5e, 0xd9, 0xf5, 0x15,
+	0x89, 0x47, 0x9a, 0xbf, 0xf4, 0x10, 0x7b, 0x8a, 0x0b, 0x46, 0xe6, 0xa9, 0x01, 0x7e, 0x28, 0x93,
+	0xe4, 0xcf, 0xcd, 0x61, 0x69, 0x1f, 0x5b, 0x7b, 0xab, 0xac, 0xfa, 0x0b, 0x07, 0xc5, 0x9f, 0x64,
+	0xb0, 0x93, 0xc1, 0xf5, 0x9d, 0xf8, 0xd4, 0x41, 0xf3, 0x5f, 0x8f, 0x3c, 0xb2, 0x14, 0x3d, 0xe9,
+	0x77, 0x58, 0x01, 0x7f, 0xa9, 0x56, 0x62, 0x08, 0xe2, 0x45, 0x80, 0xba, 0x42, 0x99, 0x64, 0x57,
+	0xee, 0x7b, 0x6c, 0x63, 0xe0, 0x4f, 0x26, 0xfd, 0x29, 0x9e, 0xed, 0xb7, 0x97, 0x64, 0x78, 0x31,
+	0xbe, 0x4b, 0x84, 0x96, 0xe0, 0xd0, 0xef, 0xb1, 0xcd, 0xd1, 0x70, 0xec, 0xb9, 0xd1, 0x68, 0xe2,
+	0xf9, 0xc7, 0x11, 0xf5, 0x3f, 0x0a, 0x00, 0x73, 0x10, 0x04, 0x24, 0x47, 0xfd, 0x60, 0x28, 0x49,
+	0xb0, 0xc9, 0x56, 0x00, 0x98, 0x20, 0xb9, 0xc9, 0x72, 0xb3, 0x60, 0xe4, 0x07, 0xa3, 0xe8, 0x2d,
+	0x75, 0xda, 0xe4, 0xb3, 0xbe, 0xc3, 0xf2, 0xd8, 0xbe, 0x02, 0xd5, 0xb1, 0xcf, 0x96, 0x43, 0x40,
+	0x93, 0x37, 0x1b, 0xfd, 0xe3, 0x08, 0x4f, 0xdd, 0xd8, 0x6a, 0xdb, 0xf0, 0x8f, 0x23, 0x7e, 0xdc,
+	0xde, 0x61, 0x79, 0x40, 0xe1, 0x76, 0x89, 0xcd, 0x36, 0xa0, 0xdd, 0xe5, 0x19, 0x55, 0xf6, 0x3b,
+	0x0b, 0x6a, 0xbf, 0xf3, 0x2f, 0xb1, 0x35, 0xde, 0x81, 0xe1, 0xe7, 0xd9, 0xc2, 0xc3, 0x6b, 0xcb,
+	0xfb, 0x33, 0x16, 0x12, 0xe9, 0x4f, 0xd8, 0xa6, 0xb2, 0xe0, 0x61, 0x71, 0x8b, 0x3b, 0xd8, 0xad,
+	0xb3, 0x62, 0xcd, 0x4a, 0x70, 0x94, 0x7e, 0x92, 0xc2, 0xd2, 0xe7, 0xe0, 0x78, 0xf0, 0xda, 0x8b,
+	0x60, 0x71, 0xdf, 0x78, 0xa3, 0xc3, 0x23, 0xb1, 0x83, 0xd1, 0x13, 0x14, 0x59, 0x6f, 0x78, 0x63,
+	0x88, 0x4f, 0x13, 0xb7, 0xb1, 0x3c, 0x87, 0xf0, 0x89, 0xde, 0x61, 0x05, 0x44, 0xe3, 0x54, 0x71,
+	0x75, 0x91, 0x03, 0x27, 0xfb, 0x40, 0x4d, 0x49, 0xe7, 0x0b, 0x82, 0xff, 0x44, 0xcd, 0x23, 0xdc,
+	0x76, 0xc0, 0xf3, 0xbe, 0x1f, 0x7b, 0x09, 0x96, 0x66, 0x8b, 0x79, 0x49, 0x12, 0x2f, 0xba, 0xc9,
+	0xfd, 0x44, 0x9b, 0x7f, 0x67, 0x05, 0xab, 0x52, 0xd4, 0xa9, 0x5b, 0x5e, 0x26, 0xb1, 0xe5, 0xc1,
+	0x74, 0xd0, 0x60, 0xab, 0xa7, 0x83, 0x78, 0x4b, 0xd0, 0x95, 0x7e, 0x3b, 0xc5, 0xb6, 0x79, 0x47,
+	0xb0, 0x0f, 0xcf, 0x50, 0x2f, 0x24, 0xdd, 0x2a, 0x35, 0xe7, 0x56, 0xd7, 0xd9, 0xc6, 0x68, 0xaa,
+	0x9a, 0x7b, 0x7d, 0x34, 0xe5, 0xb6, 0x56, 0x4c, 0x99, 0x39, 0x9f, 0x29, 0x65, 0x5c, 0x67, 0xd5,
+	0xb8, 0x26, 0xf3, 0x92, 0x3e, 0xa3, 0xe9, 0xd9, 0xea, 0xfc, 0xaa, 0xec, 0x98, 0xa6, 0x57, 0x04,
+	0xa8, 0x14, 0x34, 0xdf, 0x36, 0x3d, 0x23, 0xee, 0xe3, 0x5c, 0x92, 0x4d, 0xe4, 0x12, 0x19, 0x05,
+	0x6b, 0xe7, 0x89, 0x02, 0x31, 0xbd, 0x75, 0x65, 0x7a, 0xff, 0x24, 0x83, 0x45, 0x0c, 0x67, 0x0a,
+	0xbc, 0x89, 0x7f, 0xe2, 0xad, 0x4e, 0x5d, 0x6a, 0xec, 0xa7, 0xe7, 0x62, 0xff, 0xfb, 0x72, 0xe2,
+	0x19, 0x3e, 0xf1, 0x0f, 0x97, 0x67, 0x26, 0x1a, 0xe2, 0xac, 0xb9, 0x67, 0x93, 0x73, 0xbf, 0xc7,
+	0x36, 0x87, 0xc7, 0x41, 0x9f, 0x0a, 0xa1, 0x81, 0x48, 0x5b, 0x02, 0x66, 0x7b, 0x03, 0xd8, 0x7a,
+	0x24, 0xc9, 0x14, 0x68, 0x30, 0x6f, 0x49, 0xbe, 0x4e, 0xe8, 0x0d, 0x16, 0xd2, 0xdf, 0xc6, 0x97,
+	0xa7, 0xbf, 0xdc, 0x62, 0xfa, 0xbb, 0xc7, 0x36, 0x69, 0x01, 0x07, 0xfe, 0xf1, 0x14, 0x33, 0x59,
+	0xd6, 0x2a, 0x20, 0xac, 0x0a, 0x20, 0xc8, 0x01, 0x07, 0x6f, 0x23, 0x8f, 0x08, 0x18, 0x27, 0xc8,
+	0x03, 0x04, 0xd1, 0x72, 0xcd, 0xde, 0x9e, 0x63, 0xcd, 0x4a, 0x7f, 0x92, 0xc6, 0x3d, 0x0e, 0xb7,
+	0xb3, 0x83, 0xfe, 0x74, 0x78, 0xde, 0xf7, 0x66, 0x0a, 0x87, 0x12, 0xac, 0x3a, 0xcb, 0x06, 0xfd,
+	0xc8, 0xa3, 0xe5, 0xe3, 0xbf, 0xb9, 0xc2, 0xc7, 0x41, 0x18, 0xb9, 0xe1, 0xe8, 0x37, 0x3c, 0x72,
+	0xbd, 0x3c, 0x87, 0xd8, 0xa3, 0xdf, 0xf0, 0xf4, 0xc7, 0x2c, 0x3b, 0x0c, 0xfc, 0x19, 0xd5, 0x48,
+	0x67, 0x0e, 0x04, 0x74, 0x70, 0x7e, 0x82, 0x7f, 0xf5, 0x2f, 0x58, 0x61, 0x18, 0x0e, 0x66, 0xb0,
+	0xe4, 0xfd, 0xe0, 0xf5, 0xca, 0x26, 0xb2, 0xca, 0x1e, 0x93, 0x37, 0x2e, 0x58, 0x0c, 0x1e, 0x2d,
+	0xfe, 0xa4, 0x77, 0x96, 0x16, 0x4b, 0x1f, 0x9f, 0x25, 0xec, 0x5c, 0xb5, 0xd2, 0x55, 0xac, 0xfb,
+	0xe7, 0xa6, 0x50, 0xfa, 0x1e, 0x96, 0x50, 0xcb, 0x55, 0x03, 0x7b, 0xcd, 0x02, 0x6f, 0xe0, 0x8e,
+	0xbd, 0x13, 0x4f, 0xd4, 0xed, 0x79, 0x80, 0xb4, 0x00, 0x50, 0x32, 0xd8, 0xce, 0x19, 0xaa, 0x9c,
+	0xa7, 0xc0, 0x28, 0xfd, 0x3b, 0x4a, 0x3a, 0x28, 0xe3, 0x9c, 0x39, 0x5d, 0x12, 0x2f, 0xe6, 0x74,
+	0xb9, 0x87, 0xa6, 0xd5, 0x3d, 0x54, 0xad, 0x92, 0x32, 0x89, 0x2a, 0x49, 0xff, 0x0e, 0x5b, 0x03,
+	0xcd, 0x45, 0xda, 0x2e, 0x9d, 0x65, 0x68, 0x7a, 0x6d, 0x89, 0x0c, 0xa5, 0x1f, 0xa3, 0xe6, 0x5e,
+	0x10, 0xf8, 0x81, 0x3b, 0x09, 0x0f, 0xf5, 0xfb, 0x6c, 0x5d, 0xe9, 0x39, 0x2c, 0x4b, 0xc3, 0x24,
+	0x80, 0xc8, 0xe4, 0x51, 0x22, 0xad, 0x1c, 0x25, 0x74, 0x96, 0xe5, 0x7d, 0xc5, 0x0c, 0xbd, 0xf5,
+	0xf3, 0x87, 0xde, 0xd2, 0x6c, 0xfd, 0x5b, 0x29, 0x5c, 0x39, 0x1c, 0x3e, 0xd1, 0x05, 0x01, 0x5d,
+	0x96, 0x9d, 0x52, 0x6e, 0xb0, 0x9c, 0x77, 0x8a, 0x1b, 0x1a, 0x0d, 0xb9, 0xe1, 0x9d, 0xce, 0x78,
+	0x53, 0x73, 0x7e, 0xa9, 0x32, 0x67, 0xd4, 0x82, 0xaa, 0x16, 0x27, 0x14, 0xb3, 0xc7, 0xe3, 0x68,
+	0x34, 0xeb, 0xf3, 0x17, 0x64, 0x3f, 0x3a, 0xf6, 0xc2, 0x48, 0xff, 0x2c, 0x11, 0xb3, 0x77, 0x16,
+	0xad, 0x2a, 0x39, 0x94, 0x90, 0x5d, 0xbe, 0x78, 0x3a, 0xcb, 0x1e, 0xf8, 0xc3, 0xb7, 0x5c, 0xa7,
+	0x4d, 0x8b, 0xff, 0x2e, 0x45, 0xe4, 0xcd, 0xca, 0xb8, 0xb3, 0xf1, 0xdb, 0x5f, 0xf6, 0xa8, 0xbf,
+	0x9b, 0xc2, 0x57, 0xc2, 0x43, 0x2f, 0x1c, 0x70, 0x9f, 0x7a, 0x15, 0xf0, 0xdf, 0x7c, 0xbc, 0xbc,
+	0xb5, 0x31, 0x79, 0x15, 0xd4, 0x00, 0x85, 0x6f, 0xf0, 0xe4, 0x9b, 0xc1, 0xbc, 0xb5, 0x7e, 0xf4,
+	0x46, 0x20, 0x42, 0x42, 0xe0, 0x7b, 0xe1, 0xf5, 0x10, 0x11, 0xb7, 0x19, 0x0b, 0xbd, 0x60, 0xd4,
+	0x1f, 0xbb, 0xd3, 0xe3, 0x09, 0xb7, 0x70, 0xde, 0xca, 0x23, 0xa4, 0x73, 0x3c, 0x01, 0xbe, 0x21,
+	0x0e, 0xcb, 0x93, 0x4b, 0xde, 0x5a, 0x1f, 0xce, 0x80, 0xaf, 0xf4, 0x47, 0x29, 0x76, 0x4d, 0xee,
+	0x38, 0x61, 0xd4, 0x8f, 0x42, 0xb9, 0x02, 0x67, 0xbc, 0xf2, 0x56, 0x0b, 0xd4, 0xf4, 0x19, 0x05,
+	0x6a, 0x66, 0xae, 0x40, 0x5d, 0xb5, 0x39, 0xcf, 0x15, 0xfa, 0x6b, 0x0b, 0x85, 0xbe, 0xdc, 0x09,
+	0xd6, 0xcf, 0xb3, 0x13, 0xfc, 0x61, 0x06, 0x0b, 0xa3, 0x78, 0x52, 0xfa, 0x36, 0x4b, 0x8f, 0x86,
+	0xfc, 0xcd, 0x4c, 0xd6, 0x4a, 0x8f, 0xce, 0x7c, 0x9f, 0x3f, 0xbf, 0x8b, 0xa6, 0xcf, 0xb1, 0x8b,
+	0x66, 0x96, 0xec, 0xa2, 0x6a, 0x09, 0x90, 0x9d, 0x2b, 0x01, 0xbe, 0x9a, 0x03, 0x86, 0x74, 0xbc,
+	0x0d, 0xd5, 0xf1, 0x62, 0x23, 0xe7, 0x12, 0x46, 0xfe, 0x0a, 0xf7, 0xe3, 0xff, 0x47, 0x27, 0x89,
+	0x3f, 0x4e, 0xe1, 0xfe, 0xd0, 0x3f, 0x3c, 0x0c, 0xbc, 0xc3, 0x7e, 0xe4, 0xfd, 0x7f, 0xe3, 0xa1,
+	0x3f, 0x66, 0x37, 0x96, 0x4f, 0x0c, 0x92, 0xd0, 0xfc, 0x42, 0xa5, 0xbe, 0x6c, 0xa1, 0xd2, 0xf3,
+	0x0b, 0x75, 0x9b, 0x31, 0x3e, 0x34, 0xa2, 0xa9, 0x4c, 0x01, 0x08, 0x47, 0x97, 0xfe, 0x3c, 0x83,
+	0xa9, 0x1f, 0x8d, 0x47, 0xb7, 0x2e, 0xdc, 0x59, 0xe0, 0xcf, 0xbc, 0x80, 0xd7, 0xa7, 0x6a, 0x12,
+	0x5c, 0xac, 0x1c, 0x16, 0xd9, 0xd4, 0x6c, 0xb8, 0x3f, 0xb7, 0xec, 0xd8, 0xcc, 0xfa, 0xf4, 0x3c,
+	0x52, 0x54, 0x3e, 0xfe, 0xae, 0x4b, 0x79, 0xd6, 0x2d, 0x56, 0x98, 0x7a, 0xa7, 0x91, 0x7a, 0xb1,
+	0xa3, 0xf0, 0xf0, 0xfe, 0x79, 0xc4, 0x2a, 0x6c, 0x50, 0x2b, 0xc1, 0x23, 0x5d, 0x07, 0xd9, 0x9d,
+	0x6f, 0x6b, 0x7d, 0xeb, 0x3c, 0xf2, 0x96, 0x74, 0xb7, 0xbe, 0xc7, 0x32, 0xfe, 0xe9, 0x64, 0x65,
+	0xe1, 0xb6, 0x44, 0x88, 0x7f, 0x3a, 0x69, 0x5c, 0xb0, 0x80, 0x0b, 0x2c, 0xb6, 0xa4, 0x62, 0x3b,
+	0x97, 0xc5, 0xce, 0xac, 0xdc, 0xc4, 0x5b, 0x8f, 0xd2, 0x21, 0xfb, 0xda, 0x39, 0x2c, 0xbe, 0x10,
+	0xb0, 0xa9, 0x9f, 0x3b, 0x60, 0xbf, 0x60, 0xa5, 0x2f, 0x5f, 0x03, 0xfd, 0x43, 0xb6, 0x1d, 0x3f,
+	0xba, 0xa3, 0x21, 0x8e, 0xb4, 0x65, 0x6d, 0xca, 0x95, 0x69, 0x0e, 0xc3, 0x92, 0x8d, 0x2d, 0xb6,
+	0xd5, 0xf6, 0xff, 0x45, 0xda, 0x60, 0x9f, 0xaf, 0x72, 0x7c, 0x58, 0x0f, 0xd8, 0x25, 0xfd, 0xd3,
+	0x09, 0xd7, 0x28, 0x83, 0x17, 0x67, 0xfc, 0xd3, 0x09, 0xe8, 0xf2, 0x0f, 0x53, 0x2b, 0x2d, 0x78,
+	0x66, 0xc1, 0xba, 0xe4, 0xcd, 0x50, 0xa2, 0x88, 0xca, 0x24, 0x8b, 0xa8, 0x6f, 0xb1, 0xc4, 0x6d,
+	0x10, 0x97, 0xaa, 0x25, 0xd0, 0x44, 0x53, 0x11, 0x35, 0xa8, 0x9c, 0x7e, 0x2f, 0xcd, 0xf4, 0x05,
+	0x9d, 0xc2, 0xb3, 0x72, 0xa2, 0xb8, 0x51, 0x96, 0x56, 0x6e, 0x94, 0x7d, 0xc4, 0xb6, 0x95, 0x56,
+	0x24, 0xe4, 0xaf, 0x0c, 0x4f, 0x26, 0x5b, 0x71, 0x2f, 0x12, 0x72, 0xb9, 0x4a, 0xc6, 0x1b, 0x9d,
+	0x94, 0x1e, 0x25, 0xd9, 0x73, 0x00, 0x2a, 0x17, 0x8c, 0xd6, 0x12, 0x17, 0x8c, 0xee, 0xb0, 0xc2,
+	0xa4, 0x7f, 0xea, 0x7a, 0xd3, 0x28, 0x18, 0x79, 0x21, 0x6d, 0x65, 0x6c, 0xd2, 0x3f, 0x35, 0x11,
+	0xa2, 0xef, 0xc2, 0x39, 0x81, 0xa7, 0x1f, 0xc0, 0x6f, 0xf0, 0xd5, 0x3c, 0x4f, 0x18, 0x41, 0xbe,
+	0xb2, 0x14, 0xd6, 0xd2, 0x4f, 0x52, 0xd8, 0x70, 0x47, 0x52, 0xdc, 0xfb, 0xcf, 0xde, 0xeb, 0xc1,
+	0x35, 0x4e, 0xd4, 0x4c, 0xba, 0x65, 0x15, 0x10, 0x86, 0xb9, 0xf4, 0x1e, 0xdb, 0x1c, 0xfb, 0xfe,
+	0xeb, 0xe3, 0x99, 0x92, 0x4d, 0xb3, 0x56, 0x01, 0x61, 0x48, 0xf2, 0x35, 0xb6, 0xc5, 0x6d, 0xe7,
+	0x0d, 0x89, 0x26, 0x4b, 0xfd, 0x5c, 0x04, 0x62, 0xd2, 0x7d, 0x80, 0x85, 0x96, 0xbc, 0x33, 0x16,
+	0x6f, 0x63, 0xab, 0x2e, 0x72, 0x95, 0xfe, 0x94, 0xea, 0x98, 0x98, 0x67, 0xf5, 0xa5, 0xaf, 0xdb,
+	0x8c, 0x05, 0xa7, 0xd4, 0x31, 0x09, 0xc5, 0x8e, 0x10, 0x9c, 0xf6, 0x10, 0x00, 0xe8, 0x28, 0x46,
+	0xe3, 0x1c, 0xf2, 0x91, 0x44, 0xdf, 0x60, 0xb9, 0xe0, 0xd4, 0x85, 0x0d, 0x24, 0x24, 0xe5, 0x37,
+	0x82, 0xd3, 0x0a, 0x3c, 0x72, 0xeb, 0x09, 0x14, 0x6e, 0x7b, 0x1b, 0x11, 0xa1, 0x70, 0x4c, 0x38,
+	0x06, 0xce, 0xbc, 0x21, 0x5f, 0x55, 0x3e, 0x66, 0x0d, 0x01, 0x34, 0xa6, 0x40, 0x6f, 0x88, 0x31,
+	0x05, 0x7a, 0x87, 0xe5, 0x83, 0x53, 0x3c, 0x7e, 0x84, 0x54, 0xaa, 0xe4, 0x82, 0x53, 0x93, 0x3f,
+	0x03, 0x32, 0x92, 0x48, 0xac, 0x54, 0x72, 0x91, 0x40, 0xde, 0x65, 0x9b, 0xc1, 0xa9, 0xfb, 0x2a,
+	0xe8, 0x4f, 0x3c, 0x20, 0xa1, 0x42, 0x85, 0x05, 0xa7, 0x75, 0x00, 0x99, 0xfc, 0x9a, 0x63, 0x21,
+	0x38, 0x75, 0xfd, 0x13, 0x2f, 0xe0, 0x04, 0x05, 0xa1, 0x5a, 0xf7, 0xc4, 0x0b, 0x00, 0x7f, 0x8b,
+	0x6b, 0x3e, 0x08, 0x06, 0x1c, 0xbd, 0x29, 0x06, 0xaf, 0x06, 0x03, 0xe4, 0x66, 0x03, 0x7f, 0x3c,
+	0x1e, 0x85, 0x54, 0xb7, 0xd0, 0x5e, 0x2f, 0x20, 0x0b, 0x15, 0xe2, 0xf6, 0x39, 0x2a, 0xc4, 0x8b,
+	0x8b, 0x15, 0x62, 0xe9, 0x11, 0xb6, 0xf8, 0xb1, 0x25, 0xb8, 0x50, 0xda, 0xac, 0x7a, 0x39, 0xb6,
+	0x8f, 0x71, 0x8f, 0x5d, 0x40, 0x74, 0x38, 0x2f, 0xf8, 0xbf, 0x2f, 0x1a, 0x4a, 0x3f, 0x49, 0x63,
+	0xe8, 0x28, 0xea, 0x9c, 0xa1, 0x06, 0x5f, 0x3e, 0xef, 0x55, 0x22, 0x6e, 0x72, 0x81, 0xf7, 0x4a,
+	0x06, 0x4d, 0x42, 0x9b, 0xcc, 0x97, 0x69, 0x93, 0x9d, 0x2f, 0x61, 0xbe, 0xaa, 0x5e, 0x56, 0x85,
+	0x6d, 0x92, 0xa5, 0xf8, 0x8c, 0x28, 0xb7, 0xdc, 0x59, 0xd1, 0x5c, 0x15, 0xe6, 0xb4, 0x0a, 0xf8,
+	0x6c, 0x03, 0x0f, 0x1c, 0xdb, 0xb6, 0x63, 0xcb, 0xf0, 0xc3, 0xdb, 0x97, 0xdd, 0x79, 0x3c, 0xb3,
+	0xf5, 0x9b, 0x5e, 0xd9, 0xfa, 0xcd, 0x9c, 0xb3, 0xf5, 0x7b, 0xa2, 0x2e, 0x15, 0xa4, 0xd5, 0xb7,
+	0xa0, 0x91, 0x3c, 0x4a, 0x16, 0x56, 0x6a, 0x04, 0x24, 0x78, 0xa1, 0x54, 0x7f, 0x88, 0x97, 0x86,
+	0x45, 0x85, 0x76, 0x6b, 0x05, 0x07, 0xa7, 0xc1, 0x2b, 0xc5, 0x61, 0xe9, 0xef, 0xa4, 0xd0, 0xf9,
+	0x10, 0x25, 0x37, 0x9d, 0x2b, 0x6c, 0x8d, 0xdf, 0x35, 0x14, 0x6f, 0x66, 0xf9, 0xc3, 0xc2, 0x4d,
+	0xda, 0xf4, 0xe2, 0x4d, 0x5a, 0xf0, 0x02, 0xd8, 0x19, 0xb8, 0x3c, 0xb1, 0xeb, 0xe6, 0x27, 0xfd,
+	0x53, 0x5e, 0x8d, 0x87, 0x7a, 0x31, 0xd9, 0xe4, 0xdf, 0x8a, 0x77, 0xf2, 0xef, 0xa8, 0xad, 0xa3,
+	0xc5, 0xf6, 0xc1, 0x19, 0xaf, 0xb5, 0x7e, 0x1d, 0x5f, 0x18, 0x2b, 0x6d, 0x19, 0xf4, 0xf5, 0x32,
+	0xbb, 0x44, 0x3e, 0xcb, 0x81, 0x6a, 0x18, 0x5d, 0x44, 0x44, 0xa5, 0x3f, 0xc5, 0x64, 0xae, 0x7f,
+	0x9d, 0x5d, 0xe4, 0xce, 0xab, 0x50, 0x62, 0x3c, 0x6d, 0x01, 0x58, 0xd2, 0x95, 0xfe, 0x80, 0x62,
+	0x0a, 0x07, 0x93, 0x31, 0xb5, 0x42, 0xb5, 0xb9, 0xba, 0x3d, 0x3d, 0x57, 0xb7, 0xc3, 0xa8, 0x71,
+	0x4b, 0x5c, 0x0d, 0xac, 0x2d, 0x04, 0x37, 0xa7, 0x48, 0x57, 0x62, 0x5c, 0x8d, 0x98, 0x0a, 0xa3,
+	0xab, 0x00, 0x40, 0x41, 0xf3, 0x55, 0xc5, 0xd7, 0x13, 0xc6, 0x62, 0x1b, 0x52, 0x74, 0xdd, 0x3b,
+	0xab, 0x07, 0x86, 0xfe, 0x94, 0x87, 0xdf, 0x18, 0x5d, 0xbf, 0x89, 0x6d, 0x75, 0x24, 0x39, 0xf3,
+	0xe6, 0xbe, 0x6a, 0xb9, 0xf4, 0x8a, 0x2e, 0x5c, 0xe6, 0xe7, 0xed, 0xc2, 0xfd, 0x6b, 0x72, 0x69,
+	0x24, 0x90, 0x2e, 0x4d, 0xf7, 0xd6, 0xf1, 0x9d, 0x75, 0x4a, 0xde, 0x5b, 0x6f, 0xf3, 0x97, 0xa6,
+	0xb7, 0x69, 0xd2, 0xe8, 0xf4, 0xb4, 0x4e, 0x00, 0x71, 0x96, 0x3a, 0x7e, 0x66, 0x89, 0xe3, 0x93,
+	0x7c, 0xd1, 0x3a, 0x14, 0xf2, 0xc1, 0x75, 0x24, 0x72, 0xe0, 0x8f, 0xfd, 0x80, 0x56, 0x06, 0x90,
+	0x55, 0x78, 0x2e, 0xfd, 0x58, 0x75, 0x29, 0x8c, 0xfd, 0xcf, 0x65, 0xdd, 0x95, 0x5a, 0x71, 0x83,
+	0x46, 0xb5, 0xae, 0x2c, 0xcb, 0xbe, 0x34, 0x03, 0x28, 0x6e, 0x2b, 0x32, 0xc0, 0x09, 0xbb, 0xc7,
+	0xbb, 0x86, 0x89, 0x7e, 0xa1, 0x0c, 0xbf, 0xa3, 0xe5, 0x37, 0xa4, 0x52, 0x5f, 0x52, 0x07, 0xcf,
+	0x35, 0x13, 0x45, 0xa3, 0x30, 0xa3, 0x34, 0x0a, 0xc7, 0xb8, 0x57, 0x26, 0xc6, 0xfd, 0xe5, 0x8d,
+	0x66, 0x62, 0x5b, 0xf2, 0x47, 0xc7, 0xde, 0x31, 0xd5, 0xf9, 0x34, 0x16, 0x6f, 0xea, 0x60, 0xdd,
+	0x29, 0xbc, 0x42, 0x9e, 0x9b, 0x35, 0x96, 0x89, 0x6f, 0xe4, 0xc0, 0xcf, 0x52, 0x80, 0x4a, 0x2b,
+	0x62, 0x26, 0xa3, 0xa9, 0xcb, 0xdf, 0x24, 0x54, 0x59, 0x41, 0x91, 0x4b, 0xeb, 0xb6, 0xe8, 0xb6,
+	0x0b, 0x1a, 0x60, 0xb5, 0xdb, 0x90, 0xfd, 0xdf, 0xf9, 0x57, 0x14, 0xcb, 0xc6, 0xec, 0x9f, 0xfe,
+	0x92, 0xc7, 0xfc, 0x47, 0xd4, 0xa8, 0x51, 0x38, 0x13, 0xd6, 0xff, 0x4a, 0x06, 0x3e, 0xcf, 0xe1,
+	0x6a, 0xd9, 0x5a, 0xfe, 0xed, 0x14, 0x26, 0x18, 0x4a, 0x9d, 0x7c, 0x10, 0xf0, 0x07, 0x1c, 0x2d,
+	0x4e, 0xc2, 0xfc, 0x19, 0x8f, 0x49, 0x4a, 0xdb, 0x08, 0x2f, 0x5c, 0x55, 0x12, 0xe7, 0x93, 0x55,
+	0x9d, 0xfe, 0x15, 0xfa, 0xd3, 0xd1, 0xe4, 0x21, 0x5e, 0xa7, 0x40, 0xa2, 0x43, 0x5e, 0x6b, 0x40,
+	0x14, 0xca, 0x2d, 0x6b, 0xc9, 0x45, 0xaf, 0xd2, 0x21, 0x9e, 0x57, 0x97, 0xf0, 0xcc, 0xc6, 0x6f,
+	0x97, 0x5e, 0x0d, 0xfb, 0x9c, 0xad, 0x73, 0x6a, 0xf1, 0x5d, 0xc5, 0xed, 0x55, 0x6f, 0x55, 0x39,
+	0x95, 0x45, 0xc4, 0x25, 0x73, 0xe1, 0x16, 0x15, 0xda, 0x69, 0xc5, 0x6b, 0x00, 0x69, 0xbb, 0x4c,
+	0xc2, 0x76, 0xa5, 0xb6, 0xea, 0x7c, 0xe7, 0x3b, 0xe5, 0x24, 0xc4, 0xa5, 0x93, 0xe2, 0xfe, 0x8c,
+	0x4e, 0x73, 0x8a, 0xbc, 0x5f, 0x44, 0x4e, 0xe2, 0x0c, 0x93, 0x59, 0x38, 0xc3, 0x28, 0x07, 0xa3,
+	0xec, 0xfc, 0xc1, 0x28, 0x71, 0x0e, 0x59, 0x9b, 0x3b, 0x87, 0xcc, 0xef, 0xa1, 0xeb, 0xe7, 0xd8,
+	0x43, 0x37, 0x96, 0x9c, 0x03, 0x26, 0xe8, 0xa0, 0x81, 0x3f, 0xf6, 0xa4, 0xb9, 0x1e, 0xb1, 0x2c,
+	0x3c, 0xaf, 0x7c, 0x67, 0x39, 0xf0, 0xa7, 0x51, 0xe0, 0x8f, 0xc7, 0x5e, 0xc0, 0xf9, 0x2c, 0x4e,
+	0x0d, 0xc3, 0x1d, 0x7a, 0x53, 0x8f, 0x06, 0x24, 0x43, 0x64, 0xad, 0xcd, 0x18, 0xd8, 0x1c, 0x96,
+	0x7e, 0x87, 0x02, 0xa2, 0x1f, 0xbe, 0x9d, 0x0e, 0xc4, 0x8e, 0xfb, 0x21, 0xdb, 0x8e, 0x6b, 0x0b,
+	0xde, 0xe3, 0xa4, 0xa6, 0x8c, 0x28, 0x2d, 0x78, 0x97, 0xf3, 0x63, 0xa6, 0x29, 0x1f, 0x3d, 0x89,
+	0x6b, 0x39, 0x40, 0xb7, 0x0d, 0x70, 0x9b, 0x83, 0x39, 0x65, 0x99, 0x5d, 0x4a, 0xbc, 0xc5, 0xe6,
+	0xa4, 0x58, 0xdf, 0x5d, 0x04, 0x84, 0x85, 0x70, 0x7e, 0xd5, 0xe9, 0x35, 0xdb, 0xe6, 0xfb, 0x6a,
+	0xdb, 0x1f, 0xee, 0xcd, 0x86, 0x90, 0xa9, 0xb0, 0x5d, 0x8f, 0x6f, 0x45, 0xd2, 0x23, 0xfe, 0x91,
+	0x8f, 0x7c, 0x67, 0x47, 0xbb, 0xd5, 0xcd, 0xd5, 0x6f, 0xf5, 0x2c, 0x2c, 0x13, 0xda, 0xfe, 0x70,
+	0xc9, 0xe7, 0x95, 0x3d, 0x76, 0x91, 0x0f, 0xc6, 0x8b, 0x0f, 0x8b, 0xc7, 0xd1, 0x0f, 0x58, 0x41,
+	0xd9, 0xe9, 0x56, 0xf6, 0xbd, 0xd4, 0xdd, 0x90, 0x4d, 0xa4, 0x8c, 0xd2, 0x88, 0x5d, 0xac, 0x8f,
+	0xfd, 0x37, 0xbc, 0x73, 0xb5, 0x42, 0xff, 0x47, 0x2c, 0x27, 0x6e, 0x1b, 0x91, 0xfa, 0x37, 0x56,
+	0x5e, 0x47, 0xb2, 0x36, 0xe0, 0xd7, 0x72, 0xe5, 0x7f, 0xc4, 0xae, 0xc0, 0x50, 0xbc, 0x3a, 0x3e,
+	0x6b, 0xbc, 0x6f, 0xb3, 0xbc, 0xbc, 0xb7, 0xb2, 0xd2, 0x5e, 0x92, 0xc2, 0xc2, 0xa3, 0xc9, 0xf2,
+	0x21, 0xbf, 0xcb, 0xd6, 0x60, 0xc8, 0x50, 0x7f, 0xc0, 0xd6, 0x46, 0x91, 0x37, 0x11, 0xf6, 0xd9,
+	0x59, 0x3e, 0x01, 0x2a, 0x16, 0x38, 0x65, 0xe9, 0xfb, 0x6c, 0x9d, 0xdb, 0x3a, 0x84, 0x52, 0x43,
+	0x65, 0x5e, 0x65, 0x5c, 0x5e, 0xce, 0x08, 0xee, 0x27, 0x8c, 0xc9, 0xc9, 0x9e, 0x43, 0x82, 0x72,
+	0x18, 0x12, 0x12, 0x46, 0xac, 0x00, 0x12, 0xaa, 0x47, 0xfd, 0xe9, 0xa1, 0x17, 0xea, 0xdf, 0x64,
+	0xeb, 0x91, 0xef, 0xf6, 0x87, 0xe2, 0x4e, 0xa8, 0x9e, 0x90, 0xc1, 0x67, 0x69, 0xad, 0x45, 0xbe,
+	0x31, 0x1c, 0xea, 0xf7, 0x59, 0x3e, 0xf2, 0xc9, 0x79, 0xc9, 0x80, 0xcb, 0xa8, 0x73, 0x91, 0x8f,
+	0x8e, 0x0c, 0x65, 0xa4, 0x26, 0xb5, 0x15, 0x03, 0x7e, 0x32, 0x37, 0xe0, 0xf5, 0x05, 0x11, 0x38,
+	0x39, 0x31, 0xea, 0xa3, 0xc5, 0x51, 0x57, 0xb2, 0xc8, 0xa1, 0x89, 0xeb, 0x98, 0x7b, 0x02, 0xf5,
+	0xc5, 0xcf, 0xe2, 0x42, 0x97, 0x29, 0xd9, 0x2c, 0xd7, 0xa3, 0xd0, 0x5e, 0xe6, 0x3e, 0x32, 0x19,
+	0xac, 0x74, 0x1f, 0x49, 0x61, 0xe5, 0x44, 0x8e, 0x28, 0x3d, 0x67, 0x79, 0x14, 0xda, 0x3d, 0x8e,
+	0x16, 0xa4, 0x7e, 0x97, 0xb1, 0xf8, 0xaa, 0x12, 0x89, 0xdd, 0x59, 0x25, 0xd6, 0x3f, 0x8e, 0x2c,
+	0x52, 0xa2, 0x7b, 0x1c, 0x95, 0xfe, 0x71, 0x8a, 0x15, 0xd0, 0xaa, 0xe6, 0x89, 0x37, 0x5d, 0x94,
+	0xfd, 0x97, 0x59, 0x41, 0x49, 0x4c, 0x2b, 0x0b, 0x5a, 0x85, 0xa6, 0x71, 0xc1, 0x62, 0x71, 0xce,
+	0x02, 0xf7, 0xe2, 0x59, 0x9e, 0x0c, 0xb8, 0x38, 0x5d, 0xf9, 0x9a, 0xbe, 0x71, 0xc1, 0x42, 0xd2,
+	0xca, 0x06, 0x5b, 0xf3, 0x40, 0x9b, 0xf2, 0x7f, 0x4d, 0xb1, 0x82, 0x14, 0x3f, 0xf5, 0x75, 0x8d,
+	0x6d, 0x76, 0xeb, 0xbd, 0x9e, 0xdb, 0xec, 0xec, 0x1b, 0xad, 0x66, 0x4d, 0xbb, 0xa0, 0x6b, 0x2c,
+	0xc7, 0x21, 0x6d, 0xe3, 0x85, 0xf6, 0xee, 0x67, 0xef, 0xdf, 0x6f, 0xe8, 0x57, 0x24, 0x8d, 0xdb,
+	0xeb, 0x5a, 0x8e, 0xf6, 0x3f, 0xde, 0x03, 0x54, 0x67, 0x8c, 0x43, 0x1d, 0xa3, 0xd2, 0x32, 0xb5,
+	0xff, 0xc9, 0x61, 0x97, 0x59, 0x81, 0xc3, 0x3a, 0x5d, 0xab, 0x6d, 0xb4, 0xb4, 0xbf, 0x48, 0x10,
+	0xd6, 0x5b, 0xdd, 0x6e, 0x4d, 0xfb, 0x5f, 0x1c, 0x26, 0x06, 0x31, 0x5a, 0x2d, 0xed, 0xa7, 0x1c,
+	0x72, 0x9d, 0x5d, 0xe4, 0x90, 0x6a, 0xb7, 0xe3, 0x58, 0xdd, 0x56, 0xcb, 0xb4, 0xb4, 0xff, 0x9d,
+	0x60, 0x6f, 0x75, 0xab, 0x46, 0x4b, 0xfb, 0x59, 0x92, 0xbd, 0xf3, 0x52, 0x7b, 0x0f, 0x90, 0xf2,
+	0x7f, 0x58, 0xc3, 0xf7, 0xd5, 0x7c, 0xdb, 0xdf, 0xe6, 0x2c, 0x8e, 0xdb, 0x30, 0x5b, 0xad, 0xae,
+	0x76, 0x41, 0x3e, 0x9b, 0x96, 0xd5, 0xb5, 0xb4, 0x94, 0x7e, 0x95, 0x5d, 0xc2, 0xe7, 0x6a, 0xa3,
+	0xeb, 0x5a, 0xe6, 0xb3, 0x3d, 0xd3, 0x76, 0xb4, 0xb4, 0x7e, 0x99, 0xab, 0x20, 0xc1, 0xbd, 0xd6,
+	0x4b, 0x2d, 0x13, 0xd3, 0xbe, 0xe8, 0x99, 0x56, 0xb3, 0x6d, 0x76, 0x1c, 0xd3, 0xd2, 0xb2, 0xfa,
+	0x0d, 0x76, 0x95, 0x83, 0xeb, 0xa6, 0xe1, 0xec, 0x59, 0xa6, 0x2d, 0xc5, 0xac, 0xe9, 0xd7, 0xd9,
+	0xe5, 0x79, 0x14, 0x88, 0x5a, 0xd7, 0x77, 0xd8, 0x75, 0x8e, 0xd8, 0x35, 0x1d, 0x98, 0x66, 0xbd,
+	0xb9, 0x2b, 0xb9, 0x36, 0xa4, 0xc0, 0x04, 0x12, 0xf8, 0x72, 0x52, 0x2f, 0x5b, 0xa2, 0xb4, 0xbc,
+	0xae, 0xb3, 0x6d, 0x0e, 0xec, 0x19, 0xd5, 0xa7, 0xa6, 0xe3, 0x36, 0x3b, 0x1a, 0x93, 0xba, 0xd6,
+	0x5b, 0xdd, 0xe7, 0xae, 0x65, 0xb6, 0xbb, 0xfb, 0x66, 0x4d, 0x2b, 0xe8, 0x57, 0x98, 0x86, 0xa4,
+	0x5d, 0xcb, 0x71, 0x6d, 0xc7, 0x70, 0xf6, 0x6c, 0x6d, 0x53, 0x4a, 0x25, 0x01, 0xdd, 0x3d, 0x47,
+	0xdb, 0xd2, 0x2f, 0xb1, 0xad, 0x58, 0x42, 0xbb, 0x5b, 0xd3, 0xb6, 0xe5, 0x40, 0xbb, 0x56, 0x77,
+	0xaf, 0xc7, 0x61, 0x17, 0x25, 0x19, 0x97, 0x08, 0x20, 0x4d, 0x92, 0x71, 0x77, 0xe0, 0xb0, 0x4b,
+	0xfa, 0x4d, 0x76, 0x8d, 0xc3, 0xda, 0x7b, 0x2d, 0xa7, 0xd9, 0x33, 0x2c, 0x47, 0xce, 0x57, 0xd7,
+	0x8b, 0xec, 0xca, 0x02, 0x0e, 0xa6, 0x7b, 0x59, 0x62, 0x2a, 0x86, 0x65, 0x35, 0x4d, 0x4b, 0xf2,
+	0x5c, 0xd1, 0xaf, 0x31, 0x7d, 0x0e, 0x03, 0x1c, 0x57, 0xf5, 0x7b, 0xec, 0x36, 0x87, 0x3f, 0xdb,
+	0x33, 0xf7, 0xcc, 0x65, 0xe6, 0xbd, 0xa6, 0xdf, 0x61, 0x3b, 0xab, 0x48, 0x40, 0xc6, 0x75, 0x69,
+	0x3b, 0xab, 0xdb, 0x32, 0x25, 0x5f, 0x51, 0x5a, 0x89, 0xc0, 0x40, 0x7b, 0x43, 0xce, 0x0b, 0xc4,
+	0x18, 0xf6, 0xcb, 0x4e, 0x55, 0x32, 0xdc, 0x94, 0xda, 0xab, 0x38, 0xe0, 0xda, 0x91, 0x16, 0xb2,
+	0x05, 0x46, 0xbb, 0x25, 0x61, 0x6d, 0xd3, 0x31, 0x2d, 0x6e, 0xb5, 0xdb, 0xe5, 0x2a, 0x5e, 0xf8,
+	0x98, 0xfb, 0x13, 0x09, 0x44, 0xda, 0xe0, 0x6b, 0x2d, 0x62, 0x15, 0x07, 0x03, 0xd8, 0xbe, 0x69,
+	0xd9, 0xcd, 0x6e, 0xa7, 0xd2, 0x74, 0xda, 0x46, 0x4f, 0x4b, 0x95, 0x3d, 0x2c, 0x9c, 0xa8, 0x08,
+	0xc7, 0xa6, 0x04, 0xfa, 0x41, 0xd5, 0xad, 0x5b, 0xc6, 0xae, 0x08, 0xd1, 0x0b, 0x24, 0x97, 0xa0,
+	0x35, 0xab, 0xdb, 0xd3, 0x52, 0x34, 0x6b, 0x82, 0x59, 0xa6, 0x61, 0xb7, 0xb5, 0x74, 0x92, 0xb0,
+	0x6d, 0xd8, 0x4f, 0xb5, 0x4c, 0xf9, 0x09, 0x0e, 0x83, 0x2f, 0x2d, 0xa8, 0x3e, 0x23, 0xe7, 0xa8,
+	0x2a, 0x7a, 0x92, 0x73, 0x57, 0xdd, 0x9a, 0xd9, 0xb3, 0xcc, 0xaa, 0xe1, 0x98, 0x35, 0x21, 0xe1,
+	0xd7, 0xf0, 0xfb, 0x6c, 0xbc, 0xc7, 0x4e, 0xac, 0xea, 0x14, 0xb7, 0x59, 0x1e, 0x41, 0x90, 0x8f,
+	0x7e, 0x96, 0x8a, 0x9f, 0x21, 0x75, 0xbc, 0x4f, 0x95, 0xff, 0x3d, 0x95, 0x88, 0x89, 0x96, 0x05,
+	0x66, 0x35, 0x55, 0x03, 0x39, 0x23, 0x70, 0x6c, 0x88, 0x01, 0x5b, 0x4b, 0x49, 0x83, 0xa0, 0xcf,
+	0x22, 0x34, 0x2d, 0x49, 0x65, 0xb8, 0xd8, 0x5a, 0x56, 0x92, 0x62, 0x14, 0x20, 0x34, 0x47, 0xfa,
+	0x56, 0xdd, 0x66, 0x8f, 0xac, 0x74, 0x57, 0x12, 0xa2, 0xa3, 0x21, 0xe1, 0x13, 0xfd, 0x1a, 0xf7,
+	0x2e, 0x92, 0x59, 0x69, 0x75, 0xab, 0x4f, 0xcd, 0x9a, 0xf6, 0x2e, 0x5d, 0x3e, 0x51, 0x3e, 0xcf,
+	0x4f, 0x98, 0x6f, 0x89, 0xf2, 0x82, 0xbd, 0xd6, 0x7d, 0xde, 0xd1, 0x52, 0x31, 0x5d, 0x07, 0x92,
+	0x55, 0x75, 0x5f, 0xcb, 0x8a, 0x64, 0xce, 0x41, 0xf5, 0xe7, 0x35, 0xed, 0x2e, 0x45, 0x0c, 0x42,
+	0xe2, 0x4c, 0xf1, 0xa4, 0xfc, 0x57, 0xe6, 0x5e, 0xd7, 0x08, 0xd3, 0xf7, 0xec, 0xc5, 0x61, 0x6d,
+	0xb7, 0xd5, 0xec, 0x3c, 0x9d, 0x1b, 0xd6, 0x96, 0xb3, 0x48, 0x53, 0x7a, 0xe5, 0x74, 0xfb, 0xa6,
+	0x96, 0x2d, 0xff, 0x69, 0x1a, 0x3f, 0x8a, 0xe1, 0xd2, 0x65, 0x9b, 0x8a, 0x18, 0xeb, 0xca, 0x00,
+	0x12, 0xf4, 0xe0, 0xd3, 0x76, 0xc5, 0x6d, 0xd4, 0x62, 0xf1, 0x04, 0xaa, 0xd7, 0xa4, 0xdf, 0x71,
+	0x10, 0x91, 0x65, 0xe7, 0x61, 0xf5, 0x9a, 0x96, 0x13, 0xb3, 0xaf, 0xbb, 0x0f, 0x76, 0x39, 0x95,
+	0x96, 0x84, 0xd4, 0xc1, 0x1e, 0x8a, 0x78, 0x04, 0x3d, 0xd1, 0x75, 0x01, 0x7a, 0x44, 0xa0, 0x77,
+	0xe0, 0xff, 0xb1, 0x78, 0x02, 0xa6, 0xf5, 0x4b, 0x52, 0x9a, 0x83, 0x20, 0x30, 0x78, 0x01, 0x41,
+	0x5d, 0xa7, 0x61, 0x5a, 0xda, 0xbb, 0x5c, 0x4c, 0x54, 0xed, 0xf6, 0x7a, 0x00, 0xd2, 0x62, 0xa2,
+	0x7a, 0xb3, 0x02, 0x90, 0xbb, 0xf1, 0x90, 0xc6, 0x9e, 0xd3, 0xed, 0x98, 0xbb, 0xda, 0xbb, 0x27,
+	0xfa, 0x25, 0x41, 0xd5, 0x33, 0xf6, 0x6c, 0x53, 0x7b, 0xf7, 0x2e, 0xa5, 0x5f, 0xe3, 0xae, 0x24,
+	0x40, 0x90, 0x33, 0xda, 0xda, 0xbb, 0x77, 0xe9, 0x72, 0x4d, 0x71, 0x1a, 0xba, 0x6a, 0xbb, 0xc5,
+	0xa3, 0xa2, 0x67, 0xb9, 0x46, 0x0d, 0xf7, 0xf0, 0x4d, 0x7c, 0xac, 0x99, 0x2d, 0xd3, 0x31, 0xb5,
+	0x54, 0x0c, 0x69, 0x77, 0x6b, 0xcd, 0xfa, 0x4b, 0x2d, 0x5d, 0xfe, 0x0c, 0x5d, 0x20, 0xfe, 0x93,
+	0x07, 0x64, 0xd4, 0x36, 0x77, 0xfa, 0x4e, 0xcd, 0xb0, 0x40, 0x12, 0x0a, 0x6e, 0x3b, 0x6e, 0xf7,
+	0x45, 0x5b, 0x4b, 0x95, 0x5f, 0xc7, 0x7f, 0xd3, 0x80, 0xff, 0x91, 0x02, 0x92, 0xfb, 0xa2, 0x5d,
+	0x75, 0x3b, 0x2f, 0xda, 0xee, 0xa7, 0x72, 0x6c, 0x01, 0x79, 0xa0, 0xa5, 0xf4, 0x1d, 0x1e, 0xfd,
+	0x00, 0xe9, 0xf6, 0xcc, 0x0e, 0x8f, 0xc0, 0x8a, 0x61, 0x37, 0xab, 0x30, 0x19, 0xfd, 0x06, 0xdf,
+	0x2d, 0x01, 0x99, 0xd8, 0x61, 0xdf, 0xbf, 0xcf, 0x94, 0xff, 0x41, 0x8e, 0x5d, 0x5e, 0xf2, 0x67,
+	0x02, 0xc8, 0xa9, 0x5f, 0x80, 0x52, 0xf5, 0x8a, 0xac, 0x4a, 0x2e, 0x50, 0x5a, 0x56, 0xe1, 0x8d,
+	0x97, 0x88, 0x4b, 0xd1, 0xa6, 0x2c, 0x70, 0x6d, 0xd3, 0x31, 0x6a, 0x86, 0x63, 0x68, 0xe9, 0x39,
+	0x61, 0xa6, 0xd3, 0x70, 0x6b, 0xb6, 0xa3, 0x65, 0x96, 0xc0, 0x6d, 0xab, 0xaa, 0x65, 0xe7, 0x04,
+	0x01, 0xdc, 0x79, 0xd9, 0x33, 0xe5, 0xb6, 0x2f, 0x10, 0xfb, 0x2d, 0xa3, 0xe3, 0xee, 0x37, 0x6b,
+	0xda, 0xfa, 0x32, 0x44, 0xaf, 0xda, 0xd3, 0x36, 0xe6, 0xe7, 0xd1, 0x73, 0x6b, 0x76, 0xb5, 0xa7,
+	0xe5, 0x68, 0x2b, 0x52, 0xe0, 0x66, 0xb5, 0xa3, 0xe5, 0xe7, 0xe4, 0x34, 0x7b, 0x6e, 0xcf, 0xea,
+	0x3a, 0x5d, 0x8d, 0x2d, 0x20, 0xf6, 0x1f, 0x71, 0x5d, 0x0b, 0xcb, 0x10, 0x30, 0xb9, 0xcd, 0xb9,
+	0x91, 0x9d, 0x6a, 0x8f, 0x33, 0x6c, 0x2d, 0x81, 0x03, 0xfd, 0xf6, 0x1c, 0x7c, 0xaf, 0x86, 0xf4,
+	0x17, 0x97, 0xc0, 0x81, 0x5e, 0x9b, 0x1b, 0xd8, 0xae, 0x3a, 0xc8, 0x70, 0x69, 0x19, 0xa2, 0xc6,
+	0xcb, 0x81, 0xb9, 0xb5, 0xab, 0xb6, 0x41, 0x59, 0x6e, 0xd9, 0xcb, 0xcb, 0x71, 0xd5, 0x6e, 0xcd,
+	0xd4, 0xae, 0xcc, 0xd9, 0xca, 0xb0, 0x7a, 0x6e, 0xb7, 0xa7, 0x5d, 0x9d, 0x53, 0x0c, 0xc0, 0x76,
+	0xcf, 0xd0, 0xae, 0x2d, 0x81, 0x3b, 0x3d, 0x43, 0xbb, 0xbe, 0x8c, 0xbe, 0x61, 0x68, 0xc5, 0x65,
+	0xf4, 0x0d, 0x43, 0xbb, 0xb1, 0x68, 0xd9, 0xc7, 0x7c, 0x82, 0x37, 0x97, 0x21, 0x60, 0x82, 0x3b,
+	0xf3, 0x93, 0x00, 0x44, 0xbd, 0x65, 0x54, 0xcc, 0x96, 0x76, 0x6b, 0xd9, 0x04, 0x1f, 0xe3, 0xe4,
+	0x6f, 0x2f, 0xc7, 0xf1, 0xc9, 0x7f, 0xa0, 0xdf, 0x66, 0x37, 0xe6, 0x65, 0x76, 0x6a, 0xae, 0x63,
+	0x58, 0xbb, 0xa6, 0xa3, 0xdd, 0x59, 0x36, 0x64, 0xa7, 0xe6, 0xda, 0xad, 0x96, 0x76, 0x77, 0x05,
+	0xce, 0x69, 0xb5, 0xb4, 0x7b, 0xb4, 0x5b, 0xcb, 0x58, 0xe9, 0xb5, 0x6c, 0x17, 0x35, 0x2d, 0xcd,
+	0xd9, 0x83, 0xa3, 0x9c, 0xaa, 0xf6, 0xb5, 0xf9, 0xf0, 0x02, 0x78, 0xa5, 0x6b, 0x6b, 0x1f, 0xce,
+	0x21, 0x7a, 0x95, 0x8a, 0xdb, 0xb4, 0x9b, 0x35, 0xed, 0x23, 0x2a, 0x5d, 0xa4, 0xab, 0xed, 0x75,
+	0x3a, 0x66, 0xcb, 0x6d, 0xd6, 0xb4, 0xaf, 0x2f, 0x53, 0xcd, 0x7c, 0xe1, 0x34, 0x6a, 0x96, 0xf6,
+	0x8d, 0xf2, 0x67, 0x78, 0x7a, 0xe1, 0x1f, 0xb5, 0x8f, 0x86, 0xfa, 0x45, 0x9e, 0x34, 0xf7, 0x9b,
+	0x35, 0xb7, 0xd3, 0xed, 0x98, 0x7c, 0xcb, 0xda, 0x26, 0x40, 0xcf, 0x32, 0x6d, 0xb3, 0xe3, 0x68,
+	0xef, 0xee, 0x96, 0xff, 0x63, 0x0a, 0x5b, 0x86, 0xa3, 0xd9, 0xc9, 0x63, 0xfa, 0x08, 0x5b, 0x5c,
+	0x7c, 0x05, 0xea, 0xa6, 0xd9, 0x58, 0xd8, 0x93, 0x00, 0x06, 0x22, 0x5f, 0x40, 0xee, 0xc0, 0xfd,
+	0x0d, 0x40, 0xa6, 0xdd, 0xd3, 0xd2, 0x34, 0x2a, 0x3c, 0x1b, 0x7b, 0x4e, 0x43, 0xcb, 0x2a, 0x80,
+	0x1a, 0x14, 0x81, 0x39, 0x05, 0x00, 0xc5, 0x92, 0xa6, 0x29, 0x52, 0xad, 0xee, 0x1e, 0xe4, 0xb7,
+	0xbb, 0x8a, 0xd4, 0x46, 0xb7, 0xa7, 0x3d, 0xa1, 0x9d, 0x03, 0x9e, 0xf7, 0x3a, 0x96, 0xd9, 0x83,
+	0x6d, 0x48, 0x05, 0xd9, 0xe6, 0x33, 0x28, 0x18, 0x7e, 0x9a, 0x4e, 0x7c, 0x05, 0x4b, 0x7f, 0x98,
+	0x0b, 0xc8, 0x0c, 0x5e, 0xc3, 0xf7, 0xf6, 0x20, 0x13, 0xe2, 0x32, 0x19, 0x50, 0xe4, 0xf6, 0x5e,
+	0xba, 0x8e, 0xd3, 0xe2, 0xe5, 0x7d, 0x81, 0xa2, 0x45, 0x85, 0x37, 0x3b, 0x32, 0x1d, 0x18, 0x58,
+	0x9a, 0xe2, 0xa2, 0x3a, 0x2d, 0x19, 0xde, 0x86, 0xe3, 0xd6, 0xcc, 0x6a, 0x0c, 0xd7, 0xa8, 0x30,
+	0x30, 0x1c, 0xb7, 0xb7, 0x67, 0x37, 0x78, 0x46, 0xd3, 0x2e, 0x91, 0x31, 0x01, 0xd8, 0xed, 0x21,
+	0x4c, 0x9f, 0x23, 0x04, 0x09, 0xda, 0xe5, 0x24, 0x21, 0x87, 0x5d, 0x89, 0x09, 0x41, 0x03, 0x5e,
+	0x3a, 0x69, 0x57, 0xc9, 0x8a, 0x06, 0x1d, 0x3d, 0xb4, 0x6b, 0x54, 0x5b, 0x11, 0x55, 0xe7, 0x39,
+	0xd7, 0xe6, 0x7a, 0x0c, 0x05, 0x2d, 0x09, 0x5a, 0x4c, 0x4a, 0xac, 0x37, 0xcd, 0x56, 0x4d, 0xbb,
+	0xa1, 0x0c, 0x0d, 0xfa, 0xf4, 0x2a, 0x15, 0xed, 0x26, 0x2d, 0x0d, 0xa9, 0x03, 0xa0, 0x1d, 0xbd,
+	0x28, 0xe6, 0xbd, 0xb0, 0x25, 0xed, 0xe3, 0xd5, 0x18, 0xa5, 0xa5, 0x49, 0x5f, 0x37, 0x8b, 0xea,
+	0xb8, 0xdd, 0x4a, 0x1c, 0xa5, 0x19, 0xc1, 0xa0, 0x78, 0xfd, 0x6f, 0xef, 0x33, 0xb4, 0xa5, 0x03,
+	0xa4, 0xd3, 0x75, 0x2b, 0x7b, 0xf5, 0x3a, 0xc9, 0xfd, 0x2f, 0xc2, 0x45, 0x95, 0x2f, 0x18, 0xf9,
+	0xda, 0x92, 0xe3, 0xa8, 0x15, 0x31, 0xce, 0xb7, 0xe9, 0xb8, 0xbb, 0x5d, 0xa7, 0x4b, 0xc7, 0xef,
+	0x14, 0xc5, 0x53, 0xd3, 0x71, 0x9f, 0x5b, 0x4d, 0xc7, 0x54, 0x77, 0x38, 0x0c, 0x41, 0x89, 0x31,
+	0xaa, 0x4e, 0xb3, 0xdb, 0xb1, 0xb5, 0x4c, 0x8c, 0x30, 0x7a, 0xbd, 0xd6, 0x4b, 0x89, 0xc8, 0xc6,
+	0x88, 0x6a, 0xcb, 0x34, 0x2c, 0x89, 0x58, 0x13, 0x7e, 0x4d, 0xe7, 0x15, 0x6d, 0x9d, 0x2c, 0xd5,
+	0x5c, 0x62, 0xa9, 0xbf, 0x8e, 0x13, 0x9a, 0xff, 0x72, 0x91, 0x0a, 0x8a, 0x7a, 0x35, 0x51, 0xa9,
+	0xd4, 0xab, 0xa2, 0x2e, 0x11, 0x3b, 0xb5, 0x84, 0xb8, 0xb6, 0x63, 0x35, 0xab, 0x70, 0x3c, 0x97,
+	0xa4, 0x54, 0xd4, 0x64, 0x62, 0x52, 0x84, 0x08, 0xd2, 0x6c, 0xf9, 0x9f, 0xd1, 0x9b, 0x52, 0x39,
+	0x3a, 0xc6, 0x3b, 0x1a, 0xb3, 0xae, 0x96, 0xa0, 0x24, 0xa2, 0xee, 0xda, 0x66, 0xa7, 0x26, 0x0f,
+	0xce, 0xb1, 0x1a, 0x75, 0xb7, 0xda, 0x30, 0xab, 0x4f, 0xdd, 0xee, 0xbe, 0x69, 0xb5, 0x8c, 0x9e,
+	0x2c, 0x18, 0xea, 0x75, 0x17, 0x12, 0x0c, 0x44, 0xd2, 0x5e, 0xc7, 0x89, 0x8d, 0x56, 0xaf, 0xf3,
+	0x52, 0xfb, 0xa9, 0x44, 0xe4, 0x12, 0x88, 0xca, 0x4b, 0x89, 0xd0, 0xca, 0x36, 0x1e, 0x7d, 0xf0,
+	0x1b, 0x73, 0x9c, 0xdd, 0xee, 0x42, 0x23, 0x66, 0x57, 0x69, 0xc4, 0x08, 0x48, 0xdc, 0x35, 0x91,
+	0x10, 0xd9, 0x08, 0xf9, 0x02, 0xdf, 0x07, 0x2e, 0x7c, 0x0b, 0x48, 0x86, 0xdf, 0x4d, 0x1a, 0x7e,
+	0x57, 0x31, 0xbc, 0x84, 0x90, 0x7d, 0xd3, 0x65, 0x5b, 0xbd, 0x4c, 0xc2, 0xdd, 0x91, 0x84, 0xe0,
+	0xe9, 0x4b, 0x0a, 0x81, 0x20, 0x6b, 0x99, 0x55, 0xc8, 0x95, 0x18, 0x06, 0xbb, 0xe0, 0xaf, 0xb5,
+	0xa6, 0x65, 0xf2, 0x85, 0xdb, 0x44, 0x25, 0x1d, 0xb7, 0x5e, 0xd7, 0x32, 0xe5, 0x1e, 0x3a, 0xc6,
+	0xfc, 0x17, 0x73, 0xb4, 0x38, 0x16, 0x58, 0xa9, 0x6d, 0x38, 0xd5, 0x86, 0x76, 0x81, 0xdc, 0x4d,
+	0x38, 0xa0, 0x3c, 0xb0, 0x59, 0xc2, 0x48, 0x3c, 0xd4, 0xd3, 0xe5, 0xbf, 0x97, 0xc2, 0x77, 0x39,
+	0x4b, 0xbe, 0x45, 0xa3, 0xd5, 0xb2, 0x2c, 0xb7, 0x59, 0x6b, 0x99, 0xae, 0xd3, 0x6c, 0x9b, 0x5d,
+	0x25, 0x43, 0x5a, 0x96, 0xdb, 0x30, 0xac, 0x9a, 0x84, 0x0b, 0x23, 0x58, 0xb2, 0x72, 0x4e, 0xc7,
+	0x94, 0x78, 0xf4, 0x93, 0xce, 0x27, 0xe1, 0x78, 0x76, 0x27, 0x78, 0xb6, 0x3c, 0xa5, 0x3f, 0x38,
+	0xc6, 0x5f, 0xbf, 0x53, 0xf9, 0xec, 0xfe, 0xd0, 0xb4, 0xba, 0x72, 0x49, 0xdb, 0xb8, 0xa4, 0xef,
+	0x7e, 0xfa, 0x7e, 0x43, 0xbf, 0xca, 0x67, 0xdd, 0x76, 0xed, 0x56, 0xf7, 0x79, 0xcf, 0x70, 0x1a,
+	0xd4, 0xf4, 0xc2, 0x6e, 0x58, 0x5b, 0xed, 0x86, 0xa9, 0x9d, 0xaf, 0x36, 0x9e, 0x7e, 0xf9, 0x82,
+	0x4f, 0x16, 0xbe, 0x76, 0x52, 0x8b, 0xf9, 0x8a, 0x9a, 0x39, 0xd0, 0x9e, 0x00, 0xa3, 0x73, 0x3e,
+	0xce, 0x81, 0x03, 0xec, 0x2a, 0x9c, 0x61, 0xdb, 0x86, 0xf5, 0x54, 0x13, 0x45, 0x39, 0xc0, 0x17,
+	0xe2, 0xfa, 0x0b, 0xf5, 0xd3, 0xb5, 0x45, 0xff, 0x6a, 0x27, 0xfd, 0xab, 0xbd, 0xe0, 0x5f, 0x6d,
+	0xc5, 0xbf, 0x0e, 0xd5, 0xfb, 0x01, 0x6a, 0x88, 0xb6, 0xeb, 0x89, 0x0e, 0x00, 0x43, 0xd0, 0xd3,
+	0x4a, 0x0f, 0x4e, 0xed, 0x34, 0x8b, 0x3a, 0x44, 0x59, 0xcf, 0x96, 0xfb, 0x71, 0xbb, 0xee, 0x56,
+	0xf6, 0x2c, 0xdb, 0x91, 0xfb, 0x71, 0xbb, 0x2e, 0xce, 0xe9, 0xe5, 0x7f, 0x4e, 0xd7, 0x13, 0xb1,
+	0x33, 0xca, 0xed, 0x83, 0x53, 0x37, 0xa9, 0x49, 0xe8, 0xd6, 0x8d, 0x66, 0xcb, 0x84, 0xd1, 0x70,
+	0x8b, 0x34, 0x1d, 0xb7, 0x62, 0xd4, 0x64, 0x5b, 0x47, 0x78, 0x1e, 0x81, 0xc9, 0x1f, 0xd3, 0x54,
+	0x29, 0x11, 0xb4, 0xd9, 0xb1, 0x1d, 0x6b, 0x0f, 0x51, 0x19, 0xda, 0x7f, 0x08, 0x85, 0x0e, 0x9d,
+	0x8d, 0xe9, 0x45, 0x7f, 0x4d, 0x8c, 0xbb, 0x46, 0x55, 0x8f, 0xa9, 0xf4, 0xd9, 0x04, 0x6e, 0x3d,
+	0x66, 0x13, 0xfd, 0x36, 0x81, 0xda, 0x88, 0xd9, 0x64, 0xdf, 0x4d, 0xe0, 0x72, 0x31, 0x1b, 0xf6,
+	0x22, 0xba, 0x3d, 0x81, 0xca, 0xeb, 0x1f, 0xb0, 0x9b, 0x88, 0xb2, 0x9f, 0x37, 0x9d, 0x6a, 0x43,
+	0x34, 0xc3, 0x08, 0xcf, 0xa8, 0xb2, 0x34, 0x93, 0xed, 0x30, 0x81, 0x2e, 0xc4, 0xa3, 0xca, 0xbe,
+	0x95, 0xc0, 0x6d, 0x52, 0xa7, 0x4d, 0x6a, 0x24, 0xbb, 0xa0, 0x44, 0xb0, 0x45, 0x7b, 0x86, 0xb9,
+	0xc4, 0xb7, 0x2a, 0xea, 0x9f, 0x13, 0x7d, 0xd5, 0x1f, 0x8d, 0xf9, 0x35, 0x55, 0xfe, 0xd7, 0xb8,
+	0xc0, 0x1f, 0x1b, 0xf5, 0xaa, 0xdb, 0xec, 0x54, 0xbb, 0xed, 0x9e, 0xe1, 0x34, 0x61, 0xd7, 0x13,
+	0x5e, 0x06, 0x08, 0xb3, 0x67, 0x5a, 0x70, 0x42, 0xfd, 0xf3, 0x34, 0xe6, 0x97, 0x83, 0xfe, 0x50,
+	0xbc, 0xa1, 0x44, 0x19, 0xb8, 0xe0, 0x15, 0xab, 0xca, 0x57, 0x84, 0xfa, 0x65, 0xb2, 0xcb, 0x21,
+	0xe0, 0xbc, 0xea, 0x16, 0xbb, 0xa9, 0x00, 0xca, 0x1e, 0xa5, 0x96, 0xa6, 0x26, 0xae, 0xc0, 0x24,
+	0xa6, 0x20, 0x36, 0x24, 0x05, 0x89, 0xf2, 0x44, 0x67, 0x06, 0x10, 0xa8, 0xe7, 0x1a, 0xc5, 0xa7,
+	0x20, 0x6d, 0x99, 0x1d, 0x79, 0x52, 0xe4, 0x30, 0x5e, 0x1a, 0xb8, 0x66, 0xbb, 0xe7, 0xbc, 0x94,
+	0xcd, 0x61, 0x05, 0xb1, 0xd7, 0x79, 0xda, 0xe9, 0x3e, 0xef, 0xc8, 0xdd, 0x45, 0xaa, 0xcf, 0x6d,
+	0xde, 0x84, 0x25, 0x8e, 0xe7, 0xd5, 0xb4, 0x5d, 0xbb, 0x65, 0xec, 0x9b, 0x1a, 0x9b, 0x9b, 0x2c,
+	0x3f, 0x1b, 0x8b, 0xaa, 0x50, 0x02, 0x79, 0x9b, 0x48, 0xdb, 0xd4, 0x3f, 0x64, 0x77, 0x09, 0x1c,
+	0xf7, 0x68, 0x69, 0x78, 0xd8, 0x0d, 0xc1, 0x85, 0xb5, 0xad, 0xf2, 0xef, 0x67, 0x30, 0xff, 0x80,
+	0xbd, 0xa9, 0x28, 0xe5, 0xe6, 0xa6, 0x91, 0x0c, 0xc5, 0xac, 0xa2, 0xd7, 0x28, 0x80, 0x30, 0xe9,
+	0x94, 0x30, 0xa8, 0xb1, 0xc4, 0xa0, 0xa2, 0x76, 0x51, 0x90, 0x28, 0x29, 0x33, 0x87, 0xe8, 0xee,
+	0x61, 0x6c, 0xc8, 0x6d, 0x58, 0x20, 0x0c, 0x6b, 0x77, 0x0f, 0x84, 0x69, 0x6b, 0x62, 0x09, 0x0c,
+	0xb1, 0x04, 0xeb, 0x8a, 0x8a, 0x4e, 0x17, 0x36, 0x9d, 0x0e, 0x98, 0x1a, 0x03, 0x5d, 0xf0, 0x63,
+	0x29, 0x9a, 0x13, 0xfe, 0xa0, 0x0c, 0x87, 0x35, 0x69, 0x9e, 0x22, 0x05, 0x30, 0x3c, 0xc8, 0xb9,
+	0x83, 0x76, 0xec, 0xa6, 0xed, 0xc0, 0xa8, 0x4c, 0xbf, 0xc5, 0x8a, 0x84, 0xde, 0xeb, 0xd8, 0x7b,
+	0x3d, 0x50, 0xd2, 0xac, 0xb9, 0x5d, 0xab, 0x66, 0x5a, 0x5a, 0x61, 0xce, 0x1e, 0x8e, 0xb1, 0xab,
+	0x6d, 0xce, 0x4d, 0x00, 0x4a, 0x0c, 0x3e, 0x65, 0x71, 0x38, 0x57, 0x11, 0x60, 0xc0, 0xed, 0x39,
+	0x03, 0xf2, 0xee, 0xb2, 0x98, 0xf5, 0xc5, 0xf2, 0x5f, 0xa4, 0x58, 0x51, 0x2c, 0x8f, 0x5a, 0x5c,
+	0x2a, 0x61, 0x55, 0x69, 0x56, 0x85, 0x3f, 0xf1, 0x1c, 0x26, 0x93, 0x20, 0x22, 0xec, 0xbd, 0x1e,
+	0x82, 0x53, 0x0a, 0x7d, 0xc2, 0xd7, 0x44, 0x1e, 0x8c, 0xe9, 0x65, 0xf5, 0x99, 0xa1, 0x4c, 0xb3,
+	0x88, 0xc2, 0xfe, 0x6f, 0x56, 0x68, 0xdf, 0x5c, 0xb2, 0xfc, 0x6b, 0x73, 0x03, 0xca, 0xe5, 0x5f,
+	0x17, 0x86, 0x6b, 0xc6, 0x8e, 0xb4, 0x21, 0x16, 0xb8, 0x29, 0x16, 0x38, 0x57, 0xfe, 0x17, 0xf4,
+	0xe1, 0x01, 0x4c, 0x1e, 0xfb, 0x5c, 0xaa, 0x6b, 0xb6, 0x97, 0xb9, 0x66, 0x5b, 0x75, 0xcd, 0x24,
+	0x0c, 0x96, 0x47, 0xc6, 0x3f, 0xc1, 0x6a, 0x2d, 0xd8, 0xee, 0x2c, 0x6a, 0x66, 0xcf, 0x21, 0x3b,
+	0xcf, 0x15, 0x64, 0x56, 0xf8, 0x10, 0x21, 0x9f, 0x37, 0x5b, 0xb5, 0xaa, 0x61, 0xd5, 0xa0, 0xac,
+	0x26, 0x9f, 0x23, 0x0c, 0x1e, 0x56, 0xd6, 0xe7, 0xa0, 0xfb, 0x46, 0x6b, 0xcf, 0xd4, 0x36, 0xe6,
+	0x94, 0xe7, 0xa2, 0x45, 0xc7, 0x48, 0x00, 0x7b, 0x96, 0x69, 0x99, 0xcf, 0xb4, 0xbc, 0x22, 0xa1,
+	0xb6, 0xd7, 0x23, 0xb9, 0x4c, 0xd8, 0xa9, 0x2d, 0xec, 0x54, 0x28, 0xff, 0x11, 0x39, 0x49, 0x5c,
+	0x2e, 0x2b, 0xb9, 0x17, 0x07, 0xac, 0xb7, 0xeb, 0xd2, 0x4b, 0x64, 0xf9, 0xc4, 0x81, 0x94, 0xe6,
+	0xf7, 0x5a, 0x2d, 0x99, 0x37, 0x39, 0x7c, 0xce, 0x45, 0x14, 0x31, 0xa2, 0x96, 0xce, 0x88, 0x82,
+	0xbc, 0x2d, 0xf3, 0xb7, 0x2c, 0xa3, 0xa5, 0x04, 0xaa, 0xcc, 0xd6, 0xe6, 0x11, 0xd5, 0x6e, 0xbb,
+	0x6d, 0x74, 0xc0, 0x4e, 0x38, 0x79, 0x89, 0xa8, 0xb7, 0x8c, 0x5d, 0x5b, 0xdb, 0x28, 0xff, 0x41,
+	0x06, 0xbf, 0x5c, 0x8b, 0x2b, 0x61, 0x75, 0x56, 0xa8, 0xe8, 0x2e, 0x30, 0xe1, 0x86, 0x6b, 0xbe,
+	0x68, 0xda, 0x8e, 0x2d, 0xdf, 0x55, 0x70, 0x8c, 0x28, 0x33, 0x31, 0xd6, 0x53, 0xe4, 0xcb, 0x1c,
+	0xf5, 0xdc, 0x6c, 0xee, 0x36, 0x1c, 0x35, 0xa8, 0x65, 0x18, 0x70, 0x3c, 0xa4, 0x88, 0x6e, 0x1d,
+	0x39, 0xe1, 0xac, 0x85, 0x3b, 0xa6, 0x8a, 0xaa, 0xec, 0x41, 0x9e, 0x85, 0x93, 0xc3, 0x5d, 0x76,
+	0x4b, 0xe0, 0xaa, 0x0d, 0xa3, 0xd9, 0x69, 0x76, 0x76, 0x13, 0x82, 0xd7, 0x28, 0xc9, 0xe0, 0xc0,
+	0x3c, 0xcb, 0xa8, 0xe8, 0x75, 0x51, 0x86, 0x03, 0xba, 0xd5, 0xed, 0xf6, 0xe4, 0x86, 0xb1, 0xab,
+	0x2c, 0x1a, 0x4d, 0x22, 0xa7, 0xa2, 0xf8, 0x68, 0x66, 0x4d, 0xe6, 0x32, 0xf4, 0x97, 0x5d, 0x69,
+	0x7b, 0x88, 0x0c, 0xd1, 0x5e, 0xdc, 0x9d, 0x37, 0x7c, 0x81, 0x9c, 0x40, 0x22, 0x70, 0x42, 0xda,
+	0x26, 0x2d, 0x88, 0x84, 0x73, 0x8d, 0xe5, 0xbb, 0xc5, 0xdd, 0x78, 0xb1, 0xb7, 0xcb, 0xbf, 0x4b,
+	0x8e, 0x27, 0xfe, 0x52, 0x70, 0x62, 0x89, 0x50, 0x9b, 0x9e, 0x10, 0x43, 0x4d, 0x5e, 0xd4, 0x46,
+	0x42, 0x1b, 0x18, 0x63, 0xb2, 0x96, 0xed, 0xc5, 0x6a, 0xf2, 0x17, 0xa5, 0x62, 0x51, 0x24, 0xdc,
+	0xa8, 0xed, 0x9b, 0x96, 0xd3, 0xb4, 0x4d, 0xe9, 0x7e, 0x3d, 0xc5, 0xfd, 0xca, 0x7f, 0x15, 0x9d,
+	0x46, 0xfe, 0x39, 0xed, 0x84, 0x46, 0xf4, 0x8e, 0x30, 0xe1, 0xdd, 0x32, 0x18, 0x9c, 0xb9, 0x91,
+	0xc5, 0xbb, 0x0c, 0x27, 0x16, 0x9f, 0x2e, 0xff, 0x10, 0xe7, 0x8b, 0xb7, 0xbf, 0xfc, 0xd9, 0x92,
+	0xf9, 0x3e, 0xeb, 0x26, 0xe7, 0x8b, 0x63, 0x4a, 0x28, 0x6e, 0x48, 0x42, 0x36, 0x07, 0x0b, 0xd9,
+	0x7f, 0x8d, 0xdd, 0x5e, 0xf8, 0xc3, 0xe2, 0x4b, 0xd4, 0xb7, 0xab, 0x89, 0x40, 0x11, 0x05, 0x90,
+	0x04, 0x63, 0xea, 0x43, 0xf9, 0x1c, 0x18, 0xeb, 0x7e, 0x6b, 0xfe, 0xee, 0x57, 0x42, 0x3c, 0x1d,
+	0xe0, 0xac, 0x7a, 0x15, 0xea, 0x6e, 0x6e, 0x19, 0x05, 0xc4, 0x3d, 0x36, 0x3e, 0xc2, 0x59, 0x34,
+	0x1a, 0xd4, 0x97, 0x5a, 0xba, 0xfc, 0x87, 0x69, 0xb4, 0x7b, 0x7c, 0xac, 0x58, 0x4c, 0x41, 0xed,
+	0x64, 0x0a, 0xc2, 0x08, 0xe6, 0x40, 0xac, 0x42, 0x29, 0x82, 0x53, 0xb4, 0xe2, 0x6d, 0x35, 0x82,
+	0xb1, 0x5f, 0x91, 0x56, 0x51, 0x22, 0x2e, 0x10, 0x25, 0x2a, 0x8a, 0xf6, 0xbc, 0x9b, 0x67, 0xc9,
+	0x6c, 0xed, 0x64, 0x7e, 0x11, 0x49, 0x5b, 0x82, 0x2d, 0xc3, 0x31, 0x65, 0x32, 0x6a, 0xc7, 0x31,
+	0x61, 0xf1, 0xb7, 0xfb, 0x73, 0xc4, 0x15, 0x90, 0x9c, 0xa3, 0xed, 0x22, 0x01, 0x75, 0x6b, 0xa6,
+	0x63, 0x34, 0x5b, 0x5a, 0x5e, 0x55, 0x95, 0x32, 0x06, 0xd7, 0xd4, 0xd6, 0x98, 0x3a, 0x75, 0x91,
+	0x4c, 0x8c, 0x4e, 0xcd, 0xd6, 0x0a, 0xe5, 0x7f, 0x99, 0x5a, 0xf2, 0x2d, 0x62, 0xb8, 0xcc, 0x89,
+	0xeb, 0x73, 0x4e, 0x4c, 0xef, 0xad, 0x05, 0x58, 0xee, 0xe0, 0x62, 0xc5, 0x62, 0x06, 0xc8, 0x0a,
+	0xf2, 0xb2, 0x44, 0x5d, 0xf1, 0x9a, 0xcc, 0xbc, 0x10, 0x59, 0x87, 0x64, 0x45, 0x2c, 0xd4, 0xa5,
+	0x3f, 0xad, 0x95, 0xff, 0x33, 0xed, 0xce, 0xc9, 0xbf, 0x54, 0x20, 0x8e, 0x7b, 0x70, 0xd2, 0xb6,
+	0xab, 0xf1, 0xf1, 0x8f, 0xdf, 0x1f, 0x79, 0x2e, 0xdf, 0x4d, 0xb7, 0x7b, 0xae, 0xb1, 0xbb, 0x6b,
+	0x99, 0xbb, 0x06, 0x3f, 0xa4, 0xd3, 0x89, 0x4f, 0xdc, 0x46, 0xc9, 0x08, 0x83, 0xf7, 0x92, 0x6f,
+	0x71, 0x25, 0x19, 0x86, 0xd1, 0x5a, 0x0c, 0xc0, 0x14, 0xb8, 0x1e, 0xf3, 0x89, 0xd3, 0xbe, 0x5d,
+	0xd5, 0x36, 0x84, 0xc1, 0x05, 0x54, 0x9c, 0x69, 0x64, 0xa7, 0xb7, 0xdd, 0x23, 0x37, 0xca, 0x8b,
+	0x23, 0x35, 0x01, 0x44, 0x32, 0x60, 0xb1, 0x08, 0x84, 0x4b, 0x11, 0x85, 0x18, 0x93, 0x3c, 0x30,
+	0xc9, 0x2b, 0x1a, 0x62, 0x12, 0x5c, 0x17, 0x71, 0x7c, 0x6a, 0xf7, 0x96, 0x1d, 0xcd, 0x77, 0x96,
+	0xfe, 0x85, 0x0a, 0x57, 0x7c, 0x6d, 0x8f, 0x8c, 0x75, 0x38, 0xcf, 0x2d, 0xbc, 0xe6, 0x15, 0xf0,
+	0x76, 0xd7, 0x32, 0xb5, 0x54, 0xb9, 0x45, 0xf1, 0x98, 0xfc, 0xab, 0x13, 0x24, 0x49, 0x68, 0x5c,
+	0xc7, 0xbb, 0x0d, 0x8a, 0x2c, 0x72, 0x7f, 0x89, 0x21, 0x69, 0x7f, 0x96, 0x41, 0xd5, 0x56, 0x7c,
+	0x8f, 0x2d, 0xfd, 0xa6, 0xe7, 0xa8, 0xa7, 0x68, 0x48, 0x4e, 0xb8, 0xf3, 0x2d, 0x60, 0xdc, 0x76,
+	0xd3, 0xb6, 0x65, 0x45, 0xca, 0xd1, 0x1d, 0xf3, 0x05, 0x9d, 0x39, 0x6d, 0x2d, 0x4d, 0x75, 0xf7,
+	0x3c, 0x02, 0xd9, 0x32, 0xe2, 0x3e, 0x02, 0x60, 0x93, 0x4d, 0xd1, 0x2c, 0xed, 0xf1, 0x8b, 0x28,
+	0x64, 0x5d, 0x53, 0x59, 0x93, 0x6d, 0xd3, 0x75, 0x95, 0x35, 0x81, 0x42, 0xd6, 0x0d, 0x19, 0x03,
+	0x3d, 0x87, 0x1a, 0x02, 0x39, 0x19, 0x8c, 0x30, 0x9a, 0x2c, 0x08, 0x99, 0xb8, 0x60, 0x12, 0x2b,
+	0x61, 0x9b, 0x0e, 0x96, 0x6f, 0xe2, 0x7c, 0xbd, 0x04, 0x87, 0xc3, 0x6c, 0xa9, 0xcc, 0xa8, 0x86,
+	0x64, 0xde, 0x56, 0x99, 0x93, 0x38, 0x64, 0xbe, 0xa8, 0xdf, 0x8c, 0x57, 0x22, 0xe1, 0x5f, 0x3f,
+	0x7b, 0x9f, 0xd1, 0xef, 0xc4, 0x6b, 0xa1, 0xe2, 0x90, 0x15, 0x1c, 0xf0, 0xf7, 0xe8, 0x4f, 0x74,
+	0x60, 0xc9, 0x95, 0xb8, 0x91, 0x41, 0x7d, 0xc1, 0x7a, 0x75, 0xe1, 0xf6, 0x0a, 0xc0, 0xb0, 0x7d,
+	0x48, 0x45, 0x95, 0x96, 0x12, 0xd5, 0x52, 0x8c, 0x69, 0x35, 0xf7, 0xcd, 0x8e, 0x69, 0xc7, 0xd7,
+	0x33, 0x76, 0x95, 0x62, 0x49, 0xcb, 0x2a, 0x0c, 0xb2, 0x82, 0xe2, 0x7d, 0x5b, 0x5b, 0xcb, 0x95,
+	0x5f, 0x63, 0x43, 0x20, 0xbe, 0xf2, 0x8e, 0xb7, 0xdc, 0xc5, 0x16, 0xaa, 0x36, 0xc8, 0x50, 0xcb,
+	0x67, 0x8e, 0xdb, 0x6e, 0x76, 0x30, 0xa3, 0xa7, 0x14, 0x98, 0xf1, 0x02, 0x61, 0x69, 0x8a, 0xc1,
+	0x67, 0x4b, 0x5a, 0x18, 0x3f, 0xc2, 0xd3, 0xf0, 0xdc, 0x9d, 0x67, 0xf2, 0xd3, 0xaa, 0x85, 0xfd,
+	0x94, 0x4e, 0xb7, 0xda, 0x30, 0x3a, 0xbb, 0xa6, 0x6c, 0xe6, 0x0b, 0x84, 0xf9, 0x6c, 0xcf, 0x68,
+	0xc9, 0x0b, 0x6a, 0x02, 0xda, 0x36, 0x6c, 0xdc, 0xbd, 0x92, 0xc4, 0x78, 0xa6, 0xcf, 0x54, 0xf6,
+	0xd8, 0x07, 0x7e, 0x70, 0xc8, 0x6f, 0x01, 0x0e, 0xfc, 0x60, 0xf8, 0x09, 0xfe, 0x2f, 0x32, 0xf2,
+	0x56, 0xe0, 0x83, 0xcf, 0x7e, 0xf8, 0xd9, 0xe1, 0x28, 0x3a, 0x3a, 0x3e, 0xf8, 0x64, 0xe0, 0x4f,
+	0xee, 0x0b, 0xb2, 0xfb, 0x48, 0xf6, 0x2b, 0xf4, 0x9f, 0xcd, 0x9c, 0x7c, 0x7e, 0xff, 0xd0, 0x57,
+	0xff, 0xcb, 0x99, 0x83, 0x75, 0x8e, 0xf9, 0xec, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x7c, 0x46,
+	0xe4, 0xda, 0x96, 0x66, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/openolt/openolt.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/openolt/openolt.pb.go
new file mode 100644
index 0000000..b23fed0
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/openolt/openolt.pb.go
@@ -0,0 +1,6405 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/openolt.proto
+
+package openolt
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	config "github.com/opencord/voltha-protos/v5/go/ext/config"
+	tech_profile "github.com/opencord/voltha-protos/v5/go/tech_profile"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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
+
+// SchedulerConfig from public import voltha_protos/tech_profile.proto
+type SchedulerConfig = tech_profile.SchedulerConfig
+
+// TrafficShapingInfo from public import voltha_protos/tech_profile.proto
+type TrafficShapingInfo = tech_profile.TrafficShapingInfo
+
+// TrafficScheduler from public import voltha_protos/tech_profile.proto
+type TrafficScheduler = tech_profile.TrafficScheduler
+
+// TrafficSchedulers from public import voltha_protos/tech_profile.proto
+type TrafficSchedulers = tech_profile.TrafficSchedulers
+
+// TailDropDiscardConfig from public import voltha_protos/tech_profile.proto
+type TailDropDiscardConfig = tech_profile.TailDropDiscardConfig
+
+// RedDiscardConfig from public import voltha_protos/tech_profile.proto
+type RedDiscardConfig = tech_profile.RedDiscardConfig
+
+// WRedDiscardConfig from public import voltha_protos/tech_profile.proto
+type WRedDiscardConfig = tech_profile.WRedDiscardConfig
+
+// DiscardConfig from public import voltha_protos/tech_profile.proto
+type DiscardConfig = tech_profile.DiscardConfig
+type DiscardConfig_TailDropDiscardConfig = tech_profile.DiscardConfig_TailDropDiscardConfig
+type DiscardConfig_RedDiscardConfig = tech_profile.DiscardConfig_RedDiscardConfig
+type DiscardConfig_WredDiscardConfig = tech_profile.DiscardConfig_WredDiscardConfig
+
+// TrafficQueue from public import voltha_protos/tech_profile.proto
+type TrafficQueue = tech_profile.TrafficQueue
+
+// TrafficQueues from public import voltha_protos/tech_profile.proto
+type TrafficQueues = tech_profile.TrafficQueues
+
+// InstanceControl from public import voltha_protos/tech_profile.proto
+type InstanceControl = tech_profile.InstanceControl
+
+// QThresholds from public import voltha_protos/tech_profile.proto
+type QThresholds = tech_profile.QThresholds
+
+// GemPortAttributes from public import voltha_protos/tech_profile.proto
+type GemPortAttributes = tech_profile.GemPortAttributes
+
+// SchedulerAttributes from public import voltha_protos/tech_profile.proto
+type SchedulerAttributes = tech_profile.SchedulerAttributes
+
+// EPONQueueAttributes from public import voltha_protos/tech_profile.proto
+type EPONQueueAttributes = tech_profile.EPONQueueAttributes
+
+// TechProfile from public import voltha_protos/tech_profile.proto
+type TechProfile = tech_profile.TechProfile
+
+// EponTechProfile from public import voltha_protos/tech_profile.proto
+type EponTechProfile = tech_profile.EponTechProfile
+
+// TechProfileInstance from public import voltha_protos/tech_profile.proto
+type TechProfileInstance = tech_profile.TechProfileInstance
+
+// EponTechProfileInstance from public import voltha_protos/tech_profile.proto
+type EponTechProfileInstance = tech_profile.EponTechProfileInstance
+
+// ResourceInstance from public import voltha_protos/tech_profile.proto
+type ResourceInstance = tech_profile.ResourceInstance
+
+// Direction from public import voltha_protos/tech_profile.proto
+type Direction = tech_profile.Direction
+
+var Direction_name = tech_profile.Direction_name
+var Direction_value = tech_profile.Direction_value
+
+const Direction_UPSTREAM = Direction(tech_profile.Direction_UPSTREAM)
+const Direction_DOWNSTREAM = Direction(tech_profile.Direction_DOWNSTREAM)
+const Direction_BIDIRECTIONAL = Direction(tech_profile.Direction_BIDIRECTIONAL)
+
+// SchedulingPolicy from public import voltha_protos/tech_profile.proto
+type SchedulingPolicy = tech_profile.SchedulingPolicy
+
+var SchedulingPolicy_name = tech_profile.SchedulingPolicy_name
+var SchedulingPolicy_value = tech_profile.SchedulingPolicy_value
+
+const SchedulingPolicy_WRR = SchedulingPolicy(tech_profile.SchedulingPolicy_WRR)
+const SchedulingPolicy_StrictPriority = SchedulingPolicy(tech_profile.SchedulingPolicy_StrictPriority)
+const SchedulingPolicy_Hybrid = SchedulingPolicy(tech_profile.SchedulingPolicy_Hybrid)
+
+// AdditionalBW from public import voltha_protos/tech_profile.proto
+type AdditionalBW = tech_profile.AdditionalBW
+
+var AdditionalBW_name = tech_profile.AdditionalBW_name
+var AdditionalBW_value = tech_profile.AdditionalBW_value
+
+const AdditionalBW_AdditionalBW_None = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_None)
+const AdditionalBW_AdditionalBW_NA = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_NA)
+const AdditionalBW_AdditionalBW_BestEffort = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_BestEffort)
+const AdditionalBW_AdditionalBW_Auto = AdditionalBW(tech_profile.AdditionalBW_AdditionalBW_Auto)
+
+// DiscardPolicy from public import voltha_protos/tech_profile.proto
+type DiscardPolicy = tech_profile.DiscardPolicy
+
+var DiscardPolicy_name = tech_profile.DiscardPolicy_name
+var DiscardPolicy_value = tech_profile.DiscardPolicy_value
+
+const DiscardPolicy_TailDrop = DiscardPolicy(tech_profile.DiscardPolicy_TailDrop)
+const DiscardPolicy_WTailDrop = DiscardPolicy(tech_profile.DiscardPolicy_WTailDrop)
+const DiscardPolicy_Red = DiscardPolicy(tech_profile.DiscardPolicy_Red)
+const DiscardPolicy_WRed = DiscardPolicy(tech_profile.DiscardPolicy_WRed)
+
+// InferredAdditionBWIndication from public import voltha_protos/tech_profile.proto
+type InferredAdditionBWIndication = tech_profile.InferredAdditionBWIndication
+
+var InferredAdditionBWIndication_name = tech_profile.InferredAdditionBWIndication_name
+var InferredAdditionBWIndication_value = tech_profile.InferredAdditionBWIndication_value
+
+const InferredAdditionBWIndication_InferredAdditionBWIndication_None = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_None)
+const InferredAdditionBWIndication_InferredAdditionBWIndication_Assured = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_Assured)
+const InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort = InferredAdditionBWIndication(tech_profile.InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort)
+
+// ID from public import voltha_protos/common.proto
+type ID = common.ID
+
+// IDs from public import voltha_protos/common.proto
+type IDs = common.IDs
+
+// AdminState from public import voltha_protos/common.proto
+type AdminState = common.AdminState
+
+// OperStatus from public import voltha_protos/common.proto
+type OperStatus = common.OperStatus
+
+// ConnectStatus from public import voltha_protos/common.proto
+type ConnectStatus = common.ConnectStatus
+
+// OperationResp from public import voltha_protos/common.proto
+type OperationResp = common.OperationResp
+
+// ValueType from public import voltha_protos/common.proto
+type ValueType = common.ValueType
+
+// ValueSpecifier from public import voltha_protos/common.proto
+type ValueSpecifier = common.ValueSpecifier
+
+// ReturnValues from public import voltha_protos/common.proto
+type ReturnValues = common.ReturnValues
+
+// TestModeKeys from public import voltha_protos/common.proto
+type TestModeKeys = common.TestModeKeys
+
+var TestModeKeys_name = common.TestModeKeys_name
+var TestModeKeys_value = common.TestModeKeys_value
+
+const TestModeKeys_api_test = TestModeKeys(common.TestModeKeys_api_test)
+
+// AdminState_Types from public import voltha_protos/common.proto
+type AdminState_Types = common.AdminState_Types
+
+var AdminState_Types_name = common.AdminState_Types_name
+var AdminState_Types_value = common.AdminState_Types_value
+
+const AdminState_UNKNOWN = AdminState_Types(common.AdminState_UNKNOWN)
+const AdminState_PREPROVISIONED = AdminState_Types(common.AdminState_PREPROVISIONED)
+const AdminState_ENABLED = AdminState_Types(common.AdminState_ENABLED)
+const AdminState_DISABLED = AdminState_Types(common.AdminState_DISABLED)
+const AdminState_DOWNLOADING_IMAGE = AdminState_Types(common.AdminState_DOWNLOADING_IMAGE)
+
+// OperStatus_Types from public import voltha_protos/common.proto
+type OperStatus_Types = common.OperStatus_Types
+
+var OperStatus_Types_name = common.OperStatus_Types_name
+var OperStatus_Types_value = common.OperStatus_Types_value
+
+const OperStatus_UNKNOWN = OperStatus_Types(common.OperStatus_UNKNOWN)
+const OperStatus_DISCOVERED = OperStatus_Types(common.OperStatus_DISCOVERED)
+const OperStatus_ACTIVATING = OperStatus_Types(common.OperStatus_ACTIVATING)
+const OperStatus_TESTING = OperStatus_Types(common.OperStatus_TESTING)
+const OperStatus_ACTIVE = OperStatus_Types(common.OperStatus_ACTIVE)
+const OperStatus_FAILED = OperStatus_Types(common.OperStatus_FAILED)
+const OperStatus_RECONCILING = OperStatus_Types(common.OperStatus_RECONCILING)
+const OperStatus_RECONCILING_FAILED = OperStatus_Types(common.OperStatus_RECONCILING_FAILED)
+
+// ConnectStatus_Types from public import voltha_protos/common.proto
+type ConnectStatus_Types = common.ConnectStatus_Types
+
+var ConnectStatus_Types_name = common.ConnectStatus_Types_name
+var ConnectStatus_Types_value = common.ConnectStatus_Types_value
+
+const ConnectStatus_UNKNOWN = ConnectStatus_Types(common.ConnectStatus_UNKNOWN)
+const ConnectStatus_UNREACHABLE = ConnectStatus_Types(common.ConnectStatus_UNREACHABLE)
+const ConnectStatus_REACHABLE = ConnectStatus_Types(common.ConnectStatus_REACHABLE)
+
+// OperationResp_OperationReturnCode from public import voltha_protos/common.proto
+type OperationResp_OperationReturnCode = common.OperationResp_OperationReturnCode
+
+var OperationResp_OperationReturnCode_name = common.OperationResp_OperationReturnCode_name
+var OperationResp_OperationReturnCode_value = common.OperationResp_OperationReturnCode_value
+
+const OperationResp_OPERATION_SUCCESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_SUCCESS)
+const OperationResp_OPERATION_FAILURE = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_FAILURE)
+const OperationResp_OPERATION_UNSUPPORTED = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_UNSUPPORTED)
+const OperationResp_OPERATION_IN_PROGRESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_IN_PROGRESS)
+
+// ValueType_Type from public import voltha_protos/common.proto
+type ValueType_Type = common.ValueType_Type
+
+var ValueType_Type_name = common.ValueType_Type_name
+var ValueType_Type_value = common.ValueType_Type_value
+
+const ValueType_EMPTY = ValueType_Type(common.ValueType_EMPTY)
+const ValueType_DISTANCE = ValueType_Type(common.ValueType_DISTANCE)
+
+//* activation fail reason.
+type OnuIndication_ActivationFailReason int32
+
+const (
+	OnuIndication_ONU_ACTIVATION_FAIL_REASON_NONE                    OnuIndication_ActivationFailReason = 0
+	OnuIndication_ONU_ACTIVATION_FAIL_REASON_RANGING                 OnuIndication_ActivationFailReason = 1
+	OnuIndication_ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION OnuIndication_ActivationFailReason = 2
+	OnuIndication_ONU_ACTIVATION_FAIL_REASON_LOS                     OnuIndication_ActivationFailReason = 3
+	OnuIndication_ONU_ACTIVATION_FAIL_ONU_ALARM                      OnuIndication_ActivationFailReason = 4
+	OnuIndication_ONU_ACTIVATION_FAIL_SWITCH_OVER                    OnuIndication_ActivationFailReason = 5
+)
+
+var OnuIndication_ActivationFailReason_name = map[int32]string{
+	0: "ONU_ACTIVATION_FAIL_REASON_NONE",
+	1: "ONU_ACTIVATION_FAIL_REASON_RANGING",
+	2: "ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION",
+	3: "ONU_ACTIVATION_FAIL_REASON_LOS",
+	4: "ONU_ACTIVATION_FAIL_ONU_ALARM",
+	5: "ONU_ACTIVATION_FAIL_SWITCH_OVER",
+}
+
+var OnuIndication_ActivationFailReason_value = map[string]int32{
+	"ONU_ACTIVATION_FAIL_REASON_NONE":                    0,
+	"ONU_ACTIVATION_FAIL_REASON_RANGING":                 1,
+	"ONU_ACTIVATION_FAIL_REASON_PASSWORD_AUTHENTICATION": 2,
+	"ONU_ACTIVATION_FAIL_REASON_LOS":                     3,
+	"ONU_ACTIVATION_FAIL_ONU_ALARM":                      4,
+	"ONU_ACTIVATION_FAIL_SWITCH_OVER":                    5,
+}
+
+func (x OnuIndication_ActivationFailReason) String() string {
+	return proto.EnumName(OnuIndication_ActivationFailReason_name, int32(x))
+}
+
+func (OnuIndication_ActivationFailReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{5, 0}
+}
+
+type DeviceInfo_DeviceResourceRanges_Pool_PoolType int32
+
+const (
+	DeviceInfo_DeviceResourceRanges_Pool_ONU_ID     DeviceInfo_DeviceResourceRanges_Pool_PoolType = 0
+	DeviceInfo_DeviceResourceRanges_Pool_ALLOC_ID   DeviceInfo_DeviceResourceRanges_Pool_PoolType = 1
+	DeviceInfo_DeviceResourceRanges_Pool_GEMPORT_ID DeviceInfo_DeviceResourceRanges_Pool_PoolType = 2
+	DeviceInfo_DeviceResourceRanges_Pool_FLOW_ID    DeviceInfo_DeviceResourceRanges_Pool_PoolType = 3
+)
+
+var DeviceInfo_DeviceResourceRanges_Pool_PoolType_name = map[int32]string{
+	0: "ONU_ID",
+	1: "ALLOC_ID",
+	2: "GEMPORT_ID",
+	3: "FLOW_ID",
+}
+
+var DeviceInfo_DeviceResourceRanges_Pool_PoolType_value = map[string]int32{
+	"ONU_ID":     0,
+	"ALLOC_ID":   1,
+	"GEMPORT_ID": 2,
+	"FLOW_ID":    3,
+}
+
+func (x DeviceInfo_DeviceResourceRanges_Pool_PoolType) String() string {
+	return proto.EnumName(DeviceInfo_DeviceResourceRanges_Pool_PoolType_name, int32(x))
+}
+
+func (DeviceInfo_DeviceResourceRanges_Pool_PoolType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0, 0}
+}
+
+type DeviceInfo_DeviceResourceRanges_Pool_SharingType int32
+
+const (
+	DeviceInfo_DeviceResourceRanges_Pool_DEDICATED_PER_INTF           DeviceInfo_DeviceResourceRanges_Pool_SharingType = 0
+	DeviceInfo_DeviceResourceRanges_Pool_SHARED_BY_ALL_INTF_ALL_TECH  DeviceInfo_DeviceResourceRanges_Pool_SharingType = 1
+	DeviceInfo_DeviceResourceRanges_Pool_SHARED_BY_ALL_INTF_SAME_TECH DeviceInfo_DeviceResourceRanges_Pool_SharingType = 2
+)
+
+var DeviceInfo_DeviceResourceRanges_Pool_SharingType_name = map[int32]string{
+	0: "DEDICATED_PER_INTF",
+	1: "SHARED_BY_ALL_INTF_ALL_TECH",
+	2: "SHARED_BY_ALL_INTF_SAME_TECH",
+}
+
+var DeviceInfo_DeviceResourceRanges_Pool_SharingType_value = map[string]int32{
+	"DEDICATED_PER_INTF":           0,
+	"SHARED_BY_ALL_INTF_ALL_TECH":  1,
+	"SHARED_BY_ALL_INTF_SAME_TECH": 2,
+}
+
+func (x DeviceInfo_DeviceResourceRanges_Pool_SharingType) String() string {
+	return proto.EnumName(DeviceInfo_DeviceResourceRanges_Pool_SharingType_name, int32(x))
+}
+
+func (DeviceInfo_DeviceResourceRanges_Pool_SharingType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0, 1}
+}
+
+type GroupMember_InterfaceType int32
+
+const (
+	GroupMember_PON           GroupMember_InterfaceType = 0
+	GroupMember_EPON_1G_PATH  GroupMember_InterfaceType = 1
+	GroupMember_EPON_10G_PATH GroupMember_InterfaceType = 2
+)
+
+var GroupMember_InterfaceType_name = map[int32]string{
+	0: "PON",
+	1: "EPON_1G_PATH",
+	2: "EPON_10G_PATH",
+}
+
+var GroupMember_InterfaceType_value = map[string]int32{
+	"PON":           0,
+	"EPON_1G_PATH":  1,
+	"EPON_10G_PATH": 2,
+}
+
+func (x GroupMember_InterfaceType) String() string {
+	return proto.EnumName(GroupMember_InterfaceType_name, int32(x))
+}
+
+func (GroupMember_InterfaceType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{46, 0}
+}
+
+type Group_GroupMembersCommand int32
+
+const (
+	Group_ADD_MEMBERS    Group_GroupMembersCommand = 0
+	Group_REMOVE_MEMBERS Group_GroupMembersCommand = 1
+	Group_SET_MEMBERS    Group_GroupMembersCommand = 2
+)
+
+var Group_GroupMembersCommand_name = map[int32]string{
+	0: "ADD_MEMBERS",
+	1: "REMOVE_MEMBERS",
+	2: "SET_MEMBERS",
+}
+
+var Group_GroupMembersCommand_value = map[string]int32{
+	"ADD_MEMBERS":    0,
+	"REMOVE_MEMBERS": 1,
+	"SET_MEMBERS":    2,
+}
+
+func (x Group_GroupMembersCommand) String() string {
+	return proto.EnumName(Group_GroupMembersCommand_name, int32(x))
+}
+
+func (Group_GroupMembersCommand) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{47, 0}
+}
+
+type PonRxPowerData_RssiMeasurementFailReason int32
+
+const (
+	PonRxPowerData_FAIL_REASON_NONE         PonRxPowerData_RssiMeasurementFailReason = 0
+	PonRxPowerData_FAIL_REASON_NO_DELIMITER PonRxPowerData_RssiMeasurementFailReason = 1
+	PonRxPowerData_FAIL_REASON_NO_ACCESS    PonRxPowerData_RssiMeasurementFailReason = 2
+)
+
+var PonRxPowerData_RssiMeasurementFailReason_name = map[int32]string{
+	0: "FAIL_REASON_NONE",
+	1: "FAIL_REASON_NO_DELIMITER",
+	2: "FAIL_REASON_NO_ACCESS",
+}
+
+var PonRxPowerData_RssiMeasurementFailReason_value = map[string]int32{
+	"FAIL_REASON_NONE":         0,
+	"FAIL_REASON_NO_DELIMITER": 1,
+	"FAIL_REASON_NO_ACCESS":    2,
+}
+
+func (x PonRxPowerData_RssiMeasurementFailReason) String() string {
+	return proto.EnumName(PonRxPowerData_RssiMeasurementFailReason_name, int32(x))
+}
+
+func (PonRxPowerData_RssiMeasurementFailReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{49, 0}
+}
+
+type Indication struct {
+	// Types that are valid to be assigned to Data:
+	//	*Indication_OltInd
+	//	*Indication_IntfInd
+	//	*Indication_IntfOperInd
+	//	*Indication_OnuDiscInd
+	//	*Indication_OnuInd
+	//	*Indication_OmciInd
+	//	*Indication_PktInd
+	//	*Indication_PortStats
+	//	*Indication_FlowStats
+	//	*Indication_AlarmInd
+	Data                 isIndication_Data `protobuf_oneof:"data"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *Indication) Reset()         { *m = Indication{} }
+func (m *Indication) String() string { return proto.CompactTextString(m) }
+func (*Indication) ProtoMessage()    {}
+func (*Indication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{0}
+}
+
+func (m *Indication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Indication.Unmarshal(m, b)
+}
+func (m *Indication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Indication.Marshal(b, m, deterministic)
+}
+func (m *Indication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Indication.Merge(m, src)
+}
+func (m *Indication) XXX_Size() int {
+	return xxx_messageInfo_Indication.Size(m)
+}
+func (m *Indication) XXX_DiscardUnknown() {
+	xxx_messageInfo_Indication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Indication proto.InternalMessageInfo
+
+type isIndication_Data interface {
+	isIndication_Data()
+}
+
+type Indication_OltInd struct {
+	OltInd *OltIndication `protobuf:"bytes,1,opt,name=olt_ind,json=oltInd,proto3,oneof"`
+}
+
+type Indication_IntfInd struct {
+	IntfInd *IntfIndication `protobuf:"bytes,2,opt,name=intf_ind,json=intfInd,proto3,oneof"`
+}
+
+type Indication_IntfOperInd struct {
+	IntfOperInd *IntfOperIndication `protobuf:"bytes,3,opt,name=intf_oper_ind,json=intfOperInd,proto3,oneof"`
+}
+
+type Indication_OnuDiscInd struct {
+	OnuDiscInd *OnuDiscIndication `protobuf:"bytes,4,opt,name=onu_disc_ind,json=onuDiscInd,proto3,oneof"`
+}
+
+type Indication_OnuInd struct {
+	OnuInd *OnuIndication `protobuf:"bytes,5,opt,name=onu_ind,json=onuInd,proto3,oneof"`
+}
+
+type Indication_OmciInd struct {
+	OmciInd *OmciIndication `protobuf:"bytes,6,opt,name=omci_ind,json=omciInd,proto3,oneof"`
+}
+
+type Indication_PktInd struct {
+	PktInd *PacketIndication `protobuf:"bytes,7,opt,name=pkt_ind,json=pktInd,proto3,oneof"`
+}
+
+type Indication_PortStats struct {
+	PortStats *PortStatistics `protobuf:"bytes,8,opt,name=port_stats,json=portStats,proto3,oneof"`
+}
+
+type Indication_FlowStats struct {
+	FlowStats *FlowStatistics `protobuf:"bytes,9,opt,name=flow_stats,json=flowStats,proto3,oneof"`
+}
+
+type Indication_AlarmInd struct {
+	AlarmInd *AlarmIndication `protobuf:"bytes,10,opt,name=alarm_ind,json=alarmInd,proto3,oneof"`
+}
+
+func (*Indication_OltInd) isIndication_Data() {}
+
+func (*Indication_IntfInd) isIndication_Data() {}
+
+func (*Indication_IntfOperInd) isIndication_Data() {}
+
+func (*Indication_OnuDiscInd) isIndication_Data() {}
+
+func (*Indication_OnuInd) isIndication_Data() {}
+
+func (*Indication_OmciInd) isIndication_Data() {}
+
+func (*Indication_PktInd) isIndication_Data() {}
+
+func (*Indication_PortStats) isIndication_Data() {}
+
+func (*Indication_FlowStats) isIndication_Data() {}
+
+func (*Indication_AlarmInd) isIndication_Data() {}
+
+func (m *Indication) GetData() isIndication_Data {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+func (m *Indication) GetOltInd() *OltIndication {
+	if x, ok := m.GetData().(*Indication_OltInd); ok {
+		return x.OltInd
+	}
+	return nil
+}
+
+func (m *Indication) GetIntfInd() *IntfIndication {
+	if x, ok := m.GetData().(*Indication_IntfInd); ok {
+		return x.IntfInd
+	}
+	return nil
+}
+
+func (m *Indication) GetIntfOperInd() *IntfOperIndication {
+	if x, ok := m.GetData().(*Indication_IntfOperInd); ok {
+		return x.IntfOperInd
+	}
+	return nil
+}
+
+func (m *Indication) GetOnuDiscInd() *OnuDiscIndication {
+	if x, ok := m.GetData().(*Indication_OnuDiscInd); ok {
+		return x.OnuDiscInd
+	}
+	return nil
+}
+
+func (m *Indication) GetOnuInd() *OnuIndication {
+	if x, ok := m.GetData().(*Indication_OnuInd); ok {
+		return x.OnuInd
+	}
+	return nil
+}
+
+func (m *Indication) GetOmciInd() *OmciIndication {
+	if x, ok := m.GetData().(*Indication_OmciInd); ok {
+		return x.OmciInd
+	}
+	return nil
+}
+
+func (m *Indication) GetPktInd() *PacketIndication {
+	if x, ok := m.GetData().(*Indication_PktInd); ok {
+		return x.PktInd
+	}
+	return nil
+}
+
+func (m *Indication) GetPortStats() *PortStatistics {
+	if x, ok := m.GetData().(*Indication_PortStats); ok {
+		return x.PortStats
+	}
+	return nil
+}
+
+func (m *Indication) GetFlowStats() *FlowStatistics {
+	if x, ok := m.GetData().(*Indication_FlowStats); ok {
+		return x.FlowStats
+	}
+	return nil
+}
+
+func (m *Indication) GetAlarmInd() *AlarmIndication {
+	if x, ok := m.GetData().(*Indication_AlarmInd); ok {
+		return x.AlarmInd
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*Indication) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*Indication_OltInd)(nil),
+		(*Indication_IntfInd)(nil),
+		(*Indication_IntfOperInd)(nil),
+		(*Indication_OnuDiscInd)(nil),
+		(*Indication_OnuInd)(nil),
+		(*Indication_OmciInd)(nil),
+		(*Indication_PktInd)(nil),
+		(*Indication_PortStats)(nil),
+		(*Indication_FlowStats)(nil),
+		(*Indication_AlarmInd)(nil),
+	}
+}
+
+type AlarmIndication struct {
+	// Types that are valid to be assigned to Data:
+	//	*AlarmIndication_LosInd
+	//	*AlarmIndication_DyingGaspInd
+	//	*AlarmIndication_OnuAlarmInd
+	//	*AlarmIndication_OnuStartupFailInd
+	//	*AlarmIndication_OnuSignalDegradeInd
+	//	*AlarmIndication_OnuDriftOfWindowInd
+	//	*AlarmIndication_OnuLossOmciInd
+	//	*AlarmIndication_OnuSignalsFailInd
+	//	*AlarmIndication_OnuTiwiInd
+	//	*AlarmIndication_OnuActivationFailInd
+	//	*AlarmIndication_OnuProcessingErrorInd
+	//	*AlarmIndication_OnuLossOfSyncFailInd
+	//	*AlarmIndication_OnuItuPonStatsInd
+	//	*AlarmIndication_OnuDeactivationFailureInd
+	//	*AlarmIndication_OnuRemoteDefectInd
+	//	*AlarmIndication_OnuLossGemDelineationInd
+	//	*AlarmIndication_OnuPhysicalEquipmentErrorInd
+	//	*AlarmIndication_OnuLossOfAckInd
+	//	*AlarmIndication_OnuDiffReachExceededInd
+	Data                 isAlarmIndication_Data `protobuf_oneof:"data"`
+	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
+	XXX_unrecognized     []byte                 `json:"-"`
+	XXX_sizecache        int32                  `json:"-"`
+}
+
+func (m *AlarmIndication) Reset()         { *m = AlarmIndication{} }
+func (m *AlarmIndication) String() string { return proto.CompactTextString(m) }
+func (*AlarmIndication) ProtoMessage()    {}
+func (*AlarmIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{1}
+}
+
+func (m *AlarmIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AlarmIndication.Unmarshal(m, b)
+}
+func (m *AlarmIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AlarmIndication.Marshal(b, m, deterministic)
+}
+func (m *AlarmIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AlarmIndication.Merge(m, src)
+}
+func (m *AlarmIndication) XXX_Size() int {
+	return xxx_messageInfo_AlarmIndication.Size(m)
+}
+func (m *AlarmIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_AlarmIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AlarmIndication proto.InternalMessageInfo
+
+type isAlarmIndication_Data interface {
+	isAlarmIndication_Data()
+}
+
+type AlarmIndication_LosInd struct {
+	LosInd *LosIndication `protobuf:"bytes,1,opt,name=los_ind,json=losInd,proto3,oneof"`
+}
+
+type AlarmIndication_DyingGaspInd struct {
+	DyingGaspInd *DyingGaspIndication `protobuf:"bytes,2,opt,name=dying_gasp_ind,json=dyingGaspInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuAlarmInd struct {
+	OnuAlarmInd *OnuAlarmIndication `protobuf:"bytes,3,opt,name=onu_alarm_ind,json=onuAlarmInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuStartupFailInd struct {
+	OnuStartupFailInd *OnuStartupFailureIndication `protobuf:"bytes,4,opt,name=onu_startup_fail_ind,json=onuStartupFailInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuSignalDegradeInd struct {
+	OnuSignalDegradeInd *OnuSignalDegradeIndication `protobuf:"bytes,5,opt,name=onu_signal_degrade_ind,json=onuSignalDegradeInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuDriftOfWindowInd struct {
+	OnuDriftOfWindowInd *OnuDriftOfWindowIndication `protobuf:"bytes,6,opt,name=onu_drift_of_window_ind,json=onuDriftOfWindowInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuLossOmciInd struct {
+	OnuLossOmciInd *OnuLossOfOmciChannelIndication `protobuf:"bytes,7,opt,name=onu_loss_omci_ind,json=onuLossOmciInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuSignalsFailInd struct {
+	OnuSignalsFailInd *OnuSignalsFailureIndication `protobuf:"bytes,8,opt,name=onu_signals_fail_ind,json=onuSignalsFailInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuTiwiInd struct {
+	OnuTiwiInd *OnuTransmissionInterferenceWarning `protobuf:"bytes,9,opt,name=onu_tiwi_ind,json=onuTiwiInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuActivationFailInd struct {
+	OnuActivationFailInd *OnuActivationFailureIndication `protobuf:"bytes,10,opt,name=onu_activation_fail_ind,json=onuActivationFailInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuProcessingErrorInd struct {
+	OnuProcessingErrorInd *OnuProcessingErrorIndication `protobuf:"bytes,11,opt,name=onu_processing_error_ind,json=onuProcessingErrorInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuLossOfSyncFailInd struct {
+	OnuLossOfSyncFailInd *OnuLossOfKeySyncFailureIndication `protobuf:"bytes,12,opt,name=onu_loss_of_sync_fail_ind,json=onuLossOfSyncFailInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuItuPonStatsInd struct {
+	OnuItuPonStatsInd *OnuItuPonStatsIndication `protobuf:"bytes,13,opt,name=onu_itu_pon_stats_ind,json=onuItuPonStatsInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuDeactivationFailureInd struct {
+	OnuDeactivationFailureInd *OnuDeactivationFailureIndication `protobuf:"bytes,14,opt,name=onu_deactivation_failure_ind,json=onuDeactivationFailureInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuRemoteDefectInd struct {
+	OnuRemoteDefectInd *OnuRemoteDefectIndication `protobuf:"bytes,15,opt,name=onu_remote_defect_ind,json=onuRemoteDefectInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuLossGemDelineationInd struct {
+	OnuLossGemDelineationInd *OnuLossOfGEMChannelDelineationIndication `protobuf:"bytes,16,opt,name=onu_loss_gem_delineation_ind,json=onuLossGemDelineationInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuPhysicalEquipmentErrorInd struct {
+	OnuPhysicalEquipmentErrorInd *OnuPhysicalEquipmentErrorIndication `protobuf:"bytes,17,opt,name=onu_physical_equipment_error_ind,json=onuPhysicalEquipmentErrorInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuLossOfAckInd struct {
+	OnuLossOfAckInd *OnuLossOfAcknowledgementIndication `protobuf:"bytes,18,opt,name=onu_loss_of_ack_ind,json=onuLossOfAckInd,proto3,oneof"`
+}
+
+type AlarmIndication_OnuDiffReachExceededInd struct {
+	OnuDiffReachExceededInd *OnuDifferentialReachExceededIndication `protobuf:"bytes,19,opt,name=onu_diff_reach_exceeded_ind,json=onuDiffReachExceededInd,proto3,oneof"`
+}
+
+func (*AlarmIndication_LosInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_DyingGaspInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuAlarmInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuStartupFailInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuSignalDegradeInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuDriftOfWindowInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuLossOmciInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuSignalsFailInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuTiwiInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuActivationFailInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuProcessingErrorInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuLossOfSyncFailInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuItuPonStatsInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuDeactivationFailureInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuRemoteDefectInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuLossGemDelineationInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuPhysicalEquipmentErrorInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuLossOfAckInd) isAlarmIndication_Data() {}
+
+func (*AlarmIndication_OnuDiffReachExceededInd) isAlarmIndication_Data() {}
+
+func (m *AlarmIndication) GetData() isAlarmIndication_Data {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetLosInd() *LosIndication {
+	if x, ok := m.GetData().(*AlarmIndication_LosInd); ok {
+		return x.LosInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetDyingGaspInd() *DyingGaspIndication {
+	if x, ok := m.GetData().(*AlarmIndication_DyingGaspInd); ok {
+		return x.DyingGaspInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuAlarmInd() *OnuAlarmIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuAlarmInd); ok {
+		return x.OnuAlarmInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuStartupFailInd() *OnuStartupFailureIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuStartupFailInd); ok {
+		return x.OnuStartupFailInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuSignalDegradeInd() *OnuSignalDegradeIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuSignalDegradeInd); ok {
+		return x.OnuSignalDegradeInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuDriftOfWindowInd() *OnuDriftOfWindowIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuDriftOfWindowInd); ok {
+		return x.OnuDriftOfWindowInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuLossOmciInd() *OnuLossOfOmciChannelIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuLossOmciInd); ok {
+		return x.OnuLossOmciInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuSignalsFailInd() *OnuSignalsFailureIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuSignalsFailInd); ok {
+		return x.OnuSignalsFailInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuTiwiInd() *OnuTransmissionInterferenceWarning {
+	if x, ok := m.GetData().(*AlarmIndication_OnuTiwiInd); ok {
+		return x.OnuTiwiInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuActivationFailInd() *OnuActivationFailureIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuActivationFailInd); ok {
+		return x.OnuActivationFailInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuProcessingErrorInd() *OnuProcessingErrorIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuProcessingErrorInd); ok {
+		return x.OnuProcessingErrorInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuLossOfSyncFailInd() *OnuLossOfKeySyncFailureIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuLossOfSyncFailInd); ok {
+		return x.OnuLossOfSyncFailInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuItuPonStatsInd() *OnuItuPonStatsIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuItuPonStatsInd); ok {
+		return x.OnuItuPonStatsInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuDeactivationFailureInd() *OnuDeactivationFailureIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuDeactivationFailureInd); ok {
+		return x.OnuDeactivationFailureInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuRemoteDefectInd() *OnuRemoteDefectIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuRemoteDefectInd); ok {
+		return x.OnuRemoteDefectInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuLossGemDelineationInd() *OnuLossOfGEMChannelDelineationIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuLossGemDelineationInd); ok {
+		return x.OnuLossGemDelineationInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuPhysicalEquipmentErrorInd() *OnuPhysicalEquipmentErrorIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuPhysicalEquipmentErrorInd); ok {
+		return x.OnuPhysicalEquipmentErrorInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuLossOfAckInd() *OnuLossOfAcknowledgementIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuLossOfAckInd); ok {
+		return x.OnuLossOfAckInd
+	}
+	return nil
+}
+
+func (m *AlarmIndication) GetOnuDiffReachExceededInd() *OnuDifferentialReachExceededIndication {
+	if x, ok := m.GetData().(*AlarmIndication_OnuDiffReachExceededInd); ok {
+		return x.OnuDiffReachExceededInd
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*AlarmIndication) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*AlarmIndication_LosInd)(nil),
+		(*AlarmIndication_DyingGaspInd)(nil),
+		(*AlarmIndication_OnuAlarmInd)(nil),
+		(*AlarmIndication_OnuStartupFailInd)(nil),
+		(*AlarmIndication_OnuSignalDegradeInd)(nil),
+		(*AlarmIndication_OnuDriftOfWindowInd)(nil),
+		(*AlarmIndication_OnuLossOmciInd)(nil),
+		(*AlarmIndication_OnuSignalsFailInd)(nil),
+		(*AlarmIndication_OnuTiwiInd)(nil),
+		(*AlarmIndication_OnuActivationFailInd)(nil),
+		(*AlarmIndication_OnuProcessingErrorInd)(nil),
+		(*AlarmIndication_OnuLossOfSyncFailInd)(nil),
+		(*AlarmIndication_OnuItuPonStatsInd)(nil),
+		(*AlarmIndication_OnuDeactivationFailureInd)(nil),
+		(*AlarmIndication_OnuRemoteDefectInd)(nil),
+		(*AlarmIndication_OnuLossGemDelineationInd)(nil),
+		(*AlarmIndication_OnuPhysicalEquipmentErrorInd)(nil),
+		(*AlarmIndication_OnuLossOfAckInd)(nil),
+		(*AlarmIndication_OnuDiffReachExceededInd)(nil),
+	}
+}
+
+type OltIndication struct {
+	OperState            string   `protobuf:"bytes,1,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OltIndication) Reset()         { *m = OltIndication{} }
+func (m *OltIndication) String() string { return proto.CompactTextString(m) }
+func (*OltIndication) ProtoMessage()    {}
+func (*OltIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{2}
+}
+
+func (m *OltIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OltIndication.Unmarshal(m, b)
+}
+func (m *OltIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OltIndication.Marshal(b, m, deterministic)
+}
+func (m *OltIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OltIndication.Merge(m, src)
+}
+func (m *OltIndication) XXX_Size() int {
+	return xxx_messageInfo_OltIndication.Size(m)
+}
+func (m *OltIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OltIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OltIndication proto.InternalMessageInfo
+
+func (m *OltIndication) GetOperState() string {
+	if m != nil {
+		return m.OperState
+	}
+	return ""
+}
+
+type IntfIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OperState            string   `protobuf:"bytes,2,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *IntfIndication) Reset()         { *m = IntfIndication{} }
+func (m *IntfIndication) String() string { return proto.CompactTextString(m) }
+func (*IntfIndication) ProtoMessage()    {}
+func (*IntfIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{3}
+}
+
+func (m *IntfIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_IntfIndication.Unmarshal(m, b)
+}
+func (m *IntfIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_IntfIndication.Marshal(b, m, deterministic)
+}
+func (m *IntfIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IntfIndication.Merge(m, src)
+}
+func (m *IntfIndication) XXX_Size() int {
+	return xxx_messageInfo_IntfIndication.Size(m)
+}
+func (m *IntfIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_IntfIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_IntfIndication proto.InternalMessageInfo
+
+func (m *IntfIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *IntfIndication) GetOperState() string {
+	if m != nil {
+		return m.OperState
+	}
+	return ""
+}
+
+type OnuDiscIndication struct {
+	IntfId               uint32        `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	SerialNumber         *SerialNumber `protobuf:"bytes,2,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *OnuDiscIndication) Reset()         { *m = OnuDiscIndication{} }
+func (m *OnuDiscIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuDiscIndication) ProtoMessage()    {}
+func (*OnuDiscIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{4}
+}
+
+func (m *OnuDiscIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuDiscIndication.Unmarshal(m, b)
+}
+func (m *OnuDiscIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuDiscIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuDiscIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuDiscIndication.Merge(m, src)
+}
+func (m *OnuDiscIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuDiscIndication.Size(m)
+}
+func (m *OnuDiscIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuDiscIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuDiscIndication proto.InternalMessageInfo
+
+func (m *OnuDiscIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuDiscIndication) GetSerialNumber() *SerialNumber {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return nil
+}
+
+type OnuIndication struct {
+	IntfId               uint32                             `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32                             `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	OperState            string                             `protobuf:"bytes,3,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
+	AdminState           string                             `protobuf:"bytes,5,opt,name=admin_state,json=adminState,proto3" json:"admin_state,omitempty"`
+	SerialNumber         *SerialNumber                      `protobuf:"bytes,4,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	FailReason           OnuIndication_ActivationFailReason `protobuf:"varint,6,opt,name=fail_reason,json=failReason,proto3,enum=openolt.OnuIndication_ActivationFailReason" json:"fail_reason,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
+	XXX_unrecognized     []byte                             `json:"-"`
+	XXX_sizecache        int32                              `json:"-"`
+}
+
+func (m *OnuIndication) Reset()         { *m = OnuIndication{} }
+func (m *OnuIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuIndication) ProtoMessage()    {}
+func (*OnuIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{5}
+}
+
+func (m *OnuIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuIndication.Unmarshal(m, b)
+}
+func (m *OnuIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuIndication.Merge(m, src)
+}
+func (m *OnuIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuIndication.Size(m)
+}
+func (m *OnuIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuIndication proto.InternalMessageInfo
+
+func (m *OnuIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuIndication) GetOperState() string {
+	if m != nil {
+		return m.OperState
+	}
+	return ""
+}
+
+func (m *OnuIndication) GetAdminState() string {
+	if m != nil {
+		return m.AdminState
+	}
+	return ""
+}
+
+func (m *OnuIndication) GetSerialNumber() *SerialNumber {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return nil
+}
+
+func (m *OnuIndication) GetFailReason() OnuIndication_ActivationFailReason {
+	if m != nil {
+		return m.FailReason
+	}
+	return OnuIndication_ONU_ACTIVATION_FAIL_REASON_NONE
+}
+
+type IntfOperIndication struct {
+	Type                 string   `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
+	IntfId               uint32   `protobuf:"fixed32,2,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OperState            string   `protobuf:"bytes,3,opt,name=oper_state,json=operState,proto3" json:"oper_state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *IntfOperIndication) Reset()         { *m = IntfOperIndication{} }
+func (m *IntfOperIndication) String() string { return proto.CompactTextString(m) }
+func (*IntfOperIndication) ProtoMessage()    {}
+func (*IntfOperIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{6}
+}
+
+func (m *IntfOperIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_IntfOperIndication.Unmarshal(m, b)
+}
+func (m *IntfOperIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_IntfOperIndication.Marshal(b, m, deterministic)
+}
+func (m *IntfOperIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IntfOperIndication.Merge(m, src)
+}
+func (m *IntfOperIndication) XXX_Size() int {
+	return xxx_messageInfo_IntfOperIndication.Size(m)
+}
+func (m *IntfOperIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_IntfOperIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_IntfOperIndication proto.InternalMessageInfo
+
+func (m *IntfOperIndication) GetType() string {
+	if m != nil {
+		return m.Type
+	}
+	return ""
+}
+
+func (m *IntfOperIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *IntfOperIndication) GetOperState() string {
+	if m != nil {
+		return m.OperState
+	}
+	return ""
+}
+
+type OmciIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OmciIndication) Reset()         { *m = OmciIndication{} }
+func (m *OmciIndication) String() string { return proto.CompactTextString(m) }
+func (*OmciIndication) ProtoMessage()    {}
+func (*OmciIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{7}
+}
+
+func (m *OmciIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OmciIndication.Unmarshal(m, b)
+}
+func (m *OmciIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OmciIndication.Marshal(b, m, deterministic)
+}
+func (m *OmciIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OmciIndication.Merge(m, src)
+}
+func (m *OmciIndication) XXX_Size() int {
+	return xxx_messageInfo_OmciIndication.Size(m)
+}
+func (m *OmciIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OmciIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OmciIndication proto.InternalMessageInfo
+
+func (m *OmciIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OmciIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OmciIndication) GetPkt() []byte {
+	if m != nil {
+		return m.Pkt
+	}
+	return nil
+}
+
+type PacketIndication struct {
+	IntfType             string   `protobuf:"bytes,5,opt,name=intf_type,json=intfType,proto3" json:"intf_type,omitempty"`
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,8,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	UniId                uint32   `protobuf:"fixed32,9,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	GemportId            uint32   `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	FlowId               uint32   `protobuf:"fixed32,3,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
+	PortNo               uint32   `protobuf:"fixed32,6,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	Cookie               uint64   `protobuf:"fixed64,7,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	Pkt                  []byte   `protobuf:"bytes,4,opt,name=pkt,proto3" json:"pkt,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *PacketIndication) Reset()         { *m = PacketIndication{} }
+func (m *PacketIndication) String() string { return proto.CompactTextString(m) }
+func (*PacketIndication) ProtoMessage()    {}
+func (*PacketIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{8}
+}
+
+func (m *PacketIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PacketIndication.Unmarshal(m, b)
+}
+func (m *PacketIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PacketIndication.Marshal(b, m, deterministic)
+}
+func (m *PacketIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PacketIndication.Merge(m, src)
+}
+func (m *PacketIndication) XXX_Size() int {
+	return xxx_messageInfo_PacketIndication.Size(m)
+}
+func (m *PacketIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_PacketIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PacketIndication proto.InternalMessageInfo
+
+func (m *PacketIndication) GetIntfType() string {
+	if m != nil {
+		return m.IntfType
+	}
+	return ""
+}
+
+func (m *PacketIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetFlowId() uint32 {
+	if m != nil {
+		return m.FlowId
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *PacketIndication) GetPkt() []byte {
+	if m != nil {
+		return m.Pkt
+	}
+	return nil
+}
+
+type Interface struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Interface) Reset()         { *m = Interface{} }
+func (m *Interface) String() string { return proto.CompactTextString(m) }
+func (*Interface) ProtoMessage()    {}
+func (*Interface) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{9}
+}
+
+func (m *Interface) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Interface.Unmarshal(m, b)
+}
+func (m *Interface) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Interface.Marshal(b, m, deterministic)
+}
+func (m *Interface) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Interface.Merge(m, src)
+}
+func (m *Interface) XXX_Size() int {
+	return xxx_messageInfo_Interface.Size(m)
+}
+func (m *Interface) XXX_DiscardUnknown() {
+	xxx_messageInfo_Interface.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Interface proto.InternalMessageInfo
+
+func (m *Interface) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+type Heartbeat struct {
+	HeartbeatSignature   uint32   `protobuf:"fixed32,1,opt,name=heartbeat_signature,json=heartbeatSignature,proto3" json:"heartbeat_signature,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Heartbeat) Reset()         { *m = Heartbeat{} }
+func (m *Heartbeat) String() string { return proto.CompactTextString(m) }
+func (*Heartbeat) ProtoMessage()    {}
+func (*Heartbeat) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{10}
+}
+
+func (m *Heartbeat) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Heartbeat.Unmarshal(m, b)
+}
+func (m *Heartbeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Heartbeat.Marshal(b, m, deterministic)
+}
+func (m *Heartbeat) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Heartbeat.Merge(m, src)
+}
+func (m *Heartbeat) XXX_Size() int {
+	return xxx_messageInfo_Heartbeat.Size(m)
+}
+func (m *Heartbeat) XXX_DiscardUnknown() {
+	xxx_messageInfo_Heartbeat.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Heartbeat proto.InternalMessageInfo
+
+func (m *Heartbeat) GetHeartbeatSignature() uint32 {
+	if m != nil {
+		return m.HeartbeatSignature
+	}
+	return 0
+}
+
+type Onu struct {
+	IntfId               uint32        `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32        `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	SerialNumber         *SerialNumber `protobuf:"bytes,3,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	Pir                  uint32        `protobuf:"fixed32,4,opt,name=pir,proto3" json:"pir,omitempty"`
+	OmccEncryption       bool          `protobuf:"varint,5,opt,name=omcc_encryption,json=omccEncryption,proto3" json:"omcc_encryption,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *Onu) Reset()         { *m = Onu{} }
+func (m *Onu) String() string { return proto.CompactTextString(m) }
+func (*Onu) ProtoMessage()    {}
+func (*Onu) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{11}
+}
+
+func (m *Onu) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Onu.Unmarshal(m, b)
+}
+func (m *Onu) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Onu.Marshal(b, m, deterministic)
+}
+func (m *Onu) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Onu.Merge(m, src)
+}
+func (m *Onu) XXX_Size() int {
+	return xxx_messageInfo_Onu.Size(m)
+}
+func (m *Onu) XXX_DiscardUnknown() {
+	xxx_messageInfo_Onu.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Onu proto.InternalMessageInfo
+
+func (m *Onu) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *Onu) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *Onu) GetSerialNumber() *SerialNumber {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return nil
+}
+
+func (m *Onu) GetPir() uint32 {
+	if m != nil {
+		return m.Pir
+	}
+	return 0
+}
+
+func (m *Onu) GetOmccEncryption() bool {
+	if m != nil {
+		return m.OmccEncryption
+	}
+	return false
+}
+
+type OnuLogicalDistance struct {
+	IntfId                 uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                  uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	LogicalOnuDistanceZero uint32   `protobuf:"fixed32,3,opt,name=logical_onu_distance_zero,json=logicalOnuDistanceZero,proto3" json:"logical_onu_distance_zero,omitempty"`
+	LogicalOnuDistance     uint32   `protobuf:"fixed32,4,opt,name=logical_onu_distance,json=logicalOnuDistance,proto3" json:"logical_onu_distance,omitempty"`
+	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
+	XXX_unrecognized       []byte   `json:"-"`
+	XXX_sizecache          int32    `json:"-"`
+}
+
+func (m *OnuLogicalDistance) Reset()         { *m = OnuLogicalDistance{} }
+func (m *OnuLogicalDistance) String() string { return proto.CompactTextString(m) }
+func (*OnuLogicalDistance) ProtoMessage()    {}
+func (*OnuLogicalDistance) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{12}
+}
+
+func (m *OnuLogicalDistance) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuLogicalDistance.Unmarshal(m, b)
+}
+func (m *OnuLogicalDistance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuLogicalDistance.Marshal(b, m, deterministic)
+}
+func (m *OnuLogicalDistance) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuLogicalDistance.Merge(m, src)
+}
+func (m *OnuLogicalDistance) XXX_Size() int {
+	return xxx_messageInfo_OnuLogicalDistance.Size(m)
+}
+func (m *OnuLogicalDistance) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuLogicalDistance.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuLogicalDistance proto.InternalMessageInfo
+
+func (m *OnuLogicalDistance) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuLogicalDistance) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuLogicalDistance) GetLogicalOnuDistanceZero() uint32 {
+	if m != nil {
+		return m.LogicalOnuDistanceZero
+	}
+	return 0
+}
+
+func (m *OnuLogicalDistance) GetLogicalOnuDistance() uint32 {
+	if m != nil {
+		return m.LogicalOnuDistance
+	}
+	return 0
+}
+
+type OmciMsg struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OmciMsg) Reset()         { *m = OmciMsg{} }
+func (m *OmciMsg) String() string { return proto.CompactTextString(m) }
+func (*OmciMsg) ProtoMessage()    {}
+func (*OmciMsg) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{13}
+}
+
+func (m *OmciMsg) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OmciMsg.Unmarshal(m, b)
+}
+func (m *OmciMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OmciMsg.Marshal(b, m, deterministic)
+}
+func (m *OmciMsg) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OmciMsg.Merge(m, src)
+}
+func (m *OmciMsg) XXX_Size() int {
+	return xxx_messageInfo_OmciMsg.Size(m)
+}
+func (m *OmciMsg) XXX_DiscardUnknown() {
+	xxx_messageInfo_OmciMsg.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OmciMsg proto.InternalMessageInfo
+
+func (m *OmciMsg) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OmciMsg) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OmciMsg) GetPkt() []byte {
+	if m != nil {
+		return m.Pkt
+	}
+	return nil
+}
+
+type OnuPacket struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	PortNo               uint32   `protobuf:"fixed32,4,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	GemportId            uint32   `protobuf:"fixed32,5,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	Pkt                  []byte   `protobuf:"bytes,3,opt,name=pkt,proto3" json:"pkt,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuPacket) Reset()         { *m = OnuPacket{} }
+func (m *OnuPacket) String() string { return proto.CompactTextString(m) }
+func (*OnuPacket) ProtoMessage()    {}
+func (*OnuPacket) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{14}
+}
+
+func (m *OnuPacket) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuPacket.Unmarshal(m, b)
+}
+func (m *OnuPacket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuPacket.Marshal(b, m, deterministic)
+}
+func (m *OnuPacket) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuPacket.Merge(m, src)
+}
+func (m *OnuPacket) XXX_Size() int {
+	return xxx_messageInfo_OnuPacket.Size(m)
+}
+func (m *OnuPacket) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuPacket.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuPacket proto.InternalMessageInfo
+
+func (m *OnuPacket) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuPacket) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuPacket) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *OnuPacket) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *OnuPacket) GetPkt() []byte {
+	if m != nil {
+		return m.Pkt
+	}
+	return nil
+}
+
+type UplinkPacket struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	Pkt                  []byte   `protobuf:"bytes,2,opt,name=pkt,proto3" json:"pkt,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *UplinkPacket) Reset()         { *m = UplinkPacket{} }
+func (m *UplinkPacket) String() string { return proto.CompactTextString(m) }
+func (*UplinkPacket) ProtoMessage()    {}
+func (*UplinkPacket) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{15}
+}
+
+func (m *UplinkPacket) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_UplinkPacket.Unmarshal(m, b)
+}
+func (m *UplinkPacket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_UplinkPacket.Marshal(b, m, deterministic)
+}
+func (m *UplinkPacket) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_UplinkPacket.Merge(m, src)
+}
+func (m *UplinkPacket) XXX_Size() int {
+	return xxx_messageInfo_UplinkPacket.Size(m)
+}
+func (m *UplinkPacket) XXX_DiscardUnknown() {
+	xxx_messageInfo_UplinkPacket.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_UplinkPacket proto.InternalMessageInfo
+
+func (m *UplinkPacket) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *UplinkPacket) GetPkt() []byte {
+	if m != nil {
+		return m.Pkt
+	}
+	return nil
+}
+
+type DeviceInfo struct {
+	Vendor              string `protobuf:"bytes,1,opt,name=vendor,proto3" json:"vendor,omitempty"`
+	Model               string `protobuf:"bytes,2,opt,name=model,proto3" json:"model,omitempty"`
+	HardwareVersion     string `protobuf:"bytes,3,opt,name=hardware_version,json=hardwareVersion,proto3" json:"hardware_version,omitempty"`
+	FirmwareVersion     string `protobuf:"bytes,4,opt,name=firmware_version,json=firmwareVersion,proto3" json:"firmware_version,omitempty"`
+	DeviceId            string `protobuf:"bytes,16,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	DeviceSerialNumber  string `protobuf:"bytes,17,opt,name=device_serial_number,json=deviceSerialNumber,proto3" json:"device_serial_number,omitempty"`
+	PreviouslyConnected bool   `protobuf:"varint,19,opt,name=previously_connected,json=previouslyConnected,proto3" json:"previously_connected,omitempty"`
+	// Total number of pon intf ports on the device
+	PonPorts uint32 `protobuf:"fixed32,12,opt,name=pon_ports,json=ponPorts,proto3" json:"pon_ports,omitempty"`
+	// If using global per-device technology profile. To be deprecated
+	Technology           string                             `protobuf:"bytes,5,opt,name=technology,proto3" json:"technology,omitempty"`
+	OnuIdStart           uint32                             `protobuf:"fixed32,6,opt,name=onu_id_start,json=onuIdStart,proto3" json:"onu_id_start,omitempty"`
+	OnuIdEnd             uint32                             `protobuf:"fixed32,7,opt,name=onu_id_end,json=onuIdEnd,proto3" json:"onu_id_end,omitempty"`
+	AllocIdStart         uint32                             `protobuf:"fixed32,8,opt,name=alloc_id_start,json=allocIdStart,proto3" json:"alloc_id_start,omitempty"`
+	AllocIdEnd           uint32                             `protobuf:"fixed32,9,opt,name=alloc_id_end,json=allocIdEnd,proto3" json:"alloc_id_end,omitempty"`
+	GemportIdStart       uint32                             `protobuf:"fixed32,10,opt,name=gemport_id_start,json=gemportIdStart,proto3" json:"gemport_id_start,omitempty"`
+	GemportIdEnd         uint32                             `protobuf:"fixed32,11,opt,name=gemport_id_end,json=gemportIdEnd,proto3" json:"gemport_id_end,omitempty"`
+	FlowIdStart          uint32                             `protobuf:"fixed32,13,opt,name=flow_id_start,json=flowIdStart,proto3" json:"flow_id_start,omitempty"`
+	FlowIdEnd            uint32                             `protobuf:"fixed32,14,opt,name=flow_id_end,json=flowIdEnd,proto3" json:"flow_id_end,omitempty"`
+	Ranges               []*DeviceInfo_DeviceResourceRanges `protobuf:"bytes,15,rep,name=ranges,proto3" json:"ranges,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
+	XXX_unrecognized     []byte                             `json:"-"`
+	XXX_sizecache        int32                              `json:"-"`
+}
+
+func (m *DeviceInfo) Reset()         { *m = DeviceInfo{} }
+func (m *DeviceInfo) String() string { return proto.CompactTextString(m) }
+func (*DeviceInfo) ProtoMessage()    {}
+func (*DeviceInfo) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{16}
+}
+
+func (m *DeviceInfo) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceInfo.Unmarshal(m, b)
+}
+func (m *DeviceInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceInfo.Marshal(b, m, deterministic)
+}
+func (m *DeviceInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceInfo.Merge(m, src)
+}
+func (m *DeviceInfo) XXX_Size() int {
+	return xxx_messageInfo_DeviceInfo.Size(m)
+}
+func (m *DeviceInfo) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceInfo.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceInfo proto.InternalMessageInfo
+
+func (m *DeviceInfo) GetVendor() string {
+	if m != nil {
+		return m.Vendor
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetModel() string {
+	if m != nil {
+		return m.Model
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetHardwareVersion() string {
+	if m != nil {
+		return m.HardwareVersion
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetFirmwareVersion() string {
+	if m != nil {
+		return m.FirmwareVersion
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetDeviceSerialNumber() string {
+	if m != nil {
+		return m.DeviceSerialNumber
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetPreviouslyConnected() bool {
+	if m != nil {
+		return m.PreviouslyConnected
+	}
+	return false
+}
+
+func (m *DeviceInfo) GetPonPorts() uint32 {
+	if m != nil {
+		return m.PonPorts
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetTechnology() string {
+	if m != nil {
+		return m.Technology
+	}
+	return ""
+}
+
+func (m *DeviceInfo) GetOnuIdStart() uint32 {
+	if m != nil {
+		return m.OnuIdStart
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetOnuIdEnd() uint32 {
+	if m != nil {
+		return m.OnuIdEnd
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetAllocIdStart() uint32 {
+	if m != nil {
+		return m.AllocIdStart
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetAllocIdEnd() uint32 {
+	if m != nil {
+		return m.AllocIdEnd
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetGemportIdStart() uint32 {
+	if m != nil {
+		return m.GemportIdStart
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetGemportIdEnd() uint32 {
+	if m != nil {
+		return m.GemportIdEnd
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetFlowIdStart() uint32 {
+	if m != nil {
+		return m.FlowIdStart
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetFlowIdEnd() uint32 {
+	if m != nil {
+		return m.FlowIdEnd
+	}
+	return 0
+}
+
+func (m *DeviceInfo) GetRanges() []*DeviceInfo_DeviceResourceRanges {
+	if m != nil {
+		return m.Ranges
+	}
+	return nil
+}
+
+type DeviceInfo_DeviceResourceRanges struct {
+	// List of 0 or more intf_ids that use the same technology and pools.
+	// If 0 intf_ids supplied, it implies ALL interfaces
+	IntfIds []uint32 `protobuf:"fixed32,1,rep,packed,name=intf_ids,json=intfIds,proto3" json:"intf_ids,omitempty"`
+	// Technology profile for this pool
+	Technology           string                                  `protobuf:"bytes,2,opt,name=technology,proto3" json:"technology,omitempty"`
+	Pools                []*DeviceInfo_DeviceResourceRanges_Pool `protobuf:"bytes,3,rep,name=pools,proto3" json:"pools,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                `json:"-"`
+	XXX_unrecognized     []byte                                  `json:"-"`
+	XXX_sizecache        int32                                   `json:"-"`
+}
+
+func (m *DeviceInfo_DeviceResourceRanges) Reset()         { *m = DeviceInfo_DeviceResourceRanges{} }
+func (m *DeviceInfo_DeviceResourceRanges) String() string { return proto.CompactTextString(m) }
+func (*DeviceInfo_DeviceResourceRanges) ProtoMessage()    {}
+func (*DeviceInfo_DeviceResourceRanges) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0}
+}
+
+func (m *DeviceInfo_DeviceResourceRanges) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Unmarshal(m, b)
+}
+func (m *DeviceInfo_DeviceResourceRanges) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Marshal(b, m, deterministic)
+}
+func (m *DeviceInfo_DeviceResourceRanges) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Merge(m, src)
+}
+func (m *DeviceInfo_DeviceResourceRanges) XXX_Size() int {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges.Size(m)
+}
+func (m *DeviceInfo_DeviceResourceRanges) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceInfo_DeviceResourceRanges.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceInfo_DeviceResourceRanges proto.InternalMessageInfo
+
+func (m *DeviceInfo_DeviceResourceRanges) GetIntfIds() []uint32 {
+	if m != nil {
+		return m.IntfIds
+	}
+	return nil
+}
+
+func (m *DeviceInfo_DeviceResourceRanges) GetTechnology() string {
+	if m != nil {
+		return m.Technology
+	}
+	return ""
+}
+
+func (m *DeviceInfo_DeviceResourceRanges) GetPools() []*DeviceInfo_DeviceResourceRanges_Pool {
+	if m != nil {
+		return m.Pools
+	}
+	return nil
+}
+
+type DeviceInfo_DeviceResourceRanges_Pool struct {
+	Type                 DeviceInfo_DeviceResourceRanges_Pool_PoolType    `protobuf:"varint,1,opt,name=type,proto3,enum=openolt.DeviceInfo_DeviceResourceRanges_Pool_PoolType" json:"type,omitempty"`
+	Sharing              DeviceInfo_DeviceResourceRanges_Pool_SharingType `protobuf:"varint,2,opt,name=sharing,proto3,enum=openolt.DeviceInfo_DeviceResourceRanges_Pool_SharingType" json:"sharing,omitempty"`
+	Start                uint32                                           `protobuf:"fixed32,3,opt,name=start,proto3" json:"start,omitempty"`
+	End                  uint32                                           `protobuf:"fixed32,4,opt,name=end,proto3" json:"end,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                         `json:"-"`
+	XXX_unrecognized     []byte                                           `json:"-"`
+	XXX_sizecache        int32                                            `json:"-"`
+}
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) Reset()         { *m = DeviceInfo_DeviceResourceRanges_Pool{} }
+func (m *DeviceInfo_DeviceResourceRanges_Pool) String() string { return proto.CompactTextString(m) }
+func (*DeviceInfo_DeviceResourceRanges_Pool) ProtoMessage()    {}
+func (*DeviceInfo_DeviceResourceRanges_Pool) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{16, 0, 0}
+}
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Unmarshal(m, b)
+}
+func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Marshal(b, m, deterministic)
+}
+func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Merge(m, src)
+}
+func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_Size() int {
+	return xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.Size(m)
+}
+func (m *DeviceInfo_DeviceResourceRanges_Pool) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceInfo_DeviceResourceRanges_Pool proto.InternalMessageInfo
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) GetType() DeviceInfo_DeviceResourceRanges_Pool_PoolType {
+	if m != nil {
+		return m.Type
+	}
+	return DeviceInfo_DeviceResourceRanges_Pool_ONU_ID
+}
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) GetSharing() DeviceInfo_DeviceResourceRanges_Pool_SharingType {
+	if m != nil {
+		return m.Sharing
+	}
+	return DeviceInfo_DeviceResourceRanges_Pool_DEDICATED_PER_INTF
+}
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) GetStart() uint32 {
+	if m != nil {
+		return m.Start
+	}
+	return 0
+}
+
+func (m *DeviceInfo_DeviceResourceRanges_Pool) GetEnd() uint32 {
+	if m != nil {
+		return m.End
+	}
+	return 0
+}
+
+type Classifier struct {
+	OTpid                uint32   `protobuf:"fixed32,1,opt,name=o_tpid,json=oTpid,proto3" json:"o_tpid,omitempty"`
+	OVid                 uint32   `protobuf:"fixed32,2,opt,name=o_vid,json=oVid,proto3" json:"o_vid,omitempty"`
+	ITpid                uint32   `protobuf:"fixed32,3,opt,name=i_tpid,json=iTpid,proto3" json:"i_tpid,omitempty"`
+	IVid                 uint32   `protobuf:"fixed32,4,opt,name=i_vid,json=iVid,proto3" json:"i_vid,omitempty"`
+	OPbits               uint32   `protobuf:"fixed32,5,opt,name=o_pbits,json=oPbits,proto3" json:"o_pbits,omitempty"`
+	IPbits               uint32   `protobuf:"fixed32,6,opt,name=i_pbits,json=iPbits,proto3" json:"i_pbits,omitempty"`
+	EthType              uint32   `protobuf:"fixed32,7,opt,name=eth_type,json=ethType,proto3" json:"eth_type,omitempty"`
+	DstMac               []byte   `protobuf:"bytes,8,opt,name=dst_mac,json=dstMac,proto3" json:"dst_mac,omitempty"`
+	SrcMac               []byte   `protobuf:"bytes,9,opt,name=src_mac,json=srcMac,proto3" json:"src_mac,omitempty"`
+	IpProto              uint32   `protobuf:"fixed32,10,opt,name=ip_proto,json=ipProto,proto3" json:"ip_proto,omitempty"`
+	DstIp                uint32   `protobuf:"fixed32,11,opt,name=dst_ip,json=dstIp,proto3" json:"dst_ip,omitempty"`
+	SrcIp                uint32   `protobuf:"fixed32,12,opt,name=src_ip,json=srcIp,proto3" json:"src_ip,omitempty"`
+	SrcPort              uint32   `protobuf:"fixed32,13,opt,name=src_port,json=srcPort,proto3" json:"src_port,omitempty"`
+	DstPort              uint32   `protobuf:"fixed32,14,opt,name=dst_port,json=dstPort,proto3" json:"dst_port,omitempty"`
+	PktTagType           string   `protobuf:"bytes,15,opt,name=pkt_tag_type,json=pktTagType,proto3" json:"pkt_tag_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Classifier) Reset()         { *m = Classifier{} }
+func (m *Classifier) String() string { return proto.CompactTextString(m) }
+func (*Classifier) ProtoMessage()    {}
+func (*Classifier) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{17}
+}
+
+func (m *Classifier) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Classifier.Unmarshal(m, b)
+}
+func (m *Classifier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Classifier.Marshal(b, m, deterministic)
+}
+func (m *Classifier) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Classifier.Merge(m, src)
+}
+func (m *Classifier) XXX_Size() int {
+	return xxx_messageInfo_Classifier.Size(m)
+}
+func (m *Classifier) XXX_DiscardUnknown() {
+	xxx_messageInfo_Classifier.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Classifier proto.InternalMessageInfo
+
+func (m *Classifier) GetOTpid() uint32 {
+	if m != nil {
+		return m.OTpid
+	}
+	return 0
+}
+
+func (m *Classifier) GetOVid() uint32 {
+	if m != nil {
+		return m.OVid
+	}
+	return 0
+}
+
+func (m *Classifier) GetITpid() uint32 {
+	if m != nil {
+		return m.ITpid
+	}
+	return 0
+}
+
+func (m *Classifier) GetIVid() uint32 {
+	if m != nil {
+		return m.IVid
+	}
+	return 0
+}
+
+func (m *Classifier) GetOPbits() uint32 {
+	if m != nil {
+		return m.OPbits
+	}
+	return 0
+}
+
+func (m *Classifier) GetIPbits() uint32 {
+	if m != nil {
+		return m.IPbits
+	}
+	return 0
+}
+
+func (m *Classifier) GetEthType() uint32 {
+	if m != nil {
+		return m.EthType
+	}
+	return 0
+}
+
+func (m *Classifier) GetDstMac() []byte {
+	if m != nil {
+		return m.DstMac
+	}
+	return nil
+}
+
+func (m *Classifier) GetSrcMac() []byte {
+	if m != nil {
+		return m.SrcMac
+	}
+	return nil
+}
+
+func (m *Classifier) GetIpProto() uint32 {
+	if m != nil {
+		return m.IpProto
+	}
+	return 0
+}
+
+func (m *Classifier) GetDstIp() uint32 {
+	if m != nil {
+		return m.DstIp
+	}
+	return 0
+}
+
+func (m *Classifier) GetSrcIp() uint32 {
+	if m != nil {
+		return m.SrcIp
+	}
+	return 0
+}
+
+func (m *Classifier) GetSrcPort() uint32 {
+	if m != nil {
+		return m.SrcPort
+	}
+	return 0
+}
+
+func (m *Classifier) GetDstPort() uint32 {
+	if m != nil {
+		return m.DstPort
+	}
+	return 0
+}
+
+func (m *Classifier) GetPktTagType() string {
+	if m != nil {
+		return m.PktTagType
+	}
+	return ""
+}
+
+type ActionCmd struct {
+	AddOuterTag          bool     `protobuf:"varint,1,opt,name=add_outer_tag,json=addOuterTag,proto3" json:"add_outer_tag,omitempty"`
+	RemoveOuterTag       bool     `protobuf:"varint,2,opt,name=remove_outer_tag,json=removeOuterTag,proto3" json:"remove_outer_tag,omitempty"`
+	TrapToHost           bool     `protobuf:"varint,3,opt,name=trap_to_host,json=trapToHost,proto3" json:"trap_to_host,omitempty"`
+	RemarkOuterPbits     bool     `protobuf:"varint,4,opt,name=remark_outer_pbits,json=remarkOuterPbits,proto3" json:"remark_outer_pbits,omitempty"`
+	RemarkInnerPbits     bool     `protobuf:"varint,5,opt,name=remark_inner_pbits,json=remarkInnerPbits,proto3" json:"remark_inner_pbits,omitempty"`
+	AddInnerTag          bool     `protobuf:"varint,6,opt,name=add_inner_tag,json=addInnerTag,proto3" json:"add_inner_tag,omitempty"`
+	RemoveInnerTag       bool     `protobuf:"varint,7,opt,name=remove_inner_tag,json=removeInnerTag,proto3" json:"remove_inner_tag,omitempty"`
+	TranslateInnerTag    bool     `protobuf:"varint,8,opt,name=translate_inner_tag,json=translateInnerTag,proto3" json:"translate_inner_tag,omitempty"`
+	TranslateOuterTag    bool     `protobuf:"varint,9,opt,name=translate_outer_tag,json=translateOuterTag,proto3" json:"translate_outer_tag,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ActionCmd) Reset()         { *m = ActionCmd{} }
+func (m *ActionCmd) String() string { return proto.CompactTextString(m) }
+func (*ActionCmd) ProtoMessage()    {}
+func (*ActionCmd) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{18}
+}
+
+func (m *ActionCmd) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ActionCmd.Unmarshal(m, b)
+}
+func (m *ActionCmd) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ActionCmd.Marshal(b, m, deterministic)
+}
+func (m *ActionCmd) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ActionCmd.Merge(m, src)
+}
+func (m *ActionCmd) XXX_Size() int {
+	return xxx_messageInfo_ActionCmd.Size(m)
+}
+func (m *ActionCmd) XXX_DiscardUnknown() {
+	xxx_messageInfo_ActionCmd.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ActionCmd proto.InternalMessageInfo
+
+func (m *ActionCmd) GetAddOuterTag() bool {
+	if m != nil {
+		return m.AddOuterTag
+	}
+	return false
+}
+
+func (m *ActionCmd) GetRemoveOuterTag() bool {
+	if m != nil {
+		return m.RemoveOuterTag
+	}
+	return false
+}
+
+func (m *ActionCmd) GetTrapToHost() bool {
+	if m != nil {
+		return m.TrapToHost
+	}
+	return false
+}
+
+func (m *ActionCmd) GetRemarkOuterPbits() bool {
+	if m != nil {
+		return m.RemarkOuterPbits
+	}
+	return false
+}
+
+func (m *ActionCmd) GetRemarkInnerPbits() bool {
+	if m != nil {
+		return m.RemarkInnerPbits
+	}
+	return false
+}
+
+func (m *ActionCmd) GetAddInnerTag() bool {
+	if m != nil {
+		return m.AddInnerTag
+	}
+	return false
+}
+
+func (m *ActionCmd) GetRemoveInnerTag() bool {
+	if m != nil {
+		return m.RemoveInnerTag
+	}
+	return false
+}
+
+func (m *ActionCmd) GetTranslateInnerTag() bool {
+	if m != nil {
+		return m.TranslateInnerTag
+	}
+	return false
+}
+
+func (m *ActionCmd) GetTranslateOuterTag() bool {
+	if m != nil {
+		return m.TranslateOuterTag
+	}
+	return false
+}
+
+type Action struct {
+	Cmd                  *ActionCmd `protobuf:"bytes,1,opt,name=cmd,proto3" json:"cmd,omitempty"`
+	OVid                 uint32     `protobuf:"fixed32,2,opt,name=o_vid,json=oVid,proto3" json:"o_vid,omitempty"`
+	OPbits               uint32     `protobuf:"fixed32,3,opt,name=o_pbits,json=oPbits,proto3" json:"o_pbits,omitempty"`
+	OTpid                uint32     `protobuf:"fixed32,4,opt,name=o_tpid,json=oTpid,proto3" json:"o_tpid,omitempty"`
+	IVid                 uint32     `protobuf:"fixed32,5,opt,name=i_vid,json=iVid,proto3" json:"i_vid,omitempty"`
+	IPbits               uint32     `protobuf:"fixed32,6,opt,name=i_pbits,json=iPbits,proto3" json:"i_pbits,omitempty"`
+	ITpid                uint32     `protobuf:"fixed32,7,opt,name=i_tpid,json=iTpid,proto3" json:"i_tpid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
+	XXX_unrecognized     []byte     `json:"-"`
+	XXX_sizecache        int32      `json:"-"`
+}
+
+func (m *Action) Reset()         { *m = Action{} }
+func (m *Action) String() string { return proto.CompactTextString(m) }
+func (*Action) ProtoMessage()    {}
+func (*Action) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{19}
+}
+
+func (m *Action) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Action.Unmarshal(m, b)
+}
+func (m *Action) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Action.Marshal(b, m, deterministic)
+}
+func (m *Action) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Action.Merge(m, src)
+}
+func (m *Action) XXX_Size() int {
+	return xxx_messageInfo_Action.Size(m)
+}
+func (m *Action) XXX_DiscardUnknown() {
+	xxx_messageInfo_Action.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Action proto.InternalMessageInfo
+
+func (m *Action) GetCmd() *ActionCmd {
+	if m != nil {
+		return m.Cmd
+	}
+	return nil
+}
+
+func (m *Action) GetOVid() uint32 {
+	if m != nil {
+		return m.OVid
+	}
+	return 0
+}
+
+func (m *Action) GetOPbits() uint32 {
+	if m != nil {
+		return m.OPbits
+	}
+	return 0
+}
+
+func (m *Action) GetOTpid() uint32 {
+	if m != nil {
+		return m.OTpid
+	}
+	return 0
+}
+
+func (m *Action) GetIVid() uint32 {
+	if m != nil {
+		return m.IVid
+	}
+	return 0
+}
+
+func (m *Action) GetIPbits() uint32 {
+	if m != nil {
+		return m.IPbits
+	}
+	return 0
+}
+
+func (m *Action) GetITpid() uint32 {
+	if m != nil {
+		return m.ITpid
+	}
+	return 0
+}
+
+type Flow struct {
+	AccessIntfId         int32             `protobuf:"fixed32,1,opt,name=access_intf_id,json=accessIntfId,proto3" json:"access_intf_id,omitempty"`
+	OnuId                int32             `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	UniId                int32             `protobuf:"fixed32,11,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	FlowId               uint64            `protobuf:"fixed64,3,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
+	SymmetricFlowId      uint64            `protobuf:"fixed64,18,opt,name=symmetric_flow_id,json=symmetricFlowId,proto3" json:"symmetric_flow_id,omitempty"`
+	FlowType             string            `protobuf:"bytes,4,opt,name=flow_type,json=flowType,proto3" json:"flow_type,omitempty"`
+	AllocId              int32             `protobuf:"fixed32,10,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	NetworkIntfId        int32             `protobuf:"fixed32,5,opt,name=network_intf_id,json=networkIntfId,proto3" json:"network_intf_id,omitempty"`
+	GemportId            int32             `protobuf:"fixed32,6,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	Classifier           *Classifier       `protobuf:"bytes,7,opt,name=classifier,proto3" json:"classifier,omitempty"`
+	Action               *Action           `protobuf:"bytes,8,opt,name=action,proto3" json:"action,omitempty"`
+	Priority             int32             `protobuf:"fixed32,9,opt,name=priority,proto3" json:"priority,omitempty"`
+	Cookie               uint64            `protobuf:"fixed64,12,opt,name=cookie,proto3" json:"cookie,omitempty"`
+	PortNo               uint32            `protobuf:"fixed32,13,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	GroupId              uint32            `protobuf:"fixed32,14,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	TechProfileId        uint32            `protobuf:"fixed32,15,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
+	ReplicateFlow        bool              `protobuf:"varint,16,opt,name=replicate_flow,json=replicateFlow,proto3" json:"replicate_flow,omitempty"`
+	PbitToGemport        map[uint32]uint32 `protobuf:"bytes,17,rep,name=pbit_to_gemport,json=pbitToGemport,proto3" json:"pbit_to_gemport,omitempty" protobuf_key:"fixed32,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
+	GemportToAes         map[uint32]bool   `protobuf:"bytes,19,rep,name=gemport_to_aes,json=gemportToAes,proto3" json:"gemport_to_aes,omitempty" protobuf_key:"fixed32,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *Flow) Reset()         { *m = Flow{} }
+func (m *Flow) String() string { return proto.CompactTextString(m) }
+func (*Flow) ProtoMessage()    {}
+func (*Flow) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{20}
+}
+
+func (m *Flow) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Flow.Unmarshal(m, b)
+}
+func (m *Flow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Flow.Marshal(b, m, deterministic)
+}
+func (m *Flow) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Flow.Merge(m, src)
+}
+func (m *Flow) XXX_Size() int {
+	return xxx_messageInfo_Flow.Size(m)
+}
+func (m *Flow) XXX_DiscardUnknown() {
+	xxx_messageInfo_Flow.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Flow proto.InternalMessageInfo
+
+func (m *Flow) GetAccessIntfId() int32 {
+	if m != nil {
+		return m.AccessIntfId
+	}
+	return 0
+}
+
+func (m *Flow) GetOnuId() int32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *Flow) GetUniId() int32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *Flow) GetFlowId() uint64 {
+	if m != nil {
+		return m.FlowId
+	}
+	return 0
+}
+
+func (m *Flow) GetSymmetricFlowId() uint64 {
+	if m != nil {
+		return m.SymmetricFlowId
+	}
+	return 0
+}
+
+func (m *Flow) GetFlowType() string {
+	if m != nil {
+		return m.FlowType
+	}
+	return ""
+}
+
+func (m *Flow) GetAllocId() int32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+func (m *Flow) GetNetworkIntfId() int32 {
+	if m != nil {
+		return m.NetworkIntfId
+	}
+	return 0
+}
+
+func (m *Flow) GetGemportId() int32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *Flow) GetClassifier() *Classifier {
+	if m != nil {
+		return m.Classifier
+	}
+	return nil
+}
+
+func (m *Flow) GetAction() *Action {
+	if m != nil {
+		return m.Action
+	}
+	return nil
+}
+
+func (m *Flow) GetPriority() int32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *Flow) GetCookie() uint64 {
+	if m != nil {
+		return m.Cookie
+	}
+	return 0
+}
+
+func (m *Flow) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *Flow) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+func (m *Flow) GetTechProfileId() uint32 {
+	if m != nil {
+		return m.TechProfileId
+	}
+	return 0
+}
+
+func (m *Flow) GetReplicateFlow() bool {
+	if m != nil {
+		return m.ReplicateFlow
+	}
+	return false
+}
+
+func (m *Flow) GetPbitToGemport() map[uint32]uint32 {
+	if m != nil {
+		return m.PbitToGemport
+	}
+	return nil
+}
+
+func (m *Flow) GetGemportToAes() map[uint32]bool {
+	if m != nil {
+		return m.GemportToAes
+	}
+	return nil
+}
+
+type SerialNumber struct {
+	VendorId             []byte   `protobuf:"bytes,1,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
+	VendorSpecific       []byte   `protobuf:"bytes,2,opt,name=vendor_specific,json=vendorSpecific,proto3" json:"vendor_specific,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *SerialNumber) Reset()         { *m = SerialNumber{} }
+func (m *SerialNumber) String() string { return proto.CompactTextString(m) }
+func (*SerialNumber) ProtoMessage()    {}
+func (*SerialNumber) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{21}
+}
+
+func (m *SerialNumber) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SerialNumber.Unmarshal(m, b)
+}
+func (m *SerialNumber) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SerialNumber.Marshal(b, m, deterministic)
+}
+func (m *SerialNumber) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SerialNumber.Merge(m, src)
+}
+func (m *SerialNumber) XXX_Size() int {
+	return xxx_messageInfo_SerialNumber.Size(m)
+}
+func (m *SerialNumber) XXX_DiscardUnknown() {
+	xxx_messageInfo_SerialNumber.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SerialNumber proto.InternalMessageInfo
+
+func (m *SerialNumber) GetVendorId() []byte {
+	if m != nil {
+		return m.VendorId
+	}
+	return nil
+}
+
+func (m *SerialNumber) GetVendorSpecific() []byte {
+	if m != nil {
+		return m.VendorSpecific
+	}
+	return nil
+}
+
+type PortStatistics struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	RxBytes              uint64   `protobuf:"fixed64,2,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	RxUcastPackets       uint64   `protobuf:"fixed64,4,opt,name=rx_ucast_packets,json=rxUcastPackets,proto3" json:"rx_ucast_packets,omitempty"`
+	RxMcastPackets       uint64   `protobuf:"fixed64,5,opt,name=rx_mcast_packets,json=rxMcastPackets,proto3" json:"rx_mcast_packets,omitempty"`
+	RxBcastPackets       uint64   `protobuf:"fixed64,6,opt,name=rx_bcast_packets,json=rxBcastPackets,proto3" json:"rx_bcast_packets,omitempty"`
+	RxErrorPackets       uint64   `protobuf:"fixed64,7,opt,name=rx_error_packets,json=rxErrorPackets,proto3" json:"rx_error_packets,omitempty"`
+	RxFrames             uint64   `protobuf:"fixed64,17,opt,name=rx_frames,json=rxFrames,proto3" json:"rx_frames,omitempty"`
+	RxFrames_64          uint64   `protobuf:"fixed64,18,opt,name=rx_frames_64,json=rxFrames64,proto3" json:"rx_frames_64,omitempty"`
+	RxFrames_65_127      uint64   `protobuf:"fixed64,19,opt,name=rx_frames_65_127,json=rxFrames65127,proto3" json:"rx_frames_65_127,omitempty"`
+	RxFrames_128_255     uint64   `protobuf:"fixed64,20,opt,name=rx_frames_128_255,json=rxFrames128255,proto3" json:"rx_frames_128_255,omitempty"`
+	RxFrames_256_511     uint64   `protobuf:"fixed64,21,opt,name=rx_frames_256_511,json=rxFrames256511,proto3" json:"rx_frames_256_511,omitempty"`
+	RxFrames_512_1023    uint64   `protobuf:"fixed64,22,opt,name=rx_frames_512_1023,json=rxFrames5121023,proto3" json:"rx_frames_512_1023,omitempty"`
+	RxFrames_1024_1518   uint64   `protobuf:"fixed64,23,opt,name=rx_frames_1024_1518,json=rxFrames10241518,proto3" json:"rx_frames_1024_1518,omitempty"`
+	RxFrames_1519_2047   uint64   `protobuf:"fixed64,24,opt,name=rx_frames_1519_2047,json=rxFrames15192047,proto3" json:"rx_frames_1519_2047,omitempty"`
+	RxFrames_2048_4095   uint64   `protobuf:"fixed64,25,opt,name=rx_frames_2048_4095,json=rxFrames20484095,proto3" json:"rx_frames_2048_4095,omitempty"`
+	RxFrames_4096_9216   uint64   `protobuf:"fixed64,26,opt,name=rx_frames_4096_9216,json=rxFrames40969216,proto3" json:"rx_frames_4096_9216,omitempty"`
+	RxFrames_9217_16383  uint64   `protobuf:"fixed64,27,opt,name=rx_frames_9217_16383,json=rxFrames921716383,proto3" json:"rx_frames_9217_16383,omitempty"`
+	RxCrcErrors          uint64   `protobuf:"fixed64,14,opt,name=rx_crc_errors,json=rxCrcErrors,proto3" json:"rx_crc_errors,omitempty"`
+	TxBytes              uint64   `protobuf:"fixed64,8,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	TxPackets            uint64   `protobuf:"fixed64,9,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	TxUcastPackets       uint64   `protobuf:"fixed64,10,opt,name=tx_ucast_packets,json=txUcastPackets,proto3" json:"tx_ucast_packets,omitempty"`
+	TxMcastPackets       uint64   `protobuf:"fixed64,11,opt,name=tx_mcast_packets,json=txMcastPackets,proto3" json:"tx_mcast_packets,omitempty"`
+	TxBcastPackets       uint64   `protobuf:"fixed64,12,opt,name=tx_bcast_packets,json=txBcastPackets,proto3" json:"tx_bcast_packets,omitempty"`
+	TxErrorPackets       uint64   `protobuf:"fixed64,13,opt,name=tx_error_packets,json=txErrorPackets,proto3" json:"tx_error_packets,omitempty"`
+	TxFrames             uint64   `protobuf:"fixed64,28,opt,name=tx_frames,json=txFrames,proto3" json:"tx_frames,omitempty"`
+	TxFrames_64          uint64   `protobuf:"fixed64,29,opt,name=tx_frames_64,json=txFrames64,proto3" json:"tx_frames_64,omitempty"`
+	TxFrames_65_127      uint64   `protobuf:"fixed64,30,opt,name=tx_frames_65_127,json=txFrames65127,proto3" json:"tx_frames_65_127,omitempty"`
+	TxFrames_128_255     uint64   `protobuf:"fixed64,31,opt,name=tx_frames_128_255,json=txFrames128255,proto3" json:"tx_frames_128_255,omitempty"`
+	TxFrames_256_511     uint64   `protobuf:"fixed64,32,opt,name=tx_frames_256_511,json=txFrames256511,proto3" json:"tx_frames_256_511,omitempty"`
+	TxFrames_512_1023    uint64   `protobuf:"fixed64,33,opt,name=tx_frames_512_1023,json=txFrames5121023,proto3" json:"tx_frames_512_1023,omitempty"`
+	TxFrames_1024_1518   uint64   `protobuf:"fixed64,34,opt,name=tx_frames_1024_1518,json=txFrames10241518,proto3" json:"tx_frames_1024_1518,omitempty"`
+	TxFrames_1519_2047   uint64   `protobuf:"fixed64,35,opt,name=tx_frames_1519_2047,json=txFrames15192047,proto3" json:"tx_frames_1519_2047,omitempty"`
+	TxFrames_2048_4095   uint64   `protobuf:"fixed64,36,opt,name=tx_frames_2048_4095,json=txFrames20484095,proto3" json:"tx_frames_2048_4095,omitempty"`
+	TxFrames_4096_9216   uint64   `protobuf:"fixed64,37,opt,name=tx_frames_4096_9216,json=txFrames40969216,proto3" json:"tx_frames_4096_9216,omitempty"`
+	TxFrames_9217_16383  uint64   `protobuf:"fixed64,38,opt,name=tx_frames_9217_16383,json=txFrames921716383,proto3" json:"tx_frames_9217_16383,omitempty"`
+	BipErrors            uint64   `protobuf:"fixed64,15,opt,name=bip_errors,json=bipErrors,proto3" json:"bip_errors,omitempty"`
+	Timestamp            uint32   `protobuf:"fixed32,16,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *PortStatistics) Reset()         { *m = PortStatistics{} }
+func (m *PortStatistics) String() string { return proto.CompactTextString(m) }
+func (*PortStatistics) ProtoMessage()    {}
+func (*PortStatistics) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{22}
+}
+
+func (m *PortStatistics) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PortStatistics.Unmarshal(m, b)
+}
+func (m *PortStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PortStatistics.Marshal(b, m, deterministic)
+}
+func (m *PortStatistics) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PortStatistics.Merge(m, src)
+}
+func (m *PortStatistics) XXX_Size() int {
+	return xxx_messageInfo_PortStatistics.Size(m)
+}
+func (m *PortStatistics) XXX_DiscardUnknown() {
+	xxx_messageInfo_PortStatistics.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PortStatistics proto.InternalMessageInfo
+
+func (m *PortStatistics) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxUcastPackets() uint64 {
+	if m != nil {
+		return m.RxUcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxMcastPackets() uint64 {
+	if m != nil {
+		return m.RxMcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxBcastPackets() uint64 {
+	if m != nil {
+		return m.RxBcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxErrorPackets() uint64 {
+	if m != nil {
+		return m.RxErrorPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames() uint64 {
+	if m != nil {
+		return m.RxFrames
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_64() uint64 {
+	if m != nil {
+		return m.RxFrames_64
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_65_127() uint64 {
+	if m != nil {
+		return m.RxFrames_65_127
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_128_255() uint64 {
+	if m != nil {
+		return m.RxFrames_128_255
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_256_511() uint64 {
+	if m != nil {
+		return m.RxFrames_256_511
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_512_1023() uint64 {
+	if m != nil {
+		return m.RxFrames_512_1023
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_1024_1518() uint64 {
+	if m != nil {
+		return m.RxFrames_1024_1518
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_1519_2047() uint64 {
+	if m != nil {
+		return m.RxFrames_1519_2047
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_2048_4095() uint64 {
+	if m != nil {
+		return m.RxFrames_2048_4095
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_4096_9216() uint64 {
+	if m != nil {
+		return m.RxFrames_4096_9216
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxFrames_9217_16383() uint64 {
+	if m != nil {
+		return m.RxFrames_9217_16383
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetRxCrcErrors() uint64 {
+	if m != nil {
+		return m.RxCrcErrors
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxUcastPackets() uint64 {
+	if m != nil {
+		return m.TxUcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxMcastPackets() uint64 {
+	if m != nil {
+		return m.TxMcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxBcastPackets() uint64 {
+	if m != nil {
+		return m.TxBcastPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxErrorPackets() uint64 {
+	if m != nil {
+		return m.TxErrorPackets
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames() uint64 {
+	if m != nil {
+		return m.TxFrames
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_64() uint64 {
+	if m != nil {
+		return m.TxFrames_64
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_65_127() uint64 {
+	if m != nil {
+		return m.TxFrames_65_127
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_128_255() uint64 {
+	if m != nil {
+		return m.TxFrames_128_255
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_256_511() uint64 {
+	if m != nil {
+		return m.TxFrames_256_511
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_512_1023() uint64 {
+	if m != nil {
+		return m.TxFrames_512_1023
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_1024_1518() uint64 {
+	if m != nil {
+		return m.TxFrames_1024_1518
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_1519_2047() uint64 {
+	if m != nil {
+		return m.TxFrames_1519_2047
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_2048_4095() uint64 {
+	if m != nil {
+		return m.TxFrames_2048_4095
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_4096_9216() uint64 {
+	if m != nil {
+		return m.TxFrames_4096_9216
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTxFrames_9217_16383() uint64 {
+	if m != nil {
+		return m.TxFrames_9217_16383
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetBipErrors() uint64 {
+	if m != nil {
+		return m.BipErrors
+	}
+	return 0
+}
+
+func (m *PortStatistics) GetTimestamp() uint32 {
+	if m != nil {
+		return m.Timestamp
+	}
+	return 0
+}
+
+type OnuStatistics struct {
+	IntfId                    uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                     uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	PositiveDrift             uint64   `protobuf:"fixed64,3,opt,name=positive_drift,json=positiveDrift,proto3" json:"positive_drift,omitempty"`
+	NegativeDrift             uint64   `protobuf:"fixed64,4,opt,name=negative_drift,json=negativeDrift,proto3" json:"negative_drift,omitempty"`
+	DelimiterMissDetection    uint64   `protobuf:"fixed64,5,opt,name=delimiter_miss_detection,json=delimiterMissDetection,proto3" json:"delimiter_miss_detection,omitempty"`
+	BipErrors                 uint64   `protobuf:"fixed64,6,opt,name=bip_errors,json=bipErrors,proto3" json:"bip_errors,omitempty"`
+	BipUnits                  uint64   `protobuf:"fixed64,7,opt,name=bip_units,json=bipUnits,proto3" json:"bip_units,omitempty"`
+	FecCorrectedSymbols       uint64   `protobuf:"fixed64,8,opt,name=fec_corrected_symbols,json=fecCorrectedSymbols,proto3" json:"fec_corrected_symbols,omitempty"`
+	FecCodewordsCorrected     uint64   `protobuf:"fixed64,9,opt,name=fec_codewords_corrected,json=fecCodewordsCorrected,proto3" json:"fec_codewords_corrected,omitempty"`
+	FecCodewordsUncorrectable uint64   `protobuf:"fixed64,10,opt,name=fec_codewords_uncorrectable,json=fecCodewordsUncorrectable,proto3" json:"fec_codewords_uncorrectable,omitempty"`
+	FecCodewords              uint64   `protobuf:"fixed64,11,opt,name=fec_codewords,json=fecCodewords,proto3" json:"fec_codewords,omitempty"`
+	FecCorrectedUnits         uint64   `protobuf:"fixed64,12,opt,name=fec_corrected_units,json=fecCorrectedUnits,proto3" json:"fec_corrected_units,omitempty"`
+	XgemKeyErrors             uint64   `protobuf:"fixed64,13,opt,name=xgem_key_errors,json=xgemKeyErrors,proto3" json:"xgem_key_errors,omitempty"`
+	XgemLoss                  uint64   `protobuf:"fixed64,14,opt,name=xgem_loss,json=xgemLoss,proto3" json:"xgem_loss,omitempty"`
+	RxPloamsError             uint64   `protobuf:"fixed64,15,opt,name=rx_ploams_error,json=rxPloamsError,proto3" json:"rx_ploams_error,omitempty"`
+	RxPloamsNonIdle           uint64   `protobuf:"fixed64,16,opt,name=rx_ploams_non_idle,json=rxPloamsNonIdle,proto3" json:"rx_ploams_non_idle,omitempty"`
+	RxOmci                    uint64   `protobuf:"fixed64,17,opt,name=rx_omci,json=rxOmci,proto3" json:"rx_omci,omitempty"`
+	RxOmciPacketsCrcError     uint64   `protobuf:"fixed64,18,opt,name=rx_omci_packets_crc_error,json=rxOmciPacketsCrcError,proto3" json:"rx_omci_packets_crc_error,omitempty"`
+	RxBytes                   uint64   `protobuf:"fixed64,19,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	RxPackets                 uint64   `protobuf:"fixed64,20,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	TxBytes                   uint64   `protobuf:"fixed64,21,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	TxPackets                 uint64   `protobuf:"fixed64,22,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	BerReported               uint64   `protobuf:"fixed64,23,opt,name=ber_reported,json=berReported,proto3" json:"ber_reported,omitempty"`
+	LcdgErrors                uint64   `protobuf:"fixed64,24,opt,name=lcdg_errors,json=lcdgErrors,proto3" json:"lcdg_errors,omitempty"`
+	RdiErrors                 uint64   `protobuf:"fixed64,25,opt,name=rdi_errors,json=rdiErrors,proto3" json:"rdi_errors,omitempty"`
+	Timestamp                 uint32   `protobuf:"fixed32,26,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	XXX_NoUnkeyedLiteral      struct{} `json:"-"`
+	XXX_unrecognized          []byte   `json:"-"`
+	XXX_sizecache             int32    `json:"-"`
+}
+
+func (m *OnuStatistics) Reset()         { *m = OnuStatistics{} }
+func (m *OnuStatistics) String() string { return proto.CompactTextString(m) }
+func (*OnuStatistics) ProtoMessage()    {}
+func (*OnuStatistics) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{23}
+}
+
+func (m *OnuStatistics) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuStatistics.Unmarshal(m, b)
+}
+func (m *OnuStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuStatistics.Marshal(b, m, deterministic)
+}
+func (m *OnuStatistics) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuStatistics.Merge(m, src)
+}
+func (m *OnuStatistics) XXX_Size() int {
+	return xxx_messageInfo_OnuStatistics.Size(m)
+}
+func (m *OnuStatistics) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuStatistics.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuStatistics proto.InternalMessageInfo
+
+func (m *OnuStatistics) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetPositiveDrift() uint64 {
+	if m != nil {
+		return m.PositiveDrift
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetNegativeDrift() uint64 {
+	if m != nil {
+		return m.NegativeDrift
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetDelimiterMissDetection() uint64 {
+	if m != nil {
+		return m.DelimiterMissDetection
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetBipErrors() uint64 {
+	if m != nil {
+		return m.BipErrors
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetBipUnits() uint64 {
+	if m != nil {
+		return m.BipUnits
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetFecCorrectedSymbols() uint64 {
+	if m != nil {
+		return m.FecCorrectedSymbols
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetFecCodewordsCorrected() uint64 {
+	if m != nil {
+		return m.FecCodewordsCorrected
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetFecCodewordsUncorrectable() uint64 {
+	if m != nil {
+		return m.FecCodewordsUncorrectable
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetFecCodewords() uint64 {
+	if m != nil {
+		return m.FecCodewords
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetFecCorrectedUnits() uint64 {
+	if m != nil {
+		return m.FecCorrectedUnits
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetXgemKeyErrors() uint64 {
+	if m != nil {
+		return m.XgemKeyErrors
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetXgemLoss() uint64 {
+	if m != nil {
+		return m.XgemLoss
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxPloamsError() uint64 {
+	if m != nil {
+		return m.RxPloamsError
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxPloamsNonIdle() uint64 {
+	if m != nil {
+		return m.RxPloamsNonIdle
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxOmci() uint64 {
+	if m != nil {
+		return m.RxOmci
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxOmciPacketsCrcError() uint64 {
+	if m != nil {
+		return m.RxOmciPacketsCrcError
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetBerReported() uint64 {
+	if m != nil {
+		return m.BerReported
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetLcdgErrors() uint64 {
+	if m != nil {
+		return m.LcdgErrors
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetRdiErrors() uint64 {
+	if m != nil {
+		return m.RdiErrors
+	}
+	return 0
+}
+
+func (m *OnuStatistics) GetTimestamp() uint32 {
+	if m != nil {
+		return m.Timestamp
+	}
+	return 0
+}
+
+type GemPortStatistics struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	GemportId            uint32   `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	RxBytes              uint64   `protobuf:"fixed64,4,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	TxPackets            uint64   `protobuf:"fixed64,5,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	TxBytes              uint64   `protobuf:"fixed64,6,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	Timestamp            uint32   `protobuf:"fixed32,26,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *GemPortStatistics) Reset()         { *m = GemPortStatistics{} }
+func (m *GemPortStatistics) String() string { return proto.CompactTextString(m) }
+func (*GemPortStatistics) ProtoMessage()    {}
+func (*GemPortStatistics) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{24}
+}
+
+func (m *GemPortStatistics) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GemPortStatistics.Unmarshal(m, b)
+}
+func (m *GemPortStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GemPortStatistics.Marshal(b, m, deterministic)
+}
+func (m *GemPortStatistics) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GemPortStatistics.Merge(m, src)
+}
+func (m *GemPortStatistics) XXX_Size() int {
+	return xxx_messageInfo_GemPortStatistics.Size(m)
+}
+func (m *GemPortStatistics) XXX_DiscardUnknown() {
+	xxx_messageInfo_GemPortStatistics.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GemPortStatistics proto.InternalMessageInfo
+
+func (m *GemPortStatistics) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *GemPortStatistics) GetTimestamp() uint32 {
+	if m != nil {
+		return m.Timestamp
+	}
+	return 0
+}
+
+type FlowStatistics struct {
+	FlowId               uint32   `protobuf:"fixed32,1,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
+	RxBytes              uint64   `protobuf:"fixed64,2,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	RxPackets            uint64   `protobuf:"fixed64,3,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	TxBytes              uint64   `protobuf:"fixed64,8,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	TxPackets            uint64   `protobuf:"fixed64,9,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	Timestamp            uint32   `protobuf:"fixed32,16,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *FlowStatistics) Reset()         { *m = FlowStatistics{} }
+func (m *FlowStatistics) String() string { return proto.CompactTextString(m) }
+func (*FlowStatistics) ProtoMessage()    {}
+func (*FlowStatistics) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{25}
+}
+
+func (m *FlowStatistics) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowStatistics.Unmarshal(m, b)
+}
+func (m *FlowStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowStatistics.Marshal(b, m, deterministic)
+}
+func (m *FlowStatistics) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowStatistics.Merge(m, src)
+}
+func (m *FlowStatistics) XXX_Size() int {
+	return xxx_messageInfo_FlowStatistics.Size(m)
+}
+func (m *FlowStatistics) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowStatistics.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowStatistics proto.InternalMessageInfo
+
+func (m *FlowStatistics) GetFlowId() uint32 {
+	if m != nil {
+		return m.FlowId
+	}
+	return 0
+}
+
+func (m *FlowStatistics) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *FlowStatistics) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *FlowStatistics) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *FlowStatistics) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *FlowStatistics) GetTimestamp() uint32 {
+	if m != nil {
+		return m.Timestamp
+	}
+	return 0
+}
+
+type LosIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	Status               string   `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LosIndication) Reset()         { *m = LosIndication{} }
+func (m *LosIndication) String() string { return proto.CompactTextString(m) }
+func (*LosIndication) ProtoMessage()    {}
+func (*LosIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{26}
+}
+
+func (m *LosIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LosIndication.Unmarshal(m, b)
+}
+func (m *LosIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LosIndication.Marshal(b, m, deterministic)
+}
+func (m *LosIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LosIndication.Merge(m, src)
+}
+func (m *LosIndication) XXX_Size() int {
+	return xxx_messageInfo_LosIndication.Size(m)
+}
+func (m *LosIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_LosIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LosIndication proto.InternalMessageInfo
+
+func (m *LosIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *LosIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type DyingGaspIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DyingGaspIndication) Reset()         { *m = DyingGaspIndication{} }
+func (m *DyingGaspIndication) String() string { return proto.CompactTextString(m) }
+func (*DyingGaspIndication) ProtoMessage()    {}
+func (*DyingGaspIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{27}
+}
+
+func (m *DyingGaspIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DyingGaspIndication.Unmarshal(m, b)
+}
+func (m *DyingGaspIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DyingGaspIndication.Marshal(b, m, deterministic)
+}
+func (m *DyingGaspIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DyingGaspIndication.Merge(m, src)
+}
+func (m *DyingGaspIndication) XXX_Size() int {
+	return xxx_messageInfo_DyingGaspIndication.Size(m)
+}
+func (m *DyingGaspIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_DyingGaspIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DyingGaspIndication proto.InternalMessageInfo
+
+func (m *DyingGaspIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *DyingGaspIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *DyingGaspIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuAlarmIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	LosStatus            string   `protobuf:"bytes,3,opt,name=los_status,json=losStatus,proto3" json:"los_status,omitempty"`
+	LobStatus            string   `protobuf:"bytes,4,opt,name=lob_status,json=lobStatus,proto3" json:"lob_status,omitempty"`
+	LopcMissStatus       string   `protobuf:"bytes,5,opt,name=lopc_miss_status,json=lopcMissStatus,proto3" json:"lopc_miss_status,omitempty"`
+	LopcMicErrorStatus   string   `protobuf:"bytes,6,opt,name=lopc_mic_error_status,json=lopcMicErrorStatus,proto3" json:"lopc_mic_error_status,omitempty"`
+	LofiStatus           string   `protobuf:"bytes,7,opt,name=lofi_status,json=lofiStatus,proto3" json:"lofi_status,omitempty"`
+	LoamiStatus          string   `protobuf:"bytes,8,opt,name=loami_status,json=loamiStatus,proto3" json:"loami_status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuAlarmIndication) Reset()         { *m = OnuAlarmIndication{} }
+func (m *OnuAlarmIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuAlarmIndication) ProtoMessage()    {}
+func (*OnuAlarmIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{28}
+}
+
+func (m *OnuAlarmIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuAlarmIndication.Unmarshal(m, b)
+}
+func (m *OnuAlarmIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuAlarmIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuAlarmIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuAlarmIndication.Merge(m, src)
+}
+func (m *OnuAlarmIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuAlarmIndication.Size(m)
+}
+func (m *OnuAlarmIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuAlarmIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuAlarmIndication proto.InternalMessageInfo
+
+func (m *OnuAlarmIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuAlarmIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuAlarmIndication) GetLosStatus() string {
+	if m != nil {
+		return m.LosStatus
+	}
+	return ""
+}
+
+func (m *OnuAlarmIndication) GetLobStatus() string {
+	if m != nil {
+		return m.LobStatus
+	}
+	return ""
+}
+
+func (m *OnuAlarmIndication) GetLopcMissStatus() string {
+	if m != nil {
+		return m.LopcMissStatus
+	}
+	return ""
+}
+
+func (m *OnuAlarmIndication) GetLopcMicErrorStatus() string {
+	if m != nil {
+		return m.LopcMicErrorStatus
+	}
+	return ""
+}
+
+func (m *OnuAlarmIndication) GetLofiStatus() string {
+	if m != nil {
+		return m.LofiStatus
+	}
+	return ""
+}
+
+func (m *OnuAlarmIndication) GetLoamiStatus() string {
+	if m != nil {
+		return m.LoamiStatus
+	}
+	return ""
+}
+
+type OnuStartupFailureIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuStartupFailureIndication) Reset()         { *m = OnuStartupFailureIndication{} }
+func (m *OnuStartupFailureIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuStartupFailureIndication) ProtoMessage()    {}
+func (*OnuStartupFailureIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{29}
+}
+
+func (m *OnuStartupFailureIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuStartupFailureIndication.Unmarshal(m, b)
+}
+func (m *OnuStartupFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuStartupFailureIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuStartupFailureIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuStartupFailureIndication.Merge(m, src)
+}
+func (m *OnuStartupFailureIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuStartupFailureIndication.Size(m)
+}
+func (m *OnuStartupFailureIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuStartupFailureIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuStartupFailureIndication proto.InternalMessageInfo
+
+func (m *OnuStartupFailureIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuStartupFailureIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuStartupFailureIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuSignalDegradeIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	InverseBitErrorRate  uint32   `protobuf:"fixed32,4,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuSignalDegradeIndication) Reset()         { *m = OnuSignalDegradeIndication{} }
+func (m *OnuSignalDegradeIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuSignalDegradeIndication) ProtoMessage()    {}
+func (*OnuSignalDegradeIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{30}
+}
+
+func (m *OnuSignalDegradeIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuSignalDegradeIndication.Unmarshal(m, b)
+}
+func (m *OnuSignalDegradeIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuSignalDegradeIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuSignalDegradeIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuSignalDegradeIndication.Merge(m, src)
+}
+func (m *OnuSignalDegradeIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuSignalDegradeIndication.Size(m)
+}
+func (m *OnuSignalDegradeIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuSignalDegradeIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuSignalDegradeIndication proto.InternalMessageInfo
+
+func (m *OnuSignalDegradeIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuSignalDegradeIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuSignalDegradeIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuSignalDegradeIndication) GetInverseBitErrorRate() uint32 {
+	if m != nil {
+		return m.InverseBitErrorRate
+	}
+	return 0
+}
+
+type OnuDriftOfWindowIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	Drift                uint32   `protobuf:"fixed32,4,opt,name=drift,proto3" json:"drift,omitempty"`
+	NewEqd               uint32   `protobuf:"fixed32,5,opt,name=new_eqd,json=newEqd,proto3" json:"new_eqd,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuDriftOfWindowIndication) Reset()         { *m = OnuDriftOfWindowIndication{} }
+func (m *OnuDriftOfWindowIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuDriftOfWindowIndication) ProtoMessage()    {}
+func (*OnuDriftOfWindowIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{31}
+}
+
+func (m *OnuDriftOfWindowIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuDriftOfWindowIndication.Unmarshal(m, b)
+}
+func (m *OnuDriftOfWindowIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuDriftOfWindowIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuDriftOfWindowIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuDriftOfWindowIndication.Merge(m, src)
+}
+func (m *OnuDriftOfWindowIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuDriftOfWindowIndication.Size(m)
+}
+func (m *OnuDriftOfWindowIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuDriftOfWindowIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuDriftOfWindowIndication proto.InternalMessageInfo
+
+func (m *OnuDriftOfWindowIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuDriftOfWindowIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuDriftOfWindowIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuDriftOfWindowIndication) GetDrift() uint32 {
+	if m != nil {
+		return m.Drift
+	}
+	return 0
+}
+
+func (m *OnuDriftOfWindowIndication) GetNewEqd() uint32 {
+	if m != nil {
+		return m.NewEqd
+	}
+	return 0
+}
+
+type OnuLossOfOmciChannelIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuLossOfOmciChannelIndication) Reset()         { *m = OnuLossOfOmciChannelIndication{} }
+func (m *OnuLossOfOmciChannelIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuLossOfOmciChannelIndication) ProtoMessage()    {}
+func (*OnuLossOfOmciChannelIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{32}
+}
+
+func (m *OnuLossOfOmciChannelIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Unmarshal(m, b)
+}
+func (m *OnuLossOfOmciChannelIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuLossOfOmciChannelIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuLossOfOmciChannelIndication.Merge(m, src)
+}
+func (m *OnuLossOfOmciChannelIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuLossOfOmciChannelIndication.Size(m)
+}
+func (m *OnuLossOfOmciChannelIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuLossOfOmciChannelIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuLossOfOmciChannelIndication proto.InternalMessageInfo
+
+func (m *OnuLossOfOmciChannelIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuLossOfOmciChannelIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuLossOfOmciChannelIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuSignalsFailureIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	InverseBitErrorRate  uint32   `protobuf:"fixed32,4,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuSignalsFailureIndication) Reset()         { *m = OnuSignalsFailureIndication{} }
+func (m *OnuSignalsFailureIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuSignalsFailureIndication) ProtoMessage()    {}
+func (*OnuSignalsFailureIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{33}
+}
+
+func (m *OnuSignalsFailureIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuSignalsFailureIndication.Unmarshal(m, b)
+}
+func (m *OnuSignalsFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuSignalsFailureIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuSignalsFailureIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuSignalsFailureIndication.Merge(m, src)
+}
+func (m *OnuSignalsFailureIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuSignalsFailureIndication.Size(m)
+}
+func (m *OnuSignalsFailureIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuSignalsFailureIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuSignalsFailureIndication proto.InternalMessageInfo
+
+func (m *OnuSignalsFailureIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuSignalsFailureIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuSignalsFailureIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuSignalsFailureIndication) GetInverseBitErrorRate() uint32 {
+	if m != nil {
+		return m.InverseBitErrorRate
+	}
+	return 0
+}
+
+type OnuTransmissionInterferenceWarning struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	Drift                uint32   `protobuf:"fixed32,4,opt,name=drift,proto3" json:"drift,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuTransmissionInterferenceWarning) Reset()         { *m = OnuTransmissionInterferenceWarning{} }
+func (m *OnuTransmissionInterferenceWarning) String() string { return proto.CompactTextString(m) }
+func (*OnuTransmissionInterferenceWarning) ProtoMessage()    {}
+func (*OnuTransmissionInterferenceWarning) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{34}
+}
+
+func (m *OnuTransmissionInterferenceWarning) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Unmarshal(m, b)
+}
+func (m *OnuTransmissionInterferenceWarning) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Marshal(b, m, deterministic)
+}
+func (m *OnuTransmissionInterferenceWarning) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuTransmissionInterferenceWarning.Merge(m, src)
+}
+func (m *OnuTransmissionInterferenceWarning) XXX_Size() int {
+	return xxx_messageInfo_OnuTransmissionInterferenceWarning.Size(m)
+}
+func (m *OnuTransmissionInterferenceWarning) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuTransmissionInterferenceWarning.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuTransmissionInterferenceWarning proto.InternalMessageInfo
+
+func (m *OnuTransmissionInterferenceWarning) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuTransmissionInterferenceWarning) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuTransmissionInterferenceWarning) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuTransmissionInterferenceWarning) GetDrift() uint32 {
+	if m != nil {
+		return m.Drift
+	}
+	return 0
+}
+
+type OnuActivationFailureIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	FailReason           uint32   `protobuf:"fixed32,3,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuActivationFailureIndication) Reset()         { *m = OnuActivationFailureIndication{} }
+func (m *OnuActivationFailureIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuActivationFailureIndication) ProtoMessage()    {}
+func (*OnuActivationFailureIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{35}
+}
+
+func (m *OnuActivationFailureIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuActivationFailureIndication.Unmarshal(m, b)
+}
+func (m *OnuActivationFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuActivationFailureIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuActivationFailureIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuActivationFailureIndication.Merge(m, src)
+}
+func (m *OnuActivationFailureIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuActivationFailureIndication.Size(m)
+}
+func (m *OnuActivationFailureIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuActivationFailureIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuActivationFailureIndication proto.InternalMessageInfo
+
+func (m *OnuActivationFailureIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuActivationFailureIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuActivationFailureIndication) GetFailReason() uint32 {
+	if m != nil {
+		return m.FailReason
+	}
+	return 0
+}
+
+type OnuLossOfKeySyncFailureIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuLossOfKeySyncFailureIndication) Reset()         { *m = OnuLossOfKeySyncFailureIndication{} }
+func (m *OnuLossOfKeySyncFailureIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuLossOfKeySyncFailureIndication) ProtoMessage()    {}
+func (*OnuLossOfKeySyncFailureIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{36}
+}
+
+func (m *OnuLossOfKeySyncFailureIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Unmarshal(m, b)
+}
+func (m *OnuLossOfKeySyncFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuLossOfKeySyncFailureIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Merge(m, src)
+}
+func (m *OnuLossOfKeySyncFailureIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuLossOfKeySyncFailureIndication.Size(m)
+}
+func (m *OnuLossOfKeySyncFailureIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuLossOfKeySyncFailureIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuLossOfKeySyncFailureIndication proto.InternalMessageInfo
+
+func (m *OnuLossOfKeySyncFailureIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuLossOfKeySyncFailureIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuLossOfKeySyncFailureIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type RdiErrorIndication struct {
+	RdiErrorCount        uint64   `protobuf:"fixed64,1,opt,name=rdi_error_count,json=rdiErrorCount,proto3" json:"rdi_error_count,omitempty"`
+	Status               string   `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *RdiErrorIndication) Reset()         { *m = RdiErrorIndication{} }
+func (m *RdiErrorIndication) String() string { return proto.CompactTextString(m) }
+func (*RdiErrorIndication) ProtoMessage()    {}
+func (*RdiErrorIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{37}
+}
+
+func (m *RdiErrorIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_RdiErrorIndication.Unmarshal(m, b)
+}
+func (m *RdiErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_RdiErrorIndication.Marshal(b, m, deterministic)
+}
+func (m *RdiErrorIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RdiErrorIndication.Merge(m, src)
+}
+func (m *RdiErrorIndication) XXX_Size() int {
+	return xxx_messageInfo_RdiErrorIndication.Size(m)
+}
+func (m *RdiErrorIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_RdiErrorIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RdiErrorIndication proto.InternalMessageInfo
+
+func (m *RdiErrorIndication) GetRdiErrorCount() uint64 {
+	if m != nil {
+		return m.RdiErrorCount
+	}
+	return 0
+}
+
+func (m *RdiErrorIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuItuPonStatsIndication struct {
+	IntfId uint32 `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId  uint32 `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	// Types that are valid to be assigned to Stats:
+	//	*OnuItuPonStatsIndication_RdiErrorInd
+	Stats                isOnuItuPonStatsIndication_Stats `protobuf_oneof:"stats"`
+	XXX_NoUnkeyedLiteral struct{}                         `json:"-"`
+	XXX_unrecognized     []byte                           `json:"-"`
+	XXX_sizecache        int32                            `json:"-"`
+}
+
+func (m *OnuItuPonStatsIndication) Reset()         { *m = OnuItuPonStatsIndication{} }
+func (m *OnuItuPonStatsIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuItuPonStatsIndication) ProtoMessage()    {}
+func (*OnuItuPonStatsIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{38}
+}
+
+func (m *OnuItuPonStatsIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuItuPonStatsIndication.Unmarshal(m, b)
+}
+func (m *OnuItuPonStatsIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuItuPonStatsIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuItuPonStatsIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuItuPonStatsIndication.Merge(m, src)
+}
+func (m *OnuItuPonStatsIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuItuPonStatsIndication.Size(m)
+}
+func (m *OnuItuPonStatsIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuItuPonStatsIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuItuPonStatsIndication proto.InternalMessageInfo
+
+func (m *OnuItuPonStatsIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuItuPonStatsIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+type isOnuItuPonStatsIndication_Stats interface {
+	isOnuItuPonStatsIndication_Stats()
+}
+
+type OnuItuPonStatsIndication_RdiErrorInd struct {
+	RdiErrorInd *RdiErrorIndication `protobuf:"bytes,3,opt,name=rdi_error_ind,json=rdiErrorInd,proto3,oneof"`
+}
+
+func (*OnuItuPonStatsIndication_RdiErrorInd) isOnuItuPonStatsIndication_Stats() {}
+
+func (m *OnuItuPonStatsIndication) GetStats() isOnuItuPonStatsIndication_Stats {
+	if m != nil {
+		return m.Stats
+	}
+	return nil
+}
+
+func (m *OnuItuPonStatsIndication) GetRdiErrorInd() *RdiErrorIndication {
+	if x, ok := m.GetStats().(*OnuItuPonStatsIndication_RdiErrorInd); ok {
+		return x.RdiErrorInd
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*OnuItuPonStatsIndication) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*OnuItuPonStatsIndication_RdiErrorInd)(nil),
+	}
+}
+
+type OnuProcessingErrorIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuProcessingErrorIndication) Reset()         { *m = OnuProcessingErrorIndication{} }
+func (m *OnuProcessingErrorIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuProcessingErrorIndication) ProtoMessage()    {}
+func (*OnuProcessingErrorIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{39}
+}
+
+func (m *OnuProcessingErrorIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuProcessingErrorIndication.Unmarshal(m, b)
+}
+func (m *OnuProcessingErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuProcessingErrorIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuProcessingErrorIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuProcessingErrorIndication.Merge(m, src)
+}
+func (m *OnuProcessingErrorIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuProcessingErrorIndication.Size(m)
+}
+func (m *OnuProcessingErrorIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuProcessingErrorIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuProcessingErrorIndication proto.InternalMessageInfo
+
+func (m *OnuProcessingErrorIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuProcessingErrorIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+type OnuDeactivationFailureIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuDeactivationFailureIndication) Reset()         { *m = OnuDeactivationFailureIndication{} }
+func (m *OnuDeactivationFailureIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuDeactivationFailureIndication) ProtoMessage()    {}
+func (*OnuDeactivationFailureIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{40}
+}
+
+func (m *OnuDeactivationFailureIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuDeactivationFailureIndication.Unmarshal(m, b)
+}
+func (m *OnuDeactivationFailureIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuDeactivationFailureIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuDeactivationFailureIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuDeactivationFailureIndication.Merge(m, src)
+}
+func (m *OnuDeactivationFailureIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuDeactivationFailureIndication.Size(m)
+}
+func (m *OnuDeactivationFailureIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuDeactivationFailureIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuDeactivationFailureIndication proto.InternalMessageInfo
+
+func (m *OnuDeactivationFailureIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuDeactivationFailureIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuDeactivationFailureIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuRemoteDefectIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	RdiErrors            uint64   `protobuf:"fixed64,3,opt,name=rdi_errors,json=rdiErrors,proto3" json:"rdi_errors,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuRemoteDefectIndication) Reset()         { *m = OnuRemoteDefectIndication{} }
+func (m *OnuRemoteDefectIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuRemoteDefectIndication) ProtoMessage()    {}
+func (*OnuRemoteDefectIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{41}
+}
+
+func (m *OnuRemoteDefectIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuRemoteDefectIndication.Unmarshal(m, b)
+}
+func (m *OnuRemoteDefectIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuRemoteDefectIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuRemoteDefectIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuRemoteDefectIndication.Merge(m, src)
+}
+func (m *OnuRemoteDefectIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuRemoteDefectIndication.Size(m)
+}
+func (m *OnuRemoteDefectIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuRemoteDefectIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuRemoteDefectIndication proto.InternalMessageInfo
+
+func (m *OnuRemoteDefectIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuRemoteDefectIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuRemoteDefectIndication) GetRdiErrors() uint64 {
+	if m != nil {
+		return m.RdiErrors
+	}
+	return 0
+}
+
+type OnuLossOfGEMChannelDelineationIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	DelineationErrors    uint32   `protobuf:"fixed32,4,opt,name=delineation_errors,json=delineationErrors,proto3" json:"delineation_errors,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuLossOfGEMChannelDelineationIndication) Reset() {
+	*m = OnuLossOfGEMChannelDelineationIndication{}
+}
+func (m *OnuLossOfGEMChannelDelineationIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuLossOfGEMChannelDelineationIndication) ProtoMessage()    {}
+func (*OnuLossOfGEMChannelDelineationIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{42}
+}
+
+func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Unmarshal(m, b)
+}
+func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Merge(m, src)
+}
+func (m *OnuLossOfGEMChannelDelineationIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.Size(m)
+}
+func (m *OnuLossOfGEMChannelDelineationIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuLossOfGEMChannelDelineationIndication proto.InternalMessageInfo
+
+func (m *OnuLossOfGEMChannelDelineationIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuLossOfGEMChannelDelineationIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuLossOfGEMChannelDelineationIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuLossOfGEMChannelDelineationIndication) GetDelineationErrors() uint32 {
+	if m != nil {
+		return m.DelineationErrors
+	}
+	return 0
+}
+
+type OnuPhysicalEquipmentErrorIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuPhysicalEquipmentErrorIndication) Reset()         { *m = OnuPhysicalEquipmentErrorIndication{} }
+func (m *OnuPhysicalEquipmentErrorIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuPhysicalEquipmentErrorIndication) ProtoMessage()    {}
+func (*OnuPhysicalEquipmentErrorIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{43}
+}
+
+func (m *OnuPhysicalEquipmentErrorIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Unmarshal(m, b)
+}
+func (m *OnuPhysicalEquipmentErrorIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuPhysicalEquipmentErrorIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Merge(m, src)
+}
+func (m *OnuPhysicalEquipmentErrorIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.Size(m)
+}
+func (m *OnuPhysicalEquipmentErrorIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuPhysicalEquipmentErrorIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuPhysicalEquipmentErrorIndication proto.InternalMessageInfo
+
+func (m *OnuPhysicalEquipmentErrorIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuPhysicalEquipmentErrorIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuPhysicalEquipmentErrorIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuLossOfAcknowledgementIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuLossOfAcknowledgementIndication) Reset()         { *m = OnuLossOfAcknowledgementIndication{} }
+func (m *OnuLossOfAcknowledgementIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuLossOfAcknowledgementIndication) ProtoMessage()    {}
+func (*OnuLossOfAcknowledgementIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{44}
+}
+
+func (m *OnuLossOfAcknowledgementIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Unmarshal(m, b)
+}
+func (m *OnuLossOfAcknowledgementIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuLossOfAcknowledgementIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuLossOfAcknowledgementIndication.Merge(m, src)
+}
+func (m *OnuLossOfAcknowledgementIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuLossOfAcknowledgementIndication.Size(m)
+}
+func (m *OnuLossOfAcknowledgementIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuLossOfAcknowledgementIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuLossOfAcknowledgementIndication proto.InternalMessageInfo
+
+func (m *OnuLossOfAcknowledgementIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuLossOfAcknowledgementIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuLossOfAcknowledgementIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+type OnuDifferentialReachExceededIndication struct {
+	IntfId               uint32   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	Distance             uint32   `protobuf:"fixed32,4,opt,name=distance,proto3" json:"distance,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuDifferentialReachExceededIndication) Reset() {
+	*m = OnuDifferentialReachExceededIndication{}
+}
+func (m *OnuDifferentialReachExceededIndication) String() string { return proto.CompactTextString(m) }
+func (*OnuDifferentialReachExceededIndication) ProtoMessage()    {}
+func (*OnuDifferentialReachExceededIndication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{45}
+}
+
+func (m *OnuDifferentialReachExceededIndication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Unmarshal(m, b)
+}
+func (m *OnuDifferentialReachExceededIndication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Marshal(b, m, deterministic)
+}
+func (m *OnuDifferentialReachExceededIndication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuDifferentialReachExceededIndication.Merge(m, src)
+}
+func (m *OnuDifferentialReachExceededIndication) XXX_Size() int {
+	return xxx_messageInfo_OnuDifferentialReachExceededIndication.Size(m)
+}
+func (m *OnuDifferentialReachExceededIndication) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuDifferentialReachExceededIndication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuDifferentialReachExceededIndication proto.InternalMessageInfo
+
+func (m *OnuDifferentialReachExceededIndication) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *OnuDifferentialReachExceededIndication) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *OnuDifferentialReachExceededIndication) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *OnuDifferentialReachExceededIndication) GetDistance() uint32 {
+	if m != nil {
+		return m.Distance
+	}
+	return 0
+}
+
+type GroupMember struct {
+	InterfaceId          uint32                    `protobuf:"varint,1,opt,name=interface_id,json=interfaceId,proto3" json:"interface_id,omitempty"`
+	InterfaceType        GroupMember_InterfaceType `protobuf:"varint,2,opt,name=interface_type,json=interfaceType,proto3,enum=openolt.GroupMember_InterfaceType" json:"interface_type,omitempty"`
+	GemPortId            uint32                    `protobuf:"varint,3,opt,name=gem_port_id,json=gemPortId,proto3" json:"gem_port_id,omitempty"`
+	Priority             uint32                    `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *GroupMember) Reset()         { *m = GroupMember{} }
+func (m *GroupMember) String() string { return proto.CompactTextString(m) }
+func (*GroupMember) ProtoMessage()    {}
+func (*GroupMember) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{46}
+}
+
+func (m *GroupMember) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GroupMember.Unmarshal(m, b)
+}
+func (m *GroupMember) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GroupMember.Marshal(b, m, deterministic)
+}
+func (m *GroupMember) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GroupMember.Merge(m, src)
+}
+func (m *GroupMember) XXX_Size() int {
+	return xxx_messageInfo_GroupMember.Size(m)
+}
+func (m *GroupMember) XXX_DiscardUnknown() {
+	xxx_messageInfo_GroupMember.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GroupMember proto.InternalMessageInfo
+
+func (m *GroupMember) GetInterfaceId() uint32 {
+	if m != nil {
+		return m.InterfaceId
+	}
+	return 0
+}
+
+func (m *GroupMember) GetInterfaceType() GroupMember_InterfaceType {
+	if m != nil {
+		return m.InterfaceType
+	}
+	return GroupMember_PON
+}
+
+func (m *GroupMember) GetGemPortId() uint32 {
+	if m != nil {
+		return m.GemPortId
+	}
+	return 0
+}
+
+func (m *GroupMember) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+type Group struct {
+	GroupId              uint32                    `protobuf:"varint,1,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"`
+	Command              Group_GroupMembersCommand `protobuf:"varint,2,opt,name=command,proto3,enum=openolt.Group_GroupMembersCommand" json:"command,omitempty"`
+	Members              []*GroupMember            `protobuf:"bytes,3,rep,name=members,proto3" json:"members,omitempty"`
+	Action               *Action                   `protobuf:"bytes,4,opt,name=action,proto3" json:"action,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *Group) Reset()         { *m = Group{} }
+func (m *Group) String() string { return proto.CompactTextString(m) }
+func (*Group) ProtoMessage()    {}
+func (*Group) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{47}
+}
+
+func (m *Group) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Group.Unmarshal(m, b)
+}
+func (m *Group) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Group.Marshal(b, m, deterministic)
+}
+func (m *Group) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Group.Merge(m, src)
+}
+func (m *Group) XXX_Size() int {
+	return xxx_messageInfo_Group.Size(m)
+}
+func (m *Group) XXX_DiscardUnknown() {
+	xxx_messageInfo_Group.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Group proto.InternalMessageInfo
+
+func (m *Group) GetGroupId() uint32 {
+	if m != nil {
+		return m.GroupId
+	}
+	return 0
+}
+
+func (m *Group) GetCommand() Group_GroupMembersCommand {
+	if m != nil {
+		return m.Command
+	}
+	return Group_ADD_MEMBERS
+}
+
+func (m *Group) GetMembers() []*GroupMember {
+	if m != nil {
+		return m.Members
+	}
+	return nil
+}
+
+func (m *Group) GetAction() *Action {
+	if m != nil {
+		return m.Action
+	}
+	return nil
+}
+
+type ValueParam struct {
+	Onu                  *Onu                  `protobuf:"bytes,1,opt,name=onu,proto3" json:"onu,omitempty"`
+	Value                common.ValueType_Type `protobuf:"varint,2,opt,name=value,proto3,enum=common.ValueType_Type" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *ValueParam) Reset()         { *m = ValueParam{} }
+func (m *ValueParam) String() string { return proto.CompactTextString(m) }
+func (*ValueParam) ProtoMessage()    {}
+func (*ValueParam) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{48}
+}
+
+func (m *ValueParam) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ValueParam.Unmarshal(m, b)
+}
+func (m *ValueParam) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ValueParam.Marshal(b, m, deterministic)
+}
+func (m *ValueParam) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ValueParam.Merge(m, src)
+}
+func (m *ValueParam) XXX_Size() int {
+	return xxx_messageInfo_ValueParam.Size(m)
+}
+func (m *ValueParam) XXX_DiscardUnknown() {
+	xxx_messageInfo_ValueParam.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ValueParam proto.InternalMessageInfo
+
+func (m *ValueParam) GetOnu() *Onu {
+	if m != nil {
+		return m.Onu
+	}
+	return nil
+}
+
+func (m *ValueParam) GetValue() common.ValueType_Type {
+	if m != nil {
+		return m.Value
+	}
+	return common.ValueType_EMPTY
+}
+
+type PonRxPowerData struct {
+	IntfId               uint32                                   `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32                                   `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	Status               string                                   `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+	FailReason           PonRxPowerData_RssiMeasurementFailReason `protobuf:"varint,4,opt,name=fail_reason,json=failReason,proto3,enum=openolt.PonRxPowerData_RssiMeasurementFailReason" json:"fail_reason,omitempty"`
+	RxPowerMeanDbm       float64                                  `protobuf:"fixed64,5,opt,name=rx_power_mean_dbm,json=rxPowerMeanDbm,proto3" json:"rx_power_mean_dbm,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                                 `json:"-"`
+	XXX_unrecognized     []byte                                   `json:"-"`
+	XXX_sizecache        int32                                    `json:"-"`
+}
+
+func (m *PonRxPowerData) Reset()         { *m = PonRxPowerData{} }
+func (m *PonRxPowerData) String() string { return proto.CompactTextString(m) }
+func (*PonRxPowerData) ProtoMessage()    {}
+func (*PonRxPowerData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{49}
+}
+
+func (m *PonRxPowerData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PonRxPowerData.Unmarshal(m, b)
+}
+func (m *PonRxPowerData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PonRxPowerData.Marshal(b, m, deterministic)
+}
+func (m *PonRxPowerData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PonRxPowerData.Merge(m, src)
+}
+func (m *PonRxPowerData) XXX_Size() int {
+	return xxx_messageInfo_PonRxPowerData.Size(m)
+}
+func (m *PonRxPowerData) XXX_DiscardUnknown() {
+	xxx_messageInfo_PonRxPowerData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PonRxPowerData proto.InternalMessageInfo
+
+func (m *PonRxPowerData) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *PonRxPowerData) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *PonRxPowerData) GetStatus() string {
+	if m != nil {
+		return m.Status
+	}
+	return ""
+}
+
+func (m *PonRxPowerData) GetFailReason() PonRxPowerData_RssiMeasurementFailReason {
+	if m != nil {
+		return m.FailReason
+	}
+	return PonRxPowerData_FAIL_REASON_NONE
+}
+
+func (m *PonRxPowerData) GetRxPowerMeanDbm() float64 {
+	if m != nil {
+		return m.RxPowerMeanDbm
+	}
+	return 0
+}
+
+type Empty struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Empty) Reset()         { *m = Empty{} }
+func (m *Empty) String() string { return proto.CompactTextString(m) }
+func (*Empty) ProtoMessage()    {}
+func (*Empty) Descriptor() ([]byte, []int) {
+	return fileDescriptor_c072e7aa0dfd74d5, []int{50}
+}
+
+func (m *Empty) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Empty.Unmarshal(m, b)
+}
+func (m *Empty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Empty.Marshal(b, m, deterministic)
+}
+func (m *Empty) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Empty.Merge(m, src)
+}
+func (m *Empty) XXX_Size() int {
+	return xxx_messageInfo_Empty.Size(m)
+}
+func (m *Empty) XXX_DiscardUnknown() {
+	xxx_messageInfo_Empty.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Empty proto.InternalMessageInfo
+
+func init() {
+	proto.RegisterEnum("openolt.OnuIndication_ActivationFailReason", OnuIndication_ActivationFailReason_name, OnuIndication_ActivationFailReason_value)
+	proto.RegisterEnum("openolt.DeviceInfo_DeviceResourceRanges_Pool_PoolType", DeviceInfo_DeviceResourceRanges_Pool_PoolType_name, DeviceInfo_DeviceResourceRanges_Pool_PoolType_value)
+	proto.RegisterEnum("openolt.DeviceInfo_DeviceResourceRanges_Pool_SharingType", DeviceInfo_DeviceResourceRanges_Pool_SharingType_name, DeviceInfo_DeviceResourceRanges_Pool_SharingType_value)
+	proto.RegisterEnum("openolt.GroupMember_InterfaceType", GroupMember_InterfaceType_name, GroupMember_InterfaceType_value)
+	proto.RegisterEnum("openolt.Group_GroupMembersCommand", Group_GroupMembersCommand_name, Group_GroupMembersCommand_value)
+	proto.RegisterEnum("openolt.PonRxPowerData_RssiMeasurementFailReason", PonRxPowerData_RssiMeasurementFailReason_name, PonRxPowerData_RssiMeasurementFailReason_value)
+	proto.RegisterType((*Indication)(nil), "openolt.Indication")
+	proto.RegisterType((*AlarmIndication)(nil), "openolt.AlarmIndication")
+	proto.RegisterType((*OltIndication)(nil), "openolt.OltIndication")
+	proto.RegisterType((*IntfIndication)(nil), "openolt.IntfIndication")
+	proto.RegisterType((*OnuDiscIndication)(nil), "openolt.OnuDiscIndication")
+	proto.RegisterType((*OnuIndication)(nil), "openolt.OnuIndication")
+	proto.RegisterType((*IntfOperIndication)(nil), "openolt.IntfOperIndication")
+	proto.RegisterType((*OmciIndication)(nil), "openolt.OmciIndication")
+	proto.RegisterType((*PacketIndication)(nil), "openolt.PacketIndication")
+	proto.RegisterType((*Interface)(nil), "openolt.Interface")
+	proto.RegisterType((*Heartbeat)(nil), "openolt.Heartbeat")
+	proto.RegisterType((*Onu)(nil), "openolt.Onu")
+	proto.RegisterType((*OnuLogicalDistance)(nil), "openolt.OnuLogicalDistance")
+	proto.RegisterType((*OmciMsg)(nil), "openolt.OmciMsg")
+	proto.RegisterType((*OnuPacket)(nil), "openolt.OnuPacket")
+	proto.RegisterType((*UplinkPacket)(nil), "openolt.UplinkPacket")
+	proto.RegisterType((*DeviceInfo)(nil), "openolt.DeviceInfo")
+	proto.RegisterType((*DeviceInfo_DeviceResourceRanges)(nil), "openolt.DeviceInfo.DeviceResourceRanges")
+	proto.RegisterType((*DeviceInfo_DeviceResourceRanges_Pool)(nil), "openolt.DeviceInfo.DeviceResourceRanges.Pool")
+	proto.RegisterType((*Classifier)(nil), "openolt.Classifier")
+	proto.RegisterType((*ActionCmd)(nil), "openolt.ActionCmd")
+	proto.RegisterType((*Action)(nil), "openolt.Action")
+	proto.RegisterType((*Flow)(nil), "openolt.Flow")
+	proto.RegisterMapType((map[uint32]bool)(nil), "openolt.Flow.GemportToAesEntry")
+	proto.RegisterMapType((map[uint32]uint32)(nil), "openolt.Flow.PbitToGemportEntry")
+	proto.RegisterType((*SerialNumber)(nil), "openolt.SerialNumber")
+	proto.RegisterType((*PortStatistics)(nil), "openolt.PortStatistics")
+	proto.RegisterType((*OnuStatistics)(nil), "openolt.OnuStatistics")
+	proto.RegisterType((*GemPortStatistics)(nil), "openolt.GemPortStatistics")
+	proto.RegisterType((*FlowStatistics)(nil), "openolt.FlowStatistics")
+	proto.RegisterType((*LosIndication)(nil), "openolt.LosIndication")
+	proto.RegisterType((*DyingGaspIndication)(nil), "openolt.DyingGaspIndication")
+	proto.RegisterType((*OnuAlarmIndication)(nil), "openolt.OnuAlarmIndication")
+	proto.RegisterType((*OnuStartupFailureIndication)(nil), "openolt.OnuStartupFailureIndication")
+	proto.RegisterType((*OnuSignalDegradeIndication)(nil), "openolt.OnuSignalDegradeIndication")
+	proto.RegisterType((*OnuDriftOfWindowIndication)(nil), "openolt.OnuDriftOfWindowIndication")
+	proto.RegisterType((*OnuLossOfOmciChannelIndication)(nil), "openolt.OnuLossOfOmciChannelIndication")
+	proto.RegisterType((*OnuSignalsFailureIndication)(nil), "openolt.OnuSignalsFailureIndication")
+	proto.RegisterType((*OnuTransmissionInterferenceWarning)(nil), "openolt.OnuTransmissionInterferenceWarning")
+	proto.RegisterType((*OnuActivationFailureIndication)(nil), "openolt.OnuActivationFailureIndication")
+	proto.RegisterType((*OnuLossOfKeySyncFailureIndication)(nil), "openolt.OnuLossOfKeySyncFailureIndication")
+	proto.RegisterType((*RdiErrorIndication)(nil), "openolt.RdiErrorIndication")
+	proto.RegisterType((*OnuItuPonStatsIndication)(nil), "openolt.OnuItuPonStatsIndication")
+	proto.RegisterType((*OnuProcessingErrorIndication)(nil), "openolt.OnuProcessingErrorIndication")
+	proto.RegisterType((*OnuDeactivationFailureIndication)(nil), "openolt.OnuDeactivationFailureIndication")
+	proto.RegisterType((*OnuRemoteDefectIndication)(nil), "openolt.OnuRemoteDefectIndication")
+	proto.RegisterType((*OnuLossOfGEMChannelDelineationIndication)(nil), "openolt.OnuLossOfGEMChannelDelineationIndication")
+	proto.RegisterType((*OnuPhysicalEquipmentErrorIndication)(nil), "openolt.OnuPhysicalEquipmentErrorIndication")
+	proto.RegisterType((*OnuLossOfAcknowledgementIndication)(nil), "openolt.OnuLossOfAcknowledgementIndication")
+	proto.RegisterType((*OnuDifferentialReachExceededIndication)(nil), "openolt.OnuDifferentialReachExceededIndication")
+	proto.RegisterType((*GroupMember)(nil), "openolt.GroupMember")
+	proto.RegisterType((*Group)(nil), "openolt.Group")
+	proto.RegisterType((*ValueParam)(nil), "openolt.ValueParam")
+	proto.RegisterType((*PonRxPowerData)(nil), "openolt.PonRxPowerData")
+	proto.RegisterType((*Empty)(nil), "openolt.Empty")
+}
+
+func init() { proto.RegisterFile("voltha_protos/openolt.proto", fileDescriptor_c072e7aa0dfd74d5) }
+
+var fileDescriptor_c072e7aa0dfd74d5 = []byte{
+	// 5347 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5c, 0x4d, 0x70, 0x1c, 0x49,
+	0x56, 0x76, 0xeb, 0xa7, 0xbb, 0xf5, 0xfa, 0x47, 0xad, 0xd4, 0x8f, 0xf5, 0xe3, 0xb1, 0xe5, 0x1a,
+	0xcf, 0x8c, 0x77, 0x76, 0x47, 0xb6, 0x7a, 0x2c, 0x8f, 0x3d, 0x2c, 0xbb, 0x23, 0x4b, 0x6d, 0xa9,
+	0x19, 0x49, 0x2d, 0x4a, 0x6d, 0x9b, 0x9d, 0x8d, 0x89, 0xda, 0x52, 0x55, 0x76, 0xab, 0x56, 0xd5,
+	0x95, 0x35, 0x55, 0xd9, 0xfa, 0xe1, 0xb8, 0xc1, 0xc2, 0x85, 0xdb, 0x06, 0x44, 0x40, 0x70, 0x21,
+	0xb8, 0x72, 0xe1, 0x46, 0x04, 0x47, 0x82, 0xe0, 0xc2, 0x8d, 0x33, 0x37, 0x82, 0x0b, 0x27, 0x2e,
+	0x9c, 0x08, 0x82, 0xc8, 0x97, 0x59, 0x7f, 0xdd, 0x2d, 0xd9, 0x1e, 0x44, 0x70, 0x71, 0x28, 0xdf,
+	0xfb, 0xde, 0xf7, 0xf2, 0xe7, 0xe5, 0xcb, 0x57, 0x55, 0xd9, 0x86, 0x95, 0x33, 0xe6, 0xf2, 0x13,
+	0xd3, 0xf0, 0x03, 0xc6, 0x59, 0xf8, 0x88, 0xf9, 0xd4, 0x63, 0x2e, 0x5f, 0xc3, 0x26, 0x29, 0xa8,
+	0xe6, 0xf2, 0x9d, 0x2e, 0x63, 0x5d, 0x97, 0x3e, 0x32, 0x7d, 0xe7, 0x91, 0xe9, 0x79, 0x8c, 0x9b,
+	0xdc, 0x61, 0x5e, 0x28, 0x61, 0xcb, 0xab, 0x59, 0x0e, 0x4e, 0xad, 0x13, 0xf1, 0x77, 0xc7, 0x71,
+	0xa9, 0x42, 0x2c, 0x67, 0x11, 0x16, 0xeb, 0xf5, 0x98, 0xa7, 0x74, 0x77, 0xb3, 0x3a, 0x7a, 0xc1,
+	0x0d, 0x8b, 0x79, 0x1d, 0xa7, 0x2b, 0xf5, 0xda, 0x3f, 0x4d, 0x00, 0x34, 0x3d, 0xdb, 0xb1, 0xd0,
+	0x27, 0x59, 0x87, 0x02, 0x73, 0xb9, 0xe1, 0x78, 0xf6, 0x62, 0x6e, 0x35, 0xf7, 0xb0, 0x54, 0x5f,
+	0x58, 0x8b, 0x3a, 0xdd, 0x72, 0x79, 0x02, 0xdc, 0xbd, 0xa5, 0xe7, 0x19, 0x0a, 0xc8, 0x13, 0x28,
+	0x3a, 0x1e, 0xef, 0xa0, 0xcd, 0x18, 0xda, 0xdc, 0x8e, 0x6d, 0x9a, 0x1e, 0xef, 0x64, 0x8c, 0x0a,
+	0x8e, 0x94, 0x90, 0x4d, 0xa8, 0xa0, 0x15, 0xf3, 0x69, 0x80, 0xa6, 0xe3, 0x68, 0xba, 0x92, 0x31,
+	0x6d, 0xf9, 0x34, 0xc8, 0x98, 0x97, 0x9c, 0x44, 0x4a, 0x7e, 0x02, 0x65, 0xe6, 0xf5, 0x0d, 0xdb,
+	0x09, 0x2d, 0x64, 0x98, 0x40, 0x86, 0xe5, 0xa4, 0xc3, 0x5e, 0x7f, 0xdb, 0x09, 0xad, 0x0c, 0x01,
+	0xb0, 0x58, 0x88, 0x63, 0xf5, 0xfa, 0x68, 0x3a, 0x39, 0x38, 0x56, 0xaf, 0x3f, 0x30, 0x56, 0x14,
+	0x88, 0xb1, 0xb2, 0x9e, 0xe5, 0xa0, 0x4d, 0x7e, 0x60, 0xac, 0xad, 0x9e, 0xe5, 0x64, 0xc7, 0xca,
+	0xa4, 0x84, 0x3c, 0x81, 0x82, 0x7f, 0x2a, 0x27, 0xb5, 0x80, 0x46, 0x4b, 0xb1, 0xd1, 0xa1, 0x69,
+	0x9d, 0xd2, 0x81, 0x79, 0xf5, 0x4f, 0x71, 0x5e, 0x9f, 0x01, 0xf8, 0x2c, 0xe0, 0x46, 0xc8, 0x4d,
+	0x1e, 0x2e, 0x16, 0x07, 0xbc, 0x1d, 0xb2, 0x80, 0x1f, 0x89, 0x40, 0x09, 0xb9, 0x63, 0x85, 0xbb,
+	0xb7, 0xf4, 0x29, 0x5f, 0x49, 0x42, 0x61, 0xd9, 0x71, 0xd9, 0xb9, 0xb2, 0x9c, 0x1a, 0xb0, 0x7c,
+	0xe9, 0xb2, 0xf3, 0xac, 0x65, 0x47, 0x49, 0x42, 0xf2, 0x05, 0x4c, 0x99, 0xae, 0x19, 0xf4, 0xb0,
+	0xaf, 0x80, 0x86, 0x8b, 0xb1, 0xe1, 0xa6, 0xd0, 0x64, 0xba, 0x5a, 0x34, 0x95, 0xe8, 0x45, 0x1e,
+	0x26, 0x6c, 0x93, 0x9b, 0xda, 0xbf, 0x57, 0x60, 0x7a, 0x00, 0x27, 0xe6, 0xd9, 0x65, 0xe1, 0xc8,
+	0x98, 0xda, 0x63, 0x61, 0x76, 0xec, 0x2e, 0x0a, 0xc8, 0x36, 0x54, 0xed, 0x4b, 0xc7, 0xeb, 0x1a,
+	0x5d, 0x33, 0xf4, 0x53, 0x91, 0x75, 0x27, 0xb6, 0xdc, 0x16, 0xea, 0x1d, 0x33, 0xf4, 0x33, 0xf6,
+	0x65, 0x3b, 0x25, 0x16, 0x31, 0x26, 0x16, 0x38, 0x19, 0xd1, 0x60, 0x8c, 0xb5, 0xbc, 0xfe, 0xf0,
+	0xa0, 0x4a, 0x2c, 0x91, 0x92, 0x37, 0x30, 0x27, 0x28, 0x42, 0x6e, 0x06, 0xbc, 0xef, 0x1b, 0x1d,
+	0xd3, 0x71, 0x53, 0xb1, 0xf6, 0x20, 0xcd, 0x74, 0x24, 0x31, 0x2f, 0x4d, 0xc7, 0xed, 0x07, 0x34,
+	0x43, 0x39, 0xc3, 0x32, 0x6a, 0x41, 0xfc, 0x0d, 0x2c, 0x20, 0xb1, 0xd3, 0xf5, 0x4c, 0xd7, 0xb0,
+	0x69, 0x37, 0x30, 0x6d, 0x9a, 0x8a, 0xc5, 0x0f, 0x33, 0xd4, 0x88, 0xda, 0x96, 0xa0, 0x0c, 0xf3,
+	0x2c, 0x1b, 0xd6, 0x92, 0x9f, 0xc3, 0x6d, 0xdc, 0x18, 0x81, 0xd3, 0xe1, 0x06, 0xeb, 0x18, 0xe7,
+	0x8e, 0x67, 0xb3, 0xf3, 0x54, 0xd0, 0x66, 0xc8, 0xb7, 0x05, 0xac, 0xd5, 0x79, 0x83, 0xa0, 0x21,
+	0xf2, 0x41, 0x2d, 0x69, 0x83, 0x18, 0x8d, 0xe1, 0xb2, 0x30, 0x34, 0xe2, 0xbd, 0x20, 0xc3, 0xfa,
+	0x93, 0x34, 0xed, 0x1e, 0x0b, 0xc3, 0x56, 0x47, 0x6c, 0x8a, 0xad, 0x13, 0xd3, 0xf3, 0xa8, 0x9b,
+	0xa1, 0xae, 0x32, 0x85, 0x50, 0x5b, 0x24, 0x9a, 0x67, 0x1c, 0x4a, 0x98, 0xcc, 0x73, 0x71, 0xc4,
+	0x3c, 0x4b, 0xcc, 0x95, 0xf3, 0x9c, 0xa8, 0x05, 0x71, 0x4b, 0x26, 0x09, 0xee, 0x9c, 0xcb, 0x9e,
+	0xca, 0xdd, 0xf0, 0xc3, 0x34, 0x61, 0x3b, 0x30, 0xbd, 0xb0, 0xe7, 0x84, 0xa1, 0xc3, 0xbc, 0xa6,
+	0xc7, 0x69, 0xd0, 0xa1, 0x01, 0xf5, 0x2c, 0xfa, 0xc6, 0x0c, 0x3c, 0xc7, 0xeb, 0xaa, 0xac, 0xd1,
+	0x76, 0xce, 0xb1, 0xa7, 0xbf, 0x90, 0x93, 0x6b, 0x5a, 0xdc, 0x39, 0x43, 0xbf, 0x49, 0x67, 0x61,
+	0x78, 0x16, 0x36, 0x63, 0xd8, 0xa8, 0xfe, 0x8a, 0x31, 0x67, 0x11, 0xd2, 0xc3, 0xa2, 0xf0, 0xe0,
+	0x07, 0xcc, 0xa2, 0x61, 0x28, 0x76, 0x01, 0x0d, 0x02, 0x26, 0xb3, 0x64, 0x09, 0x5d, 0x7c, 0x94,
+	0x76, 0x71, 0x18, 0xe3, 0x1a, 0x02, 0x96, 0x71, 0x30, 0xcf, 0x46, 0xe9, 0x09, 0x85, 0xa5, 0x64,
+	0x0d, 0x3b, 0x46, 0x78, 0xe9, 0x59, 0xc9, 0x28, 0xca, 0xe8, 0xe2, 0xd3, 0xe1, 0xb5, 0xfc, 0x9a,
+	0x5e, 0x1e, 0x5d, 0x7a, 0xd6, 0x55, 0x03, 0x91, 0xa0, 0x08, 0x21, 0xdc, 0xbc, 0x82, 0x79, 0x4c,
+	0xb0, 0xbc, 0x6f, 0xf8, 0xcc, 0x93, 0xe9, 0x08, 0x5d, 0x54, 0xd0, 0xc5, 0xfd, 0x4c, 0xba, 0xe5,
+	0xfd, 0x43, 0xe6, 0x61, 0x16, 0x1a, 0x5a, 0xd2, 0xac, 0x8e, 0xb8, 0x70, 0x07, 0xc3, 0x9b, 0x0e,
+	0xac, 0x41, 0x3f, 0x90, 0x1b, 0xa8, 0x8a, 0xec, 0x3f, 0xc8, 0xc4, 0x78, 0x0a, 0x3b, 0xaa, 0xff,
+	0x62, 0x3a, 0x46, 0x63, 0xc8, 0x1b, 0x39, 0x88, 0x80, 0xf6, 0x18, 0xa7, 0x86, 0x4d, 0x3b, 0xd4,
+	0x92, 0xa9, 0x7c, 0x1a, 0xdd, 0x68, 0x69, 0x37, 0x3a, 0x82, 0xb6, 0x11, 0x93, 0xe1, 0x27, 0x6c,
+	0x48, 0x49, 0x42, 0x39, 0x0c, 0x5c, 0x84, 0x2e, 0xed, 0x19, 0x36, 0x75, 0x1d, 0x8f, 0xca, 0xe1,
+	0x08, 0xfe, 0x1a, 0xf2, 0xaf, 0x0f, 0xaf, 0xc3, 0x4e, 0x63, 0x5f, 0x6d, 0xa9, 0xed, 0xc4, 0x24,
+	0xe3, 0x6e, 0x51, 0x2d, 0xc7, 0x0e, 0xed, 0x65, 0x21, 0xe4, 0x0c, 0x56, 0x31, 0xb6, 0x4e, 0x2e,
+	0x43, 0xc7, 0x32, 0x5d, 0x83, 0x7e, 0xd7, 0x77, 0xfc, 0x1e, 0xf5, 0x78, 0x2a, 0xc6, 0x66, 0xd0,
+	0xf1, 0x8f, 0x32, 0x31, 0xa6, 0xf0, 0x8d, 0x08, 0x3e, 0x1c, 0x6a, 0x62, 0x30, 0x57, 0xc2, 0xc8,
+	0xcf, 0x61, 0x36, 0x1d, 0x71, 0xa6, 0x75, 0x8a, 0xae, 0xc8, 0xf0, 0x6e, 0x94, 0x63, 0xdc, 0xb4,
+	0x4e, 0x3d, 0x76, 0xee, 0x52, 0xbb, 0x4b, 0x05, 0x4f, 0xc6, 0xd3, 0x34, 0x4b, 0xa1, 0x04, 0x39,
+	0x83, 0x15, 0x59, 0x08, 0x74, 0x3a, 0x46, 0x40, 0x4d, 0xeb, 0xc4, 0xa0, 0x17, 0x16, 0xa5, 0x36,
+	0xb5, 0xd1, 0xc9, 0x2c, 0x3a, 0x79, 0x94, 0xad, 0x0b, 0x3a, 0xb8, 0xc9, 0xb9, 0x63, 0xba, 0xba,
+	0xb0, 0x68, 0x28, 0x83, 0x8c, 0xa3, 0xdb, 0x4c, 0x22, 0x07, 0x11, 0xf1, 0x69, 0xb7, 0x06, 0x95,
+	0x4c, 0x55, 0x44, 0x3e, 0x00, 0xc0, 0x82, 0x46, 0x84, 0x3a, 0xc5, 0xd3, 0x6e, 0x4a, 0x9f, 0x12,
+	0x12, 0x11, 0xbc, 0x54, 0xdb, 0x85, 0x6a, 0xb6, 0x22, 0x22, 0xb7, 0xa1, 0x20, 0x8b, 0x27, 0x79,
+	0x36, 0x16, 0xf4, 0x3c, 0x16, 0x48, 0xf6, 0x00, 0xd3, 0xd8, 0x20, 0xd3, 0x09, 0xcc, 0x0c, 0x95,
+	0x37, 0x57, 0x93, 0x7d, 0x09, 0x95, 0x90, 0x06, 0x8e, 0xe9, 0x1a, 0x5e, 0xbf, 0x77, 0x4c, 0x03,
+	0x75, 0x9a, 0xce, 0xc7, 0x53, 0x72, 0x84, 0xda, 0x03, 0x54, 0xea, 0xe5, 0x30, 0xd5, 0xd2, 0x7e,
+	0x33, 0x01, 0x95, 0x4c, 0x39, 0x74, 0xb5, 0x9b, 0x79, 0xc8, 0xe3, 0x7e, 0x97, 0xa7, 0x75, 0x41,
+	0x9f, 0x14, 0x7b, 0x77, 0x70, 0x28, 0xe3, 0x03, 0x43, 0x21, 0xf7, 0xa0, 0x64, 0xda, 0x3d, 0xc7,
+	0x53, 0xfa, 0x49, 0xd4, 0x03, 0x8a, 0x24, 0x60, 0xa8, 0xf7, 0x13, 0xef, 0xdc, 0x7b, 0xb2, 0x07,
+	0x25, 0x4c, 0x6c, 0x01, 0x35, 0x43, 0xe6, 0xe1, 0xf1, 0x57, 0xcd, 0xc6, 0x5b, 0x32, 0xb0, 0xb5,
+	0x6c, 0x2a, 0xd6, 0xd1, 0x44, 0x87, 0x4e, 0xfc, 0xb7, 0xf6, 0x47, 0x63, 0x30, 0x37, 0x0a, 0x44,
+	0x3e, 0x84, 0x7b, 0xad, 0x83, 0x57, 0xc6, 0xe6, 0x56, 0xbb, 0xf9, 0x7a, 0xb3, 0xdd, 0x6c, 0x1d,
+	0x18, 0x2f, 0x37, 0x9b, 0x7b, 0x86, 0xde, 0xd8, 0x3c, 0x6a, 0x1d, 0x18, 0x07, 0xad, 0x83, 0x46,
+	0xed, 0x16, 0xf9, 0x18, 0xb4, 0x6b, 0x40, 0xfa, 0xe6, 0xc1, 0x4e, 0xf3, 0x60, 0xa7, 0x96, 0x23,
+	0x4f, 0xa1, 0x7e, 0x0d, 0xee, 0x70, 0xf3, 0xe8, 0xe8, 0x4d, 0x4b, 0xdf, 0x36, 0x36, 0x5f, 0xb5,
+	0x77, 0x1b, 0x07, 0xed, 0xe6, 0x16, 0x62, 0x6a, 0x63, 0x44, 0x83, 0xbb, 0xd7, 0xd8, 0xed, 0xb5,
+	0x8e, 0x6a, 0xe3, 0xe4, 0x3e, 0x7c, 0x30, 0x0a, 0x83, 0xb2, 0xbd, 0x4d, 0x7d, 0xbf, 0x36, 0x71,
+	0xd5, 0x58, 0x8e, 0xde, 0x34, 0xdb, 0x5b, 0xbb, 0x46, 0xeb, 0x75, 0x43, 0xaf, 0x4d, 0x6a, 0xbf,
+	0x00, 0x32, 0x5c, 0xa0, 0x13, 0x02, 0x13, 0xfc, 0xd2, 0x8f, 0x02, 0x1f, 0xff, 0x4e, 0x47, 0xcb,
+	0xd8, 0x35, 0x11, 0x3e, 0x18, 0x16, 0x9a, 0x0e, 0xd5, 0x6c, 0x45, 0xfd, 0xde, 0x71, 0x57, 0x83,
+	0x71, 0xff, 0x94, 0x23, 0x73, 0x59, 0x17, 0x7f, 0x6a, 0xff, 0x91, 0x83, 0xda, 0x60, 0xc5, 0x4d,
+	0x56, 0x60, 0x0a, 0x69, 0xb1, 0xe7, 0x32, 0xfa, 0xf0, 0x81, 0xa6, 0x3d, 0xd0, 0xfb, 0xab, 0x7c,
+	0x16, 0xd3, 0x3e, 0xe7, 0x21, 0xdf, 0xf7, 0x1c, 0x21, 0x9e, 0x92, 0xe2, 0xbe, 0xe7, 0xc8, 0xb1,
+	0x76, 0x69, 0x0f, 0xcb, 0xf9, 0xb8, 0x97, 0x53, 0x4a, 0xd2, 0xb4, 0x85, 0x17, 0x2c, 0xd8, 0x1d,
+	0x59, 0xa2, 0x16, 0xf4, 0xbc, 0x68, 0x4a, 0x05, 0x1a, 0x79, 0x0c, 0x43, 0xb7, 0xa0, 0xe7, 0x45,
+	0xf3, 0x80, 0x91, 0x05, 0xc8, 0x5b, 0x8c, 0x9d, 0x3a, 0x14, 0x4b, 0xaf, 0xbc, 0xae, 0x5a, 0xd1,
+	0x98, 0x27, 0x92, 0x31, 0x3f, 0x80, 0x29, 0x59, 0xd4, 0x98, 0xd6, 0xd5, 0xc3, 0xd1, 0x7e, 0x0c,
+	0x53, 0xbb, 0xd4, 0x0c, 0xf8, 0x31, 0x35, 0x39, 0x79, 0x04, 0xb3, 0x27, 0x51, 0x43, 0x96, 0x64,
+	0xbc, 0x1f, 0x50, 0x65, 0x41, 0x62, 0xd5, 0x51, 0xa4, 0xd1, 0xfe, 0x3a, 0x07, 0xe3, 0x2d, 0xaf,
+	0xff, 0xde, 0x2b, 0x34, 0xb4, 0xb3, 0xc7, 0xdf, 0x7d, 0x67, 0x8b, 0x91, 0x3a, 0x32, 0x17, 0x14,
+	0x74, 0xf1, 0x27, 0xf9, 0x04, 0xa6, 0x59, 0xcf, 0xb2, 0x0c, 0xea, 0x59, 0xc1, 0xa5, 0x2f, 0xd6,
+	0x16, 0x97, 0xb3, 0xa8, 0x57, 0x85, 0xb8, 0x11, 0x4b, 0xb5, 0xbf, 0xc9, 0x01, 0xc1, 0x93, 0xa6,
+	0x2b, 0x0e, 0xab, 0x6d, 0x27, 0xe4, 0xa6, 0x67, 0xd1, 0xf7, 0xee, 0xfd, 0x73, 0x58, 0x72, 0x25,
+	0x85, 0xa1, 0x9e, 0x43, 0x91, 0xc7, 0xf8, 0x7d, 0x1a, 0x30, 0xb5, 0x8e, 0x0b, 0x0a, 0x20, 0x73,
+	0x35, 0xaa, 0xbf, 0xa1, 0x01, 0x23, 0x8f, 0x61, 0x6e, 0x94, 0xa9, 0x1a, 0x0d, 0x19, 0xb6, 0xd2,
+	0xbe, 0x86, 0x82, 0xd8, 0x0e, 0xfb, 0x61, 0xf7, 0x06, 0xf6, 0xc1, 0xaf, 0x73, 0x30, 0x25, 0x4e,
+	0x75, 0xdc, 0x0a, 0xef, 0xcd, 0x97, 0x0a, 0xca, 0x89, 0x4c, 0x50, 0x66, 0xa3, 0x7c, 0x72, 0x30,
+	0xca, 0x87, 0xfb, 0xf1, 0x1c, 0xca, 0xaf, 0x7c, 0xd7, 0xf1, 0x4e, 0xdf, 0xd6, 0x13, 0x65, 0x3a,
+	0x96, 0x98, 0xfe, 0xc3, 0x14, 0xc0, 0x36, 0x3d, 0x73, 0x2c, 0xda, 0xf4, 0x3a, 0xb8, 0x1f, 0xce,
+	0xa8, 0x67, 0xb3, 0x40, 0xe5, 0x1e, 0xd5, 0x22, 0x73, 0x30, 0xd9, 0x63, 0x36, 0x75, 0xd5, 0x09,
+	0x2a, 0x1b, 0xe4, 0x07, 0x50, 0x3b, 0x31, 0x03, 0xfb, 0xdc, 0x0c, 0xa8, 0x71, 0x46, 0x03, 0x51,
+	0xf8, 0xab, 0x04, 0x34, 0x1d, 0xc9, 0x5f, 0x4b, 0xb1, 0x80, 0x76, 0x9c, 0xa0, 0x97, 0x81, 0x4e,
+	0x48, 0x68, 0x24, 0x8f, 0xa0, 0x2b, 0x30, 0x65, 0x63, 0x8f, 0x44, 0xff, 0x6b, 0x32, 0x91, 0x48,
+	0x41, 0xd3, 0x16, 0x2b, 0xae, 0x94, 0xd9, 0x88, 0x9f, 0x41, 0x1c, 0x91, 0xba, 0x74, 0xb8, 0x93,
+	0x75, 0x98, 0xf3, 0x03, 0x7a, 0xe6, 0xb0, 0x7e, 0xe8, 0x5e, 0x1a, 0x16, 0xf3, 0x3c, 0x6a, 0x71,
+	0x2a, 0xcb, 0x99, 0xa2, 0x3e, 0x9b, 0xe8, 0xb6, 0x22, 0x95, 0xe8, 0x81, 0x28, 0xb4, 0xc5, 0x7c,
+	0x87, 0x58, 0xc7, 0x17, 0xf4, 0xa2, 0xcf, 0xbc, 0x43, 0xd1, 0x26, 0x77, 0x01, 0x38, 0xb5, 0x4e,
+	0x3c, 0xe6, 0xb2, 0xee, 0x65, 0x74, 0xcc, 0x26, 0x12, 0xb2, 0x2a, 0x9f, 0x94, 0x1c, 0x5b, 0x3e,
+	0xed, 0xaa, 0x84, 0x03, 0xb8, 0xe6, 0xf8, 0xf0, 0x4a, 0xee, 0x00, 0x28, 0x04, 0x55, 0xcf, 0x7c,
+	0x05, 0xbd, 0x88, 0xfa, 0x86, 0x67, 0x93, 0x07, 0x50, 0x35, 0x5d, 0x97, 0x59, 0x09, 0x83, 0xcc,
+	0x8c, 0x65, 0x94, 0x46, 0x1c, 0xab, 0x50, 0x8e, 0x51, 0xd4, 0x8b, 0xd2, 0x24, 0x28, 0x8c, 0xe0,
+	0x79, 0x08, 0xb5, 0x24, 0x8a, 0x14, 0x13, 0x20, 0xaa, 0x1a, 0xc7, 0x92, 0xe4, 0x7a, 0x00, 0xd5,
+	0x14, 0x92, 0xaa, 0xc7, 0xa3, 0x82, 0x5e, 0x8e, 0x71, 0x82, 0x4f, 0x83, 0x8a, 0x4a, 0xae, 0x8a,
+	0xac, 0x82, 0xa0, 0x92, 0x4c, 0xb1, 0x92, 0xe9, 0x2e, 0x94, 0x22, 0x0c, 0x55, 0x4f, 0x10, 0x05,
+	0xf9, 0x5e, 0x44, 0x72, 0x7c, 0x05, 0xf9, 0xc0, 0xf4, 0xba, 0x34, 0x5c, 0x9c, 0x5e, 0x1d, 0x7f,
+	0x58, 0xaa, 0x3f, 0x4c, 0xde, 0x43, 0xc4, 0x31, 0xa8, 0xfe, 0xd4, 0x69, 0xc8, 0xfa, 0x81, 0x45,
+	0x75, 0xc4, 0xeb, 0xca, 0x6e, 0xf9, 0x4f, 0x26, 0x60, 0x6e, 0x14, 0x80, 0x2c, 0x45, 0xaf, 0xcf,
+	0xec, 0x70, 0x31, 0xb7, 0x3a, 0xfe, 0xb0, 0xa0, 0xde, 0x91, 0xd9, 0x83, 0x2b, 0x36, 0x36, 0xb4,
+	0x62, 0x5b, 0x30, 0xe9, 0x33, 0xe6, 0x86, 0x8b, 0xe3, 0xd8, 0xa9, 0xcf, 0xde, 0xb5, 0x53, 0x6b,
+	0x87, 0x8c, 0xb9, 0xba, 0xb4, 0x5d, 0xfe, 0xaf, 0x31, 0x98, 0x10, 0x6d, 0xf2, 0x3b, 0xa9, 0xc3,
+	0xbb, 0x5a, 0x7f, 0xfa, 0x5e, 0x64, 0xf8, 0x8f, 0x38, 0x30, 0xd5, 0xa1, 0x7f, 0x04, 0x85, 0xf0,
+	0xc4, 0x0c, 0x1c, 0xaf, 0x8b, 0xdd, 0xae, 0xd6, 0x9f, 0xbf, 0x1f, 0xdd, 0x91, 0x34, 0x46, 0xc6,
+	0x88, 0x49, 0xec, 0x65, 0xb9, 0x80, 0x32, 0xb7, 0xca, 0x86, 0x48, 0x0d, 0x54, 0xbd, 0x90, 0x29,
+	0xe8, 0xe2, 0x4f, 0x6d, 0x13, 0x8a, 0x51, 0x77, 0x08, 0x40, 0x5e, 0x14, 0x33, 0xcd, 0xed, 0xda,
+	0x2d, 0x52, 0x86, 0xe2, 0xe6, 0xde, 0x5e, 0x6b, 0x4b, 0xb4, 0x72, 0xa4, 0x0a, 0xb0, 0xd3, 0xd8,
+	0x3f, 0x6c, 0xe9, 0x6d, 0xd1, 0x1e, 0x23, 0x25, 0x28, 0xbc, 0xdc, 0x6b, 0xbd, 0x11, 0x8d, 0x71,
+	0xed, 0x04, 0x4a, 0xa9, 0x2e, 0x90, 0x05, 0x20, 0xdb, 0x8d, 0x6d, 0x51, 0x69, 0x35, 0xb6, 0x8d,
+	0xc3, 0x86, 0x6e, 0x34, 0x0f, 0xda, 0x2f, 0x6b, 0xb7, 0xc8, 0x3d, 0x58, 0x39, 0xda, 0xdd, 0xd4,
+	0x1b, 0xdb, 0xc6, 0x8b, 0x9f, 0x19, 0x9b, 0x7b, 0x7b, 0x28, 0xc7, 0x3f, 0xda, 0x8d, 0xad, 0xdd,
+	0x5a, 0x8e, 0xac, 0xc2, 0x9d, 0x11, 0x80, 0xa3, 0xcd, 0xfd, 0x86, 0x44, 0x8c, 0x69, 0x7f, 0x30,
+	0x0e, 0xb0, 0xe5, 0x9a, 0x61, 0xe8, 0x74, 0x1c, 0x1a, 0x60, 0xca, 0x35, 0xb8, 0x1f, 0x27, 0xc0,
+	0x49, 0xd6, 0xf6, 0x1d, 0x9b, 0xcc, 0xc2, 0x24, 0x33, 0xce, 0xe2, 0x44, 0x3c, 0xc1, 0x5e, 0x3b,
+	0x98, 0x9e, 0x1d, 0x89, 0x55, 0x13, 0xe2, 0x44, 0x58, 0x07, 0xb1, 0x72, 0x4a, 0x26, 0x1c, 0x81,
+	0xbd, 0x0d, 0x05, 0x66, 0xf8, 0xc7, 0x0e, 0x0f, 0x55, 0x5e, 0xce, 0xb3, 0x43, 0xd1, 0xc2, 0x94,
+	0xab, 0x14, 0xaa, 0xc2, 0x70, 0xa4, 0x62, 0x09, 0x8a, 0x94, 0x9f, 0xc8, 0xaa, 0x48, 0x6e, 0xf5,
+	0x02, 0xe5, 0x27, 0x51, 0x51, 0x64, 0x87, 0xdc, 0xe8, 0x99, 0x16, 0x6e, 0xf1, 0xb2, 0x9e, 0xb7,
+	0x43, 0xbe, 0x6f, 0x5a, 0x42, 0x11, 0x06, 0x16, 0x2a, 0xa6, 0xa4, 0x22, 0x0c, 0x2c, 0xa1, 0x10,
+	0x41, 0xee, 0xcb, 0x77, 0xd0, 0x6a, 0x2f, 0x17, 0x1c, 0xff, 0x10, 0xdf, 0x82, 0xcf, 0x83, 0xb0,
+	0x36, 0x1c, 0x5f, 0x6d, 0xde, 0x49, 0x3b, 0xe4, 0x4d, 0x5f, 0x88, 0x05, 0x95, 0xe3, 0xab, 0x3c,
+	0x36, 0x19, 0x06, 0x56, 0xd3, 0x17, 0x44, 0x42, 0x2c, 0x76, 0xb7, 0xda, 0xc7, 0xc2, 0xa3, 0x48,
+	0x70, 0x42, 0x25, 0x88, 0x50, 0x25, 0x37, 0xb0, 0xe8, 0x25, 0xaa, 0x56, 0xa1, 0xec, 0x9f, 0x72,
+	0x83, 0x9b, 0x5d, 0x39, 0x9e, 0x69, 0xb9, 0x95, 0xfc, 0x53, 0xde, 0x36, 0x71, 0x85, 0xb5, 0x5f,
+	0x8f, 0xc3, 0x94, 0xa8, 0xec, 0x99, 0xb7, 0xd5, 0xc3, 0x94, 0x61, 0xda, 0xb6, 0xc1, 0xfa, 0x9c,
+	0x06, 0xc2, 0x0a, 0x17, 0xa3, 0xa8, 0x97, 0x4c, 0xdb, 0x6e, 0x09, 0x59, 0xdb, 0xec, 0x8a, 0x34,
+	0x15, 0xd0, 0x1e, 0x3b, 0xa3, 0x29, 0xd8, 0x98, 0x2c, 0x37, 0xa4, 0x3c, 0x46, 0xae, 0x42, 0x99,
+	0x07, 0xa6, 0x6f, 0x70, 0x66, 0x9c, 0xb0, 0x50, 0x86, 0x6f, 0x51, 0x07, 0x21, 0x6b, 0xb3, 0x5d,
+	0x16, 0x72, 0xf2, 0x23, 0x20, 0x01, 0xed, 0x99, 0xc1, 0xa9, 0xe2, 0x92, 0xeb, 0x31, 0x81, 0xb8,
+	0x9a, 0xd4, 0x20, 0x9b, 0x5c, 0x99, 0x04, 0xed, 0x78, 0x5e, 0x8c, 0x9e, 0x4c, 0xa3, 0x9b, 0x42,
+	0x21, 0xd1, 0x6a, 0x2c, 0x12, 0x2a, 0x3a, 0x99, 0x8f, 0xc7, 0x82, 0xa8, 0xec, 0x58, 0x12, 0x58,
+	0x21, 0x3d, 0x96, 0x18, 0xb9, 0x06, 0xb3, 0x3c, 0x30, 0xbd, 0xd0, 0x35, 0x79, 0x1a, 0x5c, 0x44,
+	0xf0, 0x4c, 0xac, 0x1a, 0x8d, 0x4f, 0x26, 0x6a, 0x6a, 0x00, 0x1f, 0xcd, 0x95, 0xf6, 0xb7, 0x39,
+	0xc8, 0xcb, 0x75, 0x20, 0x0f, 0x60, 0xdc, 0xea, 0x45, 0xaf, 0x8c, 0x49, 0xf2, 0x16, 0x3a, 0x5a,
+	0x25, 0x5d, 0xa8, 0x47, 0xef, 0x8c, 0x54, 0xb4, 0x8f, 0x67, 0xa2, 0x3d, 0xd9, 0x5e, 0x13, 0x03,
+	0xdb, 0x4b, 0x6e, 0x99, 0xc9, 0xec, 0x96, 0x19, 0xbd, 0x33, 0x92, 0x7d, 0x57, 0x48, 0xed, 0x3b,
+	0xed, 0x1f, 0xf3, 0x30, 0xf1, 0xd2, 0x65, 0xe7, 0x78, 0x10, 0x5a, 0x16, 0x0d, 0x43, 0x23, 0x5d,
+	0xcc, 0x4c, 0xeb, 0x65, 0x29, 0x6d, 0x8e, 0x2a, 0xae, 0xa6, 0x87, 0x1f, 0x20, 0x4a, 0x52, 0x2c,
+	0x1f, 0x20, 0x06, 0x9e, 0x10, 0xf2, 0xf1, 0x13, 0xc2, 0xa7, 0x30, 0x13, 0x5e, 0xf6, 0x7a, 0x94,
+	0x07, 0x8e, 0x65, 0x44, 0x10, 0x82, 0x90, 0xe9, 0x58, 0xf1, 0x52, 0x62, 0x57, 0x00, 0x8f, 0x34,
+	0xb9, 0x07, 0x64, 0x11, 0x53, 0x14, 0x02, 0xdc, 0xd4, 0x4b, 0x50, 0x8c, 0x0e, 0x66, 0xdc, 0xa2,
+	0xd3, 0x7a, 0x41, 0x1d, 0xca, 0xe4, 0x63, 0x98, 0xf6, 0x28, 0x3f, 0x67, 0x18, 0x71, 0x72, 0x44,
+	0x93, 0x88, 0xa8, 0x28, 0x71, 0x33, 0x7e, 0xa2, 0x4b, 0xd5, 0x7f, 0x79, 0x84, 0xa4, 0xea, 0xbf,
+	0xcf, 0x01, 0xac, 0x38, 0xd3, 0xa9, 0x57, 0xc6, 0xb3, 0xf1, 0xba, 0x26, 0x49, 0x50, 0x4f, 0xc1,
+	0xc8, 0x27, 0x90, 0x37, 0x71, 0xc5, 0xd5, 0xab, 0xe0, 0xe9, 0x81, 0x40, 0xd0, 0x95, 0x9a, 0x2c,
+	0x43, 0xd1, 0x0f, 0x1c, 0x16, 0x38, 0xfc, 0x12, 0xc3, 0x6b, 0x5a, 0x8f, 0xdb, 0xa9, 0xa7, 0xa5,
+	0x72, 0xe6, 0x69, 0x29, 0x55, 0xc9, 0x56, 0x32, 0x95, 0xec, 0x12, 0x14, 0xbb, 0x01, 0xeb, 0xfb,
+	0x62, 0x1c, 0x2a, 0x97, 0x60, 0x5b, 0x4e, 0x46, 0xfa, 0x13, 0x9c, 0x40, 0x4c, 0x23, 0xa2, 0x22,
+	0xc4, 0x87, 0x52, 0xda, 0xb4, 0xc9, 0x47, 0x50, 0x0d, 0xa8, 0xef, 0x8a, 0xa7, 0x4c, 0x8a, 0x0b,
+	0x83, 0x25, 0x61, 0x51, 0xaf, 0xc4, 0x52, 0x0c, 0x96, 0x5d, 0x98, 0x16, 0x31, 0x26, 0x92, 0x83,
+	0x9a, 0xa9, 0xc5, 0x19, 0x3c, 0xcd, 0x57, 0x33, 0x1f, 0x6c, 0xd6, 0x44, 0xe8, 0xb5, 0xd9, 0x8e,
+	0x84, 0x34, 0x3c, 0x1e, 0x5c, 0xea, 0x15, 0x3f, 0x2d, 0x23, 0x8d, 0xa4, 0x1a, 0xe2, 0xcc, 0x30,
+	0x69, 0xb8, 0x38, 0x8b, 0x44, 0xf7, 0xb2, 0x44, 0x0a, 0xde, 0x66, 0x9b, 0x34, 0x94, 0x3c, 0x51,
+	0xb9, 0x84, 0xa2, 0xe5, 0xaf, 0x80, 0x0c, 0xfb, 0x12, 0xa7, 0xec, 0x29, 0xbd, 0x54, 0x87, 0x92,
+	0xf8, 0x53, 0x9c, 0xc6, 0x67, 0xa6, 0xdb, 0xa7, 0xd1, 0xb3, 0x01, 0x36, 0xbe, 0x1c, 0x7b, 0x96,
+	0x5b, 0xfe, 0x29, 0xcc, 0x0c, 0x39, 0x79, 0x1b, 0x41, 0x31, 0x45, 0xa0, 0xb5, 0xa1, 0x9c, 0xa9,
+	0x84, 0x57, 0x60, 0x4a, 0x96, 0xf3, 0xd1, 0x5e, 0x2a, 0xeb, 0x45, 0x29, 0x68, 0xda, 0xe2, 0xa9,
+	0x4f, 0x29, 0x43, 0x9f, 0x5a, 0x4e, 0xc7, 0xb1, 0xd4, 0x63, 0x42, 0x55, 0x8a, 0x8f, 0x94, 0x54,
+	0xfb, 0xef, 0x12, 0x54, 0xb3, 0x5f, 0xcd, 0xae, 0x7e, 0xde, 0x58, 0x82, 0x62, 0x70, 0x61, 0x1c,
+	0x5f, 0x72, 0x1a, 0x22, 0x5b, 0x5e, 0x2f, 0x04, 0x17, 0x2f, 0x44, 0x53, 0x04, 0x79, 0x70, 0x61,
+	0xf8, 0xf8, 0xc0, 0x12, 0xaa, 0xcd, 0x38, 0x15, 0x5c, 0xc8, 0x27, 0x98, 0x10, 0x53, 0xe9, 0x85,
+	0xd1, 0xb7, 0x4c, 0x71, 0x14, 0x29, 0xd0, 0x04, 0x82, 0xaa, 0xc1, 0xc5, 0x2b, 0x21, 0xce, 0x22,
+	0x7b, 0x19, 0xe4, 0x64, 0x84, 0xdc, 0x1f, 0x46, 0x1e, 0x67, 0x90, 0xf9, 0x08, 0xf9, 0x62, 0x18,
+	0x29, 0x5f, 0xe5, 0x46, 0xc8, 0x42, 0x84, 0xc4, 0x97, 0xb1, 0x11, 0x72, 0x05, 0xa6, 0x82, 0x0b,
+	0xa3, 0x13, 0x98, 0x3d, 0x1a, 0xe2, 0x43, 0x48, 0x5e, 0x2f, 0x06, 0x17, 0x2f, 0xb1, 0x2d, 0x4e,
+	0xac, 0x58, 0x69, 0x3c, 0x7d, 0xa2, 0xf2, 0x09, 0x44, 0xfa, 0xa7, 0x4f, 0xc8, 0x27, 0xe8, 0x28,
+	0x42, 0x6c, 0x18, 0xeb, 0xf5, 0x2f, 0xf0, 0xc1, 0x24, 0xaf, 0x57, 0x62, 0xd4, 0xc6, 0x7a, 0xfd,
+	0x0b, 0xf2, 0x03, 0x98, 0x49, 0x80, 0xeb, 0xf5, 0x67, 0x46, 0x7d, 0x63, 0x63, 0x71, 0x2e, 0xea,
+	0x92, 0x44, 0xae, 0xd7, 0x9f, 0xd5, 0x37, 0x36, 0xb2, 0xd0, 0xfa, 0xc6, 0x53, 0x63, 0x63, 0x7d,
+	0x7d, 0x71, 0x3e, 0x0b, 0xad, 0x6f, 0x3c, 0xdd, 0x58, 0x5f, 0x27, 0x3f, 0x04, 0x92, 0x40, 0x37,
+	0xd6, 0xeb, 0xc6, 0xfa, 0xe3, 0xfa, 0xe7, 0x8b, 0x0b, 0x32, 0xed, 0x45, 0xd8, 0x8d, 0xf5, 0xba,
+	0x10, 0x93, 0xcf, 0x60, 0x36, 0xd5, 0x85, 0xc7, 0xf5, 0x27, 0xc6, 0xfa, 0xc6, 0xfa, 0xb3, 0xc5,
+	0xdb, 0x88, 0xae, 0xc5, 0x9d, 0x78, 0x5c, 0x7f, 0x22, 0xe4, 0x03, 0xf0, 0x8d, 0xf5, 0xe7, 0x46,
+	0xfd, 0xf1, 0x93, 0x2f, 0x16, 0x17, 0x07, 0xe0, 0x1b, 0xeb, 0xcf, 0x85, 0x3c, 0x0b, 0xaf, 0x3f,
+	0x7e, 0xf2, 0xcc, 0x78, 0xf2, 0xf8, 0xf9, 0xc6, 0xe2, 0x52, 0x16, 0x2e, 0x14, 0x42, 0x9e, 0x85,
+	0x3f, 0x79, 0xfc, 0xfc, 0xa9, 0xf1, 0xbc, 0xbe, 0xfe, 0x74, 0x71, 0x39, 0x0b, 0x17, 0x0a, 0x21,
+	0x27, 0x8f, 0x60, 0x2e, 0x81, 0x3f, 0xaf, 0xaf, 0x7f, 0x61, 0xac, 0x3f, 0xfd, 0xfc, 0xd9, 0xe7,
+	0x8b, 0x2b, 0x88, 0x9f, 0x89, 0xf0, 0x42, 0x83, 0x0a, 0x71, 0xdc, 0x07, 0x17, 0x86, 0x15, 0x58,
+	0x32, 0x0a, 0x42, 0x4c, 0x5f, 0x79, 0xbd, 0x14, 0x5c, 0x6c, 0x05, 0x16, 0x46, 0x00, 0x96, 0x76,
+	0x3c, 0x8a, 0xee, 0xa2, 0x8c, 0x6e, 0x9e, 0x44, 0x37, 0x4f, 0xa2, 0x7b, 0x4a, 0x46, 0x37, 0x4f,
+	0x47, 0x37, 0x1f, 0x8c, 0x6e, 0x90, 0x2b, 0xc4, 0x87, 0xa2, 0x9b, 0x0f, 0x46, 0x77, 0x29, 0x42,
+	0xee, 0x0f, 0x23, 0xb3, 0xd1, 0x5d, 0x8e, 0x90, 0x2f, 0x86, 0x91, 0xd9, 0xe8, 0xae, 0x44, 0xc8,
+	0xc1, 0xe8, 0xe6, 0x71, 0x74, 0xdf, 0x91, 0xd1, 0xcd, 0x53, 0xd1, 0xcd, 0xd3, 0xd1, 0xfd, 0x81,
+	0x8c, 0x6e, 0x9e, 0x89, 0x6e, 0x3e, 0x18, 0xdd, 0x77, 0x65, 0x74, 0xf3, 0xc1, 0xe8, 0xe6, 0x43,
+	0xd1, 0x7d, 0x2f, 0xea, 0xd2, 0x60, 0x74, 0xf3, 0xa1, 0xe8, 0x5e, 0xcd, 0x42, 0x93, 0xe8, 0xe6,
+	0xc3, 0xd1, 0x7d, 0x5f, 0x46, 0x37, 0x1f, 0x8e, 0x6e, 0x3e, 0x22, 0xba, 0x35, 0x19, 0x50, 0x7c,
+	0x44, 0x74, 0xf3, 0x11, 0xd1, 0xfd, 0xe1, 0x00, 0x3c, 0x15, 0xdd, 0x7c, 0x44, 0x74, 0x3f, 0xc8,
+	0xc2, 0xd3, 0xd1, 0xcd, 0x47, 0x44, 0xf7, 0x47, 0x59, 0x78, 0x3a, 0xba, 0xf9, 0xa8, 0xe8, 0xfe,
+	0x58, 0x46, 0x37, 0x1f, 0x8a, 0xee, 0x0f, 0x00, 0x8e, 0x1d, 0x3f, 0x0a, 0xed, 0x69, 0x19, 0x9e,
+	0xc7, 0x8e, 0xaf, 0x02, 0xfb, 0x0e, 0x4c, 0x71, 0xa7, 0x47, 0x43, 0x6e, 0xf6, 0x7c, 0x3c, 0x6e,
+	0x0b, 0x7a, 0x22, 0xd0, 0xfe, 0xad, 0x80, 0x5f, 0x32, 0xde, 0x25, 0xff, 0x5f, 0xf1, 0xe6, 0xeb,
+	0x23, 0xa8, 0xfa, 0x2c, 0x74, 0xb8, 0x73, 0x46, 0xe5, 0xd7, 0x75, 0x95, 0xff, 0x2b, 0x91, 0x14,
+	0xbf, 0x96, 0x0b, 0x98, 0x47, 0xbb, 0x66, 0x0a, 0x26, 0x4f, 0x80, 0x4a, 0x24, 0x95, 0xb0, 0x67,
+	0xb0, 0x68, 0x53, 0xd7, 0xe9, 0x39, 0xa2, 0x2a, 0xee, 0x39, 0x61, 0x68, 0xd8, 0x94, 0x53, 0x2b,
+	0x7e, 0x71, 0x99, 0xd7, 0x17, 0x62, 0xfd, 0xbe, 0x13, 0x86, 0xdb, 0x91, 0x76, 0x60, 0x1a, 0xf2,
+	0x83, 0xd3, 0xb0, 0x02, 0xa2, 0x61, 0xf4, 0x3d, 0x27, 0x4e, 0xff, 0xc5, 0x63, 0xc7, 0x7f, 0x25,
+	0xda, 0xa4, 0x0e, 0xf3, 0x1d, 0x6a, 0x19, 0x16, 0x0b, 0x02, 0x7c, 0x69, 0x64, 0x84, 0x97, 0xbd,
+	0x63, 0xe6, 0x46, 0x99, 0x60, 0xb6, 0x43, 0xad, 0xad, 0x48, 0x77, 0x24, 0x55, 0xe4, 0x29, 0xdc,
+	0x96, 0x36, 0x36, 0x3d, 0x67, 0x81, 0x1d, 0x26, 0xd6, 0x2a, 0x45, 0xcc, 0xa3, 0x95, 0xd2, 0xc6,
+	0xe6, 0xe4, 0x27, 0xb0, 0x92, 0xb5, 0xeb, 0x7b, 0xca, 0xd2, 0x3c, 0x76, 0xa9, 0xca, 0x1c, 0x4b,
+	0x69, 0xdb, 0x57, 0x69, 0x00, 0xf9, 0x10, 0x2a, 0x19, 0x7b, 0x95, 0x41, 0xca, 0x69, 0x0b, 0xf1,
+	0x88, 0x91, 0x1d, 0x90, 0x1c, 0xb7, 0x4c, 0x21, 0x33, 0xe9, 0xe1, 0xc8, 0x09, 0xf8, 0x18, 0xa6,
+	0x2f, 0xba, 0xb4, 0x67, 0x9c, 0xd2, 0xcb, 0x68, 0x06, 0x65, 0x12, 0xa9, 0x08, 0xf1, 0xd7, 0xf4,
+	0x32, 0x99, 0x45, 0xc4, 0xb9, 0x2c, 0x8c, 0xb2, 0x68, 0x51, 0x08, 0xf6, 0x58, 0x88, 0x24, 0xa2,
+	0x0a, 0x70, 0x99, 0xd9, 0x0b, 0x25, 0x8b, 0x8a, 0xc6, 0x4a, 0x70, 0x71, 0x88, 0x52, 0x64, 0x51,
+	0x07, 0x95, 0xc2, 0x79, 0xcc, 0x33, 0x1c, 0xdb, 0xa5, 0x18, 0x9a, 0x78, 0x50, 0x49, 0xe8, 0x01,
+	0xf3, 0x9a, 0xb6, 0x8b, 0xe5, 0x68, 0x70, 0x81, 0x97, 0x2a, 0xd4, 0x89, 0x9c, 0x0f, 0x2e, 0x5a,
+	0x3d, 0xcb, 0x21, 0xcf, 0x60, 0x49, 0x29, 0xa2, 0xbc, 0x97, 0x64, 0x78, 0x75, 0x38, 0xcf, 0x4b,
+	0xa8, 0x4a, 0x80, 0x51, 0xae, 0xcf, 0x14, 0x32, 0xb3, 0xd7, 0x15, 0x32, 0x73, 0x83, 0x85, 0x4c,
+	0xfa, 0x90, 0x98, 0xbf, 0xee, 0x90, 0x58, 0x18, 0x3c, 0x24, 0xee, 0x43, 0xf9, 0x98, 0x06, 0x46,
+	0x40, 0x45, 0x09, 0x48, 0x6d, 0x75, 0xd0, 0x96, 0x8e, 0x69, 0xa0, 0x2b, 0x11, 0xb9, 0x07, 0x25,
+	0xd7, 0xb2, 0xbb, 0xd1, 0xfc, 0xcb, 0xb3, 0x15, 0x84, 0x48, 0x4d, 0xbe, 0xe8, 0x9c, 0xed, 0x44,
+	0xfa, 0x25, 0xd5, 0x39, 0xdb, 0x19, 0xb5, 0xd1, 0x97, 0x07, 0x37, 0xfa, 0xbf, 0xe4, 0xb0, 0x02,
+	0x7d, 0xd7, 0x62, 0xef, 0x2d, 0x1f, 0x67, 0xde, 0x52, 0xf0, 0xa5, 0x67, 0x78, 0x62, 0x68, 0x86,
+	0x53, 0xf3, 0x34, 0x39, 0x38, 0x4f, 0xe9, 0x19, 0xce, 0x67, 0x67, 0xf8, 0xfa, 0xf1, 0xfd, 0x5d,
+	0x0e, 0xaa, 0xd9, 0x5b, 0x5c, 0xe9, 0xe7, 0xc3, 0x5c, 0xe6, 0x0b, 0xd2, 0xf7, 0xaf, 0x64, 0xbf,
+	0x7f, 0x95, 0x70, 0x7d, 0x1a, 0xfe, 0x0a, 0x2a, 0x99, 0x6b, 0x5f, 0x57, 0x2f, 0xcc, 0x02, 0xe4,
+	0x43, 0x6e, 0xf2, 0x7e, 0xa8, 0xde, 0x7d, 0xaa, 0x96, 0xf6, 0x2d, 0xcc, 0x8e, 0xb8, 0xfe, 0xf5,
+	0xde, 0xd9, 0x3c, 0xa1, 0x1f, 0xcf, 0xd0, 0xff, 0xd5, 0x18, 0x7e, 0x1e, 0x1a, 0xbc, 0xc6, 0xf6,
+	0x3d, 0x3e, 0x7b, 0xbb, 0x2c, 0x34, 0x32, 0x2e, 0xa6, 0x5c, 0x16, 0x1e, 0xa1, 0x40, 0xaa, 0x8f,
+	0x23, 0xf5, 0x44, 0xa4, 0x3e, 0x56, 0xea, 0x87, 0x50, 0x73, 0x99, 0x6f, 0xc9, 0x73, 0x41, 0x81,
+	0xe4, 0x3b, 0xfb, 0xaa, 0x90, 0x8b, 0xf3, 0x40, 0x21, 0xd7, 0x61, 0x5e, 0x21, 0x55, 0x46, 0x88,
+	0xe0, 0x79, 0xf9, 0x69, 0x41, 0xc2, 0x65, 0x3e, 0x50, 0x26, 0x62, 0xfb, 0xb1, 0x8e, 0x13, 0x01,
+	0x0b, 0xf2, 0x75, 0x98, 0x10, 0x29, 0xc0, 0x7d, 0x28, 0x8b, 0xcc, 0x14, 0x23, 0x8a, 0x88, 0x28,
+	0xa1, 0x4c, 0x42, 0x34, 0x0a, 0x2b, 0xd7, 0x5c, 0x7a, 0xbb, 0xb1, 0xc5, 0xf8, 0xb3, 0x1c, 0x2c,
+	0x5f, 0x7d, 0x03, 0xee, 0xa6, 0xdc, 0x90, 0xcf, 0x61, 0xc1, 0xf1, 0xce, 0x68, 0x10, 0x52, 0x43,
+	0x3c, 0x8d, 0xcb, 0x79, 0x0c, 0x4c, 0x1e, 0x7d, 0x92, 0x9b, 0x55, 0xda, 0x17, 0x8e, 0xbc, 0xd3,
+	0xa2, 0x9b, 0x9c, 0x6a, 0xbf, 0x91, 0x7d, 0xbb, 0xe2, 0x02, 0xdd, 0x8d, 0xf5, 0x6d, 0x0e, 0x26,
+	0x93, 0x2a, 0xa2, 0xa0, 0xcb, 0x86, 0x60, 0xf7, 0xe8, 0xb9, 0x41, 0xbf, 0x8b, 0xde, 0x5a, 0xe5,
+	0x3d, 0x7a, 0xde, 0xf8, 0xce, 0xd6, 0x4e, 0xe0, 0xee, 0xf5, 0xd7, 0xef, 0x6e, 0x6c, 0x6d, 0xfe,
+	0x3c, 0x27, 0x63, 0xe0, 0x8a, 0x0b, 0x79, 0xff, 0xbf, 0x8b, 0xf3, 0xab, 0x1c, 0x68, 0x6f, 0xbf,
+	0xdc, 0xf7, 0x7f, 0xbb, 0x48, 0xda, 0x77, 0xb8, 0x16, 0xd7, 0x5c, 0x02, 0x7c, 0x6f, 0xff, 0xf7,
+	0xb2, 0x17, 0x5a, 0xe4, 0xeb, 0xcd, 0xf4, 0x1d, 0x95, 0x53, 0xb8, 0xff, 0xd6, 0x1b, 0x7b, 0x37,
+	0x16, 0x01, 0x6d, 0x20, 0xba, 0x3a, 0x94, 0x53, 0xec, 0xa2, 0x38, 0x8a, 0x0e, 0x6f, 0xc3, 0x62,
+	0x7d, 0x8f, 0xa3, 0x17, 0x51, 0x1c, 0x29, 0xf0, 0x96, 0x10, 0x5e, 0x99, 0xdf, 0xff, 0x34, 0x07,
+	0x8b, 0x57, 0x5d, 0x09, 0x7c, 0xef, 0xae, 0x6f, 0x42, 0x25, 0xe9, 0xcc, 0xa8, 0x4b, 0xc0, 0xc3,
+	0x03, 0xd8, 0xbd, 0xa5, 0x97, 0x82, 0x44, 0xfa, 0xa2, 0x80, 0x1f, 0x9e, 0x78, 0xa8, 0x1d, 0xc0,
+	0x9d, 0xeb, 0x2e, 0x5c, 0xbe, 0x6f, 0xdf, 0xb4, 0x5f, 0xc2, 0xea, 0xdb, 0x2e, 0x27, 0xde, 0xd8,
+	0x52, 0xfd, 0x12, 0x96, 0xae, 0xbc, 0xa1, 0xf8, 0x7d, 0xce, 0xb6, 0x54, 0x79, 0x36, 0x3e, 0x50,
+	0x9e, 0x69, 0x7f, 0x99, 0x83, 0x87, 0xef, 0x7a, 0x5d, 0xf1, 0xc6, 0x76, 0xe0, 0x67, 0x40, 0xd2,
+	0x57, 0x28, 0x55, 0xdf, 0xe4, 0x76, 0x9c, 0x49, 0x69, 0x54, 0x1f, 0x7b, 0xf0, 0xe1, 0x3b, 0x5c,
+	0x6c, 0xbc, 0xb1, 0xe9, 0x77, 0x31, 0x1b, 0xbd, 0xe5, 0x72, 0xe3, 0x8d, 0x79, 0xfb, 0xe3, 0x1c,
+	0x7c, 0xfc, 0x6e, 0xd7, 0x1c, 0x6f, 0x6c, 0xfa, 0x97, 0xa1, 0x38, 0x70, 0x8d, 0x25, 0x6e, 0x6b,
+	0xff, 0x99, 0x83, 0xd2, 0x4e, 0xc0, 0xfa, 0xfe, 0x3e, 0xc5, 0x17, 0xba, 0xf7, 0xa1, 0xec, 0x44,
+	0x77, 0x92, 0x22, 0xc7, 0x15, 0xfc, 0x71, 0x87, 0x94, 0x35, 0x6d, 0xd2, 0x84, 0x6a, 0x02, 0xc1,
+	0x0f, 0x16, 0xf2, 0x43, 0x72, 0x72, 0xdf, 0x36, 0x45, 0xb8, 0x16, 0xdf, 0x70, 0xc2, 0x2f, 0xc6,
+	0x15, 0x27, 0xdd, 0x24, 0x77, 0xa1, 0x24, 0x9e, 0xe3, 0xa2, 0x02, 0x7f, 0x1c, 0x9d, 0x89, 0x02,
+	0xff, 0x50, 0x16, 0xf8, 0xe9, 0x2f, 0x07, 0x13, 0xa8, 0x8c, 0xdb, 0xda, 0x6f, 0x43, 0x25, 0xc3,
+	0x4d, 0x0a, 0x30, 0x7e, 0xd8, 0x3a, 0xa8, 0xdd, 0x22, 0x35, 0x28, 0x37, 0x0e, 0x5b, 0x07, 0xc6,
+	0xfa, 0x8e, 0x71, 0xb8, 0xd9, 0xde, 0xad, 0xe5, 0xc8, 0x0c, 0x54, 0xa4, 0xe4, 0xb1, 0x12, 0x8d,
+	0x69, 0x7f, 0x38, 0x06, 0x93, 0xd8, 0xcf, 0xcc, 0x17, 0x05, 0x39, 0xdc, 0xf8, 0x8b, 0xc2, 0x8f,
+	0xa1, 0x60, 0xb1, 0x5e, 0xcf, 0x54, 0xbf, 0x72, 0x18, 0x1a, 0x63, 0x7a, 0xa4, 0xe1, 0x96, 0x44,
+	0xea, 0x91, 0x09, 0x59, 0x83, 0x42, 0x4f, 0xaa, 0xd4, 0x35, 0x80, 0xb9, 0x51, 0x33, 0xa4, 0x47,
+	0xa0, 0xd4, 0x07, 0x95, 0x89, 0x6b, 0x3f, 0xa8, 0x68, 0x5f, 0xc3, 0xec, 0x08, 0xc7, 0x64, 0x1a,
+	0x4a, 0x9b, 0xdb, 0xdb, 0xc6, 0x7e, 0x63, 0xff, 0x45, 0x43, 0x3f, 0xaa, 0xdd, 0x22, 0x04, 0xaa,
+	0x7a, 0x63, 0xbf, 0xf5, 0xba, 0x11, 0xcb, 0x72, 0x02, 0x74, 0xd4, 0x68, 0xc7, 0x82, 0x31, 0xed,
+	0x1b, 0x80, 0xd7, 0xa6, 0xdb, 0xa7, 0x87, 0x66, 0x60, 0xf6, 0xc8, 0x5d, 0x18, 0x67, 0x5e, 0x5f,
+	0x7d, 0xda, 0x2b, 0x67, 0x6e, 0x50, 0x0b, 0x05, 0xf9, 0x51, 0xfa, 0xd3, 0x40, 0xb5, 0xbe, 0xb0,
+	0xa6, 0x7e, 0xd2, 0x84, 0x14, 0x62, 0x19, 0xd6, 0x70, 0x9d, 0x25, 0x48, 0xfb, 0xfb, 0x31, 0xa8,
+	0x1e, 0x32, 0x4f, 0xbf, 0x38, 0x64, 0xe7, 0x34, 0xd8, 0x36, 0xb9, 0x79, 0x63, 0x41, 0xad, 0x67,
+	0x4f, 0xdb, 0x09, 0xec, 0xce, 0x7a, 0xea, 0x47, 0x38, 0x69, 0xaf, 0x6b, 0x7a, 0x18, 0x3a, 0xfb,
+	0xd4, 0x0c, 0xfb, 0x01, 0xee, 0xeb, 0xd1, 0x97, 0x48, 0xd5, 0x6b, 0x6e, 0x5f, 0x58, 0x19, 0x3d,
+	0x6a, 0x7a, 0x86, 0x7d, 0xdc, 0xc3, 0x12, 0x2e, 0xa7, 0x57, 0x03, 0xc9, 0xb6, 0x4f, 0x4d, 0x6f,
+	0xfb, 0xb8, 0x27, 0x72, 0xf6, 0x95, 0x9c, 0x64, 0x0e, 0x6a, 0x23, 0x2e, 0x99, 0xde, 0x81, 0xc5,
+	0xac, 0xd4, 0xd8, 0x6e, 0xec, 0x35, 0xf7, 0x9b, 0xed, 0x86, 0x5e, 0xcb, 0x91, 0x25, 0x98, 0x1f,
+	0xd0, 0x6e, 0x6e, 0x6d, 0x35, 0x8e, 0xc4, 0x0a, 0x15, 0x60, 0xb2, 0xd1, 0xf3, 0xf9, 0x65, 0xfd,
+	0x2f, 0xe6, 0xa0, 0xd0, 0x92, 0x03, 0x24, 0xdb, 0x00, 0xdb, 0x4e, 0x68, 0x1e, 0xbb, 0xb4, 0xe5,
+	0x72, 0x52, 0x8d, 0x07, 0x8e, 0xc8, 0xe5, 0x81, 0xb6, 0xb6, 0xf0, 0xab, 0x7f, 0xfe, 0xd7, 0xdf,
+	0x8c, 0xd5, 0xb4, 0xd2, 0xa3, 0xb3, 0xf5, 0x47, 0xca, 0xee, 0xcb, 0xdc, 0xa7, 0xe4, 0x25, 0x94,
+	0x74, 0x4a, 0xbd, 0x77, 0xa5, 0xb9, 0x8d, 0x34, 0x33, 0x5a, 0x59, 0xd0, 0x44, 0x86, 0x82, 0xa7,
+	0x01, 0x25, 0x55, 0x4a, 0xd1, 0x96, 0xd7, 0x27, 0x99, 0xc0, 0x19, 0x62, 0x59, 0x44, 0x16, 0xa2,
+	0x55, 0x04, 0x4b, 0x43, 0x3a, 0xf7, 0xfa, 0x82, 0x66, 0x17, 0x2a, 0xf1, 0x91, 0xfb, 0x0e, 0x44,
+	0x4b, 0x48, 0x34, 0xab, 0x55, 0x53, 0xa3, 0x52, 0x4c, 0x5b, 0x30, 0xb5, 0x4d, 0x5d, 0xfa, 0xde,
+	0xdd, 0x89, 0x8d, 0x04, 0x49, 0x13, 0x40, 0xdd, 0xec, 0x6b, 0xf5, 0x39, 0xa9, 0x65, 0x7e, 0x4f,
+	0xb6, 0x1f, 0x76, 0xaf, 0xef, 0x4f, 0x62, 0x29, 0xa8, 0x5a, 0x50, 0x8e, 0xaf, 0xf5, 0x09, 0x32,
+	0x92, 0xb9, 0xc3, 0x8f, 0xe2, 0x21, 0xba, 0x15, 0xa4, 0x9b, 0xd7, 0x6a, 0x48, 0x97, 0xb2, 0x16,
+	0x84, 0xbf, 0x07, 0xd3, 0xe9, 0x0b, 0x7a, 0x82, 0x33, 0xb9, 0x9c, 0x99, 0xd6, 0x0c, 0xd1, 0xde,
+	0x45, 0xda, 0x45, 0x6d, 0x56, 0xd0, 0x0e, 0x70, 0x08, 0xe6, 0xaf, 0xa0, 0xf0, 0xd2, 0x65, 0xe7,
+	0x9b, 0xb6, 0x4d, 0x2a, 0x99, 0x0f, 0x94, 0xd7, 0x47, 0x95, 0xb2, 0x91, 0x51, 0x05, 0xa2, 0xa5,
+	0xe3, 0x05, 0x85, 0xb7, 0x91, 0x64, 0x26, 0x2d, 0x31, 0x13, 0x3c, 0x47, 0x50, 0x8d, 0xaf, 0xbe,
+	0x6e, 0x9d, 0x50, 0xeb, 0x74, 0x28, 0x40, 0x93, 0x69, 0x8c, 0x81, 0xda, 0x07, 0x48, 0x78, 0x5b,
+	0x23, 0x82, 0x30, 0x6b, 0x2f, 0x48, 0xf7, 0xa1, 0x24, 0x63, 0xee, 0x90, 0x79, 0xcd, 0x4e, 0x6a,
+	0x21, 0xe2, 0xd3, 0x64, 0xa8, 0x8b, 0xcb, 0xc8, 0x38, 0xa7, 0x4d, 0x27, 0x01, 0x8b, 0xc6, 0x6a,
+	0x61, 0x55, 0xe4, 0xbd, 0x3b, 0x5f, 0x66, 0x61, 0xd3, 0xd6, 0x82, 0x50, 0x87, 0xca, 0x0e, 0xe5,
+	0xa9, 0x0b, 0x94, 0x83, 0x63, 0x9e, 0x1d, 0x71, 0x61, 0x4b, 0xbb, 0x83, 0x94, 0x0b, 0xda, 0x8c,
+	0xa0, 0xcc, 0xd8, 0x0b, 0xce, 0x9f, 0x42, 0x5e, 0xa7, 0xc7, 0x8c, 0xbd, 0x7d, 0x87, 0xcf, 0x23,
+	0xcf, 0xb4, 0x06, 0x72, 0x87, 0x0b, 0x1b, 0x41, 0xf0, 0x0a, 0x66, 0xb6, 0x98, 0xeb, 0x52, 0x2b,
+	0xfd, 0xda, 0xee, 0x6d, 0x5c, 0xab, 0xc8, 0xb5, 0xac, 0xcd, 0x0b, 0xae, 0x21, 0x73, 0x41, 0xfb,
+	0x33, 0xa8, 0xed, 0x50, 0x9e, 0x7d, 0xf3, 0x9f, 0xdd, 0xac, 0x0b, 0x03, 0xbf, 0xe3, 0x53, 0x28,
+	0xed, 0x1e, 0x72, 0x2f, 0x69, 0x73, 0x6a, 0xbc, 0x19, 0xad, 0xa0, 0x3e, 0x85, 0xb9, 0x1d, 0xca,
+	0x87, 0xdf, 0x35, 0x8e, 0xda, 0x78, 0xc9, 0x0f, 0x53, 0x87, 0xf0, 0xda, 0x87, 0xe8, 0xe8, 0x03,
+	0x6d, 0x51, 0x39, 0x1a, 0x42, 0x08, 0x67, 0x01, 0xdc, 0xde, 0x0a, 0xa8, 0xc9, 0x69, 0x3b, 0x30,
+	0x3b, 0x1d, 0xc7, 0x3a, 0xb2, 0x4e, 0xa8, 0xdd, 0x77, 0xc5, 0xa1, 0x7e, 0x6f, 0x2d, 0xf3, 0xb3,
+	0xe0, 0x21, 0xc0, 0xd0, 0xac, 0x7d, 0x8c, 0x0e, 0x57, 0xb5, 0x15, 0x9c, 0xb5, 0xd1, 0xac, 0xca,
+	0xa7, 0xdc, 0x29, 0x37, 0xed, 0xf3, 0x0a, 0x56, 0xe1, 0xb3, 0x03, 0xb3, 0x99, 0x1e, 0xfd, 0x6e,
+	0x9f, 0xf6, 0x69, 0x48, 0x56, 0x46, 0xfa, 0x93, 0xca, 0x21, 0x5f, 0x1a, 0xfa, 0xba, 0xa3, 0xdd,
+	0x1e, 0x1a, 0x9f, 0x34, 0x50, 0x7e, 0x32, 0xbd, 0xf8, 0x5f, 0xfb, 0x19, 0xc1, 0x26, 0xfc, 0xfc,
+	0x16, 0xd4, 0xe4, 0x76, 0x4e, 0x55, 0xdd, 0x57, 0x6f, 0xb7, 0x04, 0xa4, 0xdd, 0x7a, 0x9c, 0x23,
+	0xdf, 0xc2, 0xfc, 0x21, 0x0d, 0x3a, 0x2c, 0xe8, 0x61, 0x31, 0xd6, 0xf2, 0x69, 0x30, 0xc8, 0x80,
+	0x8a, 0xa1, 0x9e, 0x3d, 0xc0, 0x9e, 0xdd, 0xd5, 0x96, 0x44, 0xcf, 0x46, 0x52, 0xc8, 0xc3, 0xa7,
+	0x24, 0x0f, 0x23, 0x59, 0xa5, 0xbe, 0x8d, 0x34, 0x93, 0xa3, 0x52, 0x86, 0x82, 0xaa, 0x0d, 0xa5,
+	0x1d, 0xca, 0x1b, 0x17, 0x1c, 0xab, 0x34, 0x92, 0x8c, 0x28, 0x29, 0xfc, 0x96, 0xe7, 0xa2, 0x4a,
+	0x4e, 0xa7, 0xbc, 0x1f, 0x78, 0xa8, 0x09, 0xb3, 0xac, 0x29, 0x0e, 0xc1, 0xfa, 0x2d, 0xfe, 0xd0,
+	0x49, 0xbe, 0x0a, 0xc0, 0x37, 0xb2, 0x47, 0x94, 0x13, 0x51, 0x10, 0xe2, 0x6f, 0xd8, 0xb3, 0xaa,
+	0xeb, 0x73, 0xc3, 0x10, 0x8d, 0xa0, 0xf7, 0x61, 0x69, 0x87, 0xf2, 0xbd, 0xd1, 0xb7, 0xf4, 0xb3,
+	0x49, 0x62, 0x25, 0xfb, 0x2b, 0xb5, 0xcc, 0x6f, 0x07, 0xb4, 0x87, 0xe8, 0x49, 0xd3, 0x3e, 0x50,
+	0x43, 0x18, 0xcd, 0x28, 0x3c, 0x9e, 0xc0, 0xfc, 0x48, 0xfd, 0xfb, 0x78, 0xcb, 0xac, 0xed, 0x48,
+	0x36, 0xb9, 0x20, 0x22, 0xc7, 0x27, 0x35, 0xea, 0x80, 0x87, 0xdb, 0x57, 0x94, 0xb1, 0x43, 0x59,
+	0x3e, 0x51, 0x7f, 0x99, 0xfb, 0xf4, 0xc5, 0xb7, 0xb0, 0xc2, 0x82, 0x2e, 0xda, 0x5a, 0x2c, 0xb0,
+	0xd7, 0xe4, 0xff, 0x2f, 0x10, 0x71, 0xbd, 0xa8, 0xbc, 0xc6, 0xb6, 0x28, 0x20, 0x5b, 0x7b, 0xed,
+	0x6f, 0x1e, 0x75, 0x1d, 0x7e, 0xd2, 0x3f, 0x16, 0xcb, 0xfe, 0x28, 0x32, 0x79, 0x24, 0x4d, 0x3e,
+	0x53, 0xff, 0x25, 0xc1, 0xd9, 0xc6, 0xa3, 0x2e, 0x8b, 0xfe, 0x6b, 0x84, 0xc3, 0xdc, 0xe1, 0xd8,
+	0x71, 0x1e, 0x35, 0x9f, 0xff, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x07, 0x3d, 0xbf, 0x02, 0x3e,
+	0x41, 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
+
+// OpenoltClient is the client API for Openolt service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type OpenoltClient interface {
+	DisableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
+	ReenableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
+	ActivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
+	DeactivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
+	DeleteOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error)
+	OmciMsgOut(ctx context.Context, in *OmciMsg, opts ...grpc.CallOption) (*Empty, error)
+	OnuPacketOut(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*Empty, error)
+	UplinkPacketOut(ctx context.Context, in *UplinkPacket, opts ...grpc.CallOption) (*Empty, error)
+	FlowAdd(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error)
+	FlowRemove(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error)
+	HeartbeatCheck(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Heartbeat, error)
+	EnablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error)
+	DisablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error)
+	GetDeviceInfo(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*DeviceInfo, error)
+	Reboot(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
+	CollectStatistics(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error)
+	GetOnuStatistics(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuStatistics, error)
+	GetGemPortStatistics(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*GemPortStatistics, error)
+	CreateTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error)
+	RemoveTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error)
+	CreateTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error)
+	RemoveTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error)
+	EnableIndication(ctx context.Context, in *Empty, opts ...grpc.CallOption) (Openolt_EnableIndicationClient, error)
+	PerformGroupOperation(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error)
+	DeleteGroup(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error)
+	GetExtValue(ctx context.Context, in *ValueParam, opts ...grpc.CallOption) (*common.ReturnValues, error)
+	OnuItuPonAlarmSet(ctx context.Context, in *config.OnuItuPonAlarm, opts ...grpc.CallOption) (*Empty, error)
+	GetLogicalOnuDistanceZero(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error)
+	GetLogicalOnuDistance(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error)
+	GetPonRxPower(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*PonRxPowerData, error)
+}
+
+type openoltClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewOpenoltClient(cc *grpc.ClientConn) OpenoltClient {
+	return &openoltClient{cc}
+}
+
+func (c *openoltClient) DisableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/DisableOlt", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) ReenableOlt(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/ReenableOlt", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) ActivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/ActivateOnu", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) DeactivateOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeactivateOnu", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) DeleteOnu(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeleteOnu", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) OmciMsgOut(ctx context.Context, in *OmciMsg, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/OmciMsgOut", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) OnuPacketOut(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/OnuPacketOut", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) UplinkPacketOut(ctx context.Context, in *UplinkPacket, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/UplinkPacketOut", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) FlowAdd(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/FlowAdd", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) FlowRemove(ctx context.Context, in *Flow, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/FlowRemove", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) HeartbeatCheck(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Heartbeat, error) {
+	out := new(Heartbeat)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/HeartbeatCheck", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) EnablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/EnablePonIf", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) DisablePonIf(ctx context.Context, in *Interface, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/DisablePonIf", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetDeviceInfo(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*DeviceInfo, error) {
+	out := new(DeviceInfo)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetDeviceInfo", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) Reboot(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/Reboot", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) CollectStatistics(ctx context.Context, in *Empty, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/CollectStatistics", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetOnuStatistics(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuStatistics, error) {
+	out := new(OnuStatistics)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetOnuStatistics", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetGemPortStatistics(ctx context.Context, in *OnuPacket, opts ...grpc.CallOption) (*GemPortStatistics, error) {
+	out := new(GemPortStatistics)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetGemPortStatistics", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) CreateTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/CreateTrafficSchedulers", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) RemoveTrafficSchedulers(ctx context.Context, in *tech_profile.TrafficSchedulers, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/RemoveTrafficSchedulers", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) CreateTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/CreateTrafficQueues", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) RemoveTrafficQueues(ctx context.Context, in *tech_profile.TrafficQueues, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/RemoveTrafficQueues", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) EnableIndication(ctx context.Context, in *Empty, opts ...grpc.CallOption) (Openolt_EnableIndicationClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_Openolt_serviceDesc.Streams[0], "/openolt.Openolt/EnableIndication", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &openoltEnableIndicationClient{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 Openolt_EnableIndicationClient interface {
+	Recv() (*Indication, error)
+	grpc.ClientStream
+}
+
+type openoltEnableIndicationClient struct {
+	grpc.ClientStream
+}
+
+func (x *openoltEnableIndicationClient) Recv() (*Indication, error) {
+	m := new(Indication)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *openoltClient) PerformGroupOperation(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/PerformGroupOperation", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) DeleteGroup(ctx context.Context, in *Group, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/DeleteGroup", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetExtValue(ctx context.Context, in *ValueParam, opts ...grpc.CallOption) (*common.ReturnValues, error) {
+	out := new(common.ReturnValues)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) OnuItuPonAlarmSet(ctx context.Context, in *config.OnuItuPonAlarm, opts ...grpc.CallOption) (*Empty, error) {
+	out := new(Empty)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/OnuItuPonAlarmSet", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetLogicalOnuDistanceZero(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error) {
+	out := new(OnuLogicalDistance)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetLogicalOnuDistanceZero", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetLogicalOnuDistance(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*OnuLogicalDistance, error) {
+	out := new(OnuLogicalDistance)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetLogicalOnuDistance", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *openoltClient) GetPonRxPower(ctx context.Context, in *Onu, opts ...grpc.CallOption) (*PonRxPowerData, error) {
+	out := new(PonRxPowerData)
+	err := c.cc.Invoke(ctx, "/openolt.Openolt/GetPonRxPower", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// OpenoltServer is the server API for Openolt service.
+type OpenoltServer interface {
+	DisableOlt(context.Context, *Empty) (*Empty, error)
+	ReenableOlt(context.Context, *Empty) (*Empty, error)
+	ActivateOnu(context.Context, *Onu) (*Empty, error)
+	DeactivateOnu(context.Context, *Onu) (*Empty, error)
+	DeleteOnu(context.Context, *Onu) (*Empty, error)
+	OmciMsgOut(context.Context, *OmciMsg) (*Empty, error)
+	OnuPacketOut(context.Context, *OnuPacket) (*Empty, error)
+	UplinkPacketOut(context.Context, *UplinkPacket) (*Empty, error)
+	FlowAdd(context.Context, *Flow) (*Empty, error)
+	FlowRemove(context.Context, *Flow) (*Empty, error)
+	HeartbeatCheck(context.Context, *Empty) (*Heartbeat, error)
+	EnablePonIf(context.Context, *Interface) (*Empty, error)
+	DisablePonIf(context.Context, *Interface) (*Empty, error)
+	GetDeviceInfo(context.Context, *Empty) (*DeviceInfo, error)
+	Reboot(context.Context, *Empty) (*Empty, error)
+	CollectStatistics(context.Context, *Empty) (*Empty, error)
+	GetOnuStatistics(context.Context, *Onu) (*OnuStatistics, error)
+	GetGemPortStatistics(context.Context, *OnuPacket) (*GemPortStatistics, error)
+	CreateTrafficSchedulers(context.Context, *tech_profile.TrafficSchedulers) (*Empty, error)
+	RemoveTrafficSchedulers(context.Context, *tech_profile.TrafficSchedulers) (*Empty, error)
+	CreateTrafficQueues(context.Context, *tech_profile.TrafficQueues) (*Empty, error)
+	RemoveTrafficQueues(context.Context, *tech_profile.TrafficQueues) (*Empty, error)
+	EnableIndication(*Empty, Openolt_EnableIndicationServer) error
+	PerformGroupOperation(context.Context, *Group) (*Empty, error)
+	DeleteGroup(context.Context, *Group) (*Empty, error)
+	GetExtValue(context.Context, *ValueParam) (*common.ReturnValues, error)
+	OnuItuPonAlarmSet(context.Context, *config.OnuItuPonAlarm) (*Empty, error)
+	GetLogicalOnuDistanceZero(context.Context, *Onu) (*OnuLogicalDistance, error)
+	GetLogicalOnuDistance(context.Context, *Onu) (*OnuLogicalDistance, error)
+	GetPonRxPower(context.Context, *Onu) (*PonRxPowerData, error)
+}
+
+// UnimplementedOpenoltServer can be embedded to have forward compatible implementations.
+type UnimplementedOpenoltServer struct {
+}
+
+func (*UnimplementedOpenoltServer) DisableOlt(ctx context.Context, req *Empty) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisableOlt not implemented")
+}
+func (*UnimplementedOpenoltServer) ReenableOlt(ctx context.Context, req *Empty) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ReenableOlt not implemented")
+}
+func (*UnimplementedOpenoltServer) ActivateOnu(ctx context.Context, req *Onu) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ActivateOnu not implemented")
+}
+func (*UnimplementedOpenoltServer) DeactivateOnu(ctx context.Context, req *Onu) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeactivateOnu not implemented")
+}
+func (*UnimplementedOpenoltServer) DeleteOnu(ctx context.Context, req *Onu) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteOnu not implemented")
+}
+func (*UnimplementedOpenoltServer) OmciMsgOut(ctx context.Context, req *OmciMsg) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method OmciMsgOut not implemented")
+}
+func (*UnimplementedOpenoltServer) OnuPacketOut(ctx context.Context, req *OnuPacket) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method OnuPacketOut not implemented")
+}
+func (*UnimplementedOpenoltServer) UplinkPacketOut(ctx context.Context, req *UplinkPacket) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UplinkPacketOut not implemented")
+}
+func (*UnimplementedOpenoltServer) FlowAdd(ctx context.Context, req *Flow) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method FlowAdd not implemented")
+}
+func (*UnimplementedOpenoltServer) FlowRemove(ctx context.Context, req *Flow) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method FlowRemove not implemented")
+}
+func (*UnimplementedOpenoltServer) HeartbeatCheck(ctx context.Context, req *Empty) (*Heartbeat, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method HeartbeatCheck not implemented")
+}
+func (*UnimplementedOpenoltServer) EnablePonIf(ctx context.Context, req *Interface) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method EnablePonIf not implemented")
+}
+func (*UnimplementedOpenoltServer) DisablePonIf(ctx context.Context, req *Interface) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisablePonIf not implemented")
+}
+func (*UnimplementedOpenoltServer) GetDeviceInfo(ctx context.Context, req *Empty) (*DeviceInfo, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceInfo not implemented")
+}
+func (*UnimplementedOpenoltServer) Reboot(ctx context.Context, req *Empty) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Reboot not implemented")
+}
+func (*UnimplementedOpenoltServer) CollectStatistics(ctx context.Context, req *Empty) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CollectStatistics not implemented")
+}
+func (*UnimplementedOpenoltServer) GetOnuStatistics(ctx context.Context, req *Onu) (*OnuStatistics, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetOnuStatistics not implemented")
+}
+func (*UnimplementedOpenoltServer) GetGemPortStatistics(ctx context.Context, req *OnuPacket) (*GemPortStatistics, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetGemPortStatistics not implemented")
+}
+func (*UnimplementedOpenoltServer) CreateTrafficSchedulers(ctx context.Context, req *tech_profile.TrafficSchedulers) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CreateTrafficSchedulers not implemented")
+}
+func (*UnimplementedOpenoltServer) RemoveTrafficSchedulers(ctx context.Context, req *tech_profile.TrafficSchedulers) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RemoveTrafficSchedulers not implemented")
+}
+func (*UnimplementedOpenoltServer) CreateTrafficQueues(ctx context.Context, req *tech_profile.TrafficQueues) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CreateTrafficQueues not implemented")
+}
+func (*UnimplementedOpenoltServer) RemoveTrafficQueues(ctx context.Context, req *tech_profile.TrafficQueues) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RemoveTrafficQueues not implemented")
+}
+func (*UnimplementedOpenoltServer) EnableIndication(req *Empty, srv Openolt_EnableIndicationServer) error {
+	return status.Errorf(codes.Unimplemented, "method EnableIndication not implemented")
+}
+func (*UnimplementedOpenoltServer) PerformGroupOperation(ctx context.Context, req *Group) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method PerformGroupOperation not implemented")
+}
+func (*UnimplementedOpenoltServer) DeleteGroup(ctx context.Context, req *Group) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteGroup not implemented")
+}
+func (*UnimplementedOpenoltServer) GetExtValue(ctx context.Context, req *ValueParam) (*common.ReturnValues, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
+}
+func (*UnimplementedOpenoltServer) OnuItuPonAlarmSet(ctx context.Context, req *config.OnuItuPonAlarm) (*Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method OnuItuPonAlarmSet not implemented")
+}
+func (*UnimplementedOpenoltServer) GetLogicalOnuDistanceZero(ctx context.Context, req *Onu) (*OnuLogicalDistance, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalOnuDistanceZero not implemented")
+}
+func (*UnimplementedOpenoltServer) GetLogicalOnuDistance(ctx context.Context, req *Onu) (*OnuLogicalDistance, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalOnuDistance not implemented")
+}
+func (*UnimplementedOpenoltServer) GetPonRxPower(ctx context.Context, req *Onu) (*PonRxPowerData, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetPonRxPower not implemented")
+}
+
+func RegisterOpenoltServer(s *grpc.Server, srv OpenoltServer) {
+	s.RegisterService(&_Openolt_serviceDesc, srv)
+}
+
+func _Openolt_DisableOlt_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).DisableOlt(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/DisableOlt",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).DisableOlt(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_ReenableOlt_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).ReenableOlt(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/ReenableOlt",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).ReenableOlt(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_ActivateOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).ActivateOnu(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/ActivateOnu",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).ActivateOnu(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_DeactivateOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).DeactivateOnu(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/DeactivateOnu",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).DeactivateOnu(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_DeleteOnu_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).DeleteOnu(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/DeleteOnu",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).DeleteOnu(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_OmciMsgOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(OmciMsg)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).OmciMsgOut(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/OmciMsgOut",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).OmciMsgOut(ctx, req.(*OmciMsg))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_OnuPacketOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(OnuPacket)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).OnuPacketOut(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/OnuPacketOut",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).OnuPacketOut(ctx, req.(*OnuPacket))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_UplinkPacketOut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(UplinkPacket)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).UplinkPacketOut(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/UplinkPacketOut",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).UplinkPacketOut(ctx, req.(*UplinkPacket))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_FlowAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Flow)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).FlowAdd(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/FlowAdd",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).FlowAdd(ctx, req.(*Flow))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_FlowRemove_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Flow)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).FlowRemove(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/FlowRemove",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).FlowRemove(ctx, req.(*Flow))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_HeartbeatCheck_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).HeartbeatCheck(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/HeartbeatCheck",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).HeartbeatCheck(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_EnablePonIf_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Interface)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).EnablePonIf(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/EnablePonIf",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).EnablePonIf(ctx, req.(*Interface))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_DisablePonIf_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Interface)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).DisablePonIf(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/DisablePonIf",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).DisablePonIf(ctx, req.(*Interface))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetDeviceInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetDeviceInfo(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetDeviceInfo",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetDeviceInfo(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_Reboot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).Reboot(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/Reboot",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).Reboot(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_CollectStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).CollectStatistics(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/CollectStatistics",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).CollectStatistics(ctx, req.(*Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetOnuStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetOnuStatistics(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetOnuStatistics",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetOnuStatistics(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetGemPortStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(OnuPacket)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetGemPortStatistics(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetGemPortStatistics",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetGemPortStatistics(ctx, req.(*OnuPacket))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_CreateTrafficSchedulers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(tech_profile.TrafficSchedulers)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).CreateTrafficSchedulers(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/CreateTrafficSchedulers",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).CreateTrafficSchedulers(ctx, req.(*tech_profile.TrafficSchedulers))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_RemoveTrafficSchedulers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(tech_profile.TrafficSchedulers)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).RemoveTrafficSchedulers(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/RemoveTrafficSchedulers",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).RemoveTrafficSchedulers(ctx, req.(*tech_profile.TrafficSchedulers))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_CreateTrafficQueues_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(tech_profile.TrafficQueues)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).CreateTrafficQueues(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/CreateTrafficQueues",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).CreateTrafficQueues(ctx, req.(*tech_profile.TrafficQueues))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_RemoveTrafficQueues_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(tech_profile.TrafficQueues)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).RemoveTrafficQueues(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/RemoveTrafficQueues",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).RemoveTrafficQueues(ctx, req.(*tech_profile.TrafficQueues))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_EnableIndication_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(Empty)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(OpenoltServer).EnableIndication(m, &openoltEnableIndicationServer{stream})
+}
+
+type Openolt_EnableIndicationServer interface {
+	Send(*Indication) error
+	grpc.ServerStream
+}
+
+type openoltEnableIndicationServer struct {
+	grpc.ServerStream
+}
+
+func (x *openoltEnableIndicationServer) Send(m *Indication) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _Openolt_PerformGroupOperation_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Group)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).PerformGroupOperation(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/PerformGroupOperation",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).PerformGroupOperation(ctx, req.(*Group))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_DeleteGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Group)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).DeleteGroup(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/DeleteGroup",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).DeleteGroup(ctx, req.(*Group))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ValueParam)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetExtValue(ctx, req.(*ValueParam))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_OnuItuPonAlarmSet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(config.OnuItuPonAlarm)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).OnuItuPonAlarmSet(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/OnuItuPonAlarmSet",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).OnuItuPonAlarmSet(ctx, req.(*config.OnuItuPonAlarm))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetLogicalOnuDistanceZero_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetLogicalOnuDistanceZero(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetLogicalOnuDistanceZero",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetLogicalOnuDistanceZero(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetLogicalOnuDistance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetLogicalOnuDistance(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetLogicalOnuDistance",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetLogicalOnuDistance(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Openolt_GetPonRxPower_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Onu)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(OpenoltServer).GetPonRxPower(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/openolt.Openolt/GetPonRxPower",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(OpenoltServer).GetPonRxPower(ctx, req.(*Onu))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Openolt_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "openolt.Openolt",
+	HandlerType: (*OpenoltServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "DisableOlt",
+			Handler:    _Openolt_DisableOlt_Handler,
+		},
+		{
+			MethodName: "ReenableOlt",
+			Handler:    _Openolt_ReenableOlt_Handler,
+		},
+		{
+			MethodName: "ActivateOnu",
+			Handler:    _Openolt_ActivateOnu_Handler,
+		},
+		{
+			MethodName: "DeactivateOnu",
+			Handler:    _Openolt_DeactivateOnu_Handler,
+		},
+		{
+			MethodName: "DeleteOnu",
+			Handler:    _Openolt_DeleteOnu_Handler,
+		},
+		{
+			MethodName: "OmciMsgOut",
+			Handler:    _Openolt_OmciMsgOut_Handler,
+		},
+		{
+			MethodName: "OnuPacketOut",
+			Handler:    _Openolt_OnuPacketOut_Handler,
+		},
+		{
+			MethodName: "UplinkPacketOut",
+			Handler:    _Openolt_UplinkPacketOut_Handler,
+		},
+		{
+			MethodName: "FlowAdd",
+			Handler:    _Openolt_FlowAdd_Handler,
+		},
+		{
+			MethodName: "FlowRemove",
+			Handler:    _Openolt_FlowRemove_Handler,
+		},
+		{
+			MethodName: "HeartbeatCheck",
+			Handler:    _Openolt_HeartbeatCheck_Handler,
+		},
+		{
+			MethodName: "EnablePonIf",
+			Handler:    _Openolt_EnablePonIf_Handler,
+		},
+		{
+			MethodName: "DisablePonIf",
+			Handler:    _Openolt_DisablePonIf_Handler,
+		},
+		{
+			MethodName: "GetDeviceInfo",
+			Handler:    _Openolt_GetDeviceInfo_Handler,
+		},
+		{
+			MethodName: "Reboot",
+			Handler:    _Openolt_Reboot_Handler,
+		},
+		{
+			MethodName: "CollectStatistics",
+			Handler:    _Openolt_CollectStatistics_Handler,
+		},
+		{
+			MethodName: "GetOnuStatistics",
+			Handler:    _Openolt_GetOnuStatistics_Handler,
+		},
+		{
+			MethodName: "GetGemPortStatistics",
+			Handler:    _Openolt_GetGemPortStatistics_Handler,
+		},
+		{
+			MethodName: "CreateTrafficSchedulers",
+			Handler:    _Openolt_CreateTrafficSchedulers_Handler,
+		},
+		{
+			MethodName: "RemoveTrafficSchedulers",
+			Handler:    _Openolt_RemoveTrafficSchedulers_Handler,
+		},
+		{
+			MethodName: "CreateTrafficQueues",
+			Handler:    _Openolt_CreateTrafficQueues_Handler,
+		},
+		{
+			MethodName: "RemoveTrafficQueues",
+			Handler:    _Openolt_RemoveTrafficQueues_Handler,
+		},
+		{
+			MethodName: "PerformGroupOperation",
+			Handler:    _Openolt_PerformGroupOperation_Handler,
+		},
+		{
+			MethodName: "DeleteGroup",
+			Handler:    _Openolt_DeleteGroup_Handler,
+		},
+		{
+			MethodName: "GetExtValue",
+			Handler:    _Openolt_GetExtValue_Handler,
+		},
+		{
+			MethodName: "OnuItuPonAlarmSet",
+			Handler:    _Openolt_OnuItuPonAlarmSet_Handler,
+		},
+		{
+			MethodName: "GetLogicalOnuDistanceZero",
+			Handler:    _Openolt_GetLogicalOnuDistanceZero_Handler,
+		},
+		{
+			MethodName: "GetLogicalOnuDistance",
+			Handler:    _Openolt_GetLogicalOnuDistance_Handler,
+		},
+		{
+			MethodName: "GetPonRxPower",
+			Handler:    _Openolt_GetPonRxPower_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "EnableIndication",
+			Handler:       _Openolt_EnableIndication_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "voltha_protos/openolt.proto",
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/tech_profile/tech_profile.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/tech_profile/tech_profile.pb.go
new file mode 100644
index 0000000..85f717a
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/tech_profile/tech_profile.pb.go
@@ -0,0 +1,2089 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/tech_profile.proto
+
+package tech_profile
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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 Direction int32
+
+const (
+	Direction_UPSTREAM      Direction = 0
+	Direction_DOWNSTREAM    Direction = 1
+	Direction_BIDIRECTIONAL Direction = 2
+)
+
+var Direction_name = map[int32]string{
+	0: "UPSTREAM",
+	1: "DOWNSTREAM",
+	2: "BIDIRECTIONAL",
+}
+
+var Direction_value = map[string]int32{
+	"UPSTREAM":      0,
+	"DOWNSTREAM":    1,
+	"BIDIRECTIONAL": 2,
+}
+
+func (x Direction) String() string {
+	return proto.EnumName(Direction_name, int32(x))
+}
+
+func (Direction) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{0}
+}
+
+type SchedulingPolicy int32
+
+const (
+	SchedulingPolicy_WRR            SchedulingPolicy = 0
+	SchedulingPolicy_StrictPriority SchedulingPolicy = 1
+	SchedulingPolicy_Hybrid         SchedulingPolicy = 2
+)
+
+var SchedulingPolicy_name = map[int32]string{
+	0: "WRR",
+	1: "StrictPriority",
+	2: "Hybrid",
+}
+
+var SchedulingPolicy_value = map[string]int32{
+	"WRR":            0,
+	"StrictPriority": 1,
+	"Hybrid":         2,
+}
+
+func (x SchedulingPolicy) String() string {
+	return proto.EnumName(SchedulingPolicy_name, int32(x))
+}
+
+func (SchedulingPolicy) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{1}
+}
+
+type AdditionalBW int32
+
+const (
+	AdditionalBW_AdditionalBW_None       AdditionalBW = 0
+	AdditionalBW_AdditionalBW_NA         AdditionalBW = 1
+	AdditionalBW_AdditionalBW_BestEffort AdditionalBW = 2
+	AdditionalBW_AdditionalBW_Auto       AdditionalBW = 3
+)
+
+var AdditionalBW_name = map[int32]string{
+	0: "AdditionalBW_None",
+	1: "AdditionalBW_NA",
+	2: "AdditionalBW_BestEffort",
+	3: "AdditionalBW_Auto",
+}
+
+var AdditionalBW_value = map[string]int32{
+	"AdditionalBW_None":       0,
+	"AdditionalBW_NA":         1,
+	"AdditionalBW_BestEffort": 2,
+	"AdditionalBW_Auto":       3,
+}
+
+func (x AdditionalBW) String() string {
+	return proto.EnumName(AdditionalBW_name, int32(x))
+}
+
+func (AdditionalBW) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{2}
+}
+
+type DiscardPolicy int32
+
+const (
+	DiscardPolicy_TailDrop  DiscardPolicy = 0
+	DiscardPolicy_WTailDrop DiscardPolicy = 1
+	DiscardPolicy_Red       DiscardPolicy = 2
+	DiscardPolicy_WRed      DiscardPolicy = 3
+)
+
+var DiscardPolicy_name = map[int32]string{
+	0: "TailDrop",
+	1: "WTailDrop",
+	2: "Red",
+	3: "WRed",
+}
+
+var DiscardPolicy_value = map[string]int32{
+	"TailDrop":  0,
+	"WTailDrop": 1,
+	"Red":       2,
+	"WRed":      3,
+}
+
+func (x DiscardPolicy) String() string {
+	return proto.EnumName(DiscardPolicy_name, int32(x))
+}
+
+func (DiscardPolicy) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{3}
+}
+
+type InferredAdditionBWIndication int32
+
+const (
+	InferredAdditionBWIndication_InferredAdditionBWIndication_None       InferredAdditionBWIndication = 0
+	InferredAdditionBWIndication_InferredAdditionBWIndication_Assured    InferredAdditionBWIndication = 1
+	InferredAdditionBWIndication_InferredAdditionBWIndication_BestEffort InferredAdditionBWIndication = 2
+)
+
+var InferredAdditionBWIndication_name = map[int32]string{
+	0: "InferredAdditionBWIndication_None",
+	1: "InferredAdditionBWIndication_Assured",
+	2: "InferredAdditionBWIndication_BestEffort",
+}
+
+var InferredAdditionBWIndication_value = map[string]int32{
+	"InferredAdditionBWIndication_None":       0,
+	"InferredAdditionBWIndication_Assured":    1,
+	"InferredAdditionBWIndication_BestEffort": 2,
+}
+
+func (x InferredAdditionBWIndication) String() string {
+	return proto.EnumName(InferredAdditionBWIndication_name, int32(x))
+}
+
+func (InferredAdditionBWIndication) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{4}
+}
+
+type SchedulerConfig struct {
+	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
+	AdditionalBw         AdditionalBW     `protobuf:"varint,2,opt,name=additional_bw,json=additionalBw,proto3,enum=tech_profile.AdditionalBW" json:"additional_bw,omitempty"`
+	Priority             uint32           `protobuf:"fixed32,3,opt,name=priority,proto3" json:"priority,omitempty"`
+	Weight               uint32           `protobuf:"fixed32,4,opt,name=weight,proto3" json:"weight,omitempty"`
+	SchedPolicy          SchedulingPolicy `protobuf:"varint,5,opt,name=sched_policy,json=schedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"sched_policy,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *SchedulerConfig) Reset()         { *m = SchedulerConfig{} }
+func (m *SchedulerConfig) String() string { return proto.CompactTextString(m) }
+func (*SchedulerConfig) ProtoMessage()    {}
+func (*SchedulerConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{0}
+}
+
+func (m *SchedulerConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SchedulerConfig.Unmarshal(m, b)
+}
+func (m *SchedulerConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SchedulerConfig.Marshal(b, m, deterministic)
+}
+func (m *SchedulerConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SchedulerConfig.Merge(m, src)
+}
+func (m *SchedulerConfig) XXX_Size() int {
+	return xxx_messageInfo_SchedulerConfig.Size(m)
+}
+func (m *SchedulerConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_SchedulerConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SchedulerConfig proto.InternalMessageInfo
+
+func (m *SchedulerConfig) GetDirection() Direction {
+	if m != nil {
+		return m.Direction
+	}
+	return Direction_UPSTREAM
+}
+
+func (m *SchedulerConfig) GetAdditionalBw() AdditionalBW {
+	if m != nil {
+		return m.AdditionalBw
+	}
+	return AdditionalBW_AdditionalBW_None
+}
+
+func (m *SchedulerConfig) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *SchedulerConfig) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *SchedulerConfig) GetSchedPolicy() SchedulingPolicy {
+	if m != nil {
+		return m.SchedPolicy
+	}
+	return SchedulingPolicy_WRR
+}
+
+type TrafficShapingInfo struct {
+	Cir                  uint32                       `protobuf:"fixed32,1,opt,name=cir,proto3" json:"cir,omitempty"`
+	Cbs                  uint32                       `protobuf:"fixed32,2,opt,name=cbs,proto3" json:"cbs,omitempty"`
+	Pir                  uint32                       `protobuf:"fixed32,3,opt,name=pir,proto3" json:"pir,omitempty"`
+	Pbs                  uint32                       `protobuf:"fixed32,4,opt,name=pbs,proto3" json:"pbs,omitempty"`
+	Gir                  uint32                       `protobuf:"fixed32,5,opt,name=gir,proto3" json:"gir,omitempty"`
+	AddBwInd             InferredAdditionBWIndication `protobuf:"varint,6,opt,name=add_bw_ind,json=addBwInd,proto3,enum=tech_profile.InferredAdditionBWIndication" json:"add_bw_ind,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
+	XXX_unrecognized     []byte                       `json:"-"`
+	XXX_sizecache        int32                        `json:"-"`
+}
+
+func (m *TrafficShapingInfo) Reset()         { *m = TrafficShapingInfo{} }
+func (m *TrafficShapingInfo) String() string { return proto.CompactTextString(m) }
+func (*TrafficShapingInfo) ProtoMessage()    {}
+func (*TrafficShapingInfo) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{1}
+}
+
+func (m *TrafficShapingInfo) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TrafficShapingInfo.Unmarshal(m, b)
+}
+func (m *TrafficShapingInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TrafficShapingInfo.Marshal(b, m, deterministic)
+}
+func (m *TrafficShapingInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TrafficShapingInfo.Merge(m, src)
+}
+func (m *TrafficShapingInfo) XXX_Size() int {
+	return xxx_messageInfo_TrafficShapingInfo.Size(m)
+}
+func (m *TrafficShapingInfo) XXX_DiscardUnknown() {
+	xxx_messageInfo_TrafficShapingInfo.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TrafficShapingInfo proto.InternalMessageInfo
+
+func (m *TrafficShapingInfo) GetCir() uint32 {
+	if m != nil {
+		return m.Cir
+	}
+	return 0
+}
+
+func (m *TrafficShapingInfo) GetCbs() uint32 {
+	if m != nil {
+		return m.Cbs
+	}
+	return 0
+}
+
+func (m *TrafficShapingInfo) GetPir() uint32 {
+	if m != nil {
+		return m.Pir
+	}
+	return 0
+}
+
+func (m *TrafficShapingInfo) GetPbs() uint32 {
+	if m != nil {
+		return m.Pbs
+	}
+	return 0
+}
+
+func (m *TrafficShapingInfo) GetGir() uint32 {
+	if m != nil {
+		return m.Gir
+	}
+	return 0
+}
+
+func (m *TrafficShapingInfo) GetAddBwInd() InferredAdditionBWIndication {
+	if m != nil {
+		return m.AddBwInd
+	}
+	return InferredAdditionBWIndication_InferredAdditionBWIndication_None
+}
+
+type TrafficScheduler struct {
+	Direction            Direction           `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
+	AllocId              uint32              `protobuf:"fixed32,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	Scheduler            *SchedulerConfig    `protobuf:"bytes,3,opt,name=scheduler,proto3" json:"scheduler,omitempty"`
+	TrafficShapingInfo   *TrafficShapingInfo `protobuf:"bytes,4,opt,name=traffic_shaping_info,json=trafficShapingInfo,proto3" json:"traffic_shaping_info,omitempty"`
+	TechProfileId        uint32              `protobuf:"fixed32,5,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *TrafficScheduler) Reset()         { *m = TrafficScheduler{} }
+func (m *TrafficScheduler) String() string { return proto.CompactTextString(m) }
+func (*TrafficScheduler) ProtoMessage()    {}
+func (*TrafficScheduler) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{2}
+}
+
+func (m *TrafficScheduler) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TrafficScheduler.Unmarshal(m, b)
+}
+func (m *TrafficScheduler) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TrafficScheduler.Marshal(b, m, deterministic)
+}
+func (m *TrafficScheduler) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TrafficScheduler.Merge(m, src)
+}
+func (m *TrafficScheduler) XXX_Size() int {
+	return xxx_messageInfo_TrafficScheduler.Size(m)
+}
+func (m *TrafficScheduler) XXX_DiscardUnknown() {
+	xxx_messageInfo_TrafficScheduler.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TrafficScheduler proto.InternalMessageInfo
+
+func (m *TrafficScheduler) GetDirection() Direction {
+	if m != nil {
+		return m.Direction
+	}
+	return Direction_UPSTREAM
+}
+
+func (m *TrafficScheduler) GetAllocId() uint32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+func (m *TrafficScheduler) GetScheduler() *SchedulerConfig {
+	if m != nil {
+		return m.Scheduler
+	}
+	return nil
+}
+
+func (m *TrafficScheduler) GetTrafficShapingInfo() *TrafficShapingInfo {
+	if m != nil {
+		return m.TrafficShapingInfo
+	}
+	return nil
+}
+
+func (m *TrafficScheduler) GetTechProfileId() uint32 {
+	if m != nil {
+		return m.TechProfileId
+	}
+	return 0
+}
+
+type TrafficSchedulers struct {
+	IntfId               uint32              `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32              `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	UniId                uint32              `protobuf:"fixed32,4,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	PortNo               uint32              `protobuf:"fixed32,5,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	TrafficScheds        []*TrafficScheduler `protobuf:"bytes,3,rep,name=traffic_scheds,json=trafficScheds,proto3" json:"traffic_scheds,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *TrafficSchedulers) Reset()         { *m = TrafficSchedulers{} }
+func (m *TrafficSchedulers) String() string { return proto.CompactTextString(m) }
+func (*TrafficSchedulers) ProtoMessage()    {}
+func (*TrafficSchedulers) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{3}
+}
+
+func (m *TrafficSchedulers) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TrafficSchedulers.Unmarshal(m, b)
+}
+func (m *TrafficSchedulers) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TrafficSchedulers.Marshal(b, m, deterministic)
+}
+func (m *TrafficSchedulers) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TrafficSchedulers.Merge(m, src)
+}
+func (m *TrafficSchedulers) XXX_Size() int {
+	return xxx_messageInfo_TrafficSchedulers.Size(m)
+}
+func (m *TrafficSchedulers) XXX_DiscardUnknown() {
+	xxx_messageInfo_TrafficSchedulers.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TrafficSchedulers proto.InternalMessageInfo
+
+func (m *TrafficSchedulers) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *TrafficSchedulers) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *TrafficSchedulers) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *TrafficSchedulers) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *TrafficSchedulers) GetTrafficScheds() []*TrafficScheduler {
+	if m != nil {
+		return m.TrafficScheds
+	}
+	return nil
+}
+
+type TailDropDiscardConfig struct {
+	QueueSize            uint32   `protobuf:"fixed32,1,opt,name=queue_size,json=queueSize,proto3" json:"queue_size,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *TailDropDiscardConfig) Reset()         { *m = TailDropDiscardConfig{} }
+func (m *TailDropDiscardConfig) String() string { return proto.CompactTextString(m) }
+func (*TailDropDiscardConfig) ProtoMessage()    {}
+func (*TailDropDiscardConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{4}
+}
+
+func (m *TailDropDiscardConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TailDropDiscardConfig.Unmarshal(m, b)
+}
+func (m *TailDropDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TailDropDiscardConfig.Marshal(b, m, deterministic)
+}
+func (m *TailDropDiscardConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TailDropDiscardConfig.Merge(m, src)
+}
+func (m *TailDropDiscardConfig) XXX_Size() int {
+	return xxx_messageInfo_TailDropDiscardConfig.Size(m)
+}
+func (m *TailDropDiscardConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_TailDropDiscardConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TailDropDiscardConfig proto.InternalMessageInfo
+
+func (m *TailDropDiscardConfig) GetQueueSize() uint32 {
+	if m != nil {
+		return m.QueueSize
+	}
+	return 0
+}
+
+type RedDiscardConfig struct {
+	MinThreshold         uint32   `protobuf:"fixed32,1,opt,name=min_threshold,json=minThreshold,proto3" json:"min_threshold,omitempty"`
+	MaxThreshold         uint32   `protobuf:"fixed32,2,opt,name=max_threshold,json=maxThreshold,proto3" json:"max_threshold,omitempty"`
+	MaxProbability       uint32   `protobuf:"fixed32,3,opt,name=max_probability,json=maxProbability,proto3" json:"max_probability,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *RedDiscardConfig) Reset()         { *m = RedDiscardConfig{} }
+func (m *RedDiscardConfig) String() string { return proto.CompactTextString(m) }
+func (*RedDiscardConfig) ProtoMessage()    {}
+func (*RedDiscardConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{5}
+}
+
+func (m *RedDiscardConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_RedDiscardConfig.Unmarshal(m, b)
+}
+func (m *RedDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_RedDiscardConfig.Marshal(b, m, deterministic)
+}
+func (m *RedDiscardConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RedDiscardConfig.Merge(m, src)
+}
+func (m *RedDiscardConfig) XXX_Size() int {
+	return xxx_messageInfo_RedDiscardConfig.Size(m)
+}
+func (m *RedDiscardConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_RedDiscardConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RedDiscardConfig proto.InternalMessageInfo
+
+func (m *RedDiscardConfig) GetMinThreshold() uint32 {
+	if m != nil {
+		return m.MinThreshold
+	}
+	return 0
+}
+
+func (m *RedDiscardConfig) GetMaxThreshold() uint32 {
+	if m != nil {
+		return m.MaxThreshold
+	}
+	return 0
+}
+
+func (m *RedDiscardConfig) GetMaxProbability() uint32 {
+	if m != nil {
+		return m.MaxProbability
+	}
+	return 0
+}
+
+type WRedDiscardConfig struct {
+	Green                *RedDiscardConfig `protobuf:"bytes,1,opt,name=green,proto3" json:"green,omitempty"`
+	Yellow               *RedDiscardConfig `protobuf:"bytes,2,opt,name=yellow,proto3" json:"yellow,omitempty"`
+	Red                  *RedDiscardConfig `protobuf:"bytes,3,opt,name=red,proto3" json:"red,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *WRedDiscardConfig) Reset()         { *m = WRedDiscardConfig{} }
+func (m *WRedDiscardConfig) String() string { return proto.CompactTextString(m) }
+func (*WRedDiscardConfig) ProtoMessage()    {}
+func (*WRedDiscardConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{6}
+}
+
+func (m *WRedDiscardConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_WRedDiscardConfig.Unmarshal(m, b)
+}
+func (m *WRedDiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_WRedDiscardConfig.Marshal(b, m, deterministic)
+}
+func (m *WRedDiscardConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WRedDiscardConfig.Merge(m, src)
+}
+func (m *WRedDiscardConfig) XXX_Size() int {
+	return xxx_messageInfo_WRedDiscardConfig.Size(m)
+}
+func (m *WRedDiscardConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_WRedDiscardConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WRedDiscardConfig proto.InternalMessageInfo
+
+func (m *WRedDiscardConfig) GetGreen() *RedDiscardConfig {
+	if m != nil {
+		return m.Green
+	}
+	return nil
+}
+
+func (m *WRedDiscardConfig) GetYellow() *RedDiscardConfig {
+	if m != nil {
+		return m.Yellow
+	}
+	return nil
+}
+
+func (m *WRedDiscardConfig) GetRed() *RedDiscardConfig {
+	if m != nil {
+		return m.Red
+	}
+	return nil
+}
+
+type DiscardConfig struct {
+	DiscardPolicy DiscardPolicy `protobuf:"varint,1,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
+	// Types that are valid to be assigned to DiscardConfig:
+	//	*DiscardConfig_TailDropDiscardConfig
+	//	*DiscardConfig_RedDiscardConfig
+	//	*DiscardConfig_WredDiscardConfig
+	DiscardConfig        isDiscardConfig_DiscardConfig `protobuf_oneof:"discard_config"`
+	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
+	XXX_unrecognized     []byte                        `json:"-"`
+	XXX_sizecache        int32                         `json:"-"`
+}
+
+func (m *DiscardConfig) Reset()         { *m = DiscardConfig{} }
+func (m *DiscardConfig) String() string { return proto.CompactTextString(m) }
+func (*DiscardConfig) ProtoMessage()    {}
+func (*DiscardConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{7}
+}
+
+func (m *DiscardConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DiscardConfig.Unmarshal(m, b)
+}
+func (m *DiscardConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DiscardConfig.Marshal(b, m, deterministic)
+}
+func (m *DiscardConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DiscardConfig.Merge(m, src)
+}
+func (m *DiscardConfig) XXX_Size() int {
+	return xxx_messageInfo_DiscardConfig.Size(m)
+}
+func (m *DiscardConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_DiscardConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DiscardConfig proto.InternalMessageInfo
+
+func (m *DiscardConfig) GetDiscardPolicy() DiscardPolicy {
+	if m != nil {
+		return m.DiscardPolicy
+	}
+	return DiscardPolicy_TailDrop
+}
+
+type isDiscardConfig_DiscardConfig interface {
+	isDiscardConfig_DiscardConfig()
+}
+
+type DiscardConfig_TailDropDiscardConfig struct {
+	TailDropDiscardConfig *TailDropDiscardConfig `protobuf:"bytes,2,opt,name=tail_drop_discard_config,json=tailDropDiscardConfig,proto3,oneof"`
+}
+
+type DiscardConfig_RedDiscardConfig struct {
+	RedDiscardConfig *RedDiscardConfig `protobuf:"bytes,3,opt,name=red_discard_config,json=redDiscardConfig,proto3,oneof"`
+}
+
+type DiscardConfig_WredDiscardConfig struct {
+	WredDiscardConfig *WRedDiscardConfig `protobuf:"bytes,4,opt,name=wred_discard_config,json=wredDiscardConfig,proto3,oneof"`
+}
+
+func (*DiscardConfig_TailDropDiscardConfig) isDiscardConfig_DiscardConfig() {}
+
+func (*DiscardConfig_RedDiscardConfig) isDiscardConfig_DiscardConfig() {}
+
+func (*DiscardConfig_WredDiscardConfig) isDiscardConfig_DiscardConfig() {}
+
+func (m *DiscardConfig) GetDiscardConfig() isDiscardConfig_DiscardConfig {
+	if m != nil {
+		return m.DiscardConfig
+	}
+	return nil
+}
+
+func (m *DiscardConfig) GetTailDropDiscardConfig() *TailDropDiscardConfig {
+	if x, ok := m.GetDiscardConfig().(*DiscardConfig_TailDropDiscardConfig); ok {
+		return x.TailDropDiscardConfig
+	}
+	return nil
+}
+
+func (m *DiscardConfig) GetRedDiscardConfig() *RedDiscardConfig {
+	if x, ok := m.GetDiscardConfig().(*DiscardConfig_RedDiscardConfig); ok {
+		return x.RedDiscardConfig
+	}
+	return nil
+}
+
+func (m *DiscardConfig) GetWredDiscardConfig() *WRedDiscardConfig {
+	if x, ok := m.GetDiscardConfig().(*DiscardConfig_WredDiscardConfig); ok {
+		return x.WredDiscardConfig
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*DiscardConfig) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*DiscardConfig_TailDropDiscardConfig)(nil),
+		(*DiscardConfig_RedDiscardConfig)(nil),
+		(*DiscardConfig_WredDiscardConfig)(nil),
+	}
+}
+
+type TrafficQueue struct {
+	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
+	GemportId            uint32           `protobuf:"fixed32,2,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	PbitMap              string           `protobuf:"bytes,3,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
+	AesEncryption        bool             `protobuf:"varint,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
+	SchedPolicy          SchedulingPolicy `protobuf:"varint,5,opt,name=sched_policy,json=schedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"sched_policy,omitempty"`
+	Priority             uint32           `protobuf:"fixed32,6,opt,name=priority,proto3" json:"priority,omitempty"`
+	Weight               uint32           `protobuf:"fixed32,7,opt,name=weight,proto3" json:"weight,omitempty"`
+	DiscardPolicy        DiscardPolicy    `protobuf:"varint,8,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
+	DiscardConfig        *DiscardConfig   `protobuf:"bytes,9,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *TrafficQueue) Reset()         { *m = TrafficQueue{} }
+func (m *TrafficQueue) String() string { return proto.CompactTextString(m) }
+func (*TrafficQueue) ProtoMessage()    {}
+func (*TrafficQueue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{8}
+}
+
+func (m *TrafficQueue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TrafficQueue.Unmarshal(m, b)
+}
+func (m *TrafficQueue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TrafficQueue.Marshal(b, m, deterministic)
+}
+func (m *TrafficQueue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TrafficQueue.Merge(m, src)
+}
+func (m *TrafficQueue) XXX_Size() int {
+	return xxx_messageInfo_TrafficQueue.Size(m)
+}
+func (m *TrafficQueue) XXX_DiscardUnknown() {
+	xxx_messageInfo_TrafficQueue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TrafficQueue proto.InternalMessageInfo
+
+func (m *TrafficQueue) GetDirection() Direction {
+	if m != nil {
+		return m.Direction
+	}
+	return Direction_UPSTREAM
+}
+
+func (m *TrafficQueue) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *TrafficQueue) GetPbitMap() string {
+	if m != nil {
+		return m.PbitMap
+	}
+	return ""
+}
+
+func (m *TrafficQueue) GetAesEncryption() bool {
+	if m != nil {
+		return m.AesEncryption
+	}
+	return false
+}
+
+func (m *TrafficQueue) GetSchedPolicy() SchedulingPolicy {
+	if m != nil {
+		return m.SchedPolicy
+	}
+	return SchedulingPolicy_WRR
+}
+
+func (m *TrafficQueue) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *TrafficQueue) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *TrafficQueue) GetDiscardPolicy() DiscardPolicy {
+	if m != nil {
+		return m.DiscardPolicy
+	}
+	return DiscardPolicy_TailDrop
+}
+
+func (m *TrafficQueue) GetDiscardConfig() *DiscardConfig {
+	if m != nil {
+		return m.DiscardConfig
+	}
+	return nil
+}
+
+type TrafficQueues struct {
+	IntfId               uint32          `protobuf:"fixed32,1,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	OnuId                uint32          `protobuf:"fixed32,2,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	UniId                uint32          `protobuf:"fixed32,4,opt,name=uni_id,json=uniId,proto3" json:"uni_id,omitempty"`
+	PortNo               uint32          `protobuf:"fixed32,5,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	TrafficQueues        []*TrafficQueue `protobuf:"bytes,6,rep,name=traffic_queues,json=trafficQueues,proto3" json:"traffic_queues,omitempty"`
+	TechProfileId        uint32          `protobuf:"fixed32,7,opt,name=tech_profile_id,json=techProfileId,proto3" json:"tech_profile_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *TrafficQueues) Reset()         { *m = TrafficQueues{} }
+func (m *TrafficQueues) String() string { return proto.CompactTextString(m) }
+func (*TrafficQueues) ProtoMessage()    {}
+func (*TrafficQueues) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{9}
+}
+
+func (m *TrafficQueues) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TrafficQueues.Unmarshal(m, b)
+}
+func (m *TrafficQueues) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TrafficQueues.Marshal(b, m, deterministic)
+}
+func (m *TrafficQueues) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TrafficQueues.Merge(m, src)
+}
+func (m *TrafficQueues) XXX_Size() int {
+	return xxx_messageInfo_TrafficQueues.Size(m)
+}
+func (m *TrafficQueues) XXX_DiscardUnknown() {
+	xxx_messageInfo_TrafficQueues.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TrafficQueues proto.InternalMessageInfo
+
+func (m *TrafficQueues) GetIntfId() uint32 {
+	if m != nil {
+		return m.IntfId
+	}
+	return 0
+}
+
+func (m *TrafficQueues) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *TrafficQueues) GetUniId() uint32 {
+	if m != nil {
+		return m.UniId
+	}
+	return 0
+}
+
+func (m *TrafficQueues) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *TrafficQueues) GetTrafficQueues() []*TrafficQueue {
+	if m != nil {
+		return m.TrafficQueues
+	}
+	return nil
+}
+
+func (m *TrafficQueues) GetTechProfileId() uint32 {
+	if m != nil {
+		return m.TechProfileId
+	}
+	return 0
+}
+
+type InstanceControl struct {
+	Onu                  string   `protobuf:"bytes,1,opt,name=onu,proto3" json:"onu,omitempty"`
+	Uni                  string   `protobuf:"bytes,2,opt,name=uni,proto3" json:"uni,omitempty"`
+	MaxGemPayloadSize    string   `protobuf:"bytes,3,opt,name=max_gem_payload_size,json=maxGemPayloadSize,proto3" json:"max_gem_payload_size,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *InstanceControl) Reset()         { *m = InstanceControl{} }
+func (m *InstanceControl) String() string { return proto.CompactTextString(m) }
+func (*InstanceControl) ProtoMessage()    {}
+func (*InstanceControl) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{10}
+}
+
+func (m *InstanceControl) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_InstanceControl.Unmarshal(m, b)
+}
+func (m *InstanceControl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_InstanceControl.Marshal(b, m, deterministic)
+}
+func (m *InstanceControl) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_InstanceControl.Merge(m, src)
+}
+func (m *InstanceControl) XXX_Size() int {
+	return xxx_messageInfo_InstanceControl.Size(m)
+}
+func (m *InstanceControl) XXX_DiscardUnknown() {
+	xxx_messageInfo_InstanceControl.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_InstanceControl proto.InternalMessageInfo
+
+func (m *InstanceControl) GetOnu() string {
+	if m != nil {
+		return m.Onu
+	}
+	return ""
+}
+
+func (m *InstanceControl) GetUni() string {
+	if m != nil {
+		return m.Uni
+	}
+	return ""
+}
+
+func (m *InstanceControl) GetMaxGemPayloadSize() string {
+	if m != nil {
+		return m.MaxGemPayloadSize
+	}
+	return ""
+}
+
+type QThresholds struct {
+	QThreshold1          uint32   `protobuf:"varint,1,opt,name=q_threshold1,json=qThreshold1,proto3" json:"q_threshold1,omitempty"`
+	QThreshold2          uint32   `protobuf:"varint,2,opt,name=q_threshold2,json=qThreshold2,proto3" json:"q_threshold2,omitempty"`
+	QThreshold3          uint32   `protobuf:"varint,3,opt,name=q_threshold3,json=qThreshold3,proto3" json:"q_threshold3,omitempty"`
+	QThreshold4          uint32   `protobuf:"varint,4,opt,name=q_threshold4,json=qThreshold4,proto3" json:"q_threshold4,omitempty"`
+	QThreshold5          uint32   `protobuf:"varint,5,opt,name=q_threshold5,json=qThreshold5,proto3" json:"q_threshold5,omitempty"`
+	QThreshold6          uint32   `protobuf:"varint,6,opt,name=q_threshold6,json=qThreshold6,proto3" json:"q_threshold6,omitempty"`
+	QThreshold7          uint32   `protobuf:"varint,7,opt,name=q_threshold7,json=qThreshold7,proto3" json:"q_threshold7,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *QThresholds) Reset()         { *m = QThresholds{} }
+func (m *QThresholds) String() string { return proto.CompactTextString(m) }
+func (*QThresholds) ProtoMessage()    {}
+func (*QThresholds) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{11}
+}
+
+func (m *QThresholds) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_QThresholds.Unmarshal(m, b)
+}
+func (m *QThresholds) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_QThresholds.Marshal(b, m, deterministic)
+}
+func (m *QThresholds) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_QThresholds.Merge(m, src)
+}
+func (m *QThresholds) XXX_Size() int {
+	return xxx_messageInfo_QThresholds.Size(m)
+}
+func (m *QThresholds) XXX_DiscardUnknown() {
+	xxx_messageInfo_QThresholds.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_QThresholds proto.InternalMessageInfo
+
+func (m *QThresholds) GetQThreshold1() uint32 {
+	if m != nil {
+		return m.QThreshold1
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold2() uint32 {
+	if m != nil {
+		return m.QThreshold2
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold3() uint32 {
+	if m != nil {
+		return m.QThreshold3
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold4() uint32 {
+	if m != nil {
+		return m.QThreshold4
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold5() uint32 {
+	if m != nil {
+		return m.QThreshold5
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold6() uint32 {
+	if m != nil {
+		return m.QThreshold6
+	}
+	return 0
+}
+
+func (m *QThresholds) GetQThreshold7() uint32 {
+	if m != nil {
+		return m.QThreshold7
+	}
+	return 0
+}
+
+type GemPortAttributes struct {
+	GemportId                uint32            `protobuf:"fixed32,1,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	MaxQSize                 string            `protobuf:"bytes,2,opt,name=max_q_size,json=maxQSize,proto3" json:"max_q_size,omitempty"`
+	PbitMap                  string            `protobuf:"bytes,3,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
+	AesEncryption            string            `protobuf:"bytes,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
+	SchedulingPolicy         SchedulingPolicy  `protobuf:"varint,5,opt,name=scheduling_policy,json=schedulingPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"scheduling_policy,omitempty"`
+	PriorityQ                uint32            `protobuf:"fixed32,6,opt,name=priority_q,json=priorityQ,proto3" json:"priority_q,omitempty"`
+	Weight                   uint32            `protobuf:"fixed32,7,opt,name=weight,proto3" json:"weight,omitempty"`
+	DiscardPolicy            DiscardPolicy     `protobuf:"varint,8,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
+	DiscardConfig            *RedDiscardConfig `protobuf:"bytes,9,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
+	DiscardConfigV2          *DiscardConfig    `protobuf:"bytes,14,opt,name=discard_config_v2,json=discardConfigV2,proto3" json:"discard_config_v2,omitempty"`
+	IsMulticast              string            `protobuf:"bytes,10,opt,name=is_multicast,json=isMulticast,proto3" json:"is_multicast,omitempty"`
+	MulticastGemId           uint32            `protobuf:"fixed32,11,opt,name=multicast_gem_id,json=multicastGemId,proto3" json:"multicast_gem_id,omitempty"`
+	DynamicAccessControlList string            `protobuf:"bytes,12,opt,name=dynamic_access_control_list,json=dynamicAccessControlList,proto3" json:"dynamic_access_control_list,omitempty"`
+	StaticAccessControlList  string            `protobuf:"bytes,13,opt,name=static_access_control_list,json=staticAccessControlList,proto3" json:"static_access_control_list,omitempty"`
+	XXX_NoUnkeyedLiteral     struct{}          `json:"-"`
+	XXX_unrecognized         []byte            `json:"-"`
+	XXX_sizecache            int32             `json:"-"`
+}
+
+func (m *GemPortAttributes) Reset()         { *m = GemPortAttributes{} }
+func (m *GemPortAttributes) String() string { return proto.CompactTextString(m) }
+func (*GemPortAttributes) ProtoMessage()    {}
+func (*GemPortAttributes) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{12}
+}
+
+func (m *GemPortAttributes) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_GemPortAttributes.Unmarshal(m, b)
+}
+func (m *GemPortAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_GemPortAttributes.Marshal(b, m, deterministic)
+}
+func (m *GemPortAttributes) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GemPortAttributes.Merge(m, src)
+}
+func (m *GemPortAttributes) XXX_Size() int {
+	return xxx_messageInfo_GemPortAttributes.Size(m)
+}
+func (m *GemPortAttributes) XXX_DiscardUnknown() {
+	xxx_messageInfo_GemPortAttributes.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_GemPortAttributes proto.InternalMessageInfo
+
+func (m *GemPortAttributes) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *GemPortAttributes) GetMaxQSize() string {
+	if m != nil {
+		return m.MaxQSize
+	}
+	return ""
+}
+
+func (m *GemPortAttributes) GetPbitMap() string {
+	if m != nil {
+		return m.PbitMap
+	}
+	return ""
+}
+
+func (m *GemPortAttributes) GetAesEncryption() string {
+	if m != nil {
+		return m.AesEncryption
+	}
+	return ""
+}
+
+func (m *GemPortAttributes) GetSchedulingPolicy() SchedulingPolicy {
+	if m != nil {
+		return m.SchedulingPolicy
+	}
+	return SchedulingPolicy_WRR
+}
+
+func (m *GemPortAttributes) GetPriorityQ() uint32 {
+	if m != nil {
+		return m.PriorityQ
+	}
+	return 0
+}
+
+func (m *GemPortAttributes) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *GemPortAttributes) GetDiscardPolicy() DiscardPolicy {
+	if m != nil {
+		return m.DiscardPolicy
+	}
+	return DiscardPolicy_TailDrop
+}
+
+func (m *GemPortAttributes) GetDiscardConfig() *RedDiscardConfig {
+	if m != nil {
+		return m.DiscardConfig
+	}
+	return nil
+}
+
+func (m *GemPortAttributes) GetDiscardConfigV2() *DiscardConfig {
+	if m != nil {
+		return m.DiscardConfigV2
+	}
+	return nil
+}
+
+func (m *GemPortAttributes) GetIsMulticast() string {
+	if m != nil {
+		return m.IsMulticast
+	}
+	return ""
+}
+
+func (m *GemPortAttributes) GetMulticastGemId() uint32 {
+	if m != nil {
+		return m.MulticastGemId
+	}
+	return 0
+}
+
+func (m *GemPortAttributes) GetDynamicAccessControlList() string {
+	if m != nil {
+		return m.DynamicAccessControlList
+	}
+	return ""
+}
+
+func (m *GemPortAttributes) GetStaticAccessControlList() string {
+	if m != nil {
+		return m.StaticAccessControlList
+	}
+	return ""
+}
+
+type SchedulerAttributes struct {
+	Direction            Direction        `protobuf:"varint,1,opt,name=direction,proto3,enum=tech_profile.Direction" json:"direction,omitempty"`
+	AllocId              uint32           `protobuf:"varint,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	AdditionalBw         AdditionalBW     `protobuf:"varint,3,opt,name=additional_bw,json=additionalBw,proto3,enum=tech_profile.AdditionalBW" json:"additional_bw,omitempty"`
+	Priority             uint32           `protobuf:"fixed32,4,opt,name=priority,proto3" json:"priority,omitempty"`
+	Weight               uint32           `protobuf:"fixed32,5,opt,name=weight,proto3" json:"weight,omitempty"`
+	QSchedPolicy         SchedulingPolicy `protobuf:"varint,6,opt,name=q_sched_policy,json=qSchedPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"q_sched_policy,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *SchedulerAttributes) Reset()         { *m = SchedulerAttributes{} }
+func (m *SchedulerAttributes) String() string { return proto.CompactTextString(m) }
+func (*SchedulerAttributes) ProtoMessage()    {}
+func (*SchedulerAttributes) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{13}
+}
+
+func (m *SchedulerAttributes) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SchedulerAttributes.Unmarshal(m, b)
+}
+func (m *SchedulerAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SchedulerAttributes.Marshal(b, m, deterministic)
+}
+func (m *SchedulerAttributes) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SchedulerAttributes.Merge(m, src)
+}
+func (m *SchedulerAttributes) XXX_Size() int {
+	return xxx_messageInfo_SchedulerAttributes.Size(m)
+}
+func (m *SchedulerAttributes) XXX_DiscardUnknown() {
+	xxx_messageInfo_SchedulerAttributes.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SchedulerAttributes proto.InternalMessageInfo
+
+func (m *SchedulerAttributes) GetDirection() Direction {
+	if m != nil {
+		return m.Direction
+	}
+	return Direction_UPSTREAM
+}
+
+func (m *SchedulerAttributes) GetAllocId() uint32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+func (m *SchedulerAttributes) GetAdditionalBw() AdditionalBW {
+	if m != nil {
+		return m.AdditionalBw
+	}
+	return AdditionalBW_AdditionalBW_None
+}
+
+func (m *SchedulerAttributes) GetPriority() uint32 {
+	if m != nil {
+		return m.Priority
+	}
+	return 0
+}
+
+func (m *SchedulerAttributes) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *SchedulerAttributes) GetQSchedPolicy() SchedulingPolicy {
+	if m != nil {
+		return m.QSchedPolicy
+	}
+	return SchedulingPolicy_WRR
+}
+
+type EPONQueueAttributes struct {
+	MaxQSize                  string            `protobuf:"bytes,1,opt,name=max_q_size,json=maxQSize,proto3" json:"max_q_size,omitempty"`
+	PbitMap                   string            `protobuf:"bytes,2,opt,name=pbit_map,json=pbitMap,proto3" json:"pbit_map,omitempty"`
+	GemportId                 uint32            `protobuf:"varint,3,opt,name=gemport_id,json=gemportId,proto3" json:"gemport_id,omitempty"`
+	AesEncryption             string            `protobuf:"bytes,4,opt,name=aes_encryption,json=aesEncryption,proto3" json:"aes_encryption,omitempty"`
+	TrafficType               string            `protobuf:"bytes,5,opt,name=traffic_type,json=trafficType,proto3" json:"traffic_type,omitempty"`
+	UnsolicitedGrantSize      uint32            `protobuf:"varint,6,opt,name=unsolicited_grant_size,json=unsolicitedGrantSize,proto3" json:"unsolicited_grant_size,omitempty"`
+	NominalInterval           uint32            `protobuf:"varint,7,opt,name=nominal_interval,json=nominalInterval,proto3" json:"nominal_interval,omitempty"`
+	ToleratedPollJitter       uint32            `protobuf:"varint,8,opt,name=tolerated_poll_jitter,json=toleratedPollJitter,proto3" json:"tolerated_poll_jitter,omitempty"`
+	RequestTransmissionPolicy uint32            `protobuf:"varint,9,opt,name=request_transmission_policy,json=requestTransmissionPolicy,proto3" json:"request_transmission_policy,omitempty"`
+	NumQSets                  uint32            `protobuf:"varint,10,opt,name=num_q_sets,json=numQSets,proto3" json:"num_q_sets,omitempty"`
+	QThresholds               *QThresholds      `protobuf:"bytes,11,opt,name=q_thresholds,json=qThresholds,proto3" json:"q_thresholds,omitempty"`
+	SchedulingPolicy          SchedulingPolicy  `protobuf:"varint,12,opt,name=scheduling_policy,json=schedulingPolicy,proto3,enum=tech_profile.SchedulingPolicy" json:"scheduling_policy,omitempty"`
+	PriorityQ                 uint32            `protobuf:"varint,13,opt,name=priority_q,json=priorityQ,proto3" json:"priority_q,omitempty"`
+	Weight                    uint32            `protobuf:"varint,14,opt,name=weight,proto3" json:"weight,omitempty"`
+	DiscardPolicy             DiscardPolicy     `protobuf:"varint,15,opt,name=discard_policy,json=discardPolicy,proto3,enum=tech_profile.DiscardPolicy" json:"discard_policy,omitempty"`
+	DiscardConfig             *RedDiscardConfig `protobuf:"bytes,16,opt,name=discard_config,json=discardConfig,proto3" json:"discard_config,omitempty"`
+	DiscardConfigV2           *DiscardConfig    `protobuf:"bytes,17,opt,name=discard_config_v2,json=discardConfigV2,proto3" json:"discard_config_v2,omitempty"`
+	XXX_NoUnkeyedLiteral      struct{}          `json:"-"`
+	XXX_unrecognized          []byte            `json:"-"`
+	XXX_sizecache             int32             `json:"-"`
+}
+
+func (m *EPONQueueAttributes) Reset()         { *m = EPONQueueAttributes{} }
+func (m *EPONQueueAttributes) String() string { return proto.CompactTextString(m) }
+func (*EPONQueueAttributes) ProtoMessage()    {}
+func (*EPONQueueAttributes) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{14}
+}
+
+func (m *EPONQueueAttributes) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EPONQueueAttributes.Unmarshal(m, b)
+}
+func (m *EPONQueueAttributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EPONQueueAttributes.Marshal(b, m, deterministic)
+}
+func (m *EPONQueueAttributes) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EPONQueueAttributes.Merge(m, src)
+}
+func (m *EPONQueueAttributes) XXX_Size() int {
+	return xxx_messageInfo_EPONQueueAttributes.Size(m)
+}
+func (m *EPONQueueAttributes) XXX_DiscardUnknown() {
+	xxx_messageInfo_EPONQueueAttributes.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EPONQueueAttributes proto.InternalMessageInfo
+
+func (m *EPONQueueAttributes) GetMaxQSize() string {
+	if m != nil {
+		return m.MaxQSize
+	}
+	return ""
+}
+
+func (m *EPONQueueAttributes) GetPbitMap() string {
+	if m != nil {
+		return m.PbitMap
+	}
+	return ""
+}
+
+func (m *EPONQueueAttributes) GetGemportId() uint32 {
+	if m != nil {
+		return m.GemportId
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetAesEncryption() string {
+	if m != nil {
+		return m.AesEncryption
+	}
+	return ""
+}
+
+func (m *EPONQueueAttributes) GetTrafficType() string {
+	if m != nil {
+		return m.TrafficType
+	}
+	return ""
+}
+
+func (m *EPONQueueAttributes) GetUnsolicitedGrantSize() uint32 {
+	if m != nil {
+		return m.UnsolicitedGrantSize
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetNominalInterval() uint32 {
+	if m != nil {
+		return m.NominalInterval
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetToleratedPollJitter() uint32 {
+	if m != nil {
+		return m.ToleratedPollJitter
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetRequestTransmissionPolicy() uint32 {
+	if m != nil {
+		return m.RequestTransmissionPolicy
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetNumQSets() uint32 {
+	if m != nil {
+		return m.NumQSets
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetQThresholds() *QThresholds {
+	if m != nil {
+		return m.QThresholds
+	}
+	return nil
+}
+
+func (m *EPONQueueAttributes) GetSchedulingPolicy() SchedulingPolicy {
+	if m != nil {
+		return m.SchedulingPolicy
+	}
+	return SchedulingPolicy_WRR
+}
+
+func (m *EPONQueueAttributes) GetPriorityQ() uint32 {
+	if m != nil {
+		return m.PriorityQ
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetWeight() uint32 {
+	if m != nil {
+		return m.Weight
+	}
+	return 0
+}
+
+func (m *EPONQueueAttributes) GetDiscardPolicy() DiscardPolicy {
+	if m != nil {
+		return m.DiscardPolicy
+	}
+	return DiscardPolicy_TailDrop
+}
+
+func (m *EPONQueueAttributes) GetDiscardConfig() *RedDiscardConfig {
+	if m != nil {
+		return m.DiscardConfig
+	}
+	return nil
+}
+
+func (m *EPONQueueAttributes) GetDiscardConfigV2() *DiscardConfig {
+	if m != nil {
+		return m.DiscardConfigV2
+	}
+	return nil
+}
+
+// TechProfile definition (relevant for GPON, XGPON and XGS-PON technologies)
+type TechProfile struct {
+	Name                           string               `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Version                        uint32               `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
+	ProfileType                    string               `protobuf:"bytes,3,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
+	NumGemPorts                    uint32               `protobuf:"varint,4,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
+	InstanceControl                *InstanceControl     `protobuf:"bytes,5,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
+	UsScheduler                    *SchedulerAttributes `protobuf:"bytes,6,opt,name=us_scheduler,json=usScheduler,proto3" json:"us_scheduler,omitempty"`
+	DsScheduler                    *SchedulerAttributes `protobuf:"bytes,7,opt,name=ds_scheduler,json=dsScheduler,proto3" json:"ds_scheduler,omitempty"`
+	UpstreamGemPortAttributeList   []*GemPortAttributes `protobuf:"bytes,8,rep,name=upstream_gem_port_attribute_list,json=upstreamGemPortAttributeList,proto3" json:"upstream_gem_port_attribute_list,omitempty"`
+	DownstreamGemPortAttributeList []*GemPortAttributes `protobuf:"bytes,9,rep,name=downstream_gem_port_attribute_list,json=downstreamGemPortAttributeList,proto3" json:"downstream_gem_port_attribute_list,omitempty"`
+	XXX_NoUnkeyedLiteral           struct{}             `json:"-"`
+	XXX_unrecognized               []byte               `json:"-"`
+	XXX_sizecache                  int32                `json:"-"`
+}
+
+func (m *TechProfile) Reset()         { *m = TechProfile{} }
+func (m *TechProfile) String() string { return proto.CompactTextString(m) }
+func (*TechProfile) ProtoMessage()    {}
+func (*TechProfile) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{15}
+}
+
+func (m *TechProfile) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TechProfile.Unmarshal(m, b)
+}
+func (m *TechProfile) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TechProfile.Marshal(b, m, deterministic)
+}
+func (m *TechProfile) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TechProfile.Merge(m, src)
+}
+func (m *TechProfile) XXX_Size() int {
+	return xxx_messageInfo_TechProfile.Size(m)
+}
+func (m *TechProfile) XXX_DiscardUnknown() {
+	xxx_messageInfo_TechProfile.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TechProfile proto.InternalMessageInfo
+
+func (m *TechProfile) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *TechProfile) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *TechProfile) GetProfileType() string {
+	if m != nil {
+		return m.ProfileType
+	}
+	return ""
+}
+
+func (m *TechProfile) GetNumGemPorts() uint32 {
+	if m != nil {
+		return m.NumGemPorts
+	}
+	return 0
+}
+
+func (m *TechProfile) GetInstanceControl() *InstanceControl {
+	if m != nil {
+		return m.InstanceControl
+	}
+	return nil
+}
+
+func (m *TechProfile) GetUsScheduler() *SchedulerAttributes {
+	if m != nil {
+		return m.UsScheduler
+	}
+	return nil
+}
+
+func (m *TechProfile) GetDsScheduler() *SchedulerAttributes {
+	if m != nil {
+		return m.DsScheduler
+	}
+	return nil
+}
+
+func (m *TechProfile) GetUpstreamGemPortAttributeList() []*GemPortAttributes {
+	if m != nil {
+		return m.UpstreamGemPortAttributeList
+	}
+	return nil
+}
+
+func (m *TechProfile) GetDownstreamGemPortAttributeList() []*GemPortAttributes {
+	if m != nil {
+		return m.DownstreamGemPortAttributeList
+	}
+	return nil
+}
+
+// EPON TechProfile definition
+type EponTechProfile struct {
+	Name                         string                 `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Version                      uint32                 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
+	ProfileType                  string                 `protobuf:"bytes,3,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
+	NumGemPorts                  uint32                 `protobuf:"varint,4,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
+	InstanceControl              *InstanceControl       `protobuf:"bytes,5,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
+	PackageType                  string                 `protobuf:"bytes,6,opt,name=package_type,json=packageType,proto3" json:"package_type,omitempty"`
+	UpstreamQueueAttributeList   []*EPONQueueAttributes `protobuf:"bytes,7,rep,name=upstream_queue_attribute_list,json=upstreamQueueAttributeList,proto3" json:"upstream_queue_attribute_list,omitempty"`
+	DownstreamQueueAttributeList []*EPONQueueAttributes `protobuf:"bytes,8,rep,name=downstream_queue_attribute_list,json=downstreamQueueAttributeList,proto3" json:"downstream_queue_attribute_list,omitempty"`
+	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
+	XXX_unrecognized             []byte                 `json:"-"`
+	XXX_sizecache                int32                  `json:"-"`
+}
+
+func (m *EponTechProfile) Reset()         { *m = EponTechProfile{} }
+func (m *EponTechProfile) String() string { return proto.CompactTextString(m) }
+func (*EponTechProfile) ProtoMessage()    {}
+func (*EponTechProfile) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{16}
+}
+
+func (m *EponTechProfile) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EponTechProfile.Unmarshal(m, b)
+}
+func (m *EponTechProfile) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EponTechProfile.Marshal(b, m, deterministic)
+}
+func (m *EponTechProfile) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EponTechProfile.Merge(m, src)
+}
+func (m *EponTechProfile) XXX_Size() int {
+	return xxx_messageInfo_EponTechProfile.Size(m)
+}
+func (m *EponTechProfile) XXX_DiscardUnknown() {
+	xxx_messageInfo_EponTechProfile.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EponTechProfile proto.InternalMessageInfo
+
+func (m *EponTechProfile) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *EponTechProfile) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *EponTechProfile) GetProfileType() string {
+	if m != nil {
+		return m.ProfileType
+	}
+	return ""
+}
+
+func (m *EponTechProfile) GetNumGemPorts() uint32 {
+	if m != nil {
+		return m.NumGemPorts
+	}
+	return 0
+}
+
+func (m *EponTechProfile) GetInstanceControl() *InstanceControl {
+	if m != nil {
+		return m.InstanceControl
+	}
+	return nil
+}
+
+func (m *EponTechProfile) GetPackageType() string {
+	if m != nil {
+		return m.PackageType
+	}
+	return ""
+}
+
+func (m *EponTechProfile) GetUpstreamQueueAttributeList() []*EPONQueueAttributes {
+	if m != nil {
+		return m.UpstreamQueueAttributeList
+	}
+	return nil
+}
+
+func (m *EponTechProfile) GetDownstreamQueueAttributeList() []*EPONQueueAttributes {
+	if m != nil {
+		return m.DownstreamQueueAttributeList
+	}
+	return nil
+}
+
+// TechProfile Instance definition (relevant for GPON, XGPON and XGS-PON technologies)
+type TechProfileInstance struct {
+	Name                           string               `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Version                        uint32               `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
+	SubscriberIdentifier           string               `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
+	ProfileType                    string               `protobuf:"bytes,4,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
+	NumGemPorts                    uint32               `protobuf:"varint,5,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
+	InstanceControl                *InstanceControl     `protobuf:"bytes,6,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
+	UsScheduler                    *SchedulerAttributes `protobuf:"bytes,7,opt,name=us_scheduler,json=usScheduler,proto3" json:"us_scheduler,omitempty"`
+	DsScheduler                    *SchedulerAttributes `protobuf:"bytes,8,opt,name=ds_scheduler,json=dsScheduler,proto3" json:"ds_scheduler,omitempty"`
+	UpstreamGemPortAttributeList   []*GemPortAttributes `protobuf:"bytes,9,rep,name=upstream_gem_port_attribute_list,json=upstreamGemPortAttributeList,proto3" json:"upstream_gem_port_attribute_list,omitempty"`
+	DownstreamGemPortAttributeList []*GemPortAttributes `protobuf:"bytes,10,rep,name=downstream_gem_port_attribute_list,json=downstreamGemPortAttributeList,proto3" json:"downstream_gem_port_attribute_list,omitempty"`
+	XXX_NoUnkeyedLiteral           struct{}             `json:"-"`
+	XXX_unrecognized               []byte               `json:"-"`
+	XXX_sizecache                  int32                `json:"-"`
+}
+
+func (m *TechProfileInstance) Reset()         { *m = TechProfileInstance{} }
+func (m *TechProfileInstance) String() string { return proto.CompactTextString(m) }
+func (*TechProfileInstance) ProtoMessage()    {}
+func (*TechProfileInstance) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{17}
+}
+
+func (m *TechProfileInstance) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TechProfileInstance.Unmarshal(m, b)
+}
+func (m *TechProfileInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TechProfileInstance.Marshal(b, m, deterministic)
+}
+func (m *TechProfileInstance) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TechProfileInstance.Merge(m, src)
+}
+func (m *TechProfileInstance) XXX_Size() int {
+	return xxx_messageInfo_TechProfileInstance.Size(m)
+}
+func (m *TechProfileInstance) XXX_DiscardUnknown() {
+	xxx_messageInfo_TechProfileInstance.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TechProfileInstance proto.InternalMessageInfo
+
+func (m *TechProfileInstance) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *TechProfileInstance) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *TechProfileInstance) GetSubscriberIdentifier() string {
+	if m != nil {
+		return m.SubscriberIdentifier
+	}
+	return ""
+}
+
+func (m *TechProfileInstance) GetProfileType() string {
+	if m != nil {
+		return m.ProfileType
+	}
+	return ""
+}
+
+func (m *TechProfileInstance) GetNumGemPorts() uint32 {
+	if m != nil {
+		return m.NumGemPorts
+	}
+	return 0
+}
+
+func (m *TechProfileInstance) GetInstanceControl() *InstanceControl {
+	if m != nil {
+		return m.InstanceControl
+	}
+	return nil
+}
+
+func (m *TechProfileInstance) GetUsScheduler() *SchedulerAttributes {
+	if m != nil {
+		return m.UsScheduler
+	}
+	return nil
+}
+
+func (m *TechProfileInstance) GetDsScheduler() *SchedulerAttributes {
+	if m != nil {
+		return m.DsScheduler
+	}
+	return nil
+}
+
+func (m *TechProfileInstance) GetUpstreamGemPortAttributeList() []*GemPortAttributes {
+	if m != nil {
+		return m.UpstreamGemPortAttributeList
+	}
+	return nil
+}
+
+func (m *TechProfileInstance) GetDownstreamGemPortAttributeList() []*GemPortAttributes {
+	if m != nil {
+		return m.DownstreamGemPortAttributeList
+	}
+	return nil
+}
+
+// EPON TechProfile Instance definition.
+type EponTechProfileInstance struct {
+	Name                         string                 `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Version                      uint32                 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"`
+	SubscriberIdentifier         string                 `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
+	ProfileType                  string                 `protobuf:"bytes,4,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
+	NumGemPorts                  uint32                 `protobuf:"varint,5,opt,name=num_gem_ports,json=numGemPorts,proto3" json:"num_gem_ports,omitempty"`
+	AllocId                      uint32                 `protobuf:"varint,6,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	InstanceControl              *InstanceControl       `protobuf:"bytes,7,opt,name=instance_control,json=instanceControl,proto3" json:"instance_control,omitempty"`
+	PackageType                  string                 `protobuf:"bytes,8,opt,name=package_type,json=packageType,proto3" json:"package_type,omitempty"`
+	UpstreamQueueAttributeList   []*EPONQueueAttributes `protobuf:"bytes,9,rep,name=upstream_queue_attribute_list,json=upstreamQueueAttributeList,proto3" json:"upstream_queue_attribute_list,omitempty"`
+	DownstreamQueueAttributeList []*EPONQueueAttributes `protobuf:"bytes,10,rep,name=downstream_queue_attribute_list,json=downstreamQueueAttributeList,proto3" json:"downstream_queue_attribute_list,omitempty"`
+	XXX_NoUnkeyedLiteral         struct{}               `json:"-"`
+	XXX_unrecognized             []byte                 `json:"-"`
+	XXX_sizecache                int32                  `json:"-"`
+}
+
+func (m *EponTechProfileInstance) Reset()         { *m = EponTechProfileInstance{} }
+func (m *EponTechProfileInstance) String() string { return proto.CompactTextString(m) }
+func (*EponTechProfileInstance) ProtoMessage()    {}
+func (*EponTechProfileInstance) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{18}
+}
+
+func (m *EponTechProfileInstance) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EponTechProfileInstance.Unmarshal(m, b)
+}
+func (m *EponTechProfileInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EponTechProfileInstance.Marshal(b, m, deterministic)
+}
+func (m *EponTechProfileInstance) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EponTechProfileInstance.Merge(m, src)
+}
+func (m *EponTechProfileInstance) XXX_Size() int {
+	return xxx_messageInfo_EponTechProfileInstance.Size(m)
+}
+func (m *EponTechProfileInstance) XXX_DiscardUnknown() {
+	xxx_messageInfo_EponTechProfileInstance.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EponTechProfileInstance proto.InternalMessageInfo
+
+func (m *EponTechProfileInstance) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *EponTechProfileInstance) GetVersion() uint32 {
+	if m != nil {
+		return m.Version
+	}
+	return 0
+}
+
+func (m *EponTechProfileInstance) GetSubscriberIdentifier() string {
+	if m != nil {
+		return m.SubscriberIdentifier
+	}
+	return ""
+}
+
+func (m *EponTechProfileInstance) GetProfileType() string {
+	if m != nil {
+		return m.ProfileType
+	}
+	return ""
+}
+
+func (m *EponTechProfileInstance) GetNumGemPorts() uint32 {
+	if m != nil {
+		return m.NumGemPorts
+	}
+	return 0
+}
+
+func (m *EponTechProfileInstance) GetAllocId() uint32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+func (m *EponTechProfileInstance) GetInstanceControl() *InstanceControl {
+	if m != nil {
+		return m.InstanceControl
+	}
+	return nil
+}
+
+func (m *EponTechProfileInstance) GetPackageType() string {
+	if m != nil {
+		return m.PackageType
+	}
+	return ""
+}
+
+func (m *EponTechProfileInstance) GetUpstreamQueueAttributeList() []*EPONQueueAttributes {
+	if m != nil {
+		return m.UpstreamQueueAttributeList
+	}
+	return nil
+}
+
+func (m *EponTechProfileInstance) GetDownstreamQueueAttributeList() []*EPONQueueAttributes {
+	if m != nil {
+		return m.DownstreamQueueAttributeList
+	}
+	return nil
+}
+
+// Resource Instance definition
+type ResourceInstance struct {
+	TpId                 uint32   `protobuf:"varint,1,opt,name=tp_id,json=tpId,proto3" json:"tp_id,omitempty"`
+	ProfileType          string   `protobuf:"bytes,2,opt,name=profile_type,json=profileType,proto3" json:"profile_type,omitempty"`
+	SubscriberIdentifier string   `protobuf:"bytes,3,opt,name=subscriber_identifier,json=subscriberIdentifier,proto3" json:"subscriber_identifier,omitempty"`
+	AllocId              uint32   `protobuf:"varint,4,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
+	GemportIds           []uint32 `protobuf:"varint,5,rep,packed,name=gemport_ids,json=gemportIds,proto3" json:"gemport_ids,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ResourceInstance) Reset()         { *m = ResourceInstance{} }
+func (m *ResourceInstance) String() string { return proto.CompactTextString(m) }
+func (*ResourceInstance) ProtoMessage()    {}
+func (*ResourceInstance) Descriptor() ([]byte, []int) {
+	return fileDescriptor_d019a68bffe14cae, []int{19}
+}
+
+func (m *ResourceInstance) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ResourceInstance.Unmarshal(m, b)
+}
+func (m *ResourceInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ResourceInstance.Marshal(b, m, deterministic)
+}
+func (m *ResourceInstance) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ResourceInstance.Merge(m, src)
+}
+func (m *ResourceInstance) XXX_Size() int {
+	return xxx_messageInfo_ResourceInstance.Size(m)
+}
+func (m *ResourceInstance) XXX_DiscardUnknown() {
+	xxx_messageInfo_ResourceInstance.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ResourceInstance proto.InternalMessageInfo
+
+func (m *ResourceInstance) GetTpId() uint32 {
+	if m != nil {
+		return m.TpId
+	}
+	return 0
+}
+
+func (m *ResourceInstance) GetProfileType() string {
+	if m != nil {
+		return m.ProfileType
+	}
+	return ""
+}
+
+func (m *ResourceInstance) GetSubscriberIdentifier() string {
+	if m != nil {
+		return m.SubscriberIdentifier
+	}
+	return ""
+}
+
+func (m *ResourceInstance) GetAllocId() uint32 {
+	if m != nil {
+		return m.AllocId
+	}
+	return 0
+}
+
+func (m *ResourceInstance) GetGemportIds() []uint32 {
+	if m != nil {
+		return m.GemportIds
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("tech_profile.Direction", Direction_name, Direction_value)
+	proto.RegisterEnum("tech_profile.SchedulingPolicy", SchedulingPolicy_name, SchedulingPolicy_value)
+	proto.RegisterEnum("tech_profile.AdditionalBW", AdditionalBW_name, AdditionalBW_value)
+	proto.RegisterEnum("tech_profile.DiscardPolicy", DiscardPolicy_name, DiscardPolicy_value)
+	proto.RegisterEnum("tech_profile.InferredAdditionBWIndication", InferredAdditionBWIndication_name, InferredAdditionBWIndication_value)
+	proto.RegisterType((*SchedulerConfig)(nil), "tech_profile.SchedulerConfig")
+	proto.RegisterType((*TrafficShapingInfo)(nil), "tech_profile.TrafficShapingInfo")
+	proto.RegisterType((*TrafficScheduler)(nil), "tech_profile.TrafficScheduler")
+	proto.RegisterType((*TrafficSchedulers)(nil), "tech_profile.TrafficSchedulers")
+	proto.RegisterType((*TailDropDiscardConfig)(nil), "tech_profile.TailDropDiscardConfig")
+	proto.RegisterType((*RedDiscardConfig)(nil), "tech_profile.RedDiscardConfig")
+	proto.RegisterType((*WRedDiscardConfig)(nil), "tech_profile.WRedDiscardConfig")
+	proto.RegisterType((*DiscardConfig)(nil), "tech_profile.DiscardConfig")
+	proto.RegisterType((*TrafficQueue)(nil), "tech_profile.TrafficQueue")
+	proto.RegisterType((*TrafficQueues)(nil), "tech_profile.TrafficQueues")
+	proto.RegisterType((*InstanceControl)(nil), "tech_profile.InstanceControl")
+	proto.RegisterType((*QThresholds)(nil), "tech_profile.QThresholds")
+	proto.RegisterType((*GemPortAttributes)(nil), "tech_profile.GemPortAttributes")
+	proto.RegisterType((*SchedulerAttributes)(nil), "tech_profile.SchedulerAttributes")
+	proto.RegisterType((*EPONQueueAttributes)(nil), "tech_profile.EPONQueueAttributes")
+	proto.RegisterType((*TechProfile)(nil), "tech_profile.TechProfile")
+	proto.RegisterType((*EponTechProfile)(nil), "tech_profile.EponTechProfile")
+	proto.RegisterType((*TechProfileInstance)(nil), "tech_profile.TechProfileInstance")
+	proto.RegisterType((*EponTechProfileInstance)(nil), "tech_profile.EponTechProfileInstance")
+	proto.RegisterType((*ResourceInstance)(nil), "tech_profile.ResourceInstance")
+}
+
+func init() { proto.RegisterFile("voltha_protos/tech_profile.proto", fileDescriptor_d019a68bffe14cae) }
+
+var fileDescriptor_d019a68bffe14cae = []byte{
+	// 2138 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0x4d, 0x73, 0x1c, 0x39,
+	0xf9, 0xf7, 0xcc, 0xd8, 0xf3, 0xf2, 0xcc, 0x8c, 0xdd, 0x96, 0xe3, 0xcd, 0xc4, 0x71, 0x36, 0xce,
+	0xec, 0x7f, 0xff, 0x6b, 0x4c, 0x11, 0xb3, 0xce, 0xcb, 0x1e, 0xb2, 0xb0, 0x65, 0xc7, 0xae, 0x64,
+	0x60, 0xe3, 0xd8, 0x6d, 0x83, 0x29, 0x0e, 0x74, 0xf5, 0xb4, 0x34, 0x63, 0xb1, 0x3d, 0x52, 0x5b,
+	0x52, 0xc7, 0xf1, 0x9e, 0x28, 0xaa, 0xf8, 0x14, 0xc0, 0x85, 0x2a, 0xae, 0x5c, 0xf6, 0x42, 0xc1,
+	0x85, 0x2a, 0xbe, 0x05, 0x9f, 0x80, 0x2a, 0xbe, 0x04, 0x25, 0x75, 0xf7, 0xf4, 0xcb, 0x4c, 0x12,
+	0x27, 0x38, 0x5b, 0x05, 0x37, 0xe9, 0xd1, 0x4f, 0x8f, 0x1e, 0x3d, 0x2f, 0x3f, 0x49, 0xdd, 0xb0,
+	0xf6, 0x82, 0xfb, 0xea, 0xd4, 0x75, 0x02, 0xc1, 0x15, 0x97, 0x9b, 0x8a, 0x78, 0xa7, 0xba, 0x3d,
+	0xa0, 0x3e, 0xb9, 0x6b, 0x64, 0xa8, 0x95, 0x95, 0xad, 0xac, 0x0e, 0x39, 0x1f, 0xfa, 0x64, 0xd3,
+	0x0d, 0xe8, 0xa6, 0xcb, 0x18, 0x57, 0xae, 0xa2, 0x9c, 0xc9, 0x08, 0xdb, 0xfd, 0x55, 0x19, 0x16,
+	0x8e, 0xbc, 0x53, 0x82, 0x43, 0x9f, 0x88, 0xc7, 0x9c, 0x0d, 0xe8, 0x10, 0x3d, 0x80, 0x06, 0xa6,
+	0x82, 0x78, 0x1a, 0xd7, 0x29, 0xad, 0x95, 0xd6, 0xe7, 0xb7, 0xae, 0xdf, 0xcd, 0xad, 0xb3, 0x9b,
+	0x0c, 0xdb, 0x29, 0x12, 0x7d, 0x01, 0x6d, 0x17, 0x63, 0xaa, 0xdb, 0xae, 0xef, 0xf4, 0xcf, 0x3b,
+	0x65, 0x33, 0x75, 0x25, 0x3f, 0x75, 0x7b, 0x0c, 0xd9, 0x39, 0xb1, 0x5b, 0xe9, 0x84, 0x9d, 0x73,
+	0xb4, 0x02, 0xf5, 0x40, 0x50, 0x2e, 0xa8, 0xba, 0xe8, 0x54, 0xd6, 0x4a, 0xeb, 0x35, 0x7b, 0xdc,
+	0x47, 0x1f, 0x40, 0xf5, 0x9c, 0xd0, 0xe1, 0xa9, 0xea, 0xcc, 0x9a, 0x91, 0xb8, 0x87, 0xb6, 0xa1,
+	0x25, 0xb5, 0xf9, 0x4e, 0xc0, 0x7d, 0xea, 0x5d, 0x74, 0xe6, 0xcc, 0x9a, 0x1f, 0xe6, 0xd7, 0x8c,
+	0x37, 0x48, 0xd9, 0xf0, 0xc0, 0xa0, 0xec, 0xa6, 0x99, 0x13, 0x75, 0xba, 0x7f, 0x2e, 0x01, 0x3a,
+	0x16, 0xee, 0x60, 0x40, 0xbd, 0xa3, 0x53, 0x37, 0xa0, 0x6c, 0xd8, 0x63, 0x03, 0x8e, 0x2c, 0xa8,
+	0x78, 0x54, 0x98, 0xfd, 0xd7, 0x6c, 0xdd, 0x34, 0x92, 0xbe, 0x34, 0xdb, 0xd2, 0x92, 0xbe, 0xd4,
+	0x92, 0x80, 0x8a, 0xd8, 0x58, 0xdd, 0x34, 0x92, 0xbe, 0x8c, 0x8d, 0xd4, 0x4d, 0x2d, 0x19, 0x52,
+	0x61, 0x0c, 0xab, 0xd9, 0xba, 0x89, 0x9e, 0x02, 0xb8, 0x18, 0x3b, 0xfd, 0x73, 0x87, 0x32, 0xdc,
+	0xa9, 0x1a, 0x8b, 0x37, 0xf2, 0x16, 0xf7, 0xd8, 0x80, 0x08, 0x41, 0x70, 0xe2, 0xad, 0x9d, 0x93,
+	0x1e, 0xc3, 0xd4, 0x33, 0xa1, 0xb3, 0xeb, 0x2e, 0xc6, 0x3b, 0xe7, 0x3d, 0x86, 0xbb, 0xbf, 0x2f,
+	0x83, 0x95, 0x98, 0x9e, 0x04, 0xf1, 0x5d, 0xc3, 0x77, 0x03, 0xea, 0xae, 0xef, 0x73, 0xcf, 0xa1,
+	0x38, 0xde, 0x62, 0xcd, 0xf4, 0x7b, 0x18, 0x3d, 0x82, 0x86, 0x4c, 0xd4, 0x9b, 0xcd, 0x36, 0xb7,
+	0x6e, 0x4d, 0xf5, 0x70, 0x92, 0x42, 0x76, 0x8a, 0x47, 0x36, 0x5c, 0x53, 0x91, 0x89, 0x8e, 0x8c,
+	0xdc, 0xeb, 0x50, 0x36, 0xe0, 0xc6, 0x45, 0xcd, 0xad, 0xb5, 0xbc, 0x9e, 0xc9, 0x38, 0xd8, 0x48,
+	0x4d, 0xc6, 0xe6, 0xff, 0x61, 0x21, 0x3b, 0x4d, 0x9b, 0x1c, 0xf9, 0xb7, 0xad, 0xc5, 0x07, 0x91,
+	0xb4, 0x87, 0xbb, 0x7f, 0x29, 0xc1, 0x62, 0xd1, 0x3f, 0x12, 0x5d, 0x87, 0x1a, 0x65, 0x6a, 0xa0,
+	0x67, 0x45, 0xd1, 0xad, 0xea, 0x6e, 0x0f, 0xa3, 0x65, 0xa8, 0x72, 0x16, 0xa6, 0x0e, 0x98, 0xe3,
+	0x2c, 0x8c, 0xc4, 0x21, 0xa3, 0x5a, 0x1c, 0x85, 0x75, 0x2e, 0x64, 0xb4, 0x87, 0xb5, 0x9a, 0x80,
+	0x0b, 0xe5, 0x30, 0x1e, 0x2f, 0x5e, 0xd5, 0xdd, 0x7d, 0x8e, 0xf6, 0x60, 0x7e, 0xbc, 0x63, 0xbd,
+	0xaa, 0xec, 0x54, 0xd6, 0x2a, 0xeb, 0xcd, 0x62, 0x56, 0x16, 0x0d, 0xb3, 0xdb, 0x2a, 0x23, 0x91,
+	0xdd, 0x87, 0xb0, 0x7c, 0xec, 0x52, 0x7f, 0x57, 0xf0, 0x60, 0x97, 0x4a, 0xcf, 0x15, 0x38, 0xae,
+	0xcf, 0x5b, 0x00, 0x67, 0x21, 0x09, 0x89, 0x23, 0xe9, 0xd7, 0x24, 0xde, 0x42, 0xc3, 0x48, 0x8e,
+	0xe8, 0xd7, 0xa4, 0xfb, 0x9b, 0x12, 0x58, 0x36, 0xc1, 0xf9, 0x39, 0x1f, 0x41, 0x7b, 0x44, 0x99,
+	0xa3, 0x4e, 0x05, 0x91, 0xa7, 0xdc, 0x4f, 0x76, 0xde, 0x1a, 0x51, 0x76, 0x9c, 0xc8, 0x0c, 0xc8,
+	0x7d, 0x99, 0x01, 0x95, 0x63, 0x90, 0xfb, 0x32, 0x05, 0x7d, 0x02, 0x0b, 0x1a, 0x14, 0x08, 0xde,
+	0x77, 0xfb, 0xd4, 0x4f, 0x8b, 0x75, 0x7e, 0xe4, 0xbe, 0x3c, 0x48, 0xa5, 0xdd, 0x6f, 0x4a, 0xb0,
+	0x78, 0x32, 0x61, 0xc8, 0x7d, 0x98, 0x1b, 0x0a, 0x42, 0xa2, 0xcc, 0x9c, 0xf0, 0x49, 0x11, 0x6e,
+	0x47, 0x60, 0xf4, 0x10, 0xaa, 0x17, 0xc4, 0xf7, 0x79, 0x44, 0x2a, 0x6f, 0x9e, 0x16, 0xa3, 0xd1,
+	0xf7, 0xa1, 0x22, 0x08, 0x8e, 0x73, 0xf6, 0x4d, 0x93, 0x34, 0xb4, 0xfb, 0xcf, 0x32, 0xb4, 0xf3,
+	0x16, 0xef, 0xc0, 0x3c, 0x8e, 0x04, 0x09, 0xc9, 0x44, 0x45, 0x75, 0xb3, 0x58, 0x54, 0x06, 0x13,
+	0x33, 0x4c, 0x1b, 0x67, 0xbb, 0xe8, 0x17, 0xd0, 0x51, 0x2e, 0xf5, 0x1d, 0x2c, 0x78, 0xe0, 0x24,
+	0xda, 0x3c, 0xa3, 0x3f, 0xde, 0xd1, 0x47, 0x85, 0xe4, 0x98, 0x16, 0xf9, 0xa7, 0x33, 0xf6, 0xb2,
+	0x9a, 0x9a, 0x12, 0xfb, 0x80, 0x04, 0xc1, 0x45, 0xcd, 0x97, 0xda, 0xf6, 0xd3, 0x19, 0xdb, 0x12,
+	0xc5, 0x28, 0x1d, 0xc2, 0xd2, 0xf9, 0x14, 0x85, 0x51, 0xcd, 0xde, 0xce, 0x2b, 0x3c, 0x99, 0xa2,
+	0x71, 0xf1, 0xbc, 0xa8, 0x72, 0xc7, 0x4a, 0xdd, 0x18, 0x69, 0xeb, 0xfe, 0xb1, 0x02, 0xad, 0xb8,
+	0x08, 0x0e, 0x75, 0xf6, 0xbe, 0x2b, 0x73, 0xdd, 0x02, 0x18, 0x92, 0x91, 0xa9, 0xc5, 0x71, 0xe9,
+	0x36, 0x62, 0x49, 0x0f, 0x6b, 0x62, 0x0b, 0xfa, 0x54, 0x39, 0x23, 0x37, 0x30, 0x1e, 0x69, 0xd8,
+	0x35, 0xdd, 0x7f, 0xe6, 0x06, 0xe8, 0x63, 0x98, 0x77, 0x89, 0x74, 0x08, 0xf3, 0xc4, 0x45, 0x60,
+	0x56, 0xd5, 0x3b, 0xac, 0xdb, 0x6d, 0x97, 0xc8, 0xbd, 0xb1, 0xf0, 0x0a, 0x0e, 0x99, 0xdc, 0xd9,
+	0x56, 0x7d, 0xe5, 0xd9, 0x56, 0xcb, 0x9d, 0x6d, 0x93, 0x89, 0x57, 0x7f, 0xeb, 0xc4, 0xdb, 0x29,
+	0x7a, 0xbd, 0xd3, 0x30, 0x31, 0x9c, 0xae, 0x23, 0x2e, 0x84, 0x44, 0x47, 0xd4, 0xed, 0xfe, 0xa3,
+	0x04, 0xed, 0x6c, 0x9c, 0xde, 0x3f, 0x83, 0x6e, 0xa7, 0x0c, 0x6a, 0x78, 0x4d, 0x76, 0xaa, 0x86,
+	0x41, 0x57, 0xa6, 0x32, 0xa8, 0x31, 0x6a, 0xcc, 0x9e, 0xb1, 0x89, 0x53, 0x8e, 0x88, 0xda, 0xb4,
+	0x23, 0x62, 0x00, 0x0b, 0x3d, 0x26, 0x95, 0xcb, 0x3c, 0xf2, 0x98, 0x33, 0x25, 0xb8, 0xaf, 0x4f,
+	0x6c, 0xce, 0x42, 0xb3, 0xb3, 0x86, 0xad, 0x9b, 0x5a, 0x12, 0x32, 0x6a, 0xf6, 0xd4, 0xb0, 0x75,
+	0x13, 0x6d, 0xc2, 0x35, 0xcd, 0x82, 0x43, 0x32, 0x72, 0x02, 0xf7, 0xc2, 0xe7, 0x2e, 0x8e, 0xd8,
+	0x38, 0x4a, 0xb0, 0xc5, 0x91, 0xfb, 0xf2, 0x09, 0x19, 0x1d, 0x44, 0x23, 0x86, 0x95, 0x7f, 0x5d,
+	0x86, 0xe6, 0xe1, 0x98, 0x45, 0x25, 0xba, 0x03, 0xad, 0xb3, 0x94, 0x69, 0x3f, 0x35, 0xab, 0xb5,
+	0xed, 0xe6, 0xd9, 0x18, 0xf2, 0x69, 0x01, 0xb2, 0x65, 0x96, 0xcf, 0x41, 0xb6, 0x0a, 0x90, 0x7b,
+	0x66, 0xf9, 0x1c, 0xe4, 0x5e, 0x01, 0x72, 0xdf, 0x44, 0x20, 0x07, 0xb9, 0x5f, 0x80, 0x3c, 0x30,
+	0xc1, 0xc8, 0x41, 0x1e, 0x14, 0x20, 0x0f, 0x4d, 0x0e, 0xe7, 0x20, 0x0f, 0x0b, 0x90, 0xcf, 0x8c,
+	0xbb, 0x73, 0x90, 0xcf, 0xba, 0xdf, 0xcc, 0xc1, 0xa2, 0xf6, 0x0b, 0x17, 0x6a, 0x5b, 0x29, 0x41,
+	0xfb, 0xa1, 0x22, 0xb2, 0x50, 0xbf, 0xa5, 0x62, 0xfd, 0xae, 0x02, 0x68, 0x57, 0x9f, 0x45, 0x0e,
+	0x8e, 0x62, 0x50, 0x1f, 0xb9, 0x2f, 0x0f, 0xb5, 0x5f, 0xdf, 0xbe, 0xba, 0x1b, 0xc5, 0xea, 0xfe,
+	0x31, 0x2c, 0xca, 0x71, 0xed, 0xbe, 0x5d, 0x89, 0x5b, 0xb2, 0x20, 0xd1, 0x7b, 0x49, 0xea, 0xda,
+	0x39, 0x8b, 0x2b, 0xbd, 0x91, 0x48, 0x0e, 0xdf, 0x6b, 0xa9, 0xef, 0xbd, 0xa2, 0xd4, 0xdf, 0x74,
+	0xec, 0xe5, 0xab, 0x1d, 0x3d, 0x81, 0xc5, 0xbc, 0x1a, 0xe7, 0xc5, 0x56, 0x67, 0xfe, 0xcd, 0xa4,
+	0xb1, 0x90, 0x53, 0xf3, 0x53, 0x93, 0x9b, 0x54, 0x3a, 0xa3, 0xd0, 0x57, 0xd4, 0x73, 0xa5, 0xea,
+	0x80, 0x71, 0x7e, 0x93, 0xca, 0x67, 0x89, 0x08, 0xad, 0x83, 0x35, 0x1e, 0x37, 0xb5, 0x44, 0x71,
+	0xa7, 0x19, 0x5f, 0x26, 0x12, 0xf9, 0x13, 0x32, 0xea, 0x61, 0xf4, 0x03, 0xb8, 0x89, 0x2f, 0x98,
+	0x3b, 0xa2, 0x9e, 0xe3, 0x7a, 0x1e, 0x91, 0x52, 0x1b, 0xa7, 0xab, 0xd5, 0xf1, 0xa9, 0x54, 0x9d,
+	0x96, 0xd1, 0xdd, 0x89, 0x21, 0xdb, 0x06, 0x11, 0x97, 0xf3, 0x97, 0x54, 0x2a, 0xf4, 0x08, 0x56,
+	0xa4, 0x7e, 0xf8, 0x4c, 0x9f, 0xdd, 0x36, 0xb3, 0xaf, 0x47, 0x88, 0x89, 0xc9, 0xdd, 0x3f, 0x94,
+	0x61, 0x69, 0x7c, 0x4b, 0xcb, 0xe4, 0xed, 0x15, 0x5d, 0xb4, 0xdb, 0xe9, 0x45, 0x7b, 0xe2, 0x09,
+	0x55, 0xf9, 0x0f, 0x9e, 0x50, 0xb3, 0xaf, 0x3c, 0x66, 0xe6, 0x72, 0xb9, 0xb7, 0x0b, 0xf3, 0x67,
+	0x4e, 0xee, 0x7c, 0xab, 0x5e, 0x2a, 0xf9, 0x5b, 0x67, 0x47, 0x99, 0x57, 0xd4, 0xdf, 0xaa, 0xb0,
+	0xb4, 0x77, 0xf0, 0x7c, 0xdf, 0xd0, 0x6f, 0xc6, 0x49, 0xf9, 0xea, 0x2d, 0xbd, 0xa6, 0x7a, 0xcb,
+	0xf9, 0xea, 0xcd, 0xb3, 0x42, 0x44, 0x6c, 0x19, 0x56, 0xb8, 0x64, 0x71, 0xdf, 0x81, 0x56, 0x72,
+	0x92, 0xa8, 0x8b, 0x80, 0x98, 0xad, 0x37, 0xec, 0x66, 0x2c, 0x3b, 0xbe, 0x08, 0x08, 0xba, 0x0f,
+	0x1f, 0x84, 0x4c, 0xea, 0x5d, 0x50, 0x45, 0xb0, 0x33, 0x14, 0x2e, 0x53, 0x91, 0xb5, 0x11, 0xc9,
+	0x5d, 0xcb, 0x8c, 0x3e, 0xd1, 0x83, 0xc6, 0xf2, 0xef, 0x80, 0xc5, 0xf8, 0x88, 0xea, 0x38, 0x51,
+	0xa6, 0x88, 0x78, 0xe1, 0xfa, 0x31, 0xe3, 0x2d, 0xc4, 0xf2, 0x5e, 0x2c, 0x46, 0x5b, 0xb0, 0xac,
+	0xb8, 0x4f, 0x84, 0xab, 0x22, 0x17, 0xfb, 0xce, 0x2f, 0xa9, 0x52, 0x44, 0x98, 0x1a, 0x6f, 0xdb,
+	0x4b, 0xe3, 0xc1, 0x03, 0xee, 0xfb, 0x3f, 0x32, 0x43, 0xe8, 0x87, 0x70, 0x53, 0x90, 0xb3, 0x90,
+	0x48, 0xe5, 0x28, 0xe1, 0x32, 0x39, 0xa2, 0x52, 0x52, 0xce, 0x92, 0x08, 0x35, 0xcc, 0xcc, 0x1b,
+	0x31, 0xe4, 0x38, 0x83, 0x88, 0xc9, 0x60, 0x15, 0x80, 0x85, 0x23, 0xed, 0x76, 0xa2, 0xa4, 0x29,
+	0xbd, 0xb6, 0x5d, 0x67, 0xe1, 0xe8, 0xf0, 0x88, 0x28, 0x89, 0x3e, 0xcf, 0x51, 0xb5, 0x34, 0x35,
+	0xd7, 0xdc, 0xba, 0x91, 0x0f, 0x78, 0xe6, 0xb4, 0xca, 0xb2, 0xb8, 0x9c, 0x4e, 0x98, 0xad, 0x2b,
+	0x21, 0xcc, 0x76, 0x14, 0xe6, 0x69, 0x84, 0x39, 0x6f, 0x86, 0x5e, 0x4d, 0x98, 0x0b, 0x57, 0x40,
+	0x98, 0xd6, 0x95, 0x11, 0xe6, 0xe2, 0xdb, 0x13, 0x66, 0xf7, 0x4f, 0xb3, 0xd0, 0x3c, 0x4e, 0x2f,
+	0x27, 0x08, 0xc1, 0x2c, 0x73, 0x47, 0x49, 0xd1, 0x98, 0x36, 0xea, 0x40, 0xed, 0x05, 0x11, 0x3a,
+	0xd0, 0x09, 0x77, 0xc4, 0x5d, 0x9d, 0xe9, 0xc9, 0x5d, 0xc7, 0x64, 0x7a, 0x74, 0x18, 0x36, 0x63,
+	0x99, 0xc9, 0xf4, 0x2e, 0xb4, 0x75, 0x52, 0x98, 0x4b, 0x0b, 0x17, 0x4a, 0x26, 0x77, 0x01, 0x16,
+	0x8e, 0xe2, 0x53, 0x59, 0xa2, 0xa7, 0x60, 0xd1, 0xf8, 0x3e, 0x94, 0x90, 0xa4, 0x29, 0x9a, 0x89,
+	0x27, 0x7f, 0xe1, 0xd6, 0x64, 0x2f, 0xd0, 0xc2, 0x35, 0x6a, 0x17, 0x5a, 0xa1, 0x74, 0xd2, 0x0f,
+	0x07, 0x55, 0xa3, 0xe5, 0xce, 0x2b, 0x3e, 0x1c, 0xa4, 0x94, 0x61, 0x37, 0x43, 0x99, 0x7e, 0xcd,
+	0xd8, 0x85, 0x16, 0xce, 0x6a, 0xa9, 0x5d, 0x5a, 0x0b, 0xce, 0x68, 0x19, 0xc2, 0x5a, 0x18, 0x48,
+	0x25, 0x88, 0x9b, 0x6e, 0xdf, 0x71, 0x13, 0x70, 0x74, 0x0a, 0xd4, 0xcd, 0x15, 0xb3, 0xf0, 0xb8,
+	0x99, 0xb8, 0xad, 0xd8, 0xab, 0x89, 0xa2, 0xe2, 0x90, 0x39, 0x68, 0xbe, 0x82, 0x2e, 0xe6, 0xe7,
+	0xec, 0x0d, 0x4b, 0x35, 0x2e, 0xb7, 0xd4, 0x87, 0xa9, 0xaa, 0x69, 0x8b, 0x75, 0xff, 0x5e, 0x81,
+	0x85, 0xbd, 0x80, 0xb3, 0xff, 0xa1, 0xa4, 0xd1, 0x06, 0xb9, 0xde, 0x57, 0xee, 0x30, 0x36, 0xa8,
+	0x1a, 0x1b, 0x14, 0xc9, 0x8c, 0x41, 0x18, 0x6e, 0x8d, 0x63, 0x19, 0x7d, 0x07, 0x29, 0x78, 0xb7,
+	0x66, 0xbc, 0x5b, 0x48, 0x91, 0x29, 0x67, 0x93, 0xbd, 0x92, 0xe8, 0xc9, 0x0f, 0x98, 0x40, 0x9e,
+	0xc2, 0xed, 0x4c, 0x20, 0xa7, 0xae, 0x53, 0xbf, 0xec, 0x3a, 0xab, 0xa9, 0xa6, 0xc9, 0x95, 0xba,
+	0xff, 0x9a, 0x85, 0xa5, 0x4c, 0x04, 0x13, 0x17, 0xbd, 0x65, 0x24, 0xef, 0xc1, 0xb2, 0x0c, 0xfb,
+	0xd2, 0x13, 0xb4, 0x4f, 0x84, 0x43, 0x31, 0x61, 0x8a, 0x0e, 0x68, 0xfc, 0xbd, 0xae, 0x61, 0x5f,
+	0x4b, 0x07, 0x7b, 0xe3, 0xb1, 0x89, 0xf0, 0xcf, 0x5e, 0x22, 0xfc, 0x73, 0x97, 0x0b, 0x7f, 0xf5,
+	0x4a, 0x38, 0xa3, 0x76, 0x25, 0x9c, 0x51, 0x7f, 0x6f, 0x9c, 0xd1, 0xf8, 0xf6, 0x38, 0x03, 0xae,
+	0x86, 0x33, 0x7e, 0x3b, 0x0b, 0xd7, 0x0b, 0x9c, 0xf1, 0x5f, 0x98, 0x71, 0xd9, 0x3b, 0x74, 0x35,
+	0x7f, 0x87, 0x9e, 0x96, 0x8c, 0xb5, 0x2b, 0xe1, 0xa2, 0xfa, 0x3b, 0x70, 0x51, 0xe3, 0x5b, 0xe2,
+	0x22, 0xb8, 0x1a, 0x2e, 0xfa, 0xab, 0xf9, 0x76, 0x2c, 0x79, 0x28, 0xbc, 0x34, 0x2d, 0x96, 0x60,
+	0x4e, 0x05, 0xc9, 0xd3, 0xbc, 0x6d, 0xcf, 0xaa, 0xa0, 0x87, 0x27, 0x02, 0x59, 0x9e, 0x0c, 0xe4,
+	0x3b, 0x25, 0x48, 0x36, 0xb2, 0xb3, 0xf9, 0xc8, 0xde, 0x86, 0x66, 0xfa, 0x22, 0xd0, 0x69, 0x51,
+	0x59, 0x6f, 0xdb, 0x30, 0x7e, 0x12, 0xc8, 0x8d, 0xcf, 0xa1, 0x31, 0x7e, 0x71, 0xa1, 0x16, 0xd4,
+	0x7f, 0x72, 0x70, 0x74, 0x6c, 0xef, 0x6d, 0x3f, 0xb3, 0x66, 0xd0, 0x3c, 0xc0, 0xee, 0xf3, 0x93,
+	0xfd, 0xb8, 0x5f, 0x42, 0x8b, 0xd0, 0xde, 0xe9, 0xed, 0xf6, 0xec, 0xbd, 0xc7, 0xc7, 0xbd, 0xe7,
+	0xfb, 0xdb, 0x5f, 0x5a, 0xe5, 0x8d, 0x47, 0x60, 0x15, 0xef, 0xab, 0xa8, 0x06, 0x95, 0x13, 0xdb,
+	0xb6, 0x66, 0x10, 0x82, 0xf9, 0x23, 0x25, 0xa8, 0xa7, 0x0e, 0xe2, 0xab, 0xa9, 0x55, 0x42, 0x00,
+	0xd5, 0xa7, 0x17, 0x7d, 0x41, 0xb1, 0x55, 0xde, 0x60, 0xd0, 0xca, 0x3e, 0xcb, 0xd0, 0x32, 0x2c,
+	0x66, 0xfb, 0xce, 0x3e, 0x67, 0xc4, 0x9a, 0x41, 0x4b, 0xb0, 0x90, 0x17, 0x6f, 0x5b, 0x25, 0x74,
+	0x13, 0xae, 0xe7, 0x84, 0x3b, 0x44, 0xaa, 0xbd, 0xc1, 0x80, 0x0b, 0x65, 0x95, 0x27, 0x14, 0x6d,
+	0x87, 0x8a, 0x5b, 0x95, 0x8d, 0x2f, 0xc6, 0x5f, 0xa9, 0x63, 0x4b, 0x5b, 0x50, 0x4f, 0xbe, 0x19,
+	0x5b, 0x33, 0xa8, 0x0d, 0x8d, 0x93, 0x71, 0xb7, 0xa4, 0xb7, 0x61, 0x13, 0x6c, 0x95, 0x51, 0x1d,
+	0x66, 0x4f, 0x74, 0xab, 0xb2, 0xf1, 0xbb, 0x12, 0xac, 0xbe, 0xee, 0x2f, 0x13, 0xfa, 0x18, 0xee,
+	0xbc, 0x6e, 0x3c, 0xd9, 0xd1, 0x3a, 0xfc, 0xdf, 0x6b, 0x61, 0xdb, 0x52, 0x86, 0x82, 0x60, 0xab,
+	0x84, 0xbe, 0x0b, 0x9f, 0xbc, 0x16, 0x99, 0xdd, 0xf6, 0xce, 0xcf, 0x60, 0x8d, 0x8b, 0xe1, 0x5d,
+	0x1e, 0x10, 0xe6, 0x71, 0x81, 0xef, 0x46, 0x3f, 0x3d, 0x73, 0xe9, 0xfd, 0xf3, 0xfb, 0x43, 0xaa,
+	0x4e, 0xc3, 0xfe, 0x5d, 0x8f, 0x8f, 0x36, 0x13, 0xe0, 0x66, 0x04, 0xfc, 0x5e, 0xfc, 0x77, 0xf4,
+	0xc5, 0x83, 0xcd, 0x21, 0xcf, 0xfd, 0x23, 0xed, 0x57, 0xcd, 0xd0, 0xbd, 0x7f, 0x07, 0x00, 0x00,
+	0xff, 0xff, 0xc9, 0xfb, 0x08, 0x8b, 0x48, 0x1d, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/adapter.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/adapter.pb.go
new file mode 100644
index 0000000..85aeb0a
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/adapter.pb.go
@@ -0,0 +1,269 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/adapter.proto
+
+package voltha
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	any "github.com/golang/protobuf/ptypes/any"
+	_ "github.com/opencord/voltha-protos/v5/go/common"
+	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 AdapterConfig struct {
+	// Custom (vendor-specific) configuration attributes
+	AdditionalConfig     *any.Any `protobuf:"bytes,64,opt,name=additional_config,json=additionalConfig,proto3" json:"additional_config,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *AdapterConfig) Reset()         { *m = AdapterConfig{} }
+func (m *AdapterConfig) String() string { return proto.CompactTextString(m) }
+func (*AdapterConfig) ProtoMessage()    {}
+func (*AdapterConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7e998ce153307274, []int{0}
+}
+
+func (m *AdapterConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_AdapterConfig.Unmarshal(m, b)
+}
+func (m *AdapterConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_AdapterConfig.Marshal(b, m, deterministic)
+}
+func (m *AdapterConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AdapterConfig.Merge(m, src)
+}
+func (m *AdapterConfig) XXX_Size() int {
+	return xxx_messageInfo_AdapterConfig.Size(m)
+}
+func (m *AdapterConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_AdapterConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AdapterConfig proto.InternalMessageInfo
+
+func (m *AdapterConfig) GetAdditionalConfig() *any.Any {
+	if m != nil {
+		return m.AdditionalConfig
+	}
+	return nil
+}
+
+// Adapter (software plugin)
+type Adapter struct {
+	// the adapter ID has to be unique,
+	// it will be generated as Type + CurrentReplica
+	Id      string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	Vendor  string `protobuf:"bytes,2,opt,name=vendor,proto3" json:"vendor,omitempty"`
+	Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"`
+	// Adapter configuration
+	Config *AdapterConfig `protobuf:"bytes,16,opt,name=config,proto3" json:"config,omitempty"`
+	// Custom descriptors and custom configuration
+	AdditionalDescription *any.Any `protobuf:"bytes,64,opt,name=additional_description,json=additionalDescription,proto3" json:"additional_description,omitempty"`
+	LogicalDeviceIds      []string `protobuf:"bytes,4,rep,name=logical_device_ids,json=logicalDeviceIds,proto3" json:"logical_device_ids,omitempty"`
+	// timestamp when the adapter last sent a message to the core
+	LastCommunication int64  `protobuf:"varint,5,opt,name=last_communication,json=lastCommunication,proto3" json:"last_communication,omitempty"`
+	CurrentReplica    int32  `protobuf:"varint,6,opt,name=currentReplica,proto3" json:"currentReplica,omitempty"`
+	TotalReplicas     int32  `protobuf:"varint,7,opt,name=totalReplicas,proto3" json:"totalReplicas,omitempty"`
+	Endpoint          string `protobuf:"bytes,8,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
+	// all replicas of the same adapter will have the same type
+	// it is used to associate a device to an adapter
+	Type                 string   `protobuf:"bytes,9,opt,name=type,proto3" json:"type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Adapter) Reset()         { *m = Adapter{} }
+func (m *Adapter) String() string { return proto.CompactTextString(m) }
+func (*Adapter) ProtoMessage()    {}
+func (*Adapter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7e998ce153307274, []int{1}
+}
+
+func (m *Adapter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Adapter.Unmarshal(m, b)
+}
+func (m *Adapter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Adapter.Marshal(b, m, deterministic)
+}
+func (m *Adapter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Adapter.Merge(m, src)
+}
+func (m *Adapter) XXX_Size() int {
+	return xxx_messageInfo_Adapter.Size(m)
+}
+func (m *Adapter) XXX_DiscardUnknown() {
+	xxx_messageInfo_Adapter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Adapter proto.InternalMessageInfo
+
+func (m *Adapter) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *Adapter) GetVendor() string {
+	if m != nil {
+		return m.Vendor
+	}
+	return ""
+}
+
+func (m *Adapter) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *Adapter) GetConfig() *AdapterConfig {
+	if m != nil {
+		return m.Config
+	}
+	return nil
+}
+
+func (m *Adapter) GetAdditionalDescription() *any.Any {
+	if m != nil {
+		return m.AdditionalDescription
+	}
+	return nil
+}
+
+func (m *Adapter) GetLogicalDeviceIds() []string {
+	if m != nil {
+		return m.LogicalDeviceIds
+	}
+	return nil
+}
+
+func (m *Adapter) GetLastCommunication() int64 {
+	if m != nil {
+		return m.LastCommunication
+	}
+	return 0
+}
+
+func (m *Adapter) GetCurrentReplica() int32 {
+	if m != nil {
+		return m.CurrentReplica
+	}
+	return 0
+}
+
+func (m *Adapter) GetTotalReplicas() int32 {
+	if m != nil {
+		return m.TotalReplicas
+	}
+	return 0
+}
+
+func (m *Adapter) GetEndpoint() string {
+	if m != nil {
+		return m.Endpoint
+	}
+	return ""
+}
+
+func (m *Adapter) GetType() string {
+	if m != nil {
+		return m.Type
+	}
+	return ""
+}
+
+type Adapters struct {
+	Items                []*Adapter `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
+	XXX_unrecognized     []byte     `json:"-"`
+	XXX_sizecache        int32      `json:"-"`
+}
+
+func (m *Adapters) Reset()         { *m = Adapters{} }
+func (m *Adapters) String() string { return proto.CompactTextString(m) }
+func (*Adapters) ProtoMessage()    {}
+func (*Adapters) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7e998ce153307274, []int{2}
+}
+
+func (m *Adapters) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Adapters.Unmarshal(m, b)
+}
+func (m *Adapters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Adapters.Marshal(b, m, deterministic)
+}
+func (m *Adapters) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Adapters.Merge(m, src)
+}
+func (m *Adapters) XXX_Size() int {
+	return xxx_messageInfo_Adapters.Size(m)
+}
+func (m *Adapters) XXX_DiscardUnknown() {
+	xxx_messageInfo_Adapters.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Adapters proto.InternalMessageInfo
+
+func (m *Adapters) GetItems() []*Adapter {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*AdapterConfig)(nil), "voltha.AdapterConfig")
+	proto.RegisterType((*Adapter)(nil), "voltha.Adapter")
+	proto.RegisterType((*Adapters)(nil), "voltha.Adapters")
+}
+
+func init() { proto.RegisterFile("voltha_protos/adapter.proto", fileDescriptor_7e998ce153307274) }
+
+var fileDescriptor_7e998ce153307274 = []byte{
+	// 439 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x92, 0xc1, 0x8e, 0xd3, 0x30,
+	0x14, 0x45, 0x95, 0x66, 0x9a, 0xb6, 0x6f, 0x54, 0xe8, 0x18, 0x8a, 0x4c, 0xd1, 0x88, 0xa8, 0x02,
+	0x94, 0x05, 0x4d, 0xc4, 0x20, 0xf6, 0xb4, 0x33, 0x1b, 0xb6, 0x16, 0x62, 0xc1, 0xa6, 0x72, 0x6d,
+	0x4f, 0xc6, 0x52, 0xe2, 0x17, 0x25, 0x6e, 0xa4, 0xfe, 0x1c, 0x7f, 0xc0, 0x7f, 0xf0, 0x05, 0xac,
+	0x51, 0x1d, 0x97, 0xb6, 0xb3, 0x98, 0x55, 0x7b, 0xef, 0x3d, 0xbe, 0xf6, 0x7b, 0x0a, 0xbc, 0x69,
+	0xb1, 0xb0, 0x0f, 0x7c, 0x5d, 0xd5, 0x68, 0xb1, 0xc9, 0xb8, 0xe4, 0x95, 0x55, 0x75, 0xea, 0x24,
+	0x89, 0xba, 0x70, 0x46, 0xcf, 0xa1, 0x52, 0x59, 0xde, 0x11, 0xb3, 0xd7, 0x39, 0x62, 0x5e, 0xa8,
+	0xcc, 0xa9, 0xcd, 0xf6, 0x3e, 0xe3, 0x66, 0xd7, 0x45, 0x73, 0x06, 0xe3, 0x65, 0xd7, 0x76, 0x8b,
+	0xe6, 0x5e, 0xe7, 0x64, 0x09, 0x57, 0x5c, 0x4a, 0x6d, 0x35, 0x1a, 0x5e, 0xac, 0x85, 0x33, 0xe9,
+	0xd7, 0x38, 0x48, 0x2e, 0x6f, 0x5e, 0xa6, 0x5d, 0x4f, 0x7a, 0xe8, 0x49, 0x97, 0x66, 0xc7, 0x26,
+	0x47, 0xbc, 0xab, 0x98, 0xff, 0x0a, 0x61, 0xe0, 0x4b, 0xc9, 0x14, 0x7a, 0x5a, 0xd2, 0x20, 0x0e,
+	0x92, 0xd1, 0xaa, 0xff, 0xe7, 0xef, 0xef, 0xeb, 0x80, 0xf5, 0xb4, 0x24, 0xd7, 0x10, 0xb5, 0xca,
+	0x48, 0xac, 0x69, 0xef, 0x34, 0xf2, 0x26, 0x79, 0x0b, 0x83, 0x56, 0xd5, 0x8d, 0x46, 0x43, 0xc3,
+	0xd3, 0xfc, 0xe0, 0x92, 0x05, 0x44, 0xfe, 0x69, 0x13, 0xf7, 0xb4, 0x69, 0xda, 0x0d, 0x9f, 0x9e,
+	0x0d, 0xc3, 0x3c, 0x44, 0x18, 0xbc, 0x3a, 0x19, 0x4a, 0xaa, 0x46, 0xd4, 0xba, 0xda, 0xab, 0xa7,
+	0x26, 0x3b, 0x5c, 0x3a, 0x3d, 0x1e, 0xbd, 0x3b, 0x9e, 0x24, 0x1f, 0x81, 0x14, 0x98, 0x6b, 0xe1,
+	0x0a, 0x5b, 0x2d, 0xd4, 0x5a, 0xcb, 0x86, 0x5e, 0xc4, 0x61, 0x32, 0x62, 0x13, 0x9f, 0xdc, 0xb9,
+	0xe0, 0x9b, 0x6c, 0xc8, 0x02, 0x48, 0xc1, 0x1b, 0xbb, 0x16, 0x58, 0x96, 0x5b, 0xa3, 0x05, 0x77,
+	0xb7, 0xf7, 0xe3, 0x20, 0x09, 0xd9, 0xd5, 0x3e, 0xb9, 0x3d, 0x0d, 0xc8, 0x07, 0x78, 0x26, 0xb6,
+	0x75, 0xad, 0x8c, 0x65, 0xaa, 0x2a, 0xb4, 0xe0, 0x34, 0x8a, 0x83, 0xa4, 0xcf, 0x1e, 0xb9, 0xe4,
+	0x1d, 0x8c, 0x2d, 0x5a, 0x5e, 0x78, 0xdd, 0xd0, 0x81, 0xc3, 0xce, 0x4d, 0x32, 0x83, 0xa1, 0x32,
+	0xb2, 0x42, 0x6d, 0x2c, 0x1d, 0xee, 0xf7, 0xc9, 0xfe, 0x6b, 0x42, 0xe0, 0xc2, 0xee, 0x2a, 0x45,
+	0x47, 0xce, 0x77, 0xff, 0xe7, 0x9f, 0x60, 0xe8, 0xf7, 0xd8, 0x90, 0xf7, 0xd0, 0xd7, 0x56, 0x95,
+	0x0d, 0x0d, 0xe2, 0x30, 0xb9, 0xbc, 0x79, 0xfe, 0x68, 0xd1, 0xac, 0x4b, 0x57, 0xdf, 0xe1, 0x05,
+	0xd6, 0x79, 0x8a, 0x95, 0x32, 0x02, 0x6b, 0xe9, 0xa9, 0xd5, 0xf8, 0x87, 0xfb, 0xf5, 0xf0, 0xcf,
+	0x34, 0xd7, 0xf6, 0x61, 0xbb, 0x49, 0x05, 0x96, 0xd9, 0x01, 0xcd, 0x3a, 0x74, 0xe1, 0x3f, 0xdb,
+	0xf6, 0x4b, 0x96, 0xa3, 0xf7, 0x36, 0x91, 0x33, 0x3f, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xe1,
+	0x88, 0x17, 0x78, 0x00, 0x03, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/device.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/device.pb.go
new file mode 100644
index 0000000..7da41c9
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/device.pb.go
@@ -0,0 +1,2344 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/device.proto
+
+package voltha
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	any "github.com/golang/protobuf/ptypes/any"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	openflow_13 "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	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 PmConfig_PmType int32
+
+const (
+	PmConfig_COUNTER PmConfig_PmType = 0
+	PmConfig_GAUGE   PmConfig_PmType = 1
+	PmConfig_STATE   PmConfig_PmType = 2
+	PmConfig_CONTEXT PmConfig_PmType = 3
+)
+
+var PmConfig_PmType_name = map[int32]string{
+	0: "COUNTER",
+	1: "GAUGE",
+	2: "STATE",
+	3: "CONTEXT",
+}
+
+var PmConfig_PmType_value = map[string]int32{
+	"COUNTER": 0,
+	"GAUGE":   1,
+	"STATE":   2,
+	"CONTEXT": 3,
+}
+
+func (x PmConfig_PmType) String() string {
+	return proto.EnumName(PmConfig_PmType_name, int32(x))
+}
+
+func (PmConfig_PmType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{2, 0}
+}
+
+type ImageDownload_ImageDownloadState int32
+
+const (
+	ImageDownload_DOWNLOAD_UNKNOWN     ImageDownload_ImageDownloadState = 0
+	ImageDownload_DOWNLOAD_SUCCEEDED   ImageDownload_ImageDownloadState = 1
+	ImageDownload_DOWNLOAD_REQUESTED   ImageDownload_ImageDownloadState = 2
+	ImageDownload_DOWNLOAD_STARTED     ImageDownload_ImageDownloadState = 3
+	ImageDownload_DOWNLOAD_FAILED      ImageDownload_ImageDownloadState = 4
+	ImageDownload_DOWNLOAD_UNSUPPORTED ImageDownload_ImageDownloadState = 5
+	ImageDownload_DOWNLOAD_CANCELLED   ImageDownload_ImageDownloadState = 6
+)
+
+var ImageDownload_ImageDownloadState_name = map[int32]string{
+	0: "DOWNLOAD_UNKNOWN",
+	1: "DOWNLOAD_SUCCEEDED",
+	2: "DOWNLOAD_REQUESTED",
+	3: "DOWNLOAD_STARTED",
+	4: "DOWNLOAD_FAILED",
+	5: "DOWNLOAD_UNSUPPORTED",
+	6: "DOWNLOAD_CANCELLED",
+}
+
+var ImageDownload_ImageDownloadState_value = map[string]int32{
+	"DOWNLOAD_UNKNOWN":     0,
+	"DOWNLOAD_SUCCEEDED":   1,
+	"DOWNLOAD_REQUESTED":   2,
+	"DOWNLOAD_STARTED":     3,
+	"DOWNLOAD_FAILED":      4,
+	"DOWNLOAD_UNSUPPORTED": 5,
+	"DOWNLOAD_CANCELLED":   6,
+}
+
+func (x ImageDownload_ImageDownloadState) String() string {
+	return proto.EnumName(ImageDownload_ImageDownloadState_name, int32(x))
+}
+
+func (ImageDownload_ImageDownloadState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{6, 0}
+}
+
+type ImageDownload_ImageDownloadFailureReason int32
+
+const (
+	ImageDownload_NO_ERROR           ImageDownload_ImageDownloadFailureReason = 0
+	ImageDownload_INVALID_URL        ImageDownload_ImageDownloadFailureReason = 1
+	ImageDownload_DEVICE_BUSY        ImageDownload_ImageDownloadFailureReason = 2
+	ImageDownload_INSUFFICIENT_SPACE ImageDownload_ImageDownloadFailureReason = 3
+	ImageDownload_UNKNOWN_ERROR      ImageDownload_ImageDownloadFailureReason = 4
+	ImageDownload_CANCELLED          ImageDownload_ImageDownloadFailureReason = 5
+)
+
+var ImageDownload_ImageDownloadFailureReason_name = map[int32]string{
+	0: "NO_ERROR",
+	1: "INVALID_URL",
+	2: "DEVICE_BUSY",
+	3: "INSUFFICIENT_SPACE",
+	4: "UNKNOWN_ERROR",
+	5: "CANCELLED",
+}
+
+var ImageDownload_ImageDownloadFailureReason_value = map[string]int32{
+	"NO_ERROR":           0,
+	"INVALID_URL":        1,
+	"DEVICE_BUSY":        2,
+	"INSUFFICIENT_SPACE": 3,
+	"UNKNOWN_ERROR":      4,
+	"CANCELLED":          5,
+}
+
+func (x ImageDownload_ImageDownloadFailureReason) String() string {
+	return proto.EnumName(ImageDownload_ImageDownloadFailureReason_name, int32(x))
+}
+
+func (ImageDownload_ImageDownloadFailureReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{6, 1}
+}
+
+type ImageDownload_ImageActivateState int32
+
+const (
+	ImageDownload_IMAGE_UNKNOWN    ImageDownload_ImageActivateState = 0
+	ImageDownload_IMAGE_INACTIVE   ImageDownload_ImageActivateState = 1
+	ImageDownload_IMAGE_ACTIVATING ImageDownload_ImageActivateState = 2
+	ImageDownload_IMAGE_ACTIVE     ImageDownload_ImageActivateState = 3
+	ImageDownload_IMAGE_REVERTING  ImageDownload_ImageActivateState = 4
+	ImageDownload_IMAGE_REVERTED   ImageDownload_ImageActivateState = 5
+)
+
+var ImageDownload_ImageActivateState_name = map[int32]string{
+	0: "IMAGE_UNKNOWN",
+	1: "IMAGE_INACTIVE",
+	2: "IMAGE_ACTIVATING",
+	3: "IMAGE_ACTIVE",
+	4: "IMAGE_REVERTING",
+	5: "IMAGE_REVERTED",
+}
+
+var ImageDownload_ImageActivateState_value = map[string]int32{
+	"IMAGE_UNKNOWN":    0,
+	"IMAGE_INACTIVE":   1,
+	"IMAGE_ACTIVATING": 2,
+	"IMAGE_ACTIVE":     3,
+	"IMAGE_REVERTING":  4,
+	"IMAGE_REVERTED":   5,
+}
+
+func (x ImageDownload_ImageActivateState) String() string {
+	return proto.EnumName(ImageDownload_ImageActivateState_name, int32(x))
+}
+
+func (ImageDownload_ImageActivateState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{6, 2}
+}
+
+type ImageState_ImageDownloadState int32
+
+const (
+	ImageState_DOWNLOAD_UNKNOWN     ImageState_ImageDownloadState = 0
+	ImageState_DOWNLOAD_SUCCEEDED   ImageState_ImageDownloadState = 1
+	ImageState_DOWNLOAD_REQUESTED   ImageState_ImageDownloadState = 2
+	ImageState_DOWNLOAD_STARTED     ImageState_ImageDownloadState = 3
+	ImageState_DOWNLOAD_FAILED      ImageState_ImageDownloadState = 4
+	ImageState_DOWNLOAD_UNSUPPORTED ImageState_ImageDownloadState = 5
+	ImageState_DOWNLOAD_CANCELLING  ImageState_ImageDownloadState = 6
+	ImageState_DOWNLOAD_CANCELLED   ImageState_ImageDownloadState = 7
+)
+
+var ImageState_ImageDownloadState_name = map[int32]string{
+	0: "DOWNLOAD_UNKNOWN",
+	1: "DOWNLOAD_SUCCEEDED",
+	2: "DOWNLOAD_REQUESTED",
+	3: "DOWNLOAD_STARTED",
+	4: "DOWNLOAD_FAILED",
+	5: "DOWNLOAD_UNSUPPORTED",
+	6: "DOWNLOAD_CANCELLING",
+	7: "DOWNLOAD_CANCELLED",
+}
+
+var ImageState_ImageDownloadState_value = map[string]int32{
+	"DOWNLOAD_UNKNOWN":     0,
+	"DOWNLOAD_SUCCEEDED":   1,
+	"DOWNLOAD_REQUESTED":   2,
+	"DOWNLOAD_STARTED":     3,
+	"DOWNLOAD_FAILED":      4,
+	"DOWNLOAD_UNSUPPORTED": 5,
+	"DOWNLOAD_CANCELLING":  6,
+	"DOWNLOAD_CANCELLED":   7,
+}
+
+func (x ImageState_ImageDownloadState) String() string {
+	return proto.EnumName(ImageState_ImageDownloadState_name, int32(x))
+}
+
+func (ImageState_ImageDownloadState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{12, 0}
+}
+
+type ImageState_ImageFailureReason int32
+
+const (
+	ImageState_NO_ERROR               ImageState_ImageFailureReason = 0
+	ImageState_INVALID_URL            ImageState_ImageFailureReason = 1
+	ImageState_DEVICE_BUSY            ImageState_ImageFailureReason = 2
+	ImageState_INSUFFICIENT_SPACE     ImageState_ImageFailureReason = 3
+	ImageState_UNKNOWN_ERROR          ImageState_ImageFailureReason = 4
+	ImageState_CANCELLED_ON_REQUEST   ImageState_ImageFailureReason = 5
+	ImageState_CANCELLED_ON_ONU_STATE ImageState_ImageFailureReason = 6
+	ImageState_VENDOR_DEVICE_MISMATCH ImageState_ImageFailureReason = 7
+	ImageState_OMCI_TRANSFER_ERROR    ImageState_ImageFailureReason = 8
+	ImageState_IMAGE_REFUSED_BY_ONU   ImageState_ImageFailureReason = 9
+)
+
+var ImageState_ImageFailureReason_name = map[int32]string{
+	0: "NO_ERROR",
+	1: "INVALID_URL",
+	2: "DEVICE_BUSY",
+	3: "INSUFFICIENT_SPACE",
+	4: "UNKNOWN_ERROR",
+	5: "CANCELLED_ON_REQUEST",
+	6: "CANCELLED_ON_ONU_STATE",
+	7: "VENDOR_DEVICE_MISMATCH",
+	8: "OMCI_TRANSFER_ERROR",
+	9: "IMAGE_REFUSED_BY_ONU",
+}
+
+var ImageState_ImageFailureReason_value = map[string]int32{
+	"NO_ERROR":               0,
+	"INVALID_URL":            1,
+	"DEVICE_BUSY":            2,
+	"INSUFFICIENT_SPACE":     3,
+	"UNKNOWN_ERROR":          4,
+	"CANCELLED_ON_REQUEST":   5,
+	"CANCELLED_ON_ONU_STATE": 6,
+	"VENDOR_DEVICE_MISMATCH": 7,
+	"OMCI_TRANSFER_ERROR":    8,
+	"IMAGE_REFUSED_BY_ONU":   9,
+}
+
+func (x ImageState_ImageFailureReason) String() string {
+	return proto.EnumName(ImageState_ImageFailureReason_name, int32(x))
+}
+
+func (ImageState_ImageFailureReason) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{12, 1}
+}
+
+type ImageState_ImageActivationState int32
+
+const (
+	ImageState_IMAGE_UNKNOWN             ImageState_ImageActivationState = 0
+	ImageState_IMAGE_INACTIVE            ImageState_ImageActivationState = 1
+	ImageState_IMAGE_ACTIVATING          ImageState_ImageActivationState = 2
+	ImageState_IMAGE_ACTIVE              ImageState_ImageActivationState = 3
+	ImageState_IMAGE_COMMITTING          ImageState_ImageActivationState = 4
+	ImageState_IMAGE_COMMITTED           ImageState_ImageActivationState = 5
+	ImageState_IMAGE_ACTIVATION_ABORTING ImageState_ImageActivationState = 6
+	ImageState_IMAGE_ACTIVATION_ABORTED  ImageState_ImageActivationState = 7
+	ImageState_IMAGE_COMMIT_ABORTING     ImageState_ImageActivationState = 8
+	ImageState_IMAGE_COMMIT_ABORTED      ImageState_ImageActivationState = 9
+	ImageState_IMAGE_DOWNLOADING         ImageState_ImageActivationState = 10
+)
+
+var ImageState_ImageActivationState_name = map[int32]string{
+	0:  "IMAGE_UNKNOWN",
+	1:  "IMAGE_INACTIVE",
+	2:  "IMAGE_ACTIVATING",
+	3:  "IMAGE_ACTIVE",
+	4:  "IMAGE_COMMITTING",
+	5:  "IMAGE_COMMITTED",
+	6:  "IMAGE_ACTIVATION_ABORTING",
+	7:  "IMAGE_ACTIVATION_ABORTED",
+	8:  "IMAGE_COMMIT_ABORTING",
+	9:  "IMAGE_COMMIT_ABORTED",
+	10: "IMAGE_DOWNLOADING",
+}
+
+var ImageState_ImageActivationState_value = map[string]int32{
+	"IMAGE_UNKNOWN":             0,
+	"IMAGE_INACTIVE":            1,
+	"IMAGE_ACTIVATING":          2,
+	"IMAGE_ACTIVE":              3,
+	"IMAGE_COMMITTING":          4,
+	"IMAGE_COMMITTED":           5,
+	"IMAGE_ACTIVATION_ABORTING": 6,
+	"IMAGE_ACTIVATION_ABORTED":  7,
+	"IMAGE_COMMIT_ABORTING":     8,
+	"IMAGE_COMMIT_ABORTED":      9,
+	"IMAGE_DOWNLOADING":         10,
+}
+
+func (x ImageState_ImageActivationState) String() string {
+	return proto.EnumName(ImageState_ImageActivationState_name, int32(x))
+}
+
+func (ImageState_ImageActivationState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{12, 2}
+}
+
+type Port_PortType int32
+
+const (
+	Port_UNKNOWN      Port_PortType = 0
+	Port_ETHERNET_NNI Port_PortType = 1
+	Port_ETHERNET_UNI Port_PortType = 2
+	Port_PON_OLT      Port_PortType = 3
+	Port_PON_ONU      Port_PortType = 4
+	Port_VENET_OLT    Port_PortType = 5
+	Port_VENET_ONU    Port_PortType = 6
+)
+
+var Port_PortType_name = map[int32]string{
+	0: "UNKNOWN",
+	1: "ETHERNET_NNI",
+	2: "ETHERNET_UNI",
+	3: "PON_OLT",
+	4: "PON_ONU",
+	5: "VENET_OLT",
+	6: "VENET_ONU",
+}
+
+var Port_PortType_value = map[string]int32{
+	"UNKNOWN":      0,
+	"ETHERNET_NNI": 1,
+	"ETHERNET_UNI": 2,
+	"PON_OLT":      3,
+	"PON_ONU":      4,
+	"VENET_OLT":    5,
+	"VENET_ONU":    6,
+}
+
+func (x Port_PortType) String() string {
+	return proto.EnumName(Port_PortType_name, int32(x))
+}
+
+func (Port_PortType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{13, 0}
+}
+
+type SimulateAlarmRequest_OperationType int32
+
+const (
+	SimulateAlarmRequest_RAISE SimulateAlarmRequest_OperationType = 0
+	SimulateAlarmRequest_CLEAR SimulateAlarmRequest_OperationType = 1
+)
+
+var SimulateAlarmRequest_OperationType_name = map[int32]string{
+	0: "RAISE",
+	1: "CLEAR",
+}
+
+var SimulateAlarmRequest_OperationType_value = map[string]int32{
+	"RAISE": 0,
+	"CLEAR": 1,
+}
+
+func (x SimulateAlarmRequest_OperationType) String() string {
+	return proto.EnumName(SimulateAlarmRequest_OperationType_name, int32(x))
+}
+
+func (SimulateAlarmRequest_OperationType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{17, 0}
+}
+
+// A Device Type
+type DeviceType struct {
+	// Unique name for the device type
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Unique vendor id for the device type applicable to ONU
+	// 4 bytes of vendor id from ONU serial number
+	VendorId  string   `protobuf:"bytes,5,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
+	VendorIds []string `protobuf:"bytes,6,rep,name=vendor_ids,json=vendorIds,proto3" json:"vendor_ids,omitempty"`
+	// Name of the adapter that handles device type
+	// Deprecated and replaced by adapterType
+	Adapter string `protobuf:"bytes,2,opt,name=adapter,proto3" json:"adapter,omitempty"`
+	// Capabilities
+	AcceptsBulkFlowUpdate           bool `protobuf:"varint,3,opt,name=accepts_bulk_flow_update,json=acceptsBulkFlowUpdate,proto3" json:"accepts_bulk_flow_update,omitempty"`
+	AcceptsAddRemoveFlowUpdates     bool `protobuf:"varint,4,opt,name=accepts_add_remove_flow_updates,json=acceptsAddRemoveFlowUpdates,proto3" json:"accepts_add_remove_flow_updates,omitempty"`
+	AcceptsDirectLogicalFlowsUpdate bool `protobuf:"varint,7,opt,name=accepts_direct_logical_flows_update,json=acceptsDirectLogicalFlowsUpdate,proto3" json:"accepts_direct_logical_flows_update,omitempty"`
+	// Type of adapter that can handle this device type
+	AdapterType          string   `protobuf:"bytes,8,opt,name=adapter_type,json=adapterType,proto3" json:"adapter_type,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeviceType) Reset()         { *m = DeviceType{} }
+func (m *DeviceType) String() string { return proto.CompactTextString(m) }
+func (*DeviceType) ProtoMessage()    {}
+func (*DeviceType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{0}
+}
+
+func (m *DeviceType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceType.Unmarshal(m, b)
+}
+func (m *DeviceType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceType.Marshal(b, m, deterministic)
+}
+func (m *DeviceType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceType.Merge(m, src)
+}
+func (m *DeviceType) XXX_Size() int {
+	return xxx_messageInfo_DeviceType.Size(m)
+}
+func (m *DeviceType) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceType proto.InternalMessageInfo
+
+func (m *DeviceType) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *DeviceType) GetVendorId() string {
+	if m != nil {
+		return m.VendorId
+	}
+	return ""
+}
+
+func (m *DeviceType) GetVendorIds() []string {
+	if m != nil {
+		return m.VendorIds
+	}
+	return nil
+}
+
+func (m *DeviceType) GetAdapter() string {
+	if m != nil {
+		return m.Adapter
+	}
+	return ""
+}
+
+func (m *DeviceType) GetAcceptsBulkFlowUpdate() bool {
+	if m != nil {
+		return m.AcceptsBulkFlowUpdate
+	}
+	return false
+}
+
+func (m *DeviceType) GetAcceptsAddRemoveFlowUpdates() bool {
+	if m != nil {
+		return m.AcceptsAddRemoveFlowUpdates
+	}
+	return false
+}
+
+func (m *DeviceType) GetAcceptsDirectLogicalFlowsUpdate() bool {
+	if m != nil {
+		return m.AcceptsDirectLogicalFlowsUpdate
+	}
+	return false
+}
+
+func (m *DeviceType) GetAdapterType() string {
+	if m != nil {
+		return m.AdapterType
+	}
+	return ""
+}
+
+// A plurality of device types
+type DeviceTypes struct {
+	Items                []*DeviceType `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *DeviceTypes) Reset()         { *m = DeviceTypes{} }
+func (m *DeviceTypes) String() string { return proto.CompactTextString(m) }
+func (*DeviceTypes) ProtoMessage()    {}
+func (*DeviceTypes) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{1}
+}
+
+func (m *DeviceTypes) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceTypes.Unmarshal(m, b)
+}
+func (m *DeviceTypes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceTypes.Marshal(b, m, deterministic)
+}
+func (m *DeviceTypes) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceTypes.Merge(m, src)
+}
+func (m *DeviceTypes) XXX_Size() int {
+	return xxx_messageInfo_DeviceTypes.Size(m)
+}
+func (m *DeviceTypes) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceTypes.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceTypes proto.InternalMessageInfo
+
+func (m *DeviceTypes) GetItems() []*DeviceType {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type PmConfig struct {
+	Name                 string          `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Type                 PmConfig_PmType `protobuf:"varint,2,opt,name=type,proto3,enum=voltha.PmConfig_PmType" json:"type,omitempty"`
+	Enabled              bool            `protobuf:"varint,3,opt,name=enabled,proto3" json:"enabled,omitempty"`
+	SampleFreq           uint32          `protobuf:"varint,4,opt,name=sample_freq,json=sampleFreq,proto3" json:"sample_freq,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *PmConfig) Reset()         { *m = PmConfig{} }
+func (m *PmConfig) String() string { return proto.CompactTextString(m) }
+func (*PmConfig) ProtoMessage()    {}
+func (*PmConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{2}
+}
+
+func (m *PmConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PmConfig.Unmarshal(m, b)
+}
+func (m *PmConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PmConfig.Marshal(b, m, deterministic)
+}
+func (m *PmConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PmConfig.Merge(m, src)
+}
+func (m *PmConfig) XXX_Size() int {
+	return xxx_messageInfo_PmConfig.Size(m)
+}
+func (m *PmConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_PmConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PmConfig proto.InternalMessageInfo
+
+func (m *PmConfig) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *PmConfig) GetType() PmConfig_PmType {
+	if m != nil {
+		return m.Type
+	}
+	return PmConfig_COUNTER
+}
+
+func (m *PmConfig) GetEnabled() bool {
+	if m != nil {
+		return m.Enabled
+	}
+	return false
+}
+
+func (m *PmConfig) GetSampleFreq() uint32 {
+	if m != nil {
+		return m.SampleFreq
+	}
+	return 0
+}
+
+type PmGroupConfig struct {
+	GroupName            string      `protobuf:"bytes,1,opt,name=group_name,json=groupName,proto3" json:"group_name,omitempty"`
+	GroupFreq            uint32      `protobuf:"varint,2,opt,name=group_freq,json=groupFreq,proto3" json:"group_freq,omitempty"`
+	Enabled              bool        `protobuf:"varint,3,opt,name=enabled,proto3" json:"enabled,omitempty"`
+	Metrics              []*PmConfig `protobuf:"bytes,4,rep,name=metrics,proto3" json:"metrics,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *PmGroupConfig) Reset()         { *m = PmGroupConfig{} }
+func (m *PmGroupConfig) String() string { return proto.CompactTextString(m) }
+func (*PmGroupConfig) ProtoMessage()    {}
+func (*PmGroupConfig) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{3}
+}
+
+func (m *PmGroupConfig) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PmGroupConfig.Unmarshal(m, b)
+}
+func (m *PmGroupConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PmGroupConfig.Marshal(b, m, deterministic)
+}
+func (m *PmGroupConfig) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PmGroupConfig.Merge(m, src)
+}
+func (m *PmGroupConfig) XXX_Size() int {
+	return xxx_messageInfo_PmGroupConfig.Size(m)
+}
+func (m *PmGroupConfig) XXX_DiscardUnknown() {
+	xxx_messageInfo_PmGroupConfig.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PmGroupConfig proto.InternalMessageInfo
+
+func (m *PmGroupConfig) GetGroupName() string {
+	if m != nil {
+		return m.GroupName
+	}
+	return ""
+}
+
+func (m *PmGroupConfig) GetGroupFreq() uint32 {
+	if m != nil {
+		return m.GroupFreq
+	}
+	return 0
+}
+
+func (m *PmGroupConfig) GetEnabled() bool {
+	if m != nil {
+		return m.Enabled
+	}
+	return false
+}
+
+func (m *PmGroupConfig) GetMetrics() []*PmConfig {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+type PmConfigs struct {
+	Id          string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	DefaultFreq uint32 `protobuf:"varint,2,opt,name=default_freq,json=defaultFreq,proto3" json:"default_freq,omitempty"`
+	// Forces group names and group semantics
+	Grouped bool `protobuf:"varint,3,opt,name=grouped,proto3" json:"grouped,omitempty"`
+	// Allows Pm to set an individual sample frequency
+	FreqOverride         bool             `protobuf:"varint,4,opt,name=freq_override,json=freqOverride,proto3" json:"freq_override,omitempty"`
+	Groups               []*PmGroupConfig `protobuf:"bytes,5,rep,name=groups,proto3" json:"groups,omitempty"`
+	Metrics              []*PmConfig      `protobuf:"bytes,6,rep,name=metrics,proto3" json:"metrics,omitempty"`
+	MaxSkew              uint32           `protobuf:"varint,7,opt,name=max_skew,json=maxSkew,proto3" json:"max_skew,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *PmConfigs) Reset()         { *m = PmConfigs{} }
+func (m *PmConfigs) String() string { return proto.CompactTextString(m) }
+func (*PmConfigs) ProtoMessage()    {}
+func (*PmConfigs) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{4}
+}
+
+func (m *PmConfigs) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_PmConfigs.Unmarshal(m, b)
+}
+func (m *PmConfigs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_PmConfigs.Marshal(b, m, deterministic)
+}
+func (m *PmConfigs) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PmConfigs.Merge(m, src)
+}
+func (m *PmConfigs) XXX_Size() int {
+	return xxx_messageInfo_PmConfigs.Size(m)
+}
+func (m *PmConfigs) XXX_DiscardUnknown() {
+	xxx_messageInfo_PmConfigs.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_PmConfigs proto.InternalMessageInfo
+
+func (m *PmConfigs) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *PmConfigs) GetDefaultFreq() uint32 {
+	if m != nil {
+		return m.DefaultFreq
+	}
+	return 0
+}
+
+func (m *PmConfigs) GetGrouped() bool {
+	if m != nil {
+		return m.Grouped
+	}
+	return false
+}
+
+func (m *PmConfigs) GetFreqOverride() bool {
+	if m != nil {
+		return m.FreqOverride
+	}
+	return false
+}
+
+func (m *PmConfigs) GetGroups() []*PmGroupConfig {
+	if m != nil {
+		return m.Groups
+	}
+	return nil
+}
+
+func (m *PmConfigs) GetMetrics() []*PmConfig {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+func (m *PmConfigs) GetMaxSkew() uint32 {
+	if m != nil {
+		return m.MaxSkew
+	}
+	return 0
+}
+
+//Object representing an image
+type Image struct {
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// Version, this is the sole identifier of the image. it's the vendor specified OMCI version
+	// must be known at the time of initiating a download, activate, commit image on an onu.
+	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
+	// hash of the image to be verified against
+	// Deprecated in voltha 2.8, will be removed
+	Hash uint32 `protobuf:"varint,3,opt,name=hash,proto3" json:"hash,omitempty"`
+	// Deprecated in voltha 2.8, will be removed
+	InstallDatetime string `protobuf:"bytes,4,opt,name=install_datetime,json=installDatetime,proto3" json:"install_datetime,omitempty"`
+	// The active software image is one that is currently loaded and executing
+	// in the ONU or circuit pack. Under normal operation, one software image
+	// is always active while the other is inactive. Under no circumstances are
+	// both software images allowed to be active at the same time
+	// Deprecated in voltha 2.8, will be removed
+	IsActive bool `protobuf:"varint,5,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"`
+	// The committed software image is loaded and executed upon reboot of the
+	// ONU and/or circuit pack. During normal operation, one software image is
+	// always committed, while the other is uncommitted.
+	// Deprecated in voltha 2.8, will be removed
+	IsCommitted bool `protobuf:"varint,6,opt,name=is_committed,json=isCommitted,proto3" json:"is_committed,omitempty"`
+	// A software image is valid if it has been verified to be an executable
+	// code image. The verification mechanism is not subject to standardization;
+	// however, it should include at least a data integrity (e.g., CRC) check of
+	// the entire code image.
+	// Deprecated in voltha 2.8, will be removed
+	IsValid bool `protobuf:"varint,7,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"`
+	// URL where the image is available
+	// URL MUST be fully qualified,
+	// including filename, username and password
+	Url string `protobuf:"bytes,8,opt,name=url,proto3" json:"url,omitempty"`
+	// Represents the vendor/device mfr
+	// Needs to match the vendor of the device the image will be installed on
+	// optional, if not matched no check will be performed
+	Vendor string `protobuf:"bytes,9,opt,name=vendor,proto3" json:"vendor,omitempty"`
+	// Represents the ONU Image CRC value.
+	// Default to value 0 if not specified.
+	// If different then 0 it's used to verify the image retrieved from outside before sending it to the ONU.
+	// Calculation of this needs to be done according to ITU-T I.363.5 as per OMCI spec (section A.2.27)
+	Crc32                uint32   `protobuf:"varint,10,opt,name=crc32,proto3" json:"crc32,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Image) Reset()         { *m = Image{} }
+func (m *Image) String() string { return proto.CompactTextString(m) }
+func (*Image) ProtoMessage()    {}
+func (*Image) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{5}
+}
+
+func (m *Image) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Image.Unmarshal(m, b)
+}
+func (m *Image) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Image.Marshal(b, m, deterministic)
+}
+func (m *Image) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Image.Merge(m, src)
+}
+func (m *Image) XXX_Size() int {
+	return xxx_messageInfo_Image.Size(m)
+}
+func (m *Image) XXX_DiscardUnknown() {
+	xxx_messageInfo_Image.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Image proto.InternalMessageInfo
+
+func (m *Image) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *Image) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *Image) GetHash() uint32 {
+	if m != nil {
+		return m.Hash
+	}
+	return 0
+}
+
+func (m *Image) GetInstallDatetime() string {
+	if m != nil {
+		return m.InstallDatetime
+	}
+	return ""
+}
+
+func (m *Image) GetIsActive() bool {
+	if m != nil {
+		return m.IsActive
+	}
+	return false
+}
+
+func (m *Image) GetIsCommitted() bool {
+	if m != nil {
+		return m.IsCommitted
+	}
+	return false
+}
+
+func (m *Image) GetIsValid() bool {
+	if m != nil {
+		return m.IsValid
+	}
+	return false
+}
+
+func (m *Image) GetUrl() string {
+	if m != nil {
+		return m.Url
+	}
+	return ""
+}
+
+func (m *Image) GetVendor() string {
+	if m != nil {
+		return m.Vendor
+	}
+	return ""
+}
+
+func (m *Image) GetCrc32() uint32 {
+	if m != nil {
+		return m.Crc32
+	}
+	return 0
+}
+
+// Older version of the API please see DeviceImageDownloadRequest
+// Deprecated in voltha 2.8, will be removed
+type ImageDownload struct {
+	// Device Identifier
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Image unique identifier
+	Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+	// URL where the image is available
+	// should include username password
+	Url string `protobuf:"bytes,3,opt,name=url,proto3" json:"url,omitempty"`
+	// CRC of the image to be verified aginst
+	Crc uint32 `protobuf:"varint,4,opt,name=crc,proto3" json:"crc,omitempty"`
+	// Download state
+	DownloadState ImageDownload_ImageDownloadState `protobuf:"varint,5,opt,name=download_state,json=downloadState,proto3,enum=voltha.ImageDownload_ImageDownloadState" json:"download_state,omitempty"`
+	// Downloaded version
+	ImageVersion string `protobuf:"bytes,6,opt,name=image_version,json=imageVersion,proto3" json:"image_version,omitempty"`
+	// Bytes downloaded
+	DownloadedBytes uint32 `protobuf:"varint,7,opt,name=downloaded_bytes,json=downloadedBytes,proto3" json:"downloaded_bytes,omitempty"`
+	// Download failure reason
+	Reason ImageDownload_ImageDownloadFailureReason `protobuf:"varint,8,opt,name=reason,proto3,enum=voltha.ImageDownload_ImageDownloadFailureReason" json:"reason,omitempty"`
+	// Additional info
+	AdditionalInfo string `protobuf:"bytes,9,opt,name=additional_info,json=additionalInfo,proto3" json:"additional_info,omitempty"`
+	// Save current configuration
+	SaveConfig bool `protobuf:"varint,10,opt,name=save_config,json=saveConfig,proto3" json:"save_config,omitempty"`
+	// Image local location
+	LocalDir string `protobuf:"bytes,11,opt,name=local_dir,json=localDir,proto3" json:"local_dir,omitempty"`
+	// Image activation state
+	ImageState ImageDownload_ImageActivateState `protobuf:"varint,12,opt,name=image_state,json=imageState,proto3,enum=voltha.ImageDownload_ImageActivateState" json:"image_state,omitempty"`
+	// Image file size
+	FileSize             uint32   `protobuf:"varint,13,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ImageDownload) Reset()         { *m = ImageDownload{} }
+func (m *ImageDownload) String() string { return proto.CompactTextString(m) }
+func (*ImageDownload) ProtoMessage()    {}
+func (*ImageDownload) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{6}
+}
+
+func (m *ImageDownload) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ImageDownload.Unmarshal(m, b)
+}
+func (m *ImageDownload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ImageDownload.Marshal(b, m, deterministic)
+}
+func (m *ImageDownload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ImageDownload.Merge(m, src)
+}
+func (m *ImageDownload) XXX_Size() int {
+	return xxx_messageInfo_ImageDownload.Size(m)
+}
+func (m *ImageDownload) XXX_DiscardUnknown() {
+	xxx_messageInfo_ImageDownload.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ImageDownload proto.InternalMessageInfo
+
+func (m *ImageDownload) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetUrl() string {
+	if m != nil {
+		return m.Url
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetCrc() uint32 {
+	if m != nil {
+		return m.Crc
+	}
+	return 0
+}
+
+func (m *ImageDownload) GetDownloadState() ImageDownload_ImageDownloadState {
+	if m != nil {
+		return m.DownloadState
+	}
+	return ImageDownload_DOWNLOAD_UNKNOWN
+}
+
+func (m *ImageDownload) GetImageVersion() string {
+	if m != nil {
+		return m.ImageVersion
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetDownloadedBytes() uint32 {
+	if m != nil {
+		return m.DownloadedBytes
+	}
+	return 0
+}
+
+func (m *ImageDownload) GetReason() ImageDownload_ImageDownloadFailureReason {
+	if m != nil {
+		return m.Reason
+	}
+	return ImageDownload_NO_ERROR
+}
+
+func (m *ImageDownload) GetAdditionalInfo() string {
+	if m != nil {
+		return m.AdditionalInfo
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetSaveConfig() bool {
+	if m != nil {
+		return m.SaveConfig
+	}
+	return false
+}
+
+func (m *ImageDownload) GetLocalDir() string {
+	if m != nil {
+		return m.LocalDir
+	}
+	return ""
+}
+
+func (m *ImageDownload) GetImageState() ImageDownload_ImageActivateState {
+	if m != nil {
+		return m.ImageState
+	}
+	return ImageDownload_IMAGE_UNKNOWN
+}
+
+func (m *ImageDownload) GetFileSize() uint32 {
+	if m != nil {
+		return m.FileSize
+	}
+	return 0
+}
+
+// Deprecated in voltha 2.8, will be removed
+type ImageDownloads struct {
+	Items                []*ImageDownload `protobuf:"bytes,2,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *ImageDownloads) Reset()         { *m = ImageDownloads{} }
+func (m *ImageDownloads) String() string { return proto.CompactTextString(m) }
+func (*ImageDownloads) ProtoMessage()    {}
+func (*ImageDownloads) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{7}
+}
+
+func (m *ImageDownloads) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ImageDownloads.Unmarshal(m, b)
+}
+func (m *ImageDownloads) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ImageDownloads.Marshal(b, m, deterministic)
+}
+func (m *ImageDownloads) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ImageDownloads.Merge(m, src)
+}
+func (m *ImageDownloads) XXX_Size() int {
+	return xxx_messageInfo_ImageDownloads.Size(m)
+}
+func (m *ImageDownloads) XXX_DiscardUnknown() {
+	xxx_messageInfo_ImageDownloads.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ImageDownloads proto.InternalMessageInfo
+
+func (m *ImageDownloads) GetItems() []*ImageDownload {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type Images struct {
+	Image                []*Image `protobuf:"bytes,1,rep,name=image,proto3" json:"image,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Images) Reset()         { *m = Images{} }
+func (m *Images) String() string { return proto.CompactTextString(m) }
+func (*Images) ProtoMessage()    {}
+func (*Images) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{8}
+}
+
+func (m *Images) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Images.Unmarshal(m, b)
+}
+func (m *Images) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Images.Marshal(b, m, deterministic)
+}
+func (m *Images) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Images.Merge(m, src)
+}
+func (m *Images) XXX_Size() int {
+	return xxx_messageInfo_Images.Size(m)
+}
+func (m *Images) XXX_DiscardUnknown() {
+	xxx_messageInfo_Images.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Images proto.InternalMessageInfo
+
+func (m *Images) GetImage() []*Image {
+	if m != nil {
+		return m.Image
+	}
+	return nil
+}
+
+// OnuImage represents the OMCI information as per OMCI spec
+// the information will be populates exactly as extracted from the device.
+type OnuImage struct {
+	//image version
+	Version     string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
+	IsCommited  bool   `protobuf:"varint,2,opt,name=isCommited,proto3" json:"isCommited,omitempty"`
+	IsActive    bool   `protobuf:"varint,3,opt,name=isActive,proto3" json:"isActive,omitempty"`
+	IsValid     bool   `protobuf:"varint,4,opt,name=isValid,proto3" json:"isValid,omitempty"`
+	ProductCode string `protobuf:"bytes,5,opt,name=productCode,proto3" json:"productCode,omitempty"`
+	// Hash is computed by the ONU and is optional as per OMCI spec (paragraph 9.1.4)
+	// No assumption should be made on the existence of this attribute at any time.
+	Hash                 string   `protobuf:"bytes,6,opt,name=hash,proto3" json:"hash,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OnuImage) Reset()         { *m = OnuImage{} }
+func (m *OnuImage) String() string { return proto.CompactTextString(m) }
+func (*OnuImage) ProtoMessage()    {}
+func (*OnuImage) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{9}
+}
+
+func (m *OnuImage) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuImage.Unmarshal(m, b)
+}
+func (m *OnuImage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuImage.Marshal(b, m, deterministic)
+}
+func (m *OnuImage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuImage.Merge(m, src)
+}
+func (m *OnuImage) XXX_Size() int {
+	return xxx_messageInfo_OnuImage.Size(m)
+}
+func (m *OnuImage) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuImage.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuImage proto.InternalMessageInfo
+
+func (m *OnuImage) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *OnuImage) GetIsCommited() bool {
+	if m != nil {
+		return m.IsCommited
+	}
+	return false
+}
+
+func (m *OnuImage) GetIsActive() bool {
+	if m != nil {
+		return m.IsActive
+	}
+	return false
+}
+
+func (m *OnuImage) GetIsValid() bool {
+	if m != nil {
+		return m.IsValid
+	}
+	return false
+}
+
+func (m *OnuImage) GetProductCode() string {
+	if m != nil {
+		return m.ProductCode
+	}
+	return ""
+}
+
+func (m *OnuImage) GetHash() string {
+	if m != nil {
+		return m.Hash
+	}
+	return ""
+}
+
+type OnuImages struct {
+	Items                []*OnuImage `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *OnuImages) Reset()         { *m = OnuImages{} }
+func (m *OnuImages) String() string { return proto.CompactTextString(m) }
+func (*OnuImages) ProtoMessage()    {}
+func (*OnuImages) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{10}
+}
+
+func (m *OnuImages) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OnuImages.Unmarshal(m, b)
+}
+func (m *OnuImages) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OnuImages.Marshal(b, m, deterministic)
+}
+func (m *OnuImages) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnuImages.Merge(m, src)
+}
+func (m *OnuImages) XXX_Size() int {
+	return xxx_messageInfo_OnuImages.Size(m)
+}
+func (m *OnuImages) XXX_DiscardUnknown() {
+	xxx_messageInfo_OnuImages.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OnuImages proto.InternalMessageInfo
+
+func (m *OnuImages) GetItems() []*OnuImage {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type DeviceImageState struct {
+	DeviceId             string      `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	ImageState           *ImageState `protobuf:"bytes,2,opt,name=imageState,proto3" json:"imageState,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *DeviceImageState) Reset()         { *m = DeviceImageState{} }
+func (m *DeviceImageState) String() string { return proto.CompactTextString(m) }
+func (*DeviceImageState) ProtoMessage()    {}
+func (*DeviceImageState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{11}
+}
+
+func (m *DeviceImageState) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceImageState.Unmarshal(m, b)
+}
+func (m *DeviceImageState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceImageState.Marshal(b, m, deterministic)
+}
+func (m *DeviceImageState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceImageState.Merge(m, src)
+}
+func (m *DeviceImageState) XXX_Size() int {
+	return xxx_messageInfo_DeviceImageState.Size(m)
+}
+func (m *DeviceImageState) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceImageState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceImageState proto.InternalMessageInfo
+
+func (m *DeviceImageState) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *DeviceImageState) GetImageState() *ImageState {
+	if m != nil {
+		return m.ImageState
+	}
+	return nil
+}
+
+type ImageState struct {
+	// image version
+	Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
+	// Download state
+	DownloadState ImageState_ImageDownloadState `protobuf:"varint,2,opt,name=download_state,json=downloadState,proto3,enum=voltha.ImageState_ImageDownloadState" json:"download_state,omitempty"`
+	// Image Operation Failure reason (use for both Download and Activate)
+	Reason ImageState_ImageFailureReason `protobuf:"varint,3,opt,name=reason,proto3,enum=voltha.ImageState_ImageFailureReason" json:"reason,omitempty"`
+	// Image activation state
+	ImageState           ImageState_ImageActivationState `protobuf:"varint,4,opt,name=image_state,json=imageState,proto3,enum=voltha.ImageState_ImageActivationState" json:"image_state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
+	XXX_unrecognized     []byte                          `json:"-"`
+	XXX_sizecache        int32                           `json:"-"`
+}
+
+func (m *ImageState) Reset()         { *m = ImageState{} }
+func (m *ImageState) String() string { return proto.CompactTextString(m) }
+func (*ImageState) ProtoMessage()    {}
+func (*ImageState) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{12}
+}
+
+func (m *ImageState) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ImageState.Unmarshal(m, b)
+}
+func (m *ImageState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ImageState.Marshal(b, m, deterministic)
+}
+func (m *ImageState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ImageState.Merge(m, src)
+}
+func (m *ImageState) XXX_Size() int {
+	return xxx_messageInfo_ImageState.Size(m)
+}
+func (m *ImageState) XXX_DiscardUnknown() {
+	xxx_messageInfo_ImageState.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ImageState proto.InternalMessageInfo
+
+func (m *ImageState) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *ImageState) GetDownloadState() ImageState_ImageDownloadState {
+	if m != nil {
+		return m.DownloadState
+	}
+	return ImageState_DOWNLOAD_UNKNOWN
+}
+
+func (m *ImageState) GetReason() ImageState_ImageFailureReason {
+	if m != nil {
+		return m.Reason
+	}
+	return ImageState_NO_ERROR
+}
+
+func (m *ImageState) GetImageState() ImageState_ImageActivationState {
+	if m != nil {
+		return m.ImageState
+	}
+	return ImageState_IMAGE_UNKNOWN
+}
+
+type Port struct {
+	PortNo     uint32                  `protobuf:"varint,1,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	Label      string                  `protobuf:"bytes,2,opt,name=label,proto3" json:"label,omitempty"`
+	Type       Port_PortType           `protobuf:"varint,3,opt,name=type,proto3,enum=voltha.Port_PortType" json:"type,omitempty"`
+	AdminState common.AdminState_Types `protobuf:"varint,5,opt,name=admin_state,json=adminState,proto3,enum=common.AdminState_Types" json:"admin_state,omitempty"`
+	OperStatus common.OperStatus_Types `protobuf:"varint,6,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
+	DeviceId   string                  `protobuf:"bytes,7,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Peers      []*Port_PeerPort        `protobuf:"bytes,8,rep,name=peers,proto3" json:"peers,omitempty"`
+	RxPackets  uint64                  `protobuf:"fixed64,9,opt,name=rx_packets,json=rxPackets,proto3" json:"rx_packets,omitempty"`
+	RxBytes    uint64                  `protobuf:"fixed64,10,opt,name=rx_bytes,json=rxBytes,proto3" json:"rx_bytes,omitempty"`
+	RxErrors   uint64                  `protobuf:"fixed64,11,opt,name=rx_errors,json=rxErrors,proto3" json:"rx_errors,omitempty"`
+	TxPackets  uint64                  `protobuf:"fixed64,12,opt,name=tx_packets,json=txPackets,proto3" json:"tx_packets,omitempty"`
+	TxBytes    uint64                  `protobuf:"fixed64,13,opt,name=tx_bytes,json=txBytes,proto3" json:"tx_bytes,omitempty"`
+	TxErrors   uint64                  `protobuf:"fixed64,14,opt,name=tx_errors,json=txErrors,proto3" json:"tx_errors,omitempty"`
+	// ofp_port represents the characteristics of a port, e.g. hardware
+	// address and supported features.  This field is relevant only for
+	// UNI and NNI ports.   For PON ports, it can be left empty.
+	OfpPort              *openflow_13.OfpPort `protobuf:"bytes,15,opt,name=ofp_port,json=ofpPort,proto3" json:"ofp_port,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *Port) Reset()         { *m = Port{} }
+func (m *Port) String() string { return proto.CompactTextString(m) }
+func (*Port) ProtoMessage()    {}
+func (*Port) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{13}
+}
+
+func (m *Port) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Port.Unmarshal(m, b)
+}
+func (m *Port) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Port.Marshal(b, m, deterministic)
+}
+func (m *Port) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Port.Merge(m, src)
+}
+func (m *Port) XXX_Size() int {
+	return xxx_messageInfo_Port.Size(m)
+}
+func (m *Port) XXX_DiscardUnknown() {
+	xxx_messageInfo_Port.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Port proto.InternalMessageInfo
+
+func (m *Port) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+func (m *Port) GetLabel() string {
+	if m != nil {
+		return m.Label
+	}
+	return ""
+}
+
+func (m *Port) GetType() Port_PortType {
+	if m != nil {
+		return m.Type
+	}
+	return Port_UNKNOWN
+}
+
+func (m *Port) GetAdminState() common.AdminState_Types {
+	if m != nil {
+		return m.AdminState
+	}
+	return common.AdminState_UNKNOWN
+}
+
+func (m *Port) GetOperStatus() common.OperStatus_Types {
+	if m != nil {
+		return m.OperStatus
+	}
+	return common.OperStatus_UNKNOWN
+}
+
+func (m *Port) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *Port) GetPeers() []*Port_PeerPort {
+	if m != nil {
+		return m.Peers
+	}
+	return nil
+}
+
+func (m *Port) GetRxPackets() uint64 {
+	if m != nil {
+		return m.RxPackets
+	}
+	return 0
+}
+
+func (m *Port) GetRxBytes() uint64 {
+	if m != nil {
+		return m.RxBytes
+	}
+	return 0
+}
+
+func (m *Port) GetRxErrors() uint64 {
+	if m != nil {
+		return m.RxErrors
+	}
+	return 0
+}
+
+func (m *Port) GetTxPackets() uint64 {
+	if m != nil {
+		return m.TxPackets
+	}
+	return 0
+}
+
+func (m *Port) GetTxBytes() uint64 {
+	if m != nil {
+		return m.TxBytes
+	}
+	return 0
+}
+
+func (m *Port) GetTxErrors() uint64 {
+	if m != nil {
+		return m.TxErrors
+	}
+	return 0
+}
+
+func (m *Port) GetOfpPort() *openflow_13.OfpPort {
+	if m != nil {
+		return m.OfpPort
+	}
+	return nil
+}
+
+type Port_PeerPort struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	PortNo               uint32   `protobuf:"varint,2,opt,name=port_no,json=portNo,proto3" json:"port_no,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Port_PeerPort) Reset()         { *m = Port_PeerPort{} }
+func (m *Port_PeerPort) String() string { return proto.CompactTextString(m) }
+func (*Port_PeerPort) ProtoMessage()    {}
+func (*Port_PeerPort) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{13, 0}
+}
+
+func (m *Port_PeerPort) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Port_PeerPort.Unmarshal(m, b)
+}
+func (m *Port_PeerPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Port_PeerPort.Marshal(b, m, deterministic)
+}
+func (m *Port_PeerPort) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Port_PeerPort.Merge(m, src)
+}
+func (m *Port_PeerPort) XXX_Size() int {
+	return xxx_messageInfo_Port_PeerPort.Size(m)
+}
+func (m *Port_PeerPort) XXX_DiscardUnknown() {
+	xxx_messageInfo_Port_PeerPort.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Port_PeerPort proto.InternalMessageInfo
+
+func (m *Port_PeerPort) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *Port_PeerPort) GetPortNo() uint32 {
+	if m != nil {
+		return m.PortNo
+	}
+	return 0
+}
+
+type Ports struct {
+	Items                []*Port  `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Ports) Reset()         { *m = Ports{} }
+func (m *Ports) String() string { return proto.CompactTextString(m) }
+func (*Ports) ProtoMessage()    {}
+func (*Ports) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{14}
+}
+
+func (m *Ports) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Ports.Unmarshal(m, b)
+}
+func (m *Ports) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Ports.Marshal(b, m, deterministic)
+}
+func (m *Ports) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Ports.Merge(m, src)
+}
+func (m *Ports) XXX_Size() int {
+	return xxx_messageInfo_Ports.Size(m)
+}
+func (m *Ports) XXX_DiscardUnknown() {
+	xxx_messageInfo_Ports.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Ports proto.InternalMessageInfo
+
+func (m *Ports) GetItems() []*Port {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+// A Physical Device instance
+type Device struct {
+	// Voltha's device identifier
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Device type, refers to one of the registered device types
+	Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"`
+	// Is this device a root device. Each logical switch has one root
+	// device that is associated with the logical flow switch.
+	Root bool `protobuf:"varint,3,opt,name=root,proto3" json:"root,omitempty"`
+	// Parent device id, in the device tree (for a root device, the parent_id
+	// is the logical_device.id)
+	ParentId     string `protobuf:"bytes,4,opt,name=parent_id,json=parentId,proto3" json:"parent_id,omitempty"`
+	ParentPortNo uint32 `protobuf:"varint,20,opt,name=parent_port_no,json=parentPortNo,proto3" json:"parent_port_no,omitempty"`
+	// Vendor, version, serial number, etc.
+	Vendor          string `protobuf:"bytes,5,opt,name=vendor,proto3" json:"vendor,omitempty"`
+	Model           string `protobuf:"bytes,6,opt,name=model,proto3" json:"model,omitempty"`
+	HardwareVersion string `protobuf:"bytes,7,opt,name=hardware_version,json=hardwareVersion,proto3" json:"hardware_version,omitempty"`
+	FirmwareVersion string `protobuf:"bytes,8,opt,name=firmware_version,json=firmwareVersion,proto3" json:"firmware_version,omitempty"`
+	// List of software on the device
+	Images       *Images `protobuf:"bytes,9,opt,name=images,proto3" json:"images,omitempty"`
+	SerialNumber string  `protobuf:"bytes,10,opt,name=serial_number,json=serialNumber,proto3" json:"serial_number,omitempty"`
+	VendorId     string  `protobuf:"bytes,24,opt,name=vendor_id,json=vendorId,proto3" json:"vendor_id,omitempty"`
+	// Adapter that takes care of device
+	// Deprecated and replaced by adapter_endpoint
+	Adapter string `protobuf:"bytes,11,opt,name=adapter,proto3" json:"adapter,omitempty"`
+	// Indicates how to reach the adapter instance that manages this device
+	AdapterEndpoint string `protobuf:"bytes,25,opt,name=adapter_endpoint,json=adapterEndpoint,proto3" json:"adapter_endpoint,omitempty"`
+	// Device contact on vlan (if 0, no vlan)
+	Vlan uint32 `protobuf:"varint,12,opt,name=vlan,proto3" json:"vlan,omitempty"`
+	// Device contact MAC address (format: "xx:xx:xx:xx:xx:xx")
+	MacAddress string `protobuf:"bytes,13,opt,name=mac_address,json=macAddress,proto3" json:"mac_address,omitempty"`
+	// Types that are valid to be assigned to Address:
+	//	*Device_Ipv4Address
+	//	*Device_Ipv6Address
+	//	*Device_HostAndPort
+	Address       isDevice_Address           `protobuf_oneof:"address"`
+	ExtraArgs     string                     `protobuf:"bytes,23,opt,name=extra_args,json=extraArgs,proto3" json:"extra_args,omitempty"`
+	ProxyAddress  *Device_ProxyAddress       `protobuf:"bytes,19,opt,name=proxy_address,json=proxyAddress,proto3" json:"proxy_address,omitempty"`
+	AdminState    common.AdminState_Types    `protobuf:"varint,16,opt,name=admin_state,json=adminState,proto3,enum=common.AdminState_Types" json:"admin_state,omitempty"`
+	OperStatus    common.OperStatus_Types    `protobuf:"varint,17,opt,name=oper_status,json=operStatus,proto3,enum=common.OperStatus_Types" json:"oper_status,omitempty"`
+	Reason        string                     `protobuf:"bytes,22,opt,name=reason,proto3" json:"reason,omitempty"`
+	ConnectStatus common.ConnectStatus_Types `protobuf:"varint,18,opt,name=connect_status,json=connectStatus,proto3,enum=common.ConnectStatus_Types" json:"connect_status,omitempty"`
+	// Device type specific attributes
+	Custom *any.Any `protobuf:"bytes,64,opt,name=custom,proto3" json:"custom,omitempty"`
+	// PmConfigs will eventually converted to a child node of the
+	// device to falicitata callbacks and to simplify manipulation.
+	PmConfigs            *PmConfigs       `protobuf:"bytes,131,opt,name=pm_configs,json=pmConfigs,proto3" json:"pm_configs,omitempty"`
+	ImageDownloads       []*ImageDownload `protobuf:"bytes,133,rep,name=image_downloads,json=imageDownloads,proto3" json:"image_downloads,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *Device) Reset()         { *m = Device{} }
+func (m *Device) String() string { return proto.CompactTextString(m) }
+func (*Device) ProtoMessage()    {}
+func (*Device) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{15}
+}
+
+func (m *Device) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Device.Unmarshal(m, b)
+}
+func (m *Device) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Device.Marshal(b, m, deterministic)
+}
+func (m *Device) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Device.Merge(m, src)
+}
+func (m *Device) XXX_Size() int {
+	return xxx_messageInfo_Device.Size(m)
+}
+func (m *Device) XXX_DiscardUnknown() {
+	xxx_messageInfo_Device.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Device proto.InternalMessageInfo
+
+func (m *Device) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *Device) GetType() string {
+	if m != nil {
+		return m.Type
+	}
+	return ""
+}
+
+func (m *Device) GetRoot() bool {
+	if m != nil {
+		return m.Root
+	}
+	return false
+}
+
+func (m *Device) GetParentId() string {
+	if m != nil {
+		return m.ParentId
+	}
+	return ""
+}
+
+func (m *Device) GetParentPortNo() uint32 {
+	if m != nil {
+		return m.ParentPortNo
+	}
+	return 0
+}
+
+func (m *Device) GetVendor() string {
+	if m != nil {
+		return m.Vendor
+	}
+	return ""
+}
+
+func (m *Device) GetModel() string {
+	if m != nil {
+		return m.Model
+	}
+	return ""
+}
+
+func (m *Device) GetHardwareVersion() string {
+	if m != nil {
+		return m.HardwareVersion
+	}
+	return ""
+}
+
+func (m *Device) GetFirmwareVersion() string {
+	if m != nil {
+		return m.FirmwareVersion
+	}
+	return ""
+}
+
+func (m *Device) GetImages() *Images {
+	if m != nil {
+		return m.Images
+	}
+	return nil
+}
+
+func (m *Device) GetSerialNumber() string {
+	if m != nil {
+		return m.SerialNumber
+	}
+	return ""
+}
+
+func (m *Device) GetVendorId() string {
+	if m != nil {
+		return m.VendorId
+	}
+	return ""
+}
+
+func (m *Device) GetAdapter() string {
+	if m != nil {
+		return m.Adapter
+	}
+	return ""
+}
+
+func (m *Device) GetAdapterEndpoint() string {
+	if m != nil {
+		return m.AdapterEndpoint
+	}
+	return ""
+}
+
+func (m *Device) GetVlan() uint32 {
+	if m != nil {
+		return m.Vlan
+	}
+	return 0
+}
+
+func (m *Device) GetMacAddress() string {
+	if m != nil {
+		return m.MacAddress
+	}
+	return ""
+}
+
+type isDevice_Address interface {
+	isDevice_Address()
+}
+
+type Device_Ipv4Address struct {
+	Ipv4Address string `protobuf:"bytes,14,opt,name=ipv4_address,json=ipv4Address,proto3,oneof"`
+}
+
+type Device_Ipv6Address struct {
+	Ipv6Address string `protobuf:"bytes,15,opt,name=ipv6_address,json=ipv6Address,proto3,oneof"`
+}
+
+type Device_HostAndPort struct {
+	HostAndPort string `protobuf:"bytes,21,opt,name=host_and_port,json=hostAndPort,proto3,oneof"`
+}
+
+func (*Device_Ipv4Address) isDevice_Address() {}
+
+func (*Device_Ipv6Address) isDevice_Address() {}
+
+func (*Device_HostAndPort) isDevice_Address() {}
+
+func (m *Device) GetAddress() isDevice_Address {
+	if m != nil {
+		return m.Address
+	}
+	return nil
+}
+
+func (m *Device) GetIpv4Address() string {
+	if x, ok := m.GetAddress().(*Device_Ipv4Address); ok {
+		return x.Ipv4Address
+	}
+	return ""
+}
+
+func (m *Device) GetIpv6Address() string {
+	if x, ok := m.GetAddress().(*Device_Ipv6Address); ok {
+		return x.Ipv6Address
+	}
+	return ""
+}
+
+func (m *Device) GetHostAndPort() string {
+	if x, ok := m.GetAddress().(*Device_HostAndPort); ok {
+		return x.HostAndPort
+	}
+	return ""
+}
+
+func (m *Device) GetExtraArgs() string {
+	if m != nil {
+		return m.ExtraArgs
+	}
+	return ""
+}
+
+func (m *Device) GetProxyAddress() *Device_ProxyAddress {
+	if m != nil {
+		return m.ProxyAddress
+	}
+	return nil
+}
+
+func (m *Device) GetAdminState() common.AdminState_Types {
+	if m != nil {
+		return m.AdminState
+	}
+	return common.AdminState_UNKNOWN
+}
+
+func (m *Device) GetOperStatus() common.OperStatus_Types {
+	if m != nil {
+		return m.OperStatus
+	}
+	return common.OperStatus_UNKNOWN
+}
+
+func (m *Device) GetReason() string {
+	if m != nil {
+		return m.Reason
+	}
+	return ""
+}
+
+func (m *Device) GetConnectStatus() common.ConnectStatus_Types {
+	if m != nil {
+		return m.ConnectStatus
+	}
+	return common.ConnectStatus_UNKNOWN
+}
+
+func (m *Device) GetCustom() *any.Any {
+	if m != nil {
+		return m.Custom
+	}
+	return nil
+}
+
+func (m *Device) GetPmConfigs() *PmConfigs {
+	if m != nil {
+		return m.PmConfigs
+	}
+	return nil
+}
+
+func (m *Device) GetImageDownloads() []*ImageDownload {
+	if m != nil {
+		return m.ImageDownloads
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*Device) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*Device_Ipv4Address)(nil),
+		(*Device_Ipv6Address)(nil),
+		(*Device_HostAndPort)(nil),
+	}
+}
+
+type Device_ProxyAddress struct {
+	DeviceId             string   `protobuf:"bytes,1,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	DeviceType           string   `protobuf:"bytes,2,opt,name=device_type,json=deviceType,proto3" json:"device_type,omitempty"`
+	ChannelId            uint32   `protobuf:"varint,3,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"`
+	ChannelGroupId       uint32   `protobuf:"varint,4,opt,name=channel_group_id,json=channelGroupId,proto3" json:"channel_group_id,omitempty"`
+	ChannelTermination   string   `protobuf:"bytes,5,opt,name=channel_termination,json=channelTermination,proto3" json:"channel_termination,omitempty"`
+	OnuId                uint32   `protobuf:"varint,6,opt,name=onu_id,json=onuId,proto3" json:"onu_id,omitempty"`
+	OnuSessionId         uint32   `protobuf:"varint,7,opt,name=onu_session_id,json=onuSessionId,proto3" json:"onu_session_id,omitempty"`
+	AdapterEndpoint      string   `protobuf:"bytes,8,opt,name=adapter_endpoint,json=adapterEndpoint,proto3" json:"adapter_endpoint,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Device_ProxyAddress) Reset()         { *m = Device_ProxyAddress{} }
+func (m *Device_ProxyAddress) String() string { return proto.CompactTextString(m) }
+func (*Device_ProxyAddress) ProtoMessage()    {}
+func (*Device_ProxyAddress) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{15, 0}
+}
+
+func (m *Device_ProxyAddress) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Device_ProxyAddress.Unmarshal(m, b)
+}
+func (m *Device_ProxyAddress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Device_ProxyAddress.Marshal(b, m, deterministic)
+}
+func (m *Device_ProxyAddress) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Device_ProxyAddress.Merge(m, src)
+}
+func (m *Device_ProxyAddress) XXX_Size() int {
+	return xxx_messageInfo_Device_ProxyAddress.Size(m)
+}
+func (m *Device_ProxyAddress) XXX_DiscardUnknown() {
+	xxx_messageInfo_Device_ProxyAddress.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Device_ProxyAddress proto.InternalMessageInfo
+
+func (m *Device_ProxyAddress) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *Device_ProxyAddress) GetDeviceType() string {
+	if m != nil {
+		return m.DeviceType
+	}
+	return ""
+}
+
+func (m *Device_ProxyAddress) GetChannelId() uint32 {
+	if m != nil {
+		return m.ChannelId
+	}
+	return 0
+}
+
+func (m *Device_ProxyAddress) GetChannelGroupId() uint32 {
+	if m != nil {
+		return m.ChannelGroupId
+	}
+	return 0
+}
+
+func (m *Device_ProxyAddress) GetChannelTermination() string {
+	if m != nil {
+		return m.ChannelTermination
+	}
+	return ""
+}
+
+func (m *Device_ProxyAddress) GetOnuId() uint32 {
+	if m != nil {
+		return m.OnuId
+	}
+	return 0
+}
+
+func (m *Device_ProxyAddress) GetOnuSessionId() uint32 {
+	if m != nil {
+		return m.OnuSessionId
+	}
+	return 0
+}
+
+func (m *Device_ProxyAddress) GetAdapterEndpoint() string {
+	if m != nil {
+		return m.AdapterEndpoint
+	}
+	return ""
+}
+
+type Devices struct {
+	Items                []*Device `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *Devices) Reset()         { *m = Devices{} }
+func (m *Devices) String() string { return proto.CompactTextString(m) }
+func (*Devices) ProtoMessage()    {}
+func (*Devices) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{16}
+}
+
+func (m *Devices) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Devices.Unmarshal(m, b)
+}
+func (m *Devices) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Devices.Marshal(b, m, deterministic)
+}
+func (m *Devices) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Devices.Merge(m, src)
+}
+func (m *Devices) XXX_Size() int {
+	return xxx_messageInfo_Devices.Size(m)
+}
+func (m *Devices) XXX_DiscardUnknown() {
+	xxx_messageInfo_Devices.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Devices proto.InternalMessageInfo
+
+func (m *Devices) GetItems() []*Device {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type SimulateAlarmRequest struct {
+	// Device Identifier
+	Id                   string                             `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	Indicator            string                             `protobuf:"bytes,2,opt,name=indicator,proto3" json:"indicator,omitempty"`
+	IntfId               string                             `protobuf:"bytes,3,opt,name=intf_id,json=intfId,proto3" json:"intf_id,omitempty"`
+	PortTypeName         string                             `protobuf:"bytes,4,opt,name=port_type_name,json=portTypeName,proto3" json:"port_type_name,omitempty"`
+	OnuDeviceId          string                             `protobuf:"bytes,5,opt,name=onu_device_id,json=onuDeviceId,proto3" json:"onu_device_id,omitempty"`
+	InverseBitErrorRate  int32                              `protobuf:"varint,6,opt,name=inverse_bit_error_rate,json=inverseBitErrorRate,proto3" json:"inverse_bit_error_rate,omitempty"`
+	Drift                int32                              `protobuf:"varint,7,opt,name=drift,proto3" json:"drift,omitempty"`
+	NewEqd               int32                              `protobuf:"varint,8,opt,name=new_eqd,json=newEqd,proto3" json:"new_eqd,omitempty"`
+	OnuSerialNumber      string                             `protobuf:"bytes,9,opt,name=onu_serial_number,json=onuSerialNumber,proto3" json:"onu_serial_number,omitempty"`
+	Operation            SimulateAlarmRequest_OperationType `protobuf:"varint,10,opt,name=operation,proto3,enum=voltha.SimulateAlarmRequest_OperationType" json:"operation,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
+	XXX_unrecognized     []byte                             `json:"-"`
+	XXX_sizecache        int32                              `json:"-"`
+}
+
+func (m *SimulateAlarmRequest) Reset()         { *m = SimulateAlarmRequest{} }
+func (m *SimulateAlarmRequest) String() string { return proto.CompactTextString(m) }
+func (*SimulateAlarmRequest) ProtoMessage()    {}
+func (*SimulateAlarmRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_200940f73d155856, []int{17}
+}
+
+func (m *SimulateAlarmRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SimulateAlarmRequest.Unmarshal(m, b)
+}
+func (m *SimulateAlarmRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SimulateAlarmRequest.Marshal(b, m, deterministic)
+}
+func (m *SimulateAlarmRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SimulateAlarmRequest.Merge(m, src)
+}
+func (m *SimulateAlarmRequest) XXX_Size() int {
+	return xxx_messageInfo_SimulateAlarmRequest.Size(m)
+}
+func (m *SimulateAlarmRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_SimulateAlarmRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SimulateAlarmRequest proto.InternalMessageInfo
+
+func (m *SimulateAlarmRequest) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetIndicator() string {
+	if m != nil {
+		return m.Indicator
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetIntfId() string {
+	if m != nil {
+		return m.IntfId
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetPortTypeName() string {
+	if m != nil {
+		return m.PortTypeName
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetOnuDeviceId() string {
+	if m != nil {
+		return m.OnuDeviceId
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetInverseBitErrorRate() int32 {
+	if m != nil {
+		return m.InverseBitErrorRate
+	}
+	return 0
+}
+
+func (m *SimulateAlarmRequest) GetDrift() int32 {
+	if m != nil {
+		return m.Drift
+	}
+	return 0
+}
+
+func (m *SimulateAlarmRequest) GetNewEqd() int32 {
+	if m != nil {
+		return m.NewEqd
+	}
+	return 0
+}
+
+func (m *SimulateAlarmRequest) GetOnuSerialNumber() string {
+	if m != nil {
+		return m.OnuSerialNumber
+	}
+	return ""
+}
+
+func (m *SimulateAlarmRequest) GetOperation() SimulateAlarmRequest_OperationType {
+	if m != nil {
+		return m.Operation
+	}
+	return SimulateAlarmRequest_RAISE
+}
+
+func init() {
+	proto.RegisterEnum("voltha.PmConfig_PmType", PmConfig_PmType_name, PmConfig_PmType_value)
+	proto.RegisterEnum("voltha.ImageDownload_ImageDownloadState", ImageDownload_ImageDownloadState_name, ImageDownload_ImageDownloadState_value)
+	proto.RegisterEnum("voltha.ImageDownload_ImageDownloadFailureReason", ImageDownload_ImageDownloadFailureReason_name, ImageDownload_ImageDownloadFailureReason_value)
+	proto.RegisterEnum("voltha.ImageDownload_ImageActivateState", ImageDownload_ImageActivateState_name, ImageDownload_ImageActivateState_value)
+	proto.RegisterEnum("voltha.ImageState_ImageDownloadState", ImageState_ImageDownloadState_name, ImageState_ImageDownloadState_value)
+	proto.RegisterEnum("voltha.ImageState_ImageFailureReason", ImageState_ImageFailureReason_name, ImageState_ImageFailureReason_value)
+	proto.RegisterEnum("voltha.ImageState_ImageActivationState", ImageState_ImageActivationState_name, ImageState_ImageActivationState_value)
+	proto.RegisterEnum("voltha.Port_PortType", Port_PortType_name, Port_PortType_value)
+	proto.RegisterEnum("voltha.SimulateAlarmRequest_OperationType", SimulateAlarmRequest_OperationType_name, SimulateAlarmRequest_OperationType_value)
+	proto.RegisterType((*DeviceType)(nil), "voltha.DeviceType")
+	proto.RegisterType((*DeviceTypes)(nil), "voltha.DeviceTypes")
+	proto.RegisterType((*PmConfig)(nil), "voltha.PmConfig")
+	proto.RegisterType((*PmGroupConfig)(nil), "voltha.PmGroupConfig")
+	proto.RegisterType((*PmConfigs)(nil), "voltha.PmConfigs")
+	proto.RegisterType((*Image)(nil), "voltha.Image")
+	proto.RegisterType((*ImageDownload)(nil), "voltha.ImageDownload")
+	proto.RegisterType((*ImageDownloads)(nil), "voltha.ImageDownloads")
+	proto.RegisterType((*Images)(nil), "voltha.Images")
+	proto.RegisterType((*OnuImage)(nil), "voltha.OnuImage")
+	proto.RegisterType((*OnuImages)(nil), "voltha.OnuImages")
+	proto.RegisterType((*DeviceImageState)(nil), "voltha.DeviceImageState")
+	proto.RegisterType((*ImageState)(nil), "voltha.ImageState")
+	proto.RegisterType((*Port)(nil), "voltha.Port")
+	proto.RegisterType((*Port_PeerPort)(nil), "voltha.Port.PeerPort")
+	proto.RegisterType((*Ports)(nil), "voltha.Ports")
+	proto.RegisterType((*Device)(nil), "voltha.Device")
+	proto.RegisterType((*Device_ProxyAddress)(nil), "voltha.Device.ProxyAddress")
+	proto.RegisterType((*Devices)(nil), "voltha.Devices")
+	proto.RegisterType((*SimulateAlarmRequest)(nil), "voltha.SimulateAlarmRequest")
+}
+
+func init() { proto.RegisterFile("voltha_protos/device.proto", fileDescriptor_200940f73d155856) }
+
+var fileDescriptor_200940f73d155856 = []byte{
+	// 2743 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x59, 0x4d, 0x73, 0xdb, 0xc6,
+	0xf9, 0x37, 0x29, 0x11, 0x24, 0x1e, 0xbe, 0x08, 0x5e, 0x4b, 0x36, 0x2c, 0x45, 0x7f, 0x3b, 0x74,
+	0xf2, 0x8f, 0x6c, 0xd7, 0x92, 0x6b, 0xb7, 0x49, 0x7a, 0xc8, 0x8c, 0x29, 0x12, 0xb2, 0x31, 0x95,
+	0x41, 0x75, 0x49, 0x2a, 0x4d, 0x2f, 0x18, 0x88, 0x58, 0x49, 0x18, 0x83, 0x00, 0xbd, 0x00, 0x29,
+	0x39, 0xb7, 0x4e, 0xa7, 0x3d, 0x75, 0xa6, 0x87, 0x9e, 0xfa, 0x0d, 0x7a, 0xea, 0xf4, 0x96, 0x6b,
+	0xbf, 0x40, 0x66, 0xfa, 0x19, 0xd2, 0x4b, 0x8f, 0x3d, 0x74, 0x72, 0xee, 0xec, 0x1b, 0x08, 0x48,
+	0xb2, 0x93, 0xce, 0x74, 0x3a, 0xbd, 0x48, 0xd8, 0xdf, 0xf3, 0xb2, 0xbb, 0xcf, 0xee, 0xf3, 0xdb,
+	0x67, 0x97, 0xb0, 0x3e, 0x8f, 0xc3, 0xf4, 0xd4, 0x73, 0xa7, 0x34, 0x4e, 0xe3, 0x64, 0xc7, 0x27,
+	0xf3, 0x60, 0x4c, 0xb6, 0x79, 0x0b, 0x69, 0x42, 0xb6, 0x7e, 0xfb, 0x24, 0x8e, 0x4f, 0x42, 0xb2,
+	0xc3, 0xd1, 0xa3, 0xd9, 0xf1, 0x8e, 0x17, 0xbd, 0x11, 0x2a, 0xeb, 0x17, 0xcc, 0xc7, 0xf1, 0x64,
+	0x12, 0x47, 0x52, 0x66, 0x16, 0x65, 0x13, 0x92, 0x7a, 0x52, 0x72, 0xa7, 0x28, 0x89, 0xa7, 0x24,
+	0x3a, 0x0e, 0xe3, 0x33, 0xf7, 0x87, 0x4f, 0x85, 0x42, 0xfb, 0x9b, 0x32, 0x40, 0x8f, 0x0f, 0x65,
+	0xf8, 0x66, 0x4a, 0x50, 0x0b, 0xca, 0x81, 0x6f, 0x96, 0xee, 0x96, 0xb6, 0x74, 0x5c, 0x0e, 0x7c,
+	0xb4, 0x01, 0xfa, 0x9c, 0x44, 0x7e, 0x4c, 0xdd, 0xc0, 0x37, 0x2b, 0x1c, 0xae, 0x09, 0xc0, 0xf6,
+	0xd1, 0x26, 0x40, 0x26, 0x4c, 0x4c, 0xed, 0xee, 0xd2, 0x96, 0x8e, 0x75, 0x25, 0x4d, 0x90, 0x09,
+	0x55, 0xcf, 0xf7, 0xa6, 0x29, 0xa1, 0x66, 0x99, 0x5b, 0xaa, 0x26, 0xfa, 0x04, 0x4c, 0x6f, 0x3c,
+	0x26, 0xd3, 0x34, 0x71, 0x8f, 0x66, 0xe1, 0x2b, 0x97, 0x0f, 0x69, 0x36, 0xf5, 0xbd, 0x94, 0x98,
+	0x4b, 0x77, 0x4b, 0x5b, 0x35, 0xbc, 0x26, 0xe5, 0xbb, 0xb3, 0xf0, 0xd5, 0x5e, 0x18, 0x9f, 0x8d,
+	0xb8, 0x10, 0xf5, 0xe0, 0x8e, 0x32, 0xf4, 0x7c, 0xdf, 0xa5, 0x64, 0x12, 0xcf, 0x49, 0xde, 0x3c,
+	0x31, 0x97, 0xb9, 0xfd, 0x86, 0x54, 0xeb, 0xf8, 0x3e, 0xe6, 0x4a, 0x0b, 0x27, 0x09, 0xda, 0x87,
+	0x7b, 0xca, 0x8b, 0x1f, 0x50, 0x32, 0x4e, 0xdd, 0x30, 0x3e, 0x09, 0xc6, 0x5e, 0xc8, 0x3d, 0x25,
+	0x6a, 0x24, 0x55, 0xee, 0x49, 0x75, 0xd8, 0xe3, 0x9a, 0xfb, 0x42, 0x91, 0x79, 0x4b, 0xe4, 0x98,
+	0xde, 0x87, 0x86, 0x9c, 0x97, 0x9b, 0xbe, 0x99, 0x12, 0xb3, 0xc6, 0xe7, 0x5a, 0x97, 0x18, 0x8b,
+	0x6a, 0xfb, 0x13, 0xa8, 0x2f, 0x62, 0x9c, 0xa0, 0x2d, 0xa8, 0x04, 0x29, 0x99, 0x24, 0x66, 0xe9,
+	0xee, 0xd2, 0x56, 0xfd, 0x09, 0xda, 0x16, 0x8b, 0xb4, 0xbd, 0xd0, 0xc1, 0x42, 0xa1, 0xfd, 0x97,
+	0x12, 0xd4, 0x0e, 0x26, 0xdd, 0x38, 0x3a, 0x0e, 0x4e, 0x10, 0x82, 0xe5, 0xc8, 0x9b, 0x10, 0xb9,
+	0x3a, 0xfc, 0x1b, 0x3d, 0x84, 0x65, 0xde, 0x29, 0x0b, 0x70, 0xeb, 0xc9, 0x2d, 0xe5, 0x49, 0xd9,
+	0x6c, 0x1f, 0x4c, 0xb8, 0x3b, 0xae, 0xc4, 0x16, 0x84, 0x44, 0xde, 0x51, 0x48, 0x7c, 0x19, 0x65,
+	0xd5, 0x44, 0x77, 0xa0, 0x9e, 0x78, 0x93, 0x69, 0x48, 0xdc, 0x63, 0x4a, 0x5e, 0xf3, 0x18, 0x36,
+	0x31, 0x08, 0x68, 0x8f, 0x92, 0xd7, 0xed, 0x4f, 0x41, 0x13, 0xae, 0x50, 0x1d, 0xaa, 0xdd, 0xfe,
+	0xc8, 0x19, 0x5a, 0xd8, 0xb8, 0x86, 0x74, 0xa8, 0x3c, 0xef, 0x8c, 0x9e, 0x5b, 0x46, 0x89, 0x7d,
+	0x0e, 0x86, 0x9d, 0xa1, 0x65, 0x94, 0x85, 0x8a, 0x33, 0xb4, 0x7e, 0x3e, 0x34, 0x96, 0xda, 0xbf,
+	0x2f, 0x41, 0xf3, 0x60, 0xf2, 0x9c, 0xc6, 0xb3, 0xa9, 0x9c, 0xc7, 0x26, 0xc0, 0x09, 0x6b, 0xba,
+	0xb9, 0xd9, 0xe8, 0x1c, 0x71, 0xd8, 0x94, 0x32, 0x31, 0x1f, 0x4a, 0x99, 0x0f, 0x45, 0x88, 0xd9,
+	0x48, 0xde, 0x31, 0x89, 0x07, 0x50, 0x9d, 0x90, 0x94, 0x06, 0x63, 0xb6, 0x09, 0x58, 0x60, 0x8d,
+	0x8b, 0xe1, 0xc0, 0x4a, 0xa1, 0xfd, 0xcb, 0x32, 0xe8, 0x0a, 0x4d, 0x2e, 0xed, 0xfa, 0xf7, 0xa1,
+	0xe1, 0x93, 0x63, 0x6f, 0x16, 0xa6, 0xf9, 0x41, 0xd4, 0x25, 0xc6, 0x87, 0x71, 0x07, 0xaa, 0x7c,
+	0x4c, 0x6a, 0x18, 0xbb, 0x95, 0xbf, 0x7f, 0xfb, 0xf5, 0x66, 0x09, 0x2b, 0x14, 0x3d, 0x80, 0x26,
+	0xb3, 0x75, 0xe3, 0x39, 0xa1, 0x34, 0xf0, 0x89, 0xd8, 0x98, 0x4a, 0xad, 0xc1, 0x64, 0x7d, 0x29,
+	0x42, 0x8f, 0x40, 0xe3, 0x66, 0x89, 0x59, 0xe1, 0x03, 0x5f, 0x5b, 0x0c, 0x3c, 0x17, 0x38, 0x2c,
+	0x95, 0xf2, 0x13, 0xd5, 0xbe, 0x63, 0xa2, 0xe8, 0x36, 0xd4, 0x26, 0xde, 0xb9, 0x9b, 0xbc, 0x22,
+	0x67, 0x7c, 0x43, 0x37, 0x71, 0x75, 0xe2, 0x9d, 0x0f, 0x5e, 0x91, 0xb3, 0xf6, 0xef, 0xca, 0x50,
+	0xb1, 0x27, 0xde, 0x09, 0xb9, 0x72, 0x67, 0x99, 0x50, 0x9d, 0x13, 0x9a, 0x04, 0x71, 0xa4, 0xb2,
+	0x57, 0x36, 0x99, 0xf6, 0xa9, 0x97, 0x9c, 0xf2, 0x79, 0x37, 0x31, 0xff, 0x46, 0xf7, 0xc1, 0x08,
+	0xa2, 0x24, 0xf5, 0xc2, 0xd0, 0x65, 0x49, 0x91, 0x06, 0x13, 0x31, 0x61, 0x1d, 0xaf, 0x48, 0xbc,
+	0x27, 0x61, 0x46, 0x29, 0x41, 0xe2, 0x7a, 0xe3, 0x34, 0x98, 0x13, 0x4e, 0x29, 0x35, 0x5c, 0x0b,
+	0x92, 0x0e, 0x6f, 0xb3, 0xc8, 0x07, 0x89, 0xcb, 0xc8, 0x2d, 0x48, 0x53, 0xe2, 0x9b, 0x1a, 0x97,
+	0xd7, 0x83, 0xa4, 0xab, 0x20, 0x36, 0xa3, 0x20, 0x71, 0xe7, 0x5e, 0x18, 0xf8, 0x32, 0x45, 0xab,
+	0x41, 0x72, 0xc8, 0x9a, 0xc8, 0x80, 0xa5, 0x19, 0x0d, 0x65, 0x06, 0xb2, 0x4f, 0x74, 0x13, 0x34,
+	0x41, 0x48, 0xa6, 0xce, 0x41, 0xd9, 0x42, 0xab, 0x50, 0x19, 0xd3, 0xf1, 0xd3, 0x27, 0x26, 0xf0,
+	0x49, 0x88, 0x46, 0xfb, 0x6f, 0x55, 0x68, 0xf2, 0x88, 0xf4, 0xe2, 0xb3, 0x28, 0x8c, 0x3d, 0xff,
+	0xd2, 0xce, 0x50, 0x91, 0x2a, 0xe7, 0x22, 0x25, 0x7b, 0x5d, 0x5a, 0xf4, 0x6a, 0xc0, 0xd2, 0x98,
+	0x8e, 0x65, 0x1a, 0xb1, 0x4f, 0xd4, 0x87, 0x96, 0x2f, 0x7d, 0xba, 0x49, 0xca, 0xd8, 0xa5, 0xc2,
+	0x33, 0x76, 0x4b, 0xad, 0x5c, 0xa1, 0xdb, 0x62, 0x6b, 0xc0, 0xf4, 0x71, 0xd3, 0xcf, 0x37, 0xd1,
+	0x3d, 0x68, 0x06, 0x4c, 0xc9, 0x55, 0x8b, 0xa4, 0xf1, 0xee, 0x1b, 0x1c, 0x3c, 0x94, 0x2b, 0x75,
+	0x1f, 0x0c, 0x65, 0x45, 0x7c, 0xf7, 0xe8, 0x0d, 0xe3, 0x47, 0xb1, 0x09, 0x56, 0x16, 0xf8, 0x2e,
+	0x83, 0xd1, 0x0b, 0xd0, 0x28, 0xf1, 0x92, 0x38, 0xe2, 0xd1, 0x6b, 0x3d, 0x79, 0xfc, 0x3d, 0x06,
+	0xb6, 0xe7, 0x05, 0xe1, 0x8c, 0x12, 0xcc, 0xed, 0xb0, 0xb4, 0x47, 0x1f, 0xc1, 0x8a, 0xe7, 0xfb,
+	0x41, 0x1a, 0xc4, 0x91, 0x17, 0xba, 0x41, 0x74, 0x1c, 0xcb, 0xd8, 0xb7, 0x16, 0xb0, 0x1d, 0x1d,
+	0xc7, 0x82, 0x74, 0xe6, 0xc4, 0x1d, 0xf3, 0x2d, 0xcb, 0x57, 0xa2, 0xc6, 0x48, 0x67, 0x4e, 0x24,
+	0x51, 0x6c, 0x80, 0x1e, 0xc6, 0x8c, 0x96, 0xfd, 0x80, 0x9a, 0x75, 0x71, 0xf8, 0x70, 0xa0, 0x17,
+	0x50, 0x64, 0x43, 0x5d, 0x04, 0x40, 0x84, 0xb3, 0xf1, 0x9d, 0xe1, 0xe4, 0x3b, 0xcc, 0x4b, 0x89,
+	0x08, 0x27, 0x70, 0x63, 0x11, 0xcb, 0x0d, 0xd0, 0x8f, 0x83, 0x90, 0xb8, 0x49, 0xf0, 0x25, 0x31,
+	0x9b, 0x3c, 0x3e, 0x35, 0x06, 0x0c, 0x82, 0x2f, 0x49, 0xfb, 0xab, 0x12, 0xa0, 0xcb, 0xcb, 0x81,
+	0x56, 0xc1, 0xe8, 0xf5, 0x3f, 0x77, 0xf6, 0xfb, 0x9d, 0x9e, 0x3b, 0x72, 0x7e, 0xea, 0xf4, 0x3f,
+	0x77, 0x8c, 0x6b, 0xe8, 0x26, 0xa0, 0x0c, 0x1d, 0x8c, 0xba, 0x5d, 0xcb, 0xea, 0x59, 0x3d, 0xa3,
+	0x54, 0xc0, 0xb1, 0xf5, 0xb3, 0x91, 0x35, 0x18, 0x5a, 0x3d, 0xa3, 0x5c, 0xf0, 0x32, 0x18, 0x76,
+	0x30, 0x43, 0x97, 0xd0, 0x0d, 0x58, 0xc9, 0xd0, 0xbd, 0x8e, 0xbd, 0x6f, 0xf5, 0x8c, 0x65, 0x64,
+	0xc2, 0x6a, 0xae, 0xc3, 0xc1, 0xe8, 0xe0, 0xa0, 0xcf, 0xd5, 0x2b, 0x05, 0xe7, 0xdd, 0x8e, 0xd3,
+	0xb5, 0xf6, 0x99, 0x85, 0xd6, 0xfe, 0x4d, 0x09, 0xd6, 0xdf, 0xbe, 0x5e, 0xa8, 0x01, 0x35, 0xa7,
+	0xef, 0x5a, 0x18, 0xf7, 0x19, 0x93, 0xaf, 0x40, 0xdd, 0x76, 0x0e, 0x3b, 0xfb, 0x76, 0xcf, 0x1d,
+	0xe1, 0x7d, 0xa3, 0xc4, 0x80, 0x9e, 0x75, 0x68, 0x77, 0x2d, 0x77, 0x77, 0x34, 0xf8, 0xc2, 0x28,
+	0xb3, 0x6e, 0x6c, 0x67, 0x30, 0xda, 0xdb, 0xb3, 0xbb, 0xb6, 0xe5, 0x0c, 0xdd, 0xc1, 0x41, 0xa7,
+	0x6b, 0x19, 0x4b, 0xe8, 0x3a, 0x34, 0x65, 0x00, 0xa4, 0xb3, 0x65, 0xd4, 0x04, 0x7d, 0x31, 0x90,
+	0x4a, 0xfb, 0xb7, 0x2a, 0x84, 0x85, 0x25, 0x60, 0x86, 0xf6, 0xcb, 0xce, 0x73, 0x2b, 0x17, 0x3f,
+	0x04, 0x2d, 0x01, 0xd9, 0x4e, 0xa7, 0x3b, 0xb4, 0x0f, 0xd9, 0xc1, 0xb2, 0x0a, 0x86, 0xc0, 0x38,
+	0xd2, 0x19, 0xda, 0xce, 0x73, 0xa3, 0x8c, 0x0c, 0x68, 0xe4, 0x50, 0x4b, 0x44, 0x4d, 0x20, 0xd8,
+	0x3a, 0xb4, 0x30, 0x57, 0x5b, 0x5e, 0x38, 0x14, 0x20, 0x1f, 0xce, 0x67, 0xd0, 0x2a, 0x84, 0x25,
+	0x41, 0x0f, 0xd5, 0x81, 0x5c, 0x2e, 0xd2, 0x6f, 0x41, 0x4d, 0x9d, 0xc9, 0x8f, 0x40, 0xe3, 0x78,
+	0x82, 0xee, 0x41, 0x85, 0xef, 0x22, 0x79, 0x8e, 0x37, 0x0b, 0x66, 0x58, 0xc8, 0xda, 0x7f, 0x2e,
+	0x41, 0xad, 0x1f, 0xcd, 0x04, 0xd1, 0xe6, 0x48, 0xb5, 0x54, 0x24, 0xd5, 0xff, 0x03, 0x50, 0x24,
+	0x47, 0x7c, 0x4e, 0x2f, 0x35, 0x9c, 0x43, 0xd0, 0x3a, 0x64, 0x24, 0x29, 0xcf, 0xbd, 0x05, 0x69,
+	0x9a, 0xa0, 0x18, 0x50, 0x56, 0x3f, 0x19, 0x21, 0xde, 0x85, 0xfa, 0x94, 0xc6, 0xfe, 0x6c, 0x9c,
+	0x76, 0x63, 0x9f, 0xc8, 0x02, 0x2e, 0x0f, 0x65, 0x64, 0x2e, 0xe8, 0x83, 0x7f, 0xb7, 0x9f, 0x82,
+	0xae, 0x46, 0x9c, 0xa0, 0xff, 0x2f, 0x16, 0x2b, 0xd9, 0x51, 0xa3, 0x34, 0x54, 0x58, 0xc6, 0x60,
+	0x88, 0xfa, 0xc5, 0x2e, 0x24, 0x96, 0x28, 0x73, 0xdd, 0x8c, 0x44, 0x6b, 0x02, 0xb0, 0x7d, 0xf4,
+	0x04, 0x72, 0x39, 0xc8, 0x67, 0x9c, 0x2b, 0x85, 0x16, 0x4e, 0xf2, 0x99, 0xca, 0x08, 0x1a, 0x72,
+	0xfe, 0xdf, 0x1e, 0xce, 0xfd, 0x4b, 0x7c, 0x2b, 0x2a, 0xa4, 0x0f, 0x2f, 0x77, 0xf0, 0x3d, 0xc8,
+	0xf6, 0xb3, 0x8c, 0x1c, 0x97, 0xde, 0xed, 0xe5, 0x6a, 0x46, 0x7c, 0x51, 0xa4, 0xaa, 0x65, 0xee,
+	0xe3, 0xa3, 0xb7, 0xf9, 0x90, 0x49, 0x12, 0xc4, 0xd1, 0xe5, 0xf9, 0xff, 0xf5, 0x7f, 0x9e, 0x8c,
+	0x6e, 0xc1, 0x8d, 0x8b, 0x64, 0xc4, 0x32, 0x51, 0x7b, 0x0b, 0x4b, 0x55, 0xdb, 0xff, 0x54, 0x53,
+	0xfa, 0xaf, 0xb1, 0x93, 0x09, 0xab, 0xd9, 0x00, 0xdc, 0xbe, 0xa3, 0x62, 0x60, 0x54, 0xd0, 0x3a,
+	0xdc, 0x2c, 0x48, 0xfa, 0xce, 0xc8, 0x15, 0x45, 0xad, 0xc6, 0x64, 0x87, 0x96, 0xd3, 0xeb, 0x63,
+	0x57, 0x76, 0xfc, 0xd2, 0x1e, 0xbc, 0xec, 0x0c, 0xbb, 0x2f, 0x8c, 0x2a, 0x9b, 0x74, 0xff, 0x65,
+	0xd7, 0x76, 0x87, 0xb8, 0xe3, 0x0c, 0xf6, 0x2c, 0x2c, 0xbb, 0xaa, 0xb1, 0xae, 0x14, 0xfd, 0xec,
+	0x8d, 0x06, 0x56, 0xcf, 0xdd, 0xfd, 0x82, 0x39, 0x35, 0xf4, 0xf6, 0x1f, 0xca, 0xb0, 0x7a, 0xd5,
+	0x72, 0xff, 0xa7, 0x59, 0x31, 0xd3, 0xeb, 0xf6, 0x5f, 0xbe, 0xb4, 0x87, 0x92, 0x16, 0x33, 0xae,
+	0x94, 0x28, 0x5f, 0xba, 0x4d, 0xb8, 0x5d, 0x74, 0xd9, 0x77, 0xdc, 0xce, 0x6e, 0x5f, 0x50, 0xa9,
+	0x86, 0xde, 0x03, 0xf3, 0x6a, 0x31, 0x5b, 0x46, 0x74, 0x1b, 0xd6, 0xf2, 0x1e, 0x17, 0x86, 0xb9,
+	0x20, 0xe4, 0x45, 0x56, 0xcf, 0xd0, 0xd1, 0x1a, 0x5c, 0x17, 0x12, 0xb5, 0x33, 0x98, 0x01, 0xb4,
+	0xbf, 0xaa, 0xc0, 0xf2, 0x41, 0x4c, 0x53, 0x74, 0x0b, 0xaa, 0xd3, 0x98, 0xa6, 0x6e, 0x14, 0xf3,
+	0xfc, 0x6e, 0x62, 0x8d, 0x35, 0x9d, 0x98, 0x95, 0x6f, 0xa1, 0x77, 0x44, 0x42, 0x59, 0x87, 0x89,
+	0x06, 0xba, 0x2f, 0x2f, 0x43, 0x22, 0x49, 0x17, 0x45, 0x74, 0x4c, 0x53, 0xfe, 0x27, 0x77, 0x15,
+	0xfa, 0x09, 0xd4, 0x3d, 0x7f, 0x12, 0x44, 0x85, 0x62, 0xcc, 0xdc, 0x96, 0xb7, 0xea, 0x0e, 0x13,
+	0x89, 0x94, 0xe4, 0x37, 0x36, 0x0c, 0x5e, 0x86, 0x30, 0xd3, 0x78, 0x4a, 0x28, 0xb7, 0x9c, 0x25,
+	0x9c, 0x38, 0x73, 0xa6, 0xfd, 0x29, 0xa1, 0x03, 0x2e, 0x51, 0xa6, 0x71, 0x86, 0x14, 0xf9, 0xb0,
+	0x7a, 0x81, 0x0f, 0x1f, 0x42, 0x65, 0x4a, 0x08, 0x4d, 0xcc, 0xda, 0x85, 0x3b, 0x00, 0x1f, 0x3e,
+	0x21, 0x94, 0x7d, 0x60, 0xa1, 0xc3, 0x2e, 0x49, 0xf4, 0xdc, 0x9d, 0x7a, 0xe3, 0x57, 0x24, 0x4d,
+	0x78, 0x7d, 0xa5, 0x61, 0x9d, 0x9e, 0x1f, 0x08, 0x80, 0xd5, 0xc8, 0xf4, 0x5c, 0x16, 0x7c, 0xc0,
+	0x85, 0x55, 0x7a, 0x2e, 0x0a, 0xbd, 0x0d, 0xd0, 0xe9, 0xb9, 0x4b, 0x28, 0x8d, 0x69, 0xc2, 0x8b,
+	0x2a, 0x0d, 0xd7, 0xe8, 0xb9, 0xc5, 0xdb, 0xcc, 0x6d, 0xba, 0x70, 0xdb, 0x10, 0x6e, 0xd3, 0xbc,
+	0xdb, 0x54, 0xb9, 0x6d, 0x0a, 0xb7, 0xe9, 0xc2, 0x6d, 0x9a, 0xb9, 0x6d, 0x09, 0xb7, 0xa9, 0x72,
+	0xfb, 0x18, 0x6a, 0xf1, 0xf1, 0xd4, 0x65, 0x8b, 0x67, 0xae, 0x70, 0xa2, 0x5f, 0xdb, 0xce, 0x3f,
+	0x45, 0x28, 0x21, 0xae, 0xc6, 0xc7, 0x53, 0x36, 0xcd, 0xf5, 0x67, 0x50, 0x53, 0x53, 0x7e, 0xf7,
+	0x29, 0x92, 0xdb, 0x22, 0xe5, 0xfc, 0x16, 0x69, 0x27, 0x50, 0x53, 0x6b, 0xce, 0x2e, 0xa4, 0x8b,
+	0x6c, 0x32, 0xa0, 0x61, 0x0d, 0x5f, 0x58, 0xd8, 0xb1, 0x86, 0xae, 0xe3, 0xd8, 0x46, 0xa9, 0x80,
+	0x8c, 0x1c, 0x5b, 0xdc, 0x60, 0x0f, 0x58, 0xfe, 0xef, 0x0f, 0x8d, 0xa5, 0xac, 0xe1, 0x8c, 0x44,
+	0x69, 0x73, 0x68, 0x31, 0x45, 0x26, 0xab, 0xe4, 0x9a, 0xce, 0xc8, 0xd0, 0xda, 0x0f, 0xa1, 0xc2,
+	0x3a, 0x4d, 0x50, 0xbb, 0x78, 0x6a, 0x36, 0xf2, 0x8b, 0xa9, 0x4e, 0xcc, 0x7f, 0xd4, 0x41, 0x13,
+	0x47, 0x26, 0x5a, 0x5b, 0x5c, 0x33, 0xd4, 0x0d, 0x91, 0xdd, 0x36, 0x6e, 0xe7, 0x6e, 0xf7, 0x99,
+	0x40, 0x6c, 0xe0, 0xdb, 0xb0, 0x4c, 0xe3, 0x38, 0x2d, 0x5e, 0x3e, 0x39, 0x84, 0xda, 0xa0, 0x4f,
+	0x3d, 0x4a, 0xa2, 0xd4, 0x95, 0x05, 0x41, 0x66, 0x5a, 0x13, 0x38, 0xdf, 0x6c, 0x2d, 0xa9, 0xa3,
+	0xa2, 0xb7, 0xca, 0xa2, 0x97, 0x5d, 0x4f, 0x85, 0xf0, 0x40, 0x64, 0xdb, 0x66, 0x76, 0x89, 0xaa,
+	0xe4, 0xbd, 0xa9, 0xbb, 0xd4, 0x06, 0x54, 0x26, 0xb1, 0x4f, 0x42, 0x51, 0x43, 0x28, 0xa9, 0xc0,
+	0xd0, 0x63, 0x30, 0x4e, 0x3d, 0xea, 0x9f, 0x79, 0x74, 0x71, 0x55, 0xa9, 0xe6, 0xf5, 0x56, 0x94,
+	0x58, 0x5d, 0x5a, 0x1e, 0x83, 0x71, 0x1c, 0xd0, 0x49, 0xc1, 0xa2, 0x56, 0xb0, 0x50, 0x62, 0x65,
+	0xf1, 0x08, 0x34, 0x7e, 0x46, 0x8a, 0x44, 0xa8, 0x3f, 0x69, 0x15, 0x8e, 0xd6, 0x24, 0x1b, 0xaf,
+	0x50, 0x62, 0x37, 0xf3, 0x84, 0xd0, 0xc0, 0x0b, 0xdd, 0x68, 0x36, 0x39, 0x22, 0x94, 0x67, 0x48,
+	0xe6, 0xbd, 0x21, 0x64, 0x0e, 0x17, 0xb1, 0x58, 0x2e, 0xde, 0xbf, 0xcc, 0x42, 0x2c, 0xb3, 0x67,
+	0xb0, 0x3b, 0x8b, 0x77, 0xae, 0x7a, 0x5e, 0x23, 0x7b, 0xee, 0xba, 0x0f, 0x86, 0x7a, 0x21, 0x22,
+	0x91, 0x3f, 0x8d, 0x83, 0x28, 0x35, 0x6f, 0x8b, 0xcb, 0xb1, 0xc4, 0x2d, 0x09, 0xb3, 0x72, 0x6c,
+	0x1e, 0x7a, 0x11, 0x4f, 0xbd, 0x26, 0xe6, 0xdf, 0xec, 0x9e, 0x34, 0xf1, 0xc6, 0xae, 0xe7, 0xfb,
+	0x94, 0x24, 0x22, 0xf1, 0x74, 0x0c, 0x13, 0x6f, 0xdc, 0x11, 0x08, 0xba, 0x07, 0x8d, 0x60, 0x3a,
+	0xff, 0x51, 0xa6, 0xc1, 0xd2, 0x4f, 0x7f, 0x71, 0x0d, 0xd7, 0x19, 0x5a, 0x54, 0xfa, 0x38, 0x53,
+	0x5a, 0xc9, 0x29, 0x7d, 0xac, 0x94, 0x3e, 0x80, 0xe6, 0x69, 0x9c, 0xa4, 0xae, 0x17, 0xf9, 0x22,
+	0x5b, 0xd7, 0x94, 0x16, 0x83, 0x3b, 0x91, 0xcf, 0x13, 0x72, 0x13, 0x80, 0x9c, 0xa7, 0xd4, 0x73,
+	0x3d, 0x7a, 0x92, 0x98, 0xb7, 0xc4, 0x03, 0x0e, 0x47, 0x3a, 0xf4, 0x24, 0x41, 0xcf, 0xa0, 0x39,
+	0xa5, 0xf1, 0xf9, 0x9b, 0xac, 0xab, 0x1b, 0x7c, 0x55, 0x36, 0x8a, 0xcf, 0x5c, 0xdb, 0x07, 0x4c,
+	0x47, 0x76, 0x8c, 0x1b, 0xd3, 0x5c, 0xeb, 0x22, 0x3b, 0x1b, 0xff, 0x06, 0x3b, 0x3f, 0x2b, 0xb2,
+	0xf3, 0xf5, 0x77, 0xb3, 0xb3, 0x5a, 0xaa, 0x3c, 0x49, 0x6f, 0x66, 0xc5, 0xde, 0xcd, 0xc2, 0x6e,
+	0x97, 0xc5, 0x9c, 0x0d, 0xad, 0x71, 0x1c, 0x45, 0x64, 0x9c, 0xaa, 0x3e, 0x10, 0xef, 0x63, 0x43,
+	0xf5, 0xd1, 0x15, 0xd2, 0xab, 0xba, 0x69, 0x8e, 0xf3, 0x32, 0xf4, 0x03, 0xd0, 0xc6, 0xb3, 0x24,
+	0x8d, 0x27, 0xe6, 0x33, 0x1e, 0xa1, 0xd5, 0x6d, 0xf1, 0xfc, 0xbb, 0xad, 0x9e, 0x7f, 0xb7, 0x3b,
+	0xd1, 0x1b, 0x2c, 0x75, 0xd0, 0xa7, 0x00, 0xd3, 0x89, 0xbc, 0x2c, 0x27, 0xe6, 0xaf, 0x4a, 0xdc,
+	0xe4, 0xfa, 0xc5, 0x97, 0x9f, 0x64, 0xb7, 0xf2, 0xcd, 0xb7, 0x5f, 0x6f, 0x5e, 0xc3, 0xfa, 0x34,
+	0x7b, 0xde, 0xda, 0x87, 0x15, 0x51, 0x7f, 0xaa, 0xaa, 0x36, 0x31, 0x7f, 0x5d, 0x7a, 0xc7, 0x4d,
+	0x67, 0xb7, 0xce, 0x5c, 0x68, 0xe2, 0xa9, 0x03, 0xb7, 0x82, 0xc2, 0x65, 0x69, 0xfd, 0x4f, 0x65,
+	0x68, 0xe4, 0xd7, 0xee, 0xdd, 0xfc, 0x7c, 0x07, 0xea, 0x52, 0xb8, 0x60, 0x32, 0x0c, 0xfe, 0xe2,
+	0xc5, 0x79, 0x13, 0x60, 0x7c, 0xea, 0x45, 0x11, 0x09, 0x99, 0xb9, 0x78, 0x53, 0xd2, 0x25, 0x62,
+	0xfb, 0x68, 0x0b, 0x0c, 0x25, 0x16, 0xaf, 0x82, 0x92, 0xd3, 0x9a, 0xb8, 0x25, 0x71, 0xfe, 0x42,
+	0x66, 0xfb, 0x68, 0x07, 0x6e, 0x28, 0xcd, 0x94, 0xd0, 0x49, 0x10, 0xf1, 0xa2, 0x4a, 0xde, 0x79,
+	0x90, 0x14, 0x0d, 0x17, 0x12, 0xb4, 0x06, 0x5a, 0x1c, 0xcd, 0x98, 0x43, 0x4d, 0x3c, 0x02, 0xc5,
+	0xd1, 0xcc, 0xf6, 0xd1, 0x07, 0xd0, 0x62, 0x70, 0x42, 0x12, 0x46, 0x2e, 0xea, 0xa4, 0x6e, 0xe2,
+	0x46, 0x1c, 0xcd, 0x06, 0x02, 0xb4, 0xfd, 0x2b, 0x73, 0xba, 0x76, 0x65, 0x4e, 0xef, 0xea, 0x8c,
+	0x1f, 0x78, 0xa8, 0xda, 0x3b, 0x50, 0x15, 0xbb, 0x9f, 0xa5, 0x5a, 0xe1, 0x84, 0x68, 0x15, 0xb3,
+	0x43, 0x9d, 0x11, 0x7f, 0x5c, 0x82, 0xd5, 0x41, 0x30, 0x99, 0x85, 0x5e, 0x4a, 0x3a, 0xa1, 0x47,
+	0x27, 0x98, 0xbc, 0x9e, 0x91, 0x24, 0xbd, 0xf4, 0x30, 0xf5, 0x1e, 0xe8, 0x41, 0xe4, 0x07, 0x63,
+	0x2f, 0x8d, 0xd5, 0x73, 0xfb, 0x02, 0x60, 0xa7, 0x64, 0x10, 0xa5, 0xc7, 0x2a, 0xc2, 0x3a, 0xd6,
+	0x58, 0x53, 0x4c, 0x96, 0x1f, 0x00, 0x6c, 0x71, 0xc4, 0x7b, 0xac, 0x78, 0xb5, 0x6b, 0x4c, 0xe5,
+	0xd9, 0xc9, 0x9f, 0x64, 0xdb, 0xd0, 0x64, 0x21, 0x59, 0xac, 0xb2, 0xbc, 0x48, 0xc6, 0xd1, 0xac,
+	0xa7, 0x16, 0xfa, 0x29, 0xdc, 0x0c, 0x22, 0xc6, 0xd7, 0xc4, 0x3d, 0x0a, 0x52, 0x51, 0x09, 0xb8,
+	0x94, 0xa5, 0x2f, 0x8b, 0x6e, 0x05, 0xdf, 0x90, 0xd2, 0xdd, 0x20, 0xe5, 0x55, 0x01, 0x16, 0x17,
+	0x97, 0x8a, 0x4f, 0x83, 0xe3, 0x94, 0x87, 0xb8, 0x82, 0x45, 0x83, 0x8d, 0x36, 0x22, 0x67, 0x2e,
+	0x79, 0xed, 0xf3, 0x90, 0x56, 0xb0, 0x16, 0x91, 0x33, 0xeb, 0xb5, 0x8f, 0x1e, 0xc0, 0x75, 0xb1,
+	0x34, 0x79, 0xf6, 0x16, 0x8f, 0x4b, 0x2b, 0x7c, 0x75, 0x72, 0xcc, 0xfd, 0x02, 0x74, 0x96, 0xd4,
+	0x62, 0x13, 0x00, 0x4f, 0xd1, 0x07, 0x2a, 0xc6, 0x57, 0x45, 0x94, 0x73, 0x03, 0xd7, 0xe6, 0x65,
+	0xe2, 0xc2, 0xb8, 0xfd, 0x21, 0x34, 0x0b, 0x32, 0xa4, 0x43, 0x05, 0x77, 0xec, 0x81, 0x25, 0x1e,
+	0xc0, 0xbb, 0xfb, 0x56, 0x07, 0x1b, 0xa5, 0xdd, 0x01, 0xdc, 0x88, 0xe9, 0x09, 0xaf, 0x6b, 0xc6,
+	0x31, 0xf5, 0x65, 0x5f, 0xbb, 0x8d, 0x43, 0xfe, 0x5f, 0xc4, 0xe9, 0x17, 0xdb, 0x27, 0x41, 0x7a,
+	0x3a, 0x3b, 0x62, 0x5c, 0xb1, 0xa3, 0x34, 0x77, 0x84, 0xe6, 0x23, 0xf9, 0x1b, 0xcd, 0xfc, 0xc7,
+	0x3b, 0x27, 0xb1, 0xc4, 0x8e, 0x34, 0x0e, 0x3e, 0xfd, 0x57, 0x00, 0x00, 0x00, 0xff, 0xff, 0x7b,
+	0x77, 0x4a, 0x8d, 0x3d, 0x1a, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/events.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/events.pb.go
new file mode 100644
index 0000000..b61c610
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/events.pb.go
@@ -0,0 +1,1237 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/events.proto
+
+package voltha
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	timestamp "github.com/golang/protobuf/ptypes/timestamp"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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 ConfigEventType_Types int32
+
+const (
+	ConfigEventType_add    ConfigEventType_Types = 0
+	ConfigEventType_remove ConfigEventType_Types = 1
+	ConfigEventType_update ConfigEventType_Types = 2
+)
+
+var ConfigEventType_Types_name = map[int32]string{
+	0: "add",
+	1: "remove",
+	2: "update",
+}
+
+var ConfigEventType_Types_value = map[string]int32{
+	"add":    0,
+	"remove": 1,
+	"update": 2,
+}
+
+func (x ConfigEventType_Types) String() string {
+	return proto.EnumName(ConfigEventType_Types_name, int32(x))
+}
+
+func (ConfigEventType_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{0, 0}
+}
+
+type KpiEventType_Types int32
+
+const (
+	KpiEventType_slice KpiEventType_Types = 0
+	KpiEventType_ts    KpiEventType_Types = 1
+)
+
+var KpiEventType_Types_name = map[int32]string{
+	0: "slice",
+	1: "ts",
+}
+
+var KpiEventType_Types_value = map[string]int32{
+	"slice": 0,
+	"ts":    1,
+}
+
+func (x KpiEventType_Types) String() string {
+	return proto.EnumName(KpiEventType_Types_name, int32(x))
+}
+
+func (KpiEventType_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{2, 0}
+}
+
+type EventCategory_Types int32
+
+const (
+	EventCategory_COMMUNICATION EventCategory_Types = 0
+	EventCategory_ENVIRONMENT   EventCategory_Types = 1
+	EventCategory_EQUIPMENT     EventCategory_Types = 2
+	EventCategory_SERVICE       EventCategory_Types = 3
+	EventCategory_PROCESSING    EventCategory_Types = 4
+	EventCategory_SECURITY      EventCategory_Types = 5
+)
+
+var EventCategory_Types_name = map[int32]string{
+	0: "COMMUNICATION",
+	1: "ENVIRONMENT",
+	2: "EQUIPMENT",
+	3: "SERVICE",
+	4: "PROCESSING",
+	5: "SECURITY",
+}
+
+var EventCategory_Types_value = map[string]int32{
+	"COMMUNICATION": 0,
+	"ENVIRONMENT":   1,
+	"EQUIPMENT":     2,
+	"SERVICE":       3,
+	"PROCESSING":    4,
+	"SECURITY":      5,
+}
+
+func (x EventCategory_Types) String() string {
+	return proto.EnumName(EventCategory_Types_name, int32(x))
+}
+
+func (EventCategory_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{10, 0}
+}
+
+type EventSubCategory_Types int32
+
+const (
+	EventSubCategory_PON  EventSubCategory_Types = 0
+	EventSubCategory_OLT  EventSubCategory_Types = 1
+	EventSubCategory_ONT  EventSubCategory_Types = 2
+	EventSubCategory_ONU  EventSubCategory_Types = 3
+	EventSubCategory_NNI  EventSubCategory_Types = 4
+	EventSubCategory_NONE EventSubCategory_Types = 5
+)
+
+var EventSubCategory_Types_name = map[int32]string{
+	0: "PON",
+	1: "OLT",
+	2: "ONT",
+	3: "ONU",
+	4: "NNI",
+	5: "NONE",
+}
+
+var EventSubCategory_Types_value = map[string]int32{
+	"PON":  0,
+	"OLT":  1,
+	"ONT":  2,
+	"ONU":  3,
+	"NNI":  4,
+	"NONE": 5,
+}
+
+func (x EventSubCategory_Types) String() string {
+	return proto.EnumName(EventSubCategory_Types_name, int32(x))
+}
+
+func (EventSubCategory_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{11, 0}
+}
+
+type EventType_Types int32
+
+const (
+	EventType_CONFIG_EVENT EventType_Types = 0
+	EventType_KPI_EVENT    EventType_Types = 1
+	EventType_KPI_EVENT2   EventType_Types = 2
+	EventType_DEVICE_EVENT EventType_Types = 3
+	EventType_RPC_EVENT    EventType_Types = 4
+)
+
+var EventType_Types_name = map[int32]string{
+	0: "CONFIG_EVENT",
+	1: "KPI_EVENT",
+	2: "KPI_EVENT2",
+	3: "DEVICE_EVENT",
+	4: "RPC_EVENT",
+}
+
+var EventType_Types_value = map[string]int32{
+	"CONFIG_EVENT": 0,
+	"KPI_EVENT":    1,
+	"KPI_EVENT2":   2,
+	"DEVICE_EVENT": 3,
+	"RPC_EVENT":    4,
+}
+
+func (x EventType_Types) String() string {
+	return proto.EnumName(EventType_Types_name, int32(x))
+}
+
+func (EventType_Types) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{12, 0}
+}
+
+type ConfigEventType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ConfigEventType) Reset()         { *m = ConfigEventType{} }
+func (m *ConfigEventType) String() string { return proto.CompactTextString(m) }
+func (*ConfigEventType) ProtoMessage()    {}
+func (*ConfigEventType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{0}
+}
+
+func (m *ConfigEventType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ConfigEventType.Unmarshal(m, b)
+}
+func (m *ConfigEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ConfigEventType.Marshal(b, m, deterministic)
+}
+func (m *ConfigEventType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ConfigEventType.Merge(m, src)
+}
+func (m *ConfigEventType) XXX_Size() int {
+	return xxx_messageInfo_ConfigEventType.Size(m)
+}
+func (m *ConfigEventType) XXX_DiscardUnknown() {
+	xxx_messageInfo_ConfigEventType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ConfigEventType proto.InternalMessageInfo
+
+type ConfigEvent struct {
+	Type                 ConfigEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.ConfigEventType_Types" json:"type,omitempty"`
+	Hash                 string                `protobuf:"bytes,2,opt,name=hash,proto3" json:"hash,omitempty"`
+	Data                 string                `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *ConfigEvent) Reset()         { *m = ConfigEvent{} }
+func (m *ConfigEvent) String() string { return proto.CompactTextString(m) }
+func (*ConfigEvent) ProtoMessage()    {}
+func (*ConfigEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{1}
+}
+
+func (m *ConfigEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ConfigEvent.Unmarshal(m, b)
+}
+func (m *ConfigEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ConfigEvent.Marshal(b, m, deterministic)
+}
+func (m *ConfigEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ConfigEvent.Merge(m, src)
+}
+func (m *ConfigEvent) XXX_Size() int {
+	return xxx_messageInfo_ConfigEvent.Size(m)
+}
+func (m *ConfigEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_ConfigEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ConfigEvent proto.InternalMessageInfo
+
+func (m *ConfigEvent) GetType() ConfigEventType_Types {
+	if m != nil {
+		return m.Type
+	}
+	return ConfigEventType_add
+}
+
+func (m *ConfigEvent) GetHash() string {
+	if m != nil {
+		return m.Hash
+	}
+	return ""
+}
+
+func (m *ConfigEvent) GetData() string {
+	if m != nil {
+		return m.Data
+	}
+	return ""
+}
+
+type KpiEventType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *KpiEventType) Reset()         { *m = KpiEventType{} }
+func (m *KpiEventType) String() string { return proto.CompactTextString(m) }
+func (*KpiEventType) ProtoMessage()    {}
+func (*KpiEventType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{2}
+}
+
+func (m *KpiEventType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_KpiEventType.Unmarshal(m, b)
+}
+func (m *KpiEventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_KpiEventType.Marshal(b, m, deterministic)
+}
+func (m *KpiEventType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_KpiEventType.Merge(m, src)
+}
+func (m *KpiEventType) XXX_Size() int {
+	return xxx_messageInfo_KpiEventType.Size(m)
+}
+func (m *KpiEventType) XXX_DiscardUnknown() {
+	xxx_messageInfo_KpiEventType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_KpiEventType proto.InternalMessageInfo
+
+//
+// Struct to convey a dictionary of metric metadata.
+type MetricMetaData struct {
+	Title           string  `protobuf:"bytes,1,opt,name=title,proto3" json:"title,omitempty"`
+	Ts              float64 `protobuf:"fixed64,2,opt,name=ts,proto3" json:"ts,omitempty"`
+	LogicalDeviceId string  `protobuf:"bytes,3,opt,name=logical_device_id,json=logicalDeviceId,proto3" json:"logical_device_id,omitempty"`
+	// (equivalent to the DPID that ONOS has
+	// for the VOLTHA device without the
+	//  'of:' prefix
+	SerialNo             string            `protobuf:"bytes,4,opt,name=serial_no,json=serialNo,proto3" json:"serial_no,omitempty"`
+	DeviceId             string            `protobuf:"bytes,5,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	Context              map[string]string `protobuf:"bytes,6,rep,name=context,proto3" json:"context,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+	Uuid                 string            `protobuf:"bytes,7,opt,name=uuid,proto3" json:"uuid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *MetricMetaData) Reset()         { *m = MetricMetaData{} }
+func (m *MetricMetaData) String() string { return proto.CompactTextString(m) }
+func (*MetricMetaData) ProtoMessage()    {}
+func (*MetricMetaData) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{3}
+}
+
+func (m *MetricMetaData) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MetricMetaData.Unmarshal(m, b)
+}
+func (m *MetricMetaData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MetricMetaData.Marshal(b, m, deterministic)
+}
+func (m *MetricMetaData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MetricMetaData.Merge(m, src)
+}
+func (m *MetricMetaData) XXX_Size() int {
+	return xxx_messageInfo_MetricMetaData.Size(m)
+}
+func (m *MetricMetaData) XXX_DiscardUnknown() {
+	xxx_messageInfo_MetricMetaData.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MetricMetaData proto.InternalMessageInfo
+
+func (m *MetricMetaData) GetTitle() string {
+	if m != nil {
+		return m.Title
+	}
+	return ""
+}
+
+func (m *MetricMetaData) GetTs() float64 {
+	if m != nil {
+		return m.Ts
+	}
+	return 0
+}
+
+func (m *MetricMetaData) GetLogicalDeviceId() string {
+	if m != nil {
+		return m.LogicalDeviceId
+	}
+	return ""
+}
+
+func (m *MetricMetaData) GetSerialNo() string {
+	if m != nil {
+		return m.SerialNo
+	}
+	return ""
+}
+
+func (m *MetricMetaData) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *MetricMetaData) GetContext() map[string]string {
+	if m != nil {
+		return m.Context
+	}
+	return nil
+}
+
+func (m *MetricMetaData) GetUuid() string {
+	if m != nil {
+		return m.Uuid
+	}
+	return ""
+}
+
+//
+// Struct to convey a dictionary of metric->value pairs. Typically used in
+// pure shared-timestamp or shared-timestamp + shared object prefix situations.
+type MetricValuePairs struct {
+	// Metric / value pairs.
+	Metrics              map[string]float32 `protobuf:"bytes,1,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *MetricValuePairs) Reset()         { *m = MetricValuePairs{} }
+func (m *MetricValuePairs) String() string { return proto.CompactTextString(m) }
+func (*MetricValuePairs) ProtoMessage()    {}
+func (*MetricValuePairs) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{4}
+}
+
+func (m *MetricValuePairs) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MetricValuePairs.Unmarshal(m, b)
+}
+func (m *MetricValuePairs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MetricValuePairs.Marshal(b, m, deterministic)
+}
+func (m *MetricValuePairs) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MetricValuePairs.Merge(m, src)
+}
+func (m *MetricValuePairs) XXX_Size() int {
+	return xxx_messageInfo_MetricValuePairs.Size(m)
+}
+func (m *MetricValuePairs) XXX_DiscardUnknown() {
+	xxx_messageInfo_MetricValuePairs.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MetricValuePairs proto.InternalMessageInfo
+
+func (m *MetricValuePairs) GetMetrics() map[string]float32 {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+//
+// Struct to group metadata for a metric (or group of metrics) with the key-value
+// pairs of collected metrics
+type MetricInformation struct {
+	Metadata             *MetricMetaData    `protobuf:"bytes,1,opt,name=metadata,proto3" json:"metadata,omitempty"`
+	Metrics              map[string]float32 `protobuf:"bytes,2,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed32,2,opt,name=value,proto3"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *MetricInformation) Reset()         { *m = MetricInformation{} }
+func (m *MetricInformation) String() string { return proto.CompactTextString(m) }
+func (*MetricInformation) ProtoMessage()    {}
+func (*MetricInformation) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{5}
+}
+
+func (m *MetricInformation) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MetricInformation.Unmarshal(m, b)
+}
+func (m *MetricInformation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MetricInformation.Marshal(b, m, deterministic)
+}
+func (m *MetricInformation) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MetricInformation.Merge(m, src)
+}
+func (m *MetricInformation) XXX_Size() int {
+	return xxx_messageInfo_MetricInformation.Size(m)
+}
+func (m *MetricInformation) XXX_DiscardUnknown() {
+	xxx_messageInfo_MetricInformation.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MetricInformation proto.InternalMessageInfo
+
+func (m *MetricInformation) GetMetadata() *MetricMetaData {
+	if m != nil {
+		return m.Metadata
+	}
+	return nil
+}
+
+func (m *MetricInformation) GetMetrics() map[string]float32 {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+//
+// Legacy KPI Event structured.  In mid-August, the KPI event format was updated
+//                               to a more easily parsable format. See VOL-1140
+//                               for more information.
+type KpiEvent struct {
+	Type                 KpiEventType_Types           `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.KpiEventType_Types" json:"type,omitempty"`
+	Ts                   float32                      `protobuf:"fixed32,2,opt,name=ts,proto3" json:"ts,omitempty"`
+	Prefixes             map[string]*MetricValuePairs `protobuf:"bytes,3,rep,name=prefixes,proto3" json:"prefixes,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 *KpiEvent) Reset()         { *m = KpiEvent{} }
+func (m *KpiEvent) String() string { return proto.CompactTextString(m) }
+func (*KpiEvent) ProtoMessage()    {}
+func (*KpiEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{6}
+}
+
+func (m *KpiEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_KpiEvent.Unmarshal(m, b)
+}
+func (m *KpiEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_KpiEvent.Marshal(b, m, deterministic)
+}
+func (m *KpiEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_KpiEvent.Merge(m, src)
+}
+func (m *KpiEvent) XXX_Size() int {
+	return xxx_messageInfo_KpiEvent.Size(m)
+}
+func (m *KpiEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_KpiEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_KpiEvent proto.InternalMessageInfo
+
+func (m *KpiEvent) GetType() KpiEventType_Types {
+	if m != nil {
+		return m.Type
+	}
+	return KpiEventType_slice
+}
+
+func (m *KpiEvent) GetTs() float32 {
+	if m != nil {
+		return m.Ts
+	}
+	return 0
+}
+
+func (m *KpiEvent) GetPrefixes() map[string]*MetricValuePairs {
+	if m != nil {
+		return m.Prefixes
+	}
+	return nil
+}
+
+type KpiEvent2 struct {
+	// Type of KPI Event
+	Type KpiEventType_Types `protobuf:"varint,1,opt,name=type,proto3,enum=voltha.KpiEventType_Types" json:"type,omitempty"`
+	// Fields used when for slice:
+	Ts                   float64              `protobuf:"fixed64,2,opt,name=ts,proto3" json:"ts,omitempty"`
+	SliceData            []*MetricInformation `protobuf:"bytes,3,rep,name=slice_data,json=sliceData,proto3" json:"slice_data,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *KpiEvent2) Reset()         { *m = KpiEvent2{} }
+func (m *KpiEvent2) String() string { return proto.CompactTextString(m) }
+func (*KpiEvent2) ProtoMessage()    {}
+func (*KpiEvent2) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{7}
+}
+
+func (m *KpiEvent2) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_KpiEvent2.Unmarshal(m, b)
+}
+func (m *KpiEvent2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_KpiEvent2.Marshal(b, m, deterministic)
+}
+func (m *KpiEvent2) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_KpiEvent2.Merge(m, src)
+}
+func (m *KpiEvent2) XXX_Size() int {
+	return xxx_messageInfo_KpiEvent2.Size(m)
+}
+func (m *KpiEvent2) XXX_DiscardUnknown() {
+	xxx_messageInfo_KpiEvent2.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_KpiEvent2 proto.InternalMessageInfo
+
+func (m *KpiEvent2) GetType() KpiEventType_Types {
+	if m != nil {
+		return m.Type
+	}
+	return KpiEventType_slice
+}
+
+func (m *KpiEvent2) GetTs() float64 {
+	if m != nil {
+		return m.Ts
+	}
+	return 0
+}
+
+func (m *KpiEvent2) GetSliceData() []*MetricInformation {
+	if m != nil {
+		return m.SliceData
+	}
+	return nil
+}
+
+//
+// Describes the events specific to device
+type DeviceEvent struct {
+	// Identifier of the originating resource of the event, for ex: device_id
+	ResourceId string `protobuf:"bytes,1,opt,name=resource_id,json=resourceId,proto3" json:"resource_id,omitempty"`
+	// device_event_name indicates clearly the name of the device event
+	DeviceEventName string `protobuf:"bytes,2,opt,name=device_event_name,json=deviceEventName,proto3" json:"device_event_name,omitempty"`
+	// Textual explanation of the device event
+	Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"`
+	// Key/Value storage for extra information that may give context to the event
+	Context              map[string]string `protobuf:"bytes,4,rep,name=context,proto3" json:"context,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 *DeviceEvent) Reset()         { *m = DeviceEvent{} }
+func (m *DeviceEvent) String() string { return proto.CompactTextString(m) }
+func (*DeviceEvent) ProtoMessage()    {}
+func (*DeviceEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{8}
+}
+
+func (m *DeviceEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceEvent.Unmarshal(m, b)
+}
+func (m *DeviceEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceEvent.Marshal(b, m, deterministic)
+}
+func (m *DeviceEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceEvent.Merge(m, src)
+}
+func (m *DeviceEvent) XXX_Size() int {
+	return xxx_messageInfo_DeviceEvent.Size(m)
+}
+func (m *DeviceEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceEvent proto.InternalMessageInfo
+
+func (m *DeviceEvent) GetResourceId() string {
+	if m != nil {
+		return m.ResourceId
+	}
+	return ""
+}
+
+func (m *DeviceEvent) GetDeviceEventName() string {
+	if m != nil {
+		return m.DeviceEventName
+	}
+	return ""
+}
+
+func (m *DeviceEvent) GetDescription() string {
+	if m != nil {
+		return m.Description
+	}
+	return ""
+}
+
+func (m *DeviceEvent) GetContext() map[string]string {
+	if m != nil {
+		return m.Context
+	}
+	return nil
+}
+
+//
+// Describes the events specific to an RPC request
+type RPCEvent struct {
+	// RPC name
+	Rpc string `protobuf:"bytes,1,opt,name=rpc,proto3" json:"rpc,omitempty"`
+	// The operation id of that request.  Can be a log correlation ID
+	OperationId string `protobuf:"bytes,2,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"`
+	// Identifies the service name originating the event
+	Service string `protobuf:"bytes,3,opt,name=service,proto3" json:"service,omitempty"`
+	// Identifies the stack originating the event
+	StackId string `protobuf:"bytes,4,opt,name=stack_id,json=stackId,proto3" json:"stack_id,omitempty"`
+	// Identifies the resource upon which the action is taken, e.g. device_id
+	ResourceId string `protobuf:"bytes,5,opt,name=resource_id,json=resourceId,proto3" json:"resource_id,omitempty"`
+	// Textual explanation of the event
+	Description string `protobuf:"bytes,6,opt,name=description,proto3" json:"description,omitempty"`
+	// Key/Value storage for extra information that may give context to the event
+	Context map[string]string `protobuf:"bytes,7,rep,name=context,proto3" json:"context,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+	// Status of the RPC Event
+	Status               *common.OperationResp `protobuf:"bytes,8,opt,name=status,proto3" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
+}
+
+func (m *RPCEvent) Reset()         { *m = RPCEvent{} }
+func (m *RPCEvent) String() string { return proto.CompactTextString(m) }
+func (*RPCEvent) ProtoMessage()    {}
+func (*RPCEvent) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{9}
+}
+
+func (m *RPCEvent) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_RPCEvent.Unmarshal(m, b)
+}
+func (m *RPCEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_RPCEvent.Marshal(b, m, deterministic)
+}
+func (m *RPCEvent) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RPCEvent.Merge(m, src)
+}
+func (m *RPCEvent) XXX_Size() int {
+	return xxx_messageInfo_RPCEvent.Size(m)
+}
+func (m *RPCEvent) XXX_DiscardUnknown() {
+	xxx_messageInfo_RPCEvent.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_RPCEvent proto.InternalMessageInfo
+
+func (m *RPCEvent) GetRpc() string {
+	if m != nil {
+		return m.Rpc
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetOperationId() string {
+	if m != nil {
+		return m.OperationId
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetService() string {
+	if m != nil {
+		return m.Service
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetStackId() string {
+	if m != nil {
+		return m.StackId
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetResourceId() string {
+	if m != nil {
+		return m.ResourceId
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetDescription() string {
+	if m != nil {
+		return m.Description
+	}
+	return ""
+}
+
+func (m *RPCEvent) GetContext() map[string]string {
+	if m != nil {
+		return m.Context
+	}
+	return nil
+}
+
+func (m *RPCEvent) GetStatus() *common.OperationResp {
+	if m != nil {
+		return m.Status
+	}
+	return nil
+}
+
+//
+// Identify the area of the system impacted by the event.
+type EventCategory struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EventCategory) Reset()         { *m = EventCategory{} }
+func (m *EventCategory) String() string { return proto.CompactTextString(m) }
+func (*EventCategory) ProtoMessage()    {}
+func (*EventCategory) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{10}
+}
+
+func (m *EventCategory) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventCategory.Unmarshal(m, b)
+}
+func (m *EventCategory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventCategory.Marshal(b, m, deterministic)
+}
+func (m *EventCategory) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventCategory.Merge(m, src)
+}
+func (m *EventCategory) XXX_Size() int {
+	return xxx_messageInfo_EventCategory.Size(m)
+}
+func (m *EventCategory) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventCategory.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventCategory proto.InternalMessageInfo
+
+//
+// Identify the functional category originating the event
+type EventSubCategory struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EventSubCategory) Reset()         { *m = EventSubCategory{} }
+func (m *EventSubCategory) String() string { return proto.CompactTextString(m) }
+func (*EventSubCategory) ProtoMessage()    {}
+func (*EventSubCategory) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{11}
+}
+
+func (m *EventSubCategory) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventSubCategory.Unmarshal(m, b)
+}
+func (m *EventSubCategory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventSubCategory.Marshal(b, m, deterministic)
+}
+func (m *EventSubCategory) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventSubCategory.Merge(m, src)
+}
+func (m *EventSubCategory) XXX_Size() int {
+	return xxx_messageInfo_EventSubCategory.Size(m)
+}
+func (m *EventSubCategory) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventSubCategory.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventSubCategory proto.InternalMessageInfo
+
+//
+// Identify the type of event
+type EventType struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EventType) Reset()         { *m = EventType{} }
+func (m *EventType) String() string { return proto.CompactTextString(m) }
+func (*EventType) ProtoMessage()    {}
+func (*EventType) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{12}
+}
+
+func (m *EventType) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventType.Unmarshal(m, b)
+}
+func (m *EventType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventType.Marshal(b, m, deterministic)
+}
+func (m *EventType) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventType.Merge(m, src)
+}
+func (m *EventType) XXX_Size() int {
+	return xxx_messageInfo_EventType.Size(m)
+}
+func (m *EventType) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventType.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventType proto.InternalMessageInfo
+
+//
+// Identify the functional category originating the event
+type EventHeader struct {
+	// Unique ID for this event.  e.g. voltha.some_olt.1234
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Refers to the functional area affect by the event
+	Category EventCategory_Types `protobuf:"varint,2,opt,name=category,proto3,enum=voltha.EventCategory_Types" json:"category,omitempty"`
+	// Refers to functional category of the event
+	SubCategory EventSubCategory_Types `protobuf:"varint,3,opt,name=sub_category,json=subCategory,proto3,enum=voltha.EventSubCategory_Types" json:"sub_category,omitempty"`
+	// Refers to the type of the event
+	Type EventType_Types `protobuf:"varint,4,opt,name=type,proto3,enum=voltha.EventType_Types" json:"type,omitempty"`
+	// The version identifier for this event type, thus allowing each
+	// event type to evolve independently. The version should be in the
+	// format “MAJOR.MINOR” format and minor changes must only be additive
+	// and non-breaking.
+	TypeVersion string `protobuf:"bytes,5,opt,name=type_version,json=typeVersion,proto3" json:"type_version,omitempty"`
+	// Timestamp at which the event was first raised.
+	// This represents the UTC time stamp since epoch (in seconds) when the
+	// the event was first raised from the source entity.
+	// If the source entity doesn't send the raised_ts, this shall be set
+	// to timestamp when the event was received.
+	RaisedTs *timestamp.Timestamp `protobuf:"bytes,6,opt,name=raised_ts,json=raisedTs,proto3" json:"raised_ts,omitempty"`
+	// Timestamp at which the event was reported.
+	// This represents the UTC time stamp since epoch (in seconds) when the
+	// the event was reported (this time stamp is >= raised_ts).
+	// If the source entity that reported this event doesn't send the
+	// reported_ts, this shall be set to the same value as raised_ts.
+	ReportedTs           *timestamp.Timestamp `protobuf:"bytes,7,opt,name=reported_ts,json=reportedTs,proto3" json:"reported_ts,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
+}
+
+func (m *EventHeader) Reset()         { *m = EventHeader{} }
+func (m *EventHeader) String() string { return proto.CompactTextString(m) }
+func (*EventHeader) ProtoMessage()    {}
+func (*EventHeader) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{13}
+}
+
+func (m *EventHeader) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventHeader.Unmarshal(m, b)
+}
+func (m *EventHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventHeader.Marshal(b, m, deterministic)
+}
+func (m *EventHeader) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventHeader.Merge(m, src)
+}
+func (m *EventHeader) XXX_Size() int {
+	return xxx_messageInfo_EventHeader.Size(m)
+}
+func (m *EventHeader) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventHeader.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventHeader proto.InternalMessageInfo
+
+func (m *EventHeader) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *EventHeader) GetCategory() EventCategory_Types {
+	if m != nil {
+		return m.Category
+	}
+	return EventCategory_COMMUNICATION
+}
+
+func (m *EventHeader) GetSubCategory() EventSubCategory_Types {
+	if m != nil {
+		return m.SubCategory
+	}
+	return EventSubCategory_PON
+}
+
+func (m *EventHeader) GetType() EventType_Types {
+	if m != nil {
+		return m.Type
+	}
+	return EventType_CONFIG_EVENT
+}
+
+func (m *EventHeader) GetTypeVersion() string {
+	if m != nil {
+		return m.TypeVersion
+	}
+	return ""
+}
+
+func (m *EventHeader) GetRaisedTs() *timestamp.Timestamp {
+	if m != nil {
+		return m.RaisedTs
+	}
+	return nil
+}
+
+func (m *EventHeader) GetReportedTs() *timestamp.Timestamp {
+	if m != nil {
+		return m.ReportedTs
+	}
+	return nil
+}
+
+//
+// Event Structure
+type Event struct {
+	// event header
+	Header *EventHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
+	// oneof event types referred by EventType.
+	//
+	// Types that are valid to be assigned to EventType:
+	//	*Event_ConfigEvent
+	//	*Event_KpiEvent
+	//	*Event_KpiEvent2
+	//	*Event_DeviceEvent
+	//	*Event_RpcEvent
+	EventType            isEvent_EventType `protobuf_oneof:"event_type"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *Event) Reset()         { *m = Event{} }
+func (m *Event) String() string { return proto.CompactTextString(m) }
+func (*Event) ProtoMessage()    {}
+func (*Event) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e63e6c07044fd2c4, []int{14}
+}
+
+func (m *Event) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Event.Unmarshal(m, b)
+}
+func (m *Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Event.Marshal(b, m, deterministic)
+}
+func (m *Event) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Event.Merge(m, src)
+}
+func (m *Event) XXX_Size() int {
+	return xxx_messageInfo_Event.Size(m)
+}
+func (m *Event) XXX_DiscardUnknown() {
+	xxx_messageInfo_Event.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Event proto.InternalMessageInfo
+
+func (m *Event) GetHeader() *EventHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type isEvent_EventType interface {
+	isEvent_EventType()
+}
+
+type Event_ConfigEvent struct {
+	ConfigEvent *ConfigEvent `protobuf:"bytes,2,opt,name=config_event,json=configEvent,proto3,oneof"`
+}
+
+type Event_KpiEvent struct {
+	KpiEvent *KpiEvent `protobuf:"bytes,3,opt,name=kpi_event,json=kpiEvent,proto3,oneof"`
+}
+
+type Event_KpiEvent2 struct {
+	KpiEvent2 *KpiEvent2 `protobuf:"bytes,4,opt,name=kpi_event2,json=kpiEvent2,proto3,oneof"`
+}
+
+type Event_DeviceEvent struct {
+	DeviceEvent *DeviceEvent `protobuf:"bytes,5,opt,name=device_event,json=deviceEvent,proto3,oneof"`
+}
+
+type Event_RpcEvent struct {
+	RpcEvent *RPCEvent `protobuf:"bytes,6,opt,name=rpc_event,json=rpcEvent,proto3,oneof"`
+}
+
+func (*Event_ConfigEvent) isEvent_EventType() {}
+
+func (*Event_KpiEvent) isEvent_EventType() {}
+
+func (*Event_KpiEvent2) isEvent_EventType() {}
+
+func (*Event_DeviceEvent) isEvent_EventType() {}
+
+func (*Event_RpcEvent) isEvent_EventType() {}
+
+func (m *Event) GetEventType() isEvent_EventType {
+	if m != nil {
+		return m.EventType
+	}
+	return nil
+}
+
+func (m *Event) GetConfigEvent() *ConfigEvent {
+	if x, ok := m.GetEventType().(*Event_ConfigEvent); ok {
+		return x.ConfigEvent
+	}
+	return nil
+}
+
+func (m *Event) GetKpiEvent() *KpiEvent {
+	if x, ok := m.GetEventType().(*Event_KpiEvent); ok {
+		return x.KpiEvent
+	}
+	return nil
+}
+
+func (m *Event) GetKpiEvent2() *KpiEvent2 {
+	if x, ok := m.GetEventType().(*Event_KpiEvent2); ok {
+		return x.KpiEvent2
+	}
+	return nil
+}
+
+func (m *Event) GetDeviceEvent() *DeviceEvent {
+	if x, ok := m.GetEventType().(*Event_DeviceEvent); ok {
+		return x.DeviceEvent
+	}
+	return nil
+}
+
+func (m *Event) GetRpcEvent() *RPCEvent {
+	if x, ok := m.GetEventType().(*Event_RpcEvent); ok {
+		return x.RpcEvent
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*Event) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*Event_ConfigEvent)(nil),
+		(*Event_KpiEvent)(nil),
+		(*Event_KpiEvent2)(nil),
+		(*Event_DeviceEvent)(nil),
+		(*Event_RpcEvent)(nil),
+	}
+}
+
+func init() {
+	proto.RegisterEnum("voltha.ConfigEventType_Types", ConfigEventType_Types_name, ConfigEventType_Types_value)
+	proto.RegisterEnum("voltha.KpiEventType_Types", KpiEventType_Types_name, KpiEventType_Types_value)
+	proto.RegisterEnum("voltha.EventCategory_Types", EventCategory_Types_name, EventCategory_Types_value)
+	proto.RegisterEnum("voltha.EventSubCategory_Types", EventSubCategory_Types_name, EventSubCategory_Types_value)
+	proto.RegisterEnum("voltha.EventType_Types", EventType_Types_name, EventType_Types_value)
+	proto.RegisterType((*ConfigEventType)(nil), "voltha.ConfigEventType")
+	proto.RegisterType((*ConfigEvent)(nil), "voltha.ConfigEvent")
+	proto.RegisterType((*KpiEventType)(nil), "voltha.KpiEventType")
+	proto.RegisterType((*MetricMetaData)(nil), "voltha.MetricMetaData")
+	proto.RegisterMapType((map[string]string)(nil), "voltha.MetricMetaData.ContextEntry")
+	proto.RegisterType((*MetricValuePairs)(nil), "voltha.MetricValuePairs")
+	proto.RegisterMapType((map[string]float32)(nil), "voltha.MetricValuePairs.MetricsEntry")
+	proto.RegisterType((*MetricInformation)(nil), "voltha.MetricInformation")
+	proto.RegisterMapType((map[string]float32)(nil), "voltha.MetricInformation.MetricsEntry")
+	proto.RegisterType((*KpiEvent)(nil), "voltha.KpiEvent")
+	proto.RegisterMapType((map[string]*MetricValuePairs)(nil), "voltha.KpiEvent.PrefixesEntry")
+	proto.RegisterType((*KpiEvent2)(nil), "voltha.KpiEvent2")
+	proto.RegisterType((*DeviceEvent)(nil), "voltha.DeviceEvent")
+	proto.RegisterMapType((map[string]string)(nil), "voltha.DeviceEvent.ContextEntry")
+	proto.RegisterType((*RPCEvent)(nil), "voltha.RPCEvent")
+	proto.RegisterMapType((map[string]string)(nil), "voltha.RPCEvent.ContextEntry")
+	proto.RegisterType((*EventCategory)(nil), "voltha.EventCategory")
+	proto.RegisterType((*EventSubCategory)(nil), "voltha.EventSubCategory")
+	proto.RegisterType((*EventType)(nil), "voltha.EventType")
+	proto.RegisterType((*EventHeader)(nil), "voltha.EventHeader")
+	proto.RegisterType((*Event)(nil), "voltha.Event")
+}
+
+func init() { proto.RegisterFile("voltha_protos/events.proto", fileDescriptor_e63e6c07044fd2c4) }
+
+var fileDescriptor_e63e6c07044fd2c4 = []byte{
+	// 1282 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xdb, 0x6e, 0xdb, 0x46,
+	0x13, 0x16, 0xa9, 0xf3, 0x50, 0xb6, 0xe9, 0xcd, 0xff, 0xb7, 0x8a, 0xd2, 0x26, 0x8e, 0x8a, 0x16,
+	0x46, 0x82, 0x50, 0x28, 0x8b, 0x22, 0x86, 0x83, 0x1e, 0x12, 0x85, 0x8d, 0x89, 0xd4, 0x94, 0x4a,
+	0xcb, 0x06, 0xd2, 0x1b, 0x61, 0x4d, 0xae, 0x65, 0xc2, 0x92, 0x48, 0x70, 0x57, 0x42, 0xfc, 0x00,
+	0xbd, 0xee, 0x83, 0xf4, 0x39, 0x7a, 0xd7, 0x37, 0x28, 0x8a, 0xbe, 0x44, 0x1f, 0xa0, 0xd8, 0x03,
+	0x25, 0x52, 0x71, 0x90, 0x0b, 0xa3, 0x57, 0xdc, 0x9d, 0x9d, 0x6f, 0xe7, 0x9b, 0x8f, 0x3b, 0xb3,
+	0x0b, 0x9d, 0x65, 0x3c, 0x65, 0x97, 0x78, 0x9c, 0xa4, 0x31, 0x8b, 0x69, 0x8f, 0x2c, 0xc9, 0x9c,
+	0x51, 0x4b, 0xcc, 0x50, 0x4d, 0xae, 0x75, 0xda, 0x45, 0x9f, 0x19, 0x61, 0x58, 0x7a, 0x74, 0x3e,
+	0x99, 0xc4, 0xf1, 0x64, 0x4a, 0x7a, 0x38, 0x89, 0x7a, 0x78, 0x3e, 0x8f, 0x19, 0x66, 0x51, 0x3c,
+	0x57, 0xf8, 0xce, 0x03, 0xb5, 0x2a, 0x66, 0xe7, 0x8b, 0x8b, 0x1e, 0x8b, 0x66, 0x84, 0x32, 0x3c,
+	0x4b, 0x94, 0xc3, 0x46, 0xf0, 0x20, 0x9e, 0xcd, 0xe2, 0xb9, 0x5c, 0xeb, 0x3e, 0x83, 0x9d, 0x7e,
+	0x3c, 0xbf, 0x88, 0x26, 0x0e, 0xa7, 0x34, 0xba, 0x4e, 0x48, 0x77, 0x1f, 0xaa, 0xfc, 0x4b, 0x51,
+	0x1d, 0xca, 0x38, 0x0c, 0xcd, 0x12, 0x02, 0xa8, 0xa5, 0x64, 0x16, 0x2f, 0x89, 0xa9, 0xf1, 0xf1,
+	0x22, 0x09, 0x31, 0x23, 0xa6, 0xde, 0xbd, 0x04, 0x23, 0x07, 0x46, 0x5f, 0x42, 0x85, 0x5d, 0x27,
+	0xa4, 0xad, 0xed, 0x69, 0xfb, 0xdb, 0xf6, 0xa7, 0x96, 0x0c, 0x6b, 0x6d, 0xec, 0x6f, 0x89, 0xcd,
+	0x7d, 0xe1, 0x8a, 0x10, 0x54, 0x2e, 0x31, 0xbd, 0x6c, 0xeb, 0x7b, 0xda, 0x7e, 0xd3, 0x17, 0x63,
+	0x6e, 0x0b, 0x31, 0xc3, 0xed, 0xb2, 0xb4, 0xf1, 0x71, 0xf7, 0x11, 0xb4, 0x5e, 0x27, 0xd1, 0x9a,
+	0x63, 0x27, 0xe3, 0xd8, 0x84, 0x2a, 0x9d, 0x46, 0x01, 0x31, 0x4b, 0xa8, 0x06, 0x3a, 0xa3, 0xa6,
+	0xd6, 0xfd, 0x4d, 0x87, 0xed, 0x63, 0xc2, 0xd2, 0x28, 0x38, 0x26, 0x0c, 0xbf, 0xc4, 0x0c, 0xa3,
+	0xff, 0x41, 0x95, 0x45, 0x6c, 0x2a, 0xa9, 0x35, 0x7d, 0x39, 0x41, 0xdb, 0x1c, 0x20, 0x42, 0x6b,
+	0xbe, 0xce, 0x28, 0x7a, 0x04, 0xbb, 0xd3, 0x78, 0x12, 0x05, 0x78, 0x3a, 0x0e, 0xc9, 0x32, 0x0a,
+	0xc8, 0x38, 0x0a, 0x15, 0x8b, 0x1d, 0xb5, 0xf0, 0x52, 0xd8, 0xdd, 0x10, 0xdd, 0x83, 0x26, 0x25,
+	0x69, 0x84, 0xa7, 0xe3, 0x79, 0xdc, 0xae, 0x08, 0x9f, 0x86, 0x34, 0x78, 0x31, 0x5f, 0x5c, 0x6f,
+	0x50, 0x95, 0x8b, 0x61, 0x86, 0xfc, 0x06, 0xea, 0x41, 0x3c, 0x67, 0xe4, 0x2d, 0x6b, 0xd7, 0xf6,
+	0xca, 0xfb, 0x86, 0xfd, 0x59, 0x26, 0x54, 0x91, 0x34, 0xd7, 0x8d, 0x7b, 0x39, 0x73, 0x96, 0x5e,
+	0xfb, 0x19, 0x86, 0xab, 0xb3, 0x58, 0x44, 0x61, 0xbb, 0x2e, 0xd5, 0xe1, 0xe3, 0xce, 0x21, 0xb4,
+	0xf2, 0xce, 0xc8, 0x84, 0xf2, 0x15, 0xb9, 0x56, 0xc9, 0xf2, 0x21, 0x17, 0x60, 0x89, 0xa7, 0x0b,
+	0xa2, 0x84, 0x96, 0x93, 0x43, 0xfd, 0x40, 0xeb, 0xfe, 0xaa, 0x81, 0x29, 0x03, 0x9f, 0x71, 0xdb,
+	0x10, 0x47, 0x29, 0x45, 0xdf, 0x41, 0x7d, 0x26, 0x6c, 0xb4, 0xad, 0x09, 0x8e, 0x9f, 0x17, 0x39,
+	0xae, 0x5d, 0x95, 0x81, 0x2a, 0x96, 0x0a, 0xc5, 0x19, 0xe5, 0x17, 0x3e, 0xc4, 0x48, 0xcf, 0x33,
+	0xfa, 0x5d, 0x83, 0x5d, 0x09, 0x76, 0xe7, 0x17, 0x71, 0x3a, 0x13, 0x87, 0x1d, 0xd9, 0xd0, 0xe0,
+	0x15, 0x21, 0x4e, 0x06, 0xdf, 0xc6, 0xb0, 0x3f, 0xba, 0x59, 0x37, 0x7f, 0xe5, 0x87, 0xbe, 0x5f,
+	0xa7, 0xa1, 0x8b, 0x34, 0xbe, 0x28, 0x42, 0x72, 0xfb, 0xff, 0x07, 0x79, 0xfc, 0xa5, 0x41, 0x23,
+	0x3b, 0xb4, 0xc8, 0x2a, 0xd4, 0x46, 0x27, 0xe3, 0x91, 0x3f, 0xd4, 0x85, 0xc2, 0x58, 0x9f, 0x4d,
+	0x5d, 0x9c, 0xcd, 0x43, 0x68, 0x24, 0x29, 0xb9, 0x88, 0xde, 0x12, 0xda, 0x2e, 0x8b, 0x5c, 0xee,
+	0x6f, 0xee, 0x61, 0x0d, 0x95, 0x83, 0xcc, 0x61, 0xe5, 0xdf, 0x39, 0x85, 0xad, 0xc2, 0xd2, 0x0d,
+	0x59, 0x58, 0xf9, 0x2c, 0x0c, 0xbb, 0xfd, 0xbe, 0xdf, 0x9d, 0xcf, 0xef, 0x17, 0x0d, 0x9a, 0x59,
+	0x6c, 0xfb, 0x16, 0x09, 0xca, 0xe2, 0x3b, 0x00, 0x10, 0x85, 0x3c, 0x56, 0xb5, 0xcf, 0x53, 0xbc,
+	0xfb, 0xde, 0xdf, 0xe5, 0x37, 0x85, 0x33, 0xff, 0xdf, 0xdd, 0x7f, 0x34, 0x30, 0x64, 0x5d, 0x4a,
+	0xa9, 0x1f, 0x80, 0x91, 0x12, 0x1a, 0x2f, 0x52, 0x59, 0x7f, 0x32, 0x4b, 0xc8, 0x4c, 0x6e, 0xc8,
+	0xeb, 0x5c, 0x95, 0xa7, 0xe8, 0xc3, 0xe3, 0x39, 0x9e, 0x65, 0x85, 0xb1, 0x13, 0xae, 0x37, 0xf2,
+	0xf0, 0x8c, 0xa0, 0x3d, 0x30, 0x42, 0x42, 0x83, 0x34, 0x4a, 0x78, 0x58, 0xd5, 0x0d, 0xf2, 0x26,
+	0x74, 0xb8, 0xae, 0xe7, 0x8a, 0x60, 0xbd, 0x97, 0xb1, 0xce, 0x91, 0xba, 0xb9, 0x98, 0x6f, 0x55,
+	0xb8, 0x7f, 0xea, 0xd0, 0xf0, 0x87, 0x7d, 0x99, 0xb3, 0x09, 0xe5, 0x34, 0x09, 0x32, 0x60, 0x9a,
+	0x04, 0xe8, 0x21, 0xb4, 0xe2, 0x84, 0xa4, 0x42, 0x2d, 0x2e, 0x83, 0xc4, 0x1b, 0x2b, 0x9b, 0x1b,
+	0xa2, 0x36, 0xd4, 0x29, 0x49, 0x39, 0x47, 0x95, 0x57, 0x36, 0x45, 0x77, 0xa1, 0x41, 0x19, 0x0e,
+	0xae, 0x38, 0xb0, 0xa2, 0x96, 0xf8, 0xdc, 0x0d, 0x37, 0xd5, 0xad, 0xbe, 0xa3, 0xee, 0x86, 0x62,
+	0xb5, 0x77, 0x15, 0x7b, 0xba, 0x56, 0xac, 0x2e, 0x14, 0x5b, 0x5d, 0x15, 0x59, 0x3e, 0xef, 0xe9,
+	0x7d, 0x4f, 0xa0, 0x46, 0x19, 0x66, 0x0b, 0xda, 0x6e, 0x88, 0x63, 0xfa, 0x7f, 0x4b, 0xdd, 0x65,
+	0x83, 0x2c, 0x2b, 0x9f, 0xd0, 0xc4, 0x57, 0x4e, 0xb7, 0x52, 0x77, 0x09, 0x5b, 0x82, 0x49, 0x1f,
+	0x33, 0x32, 0x89, 0xd3, 0xeb, 0x2e, 0xc9, 0x6e, 0x9c, 0x5d, 0xd8, 0xea, 0x0f, 0x8e, 0x8f, 0x4f,
+	0x3d, 0xb7, 0xff, 0x7c, 0xe4, 0x0e, 0x3c, 0xb3, 0x84, 0x76, 0xc0, 0x70, 0xbc, 0x33, 0xd7, 0x1f,
+	0x78, 0xc7, 0x8e, 0x37, 0x32, 0x35, 0xb4, 0x05, 0x4d, 0xe7, 0xa7, 0x53, 0x77, 0x28, 0xa6, 0x3a,
+	0x32, 0xa0, 0x7e, 0xe2, 0xf8, 0x67, 0x6e, 0xdf, 0x31, 0xcb, 0x68, 0x1b, 0x60, 0xe8, 0x0f, 0xfa,
+	0xce, 0xc9, 0x89, 0xeb, 0xbd, 0x32, 0x2b, 0xa8, 0x05, 0x8d, 0x13, 0xa7, 0x7f, 0xea, 0xbb, 0xa3,
+	0x37, 0x66, 0xb5, 0xeb, 0x83, 0x29, 0xe2, 0x9e, 0x2c, 0xce, 0x57, 0xa1, 0xbf, 0xcd, 0x5d, 0xc8,
+	0x43, 0x11, 0xb0, 0x0e, 0xe5, 0xc1, 0x8f, 0x3c, 0x10, 0x1f, 0x88, 0x10, 0x62, 0x70, 0x6a, 0x96,
+	0xf9, 0xc0, 0xf3, 0x5c, 0xb3, 0x82, 0x1a, 0x50, 0xf1, 0x06, 0x9e, 0x63, 0x56, 0xbb, 0x17, 0xd0,
+	0x5c, 0xdf, 0x9c, 0x6f, 0xb2, 0xcd, 0x4c, 0x68, 0xf5, 0x07, 0xde, 0x0f, 0xee, 0xab, 0xb1, 0x73,
+	0xc6, 0x69, 0x96, 0x38, 0xeb, 0xd7, 0x43, 0x57, 0x4d, 0x35, 0x4e, 0x74, 0x35, 0xb5, 0x4d, 0x9d,
+	0x03, 0x5e, 0x3a, 0x3c, 0x09, 0xe5, 0x51, 0xe6, 0x00, 0x7f, 0xd8, 0x57, 0xd3, 0x4a, 0xf7, 0x6f,
+	0x1d, 0x0c, 0x11, 0xe8, 0x88, 0xe0, 0x90, 0xa4, 0xbc, 0xc4, 0x57, 0xf5, 0xa7, 0x47, 0x21, 0x7a,
+	0x0a, 0x8d, 0x40, 0xe5, 0x24, 0x04, 0xdf, 0xb6, 0xef, 0x65, 0x3f, 0xbe, 0xa0, 0xb5, 0xea, 0x13,
+	0x2b, 0x67, 0xf4, 0x1c, 0x5a, 0x74, 0x71, 0x3e, 0x5e, 0x81, 0xcb, 0x02, 0x7c, 0xbf, 0x00, 0xce,
+	0x09, 0xa6, 0xf0, 0x06, 0x5d, 0x9b, 0xd0, 0x63, 0xd5, 0x9e, 0x2a, 0x02, 0xfa, 0x71, 0x01, 0xfa,
+	0x4e, 0x6f, 0x7a, 0x08, 0x2d, 0xfe, 0x1d, 0x2f, 0x49, 0x4a, 0xf9, 0x19, 0x96, 0x87, 0xdc, 0xe0,
+	0xb6, 0x33, 0x69, 0x42, 0x4f, 0xa1, 0x99, 0xe2, 0x88, 0x92, 0x70, 0xcc, 0xa8, 0x38, 0xe3, 0x86,
+	0xdd, 0xb1, 0xe4, 0x43, 0xcc, 0xca, 0x1e, 0x62, 0xd6, 0x28, 0x7b, 0x88, 0xf9, 0x0d, 0xe9, 0x3c,
+	0xa2, 0xe8, 0x19, 0xaf, 0x9f, 0x24, 0x4e, 0x99, 0x84, 0xd6, 0x3f, 0x08, 0x85, 0xcc, 0x7d, 0x44,
+	0xbb, 0x7f, 0xe8, 0x50, 0x95, 0x05, 0xff, 0x18, 0x6a, 0x97, 0x42, 0x65, 0x75, 0x19, 0xde, 0x29,
+	0x64, 0x24, 0x7f, 0x80, 0xaf, 0x5c, 0xd0, 0x01, 0xb4, 0x02, 0xf1, 0x08, 0x93, 0x0d, 0x4f, 0x35,
+	0xf9, 0x3b, 0x37, 0x3c, 0xd0, 0x8e, 0x4a, 0xbe, 0x11, 0xe4, 0x9e, 0x74, 0x3d, 0x68, 0x5e, 0x25,
+	0x91, 0x82, 0x95, 0x05, 0xcc, 0xdc, 0x6c, 0xed, 0x47, 0x25, 0xbf, 0x71, 0x95, 0xdd, 0x73, 0x36,
+	0xc0, 0x0a, 0x60, 0x0b, 0xb5, 0x0d, 0x7b, 0x77, 0x13, 0x61, 0x1f, 0x95, 0xfc, 0xe6, 0xd5, 0xea,
+	0xea, 0x38, 0x80, 0x56, 0xbe, 0x1f, 0x0b, 0xb9, 0x73, 0xf4, 0x72, 0x6d, 0x94, 0xd3, 0xcb, 0x75,
+	0x68, 0x4e, 0x2f, 0x4d, 0x02, 0x05, 0xab, 0x15, 0xe9, 0x65, 0xbd, 0x84, 0xd3, 0x4b, 0x93, 0x40,
+	0x8c, 0x5f, 0xb4, 0x00, 0x64, 0xcf, 0xe7, 0xff, 0xf2, 0x85, 0x03, 0x77, 0xe2, 0x74, 0x62, 0xc5,
+	0x09, 0x99, 0x07, 0x71, 0x1a, 0x2a, 0xe4, 0xcf, 0xd6, 0x24, 0x62, 0x97, 0x8b, 0x73, 0xde, 0x5c,
+	0x7a, 0xd9, 0x5a, 0x4f, 0xae, 0x3d, 0x51, 0x6f, 0xe8, 0xe5, 0xd7, 0xbd, 0x49, 0xac, 0x6c, 0xe7,
+	0x35, 0x61, 0xfc, 0xea, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x16, 0x55, 0x00, 0xec, 0xe5, 0x0b,
+	0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/health.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/health.pb.go
new file mode 100644
index 0000000..619ff26
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/health.pb.go
@@ -0,0 +1,210 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/health.proto
+
+package voltha
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	empty "github.com/golang/protobuf/ptypes/empty"
+	_ "github.com/opencord/voltha-protos/v5/go/common"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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
+
+// Health states
+type HealthStatus_HealthState int32
+
+const (
+	HealthStatus_HEALTHY    HealthStatus_HealthState = 0
+	HealthStatus_OVERLOADED HealthStatus_HealthState = 1
+	HealthStatus_DYING      HealthStatus_HealthState = 2
+)
+
+var HealthStatus_HealthState_name = map[int32]string{
+	0: "HEALTHY",
+	1: "OVERLOADED",
+	2: "DYING",
+}
+
+var HealthStatus_HealthState_value = map[string]int32{
+	"HEALTHY":    0,
+	"OVERLOADED": 1,
+	"DYING":      2,
+}
+
+func (x HealthStatus_HealthState) String() string {
+	return proto.EnumName(HealthStatus_HealthState_name, int32(x))
+}
+
+func (HealthStatus_HealthState) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_dd1fc2b2d96d69b8, []int{0, 0}
+}
+
+// Encode health status of a Voltha instance
+type HealthStatus struct {
+	// Current state of health of this Voltha instance
+	State                HealthStatus_HealthState `protobuf:"varint,1,opt,name=state,proto3,enum=voltha.HealthStatus_HealthState" json:"state,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
+	XXX_unrecognized     []byte                   `json:"-"`
+	XXX_sizecache        int32                    `json:"-"`
+}
+
+func (m *HealthStatus) Reset()         { *m = HealthStatus{} }
+func (m *HealthStatus) String() string { return proto.CompactTextString(m) }
+func (*HealthStatus) ProtoMessage()    {}
+func (*HealthStatus) Descriptor() ([]byte, []int) {
+	return fileDescriptor_dd1fc2b2d96d69b8, []int{0}
+}
+
+func (m *HealthStatus) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_HealthStatus.Unmarshal(m, b)
+}
+func (m *HealthStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_HealthStatus.Marshal(b, m, deterministic)
+}
+func (m *HealthStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HealthStatus.Merge(m, src)
+}
+func (m *HealthStatus) XXX_Size() int {
+	return xxx_messageInfo_HealthStatus.Size(m)
+}
+func (m *HealthStatus) XXX_DiscardUnknown() {
+	xxx_messageInfo_HealthStatus.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HealthStatus proto.InternalMessageInfo
+
+func (m *HealthStatus) GetState() HealthStatus_HealthState {
+	if m != nil {
+		return m.State
+	}
+	return HealthStatus_HEALTHY
+}
+
+func init() {
+	proto.RegisterEnum("voltha.HealthStatus_HealthState", HealthStatus_HealthState_name, HealthStatus_HealthState_value)
+	proto.RegisterType((*HealthStatus)(nil), "voltha.HealthStatus")
+}
+
+func init() { proto.RegisterFile("voltha_protos/health.proto", fileDescriptor_dd1fc2b2d96d69b8) }
+
+var fileDescriptor_dd1fc2b2d96d69b8 = []byte{
+	// 302 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2a, 0xcb, 0xcf, 0x29,
+	0xc9, 0x48, 0x8c, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x2f, 0xd6, 0xcf, 0x48, 0x4d, 0xcc, 0x29, 0xc9,
+	0xd0, 0x03, 0xf3, 0x84, 0xd8, 0x20, 0x72, 0x52, 0x32, 0xe9, 0xf9, 0xf9, 0xe9, 0x39, 0xa9, 0xfa,
+	0x89, 0x05, 0x99, 0xfa, 0x89, 0x79, 0x79, 0xf9, 0x25, 0x89, 0x25, 0x99, 0xf9, 0x79, 0xc5, 0x10,
+	0x55, 0x52, 0xd2, 0x50, 0x59, 0x30, 0x2f, 0xa9, 0x34, 0x4d, 0x3f, 0x35, 0xb7, 0xa0, 0xa4, 0x12,
+	0x2a, 0x29, 0x81, 0x6a, 0x7c, 0x6e, 0x6a, 0x49, 0x22, 0x44, 0x46, 0xa9, 0x85, 0x91, 0x8b, 0xc7,
+	0x03, 0x6c, 0x5b, 0x70, 0x49, 0x62, 0x49, 0x69, 0xb1, 0x90, 0x2d, 0x17, 0x6b, 0x71, 0x49, 0x62,
+	0x49, 0xaa, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x9f, 0x91, 0x82, 0x1e, 0x44, 0xab, 0x1e, 0xb2, 0x22,
+	0x24, 0x4e, 0xaa, 0x13, 0xeb, 0x8b, 0x6f, 0x67, 0x65, 0x19, 0x83, 0x20, 0xba, 0x94, 0x4c, 0xb9,
+	0xb8, 0x91, 0x24, 0x85, 0xb8, 0xb9, 0xd8, 0x3d, 0x5c, 0x1d, 0x7d, 0x42, 0x3c, 0x22, 0x05, 0x18,
+	0x84, 0xf8, 0xb8, 0xb8, 0xfc, 0xc3, 0x5c, 0x83, 0x7c, 0xfc, 0x1d, 0x5d, 0x5c, 0x5d, 0x04, 0x18,
+	0x85, 0x38, 0xb9, 0x58, 0x5d, 0x22, 0x3d, 0xfd, 0xdc, 0x05, 0x98, 0x8c, 0x12, 0xb9, 0x78, 0xa1,
+	0xda, 0x52, 0x8b, 0xca, 0x32, 0x93, 0x53, 0x85, 0x02, 0xb8, 0xf8, 0xdd, 0x53, 0x4b, 0x50, 0x5c,
+	0x26, 0xa6, 0x07, 0xf1, 0xa2, 0x1e, 0xcc, 0x8b, 0x7a, 0xae, 0x20, 0x2f, 0x4a, 0x89, 0x60, 0x73,
+	0xa2, 0x12, 0x7f, 0xd3, 0xe5, 0x27, 0x93, 0x99, 0x38, 0x85, 0xd8, 0xa1, 0x81, 0xe9, 0xe4, 0xca,
+	0x25, 0x9c, 0x5f, 0x94, 0xae, 0x97, 0x5f, 0x90, 0x9a, 0x97, 0x9c, 0x5f, 0x94, 0x02, 0xd5, 0x14,
+	0xa5, 0x97, 0x9e, 0x59, 0x92, 0x51, 0x9a, 0xa4, 0x97, 0x9c, 0x9f, 0xab, 0x0f, 0x93, 0xd3, 0x87,
+	0xc8, 0xe9, 0x42, 0x83, 0xab, 0xcc, 0x54, 0x3f, 0x3d, 0x1f, 0x2a, 0x96, 0xc4, 0x06, 0x16, 0x34,
+	0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0x2a, 0x8c, 0xb8, 0x4a, 0xb2, 0x01, 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
+
+// HealthServiceClient is the client API for HealthService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type HealthServiceClient interface {
+	// Return current health status of a Voltha instance
+	GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*HealthStatus, error)
+}
+
+type healthServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewHealthServiceClient(cc *grpc.ClientConn) HealthServiceClient {
+	return &healthServiceClient{cc}
+}
+
+func (c *healthServiceClient) GetHealthStatus(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*HealthStatus, error) {
+	out := new(HealthStatus)
+	err := c.cc.Invoke(ctx, "/voltha.HealthService/GetHealthStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// HealthServiceServer is the server API for HealthService service.
+type HealthServiceServer interface {
+	// Return current health status of a Voltha instance
+	GetHealthStatus(context.Context, *empty.Empty) (*HealthStatus, error)
+}
+
+// UnimplementedHealthServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedHealthServiceServer struct {
+}
+
+func (*UnimplementedHealthServiceServer) GetHealthStatus(ctx context.Context, req *empty.Empty) (*HealthStatus, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetHealthStatus not implemented")
+}
+
+func RegisterHealthServiceServer(s *grpc.Server, srv HealthServiceServer) {
+	s.RegisterService(&_HealthService_serviceDesc, srv)
+}
+
+func _HealthService_GetHealthStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(HealthServiceServer).GetHealthStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.HealthService/GetHealthStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(HealthServiceServer).GetHealthStatus(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _HealthService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.HealthService",
+	HandlerType: (*HealthServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetHealthStatus",
+			Handler:    _HealthService_GetHealthStatus_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "voltha_protos/health.proto",
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/logical_device.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/logical_device.pb.go
new file mode 100644
index 0000000..2f37696
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/logical_device.pb.go
@@ -0,0 +1,351 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/logical_device.proto
+
+package voltha
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_ "github.com/opencord/voltha-protos/v5/go/common"
+	openflow_13 "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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 LogicalPortId struct {
+	// unique id of logical device
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// id of the port on the logical device
+	PortId               string   `protobuf:"bytes,2,opt,name=port_id,json=portId,proto3" json:"port_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LogicalPortId) Reset()         { *m = LogicalPortId{} }
+func (m *LogicalPortId) String() string { return proto.CompactTextString(m) }
+func (*LogicalPortId) ProtoMessage()    {}
+func (*LogicalPortId) Descriptor() ([]byte, []int) {
+	return fileDescriptor_caf139ab3abc8240, []int{0}
+}
+
+func (m *LogicalPortId) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LogicalPortId.Unmarshal(m, b)
+}
+func (m *LogicalPortId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LogicalPortId.Marshal(b, m, deterministic)
+}
+func (m *LogicalPortId) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogicalPortId.Merge(m, src)
+}
+func (m *LogicalPortId) XXX_Size() int {
+	return xxx_messageInfo_LogicalPortId.Size(m)
+}
+func (m *LogicalPortId) XXX_DiscardUnknown() {
+	xxx_messageInfo_LogicalPortId.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LogicalPortId proto.InternalMessageInfo
+
+func (m *LogicalPortId) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *LogicalPortId) GetPortId() string {
+	if m != nil {
+		return m.PortId
+	}
+	return ""
+}
+
+type LogicalPort struct {
+	Id                   string                    `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	OfpPort              *openflow_13.OfpPort      `protobuf:"bytes,2,opt,name=ofp_port,json=ofpPort,proto3" json:"ofp_port,omitempty"`
+	DeviceId             string                    `protobuf:"bytes,3,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	DevicePortNo         uint32                    `protobuf:"varint,4,opt,name=device_port_no,json=devicePortNo,proto3" json:"device_port_no,omitempty"`
+	RootPort             bool                      `protobuf:"varint,5,opt,name=root_port,json=rootPort,proto3" json:"root_port,omitempty"`
+	OfpPortStats         *openflow_13.OfpPortStats `protobuf:"bytes,6,opt,name=ofp_port_stats,json=ofpPortStats,proto3" json:"ofp_port_stats,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
+	XXX_unrecognized     []byte                    `json:"-"`
+	XXX_sizecache        int32                     `json:"-"`
+}
+
+func (m *LogicalPort) Reset()         { *m = LogicalPort{} }
+func (m *LogicalPort) String() string { return proto.CompactTextString(m) }
+func (*LogicalPort) ProtoMessage()    {}
+func (*LogicalPort) Descriptor() ([]byte, []int) {
+	return fileDescriptor_caf139ab3abc8240, []int{1}
+}
+
+func (m *LogicalPort) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LogicalPort.Unmarshal(m, b)
+}
+func (m *LogicalPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LogicalPort.Marshal(b, m, deterministic)
+}
+func (m *LogicalPort) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogicalPort.Merge(m, src)
+}
+func (m *LogicalPort) XXX_Size() int {
+	return xxx_messageInfo_LogicalPort.Size(m)
+}
+func (m *LogicalPort) XXX_DiscardUnknown() {
+	xxx_messageInfo_LogicalPort.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LogicalPort proto.InternalMessageInfo
+
+func (m *LogicalPort) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *LogicalPort) GetOfpPort() *openflow_13.OfpPort {
+	if m != nil {
+		return m.OfpPort
+	}
+	return nil
+}
+
+func (m *LogicalPort) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *LogicalPort) GetDevicePortNo() uint32 {
+	if m != nil {
+		return m.DevicePortNo
+	}
+	return 0
+}
+
+func (m *LogicalPort) GetRootPort() bool {
+	if m != nil {
+		return m.RootPort
+	}
+	return false
+}
+
+func (m *LogicalPort) GetOfpPortStats() *openflow_13.OfpPortStats {
+	if m != nil {
+		return m.OfpPortStats
+	}
+	return nil
+}
+
+type LogicalPorts struct {
+	Items                []*LogicalPort `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *LogicalPorts) Reset()         { *m = LogicalPorts{} }
+func (m *LogicalPorts) String() string { return proto.CompactTextString(m) }
+func (*LogicalPorts) ProtoMessage()    {}
+func (*LogicalPorts) Descriptor() ([]byte, []int) {
+	return fileDescriptor_caf139ab3abc8240, []int{2}
+}
+
+func (m *LogicalPorts) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LogicalPorts.Unmarshal(m, b)
+}
+func (m *LogicalPorts) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LogicalPorts.Marshal(b, m, deterministic)
+}
+func (m *LogicalPorts) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogicalPorts.Merge(m, src)
+}
+func (m *LogicalPorts) XXX_Size() int {
+	return xxx_messageInfo_LogicalPorts.Size(m)
+}
+func (m *LogicalPorts) XXX_DiscardUnknown() {
+	xxx_messageInfo_LogicalPorts.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LogicalPorts proto.InternalMessageInfo
+
+func (m *LogicalPorts) GetItems() []*LogicalPort {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type LogicalDevice struct {
+	// unique id of logical device
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// unique datapath id for the logical device (used by the SDN controller)
+	DatapathId uint64 `protobuf:"varint,2,opt,name=datapath_id,json=datapathId,proto3" json:"datapath_id,omitempty"`
+	// device description
+	Desc *openflow_13.OfpDesc `protobuf:"bytes,3,opt,name=desc,proto3" json:"desc,omitempty"`
+	// device features
+	SwitchFeatures *openflow_13.OfpSwitchFeatures `protobuf:"bytes,4,opt,name=switch_features,json=switchFeatures,proto3" json:"switch_features,omitempty"`
+	// name of the root device anchoring logical device
+	RootDeviceId         string   `protobuf:"bytes,5,opt,name=root_device_id,json=rootDeviceId,proto3" json:"root_device_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LogicalDevice) Reset()         { *m = LogicalDevice{} }
+func (m *LogicalDevice) String() string { return proto.CompactTextString(m) }
+func (*LogicalDevice) ProtoMessage()    {}
+func (*LogicalDevice) Descriptor() ([]byte, []int) {
+	return fileDescriptor_caf139ab3abc8240, []int{3}
+}
+
+func (m *LogicalDevice) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LogicalDevice.Unmarshal(m, b)
+}
+func (m *LogicalDevice) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LogicalDevice.Marshal(b, m, deterministic)
+}
+func (m *LogicalDevice) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogicalDevice.Merge(m, src)
+}
+func (m *LogicalDevice) XXX_Size() int {
+	return xxx_messageInfo_LogicalDevice.Size(m)
+}
+func (m *LogicalDevice) XXX_DiscardUnknown() {
+	xxx_messageInfo_LogicalDevice.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LogicalDevice proto.InternalMessageInfo
+
+func (m *LogicalDevice) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *LogicalDevice) GetDatapathId() uint64 {
+	if m != nil {
+		return m.DatapathId
+	}
+	return 0
+}
+
+func (m *LogicalDevice) GetDesc() *openflow_13.OfpDesc {
+	if m != nil {
+		return m.Desc
+	}
+	return nil
+}
+
+func (m *LogicalDevice) GetSwitchFeatures() *openflow_13.OfpSwitchFeatures {
+	if m != nil {
+		return m.SwitchFeatures
+	}
+	return nil
+}
+
+func (m *LogicalDevice) GetRootDeviceId() string {
+	if m != nil {
+		return m.RootDeviceId
+	}
+	return ""
+}
+
+type LogicalDevices struct {
+	Items                []*LogicalDevice `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *LogicalDevices) Reset()         { *m = LogicalDevices{} }
+func (m *LogicalDevices) String() string { return proto.CompactTextString(m) }
+func (*LogicalDevices) ProtoMessage()    {}
+func (*LogicalDevices) Descriptor() ([]byte, []int) {
+	return fileDescriptor_caf139ab3abc8240, []int{4}
+}
+
+func (m *LogicalDevices) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LogicalDevices.Unmarshal(m, b)
+}
+func (m *LogicalDevices) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LogicalDevices.Marshal(b, m, deterministic)
+}
+func (m *LogicalDevices) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogicalDevices.Merge(m, src)
+}
+func (m *LogicalDevices) XXX_Size() int {
+	return xxx_messageInfo_LogicalDevices.Size(m)
+}
+func (m *LogicalDevices) XXX_DiscardUnknown() {
+	xxx_messageInfo_LogicalDevices.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LogicalDevices proto.InternalMessageInfo
+
+func (m *LogicalDevices) GetItems() []*LogicalDevice {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*LogicalPortId)(nil), "voltha.LogicalPortId")
+	proto.RegisterType((*LogicalPort)(nil), "voltha.LogicalPort")
+	proto.RegisterType((*LogicalPorts)(nil), "voltha.LogicalPorts")
+	proto.RegisterType((*LogicalDevice)(nil), "voltha.LogicalDevice")
+	proto.RegisterType((*LogicalDevices)(nil), "voltha.LogicalDevices")
+}
+
+func init() {
+	proto.RegisterFile("voltha_protos/logical_device.proto", fileDescriptor_caf139ab3abc8240)
+}
+
+var fileDescriptor_caf139ab3abc8240 = []byte{
+	// 456 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xc1, 0x6a, 0xdb, 0x40,
+	0x10, 0x86, 0x91, 0x63, 0x3b, 0xce, 0xca, 0x51, 0x40, 0x21, 0x54, 0x24, 0x85, 0x08, 0xd1, 0x83,
+	0x43, 0xa9, 0xd4, 0x3a, 0x14, 0xda, 0x43, 0x0f, 0x0d, 0x21, 0x60, 0x28, 0x6d, 0xd9, 0x42, 0x0f,
+	0xbd, 0x88, 0x8d, 0x76, 0x2d, 0x2f, 0xc8, 0x1a, 0xa1, 0xdd, 0x38, 0x2f, 0xdb, 0xa7, 0xe8, 0x13,
+	0x94, 0x9d, 0x5d, 0xb5, 0x52, 0x9c, 0xa3, 0xfe, 0xf9, 0x66, 0xe6, 0x9f, 0x7f, 0x45, 0x92, 0x1d,
+	0x54, 0x7a, 0xc3, 0xf2, 0xa6, 0x05, 0x0d, 0x2a, 0xab, 0xa0, 0x94, 0x05, 0xab, 0x72, 0x2e, 0x76,
+	0xb2, 0x10, 0x29, 0xaa, 0xe1, 0xd4, 0x32, 0xe7, 0x2f, 0x4b, 0x80, 0xb2, 0x12, 0x19, 0x6b, 0x64,
+	0xc6, 0xea, 0x1a, 0x34, 0xd3, 0x12, 0x6a, 0x65, 0xa9, 0xf3, 0x68, 0x38, 0x69, 0x2b, 0x34, 0x73,
+	0x95, 0xcb, 0x61, 0x05, 0x1a, 0x51, 0xaf, 0x2b, 0x78, 0xcc, 0xdf, 0x5d, 0x5b, 0x20, 0xf9, 0x40,
+	0x8e, 0xbf, 0xd8, 0xc5, 0xdf, 0xa1, 0xd5, 0x2b, 0x1e, 0x06, 0x64, 0x24, 0x79, 0xe4, 0xc5, 0xde,
+	0xe2, 0x88, 0x8e, 0x24, 0x0f, 0x5f, 0x90, 0xc3, 0x06, 0x5a, 0x9d, 0x4b, 0x1e, 0x8d, 0x50, 0x9c,
+	0x36, 0x08, 0x26, 0x7f, 0x3c, 0xe2, 0xf7, 0x5a, 0xf7, 0x1a, 0xdf, 0x92, 0x19, 0xac, 0x9b, 0xdc,
+	0xd0, 0xd8, 0xe9, 0x2f, 0xcf, 0xd2, 0xfe, 0xfe, 0xae, 0x48, 0x0f, 0x61, 0xdd, 0xe0, 0x84, 0x0b,
+	0x72, 0x64, 0x8f, 0x37, 0xcb, 0x0e, 0x70, 0xd0, 0xcc, 0x0a, 0x2b, 0x1e, 0xbe, 0x22, 0x81, 0x2b,
+	0xa2, 0x9d, 0x1a, 0xa2, 0x71, 0xec, 0x2d, 0x8e, 0xe9, 0xdc, 0xaa, 0x66, 0xc0, 0x57, 0x30, 0x23,
+	0x5a, 0x00, 0x6d, 0xb7, 0x4e, 0x62, 0x6f, 0x31, 0xa3, 0x33, 0x23, 0xe0, 0xfc, 0xcf, 0x24, 0xe8,
+	0x96, 0xe6, 0x4a, 0x33, 0xad, 0xa2, 0x29, 0xfa, 0xba, 0x78, 0xd6, 0x97, 0x45, 0xe8, 0xdc, 0xb9,
+	0xfb, 0x61, 0xbe, 0x92, 0x8f, 0x64, 0xde, 0xbb, 0x59, 0x85, 0x57, 0x64, 0x22, 0xb5, 0xd8, 0xaa,
+	0xc8, 0x8b, 0x0f, 0x16, 0xfe, 0xf2, 0x34, 0xb5, 0x79, 0xa7, 0x3d, 0x88, 0x5a, 0x22, 0xf9, 0xed,
+	0xfd, 0x8b, 0xfa, 0x16, 0x2d, 0xef, 0x25, 0x76, 0x49, 0x7c, 0xce, 0x34, 0x6b, 0x98, 0xde, 0x74,
+	0x71, 0x8f, 0x29, 0xe9, 0xa4, 0x15, 0x0f, 0xaf, 0xc8, 0x98, 0x0b, 0x55, 0x60, 0x36, 0xcf, 0xc5,
+	0x69, 0x8a, 0x14, 0x91, 0x70, 0x45, 0x4e, 0xd4, 0xa3, 0xd4, 0xc5, 0x26, 0x5f, 0x0b, 0xa6, 0x1f,
+	0x5a, 0xa1, 0x30, 0x2f, 0x7f, 0x19, 0xef, 0x75, 0x3d, 0xe1, 0x68, 0x60, 0x85, 0x3b, 0xf7, 0x6d,
+	0x92, 0xc7, 0x4c, 0xff, 0xbf, 0xcd, 0x04, 0x2d, 0xcf, 0x8d, 0x7a, 0xeb, 0xde, 0x27, 0xf9, 0x44,
+	0x82, 0xc1, 0x75, 0x2a, 0x7c, 0x3d, 0xcc, 0xe6, 0xec, 0x49, 0x36, 0x16, 0x73, 0xe9, 0xdc, 0xfc,
+	0x24, 0xa7, 0xd0, 0x96, 0xe8, 0xad, 0x80, 0x96, 0x3b, 0xf6, 0xe6, 0xe4, 0xdb, 0xdd, 0x00, 0xff,
+	0x95, 0x96, 0x52, 0x6f, 0x1e, 0xee, 0xd3, 0x02, 0xb6, 0x59, 0x07, 0x67, 0x16, 0x7e, 0xe3, 0x7e,
+	0xf2, 0xdd, 0xfb, 0xac, 0x04, 0xa7, 0xdd, 0x4f, 0x51, 0xbc, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff,
+	0x82, 0x07, 0x28, 0x6f, 0x6d, 0x03, 0x00, 0x00,
+}
diff --git a/vendor/github.com/opencord/voltha-protos/v5/go/voltha/voltha.pb.go b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/voltha.pb.go
new file mode 100644
index 0000000..89368cf
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-protos/v5/go/voltha/voltha.pb.go
@@ -0,0 +1,5370 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: voltha_protos/voltha.proto
+
+package voltha
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	empty "github.com/golang/protobuf/ptypes/empty"
+	common "github.com/opencord/voltha-protos/v5/go/common"
+	config "github.com/opencord/voltha-protos/v5/go/ext/config"
+	omci "github.com/opencord/voltha-protos/v5/go/omci"
+	openflow_13 "github.com/opencord/voltha-protos/v5/go/openflow_13"
+	_ "google.golang.org/genproto/googleapis/api/annotations"
+	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
+
+// ChildNode from public import voltha_protos/meta.proto
+type ChildNode = common.ChildNode
+
+// Access from public import voltha_protos/meta.proto
+type Access = common.Access
+
+var Access_name = common.Access_name
+var Access_value = common.Access_value
+
+const Access_CONFIG = Access(common.Access_CONFIG)
+const Access_READ_ONLY = Access(common.Access_READ_ONLY)
+const Access_REAL_TIME = Access(common.Access_REAL_TIME)
+
+var E_ChildNode = common.E_ChildNode
+
+var E_Access = common.E_Access
+
+// ID from public import voltha_protos/common.proto
+type ID = common.ID
+
+// IDs from public import voltha_protos/common.proto
+type IDs = common.IDs
+
+// AdminState from public import voltha_protos/common.proto
+type AdminState = common.AdminState
+
+// OperStatus from public import voltha_protos/common.proto
+type OperStatus = common.OperStatus
+
+// ConnectStatus from public import voltha_protos/common.proto
+type ConnectStatus = common.ConnectStatus
+
+// OperationResp from public import voltha_protos/common.proto
+type OperationResp = common.OperationResp
+
+// ValueType from public import voltha_protos/common.proto
+type ValueType = common.ValueType
+
+// ValueSpecifier from public import voltha_protos/common.proto
+type ValueSpecifier = common.ValueSpecifier
+
+// ReturnValues from public import voltha_protos/common.proto
+type ReturnValues = common.ReturnValues
+
+// TestModeKeys from public import voltha_protos/common.proto
+type TestModeKeys = common.TestModeKeys
+
+var TestModeKeys_name = common.TestModeKeys_name
+var TestModeKeys_value = common.TestModeKeys_value
+
+const TestModeKeys_api_test = TestModeKeys(common.TestModeKeys_api_test)
+
+// AdminState_Types from public import voltha_protos/common.proto
+type AdminState_Types = common.AdminState_Types
+
+var AdminState_Types_name = common.AdminState_Types_name
+var AdminState_Types_value = common.AdminState_Types_value
+
+const AdminState_UNKNOWN = AdminState_Types(common.AdminState_UNKNOWN)
+const AdminState_PREPROVISIONED = AdminState_Types(common.AdminState_PREPROVISIONED)
+const AdminState_ENABLED = AdminState_Types(common.AdminState_ENABLED)
+const AdminState_DISABLED = AdminState_Types(common.AdminState_DISABLED)
+const AdminState_DOWNLOADING_IMAGE = AdminState_Types(common.AdminState_DOWNLOADING_IMAGE)
+
+// OperStatus_Types from public import voltha_protos/common.proto
+type OperStatus_Types = common.OperStatus_Types
+
+var OperStatus_Types_name = common.OperStatus_Types_name
+var OperStatus_Types_value = common.OperStatus_Types_value
+
+const OperStatus_UNKNOWN = OperStatus_Types(common.OperStatus_UNKNOWN)
+const OperStatus_DISCOVERED = OperStatus_Types(common.OperStatus_DISCOVERED)
+const OperStatus_ACTIVATING = OperStatus_Types(common.OperStatus_ACTIVATING)
+const OperStatus_TESTING = OperStatus_Types(common.OperStatus_TESTING)
+const OperStatus_ACTIVE = OperStatus_Types(common.OperStatus_ACTIVE)
+const OperStatus_FAILED = OperStatus_Types(common.OperStatus_FAILED)
+const OperStatus_RECONCILING = OperStatus_Types(common.OperStatus_RECONCILING)
+const OperStatus_RECONCILING_FAILED = OperStatus_Types(common.OperStatus_RECONCILING_FAILED)
+
+// ConnectStatus_Types from public import voltha_protos/common.proto
+type ConnectStatus_Types = common.ConnectStatus_Types
+
+var ConnectStatus_Types_name = common.ConnectStatus_Types_name
+var ConnectStatus_Types_value = common.ConnectStatus_Types_value
+
+const ConnectStatus_UNKNOWN = ConnectStatus_Types(common.ConnectStatus_UNKNOWN)
+const ConnectStatus_UNREACHABLE = ConnectStatus_Types(common.ConnectStatus_UNREACHABLE)
+const ConnectStatus_REACHABLE = ConnectStatus_Types(common.ConnectStatus_REACHABLE)
+
+// OperationResp_OperationReturnCode from public import voltha_protos/common.proto
+type OperationResp_OperationReturnCode = common.OperationResp_OperationReturnCode
+
+var OperationResp_OperationReturnCode_name = common.OperationResp_OperationReturnCode_name
+var OperationResp_OperationReturnCode_value = common.OperationResp_OperationReturnCode_value
+
+const OperationResp_OPERATION_SUCCESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_SUCCESS)
+const OperationResp_OPERATION_FAILURE = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_FAILURE)
+const OperationResp_OPERATION_UNSUPPORTED = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_UNSUPPORTED)
+const OperationResp_OPERATION_IN_PROGRESS = OperationResp_OperationReturnCode(common.OperationResp_OPERATION_IN_PROGRESS)
+
+// ValueType_Type from public import voltha_protos/common.proto
+type ValueType_Type = common.ValueType_Type
+
+var ValueType_Type_name = common.ValueType_Type_name
+var ValueType_Type_value = common.ValueType_Type_value
+
+const ValueType_EMPTY = ValueType_Type(common.ValueType_EMPTY)
+const ValueType_DISTANCE = ValueType_Type(common.ValueType_DISTANCE)
+
+// OfpHeader from public import voltha_protos/openflow_13.proto
+type OfpHeader = openflow_13.OfpHeader
+
+// OfpHelloElemHeader from public import voltha_protos/openflow_13.proto
+type OfpHelloElemHeader = openflow_13.OfpHelloElemHeader
+type OfpHelloElemHeader_Versionbitmap = openflow_13.OfpHelloElemHeader_Versionbitmap
+
+// OfpHelloElemVersionbitmap from public import voltha_protos/openflow_13.proto
+type OfpHelloElemVersionbitmap = openflow_13.OfpHelloElemVersionbitmap
+
+// OfpHello from public import voltha_protos/openflow_13.proto
+type OfpHello = openflow_13.OfpHello
+
+// OfpSwitchConfig from public import voltha_protos/openflow_13.proto
+type OfpSwitchConfig = openflow_13.OfpSwitchConfig
+
+// OfpTableMod from public import voltha_protos/openflow_13.proto
+type OfpTableMod = openflow_13.OfpTableMod
+
+// OfpPort from public import voltha_protos/openflow_13.proto
+type OfpPort = openflow_13.OfpPort
+
+// OfpSwitchFeatures from public import voltha_protos/openflow_13.proto
+type OfpSwitchFeatures = openflow_13.OfpSwitchFeatures
+
+// OfpPortStatus from public import voltha_protos/openflow_13.proto
+type OfpPortStatus = openflow_13.OfpPortStatus
+
+// OfpPortMod from public import voltha_protos/openflow_13.proto
+type OfpPortMod = openflow_13.OfpPortMod
+
+// OfpMatch from public import voltha_protos/openflow_13.proto
+type OfpMatch = openflow_13.OfpMatch
+
+// OfpOxmField from public import voltha_protos/openflow_13.proto
+type OfpOxmField = openflow_13.OfpOxmField
+type OfpOxmField_OfbField = openflow_13.OfpOxmField_OfbField
+type OfpOxmField_ExperimenterField = openflow_13.OfpOxmField_ExperimenterField
+
+// OfpOxmOfbField from public import voltha_protos/openflow_13.proto
+type OfpOxmOfbField = openflow_13.OfpOxmOfbField
+type OfpOxmOfbField_Port = openflow_13.OfpOxmOfbField_Port
+type OfpOxmOfbField_PhysicalPort = openflow_13.OfpOxmOfbField_PhysicalPort
+type OfpOxmOfbField_TableMetadata = openflow_13.OfpOxmOfbField_TableMetadata
+type OfpOxmOfbField_EthDst = openflow_13.OfpOxmOfbField_EthDst
+type OfpOxmOfbField_EthSrc = openflow_13.OfpOxmOfbField_EthSrc
+type OfpOxmOfbField_EthType = openflow_13.OfpOxmOfbField_EthType
+type OfpOxmOfbField_VlanVid = openflow_13.OfpOxmOfbField_VlanVid
+type OfpOxmOfbField_VlanPcp = openflow_13.OfpOxmOfbField_VlanPcp
+type OfpOxmOfbField_IpDscp = openflow_13.OfpOxmOfbField_IpDscp
+type OfpOxmOfbField_IpEcn = openflow_13.OfpOxmOfbField_IpEcn
+type OfpOxmOfbField_IpProto = openflow_13.OfpOxmOfbField_IpProto
+type OfpOxmOfbField_Ipv4Src = openflow_13.OfpOxmOfbField_Ipv4Src
+type OfpOxmOfbField_Ipv4Dst = openflow_13.OfpOxmOfbField_Ipv4Dst
+type OfpOxmOfbField_TcpSrc = openflow_13.OfpOxmOfbField_TcpSrc
+type OfpOxmOfbField_TcpDst = openflow_13.OfpOxmOfbField_TcpDst
+type OfpOxmOfbField_UdpSrc = openflow_13.OfpOxmOfbField_UdpSrc
+type OfpOxmOfbField_UdpDst = openflow_13.OfpOxmOfbField_UdpDst
+type OfpOxmOfbField_SctpSrc = openflow_13.OfpOxmOfbField_SctpSrc
+type OfpOxmOfbField_SctpDst = openflow_13.OfpOxmOfbField_SctpDst
+type OfpOxmOfbField_Icmpv4Type = openflow_13.OfpOxmOfbField_Icmpv4Type
+type OfpOxmOfbField_Icmpv4Code = openflow_13.OfpOxmOfbField_Icmpv4Code
+type OfpOxmOfbField_ArpOp = openflow_13.OfpOxmOfbField_ArpOp
+type OfpOxmOfbField_ArpSpa = openflow_13.OfpOxmOfbField_ArpSpa
+type OfpOxmOfbField_ArpTpa = openflow_13.OfpOxmOfbField_ArpTpa
+type OfpOxmOfbField_ArpSha = openflow_13.OfpOxmOfbField_ArpSha
+type OfpOxmOfbField_ArpTha = openflow_13.OfpOxmOfbField_ArpTha
+type OfpOxmOfbField_Ipv6Src = openflow_13.OfpOxmOfbField_Ipv6Src
+type OfpOxmOfbField_Ipv6Dst = openflow_13.OfpOxmOfbField_Ipv6Dst
+type OfpOxmOfbField_Ipv6Flabel = openflow_13.OfpOxmOfbField_Ipv6Flabel
+type OfpOxmOfbField_Icmpv6Type = openflow_13.OfpOxmOfbField_Icmpv6Type
+type OfpOxmOfbField_Icmpv6Code = openflow_13.OfpOxmOfbField_Icmpv6Code
+type OfpOxmOfbField_Ipv6NdTarget = openflow_13.OfpOxmOfbField_Ipv6NdTarget
+type OfpOxmOfbField_Ipv6NdSsl = openflow_13.OfpOxmOfbField_Ipv6NdSsl
+type OfpOxmOfbField_Ipv6NdTll = openflow_13.OfpOxmOfbField_Ipv6NdTll
+type OfpOxmOfbField_MplsLabel = openflow_13.OfpOxmOfbField_MplsLabel
+type OfpOxmOfbField_MplsTc = openflow_13.OfpOxmOfbField_MplsTc
+type OfpOxmOfbField_MplsBos = openflow_13.OfpOxmOfbField_MplsBos
+type OfpOxmOfbField_PbbIsid = openflow_13.OfpOxmOfbField_PbbIsid
+type OfpOxmOfbField_TunnelId = openflow_13.OfpOxmOfbField_TunnelId
+type OfpOxmOfbField_Ipv6Exthdr = openflow_13.OfpOxmOfbField_Ipv6Exthdr
+type OfpOxmOfbField_TableMetadataMask = openflow_13.OfpOxmOfbField_TableMetadataMask
+type OfpOxmOfbField_EthDstMask = openflow_13.OfpOxmOfbField_EthDstMask
+type OfpOxmOfbField_EthSrcMask = openflow_13.OfpOxmOfbField_EthSrcMask
+type OfpOxmOfbField_VlanVidMask = openflow_13.OfpOxmOfbField_VlanVidMask
+type OfpOxmOfbField_Ipv4SrcMask = openflow_13.OfpOxmOfbField_Ipv4SrcMask
+type OfpOxmOfbField_Ipv4DstMask = openflow_13.OfpOxmOfbField_Ipv4DstMask
+type OfpOxmOfbField_ArpSpaMask = openflow_13.OfpOxmOfbField_ArpSpaMask
+type OfpOxmOfbField_ArpTpaMask = openflow_13.OfpOxmOfbField_ArpTpaMask
+type OfpOxmOfbField_Ipv6SrcMask = openflow_13.OfpOxmOfbField_Ipv6SrcMask
+type OfpOxmOfbField_Ipv6DstMask = openflow_13.OfpOxmOfbField_Ipv6DstMask
+type OfpOxmOfbField_Ipv6FlabelMask = openflow_13.OfpOxmOfbField_Ipv6FlabelMask
+type OfpOxmOfbField_PbbIsidMask = openflow_13.OfpOxmOfbField_PbbIsidMask
+type OfpOxmOfbField_TunnelIdMask = openflow_13.OfpOxmOfbField_TunnelIdMask
+type OfpOxmOfbField_Ipv6ExthdrMask = openflow_13.OfpOxmOfbField_Ipv6ExthdrMask
+
+// OfpOxmExperimenterField from public import voltha_protos/openflow_13.proto
+type OfpOxmExperimenterField = openflow_13.OfpOxmExperimenterField
+
+// OfpAction from public import voltha_protos/openflow_13.proto
+type OfpAction = openflow_13.OfpAction
+type OfpAction_Output = openflow_13.OfpAction_Output
+type OfpAction_MplsTtl = openflow_13.OfpAction_MplsTtl
+type OfpAction_Push = openflow_13.OfpAction_Push
+type OfpAction_PopMpls = openflow_13.OfpAction_PopMpls
+type OfpAction_Group = openflow_13.OfpAction_Group
+type OfpAction_NwTtl = openflow_13.OfpAction_NwTtl
+type OfpAction_SetField = openflow_13.OfpAction_SetField
+type OfpAction_Experimenter = openflow_13.OfpAction_Experimenter
+
+// OfpActionOutput from public import voltha_protos/openflow_13.proto
+type OfpActionOutput = openflow_13.OfpActionOutput
+
+// OfpActionMplsTtl from public import voltha_protos/openflow_13.proto
+type OfpActionMplsTtl = openflow_13.OfpActionMplsTtl
+
+// OfpActionPush from public import voltha_protos/openflow_13.proto
+type OfpActionPush = openflow_13.OfpActionPush
+
+// OfpActionPopMpls from public import voltha_protos/openflow_13.proto
+type OfpActionPopMpls = openflow_13.OfpActionPopMpls
+
+// OfpActionGroup from public import voltha_protos/openflow_13.proto
+type OfpActionGroup = openflow_13.OfpActionGroup
+
+// OfpActionNwTtl from public import voltha_protos/openflow_13.proto
+type OfpActionNwTtl = openflow_13.OfpActionNwTtl
+
+// OfpActionSetField from public import voltha_protos/openflow_13.proto
+type OfpActionSetField = openflow_13.OfpActionSetField
+
+// OfpActionExperimenter from public import voltha_protos/openflow_13.proto
+type OfpActionExperimenter = openflow_13.OfpActionExperimenter
+
+// OfpInstruction from public import voltha_protos/openflow_13.proto
+type OfpInstruction = openflow_13.OfpInstruction
+type OfpInstruction_GotoTable = openflow_13.OfpInstruction_GotoTable
+type OfpInstruction_WriteMetadata = openflow_13.OfpInstruction_WriteMetadata
+type OfpInstruction_Actions = openflow_13.OfpInstruction_Actions
+type OfpInstruction_Meter = openflow_13.OfpInstruction_Meter
+type OfpInstruction_Experimenter = openflow_13.OfpInstruction_Experimenter
+
+// OfpInstructionGotoTable from public import voltha_protos/openflow_13.proto
+type OfpInstructionGotoTable = openflow_13.OfpInstructionGotoTable
+
+// OfpInstructionWriteMetadata from public import voltha_protos/openflow_13.proto
+type OfpInstructionWriteMetadata = openflow_13.OfpInstructionWriteMetadata
+
+// OfpInstructionActions from public import voltha_protos/openflow_13.proto
+type OfpInstructionActions = openflow_13.OfpInstructionActions
+
+// OfpInstructionMeter from public import voltha_protos/openflow_13.proto
+type OfpInstructionMeter = openflow_13.OfpInstructionMeter
+
+// OfpInstructionExperimenter from public import voltha_protos/openflow_13.proto
+type OfpInstructionExperimenter = openflow_13.OfpInstructionExperimenter
+
+// OfpFlowMod from public import voltha_protos/openflow_13.proto
+type OfpFlowMod = openflow_13.OfpFlowMod
+
+// OfpBucket from public import voltha_protos/openflow_13.proto
+type OfpBucket = openflow_13.OfpBucket
+
+// OfpGroupMod from public import voltha_protos/openflow_13.proto
+type OfpGroupMod = openflow_13.OfpGroupMod
+
+// OfpPacketOut from public import voltha_protos/openflow_13.proto
+type OfpPacketOut = openflow_13.OfpPacketOut
+
+// OfpPacketIn from public import voltha_protos/openflow_13.proto
+type OfpPacketIn = openflow_13.OfpPacketIn
+
+// OfpFlowRemoved from public import voltha_protos/openflow_13.proto
+type OfpFlowRemoved = openflow_13.OfpFlowRemoved
+
+// OfpMeterBandHeader from public import voltha_protos/openflow_13.proto
+type OfpMeterBandHeader = openflow_13.OfpMeterBandHeader
+type OfpMeterBandHeader_Drop = openflow_13.OfpMeterBandHeader_Drop
+type OfpMeterBandHeader_DscpRemark = openflow_13.OfpMeterBandHeader_DscpRemark
+type OfpMeterBandHeader_Experimenter = openflow_13.OfpMeterBandHeader_Experimenter
+
+// OfpMeterBandDrop from public import voltha_protos/openflow_13.proto
+type OfpMeterBandDrop = openflow_13.OfpMeterBandDrop
+
+// OfpMeterBandDscpRemark from public import voltha_protos/openflow_13.proto
+type OfpMeterBandDscpRemark = openflow_13.OfpMeterBandDscpRemark
+
+// OfpMeterBandExperimenter from public import voltha_protos/openflow_13.proto
+type OfpMeterBandExperimenter = openflow_13.OfpMeterBandExperimenter
+
+// OfpMeterMod from public import voltha_protos/openflow_13.proto
+type OfpMeterMod = openflow_13.OfpMeterMod
+
+// OfpErrorMsg from public import voltha_protos/openflow_13.proto
+type OfpErrorMsg = openflow_13.OfpErrorMsg
+
+// OfpErrorExperimenterMsg from public import voltha_protos/openflow_13.proto
+type OfpErrorExperimenterMsg = openflow_13.OfpErrorExperimenterMsg
+
+// OfpMultipartRequest from public import voltha_protos/openflow_13.proto
+type OfpMultipartRequest = openflow_13.OfpMultipartRequest
+
+// OfpMultipartReply from public import voltha_protos/openflow_13.proto
+type OfpMultipartReply = openflow_13.OfpMultipartReply
+
+// OfpDesc from public import voltha_protos/openflow_13.proto
+type OfpDesc = openflow_13.OfpDesc
+
+// OfpFlowStatsRequest from public import voltha_protos/openflow_13.proto
+type OfpFlowStatsRequest = openflow_13.OfpFlowStatsRequest
+
+// OfpFlowStats from public import voltha_protos/openflow_13.proto
+type OfpFlowStats = openflow_13.OfpFlowStats
+
+// OfpAggregateStatsRequest from public import voltha_protos/openflow_13.proto
+type OfpAggregateStatsRequest = openflow_13.OfpAggregateStatsRequest
+
+// OfpAggregateStatsReply from public import voltha_protos/openflow_13.proto
+type OfpAggregateStatsReply = openflow_13.OfpAggregateStatsReply
+
+// OfpTableFeatureProperty from public import voltha_protos/openflow_13.proto
+type OfpTableFeatureProperty = openflow_13.OfpTableFeatureProperty
+type OfpTableFeatureProperty_Instructions = openflow_13.OfpTableFeatureProperty_Instructions
+type OfpTableFeatureProperty_NextTables = openflow_13.OfpTableFeatureProperty_NextTables
+type OfpTableFeatureProperty_Actions = openflow_13.OfpTableFeatureProperty_Actions
+type OfpTableFeatureProperty_Oxm = openflow_13.OfpTableFeatureProperty_Oxm
+type OfpTableFeatureProperty_Experimenter = openflow_13.OfpTableFeatureProperty_Experimenter
+
+// OfpTableFeaturePropInstructions from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropInstructions = openflow_13.OfpTableFeaturePropInstructions
+
+// OfpTableFeaturePropNextTables from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropNextTables = openflow_13.OfpTableFeaturePropNextTables
+
+// OfpTableFeaturePropActions from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropActions = openflow_13.OfpTableFeaturePropActions
+
+// OfpTableFeaturePropOxm from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropOxm = openflow_13.OfpTableFeaturePropOxm
+
+// OfpTableFeaturePropExperimenter from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropExperimenter = openflow_13.OfpTableFeaturePropExperimenter
+
+// OfpTableFeatures from public import voltha_protos/openflow_13.proto
+type OfpTableFeatures = openflow_13.OfpTableFeatures
+
+// OfpTableStats from public import voltha_protos/openflow_13.proto
+type OfpTableStats = openflow_13.OfpTableStats
+
+// OfpPortStatsRequest from public import voltha_protos/openflow_13.proto
+type OfpPortStatsRequest = openflow_13.OfpPortStatsRequest
+
+// OfpPortStats from public import voltha_protos/openflow_13.proto
+type OfpPortStats = openflow_13.OfpPortStats
+
+// OfpGroupStatsRequest from public import voltha_protos/openflow_13.proto
+type OfpGroupStatsRequest = openflow_13.OfpGroupStatsRequest
+
+// OfpBucketCounter from public import voltha_protos/openflow_13.proto
+type OfpBucketCounter = openflow_13.OfpBucketCounter
+
+// OfpGroupStats from public import voltha_protos/openflow_13.proto
+type OfpGroupStats = openflow_13.OfpGroupStats
+
+// OfpGroupDesc from public import voltha_protos/openflow_13.proto
+type OfpGroupDesc = openflow_13.OfpGroupDesc
+
+// OfpGroupEntry from public import voltha_protos/openflow_13.proto
+type OfpGroupEntry = openflow_13.OfpGroupEntry
+
+// OfpGroupFeatures from public import voltha_protos/openflow_13.proto
+type OfpGroupFeatures = openflow_13.OfpGroupFeatures
+
+// OfpMeterMultipartRequest from public import voltha_protos/openflow_13.proto
+type OfpMeterMultipartRequest = openflow_13.OfpMeterMultipartRequest
+
+// OfpMeterBandStats from public import voltha_protos/openflow_13.proto
+type OfpMeterBandStats = openflow_13.OfpMeterBandStats
+
+// OfpMeterStats from public import voltha_protos/openflow_13.proto
+type OfpMeterStats = openflow_13.OfpMeterStats
+
+// OfpMeterConfig from public import voltha_protos/openflow_13.proto
+type OfpMeterConfig = openflow_13.OfpMeterConfig
+
+// OfpMeterFeatures from public import voltha_protos/openflow_13.proto
+type OfpMeterFeatures = openflow_13.OfpMeterFeatures
+
+// OfpMeterEntry from public import voltha_protos/openflow_13.proto
+type OfpMeterEntry = openflow_13.OfpMeterEntry
+
+// OfpExperimenterMultipartHeader from public import voltha_protos/openflow_13.proto
+type OfpExperimenterMultipartHeader = openflow_13.OfpExperimenterMultipartHeader
+
+// OfpExperimenterHeader from public import voltha_protos/openflow_13.proto
+type OfpExperimenterHeader = openflow_13.OfpExperimenterHeader
+
+// OfpQueuePropHeader from public import voltha_protos/openflow_13.proto
+type OfpQueuePropHeader = openflow_13.OfpQueuePropHeader
+
+// OfpQueuePropMinRate from public import voltha_protos/openflow_13.proto
+type OfpQueuePropMinRate = openflow_13.OfpQueuePropMinRate
+
+// OfpQueuePropMaxRate from public import voltha_protos/openflow_13.proto
+type OfpQueuePropMaxRate = openflow_13.OfpQueuePropMaxRate
+
+// OfpQueuePropExperimenter from public import voltha_protos/openflow_13.proto
+type OfpQueuePropExperimenter = openflow_13.OfpQueuePropExperimenter
+
+// OfpPacketQueue from public import voltha_protos/openflow_13.proto
+type OfpPacketQueue = openflow_13.OfpPacketQueue
+
+// OfpQueueGetConfigRequest from public import voltha_protos/openflow_13.proto
+type OfpQueueGetConfigRequest = openflow_13.OfpQueueGetConfigRequest
+
+// OfpQueueGetConfigReply from public import voltha_protos/openflow_13.proto
+type OfpQueueGetConfigReply = openflow_13.OfpQueueGetConfigReply
+
+// OfpActionSetQueue from public import voltha_protos/openflow_13.proto
+type OfpActionSetQueue = openflow_13.OfpActionSetQueue
+
+// OfpQueueStatsRequest from public import voltha_protos/openflow_13.proto
+type OfpQueueStatsRequest = openflow_13.OfpQueueStatsRequest
+
+// OfpQueueStats from public import voltha_protos/openflow_13.proto
+type OfpQueueStats = openflow_13.OfpQueueStats
+
+// OfpRoleRequest from public import voltha_protos/openflow_13.proto
+type OfpRoleRequest = openflow_13.OfpRoleRequest
+
+// OfpAsyncConfig from public import voltha_protos/openflow_13.proto
+type OfpAsyncConfig = openflow_13.OfpAsyncConfig
+
+// MeterModUpdate from public import voltha_protos/openflow_13.proto
+type MeterModUpdate = openflow_13.MeterModUpdate
+
+// MeterStatsReply from public import voltha_protos/openflow_13.proto
+type MeterStatsReply = openflow_13.MeterStatsReply
+
+// FlowTableUpdate from public import voltha_protos/openflow_13.proto
+type FlowTableUpdate = openflow_13.FlowTableUpdate
+
+// FlowGroupTableUpdate from public import voltha_protos/openflow_13.proto
+type FlowGroupTableUpdate = openflow_13.FlowGroupTableUpdate
+
+// Flows from public import voltha_protos/openflow_13.proto
+type Flows = openflow_13.Flows
+
+// Meters from public import voltha_protos/openflow_13.proto
+type Meters = openflow_13.Meters
+
+// FlowGroups from public import voltha_protos/openflow_13.proto
+type FlowGroups = openflow_13.FlowGroups
+
+// FlowChanges from public import voltha_protos/openflow_13.proto
+type FlowChanges = openflow_13.FlowChanges
+
+// FlowGroupChanges from public import voltha_protos/openflow_13.proto
+type FlowGroupChanges = openflow_13.FlowGroupChanges
+
+// PacketIn from public import voltha_protos/openflow_13.proto
+type PacketIn = openflow_13.PacketIn
+
+// PacketOut from public import voltha_protos/openflow_13.proto
+type PacketOut = openflow_13.PacketOut
+
+// ChangeEvent from public import voltha_protos/openflow_13.proto
+type ChangeEvent = openflow_13.ChangeEvent
+type ChangeEvent_PortStatus = openflow_13.ChangeEvent_PortStatus
+type ChangeEvent_Error = openflow_13.ChangeEvent_Error
+
+// OfpPortNo from public import voltha_protos/openflow_13.proto
+type OfpPortNo = openflow_13.OfpPortNo
+
+var OfpPortNo_name = openflow_13.OfpPortNo_name
+var OfpPortNo_value = openflow_13.OfpPortNo_value
+
+const OfpPortNo_OFPP_INVALID = OfpPortNo(openflow_13.OfpPortNo_OFPP_INVALID)
+const OfpPortNo_OFPP_MAX = OfpPortNo(openflow_13.OfpPortNo_OFPP_MAX)
+const OfpPortNo_OFPP_IN_PORT = OfpPortNo(openflow_13.OfpPortNo_OFPP_IN_PORT)
+const OfpPortNo_OFPP_TABLE = OfpPortNo(openflow_13.OfpPortNo_OFPP_TABLE)
+const OfpPortNo_OFPP_NORMAL = OfpPortNo(openflow_13.OfpPortNo_OFPP_NORMAL)
+const OfpPortNo_OFPP_FLOOD = OfpPortNo(openflow_13.OfpPortNo_OFPP_FLOOD)
+const OfpPortNo_OFPP_ALL = OfpPortNo(openflow_13.OfpPortNo_OFPP_ALL)
+const OfpPortNo_OFPP_CONTROLLER = OfpPortNo(openflow_13.OfpPortNo_OFPP_CONTROLLER)
+const OfpPortNo_OFPP_LOCAL = OfpPortNo(openflow_13.OfpPortNo_OFPP_LOCAL)
+const OfpPortNo_OFPP_ANY = OfpPortNo(openflow_13.OfpPortNo_OFPP_ANY)
+
+// OfpType from public import voltha_protos/openflow_13.proto
+type OfpType = openflow_13.OfpType
+
+var OfpType_name = openflow_13.OfpType_name
+var OfpType_value = openflow_13.OfpType_value
+
+const OfpType_OFPT_HELLO = OfpType(openflow_13.OfpType_OFPT_HELLO)
+const OfpType_OFPT_ERROR = OfpType(openflow_13.OfpType_OFPT_ERROR)
+const OfpType_OFPT_ECHO_REQUEST = OfpType(openflow_13.OfpType_OFPT_ECHO_REQUEST)
+const OfpType_OFPT_ECHO_REPLY = OfpType(openflow_13.OfpType_OFPT_ECHO_REPLY)
+const OfpType_OFPT_EXPERIMENTER = OfpType(openflow_13.OfpType_OFPT_EXPERIMENTER)
+const OfpType_OFPT_FEATURES_REQUEST = OfpType(openflow_13.OfpType_OFPT_FEATURES_REQUEST)
+const OfpType_OFPT_FEATURES_REPLY = OfpType(openflow_13.OfpType_OFPT_FEATURES_REPLY)
+const OfpType_OFPT_GET_CONFIG_REQUEST = OfpType(openflow_13.OfpType_OFPT_GET_CONFIG_REQUEST)
+const OfpType_OFPT_GET_CONFIG_REPLY = OfpType(openflow_13.OfpType_OFPT_GET_CONFIG_REPLY)
+const OfpType_OFPT_SET_CONFIG = OfpType(openflow_13.OfpType_OFPT_SET_CONFIG)
+const OfpType_OFPT_PACKET_IN = OfpType(openflow_13.OfpType_OFPT_PACKET_IN)
+const OfpType_OFPT_FLOW_REMOVED = OfpType(openflow_13.OfpType_OFPT_FLOW_REMOVED)
+const OfpType_OFPT_PORT_STATUS = OfpType(openflow_13.OfpType_OFPT_PORT_STATUS)
+const OfpType_OFPT_PACKET_OUT = OfpType(openflow_13.OfpType_OFPT_PACKET_OUT)
+const OfpType_OFPT_FLOW_MOD = OfpType(openflow_13.OfpType_OFPT_FLOW_MOD)
+const OfpType_OFPT_GROUP_MOD = OfpType(openflow_13.OfpType_OFPT_GROUP_MOD)
+const OfpType_OFPT_PORT_MOD = OfpType(openflow_13.OfpType_OFPT_PORT_MOD)
+const OfpType_OFPT_TABLE_MOD = OfpType(openflow_13.OfpType_OFPT_TABLE_MOD)
+const OfpType_OFPT_MULTIPART_REQUEST = OfpType(openflow_13.OfpType_OFPT_MULTIPART_REQUEST)
+const OfpType_OFPT_MULTIPART_REPLY = OfpType(openflow_13.OfpType_OFPT_MULTIPART_REPLY)
+const OfpType_OFPT_BARRIER_REQUEST = OfpType(openflow_13.OfpType_OFPT_BARRIER_REQUEST)
+const OfpType_OFPT_BARRIER_REPLY = OfpType(openflow_13.OfpType_OFPT_BARRIER_REPLY)
+const OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST = OfpType(openflow_13.OfpType_OFPT_QUEUE_GET_CONFIG_REQUEST)
+const OfpType_OFPT_QUEUE_GET_CONFIG_REPLY = OfpType(openflow_13.OfpType_OFPT_QUEUE_GET_CONFIG_REPLY)
+const OfpType_OFPT_ROLE_REQUEST = OfpType(openflow_13.OfpType_OFPT_ROLE_REQUEST)
+const OfpType_OFPT_ROLE_REPLY = OfpType(openflow_13.OfpType_OFPT_ROLE_REPLY)
+const OfpType_OFPT_GET_ASYNC_REQUEST = OfpType(openflow_13.OfpType_OFPT_GET_ASYNC_REQUEST)
+const OfpType_OFPT_GET_ASYNC_REPLY = OfpType(openflow_13.OfpType_OFPT_GET_ASYNC_REPLY)
+const OfpType_OFPT_SET_ASYNC = OfpType(openflow_13.OfpType_OFPT_SET_ASYNC)
+const OfpType_OFPT_METER_MOD = OfpType(openflow_13.OfpType_OFPT_METER_MOD)
+
+// OfpHelloElemType from public import voltha_protos/openflow_13.proto
+type OfpHelloElemType = openflow_13.OfpHelloElemType
+
+var OfpHelloElemType_name = openflow_13.OfpHelloElemType_name
+var OfpHelloElemType_value = openflow_13.OfpHelloElemType_value
+
+const OfpHelloElemType_OFPHET_INVALID = OfpHelloElemType(openflow_13.OfpHelloElemType_OFPHET_INVALID)
+const OfpHelloElemType_OFPHET_VERSIONBITMAP = OfpHelloElemType(openflow_13.OfpHelloElemType_OFPHET_VERSIONBITMAP)
+
+// OfpConfigFlags from public import voltha_protos/openflow_13.proto
+type OfpConfigFlags = openflow_13.OfpConfigFlags
+
+var OfpConfigFlags_name = openflow_13.OfpConfigFlags_name
+var OfpConfigFlags_value = openflow_13.OfpConfigFlags_value
+
+const OfpConfigFlags_OFPC_FRAG_NORMAL = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_NORMAL)
+const OfpConfigFlags_OFPC_FRAG_DROP = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_DROP)
+const OfpConfigFlags_OFPC_FRAG_REASM = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_REASM)
+const OfpConfigFlags_OFPC_FRAG_MASK = OfpConfigFlags(openflow_13.OfpConfigFlags_OFPC_FRAG_MASK)
+
+// OfpTableConfig from public import voltha_protos/openflow_13.proto
+type OfpTableConfig = openflow_13.OfpTableConfig
+
+var OfpTableConfig_name = openflow_13.OfpTableConfig_name
+var OfpTableConfig_value = openflow_13.OfpTableConfig_value
+
+const OfpTableConfig_OFPTC_INVALID = OfpTableConfig(openflow_13.OfpTableConfig_OFPTC_INVALID)
+const OfpTableConfig_OFPTC_DEPRECATED_MASK = OfpTableConfig(openflow_13.OfpTableConfig_OFPTC_DEPRECATED_MASK)
+
+// OfpTable from public import voltha_protos/openflow_13.proto
+type OfpTable = openflow_13.OfpTable
+
+var OfpTable_name = openflow_13.OfpTable_name
+var OfpTable_value = openflow_13.OfpTable_value
+
+const OfpTable_OFPTT_INVALID = OfpTable(openflow_13.OfpTable_OFPTT_INVALID)
+const OfpTable_OFPTT_MAX = OfpTable(openflow_13.OfpTable_OFPTT_MAX)
+const OfpTable_OFPTT_ALL = OfpTable(openflow_13.OfpTable_OFPTT_ALL)
+
+// OfpCapabilities from public import voltha_protos/openflow_13.proto
+type OfpCapabilities = openflow_13.OfpCapabilities
+
+var OfpCapabilities_name = openflow_13.OfpCapabilities_name
+var OfpCapabilities_value = openflow_13.OfpCapabilities_value
+
+const OfpCapabilities_OFPC_INVALID = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_INVALID)
+const OfpCapabilities_OFPC_FLOW_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_FLOW_STATS)
+const OfpCapabilities_OFPC_TABLE_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_TABLE_STATS)
+const OfpCapabilities_OFPC_PORT_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_PORT_STATS)
+const OfpCapabilities_OFPC_GROUP_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_GROUP_STATS)
+const OfpCapabilities_OFPC_IP_REASM = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_IP_REASM)
+const OfpCapabilities_OFPC_QUEUE_STATS = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_QUEUE_STATS)
+const OfpCapabilities_OFPC_PORT_BLOCKED = OfpCapabilities(openflow_13.OfpCapabilities_OFPC_PORT_BLOCKED)
+
+// OfpPortConfig from public import voltha_protos/openflow_13.proto
+type OfpPortConfig = openflow_13.OfpPortConfig
+
+var OfpPortConfig_name = openflow_13.OfpPortConfig_name
+var OfpPortConfig_value = openflow_13.OfpPortConfig_value
+
+const OfpPortConfig_OFPPC_INVALID = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_INVALID)
+const OfpPortConfig_OFPPC_PORT_DOWN = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_PORT_DOWN)
+const OfpPortConfig_OFPPC_NO_RECV = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_RECV)
+const OfpPortConfig_OFPPC_NO_FWD = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_FWD)
+const OfpPortConfig_OFPPC_NO_PACKET_IN = OfpPortConfig(openflow_13.OfpPortConfig_OFPPC_NO_PACKET_IN)
+
+// OfpPortState from public import voltha_protos/openflow_13.proto
+type OfpPortState = openflow_13.OfpPortState
+
+var OfpPortState_name = openflow_13.OfpPortState_name
+var OfpPortState_value = openflow_13.OfpPortState_value
+
+const OfpPortState_OFPPS_INVALID = OfpPortState(openflow_13.OfpPortState_OFPPS_INVALID)
+const OfpPortState_OFPPS_LINK_DOWN = OfpPortState(openflow_13.OfpPortState_OFPPS_LINK_DOWN)
+const OfpPortState_OFPPS_BLOCKED = OfpPortState(openflow_13.OfpPortState_OFPPS_BLOCKED)
+const OfpPortState_OFPPS_LIVE = OfpPortState(openflow_13.OfpPortState_OFPPS_LIVE)
+
+// OfpPortFeatures from public import voltha_protos/openflow_13.proto
+type OfpPortFeatures = openflow_13.OfpPortFeatures
+
+var OfpPortFeatures_name = openflow_13.OfpPortFeatures_name
+var OfpPortFeatures_value = openflow_13.OfpPortFeatures_value
+
+const OfpPortFeatures_OFPPF_INVALID = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_INVALID)
+const OfpPortFeatures_OFPPF_10MB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10MB_HD)
+const OfpPortFeatures_OFPPF_10MB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10MB_FD)
+const OfpPortFeatures_OFPPF_100MB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100MB_HD)
+const OfpPortFeatures_OFPPF_100MB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100MB_FD)
+const OfpPortFeatures_OFPPF_1GB_HD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1GB_HD)
+const OfpPortFeatures_OFPPF_1GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1GB_FD)
+const OfpPortFeatures_OFPPF_10GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_10GB_FD)
+const OfpPortFeatures_OFPPF_40GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_40GB_FD)
+const OfpPortFeatures_OFPPF_100GB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_100GB_FD)
+const OfpPortFeatures_OFPPF_1TB_FD = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_1TB_FD)
+const OfpPortFeatures_OFPPF_OTHER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_OTHER)
+const OfpPortFeatures_OFPPF_COPPER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_COPPER)
+const OfpPortFeatures_OFPPF_FIBER = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_FIBER)
+const OfpPortFeatures_OFPPF_AUTONEG = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_AUTONEG)
+const OfpPortFeatures_OFPPF_PAUSE = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_PAUSE)
+const OfpPortFeatures_OFPPF_PAUSE_ASYM = OfpPortFeatures(openflow_13.OfpPortFeatures_OFPPF_PAUSE_ASYM)
+
+// OfpPortReason from public import voltha_protos/openflow_13.proto
+type OfpPortReason = openflow_13.OfpPortReason
+
+var OfpPortReason_name = openflow_13.OfpPortReason_name
+var OfpPortReason_value = openflow_13.OfpPortReason_value
+
+const OfpPortReason_OFPPR_ADD = OfpPortReason(openflow_13.OfpPortReason_OFPPR_ADD)
+const OfpPortReason_OFPPR_DELETE = OfpPortReason(openflow_13.OfpPortReason_OFPPR_DELETE)
+const OfpPortReason_OFPPR_MODIFY = OfpPortReason(openflow_13.OfpPortReason_OFPPR_MODIFY)
+
+// OfpMatchType from public import voltha_protos/openflow_13.proto
+type OfpMatchType = openflow_13.OfpMatchType
+
+var OfpMatchType_name = openflow_13.OfpMatchType_name
+var OfpMatchType_value = openflow_13.OfpMatchType_value
+
+const OfpMatchType_OFPMT_STANDARD = OfpMatchType(openflow_13.OfpMatchType_OFPMT_STANDARD)
+const OfpMatchType_OFPMT_OXM = OfpMatchType(openflow_13.OfpMatchType_OFPMT_OXM)
+
+// OfpOxmClass from public import voltha_protos/openflow_13.proto
+type OfpOxmClass = openflow_13.OfpOxmClass
+
+var OfpOxmClass_name = openflow_13.OfpOxmClass_name
+var OfpOxmClass_value = openflow_13.OfpOxmClass_value
+
+const OfpOxmClass_OFPXMC_NXM_0 = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_NXM_0)
+const OfpOxmClass_OFPXMC_NXM_1 = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_NXM_1)
+const OfpOxmClass_OFPXMC_OPENFLOW_BASIC = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_OPENFLOW_BASIC)
+const OfpOxmClass_OFPXMC_EXPERIMENTER = OfpOxmClass(openflow_13.OfpOxmClass_OFPXMC_EXPERIMENTER)
+
+// OxmOfbFieldTypes from public import voltha_protos/openflow_13.proto
+type OxmOfbFieldTypes = openflow_13.OxmOfbFieldTypes
+
+var OxmOfbFieldTypes_name = openflow_13.OxmOfbFieldTypes_name
+var OxmOfbFieldTypes_value = openflow_13.OxmOfbFieldTypes_value
+
+const OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PORT)
+const OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IN_PHY_PORT)
+const OxmOfbFieldTypes_OFPXMT_OFB_METADATA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_METADATA)
+const OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ETH_TYPE)
+const OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_VID)
+const OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_VLAN_PCP)
+const OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_DSCP)
+const OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_ECN)
+const OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IP_PROTO)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV4_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TCP_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TCP_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_UDP_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_SCTP_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_TYPE)
+const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV4_CODE)
+const OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_OP)
+const OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SPA)
+const OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_TPA)
+const OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_SHA)
+const OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ARP_THA)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_SRC)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_DST)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_FLABEL)
+const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_TYPE)
+const OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_ICMPV6_CODE)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TARGET)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_SLL)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_ND_TLL)
+const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_LABEL)
+const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_TC)
+const OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_MPLS_BOS)
+const OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_PBB_ISID)
+const OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_TUNNEL_ID)
+const OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR = OxmOfbFieldTypes(openflow_13.OxmOfbFieldTypes_OFPXMT_OFB_IPV6_EXTHDR)
+
+// OfpVlanId from public import voltha_protos/openflow_13.proto
+type OfpVlanId = openflow_13.OfpVlanId
+
+var OfpVlanId_name = openflow_13.OfpVlanId_name
+var OfpVlanId_value = openflow_13.OfpVlanId_value
+
+const OfpVlanId_OFPVID_NONE = OfpVlanId(openflow_13.OfpVlanId_OFPVID_NONE)
+const OfpVlanId_OFPVID_PRESENT = OfpVlanId(openflow_13.OfpVlanId_OFPVID_PRESENT)
+
+// OfpIpv6ExthdrFlags from public import voltha_protos/openflow_13.proto
+type OfpIpv6ExthdrFlags = openflow_13.OfpIpv6ExthdrFlags
+
+var OfpIpv6ExthdrFlags_name = openflow_13.OfpIpv6ExthdrFlags_name
+var OfpIpv6ExthdrFlags_value = openflow_13.OfpIpv6ExthdrFlags_value
+
+const OfpIpv6ExthdrFlags_OFPIEH_INVALID = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_INVALID)
+const OfpIpv6ExthdrFlags_OFPIEH_NONEXT = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_NONEXT)
+const OfpIpv6ExthdrFlags_OFPIEH_ESP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_ESP)
+const OfpIpv6ExthdrFlags_OFPIEH_AUTH = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_AUTH)
+const OfpIpv6ExthdrFlags_OFPIEH_DEST = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_DEST)
+const OfpIpv6ExthdrFlags_OFPIEH_FRAG = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_FRAG)
+const OfpIpv6ExthdrFlags_OFPIEH_ROUTER = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_ROUTER)
+const OfpIpv6ExthdrFlags_OFPIEH_HOP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_HOP)
+const OfpIpv6ExthdrFlags_OFPIEH_UNREP = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_UNREP)
+const OfpIpv6ExthdrFlags_OFPIEH_UNSEQ = OfpIpv6ExthdrFlags(openflow_13.OfpIpv6ExthdrFlags_OFPIEH_UNSEQ)
+
+// OfpActionType from public import voltha_protos/openflow_13.proto
+type OfpActionType = openflow_13.OfpActionType
+
+var OfpActionType_name = openflow_13.OfpActionType_name
+var OfpActionType_value = openflow_13.OfpActionType_value
+
+const OfpActionType_OFPAT_OUTPUT = OfpActionType(openflow_13.OfpActionType_OFPAT_OUTPUT)
+const OfpActionType_OFPAT_COPY_TTL_OUT = OfpActionType(openflow_13.OfpActionType_OFPAT_COPY_TTL_OUT)
+const OfpActionType_OFPAT_COPY_TTL_IN = OfpActionType(openflow_13.OfpActionType_OFPAT_COPY_TTL_IN)
+const OfpActionType_OFPAT_SET_MPLS_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_MPLS_TTL)
+const OfpActionType_OFPAT_DEC_MPLS_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_DEC_MPLS_TTL)
+const OfpActionType_OFPAT_PUSH_VLAN = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_VLAN)
+const OfpActionType_OFPAT_POP_VLAN = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_VLAN)
+const OfpActionType_OFPAT_PUSH_MPLS = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_MPLS)
+const OfpActionType_OFPAT_POP_MPLS = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_MPLS)
+const OfpActionType_OFPAT_SET_QUEUE = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_QUEUE)
+const OfpActionType_OFPAT_GROUP = OfpActionType(openflow_13.OfpActionType_OFPAT_GROUP)
+const OfpActionType_OFPAT_SET_NW_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_NW_TTL)
+const OfpActionType_OFPAT_DEC_NW_TTL = OfpActionType(openflow_13.OfpActionType_OFPAT_DEC_NW_TTL)
+const OfpActionType_OFPAT_SET_FIELD = OfpActionType(openflow_13.OfpActionType_OFPAT_SET_FIELD)
+const OfpActionType_OFPAT_PUSH_PBB = OfpActionType(openflow_13.OfpActionType_OFPAT_PUSH_PBB)
+const OfpActionType_OFPAT_POP_PBB = OfpActionType(openflow_13.OfpActionType_OFPAT_POP_PBB)
+const OfpActionType_OFPAT_EXPERIMENTER = OfpActionType(openflow_13.OfpActionType_OFPAT_EXPERIMENTER)
+
+// OfpControllerMaxLen from public import voltha_protos/openflow_13.proto
+type OfpControllerMaxLen = openflow_13.OfpControllerMaxLen
+
+var OfpControllerMaxLen_name = openflow_13.OfpControllerMaxLen_name
+var OfpControllerMaxLen_value = openflow_13.OfpControllerMaxLen_value
+
+const OfpControllerMaxLen_OFPCML_INVALID = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_INVALID)
+const OfpControllerMaxLen_OFPCML_MAX = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_MAX)
+const OfpControllerMaxLen_OFPCML_NO_BUFFER = OfpControllerMaxLen(openflow_13.OfpControllerMaxLen_OFPCML_NO_BUFFER)
+
+// OfpInstructionType from public import voltha_protos/openflow_13.proto
+type OfpInstructionType = openflow_13.OfpInstructionType
+
+var OfpInstructionType_name = openflow_13.OfpInstructionType_name
+var OfpInstructionType_value = openflow_13.OfpInstructionType_value
+
+const OfpInstructionType_OFPIT_INVALID = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_INVALID)
+const OfpInstructionType_OFPIT_GOTO_TABLE = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_GOTO_TABLE)
+const OfpInstructionType_OFPIT_WRITE_METADATA = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_WRITE_METADATA)
+const OfpInstructionType_OFPIT_WRITE_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_WRITE_ACTIONS)
+const OfpInstructionType_OFPIT_APPLY_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_APPLY_ACTIONS)
+const OfpInstructionType_OFPIT_CLEAR_ACTIONS = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_CLEAR_ACTIONS)
+const OfpInstructionType_OFPIT_METER = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_METER)
+const OfpInstructionType_OFPIT_EXPERIMENTER = OfpInstructionType(openflow_13.OfpInstructionType_OFPIT_EXPERIMENTER)
+
+// OfpFlowModCommand from public import voltha_protos/openflow_13.proto
+type OfpFlowModCommand = openflow_13.OfpFlowModCommand
+
+var OfpFlowModCommand_name = openflow_13.OfpFlowModCommand_name
+var OfpFlowModCommand_value = openflow_13.OfpFlowModCommand_value
+
+const OfpFlowModCommand_OFPFC_ADD = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_ADD)
+const OfpFlowModCommand_OFPFC_MODIFY = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_MODIFY)
+const OfpFlowModCommand_OFPFC_MODIFY_STRICT = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_MODIFY_STRICT)
+const OfpFlowModCommand_OFPFC_DELETE = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_DELETE)
+const OfpFlowModCommand_OFPFC_DELETE_STRICT = OfpFlowModCommand(openflow_13.OfpFlowModCommand_OFPFC_DELETE_STRICT)
+
+// OfpFlowModFlags from public import voltha_protos/openflow_13.proto
+type OfpFlowModFlags = openflow_13.OfpFlowModFlags
+
+var OfpFlowModFlags_name = openflow_13.OfpFlowModFlags_name
+var OfpFlowModFlags_value = openflow_13.OfpFlowModFlags_value
+
+const OfpFlowModFlags_OFPFF_INVALID = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_INVALID)
+const OfpFlowModFlags_OFPFF_SEND_FLOW_REM = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_SEND_FLOW_REM)
+const OfpFlowModFlags_OFPFF_CHECK_OVERLAP = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_CHECK_OVERLAP)
+const OfpFlowModFlags_OFPFF_RESET_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_RESET_COUNTS)
+const OfpFlowModFlags_OFPFF_NO_PKT_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_NO_PKT_COUNTS)
+const OfpFlowModFlags_OFPFF_NO_BYT_COUNTS = OfpFlowModFlags(openflow_13.OfpFlowModFlags_OFPFF_NO_BYT_COUNTS)
+
+// OfpGroup from public import voltha_protos/openflow_13.proto
+type OfpGroup = openflow_13.OfpGroup
+
+var OfpGroup_name = openflow_13.OfpGroup_name
+var OfpGroup_value = openflow_13.OfpGroup_value
+
+const OfpGroup_OFPG_INVALID = OfpGroup(openflow_13.OfpGroup_OFPG_INVALID)
+const OfpGroup_OFPG_MAX = OfpGroup(openflow_13.OfpGroup_OFPG_MAX)
+const OfpGroup_OFPG_ALL = OfpGroup(openflow_13.OfpGroup_OFPG_ALL)
+const OfpGroup_OFPG_ANY = OfpGroup(openflow_13.OfpGroup_OFPG_ANY)
+
+// OfpGroupModCommand from public import voltha_protos/openflow_13.proto
+type OfpGroupModCommand = openflow_13.OfpGroupModCommand
+
+var OfpGroupModCommand_name = openflow_13.OfpGroupModCommand_name
+var OfpGroupModCommand_value = openflow_13.OfpGroupModCommand_value
+
+const OfpGroupModCommand_OFPGC_ADD = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_ADD)
+const OfpGroupModCommand_OFPGC_MODIFY = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_MODIFY)
+const OfpGroupModCommand_OFPGC_DELETE = OfpGroupModCommand(openflow_13.OfpGroupModCommand_OFPGC_DELETE)
+
+// OfpGroupType from public import voltha_protos/openflow_13.proto
+type OfpGroupType = openflow_13.OfpGroupType
+
+var OfpGroupType_name = openflow_13.OfpGroupType_name
+var OfpGroupType_value = openflow_13.OfpGroupType_value
+
+const OfpGroupType_OFPGT_ALL = OfpGroupType(openflow_13.OfpGroupType_OFPGT_ALL)
+const OfpGroupType_OFPGT_SELECT = OfpGroupType(openflow_13.OfpGroupType_OFPGT_SELECT)
+const OfpGroupType_OFPGT_INDIRECT = OfpGroupType(openflow_13.OfpGroupType_OFPGT_INDIRECT)
+const OfpGroupType_OFPGT_FF = OfpGroupType(openflow_13.OfpGroupType_OFPGT_FF)
+
+// OfpPacketInReason from public import voltha_protos/openflow_13.proto
+type OfpPacketInReason = openflow_13.OfpPacketInReason
+
+var OfpPacketInReason_name = openflow_13.OfpPacketInReason_name
+var OfpPacketInReason_value = openflow_13.OfpPacketInReason_value
+
+const OfpPacketInReason_OFPR_NO_MATCH = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_NO_MATCH)
+const OfpPacketInReason_OFPR_ACTION = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_ACTION)
+const OfpPacketInReason_OFPR_INVALID_TTL = OfpPacketInReason(openflow_13.OfpPacketInReason_OFPR_INVALID_TTL)
+
+// OfpFlowRemovedReason from public import voltha_protos/openflow_13.proto
+type OfpFlowRemovedReason = openflow_13.OfpFlowRemovedReason
+
+var OfpFlowRemovedReason_name = openflow_13.OfpFlowRemovedReason_name
+var OfpFlowRemovedReason_value = openflow_13.OfpFlowRemovedReason_value
+
+const OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_IDLE_TIMEOUT)
+const OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_HARD_TIMEOUT)
+const OfpFlowRemovedReason_OFPRR_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_DELETE)
+const OfpFlowRemovedReason_OFPRR_GROUP_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_GROUP_DELETE)
+const OfpFlowRemovedReason_OFPRR_METER_DELETE = OfpFlowRemovedReason(openflow_13.OfpFlowRemovedReason_OFPRR_METER_DELETE)
+
+// OfpMeter from public import voltha_protos/openflow_13.proto
+type OfpMeter = openflow_13.OfpMeter
+
+var OfpMeter_name = openflow_13.OfpMeter_name
+var OfpMeter_value = openflow_13.OfpMeter_value
+
+const OfpMeter_OFPM_ZERO = OfpMeter(openflow_13.OfpMeter_OFPM_ZERO)
+const OfpMeter_OFPM_MAX = OfpMeter(openflow_13.OfpMeter_OFPM_MAX)
+const OfpMeter_OFPM_SLOWPATH = OfpMeter(openflow_13.OfpMeter_OFPM_SLOWPATH)
+const OfpMeter_OFPM_CONTROLLER = OfpMeter(openflow_13.OfpMeter_OFPM_CONTROLLER)
+const OfpMeter_OFPM_ALL = OfpMeter(openflow_13.OfpMeter_OFPM_ALL)
+
+// OfpMeterBandType from public import voltha_protos/openflow_13.proto
+type OfpMeterBandType = openflow_13.OfpMeterBandType
+
+var OfpMeterBandType_name = openflow_13.OfpMeterBandType_name
+var OfpMeterBandType_value = openflow_13.OfpMeterBandType_value
+
+const OfpMeterBandType_OFPMBT_INVALID = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_INVALID)
+const OfpMeterBandType_OFPMBT_DROP = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_DROP)
+const OfpMeterBandType_OFPMBT_DSCP_REMARK = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_DSCP_REMARK)
+const OfpMeterBandType_OFPMBT_EXPERIMENTER = OfpMeterBandType(openflow_13.OfpMeterBandType_OFPMBT_EXPERIMENTER)
+
+// OfpMeterModCommand from public import voltha_protos/openflow_13.proto
+type OfpMeterModCommand = openflow_13.OfpMeterModCommand
+
+var OfpMeterModCommand_name = openflow_13.OfpMeterModCommand_name
+var OfpMeterModCommand_value = openflow_13.OfpMeterModCommand_value
+
+const OfpMeterModCommand_OFPMC_ADD = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_ADD)
+const OfpMeterModCommand_OFPMC_MODIFY = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_MODIFY)
+const OfpMeterModCommand_OFPMC_DELETE = OfpMeterModCommand(openflow_13.OfpMeterModCommand_OFPMC_DELETE)
+
+// OfpMeterFlags from public import voltha_protos/openflow_13.proto
+type OfpMeterFlags = openflow_13.OfpMeterFlags
+
+var OfpMeterFlags_name = openflow_13.OfpMeterFlags_name
+var OfpMeterFlags_value = openflow_13.OfpMeterFlags_value
+
+const OfpMeterFlags_OFPMF_INVALID = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_INVALID)
+const OfpMeterFlags_OFPMF_KBPS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_KBPS)
+const OfpMeterFlags_OFPMF_PKTPS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_PKTPS)
+const OfpMeterFlags_OFPMF_BURST = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_BURST)
+const OfpMeterFlags_OFPMF_STATS = OfpMeterFlags(openflow_13.OfpMeterFlags_OFPMF_STATS)
+
+// OfpErrorType from public import voltha_protos/openflow_13.proto
+type OfpErrorType = openflow_13.OfpErrorType
+
+var OfpErrorType_name = openflow_13.OfpErrorType_name
+var OfpErrorType_value = openflow_13.OfpErrorType_value
+
+const OfpErrorType_OFPET_HELLO_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_HELLO_FAILED)
+const OfpErrorType_OFPET_BAD_REQUEST = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_REQUEST)
+const OfpErrorType_OFPET_BAD_ACTION = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_ACTION)
+const OfpErrorType_OFPET_BAD_INSTRUCTION = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_INSTRUCTION)
+const OfpErrorType_OFPET_BAD_MATCH = OfpErrorType(openflow_13.OfpErrorType_OFPET_BAD_MATCH)
+const OfpErrorType_OFPET_FLOW_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_FLOW_MOD_FAILED)
+const OfpErrorType_OFPET_GROUP_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_GROUP_MOD_FAILED)
+const OfpErrorType_OFPET_PORT_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_PORT_MOD_FAILED)
+const OfpErrorType_OFPET_TABLE_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_TABLE_MOD_FAILED)
+const OfpErrorType_OFPET_QUEUE_OP_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_QUEUE_OP_FAILED)
+const OfpErrorType_OFPET_SWITCH_CONFIG_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_SWITCH_CONFIG_FAILED)
+const OfpErrorType_OFPET_ROLE_REQUEST_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_ROLE_REQUEST_FAILED)
+const OfpErrorType_OFPET_METER_MOD_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_METER_MOD_FAILED)
+const OfpErrorType_OFPET_TABLE_FEATURES_FAILED = OfpErrorType(openflow_13.OfpErrorType_OFPET_TABLE_FEATURES_FAILED)
+const OfpErrorType_OFPET_EXPERIMENTER = OfpErrorType(openflow_13.OfpErrorType_OFPET_EXPERIMENTER)
+
+// OfpHelloFailedCode from public import voltha_protos/openflow_13.proto
+type OfpHelloFailedCode = openflow_13.OfpHelloFailedCode
+
+var OfpHelloFailedCode_name = openflow_13.OfpHelloFailedCode_name
+var OfpHelloFailedCode_value = openflow_13.OfpHelloFailedCode_value
+
+const OfpHelloFailedCode_OFPHFC_INCOMPATIBLE = OfpHelloFailedCode(openflow_13.OfpHelloFailedCode_OFPHFC_INCOMPATIBLE)
+const OfpHelloFailedCode_OFPHFC_EPERM = OfpHelloFailedCode(openflow_13.OfpHelloFailedCode_OFPHFC_EPERM)
+
+// OfpBadRequestCode from public import voltha_protos/openflow_13.proto
+type OfpBadRequestCode = openflow_13.OfpBadRequestCode
+
+var OfpBadRequestCode_name = openflow_13.OfpBadRequestCode_name
+var OfpBadRequestCode_value = openflow_13.OfpBadRequestCode_value
+
+const OfpBadRequestCode_OFPBRC_BAD_VERSION = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_VERSION)
+const OfpBadRequestCode_OFPBRC_BAD_TYPE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_TYPE)
+const OfpBadRequestCode_OFPBRC_BAD_MULTIPART = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_MULTIPART)
+const OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_EXPERIMENTER)
+const OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_EXP_TYPE)
+const OfpBadRequestCode_OFPBRC_EPERM = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_EPERM)
+const OfpBadRequestCode_OFPBRC_BAD_LEN = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_LEN)
+const OfpBadRequestCode_OFPBRC_BUFFER_EMPTY = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BUFFER_EMPTY)
+const OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BUFFER_UNKNOWN)
+const OfpBadRequestCode_OFPBRC_BAD_TABLE_ID = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_TABLE_ID)
+const OfpBadRequestCode_OFPBRC_IS_SLAVE = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_IS_SLAVE)
+const OfpBadRequestCode_OFPBRC_BAD_PORT = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_PORT)
+const OfpBadRequestCode_OFPBRC_BAD_PACKET = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_BAD_PACKET)
+const OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW = OfpBadRequestCode(openflow_13.OfpBadRequestCode_OFPBRC_MULTIPART_BUFFER_OVERFLOW)
+
+// OfpBadActionCode from public import voltha_protos/openflow_13.proto
+type OfpBadActionCode = openflow_13.OfpBadActionCode
+
+var OfpBadActionCode_name = openflow_13.OfpBadActionCode_name
+var OfpBadActionCode_value = openflow_13.OfpBadActionCode_value
+
+const OfpBadActionCode_OFPBAC_BAD_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_TYPE)
+const OfpBadActionCode_OFPBAC_BAD_LEN = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_LEN)
+const OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_EXPERIMENTER)
+const OfpBadActionCode_OFPBAC_BAD_EXP_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_EXP_TYPE)
+const OfpBadActionCode_OFPBAC_BAD_OUT_PORT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_OUT_PORT)
+const OfpBadActionCode_OFPBAC_BAD_ARGUMENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_ARGUMENT)
+const OfpBadActionCode_OFPBAC_EPERM = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_EPERM)
+const OfpBadActionCode_OFPBAC_TOO_MANY = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_TOO_MANY)
+const OfpBadActionCode_OFPBAC_BAD_QUEUE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_QUEUE)
+const OfpBadActionCode_OFPBAC_BAD_OUT_GROUP = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_OUT_GROUP)
+const OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_MATCH_INCONSISTENT)
+const OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_UNSUPPORTED_ORDER)
+const OfpBadActionCode_OFPBAC_BAD_TAG = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_TAG)
+const OfpBadActionCode_OFPBAC_BAD_SET_TYPE = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_TYPE)
+const OfpBadActionCode_OFPBAC_BAD_SET_LEN = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_LEN)
+const OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT = OfpBadActionCode(openflow_13.OfpBadActionCode_OFPBAC_BAD_SET_ARGUMENT)
+
+// OfpBadInstructionCode from public import voltha_protos/openflow_13.proto
+type OfpBadInstructionCode = openflow_13.OfpBadInstructionCode
+
+var OfpBadInstructionCode_name = openflow_13.OfpBadInstructionCode_name
+var OfpBadInstructionCode_value = openflow_13.OfpBadInstructionCode_value
+
+const OfpBadInstructionCode_OFPBIC_UNKNOWN_INST = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNKNOWN_INST)
+const OfpBadInstructionCode_OFPBIC_UNSUP_INST = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_INST)
+const OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_TABLE_ID)
+const OfpBadInstructionCode_OFPBIC_UNSUP_METADATA = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_METADATA)
+const OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_UNSUP_METADATA_MASK)
+const OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_EXPERIMENTER)
+const OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_EXP_TYPE)
+const OfpBadInstructionCode_OFPBIC_BAD_LEN = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_BAD_LEN)
+const OfpBadInstructionCode_OFPBIC_EPERM = OfpBadInstructionCode(openflow_13.OfpBadInstructionCode_OFPBIC_EPERM)
+
+// OfpBadMatchCode from public import voltha_protos/openflow_13.proto
+type OfpBadMatchCode = openflow_13.OfpBadMatchCode
+
+var OfpBadMatchCode_name = openflow_13.OfpBadMatchCode_name
+var OfpBadMatchCode_value = openflow_13.OfpBadMatchCode_value
+
+const OfpBadMatchCode_OFPBMC_BAD_TYPE = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_TYPE)
+const OfpBadMatchCode_OFPBMC_BAD_LEN = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_LEN)
+const OfpBadMatchCode_OFPBMC_BAD_TAG = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_TAG)
+const OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_DL_ADDR_MASK)
+const OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_NW_ADDR_MASK)
+const OfpBadMatchCode_OFPBMC_BAD_WILDCARDS = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_WILDCARDS)
+const OfpBadMatchCode_OFPBMC_BAD_FIELD = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_FIELD)
+const OfpBadMatchCode_OFPBMC_BAD_VALUE = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_VALUE)
+const OfpBadMatchCode_OFPBMC_BAD_MASK = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_MASK)
+const OfpBadMatchCode_OFPBMC_BAD_PREREQ = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_BAD_PREREQ)
+const OfpBadMatchCode_OFPBMC_DUP_FIELD = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_DUP_FIELD)
+const OfpBadMatchCode_OFPBMC_EPERM = OfpBadMatchCode(openflow_13.OfpBadMatchCode_OFPBMC_EPERM)
+
+// OfpFlowModFailedCode from public import voltha_protos/openflow_13.proto
+type OfpFlowModFailedCode = openflow_13.OfpFlowModFailedCode
+
+var OfpFlowModFailedCode_name = openflow_13.OfpFlowModFailedCode_name
+var OfpFlowModFailedCode_value = openflow_13.OfpFlowModFailedCode_value
+
+const OfpFlowModFailedCode_OFPFMFC_UNKNOWN = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_UNKNOWN)
+const OfpFlowModFailedCode_OFPFMFC_TABLE_FULL = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_TABLE_FULL)
+const OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_TABLE_ID)
+const OfpFlowModFailedCode_OFPFMFC_OVERLAP = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_OVERLAP)
+const OfpFlowModFailedCode_OFPFMFC_EPERM = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_EPERM)
+const OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_TIMEOUT)
+const OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_COMMAND)
+const OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS = OfpFlowModFailedCode(openflow_13.OfpFlowModFailedCode_OFPFMFC_BAD_FLAGS)
+
+// OfpGroupModFailedCode from public import voltha_protos/openflow_13.proto
+type OfpGroupModFailedCode = openflow_13.OfpGroupModFailedCode
+
+var OfpGroupModFailedCode_name = openflow_13.OfpGroupModFailedCode_name
+var OfpGroupModFailedCode_value = openflow_13.OfpGroupModFailedCode_value
+
+const OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_GROUP_EXISTS)
+const OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_INVALID_GROUP)
+const OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_WEIGHT_UNSUPPORTED)
+const OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_OUT_OF_GROUPS)
+const OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_OUT_OF_BUCKETS)
+const OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_CHAINING_UNSUPPORTED)
+const OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_WATCH_UNSUPPORTED)
+const OfpGroupModFailedCode_OFPGMFC_LOOP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_LOOP)
+const OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_UNKNOWN_GROUP)
+const OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_CHAINED_GROUP)
+const OfpGroupModFailedCode_OFPGMFC_BAD_TYPE = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_TYPE)
+const OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_COMMAND)
+const OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_BUCKET)
+const OfpGroupModFailedCode_OFPGMFC_BAD_WATCH = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_BAD_WATCH)
+const OfpGroupModFailedCode_OFPGMFC_EPERM = OfpGroupModFailedCode(openflow_13.OfpGroupModFailedCode_OFPGMFC_EPERM)
+
+// OfpPortModFailedCode from public import voltha_protos/openflow_13.proto
+type OfpPortModFailedCode = openflow_13.OfpPortModFailedCode
+
+var OfpPortModFailedCode_name = openflow_13.OfpPortModFailedCode_name
+var OfpPortModFailedCode_value = openflow_13.OfpPortModFailedCode_value
+
+const OfpPortModFailedCode_OFPPMFC_BAD_PORT = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_PORT)
+const OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_HW_ADDR)
+const OfpPortModFailedCode_OFPPMFC_BAD_CONFIG = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_CONFIG)
+const OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_BAD_ADVERTISE)
+const OfpPortModFailedCode_OFPPMFC_EPERM = OfpPortModFailedCode(openflow_13.OfpPortModFailedCode_OFPPMFC_EPERM)
+
+// OfpTableModFailedCode from public import voltha_protos/openflow_13.proto
+type OfpTableModFailedCode = openflow_13.OfpTableModFailedCode
+
+var OfpTableModFailedCode_name = openflow_13.OfpTableModFailedCode_name
+var OfpTableModFailedCode_value = openflow_13.OfpTableModFailedCode_value
+
+const OfpTableModFailedCode_OFPTMFC_BAD_TABLE = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_BAD_TABLE)
+const OfpTableModFailedCode_OFPTMFC_BAD_CONFIG = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_BAD_CONFIG)
+const OfpTableModFailedCode_OFPTMFC_EPERM = OfpTableModFailedCode(openflow_13.OfpTableModFailedCode_OFPTMFC_EPERM)
+
+// OfpQueueOpFailedCode from public import voltha_protos/openflow_13.proto
+type OfpQueueOpFailedCode = openflow_13.OfpQueueOpFailedCode
+
+var OfpQueueOpFailedCode_name = openflow_13.OfpQueueOpFailedCode_name
+var OfpQueueOpFailedCode_value = openflow_13.OfpQueueOpFailedCode_value
+
+const OfpQueueOpFailedCode_OFPQOFC_BAD_PORT = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_BAD_PORT)
+const OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_BAD_QUEUE)
+const OfpQueueOpFailedCode_OFPQOFC_EPERM = OfpQueueOpFailedCode(openflow_13.OfpQueueOpFailedCode_OFPQOFC_EPERM)
+
+// OfpSwitchConfigFailedCode from public import voltha_protos/openflow_13.proto
+type OfpSwitchConfigFailedCode = openflow_13.OfpSwitchConfigFailedCode
+
+var OfpSwitchConfigFailedCode_name = openflow_13.OfpSwitchConfigFailedCode_name
+var OfpSwitchConfigFailedCode_value = openflow_13.OfpSwitchConfigFailedCode_value
+
+const OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_BAD_FLAGS)
+const OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_BAD_LEN)
+const OfpSwitchConfigFailedCode_OFPSCFC_EPERM = OfpSwitchConfigFailedCode(openflow_13.OfpSwitchConfigFailedCode_OFPSCFC_EPERM)
+
+// OfpRoleRequestFailedCode from public import voltha_protos/openflow_13.proto
+type OfpRoleRequestFailedCode = openflow_13.OfpRoleRequestFailedCode
+
+var OfpRoleRequestFailedCode_name = openflow_13.OfpRoleRequestFailedCode_name
+var OfpRoleRequestFailedCode_value = openflow_13.OfpRoleRequestFailedCode_value
+
+const OfpRoleRequestFailedCode_OFPRRFC_STALE = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_STALE)
+const OfpRoleRequestFailedCode_OFPRRFC_UNSUP = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_UNSUP)
+const OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE = OfpRoleRequestFailedCode(openflow_13.OfpRoleRequestFailedCode_OFPRRFC_BAD_ROLE)
+
+// OfpMeterModFailedCode from public import voltha_protos/openflow_13.proto
+type OfpMeterModFailedCode = openflow_13.OfpMeterModFailedCode
+
+var OfpMeterModFailedCode_name = openflow_13.OfpMeterModFailedCode_name
+var OfpMeterModFailedCode_value = openflow_13.OfpMeterModFailedCode_value
+
+const OfpMeterModFailedCode_OFPMMFC_UNKNOWN = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_UNKNOWN)
+const OfpMeterModFailedCode_OFPMMFC_METER_EXISTS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_METER_EXISTS)
+const OfpMeterModFailedCode_OFPMMFC_INVALID_METER = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_INVALID_METER)
+const OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_UNKNOWN_METER)
+const OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_COMMAND)
+const OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_FLAGS)
+const OfpMeterModFailedCode_OFPMMFC_BAD_RATE = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_RATE)
+const OfpMeterModFailedCode_OFPMMFC_BAD_BURST = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BURST)
+const OfpMeterModFailedCode_OFPMMFC_BAD_BAND = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BAND)
+const OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_BAD_BAND_DETAIL)
+const OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_OUT_OF_METERS)
+const OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS = OfpMeterModFailedCode(openflow_13.OfpMeterModFailedCode_OFPMMFC_OUT_OF_BANDS)
+
+// OfpTableFeaturesFailedCode from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturesFailedCode = openflow_13.OfpTableFeaturesFailedCode
+
+var OfpTableFeaturesFailedCode_name = openflow_13.OfpTableFeaturesFailedCode_name
+var OfpTableFeaturesFailedCode_value = openflow_13.OfpTableFeaturesFailedCode_value
+
+const OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_TABLE)
+const OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_METADATA)
+const OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_TYPE)
+const OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_LEN)
+const OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_BAD_ARGUMENT)
+const OfpTableFeaturesFailedCode_OFPTFFC_EPERM = OfpTableFeaturesFailedCode(openflow_13.OfpTableFeaturesFailedCode_OFPTFFC_EPERM)
+
+// OfpMultipartType from public import voltha_protos/openflow_13.proto
+type OfpMultipartType = openflow_13.OfpMultipartType
+
+var OfpMultipartType_name = openflow_13.OfpMultipartType_name
+var OfpMultipartType_value = openflow_13.OfpMultipartType_value
+
+const OfpMultipartType_OFPMP_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_DESC)
+const OfpMultipartType_OFPMP_FLOW = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_FLOW)
+const OfpMultipartType_OFPMP_AGGREGATE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_AGGREGATE)
+const OfpMultipartType_OFPMP_TABLE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_TABLE)
+const OfpMultipartType_OFPMP_PORT_STATS = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_PORT_STATS)
+const OfpMultipartType_OFPMP_QUEUE = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_QUEUE)
+const OfpMultipartType_OFPMP_GROUP = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP)
+const OfpMultipartType_OFPMP_GROUP_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP_DESC)
+const OfpMultipartType_OFPMP_GROUP_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_GROUP_FEATURES)
+const OfpMultipartType_OFPMP_METER = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER)
+const OfpMultipartType_OFPMP_METER_CONFIG = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER_CONFIG)
+const OfpMultipartType_OFPMP_METER_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_METER_FEATURES)
+const OfpMultipartType_OFPMP_TABLE_FEATURES = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_TABLE_FEATURES)
+const OfpMultipartType_OFPMP_PORT_DESC = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_PORT_DESC)
+const OfpMultipartType_OFPMP_EXPERIMENTER = OfpMultipartType(openflow_13.OfpMultipartType_OFPMP_EXPERIMENTER)
+
+// OfpMultipartRequestFlags from public import voltha_protos/openflow_13.proto
+type OfpMultipartRequestFlags = openflow_13.OfpMultipartRequestFlags
+
+var OfpMultipartRequestFlags_name = openflow_13.OfpMultipartRequestFlags_name
+var OfpMultipartRequestFlags_value = openflow_13.OfpMultipartRequestFlags_value
+
+const OfpMultipartRequestFlags_OFPMPF_REQ_INVALID = OfpMultipartRequestFlags(openflow_13.OfpMultipartRequestFlags_OFPMPF_REQ_INVALID)
+const OfpMultipartRequestFlags_OFPMPF_REQ_MORE = OfpMultipartRequestFlags(openflow_13.OfpMultipartRequestFlags_OFPMPF_REQ_MORE)
+
+// OfpMultipartReplyFlags from public import voltha_protos/openflow_13.proto
+type OfpMultipartReplyFlags = openflow_13.OfpMultipartReplyFlags
+
+var OfpMultipartReplyFlags_name = openflow_13.OfpMultipartReplyFlags_name
+var OfpMultipartReplyFlags_value = openflow_13.OfpMultipartReplyFlags_value
+
+const OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID = OfpMultipartReplyFlags(openflow_13.OfpMultipartReplyFlags_OFPMPF_REPLY_INVALID)
+const OfpMultipartReplyFlags_OFPMPF_REPLY_MORE = OfpMultipartReplyFlags(openflow_13.OfpMultipartReplyFlags_OFPMPF_REPLY_MORE)
+
+// OfpTableFeaturePropType from public import voltha_protos/openflow_13.proto
+type OfpTableFeaturePropType = openflow_13.OfpTableFeaturePropType
+
+var OfpTableFeaturePropType_name = openflow_13.OfpTableFeaturePropType_name
+var OfpTableFeaturePropType_value = openflow_13.OfpTableFeaturePropType_value
+
+const OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS)
+const OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_INSTRUCTIONS_MISS)
+const OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES)
+const OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_NEXT_TABLES_MISS)
+const OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS)
+const OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_ACTIONS_MISS)
+const OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS)
+const OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_ACTIONS_MISS)
+const OfpTableFeaturePropType_OFPTFPT_MATCH = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_MATCH)
+const OfpTableFeaturePropType_OFPTFPT_WILDCARDS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WILDCARDS)
+const OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD)
+const OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_WRITE_SETFIELD_MISS)
+const OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD)
+const OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_APPLY_SETFIELD_MISS)
+const OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER)
+const OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS = OfpTableFeaturePropType(openflow_13.OfpTableFeaturePropType_OFPTFPT_EXPERIMENTER_MISS)
+
+// OfpGroupCapabilities from public import voltha_protos/openflow_13.proto
+type OfpGroupCapabilities = openflow_13.OfpGroupCapabilities
+
+var OfpGroupCapabilities_name = openflow_13.OfpGroupCapabilities_name
+var OfpGroupCapabilities_value = openflow_13.OfpGroupCapabilities_value
+
+const OfpGroupCapabilities_OFPGFC_INVALID = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_INVALID)
+const OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_SELECT_WEIGHT)
+const OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_SELECT_LIVENESS)
+const OfpGroupCapabilities_OFPGFC_CHAINING = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_CHAINING)
+const OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS = OfpGroupCapabilities(openflow_13.OfpGroupCapabilities_OFPGFC_CHAINING_CHECKS)
+
+// OfpQueueProperties from public import voltha_protos/openflow_13.proto
+type OfpQueueProperties = openflow_13.OfpQueueProperties
+
+var OfpQueueProperties_name = openflow_13.OfpQueueProperties_name
+var OfpQueueProperties_value = openflow_13.OfpQueueProperties_value
+
+const OfpQueueProperties_OFPQT_INVALID = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_INVALID)
+const OfpQueueProperties_OFPQT_MIN_RATE = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_MIN_RATE)
+const OfpQueueProperties_OFPQT_MAX_RATE = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_MAX_RATE)
+const OfpQueueProperties_OFPQT_EXPERIMENTER = OfpQueueProperties(openflow_13.OfpQueueProperties_OFPQT_EXPERIMENTER)
+
+// OfpControllerRole from public import voltha_protos/openflow_13.proto
+type OfpControllerRole = openflow_13.OfpControllerRole
+
+var OfpControllerRole_name = openflow_13.OfpControllerRole_name
+var OfpControllerRole_value = openflow_13.OfpControllerRole_value
+
+const OfpControllerRole_OFPCR_ROLE_NOCHANGE = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_NOCHANGE)
+const OfpControllerRole_OFPCR_ROLE_EQUAL = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_EQUAL)
+const OfpControllerRole_OFPCR_ROLE_MASTER = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_MASTER)
+const OfpControllerRole_OFPCR_ROLE_SLAVE = OfpControllerRole(openflow_13.OfpControllerRole_OFPCR_ROLE_SLAVE)
+
+type EventFilterRuleKey_EventFilterRuleType int32
+
+const (
+	EventFilterRuleKey_filter_all        EventFilterRuleKey_EventFilterRuleType = 0
+	EventFilterRuleKey_category          EventFilterRuleKey_EventFilterRuleType = 1
+	EventFilterRuleKey_sub_category      EventFilterRuleKey_EventFilterRuleType = 2
+	EventFilterRuleKey_kpi_event_type    EventFilterRuleKey_EventFilterRuleType = 3
+	EventFilterRuleKey_config_event_type EventFilterRuleKey_EventFilterRuleType = 4
+	EventFilterRuleKey_device_event_type EventFilterRuleKey_EventFilterRuleType = 5
+)
+
+var EventFilterRuleKey_EventFilterRuleType_name = map[int32]string{
+	0: "filter_all",
+	1: "category",
+	2: "sub_category",
+	3: "kpi_event_type",
+	4: "config_event_type",
+	5: "device_event_type",
+}
+
+var EventFilterRuleKey_EventFilterRuleType_value = map[string]int32{
+	"filter_all":        0,
+	"category":          1,
+	"sub_category":      2,
+	"kpi_event_type":    3,
+	"config_event_type": 4,
+	"device_event_type": 5,
+}
+
+func (x EventFilterRuleKey_EventFilterRuleType) String() string {
+	return proto.EnumName(EventFilterRuleKey_EventFilterRuleType_name, int32(x))
+}
+
+func (EventFilterRuleKey_EventFilterRuleType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{2, 0}
+}
+
+type TestResponse_TestResponseResult int32
+
+const (
+	TestResponse_SUCCESS TestResponse_TestResponseResult = 0
+	TestResponse_FAILURE TestResponse_TestResponseResult = 1
+)
+
+var TestResponse_TestResponseResult_name = map[int32]string{
+	0: "SUCCESS",
+	1: "FAILURE",
+}
+
+var TestResponse_TestResponseResult_value = map[string]int32{
+	"SUCCESS": 0,
+	"FAILURE": 1,
+}
+
+func (x TestResponse_TestResponseResult) String() string {
+	return proto.EnumName(TestResponse_TestResponseResult_name, int32(x))
+}
+
+func (TestResponse_TestResponseResult) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{9, 0}
+}
+
+type SelfTestResponse_SelfTestResult int32
+
+const (
+	SelfTestResponse_SUCCESS       SelfTestResponse_SelfTestResult = 0
+	SelfTestResponse_FAILURE       SelfTestResponse_SelfTestResult = 1
+	SelfTestResponse_NOT_SUPPORTED SelfTestResponse_SelfTestResult = 2
+	SelfTestResponse_UNKNOWN_ERROR SelfTestResponse_SelfTestResult = 3
+)
+
+var SelfTestResponse_SelfTestResult_name = map[int32]string{
+	0: "SUCCESS",
+	1: "FAILURE",
+	2: "NOT_SUPPORTED",
+	3: "UNKNOWN_ERROR",
+}
+
+var SelfTestResponse_SelfTestResult_value = map[string]int32{
+	"SUCCESS":       0,
+	"FAILURE":       1,
+	"NOT_SUPPORTED": 2,
+	"UNKNOWN_ERROR": 3,
+}
+
+func (x SelfTestResponse_SelfTestResult) String() string {
+	return proto.EnumName(SelfTestResponse_SelfTestResult_name, int32(x))
+}
+
+func (SelfTestResponse_SelfTestResult) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{12, 0}
+}
+
+type DeviceGroup struct {
+	Id                   string           `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	LogicalDevices       []*LogicalDevice `protobuf:"bytes,2,rep,name=logical_devices,json=logicalDevices,proto3" json:"logical_devices,omitempty"`
+	Devices              []*Device        `protobuf:"bytes,3,rep,name=devices,proto3" json:"devices,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *DeviceGroup) Reset()         { *m = DeviceGroup{} }
+func (m *DeviceGroup) String() string { return proto.CompactTextString(m) }
+func (*DeviceGroup) ProtoMessage()    {}
+func (*DeviceGroup) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{0}
+}
+
+func (m *DeviceGroup) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceGroup.Unmarshal(m, b)
+}
+func (m *DeviceGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceGroup.Marshal(b, m, deterministic)
+}
+func (m *DeviceGroup) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceGroup.Merge(m, src)
+}
+func (m *DeviceGroup) XXX_Size() int {
+	return xxx_messageInfo_DeviceGroup.Size(m)
+}
+func (m *DeviceGroup) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceGroup.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceGroup proto.InternalMessageInfo
+
+func (m *DeviceGroup) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *DeviceGroup) GetLogicalDevices() []*LogicalDevice {
+	if m != nil {
+		return m.LogicalDevices
+	}
+	return nil
+}
+
+func (m *DeviceGroup) GetDevices() []*Device {
+	if m != nil {
+		return m.Devices
+	}
+	return nil
+}
+
+type DeviceGroups struct {
+	Items                []*DeviceGroup `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *DeviceGroups) Reset()         { *m = DeviceGroups{} }
+func (m *DeviceGroups) String() string { return proto.CompactTextString(m) }
+func (*DeviceGroups) ProtoMessage()    {}
+func (*DeviceGroups) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{1}
+}
+
+func (m *DeviceGroups) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceGroups.Unmarshal(m, b)
+}
+func (m *DeviceGroups) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceGroups.Marshal(b, m, deterministic)
+}
+func (m *DeviceGroups) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceGroups.Merge(m, src)
+}
+func (m *DeviceGroups) XXX_Size() int {
+	return xxx_messageInfo_DeviceGroups.Size(m)
+}
+func (m *DeviceGroups) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceGroups.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceGroups proto.InternalMessageInfo
+
+func (m *DeviceGroups) GetItems() []*DeviceGroup {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type EventFilterRuleKey struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EventFilterRuleKey) Reset()         { *m = EventFilterRuleKey{} }
+func (m *EventFilterRuleKey) String() string { return proto.CompactTextString(m) }
+func (*EventFilterRuleKey) ProtoMessage()    {}
+func (*EventFilterRuleKey) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{2}
+}
+
+func (m *EventFilterRuleKey) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventFilterRuleKey.Unmarshal(m, b)
+}
+func (m *EventFilterRuleKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventFilterRuleKey.Marshal(b, m, deterministic)
+}
+func (m *EventFilterRuleKey) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventFilterRuleKey.Merge(m, src)
+}
+func (m *EventFilterRuleKey) XXX_Size() int {
+	return xxx_messageInfo_EventFilterRuleKey.Size(m)
+}
+func (m *EventFilterRuleKey) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventFilterRuleKey.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventFilterRuleKey proto.InternalMessageInfo
+
+type EventFilterRule struct {
+	Key                  EventFilterRuleKey_EventFilterRuleType `protobuf:"varint,1,opt,name=key,proto3,enum=voltha.EventFilterRuleKey_EventFilterRuleType" json:"key,omitempty"`
+	Value                string                                 `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                               `json:"-"`
+	XXX_unrecognized     []byte                                 `json:"-"`
+	XXX_sizecache        int32                                  `json:"-"`
+}
+
+func (m *EventFilterRule) Reset()         { *m = EventFilterRule{} }
+func (m *EventFilterRule) String() string { return proto.CompactTextString(m) }
+func (*EventFilterRule) ProtoMessage()    {}
+func (*EventFilterRule) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{3}
+}
+
+func (m *EventFilterRule) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventFilterRule.Unmarshal(m, b)
+}
+func (m *EventFilterRule) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventFilterRule.Marshal(b, m, deterministic)
+}
+func (m *EventFilterRule) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventFilterRule.Merge(m, src)
+}
+func (m *EventFilterRule) XXX_Size() int {
+	return xxx_messageInfo_EventFilterRule.Size(m)
+}
+func (m *EventFilterRule) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventFilterRule.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventFilterRule proto.InternalMessageInfo
+
+func (m *EventFilterRule) GetKey() EventFilterRuleKey_EventFilterRuleType {
+	if m != nil {
+		return m.Key
+	}
+	return EventFilterRuleKey_filter_all
+}
+
+func (m *EventFilterRule) GetValue() string {
+	if m != nil {
+		return m.Value
+	}
+	return ""
+}
+
+type EventFilter struct {
+	Id                   string             `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	Enable               bool               `protobuf:"varint,2,opt,name=enable,proto3" json:"enable,omitempty"`
+	DeviceId             string             `protobuf:"bytes,3,opt,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	EventType            string             `protobuf:"bytes,4,opt,name=event_type,json=eventType,proto3" json:"event_type,omitempty"`
+	Rules                []*EventFilterRule `protobuf:"bytes,5,rep,name=rules,proto3" json:"rules,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *EventFilter) Reset()         { *m = EventFilter{} }
+func (m *EventFilter) String() string { return proto.CompactTextString(m) }
+func (*EventFilter) ProtoMessage()    {}
+func (*EventFilter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{4}
+}
+
+func (m *EventFilter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventFilter.Unmarshal(m, b)
+}
+func (m *EventFilter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventFilter.Marshal(b, m, deterministic)
+}
+func (m *EventFilter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventFilter.Merge(m, src)
+}
+func (m *EventFilter) XXX_Size() int {
+	return xxx_messageInfo_EventFilter.Size(m)
+}
+func (m *EventFilter) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventFilter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventFilter proto.InternalMessageInfo
+
+func (m *EventFilter) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *EventFilter) GetEnable() bool {
+	if m != nil {
+		return m.Enable
+	}
+	return false
+}
+
+func (m *EventFilter) GetDeviceId() string {
+	if m != nil {
+		return m.DeviceId
+	}
+	return ""
+}
+
+func (m *EventFilter) GetEventType() string {
+	if m != nil {
+		return m.EventType
+	}
+	return ""
+}
+
+func (m *EventFilter) GetRules() []*EventFilterRule {
+	if m != nil {
+		return m.Rules
+	}
+	return nil
+}
+
+type EventFilters struct {
+	Filters              []*EventFilter `protobuf:"bytes,1,rep,name=filters,proto3" json:"filters,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
+	XXX_unrecognized     []byte         `json:"-"`
+	XXX_sizecache        int32          `json:"-"`
+}
+
+func (m *EventFilters) Reset()         { *m = EventFilters{} }
+func (m *EventFilters) String() string { return proto.CompactTextString(m) }
+func (*EventFilters) ProtoMessage()    {}
+func (*EventFilters) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{5}
+}
+
+func (m *EventFilters) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventFilters.Unmarshal(m, b)
+}
+func (m *EventFilters) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventFilters.Marshal(b, m, deterministic)
+}
+func (m *EventFilters) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventFilters.Merge(m, src)
+}
+func (m *EventFilters) XXX_Size() int {
+	return xxx_messageInfo_EventFilters.Size(m)
+}
+func (m *EventFilters) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventFilters.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventFilters proto.InternalMessageInfo
+
+func (m *EventFilters) GetFilters() []*EventFilter {
+	if m != nil {
+		return m.Filters
+	}
+	return nil
+}
+
+// CoreInstance represents a core instance.  It is data held in memory when a core
+// is running.  This data is not persistent.
+type CoreInstance struct {
+	InstanceId           string        `protobuf:"bytes,1,opt,name=instance_id,json=instanceId,proto3" json:"instance_id,omitempty"`
+	Health               *HealthStatus `protobuf:"bytes,2,opt,name=health,proto3" json:"health,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
+	XXX_unrecognized     []byte        `json:"-"`
+	XXX_sizecache        int32         `json:"-"`
+}
+
+func (m *CoreInstance) Reset()         { *m = CoreInstance{} }
+func (m *CoreInstance) String() string { return proto.CompactTextString(m) }
+func (*CoreInstance) ProtoMessage()    {}
+func (*CoreInstance) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{6}
+}
+
+func (m *CoreInstance) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_CoreInstance.Unmarshal(m, b)
+}
+func (m *CoreInstance) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_CoreInstance.Marshal(b, m, deterministic)
+}
+func (m *CoreInstance) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CoreInstance.Merge(m, src)
+}
+func (m *CoreInstance) XXX_Size() int {
+	return xxx_messageInfo_CoreInstance.Size(m)
+}
+func (m *CoreInstance) XXX_DiscardUnknown() {
+	xxx_messageInfo_CoreInstance.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CoreInstance proto.InternalMessageInfo
+
+func (m *CoreInstance) GetInstanceId() string {
+	if m != nil {
+		return m.InstanceId
+	}
+	return ""
+}
+
+func (m *CoreInstance) GetHealth() *HealthStatus {
+	if m != nil {
+		return m.Health
+	}
+	return nil
+}
+
+type CoreInstances struct {
+	Items                []*CoreInstance `protobuf:"bytes,1,rep,name=items,proto3" json:"items,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
+	XXX_unrecognized     []byte          `json:"-"`
+	XXX_sizecache        int32           `json:"-"`
+}
+
+func (m *CoreInstances) Reset()         { *m = CoreInstances{} }
+func (m *CoreInstances) String() string { return proto.CompactTextString(m) }
+func (*CoreInstances) ProtoMessage()    {}
+func (*CoreInstances) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{7}
+}
+
+func (m *CoreInstances) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_CoreInstances.Unmarshal(m, b)
+}
+func (m *CoreInstances) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_CoreInstances.Marshal(b, m, deterministic)
+}
+func (m *CoreInstances) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CoreInstances.Merge(m, src)
+}
+func (m *CoreInstances) XXX_Size() int {
+	return xxx_messageInfo_CoreInstances.Size(m)
+}
+func (m *CoreInstances) XXX_DiscardUnknown() {
+	xxx_messageInfo_CoreInstances.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CoreInstances proto.InternalMessageInfo
+
+func (m *CoreInstances) GetItems() []*CoreInstance {
+	if m != nil {
+		return m.Items
+	}
+	return nil
+}
+
+type OmciTestRequest struct {
+	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	Uuid                 string   `protobuf:"bytes,2,opt,name=uuid,proto3" json:"uuid,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OmciTestRequest) Reset()         { *m = OmciTestRequest{} }
+func (m *OmciTestRequest) String() string { return proto.CompactTextString(m) }
+func (*OmciTestRequest) ProtoMessage()    {}
+func (*OmciTestRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{8}
+}
+
+func (m *OmciTestRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OmciTestRequest.Unmarshal(m, b)
+}
+func (m *OmciTestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OmciTestRequest.Marshal(b, m, deterministic)
+}
+func (m *OmciTestRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OmciTestRequest.Merge(m, src)
+}
+func (m *OmciTestRequest) XXX_Size() int {
+	return xxx_messageInfo_OmciTestRequest.Size(m)
+}
+func (m *OmciTestRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_OmciTestRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OmciTestRequest proto.InternalMessageInfo
+
+func (m *OmciTestRequest) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *OmciTestRequest) GetUuid() string {
+	if m != nil {
+		return m.Uuid
+	}
+	return ""
+}
+
+type TestResponse struct {
+	Result               TestResponse_TestResponseResult `protobuf:"varint,1,opt,name=result,proto3,enum=voltha.TestResponse_TestResponseResult" json:"result,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
+	XXX_unrecognized     []byte                          `json:"-"`
+	XXX_sizecache        int32                           `json:"-"`
+}
+
+func (m *TestResponse) Reset()         { *m = TestResponse{} }
+func (m *TestResponse) String() string { return proto.CompactTextString(m) }
+func (*TestResponse) ProtoMessage()    {}
+func (*TestResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{9}
+}
+
+func (m *TestResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_TestResponse.Unmarshal(m, b)
+}
+func (m *TestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_TestResponse.Marshal(b, m, deterministic)
+}
+func (m *TestResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestResponse.Merge(m, src)
+}
+func (m *TestResponse) XXX_Size() int {
+	return xxx_messageInfo_TestResponse.Size(m)
+}
+func (m *TestResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_TestResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_TestResponse proto.InternalMessageInfo
+
+func (m *TestResponse) GetResult() TestResponse_TestResponseResult {
+	if m != nil {
+		return m.Result
+	}
+	return TestResponse_SUCCESS
+}
+
+type ValueSet struct {
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Types that are valid to be assigned to Value:
+	//	*ValueSet_AlarmConfig
+	Value                isValueSet_Value `protobuf_oneof:"value"`
+	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
+	XXX_unrecognized     []byte           `json:"-"`
+	XXX_sizecache        int32            `json:"-"`
+}
+
+func (m *ValueSet) Reset()         { *m = ValueSet{} }
+func (m *ValueSet) String() string { return proto.CompactTextString(m) }
+func (*ValueSet) ProtoMessage()    {}
+func (*ValueSet) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{10}
+}
+
+func (m *ValueSet) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ValueSet.Unmarshal(m, b)
+}
+func (m *ValueSet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ValueSet.Marshal(b, m, deterministic)
+}
+func (m *ValueSet) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ValueSet.Merge(m, src)
+}
+func (m *ValueSet) XXX_Size() int {
+	return xxx_messageInfo_ValueSet.Size(m)
+}
+func (m *ValueSet) XXX_DiscardUnknown() {
+	xxx_messageInfo_ValueSet.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ValueSet proto.InternalMessageInfo
+
+func (m *ValueSet) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+type isValueSet_Value interface {
+	isValueSet_Value()
+}
+
+type ValueSet_AlarmConfig struct {
+	AlarmConfig *config.AlarmConfig `protobuf:"bytes,2,opt,name=alarm_config,json=alarmConfig,proto3,oneof"`
+}
+
+func (*ValueSet_AlarmConfig) isValueSet_Value() {}
+
+func (m *ValueSet) GetValue() isValueSet_Value {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+func (m *ValueSet) GetAlarmConfig() *config.AlarmConfig {
+	if x, ok := m.GetValue().(*ValueSet_AlarmConfig); ok {
+		return x.AlarmConfig
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*ValueSet) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*ValueSet_AlarmConfig)(nil),
+	}
+}
+
+// Voltha represents the Voltha cluster data.  Each Core instance will hold a subset of
+// the entire cluster. However, some items (e.g. adapters) will be held by all cores
+// for better performance
+type Voltha struct {
+	Version              string                  `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"`
+	Adapters             []*Adapter              `protobuf:"bytes,2,rep,name=adapters,proto3" json:"adapters,omitempty"`
+	LogicalDevices       []*LogicalDevice        `protobuf:"bytes,3,rep,name=logical_devices,json=logicalDevices,proto3" json:"logical_devices,omitempty"`
+	Devices              []*Device               `protobuf:"bytes,4,rep,name=devices,proto3" json:"devices,omitempty"`
+	DeviceTypes          []*DeviceType           `protobuf:"bytes,5,rep,name=device_types,json=deviceTypes,proto3" json:"device_types,omitempty"`
+	DeviceGroups         []*DeviceGroup          `protobuf:"bytes,6,rep,name=device_groups,json=deviceGroups,proto3" json:"device_groups,omitempty"`
+	EventFilters         []*EventFilter          `protobuf:"bytes,7,rep,name=event_filters,json=eventFilters,proto3" json:"event_filters,omitempty"`
+	OmciMibDatabase      []*omci.MibDeviceData   `protobuf:"bytes,28,rep,name=omci_mib_database,json=omciMibDatabase,proto3" json:"omci_mib_database,omitempty"`
+	OmciAlarmDatabase    []*omci.AlarmDeviceData `protobuf:"bytes,29,rep,name=omci_alarm_database,json=omciAlarmDatabase,proto3" json:"omci_alarm_database,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
+	XXX_unrecognized     []byte                  `json:"-"`
+	XXX_sizecache        int32                   `json:"-"`
+}
+
+func (m *Voltha) Reset()         { *m = Voltha{} }
+func (m *Voltha) String() string { return proto.CompactTextString(m) }
+func (*Voltha) ProtoMessage()    {}
+func (*Voltha) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{11}
+}
+
+func (m *Voltha) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Voltha.Unmarshal(m, b)
+}
+func (m *Voltha) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Voltha.Marshal(b, m, deterministic)
+}
+func (m *Voltha) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Voltha.Merge(m, src)
+}
+func (m *Voltha) XXX_Size() int {
+	return xxx_messageInfo_Voltha.Size(m)
+}
+func (m *Voltha) XXX_DiscardUnknown() {
+	xxx_messageInfo_Voltha.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Voltha proto.InternalMessageInfo
+
+func (m *Voltha) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *Voltha) GetAdapters() []*Adapter {
+	if m != nil {
+		return m.Adapters
+	}
+	return nil
+}
+
+func (m *Voltha) GetLogicalDevices() []*LogicalDevice {
+	if m != nil {
+		return m.LogicalDevices
+	}
+	return nil
+}
+
+func (m *Voltha) GetDevices() []*Device {
+	if m != nil {
+		return m.Devices
+	}
+	return nil
+}
+
+func (m *Voltha) GetDeviceTypes() []*DeviceType {
+	if m != nil {
+		return m.DeviceTypes
+	}
+	return nil
+}
+
+func (m *Voltha) GetDeviceGroups() []*DeviceGroup {
+	if m != nil {
+		return m.DeviceGroups
+	}
+	return nil
+}
+
+func (m *Voltha) GetEventFilters() []*EventFilter {
+	if m != nil {
+		return m.EventFilters
+	}
+	return nil
+}
+
+func (m *Voltha) GetOmciMibDatabase() []*omci.MibDeviceData {
+	if m != nil {
+		return m.OmciMibDatabase
+	}
+	return nil
+}
+
+func (m *Voltha) GetOmciAlarmDatabase() []*omci.AlarmDeviceData {
+	if m != nil {
+		return m.OmciAlarmDatabase
+	}
+	return nil
+}
+
+// Device Self Test Response
+type SelfTestResponse struct {
+	Result               SelfTestResponse_SelfTestResult `protobuf:"varint,1,opt,name=result,proto3,enum=voltha.SelfTestResponse_SelfTestResult" json:"result,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
+	XXX_unrecognized     []byte                          `json:"-"`
+	XXX_sizecache        int32                           `json:"-"`
+}
+
+func (m *SelfTestResponse) Reset()         { *m = SelfTestResponse{} }
+func (m *SelfTestResponse) String() string { return proto.CompactTextString(m) }
+func (*SelfTestResponse) ProtoMessage()    {}
+func (*SelfTestResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{12}
+}
+
+func (m *SelfTestResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SelfTestResponse.Unmarshal(m, b)
+}
+func (m *SelfTestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SelfTestResponse.Marshal(b, m, deterministic)
+}
+func (m *SelfTestResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SelfTestResponse.Merge(m, src)
+}
+func (m *SelfTestResponse) XXX_Size() int {
+	return xxx_messageInfo_SelfTestResponse.Size(m)
+}
+func (m *SelfTestResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_SelfTestResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SelfTestResponse proto.InternalMessageInfo
+
+func (m *SelfTestResponse) GetResult() SelfTestResponse_SelfTestResult {
+	if m != nil {
+		return m.Result
+	}
+	return SelfTestResponse_SUCCESS
+}
+
+type OfAgentSubscriber struct {
+	// ID of ofagent instance
+	OfagentId string `protobuf:"bytes,1,opt,name=ofagent_id,json=ofagentId,proto3" json:"ofagent_id,omitempty"`
+	// ID of voltha instance to which the ofagent is subscribed
+	VolthaId             string   `protobuf:"bytes,2,opt,name=voltha_id,json=volthaId,proto3" json:"voltha_id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *OfAgentSubscriber) Reset()         { *m = OfAgentSubscriber{} }
+func (m *OfAgentSubscriber) String() string { return proto.CompactTextString(m) }
+func (*OfAgentSubscriber) ProtoMessage()    {}
+func (*OfAgentSubscriber) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{13}
+}
+
+func (m *OfAgentSubscriber) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_OfAgentSubscriber.Unmarshal(m, b)
+}
+func (m *OfAgentSubscriber) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_OfAgentSubscriber.Marshal(b, m, deterministic)
+}
+func (m *OfAgentSubscriber) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OfAgentSubscriber.Merge(m, src)
+}
+func (m *OfAgentSubscriber) XXX_Size() int {
+	return xxx_messageInfo_OfAgentSubscriber.Size(m)
+}
+func (m *OfAgentSubscriber) XXX_DiscardUnknown() {
+	xxx_messageInfo_OfAgentSubscriber.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_OfAgentSubscriber proto.InternalMessageInfo
+
+func (m *OfAgentSubscriber) GetOfagentId() string {
+	if m != nil {
+		return m.OfagentId
+	}
+	return ""
+}
+
+func (m *OfAgentSubscriber) GetVolthaId() string {
+	if m != nil {
+		return m.VolthaId
+	}
+	return ""
+}
+
+// Identifies a membership group a Core belongs to
+type Membership struct {
+	//  Group name
+	GroupName string `protobuf:"bytes,1,opt,name=group_name,json=groupName,proto3" json:"group_name,omitempty"`
+	// Unique ID of a container within that group
+	Id                   string   `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Membership) Reset()         { *m = Membership{} }
+func (m *Membership) String() string { return proto.CompactTextString(m) }
+func (*Membership) ProtoMessage()    {}
+func (*Membership) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{14}
+}
+
+func (m *Membership) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Membership.Unmarshal(m, b)
+}
+func (m *Membership) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Membership.Marshal(b, m, deterministic)
+}
+func (m *Membership) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Membership.Merge(m, src)
+}
+func (m *Membership) XXX_Size() int {
+	return xxx_messageInfo_Membership.Size(m)
+}
+func (m *Membership) XXX_DiscardUnknown() {
+	xxx_messageInfo_Membership.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Membership proto.InternalMessageInfo
+
+func (m *Membership) GetGroupName() string {
+	if m != nil {
+		return m.GroupName
+	}
+	return ""
+}
+
+func (m *Membership) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+type DeviceImageDownloadRequest struct {
+	// Device Id
+	// allows for operations on multiple devices.
+	DeviceId []*common.ID `protobuf:"bytes,1,rep,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	//The image for the device containing all the information
+	Image *Image `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"`
+	//Activate the image if the download to the device is successful
+	ActivateOnSuccess bool `protobuf:"varint,3,opt,name=activateOnSuccess,proto3" json:"activateOnSuccess,omitempty"`
+	//Automatically commit the image if the activation on the device is successful
+	CommitOnSuccess      bool     `protobuf:"varint,4,opt,name=commitOnSuccess,proto3" json:"commitOnSuccess,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeviceImageDownloadRequest) Reset()         { *m = DeviceImageDownloadRequest{} }
+func (m *DeviceImageDownloadRequest) String() string { return proto.CompactTextString(m) }
+func (*DeviceImageDownloadRequest) ProtoMessage()    {}
+func (*DeviceImageDownloadRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{15}
+}
+
+func (m *DeviceImageDownloadRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceImageDownloadRequest.Unmarshal(m, b)
+}
+func (m *DeviceImageDownloadRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceImageDownloadRequest.Marshal(b, m, deterministic)
+}
+func (m *DeviceImageDownloadRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceImageDownloadRequest.Merge(m, src)
+}
+func (m *DeviceImageDownloadRequest) XXX_Size() int {
+	return xxx_messageInfo_DeviceImageDownloadRequest.Size(m)
+}
+func (m *DeviceImageDownloadRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceImageDownloadRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceImageDownloadRequest proto.InternalMessageInfo
+
+func (m *DeviceImageDownloadRequest) GetDeviceId() []*common.ID {
+	if m != nil {
+		return m.DeviceId
+	}
+	return nil
+}
+
+func (m *DeviceImageDownloadRequest) GetImage() *Image {
+	if m != nil {
+		return m.Image
+	}
+	return nil
+}
+
+func (m *DeviceImageDownloadRequest) GetActivateOnSuccess() bool {
+	if m != nil {
+		return m.ActivateOnSuccess
+	}
+	return false
+}
+
+func (m *DeviceImageDownloadRequest) GetCommitOnSuccess() bool {
+	if m != nil {
+		return m.CommitOnSuccess
+	}
+	return false
+}
+
+type DeviceImageRequest struct {
+	//Device Id
+	//allows for operations on multiple adapters.
+	DeviceId []*common.ID `protobuf:"bytes,1,rep,name=device_id,json=deviceId,proto3" json:"device_id,omitempty"`
+	// Image Version, this is the sole identifier of the image. it's the vendor specified OMCI version
+	// must be known at the time of initiating a download, activate, commit image on an onu.
+	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
+	//Automatically commit the image if the activation on the device is successful
+	CommitOnSuccess      bool     `protobuf:"varint,3,opt,name=commitOnSuccess,proto3" json:"commitOnSuccess,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DeviceImageRequest) Reset()         { *m = DeviceImageRequest{} }
+func (m *DeviceImageRequest) String() string { return proto.CompactTextString(m) }
+func (*DeviceImageRequest) ProtoMessage()    {}
+func (*DeviceImageRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{16}
+}
+
+func (m *DeviceImageRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceImageRequest.Unmarshal(m, b)
+}
+func (m *DeviceImageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceImageRequest.Marshal(b, m, deterministic)
+}
+func (m *DeviceImageRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceImageRequest.Merge(m, src)
+}
+func (m *DeviceImageRequest) XXX_Size() int {
+	return xxx_messageInfo_DeviceImageRequest.Size(m)
+}
+func (m *DeviceImageRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceImageRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceImageRequest proto.InternalMessageInfo
+
+func (m *DeviceImageRequest) GetDeviceId() []*common.ID {
+	if m != nil {
+		return m.DeviceId
+	}
+	return nil
+}
+
+func (m *DeviceImageRequest) GetVersion() string {
+	if m != nil {
+		return m.Version
+	}
+	return ""
+}
+
+func (m *DeviceImageRequest) GetCommitOnSuccess() bool {
+	if m != nil {
+		return m.CommitOnSuccess
+	}
+	return false
+}
+
+type DeviceImageResponse struct {
+	//Image state for the different devices
+	DeviceImageStates    []*DeviceImageState `protobuf:"bytes,1,rep,name=device_image_states,json=deviceImageStates,proto3" json:"device_image_states,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *DeviceImageResponse) Reset()         { *m = DeviceImageResponse{} }
+func (m *DeviceImageResponse) String() string { return proto.CompactTextString(m) }
+func (*DeviceImageResponse) ProtoMessage()    {}
+func (*DeviceImageResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{17}
+}
+
+func (m *DeviceImageResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DeviceImageResponse.Unmarshal(m, b)
+}
+func (m *DeviceImageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DeviceImageResponse.Marshal(b, m, deterministic)
+}
+func (m *DeviceImageResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DeviceImageResponse.Merge(m, src)
+}
+func (m *DeviceImageResponse) XXX_Size() int {
+	return xxx_messageInfo_DeviceImageResponse.Size(m)
+}
+func (m *DeviceImageResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_DeviceImageResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DeviceImageResponse proto.InternalMessageInfo
+
+func (m *DeviceImageResponse) GetDeviceImageStates() []*DeviceImageState {
+	if m != nil {
+		return m.DeviceImageStates
+	}
+	return nil
+}
+
+// Additional information required to process flow at device adapters
+type FlowMetadata struct {
+	// Meters associated with flow-update to adapter
+	Meters               []*openflow_13.OfpMeterConfig `protobuf:"bytes,1,rep,name=meters,proto3" json:"meters,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
+	XXX_unrecognized     []byte                        `json:"-"`
+	XXX_sizecache        int32                         `json:"-"`
+}
+
+func (m *FlowMetadata) Reset()         { *m = FlowMetadata{} }
+func (m *FlowMetadata) String() string { return proto.CompactTextString(m) }
+func (*FlowMetadata) ProtoMessage()    {}
+func (*FlowMetadata) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e084f1a60ce7016c, []int{18}
+}
+
+func (m *FlowMetadata) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FlowMetadata.Unmarshal(m, b)
+}
+func (m *FlowMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FlowMetadata.Marshal(b, m, deterministic)
+}
+func (m *FlowMetadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FlowMetadata.Merge(m, src)
+}
+func (m *FlowMetadata) XXX_Size() int {
+	return xxx_messageInfo_FlowMetadata.Size(m)
+}
+func (m *FlowMetadata) XXX_DiscardUnknown() {
+	xxx_messageInfo_FlowMetadata.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FlowMetadata proto.InternalMessageInfo
+
+func (m *FlowMetadata) GetMeters() []*openflow_13.OfpMeterConfig {
+	if m != nil {
+		return m.Meters
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("voltha.EventFilterRuleKey_EventFilterRuleType", EventFilterRuleKey_EventFilterRuleType_name, EventFilterRuleKey_EventFilterRuleType_value)
+	proto.RegisterEnum("voltha.TestResponse_TestResponseResult", TestResponse_TestResponseResult_name, TestResponse_TestResponseResult_value)
+	proto.RegisterEnum("voltha.SelfTestResponse_SelfTestResult", SelfTestResponse_SelfTestResult_name, SelfTestResponse_SelfTestResult_value)
+	proto.RegisterType((*DeviceGroup)(nil), "voltha.DeviceGroup")
+	proto.RegisterType((*DeviceGroups)(nil), "voltha.DeviceGroups")
+	proto.RegisterType((*EventFilterRuleKey)(nil), "voltha.EventFilterRuleKey")
+	proto.RegisterType((*EventFilterRule)(nil), "voltha.EventFilterRule")
+	proto.RegisterType((*EventFilter)(nil), "voltha.EventFilter")
+	proto.RegisterType((*EventFilters)(nil), "voltha.EventFilters")
+	proto.RegisterType((*CoreInstance)(nil), "voltha.CoreInstance")
+	proto.RegisterType((*CoreInstances)(nil), "voltha.CoreInstances")
+	proto.RegisterType((*OmciTestRequest)(nil), "voltha.OmciTestRequest")
+	proto.RegisterType((*TestResponse)(nil), "voltha.TestResponse")
+	proto.RegisterType((*ValueSet)(nil), "voltha.ValueSet")
+	proto.RegisterType((*Voltha)(nil), "voltha.Voltha")
+	proto.RegisterType((*SelfTestResponse)(nil), "voltha.SelfTestResponse")
+	proto.RegisterType((*OfAgentSubscriber)(nil), "voltha.OfAgentSubscriber")
+	proto.RegisterType((*Membership)(nil), "voltha.Membership")
+	proto.RegisterType((*DeviceImageDownloadRequest)(nil), "voltha.DeviceImageDownloadRequest")
+	proto.RegisterType((*DeviceImageRequest)(nil), "voltha.DeviceImageRequest")
+	proto.RegisterType((*DeviceImageResponse)(nil), "voltha.DeviceImageResponse")
+	proto.RegisterType((*FlowMetadata)(nil), "voltha.FlowMetadata")
+}
+
+func init() { proto.RegisterFile("voltha_protos/voltha.proto", fileDescriptor_e084f1a60ce7016c) }
+
+var fileDescriptor_e084f1a60ce7016c = []byte{
+	// 2901 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcb, 0x73, 0x1b, 0xb7,
+	0x19, 0x37, 0xf5, 0xd6, 0x47, 0x4a, 0x22, 0x41, 0x3d, 0x68, 0x4a, 0xf2, 0x03, 0x4e, 0x6c, 0x45,
+	0x89, 0xc9, 0xd8, 0x8e, 0x3d, 0x69, 0xd2, 0x4c, 0xa3, 0x97, 0x15, 0x36, 0xb6, 0xc9, 0x2e, 0x25,
+	0x3b, 0x6d, 0xe3, 0xe1, 0x2c, 0xb9, 0x20, 0xb5, 0x63, 0x92, 0xcb, 0xee, 0x82, 0xb2, 0x3d, 0x9e,
+	0x4c, 0xa7, 0xe9, 0x23, 0xe9, 0x39, 0xf7, 0x9e, 0xda, 0xe9, 0x4c, 0x2f, 0xfd, 0x1f, 0x7a, 0xcf,
+	0xa9, 0xa7, 0x5e, 0x3b, 0x3d, 0xf4, 0x2f, 0xc8, 0xb9, 0x83, 0x0f, 0x00, 0xb9, 0xcb, 0xdd, 0xd5,
+	0xc3, 0xc9, 0x4c, 0x4f, 0xd2, 0x02, 0x1f, 0x7e, 0xbf, 0x0f, 0x1f, 0xf0, 0x3d, 0x00, 0x10, 0xf2,
+	0xc7, 0x4e, 0x9b, 0x1f, 0x99, 0xb5, 0x9e, 0xeb, 0x70, 0xc7, 0x2b, 0xca, 0xaf, 0x02, 0x7e, 0x91,
+	0x29, 0xf9, 0x95, 0x5f, 0x6b, 0x39, 0x4e, 0xab, 0xcd, 0x8a, 0x66, 0xcf, 0x2e, 0x9a, 0xdd, 0xae,
+	0xc3, 0x4d, 0x6e, 0x3b, 0x5d, 0x4f, 0x4a, 0xe5, 0x57, 0x55, 0x2f, 0x7e, 0xd5, 0xfb, 0xcd, 0x22,
+	0xeb, 0xf4, 0xf8, 0x4b, 0xd5, 0x99, 0x0b, 0xc2, 0x77, 0x18, 0x57, 0xe0, 0xf9, 0x11, 0xe2, 0x86,
+	0xd3, 0xe9, 0x38, 0xdd, 0xe8, 0xbe, 0x23, 0x66, 0xb6, 0xf9, 0x91, 0xea, 0xa3, 0xc1, 0xbe, 0xb6,
+	0xd3, 0xb2, 0x1b, 0x66, 0xbb, 0x66, 0xb1, 0x63, 0xbb, 0xc1, 0xa2, 0xc7, 0x07, 0xfa, 0x56, 0x83,
+	0x7d, 0xa6, 0x65, 0xf6, 0x38, 0x73, 0x55, 0xe7, 0xe5, 0x60, 0xa7, 0xd3, 0x63, 0xdd, 0x66, 0xdb,
+	0x79, 0x5e, 0xbb, 0x75, 0x27, 0x46, 0xa0, 0xd3, 0xb0, 0x6b, 0x1d, 0xbb, 0x5e, 0xb3, 0xea, 0x4a,
+	0xe0, 0x6a, 0x84, 0x80, 0xd9, 0x36, 0xdd, 0xce, 0x50, 0xe4, 0x52, 0x50, 0x84, 0xbd, 0xe0, 0xb5,
+	0x86, 0xd3, 0x6d, 0xda, 0x2d, 0xd9, 0x4f, 0xff, 0x9c, 0x80, 0xe4, 0x2e, 0xaa, 0xbc, 0xef, 0x3a,
+	0xfd, 0x1e, 0x59, 0x82, 0x31, 0xdb, 0xca, 0x25, 0xae, 0x24, 0x36, 0x66, 0xb7, 0x27, 0xff, 0xfb,
+	0xdd, 0xb7, 0xeb, 0x09, 0x63, 0xcc, 0xb6, 0x48, 0x09, 0x16, 0x82, 0x93, 0xf7, 0x72, 0x63, 0x57,
+	0xc6, 0x37, 0x92, 0xb7, 0x97, 0x0a, 0x6a, 0x15, 0x1f, 0xc8, 0x6e, 0x89, 0xb5, 0x3d, 0xfb, 0xef,
+	0xef, 0xbe, 0x5d, 0x9f, 0x10, 0x58, 0xc6, 0x7c, 0xdb, 0xdf, 0xe3, 0x91, 0x3b, 0x30, 0xad, 0x21,
+	0xc6, 0x11, 0x62, 0x5e, 0x43, 0x84, 0xc7, 0x6a, 0x49, 0xfa, 0x23, 0x48, 0xf9, 0xb4, 0xf4, 0xc8,
+	0x5b, 0x30, 0x69, 0x73, 0xd6, 0xf1, 0x72, 0x09, 0x84, 0xc8, 0x06, 0x21, 0x50, 0xc8, 0x90, 0x12,
+	0xf4, 0x4f, 0x09, 0x20, 0x7b, 0xc7, 0xac, 0xcb, 0xef, 0xdb, 0x6d, 0xce, 0x5c, 0xa3, 0xdf, 0x66,
+	0x9f, 0xb2, 0x97, 0xf4, 0xab, 0x04, 0x64, 0x47, 0x9a, 0x0f, 0x5e, 0xf6, 0x18, 0x99, 0x07, 0x68,
+	0x62, 0x4b, 0xcd, 0x6c, 0xb7, 0xd3, 0x17, 0x48, 0x0a, 0x66, 0x1a, 0x26, 0x67, 0x2d, 0xc7, 0x7d,
+	0x99, 0x4e, 0x90, 0x34, 0xa4, 0xbc, 0x7e, 0xbd, 0x36, 0x68, 0x19, 0x23, 0x04, 0xe6, 0x9f, 0xf5,
+	0xec, 0x1a, 0x13, 0x50, 0x35, 0xfe, 0xb2, 0xc7, 0xd2, 0xe3, 0x64, 0x09, 0x32, 0xd2, 0xc8, 0xfe,
+	0xe6, 0x09, 0xd1, 0x2c, 0xe7, 0xe3, 0x6f, 0x9e, 0xa4, 0x36, 0x2c, 0x8c, 0x28, 0x42, 0x3e, 0x86,
+	0xf1, 0x67, 0xec, 0x25, 0x2e, 0xc3, 0xfc, 0xed, 0x82, 0x9e, 0x5c, 0x78, 0x16, 0x85, 0x88, 0x19,
+	0x18, 0x62, 0x28, 0x59, 0x84, 0xc9, 0x63, 0xb3, 0xdd, 0x67, 0xb9, 0x31, 0xb1, 0x94, 0x86, 0xfc,
+	0xa0, 0x7f, 0x4d, 0x40, 0xd2, 0x37, 0x24, 0x6e, 0xb5, 0x97, 0x61, 0x8a, 0x75, 0xcd, 0x7a, 0x5b,
+	0x8e, 0x9e, 0x31, 0xd4, 0x17, 0x59, 0x85, 0x59, 0x35, 0x01, 0xdb, 0xca, 0x8d, 0x23, 0xf0, 0x8c,
+	0x6c, 0x28, 0x59, 0x64, 0x1d, 0x60, 0x38, 0xad, 0xdc, 0x04, 0xf6, 0xce, 0x62, 0x0b, 0xda, 0xf5,
+	0x26, 0x4c, 0xba, 0xfd, 0x36, 0xf3, 0x72, 0x93, 0xb8, 0x62, 0x2b, 0x31, 0x93, 0x32, 0xa4, 0x14,
+	0xfd, 0x08, 0x52, 0xbe, 0x1e, 0x8f, 0xdc, 0x84, 0x69, 0xb9, 0x2c, 0xa1, 0x25, 0xf7, 0x03, 0x68,
+	0x19, 0xfa, 0x0c, 0x52, 0x3b, 0x8e, 0xcb, 0x4a, 0x5d, 0x8f, 0x9b, 0xdd, 0x06, 0x23, 0xd7, 0x21,
+	0x69, 0xab, 0xff, 0x6b, 0xa3, 0x33, 0x06, 0xdd, 0x53, 0xb2, 0xc8, 0x1d, 0x98, 0x92, 0x01, 0x00,
+	0x67, 0x9e, 0xbc, 0xbd, 0xa8, 0x59, 0x3e, 0xc1, 0xd6, 0x2a, 0x37, 0x79, 0xdf, 0xdb, 0x9e, 0x14,
+	0x3b, 0xf4, 0x82, 0xa1, 0x44, 0xe9, 0x87, 0x30, 0xe7, 0x27, 0xf3, 0xc8, 0x66, 0x70, 0x77, 0x0e,
+	0x40, 0xfc, 0x52, 0x7a, 0x7b, 0xde, 0x85, 0x85, 0x72, 0xa7, 0x61, 0x1f, 0x30, 0x8f, 0x1b, 0xec,
+	0x57, 0x7d, 0xe6, 0x71, 0x32, 0x3f, 0x5c, 0x15, 0x5c, 0x0e, 0x02, 0x13, 0xfd, 0xbe, 0x6d, 0xa9,
+	0xa5, 0xc4, 0xff, 0xe9, 0xaf, 0x21, 0x25, 0x87, 0x78, 0x3d, 0xa7, 0xeb, 0x31, 0xf2, 0x13, 0x98,
+	0x72, 0x99, 0xd7, 0x6f, 0x73, 0xb5, 0x69, 0x6e, 0x68, 0x4e, 0xbf, 0x54, 0xe0, 0xc3, 0x40, 0x71,
+	0x43, 0x0d, 0xa3, 0x05, 0x20, 0xe1, 0x5e, 0x92, 0x84, 0xe9, 0xea, 0xe1, 0xce, 0xce, 0x5e, 0xb5,
+	0x9a, 0xbe, 0x20, 0x3e, 0xee, 0x6f, 0x95, 0x1e, 0x1c, 0x1a, 0x7b, 0xe9, 0x04, 0x7d, 0x0a, 0x33,
+	0x8f, 0xc5, 0x9e, 0xaa, 0xb2, 0xb0, 0xc2, 0xef, 0x43, 0x4a, 0x86, 0x21, 0xe9, 0x05, 0xca, 0x96,
+	0xd9, 0x82, 0x8a, 0x3c, 0x5b, 0xa2, 0x6f, 0x07, 0xff, 0xff, 0xe4, 0x82, 0x91, 0x34, 0x87, 0x9f,
+	0xdb, 0xd3, 0x6a, 0xdb, 0xd2, 0x7f, 0x4d, 0xc0, 0xd4, 0x63, 0x9c, 0x01, 0xb9, 0x0c, 0xd3, 0xc7,
+	0xcc, 0xf5, 0x6c, 0xa7, 0x1b, 0x5c, 0x37, 0xdd, 0x4a, 0xee, 0xc1, 0x8c, 0x8a, 0xac, 0x3a, 0x2a,
+	0x2d, 0xe8, 0xd9, 0x6f, 0xc9, 0x76, 0x7f, 0x4c, 0x19, 0xc8, 0x46, 0x05, 0xb5, 0xf1, 0xef, 0x1f,
+	0xd4, 0x26, 0xce, 0x1a, 0xd4, 0xc8, 0xc7, 0x90, 0x52, 0xee, 0x24, 0x5c, 0x46, 0x7b, 0x06, 0x09,
+	0x8e, 0x14, 0xce, 0xe3, 0x1f, 0x9d, 0xb4, 0x06, 0xcd, 0x1e, 0xd9, 0x81, 0x39, 0x85, 0xd0, 0xc2,
+	0xb8, 0x98, 0x9b, 0x8a, 0x0d, 0x87, 0x7e, 0x0c, 0x45, 0xab, 0x62, 0xe9, 0x0e, 0xcc, 0x49, 0xc7,
+	0xd5, 0x0e, 0x36, 0x1d, 0xeb, 0x60, 0x01, 0x10, 0xe6, 0xf7, 0xcf, 0x9f, 0x41, 0x66, 0x98, 0x9f,
+	0x4c, 0x6e, 0xd6, 0x4d, 0x8f, 0xe5, 0xd6, 0x14, 0x90, 0xe8, 0x29, 0x3c, 0xb4, 0xeb, 0x52, 0x9d,
+	0x5d, 0x93, 0x9b, 0xdb, 0x69, 0x01, 0x94, 0xf4, 0xc5, 0x13, 0x63, 0x41, 0x48, 0x09, 0x21, 0x35,
+	0x9a, 0x3c, 0x81, 0xac, 0x3f, 0xa3, 0x69, 0xd0, 0x75, 0xb5, 0x44, 0x08, 0x8a, 0x5b, 0xe9, 0x44,
+	0x58, 0x54, 0x4b, 0x8a, 0x29, 0x04, 0xfa, 0x97, 0x04, 0xa4, 0xab, 0xac, 0xdd, 0x3c, 0x9b, 0x03,
+	0x8d, 0x4a, 0xfa, 0x1b, 0xfc, 0x0e, 0x54, 0x81, 0xf9, 0x60, 0x4f, 0xbc, 0xf3, 0x90, 0x0c, 0xcc,
+	0x3d, 0x2a, 0x1f, 0xd4, 0xaa, 0x87, 0x95, 0x4a, 0xd9, 0x38, 0xd8, 0xdb, 0x4d, 0x8f, 0x89, 0xa6,
+	0xc3, 0x47, 0x9f, 0x3e, 0x2a, 0x3f, 0x79, 0x54, 0xdb, 0x33, 0x8c, 0xb2, 0x91, 0x1e, 0xa7, 0x65,
+	0xc8, 0x94, 0x9b, 0x5b, 0x2d, 0xd6, 0xe5, 0xd5, 0x7e, 0xdd, 0x6b, 0xb8, 0x76, 0x9d, 0xb9, 0x22,
+	0xcc, 0x3a, 0x4d, 0x53, 0x34, 0x0e, 0x02, 0x99, 0x31, 0xab, 0x5a, 0x4a, 0x96, 0x08, 0xd1, 0x2a,
+	0xe3, 0x0f, 0x02, 0xc6, 0x8c, 0x6c, 0x28, 0x59, 0xf4, 0x43, 0x80, 0x87, 0xac, 0x53, 0x67, 0xae,
+	0x77, 0x64, 0xf7, 0x04, 0x12, 0xee, 0x9a, 0x5a, 0xd7, 0xec, 0x30, 0x8d, 0x84, 0x2d, 0x8f, 0xcc,
+	0x0e, 0x53, 0x4e, 0x3d, 0xa6, 0x9d, 0x9a, 0xfe, 0x23, 0x01, 0x79, 0x69, 0xe9, 0x52, 0xc7, 0x6c,
+	0xb1, 0x5d, 0xe7, 0x79, 0xb7, 0xed, 0x98, 0x96, 0x0e, 0x5a, 0x37, 0xfc, 0xb9, 0x41, 0xc6, 0x3d,
+	0x28, 0xa8, 0x42, 0xab, 0xb4, 0xeb, 0xcb, 0x13, 0xd7, 0x60, 0xd2, 0x16, 0x00, 0x2a, 0x2a, 0xcc,
+	0x69, 0x3b, 0x23, 0xaa, 0x21, 0xfb, 0xc8, 0x3b, 0x90, 0x31, 0x1b, 0xdc, 0x3e, 0x36, 0x39, 0x2b,
+	0x77, 0xab, 0xfd, 0x46, 0x83, 0x79, 0x1e, 0x66, 0x9c, 0x19, 0x23, 0xdc, 0x41, 0x36, 0x60, 0x41,
+	0x30, 0xd9, 0x7c, 0x28, 0x3b, 0x81, 0xb2, 0xa3, 0xcd, 0xf4, 0x37, 0x09, 0x20, 0xbe, 0x49, 0x9c,
+	0x5b, 0xf9, 0xdc, 0x30, 0x16, 0x49, 0xcb, 0x0c, 0x82, 0x50, 0x84, 0x0e, 0xe3, 0xd1, 0x3a, 0xd4,
+	0x20, 0x1b, 0x50, 0x41, 0x6d, 0xc0, 0x4f, 0x20, 0xab, 0x75, 0x10, 0xed, 0x35, 0x8f, 0x9b, 0x9c,
+	0xe9, 0x14, 0x92, 0x0b, 0x7a, 0x34, 0x8e, 0x14, 0xc9, 0x88, 0x19, 0xaa, 0xa4, 0x18, 0xb6, 0x78,
+	0x74, 0x0f, 0x52, 0xf7, 0xdb, 0xce, 0xf3, 0x87, 0x8c, 0x9b, 0xc2, 0x6b, 0xc8, 0x5d, 0x98, 0xea,
+	0x30, 0x5f, 0xea, 0x5c, 0x2f, 0xf8, 0x6b, 0x4d, 0xa7, 0xd9, 0xab, 0x61, 0xb7, 0x8a, 0xd6, 0x86,
+	0x12, 0xbe, 0xfd, 0xf7, 0x7b, 0x30, 0x27, 0x43, 0x70, 0x95, 0xb9, 0x82, 0x83, 0x3c, 0x81, 0xb9,
+	0x7d, 0xc6, 0x7d, 0x5b, 0x68, 0xb9, 0x20, 0xeb, 0xf1, 0x82, 0xae, 0xc7, 0x0b, 0x7b, 0xa2, 0x1e,
+	0xcf, 0x0f, 0x62, 0xd8, 0x50, 0x96, 0xe6, 0xbf, 0xfc, 0xe7, 0x7f, 0xbe, 0x19, 0x5b, 0x24, 0x04,
+	0x4b, 0xfb, 0xe3, 0x5b, 0xc5, 0xce, 0x10, 0xe7, 0x29, 0xa4, 0x0f, 0x7b, 0x96, 0xc9, 0x99, 0x0f,
+	0x3b, 0x02, 0x23, 0x1f, 0xc3, 0x47, 0xd7, 0x11, 0x7b, 0x85, 0x46, 0x60, 0x7f, 0x90, 0xd8, 0x24,
+	0xbb, 0x30, 0xbb, 0xcf, 0xb8, 0x4a, 0x27, 0x71, 0x3a, 0x0f, 0x22, 0xb6, 0x94, 0xa3, 0x0b, 0x88,
+	0x39, 0x4b, 0xa6, 0x15, 0x26, 0x79, 0x0a, 0x99, 0x07, 0xb6, 0xc7, 0x83, 0xa9, 0x3e, 0x0e, 0x6d,
+	0x29, 0x2a, 0xe7, 0x7b, 0xf4, 0x22, 0x82, 0x66, 0x49, 0x46, 0x2b, 0x6a, 0x0f, 0x90, 0xaa, 0xb0,
+	0xb0, 0xcf, 0x02, 0xe8, 0xc4, 0xb7, 0x07, 0xf3, 0x91, 0x45, 0x04, 0xbd, 0x84, 0x78, 0x39, 0xb2,
+	0x1c, 0xc2, 0x2b, 0xbe, 0xb2, 0xad, 0x2f, 0x88, 0x01, 0x29, 0xa1, 0xf3, 0x96, 0x4e, 0x79, 0x71,
+	0xea, 0xa6, 0x47, 0x12, 0xa6, 0x47, 0x73, 0x88, 0x4c, 0x48, 0x5a, 0x23, 0x0f, 0xd2, 0x26, 0x03,
+	0x22, 0x30, 0x1f, 0x04, 0x33, 0x60, 0x1c, 0xf2, 0x72, 0x64, 0x2e, 0xf5, 0xe8, 0x65, 0xc4, 0xbf,
+	0x48, 0x56, 0x34, 0xfe, 0x48, 0x2a, 0x26, 0xbf, 0x84, 0xf4, 0x3e, 0x0b, 0xb2, 0x04, 0x0c, 0x12,
+	0x9d, 0xa4, 0xe9, 0x1b, 0x88, 0x7b, 0x89, 0xac, 0xc5, 0xe0, 0x4a, 0xbb, 0x34, 0x61, 0x39, 0x34,
+	0x87, 0x8a, 0xe3, 0x72, 0x2f, 0xda, 0xe6, 0x4a, 0x0e, 0x25, 0xe8, 0x26, 0x32, 0xbc, 0x41, 0xe8,
+	0x49, 0x0c, 0xc5, 0x1e, 0xa2, 0xbd, 0x80, 0xc5, 0xd1, 0x49, 0x08, 0x10, 0xb2, 0x14, 0x81, 0x5c,
+	0xb2, 0xf2, 0xd9, 0x88, 0x66, 0xfa, 0x1e, 0xf2, 0x15, 0xc8, 0x3b, 0xa7, 0xf3, 0x15, 0x5f, 0x89,
+	0x3f, 0x35, 0x31, 0xc3, 0xdf, 0x27, 0x60, 0x65, 0x0f, 0xcb, 0xf6, 0x33, 0xb3, 0xc7, 0x79, 0xd7,
+	0x87, 0xa8, 0xc0, 0x5d, 0x7a, 0xe7, 0x3c, 0x0a, 0x14, 0xd5, 0x99, 0xe1, 0xab, 0x04, 0xe4, 0x76,
+	0x6d, 0xef, 0x07, 0x51, 0xe4, 0xc7, 0xa8, 0xc8, 0x3d, 0xfa, 0xde, 0xb9, 0x14, 0xb1, 0x24, 0x3b,
+	0xb1, 0x22, 0xd6, 0x5c, 0xc4, 0xc9, 0xe0, 0x9a, 0x93, 0x40, 0x70, 0xc4, 0xfe, 0x33, 0xae, 0x78,
+	0x13, 0xb1, 0x7e, 0x9b, 0x80, 0x35, 0x19, 0xcb, 0x42, 0x44, 0x07, 0xa8, 0xc6, 0x5a, 0x88, 0x00,
+	0xdb, 0xe5, 0x98, 0xd8, 0xa9, 0xdf, 0x44, 0x15, 0x6e, 0xd0, 0x33, 0xa8, 0x20, 0x22, 0xde, 0xef,
+	0x12, 0xb0, 0x1e, 0xa1, 0xc5, 0x43, 0x11, 0xd9, 0xa5, 0x1a, 0xab, 0x01, 0x35, 0xb0, 0xe3, 0xa1,
+	0x63, 0x9d, 0xa2, 0x45, 0x01, 0xb5, 0xd8, 0xa0, 0xd7, 0x4e, 0xd4, 0x42, 0xe6, 0x0f, 0xa1, 0x46,
+	0x0b, 0x56, 0x42, 0x26, 0x47, 0xaa, 0xa0, 0xcd, 0xb3, 0x61, 0x5d, 0x3c, 0xfa, 0x36, 0x72, 0xbd,
+	0x49, 0xce, 0xc2, 0x45, 0x38, 0xac, 0x46, 0xae, 0xad, 0x2a, 0x71, 0xfd, 0x64, 0x2b, 0x21, 0xfb,
+	0x4b, 0x21, 0xfa, 0x2e, 0x12, 0x6e, 0x92, 0x8d, 0x53, 0x4d, 0xac, 0xaa, 0x6d, 0xf2, 0x4d, 0x02,
+	0xae, 0xc6, 0xac, 0x35, 0x62, 0x4a, 0x4b, 0x5f, 0x8d, 0x26, 0x3c, 0xcb, 0xaa, 0xdf, 0x41, 0x95,
+	0x6e, 0xd2, 0x33, 0xab, 0x24, 0x8c, 0x5e, 0x86, 0xa4, 0xb0, 0xc5, 0x69, 0x81, 0x79, 0x21, 0x58,
+	0x52, 0x78, 0x74, 0x05, 0xc9, 0x32, 0x64, 0x41, 0x93, 0xe9, 0x48, 0x5c, 0x86, 0xb9, 0x21, 0x60,
+	0xc9, 0x8a, 0x87, 0x4c, 0x0e, 0xcd, 0x1c, 0x91, 0xea, 0x24, 0x9c, 0x6d, 0x79, 0xe4, 0x10, 0xd2,
+	0x06, 0x6b, 0x38, 0xdd, 0x86, 0xdd, 0x66, 0x5a, 0x4d, 0xff, 0xd8, 0x58, 0x7b, 0xac, 0x21, 0xe6,
+	0x32, 0x0d, 0x63, 0x8a, 0x89, 0xef, 0x61, 0x9a, 0x8f, 0x48, 0x15, 0x23, 0x87, 0x31, 0x0d, 0x43,
+	0x16, 0x47, 0x66, 0x2a, 0x73, 0xc3, 0x4f, 0x21, 0xb5, 0xe3, 0x32, 0x93, 0x2b, 0xd5, 0xc8, 0xc8,
+	0xe8, 0x10, 0x9a, 0x2a, 0x6c, 0xe8, 0xa8, 0xdd, 0x84, 0x4a, 0x4f, 0x20, 0x25, 0x83, 0x70, 0x84,
+	0x56, 0x71, 0x93, 0xbc, 0x86, 0x78, 0xeb, 0x74, 0x35, 0x4a, 0x3b, 0x1d, 0x56, 0x7f, 0x0e, 0x73,
+	0x2a, 0xaa, 0x9e, 0x03, 0x59, 0xe5, 0x46, 0xba, 0x16, 0x89, 0xac, 0xe3, 0xe4, 0x13, 0x48, 0x19,
+	0xac, 0xee, 0x38, 0xfc, 0x07, 0xd3, 0xd9, 0x45, 0x38, 0x01, 0xbc, 0xcb, 0xda, 0x8c, 0xbf, 0x86,
+	0x31, 0x36, 0xa3, 0x81, 0x2d, 0x84, 0x23, 0x75, 0xc8, 0xdc, 0x77, 0xdc, 0x06, 0x3b, 0x37, 0xfa,
+	0x5b, 0x88, 0x7e, 0x6d, 0xf3, 0x6a, 0x24, 0x7a, 0x53, 0x60, 0xd6, 0x14, 0x47, 0x1f, 0xe6, 0xf4,
+	0x91, 0x07, 0x6b, 0xed, 0x61, 0xee, 0x0a, 0x1c, 0x87, 0xf2, 0x4b, 0x9a, 0xb6, 0xdc, 0x63, 0x2e,
+	0xde, 0x5d, 0x8b, 0x0a, 0x9f, 0xde, 0x43, 0xa6, 0x77, 0xe9, 0xdb, 0x91, 0x4c, 0xb2, 0xe2, 0xb7,
+	0x14, 0x86, 0x57, 0x7c, 0x25, 0x8e, 0x64, 0x5f, 0x88, 0x0d, 0xf4, 0x65, 0x02, 0x96, 0xf7, 0x19,
+	0x0f, 0x70, 0xc8, 0x5b, 0xa8, 0x78, 0x05, 0xa2, 0x9a, 0xe9, 0x07, 0xa8, 0xc0, 0x7b, 0xe4, 0xf6,
+	0x39, 0x14, 0x28, 0x7a, 0x92, 0xa9, 0x8f, 0xa5, 0x58, 0x00, 0xef, 0x9c, 0xec, 0x2a, 0x90, 0x91,
+	0xf3, 0x4c, 0x9f, 0x34, 0x65, 0xa1, 0x19, 0x40, 0xf2, 0x46, 0xd6, 0x35, 0x8a, 0xcd, 0xa3, 0xef,
+	0x20, 0xdd, 0x75, 0xf2, 0xc6, 0x59, 0xe8, 0xc8, 0x0b, 0xc8, 0xee, 0x88, 0x9a, 0xb9, 0x7d, 0xc6,
+	0x19, 0x46, 0x2e, 0xb0, 0x9a, 0xe1, 0xe6, 0xb9, 0x66, 0xf8, 0x75, 0x02, 0xb2, 0x5b, 0xea, 0x38,
+	0x8b, 0x2c, 0x32, 0x1f, 0x9c, 0x93, 0x7a, 0x07, 0xa9, 0x3f, 0xa2, 0xef, 0x9f, 0x67, 0x69, 0x65,
+	0x73, 0x1f, 0xf9, 0xc4, 0x46, 0xfb, 0x43, 0x02, 0x32, 0x06, 0x3b, 0x66, 0x2e, 0xff, 0xbf, 0x28,
+	0xe2, 0x22, 0xb5, 0x50, 0xe4, 0xeb, 0x04, 0x2c, 0x05, 0x3c, 0xed, 0xc0, 0x51, 0x1e, 0x4d, 0x23,
+	0x0e, 0xc1, 0x23, 0xd7, 0x10, 0xf9, 0xd5, 0x08, 0x19, 0x7d, 0xc4, 0xd6, 0xe5, 0x0b, 0xb9, 0x3e,
+	0xaa, 0x1f, 0xea, 0xe0, 0x15, 0xb5, 0x6e, 0xf2, 0x04, 0xee, 0x91, 0xe7, 0x30, 0xaf, 0xf7, 0xbd,
+	0xf2, 0xb9, 0x7c, 0x24, 0xfc, 0x19, 0xa8, 0x63, 0x77, 0xa4, 0xa2, 0x96, 0x7f, 0x6a, 0xca, 0xe1,
+	0xfe, 0x98, 0x80, 0x8b, 0x5b, 0x75, 0x67, 0xb0, 0x16, 0x2d, 0xd7, 0xb4, 0x86, 0x76, 0x78, 0x6d,
+	0x25, 0x62, 0xbd, 0x50, 0x29, 0x61, 0x0a, 0xca, 0x5a, 0x5f, 0xd2, 0x69, 0x23, 0x3c, 0x86, 0xd4,
+	0x3e, 0xe3, 0xe5, 0x6e, 0xbf, 0x24, 0xbf, 0xfd, 0xfe, 0x97, 0xd1, 0x6c, 0x83, 0x6e, 0x7a, 0x03,
+	0x39, 0xae, 0x92, 0xcb, 0x91, 0x7b, 0xc0, 0xe9, 0xf6, 0x35, 0xee, 0x2b, 0x98, 0x0b, 0x6c, 0xfd,
+	0xd7, 0x9f, 0xd6, 0x2d, 0xa4, 0x7c, 0x9b, 0xc6, 0x2d, 0xab, 0xbe, 0x30, 0x52, 0xcc, 0x62, 0x93,
+	0x3d, 0x87, 0xe4, 0x0e, 0x5e, 0xcb, 0x7c, 0x4f, 0xea, 0x22, 0x52, 0xbf, 0x45, 0xe3, 0x96, 0x55,
+	0xde, 0xfd, 0xf8, 0x88, 0x2b, 0xb0, 0x30, 0xac, 0xa5, 0xc2, 0x27, 0xce, 0xc1, 0x6d, 0x98, 0x3c,
+	0x6a, 0x52, 0x84, 0x5f, 0x23, 0xf9, 0x48, 0x63, 0xca, 0x23, 0xe6, 0x53, 0xc8, 0xfa, 0x10, 0xd5,
+	0x45, 0x7a, 0xcc, 0x32, 0x0d, 0xba, 0x4f, 0x59, 0xa6, 0x9e, 0xbe, 0xba, 0xf7, 0x48, 0x17, 0x96,
+	0x64, 0x2c, 0x18, 0x25, 0x08, 0x83, 0xc6, 0xa6, 0x59, 0x75, 0x7e, 0xa2, 0xa7, 0x91, 0x09, 0x03,
+	0x1d, 0xfa, 0x0d, 0x74, 0xb6, 0xe3, 0xd9, 0xc9, 0x56, 0x92, 0xc7, 0x32, 0x06, 0x8b, 0x41, 0xd8,
+	0xf3, 0x9c, 0x0c, 0x36, 0x90, 0x80, 0x92, 0x2b, 0xb1, 0x04, 0xfa, 0x44, 0xf0, 0xb9, 0x5f, 0x7b,
+	0x79, 0x47, 0x1f, 0x57, 0x2c, 0x67, 0xc3, 0xf7, 0xfc, 0x5e, 0x5c, 0x65, 0x2a, 0x1f, 0x08, 0x88,
+	0x81, 0xf7, 0x6f, 0x43, 0xf9, 0x11, 0xcb, 0x84, 0xf0, 0xe8, 0x55, 0x84, 0x5b, 0x25, 0x17, 0xa3,
+	0xe0, 0x64, 0xb5, 0x5b, 0x83, 0xf4, 0x50, 0x63, 0x65, 0x94, 0x38, 0x95, 0x17, 0x23, 0xde, 0x15,
+	0x3c, 0x7d, 0xf9, 0x46, 0x96, 0x46, 0x48, 0x94, 0x49, 0xee, 0x43, 0xba, 0xca, 0x5d, 0x66, 0x76,
+	0x2a, 0x66, 0xe3, 0x19, 0xe3, 0x5e, 0xb9, 0xcf, 0xc9, 0x72, 0xc0, 0xd2, 0xb2, 0xa3, 0xdc, 0xe7,
+	0xb1, 0x1b, 0xe8, 0xc2, 0x46, 0x82, 0xec, 0xe1, 0xa1, 0x81, 0xd9, 0xc7, 0x4c, 0x01, 0x95, 0xba,
+	0x27, 0xdc, 0xbe, 0x85, 0xf1, 0x4b, 0x5d, 0x7a, 0xe1, 0xdd, 0x04, 0xf9, 0x14, 0xb2, 0x0a, 0x66,
+	0xe7, 0xc8, 0xec, 0xb6, 0x18, 0xbe, 0x6e, 0xc4, 0x4f, 0x39, 0x17, 0x40, 0xf2, 0x0d, 0x41, 0xb0,
+	0x43, 0x4c, 0x10, 0xfe, 0xf7, 0xf3, 0xe0, 0xb1, 0x36, 0x6c, 0xae, 0xb8, 0xcd, 0xaa, 0xac, 0xa5,
+	0xd7, 0x24, 0x23, 0x4f, 0x20, 0xfe, 0xb7, 0xda, 0xa8, 0xf7, 0x98, 0x7c, 0x54, 0x23, 0xbd, 0x82,
+	0x14, 0x79, 0x3a, 0x58, 0x90, 0xc0, 0xf3, 0x8e, 0x70, 0xb2, 0xc7, 0xa8, 0xb7, 0x1f, 0x3d, 0xf2,
+	0xda, 0xcb, 0xff, 0x02, 0x1b, 0x56, 0x3c, 0x80, 0x2a, 0x15, 0xb7, 0x20, 0x23, 0x83, 0xc5, 0xeb,
+	0x29, 0xfe, 0x26, 0x52, 0x5c, 0xce, 0x9f, 0x40, 0x21, 0xb4, 0xb7, 0x20, 0x23, 0x2b, 0xfd, 0x53,
+	0x59, 0xe2, 0xf6, 0x93, 0x9a, 0xcb, 0xe6, 0x49, 0x73, 0x51, 0x8e, 0x11, 0x78, 0x85, 0x3e, 0xd5,
+	0x31, 0x02, 0x16, 0x0b, 0x39, 0x46, 0x80, 0x85, 0x3c, 0xc0, 0xe3, 0x6a, 0x44, 0x56, 0x9d, 0x0f,
+	0x14, 0x5d, 0x9e, 0x3e, 0x03, 0x91, 0xd5, 0xf8, 0xb2, 0xca, 0x23, 0x9f, 0xc1, 0x8c, 0x7e, 0x7e,
+	0x0a, 0x80, 0xe5, 0xe2, 0xde, 0xb1, 0xe8, 0x75, 0x84, 0xbd, 0x42, 0x2f, 0x45, 0xc2, 0x7a, 0xac,
+	0xdd, 0xac, 0x71, 0x81, 0xf6, 0x18, 0xab, 0xff, 0xc0, 0xf3, 0xdd, 0xe8, 0xed, 0x4d, 0xe8, 0x7d,
+	0x2f, 0x1c, 0x79, 0x84, 0x1b, 0x09, 0x39, 0x75, 0x6d, 0x63, 0xd7, 0xc9, 0xe7, 0x40, 0xf6, 0x19,
+	0x1f, 0x79, 0xc1, 0x1b, 0xb9, 0xe2, 0x8d, 0x7a, 0xe4, 0x0b, 0xdb, 0x23, 0x88, 0x8d, 0xef, 0x85,
+	0xc4, 0x83, 0xb9, 0xaa, 0xdd, 0xe9, 0xb7, 0x4d, 0xce, 0x70, 0x3c, 0x59, 0x1b, 0x18, 0xc2, 0xdf,
+	0xac, 0xb3, 0x7c, 0x4c, 0x45, 0x1b, 0xba, 0x76, 0x0b, 0xda, 0x48, 0x21, 0xd5, 0x04, 0x92, 0xd8,
+	0x99, 0x3b, 0x30, 0x3b, 0x78, 0xaa, 0x23, 0x17, 0x07, 0xc5, 0xd1, 0xe8, 0x23, 0x5e, 0x3e, 0xbe,
+	0x8b, 0x5e, 0x20, 0x0f, 0x01, 0xe4, 0x9d, 0x01, 0x5e, 0x91, 0xa6, 0xfc, 0x15, 0x41, 0xec, 0x86,
+	0x56, 0x97, 0x2d, 0x74, 0x5e, 0xe8, 0x38, 0x1c, 0xad, 0xae, 0x83, 0xd4, 0x4d, 0xc1, 0x39, 0xf0,
+	0x86, 0x77, 0x1a, 0xc7, 0xb7, 0x8a, 0xbe, 0xe1, 0x02, 0xf0, 0x33, 0x48, 0x8a, 0xe0, 0xf1, 0x82,
+	0xe3, 0xfb, 0x3f, 0x59, 0xd6, 0x96, 0x93, 0x3f, 0x07, 0xe8, 0xb1, 0x86, 0xdd, 0xb4, 0x99, 0x9b,
+	0x5f, 0xd4, 0xed, 0x06, 0xe3, 0x7d, 0xb7, 0x8b, 0xbd, 0x1e, 0x5d, 0x45, 0xe0, 0x25, 0x92, 0xd5,
+	0x06, 0xf5, 0x43, 0x1d, 0x40, 0xb2, 0xea, 0xfb, 0x1c, 0x3c, 0x4a, 0xe8, 0x1f, 0x1a, 0xc4, 0xaa,
+	0x1b, 0x42, 0xf5, 0xc3, 0x1c, 0x41, 0xb6, 0xca, 0x4d, 0x97, 0xeb, 0x9f, 0x59, 0x88, 0xaa, 0xd3,
+	0xe9, 0x92, 0xc1, 0x2f, 0x50, 0x46, 0x7e, 0x7e, 0x31, 0x74, 0xe6, 0x80, 0xb7, 0xa8, 0x90, 0x41,
+	0x07, 0xef, 0x15, 0x9e, 0xc0, 0xac, 0xe1, 0x0b, 0xb5, 0x70, 0x93, 0x0f, 0x12, 0x9b, 0xdb, 0x6d,
+	0xc8, 0x3a, 0x6e, 0x0b, 0x33, 0x46, 0xc3, 0x71, 0x2d, 0x85, 0xb3, 0x9d, 0x92, 0x2f, 0x4a, 0x15,
+	0xfc, 0x09, 0xd6, 0x2f, 0x0a, 0x2d, 0x9b, 0x1f, 0xf5, 0xeb, 0xc2, 0x3a, 0x45, 0x2d, 0xa9, 0x7e,
+	0x0a, 0x77, 0x53, 0xff, 0x30, 0xee, 0x6e, 0xb1, 0xe5, 0xa8, 0xb6, 0xbf, 0x8d, 0x2d, 0x97, 0x35,
+	0xde, 0x63, 0xff, 0x03, 0x55, 0x65, 0xac, 0x32, 0x5e, 0x99, 0xa8, 0x4c, 0x56, 0xa6, 0x2a, 0xd3,
+	0x95, 0x99, 0xfa, 0x14, 0x8e, 0xbd, 0xf3, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x14, 0x94, 0xf1,
+	0xbe, 0x64, 0x27, 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
+
+// VolthaServiceClient is the client API for VolthaService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type VolthaServiceClient interface {
+	// Get the membership group of a Voltha Core
+	GetMembership(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Membership, error)
+	// Set the membership group of a Voltha Core
+	UpdateMembership(ctx context.Context, in *Membership, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Get high level information on the Voltha cluster
+	GetVoltha(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Voltha, error)
+	// List all Voltha cluster core instances
+	ListCoreInstances(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*CoreInstances, error)
+	// Get details on a Voltha cluster instance
+	GetCoreInstance(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*CoreInstance, error)
+	// List all active adapters (plugins) in the Voltha cluster
+	ListAdapters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Adapters, error)
+	// List all logical devices managed by the Voltha cluster
+	ListLogicalDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*LogicalDevices, error)
+	// Get additional information on a given logical device
+	GetLogicalDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalDevice, error)
+	// List ports of a logical device
+	ListLogicalDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalPorts, error)
+	// Gets a logical device port
+	GetLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*LogicalPort, error)
+	// Enables a logical device port
+	EnableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Disables a logical device port
+	DisableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error)
+	// List all flows of a logical device
+	ListLogicalDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error)
+	// Update flow table for logical device
+	UpdateLogicalDeviceFlowTable(ctx context.Context, in *openflow_13.FlowTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Update meter table for logical device
+	UpdateLogicalDeviceMeterTable(ctx context.Context, in *openflow_13.MeterModUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
+	// List all meters of a logical device
+	ListLogicalDeviceMeters(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Meters, error)
+	// List all flow groups of a logical device
+	ListLogicalDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error)
+	// Update group table for device
+	UpdateLogicalDeviceFlowGroupTable(ctx context.Context, in *openflow_13.FlowGroupTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error)
+	// List all physical devices controlled by the Voltha cluster
+	ListDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Devices, error)
+	// List all physical devices IDs controlled by the Voltha cluster
+	ListDeviceIds(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*common.IDs, error)
+	// Request to a voltha Core to reconcile a set of devices based on their IDs
+	ReconcileDevices(ctx context.Context, in *common.IDs, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Get more information on a given physical device
+	GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Device, error)
+	// Pre-provision a new physical device
+	CreateDevice(ctx context.Context, in *Device, opts ...grpc.CallOption) (*Device, error)
+	// Enable a device.  If the device was in pre-provisioned state then it
+	// will transition to ENABLED state.  If it was is DISABLED state then it
+	// will transition to ENABLED state as well.
+	EnableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Disable a device
+	DisableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Reboot a device
+	RebootDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Delete a device
+	DeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Forcefully delete a device
+	ForceDeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Request an image download to the standby partition
+	// of a device.
+	// Note that the call is expected to be non-blocking.
+	DownloadImage(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
+	// Get image download status on a device
+	// The request retrieves progress on device and updates db record
+	// Deprecated in voltha 2.8, will be removed
+	GetImageDownloadStatus(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error)
+	// Get image download db record
+	// Deprecated in voltha 2.8, will be removed
+	GetImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error)
+	// List image download db records for a given device
+	// Deprecated in voltha 2.8, will be removed
+	ListImageDownloads(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*ImageDownloads, error)
+	// Cancel an existing image download process on a device
+	// Deprecated in voltha 2.8, will be removed
+	CancelImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
+	// Activate the specified image at a standby partition
+	// to active partition.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device
+	// Note that the call is expected to be non-blocking.
+	// Deprecated in voltha 2.8, will be removed
+	ActivateImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
+	// Revert the specified image at standby partition
+	// to active partition, and revert to previous image
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// previous image running on device
+	// Note that the call is expected to be non-blocking.
+	// Deprecated in voltha 2.8, will be removed
+	RevertImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error)
+	// Downloads a certain image to the standby partition of the devices
+	// Note that the call is expected to be non-blocking.
+	DownloadImageToDevice(ctx context.Context, in *DeviceImageDownloadRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
+	// Get image status on a number of devices devices
+	// Polled from northbound systems to get state of download/activate/commit
+	GetImageStatus(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
+	// Aborts the upgrade of an image on a device
+	// To be used carefully, stops any further operations for the Image on the given devices
+	// Might also stop if possible existing work, but no guarantees are given,
+	// depends on implementation and procedure status.
+	AbortImageUpgradeToDevice(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
+	// Get Both Active and Standby image for a given device
+	GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*OnuImages, error)
+	// Activate the specified image from a standby partition
+	// to active partition.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device
+	// Note that the call is expected to be non-blocking.
+	ActivateImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
+	// Commit the specified image to be default.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device upon next reboot
+	// Note that the call is expected to be non-blocking.
+	CommitImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error)
+	// List ports of a device
+	ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Ports, error)
+	// List pm config of a device
+	ListDevicePmConfigs(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*PmConfigs, error)
+	// Update the pm config of a device
+	UpdateDevicePmConfigs(ctx context.Context, in *PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error)
+	// List all flows of a device
+	ListDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error)
+	// List all flow groups of a device
+	ListDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error)
+	// List device types known to Voltha
+	ListDeviceTypes(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceTypes, error)
+	// Get additional information on a device type
+	GetDeviceType(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceType, error)
+	// List all device sharding groups
+	ListDeviceGroups(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceGroups, error)
+	// Stream control packets to the dataplane
+	StreamPacketsOut(ctx context.Context, opts ...grpc.CallOption) (VolthaService_StreamPacketsOutClient, error)
+	// Receive control packet stream
+	ReceivePacketsIn(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceivePacketsInClient, error)
+	ReceiveChangeEvents(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceiveChangeEventsClient, error)
+	// Get additional information on a device group
+	GetDeviceGroup(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceGroup, error)
+	CreateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error)
+	// Get all filters present for a device
+	GetEventFilter(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*EventFilters, error)
+	UpdateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error)
+	DeleteEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*empty.Empty, error)
+	// Get all the filters present
+	ListEventFilters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*EventFilters, error)
+	GetImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Images, error)
+	SelfTest(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*SelfTestResponse, error)
+	// OpenOMCI MIB information
+	GetMibDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.MibDeviceData, error)
+	// OpenOMCI ALARM information
+	GetAlarmDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.AlarmDeviceData, error)
+	// Simulate an Alarm
+	SimulateAlarm(ctx context.Context, in *SimulateAlarmRequest, opts ...grpc.CallOption) (*common.OperationResp, error)
+	Subscribe(ctx context.Context, in *OfAgentSubscriber, opts ...grpc.CallOption) (*OfAgentSubscriber, error)
+	EnablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error)
+	DisablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error)
+	GetExtValue(ctx context.Context, in *common.ValueSpecifier, opts ...grpc.CallOption) (*common.ReturnValues, error)
+	SetExtValue(ctx context.Context, in *ValueSet, opts ...grpc.CallOption) (*empty.Empty, error)
+	// omci start and stop cli implementation
+	StartOmciTestAction(ctx context.Context, in *OmciTestRequest, opts ...grpc.CallOption) (*TestResponse, error)
+}
+
+type volthaServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewVolthaServiceClient(cc *grpc.ClientConn) VolthaServiceClient {
+	return &volthaServiceClient{cc}
+}
+
+func (c *volthaServiceClient) GetMembership(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Membership, error) {
+	out := new(Membership)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetMembership", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateMembership(ctx context.Context, in *Membership, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateMembership", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetVoltha(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Voltha, error) {
+	out := new(Voltha)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetVoltha", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListCoreInstances(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*CoreInstances, error) {
+	out := new(CoreInstances)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListCoreInstances", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetCoreInstance(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*CoreInstance, error) {
+	out := new(CoreInstance)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetCoreInstance", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListAdapters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Adapters, error) {
+	out := new(Adapters)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListAdapters", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListLogicalDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*LogicalDevices, error) {
+	out := new(LogicalDevices)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDevices", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetLogicalDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalDevice, error) {
+	out := new(LogicalDevice)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetLogicalDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListLogicalDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*LogicalPorts, error) {
+	out := new(LogicalPorts)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDevicePorts", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*LogicalPort, error) {
+	out := new(LogicalPort)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetLogicalDevicePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) EnableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnableLogicalDevicePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DisableLogicalDevicePort(ctx context.Context, in *LogicalPortId, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisableLogicalDevicePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListLogicalDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error) {
+	out := new(openflow_13.Flows)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceFlows", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateLogicalDeviceFlowTable(ctx context.Context, in *openflow_13.FlowTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceFlowTable", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateLogicalDeviceMeterTable(ctx context.Context, in *openflow_13.MeterModUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceMeterTable", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListLogicalDeviceMeters(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Meters, error) {
+	out := new(openflow_13.Meters)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceMeters", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListLogicalDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error) {
+	out := new(openflow_13.FlowGroups)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListLogicalDeviceFlowGroups", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateLogicalDeviceFlowGroupTable(ctx context.Context, in *openflow_13.FlowGroupTableUpdate, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateLogicalDeviceFlowGroupTable", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDevices(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*Devices, error) {
+	out := new(Devices)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevices", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDeviceIds(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*common.IDs, error) {
+	out := new(common.IDs)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceIds", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ReconcileDevices(ctx context.Context, in *common.IDs, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ReconcileDevices", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Device, error) {
+	out := new(Device)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) CreateDevice(ctx context.Context, in *Device, opts ...grpc.CallOption) (*Device, error) {
+	out := new(Device)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CreateDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) EnableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnableDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DisableDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisableDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) RebootDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/RebootDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DeleteDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ForceDeleteDevice(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ForceDeleteDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DownloadImage(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DownloadImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetImageDownloadStatus(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error) {
+	out := new(ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageDownloadStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*ImageDownload, error) {
+	out := new(ImageDownload)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageDownload", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListImageDownloads(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*ImageDownloads, error) {
+	out := new(ImageDownloads)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListImageDownloads", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) CancelImageDownload(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CancelImageDownload", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ActivateImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ActivateImageUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) RevertImageUpdate(ctx context.Context, in *ImageDownload, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/RevertImageUpdate", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DownloadImageToDevice(ctx context.Context, in *DeviceImageDownloadRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
+	out := new(DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DownloadImageToDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetImageStatus(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
+	out := new(DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImageStatus", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) AbortImageUpgradeToDevice(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
+	out := new(DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/AbortImageUpgradeToDevice", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetOnuImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*OnuImages, error) {
+	out := new(OnuImages)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetOnuImages", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ActivateImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
+	out := new(DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ActivateImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) CommitImage(ctx context.Context, in *DeviceImageRequest, opts ...grpc.CallOption) (*DeviceImageResponse, error) {
+	out := new(DeviceImageResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CommitImage", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDevicePorts(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Ports, error) {
+	out := new(Ports)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevicePorts", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDevicePmConfigs(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*PmConfigs, error) {
+	out := new(PmConfigs)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDevicePmConfigs", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateDevicePmConfigs(ctx context.Context, in *PmConfigs, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateDevicePmConfigs", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDeviceFlows(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.Flows, error) {
+	out := new(openflow_13.Flows)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceFlows", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDeviceFlowGroups(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*openflow_13.FlowGroups, error) {
+	out := new(openflow_13.FlowGroups)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceFlowGroups", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDeviceTypes(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceTypes, error) {
+	out := new(DeviceTypes)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceTypes", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetDeviceType(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceType, error) {
+	out := new(DeviceType)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDeviceType", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListDeviceGroups(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*DeviceGroups, error) {
+	out := new(DeviceGroups)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListDeviceGroups", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) StreamPacketsOut(ctx context.Context, opts ...grpc.CallOption) (VolthaService_StreamPacketsOutClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[0], "/voltha.VolthaService/StreamPacketsOut", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &volthaServiceStreamPacketsOutClient{stream}
+	return x, nil
+}
+
+type VolthaService_StreamPacketsOutClient interface {
+	Send(*openflow_13.PacketOut) error
+	CloseAndRecv() (*empty.Empty, error)
+	grpc.ClientStream
+}
+
+type volthaServiceStreamPacketsOutClient struct {
+	grpc.ClientStream
+}
+
+func (x *volthaServiceStreamPacketsOutClient) Send(m *openflow_13.PacketOut) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *volthaServiceStreamPacketsOutClient) CloseAndRecv() (*empty.Empty, error) {
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	m := new(empty.Empty)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *volthaServiceClient) ReceivePacketsIn(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceivePacketsInClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[1], "/voltha.VolthaService/ReceivePacketsIn", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &volthaServiceReceivePacketsInClient{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 VolthaService_ReceivePacketsInClient interface {
+	Recv() (*openflow_13.PacketIn, error)
+	grpc.ClientStream
+}
+
+type volthaServiceReceivePacketsInClient struct {
+	grpc.ClientStream
+}
+
+func (x *volthaServiceReceivePacketsInClient) Recv() (*openflow_13.PacketIn, error) {
+	m := new(openflow_13.PacketIn)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *volthaServiceClient) ReceiveChangeEvents(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (VolthaService_ReceiveChangeEventsClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_VolthaService_serviceDesc.Streams[2], "/voltha.VolthaService/ReceiveChangeEvents", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &volthaServiceReceiveChangeEventsClient{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 VolthaService_ReceiveChangeEventsClient interface {
+	Recv() (*openflow_13.ChangeEvent, error)
+	grpc.ClientStream
+}
+
+type volthaServiceReceiveChangeEventsClient struct {
+	grpc.ClientStream
+}
+
+func (x *volthaServiceReceiveChangeEventsClient) Recv() (*openflow_13.ChangeEvent, error) {
+	m := new(openflow_13.ChangeEvent)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *volthaServiceClient) GetDeviceGroup(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*DeviceGroup, error) {
+	out := new(DeviceGroup)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetDeviceGroup", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) CreateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error) {
+	out := new(EventFilter)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/CreateEventFilter", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetEventFilter(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*EventFilters, error) {
+	out := new(EventFilters)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetEventFilter", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) UpdateEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*EventFilter, error) {
+	out := new(EventFilter)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/UpdateEventFilter", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DeleteEventFilter(ctx context.Context, in *EventFilter, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DeleteEventFilter", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) ListEventFilters(ctx context.Context, in *empty.Empty, opts ...grpc.CallOption) (*EventFilters, error) {
+	out := new(EventFilters)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/ListEventFilters", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetImages(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*Images, error) {
+	out := new(Images)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetImages", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) SelfTest(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*SelfTestResponse, error) {
+	out := new(SelfTestResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SelfTest", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetMibDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.MibDeviceData, error) {
+	out := new(omci.MibDeviceData)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetMibDeviceData", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetAlarmDeviceData(ctx context.Context, in *common.ID, opts ...grpc.CallOption) (*omci.AlarmDeviceData, error) {
+	out := new(omci.AlarmDeviceData)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetAlarmDeviceData", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) SimulateAlarm(ctx context.Context, in *SimulateAlarmRequest, opts ...grpc.CallOption) (*common.OperationResp, error) {
+	out := new(common.OperationResp)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SimulateAlarm", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) Subscribe(ctx context.Context, in *OfAgentSubscriber, opts ...grpc.CallOption) (*OfAgentSubscriber, error) {
+	out := new(OfAgentSubscriber)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/Subscribe", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) EnablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/EnablePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) DisablePort(ctx context.Context, in *Port, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/DisablePort", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) GetExtValue(ctx context.Context, in *common.ValueSpecifier, opts ...grpc.CallOption) (*common.ReturnValues, error) {
+	out := new(common.ReturnValues)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/GetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) SetExtValue(ctx context.Context, in *ValueSet, opts ...grpc.CallOption) (*empty.Empty, error) {
+	out := new(empty.Empty)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/SetExtValue", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *volthaServiceClient) StartOmciTestAction(ctx context.Context, in *OmciTestRequest, opts ...grpc.CallOption) (*TestResponse, error) {
+	out := new(TestResponse)
+	err := c.cc.Invoke(ctx, "/voltha.VolthaService/StartOmciTestAction", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// VolthaServiceServer is the server API for VolthaService service.
+type VolthaServiceServer interface {
+	// Get the membership group of a Voltha Core
+	GetMembership(context.Context, *empty.Empty) (*Membership, error)
+	// Set the membership group of a Voltha Core
+	UpdateMembership(context.Context, *Membership) (*empty.Empty, error)
+	// Get high level information on the Voltha cluster
+	GetVoltha(context.Context, *empty.Empty) (*Voltha, error)
+	// List all Voltha cluster core instances
+	ListCoreInstances(context.Context, *empty.Empty) (*CoreInstances, error)
+	// Get details on a Voltha cluster instance
+	GetCoreInstance(context.Context, *common.ID) (*CoreInstance, error)
+	// List all active adapters (plugins) in the Voltha cluster
+	ListAdapters(context.Context, *empty.Empty) (*Adapters, error)
+	// List all logical devices managed by the Voltha cluster
+	ListLogicalDevices(context.Context, *empty.Empty) (*LogicalDevices, error)
+	// Get additional information on a given logical device
+	GetLogicalDevice(context.Context, *common.ID) (*LogicalDevice, error)
+	// List ports of a logical device
+	ListLogicalDevicePorts(context.Context, *common.ID) (*LogicalPorts, error)
+	// Gets a logical device port
+	GetLogicalDevicePort(context.Context, *LogicalPortId) (*LogicalPort, error)
+	// Enables a logical device port
+	EnableLogicalDevicePort(context.Context, *LogicalPortId) (*empty.Empty, error)
+	// Disables a logical device port
+	DisableLogicalDevicePort(context.Context, *LogicalPortId) (*empty.Empty, error)
+	// List all flows of a logical device
+	ListLogicalDeviceFlows(context.Context, *common.ID) (*openflow_13.Flows, error)
+	// Update flow table for logical device
+	UpdateLogicalDeviceFlowTable(context.Context, *openflow_13.FlowTableUpdate) (*empty.Empty, error)
+	// Update meter table for logical device
+	UpdateLogicalDeviceMeterTable(context.Context, *openflow_13.MeterModUpdate) (*empty.Empty, error)
+	// List all meters of a logical device
+	ListLogicalDeviceMeters(context.Context, *common.ID) (*openflow_13.Meters, error)
+	// List all flow groups of a logical device
+	ListLogicalDeviceFlowGroups(context.Context, *common.ID) (*openflow_13.FlowGroups, error)
+	// Update group table for device
+	UpdateLogicalDeviceFlowGroupTable(context.Context, *openflow_13.FlowGroupTableUpdate) (*empty.Empty, error)
+	// List all physical devices controlled by the Voltha cluster
+	ListDevices(context.Context, *empty.Empty) (*Devices, error)
+	// List all physical devices IDs controlled by the Voltha cluster
+	ListDeviceIds(context.Context, *empty.Empty) (*common.IDs, error)
+	// Request to a voltha Core to reconcile a set of devices based on their IDs
+	ReconcileDevices(context.Context, *common.IDs) (*empty.Empty, error)
+	// Get more information on a given physical device
+	GetDevice(context.Context, *common.ID) (*Device, error)
+	// Pre-provision a new physical device
+	CreateDevice(context.Context, *Device) (*Device, error)
+	// Enable a device.  If the device was in pre-provisioned state then it
+	// will transition to ENABLED state.  If it was is DISABLED state then it
+	// will transition to ENABLED state as well.
+	EnableDevice(context.Context, *common.ID) (*empty.Empty, error)
+	// Disable a device
+	DisableDevice(context.Context, *common.ID) (*empty.Empty, error)
+	// Reboot a device
+	RebootDevice(context.Context, *common.ID) (*empty.Empty, error)
+	// Delete a device
+	DeleteDevice(context.Context, *common.ID) (*empty.Empty, error)
+	// Forcefully delete a device
+	ForceDeleteDevice(context.Context, *common.ID) (*empty.Empty, error)
+	// Request an image download to the standby partition
+	// of a device.
+	// Note that the call is expected to be non-blocking.
+	DownloadImage(context.Context, *ImageDownload) (*common.OperationResp, error)
+	// Get image download status on a device
+	// The request retrieves progress on device and updates db record
+	// Deprecated in voltha 2.8, will be removed
+	GetImageDownloadStatus(context.Context, *ImageDownload) (*ImageDownload, error)
+	// Get image download db record
+	// Deprecated in voltha 2.8, will be removed
+	GetImageDownload(context.Context, *ImageDownload) (*ImageDownload, error)
+	// List image download db records for a given device
+	// Deprecated in voltha 2.8, will be removed
+	ListImageDownloads(context.Context, *common.ID) (*ImageDownloads, error)
+	// Cancel an existing image download process on a device
+	// Deprecated in voltha 2.8, will be removed
+	CancelImageDownload(context.Context, *ImageDownload) (*common.OperationResp, error)
+	// Activate the specified image at a standby partition
+	// to active partition.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device
+	// Note that the call is expected to be non-blocking.
+	// Deprecated in voltha 2.8, will be removed
+	ActivateImageUpdate(context.Context, *ImageDownload) (*common.OperationResp, error)
+	// Revert the specified image at standby partition
+	// to active partition, and revert to previous image
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// previous image running on device
+	// Note that the call is expected to be non-blocking.
+	// Deprecated in voltha 2.8, will be removed
+	RevertImageUpdate(context.Context, *ImageDownload) (*common.OperationResp, error)
+	// Downloads a certain image to the standby partition of the devices
+	// Note that the call is expected to be non-blocking.
+	DownloadImageToDevice(context.Context, *DeviceImageDownloadRequest) (*DeviceImageResponse, error)
+	// Get image status on a number of devices devices
+	// Polled from northbound systems to get state of download/activate/commit
+	GetImageStatus(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
+	// Aborts the upgrade of an image on a device
+	// To be used carefully, stops any further operations for the Image on the given devices
+	// Might also stop if possible existing work, but no guarantees are given,
+	// depends on implementation and procedure status.
+	AbortImageUpgradeToDevice(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
+	// Get Both Active and Standby image for a given device
+	GetOnuImages(context.Context, *common.ID) (*OnuImages, error)
+	// Activate the specified image from a standby partition
+	// to active partition.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device
+	// Note that the call is expected to be non-blocking.
+	ActivateImage(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
+	// Commit the specified image to be default.
+	// Depending on the device implementation, this call
+	// may or may not cause device reboot.
+	// If no reboot, then a reboot is required to make the
+	// activated image running on device upon next reboot
+	// Note that the call is expected to be non-blocking.
+	CommitImage(context.Context, *DeviceImageRequest) (*DeviceImageResponse, error)
+	// List ports of a device
+	ListDevicePorts(context.Context, *common.ID) (*Ports, error)
+	// List pm config of a device
+	ListDevicePmConfigs(context.Context, *common.ID) (*PmConfigs, error)
+	// Update the pm config of a device
+	UpdateDevicePmConfigs(context.Context, *PmConfigs) (*empty.Empty, error)
+	// List all flows of a device
+	ListDeviceFlows(context.Context, *common.ID) (*openflow_13.Flows, error)
+	// List all flow groups of a device
+	ListDeviceFlowGroups(context.Context, *common.ID) (*openflow_13.FlowGroups, error)
+	// List device types known to Voltha
+	ListDeviceTypes(context.Context, *empty.Empty) (*DeviceTypes, error)
+	// Get additional information on a device type
+	GetDeviceType(context.Context, *common.ID) (*DeviceType, error)
+	// List all device sharding groups
+	ListDeviceGroups(context.Context, *empty.Empty) (*DeviceGroups, error)
+	// Stream control packets to the dataplane
+	StreamPacketsOut(VolthaService_StreamPacketsOutServer) error
+	// Receive control packet stream
+	ReceivePacketsIn(*empty.Empty, VolthaService_ReceivePacketsInServer) error
+	ReceiveChangeEvents(*empty.Empty, VolthaService_ReceiveChangeEventsServer) error
+	// Get additional information on a device group
+	GetDeviceGroup(context.Context, *common.ID) (*DeviceGroup, error)
+	CreateEventFilter(context.Context, *EventFilter) (*EventFilter, error)
+	// Get all filters present for a device
+	GetEventFilter(context.Context, *common.ID) (*EventFilters, error)
+	UpdateEventFilter(context.Context, *EventFilter) (*EventFilter, error)
+	DeleteEventFilter(context.Context, *EventFilter) (*empty.Empty, error)
+	// Get all the filters present
+	ListEventFilters(context.Context, *empty.Empty) (*EventFilters, error)
+	GetImages(context.Context, *common.ID) (*Images, error)
+	SelfTest(context.Context, *common.ID) (*SelfTestResponse, error)
+	// OpenOMCI MIB information
+	GetMibDeviceData(context.Context, *common.ID) (*omci.MibDeviceData, error)
+	// OpenOMCI ALARM information
+	GetAlarmDeviceData(context.Context, *common.ID) (*omci.AlarmDeviceData, error)
+	// Simulate an Alarm
+	SimulateAlarm(context.Context, *SimulateAlarmRequest) (*common.OperationResp, error)
+	Subscribe(context.Context, *OfAgentSubscriber) (*OfAgentSubscriber, error)
+	EnablePort(context.Context, *Port) (*empty.Empty, error)
+	DisablePort(context.Context, *Port) (*empty.Empty, error)
+	GetExtValue(context.Context, *common.ValueSpecifier) (*common.ReturnValues, error)
+	SetExtValue(context.Context, *ValueSet) (*empty.Empty, error)
+	// omci start and stop cli implementation
+	StartOmciTestAction(context.Context, *OmciTestRequest) (*TestResponse, error)
+}
+
+// UnimplementedVolthaServiceServer can be embedded to have forward compatible implementations.
+type UnimplementedVolthaServiceServer struct {
+}
+
+func (*UnimplementedVolthaServiceServer) GetMembership(ctx context.Context, req *empty.Empty) (*Membership, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetMembership not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateMembership(ctx context.Context, req *Membership) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateMembership not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetVoltha(ctx context.Context, req *empty.Empty) (*Voltha, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetVoltha not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListCoreInstances(ctx context.Context, req *empty.Empty) (*CoreInstances, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListCoreInstances not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetCoreInstance(ctx context.Context, req *common.ID) (*CoreInstance, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetCoreInstance not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListAdapters(ctx context.Context, req *empty.Empty) (*Adapters, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListAdapters not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListLogicalDevices(ctx context.Context, req *empty.Empty) (*LogicalDevices, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDevices not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetLogicalDevice(ctx context.Context, req *common.ID) (*LogicalDevice, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListLogicalDevicePorts(ctx context.Context, req *common.ID) (*LogicalPorts, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDevicePorts not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*LogicalPort, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetLogicalDevicePort not implemented")
+}
+func (*UnimplementedVolthaServiceServer) EnableLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method EnableLogicalDevicePort not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DisableLogicalDevicePort(ctx context.Context, req *LogicalPortId) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisableLogicalDevicePort not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListLogicalDeviceFlows(ctx context.Context, req *common.ID) (*openflow_13.Flows, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceFlows not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceFlowTable(ctx context.Context, req *openflow_13.FlowTableUpdate) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceFlowTable not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceMeterTable(ctx context.Context, req *openflow_13.MeterModUpdate) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceMeterTable not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListLogicalDeviceMeters(ctx context.Context, req *common.ID) (*openflow_13.Meters, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceMeters not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListLogicalDeviceFlowGroups(ctx context.Context, req *common.ID) (*openflow_13.FlowGroups, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListLogicalDeviceFlowGroups not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateLogicalDeviceFlowGroupTable(ctx context.Context, req *openflow_13.FlowGroupTableUpdate) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateLogicalDeviceFlowGroupTable not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDevices(ctx context.Context, req *empty.Empty) (*Devices, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDevices not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDeviceIds(ctx context.Context, req *empty.Empty) (*common.IDs, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceIds not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ReconcileDevices(ctx context.Context, req *common.IDs) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ReconcileDevices not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetDevice(ctx context.Context, req *common.ID) (*Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) CreateDevice(ctx context.Context, req *Device) (*Device, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CreateDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) EnableDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method EnableDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DisableDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisableDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) RebootDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RebootDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DeleteDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ForceDeleteDevice(ctx context.Context, req *common.ID) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ForceDeleteDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DownloadImage(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DownloadImage not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetImageDownloadStatus(ctx context.Context, req *ImageDownload) (*ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetImageDownloadStatus not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetImageDownload(ctx context.Context, req *ImageDownload) (*ImageDownload, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetImageDownload not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListImageDownloads(ctx context.Context, req *common.ID) (*ImageDownloads, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListImageDownloads not implemented")
+}
+func (*UnimplementedVolthaServiceServer) CancelImageDownload(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CancelImageDownload not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ActivateImageUpdate(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ActivateImageUpdate not implemented")
+}
+func (*UnimplementedVolthaServiceServer) RevertImageUpdate(ctx context.Context, req *ImageDownload) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method RevertImageUpdate not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DownloadImageToDevice(ctx context.Context, req *DeviceImageDownloadRequest) (*DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DownloadImageToDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetImageStatus(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetImageStatus not implemented")
+}
+func (*UnimplementedVolthaServiceServer) AbortImageUpgradeToDevice(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method AbortImageUpgradeToDevice not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetOnuImages(ctx context.Context, req *common.ID) (*OnuImages, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetOnuImages not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ActivateImage(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ActivateImage not implemented")
+}
+func (*UnimplementedVolthaServiceServer) CommitImage(ctx context.Context, req *DeviceImageRequest) (*DeviceImageResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CommitImage not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDevicePorts(ctx context.Context, req *common.ID) (*Ports, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDevicePorts not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDevicePmConfigs(ctx context.Context, req *common.ID) (*PmConfigs, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDevicePmConfigs not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateDevicePmConfigs(ctx context.Context, req *PmConfigs) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateDevicePmConfigs not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDeviceFlows(ctx context.Context, req *common.ID) (*openflow_13.Flows, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceFlows not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDeviceFlowGroups(ctx context.Context, req *common.ID) (*openflow_13.FlowGroups, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceFlowGroups not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDeviceTypes(ctx context.Context, req *empty.Empty) (*DeviceTypes, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceTypes not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetDeviceType(ctx context.Context, req *common.ID) (*DeviceType, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceType not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListDeviceGroups(ctx context.Context, req *empty.Empty) (*DeviceGroups, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListDeviceGroups not implemented")
+}
+func (*UnimplementedVolthaServiceServer) StreamPacketsOut(srv VolthaService_StreamPacketsOutServer) error {
+	return status.Errorf(codes.Unimplemented, "method StreamPacketsOut not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ReceivePacketsIn(req *empty.Empty, srv VolthaService_ReceivePacketsInServer) error {
+	return status.Errorf(codes.Unimplemented, "method ReceivePacketsIn not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ReceiveChangeEvents(req *empty.Empty, srv VolthaService_ReceiveChangeEventsServer) error {
+	return status.Errorf(codes.Unimplemented, "method ReceiveChangeEvents not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetDeviceGroup(ctx context.Context, req *common.ID) (*DeviceGroup, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetDeviceGroup not implemented")
+}
+func (*UnimplementedVolthaServiceServer) CreateEventFilter(ctx context.Context, req *EventFilter) (*EventFilter, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method CreateEventFilter not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetEventFilter(ctx context.Context, req *common.ID) (*EventFilters, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetEventFilter not implemented")
+}
+func (*UnimplementedVolthaServiceServer) UpdateEventFilter(ctx context.Context, req *EventFilter) (*EventFilter, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method UpdateEventFilter not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DeleteEventFilter(ctx context.Context, req *EventFilter) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DeleteEventFilter not implemented")
+}
+func (*UnimplementedVolthaServiceServer) ListEventFilters(ctx context.Context, req *empty.Empty) (*EventFilters, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ListEventFilters not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetImages(ctx context.Context, req *common.ID) (*Images, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetImages not implemented")
+}
+func (*UnimplementedVolthaServiceServer) SelfTest(ctx context.Context, req *common.ID) (*SelfTestResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SelfTest not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetMibDeviceData(ctx context.Context, req *common.ID) (*omci.MibDeviceData, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetMibDeviceData not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetAlarmDeviceData(ctx context.Context, req *common.ID) (*omci.AlarmDeviceData, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetAlarmDeviceData not implemented")
+}
+func (*UnimplementedVolthaServiceServer) SimulateAlarm(ctx context.Context, req *SimulateAlarmRequest) (*common.OperationResp, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SimulateAlarm not implemented")
+}
+func (*UnimplementedVolthaServiceServer) Subscribe(ctx context.Context, req *OfAgentSubscriber) (*OfAgentSubscriber, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
+}
+func (*UnimplementedVolthaServiceServer) EnablePort(ctx context.Context, req *Port) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method EnablePort not implemented")
+}
+func (*UnimplementedVolthaServiceServer) DisablePort(ctx context.Context, req *Port) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method DisablePort not implemented")
+}
+func (*UnimplementedVolthaServiceServer) GetExtValue(ctx context.Context, req *common.ValueSpecifier) (*common.ReturnValues, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method GetExtValue not implemented")
+}
+func (*UnimplementedVolthaServiceServer) SetExtValue(ctx context.Context, req *ValueSet) (*empty.Empty, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method SetExtValue not implemented")
+}
+func (*UnimplementedVolthaServiceServer) StartOmciTestAction(ctx context.Context, req *OmciTestRequest) (*TestResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method StartOmciTestAction not implemented")
+}
+
+func RegisterVolthaServiceServer(s *grpc.Server, srv VolthaServiceServer) {
+	s.RegisterService(&_VolthaService_serviceDesc, srv)
+}
+
+func _VolthaService_GetMembership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetMembership(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetMembership",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetMembership(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateMembership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Membership)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateMembership(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateMembership",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateMembership(ctx, req.(*Membership))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetVoltha_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetVoltha(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetVoltha",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetVoltha(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListCoreInstances_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListCoreInstances(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListCoreInstances",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListCoreInstances(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetCoreInstance_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetCoreInstance(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetCoreInstance",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetCoreInstance(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListAdapters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListAdapters(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListAdapters",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListAdapters(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListLogicalDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListLogicalDevices(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListLogicalDevices",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListLogicalDevices(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetLogicalDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetLogicalDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetLogicalDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetLogicalDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListLogicalDevicePorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListLogicalDevicePorts(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListLogicalDevicePorts",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListLogicalDevicePorts(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LogicalPortId)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetLogicalDevicePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetLogicalDevicePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetLogicalDevicePort(ctx, req.(*LogicalPortId))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_EnableLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LogicalPortId)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).EnableLogicalDevicePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/EnableLogicalDevicePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).EnableLogicalDevicePort(ctx, req.(*LogicalPortId))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DisableLogicalDevicePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LogicalPortId)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DisableLogicalDevicePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DisableLogicalDevicePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DisableLogicalDevicePort(ctx, req.(*LogicalPortId))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListLogicalDeviceFlows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListLogicalDeviceFlows(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListLogicalDeviceFlows",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListLogicalDeviceFlows(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateLogicalDeviceFlowTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(openflow_13.FlowTableUpdate)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowTable(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceFlowTable",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowTable(ctx, req.(*openflow_13.FlowTableUpdate))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateLogicalDeviceMeterTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(openflow_13.MeterModUpdate)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceMeterTable(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceMeterTable",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceMeterTable(ctx, req.(*openflow_13.MeterModUpdate))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListLogicalDeviceMeters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListLogicalDeviceMeters(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListLogicalDeviceMeters",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListLogicalDeviceMeters(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListLogicalDeviceFlowGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListLogicalDeviceFlowGroups(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListLogicalDeviceFlowGroups",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListLogicalDeviceFlowGroups(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateLogicalDeviceFlowGroupTable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(openflow_13.FlowGroupTableUpdate)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowGroupTable(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateLogicalDeviceFlowGroupTable",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateLogicalDeviceFlowGroupTable(ctx, req.(*openflow_13.FlowGroupTableUpdate))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDevices(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDevices",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDevices(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDeviceIds_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDeviceIds(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDeviceIds",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDeviceIds(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ReconcileDevices_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.IDs)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ReconcileDevices(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ReconcileDevices",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ReconcileDevices(ctx, req.(*common.IDs))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_CreateDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Device)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).CreateDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/CreateDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).CreateDevice(ctx, req.(*Device))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_EnableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).EnableDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/EnableDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).EnableDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DisableDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DisableDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DisableDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DisableDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_RebootDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).RebootDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/RebootDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).RebootDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DeleteDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DeleteDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DeleteDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DeleteDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ForceDeleteDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ForceDeleteDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ForceDeleteDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ForceDeleteDevice(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DownloadImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DownloadImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DownloadImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DownloadImage(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetImageDownloadStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetImageDownloadStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetImageDownloadStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetImageDownloadStatus(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetImageDownload(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetImageDownload",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetImageDownload(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListImageDownloads_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListImageDownloads(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListImageDownloads",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListImageDownloads(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_CancelImageDownload_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).CancelImageDownload(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/CancelImageDownload",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).CancelImageDownload(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ActivateImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ActivateImageUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ActivateImageUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ActivateImageUpdate(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_RevertImageUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ImageDownload)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).RevertImageUpdate(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/RevertImageUpdate",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).RevertImageUpdate(ctx, req.(*ImageDownload))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DownloadImageToDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeviceImageDownloadRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DownloadImageToDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DownloadImageToDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DownloadImageToDevice(ctx, req.(*DeviceImageDownloadRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetImageStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetImageStatus(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetImageStatus",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetImageStatus(ctx, req.(*DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_AbortImageUpgradeToDevice_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).AbortImageUpgradeToDevice(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/AbortImageUpgradeToDevice",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).AbortImageUpgradeToDevice(ctx, req.(*DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetOnuImages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetOnuImages(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetOnuImages",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetOnuImages(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ActivateImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ActivateImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ActivateImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ActivateImage(ctx, req.(*DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_CommitImage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(DeviceImageRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).CommitImage(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/CommitImage",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).CommitImage(ctx, req.(*DeviceImageRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDevicePorts_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDevicePorts(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDevicePorts",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDevicePorts(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDevicePmConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDevicePmConfigs(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDevicePmConfigs",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDevicePmConfigs(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateDevicePmConfigs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(PmConfigs)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateDevicePmConfigs(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateDevicePmConfigs",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateDevicePmConfigs(ctx, req.(*PmConfigs))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDeviceFlows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDeviceFlows(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDeviceFlows",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDeviceFlows(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDeviceFlowGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDeviceFlowGroups(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDeviceFlowGroups",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDeviceFlowGroups(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDeviceTypes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDeviceTypes(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDeviceTypes",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDeviceTypes(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetDeviceType_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetDeviceType(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetDeviceType",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetDeviceType(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListDeviceGroups_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListDeviceGroups(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListDeviceGroups",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListDeviceGroups(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_StreamPacketsOut_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(VolthaServiceServer).StreamPacketsOut(&volthaServiceStreamPacketsOutServer{stream})
+}
+
+type VolthaService_StreamPacketsOutServer interface {
+	SendAndClose(*empty.Empty) error
+	Recv() (*openflow_13.PacketOut, error)
+	grpc.ServerStream
+}
+
+type volthaServiceStreamPacketsOutServer struct {
+	grpc.ServerStream
+}
+
+func (x *volthaServiceStreamPacketsOutServer) SendAndClose(m *empty.Empty) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *volthaServiceStreamPacketsOutServer) Recv() (*openflow_13.PacketOut, error) {
+	m := new(openflow_13.PacketOut)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func _VolthaService_ReceivePacketsIn_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(empty.Empty)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(VolthaServiceServer).ReceivePacketsIn(m, &volthaServiceReceivePacketsInServer{stream})
+}
+
+type VolthaService_ReceivePacketsInServer interface {
+	Send(*openflow_13.PacketIn) error
+	grpc.ServerStream
+}
+
+type volthaServiceReceivePacketsInServer struct {
+	grpc.ServerStream
+}
+
+func (x *volthaServiceReceivePacketsInServer) Send(m *openflow_13.PacketIn) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _VolthaService_ReceiveChangeEvents_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(empty.Empty)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(VolthaServiceServer).ReceiveChangeEvents(m, &volthaServiceReceiveChangeEventsServer{stream})
+}
+
+type VolthaService_ReceiveChangeEventsServer interface {
+	Send(*openflow_13.ChangeEvent) error
+	grpc.ServerStream
+}
+
+type volthaServiceReceiveChangeEventsServer struct {
+	grpc.ServerStream
+}
+
+func (x *volthaServiceReceiveChangeEventsServer) Send(m *openflow_13.ChangeEvent) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _VolthaService_GetDeviceGroup_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetDeviceGroup(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetDeviceGroup",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetDeviceGroup(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_CreateEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(EventFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).CreateEventFilter(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/CreateEventFilter",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).CreateEventFilter(ctx, req.(*EventFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetEventFilter(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetEventFilter",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetEventFilter(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_UpdateEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(EventFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).UpdateEventFilter(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/UpdateEventFilter",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).UpdateEventFilter(ctx, req.(*EventFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DeleteEventFilter_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(EventFilter)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DeleteEventFilter(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DeleteEventFilter",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DeleteEventFilter(ctx, req.(*EventFilter))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_ListEventFilters_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(empty.Empty)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).ListEventFilters(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/ListEventFilters",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).ListEventFilters(ctx, req.(*empty.Empty))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetImages_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetImages(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetImages",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetImages(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_SelfTest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).SelfTest(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/SelfTest",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).SelfTest(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetMibDeviceData_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetMibDeviceData(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetMibDeviceData",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetMibDeviceData(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetAlarmDeviceData_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ID)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetAlarmDeviceData(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetAlarmDeviceData",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetAlarmDeviceData(ctx, req.(*common.ID))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_SimulateAlarm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(SimulateAlarmRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).SimulateAlarm(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/SimulateAlarm",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).SimulateAlarm(ctx, req.(*SimulateAlarmRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_Subscribe_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(OfAgentSubscriber)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).Subscribe(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/Subscribe",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).Subscribe(ctx, req.(*OfAgentSubscriber))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_EnablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Port)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).EnablePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/EnablePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).EnablePort(ctx, req.(*Port))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_DisablePort_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(Port)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).DisablePort(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/DisablePort",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).DisablePort(ctx, req.(*Port))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_GetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(common.ValueSpecifier)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).GetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/GetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).GetExtValue(ctx, req.(*common.ValueSpecifier))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_SetExtValue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ValueSet)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).SetExtValue(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/SetExtValue",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).SetExtValue(ctx, req.(*ValueSet))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _VolthaService_StartOmciTestAction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(OmciTestRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(VolthaServiceServer).StartOmciTestAction(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/voltha.VolthaService/StartOmciTestAction",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(VolthaServiceServer).StartOmciTestAction(ctx, req.(*OmciTestRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _VolthaService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "voltha.VolthaService",
+	HandlerType: (*VolthaServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "GetMembership",
+			Handler:    _VolthaService_GetMembership_Handler,
+		},
+		{
+			MethodName: "UpdateMembership",
+			Handler:    _VolthaService_UpdateMembership_Handler,
+		},
+		{
+			MethodName: "GetVoltha",
+			Handler:    _VolthaService_GetVoltha_Handler,
+		},
+		{
+			MethodName: "ListCoreInstances",
+			Handler:    _VolthaService_ListCoreInstances_Handler,
+		},
+		{
+			MethodName: "GetCoreInstance",
+			Handler:    _VolthaService_GetCoreInstance_Handler,
+		},
+		{
+			MethodName: "ListAdapters",
+			Handler:    _VolthaService_ListAdapters_Handler,
+		},
+		{
+			MethodName: "ListLogicalDevices",
+			Handler:    _VolthaService_ListLogicalDevices_Handler,
+		},
+		{
+			MethodName: "GetLogicalDevice",
+			Handler:    _VolthaService_GetLogicalDevice_Handler,
+		},
+		{
+			MethodName: "ListLogicalDevicePorts",
+			Handler:    _VolthaService_ListLogicalDevicePorts_Handler,
+		},
+		{
+			MethodName: "GetLogicalDevicePort",
+			Handler:    _VolthaService_GetLogicalDevicePort_Handler,
+		},
+		{
+			MethodName: "EnableLogicalDevicePort",
+			Handler:    _VolthaService_EnableLogicalDevicePort_Handler,
+		},
+		{
+			MethodName: "DisableLogicalDevicePort",
+			Handler:    _VolthaService_DisableLogicalDevicePort_Handler,
+		},
+		{
+			MethodName: "ListLogicalDeviceFlows",
+			Handler:    _VolthaService_ListLogicalDeviceFlows_Handler,
+		},
+		{
+			MethodName: "UpdateLogicalDeviceFlowTable",
+			Handler:    _VolthaService_UpdateLogicalDeviceFlowTable_Handler,
+		},
+		{
+			MethodName: "UpdateLogicalDeviceMeterTable",
+			Handler:    _VolthaService_UpdateLogicalDeviceMeterTable_Handler,
+		},
+		{
+			MethodName: "ListLogicalDeviceMeters",
+			Handler:    _VolthaService_ListLogicalDeviceMeters_Handler,
+		},
+		{
+			MethodName: "ListLogicalDeviceFlowGroups",
+			Handler:    _VolthaService_ListLogicalDeviceFlowGroups_Handler,
+		},
+		{
+			MethodName: "UpdateLogicalDeviceFlowGroupTable",
+			Handler:    _VolthaService_UpdateLogicalDeviceFlowGroupTable_Handler,
+		},
+		{
+			MethodName: "ListDevices",
+			Handler:    _VolthaService_ListDevices_Handler,
+		},
+		{
+			MethodName: "ListDeviceIds",
+			Handler:    _VolthaService_ListDeviceIds_Handler,
+		},
+		{
+			MethodName: "ReconcileDevices",
+			Handler:    _VolthaService_ReconcileDevices_Handler,
+		},
+		{
+			MethodName: "GetDevice",
+			Handler:    _VolthaService_GetDevice_Handler,
+		},
+		{
+			MethodName: "CreateDevice",
+			Handler:    _VolthaService_CreateDevice_Handler,
+		},
+		{
+			MethodName: "EnableDevice",
+			Handler:    _VolthaService_EnableDevice_Handler,
+		},
+		{
+			MethodName: "DisableDevice",
+			Handler:    _VolthaService_DisableDevice_Handler,
+		},
+		{
+			MethodName: "RebootDevice",
+			Handler:    _VolthaService_RebootDevice_Handler,
+		},
+		{
+			MethodName: "DeleteDevice",
+			Handler:    _VolthaService_DeleteDevice_Handler,
+		},
+		{
+			MethodName: "ForceDeleteDevice",
+			Handler:    _VolthaService_ForceDeleteDevice_Handler,
+		},
+		{
+			MethodName: "DownloadImage",
+			Handler:    _VolthaService_DownloadImage_Handler,
+		},
+		{
+			MethodName: "GetImageDownloadStatus",
+			Handler:    _VolthaService_GetImageDownloadStatus_Handler,
+		},
+		{
+			MethodName: "GetImageDownload",
+			Handler:    _VolthaService_GetImageDownload_Handler,
+		},
+		{
+			MethodName: "ListImageDownloads",
+			Handler:    _VolthaService_ListImageDownloads_Handler,
+		},
+		{
+			MethodName: "CancelImageDownload",
+			Handler:    _VolthaService_CancelImageDownload_Handler,
+		},
+		{
+			MethodName: "ActivateImageUpdate",
+			Handler:    _VolthaService_ActivateImageUpdate_Handler,
+		},
+		{
+			MethodName: "RevertImageUpdate",
+			Handler:    _VolthaService_RevertImageUpdate_Handler,
+		},
+		{
+			MethodName: "DownloadImageToDevice",
+			Handler:    _VolthaService_DownloadImageToDevice_Handler,
+		},
+		{
+			MethodName: "GetImageStatus",
+			Handler:    _VolthaService_GetImageStatus_Handler,
+		},
+		{
+			MethodName: "AbortImageUpgradeToDevice",
+			Handler:    _VolthaService_AbortImageUpgradeToDevice_Handler,
+		},
+		{
+			MethodName: "GetOnuImages",
+			Handler:    _VolthaService_GetOnuImages_Handler,
+		},
+		{
+			MethodName: "ActivateImage",
+			Handler:    _VolthaService_ActivateImage_Handler,
+		},
+		{
+			MethodName: "CommitImage",
+			Handler:    _VolthaService_CommitImage_Handler,
+		},
+		{
+			MethodName: "ListDevicePorts",
+			Handler:    _VolthaService_ListDevicePorts_Handler,
+		},
+		{
+			MethodName: "ListDevicePmConfigs",
+			Handler:    _VolthaService_ListDevicePmConfigs_Handler,
+		},
+		{
+			MethodName: "UpdateDevicePmConfigs",
+			Handler:    _VolthaService_UpdateDevicePmConfigs_Handler,
+		},
+		{
+			MethodName: "ListDeviceFlows",
+			Handler:    _VolthaService_ListDeviceFlows_Handler,
+		},
+		{
+			MethodName: "ListDeviceFlowGroups",
+			Handler:    _VolthaService_ListDeviceFlowGroups_Handler,
+		},
+		{
+			MethodName: "ListDeviceTypes",
+			Handler:    _VolthaService_ListDeviceTypes_Handler,
+		},
+		{
+			MethodName: "GetDeviceType",
+			Handler:    _VolthaService_GetDeviceType_Handler,
+		},
+		{
+			MethodName: "ListDeviceGroups",
+			Handler:    _VolthaService_ListDeviceGroups_Handler,
+		},
+		{
+			MethodName: "GetDeviceGroup",
+			Handler:    _VolthaService_GetDeviceGroup_Handler,
+		},
+		{
+			MethodName: "CreateEventFilter",
+			Handler:    _VolthaService_CreateEventFilter_Handler,
+		},
+		{
+			MethodName: "GetEventFilter",
+			Handler:    _VolthaService_GetEventFilter_Handler,
+		},
+		{
+			MethodName: "UpdateEventFilter",
+			Handler:    _VolthaService_UpdateEventFilter_Handler,
+		},
+		{
+			MethodName: "DeleteEventFilter",
+			Handler:    _VolthaService_DeleteEventFilter_Handler,
+		},
+		{
+			MethodName: "ListEventFilters",
+			Handler:    _VolthaService_ListEventFilters_Handler,
+		},
+		{
+			MethodName: "GetImages",
+			Handler:    _VolthaService_GetImages_Handler,
+		},
+		{
+			MethodName: "SelfTest",
+			Handler:    _VolthaService_SelfTest_Handler,
+		},
+		{
+			MethodName: "GetMibDeviceData",
+			Handler:    _VolthaService_GetMibDeviceData_Handler,
+		},
+		{
+			MethodName: "GetAlarmDeviceData",
+			Handler:    _VolthaService_GetAlarmDeviceData_Handler,
+		},
+		{
+			MethodName: "SimulateAlarm",
+			Handler:    _VolthaService_SimulateAlarm_Handler,
+		},
+		{
+			MethodName: "Subscribe",
+			Handler:    _VolthaService_Subscribe_Handler,
+		},
+		{
+			MethodName: "EnablePort",
+			Handler:    _VolthaService_EnablePort_Handler,
+		},
+		{
+			MethodName: "DisablePort",
+			Handler:    _VolthaService_DisablePort_Handler,
+		},
+		{
+			MethodName: "GetExtValue",
+			Handler:    _VolthaService_GetExtValue_Handler,
+		},
+		{
+			MethodName: "SetExtValue",
+			Handler:    _VolthaService_SetExtValue_Handler,
+		},
+		{
+			MethodName: "StartOmciTestAction",
+			Handler:    _VolthaService_StartOmciTestAction_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "StreamPacketsOut",
+			Handler:       _VolthaService_StreamPacketsOut_Handler,
+			ClientStreams: true,
+		},
+		{
+			StreamName:    "ReceivePacketsIn",
+			Handler:       _VolthaService_ReceivePacketsIn_Handler,
+			ServerStreams: true,
+		},
+		{
+			StreamName:    "ReceiveChangeEvents",
+			Handler:       _VolthaService_ReceiveChangeEvents_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "voltha_protos/voltha.proto",
+}
diff --git a/vendor/github.com/opentracing/opentracing-go/.travis.yml b/vendor/github.com/opentracing/opentracing-go/.travis.yml
index 8d5b75e..b950e42 100644
--- a/vendor/github.com/opentracing/opentracing-go/.travis.yml
+++ b/vendor/github.com/opentracing/opentracing-go/.travis.yml
@@ -2,8 +2,8 @@
 
 matrix:
   include:
-  - go: "1.11.x"
-  - go: "1.12.x"
+  - go: "1.13.x"
+  - go: "1.14.x"
   - go: "tip"
     env:
     - LINT=true
diff --git a/vendor/github.com/opentracing/opentracing-go/CHANGELOG.md b/vendor/github.com/opentracing/opentracing-go/CHANGELOG.md
index 7c14feb..d3bfcf6 100644
--- a/vendor/github.com/opentracing/opentracing-go/CHANGELOG.md
+++ b/vendor/github.com/opentracing/opentracing-go/CHANGELOG.md
@@ -1,6 +1,23 @@
 Changes by Version
 ==================
 
+
+1.2.0 (2020-07-01)
+-------------------
+
+* Restore the ability to reset the current span in context to nil (#231) -- Yuri Shkuro
+* Use error.object per OpenTracing Semantic Conventions (#179) -- Rahman Syed
+* Convert nil pointer log field value to string "nil" (#230) -- Cyril Tovena
+* Add Go module support (#215) -- Zaba505
+* Make SetTag helper types in ext public (#229) -- Blake Edwards
+* Add log/fields helpers for keys from specification (#226) -- Dmitry Monakhov
+* Improve noop impementation (#223) -- chanxuehong
+* Add an extension to Tracer interface for custom go context creation (#220) -- Krzesimir Nowak
+* Fix typo in comments (#222) -- meteorlxy
+* Improve documentation for log.Object() to emphasize the requirement to pass immutable arguments (#219) -- 疯狂的小企鹅
+* [mock] Return ErrInvalidSpanContext if span context is not MockSpanContext (#216) -- Milad Irannejad
+
+
 1.1.0 (2019-03-23)
 -------------------
 
diff --git a/vendor/github.com/opentracing/opentracing-go/ext.go b/vendor/github.com/opentracing/opentracing-go/ext.go
new file mode 100644
index 0000000..e11977e
--- /dev/null
+++ b/vendor/github.com/opentracing/opentracing-go/ext.go
@@ -0,0 +1,24 @@
+package opentracing
+
+import (
+	"context"
+)
+
+// TracerContextWithSpanExtension is an extension interface that the
+// implementation of the Tracer interface may want to implement. It
+// allows to have some control over the go context when the
+// ContextWithSpan is invoked.
+//
+// The primary purpose of this extension are adapters from opentracing
+// API to some other tracing API.
+type TracerContextWithSpanExtension interface {
+	// ContextWithSpanHook gets called by the ContextWithSpan
+	// function, when the Tracer implementation also implements
+	// this interface. It allows to put extra information into the
+	// context and make it available to the callers of the
+	// ContextWithSpan.
+	//
+	// This hook is invoked before the ContextWithSpan function
+	// actually puts the span into the context.
+	ContextWithSpanHook(ctx context.Context, span Span) context.Context
+}
diff --git a/vendor/github.com/opentracing/opentracing-go/ext/field.go b/vendor/github.com/opentracing/opentracing-go/ext/field.go
new file mode 100644
index 0000000..8282bd7
--- /dev/null
+++ b/vendor/github.com/opentracing/opentracing-go/ext/field.go
@@ -0,0 +1,17 @@
+package ext
+
+import (
+	"github.com/opentracing/opentracing-go"
+	"github.com/opentracing/opentracing-go/log"
+)
+
+// LogError sets the error=true tag on the Span and logs err as an "error" event.
+func LogError(span opentracing.Span, err error, fields ...log.Field) {
+	Error.Set(span, true)
+	ef := []log.Field{
+		log.Event("error"),
+		log.Error(err),
+	}
+	ef = append(ef, fields...)
+	span.LogFields(ef...)
+}
diff --git a/vendor/github.com/opentracing/opentracing-go/ext/tags.go b/vendor/github.com/opentracing/opentracing-go/ext/tags.go
index 52e8895..a414b59 100644
--- a/vendor/github.com/opentracing/opentracing-go/ext/tags.go
+++ b/vendor/github.com/opentracing/opentracing-go/ext/tags.go
@@ -47,40 +47,40 @@
 
 	// Component is a low-cardinality identifier of the module, library,
 	// or package that is generating a span.
-	Component = stringTagName("component")
+	Component = StringTagName("component")
 
 	//////////////////////////////////////////////////////////////////////
 	// Sampling hint
 	//////////////////////////////////////////////////////////////////////
 
 	// SamplingPriority determines the priority of sampling this Span.
-	SamplingPriority = uint16TagName("sampling.priority")
+	SamplingPriority = Uint16TagName("sampling.priority")
 
 	//////////////////////////////////////////////////////////////////////
-	// Peer tags. These tags can be emitted by either client-side of
+	// Peer tags. These tags can be emitted by either client-side or
 	// server-side to describe the other side/service in a peer-to-peer
 	// communications, like an RPC call.
 	//////////////////////////////////////////////////////////////////////
 
 	// PeerService records the service name of the peer.
-	PeerService = stringTagName("peer.service")
+	PeerService = StringTagName("peer.service")
 
 	// PeerAddress records the address name of the peer. This may be a "ip:port",
 	// a bare "hostname", a FQDN or even a database DSN substring
 	// like "mysql://username@127.0.0.1:3306/dbname"
-	PeerAddress = stringTagName("peer.address")
+	PeerAddress = StringTagName("peer.address")
 
 	// PeerHostname records the host name of the peer
-	PeerHostname = stringTagName("peer.hostname")
+	PeerHostname = StringTagName("peer.hostname")
 
 	// PeerHostIPv4 records IP v4 host address of the peer
-	PeerHostIPv4 = ipv4Tag("peer.ipv4")
+	PeerHostIPv4 = IPv4TagName("peer.ipv4")
 
 	// PeerHostIPv6 records IP v6 host address of the peer
-	PeerHostIPv6 = stringTagName("peer.ipv6")
+	PeerHostIPv6 = StringTagName("peer.ipv6")
 
 	// PeerPort records port number of the peer
-	PeerPort = uint16TagName("peer.port")
+	PeerPort = Uint16TagName("peer.port")
 
 	//////////////////////////////////////////////////////////////////////
 	// HTTP Tags
@@ -88,46 +88,46 @@
 
 	// HTTPUrl should be the URL of the request being handled in this segment
 	// of the trace, in standard URI format. The protocol is optional.
-	HTTPUrl = stringTagName("http.url")
+	HTTPUrl = StringTagName("http.url")
 
 	// HTTPMethod is the HTTP method of the request, and is case-insensitive.
-	HTTPMethod = stringTagName("http.method")
+	HTTPMethod = StringTagName("http.method")
 
 	// HTTPStatusCode is the numeric HTTP status code (200, 404, etc) of the
 	// HTTP response.
-	HTTPStatusCode = uint16TagName("http.status_code")
+	HTTPStatusCode = Uint16TagName("http.status_code")
 
 	//////////////////////////////////////////////////////////////////////
 	// DB Tags
 	//////////////////////////////////////////////////////////////////////
 
 	// DBInstance is database instance name.
-	DBInstance = stringTagName("db.instance")
+	DBInstance = StringTagName("db.instance")
 
 	// DBStatement is a database statement for the given database type.
 	// It can be a query or a prepared statement (i.e., before substitution).
-	DBStatement = stringTagName("db.statement")
+	DBStatement = StringTagName("db.statement")
 
 	// DBType is a database type. For any SQL database, "sql".
 	// For others, the lower-case database category, e.g. "redis"
-	DBType = stringTagName("db.type")
+	DBType = StringTagName("db.type")
 
 	// DBUser is a username for accessing database.
-	DBUser = stringTagName("db.user")
+	DBUser = StringTagName("db.user")
 
 	//////////////////////////////////////////////////////////////////////
 	// Message Bus Tag
 	//////////////////////////////////////////////////////////////////////
 
 	// MessageBusDestination is an address at which messages can be exchanged
-	MessageBusDestination = stringTagName("message_bus.destination")
+	MessageBusDestination = StringTagName("message_bus.destination")
 
 	//////////////////////////////////////////////////////////////////////
 	// Error Tag
 	//////////////////////////////////////////////////////////////////////
 
 	// Error indicates that operation represented by the span resulted in an error.
-	Error = boolTagName("error")
+	Error = BoolTagName("error")
 )
 
 // ---
@@ -163,48 +163,53 @@
 
 // ---
 
-type stringTagName string
+// StringTagName is a common tag name to be set to a string value
+type StringTagName string
 
 // Set adds a string tag to the `span`
-func (tag stringTagName) Set(span opentracing.Span, value string) {
+func (tag StringTagName) Set(span opentracing.Span, value string) {
 	span.SetTag(string(tag), value)
 }
 
 // ---
 
-type uint32TagName string
+// Uint32TagName is a common tag name to be set to a uint32 value
+type Uint32TagName string
 
 // Set adds a uint32 tag to the `span`
-func (tag uint32TagName) Set(span opentracing.Span, value uint32) {
+func (tag Uint32TagName) Set(span opentracing.Span, value uint32) {
 	span.SetTag(string(tag), value)
 }
 
 // ---
 
-type uint16TagName string
+// Uint16TagName is a common tag name to be set to a uint16 value
+type Uint16TagName string
 
 // Set adds a uint16 tag to the `span`
-func (tag uint16TagName) Set(span opentracing.Span, value uint16) {
+func (tag Uint16TagName) Set(span opentracing.Span, value uint16) {
 	span.SetTag(string(tag), value)
 }
 
 // ---
 
-type boolTagName string
+// BoolTagName is a common tag name to be set to a bool value
+type BoolTagName string
 
-// Add adds a bool tag to the `span`
-func (tag boolTagName) Set(span opentracing.Span, value bool) {
+// Set adds a bool tag to the `span`
+func (tag BoolTagName) Set(span opentracing.Span, value bool) {
 	span.SetTag(string(tag), value)
 }
 
-type ipv4Tag string
+// IPv4TagName is a common tag name to be set to an ipv4 value
+type IPv4TagName string
 
 // Set adds IP v4 host address of the peer as an uint32 value to the `span`, keep this for backward and zipkin compatibility
-func (tag ipv4Tag) Set(span opentracing.Span, value uint32) {
+func (tag IPv4TagName) Set(span opentracing.Span, value uint32) {
 	span.SetTag(string(tag), value)
 }
 
 // SetString records IP v4 host address of the peer as a .-separated tuple to the `span`. E.g., "127.0.0.1"
-func (tag ipv4Tag) SetString(span opentracing.Span, value string) {
+func (tag IPv4TagName) SetString(span opentracing.Span, value string) {
 	span.SetTag(string(tag), value)
 }
diff --git a/vendor/github.com/opentracing/opentracing-go/go.mod b/vendor/github.com/opentracing/opentracing-go/go.mod
new file mode 100644
index 0000000..bf48bb5
--- /dev/null
+++ b/vendor/github.com/opentracing/opentracing-go/go.mod
@@ -0,0 +1,5 @@
+module github.com/opentracing/opentracing-go
+
+go 1.14
+
+require github.com/stretchr/testify v1.3.0
diff --git a/vendor/github.com/opentracing/opentracing-go/go.sum b/vendor/github.com/opentracing/opentracing-go/go.sum
new file mode 100644
index 0000000..4347755
--- /dev/null
+++ b/vendor/github.com/opentracing/opentracing-go/go.sum
@@ -0,0 +1,7 @@
+github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+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/opentracing/opentracing-go/gocontext.go b/vendor/github.com/opentracing/opentracing-go/gocontext.go
index 08c00c0..1831bc9 100644
--- a/vendor/github.com/opentracing/opentracing-go/gocontext.go
+++ b/vendor/github.com/opentracing/opentracing-go/gocontext.go
@@ -7,8 +7,13 @@
 var activeSpanKey = contextKey{}
 
 // ContextWithSpan returns a new `context.Context` that holds a reference to
-// `span`'s SpanContext.
+// the span. If span is nil, a new context without an active span is returned.
 func ContextWithSpan(ctx context.Context, span Span) context.Context {
+	if span != nil {
+		if tracerWithHook, ok := span.Tracer().(TracerContextWithSpanExtension); ok {
+			ctx = tracerWithHook.ContextWithSpanHook(ctx, span)
+		}
+	}
 	return context.WithValue(ctx, activeSpanKey, span)
 }
 
diff --git a/vendor/github.com/opentracing/opentracing-go/log/field.go b/vendor/github.com/opentracing/opentracing-go/log/field.go
index 50feea3..f222ded 100644
--- a/vendor/github.com/opentracing/opentracing-go/log/field.go
+++ b/vendor/github.com/opentracing/opentracing-go/log/field.go
@@ -122,16 +122,19 @@
 	}
 }
 
-// Error adds an error with the key "error" to a Span.LogFields() record
+// Error adds an error with the key "error.object" to a Span.LogFields() record
 func Error(err error) Field {
 	return Field{
-		key:          "error",
+		key:          "error.object",
 		fieldType:    errorType,
 		interfaceVal: err,
 	}
 }
 
 // Object adds an object-valued key:value pair to a Span.LogFields() record
+// Please pass in an immutable object, otherwise there may be concurrency issues.
+// Such as passing in the map, log.Object may result in "fatal error: concurrent map iteration and map write".
+// Because span is sent asynchronously, it is possible that this map will also be modified.
 func Object(key string, obj interface{}) Field {
 	return Field{
 		key:          key,
@@ -140,6 +143,16 @@
 	}
 }
 
+// Event creates a string-valued Field for span logs with key="event" and value=val.
+func Event(val string) Field {
+	return String("event", val)
+}
+
+// Message creates a string-valued Field for span logs with key="message" and value=val.
+func Message(val string) Field {
+	return String("message", val)
+}
+
 // LazyLogger allows for user-defined, late-bound logging of arbitrary data
 type LazyLogger func(fv Encoder)
 
diff --git a/vendor/github.com/opentracing/opentracing-go/log/util.go b/vendor/github.com/opentracing/opentracing-go/log/util.go
index 3832feb..d57e28a 100644
--- a/vendor/github.com/opentracing/opentracing-go/log/util.go
+++ b/vendor/github.com/opentracing/opentracing-go/log/util.go
@@ -1,6 +1,9 @@
 package log
 
-import "fmt"
+import (
+	"fmt"
+	"reflect"
+)
 
 // InterleavedKVToFields converts keyValues a la Span.LogKV() to a Field slice
 // a la Span.LogFields().
@@ -46,6 +49,10 @@
 		case float64:
 			fields[i] = Float64(key, typedVal)
 		default:
+			if typedVal == nil || (reflect.ValueOf(typedVal).Kind() == reflect.Ptr && reflect.ValueOf(typedVal).IsNil()) {
+				fields[i] = String(key, "nil")
+				continue
+			}
 			// When in doubt, coerce to a string
 			fields[i] = String(key, fmt.Sprint(typedVal))
 		}
diff --git a/vendor/github.com/opentracing/opentracing-go/noop.go b/vendor/github.com/opentracing/opentracing-go/noop.go
index 0d32f69..f9b680a 100644
--- a/vendor/github.com/opentracing/opentracing-go/noop.go
+++ b/vendor/github.com/opentracing/opentracing-go/noop.go
@@ -21,9 +21,9 @@
 type noopSpanContext struct{}
 
 var (
-	defaultNoopSpanContext = noopSpanContext{}
-	defaultNoopSpan        = noopSpan{}
-	defaultNoopTracer      = NoopTracer{}
+	defaultNoopSpanContext SpanContext = noopSpanContext{}
+	defaultNoopSpan        Span        = noopSpan{}
+	defaultNoopTracer      Tracer      = NoopTracer{}
 )
 
 const (
@@ -35,7 +35,7 @@
 
 // noopSpan:
 func (n noopSpan) Context() SpanContext                                  { return defaultNoopSpanContext }
-func (n noopSpan) SetBaggageItem(key, val string) Span                   { return defaultNoopSpan }
+func (n noopSpan) SetBaggageItem(key, val string) Span                   { return n }
 func (n noopSpan) BaggageItem(key string) string                         { return emptyString }
 func (n noopSpan) SetTag(key string, value interface{}) Span             { return n }
 func (n noopSpan) LogFields(fields ...log.Field)                         {}
diff --git a/vendor/github.com/pierrec/lz4/README.md b/vendor/github.com/pierrec/lz4/README.md
index be1f52a..4ee388e 100644
--- a/vendor/github.com/pierrec/lz4/README.md
+++ b/vendor/github.com/pierrec/lz4/README.md
@@ -83,24 +83,8 @@
 
 ## Contributors
 
-Thanks to all contributors so far:
+Thanks to all [contributors](https://github.com/pierrec/lz4/graphs/contributors)  so far!
 
-- [@klauspost](https://github.com/klauspost)
-- [@heidawei](https://github.com/heidawei)
-- [@x4m](https://github.com/x4m)
-- [@Zariel](https://github.com/Zariel)
-- [@edwingeng](https://github.com/edwingeng)
-- [@danielmoy-google](https://github.com/danielmoy-google)
-- [@honda-tatsuya](https://github.com/honda-tatsuya)
-- [@h8liu](https://github.com/h8liu)
-- [@sbinet](https://github.com/sbinet)
-- [@fingon](https://github.com/fingon)
-- [@emfree](https://github.com/emfree)
-- [@lhemala](https://github.com/lhemala)
-- [@connor4312](https://github.com/connor4312)
-- [@oov](https://github.com/oov)
-- [@arya](https://github.com/arya)
-- [@ikkeps](https://github.com/ikkeps)
+Special thanks to [@Zariel](https://github.com/Zariel) for his asm implementation of the decoder.
 
-Special thanks to [@Zariel](https://github.com/Zariel) for his asm implementation of the decoder
-Special thanks to [@klauspost](https://github.com/klauspost) for his work on optimizing the code
+Special thanks to [@klauspost](https://github.com/klauspost) for his work on optimizing the code.
diff --git a/vendor/github.com/pierrec/lz4/block.go b/vendor/github.com/pierrec/lz4/block.go
index 5755cda..664d9be 100644
--- a/vendor/github.com/pierrec/lz4/block.go
+++ b/vendor/github.com/pierrec/lz4/block.go
@@ -2,8 +2,8 @@
 
 import (
 	"encoding/binary"
-	"fmt"
 	"math/bits"
+	"sync"
 )
 
 // blockHash hashes the lower 6 bytes into a value < htSize.
@@ -35,24 +35,31 @@
 
 // CompressBlock compresses the source buffer into the destination one.
 // This is the fast version of LZ4 compression and also the default one.
-// The size of hashTable must be at least 64Kb.
 //
-// The size of the compressed data is returned. If it is 0 and no error, then the data is incompressible.
+// The argument hashTable is scratch space for a hash table used by the
+// compressor. If provided, it should have length at least 1<<16. If it is
+// shorter (or nil), CompressBlock allocates its own hash table.
+//
+// The size of the compressed data is returned.
+//
+// If the destination buffer size is lower than CompressBlockBound and
+// the compressed size is 0 and no error, then the data is incompressible.
 //
 // An error is returned if the destination buffer is too small.
-func CompressBlock(src, dst []byte, hashTable []int) (di int, err error) {
+func CompressBlock(src, dst []byte, hashTable []int) (_ int, err error) {
 	defer recoverBlock(&err)
 
+	// Return 0, nil only if the destination buffer size is < CompressBlockBound.
+	isNotCompressible := len(dst) < CompressBlockBound(len(src))
+
 	// adaptSkipLog sets how quickly the compressor begins skipping blocks when data is incompressible.
-	// This significantly speeds up incompressible data and usually has very small impact on compresssion.
+	// This significantly speeds up incompressible data and usually has very small impact on compression.
 	// bytes to skip =  1 + (bytes since last match >> adaptSkipLog)
 	const adaptSkipLog = 7
-	sn, dn := len(src)-mfLimit, len(dst)
-	if sn <= 0 || dn == 0 {
-		return 0, nil
-	}
 	if len(hashTable) < htSize {
-		return 0, fmt.Errorf("hash table too small, should be at least %d in size", htSize)
+		htIface := htPool.Get()
+		defer htPool.Put(htIface)
+		hashTable = (*(htIface).(*[htSize]int))[:]
 	}
 	// Prove to the compiler the table has at least htSize elements.
 	// The compiler can see that "uint32() >> hashShift" cannot be out of bounds.
@@ -60,7 +67,11 @@
 
 	// si: Current position of the search.
 	// anchor: Position of the current literals.
-	var si, anchor int
+	var si, di, anchor int
+	sn := len(src) - mfLimit
+	if sn <= 0 {
+		goto lastLiterals
+	}
 
 	// Fast scan strategy: the hash table only stores the last 4 bytes sequences.
 	for si < sn {
@@ -124,7 +135,7 @@
 		si, mLen = si+mLen, si+minMatch
 
 		// Find the longest match by looking by batches of 8 bytes.
-		for si < sn {
+		for si+8 < sn {
 			x := binary.LittleEndian.Uint64(src[si:]) ^ binary.LittleEndian.Uint64(src[si-offset:])
 			if x == 0 {
 				si += 8
@@ -184,7 +195,8 @@
 		hashTable[h] = si - 2
 	}
 
-	if anchor == 0 {
+lastLiterals:
+	if isNotCompressible && anchor == 0 {
 		// Incompressible.
 		return 0, nil
 	}
@@ -205,7 +217,7 @@
 	di++
 
 	// Write the last literals.
-	if di >= anchor {
+	if isNotCompressible && di >= anchor {
 		// Incompressible.
 		return 0, nil
 	}
@@ -213,6 +225,13 @@
 	return di, nil
 }
 
+// Pool of hash tables for CompressBlock.
+var htPool = sync.Pool{
+	New: func() interface{} {
+		return new([htSize]int)
+	},
+}
+
 // blockHash hashes 4 bytes into a value < winSize.
 func blockHashHC(x uint32) uint32 {
 	const hasher uint32 = 2654435761 // Knuth multiplicative hash.
@@ -224,22 +243,24 @@
 //
 // CompressBlockHC compression ratio is better than CompressBlock but it is also slower.
 //
-// The size of the compressed data is returned. If it is 0 and no error, then the data is not compressible.
+// The size of the compressed data is returned.
+//
+// If the destination buffer size is lower than CompressBlockBound and
+// the compressed size is 0 and no error, then the data is incompressible.
 //
 // An error is returned if the destination buffer is too small.
-func CompressBlockHC(src, dst []byte, depth int) (di int, err error) {
+func CompressBlockHC(src, dst []byte, depth int) (_ int, err error) {
 	defer recoverBlock(&err)
 
+	// Return 0, nil only if the destination buffer size is < CompressBlockBound.
+	isNotCompressible := len(dst) < CompressBlockBound(len(src))
+
 	// adaptSkipLog sets how quickly the compressor begins skipping blocks when data is incompressible.
-	// This significantly speeds up incompressible data and usually has very small impact on compresssion.
+	// This significantly speeds up incompressible data and usually has very small impact on compression.
 	// bytes to skip =  1 + (bytes since last match >> adaptSkipLog)
 	const adaptSkipLog = 7
 
-	sn, dn := len(src)-mfLimit, len(dst)
-	if sn <= 0 || dn == 0 {
-		return 0, nil
-	}
-	var si int
+	var si, di, anchor int
 
 	// hashTable: stores the last position found for a given hash
 	// chainTable: stores previous positions for a given hash
@@ -249,7 +270,11 @@
 		depth = winSize
 	}
 
-	anchor := si
+	sn := len(src) - mfLimit
+	if sn <= 0 {
+		goto lastLiterals
+	}
+
 	for si < sn {
 		// Hash the next 4 bytes (sequence).
 		match := binary.LittleEndian.Uint32(src[si:])
@@ -356,12 +381,13 @@
 		}
 	}
 
-	if anchor == 0 {
+	if isNotCompressible && anchor == 0 {
 		// Incompressible.
 		return 0, nil
 	}
 
 	// Last literals.
+lastLiterals:
 	lLen := len(src) - anchor
 	if lLen < 0xF {
 		dst[di] = byte(lLen << 4)
@@ -378,7 +404,7 @@
 	di++
 
 	// Write the last literals.
-	if di >= anchor {
+	if isNotCompressible && di >= anchor {
 		// Incompressible.
 		return 0, nil
 	}
diff --git a/vendor/github.com/pierrec/lz4/lz4.go b/vendor/github.com/pierrec/lz4/lz4.go
index cdbf961..a3284bd 100644
--- a/vendor/github.com/pierrec/lz4/lz4.go
+++ b/vendor/github.com/pierrec/lz4/lz4.go
@@ -10,14 +10,20 @@
 //
 package lz4
 
+import (
+	"math/bits"
+	"sync"
+)
+
 const (
 	// Extension is the LZ4 frame file name extension
 	Extension = ".lz4"
 	// Version is the LZ4 frame format version
 	Version = 1
 
-	frameMagic     uint32 = 0x184D2204
-	frameSkipMagic uint32 = 0x184D2A50
+	frameMagic       uint32 = 0x184D2204
+	frameSkipMagic   uint32 = 0x184D2A50
+	frameMagicLegacy uint32 = 0x184C2102
 
 	// The following constants are used to setup the compression algorithm.
 	minMatch            = 4  // the minimum size of the match sequence size (4 bytes)
@@ -34,28 +40,67 @@
 	hashLog = 16
 	htSize  = 1 << hashLog
 
-	mfLimit = 8 + minMatch // The last match cannot start within the last 12 bytes.
+	mfLimit = 10 + minMatch // The last match cannot start within the last 14 bytes.
 )
 
 // map the block max size id with its value in bytes: 64Kb, 256Kb, 1Mb and 4Mb.
 const (
-	blockSize64K  = 64 << 10
-	blockSize256K = 256 << 10
-	blockSize1M   = 1 << 20
-	blockSize4M   = 4 << 20
+	blockSize64K = 1 << (16 + 2*iota)
+	blockSize256K
+	blockSize1M
+	blockSize4M
 )
 
 var (
-	bsMapID    = map[byte]int{4: blockSize64K, 5: blockSize256K, 6: blockSize1M, 7: blockSize4M}
-	bsMapValue = map[int]byte{blockSize64K: 4, blockSize256K: 5, blockSize1M: 6, blockSize4M: 7}
+	// Keep a pool of buffers for each valid block sizes.
+	bsMapValue = [...]*sync.Pool{
+		newBufferPool(2 * blockSize64K),
+		newBufferPool(2 * blockSize256K),
+		newBufferPool(2 * blockSize1M),
+		newBufferPool(2 * blockSize4M),
+	}
 )
 
+// newBufferPool returns a pool for buffers of the given size.
+func newBufferPool(size int) *sync.Pool {
+	return &sync.Pool{
+		New: func() interface{} {
+			return make([]byte, size)
+		},
+	}
+}
+
+// getBuffer returns a buffer to its pool.
+func getBuffer(size int) []byte {
+	idx := blockSizeValueToIndex(size) - 4
+	return bsMapValue[idx].Get().([]byte)
+}
+
+// putBuffer returns a buffer to its pool.
+func putBuffer(size int, buf []byte) {
+	if cap(buf) > 0 {
+		idx := blockSizeValueToIndex(size) - 4
+		bsMapValue[idx].Put(buf[:cap(buf)])
+	}
+}
+func blockSizeIndexToValue(i byte) int {
+	return 1 << (16 + 2*uint(i))
+}
+func isValidBlockSize(size int) bool {
+	const blockSizeMask = blockSize64K | blockSize256K | blockSize1M | blockSize4M
+
+	return size&blockSizeMask > 0 && bits.OnesCount(uint(size)) == 1
+}
+func blockSizeValueToIndex(size int) byte {
+	return 4 + byte(bits.TrailingZeros(uint(size)>>16)/2)
+}
+
 // Header describes the various flags that can be set on a Writer or obtained from a Reader.
 // The default values match those of the LZ4 frame format definition
 // (http://fastcompression.blogspot.com/2013/04/lz4-streaming-format-final.html).
 //
 // NB. in a Reader, in case of concatenated frames, the Header values may change between Read() calls.
-// It is the caller responsibility to check them if necessary.
+// It is the caller's responsibility to check them if necessary.
 type Header struct {
 	BlockChecksum    bool   // Compressed blocks checksum flag.
 	NoChecksum       bool   // Frame checksum flag.
@@ -64,3 +109,8 @@
 	CompressionLevel int    // Compression level (higher is better, use 0 for fastest compression).
 	done             bool   // Header processed flag (Read or Write and checked).
 }
+
+// Reset reset internal status
+func (h *Header) Reset() {
+	h.done = false
+}
diff --git a/vendor/github.com/pierrec/lz4/reader.go b/vendor/github.com/pierrec/lz4/reader.go
index 126b792..87dd72b 100644
--- a/vendor/github.com/pierrec/lz4/reader.go
+++ b/vendor/github.com/pierrec/lz4/reader.go
@@ -88,10 +88,10 @@
 	z.NoChecksum = b>>2&1 == 0
 
 	bmsID := buf[1] >> 4 & 0x7
-	bSize, ok := bsMapID[bmsID]
-	if !ok {
+	if bmsID < 4 || bmsID > 7 {
 		return fmt.Errorf("lz4: invalid block max size ID: %d", bmsID)
 	}
+	bSize := blockSizeIndexToValue(bmsID - 4)
 	z.BlockMaxSize = bSize
 
 	// Allocate the compressed/uncompressed buffers.
diff --git a/vendor/github.com/pierrec/lz4/reader_legacy.go b/vendor/github.com/pierrec/lz4/reader_legacy.go
new file mode 100644
index 0000000..1670a77
--- /dev/null
+++ b/vendor/github.com/pierrec/lz4/reader_legacy.go
@@ -0,0 +1,207 @@
+package lz4
+
+import (
+	"encoding/binary"
+	"fmt"
+	"io"
+)
+
+// ReaderLegacy implements the LZ4Demo frame decoder.
+// The Header is set after the first call to Read().
+type ReaderLegacy struct {
+	Header
+	// Handler called when a block has been successfully read.
+	// It provides the number of bytes read.
+	OnBlockDone func(size int)
+
+	lastBlock bool
+	buf       [8]byte   // Scrap buffer.
+	pos       int64     // Current position in src.
+	src       io.Reader // Source.
+	zdata     []byte    // Compressed data.
+	data      []byte    // Uncompressed data.
+	idx       int       // Index of unread bytes into data.
+	skip      int64     // Bytes to skip before next read.
+	dpos      int64     // Position in dest
+}
+
+// NewReaderLegacy returns a new LZ4Demo frame decoder.
+// No access to the underlying io.Reader is performed.
+func NewReaderLegacy(src io.Reader) *ReaderLegacy {
+	r := &ReaderLegacy{src: src}
+	return r
+}
+
+// readHeader checks the frame magic number and parses the frame descriptoz.
+// Skippable frames are supported even as a first frame although the LZ4
+// specifications recommends skippable frames not to be used as first frames.
+func (z *ReaderLegacy) readLegacyHeader() error {
+	z.lastBlock = false
+	magic, err := z.readUint32()
+	if err != nil {
+		z.pos += 4
+		if err == io.ErrUnexpectedEOF {
+			return io.EOF
+		}
+		return err
+	}
+	if magic != frameMagicLegacy {
+		return ErrInvalid
+	}
+	z.pos += 4
+
+	// Legacy has fixed 8MB blocksizes
+	// https://github.com/lz4/lz4/blob/dev/doc/lz4_Frame_format.md#legacy-frame
+	bSize := blockSize4M * 2
+
+	// Allocate the compressed/uncompressed buffers.
+	// The compressed buffer cannot exceed the uncompressed one.
+	if n := 2 * bSize; cap(z.zdata) < n {
+		z.zdata = make([]byte, n, n)
+	}
+	if debugFlag {
+		debug("header block max size size=%d", bSize)
+	}
+	z.zdata = z.zdata[:bSize]
+	z.data = z.zdata[:cap(z.zdata)][bSize:]
+	z.idx = len(z.data)
+
+	z.Header.done = true
+	if debugFlag {
+		debug("header read: %v", z.Header)
+	}
+
+	return nil
+}
+
+// Read decompresses data from the underlying source into the supplied buffer.
+//
+// Since there can be multiple streams concatenated, Header values may
+// change between calls to Read(). If that is the case, no data is actually read from
+// the underlying io.Reader, to allow for potential input buffer resizing.
+func (z *ReaderLegacy) Read(buf []byte) (int, error) {
+	if debugFlag {
+		debug("Read buf len=%d", len(buf))
+	}
+	if !z.Header.done {
+		if err := z.readLegacyHeader(); err != nil {
+			return 0, err
+		}
+		if debugFlag {
+			debug("header read OK compressed buffer %d / %d uncompressed buffer %d : %d index=%d",
+				len(z.zdata), cap(z.zdata), len(z.data), cap(z.data), z.idx)
+		}
+	}
+
+	if len(buf) == 0 {
+		return 0, nil
+	}
+
+	if z.idx == len(z.data) {
+		// No data ready for reading, process the next block.
+		if debugFlag {
+			debug("  reading block from writer %d %d", z.idx, blockSize4M*2)
+		}
+
+		// Reset uncompressed buffer
+		z.data = z.zdata[:cap(z.zdata)][len(z.zdata):]
+
+		bLen, err := z.readUint32()
+		if err != nil {
+			return 0, err
+		}
+		if debugFlag {
+			debug("   bLen %d (0x%x) offset = %d (0x%x)", bLen, bLen, z.pos, z.pos)
+		}
+		z.pos += 4
+
+		// Legacy blocks are always compressed, even when detrimental
+		if debugFlag {
+			debug("   compressed block size %d", bLen)
+		}
+
+		if int(bLen) > cap(z.data) {
+			return 0, fmt.Errorf("lz4: invalid block size: %d", bLen)
+		}
+		zdata := z.zdata[:bLen]
+		if _, err := io.ReadFull(z.src, zdata); err != nil {
+			return 0, err
+		}
+		z.pos += int64(bLen)
+
+		n, err := UncompressBlock(zdata, z.data)
+		if err != nil {
+			return 0, err
+		}
+
+		z.data = z.data[:n]
+		if z.OnBlockDone != nil {
+			z.OnBlockDone(n)
+		}
+
+		z.idx = 0
+
+		// Legacy blocks are fixed to 8MB, if we read a decompressed block smaller than this
+		// it means we've reached the end...
+		if n < blockSize4M*2 {
+			z.lastBlock = true
+		}
+	}
+
+	if z.skip > int64(len(z.data[z.idx:])) {
+		z.skip -= int64(len(z.data[z.idx:]))
+		z.dpos += int64(len(z.data[z.idx:]))
+		z.idx = len(z.data)
+		return 0, nil
+	}
+
+	z.idx += int(z.skip)
+	z.dpos += z.skip
+	z.skip = 0
+
+	n := copy(buf, z.data[z.idx:])
+	z.idx += n
+	z.dpos += int64(n)
+	if debugFlag {
+		debug("%v] copied %d bytes to input (%d:%d)", z.lastBlock, n, z.idx, len(z.data))
+	}
+	if z.lastBlock && len(z.data) == z.idx {
+		return n, io.EOF
+	}
+	return n, nil
+}
+
+// Seek implements io.Seeker, but supports seeking forward from the current
+// position only. Any other seek will return an error. Allows skipping output
+// bytes which aren't needed, which in some scenarios is faster than reading
+// and discarding them.
+// Note this may cause future calls to Read() to read 0 bytes if all of the
+// data they would have returned is skipped.
+func (z *ReaderLegacy) Seek(offset int64, whence int) (int64, error) {
+	if offset < 0 || whence != io.SeekCurrent {
+		return z.dpos + z.skip, ErrUnsupportedSeek
+	}
+	z.skip += offset
+	return z.dpos + z.skip, nil
+}
+
+// Reset discards the Reader's state and makes it equivalent to the
+// result of its original state from NewReader, but reading from r instead.
+// This permits reusing a Reader rather than allocating a new one.
+func (z *ReaderLegacy) Reset(r io.Reader) {
+	z.Header = Header{}
+	z.pos = 0
+	z.src = r
+	z.zdata = z.zdata[:0]
+	z.data = z.data[:0]
+	z.idx = 0
+}
+
+// readUint32 reads an uint32 into the supplied buffer.
+// The idea is to make use of the already allocated buffers avoiding additional allocations.
+func (z *ReaderLegacy) readUint32() (uint32, error) {
+	buf := z.buf[:4]
+	_, err := io.ReadFull(z.src, buf)
+	x := binary.LittleEndian.Uint32(buf)
+	return x, err
+}
diff --git a/vendor/github.com/pierrec/lz4/writer.go b/vendor/github.com/pierrec/lz4/writer.go
index 2cc8d95..6a60a9a 100644
--- a/vendor/github.com/pierrec/lz4/writer.go
+++ b/vendor/github.com/pierrec/lz4/writer.go
@@ -4,10 +4,18 @@
 	"encoding/binary"
 	"fmt"
 	"io"
+	"runtime"
 
 	"github.com/pierrec/lz4/internal/xxh32"
 )
 
+// zResult contains the results of compressing a block.
+type zResult struct {
+	size     uint32 // Block header
+	data     []byte // Compressed data
+	checksum uint32 // Data checksum
+}
+
 // Writer implements the LZ4 frame encoder.
 type Writer struct {
 	Header
@@ -18,10 +26,13 @@
 	buf       [19]byte      // magic number(4) + header(flags(2)+[Size(8)+DictID(4)]+checksum(1)) does not exceed 19 bytes
 	dst       io.Writer     // Destination.
 	checksum  xxh32.XXHZero // Frame checksum.
-	zdata     []byte        // Compressed data.
-	data      []byte        // Data to be compressed.
+	data      []byte        // Data to be compressed + buffer for compressed data.
 	idx       int           // Index into data.
 	hashtable [winSize]int  // Hash table used in CompressBlock().
+
+	// For concurrency.
+	c   chan chan zResult // Channel for block compression goroutines and writer goroutine.
+	err error             // Any error encountered while writing to the underlying destination.
 }
 
 // NewWriter returns a new LZ4 frame encoder.
@@ -29,30 +40,92 @@
 // The supplied Header is checked at the first Write.
 // It is ok to change it before the first Write but then not until a Reset() is performed.
 func NewWriter(dst io.Writer) *Writer {
-	return &Writer{dst: dst}
+	z := new(Writer)
+	z.Reset(dst)
+	return z
+}
+
+// WithConcurrency sets the number of concurrent go routines used for compression.
+// A negative value sets the concurrency to GOMAXPROCS.
+func (z *Writer) WithConcurrency(n int) *Writer {
+	switch {
+	case n == 0 || n == 1:
+		z.c = nil
+		return z
+	case n < 0:
+		n = runtime.GOMAXPROCS(0)
+	}
+	z.c = make(chan chan zResult, n)
+	// Writer goroutine managing concurrent block compression goroutines.
+	go func() {
+		// Process next block compression item.
+		for c := range z.c {
+			// Read the next compressed block result.
+			// Waiting here ensures that the blocks are output in the order they were sent.
+			// The incoming channel is always closed as it indicates to the caller that
+			// the block has been processed.
+			res := <-c
+			n := len(res.data)
+			if n == 0 {
+				// Notify the block compression routine that we are done with its result.
+				// This is used when a sentinel block is sent to terminate the compression.
+				close(c)
+				return
+			}
+			// Write the block.
+			if err := z.writeUint32(res.size); err != nil && z.err == nil {
+				z.err = err
+			}
+			if _, err := z.dst.Write(res.data); err != nil && z.err == nil {
+				z.err = err
+			}
+			if z.BlockChecksum {
+				if err := z.writeUint32(res.checksum); err != nil && z.err == nil {
+					z.err = err
+				}
+			}
+			if isCompressed := res.size&compressedBlockFlag == 0; isCompressed {
+				// It is now safe to release the buffer as no longer in use by any goroutine.
+				putBuffer(cap(res.data), res.data)
+			}
+			if h := z.OnBlockDone; h != nil {
+				h(n)
+			}
+			close(c)
+		}
+	}()
+	return z
+}
+
+// newBuffers instantiates new buffers which size matches the one in Header.
+// The returned buffers are for decompression and compression respectively.
+func (z *Writer) newBuffers() {
+	bSize := z.Header.BlockMaxSize
+	buf := getBuffer(bSize)
+	z.data = buf[:bSize] // Uncompressed buffer is the first half.
+}
+
+// freeBuffers puts the writer's buffers back to the pool.
+func (z *Writer) freeBuffers() {
+	// Put the buffer back into the pool, if any.
+	putBuffer(z.Header.BlockMaxSize, z.data)
+	z.data = nil
 }
 
 // writeHeader builds and writes the header (magic+header) to the underlying io.Writer.
 func (z *Writer) writeHeader() error {
 	// Default to 4Mb if BlockMaxSize is not set.
 	if z.Header.BlockMaxSize == 0 {
-		z.Header.BlockMaxSize = bsMapID[7]
+		z.Header.BlockMaxSize = blockSize4M
 	}
 	// The only option that needs to be validated.
 	bSize := z.Header.BlockMaxSize
-	bSizeID, ok := bsMapValue[bSize]
-	if !ok {
+	if !isValidBlockSize(z.Header.BlockMaxSize) {
 		return fmt.Errorf("lz4: invalid block max size: %d", bSize)
 	}
 	// Allocate the compressed/uncompressed buffers.
 	// The compressed buffer cannot exceed the uncompressed one.
-	if cap(z.zdata) < bSize {
-		// Only allocate if there is not enough capacity.
-		// Allocate both buffers at once.
-		z.zdata = make([]byte, 2*bSize)
-	}
-	z.data = z.zdata[:bSize]                 // Uncompressed buffer is the first half.
-	z.zdata = z.zdata[:cap(z.zdata)][bSize:] // Compressed buffer is the second half.
+	z.newBuffers()
 	z.idx = 0
 
 	// Size is optional.
@@ -72,7 +145,7 @@
 		flg |= 1 << 2
 	}
 	buf[4] = flg
-	buf[5] = bSizeID << 4
+	buf[5] = blockSizeValueToIndex(z.Header.BlockMaxSize) << 4
 
 	// Current buffer size: magic(4) + flags(1) + block max size (1).
 	n := 6
@@ -152,28 +225,34 @@
 // compressBlock compresses a block.
 func (z *Writer) compressBlock(data []byte) error {
 	if !z.NoChecksum {
-		z.checksum.Write(data)
+		_, _ = z.checksum.Write(data)
 	}
 
+	if z.c != nil {
+		c := make(chan zResult)
+		z.c <- c // Send now to guarantee order
+		go writerCompressBlock(c, z.Header, data)
+		return nil
+	}
+
+	zdata := z.data[z.Header.BlockMaxSize:cap(z.data)]
 	// The compressed block size cannot exceed the input's.
 	var zn int
-	var err error
 
 	if level := z.Header.CompressionLevel; level != 0 {
-		zn, err = CompressBlockHC(data, z.zdata, level)
+		zn, _ = CompressBlockHC(data, zdata, level)
 	} else {
-		zn, err = CompressBlock(data, z.zdata, z.hashtable[:])
+		zn, _ = CompressBlock(data, zdata, z.hashtable[:])
 	}
 
-	var zdata []byte
 	var bLen uint32
 	if debugFlag {
 		debug("block compression %d => %d", len(data), zn)
 	}
-	if err == nil && zn > 0 && zn < len(data) {
+	if zn > 0 && zn < len(data) {
 		// Compressible and compressed size smaller than uncompressed: ok!
 		bLen = uint32(zn)
-		zdata = z.zdata[:zn]
+		zdata = zdata[:zn]
 	} else {
 		// Uncompressed block.
 		bLen = uint32(len(data)) | compressedBlockFlag
@@ -220,13 +299,35 @@
 		return nil
 	}
 
-	if err := z.compressBlock(z.data[:z.idx]); err != nil {
-		return err
-	}
+	data := z.data[:z.idx]
 	z.idx = 0
+	if z.c == nil {
+		return z.compressBlock(data)
+	}
+	if !z.NoChecksum {
+		_, _ = z.checksum.Write(data)
+	}
+	c := make(chan zResult)
+	z.c <- c
+	writerCompressBlock(c, z.Header, data)
 	return nil
 }
 
+func (z *Writer) close() error {
+	if z.c == nil {
+		return nil
+	}
+	// Send a sentinel block (no data to compress) to terminate the writer main goroutine.
+	c := make(chan zResult)
+	z.c <- c
+	c <- zResult{}
+	// Wait for the main goroutine to complete.
+	<-c
+	// At this point the main goroutine has shut down or is about to return.
+	z.c = nil
+	return z.err
+}
+
 // Close closes the Writer, flushing any unwritten data to the underlying io.Writer, but does not close the underlying io.Writer.
 func (z *Writer) Close() error {
 	if !z.Header.done {
@@ -237,6 +338,10 @@
 	if err := z.Flush(); err != nil {
 		return err
 	}
+	if err := z.close(); err != nil {
+		return err
+	}
+	z.freeBuffers()
 
 	if debugFlag {
 		debug("writing last empty block")
@@ -258,12 +363,19 @@
 // initial state from NewWriter, but instead writing to w.
 // No access to the underlying io.Writer is performed.
 func (z *Writer) Reset(w io.Writer) {
-	z.Header = Header{}
+	n := cap(z.c)
+	_ = z.close()
+	z.freeBuffers()
+	z.Header.Reset()
 	z.dst = w
 	z.checksum.Reset()
-	z.zdata = z.zdata[:0]
-	z.data = z.data[:0]
 	z.idx = 0
+	z.err = nil
+	// reset hashtable to ensure deterministic output.
+	for i := range z.hashtable {
+		z.hashtable[i] = 0
+	}
+	z.WithConcurrency(n)
 }
 
 // writeUint32 writes a uint32 to the underlying writer.
@@ -273,3 +385,29 @@
 	_, err := z.dst.Write(buf)
 	return err
 }
+
+// writerCompressBlock compresses data into a pooled buffer and writes its result
+// out to the input channel.
+func writerCompressBlock(c chan zResult, header Header, data []byte) {
+	zdata := getBuffer(header.BlockMaxSize)
+	// The compressed block size cannot exceed the input's.
+	var zn int
+	if level := header.CompressionLevel; level != 0 {
+		zn, _ = CompressBlockHC(data, zdata, level)
+	} else {
+		var hashTable [winSize]int
+		zn, _ = CompressBlock(data, zdata, hashTable[:])
+	}
+	var res zResult
+	if zn > 0 && zn < len(data) {
+		res.size = uint32(zn)
+		res.data = zdata[:zn]
+	} else {
+		res.size = uint32(len(data)) | compressedBlockFlag
+		res.data = data
+	}
+	if header.BlockChecksum {
+		res.checksum = xxh32.ChecksumZero(res.data)
+	}
+	c <- res
+}
diff --git a/vendor/github.com/pkg/errors/.travis.yml b/vendor/github.com/pkg/errors/.travis.yml
index d4b9266..9159de0 100644
--- a/vendor/github.com/pkg/errors/.travis.yml
+++ b/vendor/github.com/pkg/errors/.travis.yml
@@ -1,15 +1,10 @@
 language: go
 go_import_path: github.com/pkg/errors
 go:
-  - 1.4.x
-  - 1.5.x
-  - 1.6.x
-  - 1.7.x
-  - 1.8.x
-  - 1.9.x
-  - 1.10.x
   - 1.11.x
+  - 1.12.x
+  - 1.13.x
   - tip
 
 script:
-  - go test -v ./...
+  - make check
diff --git a/vendor/github.com/pkg/errors/Makefile b/vendor/github.com/pkg/errors/Makefile
new file mode 100644
index 0000000..ce9d7cd
--- /dev/null
+++ b/vendor/github.com/pkg/errors/Makefile
@@ -0,0 +1,44 @@
+PKGS := github.com/pkg/errors
+SRCDIRS := $(shell go list -f '{{.Dir}}' $(PKGS))
+GO := go
+
+check: test vet gofmt misspell unconvert staticcheck ineffassign unparam
+
+test: 
+	$(GO) test $(PKGS)
+
+vet: | test
+	$(GO) vet $(PKGS)
+
+staticcheck:
+	$(GO) get honnef.co/go/tools/cmd/staticcheck
+	staticcheck -checks all $(PKGS)
+
+misspell:
+	$(GO) get github.com/client9/misspell/cmd/misspell
+	misspell \
+		-locale GB \
+		-error \
+		*.md *.go
+
+unconvert:
+	$(GO) get github.com/mdempsky/unconvert
+	unconvert -v $(PKGS)
+
+ineffassign:
+	$(GO) get github.com/gordonklaus/ineffassign
+	find $(SRCDIRS) -name '*.go' | xargs ineffassign
+
+pedantic: check errcheck
+
+unparam:
+	$(GO) get mvdan.cc/unparam
+	unparam ./...
+
+errcheck:
+	$(GO) get github.com/kisielk/errcheck
+	errcheck $(PKGS)
+
+gofmt:  
+	@echo Checking code is gofmted
+	@test -z "$(shell gofmt -s -l -d -e $(SRCDIRS) | tee /dev/stderr)"
diff --git a/vendor/github.com/pkg/errors/README.md b/vendor/github.com/pkg/errors/README.md
index 6483ba2..54dfdcb 100644
--- a/vendor/github.com/pkg/errors/README.md
+++ b/vendor/github.com/pkg/errors/README.md
@@ -41,11 +41,18 @@
 
 [Read the package documentation for more information](https://godoc.org/github.com/pkg/errors).
 
+## Roadmap
+
+With the upcoming [Go2 error proposals](https://go.googlesource.com/proposal/+/master/design/go2draft.md) this package is moving into maintenance mode. The roadmap for a 1.0 release is as follows:
+
+- 0.9. Remove pre Go 1.9 and Go 1.10 support, address outstanding pull requests (if possible)
+- 1.0. Final release.
+
 ## Contributing
 
-We welcome pull requests, bug fixes and issue reports. With that said, the bar for adding new symbols to this package is intentionally set high.
+Because of the Go2 errors changes, this package is not accepting proposals for new functionality. With that said, we welcome pull requests, bug fixes and issue reports. 
 
-Before proposing a change, please discuss your change by raising an issue.
+Before sending a PR, please discuss your change by raising an issue.
 
 ## License
 
diff --git a/vendor/github.com/pkg/errors/errors.go b/vendor/github.com/pkg/errors/errors.go
index 7421f32..161aea2 100644
--- a/vendor/github.com/pkg/errors/errors.go
+++ b/vendor/github.com/pkg/errors/errors.go
@@ -82,7 +82,7 @@
 //
 //     if err, ok := err.(stackTracer); ok {
 //             for _, f := range err.StackTrace() {
-//                     fmt.Printf("%+s:%d", f)
+//                     fmt.Printf("%+s:%d\n", f, f)
 //             }
 //     }
 //
@@ -159,6 +159,9 @@
 
 func (w *withStack) Cause() error { return w.error }
 
+// Unwrap provides compatibility for Go 1.13 error chains.
+func (w *withStack) Unwrap() error { return w.error }
+
 func (w *withStack) Format(s fmt.State, verb rune) {
 	switch verb {
 	case 'v':
@@ -241,6 +244,9 @@
 func (w *withMessage) Error() string { return w.msg + ": " + w.cause.Error() }
 func (w *withMessage) Cause() error  { return w.cause }
 
+// Unwrap provides compatibility for Go 1.13 error chains.
+func (w *withMessage) Unwrap() error { return w.cause }
+
 func (w *withMessage) Format(s fmt.State, verb rune) {
 	switch verb {
 	case 'v':
diff --git a/vendor/github.com/pkg/errors/go113.go b/vendor/github.com/pkg/errors/go113.go
new file mode 100644
index 0000000..be0d10d
--- /dev/null
+++ b/vendor/github.com/pkg/errors/go113.go
@@ -0,0 +1,38 @@
+// +build go1.13
+
+package errors
+
+import (
+	stderrors "errors"
+)
+
+// Is reports whether any error in err's chain matches target.
+//
+// The chain consists of err itself followed by the sequence of errors obtained by
+// repeatedly calling Unwrap.
+//
+// An error is considered to match a target if it is equal to that target or if
+// it implements a method Is(error) bool such that Is(target) returns true.
+func Is(err, target error) bool { return stderrors.Is(err, target) }
+
+// As finds the first error in err's chain that matches target, and if so, sets
+// target to that error value and returns true.
+//
+// The chain consists of err itself followed by the sequence of errors obtained by
+// repeatedly calling Unwrap.
+//
+// An error matches target if the error's concrete value is assignable to the value
+// pointed to by target, or if the error has a method As(interface{}) bool such that
+// As(target) returns true. In the latter case, the As method is responsible for
+// setting target.
+//
+// As will panic if target is not a non-nil pointer to either a type that implements
+// error, or to any interface type. As returns false if err is nil.
+func As(err error, target interface{}) bool { return stderrors.As(err, target) }
+
+// Unwrap returns the result of calling the Unwrap method on err, if err's
+// type contains an Unwrap method returning error.
+// Otherwise, Unwrap returns nil.
+func Unwrap(err error) error {
+	return stderrors.Unwrap(err)
+}
diff --git a/vendor/github.com/pkg/errors/stack.go b/vendor/github.com/pkg/errors/stack.go
index 2874a04..779a834 100644
--- a/vendor/github.com/pkg/errors/stack.go
+++ b/vendor/github.com/pkg/errors/stack.go
@@ -5,10 +5,13 @@
 	"io"
 	"path"
 	"runtime"
+	"strconv"
 	"strings"
 )
 
 // Frame represents a program counter inside a stack frame.
+// For historical reasons if Frame is interpreted as a uintptr
+// its value represents the program counter + 1.
 type Frame uintptr
 
 // pc returns the program counter for this frame;
@@ -37,6 +40,15 @@
 	return line
 }
 
+// name returns the name of this function, if known.
+func (f Frame) name() string {
+	fn := runtime.FuncForPC(f.pc())
+	if fn == nil {
+		return "unknown"
+	}
+	return fn.Name()
+}
+
 // Format formats the frame according to the fmt.Formatter interface.
 //
 //    %s    source file
@@ -54,22 +66,16 @@
 	case 's':
 		switch {
 		case s.Flag('+'):
-			pc := f.pc()
-			fn := runtime.FuncForPC(pc)
-			if fn == nil {
-				io.WriteString(s, "unknown")
-			} else {
-				file, _ := fn.FileLine(pc)
-				fmt.Fprintf(s, "%s\n\t%s", fn.Name(), file)
-			}
+			io.WriteString(s, f.name())
+			io.WriteString(s, "\n\t")
+			io.WriteString(s, f.file())
 		default:
 			io.WriteString(s, path.Base(f.file()))
 		}
 	case 'd':
-		fmt.Fprintf(s, "%d", f.line())
+		io.WriteString(s, strconv.Itoa(f.line()))
 	case 'n':
-		name := runtime.FuncForPC(f.pc()).Name()
-		io.WriteString(s, funcname(name))
+		io.WriteString(s, funcname(f.name()))
 	case 'v':
 		f.Format(s, 's')
 		io.WriteString(s, ":")
@@ -77,6 +83,16 @@
 	}
 }
 
+// MarshalText formats a stacktrace Frame as a text string. The output is the
+// same as that of fmt.Sprintf("%+v", f), but without newlines or tabs.
+func (f Frame) MarshalText() ([]byte, error) {
+	name := f.name()
+	if name == "unknown" {
+		return []byte(name), nil
+	}
+	return []byte(fmt.Sprintf("%s %s:%d", name, f.file(), f.line())), nil
+}
+
 // StackTrace is stack of Frames from innermost (newest) to outermost (oldest).
 type StackTrace []Frame
 
@@ -94,18 +110,32 @@
 		switch {
 		case s.Flag('+'):
 			for _, f := range st {
-				fmt.Fprintf(s, "\n%+v", f)
+				io.WriteString(s, "\n")
+				f.Format(s, verb)
 			}
 		case s.Flag('#'):
 			fmt.Fprintf(s, "%#v", []Frame(st))
 		default:
-			fmt.Fprintf(s, "%v", []Frame(st))
+			st.formatSlice(s, verb)
 		}
 	case 's':
-		fmt.Fprintf(s, "%s", []Frame(st))
+		st.formatSlice(s, verb)
 	}
 }
 
+// formatSlice will format this StackTrace into the given buffer as a slice of
+// Frame, only valid when called with '%s' or '%v'.
+func (st StackTrace) formatSlice(s fmt.State, verb rune) {
+	io.WriteString(s, "[")
+	for i, f := range st {
+		if i > 0 {
+			io.WriteString(s, " ")
+		}
+		f.Format(s, verb)
+	}
+	io.WriteString(s, "]")
+}
+
 // stack represents a stack of program counters.
 type stack []uintptr
 
diff --git a/vendor/github.com/rcrowley/go-metrics/.travis.yml b/vendor/github.com/rcrowley/go-metrics/.travis.yml
index aead076..ce9afea 100644
--- a/vendor/github.com/rcrowley/go-metrics/.travis.yml
+++ b/vendor/github.com/rcrowley/go-metrics/.travis.yml
@@ -11,6 +11,9 @@
     - "1.10"
     - "1.11"
     - "1.12"
+    - "1.13"
+    - "1.14"
+    - "1.15"
 
 script:
     - ./validate.sh
diff --git a/vendor/github.com/stretchr/testify/assert/assertion_compare.go b/vendor/github.com/stretchr/testify/assert/assertion_compare.go
index dc20039..41649d2 100644
--- a/vendor/github.com/stretchr/testify/assert/assertion_compare.go
+++ b/vendor/github.com/stretchr/testify/assert/assertion_compare.go
@@ -13,12 +13,42 @@
 	compareGreater
 )
 
+var (
+	intType   = reflect.TypeOf(int(1))
+	int8Type  = reflect.TypeOf(int8(1))
+	int16Type = reflect.TypeOf(int16(1))
+	int32Type = reflect.TypeOf(int32(1))
+	int64Type = reflect.TypeOf(int64(1))
+
+	uintType   = reflect.TypeOf(uint(1))
+	uint8Type  = reflect.TypeOf(uint8(1))
+	uint16Type = reflect.TypeOf(uint16(1))
+	uint32Type = reflect.TypeOf(uint32(1))
+	uint64Type = reflect.TypeOf(uint64(1))
+
+	float32Type = reflect.TypeOf(float32(1))
+	float64Type = reflect.TypeOf(float64(1))
+
+	stringType = reflect.TypeOf("")
+)
+
 func compare(obj1, obj2 interface{}, kind reflect.Kind) (CompareType, bool) {
+	obj1Value := reflect.ValueOf(obj1)
+	obj2Value := reflect.ValueOf(obj2)
+
+	// throughout this switch we try and avoid calling .Convert() if possible,
+	// as this has a pretty big performance impact
 	switch kind {
 	case reflect.Int:
 		{
-			intobj1 := obj1.(int)
-			intobj2 := obj2.(int)
+			intobj1, ok := obj1.(int)
+			if !ok {
+				intobj1 = obj1Value.Convert(intType).Interface().(int)
+			}
+			intobj2, ok := obj2.(int)
+			if !ok {
+				intobj2 = obj2Value.Convert(intType).Interface().(int)
+			}
 			if intobj1 > intobj2 {
 				return compareGreater, true
 			}
@@ -31,8 +61,14 @@
 		}
 	case reflect.Int8:
 		{
-			int8obj1 := obj1.(int8)
-			int8obj2 := obj2.(int8)
+			int8obj1, ok := obj1.(int8)
+			if !ok {
+				int8obj1 = obj1Value.Convert(int8Type).Interface().(int8)
+			}
+			int8obj2, ok := obj2.(int8)
+			if !ok {
+				int8obj2 = obj2Value.Convert(int8Type).Interface().(int8)
+			}
 			if int8obj1 > int8obj2 {
 				return compareGreater, true
 			}
@@ -45,8 +81,14 @@
 		}
 	case reflect.Int16:
 		{
-			int16obj1 := obj1.(int16)
-			int16obj2 := obj2.(int16)
+			int16obj1, ok := obj1.(int16)
+			if !ok {
+				int16obj1 = obj1Value.Convert(int16Type).Interface().(int16)
+			}
+			int16obj2, ok := obj2.(int16)
+			if !ok {
+				int16obj2 = obj2Value.Convert(int16Type).Interface().(int16)
+			}
 			if int16obj1 > int16obj2 {
 				return compareGreater, true
 			}
@@ -59,8 +101,14 @@
 		}
 	case reflect.Int32:
 		{
-			int32obj1 := obj1.(int32)
-			int32obj2 := obj2.(int32)
+			int32obj1, ok := obj1.(int32)
+			if !ok {
+				int32obj1 = obj1Value.Convert(int32Type).Interface().(int32)
+			}
+			int32obj2, ok := obj2.(int32)
+			if !ok {
+				int32obj2 = obj2Value.Convert(int32Type).Interface().(int32)
+			}
 			if int32obj1 > int32obj2 {
 				return compareGreater, true
 			}
@@ -73,8 +121,14 @@
 		}
 	case reflect.Int64:
 		{
-			int64obj1 := obj1.(int64)
-			int64obj2 := obj2.(int64)
+			int64obj1, ok := obj1.(int64)
+			if !ok {
+				int64obj1 = obj1Value.Convert(int64Type).Interface().(int64)
+			}
+			int64obj2, ok := obj2.(int64)
+			if !ok {
+				int64obj2 = obj2Value.Convert(int64Type).Interface().(int64)
+			}
 			if int64obj1 > int64obj2 {
 				return compareGreater, true
 			}
@@ -87,8 +141,14 @@
 		}
 	case reflect.Uint:
 		{
-			uintobj1 := obj1.(uint)
-			uintobj2 := obj2.(uint)
+			uintobj1, ok := obj1.(uint)
+			if !ok {
+				uintobj1 = obj1Value.Convert(uintType).Interface().(uint)
+			}
+			uintobj2, ok := obj2.(uint)
+			if !ok {
+				uintobj2 = obj2Value.Convert(uintType).Interface().(uint)
+			}
 			if uintobj1 > uintobj2 {
 				return compareGreater, true
 			}
@@ -101,8 +161,14 @@
 		}
 	case reflect.Uint8:
 		{
-			uint8obj1 := obj1.(uint8)
-			uint8obj2 := obj2.(uint8)
+			uint8obj1, ok := obj1.(uint8)
+			if !ok {
+				uint8obj1 = obj1Value.Convert(uint8Type).Interface().(uint8)
+			}
+			uint8obj2, ok := obj2.(uint8)
+			if !ok {
+				uint8obj2 = obj2Value.Convert(uint8Type).Interface().(uint8)
+			}
 			if uint8obj1 > uint8obj2 {
 				return compareGreater, true
 			}
@@ -115,8 +181,14 @@
 		}
 	case reflect.Uint16:
 		{
-			uint16obj1 := obj1.(uint16)
-			uint16obj2 := obj2.(uint16)
+			uint16obj1, ok := obj1.(uint16)
+			if !ok {
+				uint16obj1 = obj1Value.Convert(uint16Type).Interface().(uint16)
+			}
+			uint16obj2, ok := obj2.(uint16)
+			if !ok {
+				uint16obj2 = obj2Value.Convert(uint16Type).Interface().(uint16)
+			}
 			if uint16obj1 > uint16obj2 {
 				return compareGreater, true
 			}
@@ -129,8 +201,14 @@
 		}
 	case reflect.Uint32:
 		{
-			uint32obj1 := obj1.(uint32)
-			uint32obj2 := obj2.(uint32)
+			uint32obj1, ok := obj1.(uint32)
+			if !ok {
+				uint32obj1 = obj1Value.Convert(uint32Type).Interface().(uint32)
+			}
+			uint32obj2, ok := obj2.(uint32)
+			if !ok {
+				uint32obj2 = obj2Value.Convert(uint32Type).Interface().(uint32)
+			}
 			if uint32obj1 > uint32obj2 {
 				return compareGreater, true
 			}
@@ -143,8 +221,14 @@
 		}
 	case reflect.Uint64:
 		{
-			uint64obj1 := obj1.(uint64)
-			uint64obj2 := obj2.(uint64)
+			uint64obj1, ok := obj1.(uint64)
+			if !ok {
+				uint64obj1 = obj1Value.Convert(uint64Type).Interface().(uint64)
+			}
+			uint64obj2, ok := obj2.(uint64)
+			if !ok {
+				uint64obj2 = obj2Value.Convert(uint64Type).Interface().(uint64)
+			}
 			if uint64obj1 > uint64obj2 {
 				return compareGreater, true
 			}
@@ -157,8 +241,14 @@
 		}
 	case reflect.Float32:
 		{
-			float32obj1 := obj1.(float32)
-			float32obj2 := obj2.(float32)
+			float32obj1, ok := obj1.(float32)
+			if !ok {
+				float32obj1 = obj1Value.Convert(float32Type).Interface().(float32)
+			}
+			float32obj2, ok := obj2.(float32)
+			if !ok {
+				float32obj2 = obj2Value.Convert(float32Type).Interface().(float32)
+			}
 			if float32obj1 > float32obj2 {
 				return compareGreater, true
 			}
@@ -171,8 +261,14 @@
 		}
 	case reflect.Float64:
 		{
-			float64obj1 := obj1.(float64)
-			float64obj2 := obj2.(float64)
+			float64obj1, ok := obj1.(float64)
+			if !ok {
+				float64obj1 = obj1Value.Convert(float64Type).Interface().(float64)
+			}
+			float64obj2, ok := obj2.(float64)
+			if !ok {
+				float64obj2 = obj2Value.Convert(float64Type).Interface().(float64)
+			}
 			if float64obj1 > float64obj2 {
 				return compareGreater, true
 			}
@@ -185,8 +281,14 @@
 		}
 	case reflect.String:
 		{
-			stringobj1 := obj1.(string)
-			stringobj2 := obj2.(string)
+			stringobj1, ok := obj1.(string)
+			if !ok {
+				stringobj1 = obj1Value.Convert(stringType).Interface().(string)
+			}
+			stringobj2, ok := obj2.(string)
+			if !ok {
+				stringobj2 = obj2Value.Convert(stringType).Interface().(string)
+			}
 			if stringobj1 > stringobj2 {
 				return compareGreater, true
 			}
@@ -240,6 +342,24 @@
 	return compareTwoValues(t, e1, e2, []CompareType{compareLess, compareEqual}, "\"%v\" is not less than or equal to \"%v\"", msgAndArgs)
 }
 
+// Positive asserts that the specified element is positive
+//
+//    assert.Positive(t, 1)
+//    assert.Positive(t, 1.23)
+func Positive(t TestingT, e interface{}, msgAndArgs ...interface{}) bool {
+	zero := reflect.Zero(reflect.TypeOf(e))
+	return compareTwoValues(t, e, zero.Interface(), []CompareType{compareGreater}, "\"%v\" is not positive", msgAndArgs)
+}
+
+// Negative asserts that the specified element is negative
+//
+//    assert.Negative(t, -1)
+//    assert.Negative(t, -1.23)
+func Negative(t TestingT, e interface{}, msgAndArgs ...interface{}) bool {
+	zero := reflect.Zero(reflect.TypeOf(e))
+	return compareTwoValues(t, e, zero.Interface(), []CompareType{compareLess}, "\"%v\" is not negative", msgAndArgs)
+}
+
 func compareTwoValues(t TestingT, e1 interface{}, e2 interface{}, allowedComparesResults []CompareType, failMessage string, msgAndArgs ...interface{}) bool {
 	if h, ok := t.(tHelper); ok {
 		h.Helper()
diff --git a/vendor/github.com/stretchr/testify/assert/assertion_format.go b/vendor/github.com/stretchr/testify/assert/assertion_format.go
index 49370eb..4dfd122 100644
--- a/vendor/github.com/stretchr/testify/assert/assertion_format.go
+++ b/vendor/github.com/stretchr/testify/assert/assertion_format.go
@@ -114,6 +114,24 @@
 	return Error(t, err, append([]interface{}{msg}, args...)...)
 }
 
+// ErrorAsf asserts that at least one of the errors in err's chain matches target, and if so, sets target to that error value.
+// This is a wrapper for errors.As.
+func ErrorAsf(t TestingT, err error, target interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorAs(t, err, target, append([]interface{}{msg}, args...)...)
+}
+
+// ErrorIsf asserts that at least one of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func ErrorIsf(t TestingT, err error, target error, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorIs(t, err, target, append([]interface{}{msg}, args...)...)
+}
+
 // Eventuallyf asserts that given condition will be met in waitFor time,
 // periodically checking target function each tick.
 //
@@ -321,6 +339,54 @@
 	return InEpsilonSlice(t, expected, actual, epsilon, append([]interface{}{msg}, args...)...)
 }
 
+// IsDecreasingf asserts that the collection is decreasing
+//
+//    assert.IsDecreasingf(t, []int{2, 1, 0}, "error message %s", "formatted")
+//    assert.IsDecreasingf(t, []float{2, 1}, "error message %s", "formatted")
+//    assert.IsDecreasingf(t, []string{"b", "a"}, "error message %s", "formatted")
+func IsDecreasingf(t TestingT, object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsDecreasing(t, object, append([]interface{}{msg}, args...)...)
+}
+
+// IsIncreasingf asserts that the collection is increasing
+//
+//    assert.IsIncreasingf(t, []int{1, 2, 3}, "error message %s", "formatted")
+//    assert.IsIncreasingf(t, []float{1, 2}, "error message %s", "formatted")
+//    assert.IsIncreasingf(t, []string{"a", "b"}, "error message %s", "formatted")
+func IsIncreasingf(t TestingT, object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsIncreasing(t, object, append([]interface{}{msg}, args...)...)
+}
+
+// IsNonDecreasingf asserts that the collection is not decreasing
+//
+//    assert.IsNonDecreasingf(t, []int{1, 1, 2}, "error message %s", "formatted")
+//    assert.IsNonDecreasingf(t, []float{1, 2}, "error message %s", "formatted")
+//    assert.IsNonDecreasingf(t, []string{"a", "b"}, "error message %s", "formatted")
+func IsNonDecreasingf(t TestingT, object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonDecreasing(t, object, append([]interface{}{msg}, args...)...)
+}
+
+// IsNonIncreasingf asserts that the collection is not increasing
+//
+//    assert.IsNonIncreasingf(t, []int{2, 1, 1}, "error message %s", "formatted")
+//    assert.IsNonIncreasingf(t, []float{2, 1}, "error message %s", "formatted")
+//    assert.IsNonIncreasingf(t, []string{"b", "a"}, "error message %s", "formatted")
+func IsNonIncreasingf(t TestingT, object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonIncreasing(t, object, append([]interface{}{msg}, args...)...)
+}
+
 // IsTypef asserts that the specified objects are of the same type.
 func IsTypef(t TestingT, expectedType interface{}, object interface{}, msg string, args ...interface{}) bool {
 	if h, ok := t.(tHelper); ok {
@@ -375,6 +441,17 @@
 	return LessOrEqual(t, e1, e2, append([]interface{}{msg}, args...)...)
 }
 
+// Negativef asserts that the specified element is negative
+//
+//    assert.Negativef(t, -1, "error message %s", "formatted")
+//    assert.Negativef(t, -1.23, "error message %s", "formatted")
+func Negativef(t TestingT, e interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return Negative(t, e, append([]interface{}{msg}, args...)...)
+}
+
 // Neverf asserts that the given condition doesn't satisfy in waitFor time,
 // periodically checking the target function each tick.
 //
@@ -476,6 +553,15 @@
 	return NotEqualValues(t, expected, actual, append([]interface{}{msg}, args...)...)
 }
 
+// NotErrorIsf asserts that at none of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func NotErrorIsf(t TestingT, err error, target error, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return NotErrorIs(t, err, target, append([]interface{}{msg}, args...)...)
+}
+
 // NotNilf asserts that the specified object is not nil.
 //
 //    assert.NotNilf(t, err, "error message %s", "formatted")
@@ -572,6 +658,17 @@
 	return PanicsWithValue(t, expected, f, append([]interface{}{msg}, args...)...)
 }
 
+// Positivef asserts that the specified element is positive
+//
+//    assert.Positivef(t, 1, "error message %s", "formatted")
+//    assert.Positivef(t, 1.23, "error message %s", "formatted")
+func Positivef(t TestingT, e interface{}, msg string, args ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	return Positive(t, e, append([]interface{}{msg}, args...)...)
+}
+
 // Regexpf asserts that a specified regexp matches a string.
 //
 //  assert.Regexpf(t, regexp.MustCompile("start"), "it's starting", "error message %s", "formatted")
diff --git a/vendor/github.com/stretchr/testify/assert/assertion_forward.go b/vendor/github.com/stretchr/testify/assert/assertion_forward.go
index 9db8894..25337a6 100644
--- a/vendor/github.com/stretchr/testify/assert/assertion_forward.go
+++ b/vendor/github.com/stretchr/testify/assert/assertion_forward.go
@@ -204,6 +204,42 @@
 	return Error(a.t, err, msgAndArgs...)
 }
 
+// ErrorAs asserts that at least one of the errors in err's chain matches target, and if so, sets target to that error value.
+// This is a wrapper for errors.As.
+func (a *Assertions) ErrorAs(err error, target interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorAs(a.t, err, target, msgAndArgs...)
+}
+
+// ErrorAsf asserts that at least one of the errors in err's chain matches target, and if so, sets target to that error value.
+// This is a wrapper for errors.As.
+func (a *Assertions) ErrorAsf(err error, target interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorAsf(a.t, err, target, msg, args...)
+}
+
+// ErrorIs asserts that at least one of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func (a *Assertions) ErrorIs(err error, target error, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorIs(a.t, err, target, msgAndArgs...)
+}
+
+// ErrorIsf asserts that at least one of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func (a *Assertions) ErrorIsf(err error, target error, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return ErrorIsf(a.t, err, target, msg, args...)
+}
+
 // Errorf asserts that a function returned an error (i.e. not `nil`).
 //
 //   actualObj, err := SomeFunction()
@@ -631,6 +667,102 @@
 	return InEpsilonf(a.t, expected, actual, epsilon, msg, args...)
 }
 
+// IsDecreasing asserts that the collection is decreasing
+//
+//    a.IsDecreasing([]int{2, 1, 0})
+//    a.IsDecreasing([]float{2, 1})
+//    a.IsDecreasing([]string{"b", "a"})
+func (a *Assertions) IsDecreasing(object interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsDecreasing(a.t, object, msgAndArgs...)
+}
+
+// IsDecreasingf asserts that the collection is decreasing
+//
+//    a.IsDecreasingf([]int{2, 1, 0}, "error message %s", "formatted")
+//    a.IsDecreasingf([]float{2, 1}, "error message %s", "formatted")
+//    a.IsDecreasingf([]string{"b", "a"}, "error message %s", "formatted")
+func (a *Assertions) IsDecreasingf(object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsDecreasingf(a.t, object, msg, args...)
+}
+
+// IsIncreasing asserts that the collection is increasing
+//
+//    a.IsIncreasing([]int{1, 2, 3})
+//    a.IsIncreasing([]float{1, 2})
+//    a.IsIncreasing([]string{"a", "b"})
+func (a *Assertions) IsIncreasing(object interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsIncreasing(a.t, object, msgAndArgs...)
+}
+
+// IsIncreasingf asserts that the collection is increasing
+//
+//    a.IsIncreasingf([]int{1, 2, 3}, "error message %s", "formatted")
+//    a.IsIncreasingf([]float{1, 2}, "error message %s", "formatted")
+//    a.IsIncreasingf([]string{"a", "b"}, "error message %s", "formatted")
+func (a *Assertions) IsIncreasingf(object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsIncreasingf(a.t, object, msg, args...)
+}
+
+// IsNonDecreasing asserts that the collection is not decreasing
+//
+//    a.IsNonDecreasing([]int{1, 1, 2})
+//    a.IsNonDecreasing([]float{1, 2})
+//    a.IsNonDecreasing([]string{"a", "b"})
+func (a *Assertions) IsNonDecreasing(object interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonDecreasing(a.t, object, msgAndArgs...)
+}
+
+// IsNonDecreasingf asserts that the collection is not decreasing
+//
+//    a.IsNonDecreasingf([]int{1, 1, 2}, "error message %s", "formatted")
+//    a.IsNonDecreasingf([]float{1, 2}, "error message %s", "formatted")
+//    a.IsNonDecreasingf([]string{"a", "b"}, "error message %s", "formatted")
+func (a *Assertions) IsNonDecreasingf(object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonDecreasingf(a.t, object, msg, args...)
+}
+
+// IsNonIncreasing asserts that the collection is not increasing
+//
+//    a.IsNonIncreasing([]int{2, 1, 1})
+//    a.IsNonIncreasing([]float{2, 1})
+//    a.IsNonIncreasing([]string{"b", "a"})
+func (a *Assertions) IsNonIncreasing(object interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonIncreasing(a.t, object, msgAndArgs...)
+}
+
+// IsNonIncreasingf asserts that the collection is not increasing
+//
+//    a.IsNonIncreasingf([]int{2, 1, 1}, "error message %s", "formatted")
+//    a.IsNonIncreasingf([]float{2, 1}, "error message %s", "formatted")
+//    a.IsNonIncreasingf([]string{"b", "a"}, "error message %s", "formatted")
+func (a *Assertions) IsNonIncreasingf(object interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return IsNonIncreasingf(a.t, object, msg, args...)
+}
+
 // IsType asserts that the specified objects are of the same type.
 func (a *Assertions) IsType(expectedType interface{}, object interface{}, msgAndArgs ...interface{}) bool {
 	if h, ok := a.t.(tHelper); ok {
@@ -739,6 +871,28 @@
 	return Lessf(a.t, e1, e2, msg, args...)
 }
 
+// Negative asserts that the specified element is negative
+//
+//    a.Negative(-1)
+//    a.Negative(-1.23)
+func (a *Assertions) Negative(e interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return Negative(a.t, e, msgAndArgs...)
+}
+
+// Negativef asserts that the specified element is negative
+//
+//    a.Negativef(-1, "error message %s", "formatted")
+//    a.Negativef(-1.23, "error message %s", "formatted")
+func (a *Assertions) Negativef(e interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return Negativef(a.t, e, msg, args...)
+}
+
 // Never asserts that the given condition doesn't satisfy in waitFor time,
 // periodically checking the target function each tick.
 //
@@ -941,6 +1095,24 @@
 	return NotEqualf(a.t, expected, actual, msg, args...)
 }
 
+// NotErrorIs asserts that at none of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func (a *Assertions) NotErrorIs(err error, target error, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return NotErrorIs(a.t, err, target, msgAndArgs...)
+}
+
+// NotErrorIsf asserts that at none of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func (a *Assertions) NotErrorIsf(err error, target error, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return NotErrorIsf(a.t, err, target, msg, args...)
+}
+
 // NotNil asserts that the specified object is not nil.
 //
 //    a.NotNil(err)
@@ -1133,6 +1305,28 @@
 	return Panicsf(a.t, f, msg, args...)
 }
 
+// Positive asserts that the specified element is positive
+//
+//    a.Positive(1)
+//    a.Positive(1.23)
+func (a *Assertions) Positive(e interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return Positive(a.t, e, msgAndArgs...)
+}
+
+// Positivef asserts that the specified element is positive
+//
+//    a.Positivef(1, "error message %s", "formatted")
+//    a.Positivef(1.23, "error message %s", "formatted")
+func (a *Assertions) Positivef(e interface{}, msg string, args ...interface{}) bool {
+	if h, ok := a.t.(tHelper); ok {
+		h.Helper()
+	}
+	return Positivef(a.t, e, msg, args...)
+}
+
 // Regexp asserts that a specified regexp matches a string.
 //
 //  a.Regexp(regexp.MustCompile("start"), "it's starting")
diff --git a/vendor/github.com/stretchr/testify/assert/assertion_order.go b/vendor/github.com/stretchr/testify/assert/assertion_order.go
new file mode 100644
index 0000000..1c3b471
--- /dev/null
+++ b/vendor/github.com/stretchr/testify/assert/assertion_order.go
@@ -0,0 +1,81 @@
+package assert
+
+import (
+	"fmt"
+	"reflect"
+)
+
+// isOrdered checks that collection contains orderable elements.
+func isOrdered(t TestingT, object interface{}, allowedComparesResults []CompareType, failMessage string, msgAndArgs ...interface{}) bool {
+	objKind := reflect.TypeOf(object).Kind()
+	if objKind != reflect.Slice && objKind != reflect.Array {
+		return false
+	}
+
+	objValue := reflect.ValueOf(object)
+	objLen := objValue.Len()
+
+	if objLen <= 1 {
+		return true
+	}
+
+	value := objValue.Index(0)
+	valueInterface := value.Interface()
+	firstValueKind := value.Kind()
+
+	for i := 1; i < objLen; i++ {
+		prevValue := value
+		prevValueInterface := valueInterface
+
+		value = objValue.Index(i)
+		valueInterface = value.Interface()
+
+		compareResult, isComparable := compare(prevValueInterface, valueInterface, firstValueKind)
+
+		if !isComparable {
+			return Fail(t, fmt.Sprintf("Can not compare type \"%s\" and \"%s\"", reflect.TypeOf(value), reflect.TypeOf(prevValue)), msgAndArgs...)
+		}
+
+		if !containsValue(allowedComparesResults, compareResult) {
+			return Fail(t, fmt.Sprintf(failMessage, prevValue, value), msgAndArgs...)
+		}
+	}
+
+	return true
+}
+
+// IsIncreasing asserts that the collection is increasing
+//
+//    assert.IsIncreasing(t, []int{1, 2, 3})
+//    assert.IsIncreasing(t, []float{1, 2})
+//    assert.IsIncreasing(t, []string{"a", "b"})
+func IsIncreasing(t TestingT, object interface{}, msgAndArgs ...interface{}) bool {
+	return isOrdered(t, object, []CompareType{compareLess}, "\"%v\" is not less than \"%v\"", msgAndArgs)
+}
+
+// IsNonIncreasing asserts that the collection is not increasing
+//
+//    assert.IsNonIncreasing(t, []int{2, 1, 1})
+//    assert.IsNonIncreasing(t, []float{2, 1})
+//    assert.IsNonIncreasing(t, []string{"b", "a"})
+func IsNonIncreasing(t TestingT, object interface{}, msgAndArgs ...interface{}) bool {
+	return isOrdered(t, object, []CompareType{compareEqual, compareGreater}, "\"%v\" is not greater than or equal to \"%v\"", msgAndArgs)
+}
+
+// IsDecreasing asserts that the collection is decreasing
+//
+//    assert.IsDecreasing(t, []int{2, 1, 0})
+//    assert.IsDecreasing(t, []float{2, 1})
+//    assert.IsDecreasing(t, []string{"b", "a"})
+func IsDecreasing(t TestingT, object interface{}, msgAndArgs ...interface{}) bool {
+	return isOrdered(t, object, []CompareType{compareGreater}, "\"%v\" is not greater than \"%v\"", msgAndArgs)
+}
+
+// IsNonDecreasing asserts that the collection is not decreasing
+//
+//    assert.IsNonDecreasing(t, []int{1, 1, 2})
+//    assert.IsNonDecreasing(t, []float{1, 2})
+//    assert.IsNonDecreasing(t, []string{"a", "b"})
+func IsNonDecreasing(t TestingT, object interface{}, msgAndArgs ...interface{}) bool {
+	return isOrdered(t, object, []CompareType{compareLess, compareEqual}, "\"%v\" is not less than or equal to \"%v\"", msgAndArgs)
+}
diff --git a/vendor/github.com/stretchr/testify/assert/assertions.go b/vendor/github.com/stretchr/testify/assert/assertions.go
index 914a10d..bcac440 100644
--- a/vendor/github.com/stretchr/testify/assert/assertions.go
+++ b/vendor/github.com/stretchr/testify/assert/assertions.go
@@ -172,8 +172,8 @@
 	if len(name) == len(prefix) { // "Test" is ok
 		return true
 	}
-	rune, _ := utf8.DecodeRuneInString(name[len(prefix):])
-	return !unicode.IsLower(rune)
+	r, _ := utf8.DecodeRuneInString(name[len(prefix):])
+	return !unicode.IsLower(r)
 }
 
 func messageFromMsgAndArgs(msgAndArgs ...interface{}) string {
@@ -1622,6 +1622,7 @@
 	DisableCapacities:       true,
 	SortKeys:                true,
 	DisableMethods:          true,
+	MaxDepth:                10,
 }
 
 type tHelper interface {
@@ -1693,3 +1694,81 @@
 		}
 	}
 }
+
+// ErrorIs asserts that at least one of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func ErrorIs(t TestingT, err, target error, msgAndArgs ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	if errors.Is(err, target) {
+		return true
+	}
+
+	var expectedText string
+	if target != nil {
+		expectedText = target.Error()
+	}
+
+	chain := buildErrorChainString(err)
+
+	return Fail(t, fmt.Sprintf("Target error should be in err chain:\n"+
+		"expected: %q\n"+
+		"in chain: %s", expectedText, chain,
+	), msgAndArgs...)
+}
+
+// NotErrorIs asserts that at none of the errors in err's chain matches target.
+// This is a wrapper for errors.Is.
+func NotErrorIs(t TestingT, err, target error, msgAndArgs ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	if !errors.Is(err, target) {
+		return true
+	}
+
+	var expectedText string
+	if target != nil {
+		expectedText = target.Error()
+	}
+
+	chain := buildErrorChainString(err)
+
+	return Fail(t, fmt.Sprintf("Target error should not be in err chain:\n"+
+		"found: %q\n"+
+		"in chain: %s", expectedText, chain,
+	), msgAndArgs...)
+}
+
+// ErrorAs asserts that at least one of the errors in err's chain matches target, and if so, sets target to that error value.
+// This is a wrapper for errors.As.
+func ErrorAs(t TestingT, err error, target interface{}, msgAndArgs ...interface{}) bool {
+	if h, ok := t.(tHelper); ok {
+		h.Helper()
+	}
+	if errors.As(err, target) {
+		return true
+	}
+
+	chain := buildErrorChainString(err)
+
+	return Fail(t, fmt.Sprintf("Should be in error chain:\n"+
+		"expected: %q\n"+
+		"in chain: %s", target, chain,
+	), msgAndArgs...)
+}
+
+func buildErrorChainString(err error) string {
+	if err == nil {
+		return ""
+	}
+
+	e := errors.Unwrap(err)
+	chain := fmt.Sprintf("%q", err.Error())
+	for e != nil {
+		chain += fmt.Sprintf("\n\t%q", e.Error())
+		e = errors.Unwrap(e)
+	}
+	return chain
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/.travis.yml b/vendor/github.com/uber/jaeger-client-go/.travis.yml
index e81cc88..435aea1 100644
--- a/vendor/github.com/uber/jaeger-client-go/.travis.yml
+++ b/vendor/github.com/uber/jaeger-client-go/.travis.yml
@@ -7,26 +7,26 @@
 
 matrix:
   include:
-  - go: 1.13.x
-    env:
-    - TESTS=true
-    - USE_DEP=true
-    - COVERAGE=true
-  - go: 1.13.x
+  # - go: 1.15.x
+  #   env:
+  #   - TESTS=true
+  #   - USE_DEP=true
+  #   - COVERAGE=true
+  - go: 1.15.x
     env:
     - USE_DEP=true
     - CROSSDOCK=true
-  - go: 1.13.x
-    env:
-    - TESTS=true
-    - USE_DEP=false
-    - USE_GLIDE=true
+  # - go: 1.15.x
+  #   env:
+  #   - TESTS=true
+  #   - USE_DEP=false
+  #   - USE_GLIDE=true
   # test with previous version of Go
-  - go: 1.12.x
-    env:
-    - TESTS=true
-    - USE_DEP=true
-    - CI_SKIP_LINT=true
+  # - go: 1.14.x
+  #   env:
+  #   - TESTS=true
+  #   - USE_DEP=true
+  #   - CI_SKIP_LINT=true
 
 services:
   - docker
diff --git a/vendor/github.com/uber/jaeger-client-go/CHANGELOG.md b/vendor/github.com/uber/jaeger-client-go/CHANGELOG.md
index 944feb2..956790e 100644
--- a/vendor/github.com/uber/jaeger-client-go/CHANGELOG.md
+++ b/vendor/github.com/uber/jaeger-client-go/CHANGELOG.md
@@ -1,6 +1,66 @@
 Changes by Version
 ==================
 
+2.29.2 (unreleased)
+-------------------
+- Nothing yet.
+
+
+2.29.1 (2021-05-24)
+-------------------
+- Remove dependency on "testing" in "thrift" (#586) -- @yurishkuro
+
+
+2.29.0 (2021-05-20)
+-------------------
+- Update vendored thrift to 0.14.1 (#584) -- @nhatthm
+
+
+2.28.0 (2021-04-30)
+-------------------
+- HTTPSamplingStrategyFetcher: Use http client with 10 second timeout (#578) -- Joe Elliott
+
+
+2.27.0 (2021-04-19)
+-------------------
+- Don't override HTTP Reporter batch size to 1; default to 100, user can override (#571) -- R. Aidan Campbell
+
+
+2.26.0 (2021-04-16)
+-------------------
+- Delete a baggage item when value is blank (#562) -- evan.kim
+- Trim baggage key when parsing (#566) -- sicong.huang
+- feat: extend configuration to support custom randomNumber func (#555) -- NemoO_o
+- Support JAEGER_TRACEID_128BIT env var (#547) -- Yuri Shkuro
+- Additional context protections (#544) -- Joe Elliott
+- Lock RemotelyControlledSampler.sampler on callbacks (#543) -- Dima
+- Upgrade build to Go 1.15 (#539) -- Yuri Shkuro
+- Upgrade to jaeger-lib@2.3.0 to fix broken codahale/hdrhistogram dependency (#537) -- Yuri Shkuro
+- Prefix TraceID/SpanID.String() with zeroes (#533) -- Lukas Vogel
+- Upgrade to OpenTracing Go 1.2 (#525) -- Yuri Shkuro
+
+
+2.25.0 (2020-07-13)
+-------------------
+## Breaking changes
+- [feat] Periodically re-resolve UDP server address, with opt-out (#520) -- Trevor Foster
+
+  The re-resolving of UDP address is now enabled by default, to make the client more robust in Kubernetes deployments.
+  The old resolve-once behavior can be restored by setting DisableAttemptReconnecting=true in the Configuration struct,
+  or via JAEGER_REPORTER_ATTEMPT_RECONNECTING_DISABLED=true environment variable.
+
+## Bug fixes
+- Do not add invalid context to references (#521) -- Yuri Shkuro
+
+
+2.24.0 (2020-06-14)
+-------------------
+- Mention FromEnv() in the README, docs, and examples (#518) -- Martin Lercher
+- Serialize access to RemotelyControlledSampler.sampler (#515) -- Dima
+- Override reporter config only when agent host/port is set in env (#513) -- ilylia
+- Converge on JAEGER_SAMPLING_ENDPOINT env variable (#511) -- Eundoo Song
+
+
 2.23.1 (2020-04-28)
 -------------------
 - Fix regression by handling nil logger correctly ([#507](https://github.com/jaegertracing/jaeger-client-go/pull/507)) -- Prithvi Raj
diff --git a/vendor/github.com/uber/jaeger-client-go/CODEOWNERS b/vendor/github.com/uber/jaeger-client-go/CODEOWNERS
new file mode 100644
index 0000000..0572efc
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/CODEOWNERS
@@ -0,0 +1,2 @@
+
+* @jaegertracing/jaeger-maintainers
diff --git a/vendor/github.com/uber/jaeger-client-go/Gopkg.lock b/vendor/github.com/uber/jaeger-client-go/Gopkg.lock
index 2a5215a..268289b 100644
--- a/vendor/github.com/uber/jaeger-client-go/Gopkg.lock
+++ b/vendor/github.com/uber/jaeger-client-go/Gopkg.lock
@@ -2,12 +2,12 @@
 
 
 [[projects]]
-  digest = "1:9f3b30d9f8e0d7040f729b82dcbc8f0dead820a133b3147ce355fc451f32d761"
-  name = "github.com/BurntSushi/toml"
+  digest = "1:4c4c33075b704791d6a7f09dfb55c66769e8a1dc6adf87026292d274fe8ad113"
+  name = "github.com/HdrHistogram/hdrhistogram-go"
   packages = ["."]
   pruneopts = "UT"
-  revision = "3012a1dbe2e4bd1391d42b32f0577cb7bbc7f005"
-  version = "v0.3.1"
+  revision = "3a0bb77429bd3a61596f5e8a3172445844342120"
+  version = "0.9.0"
 
 [[projects]]
   digest = "1:d6afaeed1502aa28e80a4ed0981d570ad91b2579193404256ce672ed0a609e0d"
@@ -19,14 +19,6 @@
 
 [[projects]]
   branch = "master"
-  digest = "1:4c4c33075b704791d6a7f09dfb55c66769e8a1dc6adf87026292d274fe8ad113"
-  name = "github.com/codahale/hdrhistogram"
-  packages = ["."]
-  pruneopts = "UT"
-  revision = "3a0bb77429bd3a61596f5e8a3172445844342120"
-
-[[projects]]
-  branch = "master"
   digest = "1:a382acd6150713655ded76ab5fbcbc7924a7808dab4312dda5d1f23dd8ce5277"
   name = "github.com/crossdock/crossdock-go"
   packages = [
@@ -50,16 +42,22 @@
   name = "github.com/golang/mock"
   packages = ["gomock"]
   pruneopts = "UT"
-  revision = "3a35fb6e3e18b9dbfee291262260dee7372d2a92"
-  version = "v1.4.3"
+  revision = "f7b1909c82a8958747e5c87c6a5c3b2eaed8a33d"
+  version = "v1.4.4"
 
 [[projects]]
-  digest = "1:573ca21d3669500ff845bdebee890eb7fc7f0f50c59f2132f2a0c6b03d85086a"
+  digest = "1:4a32eb57407190eced21a21abee9ce4d4ab6f0bf113ca61cb1cb2d549a65c985"
   name = "github.com/golang/protobuf"
-  packages = ["proto"]
+  packages = [
+    "proto",
+    "ptypes",
+    "ptypes/any",
+    "ptypes/duration",
+    "ptypes/timestamp",
+  ]
   pruneopts = "UT"
-  revision = "6c65a5562fc06764971b7c5d05c76c75e84bdbf7"
-  version = "v1.3.2"
+  revision = "d04d7b157bb510b1e0c10132224b616ac0e26b17"
+  version = "v1.4.2"
 
 [[projects]]
   digest = "1:ff5ebae34cfbf047d505ee150de27e60570e8c394b3b8fdbb720ff6ac71985fc"
@@ -70,7 +68,7 @@
   version = "v1.0.1"
 
 [[projects]]
-  digest = "1:727b8f567a30d0739d6c26b9472b3422b351c93cf62095164c845a54b16fc18e"
+  digest = "1:fe5217d44ae8fb84f711968816fe50077cea9dfa8f44425b8e44e7e3de896d01"
   name = "github.com/opentracing/opentracing-go"
   packages = [
     ".",
@@ -79,16 +77,16 @@
     "log",
   ]
   pruneopts = "UT"
-  revision = "659c90643e714681897ec2521c60567dd21da733"
-  version = "v1.1.0"
+  revision = "d34af3eaa63c4d08ab54863a4bdd0daa45212e12"
+  version = "v1.2.0"
 
 [[projects]]
-  digest = "1:cf31692c14422fa27c83a05292eb5cbe0fb2775972e8f1f8446a71549bd8980b"
+  digest = "1:9e1d37b58d17113ec3cb5608ac0382313c5b59470b94ed97d0976e69c7022314"
   name = "github.com/pkg/errors"
   packages = ["."]
   pruneopts = "UT"
-  revision = "ba968bfe8b2f7e042a574c888954fccecfa385b4"
-  version = "v0.8.1"
+  revision = "614d223910a179a466c1767a985424175c39b465"
+  version = "v0.9.1"
 
 [[projects]]
   digest = "1:0028cb19b2e4c3112225cd871870f2d9cf49b9b4276531f03438a88e94be86fe"
@@ -110,15 +108,15 @@
   version = "v1.1.0"
 
 [[projects]]
-  branch = "master"
-  digest = "1:2d5cd61daa5565187e1d96bae64dbbc6080dacf741448e9629c64fd93203b0d4"
+  digest = "1:0db23933b8052702d980a3f029149b3f175f7c0eea0cff85b175017d0f2722c0"
   name = "github.com/prometheus/client_model"
   packages = ["go"]
   pruneopts = "UT"
-  revision = "14fe0d1b01d4d5fc031dd4bec1823bd3ebbe8016"
+  revision = "7bc5445566f0fe75b15de23e6b93886e982d7bf9"
+  version = "v0.2.0"
 
 [[projects]]
-  digest = "1:f119e3205d3a1f0f19dbd7038eb37528e2c6f0933269dc344e305951fb87d632"
+  digest = "1:4407525bde4e6ad9c1f60113d38cbb255d769e0ea506c8cf877db39da7753b3a"
   name = "github.com/prometheus/common"
   packages = [
     "expfmt",
@@ -126,11 +124,11 @@
     "model",
   ]
   pruneopts = "UT"
-  revision = "287d3e634a1e550c9e463dd7e5a75a422c614505"
-  version = "v0.7.0"
+  revision = "317b7b125e8fddda956d0c9574e5f03f438ed5bc"
+  version = "v0.14.0"
 
 [[projects]]
-  digest = "1:a210815b437763623ecca8eb91e6a0bf4f2d6773c5a6c9aec0e28f19e5fd6deb"
+  digest = "1:b2268435af85ee1a0fca0e37de4225f78e2d9d8b0b66acde3a29f127634efa87"
   name = "github.com/prometheus/procfs"
   packages = [
     ".",
@@ -138,23 +136,32 @@
     "internal/util",
   ]
   pruneopts = "UT"
-  revision = "499c85531f756d1129edd26485a5f73871eeb308"
-  version = "v0.0.5"
+  revision = "9dece15c53cd5e9fbfbd72d5108adcf526a3f486"
+  version = "v0.2.0"
 
 [[projects]]
-  digest = "1:0496f0e99014b7fd0a560c539f51d0882731137b85494142f47e550e4657176a"
+  digest = "1:86ff4af7b6bb3d27c2e89b5ef8c139678acff1cad74a3c5235fc5af6b94fcc9e"
+  name = "github.com/stretchr/objx"
+  packages = ["."]
+  pruneopts = "UT"
+  revision = "35313a95ee26395aa17d366c71a2ccf788fa69b6"
+  version = "v0.3.0"
+
+[[projects]]
+  digest = "1:5201127841a78d84d0ca68a2e564c08e3882c0fb9321a75997ce87926e0d63ea"
   name = "github.com/stretchr/testify"
   packages = [
     "assert",
+    "mock",
     "require",
     "suite",
   ]
   pruneopts = "UT"
-  revision = "221dbe5ed46703ee255b1da0dec05086f5035f62"
-  version = "v1.4.0"
+  revision = "f654a9112bbeac49ca2cd45bfbe11533c4666cf8"
+  version = "v1.6.1"
 
 [[projects]]
-  digest = "1:0ec60ffd594af00ba1660bc746aa0e443d27dd4003dee55f9d08a0b4ff5431a3"
+  digest = "1:4af46f2faea30e52c96ec9ec32bb654d2729579a80d242b0acfa193ad321eb61"
   name = "github.com/uber/jaeger-lib"
   packages = [
     "metrics",
@@ -162,35 +169,27 @@
     "metrics/prometheus",
   ]
   pruneopts = "UT"
-  revision = "a87ae9d84fb038a8d79266298970720be7c80fcd"
-  version = "v2.2.0"
+  revision = "48cc1df63e6be0d63b95677f0d22beb880bce1e4"
+  version = "v2.3.0"
 
 [[projects]]
-  digest = "1:0bdcb0c740d79d400bd3f7946ac22a715c94db62b20bfd2e01cd50693aba0600"
+  digest = "1:7a3de4371d6b68c6f37a0df2c09905664d9de59026c91cbe275aae55f4fe760f"
   name = "go.uber.org/atomic"
   packages = ["."]
   pruneopts = "UT"
-  revision = "9dc4df04d0d1c39369750a9f6c32c39560672089"
-  version = "v1.5.0"
+  revision = "12f27ba2637fa0e13772a4f05fa46a5d18d53182"
+  version = "v1.7.0"
 
 [[projects]]
-  digest = "1:002ebc50f3ef475ac325e1904be931d9dcba6dc6d73b5682afce0c63436e3902"
+  digest = "1:e9eeeabfd025a5e69b9c8e2857d3517ea67e747ae913bcb0a9e1e7bafdb9c298"
   name = "go.uber.org/multierr"
   packages = ["."]
   pruneopts = "UT"
-  revision = "c3fc3d02ec864719d8e25be2d7dde1e35a36aa27"
-  version = "v1.3.0"
+  revision = "3114a8b704d2d28dbacda34a872690aaef66aeed"
+  version = "v1.6.0"
 
 [[projects]]
-  branch = "master"
-  digest = "1:3032e90a153750ea149f68bf081f97ca738f041fba45c41c80737f572ffdf2f4"
-  name = "go.uber.org/tools"
-  packages = ["update-license"]
-  pruneopts = "UT"
-  revision = "2cfd321de3ee5d5f8a5fda2521d1703478334d98"
-
-[[projects]]
-  digest = "1:98a70115729234dc73ee7bb83973cb39cb8fedf278d17df77264382bad0183ec"
+  digest = "1:0398f5f0e2e9233f25fad702f3b323241daf9f876cc869ab259238cf1bced236"
   name = "go.uber.org/zap"
   packages = [
     ".",
@@ -202,19 +201,8 @@
     "zaptest/observer",
   ]
   pruneopts = "UT"
-  revision = "a6015e13fab9b744d96085308ce4e8f11bad1996"
-  version = "v1.12.0"
-
-[[projects]]
-  branch = "master"
-  digest = "1:21d7bad9b7da270fd2d50aba8971a041bd691165c95096a2a4c68db823cbc86a"
-  name = "golang.org/x/lint"
-  packages = [
-    ".",
-    "golint",
-  ]
-  pruneopts = "UT"
-  revision = "16217165b5de779cb6a5e4fc81fa9c1166fda457"
+  revision = "404189cf44aea95b0cd9bddcb0242dd4cf88c510"
+  version = "v1.16.0"
 
 [[projects]]
   branch = "master"
@@ -225,81 +213,65 @@
     "context/ctxhttp",
   ]
   pruneopts = "UT"
-  revision = "0deb6923b6d97481cb43bc1043fe5b72a0143032"
+  revision = "328152dc79b1547da63f950cd4cdd9afd50b2774"
 
 [[projects]]
   branch = "master"
-  digest = "1:5dfb17d45415b7b8927382f53955a66f55f9d9d11557aa82f7f481d642ab247a"
+  digest = "1:1e581fa394685ef0d84008ae04cf3414390c1a700c04846853869cb4ac2fec86"
   name = "golang.org/x/sys"
-  packages = ["windows"]
-  pruneopts = "UT"
-  revision = "f43be2a4598cf3a47be9f94f0c28197ed9eae611"
-
-[[projects]]
-  branch = "master"
-  digest = "1:bae8b3bf837d9d7f601776f37f44e031d46943677beff8fb2eb9c7317d44de2f"
-  name = "golang.org/x/tools"
   packages = [
-    "go/analysis",
-    "go/analysis/passes/inspect",
-    "go/ast/astutil",
-    "go/ast/inspector",
-    "go/buildutil",
-    "go/gcexportdata",
-    "go/internal/gcimporter",
-    "go/internal/packagesdriver",
-    "go/packages",
-    "go/types/objectpath",
-    "go/types/typeutil",
-    "internal/fastwalk",
-    "internal/gopathwalk",
-    "internal/semver",
-    "internal/span",
+    "internal/unsafeheader",
+    "unix",
+    "windows",
   ]
   pruneopts = "UT"
-  revision = "8dbcdeb83d3faec5315146800b375c4962a42fc6"
+  revision = "d9f96fdee20d1e5115ee34ba4016eae6cfb66eb9"
 
 [[projects]]
-  digest = "1:59f10c1537d2199d9115d946927fe31165959a95190849c82ff11e05803528b0"
-  name = "gopkg.in/yaml.v2"
+  digest = "1:fd328c5b52e433ea3ffc891bcc4f94469a82bf478558208db2b386aad8a304a1"
+  name = "google.golang.org/protobuf"
+  packages = [
+    "encoding/prototext",
+    "encoding/protowire",
+    "internal/descfmt",
+    "internal/descopts",
+    "internal/detrand",
+    "internal/encoding/defval",
+    "internal/encoding/messageset",
+    "internal/encoding/tag",
+    "internal/encoding/text",
+    "internal/errors",
+    "internal/fieldsort",
+    "internal/filedesc",
+    "internal/filetype",
+    "internal/flags",
+    "internal/genid",
+    "internal/impl",
+    "internal/mapsort",
+    "internal/pragma",
+    "internal/set",
+    "internal/strs",
+    "internal/version",
+    "proto",
+    "reflect/protoreflect",
+    "reflect/protoregistry",
+    "runtime/protoiface",
+    "runtime/protoimpl",
+    "types/known/anypb",
+    "types/known/durationpb",
+    "types/known/timestamppb",
+  ]
+  pruneopts = "UT"
+  revision = "3f7a61f89bb6813f89d981d1870ed68da0b3c3f1"
+  version = "v1.25.0"
+
+[[projects]]
+  branch = "v3"
+  digest = "1:229cb0f6192914f518cc1241ede6d6f1f458b31debfa18bf3a5c9e4f7b01e24b"
+  name = "gopkg.in/yaml.v3"
   packages = ["."]
   pruneopts = "UT"
-  revision = "f221b8435cfb71e54062f6c6e99e9ade30b124d5"
-  version = "v2.2.4"
-
-[[projects]]
-  digest = "1:131158a88aad1f94854d0aa21a64af2802d0a470fb0f01cb33c04fafd2047111"
-  name = "honnef.co/go/tools"
-  packages = [
-    "arg",
-    "cmd/staticcheck",
-    "config",
-    "deprecated",
-    "facts",
-    "functions",
-    "go/types/typeutil",
-    "internal/cache",
-    "internal/passes/buildssa",
-    "internal/renameio",
-    "internal/sharedcheck",
-    "lint",
-    "lint/lintdsl",
-    "lint/lintutil",
-    "lint/lintutil/format",
-    "loader",
-    "printf",
-    "simple",
-    "ssa",
-    "ssautil",
-    "staticcheck",
-    "staticcheck/vrp",
-    "stylecheck",
-    "unused",
-    "version",
-  ]
-  pruneopts = "UT"
-  revision = "afd67930eec2a9ed3e9b19f684d17a062285f16a"
-  version = "2019.2.3"
+  revision = "eeeca48fe7764f320e4870d231902bf9c1be2c08"
 
 [solve-meta]
   analyzer-name = "dep"
@@ -314,6 +286,7 @@
     "github.com/pkg/errors",
     "github.com/prometheus/client_golang/prometheus",
     "github.com/stretchr/testify/assert",
+    "github.com/stretchr/testify/mock",
     "github.com/stretchr/testify/require",
     "github.com/stretchr/testify/suite",
     "github.com/uber/jaeger-lib/metrics",
diff --git a/vendor/github.com/uber/jaeger-client-go/Gopkg.toml b/vendor/github.com/uber/jaeger-client-go/Gopkg.toml
index 1fed7f8..3aa307a 100644
--- a/vendor/github.com/uber/jaeger-client-go/Gopkg.toml
+++ b/vendor/github.com/uber/jaeger-client-go/Gopkg.toml
@@ -4,7 +4,7 @@
 
 [[constraint]]
   name = "github.com/opentracing/opentracing-go"
-  version = "^1.1"
+  version = "^1.2"
 
 [[constraint]]
   name = "github.com/prometheus/client_golang"
@@ -20,7 +20,7 @@
 
 [[constraint]]
   name = "github.com/uber/jaeger-lib"
-  version = "^2.2"
+  version = "^2.3"
 
 [[constraint]]
   name = "go.uber.org/zap"
diff --git a/vendor/github.com/uber/jaeger-client-go/Makefile b/vendor/github.com/uber/jaeger-client-go/Makefile
index d5e962c..bb7463c 100644
--- a/vendor/github.com/uber/jaeger-client-go/Makefile
+++ b/vendor/github.com/uber/jaeger-client-go/Makefile
@@ -1,5 +1,6 @@
 PROJECT_ROOT=github.com/uber/jaeger-client-go
-PACKAGES := . $(shell go list ./... | awk -F/ 'NR>1 {print "./"$$4"/..."}' | grep -v -e ./thrift-gen/... -e ./thrift/... | sort -u)
+export GO111MODULE=off
+PACKAGES := . $(shell GO111MODULE=off go list ./... | awk -F/ 'NR>1 {print "./"$$4"/..."}' | grep -v -e ./thrift-gen/... -e ./thrift/... | sort -u)
 # all .go files that don't exist in hidden directories
 ALL_SRC := $(shell find . -name "*.go" | grep -v -e vendor -e thrift-gen -e ./thrift/ \
         -e ".*/\..*" \
@@ -18,8 +19,8 @@
 FMT_LOG=fmt.log
 LINT_LOG=lint.log
 
-THRIFT_VER=0.9.3
-THRIFT_IMG=thrift:$(THRIFT_VER)
+THRIFT_VER=0.14
+THRIFT_IMG=jaegertracing/thrift:$(THRIFT_VER)
 THRIFT=docker run -v "${PWD}:/data" $(THRIFT_IMG) thrift
 THRIFT_GO_ARGS=thrift_import="github.com/apache/thrift/lib/go/thrift"
 THRIFT_GEN_DIR=thrift-gen
@@ -46,22 +47,37 @@
 	./scripts/updateLicenses.sh
 
 .PHONY: lint
-lint:
+lint: vet golint lint-fmt lint-thrift-testing
+
+.PHONY: vet
+vet:
 	$(GOVET) $(PACKAGES)
+
+.PHONY: golint
+golint:
 	@cat /dev/null > $(LINT_LOG)
 	@$(foreach pkg, $(PACKAGES), $(GOLINT) $(pkg) | grep -v crossdock/thrift >> $(LINT_LOG) || true;)
 	@[ ! -s "$(LINT_LOG)" ] || (echo "Lint Failures" | cat - $(LINT_LOG) && false)
+
+.PHONY: lint-fmt
+lint-fmt:
 	@$(GOFMT) -e -s -l $(ALL_SRC) > $(FMT_LOG)
 	./scripts/updateLicenses.sh >> $(FMT_LOG)
 	@[ ! -s "$(FMT_LOG)" ] || (echo "go fmt or license check failures, run 'make fmt'" | cat - $(FMT_LOG) && false)
 
+# make sure thrift/ module does not import "testing"
+.PHONY: lint-thrift-testing
+lint-thrift-testing:
+	@cat /dev/null > $(LINT_LOG)
+	@(grep -rn '"testing"' thrift | grep -v README.md > $(LINT_LOG)) || true
+	@[ ! -s "$(LINT_LOG)" ] || (echo '"thrift" module must not import "testing", see issue #585' | cat - $(LINT_LOG) && false)
 
 .PHONY: install
 install:
 	@echo install: USE_DEP=$(USE_DEP) USE_GLIDE=$(USE_GLIDE)
 ifeq ($(USE_DEP),true)
 	dep version || make install-dep
-	dep ensure
+	dep ensure -vendor-only -v
 endif
 ifeq ($(USE_GLIDE),true)
 	glide --version || go get github.com/Masterminds/glide
@@ -131,4 +147,3 @@
 else
 	make lint
 endif
-
diff --git a/vendor/github.com/uber/jaeger-client-go/README.md b/vendor/github.com/uber/jaeger-client-go/README.md
index 7c348e7..687f578 100644
--- a/vendor/github.com/uber/jaeger-client-go/README.md
+++ b/vendor/github.com/uber/jaeger-client-go/README.md
@@ -44,28 +44,34 @@
 
 ### Environment variables
 
-The tracer can be initialized with values coming from environment variables. None of the env vars are required
-and all of them can be overridden via direct setting of the property on the configuration object.
+The tracer can be initialized with values coming from environment variables, if it is
+[built from a config](https://pkg.go.dev/github.com/uber/jaeger-client-go/config?tab=doc#Configuration.NewTracer)
+that was created via [FromEnv()](https://pkg.go.dev/github.com/uber/jaeger-client-go/config?tab=doc#FromEnv).
+None of the env vars are required and all of them can be overridden via direct setting 
+of the property on the configuration object.
 
 Property| Description
 --- | ---
-JAEGER_SERVICE_NAME | The service name
-JAEGER_AGENT_HOST | The hostname for communicating with agent via UDP
-JAEGER_AGENT_PORT | The port for communicating with agent via UDP
-JAEGER_ENDPOINT | The HTTP endpoint for sending spans directly to a collector, i.e. http://jaeger-collector:14268/api/traces
-JAEGER_USER | Username to send as part of "Basic" authentication to the collector endpoint
-JAEGER_PASSWORD | Password to send as part of "Basic" authentication to the collector endpoint
-JAEGER_REPORTER_LOG_SPANS | Whether the reporter should also log the spans
-JAEGER_REPORTER_MAX_QUEUE_SIZE | The reporter's maximum queue size
-JAEGER_REPORTER_FLUSH_INTERVAL | The reporter's flush interval, with units, e.g. "500ms" or "2s" ([valid units][timeunits])
-JAEGER_SAMPLER_TYPE | The sampler type
-JAEGER_SAMPLER_PARAM | The sampler parameter (number)
-JAEGER_SAMPLER_MANAGER_HOST_PORT | The HTTP endpoint when using the remote sampler, i.e. http://jaeger-agent:5778/sampling
-JAEGER_SAMPLER_MAX_OPERATIONS | The maximum number of operations that the sampler will keep track of
-JAEGER_SAMPLER_REFRESH_INTERVAL | How often the remotely controlled sampler will poll jaeger-agent for the appropriate sampling strategy, with units, e.g. "1m" or "30s" ([valid units][timeunits])
-JAEGER_TAGS | A comma separated list of `name = value` tracer level tags, which get added to all reported spans. The value can also refer to an environment variable using the format `${envVarName:default}`, where the `:default` is optional, and identifies a value to be used if the environment variable cannot be found
-JAEGER_DISABLED | Whether the tracer is disabled or not. If true, the default `opentracing.NoopTracer` is used.
-JAEGER_RPC_METRICS | Whether to store RPC metrics
+JAEGER_SERVICE_NAME | The service name.
+JAEGER_AGENT_HOST | The hostname for communicating with agent via UDP (default `localhost`).
+JAEGER_AGENT_PORT | The port for communicating with agent via UDP (default `6831`).
+JAEGER_ENDPOINT | The HTTP endpoint for sending spans directly to a collector, i.e. http://jaeger-collector:14268/api/traces. If specified, the agent host/port are ignored.
+JAEGER_USER | Username to send as part of "Basic" authentication to the collector endpoint.
+JAEGER_PASSWORD | Password to send as part of "Basic" authentication to the collector endpoint.
+JAEGER_REPORTER_LOG_SPANS | Whether the reporter should also log the spans" `true` or `false` (default `false`).
+JAEGER_REPORTER_MAX_QUEUE_SIZE | The reporter's maximum queue size (default `100`).
+JAEGER_REPORTER_FLUSH_INTERVAL | The reporter's flush interval, with units, e.g. `500ms` or `2s` ([valid units][timeunits]; default `1s`).
+JAEGER_REPORTER_ATTEMPT_RECONNECTING_DISABLED | When true, disables udp connection helper that periodically re-resolves the agent's hostname and reconnects if there was a change (default `false`).
+JAEGER_REPORTER_ATTEMPT_RECONNECT_INTERVAL | Controls how often the agent client re-resolves the provided hostname in order to detect address changes ([valid units][timeunits]; default `30s`).
+JAEGER_SAMPLER_TYPE | The sampler type: `remote`, `const`, `probabilistic`, `ratelimiting` (default `remote`). See also https://www.jaegertracing.io/docs/latest/sampling/.
+JAEGER_SAMPLER_PARAM | The sampler parameter (number).
+JAEGER_SAMPLER_MANAGER_HOST_PORT | (deprecated) The HTTP endpoint when using the `remote` sampler.
+JAEGER_SAMPLING_ENDPOINT | The URL for the sampling configuration server when using sampler type `remote` (default `http://127.0.0.1:5778/sampling`).
+JAEGER_SAMPLER_MAX_OPERATIONS | The maximum number of operations that the sampler will keep track of (default `2000`).
+JAEGER_SAMPLER_REFRESH_INTERVAL | How often the `remote` sampler should poll the configuration server for the appropriate sampling strategy, e.g. "1m" or "30s" ([valid units][timeunits]; default `1m`).
+JAEGER_TAGS | A comma separated list of `name=value` tracer-level tags, which get added to all reported spans. The value can also refer to an environment variable using the format `${envVarName:defaultValue}`.
+JAEGER_DISABLED | Whether the tracer is disabled or not. If `true`, the `opentracing.NoopTracer` is used (default `false`).
+JAEGER_RPC_METRICS | Whether to store RPC metrics, `true` or `false` (default `false`).
 
 By default, the client sends traces via UDP to the agent at `localhost:6831`. Use `JAEGER_AGENT_HOST` and
 `JAEGER_AGENT_PORT` to send UDP traces to a different `host:port`. If `JAEGER_ENDPOINT` is set, the client sends traces
diff --git a/vendor/github.com/uber/jaeger-client-go/RELEASE.md b/vendor/github.com/uber/jaeger-client-go/RELEASE.md
index 115e49a..12438d8 100644
--- a/vendor/github.com/uber/jaeger-client-go/RELEASE.md
+++ b/vendor/github.com/uber/jaeger-client-go/RELEASE.md
@@ -2,6 +2,7 @@
 
 1. Create a PR "Preparing for release X.Y.Z" against master branch
     * Alter CHANGELOG.md from `<placeholder_version> (unreleased)` to `<X.Y.Z> (YYYY-MM-DD)`
+    * Use `git log --pretty=format:'- %s -- %an'` as the basis for for changelog entries
     * Update `JaegerClientVersion` in constants.go to `Go-X.Y.Z`
 2. Create a release "Release X.Y.Z" on Github
     * Create Tag `vX.Y.Z`
diff --git a/vendor/github.com/uber/jaeger-client-go/config/config.go b/vendor/github.com/uber/jaeger-client-go/config/config.go
index 44e9353..c2222f1 100644
--- a/vendor/github.com/uber/jaeger-client-go/config/config.go
+++ b/vendor/github.com/uber/jaeger-client-go/config/config.go
@@ -22,6 +22,7 @@
 	"time"
 
 	"github.com/opentracing/opentracing-go"
+	"github.com/uber/jaeger-client-go/utils"
 
 	"github.com/uber/jaeger-client-go"
 	"github.com/uber/jaeger-client-go/internal/baggage/remote"
@@ -36,16 +37,22 @@
 // Configuration configures and creates Jaeger Tracer
 type Configuration struct {
 	// ServiceName specifies the service name to use on the tracer.
-	// Can be provided via environment variable named JAEGER_SERVICE_NAME
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SERVICE_NAME
 	ServiceName string `yaml:"serviceName"`
 
-	// Disabled can be provided via environment variable named JAEGER_DISABLED
+	// Disabled makes the config return opentracing.NoopTracer.
+	// Value can be provided by FromEnv() via the environment variable named JAEGER_DISABLED.
 	Disabled bool `yaml:"disabled"`
 
-	// RPCMetrics can be provided via environment variable named JAEGER_RPC_METRICS
+	// RPCMetrics enables generations of RPC metrics (requires metrics factory to be provided).
+	// Value can be provided by FromEnv() via the environment variable named JAEGER_RPC_METRICS
 	RPCMetrics bool `yaml:"rpc_metrics"`
 
-	// Tags can be provided via environment variable named JAEGER_TAGS
+	// Gen128Bit instructs the tracer to generate 128-bit wide trace IDs, compatible with W3C Trace Context.
+	// Value can be provided by FromEnv() via the environment variable named JAEGER_TRACEID_128BIT.
+	Gen128Bit bool `yaml:"traceid_128bit"`
+
+	// Tags can be provided by FromEnv() via the environment variable named JAEGER_TAGS
 	Tags []opentracing.Tag `yaml:"tags"`
 
 	Sampler             *SamplerConfig             `yaml:"sampler"`
@@ -57,8 +64,8 @@
 
 // SamplerConfig allows initializing a non-default sampler.  All fields are optional.
 type SamplerConfig struct {
-	// Type specifies the type of the sampler: const, probabilistic, rateLimiting, or remote
-	// Can be set by exporting an environment variable named JAEGER_SAMPLER_TYPE
+	// Type specifies the type of the sampler: const, probabilistic, rateLimiting, or remote.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SAMPLER_TYPE
 	Type string `yaml:"type"`
 
 	// Param is a value passed to the sampler.
@@ -69,22 +76,23 @@
 	// - for "remote" sampler, param is the same as for "probabilistic"
 	//   and indicates the initial sampling rate before the actual one
 	//   is received from the mothership.
-	// Can be set by exporting an environment variable named JAEGER_SAMPLER_PARAM
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SAMPLER_PARAM
 	Param float64 `yaml:"param"`
 
-	// SamplingServerURL is the address of jaeger-agent's HTTP sampling server
-	// Can be set by exporting an environment variable named JAEGER_SAMPLER_MANAGER_HOST_PORT
+	// SamplingServerURL is the URL of sampling manager that can provide
+	// sampling strategy to this service.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SAMPLING_ENDPOINT
 	SamplingServerURL string `yaml:"samplingServerURL"`
 
 	// SamplingRefreshInterval controls how often the remotely controlled sampler will poll
-	// jaeger-agent for the appropriate sampling strategy.
-	// Can be set by exporting an environment variable named JAEGER_SAMPLER_REFRESH_INTERVAL
+	// sampling manager for the appropriate sampling strategy.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SAMPLER_REFRESH_INTERVAL
 	SamplingRefreshInterval time.Duration `yaml:"samplingRefreshInterval"`
 
 	// MaxOperations is the maximum number of operations that the PerOperationSampler
 	// will keep track of. If an operation is not tracked, a default probabilistic
 	// sampler will be used rather than the per operation specific sampler.
-	// Can be set by exporting an environment variable named JAEGER_SAMPLER_MAX_OPERATIONS.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_SAMPLER_MAX_OPERATIONS.
 	MaxOperations int `yaml:"maxOperations"`
 
 	// Opt-in feature for applications that require late binding of span name via explicit
@@ -105,34 +113,46 @@
 	// QueueSize controls how many spans the reporter can keep in memory before it starts dropping
 	// new spans. The queue is continuously drained by a background go-routine, as fast as spans
 	// can be sent out of process.
-	// Can be set by exporting an environment variable named JAEGER_REPORTER_MAX_QUEUE_SIZE
+	// Can be provided by FromEnv() via the environment variable named JAEGER_REPORTER_MAX_QUEUE_SIZE
 	QueueSize int `yaml:"queueSize"`
 
 	// BufferFlushInterval controls how often the buffer is force-flushed, even if it's not full.
 	// It is generally not useful, as it only matters for very low traffic services.
-	// Can be set by exporting an environment variable named JAEGER_REPORTER_FLUSH_INTERVAL
+	// Can be provided by FromEnv() via the environment variable named JAEGER_REPORTER_FLUSH_INTERVAL
 	BufferFlushInterval time.Duration
 
 	// LogSpans, when true, enables LoggingReporter that runs in parallel with the main reporter
 	// and logs all submitted spans. Main Configuration.Logger must be initialized in the code
 	// for this option to have any effect.
-	// Can be set by exporting an environment variable named JAEGER_REPORTER_LOG_SPANS
+	// Can be provided by FromEnv() via the environment variable named JAEGER_REPORTER_LOG_SPANS
 	LogSpans bool `yaml:"logSpans"`
 
-	// LocalAgentHostPort instructs reporter to send spans to jaeger-agent at this address
-	// Can be set by exporting an environment variable named JAEGER_AGENT_HOST / JAEGER_AGENT_PORT
+	// LocalAgentHostPort instructs reporter to send spans to jaeger-agent at this address.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_AGENT_HOST / JAEGER_AGENT_PORT
 	LocalAgentHostPort string `yaml:"localAgentHostPort"`
 
-	// CollectorEndpoint instructs reporter to send spans to jaeger-collector at this URL
-	// Can be set by exporting an environment variable named JAEGER_ENDPOINT
+	// DisableAttemptReconnecting when true, disables udp connection helper that periodically re-resolves
+	// the agent's hostname and reconnects if there was a change. This option only
+	// applies if LocalAgentHostPort is specified.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_REPORTER_ATTEMPT_RECONNECTING_DISABLED
+	DisableAttemptReconnecting bool `yaml:"disableAttemptReconnecting"`
+
+	// AttemptReconnectInterval controls how often the agent client re-resolves the provided hostname
+	// in order to detect address changes. This option only applies if DisableAttemptReconnecting is false.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_REPORTER_ATTEMPT_RECONNECT_INTERVAL
+	AttemptReconnectInterval time.Duration
+
+	// CollectorEndpoint instructs reporter to send spans to jaeger-collector at this URL.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_ENDPOINT
 	CollectorEndpoint string `yaml:"collectorEndpoint"`
 
 	// User instructs reporter to include a user for basic http authentication when sending spans to jaeger-collector.
-	// Can be set by exporting an environment variable named JAEGER_USER
+	// Can be provided by FromEnv() via the environment variable named JAEGER_USER
 	User string `yaml:"user"`
 
 	// Password instructs reporter to include a password for basic http authentication when sending spans to
-	// jaeger-collector. Can be set by exporting an environment variable named JAEGER_PASSWORD
+	// jaeger-collector.
+	// Can be provided by FromEnv() via the environment variable named JAEGER_PASSWORD
 	Password string `yaml:"password"`
 
 	// HTTPHeaders instructs the reporter to add these headers to the http request when reporting spans.
@@ -247,13 +267,20 @@
 		jaeger.TracerOptions.Metrics(tracerMetrics),
 		jaeger.TracerOptions.Logger(opts.logger),
 		jaeger.TracerOptions.CustomHeaderKeys(c.Headers),
-		jaeger.TracerOptions.Gen128Bit(opts.gen128Bit),
 		jaeger.TracerOptions.PoolSpans(opts.poolSpans),
 		jaeger.TracerOptions.ZipkinSharedRPCSpan(opts.zipkinSharedRPCSpan),
 		jaeger.TracerOptions.MaxTagValueLength(opts.maxTagValueLength),
 		jaeger.TracerOptions.NoDebugFlagOnForcedSampling(opts.noDebugFlagOnForcedSampling),
 	}
 
+	if c.Gen128Bit || opts.gen128Bit {
+		tracerOptions = append(tracerOptions, jaeger.TracerOptions.Gen128Bit(true))
+	}
+
+	if opts.randomNumber != nil {
+		tracerOptions = append(tracerOptions, jaeger.TracerOptions.RandomNumber(opts.randomNumber))
+	}
+
 	for _, tag := range opts.tags {
 		tracerOptions = append(tracerOptions, jaeger.TracerOptions.Tag(tag.Key, tag.Value))
 	}
@@ -382,7 +409,7 @@
 	metrics *jaeger.Metrics,
 	logger jaeger.Logger,
 ) (jaeger.Reporter, error) {
-	sender, err := rc.newTransport()
+	sender, err := rc.newTransport(logger)
 	if err != nil {
 		return nil, err
 	}
@@ -399,15 +426,22 @@
 	return reporter, err
 }
 
-func (rc *ReporterConfig) newTransport() (jaeger.Transport, error) {
+func (rc *ReporterConfig) newTransport(logger jaeger.Logger) (jaeger.Transport, error) {
 	switch {
 	case rc.CollectorEndpoint != "":
-		httpOptions := []transport.HTTPOption{transport.HTTPBatchSize(1), transport.HTTPHeaders(rc.HTTPHeaders)}
+		httpOptions := []transport.HTTPOption{transport.HTTPHeaders(rc.HTTPHeaders)}
 		if rc.User != "" && rc.Password != "" {
 			httpOptions = append(httpOptions, transport.HTTPBasicAuth(rc.User, rc.Password))
 		}
 		return transport.NewHTTPTransport(rc.CollectorEndpoint, httpOptions...), nil
 	default:
-		return jaeger.NewUDPTransport(rc.LocalAgentHostPort, 0)
+		return jaeger.NewUDPTransportWithParams(jaeger.UDPTransportParams{
+			AgentClientUDPParams: utils.AgentClientUDPParams{
+				HostPort:                   rc.LocalAgentHostPort,
+				Logger:                     logger,
+				DisableAttemptReconnecting: rc.DisableAttemptReconnecting,
+				AttemptReconnectInterval:   rc.AttemptReconnectInterval,
+			},
+		})
 	}
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/config/config_env.go b/vendor/github.com/uber/jaeger-client-go/config/config_env.go
index a729bd8..0fc3c53 100644
--- a/vendor/github.com/uber/jaeger-client-go/config/config_env.go
+++ b/vendor/github.com/uber/jaeger-client-go/config/config_env.go
@@ -24,29 +24,32 @@
 
 	"github.com/opentracing/opentracing-go"
 	"github.com/pkg/errors"
-
 	"github.com/uber/jaeger-client-go"
 )
 
 const (
 	// environment variable names
-	envServiceName            = "JAEGER_SERVICE_NAME"
-	envDisabled               = "JAEGER_DISABLED"
-	envRPCMetrics             = "JAEGER_RPC_METRICS"
-	envTags                   = "JAEGER_TAGS"
-	envSamplerType            = "JAEGER_SAMPLER_TYPE"
-	envSamplerParam           = "JAEGER_SAMPLER_PARAM"
-	envSamplerManagerHostPort = "JAEGER_SAMPLER_MANAGER_HOST_PORT"
-	envSamplerMaxOperations   = "JAEGER_SAMPLER_MAX_OPERATIONS"
-	envSamplerRefreshInterval = "JAEGER_SAMPLER_REFRESH_INTERVAL"
-	envReporterMaxQueueSize   = "JAEGER_REPORTER_MAX_QUEUE_SIZE"
-	envReporterFlushInterval  = "JAEGER_REPORTER_FLUSH_INTERVAL"
-	envReporterLogSpans       = "JAEGER_REPORTER_LOG_SPANS"
-	envEndpoint               = "JAEGER_ENDPOINT"
-	envUser                   = "JAEGER_USER"
-	envPassword               = "JAEGER_PASSWORD"
-	envAgentHost              = "JAEGER_AGENT_HOST"
-	envAgentPort              = "JAEGER_AGENT_PORT"
+	envServiceName                         = "JAEGER_SERVICE_NAME"
+	envDisabled                            = "JAEGER_DISABLED"
+	envRPCMetrics                          = "JAEGER_RPC_METRICS"
+	envTags                                = "JAEGER_TAGS"
+	envSamplerType                         = "JAEGER_SAMPLER_TYPE"
+	envSamplerParam                        = "JAEGER_SAMPLER_PARAM"
+	envSamplerManagerHostPort              = "JAEGER_SAMPLER_MANAGER_HOST_PORT" // Deprecated by envSamplingEndpoint
+	envSamplingEndpoint                    = "JAEGER_SAMPLING_ENDPOINT"
+	envSamplerMaxOperations                = "JAEGER_SAMPLER_MAX_OPERATIONS"
+	envSamplerRefreshInterval              = "JAEGER_SAMPLER_REFRESH_INTERVAL"
+	envReporterMaxQueueSize                = "JAEGER_REPORTER_MAX_QUEUE_SIZE"
+	envReporterFlushInterval               = "JAEGER_REPORTER_FLUSH_INTERVAL"
+	envReporterLogSpans                    = "JAEGER_REPORTER_LOG_SPANS"
+	envReporterAttemptReconnectingDisabled = "JAEGER_REPORTER_ATTEMPT_RECONNECTING_DISABLED"
+	envReporterAttemptReconnectInterval    = "JAEGER_REPORTER_ATTEMPT_RECONNECT_INTERVAL"
+	envEndpoint                            = "JAEGER_ENDPOINT"
+	envUser                                = "JAEGER_USER"
+	envPassword                            = "JAEGER_PASSWORD"
+	envAgentHost                           = "JAEGER_AGENT_HOST"
+	envAgentPort                           = "JAEGER_AGENT_PORT"
+	env128bit                              = "JAEGER_TRACEID_128BIT"
 )
 
 // FromEnv uses environment variables to set the tracer's Configuration
@@ -81,6 +84,14 @@
 		c.Tags = parseTags(e)
 	}
 
+	if e := os.Getenv(env128bit); e != "" {
+		if value, err := strconv.ParseBool(e); err == nil {
+			c.Gen128Bit = value
+		} else {
+			return nil, errors.Wrapf(err, "cannot parse env var %s=%s", env128bit, e)
+		}
+	}
+
 	if c.Sampler == nil {
 		c.Sampler = &SamplerConfig{}
 	}
@@ -118,7 +129,9 @@
 		}
 	}
 
-	if e := os.Getenv(envSamplerManagerHostPort); e != "" {
+	if e := os.Getenv(envSamplingEndpoint); e != "" {
+		sc.SamplingServerURL = e
+	} else if e := os.Getenv(envSamplerManagerHostPort); e != "" {
 		sc.SamplingServerURL = e
 	} else if e := os.Getenv(envAgentHost); e != "" {
 		// Fallback if we know the agent host - try the sampling endpoint there
@@ -184,20 +197,43 @@
 		rc.User = user
 		rc.Password = pswd
 	} else {
+		useEnv := false
 		host := jaeger.DefaultUDPSpanServerHost
 		if e := os.Getenv(envAgentHost); e != "" {
 			host = e
+			useEnv = true
 		}
 
 		port := jaeger.DefaultUDPSpanServerPort
 		if e := os.Getenv(envAgentPort); e != "" {
 			if value, err := strconv.ParseInt(e, 10, 0); err == nil {
 				port = int(value)
+				useEnv = true
 			} else {
 				return nil, errors.Wrapf(err, "cannot parse env var %s=%s", envAgentPort, e)
 			}
 		}
-		rc.LocalAgentHostPort = fmt.Sprintf("%s:%d", host, port)
+		if useEnv || rc.LocalAgentHostPort == "" {
+			rc.LocalAgentHostPort = fmt.Sprintf("%s:%d", host, port)
+		}
+
+		if e := os.Getenv(envReporterAttemptReconnectingDisabled); e != "" {
+			if value, err := strconv.ParseBool(e); err == nil {
+				rc.DisableAttemptReconnecting = value
+			} else {
+				return nil, errors.Wrapf(err, "cannot parse env var %s=%s", envReporterAttemptReconnectingDisabled, e)
+			}
+		}
+
+		if !rc.DisableAttemptReconnecting {
+			if e := os.Getenv(envReporterAttemptReconnectInterval); e != "" {
+				if value, err := time.ParseDuration(e); err == nil {
+					rc.AttemptReconnectInterval = value
+				} else {
+					return nil, errors.Wrapf(err, "cannot parse env var %s=%s", envReporterAttemptReconnectInterval, e)
+				}
+			}
+		}
 	}
 
 	return rc, nil
diff --git a/vendor/github.com/uber/jaeger-client-go/config/options.go b/vendor/github.com/uber/jaeger-client-go/config/options.go
index e0e50e8..a2b9cbc 100644
--- a/vendor/github.com/uber/jaeger-client-go/config/options.go
+++ b/vendor/github.com/uber/jaeger-client-go/config/options.go
@@ -40,6 +40,7 @@
 	tags                        []opentracing.Tag
 	injectors                   map[interface{}]jaeger.Injector
 	extractors                  map[interface{}]jaeger.Extractor
+	randomNumber                func() uint64
 }
 
 // Metrics creates an Option that initializes Metrics in the tracer,
@@ -147,6 +148,13 @@
 	}
 }
 
+// WithRandomNumber supplies a random number generator function to the Tracer used to generate trace and span IDs.
+func WithRandomNumber(f func() uint64) Option {
+	return func(c *Options) {
+		c.randomNumber = f
+	}
+}
+
 func applyOptions(options ...Option) Options {
 	opts := Options{
 		injectors:  make(map[interface{}]jaeger.Injector),
diff --git a/vendor/github.com/uber/jaeger-client-go/constants.go b/vendor/github.com/uber/jaeger-client-go/constants.go
index 1f8578f..d8eb698 100644
--- a/vendor/github.com/uber/jaeger-client-go/constants.go
+++ b/vendor/github.com/uber/jaeger-client-go/constants.go
@@ -22,7 +22,7 @@
 
 const (
 	// JaegerClientVersion is the version of the client library reported as Span tag.
-	JaegerClientVersion = "Go-2.23.1"
+	JaegerClientVersion = "Go-2.29.1"
 
 	// JaegerClientVersionTagKey is the name of the tag used to report client version.
 	JaegerClientVersionTagKey = "jaeger.version"
@@ -102,5 +102,5 @@
 
 var (
 	// DefaultSamplingServerURL is the default url to fetch sampling config from, via http
-	DefaultSamplingServerURL = fmt.Sprintf("http://localhost:%d/sampling", DefaultSamplingServerPort)
+	DefaultSamplingServerURL = fmt.Sprintf("http://127.0.0.1:%d/sampling", DefaultSamplingServerPort)
 )
diff --git a/vendor/github.com/uber/jaeger-client-go/glide.lock b/vendor/github.com/uber/jaeger-client-go/glide.lock
index f4c05b2..c1ec339 100644
--- a/vendor/github.com/uber/jaeger-client-go/glide.lock
+++ b/vendor/github.com/uber/jaeger-client-go/glide.lock
@@ -1,11 +1,11 @@
-hash: a4a449cfc060c2d7be850a69b171e4382a3bd00d1a0a72cfc944facc3fe263bf
-updated: 2019-09-23T17:10:15.213856-04:00
+hash: 63bec420a22b7e5abac8c602c5cc9b66a33d6a1bfec8918eecc77fd344b759ed
+updated: 2020-07-31T13:30:37.242608-04:00
 imports:
 - name: github.com/beorn7/perks
-  version: 37c8de3658fcb183f997c4e13e8337516ab753e6
+  version: 3a771d992973f24aa725d07868b467d1ddfceafb
   subpackages:
   - quantile
-- name: github.com/codahale/hdrhistogram
+- name: github.com/HdrHistogram/hdrhistogram-go
   version: 3a0bb77429bd3a61596f5e8a3172445844342120
 - name: github.com/crossdock/crossdock-go
   version: 049aabb0122b03bc9bd30cab8f3f91fb60166361
@@ -13,11 +13,15 @@
   - assert
   - require
 - name: github.com/davecgh/go-spew
-  version: d8f796af33cc11cb798c1aaeb27a4ebc5099927d
+  version: 8991bc29aa16c548c550c7ff78260e27b9ab7c73
   subpackages:
   - spew
+- name: github.com/golang/mock
+  version: 51421b967af1f557f93a59e0057aaf15ca02e29c
+  subpackages:
+  - gomock
 - name: github.com/golang/protobuf
-  version: 1680a479a2cfb3fa22b972af7e36d0a0fde47bf8
+  version: b5d812f8a3706043e23a9cd5babf2e5423744d30
   subpackages:
   - proto
 - name: github.com/matttproud/golang_protobuf_extensions
@@ -25,7 +29,7 @@
   subpackages:
   - pbutil
 - name: github.com/opentracing/opentracing-go
-  version: 659c90643e714681897ec2521c60567dd21da733
+  version: d34af3eaa63c4d08ab54863a4bdd0daa45212e12
   subpackages:
   - ext
   - harness
@@ -42,57 +46,60 @@
   - prometheus
   - prometheus/internal
 - name: github.com/prometheus/client_model
-  version: 14fe0d1b01d4d5fc031dd4bec1823bd3ebbe8016
+  version: fd36f4220a901265f90734c3183c5f0c91daa0b8
   subpackages:
   - go
 - name: github.com/prometheus/common
-  version: 287d3e634a1e550c9e463dd7e5a75a422c614505
+  version: 1ab4d74fc89940cfbc3c2b3a89821336cdefa119
   subpackages:
   - expfmt
   - internal/bitbucket.org/ww/goautoneg
   - model
 - name: github.com/prometheus/procfs
-  version: de25ac347ef9305868b04dc42425c973b863b18c
+  version: 8a055596020d692cf491851e47ba3e302d9f90ce
   subpackages:
   - internal/fs
   - internal/util
 - name: github.com/stretchr/testify
-  version: 85f2b59c4459e5bf57488796be8c3667cb8246d6
+  version: f654a9112bbeac49ca2cd45bfbe11533c4666cf8
   subpackages:
   - assert
+  - mock
   - require
   - suite
 - name: github.com/uber-go/atomic
-  version: df976f2515e274675050de7b3f42545de80594fd
+  version: 845920076a298bdb984fb0f1b86052e4ca0a281c
 - name: github.com/uber/jaeger-lib
-  version: a87ae9d84fb038a8d79266298970720be7c80fcd
+  version: 48cc1df63e6be0d63b95677f0d22beb880bce1e4
   subpackages:
   - metrics
   - metrics/metricstest
   - metrics/prometheus
 - name: go.uber.org/atomic
-  version: df976f2515e274675050de7b3f42545de80594fd
+  version: 845920076a298bdb984fb0f1b86052e4ca0a281c
 - name: go.uber.org/multierr
-  version: 3c4937480c32f4c13a875a1829af76c98ca3d40a
+  version: b587143a48b62b01d337824eab43700af6ffe222
 - name: go.uber.org/zap
-  version: 27376062155ad36be76b0f12cf1572a221d3a48c
+  version: feeb9a050b31b40eec6f2470e7599eeeadfe5bdd
   subpackages:
   - buffer
   - internal/bufferpool
   - internal/color
   - internal/exit
   - zapcore
+  - zaptest/observer
 - name: golang.org/x/net
-  version: aa69164e4478b84860dc6769c710c699c67058a3
+  version: addf6b3196f61cd44ce5a76657913698c73479d0
   subpackages:
   - context
   - context/ctxhttp
 - name: golang.org/x/sys
-  version: 0a153f010e6963173baba2306531d173aa843137
+  version: 3e129f6d46b10b0e1da36b3deffcb55e09631b64
   subpackages:
+  - internal/unsafeheader
   - windows
-- name: gopkg.in/yaml.v2
-  version: 51d6538a90f86fe93ac480b35f37b2be17fef232
-- name: github.com/golang/mock 
-  version: 3a35fb6e3e18b9dbfee291262260dee7372d2a92
-testImports: []
+- name: gopkg.in/yaml.v3
+  version: eeeca48fe7764f320e4870d231902bf9c1be2c08
+testImports:
+- name: github.com/stretchr/objx
+  version: 35313a95ee26395aa17d366c71a2ccf788fa69b6
diff --git a/vendor/github.com/uber/jaeger-client-go/glide.yaml b/vendor/github.com/uber/jaeger-client-go/glide.yaml
index eb58c67..295678c 100644
--- a/vendor/github.com/uber/jaeger-client-go/glide.yaml
+++ b/vendor/github.com/uber/jaeger-client-go/glide.yaml
@@ -1,13 +1,13 @@
 package: github.com/uber/jaeger-client-go
 import:
 - package: github.com/opentracing/opentracing-go
-  version: ^1.1
+  version: ^1.2
   subpackages:
   - ext
   - log
 - package: github.com/crossdock/crossdock-go
 - package: github.com/uber/jaeger-lib
-  version: ^2.0.0
+  version: ^2.3.0
   subpackages:
   - metrics
 - package: github.com/pkg/errors
@@ -18,7 +18,9 @@
 - package: github.com/uber-go/atomic
   version: ^1
 - package: github.com/prometheus/client_golang
-  version: ^1
+  version: 1.1
+- package: github.com/prometheus/procfs
+  version: 0.0.6
 testImport:
 - package: github.com/stretchr/testify
   subpackages:
diff --git a/vendor/github.com/uber/jaeger-client-go/internal/baggage/remote/restriction_manager.go b/vendor/github.com/uber/jaeger-client-go/internal/baggage/remote/restriction_manager.go
index a56515a..2f58bb5 100644
--- a/vendor/github.com/uber/jaeger-client-go/internal/baggage/remote/restriction_manager.go
+++ b/vendor/github.com/uber/jaeger-client-go/internal/baggage/remote/restriction_manager.go
@@ -15,6 +15,7 @@
 package remote
 
 import (
+	"context"
 	"fmt"
 	"net/url"
 	"sync"
@@ -37,7 +38,7 @@
 	}
 }
 
-func (s *httpBaggageRestrictionManagerProxy) GetBaggageRestrictions(serviceName string) ([]*thrift.BaggageRestriction, error) {
+func (s *httpBaggageRestrictionManagerProxy) GetBaggageRestrictions(context.Context, string) ([]*thrift.BaggageRestriction, error) {
 	var out []*thrift.BaggageRestriction
 	if err := utils.GetJSON(s.url, &out); err != nil {
 		return nil, err
@@ -134,7 +135,7 @@
 }
 
 func (m *RestrictionManager) updateRestrictions() error {
-	restrictions, err := m.thriftProxy.GetBaggageRestrictions(m.serviceName)
+	restrictions, err := m.thriftProxy.GetBaggageRestrictions(context.Background(), m.serviceName)
 	if err != nil {
 		m.metrics.BaggageRestrictionsUpdateFailure.Inc(1)
 		return err
diff --git a/vendor/github.com/uber/jaeger-client-go/propagation.go b/vendor/github.com/uber/jaeger-client-go/propagation.go
index 42fd64b..e06459b 100644
--- a/vendor/github.com/uber/jaeger-client-go/propagation.go
+++ b/vendor/github.com/uber/jaeger-client-go/propagation.go
@@ -215,6 +215,12 @@
 	return nil
 }
 
+// W3C limits https://github.com/w3c/baggage/blob/master/baggage/HTTP_HEADER_FORMAT.md#limits
+const (
+	maxBinaryBaggage      = 180
+	maxBinaryNameValueLen = 4096
+)
+
 // Extract implements Extractor of BinaryPropagator
 func (p *BinaryPropagator) Extract(abstractCarrier interface{}) (SpanContext, error) {
 	carrier, ok := abstractCarrier.(io.Reader)
@@ -245,6 +251,9 @@
 	if err := binary.Read(carrier, binary.BigEndian, &numBaggage); err != nil {
 		return emptyContext, opentracing.ErrSpanContextCorrupted
 	}
+	if numBaggage > maxBinaryBaggage {
+		return emptyContext, opentracing.ErrSpanContextCorrupted
+	}
 	if iNumBaggage := int(numBaggage); iNumBaggage > 0 {
 		ctx.baggage = make(map[string]string, iNumBaggage)
 		buf := p.buffers.Get().(*bytes.Buffer)
@@ -265,6 +274,9 @@
 			if err := binary.Read(carrier, binary.BigEndian, &valLen); err != nil {
 				return emptyContext, opentracing.ErrSpanContextCorrupted
 			}
+			if keyLen+valLen > maxBinaryNameValueLen {
+				return emptyContext, opentracing.ErrSpanContextCorrupted
+			}
 			buf.Reset()
 			buf.Grow(int(valLen))
 			if n, err := io.CopyN(buf, carrier, int64(valLen)); err != nil || int32(n) != valLen {
@@ -292,7 +304,7 @@
 	for _, kvpair := range strings.Split(value, ",") {
 		kv := strings.Split(strings.TrimSpace(kvpair), "=")
 		if len(kv) == 2 {
-			baggage[kv[0]] = kv[1]
+			baggage[strings.TrimSpace(kv[0])] = kv[1]
 		} else {
 			log.Printf("Malformed value passed in for %s", p.headerKeys.JaegerBaggageHeader)
 		}
diff --git a/vendor/github.com/uber/jaeger-client-go/sampler_remote.go b/vendor/github.com/uber/jaeger-client-go/sampler_remote.go
index 112e3e1..119f0a1 100644
--- a/vendor/github.com/uber/jaeger-client-go/sampler_remote.go
+++ b/vendor/github.com/uber/jaeger-client-go/sampler_remote.go
@@ -29,6 +29,7 @@
 )
 
 const (
+	defaultRemoteSamplingTimeout   = 10 * time.Second
 	defaultSamplingRefreshInterval = time.Minute
 )
 
@@ -64,7 +65,7 @@
 	// Cf. https://github.com/uber/jaeger-client-go/issues/155, https://goo.gl/zW7dgq
 	closed int64 // 0 - not closed, 1 - closed
 
-	sync.RWMutex
+	sync.RWMutex // used to serialize access to samplerOptions.sampler
 	samplerOptions
 
 	serviceName string
@@ -95,21 +96,29 @@
 
 // OnCreateSpan implements OnCreateSpan of SamplerV2.
 func (s *RemotelyControlledSampler) OnCreateSpan(span *Span) SamplingDecision {
+	s.RLock()
+	defer s.RUnlock()
 	return s.sampler.OnCreateSpan(span)
 }
 
 // OnSetOperationName implements OnSetOperationName of SamplerV2.
 func (s *RemotelyControlledSampler) OnSetOperationName(span *Span, operationName string) SamplingDecision {
+	s.RLock()
+	defer s.RUnlock()
 	return s.sampler.OnSetOperationName(span, operationName)
 }
 
 // OnSetTag implements OnSetTag of SamplerV2.
 func (s *RemotelyControlledSampler) OnSetTag(span *Span, key string, value interface{}) SamplingDecision {
+	s.RLock()
+	defer s.RUnlock()
 	return s.sampler.OnSetTag(span, key, value)
 }
 
 // OnFinishSpan implements OnFinishSpan of SamplerV2.
 func (s *RemotelyControlledSampler) OnFinishSpan(span *Span) SamplingDecision {
+	s.RLock()
+	defer s.RUnlock()
 	return s.sampler.OnFinishSpan(span)
 }
 
@@ -153,11 +162,10 @@
 
 // Sampler returns the currently active sampler.
 func (s *RemotelyControlledSampler) Sampler() SamplerV2 {
-	s.Lock()
-	defer s.Unlock()
+	s.RLock()
+	defer s.RUnlock()
 	return s.sampler
 }
-
 func (s *RemotelyControlledSampler) setSampler(sampler SamplerV2) {
 	s.Lock()
 	defer s.Unlock()
@@ -291,8 +299,22 @@
 // -----------------------
 
 type httpSamplingStrategyFetcher struct {
-	serverURL string
-	logger    log.DebugLogger
+	serverURL  string
+	logger     log.DebugLogger
+	httpClient http.Client
+}
+
+func newHTTPSamplingStrategyFetcher(serverURL string, logger log.DebugLogger) *httpSamplingStrategyFetcher {
+	customTransport := http.DefaultTransport.(*http.Transport).Clone()
+	customTransport.ResponseHeaderTimeout = defaultRemoteSamplingTimeout
+
+	return &httpSamplingStrategyFetcher{
+		serverURL: serverURL,
+		logger:    logger,
+		httpClient: http.Client{
+			Transport: customTransport,
+		},
+	}
 }
 
 func (f *httpSamplingStrategyFetcher) Fetch(serviceName string) ([]byte, error) {
@@ -300,8 +322,7 @@
 	v.Set("service", serviceName)
 	uri := f.serverURL + "?" + v.Encode()
 
-	// TODO create and reuse http.Client with proper timeout settings, etc.
-	resp, err := http.Get(uri)
+	resp, err := f.httpClient.Get(uri)
 	if err != nil {
 		return nil, err
 	}
diff --git a/vendor/github.com/uber/jaeger-client-go/sampler_remote_options.go b/vendor/github.com/uber/jaeger-client-go/sampler_remote_options.go
index e4a6108..64b028b 100644
--- a/vendor/github.com/uber/jaeger-client-go/sampler_remote_options.go
+++ b/vendor/github.com/uber/jaeger-client-go/sampler_remote_options.go
@@ -140,10 +140,7 @@
 		o.samplingRefreshInterval = defaultSamplingRefreshInterval
 	}
 	if o.samplingFetcher == nil {
-		o.samplingFetcher = &httpSamplingStrategyFetcher{
-			serverURL: o.samplingServerURL,
-			logger:    o.logger,
-		}
+		o.samplingFetcher = newHTTPSamplingStrategyFetcher(o.samplingServerURL, o.logger)
 	}
 	if o.samplingParser == nil {
 		o.samplingParser = new(samplingStrategyParser)
diff --git a/vendor/github.com/uber/jaeger-client-go/span.go b/vendor/github.com/uber/jaeger-client-go/span.go
index 42c9112..997cffd 100644
--- a/vendor/github.com/uber/jaeger-client-go/span.go
+++ b/vendor/github.com/uber/jaeger-client-go/span.go
@@ -85,8 +85,9 @@
 func (s *Span) SetOperationName(operationName string) opentracing.Span {
 	s.Lock()
 	s.operationName = operationName
+	ctx := s.context
 	s.Unlock()
-	if !s.isSamplingFinalized() {
+	if !ctx.isSamplingFinalized() {
 		decision := s.tracer.sampler.OnSetOperationName(s, operationName)
 		s.applySamplingDecision(decision, true)
 	}
@@ -100,15 +101,25 @@
 }
 
 func (s *Span) setTagInternal(key string, value interface{}, lock bool) opentracing.Span {
+	var ctx SpanContext
+	var operationName string
+	if lock {
+		ctx = s.SpanContext()
+		operationName = s.OperationName()
+	} else {
+		ctx = s.context
+		operationName = s.operationName
+	}
+
 	s.observer.OnSetTag(key, value)
-	if key == string(ext.SamplingPriority) && !setSamplingPriority(s, value) {
+	if key == string(ext.SamplingPriority) && !setSamplingPriority(ctx.samplingState, operationName, s.tracer, value) {
 		return s
 	}
-	if !s.isSamplingFinalized() {
+	if !ctx.isSamplingFinalized() {
 		decision := s.tracer.sampler.OnSetTag(s, key, value)
 		s.applySamplingDecision(decision, lock)
 	}
-	if s.isWriteable() {
+	if ctx.isWriteable() {
 		if lock {
 			s.Lock()
 			defer s.Unlock()
@@ -303,7 +314,14 @@
 	s.numDroppedLogs = 0
 }
 
-// SetBaggageItem implements SetBaggageItem() of opentracing.SpanContext
+// SetBaggageItem implements SetBaggageItem() of opentracing.SpanContext.
+// The call is proxied via tracer.baggageSetter to allow policies to be applied
+// before allowing to set/replace baggage keys.
+// The setter eventually stores a new SpanContext with extended baggage:
+//
+//     span.context = span.context.WithBaggageItem(key, value)
+//
+//  See SpanContext.WithBaggageItem() for explanation why it's done this way.
 func (s *Span) SetBaggageItem(key, value string) opentracing.Span {
 	s.Lock()
 	defer s.Unlock()
@@ -333,12 +351,13 @@
 	s.observer.OnFinish(options)
 	s.Lock()
 	s.duration = options.FinishTime.Sub(s.startTime)
+	ctx := s.context
 	s.Unlock()
-	if !s.isSamplingFinalized() {
+	if !ctx.isSamplingFinalized() {
 		decision := s.tracer.sampler.OnFinishSpan(s)
 		s.applySamplingDecision(decision, true)
 	}
-	if s.context.IsSampled() {
+	if ctx.IsSampled() {
 		s.Lock()
 		s.fixLogsIfDropped()
 		if len(options.LogRecords) > 0 || len(options.BulkLogData) > 0 {
@@ -419,11 +438,18 @@
 }
 
 func (s *Span) applySamplingDecision(decision SamplingDecision, lock bool) {
+	var ctx SpanContext
+	if lock {
+		ctx = s.SpanContext()
+	} else {
+		ctx = s.context
+	}
+
 	if !decision.Retryable {
-		s.context.samplingState.setFinal()
+		ctx.samplingState.setFinal()
 	}
 	if decision.Sample {
-		s.context.samplingState.setSampled()
+		ctx.samplingState.setSampled()
 		if len(decision.Tags) > 0 {
 			if lock {
 				s.Lock()
@@ -436,44 +462,34 @@
 	}
 }
 
-// Span can be written to if it is sampled or the sampling decision has not been finalized.
-func (s *Span) isWriteable() bool {
-	state := s.context.samplingState
-	return !state.isFinal() || state.isSampled()
-}
-
-func (s *Span) isSamplingFinalized() bool {
-	return s.context.samplingState.isFinal()
-}
-
 // setSamplingPriority returns true if the flag was updated successfully, false otherwise.
 // The behavior of setSamplingPriority is surprising
 // If noDebugFlagOnForcedSampling is set
-//     setSamplingPriority(span, 1) always sets only flagSampled
+//     setSamplingPriority(..., 1) always sets only flagSampled
 // If noDebugFlagOnForcedSampling is unset, and isDebugAllowed passes
-//     setSamplingPriority(span, 1) sets both flagSampled and flagDebug
+//     setSamplingPriority(..., 1) sets both flagSampled and flagDebug
 // However,
-//     setSamplingPriority(span, 0) always only resets flagSampled
+//     setSamplingPriority(..., 0) always only resets flagSampled
 //
-// This means that doing a setSamplingPriority(span, 1) followed by setSamplingPriority(span, 0) can
+// This means that doing a setSamplingPriority(..., 1) followed by setSamplingPriority(..., 0) can
 // leave flagDebug set
-func setSamplingPriority(s *Span, value interface{}) bool {
+func setSamplingPriority(state *samplingState, operationName string, tracer *Tracer, value interface{}) bool {
 	val, ok := value.(uint16)
 	if !ok {
 		return false
 	}
 	if val == 0 {
-		s.context.samplingState.unsetSampled()
-		s.context.samplingState.setFinal()
+		state.unsetSampled()
+		state.setFinal()
 		return true
 	}
-	if s.tracer.options.noDebugFlagOnForcedSampling {
-		s.context.samplingState.setSampled()
-		s.context.samplingState.setFinal()
+	if tracer.options.noDebugFlagOnForcedSampling {
+		state.setSampled()
+		state.setFinal()
 		return true
-	} else if s.tracer.isDebugAllowed(s.operationName) {
-		s.context.samplingState.setDebugAndSampled()
-		s.context.samplingState.setFinal()
+	} else if tracer.isDebugAllowed(operationName) {
+		state.setDebugAndSampled()
+		state.setFinal()
 		return true
 	}
 	return false
diff --git a/vendor/github.com/uber/jaeger-client-go/span_context.go b/vendor/github.com/uber/jaeger-client-go/span_context.go
index 1b44f3f..5b2307b 100644
--- a/vendor/github.com/uber/jaeger-client-go/span_context.go
+++ b/vendor/github.com/uber/jaeger-client-go/span_context.go
@@ -212,10 +212,14 @@
 }
 
 func (c SpanContext) String() string {
-	if c.traceID.High == 0 {
-		return fmt.Sprintf("%016x:%016x:%016x:%x", c.traceID.Low, uint64(c.spanID), uint64(c.parentID), c.samplingState.stateFlags.Load())
+	var flags int32
+	if c.samplingState != nil {
+		flags = c.samplingState.stateFlags.Load()
 	}
-	return fmt.Sprintf("%016x%016x:%016x:%016x:%x", c.traceID.High, c.traceID.Low, uint64(c.spanID), uint64(c.parentID), c.samplingState.stateFlags.Load())
+	if c.traceID.High == 0 {
+		return fmt.Sprintf("%016x:%016x:%016x:%x", c.traceID.Low, uint64(c.spanID), uint64(c.parentID), flags)
+	}
+	return fmt.Sprintf("%016x%016x:%016x:%016x:%x", c.traceID.High, c.traceID.Low, uint64(c.spanID), uint64(c.parentID), flags)
 }
 
 // ContextFromString reconstructs the Context encoded in a string
@@ -267,6 +271,16 @@
 	return c.samplingState.flags()
 }
 
+// Span can be written to if it is sampled or the sampling decision has not been finalized.
+func (c SpanContext) isWriteable() bool {
+	state := c.samplingState
+	return !state.isFinal() || state.isSampled()
+}
+
+func (c SpanContext) isSamplingFinalized() bool {
+	return c.samplingState.isFinal()
+}
+
 // NewSpanContext creates a new instance of SpanContext
 func NewSpanContext(traceID TraceID, spanID, parentID SpanID, sampled bool, baggage map[string]string) SpanContext {
 	samplingState := &samplingState{}
@@ -300,8 +314,29 @@
 }
 
 // WithBaggageItem creates a new context with an extra baggage item.
+// Delete a baggage item if provided blank value.
+//
+// The SpanContext is designed to be immutable and passed by value. As such,
+// it cannot contain any locks, and should only hold immutable data, including baggage.
+// Another reason for why baggage is immutable is when the span context is passed
+// as a parent when starting a new span. The new span's baggage cannot affect the parent
+// span's baggage, so the child span either needs to take a copy of the parent baggage
+// (which is expensive and unnecessary since baggage rarely changes in the life span of
+// a trace), or it needs to do a copy-on-write, which is the approach taken here.
 func (c SpanContext) WithBaggageItem(key, value string) SpanContext {
 	var newBaggage map[string]string
+	// unset baggage item
+	if value == "" {
+		if _, ok := c.baggage[key]; !ok {
+			return c
+		}
+		newBaggage = make(map[string]string, len(c.baggage))
+		for k, v := range c.baggage {
+			newBaggage[k] = v
+		}
+		delete(newBaggage, key)
+		return SpanContext{c.traceID, c.spanID, c.parentID, newBaggage, "", c.samplingState, c.remote}
+	}
 	if c.baggage == nil {
 		newBaggage = map[string]string{key: value}
 	} else {
@@ -332,9 +367,9 @@
 
 func (t TraceID) String() string {
 	if t.High == 0 {
-		return fmt.Sprintf("%x", t.Low)
+		return fmt.Sprintf("%016x", t.Low)
 	}
-	return fmt.Sprintf("%x%016x", t.High, t.Low)
+	return fmt.Sprintf("%016x%016x", t.High, t.Low)
 }
 
 // TraceIDFromString creates a TraceID from a hexadecimal string
@@ -367,7 +402,7 @@
 // ------- SpanID -------
 
 func (s SpanID) String() string {
-	return fmt.Sprintf("%x", uint64(s))
+	return fmt.Sprintf("%016x", uint64(s))
 }
 
 // SpanIDFromString creates a SpanID from a hexadecimal string
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/GoUnusedProtection__.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/GoUnusedProtection__.go
new file mode 100644
index 0000000..54cd3b0
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/GoUnusedProtection__.go
@@ -0,0 +1,6 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package agent
+
+var GoUnusedProtection__ int;
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent-consts.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent-consts.go
new file mode 100644
index 0000000..a0df507
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent-consts.go
@@ -0,0 +1,28 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package agent
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+	"github.com/uber/jaeger-client-go/thrift-gen/jaeger"
+	"github.com/uber/jaeger-client-go/thrift-gen/zipkincore"
+
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+var _ = jaeger.GoUnusedProtection__
+var _ = zipkincore.GoUnusedProtection__
+
+func init() {
+}
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent.go
index e48811c..6472e84 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/agent.go
@@ -1,411 +1,396 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
 
 package agent
 
-import (
+import(
 	"bytes"
+	"context"
 	"fmt"
+	"time"
 	"github.com/uber/jaeger-client-go/thrift"
 	"github.com/uber/jaeger-client-go/thrift-gen/jaeger"
 	"github.com/uber/jaeger-client-go/thrift-gen/zipkincore"
+
 )
 
 // (needed to ensure safety because of naive import list construction.)
 var _ = thrift.ZERO
 var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
 var _ = bytes.Equal
 
 var _ = jaeger.GoUnusedProtection__
 var _ = zipkincore.GoUnusedProtection__
-
 type Agent interface {
-	// Parameters:
-	//  - Spans
-	EmitZipkinBatch(spans []*zipkincore.Span) (err error)
-	// Parameters:
-	//  - Batch
-	EmitBatch(batch *jaeger.Batch) (err error)
+  // Parameters:
+  //  - Spans
+  EmitZipkinBatch(ctx context.Context, spans []*zipkincore.Span) (_err error)
+  // Parameters:
+  //  - Batch
+  EmitBatch(ctx context.Context, batch *jaeger.Batch) (_err error)
 }
 
 type AgentClient struct {
-	Transport       thrift.TTransport
-	ProtocolFactory thrift.TProtocolFactory
-	InputProtocol   thrift.TProtocol
-	OutputProtocol  thrift.TProtocol
-	SeqId           int32
+  c thrift.TClient
+  meta thrift.ResponseMeta
 }
 
 func NewAgentClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *AgentClient {
-	return &AgentClient{Transport: t,
-		ProtocolFactory: f,
-		InputProtocol:   f.GetProtocol(t),
-		OutputProtocol:  f.GetProtocol(t),
-		SeqId:           0,
-	}
+  return &AgentClient{
+    c: thrift.NewTStandardClient(f.GetProtocol(t), f.GetProtocol(t)),
+  }
 }
 
 func NewAgentClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *AgentClient {
-	return &AgentClient{Transport: t,
-		ProtocolFactory: nil,
-		InputProtocol:   iprot,
-		OutputProtocol:  oprot,
-		SeqId:           0,
-	}
+  return &AgentClient{
+    c: thrift.NewTStandardClient(iprot, oprot),
+  }
+}
+
+func NewAgentClient(c thrift.TClient) *AgentClient {
+  return &AgentClient{
+    c: c,
+  }
+}
+
+func (p *AgentClient) Client_() thrift.TClient {
+  return p.c
+}
+
+func (p *AgentClient) LastResponseMeta_() thrift.ResponseMeta {
+  return p.meta
+}
+
+func (p *AgentClient) SetLastResponseMeta_(meta thrift.ResponseMeta) {
+  p.meta = meta
 }
 
 // Parameters:
 //  - Spans
-func (p *AgentClient) EmitZipkinBatch(spans []*zipkincore.Span) (err error) {
-	if err = p.sendEmitZipkinBatch(spans); err != nil {
-		return
-	}
-	return
-}
-
-func (p *AgentClient) sendEmitZipkinBatch(spans []*zipkincore.Span) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("emitZipkinBatch", thrift.ONEWAY, p.SeqId); err != nil {
-		return
-	}
-	args := AgentEmitZipkinBatchArgs{
-		Spans: spans,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
+func (p *AgentClient) EmitZipkinBatch(ctx context.Context, spans []*zipkincore.Span) (_err error) {
+  var _args0 AgentEmitZipkinBatchArgs
+  _args0.Spans = spans
+  p.SetLastResponseMeta_(thrift.ResponseMeta{})
+  if _, err := p.Client_().Call(ctx, "emitZipkinBatch", &_args0, nil); err != nil {
+    return err
+  }
+  return nil
 }
 
 // Parameters:
 //  - Batch
-func (p *AgentClient) EmitBatch(batch *jaeger.Batch) (err error) {
-	if err = p.sendEmitBatch(batch); err != nil {
-		return
-	}
-	return
-}
-
-func (p *AgentClient) sendEmitBatch(batch *jaeger.Batch) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("emitBatch", thrift.ONEWAY, p.SeqId); err != nil {
-		return
-	}
-	args := AgentEmitBatchArgs{
-		Batch: batch,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
+func (p *AgentClient) EmitBatch(ctx context.Context, batch *jaeger.Batch) (_err error) {
+  var _args1 AgentEmitBatchArgs
+  _args1.Batch = batch
+  p.SetLastResponseMeta_(thrift.ResponseMeta{})
+  if _, err := p.Client_().Call(ctx, "emitBatch", &_args1, nil); err != nil {
+    return err
+  }
+  return nil
 }
 
 type AgentProcessor struct {
-	processorMap map[string]thrift.TProcessorFunction
-	handler      Agent
+  processorMap map[string]thrift.TProcessorFunction
+  handler Agent
 }
 
 func (p *AgentProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
-	p.processorMap[key] = processor
+  p.processorMap[key] = processor
 }
 
 func (p *AgentProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
-	processor, ok = p.processorMap[key]
-	return processor, ok
+  processor, ok = p.processorMap[key]
+  return processor, ok
 }
 
 func (p *AgentProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
-	return p.processorMap
+  return p.processorMap
 }
 
 func NewAgentProcessor(handler Agent) *AgentProcessor {
 
-	self0 := &AgentProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
-	self0.processorMap["emitZipkinBatch"] = &agentProcessorEmitZipkinBatch{handler: handler}
-	self0.processorMap["emitBatch"] = &agentProcessorEmitBatch{handler: handler}
-	return self0
+  self2 := &AgentProcessor{handler:handler, processorMap:make(map[string]thrift.TProcessorFunction)}
+  self2.processorMap["emitZipkinBatch"] = &agentProcessorEmitZipkinBatch{handler:handler}
+  self2.processorMap["emitBatch"] = &agentProcessorEmitBatch{handler:handler}
+return self2
 }
 
-func (p *AgentProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	name, _, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return false, err
-	}
-	if processor, ok := p.GetProcessorFunction(name); ok {
-		return processor.Process(seqId, iprot, oprot)
-	}
-	iprot.Skip(thrift.STRUCT)
-	iprot.ReadMessageEnd()
-	x1 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
-	oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
-	x1.Write(oprot)
-	oprot.WriteMessageEnd()
-	oprot.Flush()
-	return false, x1
+func (p *AgentProcessor) Process(ctx context.Context, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  name, _, seqId, err2 := iprot.ReadMessageBegin(ctx)
+  if err2 != nil { return false, thrift.WrapTException(err2) }
+  if processor, ok := p.GetProcessorFunction(name); ok {
+    return processor.Process(ctx, seqId, iprot, oprot)
+  }
+  iprot.Skip(ctx, thrift.STRUCT)
+  iprot.ReadMessageEnd(ctx)
+  x3 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function " + name)
+  oprot.WriteMessageBegin(ctx, name, thrift.EXCEPTION, seqId)
+  x3.Write(ctx, oprot)
+  oprot.WriteMessageEnd(ctx)
+  oprot.Flush(ctx)
+  return false, x3
 
 }
 
 type agentProcessorEmitZipkinBatch struct {
-	handler Agent
+  handler Agent
 }
 
-func (p *agentProcessorEmitZipkinBatch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := AgentEmitZipkinBatchArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		return false, err
-	}
+func (p *agentProcessorEmitZipkinBatch) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := AgentEmitZipkinBatchArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
 
-	iprot.ReadMessageEnd()
-	var err2 error
-	if err2 = p.handler.EmitZipkinBatch(args.Spans); err2 != nil {
-		return true, err2
-	}
-	return true, nil
+  tickerCancel := func() {}
+  _ = tickerCancel
+
+  if err2 = p.handler.EmitZipkinBatch(ctx, args.Spans); err2 != nil {
+    tickerCancel()
+    return true, thrift.WrapTException(err2)
+  }
+  tickerCancel()
+  return true, nil
 }
 
 type agentProcessorEmitBatch struct {
-	handler Agent
+  handler Agent
 }
 
-func (p *agentProcessorEmitBatch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := AgentEmitBatchArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		return false, err
-	}
+func (p *agentProcessorEmitBatch) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := AgentEmitBatchArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
 
-	iprot.ReadMessageEnd()
-	var err2 error
-	if err2 = p.handler.EmitBatch(args.Batch); err2 != nil {
-		return true, err2
-	}
-	return true, nil
+  tickerCancel := func() {}
+  _ = tickerCancel
+
+  if err2 = p.handler.EmitBatch(ctx, args.Batch); err2 != nil {
+    tickerCancel()
+    return true, thrift.WrapTException(err2)
+  }
+  tickerCancel()
+  return true, nil
 }
 
+
 // HELPER FUNCTIONS AND STRUCTURES
 
 // Attributes:
 //  - Spans
 type AgentEmitZipkinBatchArgs struct {
-	Spans []*zipkincore.Span `thrift:"spans,1" json:"spans"`
+  Spans []*zipkincore.Span `thrift:"spans,1" db:"spans" json:"spans"`
 }
 
 func NewAgentEmitZipkinBatchArgs() *AgentEmitZipkinBatchArgs {
-	return &AgentEmitZipkinBatchArgs{}
+  return &AgentEmitZipkinBatchArgs{}
 }
 
+
 func (p *AgentEmitZipkinBatchArgs) GetSpans() []*zipkincore.Span {
-	return p.Spans
+  return p.Spans
 }
-func (p *AgentEmitZipkinBatchArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
+func (p *AgentEmitZipkinBatchArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
 
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
 }
 
-func (p *AgentEmitZipkinBatchArgs) readField1(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*zipkincore.Span, 0, size)
-	p.Spans = tSlice
-	for i := 0; i < size; i++ {
-		_elem2 := &zipkincore.Span{}
-		if err := _elem2.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem2), err)
-		}
-		p.Spans = append(p.Spans, _elem2)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
+func (p *AgentEmitZipkinBatchArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*zipkincore.Span, 0, size)
+  p.Spans =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem4 := &zipkincore.Span{}
+    if err := _elem4.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err)
+    }
+    p.Spans = append(p.Spans, _elem4)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
 }
 
-func (p *AgentEmitZipkinBatchArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("emitZipkinBatch_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
+func (p *AgentEmitZipkinBatchArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "emitZipkinBatch_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
 }
 
-func (p *AgentEmitZipkinBatchArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("spans", thrift.LIST, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:spans: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Spans)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Spans {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:spans: ", p), err)
-	}
-	return err
+func (p *AgentEmitZipkinBatchArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "spans", thrift.LIST, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:spans: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Spans)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Spans {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:spans: ", p), err) }
+  return err
 }
 
 func (p *AgentEmitZipkinBatchArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("AgentEmitZipkinBatchArgs(%+v)", *p)
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("AgentEmitZipkinBatchArgs(%+v)", *p)
 }
 
 // Attributes:
 //  - Batch
 type AgentEmitBatchArgs struct {
-	Batch *jaeger.Batch `thrift:"batch,1" json:"batch"`
+  Batch *jaeger.Batch `thrift:"batch,1" db:"batch" json:"batch"`
 }
 
 func NewAgentEmitBatchArgs() *AgentEmitBatchArgs {
-	return &AgentEmitBatchArgs{}
+  return &AgentEmitBatchArgs{}
 }
 
 var AgentEmitBatchArgs_Batch_DEFAULT *jaeger.Batch
-
 func (p *AgentEmitBatchArgs) GetBatch() *jaeger.Batch {
-	if !p.IsSetBatch() {
-		return AgentEmitBatchArgs_Batch_DEFAULT
-	}
-	return p.Batch
+  if !p.IsSetBatch() {
+    return AgentEmitBatchArgs_Batch_DEFAULT
+  }
+return p.Batch
 }
 func (p *AgentEmitBatchArgs) IsSetBatch() bool {
-	return p.Batch != nil
+  return p.Batch != nil
 }
 
-func (p *AgentEmitBatchArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
+func (p *AgentEmitBatchArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
 
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
 }
 
-func (p *AgentEmitBatchArgs) readField1(iprot thrift.TProtocol) error {
-	p.Batch = &jaeger.Batch{}
-	if err := p.Batch.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Batch), err)
-	}
-	return nil
+func (p *AgentEmitBatchArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Batch = &jaeger.Batch{}
+  if err := p.Batch.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Batch), err)
+  }
+  return nil
 }
 
-func (p *AgentEmitBatchArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("emitBatch_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
+func (p *AgentEmitBatchArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "emitBatch_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
 }
 
-func (p *AgentEmitBatchArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("batch", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:batch: ", p), err)
-	}
-	if err := p.Batch.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Batch), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:batch: ", p), err)
-	}
-	return err
+func (p *AgentEmitBatchArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "batch", thrift.STRUCT, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:batch: ", p), err) }
+  if err := p.Batch.Write(ctx, oprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Batch), err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:batch: ", p), err) }
+  return err
 }
 
 func (p *AgentEmitBatchArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("AgentEmitBatchArgs(%+v)", *p)
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("AgentEmitBatchArgs(%+v)", *p)
 }
+
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/constants.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/constants.go
deleted file mode 100644
index aa9857b..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/constants.go
+++ /dev/null
@@ -1,23 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package agent
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-	"github.com/uber/jaeger-client-go/thrift-gen/jaeger"
-	"github.com/uber/jaeger-client-go/thrift-gen/zipkincore"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var _ = jaeger.GoUnusedProtection__
-var _ = zipkincore.GoUnusedProtection__
-
-func init() {
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/ttypes.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/ttypes.go
deleted file mode 100644
index 9c28f11..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/agent/ttypes.go
+++ /dev/null
@@ -1,21 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package agent
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-	"github.com/uber/jaeger-client-go/thrift-gen/jaeger"
-	"github.com/uber/jaeger-client-go/thrift-gen/zipkincore"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var _ = jaeger.GoUnusedProtection__
-var _ = zipkincore.GoUnusedProtection__
-var GoUnusedProtection__ int
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/GoUnusedProtection__.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/GoUnusedProtection__.go
new file mode 100644
index 0000000..712b6a9
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/GoUnusedProtection__.go
@@ -0,0 +1,6 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package baggage
+
+var GoUnusedProtection__ int;
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage-consts.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage-consts.go
new file mode 100644
index 0000000..39b5a7e
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage-consts.go
@@ -0,0 +1,23 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package baggage
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+
+func init() {
+}
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage.go
new file mode 100644
index 0000000..e4d89d5
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggage.go
@@ -0,0 +1,565 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package baggage
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+// Attributes:
+//  - BaggageKey
+//  - MaxValueLength
+type BaggageRestriction struct {
+  BaggageKey string `thrift:"baggageKey,1,required" db:"baggageKey" json:"baggageKey"`
+  MaxValueLength int32 `thrift:"maxValueLength,2,required" db:"maxValueLength" json:"maxValueLength"`
+}
+
+func NewBaggageRestriction() *BaggageRestriction {
+  return &BaggageRestriction{}
+}
+
+
+func (p *BaggageRestriction) GetBaggageKey() string {
+  return p.BaggageKey
+}
+
+func (p *BaggageRestriction) GetMaxValueLength() int32 {
+  return p.MaxValueLength
+}
+func (p *BaggageRestriction) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetBaggageKey bool = false;
+  var issetMaxValueLength bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetBaggageKey = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetMaxValueLength = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetBaggageKey{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field BaggageKey is not set"));
+  }
+  if !issetMaxValueLength{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field MaxValueLength is not set"));
+  }
+  return nil
+}
+
+func (p *BaggageRestriction)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.BaggageKey = v
+}
+  return nil
+}
+
+func (p *BaggageRestriction)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.MaxValueLength = v
+}
+  return nil
+}
+
+func (p *BaggageRestriction) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "BaggageRestriction"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *BaggageRestriction) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "baggageKey", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:baggageKey: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.BaggageKey)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.baggageKey (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:baggageKey: ", p), err) }
+  return err
+}
+
+func (p *BaggageRestriction) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "maxValueLength", thrift.I32, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:maxValueLength: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.MaxValueLength)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.maxValueLength (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:maxValueLength: ", p), err) }
+  return err
+}
+
+func (p *BaggageRestriction) Equals(other *BaggageRestriction) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.BaggageKey != other.BaggageKey { return false }
+  if p.MaxValueLength != other.MaxValueLength { return false }
+  return true
+}
+
+func (p *BaggageRestriction) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("BaggageRestriction(%+v)", *p)
+}
+
+type BaggageRestrictionManager interface {
+  // getBaggageRestrictions retrieves the baggage restrictions for a specific service.
+  // Usually, baggageRestrictions apply to all services however there may be situations
+  // where a baggageKey might only be allowed to be set by a specific service.
+  // 
+  // Parameters:
+  //  - ServiceName
+  GetBaggageRestrictions(ctx context.Context, serviceName string) (_r []*BaggageRestriction, _err error)
+}
+
+type BaggageRestrictionManagerClient struct {
+  c thrift.TClient
+  meta thrift.ResponseMeta
+}
+
+func NewBaggageRestrictionManagerClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *BaggageRestrictionManagerClient {
+  return &BaggageRestrictionManagerClient{
+    c: thrift.NewTStandardClient(f.GetProtocol(t), f.GetProtocol(t)),
+  }
+}
+
+func NewBaggageRestrictionManagerClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *BaggageRestrictionManagerClient {
+  return &BaggageRestrictionManagerClient{
+    c: thrift.NewTStandardClient(iprot, oprot),
+  }
+}
+
+func NewBaggageRestrictionManagerClient(c thrift.TClient) *BaggageRestrictionManagerClient {
+  return &BaggageRestrictionManagerClient{
+    c: c,
+  }
+}
+
+func (p *BaggageRestrictionManagerClient) Client_() thrift.TClient {
+  return p.c
+}
+
+func (p *BaggageRestrictionManagerClient) LastResponseMeta_() thrift.ResponseMeta {
+  return p.meta
+}
+
+func (p *BaggageRestrictionManagerClient) SetLastResponseMeta_(meta thrift.ResponseMeta) {
+  p.meta = meta
+}
+
+// getBaggageRestrictions retrieves the baggage restrictions for a specific service.
+// Usually, baggageRestrictions apply to all services however there may be situations
+// where a baggageKey might only be allowed to be set by a specific service.
+// 
+// Parameters:
+//  - ServiceName
+func (p *BaggageRestrictionManagerClient) GetBaggageRestrictions(ctx context.Context, serviceName string) (_r []*BaggageRestriction, _err error) {
+  var _args0 BaggageRestrictionManagerGetBaggageRestrictionsArgs
+  _args0.ServiceName = serviceName
+  var _result2 BaggageRestrictionManagerGetBaggageRestrictionsResult
+  var _meta1 thrift.ResponseMeta
+  _meta1, _err = p.Client_().Call(ctx, "getBaggageRestrictions", &_args0, &_result2)
+  p.SetLastResponseMeta_(_meta1)
+  if _err != nil {
+    return
+  }
+  return _result2.GetSuccess(), nil
+}
+
+type BaggageRestrictionManagerProcessor struct {
+  processorMap map[string]thrift.TProcessorFunction
+  handler BaggageRestrictionManager
+}
+
+func (p *BaggageRestrictionManagerProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
+  p.processorMap[key] = processor
+}
+
+func (p *BaggageRestrictionManagerProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
+  processor, ok = p.processorMap[key]
+  return processor, ok
+}
+
+func (p *BaggageRestrictionManagerProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
+  return p.processorMap
+}
+
+func NewBaggageRestrictionManagerProcessor(handler BaggageRestrictionManager) *BaggageRestrictionManagerProcessor {
+
+  self3 := &BaggageRestrictionManagerProcessor{handler:handler, processorMap:make(map[string]thrift.TProcessorFunction)}
+  self3.processorMap["getBaggageRestrictions"] = &baggageRestrictionManagerProcessorGetBaggageRestrictions{handler:handler}
+return self3
+}
+
+func (p *BaggageRestrictionManagerProcessor) Process(ctx context.Context, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  name, _, seqId, err2 := iprot.ReadMessageBegin(ctx)
+  if err2 != nil { return false, thrift.WrapTException(err2) }
+  if processor, ok := p.GetProcessorFunction(name); ok {
+    return processor.Process(ctx, seqId, iprot, oprot)
+  }
+  iprot.Skip(ctx, thrift.STRUCT)
+  iprot.ReadMessageEnd(ctx)
+  x4 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function " + name)
+  oprot.WriteMessageBegin(ctx, name, thrift.EXCEPTION, seqId)
+  x4.Write(ctx, oprot)
+  oprot.WriteMessageEnd(ctx)
+  oprot.Flush(ctx)
+  return false, x4
+
+}
+
+type baggageRestrictionManagerProcessorGetBaggageRestrictions struct {
+  handler BaggageRestrictionManager
+}
+
+func (p *baggageRestrictionManagerProcessorGetBaggageRestrictions) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := BaggageRestrictionManagerGetBaggageRestrictionsArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err2.Error())
+    oprot.WriteMessageBegin(ctx, "getBaggageRestrictions", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
+
+  tickerCancel := func() {}
+  // Start a goroutine to do server side connectivity check.
+  if thrift.ServerConnectivityCheckInterval > 0 {
+    var cancel context.CancelFunc
+    ctx, cancel = context.WithCancel(ctx)
+    defer cancel()
+    var tickerCtx context.Context
+    tickerCtx, tickerCancel = context.WithCancel(context.Background())
+    defer tickerCancel()
+    go func(ctx context.Context, cancel context.CancelFunc) {
+      ticker := time.NewTicker(thrift.ServerConnectivityCheckInterval)
+      defer ticker.Stop()
+      for {
+        select {
+        case <-ctx.Done():
+          return
+        case <-ticker.C:
+          if !iprot.Transport().IsOpen() {
+            cancel()
+            return
+          }
+        }
+      }
+    }(tickerCtx, cancel)
+  }
+
+  result := BaggageRestrictionManagerGetBaggageRestrictionsResult{}
+  var retval []*BaggageRestriction
+  if retval, err2 = p.handler.GetBaggageRestrictions(ctx, args.ServiceName); err2 != nil {
+    tickerCancel()
+    if err2 == thrift.ErrAbandonRequest {
+      return false, thrift.WrapTException(err2)
+    }
+    x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getBaggageRestrictions: " + err2.Error())
+    oprot.WriteMessageBegin(ctx, "getBaggageRestrictions", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return true, thrift.WrapTException(err2)
+  } else {
+    result.Success = retval
+  }
+  tickerCancel()
+  if err2 = oprot.WriteMessageBegin(ctx, "getBaggageRestrictions", thrift.REPLY, seqId); err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = result.Write(ctx, oprot); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.WriteMessageEnd(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.Flush(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err != nil {
+    return
+  }
+  return true, err
+}
+
+
+// HELPER FUNCTIONS AND STRUCTURES
+
+// Attributes:
+//  - ServiceName
+type BaggageRestrictionManagerGetBaggageRestrictionsArgs struct {
+  ServiceName string `thrift:"serviceName,1" db:"serviceName" json:"serviceName"`
+}
+
+func NewBaggageRestrictionManagerGetBaggageRestrictionsArgs() *BaggageRestrictionManagerGetBaggageRestrictionsArgs {
+  return &BaggageRestrictionManagerGetBaggageRestrictionsArgs{}
+}
+
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) GetServiceName() string {
+  return p.ServiceName
+}
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.ServiceName = v
+}
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "getBaggageRestrictions_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "serviceName", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.ServiceName)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err) }
+  return err
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("BaggageRestrictionManagerGetBaggageRestrictionsArgs(%+v)", *p)
+}
+
+// Attributes:
+//  - Success
+type BaggageRestrictionManagerGetBaggageRestrictionsResult struct {
+  Success []*BaggageRestriction `thrift:"success,0" db:"success" json:"success,omitempty"`
+}
+
+func NewBaggageRestrictionManagerGetBaggageRestrictionsResult() *BaggageRestrictionManagerGetBaggageRestrictionsResult {
+  return &BaggageRestrictionManagerGetBaggageRestrictionsResult{}
+}
+
+var BaggageRestrictionManagerGetBaggageRestrictionsResult_Success_DEFAULT []*BaggageRestriction
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) GetSuccess() []*BaggageRestriction {
+  return p.Success
+}
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) IsSetSuccess() bool {
+  return p.Success != nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 0:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField0(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult)  ReadField0(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*BaggageRestriction, 0, size)
+  p.Success =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem5 := &BaggageRestriction{}
+    if err := _elem5.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem5), err)
+    }
+    p.Success = append(p.Success, _elem5)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "getBaggageRestrictions_result"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField0(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) writeField0(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetSuccess() {
+    if err := oprot.WriteFieldBegin(ctx, "success", thrift.LIST, 0); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Success)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Success {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) }
+  }
+  return err
+}
+
+func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("BaggageRestrictionManagerGetBaggageRestrictionsResult(%+v)", *p)
+}
+
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggagerestrictionmanager.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggagerestrictionmanager.go
deleted file mode 100644
index 1f79c12..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/baggagerestrictionmanager.go
+++ /dev/null
@@ -1,435 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package baggage
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-type BaggageRestrictionManager interface {
-	// getBaggageRestrictions retrieves the baggage restrictions for a specific service.
-	// Usually, baggageRestrictions apply to all services however there may be situations
-	// where a baggageKey might only be allowed to be set by a specific service.
-	//
-	// Parameters:
-	//  - ServiceName
-	GetBaggageRestrictions(serviceName string) (r []*BaggageRestriction, err error)
-}
-
-type BaggageRestrictionManagerClient struct {
-	Transport       thrift.TTransport
-	ProtocolFactory thrift.TProtocolFactory
-	InputProtocol   thrift.TProtocol
-	OutputProtocol  thrift.TProtocol
-	SeqId           int32
-}
-
-func NewBaggageRestrictionManagerClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *BaggageRestrictionManagerClient {
-	return &BaggageRestrictionManagerClient{Transport: t,
-		ProtocolFactory: f,
-		InputProtocol:   f.GetProtocol(t),
-		OutputProtocol:  f.GetProtocol(t),
-		SeqId:           0,
-	}
-}
-
-func NewBaggageRestrictionManagerClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *BaggageRestrictionManagerClient {
-	return &BaggageRestrictionManagerClient{Transport: t,
-		ProtocolFactory: nil,
-		InputProtocol:   iprot,
-		OutputProtocol:  oprot,
-		SeqId:           0,
-	}
-}
-
-// getBaggageRestrictions retrieves the baggage restrictions for a specific service.
-// Usually, baggageRestrictions apply to all services however there may be situations
-// where a baggageKey might only be allowed to be set by a specific service.
-//
-// Parameters:
-//  - ServiceName
-func (p *BaggageRestrictionManagerClient) GetBaggageRestrictions(serviceName string) (r []*BaggageRestriction, err error) {
-	if err = p.sendGetBaggageRestrictions(serviceName); err != nil {
-		return
-	}
-	return p.recvGetBaggageRestrictions()
-}
-
-func (p *BaggageRestrictionManagerClient) sendGetBaggageRestrictions(serviceName string) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("getBaggageRestrictions", thrift.CALL, p.SeqId); err != nil {
-		return
-	}
-	args := BaggageRestrictionManagerGetBaggageRestrictionsArgs{
-		ServiceName: serviceName,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
-}
-
-func (p *BaggageRestrictionManagerClient) recvGetBaggageRestrictions() (value []*BaggageRestriction, err error) {
-	iprot := p.InputProtocol
-	if iprot == nil {
-		iprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.InputProtocol = iprot
-	}
-	method, mTypeId, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return
-	}
-	if method != "getBaggageRestrictions" {
-		err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getBaggageRestrictions failed: wrong method name")
-		return
-	}
-	if p.SeqId != seqId {
-		err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getBaggageRestrictions failed: out of sequence response")
-		return
-	}
-	if mTypeId == thrift.EXCEPTION {
-		error0 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception")
-		var error1 error
-		error1, err = error0.Read(iprot)
-		if err != nil {
-			return
-		}
-		if err = iprot.ReadMessageEnd(); err != nil {
-			return
-		}
-		err = error1
-		return
-	}
-	if mTypeId != thrift.REPLY {
-		err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getBaggageRestrictions failed: invalid message type")
-		return
-	}
-	result := BaggageRestrictionManagerGetBaggageRestrictionsResult{}
-	if err = result.Read(iprot); err != nil {
-		return
-	}
-	if err = iprot.ReadMessageEnd(); err != nil {
-		return
-	}
-	value = result.GetSuccess()
-	return
-}
-
-type BaggageRestrictionManagerProcessor struct {
-	processorMap map[string]thrift.TProcessorFunction
-	handler      BaggageRestrictionManager
-}
-
-func (p *BaggageRestrictionManagerProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
-	p.processorMap[key] = processor
-}
-
-func (p *BaggageRestrictionManagerProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
-	processor, ok = p.processorMap[key]
-	return processor, ok
-}
-
-func (p *BaggageRestrictionManagerProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
-	return p.processorMap
-}
-
-func NewBaggageRestrictionManagerProcessor(handler BaggageRestrictionManager) *BaggageRestrictionManagerProcessor {
-
-	self2 := &BaggageRestrictionManagerProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
-	self2.processorMap["getBaggageRestrictions"] = &baggageRestrictionManagerProcessorGetBaggageRestrictions{handler: handler}
-	return self2
-}
-
-func (p *BaggageRestrictionManagerProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	name, _, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return false, err
-	}
-	if processor, ok := p.GetProcessorFunction(name); ok {
-		return processor.Process(seqId, iprot, oprot)
-	}
-	iprot.Skip(thrift.STRUCT)
-	iprot.ReadMessageEnd()
-	x3 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
-	oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
-	x3.Write(oprot)
-	oprot.WriteMessageEnd()
-	oprot.Flush()
-	return false, x3
-
-}
-
-type baggageRestrictionManagerProcessorGetBaggageRestrictions struct {
-	handler BaggageRestrictionManager
-}
-
-func (p *baggageRestrictionManagerProcessorGetBaggageRestrictions) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := BaggageRestrictionManagerGetBaggageRestrictionsArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error())
-		oprot.WriteMessageBegin("getBaggageRestrictions", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return false, err
-	}
-
-	iprot.ReadMessageEnd()
-	result := BaggageRestrictionManagerGetBaggageRestrictionsResult{}
-	var retval []*BaggageRestriction
-	var err2 error
-	if retval, err2 = p.handler.GetBaggageRestrictions(args.ServiceName); err2 != nil {
-		x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getBaggageRestrictions: "+err2.Error())
-		oprot.WriteMessageBegin("getBaggageRestrictions", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return true, err2
-	} else {
-		result.Success = retval
-	}
-	if err2 = oprot.WriteMessageBegin("getBaggageRestrictions", thrift.REPLY, seqId); err2 != nil {
-		err = err2
-	}
-	if err2 = result.Write(oprot); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.Flush(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err != nil {
-		return
-	}
-	return true, err
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
-
-// Attributes:
-//  - ServiceName
-type BaggageRestrictionManagerGetBaggageRestrictionsArgs struct {
-	ServiceName string `thrift:"serviceName,1" json:"serviceName"`
-}
-
-func NewBaggageRestrictionManagerGetBaggageRestrictionsArgs() *BaggageRestrictionManagerGetBaggageRestrictionsArgs {
-	return &BaggageRestrictionManagerGetBaggageRestrictionsArgs{}
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) GetServiceName() string {
-	return p.ServiceName
-}
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.ServiceName = v
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("getBaggageRestrictions_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("serviceName", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.ServiceName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err)
-	}
-	return err
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BaggageRestrictionManagerGetBaggageRestrictionsArgs(%+v)", *p)
-}
-
-// Attributes:
-//  - Success
-type BaggageRestrictionManagerGetBaggageRestrictionsResult struct {
-	Success []*BaggageRestriction `thrift:"success,0" json:"success,omitempty"`
-}
-
-func NewBaggageRestrictionManagerGetBaggageRestrictionsResult() *BaggageRestrictionManagerGetBaggageRestrictionsResult {
-	return &BaggageRestrictionManagerGetBaggageRestrictionsResult{}
-}
-
-var BaggageRestrictionManagerGetBaggageRestrictionsResult_Success_DEFAULT []*BaggageRestriction
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) GetSuccess() []*BaggageRestriction {
-	return p.Success
-}
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) IsSetSuccess() bool {
-	return p.Success != nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 0:
-			if err := p.readField0(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) readField0(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*BaggageRestriction, 0, size)
-	p.Success = tSlice
-	for i := 0; i < size; i++ {
-		_elem4 := &BaggageRestriction{}
-		if err := _elem4.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err)
-		}
-		p.Success = append(p.Success, _elem4)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("getBaggageRestrictions_result"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField0(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) writeField0(oprot thrift.TProtocol) (err error) {
-	if p.IsSetSuccess() {
-		if err := oprot.WriteFieldBegin("success", thrift.LIST, 0); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Success)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Success {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *BaggageRestrictionManagerGetBaggageRestrictionsResult) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BaggageRestrictionManagerGetBaggageRestrictionsResult(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/constants.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/constants.go
deleted file mode 100644
index ed35ce9..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/constants.go
+++ /dev/null
@@ -1,18 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package baggage
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-func init() {
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/ttypes.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/ttypes.go
deleted file mode 100644
index 7888892..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/baggage/ttypes.go
+++ /dev/null
@@ -1,154 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package baggage
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var GoUnusedProtection__ int
-
-// Attributes:
-//  - BaggageKey
-//  - MaxValueLength
-type BaggageRestriction struct {
-	BaggageKey     string `thrift:"baggageKey,1,required" json:"baggageKey"`
-	MaxValueLength int32  `thrift:"maxValueLength,2,required" json:"maxValueLength"`
-}
-
-func NewBaggageRestriction() *BaggageRestriction {
-	return &BaggageRestriction{}
-}
-
-func (p *BaggageRestriction) GetBaggageKey() string {
-	return p.BaggageKey
-}
-
-func (p *BaggageRestriction) GetMaxValueLength() int32 {
-	return p.MaxValueLength
-}
-func (p *BaggageRestriction) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetBaggageKey bool = false
-	var issetMaxValueLength bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetBaggageKey = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetMaxValueLength = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetBaggageKey {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field BaggageKey is not set"))
-	}
-	if !issetMaxValueLength {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field MaxValueLength is not set"))
-	}
-	return nil
-}
-
-func (p *BaggageRestriction) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.BaggageKey = v
-	}
-	return nil
-}
-
-func (p *BaggageRestriction) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.MaxValueLength = v
-	}
-	return nil
-}
-
-func (p *BaggageRestriction) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("BaggageRestriction"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *BaggageRestriction) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("baggageKey", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:baggageKey: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.BaggageKey)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.baggageKey (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:baggageKey: ", p), err)
-	}
-	return err
-}
-
-func (p *BaggageRestriction) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("maxValueLength", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:maxValueLength: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.MaxValueLength)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.maxValueLength (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:maxValueLength: ", p), err)
-	}
-	return err
-}
-
-func (p *BaggageRestriction) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BaggageRestriction(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/GoUnusedProtection__.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/GoUnusedProtection__.go
new file mode 100644
index 0000000..fe45a9f
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/GoUnusedProtection__.go
@@ -0,0 +1,6 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package jaeger
+
+var GoUnusedProtection__ int;
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/agent.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/agent.go
deleted file mode 100644
index b32c37d..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/agent.go
+++ /dev/null
@@ -1,242 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package jaeger
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-type Agent interface {
-	// Parameters:
-	//  - Batch
-	EmitBatch(batch *Batch) (err error)
-}
-
-type AgentClient struct {
-	Transport       thrift.TTransport
-	ProtocolFactory thrift.TProtocolFactory
-	InputProtocol   thrift.TProtocol
-	OutputProtocol  thrift.TProtocol
-	SeqId           int32
-}
-
-func NewAgentClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *AgentClient {
-	return &AgentClient{Transport: t,
-		ProtocolFactory: f,
-		InputProtocol:   f.GetProtocol(t),
-		OutputProtocol:  f.GetProtocol(t),
-		SeqId:           0,
-	}
-}
-
-func NewAgentClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *AgentClient {
-	return &AgentClient{Transport: t,
-		ProtocolFactory: nil,
-		InputProtocol:   iprot,
-		OutputProtocol:  oprot,
-		SeqId:           0,
-	}
-}
-
-// Parameters:
-//  - Batch
-func (p *AgentClient) EmitBatch(batch *Batch) (err error) {
-	if err = p.sendEmitBatch(batch); err != nil {
-		return
-	}
-	return
-}
-
-func (p *AgentClient) sendEmitBatch(batch *Batch) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("emitBatch", thrift.ONEWAY, p.SeqId); err != nil {
-		return
-	}
-	args := AgentEmitBatchArgs{
-		Batch: batch,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
-}
-
-type AgentProcessor struct {
-	processorMap map[string]thrift.TProcessorFunction
-	handler      Agent
-}
-
-func (p *AgentProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
-	p.processorMap[key] = processor
-}
-
-func (p *AgentProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
-	processor, ok = p.processorMap[key]
-	return processor, ok
-}
-
-func (p *AgentProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
-	return p.processorMap
-}
-
-func NewAgentProcessor(handler Agent) *AgentProcessor {
-
-	self6 := &AgentProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
-	self6.processorMap["emitBatch"] = &agentProcessorEmitBatch{handler: handler}
-	return self6
-}
-
-func (p *AgentProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	name, _, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return false, err
-	}
-	if processor, ok := p.GetProcessorFunction(name); ok {
-		return processor.Process(seqId, iprot, oprot)
-	}
-	iprot.Skip(thrift.STRUCT)
-	iprot.ReadMessageEnd()
-	x7 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
-	oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
-	x7.Write(oprot)
-	oprot.WriteMessageEnd()
-	oprot.Flush()
-	return false, x7
-
-}
-
-type agentProcessorEmitBatch struct {
-	handler Agent
-}
-
-func (p *agentProcessorEmitBatch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := AgentEmitBatchArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		return false, err
-	}
-
-	iprot.ReadMessageEnd()
-	var err2 error
-	if err2 = p.handler.EmitBatch(args.Batch); err2 != nil {
-		return true, err2
-	}
-	return true, nil
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
-
-// Attributes:
-//  - Batch
-type AgentEmitBatchArgs struct {
-	Batch *Batch `thrift:"batch,1" json:"batch"`
-}
-
-func NewAgentEmitBatchArgs() *AgentEmitBatchArgs {
-	return &AgentEmitBatchArgs{}
-}
-
-var AgentEmitBatchArgs_Batch_DEFAULT *Batch
-
-func (p *AgentEmitBatchArgs) GetBatch() *Batch {
-	if !p.IsSetBatch() {
-		return AgentEmitBatchArgs_Batch_DEFAULT
-	}
-	return p.Batch
-}
-func (p *AgentEmitBatchArgs) IsSetBatch() bool {
-	return p.Batch != nil
-}
-
-func (p *AgentEmitBatchArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *AgentEmitBatchArgs) readField1(iprot thrift.TProtocol) error {
-	p.Batch = &Batch{}
-	if err := p.Batch.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Batch), err)
-	}
-	return nil
-}
-
-func (p *AgentEmitBatchArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("emitBatch_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *AgentEmitBatchArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("batch", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:batch: ", p), err)
-	}
-	if err := p.Batch.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Batch), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:batch: ", p), err)
-	}
-	return err
-}
-
-func (p *AgentEmitBatchArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("AgentEmitBatchArgs(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/constants.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/constants.go
deleted file mode 100644
index 621b8b1..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/constants.go
+++ /dev/null
@@ -1,18 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package jaeger
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-func init() {
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger-consts.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger-consts.go
new file mode 100644
index 0000000..b6ce855
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger-consts.go
@@ -0,0 +1,23 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package jaeger
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+
+func init() {
+}
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger.go
new file mode 100644
index 0000000..d55cca0
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/jaeger.go
@@ -0,0 +1,2698 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package jaeger
+
+import(
+	"bytes"
+	"context"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+type TagType int64
+const (
+  TagType_STRING TagType = 0
+  TagType_DOUBLE TagType = 1
+  TagType_BOOL TagType = 2
+  TagType_LONG TagType = 3
+  TagType_BINARY TagType = 4
+)
+
+func (p TagType) String() string {
+  switch p {
+  case TagType_STRING: return "STRING"
+  case TagType_DOUBLE: return "DOUBLE"
+  case TagType_BOOL: return "BOOL"
+  case TagType_LONG: return "LONG"
+  case TagType_BINARY: return "BINARY"
+  }
+  return "<UNSET>"
+}
+
+func TagTypeFromString(s string) (TagType, error) {
+  switch s {
+  case "STRING": return TagType_STRING, nil 
+  case "DOUBLE": return TagType_DOUBLE, nil 
+  case "BOOL": return TagType_BOOL, nil 
+  case "LONG": return TagType_LONG, nil 
+  case "BINARY": return TagType_BINARY, nil 
+  }
+  return TagType(0), fmt.Errorf("not a valid TagType string")
+}
+
+
+func TagTypePtr(v TagType) *TagType { return &v }
+
+func (p TagType) MarshalText() ([]byte, error) {
+return []byte(p.String()), nil
+}
+
+func (p *TagType) UnmarshalText(text []byte) error {
+q, err := TagTypeFromString(string(text))
+if (err != nil) {
+return err
+}
+*p = q
+return nil
+}
+
+func (p *TagType) Scan(value interface{}) error {
+v, ok := value.(int64)
+if !ok {
+return errors.New("Scan value is not int64")
+}
+*p = TagType(v)
+return nil
+}
+
+func (p * TagType) Value() (driver.Value, error) {
+  if p == nil {
+    return nil, nil
+  }
+return int64(*p), nil
+}
+type SpanRefType int64
+const (
+  SpanRefType_CHILD_OF SpanRefType = 0
+  SpanRefType_FOLLOWS_FROM SpanRefType = 1
+)
+
+func (p SpanRefType) String() string {
+  switch p {
+  case SpanRefType_CHILD_OF: return "CHILD_OF"
+  case SpanRefType_FOLLOWS_FROM: return "FOLLOWS_FROM"
+  }
+  return "<UNSET>"
+}
+
+func SpanRefTypeFromString(s string) (SpanRefType, error) {
+  switch s {
+  case "CHILD_OF": return SpanRefType_CHILD_OF, nil 
+  case "FOLLOWS_FROM": return SpanRefType_FOLLOWS_FROM, nil 
+  }
+  return SpanRefType(0), fmt.Errorf("not a valid SpanRefType string")
+}
+
+
+func SpanRefTypePtr(v SpanRefType) *SpanRefType { return &v }
+
+func (p SpanRefType) MarshalText() ([]byte, error) {
+return []byte(p.String()), nil
+}
+
+func (p *SpanRefType) UnmarshalText(text []byte) error {
+q, err := SpanRefTypeFromString(string(text))
+if (err != nil) {
+return err
+}
+*p = q
+return nil
+}
+
+func (p *SpanRefType) Scan(value interface{}) error {
+v, ok := value.(int64)
+if !ok {
+return errors.New("Scan value is not int64")
+}
+*p = SpanRefType(v)
+return nil
+}
+
+func (p * SpanRefType) Value() (driver.Value, error) {
+  if p == nil {
+    return nil, nil
+  }
+return int64(*p), nil
+}
+// Attributes:
+//  - Key
+//  - VType
+//  - VStr
+//  - VDouble
+//  - VBool
+//  - VLong
+//  - VBinary
+type Tag struct {
+  Key string `thrift:"key,1,required" db:"key" json:"key"`
+  VType TagType `thrift:"vType,2,required" db:"vType" json:"vType"`
+  VStr *string `thrift:"vStr,3" db:"vStr" json:"vStr,omitempty"`
+  VDouble *float64 `thrift:"vDouble,4" db:"vDouble" json:"vDouble,omitempty"`
+  VBool *bool `thrift:"vBool,5" db:"vBool" json:"vBool,omitempty"`
+  VLong *int64 `thrift:"vLong,6" db:"vLong" json:"vLong,omitempty"`
+  VBinary []byte `thrift:"vBinary,7" db:"vBinary" json:"vBinary,omitempty"`
+}
+
+func NewTag() *Tag {
+  return &Tag{}
+}
+
+
+func (p *Tag) GetKey() string {
+  return p.Key
+}
+
+func (p *Tag) GetVType() TagType {
+  return p.VType
+}
+var Tag_VStr_DEFAULT string
+func (p *Tag) GetVStr() string {
+  if !p.IsSetVStr() {
+    return Tag_VStr_DEFAULT
+  }
+return *p.VStr
+}
+var Tag_VDouble_DEFAULT float64
+func (p *Tag) GetVDouble() float64 {
+  if !p.IsSetVDouble() {
+    return Tag_VDouble_DEFAULT
+  }
+return *p.VDouble
+}
+var Tag_VBool_DEFAULT bool
+func (p *Tag) GetVBool() bool {
+  if !p.IsSetVBool() {
+    return Tag_VBool_DEFAULT
+  }
+return *p.VBool
+}
+var Tag_VLong_DEFAULT int64
+func (p *Tag) GetVLong() int64 {
+  if !p.IsSetVLong() {
+    return Tag_VLong_DEFAULT
+  }
+return *p.VLong
+}
+var Tag_VBinary_DEFAULT []byte
+
+func (p *Tag) GetVBinary() []byte {
+  return p.VBinary
+}
+func (p *Tag) IsSetVStr() bool {
+  return p.VStr != nil
+}
+
+func (p *Tag) IsSetVDouble() bool {
+  return p.VDouble != nil
+}
+
+func (p *Tag) IsSetVBool() bool {
+  return p.VBool != nil
+}
+
+func (p *Tag) IsSetVLong() bool {
+  return p.VLong != nil
+}
+
+func (p *Tag) IsSetVBinary() bool {
+  return p.VBinary != nil
+}
+
+func (p *Tag) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetKey bool = false;
+  var issetVType bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetKey = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetVType = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.DOUBLE {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 5:
+      if fieldTypeId == thrift.BOOL {
+        if err := p.ReadField5(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 6:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField6(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 7:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField7(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetKey{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Key is not set"));
+  }
+  if !issetVType{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field VType is not set"));
+  }
+  return nil
+}
+
+func (p *Tag)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Key = v
+}
+  return nil
+}
+
+func (p *Tag)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  temp := TagType(v)
+  p.VType = temp
+}
+  return nil
+}
+
+func (p *Tag)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.VStr = &v
+}
+  return nil
+}
+
+func (p *Tag)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadDouble(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.VDouble = &v
+}
+  return nil
+}
+
+func (p *Tag)  ReadField5(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBool(ctx); err != nil {
+  return thrift.PrependError("error reading field 5: ", err)
+} else {
+  p.VBool = &v
+}
+  return nil
+}
+
+func (p *Tag)  ReadField6(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 6: ", err)
+} else {
+  p.VLong = &v
+}
+  return nil
+}
+
+func (p *Tag)  ReadField7(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBinary(ctx); err != nil {
+  return thrift.PrependError("error reading field 7: ", err)
+} else {
+  p.VBinary = v
+}
+  return nil
+}
+
+func (p *Tag) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Tag"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+    if err := p.writeField5(ctx, oprot); err != nil { return err }
+    if err := p.writeField6(ctx, oprot); err != nil { return err }
+    if err := p.writeField7(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Tag) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "key", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:key: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.Key)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.key (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:key: ", p), err) }
+  return err
+}
+
+func (p *Tag) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "vType", thrift.I32, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:vType: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.VType)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.vType (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:vType: ", p), err) }
+  return err
+}
+
+func (p *Tag) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetVStr() {
+    if err := oprot.WriteFieldBegin(ctx, "vStr", thrift.STRING, 3); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:vStr: ", p), err) }
+    if err := oprot.WriteString(ctx, string(*p.VStr)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.vStr (3) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 3:vStr: ", p), err) }
+  }
+  return err
+}
+
+func (p *Tag) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetVDouble() {
+    if err := oprot.WriteFieldBegin(ctx, "vDouble", thrift.DOUBLE, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:vDouble: ", p), err) }
+    if err := oprot.WriteDouble(ctx, float64(*p.VDouble)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.vDouble (4) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:vDouble: ", p), err) }
+  }
+  return err
+}
+
+func (p *Tag) writeField5(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetVBool() {
+    if err := oprot.WriteFieldBegin(ctx, "vBool", thrift.BOOL, 5); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:vBool: ", p), err) }
+    if err := oprot.WriteBool(ctx, bool(*p.VBool)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.vBool (5) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 5:vBool: ", p), err) }
+  }
+  return err
+}
+
+func (p *Tag) writeField6(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetVLong() {
+    if err := oprot.WriteFieldBegin(ctx, "vLong", thrift.I64, 6); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:vLong: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.VLong)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.vLong (6) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 6:vLong: ", p), err) }
+  }
+  return err
+}
+
+func (p *Tag) writeField7(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetVBinary() {
+    if err := oprot.WriteFieldBegin(ctx, "vBinary", thrift.STRING, 7); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:vBinary: ", p), err) }
+    if err := oprot.WriteBinary(ctx, p.VBinary); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.vBinary (7) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 7:vBinary: ", p), err) }
+  }
+  return err
+}
+
+func (p *Tag) Equals(other *Tag) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Key != other.Key { return false }
+  if p.VType != other.VType { return false }
+  if p.VStr != other.VStr {
+    if p.VStr == nil || other.VStr == nil {
+      return false
+    }
+    if (*p.VStr) != (*other.VStr) { return false }
+  }
+  if p.VDouble != other.VDouble {
+    if p.VDouble == nil || other.VDouble == nil {
+      return false
+    }
+    if (*p.VDouble) != (*other.VDouble) { return false }
+  }
+  if p.VBool != other.VBool {
+    if p.VBool == nil || other.VBool == nil {
+      return false
+    }
+    if (*p.VBool) != (*other.VBool) { return false }
+  }
+  if p.VLong != other.VLong {
+    if p.VLong == nil || other.VLong == nil {
+      return false
+    }
+    if (*p.VLong) != (*other.VLong) { return false }
+  }
+  if bytes.Compare(p.VBinary, other.VBinary) != 0 { return false }
+  return true
+}
+
+func (p *Tag) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Tag(%+v)", *p)
+}
+
+// Attributes:
+//  - Timestamp
+//  - Fields
+type Log struct {
+  Timestamp int64 `thrift:"timestamp,1,required" db:"timestamp" json:"timestamp"`
+  Fields []*Tag `thrift:"fields,2,required" db:"fields" json:"fields"`
+}
+
+func NewLog() *Log {
+  return &Log{}
+}
+
+
+func (p *Log) GetTimestamp() int64 {
+  return p.Timestamp
+}
+
+func (p *Log) GetFields() []*Tag {
+  return p.Fields
+}
+func (p *Log) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetTimestamp bool = false;
+  var issetFields bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetTimestamp = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetFields = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetTimestamp{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Timestamp is not set"));
+  }
+  if !issetFields{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Fields is not set"));
+  }
+  return nil
+}
+
+func (p *Log)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Timestamp = v
+}
+  return nil
+}
+
+func (p *Log)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Tag, 0, size)
+  p.Fields =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem0 := &Tag{}
+    if err := _elem0.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
+    }
+    p.Fields = append(p.Fields, _elem0)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Log) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Log"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Log) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "timestamp", thrift.I64, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:timestamp: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.Timestamp)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.timestamp (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:timestamp: ", p), err) }
+  return err
+}
+
+func (p *Log) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "fields", thrift.LIST, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:fields: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Fields)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Fields {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:fields: ", p), err) }
+  return err
+}
+
+func (p *Log) Equals(other *Log) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Timestamp != other.Timestamp { return false }
+  if len(p.Fields) != len(other.Fields) { return false }
+  for i, _tgt := range p.Fields {
+    _src1 := other.Fields[i]
+    if !_tgt.Equals(_src1) { return false }
+  }
+  return true
+}
+
+func (p *Log) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Log(%+v)", *p)
+}
+
+// Attributes:
+//  - RefType
+//  - TraceIdLow
+//  - TraceIdHigh
+//  - SpanId
+type SpanRef struct {
+  RefType SpanRefType `thrift:"refType,1,required" db:"refType" json:"refType"`
+  TraceIdLow int64 `thrift:"traceIdLow,2,required" db:"traceIdLow" json:"traceIdLow"`
+  TraceIdHigh int64 `thrift:"traceIdHigh,3,required" db:"traceIdHigh" json:"traceIdHigh"`
+  SpanId int64 `thrift:"spanId,4,required" db:"spanId" json:"spanId"`
+}
+
+func NewSpanRef() *SpanRef {
+  return &SpanRef{}
+}
+
+
+func (p *SpanRef) GetRefType() SpanRefType {
+  return p.RefType
+}
+
+func (p *SpanRef) GetTraceIdLow() int64 {
+  return p.TraceIdLow
+}
+
+func (p *SpanRef) GetTraceIdHigh() int64 {
+  return p.TraceIdHigh
+}
+
+func (p *SpanRef) GetSpanId() int64 {
+  return p.SpanId
+}
+func (p *SpanRef) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetRefType bool = false;
+  var issetTraceIdLow bool = false;
+  var issetTraceIdHigh bool = false;
+  var issetSpanId bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetRefType = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetTraceIdLow = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+        issetTraceIdHigh = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+        issetSpanId = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetRefType{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RefType is not set"));
+  }
+  if !issetTraceIdLow{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdLow is not set"));
+  }
+  if !issetTraceIdHigh{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdHigh is not set"));
+  }
+  if !issetSpanId{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SpanId is not set"));
+  }
+  return nil
+}
+
+func (p *SpanRef)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  temp := SpanRefType(v)
+  p.RefType = temp
+}
+  return nil
+}
+
+func (p *SpanRef)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.TraceIdLow = v
+}
+  return nil
+}
+
+func (p *SpanRef)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.TraceIdHigh = v
+}
+  return nil
+}
+
+func (p *SpanRef)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.SpanId = v
+}
+  return nil
+}
+
+func (p *SpanRef) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "SpanRef"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *SpanRef) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "refType", thrift.I32, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:refType: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.RefType)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.refType (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:refType: ", p), err) }
+  return err
+}
+
+func (p *SpanRef) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "traceIdLow", thrift.I64, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:traceIdLow: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TraceIdLow)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.traceIdLow (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:traceIdLow: ", p), err) }
+  return err
+}
+
+func (p *SpanRef) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "traceIdHigh", thrift.I64, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:traceIdHigh: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TraceIdHigh)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.traceIdHigh (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:traceIdHigh: ", p), err) }
+  return err
+}
+
+func (p *SpanRef) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "spanId", thrift.I64, 4); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:spanId: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.SpanId)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.spanId (4) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 4:spanId: ", p), err) }
+  return err
+}
+
+func (p *SpanRef) Equals(other *SpanRef) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.RefType != other.RefType { return false }
+  if p.TraceIdLow != other.TraceIdLow { return false }
+  if p.TraceIdHigh != other.TraceIdHigh { return false }
+  if p.SpanId != other.SpanId { return false }
+  return true
+}
+
+func (p *SpanRef) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("SpanRef(%+v)", *p)
+}
+
+// Attributes:
+//  - TraceIdLow
+//  - TraceIdHigh
+//  - SpanId
+//  - ParentSpanId
+//  - OperationName
+//  - References
+//  - Flags
+//  - StartTime
+//  - Duration
+//  - Tags
+//  - Logs
+type Span struct {
+  TraceIdLow int64 `thrift:"traceIdLow,1,required" db:"traceIdLow" json:"traceIdLow"`
+  TraceIdHigh int64 `thrift:"traceIdHigh,2,required" db:"traceIdHigh" json:"traceIdHigh"`
+  SpanId int64 `thrift:"spanId,3,required" db:"spanId" json:"spanId"`
+  ParentSpanId int64 `thrift:"parentSpanId,4,required" db:"parentSpanId" json:"parentSpanId"`
+  OperationName string `thrift:"operationName,5,required" db:"operationName" json:"operationName"`
+  References []*SpanRef `thrift:"references,6" db:"references" json:"references,omitempty"`
+  Flags int32 `thrift:"flags,7,required" db:"flags" json:"flags"`
+  StartTime int64 `thrift:"startTime,8,required" db:"startTime" json:"startTime"`
+  Duration int64 `thrift:"duration,9,required" db:"duration" json:"duration"`
+  Tags []*Tag `thrift:"tags,10" db:"tags" json:"tags,omitempty"`
+  Logs []*Log `thrift:"logs,11" db:"logs" json:"logs,omitempty"`
+}
+
+func NewSpan() *Span {
+  return &Span{}
+}
+
+
+func (p *Span) GetTraceIdLow() int64 {
+  return p.TraceIdLow
+}
+
+func (p *Span) GetTraceIdHigh() int64 {
+  return p.TraceIdHigh
+}
+
+func (p *Span) GetSpanId() int64 {
+  return p.SpanId
+}
+
+func (p *Span) GetParentSpanId() int64 {
+  return p.ParentSpanId
+}
+
+func (p *Span) GetOperationName() string {
+  return p.OperationName
+}
+var Span_References_DEFAULT []*SpanRef
+
+func (p *Span) GetReferences() []*SpanRef {
+  return p.References
+}
+
+func (p *Span) GetFlags() int32 {
+  return p.Flags
+}
+
+func (p *Span) GetStartTime() int64 {
+  return p.StartTime
+}
+
+func (p *Span) GetDuration() int64 {
+  return p.Duration
+}
+var Span_Tags_DEFAULT []*Tag
+
+func (p *Span) GetTags() []*Tag {
+  return p.Tags
+}
+var Span_Logs_DEFAULT []*Log
+
+func (p *Span) GetLogs() []*Log {
+  return p.Logs
+}
+func (p *Span) IsSetReferences() bool {
+  return p.References != nil
+}
+
+func (p *Span) IsSetTags() bool {
+  return p.Tags != nil
+}
+
+func (p *Span) IsSetLogs() bool {
+  return p.Logs != nil
+}
+
+func (p *Span) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetTraceIdLow bool = false;
+  var issetTraceIdHigh bool = false;
+  var issetSpanId bool = false;
+  var issetParentSpanId bool = false;
+  var issetOperationName bool = false;
+  var issetFlags bool = false;
+  var issetStartTime bool = false;
+  var issetDuration bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetTraceIdLow = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetTraceIdHigh = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+        issetSpanId = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+        issetParentSpanId = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 5:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField5(ctx, iprot); err != nil {
+          return err
+        }
+        issetOperationName = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 6:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField6(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 7:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField7(ctx, iprot); err != nil {
+          return err
+        }
+        issetFlags = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 8:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField8(ctx, iprot); err != nil {
+          return err
+        }
+        issetStartTime = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 9:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField9(ctx, iprot); err != nil {
+          return err
+        }
+        issetDuration = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 10:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField10(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 11:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField11(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetTraceIdLow{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdLow is not set"));
+  }
+  if !issetTraceIdHigh{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdHigh is not set"));
+  }
+  if !issetSpanId{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SpanId is not set"));
+  }
+  if !issetParentSpanId{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ParentSpanId is not set"));
+  }
+  if !issetOperationName{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field OperationName is not set"));
+  }
+  if !issetFlags{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Flags is not set"));
+  }
+  if !issetStartTime{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field StartTime is not set"));
+  }
+  if !issetDuration{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Duration is not set"));
+  }
+  return nil
+}
+
+func (p *Span)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.TraceIdLow = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.TraceIdHigh = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.SpanId = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.ParentSpanId = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField5(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 5: ", err)
+} else {
+  p.OperationName = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField6(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*SpanRef, 0, size)
+  p.References =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem2 := &SpanRef{}
+    if err := _elem2.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem2), err)
+    }
+    p.References = append(p.References, _elem2)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Span)  ReadField7(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 7: ", err)
+} else {
+  p.Flags = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField8(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 8: ", err)
+} else {
+  p.StartTime = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField9(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 9: ", err)
+} else {
+  p.Duration = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField10(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Tag, 0, size)
+  p.Tags =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem3 := &Tag{}
+    if err := _elem3.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem3), err)
+    }
+    p.Tags = append(p.Tags, _elem3)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Span)  ReadField11(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Log, 0, size)
+  p.Logs =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem4 := &Log{}
+    if err := _elem4.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err)
+    }
+    p.Logs = append(p.Logs, _elem4)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Span) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Span"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+    if err := p.writeField5(ctx, oprot); err != nil { return err }
+    if err := p.writeField6(ctx, oprot); err != nil { return err }
+    if err := p.writeField7(ctx, oprot); err != nil { return err }
+    if err := p.writeField8(ctx, oprot); err != nil { return err }
+    if err := p.writeField9(ctx, oprot); err != nil { return err }
+    if err := p.writeField10(ctx, oprot); err != nil { return err }
+    if err := p.writeField11(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Span) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "traceIdLow", thrift.I64, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:traceIdLow: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TraceIdLow)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.traceIdLow (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:traceIdLow: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "traceIdHigh", thrift.I64, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:traceIdHigh: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TraceIdHigh)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.traceIdHigh (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:traceIdHigh: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "spanId", thrift.I64, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:spanId: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.SpanId)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.spanId (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:spanId: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "parentSpanId", thrift.I64, 4); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:parentSpanId: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.ParentSpanId)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.parentSpanId (4) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 4:parentSpanId: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField5(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "operationName", thrift.STRING, 5); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:operationName: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.OperationName)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.operationName (5) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 5:operationName: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField6(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetReferences() {
+    if err := oprot.WriteFieldBegin(ctx, "references", thrift.LIST, 6); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:references: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.References)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.References {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 6:references: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField7(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "flags", thrift.I32, 7); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:flags: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.Flags)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.flags (7) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 7:flags: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField8(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "startTime", thrift.I64, 8); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:startTime: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.StartTime)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.startTime (8) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 8:startTime: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField9(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "duration", thrift.I64, 9); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 9:duration: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.Duration)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.duration (9) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 9:duration: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField10(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetTags() {
+    if err := oprot.WriteFieldBegin(ctx, "tags", thrift.LIST, 10); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 10:tags: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Tags)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Tags {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 10:tags: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField11(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetLogs() {
+    if err := oprot.WriteFieldBegin(ctx, "logs", thrift.LIST, 11); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 11:logs: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Logs)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Logs {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 11:logs: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) Equals(other *Span) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.TraceIdLow != other.TraceIdLow { return false }
+  if p.TraceIdHigh != other.TraceIdHigh { return false }
+  if p.SpanId != other.SpanId { return false }
+  if p.ParentSpanId != other.ParentSpanId { return false }
+  if p.OperationName != other.OperationName { return false }
+  if len(p.References) != len(other.References) { return false }
+  for i, _tgt := range p.References {
+    _src5 := other.References[i]
+    if !_tgt.Equals(_src5) { return false }
+  }
+  if p.Flags != other.Flags { return false }
+  if p.StartTime != other.StartTime { return false }
+  if p.Duration != other.Duration { return false }
+  if len(p.Tags) != len(other.Tags) { return false }
+  for i, _tgt := range p.Tags {
+    _src6 := other.Tags[i]
+    if !_tgt.Equals(_src6) { return false }
+  }
+  if len(p.Logs) != len(other.Logs) { return false }
+  for i, _tgt := range p.Logs {
+    _src7 := other.Logs[i]
+    if !_tgt.Equals(_src7) { return false }
+  }
+  return true
+}
+
+func (p *Span) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Span(%+v)", *p)
+}
+
+// Attributes:
+//  - ServiceName
+//  - Tags
+type Process struct {
+  ServiceName string `thrift:"serviceName,1,required" db:"serviceName" json:"serviceName"`
+  Tags []*Tag `thrift:"tags,2" db:"tags" json:"tags,omitempty"`
+}
+
+func NewProcess() *Process {
+  return &Process{}
+}
+
+
+func (p *Process) GetServiceName() string {
+  return p.ServiceName
+}
+var Process_Tags_DEFAULT []*Tag
+
+func (p *Process) GetTags() []*Tag {
+  return p.Tags
+}
+func (p *Process) IsSetTags() bool {
+  return p.Tags != nil
+}
+
+func (p *Process) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetServiceName bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetServiceName = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetServiceName{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ServiceName is not set"));
+  }
+  return nil
+}
+
+func (p *Process)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.ServiceName = v
+}
+  return nil
+}
+
+func (p *Process)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Tag, 0, size)
+  p.Tags =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem8 := &Tag{}
+    if err := _elem8.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem8), err)
+    }
+    p.Tags = append(p.Tags, _elem8)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Process) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Process"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Process) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "serviceName", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.ServiceName)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err) }
+  return err
+}
+
+func (p *Process) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetTags() {
+    if err := oprot.WriteFieldBegin(ctx, "tags", thrift.LIST, 2); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tags: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Tags)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Tags {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tags: ", p), err) }
+  }
+  return err
+}
+
+func (p *Process) Equals(other *Process) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.ServiceName != other.ServiceName { return false }
+  if len(p.Tags) != len(other.Tags) { return false }
+  for i, _tgt := range p.Tags {
+    _src9 := other.Tags[i]
+    if !_tgt.Equals(_src9) { return false }
+  }
+  return true
+}
+
+func (p *Process) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Process(%+v)", *p)
+}
+
+// Attributes:
+//  - FullQueueDroppedSpans
+//  - TooLargeDroppedSpans
+//  - FailedToEmitSpans
+type ClientStats struct {
+  FullQueueDroppedSpans int64 `thrift:"fullQueueDroppedSpans,1,required" db:"fullQueueDroppedSpans" json:"fullQueueDroppedSpans"`
+  TooLargeDroppedSpans int64 `thrift:"tooLargeDroppedSpans,2,required" db:"tooLargeDroppedSpans" json:"tooLargeDroppedSpans"`
+  FailedToEmitSpans int64 `thrift:"failedToEmitSpans,3,required" db:"failedToEmitSpans" json:"failedToEmitSpans"`
+}
+
+func NewClientStats() *ClientStats {
+  return &ClientStats{}
+}
+
+
+func (p *ClientStats) GetFullQueueDroppedSpans() int64 {
+  return p.FullQueueDroppedSpans
+}
+
+func (p *ClientStats) GetTooLargeDroppedSpans() int64 {
+  return p.TooLargeDroppedSpans
+}
+
+func (p *ClientStats) GetFailedToEmitSpans() int64 {
+  return p.FailedToEmitSpans
+}
+func (p *ClientStats) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetFullQueueDroppedSpans bool = false;
+  var issetTooLargeDroppedSpans bool = false;
+  var issetFailedToEmitSpans bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetFullQueueDroppedSpans = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetTooLargeDroppedSpans = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+        issetFailedToEmitSpans = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetFullQueueDroppedSpans{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field FullQueueDroppedSpans is not set"));
+  }
+  if !issetTooLargeDroppedSpans{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TooLargeDroppedSpans is not set"));
+  }
+  if !issetFailedToEmitSpans{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field FailedToEmitSpans is not set"));
+  }
+  return nil
+}
+
+func (p *ClientStats)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.FullQueueDroppedSpans = v
+}
+  return nil
+}
+
+func (p *ClientStats)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.TooLargeDroppedSpans = v
+}
+  return nil
+}
+
+func (p *ClientStats)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.FailedToEmitSpans = v
+}
+  return nil
+}
+
+func (p *ClientStats) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "ClientStats"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *ClientStats) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "fullQueueDroppedSpans", thrift.I64, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:fullQueueDroppedSpans: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.FullQueueDroppedSpans)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.fullQueueDroppedSpans (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:fullQueueDroppedSpans: ", p), err) }
+  return err
+}
+
+func (p *ClientStats) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "tooLargeDroppedSpans", thrift.I64, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tooLargeDroppedSpans: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TooLargeDroppedSpans)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.tooLargeDroppedSpans (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tooLargeDroppedSpans: ", p), err) }
+  return err
+}
+
+func (p *ClientStats) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "failedToEmitSpans", thrift.I64, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:failedToEmitSpans: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.FailedToEmitSpans)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.failedToEmitSpans (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:failedToEmitSpans: ", p), err) }
+  return err
+}
+
+func (p *ClientStats) Equals(other *ClientStats) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.FullQueueDroppedSpans != other.FullQueueDroppedSpans { return false }
+  if p.TooLargeDroppedSpans != other.TooLargeDroppedSpans { return false }
+  if p.FailedToEmitSpans != other.FailedToEmitSpans { return false }
+  return true
+}
+
+func (p *ClientStats) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("ClientStats(%+v)", *p)
+}
+
+// Attributes:
+//  - Process
+//  - Spans
+//  - SeqNo
+//  - Stats
+type Batch struct {
+  Process *Process `thrift:"process,1,required" db:"process" json:"process"`
+  Spans []*Span `thrift:"spans,2,required" db:"spans" json:"spans"`
+  SeqNo *int64 `thrift:"seqNo,3" db:"seqNo" json:"seqNo,omitempty"`
+  Stats *ClientStats `thrift:"stats,4" db:"stats" json:"stats,omitempty"`
+}
+
+func NewBatch() *Batch {
+  return &Batch{}
+}
+
+var Batch_Process_DEFAULT *Process
+func (p *Batch) GetProcess() *Process {
+  if !p.IsSetProcess() {
+    return Batch_Process_DEFAULT
+  }
+return p.Process
+}
+
+func (p *Batch) GetSpans() []*Span {
+  return p.Spans
+}
+var Batch_SeqNo_DEFAULT int64
+func (p *Batch) GetSeqNo() int64 {
+  if !p.IsSetSeqNo() {
+    return Batch_SeqNo_DEFAULT
+  }
+return *p.SeqNo
+}
+var Batch_Stats_DEFAULT *ClientStats
+func (p *Batch) GetStats() *ClientStats {
+  if !p.IsSetStats() {
+    return Batch_Stats_DEFAULT
+  }
+return p.Stats
+}
+func (p *Batch) IsSetProcess() bool {
+  return p.Process != nil
+}
+
+func (p *Batch) IsSetSeqNo() bool {
+  return p.SeqNo != nil
+}
+
+func (p *Batch) IsSetStats() bool {
+  return p.Stats != nil
+}
+
+func (p *Batch) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetProcess bool = false;
+  var issetSpans bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetProcess = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetSpans = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetProcess{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Process is not set"));
+  }
+  if !issetSpans{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Spans is not set"));
+  }
+  return nil
+}
+
+func (p *Batch)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Process = &Process{}
+  if err := p.Process.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Process), err)
+  }
+  return nil
+}
+
+func (p *Batch)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Span, 0, size)
+  p.Spans =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem10 := &Span{}
+    if err := _elem10.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem10), err)
+    }
+    p.Spans = append(p.Spans, _elem10)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Batch)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.SeqNo = &v
+}
+  return nil
+}
+
+func (p *Batch)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Stats = &ClientStats{}
+  if err := p.Stats.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Stats), err)
+  }
+  return nil
+}
+
+func (p *Batch) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Batch"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Batch) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "process", thrift.STRUCT, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:process: ", p), err) }
+  if err := p.Process.Write(ctx, oprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Process), err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:process: ", p), err) }
+  return err
+}
+
+func (p *Batch) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "spans", thrift.LIST, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:spans: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Spans)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Spans {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:spans: ", p), err) }
+  return err
+}
+
+func (p *Batch) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetSeqNo() {
+    if err := oprot.WriteFieldBegin(ctx, "seqNo", thrift.I64, 3); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:seqNo: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.SeqNo)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.seqNo (3) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 3:seqNo: ", p), err) }
+  }
+  return err
+}
+
+func (p *Batch) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetStats() {
+    if err := oprot.WriteFieldBegin(ctx, "stats", thrift.STRUCT, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:stats: ", p), err) }
+    if err := p.Stats.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Stats), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:stats: ", p), err) }
+  }
+  return err
+}
+
+func (p *Batch) Equals(other *Batch) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if !p.Process.Equals(other.Process) { return false }
+  if len(p.Spans) != len(other.Spans) { return false }
+  for i, _tgt := range p.Spans {
+    _src11 := other.Spans[i]
+    if !_tgt.Equals(_src11) { return false }
+  }
+  if p.SeqNo != other.SeqNo {
+    if p.SeqNo == nil || other.SeqNo == nil {
+      return false
+    }
+    if (*p.SeqNo) != (*other.SeqNo) { return false }
+  }
+  if !p.Stats.Equals(other.Stats) { return false }
+  return true
+}
+
+func (p *Batch) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Batch(%+v)", *p)
+}
+
+// Attributes:
+//  - Ok
+type BatchSubmitResponse struct {
+  Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"`
+}
+
+func NewBatchSubmitResponse() *BatchSubmitResponse {
+  return &BatchSubmitResponse{}
+}
+
+
+func (p *BatchSubmitResponse) GetOk() bool {
+  return p.Ok
+}
+func (p *BatchSubmitResponse) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetOk bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.BOOL {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetOk = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetOk{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set"));
+  }
+  return nil
+}
+
+func (p *BatchSubmitResponse)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBool(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Ok = v
+}
+  return nil
+}
+
+func (p *BatchSubmitResponse) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "BatchSubmitResponse"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *BatchSubmitResponse) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "ok", thrift.BOOL, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) }
+  if err := oprot.WriteBool(ctx, bool(p.Ok)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) }
+  return err
+}
+
+func (p *BatchSubmitResponse) Equals(other *BatchSubmitResponse) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Ok != other.Ok { return false }
+  return true
+}
+
+func (p *BatchSubmitResponse) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("BatchSubmitResponse(%+v)", *p)
+}
+
+type Collector interface {
+  // Parameters:
+  //  - Batches
+  SubmitBatches(ctx context.Context, batches []*Batch) (_r []*BatchSubmitResponse, _err error)
+}
+
+type CollectorClient struct {
+  c thrift.TClient
+  meta thrift.ResponseMeta
+}
+
+func NewCollectorClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *CollectorClient {
+  return &CollectorClient{
+    c: thrift.NewTStandardClient(f.GetProtocol(t), f.GetProtocol(t)),
+  }
+}
+
+func NewCollectorClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *CollectorClient {
+  return &CollectorClient{
+    c: thrift.NewTStandardClient(iprot, oprot),
+  }
+}
+
+func NewCollectorClient(c thrift.TClient) *CollectorClient {
+  return &CollectorClient{
+    c: c,
+  }
+}
+
+func (p *CollectorClient) Client_() thrift.TClient {
+  return p.c
+}
+
+func (p *CollectorClient) LastResponseMeta_() thrift.ResponseMeta {
+  return p.meta
+}
+
+func (p *CollectorClient) SetLastResponseMeta_(meta thrift.ResponseMeta) {
+  p.meta = meta
+}
+
+// Parameters:
+//  - Batches
+func (p *CollectorClient) SubmitBatches(ctx context.Context, batches []*Batch) (_r []*BatchSubmitResponse, _err error) {
+  var _args12 CollectorSubmitBatchesArgs
+  _args12.Batches = batches
+  var _result14 CollectorSubmitBatchesResult
+  var _meta13 thrift.ResponseMeta
+  _meta13, _err = p.Client_().Call(ctx, "submitBatches", &_args12, &_result14)
+  p.SetLastResponseMeta_(_meta13)
+  if _err != nil {
+    return
+  }
+  return _result14.GetSuccess(), nil
+}
+
+type CollectorProcessor struct {
+  processorMap map[string]thrift.TProcessorFunction
+  handler Collector
+}
+
+func (p *CollectorProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
+  p.processorMap[key] = processor
+}
+
+func (p *CollectorProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
+  processor, ok = p.processorMap[key]
+  return processor, ok
+}
+
+func (p *CollectorProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
+  return p.processorMap
+}
+
+func NewCollectorProcessor(handler Collector) *CollectorProcessor {
+
+  self15 := &CollectorProcessor{handler:handler, processorMap:make(map[string]thrift.TProcessorFunction)}
+  self15.processorMap["submitBatches"] = &collectorProcessorSubmitBatches{handler:handler}
+return self15
+}
+
+func (p *CollectorProcessor) Process(ctx context.Context, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  name, _, seqId, err2 := iprot.ReadMessageBegin(ctx)
+  if err2 != nil { return false, thrift.WrapTException(err2) }
+  if processor, ok := p.GetProcessorFunction(name); ok {
+    return processor.Process(ctx, seqId, iprot, oprot)
+  }
+  iprot.Skip(ctx, thrift.STRUCT)
+  iprot.ReadMessageEnd(ctx)
+  x16 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function " + name)
+  oprot.WriteMessageBegin(ctx, name, thrift.EXCEPTION, seqId)
+  x16.Write(ctx, oprot)
+  oprot.WriteMessageEnd(ctx)
+  oprot.Flush(ctx)
+  return false, x16
+
+}
+
+type collectorProcessorSubmitBatches struct {
+  handler Collector
+}
+
+func (p *collectorProcessorSubmitBatches) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := CollectorSubmitBatchesArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err2.Error())
+    oprot.WriteMessageBegin(ctx, "submitBatches", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
+
+  tickerCancel := func() {}
+  // Start a goroutine to do server side connectivity check.
+  if thrift.ServerConnectivityCheckInterval > 0 {
+    var cancel context.CancelFunc
+    ctx, cancel = context.WithCancel(ctx)
+    defer cancel()
+    var tickerCtx context.Context
+    tickerCtx, tickerCancel = context.WithCancel(context.Background())
+    defer tickerCancel()
+    go func(ctx context.Context, cancel context.CancelFunc) {
+      ticker := time.NewTicker(thrift.ServerConnectivityCheckInterval)
+      defer ticker.Stop()
+      for {
+        select {
+        case <-ctx.Done():
+          return
+        case <-ticker.C:
+          if !iprot.Transport().IsOpen() {
+            cancel()
+            return
+          }
+        }
+      }
+    }(tickerCtx, cancel)
+  }
+
+  result := CollectorSubmitBatchesResult{}
+  var retval []*BatchSubmitResponse
+  if retval, err2 = p.handler.SubmitBatches(ctx, args.Batches); err2 != nil {
+    tickerCancel()
+    if err2 == thrift.ErrAbandonRequest {
+      return false, thrift.WrapTException(err2)
+    }
+    x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing submitBatches: " + err2.Error())
+    oprot.WriteMessageBegin(ctx, "submitBatches", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return true, thrift.WrapTException(err2)
+  } else {
+    result.Success = retval
+  }
+  tickerCancel()
+  if err2 = oprot.WriteMessageBegin(ctx, "submitBatches", thrift.REPLY, seqId); err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = result.Write(ctx, oprot); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.WriteMessageEnd(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.Flush(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err != nil {
+    return
+  }
+  return true, err
+}
+
+
+// HELPER FUNCTIONS AND STRUCTURES
+
+// Attributes:
+//  - Batches
+type CollectorSubmitBatchesArgs struct {
+  Batches []*Batch `thrift:"batches,1" db:"batches" json:"batches"`
+}
+
+func NewCollectorSubmitBatchesArgs() *CollectorSubmitBatchesArgs {
+  return &CollectorSubmitBatchesArgs{}
+}
+
+
+func (p *CollectorSubmitBatchesArgs) GetBatches() []*Batch {
+  return p.Batches
+}
+func (p *CollectorSubmitBatchesArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Batch, 0, size)
+  p.Batches =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem17 := &Batch{}
+    if err := _elem17.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem17), err)
+    }
+    p.Batches = append(p.Batches, _elem17)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "submitBatches_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "batches", thrift.LIST, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:batches: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Batches)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Batches {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:batches: ", p), err) }
+  return err
+}
+
+func (p *CollectorSubmitBatchesArgs) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("CollectorSubmitBatchesArgs(%+v)", *p)
+}
+
+// Attributes:
+//  - Success
+type CollectorSubmitBatchesResult struct {
+  Success []*BatchSubmitResponse `thrift:"success,0" db:"success" json:"success,omitempty"`
+}
+
+func NewCollectorSubmitBatchesResult() *CollectorSubmitBatchesResult {
+  return &CollectorSubmitBatchesResult{}
+}
+
+var CollectorSubmitBatchesResult_Success_DEFAULT []*BatchSubmitResponse
+
+func (p *CollectorSubmitBatchesResult) GetSuccess() []*BatchSubmitResponse {
+  return p.Success
+}
+func (p *CollectorSubmitBatchesResult) IsSetSuccess() bool {
+  return p.Success != nil
+}
+
+func (p *CollectorSubmitBatchesResult) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 0:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField0(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesResult)  ReadField0(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*BatchSubmitResponse, 0, size)
+  p.Success =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem18 := &BatchSubmitResponse{}
+    if err := _elem18.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem18), err)
+    }
+    p.Success = append(p.Success, _elem18)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesResult) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "submitBatches_result"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField0(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *CollectorSubmitBatchesResult) writeField0(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetSuccess() {
+    if err := oprot.WriteFieldBegin(ctx, "success", thrift.LIST, 0); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Success)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Success {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) }
+  }
+  return err
+}
+
+func (p *CollectorSubmitBatchesResult) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("CollectorSubmitBatchesResult(%+v)", *p)
+}
+
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/ttypes.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/ttypes.go
deleted file mode 100644
index e69c6d6..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/jaeger/ttypes.go
+++ /dev/null
@@ -1,2106 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package jaeger
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var GoUnusedProtection__ int
-
-type TagType int64
-
-const (
-	TagType_STRING TagType = 0
-	TagType_DOUBLE TagType = 1
-	TagType_BOOL   TagType = 2
-	TagType_LONG   TagType = 3
-	TagType_BINARY TagType = 4
-)
-
-func (p TagType) String() string {
-	switch p {
-	case TagType_STRING:
-		return "STRING"
-	case TagType_DOUBLE:
-		return "DOUBLE"
-	case TagType_BOOL:
-		return "BOOL"
-	case TagType_LONG:
-		return "LONG"
-	case TagType_BINARY:
-		return "BINARY"
-	}
-	return "<UNSET>"
-}
-
-func TagTypeFromString(s string) (TagType, error) {
-	switch s {
-	case "STRING":
-		return TagType_STRING, nil
-	case "DOUBLE":
-		return TagType_DOUBLE, nil
-	case "BOOL":
-		return TagType_BOOL, nil
-	case "LONG":
-		return TagType_LONG, nil
-	case "BINARY":
-		return TagType_BINARY, nil
-	}
-	return TagType(0), fmt.Errorf("not a valid TagType string")
-}
-
-func TagTypePtr(v TagType) *TagType { return &v }
-
-func (p TagType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *TagType) UnmarshalText(text []byte) error {
-	q, err := TagTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-type SpanRefType int64
-
-const (
-	SpanRefType_CHILD_OF     SpanRefType = 0
-	SpanRefType_FOLLOWS_FROM SpanRefType = 1
-)
-
-func (p SpanRefType) String() string {
-	switch p {
-	case SpanRefType_CHILD_OF:
-		return "CHILD_OF"
-	case SpanRefType_FOLLOWS_FROM:
-		return "FOLLOWS_FROM"
-	}
-	return "<UNSET>"
-}
-
-func SpanRefTypeFromString(s string) (SpanRefType, error) {
-	switch s {
-	case "CHILD_OF":
-		return SpanRefType_CHILD_OF, nil
-	case "FOLLOWS_FROM":
-		return SpanRefType_FOLLOWS_FROM, nil
-	}
-	return SpanRefType(0), fmt.Errorf("not a valid SpanRefType string")
-}
-
-func SpanRefTypePtr(v SpanRefType) *SpanRefType { return &v }
-
-func (p SpanRefType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *SpanRefType) UnmarshalText(text []byte) error {
-	q, err := SpanRefTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-// Attributes:
-//  - Key
-//  - VType
-//  - VStr
-//  - VDouble
-//  - VBool
-//  - VLong
-//  - VBinary
-type Tag struct {
-	Key     string   `thrift:"key,1,required" json:"key"`
-	VType   TagType  `thrift:"vType,2,required" json:"vType"`
-	VStr    *string  `thrift:"vStr,3" json:"vStr,omitempty"`
-	VDouble *float64 `thrift:"vDouble,4" json:"vDouble,omitempty"`
-	VBool   *bool    `thrift:"vBool,5" json:"vBool,omitempty"`
-	VLong   *int64   `thrift:"vLong,6" json:"vLong,omitempty"`
-	VBinary []byte   `thrift:"vBinary,7" json:"vBinary,omitempty"`
-}
-
-func NewTag() *Tag {
-	return &Tag{}
-}
-
-func (p *Tag) GetKey() string {
-	return p.Key
-}
-
-func (p *Tag) GetVType() TagType {
-	return p.VType
-}
-
-var Tag_VStr_DEFAULT string
-
-func (p *Tag) GetVStr() string {
-	if !p.IsSetVStr() {
-		return Tag_VStr_DEFAULT
-	}
-	return *p.VStr
-}
-
-var Tag_VDouble_DEFAULT float64
-
-func (p *Tag) GetVDouble() float64 {
-	if !p.IsSetVDouble() {
-		return Tag_VDouble_DEFAULT
-	}
-	return *p.VDouble
-}
-
-var Tag_VBool_DEFAULT bool
-
-func (p *Tag) GetVBool() bool {
-	if !p.IsSetVBool() {
-		return Tag_VBool_DEFAULT
-	}
-	return *p.VBool
-}
-
-var Tag_VLong_DEFAULT int64
-
-func (p *Tag) GetVLong() int64 {
-	if !p.IsSetVLong() {
-		return Tag_VLong_DEFAULT
-	}
-	return *p.VLong
-}
-
-var Tag_VBinary_DEFAULT []byte
-
-func (p *Tag) GetVBinary() []byte {
-	return p.VBinary
-}
-func (p *Tag) IsSetVStr() bool {
-	return p.VStr != nil
-}
-
-func (p *Tag) IsSetVDouble() bool {
-	return p.VDouble != nil
-}
-
-func (p *Tag) IsSetVBool() bool {
-	return p.VBool != nil
-}
-
-func (p *Tag) IsSetVLong() bool {
-	return p.VLong != nil
-}
-
-func (p *Tag) IsSetVBinary() bool {
-	return p.VBinary != nil
-}
-
-func (p *Tag) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetKey bool = false
-	var issetVType bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetKey = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetVType = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		case 5:
-			if err := p.readField5(iprot); err != nil {
-				return err
-			}
-		case 6:
-			if err := p.readField6(iprot); err != nil {
-				return err
-			}
-		case 7:
-			if err := p.readField7(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetKey {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Key is not set"))
-	}
-	if !issetVType {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field VType is not set"))
-	}
-	return nil
-}
-
-func (p *Tag) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Key = v
-	}
-	return nil
-}
-
-func (p *Tag) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		temp := TagType(v)
-		p.VType = temp
-	}
-	return nil
-}
-
-func (p *Tag) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.VStr = &v
-	}
-	return nil
-}
-
-func (p *Tag) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadDouble(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.VDouble = &v
-	}
-	return nil
-}
-
-func (p *Tag) readField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.VBool = &v
-	}
-	return nil
-}
-
-func (p *Tag) readField6(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 6: ", err)
-	} else {
-		p.VLong = &v
-	}
-	return nil
-}
-
-func (p *Tag) readField7(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBinary(); err != nil {
-		return thrift.PrependError("error reading field 7: ", err)
-	} else {
-		p.VBinary = v
-	}
-	return nil
-}
-
-func (p *Tag) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Tag"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField5(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField6(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField7(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Tag) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("key", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:key: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Key)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.key (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:key: ", p), err)
-	}
-	return err
-}
-
-func (p *Tag) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("vType", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:vType: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.VType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.vType (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:vType: ", p), err)
-	}
-	return err
-}
-
-func (p *Tag) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetVStr() {
-		if err := oprot.WriteFieldBegin("vStr", thrift.STRING, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:vStr: ", p), err)
-		}
-		if err := oprot.WriteString(string(*p.VStr)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.vStr (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:vStr: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Tag) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetVDouble() {
-		if err := oprot.WriteFieldBegin("vDouble", thrift.DOUBLE, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:vDouble: ", p), err)
-		}
-		if err := oprot.WriteDouble(float64(*p.VDouble)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.vDouble (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:vDouble: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Tag) writeField5(oprot thrift.TProtocol) (err error) {
-	if p.IsSetVBool() {
-		if err := oprot.WriteFieldBegin("vBool", thrift.BOOL, 5); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:vBool: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(*p.VBool)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.vBool (5) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 5:vBool: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Tag) writeField6(oprot thrift.TProtocol) (err error) {
-	if p.IsSetVLong() {
-		if err := oprot.WriteFieldBegin("vLong", thrift.I64, 6); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:vLong: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.VLong)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.vLong (6) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 6:vLong: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Tag) writeField7(oprot thrift.TProtocol) (err error) {
-	if p.IsSetVBinary() {
-		if err := oprot.WriteFieldBegin("vBinary", thrift.STRING, 7); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:vBinary: ", p), err)
-		}
-		if err := oprot.WriteBinary(p.VBinary); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.vBinary (7) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 7:vBinary: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Tag) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Tag(%+v)", *p)
-}
-
-// Attributes:
-//  - Timestamp
-//  - Fields
-type Log struct {
-	Timestamp int64  `thrift:"timestamp,1,required" json:"timestamp"`
-	Fields    []*Tag `thrift:"fields,2,required" json:"fields"`
-}
-
-func NewLog() *Log {
-	return &Log{}
-}
-
-func (p *Log) GetTimestamp() int64 {
-	return p.Timestamp
-}
-
-func (p *Log) GetFields() []*Tag {
-	return p.Fields
-}
-func (p *Log) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetTimestamp bool = false
-	var issetFields bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetTimestamp = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetFields = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetTimestamp {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Timestamp is not set"))
-	}
-	if !issetFields {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Fields is not set"))
-	}
-	return nil
-}
-
-func (p *Log) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Timestamp = v
-	}
-	return nil
-}
-
-func (p *Log) readField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Tag, 0, size)
-	p.Fields = tSlice
-	for i := 0; i < size; i++ {
-		_elem0 := &Tag{}
-		if err := _elem0.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
-		}
-		p.Fields = append(p.Fields, _elem0)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Log) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Log"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Log) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("timestamp", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:timestamp: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.Timestamp)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.timestamp (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:timestamp: ", p), err)
-	}
-	return err
-}
-
-func (p *Log) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("fields", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:fields: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Fields)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Fields {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:fields: ", p), err)
-	}
-	return err
-}
-
-func (p *Log) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Log(%+v)", *p)
-}
-
-// Attributes:
-//  - RefType
-//  - TraceIdLow
-//  - TraceIdHigh
-//  - SpanId
-type SpanRef struct {
-	RefType     SpanRefType `thrift:"refType,1,required" json:"refType"`
-	TraceIdLow  int64       `thrift:"traceIdLow,2,required" json:"traceIdLow"`
-	TraceIdHigh int64       `thrift:"traceIdHigh,3,required" json:"traceIdHigh"`
-	SpanId      int64       `thrift:"spanId,4,required" json:"spanId"`
-}
-
-func NewSpanRef() *SpanRef {
-	return &SpanRef{}
-}
-
-func (p *SpanRef) GetRefType() SpanRefType {
-	return p.RefType
-}
-
-func (p *SpanRef) GetTraceIdLow() int64 {
-	return p.TraceIdLow
-}
-
-func (p *SpanRef) GetTraceIdHigh() int64 {
-	return p.TraceIdHigh
-}
-
-func (p *SpanRef) GetSpanId() int64 {
-	return p.SpanId
-}
-func (p *SpanRef) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetRefType bool = false
-	var issetTraceIdLow bool = false
-	var issetTraceIdHigh bool = false
-	var issetSpanId bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetRefType = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetTraceIdLow = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-			issetTraceIdHigh = true
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-			issetSpanId = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetRefType {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RefType is not set"))
-	}
-	if !issetTraceIdLow {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdLow is not set"))
-	}
-	if !issetTraceIdHigh {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdHigh is not set"))
-	}
-	if !issetSpanId {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SpanId is not set"))
-	}
-	return nil
-}
-
-func (p *SpanRef) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := SpanRefType(v)
-		p.RefType = temp
-	}
-	return nil
-}
-
-func (p *SpanRef) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.TraceIdLow = v
-	}
-	return nil
-}
-
-func (p *SpanRef) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.TraceIdHigh = v
-	}
-	return nil
-}
-
-func (p *SpanRef) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.SpanId = v
-	}
-	return nil
-}
-
-func (p *SpanRef) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("SpanRef"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *SpanRef) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("refType", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:refType: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.RefType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.refType (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:refType: ", p), err)
-	}
-	return err
-}
-
-func (p *SpanRef) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("traceIdLow", thrift.I64, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:traceIdLow: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TraceIdLow)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.traceIdLow (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:traceIdLow: ", p), err)
-	}
-	return err
-}
-
-func (p *SpanRef) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("traceIdHigh", thrift.I64, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:traceIdHigh: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TraceIdHigh)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.traceIdHigh (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:traceIdHigh: ", p), err)
-	}
-	return err
-}
-
-func (p *SpanRef) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("spanId", thrift.I64, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:spanId: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.SpanId)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.spanId (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:spanId: ", p), err)
-	}
-	return err
-}
-
-func (p *SpanRef) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("SpanRef(%+v)", *p)
-}
-
-// Attributes:
-//  - TraceIdLow
-//  - TraceIdHigh
-//  - SpanId
-//  - ParentSpanId
-//  - OperationName
-//  - References
-//  - Flags
-//  - StartTime
-//  - Duration
-//  - Tags
-//  - Logs
-type Span struct {
-	TraceIdLow    int64      `thrift:"traceIdLow,1,required" json:"traceIdLow"`
-	TraceIdHigh   int64      `thrift:"traceIdHigh,2,required" json:"traceIdHigh"`
-	SpanId        int64      `thrift:"spanId,3,required" json:"spanId"`
-	ParentSpanId  int64      `thrift:"parentSpanId,4,required" json:"parentSpanId"`
-	OperationName string     `thrift:"operationName,5,required" json:"operationName"`
-	References    []*SpanRef `thrift:"references,6" json:"references,omitempty"`
-	Flags         int32      `thrift:"flags,7,required" json:"flags"`
-	StartTime     int64      `thrift:"startTime,8,required" json:"startTime"`
-	Duration      int64      `thrift:"duration,9,required" json:"duration"`
-	Tags          []*Tag     `thrift:"tags,10" json:"tags,omitempty"`
-	Logs          []*Log     `thrift:"logs,11" json:"logs,omitempty"`
-}
-
-func NewSpan() *Span {
-	return &Span{}
-}
-
-func (p *Span) GetTraceIdLow() int64 {
-	return p.TraceIdLow
-}
-
-func (p *Span) GetTraceIdHigh() int64 {
-	return p.TraceIdHigh
-}
-
-func (p *Span) GetSpanId() int64 {
-	return p.SpanId
-}
-
-func (p *Span) GetParentSpanId() int64 {
-	return p.ParentSpanId
-}
-
-func (p *Span) GetOperationName() string {
-	return p.OperationName
-}
-
-var Span_References_DEFAULT []*SpanRef
-
-func (p *Span) GetReferences() []*SpanRef {
-	return p.References
-}
-
-func (p *Span) GetFlags() int32 {
-	return p.Flags
-}
-
-func (p *Span) GetStartTime() int64 {
-	return p.StartTime
-}
-
-func (p *Span) GetDuration() int64 {
-	return p.Duration
-}
-
-var Span_Tags_DEFAULT []*Tag
-
-func (p *Span) GetTags() []*Tag {
-	return p.Tags
-}
-
-var Span_Logs_DEFAULT []*Log
-
-func (p *Span) GetLogs() []*Log {
-	return p.Logs
-}
-func (p *Span) IsSetReferences() bool {
-	return p.References != nil
-}
-
-func (p *Span) IsSetTags() bool {
-	return p.Tags != nil
-}
-
-func (p *Span) IsSetLogs() bool {
-	return p.Logs != nil
-}
-
-func (p *Span) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetTraceIdLow bool = false
-	var issetTraceIdHigh bool = false
-	var issetSpanId bool = false
-	var issetParentSpanId bool = false
-	var issetOperationName bool = false
-	var issetFlags bool = false
-	var issetStartTime bool = false
-	var issetDuration bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetTraceIdLow = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetTraceIdHigh = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-			issetSpanId = true
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-			issetParentSpanId = true
-		case 5:
-			if err := p.readField5(iprot); err != nil {
-				return err
-			}
-			issetOperationName = true
-		case 6:
-			if err := p.readField6(iprot); err != nil {
-				return err
-			}
-		case 7:
-			if err := p.readField7(iprot); err != nil {
-				return err
-			}
-			issetFlags = true
-		case 8:
-			if err := p.readField8(iprot); err != nil {
-				return err
-			}
-			issetStartTime = true
-		case 9:
-			if err := p.readField9(iprot); err != nil {
-				return err
-			}
-			issetDuration = true
-		case 10:
-			if err := p.readField10(iprot); err != nil {
-				return err
-			}
-		case 11:
-			if err := p.readField11(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetTraceIdLow {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdLow is not set"))
-	}
-	if !issetTraceIdHigh {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TraceIdHigh is not set"))
-	}
-	if !issetSpanId {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SpanId is not set"))
-	}
-	if !issetParentSpanId {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ParentSpanId is not set"))
-	}
-	if !issetOperationName {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field OperationName is not set"))
-	}
-	if !issetFlags {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Flags is not set"))
-	}
-	if !issetStartTime {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field StartTime is not set"))
-	}
-	if !issetDuration {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Duration is not set"))
-	}
-	return nil
-}
-
-func (p *Span) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.TraceIdLow = v
-	}
-	return nil
-}
-
-func (p *Span) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.TraceIdHigh = v
-	}
-	return nil
-}
-
-func (p *Span) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.SpanId = v
-	}
-	return nil
-}
-
-func (p *Span) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.ParentSpanId = v
-	}
-	return nil
-}
-
-func (p *Span) readField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.OperationName = v
-	}
-	return nil
-}
-
-func (p *Span) readField6(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*SpanRef, 0, size)
-	p.References = tSlice
-	for i := 0; i < size; i++ {
-		_elem1 := &SpanRef{}
-		if err := _elem1.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem1), err)
-		}
-		p.References = append(p.References, _elem1)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Span) readField7(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 7: ", err)
-	} else {
-		p.Flags = v
-	}
-	return nil
-}
-
-func (p *Span) readField8(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 8: ", err)
-	} else {
-		p.StartTime = v
-	}
-	return nil
-}
-
-func (p *Span) readField9(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 9: ", err)
-	} else {
-		p.Duration = v
-	}
-	return nil
-}
-
-func (p *Span) readField10(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Tag, 0, size)
-	p.Tags = tSlice
-	for i := 0; i < size; i++ {
-		_elem2 := &Tag{}
-		if err := _elem2.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem2), err)
-		}
-		p.Tags = append(p.Tags, _elem2)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Span) readField11(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Log, 0, size)
-	p.Logs = tSlice
-	for i := 0; i < size; i++ {
-		_elem3 := &Log{}
-		if err := _elem3.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem3), err)
-		}
-		p.Logs = append(p.Logs, _elem3)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Span) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Span"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField5(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField6(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField7(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField8(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField9(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField10(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField11(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Span) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("traceIdLow", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:traceIdLow: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TraceIdLow)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.traceIdLow (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:traceIdLow: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("traceIdHigh", thrift.I64, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:traceIdHigh: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TraceIdHigh)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.traceIdHigh (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:traceIdHigh: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("spanId", thrift.I64, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:spanId: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.SpanId)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.spanId (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:spanId: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("parentSpanId", thrift.I64, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:parentSpanId: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.ParentSpanId)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.parentSpanId (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:parentSpanId: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField5(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("operationName", thrift.STRING, 5); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:operationName: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.OperationName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.operationName (5) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 5:operationName: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField6(oprot thrift.TProtocol) (err error) {
-	if p.IsSetReferences() {
-		if err := oprot.WriteFieldBegin("references", thrift.LIST, 6); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:references: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.References)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.References {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 6:references: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField7(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("flags", thrift.I32, 7); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:flags: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Flags)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.flags (7) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 7:flags: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField8(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("startTime", thrift.I64, 8); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:startTime: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.StartTime)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.startTime (8) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 8:startTime: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField9(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("duration", thrift.I64, 9); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 9:duration: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.Duration)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.duration (9) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 9:duration: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField10(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTags() {
-		if err := oprot.WriteFieldBegin("tags", thrift.LIST, 10); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 10:tags: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Tags)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Tags {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 10:tags: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField11(oprot thrift.TProtocol) (err error) {
-	if p.IsSetLogs() {
-		if err := oprot.WriteFieldBegin("logs", thrift.LIST, 11); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 11:logs: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Logs)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Logs {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 11:logs: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Span(%+v)", *p)
-}
-
-// Attributes:
-//  - ServiceName
-//  - Tags
-type Process struct {
-	ServiceName string `thrift:"serviceName,1,required" json:"serviceName"`
-	Tags        []*Tag `thrift:"tags,2" json:"tags,omitempty"`
-}
-
-func NewProcess() *Process {
-	return &Process{}
-}
-
-func (p *Process) GetServiceName() string {
-	return p.ServiceName
-}
-
-var Process_Tags_DEFAULT []*Tag
-
-func (p *Process) GetTags() []*Tag {
-	return p.Tags
-}
-func (p *Process) IsSetTags() bool {
-	return p.Tags != nil
-}
-
-func (p *Process) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetServiceName bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetServiceName = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetServiceName {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ServiceName is not set"))
-	}
-	return nil
-}
-
-func (p *Process) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.ServiceName = v
-	}
-	return nil
-}
-
-func (p *Process) readField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Tag, 0, size)
-	p.Tags = tSlice
-	for i := 0; i < size; i++ {
-		_elem4 := &Tag{}
-		if err := _elem4.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err)
-		}
-		p.Tags = append(p.Tags, _elem4)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Process) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Process"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Process) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("serviceName", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.ServiceName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err)
-	}
-	return err
-}
-
-func (p *Process) writeField2(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTags() {
-		if err := oprot.WriteFieldBegin("tags", thrift.LIST, 2); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tags: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Tags)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Tags {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tags: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Process) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Process(%+v)", *p)
-}
-
-// Attributes:
-//  - FullQueueDroppedSpans
-//  - TooLargeDroppedSpans
-//  - FailedToEmitSpans
-type ClientStats struct {
-	FullQueueDroppedSpans int64 `thrift:"fullQueueDroppedSpans,1,required" json:"fullQueueDroppedSpans"`
-	TooLargeDroppedSpans  int64 `thrift:"tooLargeDroppedSpans,2,required" json:"tooLargeDroppedSpans"`
-	FailedToEmitSpans     int64 `thrift:"failedToEmitSpans,3,required" json:"failedToEmitSpans"`
-}
-
-func NewClientStats() *ClientStats {
-	return &ClientStats{}
-}
-
-func (p *ClientStats) GetFullQueueDroppedSpans() int64 {
-	return p.FullQueueDroppedSpans
-}
-
-func (p *ClientStats) GetTooLargeDroppedSpans() int64 {
-	return p.TooLargeDroppedSpans
-}
-
-func (p *ClientStats) GetFailedToEmitSpans() int64 {
-	return p.FailedToEmitSpans
-}
-func (p *ClientStats) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetFullQueueDroppedSpans bool = false
-	var issetTooLargeDroppedSpans bool = false
-	var issetFailedToEmitSpans bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetFullQueueDroppedSpans = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetTooLargeDroppedSpans = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-			issetFailedToEmitSpans = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetFullQueueDroppedSpans {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field FullQueueDroppedSpans is not set"))
-	}
-	if !issetTooLargeDroppedSpans {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TooLargeDroppedSpans is not set"))
-	}
-	if !issetFailedToEmitSpans {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field FailedToEmitSpans is not set"))
-	}
-	return nil
-}
-
-func (p *ClientStats) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.FullQueueDroppedSpans = v
-	}
-	return nil
-}
-
-func (p *ClientStats) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.TooLargeDroppedSpans = v
-	}
-	return nil
-}
-
-func (p *ClientStats) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.FailedToEmitSpans = v
-	}
-	return nil
-}
-
-func (p *ClientStats) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("ClientStats"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ClientStats) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("fullQueueDroppedSpans", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:fullQueueDroppedSpans: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.FullQueueDroppedSpans)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.fullQueueDroppedSpans (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:fullQueueDroppedSpans: ", p), err)
-	}
-	return err
-}
-
-func (p *ClientStats) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("tooLargeDroppedSpans", thrift.I64, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tooLargeDroppedSpans: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TooLargeDroppedSpans)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.tooLargeDroppedSpans (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tooLargeDroppedSpans: ", p), err)
-	}
-	return err
-}
-
-func (p *ClientStats) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("failedToEmitSpans", thrift.I64, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:failedToEmitSpans: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.FailedToEmitSpans)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.failedToEmitSpans (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:failedToEmitSpans: ", p), err)
-	}
-	return err
-}
-
-func (p *ClientStats) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ClientStats(%+v)", *p)
-}
-
-// Attributes:
-//  - Process
-//  - Spans
-//  - SeqNo
-//  - Stats
-type Batch struct {
-	Process *Process     `thrift:"process,1,required" json:"process"`
-	Spans   []*Span      `thrift:"spans,2,required" json:"spans"`
-	SeqNo   *int64       `thrift:"seqNo,3" json:"seqNo,omitempty"`
-	Stats   *ClientStats `thrift:"stats,4" json:"stats,omitempty"`
-}
-
-func NewBatch() *Batch {
-	return &Batch{}
-}
-
-var Batch_Process_DEFAULT *Process
-
-func (p *Batch) GetProcess() *Process {
-	if !p.IsSetProcess() {
-		return Batch_Process_DEFAULT
-	}
-	return p.Process
-}
-
-func (p *Batch) GetSpans() []*Span {
-	return p.Spans
-}
-
-var Batch_SeqNo_DEFAULT int64
-
-func (p *Batch) GetSeqNo() int64 {
-	if !p.IsSetSeqNo() {
-		return Batch_SeqNo_DEFAULT
-	}
-	return *p.SeqNo
-}
-
-var Batch_Stats_DEFAULT *ClientStats
-
-func (p *Batch) GetStats() *ClientStats {
-	if !p.IsSetStats() {
-		return Batch_Stats_DEFAULT
-	}
-	return p.Stats
-}
-func (p *Batch) IsSetProcess() bool {
-	return p.Process != nil
-}
-
-func (p *Batch) IsSetSeqNo() bool {
-	return p.SeqNo != nil
-}
-
-func (p *Batch) IsSetStats() bool {
-	return p.Stats != nil
-}
-
-func (p *Batch) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetProcess bool = false
-	var issetSpans bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetProcess = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetSpans = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetProcess {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Process is not set"))
-	}
-	if !issetSpans {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Spans is not set"))
-	}
-	return nil
-}
-
-func (p *Batch) readField1(iprot thrift.TProtocol) error {
-	p.Process = &Process{}
-	if err := p.Process.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Process), err)
-	}
-	return nil
-}
-
-func (p *Batch) readField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Span, 0, size)
-	p.Spans = tSlice
-	for i := 0; i < size; i++ {
-		_elem5 := &Span{}
-		if err := _elem5.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem5), err)
-		}
-		p.Spans = append(p.Spans, _elem5)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Batch) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.SeqNo = &v
-	}
-	return nil
-}
-
-func (p *Batch) readField4(iprot thrift.TProtocol) error {
-	p.Stats = &ClientStats{}
-	if err := p.Stats.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Stats), err)
-	}
-	return nil
-}
-
-func (p *Batch) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Batch"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Batch) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("process", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:process: ", p), err)
-	}
-	if err := p.Process.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Process), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:process: ", p), err)
-	}
-	return err
-}
-
-func (p *Batch) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("spans", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:spans: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Spans)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Spans {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:spans: ", p), err)
-	}
-	return err
-}
-
-func (p *Batch) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetSeqNo() {
-		if err := oprot.WriteFieldBegin("seqNo", thrift.I64, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:seqNo: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.SeqNo)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.seqNo (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:seqNo: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Batch) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetStats() {
-		if err := oprot.WriteFieldBegin("stats", thrift.STRUCT, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:stats: ", p), err)
-		}
-		if err := p.Stats.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Stats), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:stats: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Batch) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Batch(%+v)", *p)
-}
-
-// Attributes:
-//  - Ok
-type BatchSubmitResponse struct {
-	Ok bool `thrift:"ok,1,required" json:"ok"`
-}
-
-func NewBatchSubmitResponse() *BatchSubmitResponse {
-	return &BatchSubmitResponse{}
-}
-
-func (p *BatchSubmitResponse) GetOk() bool {
-	return p.Ok
-}
-func (p *BatchSubmitResponse) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetOk bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetOk = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetOk {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set"))
-	}
-	return nil
-}
-
-func (p *BatchSubmitResponse) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Ok = v
-	}
-	return nil
-}
-
-func (p *BatchSubmitResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("BatchSubmitResponse"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *BatchSubmitResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.Ok)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err)
-	}
-	return err
-}
-
-func (p *BatchSubmitResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BatchSubmitResponse(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/GoUnusedProtection__.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/GoUnusedProtection__.go
new file mode 100644
index 0000000..015ad4b
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/GoUnusedProtection__.go
@@ -0,0 +1,6 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package sampling
+
+var GoUnusedProtection__ int;
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/constants.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/constants.go
deleted file mode 100644
index 0f6e3a8..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/constants.go
+++ /dev/null
@@ -1,18 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package sampling
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-func init() {
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling-consts.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling-consts.go
new file mode 100644
index 0000000..5cc7628
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling-consts.go
@@ -0,0 +1,23 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package sampling
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+
+func init() {
+}
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling.go
new file mode 100644
index 0000000..3bffa5b
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/sampling.go
@@ -0,0 +1,1323 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package sampling
+
+import(
+	"bytes"
+	"context"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+type SamplingStrategyType int64
+const (
+  SamplingStrategyType_PROBABILISTIC SamplingStrategyType = 0
+  SamplingStrategyType_RATE_LIMITING SamplingStrategyType = 1
+)
+
+func (p SamplingStrategyType) String() string {
+  switch p {
+  case SamplingStrategyType_PROBABILISTIC: return "PROBABILISTIC"
+  case SamplingStrategyType_RATE_LIMITING: return "RATE_LIMITING"
+  }
+  return "<UNSET>"
+}
+
+func SamplingStrategyTypeFromString(s string) (SamplingStrategyType, error) {
+  switch s {
+  case "PROBABILISTIC": return SamplingStrategyType_PROBABILISTIC, nil 
+  case "RATE_LIMITING": return SamplingStrategyType_RATE_LIMITING, nil 
+  }
+  return SamplingStrategyType(0), fmt.Errorf("not a valid SamplingStrategyType string")
+}
+
+
+func SamplingStrategyTypePtr(v SamplingStrategyType) *SamplingStrategyType { return &v }
+
+func (p SamplingStrategyType) MarshalText() ([]byte, error) {
+return []byte(p.String()), nil
+}
+
+func (p *SamplingStrategyType) UnmarshalText(text []byte) error {
+q, err := SamplingStrategyTypeFromString(string(text))
+if (err != nil) {
+return err
+}
+*p = q
+return nil
+}
+
+func (p *SamplingStrategyType) Scan(value interface{}) error {
+v, ok := value.(int64)
+if !ok {
+return errors.New("Scan value is not int64")
+}
+*p = SamplingStrategyType(v)
+return nil
+}
+
+func (p * SamplingStrategyType) Value() (driver.Value, error) {
+  if p == nil {
+    return nil, nil
+  }
+return int64(*p), nil
+}
+// Attributes:
+//  - SamplingRate
+type ProbabilisticSamplingStrategy struct {
+  SamplingRate float64 `thrift:"samplingRate,1,required" db:"samplingRate" json:"samplingRate"`
+}
+
+func NewProbabilisticSamplingStrategy() *ProbabilisticSamplingStrategy {
+  return &ProbabilisticSamplingStrategy{}
+}
+
+
+func (p *ProbabilisticSamplingStrategy) GetSamplingRate() float64 {
+  return p.SamplingRate
+}
+func (p *ProbabilisticSamplingStrategy) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetSamplingRate bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.DOUBLE {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetSamplingRate = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetSamplingRate{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SamplingRate is not set"));
+  }
+  return nil
+}
+
+func (p *ProbabilisticSamplingStrategy)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadDouble(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.SamplingRate = v
+}
+  return nil
+}
+
+func (p *ProbabilisticSamplingStrategy) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "ProbabilisticSamplingStrategy"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *ProbabilisticSamplingStrategy) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "samplingRate", thrift.DOUBLE, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:samplingRate: ", p), err) }
+  if err := oprot.WriteDouble(ctx, float64(p.SamplingRate)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.samplingRate (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:samplingRate: ", p), err) }
+  return err
+}
+
+func (p *ProbabilisticSamplingStrategy) Equals(other *ProbabilisticSamplingStrategy) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.SamplingRate != other.SamplingRate { return false }
+  return true
+}
+
+func (p *ProbabilisticSamplingStrategy) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("ProbabilisticSamplingStrategy(%+v)", *p)
+}
+
+// Attributes:
+//  - MaxTracesPerSecond
+type RateLimitingSamplingStrategy struct {
+  MaxTracesPerSecond int16 `thrift:"maxTracesPerSecond,1,required" db:"maxTracesPerSecond" json:"maxTracesPerSecond"`
+}
+
+func NewRateLimitingSamplingStrategy() *RateLimitingSamplingStrategy {
+  return &RateLimitingSamplingStrategy{}
+}
+
+
+func (p *RateLimitingSamplingStrategy) GetMaxTracesPerSecond() int16 {
+  return p.MaxTracesPerSecond
+}
+func (p *RateLimitingSamplingStrategy) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetMaxTracesPerSecond bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I16 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetMaxTracesPerSecond = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetMaxTracesPerSecond{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field MaxTracesPerSecond is not set"));
+  }
+  return nil
+}
+
+func (p *RateLimitingSamplingStrategy)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI16(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.MaxTracesPerSecond = v
+}
+  return nil
+}
+
+func (p *RateLimitingSamplingStrategy) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "RateLimitingSamplingStrategy"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *RateLimitingSamplingStrategy) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "maxTracesPerSecond", thrift.I16, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:maxTracesPerSecond: ", p), err) }
+  if err := oprot.WriteI16(ctx, int16(p.MaxTracesPerSecond)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.maxTracesPerSecond (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:maxTracesPerSecond: ", p), err) }
+  return err
+}
+
+func (p *RateLimitingSamplingStrategy) Equals(other *RateLimitingSamplingStrategy) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.MaxTracesPerSecond != other.MaxTracesPerSecond { return false }
+  return true
+}
+
+func (p *RateLimitingSamplingStrategy) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("RateLimitingSamplingStrategy(%+v)", *p)
+}
+
+// Attributes:
+//  - Operation
+//  - ProbabilisticSampling
+type OperationSamplingStrategy struct {
+  Operation string `thrift:"operation,1,required" db:"operation" json:"operation"`
+  ProbabilisticSampling *ProbabilisticSamplingStrategy `thrift:"probabilisticSampling,2,required" db:"probabilisticSampling" json:"probabilisticSampling"`
+}
+
+func NewOperationSamplingStrategy() *OperationSamplingStrategy {
+  return &OperationSamplingStrategy{}
+}
+
+
+func (p *OperationSamplingStrategy) GetOperation() string {
+  return p.Operation
+}
+var OperationSamplingStrategy_ProbabilisticSampling_DEFAULT *ProbabilisticSamplingStrategy
+func (p *OperationSamplingStrategy) GetProbabilisticSampling() *ProbabilisticSamplingStrategy {
+  if !p.IsSetProbabilisticSampling() {
+    return OperationSamplingStrategy_ProbabilisticSampling_DEFAULT
+  }
+return p.ProbabilisticSampling
+}
+func (p *OperationSamplingStrategy) IsSetProbabilisticSampling() bool {
+  return p.ProbabilisticSampling != nil
+}
+
+func (p *OperationSamplingStrategy) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetOperation bool = false;
+  var issetProbabilisticSampling bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetOperation = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetProbabilisticSampling = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetOperation{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Operation is not set"));
+  }
+  if !issetProbabilisticSampling{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ProbabilisticSampling is not set"));
+  }
+  return nil
+}
+
+func (p *OperationSamplingStrategy)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Operation = v
+}
+  return nil
+}
+
+func (p *OperationSamplingStrategy)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  p.ProbabilisticSampling = &ProbabilisticSamplingStrategy{}
+  if err := p.ProbabilisticSampling.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.ProbabilisticSampling), err)
+  }
+  return nil
+}
+
+func (p *OperationSamplingStrategy) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "OperationSamplingStrategy"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *OperationSamplingStrategy) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "operation", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:operation: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.Operation)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.operation (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:operation: ", p), err) }
+  return err
+}
+
+func (p *OperationSamplingStrategy) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "probabilisticSampling", thrift.STRUCT, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:probabilisticSampling: ", p), err) }
+  if err := p.ProbabilisticSampling.Write(ctx, oprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.ProbabilisticSampling), err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:probabilisticSampling: ", p), err) }
+  return err
+}
+
+func (p *OperationSamplingStrategy) Equals(other *OperationSamplingStrategy) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Operation != other.Operation { return false }
+  if !p.ProbabilisticSampling.Equals(other.ProbabilisticSampling) { return false }
+  return true
+}
+
+func (p *OperationSamplingStrategy) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("OperationSamplingStrategy(%+v)", *p)
+}
+
+// Attributes:
+//  - DefaultSamplingProbability
+//  - DefaultLowerBoundTracesPerSecond
+//  - PerOperationStrategies
+//  - DefaultUpperBoundTracesPerSecond
+type PerOperationSamplingStrategies struct {
+  DefaultSamplingProbability float64 `thrift:"defaultSamplingProbability,1,required" db:"defaultSamplingProbability" json:"defaultSamplingProbability"`
+  DefaultLowerBoundTracesPerSecond float64 `thrift:"defaultLowerBoundTracesPerSecond,2,required" db:"defaultLowerBoundTracesPerSecond" json:"defaultLowerBoundTracesPerSecond"`
+  PerOperationStrategies []*OperationSamplingStrategy `thrift:"perOperationStrategies,3,required" db:"perOperationStrategies" json:"perOperationStrategies"`
+  DefaultUpperBoundTracesPerSecond *float64 `thrift:"defaultUpperBoundTracesPerSecond,4" db:"defaultUpperBoundTracesPerSecond" json:"defaultUpperBoundTracesPerSecond,omitempty"`
+}
+
+func NewPerOperationSamplingStrategies() *PerOperationSamplingStrategies {
+  return &PerOperationSamplingStrategies{}
+}
+
+
+func (p *PerOperationSamplingStrategies) GetDefaultSamplingProbability() float64 {
+  return p.DefaultSamplingProbability
+}
+
+func (p *PerOperationSamplingStrategies) GetDefaultLowerBoundTracesPerSecond() float64 {
+  return p.DefaultLowerBoundTracesPerSecond
+}
+
+func (p *PerOperationSamplingStrategies) GetPerOperationStrategies() []*OperationSamplingStrategy {
+  return p.PerOperationStrategies
+}
+var PerOperationSamplingStrategies_DefaultUpperBoundTracesPerSecond_DEFAULT float64
+func (p *PerOperationSamplingStrategies) GetDefaultUpperBoundTracesPerSecond() float64 {
+  if !p.IsSetDefaultUpperBoundTracesPerSecond() {
+    return PerOperationSamplingStrategies_DefaultUpperBoundTracesPerSecond_DEFAULT
+  }
+return *p.DefaultUpperBoundTracesPerSecond
+}
+func (p *PerOperationSamplingStrategies) IsSetDefaultUpperBoundTracesPerSecond() bool {
+  return p.DefaultUpperBoundTracesPerSecond != nil
+}
+
+func (p *PerOperationSamplingStrategies) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetDefaultSamplingProbability bool = false;
+  var issetDefaultLowerBoundTracesPerSecond bool = false;
+  var issetPerOperationStrategies bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.DOUBLE {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetDefaultSamplingProbability = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.DOUBLE {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+        issetDefaultLowerBoundTracesPerSecond = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+        issetPerOperationStrategies = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.DOUBLE {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetDefaultSamplingProbability{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DefaultSamplingProbability is not set"));
+  }
+  if !issetDefaultLowerBoundTracesPerSecond{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DefaultLowerBoundTracesPerSecond is not set"));
+  }
+  if !issetPerOperationStrategies{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field PerOperationStrategies is not set"));
+  }
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadDouble(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.DefaultSamplingProbability = v
+}
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadDouble(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.DefaultLowerBoundTracesPerSecond = v
+}
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*OperationSamplingStrategy, 0, size)
+  p.PerOperationStrategies =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem0 := &OperationSamplingStrategy{}
+    if err := _elem0.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
+    }
+    p.PerOperationStrategies = append(p.PerOperationStrategies, _elem0)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadDouble(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.DefaultUpperBoundTracesPerSecond = &v
+}
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "PerOperationSamplingStrategies"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *PerOperationSamplingStrategies) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "defaultSamplingProbability", thrift.DOUBLE, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:defaultSamplingProbability: ", p), err) }
+  if err := oprot.WriteDouble(ctx, float64(p.DefaultSamplingProbability)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.defaultSamplingProbability (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:defaultSamplingProbability: ", p), err) }
+  return err
+}
+
+func (p *PerOperationSamplingStrategies) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "defaultLowerBoundTracesPerSecond", thrift.DOUBLE, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:defaultLowerBoundTracesPerSecond: ", p), err) }
+  if err := oprot.WriteDouble(ctx, float64(p.DefaultLowerBoundTracesPerSecond)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.defaultLowerBoundTracesPerSecond (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:defaultLowerBoundTracesPerSecond: ", p), err) }
+  return err
+}
+
+func (p *PerOperationSamplingStrategies) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "perOperationStrategies", thrift.LIST, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:perOperationStrategies: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.PerOperationStrategies)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.PerOperationStrategies {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:perOperationStrategies: ", p), err) }
+  return err
+}
+
+func (p *PerOperationSamplingStrategies) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetDefaultUpperBoundTracesPerSecond() {
+    if err := oprot.WriteFieldBegin(ctx, "defaultUpperBoundTracesPerSecond", thrift.DOUBLE, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:defaultUpperBoundTracesPerSecond: ", p), err) }
+    if err := oprot.WriteDouble(ctx, float64(*p.DefaultUpperBoundTracesPerSecond)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.defaultUpperBoundTracesPerSecond (4) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:defaultUpperBoundTracesPerSecond: ", p), err) }
+  }
+  return err
+}
+
+func (p *PerOperationSamplingStrategies) Equals(other *PerOperationSamplingStrategies) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.DefaultSamplingProbability != other.DefaultSamplingProbability { return false }
+  if p.DefaultLowerBoundTracesPerSecond != other.DefaultLowerBoundTracesPerSecond { return false }
+  if len(p.PerOperationStrategies) != len(other.PerOperationStrategies) { return false }
+  for i, _tgt := range p.PerOperationStrategies {
+    _src1 := other.PerOperationStrategies[i]
+    if !_tgt.Equals(_src1) { return false }
+  }
+  if p.DefaultUpperBoundTracesPerSecond != other.DefaultUpperBoundTracesPerSecond {
+    if p.DefaultUpperBoundTracesPerSecond == nil || other.DefaultUpperBoundTracesPerSecond == nil {
+      return false
+    }
+    if (*p.DefaultUpperBoundTracesPerSecond) != (*other.DefaultUpperBoundTracesPerSecond) { return false }
+  }
+  return true
+}
+
+func (p *PerOperationSamplingStrategies) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("PerOperationSamplingStrategies(%+v)", *p)
+}
+
+// Attributes:
+//  - StrategyType
+//  - ProbabilisticSampling
+//  - RateLimitingSampling
+//  - OperationSampling
+type SamplingStrategyResponse struct {
+  StrategyType SamplingStrategyType `thrift:"strategyType,1,required" db:"strategyType" json:"strategyType"`
+  ProbabilisticSampling *ProbabilisticSamplingStrategy `thrift:"probabilisticSampling,2" db:"probabilisticSampling" json:"probabilisticSampling,omitempty"`
+  RateLimitingSampling *RateLimitingSamplingStrategy `thrift:"rateLimitingSampling,3" db:"rateLimitingSampling" json:"rateLimitingSampling,omitempty"`
+  OperationSampling *PerOperationSamplingStrategies `thrift:"operationSampling,4" db:"operationSampling" json:"operationSampling,omitempty"`
+}
+
+func NewSamplingStrategyResponse() *SamplingStrategyResponse {
+  return &SamplingStrategyResponse{}
+}
+
+
+func (p *SamplingStrategyResponse) GetStrategyType() SamplingStrategyType {
+  return p.StrategyType
+}
+var SamplingStrategyResponse_ProbabilisticSampling_DEFAULT *ProbabilisticSamplingStrategy
+func (p *SamplingStrategyResponse) GetProbabilisticSampling() *ProbabilisticSamplingStrategy {
+  if !p.IsSetProbabilisticSampling() {
+    return SamplingStrategyResponse_ProbabilisticSampling_DEFAULT
+  }
+return p.ProbabilisticSampling
+}
+var SamplingStrategyResponse_RateLimitingSampling_DEFAULT *RateLimitingSamplingStrategy
+func (p *SamplingStrategyResponse) GetRateLimitingSampling() *RateLimitingSamplingStrategy {
+  if !p.IsSetRateLimitingSampling() {
+    return SamplingStrategyResponse_RateLimitingSampling_DEFAULT
+  }
+return p.RateLimitingSampling
+}
+var SamplingStrategyResponse_OperationSampling_DEFAULT *PerOperationSamplingStrategies
+func (p *SamplingStrategyResponse) GetOperationSampling() *PerOperationSamplingStrategies {
+  if !p.IsSetOperationSampling() {
+    return SamplingStrategyResponse_OperationSampling_DEFAULT
+  }
+return p.OperationSampling
+}
+func (p *SamplingStrategyResponse) IsSetProbabilisticSampling() bool {
+  return p.ProbabilisticSampling != nil
+}
+
+func (p *SamplingStrategyResponse) IsSetRateLimitingSampling() bool {
+  return p.RateLimitingSampling != nil
+}
+
+func (p *SamplingStrategyResponse) IsSetOperationSampling() bool {
+  return p.OperationSampling != nil
+}
+
+func (p *SamplingStrategyResponse) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetStrategyType bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetStrategyType = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetStrategyType{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field StrategyType is not set"));
+  }
+  return nil
+}
+
+func (p *SamplingStrategyResponse)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  temp := SamplingStrategyType(v)
+  p.StrategyType = temp
+}
+  return nil
+}
+
+func (p *SamplingStrategyResponse)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  p.ProbabilisticSampling = &ProbabilisticSamplingStrategy{}
+  if err := p.ProbabilisticSampling.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.ProbabilisticSampling), err)
+  }
+  return nil
+}
+
+func (p *SamplingStrategyResponse)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  p.RateLimitingSampling = &RateLimitingSamplingStrategy{}
+  if err := p.RateLimitingSampling.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.RateLimitingSampling), err)
+  }
+  return nil
+}
+
+func (p *SamplingStrategyResponse)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  p.OperationSampling = &PerOperationSamplingStrategies{}
+  if err := p.OperationSampling.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.OperationSampling), err)
+  }
+  return nil
+}
+
+func (p *SamplingStrategyResponse) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "SamplingStrategyResponse"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *SamplingStrategyResponse) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "strategyType", thrift.I32, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:strategyType: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.StrategyType)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.strategyType (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:strategyType: ", p), err) }
+  return err
+}
+
+func (p *SamplingStrategyResponse) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetProbabilisticSampling() {
+    if err := oprot.WriteFieldBegin(ctx, "probabilisticSampling", thrift.STRUCT, 2); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:probabilisticSampling: ", p), err) }
+    if err := p.ProbabilisticSampling.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.ProbabilisticSampling), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 2:probabilisticSampling: ", p), err) }
+  }
+  return err
+}
+
+func (p *SamplingStrategyResponse) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetRateLimitingSampling() {
+    if err := oprot.WriteFieldBegin(ctx, "rateLimitingSampling", thrift.STRUCT, 3); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rateLimitingSampling: ", p), err) }
+    if err := p.RateLimitingSampling.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.RateLimitingSampling), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 3:rateLimitingSampling: ", p), err) }
+  }
+  return err
+}
+
+func (p *SamplingStrategyResponse) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetOperationSampling() {
+    if err := oprot.WriteFieldBegin(ctx, "operationSampling", thrift.STRUCT, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:operationSampling: ", p), err) }
+    if err := p.OperationSampling.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.OperationSampling), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:operationSampling: ", p), err) }
+  }
+  return err
+}
+
+func (p *SamplingStrategyResponse) Equals(other *SamplingStrategyResponse) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.StrategyType != other.StrategyType { return false }
+  if !p.ProbabilisticSampling.Equals(other.ProbabilisticSampling) { return false }
+  if !p.RateLimitingSampling.Equals(other.RateLimitingSampling) { return false }
+  if !p.OperationSampling.Equals(other.OperationSampling) { return false }
+  return true
+}
+
+func (p *SamplingStrategyResponse) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("SamplingStrategyResponse(%+v)", *p)
+}
+
+type SamplingManager interface {
+  // Parameters:
+  //  - ServiceName
+  GetSamplingStrategy(ctx context.Context, serviceName string) (_r *SamplingStrategyResponse, _err error)
+}
+
+type SamplingManagerClient struct {
+  c thrift.TClient
+  meta thrift.ResponseMeta
+}
+
+func NewSamplingManagerClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *SamplingManagerClient {
+  return &SamplingManagerClient{
+    c: thrift.NewTStandardClient(f.GetProtocol(t), f.GetProtocol(t)),
+  }
+}
+
+func NewSamplingManagerClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *SamplingManagerClient {
+  return &SamplingManagerClient{
+    c: thrift.NewTStandardClient(iprot, oprot),
+  }
+}
+
+func NewSamplingManagerClient(c thrift.TClient) *SamplingManagerClient {
+  return &SamplingManagerClient{
+    c: c,
+  }
+}
+
+func (p *SamplingManagerClient) Client_() thrift.TClient {
+  return p.c
+}
+
+func (p *SamplingManagerClient) LastResponseMeta_() thrift.ResponseMeta {
+  return p.meta
+}
+
+func (p *SamplingManagerClient) SetLastResponseMeta_(meta thrift.ResponseMeta) {
+  p.meta = meta
+}
+
+// Parameters:
+//  - ServiceName
+func (p *SamplingManagerClient) GetSamplingStrategy(ctx context.Context, serviceName string) (_r *SamplingStrategyResponse, _err error) {
+  var _args2 SamplingManagerGetSamplingStrategyArgs
+  _args2.ServiceName = serviceName
+  var _result4 SamplingManagerGetSamplingStrategyResult
+  var _meta3 thrift.ResponseMeta
+  _meta3, _err = p.Client_().Call(ctx, "getSamplingStrategy", &_args2, &_result4)
+  p.SetLastResponseMeta_(_meta3)
+  if _err != nil {
+    return
+  }
+  return _result4.GetSuccess(), nil
+}
+
+type SamplingManagerProcessor struct {
+  processorMap map[string]thrift.TProcessorFunction
+  handler SamplingManager
+}
+
+func (p *SamplingManagerProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
+  p.processorMap[key] = processor
+}
+
+func (p *SamplingManagerProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
+  processor, ok = p.processorMap[key]
+  return processor, ok
+}
+
+func (p *SamplingManagerProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
+  return p.processorMap
+}
+
+func NewSamplingManagerProcessor(handler SamplingManager) *SamplingManagerProcessor {
+
+  self5 := &SamplingManagerProcessor{handler:handler, processorMap:make(map[string]thrift.TProcessorFunction)}
+  self5.processorMap["getSamplingStrategy"] = &samplingManagerProcessorGetSamplingStrategy{handler:handler}
+return self5
+}
+
+func (p *SamplingManagerProcessor) Process(ctx context.Context, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  name, _, seqId, err2 := iprot.ReadMessageBegin(ctx)
+  if err2 != nil { return false, thrift.WrapTException(err2) }
+  if processor, ok := p.GetProcessorFunction(name); ok {
+    return processor.Process(ctx, seqId, iprot, oprot)
+  }
+  iprot.Skip(ctx, thrift.STRUCT)
+  iprot.ReadMessageEnd(ctx)
+  x6 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function " + name)
+  oprot.WriteMessageBegin(ctx, name, thrift.EXCEPTION, seqId)
+  x6.Write(ctx, oprot)
+  oprot.WriteMessageEnd(ctx)
+  oprot.Flush(ctx)
+  return false, x6
+
+}
+
+type samplingManagerProcessorGetSamplingStrategy struct {
+  handler SamplingManager
+}
+
+func (p *samplingManagerProcessorGetSamplingStrategy) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := SamplingManagerGetSamplingStrategyArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err2.Error())
+    oprot.WriteMessageBegin(ctx, "getSamplingStrategy", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
+
+  tickerCancel := func() {}
+  // Start a goroutine to do server side connectivity check.
+  if thrift.ServerConnectivityCheckInterval > 0 {
+    var cancel context.CancelFunc
+    ctx, cancel = context.WithCancel(ctx)
+    defer cancel()
+    var tickerCtx context.Context
+    tickerCtx, tickerCancel = context.WithCancel(context.Background())
+    defer tickerCancel()
+    go func(ctx context.Context, cancel context.CancelFunc) {
+      ticker := time.NewTicker(thrift.ServerConnectivityCheckInterval)
+      defer ticker.Stop()
+      for {
+        select {
+        case <-ctx.Done():
+          return
+        case <-ticker.C:
+          if !iprot.Transport().IsOpen() {
+            cancel()
+            return
+          }
+        }
+      }
+    }(tickerCtx, cancel)
+  }
+
+  result := SamplingManagerGetSamplingStrategyResult{}
+  var retval *SamplingStrategyResponse
+  if retval, err2 = p.handler.GetSamplingStrategy(ctx, args.ServiceName); err2 != nil {
+    tickerCancel()
+    if err2 == thrift.ErrAbandonRequest {
+      return false, thrift.WrapTException(err2)
+    }
+    x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getSamplingStrategy: " + err2.Error())
+    oprot.WriteMessageBegin(ctx, "getSamplingStrategy", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return true, thrift.WrapTException(err2)
+  } else {
+    result.Success = retval
+  }
+  tickerCancel()
+  if err2 = oprot.WriteMessageBegin(ctx, "getSamplingStrategy", thrift.REPLY, seqId); err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = result.Write(ctx, oprot); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.WriteMessageEnd(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.Flush(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err != nil {
+    return
+  }
+  return true, err
+}
+
+
+// HELPER FUNCTIONS AND STRUCTURES
+
+// Attributes:
+//  - ServiceName
+type SamplingManagerGetSamplingStrategyArgs struct {
+  ServiceName string `thrift:"serviceName,1" db:"serviceName" json:"serviceName"`
+}
+
+func NewSamplingManagerGetSamplingStrategyArgs() *SamplingManagerGetSamplingStrategyArgs {
+  return &SamplingManagerGetSamplingStrategyArgs{}
+}
+
+
+func (p *SamplingManagerGetSamplingStrategyArgs) GetServiceName() string {
+  return p.ServiceName
+}
+func (p *SamplingManagerGetSamplingStrategyArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.ServiceName = v
+}
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "getSamplingStrategy_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "serviceName", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.ServiceName)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err) }
+  return err
+}
+
+func (p *SamplingManagerGetSamplingStrategyArgs) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("SamplingManagerGetSamplingStrategyArgs(%+v)", *p)
+}
+
+// Attributes:
+//  - Success
+type SamplingManagerGetSamplingStrategyResult struct {
+  Success *SamplingStrategyResponse `thrift:"success,0" db:"success" json:"success,omitempty"`
+}
+
+func NewSamplingManagerGetSamplingStrategyResult() *SamplingManagerGetSamplingStrategyResult {
+  return &SamplingManagerGetSamplingStrategyResult{}
+}
+
+var SamplingManagerGetSamplingStrategyResult_Success_DEFAULT *SamplingStrategyResponse
+func (p *SamplingManagerGetSamplingStrategyResult) GetSuccess() *SamplingStrategyResponse {
+  if !p.IsSetSuccess() {
+    return SamplingManagerGetSamplingStrategyResult_Success_DEFAULT
+  }
+return p.Success
+}
+func (p *SamplingManagerGetSamplingStrategyResult) IsSetSuccess() bool {
+  return p.Success != nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyResult) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 0:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField0(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyResult)  ReadField0(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Success = &SamplingStrategyResponse{}
+  if err := p.Success.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err)
+  }
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyResult) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "getSamplingStrategy_result"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField0(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *SamplingManagerGetSamplingStrategyResult) writeField0(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetSuccess() {
+    if err := oprot.WriteFieldBegin(ctx, "success", thrift.STRUCT, 0); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) }
+    if err := p.Success.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) }
+  }
+  return err
+}
+
+func (p *SamplingManagerGetSamplingStrategyResult) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("SamplingManagerGetSamplingStrategyResult(%+v)", *p)
+}
+
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/samplingmanager.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/samplingmanager.go
deleted file mode 100644
index 33179cf..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/samplingmanager.go
+++ /dev/null
@@ -1,410 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package sampling
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-type SamplingManager interface {
-	// Parameters:
-	//  - ServiceName
-	GetSamplingStrategy(serviceName string) (r *SamplingStrategyResponse, err error)
-}
-
-type SamplingManagerClient struct {
-	Transport       thrift.TTransport
-	ProtocolFactory thrift.TProtocolFactory
-	InputProtocol   thrift.TProtocol
-	OutputProtocol  thrift.TProtocol
-	SeqId           int32
-}
-
-func NewSamplingManagerClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *SamplingManagerClient {
-	return &SamplingManagerClient{Transport: t,
-		ProtocolFactory: f,
-		InputProtocol:   f.GetProtocol(t),
-		OutputProtocol:  f.GetProtocol(t),
-		SeqId:           0,
-	}
-}
-
-func NewSamplingManagerClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *SamplingManagerClient {
-	return &SamplingManagerClient{Transport: t,
-		ProtocolFactory: nil,
-		InputProtocol:   iprot,
-		OutputProtocol:  oprot,
-		SeqId:           0,
-	}
-}
-
-// Parameters:
-//  - ServiceName
-func (p *SamplingManagerClient) GetSamplingStrategy(serviceName string) (r *SamplingStrategyResponse, err error) {
-	if err = p.sendGetSamplingStrategy(serviceName); err != nil {
-		return
-	}
-	return p.recvGetSamplingStrategy()
-}
-
-func (p *SamplingManagerClient) sendGetSamplingStrategy(serviceName string) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("getSamplingStrategy", thrift.CALL, p.SeqId); err != nil {
-		return
-	}
-	args := SamplingManagerGetSamplingStrategyArgs{
-		ServiceName: serviceName,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
-}
-
-func (p *SamplingManagerClient) recvGetSamplingStrategy() (value *SamplingStrategyResponse, err error) {
-	iprot := p.InputProtocol
-	if iprot == nil {
-		iprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.InputProtocol = iprot
-	}
-	method, mTypeId, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return
-	}
-	if method != "getSamplingStrategy" {
-		err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getSamplingStrategy failed: wrong method name")
-		return
-	}
-	if p.SeqId != seqId {
-		err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getSamplingStrategy failed: out of sequence response")
-		return
-	}
-	if mTypeId == thrift.EXCEPTION {
-		error1 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception")
-		var error2 error
-		error2, err = error1.Read(iprot)
-		if err != nil {
-			return
-		}
-		if err = iprot.ReadMessageEnd(); err != nil {
-			return
-		}
-		err = error2
-		return
-	}
-	if mTypeId != thrift.REPLY {
-		err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getSamplingStrategy failed: invalid message type")
-		return
-	}
-	result := SamplingManagerGetSamplingStrategyResult{}
-	if err = result.Read(iprot); err != nil {
-		return
-	}
-	if err = iprot.ReadMessageEnd(); err != nil {
-		return
-	}
-	value = result.GetSuccess()
-	return
-}
-
-type SamplingManagerProcessor struct {
-	processorMap map[string]thrift.TProcessorFunction
-	handler      SamplingManager
-}
-
-func (p *SamplingManagerProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
-	p.processorMap[key] = processor
-}
-
-func (p *SamplingManagerProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
-	processor, ok = p.processorMap[key]
-	return processor, ok
-}
-
-func (p *SamplingManagerProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
-	return p.processorMap
-}
-
-func NewSamplingManagerProcessor(handler SamplingManager) *SamplingManagerProcessor {
-
-	self3 := &SamplingManagerProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
-	self3.processorMap["getSamplingStrategy"] = &samplingManagerProcessorGetSamplingStrategy{handler: handler}
-	return self3
-}
-
-func (p *SamplingManagerProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	name, _, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return false, err
-	}
-	if processor, ok := p.GetProcessorFunction(name); ok {
-		return processor.Process(seqId, iprot, oprot)
-	}
-	iprot.Skip(thrift.STRUCT)
-	iprot.ReadMessageEnd()
-	x4 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
-	oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
-	x4.Write(oprot)
-	oprot.WriteMessageEnd()
-	oprot.Flush()
-	return false, x4
-
-}
-
-type samplingManagerProcessorGetSamplingStrategy struct {
-	handler SamplingManager
-}
-
-func (p *samplingManagerProcessorGetSamplingStrategy) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := SamplingManagerGetSamplingStrategyArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error())
-		oprot.WriteMessageBegin("getSamplingStrategy", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return false, err
-	}
-
-	iprot.ReadMessageEnd()
-	result := SamplingManagerGetSamplingStrategyResult{}
-	var retval *SamplingStrategyResponse
-	var err2 error
-	if retval, err2 = p.handler.GetSamplingStrategy(args.ServiceName); err2 != nil {
-		x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getSamplingStrategy: "+err2.Error())
-		oprot.WriteMessageBegin("getSamplingStrategy", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return true, err2
-	} else {
-		result.Success = retval
-	}
-	if err2 = oprot.WriteMessageBegin("getSamplingStrategy", thrift.REPLY, seqId); err2 != nil {
-		err = err2
-	}
-	if err2 = result.Write(oprot); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.Flush(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err != nil {
-		return
-	}
-	return true, err
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
-
-// Attributes:
-//  - ServiceName
-type SamplingManagerGetSamplingStrategyArgs struct {
-	ServiceName string `thrift:"serviceName,1" json:"serviceName"`
-}
-
-func NewSamplingManagerGetSamplingStrategyArgs() *SamplingManagerGetSamplingStrategyArgs {
-	return &SamplingManagerGetSamplingStrategyArgs{}
-}
-
-func (p *SamplingManagerGetSamplingStrategyArgs) GetServiceName() string {
-	return p.ServiceName
-}
-func (p *SamplingManagerGetSamplingStrategyArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyArgs) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.ServiceName = v
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("getSamplingStrategy_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("serviceName", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:serviceName: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.ServiceName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.serviceName (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:serviceName: ", p), err)
-	}
-	return err
-}
-
-func (p *SamplingManagerGetSamplingStrategyArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("SamplingManagerGetSamplingStrategyArgs(%+v)", *p)
-}
-
-// Attributes:
-//  - Success
-type SamplingManagerGetSamplingStrategyResult struct {
-	Success *SamplingStrategyResponse `thrift:"success,0" json:"success,omitempty"`
-}
-
-func NewSamplingManagerGetSamplingStrategyResult() *SamplingManagerGetSamplingStrategyResult {
-	return &SamplingManagerGetSamplingStrategyResult{}
-}
-
-var SamplingManagerGetSamplingStrategyResult_Success_DEFAULT *SamplingStrategyResponse
-
-func (p *SamplingManagerGetSamplingStrategyResult) GetSuccess() *SamplingStrategyResponse {
-	if !p.IsSetSuccess() {
-		return SamplingManagerGetSamplingStrategyResult_Success_DEFAULT
-	}
-	return p.Success
-}
-func (p *SamplingManagerGetSamplingStrategyResult) IsSetSuccess() bool {
-	return p.Success != nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyResult) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 0:
-			if err := p.readField0(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyResult) readField0(iprot thrift.TProtocol) error {
-	p.Success = &SamplingStrategyResponse{}
-	if err := p.Success.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err)
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyResult) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("getSamplingStrategy_result"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField0(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *SamplingManagerGetSamplingStrategyResult) writeField0(oprot thrift.TProtocol) (err error) {
-	if p.IsSetSuccess() {
-		if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err)
-		}
-		if err := p.Success.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *SamplingManagerGetSamplingStrategyResult) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("SamplingManagerGetSamplingStrategyResult(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/ttypes.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/ttypes.go
deleted file mode 100644
index 9abaf05..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/sampling/ttypes.go
+++ /dev/null
@@ -1,873 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package sampling
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var GoUnusedProtection__ int
-
-type SamplingStrategyType int64
-
-const (
-	SamplingStrategyType_PROBABILISTIC SamplingStrategyType = 0
-	SamplingStrategyType_RATE_LIMITING SamplingStrategyType = 1
-)
-
-func (p SamplingStrategyType) String() string {
-	switch p {
-	case SamplingStrategyType_PROBABILISTIC:
-		return "PROBABILISTIC"
-	case SamplingStrategyType_RATE_LIMITING:
-		return "RATE_LIMITING"
-	}
-	return "<UNSET>"
-}
-
-func SamplingStrategyTypeFromString(s string) (SamplingStrategyType, error) {
-	switch s {
-	case "PROBABILISTIC":
-		return SamplingStrategyType_PROBABILISTIC, nil
-	case "RATE_LIMITING":
-		return SamplingStrategyType_RATE_LIMITING, nil
-	}
-	return SamplingStrategyType(0), fmt.Errorf("not a valid SamplingStrategyType string")
-}
-
-func SamplingStrategyTypePtr(v SamplingStrategyType) *SamplingStrategyType { return &v }
-
-func (p SamplingStrategyType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *SamplingStrategyType) UnmarshalText(text []byte) error {
-	q, err := SamplingStrategyTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-// Attributes:
-//  - SamplingRate
-type ProbabilisticSamplingStrategy struct {
-	SamplingRate float64 `thrift:"samplingRate,1,required" json:"samplingRate"`
-}
-
-func NewProbabilisticSamplingStrategy() *ProbabilisticSamplingStrategy {
-	return &ProbabilisticSamplingStrategy{}
-}
-
-func (p *ProbabilisticSamplingStrategy) GetSamplingRate() float64 {
-	return p.SamplingRate
-}
-func (p *ProbabilisticSamplingStrategy) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetSamplingRate bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetSamplingRate = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetSamplingRate {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field SamplingRate is not set"))
-	}
-	return nil
-}
-
-func (p *ProbabilisticSamplingStrategy) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadDouble(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.SamplingRate = v
-	}
-	return nil
-}
-
-func (p *ProbabilisticSamplingStrategy) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("ProbabilisticSamplingStrategy"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ProbabilisticSamplingStrategy) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("samplingRate", thrift.DOUBLE, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:samplingRate: ", p), err)
-	}
-	if err := oprot.WriteDouble(float64(p.SamplingRate)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.samplingRate (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:samplingRate: ", p), err)
-	}
-	return err
-}
-
-func (p *ProbabilisticSamplingStrategy) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ProbabilisticSamplingStrategy(%+v)", *p)
-}
-
-// Attributes:
-//  - MaxTracesPerSecond
-type RateLimitingSamplingStrategy struct {
-	MaxTracesPerSecond int16 `thrift:"maxTracesPerSecond,1,required" json:"maxTracesPerSecond"`
-}
-
-func NewRateLimitingSamplingStrategy() *RateLimitingSamplingStrategy {
-	return &RateLimitingSamplingStrategy{}
-}
-
-func (p *RateLimitingSamplingStrategy) GetMaxTracesPerSecond() int16 {
-	return p.MaxTracesPerSecond
-}
-func (p *RateLimitingSamplingStrategy) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetMaxTracesPerSecond bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetMaxTracesPerSecond = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetMaxTracesPerSecond {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field MaxTracesPerSecond is not set"))
-	}
-	return nil
-}
-
-func (p *RateLimitingSamplingStrategy) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI16(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.MaxTracesPerSecond = v
-	}
-	return nil
-}
-
-func (p *RateLimitingSamplingStrategy) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("RateLimitingSamplingStrategy"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *RateLimitingSamplingStrategy) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("maxTracesPerSecond", thrift.I16, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:maxTracesPerSecond: ", p), err)
-	}
-	if err := oprot.WriteI16(int16(p.MaxTracesPerSecond)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.maxTracesPerSecond (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:maxTracesPerSecond: ", p), err)
-	}
-	return err
-}
-
-func (p *RateLimitingSamplingStrategy) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("RateLimitingSamplingStrategy(%+v)", *p)
-}
-
-// Attributes:
-//  - Operation
-//  - ProbabilisticSampling
-type OperationSamplingStrategy struct {
-	Operation             string                         `thrift:"operation,1,required" json:"operation"`
-	ProbabilisticSampling *ProbabilisticSamplingStrategy `thrift:"probabilisticSampling,2,required" json:"probabilisticSampling"`
-}
-
-func NewOperationSamplingStrategy() *OperationSamplingStrategy {
-	return &OperationSamplingStrategy{}
-}
-
-func (p *OperationSamplingStrategy) GetOperation() string {
-	return p.Operation
-}
-
-var OperationSamplingStrategy_ProbabilisticSampling_DEFAULT *ProbabilisticSamplingStrategy
-
-func (p *OperationSamplingStrategy) GetProbabilisticSampling() *ProbabilisticSamplingStrategy {
-	if !p.IsSetProbabilisticSampling() {
-		return OperationSamplingStrategy_ProbabilisticSampling_DEFAULT
-	}
-	return p.ProbabilisticSampling
-}
-func (p *OperationSamplingStrategy) IsSetProbabilisticSampling() bool {
-	return p.ProbabilisticSampling != nil
-}
-
-func (p *OperationSamplingStrategy) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetOperation bool = false
-	var issetProbabilisticSampling bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetOperation = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetProbabilisticSampling = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetOperation {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Operation is not set"))
-	}
-	if !issetProbabilisticSampling {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ProbabilisticSampling is not set"))
-	}
-	return nil
-}
-
-func (p *OperationSamplingStrategy) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Operation = v
-	}
-	return nil
-}
-
-func (p *OperationSamplingStrategy) readField2(iprot thrift.TProtocol) error {
-	p.ProbabilisticSampling = &ProbabilisticSamplingStrategy{}
-	if err := p.ProbabilisticSampling.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.ProbabilisticSampling), err)
-	}
-	return nil
-}
-
-func (p *OperationSamplingStrategy) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("OperationSamplingStrategy"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *OperationSamplingStrategy) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("operation", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:operation: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Operation)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.operation (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:operation: ", p), err)
-	}
-	return err
-}
-
-func (p *OperationSamplingStrategy) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("probabilisticSampling", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:probabilisticSampling: ", p), err)
-	}
-	if err := p.ProbabilisticSampling.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.ProbabilisticSampling), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:probabilisticSampling: ", p), err)
-	}
-	return err
-}
-
-func (p *OperationSamplingStrategy) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("OperationSamplingStrategy(%+v)", *p)
-}
-
-// Attributes:
-//  - DefaultSamplingProbability
-//  - DefaultLowerBoundTracesPerSecond
-//  - PerOperationStrategies
-//  - DefaultUpperBoundTracesPerSecond
-type PerOperationSamplingStrategies struct {
-	DefaultSamplingProbability       float64                      `thrift:"defaultSamplingProbability,1,required" json:"defaultSamplingProbability"`
-	DefaultLowerBoundTracesPerSecond float64                      `thrift:"defaultLowerBoundTracesPerSecond,2,required" json:"defaultLowerBoundTracesPerSecond"`
-	PerOperationStrategies           []*OperationSamplingStrategy `thrift:"perOperationStrategies,3,required" json:"perOperationStrategies"`
-	DefaultUpperBoundTracesPerSecond *float64                     `thrift:"defaultUpperBoundTracesPerSecond,4" json:"defaultUpperBoundTracesPerSecond,omitempty"`
-}
-
-func NewPerOperationSamplingStrategies() *PerOperationSamplingStrategies {
-	return &PerOperationSamplingStrategies{}
-}
-
-func (p *PerOperationSamplingStrategies) GetDefaultSamplingProbability() float64 {
-	return p.DefaultSamplingProbability
-}
-
-func (p *PerOperationSamplingStrategies) GetDefaultLowerBoundTracesPerSecond() float64 {
-	return p.DefaultLowerBoundTracesPerSecond
-}
-
-func (p *PerOperationSamplingStrategies) GetPerOperationStrategies() []*OperationSamplingStrategy {
-	return p.PerOperationStrategies
-}
-
-var PerOperationSamplingStrategies_DefaultUpperBoundTracesPerSecond_DEFAULT float64
-
-func (p *PerOperationSamplingStrategies) GetDefaultUpperBoundTracesPerSecond() float64 {
-	if !p.IsSetDefaultUpperBoundTracesPerSecond() {
-		return PerOperationSamplingStrategies_DefaultUpperBoundTracesPerSecond_DEFAULT
-	}
-	return *p.DefaultUpperBoundTracesPerSecond
-}
-func (p *PerOperationSamplingStrategies) IsSetDefaultUpperBoundTracesPerSecond() bool {
-	return p.DefaultUpperBoundTracesPerSecond != nil
-}
-
-func (p *PerOperationSamplingStrategies) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetDefaultSamplingProbability bool = false
-	var issetDefaultLowerBoundTracesPerSecond bool = false
-	var issetPerOperationStrategies bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetDefaultSamplingProbability = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-			issetDefaultLowerBoundTracesPerSecond = true
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-			issetPerOperationStrategies = true
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetDefaultSamplingProbability {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DefaultSamplingProbability is not set"))
-	}
-	if !issetDefaultLowerBoundTracesPerSecond {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DefaultLowerBoundTracesPerSecond is not set"))
-	}
-	if !issetPerOperationStrategies {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field PerOperationStrategies is not set"))
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadDouble(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.DefaultSamplingProbability = v
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadDouble(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.DefaultLowerBoundTracesPerSecond = v
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) readField3(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*OperationSamplingStrategy, 0, size)
-	p.PerOperationStrategies = tSlice
-	for i := 0; i < size; i++ {
-		_elem0 := &OperationSamplingStrategy{}
-		if err := _elem0.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
-		}
-		p.PerOperationStrategies = append(p.PerOperationStrategies, _elem0)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadDouble(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.DefaultUpperBoundTracesPerSecond = &v
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("PerOperationSamplingStrategies"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *PerOperationSamplingStrategies) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("defaultSamplingProbability", thrift.DOUBLE, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:defaultSamplingProbability: ", p), err)
-	}
-	if err := oprot.WriteDouble(float64(p.DefaultSamplingProbability)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.defaultSamplingProbability (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:defaultSamplingProbability: ", p), err)
-	}
-	return err
-}
-
-func (p *PerOperationSamplingStrategies) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("defaultLowerBoundTracesPerSecond", thrift.DOUBLE, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:defaultLowerBoundTracesPerSecond: ", p), err)
-	}
-	if err := oprot.WriteDouble(float64(p.DefaultLowerBoundTracesPerSecond)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.defaultLowerBoundTracesPerSecond (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:defaultLowerBoundTracesPerSecond: ", p), err)
-	}
-	return err
-}
-
-func (p *PerOperationSamplingStrategies) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("perOperationStrategies", thrift.LIST, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:perOperationStrategies: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.PerOperationStrategies)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.PerOperationStrategies {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:perOperationStrategies: ", p), err)
-	}
-	return err
-}
-
-func (p *PerOperationSamplingStrategies) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetDefaultUpperBoundTracesPerSecond() {
-		if err := oprot.WriteFieldBegin("defaultUpperBoundTracesPerSecond", thrift.DOUBLE, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:defaultUpperBoundTracesPerSecond: ", p), err)
-		}
-		if err := oprot.WriteDouble(float64(*p.DefaultUpperBoundTracesPerSecond)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.defaultUpperBoundTracesPerSecond (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:defaultUpperBoundTracesPerSecond: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *PerOperationSamplingStrategies) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("PerOperationSamplingStrategies(%+v)", *p)
-}
-
-// Attributes:
-//  - StrategyType
-//  - ProbabilisticSampling
-//  - RateLimitingSampling
-//  - OperationSampling
-type SamplingStrategyResponse struct {
-	StrategyType          SamplingStrategyType            `thrift:"strategyType,1,required" json:"strategyType"`
-	ProbabilisticSampling *ProbabilisticSamplingStrategy  `thrift:"probabilisticSampling,2" json:"probabilisticSampling,omitempty"`
-	RateLimitingSampling  *RateLimitingSamplingStrategy   `thrift:"rateLimitingSampling,3" json:"rateLimitingSampling,omitempty"`
-	OperationSampling     *PerOperationSamplingStrategies `thrift:"operationSampling,4" json:"operationSampling,omitempty"`
-}
-
-func NewSamplingStrategyResponse() *SamplingStrategyResponse {
-	return &SamplingStrategyResponse{}
-}
-
-func (p *SamplingStrategyResponse) GetStrategyType() SamplingStrategyType {
-	return p.StrategyType
-}
-
-var SamplingStrategyResponse_ProbabilisticSampling_DEFAULT *ProbabilisticSamplingStrategy
-
-func (p *SamplingStrategyResponse) GetProbabilisticSampling() *ProbabilisticSamplingStrategy {
-	if !p.IsSetProbabilisticSampling() {
-		return SamplingStrategyResponse_ProbabilisticSampling_DEFAULT
-	}
-	return p.ProbabilisticSampling
-}
-
-var SamplingStrategyResponse_RateLimitingSampling_DEFAULT *RateLimitingSamplingStrategy
-
-func (p *SamplingStrategyResponse) GetRateLimitingSampling() *RateLimitingSamplingStrategy {
-	if !p.IsSetRateLimitingSampling() {
-		return SamplingStrategyResponse_RateLimitingSampling_DEFAULT
-	}
-	return p.RateLimitingSampling
-}
-
-var SamplingStrategyResponse_OperationSampling_DEFAULT *PerOperationSamplingStrategies
-
-func (p *SamplingStrategyResponse) GetOperationSampling() *PerOperationSamplingStrategies {
-	if !p.IsSetOperationSampling() {
-		return SamplingStrategyResponse_OperationSampling_DEFAULT
-	}
-	return p.OperationSampling
-}
-func (p *SamplingStrategyResponse) IsSetProbabilisticSampling() bool {
-	return p.ProbabilisticSampling != nil
-}
-
-func (p *SamplingStrategyResponse) IsSetRateLimitingSampling() bool {
-	return p.RateLimitingSampling != nil
-}
-
-func (p *SamplingStrategyResponse) IsSetOperationSampling() bool {
-	return p.OperationSampling != nil
-}
-
-func (p *SamplingStrategyResponse) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetStrategyType bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetStrategyType = true
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetStrategyType {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field StrategyType is not set"))
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := SamplingStrategyType(v)
-		p.StrategyType = temp
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) readField2(iprot thrift.TProtocol) error {
-	p.ProbabilisticSampling = &ProbabilisticSamplingStrategy{}
-	if err := p.ProbabilisticSampling.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.ProbabilisticSampling), err)
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) readField3(iprot thrift.TProtocol) error {
-	p.RateLimitingSampling = &RateLimitingSamplingStrategy{}
-	if err := p.RateLimitingSampling.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.RateLimitingSampling), err)
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) readField4(iprot thrift.TProtocol) error {
-	p.OperationSampling = &PerOperationSamplingStrategies{}
-	if err := p.OperationSampling.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.OperationSampling), err)
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("SamplingStrategyResponse"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *SamplingStrategyResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("strategyType", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:strategyType: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.StrategyType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.strategyType (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:strategyType: ", p), err)
-	}
-	return err
-}
-
-func (p *SamplingStrategyResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if p.IsSetProbabilisticSampling() {
-		if err := oprot.WriteFieldBegin("probabilisticSampling", thrift.STRUCT, 2); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:probabilisticSampling: ", p), err)
-		}
-		if err := p.ProbabilisticSampling.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.ProbabilisticSampling), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 2:probabilisticSampling: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *SamplingStrategyResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetRateLimitingSampling() {
-		if err := oprot.WriteFieldBegin("rateLimitingSampling", thrift.STRUCT, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rateLimitingSampling: ", p), err)
-		}
-		if err := p.RateLimitingSampling.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.RateLimitingSampling), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:rateLimitingSampling: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *SamplingStrategyResponse) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetOperationSampling() {
-		if err := oprot.WriteFieldBegin("operationSampling", thrift.STRUCT, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:operationSampling: ", p), err)
-		}
-		if err := p.OperationSampling.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.OperationSampling), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:operationSampling: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *SamplingStrategyResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("SamplingStrategyResponse(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/GoUnusedProtection__.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/GoUnusedProtection__.go
new file mode 100644
index 0000000..ebf4301
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/GoUnusedProtection__.go
@@ -0,0 +1,6 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package zipkincore
+
+var GoUnusedProtection__ int;
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/constants.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/constants.go
deleted file mode 100644
index a53d46f..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/constants.go
+++ /dev/null
@@ -1,35 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package zipkincore
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-const CLIENT_SEND = "cs"
-const CLIENT_RECV = "cr"
-const SERVER_SEND = "ss"
-const SERVER_RECV = "sr"
-const MESSAGE_SEND = "ms"
-const MESSAGE_RECV = "mr"
-const WIRE_SEND = "ws"
-const WIRE_RECV = "wr"
-const CLIENT_SEND_FRAGMENT = "csf"
-const CLIENT_RECV_FRAGMENT = "crf"
-const SERVER_SEND_FRAGMENT = "ssf"
-const SERVER_RECV_FRAGMENT = "srf"
-const LOCAL_COMPONENT = "lc"
-const CLIENT_ADDR = "ca"
-const SERVER_ADDR = "sa"
-const MESSAGE_ADDR = "ma"
-
-func init() {
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/ttypes.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/ttypes.go
deleted file mode 100644
index 15583e5..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/ttypes.go
+++ /dev/null
@@ -1,1337 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package zipkincore
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-var GoUnusedProtection__ int
-
-type AnnotationType int64
-
-const (
-	AnnotationType_BOOL   AnnotationType = 0
-	AnnotationType_BYTES  AnnotationType = 1
-	AnnotationType_I16    AnnotationType = 2
-	AnnotationType_I32    AnnotationType = 3
-	AnnotationType_I64    AnnotationType = 4
-	AnnotationType_DOUBLE AnnotationType = 5
-	AnnotationType_STRING AnnotationType = 6
-)
-
-func (p AnnotationType) String() string {
-	switch p {
-	case AnnotationType_BOOL:
-		return "BOOL"
-	case AnnotationType_BYTES:
-		return "BYTES"
-	case AnnotationType_I16:
-		return "I16"
-	case AnnotationType_I32:
-		return "I32"
-	case AnnotationType_I64:
-		return "I64"
-	case AnnotationType_DOUBLE:
-		return "DOUBLE"
-	case AnnotationType_STRING:
-		return "STRING"
-	}
-	return "<UNSET>"
-}
-
-func AnnotationTypeFromString(s string) (AnnotationType, error) {
-	switch s {
-	case "BOOL":
-		return AnnotationType_BOOL, nil
-	case "BYTES":
-		return AnnotationType_BYTES, nil
-	case "I16":
-		return AnnotationType_I16, nil
-	case "I32":
-		return AnnotationType_I32, nil
-	case "I64":
-		return AnnotationType_I64, nil
-	case "DOUBLE":
-		return AnnotationType_DOUBLE, nil
-	case "STRING":
-		return AnnotationType_STRING, nil
-	}
-	return AnnotationType(0), fmt.Errorf("not a valid AnnotationType string")
-}
-
-func AnnotationTypePtr(v AnnotationType) *AnnotationType { return &v }
-
-func (p AnnotationType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *AnnotationType) UnmarshalText(text []byte) error {
-	q, err := AnnotationTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-// Indicates the network context of a service recording an annotation with two
-// exceptions.
-//
-// When a BinaryAnnotation, and key is CLIENT_ADDR or SERVER_ADDR,
-// the endpoint indicates the source or destination of an RPC. This exception
-// allows zipkin to display network context of uninstrumented services, or
-// clients such as web browsers.
-//
-// Attributes:
-//  - Ipv4: IPv4 host address packed into 4 bytes.
-//
-// Ex for the ip 1.2.3.4, it would be (1 << 24) | (2 << 16) | (3 << 8) | 4
-//  - Port: IPv4 port
-//
-// Note: this is to be treated as an unsigned integer, so watch for negatives.
-//
-// Conventionally, when the port isn't known, port = 0.
-//  - ServiceName: Service name in lowercase, such as "memcache" or "zipkin-web"
-//
-// Conventionally, when the service name isn't known, service_name = "unknown".
-//  - Ipv6: IPv6 host address packed into 16 bytes. Ex Inet6Address.getBytes()
-type Endpoint struct {
-	Ipv4        int32  `thrift:"ipv4,1" json:"ipv4"`
-	Port        int16  `thrift:"port,2" json:"port"`
-	ServiceName string `thrift:"service_name,3" json:"service_name"`
-	Ipv6        []byte `thrift:"ipv6,4" json:"ipv6,omitempty"`
-}
-
-func NewEndpoint() *Endpoint {
-	return &Endpoint{}
-}
-
-func (p *Endpoint) GetIpv4() int32 {
-	return p.Ipv4
-}
-
-func (p *Endpoint) GetPort() int16 {
-	return p.Port
-}
-
-func (p *Endpoint) GetServiceName() string {
-	return p.ServiceName
-}
-
-var Endpoint_Ipv6_DEFAULT []byte
-
-func (p *Endpoint) GetIpv6() []byte {
-	return p.Ipv6
-}
-func (p *Endpoint) IsSetIpv6() bool {
-	return p.Ipv6 != nil
-}
-
-func (p *Endpoint) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *Endpoint) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Ipv4 = v
-	}
-	return nil
-}
-
-func (p *Endpoint) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI16(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Port = v
-	}
-	return nil
-}
-
-func (p *Endpoint) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.ServiceName = v
-	}
-	return nil
-}
-
-func (p *Endpoint) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBinary(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.Ipv6 = v
-	}
-	return nil
-}
-
-func (p *Endpoint) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Endpoint"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Endpoint) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("ipv4", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ipv4: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Ipv4)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.ipv4 (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ipv4: ", p), err)
-	}
-	return err
-}
-
-func (p *Endpoint) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("port", thrift.I16, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:port: ", p), err)
-	}
-	if err := oprot.WriteI16(int16(p.Port)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.port (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:port: ", p), err)
-	}
-	return err
-}
-
-func (p *Endpoint) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("service_name", thrift.STRING, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:service_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.ServiceName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.service_name (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:service_name: ", p), err)
-	}
-	return err
-}
-
-func (p *Endpoint) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetIpv6() {
-		if err := oprot.WriteFieldBegin("ipv6", thrift.STRING, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:ipv6: ", p), err)
-		}
-		if err := oprot.WriteBinary(p.Ipv6); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.ipv6 (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:ipv6: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Endpoint) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Endpoint(%+v)", *p)
-}
-
-// An annotation is similar to a log statement. It includes a host field which
-// allows these events to be attributed properly, and also aggregatable.
-//
-// Attributes:
-//  - Timestamp: Microseconds from epoch.
-//
-// This value should use the most precise value possible. For example,
-// gettimeofday or syncing nanoTime against a tick of currentTimeMillis.
-//  - Value
-//  - Host: Always the host that recorded the event. By specifying the host you allow
-// rollup of all events (such as client requests to a service) by IP address.
-type Annotation struct {
-	Timestamp int64     `thrift:"timestamp,1" json:"timestamp"`
-	Value     string    `thrift:"value,2" json:"value"`
-	Host      *Endpoint `thrift:"host,3" json:"host,omitempty"`
-}
-
-func NewAnnotation() *Annotation {
-	return &Annotation{}
-}
-
-func (p *Annotation) GetTimestamp() int64 {
-	return p.Timestamp
-}
-
-func (p *Annotation) GetValue() string {
-	return p.Value
-}
-
-var Annotation_Host_DEFAULT *Endpoint
-
-func (p *Annotation) GetHost() *Endpoint {
-	if !p.IsSetHost() {
-		return Annotation_Host_DEFAULT
-	}
-	return p.Host
-}
-func (p *Annotation) IsSetHost() bool {
-	return p.Host != nil
-}
-
-func (p *Annotation) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *Annotation) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Timestamp = v
-	}
-	return nil
-}
-
-func (p *Annotation) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Value = v
-	}
-	return nil
-}
-
-func (p *Annotation) readField3(iprot thrift.TProtocol) error {
-	p.Host = &Endpoint{}
-	if err := p.Host.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Host), err)
-	}
-	return nil
-}
-
-func (p *Annotation) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Annotation"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Annotation) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("timestamp", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:timestamp: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.Timestamp)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.timestamp (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:timestamp: ", p), err)
-	}
-	return err
-}
-
-func (p *Annotation) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("value", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:value: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Value)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.value (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:value: ", p), err)
-	}
-	return err
-}
-
-func (p *Annotation) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetHost() {
-		if err := oprot.WriteFieldBegin("host", thrift.STRUCT, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:host: ", p), err)
-		}
-		if err := p.Host.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Host), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:host: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Annotation) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Annotation(%+v)", *p)
-}
-
-// Binary annotations are tags applied to a Span to give it context. For
-// example, a binary annotation of "http.uri" could the path to a resource in a
-// RPC call.
-//
-// Binary annotations of type STRING are always queryable, though more a
-// historical implementation detail than a structural concern.
-//
-// Binary annotations can repeat, and vary on the host. Similar to Annotation,
-// the host indicates who logged the event. This allows you to tell the
-// difference between the client and server side of the same key. For example,
-// the key "http.uri" might be different on the client and server side due to
-// rewriting, like "/api/v1/myresource" vs "/myresource. Via the host field,
-// you can see the different points of view, which often help in debugging.
-//
-// Attributes:
-//  - Key
-//  - Value
-//  - AnnotationType
-//  - Host: The host that recorded tag, which allows you to differentiate between
-// multiple tags with the same key. There are two exceptions to this.
-//
-// When the key is CLIENT_ADDR or SERVER_ADDR, host indicates the source or
-// destination of an RPC. This exception allows zipkin to display network
-// context of uninstrumented services, or clients such as web browsers.
-type BinaryAnnotation struct {
-	Key            string         `thrift:"key,1" json:"key"`
-	Value          []byte         `thrift:"value,2" json:"value"`
-	AnnotationType AnnotationType `thrift:"annotation_type,3" json:"annotation_type"`
-	Host           *Endpoint      `thrift:"host,4" json:"host,omitempty"`
-}
-
-func NewBinaryAnnotation() *BinaryAnnotation {
-	return &BinaryAnnotation{}
-}
-
-func (p *BinaryAnnotation) GetKey() string {
-	return p.Key
-}
-
-func (p *BinaryAnnotation) GetValue() []byte {
-	return p.Value
-}
-
-func (p *BinaryAnnotation) GetAnnotationType() AnnotationType {
-	return p.AnnotationType
-}
-
-var BinaryAnnotation_Host_DEFAULT *Endpoint
-
-func (p *BinaryAnnotation) GetHost() *Endpoint {
-	if !p.IsSetHost() {
-		return BinaryAnnotation_Host_DEFAULT
-	}
-	return p.Host
-}
-func (p *BinaryAnnotation) IsSetHost() bool {
-	return p.Host != nil
-}
-
-func (p *BinaryAnnotation) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		case 2:
-			if err := p.readField2(iprot); err != nil {
-				return err
-			}
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Key = v
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) readField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBinary(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Value = v
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		temp := AnnotationType(v)
-		p.AnnotationType = temp
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) readField4(iprot thrift.TProtocol) error {
-	p.Host = &Endpoint{}
-	if err := p.Host.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Host), err)
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("BinaryAnnotation"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField2(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *BinaryAnnotation) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("key", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:key: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Key)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.key (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:key: ", p), err)
-	}
-	return err
-}
-
-func (p *BinaryAnnotation) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("value", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:value: ", p), err)
-	}
-	if err := oprot.WriteBinary(p.Value); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.value (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:value: ", p), err)
-	}
-	return err
-}
-
-func (p *BinaryAnnotation) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("annotation_type", thrift.I32, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:annotation_type: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.AnnotationType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.annotation_type (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:annotation_type: ", p), err)
-	}
-	return err
-}
-
-func (p *BinaryAnnotation) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetHost() {
-		if err := oprot.WriteFieldBegin("host", thrift.STRUCT, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:host: ", p), err)
-		}
-		if err := p.Host.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Host), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:host: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *BinaryAnnotation) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BinaryAnnotation(%+v)", *p)
-}
-
-// A trace is a series of spans (often RPC calls) which form a latency tree.
-//
-// The root span is where trace_id = id and parent_id = Nil. The root span is
-// usually the longest interval in the trace, starting with a SERVER_RECV
-// annotation and ending with a SERVER_SEND.
-//
-// Attributes:
-//  - TraceID
-//  - Name: Span name in lowercase, rpc method for example
-//
-// Conventionally, when the span name isn't known, name = "unknown".
-//  - ID
-//  - ParentID
-//  - Annotations
-//  - BinaryAnnotations
-//  - Debug
-//  - Timestamp: Microseconds from epoch of the creation of this span.
-//
-// This value should be set directly by instrumentation, using the most
-// precise value possible. For example, gettimeofday or syncing nanoTime
-// against a tick of currentTimeMillis.
-//
-// For compatibility with instrumentation that precede this field, collectors
-// or span stores can derive this via Annotation.timestamp.
-// For example, SERVER_RECV.timestamp or CLIENT_SEND.timestamp.
-//
-// This field is optional for compatibility with old data: first-party span
-// stores are expected to support this at time of introduction.
-//  - Duration: Measurement of duration in microseconds, used to support queries.
-//
-// This value should be set directly, where possible. Doing so encourages
-// precise measurement decoupled from problems of clocks, such as skew or NTP
-// updates causing time to move backwards.
-//
-// For compatibility with instrumentation that precede this field, collectors
-// or span stores can derive this by subtracting Annotation.timestamp.
-// For example, SERVER_SEND.timestamp - SERVER_RECV.timestamp.
-//
-// If this field is persisted as unset, zipkin will continue to work, except
-// duration query support will be implementation-specific. Similarly, setting
-// this field non-atomically is implementation-specific.
-//
-// This field is i64 vs i32 to support spans longer than 35 minutes.
-//  - TraceIDHigh: Optional unique 8-byte additional identifier for a trace. If non zero, this
-// means the trace uses 128 bit traceIds instead of 64 bit.
-type Span struct {
-	TraceID int64 `thrift:"trace_id,1" json:"trace_id"`
-	// unused field # 2
-	Name        string        `thrift:"name,3" json:"name"`
-	ID          int64         `thrift:"id,4" json:"id"`
-	ParentID    *int64        `thrift:"parent_id,5" json:"parent_id,omitempty"`
-	Annotations []*Annotation `thrift:"annotations,6" json:"annotations"`
-	// unused field # 7
-	BinaryAnnotations []*BinaryAnnotation `thrift:"binary_annotations,8" json:"binary_annotations"`
-	Debug             bool                `thrift:"debug,9" json:"debug,omitempty"`
-	Timestamp         *int64              `thrift:"timestamp,10" json:"timestamp,omitempty"`
-	Duration          *int64              `thrift:"duration,11" json:"duration,omitempty"`
-	TraceIDHigh       *int64              `thrift:"trace_id_high,12" json:"trace_id_high,omitempty"`
-}
-
-func NewSpan() *Span {
-	return &Span{}
-}
-
-func (p *Span) GetTraceID() int64 {
-	return p.TraceID
-}
-
-func (p *Span) GetName() string {
-	return p.Name
-}
-
-func (p *Span) GetID() int64 {
-	return p.ID
-}
-
-var Span_ParentID_DEFAULT int64
-
-func (p *Span) GetParentID() int64 {
-	if !p.IsSetParentID() {
-		return Span_ParentID_DEFAULT
-	}
-	return *p.ParentID
-}
-
-func (p *Span) GetAnnotations() []*Annotation {
-	return p.Annotations
-}
-
-func (p *Span) GetBinaryAnnotations() []*BinaryAnnotation {
-	return p.BinaryAnnotations
-}
-
-var Span_Debug_DEFAULT bool = false
-
-func (p *Span) GetDebug() bool {
-	return p.Debug
-}
-
-var Span_Timestamp_DEFAULT int64
-
-func (p *Span) GetTimestamp() int64 {
-	if !p.IsSetTimestamp() {
-		return Span_Timestamp_DEFAULT
-	}
-	return *p.Timestamp
-}
-
-var Span_Duration_DEFAULT int64
-
-func (p *Span) GetDuration() int64 {
-	if !p.IsSetDuration() {
-		return Span_Duration_DEFAULT
-	}
-	return *p.Duration
-}
-
-var Span_TraceIDHigh_DEFAULT int64
-
-func (p *Span) GetTraceIDHigh() int64 {
-	if !p.IsSetTraceIDHigh() {
-		return Span_TraceIDHigh_DEFAULT
-	}
-	return *p.TraceIDHigh
-}
-func (p *Span) IsSetParentID() bool {
-	return p.ParentID != nil
-}
-
-func (p *Span) IsSetDebug() bool {
-	return p.Debug != Span_Debug_DEFAULT
-}
-
-func (p *Span) IsSetTimestamp() bool {
-	return p.Timestamp != nil
-}
-
-func (p *Span) IsSetDuration() bool {
-	return p.Duration != nil
-}
-
-func (p *Span) IsSetTraceIDHigh() bool {
-	return p.TraceIDHigh != nil
-}
-
-func (p *Span) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		case 3:
-			if err := p.readField3(iprot); err != nil {
-				return err
-			}
-		case 4:
-			if err := p.readField4(iprot); err != nil {
-				return err
-			}
-		case 5:
-			if err := p.readField5(iprot); err != nil {
-				return err
-			}
-		case 6:
-			if err := p.readField6(iprot); err != nil {
-				return err
-			}
-		case 8:
-			if err := p.readField8(iprot); err != nil {
-				return err
-			}
-		case 9:
-			if err := p.readField9(iprot); err != nil {
-				return err
-			}
-		case 10:
-			if err := p.readField10(iprot); err != nil {
-				return err
-			}
-		case 11:
-			if err := p.readField11(iprot); err != nil {
-				return err
-			}
-		case 12:
-			if err := p.readField12(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *Span) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.TraceID = v
-	}
-	return nil
-}
-
-func (p *Span) readField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Name = v
-	}
-	return nil
-}
-
-func (p *Span) readField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.ID = v
-	}
-	return nil
-}
-
-func (p *Span) readField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.ParentID = &v
-	}
-	return nil
-}
-
-func (p *Span) readField6(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Annotation, 0, size)
-	p.Annotations = tSlice
-	for i := 0; i < size; i++ {
-		_elem0 := &Annotation{}
-		if err := _elem0.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
-		}
-		p.Annotations = append(p.Annotations, _elem0)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Span) readField8(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*BinaryAnnotation, 0, size)
-	p.BinaryAnnotations = tSlice
-	for i := 0; i < size; i++ {
-		_elem1 := &BinaryAnnotation{}
-		if err := _elem1.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem1), err)
-		}
-		p.BinaryAnnotations = append(p.BinaryAnnotations, _elem1)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *Span) readField9(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 9: ", err)
-	} else {
-		p.Debug = v
-	}
-	return nil
-}
-
-func (p *Span) readField10(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 10: ", err)
-	} else {
-		p.Timestamp = &v
-	}
-	return nil
-}
-
-func (p *Span) readField11(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 11: ", err)
-	} else {
-		p.Duration = &v
-	}
-	return nil
-}
-
-func (p *Span) readField12(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 12: ", err)
-	} else {
-		p.TraceIDHigh = &v
-	}
-	return nil
-}
-
-func (p *Span) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Span"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField3(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField4(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField5(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField6(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField8(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField9(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField10(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField11(oprot); err != nil {
-		return err
-	}
-	if err := p.writeField12(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Span) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("trace_id", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:trace_id: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.TraceID)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.trace_id (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:trace_id: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("name", thrift.STRING, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Name)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.name (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:name: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("id", thrift.I64, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:id: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.ID)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.id (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:id: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField5(oprot thrift.TProtocol) (err error) {
-	if p.IsSetParentID() {
-		if err := oprot.WriteFieldBegin("parent_id", thrift.I64, 5); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:parent_id: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.ParentID)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.parent_id (5) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 5:parent_id: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField6(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("annotations", thrift.LIST, 6); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:annotations: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Annotations)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Annotations {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 6:annotations: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField8(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("binary_annotations", thrift.LIST, 8); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:binary_annotations: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.BinaryAnnotations)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.BinaryAnnotations {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 8:binary_annotations: ", p), err)
-	}
-	return err
-}
-
-func (p *Span) writeField9(oprot thrift.TProtocol) (err error) {
-	if p.IsSetDebug() {
-		if err := oprot.WriteFieldBegin("debug", thrift.BOOL, 9); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 9:debug: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(p.Debug)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.debug (9) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 9:debug: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField10(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTimestamp() {
-		if err := oprot.WriteFieldBegin("timestamp", thrift.I64, 10); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 10:timestamp: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.Timestamp)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.timestamp (10) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 10:timestamp: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField11(oprot thrift.TProtocol) (err error) {
-	if p.IsSetDuration() {
-		if err := oprot.WriteFieldBegin("duration", thrift.I64, 11); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 11:duration: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.Duration)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.duration (11) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 11:duration: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) writeField12(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTraceIDHigh() {
-		if err := oprot.WriteFieldBegin("trace_id_high", thrift.I64, 12); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 12:trace_id_high: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.TraceIDHigh)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.trace_id_high (12) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 12:trace_id_high: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *Span) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Span(%+v)", *p)
-}
-
-// Attributes:
-//  - Ok
-type Response struct {
-	Ok bool `thrift:"ok,1,required" json:"ok"`
-}
-
-func NewResponse() *Response {
-	return &Response{}
-}
-
-func (p *Response) GetOk() bool {
-	return p.Ok
-}
-func (p *Response) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	var issetOk bool = false
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-			issetOk = true
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	if !issetOk {
-		return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set"))
-	}
-	return nil
-}
-
-func (p *Response) readField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Ok = v
-	}
-	return nil
-}
-
-func (p *Response) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("Response"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *Response) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.Ok)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err)
-	}
-	return err
-}
-
-func (p *Response) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("Response(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincollector.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincollector.go
deleted file mode 100644
index 417e883..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincollector.go
+++ /dev/null
@@ -1,446 +0,0 @@
-// Autogenerated by Thrift Compiler (0.9.3)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package zipkincore
-
-import (
-	"bytes"
-	"fmt"
-	"github.com/uber/jaeger-client-go/thrift"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = bytes.Equal
-
-type ZipkinCollector interface {
-	// Parameters:
-	//  - Spans
-	SubmitZipkinBatch(spans []*Span) (r []*Response, err error)
-}
-
-type ZipkinCollectorClient struct {
-	Transport       thrift.TTransport
-	ProtocolFactory thrift.TProtocolFactory
-	InputProtocol   thrift.TProtocol
-	OutputProtocol  thrift.TProtocol
-	SeqId           int32
-}
-
-func NewZipkinCollectorClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *ZipkinCollectorClient {
-	return &ZipkinCollectorClient{Transport: t,
-		ProtocolFactory: f,
-		InputProtocol:   f.GetProtocol(t),
-		OutputProtocol:  f.GetProtocol(t),
-		SeqId:           0,
-	}
-}
-
-func NewZipkinCollectorClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *ZipkinCollectorClient {
-	return &ZipkinCollectorClient{Transport: t,
-		ProtocolFactory: nil,
-		InputProtocol:   iprot,
-		OutputProtocol:  oprot,
-		SeqId:           0,
-	}
-}
-
-// Parameters:
-//  - Spans
-func (p *ZipkinCollectorClient) SubmitZipkinBatch(spans []*Span) (r []*Response, err error) {
-	if err = p.sendSubmitZipkinBatch(spans); err != nil {
-		return
-	}
-	return p.recvSubmitZipkinBatch()
-}
-
-func (p *ZipkinCollectorClient) sendSubmitZipkinBatch(spans []*Span) (err error) {
-	oprot := p.OutputProtocol
-	if oprot == nil {
-		oprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.OutputProtocol = oprot
-	}
-	p.SeqId++
-	if err = oprot.WriteMessageBegin("submitZipkinBatch", thrift.CALL, p.SeqId); err != nil {
-		return
-	}
-	args := ZipkinCollectorSubmitZipkinBatchArgs{
-		Spans: spans,
-	}
-	if err = args.Write(oprot); err != nil {
-		return
-	}
-	if err = oprot.WriteMessageEnd(); err != nil {
-		return
-	}
-	return oprot.Flush()
-}
-
-func (p *ZipkinCollectorClient) recvSubmitZipkinBatch() (value []*Response, err error) {
-	iprot := p.InputProtocol
-	if iprot == nil {
-		iprot = p.ProtocolFactory.GetProtocol(p.Transport)
-		p.InputProtocol = iprot
-	}
-	method, mTypeId, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return
-	}
-	if method != "submitZipkinBatch" {
-		err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "submitZipkinBatch failed: wrong method name")
-		return
-	}
-	if p.SeqId != seqId {
-		err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "submitZipkinBatch failed: out of sequence response")
-		return
-	}
-	if mTypeId == thrift.EXCEPTION {
-		error2 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception")
-		var error3 error
-		error3, err = error2.Read(iprot)
-		if err != nil {
-			return
-		}
-		if err = iprot.ReadMessageEnd(); err != nil {
-			return
-		}
-		err = error3
-		return
-	}
-	if mTypeId != thrift.REPLY {
-		err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "submitZipkinBatch failed: invalid message type")
-		return
-	}
-	result := ZipkinCollectorSubmitZipkinBatchResult{}
-	if err = result.Read(iprot); err != nil {
-		return
-	}
-	if err = iprot.ReadMessageEnd(); err != nil {
-		return
-	}
-	value = result.GetSuccess()
-	return
-}
-
-type ZipkinCollectorProcessor struct {
-	processorMap map[string]thrift.TProcessorFunction
-	handler      ZipkinCollector
-}
-
-func (p *ZipkinCollectorProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
-	p.processorMap[key] = processor
-}
-
-func (p *ZipkinCollectorProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
-	processor, ok = p.processorMap[key]
-	return processor, ok
-}
-
-func (p *ZipkinCollectorProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
-	return p.processorMap
-}
-
-func NewZipkinCollectorProcessor(handler ZipkinCollector) *ZipkinCollectorProcessor {
-
-	self4 := &ZipkinCollectorProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)}
-	self4.processorMap["submitZipkinBatch"] = &zipkinCollectorProcessorSubmitZipkinBatch{handler: handler}
-	return self4
-}
-
-func (p *ZipkinCollectorProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	name, _, seqId, err := iprot.ReadMessageBegin()
-	if err != nil {
-		return false, err
-	}
-	if processor, ok := p.GetProcessorFunction(name); ok {
-		return processor.Process(seqId, iprot, oprot)
-	}
-	iprot.Skip(thrift.STRUCT)
-	iprot.ReadMessageEnd()
-	x5 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name)
-	oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId)
-	x5.Write(oprot)
-	oprot.WriteMessageEnd()
-	oprot.Flush()
-	return false, x5
-
-}
-
-type zipkinCollectorProcessorSubmitZipkinBatch struct {
-	handler ZipkinCollector
-}
-
-func (p *zipkinCollectorProcessorSubmitZipkinBatch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
-	args := ZipkinCollectorSubmitZipkinBatchArgs{}
-	if err = args.Read(iprot); err != nil {
-		iprot.ReadMessageEnd()
-		x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error())
-		oprot.WriteMessageBegin("submitZipkinBatch", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return false, err
-	}
-
-	iprot.ReadMessageEnd()
-	result := ZipkinCollectorSubmitZipkinBatchResult{}
-	var retval []*Response
-	var err2 error
-	if retval, err2 = p.handler.SubmitZipkinBatch(args.Spans); err2 != nil {
-		x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing submitZipkinBatch: "+err2.Error())
-		oprot.WriteMessageBegin("submitZipkinBatch", thrift.EXCEPTION, seqId)
-		x.Write(oprot)
-		oprot.WriteMessageEnd()
-		oprot.Flush()
-		return true, err2
-	} else {
-		result.Success = retval
-	}
-	if err2 = oprot.WriteMessageBegin("submitZipkinBatch", thrift.REPLY, seqId); err2 != nil {
-		err = err2
-	}
-	if err2 = result.Write(oprot); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err2 = oprot.Flush(); err == nil && err2 != nil {
-		err = err2
-	}
-	if err != nil {
-		return
-	}
-	return true, err
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
-
-// Attributes:
-//  - Spans
-type ZipkinCollectorSubmitZipkinBatchArgs struct {
-	Spans []*Span `thrift:"spans,1" json:"spans"`
-}
-
-func NewZipkinCollectorSubmitZipkinBatchArgs() *ZipkinCollectorSubmitZipkinBatchArgs {
-	return &ZipkinCollectorSubmitZipkinBatchArgs{}
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) GetSpans() []*Span {
-	return p.Spans
-}
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 1:
-			if err := p.readField1(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) readField1(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Span, 0, size)
-	p.Spans = tSlice
-	for i := 0; i < size; i++ {
-		_elem6 := &Span{}
-		if err := _elem6.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem6), err)
-		}
-		p.Spans = append(p.Spans, _elem6)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("submitZipkinBatch_args"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField1(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("spans", thrift.LIST, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:spans: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Spans)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Spans {
-		if err := v.Write(oprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:spans: ", p), err)
-	}
-	return err
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchArgs) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ZipkinCollectorSubmitZipkinBatchArgs(%+v)", *p)
-}
-
-// Attributes:
-//  - Success
-type ZipkinCollectorSubmitZipkinBatchResult struct {
-	Success []*Response `thrift:"success,0" json:"success,omitempty"`
-}
-
-func NewZipkinCollectorSubmitZipkinBatchResult() *ZipkinCollectorSubmitZipkinBatchResult {
-	return &ZipkinCollectorSubmitZipkinBatchResult{}
-}
-
-var ZipkinCollectorSubmitZipkinBatchResult_Success_DEFAULT []*Response
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) GetSuccess() []*Response {
-	return p.Success
-}
-func (p *ZipkinCollectorSubmitZipkinBatchResult) IsSetSuccess() bool {
-	return p.Success != nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) Read(iprot thrift.TProtocol) error {
-	if _, err := iprot.ReadStructBegin(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
-	}
-
-	for {
-		_, fieldTypeId, fieldId, err := iprot.ReadFieldBegin()
-		if err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
-		}
-		if fieldTypeId == thrift.STOP {
-			break
-		}
-		switch fieldId {
-		case 0:
-			if err := p.readField0(iprot); err != nil {
-				return err
-			}
-		default:
-			if err := iprot.Skip(fieldTypeId); err != nil {
-				return err
-			}
-		}
-		if err := iprot.ReadFieldEnd(); err != nil {
-			return err
-		}
-	}
-	if err := iprot.ReadStructEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) readField0(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*Response, 0, size)
-	p.Success = tSlice
-	for i := 0; i < size; i++ {
-		_elem7 := &Response{}
-		if err := _elem7.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem7), err)
-		}
-		p.Success = append(p.Success, _elem7)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("submitZipkinBatch_result"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if err := p.writeField0(oprot); err != nil {
-		return err
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) writeField0(oprot thrift.TProtocol) (err error) {
-	if p.IsSetSuccess() {
-		if err := oprot.WriteFieldBegin("success", thrift.LIST, 0); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Success)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Success {
-			if err := v.Write(oprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *ZipkinCollectorSubmitZipkinBatchResult) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ZipkinCollectorSubmitZipkinBatchResult(%+v)", *p)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore-consts.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore-consts.go
new file mode 100644
index 0000000..7a924b9
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore-consts.go
@@ -0,0 +1,39 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package zipkincore
+
+import(
+	"bytes"
+	"context"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+const CLIENT_SEND = "cs"
+const CLIENT_RECV = "cr"
+const SERVER_SEND = "ss"
+const SERVER_RECV = "sr"
+const MESSAGE_SEND = "ms"
+const MESSAGE_RECV = "mr"
+const WIRE_SEND = "ws"
+const WIRE_RECV = "wr"
+const CLIENT_SEND_FRAGMENT = "csf"
+const CLIENT_RECV_FRAGMENT = "crf"
+const SERVER_SEND_FRAGMENT = "ssf"
+const SERVER_RECV_FRAGMENT = "srf"
+const LOCAL_COMPONENT = "lc"
+const CLIENT_ADDR = "ca"
+const SERVER_ADDR = "sa"
+const MESSAGE_ADDR = "ma"
+
+func init() {
+}
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore.go b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore.go
new file mode 100644
index 0000000..b00ecd2
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift-gen/zipkincore/zipkincore.go
@@ -0,0 +1,1853 @@
+// Code generated by Thrift Compiler (0.14.1). DO NOT EDIT.
+
+package zipkincore
+
+import(
+	"bytes"
+	"context"
+	"database/sql/driver"
+	"errors"
+	"fmt"
+	"time"
+	"github.com/uber/jaeger-client-go/thrift"
+)
+
+// (needed to ensure safety because of naive import list construction.)
+var _ = thrift.ZERO
+var _ = fmt.Printf
+var _ = context.Background
+var _ = time.Now
+var _ = bytes.Equal
+
+type AnnotationType int64
+const (
+  AnnotationType_BOOL AnnotationType = 0
+  AnnotationType_BYTES AnnotationType = 1
+  AnnotationType_I16 AnnotationType = 2
+  AnnotationType_I32 AnnotationType = 3
+  AnnotationType_I64 AnnotationType = 4
+  AnnotationType_DOUBLE AnnotationType = 5
+  AnnotationType_STRING AnnotationType = 6
+)
+
+func (p AnnotationType) String() string {
+  switch p {
+  case AnnotationType_BOOL: return "BOOL"
+  case AnnotationType_BYTES: return "BYTES"
+  case AnnotationType_I16: return "I16"
+  case AnnotationType_I32: return "I32"
+  case AnnotationType_I64: return "I64"
+  case AnnotationType_DOUBLE: return "DOUBLE"
+  case AnnotationType_STRING: return "STRING"
+  }
+  return "<UNSET>"
+}
+
+func AnnotationTypeFromString(s string) (AnnotationType, error) {
+  switch s {
+  case "BOOL": return AnnotationType_BOOL, nil 
+  case "BYTES": return AnnotationType_BYTES, nil 
+  case "I16": return AnnotationType_I16, nil 
+  case "I32": return AnnotationType_I32, nil 
+  case "I64": return AnnotationType_I64, nil 
+  case "DOUBLE": return AnnotationType_DOUBLE, nil 
+  case "STRING": return AnnotationType_STRING, nil 
+  }
+  return AnnotationType(0), fmt.Errorf("not a valid AnnotationType string")
+}
+
+
+func AnnotationTypePtr(v AnnotationType) *AnnotationType { return &v }
+
+func (p AnnotationType) MarshalText() ([]byte, error) {
+return []byte(p.String()), nil
+}
+
+func (p *AnnotationType) UnmarshalText(text []byte) error {
+q, err := AnnotationTypeFromString(string(text))
+if (err != nil) {
+return err
+}
+*p = q
+return nil
+}
+
+func (p *AnnotationType) Scan(value interface{}) error {
+v, ok := value.(int64)
+if !ok {
+return errors.New("Scan value is not int64")
+}
+*p = AnnotationType(v)
+return nil
+}
+
+func (p * AnnotationType) Value() (driver.Value, error) {
+  if p == nil {
+    return nil, nil
+  }
+return int64(*p), nil
+}
+// Indicates the network context of a service recording an annotation with two
+// exceptions.
+// 
+// When a BinaryAnnotation, and key is CLIENT_ADDR or SERVER_ADDR,
+// the endpoint indicates the source or destination of an RPC. This exception
+// allows zipkin to display network context of uninstrumented services, or
+// clients such as web browsers.
+// 
+// Attributes:
+//  - Ipv4: IPv4 host address packed into 4 bytes.
+// 
+// Ex for the ip 1.2.3.4, it would be (1 << 24) | (2 << 16) | (3 << 8) | 4
+//  - Port: IPv4 port
+// 
+// Note: this is to be treated as an unsigned integer, so watch for negatives.
+// 
+// Conventionally, when the port isn't known, port = 0.
+//  - ServiceName: Service name in lowercase, such as "memcache" or "zipkin-web"
+// 
+// Conventionally, when the service name isn't known, service_name = "unknown".
+//  - Ipv6: IPv6 host address packed into 16 bytes. Ex Inet6Address.getBytes()
+type Endpoint struct {
+  Ipv4 int32 `thrift:"ipv4,1" db:"ipv4" json:"ipv4"`
+  Port int16 `thrift:"port,2" db:"port" json:"port"`
+  ServiceName string `thrift:"service_name,3" db:"service_name" json:"service_name"`
+  Ipv6 []byte `thrift:"ipv6,4" db:"ipv6" json:"ipv6,omitempty"`
+}
+
+func NewEndpoint() *Endpoint {
+  return &Endpoint{}
+}
+
+
+func (p *Endpoint) GetIpv4() int32 {
+  return p.Ipv4
+}
+
+func (p *Endpoint) GetPort() int16 {
+  return p.Port
+}
+
+func (p *Endpoint) GetServiceName() string {
+  return p.ServiceName
+}
+var Endpoint_Ipv6_DEFAULT []byte
+
+func (p *Endpoint) GetIpv6() []byte {
+  return p.Ipv6
+}
+func (p *Endpoint) IsSetIpv6() bool {
+  return p.Ipv6 != nil
+}
+
+func (p *Endpoint) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.I16 {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *Endpoint)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Ipv4 = v
+}
+  return nil
+}
+
+func (p *Endpoint)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI16(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.Port = v
+}
+  return nil
+}
+
+func (p *Endpoint)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.ServiceName = v
+}
+  return nil
+}
+
+func (p *Endpoint)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBinary(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.Ipv6 = v
+}
+  return nil
+}
+
+func (p *Endpoint) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Endpoint"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Endpoint) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "ipv4", thrift.I32, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ipv4: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.Ipv4)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.ipv4 (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ipv4: ", p), err) }
+  return err
+}
+
+func (p *Endpoint) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "port", thrift.I16, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:port: ", p), err) }
+  if err := oprot.WriteI16(ctx, int16(p.Port)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.port (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:port: ", p), err) }
+  return err
+}
+
+func (p *Endpoint) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "service_name", thrift.STRING, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:service_name: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.ServiceName)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.service_name (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:service_name: ", p), err) }
+  return err
+}
+
+func (p *Endpoint) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetIpv6() {
+    if err := oprot.WriteFieldBegin(ctx, "ipv6", thrift.STRING, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:ipv6: ", p), err) }
+    if err := oprot.WriteBinary(ctx, p.Ipv6); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.ipv6 (4) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:ipv6: ", p), err) }
+  }
+  return err
+}
+
+func (p *Endpoint) Equals(other *Endpoint) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Ipv4 != other.Ipv4 { return false }
+  if p.Port != other.Port { return false }
+  if p.ServiceName != other.ServiceName { return false }
+  if bytes.Compare(p.Ipv6, other.Ipv6) != 0 { return false }
+  return true
+}
+
+func (p *Endpoint) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Endpoint(%+v)", *p)
+}
+
+// An annotation is similar to a log statement. It includes a host field which
+// allows these events to be attributed properly, and also aggregatable.
+// 
+// Attributes:
+//  - Timestamp: Microseconds from epoch.
+// 
+// This value should use the most precise value possible. For example,
+// gettimeofday or syncing nanoTime against a tick of currentTimeMillis.
+//  - Value
+//  - Host: Always the host that recorded the event. By specifying the host you allow
+// rollup of all events (such as client requests to a service) by IP address.
+type Annotation struct {
+  Timestamp int64 `thrift:"timestamp,1" db:"timestamp" json:"timestamp"`
+  Value string `thrift:"value,2" db:"value" json:"value"`
+  Host *Endpoint `thrift:"host,3" db:"host" json:"host,omitempty"`
+}
+
+func NewAnnotation() *Annotation {
+  return &Annotation{}
+}
+
+
+func (p *Annotation) GetTimestamp() int64 {
+  return p.Timestamp
+}
+
+func (p *Annotation) GetValue() string {
+  return p.Value
+}
+var Annotation_Host_DEFAULT *Endpoint
+func (p *Annotation) GetHost() *Endpoint {
+  if !p.IsSetHost() {
+    return Annotation_Host_DEFAULT
+  }
+return p.Host
+}
+func (p *Annotation) IsSetHost() bool {
+  return p.Host != nil
+}
+
+func (p *Annotation) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *Annotation)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Timestamp = v
+}
+  return nil
+}
+
+func (p *Annotation)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.Value = v
+}
+  return nil
+}
+
+func (p *Annotation)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Host = &Endpoint{}
+  if err := p.Host.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Host), err)
+  }
+  return nil
+}
+
+func (p *Annotation) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Annotation"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Annotation) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "timestamp", thrift.I64, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:timestamp: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.Timestamp)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.timestamp (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:timestamp: ", p), err) }
+  return err
+}
+
+func (p *Annotation) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "value", thrift.STRING, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:value: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.Value)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.value (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:value: ", p), err) }
+  return err
+}
+
+func (p *Annotation) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetHost() {
+    if err := oprot.WriteFieldBegin(ctx, "host", thrift.STRUCT, 3); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:host: ", p), err) }
+    if err := p.Host.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Host), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 3:host: ", p), err) }
+  }
+  return err
+}
+
+func (p *Annotation) Equals(other *Annotation) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Timestamp != other.Timestamp { return false }
+  if p.Value != other.Value { return false }
+  if !p.Host.Equals(other.Host) { return false }
+  return true
+}
+
+func (p *Annotation) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Annotation(%+v)", *p)
+}
+
+// Binary annotations are tags applied to a Span to give it context. For
+// example, a binary annotation of "http.uri" could the path to a resource in a
+// RPC call.
+// 
+// Binary annotations of type STRING are always queryable, though more a
+// historical implementation detail than a structural concern.
+// 
+// Binary annotations can repeat, and vary on the host. Similar to Annotation,
+// the host indicates who logged the event. This allows you to tell the
+// difference between the client and server side of the same key. For example,
+// the key "http.uri" might be different on the client and server side due to
+// rewriting, like "/api/v1/myresource" vs "/myresource. Via the host field,
+// you can see the different points of view, which often help in debugging.
+// 
+// Attributes:
+//  - Key
+//  - Value
+//  - AnnotationType
+//  - Host: The host that recorded tag, which allows you to differentiate between
+// multiple tags with the same key. There are two exceptions to this.
+// 
+// When the key is CLIENT_ADDR or SERVER_ADDR, host indicates the source or
+// destination of an RPC. This exception allows zipkin to display network
+// context of uninstrumented services, or clients such as web browsers.
+type BinaryAnnotation struct {
+  Key string `thrift:"key,1" db:"key" json:"key"`
+  Value []byte `thrift:"value,2" db:"value" json:"value"`
+  AnnotationType AnnotationType `thrift:"annotation_type,3" db:"annotation_type" json:"annotation_type"`
+  Host *Endpoint `thrift:"host,4" db:"host" json:"host,omitempty"`
+}
+
+func NewBinaryAnnotation() *BinaryAnnotation {
+  return &BinaryAnnotation{}
+}
+
+
+func (p *BinaryAnnotation) GetKey() string {
+  return p.Key
+}
+
+func (p *BinaryAnnotation) GetValue() []byte {
+  return p.Value
+}
+
+func (p *BinaryAnnotation) GetAnnotationType() AnnotationType {
+  return p.AnnotationType
+}
+var BinaryAnnotation_Host_DEFAULT *Endpoint
+func (p *BinaryAnnotation) GetHost() *Endpoint {
+  if !p.IsSetHost() {
+    return BinaryAnnotation_Host_DEFAULT
+  }
+return p.Host
+}
+func (p *BinaryAnnotation) IsSetHost() bool {
+  return p.Host != nil
+}
+
+func (p *BinaryAnnotation) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 2:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField2(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.I32 {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.STRUCT {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *BinaryAnnotation)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Key = v
+}
+  return nil
+}
+
+func (p *BinaryAnnotation)  ReadField2(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBinary(ctx); err != nil {
+  return thrift.PrependError("error reading field 2: ", err)
+} else {
+  p.Value = v
+}
+  return nil
+}
+
+func (p *BinaryAnnotation)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI32(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  temp := AnnotationType(v)
+  p.AnnotationType = temp
+}
+  return nil
+}
+
+func (p *BinaryAnnotation)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  p.Host = &Endpoint{}
+  if err := p.Host.Read(ctx, iprot); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Host), err)
+  }
+  return nil
+}
+
+func (p *BinaryAnnotation) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "BinaryAnnotation"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField2(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *BinaryAnnotation) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "key", thrift.STRING, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:key: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.Key)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.key (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:key: ", p), err) }
+  return err
+}
+
+func (p *BinaryAnnotation) writeField2(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "value", thrift.STRING, 2); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:value: ", p), err) }
+  if err := oprot.WriteBinary(ctx, p.Value); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.value (2) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 2:value: ", p), err) }
+  return err
+}
+
+func (p *BinaryAnnotation) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "annotation_type", thrift.I32, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:annotation_type: ", p), err) }
+  if err := oprot.WriteI32(ctx, int32(p.AnnotationType)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.annotation_type (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:annotation_type: ", p), err) }
+  return err
+}
+
+func (p *BinaryAnnotation) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetHost() {
+    if err := oprot.WriteFieldBegin(ctx, "host", thrift.STRUCT, 4); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:host: ", p), err) }
+    if err := p.Host.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Host), err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 4:host: ", p), err) }
+  }
+  return err
+}
+
+func (p *BinaryAnnotation) Equals(other *BinaryAnnotation) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Key != other.Key { return false }
+  if bytes.Compare(p.Value, other.Value) != 0 { return false }
+  if p.AnnotationType != other.AnnotationType { return false }
+  if !p.Host.Equals(other.Host) { return false }
+  return true
+}
+
+func (p *BinaryAnnotation) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("BinaryAnnotation(%+v)", *p)
+}
+
+// A trace is a series of spans (often RPC calls) which form a latency tree.
+// 
+// The root span is where trace_id = id and parent_id = Nil. The root span is
+// usually the longest interval in the trace, starting with a SERVER_RECV
+// annotation and ending with a SERVER_SEND.
+// 
+// Attributes:
+//  - TraceID
+//  - Name: Span name in lowercase, rpc method for example
+// 
+// Conventionally, when the span name isn't known, name = "unknown".
+//  - ID
+//  - ParentID
+//  - Annotations
+//  - BinaryAnnotations
+//  - Debug
+//  - Timestamp: Microseconds from epoch of the creation of this span.
+// 
+// This value should be set directly by instrumentation, using the most
+// precise value possible. For example, gettimeofday or syncing nanoTime
+// against a tick of currentTimeMillis.
+// 
+// For compatibility with instrumentation that precede this field, collectors
+// or span stores can derive this via Annotation.timestamp.
+// For example, SERVER_RECV.timestamp or CLIENT_SEND.timestamp.
+// 
+// This field is optional for compatibility with old data: first-party span
+// stores are expected to support this at time of introduction.
+//  - Duration: Measurement of duration in microseconds, used to support queries.
+// 
+// This value should be set directly, where possible. Doing so encourages
+// precise measurement decoupled from problems of clocks, such as skew or NTP
+// updates causing time to move backwards.
+// 
+// For compatibility with instrumentation that precede this field, collectors
+// or span stores can derive this by subtracting Annotation.timestamp.
+// For example, SERVER_SEND.timestamp - SERVER_RECV.timestamp.
+// 
+// If this field is persisted as unset, zipkin will continue to work, except
+// duration query support will be implementation-specific. Similarly, setting
+// this field non-atomically is implementation-specific.
+// 
+// This field is i64 vs i32 to support spans longer than 35 minutes.
+//  - TraceIDHigh: Optional unique 8-byte additional identifier for a trace. If non zero, this
+// means the trace uses 128 bit traceIds instead of 64 bit.
+type Span struct {
+  TraceID int64 `thrift:"trace_id,1" db:"trace_id" json:"trace_id"`
+  // unused field # 2
+  Name string `thrift:"name,3" db:"name" json:"name"`
+  ID int64 `thrift:"id,4" db:"id" json:"id"`
+  ParentID *int64 `thrift:"parent_id,5" db:"parent_id" json:"parent_id,omitempty"`
+  Annotations []*Annotation `thrift:"annotations,6" db:"annotations" json:"annotations"`
+  // unused field # 7
+  BinaryAnnotations []*BinaryAnnotation `thrift:"binary_annotations,8" db:"binary_annotations" json:"binary_annotations"`
+  Debug bool `thrift:"debug,9" db:"debug" json:"debug"`
+  Timestamp *int64 `thrift:"timestamp,10" db:"timestamp" json:"timestamp,omitempty"`
+  Duration *int64 `thrift:"duration,11" db:"duration" json:"duration,omitempty"`
+  TraceIDHigh *int64 `thrift:"trace_id_high,12" db:"trace_id_high" json:"trace_id_high,omitempty"`
+}
+
+func NewSpan() *Span {
+  return &Span{}
+}
+
+
+func (p *Span) GetTraceID() int64 {
+  return p.TraceID
+}
+
+func (p *Span) GetName() string {
+  return p.Name
+}
+
+func (p *Span) GetID() int64 {
+  return p.ID
+}
+var Span_ParentID_DEFAULT int64
+func (p *Span) GetParentID() int64 {
+  if !p.IsSetParentID() {
+    return Span_ParentID_DEFAULT
+  }
+return *p.ParentID
+}
+
+func (p *Span) GetAnnotations() []*Annotation {
+  return p.Annotations
+}
+
+func (p *Span) GetBinaryAnnotations() []*BinaryAnnotation {
+  return p.BinaryAnnotations
+}
+var Span_Debug_DEFAULT bool = false
+
+func (p *Span) GetDebug() bool {
+  return p.Debug
+}
+var Span_Timestamp_DEFAULT int64
+func (p *Span) GetTimestamp() int64 {
+  if !p.IsSetTimestamp() {
+    return Span_Timestamp_DEFAULT
+  }
+return *p.Timestamp
+}
+var Span_Duration_DEFAULT int64
+func (p *Span) GetDuration() int64 {
+  if !p.IsSetDuration() {
+    return Span_Duration_DEFAULT
+  }
+return *p.Duration
+}
+var Span_TraceIDHigh_DEFAULT int64
+func (p *Span) GetTraceIDHigh() int64 {
+  if !p.IsSetTraceIDHigh() {
+    return Span_TraceIDHigh_DEFAULT
+  }
+return *p.TraceIDHigh
+}
+func (p *Span) IsSetParentID() bool {
+  return p.ParentID != nil
+}
+
+func (p *Span) IsSetDebug() bool {
+  return p.Debug != Span_Debug_DEFAULT
+}
+
+func (p *Span) IsSetTimestamp() bool {
+  return p.Timestamp != nil
+}
+
+func (p *Span) IsSetDuration() bool {
+  return p.Duration != nil
+}
+
+func (p *Span) IsSetTraceIDHigh() bool {
+  return p.TraceIDHigh != nil
+}
+
+func (p *Span) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 3:
+      if fieldTypeId == thrift.STRING {
+        if err := p.ReadField3(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 4:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField4(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 5:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField5(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 6:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField6(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 8:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField8(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 9:
+      if fieldTypeId == thrift.BOOL {
+        if err := p.ReadField9(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 10:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField10(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 11:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField11(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    case 12:
+      if fieldTypeId == thrift.I64 {
+        if err := p.ReadField12(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *Span)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.TraceID = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField3(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadString(ctx); err != nil {
+  return thrift.PrependError("error reading field 3: ", err)
+} else {
+  p.Name = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField4(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 4: ", err)
+} else {
+  p.ID = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField5(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 5: ", err)
+} else {
+  p.ParentID = &v
+}
+  return nil
+}
+
+func (p *Span)  ReadField6(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Annotation, 0, size)
+  p.Annotations =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem0 := &Annotation{}
+    if err := _elem0.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem0), err)
+    }
+    p.Annotations = append(p.Annotations, _elem0)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Span)  ReadField8(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*BinaryAnnotation, 0, size)
+  p.BinaryAnnotations =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem1 := &BinaryAnnotation{}
+    if err := _elem1.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem1), err)
+    }
+    p.BinaryAnnotations = append(p.BinaryAnnotations, _elem1)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *Span)  ReadField9(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBool(ctx); err != nil {
+  return thrift.PrependError("error reading field 9: ", err)
+} else {
+  p.Debug = v
+}
+  return nil
+}
+
+func (p *Span)  ReadField10(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 10: ", err)
+} else {
+  p.Timestamp = &v
+}
+  return nil
+}
+
+func (p *Span)  ReadField11(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 11: ", err)
+} else {
+  p.Duration = &v
+}
+  return nil
+}
+
+func (p *Span)  ReadField12(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadI64(ctx); err != nil {
+  return thrift.PrependError("error reading field 12: ", err)
+} else {
+  p.TraceIDHigh = &v
+}
+  return nil
+}
+
+func (p *Span) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Span"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+    if err := p.writeField3(ctx, oprot); err != nil { return err }
+    if err := p.writeField4(ctx, oprot); err != nil { return err }
+    if err := p.writeField5(ctx, oprot); err != nil { return err }
+    if err := p.writeField6(ctx, oprot); err != nil { return err }
+    if err := p.writeField8(ctx, oprot); err != nil { return err }
+    if err := p.writeField9(ctx, oprot); err != nil { return err }
+    if err := p.writeField10(ctx, oprot); err != nil { return err }
+    if err := p.writeField11(ctx, oprot); err != nil { return err }
+    if err := p.writeField12(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Span) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "trace_id", thrift.I64, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:trace_id: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.TraceID)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.trace_id (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:trace_id: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField3(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "name", thrift.STRING, 3); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:name: ", p), err) }
+  if err := oprot.WriteString(ctx, string(p.Name)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.name (3) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 3:name: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField4(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "id", thrift.I64, 4); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:id: ", p), err) }
+  if err := oprot.WriteI64(ctx, int64(p.ID)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.id (4) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 4:id: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField5(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetParentID() {
+    if err := oprot.WriteFieldBegin(ctx, "parent_id", thrift.I64, 5); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:parent_id: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.ParentID)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.parent_id (5) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 5:parent_id: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField6(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "annotations", thrift.LIST, 6); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:annotations: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Annotations)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Annotations {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 6:annotations: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField8(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "binary_annotations", thrift.LIST, 8); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:binary_annotations: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.BinaryAnnotations)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.BinaryAnnotations {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 8:binary_annotations: ", p), err) }
+  return err
+}
+
+func (p *Span) writeField9(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetDebug() {
+    if err := oprot.WriteFieldBegin(ctx, "debug", thrift.BOOL, 9); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 9:debug: ", p), err) }
+    if err := oprot.WriteBool(ctx, bool(p.Debug)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.debug (9) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 9:debug: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField10(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetTimestamp() {
+    if err := oprot.WriteFieldBegin(ctx, "timestamp", thrift.I64, 10); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 10:timestamp: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.Timestamp)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.timestamp (10) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 10:timestamp: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField11(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetDuration() {
+    if err := oprot.WriteFieldBegin(ctx, "duration", thrift.I64, 11); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 11:duration: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.Duration)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.duration (11) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 11:duration: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) writeField12(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetTraceIDHigh() {
+    if err := oprot.WriteFieldBegin(ctx, "trace_id_high", thrift.I64, 12); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 12:trace_id_high: ", p), err) }
+    if err := oprot.WriteI64(ctx, int64(*p.TraceIDHigh)); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T.trace_id_high (12) field write error: ", p), err) }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 12:trace_id_high: ", p), err) }
+  }
+  return err
+}
+
+func (p *Span) Equals(other *Span) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.TraceID != other.TraceID { return false }
+  if p.Name != other.Name { return false }
+  if p.ID != other.ID { return false }
+  if p.ParentID != other.ParentID {
+    if p.ParentID == nil || other.ParentID == nil {
+      return false
+    }
+    if (*p.ParentID) != (*other.ParentID) { return false }
+  }
+  if len(p.Annotations) != len(other.Annotations) { return false }
+  for i, _tgt := range p.Annotations {
+    _src2 := other.Annotations[i]
+    if !_tgt.Equals(_src2) { return false }
+  }
+  if len(p.BinaryAnnotations) != len(other.BinaryAnnotations) { return false }
+  for i, _tgt := range p.BinaryAnnotations {
+    _src3 := other.BinaryAnnotations[i]
+    if !_tgt.Equals(_src3) { return false }
+  }
+  if p.Debug != other.Debug { return false }
+  if p.Timestamp != other.Timestamp {
+    if p.Timestamp == nil || other.Timestamp == nil {
+      return false
+    }
+    if (*p.Timestamp) != (*other.Timestamp) { return false }
+  }
+  if p.Duration != other.Duration {
+    if p.Duration == nil || other.Duration == nil {
+      return false
+    }
+    if (*p.Duration) != (*other.Duration) { return false }
+  }
+  if p.TraceIDHigh != other.TraceIDHigh {
+    if p.TraceIDHigh == nil || other.TraceIDHigh == nil {
+      return false
+    }
+    if (*p.TraceIDHigh) != (*other.TraceIDHigh) { return false }
+  }
+  return true
+}
+
+func (p *Span) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Span(%+v)", *p)
+}
+
+// Attributes:
+//  - Ok
+type Response struct {
+  Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"`
+}
+
+func NewResponse() *Response {
+  return &Response{}
+}
+
+
+func (p *Response) GetOk() bool {
+  return p.Ok
+}
+func (p *Response) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+  var issetOk bool = false;
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.BOOL {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+        issetOk = true
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  if !issetOk{
+    return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set"));
+  }
+  return nil
+}
+
+func (p *Response)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  if v, err := iprot.ReadBool(ctx); err != nil {
+  return thrift.PrependError("error reading field 1: ", err)
+} else {
+  p.Ok = v
+}
+  return nil
+}
+
+func (p *Response) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "Response"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *Response) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "ok", thrift.BOOL, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) }
+  if err := oprot.WriteBool(ctx, bool(p.Ok)); err != nil {
+  return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) }
+  return err
+}
+
+func (p *Response) Equals(other *Response) bool {
+  if p == other {
+    return true
+  } else if p == nil || other == nil {
+    return false
+  }
+  if p.Ok != other.Ok { return false }
+  return true
+}
+
+func (p *Response) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("Response(%+v)", *p)
+}
+
+type ZipkinCollector interface {
+  // Parameters:
+  //  - Spans
+  SubmitZipkinBatch(ctx context.Context, spans []*Span) (_r []*Response, _err error)
+}
+
+type ZipkinCollectorClient struct {
+  c thrift.TClient
+  meta thrift.ResponseMeta
+}
+
+func NewZipkinCollectorClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *ZipkinCollectorClient {
+  return &ZipkinCollectorClient{
+    c: thrift.NewTStandardClient(f.GetProtocol(t), f.GetProtocol(t)),
+  }
+}
+
+func NewZipkinCollectorClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *ZipkinCollectorClient {
+  return &ZipkinCollectorClient{
+    c: thrift.NewTStandardClient(iprot, oprot),
+  }
+}
+
+func NewZipkinCollectorClient(c thrift.TClient) *ZipkinCollectorClient {
+  return &ZipkinCollectorClient{
+    c: c,
+  }
+}
+
+func (p *ZipkinCollectorClient) Client_() thrift.TClient {
+  return p.c
+}
+
+func (p *ZipkinCollectorClient) LastResponseMeta_() thrift.ResponseMeta {
+  return p.meta
+}
+
+func (p *ZipkinCollectorClient) SetLastResponseMeta_(meta thrift.ResponseMeta) {
+  p.meta = meta
+}
+
+// Parameters:
+//  - Spans
+func (p *ZipkinCollectorClient) SubmitZipkinBatch(ctx context.Context, spans []*Span) (_r []*Response, _err error) {
+  var _args4 ZipkinCollectorSubmitZipkinBatchArgs
+  _args4.Spans = spans
+  var _result6 ZipkinCollectorSubmitZipkinBatchResult
+  var _meta5 thrift.ResponseMeta
+  _meta5, _err = p.Client_().Call(ctx, "submitZipkinBatch", &_args4, &_result6)
+  p.SetLastResponseMeta_(_meta5)
+  if _err != nil {
+    return
+  }
+  return _result6.GetSuccess(), nil
+}
+
+type ZipkinCollectorProcessor struct {
+  processorMap map[string]thrift.TProcessorFunction
+  handler ZipkinCollector
+}
+
+func (p *ZipkinCollectorProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) {
+  p.processorMap[key] = processor
+}
+
+func (p *ZipkinCollectorProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) {
+  processor, ok = p.processorMap[key]
+  return processor, ok
+}
+
+func (p *ZipkinCollectorProcessor) ProcessorMap() map[string]thrift.TProcessorFunction {
+  return p.processorMap
+}
+
+func NewZipkinCollectorProcessor(handler ZipkinCollector) *ZipkinCollectorProcessor {
+
+  self7 := &ZipkinCollectorProcessor{handler:handler, processorMap:make(map[string]thrift.TProcessorFunction)}
+  self7.processorMap["submitZipkinBatch"] = &zipkinCollectorProcessorSubmitZipkinBatch{handler:handler}
+return self7
+}
+
+func (p *ZipkinCollectorProcessor) Process(ctx context.Context, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  name, _, seqId, err2 := iprot.ReadMessageBegin(ctx)
+  if err2 != nil { return false, thrift.WrapTException(err2) }
+  if processor, ok := p.GetProcessorFunction(name); ok {
+    return processor.Process(ctx, seqId, iprot, oprot)
+  }
+  iprot.Skip(ctx, thrift.STRUCT)
+  iprot.ReadMessageEnd(ctx)
+  x8 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function " + name)
+  oprot.WriteMessageBegin(ctx, name, thrift.EXCEPTION, seqId)
+  x8.Write(ctx, oprot)
+  oprot.WriteMessageEnd(ctx)
+  oprot.Flush(ctx)
+  return false, x8
+
+}
+
+type zipkinCollectorProcessorSubmitZipkinBatch struct {
+  handler ZipkinCollector
+}
+
+func (p *zipkinCollectorProcessorSubmitZipkinBatch) Process(ctx context.Context, seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) {
+  args := ZipkinCollectorSubmitZipkinBatchArgs{}
+  var err2 error
+  if err2 = args.Read(ctx, iprot); err2 != nil {
+    iprot.ReadMessageEnd(ctx)
+    x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err2.Error())
+    oprot.WriteMessageBegin(ctx, "submitZipkinBatch", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return false, thrift.WrapTException(err2)
+  }
+  iprot.ReadMessageEnd(ctx)
+
+  tickerCancel := func() {}
+  // Start a goroutine to do server side connectivity check.
+  if thrift.ServerConnectivityCheckInterval > 0 {
+    var cancel context.CancelFunc
+    ctx, cancel = context.WithCancel(ctx)
+    defer cancel()
+    var tickerCtx context.Context
+    tickerCtx, tickerCancel = context.WithCancel(context.Background())
+    defer tickerCancel()
+    go func(ctx context.Context, cancel context.CancelFunc) {
+      ticker := time.NewTicker(thrift.ServerConnectivityCheckInterval)
+      defer ticker.Stop()
+      for {
+        select {
+        case <-ctx.Done():
+          return
+        case <-ticker.C:
+          if !iprot.Transport().IsOpen() {
+            cancel()
+            return
+          }
+        }
+      }
+    }(tickerCtx, cancel)
+  }
+
+  result := ZipkinCollectorSubmitZipkinBatchResult{}
+  var retval []*Response
+  if retval, err2 = p.handler.SubmitZipkinBatch(ctx, args.Spans); err2 != nil {
+    tickerCancel()
+    if err2 == thrift.ErrAbandonRequest {
+      return false, thrift.WrapTException(err2)
+    }
+    x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing submitZipkinBatch: " + err2.Error())
+    oprot.WriteMessageBegin(ctx, "submitZipkinBatch", thrift.EXCEPTION, seqId)
+    x.Write(ctx, oprot)
+    oprot.WriteMessageEnd(ctx)
+    oprot.Flush(ctx)
+    return true, thrift.WrapTException(err2)
+  } else {
+    result.Success = retval
+  }
+  tickerCancel()
+  if err2 = oprot.WriteMessageBegin(ctx, "submitZipkinBatch", thrift.REPLY, seqId); err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = result.Write(ctx, oprot); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.WriteMessageEnd(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err2 = oprot.Flush(ctx); err == nil && err2 != nil {
+    err = thrift.WrapTException(err2)
+  }
+  if err != nil {
+    return
+  }
+  return true, err
+}
+
+
+// HELPER FUNCTIONS AND STRUCTURES
+
+// Attributes:
+//  - Spans
+type ZipkinCollectorSubmitZipkinBatchArgs struct {
+  Spans []*Span `thrift:"spans,1" db:"spans" json:"spans"`
+}
+
+func NewZipkinCollectorSubmitZipkinBatchArgs() *ZipkinCollectorSubmitZipkinBatchArgs {
+  return &ZipkinCollectorSubmitZipkinBatchArgs{}
+}
+
+
+func (p *ZipkinCollectorSubmitZipkinBatchArgs) GetSpans() []*Span {
+  return p.Spans
+}
+func (p *ZipkinCollectorSubmitZipkinBatchArgs) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 1:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField1(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchArgs)  ReadField1(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Span, 0, size)
+  p.Spans =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem9 := &Span{}
+    if err := _elem9.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem9), err)
+    }
+    p.Spans = append(p.Spans, _elem9)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchArgs) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "submitZipkinBatch_args"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField1(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchArgs) writeField1(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if err := oprot.WriteFieldBegin(ctx, "spans", thrift.LIST, 1); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:spans: ", p), err) }
+  if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Spans)); err != nil {
+    return thrift.PrependError("error writing list begin: ", err)
+  }
+  for _, v := range p.Spans {
+    if err := v.Write(ctx, oprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+    }
+  }
+  if err := oprot.WriteListEnd(ctx); err != nil {
+    return thrift.PrependError("error writing list end: ", err)
+  }
+  if err := oprot.WriteFieldEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write field end error 1:spans: ", p), err) }
+  return err
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchArgs) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("ZipkinCollectorSubmitZipkinBatchArgs(%+v)", *p)
+}
+
+// Attributes:
+//  - Success
+type ZipkinCollectorSubmitZipkinBatchResult struct {
+  Success []*Response `thrift:"success,0" db:"success" json:"success,omitempty"`
+}
+
+func NewZipkinCollectorSubmitZipkinBatchResult() *ZipkinCollectorSubmitZipkinBatchResult {
+  return &ZipkinCollectorSubmitZipkinBatchResult{}
+}
+
+var ZipkinCollectorSubmitZipkinBatchResult_Success_DEFAULT []*Response
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult) GetSuccess() []*Response {
+  return p.Success
+}
+func (p *ZipkinCollectorSubmitZipkinBatchResult) IsSetSuccess() bool {
+  return p.Success != nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult) Read(ctx context.Context, iprot thrift.TProtocol) error {
+  if _, err := iprot.ReadStructBegin(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err)
+  }
+
+
+  for {
+    _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin(ctx)
+    if err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err)
+    }
+    if fieldTypeId == thrift.STOP { break; }
+    switch fieldId {
+    case 0:
+      if fieldTypeId == thrift.LIST {
+        if err := p.ReadField0(ctx, iprot); err != nil {
+          return err
+        }
+      } else {
+        if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+          return err
+        }
+      }
+    default:
+      if err := iprot.Skip(ctx, fieldTypeId); err != nil {
+        return err
+      }
+    }
+    if err := iprot.ReadFieldEnd(ctx); err != nil {
+      return err
+    }
+  }
+  if err := iprot.ReadStructEnd(ctx); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err)
+  }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult)  ReadField0(ctx context.Context, iprot thrift.TProtocol) error {
+  _, size, err := iprot.ReadListBegin(ctx)
+  if err != nil {
+    return thrift.PrependError("error reading list begin: ", err)
+  }
+  tSlice := make([]*Response, 0, size)
+  p.Success =  tSlice
+  for i := 0; i < size; i ++ {
+    _elem10 := &Response{}
+    if err := _elem10.Read(ctx, iprot); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem10), err)
+    }
+    p.Success = append(p.Success, _elem10)
+  }
+  if err := iprot.ReadListEnd(ctx); err != nil {
+    return thrift.PrependError("error reading list end: ", err)
+  }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult) Write(ctx context.Context, oprot thrift.TProtocol) error {
+  if err := oprot.WriteStructBegin(ctx, "submitZipkinBatch_result"); err != nil {
+    return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) }
+  if p != nil {
+    if err := p.writeField0(ctx, oprot); err != nil { return err }
+  }
+  if err := oprot.WriteFieldStop(ctx); err != nil {
+    return thrift.PrependError("write field stop error: ", err) }
+  if err := oprot.WriteStructEnd(ctx); err != nil {
+    return thrift.PrependError("write struct stop error: ", err) }
+  return nil
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult) writeField0(ctx context.Context, oprot thrift.TProtocol) (err error) {
+  if p.IsSetSuccess() {
+    if err := oprot.WriteFieldBegin(ctx, "success", thrift.LIST, 0); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) }
+    if err := oprot.WriteListBegin(ctx, thrift.STRUCT, len(p.Success)); err != nil {
+      return thrift.PrependError("error writing list begin: ", err)
+    }
+    for _, v := range p.Success {
+      if err := v.Write(ctx, oprot); err != nil {
+        return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err)
+      }
+    }
+    if err := oprot.WriteListEnd(ctx); err != nil {
+      return thrift.PrependError("error writing list end: ", err)
+    }
+    if err := oprot.WriteFieldEnd(ctx); err != nil {
+      return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) }
+  }
+  return err
+}
+
+func (p *ZipkinCollectorSubmitZipkinBatchResult) String() string {
+  if p == nil {
+    return "<nil>"
+  }
+  return fmt.Sprintf("ZipkinCollectorSubmitZipkinBatchResult(%+v)", *p)
+}
+
+
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/README.md b/vendor/github.com/uber/jaeger-client-go/thrift/README.md
index 1d8e642..c4c38ae 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/README.md
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/README.md
@@ -1,7 +1,11 @@
 # Apache Thrift
 
-This is a partial copy of Apache Thrift v0.10 (https://github.com/apache/thrift/commit/b2a4d4ae21c789b689dd162deb819665567f481c).
+This is a partial copy of Apache Thrift v0.14.1 (https://github.com/apache/thrift/commit/f6fa1794539e68ac294038ac388d6bde40a6c237).
 
-It is vendored code to avoid compatibility issues introduced in Thrift  v0.11.
+It is vendored code to avoid compatibility issues with Thrift versions.
 
-See https://github.com/jaegertracing/jaeger-client-go/pull/303.
+The file logger.go is modified to remove dependency on "testing" (see Issue #585).
+
+See:
+  * https://github.com/jaegertracing/jaeger-client-go/pull/584
+  * https://github.com/jaegertracing/jaeger-client-go/pull/303
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/application_exception.go b/vendor/github.com/uber/jaeger-client-go/thrift/application_exception.go
index 6655cc5..32d5b01 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/application_exception.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/application_exception.go
@@ -19,6 +19,10 @@
 
 package thrift
 
+import (
+	"context"
+)
+
 const (
 	UNKNOWN_APPLICATION_EXCEPTION  = 0
 	UNKNOWN_METHOD                 = 1
@@ -28,14 +32,31 @@
 	MISSING_RESULT                 = 5
 	INTERNAL_ERROR                 = 6
 	PROTOCOL_ERROR                 = 7
+	INVALID_TRANSFORM              = 8
+	INVALID_PROTOCOL               = 9
+	UNSUPPORTED_CLIENT_TYPE        = 10
 )
 
+var defaultApplicationExceptionMessage = map[int32]string{
+	UNKNOWN_APPLICATION_EXCEPTION:  "unknown application exception",
+	UNKNOWN_METHOD:                 "unknown method",
+	INVALID_MESSAGE_TYPE_EXCEPTION: "invalid message type",
+	WRONG_METHOD_NAME:              "wrong method name",
+	BAD_SEQUENCE_ID:                "bad sequence ID",
+	MISSING_RESULT:                 "missing result",
+	INTERNAL_ERROR:                 "unknown internal error",
+	PROTOCOL_ERROR:                 "unknown protocol error",
+	INVALID_TRANSFORM:              "Invalid transform",
+	INVALID_PROTOCOL:               "Invalid protocol",
+	UNSUPPORTED_CLIENT_TYPE:        "Unsupported client type",
+}
+
 // Application level Thrift exception
 type TApplicationException interface {
 	TException
 	TypeId() int32
-	Read(iprot TProtocol) (TApplicationException, error)
-	Write(oprot TProtocol) error
+	Read(ctx context.Context, iprot TProtocol) error
+	Write(ctx context.Context, oprot TProtocol) error
 }
 
 type tApplicationException struct {
@@ -43,8 +64,17 @@
 	type_   int32
 }
 
+var _ TApplicationException = (*tApplicationException)(nil)
+
+func (tApplicationException) TExceptionType() TExceptionType {
+	return TExceptionTypeApplication
+}
+
 func (e tApplicationException) Error() string {
-	return e.message
+	if e.message != "" {
+		return e.message
+	}
+	return defaultApplicationExceptionMessage[e.type_]
 }
 
 func NewTApplicationException(type_ int32, message string) TApplicationException {
@@ -55,19 +85,20 @@
 	return p.type_
 }
 
-func (p *tApplicationException) Read(iprot TProtocol) (TApplicationException, error) {
-	_, err := iprot.ReadStructBegin()
+func (p *tApplicationException) Read(ctx context.Context, iprot TProtocol) error {
+	// TODO: this should really be generated by the compiler
+	_, err := iprot.ReadStructBegin(ctx)
 	if err != nil {
-		return nil, err
+		return err
 	}
 
 	message := ""
 	type_ := int32(UNKNOWN_APPLICATION_EXCEPTION)
 
 	for {
-		_, ttype, id, err := iprot.ReadFieldBegin()
+		_, ttype, id, err := iprot.ReadFieldBegin(ctx)
 		if err != nil {
-			return nil, err
+			return err
 		}
 		if ttype == STOP {
 			break
@@ -75,68 +106,75 @@
 		switch id {
 		case 1:
 			if ttype == STRING {
-				if message, err = iprot.ReadString(); err != nil {
-					return nil, err
+				if message, err = iprot.ReadString(ctx); err != nil {
+					return err
 				}
 			} else {
-				if err = SkipDefaultDepth(iprot, ttype); err != nil {
-					return nil, err
+				if err = SkipDefaultDepth(ctx, iprot, ttype); err != nil {
+					return err
 				}
 			}
 		case 2:
 			if ttype == I32 {
-				if type_, err = iprot.ReadI32(); err != nil {
-					return nil, err
+				if type_, err = iprot.ReadI32(ctx); err != nil {
+					return err
 				}
 			} else {
-				if err = SkipDefaultDepth(iprot, ttype); err != nil {
-					return nil, err
+				if err = SkipDefaultDepth(ctx, iprot, ttype); err != nil {
+					return err
 				}
 			}
 		default:
-			if err = SkipDefaultDepth(iprot, ttype); err != nil {
-				return nil, err
+			if err = SkipDefaultDepth(ctx, iprot, ttype); err != nil {
+				return err
 			}
 		}
-		if err = iprot.ReadFieldEnd(); err != nil {
-			return nil, err
+		if err = iprot.ReadFieldEnd(ctx); err != nil {
+			return err
 		}
 	}
-	return NewTApplicationException(type_, message), iprot.ReadStructEnd()
+	if err := iprot.ReadStructEnd(ctx); err != nil {
+		return err
+	}
+
+	p.message = message
+	p.type_ = type_
+
+	return nil
 }
 
-func (p *tApplicationException) Write(oprot TProtocol) (err error) {
-	err = oprot.WriteStructBegin("TApplicationException")
+func (p *tApplicationException) Write(ctx context.Context, oprot TProtocol) (err error) {
+	err = oprot.WriteStructBegin(ctx, "TApplicationException")
 	if len(p.Error()) > 0 {
-		err = oprot.WriteFieldBegin("message", STRING, 1)
+		err = oprot.WriteFieldBegin(ctx, "message", STRING, 1)
 		if err != nil {
 			return
 		}
-		err = oprot.WriteString(p.Error())
+		err = oprot.WriteString(ctx, p.Error())
 		if err != nil {
 			return
 		}
-		err = oprot.WriteFieldEnd()
+		err = oprot.WriteFieldEnd(ctx)
 		if err != nil {
 			return
 		}
 	}
-	err = oprot.WriteFieldBegin("type", I32, 2)
+	err = oprot.WriteFieldBegin(ctx, "type", I32, 2)
 	if err != nil {
 		return
 	}
-	err = oprot.WriteI32(p.type_)
+	err = oprot.WriteI32(ctx, p.type_)
 	if err != nil {
 		return
 	}
-	err = oprot.WriteFieldEnd()
+	err = oprot.WriteFieldEnd(ctx)
 	if err != nil {
 		return
 	}
-	err = oprot.WriteFieldStop()
+	err = oprot.WriteFieldStop(ctx)
 	if err != nil {
 		return
 	}
-	err = oprot.WriteStructEnd()
+	err = oprot.WriteStructEnd(ctx)
 	return
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/binary_protocol.go b/vendor/github.com/uber/jaeger-client-go/thrift/binary_protocol.go
index 690d341..45c880d 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/binary_protocol.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/binary_protocol.go
@@ -21,6 +21,7 @@
 
 import (
 	"bytes"
+	"context"
 	"encoding/binary"
 	"errors"
 	"fmt"
@@ -31,190 +32,220 @@
 type TBinaryProtocol struct {
 	trans         TRichTransport
 	origTransport TTransport
-	reader        io.Reader
-	writer        io.Writer
-	strictRead    bool
-	strictWrite   bool
+	cfg           *TConfiguration
 	buffer        [64]byte
 }
 
 type TBinaryProtocolFactory struct {
-	strictRead  bool
-	strictWrite bool
+	cfg *TConfiguration
 }
 
+// Deprecated: Use NewTBinaryProtocolConf instead.
 func NewTBinaryProtocolTransport(t TTransport) *TBinaryProtocol {
-	return NewTBinaryProtocol(t, false, true)
+	return NewTBinaryProtocolConf(t, &TConfiguration{
+		noPropagation: true,
+	})
 }
 
+// Deprecated: Use NewTBinaryProtocolConf instead.
 func NewTBinaryProtocol(t TTransport, strictRead, strictWrite bool) *TBinaryProtocol {
-	p := &TBinaryProtocol{origTransport: t, strictRead: strictRead, strictWrite: strictWrite}
+	return NewTBinaryProtocolConf(t, &TConfiguration{
+		TBinaryStrictRead:  &strictRead,
+		TBinaryStrictWrite: &strictWrite,
+
+		noPropagation: true,
+	})
+}
+
+func NewTBinaryProtocolConf(t TTransport, conf *TConfiguration) *TBinaryProtocol {
+	PropagateTConfiguration(t, conf)
+	p := &TBinaryProtocol{
+		origTransport: t,
+		cfg:           conf,
+	}
 	if et, ok := t.(TRichTransport); ok {
 		p.trans = et
 	} else {
 		p.trans = NewTRichTransport(t)
 	}
-	p.reader = p.trans
-	p.writer = p.trans
 	return p
 }
 
+// Deprecated: Use NewTBinaryProtocolFactoryConf instead.
 func NewTBinaryProtocolFactoryDefault() *TBinaryProtocolFactory {
-	return NewTBinaryProtocolFactory(false, true)
+	return NewTBinaryProtocolFactoryConf(&TConfiguration{
+		noPropagation: true,
+	})
 }
 
+// Deprecated: Use NewTBinaryProtocolFactoryConf instead.
 func NewTBinaryProtocolFactory(strictRead, strictWrite bool) *TBinaryProtocolFactory {
-	return &TBinaryProtocolFactory{strictRead: strictRead, strictWrite: strictWrite}
+	return NewTBinaryProtocolFactoryConf(&TConfiguration{
+		TBinaryStrictRead:  &strictRead,
+		TBinaryStrictWrite: &strictWrite,
+
+		noPropagation: true,
+	})
+}
+
+func NewTBinaryProtocolFactoryConf(conf *TConfiguration) *TBinaryProtocolFactory {
+	return &TBinaryProtocolFactory{
+		cfg: conf,
+	}
 }
 
 func (p *TBinaryProtocolFactory) GetProtocol(t TTransport) TProtocol {
-	return NewTBinaryProtocol(t, p.strictRead, p.strictWrite)
+	return NewTBinaryProtocolConf(t, p.cfg)
+}
+
+func (p *TBinaryProtocolFactory) SetTConfiguration(conf *TConfiguration) {
+	p.cfg = conf
 }
 
 /**
  * Writing Methods
  */
 
-func (p *TBinaryProtocol) WriteMessageBegin(name string, typeId TMessageType, seqId int32) error {
-	if p.strictWrite {
+func (p *TBinaryProtocol) WriteMessageBegin(ctx context.Context, name string, typeId TMessageType, seqId int32) error {
+	if p.cfg.GetTBinaryStrictWrite() {
 		version := uint32(VERSION_1) | uint32(typeId)
-		e := p.WriteI32(int32(version))
+		e := p.WriteI32(ctx, int32(version))
 		if e != nil {
 			return e
 		}
-		e = p.WriteString(name)
+		e = p.WriteString(ctx, name)
 		if e != nil {
 			return e
 		}
-		e = p.WriteI32(seqId)
+		e = p.WriteI32(ctx, seqId)
 		return e
 	} else {
-		e := p.WriteString(name)
+		e := p.WriteString(ctx, name)
 		if e != nil {
 			return e
 		}
-		e = p.WriteByte(int8(typeId))
+		e = p.WriteByte(ctx, int8(typeId))
 		if e != nil {
 			return e
 		}
-		e = p.WriteI32(seqId)
+		e = p.WriteI32(ctx, seqId)
 		return e
 	}
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteMessageEnd() error {
+func (p *TBinaryProtocol) WriteMessageEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteStructBegin(name string) error {
+func (p *TBinaryProtocol) WriteStructBegin(ctx context.Context, name string) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteStructEnd() error {
+func (p *TBinaryProtocol) WriteStructEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteFieldBegin(name string, typeId TType, id int16) error {
-	e := p.WriteByte(int8(typeId))
+func (p *TBinaryProtocol) WriteFieldBegin(ctx context.Context, name string, typeId TType, id int16) error {
+	e := p.WriteByte(ctx, int8(typeId))
 	if e != nil {
 		return e
 	}
-	e = p.WriteI16(id)
+	e = p.WriteI16(ctx, id)
 	return e
 }
 
-func (p *TBinaryProtocol) WriteFieldEnd() error {
+func (p *TBinaryProtocol) WriteFieldEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteFieldStop() error {
-	e := p.WriteByte(STOP)
+func (p *TBinaryProtocol) WriteFieldStop(ctx context.Context) error {
+	e := p.WriteByte(ctx, STOP)
 	return e
 }
 
-func (p *TBinaryProtocol) WriteMapBegin(keyType TType, valueType TType, size int) error {
-	e := p.WriteByte(int8(keyType))
+func (p *TBinaryProtocol) WriteMapBegin(ctx context.Context, keyType TType, valueType TType, size int) error {
+	e := p.WriteByte(ctx, int8(keyType))
 	if e != nil {
 		return e
 	}
-	e = p.WriteByte(int8(valueType))
+	e = p.WriteByte(ctx, int8(valueType))
 	if e != nil {
 		return e
 	}
-	e = p.WriteI32(int32(size))
+	e = p.WriteI32(ctx, int32(size))
 	return e
 }
 
-func (p *TBinaryProtocol) WriteMapEnd() error {
+func (p *TBinaryProtocol) WriteMapEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteListBegin(elemType TType, size int) error {
-	e := p.WriteByte(int8(elemType))
+func (p *TBinaryProtocol) WriteListBegin(ctx context.Context, elemType TType, size int) error {
+	e := p.WriteByte(ctx, int8(elemType))
 	if e != nil {
 		return e
 	}
-	e = p.WriteI32(int32(size))
+	e = p.WriteI32(ctx, int32(size))
 	return e
 }
 
-func (p *TBinaryProtocol) WriteListEnd() error {
+func (p *TBinaryProtocol) WriteListEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteSetBegin(elemType TType, size int) error {
-	e := p.WriteByte(int8(elemType))
+func (p *TBinaryProtocol) WriteSetBegin(ctx context.Context, elemType TType, size int) error {
+	e := p.WriteByte(ctx, int8(elemType))
 	if e != nil {
 		return e
 	}
-	e = p.WriteI32(int32(size))
+	e = p.WriteI32(ctx, int32(size))
 	return e
 }
 
-func (p *TBinaryProtocol) WriteSetEnd() error {
+func (p *TBinaryProtocol) WriteSetEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) WriteBool(value bool) error {
+func (p *TBinaryProtocol) WriteBool(ctx context.Context, value bool) error {
 	if value {
-		return p.WriteByte(1)
+		return p.WriteByte(ctx, 1)
 	}
-	return p.WriteByte(0)
+	return p.WriteByte(ctx, 0)
 }
 
-func (p *TBinaryProtocol) WriteByte(value int8) error {
+func (p *TBinaryProtocol) WriteByte(ctx context.Context, value int8) error {
 	e := p.trans.WriteByte(byte(value))
 	return NewTProtocolException(e)
 }
 
-func (p *TBinaryProtocol) WriteI16(value int16) error {
+func (p *TBinaryProtocol) WriteI16(ctx context.Context, value int16) error {
 	v := p.buffer[0:2]
 	binary.BigEndian.PutUint16(v, uint16(value))
-	_, e := p.writer.Write(v)
+	_, e := p.trans.Write(v)
 	return NewTProtocolException(e)
 }
 
-func (p *TBinaryProtocol) WriteI32(value int32) error {
+func (p *TBinaryProtocol) WriteI32(ctx context.Context, value int32) error {
 	v := p.buffer[0:4]
 	binary.BigEndian.PutUint32(v, uint32(value))
-	_, e := p.writer.Write(v)
+	_, e := p.trans.Write(v)
 	return NewTProtocolException(e)
 }
 
-func (p *TBinaryProtocol) WriteI64(value int64) error {
+func (p *TBinaryProtocol) WriteI64(ctx context.Context, value int64) error {
 	v := p.buffer[0:8]
 	binary.BigEndian.PutUint64(v, uint64(value))
-	_, err := p.writer.Write(v)
+	_, err := p.trans.Write(v)
 	return NewTProtocolException(err)
 }
 
-func (p *TBinaryProtocol) WriteDouble(value float64) error {
-	return p.WriteI64(int64(math.Float64bits(value)))
+func (p *TBinaryProtocol) WriteDouble(ctx context.Context, value float64) error {
+	return p.WriteI64(ctx, int64(math.Float64bits(value)))
 }
 
-func (p *TBinaryProtocol) WriteString(value string) error {
-	e := p.WriteI32(int32(len(value)))
+func (p *TBinaryProtocol) WriteString(ctx context.Context, value string) error {
+	e := p.WriteI32(ctx, int32(len(value)))
 	if e != nil {
 		return e
 	}
@@ -222,12 +253,12 @@
 	return NewTProtocolException(err)
 }
 
-func (p *TBinaryProtocol) WriteBinary(value []byte) error {
-	e := p.WriteI32(int32(len(value)))
+func (p *TBinaryProtocol) WriteBinary(ctx context.Context, value []byte) error {
+	e := p.WriteI32(ctx, int32(len(value)))
 	if e != nil {
 		return e
 	}
-	_, err := p.writer.Write(value)
+	_, err := p.trans.Write(value)
 	return NewTProtocolException(err)
 }
 
@@ -235,8 +266,8 @@
  * Reading methods
  */
 
-func (p *TBinaryProtocol) ReadMessageBegin() (name string, typeId TMessageType, seqId int32, err error) {
-	size, e := p.ReadI32()
+func (p *TBinaryProtocol) ReadMessageBegin(ctx context.Context) (name string, typeId TMessageType, seqId int32, err error) {
+	size, e := p.ReadI32(ctx)
 	if e != nil {
 		return "", typeId, 0, NewTProtocolException(e)
 	}
@@ -246,79 +277,79 @@
 		if version != VERSION_1 {
 			return name, typeId, seqId, NewTProtocolExceptionWithType(BAD_VERSION, fmt.Errorf("Bad version in ReadMessageBegin"))
 		}
-		name, e = p.ReadString()
+		name, e = p.ReadString(ctx)
 		if e != nil {
 			return name, typeId, seqId, NewTProtocolException(e)
 		}
-		seqId, e = p.ReadI32()
+		seqId, e = p.ReadI32(ctx)
 		if e != nil {
 			return name, typeId, seqId, NewTProtocolException(e)
 		}
 		return name, typeId, seqId, nil
 	}
-	if p.strictRead {
+	if p.cfg.GetTBinaryStrictRead() {
 		return name, typeId, seqId, NewTProtocolExceptionWithType(BAD_VERSION, fmt.Errorf("Missing version in ReadMessageBegin"))
 	}
 	name, e2 := p.readStringBody(size)
 	if e2 != nil {
 		return name, typeId, seqId, e2
 	}
-	b, e3 := p.ReadByte()
+	b, e3 := p.ReadByte(ctx)
 	if e3 != nil {
 		return name, typeId, seqId, e3
 	}
 	typeId = TMessageType(b)
-	seqId, e4 := p.ReadI32()
+	seqId, e4 := p.ReadI32(ctx)
 	if e4 != nil {
 		return name, typeId, seqId, e4
 	}
 	return name, typeId, seqId, nil
 }
 
-func (p *TBinaryProtocol) ReadMessageEnd() error {
+func (p *TBinaryProtocol) ReadMessageEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) ReadStructBegin() (name string, err error) {
+func (p *TBinaryProtocol) ReadStructBegin(ctx context.Context) (name string, err error) {
 	return
 }
 
-func (p *TBinaryProtocol) ReadStructEnd() error {
+func (p *TBinaryProtocol) ReadStructEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) ReadFieldBegin() (name string, typeId TType, seqId int16, err error) {
-	t, err := p.ReadByte()
+func (p *TBinaryProtocol) ReadFieldBegin(ctx context.Context) (name string, typeId TType, seqId int16, err error) {
+	t, err := p.ReadByte(ctx)
 	typeId = TType(t)
 	if err != nil {
 		return name, typeId, seqId, err
 	}
 	if t != STOP {
-		seqId, err = p.ReadI16()
+		seqId, err = p.ReadI16(ctx)
 	}
 	return name, typeId, seqId, err
 }
 
-func (p *TBinaryProtocol) ReadFieldEnd() error {
+func (p *TBinaryProtocol) ReadFieldEnd(ctx context.Context) error {
 	return nil
 }
 
 var invalidDataLength = NewTProtocolExceptionWithType(INVALID_DATA, errors.New("Invalid data length"))
 
-func (p *TBinaryProtocol) ReadMapBegin() (kType, vType TType, size int, err error) {
-	k, e := p.ReadByte()
+func (p *TBinaryProtocol) ReadMapBegin(ctx context.Context) (kType, vType TType, size int, err error) {
+	k, e := p.ReadByte(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
 	}
 	kType = TType(k)
-	v, e := p.ReadByte()
+	v, e := p.ReadByte(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
 	}
 	vType = TType(v)
-	size32, e := p.ReadI32()
+	size32, e := p.ReadI32(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
@@ -331,18 +362,18 @@
 	return kType, vType, size, nil
 }
 
-func (p *TBinaryProtocol) ReadMapEnd() error {
+func (p *TBinaryProtocol) ReadMapEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) ReadListBegin() (elemType TType, size int, err error) {
-	b, e := p.ReadByte()
+func (p *TBinaryProtocol) ReadListBegin(ctx context.Context) (elemType TType, size int, err error) {
+	b, e := p.ReadByte(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
 	}
 	elemType = TType(b)
-	size32, e := p.ReadI32()
+	size32, e := p.ReadI32(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
@@ -356,18 +387,18 @@
 	return
 }
 
-func (p *TBinaryProtocol) ReadListEnd() error {
+func (p *TBinaryProtocol) ReadListEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) ReadSetBegin() (elemType TType, size int, err error) {
-	b, e := p.ReadByte()
+func (p *TBinaryProtocol) ReadSetBegin(ctx context.Context) (elemType TType, size int, err error) {
+	b, e := p.ReadByte(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
 	}
 	elemType = TType(b)
-	size32, e := p.ReadI32()
+	size32, e := p.ReadI32(ctx)
 	if e != nil {
 		err = NewTProtocolException(e)
 		return
@@ -380,12 +411,12 @@
 	return elemType, size, nil
 }
 
-func (p *TBinaryProtocol) ReadSetEnd() error {
+func (p *TBinaryProtocol) ReadSetEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TBinaryProtocol) ReadBool() (bool, error) {
-	b, e := p.ReadByte()
+func (p *TBinaryProtocol) ReadBool(ctx context.Context) (bool, error) {
+	b, e := p.ReadByte(ctx)
 	v := true
 	if b != 1 {
 		v = false
@@ -393,122 +424,132 @@
 	return v, e
 }
 
-func (p *TBinaryProtocol) ReadByte() (int8, error) {
+func (p *TBinaryProtocol) ReadByte(ctx context.Context) (int8, error) {
 	v, err := p.trans.ReadByte()
 	return int8(v), err
 }
 
-func (p *TBinaryProtocol) ReadI16() (value int16, err error) {
+func (p *TBinaryProtocol) ReadI16(ctx context.Context) (value int16, err error) {
 	buf := p.buffer[0:2]
-	err = p.readAll(buf)
+	err = p.readAll(ctx, buf)
 	value = int16(binary.BigEndian.Uint16(buf))
 	return value, err
 }
 
-func (p *TBinaryProtocol) ReadI32() (value int32, err error) {
+func (p *TBinaryProtocol) ReadI32(ctx context.Context) (value int32, err error) {
 	buf := p.buffer[0:4]
-	err = p.readAll(buf)
+	err = p.readAll(ctx, buf)
 	value = int32(binary.BigEndian.Uint32(buf))
 	return value, err
 }
 
-func (p *TBinaryProtocol) ReadI64() (value int64, err error) {
+func (p *TBinaryProtocol) ReadI64(ctx context.Context) (value int64, err error) {
 	buf := p.buffer[0:8]
-	err = p.readAll(buf)
+	err = p.readAll(ctx, buf)
 	value = int64(binary.BigEndian.Uint64(buf))
 	return value, err
 }
 
-func (p *TBinaryProtocol) ReadDouble() (value float64, err error) {
+func (p *TBinaryProtocol) ReadDouble(ctx context.Context) (value float64, err error) {
 	buf := p.buffer[0:8]
-	err = p.readAll(buf)
+	err = p.readAll(ctx, buf)
 	value = math.Float64frombits(binary.BigEndian.Uint64(buf))
 	return value, err
 }
 
-func (p *TBinaryProtocol) ReadString() (value string, err error) {
-	size, e := p.ReadI32()
+func (p *TBinaryProtocol) ReadString(ctx context.Context) (value string, err error) {
+	size, e := p.ReadI32(ctx)
 	if e != nil {
 		return "", e
 	}
+	err = checkSizeForProtocol(size, p.cfg)
+	if err != nil {
+		return
+	}
 	if size < 0 {
 		err = invalidDataLength
 		return
 	}
+	if size == 0 {
+		return "", nil
+	}
+	if size < int32(len(p.buffer)) {
+		// Avoid allocation on small reads
+		buf := p.buffer[:size]
+		read, e := io.ReadFull(p.trans, buf)
+		return string(buf[:read]), NewTProtocolException(e)
+	}
 
 	return p.readStringBody(size)
 }
 
-func (p *TBinaryProtocol) ReadBinary() ([]byte, error) {
-	size, e := p.ReadI32()
+func (p *TBinaryProtocol) ReadBinary(ctx context.Context) ([]byte, error) {
+	size, e := p.ReadI32(ctx)
 	if e != nil {
 		return nil, e
 	}
-	if size < 0 {
-		return nil, invalidDataLength
-	}
-	if uint64(size) > p.trans.RemainingBytes() {
-		return nil, invalidDataLength
+	if err := checkSizeForProtocol(size, p.cfg); err != nil {
+		return nil, err
 	}
 
-	isize := int(size)
-	buf := make([]byte, isize)
-	_, err := io.ReadFull(p.trans, buf)
+	buf, err := safeReadBytes(size, p.trans)
 	return buf, NewTProtocolException(err)
 }
 
-func (p *TBinaryProtocol) Flush() (err error) {
-	return NewTProtocolException(p.trans.Flush())
+func (p *TBinaryProtocol) Flush(ctx context.Context) (err error) {
+	return NewTProtocolException(p.trans.Flush(ctx))
 }
 
-func (p *TBinaryProtocol) Skip(fieldType TType) (err error) {
-	return SkipDefaultDepth(p, fieldType)
+func (p *TBinaryProtocol) Skip(ctx context.Context, fieldType TType) (err error) {
+	return SkipDefaultDepth(ctx, p, fieldType)
 }
 
 func (p *TBinaryProtocol) Transport() TTransport {
 	return p.origTransport
 }
 
-func (p *TBinaryProtocol) readAll(buf []byte) error {
-	_, err := io.ReadFull(p.reader, buf)
+func (p *TBinaryProtocol) readAll(ctx context.Context, buf []byte) (err error) {
+	var read int
+	_, deadlineSet := ctx.Deadline()
+	for {
+		read, err = io.ReadFull(p.trans, buf)
+		if deadlineSet && read == 0 && isTimeoutError(err) && ctx.Err() == nil {
+			// This is I/O timeout without anything read,
+			// and we still have time left, keep retrying.
+			continue
+		}
+		// For anything else, don't retry
+		break
+	}
 	return NewTProtocolException(err)
 }
 
-const readLimit = 32768
-
 func (p *TBinaryProtocol) readStringBody(size int32) (value string, err error) {
+	buf, err := safeReadBytes(size, p.trans)
+	return string(buf), NewTProtocolException(err)
+}
+
+func (p *TBinaryProtocol) SetTConfiguration(conf *TConfiguration) {
+	PropagateTConfiguration(p.trans, conf)
+	PropagateTConfiguration(p.origTransport, conf)
+	p.cfg = conf
+}
+
+var (
+	_ TConfigurationSetter = (*TBinaryProtocolFactory)(nil)
+	_ TConfigurationSetter = (*TBinaryProtocol)(nil)
+)
+
+// This function is shared between TBinaryProtocol and TCompactProtocol.
+//
+// It tries to read size bytes from trans, in a way that prevents large
+// allocations when size is insanely large (mostly caused by malformed message).
+func safeReadBytes(size int32, trans io.Reader) ([]byte, error) {
 	if size < 0 {
-		return "", nil
-	}
-	if uint64(size) > p.trans.RemainingBytes() {
-		return "", invalidDataLength
+		return nil, nil
 	}
 
-	var (
-		buf bytes.Buffer
-		e   error
-		b   []byte
-	)
-
-	switch {
-	case int(size) <= len(p.buffer):
-		b = p.buffer[:size] // avoids allocation for small reads
-	case int(size) < readLimit:
-		b = make([]byte, size)
-	default:
-		b = make([]byte, readLimit)
-	}
-
-	for size > 0 {
-		_, e = io.ReadFull(p.trans, b)
-		buf.Write(b)
-		if e != nil {
-			break
-		}
-		size -= readLimit
-		if size < readLimit && size > 0 {
-			b = b[:size]
-		}
-	}
-	return buf.String(), NewTProtocolException(e)
+	buf := new(bytes.Buffer)
+	_, err := io.CopyN(buf, trans, int64(size))
+	return buf.Bytes(), err
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/client.go b/vendor/github.com/uber/jaeger-client-go/thrift/client.go
new file mode 100644
index 0000000..ea2c01f
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/client.go
@@ -0,0 +1,109 @@
+package thrift
+
+import (
+	"context"
+	"fmt"
+)
+
+// ResponseMeta represents the metadata attached to the response.
+type ResponseMeta struct {
+	// The headers in the response, if any.
+	// If the underlying transport/protocol is not THeader, this will always be nil.
+	Headers THeaderMap
+}
+
+type TClient interface {
+	Call(ctx context.Context, method string, args, result TStruct) (ResponseMeta, error)
+}
+
+type TStandardClient struct {
+	seqId        int32
+	iprot, oprot TProtocol
+}
+
+// TStandardClient implements TClient, and uses the standard message format for Thrift.
+// It is not safe for concurrent use.
+func NewTStandardClient(inputProtocol, outputProtocol TProtocol) *TStandardClient {
+	return &TStandardClient{
+		iprot: inputProtocol,
+		oprot: outputProtocol,
+	}
+}
+
+func (p *TStandardClient) Send(ctx context.Context, oprot TProtocol, seqId int32, method string, args TStruct) error {
+	// Set headers from context object on THeaderProtocol
+	if headerProt, ok := oprot.(*THeaderProtocol); ok {
+		headerProt.ClearWriteHeaders()
+		for _, key := range GetWriteHeaderList(ctx) {
+			if value, ok := GetHeader(ctx, key); ok {
+				headerProt.SetWriteHeader(key, value)
+			}
+		}
+	}
+
+	if err := oprot.WriteMessageBegin(ctx, method, CALL, seqId); err != nil {
+		return err
+	}
+	if err := args.Write(ctx, oprot); err != nil {
+		return err
+	}
+	if err := oprot.WriteMessageEnd(ctx); err != nil {
+		return err
+	}
+	return oprot.Flush(ctx)
+}
+
+func (p *TStandardClient) Recv(ctx context.Context, iprot TProtocol, seqId int32, method string, result TStruct) error {
+	rMethod, rTypeId, rSeqId, err := iprot.ReadMessageBegin(ctx)
+	if err != nil {
+		return err
+	}
+
+	if method != rMethod {
+		return NewTApplicationException(WRONG_METHOD_NAME, fmt.Sprintf("%s: wrong method name", method))
+	} else if seqId != rSeqId {
+		return NewTApplicationException(BAD_SEQUENCE_ID, fmt.Sprintf("%s: out of order sequence response", method))
+	} else if rTypeId == EXCEPTION {
+		var exception tApplicationException
+		if err := exception.Read(ctx, iprot); err != nil {
+			return err
+		}
+
+		if err := iprot.ReadMessageEnd(ctx); err != nil {
+			return err
+		}
+
+		return &exception
+	} else if rTypeId != REPLY {
+		return NewTApplicationException(INVALID_MESSAGE_TYPE_EXCEPTION, fmt.Sprintf("%s: invalid message type", method))
+	}
+
+	if err := result.Read(ctx, iprot); err != nil {
+		return err
+	}
+
+	return iprot.ReadMessageEnd(ctx)
+}
+
+func (p *TStandardClient) Call(ctx context.Context, method string, args, result TStruct) (ResponseMeta, error) {
+	p.seqId++
+	seqId := p.seqId
+
+	if err := p.Send(ctx, p.oprot, seqId, method, args); err != nil {
+		return ResponseMeta{}, err
+	}
+
+	// method is oneway
+	if result == nil {
+		return ResponseMeta{}, nil
+	}
+
+	err := p.Recv(ctx, p.iprot, seqId, method, result)
+	var headers THeaderMap
+	if hp, ok := p.iprot.(*THeaderProtocol); ok {
+		headers = hp.transport.readHeaders
+	}
+	return ResponseMeta{
+		Headers: headers,
+	}, err
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/compact_protocol.go b/vendor/github.com/uber/jaeger-client-go/thrift/compact_protocol.go
index b9299f2..a49225d 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/compact_protocol.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/compact_protocol.go
@@ -20,7 +20,9 @@
 package thrift
 
 import (
+	"context"
 	"encoding/binary"
+	"errors"
 	"fmt"
 	"io"
 	"math"
@@ -73,20 +75,37 @@
 	}
 }
 
-type TCompactProtocolFactory struct{}
+type TCompactProtocolFactory struct {
+	cfg *TConfiguration
+}
 
+// Deprecated: Use NewTCompactProtocolFactoryConf instead.
 func NewTCompactProtocolFactory() *TCompactProtocolFactory {
-	return &TCompactProtocolFactory{}
+	return NewTCompactProtocolFactoryConf(&TConfiguration{
+		noPropagation: true,
+	})
+}
+
+func NewTCompactProtocolFactoryConf(conf *TConfiguration) *TCompactProtocolFactory {
+	return &TCompactProtocolFactory{
+		cfg: conf,
+	}
 }
 
 func (p *TCompactProtocolFactory) GetProtocol(trans TTransport) TProtocol {
-	return NewTCompactProtocol(trans)
+	return NewTCompactProtocolConf(trans, p.cfg)
+}
+
+func (p *TCompactProtocolFactory) SetTConfiguration(conf *TConfiguration) {
+	p.cfg = conf
 }
 
 type TCompactProtocol struct {
 	trans         TRichTransport
 	origTransport TTransport
 
+	cfg *TConfiguration
+
 	// Used to keep track of the last field for the current and previous structs,
 	// so we can do the delta stuff.
 	lastField   []int
@@ -105,9 +124,19 @@
 	buffer             [64]byte
 }
 
-// Create a TCompactProtocol given a TTransport
+// Deprecated: Use NewTCompactProtocolConf instead.
 func NewTCompactProtocol(trans TTransport) *TCompactProtocol {
-	p := &TCompactProtocol{origTransport: trans, lastField: []int{}}
+	return NewTCompactProtocolConf(trans, &TConfiguration{
+		noPropagation: true,
+	})
+}
+
+func NewTCompactProtocolConf(trans TTransport, conf *TConfiguration) *TCompactProtocol {
+	PropagateTConfiguration(trans, conf)
+	p := &TCompactProtocol{
+		origTransport: trans,
+		cfg:           conf,
+	}
 	if et, ok := trans.(TRichTransport); ok {
 		p.trans = et
 	} else {
@@ -115,7 +144,6 @@
 	}
 
 	return p
-
 }
 
 //
@@ -124,7 +152,7 @@
 
 // Write a message header to the wire. Compact Protocol messages contain the
 // protocol version so we can migrate forwards in the future if need be.
-func (p *TCompactProtocol) WriteMessageBegin(name string, typeId TMessageType, seqid int32) error {
+func (p *TCompactProtocol) WriteMessageBegin(ctx context.Context, name string, typeId TMessageType, seqid int32) error {
 	err := p.writeByteDirect(COMPACT_PROTOCOL_ID)
 	if err != nil {
 		return NewTProtocolException(err)
@@ -137,17 +165,17 @@
 	if err != nil {
 		return NewTProtocolException(err)
 	}
-	e := p.WriteString(name)
+	e := p.WriteString(ctx, name)
 	return e
 
 }
 
-func (p *TCompactProtocol) WriteMessageEnd() error { return nil }
+func (p *TCompactProtocol) WriteMessageEnd(ctx context.Context) error { return nil }
 
 // Write a struct begin. This doesn't actually put anything on the wire. We
 // use it as an opportunity to put special placeholder markers on the field
 // stack so we can get the field id deltas correct.
-func (p *TCompactProtocol) WriteStructBegin(name string) error {
+func (p *TCompactProtocol) WriteStructBegin(ctx context.Context, name string) error {
 	p.lastField = append(p.lastField, p.lastFieldId)
 	p.lastFieldId = 0
 	return nil
@@ -156,26 +184,29 @@
 // Write a struct end. This doesn't actually put anything on the wire. We use
 // this as an opportunity to pop the last field from the current struct off
 // of the field stack.
-func (p *TCompactProtocol) WriteStructEnd() error {
+func (p *TCompactProtocol) WriteStructEnd(ctx context.Context) error {
+	if len(p.lastField) <= 0 {
+		return NewTProtocolExceptionWithType(INVALID_DATA, errors.New("WriteStructEnd called without matching WriteStructBegin call before"))
+	}
 	p.lastFieldId = p.lastField[len(p.lastField)-1]
 	p.lastField = p.lastField[:len(p.lastField)-1]
 	return nil
 }
 
-func (p *TCompactProtocol) WriteFieldBegin(name string, typeId TType, id int16) error {
+func (p *TCompactProtocol) WriteFieldBegin(ctx context.Context, name string, typeId TType, id int16) error {
 	if typeId == BOOL {
 		// we want to possibly include the value, so we'll wait.
 		p.booleanFieldName, p.booleanFieldId, p.booleanFieldPending = name, id, true
 		return nil
 	}
-	_, err := p.writeFieldBeginInternal(name, typeId, id, 0xFF)
+	_, err := p.writeFieldBeginInternal(ctx, name, typeId, id, 0xFF)
 	return NewTProtocolException(err)
 }
 
 // The workhorse of writeFieldBegin. It has the option of doing a
 // 'type override' of the type header. This is used specifically in the
 // boolean field case.
-func (p *TCompactProtocol) writeFieldBeginInternal(name string, typeId TType, id int16, typeOverride byte) (int, error) {
+func (p *TCompactProtocol) writeFieldBeginInternal(ctx context.Context, name string, typeId TType, id int16, typeOverride byte) (int, error) {
 	// short lastField = lastField_.pop();
 
 	// if there's a type override, use that.
@@ -200,7 +231,7 @@
 		if err != nil {
 			return 0, err
 		}
-		err = p.WriteI16(id)
+		err = p.WriteI16(ctx, id)
 		written = 1 + 2
 		if err != nil {
 			return 0, err
@@ -208,18 +239,17 @@
 	}
 
 	p.lastFieldId = fieldId
-	// p.lastField.Push(field.id);
 	return written, nil
 }
 
-func (p *TCompactProtocol) WriteFieldEnd() error { return nil }
+func (p *TCompactProtocol) WriteFieldEnd(ctx context.Context) error { return nil }
 
-func (p *TCompactProtocol) WriteFieldStop() error {
+func (p *TCompactProtocol) WriteFieldStop(ctx context.Context) error {
 	err := p.writeByteDirect(STOP)
 	return NewTProtocolException(err)
 }
 
-func (p *TCompactProtocol) WriteMapBegin(keyType TType, valueType TType, size int) error {
+func (p *TCompactProtocol) WriteMapBegin(ctx context.Context, keyType TType, valueType TType, size int) error {
 	if size == 0 {
 		err := p.writeByteDirect(0)
 		return NewTProtocolException(err)
@@ -232,32 +262,32 @@
 	return NewTProtocolException(err)
 }
 
-func (p *TCompactProtocol) WriteMapEnd() error { return nil }
+func (p *TCompactProtocol) WriteMapEnd(ctx context.Context) error { return nil }
 
 // Write a list header.
-func (p *TCompactProtocol) WriteListBegin(elemType TType, size int) error {
+func (p *TCompactProtocol) WriteListBegin(ctx context.Context, elemType TType, size int) error {
 	_, err := p.writeCollectionBegin(elemType, size)
 	return NewTProtocolException(err)
 }
 
-func (p *TCompactProtocol) WriteListEnd() error { return nil }
+func (p *TCompactProtocol) WriteListEnd(ctx context.Context) error { return nil }
 
 // Write a set header.
-func (p *TCompactProtocol) WriteSetBegin(elemType TType, size int) error {
+func (p *TCompactProtocol) WriteSetBegin(ctx context.Context, elemType TType, size int) error {
 	_, err := p.writeCollectionBegin(elemType, size)
 	return NewTProtocolException(err)
 }
 
-func (p *TCompactProtocol) WriteSetEnd() error { return nil }
+func (p *TCompactProtocol) WriteSetEnd(ctx context.Context) error { return nil }
 
-func (p *TCompactProtocol) WriteBool(value bool) error {
+func (p *TCompactProtocol) WriteBool(ctx context.Context, value bool) error {
 	v := byte(COMPACT_BOOLEAN_FALSE)
 	if value {
 		v = byte(COMPACT_BOOLEAN_TRUE)
 	}
 	if p.booleanFieldPending {
 		// we haven't written the field header yet
-		_, err := p.writeFieldBeginInternal(p.booleanFieldName, BOOL, p.booleanFieldId, v)
+		_, err := p.writeFieldBeginInternal(ctx, p.booleanFieldName, BOOL, p.booleanFieldId, v)
 		p.booleanFieldPending = false
 		return NewTProtocolException(err)
 	}
@@ -267,31 +297,31 @@
 }
 
 // Write a byte. Nothing to see here!
-func (p *TCompactProtocol) WriteByte(value int8) error {
+func (p *TCompactProtocol) WriteByte(ctx context.Context, value int8) error {
 	err := p.writeByteDirect(byte(value))
 	return NewTProtocolException(err)
 }
 
 // Write an I16 as a zigzag varint.
-func (p *TCompactProtocol) WriteI16(value int16) error {
+func (p *TCompactProtocol) WriteI16(ctx context.Context, value int16) error {
 	_, err := p.writeVarint32(p.int32ToZigzag(int32(value)))
 	return NewTProtocolException(err)
 }
 
 // Write an i32 as a zigzag varint.
-func (p *TCompactProtocol) WriteI32(value int32) error {
+func (p *TCompactProtocol) WriteI32(ctx context.Context, value int32) error {
 	_, err := p.writeVarint32(p.int32ToZigzag(value))
 	return NewTProtocolException(err)
 }
 
 // Write an i64 as a zigzag varint.
-func (p *TCompactProtocol) WriteI64(value int64) error {
+func (p *TCompactProtocol) WriteI64(ctx context.Context, value int64) error {
 	_, err := p.writeVarint64(p.int64ToZigzag(value))
 	return NewTProtocolException(err)
 }
 
 // Write a double to the wire as 8 bytes.
-func (p *TCompactProtocol) WriteDouble(value float64) error {
+func (p *TCompactProtocol) WriteDouble(ctx context.Context, value float64) error {
 	buf := p.buffer[0:8]
 	binary.LittleEndian.PutUint64(buf, math.Float64bits(value))
 	_, err := p.trans.Write(buf)
@@ -299,7 +329,7 @@
 }
 
 // Write a string to the wire with a varint size preceding.
-func (p *TCompactProtocol) WriteString(value string) error {
+func (p *TCompactProtocol) WriteString(ctx context.Context, value string) error {
 	_, e := p.writeVarint32(int32(len(value)))
 	if e != nil {
 		return NewTProtocolException(e)
@@ -311,7 +341,7 @@
 }
 
 // Write a byte array, using a varint for the size.
-func (p *TCompactProtocol) WriteBinary(bin []byte) error {
+func (p *TCompactProtocol) WriteBinary(ctx context.Context, bin []byte) error {
 	_, e := p.writeVarint32(int32(len(bin)))
 	if e != nil {
 		return NewTProtocolException(e)
@@ -328,9 +358,20 @@
 //
 
 // Read a message header.
-func (p *TCompactProtocol) ReadMessageBegin() (name string, typeId TMessageType, seqId int32, err error) {
+func (p *TCompactProtocol) ReadMessageBegin(ctx context.Context) (name string, typeId TMessageType, seqId int32, err error) {
+	var protocolId byte
 
-	protocolId, err := p.readByteDirect()
+	_, deadlineSet := ctx.Deadline()
+	for {
+		protocolId, err = p.readByteDirect()
+		if deadlineSet && isTimeoutError(err) && ctx.Err() == nil {
+			// keep retrying I/O timeout errors since we still have
+			// time left
+			continue
+		}
+		// For anything else, don't retry
+		break
+	}
 	if err != nil {
 		return
 	}
@@ -357,15 +398,15 @@
 		err = NewTProtocolException(e)
 		return
 	}
-	name, err = p.ReadString()
+	name, err = p.ReadString(ctx)
 	return
 }
 
-func (p *TCompactProtocol) ReadMessageEnd() error { return nil }
+func (p *TCompactProtocol) ReadMessageEnd(ctx context.Context) error { return nil }
 
 // Read a struct begin. There's nothing on the wire for this, but it is our
 // opportunity to push a new struct begin marker onto the field stack.
-func (p *TCompactProtocol) ReadStructBegin() (name string, err error) {
+func (p *TCompactProtocol) ReadStructBegin(ctx context.Context) (name string, err error) {
 	p.lastField = append(p.lastField, p.lastFieldId)
 	p.lastFieldId = 0
 	return
@@ -373,15 +414,18 @@
 
 // Doesn't actually consume any wire data, just removes the last field for
 // this struct from the field stack.
-func (p *TCompactProtocol) ReadStructEnd() error {
+func (p *TCompactProtocol) ReadStructEnd(ctx context.Context) error {
 	// consume the last field we read off the wire.
+	if len(p.lastField) <= 0 {
+		return NewTProtocolExceptionWithType(INVALID_DATA, errors.New("ReadStructEnd called without matching ReadStructBegin call before"))
+	}
 	p.lastFieldId = p.lastField[len(p.lastField)-1]
 	p.lastField = p.lastField[:len(p.lastField)-1]
 	return nil
 }
 
 // Read a field header off the wire.
-func (p *TCompactProtocol) ReadFieldBegin() (name string, typeId TType, id int16, err error) {
+func (p *TCompactProtocol) ReadFieldBegin(ctx context.Context) (name string, typeId TType, id int16, err error) {
 	t, err := p.readByteDirect()
 	if err != nil {
 		return
@@ -396,7 +440,7 @@
 	modifier := int16((t & 0xf0) >> 4)
 	if modifier == 0 {
 		// not a delta. look ahead for the zigzag varint field id.
-		id, err = p.ReadI16()
+		id, err = p.ReadI16(ctx)
 		if err != nil {
 			return
 		}
@@ -422,12 +466,12 @@
 	return
 }
 
-func (p *TCompactProtocol) ReadFieldEnd() error { return nil }
+func (p *TCompactProtocol) ReadFieldEnd(ctx context.Context) error { return nil }
 
 // Read a map header off the wire. If the size is zero, skip reading the key
 // and value type. This means that 0-length maps will yield TMaps without the
 // "correct" types.
-func (p *TCompactProtocol) ReadMapBegin() (keyType TType, valueType TType, size int, err error) {
+func (p *TCompactProtocol) ReadMapBegin(ctx context.Context) (keyType TType, valueType TType, size int, err error) {
 	size32, e := p.readVarint32()
 	if e != nil {
 		err = NewTProtocolException(e)
@@ -451,13 +495,13 @@
 	return
 }
 
-func (p *TCompactProtocol) ReadMapEnd() error { return nil }
+func (p *TCompactProtocol) ReadMapEnd(ctx context.Context) error { return nil }
 
 // Read a list header off the wire. If the list size is 0-14, the size will
 // be packed into the element type header. If it's a longer list, the 4 MSB
 // of the element type header will be 0xF, and a varint will follow with the
 // true size.
-func (p *TCompactProtocol) ReadListBegin() (elemType TType, size int, err error) {
+func (p *TCompactProtocol) ReadListBegin(ctx context.Context) (elemType TType, size int, err error) {
 	size_and_type, err := p.readByteDirect()
 	if err != nil {
 		return
@@ -483,22 +527,22 @@
 	return
 }
 
-func (p *TCompactProtocol) ReadListEnd() error { return nil }
+func (p *TCompactProtocol) ReadListEnd(ctx context.Context) error { return nil }
 
 // Read a set header off the wire. If the set size is 0-14, the size will
 // be packed into the element type header. If it's a longer set, the 4 MSB
 // of the element type header will be 0xF, and a varint will follow with the
 // true size.
-func (p *TCompactProtocol) ReadSetBegin() (elemType TType, size int, err error) {
-	return p.ReadListBegin()
+func (p *TCompactProtocol) ReadSetBegin(ctx context.Context) (elemType TType, size int, err error) {
+	return p.ReadListBegin(ctx)
 }
 
-func (p *TCompactProtocol) ReadSetEnd() error { return nil }
+func (p *TCompactProtocol) ReadSetEnd(ctx context.Context) error { return nil }
 
 // Read a boolean off the wire. If this is a boolean field, the value should
 // already have been read during readFieldBegin, so we'll just consume the
 // pre-stored value. Otherwise, read a byte.
-func (p *TCompactProtocol) ReadBool() (value bool, err error) {
+func (p *TCompactProtocol) ReadBool(ctx context.Context) (value bool, err error) {
 	if p.boolValueIsNotNull {
 		p.boolValueIsNotNull = false
 		return p.boolValue, nil
@@ -508,7 +552,7 @@
 }
 
 // Read a single byte off the wire. Nothing interesting here.
-func (p *TCompactProtocol) ReadByte() (int8, error) {
+func (p *TCompactProtocol) ReadByte(ctx context.Context) (int8, error) {
 	v, err := p.readByteDirect()
 	if err != nil {
 		return 0, NewTProtocolException(err)
@@ -517,13 +561,13 @@
 }
 
 // Read an i16 from the wire as a zigzag varint.
-func (p *TCompactProtocol) ReadI16() (value int16, err error) {
-	v, err := p.ReadI32()
+func (p *TCompactProtocol) ReadI16(ctx context.Context) (value int16, err error) {
+	v, err := p.ReadI32(ctx)
 	return int16(v), err
 }
 
 // Read an i32 from the wire as a zigzag varint.
-func (p *TCompactProtocol) ReadI32() (value int32, err error) {
+func (p *TCompactProtocol) ReadI32(ctx context.Context) (value int32, err error) {
 	v, e := p.readVarint32()
 	if e != nil {
 		return 0, NewTProtocolException(e)
@@ -533,7 +577,7 @@
 }
 
 // Read an i64 from the wire as a zigzag varint.
-func (p *TCompactProtocol) ReadI64() (value int64, err error) {
+func (p *TCompactProtocol) ReadI64(ctx context.Context) (value int64, err error) {
 	v, e := p.readVarint64()
 	if e != nil {
 		return 0, NewTProtocolException(e)
@@ -543,7 +587,7 @@
 }
 
 // No magic here - just read a double off the wire.
-func (p *TCompactProtocol) ReadDouble() (value float64, err error) {
+func (p *TCompactProtocol) ReadDouble(ctx context.Context) (value float64, err error) {
 	longBits := p.buffer[0:8]
 	_, e := io.ReadFull(p.trans, longBits)
 	if e != nil {
@@ -553,58 +597,53 @@
 }
 
 // Reads a []byte (via readBinary), and then UTF-8 decodes it.
-func (p *TCompactProtocol) ReadString() (value string, err error) {
+func (p *TCompactProtocol) ReadString(ctx context.Context) (value string, err error) {
 	length, e := p.readVarint32()
 	if e != nil {
 		return "", NewTProtocolException(e)
 	}
-	if length < 0 {
-		return "", invalidDataLength
+	err = checkSizeForProtocol(length, p.cfg)
+	if err != nil {
+		return
 	}
-	if uint64(length) > p.trans.RemainingBytes() {
-		return "", invalidDataLength
-	}
-
 	if length == 0 {
 		return "", nil
 	}
-	var buf []byte
-	if length <= int32(len(p.buffer)) {
-		buf = p.buffer[0:length]
-	} else {
-		buf = make([]byte, length)
+	if length < int32(len(p.buffer)) {
+		// Avoid allocation on small reads
+		buf := p.buffer[:length]
+		read, e := io.ReadFull(p.trans, buf)
+		return string(buf[:read]), NewTProtocolException(e)
 	}
-	_, e = io.ReadFull(p.trans, buf)
+
+	buf, e := safeReadBytes(length, p.trans)
 	return string(buf), NewTProtocolException(e)
 }
 
 // Read a []byte from the wire.
-func (p *TCompactProtocol) ReadBinary() (value []byte, err error) {
+func (p *TCompactProtocol) ReadBinary(ctx context.Context) (value []byte, err error) {
 	length, e := p.readVarint32()
 	if e != nil {
 		return nil, NewTProtocolException(e)
 	}
+	err = checkSizeForProtocol(length, p.cfg)
+	if err != nil {
+		return
+	}
 	if length == 0 {
 		return []byte{}, nil
 	}
-	if length < 0 {
-		return nil, invalidDataLength
-	}
-	if uint64(length) > p.trans.RemainingBytes() {
-		return nil, invalidDataLength
-	}
 
-	buf := make([]byte, length)
-	_, e = io.ReadFull(p.trans, buf)
+	buf, e := safeReadBytes(length, p.trans)
 	return buf, NewTProtocolException(e)
 }
 
-func (p *TCompactProtocol) Flush() (err error) {
-	return NewTProtocolException(p.trans.Flush())
+func (p *TCompactProtocol) Flush(ctx context.Context) (err error) {
+	return NewTProtocolException(p.trans.Flush(ctx))
 }
 
-func (p *TCompactProtocol) Skip(fieldType TType) (err error) {
-	return SkipDefaultDepth(p, fieldType)
+func (p *TCompactProtocol) Skip(ctx context.Context, fieldType TType) (err error) {
+	return SkipDefaultDepth(ctx, p, fieldType)
 }
 
 func (p *TCompactProtocol) Transport() TTransport {
@@ -806,10 +845,21 @@
 	case COMPACT_STRUCT:
 		return STRUCT, nil
 	}
-	return STOP, TException(fmt.Errorf("don't know what type: %d", t&0x0f))
+	return STOP, NewTProtocolException(fmt.Errorf("don't know what type: %v", t&0x0f))
 }
 
 // Given a TType value, find the appropriate TCompactProtocol.Types constant.
 func (p *TCompactProtocol) getCompactType(t TType) tCompactType {
 	return ttypeToCompactType[t]
 }
+
+func (p *TCompactProtocol) SetTConfiguration(conf *TConfiguration) {
+	PropagateTConfiguration(p.trans, conf)
+	PropagateTConfiguration(p.origTransport, conf)
+	p.cfg = conf
+}
+
+var (
+	_ TConfigurationSetter = (*TCompactProtocolFactory)(nil)
+	_ TConfigurationSetter = (*TCompactProtocol)(nil)
+)
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/configuration.go b/vendor/github.com/uber/jaeger-client-go/thrift/configuration.go
new file mode 100644
index 0000000..454d9f3
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/configuration.go
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"crypto/tls"
+	"fmt"
+	"time"
+)
+
+// Default TConfiguration values.
+const (
+	DEFAULT_MAX_MESSAGE_SIZE = 100 * 1024 * 1024
+	DEFAULT_MAX_FRAME_SIZE   = 16384000
+
+	DEFAULT_TBINARY_STRICT_READ  = false
+	DEFAULT_TBINARY_STRICT_WRITE = true
+
+	DEFAULT_CONNECT_TIMEOUT = 0
+	DEFAULT_SOCKET_TIMEOUT  = 0
+)
+
+// TConfiguration defines some configurations shared between TTransport,
+// TProtocol, TTransportFactory, TProtocolFactory, and other implementations.
+//
+// When constructing TConfiguration, you only need to specify the non-default
+// fields. All zero values have sane default values.
+//
+// Not all configurations defined are applicable to all implementations.
+// Implementations are free to ignore the configurations not applicable to them.
+//
+// All functions attached to this type are nil-safe.
+//
+// See [1] for spec.
+//
+// NOTE: When using TConfiguration, fill in all the configurations you want to
+// set across the stack, not only the ones you want to set in the immediate
+// TTransport/TProtocol.
+//
+// For example, say you want to migrate this old code into using TConfiguration:
+//
+//     sccket := thrift.NewTSocketTimeout("host:port", time.Second)
+//     transFactory := thrift.NewTFramedTransportFactoryMaxLength(
+//         thrift.NewTTransportFactory(),
+//         1024 * 1024 * 256,
+//     )
+//     protoFactory := thrift.NewTBinaryProtocolFactory(true, true)
+//
+// This is the wrong way to do it because in the end the TConfiguration used by
+// socket and transFactory will be overwritten by the one used by protoFactory
+// because of TConfiguration propagation:
+//
+//     // bad example, DO NOT USE
+//     sccket := thrift.NewTSocketConf("host:port", &thrift.TConfiguration{
+//         ConnectTimeout: time.Second,
+//         SocketTimeout:  time.Second,
+//     })
+//     transFactory := thrift.NewTFramedTransportFactoryConf(
+//         thrift.NewTTransportFactory(),
+//         &thrift.TConfiguration{
+//             MaxFrameSize: 1024 * 1024 * 256,
+//         },
+//     )
+//     protoFactory := thrift.NewTBinaryProtocolFactoryConf(&thrift.TConfiguration{
+//         TBinaryStrictRead:  thrift.BoolPtr(true),
+//         TBinaryStrictWrite: thrift.BoolPtr(true),
+//     })
+//
+// This is the correct way to do it:
+//
+//     conf := &thrift.TConfiguration{
+//         ConnectTimeout: time.Second,
+//         SocketTimeout:  time.Second,
+//
+//         MaxFrameSize: 1024 * 1024 * 256,
+//
+//         TBinaryStrictRead:  thrift.BoolPtr(true),
+//         TBinaryStrictWrite: thrift.BoolPtr(true),
+//     }
+//     sccket := thrift.NewTSocketConf("host:port", conf)
+//     transFactory := thrift.NewTFramedTransportFactoryConf(thrift.NewTTransportFactory(), conf)
+//     protoFactory := thrift.NewTBinaryProtocolFactoryConf(conf)
+//
+// [1]: https://github.com/apache/thrift/blob/master/doc/specs/thrift-tconfiguration.md
+type TConfiguration struct {
+	// If <= 0, DEFAULT_MAX_MESSAGE_SIZE will be used instead.
+	MaxMessageSize int32
+
+	// If <= 0, DEFAULT_MAX_FRAME_SIZE will be used instead.
+	//
+	// Also if MaxMessageSize < MaxFrameSize,
+	// MaxMessageSize will be used instead.
+	MaxFrameSize int32
+
+	// Connect and socket timeouts to be used by TSocket and TSSLSocket.
+	//
+	// 0 means no timeout.
+	//
+	// If <0, DEFAULT_CONNECT_TIMEOUT and DEFAULT_SOCKET_TIMEOUT will be
+	// used.
+	ConnectTimeout time.Duration
+	SocketTimeout  time.Duration
+
+	// TLS config to be used by TSSLSocket.
+	TLSConfig *tls.Config
+
+	// Strict read/write configurations for TBinaryProtocol.
+	//
+	// BoolPtr helper function is available to use literal values.
+	TBinaryStrictRead  *bool
+	TBinaryStrictWrite *bool
+
+	// The wrapped protocol id to be used in THeader transport/protocol.
+	//
+	// THeaderProtocolIDPtr and THeaderProtocolIDPtrMust helper functions
+	// are provided to help filling this value.
+	THeaderProtocolID *THeaderProtocolID
+
+	// Used internally by deprecated constructors, to avoid overriding
+	// underlying TTransport/TProtocol's cfg by accidental propagations.
+	//
+	// For external users this is always false.
+	noPropagation bool
+}
+
+// GetMaxMessageSize returns the max message size an implementation should
+// follow.
+//
+// It's nil-safe. DEFAULT_MAX_MESSAGE_SIZE will be returned if tc is nil.
+func (tc *TConfiguration) GetMaxMessageSize() int32 {
+	if tc == nil || tc.MaxMessageSize <= 0 {
+		return DEFAULT_MAX_MESSAGE_SIZE
+	}
+	return tc.MaxMessageSize
+}
+
+// GetMaxFrameSize returns the max frame size an implementation should follow.
+//
+// It's nil-safe. DEFAULT_MAX_FRAME_SIZE will be returned if tc is nil.
+//
+// If the configured max message size is smaller than the configured max frame
+// size, the smaller one will be returned instead.
+func (tc *TConfiguration) GetMaxFrameSize() int32 {
+	if tc == nil {
+		return DEFAULT_MAX_FRAME_SIZE
+	}
+	maxFrameSize := tc.MaxFrameSize
+	if maxFrameSize <= 0 {
+		maxFrameSize = DEFAULT_MAX_FRAME_SIZE
+	}
+	if maxMessageSize := tc.GetMaxMessageSize(); maxMessageSize < maxFrameSize {
+		return maxMessageSize
+	}
+	return maxFrameSize
+}
+
+// GetConnectTimeout returns the connect timeout should be used by TSocket and
+// TSSLSocket.
+//
+// It's nil-safe. If tc is nil, DEFAULT_CONNECT_TIMEOUT will be returned instead.
+func (tc *TConfiguration) GetConnectTimeout() time.Duration {
+	if tc == nil || tc.ConnectTimeout < 0 {
+		return DEFAULT_CONNECT_TIMEOUT
+	}
+	return tc.ConnectTimeout
+}
+
+// GetSocketTimeout returns the socket timeout should be used by TSocket and
+// TSSLSocket.
+//
+// It's nil-safe. If tc is nil, DEFAULT_SOCKET_TIMEOUT will be returned instead.
+func (tc *TConfiguration) GetSocketTimeout() time.Duration {
+	if tc == nil || tc.SocketTimeout < 0 {
+		return DEFAULT_SOCKET_TIMEOUT
+	}
+	return tc.SocketTimeout
+}
+
+// GetTLSConfig returns the tls config should be used by TSSLSocket.
+//
+// It's nil-safe. If tc is nil, nil will be returned instead.
+func (tc *TConfiguration) GetTLSConfig() *tls.Config {
+	if tc == nil {
+		return nil
+	}
+	return tc.TLSConfig
+}
+
+// GetTBinaryStrictRead returns the strict read configuration TBinaryProtocol
+// should follow.
+//
+// It's nil-safe. DEFAULT_TBINARY_STRICT_READ will be returned if either tc or
+// tc.TBinaryStrictRead is nil.
+func (tc *TConfiguration) GetTBinaryStrictRead() bool {
+	if tc == nil || tc.TBinaryStrictRead == nil {
+		return DEFAULT_TBINARY_STRICT_READ
+	}
+	return *tc.TBinaryStrictRead
+}
+
+// GetTBinaryStrictWrite returns the strict read configuration TBinaryProtocol
+// should follow.
+//
+// It's nil-safe. DEFAULT_TBINARY_STRICT_WRITE will be returned if either tc or
+// tc.TBinaryStrictWrite is nil.
+func (tc *TConfiguration) GetTBinaryStrictWrite() bool {
+	if tc == nil || tc.TBinaryStrictWrite == nil {
+		return DEFAULT_TBINARY_STRICT_WRITE
+	}
+	return *tc.TBinaryStrictWrite
+}
+
+// GetTHeaderProtocolID returns the THeaderProtocolID should be used by
+// THeaderProtocol clients (for servers, they always use the same one as the
+// client instead).
+//
+// It's nil-safe. If either tc or tc.THeaderProtocolID is nil,
+// THeaderProtocolDefault will be returned instead.
+// THeaderProtocolDefault will also be returned if configured value is invalid.
+func (tc *TConfiguration) GetTHeaderProtocolID() THeaderProtocolID {
+	if tc == nil || tc.THeaderProtocolID == nil {
+		return THeaderProtocolDefault
+	}
+	protoID := *tc.THeaderProtocolID
+	if err := protoID.Validate(); err != nil {
+		return THeaderProtocolDefault
+	}
+	return protoID
+}
+
+// THeaderProtocolIDPtr validates and returns the pointer to id.
+//
+// If id is not a valid THeaderProtocolID, a pointer to THeaderProtocolDefault
+// and the validation error will be returned.
+func THeaderProtocolIDPtr(id THeaderProtocolID) (*THeaderProtocolID, error) {
+	err := id.Validate()
+	if err != nil {
+		id = THeaderProtocolDefault
+	}
+	return &id, err
+}
+
+// THeaderProtocolIDPtrMust validates and returns the pointer to id.
+//
+// It's similar to THeaderProtocolIDPtr, but it panics on validation errors
+// instead of returning them.
+func THeaderProtocolIDPtrMust(id THeaderProtocolID) *THeaderProtocolID {
+	ptr, err := THeaderProtocolIDPtr(id)
+	if err != nil {
+		panic(err)
+	}
+	return ptr
+}
+
+// TConfigurationSetter is an optional interface TProtocol, TTransport,
+// TProtocolFactory, TTransportFactory, and other implementations can implement.
+//
+// It's intended to be called during intializations.
+// The behavior of calling SetTConfiguration on a TTransport/TProtocol in the
+// middle of a message is undefined:
+// It may or may not change the behavior of the current processing message,
+// and it may even cause the current message to fail.
+//
+// Note for implementations: SetTConfiguration might be called multiple times
+// with the same value in quick successions due to the implementation of the
+// propagation. Implementations should make SetTConfiguration as simple as
+// possible (usually just overwrite the stored configuration and propagate it to
+// the wrapped TTransports/TProtocols).
+type TConfigurationSetter interface {
+	SetTConfiguration(*TConfiguration)
+}
+
+// PropagateTConfiguration propagates cfg to impl if impl implements
+// TConfigurationSetter and cfg is non-nil, otherwise it does nothing.
+//
+// NOTE: nil cfg is not propagated. If you want to propagate a TConfiguration
+// with everything being default value, use &TConfiguration{} explicitly instead.
+func PropagateTConfiguration(impl interface{}, cfg *TConfiguration) {
+	if cfg == nil || cfg.noPropagation {
+		return
+	}
+
+	if setter, ok := impl.(TConfigurationSetter); ok {
+		setter.SetTConfiguration(cfg)
+	}
+}
+
+func checkSizeForProtocol(size int32, cfg *TConfiguration) error {
+	if size < 0 {
+		return NewTProtocolExceptionWithType(
+			NEGATIVE_SIZE,
+			fmt.Errorf("negative size: %d", size),
+		)
+	}
+	if size > cfg.GetMaxMessageSize() {
+		return NewTProtocolExceptionWithType(
+			SIZE_LIMIT,
+			fmt.Errorf("size exceeded max allowed: %d", size),
+		)
+	}
+	return nil
+}
+
+type tTransportFactoryConf struct {
+	delegate TTransportFactory
+	cfg      *TConfiguration
+}
+
+func (f *tTransportFactoryConf) GetTransport(orig TTransport) (TTransport, error) {
+	trans, err := f.delegate.GetTransport(orig)
+	if err == nil {
+		PropagateTConfiguration(orig, f.cfg)
+		PropagateTConfiguration(trans, f.cfg)
+	}
+	return trans, err
+}
+
+func (f *tTransportFactoryConf) SetTConfiguration(cfg *TConfiguration) {
+	PropagateTConfiguration(f.delegate, f.cfg)
+	f.cfg = cfg
+}
+
+// TTransportFactoryConf wraps a TTransportFactory to propagate
+// TConfiguration on the factory's GetTransport calls.
+func TTransportFactoryConf(delegate TTransportFactory, conf *TConfiguration) TTransportFactory {
+	return &tTransportFactoryConf{
+		delegate: delegate,
+		cfg:      conf,
+	}
+}
+
+type tProtocolFactoryConf struct {
+	delegate TProtocolFactory
+	cfg      *TConfiguration
+}
+
+func (f *tProtocolFactoryConf) GetProtocol(trans TTransport) TProtocol {
+	proto := f.delegate.GetProtocol(trans)
+	PropagateTConfiguration(trans, f.cfg)
+	PropagateTConfiguration(proto, f.cfg)
+	return proto
+}
+
+func (f *tProtocolFactoryConf) SetTConfiguration(cfg *TConfiguration) {
+	PropagateTConfiguration(f.delegate, f.cfg)
+	f.cfg = cfg
+}
+
+// TProtocolFactoryConf wraps a TProtocolFactory to propagate
+// TConfiguration on the factory's GetProtocol calls.
+func TProtocolFactoryConf(delegate TProtocolFactory, conf *TConfiguration) TProtocolFactory {
+	return &tProtocolFactoryConf{
+		delegate: delegate,
+		cfg:      conf,
+	}
+}
+
+var (
+	_ TConfigurationSetter = (*tTransportFactoryConf)(nil)
+	_ TConfigurationSetter = (*tProtocolFactoryConf)(nil)
+)
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/context.go b/vendor/github.com/uber/jaeger-client-go/thrift/context.go
new file mode 100644
index 0000000..d15c1bc
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/context.go
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import "context"
+
+var defaultCtx = context.Background()
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/exception.go b/vendor/github.com/uber/jaeger-client-go/thrift/exception.go
index ea8d6f6..53bf862 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/exception.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/exception.go
@@ -26,19 +26,91 @@
 // Generic Thrift exception
 type TException interface {
 	error
+
+	TExceptionType() TExceptionType
 }
 
 // Prepends additional information to an error without losing the Thrift exception interface
 func PrependError(prepend string, err error) error {
-	if t, ok := err.(TTransportException); ok {
-		return NewTTransportException(t.TypeId(), prepend+t.Error())
-	}
-	if t, ok := err.(TProtocolException); ok {
-		return NewTProtocolExceptionWithType(t.TypeId(), errors.New(prepend+err.Error()))
-	}
-	if t, ok := err.(TApplicationException); ok {
-		return NewTApplicationException(t.TypeId(), prepend+t.Error())
+	msg := prepend + err.Error()
+
+	var te TException
+	if errors.As(err, &te) {
+		switch te.TExceptionType() {
+		case TExceptionTypeTransport:
+			if t, ok := err.(TTransportException); ok {
+				return prependTTransportException(prepend, t)
+			}
+		case TExceptionTypeProtocol:
+			if t, ok := err.(TProtocolException); ok {
+				return prependTProtocolException(prepend, t)
+			}
+		case TExceptionTypeApplication:
+			var t TApplicationException
+			if errors.As(err, &t) {
+				return NewTApplicationException(t.TypeId(), msg)
+			}
+		}
+
+		return wrappedTException{
+			err:            err,
+			msg:            msg,
+			tExceptionType: te.TExceptionType(),
+		}
 	}
 
-	return errors.New(prepend + err.Error())
+	return errors.New(msg)
 }
+
+// TExceptionType is an enum type to categorize different "subclasses" of TExceptions.
+type TExceptionType byte
+
+// TExceptionType values
+const (
+	TExceptionTypeUnknown     TExceptionType = iota
+	TExceptionTypeCompiled                   // TExceptions defined in thrift files and generated by thrift compiler
+	TExceptionTypeApplication                // TApplicationExceptions
+	TExceptionTypeProtocol                   // TProtocolExceptions
+	TExceptionTypeTransport                  // TTransportExceptions
+)
+
+// WrapTException wraps an error into TException.
+//
+// If err is nil or already TException, it's returned as-is.
+// Otherwise it will be wraped into TException with TExceptionType() returning
+// TExceptionTypeUnknown, and Unwrap() returning the original error.
+func WrapTException(err error) TException {
+	if err == nil {
+		return nil
+	}
+
+	if te, ok := err.(TException); ok {
+		return te
+	}
+
+	return wrappedTException{
+		err:            err,
+		msg:            err.Error(),
+		tExceptionType: TExceptionTypeUnknown,
+	}
+}
+
+type wrappedTException struct {
+	err            error
+	msg            string
+	tExceptionType TExceptionType
+}
+
+func (w wrappedTException) Error() string {
+	return w.msg
+}
+
+func (w wrappedTException) TExceptionType() TExceptionType {
+	return w.tExceptionType
+}
+
+func (w wrappedTException) Unwrap() error {
+	return w.err
+}
+
+var _ TException = wrappedTException{}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/header_context.go b/vendor/github.com/uber/jaeger-client-go/thrift/header_context.go
new file mode 100644
index 0000000..ac9bd48
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/header_context.go
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"context"
+)
+
+// See https://godoc.org/context#WithValue on why do we need the unexported typedefs.
+type (
+	headerKey     string
+	headerKeyList int
+)
+
+// Values for headerKeyList.
+const (
+	headerKeyListRead headerKeyList = iota
+	headerKeyListWrite
+)
+
+// SetHeader sets a header in the context.
+func SetHeader(ctx context.Context, key, value string) context.Context {
+	return context.WithValue(
+		ctx,
+		headerKey(key),
+		value,
+	)
+}
+
+// UnsetHeader unsets a previously set header in the context.
+func UnsetHeader(ctx context.Context, key string) context.Context {
+	return context.WithValue(
+		ctx,
+		headerKey(key),
+		nil,
+	)
+}
+
+// GetHeader returns a value of the given header from the context.
+func GetHeader(ctx context.Context, key string) (value string, ok bool) {
+	if v := ctx.Value(headerKey(key)); v != nil {
+		value, ok = v.(string)
+	}
+	return
+}
+
+// SetReadHeaderList sets the key list of read THeaders in the context.
+func SetReadHeaderList(ctx context.Context, keys []string) context.Context {
+	return context.WithValue(
+		ctx,
+		headerKeyListRead,
+		keys,
+	)
+}
+
+// GetReadHeaderList returns the key list of read THeaders from the context.
+func GetReadHeaderList(ctx context.Context) []string {
+	if v := ctx.Value(headerKeyListRead); v != nil {
+		if value, ok := v.([]string); ok {
+			return value
+		}
+	}
+	return nil
+}
+
+// SetWriteHeaderList sets the key list of THeaders to write in the context.
+func SetWriteHeaderList(ctx context.Context, keys []string) context.Context {
+	return context.WithValue(
+		ctx,
+		headerKeyListWrite,
+		keys,
+	)
+}
+
+// GetWriteHeaderList returns the key list of THeaders to write from the context.
+func GetWriteHeaderList(ctx context.Context) []string {
+	if v := ctx.Value(headerKeyListWrite); v != nil {
+		if value, ok := v.([]string); ok {
+			return value
+		}
+	}
+	return nil
+}
+
+// AddReadTHeaderToContext adds the whole THeader headers into context.
+func AddReadTHeaderToContext(ctx context.Context, headers THeaderMap) context.Context {
+	keys := make([]string, 0, len(headers))
+	for key, value := range headers {
+		ctx = SetHeader(ctx, key, value)
+		keys = append(keys, key)
+	}
+	return SetReadHeaderList(ctx, keys)
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/header_protocol.go b/vendor/github.com/uber/jaeger-client-go/thrift/header_protocol.go
new file mode 100644
index 0000000..878041f
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/header_protocol.go
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"context"
+	"errors"
+)
+
+// THeaderProtocol is a thrift protocol that implements THeader:
+// https://github.com/apache/thrift/blob/master/doc/specs/HeaderFormat.md
+//
+// It supports either binary or compact protocol as the wrapped protocol.
+//
+// Most of the THeader handlings are happening inside THeaderTransport.
+type THeaderProtocol struct {
+	transport *THeaderTransport
+
+	// Will be initialized on first read/write.
+	protocol TProtocol
+
+	cfg *TConfiguration
+}
+
+// Deprecated: Use NewTHeaderProtocolConf instead.
+func NewTHeaderProtocol(trans TTransport) *THeaderProtocol {
+	return newTHeaderProtocolConf(trans, &TConfiguration{
+		noPropagation: true,
+	})
+}
+
+// NewTHeaderProtocolConf creates a new THeaderProtocol from the underlying
+// transport with given TConfiguration.
+//
+// The passed in transport will be wrapped with THeaderTransport.
+//
+// Note that THeaderTransport handles frame and zlib by itself,
+// so the underlying transport should be a raw socket transports (TSocket or TSSLSocket),
+// instead of rich transports like TZlibTransport or TFramedTransport.
+func NewTHeaderProtocolConf(trans TTransport, conf *TConfiguration) *THeaderProtocol {
+	return newTHeaderProtocolConf(trans, conf)
+}
+
+func newTHeaderProtocolConf(trans TTransport, cfg *TConfiguration) *THeaderProtocol {
+	t := NewTHeaderTransportConf(trans, cfg)
+	p, _ := t.cfg.GetTHeaderProtocolID().GetProtocol(t)
+	PropagateTConfiguration(p, cfg)
+	return &THeaderProtocol{
+		transport: t,
+		protocol:  p,
+		cfg:       cfg,
+	}
+}
+
+type tHeaderProtocolFactory struct {
+	cfg *TConfiguration
+}
+
+func (f tHeaderProtocolFactory) GetProtocol(trans TTransport) TProtocol {
+	return newTHeaderProtocolConf(trans, f.cfg)
+}
+
+func (f *tHeaderProtocolFactory) SetTConfiguration(cfg *TConfiguration) {
+	f.cfg = cfg
+}
+
+// Deprecated: Use NewTHeaderProtocolFactoryConf instead.
+func NewTHeaderProtocolFactory() TProtocolFactory {
+	return NewTHeaderProtocolFactoryConf(&TConfiguration{
+		noPropagation: true,
+	})
+}
+
+// NewTHeaderProtocolFactoryConf creates a factory for THeader with given
+// TConfiguration.
+func NewTHeaderProtocolFactoryConf(conf *TConfiguration) TProtocolFactory {
+	return tHeaderProtocolFactory{
+		cfg: conf,
+	}
+}
+
+// Transport returns the underlying transport.
+//
+// It's guaranteed to be of type *THeaderTransport.
+func (p *THeaderProtocol) Transport() TTransport {
+	return p.transport
+}
+
+// GetReadHeaders returns the THeaderMap read from transport.
+func (p *THeaderProtocol) GetReadHeaders() THeaderMap {
+	return p.transport.GetReadHeaders()
+}
+
+// SetWriteHeader sets a header for write.
+func (p *THeaderProtocol) SetWriteHeader(key, value string) {
+	p.transport.SetWriteHeader(key, value)
+}
+
+// ClearWriteHeaders clears all write headers previously set.
+func (p *THeaderProtocol) ClearWriteHeaders() {
+	p.transport.ClearWriteHeaders()
+}
+
+// AddTransform add a transform for writing.
+func (p *THeaderProtocol) AddTransform(transform THeaderTransformID) error {
+	return p.transport.AddTransform(transform)
+}
+
+func (p *THeaderProtocol) Flush(ctx context.Context) error {
+	return p.transport.Flush(ctx)
+}
+
+func (p *THeaderProtocol) WriteMessageBegin(ctx context.Context, name string, typeID TMessageType, seqID int32) error {
+	newProto, err := p.transport.Protocol().GetProtocol(p.transport)
+	if err != nil {
+		return err
+	}
+	PropagateTConfiguration(newProto, p.cfg)
+	p.protocol = newProto
+	p.transport.SequenceID = seqID
+	return p.protocol.WriteMessageBegin(ctx, name, typeID, seqID)
+}
+
+func (p *THeaderProtocol) WriteMessageEnd(ctx context.Context) error {
+	if err := p.protocol.WriteMessageEnd(ctx); err != nil {
+		return err
+	}
+	return p.transport.Flush(ctx)
+}
+
+func (p *THeaderProtocol) WriteStructBegin(ctx context.Context, name string) error {
+	return p.protocol.WriteStructBegin(ctx, name)
+}
+
+func (p *THeaderProtocol) WriteStructEnd(ctx context.Context) error {
+	return p.protocol.WriteStructEnd(ctx)
+}
+
+func (p *THeaderProtocol) WriteFieldBegin(ctx context.Context, name string, typeID TType, id int16) error {
+	return p.protocol.WriteFieldBegin(ctx, name, typeID, id)
+}
+
+func (p *THeaderProtocol) WriteFieldEnd(ctx context.Context) error {
+	return p.protocol.WriteFieldEnd(ctx)
+}
+
+func (p *THeaderProtocol) WriteFieldStop(ctx context.Context) error {
+	return p.protocol.WriteFieldStop(ctx)
+}
+
+func (p *THeaderProtocol) WriteMapBegin(ctx context.Context, keyType TType, valueType TType, size int) error {
+	return p.protocol.WriteMapBegin(ctx, keyType, valueType, size)
+}
+
+func (p *THeaderProtocol) WriteMapEnd(ctx context.Context) error {
+	return p.protocol.WriteMapEnd(ctx)
+}
+
+func (p *THeaderProtocol) WriteListBegin(ctx context.Context, elemType TType, size int) error {
+	return p.protocol.WriteListBegin(ctx, elemType, size)
+}
+
+func (p *THeaderProtocol) WriteListEnd(ctx context.Context) error {
+	return p.protocol.WriteListEnd(ctx)
+}
+
+func (p *THeaderProtocol) WriteSetBegin(ctx context.Context, elemType TType, size int) error {
+	return p.protocol.WriteSetBegin(ctx, elemType, size)
+}
+
+func (p *THeaderProtocol) WriteSetEnd(ctx context.Context) error {
+	return p.protocol.WriteSetEnd(ctx)
+}
+
+func (p *THeaderProtocol) WriteBool(ctx context.Context, value bool) error {
+	return p.protocol.WriteBool(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteByte(ctx context.Context, value int8) error {
+	return p.protocol.WriteByte(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteI16(ctx context.Context, value int16) error {
+	return p.protocol.WriteI16(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteI32(ctx context.Context, value int32) error {
+	return p.protocol.WriteI32(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteI64(ctx context.Context, value int64) error {
+	return p.protocol.WriteI64(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteDouble(ctx context.Context, value float64) error {
+	return p.protocol.WriteDouble(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteString(ctx context.Context, value string) error {
+	return p.protocol.WriteString(ctx, value)
+}
+
+func (p *THeaderProtocol) WriteBinary(ctx context.Context, value []byte) error {
+	return p.protocol.WriteBinary(ctx, value)
+}
+
+// ReadFrame calls underlying THeaderTransport's ReadFrame function.
+func (p *THeaderProtocol) ReadFrame(ctx context.Context) error {
+	return p.transport.ReadFrame(ctx)
+}
+
+func (p *THeaderProtocol) ReadMessageBegin(ctx context.Context) (name string, typeID TMessageType, seqID int32, err error) {
+	if err = p.transport.ReadFrame(ctx); err != nil {
+		return
+	}
+
+	var newProto TProtocol
+	newProto, err = p.transport.Protocol().GetProtocol(p.transport)
+	if err != nil {
+		var tAppExc TApplicationException
+		if !errors.As(err, &tAppExc) {
+			return
+		}
+		if e := p.protocol.WriteMessageBegin(ctx, "", EXCEPTION, seqID); e != nil {
+			return
+		}
+		if e := tAppExc.Write(ctx, p.protocol); e != nil {
+			return
+		}
+		if e := p.protocol.WriteMessageEnd(ctx); e != nil {
+			return
+		}
+		if e := p.transport.Flush(ctx); e != nil {
+			return
+		}
+		return
+	}
+	PropagateTConfiguration(newProto, p.cfg)
+	p.protocol = newProto
+
+	return p.protocol.ReadMessageBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadMessageEnd(ctx context.Context) error {
+	return p.protocol.ReadMessageEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadStructBegin(ctx context.Context) (name string, err error) {
+	return p.protocol.ReadStructBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadStructEnd(ctx context.Context) error {
+	return p.protocol.ReadStructEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadFieldBegin(ctx context.Context) (name string, typeID TType, id int16, err error) {
+	return p.protocol.ReadFieldBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadFieldEnd(ctx context.Context) error {
+	return p.protocol.ReadFieldEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadMapBegin(ctx context.Context) (keyType TType, valueType TType, size int, err error) {
+	return p.protocol.ReadMapBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadMapEnd(ctx context.Context) error {
+	return p.protocol.ReadMapEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadListBegin(ctx context.Context) (elemType TType, size int, err error) {
+	return p.protocol.ReadListBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadListEnd(ctx context.Context) error {
+	return p.protocol.ReadListEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadSetBegin(ctx context.Context) (elemType TType, size int, err error) {
+	return p.protocol.ReadSetBegin(ctx)
+}
+
+func (p *THeaderProtocol) ReadSetEnd(ctx context.Context) error {
+	return p.protocol.ReadSetEnd(ctx)
+}
+
+func (p *THeaderProtocol) ReadBool(ctx context.Context) (value bool, err error) {
+	return p.protocol.ReadBool(ctx)
+}
+
+func (p *THeaderProtocol) ReadByte(ctx context.Context) (value int8, err error) {
+	return p.protocol.ReadByte(ctx)
+}
+
+func (p *THeaderProtocol) ReadI16(ctx context.Context) (value int16, err error) {
+	return p.protocol.ReadI16(ctx)
+}
+
+func (p *THeaderProtocol) ReadI32(ctx context.Context) (value int32, err error) {
+	return p.protocol.ReadI32(ctx)
+}
+
+func (p *THeaderProtocol) ReadI64(ctx context.Context) (value int64, err error) {
+	return p.protocol.ReadI64(ctx)
+}
+
+func (p *THeaderProtocol) ReadDouble(ctx context.Context) (value float64, err error) {
+	return p.protocol.ReadDouble(ctx)
+}
+
+func (p *THeaderProtocol) ReadString(ctx context.Context) (value string, err error) {
+	return p.protocol.ReadString(ctx)
+}
+
+func (p *THeaderProtocol) ReadBinary(ctx context.Context) (value []byte, err error) {
+	return p.protocol.ReadBinary(ctx)
+}
+
+func (p *THeaderProtocol) Skip(ctx context.Context, fieldType TType) error {
+	return p.protocol.Skip(ctx, fieldType)
+}
+
+// SetTConfiguration implements TConfigurationSetter.
+func (p *THeaderProtocol) SetTConfiguration(cfg *TConfiguration) {
+	PropagateTConfiguration(p.transport, cfg)
+	PropagateTConfiguration(p.protocol, cfg)
+	p.cfg = cfg
+}
+
+var (
+	_ TConfigurationSetter = (*tHeaderProtocolFactory)(nil)
+	_ TConfigurationSetter = (*THeaderProtocol)(nil)
+)
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/header_transport.go b/vendor/github.com/uber/jaeger-client-go/thrift/header_transport.go
new file mode 100644
index 0000000..f5736df
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/header_transport.go
@@ -0,0 +1,810 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"bufio"
+	"bytes"
+	"compress/zlib"
+	"context"
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"io"
+	"io/ioutil"
+)
+
+// Size in bytes for 32-bit ints.
+const size32 = 4
+
+type headerMeta struct {
+	MagicFlags   uint32
+	SequenceID   int32
+	HeaderLength uint16
+}
+
+const headerMetaSize = 10
+
+type clientType int
+
+const (
+	clientUnknown clientType = iota
+	clientHeaders
+	clientFramedBinary
+	clientUnframedBinary
+	clientFramedCompact
+	clientUnframedCompact
+)
+
+// Constants defined in THeader format:
+// https://github.com/apache/thrift/blob/master/doc/specs/HeaderFormat.md
+const (
+	THeaderHeaderMagic  uint32 = 0x0fff0000
+	THeaderHeaderMask   uint32 = 0xffff0000
+	THeaderFlagsMask    uint32 = 0x0000ffff
+	THeaderMaxFrameSize uint32 = 0x3fffffff
+)
+
+// THeaderMap is the type of the header map in THeader transport.
+type THeaderMap map[string]string
+
+// THeaderProtocolID is the wrapped protocol id used in THeader.
+type THeaderProtocolID int32
+
+// Supported THeaderProtocolID values.
+const (
+	THeaderProtocolBinary  THeaderProtocolID = 0x00
+	THeaderProtocolCompact THeaderProtocolID = 0x02
+	THeaderProtocolDefault                   = THeaderProtocolBinary
+)
+
+// Declared globally to avoid repetitive allocations, not really used.
+var globalMemoryBuffer = NewTMemoryBuffer()
+
+// Validate checks whether the THeaderProtocolID is a valid/supported one.
+func (id THeaderProtocolID) Validate() error {
+	_, err := id.GetProtocol(globalMemoryBuffer)
+	return err
+}
+
+// GetProtocol gets the corresponding TProtocol from the wrapped protocol id.
+func (id THeaderProtocolID) GetProtocol(trans TTransport) (TProtocol, error) {
+	switch id {
+	default:
+		return nil, NewTApplicationException(
+			INVALID_PROTOCOL,
+			fmt.Sprintf("THeader protocol id %d not supported", id),
+		)
+	case THeaderProtocolBinary:
+		return NewTBinaryProtocolTransport(trans), nil
+	case THeaderProtocolCompact:
+		return NewTCompactProtocol(trans), nil
+	}
+}
+
+// THeaderTransformID defines the numeric id of the transform used.
+type THeaderTransformID int32
+
+// THeaderTransformID values.
+//
+// Values not defined here are not currently supported, namely HMAC and Snappy.
+const (
+	TransformNone THeaderTransformID = iota // 0, no special handling
+	TransformZlib                           // 1, zlib
+)
+
+var supportedTransformIDs = map[THeaderTransformID]bool{
+	TransformNone: true,
+	TransformZlib: true,
+}
+
+// TransformReader is an io.ReadCloser that handles transforms reading.
+type TransformReader struct {
+	io.Reader
+
+	closers []io.Closer
+}
+
+var _ io.ReadCloser = (*TransformReader)(nil)
+
+// NewTransformReaderWithCapacity initializes a TransformReader with expected
+// closers capacity.
+//
+// If you don't know the closers capacity beforehand, just use
+//
+//     &TransformReader{Reader: baseReader}
+//
+// instead would be sufficient.
+func NewTransformReaderWithCapacity(baseReader io.Reader, capacity int) *TransformReader {
+	return &TransformReader{
+		Reader:  baseReader,
+		closers: make([]io.Closer, 0, capacity),
+	}
+}
+
+// Close calls the underlying closers in appropriate order,
+// stops at and returns the first error encountered.
+func (tr *TransformReader) Close() error {
+	// Call closers in reversed order
+	for i := len(tr.closers) - 1; i >= 0; i-- {
+		if err := tr.closers[i].Close(); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// AddTransform adds a transform.
+func (tr *TransformReader) AddTransform(id THeaderTransformID) error {
+	switch id {
+	default:
+		return NewTApplicationException(
+			INVALID_TRANSFORM,
+			fmt.Sprintf("THeaderTransformID %d not supported", id),
+		)
+	case TransformNone:
+		// no-op
+	case TransformZlib:
+		readCloser, err := zlib.NewReader(tr.Reader)
+		if err != nil {
+			return err
+		}
+		tr.Reader = readCloser
+		tr.closers = append(tr.closers, readCloser)
+	}
+	return nil
+}
+
+// TransformWriter is an io.WriteCloser that handles transforms writing.
+type TransformWriter struct {
+	io.Writer
+
+	closers []io.Closer
+}
+
+var _ io.WriteCloser = (*TransformWriter)(nil)
+
+// NewTransformWriter creates a new TransformWriter with base writer and transforms.
+func NewTransformWriter(baseWriter io.Writer, transforms []THeaderTransformID) (io.WriteCloser, error) {
+	writer := &TransformWriter{
+		Writer:  baseWriter,
+		closers: make([]io.Closer, 0, len(transforms)),
+	}
+	for _, id := range transforms {
+		if err := writer.AddTransform(id); err != nil {
+			return nil, err
+		}
+	}
+	return writer, nil
+}
+
+// Close calls the underlying closers in appropriate order,
+// stops at and returns the first error encountered.
+func (tw *TransformWriter) Close() error {
+	// Call closers in reversed order
+	for i := len(tw.closers) - 1; i >= 0; i-- {
+		if err := tw.closers[i].Close(); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// AddTransform adds a transform.
+func (tw *TransformWriter) AddTransform(id THeaderTransformID) error {
+	switch id {
+	default:
+		return NewTApplicationException(
+			INVALID_TRANSFORM,
+			fmt.Sprintf("THeaderTransformID %d not supported", id),
+		)
+	case TransformNone:
+		// no-op
+	case TransformZlib:
+		writeCloser := zlib.NewWriter(tw.Writer)
+		tw.Writer = writeCloser
+		tw.closers = append(tw.closers, writeCloser)
+	}
+	return nil
+}
+
+// THeaderInfoType is the type id of the info headers.
+type THeaderInfoType int32
+
+// Supported THeaderInfoType values.
+const (
+	_            THeaderInfoType = iota // Skip 0
+	InfoKeyValue                        // 1
+	// Rest of the info types are not supported.
+)
+
+// THeaderTransport is a Transport mode that implements THeader.
+//
+// Note that THeaderTransport handles frame and zlib by itself,
+// so the underlying transport should be a raw socket transports (TSocket or TSSLSocket),
+// instead of rich transports like TZlibTransport or TFramedTransport.
+type THeaderTransport struct {
+	SequenceID int32
+	Flags      uint32
+
+	transport TTransport
+
+	// THeaderMap for read and write
+	readHeaders  THeaderMap
+	writeHeaders THeaderMap
+
+	// Reading related variables.
+	reader *bufio.Reader
+	// When frame is detected, we read the frame fully into frameBuffer.
+	frameBuffer bytes.Buffer
+	// When it's non-nil, Read should read from frameReader instead of
+	// reader, and EOF error indicates end of frame instead of end of all
+	// transport.
+	frameReader io.ReadCloser
+
+	// Writing related variables
+	writeBuffer     bytes.Buffer
+	writeTransforms []THeaderTransformID
+
+	clientType clientType
+	protocolID THeaderProtocolID
+	cfg        *TConfiguration
+
+	// buffer is used in the following scenarios to avoid repetitive
+	// allocations, while 4 is big enough for all those scenarios:
+	//
+	// * header padding (max size 4)
+	// * write the frame size (size 4)
+	buffer [4]byte
+}
+
+var _ TTransport = (*THeaderTransport)(nil)
+
+// Deprecated: Use NewTHeaderTransportConf instead.
+func NewTHeaderTransport(trans TTransport) *THeaderTransport {
+	return NewTHeaderTransportConf(trans, &TConfiguration{
+		noPropagation: true,
+	})
+}
+
+// NewTHeaderTransportConf creates THeaderTransport from the
+// underlying transport, with given TConfiguration attached.
+//
+// If trans is already a *THeaderTransport, it will be returned as is,
+// but with TConfiguration overridden by the value passed in.
+//
+// The protocol ID in TConfiguration is only useful for client transports.
+// For servers,
+// the protocol ID will be overridden again to the one set by the client,
+// to ensure that servers always speak the same dialect as the client.
+func NewTHeaderTransportConf(trans TTransport, conf *TConfiguration) *THeaderTransport {
+	if ht, ok := trans.(*THeaderTransport); ok {
+		ht.SetTConfiguration(conf)
+		return ht
+	}
+	PropagateTConfiguration(trans, conf)
+	return &THeaderTransport{
+		transport:    trans,
+		reader:       bufio.NewReader(trans),
+		writeHeaders: make(THeaderMap),
+		protocolID:   conf.GetTHeaderProtocolID(),
+		cfg:          conf,
+	}
+}
+
+// Open calls the underlying transport's Open function.
+func (t *THeaderTransport) Open() error {
+	return t.transport.Open()
+}
+
+// IsOpen calls the underlying transport's IsOpen function.
+func (t *THeaderTransport) IsOpen() bool {
+	return t.transport.IsOpen()
+}
+
+// ReadFrame tries to read the frame header, guess the client type, and handle
+// unframed clients.
+func (t *THeaderTransport) ReadFrame(ctx context.Context) error {
+	if !t.needReadFrame() {
+		// No need to read frame, skipping.
+		return nil
+	}
+
+	// Peek and handle the first 32 bits.
+	// They could either be the length field of a framed message,
+	// or the first bytes of an unframed message.
+	var buf []byte
+	var err error
+	// This is also usually the first read from a connection,
+	// so handle retries around socket timeouts.
+	_, deadlineSet := ctx.Deadline()
+	for {
+		buf, err = t.reader.Peek(size32)
+		if deadlineSet && isTimeoutError(err) && ctx.Err() == nil {
+			// This is I/O timeout and we still have time,
+			// continue trying
+			continue
+		}
+		// For anything else, do not retry
+		break
+	}
+	if err != nil {
+		return err
+	}
+
+	frameSize := binary.BigEndian.Uint32(buf)
+	if frameSize&VERSION_MASK == VERSION_1 {
+		t.clientType = clientUnframedBinary
+		return nil
+	}
+	if buf[0] == COMPACT_PROTOCOL_ID && buf[1]&COMPACT_VERSION_MASK == COMPACT_VERSION {
+		t.clientType = clientUnframedCompact
+		return nil
+	}
+
+	// At this point it should be a framed message,
+	// sanity check on frameSize then discard the peeked part.
+	if frameSize > THeaderMaxFrameSize || frameSize > uint32(t.cfg.GetMaxFrameSize()) {
+		return NewTProtocolExceptionWithType(
+			SIZE_LIMIT,
+			errors.New("frame too large"),
+		)
+	}
+	t.reader.Discard(size32)
+
+	// Read the frame fully into frameBuffer.
+	_, err = io.CopyN(&t.frameBuffer, t.reader, int64(frameSize))
+	if err != nil {
+		return err
+	}
+	t.frameReader = ioutil.NopCloser(&t.frameBuffer)
+
+	// Peek and handle the next 32 bits.
+	buf = t.frameBuffer.Bytes()[:size32]
+	version := binary.BigEndian.Uint32(buf)
+	if version&THeaderHeaderMask == THeaderHeaderMagic {
+		t.clientType = clientHeaders
+		return t.parseHeaders(ctx, frameSize)
+	}
+	if version&VERSION_MASK == VERSION_1 {
+		t.clientType = clientFramedBinary
+		return nil
+	}
+	if buf[0] == COMPACT_PROTOCOL_ID && buf[1]&COMPACT_VERSION_MASK == COMPACT_VERSION {
+		t.clientType = clientFramedCompact
+		return nil
+	}
+	if err := t.endOfFrame(); err != nil {
+		return err
+	}
+	return NewTProtocolExceptionWithType(
+		NOT_IMPLEMENTED,
+		errors.New("unsupported client transport type"),
+	)
+}
+
+// endOfFrame does end of frame handling.
+//
+// It closes frameReader, and also resets frame related states.
+func (t *THeaderTransport) endOfFrame() error {
+	defer func() {
+		t.frameBuffer.Reset()
+		t.frameReader = nil
+	}()
+	return t.frameReader.Close()
+}
+
+func (t *THeaderTransport) parseHeaders(ctx context.Context, frameSize uint32) error {
+	if t.clientType != clientHeaders {
+		return nil
+	}
+
+	var err error
+	var meta headerMeta
+	if err = binary.Read(&t.frameBuffer, binary.BigEndian, &meta); err != nil {
+		return err
+	}
+	frameSize -= headerMetaSize
+	t.Flags = meta.MagicFlags & THeaderFlagsMask
+	t.SequenceID = meta.SequenceID
+	headerLength := int64(meta.HeaderLength) * 4
+	if int64(frameSize) < headerLength {
+		return NewTProtocolExceptionWithType(
+			SIZE_LIMIT,
+			errors.New("header size is larger than the whole frame"),
+		)
+	}
+	headerBuf := NewTMemoryBuffer()
+	_, err = io.CopyN(headerBuf, &t.frameBuffer, headerLength)
+	if err != nil {
+		return err
+	}
+	hp := NewTCompactProtocol(headerBuf)
+	hp.SetTConfiguration(t.cfg)
+
+	// At this point the header is already read into headerBuf,
+	// and t.frameBuffer starts from the actual payload.
+	protoID, err := hp.readVarint32()
+	if err != nil {
+		return err
+	}
+	t.protocolID = THeaderProtocolID(protoID)
+
+	var transformCount int32
+	transformCount, err = hp.readVarint32()
+	if err != nil {
+		return err
+	}
+	if transformCount > 0 {
+		reader := NewTransformReaderWithCapacity(
+			&t.frameBuffer,
+			int(transformCount),
+		)
+		t.frameReader = reader
+		transformIDs := make([]THeaderTransformID, transformCount)
+		for i := 0; i < int(transformCount); i++ {
+			id, err := hp.readVarint32()
+			if err != nil {
+				return err
+			}
+			transformIDs[i] = THeaderTransformID(id)
+		}
+		// The transform IDs on the wire was added based on the order of
+		// writing, so on the reading side we need to reverse the order.
+		for i := transformCount - 1; i >= 0; i-- {
+			id := transformIDs[i]
+			if err := reader.AddTransform(id); err != nil {
+				return err
+			}
+		}
+	}
+
+	// The info part does not use the transforms yet, so it's
+	// important to continue using headerBuf.
+	headers := make(THeaderMap)
+	for {
+		infoType, err := hp.readVarint32()
+		if errors.Is(err, io.EOF) {
+			break
+		}
+		if err != nil {
+			return err
+		}
+		if THeaderInfoType(infoType) == InfoKeyValue {
+			count, err := hp.readVarint32()
+			if err != nil {
+				return err
+			}
+			for i := 0; i < int(count); i++ {
+				key, err := hp.ReadString(ctx)
+				if err != nil {
+					return err
+				}
+				value, err := hp.ReadString(ctx)
+				if err != nil {
+					return err
+				}
+				headers[key] = value
+			}
+		} else {
+			// Skip reading info section on the first
+			// unsupported info type.
+			break
+		}
+	}
+	t.readHeaders = headers
+
+	return nil
+}
+
+func (t *THeaderTransport) needReadFrame() bool {
+	if t.clientType == clientUnknown {
+		// This is a new connection that's never read before.
+		return true
+	}
+	if t.isFramed() && t.frameReader == nil {
+		// We just finished the last frame.
+		return true
+	}
+	return false
+}
+
+func (t *THeaderTransport) Read(p []byte) (read int, err error) {
+	// Here using context.Background instead of a context passed in is safe.
+	// First is that there's no way to pass context into this function.
+	// Then, 99% of the case when calling this Read frame is already read
+	// into frameReader. ReadFrame here is more of preventing bugs that
+	// didn't call ReadFrame before calling Read.
+	err = t.ReadFrame(context.Background())
+	if err != nil {
+		return
+	}
+	if t.frameReader != nil {
+		read, err = t.frameReader.Read(p)
+		if err == nil && t.frameBuffer.Len() <= 0 {
+			// the last Read finished the frame, do endOfFrame
+			// handling here.
+			err = t.endOfFrame()
+		} else if err == io.EOF {
+			err = t.endOfFrame()
+			if err != nil {
+				return
+			}
+			if read == 0 {
+				// Try to read the next frame when we hit EOF
+				// (end of frame) immediately.
+				// When we got here, it means the last read
+				// finished the previous frame, but didn't
+				// do endOfFrame handling yet.
+				// We have to read the next frame here,
+				// as otherwise we would return 0 and nil,
+				// which is a case not handled well by most
+				// protocol implementations.
+				return t.Read(p)
+			}
+		}
+		return
+	}
+	return t.reader.Read(p)
+}
+
+// Write writes data to the write buffer.
+//
+// You need to call Flush to actually write them to the transport.
+func (t *THeaderTransport) Write(p []byte) (int, error) {
+	return t.writeBuffer.Write(p)
+}
+
+// Flush writes the appropriate header and the write buffer to the underlying transport.
+func (t *THeaderTransport) Flush(ctx context.Context) error {
+	if t.writeBuffer.Len() == 0 {
+		return nil
+	}
+
+	defer t.writeBuffer.Reset()
+
+	switch t.clientType {
+	default:
+		fallthrough
+	case clientUnknown:
+		t.clientType = clientHeaders
+		fallthrough
+	case clientHeaders:
+		headers := NewTMemoryBuffer()
+		hp := NewTCompactProtocol(headers)
+		hp.SetTConfiguration(t.cfg)
+		if _, err := hp.writeVarint32(int32(t.protocolID)); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		if _, err := hp.writeVarint32(int32(len(t.writeTransforms))); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		for _, transform := range t.writeTransforms {
+			if _, err := hp.writeVarint32(int32(transform)); err != nil {
+				return NewTTransportExceptionFromError(err)
+			}
+		}
+		if len(t.writeHeaders) > 0 {
+			if _, err := hp.writeVarint32(int32(InfoKeyValue)); err != nil {
+				return NewTTransportExceptionFromError(err)
+			}
+			if _, err := hp.writeVarint32(int32(len(t.writeHeaders))); err != nil {
+				return NewTTransportExceptionFromError(err)
+			}
+			for key, value := range t.writeHeaders {
+				if err := hp.WriteString(ctx, key); err != nil {
+					return NewTTransportExceptionFromError(err)
+				}
+				if err := hp.WriteString(ctx, value); err != nil {
+					return NewTTransportExceptionFromError(err)
+				}
+			}
+		}
+		padding := 4 - headers.Len()%4
+		if padding < 4 {
+			buf := t.buffer[:padding]
+			for i := range buf {
+				buf[i] = 0
+			}
+			if _, err := headers.Write(buf); err != nil {
+				return NewTTransportExceptionFromError(err)
+			}
+		}
+
+		var payload bytes.Buffer
+		meta := headerMeta{
+			MagicFlags:   THeaderHeaderMagic + t.Flags&THeaderFlagsMask,
+			SequenceID:   t.SequenceID,
+			HeaderLength: uint16(headers.Len() / 4),
+		}
+		if err := binary.Write(&payload, binary.BigEndian, meta); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		if _, err := io.Copy(&payload, headers); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+
+		writer, err := NewTransformWriter(&payload, t.writeTransforms)
+		if err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		if _, err := io.Copy(writer, &t.writeBuffer); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		if err := writer.Close(); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+
+		// First write frame length
+		buf := t.buffer[:size32]
+		binary.BigEndian.PutUint32(buf, uint32(payload.Len()))
+		if _, err := t.transport.Write(buf); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		// Then write the payload
+		if _, err := io.Copy(t.transport, &payload); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+
+	case clientFramedBinary, clientFramedCompact:
+		buf := t.buffer[:size32]
+		binary.BigEndian.PutUint32(buf, uint32(t.writeBuffer.Len()))
+		if _, err := t.transport.Write(buf); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+		fallthrough
+	case clientUnframedBinary, clientUnframedCompact:
+		if _, err := io.Copy(t.transport, &t.writeBuffer); err != nil {
+			return NewTTransportExceptionFromError(err)
+		}
+	}
+
+	select {
+	default:
+	case <-ctx.Done():
+		return NewTTransportExceptionFromError(ctx.Err())
+	}
+
+	return t.transport.Flush(ctx)
+}
+
+// Close closes the transport, along with its underlying transport.
+func (t *THeaderTransport) Close() error {
+	if err := t.Flush(context.Background()); err != nil {
+		return err
+	}
+	return t.transport.Close()
+}
+
+// RemainingBytes calls underlying transport's RemainingBytes.
+//
+// Even in framed cases, because of all the possible compression transforms
+// involved, the remaining frame size is likely to be different from the actual
+// remaining readable bytes, so we don't bother to keep tracking the remaining
+// frame size by ourselves and just use the underlying transport's
+// RemainingBytes directly.
+func (t *THeaderTransport) RemainingBytes() uint64 {
+	return t.transport.RemainingBytes()
+}
+
+// GetReadHeaders returns the THeaderMap read from transport.
+func (t *THeaderTransport) GetReadHeaders() THeaderMap {
+	return t.readHeaders
+}
+
+// SetWriteHeader sets a header for write.
+func (t *THeaderTransport) SetWriteHeader(key, value string) {
+	t.writeHeaders[key] = value
+}
+
+// ClearWriteHeaders clears all write headers previously set.
+func (t *THeaderTransport) ClearWriteHeaders() {
+	t.writeHeaders = make(THeaderMap)
+}
+
+// AddTransform add a transform for writing.
+func (t *THeaderTransport) AddTransform(transform THeaderTransformID) error {
+	if !supportedTransformIDs[transform] {
+		return NewTProtocolExceptionWithType(
+			NOT_IMPLEMENTED,
+			fmt.Errorf("THeaderTransformID %d not supported", transform),
+		)
+	}
+	t.writeTransforms = append(t.writeTransforms, transform)
+	return nil
+}
+
+// Protocol returns the wrapped protocol id used in this THeaderTransport.
+func (t *THeaderTransport) Protocol() THeaderProtocolID {
+	switch t.clientType {
+	default:
+		return t.protocolID
+	case clientFramedBinary, clientUnframedBinary:
+		return THeaderProtocolBinary
+	case clientFramedCompact, clientUnframedCompact:
+		return THeaderProtocolCompact
+	}
+}
+
+func (t *THeaderTransport) isFramed() bool {
+	switch t.clientType {
+	default:
+		return false
+	case clientHeaders, clientFramedBinary, clientFramedCompact:
+		return true
+	}
+}
+
+// SetTConfiguration implements TConfigurationSetter.
+func (t *THeaderTransport) SetTConfiguration(cfg *TConfiguration) {
+	PropagateTConfiguration(t.transport, cfg)
+	t.cfg = cfg
+}
+
+// THeaderTransportFactory is a TTransportFactory implementation to create
+// THeaderTransport.
+//
+// It also implements TConfigurationSetter.
+type THeaderTransportFactory struct {
+	// The underlying factory, could be nil.
+	Factory TTransportFactory
+
+	cfg *TConfiguration
+}
+
+// Deprecated: Use NewTHeaderTransportFactoryConf instead.
+func NewTHeaderTransportFactory(factory TTransportFactory) TTransportFactory {
+	return NewTHeaderTransportFactoryConf(factory, &TConfiguration{
+		noPropagation: true,
+	})
+}
+
+// NewTHeaderTransportFactoryConf creates a new *THeaderTransportFactory with
+// the given *TConfiguration.
+func NewTHeaderTransportFactoryConf(factory TTransportFactory, conf *TConfiguration) TTransportFactory {
+	return &THeaderTransportFactory{
+		Factory: factory,
+
+		cfg: conf,
+	}
+}
+
+// GetTransport implements TTransportFactory.
+func (f *THeaderTransportFactory) GetTransport(trans TTransport) (TTransport, error) {
+	if f.Factory != nil {
+		t, err := f.Factory.GetTransport(trans)
+		if err != nil {
+			return nil, err
+		}
+		return NewTHeaderTransportConf(t, f.cfg), nil
+	}
+	return NewTHeaderTransportConf(trans, f.cfg), nil
+}
+
+// SetTConfiguration implements TConfigurationSetter.
+func (f *THeaderTransportFactory) SetTConfiguration(cfg *TConfiguration) {
+	PropagateTConfiguration(f.Factory, f.cfg)
+	f.cfg = cfg
+}
+
+var (
+	_ TConfigurationSetter = (*THeaderTransportFactory)(nil)
+	_ TConfigurationSetter = (*THeaderTransport)(nil)
+)
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/logger.go b/vendor/github.com/uber/jaeger-client-go/thrift/logger.go
new file mode 100644
index 0000000..50d44ec
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/logger.go
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"log"
+	"os"
+)
+
+// Logger is a simple wrapper of a logging function.
+//
+// In reality the users might actually use different logging libraries, and they
+// are not always compatible with each other.
+//
+// Logger is meant to be a simple common ground that it's easy to wrap whatever
+// logging library they use into.
+//
+// See https://issues.apache.org/jira/browse/THRIFT-4985 for the design
+// discussion behind it.
+type Logger func(msg string)
+
+// NopLogger is a Logger implementation that does nothing.
+func NopLogger(msg string) {}
+
+// StdLogger wraps stdlib log package into a Logger.
+//
+// If logger passed in is nil, it will fallback to use stderr and default flags.
+func StdLogger(logger *log.Logger) Logger {
+	if logger == nil {
+		logger = log.New(os.Stderr, "", log.LstdFlags)
+	}
+	return func(msg string) {
+		logger.Print(msg)
+	}
+}
+
+func fallbackLogger(logger Logger) Logger {
+	if logger == nil {
+		return StdLogger(nil)
+	}
+	return logger
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/memory_buffer.go b/vendor/github.com/uber/jaeger-client-go/thrift/memory_buffer.go
index b62fd56..5936d27 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/memory_buffer.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/memory_buffer.go
@@ -21,6 +21,7 @@
 
 import (
 	"bytes"
+	"context"
 )
 
 // Memory buffer-based implementation of the TTransport interface.
@@ -33,14 +34,14 @@
 	size int
 }
 
-func (p *TMemoryBufferTransportFactory) GetTransport(trans TTransport) TTransport {
+func (p *TMemoryBufferTransportFactory) GetTransport(trans TTransport) (TTransport, error) {
 	if trans != nil {
 		t, ok := trans.(*TMemoryBuffer)
 		if ok && t.size > 0 {
-			return NewTMemoryBufferLen(t.size)
+			return NewTMemoryBufferLen(t.size), nil
 		}
 	}
-	return NewTMemoryBufferLen(p.size)
+	return NewTMemoryBufferLen(p.size), nil
 }
 
 func NewTMemoryBufferTransportFactory(size int) *TMemoryBufferTransportFactory {
@@ -70,7 +71,7 @@
 }
 
 // Flushing a memory buffer is a no-op
-func (p *TMemoryBuffer) Flush() error {
+func (p *TMemoryBuffer) Flush(ctx context.Context) error {
 	return nil
 }
 
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/numeric.go b/vendor/github.com/uber/jaeger-client-go/thrift/numeric.go
index aa8daa9..e4512d2 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/numeric.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/numeric.go
@@ -69,14 +69,14 @@
 
 func NewNumericFromI64(iValue int64) Numeric {
 	dValue := float64(iValue)
-	sValue := string(iValue)
+	sValue := strconv.FormatInt(iValue, 10)
 	isNil := false
 	return &numeric{iValue: iValue, dValue: dValue, sValue: sValue, isNil: isNil}
 }
 
 func NewNumericFromI32(iValue int32) Numeric {
 	dValue := float64(iValue)
-	sValue := string(iValue)
+	sValue := strconv.FormatInt(int64(iValue), 10)
 	isNil := false
 	return &numeric{iValue: int64(iValue), dValue: dValue, sValue: sValue, isNil: isNil}
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/processor.go b/vendor/github.com/uber/jaeger-client-go/thrift/processor.go
deleted file mode 100644
index ca0d3fa..0000000
--- a/vendor/github.com/uber/jaeger-client-go/thrift/processor.go
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package thrift
-
-// A processor is a generic object which operates upon an input stream and
-// writes to some output stream.
-type TProcessor interface {
-	Process(in, out TProtocol) (bool, TException)
-}
-
-type TProcessorFunction interface {
-	Process(seqId int32, in, out TProtocol) (bool, TException)
-}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/processor_factory.go b/vendor/github.com/uber/jaeger-client-go/thrift/processor_factory.go
new file mode 100644
index 0000000..245a3cc
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/processor_factory.go
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import "context"
+
+// A processor is a generic object which operates upon an input stream and
+// writes to some output stream.
+type TProcessor interface {
+	Process(ctx context.Context, in, out TProtocol) (bool, TException)
+
+	// ProcessorMap returns a map of thrift method names to TProcessorFunctions.
+	ProcessorMap() map[string]TProcessorFunction
+
+	// AddToProcessorMap adds the given TProcessorFunction to the internal
+	// processor map at the given key.
+	//
+	// If one is already set at the given key, it will be replaced with the new
+	// TProcessorFunction.
+	AddToProcessorMap(string, TProcessorFunction)
+}
+
+type TProcessorFunction interface {
+	Process(ctx context.Context, seqId int32, in, out TProtocol) (bool, TException)
+}
+
+// The default processor factory just returns a singleton
+// instance.
+type TProcessorFactory interface {
+	GetProcessor(trans TTransport) TProcessor
+}
+
+type tProcessorFactory struct {
+	processor TProcessor
+}
+
+func NewTProcessorFactory(p TProcessor) TProcessorFactory {
+	return &tProcessorFactory{processor: p}
+}
+
+func (p *tProcessorFactory) GetProcessor(trans TTransport) TProcessor {
+	return p.processor
+}
+
+/**
+ * The default processor factory just returns a singleton
+ * instance.
+ */
+type TProcessorFunctionFactory interface {
+	GetProcessorFunction(trans TTransport) TProcessorFunction
+}
+
+type tProcessorFunctionFactory struct {
+	processor TProcessorFunction
+}
+
+func NewTProcessorFunctionFactory(p TProcessorFunction) TProcessorFunctionFactory {
+	return &tProcessorFunctionFactory{processor: p}
+}
+
+func (p *tProcessorFunctionFactory) GetProcessorFunction(trans TTransport) TProcessorFunction {
+	return p.processor
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/protocol.go b/vendor/github.com/uber/jaeger-client-go/thrift/protocol.go
index 45fa202..0a69bd4 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/protocol.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/protocol.go
@@ -20,7 +20,9 @@
 package thrift
 
 import (
+	"context"
 	"errors"
+	"fmt"
 )
 
 const (
@@ -29,51 +31,51 @@
 )
 
 type TProtocol interface {
-	WriteMessageBegin(name string, typeId TMessageType, seqid int32) error
-	WriteMessageEnd() error
-	WriteStructBegin(name string) error
-	WriteStructEnd() error
-	WriteFieldBegin(name string, typeId TType, id int16) error
-	WriteFieldEnd() error
-	WriteFieldStop() error
-	WriteMapBegin(keyType TType, valueType TType, size int) error
-	WriteMapEnd() error
-	WriteListBegin(elemType TType, size int) error
-	WriteListEnd() error
-	WriteSetBegin(elemType TType, size int) error
-	WriteSetEnd() error
-	WriteBool(value bool) error
-	WriteByte(value int8) error
-	WriteI16(value int16) error
-	WriteI32(value int32) error
-	WriteI64(value int64) error
-	WriteDouble(value float64) error
-	WriteString(value string) error
-	WriteBinary(value []byte) error
+	WriteMessageBegin(ctx context.Context, name string, typeId TMessageType, seqid int32) error
+	WriteMessageEnd(ctx context.Context) error
+	WriteStructBegin(ctx context.Context, name string) error
+	WriteStructEnd(ctx context.Context) error
+	WriteFieldBegin(ctx context.Context, name string, typeId TType, id int16) error
+	WriteFieldEnd(ctx context.Context) error
+	WriteFieldStop(ctx context.Context) error
+	WriteMapBegin(ctx context.Context, keyType TType, valueType TType, size int) error
+	WriteMapEnd(ctx context.Context) error
+	WriteListBegin(ctx context.Context, elemType TType, size int) error
+	WriteListEnd(ctx context.Context) error
+	WriteSetBegin(ctx context.Context, elemType TType, size int) error
+	WriteSetEnd(ctx context.Context) error
+	WriteBool(ctx context.Context, value bool) error
+	WriteByte(ctx context.Context, value int8) error
+	WriteI16(ctx context.Context, value int16) error
+	WriteI32(ctx context.Context, value int32) error
+	WriteI64(ctx context.Context, value int64) error
+	WriteDouble(ctx context.Context, value float64) error
+	WriteString(ctx context.Context, value string) error
+	WriteBinary(ctx context.Context, value []byte) error
 
-	ReadMessageBegin() (name string, typeId TMessageType, seqid int32, err error)
-	ReadMessageEnd() error
-	ReadStructBegin() (name string, err error)
-	ReadStructEnd() error
-	ReadFieldBegin() (name string, typeId TType, id int16, err error)
-	ReadFieldEnd() error
-	ReadMapBegin() (keyType TType, valueType TType, size int, err error)
-	ReadMapEnd() error
-	ReadListBegin() (elemType TType, size int, err error)
-	ReadListEnd() error
-	ReadSetBegin() (elemType TType, size int, err error)
-	ReadSetEnd() error
-	ReadBool() (value bool, err error)
-	ReadByte() (value int8, err error)
-	ReadI16() (value int16, err error)
-	ReadI32() (value int32, err error)
-	ReadI64() (value int64, err error)
-	ReadDouble() (value float64, err error)
-	ReadString() (value string, err error)
-	ReadBinary() (value []byte, err error)
+	ReadMessageBegin(ctx context.Context) (name string, typeId TMessageType, seqid int32, err error)
+	ReadMessageEnd(ctx context.Context) error
+	ReadStructBegin(ctx context.Context) (name string, err error)
+	ReadStructEnd(ctx context.Context) error
+	ReadFieldBegin(ctx context.Context) (name string, typeId TType, id int16, err error)
+	ReadFieldEnd(ctx context.Context) error
+	ReadMapBegin(ctx context.Context) (keyType TType, valueType TType, size int, err error)
+	ReadMapEnd(ctx context.Context) error
+	ReadListBegin(ctx context.Context) (elemType TType, size int, err error)
+	ReadListEnd(ctx context.Context) error
+	ReadSetBegin(ctx context.Context) (elemType TType, size int, err error)
+	ReadSetEnd(ctx context.Context) error
+	ReadBool(ctx context.Context) (value bool, err error)
+	ReadByte(ctx context.Context) (value int8, err error)
+	ReadI16(ctx context.Context) (value int16, err error)
+	ReadI32(ctx context.Context) (value int32, err error)
+	ReadI64(ctx context.Context) (value int64, err error)
+	ReadDouble(ctx context.Context) (value float64, err error)
+	ReadString(ctx context.Context) (value string, err error)
+	ReadBinary(ctx context.Context) (value []byte, err error)
 
-	Skip(fieldType TType) (err error)
-	Flush() (err error)
+	Skip(ctx context.Context, fieldType TType) (err error)
+	Flush(ctx context.Context) (err error)
 
 	Transport() TTransport
 }
@@ -82,94 +84,94 @@
 const DEFAULT_RECURSION_DEPTH = 64
 
 // Skips over the next data element from the provided input TProtocol object.
-func SkipDefaultDepth(prot TProtocol, typeId TType) (err error) {
-	return Skip(prot, typeId, DEFAULT_RECURSION_DEPTH)
+func SkipDefaultDepth(ctx context.Context, prot TProtocol, typeId TType) (err error) {
+	return Skip(ctx, prot, typeId, DEFAULT_RECURSION_DEPTH)
 }
 
 // Skips over the next data element from the provided input TProtocol object.
-func Skip(self TProtocol, fieldType TType, maxDepth int) (err error) {
-	
-    if maxDepth <= 0 {
-		return NewTProtocolExceptionWithType( DEPTH_LIMIT, errors.New("Depth limit exceeded"))
+func Skip(ctx context.Context, self TProtocol, fieldType TType, maxDepth int) (err error) {
+
+	if maxDepth <= 0 {
+		return NewTProtocolExceptionWithType(DEPTH_LIMIT, errors.New("Depth limit exceeded"))
 	}
 
 	switch fieldType {
-	case STOP:
-		return
 	case BOOL:
-		_, err = self.ReadBool()
+		_, err = self.ReadBool(ctx)
 		return
 	case BYTE:
-		_, err = self.ReadByte()
+		_, err = self.ReadByte(ctx)
 		return
 	case I16:
-		_, err = self.ReadI16()
+		_, err = self.ReadI16(ctx)
 		return
 	case I32:
-		_, err = self.ReadI32()
+		_, err = self.ReadI32(ctx)
 		return
 	case I64:
-		_, err = self.ReadI64()
+		_, err = self.ReadI64(ctx)
 		return
 	case DOUBLE:
-		_, err = self.ReadDouble()
+		_, err = self.ReadDouble(ctx)
 		return
 	case STRING:
-		_, err = self.ReadString()
+		_, err = self.ReadString(ctx)
 		return
 	case STRUCT:
-		if _, err = self.ReadStructBegin(); err != nil {
+		if _, err = self.ReadStructBegin(ctx); err != nil {
 			return err
 		}
 		for {
-			_, typeId, _, _ := self.ReadFieldBegin()
+			_, typeId, _, _ := self.ReadFieldBegin(ctx)
 			if typeId == STOP {
 				break
 			}
-			err := Skip(self, typeId, maxDepth-1)
+			err := Skip(ctx, self, typeId, maxDepth-1)
 			if err != nil {
 				return err
 			}
-			self.ReadFieldEnd()
+			self.ReadFieldEnd(ctx)
 		}
-		return self.ReadStructEnd()
+		return self.ReadStructEnd(ctx)
 	case MAP:
-		keyType, valueType, size, err := self.ReadMapBegin()
+		keyType, valueType, size, err := self.ReadMapBegin(ctx)
 		if err != nil {
 			return err
 		}
 		for i := 0; i < size; i++ {
-			err := Skip(self, keyType, maxDepth-1)
+			err := Skip(ctx, self, keyType, maxDepth-1)
 			if err != nil {
 				return err
 			}
-			self.Skip(valueType)
+			self.Skip(ctx, valueType)
 		}
-		return self.ReadMapEnd()
+		return self.ReadMapEnd(ctx)
 	case SET:
-		elemType, size, err := self.ReadSetBegin()
+		elemType, size, err := self.ReadSetBegin(ctx)
 		if err != nil {
 			return err
 		}
 		for i := 0; i < size; i++ {
-			err := Skip(self, elemType, maxDepth-1)
+			err := Skip(ctx, self, elemType, maxDepth-1)
 			if err != nil {
 				return err
 			}
 		}
-		return self.ReadSetEnd()
+		return self.ReadSetEnd(ctx)
 	case LIST:
-		elemType, size, err := self.ReadListBegin()
+		elemType, size, err := self.ReadListBegin(ctx)
 		if err != nil {
 			return err
 		}
 		for i := 0; i < size; i++ {
-			err := Skip(self, elemType, maxDepth-1)
+			err := Skip(ctx, self, elemType, maxDepth-1)
 			if err != nil {
 				return err
 			}
 		}
-		return self.ReadListEnd()
+		return self.ReadListEnd(ctx)
+	default:
+		return NewTProtocolExceptionWithType(INVALID_DATA, errors.New(fmt.Sprintf("Unknown data type %d", fieldType)))
 	}
 	return nil
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/protocol_exception.go b/vendor/github.com/uber/jaeger-client-go/thrift/protocol_exception.go
index 6e357ee..9dcf4bf 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/protocol_exception.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/protocol_exception.go
@@ -21,6 +21,7 @@
 
 import (
 	"encoding/base64"
+	"errors"
 )
 
 // Thrift Protocol exception
@@ -40,8 +41,15 @@
 )
 
 type tProtocolException struct {
-	typeId  int
-	message string
+	typeId int
+	err    error
+	msg    string
+}
+
+var _ TProtocolException = (*tProtocolException)(nil)
+
+func (tProtocolException) TExceptionType() TExceptionType {
+	return TExceptionTypeProtocol
 }
 
 func (p *tProtocolException) TypeId() int {
@@ -49,30 +57,48 @@
 }
 
 func (p *tProtocolException) String() string {
-	return p.message
+	return p.msg
 }
 
 func (p *tProtocolException) Error() string {
-	return p.message
+	return p.msg
+}
+
+func (p *tProtocolException) Unwrap() error {
+	return p.err
 }
 
 func NewTProtocolException(err error) TProtocolException {
 	if err == nil {
 		return nil
 	}
-	if e,ok := err.(TProtocolException); ok {
+
+	if e, ok := err.(TProtocolException); ok {
 		return e
 	}
-	if _, ok := err.(base64.CorruptInputError); ok {
-		return &tProtocolException{INVALID_DATA, err.Error()}
+
+	if errors.As(err, new(base64.CorruptInputError)) {
+		return NewTProtocolExceptionWithType(INVALID_DATA, err)
 	}
-	return &tProtocolException{UNKNOWN_PROTOCOL_EXCEPTION, err.Error()}
+
+	return NewTProtocolExceptionWithType(UNKNOWN_PROTOCOL_EXCEPTION, err)
 }
 
 func NewTProtocolExceptionWithType(errType int, err error) TProtocolException {
 	if err == nil {
 		return nil
 	}
-	return &tProtocolException{errType, err.Error()}
+	return &tProtocolException{
+		typeId: errType,
+		err:    err,
+		msg:    err.Error(),
+	}
 }
 
+func prependTProtocolException(prepend string, err TProtocolException) TProtocolException {
+	return &tProtocolException{
+		typeId: err.TypeId(),
+		err:    err,
+		msg:    prepend + err.Error(),
+	}
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/response_helper.go b/vendor/github.com/uber/jaeger-client-go/thrift/response_helper.go
new file mode 100644
index 0000000..d884c6a
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/response_helper.go
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"context"
+)
+
+// See https://godoc.org/context#WithValue on why do we need the unexported typedefs.
+type responseHelperKey struct{}
+
+// TResponseHelper defines a object with a set of helper functions that can be
+// retrieved from the context object passed into server handler functions.
+//
+// Use GetResponseHelper to retrieve the injected TResponseHelper implementation
+// from the context object.
+//
+// The zero value of TResponseHelper is valid with all helper functions being
+// no-op.
+type TResponseHelper struct {
+	// THeader related functions
+	*THeaderResponseHelper
+}
+
+// THeaderResponseHelper defines THeader related TResponseHelper functions.
+//
+// The zero value of *THeaderResponseHelper is valid with all helper functions
+// being no-op.
+type THeaderResponseHelper struct {
+	proto *THeaderProtocol
+}
+
+// NewTHeaderResponseHelper creates a new THeaderResponseHelper from the
+// underlying TProtocol.
+func NewTHeaderResponseHelper(proto TProtocol) *THeaderResponseHelper {
+	if hp, ok := proto.(*THeaderProtocol); ok {
+		return &THeaderResponseHelper{
+			proto: hp,
+		}
+	}
+	return nil
+}
+
+// SetHeader sets a response header.
+//
+// It's no-op if the underlying protocol/transport does not support THeader.
+func (h *THeaderResponseHelper) SetHeader(key, value string) {
+	if h != nil && h.proto != nil {
+		h.proto.SetWriteHeader(key, value)
+	}
+}
+
+// ClearHeaders clears all the response headers previously set.
+//
+// It's no-op if the underlying protocol/transport does not support THeader.
+func (h *THeaderResponseHelper) ClearHeaders() {
+	if h != nil && h.proto != nil {
+		h.proto.ClearWriteHeaders()
+	}
+}
+
+// GetResponseHelper retrieves the TResponseHelper implementation injected into
+// the context object.
+//
+// If no helper was found in the context object, a nop helper with ok == false
+// will be returned.
+func GetResponseHelper(ctx context.Context) (helper TResponseHelper, ok bool) {
+	if v := ctx.Value(responseHelperKey{}); v != nil {
+		helper, ok = v.(TResponseHelper)
+	}
+	return
+}
+
+// SetResponseHelper injects TResponseHelper into the context object.
+func SetResponseHelper(ctx context.Context, helper TResponseHelper) context.Context {
+	return context.WithValue(ctx, responseHelperKey{}, helper)
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/rich_transport.go b/vendor/github.com/uber/jaeger-client-go/thrift/rich_transport.go
index 8e296a9..83fdf29 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/rich_transport.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/rich_transport.go
@@ -19,7 +19,10 @@
 
 package thrift
 
-import "io"
+import (
+	"errors"
+	"io"
+)
 
 type RichTransport struct {
 	TTransport
@@ -49,7 +52,7 @@
 func readByte(r io.Reader) (c byte, err error) {
 	v := [1]byte{0}
 	n, err := r.Read(v[0:1])
-	if n > 0 && (err == nil || err == io.EOF) {
+	if n > 0 && (err == nil || errors.Is(err, io.EOF)) {
 		return v[0], nil
 	}
 	if n > 0 && err != nil {
@@ -66,4 +69,3 @@
 	_, err := w.Write(v[0:1])
 	return err
 }
-
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/serializer.go b/vendor/github.com/uber/jaeger-client-go/thrift/serializer.go
index 7712229..c449790 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/serializer.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/serializer.go
@@ -19,57 +19,118 @@
 
 package thrift
 
+import (
+	"context"
+	"sync"
+)
+
 type TSerializer struct {
 	Transport *TMemoryBuffer
 	Protocol  TProtocol
 }
 
 type TStruct interface {
-	Write(p TProtocol) error
-	Read(p TProtocol) error
+	Write(ctx context.Context, p TProtocol) error
+	Read(ctx context.Context, p TProtocol) error
 }
 
 func NewTSerializer() *TSerializer {
 	transport := NewTMemoryBufferLen(1024)
-	protocol := NewTBinaryProtocolFactoryDefault().GetProtocol(transport)
+	protocol := NewTBinaryProtocolTransport(transport)
 
 	return &TSerializer{
-		transport,
-		protocol}
+		Transport: transport,
+		Protocol:  protocol,
+	}
 }
 
-func (t *TSerializer) WriteString(msg TStruct) (s string, err error) {
+func (t *TSerializer) WriteString(ctx context.Context, msg TStruct) (s string, err error) {
 	t.Transport.Reset()
 
-	if err = msg.Write(t.Protocol); err != nil {
+	if err = msg.Write(ctx, t.Protocol); err != nil {
 		return
 	}
 
-	if err = t.Protocol.Flush(); err != nil {
+	if err = t.Protocol.Flush(ctx); err != nil {
 		return
 	}
-	if err = t.Transport.Flush(); err != nil {
+	if err = t.Transport.Flush(ctx); err != nil {
 		return
 	}
 
 	return t.Transport.String(), nil
 }
 
-func (t *TSerializer) Write(msg TStruct) (b []byte, err error) {
+func (t *TSerializer) Write(ctx context.Context, msg TStruct) (b []byte, err error) {
 	t.Transport.Reset()
 
-	if err = msg.Write(t.Protocol); err != nil {
+	if err = msg.Write(ctx, t.Protocol); err != nil {
 		return
 	}
 
-	if err = t.Protocol.Flush(); err != nil {
+	if err = t.Protocol.Flush(ctx); err != nil {
 		return
 	}
 
-	if err = t.Transport.Flush(); err != nil {
+	if err = t.Transport.Flush(ctx); err != nil {
 		return
 	}
 
 	b = append(b, t.Transport.Bytes()...)
 	return
 }
+
+// TSerializerPool is the thread-safe version of TSerializer, it uses resource
+// pool of TSerializer under the hood.
+//
+// It must be initialized with either NewTSerializerPool or
+// NewTSerializerPoolSizeFactory.
+type TSerializerPool struct {
+	pool sync.Pool
+}
+
+// NewTSerializerPool creates a new TSerializerPool.
+//
+// NewTSerializer can be used as the arg here.
+func NewTSerializerPool(f func() *TSerializer) *TSerializerPool {
+	return &TSerializerPool{
+		pool: sync.Pool{
+			New: func() interface{} {
+				return f()
+			},
+		},
+	}
+}
+
+// NewTSerializerPoolSizeFactory creates a new TSerializerPool with the given
+// size and protocol factory.
+//
+// Note that the size is not the limit. The TMemoryBuffer underneath can grow
+// larger than that. It just dictates the initial size.
+func NewTSerializerPoolSizeFactory(size int, factory TProtocolFactory) *TSerializerPool {
+	return &TSerializerPool{
+		pool: sync.Pool{
+			New: func() interface{} {
+				transport := NewTMemoryBufferLen(size)
+				protocol := factory.GetProtocol(transport)
+
+				return &TSerializer{
+					Transport: transport,
+					Protocol:  protocol,
+				}
+			},
+		},
+	}
+}
+
+func (t *TSerializerPool) WriteString(ctx context.Context, msg TStruct) (string, error) {
+	s := t.pool.Get().(*TSerializer)
+	defer t.pool.Put(s)
+	return s.WriteString(ctx, msg)
+}
+
+func (t *TSerializerPool) Write(ctx context.Context, msg TStruct) ([]byte, error) {
+	s := t.pool.Get().(*TSerializer)
+	defer t.pool.Put(s)
+	return s.Write(ctx, msg)
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/server_transport.go b/vendor/github.com/uber/jaeger-client-go/thrift/server_transport.go
new file mode 100644
index 0000000..51c40b6
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/server_transport.go
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+// Server transport. Object which provides client transports.
+type TServerTransport interface {
+	Listen() error
+	Accept() (TTransport, error)
+	Close() error
+
+	// Optional method implementation. This signals to the server transport
+	// that it should break out of any accept() or listen() that it is currently
+	// blocked on. This method, if implemented, MUST be thread safe, as it may
+	// be called from a different thread context than the other TServerTransport
+	// methods.
+	Interrupt() error
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/simple_json_protocol.go b/vendor/github.com/uber/jaeger-client-go/thrift/simple_json_protocol.go
index 412a482..d1a8154 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/simple_json_protocol.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/simple_json_protocol.go
@@ -22,8 +22,10 @@
 import (
 	"bufio"
 	"bytes"
+	"context"
 	"encoding/base64"
 	"encoding/json"
+	"errors"
 	"fmt"
 	"io"
 	"math"
@@ -33,12 +35,13 @@
 type _ParseContext int
 
 const (
-	_CONTEXT_IN_TOPLEVEL          _ParseContext = 1
-	_CONTEXT_IN_LIST_FIRST        _ParseContext = 2
-	_CONTEXT_IN_LIST              _ParseContext = 3
-	_CONTEXT_IN_OBJECT_FIRST      _ParseContext = 4
-	_CONTEXT_IN_OBJECT_NEXT_KEY   _ParseContext = 5
-	_CONTEXT_IN_OBJECT_NEXT_VALUE _ParseContext = 6
+	_CONTEXT_INVALID              _ParseContext = iota
+	_CONTEXT_IN_TOPLEVEL                        // 1
+	_CONTEXT_IN_LIST_FIRST                      // 2
+	_CONTEXT_IN_LIST                            // 3
+	_CONTEXT_IN_OBJECT_FIRST                    // 4
+	_CONTEXT_IN_OBJECT_NEXT_KEY                 // 5
+	_CONTEXT_IN_OBJECT_NEXT_VALUE               // 6
 )
 
 func (p _ParseContext) String() string {
@@ -59,7 +62,33 @@
 	return "UNKNOWN-PARSE-CONTEXT"
 }
 
-// JSON protocol implementation for thrift.
+type jsonContextStack []_ParseContext
+
+func (s *jsonContextStack) push(v _ParseContext) {
+	*s = append(*s, v)
+}
+
+func (s jsonContextStack) peek() (v _ParseContext, ok bool) {
+	l := len(s)
+	if l <= 0 {
+		return
+	}
+	return s[l-1], true
+}
+
+func (s *jsonContextStack) pop() (v _ParseContext, ok bool) {
+	l := len(*s)
+	if l <= 0 {
+		return
+	}
+	v = (*s)[l-1]
+	*s = (*s)[0 : l-1]
+	return v, true
+}
+
+var errEmptyJSONContextStack = NewTProtocolExceptionWithType(INVALID_DATA, errors.New("Unexpected empty json protocol context stack"))
+
+// Simple JSON protocol implementation for thrift.
 //
 // This protocol produces/consumes a simple output format
 // suitable for parsing by scripting languages.  It should not be
@@ -68,8 +97,8 @@
 type TSimpleJSONProtocol struct {
 	trans TTransport
 
-	parseContextStack []int
-	dumpContext       []int
+	parseContextStack jsonContextStack
+	dumpContext       jsonContextStack
 
 	writer *bufio.Writer
 	reader *bufio.Reader
@@ -81,8 +110,8 @@
 		writer: bufio.NewWriter(t),
 		reader: bufio.NewReader(t),
 	}
-	v.parseContextStack = append(v.parseContextStack, int(_CONTEXT_IN_TOPLEVEL))
-	v.dumpContext = append(v.dumpContext, int(_CONTEXT_IN_TOPLEVEL))
+	v.parseContextStack.push(_CONTEXT_IN_TOPLEVEL)
+	v.dumpContext.push(_CONTEXT_IN_TOPLEVEL)
 	return v
 }
 
@@ -155,114 +184,113 @@
 	return fmt.Errorf("Expected '%s' but found '%s' while parsing JSON.", expected, actual)
 }
 
-func (p *TSimpleJSONProtocol) WriteMessageBegin(name string, typeId TMessageType, seqId int32) error {
+func (p *TSimpleJSONProtocol) WriteMessageBegin(ctx context.Context, name string, typeId TMessageType, seqId int32) error {
 	p.resetContextStack() // THRIFT-3735
 	if e := p.OutputListBegin(); e != nil {
 		return e
 	}
-	if e := p.WriteString(name); e != nil {
+	if e := p.WriteString(ctx, name); e != nil {
 		return e
 	}
-	if e := p.WriteByte(int8(typeId)); e != nil {
+	if e := p.WriteByte(ctx, int8(typeId)); e != nil {
 		return e
 	}
-	if e := p.WriteI32(seqId); e != nil {
+	if e := p.WriteI32(ctx, seqId); e != nil {
 		return e
 	}
 	return nil
 }
 
-func (p *TSimpleJSONProtocol) WriteMessageEnd() error {
+func (p *TSimpleJSONProtocol) WriteMessageEnd(ctx context.Context) error {
 	return p.OutputListEnd()
 }
 
-func (p *TSimpleJSONProtocol) WriteStructBegin(name string) error {
+func (p *TSimpleJSONProtocol) WriteStructBegin(ctx context.Context, name string) error {
 	if e := p.OutputObjectBegin(); e != nil {
 		return e
 	}
 	return nil
 }
 
-func (p *TSimpleJSONProtocol) WriteStructEnd() error {
+func (p *TSimpleJSONProtocol) WriteStructEnd(ctx context.Context) error {
 	return p.OutputObjectEnd()
 }
 
-func (p *TSimpleJSONProtocol) WriteFieldBegin(name string, typeId TType, id int16) error {
-	if e := p.WriteString(name); e != nil {
+func (p *TSimpleJSONProtocol) WriteFieldBegin(ctx context.Context, name string, typeId TType, id int16) error {
+	if e := p.WriteString(ctx, name); e != nil {
 		return e
 	}
 	return nil
 }
 
-func (p *TSimpleJSONProtocol) WriteFieldEnd() error {
-	//return p.OutputListEnd()
+func (p *TSimpleJSONProtocol) WriteFieldEnd(ctx context.Context) error {
 	return nil
 }
 
-func (p *TSimpleJSONProtocol) WriteFieldStop() error { return nil }
+func (p *TSimpleJSONProtocol) WriteFieldStop(ctx context.Context) error { return nil }
 
-func (p *TSimpleJSONProtocol) WriteMapBegin(keyType TType, valueType TType, size int) error {
+func (p *TSimpleJSONProtocol) WriteMapBegin(ctx context.Context, keyType TType, valueType TType, size int) error {
 	if e := p.OutputListBegin(); e != nil {
 		return e
 	}
-	if e := p.WriteByte(int8(keyType)); e != nil {
+	if e := p.WriteByte(ctx, int8(keyType)); e != nil {
 		return e
 	}
-	if e := p.WriteByte(int8(valueType)); e != nil {
+	if e := p.WriteByte(ctx, int8(valueType)); e != nil {
 		return e
 	}
-	return p.WriteI32(int32(size))
+	return p.WriteI32(ctx, int32(size))
 }
 
-func (p *TSimpleJSONProtocol) WriteMapEnd() error {
+func (p *TSimpleJSONProtocol) WriteMapEnd(ctx context.Context) error {
 	return p.OutputListEnd()
 }
 
-func (p *TSimpleJSONProtocol) WriteListBegin(elemType TType, size int) error {
+func (p *TSimpleJSONProtocol) WriteListBegin(ctx context.Context, elemType TType, size int) error {
 	return p.OutputElemListBegin(elemType, size)
 }
 
-func (p *TSimpleJSONProtocol) WriteListEnd() error {
+func (p *TSimpleJSONProtocol) WriteListEnd(ctx context.Context) error {
 	return p.OutputListEnd()
 }
 
-func (p *TSimpleJSONProtocol) WriteSetBegin(elemType TType, size int) error {
+func (p *TSimpleJSONProtocol) WriteSetBegin(ctx context.Context, elemType TType, size int) error {
 	return p.OutputElemListBegin(elemType, size)
 }
 
-func (p *TSimpleJSONProtocol) WriteSetEnd() error {
+func (p *TSimpleJSONProtocol) WriteSetEnd(ctx context.Context) error {
 	return p.OutputListEnd()
 }
 
-func (p *TSimpleJSONProtocol) WriteBool(b bool) error {
+func (p *TSimpleJSONProtocol) WriteBool(ctx context.Context, b bool) error {
 	return p.OutputBool(b)
 }
 
-func (p *TSimpleJSONProtocol) WriteByte(b int8) error {
-	return p.WriteI32(int32(b))
+func (p *TSimpleJSONProtocol) WriteByte(ctx context.Context, b int8) error {
+	return p.WriteI32(ctx, int32(b))
 }
 
-func (p *TSimpleJSONProtocol) WriteI16(v int16) error {
-	return p.WriteI32(int32(v))
+func (p *TSimpleJSONProtocol) WriteI16(ctx context.Context, v int16) error {
+	return p.WriteI32(ctx, int32(v))
 }
 
-func (p *TSimpleJSONProtocol) WriteI32(v int32) error {
+func (p *TSimpleJSONProtocol) WriteI32(ctx context.Context, v int32) error {
 	return p.OutputI64(int64(v))
 }
 
-func (p *TSimpleJSONProtocol) WriteI64(v int64) error {
+func (p *TSimpleJSONProtocol) WriteI64(ctx context.Context, v int64) error {
 	return p.OutputI64(int64(v))
 }
 
-func (p *TSimpleJSONProtocol) WriteDouble(v float64) error {
+func (p *TSimpleJSONProtocol) WriteDouble(ctx context.Context, v float64) error {
 	return p.OutputF64(v)
 }
 
-func (p *TSimpleJSONProtocol) WriteString(v string) error {
+func (p *TSimpleJSONProtocol) WriteString(ctx context.Context, v string) error {
 	return p.OutputString(v)
 }
 
-func (p *TSimpleJSONProtocol) WriteBinary(v []byte) error {
+func (p *TSimpleJSONProtocol) WriteBinary(ctx context.Context, v []byte) error {
 	// JSON library only takes in a string,
 	// not an arbitrary byte array, to ensure bytes are transmitted
 	// efficiently we must convert this into a valid JSON string
@@ -288,39 +316,39 @@
 }
 
 // Reading methods.
-func (p *TSimpleJSONProtocol) ReadMessageBegin() (name string, typeId TMessageType, seqId int32, err error) {
+func (p *TSimpleJSONProtocol) ReadMessageBegin(ctx context.Context) (name string, typeId TMessageType, seqId int32, err error) {
 	p.resetContextStack() // THRIFT-3735
 	if isNull, err := p.ParseListBegin(); isNull || err != nil {
 		return name, typeId, seqId, err
 	}
-	if name, err = p.ReadString(); err != nil {
+	if name, err = p.ReadString(ctx); err != nil {
 		return name, typeId, seqId, err
 	}
-	bTypeId, err := p.ReadByte()
+	bTypeId, err := p.ReadByte(ctx)
 	typeId = TMessageType(bTypeId)
 	if err != nil {
 		return name, typeId, seqId, err
 	}
-	if seqId, err = p.ReadI32(); err != nil {
+	if seqId, err = p.ReadI32(ctx); err != nil {
 		return name, typeId, seqId, err
 	}
 	return name, typeId, seqId, nil
 }
 
-func (p *TSimpleJSONProtocol) ReadMessageEnd() error {
+func (p *TSimpleJSONProtocol) ReadMessageEnd(ctx context.Context) error {
 	return p.ParseListEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadStructBegin() (name string, err error) {
+func (p *TSimpleJSONProtocol) ReadStructBegin(ctx context.Context) (name string, err error) {
 	_, err = p.ParseObjectStart()
 	return "", err
 }
 
-func (p *TSimpleJSONProtocol) ReadStructEnd() error {
+func (p *TSimpleJSONProtocol) ReadStructEnd(ctx context.Context) error {
 	return p.ParseObjectEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadFieldBegin() (string, TType, int16, error) {
+func (p *TSimpleJSONProtocol) ReadFieldBegin(ctx context.Context) (string, TType, int16, error) {
 	if err := p.ParsePreValue(); err != nil {
 		return "", STOP, 0, err
 	}
@@ -339,21 +367,6 @@
 				return name, STOP, 0, err
 			}
 			return name, STOP, -1, p.ParsePostValue()
-			/*
-			   if err = p.ParsePostValue(); err != nil {
-			     return name, STOP, 0, err
-			   }
-			   if isNull, err := p.ParseListBegin(); isNull || err != nil {
-			     return name, STOP, 0, err
-			   }
-			   bType, err := p.ReadByte()
-			   thetype := TType(bType)
-			   if err != nil {
-			     return name, thetype, 0, err
-			   }
-			   id, err := p.ReadI16()
-			   return name, thetype, id, err
-			*/
 		}
 		e := fmt.Errorf("Expected \"}\" or '\"', but found: '%s'", string(b))
 		return "", STOP, 0, NewTProtocolExceptionWithType(INVALID_DATA, e)
@@ -361,57 +374,56 @@
 	return "", STOP, 0, NewTProtocolException(io.EOF)
 }
 
-func (p *TSimpleJSONProtocol) ReadFieldEnd() error {
+func (p *TSimpleJSONProtocol) ReadFieldEnd(ctx context.Context) error {
 	return nil
-	//return p.ParseListEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadMapBegin() (keyType TType, valueType TType, size int, e error) {
+func (p *TSimpleJSONProtocol) ReadMapBegin(ctx context.Context) (keyType TType, valueType TType, size int, e error) {
 	if isNull, e := p.ParseListBegin(); isNull || e != nil {
 		return VOID, VOID, 0, e
 	}
 
 	// read keyType
-	bKeyType, e := p.ReadByte()
+	bKeyType, e := p.ReadByte(ctx)
 	keyType = TType(bKeyType)
 	if e != nil {
 		return keyType, valueType, size, e
 	}
 
 	// read valueType
-	bValueType, e := p.ReadByte()
+	bValueType, e := p.ReadByte(ctx)
 	valueType = TType(bValueType)
 	if e != nil {
 		return keyType, valueType, size, e
 	}
 
 	// read size
-	iSize, err := p.ReadI64()
+	iSize, err := p.ReadI64(ctx)
 	size = int(iSize)
 	return keyType, valueType, size, err
 }
 
-func (p *TSimpleJSONProtocol) ReadMapEnd() error {
+func (p *TSimpleJSONProtocol) ReadMapEnd(ctx context.Context) error {
 	return p.ParseListEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadListBegin() (elemType TType, size int, e error) {
+func (p *TSimpleJSONProtocol) ReadListBegin(ctx context.Context) (elemType TType, size int, e error) {
 	return p.ParseElemListBegin()
 }
 
-func (p *TSimpleJSONProtocol) ReadListEnd() error {
+func (p *TSimpleJSONProtocol) ReadListEnd(ctx context.Context) error {
 	return p.ParseListEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadSetBegin() (elemType TType, size int, e error) {
+func (p *TSimpleJSONProtocol) ReadSetBegin(ctx context.Context) (elemType TType, size int, e error) {
 	return p.ParseElemListBegin()
 }
 
-func (p *TSimpleJSONProtocol) ReadSetEnd() error {
+func (p *TSimpleJSONProtocol) ReadSetEnd(ctx context.Context) error {
 	return p.ParseListEnd()
 }
 
-func (p *TSimpleJSONProtocol) ReadBool() (bool, error) {
+func (p *TSimpleJSONProtocol) ReadBool(ctx context.Context) (bool, error) {
 	var value bool
 
 	if err := p.ParsePreValue(); err != nil {
@@ -466,32 +478,32 @@
 	return value, p.ParsePostValue()
 }
 
-func (p *TSimpleJSONProtocol) ReadByte() (int8, error) {
-	v, err := p.ReadI64()
+func (p *TSimpleJSONProtocol) ReadByte(ctx context.Context) (int8, error) {
+	v, err := p.ReadI64(ctx)
 	return int8(v), err
 }
 
-func (p *TSimpleJSONProtocol) ReadI16() (int16, error) {
-	v, err := p.ReadI64()
+func (p *TSimpleJSONProtocol) ReadI16(ctx context.Context) (int16, error) {
+	v, err := p.ReadI64(ctx)
 	return int16(v), err
 }
 
-func (p *TSimpleJSONProtocol) ReadI32() (int32, error) {
-	v, err := p.ReadI64()
+func (p *TSimpleJSONProtocol) ReadI32(ctx context.Context) (int32, error) {
+	v, err := p.ReadI64(ctx)
 	return int32(v), err
 }
 
-func (p *TSimpleJSONProtocol) ReadI64() (int64, error) {
+func (p *TSimpleJSONProtocol) ReadI64(ctx context.Context) (int64, error) {
 	v, _, err := p.ParseI64()
 	return v, err
 }
 
-func (p *TSimpleJSONProtocol) ReadDouble() (float64, error) {
+func (p *TSimpleJSONProtocol) ReadDouble(ctx context.Context) (float64, error) {
 	v, _, err := p.ParseF64()
 	return v, err
 }
 
-func (p *TSimpleJSONProtocol) ReadString() (string, error) {
+func (p *TSimpleJSONProtocol) ReadString(ctx context.Context) (string, error) {
 	var v string
 	if err := p.ParsePreValue(); err != nil {
 		return v, err
@@ -521,7 +533,7 @@
 	return v, p.ParsePostValue()
 }
 
-func (p *TSimpleJSONProtocol) ReadBinary() ([]byte, error) {
+func (p *TSimpleJSONProtocol) ReadBinary(ctx context.Context) ([]byte, error) {
 	var v []byte
 	if err := p.ParsePreValue(); err != nil {
 		return nil, err
@@ -552,12 +564,12 @@
 	return v, p.ParsePostValue()
 }
 
-func (p *TSimpleJSONProtocol) Flush() (err error) {
+func (p *TSimpleJSONProtocol) Flush(ctx context.Context) (err error) {
 	return NewTProtocolException(p.writer.Flush())
 }
 
-func (p *TSimpleJSONProtocol) Skip(fieldType TType) (err error) {
-	return SkipDefaultDepth(p, fieldType)
+func (p *TSimpleJSONProtocol) Skip(ctx context.Context, fieldType TType) (err error) {
+	return SkipDefaultDepth(ctx, p, fieldType)
 }
 
 func (p *TSimpleJSONProtocol) Transport() TTransport {
@@ -565,41 +577,41 @@
 }
 
 func (p *TSimpleJSONProtocol) OutputPreValue() error {
-	cxt := _ParseContext(p.dumpContext[len(p.dumpContext)-1])
+	cxt, ok := p.dumpContext.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	switch cxt {
 	case _CONTEXT_IN_LIST, _CONTEXT_IN_OBJECT_NEXT_KEY:
 		if _, e := p.write(JSON_COMMA); e != nil {
 			return NewTProtocolException(e)
 		}
-		break
 	case _CONTEXT_IN_OBJECT_NEXT_VALUE:
 		if _, e := p.write(JSON_COLON); e != nil {
 			return NewTProtocolException(e)
 		}
-		break
 	}
 	return nil
 }
 
 func (p *TSimpleJSONProtocol) OutputPostValue() error {
-	cxt := _ParseContext(p.dumpContext[len(p.dumpContext)-1])
+	cxt, ok := p.dumpContext.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	switch cxt {
 	case _CONTEXT_IN_LIST_FIRST:
-		p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
-		p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_LIST))
-		break
+		p.dumpContext.pop()
+		p.dumpContext.push(_CONTEXT_IN_LIST)
 	case _CONTEXT_IN_OBJECT_FIRST:
-		p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
-		p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_OBJECT_NEXT_VALUE))
-		break
+		p.dumpContext.pop()
+		p.dumpContext.push(_CONTEXT_IN_OBJECT_NEXT_VALUE)
 	case _CONTEXT_IN_OBJECT_NEXT_KEY:
-		p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
-		p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_OBJECT_NEXT_VALUE))
-		break
+		p.dumpContext.pop()
+		p.dumpContext.push(_CONTEXT_IN_OBJECT_NEXT_VALUE)
 	case _CONTEXT_IN_OBJECT_NEXT_VALUE:
-		p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
-		p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_OBJECT_NEXT_KEY))
-		break
+		p.dumpContext.pop()
+		p.dumpContext.push(_CONTEXT_IN_OBJECT_NEXT_KEY)
 	}
 	return nil
 }
@@ -614,10 +626,13 @@
 	} else {
 		v = string(JSON_FALSE)
 	}
-	switch _ParseContext(p.dumpContext[len(p.dumpContext)-1]) {
+	cxt, ok := p.dumpContext.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
+	switch cxt {
 	case _CONTEXT_IN_OBJECT_FIRST, _CONTEXT_IN_OBJECT_NEXT_KEY:
 		v = jsonQuote(v)
-	default:
 	}
 	if e := p.OutputStringData(v); e != nil {
 		return e
@@ -647,11 +662,14 @@
 	} else if math.IsInf(value, -1) {
 		v = string(JSON_QUOTE) + JSON_NEGATIVE_INFINITY + string(JSON_QUOTE)
 	} else {
+		cxt, ok := p.dumpContext.peek()
+		if !ok {
+			return errEmptyJSONContextStack
+		}
 		v = strconv.FormatFloat(value, 'g', -1, 64)
-		switch _ParseContext(p.dumpContext[len(p.dumpContext)-1]) {
+		switch cxt {
 		case _CONTEXT_IN_OBJECT_FIRST, _CONTEXT_IN_OBJECT_NEXT_KEY:
 			v = string(JSON_QUOTE) + v + string(JSON_QUOTE)
-		default:
 		}
 	}
 	if e := p.OutputStringData(v); e != nil {
@@ -664,11 +682,14 @@
 	if e := p.OutputPreValue(); e != nil {
 		return e
 	}
+	cxt, ok := p.dumpContext.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	v := strconv.FormatInt(value, 10)
-	switch _ParseContext(p.dumpContext[len(p.dumpContext)-1]) {
+	switch cxt {
 	case _CONTEXT_IN_OBJECT_FIRST, _CONTEXT_IN_OBJECT_NEXT_KEY:
 		v = jsonQuote(v)
-	default:
 	}
 	if e := p.OutputStringData(v); e != nil {
 		return e
@@ -698,7 +719,7 @@
 	if _, e := p.write(JSON_LBRACE); e != nil {
 		return NewTProtocolException(e)
 	}
-	p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_OBJECT_FIRST))
+	p.dumpContext.push(_CONTEXT_IN_OBJECT_FIRST)
 	return nil
 }
 
@@ -706,7 +727,10 @@
 	if _, e := p.write(JSON_RBRACE); e != nil {
 		return NewTProtocolException(e)
 	}
-	p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
+	_, ok := p.dumpContext.pop()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	if e := p.OutputPostValue(); e != nil {
 		return e
 	}
@@ -720,7 +744,7 @@
 	if _, e := p.write(JSON_LBRACKET); e != nil {
 		return NewTProtocolException(e)
 	}
-	p.dumpContext = append(p.dumpContext, int(_CONTEXT_IN_LIST_FIRST))
+	p.dumpContext.push(_CONTEXT_IN_LIST_FIRST)
 	return nil
 }
 
@@ -728,7 +752,10 @@
 	if _, e := p.write(JSON_RBRACKET); e != nil {
 		return NewTProtocolException(e)
 	}
-	p.dumpContext = p.dumpContext[:len(p.dumpContext)-1]
+	_, ok := p.dumpContext.pop()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	if e := p.OutputPostValue(); e != nil {
 		return e
 	}
@@ -739,10 +766,10 @@
 	if e := p.OutputListBegin(); e != nil {
 		return e
 	}
-	if e := p.WriteByte(int8(elemType)); e != nil {
+	if e := p.OutputI64(int64(elemType)); e != nil {
 		return e
 	}
-	if e := p.WriteI64(int64(size)); e != nil {
+	if e := p.OutputI64(int64(size)); e != nil {
 		return e
 	}
 	return nil
@@ -752,7 +779,10 @@
 	if e := p.readNonSignificantWhitespace(); e != nil {
 		return NewTProtocolException(e)
 	}
-	cxt := _ParseContext(p.parseContextStack[len(p.parseContextStack)-1])
+	cxt, ok := p.parseContextStack.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	b, _ := p.reader.Peek(1)
 	switch cxt {
 	case _CONTEXT_IN_LIST:
@@ -771,7 +801,6 @@
 				return NewTProtocolExceptionWithType(INVALID_DATA, e)
 			}
 		}
-		break
 	case _CONTEXT_IN_OBJECT_NEXT_KEY:
 		if len(b) > 0 {
 			switch b[0] {
@@ -788,7 +817,6 @@
 				return NewTProtocolExceptionWithType(INVALID_DATA, e)
 			}
 		}
-		break
 	case _CONTEXT_IN_OBJECT_NEXT_VALUE:
 		if len(b) > 0 {
 			switch b[0] {
@@ -803,7 +831,6 @@
 				return NewTProtocolExceptionWithType(INVALID_DATA, e)
 			}
 		}
-		break
 	}
 	return nil
 }
@@ -812,20 +839,20 @@
 	if e := p.readNonSignificantWhitespace(); e != nil {
 		return NewTProtocolException(e)
 	}
-	cxt := _ParseContext(p.parseContextStack[len(p.parseContextStack)-1])
+	cxt, ok := p.parseContextStack.peek()
+	if !ok {
+		return errEmptyJSONContextStack
+	}
 	switch cxt {
 	case _CONTEXT_IN_LIST_FIRST:
-		p.parseContextStack = p.parseContextStack[:len(p.parseContextStack)-1]
-		p.parseContextStack = append(p.parseContextStack, int(_CONTEXT_IN_LIST))
-		break
+		p.parseContextStack.pop()
+		p.parseContextStack.push(_CONTEXT_IN_LIST)
 	case _CONTEXT_IN_OBJECT_FIRST, _CONTEXT_IN_OBJECT_NEXT_KEY:
-		p.parseContextStack = p.parseContextStack[:len(p.parseContextStack)-1]
-		p.parseContextStack = append(p.parseContextStack, int(_CONTEXT_IN_OBJECT_NEXT_VALUE))
-		break
+		p.parseContextStack.pop()
+		p.parseContextStack.push(_CONTEXT_IN_OBJECT_NEXT_VALUE)
 	case _CONTEXT_IN_OBJECT_NEXT_VALUE:
-		p.parseContextStack = p.parseContextStack[:len(p.parseContextStack)-1]
-		p.parseContextStack = append(p.parseContextStack, int(_CONTEXT_IN_OBJECT_NEXT_KEY))
-		break
+		p.parseContextStack.pop()
+		p.parseContextStack.push(_CONTEXT_IN_OBJECT_NEXT_KEY)
 	}
 	return nil
 }
@@ -978,7 +1005,7 @@
 	}
 	if len(b) > 0 && b[0] == JSON_LBRACE[0] {
 		p.reader.ReadByte()
-		p.parseContextStack = append(p.parseContextStack, int(_CONTEXT_IN_OBJECT_FIRST))
+		p.parseContextStack.push(_CONTEXT_IN_OBJECT_FIRST)
 		return false, nil
 	} else if p.safePeekContains(JSON_NULL) {
 		return true, nil
@@ -991,7 +1018,7 @@
 	if isNull, err := p.readIfNull(); isNull || err != nil {
 		return err
 	}
-	cxt := _ParseContext(p.parseContextStack[len(p.parseContextStack)-1])
+	cxt, _ := p.parseContextStack.peek()
 	if (cxt != _CONTEXT_IN_OBJECT_FIRST) && (cxt != _CONTEXT_IN_OBJECT_NEXT_KEY) {
 		e := fmt.Errorf("Expected to be in the Object Context, but not in Object Context (%d)", cxt)
 		return NewTProtocolExceptionWithType(INVALID_DATA, e)
@@ -1009,7 +1036,7 @@
 			break
 		}
 	}
-	p.parseContextStack = p.parseContextStack[:len(p.parseContextStack)-1]
+	p.parseContextStack.pop()
 	return p.ParsePostValue()
 }
 
@@ -1023,7 +1050,7 @@
 		return false, err
 	}
 	if len(b) >= 1 && b[0] == JSON_LBRACKET[0] {
-		p.parseContextStack = append(p.parseContextStack, int(_CONTEXT_IN_LIST_FIRST))
+		p.parseContextStack.push(_CONTEXT_IN_LIST_FIRST)
 		p.reader.ReadByte()
 		isNull = false
 	} else if p.safePeekContains(JSON_NULL) {
@@ -1038,12 +1065,12 @@
 	if isNull, e := p.ParseListBegin(); isNull || e != nil {
 		return VOID, 0, e
 	}
-	bElemType, err := p.ReadByte()
+	bElemType, _, err := p.ParseI64()
 	elemType = TType(bElemType)
 	if err != nil {
 		return elemType, size, err
 	}
-	nSize, err2 := p.ReadI64()
+	nSize, _, err2 := p.ParseI64()
 	size = int(nSize)
 	return elemType, size, err2
 }
@@ -1052,7 +1079,7 @@
 	if isNull, err := p.readIfNull(); isNull || err != nil {
 		return err
 	}
-	cxt := _ParseContext(p.parseContextStack[len(p.parseContextStack)-1])
+	cxt, _ := p.parseContextStack.peek()
 	if cxt != _CONTEXT_IN_LIST {
 		e := fmt.Errorf("Expected to be in the List Context, but not in List Context (%d)", cxt)
 		return NewTProtocolExceptionWithType(INVALID_DATA, e)
@@ -1064,14 +1091,16 @@
 	for _, char := range line {
 		switch char {
 		default:
-			e := fmt.Errorf("Expecting end of list \"]\", but found: \"%s\"", line)
+			e := fmt.Errorf("Expecting end of list \"]\", but found: \"%v\"", line)
 			return NewTProtocolExceptionWithType(INVALID_DATA, e)
 		case ' ', '\n', '\r', '\t', rune(JSON_RBRACKET[0]):
 			break
 		}
 	}
-	p.parseContextStack = p.parseContextStack[:len(p.parseContextStack)-1]
-	if _ParseContext(p.parseContextStack[len(p.parseContextStack)-1]) == _CONTEXT_IN_TOPLEVEL {
+	p.parseContextStack.pop()
+	if cxt, ok := p.parseContextStack.peek(); !ok {
+		return errEmptyJSONContextStack
+	} else if cxt == _CONTEXT_IN_TOPLEVEL {
 		return nil
 	}
 	return p.ParsePostValue()
@@ -1315,7 +1344,7 @@
 func (p *TSimpleJSONProtocol) safePeekContains(b []byte) bool {
 	for i := 0; i < len(b); i++ {
 		a, _ := p.reader.Peek(i + 1)
-		if len(a) == 0 || a[i] != b[i] {
+		if len(a) < (i+1) || a[i] != b[i] {
 			return false
 		}
 	}
@@ -1324,8 +1353,8 @@
 
 // Reset the context stack to its initial state.
 func (p *TSimpleJSONProtocol) resetContextStack() {
-	p.parseContextStack = []int{int(_CONTEXT_IN_TOPLEVEL)}
-	p.dumpContext = []int{int(_CONTEXT_IN_TOPLEVEL)}
+	p.parseContextStack = jsonContextStack{_CONTEXT_IN_TOPLEVEL}
+	p.dumpContext = jsonContextStack{_CONTEXT_IN_TOPLEVEL}
 }
 
 func (p *TSimpleJSONProtocol) write(b []byte) (int, error) {
@@ -1335,3 +1364,10 @@
 	}
 	return n, err
 }
+
+// SetTConfiguration implements TConfigurationSetter for propagation.
+func (p *TSimpleJSONProtocol) SetTConfiguration(conf *TConfiguration) {
+	PropagateTConfiguration(p.trans, conf)
+}
+
+var _ TConfigurationSetter = (*TSimpleJSONProtocol)(nil)
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/simple_server.go b/vendor/github.com/uber/jaeger-client-go/thrift/simple_server.go
new file mode 100644
index 0000000..563cbfc
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/simple_server.go
@@ -0,0 +1,332 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package thrift
+
+import (
+	"errors"
+	"fmt"
+	"io"
+	"sync"
+	"sync/atomic"
+	"time"
+)
+
+// ErrAbandonRequest is a special error server handler implementations can
+// return to indicate that the request has been abandoned.
+//
+// TSimpleServer will check for this error, and close the client connection
+// instead of writing the response/error back to the client.
+//
+// It shall only be used when the server handler implementation know that the
+// client already abandoned the request (by checking that the passed in context
+// is already canceled, for example).
+var ErrAbandonRequest = errors.New("request abandoned")
+
+// ServerConnectivityCheckInterval defines the ticker interval used by
+// connectivity check in thrift compiled TProcessorFunc implementations.
+//
+// It's defined as a variable instead of constant, so that thrift server
+// implementations can change its value to control the behavior.
+//
+// If it's changed to <=0, the feature will be disabled.
+var ServerConnectivityCheckInterval = time.Millisecond * 5
+
+/*
+ * This is not a typical TSimpleServer as it is not blocked after accept a socket.
+ * It is more like a TThreadedServer that can handle different connections in different goroutines.
+ * This will work if golang user implements a conn-pool like thing in client side.
+ */
+type TSimpleServer struct {
+	closed int32
+	wg     sync.WaitGroup
+	mu     sync.Mutex
+
+	processorFactory       TProcessorFactory
+	serverTransport        TServerTransport
+	inputTransportFactory  TTransportFactory
+	outputTransportFactory TTransportFactory
+	inputProtocolFactory   TProtocolFactory
+	outputProtocolFactory  TProtocolFactory
+
+	// Headers to auto forward in THeaderProtocol
+	forwardHeaders []string
+
+	logger Logger
+}
+
+func NewTSimpleServer2(processor TProcessor, serverTransport TServerTransport) *TSimpleServer {
+	return NewTSimpleServerFactory2(NewTProcessorFactory(processor), serverTransport)
+}
+
+func NewTSimpleServer4(processor TProcessor, serverTransport TServerTransport, transportFactory TTransportFactory, protocolFactory TProtocolFactory) *TSimpleServer {
+	return NewTSimpleServerFactory4(NewTProcessorFactory(processor),
+		serverTransport,
+		transportFactory,
+		protocolFactory,
+	)
+}
+
+func NewTSimpleServer6(processor TProcessor, serverTransport TServerTransport, inputTransportFactory TTransportFactory, outputTransportFactory TTransportFactory, inputProtocolFactory TProtocolFactory, outputProtocolFactory TProtocolFactory) *TSimpleServer {
+	return NewTSimpleServerFactory6(NewTProcessorFactory(processor),
+		serverTransport,
+		inputTransportFactory,
+		outputTransportFactory,
+		inputProtocolFactory,
+		outputProtocolFactory,
+	)
+}
+
+func NewTSimpleServerFactory2(processorFactory TProcessorFactory, serverTransport TServerTransport) *TSimpleServer {
+	return NewTSimpleServerFactory6(processorFactory,
+		serverTransport,
+		NewTTransportFactory(),
+		NewTTransportFactory(),
+		NewTBinaryProtocolFactoryDefault(),
+		NewTBinaryProtocolFactoryDefault(),
+	)
+}
+
+func NewTSimpleServerFactory4(processorFactory TProcessorFactory, serverTransport TServerTransport, transportFactory TTransportFactory, protocolFactory TProtocolFactory) *TSimpleServer {
+	return NewTSimpleServerFactory6(processorFactory,
+		serverTransport,
+		transportFactory,
+		transportFactory,
+		protocolFactory,
+		protocolFactory,
+	)
+}
+
+func NewTSimpleServerFactory6(processorFactory TProcessorFactory, serverTransport TServerTransport, inputTransportFactory TTransportFactory, outputTransportFactory TTransportFactory, inputProtocolFactory TProtocolFactory, outputProtocolFactory TProtocolFactory) *TSimpleServer {
+	return &TSimpleServer{
+		processorFactory:       processorFactory,
+		serverTransport:        serverTransport,
+		inputTransportFactory:  inputTransportFactory,
+		outputTransportFactory: outputTransportFactory,
+		inputProtocolFactory:   inputProtocolFactory,
+		outputProtocolFactory:  outputProtocolFactory,
+	}
+}
+
+func (p *TSimpleServer) ProcessorFactory() TProcessorFactory {
+	return p.processorFactory
+}
+
+func (p *TSimpleServer) ServerTransport() TServerTransport {
+	return p.serverTransport
+}
+
+func (p *TSimpleServer) InputTransportFactory() TTransportFactory {
+	return p.inputTransportFactory
+}
+
+func (p *TSimpleServer) OutputTransportFactory() TTransportFactory {
+	return p.outputTransportFactory
+}
+
+func (p *TSimpleServer) InputProtocolFactory() TProtocolFactory {
+	return p.inputProtocolFactory
+}
+
+func (p *TSimpleServer) OutputProtocolFactory() TProtocolFactory {
+	return p.outputProtocolFactory
+}
+
+func (p *TSimpleServer) Listen() error {
+	return p.serverTransport.Listen()
+}
+
+// SetForwardHeaders sets the list of header keys that will be auto forwarded
+// while using THeaderProtocol.
+//
+// "forward" means that when the server is also a client to other upstream
+// thrift servers, the context object user gets in the processor functions will
+// have both read and write headers set, with write headers being forwarded.
+// Users can always override the write headers by calling SetWriteHeaderList
+// before calling thrift client functions.
+func (p *TSimpleServer) SetForwardHeaders(headers []string) {
+	size := len(headers)
+	if size == 0 {
+		p.forwardHeaders = nil
+		return
+	}
+
+	keys := make([]string, size)
+	copy(keys, headers)
+	p.forwardHeaders = keys
+}
+
+// SetLogger sets the logger used by this TSimpleServer.
+//
+// If no logger was set before Serve is called, a default logger using standard
+// log library will be used.
+func (p *TSimpleServer) SetLogger(logger Logger) {
+	p.logger = logger
+}
+
+func (p *TSimpleServer) innerAccept() (int32, error) {
+	client, err := p.serverTransport.Accept()
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	closed := atomic.LoadInt32(&p.closed)
+	if closed != 0 {
+		return closed, nil
+	}
+	if err != nil {
+		return 0, err
+	}
+	if client != nil {
+		p.wg.Add(1)
+		go func() {
+			defer p.wg.Done()
+			if err := p.processRequests(client); err != nil {
+				p.logger(fmt.Sprintf("error processing request: %v", err))
+			}
+		}()
+	}
+	return 0, nil
+}
+
+func (p *TSimpleServer) AcceptLoop() error {
+	for {
+		closed, err := p.innerAccept()
+		if err != nil {
+			return err
+		}
+		if closed != 0 {
+			return nil
+		}
+	}
+}
+
+func (p *TSimpleServer) Serve() error {
+	p.logger = fallbackLogger(p.logger)
+
+	err := p.Listen()
+	if err != nil {
+		return err
+	}
+	p.AcceptLoop()
+	return nil
+}
+
+func (p *TSimpleServer) Stop() error {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	if atomic.LoadInt32(&p.closed) != 0 {
+		return nil
+	}
+	atomic.StoreInt32(&p.closed, 1)
+	p.serverTransport.Interrupt()
+	p.wg.Wait()
+	return nil
+}
+
+// If err is actually EOF, return nil, otherwise return err as-is.
+func treatEOFErrorsAsNil(err error) error {
+	if err == nil {
+		return nil
+	}
+	if errors.Is(err, io.EOF) {
+		return nil
+	}
+	var te TTransportException
+	if errors.As(err, &te) && te.TypeId() == END_OF_FILE {
+		return nil
+	}
+	return err
+}
+
+func (p *TSimpleServer) processRequests(client TTransport) (err error) {
+	defer func() {
+		err = treatEOFErrorsAsNil(err)
+	}()
+
+	processor := p.processorFactory.GetProcessor(client)
+	inputTransport, err := p.inputTransportFactory.GetTransport(client)
+	if err != nil {
+		return err
+	}
+	inputProtocol := p.inputProtocolFactory.GetProtocol(inputTransport)
+	var outputTransport TTransport
+	var outputProtocol TProtocol
+
+	// for THeaderProtocol, we must use the same protocol instance for
+	// input and output so that the response is in the same dialect that
+	// the server detected the request was in.
+	headerProtocol, ok := inputProtocol.(*THeaderProtocol)
+	if ok {
+		outputProtocol = inputProtocol
+	} else {
+		oTrans, err := p.outputTransportFactory.GetTransport(client)
+		if err != nil {
+			return err
+		}
+		outputTransport = oTrans
+		outputProtocol = p.outputProtocolFactory.GetProtocol(outputTransport)
+	}
+
+	if inputTransport != nil {
+		defer inputTransport.Close()
+	}
+	if outputTransport != nil {
+		defer outputTransport.Close()
+	}
+	for {
+		if atomic.LoadInt32(&p.closed) != 0 {
+			return nil
+		}
+
+		ctx := SetResponseHelper(
+			defaultCtx,
+			TResponseHelper{
+				THeaderResponseHelper: NewTHeaderResponseHelper(outputProtocol),
+			},
+		)
+		if headerProtocol != nil {
+			// We need to call ReadFrame here, otherwise we won't
+			// get any headers on the AddReadTHeaderToContext call.
+			//
+			// ReadFrame is safe to be called multiple times so it
+			// won't break when it's called again later when we
+			// actually start to read the message.
+			if err := headerProtocol.ReadFrame(ctx); err != nil {
+				return err
+			}
+			ctx = AddReadTHeaderToContext(ctx, headerProtocol.GetReadHeaders())
+			ctx = SetWriteHeaderList(ctx, p.forwardHeaders)
+		}
+
+		ok, err := processor.Process(ctx, inputProtocol, outputProtocol)
+		if errors.Is(err, ErrAbandonRequest) {
+			return client.Close()
+		}
+		if errors.As(err, new(TTransportException)) && err != nil {
+			return err
+		}
+		var tae TApplicationException
+		if errors.As(err, &tae) && tae.TypeId() == UNKNOWN_METHOD {
+			continue
+		}
+		if !ok {
+			break
+		}
+	}
+	return nil
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/transport.go b/vendor/github.com/uber/jaeger-client-go/thrift/transport.go
index 4538996..ba2738a 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/transport.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/transport.go
@@ -20,6 +20,7 @@
 package thrift
 
 import (
+	"context"
 	"errors"
 	"io"
 )
@@ -30,15 +31,18 @@
 	Flush() (err error)
 }
 
+type ContextFlusher interface {
+	Flush(ctx context.Context) (err error)
+}
+
 type ReadSizeProvider interface {
 	RemainingBytes() (num_bytes uint64)
 }
 
-
 // Encapsulates the I/O layer
 type TTransport interface {
 	io.ReadWriteCloser
-	Flusher
+	ContextFlusher
 	ReadSizeProvider
 
 	// Opens the transport for communication
@@ -52,7 +56,6 @@
 	WriteString(s string) (n int, err error)
 }
 
-
 // This is "enchanced" transport with extra capabilities. You need to use one of these
 // to construct protocol.
 // Notably, TSocket does not implement this interface, and it is always a mistake to use
@@ -62,7 +65,6 @@
 	io.ByteReader
 	io.ByteWriter
 	stringWriter
-	Flusher
+	ContextFlusher
 	ReadSizeProvider
 }
-
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/transport_exception.go b/vendor/github.com/uber/jaeger-client-go/thrift/transport_exception.go
index 9505b44..0a3f076 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/transport_exception.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/transport_exception.go
@@ -46,6 +46,13 @@
 type tTransportException struct {
 	typeId int
 	err    error
+	msg    string
+}
+
+var _ TTransportException = (*tTransportException)(nil)
+
+func (tTransportException) TExceptionType() TExceptionType {
+	return TExceptionTypeTransport
 }
 
 func (p *tTransportException) TypeId() int {
@@ -53,15 +60,27 @@
 }
 
 func (p *tTransportException) Error() string {
-	return p.err.Error()
+	return p.msg
 }
 
 func (p *tTransportException) Err() error {
 	return p.err
 }
 
+func (p *tTransportException) Unwrap() error {
+	return p.err
+}
+
+func (p *tTransportException) Timeout() bool {
+	return p.typeId == TIMED_OUT
+}
+
 func NewTTransportException(t int, e string) TTransportException {
-	return &tTransportException{typeId: t, err: errors.New(e)}
+	return &tTransportException{
+		typeId: t,
+		err:    errors.New(e),
+		msg:    e,
+	}
 }
 
 func NewTTransportExceptionFromError(e error) TTransportException {
@@ -73,18 +92,40 @@
 		return t
 	}
 
-	switch v := e.(type) {
-	case TTransportException:
-		return v
-	case timeoutable:
-		if v.Timeout() {
-			return &tTransportException{typeId: TIMED_OUT, err: e}
-		}
+	te := &tTransportException{
+		typeId: UNKNOWN_TRANSPORT_EXCEPTION,
+		err:    e,
+		msg:    e.Error(),
 	}
 
-	if e == io.EOF {
-		return &tTransportException{typeId: END_OF_FILE, err: e}
+	if isTimeoutError(e) {
+		te.typeId = TIMED_OUT
+		return te
 	}
 
-	return &tTransportException{typeId: UNKNOWN_TRANSPORT_EXCEPTION, err: e}
+	if errors.Is(e, io.EOF) {
+		te.typeId = END_OF_FILE
+		return te
+	}
+
+	return te
+}
+
+func prependTTransportException(prepend string, e TTransportException) TTransportException {
+	return &tTransportException{
+		typeId: e.TypeId(),
+		err:    e,
+		msg:    prepend + e.Error(),
+	}
+}
+
+// isTimeoutError returns true when err is an error caused by timeout.
+//
+// Note that this also includes TTransportException wrapped timeout errors.
+func isTimeoutError(err error) bool {
+	var t timeoutable
+	if errors.As(err, &t) {
+		return t.Timeout()
+	}
+	return false
 }
diff --git a/vendor/github.com/uber/jaeger-client-go/thrift/transport_factory.go b/vendor/github.com/uber/jaeger-client-go/thrift/transport_factory.go
index 533d1b4..c805807 100644
--- a/vendor/github.com/uber/jaeger-client-go/thrift/transport_factory.go
+++ b/vendor/github.com/uber/jaeger-client-go/thrift/transport_factory.go
@@ -24,14 +24,14 @@
 // a ServerTransport and then may want to mutate them (i.e. create
 // a BufferedTransport from the underlying base transport)
 type TTransportFactory interface {
-	GetTransport(trans TTransport) TTransport
+	GetTransport(trans TTransport) (TTransport, error)
 }
 
 type tTransportFactory struct{}
 
 // Return a wrapped instance of the base Transport.
-func (p *tTransportFactory) GetTransport(trans TTransport) TTransport {
-	return trans
+func (p *tTransportFactory) GetTransport(trans TTransport) (TTransport, error) {
+	return trans, nil
 }
 
 func NewTTransportFactory() TTransportFactory {
diff --git a/vendor/github.com/uber/jaeger-client-go/tracer.go b/vendor/github.com/uber/jaeger-client-go/tracer.go
index 8a3fc97..9a627be 100644
--- a/vendor/github.com/uber/jaeger-client-go/tracer.go
+++ b/vendor/github.com/uber/jaeger-client-go/tracer.go
@@ -216,10 +216,10 @@
 		options.StartTime = t.timeNow()
 	}
 
-	// Predicate whether the given span context is a valid reference
-	// which may be used as parent / debug ID / baggage items source
-	isValidReference := func(ctx SpanContext) bool {
-		return ctx.IsValid() || ctx.isDebugIDContainerOnly() || len(ctx.baggage) != 0
+	// Predicate whether the given span context is an empty reference
+	// or may be used as parent / debug ID / baggage items source
+	isEmptyReference := func(ctx SpanContext) bool {
+		return !ctx.IsValid() && !ctx.isDebugIDContainerOnly() && len(ctx.baggage) == 0
 	}
 
 	var references []Reference
@@ -235,7 +235,7 @@
 				reflect.ValueOf(ref.ReferencedContext)))
 			continue
 		}
-		if !isValidReference(ctxRef) {
+		if isEmptyReference(ctxRef) {
 			continue
 		}
 
@@ -245,14 +245,17 @@
 			continue
 		}
 
-		references = append(references, Reference{Type: ref.Type, Context: ctxRef})
+		if ctxRef.IsValid() {
+			// we don't want empty context that contains only debug-id or baggage
+			references = append(references, Reference{Type: ref.Type, Context: ctxRef})
+		}
 
 		if !hasParent {
 			parent = ctxRef
 			hasParent = ref.Type == opentracing.ChildOfRef
 		}
 	}
-	if !hasParent && isValidReference(parent) {
+	if !hasParent && !isEmptyReference(parent) {
 		// If ChildOfRef wasn't found but a FollowFromRef exists, use the context from
 		// the FollowFromRef as the parent
 		hasParent = true
@@ -317,7 +320,7 @@
 	sp.references = references
 	sp.firstInProcess = rpcServer || sp.context.parentID == 0
 
-	if !sp.isSamplingFinalized() {
+	if !sp.context.isSamplingFinalized() {
 		decision := t.sampler.OnCreateSpan(sp)
 		sp.applySamplingDecision(decision, false)
 	}
@@ -410,7 +413,7 @@
 // calling client-side span, but obviously the server side span is
 // no longer a root span of the trace.
 func (t *Tracer) emitNewSpanMetrics(sp *Span, newTrace bool) {
-	if !sp.isSamplingFinalized() {
+	if !sp.context.isSamplingFinalized() {
 		t.metrics.SpansStartedDelayedSampling.Inc(1)
 		if newTrace {
 			t.metrics.TracesStartedDelayedSampling.Inc(1)
@@ -434,9 +437,11 @@
 }
 
 func (t *Tracer) reportSpan(sp *Span) {
-	if !sp.isSamplingFinalized() {
+	ctx := sp.SpanContext()
+
+	if !ctx.isSamplingFinalized() {
 		t.metrics.SpansFinishedDelayedSampling.Inc(1)
-	} else if sp.context.IsSampled() {
+	} else if ctx.IsSampled() {
 		t.metrics.SpansFinishedSampled.Inc(1)
 	} else {
 		t.metrics.SpansFinishedNotSampled.Inc(1)
@@ -445,7 +450,7 @@
 	// Note: if the reporter is processing Span asynchronously then it needs to Retain() the span,
 	// and then Release() it when no longer needed.
 	// Otherwise, the span may be reused for another trace and its data may be overwritten.
-	if sp.context.IsSampled() {
+	if ctx.IsSampled() {
 		t.reporter.Report(sp)
 	}
 
diff --git a/vendor/github.com/uber/jaeger-client-go/transport/http.go b/vendor/github.com/uber/jaeger-client-go/transport/http.go
index bb7eb00..1d6f14d 100644
--- a/vendor/github.com/uber/jaeger-client-go/transport/http.go
+++ b/vendor/github.com/uber/jaeger-client-go/transport/http.go
@@ -16,6 +16,7 @@
 
 import (
 	"bytes"
+	"context"
 	"fmt"
 	"io"
 	"io/ioutil"
@@ -167,7 +168,7 @@
 func serializeThrift(obj thrift.TStruct) (*bytes.Buffer, error) {
 	t := thrift.NewTMemoryBuffer()
 	p := thrift.NewTBinaryProtocolTransport(t)
-	if err := obj.Write(p); err != nil {
+	if err := obj.Write(context.Background(), p); err != nil {
 		return nil, err
 	}
 	return t.Buffer, nil
diff --git a/vendor/github.com/uber/jaeger-client-go/transport_udp.go b/vendor/github.com/uber/jaeger-client-go/transport_udp.go
index 7370d80..0000412 100644
--- a/vendor/github.com/uber/jaeger-client-go/transport_udp.go
+++ b/vendor/github.com/uber/jaeger-client-go/transport_udp.go
@@ -15,10 +15,12 @@
 package jaeger
 
 import (
+	"context"
 	"errors"
 	"fmt"
 
 	"github.com/uber/jaeger-client-go/internal/reporterstats"
+	"github.com/uber/jaeger-client-go/log"
 	"github.com/uber/jaeger-client-go/thrift"
 	j "github.com/uber/jaeger-client-go/thrift-gen/jaeger"
 	"github.com/uber/jaeger-client-go/utils"
@@ -57,35 +59,57 @@
 	failedToEmitSpans    int64
 }
 
-// NewUDPTransport creates a reporter that submits spans to jaeger-agent.
+// UDPTransportParams allows specifying options for initializing a UDPTransport. An instance of this struct should
+// be passed to NewUDPTransportWithParams.
+type UDPTransportParams struct {
+	utils.AgentClientUDPParams
+}
+
+// NewUDPTransportWithParams creates a reporter that submits spans to jaeger-agent.
 // TODO: (breaking change) move to transport/ package.
-func NewUDPTransport(hostPort string, maxPacketSize int) (Transport, error) {
-	if len(hostPort) == 0 {
-		hostPort = fmt.Sprintf("%s:%d", DefaultUDPSpanServerHost, DefaultUDPSpanServerPort)
+func NewUDPTransportWithParams(params UDPTransportParams) (Transport, error) {
+	if len(params.HostPort) == 0 {
+		params.HostPort = fmt.Sprintf("%s:%d", DefaultUDPSpanServerHost, DefaultUDPSpanServerPort)
 	}
-	if maxPacketSize == 0 {
-		maxPacketSize = utils.UDPPacketMaxLength
+
+	if params.Logger == nil {
+		params.Logger = log.StdLogger
+	}
+
+	if params.MaxPacketSize == 0 {
+		params.MaxPacketSize = utils.UDPPacketMaxLength
 	}
 
 	protocolFactory := thrift.NewTCompactProtocolFactory()
 
 	// Each span is first written to thriftBuffer to determine its size in bytes.
-	thriftBuffer := thrift.NewTMemoryBufferLen(maxPacketSize)
+	thriftBuffer := thrift.NewTMemoryBufferLen(params.MaxPacketSize)
 	thriftProtocol := protocolFactory.GetProtocol(thriftBuffer)
 
-	client, err := utils.NewAgentClientUDP(hostPort, maxPacketSize)
+	client, err := utils.NewAgentClientUDPWithParams(params.AgentClientUDPParams)
 	if err != nil {
 		return nil, err
 	}
 
 	return &udpSender{
 		client:         client,
-		maxSpanBytes:   maxPacketSize - emitBatchOverhead,
+		maxSpanBytes:   params.MaxPacketSize - emitBatchOverhead,
 		thriftBuffer:   thriftBuffer,
 		thriftProtocol: thriftProtocol,
 	}, nil
 }
 
+// NewUDPTransport creates a reporter that submits spans to jaeger-agent.
+// TODO: (breaking change) move to transport/ package.
+func NewUDPTransport(hostPort string, maxPacketSize int) (Transport, error) {
+	return NewUDPTransportWithParams(UDPTransportParams{
+		AgentClientUDPParams: utils.AgentClientUDPParams{
+			HostPort:      hostPort,
+			MaxPacketSize: maxPacketSize,
+		},
+	})
+}
+
 // SetReporterStats implements reporterstats.Receiver.
 func (s *udpSender) SetReporterStats(rs reporterstats.ReporterStats) {
 	s.reporterStats = rs
@@ -93,7 +117,7 @@
 
 func (s *udpSender) calcSizeOfSerializedThrift(thriftStruct thrift.TStruct) int {
 	s.thriftBuffer.Reset()
-	_ = thriftStruct.Write(s.thriftProtocol)
+	_ = thriftStruct.Write(context.Background(), s.thriftProtocol)
 	return s.thriftBuffer.Len()
 }
 
@@ -132,7 +156,7 @@
 	}
 	s.batchSeqNo++
 	batchSeqNo := int64(s.batchSeqNo)
-	err := s.client.EmitBatch(&j.Batch{
+	err := s.client.EmitBatch(context.Background(), &j.Batch{
 		Process: s.process,
 		Spans:   s.spanBuffer,
 		SeqNo:   &batchSeqNo,
diff --git a/vendor/github.com/uber/jaeger-client-go/utils/reconnecting_udp_conn.go b/vendor/github.com/uber/jaeger-client-go/utils/reconnecting_udp_conn.go
new file mode 100644
index 0000000..0dffc7f
--- /dev/null
+++ b/vendor/github.com/uber/jaeger-client-go/utils/reconnecting_udp_conn.go
@@ -0,0 +1,189 @@
+// Copyright (c) 2020 The Jaeger Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package utils
+
+import (
+	"fmt"
+	"net"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"github.com/uber/jaeger-client-go/log"
+)
+
+// reconnectingUDPConn is an implementation of udpConn that resolves hostPort every resolveTimeout, if the resolved address is
+// different than the current conn then the new address is dialed and the conn is swapped.
+type reconnectingUDPConn struct {
+	hostPort    string
+	resolveFunc resolveFunc
+	dialFunc    dialFunc
+	logger      log.Logger
+	bufferBytes int64
+
+	connMtx   sync.RWMutex
+	conn      *net.UDPConn
+	destAddr  *net.UDPAddr
+	closeChan chan struct{}
+}
+
+type resolveFunc func(network string, hostPort string) (*net.UDPAddr, error)
+type dialFunc func(network string, laddr, raddr *net.UDPAddr) (*net.UDPConn, error)
+
+// newReconnectingUDPConn returns a new udpConn that resolves hostPort every resolveTimeout, if the resolved address is
+// different than the current conn then the new address is dialed and the conn is swapped.
+func newReconnectingUDPConn(hostPort string, resolveTimeout time.Duration, resolveFunc resolveFunc, dialFunc dialFunc, logger log.Logger) (*reconnectingUDPConn, error) {
+	conn := &reconnectingUDPConn{
+		hostPort:    hostPort,
+		resolveFunc: resolveFunc,
+		dialFunc:    dialFunc,
+		logger:      logger,
+		closeChan:   make(chan struct{}),
+	}
+
+	if err := conn.attemptResolveAndDial(); err != nil {
+		logger.Error(fmt.Sprintf("failed resolving destination address on connection startup, with err: %q. retrying in %s", err.Error(), resolveTimeout))
+	}
+
+	go conn.reconnectLoop(resolveTimeout)
+
+	return conn, nil
+}
+
+func (c *reconnectingUDPConn) reconnectLoop(resolveTimeout time.Duration) {
+	ticker := time.NewTicker(resolveTimeout)
+	defer ticker.Stop()
+
+	for {
+		select {
+		case <-c.closeChan:
+			return
+		case <-ticker.C:
+			if err := c.attemptResolveAndDial(); err != nil {
+				c.logger.Error(err.Error())
+			}
+		}
+	}
+}
+
+func (c *reconnectingUDPConn) attemptResolveAndDial() error {
+	newAddr, err := c.resolveFunc("udp", c.hostPort)
+	if err != nil {
+		return fmt.Errorf("failed to resolve new addr for host %q, with err: %w", c.hostPort, err)
+	}
+
+	c.connMtx.RLock()
+	curAddr := c.destAddr
+	c.connMtx.RUnlock()
+
+	// dont attempt dial if an addr was successfully dialed previously and, resolved addr is the same as current conn
+	if curAddr != nil && newAddr.String() == curAddr.String() {
+		return nil
+	}
+
+	if err := c.attemptDialNewAddr(newAddr); err != nil {
+		return fmt.Errorf("failed to dial newly resolved addr '%s', with err: %w", newAddr, err)
+	}
+
+	return nil
+}
+
+func (c *reconnectingUDPConn) attemptDialNewAddr(newAddr *net.UDPAddr) error {
+	connUDP, err := c.dialFunc(newAddr.Network(), nil, newAddr)
+	if err != nil {
+		return err
+	}
+
+	if bufferBytes := int(atomic.LoadInt64(&c.bufferBytes)); bufferBytes != 0 {
+		if err = connUDP.SetWriteBuffer(bufferBytes); err != nil {
+			return err
+		}
+	}
+
+	c.connMtx.Lock()
+	c.destAddr = newAddr
+	// store prev to close later
+	prevConn := c.conn
+	c.conn = connUDP
+	c.connMtx.Unlock()
+
+	if prevConn != nil {
+		return prevConn.Close()
+	}
+
+	return nil
+}
+
+// Write calls net.udpConn.Write, if it fails an attempt is made to connect to a new addr, if that succeeds the write is retried before returning
+func (c *reconnectingUDPConn) Write(b []byte) (int, error) {
+	var bytesWritten int
+	var err error
+
+	c.connMtx.RLock()
+	if c.conn == nil {
+		// if connection is not initialized indicate this with err in order to hook into retry logic
+		err = fmt.Errorf("UDP connection not yet initialized, an address has not been resolved")
+	} else {
+		bytesWritten, err = c.conn.Write(b)
+	}
+	c.connMtx.RUnlock()
+
+	if err == nil {
+		return bytesWritten, nil
+	}
+
+	// attempt to resolve and dial new address in case that's the problem, if resolve and dial succeeds, try write again
+	if reconnErr := c.attemptResolveAndDial(); reconnErr == nil {
+		c.connMtx.RLock()
+		defer c.connMtx.RUnlock()
+		return c.conn.Write(b)
+	}
+
+	// return original error if reconn fails
+	return bytesWritten, err
+}
+
+// Close stops the reconnectLoop, then closes the connection via net.udpConn 's implementation
+func (c *reconnectingUDPConn) Close() error {
+	close(c.closeChan)
+
+	// acquire rw lock before closing conn to ensure calls to Write drain
+	c.connMtx.Lock()
+	defer c.connMtx.Unlock()
+
+	if c.conn != nil {
+		return c.conn.Close()
+	}
+
+	return nil
+}
+
+// SetWriteBuffer defers to the net.udpConn SetWriteBuffer implementation wrapped with a RLock. if no conn is currently held
+// and SetWriteBuffer is called store bufferBytes to be set for new conns
+func (c *reconnectingUDPConn) SetWriteBuffer(bytes int) error {
+	var err error
+
+	c.connMtx.RLock()
+	if c.conn != nil {
+		err = c.conn.SetWriteBuffer(bytes)
+	}
+	c.connMtx.RUnlock()
+
+	if err == nil {
+		atomic.StoreInt64(&c.bufferBytes, int64(bytes))
+	}
+
+	return err
+}
diff --git a/vendor/github.com/uber/jaeger-client-go/utils/udp_client.go b/vendor/github.com/uber/jaeger-client-go/utils/udp_client.go
index fadd73e..4c59ae9 100644
--- a/vendor/github.com/uber/jaeger-client-go/utils/udp_client.go
+++ b/vendor/github.com/uber/jaeger-client-go/utils/udp_client.go
@@ -15,11 +15,14 @@
 package utils
 
 import (
+	"context"
 	"errors"
 	"fmt"
 	"io"
 	"net"
+	"time"
 
+	"github.com/uber/jaeger-client-go/log"
 	"github.com/uber/jaeger-client-go/thrift"
 
 	"github.com/uber/jaeger-client-go/thrift-gen/agent"
@@ -35,53 +38,101 @@
 	agent.Agent
 	io.Closer
 
-	connUDP       *net.UDPConn
+	connUDP       udpConn
 	client        *agent.AgentClient
 	maxPacketSize int                   // max size of datagram in bytes
 	thriftBuffer  *thrift.TMemoryBuffer // buffer used to calculate byte size of a span
 }
 
-// NewAgentClientUDP creates a client that sends spans to Jaeger Agent over UDP.
-func NewAgentClientUDP(hostPort string, maxPacketSize int) (*AgentClientUDP, error) {
-	if maxPacketSize == 0 {
-		maxPacketSize = UDPPacketMaxLength
+type udpConn interface {
+	Write([]byte) (int, error)
+	SetWriteBuffer(int) error
+	Close() error
+}
+
+// AgentClientUDPParams allows specifying options for initializing an AgentClientUDP. An instance of this struct should
+// be passed to NewAgentClientUDPWithParams.
+type AgentClientUDPParams struct {
+	HostPort                   string
+	MaxPacketSize              int
+	Logger                     log.Logger
+	DisableAttemptReconnecting bool
+	AttemptReconnectInterval   time.Duration
+}
+
+// NewAgentClientUDPWithParams creates a client that sends spans to Jaeger Agent over UDP.
+func NewAgentClientUDPWithParams(params AgentClientUDPParams) (*AgentClientUDP, error) {
+	// validate hostport
+	if _, _, err := net.SplitHostPort(params.HostPort); err != nil {
+		return nil, err
 	}
 
-	thriftBuffer := thrift.NewTMemoryBufferLen(maxPacketSize)
+	if params.MaxPacketSize == 0 {
+		params.MaxPacketSize = UDPPacketMaxLength
+	}
+
+	if params.Logger == nil {
+		params.Logger = log.StdLogger
+	}
+
+	if !params.DisableAttemptReconnecting && params.AttemptReconnectInterval == 0 {
+		params.AttemptReconnectInterval = time.Second * 30
+	}
+
+	thriftBuffer := thrift.NewTMemoryBufferLen(params.MaxPacketSize)
 	protocolFactory := thrift.NewTCompactProtocolFactory()
 	client := agent.NewAgentClientFactory(thriftBuffer, protocolFactory)
 
-	destAddr, err := net.ResolveUDPAddr("udp", hostPort)
-	if err != nil {
+	var connUDP udpConn
+	var err error
+
+	if params.DisableAttemptReconnecting {
+		destAddr, err := net.ResolveUDPAddr("udp", params.HostPort)
+		if err != nil {
+			return nil, err
+		}
+
+		connUDP, err = net.DialUDP(destAddr.Network(), nil, destAddr)
+		if err != nil {
+			return nil, err
+		}
+	} else {
+		// host is hostname, setup resolver loop in case host record changes during operation
+		connUDP, err = newReconnectingUDPConn(params.HostPort, params.AttemptReconnectInterval, net.ResolveUDPAddr, net.DialUDP, params.Logger)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if err := connUDP.SetWriteBuffer(params.MaxPacketSize); err != nil {
 		return nil, err
 	}
 
-	connUDP, err := net.DialUDP(destAddr.Network(), nil, destAddr)
-	if err != nil {
-		return nil, err
-	}
-	if err := connUDP.SetWriteBuffer(maxPacketSize); err != nil {
-		return nil, err
-	}
-
-	clientUDP := &AgentClientUDP{
+	return &AgentClientUDP{
 		connUDP:       connUDP,
 		client:        client,
-		maxPacketSize: maxPacketSize,
-		thriftBuffer:  thriftBuffer}
-	return clientUDP, nil
+		maxPacketSize: params.MaxPacketSize,
+		thriftBuffer:  thriftBuffer,
+	}, nil
+}
+
+// NewAgentClientUDP creates a client that sends spans to Jaeger Agent over UDP.
+func NewAgentClientUDP(hostPort string, maxPacketSize int) (*AgentClientUDP, error) {
+	return NewAgentClientUDPWithParams(AgentClientUDPParams{
+		HostPort:      hostPort,
+		MaxPacketSize: maxPacketSize,
+	})
 }
 
 // EmitZipkinBatch implements EmitZipkinBatch() of Agent interface
-func (a *AgentClientUDP) EmitZipkinBatch(spans []*zipkincore.Span) error {
+func (a *AgentClientUDP) EmitZipkinBatch(context.Context, []*zipkincore.Span) error {
 	return errors.New("Not implemented")
 }
 
 // EmitBatch implements EmitBatch() of Agent interface
-func (a *AgentClientUDP) EmitBatch(batch *jaeger.Batch) error {
+func (a *AgentClientUDP) EmitBatch(ctx context.Context, batch *jaeger.Batch) error {
 	a.thriftBuffer.Reset()
-	a.client.SeqId = 0 // we have no need for distinct SeqIds for our one-way UDP messages
-	if err := a.client.EmitBatch(batch); err != nil {
+	if err := a.client.EmitBatch(ctx, batch); err != nil {
 		return err
 	}
 	if a.thriftBuffer.Len() > a.maxPacketSize {
diff --git a/vendor/go.etcd.io/etcd/NOTICE b/vendor/go.etcd.io/etcd/NOTICE
new file mode 100644
index 0000000..b39ddfa
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/NOTICE
@@ -0,0 +1,5 @@
+CoreOS Project
+Copyright 2014 CoreOS, Inc
+
+This product includes software developed at CoreOS, Inc.
+(http://www.coreos.com/).
diff --git a/vendor/go.etcd.io/etcd/auth/authpb/auth.pb.go b/vendor/go.etcd.io/etcd/auth/authpb/auth.pb.go
deleted file mode 100644
index 7e038df..0000000
--- a/vendor/go.etcd.io/etcd/auth/authpb/auth.pb.go
+++ /dev/null
@@ -1,977 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: auth.proto
-
-/*
-	Package authpb is a generated protocol buffer package.
-
-	It is generated from these files:
-		auth.proto
-
-	It has these top-level messages:
-		UserAddOptions
-		User
-		Permission
-		Role
-*/
-package authpb
-
-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 Permission_Type int32
-
-const (
-	READ      Permission_Type = 0
-	WRITE     Permission_Type = 1
-	READWRITE Permission_Type = 2
-)
-
-var Permission_Type_name = map[int32]string{
-	0: "READ",
-	1: "WRITE",
-	2: "READWRITE",
-}
-var Permission_Type_value = map[string]int32{
-	"READ":      0,
-	"WRITE":     1,
-	"READWRITE": 2,
-}
-
-func (x Permission_Type) String() string {
-	return proto.EnumName(Permission_Type_name, int32(x))
-}
-func (Permission_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorAuth, []int{2, 0} }
-
-type UserAddOptions struct {
-	NoPassword bool `protobuf:"varint,1,opt,name=no_password,json=noPassword,proto3" json:"no_password,omitempty"`
-}
-
-func (m *UserAddOptions) Reset()                    { *m = UserAddOptions{} }
-func (m *UserAddOptions) String() string            { return proto.CompactTextString(m) }
-func (*UserAddOptions) ProtoMessage()               {}
-func (*UserAddOptions) Descriptor() ([]byte, []int) { return fileDescriptorAuth, []int{0} }
-
-// User is a single entry in the bucket authUsers
-type User struct {
-	Name     []byte          `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Password []byte          `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
-	Roles    []string        `protobuf:"bytes,3,rep,name=roles" json:"roles,omitempty"`
-	Options  *UserAddOptions `protobuf:"bytes,4,opt,name=options" json:"options,omitempty"`
-}
-
-func (m *User) Reset()                    { *m = User{} }
-func (m *User) String() string            { return proto.CompactTextString(m) }
-func (*User) ProtoMessage()               {}
-func (*User) Descriptor() ([]byte, []int) { return fileDescriptorAuth, []int{1} }
-
-// Permission is a single entity
-type Permission struct {
-	PermType Permission_Type `protobuf:"varint,1,opt,name=permType,proto3,enum=authpb.Permission_Type" json:"permType,omitempty"`
-	Key      []byte          `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
-	RangeEnd []byte          `protobuf:"bytes,3,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-}
-
-func (m *Permission) Reset()                    { *m = Permission{} }
-func (m *Permission) String() string            { return proto.CompactTextString(m) }
-func (*Permission) ProtoMessage()               {}
-func (*Permission) Descriptor() ([]byte, []int) { return fileDescriptorAuth, []int{2} }
-
-// Role is a single entry in the bucket authRoles
-type Role struct {
-	Name          []byte        `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	KeyPermission []*Permission `protobuf:"bytes,2,rep,name=keyPermission" json:"keyPermission,omitempty"`
-}
-
-func (m *Role) Reset()                    { *m = Role{} }
-func (m *Role) String() string            { return proto.CompactTextString(m) }
-func (*Role) ProtoMessage()               {}
-func (*Role) Descriptor() ([]byte, []int) { return fileDescriptorAuth, []int{3} }
-
-func init() {
-	proto.RegisterType((*UserAddOptions)(nil), "authpb.UserAddOptions")
-	proto.RegisterType((*User)(nil), "authpb.User")
-	proto.RegisterType((*Permission)(nil), "authpb.Permission")
-	proto.RegisterType((*Role)(nil), "authpb.Role")
-	proto.RegisterEnum("authpb.Permission_Type", Permission_Type_name, Permission_Type_value)
-}
-func (m *UserAddOptions) 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 *UserAddOptions) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.NoPassword {
-		dAtA[i] = 0x8
-		i++
-		if m.NoPassword {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *User) 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 *User) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Password) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(len(m.Password)))
-		i += copy(dAtA[i:], m.Password)
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			dAtA[i] = 0x1a
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	if m.Options != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(m.Options.Size()))
-		n1, err := m.Options.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n1
-	}
-	return i, nil
-}
-
-func (m *Permission) 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 *Permission) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.PermType != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(m.PermType))
-	}
-	if len(m.Key) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	return i, nil
-}
-
-func (m *Role) 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 *Role) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintAuth(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.KeyPermission) > 0 {
-		for _, msg := range m.KeyPermission {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintAuth(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func encodeVarintAuth(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 *UserAddOptions) Size() (n int) {
-	var l int
-	_ = l
-	if m.NoPassword {
-		n += 2
-	}
-	return n
-}
-
-func (m *User) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	l = len(m.Password)
-	if l > 0 {
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			l = len(s)
-			n += 1 + l + sovAuth(uint64(l))
-		}
-	}
-	if m.Options != nil {
-		l = m.Options.Size()
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	return n
-}
-
-func (m *Permission) Size() (n int) {
-	var l int
-	_ = l
-	if m.PermType != 0 {
-		n += 1 + sovAuth(uint64(m.PermType))
-	}
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	return n
-}
-
-func (m *Role) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovAuth(uint64(l))
-	}
-	if len(m.KeyPermission) > 0 {
-		for _, e := range m.KeyPermission {
-			l = e.Size()
-			n += 1 + l + sovAuth(uint64(l))
-		}
-	}
-	return n
-}
-
-func sovAuth(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozAuth(x uint64) (n int) {
-	return sovAuth(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *UserAddOptions) 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 ErrIntOverflowAuth
-			}
-			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: UserAddOptions: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: UserAddOptions: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field NoPassword", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.NoPassword = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipAuth(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthAuth
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *User) 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 ErrIntOverflowAuth
-			}
-			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: User: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: User: 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 ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			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 Password", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Password = append(m.Password[:0], dAtA[iNdEx:postIndex]...)
-			if m.Password == nil {
-				m.Password = []byte{}
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Options == nil {
-				m.Options = &UserAddOptions{}
-			}
-			if err := m.Options.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipAuth(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthAuth
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Permission) 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 ErrIntOverflowAuth
-			}
-			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: Permission: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Permission: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PermType", wireType)
-			}
-			m.PermType = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.PermType |= (Permission_Type(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		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 ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			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 != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipAuth(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthAuth
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Role) 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 ErrIntOverflowAuth
-			}
-			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: Role: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Role: 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 ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			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 KeyPermission", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowAuth
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthAuth
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.KeyPermission = append(m.KeyPermission, &Permission{})
-			if err := m.KeyPermission[len(m.KeyPermission)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipAuth(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthAuth
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func skipAuth(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, ErrIntOverflowAuth
-			}
-			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, ErrIntOverflowAuth
-				}
-				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, ErrIntOverflowAuth
-				}
-				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, ErrInvalidLengthAuth
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowAuth
-					}
-					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 := skipAuth(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 (
-	ErrInvalidLengthAuth = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowAuth   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("auth.proto", fileDescriptorAuth) }
-
-var fileDescriptorAuth = []byte{
-	// 338 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x91, 0xcf, 0x4e, 0xea, 0x40,
-	0x14, 0xc6, 0x3b, 0xb4, 0x70, 0xdb, 0xc3, 0x85, 0x90, 0x13, 0x72, 0x6f, 0x83, 0x49, 0x6d, 0xba,
-	0x6a, 0x5c, 0x54, 0x85, 0x8d, 0x5b, 0x8c, 0x2c, 0x5c, 0x49, 0x26, 0x18, 0x97, 0xa4, 0xa4, 0x13,
-	0x24, 0xc0, 0x4c, 0x33, 0x83, 0x31, 0x6c, 0x7c, 0x0e, 0x17, 0x3e, 0x10, 0x4b, 0x1e, 0x41, 0xf0,
-	0x45, 0x4c, 0x67, 0xf8, 0x13, 0xa2, 0xbb, 0xef, 0x7c, 0xe7, 0xfb, 0x66, 0x7e, 0x99, 0x01, 0x48,
-	0x5f, 0x16, 0xcf, 0x49, 0x2e, 0xc5, 0x42, 0x60, 0xa5, 0xd0, 0xf9, 0xa8, 0xd5, 0x1c, 0x8b, 0xb1,
-	0xd0, 0xd6, 0x65, 0xa1, 0xcc, 0x36, 0xba, 0x86, 0xfa, 0xa3, 0x62, 0xb2, 0x9b, 0x65, 0x0f, 0xf9,
-	0x62, 0x22, 0xb8, 0xc2, 0x73, 0xa8, 0x72, 0x31, 0xcc, 0x53, 0xa5, 0x5e, 0x85, 0xcc, 0x7c, 0x12,
-	0x92, 0xd8, 0xa5, 0xc0, 0x45, 0x7f, 0xe7, 0x44, 0x6f, 0xe0, 0x14, 0x15, 0x44, 0x70, 0x78, 0x3a,
-	0x67, 0x3a, 0xf1, 0x97, 0x6a, 0x8d, 0x2d, 0x70, 0x0f, 0xcd, 0x92, 0xf6, 0x0f, 0x33, 0x36, 0xa1,
-	0x2c, 0xc5, 0x8c, 0x29, 0xdf, 0x0e, 0xed, 0xd8, 0xa3, 0x66, 0xc0, 0x2b, 0xf8, 0x23, 0xcc, 0xcd,
-	0xbe, 0x13, 0x92, 0xb8, 0xda, 0xfe, 0x97, 0x18, 0xe0, 0xe4, 0x94, 0x8b, 0xee, 0x63, 0xd1, 0x07,
-	0x01, 0xe8, 0x33, 0x39, 0x9f, 0x28, 0x35, 0x11, 0x1c, 0x3b, 0xe0, 0xe6, 0x4c, 0xce, 0x07, 0xcb,
-	0xdc, 0xa0, 0xd4, 0xdb, 0xff, 0xf7, 0x27, 0x1c, 0x53, 0x49, 0xb1, 0xa6, 0x87, 0x20, 0x36, 0xc0,
-	0x9e, 0xb2, 0xe5, 0x0e, 0xb1, 0x90, 0x78, 0x06, 0x9e, 0x4c, 0xf9, 0x98, 0x0d, 0x19, 0xcf, 0x7c,
-	0xdb, 0xa0, 0x6b, 0xa3, 0xc7, 0xb3, 0xe8, 0x02, 0x1c, 0x5d, 0x73, 0xc1, 0xa1, 0xbd, 0xee, 0x5d,
-	0xc3, 0x42, 0x0f, 0xca, 0x4f, 0xf4, 0x7e, 0xd0, 0x6b, 0x10, 0xac, 0x81, 0x57, 0x98, 0x66, 0x2c,
-	0x45, 0x03, 0x70, 0xa8, 0x98, 0xb1, 0x5f, 0x9f, 0xe7, 0x06, 0x6a, 0x53, 0xb6, 0x3c, 0x62, 0xf9,
-	0xa5, 0xd0, 0x8e, 0xab, 0x6d, 0xfc, 0x09, 0x4c, 0x4f, 0x83, 0xb7, 0xfe, 0x6a, 0x13, 0x58, 0xeb,
-	0x4d, 0x60, 0xad, 0xb6, 0x01, 0x59, 0x6f, 0x03, 0xf2, 0xb9, 0x0d, 0xc8, 0xfb, 0x57, 0x60, 0x8d,
-	0x2a, 0xfa, 0x23, 0x3b, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x61, 0x66, 0xc6, 0x9d, 0xf4, 0x01,
-	0x00, 0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/auth/authpb/auth.proto b/vendor/go.etcd.io/etcd/auth/authpb/auth.proto
deleted file mode 100644
index 8f82b7c..0000000
--- a/vendor/go.etcd.io/etcd/auth/authpb/auth.proto
+++ /dev/null
@@ -1,42 +0,0 @@
-syntax = "proto3";
-package authpb;
-
-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;
-option (gogoproto.goproto_enum_prefix_all) = false;
-
-message UserAddOptions {
-  bool no_password = 1;
-};
-
-// User is a single entry in the bucket authUsers
-message User {
-  bytes name = 1;
-  bytes password = 2;
-  repeated string roles = 3;
-  UserAddOptions options = 4;
-}
-
-// Permission is a single entity
-message Permission {
-  enum Type {
-    READ = 0;
-    WRITE = 1;
-    READWRITE = 2;
-  }
-  Type permType = 1;
-
-  bytes key = 2;
-  bytes range_end = 3;
-}
-
-// Role is a single entry in the bucket authRoles
-message Role {
-  bytes name = 1;
-
-  repeated Permission keyPermission = 2;
-}
diff --git a/vendor/go.etcd.io/etcd/clientv3/README.md b/vendor/go.etcd.io/etcd/clientv3/README.md
deleted file mode 100644
index 6c6fe7c..0000000
--- a/vendor/go.etcd.io/etcd/clientv3/README.md
+++ /dev/null
@@ -1,85 +0,0 @@
-# etcd/clientv3
-
-[![Docs](https://img.shields.io/badge/docs-latest-green.svg)](https://etcd.io/docs)
-[![Godoc](https://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](https://godoc.org/go.etcd.io/etcd/clientv3)
-
-`etcd/clientv3` is the official Go etcd client for v3.
-
-## Install
-
-```bash
-go get go.etcd.io/etcd/clientv3
-```
-
-## Get started
-
-Create client using `clientv3.New`:
-
-```go
-cli, err := clientv3.New(clientv3.Config{
-	Endpoints:   []string{"localhost:2379", "localhost:22379", "localhost:32379"},
-	DialTimeout: 5 * time.Second,
-})
-if err != nil {
-	// handle error!
-}
-defer cli.Close()
-```
-
-etcd v3 uses [`gRPC`](https://www.grpc.io) for remote procedure calls. And `clientv3` uses
-[`grpc-go`](https://github.com/grpc/grpc-go) to connect to etcd. Make sure to close the client after using it.
-If the client is not closed, the connection will have leaky goroutines. To specify client request timeout,
-pass `context.WithTimeout` to APIs:
-
-```go
-ctx, cancel := context.WithTimeout(context.Background(), timeout)
-resp, err := cli.Put(ctx, "sample_key", "sample_value")
-cancel()
-if err != nil {
-    // handle error!
-}
-// use the response
-```
-
-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).
-
-## Error Handling
-
-etcd client returns 2 types of errors:
-
-1. context error: canceled or deadline exceeded.
-2. gRPC error: see [api/v3rpc/rpctypes](https://godoc.org/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes).
-
-Here is the example code to handle client errors:
-
-```go
-resp, err := cli.Put(ctx, "", "")
-if err != nil {
-	switch err {
-	case context.Canceled:
-		log.Fatalf("ctx is canceled by another routine: %v", err)
-	case context.DeadlineExceeded:
-		log.Fatalf("ctx is attached with a deadline is exceeded: %v", err)
-	case rpctypes.ErrEmptyKey:
-		log.Fatalf("client-side error: %v", err)
-	default:
-		log.Fatalf("bad cluster endpoints, which are not etcd servers: %v", err)
-	}
-}
-```
-
-## Metrics
-
-The etcd client optionally exposes RPC metrics through [go-grpc-prometheus](https://github.com/grpc-ecosystem/go-grpc-prometheus). See the [examples](https://github.com/etcd-io/etcd/blob/master/clientv3/example_metrics_test.go).
-
-## Namespacing
-
-The [namespace](https://godoc.org/go.etcd.io/etcd/clientv3/namespace) package provides `clientv3` interface wrappers to transparently isolate client requests to a user-defined prefix.
-
-## Request size limit
-
-Client request size limit is configurable via `clientv3.Config.MaxCallSendMsgSize` and `MaxCallRecvMsgSize` in bytes. If none given, client request send limit defaults to 2 MiB including gRPC overhead bytes. And receive limit defaults to `math.MaxInt32`.
-
-## Examples
-
-More code examples can be found at [GoDoc](https://godoc.org/go.etcd.io/etcd/clientv3).
diff --git a/vendor/go.etcd.io/etcd/clientv3/auth.go b/vendor/go.etcd.io/etcd/clientv3/auth.go
index c954f1b..edccf1a 100644
--- a/vendor/go.etcd.io/etcd/clientv3/auth.go
+++ b/vendor/go.etcd.io/etcd/clientv3/auth.go
@@ -19,8 +19,9 @@
 	"fmt"
 	"strings"
 
-	"go.etcd.io/etcd/auth/authpb"
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/auth/authpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"google.golang.org/grpc"
 )
 
@@ -52,8 +53,6 @@
 	PermReadWrite = authpb.READWRITE
 )
 
-type UserAddOptions authpb.UserAddOptions
-
 type Auth interface {
 	// AuthEnable enables auth of an etcd cluster.
 	AuthEnable(ctx context.Context) (*AuthEnableResponse, error)
@@ -64,9 +63,6 @@
 	// UserAdd adds a new user to an etcd cluster.
 	UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error)
 
-	// UserAddWithOptions adds a new user to an etcd cluster with some options.
-	UserAddWithOptions(ctx context.Context, name string, password string, opt *UserAddOptions) (*AuthUserAddResponse, error)
-
 	// UserDelete deletes a user from an etcd cluster.
 	UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error)
 
@@ -104,75 +100,70 @@
 	RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error)
 }
 
-type authClient struct {
+type auth struct {
 	remote   pb.AuthClient
 	callOpts []grpc.CallOption
 }
 
 func NewAuth(c *Client) Auth {
-	api := &authClient{remote: RetryAuthClient(c)}
+	api := &auth{remote: RetryAuthClient(c)}
 	if c != nil {
 		api.callOpts = c.callOpts
 	}
 	return api
 }
 
-func (auth *authClient) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
+func (auth *auth) AuthEnable(ctx context.Context) (*AuthEnableResponse, error) {
 	resp, err := auth.remote.AuthEnable(ctx, &pb.AuthEnableRequest{}, auth.callOpts...)
 	return (*AuthEnableResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
+func (auth *auth) AuthDisable(ctx context.Context) (*AuthDisableResponse, error) {
 	resp, err := auth.remote.AuthDisable(ctx, &pb.AuthDisableRequest{}, auth.callOpts...)
 	return (*AuthDisableResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error) {
-	resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password, Options: &authpb.UserAddOptions{NoPassword: false}}, auth.callOpts...)
+func (auth *auth) UserAdd(ctx context.Context, name string, password string) (*AuthUserAddResponse, error) {
+	resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password}, auth.callOpts...)
 	return (*AuthUserAddResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserAddWithOptions(ctx context.Context, name string, password string, options *UserAddOptions) (*AuthUserAddResponse, error) {
-	resp, err := auth.remote.UserAdd(ctx, &pb.AuthUserAddRequest{Name: name, Password: password, Options: (*authpb.UserAddOptions)(options)}, auth.callOpts...)
-	return (*AuthUserAddResponse)(resp), toErr(ctx, err)
-}
-
-func (auth *authClient) UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error) {
+func (auth *auth) UserDelete(ctx context.Context, name string) (*AuthUserDeleteResponse, error) {
 	resp, err := auth.remote.UserDelete(ctx, &pb.AuthUserDeleteRequest{Name: name}, auth.callOpts...)
 	return (*AuthUserDeleteResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserChangePassword(ctx context.Context, name string, password string) (*AuthUserChangePasswordResponse, error) {
+func (auth *auth) UserChangePassword(ctx context.Context, name string, password string) (*AuthUserChangePasswordResponse, error) {
 	resp, err := auth.remote.UserChangePassword(ctx, &pb.AuthUserChangePasswordRequest{Name: name, Password: password}, auth.callOpts...)
 	return (*AuthUserChangePasswordResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserGrantRole(ctx context.Context, user string, role string) (*AuthUserGrantRoleResponse, error) {
+func (auth *auth) UserGrantRole(ctx context.Context, user string, role string) (*AuthUserGrantRoleResponse, error) {
 	resp, err := auth.remote.UserGrantRole(ctx, &pb.AuthUserGrantRoleRequest{User: user, Role: role}, auth.callOpts...)
 	return (*AuthUserGrantRoleResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserGet(ctx context.Context, name string) (*AuthUserGetResponse, error) {
+func (auth *auth) UserGet(ctx context.Context, name string) (*AuthUserGetResponse, error) {
 	resp, err := auth.remote.UserGet(ctx, &pb.AuthUserGetRequest{Name: name}, auth.callOpts...)
 	return (*AuthUserGetResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserList(ctx context.Context) (*AuthUserListResponse, error) {
+func (auth *auth) UserList(ctx context.Context) (*AuthUserListResponse, error) {
 	resp, err := auth.remote.UserList(ctx, &pb.AuthUserListRequest{}, auth.callOpts...)
 	return (*AuthUserListResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) UserRevokeRole(ctx context.Context, name string, role string) (*AuthUserRevokeRoleResponse, error) {
+func (auth *auth) UserRevokeRole(ctx context.Context, name string, role string) (*AuthUserRevokeRoleResponse, error) {
 	resp, err := auth.remote.UserRevokeRole(ctx, &pb.AuthUserRevokeRoleRequest{Name: name, Role: role}, auth.callOpts...)
 	return (*AuthUserRevokeRoleResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleAdd(ctx context.Context, name string) (*AuthRoleAddResponse, error) {
+func (auth *auth) RoleAdd(ctx context.Context, name string) (*AuthRoleAddResponse, error) {
 	resp, err := auth.remote.RoleAdd(ctx, &pb.AuthRoleAddRequest{Name: name}, auth.callOpts...)
 	return (*AuthRoleAddResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleGrantPermission(ctx context.Context, name string, key, rangeEnd string, permType PermissionType) (*AuthRoleGrantPermissionResponse, error) {
+func (auth *auth) RoleGrantPermission(ctx context.Context, name string, key, rangeEnd string, permType PermissionType) (*AuthRoleGrantPermissionResponse, error) {
 	perm := &authpb.Permission{
 		Key:      []byte(key),
 		RangeEnd: []byte(rangeEnd),
@@ -182,22 +173,22 @@
 	return (*AuthRoleGrantPermissionResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleGet(ctx context.Context, role string) (*AuthRoleGetResponse, error) {
+func (auth *auth) RoleGet(ctx context.Context, role string) (*AuthRoleGetResponse, error) {
 	resp, err := auth.remote.RoleGet(ctx, &pb.AuthRoleGetRequest{Role: role}, auth.callOpts...)
 	return (*AuthRoleGetResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleList(ctx context.Context) (*AuthRoleListResponse, error) {
+func (auth *auth) RoleList(ctx context.Context) (*AuthRoleListResponse, error) {
 	resp, err := auth.remote.RoleList(ctx, &pb.AuthRoleListRequest{}, auth.callOpts...)
 	return (*AuthRoleListResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleRevokePermission(ctx context.Context, role string, key, rangeEnd string) (*AuthRoleRevokePermissionResponse, error) {
-	resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: []byte(key), RangeEnd: []byte(rangeEnd)}, auth.callOpts...)
+func (auth *auth) RoleRevokePermission(ctx context.Context, role string, key, rangeEnd string) (*AuthRoleRevokePermissionResponse, error) {
+	resp, err := auth.remote.RoleRevokePermission(ctx, &pb.AuthRoleRevokePermissionRequest{Role: role, Key: key, RangeEnd: rangeEnd}, auth.callOpts...)
 	return (*AuthRoleRevokePermissionResponse)(resp), toErr(ctx, err)
 }
 
-func (auth *authClient) RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error) {
+func (auth *auth) RoleDelete(ctx context.Context, role string) (*AuthRoleDeleteResponse, error) {
 	resp, err := auth.remote.RoleDelete(ctx, &pb.AuthRoleDeleteRequest{Role: role}, auth.callOpts...)
 	return (*AuthRoleDeleteResponse)(resp), toErr(ctx, err)
 }
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/balancer.go b/vendor/go.etcd.io/etcd/clientv3/balancer/balancer.go
deleted file mode 100644
index d02a7ee..0000000
--- a/vendor/go.etcd.io/etcd/clientv3/balancer/balancer.go
+++ /dev/null
@@ -1,293 +0,0 @@
-// 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 balancer implements client balancer.
-package balancer
-
-import (
-	"strconv"
-	"sync"
-	"time"
-
-	"go.etcd.io/etcd/clientv3/balancer/connectivity"
-	"go.etcd.io/etcd/clientv3/balancer/picker"
-
-	"go.uber.org/zap"
-	"google.golang.org/grpc/balancer"
-	grpcconnectivity "google.golang.org/grpc/connectivity"
-	"google.golang.org/grpc/resolver"
-	_ "google.golang.org/grpc/resolver/dns"         // register DNS resolver
-	_ "google.golang.org/grpc/resolver/passthrough" // register passthrough resolver
-)
-
-// Config defines balancer configurations.
-type Config struct {
-	// Policy configures balancer policy.
-	Policy picker.Policy
-
-	// Picker implements gRPC picker.
-	// Leave empty if "Policy" field is not custom.
-	// TODO: currently custom policy is not supported.
-	// Picker picker.Picker
-
-	// Name defines an additional name for balancer.
-	// Useful for balancer testing to avoid register conflicts.
-	// If empty, defaults to policy name.
-	Name string
-
-	// Logger configures balancer logging.
-	// If nil, logs are discarded.
-	Logger *zap.Logger
-}
-
-// RegisterBuilder creates and registers a builder. Since this function calls balancer.Register, it
-// must be invoked at initialization time.
-func RegisterBuilder(cfg Config) {
-	bb := &builder{cfg}
-	balancer.Register(bb)
-
-	bb.cfg.Logger.Debug(
-		"registered balancer",
-		zap.String("policy", bb.cfg.Policy.String()),
-		zap.String("name", bb.cfg.Name),
-	)
-}
-
-type builder struct {
-	cfg Config
-}
-
-// Build is called initially when creating "ccBalancerWrapper".
-// "grpc.Dial" is called to this client connection.
-// Then, resolved addresses will be handled via "HandleResolvedAddrs".
-func (b *builder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
-	bb := &baseBalancer{
-		id:     strconv.FormatInt(time.Now().UnixNano(), 36),
-		policy: b.cfg.Policy,
-		name:   b.cfg.Name,
-		lg:     b.cfg.Logger,
-
-		addrToSc: make(map[resolver.Address]balancer.SubConn),
-		scToAddr: make(map[balancer.SubConn]resolver.Address),
-		scToSt:   make(map[balancer.SubConn]grpcconnectivity.State),
-
-		currentConn:          nil,
-		connectivityRecorder: connectivity.New(b.cfg.Logger),
-
-		// initialize picker always returns "ErrNoSubConnAvailable"
-		picker: picker.NewErr(balancer.ErrNoSubConnAvailable),
-	}
-
-	// TODO: support multiple connections
-	bb.mu.Lock()
-	bb.currentConn = cc
-	bb.mu.Unlock()
-
-	bb.lg.Info(
-		"built balancer",
-		zap.String("balancer-id", bb.id),
-		zap.String("policy", bb.policy.String()),
-		zap.String("resolver-target", cc.Target()),
-	)
-	return bb
-}
-
-// Name implements "grpc/balancer.Builder" interface.
-func (b *builder) Name() string { return b.cfg.Name }
-
-// Balancer defines client balancer interface.
-type Balancer interface {
-	// Balancer is called on specified client connection. Client initiates gRPC
-	// connection with "grpc.Dial(addr, grpc.WithBalancerName)", and then those resolved
-	// addresses are passed to "grpc/balancer.Balancer.HandleResolvedAddrs".
-	// For each resolved address, balancer calls "balancer.ClientConn.NewSubConn".
-	// "grpc/balancer.Balancer.HandleSubConnStateChange" is called when connectivity state
-	// changes, thus requires failover logic in this method.
-	balancer.Balancer
-
-	// Picker calls "Pick" for every client request.
-	picker.Picker
-}
-
-type baseBalancer struct {
-	id     string
-	policy picker.Policy
-	name   string
-	lg     *zap.Logger
-
-	mu sync.RWMutex
-
-	addrToSc map[resolver.Address]balancer.SubConn
-	scToAddr map[balancer.SubConn]resolver.Address
-	scToSt   map[balancer.SubConn]grpcconnectivity.State
-
-	currentConn          balancer.ClientConn
-	connectivityRecorder connectivity.Recorder
-
-	picker picker.Picker
-}
-
-// HandleResolvedAddrs implements "grpc/balancer.Balancer" interface.
-// gRPC sends initial or updated resolved addresses from "Build".
-func (bb *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
-	if err != nil {
-		bb.lg.Warn("HandleResolvedAddrs called with error", zap.String("balancer-id", bb.id), zap.Error(err))
-		return
-	}
-	bb.lg.Info("resolved",
-		zap.String("picker", bb.picker.String()),
-		zap.String("balancer-id", bb.id),
-		zap.Strings("addresses", addrsToStrings(addrs)),
-	)
-
-	bb.mu.Lock()
-	defer bb.mu.Unlock()
-
-	resolved := make(map[resolver.Address]struct{})
-	for _, addr := range addrs {
-		resolved[addr] = struct{}{}
-		if _, ok := bb.addrToSc[addr]; !ok {
-			sc, err := bb.currentConn.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{})
-			if err != nil {
-				bb.lg.Warn("NewSubConn failed", zap.String("picker", bb.picker.String()), zap.String("balancer-id", bb.id), zap.Error(err), zap.String("address", addr.Addr))
-				continue
-			}
-			bb.lg.Info("created subconn", zap.String("address", addr.Addr))
-			bb.addrToSc[addr] = sc
-			bb.scToAddr[sc] = addr
-			bb.scToSt[sc] = grpcconnectivity.Idle
-			sc.Connect()
-		}
-	}
-
-	for addr, sc := range bb.addrToSc {
-		if _, ok := resolved[addr]; !ok {
-			// was removed by resolver or failed to create subconn
-			bb.currentConn.RemoveSubConn(sc)
-			delete(bb.addrToSc, addr)
-
-			bb.lg.Info(
-				"removed subconn",
-				zap.String("picker", bb.picker.String()),
-				zap.String("balancer-id", bb.id),
-				zap.String("address", addr.Addr),
-				zap.String("subconn", scToString(sc)),
-			)
-
-			// Keep the state of this sc in bb.scToSt until sc's state becomes Shutdown.
-			// The entry will be deleted in HandleSubConnStateChange.
-			// (DO NOT) delete(bb.scToAddr, sc)
-			// (DO NOT) delete(bb.scToSt, sc)
-		}
-	}
-}
-
-// HandleSubConnStateChange implements "grpc/balancer.Balancer" interface.
-func (bb *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s grpcconnectivity.State) {
-	bb.mu.Lock()
-	defer bb.mu.Unlock()
-
-	old, ok := bb.scToSt[sc]
-	if !ok {
-		bb.lg.Warn(
-			"state change for an unknown subconn",
-			zap.String("picker", bb.picker.String()),
-			zap.String("balancer-id", bb.id),
-			zap.String("subconn", scToString(sc)),
-			zap.Int("subconn-size", len(bb.scToAddr)),
-			zap.String("state", s.String()),
-		)
-		return
-	}
-
-	bb.lg.Info(
-		"state changed",
-		zap.String("picker", bb.picker.String()),
-		zap.String("balancer-id", bb.id),
-		zap.Bool("connected", s == grpcconnectivity.Ready),
-		zap.String("subconn", scToString(sc)),
-		zap.Int("subconn-size", len(bb.scToAddr)),
-		zap.String("address", bb.scToAddr[sc].Addr),
-		zap.String("old-state", old.String()),
-		zap.String("new-state", s.String()),
-	)
-
-	bb.scToSt[sc] = s
-	switch s {
-	case grpcconnectivity.Idle:
-		sc.Connect()
-	case grpcconnectivity.Shutdown:
-		// When an address was removed by resolver, b called RemoveSubConn but
-		// kept the sc's state in scToSt. Remove state for this sc here.
-		delete(bb.scToAddr, sc)
-		delete(bb.scToSt, sc)
-	}
-
-	oldAggrState := bb.connectivityRecorder.GetCurrentState()
-	bb.connectivityRecorder.RecordTransition(old, s)
-
-	// Update balancer picker when one of the following happens:
-	//  - this sc became ready from not-ready
-	//  - this sc became not-ready from ready
-	//  - the aggregated state of balancer became TransientFailure from non-TransientFailure
-	//  - the aggregated state of balancer became non-TransientFailure from TransientFailure
-	if (s == grpcconnectivity.Ready) != (old == grpcconnectivity.Ready) ||
-		(bb.connectivityRecorder.GetCurrentState() == grpcconnectivity.TransientFailure) != (oldAggrState == grpcconnectivity.TransientFailure) {
-		bb.updatePicker()
-	}
-
-	bb.currentConn.UpdateBalancerState(bb.connectivityRecorder.GetCurrentState(), bb.picker)
-}
-
-func (bb *baseBalancer) updatePicker() {
-	if bb.connectivityRecorder.GetCurrentState() == grpcconnectivity.TransientFailure {
-		bb.picker = picker.NewErr(balancer.ErrTransientFailure)
-		bb.lg.Info(
-			"updated picker to transient error picker",
-			zap.String("picker", bb.picker.String()),
-			zap.String("balancer-id", bb.id),
-			zap.String("policy", bb.policy.String()),
-		)
-		return
-	}
-
-	// only pass ready subconns to picker
-	scToAddr := make(map[balancer.SubConn]resolver.Address)
-	for addr, sc := range bb.addrToSc {
-		if st, ok := bb.scToSt[sc]; ok && st == grpcconnectivity.Ready {
-			scToAddr[sc] = addr
-		}
-	}
-
-	bb.picker = picker.New(picker.Config{
-		Policy:                   bb.policy,
-		Logger:                   bb.lg,
-		SubConnToResolverAddress: scToAddr,
-	})
-	bb.lg.Info(
-		"updated picker",
-		zap.String("picker", bb.picker.String()),
-		zap.String("balancer-id", bb.id),
-		zap.String("policy", bb.policy.String()),
-		zap.Strings("subconn-ready", scsToStrings(scToAddr)),
-		zap.Int("subconn-size", len(scToAddr)),
-	)
-}
-
-// Close implements "grpc/balancer.Balancer" interface.
-// Close is a nop because base balancer doesn't have internal state to clean up,
-// and it doesn't need to call RemoveSubConn for the SubConns.
-func (bb *baseBalancer) Close() {
-	// TODO
-}
diff --git a/vendor/go.etcd.io/etcd/clientv3/balancer/resolver/endpoint/endpoint.go b/vendor/go.etcd.io/etcd/clientv3/balancer/resolver/endpoint/endpoint.go
deleted file mode 100644
index 1f32039..0000000
--- a/vendor/go.etcd.io/etcd/clientv3/balancer/resolver/endpoint/endpoint.go
+++ /dev/null
@@ -1,240 +0,0 @@
-// 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 endpoint resolves etcd entpoints using grpc targets of the form 'endpoint://<id>/<endpoint>'.
-package endpoint
-
-import (
-	"fmt"
-	"net/url"
-	"strings"
-	"sync"
-
-	"google.golang.org/grpc/resolver"
-)
-
-const scheme = "endpoint"
-
-var (
-	targetPrefix = fmt.Sprintf("%s://", scheme)
-
-	bldr *builder
-)
-
-func init() {
-	bldr = &builder{
-		resolverGroups: make(map[string]*ResolverGroup),
-	}
-	resolver.Register(bldr)
-}
-
-type builder struct {
-	mu             sync.RWMutex
-	resolverGroups map[string]*ResolverGroup
-}
-
-// NewResolverGroup creates a new ResolverGroup with the given id.
-func NewResolverGroup(id string) (*ResolverGroup, error) {
-	return bldr.newResolverGroup(id)
-}
-
-// ResolverGroup keeps all endpoints of resolvers using a common endpoint://<id>/ target
-// up-to-date.
-type ResolverGroup struct {
-	mu        sync.RWMutex
-	id        string
-	endpoints []string
-	resolvers []*Resolver
-}
-
-func (e *ResolverGroup) addResolver(r *Resolver) {
-	e.mu.Lock()
-	addrs := epsToAddrs(e.endpoints...)
-	e.resolvers = append(e.resolvers, r)
-	e.mu.Unlock()
-	r.cc.NewAddress(addrs)
-}
-
-func (e *ResolverGroup) removeResolver(r *Resolver) {
-	e.mu.Lock()
-	for i, er := range e.resolvers {
-		if er == r {
-			e.resolvers = append(e.resolvers[:i], e.resolvers[i+1:]...)
-			break
-		}
-	}
-	e.mu.Unlock()
-}
-
-// SetEndpoints updates the endpoints for ResolverGroup. All registered resolver are updated
-// immediately with the new endpoints.
-func (e *ResolverGroup) SetEndpoints(endpoints []string) {
-	addrs := epsToAddrs(endpoints...)
-	e.mu.Lock()
-	e.endpoints = endpoints
-	for _, r := range e.resolvers {
-		r.cc.NewAddress(addrs)
-	}
-	e.mu.Unlock()
-}
-
-// Target constructs a endpoint target using the endpoint id of the ResolverGroup.
-func (e *ResolverGroup) Target(endpoint string) string {
-	return Target(e.id, endpoint)
-}
-
-// Target constructs a endpoint resolver target.
-func Target(id, endpoint string) string {
-	return fmt.Sprintf("%s://%s/%s", scheme, id, endpoint)
-}
-
-// IsTarget checks if a given target string in an endpoint resolver target.
-func IsTarget(target string) bool {
-	return strings.HasPrefix(target, "endpoint://")
-}
-
-func (e *ResolverGroup) Close() {
-	bldr.close(e.id)
-}
-
-// Build creates or reuses an etcd resolver for the etcd cluster name identified by the authority part of the target.
-func (b *builder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) {
-	if len(target.Authority) < 1 {
-		return nil, fmt.Errorf("'etcd' target scheme requires non-empty authority identifying etcd cluster being routed to")
-	}
-	id := target.Authority
-	es, err := b.getResolverGroup(id)
-	if err != nil {
-		return nil, fmt.Errorf("failed to build resolver: %v", err)
-	}
-	r := &Resolver{
-		endpointID: id,
-		cc:         cc,
-	}
-	es.addResolver(r)
-	return r, nil
-}
-
-func (b *builder) newResolverGroup(id string) (*ResolverGroup, error) {
-	b.mu.RLock()
-	_, ok := b.resolverGroups[id]
-	b.mu.RUnlock()
-	if ok {
-		return nil, fmt.Errorf("Endpoint already exists for id: %s", id)
-	}
-
-	es := &ResolverGroup{id: id}
-	b.mu.Lock()
-	b.resolverGroups[id] = es
-	b.mu.Unlock()
-	return es, nil
-}
-
-func (b *builder) getResolverGroup(id string) (*ResolverGroup, error) {
-	b.mu.RLock()
-	es, ok := b.resolverGroups[id]
-	b.mu.RUnlock()
-	if !ok {
-		return nil, fmt.Errorf("ResolverGroup not found for id: %s", id)
-	}
-	return es, nil
-}
-
-func (b *builder) close(id string) {
-	b.mu.Lock()
-	delete(b.resolverGroups, id)
-	b.mu.Unlock()
-}
-
-func (b *builder) Scheme() string {
-	return scheme
-}
-
-// Resolver provides a resolver for a single etcd cluster, identified by name.
-type Resolver struct {
-	endpointID string
-	cc         resolver.ClientConn
-	sync.RWMutex
-}
-
-// TODO: use balancer.epsToAddrs
-func epsToAddrs(eps ...string) (addrs []resolver.Address) {
-	addrs = make([]resolver.Address, 0, len(eps))
-	for _, ep := range eps {
-		addrs = append(addrs, resolver.Address{Addr: ep})
-	}
-	return addrs
-}
-
-func (*Resolver) ResolveNow(o resolver.ResolveNowOption) {}
-
-func (r *Resolver) Close() {
-	es, err := bldr.getResolverGroup(r.endpointID)
-	if err != nil {
-		return
-	}
-	es.removeResolver(r)
-}
-
-// ParseEndpoint endpoint parses an endpoint of the form
-// (http|https)://<host>*|(unix|unixs)://<path>)
-// and returns a protocol ('tcp' or 'unix'),
-// host (or filepath if a unix socket),
-// scheme (http, https, unix, unixs).
-func ParseEndpoint(endpoint string) (proto string, host string, scheme string) {
-	proto = "tcp"
-	host = endpoint
-	url, uerr := url.Parse(endpoint)
-	if uerr != nil || !strings.Contains(endpoint, "://") {
-		return proto, host, scheme
-	}
-	scheme = url.Scheme
-
-	// strip scheme:// prefix since grpc dials by host
-	host = url.Host
-	switch url.Scheme {
-	case "http", "https":
-	case "unix", "unixs":
-		proto = "unix"
-		host = url.Host + url.Path
-	default:
-		proto, host = "", ""
-	}
-	return proto, host, scheme
-}
-
-// ParseTarget parses a endpoint://<id>/<endpoint> string and returns the parsed id and endpoint.
-// If the target is malformed, an error is returned.
-func ParseTarget(target string) (string, string, error) {
-	noPrefix := strings.TrimPrefix(target, targetPrefix)
-	if noPrefix == target {
-		return "", "", fmt.Errorf("malformed target, %s prefix is required: %s", targetPrefix, target)
-	}
-	parts := strings.SplitN(noPrefix, "/", 2)
-	if len(parts) != 2 {
-		return "", "", fmt.Errorf("malformed target, expected %s://<id>/<endpoint>, but got %s", scheme, target)
-	}
-	return parts[0], parts[1], nil
-}
-
-// ParseHostPort splits a "<host>:<port>" string into the host and port parts.
-// The port part is optional.
-func ParseHostPort(hostPort string) (host string, port string) {
-	parts := strings.SplitN(hostPort, ":", 2)
-	host = parts[0]
-	if len(parts) > 1 {
-		port = parts[1]
-	}
-	return host, port
-}
diff --git a/vendor/go.etcd.io/etcd/clientv3/client.go b/vendor/go.etcd.io/etcd/clientv3/client.go
index d6000a8..c49e4ba 100644
--- a/vendor/go.etcd.io/etcd/clientv3/client.go
+++ b/vendor/go.etcd.io/etcd/clientv3/client.go
@@ -25,19 +25,19 @@
 	"sync"
 	"time"
 
+	"github.com/coreos/etcd/clientv3/balancer"
+	"github.com/coreos/etcd/clientv3/balancer/picker"
+	"github.com/coreos/etcd/clientv3/balancer/resolver/endpoint"
+	"github.com/coreos/etcd/clientv3/credentials"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/pkg/logutil"
+	"github.com/coreos/pkg/capnslog"
 	"github.com/google/uuid"
-	"go.etcd.io/etcd/clientv3/balancer"
-	"go.etcd.io/etcd/clientv3/balancer/picker"
-	"go.etcd.io/etcd/clientv3/balancer/resolver/endpoint"
-	"go.etcd.io/etcd/clientv3/credentials"
-	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	"go.etcd.io/etcd/pkg/logutil"
 	"go.uber.org/zap"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 	grpccredentials "google.golang.org/grpc/credentials"
 	"google.golang.org/grpc/keepalive"
-	"google.golang.org/grpc/metadata"
 	"google.golang.org/grpc/status"
 )
 
@@ -48,6 +48,10 @@
 	roundRobinBalancerName = fmt.Sprintf("etcd-%s", picker.RoundrobinBalanced.String())
 )
 
+var (
+	plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "clientv3")
+)
+
 func init() {
 	lg := zap.NewNop()
 	if os.Getenv("ETCD_CLIENT_DEBUG") != "" {
@@ -129,12 +133,8 @@
 // Close shuts down the client's etcd connections.
 func (c *Client) Close() error {
 	c.cancel()
-	if c.Watcher != nil {
-		c.Watcher.Close()
-	}
-	if c.Lease != nil {
-		c.Lease.Close()
-	}
+	c.Watcher.Close()
+	c.Lease.Close()
 	if c.resolverGroup != nil {
 		c.resolverGroup.Close()
 	}
@@ -230,24 +230,17 @@
 	}
 	opts = append(opts, dopts...)
 
-	// Provide a net dialer that supports cancelation and timeout.
-	f := func(dialEp string, t time.Duration) (net.Conn, error) {
-		proto, host, _ := endpoint.ParseEndpoint(dialEp)
-		select {
-		case <-c.ctx.Done():
-			return nil, c.ctx.Err()
-		default:
-		}
-		dialer := &net.Dialer{Timeout: t}
-		return dialer.DialContext(c.ctx, proto, host)
-	}
-	opts = append(opts, grpc.WithDialer(f))
-
+	dialer := endpoint.Dialer
 	if creds != nil {
 		opts = append(opts, grpc.WithTransportCredentials(creds))
+		// gRPC load balancer workaround. See credentials.transportCredential for details.
+		if credsDialer, ok := creds.(TransportCredentialsWithDialer); ok {
+			dialer = credsDialer.Dialer
+		}
 	} else {
 		opts = append(opts, grpc.WithInsecure())
 	}
+	opts = append(opts, grpc.WithContextDialer(dialer))
 
 	// Interceptor retry and backoff.
 	// TODO: Replace all of clientv3/retry.go with interceptor based retry, or with
@@ -266,7 +259,10 @@
 
 // Dial connects to a single endpoint using the client's config.
 func (c *Client) Dial(ep string) (*grpc.ClientConn, error) {
-	creds := c.directDialCreds(ep)
+	creds, err := c.directDialCreds(ep)
+	if err != nil {
+		return nil, err
+	}
 	// Use the grpc passthrough resolver to directly dial a single endpoint.
 	// This resolver passes through the 'unix' and 'unixs' endpoints schemes used
 	// by etcd without modification, allowing us to directly dial endpoints and
@@ -369,8 +365,8 @@
 	return conn, nil
 }
 
-func (c *Client) directDialCreds(ep string) grpccredentials.TransportCredentials {
-	_, hostPort, scheme := endpoint.ParseEndpoint(ep)
+func (c *Client) directDialCreds(ep string) (grpccredentials.TransportCredentials, error) {
+	_, host, scheme := endpoint.ParseEndpoint(ep)
 	creds := c.creds
 	if len(scheme) != 0 {
 		creds = c.processCreds(scheme)
@@ -379,12 +375,17 @@
 			// Set the server name must to the endpoint hostname without port since grpc
 			// otherwise attempts to check if x509 cert is valid for the full endpoint
 			// including the scheme and port, which fails.
-			host, _ := endpoint.ParseHostPort(hostPort)
-			clone.OverrideServerName(host)
+			overrideServerName, _, err := net.SplitHostPort(host)
+			if err != nil {
+				// Either the host didn't have a port or the host could not be parsed. Either way, continue with the
+				// original host string.
+				overrideServerName = host
+			}
+			clone.OverrideServerName(overrideServerName)
 			creds = clone
 		}
 	}
-	return creds
+	return creds, nil
 }
 
 func (c *Client) dialWithBalancerCreds(ep string) grpccredentials.TransportCredentials {
@@ -396,13 +397,6 @@
 	return creds
 }
 
-// WithRequireLeader requires client requests to only succeed
-// when the cluster has a leader.
-func WithRequireLeader(ctx context.Context) context.Context {
-	md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
-	return metadata.NewOutgoingContext(ctx, md)
-}
-
 func newClient(cfg *Config) (*Client, error) {
 	if cfg == nil {
 		cfg = &Config{}
@@ -663,3 +657,9 @@
 	// <= gRPC v1.7.x returns 'errors.New("grpc: the client connection is closing")'
 	return strings.Contains(err.Error(), "grpc: the client connection is closing")
 }
+
+// TransportCredentialsWithDialer is for a gRPC load balancer workaround. See credentials.transportCredential for details.
+type TransportCredentialsWithDialer interface {
+	grpccredentials.TransportCredentials
+	Dialer(ctx context.Context, dialEp string) (net.Conn, error)
+}
diff --git a/vendor/go.etcd.io/etcd/clientv3/cluster.go b/vendor/go.etcd.io/etcd/clientv3/cluster.go
index ce97e5c..785672b 100644
--- a/vendor/go.etcd.io/etcd/clientv3/cluster.go
+++ b/vendor/go.etcd.io/etcd/clientv3/cluster.go
@@ -17,19 +17,18 @@
 import (
 	"context"
 
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
-	"go.etcd.io/etcd/pkg/types"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/pkg/types"
 
 	"google.golang.org/grpc"
 )
 
 type (
-	Member                pb.Member
-	MemberListResponse    pb.MemberListResponse
-	MemberAddResponse     pb.MemberAddResponse
-	MemberRemoveResponse  pb.MemberRemoveResponse
-	MemberUpdateResponse  pb.MemberUpdateResponse
-	MemberPromoteResponse pb.MemberPromoteResponse
+	Member               pb.Member
+	MemberListResponse   pb.MemberListResponse
+	MemberAddResponse    pb.MemberAddResponse
+	MemberRemoveResponse pb.MemberRemoveResponse
+	MemberUpdateResponse pb.MemberUpdateResponse
 )
 
 type Cluster interface {
@@ -39,17 +38,11 @@
 	// MemberAdd adds a new member into the cluster.
 	MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error)
 
-	// MemberAddAsLearner adds a new learner member into the cluster.
-	MemberAddAsLearner(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error)
-
 	// MemberRemove removes an existing member from the cluster.
 	MemberRemove(ctx context.Context, id uint64) (*MemberRemoveResponse, error)
 
 	// MemberUpdate updates the peer addresses of the member.
 	MemberUpdate(ctx context.Context, id uint64, peerAddrs []string) (*MemberUpdateResponse, error)
-
-	// MemberPromote promotes a member from raft learner (non-voting) to raft voting member.
-	MemberPromote(ctx context.Context, id uint64) (*MemberPromoteResponse, error)
 }
 
 type cluster struct {
@@ -74,23 +67,12 @@
 }
 
 func (c *cluster) MemberAdd(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error) {
-	return c.memberAdd(ctx, peerAddrs, false)
-}
-
-func (c *cluster) MemberAddAsLearner(ctx context.Context, peerAddrs []string) (*MemberAddResponse, error) {
-	return c.memberAdd(ctx, peerAddrs, true)
-}
-
-func (c *cluster) memberAdd(ctx context.Context, peerAddrs []string, isLearner bool) (*MemberAddResponse, error) {
 	// fail-fast before panic in rafthttp
 	if _, err := types.NewURLs(peerAddrs); err != nil {
 		return nil, err
 	}
 
-	r := &pb.MemberAddRequest{
-		PeerURLs:  peerAddrs,
-		IsLearner: isLearner,
-	}
+	r := &pb.MemberAddRequest{PeerURLs: peerAddrs}
 	resp, err := c.remote.MemberAdd(ctx, r, c.callOpts...)
 	if err != nil {
 		return nil, toErr(ctx, err)
@@ -130,12 +112,3 @@
 	}
 	return nil, toErr(ctx, err)
 }
-
-func (c *cluster) MemberPromote(ctx context.Context, id uint64) (*MemberPromoteResponse, error) {
-	r := &pb.MemberPromoteRequest{ID: id}
-	resp, err := c.remote.MemberPromote(ctx, r, c.callOpts...)
-	if err != nil {
-		return nil, toErr(ctx, err)
-	}
-	return (*MemberPromoteResponse)(resp), nil
-}
diff --git a/vendor/go.etcd.io/etcd/clientv3/compact_op.go b/vendor/go.etcd.io/etcd/clientv3/compact_op.go
index 5779713..41e80c1 100644
--- a/vendor/go.etcd.io/etcd/clientv3/compact_op.go
+++ b/vendor/go.etcd.io/etcd/clientv3/compact_op.go
@@ -15,7 +15,7 @@
 package clientv3
 
 import (
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 )
 
 // CompactOp represents a compact operation.
diff --git a/vendor/go.etcd.io/etcd/clientv3/compare.go b/vendor/go.etcd.io/etcd/clientv3/compare.go
index 01ed68e..b5f0a25 100644
--- a/vendor/go.etcd.io/etcd/clientv3/compare.go
+++ b/vendor/go.etcd.io/etcd/clientv3/compare.go
@@ -15,7 +15,7 @@
 package clientv3
 
 import (
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 )
 
 type CompareTarget int
diff --git a/vendor/go.etcd.io/etcd/clientv3/config.go b/vendor/go.etcd.io/etcd/clientv3/config.go
index 11d447d..9c17fc2 100644
--- a/vendor/go.etcd.io/etcd/clientv3/config.go
+++ b/vendor/go.etcd.io/etcd/clientv3/config.go
@@ -72,17 +72,15 @@
 	// Without this, Dial returns immediately and connecting the server happens in background.
 	DialOptions []grpc.DialOption
 
-	// Context is the default client context; it can be used to cancel grpc dial out and
-	// other operations that do not have an explicit context.
-	Context context.Context
-
 	// LogConfig configures client-side logger.
 	// If nil, use the default logger.
 	// TODO: configure gRPC logger
 	LogConfig *zap.Config
 
+	// Context is the default client context; it can be used to cancel grpc dial out and
+	// other operations that do not have an explicit context.
+	Context context.Context
+
 	// PermitWithoutStream when set will allow client to send keepalive pings to server without any active streams(RPCs).
 	PermitWithoutStream bool `json:"permit-without-stream"`
-
-	// TODO: support custom balancer picker
 }
diff --git a/vendor/go.etcd.io/etcd/clientv3/credentials/credentials.go b/vendor/go.etcd.io/etcd/clientv3/credentials/credentials.go
deleted file mode 100644
index e6fd75c..0000000
--- a/vendor/go.etcd.io/etcd/clientv3/credentials/credentials.go
+++ /dev/null
@@ -1,155 +0,0 @@
-// 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 credentials implements gRPC credential interface with etcd specific logic.
-// e.g., client handshake with custom authority parameter
-package credentials
-
-import (
-	"context"
-	"crypto/tls"
-	"net"
-	"sync"
-
-	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	grpccredentials "google.golang.org/grpc/credentials"
-)
-
-// Config defines gRPC credential configuration.
-type Config struct {
-	TLSConfig *tls.Config
-}
-
-// Bundle defines gRPC credential interface.
-type Bundle interface {
-	grpccredentials.Bundle
-	UpdateAuthToken(token string)
-}
-
-// NewBundle constructs a new gRPC credential bundle.
-func NewBundle(cfg Config) Bundle {
-	return &bundle{
-		tc: newTransportCredential(cfg.TLSConfig),
-		rc: newPerRPCCredential(),
-	}
-}
-
-// bundle implements "grpccredentials.Bundle" interface.
-type bundle struct {
-	tc *transportCredential
-	rc *perRPCCredential
-}
-
-func (b *bundle) TransportCredentials() grpccredentials.TransportCredentials {
-	return b.tc
-}
-
-func (b *bundle) PerRPCCredentials() grpccredentials.PerRPCCredentials {
-	return b.rc
-}
-
-func (b *bundle) NewWithMode(mode string) (grpccredentials.Bundle, error) {
-	// no-op
-	return nil, nil
-}
-
-// transportCredential implements "grpccredentials.TransportCredentials" interface.
-type transportCredential struct {
-	gtc grpccredentials.TransportCredentials
-}
-
-func newTransportCredential(cfg *tls.Config) *transportCredential {
-	return &transportCredential{
-		gtc: grpccredentials.NewTLS(cfg),
-	}
-}
-
-func (tc *transportCredential) ClientHandshake(ctx context.Context, authority string, rawConn net.Conn) (net.Conn, grpccredentials.AuthInfo, error) {
-	// Only overwrite when authority is an IP address!
-	// Let's say, a server runs SRV records on "etcd.local" that resolves
-	// to "m1.etcd.local", and its SAN field also includes "m1.etcd.local".
-	// But what if SAN does not include its resolved IP address (e.g. 127.0.0.1)?
-	// Then, the server should only authenticate using its DNS hostname "m1.etcd.local",
-	// instead of overwriting it with its IP address.
-	// And we do not overwrite "localhost" either. Only overwrite IP addresses!
-	if isIP(authority) {
-		target := rawConn.RemoteAddr().String()
-		if authority != target {
-			// When user dials with "grpc.WithDialer", "grpc.DialContext" "cc.parsedTarget"
-			// update only happens once. This is problematic, because when TLS is enabled,
-			// retries happen through "grpc.WithDialer" with static "cc.parsedTarget" from
-			// the initial dial call.
-			// If the server authenticates by IP addresses, we want to set a new endpoint as
-			// a new authority. Otherwise
-			// "transport: authentication handshake failed: x509: certificate is valid for 127.0.0.1, 192.168.121.180, not 192.168.223.156"
-			// when the new dial target is "192.168.121.180" whose certificate host name is also "192.168.121.180"
-			// but client tries to authenticate with previously set "cc.parsedTarget" field "192.168.223.156"
-			authority = target
-		}
-	}
-	return tc.gtc.ClientHandshake(ctx, authority, rawConn)
-}
-
-// return true if given string is an IP.
-func isIP(ep string) bool {
-	return net.ParseIP(ep) != nil
-}
-
-func (tc *transportCredential) ServerHandshake(rawConn net.Conn) (net.Conn, grpccredentials.AuthInfo, error) {
-	return tc.gtc.ServerHandshake(rawConn)
-}
-
-func (tc *transportCredential) Info() grpccredentials.ProtocolInfo {
-	return tc.gtc.Info()
-}
-
-func (tc *transportCredential) Clone() grpccredentials.TransportCredentials {
-	return &transportCredential{
-		gtc: tc.gtc.Clone(),
-	}
-}
-
-func (tc *transportCredential) OverrideServerName(serverNameOverride string) error {
-	return tc.gtc.OverrideServerName(serverNameOverride)
-}
-
-// perRPCCredential implements "grpccredentials.PerRPCCredentials" interface.
-type perRPCCredential struct {
-	authToken   string
-	authTokenMu sync.RWMutex
-}
-
-func newPerRPCCredential() *perRPCCredential { return &perRPCCredential{} }
-
-func (rc *perRPCCredential) RequireTransportSecurity() bool { return false }
-
-func (rc *perRPCCredential) GetRequestMetadata(ctx context.Context, s ...string) (map[string]string, error) {
-	rc.authTokenMu.RLock()
-	authToken := rc.authToken
-	rc.authTokenMu.RUnlock()
-	return map[string]string{rpctypes.TokenFieldNameGRPC: authToken}, nil
-}
-
-func (b *bundle) UpdateAuthToken(token string) {
-	if b.rc == nil {
-		return
-	}
-	b.rc.UpdateAuthToken(token)
-}
-
-func (rc *perRPCCredential) UpdateAuthToken(token string) {
-	rc.authTokenMu.Lock()
-	rc.authToken = token
-	rc.authTokenMu.Unlock()
-}
diff --git a/vendor/go.etcd.io/etcd/clientv3/ctx.go b/vendor/go.etcd.io/etcd/clientv3/ctx.go
new file mode 100644
index 0000000..da8297b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/clientv3/ctx.go
@@ -0,0 +1,64 @@
+// Copyright 2020 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package clientv3
+
+import (
+	"context"
+	"strings"
+
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/version"
+	"google.golang.org/grpc/metadata"
+)
+
+// WithRequireLeader requires client requests to only succeed
+// when the cluster has a leader.
+func WithRequireLeader(ctx context.Context) context.Context {
+	md, ok := metadata.FromOutgoingContext(ctx)
+	if !ok { // no outgoing metadata ctx key, create one
+		md = metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
+		return metadata.NewOutgoingContext(ctx, md)
+	}
+	copied := md.Copy() // avoid racey updates
+	// overwrite/add 'hasleader' key/value
+	metadataSet(copied, rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
+	return metadata.NewOutgoingContext(ctx, copied)
+}
+
+// embeds client version
+func withVersion(ctx context.Context) context.Context {
+	md, ok := metadata.FromOutgoingContext(ctx)
+	if !ok { // no outgoing metadata ctx key, create one
+		md = metadata.Pairs(rpctypes.MetadataClientAPIVersionKey, version.APIVersion)
+		return metadata.NewOutgoingContext(ctx, md)
+	}
+	copied := md.Copy() // avoid racey updates
+	// overwrite/add version key/value
+	metadataSet(copied, rpctypes.MetadataClientAPIVersionKey, version.APIVersion)
+	return metadata.NewOutgoingContext(ctx, copied)
+}
+
+func metadataGet(md metadata.MD, k string) []string {
+	k = strings.ToLower(k)
+	return md[k]
+}
+
+func metadataSet(md metadata.MD, k string, vals ...string) {
+	if len(vals) == 0 {
+		return
+	}
+	k = strings.ToLower(k)
+	md[k] = vals
+}
diff --git a/vendor/go.etcd.io/etcd/clientv3/doc.go b/vendor/go.etcd.io/etcd/clientv3/doc.go
index 913cd28..717fbe4 100644
--- a/vendor/go.etcd.io/etcd/clientv3/doc.go
+++ b/vendor/go.etcd.io/etcd/clientv3/doc.go
@@ -19,7 +19,7 @@
 //	// expect dial time-out on ipv4 blackhole
 //	_, err := clientv3.New(clientv3.Config{
 //		Endpoints:   []string{"http://254.0.0.1:12345"},
-//		DialTimeout: 2 * time.Second,
+//		DialTimeout: 2 * time.Second
 //	})
 //
 //	// etcd clientv3 >= v3.2.10, grpc/grpc-go >= v1.7.3
@@ -61,7 +61,7 @@
 //
 //  1. context error: canceled or deadline exceeded.
 //  2. gRPC status error: e.g. when clock drifts in server-side before client's context deadline exceeded.
-//  3. gRPC error: see https://github.com/etcd-io/etcd/blob/master/etcdserver/api/v3rpc/rpctypes/error.go
+//  3. gRPC error: see https://github.com/coreos/etcd/blob/master/etcdserver/api/v3rpc/rpctypes/error.go
 //
 // Here is the example code to handle client errors:
 //
@@ -71,14 +71,14 @@
 //			// ctx is canceled by another routine
 //		} else if err == context.DeadlineExceeded {
 //			// ctx is attached with a deadline and it exceeded
-//		} else if err == rpctypes.ErrEmptyKey {
-//			// client-side error: key is not provided
 //		} else if ev, ok := status.FromError(err); ok {
 //			code := ev.Code()
 //			if code == codes.DeadlineExceeded {
 //				// server-side context might have timed-out first (due to clock skew)
 //				// while original client-side context is not timed-out yet
 //			}
+//		} else if verr, ok := err.(*v3rpc.ErrEmptyKey); ok {
+//			// process (verr.Errors)
 //		} else {
 //			// bad cluster endpoints, which are not etcd servers
 //		}
@@ -87,20 +87,11 @@
 //	go func() { cli.Close() }()
 //	_, err := kvc.Get(ctx, "a")
 //	if err != nil {
-//		// with etcd clientv3 <= v3.3
 //		if err == context.Canceled {
 //			// grpc balancer calls 'Get' with an inflight client.Close
-//		} else if err == grpc.ErrClientConnClosing { // <= gRCP v1.7.x
+//		} else if err == grpc.ErrClientConnClosing {
 //			// grpc balancer calls 'Get' after client.Close.
 //		}
-//		// with etcd clientv3 >= v3.4
-//		if clientv3.IsConnCanceled(err) {
-//			// gRPC client connection is closed
-//		}
 //	}
 //
-// The grpc load balancer is registered statically and is shared across etcd clients.
-// To enable detailed load balancer logging, set the ETCD_CLIENT_DEBUG environment
-// variable.  E.g. "ETCD_CLIENT_DEBUG=1".
-//
 package clientv3
diff --git a/vendor/go.etcd.io/etcd/clientv3/kv.go b/vendor/go.etcd.io/etcd/clientv3/kv.go
index 2b7864a..5a7469b 100644
--- a/vendor/go.etcd.io/etcd/clientv3/kv.go
+++ b/vendor/go.etcd.io/etcd/clientv3/kv.go
@@ -17,7 +17,7 @@
 import (
 	"context"
 
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 	"google.golang.org/grpc"
 )
diff --git a/vendor/go.etcd.io/etcd/clientv3/lease.go b/vendor/go.etcd.io/etcd/clientv3/lease.go
index c2796fc..3729cf3 100644
--- a/vendor/go.etcd.io/etcd/clientv3/lease.go
+++ b/vendor/go.etcd.io/etcd/clientv3/lease.go
@@ -19,10 +19,9 @@
 	"sync"
 	"time"
 
-	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
-	"go.uber.org/zap"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/metadata"
 )
@@ -118,21 +117,22 @@
 	// Leases retrieves all leases.
 	Leases(ctx context.Context) (*LeaseLeasesResponse, error)
 
-	// KeepAlive attempts to keep the given lease alive forever. If the keepalive responses posted
-	// to the channel are not consumed promptly the channel may become full. When full, the lease
-	// client will continue sending keep alive requests to the etcd server, but will drop responses
-	// until there is capacity on the channel to send more responses.
-	//
-	// If client keep alive loop halts with an unexpected error (e.g. "etcdserver: no leader") or
-	// canceled by the caller (e.g. context.Canceled), KeepAlive returns a ErrKeepAliveHalted error
-	// containing the error reason.
+	// KeepAlive keeps the given lease alive forever. If the keepalive response
+	// posted to the channel is not consumed immediately, the lease client will
+	// continue sending keep alive requests to the etcd server at least every
+	// second until latest response is consumed.
 	//
 	// The returned "LeaseKeepAliveResponse" channel closes if underlying keep
 	// alive stream is interrupted in some way the client cannot handle itself;
-	// given context "ctx" is canceled or timed out.
+	// given context "ctx" is canceled or timed out. "LeaseKeepAliveResponse"
+	// from this closed channel is nil.
+	//
+	// If client keep alive loop halts with an unexpected error (e.g. "etcdserver:
+	// no leader") or canceled by the caller (e.g. context.Canceled), the error
+	// is returned. Otherwise, it retries.
 	//
 	// TODO(v4.0): post errors to last keep alive message before closing
-	// (see https://github.com/etcd-io/etcd/pull/7866)
+	// (see https://github.com/coreos/etcd/pull/7866)
 	KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error)
 
 	// KeepAliveOnce renews the lease once. The response corresponds to the
@@ -172,8 +172,6 @@
 	firstKeepAliveOnce sync.Once
 
 	callOpts []grpc.CallOption
-
-	lg *zap.Logger
 }
 
 // keepAlive multiplexes a keepalive for a lease over multiple channels
@@ -198,7 +196,6 @@
 		keepAlives:            make(map[LeaseID]*keepAlive),
 		remote:                remote,
 		firstKeepAliveTimeout: keepAliveTimeout,
-		lg:                    c.lg,
 	}
 	if l.firstKeepAliveTimeout == time.Second {
 		l.firstKeepAliveTimeout = defaultTTL
@@ -294,7 +291,7 @@
 	}
 	l.mu.Unlock()
 
-	go l.keepAliveCtxCloser(ctx, id, ka.donec)
+	go l.keepAliveCtxCloser(id, ctx, ka.donec)
 	l.firstKeepAliveOnce.Do(func() {
 		go l.recvKeepAliveLoop()
 		go l.deadlineLoop()
@@ -326,7 +323,7 @@
 	return nil
 }
 
-func (l *lessor) keepAliveCtxCloser(ctx context.Context, id LeaseID, donec <-chan struct{}) {
+func (l *lessor) keepAliveCtxCloser(id LeaseID, ctx context.Context, donec <-chan struct{}) {
 	select {
 	case <-donec:
 		return
@@ -462,6 +459,7 @@
 
 		select {
 		case <-time.After(retryConnWait):
+			continue
 		case <-l.stopCtx.Done():
 			return l.stopCtx.Err()
 		}
@@ -471,7 +469,7 @@
 // resetRecv opens a new lease stream and starts sending keep alive requests.
 func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
 	sctx, cancel := context.WithCancel(l.stopCtx)
-	stream, err := l.remote.LeaseKeepAlive(sctx, append(l.callOpts, withMax(0))...)
+	stream, err := l.remote.LeaseKeepAlive(sctx, l.callOpts...)
 	if err != nil {
 		cancel()
 		return nil, err
@@ -520,12 +518,6 @@
 		select {
 		case ch <- karesp:
 		default:
-			if l.lg != nil {
-				l.lg.Warn("lease keepalive response queue is full; dropping response send",
-					zap.Int("queue-size", len(ch)),
-					zap.Int("queue-capacity", cap(ch)),
-				)
-			}
 		}
 		// still advance in order to rate-limit keep-alive sends
 		ka.nextKeepAlive = nextKeepAlive
@@ -577,7 +569,7 @@
 		}
 
 		select {
-		case <-time.After(retryConnWait):
+		case <-time.After(500 * time.Millisecond):
 		case <-stream.Context().Done():
 			return
 		case <-l.donec:
diff --git a/vendor/go.etcd.io/etcd/clientv3/logger.go b/vendor/go.etcd.io/etcd/clientv3/logger.go
index f5ae010..3276372 100644
--- a/vendor/go.etcd.io/etcd/clientv3/logger.go
+++ b/vendor/go.etcd.io/etcd/clientv3/logger.go
@@ -18,7 +18,7 @@
 	"io/ioutil"
 	"sync"
 
-	"go.etcd.io/etcd/pkg/logutil"
+	"github.com/coreos/etcd/pkg/logutil"
 
 	"google.golang.org/grpc/grpclog"
 )
diff --git a/vendor/go.etcd.io/etcd/clientv3/maintenance.go b/vendor/go.etcd.io/etcd/clientv3/maintenance.go
index 744455a..5e87cf8 100644
--- a/vendor/go.etcd.io/etcd/clientv3/maintenance.go
+++ b/vendor/go.etcd.io/etcd/clientv3/maintenance.go
@@ -19,7 +19,7 @@
 	"fmt"
 	"io"
 
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 	"google.golang.org/grpc"
 )
@@ -193,23 +193,32 @@
 		return nil, toErr(ctx, err)
 	}
 
+	plog.Info("opened snapshot stream; downloading")
 	pr, pw := io.Pipe()
 	go func() {
 		for {
 			resp, err := ss.Recv()
 			if err != nil {
+				switch err {
+				case io.EOF:
+					plog.Info("completed snapshot read; closing")
+				default:
+					plog.Warningf("failed to receive from snapshot stream; closing (%v)", err)
+				}
 				pw.CloseWithError(err)
 				return
 			}
-			if resp == nil && err == nil {
-				break
-			}
+
+			// can "resp == nil && err == nil"
+			// before we receive snapshot SHA digest?
+			// No, server sends EOF with an empty response
+			// after it sends SHA digest at the end
+
 			if _, werr := pw.Write(resp.Blob); werr != nil {
 				pw.CloseWithError(werr)
 				return
 			}
 		}
-		pw.Close()
 	}()
 	return &snapshotReadCloser{ctx: ctx, ReadCloser: pr}, nil
 }
diff --git a/vendor/go.etcd.io/etcd/clientv3/op.go b/vendor/go.etcd.io/etcd/clientv3/op.go
index 81ae31f..3dca41b 100644
--- a/vendor/go.etcd.io/etcd/clientv3/op.go
+++ b/vendor/go.etcd.io/etcd/clientv3/op.go
@@ -14,7 +14,7 @@
 
 package clientv3
 
-import pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+import pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 type opType int
 
@@ -26,7 +26,9 @@
 	tTxn
 )
 
-var noPrefixEnd = []byte{0}
+var (
+	noPrefixEnd = []byte{0}
+)
 
 // Op represents an Operation that kv can execute.
 type Op struct {
@@ -81,15 +83,8 @@
 
 // accessors / mutators
 
-// IsTxn returns true if the "Op" type is transaction.
-func (op Op) IsTxn() bool {
-	return op.t == tTxn
-}
-
-// Txn returns the comparison(if) operations, "then" operations, and "else" operations.
-func (op Op) Txn() ([]Cmp, []Op, []Op) {
-	return op.cmps, op.thenOps, op.elseOps
-}
+func (op Op) IsTxn() bool              { return op.t == tTxn }
+func (op Op) Txn() ([]Cmp, []Op, []Op) { return op.cmps, op.thenOps, op.elseOps }
 
 // KeyBytes returns the byte slice holding the Op's key.
 func (op Op) KeyBytes() []byte { return op.key }
@@ -113,13 +108,13 @@
 func (op Op) IsDelete() bool { return op.t == tDeleteRange }
 
 // IsSerializable returns true if the serializable field is true.
-func (op Op) IsSerializable() bool { return op.serializable }
+func (op Op) IsSerializable() bool { return op.serializable == true }
 
 // IsKeysOnly returns whether keysOnly is set.
-func (op Op) IsKeysOnly() bool { return op.keysOnly }
+func (op Op) IsKeysOnly() bool { return op.keysOnly == true }
 
 // IsCountOnly returns whether countOnly is set.
-func (op Op) IsCountOnly() bool { return op.countOnly }
+func (op Op) IsCountOnly() bool { return op.countOnly == true }
 
 // MinModRev returns the operation's minimum modify revision.
 func (op Op) MinModRev() int64 { return op.minModRev }
@@ -216,23 +211,13 @@
 	return op.t != tRange
 }
 
-// OpGet returns "get" operation based on given key and operation options.
 func OpGet(key string, opts ...OpOption) Op {
-	// WithPrefix and WithFromKey are not supported together
-	if isWithPrefix(opts) && isWithFromKey(opts) {
-		panic("`WithPrefix` and `WithFromKey` cannot be set at the same time, choose one")
-	}
 	ret := Op{t: tRange, key: []byte(key)}
 	ret.applyOpts(opts)
 	return ret
 }
 
-// OpDelete returns "delete" operation based on given key and operation options.
 func OpDelete(key string, opts ...OpOption) Op {
-	// WithPrefix and WithFromKey are not supported together
-	if isWithPrefix(opts) && isWithFromKey(opts) {
-		panic("`WithPrefix` and `WithFromKey` cannot be set at the same time, choose one")
-	}
 	ret := Op{t: tDeleteRange, key: []byte(key)}
 	ret.applyOpts(opts)
 	switch {
@@ -260,7 +245,6 @@
 	return ret
 }
 
-// OpPut returns "put" operation based on given key-value and operation options.
 func OpPut(key, val string, opts ...OpOption) Op {
 	ret := Op{t: tPut, key: []byte(key), val: []byte(val)}
 	ret.applyOpts(opts)
@@ -289,7 +273,6 @@
 	return ret
 }
 
-// OpTxn returns "txn" operation based on given transaction conditions.
 func OpTxn(cmps []Cmp, thenOps []Op, elseOps []Op) Op {
 	return Op{t: tTxn, cmps: cmps, thenOps: thenOps, elseOps: elseOps}
 }
@@ -400,14 +383,7 @@
 
 // WithFromKey specifies the range of 'Get', 'Delete', 'Watch' requests
 // to be equal or greater than the key in the argument.
-func WithFromKey() OpOption {
-	return func(op *Op) {
-		if len(op.key) == 0 {
-			op.key = []byte{0}
-		}
-		op.end = []byte("\x00")
-	}
-}
+func WithFromKey() OpOption { return WithRange("\x00") }
 
 // WithSerializable makes 'Get' request serializable. By default,
 // it's linearizable. Serializable requests are better for lower latency
@@ -496,17 +472,6 @@
 	}
 }
 
-// WithFragment to receive raw watch response with fragmentation.
-// Fragmentation is disabled by default. If fragmentation is enabled,
-// etcd watch server will split watch response before sending to clients
-// when the total size of watch events exceed server-side request limit.
-// The default server-side request limit is 1.5 MiB, which can be configured
-// as "--max-request-bytes" flag value + gRPC-overhead 512 bytes.
-// See "etcdserver/api/v3rpc/watch.go" for more details.
-func WithFragment() OpOption {
-	return func(op *Op) { op.fragment = true }
-}
-
 // WithIgnoreValue updates the key using its current value.
 // This option can not be combined with non-empty values.
 // Returns an error if the key does not exist.
@@ -553,8 +518,13 @@
 	return &pb.LeaseTimeToLiveRequest{ID: int64(id), Keys: ret.attachedKeys}
 }
 
-// isWithPrefix returns true if WithPrefix is being called in the op
-func isWithPrefix(opts []OpOption) bool { return isOpFuncCalled("WithPrefix", opts) }
-
-// isWithFromKey returns true if WithFromKey is being called in the op
-func isWithFromKey(opts []OpOption) bool { return isOpFuncCalled("WithFromKey", opts) }
+// WithFragment to receive raw watch response with fragmentation.
+// Fragmentation is disabled by default. If fragmentation is enabled,
+// etcd watch server will split watch response before sending to clients
+// when the total size of watch events exceed server-side request limit.
+// The default server-side request limit is 1.5 MiB, which can be configured
+// as "--max-request-bytes" flag value + gRPC-overhead 512 bytes.
+// See "etcdserver/api/v3rpc/watch.go" for more details.
+func WithFragment() OpOption {
+	return func(op *Op) { op.fragment = true }
+}
diff --git a/vendor/go.etcd.io/etcd/clientv3/retry.go b/vendor/go.etcd.io/etcd/clientv3/retry.go
index 7e855de..6baa52e 100644
--- a/vendor/go.etcd.io/etcd/clientv3/retry.go
+++ b/vendor/go.etcd.io/etcd/clientv3/retry.go
@@ -17,8 +17,8 @@
 import (
 	"context"
 
-	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
@@ -179,10 +179,6 @@
 	return rcc.cc.MemberUpdate(ctx, in, opts...)
 }
 
-func (rcc *retryClusterClient) MemberPromote(ctx context.Context, in *pb.MemberPromoteRequest, opts ...grpc.CallOption) (resp *pb.MemberPromoteResponse, err error) {
-	return rcc.cc.MemberPromote(ctx, in, opts...)
-}
-
 type retryMaintenanceClient struct {
 	mc pb.MaintenanceClient
 }
diff --git a/vendor/go.etcd.io/etcd/clientv3/retry_interceptor.go b/vendor/go.etcd.io/etcd/clientv3/retry_interceptor.go
index 080490a..f3c5057 100644
--- a/vendor/go.etcd.io/etcd/clientv3/retry_interceptor.go
+++ b/vendor/go.etcd.io/etcd/clientv3/retry_interceptor.go
@@ -23,7 +23,7 @@
 	"sync"
 	"time"
 
-	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
 	"go.uber.org/zap"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
@@ -38,6 +38,7 @@
 func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.UnaryClientInterceptor {
 	intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
 	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		ctx = withVersion(ctx)
 		grpcOpts, retryOpts := filterCallOptions(opts)
 		callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
 		// short circuit for simplicity, and avoiding allocations.
@@ -103,6 +104,7 @@
 func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.StreamClientInterceptor {
 	intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
 	return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		ctx = withVersion(ctx)
 		grpcOpts, retryOpts := filterCallOptions(opts)
 		callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
 		// short circuit for simplicity, and avoiding allocations.
@@ -113,10 +115,9 @@
 			return nil, status.Errorf(codes.Unimplemented, "clientv3/retry_interceptor: cannot retry on ClientStreams, set Disable()")
 		}
 		newStreamer, err := streamer(ctx, desc, cc, method, grpcOpts...)
-		logger.Warn("retry stream intercept", zap.Error(err))
 		if err != nil {
-			// TODO(mwitkow): Maybe dial and transport errors should be retriable?
-			return nil, err
+			logger.Error("streamer failed to create ClientStream", zap.Error(err))
+			return nil, err // TODO(mwitkow): Maybe dial and transport errors should be retriable?
 		}
 		retryingStreamer := &serverStreamingRetryingStream{
 			client:       c,
@@ -185,6 +186,7 @@
 	if !attemptRetry {
 		return lastErr // success or hard failure
 	}
+
 	// We start off from attempt 1, because zeroth was already made on normal SendMsg().
 	for attempt := uint(1); attempt < s.callOpts.max; attempt++ {
 		if err := waitRetryBackoff(s.ctx, attempt, s.callOpts); err != nil {
@@ -192,12 +194,13 @@
 		}
 		newStream, err := s.reestablishStreamAndResendBuffer(s.ctx)
 		if err != nil {
-			// TODO(mwitkow): Maybe dial and transport errors should be retriable?
-			return err
+			s.client.lg.Error("failed reestablishStreamAndResendBuffer", zap.Error(err))
+			return err // TODO(mwitkow): Maybe dial and transport errors should be retriable?
 		}
 		s.setStream(newStream)
+
+		s.client.lg.Warn("retrying RecvMsg", zap.Error(lastErr))
 		attemptRetry, lastErr = s.receiveMsgAndIndicateRetry(m)
-		//fmt.Printf("Received message and indicate: %v  %v\n", attemptRetry, lastErr)
 		if !attemptRetry {
 			return lastErr
 		}
diff --git a/vendor/go.etcd.io/etcd/clientv3/txn.go b/vendor/go.etcd.io/etcd/clientv3/txn.go
index c19715d..c3c2d24 100644
--- a/vendor/go.etcd.io/etcd/clientv3/txn.go
+++ b/vendor/go.etcd.io/etcd/clientv3/txn.go
@@ -18,7 +18,7 @@
 	"context"
 	"sync"
 
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 
 	"google.golang.org/grpc"
 )
diff --git a/vendor/go.etcd.io/etcd/clientv3/watch.go b/vendor/go.etcd.io/etcd/clientv3/watch.go
index 87d222d..4a3b8cc 100644
--- a/vendor/go.etcd.io/etcd/clientv3/watch.go
+++ b/vendor/go.etcd.io/etcd/clientv3/watch.go
@@ -21,9 +21,9 @@
 	"sync"
 	"time"
 
-	v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
-	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
-	mvccpb "go.etcd.io/etcd/mvcc/mvccpb"
+	v3rpc "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
 
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/error.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/error.go
index e6a2814..bc1ad7b 100644
--- a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/error.go
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/error.go
@@ -40,9 +40,6 @@
 	ErrGRPCMemberNotEnoughStarted = status.New(codes.FailedPrecondition, "etcdserver: re-configuration failed due to not enough started members").Err()
 	ErrGRPCMemberBadURLs          = status.New(codes.InvalidArgument, "etcdserver: given member URLs are invalid").Err()
 	ErrGRPCMemberNotFound         = status.New(codes.NotFound, "etcdserver: member not found").Err()
-	ErrGRPCMemberNotLearner       = status.New(codes.FailedPrecondition, "etcdserver: can only promote a learner member").Err()
-	ErrGRPCLearnerNotReady        = status.New(codes.FailedPrecondition, "etcdserver: can only promote a learner member which is in sync with leader").Err()
-	ErrGRPCTooManyLearners        = status.New(codes.FailedPrecondition, "etcdserver: too many learner members in cluster").Err()
 
 	ErrGRPCRequestTooLarge        = status.New(codes.InvalidArgument, "etcdserver: request is too large").Err()
 	ErrGRPCRequestTooManyRequests = status.New(codes.ResourceExhausted, "etcdserver: too many requests").Err()
@@ -54,7 +51,6 @@
 	ErrGRPCUserNotFound         = status.New(codes.FailedPrecondition, "etcdserver: user name not found").Err()
 	ErrGRPCRoleAlreadyExist     = status.New(codes.FailedPrecondition, "etcdserver: role name already exists").Err()
 	ErrGRPCRoleNotFound         = status.New(codes.FailedPrecondition, "etcdserver: role name not found").Err()
-	ErrGRPCRoleEmpty            = status.New(codes.InvalidArgument, "etcdserver: role name is empty").Err()
 	ErrGRPCAuthFailed           = status.New(codes.InvalidArgument, "etcdserver: authentication failed, invalid user ID or password").Err()
 	ErrGRPCPermissionDenied     = status.New(codes.PermissionDenied, "etcdserver: permission denied").Err()
 	ErrGRPCRoleNotGranted       = status.New(codes.FailedPrecondition, "etcdserver: role is not granted to the user").Err()
@@ -73,8 +69,6 @@
 	ErrGRPCTimeoutDueToConnectionLost = status.New(codes.Unavailable, "etcdserver: request timed out, possibly due to connection lost").Err()
 	ErrGRPCUnhealthy                  = status.New(codes.Unavailable, "etcdserver: unhealthy cluster").Err()
 	ErrGRPCCorrupt                    = status.New(codes.DataLoss, "etcdserver: corrupt cluster").Err()
-	ErrGPRCNotSupportedForLearner     = status.New(codes.Unavailable, "etcdserver: rpc not supported for learner").Err()
-	ErrGRPCBadLeaderTransferee        = status.New(codes.FailedPrecondition, "etcdserver: bad leader transferee").Err()
 
 	errStringToError = map[string]error{
 		ErrorDesc(ErrGRPCEmptyKey):      ErrGRPCEmptyKey,
@@ -97,9 +91,6 @@
 		ErrorDesc(ErrGRPCMemberNotEnoughStarted): ErrGRPCMemberNotEnoughStarted,
 		ErrorDesc(ErrGRPCMemberBadURLs):          ErrGRPCMemberBadURLs,
 		ErrorDesc(ErrGRPCMemberNotFound):         ErrGRPCMemberNotFound,
-		ErrorDesc(ErrGRPCMemberNotLearner):       ErrGRPCMemberNotLearner,
-		ErrorDesc(ErrGRPCLearnerNotReady):        ErrGRPCLearnerNotReady,
-		ErrorDesc(ErrGRPCTooManyLearners):        ErrGRPCTooManyLearners,
 
 		ErrorDesc(ErrGRPCRequestTooLarge):        ErrGRPCRequestTooLarge,
 		ErrorDesc(ErrGRPCRequestTooManyRequests): ErrGRPCRequestTooManyRequests,
@@ -111,7 +102,6 @@
 		ErrorDesc(ErrGRPCUserNotFound):         ErrGRPCUserNotFound,
 		ErrorDesc(ErrGRPCRoleAlreadyExist):     ErrGRPCRoleAlreadyExist,
 		ErrorDesc(ErrGRPCRoleNotFound):         ErrGRPCRoleNotFound,
-		ErrorDesc(ErrGRPCRoleEmpty):            ErrGRPCRoleEmpty,
 		ErrorDesc(ErrGRPCAuthFailed):           ErrGRPCAuthFailed,
 		ErrorDesc(ErrGRPCPermissionDenied):     ErrGRPCPermissionDenied,
 		ErrorDesc(ErrGRPCRoleNotGranted):       ErrGRPCRoleNotGranted,
@@ -122,7 +112,6 @@
 
 		ErrorDesc(ErrGRPCNoLeader):                   ErrGRPCNoLeader,
 		ErrorDesc(ErrGRPCNotLeader):                  ErrGRPCNotLeader,
-		ErrorDesc(ErrGRPCLeaderChanged):              ErrGRPCLeaderChanged,
 		ErrorDesc(ErrGRPCNotCapable):                 ErrGRPCNotCapable,
 		ErrorDesc(ErrGRPCStopped):                    ErrGRPCStopped,
 		ErrorDesc(ErrGRPCTimeout):                    ErrGRPCTimeout,
@@ -130,8 +119,6 @@
 		ErrorDesc(ErrGRPCTimeoutDueToConnectionLost): ErrGRPCTimeoutDueToConnectionLost,
 		ErrorDesc(ErrGRPCUnhealthy):                  ErrGRPCUnhealthy,
 		ErrorDesc(ErrGRPCCorrupt):                    ErrGRPCCorrupt,
-		ErrorDesc(ErrGPRCNotSupportedForLearner):     ErrGPRCNotSupportedForLearner,
-		ErrorDesc(ErrGRPCBadLeaderTransferee):        ErrGRPCBadLeaderTransferee,
 	}
 )
 
@@ -156,9 +143,6 @@
 	ErrMemberNotEnoughStarted = Error(ErrGRPCMemberNotEnoughStarted)
 	ErrMemberBadURLs          = Error(ErrGRPCMemberBadURLs)
 	ErrMemberNotFound         = Error(ErrGRPCMemberNotFound)
-	ErrMemberNotLearner       = Error(ErrGRPCMemberNotLearner)
-	ErrMemberLearnerNotReady  = Error(ErrGRPCLearnerNotReady)
-	ErrTooManyLearners        = Error(ErrGRPCTooManyLearners)
 
 	ErrRequestTooLarge = Error(ErrGRPCRequestTooLarge)
 	ErrTooManyRequests = Error(ErrGRPCRequestTooManyRequests)
@@ -170,7 +154,6 @@
 	ErrUserNotFound         = Error(ErrGRPCUserNotFound)
 	ErrRoleAlreadyExist     = Error(ErrGRPCRoleAlreadyExist)
 	ErrRoleNotFound         = Error(ErrGRPCRoleNotFound)
-	ErrRoleEmpty            = Error(ErrGRPCRoleEmpty)
 	ErrAuthFailed           = Error(ErrGRPCAuthFailed)
 	ErrPermissionDenied     = Error(ErrGRPCPermissionDenied)
 	ErrRoleNotGranted       = Error(ErrGRPCRoleNotGranted)
@@ -189,7 +172,6 @@
 	ErrTimeoutDueToConnectionLost = Error(ErrGRPCTimeoutDueToConnectionLost)
 	ErrUnhealthy                  = Error(ErrGRPCUnhealthy)
 	ErrCorrupt                    = Error(ErrGRPCCorrupt)
-	ErrBadLeaderTransferee        = Error(ErrGRPCBadLeaderTransferee)
 )
 
 // EtcdError defines gRPC server errors.
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/md.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/md.go
index 5c590e1..90b8b83 100644
--- a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/md.go
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes/md.go
@@ -17,4 +17,6 @@
 var (
 	MetadataRequireLeaderKey = "hasleader"
 	MetadataHasLeader        = "true"
+
+	MetadataClientAPIVersionKey = "client-api-version"
 )
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/etcdserver.pb.go b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/etcdserver.pb.go
deleted file mode 100644
index 9e9b42c..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/etcdserver.pb.go
+++ /dev/null
@@ -1,1041 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: etcdserver.proto
-
-/*
-	Package etcdserverpb is a generated protocol buffer package.
-
-	It is generated from these files:
-		etcdserver.proto
-		raft_internal.proto
-		rpc.proto
-
-	It has these top-level messages:
-		Request
-		Metadata
-		RequestHeader
-		InternalRaftRequest
-		EmptyResponse
-		InternalAuthenticateRequest
-		ResponseHeader
-		RangeRequest
-		RangeResponse
-		PutRequest
-		PutResponse
-		DeleteRangeRequest
-		DeleteRangeResponse
-		RequestOp
-		ResponseOp
-		Compare
-		TxnRequest
-		TxnResponse
-		CompactionRequest
-		CompactionResponse
-		HashRequest
-		HashKVRequest
-		HashKVResponse
-		HashResponse
-		SnapshotRequest
-		SnapshotResponse
-		WatchRequest
-		WatchCreateRequest
-		WatchCancelRequest
-		WatchProgressRequest
-		WatchResponse
-		LeaseGrantRequest
-		LeaseGrantResponse
-		LeaseRevokeRequest
-		LeaseRevokeResponse
-		LeaseCheckpoint
-		LeaseCheckpointRequest
-		LeaseCheckpointResponse
-		LeaseKeepAliveRequest
-		LeaseKeepAliveResponse
-		LeaseTimeToLiveRequest
-		LeaseTimeToLiveResponse
-		LeaseLeasesRequest
-		LeaseStatus
-		LeaseLeasesResponse
-		Member
-		MemberAddRequest
-		MemberAddResponse
-		MemberRemoveRequest
-		MemberRemoveResponse
-		MemberUpdateRequest
-		MemberUpdateResponse
-		MemberListRequest
-		MemberListResponse
-		MemberPromoteRequest
-		MemberPromoteResponse
-		DefragmentRequest
-		DefragmentResponse
-		MoveLeaderRequest
-		MoveLeaderResponse
-		AlarmRequest
-		AlarmMember
-		AlarmResponse
-		StatusRequest
-		StatusResponse
-		AuthEnableRequest
-		AuthDisableRequest
-		AuthenticateRequest
-		AuthUserAddRequest
-		AuthUserGetRequest
-		AuthUserDeleteRequest
-		AuthUserChangePasswordRequest
-		AuthUserGrantRoleRequest
-		AuthUserRevokeRoleRequest
-		AuthRoleAddRequest
-		AuthRoleGetRequest
-		AuthUserListRequest
-		AuthRoleListRequest
-		AuthRoleDeleteRequest
-		AuthRoleGrantPermissionRequest
-		AuthRoleRevokePermissionRequest
-		AuthEnableResponse
-		AuthDisableResponse
-		AuthenticateResponse
-		AuthUserAddResponse
-		AuthUserGetResponse
-		AuthUserDeleteResponse
-		AuthUserChangePasswordResponse
-		AuthUserGrantRoleResponse
-		AuthUserRevokeRoleResponse
-		AuthRoleAddResponse
-		AuthRoleGetResponse
-		AuthRoleListResponse
-		AuthUserListResponse
-		AuthRoleDeleteResponse
-		AuthRoleGrantPermissionResponse
-		AuthRoleRevokePermissionResponse
-*/
-package etcdserverpb
-
-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 Request struct {
-	ID               uint64 `protobuf:"varint,1,opt,name=ID" json:"ID"`
-	Method           string `protobuf:"bytes,2,opt,name=Method" json:"Method"`
-	Path             string `protobuf:"bytes,3,opt,name=Path" json:"Path"`
-	Val              string `protobuf:"bytes,4,opt,name=Val" json:"Val"`
-	Dir              bool   `protobuf:"varint,5,opt,name=Dir" json:"Dir"`
-	PrevValue        string `protobuf:"bytes,6,opt,name=PrevValue" json:"PrevValue"`
-	PrevIndex        uint64 `protobuf:"varint,7,opt,name=PrevIndex" json:"PrevIndex"`
-	PrevExist        *bool  `protobuf:"varint,8,opt,name=PrevExist" json:"PrevExist,omitempty"`
-	Expiration       int64  `protobuf:"varint,9,opt,name=Expiration" json:"Expiration"`
-	Wait             bool   `protobuf:"varint,10,opt,name=Wait" json:"Wait"`
-	Since            uint64 `protobuf:"varint,11,opt,name=Since" json:"Since"`
-	Recursive        bool   `protobuf:"varint,12,opt,name=Recursive" json:"Recursive"`
-	Sorted           bool   `protobuf:"varint,13,opt,name=Sorted" json:"Sorted"`
-	Quorum           bool   `protobuf:"varint,14,opt,name=Quorum" json:"Quorum"`
-	Time             int64  `protobuf:"varint,15,opt,name=Time" json:"Time"`
-	Stream           bool   `protobuf:"varint,16,opt,name=Stream" json:"Stream"`
-	Refresh          *bool  `protobuf:"varint,17,opt,name=Refresh" json:"Refresh,omitempty"`
-	XXX_unrecognized []byte `json:"-"`
-}
-
-func (m *Request) Reset()                    { *m = Request{} }
-func (m *Request) String() string            { return proto.CompactTextString(m) }
-func (*Request) ProtoMessage()               {}
-func (*Request) Descriptor() ([]byte, []int) { return fileDescriptorEtcdserver, []int{0} }
-
-type Metadata struct {
-	NodeID           uint64 `protobuf:"varint,1,opt,name=NodeID" json:"NodeID"`
-	ClusterID        uint64 `protobuf:"varint,2,opt,name=ClusterID" json:"ClusterID"`
-	XXX_unrecognized []byte `json:"-"`
-}
-
-func (m *Metadata) Reset()                    { *m = Metadata{} }
-func (m *Metadata) String() string            { return proto.CompactTextString(m) }
-func (*Metadata) ProtoMessage()               {}
-func (*Metadata) Descriptor() ([]byte, []int) { return fileDescriptorEtcdserver, []int{1} }
-
-func init() {
-	proto.RegisterType((*Request)(nil), "etcdserverpb.Request")
-	proto.RegisterType((*Metadata)(nil), "etcdserverpb.Metadata")
-}
-func (m *Request) 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 *Request) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.ID))
-	dAtA[i] = 0x12
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Method)))
-	i += copy(dAtA[i:], m.Method)
-	dAtA[i] = 0x1a
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Path)))
-	i += copy(dAtA[i:], m.Path)
-	dAtA[i] = 0x22
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.Val)))
-	i += copy(dAtA[i:], m.Val)
-	dAtA[i] = 0x28
-	i++
-	if m.Dir {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x32
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(len(m.PrevValue)))
-	i += copy(dAtA[i:], m.PrevValue)
-	dAtA[i] = 0x38
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.PrevIndex))
-	if m.PrevExist != nil {
-		dAtA[i] = 0x40
-		i++
-		if *m.PrevExist {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	dAtA[i] = 0x48
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Expiration))
-	dAtA[i] = 0x50
-	i++
-	if m.Wait {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x58
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Since))
-	dAtA[i] = 0x60
-	i++
-	if m.Recursive {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x68
-	i++
-	if m.Sorted {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x70
-	i++
-	if m.Quorum {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x78
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.Time))
-	dAtA[i] = 0x80
-	i++
-	dAtA[i] = 0x1
-	i++
-	if m.Stream {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	if m.Refresh != nil {
-		dAtA[i] = 0x88
-		i++
-		dAtA[i] = 0x1
-		i++
-		if *m.Refresh {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *Metadata) 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 *Metadata) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.NodeID))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintEtcdserver(dAtA, i, uint64(m.ClusterID))
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func encodeVarintEtcdserver(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 *Request) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovEtcdserver(uint64(m.ID))
-	l = len(m.Method)
-	n += 1 + l + sovEtcdserver(uint64(l))
-	l = len(m.Path)
-	n += 1 + l + sovEtcdserver(uint64(l))
-	l = len(m.Val)
-	n += 1 + l + sovEtcdserver(uint64(l))
-	n += 2
-	l = len(m.PrevValue)
-	n += 1 + l + sovEtcdserver(uint64(l))
-	n += 1 + sovEtcdserver(uint64(m.PrevIndex))
-	if m.PrevExist != nil {
-		n += 2
-	}
-	n += 1 + sovEtcdserver(uint64(m.Expiration))
-	n += 2
-	n += 1 + sovEtcdserver(uint64(m.Since))
-	n += 2
-	n += 2
-	n += 2
-	n += 1 + sovEtcdserver(uint64(m.Time))
-	n += 3
-	if m.Refresh != nil {
-		n += 3
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *Metadata) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovEtcdserver(uint64(m.NodeID))
-	n += 1 + sovEtcdserver(uint64(m.ClusterID))
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func sovEtcdserver(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozEtcdserver(x uint64) (n int) {
-	return sovEtcdserver(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *Request) 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 ErrIntOverflowEtcdserver
-			}
-			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: Request: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Request: 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 ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Method = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Path = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Val", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Val = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Dir", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Dir = bool(v != 0)
-		case 6:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevValue", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.PrevValue = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 7:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevIndex", wireType)
-			}
-			m.PrevIndex = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.PrevIndex |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 8:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevExist", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			b := bool(v != 0)
-			m.PrevExist = &b
-		case 9:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Expiration", wireType)
-			}
-			m.Expiration = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Expiration |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 10:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Wait", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Wait = bool(v != 0)
-		case 11:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Since", wireType)
-			}
-			m.Since = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Since |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 12:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Recursive", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Recursive = bool(v != 0)
-		case 13:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Sorted", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Sorted = bool(v != 0)
-		case 14:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Quorum", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Quorum = bool(v != 0)
-		case 15:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Time", wireType)
-			}
-			m.Time = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Time |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 16:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Stream", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Stream = bool(v != 0)
-		case 17:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Refresh", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			b := bool(v != 0)
-			m.Refresh = &b
-		default:
-			iNdEx = preIndex
-			skippy, err := skipEtcdserver(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			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 *Metadata) 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 ErrIntOverflowEtcdserver
-			}
-			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: Metadata: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
-			}
-			m.NodeID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.NodeID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ClusterID", wireType)
-			}
-			m.ClusterID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowEtcdserver
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ClusterID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipEtcdserver(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthEtcdserver
-			}
-			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 skipEtcdserver(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, ErrIntOverflowEtcdserver
-			}
-			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, ErrIntOverflowEtcdserver
-				}
-				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, ErrIntOverflowEtcdserver
-				}
-				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, ErrInvalidLengthEtcdserver
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowEtcdserver
-					}
-					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 := skipEtcdserver(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 (
-	ErrInvalidLengthEtcdserver = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowEtcdserver   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("etcdserver.proto", fileDescriptorEtcdserver) }
-
-var fileDescriptorEtcdserver = []byte{
-	// 380 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0xd2, 0xdd, 0x6e, 0xda, 0x30,
-	0x14, 0x07, 0x70, 0x0c, 0xe1, 0xcb, 0x63, 0x1b, 0xb3, 0xd0, 0x74, 0x84, 0xa6, 0x2c, 0x42, 0xbb,
-	0xc8, 0xd5, 0xf6, 0x0e, 0x2c, 0x5c, 0x44, 0x2a, 0x15, 0x0d, 0x15, 0xbd, 0x76, 0xc9, 0x29, 0x58,
-	0x02, 0x4c, 0x1d, 0x07, 0xf1, 0x06, 0x7d, 0x85, 0x3e, 0x12, 0x97, 0x7d, 0x82, 0xaa, 0xa5, 0x2f,
-	0x52, 0x39, 0x24, 0xc4, 0xed, 0x5d, 0xf4, 0xfb, 0x9f, 0x1c, 0x1f, 0x7f, 0xd0, 0x2e, 0xea, 0x79,
-	0x9c, 0xa0, 0xda, 0xa1, 0xfa, 0xbb, 0x55, 0x52, 0x4b, 0xd6, 0x29, 0x65, 0x7b, 0xdb, 0xef, 0x2d,
-	0xe4, 0x42, 0x66, 0xc1, 0x3f, 0xf3, 0x75, 0xaa, 0x19, 0x3c, 0x38, 0xb4, 0x19, 0xe1, 0x7d, 0x8a,
-	0x89, 0x66, 0x3d, 0x5a, 0x0d, 0x03, 0x20, 0x1e, 0xf1, 0x9d, 0xa1, 0x73, 0x78, 0xfe, 0x5d, 0x89,
-	0xaa, 0x61, 0xc0, 0x7e, 0xd1, 0xc6, 0x18, 0xf5, 0x52, 0xc6, 0x50, 0xf5, 0x88, 0xdf, 0xce, 0x93,
-	0xdc, 0x18, 0x50, 0x67, 0xc2, 0xf5, 0x12, 0x6a, 0x56, 0x96, 0x09, 0xfb, 0x49, 0x6b, 0x33, 0xbe,
-	0x02, 0xc7, 0x0a, 0x0c, 0x18, 0x0f, 0x84, 0x82, 0xba, 0x47, 0xfc, 0x56, 0xe1, 0x81, 0x50, 0x6c,
-	0x40, 0xdb, 0x13, 0x85, 0xbb, 0x19, 0x5f, 0xa5, 0x08, 0x0d, 0xeb, 0xaf, 0x92, 0x8b, 0x9a, 0x70,
-	0x13, 0xe3, 0x1e, 0x9a, 0xd6, 0xa0, 0x25, 0x17, 0x35, 0xa3, 0xbd, 0x48, 0x34, 0xb4, 0xce, 0xab,
-	0x90, 0xa8, 0x64, 0xf6, 0x87, 0xd2, 0xd1, 0x7e, 0x2b, 0x14, 0xd7, 0x42, 0x6e, 0xa0, 0xed, 0x11,
-	0xbf, 0x96, 0x37, 0xb2, 0xdc, 0xec, 0xed, 0x86, 0x0b, 0x0d, 0xd4, 0x1a, 0x35, 0x13, 0xd6, 0xa7,
-	0xf5, 0xa9, 0xd8, 0xcc, 0x11, 0xbe, 0x58, 0x33, 0x9c, 0xc8, 0xac, 0x1f, 0xe1, 0x3c, 0x55, 0x89,
-	0xd8, 0x21, 0x74, 0xac, 0x5f, 0x4b, 0x36, 0x67, 0x3a, 0x95, 0x4a, 0x63, 0x0c, 0x5f, 0xad, 0x82,
-	0xdc, 0x4c, 0x7a, 0x95, 0x4a, 0x95, 0xae, 0xe1, 0x9b, 0x9d, 0x9e, 0xcc, 0x4c, 0x75, 0x2d, 0xd6,
-	0x08, 0xdf, 0xad, 0xa9, 0x33, 0xc9, 0xba, 0x6a, 0x85, 0x7c, 0x0d, 0xdd, 0x0f, 0x5d, 0x33, 0x63,
-	0xae, 0xb9, 0xe8, 0x3b, 0x85, 0xc9, 0x12, 0x7e, 0x58, 0xa7, 0x52, 0xe0, 0xe0, 0x82, 0xb6, 0xc6,
-	0xa8, 0x79, 0xcc, 0x35, 0x37, 0x9d, 0x2e, 0x65, 0x8c, 0x9f, 0x5e, 0x43, 0x6e, 0x66, 0x87, 0xff,
-	0x57, 0x69, 0xa2, 0x51, 0x85, 0x41, 0xf6, 0x28, 0xce, 0xb7, 0x70, 0xe6, 0x61, 0xef, 0xf0, 0xea,
-	0x56, 0x0e, 0x47, 0x97, 0x3c, 0x1d, 0x5d, 0xf2, 0x72, 0x74, 0xc9, 0xe3, 0x9b, 0x5b, 0x79, 0x0f,
-	0x00, 0x00, 0xff, 0xff, 0xee, 0x40, 0xba, 0xd6, 0xa4, 0x02, 0x00, 0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.pb.go b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.pb.go
deleted file mode 100644
index b170499..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.pb.go
+++ /dev/null
@@ -1,2127 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: raft_internal.proto
-
-package etcdserverpb
-
-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
-
-type RequestHeader struct {
-	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	// username is a username that is associated with an auth token of gRPC connection
-	Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"`
-	// auth_revision is a revision number of auth.authStore. It is not related to mvcc
-	AuthRevision uint64 `protobuf:"varint,3,opt,name=auth_revision,json=authRevision,proto3" json:"auth_revision,omitempty"`
-}
-
-func (m *RequestHeader) Reset()                    { *m = RequestHeader{} }
-func (m *RequestHeader) String() string            { return proto.CompactTextString(m) }
-func (*RequestHeader) ProtoMessage()               {}
-func (*RequestHeader) Descriptor() ([]byte, []int) { return fileDescriptorRaftInternal, []int{0} }
-
-// An InternalRaftRequest is the union of all requests which can be
-// sent via raft.
-type InternalRaftRequest struct {
-	Header                   *RequestHeader                   `protobuf:"bytes,100,opt,name=header" json:"header,omitempty"`
-	ID                       uint64                           `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	V2                       *Request                         `protobuf:"bytes,2,opt,name=v2" json:"v2,omitempty"`
-	Range                    *RangeRequest                    `protobuf:"bytes,3,opt,name=range" json:"range,omitempty"`
-	Put                      *PutRequest                      `protobuf:"bytes,4,opt,name=put" json:"put,omitempty"`
-	DeleteRange              *DeleteRangeRequest              `protobuf:"bytes,5,opt,name=delete_range,json=deleteRange" json:"delete_range,omitempty"`
-	Txn                      *TxnRequest                      `protobuf:"bytes,6,opt,name=txn" json:"txn,omitempty"`
-	Compaction               *CompactionRequest               `protobuf:"bytes,7,opt,name=compaction" json:"compaction,omitempty"`
-	LeaseGrant               *LeaseGrantRequest               `protobuf:"bytes,8,opt,name=lease_grant,json=leaseGrant" json:"lease_grant,omitempty"`
-	LeaseRevoke              *LeaseRevokeRequest              `protobuf:"bytes,9,opt,name=lease_revoke,json=leaseRevoke" json:"lease_revoke,omitempty"`
-	Alarm                    *AlarmRequest                    `protobuf:"bytes,10,opt,name=alarm" json:"alarm,omitempty"`
-	LeaseCheckpoint          *LeaseCheckpointRequest          `protobuf:"bytes,11,opt,name=lease_checkpoint,json=leaseCheckpoint" json:"lease_checkpoint,omitempty"`
-	AuthEnable               *AuthEnableRequest               `protobuf:"bytes,1000,opt,name=auth_enable,json=authEnable" json:"auth_enable,omitempty"`
-	AuthDisable              *AuthDisableRequest              `protobuf:"bytes,1011,opt,name=auth_disable,json=authDisable" json:"auth_disable,omitempty"`
-	Authenticate             *InternalAuthenticateRequest     `protobuf:"bytes,1012,opt,name=authenticate" json:"authenticate,omitempty"`
-	AuthUserAdd              *AuthUserAddRequest              `protobuf:"bytes,1100,opt,name=auth_user_add,json=authUserAdd" json:"auth_user_add,omitempty"`
-	AuthUserDelete           *AuthUserDeleteRequest           `protobuf:"bytes,1101,opt,name=auth_user_delete,json=authUserDelete" json:"auth_user_delete,omitempty"`
-	AuthUserGet              *AuthUserGetRequest              `protobuf:"bytes,1102,opt,name=auth_user_get,json=authUserGet" json:"auth_user_get,omitempty"`
-	AuthUserChangePassword   *AuthUserChangePasswordRequest   `protobuf:"bytes,1103,opt,name=auth_user_change_password,json=authUserChangePassword" json:"auth_user_change_password,omitempty"`
-	AuthUserGrantRole        *AuthUserGrantRoleRequest        `protobuf:"bytes,1104,opt,name=auth_user_grant_role,json=authUserGrantRole" json:"auth_user_grant_role,omitempty"`
-	AuthUserRevokeRole       *AuthUserRevokeRoleRequest       `protobuf:"bytes,1105,opt,name=auth_user_revoke_role,json=authUserRevokeRole" json:"auth_user_revoke_role,omitempty"`
-	AuthUserList             *AuthUserListRequest             `protobuf:"bytes,1106,opt,name=auth_user_list,json=authUserList" json:"auth_user_list,omitempty"`
-	AuthRoleList             *AuthRoleListRequest             `protobuf:"bytes,1107,opt,name=auth_role_list,json=authRoleList" json:"auth_role_list,omitempty"`
-	AuthRoleAdd              *AuthRoleAddRequest              `protobuf:"bytes,1200,opt,name=auth_role_add,json=authRoleAdd" json:"auth_role_add,omitempty"`
-	AuthRoleDelete           *AuthRoleDeleteRequest           `protobuf:"bytes,1201,opt,name=auth_role_delete,json=authRoleDelete" json:"auth_role_delete,omitempty"`
-	AuthRoleGet              *AuthRoleGetRequest              `protobuf:"bytes,1202,opt,name=auth_role_get,json=authRoleGet" json:"auth_role_get,omitempty"`
-	AuthRoleGrantPermission  *AuthRoleGrantPermissionRequest  `protobuf:"bytes,1203,opt,name=auth_role_grant_permission,json=authRoleGrantPermission" json:"auth_role_grant_permission,omitempty"`
-	AuthRoleRevokePermission *AuthRoleRevokePermissionRequest `protobuf:"bytes,1204,opt,name=auth_role_revoke_permission,json=authRoleRevokePermission" json:"auth_role_revoke_permission,omitempty"`
-}
-
-func (m *InternalRaftRequest) Reset()                    { *m = InternalRaftRequest{} }
-func (m *InternalRaftRequest) String() string            { return proto.CompactTextString(m) }
-func (*InternalRaftRequest) ProtoMessage()               {}
-func (*InternalRaftRequest) Descriptor() ([]byte, []int) { return fileDescriptorRaftInternal, []int{1} }
-
-type EmptyResponse struct {
-}
-
-func (m *EmptyResponse) Reset()                    { *m = EmptyResponse{} }
-func (m *EmptyResponse) String() string            { return proto.CompactTextString(m) }
-func (*EmptyResponse) ProtoMessage()               {}
-func (*EmptyResponse) Descriptor() ([]byte, []int) { return fileDescriptorRaftInternal, []int{2} }
-
-// What is the difference between AuthenticateRequest (defined in rpc.proto) and InternalAuthenticateRequest?
-// InternalAuthenticateRequest has a member that is filled by etcdserver and shouldn't be user-facing.
-// For avoiding misusage the field, we have an internal version of AuthenticateRequest.
-type InternalAuthenticateRequest struct {
-	Name     string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
-	// simple_token is generated in API layer (etcdserver/v3_server.go)
-	SimpleToken string `protobuf:"bytes,3,opt,name=simple_token,json=simpleToken,proto3" json:"simple_token,omitempty"`
-}
-
-func (m *InternalAuthenticateRequest) Reset()         { *m = InternalAuthenticateRequest{} }
-func (m *InternalAuthenticateRequest) String() string { return proto.CompactTextString(m) }
-func (*InternalAuthenticateRequest) ProtoMessage()    {}
-func (*InternalAuthenticateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRaftInternal, []int{3}
-}
-
-func init() {
-	proto.RegisterType((*RequestHeader)(nil), "etcdserverpb.RequestHeader")
-	proto.RegisterType((*InternalRaftRequest)(nil), "etcdserverpb.InternalRaftRequest")
-	proto.RegisterType((*EmptyResponse)(nil), "etcdserverpb.EmptyResponse")
-	proto.RegisterType((*InternalAuthenticateRequest)(nil), "etcdserverpb.InternalAuthenticateRequest")
-}
-func (m *RequestHeader) 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 *RequestHeader) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.ID))
-	}
-	if len(m.Username) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Username)))
-		i += copy(dAtA[i:], m.Username)
-	}
-	if m.AuthRevision != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRevision))
-	}
-	return i, nil
-}
-
-func (m *InternalRaftRequest) 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 *InternalRaftRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.ID))
-	}
-	if m.V2 != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.V2.Size()))
-		n1, err := m.V2.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n1
-	}
-	if m.Range != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Range.Size()))
-		n2, err := m.Range.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n2
-	}
-	if m.Put != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Put.Size()))
-		n3, err := m.Put.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n3
-	}
-	if m.DeleteRange != nil {
-		dAtA[i] = 0x2a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.DeleteRange.Size()))
-		n4, err := m.DeleteRange.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n4
-	}
-	if m.Txn != nil {
-		dAtA[i] = 0x32
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Txn.Size()))
-		n5, err := m.Txn.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n5
-	}
-	if m.Compaction != nil {
-		dAtA[i] = 0x3a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Compaction.Size()))
-		n6, err := m.Compaction.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n6
-	}
-	if m.LeaseGrant != nil {
-		dAtA[i] = 0x42
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.LeaseGrant.Size()))
-		n7, err := m.LeaseGrant.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n7
-	}
-	if m.LeaseRevoke != nil {
-		dAtA[i] = 0x4a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.LeaseRevoke.Size()))
-		n8, err := m.LeaseRevoke.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n8
-	}
-	if m.Alarm != nil {
-		dAtA[i] = 0x52
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Alarm.Size()))
-		n9, err := m.Alarm.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n9
-	}
-	if m.LeaseCheckpoint != nil {
-		dAtA[i] = 0x5a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.LeaseCheckpoint.Size()))
-		n10, err := m.LeaseCheckpoint.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n10
-	}
-	if m.Header != nil {
-		dAtA[i] = 0xa2
-		i++
-		dAtA[i] = 0x6
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Header.Size()))
-		n11, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n11
-	}
-	if m.AuthEnable != nil {
-		dAtA[i] = 0xc2
-		i++
-		dAtA[i] = 0x3e
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthEnable.Size()))
-		n12, err := m.AuthEnable.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n12
-	}
-	if m.AuthDisable != nil {
-		dAtA[i] = 0x9a
-		i++
-		dAtA[i] = 0x3f
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthDisable.Size()))
-		n13, err := m.AuthDisable.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n13
-	}
-	if m.Authenticate != nil {
-		dAtA[i] = 0xa2
-		i++
-		dAtA[i] = 0x3f
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.Authenticate.Size()))
-		n14, err := m.Authenticate.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n14
-	}
-	if m.AuthUserAdd != nil {
-		dAtA[i] = 0xe2
-		i++
-		dAtA[i] = 0x44
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserAdd.Size()))
-		n15, err := m.AuthUserAdd.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n15
-	}
-	if m.AuthUserDelete != nil {
-		dAtA[i] = 0xea
-		i++
-		dAtA[i] = 0x44
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserDelete.Size()))
-		n16, err := m.AuthUserDelete.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n16
-	}
-	if m.AuthUserGet != nil {
-		dAtA[i] = 0xf2
-		i++
-		dAtA[i] = 0x44
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserGet.Size()))
-		n17, err := m.AuthUserGet.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n17
-	}
-	if m.AuthUserChangePassword != nil {
-		dAtA[i] = 0xfa
-		i++
-		dAtA[i] = 0x44
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserChangePassword.Size()))
-		n18, err := m.AuthUserChangePassword.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n18
-	}
-	if m.AuthUserGrantRole != nil {
-		dAtA[i] = 0x82
-		i++
-		dAtA[i] = 0x45
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserGrantRole.Size()))
-		n19, err := m.AuthUserGrantRole.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n19
-	}
-	if m.AuthUserRevokeRole != nil {
-		dAtA[i] = 0x8a
-		i++
-		dAtA[i] = 0x45
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserRevokeRole.Size()))
-		n20, err := m.AuthUserRevokeRole.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n20
-	}
-	if m.AuthUserList != nil {
-		dAtA[i] = 0x92
-		i++
-		dAtA[i] = 0x45
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthUserList.Size()))
-		n21, err := m.AuthUserList.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n21
-	}
-	if m.AuthRoleList != nil {
-		dAtA[i] = 0x9a
-		i++
-		dAtA[i] = 0x45
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleList.Size()))
-		n22, err := m.AuthRoleList.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n22
-	}
-	if m.AuthRoleAdd != nil {
-		dAtA[i] = 0x82
-		i++
-		dAtA[i] = 0x4b
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleAdd.Size()))
-		n23, err := m.AuthRoleAdd.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n23
-	}
-	if m.AuthRoleDelete != nil {
-		dAtA[i] = 0x8a
-		i++
-		dAtA[i] = 0x4b
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleDelete.Size()))
-		n24, err := m.AuthRoleDelete.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n24
-	}
-	if m.AuthRoleGet != nil {
-		dAtA[i] = 0x92
-		i++
-		dAtA[i] = 0x4b
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleGet.Size()))
-		n25, err := m.AuthRoleGet.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n25
-	}
-	if m.AuthRoleGrantPermission != nil {
-		dAtA[i] = 0x9a
-		i++
-		dAtA[i] = 0x4b
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleGrantPermission.Size()))
-		n26, err := m.AuthRoleGrantPermission.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n26
-	}
-	if m.AuthRoleRevokePermission != nil {
-		dAtA[i] = 0xa2
-		i++
-		dAtA[i] = 0x4b
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(m.AuthRoleRevokePermission.Size()))
-		n27, err := m.AuthRoleRevokePermission.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n27
-	}
-	return i, nil
-}
-
-func (m *EmptyResponse) 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 *EmptyResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *InternalAuthenticateRequest) 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 *InternalAuthenticateRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Password) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.Password)))
-		i += copy(dAtA[i:], m.Password)
-	}
-	if len(m.SimpleToken) > 0 {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRaftInternal(dAtA, i, uint64(len(m.SimpleToken)))
-		i += copy(dAtA[i:], m.SimpleToken)
-	}
-	return i, nil
-}
-
-func encodeVarintRaftInternal(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 *RequestHeader) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRaftInternal(uint64(m.ID))
-	}
-	l = len(m.Username)
-	if l > 0 {
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRevision != 0 {
-		n += 1 + sovRaftInternal(uint64(m.AuthRevision))
-	}
-	return n
-}
-
-func (m *InternalRaftRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRaftInternal(uint64(m.ID))
-	}
-	if m.V2 != nil {
-		l = m.V2.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Range != nil {
-		l = m.Range.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Put != nil {
-		l = m.Put.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.DeleteRange != nil {
-		l = m.DeleteRange.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Txn != nil {
-		l = m.Txn.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Compaction != nil {
-		l = m.Compaction.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.LeaseGrant != nil {
-		l = m.LeaseGrant.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.LeaseRevoke != nil {
-		l = m.LeaseRevoke.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Alarm != nil {
-		l = m.Alarm.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.LeaseCheckpoint != nil {
-		l = m.LeaseCheckpoint.Size()
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthEnable != nil {
-		l = m.AuthEnable.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthDisable != nil {
-		l = m.AuthDisable.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.Authenticate != nil {
-		l = m.Authenticate.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserAdd != nil {
-		l = m.AuthUserAdd.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserDelete != nil {
-		l = m.AuthUserDelete.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserGet != nil {
-		l = m.AuthUserGet.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserChangePassword != nil {
-		l = m.AuthUserChangePassword.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserGrantRole != nil {
-		l = m.AuthUserGrantRole.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserRevokeRole != nil {
-		l = m.AuthUserRevokeRole.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthUserList != nil {
-		l = m.AuthUserList.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleList != nil {
-		l = m.AuthRoleList.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleAdd != nil {
-		l = m.AuthRoleAdd.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleDelete != nil {
-		l = m.AuthRoleDelete.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleGet != nil {
-		l = m.AuthRoleGet.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleGrantPermission != nil {
-		l = m.AuthRoleGrantPermission.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	if m.AuthRoleRevokePermission != nil {
-		l = m.AuthRoleRevokePermission.Size()
-		n += 2 + l + sovRaftInternal(uint64(l))
-	}
-	return n
-}
-
-func (m *EmptyResponse) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *InternalAuthenticateRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	l = len(m.Password)
-	if l > 0 {
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	l = len(m.SimpleToken)
-	if l > 0 {
-		n += 1 + l + sovRaftInternal(uint64(l))
-	}
-	return n
-}
-
-func sovRaftInternal(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozRaftInternal(x uint64) (n int) {
-	return sovRaftInternal(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *RequestHeader) 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 ErrIntOverflowRaftInternal
-			}
-			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: RequestHeader: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: RequestHeader: 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 ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Username", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Username = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRevision", wireType)
-			}
-			m.AuthRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.AuthRevision |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaftInternal(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *InternalRaftRequest) 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 ErrIntOverflowRaftInternal
-			}
-			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: InternalRaftRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: InternalRaftRequest: 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 ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field V2", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.V2 == nil {
-				m.V2 = &Request{}
-			}
-			if err := m.V2.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Range == nil {
-				m.Range = &RangeRequest{}
-			}
-			if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Put", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Put == nil {
-				m.Put = &PutRequest{}
-			}
-			if err := m.Put.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 5:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field DeleteRange", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.DeleteRange == nil {
-				m.DeleteRange = &DeleteRangeRequest{}
-			}
-			if err := m.DeleteRange.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 6:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Txn", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Txn == nil {
-				m.Txn = &TxnRequest{}
-			}
-			if err := m.Txn.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 7:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Compaction", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Compaction == nil {
-				m.Compaction = &CompactionRequest{}
-			}
-			if err := m.Compaction.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 8:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field LeaseGrant", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.LeaseGrant == nil {
-				m.LeaseGrant = &LeaseGrantRequest{}
-			}
-			if err := m.LeaseGrant.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 9:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field LeaseRevoke", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.LeaseRevoke == nil {
-				m.LeaseRevoke = &LeaseRevokeRequest{}
-			}
-			if err := m.LeaseRevoke.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 10:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Alarm == nil {
-				m.Alarm = &AlarmRequest{}
-			}
-			if err := m.Alarm.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 11:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field LeaseCheckpoint", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.LeaseCheckpoint == nil {
-				m.LeaseCheckpoint = &LeaseCheckpointRequest{}
-			}
-			if err := m.LeaseCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 100:
-			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 ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &RequestHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1000:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthEnable", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthEnable == nil {
-				m.AuthEnable = &AuthEnableRequest{}
-			}
-			if err := m.AuthEnable.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1011:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthDisable", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthDisable == nil {
-				m.AuthDisable = &AuthDisableRequest{}
-			}
-			if err := m.AuthDisable.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1012:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Authenticate", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Authenticate == nil {
-				m.Authenticate = &InternalAuthenticateRequest{}
-			}
-			if err := m.Authenticate.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1100:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserAdd", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserAdd == nil {
-				m.AuthUserAdd = &AuthUserAddRequest{}
-			}
-			if err := m.AuthUserAdd.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1101:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserDelete", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserDelete == nil {
-				m.AuthUserDelete = &AuthUserDeleteRequest{}
-			}
-			if err := m.AuthUserDelete.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1102:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserGet", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserGet == nil {
-				m.AuthUserGet = &AuthUserGetRequest{}
-			}
-			if err := m.AuthUserGet.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1103:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserChangePassword", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserChangePassword == nil {
-				m.AuthUserChangePassword = &AuthUserChangePasswordRequest{}
-			}
-			if err := m.AuthUserChangePassword.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1104:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserGrantRole", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserGrantRole == nil {
-				m.AuthUserGrantRole = &AuthUserGrantRoleRequest{}
-			}
-			if err := m.AuthUserGrantRole.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1105:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserRevokeRole", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserRevokeRole == nil {
-				m.AuthUserRevokeRole = &AuthUserRevokeRoleRequest{}
-			}
-			if err := m.AuthUserRevokeRole.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1106:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthUserList", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthUserList == nil {
-				m.AuthUserList = &AuthUserListRequest{}
-			}
-			if err := m.AuthUserList.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1107:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleList", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleList == nil {
-				m.AuthRoleList = &AuthRoleListRequest{}
-			}
-			if err := m.AuthRoleList.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1200:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleAdd", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleAdd == nil {
-				m.AuthRoleAdd = &AuthRoleAddRequest{}
-			}
-			if err := m.AuthRoleAdd.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1201:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleDelete", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleDelete == nil {
-				m.AuthRoleDelete = &AuthRoleDeleteRequest{}
-			}
-			if err := m.AuthRoleDelete.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1202:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleGet", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleGet == nil {
-				m.AuthRoleGet = &AuthRoleGetRequest{}
-			}
-			if err := m.AuthRoleGet.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1203:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleGrantPermission", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleGrantPermission == nil {
-				m.AuthRoleGrantPermission = &AuthRoleGrantPermissionRequest{}
-			}
-			if err := m.AuthRoleGrantPermission.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 1204:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AuthRoleRevokePermission", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.AuthRoleRevokePermission == nil {
-				m.AuthRoleRevokePermission = &AuthRoleRevokePermissionRequest{}
-			}
-			if err := m.AuthRoleRevokePermission.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaftInternal(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *EmptyResponse) 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 ErrIntOverflowRaftInternal
-			}
-			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: EmptyResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: EmptyResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaftInternal(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *InternalAuthenticateRequest) 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 ErrIntOverflowRaftInternal
-			}
-			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: InternalAuthenticateRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: InternalAuthenticateRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Password = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field SimpleToken", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaftInternal
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.SimpleToken = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaftInternal(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaftInternal
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func skipRaftInternal(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, ErrIntOverflowRaftInternal
-			}
-			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, ErrIntOverflowRaftInternal
-				}
-				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, ErrIntOverflowRaftInternal
-				}
-				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, ErrInvalidLengthRaftInternal
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowRaftInternal
-					}
-					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 := skipRaftInternal(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 (
-	ErrInvalidLengthRaftInternal = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowRaftInternal   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("raft_internal.proto", fileDescriptorRaftInternal) }
-
-var fileDescriptorRaftInternal = []byte{
-	// 867 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x96, 0xcb, 0x52, 0x33, 0x45,
-	0x14, 0xc7, 0x49, 0xb8, 0xa6, 0x13, 0x2e, 0x36, 0xa0, 0x6d, 0xa8, 0x8a, 0x10, 0xbc, 0xe0, 0x0d,
-	0xad, 0xf0, 0x00, 0x1a, 0x13, 0x0a, 0xa8, 0xa2, 0x90, 0x9a, 0xc2, 0x2a, 0xab, 0x5c, 0x8c, 0xcd,
-	0xcc, 0x21, 0x19, 0x99, 0xcc, 0x8c, 0x3d, 0x9d, 0x88, 0x6f, 0xe2, 0x63, 0x78, 0xdb, 0xbb, 0x65,
-	0xe1, 0x05, 0xf5, 0x05, 0x14, 0x37, 0xee, 0xbf, 0xef, 0x01, 0xbe, 0xea, 0xcb, 0xf4, 0x64, 0x92,
-	0x0e, 0xbb, 0xc9, 0x39, 0xff, 0xf3, 0xfb, 0x9f, 0x99, 0x3e, 0x07, 0x1a, 0x6d, 0x32, 0x7a, 0xc3,
-	0xdd, 0x20, 0xe2, 0xc0, 0x22, 0x1a, 0x1e, 0x26, 0x2c, 0xe6, 0x31, 0xae, 0x01, 0xf7, 0xfc, 0x14,
-	0xd8, 0x08, 0x58, 0x72, 0x5d, 0xdf, 0xea, 0xc5, 0xbd, 0x58, 0x26, 0x3e, 0x10, 0x4f, 0x4a, 0x53,
-	0xdf, 0xc8, 0x35, 0x3a, 0x52, 0x61, 0x89, 0xa7, 0x1e, 0x9b, 0x5f, 0xa2, 0x55, 0x07, 0xbe, 0x1e,
-	0x42, 0xca, 0x4f, 0x81, 0xfa, 0xc0, 0xf0, 0x1a, 0x2a, 0x9f, 0x75, 0x49, 0x69, 0xb7, 0x74, 0xb0,
-	0xe0, 0x94, 0xcf, 0xba, 0xb8, 0x8e, 0x56, 0x86, 0xa9, 0xb0, 0x1c, 0x00, 0x29, 0xef, 0x96, 0x0e,
-	0x2a, 0x8e, 0xf9, 0x8d, 0xf7, 0xd1, 0x2a, 0x1d, 0xf2, 0xbe, 0xcb, 0x60, 0x14, 0xa4, 0x41, 0x1c,
-	0x91, 0x79, 0x59, 0x56, 0x13, 0x41, 0x47, 0xc7, 0x9a, 0xbf, 0xac, 0xa3, 0xcd, 0x33, 0xdd, 0xb5,
-	0x43, 0x6f, 0xb8, 0xb6, 0x9b, 0x32, 0x7a, 0x03, 0x95, 0x47, 0x2d, 0x69, 0x51, 0x6d, 0x6d, 0x1f,
-	0x8e, 0xbf, 0xd7, 0xa1, 0x2e, 0x71, 0xca, 0xa3, 0x16, 0xfe, 0x10, 0x2d, 0x32, 0x1a, 0xf5, 0x40,
-	0x7a, 0x55, 0x5b, 0xf5, 0x09, 0xa5, 0x48, 0x65, 0x72, 0x25, 0xc4, 0xef, 0xa0, 0xf9, 0x64, 0xc8,
-	0xc9, 0x82, 0xd4, 0x93, 0xa2, 0xfe, 0x72, 0x98, 0xf5, 0xe3, 0x08, 0x11, 0xee, 0xa0, 0x9a, 0x0f,
-	0x21, 0x70, 0x70, 0x95, 0xc9, 0xa2, 0x2c, 0xda, 0x2d, 0x16, 0x75, 0xa5, 0xa2, 0x60, 0x55, 0xf5,
-	0xf3, 0x98, 0x30, 0xe4, 0x77, 0x11, 0x59, 0xb2, 0x19, 0x5e, 0xdd, 0x45, 0xc6, 0x90, 0xdf, 0x45,
-	0xf8, 0x23, 0x84, 0xbc, 0x78, 0x90, 0x50, 0x8f, 0x8b, 0xef, 0xb7, 0x2c, 0x4b, 0x5e, 0x2b, 0x96,
-	0x74, 0x4c, 0x3e, 0xab, 0x1c, 0x2b, 0xc1, 0x1f, 0xa3, 0x6a, 0x08, 0x34, 0x05, 0xb7, 0xc7, 0x68,
-	0xc4, 0xc9, 0x8a, 0x8d, 0x70, 0x2e, 0x04, 0x27, 0x22, 0x6f, 0x08, 0xa1, 0x09, 0x89, 0x77, 0x56,
-	0x04, 0x06, 0xa3, 0xf8, 0x16, 0x48, 0xc5, 0xf6, 0xce, 0x12, 0xe1, 0x48, 0x81, 0x79, 0xe7, 0x30,
-	0x8f, 0x89, 0x63, 0xa1, 0x21, 0x65, 0x03, 0x82, 0x6c, 0xc7, 0xd2, 0x16, 0x29, 0x73, 0x2c, 0x52,
-	0x88, 0x3f, 0x45, 0x1b, 0xca, 0xd6, 0xeb, 0x83, 0x77, 0x9b, 0xc4, 0x41, 0xc4, 0x49, 0x55, 0x16,
-	0xbf, 0x6e, 0xb1, 0xee, 0x18, 0x51, 0x86, 0x59, 0x0f, 0x8b, 0x71, 0x7c, 0x84, 0x96, 0xfa, 0x72,
-	0x86, 0x89, 0x2f, 0x31, 0x3b, 0xd6, 0x21, 0x52, 0x63, 0xee, 0x68, 0x29, 0x6e, 0xa3, 0xaa, 0x1c,
-	0x61, 0x88, 0xe8, 0x75, 0x08, 0xe4, 0x7f, 0xeb, 0x09, 0xb4, 0x87, 0xbc, 0x7f, 0x2c, 0x05, 0xe6,
-	0xfb, 0x51, 0x13, 0xc2, 0x5d, 0x24, 0x07, 0xde, 0xf5, 0x83, 0x54, 0x32, 0x9e, 0x2d, 0xdb, 0x3e,
-	0xa0, 0x60, 0x74, 0x95, 0xc2, 0x7c, 0x40, 0x9a, 0xc7, 0xf0, 0x85, 0xa2, 0x40, 0xc4, 0x03, 0x8f,
-	0x72, 0x20, 0xcf, 0x15, 0xe5, 0xed, 0x22, 0x25, 0x5b, 0xa4, 0xf6, 0x98, 0x34, 0xc3, 0x15, 0xea,
-	0xf1, 0xb1, 0xde, 0x4d, 0xb1, 0xac, 0x2e, 0xf5, 0x7d, 0xf2, 0xeb, 0xca, 0xac, 0xb6, 0x3e, 0x4b,
-	0x81, 0xb5, 0x7d, 0xbf, 0xd0, 0x96, 0x8e, 0xe1, 0x0b, 0xb4, 0x91, 0x63, 0xd4, 0x90, 0x93, 0xdf,
-	0x14, 0x69, 0xdf, 0x4e, 0xd2, 0xdb, 0xa1, 0x61, 0x6b, 0xb4, 0x10, 0x2e, 0xb6, 0xd5, 0x03, 0x4e,
-	0x7e, 0x7f, 0xb2, 0xad, 0x13, 0xe0, 0x53, 0x6d, 0x9d, 0x00, 0xc7, 0x3d, 0xf4, 0x6a, 0x8e, 0xf1,
-	0xfa, 0x62, 0xed, 0xdc, 0x84, 0xa6, 0xe9, 0x37, 0x31, 0xf3, 0xc9, 0x1f, 0x0a, 0xf9, 0xae, 0x1d,
-	0xd9, 0x91, 0xea, 0x4b, 0x2d, 0xce, 0xe8, 0x2f, 0x53, 0x6b, 0x1a, 0x7f, 0x8e, 0xb6, 0xc6, 0xfa,
-	0x15, 0xfb, 0xe2, 0xb2, 0x38, 0x04, 0xf2, 0xa0, 0x3c, 0xde, 0x9c, 0xd1, 0xb6, 0xdc, 0xb5, 0x38,
-	0x3f, 0xea, 0x97, 0xe8, 0x64, 0x06, 0x7f, 0x81, 0xb6, 0x73, 0xb2, 0x5a, 0x3d, 0x85, 0xfe, 0x53,
-	0xa1, 0xdf, 0xb2, 0xa3, 0xf5, 0x0e, 0x8e, 0xb1, 0x31, 0x9d, 0x4a, 0xe1, 0x53, 0xb4, 0x96, 0xc3,
-	0xc3, 0x20, 0xe5, 0xe4, 0x2f, 0x45, 0xdd, 0xb3, 0x53, 0xcf, 0x83, 0x94, 0x17, 0xe6, 0x28, 0x0b,
-	0x1a, 0x92, 0x68, 0x4d, 0x91, 0xfe, 0x9e, 0x49, 0x12, 0xd6, 0x53, 0xa4, 0x2c, 0x68, 0x8e, 0x5e,
-	0x92, 0xc4, 0x44, 0x7e, 0x5f, 0x99, 0x75, 0xf4, 0xa2, 0x66, 0x72, 0x22, 0x75, 0xcc, 0x4c, 0xa4,
-	0xc4, 0xe8, 0x89, 0xfc, 0xa1, 0x32, 0x6b, 0x22, 0x45, 0x95, 0x65, 0x22, 0xf3, 0x70, 0xb1, 0x2d,
-	0x31, 0x91, 0x3f, 0x3e, 0xd9, 0xd6, 0xe4, 0x44, 0xea, 0x18, 0xfe, 0x0a, 0xd5, 0xc7, 0x30, 0x72,
-	0x50, 0x12, 0x60, 0x83, 0x20, 0x95, 0xff, 0x18, 0x7f, 0x52, 0xcc, 0xf7, 0x66, 0x30, 0x85, 0xfc,
-	0xd2, 0xa8, 0x33, 0xfe, 0x2b, 0xd4, 0x9e, 0xc7, 0x03, 0xb4, 0x93, 0x7b, 0xe9, 0xd1, 0x19, 0x33,
-	0xfb, 0x59, 0x99, 0xbd, 0x6f, 0x37, 0x53, 0x53, 0x32, 0xed, 0x46, 0xe8, 0x0c, 0x41, 0x73, 0x1d,
-	0xad, 0x1e, 0x0f, 0x12, 0xfe, 0xad, 0x03, 0x69, 0x12, 0x47, 0x29, 0x34, 0x13, 0xb4, 0xf3, 0xc4,
-	0x1f, 0x22, 0x8c, 0xd1, 0x82, 0xbc, 0x2e, 0x94, 0xe4, 0x75, 0x41, 0x3e, 0x8b, 0x6b, 0x84, 0xd9,
-	0x4f, 0x7d, 0x8d, 0xc8, 0x7e, 0xe3, 0x3d, 0x54, 0x4b, 0x83, 0x41, 0x12, 0x82, 0xcb, 0xe3, 0x5b,
-	0x50, 0xb7, 0x88, 0x8a, 0x53, 0x55, 0xb1, 0x2b, 0x11, 0xfa, 0x64, 0xeb, 0xfe, 0xdf, 0xc6, 0xdc,
-	0xfd, 0x63, 0xa3, 0xf4, 0xf0, 0xd8, 0x28, 0xfd, 0xf3, 0xd8, 0x28, 0x7d, 0xf7, 0x5f, 0x63, 0xee,
-	0x7a, 0x49, 0xde, 0x61, 0x8e, 0x5e, 0x04, 0x00, 0x00, 0xff, 0xff, 0xed, 0x36, 0xf0, 0x6f, 0x1b,
-	0x09, 0x00, 0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.proto b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.proto
deleted file mode 100644
index 7111f45..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal.proto
+++ /dev/null
@@ -1,75 +0,0 @@
-syntax = "proto3";
-package etcdserverpb;
-
-import "gogoproto/gogo.proto";
-import "etcdserver.proto";
-import "rpc.proto";
-
-option (gogoproto.marshaler_all) = true;
-option (gogoproto.sizer_all) = true;
-option (gogoproto.unmarshaler_all) = true;
-option (gogoproto.goproto_getters_all) = false;
-
-message RequestHeader {
-  uint64 ID = 1;
-  // username is a username that is associated with an auth token of gRPC connection
-  string username = 2;
-  // auth_revision is a revision number of auth.authStore. It is not related to mvcc
-  uint64 auth_revision = 3;
-}
-
-// An InternalRaftRequest is the union of all requests which can be
-// sent via raft.
-message InternalRaftRequest {
-  RequestHeader header = 100;
-  uint64 ID = 1;
-
-  Request v2 = 2;
-
-  RangeRequest range = 3;
-  PutRequest put = 4;
-  DeleteRangeRequest delete_range = 5;
-  TxnRequest txn = 6;
-  CompactionRequest compaction = 7;
-
-  LeaseGrantRequest lease_grant = 8;
-  LeaseRevokeRequest lease_revoke = 9;
-
-  AlarmRequest alarm = 10;
-
-  LeaseCheckpointRequest lease_checkpoint = 11;
-
-  AuthEnableRequest auth_enable = 1000;
-  AuthDisableRequest auth_disable = 1011;
-
-  InternalAuthenticateRequest authenticate = 1012;
-
-  AuthUserAddRequest auth_user_add = 1100;
-  AuthUserDeleteRequest auth_user_delete = 1101;
-  AuthUserGetRequest auth_user_get = 1102;
-  AuthUserChangePasswordRequest auth_user_change_password = 1103;
-  AuthUserGrantRoleRequest auth_user_grant_role = 1104;
-  AuthUserRevokeRoleRequest auth_user_revoke_role = 1105;
-  AuthUserListRequest auth_user_list = 1106;
-  AuthRoleListRequest auth_role_list = 1107;
-
-  AuthRoleAddRequest auth_role_add = 1200;
-  AuthRoleDeleteRequest auth_role_delete = 1201;
-  AuthRoleGetRequest auth_role_get = 1202;
-  AuthRoleGrantPermissionRequest auth_role_grant_permission = 1203;
-  AuthRoleRevokePermissionRequest auth_role_revoke_permission = 1204;
-}
-
-message EmptyResponse {
-}
-
-// What is the difference between AuthenticateRequest (defined in rpc.proto) and InternalAuthenticateRequest?
-// InternalAuthenticateRequest has a member that is filled by etcdserver and shouldn't be user-facing.
-// For avoiding misusage the field, we have an internal version of AuthenticateRequest.
-message InternalAuthenticateRequest {
-  string name = 1;
-  string password = 2;
-
-  // simple_token is generated in API layer (etcdserver/v3_server.go)
-  string simple_token = 3;
-}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go
deleted file mode 100644
index 3d3536a..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/raft_internal_stringer.go
+++ /dev/null
@@ -1,183 +0,0 @@
-// 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 etcdserverpb
-
-import (
-	"fmt"
-	"strings"
-
-	proto "github.com/golang/protobuf/proto"
-)
-
-// InternalRaftStringer implements custom proto Stringer:
-// redact password, replace value fields with value_size fields.
-type InternalRaftStringer struct {
-	Request *InternalRaftRequest
-}
-
-func (as *InternalRaftStringer) String() string {
-	switch {
-	case as.Request.LeaseGrant != nil:
-		return fmt.Sprintf("header:<%s> lease_grant:<ttl:%d-second id:%016x>",
-			as.Request.Header.String(),
-			as.Request.LeaseGrant.TTL,
-			as.Request.LeaseGrant.ID,
-		)
-	case as.Request.LeaseRevoke != nil:
-		return fmt.Sprintf("header:<%s> lease_revoke:<id:%016x>",
-			as.Request.Header.String(),
-			as.Request.LeaseRevoke.ID,
-		)
-	case as.Request.Authenticate != nil:
-		return fmt.Sprintf("header:<%s> authenticate:<name:%s simple_token:%s>",
-			as.Request.Header.String(),
-			as.Request.Authenticate.Name,
-			as.Request.Authenticate.SimpleToken,
-		)
-	case as.Request.AuthUserAdd != nil:
-		return fmt.Sprintf("header:<%s> auth_user_add:<name:%s>",
-			as.Request.Header.String(),
-			as.Request.AuthUserAdd.Name,
-		)
-	case as.Request.AuthUserChangePassword != nil:
-		return fmt.Sprintf("header:<%s> auth_user_change_password:<name:%s>",
-			as.Request.Header.String(),
-			as.Request.AuthUserChangePassword.Name,
-		)
-	case as.Request.Put != nil:
-		return fmt.Sprintf("header:<%s> put:<%s>",
-			as.Request.Header.String(),
-			NewLoggablePutRequest(as.Request.Put).String(),
-		)
-	case as.Request.Txn != nil:
-		return fmt.Sprintf("header:<%s> txn:<%s>",
-			as.Request.Header.String(),
-			NewLoggableTxnRequest(as.Request.Txn).String(),
-		)
-	default:
-		// nothing to redact
-	}
-	return as.Request.String()
-}
-
-// txnRequestStringer implements a custom proto String to replace value bytes fields with value size
-// fields in any nested txn and put operations.
-type txnRequestStringer struct {
-	Request *TxnRequest
-}
-
-func NewLoggableTxnRequest(request *TxnRequest) *txnRequestStringer {
-	return &txnRequestStringer{request}
-}
-
-func (as *txnRequestStringer) String() string {
-	var compare []string
-	for _, c := range as.Request.Compare {
-		switch cv := c.TargetUnion.(type) {
-		case *Compare_Value:
-			compare = append(compare, newLoggableValueCompare(c, cv).String())
-		default:
-			// nothing to redact
-			compare = append(compare, c.String())
-		}
-	}
-	var success []string
-	for _, s := range as.Request.Success {
-		success = append(success, newLoggableRequestOp(s).String())
-	}
-	var failure []string
-	for _, f := range as.Request.Failure {
-		failure = append(failure, newLoggableRequestOp(f).String())
-	}
-	return fmt.Sprintf("compare:<%s> success:<%s> failure:<%s>",
-		strings.Join(compare, " "),
-		strings.Join(success, " "),
-		strings.Join(failure, " "),
-	)
-}
-
-// requestOpStringer implements a custom proto String to replace value bytes fields with value
-// size fields in any nested txn and put operations.
-type requestOpStringer struct {
-	Op *RequestOp
-}
-
-func newLoggableRequestOp(op *RequestOp) *requestOpStringer {
-	return &requestOpStringer{op}
-}
-
-func (as *requestOpStringer) String() string {
-	switch op := as.Op.Request.(type) {
-	case *RequestOp_RequestPut:
-		return fmt.Sprintf("request_put:<%s>", NewLoggablePutRequest(op.RequestPut).String())
-	case *RequestOp_RequestTxn:
-		return fmt.Sprintf("request_txn:<%s>", NewLoggableTxnRequest(op.RequestTxn).String())
-	default:
-		// nothing to redact
-	}
-	return as.Op.String()
-}
-
-// loggableValueCompare implements a custom proto String for Compare.Value union member types to
-// replace the value bytes field with a value size field.
-// To preserve proto encoding of the key and range_end bytes, a faked out proto type is used here.
-type loggableValueCompare struct {
-	Result    Compare_CompareResult `protobuf:"varint,1,opt,name=result,proto3,enum=etcdserverpb.Compare_CompareResult"`
-	Target    Compare_CompareTarget `protobuf:"varint,2,opt,name=target,proto3,enum=etcdserverpb.Compare_CompareTarget"`
-	Key       []byte                `protobuf:"bytes,3,opt,name=key,proto3"`
-	ValueSize int                   `protobuf:"bytes,7,opt,name=value_size,proto3"`
-	RangeEnd  []byte                `protobuf:"bytes,64,opt,name=range_end,proto3"`
-}
-
-func newLoggableValueCompare(c *Compare, cv *Compare_Value) *loggableValueCompare {
-	return &loggableValueCompare{
-		c.Result,
-		c.Target,
-		c.Key,
-		len(cv.Value),
-		c.RangeEnd,
-	}
-}
-
-func (m *loggableValueCompare) Reset()         { *m = loggableValueCompare{} }
-func (m *loggableValueCompare) String() string { return proto.CompactTextString(m) }
-func (*loggableValueCompare) ProtoMessage()    {}
-
-// loggablePutRequest implements a custom proto String to replace value bytes field with a value
-// size field.
-// To preserve proto encoding of the key bytes, a faked out proto type is used here.
-type loggablePutRequest struct {
-	Key         []byte `protobuf:"bytes,1,opt,name=key,proto3"`
-	ValueSize   int    `protobuf:"varint,2,opt,name=value_size,proto3"`
-	Lease       int64  `protobuf:"varint,3,opt,name=lease,proto3"`
-	PrevKv      bool   `protobuf:"varint,4,opt,name=prev_kv,proto3"`
-	IgnoreValue bool   `protobuf:"varint,5,opt,name=ignore_value,proto3"`
-	IgnoreLease bool   `protobuf:"varint,6,opt,name=ignore_lease,proto3"`
-}
-
-func NewLoggablePutRequest(request *PutRequest) *loggablePutRequest {
-	return &loggablePutRequest{
-		request.Key,
-		len(request.Value),
-		request.Lease,
-		request.PrevKv,
-		request.IgnoreValue,
-		request.IgnoreLease,
-	}
-}
-
-func (m *loggablePutRequest) Reset()         { *m = loggablePutRequest{} }
-func (m *loggablePutRequest) String() string { return proto.CompactTextString(m) }
-func (*loggablePutRequest) ProtoMessage()    {}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.pb.go b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.pb.go
deleted file mode 100644
index 199ee62..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.pb.go
+++ /dev/null
@@ -1,20086 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: rpc.proto
-
-package etcdserverpb
-
-import (
-	"fmt"
-
-	proto "github.com/golang/protobuf/proto"
-
-	math "math"
-
-	_ "github.com/gogo/protobuf/gogoproto"
-
-	mvccpb "go.etcd.io/etcd/mvcc/mvccpb"
-
-	authpb "go.etcd.io/etcd/auth/authpb"
-
-	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
-
-type AlarmType int32
-
-const (
-	AlarmType_NONE    AlarmType = 0
-	AlarmType_NOSPACE AlarmType = 1
-	AlarmType_CORRUPT AlarmType = 2
-)
-
-var AlarmType_name = map[int32]string{
-	0: "NONE",
-	1: "NOSPACE",
-	2: "CORRUPT",
-}
-var AlarmType_value = map[string]int32{
-	"NONE":    0,
-	"NOSPACE": 1,
-	"CORRUPT": 2,
-}
-
-func (x AlarmType) String() string {
-	return proto.EnumName(AlarmType_name, int32(x))
-}
-func (AlarmType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{0} }
-
-type RangeRequest_SortOrder int32
-
-const (
-	RangeRequest_NONE    RangeRequest_SortOrder = 0
-	RangeRequest_ASCEND  RangeRequest_SortOrder = 1
-	RangeRequest_DESCEND RangeRequest_SortOrder = 2
-)
-
-var RangeRequest_SortOrder_name = map[int32]string{
-	0: "NONE",
-	1: "ASCEND",
-	2: "DESCEND",
-}
-var RangeRequest_SortOrder_value = map[string]int32{
-	"NONE":    0,
-	"ASCEND":  1,
-	"DESCEND": 2,
-}
-
-func (x RangeRequest_SortOrder) String() string {
-	return proto.EnumName(RangeRequest_SortOrder_name, int32(x))
-}
-func (RangeRequest_SortOrder) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{1, 0} }
-
-type RangeRequest_SortTarget int32
-
-const (
-	RangeRequest_KEY     RangeRequest_SortTarget = 0
-	RangeRequest_VERSION RangeRequest_SortTarget = 1
-	RangeRequest_CREATE  RangeRequest_SortTarget = 2
-	RangeRequest_MOD     RangeRequest_SortTarget = 3
-	RangeRequest_VALUE   RangeRequest_SortTarget = 4
-)
-
-var RangeRequest_SortTarget_name = map[int32]string{
-	0: "KEY",
-	1: "VERSION",
-	2: "CREATE",
-	3: "MOD",
-	4: "VALUE",
-}
-var RangeRequest_SortTarget_value = map[string]int32{
-	"KEY":     0,
-	"VERSION": 1,
-	"CREATE":  2,
-	"MOD":     3,
-	"VALUE":   4,
-}
-
-func (x RangeRequest_SortTarget) String() string {
-	return proto.EnumName(RangeRequest_SortTarget_name, int32(x))
-}
-func (RangeRequest_SortTarget) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{1, 1} }
-
-type Compare_CompareResult int32
-
-const (
-	Compare_EQUAL     Compare_CompareResult = 0
-	Compare_GREATER   Compare_CompareResult = 1
-	Compare_LESS      Compare_CompareResult = 2
-	Compare_NOT_EQUAL Compare_CompareResult = 3
-)
-
-var Compare_CompareResult_name = map[int32]string{
-	0: "EQUAL",
-	1: "GREATER",
-	2: "LESS",
-	3: "NOT_EQUAL",
-}
-var Compare_CompareResult_value = map[string]int32{
-	"EQUAL":     0,
-	"GREATER":   1,
-	"LESS":      2,
-	"NOT_EQUAL": 3,
-}
-
-func (x Compare_CompareResult) String() string {
-	return proto.EnumName(Compare_CompareResult_name, int32(x))
-}
-func (Compare_CompareResult) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{9, 0} }
-
-type Compare_CompareTarget int32
-
-const (
-	Compare_VERSION Compare_CompareTarget = 0
-	Compare_CREATE  Compare_CompareTarget = 1
-	Compare_MOD     Compare_CompareTarget = 2
-	Compare_VALUE   Compare_CompareTarget = 3
-	Compare_LEASE   Compare_CompareTarget = 4
-)
-
-var Compare_CompareTarget_name = map[int32]string{
-	0: "VERSION",
-	1: "CREATE",
-	2: "MOD",
-	3: "VALUE",
-	4: "LEASE",
-}
-var Compare_CompareTarget_value = map[string]int32{
-	"VERSION": 0,
-	"CREATE":  1,
-	"MOD":     2,
-	"VALUE":   3,
-	"LEASE":   4,
-}
-
-func (x Compare_CompareTarget) String() string {
-	return proto.EnumName(Compare_CompareTarget_name, int32(x))
-}
-func (Compare_CompareTarget) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{9, 1} }
-
-type WatchCreateRequest_FilterType int32
-
-const (
-	// filter out put event.
-	WatchCreateRequest_NOPUT WatchCreateRequest_FilterType = 0
-	// filter out delete event.
-	WatchCreateRequest_NODELETE WatchCreateRequest_FilterType = 1
-)
-
-var WatchCreateRequest_FilterType_name = map[int32]string{
-	0: "NOPUT",
-	1: "NODELETE",
-}
-var WatchCreateRequest_FilterType_value = map[string]int32{
-	"NOPUT":    0,
-	"NODELETE": 1,
-}
-
-func (x WatchCreateRequest_FilterType) String() string {
-	return proto.EnumName(WatchCreateRequest_FilterType_name, int32(x))
-}
-func (WatchCreateRequest_FilterType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{21, 0}
-}
-
-type AlarmRequest_AlarmAction int32
-
-const (
-	AlarmRequest_GET        AlarmRequest_AlarmAction = 0
-	AlarmRequest_ACTIVATE   AlarmRequest_AlarmAction = 1
-	AlarmRequest_DEACTIVATE AlarmRequest_AlarmAction = 2
-)
-
-var AlarmRequest_AlarmAction_name = map[int32]string{
-	0: "GET",
-	1: "ACTIVATE",
-	2: "DEACTIVATE",
-}
-var AlarmRequest_AlarmAction_value = map[string]int32{
-	"GET":        0,
-	"ACTIVATE":   1,
-	"DEACTIVATE": 2,
-}
-
-func (x AlarmRequest_AlarmAction) String() string {
-	return proto.EnumName(AlarmRequest_AlarmAction_name, int32(x))
-}
-func (AlarmRequest_AlarmAction) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{54, 0}
-}
-
-type ResponseHeader struct {
-	// cluster_id is the ID of the cluster which sent the response.
-	ClusterId uint64 `protobuf:"varint,1,opt,name=cluster_id,json=clusterId,proto3" json:"cluster_id,omitempty"`
-	// member_id is the ID of the member which sent the response.
-	MemberId uint64 `protobuf:"varint,2,opt,name=member_id,json=memberId,proto3" json:"member_id,omitempty"`
-	// revision is the key-value store revision when the request was applied.
-	// For watch progress responses, the header.revision indicates progress. All future events
-	// recieved in this stream are guaranteed to have a higher revision number than the
-	// header.revision number.
-	Revision int64 `protobuf:"varint,3,opt,name=revision,proto3" json:"revision,omitempty"`
-	// raft_term is the raft term when the request was applied.
-	RaftTerm uint64 `protobuf:"varint,4,opt,name=raft_term,json=raftTerm,proto3" json:"raft_term,omitempty"`
-}
-
-func (m *ResponseHeader) Reset()                    { *m = ResponseHeader{} }
-func (m *ResponseHeader) String() string            { return proto.CompactTextString(m) }
-func (*ResponseHeader) ProtoMessage()               {}
-func (*ResponseHeader) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{0} }
-
-func (m *ResponseHeader) GetClusterId() uint64 {
-	if m != nil {
-		return m.ClusterId
-	}
-	return 0
-}
-
-func (m *ResponseHeader) GetMemberId() uint64 {
-	if m != nil {
-		return m.MemberId
-	}
-	return 0
-}
-
-func (m *ResponseHeader) GetRevision() int64 {
-	if m != nil {
-		return m.Revision
-	}
-	return 0
-}
-
-func (m *ResponseHeader) GetRaftTerm() uint64 {
-	if m != nil {
-		return m.RaftTerm
-	}
-	return 0
-}
-
-type RangeRequest struct {
-	// key is the first key for the range. If range_end is not given, the request only looks up key.
-	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	// range_end is the upper bound on the requested range [key, range_end).
-	// If range_end is '\0', the range is all keys >= key.
-	// If range_end is key plus one (e.g., "aa"+1 == "ab", "a\xff"+1 == "b"),
-	// then the range request gets all keys prefixed with key.
-	// If both key and range_end are '\0', then the range request returns all keys.
-	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-	// limit is a limit on the number of keys returned for the request. When limit is set to 0,
-	// it is treated as no limit.
-	Limit int64 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"`
-	// revision is the point-in-time of the key-value store to use for the range.
-	// If revision is less or equal to zero, the range is over the newest key-value store.
-	// If the revision has been compacted, ErrCompacted is returned as a response.
-	Revision int64 `protobuf:"varint,4,opt,name=revision,proto3" json:"revision,omitempty"`
-	// sort_order is the order for returned sorted results.
-	SortOrder RangeRequest_SortOrder `protobuf:"varint,5,opt,name=sort_order,json=sortOrder,proto3,enum=etcdserverpb.RangeRequest_SortOrder" json:"sort_order,omitempty"`
-	// sort_target is the key-value field to use for sorting.
-	SortTarget RangeRequest_SortTarget `protobuf:"varint,6,opt,name=sort_target,json=sortTarget,proto3,enum=etcdserverpb.RangeRequest_SortTarget" json:"sort_target,omitempty"`
-	// serializable sets the range request to use serializable member-local reads.
-	// Range requests are linearizable by default; linearizable requests have higher
-	// latency and lower throughput than serializable requests but reflect the current
-	// consensus of the cluster. For better performance, in exchange for possible stale reads,
-	// a serializable range request is served locally without needing to reach consensus
-	// with other nodes in the cluster.
-	Serializable bool `protobuf:"varint,7,opt,name=serializable,proto3" json:"serializable,omitempty"`
-	// keys_only when set returns only the keys and not the values.
-	KeysOnly bool `protobuf:"varint,8,opt,name=keys_only,json=keysOnly,proto3" json:"keys_only,omitempty"`
-	// count_only when set returns only the count of the keys in the range.
-	CountOnly bool `protobuf:"varint,9,opt,name=count_only,json=countOnly,proto3" json:"count_only,omitempty"`
-	// min_mod_revision is the lower bound for returned key mod revisions; all keys with
-	// lesser mod revisions will be filtered away.
-	MinModRevision int64 `protobuf:"varint,10,opt,name=min_mod_revision,json=minModRevision,proto3" json:"min_mod_revision,omitempty"`
-	// max_mod_revision is the upper bound for returned key mod revisions; all keys with
-	// greater mod revisions will be filtered away.
-	MaxModRevision int64 `protobuf:"varint,11,opt,name=max_mod_revision,json=maxModRevision,proto3" json:"max_mod_revision,omitempty"`
-	// min_create_revision is the lower bound for returned key create revisions; all keys with
-	// lesser create revisions will be filtered away.
-	MinCreateRevision int64 `protobuf:"varint,12,opt,name=min_create_revision,json=minCreateRevision,proto3" json:"min_create_revision,omitempty"`
-	// max_create_revision is the upper bound for returned key create revisions; all keys with
-	// greater create revisions will be filtered away.
-	MaxCreateRevision int64 `protobuf:"varint,13,opt,name=max_create_revision,json=maxCreateRevision,proto3" json:"max_create_revision,omitempty"`
-}
-
-func (m *RangeRequest) Reset()                    { *m = RangeRequest{} }
-func (m *RangeRequest) String() string            { return proto.CompactTextString(m) }
-func (*RangeRequest) ProtoMessage()               {}
-func (*RangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{1} }
-
-func (m *RangeRequest) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *RangeRequest) GetRangeEnd() []byte {
-	if m != nil {
-		return m.RangeEnd
-	}
-	return nil
-}
-
-func (m *RangeRequest) GetLimit() int64 {
-	if m != nil {
-		return m.Limit
-	}
-	return 0
-}
-
-func (m *RangeRequest) GetRevision() int64 {
-	if m != nil {
-		return m.Revision
-	}
-	return 0
-}
-
-func (m *RangeRequest) GetSortOrder() RangeRequest_SortOrder {
-	if m != nil {
-		return m.SortOrder
-	}
-	return RangeRequest_NONE
-}
-
-func (m *RangeRequest) GetSortTarget() RangeRequest_SortTarget {
-	if m != nil {
-		return m.SortTarget
-	}
-	return RangeRequest_KEY
-}
-
-func (m *RangeRequest) GetSerializable() bool {
-	if m != nil {
-		return m.Serializable
-	}
-	return false
-}
-
-func (m *RangeRequest) GetKeysOnly() bool {
-	if m != nil {
-		return m.KeysOnly
-	}
-	return false
-}
-
-func (m *RangeRequest) GetCountOnly() bool {
-	if m != nil {
-		return m.CountOnly
-	}
-	return false
-}
-
-func (m *RangeRequest) GetMinModRevision() int64 {
-	if m != nil {
-		return m.MinModRevision
-	}
-	return 0
-}
-
-func (m *RangeRequest) GetMaxModRevision() int64 {
-	if m != nil {
-		return m.MaxModRevision
-	}
-	return 0
-}
-
-func (m *RangeRequest) GetMinCreateRevision() int64 {
-	if m != nil {
-		return m.MinCreateRevision
-	}
-	return 0
-}
-
-func (m *RangeRequest) GetMaxCreateRevision() int64 {
-	if m != nil {
-		return m.MaxCreateRevision
-	}
-	return 0
-}
-
-type RangeResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// kvs is the list of key-value pairs matched by the range request.
-	// kvs is empty when count is requested.
-	Kvs []*mvccpb.KeyValue `protobuf:"bytes,2,rep,name=kvs" json:"kvs,omitempty"`
-	// more indicates if there are more keys to return in the requested range.
-	More bool `protobuf:"varint,3,opt,name=more,proto3" json:"more,omitempty"`
-	// count is set to the number of keys within the range when requested.
-	Count int64 `protobuf:"varint,4,opt,name=count,proto3" json:"count,omitempty"`
-}
-
-func (m *RangeResponse) Reset()                    { *m = RangeResponse{} }
-func (m *RangeResponse) String() string            { return proto.CompactTextString(m) }
-func (*RangeResponse) ProtoMessage()               {}
-func (*RangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{2} }
-
-func (m *RangeResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *RangeResponse) GetKvs() []*mvccpb.KeyValue {
-	if m != nil {
-		return m.Kvs
-	}
-	return nil
-}
-
-func (m *RangeResponse) GetMore() bool {
-	if m != nil {
-		return m.More
-	}
-	return false
-}
-
-func (m *RangeResponse) GetCount() int64 {
-	if m != nil {
-		return m.Count
-	}
-	return 0
-}
-
-type PutRequest struct {
-	// key is the key, in bytes, to put into the key-value store.
-	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	// value is the value, in bytes, to associate with the key in the key-value store.
-	Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	// lease is the lease ID to associate with the key in the key-value store. A lease
-	// value of 0 indicates no lease.
-	Lease int64 `protobuf:"varint,3,opt,name=lease,proto3" json:"lease,omitempty"`
-	// If prev_kv is set, etcd gets the previous key-value pair before changing it.
-	// The previous key-value pair will be returned in the put response.
-	PrevKv bool `protobuf:"varint,4,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
-	// If ignore_value is set, etcd updates the key using its current value.
-	// Returns an error if the key does not exist.
-	IgnoreValue bool `protobuf:"varint,5,opt,name=ignore_value,json=ignoreValue,proto3" json:"ignore_value,omitempty"`
-	// If ignore_lease is set, etcd updates the key using its current lease.
-	// Returns an error if the key does not exist.
-	IgnoreLease bool `protobuf:"varint,6,opt,name=ignore_lease,json=ignoreLease,proto3" json:"ignore_lease,omitempty"`
-}
-
-func (m *PutRequest) Reset()                    { *m = PutRequest{} }
-func (m *PutRequest) String() string            { return proto.CompactTextString(m) }
-func (*PutRequest) ProtoMessage()               {}
-func (*PutRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{3} }
-
-func (m *PutRequest) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *PutRequest) GetValue() []byte {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func (m *PutRequest) GetLease() int64 {
-	if m != nil {
-		return m.Lease
-	}
-	return 0
-}
-
-func (m *PutRequest) GetPrevKv() bool {
-	if m != nil {
-		return m.PrevKv
-	}
-	return false
-}
-
-func (m *PutRequest) GetIgnoreValue() bool {
-	if m != nil {
-		return m.IgnoreValue
-	}
-	return false
-}
-
-func (m *PutRequest) GetIgnoreLease() bool {
-	if m != nil {
-		return m.IgnoreLease
-	}
-	return false
-}
-
-type PutResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// if prev_kv is set in the request, the previous key-value pair will be returned.
-	PrevKv *mvccpb.KeyValue `protobuf:"bytes,2,opt,name=prev_kv,json=prevKv" json:"prev_kv,omitempty"`
-}
-
-func (m *PutResponse) Reset()                    { *m = PutResponse{} }
-func (m *PutResponse) String() string            { return proto.CompactTextString(m) }
-func (*PutResponse) ProtoMessage()               {}
-func (*PutResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{4} }
-
-func (m *PutResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *PutResponse) GetPrevKv() *mvccpb.KeyValue {
-	if m != nil {
-		return m.PrevKv
-	}
-	return nil
-}
-
-type DeleteRangeRequest struct {
-	// key is the first key to delete in the range.
-	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	// range_end is the key following the last key to delete for the range [key, range_end).
-	// If range_end is not given, the range is defined to contain only the key argument.
-	// If range_end is one bit larger than the given key, then the range is all the keys
-	// with the prefix (the given key).
-	// If range_end is '\0', the range is all keys greater than or equal to the key argument.
-	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-	// If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
-	// The previous key-value pairs will be returned in the delete response.
-	PrevKv bool `protobuf:"varint,3,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
-}
-
-func (m *DeleteRangeRequest) Reset()                    { *m = DeleteRangeRequest{} }
-func (m *DeleteRangeRequest) String() string            { return proto.CompactTextString(m) }
-func (*DeleteRangeRequest) ProtoMessage()               {}
-func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{5} }
-
-func (m *DeleteRangeRequest) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *DeleteRangeRequest) GetRangeEnd() []byte {
-	if m != nil {
-		return m.RangeEnd
-	}
-	return nil
-}
-
-func (m *DeleteRangeRequest) GetPrevKv() bool {
-	if m != nil {
-		return m.PrevKv
-	}
-	return false
-}
-
-type DeleteRangeResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// deleted is the number of keys deleted by the delete range request.
-	Deleted int64 `protobuf:"varint,2,opt,name=deleted,proto3" json:"deleted,omitempty"`
-	// if prev_kv is set in the request, the previous key-value pairs will be returned.
-	PrevKvs []*mvccpb.KeyValue `protobuf:"bytes,3,rep,name=prev_kvs,json=prevKvs" json:"prev_kvs,omitempty"`
-}
-
-func (m *DeleteRangeResponse) Reset()                    { *m = DeleteRangeResponse{} }
-func (m *DeleteRangeResponse) String() string            { return proto.CompactTextString(m) }
-func (*DeleteRangeResponse) ProtoMessage()               {}
-func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{6} }
-
-func (m *DeleteRangeResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *DeleteRangeResponse) GetDeleted() int64 {
-	if m != nil {
-		return m.Deleted
-	}
-	return 0
-}
-
-func (m *DeleteRangeResponse) GetPrevKvs() []*mvccpb.KeyValue {
-	if m != nil {
-		return m.PrevKvs
-	}
-	return nil
-}
-
-type RequestOp struct {
-	// request is a union of request types accepted by a transaction.
-	//
-	// Types that are valid to be assigned to Request:
-	//	*RequestOp_RequestRange
-	//	*RequestOp_RequestPut
-	//	*RequestOp_RequestDeleteRange
-	//	*RequestOp_RequestTxn
-	Request isRequestOp_Request `protobuf_oneof:"request"`
-}
-
-func (m *RequestOp) Reset()                    { *m = RequestOp{} }
-func (m *RequestOp) String() string            { return proto.CompactTextString(m) }
-func (*RequestOp) ProtoMessage()               {}
-func (*RequestOp) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{7} }
-
-type isRequestOp_Request interface {
-	isRequestOp_Request()
-	MarshalTo([]byte) (int, error)
-	Size() int
-}
-
-type RequestOp_RequestRange struct {
-	RequestRange *RangeRequest `protobuf:"bytes,1,opt,name=request_range,json=requestRange,oneof"`
-}
-type RequestOp_RequestPut struct {
-	RequestPut *PutRequest `protobuf:"bytes,2,opt,name=request_put,json=requestPut,oneof"`
-}
-type RequestOp_RequestDeleteRange struct {
-	RequestDeleteRange *DeleteRangeRequest `protobuf:"bytes,3,opt,name=request_delete_range,json=requestDeleteRange,oneof"`
-}
-type RequestOp_RequestTxn struct {
-	RequestTxn *TxnRequest `protobuf:"bytes,4,opt,name=request_txn,json=requestTxn,oneof"`
-}
-
-func (*RequestOp_RequestRange) isRequestOp_Request()       {}
-func (*RequestOp_RequestPut) isRequestOp_Request()         {}
-func (*RequestOp_RequestDeleteRange) isRequestOp_Request() {}
-func (*RequestOp_RequestTxn) isRequestOp_Request()         {}
-
-func (m *RequestOp) GetRequest() isRequestOp_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *RequestOp) GetRequestRange() *RangeRequest {
-	if x, ok := m.GetRequest().(*RequestOp_RequestRange); ok {
-		return x.RequestRange
-	}
-	return nil
-}
-
-func (m *RequestOp) GetRequestPut() *PutRequest {
-	if x, ok := m.GetRequest().(*RequestOp_RequestPut); ok {
-		return x.RequestPut
-	}
-	return nil
-}
-
-func (m *RequestOp) GetRequestDeleteRange() *DeleteRangeRequest {
-	if x, ok := m.GetRequest().(*RequestOp_RequestDeleteRange); ok {
-		return x.RequestDeleteRange
-	}
-	return nil
-}
-
-func (m *RequestOp) GetRequestTxn() *TxnRequest {
-	if x, ok := m.GetRequest().(*RequestOp_RequestTxn); ok {
-		return x.RequestTxn
-	}
-	return nil
-}
-
-// XXX_OneofFuncs is for the internal use of the proto package.
-func (*RequestOp) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
-	return _RequestOp_OneofMarshaler, _RequestOp_OneofUnmarshaler, _RequestOp_OneofSizer, []interface{}{
-		(*RequestOp_RequestRange)(nil),
-		(*RequestOp_RequestPut)(nil),
-		(*RequestOp_RequestDeleteRange)(nil),
-		(*RequestOp_RequestTxn)(nil),
-	}
-}
-
-func _RequestOp_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
-	m := msg.(*RequestOp)
-	// request
-	switch x := m.Request.(type) {
-	case *RequestOp_RequestRange:
-		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.RequestRange); err != nil {
-			return err
-		}
-	case *RequestOp_RequestPut:
-		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.RequestPut); err != nil {
-			return err
-		}
-	case *RequestOp_RequestDeleteRange:
-		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.RequestDeleteRange); err != nil {
-			return err
-		}
-	case *RequestOp_RequestTxn:
-		_ = b.EncodeVarint(4<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.RequestTxn); err != nil {
-			return err
-		}
-	case nil:
-	default:
-		return fmt.Errorf("RequestOp.Request has unexpected type %T", x)
-	}
-	return nil
-}
-
-func _RequestOp_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
-	m := msg.(*RequestOp)
-	switch tag {
-	case 1: // request.request_range
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(RangeRequest)
-		err := b.DecodeMessage(msg)
-		m.Request = &RequestOp_RequestRange{msg}
-		return true, err
-	case 2: // request.request_put
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(PutRequest)
-		err := b.DecodeMessage(msg)
-		m.Request = &RequestOp_RequestPut{msg}
-		return true, err
-	case 3: // request.request_delete_range
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(DeleteRangeRequest)
-		err := b.DecodeMessage(msg)
-		m.Request = &RequestOp_RequestDeleteRange{msg}
-		return true, err
-	case 4: // request.request_txn
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(TxnRequest)
-		err := b.DecodeMessage(msg)
-		m.Request = &RequestOp_RequestTxn{msg}
-		return true, err
-	default:
-		return false, nil
-	}
-}
-
-func _RequestOp_OneofSizer(msg proto.Message) (n int) {
-	m := msg.(*RequestOp)
-	// request
-	switch x := m.Request.(type) {
-	case *RequestOp_RequestRange:
-		s := proto.Size(x.RequestRange)
-		n += proto.SizeVarint(1<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *RequestOp_RequestPut:
-		s := proto.Size(x.RequestPut)
-		n += proto.SizeVarint(2<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *RequestOp_RequestDeleteRange:
-		s := proto.Size(x.RequestDeleteRange)
-		n += proto.SizeVarint(3<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *RequestOp_RequestTxn:
-		s := proto.Size(x.RequestTxn)
-		n += proto.SizeVarint(4<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case nil:
-	default:
-		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
-	}
-	return n
-}
-
-type ResponseOp struct {
-	// response is a union of response types returned by a transaction.
-	//
-	// Types that are valid to be assigned to Response:
-	//	*ResponseOp_ResponseRange
-	//	*ResponseOp_ResponsePut
-	//	*ResponseOp_ResponseDeleteRange
-	//	*ResponseOp_ResponseTxn
-	Response isResponseOp_Response `protobuf_oneof:"response"`
-}
-
-func (m *ResponseOp) Reset()                    { *m = ResponseOp{} }
-func (m *ResponseOp) String() string            { return proto.CompactTextString(m) }
-func (*ResponseOp) ProtoMessage()               {}
-func (*ResponseOp) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{8} }
-
-type isResponseOp_Response interface {
-	isResponseOp_Response()
-	MarshalTo([]byte) (int, error)
-	Size() int
-}
-
-type ResponseOp_ResponseRange struct {
-	ResponseRange *RangeResponse `protobuf:"bytes,1,opt,name=response_range,json=responseRange,oneof"`
-}
-type ResponseOp_ResponsePut struct {
-	ResponsePut *PutResponse `protobuf:"bytes,2,opt,name=response_put,json=responsePut,oneof"`
-}
-type ResponseOp_ResponseDeleteRange struct {
-	ResponseDeleteRange *DeleteRangeResponse `protobuf:"bytes,3,opt,name=response_delete_range,json=responseDeleteRange,oneof"`
-}
-type ResponseOp_ResponseTxn struct {
-	ResponseTxn *TxnResponse `protobuf:"bytes,4,opt,name=response_txn,json=responseTxn,oneof"`
-}
-
-func (*ResponseOp_ResponseRange) isResponseOp_Response()       {}
-func (*ResponseOp_ResponsePut) isResponseOp_Response()         {}
-func (*ResponseOp_ResponseDeleteRange) isResponseOp_Response() {}
-func (*ResponseOp_ResponseTxn) isResponseOp_Response()         {}
-
-func (m *ResponseOp) GetResponse() isResponseOp_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *ResponseOp) GetResponseRange() *RangeResponse {
-	if x, ok := m.GetResponse().(*ResponseOp_ResponseRange); ok {
-		return x.ResponseRange
-	}
-	return nil
-}
-
-func (m *ResponseOp) GetResponsePut() *PutResponse {
-	if x, ok := m.GetResponse().(*ResponseOp_ResponsePut); ok {
-		return x.ResponsePut
-	}
-	return nil
-}
-
-func (m *ResponseOp) GetResponseDeleteRange() *DeleteRangeResponse {
-	if x, ok := m.GetResponse().(*ResponseOp_ResponseDeleteRange); ok {
-		return x.ResponseDeleteRange
-	}
-	return nil
-}
-
-func (m *ResponseOp) GetResponseTxn() *TxnResponse {
-	if x, ok := m.GetResponse().(*ResponseOp_ResponseTxn); ok {
-		return x.ResponseTxn
-	}
-	return nil
-}
-
-// XXX_OneofFuncs is for the internal use of the proto package.
-func (*ResponseOp) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
-	return _ResponseOp_OneofMarshaler, _ResponseOp_OneofUnmarshaler, _ResponseOp_OneofSizer, []interface{}{
-		(*ResponseOp_ResponseRange)(nil),
-		(*ResponseOp_ResponsePut)(nil),
-		(*ResponseOp_ResponseDeleteRange)(nil),
-		(*ResponseOp_ResponseTxn)(nil),
-	}
-}
-
-func _ResponseOp_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
-	m := msg.(*ResponseOp)
-	// response
-	switch x := m.Response.(type) {
-	case *ResponseOp_ResponseRange:
-		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ResponseRange); err != nil {
-			return err
-		}
-	case *ResponseOp_ResponsePut:
-		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ResponsePut); err != nil {
-			return err
-		}
-	case *ResponseOp_ResponseDeleteRange:
-		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ResponseDeleteRange); err != nil {
-			return err
-		}
-	case *ResponseOp_ResponseTxn:
-		_ = b.EncodeVarint(4<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ResponseTxn); err != nil {
-			return err
-		}
-	case nil:
-	default:
-		return fmt.Errorf("ResponseOp.Response has unexpected type %T", x)
-	}
-	return nil
-}
-
-func _ResponseOp_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
-	m := msg.(*ResponseOp)
-	switch tag {
-	case 1: // response.response_range
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(RangeResponse)
-		err := b.DecodeMessage(msg)
-		m.Response = &ResponseOp_ResponseRange{msg}
-		return true, err
-	case 2: // response.response_put
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(PutResponse)
-		err := b.DecodeMessage(msg)
-		m.Response = &ResponseOp_ResponsePut{msg}
-		return true, err
-	case 3: // response.response_delete_range
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(DeleteRangeResponse)
-		err := b.DecodeMessage(msg)
-		m.Response = &ResponseOp_ResponseDeleteRange{msg}
-		return true, err
-	case 4: // response.response_txn
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(TxnResponse)
-		err := b.DecodeMessage(msg)
-		m.Response = &ResponseOp_ResponseTxn{msg}
-		return true, err
-	default:
-		return false, nil
-	}
-}
-
-func _ResponseOp_OneofSizer(msg proto.Message) (n int) {
-	m := msg.(*ResponseOp)
-	// response
-	switch x := m.Response.(type) {
-	case *ResponseOp_ResponseRange:
-		s := proto.Size(x.ResponseRange)
-		n += proto.SizeVarint(1<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *ResponseOp_ResponsePut:
-		s := proto.Size(x.ResponsePut)
-		n += proto.SizeVarint(2<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *ResponseOp_ResponseDeleteRange:
-		s := proto.Size(x.ResponseDeleteRange)
-		n += proto.SizeVarint(3<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *ResponseOp_ResponseTxn:
-		s := proto.Size(x.ResponseTxn)
-		n += proto.SizeVarint(4<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case nil:
-	default:
-		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
-	}
-	return n
-}
-
-type Compare struct {
-	// result is logical comparison operation for this comparison.
-	Result Compare_CompareResult `protobuf:"varint,1,opt,name=result,proto3,enum=etcdserverpb.Compare_CompareResult" json:"result,omitempty"`
-	// target is the key-value field to inspect for the comparison.
-	Target Compare_CompareTarget `protobuf:"varint,2,opt,name=target,proto3,enum=etcdserverpb.Compare_CompareTarget" json:"target,omitempty"`
-	// key is the subject key for the comparison operation.
-	Key []byte `protobuf:"bytes,3,opt,name=key,proto3" json:"key,omitempty"`
-	// Types that are valid to be assigned to TargetUnion:
-	//	*Compare_Version
-	//	*Compare_CreateRevision
-	//	*Compare_ModRevision
-	//	*Compare_Value
-	//	*Compare_Lease
-	TargetUnion isCompare_TargetUnion `protobuf_oneof:"target_union"`
-	// range_end compares the given target to all keys in the range [key, range_end).
-	// See RangeRequest for more details on key ranges.
-	RangeEnd []byte `protobuf:"bytes,64,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-}
-
-func (m *Compare) Reset()                    { *m = Compare{} }
-func (m *Compare) String() string            { return proto.CompactTextString(m) }
-func (*Compare) ProtoMessage()               {}
-func (*Compare) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{9} }
-
-type isCompare_TargetUnion interface {
-	isCompare_TargetUnion()
-	MarshalTo([]byte) (int, error)
-	Size() int
-}
-
-type Compare_Version struct {
-	Version int64 `protobuf:"varint,4,opt,name=version,proto3,oneof"`
-}
-type Compare_CreateRevision struct {
-	CreateRevision int64 `protobuf:"varint,5,opt,name=create_revision,json=createRevision,proto3,oneof"`
-}
-type Compare_ModRevision struct {
-	ModRevision int64 `protobuf:"varint,6,opt,name=mod_revision,json=modRevision,proto3,oneof"`
-}
-type Compare_Value struct {
-	Value []byte `protobuf:"bytes,7,opt,name=value,proto3,oneof"`
-}
-type Compare_Lease struct {
-	Lease int64 `protobuf:"varint,8,opt,name=lease,proto3,oneof"`
-}
-
-func (*Compare_Version) isCompare_TargetUnion()        {}
-func (*Compare_CreateRevision) isCompare_TargetUnion() {}
-func (*Compare_ModRevision) isCompare_TargetUnion()    {}
-func (*Compare_Value) isCompare_TargetUnion()          {}
-func (*Compare_Lease) isCompare_TargetUnion()          {}
-
-func (m *Compare) GetTargetUnion() isCompare_TargetUnion {
-	if m != nil {
-		return m.TargetUnion
-	}
-	return nil
-}
-
-func (m *Compare) GetResult() Compare_CompareResult {
-	if m != nil {
-		return m.Result
-	}
-	return Compare_EQUAL
-}
-
-func (m *Compare) GetTarget() Compare_CompareTarget {
-	if m != nil {
-		return m.Target
-	}
-	return Compare_VERSION
-}
-
-func (m *Compare) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *Compare) GetVersion() int64 {
-	if x, ok := m.GetTargetUnion().(*Compare_Version); ok {
-		return x.Version
-	}
-	return 0
-}
-
-func (m *Compare) GetCreateRevision() int64 {
-	if x, ok := m.GetTargetUnion().(*Compare_CreateRevision); ok {
-		return x.CreateRevision
-	}
-	return 0
-}
-
-func (m *Compare) GetModRevision() int64 {
-	if x, ok := m.GetTargetUnion().(*Compare_ModRevision); ok {
-		return x.ModRevision
-	}
-	return 0
-}
-
-func (m *Compare) GetValue() []byte {
-	if x, ok := m.GetTargetUnion().(*Compare_Value); ok {
-		return x.Value
-	}
-	return nil
-}
-
-func (m *Compare) GetLease() int64 {
-	if x, ok := m.GetTargetUnion().(*Compare_Lease); ok {
-		return x.Lease
-	}
-	return 0
-}
-
-func (m *Compare) GetRangeEnd() []byte {
-	if m != nil {
-		return m.RangeEnd
-	}
-	return nil
-}
-
-// XXX_OneofFuncs is for the internal use of the proto package.
-func (*Compare) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
-	return _Compare_OneofMarshaler, _Compare_OneofUnmarshaler, _Compare_OneofSizer, []interface{}{
-		(*Compare_Version)(nil),
-		(*Compare_CreateRevision)(nil),
-		(*Compare_ModRevision)(nil),
-		(*Compare_Value)(nil),
-		(*Compare_Lease)(nil),
-	}
-}
-
-func _Compare_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
-	m := msg.(*Compare)
-	// target_union
-	switch x := m.TargetUnion.(type) {
-	case *Compare_Version:
-		_ = b.EncodeVarint(4<<3 | proto.WireVarint)
-		_ = b.EncodeVarint(uint64(x.Version))
-	case *Compare_CreateRevision:
-		_ = b.EncodeVarint(5<<3 | proto.WireVarint)
-		_ = b.EncodeVarint(uint64(x.CreateRevision))
-	case *Compare_ModRevision:
-		_ = b.EncodeVarint(6<<3 | proto.WireVarint)
-		_ = b.EncodeVarint(uint64(x.ModRevision))
-	case *Compare_Value:
-		_ = b.EncodeVarint(7<<3 | proto.WireBytes)
-		_ = b.EncodeRawBytes(x.Value)
-	case *Compare_Lease:
-		_ = b.EncodeVarint(8<<3 | proto.WireVarint)
-		_ = b.EncodeVarint(uint64(x.Lease))
-	case nil:
-	default:
-		return fmt.Errorf("Compare.TargetUnion has unexpected type %T", x)
-	}
-	return nil
-}
-
-func _Compare_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
-	m := msg.(*Compare)
-	switch tag {
-	case 4: // target_union.version
-		if wire != proto.WireVarint {
-			return true, proto.ErrInternalBadWireType
-		}
-		x, err := b.DecodeVarint()
-		m.TargetUnion = &Compare_Version{int64(x)}
-		return true, err
-	case 5: // target_union.create_revision
-		if wire != proto.WireVarint {
-			return true, proto.ErrInternalBadWireType
-		}
-		x, err := b.DecodeVarint()
-		m.TargetUnion = &Compare_CreateRevision{int64(x)}
-		return true, err
-	case 6: // target_union.mod_revision
-		if wire != proto.WireVarint {
-			return true, proto.ErrInternalBadWireType
-		}
-		x, err := b.DecodeVarint()
-		m.TargetUnion = &Compare_ModRevision{int64(x)}
-		return true, err
-	case 7: // target_union.value
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		x, err := b.DecodeRawBytes(true)
-		m.TargetUnion = &Compare_Value{x}
-		return true, err
-	case 8: // target_union.lease
-		if wire != proto.WireVarint {
-			return true, proto.ErrInternalBadWireType
-		}
-		x, err := b.DecodeVarint()
-		m.TargetUnion = &Compare_Lease{int64(x)}
-		return true, err
-	default:
-		return false, nil
-	}
-}
-
-func _Compare_OneofSizer(msg proto.Message) (n int) {
-	m := msg.(*Compare)
-	// target_union
-	switch x := m.TargetUnion.(type) {
-	case *Compare_Version:
-		n += proto.SizeVarint(4<<3 | proto.WireVarint)
-		n += proto.SizeVarint(uint64(x.Version))
-	case *Compare_CreateRevision:
-		n += proto.SizeVarint(5<<3 | proto.WireVarint)
-		n += proto.SizeVarint(uint64(x.CreateRevision))
-	case *Compare_ModRevision:
-		n += proto.SizeVarint(6<<3 | proto.WireVarint)
-		n += proto.SizeVarint(uint64(x.ModRevision))
-	case *Compare_Value:
-		n += proto.SizeVarint(7<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(len(x.Value)))
-		n += len(x.Value)
-	case *Compare_Lease:
-		n += proto.SizeVarint(8<<3 | proto.WireVarint)
-		n += proto.SizeVarint(uint64(x.Lease))
-	case nil:
-	default:
-		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
-	}
-	return n
-}
-
-// From google paxosdb paper:
-// Our implementation hinges around a powerful primitive which we call MultiOp. All other database
-// operations except for iteration are implemented as a single call to MultiOp. A MultiOp is applied atomically
-// and consists of three components:
-// 1. A list of tests called guard. Each test in guard checks a single entry in the database. It may check
-// for the absence or presence of a value, or compare with a given value. Two different tests in the guard
-// may apply to the same or different entries in the database. All tests in the guard are applied and
-// MultiOp returns the results. If all tests are true, MultiOp executes t op (see item 2 below), otherwise
-// it executes f op (see item 3 below).
-// 2. A list of database operations called t op. Each operation in the list is either an insert, delete, or
-// lookup operation, and applies to a single database entry. Two different operations in the list may apply
-// to the same or different entries in the database. These operations are executed
-// if guard evaluates to
-// true.
-// 3. A list of database operations called f op. Like t op, but executed if guard evaluates to false.
-type TxnRequest struct {
-	// compare is a list of predicates representing a conjunction of terms.
-	// If the comparisons succeed, then the success requests will be processed in order,
-	// and the response will contain their respective responses in order.
-	// If the comparisons fail, then the failure requests will be processed in order,
-	// and the response will contain their respective responses in order.
-	Compare []*Compare `protobuf:"bytes,1,rep,name=compare" json:"compare,omitempty"`
-	// success is a list of requests which will be applied when compare evaluates to true.
-	Success []*RequestOp `protobuf:"bytes,2,rep,name=success" json:"success,omitempty"`
-	// failure is a list of requests which will be applied when compare evaluates to false.
-	Failure []*RequestOp `protobuf:"bytes,3,rep,name=failure" json:"failure,omitempty"`
-}
-
-func (m *TxnRequest) Reset()                    { *m = TxnRequest{} }
-func (m *TxnRequest) String() string            { return proto.CompactTextString(m) }
-func (*TxnRequest) ProtoMessage()               {}
-func (*TxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{10} }
-
-func (m *TxnRequest) GetCompare() []*Compare {
-	if m != nil {
-		return m.Compare
-	}
-	return nil
-}
-
-func (m *TxnRequest) GetSuccess() []*RequestOp {
-	if m != nil {
-		return m.Success
-	}
-	return nil
-}
-
-func (m *TxnRequest) GetFailure() []*RequestOp {
-	if m != nil {
-		return m.Failure
-	}
-	return nil
-}
-
-type TxnResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// succeeded is set to true if the compare evaluated to true or false otherwise.
-	Succeeded bool `protobuf:"varint,2,opt,name=succeeded,proto3" json:"succeeded,omitempty"`
-	// responses is a list of responses corresponding to the results from applying
-	// success if succeeded is true or failure if succeeded is false.
-	Responses []*ResponseOp `protobuf:"bytes,3,rep,name=responses" json:"responses,omitempty"`
-}
-
-func (m *TxnResponse) Reset()                    { *m = TxnResponse{} }
-func (m *TxnResponse) String() string            { return proto.CompactTextString(m) }
-func (*TxnResponse) ProtoMessage()               {}
-func (*TxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{11} }
-
-func (m *TxnResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *TxnResponse) GetSucceeded() bool {
-	if m != nil {
-		return m.Succeeded
-	}
-	return false
-}
-
-func (m *TxnResponse) GetResponses() []*ResponseOp {
-	if m != nil {
-		return m.Responses
-	}
-	return nil
-}
-
-// CompactionRequest compacts the key-value store up to a given revision. All superseded keys
-// with a revision less than the compaction revision will be removed.
-type CompactionRequest struct {
-	// revision is the key-value store revision for the compaction operation.
-	Revision int64 `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"`
-	// physical is set so the RPC will wait until the compaction is physically
-	// applied to the local database such that compacted entries are totally
-	// removed from the backend database.
-	Physical bool `protobuf:"varint,2,opt,name=physical,proto3" json:"physical,omitempty"`
-}
-
-func (m *CompactionRequest) Reset()                    { *m = CompactionRequest{} }
-func (m *CompactionRequest) String() string            { return proto.CompactTextString(m) }
-func (*CompactionRequest) ProtoMessage()               {}
-func (*CompactionRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{12} }
-
-func (m *CompactionRequest) GetRevision() int64 {
-	if m != nil {
-		return m.Revision
-	}
-	return 0
-}
-
-func (m *CompactionRequest) GetPhysical() bool {
-	if m != nil {
-		return m.Physical
-	}
-	return false
-}
-
-type CompactionResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *CompactionResponse) Reset()                    { *m = CompactionResponse{} }
-func (m *CompactionResponse) String() string            { return proto.CompactTextString(m) }
-func (*CompactionResponse) ProtoMessage()               {}
-func (*CompactionResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{13} }
-
-func (m *CompactionResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type HashRequest struct {
-}
-
-func (m *HashRequest) Reset()                    { *m = HashRequest{} }
-func (m *HashRequest) String() string            { return proto.CompactTextString(m) }
-func (*HashRequest) ProtoMessage()               {}
-func (*HashRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{14} }
-
-type HashKVRequest struct {
-	// revision is the key-value store revision for the hash operation.
-	Revision int64 `protobuf:"varint,1,opt,name=revision,proto3" json:"revision,omitempty"`
-}
-
-func (m *HashKVRequest) Reset()                    { *m = HashKVRequest{} }
-func (m *HashKVRequest) String() string            { return proto.CompactTextString(m) }
-func (*HashKVRequest) ProtoMessage()               {}
-func (*HashKVRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{15} }
-
-func (m *HashKVRequest) GetRevision() int64 {
-	if m != nil {
-		return m.Revision
-	}
-	return 0
-}
-
-type HashKVResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// hash is the hash value computed from the responding member's MVCC keys up to a given revision.
-	Hash uint32 `protobuf:"varint,2,opt,name=hash,proto3" json:"hash,omitempty"`
-	// compact_revision is the compacted revision of key-value store when hash begins.
-	CompactRevision int64 `protobuf:"varint,3,opt,name=compact_revision,json=compactRevision,proto3" json:"compact_revision,omitempty"`
-}
-
-func (m *HashKVResponse) Reset()                    { *m = HashKVResponse{} }
-func (m *HashKVResponse) String() string            { return proto.CompactTextString(m) }
-func (*HashKVResponse) ProtoMessage()               {}
-func (*HashKVResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{16} }
-
-func (m *HashKVResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *HashKVResponse) GetHash() uint32 {
-	if m != nil {
-		return m.Hash
-	}
-	return 0
-}
-
-func (m *HashKVResponse) GetCompactRevision() int64 {
-	if m != nil {
-		return m.CompactRevision
-	}
-	return 0
-}
-
-type HashResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// hash is the hash value computed from the responding member's KV's backend.
-	Hash uint32 `protobuf:"varint,2,opt,name=hash,proto3" json:"hash,omitempty"`
-}
-
-func (m *HashResponse) Reset()                    { *m = HashResponse{} }
-func (m *HashResponse) String() string            { return proto.CompactTextString(m) }
-func (*HashResponse) ProtoMessage()               {}
-func (*HashResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{17} }
-
-func (m *HashResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *HashResponse) GetHash() uint32 {
-	if m != nil {
-		return m.Hash
-	}
-	return 0
-}
-
-type SnapshotRequest struct {
-}
-
-func (m *SnapshotRequest) Reset()                    { *m = SnapshotRequest{} }
-func (m *SnapshotRequest) String() string            { return proto.CompactTextString(m) }
-func (*SnapshotRequest) ProtoMessage()               {}
-func (*SnapshotRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{18} }
-
-type SnapshotResponse struct {
-	// header has the current key-value store information. The first header in the snapshot
-	// stream indicates the point in time of the snapshot.
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// remaining_bytes is the number of blob bytes to be sent after this message
-	RemainingBytes uint64 `protobuf:"varint,2,opt,name=remaining_bytes,json=remainingBytes,proto3" json:"remaining_bytes,omitempty"`
-	// blob contains the next chunk of the snapshot in the snapshot stream.
-	Blob []byte `protobuf:"bytes,3,opt,name=blob,proto3" json:"blob,omitempty"`
-}
-
-func (m *SnapshotResponse) Reset()                    { *m = SnapshotResponse{} }
-func (m *SnapshotResponse) String() string            { return proto.CompactTextString(m) }
-func (*SnapshotResponse) ProtoMessage()               {}
-func (*SnapshotResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{19} }
-
-func (m *SnapshotResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *SnapshotResponse) GetRemainingBytes() uint64 {
-	if m != nil {
-		return m.RemainingBytes
-	}
-	return 0
-}
-
-func (m *SnapshotResponse) GetBlob() []byte {
-	if m != nil {
-		return m.Blob
-	}
-	return nil
-}
-
-type WatchRequest struct {
-	// request_union is a request to either create a new watcher or cancel an existing watcher.
-	//
-	// Types that are valid to be assigned to RequestUnion:
-	//	*WatchRequest_CreateRequest
-	//	*WatchRequest_CancelRequest
-	//	*WatchRequest_ProgressRequest
-	RequestUnion isWatchRequest_RequestUnion `protobuf_oneof:"request_union"`
-}
-
-func (m *WatchRequest) Reset()                    { *m = WatchRequest{} }
-func (m *WatchRequest) String() string            { return proto.CompactTextString(m) }
-func (*WatchRequest) ProtoMessage()               {}
-func (*WatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{20} }
-
-type isWatchRequest_RequestUnion interface {
-	isWatchRequest_RequestUnion()
-	MarshalTo([]byte) (int, error)
-	Size() int
-}
-
-type WatchRequest_CreateRequest struct {
-	CreateRequest *WatchCreateRequest `protobuf:"bytes,1,opt,name=create_request,json=createRequest,oneof"`
-}
-type WatchRequest_CancelRequest struct {
-	CancelRequest *WatchCancelRequest `protobuf:"bytes,2,opt,name=cancel_request,json=cancelRequest,oneof"`
-}
-type WatchRequest_ProgressRequest struct {
-	ProgressRequest *WatchProgressRequest `protobuf:"bytes,3,opt,name=progress_request,json=progressRequest,oneof"`
-}
-
-func (*WatchRequest_CreateRequest) isWatchRequest_RequestUnion()   {}
-func (*WatchRequest_CancelRequest) isWatchRequest_RequestUnion()   {}
-func (*WatchRequest_ProgressRequest) isWatchRequest_RequestUnion() {}
-
-func (m *WatchRequest) GetRequestUnion() isWatchRequest_RequestUnion {
-	if m != nil {
-		return m.RequestUnion
-	}
-	return nil
-}
-
-func (m *WatchRequest) GetCreateRequest() *WatchCreateRequest {
-	if x, ok := m.GetRequestUnion().(*WatchRequest_CreateRequest); ok {
-		return x.CreateRequest
-	}
-	return nil
-}
-
-func (m *WatchRequest) GetCancelRequest() *WatchCancelRequest {
-	if x, ok := m.GetRequestUnion().(*WatchRequest_CancelRequest); ok {
-		return x.CancelRequest
-	}
-	return nil
-}
-
-func (m *WatchRequest) GetProgressRequest() *WatchProgressRequest {
-	if x, ok := m.GetRequestUnion().(*WatchRequest_ProgressRequest); ok {
-		return x.ProgressRequest
-	}
-	return nil
-}
-
-// XXX_OneofFuncs is for the internal use of the proto package.
-func (*WatchRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
-	return _WatchRequest_OneofMarshaler, _WatchRequest_OneofUnmarshaler, _WatchRequest_OneofSizer, []interface{}{
-		(*WatchRequest_CreateRequest)(nil),
-		(*WatchRequest_CancelRequest)(nil),
-		(*WatchRequest_ProgressRequest)(nil),
-	}
-}
-
-func _WatchRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
-	m := msg.(*WatchRequest)
-	// request_union
-	switch x := m.RequestUnion.(type) {
-	case *WatchRequest_CreateRequest:
-		_ = b.EncodeVarint(1<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.CreateRequest); err != nil {
-			return err
-		}
-	case *WatchRequest_CancelRequest:
-		_ = b.EncodeVarint(2<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.CancelRequest); err != nil {
-			return err
-		}
-	case *WatchRequest_ProgressRequest:
-		_ = b.EncodeVarint(3<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ProgressRequest); err != nil {
-			return err
-		}
-	case nil:
-	default:
-		return fmt.Errorf("WatchRequest.RequestUnion has unexpected type %T", x)
-	}
-	return nil
-}
-
-func _WatchRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
-	m := msg.(*WatchRequest)
-	switch tag {
-	case 1: // request_union.create_request
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(WatchCreateRequest)
-		err := b.DecodeMessage(msg)
-		m.RequestUnion = &WatchRequest_CreateRequest{msg}
-		return true, err
-	case 2: // request_union.cancel_request
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(WatchCancelRequest)
-		err := b.DecodeMessage(msg)
-		m.RequestUnion = &WatchRequest_CancelRequest{msg}
-		return true, err
-	case 3: // request_union.progress_request
-		if wire != proto.WireBytes {
-			return true, proto.ErrInternalBadWireType
-		}
-		msg := new(WatchProgressRequest)
-		err := b.DecodeMessage(msg)
-		m.RequestUnion = &WatchRequest_ProgressRequest{msg}
-		return true, err
-	default:
-		return false, nil
-	}
-}
-
-func _WatchRequest_OneofSizer(msg proto.Message) (n int) {
-	m := msg.(*WatchRequest)
-	// request_union
-	switch x := m.RequestUnion.(type) {
-	case *WatchRequest_CreateRequest:
-		s := proto.Size(x.CreateRequest)
-		n += proto.SizeVarint(1<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *WatchRequest_CancelRequest:
-		s := proto.Size(x.CancelRequest)
-		n += proto.SizeVarint(2<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case *WatchRequest_ProgressRequest:
-		s := proto.Size(x.ProgressRequest)
-		n += proto.SizeVarint(3<<3 | proto.WireBytes)
-		n += proto.SizeVarint(uint64(s))
-		n += s
-	case nil:
-	default:
-		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
-	}
-	return n
-}
-
-type WatchCreateRequest struct {
-	// key is the key to register for watching.
-	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	// range_end is the end of the range [key, range_end) to watch. If range_end is not given,
-	// only the key argument is watched. If range_end is equal to '\0', all keys greater than
-	// or equal to the key argument are watched.
-	// If the range_end is one bit larger than the given key,
-	// then all keys with the prefix (the given key) will be watched.
-	RangeEnd []byte `protobuf:"bytes,2,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-	// start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
-	StartRevision int64 `protobuf:"varint,3,opt,name=start_revision,json=startRevision,proto3" json:"start_revision,omitempty"`
-	// progress_notify is set so that the etcd server will periodically send a WatchResponse with
-	// no events to the new watcher if there are no recent events. It is useful when clients
-	// wish to recover a disconnected watcher starting from a recent known revision.
-	// The etcd server may decide how often it will send notifications based on current load.
-	ProgressNotify bool `protobuf:"varint,4,opt,name=progress_notify,json=progressNotify,proto3" json:"progress_notify,omitempty"`
-	// filters filter the events at server side before it sends back to the watcher.
-	Filters []WatchCreateRequest_FilterType `protobuf:"varint,5,rep,packed,name=filters,enum=etcdserverpb.WatchCreateRequest_FilterType" json:"filters,omitempty"`
-	// If prev_kv is set, created watcher gets the previous KV before the event happens.
-	// If the previous KV is already compacted, nothing will be returned.
-	PrevKv bool `protobuf:"varint,6,opt,name=prev_kv,json=prevKv,proto3" json:"prev_kv,omitempty"`
-	// If watch_id is provided and non-zero, it will be assigned to this watcher.
-	// Since creating a watcher in etcd is not a synchronous operation,
-	// this can be used ensure that ordering is correct when creating multiple
-	// watchers on the same stream. Creating a watcher with an ID already in
-	// use on the stream will cause an error to be returned.
-	WatchId int64 `protobuf:"varint,7,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
-	// fragment enables splitting large revisions into multiple watch responses.
-	Fragment bool `protobuf:"varint,8,opt,name=fragment,proto3" json:"fragment,omitempty"`
-}
-
-func (m *WatchCreateRequest) Reset()                    { *m = WatchCreateRequest{} }
-func (m *WatchCreateRequest) String() string            { return proto.CompactTextString(m) }
-func (*WatchCreateRequest) ProtoMessage()               {}
-func (*WatchCreateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{21} }
-
-func (m *WatchCreateRequest) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *WatchCreateRequest) GetRangeEnd() []byte {
-	if m != nil {
-		return m.RangeEnd
-	}
-	return nil
-}
-
-func (m *WatchCreateRequest) GetStartRevision() int64 {
-	if m != nil {
-		return m.StartRevision
-	}
-	return 0
-}
-
-func (m *WatchCreateRequest) GetProgressNotify() bool {
-	if m != nil {
-		return m.ProgressNotify
-	}
-	return false
-}
-
-func (m *WatchCreateRequest) GetFilters() []WatchCreateRequest_FilterType {
-	if m != nil {
-		return m.Filters
-	}
-	return nil
-}
-
-func (m *WatchCreateRequest) GetPrevKv() bool {
-	if m != nil {
-		return m.PrevKv
-	}
-	return false
-}
-
-func (m *WatchCreateRequest) GetWatchId() int64 {
-	if m != nil {
-		return m.WatchId
-	}
-	return 0
-}
-
-func (m *WatchCreateRequest) GetFragment() bool {
-	if m != nil {
-		return m.Fragment
-	}
-	return false
-}
-
-type WatchCancelRequest struct {
-	// watch_id is the watcher id to cancel so that no more events are transmitted.
-	WatchId int64 `protobuf:"varint,1,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
-}
-
-func (m *WatchCancelRequest) Reset()                    { *m = WatchCancelRequest{} }
-func (m *WatchCancelRequest) String() string            { return proto.CompactTextString(m) }
-func (*WatchCancelRequest) ProtoMessage()               {}
-func (*WatchCancelRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{22} }
-
-func (m *WatchCancelRequest) GetWatchId() int64 {
-	if m != nil {
-		return m.WatchId
-	}
-	return 0
-}
-
-// Requests the a watch stream progress status be sent in the watch response stream as soon as
-// possible.
-type WatchProgressRequest struct {
-}
-
-func (m *WatchProgressRequest) Reset()                    { *m = WatchProgressRequest{} }
-func (m *WatchProgressRequest) String() string            { return proto.CompactTextString(m) }
-func (*WatchProgressRequest) ProtoMessage()               {}
-func (*WatchProgressRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{23} }
-
-type WatchResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// watch_id is the ID of the watcher that corresponds to the response.
-	WatchId int64 `protobuf:"varint,2,opt,name=watch_id,json=watchId,proto3" json:"watch_id,omitempty"`
-	// created is set to true if the response is for a create watch request.
-	// The client should record the watch_id and expect to receive events for
-	// the created watcher from the same stream.
-	// All events sent to the created watcher will attach with the same watch_id.
-	Created bool `protobuf:"varint,3,opt,name=created,proto3" json:"created,omitempty"`
-	// canceled is set to true if the response is for a cancel watch request.
-	// No further events will be sent to the canceled watcher.
-	Canceled bool `protobuf:"varint,4,opt,name=canceled,proto3" json:"canceled,omitempty"`
-	// compact_revision is set to the minimum index if a watcher tries to watch
-	// at a compacted index.
-	//
-	// This happens when creating a watcher at a compacted revision or the watcher cannot
-	// catch up with the progress of the key-value store.
-	//
-	// The client should treat the watcher as canceled and should not try to create any
-	// watcher with the same start_revision again.
-	CompactRevision int64 `protobuf:"varint,5,opt,name=compact_revision,json=compactRevision,proto3" json:"compact_revision,omitempty"`
-	// cancel_reason indicates the reason for canceling the watcher.
-	CancelReason string `protobuf:"bytes,6,opt,name=cancel_reason,json=cancelReason,proto3" json:"cancel_reason,omitempty"`
-	// framgment is true if large watch response was split over multiple responses.
-	Fragment bool            `protobuf:"varint,7,opt,name=fragment,proto3" json:"fragment,omitempty"`
-	Events   []*mvccpb.Event `protobuf:"bytes,11,rep,name=events" json:"events,omitempty"`
-}
-
-func (m *WatchResponse) Reset()                    { *m = WatchResponse{} }
-func (m *WatchResponse) String() string            { return proto.CompactTextString(m) }
-func (*WatchResponse) ProtoMessage()               {}
-func (*WatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{24} }
-
-func (m *WatchResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *WatchResponse) GetWatchId() int64 {
-	if m != nil {
-		return m.WatchId
-	}
-	return 0
-}
-
-func (m *WatchResponse) GetCreated() bool {
-	if m != nil {
-		return m.Created
-	}
-	return false
-}
-
-func (m *WatchResponse) GetCanceled() bool {
-	if m != nil {
-		return m.Canceled
-	}
-	return false
-}
-
-func (m *WatchResponse) GetCompactRevision() int64 {
-	if m != nil {
-		return m.CompactRevision
-	}
-	return 0
-}
-
-func (m *WatchResponse) GetCancelReason() string {
-	if m != nil {
-		return m.CancelReason
-	}
-	return ""
-}
-
-func (m *WatchResponse) GetFragment() bool {
-	if m != nil {
-		return m.Fragment
-	}
-	return false
-}
-
-func (m *WatchResponse) GetEvents() []*mvccpb.Event {
-	if m != nil {
-		return m.Events
-	}
-	return nil
-}
-
-type LeaseGrantRequest struct {
-	// TTL is the advisory time-to-live in seconds. Expired lease will return -1.
-	TTL int64 `protobuf:"varint,1,opt,name=TTL,proto3" json:"TTL,omitempty"`
-	// ID is the requested ID for the lease. If ID is set to 0, the lessor chooses an ID.
-	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *LeaseGrantRequest) Reset()                    { *m = LeaseGrantRequest{} }
-func (m *LeaseGrantRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseGrantRequest) ProtoMessage()               {}
-func (*LeaseGrantRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{25} }
-
-func (m *LeaseGrantRequest) GetTTL() int64 {
-	if m != nil {
-		return m.TTL
-	}
-	return 0
-}
-
-func (m *LeaseGrantRequest) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type LeaseGrantResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// ID is the lease ID for the granted lease.
-	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
-	// TTL is the server chosen lease time-to-live in seconds.
-	TTL   int64  `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
-	Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"`
-}
-
-func (m *LeaseGrantResponse) Reset()                    { *m = LeaseGrantResponse{} }
-func (m *LeaseGrantResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseGrantResponse) ProtoMessage()               {}
-func (*LeaseGrantResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{26} }
-
-func (m *LeaseGrantResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *LeaseGrantResponse) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *LeaseGrantResponse) GetTTL() int64 {
-	if m != nil {
-		return m.TTL
-	}
-	return 0
-}
-
-func (m *LeaseGrantResponse) GetError() string {
-	if m != nil {
-		return m.Error
-	}
-	return ""
-}
-
-type LeaseRevokeRequest struct {
-	// ID is the lease ID to revoke. When the ID is revoked, all associated keys will be deleted.
-	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *LeaseRevokeRequest) Reset()                    { *m = LeaseRevokeRequest{} }
-func (m *LeaseRevokeRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseRevokeRequest) ProtoMessage()               {}
-func (*LeaseRevokeRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{27} }
-
-func (m *LeaseRevokeRequest) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type LeaseRevokeResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *LeaseRevokeResponse) Reset()                    { *m = LeaseRevokeResponse{} }
-func (m *LeaseRevokeResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseRevokeResponse) ProtoMessage()               {}
-func (*LeaseRevokeResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{28} }
-
-func (m *LeaseRevokeResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type LeaseCheckpoint struct {
-	// ID is the lease ID to checkpoint.
-	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	// Remaining_TTL is the remaining time until expiry of the lease.
-	Remaining_TTL int64 `protobuf:"varint,2,opt,name=remaining_TTL,json=remainingTTL,proto3" json:"remaining_TTL,omitempty"`
-}
-
-func (m *LeaseCheckpoint) Reset()                    { *m = LeaseCheckpoint{} }
-func (m *LeaseCheckpoint) String() string            { return proto.CompactTextString(m) }
-func (*LeaseCheckpoint) ProtoMessage()               {}
-func (*LeaseCheckpoint) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{29} }
-
-func (m *LeaseCheckpoint) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *LeaseCheckpoint) GetRemaining_TTL() int64 {
-	if m != nil {
-		return m.Remaining_TTL
-	}
-	return 0
-}
-
-type LeaseCheckpointRequest struct {
-	Checkpoints []*LeaseCheckpoint `protobuf:"bytes,1,rep,name=checkpoints" json:"checkpoints,omitempty"`
-}
-
-func (m *LeaseCheckpointRequest) Reset()                    { *m = LeaseCheckpointRequest{} }
-func (m *LeaseCheckpointRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseCheckpointRequest) ProtoMessage()               {}
-func (*LeaseCheckpointRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{30} }
-
-func (m *LeaseCheckpointRequest) GetCheckpoints() []*LeaseCheckpoint {
-	if m != nil {
-		return m.Checkpoints
-	}
-	return nil
-}
-
-type LeaseCheckpointResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *LeaseCheckpointResponse) Reset()                    { *m = LeaseCheckpointResponse{} }
-func (m *LeaseCheckpointResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseCheckpointResponse) ProtoMessage()               {}
-func (*LeaseCheckpointResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{31} }
-
-func (m *LeaseCheckpointResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type LeaseKeepAliveRequest struct {
-	// ID is the lease ID for the lease to keep alive.
-	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *LeaseKeepAliveRequest) Reset()                    { *m = LeaseKeepAliveRequest{} }
-func (m *LeaseKeepAliveRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseKeepAliveRequest) ProtoMessage()               {}
-func (*LeaseKeepAliveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{32} }
-
-func (m *LeaseKeepAliveRequest) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type LeaseKeepAliveResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// ID is the lease ID from the keep alive request.
-	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
-	// TTL is the new time-to-live for the lease.
-	TTL int64 `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
-}
-
-func (m *LeaseKeepAliveResponse) Reset()                    { *m = LeaseKeepAliveResponse{} }
-func (m *LeaseKeepAliveResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseKeepAliveResponse) ProtoMessage()               {}
-func (*LeaseKeepAliveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{33} }
-
-func (m *LeaseKeepAliveResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *LeaseKeepAliveResponse) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *LeaseKeepAliveResponse) GetTTL() int64 {
-	if m != nil {
-		return m.TTL
-	}
-	return 0
-}
-
-type LeaseTimeToLiveRequest struct {
-	// ID is the lease ID for the lease.
-	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	// keys is true to query all the keys attached to this lease.
-	Keys bool `protobuf:"varint,2,opt,name=keys,proto3" json:"keys,omitempty"`
-}
-
-func (m *LeaseTimeToLiveRequest) Reset()                    { *m = LeaseTimeToLiveRequest{} }
-func (m *LeaseTimeToLiveRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseTimeToLiveRequest) ProtoMessage()               {}
-func (*LeaseTimeToLiveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{34} }
-
-func (m *LeaseTimeToLiveRequest) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *LeaseTimeToLiveRequest) GetKeys() bool {
-	if m != nil {
-		return m.Keys
-	}
-	return false
-}
-
-type LeaseTimeToLiveResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// ID is the lease ID from the keep alive request.
-	ID int64 `protobuf:"varint,2,opt,name=ID,proto3" json:"ID,omitempty"`
-	// TTL is the remaining TTL in seconds for the lease; the lease will expire in under TTL+1 seconds.
-	TTL int64 `protobuf:"varint,3,opt,name=TTL,proto3" json:"TTL,omitempty"`
-	// GrantedTTL is the initial granted time in seconds upon lease creation/renewal.
-	GrantedTTL int64 `protobuf:"varint,4,opt,name=grantedTTL,proto3" json:"grantedTTL,omitempty"`
-	// Keys is the list of keys attached to this lease.
-	Keys [][]byte `protobuf:"bytes,5,rep,name=keys" json:"keys,omitempty"`
-}
-
-func (m *LeaseTimeToLiveResponse) Reset()                    { *m = LeaseTimeToLiveResponse{} }
-func (m *LeaseTimeToLiveResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseTimeToLiveResponse) ProtoMessage()               {}
-func (*LeaseTimeToLiveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{35} }
-
-func (m *LeaseTimeToLiveResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *LeaseTimeToLiveResponse) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *LeaseTimeToLiveResponse) GetTTL() int64 {
-	if m != nil {
-		return m.TTL
-	}
-	return 0
-}
-
-func (m *LeaseTimeToLiveResponse) GetGrantedTTL() int64 {
-	if m != nil {
-		return m.GrantedTTL
-	}
-	return 0
-}
-
-func (m *LeaseTimeToLiveResponse) GetKeys() [][]byte {
-	if m != nil {
-		return m.Keys
-	}
-	return nil
-}
-
-type LeaseLeasesRequest struct {
-}
-
-func (m *LeaseLeasesRequest) Reset()                    { *m = LeaseLeasesRequest{} }
-func (m *LeaseLeasesRequest) String() string            { return proto.CompactTextString(m) }
-func (*LeaseLeasesRequest) ProtoMessage()               {}
-func (*LeaseLeasesRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{36} }
-
-type LeaseStatus struct {
-	ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *LeaseStatus) Reset()                    { *m = LeaseStatus{} }
-func (m *LeaseStatus) String() string            { return proto.CompactTextString(m) }
-func (*LeaseStatus) ProtoMessage()               {}
-func (*LeaseStatus) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{37} }
-
-func (m *LeaseStatus) GetID() int64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type LeaseLeasesResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	Leases []*LeaseStatus  `protobuf:"bytes,2,rep,name=leases" json:"leases,omitempty"`
-}
-
-func (m *LeaseLeasesResponse) Reset()                    { *m = LeaseLeasesResponse{} }
-func (m *LeaseLeasesResponse) String() string            { return proto.CompactTextString(m) }
-func (*LeaseLeasesResponse) ProtoMessage()               {}
-func (*LeaseLeasesResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{38} }
-
-func (m *LeaseLeasesResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *LeaseLeasesResponse) GetLeases() []*LeaseStatus {
-	if m != nil {
-		return m.Leases
-	}
-	return nil
-}
-
-type Member struct {
-	// ID is the member ID for this member.
-	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	// name is the human-readable name of the member. If the member is not started, the name will be an empty string.
-	Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	// peerURLs is the list of URLs the member exposes to the cluster for communication.
-	PeerURLs []string `protobuf:"bytes,3,rep,name=peerURLs" json:"peerURLs,omitempty"`
-	// clientURLs is the list of URLs the member exposes to clients for communication. If the member is not started, clientURLs will be empty.
-	ClientURLs []string `protobuf:"bytes,4,rep,name=clientURLs" json:"clientURLs,omitempty"`
-	// isLearner indicates if the member is raft learner.
-	IsLearner bool `protobuf:"varint,5,opt,name=isLearner,proto3" json:"isLearner,omitempty"`
-}
-
-func (m *Member) Reset()                    { *m = Member{} }
-func (m *Member) String() string            { return proto.CompactTextString(m) }
-func (*Member) ProtoMessage()               {}
-func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{39} }
-
-func (m *Member) GetID() uint64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *Member) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *Member) GetPeerURLs() []string {
-	if m != nil {
-		return m.PeerURLs
-	}
-	return nil
-}
-
-func (m *Member) GetClientURLs() []string {
-	if m != nil {
-		return m.ClientURLs
-	}
-	return nil
-}
-
-func (m *Member) GetIsLearner() bool {
-	if m != nil {
-		return m.IsLearner
-	}
-	return false
-}
-
-type MemberAddRequest struct {
-	// peerURLs is the list of URLs the added member will use to communicate with the cluster.
-	PeerURLs []string `protobuf:"bytes,1,rep,name=peerURLs" json:"peerURLs,omitempty"`
-	// isLearner indicates if the added member is raft learner.
-	IsLearner bool `protobuf:"varint,2,opt,name=isLearner,proto3" json:"isLearner,omitempty"`
-}
-
-func (m *MemberAddRequest) Reset()                    { *m = MemberAddRequest{} }
-func (m *MemberAddRequest) String() string            { return proto.CompactTextString(m) }
-func (*MemberAddRequest) ProtoMessage()               {}
-func (*MemberAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{40} }
-
-func (m *MemberAddRequest) GetPeerURLs() []string {
-	if m != nil {
-		return m.PeerURLs
-	}
-	return nil
-}
-
-func (m *MemberAddRequest) GetIsLearner() bool {
-	if m != nil {
-		return m.IsLearner
-	}
-	return false
-}
-
-type MemberAddResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// member is the member information for the added member.
-	Member *Member `protobuf:"bytes,2,opt,name=member" json:"member,omitempty"`
-	// members is a list of all members after adding the new member.
-	Members []*Member `protobuf:"bytes,3,rep,name=members" json:"members,omitempty"`
-}
-
-func (m *MemberAddResponse) Reset()                    { *m = MemberAddResponse{} }
-func (m *MemberAddResponse) String() string            { return proto.CompactTextString(m) }
-func (*MemberAddResponse) ProtoMessage()               {}
-func (*MemberAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{41} }
-
-func (m *MemberAddResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *MemberAddResponse) GetMember() *Member {
-	if m != nil {
-		return m.Member
-	}
-	return nil
-}
-
-func (m *MemberAddResponse) GetMembers() []*Member {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-type MemberRemoveRequest struct {
-	// ID is the member ID of the member to remove.
-	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *MemberRemoveRequest) Reset()                    { *m = MemberRemoveRequest{} }
-func (m *MemberRemoveRequest) String() string            { return proto.CompactTextString(m) }
-func (*MemberRemoveRequest) ProtoMessage()               {}
-func (*MemberRemoveRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{42} }
-
-func (m *MemberRemoveRequest) GetID() uint64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type MemberRemoveResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// members is a list of all members after removing the member.
-	Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"`
-}
-
-func (m *MemberRemoveResponse) Reset()                    { *m = MemberRemoveResponse{} }
-func (m *MemberRemoveResponse) String() string            { return proto.CompactTextString(m) }
-func (*MemberRemoveResponse) ProtoMessage()               {}
-func (*MemberRemoveResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{43} }
-
-func (m *MemberRemoveResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *MemberRemoveResponse) GetMembers() []*Member {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-type MemberUpdateRequest struct {
-	// ID is the member ID of the member to update.
-	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-	// peerURLs is the new list of URLs the member will use to communicate with the cluster.
-	PeerURLs []string `protobuf:"bytes,2,rep,name=peerURLs" json:"peerURLs,omitempty"`
-}
-
-func (m *MemberUpdateRequest) Reset()                    { *m = MemberUpdateRequest{} }
-func (m *MemberUpdateRequest) String() string            { return proto.CompactTextString(m) }
-func (*MemberUpdateRequest) ProtoMessage()               {}
-func (*MemberUpdateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{44} }
-
-func (m *MemberUpdateRequest) GetID() uint64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-func (m *MemberUpdateRequest) GetPeerURLs() []string {
-	if m != nil {
-		return m.PeerURLs
-	}
-	return nil
-}
-
-type MemberUpdateResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// members is a list of all members after updating the member.
-	Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"`
-}
-
-func (m *MemberUpdateResponse) Reset()                    { *m = MemberUpdateResponse{} }
-func (m *MemberUpdateResponse) String() string            { return proto.CompactTextString(m) }
-func (*MemberUpdateResponse) ProtoMessage()               {}
-func (*MemberUpdateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{45} }
-
-func (m *MemberUpdateResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *MemberUpdateResponse) GetMembers() []*Member {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-type MemberListRequest struct {
-}
-
-func (m *MemberListRequest) Reset()                    { *m = MemberListRequest{} }
-func (m *MemberListRequest) String() string            { return proto.CompactTextString(m) }
-func (*MemberListRequest) ProtoMessage()               {}
-func (*MemberListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{46} }
-
-type MemberListResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// members is a list of all members associated with the cluster.
-	Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"`
-}
-
-func (m *MemberListResponse) Reset()                    { *m = MemberListResponse{} }
-func (m *MemberListResponse) String() string            { return proto.CompactTextString(m) }
-func (*MemberListResponse) ProtoMessage()               {}
-func (*MemberListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{47} }
-
-func (m *MemberListResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *MemberListResponse) GetMembers() []*Member {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-type MemberPromoteRequest struct {
-	// ID is the member ID of the member to promote.
-	ID uint64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
-}
-
-func (m *MemberPromoteRequest) Reset()                    { *m = MemberPromoteRequest{} }
-func (m *MemberPromoteRequest) String() string            { return proto.CompactTextString(m) }
-func (*MemberPromoteRequest) ProtoMessage()               {}
-func (*MemberPromoteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{48} }
-
-func (m *MemberPromoteRequest) GetID() uint64 {
-	if m != nil {
-		return m.ID
-	}
-	return 0
-}
-
-type MemberPromoteResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// members is a list of all members after promoting the member.
-	Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"`
-}
-
-func (m *MemberPromoteResponse) Reset()                    { *m = MemberPromoteResponse{} }
-func (m *MemberPromoteResponse) String() string            { return proto.CompactTextString(m) }
-func (*MemberPromoteResponse) ProtoMessage()               {}
-func (*MemberPromoteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{49} }
-
-func (m *MemberPromoteResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *MemberPromoteResponse) GetMembers() []*Member {
-	if m != nil {
-		return m.Members
-	}
-	return nil
-}
-
-type DefragmentRequest struct {
-}
-
-func (m *DefragmentRequest) Reset()                    { *m = DefragmentRequest{} }
-func (m *DefragmentRequest) String() string            { return proto.CompactTextString(m) }
-func (*DefragmentRequest) ProtoMessage()               {}
-func (*DefragmentRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{50} }
-
-type DefragmentResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *DefragmentResponse) Reset()                    { *m = DefragmentResponse{} }
-func (m *DefragmentResponse) String() string            { return proto.CompactTextString(m) }
-func (*DefragmentResponse) ProtoMessage()               {}
-func (*DefragmentResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{51} }
-
-func (m *DefragmentResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type MoveLeaderRequest struct {
-	// targetID is the node ID for the new leader.
-	TargetID uint64 `protobuf:"varint,1,opt,name=targetID,proto3" json:"targetID,omitempty"`
-}
-
-func (m *MoveLeaderRequest) Reset()                    { *m = MoveLeaderRequest{} }
-func (m *MoveLeaderRequest) String() string            { return proto.CompactTextString(m) }
-func (*MoveLeaderRequest) ProtoMessage()               {}
-func (*MoveLeaderRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{52} }
-
-func (m *MoveLeaderRequest) GetTargetID() uint64 {
-	if m != nil {
-		return m.TargetID
-	}
-	return 0
-}
-
-type MoveLeaderResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *MoveLeaderResponse) Reset()                    { *m = MoveLeaderResponse{} }
-func (m *MoveLeaderResponse) String() string            { return proto.CompactTextString(m) }
-func (*MoveLeaderResponse) ProtoMessage()               {}
-func (*MoveLeaderResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{53} }
-
-func (m *MoveLeaderResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AlarmRequest struct {
-	// action is the kind of alarm request to issue. The action
-	// may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
-	// raised alarm.
-	Action AlarmRequest_AlarmAction `protobuf:"varint,1,opt,name=action,proto3,enum=etcdserverpb.AlarmRequest_AlarmAction" json:"action,omitempty"`
-	// memberID is the ID of the member associated with the alarm. If memberID is 0, the
-	// alarm request covers all members.
-	MemberID uint64 `protobuf:"varint,2,opt,name=memberID,proto3" json:"memberID,omitempty"`
-	// alarm is the type of alarm to consider for this request.
-	Alarm AlarmType `protobuf:"varint,3,opt,name=alarm,proto3,enum=etcdserverpb.AlarmType" json:"alarm,omitempty"`
-}
-
-func (m *AlarmRequest) Reset()                    { *m = AlarmRequest{} }
-func (m *AlarmRequest) String() string            { return proto.CompactTextString(m) }
-func (*AlarmRequest) ProtoMessage()               {}
-func (*AlarmRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{54} }
-
-func (m *AlarmRequest) GetAction() AlarmRequest_AlarmAction {
-	if m != nil {
-		return m.Action
-	}
-	return AlarmRequest_GET
-}
-
-func (m *AlarmRequest) GetMemberID() uint64 {
-	if m != nil {
-		return m.MemberID
-	}
-	return 0
-}
-
-func (m *AlarmRequest) GetAlarm() AlarmType {
-	if m != nil {
-		return m.Alarm
-	}
-	return AlarmType_NONE
-}
-
-type AlarmMember struct {
-	// memberID is the ID of the member associated with the raised alarm.
-	MemberID uint64 `protobuf:"varint,1,opt,name=memberID,proto3" json:"memberID,omitempty"`
-	// alarm is the type of alarm which has been raised.
-	Alarm AlarmType `protobuf:"varint,2,opt,name=alarm,proto3,enum=etcdserverpb.AlarmType" json:"alarm,omitempty"`
-}
-
-func (m *AlarmMember) Reset()                    { *m = AlarmMember{} }
-func (m *AlarmMember) String() string            { return proto.CompactTextString(m) }
-func (*AlarmMember) ProtoMessage()               {}
-func (*AlarmMember) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{55} }
-
-func (m *AlarmMember) GetMemberID() uint64 {
-	if m != nil {
-		return m.MemberID
-	}
-	return 0
-}
-
-func (m *AlarmMember) GetAlarm() AlarmType {
-	if m != nil {
-		return m.Alarm
-	}
-	return AlarmType_NONE
-}
-
-type AlarmResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// alarms is a list of alarms associated with the alarm request.
-	Alarms []*AlarmMember `protobuf:"bytes,2,rep,name=alarms" json:"alarms,omitempty"`
-}
-
-func (m *AlarmResponse) Reset()                    { *m = AlarmResponse{} }
-func (m *AlarmResponse) String() string            { return proto.CompactTextString(m) }
-func (*AlarmResponse) ProtoMessage()               {}
-func (*AlarmResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{56} }
-
-func (m *AlarmResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AlarmResponse) GetAlarms() []*AlarmMember {
-	if m != nil {
-		return m.Alarms
-	}
-	return nil
-}
-
-type StatusRequest struct {
-}
-
-func (m *StatusRequest) Reset()                    { *m = StatusRequest{} }
-func (m *StatusRequest) String() string            { return proto.CompactTextString(m) }
-func (*StatusRequest) ProtoMessage()               {}
-func (*StatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{57} }
-
-type StatusResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// version is the cluster protocol version used by the responding member.
-	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
-	// dbSize is the size of the backend database physically allocated, in bytes, of the responding member.
-	DbSize int64 `protobuf:"varint,3,opt,name=dbSize,proto3" json:"dbSize,omitempty"`
-	// leader is the member ID which the responding member believes is the current leader.
-	Leader uint64 `protobuf:"varint,4,opt,name=leader,proto3" json:"leader,omitempty"`
-	// raftIndex is the current raft committed index of the responding member.
-	RaftIndex uint64 `protobuf:"varint,5,opt,name=raftIndex,proto3" json:"raftIndex,omitempty"`
-	// raftTerm is the current raft term of the responding member.
-	RaftTerm uint64 `protobuf:"varint,6,opt,name=raftTerm,proto3" json:"raftTerm,omitempty"`
-	// raftAppliedIndex is the current raft applied index of the responding member.
-	RaftAppliedIndex uint64 `protobuf:"varint,7,opt,name=raftAppliedIndex,proto3" json:"raftAppliedIndex,omitempty"`
-	// errors contains alarm/health information and status.
-	Errors []string `protobuf:"bytes,8,rep,name=errors" json:"errors,omitempty"`
-	// dbSizeInUse is the size of the backend database logically in use, in bytes, of the responding member.
-	DbSizeInUse int64 `protobuf:"varint,9,opt,name=dbSizeInUse,proto3" json:"dbSizeInUse,omitempty"`
-	// isLearner indicates if the member is raft learner.
-	IsLearner bool `protobuf:"varint,10,opt,name=isLearner,proto3" json:"isLearner,omitempty"`
-}
-
-func (m *StatusResponse) Reset()                    { *m = StatusResponse{} }
-func (m *StatusResponse) String() string            { return proto.CompactTextString(m) }
-func (*StatusResponse) ProtoMessage()               {}
-func (*StatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{58} }
-
-func (m *StatusResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *StatusResponse) GetVersion() string {
-	if m != nil {
-		return m.Version
-	}
-	return ""
-}
-
-func (m *StatusResponse) GetDbSize() int64 {
-	if m != nil {
-		return m.DbSize
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetLeader() uint64 {
-	if m != nil {
-		return m.Leader
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetRaftIndex() uint64 {
-	if m != nil {
-		return m.RaftIndex
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetRaftTerm() uint64 {
-	if m != nil {
-		return m.RaftTerm
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetRaftAppliedIndex() uint64 {
-	if m != nil {
-		return m.RaftAppliedIndex
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetErrors() []string {
-	if m != nil {
-		return m.Errors
-	}
-	return nil
-}
-
-func (m *StatusResponse) GetDbSizeInUse() int64 {
-	if m != nil {
-		return m.DbSizeInUse
-	}
-	return 0
-}
-
-func (m *StatusResponse) GetIsLearner() bool {
-	if m != nil {
-		return m.IsLearner
-	}
-	return false
-}
-
-type AuthEnableRequest struct {
-}
-
-func (m *AuthEnableRequest) Reset()                    { *m = AuthEnableRequest{} }
-func (m *AuthEnableRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthEnableRequest) ProtoMessage()               {}
-func (*AuthEnableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{59} }
-
-type AuthDisableRequest struct {
-}
-
-func (m *AuthDisableRequest) Reset()                    { *m = AuthDisableRequest{} }
-func (m *AuthDisableRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthDisableRequest) ProtoMessage()               {}
-func (*AuthDisableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{60} }
-
-type AuthenticateRequest struct {
-	Name     string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
-}
-
-func (m *AuthenticateRequest) Reset()                    { *m = AuthenticateRequest{} }
-func (m *AuthenticateRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthenticateRequest) ProtoMessage()               {}
-func (*AuthenticateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{61} }
-
-func (m *AuthenticateRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AuthenticateRequest) GetPassword() string {
-	if m != nil {
-		return m.Password
-	}
-	return ""
-}
-
-type AuthUserAddRequest struct {
-	Name     string                 `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Password string                 `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
-	Options  *authpb.UserAddOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
-}
-
-func (m *AuthUserAddRequest) Reset()                    { *m = AuthUserAddRequest{} }
-func (m *AuthUserAddRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserAddRequest) ProtoMessage()               {}
-func (*AuthUserAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{62} }
-
-func (m *AuthUserAddRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AuthUserAddRequest) GetPassword() string {
-	if m != nil {
-		return m.Password
-	}
-	return ""
-}
-
-func (m *AuthUserAddRequest) GetOptions() *authpb.UserAddOptions {
-	if m != nil {
-		return m.Options
-	}
-	return nil
-}
-
-type AuthUserGetRequest struct {
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-}
-
-func (m *AuthUserGetRequest) Reset()                    { *m = AuthUserGetRequest{} }
-func (m *AuthUserGetRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserGetRequest) ProtoMessage()               {}
-func (*AuthUserGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{63} }
-
-func (m *AuthUserGetRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type AuthUserDeleteRequest struct {
-	// name is the name of the user to delete.
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-}
-
-func (m *AuthUserDeleteRequest) Reset()                    { *m = AuthUserDeleteRequest{} }
-func (m *AuthUserDeleteRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserDeleteRequest) ProtoMessage()               {}
-func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{64} }
-
-func (m *AuthUserDeleteRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type AuthUserChangePasswordRequest struct {
-	// name is the name of the user whose password is being changed.
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	// password is the new password for the user.
-	Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"`
-}
-
-func (m *AuthUserChangePasswordRequest) Reset()         { *m = AuthUserChangePasswordRequest{} }
-func (m *AuthUserChangePasswordRequest) String() string { return proto.CompactTextString(m) }
-func (*AuthUserChangePasswordRequest) ProtoMessage()    {}
-func (*AuthUserChangePasswordRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{65}
-}
-
-func (m *AuthUserChangePasswordRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AuthUserChangePasswordRequest) GetPassword() string {
-	if m != nil {
-		return m.Password
-	}
-	return ""
-}
-
-type AuthUserGrantRoleRequest struct {
-	// user is the name of the user which should be granted a given role.
-	User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"`
-	// role is the name of the role to grant to the user.
-	Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
-}
-
-func (m *AuthUserGrantRoleRequest) Reset()                    { *m = AuthUserGrantRoleRequest{} }
-func (m *AuthUserGrantRoleRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserGrantRoleRequest) ProtoMessage()               {}
-func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{66} }
-
-func (m *AuthUserGrantRoleRequest) GetUser() string {
-	if m != nil {
-		return m.User
-	}
-	return ""
-}
-
-func (m *AuthUserGrantRoleRequest) GetRole() string {
-	if m != nil {
-		return m.Role
-	}
-	return ""
-}
-
-type AuthUserRevokeRoleRequest struct {
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
-}
-
-func (m *AuthUserRevokeRoleRequest) Reset()                    { *m = AuthUserRevokeRoleRequest{} }
-func (m *AuthUserRevokeRoleRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserRevokeRoleRequest) ProtoMessage()               {}
-func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{67} }
-
-func (m *AuthUserRevokeRoleRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AuthUserRevokeRoleRequest) GetRole() string {
-	if m != nil {
-		return m.Role
-	}
-	return ""
-}
-
-type AuthRoleAddRequest struct {
-	// name is the name of the role to add to the authentication system.
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-}
-
-func (m *AuthRoleAddRequest) Reset()                    { *m = AuthRoleAddRequest{} }
-func (m *AuthRoleAddRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleAddRequest) ProtoMessage()               {}
-func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{68} }
-
-func (m *AuthRoleAddRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type AuthRoleGetRequest struct {
-	Role string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
-}
-
-func (m *AuthRoleGetRequest) Reset()                    { *m = AuthRoleGetRequest{} }
-func (m *AuthRoleGetRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleGetRequest) ProtoMessage()               {}
-func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{69} }
-
-func (m *AuthRoleGetRequest) GetRole() string {
-	if m != nil {
-		return m.Role
-	}
-	return ""
-}
-
-type AuthUserListRequest struct {
-}
-
-func (m *AuthUserListRequest) Reset()                    { *m = AuthUserListRequest{} }
-func (m *AuthUserListRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserListRequest) ProtoMessage()               {}
-func (*AuthUserListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{70} }
-
-type AuthRoleListRequest struct {
-}
-
-func (m *AuthRoleListRequest) Reset()                    { *m = AuthRoleListRequest{} }
-func (m *AuthRoleListRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleListRequest) ProtoMessage()               {}
-func (*AuthRoleListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{71} }
-
-type AuthRoleDeleteRequest struct {
-	Role string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
-}
-
-func (m *AuthRoleDeleteRequest) Reset()                    { *m = AuthRoleDeleteRequest{} }
-func (m *AuthRoleDeleteRequest) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleDeleteRequest) ProtoMessage()               {}
-func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{72} }
-
-func (m *AuthRoleDeleteRequest) GetRole() string {
-	if m != nil {
-		return m.Role
-	}
-	return ""
-}
-
-type AuthRoleGrantPermissionRequest struct {
-	// name is the name of the role which will be granted the permission.
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	// perm is the permission to grant to the role.
-	Perm *authpb.Permission `protobuf:"bytes,2,opt,name=perm" json:"perm,omitempty"`
-}
-
-func (m *AuthRoleGrantPermissionRequest) Reset()         { *m = AuthRoleGrantPermissionRequest{} }
-func (m *AuthRoleGrantPermissionRequest) String() string { return proto.CompactTextString(m) }
-func (*AuthRoleGrantPermissionRequest) ProtoMessage()    {}
-func (*AuthRoleGrantPermissionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{73}
-}
-
-func (m *AuthRoleGrantPermissionRequest) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *AuthRoleGrantPermissionRequest) GetPerm() *authpb.Permission {
-	if m != nil {
-		return m.Perm
-	}
-	return nil
-}
-
-type AuthRoleRevokePermissionRequest struct {
-	Role     string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
-	Key      []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
-	RangeEnd []byte `protobuf:"bytes,3,opt,name=range_end,json=rangeEnd,proto3" json:"range_end,omitempty"`
-}
-
-func (m *AuthRoleRevokePermissionRequest) Reset()         { *m = AuthRoleRevokePermissionRequest{} }
-func (m *AuthRoleRevokePermissionRequest) String() string { return proto.CompactTextString(m) }
-func (*AuthRoleRevokePermissionRequest) ProtoMessage()    {}
-func (*AuthRoleRevokePermissionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{74}
-}
-
-func (m *AuthRoleRevokePermissionRequest) GetRole() string {
-	if m != nil {
-		return m.Role
-	}
-	return ""
-}
-
-func (m *AuthRoleRevokePermissionRequest) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *AuthRoleRevokePermissionRequest) GetRangeEnd() []byte {
-	if m != nil {
-		return m.RangeEnd
-	}
-	return nil
-}
-
-type AuthEnableResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthEnableResponse) Reset()                    { *m = AuthEnableResponse{} }
-func (m *AuthEnableResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthEnableResponse) ProtoMessage()               {}
-func (*AuthEnableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{75} }
-
-func (m *AuthEnableResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthDisableResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthDisableResponse) Reset()                    { *m = AuthDisableResponse{} }
-func (m *AuthDisableResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthDisableResponse) ProtoMessage()               {}
-func (*AuthDisableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{76} }
-
-func (m *AuthDisableResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthenticateResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	// token is an authorized token that can be used in succeeding RPCs
-	Token string `protobuf:"bytes,2,opt,name=token,proto3" json:"token,omitempty"`
-}
-
-func (m *AuthenticateResponse) Reset()                    { *m = AuthenticateResponse{} }
-func (m *AuthenticateResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthenticateResponse) ProtoMessage()               {}
-func (*AuthenticateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{77} }
-
-func (m *AuthenticateResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AuthenticateResponse) GetToken() string {
-	if m != nil {
-		return m.Token
-	}
-	return ""
-}
-
-type AuthUserAddResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthUserAddResponse) Reset()                    { *m = AuthUserAddResponse{} }
-func (m *AuthUserAddResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserAddResponse) ProtoMessage()               {}
-func (*AuthUserAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{78} }
-
-func (m *AuthUserAddResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthUserGetResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	Roles  []string        `protobuf:"bytes,2,rep,name=roles" json:"roles,omitempty"`
-}
-
-func (m *AuthUserGetResponse) Reset()                    { *m = AuthUserGetResponse{} }
-func (m *AuthUserGetResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserGetResponse) ProtoMessage()               {}
-func (*AuthUserGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{79} }
-
-func (m *AuthUserGetResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AuthUserGetResponse) GetRoles() []string {
-	if m != nil {
-		return m.Roles
-	}
-	return nil
-}
-
-type AuthUserDeleteResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthUserDeleteResponse) Reset()                    { *m = AuthUserDeleteResponse{} }
-func (m *AuthUserDeleteResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserDeleteResponse) ProtoMessage()               {}
-func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{80} }
-
-func (m *AuthUserDeleteResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthUserChangePasswordResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthUserChangePasswordResponse) Reset()         { *m = AuthUserChangePasswordResponse{} }
-func (m *AuthUserChangePasswordResponse) String() string { return proto.CompactTextString(m) }
-func (*AuthUserChangePasswordResponse) ProtoMessage()    {}
-func (*AuthUserChangePasswordResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{81}
-}
-
-func (m *AuthUserChangePasswordResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthUserGrantRoleResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthUserGrantRoleResponse) Reset()                    { *m = AuthUserGrantRoleResponse{} }
-func (m *AuthUserGrantRoleResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserGrantRoleResponse) ProtoMessage()               {}
-func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{82} }
-
-func (m *AuthUserGrantRoleResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthUserRevokeRoleResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthUserRevokeRoleResponse) Reset()                    { *m = AuthUserRevokeRoleResponse{} }
-func (m *AuthUserRevokeRoleResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserRevokeRoleResponse) ProtoMessage()               {}
-func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{83} }
-
-func (m *AuthUserRevokeRoleResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthRoleAddResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthRoleAddResponse) Reset()                    { *m = AuthRoleAddResponse{} }
-func (m *AuthRoleAddResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleAddResponse) ProtoMessage()               {}
-func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{84} }
-
-func (m *AuthRoleAddResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthRoleGetResponse struct {
-	Header *ResponseHeader      `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	Perm   []*authpb.Permission `protobuf:"bytes,2,rep,name=perm" json:"perm,omitempty"`
-}
-
-func (m *AuthRoleGetResponse) Reset()                    { *m = AuthRoleGetResponse{} }
-func (m *AuthRoleGetResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleGetResponse) ProtoMessage()               {}
-func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{85} }
-
-func (m *AuthRoleGetResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AuthRoleGetResponse) GetPerm() []*authpb.Permission {
-	if m != nil {
-		return m.Perm
-	}
-	return nil
-}
-
-type AuthRoleListResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	Roles  []string        `protobuf:"bytes,2,rep,name=roles" json:"roles,omitempty"`
-}
-
-func (m *AuthRoleListResponse) Reset()                    { *m = AuthRoleListResponse{} }
-func (m *AuthRoleListResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleListResponse) ProtoMessage()               {}
-func (*AuthRoleListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{86} }
-
-func (m *AuthRoleListResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AuthRoleListResponse) GetRoles() []string {
-	if m != nil {
-		return m.Roles
-	}
-	return nil
-}
-
-type AuthUserListResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-	Users  []string        `protobuf:"bytes,2,rep,name=users" json:"users,omitempty"`
-}
-
-func (m *AuthUserListResponse) Reset()                    { *m = AuthUserListResponse{} }
-func (m *AuthUserListResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthUserListResponse) ProtoMessage()               {}
-func (*AuthUserListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{87} }
-
-func (m *AuthUserListResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func (m *AuthUserListResponse) GetUsers() []string {
-	if m != nil {
-		return m.Users
-	}
-	return nil
-}
-
-type AuthRoleDeleteResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthRoleDeleteResponse) Reset()                    { *m = AuthRoleDeleteResponse{} }
-func (m *AuthRoleDeleteResponse) String() string            { return proto.CompactTextString(m) }
-func (*AuthRoleDeleteResponse) ProtoMessage()               {}
-func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{88} }
-
-func (m *AuthRoleDeleteResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthRoleGrantPermissionResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthRoleGrantPermissionResponse) Reset()         { *m = AuthRoleGrantPermissionResponse{} }
-func (m *AuthRoleGrantPermissionResponse) String() string { return proto.CompactTextString(m) }
-func (*AuthRoleGrantPermissionResponse) ProtoMessage()    {}
-func (*AuthRoleGrantPermissionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{89}
-}
-
-func (m *AuthRoleGrantPermissionResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-type AuthRoleRevokePermissionResponse struct {
-	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
-}
-
-func (m *AuthRoleRevokePermissionResponse) Reset()         { *m = AuthRoleRevokePermissionResponse{} }
-func (m *AuthRoleRevokePermissionResponse) String() string { return proto.CompactTextString(m) }
-func (*AuthRoleRevokePermissionResponse) ProtoMessage()    {}
-func (*AuthRoleRevokePermissionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{90}
-}
-
-func (m *AuthRoleRevokePermissionResponse) GetHeader() *ResponseHeader {
-	if m != nil {
-		return m.Header
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterType((*ResponseHeader)(nil), "etcdserverpb.ResponseHeader")
-	proto.RegisterType((*RangeRequest)(nil), "etcdserverpb.RangeRequest")
-	proto.RegisterType((*RangeResponse)(nil), "etcdserverpb.RangeResponse")
-	proto.RegisterType((*PutRequest)(nil), "etcdserverpb.PutRequest")
-	proto.RegisterType((*PutResponse)(nil), "etcdserverpb.PutResponse")
-	proto.RegisterType((*DeleteRangeRequest)(nil), "etcdserverpb.DeleteRangeRequest")
-	proto.RegisterType((*DeleteRangeResponse)(nil), "etcdserverpb.DeleteRangeResponse")
-	proto.RegisterType((*RequestOp)(nil), "etcdserverpb.RequestOp")
-	proto.RegisterType((*ResponseOp)(nil), "etcdserverpb.ResponseOp")
-	proto.RegisterType((*Compare)(nil), "etcdserverpb.Compare")
-	proto.RegisterType((*TxnRequest)(nil), "etcdserverpb.TxnRequest")
-	proto.RegisterType((*TxnResponse)(nil), "etcdserverpb.TxnResponse")
-	proto.RegisterType((*CompactionRequest)(nil), "etcdserverpb.CompactionRequest")
-	proto.RegisterType((*CompactionResponse)(nil), "etcdserverpb.CompactionResponse")
-	proto.RegisterType((*HashRequest)(nil), "etcdserverpb.HashRequest")
-	proto.RegisterType((*HashKVRequest)(nil), "etcdserverpb.HashKVRequest")
-	proto.RegisterType((*HashKVResponse)(nil), "etcdserverpb.HashKVResponse")
-	proto.RegisterType((*HashResponse)(nil), "etcdserverpb.HashResponse")
-	proto.RegisterType((*SnapshotRequest)(nil), "etcdserverpb.SnapshotRequest")
-	proto.RegisterType((*SnapshotResponse)(nil), "etcdserverpb.SnapshotResponse")
-	proto.RegisterType((*WatchRequest)(nil), "etcdserverpb.WatchRequest")
-	proto.RegisterType((*WatchCreateRequest)(nil), "etcdserverpb.WatchCreateRequest")
-	proto.RegisterType((*WatchCancelRequest)(nil), "etcdserverpb.WatchCancelRequest")
-	proto.RegisterType((*WatchProgressRequest)(nil), "etcdserverpb.WatchProgressRequest")
-	proto.RegisterType((*WatchResponse)(nil), "etcdserverpb.WatchResponse")
-	proto.RegisterType((*LeaseGrantRequest)(nil), "etcdserverpb.LeaseGrantRequest")
-	proto.RegisterType((*LeaseGrantResponse)(nil), "etcdserverpb.LeaseGrantResponse")
-	proto.RegisterType((*LeaseRevokeRequest)(nil), "etcdserverpb.LeaseRevokeRequest")
-	proto.RegisterType((*LeaseRevokeResponse)(nil), "etcdserverpb.LeaseRevokeResponse")
-	proto.RegisterType((*LeaseCheckpoint)(nil), "etcdserverpb.LeaseCheckpoint")
-	proto.RegisterType((*LeaseCheckpointRequest)(nil), "etcdserverpb.LeaseCheckpointRequest")
-	proto.RegisterType((*LeaseCheckpointResponse)(nil), "etcdserverpb.LeaseCheckpointResponse")
-	proto.RegisterType((*LeaseKeepAliveRequest)(nil), "etcdserverpb.LeaseKeepAliveRequest")
-	proto.RegisterType((*LeaseKeepAliveResponse)(nil), "etcdserverpb.LeaseKeepAliveResponse")
-	proto.RegisterType((*LeaseTimeToLiveRequest)(nil), "etcdserverpb.LeaseTimeToLiveRequest")
-	proto.RegisterType((*LeaseTimeToLiveResponse)(nil), "etcdserverpb.LeaseTimeToLiveResponse")
-	proto.RegisterType((*LeaseLeasesRequest)(nil), "etcdserverpb.LeaseLeasesRequest")
-	proto.RegisterType((*LeaseStatus)(nil), "etcdserverpb.LeaseStatus")
-	proto.RegisterType((*LeaseLeasesResponse)(nil), "etcdserverpb.LeaseLeasesResponse")
-	proto.RegisterType((*Member)(nil), "etcdserverpb.Member")
-	proto.RegisterType((*MemberAddRequest)(nil), "etcdserverpb.MemberAddRequest")
-	proto.RegisterType((*MemberAddResponse)(nil), "etcdserverpb.MemberAddResponse")
-	proto.RegisterType((*MemberRemoveRequest)(nil), "etcdserverpb.MemberRemoveRequest")
-	proto.RegisterType((*MemberRemoveResponse)(nil), "etcdserverpb.MemberRemoveResponse")
-	proto.RegisterType((*MemberUpdateRequest)(nil), "etcdserverpb.MemberUpdateRequest")
-	proto.RegisterType((*MemberUpdateResponse)(nil), "etcdserverpb.MemberUpdateResponse")
-	proto.RegisterType((*MemberListRequest)(nil), "etcdserverpb.MemberListRequest")
-	proto.RegisterType((*MemberListResponse)(nil), "etcdserverpb.MemberListResponse")
-	proto.RegisterType((*MemberPromoteRequest)(nil), "etcdserverpb.MemberPromoteRequest")
-	proto.RegisterType((*MemberPromoteResponse)(nil), "etcdserverpb.MemberPromoteResponse")
-	proto.RegisterType((*DefragmentRequest)(nil), "etcdserverpb.DefragmentRequest")
-	proto.RegisterType((*DefragmentResponse)(nil), "etcdserverpb.DefragmentResponse")
-	proto.RegisterType((*MoveLeaderRequest)(nil), "etcdserverpb.MoveLeaderRequest")
-	proto.RegisterType((*MoveLeaderResponse)(nil), "etcdserverpb.MoveLeaderResponse")
-	proto.RegisterType((*AlarmRequest)(nil), "etcdserverpb.AlarmRequest")
-	proto.RegisterType((*AlarmMember)(nil), "etcdserverpb.AlarmMember")
-	proto.RegisterType((*AlarmResponse)(nil), "etcdserverpb.AlarmResponse")
-	proto.RegisterType((*StatusRequest)(nil), "etcdserverpb.StatusRequest")
-	proto.RegisterType((*StatusResponse)(nil), "etcdserverpb.StatusResponse")
-	proto.RegisterType((*AuthEnableRequest)(nil), "etcdserverpb.AuthEnableRequest")
-	proto.RegisterType((*AuthDisableRequest)(nil), "etcdserverpb.AuthDisableRequest")
-	proto.RegisterType((*AuthenticateRequest)(nil), "etcdserverpb.AuthenticateRequest")
-	proto.RegisterType((*AuthUserAddRequest)(nil), "etcdserverpb.AuthUserAddRequest")
-	proto.RegisterType((*AuthUserGetRequest)(nil), "etcdserverpb.AuthUserGetRequest")
-	proto.RegisterType((*AuthUserDeleteRequest)(nil), "etcdserverpb.AuthUserDeleteRequest")
-	proto.RegisterType((*AuthUserChangePasswordRequest)(nil), "etcdserverpb.AuthUserChangePasswordRequest")
-	proto.RegisterType((*AuthUserGrantRoleRequest)(nil), "etcdserverpb.AuthUserGrantRoleRequest")
-	proto.RegisterType((*AuthUserRevokeRoleRequest)(nil), "etcdserverpb.AuthUserRevokeRoleRequest")
-	proto.RegisterType((*AuthRoleAddRequest)(nil), "etcdserverpb.AuthRoleAddRequest")
-	proto.RegisterType((*AuthRoleGetRequest)(nil), "etcdserverpb.AuthRoleGetRequest")
-	proto.RegisterType((*AuthUserListRequest)(nil), "etcdserverpb.AuthUserListRequest")
-	proto.RegisterType((*AuthRoleListRequest)(nil), "etcdserverpb.AuthRoleListRequest")
-	proto.RegisterType((*AuthRoleDeleteRequest)(nil), "etcdserverpb.AuthRoleDeleteRequest")
-	proto.RegisterType((*AuthRoleGrantPermissionRequest)(nil), "etcdserverpb.AuthRoleGrantPermissionRequest")
-	proto.RegisterType((*AuthRoleRevokePermissionRequest)(nil), "etcdserverpb.AuthRoleRevokePermissionRequest")
-	proto.RegisterType((*AuthEnableResponse)(nil), "etcdserverpb.AuthEnableResponse")
-	proto.RegisterType((*AuthDisableResponse)(nil), "etcdserverpb.AuthDisableResponse")
-	proto.RegisterType((*AuthenticateResponse)(nil), "etcdserverpb.AuthenticateResponse")
-	proto.RegisterType((*AuthUserAddResponse)(nil), "etcdserverpb.AuthUserAddResponse")
-	proto.RegisterType((*AuthUserGetResponse)(nil), "etcdserverpb.AuthUserGetResponse")
-	proto.RegisterType((*AuthUserDeleteResponse)(nil), "etcdserverpb.AuthUserDeleteResponse")
-	proto.RegisterType((*AuthUserChangePasswordResponse)(nil), "etcdserverpb.AuthUserChangePasswordResponse")
-	proto.RegisterType((*AuthUserGrantRoleResponse)(nil), "etcdserverpb.AuthUserGrantRoleResponse")
-	proto.RegisterType((*AuthUserRevokeRoleResponse)(nil), "etcdserverpb.AuthUserRevokeRoleResponse")
-	proto.RegisterType((*AuthRoleAddResponse)(nil), "etcdserverpb.AuthRoleAddResponse")
-	proto.RegisterType((*AuthRoleGetResponse)(nil), "etcdserverpb.AuthRoleGetResponse")
-	proto.RegisterType((*AuthRoleListResponse)(nil), "etcdserverpb.AuthRoleListResponse")
-	proto.RegisterType((*AuthUserListResponse)(nil), "etcdserverpb.AuthUserListResponse")
-	proto.RegisterType((*AuthRoleDeleteResponse)(nil), "etcdserverpb.AuthRoleDeleteResponse")
-	proto.RegisterType((*AuthRoleGrantPermissionResponse)(nil), "etcdserverpb.AuthRoleGrantPermissionResponse")
-	proto.RegisterType((*AuthRoleRevokePermissionResponse)(nil), "etcdserverpb.AuthRoleRevokePermissionResponse")
-	proto.RegisterEnum("etcdserverpb.AlarmType", AlarmType_name, AlarmType_value)
-	proto.RegisterEnum("etcdserverpb.RangeRequest_SortOrder", RangeRequest_SortOrder_name, RangeRequest_SortOrder_value)
-	proto.RegisterEnum("etcdserverpb.RangeRequest_SortTarget", RangeRequest_SortTarget_name, RangeRequest_SortTarget_value)
-	proto.RegisterEnum("etcdserverpb.Compare_CompareResult", Compare_CompareResult_name, Compare_CompareResult_value)
-	proto.RegisterEnum("etcdserverpb.Compare_CompareTarget", Compare_CompareTarget_name, Compare_CompareTarget_value)
-	proto.RegisterEnum("etcdserverpb.WatchCreateRequest_FilterType", WatchCreateRequest_FilterType_name, WatchCreateRequest_FilterType_value)
-	proto.RegisterEnum("etcdserverpb.AlarmRequest_AlarmAction", AlarmRequest_AlarmAction_name, AlarmRequest_AlarmAction_value)
-}
-
-// 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 KV service
-
-type KVClient interface {
-	// Range gets the keys in the range from the key-value store.
-	Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error)
-	// Put puts the given key into the key-value store.
-	// A put request increments the revision of the key-value store
-	// and generates one event in the event history.
-	Put(ctx context.Context, in *PutRequest, opts ...grpc.CallOption) (*PutResponse, error)
-	// DeleteRange deletes the given range from the key-value store.
-	// A delete request increments the revision of the key-value store
-	// and generates a delete event in the event history for every deleted key.
-	DeleteRange(ctx context.Context, in *DeleteRangeRequest, opts ...grpc.CallOption) (*DeleteRangeResponse, error)
-	// Txn processes multiple requests in a single transaction.
-	// A txn request increments the revision of the key-value store
-	// and generates events with the same revision for every completed request.
-	// It is not allowed to modify the same key several times within one txn.
-	Txn(ctx context.Context, in *TxnRequest, opts ...grpc.CallOption) (*TxnResponse, error)
-	// Compact compacts the event history in the etcd key-value store. The key-value
-	// store should be periodically compacted or the event history will continue to grow
-	// indefinitely.
-	Compact(ctx context.Context, in *CompactionRequest, opts ...grpc.CallOption) (*CompactionResponse, error)
-}
-
-type kVClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewKVClient(cc *grpc.ClientConn) KVClient {
-	return &kVClient{cc}
-}
-
-func (c *kVClient) Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error) {
-	out := new(RangeResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.KV/Range", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *kVClient) Put(ctx context.Context, in *PutRequest, opts ...grpc.CallOption) (*PutResponse, error) {
-	out := new(PutResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.KV/Put", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *kVClient) DeleteRange(ctx context.Context, in *DeleteRangeRequest, opts ...grpc.CallOption) (*DeleteRangeResponse, error) {
-	out := new(DeleteRangeResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.KV/DeleteRange", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *kVClient) Txn(ctx context.Context, in *TxnRequest, opts ...grpc.CallOption) (*TxnResponse, error) {
-	out := new(TxnResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.KV/Txn", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *kVClient) Compact(ctx context.Context, in *CompactionRequest, opts ...grpc.CallOption) (*CompactionResponse, error) {
-	out := new(CompactionResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.KV/Compact", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// Server API for KV service
-
-type KVServer interface {
-	// Range gets the keys in the range from the key-value store.
-	Range(context.Context, *RangeRequest) (*RangeResponse, error)
-	// Put puts the given key into the key-value store.
-	// A put request increments the revision of the key-value store
-	// and generates one event in the event history.
-	Put(context.Context, *PutRequest) (*PutResponse, error)
-	// DeleteRange deletes the given range from the key-value store.
-	// A delete request increments the revision of the key-value store
-	// and generates a delete event in the event history for every deleted key.
-	DeleteRange(context.Context, *DeleteRangeRequest) (*DeleteRangeResponse, error)
-	// Txn processes multiple requests in a single transaction.
-	// A txn request increments the revision of the key-value store
-	// and generates events with the same revision for every completed request.
-	// It is not allowed to modify the same key several times within one txn.
-	Txn(context.Context, *TxnRequest) (*TxnResponse, error)
-	// Compact compacts the event history in the etcd key-value store. The key-value
-	// store should be periodically compacted or the event history will continue to grow
-	// indefinitely.
-	Compact(context.Context, *CompactionRequest) (*CompactionResponse, error)
-}
-
-func RegisterKVServer(s *grpc.Server, srv KVServer) {
-	s.RegisterService(&_KV_serviceDesc, srv)
-}
-
-func _KV_Range_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(RangeRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(KVServer).Range(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.KV/Range",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(KVServer).Range(ctx, req.(*RangeRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _KV_Put_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PutRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(KVServer).Put(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.KV/Put",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(KVServer).Put(ctx, req.(*PutRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _KV_DeleteRange_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DeleteRangeRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(KVServer).DeleteRange(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.KV/DeleteRange",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(KVServer).DeleteRange(ctx, req.(*DeleteRangeRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _KV_Txn_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(TxnRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(KVServer).Txn(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.KV/Txn",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(KVServer).Txn(ctx, req.(*TxnRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _KV_Compact_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(CompactionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(KVServer).Compact(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.KV/Compact",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(KVServer).Compact(ctx, req.(*CompactionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _KV_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.KV",
-	HandlerType: (*KVServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "Range",
-			Handler:    _KV_Range_Handler,
-		},
-		{
-			MethodName: "Put",
-			Handler:    _KV_Put_Handler,
-		},
-		{
-			MethodName: "DeleteRange",
-			Handler:    _KV_DeleteRange_Handler,
-		},
-		{
-			MethodName: "Txn",
-			Handler:    _KV_Txn_Handler,
-		},
-		{
-			MethodName: "Compact",
-			Handler:    _KV_Compact_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "rpc.proto",
-}
-
-// Client API for Watch service
-
-type WatchClient interface {
-	// Watch watches for events happening or that have happened. Both input and output
-	// are streams; the input stream is for creating and canceling watchers and the output
-	// stream sends events. One watch RPC can watch on multiple key ranges, streaming events
-	// for several watches at once. The entire event history can be watched starting from the
-	// last compaction revision.
-	Watch(ctx context.Context, opts ...grpc.CallOption) (Watch_WatchClient, error)
-}
-
-type watchClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewWatchClient(cc *grpc.ClientConn) WatchClient {
-	return &watchClient{cc}
-}
-
-func (c *watchClient) Watch(ctx context.Context, opts ...grpc.CallOption) (Watch_WatchClient, error) {
-	stream, err := grpc.NewClientStream(ctx, &_Watch_serviceDesc.Streams[0], c.cc, "/etcdserverpb.Watch/Watch", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &watchWatchClient{stream}
-	return x, nil
-}
-
-type Watch_WatchClient interface {
-	Send(*WatchRequest) error
-	Recv() (*WatchResponse, error)
-	grpc.ClientStream
-}
-
-type watchWatchClient struct {
-	grpc.ClientStream
-}
-
-func (x *watchWatchClient) Send(m *WatchRequest) error {
-	return x.ClientStream.SendMsg(m)
-}
-
-func (x *watchWatchClient) Recv() (*WatchResponse, error) {
-	m := new(WatchResponse)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-// Server API for Watch service
-
-type WatchServer interface {
-	// Watch watches for events happening or that have happened. Both input and output
-	// are streams; the input stream is for creating and canceling watchers and the output
-	// stream sends events. One watch RPC can watch on multiple key ranges, streaming events
-	// for several watches at once. The entire event history can be watched starting from the
-	// last compaction revision.
-	Watch(Watch_WatchServer) error
-}
-
-func RegisterWatchServer(s *grpc.Server, srv WatchServer) {
-	s.RegisterService(&_Watch_serviceDesc, srv)
-}
-
-func _Watch_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
-	return srv.(WatchServer).Watch(&watchWatchServer{stream})
-}
-
-type Watch_WatchServer interface {
-	Send(*WatchResponse) error
-	Recv() (*WatchRequest, error)
-	grpc.ServerStream
-}
-
-type watchWatchServer struct {
-	grpc.ServerStream
-}
-
-func (x *watchWatchServer) Send(m *WatchResponse) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func (x *watchWatchServer) Recv() (*WatchRequest, error) {
-	m := new(WatchRequest)
-	if err := x.ServerStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-var _Watch_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.Watch",
-	HandlerType: (*WatchServer)(nil),
-	Methods:     []grpc.MethodDesc{},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "Watch",
-			Handler:       _Watch_Watch_Handler,
-			ServerStreams: true,
-			ClientStreams: true,
-		},
-	},
-	Metadata: "rpc.proto",
-}
-
-// Client API for Lease service
-
-type LeaseClient interface {
-	// LeaseGrant creates a lease which expires if the server does not receive a keepAlive
-	// within a given time to live period. All keys attached to the lease will be expired and
-	// deleted if the lease expires. Each expired key generates a delete event in the event history.
-	LeaseGrant(ctx context.Context, in *LeaseGrantRequest, opts ...grpc.CallOption) (*LeaseGrantResponse, error)
-	// LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
-	LeaseRevoke(ctx context.Context, in *LeaseRevokeRequest, opts ...grpc.CallOption) (*LeaseRevokeResponse, error)
-	// LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
-	// to the server and streaming keep alive responses from the server to the client.
-	LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (Lease_LeaseKeepAliveClient, error)
-	// LeaseTimeToLive retrieves lease information.
-	LeaseTimeToLive(ctx context.Context, in *LeaseTimeToLiveRequest, opts ...grpc.CallOption) (*LeaseTimeToLiveResponse, error)
-	// LeaseLeases lists all existing leases.
-	LeaseLeases(ctx context.Context, in *LeaseLeasesRequest, opts ...grpc.CallOption) (*LeaseLeasesResponse, error)
-}
-
-type leaseClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewLeaseClient(cc *grpc.ClientConn) LeaseClient {
-	return &leaseClient{cc}
-}
-
-func (c *leaseClient) LeaseGrant(ctx context.Context, in *LeaseGrantRequest, opts ...grpc.CallOption) (*LeaseGrantResponse, error) {
-	out := new(LeaseGrantResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Lease/LeaseGrant", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *leaseClient) LeaseRevoke(ctx context.Context, in *LeaseRevokeRequest, opts ...grpc.CallOption) (*LeaseRevokeResponse, error) {
-	out := new(LeaseRevokeResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Lease/LeaseRevoke", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *leaseClient) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (Lease_LeaseKeepAliveClient, error) {
-	stream, err := grpc.NewClientStream(ctx, &_Lease_serviceDesc.Streams[0], c.cc, "/etcdserverpb.Lease/LeaseKeepAlive", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &leaseLeaseKeepAliveClient{stream}
-	return x, nil
-}
-
-type Lease_LeaseKeepAliveClient interface {
-	Send(*LeaseKeepAliveRequest) error
-	Recv() (*LeaseKeepAliveResponse, error)
-	grpc.ClientStream
-}
-
-type leaseLeaseKeepAliveClient struct {
-	grpc.ClientStream
-}
-
-func (x *leaseLeaseKeepAliveClient) Send(m *LeaseKeepAliveRequest) error {
-	return x.ClientStream.SendMsg(m)
-}
-
-func (x *leaseLeaseKeepAliveClient) Recv() (*LeaseKeepAliveResponse, error) {
-	m := new(LeaseKeepAliveResponse)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *leaseClient) LeaseTimeToLive(ctx context.Context, in *LeaseTimeToLiveRequest, opts ...grpc.CallOption) (*LeaseTimeToLiveResponse, error) {
-	out := new(LeaseTimeToLiveResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Lease/LeaseTimeToLive", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *leaseClient) LeaseLeases(ctx context.Context, in *LeaseLeasesRequest, opts ...grpc.CallOption) (*LeaseLeasesResponse, error) {
-	out := new(LeaseLeasesResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Lease/LeaseLeases", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// Server API for Lease service
-
-type LeaseServer interface {
-	// LeaseGrant creates a lease which expires if the server does not receive a keepAlive
-	// within a given time to live period. All keys attached to the lease will be expired and
-	// deleted if the lease expires. Each expired key generates a delete event in the event history.
-	LeaseGrant(context.Context, *LeaseGrantRequest) (*LeaseGrantResponse, error)
-	// LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
-	LeaseRevoke(context.Context, *LeaseRevokeRequest) (*LeaseRevokeResponse, error)
-	// LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
-	// to the server and streaming keep alive responses from the server to the client.
-	LeaseKeepAlive(Lease_LeaseKeepAliveServer) error
-	// LeaseTimeToLive retrieves lease information.
-	LeaseTimeToLive(context.Context, *LeaseTimeToLiveRequest) (*LeaseTimeToLiveResponse, error)
-	// LeaseLeases lists all existing leases.
-	LeaseLeases(context.Context, *LeaseLeasesRequest) (*LeaseLeasesResponse, error)
-}
-
-func RegisterLeaseServer(s *grpc.Server, srv LeaseServer) {
-	s.RegisterService(&_Lease_serviceDesc, srv)
-}
-
-func _Lease_LeaseGrant_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LeaseGrantRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(LeaseServer).LeaseGrant(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Lease/LeaseGrant",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(LeaseServer).LeaseGrant(ctx, req.(*LeaseGrantRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Lease_LeaseRevoke_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LeaseRevokeRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(LeaseServer).LeaseRevoke(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Lease/LeaseRevoke",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(LeaseServer).LeaseRevoke(ctx, req.(*LeaseRevokeRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Lease_LeaseKeepAlive_Handler(srv interface{}, stream grpc.ServerStream) error {
-	return srv.(LeaseServer).LeaseKeepAlive(&leaseLeaseKeepAliveServer{stream})
-}
-
-type Lease_LeaseKeepAliveServer interface {
-	Send(*LeaseKeepAliveResponse) error
-	Recv() (*LeaseKeepAliveRequest, error)
-	grpc.ServerStream
-}
-
-type leaseLeaseKeepAliveServer struct {
-	grpc.ServerStream
-}
-
-func (x *leaseLeaseKeepAliveServer) Send(m *LeaseKeepAliveResponse) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func (x *leaseLeaseKeepAliveServer) Recv() (*LeaseKeepAliveRequest, error) {
-	m := new(LeaseKeepAliveRequest)
-	if err := x.ServerStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func _Lease_LeaseTimeToLive_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LeaseTimeToLiveRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(LeaseServer).LeaseTimeToLive(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Lease/LeaseTimeToLive",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(LeaseServer).LeaseTimeToLive(ctx, req.(*LeaseTimeToLiveRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Lease_LeaseLeases_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LeaseLeasesRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(LeaseServer).LeaseLeases(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Lease/LeaseLeases",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(LeaseServer).LeaseLeases(ctx, req.(*LeaseLeasesRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Lease_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.Lease",
-	HandlerType: (*LeaseServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "LeaseGrant",
-			Handler:    _Lease_LeaseGrant_Handler,
-		},
-		{
-			MethodName: "LeaseRevoke",
-			Handler:    _Lease_LeaseRevoke_Handler,
-		},
-		{
-			MethodName: "LeaseTimeToLive",
-			Handler:    _Lease_LeaseTimeToLive_Handler,
-		},
-		{
-			MethodName: "LeaseLeases",
-			Handler:    _Lease_LeaseLeases_Handler,
-		},
-	},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "LeaseKeepAlive",
-			Handler:       _Lease_LeaseKeepAlive_Handler,
-			ServerStreams: true,
-			ClientStreams: true,
-		},
-	},
-	Metadata: "rpc.proto",
-}
-
-// Client API for Cluster service
-
-type ClusterClient interface {
-	// MemberAdd adds a member into the cluster.
-	MemberAdd(ctx context.Context, in *MemberAddRequest, opts ...grpc.CallOption) (*MemberAddResponse, error)
-	// MemberRemove removes an existing member from the cluster.
-	MemberRemove(ctx context.Context, in *MemberRemoveRequest, opts ...grpc.CallOption) (*MemberRemoveResponse, error)
-	// MemberUpdate updates the member configuration.
-	MemberUpdate(ctx context.Context, in *MemberUpdateRequest, opts ...grpc.CallOption) (*MemberUpdateResponse, error)
-	// MemberList lists all the members in the cluster.
-	MemberList(ctx context.Context, in *MemberListRequest, opts ...grpc.CallOption) (*MemberListResponse, error)
-	// MemberPromote promotes a member from raft learner (non-voting) to raft voting member.
-	MemberPromote(ctx context.Context, in *MemberPromoteRequest, opts ...grpc.CallOption) (*MemberPromoteResponse, error)
-}
-
-type clusterClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewClusterClient(cc *grpc.ClientConn) ClusterClient {
-	return &clusterClient{cc}
-}
-
-func (c *clusterClient) MemberAdd(ctx context.Context, in *MemberAddRequest, opts ...grpc.CallOption) (*MemberAddResponse, error) {
-	out := new(MemberAddResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Cluster/MemberAdd", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *clusterClient) MemberRemove(ctx context.Context, in *MemberRemoveRequest, opts ...grpc.CallOption) (*MemberRemoveResponse, error) {
-	out := new(MemberRemoveResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Cluster/MemberRemove", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *clusterClient) MemberUpdate(ctx context.Context, in *MemberUpdateRequest, opts ...grpc.CallOption) (*MemberUpdateResponse, error) {
-	out := new(MemberUpdateResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Cluster/MemberUpdate", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *clusterClient) MemberList(ctx context.Context, in *MemberListRequest, opts ...grpc.CallOption) (*MemberListResponse, error) {
-	out := new(MemberListResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Cluster/MemberList", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *clusterClient) MemberPromote(ctx context.Context, in *MemberPromoteRequest, opts ...grpc.CallOption) (*MemberPromoteResponse, error) {
-	out := new(MemberPromoteResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Cluster/MemberPromote", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// Server API for Cluster service
-
-type ClusterServer interface {
-	// MemberAdd adds a member into the cluster.
-	MemberAdd(context.Context, *MemberAddRequest) (*MemberAddResponse, error)
-	// MemberRemove removes an existing member from the cluster.
-	MemberRemove(context.Context, *MemberRemoveRequest) (*MemberRemoveResponse, error)
-	// MemberUpdate updates the member configuration.
-	MemberUpdate(context.Context, *MemberUpdateRequest) (*MemberUpdateResponse, error)
-	// MemberList lists all the members in the cluster.
-	MemberList(context.Context, *MemberListRequest) (*MemberListResponse, error)
-	// MemberPromote promotes a member from raft learner (non-voting) to raft voting member.
-	MemberPromote(context.Context, *MemberPromoteRequest) (*MemberPromoteResponse, error)
-}
-
-func RegisterClusterServer(s *grpc.Server, srv ClusterServer) {
-	s.RegisterService(&_Cluster_serviceDesc, srv)
-}
-
-func _Cluster_MemberAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MemberAddRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ClusterServer).MemberAdd(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Cluster/MemberAdd",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ClusterServer).MemberAdd(ctx, req.(*MemberAddRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Cluster_MemberRemove_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MemberRemoveRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ClusterServer).MemberRemove(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Cluster/MemberRemove",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ClusterServer).MemberRemove(ctx, req.(*MemberRemoveRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Cluster_MemberUpdate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MemberUpdateRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ClusterServer).MemberUpdate(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Cluster/MemberUpdate",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ClusterServer).MemberUpdate(ctx, req.(*MemberUpdateRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Cluster_MemberList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MemberListRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ClusterServer).MemberList(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Cluster/MemberList",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ClusterServer).MemberList(ctx, req.(*MemberListRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Cluster_MemberPromote_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MemberPromoteRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(ClusterServer).MemberPromote(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Cluster/MemberPromote",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(ClusterServer).MemberPromote(ctx, req.(*MemberPromoteRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Cluster_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.Cluster",
-	HandlerType: (*ClusterServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "MemberAdd",
-			Handler:    _Cluster_MemberAdd_Handler,
-		},
-		{
-			MethodName: "MemberRemove",
-			Handler:    _Cluster_MemberRemove_Handler,
-		},
-		{
-			MethodName: "MemberUpdate",
-			Handler:    _Cluster_MemberUpdate_Handler,
-		},
-		{
-			MethodName: "MemberList",
-			Handler:    _Cluster_MemberList_Handler,
-		},
-		{
-			MethodName: "MemberPromote",
-			Handler:    _Cluster_MemberPromote_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "rpc.proto",
-}
-
-// Client API for Maintenance service
-
-type MaintenanceClient interface {
-	// Alarm activates, deactivates, and queries alarms regarding cluster health.
-	Alarm(ctx context.Context, in *AlarmRequest, opts ...grpc.CallOption) (*AlarmResponse, error)
-	// Status gets the status of the member.
-	Status(ctx context.Context, in *StatusRequest, opts ...grpc.CallOption) (*StatusResponse, error)
-	// Defragment defragments a member's backend database to recover storage space.
-	Defragment(ctx context.Context, in *DefragmentRequest, opts ...grpc.CallOption) (*DefragmentResponse, error)
-	// Hash computes the hash of whole backend keyspace,
-	// including key, lease, and other buckets in storage.
-	// This is designed for testing ONLY!
-	// Do not rely on this in production with ongoing transactions,
-	// since Hash operation does not hold MVCC locks.
-	// Use "HashKV" API instead for "key" bucket consistency checks.
-	Hash(ctx context.Context, in *HashRequest, opts ...grpc.CallOption) (*HashResponse, error)
-	// HashKV computes the hash of all MVCC keys up to a given revision.
-	// It only iterates "key" bucket in backend storage.
-	HashKV(ctx context.Context, in *HashKVRequest, opts ...grpc.CallOption) (*HashKVResponse, error)
-	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
-	Snapshot(ctx context.Context, in *SnapshotRequest, opts ...grpc.CallOption) (Maintenance_SnapshotClient, error)
-	// MoveLeader requests current leader node to transfer its leadership to transferee.
-	MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error)
-}
-
-type maintenanceClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewMaintenanceClient(cc *grpc.ClientConn) MaintenanceClient {
-	return &maintenanceClient{cc}
-}
-
-func (c *maintenanceClient) Alarm(ctx context.Context, in *AlarmRequest, opts ...grpc.CallOption) (*AlarmResponse, error) {
-	out := new(AlarmResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/Alarm", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *maintenanceClient) Status(ctx context.Context, in *StatusRequest, opts ...grpc.CallOption) (*StatusResponse, error) {
-	out := new(StatusResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/Status", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *maintenanceClient) Defragment(ctx context.Context, in *DefragmentRequest, opts ...grpc.CallOption) (*DefragmentResponse, error) {
-	out := new(DefragmentResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/Defragment", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *maintenanceClient) Hash(ctx context.Context, in *HashRequest, opts ...grpc.CallOption) (*HashResponse, error) {
-	out := new(HashResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/Hash", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *maintenanceClient) HashKV(ctx context.Context, in *HashKVRequest, opts ...grpc.CallOption) (*HashKVResponse, error) {
-	out := new(HashKVResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/HashKV", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *maintenanceClient) Snapshot(ctx context.Context, in *SnapshotRequest, opts ...grpc.CallOption) (Maintenance_SnapshotClient, error) {
-	stream, err := grpc.NewClientStream(ctx, &_Maintenance_serviceDesc.Streams[0], c.cc, "/etcdserverpb.Maintenance/Snapshot", opts...)
-	if err != nil {
-		return nil, err
-	}
-	x := &maintenanceSnapshotClient{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 Maintenance_SnapshotClient interface {
-	Recv() (*SnapshotResponse, error)
-	grpc.ClientStream
-}
-
-type maintenanceSnapshotClient struct {
-	grpc.ClientStream
-}
-
-func (x *maintenanceSnapshotClient) Recv() (*SnapshotResponse, error) {
-	m := new(SnapshotResponse)
-	if err := x.ClientStream.RecvMsg(m); err != nil {
-		return nil, err
-	}
-	return m, nil
-}
-
-func (c *maintenanceClient) MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error) {
-	out := new(MoveLeaderResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/MoveLeader", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// Server API for Maintenance service
-
-type MaintenanceServer interface {
-	// Alarm activates, deactivates, and queries alarms regarding cluster health.
-	Alarm(context.Context, *AlarmRequest) (*AlarmResponse, error)
-	// Status gets the status of the member.
-	Status(context.Context, *StatusRequest) (*StatusResponse, error)
-	// Defragment defragments a member's backend database to recover storage space.
-	Defragment(context.Context, *DefragmentRequest) (*DefragmentResponse, error)
-	// Hash computes the hash of whole backend keyspace,
-	// including key, lease, and other buckets in storage.
-	// This is designed for testing ONLY!
-	// Do not rely on this in production with ongoing transactions,
-	// since Hash operation does not hold MVCC locks.
-	// Use "HashKV" API instead for "key" bucket consistency checks.
-	Hash(context.Context, *HashRequest) (*HashResponse, error)
-	// HashKV computes the hash of all MVCC keys up to a given revision.
-	// It only iterates "key" bucket in backend storage.
-	HashKV(context.Context, *HashKVRequest) (*HashKVResponse, error)
-	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
-	Snapshot(*SnapshotRequest, Maintenance_SnapshotServer) error
-	// MoveLeader requests current leader node to transfer its leadership to transferee.
-	MoveLeader(context.Context, *MoveLeaderRequest) (*MoveLeaderResponse, error)
-}
-
-func RegisterMaintenanceServer(s *grpc.Server, srv MaintenanceServer) {
-	s.RegisterService(&_Maintenance_serviceDesc, srv)
-}
-
-func _Maintenance_Alarm_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AlarmRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).Alarm(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/Alarm",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).Alarm(ctx, req.(*AlarmRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Maintenance_Status_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(StatusRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).Status(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/Status",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).Status(ctx, req.(*StatusRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Maintenance_Defragment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DefragmentRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).Defragment(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/Defragment",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).Defragment(ctx, req.(*DefragmentRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Maintenance_Hash_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HashRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).Hash(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/Hash",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).Hash(ctx, req.(*HashRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Maintenance_HashKV_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HashKVRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).HashKV(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/HashKV",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).HashKV(ctx, req.(*HashKVRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Maintenance_Snapshot_Handler(srv interface{}, stream grpc.ServerStream) error {
-	m := new(SnapshotRequest)
-	if err := stream.RecvMsg(m); err != nil {
-		return err
-	}
-	return srv.(MaintenanceServer).Snapshot(m, &maintenanceSnapshotServer{stream})
-}
-
-type Maintenance_SnapshotServer interface {
-	Send(*SnapshotResponse) error
-	grpc.ServerStream
-}
-
-type maintenanceSnapshotServer struct {
-	grpc.ServerStream
-}
-
-func (x *maintenanceSnapshotServer) Send(m *SnapshotResponse) error {
-	return x.ServerStream.SendMsg(m)
-}
-
-func _Maintenance_MoveLeader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(MoveLeaderRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MaintenanceServer).MoveLeader(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Maintenance/MoveLeader",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MaintenanceServer).MoveLeader(ctx, req.(*MoveLeaderRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Maintenance_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.Maintenance",
-	HandlerType: (*MaintenanceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "Alarm",
-			Handler:    _Maintenance_Alarm_Handler,
-		},
-		{
-			MethodName: "Status",
-			Handler:    _Maintenance_Status_Handler,
-		},
-		{
-			MethodName: "Defragment",
-			Handler:    _Maintenance_Defragment_Handler,
-		},
-		{
-			MethodName: "Hash",
-			Handler:    _Maintenance_Hash_Handler,
-		},
-		{
-			MethodName: "HashKV",
-			Handler:    _Maintenance_HashKV_Handler,
-		},
-		{
-			MethodName: "MoveLeader",
-			Handler:    _Maintenance_MoveLeader_Handler,
-		},
-	},
-	Streams: []grpc.StreamDesc{
-		{
-			StreamName:    "Snapshot",
-			Handler:       _Maintenance_Snapshot_Handler,
-			ServerStreams: true,
-		},
-	},
-	Metadata: "rpc.proto",
-}
-
-// Client API for Auth service
-
-type AuthClient interface {
-	// AuthEnable enables authentication.
-	AuthEnable(ctx context.Context, in *AuthEnableRequest, opts ...grpc.CallOption) (*AuthEnableResponse, error)
-	// AuthDisable disables authentication.
-	AuthDisable(ctx context.Context, in *AuthDisableRequest, opts ...grpc.CallOption) (*AuthDisableResponse, error)
-	// Authenticate processes an authenticate request.
-	Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error)
-	// UserAdd adds a new user. User name cannot be empty.
-	UserAdd(ctx context.Context, in *AuthUserAddRequest, opts ...grpc.CallOption) (*AuthUserAddResponse, error)
-	// UserGet gets detailed user information.
-	UserGet(ctx context.Context, in *AuthUserGetRequest, opts ...grpc.CallOption) (*AuthUserGetResponse, error)
-	// UserList gets a list of all users.
-	UserList(ctx context.Context, in *AuthUserListRequest, opts ...grpc.CallOption) (*AuthUserListResponse, error)
-	// UserDelete deletes a specified user.
-	UserDelete(ctx context.Context, in *AuthUserDeleteRequest, opts ...grpc.CallOption) (*AuthUserDeleteResponse, error)
-	// UserChangePassword changes the password of a specified user.
-	UserChangePassword(ctx context.Context, in *AuthUserChangePasswordRequest, opts ...grpc.CallOption) (*AuthUserChangePasswordResponse, error)
-	// UserGrant grants a role to a specified user.
-	UserGrantRole(ctx context.Context, in *AuthUserGrantRoleRequest, opts ...grpc.CallOption) (*AuthUserGrantRoleResponse, error)
-	// UserRevokeRole revokes a role of specified user.
-	UserRevokeRole(ctx context.Context, in *AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (*AuthUserRevokeRoleResponse, error)
-	// RoleAdd adds a new role. Role name cannot be empty.
-	RoleAdd(ctx context.Context, in *AuthRoleAddRequest, opts ...grpc.CallOption) (*AuthRoleAddResponse, error)
-	// RoleGet gets detailed role information.
-	RoleGet(ctx context.Context, in *AuthRoleGetRequest, opts ...grpc.CallOption) (*AuthRoleGetResponse, error)
-	// RoleList gets lists of all roles.
-	RoleList(ctx context.Context, in *AuthRoleListRequest, opts ...grpc.CallOption) (*AuthRoleListResponse, error)
-	// RoleDelete deletes a specified role.
-	RoleDelete(ctx context.Context, in *AuthRoleDeleteRequest, opts ...grpc.CallOption) (*AuthRoleDeleteResponse, error)
-	// RoleGrantPermission grants a permission of a specified key or range to a specified role.
-	RoleGrantPermission(ctx context.Context, in *AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (*AuthRoleGrantPermissionResponse, error)
-	// RoleRevokePermission revokes a key or range permission of a specified role.
-	RoleRevokePermission(ctx context.Context, in *AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (*AuthRoleRevokePermissionResponse, error)
-}
-
-type authClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewAuthClient(cc *grpc.ClientConn) AuthClient {
-	return &authClient{cc}
-}
-
-func (c *authClient) AuthEnable(ctx context.Context, in *AuthEnableRequest, opts ...grpc.CallOption) (*AuthEnableResponse, error) {
-	out := new(AuthEnableResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/AuthEnable", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) AuthDisable(ctx context.Context, in *AuthDisableRequest, opts ...grpc.CallOption) (*AuthDisableResponse, error) {
-	out := new(AuthDisableResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/AuthDisable", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) Authenticate(ctx context.Context, in *AuthenticateRequest, opts ...grpc.CallOption) (*AuthenticateResponse, error) {
-	out := new(AuthenticateResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/Authenticate", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserAdd(ctx context.Context, in *AuthUserAddRequest, opts ...grpc.CallOption) (*AuthUserAddResponse, error) {
-	out := new(AuthUserAddResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserAdd", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserGet(ctx context.Context, in *AuthUserGetRequest, opts ...grpc.CallOption) (*AuthUserGetResponse, error) {
-	out := new(AuthUserGetResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserGet", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserList(ctx context.Context, in *AuthUserListRequest, opts ...grpc.CallOption) (*AuthUserListResponse, error) {
-	out := new(AuthUserListResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserList", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserDelete(ctx context.Context, in *AuthUserDeleteRequest, opts ...grpc.CallOption) (*AuthUserDeleteResponse, error) {
-	out := new(AuthUserDeleteResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserDelete", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserChangePassword(ctx context.Context, in *AuthUserChangePasswordRequest, opts ...grpc.CallOption) (*AuthUserChangePasswordResponse, error) {
-	out := new(AuthUserChangePasswordResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserChangePassword", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserGrantRole(ctx context.Context, in *AuthUserGrantRoleRequest, opts ...grpc.CallOption) (*AuthUserGrantRoleResponse, error) {
-	out := new(AuthUserGrantRoleResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserGrantRole", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) UserRevokeRole(ctx context.Context, in *AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (*AuthUserRevokeRoleResponse, error) {
-	out := new(AuthUserRevokeRoleResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/UserRevokeRole", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleAdd(ctx context.Context, in *AuthRoleAddRequest, opts ...grpc.CallOption) (*AuthRoleAddResponse, error) {
-	out := new(AuthRoleAddResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleAdd", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleGet(ctx context.Context, in *AuthRoleGetRequest, opts ...grpc.CallOption) (*AuthRoleGetResponse, error) {
-	out := new(AuthRoleGetResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleGet", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleList(ctx context.Context, in *AuthRoleListRequest, opts ...grpc.CallOption) (*AuthRoleListResponse, error) {
-	out := new(AuthRoleListResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleList", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleDelete(ctx context.Context, in *AuthRoleDeleteRequest, opts ...grpc.CallOption) (*AuthRoleDeleteResponse, error) {
-	out := new(AuthRoleDeleteResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleDelete", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleGrantPermission(ctx context.Context, in *AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (*AuthRoleGrantPermissionResponse, error) {
-	out := new(AuthRoleGrantPermissionResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleGrantPermission", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *authClient) RoleRevokePermission(ctx context.Context, in *AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (*AuthRoleRevokePermissionResponse, error) {
-	out := new(AuthRoleRevokePermissionResponse)
-	err := grpc.Invoke(ctx, "/etcdserverpb.Auth/RoleRevokePermission", in, out, c.cc, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// Server API for Auth service
-
-type AuthServer interface {
-	// AuthEnable enables authentication.
-	AuthEnable(context.Context, *AuthEnableRequest) (*AuthEnableResponse, error)
-	// AuthDisable disables authentication.
-	AuthDisable(context.Context, *AuthDisableRequest) (*AuthDisableResponse, error)
-	// Authenticate processes an authenticate request.
-	Authenticate(context.Context, *AuthenticateRequest) (*AuthenticateResponse, error)
-	// UserAdd adds a new user. User name cannot be empty.
-	UserAdd(context.Context, *AuthUserAddRequest) (*AuthUserAddResponse, error)
-	// UserGet gets detailed user information.
-	UserGet(context.Context, *AuthUserGetRequest) (*AuthUserGetResponse, error)
-	// UserList gets a list of all users.
-	UserList(context.Context, *AuthUserListRequest) (*AuthUserListResponse, error)
-	// UserDelete deletes a specified user.
-	UserDelete(context.Context, *AuthUserDeleteRequest) (*AuthUserDeleteResponse, error)
-	// UserChangePassword changes the password of a specified user.
-	UserChangePassword(context.Context, *AuthUserChangePasswordRequest) (*AuthUserChangePasswordResponse, error)
-	// UserGrant grants a role to a specified user.
-	UserGrantRole(context.Context, *AuthUserGrantRoleRequest) (*AuthUserGrantRoleResponse, error)
-	// UserRevokeRole revokes a role of specified user.
-	UserRevokeRole(context.Context, *AuthUserRevokeRoleRequest) (*AuthUserRevokeRoleResponse, error)
-	// RoleAdd adds a new role. Role name cannot be empty.
-	RoleAdd(context.Context, *AuthRoleAddRequest) (*AuthRoleAddResponse, error)
-	// RoleGet gets detailed role information.
-	RoleGet(context.Context, *AuthRoleGetRequest) (*AuthRoleGetResponse, error)
-	// RoleList gets lists of all roles.
-	RoleList(context.Context, *AuthRoleListRequest) (*AuthRoleListResponse, error)
-	// RoleDelete deletes a specified role.
-	RoleDelete(context.Context, *AuthRoleDeleteRequest) (*AuthRoleDeleteResponse, error)
-	// RoleGrantPermission grants a permission of a specified key or range to a specified role.
-	RoleGrantPermission(context.Context, *AuthRoleGrantPermissionRequest) (*AuthRoleGrantPermissionResponse, error)
-	// RoleRevokePermission revokes a key or range permission of a specified role.
-	RoleRevokePermission(context.Context, *AuthRoleRevokePermissionRequest) (*AuthRoleRevokePermissionResponse, error)
-}
-
-func RegisterAuthServer(s *grpc.Server, srv AuthServer) {
-	s.RegisterService(&_Auth_serviceDesc, srv)
-}
-
-func _Auth_AuthEnable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthEnableRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).AuthEnable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/AuthEnable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).AuthEnable(ctx, req.(*AuthEnableRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_AuthDisable_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthDisableRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).AuthDisable(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/AuthDisable",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).AuthDisable(ctx, req.(*AuthDisableRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_Authenticate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthenticateRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).Authenticate(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/Authenticate",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).Authenticate(ctx, req.(*AuthenticateRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserAddRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserAdd(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserAdd",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserAdd(ctx, req.(*AuthUserAddRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserGetRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserGet(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserGet",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserGet(ctx, req.(*AuthUserGetRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserListRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserList(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserList",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserList(ctx, req.(*AuthUserListRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserDeleteRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserDelete(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserDelete",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserDelete(ctx, req.(*AuthUserDeleteRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserChangePassword_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserChangePasswordRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserChangePassword(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserChangePassword",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserChangePassword(ctx, req.(*AuthUserChangePasswordRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserGrantRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserGrantRoleRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserGrantRole(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserGrantRole",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserGrantRole(ctx, req.(*AuthUserGrantRoleRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_UserRevokeRole_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthUserRevokeRoleRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).UserRevokeRole(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/UserRevokeRole",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).UserRevokeRole(ctx, req.(*AuthUserRevokeRoleRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleAdd_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleAddRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleAdd(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleAdd",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleAdd(ctx, req.(*AuthRoleAddRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleGetRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleGet(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleGet",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleGet(ctx, req.(*AuthRoleGetRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleList_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleListRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleList(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleList",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleList(ctx, req.(*AuthRoleListRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleDeleteRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleDelete(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleDelete",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleDelete(ctx, req.(*AuthRoleDeleteRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleGrantPermission_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleGrantPermissionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleGrantPermission(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleGrantPermission",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleGrantPermission(ctx, req.(*AuthRoleGrantPermissionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _Auth_RoleRevokePermission_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AuthRoleRevokePermissionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AuthServer).RoleRevokePermission(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/etcdserverpb.Auth/RoleRevokePermission",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AuthServer).RoleRevokePermission(ctx, req.(*AuthRoleRevokePermissionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Auth_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "etcdserverpb.Auth",
-	HandlerType: (*AuthServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "AuthEnable",
-			Handler:    _Auth_AuthEnable_Handler,
-		},
-		{
-			MethodName: "AuthDisable",
-			Handler:    _Auth_AuthDisable_Handler,
-		},
-		{
-			MethodName: "Authenticate",
-			Handler:    _Auth_Authenticate_Handler,
-		},
-		{
-			MethodName: "UserAdd",
-			Handler:    _Auth_UserAdd_Handler,
-		},
-		{
-			MethodName: "UserGet",
-			Handler:    _Auth_UserGet_Handler,
-		},
-		{
-			MethodName: "UserList",
-			Handler:    _Auth_UserList_Handler,
-		},
-		{
-			MethodName: "UserDelete",
-			Handler:    _Auth_UserDelete_Handler,
-		},
-		{
-			MethodName: "UserChangePassword",
-			Handler:    _Auth_UserChangePassword_Handler,
-		},
-		{
-			MethodName: "UserGrantRole",
-			Handler:    _Auth_UserGrantRole_Handler,
-		},
-		{
-			MethodName: "UserRevokeRole",
-			Handler:    _Auth_UserRevokeRole_Handler,
-		},
-		{
-			MethodName: "RoleAdd",
-			Handler:    _Auth_RoleAdd_Handler,
-		},
-		{
-			MethodName: "RoleGet",
-			Handler:    _Auth_RoleGet_Handler,
-		},
-		{
-			MethodName: "RoleList",
-			Handler:    _Auth_RoleList_Handler,
-		},
-		{
-			MethodName: "RoleDelete",
-			Handler:    _Auth_RoleDelete_Handler,
-		},
-		{
-			MethodName: "RoleGrantPermission",
-			Handler:    _Auth_RoleGrantPermission_Handler,
-		},
-		{
-			MethodName: "RoleRevokePermission",
-			Handler:    _Auth_RoleRevokePermission_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "rpc.proto",
-}
-
-func (m *ResponseHeader) 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 *ResponseHeader) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ClusterId != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ClusterId))
-	}
-	if m.MemberId != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MemberId))
-	}
-	if m.Revision != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
-	}
-	if m.RaftTerm != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RaftTerm))
-	}
-	return i, nil
-}
-
-func (m *RangeRequest) 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 *RangeRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Key) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	if m.Limit != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Limit))
-	}
-	if m.Revision != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
-	}
-	if m.SortOrder != 0 {
-		dAtA[i] = 0x28
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.SortOrder))
-	}
-	if m.SortTarget != 0 {
-		dAtA[i] = 0x30
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.SortTarget))
-	}
-	if m.Serializable {
-		dAtA[i] = 0x38
-		i++
-		if m.Serializable {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.KeysOnly {
-		dAtA[i] = 0x40
-		i++
-		if m.KeysOnly {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.CountOnly {
-		dAtA[i] = 0x48
-		i++
-		if m.CountOnly {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.MinModRevision != 0 {
-		dAtA[i] = 0x50
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MinModRevision))
-	}
-	if m.MaxModRevision != 0 {
-		dAtA[i] = 0x58
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MaxModRevision))
-	}
-	if m.MinCreateRevision != 0 {
-		dAtA[i] = 0x60
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MinCreateRevision))
-	}
-	if m.MaxCreateRevision != 0 {
-		dAtA[i] = 0x68
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MaxCreateRevision))
-	}
-	return i, nil
-}
-
-func (m *RangeResponse) 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 *RangeResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n1, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n1
-	}
-	if len(m.Kvs) > 0 {
-		for _, msg := range m.Kvs {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	if m.More {
-		dAtA[i] = 0x18
-		i++
-		if m.More {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.Count != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Count))
-	}
-	return i, nil
-}
-
-func (m *PutRequest) 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 *PutRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Key) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.Value) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Value)))
-		i += copy(dAtA[i:], m.Value)
-	}
-	if m.Lease != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Lease))
-	}
-	if m.PrevKv {
-		dAtA[i] = 0x20
-		i++
-		if m.PrevKv {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.IgnoreValue {
-		dAtA[i] = 0x28
-		i++
-		if m.IgnoreValue {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.IgnoreLease {
-		dAtA[i] = 0x30
-		i++
-		if m.IgnoreLease {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *PutResponse) 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 *PutResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n2, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n2
-	}
-	if m.PrevKv != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.PrevKv.Size()))
-		n3, err := m.PrevKv.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n3
-	}
-	return i, nil
-}
-
-func (m *DeleteRangeRequest) 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 *DeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Key) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	if m.PrevKv {
-		dAtA[i] = 0x18
-		i++
-		if m.PrevKv {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *DeleteRangeResponse) 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 *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n4, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n4
-	}
-	if m.Deleted != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Deleted))
-	}
-	if len(m.PrevKvs) > 0 {
-		for _, msg := range m.PrevKvs {
-			dAtA[i] = 0x1a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *RequestOp) 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 *RequestOp) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Request != nil {
-		nn5, err := m.Request.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += nn5
-	}
-	return i, nil
-}
-
-func (m *RequestOp_RequestRange) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.RequestRange != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RequestRange.Size()))
-		n6, err := m.RequestRange.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n6
-	}
-	return i, nil
-}
-func (m *RequestOp_RequestPut) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.RequestPut != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RequestPut.Size()))
-		n7, err := m.RequestPut.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n7
-	}
-	return i, nil
-}
-func (m *RequestOp_RequestDeleteRange) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.RequestDeleteRange != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RequestDeleteRange.Size()))
-		n8, err := m.RequestDeleteRange.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n8
-	}
-	return i, nil
-}
-func (m *RequestOp_RequestTxn) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.RequestTxn != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RequestTxn.Size()))
-		n9, err := m.RequestTxn.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n9
-	}
-	return i, nil
-}
-func (m *ResponseOp) 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 *ResponseOp) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Response != nil {
-		nn10, err := m.Response.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += nn10
-	}
-	return i, nil
-}
-
-func (m *ResponseOp_ResponseRange) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.ResponseRange != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ResponseRange.Size()))
-		n11, err := m.ResponseRange.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n11
-	}
-	return i, nil
-}
-func (m *ResponseOp_ResponsePut) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.ResponsePut != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ResponsePut.Size()))
-		n12, err := m.ResponsePut.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n12
-	}
-	return i, nil
-}
-func (m *ResponseOp_ResponseDeleteRange) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.ResponseDeleteRange != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ResponseDeleteRange.Size()))
-		n13, err := m.ResponseDeleteRange.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n13
-	}
-	return i, nil
-}
-func (m *ResponseOp_ResponseTxn) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.ResponseTxn != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ResponseTxn.Size()))
-		n14, err := m.ResponseTxn.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n14
-	}
-	return i, nil
-}
-func (m *Compare) 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 *Compare) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Result != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Result))
-	}
-	if m.Target != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Target))
-	}
-	if len(m.Key) > 0 {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if m.TargetUnion != nil {
-		nn15, err := m.TargetUnion.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += nn15
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x82
-		i++
-		dAtA[i] = 0x4
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	return i, nil
-}
-
-func (m *Compare_Version) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	dAtA[i] = 0x20
-	i++
-	i = encodeVarintRpc(dAtA, i, uint64(m.Version))
-	return i, nil
-}
-func (m *Compare_CreateRevision) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	dAtA[i] = 0x28
-	i++
-	i = encodeVarintRpc(dAtA, i, uint64(m.CreateRevision))
-	return i, nil
-}
-func (m *Compare_ModRevision) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	dAtA[i] = 0x30
-	i++
-	i = encodeVarintRpc(dAtA, i, uint64(m.ModRevision))
-	return i, nil
-}
-func (m *Compare_Value) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.Value != nil {
-		dAtA[i] = 0x3a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Value)))
-		i += copy(dAtA[i:], m.Value)
-	}
-	return i, nil
-}
-func (m *Compare_Lease) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	dAtA[i] = 0x40
-	i++
-	i = encodeVarintRpc(dAtA, i, uint64(m.Lease))
-	return i, nil
-}
-func (m *TxnRequest) 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 *TxnRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Compare) > 0 {
-		for _, msg := range m.Compare {
-			dAtA[i] = 0xa
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	if len(m.Success) > 0 {
-		for _, msg := range m.Success {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	if len(m.Failure) > 0 {
-		for _, msg := range m.Failure {
-			dAtA[i] = 0x1a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *TxnResponse) 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 *TxnResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n16, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n16
-	}
-	if m.Succeeded {
-		dAtA[i] = 0x10
-		i++
-		if m.Succeeded {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if len(m.Responses) > 0 {
-		for _, msg := range m.Responses {
-			dAtA[i] = 0x1a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *CompactionRequest) 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 *CompactionRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Revision != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
-	}
-	if m.Physical {
-		dAtA[i] = 0x10
-		i++
-		if m.Physical {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *CompactionResponse) 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 *CompactionResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n17, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n17
-	}
-	return i, nil
-}
-
-func (m *HashRequest) 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 *HashRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *HashKVRequest) 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 *HashKVRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Revision != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Revision))
-	}
-	return i, nil
-}
-
-func (m *HashKVResponse) 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 *HashKVResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n18, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n18
-	}
-	if m.Hash != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Hash))
-	}
-	if m.CompactRevision != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.CompactRevision))
-	}
-	return i, nil
-}
-
-func (m *HashResponse) 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 *HashResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n19, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n19
-	}
-	if m.Hash != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Hash))
-	}
-	return i, nil
-}
-
-func (m *SnapshotRequest) 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 *SnapshotRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *SnapshotResponse) 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 *SnapshotResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n20, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n20
-	}
-	if m.RemainingBytes != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RemainingBytes))
-	}
-	if len(m.Blob) > 0 {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Blob)))
-		i += copy(dAtA[i:], m.Blob)
-	}
-	return i, nil
-}
-
-func (m *WatchRequest) 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 *WatchRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.RequestUnion != nil {
-		nn21, err := m.RequestUnion.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += nn21
-	}
-	return i, nil
-}
-
-func (m *WatchRequest_CreateRequest) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.CreateRequest != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.CreateRequest.Size()))
-		n22, err := m.CreateRequest.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n22
-	}
-	return i, nil
-}
-func (m *WatchRequest_CancelRequest) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.CancelRequest != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.CancelRequest.Size()))
-		n23, err := m.CancelRequest.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n23
-	}
-	return i, nil
-}
-func (m *WatchRequest_ProgressRequest) MarshalTo(dAtA []byte) (int, error) {
-	i := 0
-	if m.ProgressRequest != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ProgressRequest.Size()))
-		n24, err := m.ProgressRequest.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n24
-	}
-	return i, nil
-}
-func (m *WatchCreateRequest) 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 *WatchCreateRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Key) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	if m.StartRevision != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.StartRevision))
-	}
-	if m.ProgressNotify {
-		dAtA[i] = 0x20
-		i++
-		if m.ProgressNotify {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if len(m.Filters) > 0 {
-		dAtA26 := make([]byte, len(m.Filters)*10)
-		var j25 int
-		for _, num := range m.Filters {
-			for num >= 1<<7 {
-				dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j25++
-			}
-			dAtA26[j25] = uint8(num)
-			j25++
-		}
-		dAtA[i] = 0x2a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(j25))
-		i += copy(dAtA[i:], dAtA26[:j25])
-	}
-	if m.PrevKv {
-		dAtA[i] = 0x30
-		i++
-		if m.PrevKv {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.WatchId != 0 {
-		dAtA[i] = 0x38
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
-	}
-	if m.Fragment {
-		dAtA[i] = 0x40
-		i++
-		if m.Fragment {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *WatchCancelRequest) 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 *WatchCancelRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.WatchId != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
-	}
-	return i, nil
-}
-
-func (m *WatchProgressRequest) 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 *WatchProgressRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *WatchResponse) 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 *WatchResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n27, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n27
-	}
-	if m.WatchId != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.WatchId))
-	}
-	if m.Created {
-		dAtA[i] = 0x18
-		i++
-		if m.Created {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.Canceled {
-		dAtA[i] = 0x20
-		i++
-		if m.Canceled {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if m.CompactRevision != 0 {
-		dAtA[i] = 0x28
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.CompactRevision))
-	}
-	if len(m.CancelReason) > 0 {
-		dAtA[i] = 0x32
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.CancelReason)))
-		i += copy(dAtA[i:], m.CancelReason)
-	}
-	if m.Fragment {
-		dAtA[i] = 0x38
-		i++
-		if m.Fragment {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	if len(m.Events) > 0 {
-		for _, msg := range m.Events {
-			dAtA[i] = 0x5a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *LeaseGrantRequest) 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 *LeaseGrantRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.TTL != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
-	}
-	if m.ID != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *LeaseGrantResponse) 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 *LeaseGrantResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n28, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n28
-	}
-	if m.ID != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
-	}
-	if len(m.Error) > 0 {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Error)))
-		i += copy(dAtA[i:], m.Error)
-	}
-	return i, nil
-}
-
-func (m *LeaseRevokeRequest) 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 *LeaseRevokeRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *LeaseRevokeResponse) 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 *LeaseRevokeResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n29, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n29
-	}
-	return i, nil
-}
-
-func (m *LeaseCheckpoint) 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 *LeaseCheckpoint) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if m.Remaining_TTL != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Remaining_TTL))
-	}
-	return i, nil
-}
-
-func (m *LeaseCheckpointRequest) 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 *LeaseCheckpointRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Checkpoints) > 0 {
-		for _, msg := range m.Checkpoints {
-			dAtA[i] = 0xa
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *LeaseCheckpointResponse) 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 *LeaseCheckpointResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n30, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n30
-	}
-	return i, nil
-}
-
-func (m *LeaseKeepAliveRequest) 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 *LeaseKeepAliveRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *LeaseKeepAliveResponse) 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 *LeaseKeepAliveResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n31, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n31
-	}
-	if m.ID != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
-	}
-	return i, nil
-}
-
-func (m *LeaseTimeToLiveRequest) 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 *LeaseTimeToLiveRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if m.Keys {
-		dAtA[i] = 0x10
-		i++
-		if m.Keys {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *LeaseTimeToLiveResponse) 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 *LeaseTimeToLiveResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n32, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n32
-	}
-	if m.ID != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.TTL))
-	}
-	if m.GrantedTTL != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.GrantedTTL))
-	}
-	if len(m.Keys) > 0 {
-		for _, b := range m.Keys {
-			dAtA[i] = 0x2a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(len(b)))
-			i += copy(dAtA[i:], b)
-		}
-	}
-	return i, nil
-}
-
-func (m *LeaseLeasesRequest) 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 *LeaseLeasesRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *LeaseStatus) 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 *LeaseStatus) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *LeaseLeasesResponse) 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 *LeaseLeasesResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n33, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n33
-	}
-	if len(m.Leases) > 0 {
-		for _, msg := range m.Leases {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *Member) 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 *Member) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if len(m.Name) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			dAtA[i] = 0x1a
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	if len(m.ClientURLs) > 0 {
-		for _, s := range m.ClientURLs {
-			dAtA[i] = 0x22
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	if m.IsLearner {
-		dAtA[i] = 0x28
-		i++
-		if m.IsLearner {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *MemberAddRequest) 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 *MemberAddRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			dAtA[i] = 0xa
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	if m.IsLearner {
-		dAtA[i] = 0x10
-		i++
-		if m.IsLearner {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *MemberAddResponse) 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 *MemberAddResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n34, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n34
-	}
-	if m.Member != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Member.Size()))
-		n35, err := m.Member.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n35
-	}
-	if len(m.Members) > 0 {
-		for _, msg := range m.Members {
-			dAtA[i] = 0x1a
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *MemberRemoveRequest) 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 *MemberRemoveRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *MemberRemoveResponse) 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 *MemberRemoveResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n36, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n36
-	}
-	if len(m.Members) > 0 {
-		for _, msg := range m.Members {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *MemberUpdateRequest) 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 *MemberUpdateRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			dAtA[i] = 0x12
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	return i, nil
-}
-
-func (m *MemberUpdateResponse) 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 *MemberUpdateResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n37, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n37
-	}
-	if len(m.Members) > 0 {
-		for _, msg := range m.Members {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *MemberListRequest) 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 *MemberListRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *MemberListResponse) 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 *MemberListResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n38, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n38
-	}
-	if len(m.Members) > 0 {
-		for _, msg := range m.Members {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *MemberPromoteRequest) 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 *MemberPromoteRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.ID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.ID))
-	}
-	return i, nil
-}
-
-func (m *MemberPromoteResponse) 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 *MemberPromoteResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n39, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n39
-	}
-	if len(m.Members) > 0 {
-		for _, msg := range m.Members {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *DefragmentRequest) 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 *DefragmentRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *DefragmentResponse) 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 *DefragmentResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n40, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n40
-	}
-	return i, nil
-}
-
-func (m *MoveLeaderRequest) Marshal() (dAtA []byte, err error) {
-	size := m.Size()
-	dAtA = make([]byte, size)
-	n, err := m.MarshalTo(dAtA)
-	if err != nil {
-		return nil, err
-	}
-	return dAtA[:n], nil
-}
-
-func (m *MoveLeaderRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.TargetID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.TargetID))
-	}
-	return i, nil
-}
-
-func (m *MoveLeaderResponse) Marshal() (dAtA []byte, err error) {
-	size := m.Size()
-	dAtA = make([]byte, size)
-	n, err := m.MarshalTo(dAtA)
-	if err != nil {
-		return nil, err
-	}
-	return dAtA[:n], nil
-}
-
-func (m *MoveLeaderResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n41, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n41
-	}
-	return i, nil
-}
-
-func (m *AlarmRequest) 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 *AlarmRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Action != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Action))
-	}
-	if m.MemberID != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MemberID))
-	}
-	if m.Alarm != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Alarm))
-	}
-	return i, nil
-}
-
-func (m *AlarmMember) 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 *AlarmMember) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.MemberID != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.MemberID))
-	}
-	if m.Alarm != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Alarm))
-	}
-	return i, nil
-}
-
-func (m *AlarmResponse) 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 *AlarmResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n42, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n42
-	}
-	if len(m.Alarms) > 0 {
-		for _, msg := range m.Alarms {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *StatusRequest) 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 *StatusRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *StatusResponse) 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 *StatusResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n43, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n43
-	}
-	if len(m.Version) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Version)))
-		i += copy(dAtA[i:], m.Version)
-	}
-	if m.DbSize != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.DbSize))
-	}
-	if m.Leader != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Leader))
-	}
-	if m.RaftIndex != 0 {
-		dAtA[i] = 0x28
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RaftIndex))
-	}
-	if m.RaftTerm != 0 {
-		dAtA[i] = 0x30
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RaftTerm))
-	}
-	if m.RaftAppliedIndex != 0 {
-		dAtA[i] = 0x38
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.RaftAppliedIndex))
-	}
-	if len(m.Errors) > 0 {
-		for _, s := range m.Errors {
-			dAtA[i] = 0x42
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	if m.DbSizeInUse != 0 {
-		dAtA[i] = 0x48
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.DbSizeInUse))
-	}
-	if m.IsLearner {
-		dAtA[i] = 0x50
-		i++
-		if m.IsLearner {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
-		i++
-	}
-	return i, nil
-}
-
-func (m *AuthEnableRequest) 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 *AuthEnableRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *AuthDisableRequest) 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 *AuthDisableRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *AuthenticateRequest) 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 *AuthenticateRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Password) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
-		i += copy(dAtA[i:], m.Password)
-	}
-	return i, nil
-}
-
-func (m *AuthUserAddRequest) 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 *AuthUserAddRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Password) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
-		i += copy(dAtA[i:], m.Password)
-	}
-	if m.Options != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Options.Size()))
-		n44, err := m.Options.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n44
-	}
-	return i, nil
-}
-
-func (m *AuthUserGetRequest) 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 *AuthUserGetRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	return i, nil
-}
-
-func (m *AuthUserDeleteRequest) 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 *AuthUserDeleteRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	return i, nil
-}
-
-func (m *AuthUserChangePasswordRequest) 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 *AuthUserChangePasswordRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Password) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Password)))
-		i += copy(dAtA[i:], m.Password)
-	}
-	return i, nil
-}
-
-func (m *AuthUserGrantRoleRequest) 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 *AuthUserGrantRoleRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.User) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.User)))
-		i += copy(dAtA[i:], m.User)
-	}
-	if len(m.Role) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
-		i += copy(dAtA[i:], m.Role)
-	}
-	return i, nil
-}
-
-func (m *AuthUserRevokeRoleRequest) 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 *AuthUserRevokeRoleRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if len(m.Role) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
-		i += copy(dAtA[i:], m.Role)
-	}
-	return i, nil
-}
-
-func (m *AuthRoleAddRequest) 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 *AuthRoleAddRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	return i, nil
-}
-
-func (m *AuthRoleGetRequest) 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 *AuthRoleGetRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Role) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
-		i += copy(dAtA[i:], m.Role)
-	}
-	return i, nil
-}
-
-func (m *AuthUserListRequest) 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 *AuthUserListRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *AuthRoleListRequest) 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 *AuthRoleListRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	return i, nil
-}
-
-func (m *AuthRoleDeleteRequest) 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 *AuthRoleDeleteRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Role) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
-		i += copy(dAtA[i:], m.Role)
-	}
-	return i, nil
-}
-
-func (m *AuthRoleGrantPermissionRequest) 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 *AuthRoleGrantPermissionRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Name) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Name)))
-		i += copy(dAtA[i:], m.Name)
-	}
-	if m.Perm != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Perm.Size()))
-		n45, err := m.Perm.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n45
-	}
-	return i, nil
-}
-
-func (m *AuthRoleRevokePermissionRequest) 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 *AuthRoleRevokePermissionRequest) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Role) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Role)))
-		i += copy(dAtA[i:], m.Role)
-	}
-	if len(m.Key) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if len(m.RangeEnd) > 0 {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.RangeEnd)))
-		i += copy(dAtA[i:], m.RangeEnd)
-	}
-	return i, nil
-}
-
-func (m *AuthEnableResponse) 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 *AuthEnableResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n46, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n46
-	}
-	return i, nil
-}
-
-func (m *AuthDisableResponse) 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 *AuthDisableResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n47, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n47
-	}
-	return i, nil
-}
-
-func (m *AuthenticateResponse) 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 *AuthenticateResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n48, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n48
-	}
-	if len(m.Token) > 0 {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(len(m.Token)))
-		i += copy(dAtA[i:], m.Token)
-	}
-	return i, nil
-}
-
-func (m *AuthUserAddResponse) 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 *AuthUserAddResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n49, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n49
-	}
-	return i, nil
-}
-
-func (m *AuthUserGetResponse) 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 *AuthUserGetResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n50, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n50
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			dAtA[i] = 0x12
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	return i, nil
-}
-
-func (m *AuthUserDeleteResponse) 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 *AuthUserDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n51, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n51
-	}
-	return i, nil
-}
-
-func (m *AuthUserChangePasswordResponse) 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 *AuthUserChangePasswordResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n52, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n52
-	}
-	return i, nil
-}
-
-func (m *AuthUserGrantRoleResponse) 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 *AuthUserGrantRoleResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n53, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n53
-	}
-	return i, nil
-}
-
-func (m *AuthUserRevokeRoleResponse) 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 *AuthUserRevokeRoleResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n54, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n54
-	}
-	return i, nil
-}
-
-func (m *AuthRoleAddResponse) 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 *AuthRoleAddResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n55, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n55
-	}
-	return i, nil
-}
-
-func (m *AuthRoleGetResponse) 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 *AuthRoleGetResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n56, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n56
-	}
-	if len(m.Perm) > 0 {
-		for _, msg := range m.Perm {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRpc(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	return i, nil
-}
-
-func (m *AuthRoleListResponse) 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 *AuthRoleListResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n57, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n57
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			dAtA[i] = 0x12
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	return i, nil
-}
-
-func (m *AuthUserListResponse) 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 *AuthUserListResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n58, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n58
-	}
-	if len(m.Users) > 0 {
-		for _, s := range m.Users {
-			dAtA[i] = 0x12
-			i++
-			l = len(s)
-			for l >= 1<<7 {
-				dAtA[i] = uint8(uint64(l)&0x7f | 0x80)
-				l >>= 7
-				i++
-			}
-			dAtA[i] = uint8(l)
-			i++
-			i += copy(dAtA[i:], s)
-		}
-	}
-	return i, nil
-}
-
-func (m *AuthRoleDeleteResponse) 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 *AuthRoleDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n59, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n59
-	}
-	return i, nil
-}
-
-func (m *AuthRoleGrantPermissionResponse) 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 *AuthRoleGrantPermissionResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n60, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n60
-	}
-	return i, nil
-}
-
-func (m *AuthRoleRevokePermissionResponse) 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 *AuthRoleRevokePermissionResponse) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Header != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n61, err := m.Header.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n61
-	}
-	return i, nil
-}
-
-func encodeVarintRpc(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 *ResponseHeader) Size() (n int) {
-	var l int
-	_ = l
-	if m.ClusterId != 0 {
-		n += 1 + sovRpc(uint64(m.ClusterId))
-	}
-	if m.MemberId != 0 {
-		n += 1 + sovRpc(uint64(m.MemberId))
-	}
-	if m.Revision != 0 {
-		n += 1 + sovRpc(uint64(m.Revision))
-	}
-	if m.RaftTerm != 0 {
-		n += 1 + sovRpc(uint64(m.RaftTerm))
-	}
-	return n
-}
-
-func (m *RangeRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Limit != 0 {
-		n += 1 + sovRpc(uint64(m.Limit))
-	}
-	if m.Revision != 0 {
-		n += 1 + sovRpc(uint64(m.Revision))
-	}
-	if m.SortOrder != 0 {
-		n += 1 + sovRpc(uint64(m.SortOrder))
-	}
-	if m.SortTarget != 0 {
-		n += 1 + sovRpc(uint64(m.SortTarget))
-	}
-	if m.Serializable {
-		n += 2
-	}
-	if m.KeysOnly {
-		n += 2
-	}
-	if m.CountOnly {
-		n += 2
-	}
-	if m.MinModRevision != 0 {
-		n += 1 + sovRpc(uint64(m.MinModRevision))
-	}
-	if m.MaxModRevision != 0 {
-		n += 1 + sovRpc(uint64(m.MaxModRevision))
-	}
-	if m.MinCreateRevision != 0 {
-		n += 1 + sovRpc(uint64(m.MinCreateRevision))
-	}
-	if m.MaxCreateRevision != 0 {
-		n += 1 + sovRpc(uint64(m.MaxCreateRevision))
-	}
-	return n
-}
-
-func (m *RangeResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Kvs) > 0 {
-		for _, e := range m.Kvs {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if m.More {
-		n += 2
-	}
-	if m.Count != 0 {
-		n += 1 + sovRpc(uint64(m.Count))
-	}
-	return n
-}
-
-func (m *PutRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Value)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Lease != 0 {
-		n += 1 + sovRpc(uint64(m.Lease))
-	}
-	if m.PrevKv {
-		n += 2
-	}
-	if m.IgnoreValue {
-		n += 2
-	}
-	if m.IgnoreLease {
-		n += 2
-	}
-	return n
-}
-
-func (m *PutResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.PrevKv != nil {
-		l = m.PrevKv.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *DeleteRangeRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.PrevKv {
-		n += 2
-	}
-	return n
-}
-
-func (m *DeleteRangeResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Deleted != 0 {
-		n += 1 + sovRpc(uint64(m.Deleted))
-	}
-	if len(m.PrevKvs) > 0 {
-		for _, e := range m.PrevKvs {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *RequestOp) Size() (n int) {
-	var l int
-	_ = l
-	if m.Request != nil {
-		n += m.Request.Size()
-	}
-	return n
-}
-
-func (m *RequestOp_RequestRange) Size() (n int) {
-	var l int
-	_ = l
-	if m.RequestRange != nil {
-		l = m.RequestRange.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *RequestOp_RequestPut) Size() (n int) {
-	var l int
-	_ = l
-	if m.RequestPut != nil {
-		l = m.RequestPut.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *RequestOp_RequestDeleteRange) Size() (n int) {
-	var l int
-	_ = l
-	if m.RequestDeleteRange != nil {
-		l = m.RequestDeleteRange.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *RequestOp_RequestTxn) Size() (n int) {
-	var l int
-	_ = l
-	if m.RequestTxn != nil {
-		l = m.RequestTxn.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *ResponseOp) Size() (n int) {
-	var l int
-	_ = l
-	if m.Response != nil {
-		n += m.Response.Size()
-	}
-	return n
-}
-
-func (m *ResponseOp_ResponseRange) Size() (n int) {
-	var l int
-	_ = l
-	if m.ResponseRange != nil {
-		l = m.ResponseRange.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *ResponseOp_ResponsePut) Size() (n int) {
-	var l int
-	_ = l
-	if m.ResponsePut != nil {
-		l = m.ResponsePut.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *ResponseOp_ResponseDeleteRange) Size() (n int) {
-	var l int
-	_ = l
-	if m.ResponseDeleteRange != nil {
-		l = m.ResponseDeleteRange.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *ResponseOp_ResponseTxn) Size() (n int) {
-	var l int
-	_ = l
-	if m.ResponseTxn != nil {
-		l = m.ResponseTxn.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *Compare) Size() (n int) {
-	var l int
-	_ = l
-	if m.Result != 0 {
-		n += 1 + sovRpc(uint64(m.Result))
-	}
-	if m.Target != 0 {
-		n += 1 + sovRpc(uint64(m.Target))
-	}
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.TargetUnion != nil {
-		n += m.TargetUnion.Size()
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 2 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *Compare_Version) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRpc(uint64(m.Version))
-	return n
-}
-func (m *Compare_CreateRevision) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRpc(uint64(m.CreateRevision))
-	return n
-}
-func (m *Compare_ModRevision) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRpc(uint64(m.ModRevision))
-	return n
-}
-func (m *Compare_Value) Size() (n int) {
-	var l int
-	_ = l
-	if m.Value != nil {
-		l = len(m.Value)
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *Compare_Lease) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRpc(uint64(m.Lease))
-	return n
-}
-func (m *TxnRequest) Size() (n int) {
-	var l int
-	_ = l
-	if len(m.Compare) > 0 {
-		for _, e := range m.Compare {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if len(m.Success) > 0 {
-		for _, e := range m.Success {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if len(m.Failure) > 0 {
-		for _, e := range m.Failure {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *TxnResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Succeeded {
-		n += 2
-	}
-	if len(m.Responses) > 0 {
-		for _, e := range m.Responses {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *CompactionRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.Revision != 0 {
-		n += 1 + sovRpc(uint64(m.Revision))
-	}
-	if m.Physical {
-		n += 2
-	}
-	return n
-}
-
-func (m *CompactionResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *HashRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *HashKVRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.Revision != 0 {
-		n += 1 + sovRpc(uint64(m.Revision))
-	}
-	return n
-}
-
-func (m *HashKVResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Hash != 0 {
-		n += 1 + sovRpc(uint64(m.Hash))
-	}
-	if m.CompactRevision != 0 {
-		n += 1 + sovRpc(uint64(m.CompactRevision))
-	}
-	return n
-}
-
-func (m *HashResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Hash != 0 {
-		n += 1 + sovRpc(uint64(m.Hash))
-	}
-	return n
-}
-
-func (m *SnapshotRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *SnapshotResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.RemainingBytes != 0 {
-		n += 1 + sovRpc(uint64(m.RemainingBytes))
-	}
-	l = len(m.Blob)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *WatchRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.RequestUnion != nil {
-		n += m.RequestUnion.Size()
-	}
-	return n
-}
-
-func (m *WatchRequest_CreateRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.CreateRequest != nil {
-		l = m.CreateRequest.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *WatchRequest_CancelRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.CancelRequest != nil {
-		l = m.CancelRequest.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *WatchRequest_ProgressRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ProgressRequest != nil {
-		l = m.ProgressRequest.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-func (m *WatchCreateRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.StartRevision != 0 {
-		n += 1 + sovRpc(uint64(m.StartRevision))
-	}
-	if m.ProgressNotify {
-		n += 2
-	}
-	if len(m.Filters) > 0 {
-		l = 0
-		for _, e := range m.Filters {
-			l += sovRpc(uint64(e))
-		}
-		n += 1 + sovRpc(uint64(l)) + l
-	}
-	if m.PrevKv {
-		n += 2
-	}
-	if m.WatchId != 0 {
-		n += 1 + sovRpc(uint64(m.WatchId))
-	}
-	if m.Fragment {
-		n += 2
-	}
-	return n
-}
-
-func (m *WatchCancelRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.WatchId != 0 {
-		n += 1 + sovRpc(uint64(m.WatchId))
-	}
-	return n
-}
-
-func (m *WatchProgressRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *WatchResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.WatchId != 0 {
-		n += 1 + sovRpc(uint64(m.WatchId))
-	}
-	if m.Created {
-		n += 2
-	}
-	if m.Canceled {
-		n += 2
-	}
-	if m.CompactRevision != 0 {
-		n += 1 + sovRpc(uint64(m.CompactRevision))
-	}
-	l = len(m.CancelReason)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Fragment {
-		n += 2
-	}
-	if len(m.Events) > 0 {
-		for _, e := range m.Events {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *LeaseGrantRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.TTL != 0 {
-		n += 1 + sovRpc(uint64(m.TTL))
-	}
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *LeaseGrantResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		n += 1 + sovRpc(uint64(m.TTL))
-	}
-	l = len(m.Error)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *LeaseRevokeRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *LeaseRevokeResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *LeaseCheckpoint) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if m.Remaining_TTL != 0 {
-		n += 1 + sovRpc(uint64(m.Remaining_TTL))
-	}
-	return n
-}
-
-func (m *LeaseCheckpointRequest) Size() (n int) {
-	var l int
-	_ = l
-	if len(m.Checkpoints) > 0 {
-		for _, e := range m.Checkpoints {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *LeaseCheckpointResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *LeaseKeepAliveRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *LeaseKeepAliveResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		n += 1 + sovRpc(uint64(m.TTL))
-	}
-	return n
-}
-
-func (m *LeaseTimeToLiveRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if m.Keys {
-		n += 2
-	}
-	return n
-}
-
-func (m *LeaseTimeToLiveResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if m.TTL != 0 {
-		n += 1 + sovRpc(uint64(m.TTL))
-	}
-	if m.GrantedTTL != 0 {
-		n += 1 + sovRpc(uint64(m.GrantedTTL))
-	}
-	if len(m.Keys) > 0 {
-		for _, b := range m.Keys {
-			l = len(b)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *LeaseLeasesRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *LeaseStatus) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *LeaseLeasesResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Leases) > 0 {
-		for _, e := range m.Leases {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *Member) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if len(m.ClientURLs) > 0 {
-		for _, s := range m.ClientURLs {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if m.IsLearner {
-		n += 2
-	}
-	return n
-}
-
-func (m *MemberAddRequest) Size() (n int) {
-	var l int
-	_ = l
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if m.IsLearner {
-		n += 2
-	}
-	return n
-}
-
-func (m *MemberAddResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Member != nil {
-		l = m.Member.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Members) > 0 {
-		for _, e := range m.Members {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *MemberRemoveRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *MemberRemoveResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Members) > 0 {
-		for _, e := range m.Members {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *MemberUpdateRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	if len(m.PeerURLs) > 0 {
-		for _, s := range m.PeerURLs {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *MemberUpdateResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Members) > 0 {
-		for _, e := range m.Members {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *MemberListRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *MemberListResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Members) > 0 {
-		for _, e := range m.Members {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *MemberPromoteRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.ID != 0 {
-		n += 1 + sovRpc(uint64(m.ID))
-	}
-	return n
-}
-
-func (m *MemberPromoteResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Members) > 0 {
-		for _, e := range m.Members {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *DefragmentRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *DefragmentResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *MoveLeaderRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.TargetID != 0 {
-		n += 1 + sovRpc(uint64(m.TargetID))
-	}
-	return n
-}
-
-func (m *MoveLeaderResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AlarmRequest) Size() (n int) {
-	var l int
-	_ = l
-	if m.Action != 0 {
-		n += 1 + sovRpc(uint64(m.Action))
-	}
-	if m.MemberID != 0 {
-		n += 1 + sovRpc(uint64(m.MemberID))
-	}
-	if m.Alarm != 0 {
-		n += 1 + sovRpc(uint64(m.Alarm))
-	}
-	return n
-}
-
-func (m *AlarmMember) Size() (n int) {
-	var l int
-	_ = l
-	if m.MemberID != 0 {
-		n += 1 + sovRpc(uint64(m.MemberID))
-	}
-	if m.Alarm != 0 {
-		n += 1 + sovRpc(uint64(m.Alarm))
-	}
-	return n
-}
-
-func (m *AlarmResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Alarms) > 0 {
-		for _, e := range m.Alarms {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *StatusRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *StatusResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Version)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.DbSize != 0 {
-		n += 1 + sovRpc(uint64(m.DbSize))
-	}
-	if m.Leader != 0 {
-		n += 1 + sovRpc(uint64(m.Leader))
-	}
-	if m.RaftIndex != 0 {
-		n += 1 + sovRpc(uint64(m.RaftIndex))
-	}
-	if m.RaftTerm != 0 {
-		n += 1 + sovRpc(uint64(m.RaftTerm))
-	}
-	if m.RaftAppliedIndex != 0 {
-		n += 1 + sovRpc(uint64(m.RaftAppliedIndex))
-	}
-	if len(m.Errors) > 0 {
-		for _, s := range m.Errors {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	if m.DbSizeInUse != 0 {
-		n += 1 + sovRpc(uint64(m.DbSizeInUse))
-	}
-	if m.IsLearner {
-		n += 2
-	}
-	return n
-}
-
-func (m *AuthEnableRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *AuthDisableRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *AuthenticateRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Password)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserAddRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Password)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Options != nil {
-		l = m.Options.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserGetRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserDeleteRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserChangePasswordRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Password)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserGrantRoleRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.User)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Role)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserRevokeRoleRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Role)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleAddRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleGetRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Role)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserListRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *AuthRoleListRequest) Size() (n int) {
-	var l int
-	_ = l
-	return n
-}
-
-func (m *AuthRoleDeleteRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Role)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleGrantPermissionRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Name)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if m.Perm != nil {
-		l = m.Perm.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleRevokePermissionRequest) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Role)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.RangeEnd)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthEnableResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthDisableResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthenticateResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	l = len(m.Token)
-	if l > 0 {
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserAddResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserGetResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *AuthUserDeleteResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserChangePasswordResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserGrantRoleResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthUserRevokeRoleResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleAddResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleGetResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Perm) > 0 {
-		for _, e := range m.Perm {
-			l = e.Size()
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *AuthRoleListResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Roles) > 0 {
-		for _, s := range m.Roles {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *AuthUserListResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	if len(m.Users) > 0 {
-		for _, s := range m.Users {
-			l = len(s)
-			n += 1 + l + sovRpc(uint64(l))
-		}
-	}
-	return n
-}
-
-func (m *AuthRoleDeleteResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleGrantPermissionResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func (m *AuthRoleRevokePermissionResponse) Size() (n int) {
-	var l int
-	_ = l
-	if m.Header != nil {
-		l = m.Header.Size()
-		n += 1 + l + sovRpc(uint64(l))
-	}
-	return n
-}
-
-func sovRpc(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozRpc(x uint64) (n int) {
-	return sovRpc(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *ResponseHeader) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: ResponseHeader: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ResponseHeader: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ClusterId", wireType)
-			}
-			m.ClusterId = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ClusterId |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MemberId", wireType)
-			}
-			m.MemberId = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MemberId |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
-			}
-			m.Revision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Revision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RaftTerm", wireType)
-			}
-			m.RaftTerm = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RaftTerm |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *RangeRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: RangeRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: RangeRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType)
-			}
-			m.Limit = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Limit |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
-			}
-			m.Revision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Revision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field SortOrder", wireType)
-			}
-			m.SortOrder = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.SortOrder |= (RangeRequest_SortOrder(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 6:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field SortTarget", wireType)
-			}
-			m.SortTarget = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.SortTarget |= (RangeRequest_SortTarget(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 7:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Serializable", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Serializable = bool(v != 0)
-		case 8:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field KeysOnly", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.KeysOnly = bool(v != 0)
-		case 9:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CountOnly", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.CountOnly = bool(v != 0)
-		case 10:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MinModRevision", wireType)
-			}
-			m.MinModRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MinModRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 11:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MaxModRevision", wireType)
-			}
-			m.MaxModRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MaxModRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 12:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MinCreateRevision", wireType)
-			}
-			m.MinCreateRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MinCreateRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 13:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MaxCreateRevision", wireType)
-			}
-			m.MaxCreateRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MaxCreateRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *RangeResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: RangeResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: RangeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Kvs", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Kvs = append(m.Kvs, &mvccpb.KeyValue{})
-			if err := m.Kvs[len(m.Kvs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field More", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.More = bool(v != 0)
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType)
-			}
-			m.Count = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Count |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *PutRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: PutRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: PutRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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
-		case 3:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Lease |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.PrevKv = bool(v != 0)
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field IgnoreValue", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.IgnoreValue = bool(v != 0)
-		case 6:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field IgnoreLease", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.IgnoreLease = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *PutResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: PutResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: PutResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.PrevKv == nil {
-				m.PrevKv = &mvccpb.KeyValue{}
-			}
-			if err := m.PrevKv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: DeleteRangeRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: DeleteRangeRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.PrevKv = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: DeleteRangeResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: DeleteRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Deleted", wireType)
-			}
-			m.Deleted = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Deleted |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKvs", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.PrevKvs = append(m.PrevKvs, &mvccpb.KeyValue{})
-			if err := m.PrevKvs[len(m.PrevKvs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *RequestOp) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: RequestOp: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: RequestOp: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RequestRange", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &RangeRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Request = &RequestOp_RequestRange{v}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RequestPut", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &PutRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Request = &RequestOp_RequestPut{v}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RequestDeleteRange", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &DeleteRangeRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Request = &RequestOp_RequestDeleteRange{v}
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RequestTxn", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &TxnRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Request = &RequestOp_RequestTxn{v}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *ResponseOp) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: ResponseOp: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ResponseOp: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ResponseRange", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &RangeResponse{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Response = &ResponseOp_ResponseRange{v}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ResponsePut", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &PutResponse{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Response = &ResponseOp_ResponsePut{v}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ResponseDeleteRange", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &DeleteRangeResponse{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Response = &ResponseOp_ResponseDeleteRange{v}
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ResponseTxn", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &TxnResponse{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.Response = &ResponseOp_ResponseTxn{v}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Compare) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: Compare: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Compare: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
-			}
-			m.Result = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Result |= (Compare_CompareResult(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType)
-			}
-			m.Target = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Target |= (Compare_CompareTarget(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
-			}
-			var v int64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.TargetUnion = &Compare_Version{v}
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType)
-			}
-			var v int64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.TargetUnion = &Compare_CreateRevision{v}
-		case 6:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType)
-			}
-			var v int64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.TargetUnion = &Compare_ModRevision{v}
-		case 7:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := make([]byte, postIndex-iNdEx)
-			copy(v, dAtA[iNdEx:postIndex])
-			m.TargetUnion = &Compare_Value{v}
-			iNdEx = postIndex
-		case 8:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Lease", wireType)
-			}
-			var v int64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.TargetUnion = &Compare_Lease{v}
-		case 64:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *TxnRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: TxnRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: TxnRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Compare", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Compare = append(m.Compare, &Compare{})
-			if err := m.Compare[len(m.Compare)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Success", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Success = append(m.Success, &RequestOp{})
-			if err := m.Success[len(m.Success)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Failure", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Failure = append(m.Failure, &RequestOp{})
-			if err := m.Failure[len(m.Failure)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *TxnResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: TxnResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: TxnResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Succeeded", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Succeeded = bool(v != 0)
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Responses = append(m.Responses, &ResponseOp{})
-			if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *CompactionRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: CompactionRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: CompactionRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
-			}
-			m.Revision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Revision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Physical", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Physical = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *CompactionResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: CompactionResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: CompactionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *HashRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: HashRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: HashRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *HashKVRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: HashKVRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: HashKVRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Revision", wireType)
-			}
-			m.Revision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Revision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *HashKVResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: HashKVResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: HashKVResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType)
-			}
-			m.Hash = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Hash |= (uint32(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CompactRevision", wireType)
-			}
-			m.CompactRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.CompactRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *HashResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: HashResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: HashResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType)
-			}
-			m.Hash = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Hash |= (uint32(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *SnapshotRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: SnapshotRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: SnapshotRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *SnapshotResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: SnapshotResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: SnapshotResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RemainingBytes", wireType)
-			}
-			m.RemainingBytes = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RemainingBytes |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Blob", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Blob = append(m.Blob[:0], dAtA[iNdEx:postIndex]...)
-			if m.Blob == nil {
-				m.Blob = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *WatchRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: WatchRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: WatchRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CreateRequest", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &WatchCreateRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.RequestUnion = &WatchRequest_CreateRequest{v}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CancelRequest", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &WatchCancelRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.RequestUnion = &WatchRequest_CancelRequest{v}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ProgressRequest", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			v := &WatchProgressRequest{}
-			if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			m.RequestUnion = &WatchRequest_ProgressRequest{v}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *WatchCreateRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: WatchCreateRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: WatchCreateRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field StartRevision", wireType)
-			}
-			m.StartRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.StartRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ProgressNotify", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.ProgressNotify = bool(v != 0)
-		case 5:
-			if wireType == 0 {
-				var v WatchCreateRequest_FilterType
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRpc
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= (WatchCreateRequest_FilterType(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.Filters = append(m.Filters, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRpc
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= (int(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthRpc
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				for iNdEx < postIndex {
-					var v WatchCreateRequest_FilterType
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowRpc
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= (WatchCreateRequest_FilterType(b) & 0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.Filters = append(m.Filters, v)
-				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Filters", wireType)
-			}
-		case 6:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.PrevKv = bool(v != 0)
-		case 7:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
-			}
-			m.WatchId = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.WatchId |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 8:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Fragment = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *WatchCancelRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: WatchCancelRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: WatchCancelRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
-			}
-			m.WatchId = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.WatchId |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *WatchProgressRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: WatchProgressRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: WatchProgressRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *WatchResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: WatchResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: WatchResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field WatchId", wireType)
-			}
-			m.WatchId = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.WatchId |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Created", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Created = bool(v != 0)
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Canceled", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Canceled = bool(v != 0)
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CompactRevision", wireType)
-			}
-			m.CompactRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.CompactRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 6:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CancelReason", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.CancelReason = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 7:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Fragment", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Fragment = bool(v != 0)
-		case 11:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Events = append(m.Events, &mvccpb.Event{})
-			if err := m.Events[len(m.Events)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseGrantRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseGrantRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseGrantRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.TTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseGrantResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseGrantResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseGrantResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.TTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Error = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseRevokeRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseRevokeRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseRevokeRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseRevokeResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseRevokeResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseRevokeResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseCheckpoint) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseCheckpoint: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseCheckpoint: 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 ErrIntOverflowRpc
-				}
-				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 Remaining_TTL", wireType)
-			}
-			m.Remaining_TTL = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Remaining_TTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseCheckpointRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseCheckpointRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseCheckpointRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Checkpoints", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Checkpoints = append(m.Checkpoints, &LeaseCheckpoint{})
-			if err := m.Checkpoints[len(m.Checkpoints)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseCheckpointResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseCheckpointResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseCheckpointResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseKeepAliveRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseKeepAliveRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseKeepAliveRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseKeepAliveResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseKeepAliveResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseKeepAliveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.TTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseTimeToLiveRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseTimeToLiveRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseTimeToLiveRequest: 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 ErrIntOverflowRpc
-				}
-				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 Keys", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Keys = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseTimeToLiveResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseTimeToLiveResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseTimeToLiveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.TTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field GrantedTTL", wireType)
-			}
-			m.GrantedTTL = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.GrantedTTL |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 5:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx))
-			copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseLeasesRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseLeasesRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseLeasesRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseStatus) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseStatus: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseStatus: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *LeaseLeasesResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: LeaseLeasesResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeaseLeasesResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Leases", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Leases = append(m.Leases, &LeaseStatus{})
-			if err := m.Leases[len(m.Leases)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Member) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: Member: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Member: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ClientURLs", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.ClientURLs = append(m.ClientURLs, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field IsLearner", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.IsLearner = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberAddRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberAddRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberAddRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field IsLearner", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.IsLearner = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberAddResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberAddResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Member", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Member == nil {
-				m.Member = &Member{}
-			}
-			if err := m.Member.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Members = append(m.Members, &Member{})
-			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberRemoveRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberRemoveRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberRemoveRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberRemoveResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberRemoveResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberRemoveResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Members = append(m.Members, &Member{})
-			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberUpdateRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberUpdateRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberUpdateRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PeerURLs", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.PeerURLs = append(m.PeerURLs, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberUpdateResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberUpdateResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberUpdateResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Members = append(m.Members, &Member{})
-			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberListRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberListRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberListResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberListResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Members = append(m.Members, &Member{})
-			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberPromoteRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberPromoteRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberPromoteRequest: 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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MemberPromoteResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MemberPromoteResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MemberPromoteResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Members", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Members = append(m.Members, &Member{})
-			if err := m.Members[len(m.Members)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *DefragmentRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: DefragmentRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: DefragmentRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *DefragmentResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: DefragmentResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: DefragmentResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MoveLeaderRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MoveLeaderRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MoveLeaderRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field TargetID", wireType)
-			}
-			m.TargetID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.TargetID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *MoveLeaderResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: MoveLeaderResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: MoveLeaderResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AlarmRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AlarmRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AlarmRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType)
-			}
-			m.Action = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Action |= (AlarmRequest_AlarmAction(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MemberID", wireType)
-			}
-			m.MemberID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MemberID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
-			}
-			m.Alarm = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Alarm |= (AlarmType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AlarmMember) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AlarmMember: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AlarmMember: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field MemberID", wireType)
-			}
-			m.MemberID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.MemberID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Alarm", wireType)
-			}
-			m.Alarm = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Alarm |= (AlarmType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AlarmResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AlarmResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AlarmResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Alarms", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Alarms = append(m.Alarms, &AlarmMember{})
-			if err := m.Alarms[len(m.Alarms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *StatusRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: StatusRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: StatusRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *StatusResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: StatusResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: StatusResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Version = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field DbSize", wireType)
-			}
-			m.DbSize = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.DbSize |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType)
-			}
-			m.Leader = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Leader |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RaftIndex", wireType)
-			}
-			m.RaftIndex = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RaftIndex |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 6:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RaftTerm", wireType)
-			}
-			m.RaftTerm = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RaftTerm |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 7:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RaftAppliedIndex", wireType)
-			}
-			m.RaftAppliedIndex = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RaftAppliedIndex |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 8:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Errors", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Errors = append(m.Errors, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		case 9:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field DbSizeInUse", wireType)
-			}
-			m.DbSizeInUse = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.DbSizeInUse |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 10:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field IsLearner", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.IsLearner = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthEnableRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthEnableRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthEnableRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthDisableRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthDisableRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthDisableRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthenticateRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthenticateRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthenticateRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Password = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserAddRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserAddRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserAddRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Password = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Options == nil {
-				m.Options = &authpb.UserAddOptions{}
-			}
-			if err := m.Options.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserGetRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserGetRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserGetRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserDeleteRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserDeleteRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserDeleteRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserChangePasswordRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserChangePasswordRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserChangePasswordRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Password = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserGrantRoleRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserGrantRoleRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserGrantRoleRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.User = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Role = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserRevokeRoleRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserRevokeRoleRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserRevokeRoleRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Role = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleAddRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleAddRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleAddRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleGetRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleGetRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleGetRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Role = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserListRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserListRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleListRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleListRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleListRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleDeleteRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleDeleteRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Role = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleGrantPermissionRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleGrantPermissionRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleGrantPermissionRequest: 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 stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Name = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Perm", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Perm == nil {
-				m.Perm = &authpb.Permission{}
-			}
-			if err := m.Perm.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleRevokePermissionRequest) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleRevokePermissionRequest: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleRevokePermissionRequest: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Role", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Role = string(dAtA[iNdEx:postIndex])
-			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 ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			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 != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RangeEnd", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.RangeEnd = append(m.RangeEnd[:0], dAtA[iNdEx:postIndex]...)
-			if m.RangeEnd == nil {
-				m.RangeEnd = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthEnableResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthEnableResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthEnableResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthDisableResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthDisableResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthDisableResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthenticateResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthenticateResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthenticateResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Token", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Token = string(dAtA[iNdEx:postIndex])
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserAddResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserAddResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserGetResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserGetResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserGetResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserDeleteResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserDeleteResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserChangePasswordResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserChangePasswordResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserChangePasswordResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserGrantRoleResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserGrantRoleResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserGrantRoleResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserRevokeRoleResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserRevokeRoleResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserRevokeRoleResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleAddResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleAddResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleAddResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleGetResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleGetResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleGetResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Perm", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Perm = append(m.Perm, &authpb.Permission{})
-			if err := m.Perm[len(m.Perm)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleListResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleListResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Roles", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Roles = append(m.Roles, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthUserListResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthUserListResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthUserListResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Users", wireType)
-			}
-			var stringLen uint64
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				stringLen |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			intStringLen := int(stringLen)
-			if intStringLen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + intStringLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Users = append(m.Users, string(dAtA[iNdEx:postIndex]))
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleDeleteResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleDeleteResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleGrantPermissionResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleGrantPermissionResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleGrantPermissionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *AuthRoleRevokePermissionResponse) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowRpc
-			}
-			if iNdEx >= l {
-				return io.ErrUnexpectedEOF
-			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= (uint64(b) & 0x7F) << shift
-			if b < 0x80 {
-				break
-			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: AuthRoleRevokePermissionResponse: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AuthRoleRevokePermissionResponse: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRpc
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRpc
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Header == nil {
-				m.Header = &ResponseHeader{}
-			}
-			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRpc(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRpc
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func skipRpc(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, ErrIntOverflowRpc
-			}
-			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, ErrIntOverflowRpc
-				}
-				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, ErrIntOverflowRpc
-				}
-				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, ErrInvalidLengthRpc
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowRpc
-					}
-					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 := skipRpc(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 (
-	ErrInvalidLengthRpc = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowRpc   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) }
-
-var fileDescriptorRpc = []byte{
-	// 3928 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0x5b, 0x6f, 0x23, 0xc9,
-	0x75, 0x56, 0x93, 0xe2, 0xed, 0xf0, 0x22, 0xaa, 0x74, 0x19, 0x0e, 0x67, 0x46, 0xa3, 0xad, 0xd9,
-	0xd9, 0xd5, 0xce, 0xec, 0x8a, 0x6b, 0xd9, 0x4e, 0x80, 0x49, 0xe2, 0x58, 0x23, 0x71, 0x67, 0xb4,
-	0xd2, 0x88, 0xda, 0x16, 0x67, 0xf6, 0x02, 0x23, 0x42, 0x8b, 0x2c, 0x49, 0x1d, 0x91, 0xdd, 0x74,
-	0x77, 0x93, 0x23, 0x6d, 0x2e, 0x0e, 0x0c, 0xc7, 0x40, 0xf2, 0x68, 0x03, 0x41, 0xf2, 0x90, 0xa7,
-	0x20, 0x08, 0xfc, 0x90, 0xe7, 0x00, 0xf9, 0x05, 0x79, 0xca, 0x05, 0xf9, 0x03, 0xc1, 0xc6, 0x2f,
-	0xc9, 0xaf, 0x30, 0xea, 0xd6, 0x5d, 0x7d, 0xa3, 0xc6, 0xa6, 0x77, 0x5f, 0xa4, 0xae, 0x53, 0xa7,
-	0xce, 0x39, 0x75, 0xaa, 0xea, 0x9c, 0xd3, 0x5f, 0x17, 0xa1, 0xe4, 0x8c, 0x7a, 0x9b, 0x23, 0xc7,
-	0xf6, 0x6c, 0x54, 0x21, 0x5e, 0xaf, 0xef, 0x12, 0x67, 0x42, 0x9c, 0xd1, 0x69, 0x73, 0xf9, 0xdc,
-	0x3e, 0xb7, 0x59, 0x47, 0x8b, 0x3e, 0x71, 0x9e, 0xe6, 0x6d, 0xca, 0xd3, 0x1a, 0x4e, 0x7a, 0x3d,
-	0xf6, 0x67, 0x74, 0xda, 0xba, 0x9c, 0x88, 0xae, 0x3b, 0xac, 0xcb, 0x18, 0x7b, 0x17, 0xec, 0xcf,
-	0xe8, 0x94, 0xfd, 0x13, 0x9d, 0x77, 0xcf, 0x6d, 0xfb, 0x7c, 0x40, 0x5a, 0xc6, 0xc8, 0x6c, 0x19,
-	0x96, 0x65, 0x7b, 0x86, 0x67, 0xda, 0x96, 0xcb, 0x7b, 0xf1, 0x5f, 0x6a, 0x50, 0xd3, 0x89, 0x3b,
-	0xb2, 0x2d, 0x97, 0x3c, 0x27, 0x46, 0x9f, 0x38, 0xe8, 0x1e, 0x40, 0x6f, 0x30, 0x76, 0x3d, 0xe2,
-	0x9c, 0x98, 0xfd, 0x86, 0xb6, 0xae, 0x6d, 0xcc, 0xeb, 0x25, 0x41, 0xd9, 0xeb, 0xa3, 0x3b, 0x50,
-	0x1a, 0x92, 0xe1, 0x29, 0xef, 0xcd, 0xb0, 0xde, 0x22, 0x27, 0xec, 0xf5, 0x51, 0x13, 0x8a, 0x0e,
-	0x99, 0x98, 0xae, 0x69, 0x5b, 0x8d, 0xec, 0xba, 0xb6, 0x91, 0xd5, 0xfd, 0x36, 0x1d, 0xe8, 0x18,
-	0x67, 0xde, 0x89, 0x47, 0x9c, 0x61, 0x63, 0x9e, 0x0f, 0xa4, 0x84, 0x2e, 0x71, 0x86, 0xf8, 0x27,
-	0x39, 0xa8, 0xe8, 0x86, 0x75, 0x4e, 0x74, 0xf2, 0xc3, 0x31, 0x71, 0x3d, 0x54, 0x87, 0xec, 0x25,
-	0xb9, 0x66, 0xea, 0x2b, 0x3a, 0x7d, 0xe4, 0xe3, 0xad, 0x73, 0x72, 0x42, 0x2c, 0xae, 0xb8, 0x42,
-	0xc7, 0x5b, 0xe7, 0xa4, 0x6d, 0xf5, 0xd1, 0x32, 0xe4, 0x06, 0xe6, 0xd0, 0xf4, 0x84, 0x56, 0xde,
-	0x08, 0x99, 0x33, 0x1f, 0x31, 0x67, 0x07, 0xc0, 0xb5, 0x1d, 0xef, 0xc4, 0x76, 0xfa, 0xc4, 0x69,
-	0xe4, 0xd6, 0xb5, 0x8d, 0xda, 0xd6, 0xdb, 0x9b, 0xea, 0x42, 0x6c, 0xaa, 0x06, 0x6d, 0x1e, 0xdb,
-	0x8e, 0xd7, 0xa1, 0xbc, 0x7a, 0xc9, 0x95, 0x8f, 0xe8, 0x23, 0x28, 0x33, 0x21, 0x9e, 0xe1, 0x9c,
-	0x13, 0xaf, 0x91, 0x67, 0x52, 0x1e, 0xde, 0x20, 0xa5, 0xcb, 0x98, 0x75, 0xa6, 0x9e, 0x3f, 0x23,
-	0x0c, 0x15, 0x97, 0x38, 0xa6, 0x31, 0x30, 0xbf, 0x34, 0x4e, 0x07, 0xa4, 0x51, 0x58, 0xd7, 0x36,
-	0x8a, 0x7a, 0x88, 0x46, 0xe7, 0x7f, 0x49, 0xae, 0xdd, 0x13, 0xdb, 0x1a, 0x5c, 0x37, 0x8a, 0x8c,
-	0xa1, 0x48, 0x09, 0x1d, 0x6b, 0x70, 0xcd, 0x16, 0xcd, 0x1e, 0x5b, 0x1e, 0xef, 0x2d, 0xb1, 0xde,
-	0x12, 0xa3, 0xb0, 0xee, 0x0d, 0xa8, 0x0f, 0x4d, 0xeb, 0x64, 0x68, 0xf7, 0x4f, 0x7c, 0x87, 0x00,
-	0x73, 0x48, 0x6d, 0x68, 0x5a, 0x2f, 0xec, 0xbe, 0x2e, 0xdd, 0x42, 0x39, 0x8d, 0xab, 0x30, 0x67,
-	0x59, 0x70, 0x1a, 0x57, 0x2a, 0xe7, 0x26, 0x2c, 0x51, 0x99, 0x3d, 0x87, 0x18, 0x1e, 0x09, 0x98,
-	0x2b, 0x8c, 0x79, 0x71, 0x68, 0x5a, 0x3b, 0xac, 0x27, 0xc4, 0x6f, 0x5c, 0xc5, 0xf8, 0xab, 0x82,
-	0xdf, 0xb8, 0x0a, 0xf3, 0xe3, 0x4d, 0x28, 0xf9, 0x3e, 0x47, 0x45, 0x98, 0x3f, 0xec, 0x1c, 0xb6,
-	0xeb, 0x73, 0x08, 0x20, 0xbf, 0x7d, 0xbc, 0xd3, 0x3e, 0xdc, 0xad, 0x6b, 0xa8, 0x0c, 0x85, 0xdd,
-	0x36, 0x6f, 0x64, 0xf0, 0x53, 0x80, 0xc0, 0xbb, 0xa8, 0x00, 0xd9, 0xfd, 0xf6, 0xe7, 0xf5, 0x39,
-	0xca, 0xf3, 0xaa, 0xad, 0x1f, 0xef, 0x75, 0x0e, 0xeb, 0x1a, 0x1d, 0xbc, 0xa3, 0xb7, 0xb7, 0xbb,
-	0xed, 0x7a, 0x86, 0x72, 0xbc, 0xe8, 0xec, 0xd6, 0xb3, 0xa8, 0x04, 0xb9, 0x57, 0xdb, 0x07, 0x2f,
-	0xdb, 0xf5, 0x79, 0xfc, 0x73, 0x0d, 0xaa, 0x62, 0xbd, 0xf8, 0x99, 0x40, 0xdf, 0x81, 0xfc, 0x05,
-	0x3b, 0x17, 0x6c, 0x2b, 0x96, 0xb7, 0xee, 0x46, 0x16, 0x37, 0x74, 0x76, 0x74, 0xc1, 0x8b, 0x30,
-	0x64, 0x2f, 0x27, 0x6e, 0x23, 0xb3, 0x9e, 0xdd, 0x28, 0x6f, 0xd5, 0x37, 0xf9, 0x81, 0xdd, 0xdc,
-	0x27, 0xd7, 0xaf, 0x8c, 0xc1, 0x98, 0xe8, 0xb4, 0x13, 0x21, 0x98, 0x1f, 0xda, 0x0e, 0x61, 0x3b,
-	0xb6, 0xa8, 0xb3, 0x67, 0xba, 0x8d, 0xd9, 0xa2, 0x89, 0xdd, 0xca, 0x1b, 0xf8, 0x17, 0x1a, 0xc0,
-	0xd1, 0xd8, 0x4b, 0x3f, 0x1a, 0xcb, 0x90, 0x9b, 0x50, 0xc1, 0xe2, 0x58, 0xf0, 0x06, 0x3b, 0x13,
-	0xc4, 0x70, 0x89, 0x7f, 0x26, 0x68, 0x03, 0xdd, 0x82, 0xc2, 0xc8, 0x21, 0x93, 0x93, 0xcb, 0x09,
-	0x53, 0x52, 0xd4, 0xf3, 0xb4, 0xb9, 0x3f, 0x41, 0x6f, 0x41, 0xc5, 0x3c, 0xb7, 0x6c, 0x87, 0x9c,
-	0x70, 0x59, 0x39, 0xd6, 0x5b, 0xe6, 0x34, 0x66, 0xb7, 0xc2, 0xc2, 0x05, 0xe7, 0x55, 0x96, 0x03,
-	0x4a, 0xc2, 0x16, 0x94, 0x99, 0xa9, 0x33, 0xb9, 0xef, 0xbd, 0xc0, 0xc6, 0x0c, 0x1b, 0x16, 0x77,
-	0xa1, 0xb0, 0x1a, 0xff, 0x00, 0xd0, 0x2e, 0x19, 0x10, 0x8f, 0xcc, 0x12, 0x3d, 0x14, 0x9f, 0x64,
-	0x55, 0x9f, 0xe0, 0x9f, 0x69, 0xb0, 0x14, 0x12, 0x3f, 0xd3, 0xb4, 0x1a, 0x50, 0xe8, 0x33, 0x61,
-	0xdc, 0x82, 0xac, 0x2e, 0x9b, 0xe8, 0x31, 0x14, 0x85, 0x01, 0x6e, 0x23, 0x9b, 0xb2, 0x69, 0x0a,
-	0xdc, 0x26, 0x17, 0xff, 0x22, 0x03, 0x25, 0x31, 0xd1, 0xce, 0x08, 0x6d, 0x43, 0xd5, 0xe1, 0x8d,
-	0x13, 0x36, 0x1f, 0x61, 0x51, 0x33, 0x3d, 0x08, 0x3d, 0x9f, 0xd3, 0x2b, 0x62, 0x08, 0x23, 0xa3,
-	0xdf, 0x83, 0xb2, 0x14, 0x31, 0x1a, 0x7b, 0xc2, 0xe5, 0x8d, 0xb0, 0x80, 0x60, 0xff, 0x3d, 0x9f,
-	0xd3, 0x41, 0xb0, 0x1f, 0x8d, 0x3d, 0xd4, 0x85, 0x65, 0x39, 0x98, 0xcf, 0x46, 0x98, 0x91, 0x65,
-	0x52, 0xd6, 0xc3, 0x52, 0xe2, 0x4b, 0xf5, 0x7c, 0x4e, 0x47, 0x62, 0xbc, 0xd2, 0xa9, 0x9a, 0xe4,
-	0x5d, 0xf1, 0xe0, 0x1d, 0x33, 0xa9, 0x7b, 0x65, 0xc5, 0x4d, 0xea, 0x5e, 0x59, 0x4f, 0x4b, 0x50,
-	0x10, 0x2d, 0xfc, 0x2f, 0x19, 0x00, 0xb9, 0x1a, 0x9d, 0x11, 0xda, 0x85, 0x9a, 0x23, 0x5a, 0x21,
-	0x6f, 0xdd, 0x49, 0xf4, 0x96, 0x58, 0xc4, 0x39, 0xbd, 0x2a, 0x07, 0x71, 0xe3, 0xbe, 0x07, 0x15,
-	0x5f, 0x4a, 0xe0, 0xb0, 0xdb, 0x09, 0x0e, 0xf3, 0x25, 0x94, 0xe5, 0x00, 0xea, 0xb2, 0x4f, 0x61,
-	0xc5, 0x1f, 0x9f, 0xe0, 0xb3, 0xb7, 0xa6, 0xf8, 0xcc, 0x17, 0xb8, 0x24, 0x25, 0xa8, 0x5e, 0x53,
-	0x0d, 0x0b, 0xdc, 0x76, 0x3b, 0xc1, 0x6d, 0x71, 0xc3, 0xa8, 0xe3, 0x80, 0xe6, 0x4b, 0xde, 0xc4,
-	0xff, 0x97, 0x85, 0xc2, 0x8e, 0x3d, 0x1c, 0x19, 0x0e, 0x5d, 0x8d, 0xbc, 0x43, 0xdc, 0xf1, 0xc0,
-	0x63, 0xee, 0xaa, 0x6d, 0x3d, 0x08, 0x4b, 0x14, 0x6c, 0xf2, 0xbf, 0xce, 0x58, 0x75, 0x31, 0x84,
-	0x0e, 0x16, 0xe9, 0x31, 0xf3, 0x06, 0x83, 0x45, 0x72, 0x14, 0x43, 0xe4, 0x41, 0xce, 0x06, 0x07,
-	0xb9, 0x09, 0x85, 0x09, 0x71, 0x82, 0x94, 0xfe, 0x7c, 0x4e, 0x97, 0x04, 0xf4, 0x1e, 0x2c, 0x44,
-	0xd3, 0x4b, 0x4e, 0xf0, 0xd4, 0x7a, 0xe1, 0x6c, 0xf4, 0x00, 0x2a, 0xa1, 0x1c, 0x97, 0x17, 0x7c,
-	0xe5, 0xa1, 0x92, 0xe2, 0x56, 0x65, 0x5c, 0xa5, 0xf9, 0xb8, 0xf2, 0x7c, 0x4e, 0x46, 0xd6, 0x55,
-	0x19, 0x59, 0x8b, 0x62, 0x94, 0x88, 0xad, 0xa1, 0x20, 0xf3, 0xfd, 0x70, 0x90, 0xc1, 0xdf, 0x87,
-	0x6a, 0xc8, 0x41, 0x34, 0xef, 0xb4, 0x3f, 0x79, 0xb9, 0x7d, 0xc0, 0x93, 0xd4, 0x33, 0x96, 0x97,
-	0xf4, 0xba, 0x46, 0x73, 0xdd, 0x41, 0xfb, 0xf8, 0xb8, 0x9e, 0x41, 0x55, 0x28, 0x1d, 0x76, 0xba,
-	0x27, 0x9c, 0x2b, 0x8b, 0x9f, 0xf9, 0x12, 0x44, 0x92, 0x53, 0x72, 0xdb, 0x9c, 0x92, 0xdb, 0x34,
-	0x99, 0xdb, 0x32, 0x41, 0x6e, 0x63, 0x69, 0xee, 0xa0, 0xbd, 0x7d, 0xdc, 0xae, 0xcf, 0x3f, 0xad,
-	0x41, 0x85, 0xfb, 0xf7, 0x64, 0x6c, 0xd1, 0x54, 0xfb, 0x0f, 0x1a, 0x40, 0x70, 0x9a, 0x50, 0x0b,
-	0x0a, 0x3d, 0xae, 0xa7, 0xa1, 0xb1, 0x60, 0xb4, 0x92, 0xb8, 0x64, 0xba, 0xe4, 0x42, 0xdf, 0x82,
-	0x82, 0x3b, 0xee, 0xf5, 0x88, 0x2b, 0x53, 0xde, 0xad, 0x68, 0x3c, 0x14, 0xd1, 0x4a, 0x97, 0x7c,
-	0x74, 0xc8, 0x99, 0x61, 0x0e, 0xc6, 0x2c, 0x01, 0x4e, 0x1f, 0x22, 0xf8, 0xf0, 0xdf, 0x69, 0x50,
-	0x56, 0x36, 0xef, 0x6f, 0x18, 0x84, 0xef, 0x42, 0x89, 0xd9, 0x40, 0xfa, 0x22, 0x0c, 0x17, 0xf5,
-	0x80, 0x80, 0x7e, 0x07, 0x4a, 0xf2, 0x04, 0xc8, 0x48, 0xdc, 0x48, 0x16, 0xdb, 0x19, 0xe9, 0x01,
-	0x2b, 0xde, 0x87, 0x45, 0xe6, 0x95, 0x1e, 0x2d, 0xae, 0xa5, 0x1f, 0xd5, 0xf2, 0x53, 0x8b, 0x94,
-	0x9f, 0x4d, 0x28, 0x8e, 0x2e, 0xae, 0x5d, 0xb3, 0x67, 0x0c, 0x84, 0x15, 0x7e, 0x1b, 0x7f, 0x0c,
-	0x48, 0x15, 0x36, 0xcb, 0x74, 0x71, 0x15, 0xca, 0xcf, 0x0d, 0xf7, 0x42, 0x98, 0x84, 0x1f, 0x43,
-	0x95, 0x36, 0xf7, 0x5f, 0xbd, 0x81, 0x8d, 0xec, 0xe5, 0x40, 0x72, 0xcf, 0xe4, 0x73, 0x04, 0xf3,
-	0x17, 0x86, 0x7b, 0xc1, 0x26, 0x5a, 0xd5, 0xd9, 0x33, 0x7a, 0x0f, 0xea, 0x3d, 0x3e, 0xc9, 0x93,
-	0xc8, 0x2b, 0xc3, 0x82, 0xa0, 0xfb, 0x95, 0xe0, 0x67, 0x50, 0xe1, 0x73, 0xf8, 0x6d, 0x1b, 0x81,
-	0x17, 0x61, 0xe1, 0xd8, 0x32, 0x46, 0xee, 0x85, 0x2d, 0xb3, 0x1b, 0x9d, 0x74, 0x3d, 0xa0, 0xcd,
-	0xa4, 0xf1, 0x5d, 0x58, 0x70, 0xc8, 0xd0, 0x30, 0x2d, 0xd3, 0x3a, 0x3f, 0x39, 0xbd, 0xf6, 0x88,
-	0x2b, 0x5e, 0x98, 0x6a, 0x3e, 0xf9, 0x29, 0xa5, 0x52, 0xd3, 0x4e, 0x07, 0xf6, 0xa9, 0x08, 0x73,
-	0xec, 0x19, 0xff, 0x34, 0x03, 0x95, 0x4f, 0x0d, 0xaf, 0x27, 0x97, 0x0e, 0xed, 0x41, 0xcd, 0x0f,
-	0x6e, 0x8c, 0x22, 0x6c, 0x89, 0xa4, 0x58, 0x36, 0x46, 0x96, 0xd2, 0x32, 0x3b, 0x56, 0x7b, 0x2a,
-	0x81, 0x89, 0x32, 0xac, 0x1e, 0x19, 0xf8, 0xa2, 0x32, 0xe9, 0xa2, 0x18, 0xa3, 0x2a, 0x4a, 0x25,
-	0xa0, 0x0e, 0xd4, 0x47, 0x8e, 0x7d, 0xee, 0x10, 0xd7, 0xf5, 0x85, 0xf1, 0x34, 0x86, 0x13, 0x84,
-	0x1d, 0x09, 0xd6, 0x40, 0xdc, 0xc2, 0x28, 0x4c, 0x7a, 0xba, 0x10, 0xd4, 0x33, 0x3c, 0x38, 0xfd,
-	0x57, 0x06, 0x50, 0x7c, 0x52, 0xbf, 0x6e, 0x89, 0xf7, 0x10, 0x6a, 0xae, 0x67, 0x38, 0xb1, 0xcd,
-	0x56, 0x65, 0x54, 0x3f, 0xe2, 0xbf, 0x0b, 0xbe, 0x41, 0x27, 0x96, 0xed, 0x99, 0x67, 0xd7, 0xa2,
-	0x4a, 0xae, 0x49, 0xf2, 0x21, 0xa3, 0xa2, 0x36, 0x14, 0xce, 0xcc, 0x81, 0x47, 0x1c, 0xb7, 0x91,
-	0x5b, 0xcf, 0x6e, 0xd4, 0xb6, 0x1e, 0xdf, 0xb4, 0x0c, 0x9b, 0x1f, 0x31, 0xfe, 0xee, 0xf5, 0x88,
-	0xe8, 0x72, 0xac, 0x5a, 0x79, 0xe6, 0x43, 0xd5, 0xf8, 0x6d, 0x28, 0xbe, 0xa6, 0x22, 0xe8, 0x5b,
-	0x76, 0x81, 0x17, 0x8b, 0xac, 0xcd, 0x5f, 0xb2, 0xcf, 0x1c, 0xe3, 0x7c, 0x48, 0x2c, 0x4f, 0xbe,
-	0x07, 0xca, 0x36, 0x7e, 0x08, 0x10, 0xa8, 0xa1, 0x21, 0xff, 0xb0, 0x73, 0xf4, 0xb2, 0x5b, 0x9f,
-	0x43, 0x15, 0x28, 0x1e, 0x76, 0x76, 0xdb, 0x07, 0x6d, 0x9a, 0x1f, 0x70, 0x4b, 0xba, 0x34, 0xb4,
-	0x96, 0xaa, 0x4e, 0x2d, 0xa4, 0x13, 0xaf, 0xc2, 0x72, 0xd2, 0x02, 0xd2, 0x5a, 0xb4, 0x2a, 0x76,
-	0xe9, 0x4c, 0x47, 0x45, 0x55, 0x9d, 0x09, 0x4f, 0xb7, 0x01, 0x05, 0xbe, 0x7b, 0xfb, 0xa2, 0x38,
-	0x97, 0x4d, 0xea, 0x08, 0xbe, 0x19, 0x49, 0x5f, 0xac, 0x92, 0xdf, 0x4e, 0x0c, 0x2f, 0xb9, 0xc4,
-	0xf0, 0x82, 0x1e, 0x40, 0xd5, 0x3f, 0x0d, 0x86, 0x2b, 0x6a, 0x81, 0x92, 0x5e, 0x91, 0x1b, 0x9d,
-	0xd2, 0x42, 0x4e, 0x2f, 0x84, 0x9d, 0x8e, 0x1e, 0x42, 0x9e, 0x4c, 0x88, 0xe5, 0xb9, 0x8d, 0x32,
-	0xcb, 0x18, 0x55, 0x59, 0xbb, 0xb7, 0x29, 0x55, 0x17, 0x9d, 0xf8, 0xbb, 0xb0, 0xc8, 0xde, 0x91,
-	0x9e, 0x39, 0x86, 0xa5, 0xbe, 0xcc, 0x75, 0xbb, 0x07, 0xc2, 0xdd, 0xf4, 0x11, 0xd5, 0x20, 0xb3,
-	0xb7, 0x2b, 0x9c, 0x90, 0xd9, 0xdb, 0xc5, 0x3f, 0xd6, 0x00, 0xa9, 0xe3, 0x66, 0xf2, 0x73, 0x44,
-	0xb8, 0x54, 0x9f, 0x0d, 0xd4, 0x2f, 0x43, 0x8e, 0x38, 0x8e, 0xed, 0x30, 0x8f, 0x96, 0x74, 0xde,
-	0xc0, 0x6f, 0x0b, 0x1b, 0x74, 0x32, 0xb1, 0x2f, 0xfd, 0x33, 0xc8, 0xa5, 0x69, 0xbe, 0xa9, 0xfb,
-	0xb0, 0x14, 0xe2, 0x9a, 0x29, 0x73, 0x7d, 0x04, 0x0b, 0x4c, 0xd8, 0xce, 0x05, 0xe9, 0x5d, 0x8e,
-	0x6c, 0xd3, 0x8a, 0xe9, 0xa3, 0x2b, 0x17, 0x04, 0x58, 0x3a, 0x0f, 0x3e, 0xb1, 0x8a, 0x4f, 0xec,
-	0x76, 0x0f, 0xf0, 0xe7, 0xb0, 0x1a, 0x91, 0x23, 0xcd, 0xff, 0x43, 0x28, 0xf7, 0x7c, 0xa2, 0x2b,
-	0x6a, 0x9d, 0x7b, 0x61, 0xe3, 0xa2, 0x43, 0xd5, 0x11, 0xb8, 0x03, 0xb7, 0x62, 0xa2, 0x67, 0x9a,
-	0xf3, 0xbb, 0xb0, 0xc2, 0x04, 0xee, 0x13, 0x32, 0xda, 0x1e, 0x98, 0x93, 0x54, 0x4f, 0x8f, 0xc4,
-	0xa4, 0x14, 0xc6, 0xaf, 0x77, 0x5f, 0xe0, 0xdf, 0x17, 0x1a, 0xbb, 0xe6, 0x90, 0x74, 0xed, 0x83,
-	0x74, 0xdb, 0x68, 0x36, 0xbb, 0x24, 0xd7, 0xae, 0x28, 0x6b, 0xd8, 0x33, 0xfe, 0x47, 0x4d, 0xb8,
-	0x4a, 0x1d, 0xfe, 0x35, 0xef, 0xe4, 0x35, 0x80, 0x73, 0x7a, 0x64, 0x48, 0x9f, 0x76, 0x70, 0x44,
-	0x45, 0xa1, 0xf8, 0x76, 0xd2, 0xf8, 0x5d, 0x11, 0x76, 0x2e, 0x8b, 0x7d, 0xce, 0xfe, 0xf8, 0x51,
-	0xee, 0x1e, 0x94, 0x19, 0xe1, 0xd8, 0x33, 0xbc, 0xb1, 0x1b, 0x5b, 0x8c, 0x3f, 0x17, 0xdb, 0x5e,
-	0x0e, 0x9a, 0x69, 0x5e, 0xdf, 0x82, 0x3c, 0x7b, 0x99, 0x90, 0xa5, 0xf4, 0xed, 0x84, 0xfd, 0xc8,
-	0xed, 0xd0, 0x05, 0x23, 0xfe, 0xa9, 0x06, 0xf9, 0x17, 0x0c, 0x82, 0x55, 0x4c, 0x9b, 0x97, 0x6b,
-	0x61, 0x19, 0x43, 0x0e, 0x0c, 0x95, 0x74, 0xf6, 0xcc, 0x4a, 0x4f, 0x42, 0x9c, 0x97, 0xfa, 0x01,
-	0x2f, 0x71, 0x4b, 0xba, 0xdf, 0xa6, 0x3e, 0xeb, 0x0d, 0x4c, 0x62, 0x79, 0xac, 0x77, 0x9e, 0xf5,
-	0x2a, 0x14, 0x5a, 0x3d, 0x9b, 0xee, 0x01, 0x31, 0x1c, 0x4b, 0x80, 0xa6, 0x45, 0x3d, 0x20, 0xe0,
-	0x03, 0xa8, 0x73, 0x3b, 0xb6, 0xfb, 0x7d, 0xa5, 0xc0, 0xf4, 0xb5, 0x69, 0x11, 0x6d, 0x21, 0x69,
-	0x99, 0xa8, 0xb4, 0x7f, 0xd2, 0x60, 0x51, 0x11, 0x37, 0x93, 0x57, 0xdf, 0x87, 0x3c, 0x07, 0xa9,
-	0x45, 0xa5, 0xb3, 0x1c, 0x1e, 0xc5, 0xd5, 0xe8, 0x82, 0x07, 0x6d, 0x42, 0x81, 0x3f, 0xc9, 0x77,
-	0x80, 0x64, 0x76, 0xc9, 0x84, 0x1f, 0xc2, 0x92, 0x20, 0x91, 0xa1, 0x9d, 0x74, 0x30, 0xd8, 0x62,
-	0xe0, 0x3f, 0x85, 0xe5, 0x30, 0xdb, 0x4c, 0x53, 0x52, 0x8c, 0xcc, 0xbc, 0x89, 0x91, 0xdb, 0xd2,
-	0xc8, 0x97, 0xa3, 0xbe, 0x52, 0x47, 0x45, 0x77, 0x8c, 0xba, 0x5e, 0x99, 0xf0, 0x7a, 0x05, 0x13,
-	0x90, 0x22, 0xbe, 0xd1, 0x09, 0x2c, 0xc9, 0xed, 0x70, 0x60, 0xba, 0x7e, 0xb9, 0xfe, 0x25, 0x20,
-	0x95, 0xf8, 0x8d, 0x1a, 0xf4, 0x8e, 0x74, 0xc7, 0x91, 0x63, 0x0f, 0xed, 0x54, 0x97, 0xe2, 0x3f,
-	0x83, 0x95, 0x08, 0xdf, 0x37, 0xed, 0xb7, 0x5d, 0x22, 0x8b, 0x15, 0xe9, 0xb7, 0x8f, 0x01, 0xa9,
-	0xc4, 0x99, 0xb2, 0x56, 0x0b, 0x16, 0x5f, 0xd8, 0x13, 0x1a, 0xfe, 0x28, 0x35, 0x38, 0xf7, 0x1c,
-	0x63, 0xf0, 0x5d, 0xe1, 0xb7, 0xa9, 0x72, 0x75, 0xc0, 0x4c, 0xca, 0xff, 0x43, 0x83, 0xca, 0xf6,
-	0xc0, 0x70, 0x86, 0x52, 0xf1, 0xf7, 0x20, 0xcf, 0xdf, 0x9c, 0x05, 0x58, 0xf5, 0x4e, 0x58, 0x8c,
-	0xca, 0xcb, 0x1b, 0xdb, 0xfc, 0x3d, 0x5b, 0x8c, 0xa2, 0x86, 0x8b, 0xef, 0x59, 0xbb, 0x91, 0xef,
-	0x5b, 0xbb, 0xe8, 0x03, 0xc8, 0x19, 0x74, 0x08, 0x4b, 0x33, 0xb5, 0x28, 0x66, 0xc1, 0xa4, 0xb1,
-	0xfa, 0x9e, 0x73, 0xe1, 0xef, 0x40, 0x59, 0xd1, 0x80, 0x0a, 0x90, 0x7d, 0xd6, 0x16, 0xc5, 0xf8,
-	0xf6, 0x4e, 0x77, 0xef, 0x15, 0x07, 0x6b, 0x6a, 0x00, 0xbb, 0x6d, 0xbf, 0x9d, 0xc1, 0x9f, 0x89,
-	0x51, 0x22, 0xa4, 0xab, 0xf6, 0x68, 0x69, 0xf6, 0x64, 0xde, 0xc8, 0x9e, 0x2b, 0xa8, 0x8a, 0xe9,
-	0xcf, 0x9a, 0xa2, 0x98, 0xbc, 0x94, 0x14, 0xa5, 0x18, 0xaf, 0x0b, 0x46, 0xbc, 0x00, 0x55, 0x91,
-	0xb4, 0xc4, 0xfe, 0xfb, 0xf7, 0x0c, 0xd4, 0x24, 0x65, 0x56, 0x50, 0x5d, 0xe2, 0x81, 0x3c, 0xc9,
-	0xf9, 0x68, 0xe0, 0x2a, 0xe4, 0xfb, 0xa7, 0xc7, 0xe6, 0x97, 0xf2, 0x03, 0x88, 0x68, 0x51, 0xfa,
-	0x80, 0xeb, 0xe1, 0x5f, 0x21, 0x45, 0x8b, 0x66, 0x23, 0xc7, 0x38, 0xf3, 0xf6, 0xac, 0x3e, 0xb9,
-	0x62, 0xb9, 0x6d, 0x5e, 0x0f, 0x08, 0x0c, 0x28, 0x11, 0x5f, 0x2b, 0xd9, 0x0b, 0x82, 0xf2, 0xf5,
-	0x12, 0x3d, 0x82, 0x3a, 0x7d, 0xde, 0x1e, 0x8d, 0x06, 0x26, 0xe9, 0x73, 0x01, 0x05, 0xc6, 0x13,
-	0xa3, 0x53, 0xed, 0xac, 0xa4, 0x76, 0x1b, 0x45, 0x16, 0x5d, 0x45, 0x0b, 0xad, 0x43, 0x99, 0xdb,
-	0xb7, 0x67, 0xbd, 0x74, 0x09, 0xfb, 0x84, 0x97, 0xd5, 0x55, 0x52, 0x38, 0x5b, 0x42, 0x34, 0x5b,
-	0x2e, 0xc1, 0xe2, 0xf6, 0xd8, 0xbb, 0x68, 0x5b, 0xc6, 0xe9, 0x40, 0x46, 0x22, 0x5a, 0xce, 0x50,
-	0xe2, 0xae, 0xe9, 0xaa, 0xd4, 0x36, 0x2c, 0x51, 0x2a, 0xb1, 0x3c, 0xb3, 0xa7, 0x64, 0x02, 0x59,
-	0x2b, 0x68, 0x91, 0x5a, 0xc1, 0x70, 0xdd, 0xd7, 0xb6, 0xd3, 0x17, 0xee, 0xf5, 0xdb, 0x78, 0xc2,
-	0x85, 0xbf, 0x74, 0x43, 0xf9, 0xfe, 0xd7, 0x94, 0x82, 0x3e, 0x84, 0x82, 0x3d, 0x62, 0x9f, 0xa4,
-	0x05, 0x6e, 0xb0, 0xba, 0xc9, 0x3f, 0x62, 0x6f, 0x0a, 0xc1, 0x1d, 0xde, 0xab, 0x4b, 0x36, 0xbc,
-	0x11, 0xe8, 0x7d, 0x46, 0xbc, 0x29, 0x7a, 0xf1, 0x63, 0x58, 0x91, 0x9c, 0x02, 0x26, 0x9f, 0xc2,
-	0xdc, 0x81, 0x7b, 0x92, 0x79, 0xe7, 0xc2, 0xb0, 0xce, 0xc9, 0x91, 0x30, 0xf1, 0x37, 0xf5, 0xcf,
-	0x53, 0x68, 0xf8, 0x76, 0xb2, 0x57, 0x37, 0x7b, 0xa0, 0x1a, 0x30, 0x76, 0xc5, 0x4e, 0x2f, 0xe9,
-	0xec, 0x99, 0xd2, 0x1c, 0x7b, 0xe0, 0xd7, 0x6a, 0xf4, 0x19, 0xef, 0xc0, 0x6d, 0x29, 0x43, 0xbc,
-	0x54, 0x85, 0x85, 0xc4, 0x0c, 0x4a, 0x12, 0x22, 0x1c, 0x46, 0x87, 0x4e, 0x5f, 0x28, 0x95, 0x33,
-	0xec, 0x5a, 0x26, 0x53, 0x53, 0x64, 0xae, 0xf0, 0x3d, 0x44, 0x0d, 0x53, 0xd3, 0xb1, 0x20, 0x53,
-	0x01, 0x2a, 0x59, 0x2c, 0x04, 0x25, 0xc7, 0x16, 0x22, 0x26, 0xfa, 0x07, 0xb0, 0xe6, 0x1b, 0x41,
-	0xfd, 0x76, 0x44, 0x9c, 0xa1, 0xe9, 0xba, 0x0a, 0xb0, 0x9a, 0x34, 0xf1, 0x77, 0x60, 0x7e, 0x44,
-	0x44, 0x24, 0x2c, 0x6f, 0x21, 0xb9, 0x89, 0x94, 0xc1, 0xac, 0x1f, 0xf7, 0xe1, 0xbe, 0x94, 0xce,
-	0x3d, 0x9a, 0x28, 0x3e, 0x6a, 0x94, 0x84, 0x9b, 0x32, 0x29, 0x70, 0x53, 0x36, 0x02, 0xf6, 0x7f,
-	0xcc, 0x1d, 0x29, 0x4f, 0xe3, 0x4c, 0x19, 0x6e, 0x9f, 0xfb, 0xd4, 0x3f, 0xc4, 0x33, 0x09, 0x3b,
-	0x85, 0xe5, 0xf0, 0xd9, 0x9f, 0x29, 0xf8, 0x2e, 0x43, 0xce, 0xb3, 0x2f, 0x89, 0x0c, 0xbd, 0xbc,
-	0x21, 0x0d, 0xf6, 0x03, 0xc3, 0x4c, 0x06, 0x1b, 0x81, 0x30, 0xb6, 0x25, 0x67, 0xb5, 0x97, 0xae,
-	0xa6, 0xac, 0x6c, 0x79, 0x03, 0x1f, 0xc2, 0x6a, 0x34, 0x4c, 0xcc, 0x64, 0xf2, 0x2b, 0xbe, 0x81,
-	0x93, 0x22, 0xc9, 0x4c, 0x72, 0x3f, 0x09, 0x82, 0x81, 0x12, 0x50, 0x66, 0x12, 0xa9, 0x43, 0x33,
-	0x29, 0xbe, 0xfc, 0x36, 0xf6, 0xab, 0x1f, 0x6e, 0x66, 0x12, 0xe6, 0x06, 0xc2, 0x66, 0x5f, 0xfe,
-	0x20, 0x46, 0x64, 0xa7, 0xc6, 0x08, 0x71, 0x48, 0x82, 0x28, 0xf6, 0x35, 0x6c, 0x3a, 0xa1, 0x23,
-	0x08, 0xa0, 0xb3, 0xea, 0xa0, 0x39, 0xc4, 0xd7, 0xc1, 0x1a, 0x72, 0x63, 0xab, 0x61, 0x77, 0xa6,
-	0xc5, 0xf8, 0x34, 0x88, 0x9d, 0xb1, 0xc8, 0x3c, 0x93, 0xe0, 0xcf, 0x60, 0x3d, 0x3d, 0x28, 0xcf,
-	0x22, 0xf9, 0x51, 0x0b, 0x4a, 0x7e, 0x19, 0xac, 0xdc, 0x22, 0x2a, 0x43, 0xe1, 0xb0, 0x73, 0x7c,
-	0xb4, 0xbd, 0xd3, 0xe6, 0xd7, 0x88, 0x76, 0x3a, 0xba, 0xfe, 0xf2, 0xa8, 0x5b, 0xcf, 0x6c, 0xfd,
-	0x32, 0x0b, 0x99, 0xfd, 0x57, 0xe8, 0x73, 0xc8, 0xf1, 0x6f, 0xea, 0x53, 0x2e, 0x52, 0x34, 0xa7,
-	0x5d, 0x1b, 0xc0, 0xb7, 0x7e, 0xfc, 0xdf, 0xbf, 0xfc, 0x79, 0x66, 0x11, 0x57, 0x5a, 0x93, 0x6f,
-	0xb7, 0x2e, 0x27, 0x2d, 0x96, 0x1b, 0x9e, 0x68, 0x8f, 0xd0, 0x27, 0x90, 0x3d, 0x1a, 0x7b, 0x28,
-	0xf5, 0x82, 0x45, 0x33, 0xfd, 0x26, 0x01, 0x5e, 0x61, 0x42, 0x17, 0x30, 0x08, 0xa1, 0xa3, 0xb1,
-	0x47, 0x45, 0xfe, 0x10, 0xca, 0xea, 0x3d, 0x80, 0x1b, 0x6f, 0x5d, 0x34, 0x6f, 0xbe, 0x63, 0x80,
-	0xef, 0x31, 0x55, 0xb7, 0x30, 0x12, 0xaa, 0xf8, 0x4d, 0x05, 0x75, 0x16, 0xdd, 0x2b, 0x0b, 0xa5,
-	0xde, 0xc9, 0x68, 0xa6, 0x5f, 0x3b, 0x88, 0xcd, 0xc2, 0xbb, 0xb2, 0xa8, 0xc8, 0x3f, 0x16, 0x37,
-	0x0e, 0x7a, 0x1e, 0xba, 0x9f, 0xf0, 0xc5, 0x59, 0xfd, 0xb6, 0xda, 0x5c, 0x4f, 0x67, 0x10, 0x4a,
-	0xee, 0x32, 0x25, 0xab, 0x78, 0x51, 0x28, 0xe9, 0xf9, 0x2c, 0x4f, 0xb4, 0x47, 0x5b, 0x3d, 0xc8,
-	0xb1, 0xef, 0x16, 0xe8, 0x0b, 0xf9, 0xd0, 0x4c, 0xf8, 0x80, 0x93, 0xb2, 0xd0, 0xa1, 0x2f, 0x1e,
-	0x78, 0x99, 0x29, 0xaa, 0xe1, 0x12, 0x55, 0xc4, 0xbe, 0x5a, 0x3c, 0xd1, 0x1e, 0x6d, 0x68, 0x1f,
-	0x6a, 0x5b, 0xff, 0x9c, 0x83, 0x1c, 0x03, 0xec, 0xd0, 0x25, 0x40, 0x80, 0xe1, 0x47, 0x67, 0x17,
-	0xfb, 0x2a, 0x10, 0x9d, 0x5d, 0x1c, 0xfe, 0xc7, 0x4d, 0xa6, 0x74, 0x19, 0x2f, 0x50, 0xa5, 0x0c,
-	0x07, 0x6c, 0x31, 0x68, 0x93, 0xfa, 0xf1, 0xaf, 0x34, 0x81, 0x57, 0xf2, 0xb3, 0x84, 0x92, 0xa4,
-	0x85, 0x80, 0xfc, 0xe8, 0x76, 0x48, 0x00, 0xf1, 0xf1, 0x77, 0x99, 0xc2, 0x16, 0xae, 0x07, 0x0a,
-	0x1d, 0xc6, 0xf1, 0x44, 0x7b, 0xf4, 0x45, 0x03, 0x2f, 0x09, 0x2f, 0x47, 0x7a, 0xd0, 0x8f, 0xa0,
-	0x16, 0x06, 0xaa, 0xd1, 0x83, 0x04, 0x5d, 0x51, 0xbc, 0xbb, 0xf9, 0xf6, 0x74, 0x26, 0x61, 0xd3,
-	0x1a, 0xb3, 0x49, 0x28, 0xe7, 0x9a, 0x2f, 0x09, 0x19, 0x19, 0x94, 0x49, 0xac, 0x01, 0xfa, 0x7b,
-	0x4d, 0x7c, 0x47, 0x08, 0x90, 0x67, 0x94, 0x24, 0x3d, 0x86, 0x6b, 0x37, 0x1f, 0xde, 0xc0, 0x25,
-	0x8c, 0xf8, 0x03, 0x66, 0xc4, 0xef, 0xe2, 0xe5, 0xc0, 0x08, 0xcf, 0x1c, 0x12, 0xcf, 0x16, 0x56,
-	0x7c, 0x71, 0x17, 0xdf, 0x0a, 0x39, 0x27, 0xd4, 0x1b, 0x2c, 0x16, 0x47, 0x8f, 0x13, 0x17, 0x2b,
-	0x84, 0x46, 0x27, 0x2e, 0x56, 0x18, 0x7a, 0x4e, 0x5a, 0x2c, 0x8e, 0x15, 0x27, 0x2d, 0x96, 0xdf,
-	0xb3, 0xf5, 0xff, 0xf3, 0x50, 0xd8, 0xe1, 0x37, 0x7d, 0x91, 0x0d, 0x25, 0x1f, 0x7c, 0x45, 0x6b,
-	0x49, 0x08, 0x53, 0xf0, 0x2e, 0xd1, 0xbc, 0x9f, 0xda, 0x2f, 0x0c, 0x7a, 0x8b, 0x19, 0x74, 0x07,
-	0xaf, 0x52, 0xcd, 0xe2, 0x32, 0x71, 0x8b, 0xc3, 0x18, 0x2d, 0xa3, 0xdf, 0xa7, 0x8e, 0xf8, 0x13,
-	0xa8, 0xa8, 0xe8, 0x28, 0x7a, 0x2b, 0x11, 0xd5, 0x52, 0x01, 0xd6, 0x26, 0x9e, 0xc6, 0x22, 0x34,
-	0xbf, 0xcd, 0x34, 0xaf, 0xe1, 0xdb, 0x09, 0x9a, 0x1d, 0xc6, 0x1a, 0x52, 0xce, 0x91, 0xcd, 0x64,
-	0xe5, 0x21, 0xe0, 0x34, 0x59, 0x79, 0x18, 0x18, 0x9d, 0xaa, 0x7c, 0xcc, 0x58, 0xa9, 0x72, 0x17,
-	0x20, 0xc0, 0x30, 0x51, 0xa2, 0x2f, 0x95, 0x97, 0xa9, 0x68, 0x70, 0x88, 0xc3, 0x9f, 0x18, 0x33,
-	0xb5, 0x62, 0xdf, 0x45, 0xd4, 0x0e, 0x4c, 0xd7, 0xe3, 0x07, 0xb3, 0x1a, 0x02, 0x25, 0x51, 0xe2,
-	0x7c, 0xc2, 0xc8, 0x66, 0xf3, 0xc1, 0x54, 0x1e, 0xa1, 0xfd, 0x21, 0xd3, 0x7e, 0x1f, 0x37, 0x13,
-	0xb4, 0x8f, 0x38, 0x2f, 0xdd, 0x6c, 0x7f, 0x9d, 0x87, 0xf2, 0x0b, 0xc3, 0xb4, 0x3c, 0x62, 0x19,
-	0x56, 0x8f, 0xa0, 0x53, 0xc8, 0xb1, 0x4c, 0x1d, 0x0d, 0xc4, 0x2a, 0x60, 0x17, 0x0d, 0xc4, 0x21,
-	0x34, 0x0b, 0xaf, 0x33, 0xc5, 0x4d, 0xbc, 0x42, 0x15, 0x0f, 0x03, 0xd1, 0x2d, 0x06, 0x42, 0xd1,
-	0x49, 0x9f, 0x41, 0x5e, 0x7c, 0xc3, 0x89, 0x08, 0x0a, 0x81, 0x53, 0xcd, 0xbb, 0xc9, 0x9d, 0x49,
-	0x7b, 0x59, 0x55, 0xe3, 0x32, 0x3e, 0xaa, 0x67, 0x02, 0x10, 0xa0, 0xab, 0xd1, 0x15, 0x8d, 0x81,
-	0xb1, 0xcd, 0xf5, 0x74, 0x86, 0x24, 0x9f, 0xaa, 0x3a, 0xfb, 0x3e, 0x2f, 0xd5, 0xfb, 0x47, 0x30,
-	0xff, 0xdc, 0x70, 0x2f, 0x50, 0x24, 0xf7, 0x2a, 0x37, 0x80, 0x9a, 0xcd, 0xa4, 0x2e, 0xa1, 0xe5,
-	0x3e, 0xd3, 0x72, 0x9b, 0x87, 0x32, 0x55, 0xcb, 0x85, 0xe1, 0xd2, 0xa4, 0x86, 0xfa, 0x90, 0xe7,
-	0x17, 0x82, 0xa2, 0xfe, 0x0b, 0x5d, 0x2a, 0x8a, 0xfa, 0x2f, 0x7c, 0x87, 0xe8, 0x66, 0x2d, 0x23,
-	0x28, 0xca, 0x1b, 0x38, 0x28, 0xf2, 0x39, 0x36, 0x72, 0x5b, 0xa7, 0xb9, 0x96, 0xd6, 0x2d, 0x74,
-	0x3d, 0x60, 0xba, 0xee, 0xe1, 0x46, 0x6c, 0xad, 0x04, 0xe7, 0x13, 0xed, 0xd1, 0x87, 0x1a, 0xfa,
-	0x11, 0x40, 0x00, 0x48, 0xc7, 0x4e, 0x60, 0x14, 0xdb, 0x8e, 0x9d, 0xc0, 0x18, 0x96, 0x8d, 0x37,
-	0x99, 0xde, 0x0d, 0xfc, 0x20, 0xaa, 0xd7, 0x73, 0x0c, 0xcb, 0x3d, 0x23, 0xce, 0x07, 0x1c, 0x74,
-	0x74, 0x2f, 0xcc, 0x11, 0x3d, 0x0c, 0xff, 0xba, 0x00, 0xf3, 0xb4, 0x02, 0xa6, 0x85, 0x42, 0x00,
-	0x1c, 0x44, 0x2d, 0x89, 0x01, 0x7c, 0x51, 0x4b, 0xe2, 0x98, 0x43, 0xb8, 0x50, 0x60, 0xbf, 0x11,
-	0x21, 0x8c, 0x81, 0x3a, 0xda, 0x86, 0xb2, 0x82, 0x2c, 0xa0, 0x04, 0x61, 0x61, 0xe4, 0x30, 0x9a,
-	0x7a, 0x12, 0x60, 0x09, 0x7c, 0x87, 0xe9, 0x5b, 0xe1, 0xa9, 0x87, 0xe9, 0xeb, 0x73, 0x0e, 0xaa,
-	0xf0, 0x35, 0x54, 0x54, 0xf4, 0x01, 0x25, 0xc8, 0x8b, 0xa0, 0x92, 0xd1, 0x30, 0x9b, 0x04, 0x5e,
-	0x84, 0x0f, 0xbe, 0xff, 0x3b, 0x18, 0xc9, 0x46, 0x15, 0x0f, 0xa0, 0x20, 0xe0, 0x88, 0xa4, 0x59,
-	0x86, 0x21, 0xcc, 0xa4, 0x59, 0x46, 0xb0, 0x8c, 0x70, 0x71, 0xc9, 0x34, 0xd2, 0x37, 0x2e, 0x99,
-	0xca, 0x84, 0xb6, 0x67, 0xc4, 0x4b, 0xd3, 0x16, 0xa0, 0x6b, 0x69, 0xda, 0x94, 0xb7, 0xdd, 0x34,
-	0x6d, 0xe7, 0xc4, 0x13, 0xc7, 0x45, 0xbe, 0x45, 0xa2, 0x14, 0x61, 0x6a, 0xfa, 0xc0, 0xd3, 0x58,
-	0x92, 0x6a, 0xff, 0x40, 0xa1, 0xcc, 0x1d, 0x57, 0x00, 0x01, 0x58, 0x12, 0x2d, 0xe8, 0x12, 0x11,
-	0xd7, 0x68, 0x41, 0x97, 0x8c, 0xb7, 0x84, 0x43, 0x43, 0xa0, 0x97, 0xbf, 0x7a, 0x50, 0xcd, 0x3f,
-	0xd3, 0x00, 0xc5, 0x71, 0x15, 0xf4, 0x38, 0x59, 0x7a, 0x22, 0x8e, 0xdb, 0x7c, 0xff, 0xcd, 0x98,
-	0x93, 0xa2, 0x7d, 0x60, 0x52, 0x8f, 0x71, 0x8f, 0x5e, 0x53, 0xa3, 0xfe, 0x42, 0x83, 0x6a, 0x08,
-	0x94, 0x41, 0xef, 0xa4, 0xac, 0x69, 0x04, 0x06, 0x6e, 0xbe, 0x7b, 0x23, 0x5f, 0x52, 0xa5, 0xab,
-	0xec, 0x00, 0x59, 0xf2, 0xff, 0x44, 0x83, 0x5a, 0x18, 0xc4, 0x41, 0x29, 0xb2, 0x63, 0x30, 0x72,
-	0x73, 0xe3, 0x66, 0xc6, 0xe9, 0xcb, 0x13, 0x54, 0xfb, 0x03, 0x28, 0x08, 0xd8, 0x27, 0x69, 0xe3,
-	0x87, 0x01, 0xe8, 0xa4, 0x8d, 0x1f, 0xc1, 0x8c, 0x12, 0x36, 0xbe, 0x63, 0x0f, 0x88, 0x72, 0xcc,
-	0x04, 0x2e, 0x94, 0xa6, 0x6d, 0xfa, 0x31, 0x8b, 0x80, 0x4a, 0x69, 0xda, 0x82, 0x63, 0x26, 0x01,
-	0x21, 0x94, 0x22, 0xec, 0x86, 0x63, 0x16, 0xc5, 0x93, 0x12, 0x8e, 0x19, 0x53, 0xa8, 0x1c, 0xb3,
-	0x00, 0xba, 0x49, 0x3a, 0x66, 0x31, 0x3c, 0x3d, 0xe9, 0x98, 0xc5, 0xd1, 0x9f, 0x84, 0x75, 0x64,
-	0x7a, 0x43, 0xc7, 0x6c, 0x29, 0x01, 0xe5, 0x41, 0xef, 0xa7, 0x38, 0x31, 0x11, 0xa6, 0x6f, 0x7e,
-	0xf0, 0x86, 0xdc, 0xa9, 0x7b, 0x9c, 0xbb, 0x5f, 0xee, 0xf1, 0xbf, 0xd1, 0x60, 0x39, 0x09, 0x21,
-	0x42, 0x29, 0x7a, 0x52, 0xe0, 0xfd, 0xe6, 0xe6, 0x9b, 0xb2, 0x4f, 0xf7, 0x96, 0xbf, 0xeb, 0x9f,
-	0xd6, 0xff, 0xed, 0xab, 0x35, 0xed, 0x3f, 0xbf, 0x5a, 0xd3, 0xfe, 0xe7, 0xab, 0x35, 0xed, 0x6f,
-	0xff, 0x77, 0x6d, 0xee, 0x34, 0xcf, 0x7e, 0x5d, 0xf9, 0xed, 0x5f, 0x05, 0x00, 0x00, 0xff, 0xff,
-	0x52, 0x4e, 0xd7, 0x33, 0xe4, 0x39, 0x00, 0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.proto b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.proto
deleted file mode 100644
index 423eaba..0000000
--- a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/rpc.proto
+++ /dev/null
@@ -1,1146 +0,0 @@
-syntax = "proto3";
-package etcdserverpb;
-
-import "gogoproto/gogo.proto";
-import "etcd/mvcc/mvccpb/kv.proto";
-import "etcd/auth/authpb/auth.proto";
-
-// for grpc-gateway
-import "google/api/annotations.proto";
-
-option (gogoproto.marshaler_all) = true;
-option (gogoproto.unmarshaler_all) = true;
-
-service KV {
-  // Range gets the keys in the range from the key-value store.
-  rpc Range(RangeRequest) returns (RangeResponse) {
-      option (google.api.http) = {
-        post: "/v3/kv/range"
-        body: "*"
-    };
-  }
-
-  // Put puts the given key into the key-value store.
-  // A put request increments the revision of the key-value store
-  // and generates one event in the event history.
-  rpc Put(PutRequest) returns (PutResponse) {
-      option (google.api.http) = {
-        post: "/v3/kv/put"
-        body: "*"
-    };
-  }
-
-  // DeleteRange deletes the given range from the key-value store.
-  // A delete request increments the revision of the key-value store
-  // and generates a delete event in the event history for every deleted key.
-  rpc DeleteRange(DeleteRangeRequest) returns (DeleteRangeResponse) {
-      option (google.api.http) = {
-        post: "/v3/kv/deleterange"
-        body: "*"
-    };
-  }
-
-  // Txn processes multiple requests in a single transaction.
-  // A txn request increments the revision of the key-value store
-  // and generates events with the same revision for every completed request.
-  // It is not allowed to modify the same key several times within one txn.
-  rpc Txn(TxnRequest) returns (TxnResponse) {
-      option (google.api.http) = {
-        post: "/v3/kv/txn"
-        body: "*"
-    };
-  }
-
-  // Compact compacts the event history in the etcd key-value store. The key-value
-  // store should be periodically compacted or the event history will continue to grow
-  // indefinitely.
-  rpc Compact(CompactionRequest) returns (CompactionResponse) {
-      option (google.api.http) = {
-        post: "/v3/kv/compaction"
-        body: "*"
-    };
-  }
-}
-
-service Watch {
-  // Watch watches for events happening or that have happened. Both input and output
-  // are streams; the input stream is for creating and canceling watchers and the output
-  // stream sends events. One watch RPC can watch on multiple key ranges, streaming events
-  // for several watches at once. The entire event history can be watched starting from the
-  // last compaction revision.
-  rpc Watch(stream WatchRequest) returns (stream WatchResponse) {
-      option (google.api.http) = {
-        post: "/v3/watch"
-        body: "*"
-    };
-  }
-}
-
-service Lease {
-  // LeaseGrant creates a lease which expires if the server does not receive a keepAlive
-  // within a given time to live period. All keys attached to the lease will be expired and
-  // deleted if the lease expires. Each expired key generates a delete event in the event history.
-  rpc LeaseGrant(LeaseGrantRequest) returns (LeaseGrantResponse) {
-      option (google.api.http) = {
-        post: "/v3/lease/grant"
-        body: "*"
-    };
-  }
-
-  // LeaseRevoke revokes a lease. All keys attached to the lease will expire and be deleted.
-  rpc LeaseRevoke(LeaseRevokeRequest) returns (LeaseRevokeResponse) {
-      option (google.api.http) = {
-        post: "/v3/lease/revoke"
-        body: "*"
-        additional_bindings {
-            post: "/v3/kv/lease/revoke"
-            body: "*"
-        }
-    };
-  }
-
-  // LeaseKeepAlive keeps the lease alive by streaming keep alive requests from the client
-  // to the server and streaming keep alive responses from the server to the client.
-  rpc LeaseKeepAlive(stream LeaseKeepAliveRequest) returns (stream LeaseKeepAliveResponse) {
-      option (google.api.http) = {
-        post: "/v3/lease/keepalive"
-        body: "*"
-    };
-  }
-
-  // LeaseTimeToLive retrieves lease information.
-  rpc LeaseTimeToLive(LeaseTimeToLiveRequest) returns (LeaseTimeToLiveResponse) {
-      option (google.api.http) = {
-        post: "/v3/lease/timetolive"
-        body: "*"
-        additional_bindings {
-            post: "/v3/kv/lease/timetolive"
-            body: "*"
-        }
-    };
-  }
-
-  // LeaseLeases lists all existing leases.
-  rpc LeaseLeases(LeaseLeasesRequest) returns (LeaseLeasesResponse) {
-      option (google.api.http) = {
-        post: "/v3/lease/leases"
-        body: "*"
-        additional_bindings {
-            post: "/v3/kv/lease/leases"
-            body: "*"
-        }
-    };
-  }
-}
-
-service Cluster {
-  // MemberAdd adds a member into the cluster.
-  rpc MemberAdd(MemberAddRequest) returns (MemberAddResponse) {
-      option (google.api.http) = {
-        post: "/v3/cluster/member/add"
-        body: "*"
-    };
-  }
-
-  // MemberRemove removes an existing member from the cluster.
-  rpc MemberRemove(MemberRemoveRequest) returns (MemberRemoveResponse) {
-      option (google.api.http) = {
-        post: "/v3/cluster/member/remove"
-        body: "*"
-    };
-  }
-
-  // MemberUpdate updates the member configuration.
-  rpc MemberUpdate(MemberUpdateRequest) returns (MemberUpdateResponse) {
-      option (google.api.http) = {
-        post: "/v3/cluster/member/update"
-        body: "*"
-    };
-  }
-
-  // MemberList lists all the members in the cluster.
-  rpc MemberList(MemberListRequest) returns (MemberListResponse) {
-      option (google.api.http) = {
-        post: "/v3/cluster/member/list"
-        body: "*"
-    };
-  }
-
-  // MemberPromote promotes a member from raft learner (non-voting) to raft voting member.
-  rpc MemberPromote(MemberPromoteRequest) returns (MemberPromoteResponse) {
-      option (google.api.http) = {
-        post: "/v3/cluster/member/promote"
-        body: "*"
-    };
-  }
-}
-
-service Maintenance {
-  // Alarm activates, deactivates, and queries alarms regarding cluster health.
-  rpc Alarm(AlarmRequest) returns (AlarmResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/alarm"
-        body: "*"
-    };
-  }
-
-  // Status gets the status of the member.
-  rpc Status(StatusRequest) returns (StatusResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/status"
-        body: "*"
-    };
-  }
-
-  // Defragment defragments a member's backend database to recover storage space.
-  rpc Defragment(DefragmentRequest) returns (DefragmentResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/defragment"
-        body: "*"
-    };
-  }
-
-  // Hash computes the hash of whole backend keyspace,
-  // including key, lease, and other buckets in storage.
-  // This is designed for testing ONLY!
-  // Do not rely on this in production with ongoing transactions,
-  // since Hash operation does not hold MVCC locks.
-  // Use "HashKV" API instead for "key" bucket consistency checks.
-  rpc Hash(HashRequest) returns (HashResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/hash"
-        body: "*"
-    };
-  }
-
-  // HashKV computes the hash of all MVCC keys up to a given revision.
-  // It only iterates "key" bucket in backend storage.
-  rpc HashKV(HashKVRequest) returns (HashKVResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/hash"
-        body: "*"
-    };
-  }
-
-  // Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
-  rpc Snapshot(SnapshotRequest) returns (stream SnapshotResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/snapshot"
-        body: "*"
-    };
-  }
-
-  // MoveLeader requests current leader node to transfer its leadership to transferee.
-  rpc MoveLeader(MoveLeaderRequest) returns (MoveLeaderResponse) {
-      option (google.api.http) = {
-        post: "/v3/maintenance/transfer-leadership"
-        body: "*"
-    };
-  }
-}
-
-service Auth {
-  // AuthEnable enables authentication.
-  rpc AuthEnable(AuthEnableRequest) returns (AuthEnableResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/enable"
-        body: "*"
-    };
-  }
-
-  // AuthDisable disables authentication.
-  rpc AuthDisable(AuthDisableRequest) returns (AuthDisableResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/disable"
-        body: "*"
-    };
-  }
-
-  // Authenticate processes an authenticate request.
-  rpc Authenticate(AuthenticateRequest) returns (AuthenticateResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/authenticate"
-        body: "*"
-    };
-  }
-
-  // UserAdd adds a new user. User name cannot be empty.
-  rpc UserAdd(AuthUserAddRequest) returns (AuthUserAddResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/add"
-        body: "*"
-    };
-  }
-
-  // UserGet gets detailed user information.
-  rpc UserGet(AuthUserGetRequest) returns (AuthUserGetResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/get"
-        body: "*"
-    };
-  }
-
-  // UserList gets a list of all users.
-  rpc UserList(AuthUserListRequest) returns (AuthUserListResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/list"
-        body: "*"
-    };
-  }
-
-  // UserDelete deletes a specified user.
-  rpc UserDelete(AuthUserDeleteRequest) returns (AuthUserDeleteResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/delete"
-        body: "*"
-    };
-  }
-
-  // UserChangePassword changes the password of a specified user.
-  rpc UserChangePassword(AuthUserChangePasswordRequest) returns (AuthUserChangePasswordResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/changepw"
-        body: "*"
-    };
-  }
-
-  // UserGrant grants a role to a specified user.
-  rpc UserGrantRole(AuthUserGrantRoleRequest) returns (AuthUserGrantRoleResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/grant"
-        body: "*"
-    };
-  }
-
-  // UserRevokeRole revokes a role of specified user.
-  rpc UserRevokeRole(AuthUserRevokeRoleRequest) returns (AuthUserRevokeRoleResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/user/revoke"
-        body: "*"
-    };
-  }
-
-  // RoleAdd adds a new role. Role name cannot be empty.
-  rpc RoleAdd(AuthRoleAddRequest) returns (AuthRoleAddResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/add"
-        body: "*"
-    };
-  }
-
-  // RoleGet gets detailed role information.
-  rpc RoleGet(AuthRoleGetRequest) returns (AuthRoleGetResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/get"
-        body: "*"
-    };
-  }
-
-  // RoleList gets lists of all roles.
-  rpc RoleList(AuthRoleListRequest) returns (AuthRoleListResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/list"
-        body: "*"
-    };
-  }
-
-  // RoleDelete deletes a specified role.
-  rpc RoleDelete(AuthRoleDeleteRequest) returns (AuthRoleDeleteResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/delete"
-        body: "*"
-    };
-  }
-
-  // RoleGrantPermission grants a permission of a specified key or range to a specified role.
-  rpc RoleGrantPermission(AuthRoleGrantPermissionRequest) returns (AuthRoleGrantPermissionResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/grant"
-        body: "*"
-    };
-  }
-
-  // RoleRevokePermission revokes a key or range permission of a specified role.
-  rpc RoleRevokePermission(AuthRoleRevokePermissionRequest) returns (AuthRoleRevokePermissionResponse) {
-      option (google.api.http) = {
-        post: "/v3/auth/role/revoke"
-        body: "*"
-    };
-  }
-}
-
-message ResponseHeader {
-  // cluster_id is the ID of the cluster which sent the response.
-  uint64 cluster_id = 1;
-  // member_id is the ID of the member which sent the response.
-  uint64 member_id = 2;
-  // revision is the key-value store revision when the request was applied.
-  // For watch progress responses, the header.revision indicates progress. All future events
-  // recieved in this stream are guaranteed to have a higher revision number than the
-  // header.revision number.
-  int64 revision = 3;
-  // raft_term is the raft term when the request was applied.
-  uint64 raft_term = 4;
-}
-
-message RangeRequest {
-  enum SortOrder {
-	NONE = 0; // default, no sorting
-	ASCEND = 1; // lowest target value first
-	DESCEND = 2; // highest target value first
-  }
-  enum SortTarget {
-	KEY = 0;
-	VERSION = 1;
-	CREATE = 2;
-	MOD = 3;
-	VALUE = 4;
-  }
-
-  // key is the first key for the range. If range_end is not given, the request only looks up key.
-  bytes key = 1;
-  // range_end is the upper bound on the requested range [key, range_end).
-  // If range_end is '\0', the range is all keys >= key.
-  // If range_end is key plus one (e.g., "aa"+1 == "ab", "a\xff"+1 == "b"),
-  // then the range request gets all keys prefixed with key.
-  // If both key and range_end are '\0', then the range request returns all keys.
-  bytes range_end = 2;
-  // limit is a limit on the number of keys returned for the request. When limit is set to 0,
-  // it is treated as no limit.
-  int64 limit = 3;
-  // revision is the point-in-time of the key-value store to use for the range.
-  // If revision is less or equal to zero, the range is over the newest key-value store.
-  // If the revision has been compacted, ErrCompacted is returned as a response.
-  int64 revision = 4;
-
-  // sort_order is the order for returned sorted results.
-  SortOrder sort_order = 5;
-
-  // sort_target is the key-value field to use for sorting.
-  SortTarget sort_target = 6;
-
-  // serializable sets the range request to use serializable member-local reads.
-  // Range requests are linearizable by default; linearizable requests have higher
-  // latency and lower throughput than serializable requests but reflect the current
-  // consensus of the cluster. For better performance, in exchange for possible stale reads,
-  // a serializable range request is served locally without needing to reach consensus
-  // with other nodes in the cluster.
-  bool serializable = 7;
-
-  // keys_only when set returns only the keys and not the values.
-  bool keys_only = 8;
-
-  // count_only when set returns only the count of the keys in the range.
-  bool count_only = 9;
-
-  // min_mod_revision is the lower bound for returned key mod revisions; all keys with
-  // lesser mod revisions will be filtered away.
-  int64 min_mod_revision = 10;
-
-  // max_mod_revision is the upper bound for returned key mod revisions; all keys with
-  // greater mod revisions will be filtered away.
-  int64 max_mod_revision = 11;
-
-  // min_create_revision is the lower bound for returned key create revisions; all keys with
-  // lesser create revisions will be filtered away.
-  int64 min_create_revision = 12;
-
-  // max_create_revision is the upper bound for returned key create revisions; all keys with
-  // greater create revisions will be filtered away.
-  int64 max_create_revision = 13;
-}
-
-message RangeResponse {
-  ResponseHeader header = 1;
-  // kvs is the list of key-value pairs matched by the range request.
-  // kvs is empty when count is requested.
-  repeated mvccpb.KeyValue kvs = 2;
-  // more indicates if there are more keys to return in the requested range.
-  bool more = 3;
-  // count is set to the number of keys within the range when requested.
-  int64 count = 4;
-}
-
-message PutRequest {
-  // key is the key, in bytes, to put into the key-value store.
-  bytes key = 1;
-  // value is the value, in bytes, to associate with the key in the key-value store.
-  bytes value = 2;
-  // lease is the lease ID to associate with the key in the key-value store. A lease
-  // value of 0 indicates no lease.
-  int64 lease = 3;
-
-  // If prev_kv is set, etcd gets the previous key-value pair before changing it.
-  // The previous key-value pair will be returned in the put response.
-  bool prev_kv = 4;
-
-  // If ignore_value is set, etcd updates the key using its current value.
-  // Returns an error if the key does not exist.
-  bool ignore_value = 5;
-
-  // If ignore_lease is set, etcd updates the key using its current lease.
-  // Returns an error if the key does not exist.
-  bool ignore_lease = 6;
-}
-
-message PutResponse {
-  ResponseHeader header = 1;
-  // if prev_kv is set in the request, the previous key-value pair will be returned.
-  mvccpb.KeyValue prev_kv = 2;
-}
-
-message DeleteRangeRequest {
-  // key is the first key to delete in the range.
-  bytes key = 1;
-  // range_end is the key following the last key to delete for the range [key, range_end).
-  // If range_end is not given, the range is defined to contain only the key argument.
-  // If range_end is one bit larger than the given key, then the range is all the keys
-  // with the prefix (the given key).
-  // If range_end is '\0', the range is all keys greater than or equal to the key argument.
-  bytes range_end = 2;
-
-  // If prev_kv is set, etcd gets the previous key-value pairs before deleting it.
-  // The previous key-value pairs will be returned in the delete response.
-  bool prev_kv = 3;
-}
-
-message DeleteRangeResponse {
-  ResponseHeader header = 1;
-  // deleted is the number of keys deleted by the delete range request.
-  int64 deleted = 2;
-  // if prev_kv is set in the request, the previous key-value pairs will be returned.
-  repeated mvccpb.KeyValue prev_kvs = 3;
-}
-
-message RequestOp {
-  // request is a union of request types accepted by a transaction.
-  oneof request {
-    RangeRequest request_range = 1;
-    PutRequest request_put = 2;
-    DeleteRangeRequest request_delete_range = 3;
-    TxnRequest request_txn = 4;
-  }
-}
-
-message ResponseOp {
-  // response is a union of response types returned by a transaction.
-  oneof response {
-    RangeResponse response_range = 1;
-    PutResponse response_put = 2;
-    DeleteRangeResponse response_delete_range = 3;
-    TxnResponse response_txn = 4;
-  }
-}
-
-message Compare {
-  enum CompareResult {
-    EQUAL = 0;
-    GREATER = 1;
-    LESS = 2;
-    NOT_EQUAL = 3;
-  }
-  enum CompareTarget {
-    VERSION = 0;
-    CREATE = 1;
-    MOD = 2;
-    VALUE = 3;
-    LEASE = 4;
-  }
-  // result is logical comparison operation for this comparison.
-  CompareResult result = 1;
-  // target is the key-value field to inspect for the comparison.
-  CompareTarget target = 2;
-  // key is the subject key for the comparison operation.
-  bytes key = 3;
-  oneof target_union {
-    // version is the version of the given key
-    int64 version = 4;
-    // create_revision is the creation revision of the given key
-    int64 create_revision = 5;
-    // mod_revision is the last modified revision of the given key.
-    int64 mod_revision = 6;
-    // value is the value of the given key, in bytes.
-    bytes value = 7;
-    // lease is the lease id of the given key.
-    int64 lease = 8;
-    // leave room for more target_union field tags, jump to 64
-  }
-
-  // range_end compares the given target to all keys in the range [key, range_end).
-  // See RangeRequest for more details on key ranges.
-  bytes range_end = 64;
-  // TODO: fill out with most of the rest of RangeRequest fields when needed.
-}
-
-// From google paxosdb paper:
-// Our implementation hinges around a powerful primitive which we call MultiOp. All other database
-// operations except for iteration are implemented as a single call to MultiOp. A MultiOp is applied atomically
-// and consists of three components:
-// 1. A list of tests called guard. Each test in guard checks a single entry in the database. It may check
-// for the absence or presence of a value, or compare with a given value. Two different tests in the guard
-// may apply to the same or different entries in the database. All tests in the guard are applied and
-// MultiOp returns the results. If all tests are true, MultiOp executes t op (see item 2 below), otherwise
-// it executes f op (see item 3 below).
-// 2. A list of database operations called t op. Each operation in the list is either an insert, delete, or
-// lookup operation, and applies to a single database entry. Two different operations in the list may apply
-// to the same or different entries in the database. These operations are executed
-// if guard evaluates to
-// true.
-// 3. A list of database operations called f op. Like t op, but executed if guard evaluates to false.
-message TxnRequest {
-  // compare is a list of predicates representing a conjunction of terms.
-  // If the comparisons succeed, then the success requests will be processed in order,
-  // and the response will contain their respective responses in order.
-  // If the comparisons fail, then the failure requests will be processed in order,
-  // and the response will contain their respective responses in order.
-  repeated Compare compare = 1;
-  // success is a list of requests which will be applied when compare evaluates to true.
-  repeated RequestOp success = 2;
-  // failure is a list of requests which will be applied when compare evaluates to false.
-  repeated RequestOp failure = 3;
-}
-
-message TxnResponse {
-  ResponseHeader header = 1;
-  // succeeded is set to true if the compare evaluated to true or false otherwise.
-  bool succeeded = 2;
-  // responses is a list of responses corresponding to the results from applying
-  // success if succeeded is true or failure if succeeded is false.
-  repeated ResponseOp responses = 3;
-}
-
-// CompactionRequest compacts the key-value store up to a given revision. All superseded keys
-// with a revision less than the compaction revision will be removed.
-message CompactionRequest {
-  // revision is the key-value store revision for the compaction operation.
-  int64 revision = 1;
-  // physical is set so the RPC will wait until the compaction is physically
-  // applied to the local database such that compacted entries are totally
-  // removed from the backend database.
-  bool physical = 2;
-}
-
-message CompactionResponse {
-  ResponseHeader header = 1;
-}
-
-message HashRequest {
-}
-
-message HashKVRequest {
-  // revision is the key-value store revision for the hash operation.
-  int64 revision = 1;
-}
-
-message HashKVResponse {
-  ResponseHeader header = 1;
-  // hash is the hash value computed from the responding member's MVCC keys up to a given revision.
-  uint32 hash = 2;
-  // compact_revision is the compacted revision of key-value store when hash begins.
-  int64 compact_revision = 3;
-}
-
-message HashResponse {
-  ResponseHeader header = 1;
-  // hash is the hash value computed from the responding member's KV's backend.
-  uint32 hash = 2;
-}
-
-message SnapshotRequest {
-}
-
-message SnapshotResponse {
-  // header has the current key-value store information. The first header in the snapshot
-  // stream indicates the point in time of the snapshot.
-  ResponseHeader header = 1;
-
-  // remaining_bytes is the number of blob bytes to be sent after this message
-  uint64 remaining_bytes = 2;
-
-  // blob contains the next chunk of the snapshot in the snapshot stream.
-  bytes blob = 3;
-}
-
-message WatchRequest {
-  // request_union is a request to either create a new watcher or cancel an existing watcher.
-  oneof request_union {
-    WatchCreateRequest create_request = 1;
-    WatchCancelRequest cancel_request = 2;
-    WatchProgressRequest progress_request = 3;
-  }
-}
-
-message WatchCreateRequest {
-  // key is the key to register for watching.
-  bytes key = 1;
-
-  // range_end is the end of the range [key, range_end) to watch. If range_end is not given,
-  // only the key argument is watched. If range_end is equal to '\0', all keys greater than
-  // or equal to the key argument are watched.
-  // If the range_end is one bit larger than the given key,
-  // then all keys with the prefix (the given key) will be watched.
-  bytes range_end = 2;
-
-  // start_revision is an optional revision to watch from (inclusive). No start_revision is "now".
-  int64 start_revision = 3;
-
-  // progress_notify is set so that the etcd server will periodically send a WatchResponse with
-  // no events to the new watcher if there are no recent events. It is useful when clients
-  // wish to recover a disconnected watcher starting from a recent known revision.
-  // The etcd server may decide how often it will send notifications based on current load.
-  bool progress_notify = 4;
-
-  enum FilterType {
-    // filter out put event.
-    NOPUT = 0;
-    // filter out delete event.
-    NODELETE = 1;
-  }
-
-  // filters filter the events at server side before it sends back to the watcher.
-  repeated FilterType filters = 5;
-
-  // If prev_kv is set, created watcher gets the previous KV before the event happens.
-  // If the previous KV is already compacted, nothing will be returned.
-  bool prev_kv = 6;
-
-  // If watch_id is provided and non-zero, it will be assigned to this watcher.
-  // Since creating a watcher in etcd is not a synchronous operation,
-  // this can be used ensure that ordering is correct when creating multiple
-  // watchers on the same stream. Creating a watcher with an ID already in
-  // use on the stream will cause an error to be returned.
-  int64 watch_id = 7;
-
-  // fragment enables splitting large revisions into multiple watch responses.
-  bool fragment = 8;
-}
-
-message WatchCancelRequest {
-  // watch_id is the watcher id to cancel so that no more events are transmitted.
-  int64 watch_id = 1;
-}
-
-// Requests the a watch stream progress status be sent in the watch response stream as soon as
-// possible.
-message WatchProgressRequest {
-}
-
-message WatchResponse {
-  ResponseHeader header = 1;
-  // watch_id is the ID of the watcher that corresponds to the response.
-  int64 watch_id = 2;
-
-  // created is set to true if the response is for a create watch request.
-  // The client should record the watch_id and expect to receive events for
-  // the created watcher from the same stream.
-  // All events sent to the created watcher will attach with the same watch_id.
-  bool created = 3;
-
-  // canceled is set to true if the response is for a cancel watch request.
-  // No further events will be sent to the canceled watcher.
-  bool canceled = 4;
-
-  // compact_revision is set to the minimum index if a watcher tries to watch
-  // at a compacted index.
-  //
-  // This happens when creating a watcher at a compacted revision or the watcher cannot
-  // catch up with the progress of the key-value store.
-  //
-  // The client should treat the watcher as canceled and should not try to create any
-  // watcher with the same start_revision again.
-  int64 compact_revision = 5;
-
-  // cancel_reason indicates the reason for canceling the watcher.
-  string cancel_reason = 6;
-
-  // framgment is true if large watch response was split over multiple responses.
-  bool fragment = 7;
-
-  repeated mvccpb.Event events = 11;
-}
-
-message LeaseGrantRequest {
-  // TTL is the advisory time-to-live in seconds. Expired lease will return -1.
-  int64 TTL = 1;
-  // ID is the requested ID for the lease. If ID is set to 0, the lessor chooses an ID.
-  int64 ID = 2;
-}
-
-message LeaseGrantResponse {
-  ResponseHeader header = 1;
-  // ID is the lease ID for the granted lease.
-  int64 ID = 2;
-  // TTL is the server chosen lease time-to-live in seconds.
-  int64 TTL = 3;
-  string error = 4;
-}
-
-message LeaseRevokeRequest {
-  // ID is the lease ID to revoke. When the ID is revoked, all associated keys will be deleted.
-  int64 ID = 1;
-}
-
-message LeaseRevokeResponse {
-  ResponseHeader header = 1;
-}
-
-message LeaseCheckpoint {
-    // ID is the lease ID to checkpoint.
-  int64 ID = 1;
-
-  // Remaining_TTL is the remaining time until expiry of the lease.
-  int64 remaining_TTL = 2;
-}
-
-message LeaseCheckpointRequest {
-  repeated LeaseCheckpoint checkpoints = 1;
-}
-
-message LeaseCheckpointResponse {
-  ResponseHeader header = 1;
-}
-
-message LeaseKeepAliveRequest {
-  // ID is the lease ID for the lease to keep alive.
-  int64 ID = 1;
-}
-
-message LeaseKeepAliveResponse {
-  ResponseHeader header = 1;
-  // ID is the lease ID from the keep alive request.
-  int64 ID = 2;
-  // TTL is the new time-to-live for the lease.
-  int64 TTL = 3;
-}
-
-message LeaseTimeToLiveRequest {
-  // ID is the lease ID for the lease.
-  int64 ID = 1;
-  // keys is true to query all the keys attached to this lease.
-  bool keys = 2;
-}
-
-message LeaseTimeToLiveResponse {
-  ResponseHeader header = 1;
-  // ID is the lease ID from the keep alive request.
-  int64 ID = 2;
-  // TTL is the remaining TTL in seconds for the lease; the lease will expire in under TTL+1 seconds.
-  int64 TTL = 3;
-  // GrantedTTL is the initial granted time in seconds upon lease creation/renewal.
-  int64 grantedTTL = 4;
-  // Keys is the list of keys attached to this lease.
-  repeated bytes keys = 5;
-}
-
-message LeaseLeasesRequest {
-}
-
-message LeaseStatus {
-  int64 ID = 1;
-  // TODO: int64 TTL = 2;
-}
-
-message LeaseLeasesResponse {
-  ResponseHeader header = 1;
-  repeated LeaseStatus leases = 2;
-}
-
-message Member {
-  // ID is the member ID for this member.
-  uint64 ID = 1;
-  // name is the human-readable name of the member. If the member is not started, the name will be an empty string.
-  string name = 2;
-  // peerURLs is the list of URLs the member exposes to the cluster for communication.
-  repeated string peerURLs = 3;
-  // clientURLs is the list of URLs the member exposes to clients for communication. If the member is not started, clientURLs will be empty.
-  repeated string clientURLs = 4;
-  // isLearner indicates if the member is raft learner.
-  bool isLearner = 5;
-}
-
-message MemberAddRequest {
-  // peerURLs is the list of URLs the added member will use to communicate with the cluster.
-  repeated string peerURLs = 1;
-  // isLearner indicates if the added member is raft learner.
-  bool isLearner = 2;
-}
-
-message MemberAddResponse {
-  ResponseHeader header = 1;
-  // member is the member information for the added member.
-  Member member = 2;
-  // members is a list of all members after adding the new member.
-  repeated Member members = 3;
-}
-
-message MemberRemoveRequest {
-  // ID is the member ID of the member to remove.
-  uint64 ID = 1;
-}
-
-message MemberRemoveResponse {
-  ResponseHeader header = 1;
-  // members is a list of all members after removing the member.
-  repeated Member members = 2;
-}
-
-message MemberUpdateRequest {
-  // ID is the member ID of the member to update.
-  uint64 ID = 1;
-  // peerURLs is the new list of URLs the member will use to communicate with the cluster.
-  repeated string peerURLs = 2;
-}
-
-message MemberUpdateResponse{
-  ResponseHeader header = 1;
-  // members is a list of all members after updating the member.
-  repeated Member members = 2;
-}
-
-message MemberListRequest {
-}
-
-message MemberListResponse {
-  ResponseHeader header = 1;
-  // members is a list of all members associated with the cluster.
-  repeated Member members = 2;
-}
-
-message MemberPromoteRequest {
-  // ID is the member ID of the member to promote.
-  uint64 ID = 1;
-}
-
-message MemberPromoteResponse {
-  ResponseHeader header = 1;
-  // members is a list of all members after promoting the member.
-  repeated Member members = 2;
-}
-
-message DefragmentRequest {
-}
-
-message DefragmentResponse {
-  ResponseHeader header = 1;
-}
-
-message MoveLeaderRequest {
-  // targetID is the node ID for the new leader.
-  uint64 targetID = 1;
-}
-
-message MoveLeaderResponse {
-  ResponseHeader header = 1;
-}
-
-enum AlarmType {
-	NONE = 0; // default, used to query if any alarm is active
-	NOSPACE = 1; // space quota is exhausted
-	CORRUPT = 2; // kv store corruption detected
-}
-
-message AlarmRequest {
-  enum AlarmAction {
-	GET = 0;
-	ACTIVATE = 1;
-	DEACTIVATE = 2;
-  }
-  // action is the kind of alarm request to issue. The action
-  // may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
-  // raised alarm.
-  AlarmAction action = 1;
-  // memberID is the ID of the member associated with the alarm. If memberID is 0, the
-  // alarm request covers all members.
-  uint64 memberID = 2;
-  // alarm is the type of alarm to consider for this request.
-  AlarmType alarm = 3;
-}
-
-message AlarmMember {
-  // memberID is the ID of the member associated with the raised alarm.
-  uint64 memberID = 1;
-  // alarm is the type of alarm which has been raised.
-  AlarmType alarm = 2;
-}
-
-message AlarmResponse {
-  ResponseHeader header = 1;
-  // alarms is a list of alarms associated with the alarm request.
-  repeated AlarmMember alarms = 2;
-}
-
-message StatusRequest {
-}
-
-message StatusResponse {
-  ResponseHeader header = 1;
-  // version is the cluster protocol version used by the responding member.
-  string version = 2;
-  // dbSize is the size of the backend database physically allocated, in bytes, of the responding member.
-  int64 dbSize = 3;
-  // leader is the member ID which the responding member believes is the current leader.
-  uint64 leader = 4;
-  // raftIndex is the current raft committed index of the responding member.
-  uint64 raftIndex = 5;
-  // raftTerm is the current raft term of the responding member.
-  uint64 raftTerm = 6;
-  // raftAppliedIndex is the current raft applied index of the responding member.
-  uint64 raftAppliedIndex = 7;
-  // errors contains alarm/health information and status.
-  repeated string errors = 8;
-  // dbSizeInUse is the size of the backend database logically in use, in bytes, of the responding member.
-  int64 dbSizeInUse = 9;
-  // isLearner indicates if the member is raft learner.
-  bool isLearner = 10;
-}
-
-message AuthEnableRequest {
-}
-
-message AuthDisableRequest {
-}
-
-message AuthenticateRequest {
-  string name = 1;
-  string password = 2;
-}
-
-message AuthUserAddRequest {
-  string name = 1;
-  string password = 2;
-  authpb.UserAddOptions options = 3;
-}
-
-message AuthUserGetRequest {
-  string name = 1;
-}
-
-message AuthUserDeleteRequest {
-  // name is the name of the user to delete.
-  string name = 1;
-}
-
-message AuthUserChangePasswordRequest {
-  // name is the name of the user whose password is being changed.
-  string name = 1;
-  // password is the new password for the user.
-  string password = 2;
-}
-
-message AuthUserGrantRoleRequest {
-  // user is the name of the user which should be granted a given role.
-  string user = 1;
-  // role is the name of the role to grant to the user.
-  string role = 2;
-}
-
-message AuthUserRevokeRoleRequest {
-  string name = 1;
-  string role = 2;
-}
-
-message AuthRoleAddRequest {
-  // name is the name of the role to add to the authentication system.
-  string name = 1;
-}
-
-message AuthRoleGetRequest {
-  string role = 1;
-}
-
-message AuthUserListRequest {
-}
-
-message AuthRoleListRequest {
-}
-
-message AuthRoleDeleteRequest {
-  string role = 1;
-}
-
-message AuthRoleGrantPermissionRequest {
-  // name is the name of the role which will be granted the permission.
-  string name = 1;
-  // perm is the permission to grant to the role.
-  authpb.Permission perm = 2;
-}
-
-message AuthRoleRevokePermissionRequest {
-  string role = 1;
-  bytes key = 2;
-  bytes range_end = 3;
-}
-
-message AuthEnableResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthDisableResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthenticateResponse {
-  ResponseHeader header = 1;
-  // token is an authorized token that can be used in succeeding RPCs
-  string token = 2;
-}
-
-message AuthUserAddResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthUserGetResponse {
-  ResponseHeader header = 1;
-
-  repeated string roles = 2;
-}
-
-message AuthUserDeleteResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthUserChangePasswordResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthUserGrantRoleResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthUserRevokeRoleResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthRoleAddResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthRoleGetResponse {
-  ResponseHeader header = 1;
-
-  repeated authpb.Permission perm = 2;
-}
-
-message AuthRoleListResponse {
-  ResponseHeader header = 1;
-
-  repeated string roles = 2;
-}
-
-message AuthUserListResponse {
-  ResponseHeader header = 1;
-
-  repeated string users = 2;
-}
-
-message AuthRoleDeleteResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthRoleGrantPermissionResponse {
-  ResponseHeader header = 1;
-}
-
-message AuthRoleRevokePermissionResponse {
-  ResponseHeader header = 1;
-}
diff --git a/vendor/go.etcd.io/etcd/mvcc/mvccpb/kv.pb.go b/vendor/go.etcd.io/etcd/mvcc/mvccpb/kv.pb.go
deleted file mode 100644
index 23fe337..0000000
--- a/vendor/go.etcd.io/etcd/mvcc/mvccpb/kv.pb.go
+++ /dev/null
@@ -1,718 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: kv.proto
-
-/*
-	Package mvccpb is a generated protocol buffer package.
-
-	It is generated from these files:
-		kv.proto
-
-	It has these top-level messages:
-		KeyValue
-		Event
-*/
-package mvccpb
-
-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 Event_EventType int32
-
-const (
-	PUT    Event_EventType = 0
-	DELETE Event_EventType = 1
-)
-
-var Event_EventType_name = map[int32]string{
-	0: "PUT",
-	1: "DELETE",
-}
-var Event_EventType_value = map[string]int32{
-	"PUT":    0,
-	"DELETE": 1,
-}
-
-func (x Event_EventType) String() string {
-	return proto.EnumName(Event_EventType_name, int32(x))
-}
-func (Event_EventType) EnumDescriptor() ([]byte, []int) { return fileDescriptorKv, []int{1, 0} }
-
-type KeyValue struct {
-	// key is the key in bytes. An empty key is not allowed.
-	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	// create_revision is the revision of last creation on this key.
-	CreateRevision int64 `protobuf:"varint,2,opt,name=create_revision,json=createRevision,proto3" json:"create_revision,omitempty"`
-	// mod_revision is the revision of last modification on this key.
-	ModRevision int64 `protobuf:"varint,3,opt,name=mod_revision,json=modRevision,proto3" json:"mod_revision,omitempty"`
-	// version is the version of the key. A deletion resets
-	// the version to zero and any modification of the key
-	// increases its version.
-	Version int64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"`
-	// value is the value held by the key, in bytes.
-	Value []byte `protobuf:"bytes,5,opt,name=value,proto3" json:"value,omitempty"`
-	// lease is the ID of the lease that attached to key.
-	// When the attached lease expires, the key will be deleted.
-	// If lease is 0, then no lease is attached to the key.
-	Lease int64 `protobuf:"varint,6,opt,name=lease,proto3" json:"lease,omitempty"`
-}
-
-func (m *KeyValue) Reset()                    { *m = KeyValue{} }
-func (m *KeyValue) String() string            { return proto.CompactTextString(m) }
-func (*KeyValue) ProtoMessage()               {}
-func (*KeyValue) Descriptor() ([]byte, []int) { return fileDescriptorKv, []int{0} }
-
-type Event struct {
-	// type is the kind of event. If type is a PUT, it indicates
-	// new data has been stored to the key. If type is a DELETE,
-	// it indicates the key was deleted.
-	Type Event_EventType `protobuf:"varint,1,opt,name=type,proto3,enum=mvccpb.Event_EventType" json:"type,omitempty"`
-	// kv holds the KeyValue for the event.
-	// A PUT event contains current kv pair.
-	// A PUT event with kv.Version=1 indicates the creation of a key.
-	// A DELETE/EXPIRE event contains the deleted key with
-	// its modification revision set to the revision of deletion.
-	Kv *KeyValue `protobuf:"bytes,2,opt,name=kv" json:"kv,omitempty"`
-	// prev_kv holds the key-value pair before the event happens.
-	PrevKv *KeyValue `protobuf:"bytes,3,opt,name=prev_kv,json=prevKv" json:"prev_kv,omitempty"`
-}
-
-func (m *Event) Reset()                    { *m = Event{} }
-func (m *Event) String() string            { return proto.CompactTextString(m) }
-func (*Event) ProtoMessage()               {}
-func (*Event) Descriptor() ([]byte, []int) { return fileDescriptorKv, []int{1} }
-
-func init() {
-	proto.RegisterType((*KeyValue)(nil), "mvccpb.KeyValue")
-	proto.RegisterType((*Event)(nil), "mvccpb.Event")
-	proto.RegisterEnum("mvccpb.Event_EventType", Event_EventType_name, Event_EventType_value)
-}
-func (m *KeyValue) 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 *KeyValue) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Key) > 0 {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(len(m.Key)))
-		i += copy(dAtA[i:], m.Key)
-	}
-	if m.CreateRevision != 0 {
-		dAtA[i] = 0x10
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.CreateRevision))
-	}
-	if m.ModRevision != 0 {
-		dAtA[i] = 0x18
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.ModRevision))
-	}
-	if m.Version != 0 {
-		dAtA[i] = 0x20
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.Version))
-	}
-	if len(m.Value) > 0 {
-		dAtA[i] = 0x2a
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(len(m.Value)))
-		i += copy(dAtA[i:], m.Value)
-	}
-	if m.Lease != 0 {
-		dAtA[i] = 0x30
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.Lease))
-	}
-	return i, nil
-}
-
-func (m *Event) 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 *Event) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if m.Type != 0 {
-		dAtA[i] = 0x8
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.Type))
-	}
-	if m.Kv != nil {
-		dAtA[i] = 0x12
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.Kv.Size()))
-		n1, err := m.Kv.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n1
-	}
-	if m.PrevKv != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintKv(dAtA, i, uint64(m.PrevKv.Size()))
-		n2, err := m.PrevKv.MarshalTo(dAtA[i:])
-		if err != nil {
-			return 0, err
-		}
-		i += n2
-	}
-	return i, nil
-}
-
-func encodeVarintKv(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 *KeyValue) Size() (n int) {
-	var l int
-	_ = l
-	l = len(m.Key)
-	if l > 0 {
-		n += 1 + l + sovKv(uint64(l))
-	}
-	if m.CreateRevision != 0 {
-		n += 1 + sovKv(uint64(m.CreateRevision))
-	}
-	if m.ModRevision != 0 {
-		n += 1 + sovKv(uint64(m.ModRevision))
-	}
-	if m.Version != 0 {
-		n += 1 + sovKv(uint64(m.Version))
-	}
-	l = len(m.Value)
-	if l > 0 {
-		n += 1 + l + sovKv(uint64(l))
-	}
-	if m.Lease != 0 {
-		n += 1 + sovKv(uint64(m.Lease))
-	}
-	return n
-}
-
-func (m *Event) Size() (n int) {
-	var l int
-	_ = l
-	if m.Type != 0 {
-		n += 1 + sovKv(uint64(m.Type))
-	}
-	if m.Kv != nil {
-		l = m.Kv.Size()
-		n += 1 + l + sovKv(uint64(l))
-	}
-	if m.PrevKv != nil {
-		l = m.PrevKv.Size()
-		n += 1 + l + sovKv(uint64(l))
-	}
-	return n
-}
-
-func sovKv(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozKv(x uint64) (n int) {
-	return sovKv(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *KeyValue) 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 ErrIntOverflowKv
-			}
-			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: KeyValue: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: KeyValue: 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 ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthKv
-			}
-			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 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType)
-			}
-			m.CreateRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.CreateRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType)
-			}
-			m.ModRevision = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ModRevision |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType)
-			}
-			m.Version = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Version |= (int64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 5:
-			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 ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthKv
-			}
-			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
-		case 6:
-			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 ErrIntOverflowKv
-				}
-				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 := skipKv(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthKv
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Event) 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 ErrIntOverflowKv
-			}
-			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: Event: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Event: 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 ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Type |= (Event_EventType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		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 ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthKv
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.Kv == nil {
-				m.Kv = &KeyValue{}
-			}
-			if err := m.Kv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field PrevKv", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowKv
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthKv
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if m.PrevKv == nil {
-				m.PrevKv = &KeyValue{}
-			}
-			if err := m.PrevKv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipKv(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthKv
-			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
-			}
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func skipKv(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, ErrIntOverflowKv
-			}
-			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, ErrIntOverflowKv
-				}
-				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, ErrIntOverflowKv
-				}
-				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, ErrInvalidLengthKv
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowKv
-					}
-					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 := skipKv(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 (
-	ErrInvalidLengthKv = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowKv   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("kv.proto", fileDescriptorKv) }
-
-var fileDescriptorKv = []byte{
-	// 303 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0x41, 0x4e, 0xc2, 0x40,
-	0x14, 0x86, 0x3b, 0x14, 0x0a, 0x3e, 0x08, 0x36, 0x13, 0x12, 0x27, 0x2e, 0x26, 0x95, 0x8d, 0x18,
-	0x13, 0x4c, 0xf0, 0x06, 0xc6, 0xae, 0x70, 0x61, 0x1a, 0x74, 0x4b, 0x4a, 0x79, 0x21, 0xa4, 0x94,
-	0x69, 0x4a, 0x9d, 0xa4, 0x37, 0x71, 0xef, 0xde, 0x73, 0xb0, 0xe4, 0x08, 0x52, 0x2f, 0x62, 0xfa,
-	0xc6, 0xe2, 0xc6, 0xcd, 0xe4, 0xfd, 0xff, 0xff, 0x65, 0xe6, 0x7f, 0x03, 0x9d, 0x58, 0x8f, 0xd3,
-	0x4c, 0xe5, 0x8a, 0x3b, 0x89, 0x8e, 0xa2, 0x74, 0x71, 0x39, 0x58, 0xa9, 0x95, 0x22, 0xeb, 0xae,
-	0x9a, 0x4c, 0x3a, 0xfc, 0x64, 0xd0, 0x99, 0x62, 0xf1, 0x1a, 0x6e, 0xde, 0x90, 0xbb, 0x60, 0xc7,
-	0x58, 0x08, 0xe6, 0xb1, 0x51, 0x2f, 0xa8, 0x46, 0x7e, 0x0d, 0xe7, 0x51, 0x86, 0x61, 0x8e, 0xf3,
-	0x0c, 0xf5, 0x7a, 0xb7, 0x56, 0x5b, 0xd1, 0xf0, 0xd8, 0xc8, 0x0e, 0xfa, 0xc6, 0x0e, 0x7e, 0x5d,
-	0x7e, 0x05, 0xbd, 0x44, 0x2d, 0xff, 0x28, 0x9b, 0xa8, 0x6e, 0xa2, 0x96, 0x27, 0x44, 0x40, 0x5b,
-	0x63, 0x46, 0x69, 0x93, 0xd2, 0x5a, 0xf2, 0x01, 0xb4, 0x74, 0x55, 0x40, 0xb4, 0xe8, 0x65, 0x23,
-	0x2a, 0x77, 0x83, 0xe1, 0x0e, 0x85, 0x43, 0xb4, 0x11, 0xc3, 0x0f, 0x06, 0x2d, 0x5f, 0xe3, 0x36,
-	0xe7, 0xb7, 0xd0, 0xcc, 0x8b, 0x14, 0xa9, 0x6e, 0x7f, 0x72, 0x31, 0x36, 0x7b, 0x8e, 0x29, 0x34,
-	0xe7, 0xac, 0x48, 0x31, 0x20, 0x88, 0x7b, 0xd0, 0x88, 0x35, 0x75, 0xef, 0x4e, 0xdc, 0x1a, 0xad,
-	0x17, 0x0f, 0x1a, 0xb1, 0xe6, 0x37, 0xd0, 0x4e, 0x33, 0xd4, 0xf3, 0x58, 0x53, 0xf9, 0xff, 0x30,
-	0xa7, 0x02, 0xa6, 0x7a, 0xe8, 0xc1, 0xd9, 0xe9, 0x7e, 0xde, 0x06, 0xfb, 0xf9, 0x65, 0xe6, 0x5a,
-	0x1c, 0xc0, 0x79, 0xf4, 0x9f, 0xfc, 0x99, 0xef, 0xb2, 0x07, 0xb1, 0x3f, 0x4a, 0xeb, 0x70, 0x94,
-	0xd6, 0xbe, 0x94, 0xec, 0x50, 0x4a, 0xf6, 0x55, 0x4a, 0xf6, 0xfe, 0x2d, 0xad, 0x85, 0x43, 0xff,
-	0x7e, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x45, 0x92, 0x5d, 0xa1, 0x01, 0x00, 0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/package_logger.go b/vendor/go.etcd.io/etcd/pkg/logutil/package_logger.go
deleted file mode 100644
index 729cbdb..0000000
--- a/vendor/go.etcd.io/etcd/pkg/logutil/package_logger.go
+++ /dev/null
@@ -1,60 +0,0 @@
-// 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 logutil
-
-import (
-	"github.com/coreos/pkg/capnslog"
-	"google.golang.org/grpc/grpclog"
-)
-
-// assert that "packageLogger" satisfy "Logger" interface
-var _ Logger = &packageLogger{}
-
-// NewPackageLogger wraps "*capnslog.PackageLogger" that implements "Logger" interface.
-//
-// For example:
-//
-//  var defaultLogger Logger
-//  defaultLogger = NewPackageLogger("go.etcd.io/etcd", "snapshot")
-//
-func NewPackageLogger(repo, pkg string) Logger {
-	return &packageLogger{p: capnslog.NewPackageLogger(repo, pkg)}
-}
-
-type packageLogger struct {
-	p *capnslog.PackageLogger
-}
-
-func (l *packageLogger) Info(args ...interface{})                    { l.p.Info(args...) }
-func (l *packageLogger) Infoln(args ...interface{})                  { l.p.Info(args...) }
-func (l *packageLogger) Infof(format string, args ...interface{})    { l.p.Infof(format, args...) }
-func (l *packageLogger) Warning(args ...interface{})                 { l.p.Warning(args...) }
-func (l *packageLogger) Warningln(args ...interface{})               { l.p.Warning(args...) }
-func (l *packageLogger) Warningf(format string, args ...interface{}) { l.p.Warningf(format, args...) }
-func (l *packageLogger) Error(args ...interface{})                   { l.p.Error(args...) }
-func (l *packageLogger) Errorln(args ...interface{})                 { l.p.Error(args...) }
-func (l *packageLogger) Errorf(format string, args ...interface{})   { l.p.Errorf(format, args...) }
-func (l *packageLogger) Fatal(args ...interface{})                   { l.p.Fatal(args...) }
-func (l *packageLogger) Fatalln(args ...interface{})                 { l.p.Fatal(args...) }
-func (l *packageLogger) Fatalf(format string, args ...interface{})   { l.p.Fatalf(format, args...) }
-func (l *packageLogger) V(lvl int) bool {
-	return l.p.LevelAt(capnslog.LogLevel(lvl))
-}
-func (l *packageLogger) Lvl(lvl int) grpclog.LoggerV2 {
-	if l.p.LevelAt(capnslog.LogLevel(lvl)) {
-		return l
-	}
-	return &discardLogger{}
-}
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/zap.go b/vendor/go.etcd.io/etcd/pkg/logutil/zap.go
deleted file mode 100644
index 8fc6e03..0000000
--- a/vendor/go.etcd.io/etcd/pkg/logutil/zap.go
+++ /dev/null
@@ -1,91 +0,0 @@
-// 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 logutil
-
-import (
-	"sort"
-
-	"go.uber.org/zap"
-	"go.uber.org/zap/zapcore"
-)
-
-// DefaultZapLoggerConfig defines default zap logger configuration.
-var DefaultZapLoggerConfig = zap.Config{
-	Level: zap.NewAtomicLevelAt(ConvertToZapLevel(DefaultLogLevel)),
-
-	Development: false,
-	Sampling: &zap.SamplingConfig{
-		Initial:    100,
-		Thereafter: 100,
-	},
-
-	Encoding: "json",
-
-	// copied from "zap.NewProductionEncoderConfig" with some updates
-	EncoderConfig: zapcore.EncoderConfig{
-		TimeKey:        "ts",
-		LevelKey:       "level",
-		NameKey:        "logger",
-		CallerKey:      "caller",
-		MessageKey:     "msg",
-		StacktraceKey:  "stacktrace",
-		LineEnding:     zapcore.DefaultLineEnding,
-		EncodeLevel:    zapcore.LowercaseLevelEncoder,
-		EncodeTime:     zapcore.ISO8601TimeEncoder,
-		EncodeDuration: zapcore.StringDurationEncoder,
-		EncodeCaller:   zapcore.ShortCallerEncoder,
-	},
-
-	// Use "/dev/null" to discard all
-	OutputPaths:      []string{"stderr"},
-	ErrorOutputPaths: []string{"stderr"},
-}
-
-// MergeOutputPaths merges logging output paths, resolving conflicts.
-func MergeOutputPaths(cfg zap.Config) zap.Config {
-	outputs := make(map[string]struct{})
-	for _, v := range cfg.OutputPaths {
-		outputs[v] = struct{}{}
-	}
-	outputSlice := make([]string, 0)
-	if _, ok := outputs["/dev/null"]; ok {
-		// "/dev/null" to discard all
-		outputSlice = []string{"/dev/null"}
-	} else {
-		for k := range outputs {
-			outputSlice = append(outputSlice, k)
-		}
-	}
-	cfg.OutputPaths = outputSlice
-	sort.Strings(cfg.OutputPaths)
-
-	errOutputs := make(map[string]struct{})
-	for _, v := range cfg.ErrorOutputPaths {
-		errOutputs[v] = struct{}{}
-	}
-	errOutputSlice := make([]string, 0)
-	if _, ok := errOutputs["/dev/null"]; ok {
-		// "/dev/null" to discard all
-		errOutputSlice = []string{"/dev/null"}
-	} else {
-		for k := range errOutputs {
-			errOutputSlice = append(errOutputSlice, k)
-		}
-	}
-	cfg.ErrorOutputPaths = errOutputSlice
-	sort.Strings(cfg.ErrorOutputPaths)
-
-	return cfg
-}
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/zap_journal.go b/vendor/go.etcd.io/etcd/pkg/logutil/zap_journal.go
deleted file mode 100644
index fcd3903..0000000
--- a/vendor/go.etcd.io/etcd/pkg/logutil/zap_journal.go
+++ /dev/null
@@ -1,92 +0,0 @@
-// 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 logutil
-
-import (
-	"bytes"
-	"encoding/json"
-	"fmt"
-	"io"
-	"os"
-	"path/filepath"
-
-	"go.etcd.io/etcd/pkg/systemd"
-
-	"github.com/coreos/go-systemd/journal"
-	"go.uber.org/zap/zapcore"
-)
-
-// NewJournalWriter wraps "io.Writer" to redirect log output
-// to the local systemd journal. If journald send fails, it fails
-// back to writing to the original writer.
-// The decode overhead is only <30µs per write.
-// Reference: https://github.com/coreos/pkg/blob/master/capnslog/journald_formatter.go
-func NewJournalWriter(wr io.Writer) (io.Writer, error) {
-	return &journalWriter{Writer: wr}, systemd.DialJournal()
-}
-
-type journalWriter struct {
-	io.Writer
-}
-
-// WARN: assume that etcd uses default field names in zap encoder config
-// make sure to keep this up-to-date!
-type logLine struct {
-	Level  string `json:"level"`
-	Caller string `json:"caller"`
-}
-
-func (w *journalWriter) Write(p []byte) (int, error) {
-	line := &logLine{}
-	if err := json.NewDecoder(bytes.NewReader(p)).Decode(line); err != nil {
-		return 0, err
-	}
-
-	var pri journal.Priority
-	switch line.Level {
-	case zapcore.DebugLevel.String():
-		pri = journal.PriDebug
-	case zapcore.InfoLevel.String():
-		pri = journal.PriInfo
-
-	case zapcore.WarnLevel.String():
-		pri = journal.PriWarning
-	case zapcore.ErrorLevel.String():
-		pri = journal.PriErr
-
-	case zapcore.DPanicLevel.String():
-		pri = journal.PriCrit
-	case zapcore.PanicLevel.String():
-		pri = journal.PriCrit
-	case zapcore.FatalLevel.String():
-		pri = journal.PriCrit
-
-	default:
-		panic(fmt.Errorf("unknown log level: %q", line.Level))
-	}
-
-	err := journal.Send(string(p), pri, map[string]string{
-		"PACKAGE":           filepath.Dir(line.Caller),
-		"SYSLOG_IDENTIFIER": filepath.Base(os.Args[0]),
-	})
-	if err != nil {
-		// "journal" also falls back to stderr
-		// "fmt.Fprintln(os.Stderr, s)"
-		return w.Writer.Write(p)
-	}
-	return 0, nil
-}
diff --git a/vendor/go.etcd.io/etcd/pkg/logutil/zap_raft.go b/vendor/go.etcd.io/etcd/pkg/logutil/zap_raft.go
deleted file mode 100644
index f016b30..0000000
--- a/vendor/go.etcd.io/etcd/pkg/logutil/zap_raft.go
+++ /dev/null
@@ -1,102 +0,0 @@
-// 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 logutil
-
-import (
-	"errors"
-
-	"go.etcd.io/etcd/raft"
-
-	"go.uber.org/zap"
-	"go.uber.org/zap/zapcore"
-)
-
-// NewRaftLogger builds "raft.Logger" from "*zap.Config".
-func NewRaftLogger(lcfg *zap.Config) (raft.Logger, error) {
-	if lcfg == nil {
-		return nil, errors.New("nil zap.Config")
-	}
-	lg, err := lcfg.Build(zap.AddCallerSkip(1)) // to annotate caller outside of "logutil"
-	if err != nil {
-		return nil, err
-	}
-	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}, nil
-}
-
-// NewRaftLoggerZap converts "*zap.Logger" to "raft.Logger".
-func NewRaftLoggerZap(lg *zap.Logger) raft.Logger {
-	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}
-}
-
-// NewRaftLoggerFromZapCore creates "raft.Logger" from "zap.Core"
-// and "zapcore.WriteSyncer".
-func NewRaftLoggerFromZapCore(cr zapcore.Core, syncer zapcore.WriteSyncer) raft.Logger {
-	// "AddCallerSkip" to annotate caller outside of "logutil"
-	lg := zap.New(cr, zap.AddCaller(), zap.AddCallerSkip(1), zap.ErrorOutput(syncer))
-	return &zapRaftLogger{lg: lg, sugar: lg.Sugar()}
-}
-
-type zapRaftLogger struct {
-	lg    *zap.Logger
-	sugar *zap.SugaredLogger
-}
-
-func (zl *zapRaftLogger) Debug(args ...interface{}) {
-	zl.sugar.Debug(args...)
-}
-
-func (zl *zapRaftLogger) Debugf(format string, args ...interface{}) {
-	zl.sugar.Debugf(format, args...)
-}
-
-func (zl *zapRaftLogger) Error(args ...interface{}) {
-	zl.sugar.Error(args...)
-}
-
-func (zl *zapRaftLogger) Errorf(format string, args ...interface{}) {
-	zl.sugar.Errorf(format, args...)
-}
-
-func (zl *zapRaftLogger) Info(args ...interface{}) {
-	zl.sugar.Info(args...)
-}
-
-func (zl *zapRaftLogger) Infof(format string, args ...interface{}) {
-	zl.sugar.Infof(format, args...)
-}
-
-func (zl *zapRaftLogger) Warning(args ...interface{}) {
-	zl.sugar.Warn(args...)
-}
-
-func (zl *zapRaftLogger) Warningf(format string, args ...interface{}) {
-	zl.sugar.Warnf(format, args...)
-}
-
-func (zl *zapRaftLogger) Fatal(args ...interface{}) {
-	zl.sugar.Fatal(args...)
-}
-
-func (zl *zapRaftLogger) Fatalf(format string, args ...interface{}) {
-	zl.sugar.Fatalf(format, args...)
-}
-
-func (zl *zapRaftLogger) Panic(args ...interface{}) {
-	zl.sugar.Panic(args...)
-}
-
-func (zl *zapRaftLogger) Panicf(format string, args ...interface{}) {
-	zl.sugar.Panicf(format, args...)
-}
diff --git a/vendor/go.etcd.io/etcd/pkg/types/id.go b/vendor/go.etcd.io/etcd/pkg/types/id.go
deleted file mode 100644
index ae00388..0000000
--- a/vendor/go.etcd.io/etcd/pkg/types/id.go
+++ /dev/null
@@ -1,39 +0,0 @@
-// 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 types
-
-import "strconv"
-
-// ID represents a generic identifier which is canonically
-// stored as a uint64 but is typically represented as a
-// base-16 string for input/output
-type ID uint64
-
-func (i ID) String() string {
-	return strconv.FormatUint(uint64(i), 16)
-}
-
-// IDFromString attempts to create an ID from a base-16 string.
-func IDFromString(s string) (ID, error) {
-	i, err := strconv.ParseUint(s, 16, 64)
-	return ID(i), err
-}
-
-// IDSlice implements the sort interface
-type IDSlice []ID
-
-func (p IDSlice) Len() int           { return len(p) }
-func (p IDSlice) Less(i, j int) bool { return uint64(p[i]) < uint64(p[j]) }
-func (p IDSlice) Swap(i, j int)      { p[i], p[j] = p[j], p[i] }
diff --git a/vendor/go.etcd.io/etcd/pkg/types/set.go b/vendor/go.etcd.io/etcd/pkg/types/set.go
deleted file mode 100644
index e7a3cdc..0000000
--- a/vendor/go.etcd.io/etcd/pkg/types/set.go
+++ /dev/null
@@ -1,195 +0,0 @@
-// 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 types
-
-import (
-	"reflect"
-	"sort"
-	"sync"
-)
-
-type Set interface {
-	Add(string)
-	Remove(string)
-	Contains(string) bool
-	Equals(Set) bool
-	Length() int
-	Values() []string
-	Copy() Set
-	Sub(Set) Set
-}
-
-func NewUnsafeSet(values ...string) *unsafeSet {
-	set := &unsafeSet{make(map[string]struct{})}
-	for _, v := range values {
-		set.Add(v)
-	}
-	return set
-}
-
-func NewThreadsafeSet(values ...string) *tsafeSet {
-	us := NewUnsafeSet(values...)
-	return &tsafeSet{us, sync.RWMutex{}}
-}
-
-type unsafeSet struct {
-	d map[string]struct{}
-}
-
-// Add adds a new value to the set (no-op if the value is already present)
-func (us *unsafeSet) Add(value string) {
-	us.d[value] = struct{}{}
-}
-
-// Remove removes the given value from the set
-func (us *unsafeSet) Remove(value string) {
-	delete(us.d, value)
-}
-
-// Contains returns whether the set contains the given value
-func (us *unsafeSet) Contains(value string) (exists bool) {
-	_, exists = us.d[value]
-	return exists
-}
-
-// ContainsAll returns whether the set contains all given values
-func (us *unsafeSet) ContainsAll(values []string) bool {
-	for _, s := range values {
-		if !us.Contains(s) {
-			return false
-		}
-	}
-	return true
-}
-
-// Equals returns whether the contents of two sets are identical
-func (us *unsafeSet) Equals(other Set) bool {
-	v1 := sort.StringSlice(us.Values())
-	v2 := sort.StringSlice(other.Values())
-	v1.Sort()
-	v2.Sort()
-	return reflect.DeepEqual(v1, v2)
-}
-
-// Length returns the number of elements in the set
-func (us *unsafeSet) Length() int {
-	return len(us.d)
-}
-
-// Values returns the values of the Set in an unspecified order.
-func (us *unsafeSet) Values() (values []string) {
-	values = make([]string, 0)
-	for val := range us.d {
-		values = append(values, val)
-	}
-	return values
-}
-
-// Copy creates a new Set containing the values of the first
-func (us *unsafeSet) Copy() Set {
-	cp := NewUnsafeSet()
-	for val := range us.d {
-		cp.Add(val)
-	}
-
-	return cp
-}
-
-// Sub removes all elements in other from the set
-func (us *unsafeSet) Sub(other Set) Set {
-	oValues := other.Values()
-	result := us.Copy().(*unsafeSet)
-
-	for _, val := range oValues {
-		if _, ok := result.d[val]; !ok {
-			continue
-		}
-		delete(result.d, val)
-	}
-
-	return result
-}
-
-type tsafeSet struct {
-	us *unsafeSet
-	m  sync.RWMutex
-}
-
-func (ts *tsafeSet) Add(value string) {
-	ts.m.Lock()
-	defer ts.m.Unlock()
-	ts.us.Add(value)
-}
-
-func (ts *tsafeSet) Remove(value string) {
-	ts.m.Lock()
-	defer ts.m.Unlock()
-	ts.us.Remove(value)
-}
-
-func (ts *tsafeSet) Contains(value string) (exists bool) {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-	return ts.us.Contains(value)
-}
-
-func (ts *tsafeSet) Equals(other Set) bool {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-
-	// If ts and other represent the same variable, avoid calling
-	// ts.us.Equals(other), to avoid double RLock bug
-	if _other, ok := other.(*tsafeSet); ok {
-		if _other == ts {
-			return true
-		}
-	}
-	return ts.us.Equals(other)
-}
-
-func (ts *tsafeSet) Length() int {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-	return ts.us.Length()
-}
-
-func (ts *tsafeSet) Values() (values []string) {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-	return ts.us.Values()
-}
-
-func (ts *tsafeSet) Copy() Set {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-	usResult := ts.us.Copy().(*unsafeSet)
-	return &tsafeSet{usResult, sync.RWMutex{}}
-}
-
-func (ts *tsafeSet) Sub(other Set) Set {
-	ts.m.RLock()
-	defer ts.m.RUnlock()
-
-	// If ts and other represent the same variable, avoid calling
-	// ts.us.Sub(other), to avoid double RLock bug
-	if _other, ok := other.(*tsafeSet); ok {
-		if _other == ts {
-			usResult := NewUnsafeSet()
-			return &tsafeSet{usResult, sync.RWMutex{}}
-		}
-	}
-	usResult := ts.us.Sub(other).(*unsafeSet)
-	return &tsafeSet{usResult, sync.RWMutex{}}
-}
diff --git a/vendor/go.etcd.io/etcd/raft/OWNERS b/vendor/go.etcd.io/etcd/raft/OWNERS
deleted file mode 100644
index ab78106..0000000
--- a/vendor/go.etcd.io/etcd/raft/OWNERS
+++ /dev/null
@@ -1,19 +0,0 @@
-approvers:
-- heyitsanthony
-- philips
-- fanminshi
-- gyuho
-- mitake
-- jpbetz
-- xiang90
-- bdarnell
-reviewers:
-- heyitsanthony
-- philips
-- fanminshi
-- gyuho
-- mitake
-- jpbetz
-- xiang90
-- bdarnell
-- tschottdorf
diff --git a/vendor/go.etcd.io/etcd/raft/README.md b/vendor/go.etcd.io/etcd/raft/README.md
deleted file mode 100644
index 83cf040..0000000
--- a/vendor/go.etcd.io/etcd/raft/README.md
+++ /dev/null
@@ -1,197 +0,0 @@
-# Raft library
-
-Raft is a protocol with which a cluster of nodes can maintain a replicated state machine.
-The state machine is kept in sync through the use of a replicated log.
-For more details on Raft, see "In Search of an Understandable Consensus Algorithm"
-(https://raft.github.io/raft.pdf) by Diego Ongaro and John Ousterhout.
-
-This Raft library is stable and feature complete. As of 2016, it is **the most widely used** Raft library in production, serving tens of thousands clusters each day. It powers distributed systems such as etcd, Kubernetes, Docker Swarm, Cloud Foundry Diego, CockroachDB, TiDB, Project Calico, Flannel, and more.
-
-Most Raft implementations have a monolithic design, including storage handling, messaging serialization, and network transport. This library instead follows a minimalistic design philosophy by only implementing the core raft algorithm. This minimalism buys flexibility, determinism, and performance.
-
-To keep the codebase small as well as provide flexibility, the library only implements the Raft algorithm; both network and disk IO are left to the user. Library users must implement their own transportation layer for message passing between Raft peers over the wire. Similarly, users must implement their own storage layer to persist the Raft log and state.
-
-In order to easily test the Raft library, its behavior should be deterministic. To achieve this determinism, the library models Raft as a state machine.  The state machine takes a `Message` as input. A message can either be a local timer update or a network message sent from a remote peer. The state machine's output is a 3-tuple `{[]Messages, []LogEntries, NextState}` consisting of an array of `Messages`, `log entries`, and `Raft state changes`. For state machines with the same state, the same state machine input should always generate the same state machine output.
-
-A simple example application, _raftexample_, is also available to help illustrate how to use this package in practice: https://github.com/etcd-io/etcd/tree/master/contrib/raftexample
-
-# Features
-
-This raft implementation is a full feature implementation of Raft protocol. Features includes:
-
-- Leader election
-- Log replication
-- Log compaction
-- Membership changes
-- Leadership transfer extension
-- Efficient linearizable read-only queries served by both the leader and followers
-  - leader checks with quorum and bypasses Raft log before processing read-only queries
-  - followers asks leader to get a safe read index before processing read-only queries
-- More efficient lease-based linearizable read-only queries served by both the leader and followers
-  - leader bypasses Raft log and processing read-only queries locally
-  - followers asks leader to get a safe read index before processing read-only queries
-  - this approach relies on the clock of the all the machines in raft group
-
-This raft implementation also includes a few optional enhancements:
-
-- Optimistic pipelining to reduce log replication latency
-- Flow control for log replication
-- Batching Raft messages to reduce synchronized network I/O calls
-- Batching log entries to reduce disk synchronized I/O
-- Writing to leader's disk in parallel
-- Internal proposal redirection from followers to leader
-- Automatic stepping down when the leader loses quorum
-- Protection against unbounded log growth when quorum is lost
-
-## Notable Users
-
-- [cockroachdb](https://github.com/cockroachdb/cockroach) A Scalable, Survivable, Strongly-Consistent SQL Database
-- [dgraph](https://github.com/dgraph-io/dgraph) A Scalable, Distributed, Low Latency, High Throughput Graph Database
-- [etcd](https://github.com/etcd-io/etcd) A distributed reliable key-value store
-- [tikv](https://github.com/pingcap/tikv) A Distributed transactional key value database powered by Rust and Raft
-- [swarmkit](https://github.com/docker/swarmkit) A toolkit for orchestrating distributed systems at any scale.
-- [chain core](https://github.com/chain/chain) Software for operating permissioned, multi-asset blockchain networks
-
-## Usage
-
-The primary object in raft is a Node. Either start a Node from scratch using raft.StartNode or start a Node from some initial state using raft.RestartNode.
-
-To start a three-node cluster
-```go
-  storage := raft.NewMemoryStorage()
-  c := &Config{
-    ID:              0x01,
-    ElectionTick:    10,
-    HeartbeatTick:   1,
-    Storage:         storage,
-    MaxSizePerMsg:   4096,
-    MaxInflightMsgs: 256,
-  }
-  // Set peer list to the other nodes in the cluster.
-  // Note that they need to be started separately as well.
-  n := raft.StartNode(c, []raft.Peer{{ID: 0x02}, {ID: 0x03}})
-```
-
-Start a single node cluster, like so:
-```go
-  // Create storage and config as shown above.
-  // Set peer list to itself, so this node can become the leader of this single-node cluster.
-  peers := []raft.Peer{{ID: 0x01}}
-  n := raft.StartNode(c, peers)
-```
-
-To allow a new node to join this cluster, do not pass in any peers. First, add the node to the existing cluster by calling `ProposeConfChange` on any existing node inside the cluster. Then, start the node with an empty peer list, like so:
-```go
-  // Create storage and config as shown above.
-  n := raft.StartNode(c, nil)
-```
-
-To restart a node from previous state:
-```go
-  storage := raft.NewMemoryStorage()
-
-  // Recover the in-memory storage from persistent snapshot, state and entries.
-  storage.ApplySnapshot(snapshot)
-  storage.SetHardState(state)
-  storage.Append(entries)
-
-  c := &Config{
-    ID:              0x01,
-    ElectionTick:    10,
-    HeartbeatTick:   1,
-    Storage:         storage,
-    MaxSizePerMsg:   4096,
-    MaxInflightMsgs: 256,
-  }
-
-  // Restart raft without peer information.
-  // Peer information is already included in the storage.
-  n := raft.RestartNode(c)
-```
-
-After creating a Node, the user has a few responsibilities:
-
-First, read from the Node.Ready() channel and process the updates it contains. These steps may be performed in parallel, except as noted in step 2.
-
-1. Write Entries, HardState and Snapshot to persistent storage in order, i.e. Entries first, then HardState and Snapshot if they are not empty. If persistent storage supports atomic writes then all of them can be written together. Note that when writing an Entry with Index i, any previously-persisted entries with Index >= i must be discarded.
-
-2. Send all Messages to the nodes named in the To field. It is important that no messages be sent until the latest HardState has been persisted to disk, and all Entries written by any previous Ready batch (Messages may be sent while entries from the same batch are being persisted). To reduce the I/O latency, an optimization can be applied to make leader write to disk in parallel with its followers (as explained at section 10.2.1 in Raft thesis). If any Message has type MsgSnap, call Node.ReportSnapshot() after it has been sent (these messages may be large). Note: Marshalling messages is not thread-safe; it is important to make sure that no new entries are persisted while marshalling. The easiest way to achieve this is to serialise the messages directly inside the main raft loop.
-
-3. Apply Snapshot (if any) and CommittedEntries to the state machine. If any committed Entry has Type EntryConfChange, call Node.ApplyConfChange() to apply it to the node. The configuration change may be cancelled at this point by setting the NodeID field to zero before calling ApplyConfChange (but ApplyConfChange must be called one way or the other, and the decision to cancel must be based solely on the state machine and not external information such as the observed health of the node).
-
-4. Call Node.Advance() to signal readiness for the next batch of updates. This may be done at any time after step 1, although all updates must be processed in the order they were returned by Ready.
-
-Second, all persisted log entries must be made available via an implementation of the Storage interface. The provided MemoryStorage type can be used for this (if repopulating its state upon a restart), or a custom disk-backed implementation can be supplied.
-
-Third, after receiving a message from another node, pass it to Node.Step:
-
-```go
-	func recvRaftRPC(ctx context.Context, m raftpb.Message) {
-		n.Step(ctx, m)
-	}
-```
-
-Finally, call `Node.Tick()` at regular intervals (probably via a `time.Ticker`). Raft has two important timeouts: heartbeat and the election timeout. However, internally to the raft package time is represented by an abstract "tick".
-
-The total state machine handling loop will look something like this:
-
-```go
-  for {
-    select {
-    case <-s.Ticker:
-      n.Tick()
-    case rd := <-s.Node.Ready():
-      saveToStorage(rd.HardState, rd.Entries, rd.Snapshot)
-      send(rd.Messages)
-      if !raft.IsEmptySnap(rd.Snapshot) {
-        processSnapshot(rd.Snapshot)
-      }
-      for _, entry := range rd.CommittedEntries {
-        process(entry)
-        if entry.Type == raftpb.EntryConfChange {
-          var cc raftpb.ConfChange
-          cc.Unmarshal(entry.Data)
-          s.Node.ApplyConfChange(cc)
-        }
-      }
-      s.Node.Advance()
-    case <-s.done:
-      return
-    }
-  }
-```
-
-To propose changes to the state machine from the node to take application data, serialize it into a byte slice and call:
-
-```go
-	n.Propose(ctx, data)
-```
-
-If the proposal is committed, data will appear in committed entries with type raftpb.EntryNormal. There is no guarantee that a proposed command will be committed; the command may have to be reproposed after a timeout.
-
-To add or remove node in a cluster, build ConfChange struct 'cc' and call:
-
-```go
-	n.ProposeConfChange(ctx, cc)
-```
-
-After config change is committed, some committed entry with type raftpb.EntryConfChange will be returned. This must be applied to node through:
-
-```go
-	var cc raftpb.ConfChange
-	cc.Unmarshal(data)
-	n.ApplyConfChange(cc)
-```
-
-Note: An ID represents a unique node in a cluster for all time. A
-given ID MUST be used only once even if the old node has been removed.
-This means that for example IP addresses make poor node IDs since they
-may be reused. Node IDs must be non-zero.
-
-## Implementation notes
-
-This implementation is up to date with the final Raft thesis (https://github.com/ongardie/dissertation/blob/master/stanford.pdf), although this implementation of the membership change protocol differs somewhat from that described in chapter 4. The key invariant that membership changes happen one node at a time is preserved, but in our implementation the membership change takes effect when its entry is applied, not when it is added to the log (so the entry is committed under the old membership instead of the new). This is equivalent in terms of safety, since the old and new configurations are guaranteed to overlap.
-
-To ensure there is no attempt to commit two membership changes at once by matching log positions (which would be unsafe since they should have different quorum requirements), any proposed membership change is simply disallowed while any uncommitted change appears in the leader's log.
-
-This approach introduces a problem when removing a member from a two-member cluster: If one of the members dies before the other one receives the commit of the confchange entry, then the member cannot be removed any more since the cluster cannot make progress. For this reason it is highly recommended to use three or more nodes in every cluster.
diff --git a/vendor/go.etcd.io/etcd/raft/bootstrap.go b/vendor/go.etcd.io/etcd/raft/bootstrap.go
deleted file mode 100644
index bd82b20..0000000
--- a/vendor/go.etcd.io/etcd/raft/bootstrap.go
+++ /dev/null
@@ -1,80 +0,0 @@
-// 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 raft
-
-import (
-	"errors"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-// Bootstrap initializes the RawNode for first use by appending configuration
-// changes for the supplied peers. This method returns an error if the Storage
-// is nonempty.
-//
-// It is recommended that instead of calling this method, applications bootstrap
-// their state manually by setting up a Storage that has a first index > 1 and
-// which stores the desired ConfState as its InitialState.
-func (rn *RawNode) Bootstrap(peers []Peer) error {
-	if len(peers) == 0 {
-		return errors.New("must provide at least one peer to Bootstrap")
-	}
-	lastIndex, err := rn.raft.raftLog.storage.LastIndex()
-	if err != nil {
-		return err
-	}
-
-	if lastIndex != 0 {
-		return errors.New("can't bootstrap a nonempty Storage")
-	}
-
-	// We've faked out initial entries above, but nothing has been
-	// persisted. Start with an empty HardState (thus the first Ready will
-	// emit a HardState update for the app to persist).
-	rn.prevHardSt = emptyState
-
-	// TODO(tbg): remove StartNode and give the application the right tools to
-	// bootstrap the initial membership in a cleaner way.
-	rn.raft.becomeFollower(1, None)
-	ents := make([]pb.Entry, len(peers))
-	for i, peer := range peers {
-		cc := pb.ConfChange{Type: pb.ConfChangeAddNode, NodeID: peer.ID, Context: peer.Context}
-		data, err := cc.Marshal()
-		if err != nil {
-			return err
-		}
-
-		ents[i] = pb.Entry{Type: pb.EntryConfChange, Term: 1, Index: uint64(i + 1), Data: data}
-	}
-	rn.raft.raftLog.append(ents...)
-
-	// Now apply them, mainly so that the application can call Campaign
-	// immediately after StartNode in tests. Note that these nodes will
-	// be added to raft twice: here and when the application's Ready
-	// loop calls ApplyConfChange. The calls to addNode must come after
-	// all calls to raftLog.append so progress.next is set after these
-	// bootstrapping entries (it is an error if we try to append these
-	// entries since they have already been committed).
-	// We do not set raftLog.applied so the application will be able
-	// to observe all conf changes via Ready.CommittedEntries.
-	//
-	// TODO(bdarnell): These entries are still unstable; do we need to preserve
-	// the invariant that committed < unstable?
-	rn.raft.raftLog.committed = uint64(len(ents))
-	for _, peer := range peers {
-		rn.raft.applyConfChange(pb.ConfChange{NodeID: peer.ID, Type: pb.ConfChangeAddNode}.AsV2())
-	}
-	return nil
-}
diff --git a/vendor/go.etcd.io/etcd/raft/confchange/confchange.go b/vendor/go.etcd.io/etcd/raft/confchange/confchange.go
deleted file mode 100644
index a0dc486..0000000
--- a/vendor/go.etcd.io/etcd/raft/confchange/confchange.go
+++ /dev/null
@@ -1,425 +0,0 @@
-// 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 confchange
-
-import (
-	"errors"
-	"fmt"
-	"strings"
-
-	"go.etcd.io/etcd/raft/quorum"
-	pb "go.etcd.io/etcd/raft/raftpb"
-	"go.etcd.io/etcd/raft/tracker"
-)
-
-// Changer facilitates configuration changes. It exposes methods to handle
-// simple and joint consensus while performing the proper validation that allows
-// refusing invalid configuration changes before they affect the active
-// configuration.
-type Changer struct {
-	Tracker   tracker.ProgressTracker
-	LastIndex uint64
-}
-
-// EnterJoint verifies that the outgoing (=right) majority config of the joint
-// config is empty and initializes it with a copy of the incoming (=left)
-// majority config. That is, it transitions from
-//
-//     (1 2 3)&&()
-// to
-//     (1 2 3)&&(1 2 3).
-//
-// The supplied changes are then applied to the incoming majority config,
-// resulting in a joint configuration that in terms of the Raft thesis[1]
-// (Section 4.3) corresponds to `C_{new,old}`.
-//
-// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
-func (c Changer) EnterJoint(autoLeave bool, ccs ...pb.ConfChangeSingle) (tracker.Config, tracker.ProgressMap, error) {
-	cfg, prs, err := c.checkAndCopy()
-	if err != nil {
-		return c.err(err)
-	}
-	if joint(cfg) {
-		err := errors.New("config is already joint")
-		return c.err(err)
-	}
-	if len(incoming(cfg.Voters)) == 0 {
-		// We allow adding nodes to an empty config for convenience (testing and
-		// bootstrap), but you can't enter a joint state.
-		err := errors.New("can't make a zero-voter config joint")
-		return c.err(err)
-	}
-	// Clear the outgoing config.
-	*outgoingPtr(&cfg.Voters) = quorum.MajorityConfig{}
-	// Copy incoming to outgoing.
-	for id := range incoming(cfg.Voters) {
-		outgoing(cfg.Voters)[id] = struct{}{}
-	}
-
-	if err := c.apply(&cfg, prs, ccs...); err != nil {
-		return c.err(err)
-	}
-	cfg.AutoLeave = autoLeave
-	return checkAndReturn(cfg, prs)
-}
-
-// LeaveJoint transitions out of a joint configuration. It is an error to call
-// this method if the configuration is not joint, i.e. if the outgoing majority
-// config Voters[1] is empty.
-//
-// The outgoing majority config of the joint configuration will be removed,
-// that is, the incoming config is promoted as the sole decision maker. In the
-// notation of the Raft thesis[1] (Section 4.3), this method transitions from
-// `C_{new,old}` into `C_new`.
-//
-// At the same time, any staged learners (LearnersNext) the addition of which
-// was held back by an overlapping voter in the former outgoing config will be
-// inserted into Learners.
-//
-// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
-func (c Changer) LeaveJoint() (tracker.Config, tracker.ProgressMap, error) {
-	cfg, prs, err := c.checkAndCopy()
-	if err != nil {
-		return c.err(err)
-	}
-	if !joint(cfg) {
-		err := errors.New("can't leave a non-joint config")
-		return c.err(err)
-	}
-	if len(outgoing(cfg.Voters)) == 0 {
-		err := fmt.Errorf("configuration is not joint: %v", cfg)
-		return c.err(err)
-	}
-	for id := range cfg.LearnersNext {
-		nilAwareAdd(&cfg.Learners, id)
-		prs[id].IsLearner = true
-	}
-	cfg.LearnersNext = nil
-
-	for id := range outgoing(cfg.Voters) {
-		_, isVoter := incoming(cfg.Voters)[id]
-		_, isLearner := cfg.Learners[id]
-
-		if !isVoter && !isLearner {
-			delete(prs, id)
-		}
-	}
-	*outgoingPtr(&cfg.Voters) = nil
-	cfg.AutoLeave = false
-
-	return checkAndReturn(cfg, prs)
-}
-
-// Simple carries out a series of configuration changes that (in aggregate)
-// mutates the incoming majority config Voters[0] by at most one. This method
-// will return an error if that is not the case, if the resulting quorum is
-// zero, or if the configuration is in a joint state (i.e. if there is an
-// outgoing configuration).
-func (c Changer) Simple(ccs ...pb.ConfChangeSingle) (tracker.Config, tracker.ProgressMap, error) {
-	cfg, prs, err := c.checkAndCopy()
-	if err != nil {
-		return c.err(err)
-	}
-	if joint(cfg) {
-		err := errors.New("can't apply simple config change in joint config")
-		return c.err(err)
-	}
-	if err := c.apply(&cfg, prs, ccs...); err != nil {
-		return c.err(err)
-	}
-	if n := symdiff(incoming(c.Tracker.Voters), incoming(cfg.Voters)); n > 1 {
-		return tracker.Config{}, nil, errors.New("more than one voter changed without entering joint config")
-	}
-	if err := checkInvariants(cfg, prs); err != nil {
-		return tracker.Config{}, tracker.ProgressMap{}, nil
-	}
-
-	return checkAndReturn(cfg, prs)
-}
-
-// apply a change to the configuration. By convention, changes to voters are
-// always made to the incoming majority config Voters[0]. Voters[1] is either
-// empty or preserves the outgoing majority configuration while in a joint state.
-func (c Changer) apply(cfg *tracker.Config, prs tracker.ProgressMap, ccs ...pb.ConfChangeSingle) error {
-	for _, cc := range ccs {
-		if cc.NodeID == 0 {
-			// etcd replaces the NodeID with zero if it decides (downstream of
-			// raft) to not apply a change, so we have to have explicit code
-			// here to ignore these.
-			continue
-		}
-		switch cc.Type {
-		case pb.ConfChangeAddNode:
-			c.makeVoter(cfg, prs, cc.NodeID)
-		case pb.ConfChangeAddLearnerNode:
-			c.makeLearner(cfg, prs, cc.NodeID)
-		case pb.ConfChangeRemoveNode:
-			c.remove(cfg, prs, cc.NodeID)
-		case pb.ConfChangeUpdateNode:
-		default:
-			return fmt.Errorf("unexpected conf type %d", cc.Type)
-		}
-	}
-	if len(incoming(cfg.Voters)) == 0 {
-		return errors.New("removed all voters")
-	}
-	return nil
-}
-
-// makeVoter adds or promotes the given ID to be a voter in the incoming
-// majority config.
-func (c Changer) makeVoter(cfg *tracker.Config, prs tracker.ProgressMap, id uint64) {
-	pr := prs[id]
-	if pr == nil {
-		c.initProgress(cfg, prs, id, false /* isLearner */)
-		return
-	}
-
-	pr.IsLearner = false
-	nilAwareDelete(&cfg.Learners, id)
-	nilAwareDelete(&cfg.LearnersNext, id)
-	incoming(cfg.Voters)[id] = struct{}{}
-	return
-}
-
-// makeLearner makes the given ID a learner or stages it to be a learner once
-// an active joint configuration is exited.
-//
-// The former happens when the peer is not a part of the outgoing config, in
-// which case we either add a new learner or demote a voter in the incoming
-// config.
-//
-// The latter case occurs when the configuration is joint and the peer is a
-// voter in the outgoing config. In that case, we do not want to add the peer
-// as a learner because then we'd have to track a peer as a voter and learner
-// simultaneously. Instead, we add the learner to LearnersNext, so that it will
-// be added to Learners the moment the outgoing config is removed by
-// LeaveJoint().
-func (c Changer) makeLearner(cfg *tracker.Config, prs tracker.ProgressMap, id uint64) {
-	pr := prs[id]
-	if pr == nil {
-		c.initProgress(cfg, prs, id, true /* isLearner */)
-		return
-	}
-	if pr.IsLearner {
-		return
-	}
-	// Remove any existing voter in the incoming config...
-	c.remove(cfg, prs, id)
-	// ... but save the Progress.
-	prs[id] = pr
-	// Use LearnersNext if we can't add the learner to Learners directly, i.e.
-	// if the peer is still tracked as a voter in the outgoing config. It will
-	// be turned into a learner in LeaveJoint().
-	//
-	// Otherwise, add a regular learner right away.
-	if _, onRight := outgoing(cfg.Voters)[id]; onRight {
-		nilAwareAdd(&cfg.LearnersNext, id)
-	} else {
-		pr.IsLearner = true
-		nilAwareAdd(&cfg.Learners, id)
-	}
-}
-
-// remove this peer as a voter or learner from the incoming config.
-func (c Changer) remove(cfg *tracker.Config, prs tracker.ProgressMap, id uint64) {
-	if _, ok := prs[id]; !ok {
-		return
-	}
-
-	delete(incoming(cfg.Voters), id)
-	nilAwareDelete(&cfg.Learners, id)
-	nilAwareDelete(&cfg.LearnersNext, id)
-
-	// If the peer is still a voter in the outgoing config, keep the Progress.
-	if _, onRight := outgoing(cfg.Voters)[id]; !onRight {
-		delete(prs, id)
-	}
-}
-
-// initProgress initializes a new progress for the given node or learner.
-func (c Changer) initProgress(cfg *tracker.Config, prs tracker.ProgressMap, id uint64, isLearner bool) {
-	if !isLearner {
-		incoming(cfg.Voters)[id] = struct{}{}
-	} else {
-		nilAwareAdd(&cfg.Learners, id)
-	}
-	prs[id] = &tracker.Progress{
-		// Initializing the Progress with the last index means that the follower
-		// can be probed (with the last index).
-		//
-		// TODO(tbg): seems awfully optimistic. Using the first index would be
-		// better. The general expectation here is that the follower has no log
-		// at all (and will thus likely need a snapshot), though the app may
-		// have applied a snapshot out of band before adding the replica (thus
-		// making the first index the better choice).
-		Next:      c.LastIndex,
-		Match:     0,
-		Inflights: tracker.NewInflights(c.Tracker.MaxInflight),
-		IsLearner: isLearner,
-		// When a node is first added, we should mark it as recently active.
-		// Otherwise, CheckQuorum may cause us to step down if it is invoked
-		// before the added node has had a chance to communicate with us.
-		RecentActive: true,
-	}
-}
-
-// checkInvariants makes sure that the config and progress are compatible with
-// each other. This is used to check both what the Changer is initialized with,
-// as well as what it returns.
-func checkInvariants(cfg tracker.Config, prs tracker.ProgressMap) error {
-	// NB: intentionally allow the empty config. In production we'll never see a
-	// non-empty config (we prevent it from being created) but we will need to
-	// be able to *create* an initial config, for example during bootstrap (or
-	// during tests). Instead of having to hand-code this, we allow
-	// transitioning from an empty config into any other legal and non-empty
-	// config.
-	for _, ids := range []map[uint64]struct{}{
-		cfg.Voters.IDs(),
-		cfg.Learners,
-		cfg.LearnersNext,
-	} {
-		for id := range ids {
-			if _, ok := prs[id]; !ok {
-				return fmt.Errorf("no progress for %d", id)
-			}
-		}
-	}
-
-	// Any staged learner was staged because it could not be directly added due
-	// to a conflicting voter in the outgoing config.
-	for id := range cfg.LearnersNext {
-		if _, ok := outgoing(cfg.Voters)[id]; !ok {
-			return fmt.Errorf("%d is in LearnersNext, but not Voters[1]", id)
-		}
-		if prs[id].IsLearner {
-			return fmt.Errorf("%d is in LearnersNext, but is already marked as learner", id)
-		}
-	}
-	// Conversely Learners and Voters doesn't intersect at all.
-	for id := range cfg.Learners {
-		if _, ok := outgoing(cfg.Voters)[id]; ok {
-			return fmt.Errorf("%d is in Learners and Voters[1]", id)
-		}
-		if _, ok := incoming(cfg.Voters)[id]; ok {
-			return fmt.Errorf("%d is in Learners and Voters[0]", id)
-		}
-		if !prs[id].IsLearner {
-			return fmt.Errorf("%d is in Learners, but is not marked as learner", id)
-		}
-	}
-
-	if !joint(cfg) {
-		// We enforce that empty maps are nil instead of zero.
-		if outgoing(cfg.Voters) != nil {
-			return fmt.Errorf("Voters[1] must be nil when not joint")
-		}
-		if cfg.LearnersNext != nil {
-			return fmt.Errorf("LearnersNext must be nil when not joint")
-		}
-		if cfg.AutoLeave {
-			return fmt.Errorf("AutoLeave must be false when not joint")
-		}
-	}
-
-	return nil
-}
-
-// checkAndCopy copies the tracker's config and progress map (deeply enough for
-// the purposes of the Changer) and returns those copies. It returns an error
-// if checkInvariants does.
-func (c Changer) checkAndCopy() (tracker.Config, tracker.ProgressMap, error) {
-	cfg := c.Tracker.Config.Clone()
-	prs := tracker.ProgressMap{}
-
-	for id, pr := range c.Tracker.Progress {
-		// A shallow copy is enough because we only mutate the Learner field.
-		ppr := *pr
-		prs[id] = &ppr
-	}
-	return checkAndReturn(cfg, prs)
-}
-
-// checkAndReturn calls checkInvariants on the input and returns either the
-// resulting error or the input.
-func checkAndReturn(cfg tracker.Config, prs tracker.ProgressMap) (tracker.Config, tracker.ProgressMap, error) {
-	if err := checkInvariants(cfg, prs); err != nil {
-		return tracker.Config{}, tracker.ProgressMap{}, err
-	}
-	return cfg, prs, nil
-}
-
-// err returns zero values and an error.
-func (c Changer) err(err error) (tracker.Config, tracker.ProgressMap, error) {
-	return tracker.Config{}, nil, err
-}
-
-// nilAwareAdd populates a map entry, creating the map if necessary.
-func nilAwareAdd(m *map[uint64]struct{}, id uint64) {
-	if *m == nil {
-		*m = map[uint64]struct{}{}
-	}
-	(*m)[id] = struct{}{}
-}
-
-// nilAwareDelete deletes from a map, nil'ing the map itself if it is empty after.
-func nilAwareDelete(m *map[uint64]struct{}, id uint64) {
-	if *m == nil {
-		return
-	}
-	delete(*m, id)
-	if len(*m) == 0 {
-		*m = nil
-	}
-}
-
-// symdiff returns the count of the symmetric difference between the sets of
-// uint64s, i.e. len( (l - r) \union (r - l)).
-func symdiff(l, r map[uint64]struct{}) int {
-	var n int
-	pairs := [][2]quorum.MajorityConfig{
-		{l, r}, // count elems in l but not in r
-		{r, l}, // count elems in r but not in l
-	}
-	for _, p := range pairs {
-		for id := range p[0] {
-			if _, ok := p[1][id]; !ok {
-				n++
-			}
-		}
-	}
-	return n
-}
-
-func joint(cfg tracker.Config) bool {
-	return len(outgoing(cfg.Voters)) > 0
-}
-
-func incoming(voters quorum.JointConfig) quorum.MajorityConfig      { return voters[0] }
-func outgoing(voters quorum.JointConfig) quorum.MajorityConfig      { return voters[1] }
-func outgoingPtr(voters *quorum.JointConfig) *quorum.MajorityConfig { return &voters[1] }
-
-// Describe prints the type and NodeID of the configuration changes as a
-// space-delimited string.
-func Describe(ccs ...pb.ConfChangeSingle) string {
-	var buf strings.Builder
-	for _, cc := range ccs {
-		if buf.Len() > 0 {
-			buf.WriteByte(' ')
-		}
-		fmt.Fprintf(&buf, "%s(%d)", cc.Type, cc.NodeID)
-	}
-	return buf.String()
-}
diff --git a/vendor/go.etcd.io/etcd/raft/confchange/restore.go b/vendor/go.etcd.io/etcd/raft/confchange/restore.go
deleted file mode 100644
index 724068d..0000000
--- a/vendor/go.etcd.io/etcd/raft/confchange/restore.go
+++ /dev/null
@@ -1,155 +0,0 @@
-// 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 confchange
-
-import (
-	pb "go.etcd.io/etcd/raft/raftpb"
-	"go.etcd.io/etcd/raft/tracker"
-)
-
-// toConfChangeSingle translates a conf state into 1) a slice of operations creating
-// first the config that will become the outgoing one, and then the incoming one, and
-// b) another slice that, when applied to the config resulted from 1), represents the
-// ConfState.
-func toConfChangeSingle(cs pb.ConfState) (out []pb.ConfChangeSingle, in []pb.ConfChangeSingle) {
-	// Example to follow along this code:
-	// voters=(1 2 3) learners=(5) outgoing=(1 2 4 6) learners_next=(4)
-	//
-	// This means that before entering the joint config, the configuration
-	// had voters (1 2 4) and perhaps some learners that are already gone.
-	// The new set of voters is (1 2 3), i.e. (1 2) were kept around, and (4 6)
-	// are no longer voters; however 4 is poised to become a learner upon leaving
-	// the joint state.
-	// We can't tell whether 5 was a learner before entering the joint config,
-	// but it doesn't matter (we'll pretend that it wasn't).
-	//
-	// The code below will construct
-	// outgoing = add 1; add 2; add 4; add 6
-	// incoming = remove 1; remove 2; remove 4; remove 6
-	//            add 1;    add 2;    add 3;
-	//            add-learner 5;
-	//            add-learner 4;
-	//
-	// So, when starting with an empty config, after applying 'outgoing' we have
-	//
-	//   quorum=(1 2 4 6)
-	//
-	// From which we enter a joint state via 'incoming'
-	//
-	//   quorum=(1 2 3)&&(1 2 4 6) learners=(5) learners_next=(4)
-	//
-	// as desired.
-
-	for _, id := range cs.VotersOutgoing {
-		// If there are outgoing voters, first add them one by one so that the
-		// (non-joint) config has them all.
-		out = append(out, pb.ConfChangeSingle{
-			Type:   pb.ConfChangeAddNode,
-			NodeID: id,
-		})
-
-	}
-
-	// We're done constructing the outgoing slice, now on to the incoming one
-	// (which will apply on top of the config created by the outgoing slice).
-
-	// First, we'll remove all of the outgoing voters.
-	for _, id := range cs.VotersOutgoing {
-		in = append(in, pb.ConfChangeSingle{
-			Type:   pb.ConfChangeRemoveNode,
-			NodeID: id,
-		})
-	}
-	// Then we'll add the incoming voters and learners.
-	for _, id := range cs.Voters {
-		in = append(in, pb.ConfChangeSingle{
-			Type:   pb.ConfChangeAddNode,
-			NodeID: id,
-		})
-	}
-	for _, id := range cs.Learners {
-		in = append(in, pb.ConfChangeSingle{
-			Type:   pb.ConfChangeAddLearnerNode,
-			NodeID: id,
-		})
-	}
-	// Same for LearnersNext; these are nodes we want to be learners but which
-	// are currently voters in the outgoing config.
-	for _, id := range cs.LearnersNext {
-		in = append(in, pb.ConfChangeSingle{
-			Type:   pb.ConfChangeAddLearnerNode,
-			NodeID: id,
-		})
-	}
-	return out, in
-}
-
-func chain(chg Changer, ops ...func(Changer) (tracker.Config, tracker.ProgressMap, error)) (tracker.Config, tracker.ProgressMap, error) {
-	for _, op := range ops {
-		cfg, prs, err := op(chg)
-		if err != nil {
-			return tracker.Config{}, nil, err
-		}
-		chg.Tracker.Config = cfg
-		chg.Tracker.Progress = prs
-	}
-	return chg.Tracker.Config, chg.Tracker.Progress, nil
-}
-
-// Restore takes a Changer (which must represent an empty configuration), and
-// runs a sequence of changes enacting the configuration described in the
-// ConfState.
-//
-// TODO(tbg) it's silly that this takes a Changer. Unravel this by making sure
-// the Changer only needs a ProgressMap (not a whole Tracker) at which point
-// this can just take LastIndex and MaxInflight directly instead and cook up
-// the results from that alone.
-func Restore(chg Changer, cs pb.ConfState) (tracker.Config, tracker.ProgressMap, error) {
-	outgoing, incoming := toConfChangeSingle(cs)
-
-	var ops []func(Changer) (tracker.Config, tracker.ProgressMap, error)
-
-	if len(outgoing) == 0 {
-		// No outgoing config, so just apply the incoming changes one by one.
-		for _, cc := range incoming {
-			cc := cc // loop-local copy
-			ops = append(ops, func(chg Changer) (tracker.Config, tracker.ProgressMap, error) {
-				return chg.Simple(cc)
-			})
-		}
-	} else {
-		// The ConfState describes a joint configuration.
-		//
-		// First, apply all of the changes of the outgoing config one by one, so
-		// that it temporarily becomes the incoming active config. For example,
-		// if the config is (1 2 3)&(2 3 4), this will establish (2 3 4)&().
-		for _, cc := range outgoing {
-			cc := cc // loop-local copy
-			ops = append(ops, func(chg Changer) (tracker.Config, tracker.ProgressMap, error) {
-				return chg.Simple(cc)
-			})
-		}
-		// Now enter the joint state, which rotates the above additions into the
-		// outgoing config, and adds the incoming config in. Continuing the
-		// example above, we'd get (1 2 3)&(2 3 4), i.e. the incoming operations
-		// would be removing 2,3,4 and then adding in 1,2,3 while transitioning
-		// into a joint state.
-		ops = append(ops, func(chg Changer) (tracker.Config, tracker.ProgressMap, error) {
-			return chg.EnterJoint(cs.AutoLeave, incoming...)
-		})
-	}
-
-	return chain(chg, ops...)
-}
diff --git a/vendor/go.etcd.io/etcd/raft/doc.go b/vendor/go.etcd.io/etcd/raft/doc.go
deleted file mode 100644
index 68fe6f0..0000000
--- a/vendor/go.etcd.io/etcd/raft/doc.go
+++ /dev/null
@@ -1,300 +0,0 @@
-// 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 raft sends and receives messages in the Protocol Buffer format
-defined in the raftpb package.
-
-Raft is a protocol with which a cluster of nodes can maintain a replicated state machine.
-The state machine is kept in sync through the use of a replicated log.
-For more details on Raft, see "In Search of an Understandable Consensus Algorithm"
-(https://raft.github.io/raft.pdf) by Diego Ongaro and John Ousterhout.
-
-A simple example application, _raftexample_, is also available to help illustrate
-how to use this package in practice:
-https://github.com/etcd-io/etcd/tree/master/contrib/raftexample
-
-Usage
-
-The primary object in raft is a Node. You either start a Node from scratch
-using raft.StartNode or start a Node from some initial state using raft.RestartNode.
-
-To start a node from scratch:
-
-  storage := raft.NewMemoryStorage()
-  c := &Config{
-    ID:              0x01,
-    ElectionTick:    10,
-    HeartbeatTick:   1,
-    Storage:         storage,
-    MaxSizePerMsg:   4096,
-    MaxInflightMsgs: 256,
-  }
-  n := raft.StartNode(c, []raft.Peer{{ID: 0x02}, {ID: 0x03}})
-
-To restart a node from previous state:
-
-  storage := raft.NewMemoryStorage()
-
-  // recover the in-memory storage from persistent
-  // snapshot, state and entries.
-  storage.ApplySnapshot(snapshot)
-  storage.SetHardState(state)
-  storage.Append(entries)
-
-  c := &Config{
-    ID:              0x01,
-    ElectionTick:    10,
-    HeartbeatTick:   1,
-    Storage:         storage,
-    MaxSizePerMsg:   4096,
-    MaxInflightMsgs: 256,
-  }
-
-  // restart raft without peer information.
-  // peer information is already included in the storage.
-  n := raft.RestartNode(c)
-
-Now that you are holding onto a Node you have a few responsibilities:
-
-First, you must read from the Node.Ready() channel and process the updates
-it contains. These steps may be performed in parallel, except as noted in step
-2.
-
-1. Write HardState, Entries, and Snapshot to persistent storage if they are
-not empty. Note that when writing an Entry with Index i, any
-previously-persisted entries with Index >= i must be discarded.
-
-2. Send all Messages to the nodes named in the To field. It is important that
-no messages be sent until the latest HardState has been persisted to disk,
-and all Entries written by any previous Ready batch (Messages may be sent while
-entries from the same batch are being persisted). To reduce the I/O latency, an
-optimization can be applied to make leader write to disk in parallel with its
-followers (as explained at section 10.2.1 in Raft thesis). If any Message has type
-MsgSnap, call Node.ReportSnapshot() after it has been sent (these messages may be
-large).
-
-Note: Marshalling messages is not thread-safe; it is important that you
-make sure that no new entries are persisted while marshalling.
-The easiest way to achieve this is to serialize the messages directly inside
-your main raft loop.
-
-3. Apply Snapshot (if any) and CommittedEntries to the state machine.
-If any committed Entry has Type EntryConfChange, call Node.ApplyConfChange()
-to apply it to the node. The configuration change may be cancelled at this point
-by setting the NodeID field to zero before calling ApplyConfChange
-(but ApplyConfChange must be called one way or the other, and the decision to cancel
-must be based solely on the state machine and not external information such as
-the observed health of the node).
-
-4. Call Node.Advance() to signal readiness for the next batch of updates.
-This may be done at any time after step 1, although all updates must be processed
-in the order they were returned by Ready.
-
-Second, all persisted log entries must be made available via an
-implementation of the Storage interface. The provided MemoryStorage
-type can be used for this (if you repopulate its state upon a
-restart), or you can supply your own disk-backed implementation.
-
-Third, when you receive a message from another node, pass it to Node.Step:
-
-	func recvRaftRPC(ctx context.Context, m raftpb.Message) {
-		n.Step(ctx, m)
-	}
-
-Finally, you need to call Node.Tick() at regular intervals (probably
-via a time.Ticker). Raft has two important timeouts: heartbeat and the
-election timeout. However, internally to the raft package time is
-represented by an abstract "tick".
-
-The total state machine handling loop will look something like this:
-
-  for {
-    select {
-    case <-s.Ticker:
-      n.Tick()
-    case rd := <-s.Node.Ready():
-      saveToStorage(rd.State, rd.Entries, rd.Snapshot)
-      send(rd.Messages)
-      if !raft.IsEmptySnap(rd.Snapshot) {
-        processSnapshot(rd.Snapshot)
-      }
-      for _, entry := range rd.CommittedEntries {
-        process(entry)
-        if entry.Type == raftpb.EntryConfChange {
-          var cc raftpb.ConfChange
-          cc.Unmarshal(entry.Data)
-          s.Node.ApplyConfChange(cc)
-        }
-      }
-      s.Node.Advance()
-    case <-s.done:
-      return
-    }
-  }
-
-To propose changes to the state machine from your node take your application
-data, serialize it into a byte slice and call:
-
-	n.Propose(ctx, data)
-
-If the proposal is committed, data will appear in committed entries with type
-raftpb.EntryNormal. There is no guarantee that a proposed command will be
-committed; you may have to re-propose after a timeout.
-
-To add or remove a node in a cluster, build ConfChange struct 'cc' and call:
-
-	n.ProposeConfChange(ctx, cc)
-
-After config change is committed, some committed entry with type
-raftpb.EntryConfChange will be returned. You must apply it to node through:
-
-	var cc raftpb.ConfChange
-	cc.Unmarshal(data)
-	n.ApplyConfChange(cc)
-
-Note: An ID represents a unique node in a cluster for all time. A
-given ID MUST be used only once even if the old node has been removed.
-This means that for example IP addresses make poor node IDs since they
-may be reused. Node IDs must be non-zero.
-
-Implementation notes
-
-This implementation is up to date with the final Raft thesis
-(https://github.com/ongardie/dissertation/blob/master/stanford.pdf), although our
-implementation of the membership change protocol differs somewhat from
-that described in chapter 4. The key invariant that membership changes
-happen one node at a time is preserved, but in our implementation the
-membership change takes effect when its entry is applied, not when it
-is added to the log (so the entry is committed under the old
-membership instead of the new). This is equivalent in terms of safety,
-since the old and new configurations are guaranteed to overlap.
-
-To ensure that we do not attempt to commit two membership changes at
-once by matching log positions (which would be unsafe since they
-should have different quorum requirements), we simply disallow any
-proposed membership change while any uncommitted change appears in
-the leader's log.
-
-This approach introduces a problem when you try to remove a member
-from a two-member cluster: If one of the members dies before the
-other one receives the commit of the confchange entry, then the member
-cannot be removed any more since the cluster cannot make progress.
-For this reason it is highly recommended to use three or more nodes in
-every cluster.
-
-MessageType
-
-Package raft sends and receives message in Protocol Buffer format (defined
-in raftpb package). Each state (follower, candidate, leader) implements its
-own 'step' method ('stepFollower', 'stepCandidate', 'stepLeader') when
-advancing with the given raftpb.Message. Each step is determined by its
-raftpb.MessageType. Note that every step is checked by one common method
-'Step' that safety-checks the terms of node and incoming message to prevent
-stale log entries:
-
-	'MsgHup' is used for election. If a node is a follower or candidate, the
-	'tick' function in 'raft' struct is set as 'tickElection'. If a follower or
-	candidate has not received any heartbeat before the election timeout, it
-	passes 'MsgHup' to its Step method and becomes (or remains) a candidate to
-	start a new election.
-
-	'MsgBeat' is an internal type that signals the leader to send a heartbeat of
-	the 'MsgHeartbeat' type. If a node is a leader, the 'tick' function in
-	the 'raft' struct is set as 'tickHeartbeat', and triggers the leader to
-	send periodic 'MsgHeartbeat' messages to its followers.
-
-	'MsgProp' proposes to append data to its log entries. This is a special
-	type to redirect proposals to leader. Therefore, send method overwrites
-	raftpb.Message's term with its HardState's term to avoid attaching its
-	local term to 'MsgProp'. When 'MsgProp' is passed to the leader's 'Step'
-	method, the leader first calls the 'appendEntry' method to append entries
-	to its log, and then calls 'bcastAppend' method to send those entries to
-	its peers. When passed to candidate, 'MsgProp' is dropped. When passed to
-	follower, 'MsgProp' is stored in follower's mailbox(msgs) by the send
-	method. It is stored with sender's ID and later forwarded to leader by
-	rafthttp package.
-
-	'MsgApp' contains log entries to replicate. A leader calls bcastAppend,
-	which calls sendAppend, which sends soon-to-be-replicated logs in 'MsgApp'
-	type. When 'MsgApp' is passed to candidate's Step method, candidate reverts
-	back to follower, because it indicates that there is a valid leader sending
-	'MsgApp' messages. Candidate and follower respond to this message in
-	'MsgAppResp' type.
-
-	'MsgAppResp' is response to log replication request('MsgApp'). When
-	'MsgApp' is passed to candidate or follower's Step method, it responds by
-	calling 'handleAppendEntries' method, which sends 'MsgAppResp' to raft
-	mailbox.
-
-	'MsgVote' requests votes for election. When a node is a follower or
-	candidate and 'MsgHup' is passed to its Step method, then the node calls
-	'campaign' method to campaign itself to become a leader. Once 'campaign'
-	method is called, the node becomes candidate and sends 'MsgVote' to peers
-	in cluster to request votes. When passed to leader or candidate's Step
-	method and the message's Term is lower than leader's or candidate's,
-	'MsgVote' will be rejected ('MsgVoteResp' is returned with Reject true).
-	If leader or candidate receives 'MsgVote' with higher term, it will revert
-	back to follower. When 'MsgVote' is passed to follower, it votes for the
-	sender only when sender's last term is greater than MsgVote's term or
-	sender's last term is equal to MsgVote's term but sender's last committed
-	index is greater than or equal to follower's.
-
-	'MsgVoteResp' contains responses from voting request. When 'MsgVoteResp' is
-	passed to candidate, the candidate calculates how many votes it has won. If
-	it's more than majority (quorum), it becomes leader and calls 'bcastAppend'.
-	If candidate receives majority of votes of denials, it reverts back to
-	follower.
-
-	'MsgPreVote' and 'MsgPreVoteResp' are used in an optional two-phase election
-	protocol. When Config.PreVote is true, a pre-election is carried out first
-	(using the same rules as a regular election), and no node increases its term
-	number unless the pre-election indicates that the campaigning node would win.
-	This minimizes disruption when a partitioned node rejoins the cluster.
-
-	'MsgSnap' requests to install a snapshot message. When a node has just
-	become a leader or the leader receives 'MsgProp' message, it calls
-	'bcastAppend' method, which then calls 'sendAppend' method to each
-	follower. In 'sendAppend', if a leader fails to get term or entries,
-	the leader requests snapshot by sending 'MsgSnap' type message.
-
-	'MsgSnapStatus' tells the result of snapshot install message. When a
-	follower rejected 'MsgSnap', it indicates the snapshot request with
-	'MsgSnap' had failed from network issues which causes the network layer
-	to fail to send out snapshots to its followers. Then leader considers
-	follower's progress as probe. When 'MsgSnap' were not rejected, it
-	indicates that the snapshot succeeded and the leader sets follower's
-	progress to probe and resumes its log replication.
-
-	'MsgHeartbeat' sends heartbeat from leader. When 'MsgHeartbeat' is passed
-	to candidate and message's term is higher than candidate's, the candidate
-	reverts back to follower and updates its committed index from the one in
-	this heartbeat. And it sends the message to its mailbox. When
-	'MsgHeartbeat' is passed to follower's Step method and message's term is
-	higher than follower's, the follower updates its leaderID with the ID
-	from the message.
-
-	'MsgHeartbeatResp' is a response to 'MsgHeartbeat'. When 'MsgHeartbeatResp'
-	is passed to leader's Step method, the leader knows which follower
-	responded. And only when the leader's last committed index is greater than
-	follower's Match index, the leader runs 'sendAppend` method.
-
-	'MsgUnreachable' tells that request(message) wasn't delivered. When
-	'MsgUnreachable' is passed to leader's Step method, the leader discovers
-	that the follower that sent this 'MsgUnreachable' is not reachable, often
-	indicating 'MsgApp' is lost. When follower's progress state is replicate,
-	the leader sets it back to probe.
-
-*/
-package raft
diff --git a/vendor/go.etcd.io/etcd/raft/log.go b/vendor/go.etcd.io/etcd/raft/log.go
deleted file mode 100644
index 77eedfc..0000000
--- a/vendor/go.etcd.io/etcd/raft/log.go
+++ /dev/null
@@ -1,372 +0,0 @@
-// 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 raft
-
-import (
-	"fmt"
-	"log"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-type raftLog struct {
-	// storage contains all stable entries since the last snapshot.
-	storage Storage
-
-	// unstable contains all unstable entries and snapshot.
-	// they will be saved into storage.
-	unstable unstable
-
-	// committed is the highest log position that is known to be in
-	// stable storage on a quorum of nodes.
-	committed uint64
-	// applied is the highest log position that the application has
-	// been instructed to apply to its state machine.
-	// Invariant: applied <= committed
-	applied uint64
-
-	logger Logger
-
-	// maxNextEntsSize is the maximum number aggregate byte size of the messages
-	// returned from calls to nextEnts.
-	maxNextEntsSize uint64
-}
-
-// newLog returns log using the given storage and default options. It
-// recovers the log to the state that it just commits and applies the
-// latest snapshot.
-func newLog(storage Storage, logger Logger) *raftLog {
-	return newLogWithSize(storage, logger, noLimit)
-}
-
-// newLogWithSize returns a log using the given storage and max
-// message size.
-func newLogWithSize(storage Storage, logger Logger, maxNextEntsSize uint64) *raftLog {
-	if storage == nil {
-		log.Panic("storage must not be nil")
-	}
-	log := &raftLog{
-		storage:         storage,
-		logger:          logger,
-		maxNextEntsSize: maxNextEntsSize,
-	}
-	firstIndex, err := storage.FirstIndex()
-	if err != nil {
-		panic(err) // TODO(bdarnell)
-	}
-	lastIndex, err := storage.LastIndex()
-	if err != nil {
-		panic(err) // TODO(bdarnell)
-	}
-	log.unstable.offset = lastIndex + 1
-	log.unstable.logger = logger
-	// Initialize our committed and applied pointers to the time of the last compaction.
-	log.committed = firstIndex - 1
-	log.applied = firstIndex - 1
-
-	return log
-}
-
-func (l *raftLog) String() string {
-	return fmt.Sprintf("committed=%d, applied=%d, unstable.offset=%d, len(unstable.Entries)=%d", l.committed, l.applied, l.unstable.offset, len(l.unstable.entries))
-}
-
-// maybeAppend returns (0, false) if the entries cannot be appended. Otherwise,
-// it returns (last index of new entries, true).
-func (l *raftLog) maybeAppend(index, logTerm, committed uint64, ents ...pb.Entry) (lastnewi uint64, ok bool) {
-	if l.matchTerm(index, logTerm) {
-		lastnewi = index + uint64(len(ents))
-		ci := l.findConflict(ents)
-		switch {
-		case ci == 0:
-		case ci <= l.committed:
-			l.logger.Panicf("entry %d conflict with committed entry [committed(%d)]", ci, l.committed)
-		default:
-			offset := index + 1
-			l.append(ents[ci-offset:]...)
-		}
-		l.commitTo(min(committed, lastnewi))
-		return lastnewi, true
-	}
-	return 0, false
-}
-
-func (l *raftLog) append(ents ...pb.Entry) uint64 {
-	if len(ents) == 0 {
-		return l.lastIndex()
-	}
-	if after := ents[0].Index - 1; after < l.committed {
-		l.logger.Panicf("after(%d) is out of range [committed(%d)]", after, l.committed)
-	}
-	l.unstable.truncateAndAppend(ents)
-	return l.lastIndex()
-}
-
-// findConflict finds the index of the conflict.
-// It returns the first pair of conflicting entries between the existing
-// entries and the given entries, if there are any.
-// If there is no conflicting entries, and the existing entries contains
-// all the given entries, zero will be returned.
-// If there is no conflicting entries, but the given entries contains new
-// entries, the index of the first new entry will be returned.
-// An entry is considered to be conflicting if it has the same index but
-// a different term.
-// The first entry MUST have an index equal to the argument 'from'.
-// The index of the given entries MUST be continuously increasing.
-func (l *raftLog) findConflict(ents []pb.Entry) uint64 {
-	for _, ne := range ents {
-		if !l.matchTerm(ne.Index, ne.Term) {
-			if ne.Index <= l.lastIndex() {
-				l.logger.Infof("found conflict at index %d [existing term: %d, conflicting term: %d]",
-					ne.Index, l.zeroTermOnErrCompacted(l.term(ne.Index)), ne.Term)
-			}
-			return ne.Index
-		}
-	}
-	return 0
-}
-
-func (l *raftLog) unstableEntries() []pb.Entry {
-	if len(l.unstable.entries) == 0 {
-		return nil
-	}
-	return l.unstable.entries
-}
-
-// nextEnts returns all the available entries for execution.
-// If applied is smaller than the index of snapshot, it returns all committed
-// entries after the index of snapshot.
-func (l *raftLog) nextEnts() (ents []pb.Entry) {
-	off := max(l.applied+1, l.firstIndex())
-	if l.committed+1 > off {
-		ents, err := l.slice(off, l.committed+1, l.maxNextEntsSize)
-		if err != nil {
-			l.logger.Panicf("unexpected error when getting unapplied entries (%v)", err)
-		}
-		return ents
-	}
-	return nil
-}
-
-// hasNextEnts returns if there is any available entries for execution. This
-// is a fast check without heavy raftLog.slice() in raftLog.nextEnts().
-func (l *raftLog) hasNextEnts() bool {
-	off := max(l.applied+1, l.firstIndex())
-	return l.committed+1 > off
-}
-
-func (l *raftLog) snapshot() (pb.Snapshot, error) {
-	if l.unstable.snapshot != nil {
-		return *l.unstable.snapshot, nil
-	}
-	return l.storage.Snapshot()
-}
-
-func (l *raftLog) firstIndex() uint64 {
-	if i, ok := l.unstable.maybeFirstIndex(); ok {
-		return i
-	}
-	index, err := l.storage.FirstIndex()
-	if err != nil {
-		panic(err) // TODO(bdarnell)
-	}
-	return index
-}
-
-func (l *raftLog) lastIndex() uint64 {
-	if i, ok := l.unstable.maybeLastIndex(); ok {
-		return i
-	}
-	i, err := l.storage.LastIndex()
-	if err != nil {
-		panic(err) // TODO(bdarnell)
-	}
-	return i
-}
-
-func (l *raftLog) commitTo(tocommit uint64) {
-	// never decrease commit
-	if l.committed < tocommit {
-		if l.lastIndex() < tocommit {
-			l.logger.Panicf("tocommit(%d) is out of range [lastIndex(%d)]. Was the raft log corrupted, truncated, or lost?", tocommit, l.lastIndex())
-		}
-		l.committed = tocommit
-	}
-}
-
-func (l *raftLog) appliedTo(i uint64) {
-	if i == 0 {
-		return
-	}
-	if l.committed < i || i < l.applied {
-		l.logger.Panicf("applied(%d) is out of range [prevApplied(%d), committed(%d)]", i, l.applied, l.committed)
-	}
-	l.applied = i
-}
-
-func (l *raftLog) stableTo(i, t uint64) { l.unstable.stableTo(i, t) }
-
-func (l *raftLog) stableSnapTo(i uint64) { l.unstable.stableSnapTo(i) }
-
-func (l *raftLog) lastTerm() uint64 {
-	t, err := l.term(l.lastIndex())
-	if err != nil {
-		l.logger.Panicf("unexpected error when getting the last term (%v)", err)
-	}
-	return t
-}
-
-func (l *raftLog) term(i uint64) (uint64, error) {
-	// the valid term range is [index of dummy entry, last index]
-	dummyIndex := l.firstIndex() - 1
-	if i < dummyIndex || i > l.lastIndex() {
-		// TODO: return an error instead?
-		return 0, nil
-	}
-
-	if t, ok := l.unstable.maybeTerm(i); ok {
-		return t, nil
-	}
-
-	t, err := l.storage.Term(i)
-	if err == nil {
-		return t, nil
-	}
-	if err == ErrCompacted || err == ErrUnavailable {
-		return 0, err
-	}
-	panic(err) // TODO(bdarnell)
-}
-
-func (l *raftLog) entries(i, maxsize uint64) ([]pb.Entry, error) {
-	if i > l.lastIndex() {
-		return nil, nil
-	}
-	return l.slice(i, l.lastIndex()+1, maxsize)
-}
-
-// allEntries returns all entries in the log.
-func (l *raftLog) allEntries() []pb.Entry {
-	ents, err := l.entries(l.firstIndex(), noLimit)
-	if err == nil {
-		return ents
-	}
-	if err == ErrCompacted { // try again if there was a racing compaction
-		return l.allEntries()
-	}
-	// TODO (xiangli): handle error?
-	panic(err)
-}
-
-// isUpToDate determines if the given (lastIndex,term) log is more up-to-date
-// by comparing the index and term of the last entries in the existing logs.
-// If the logs have last entries with different terms, then the log with the
-// later term is more up-to-date. If the logs end with the same term, then
-// whichever log has the larger lastIndex is more up-to-date. If the logs are
-// the same, the given log is up-to-date.
-func (l *raftLog) isUpToDate(lasti, term uint64) bool {
-	return term > l.lastTerm() || (term == l.lastTerm() && lasti >= l.lastIndex())
-}
-
-func (l *raftLog) matchTerm(i, term uint64) bool {
-	t, err := l.term(i)
-	if err != nil {
-		return false
-	}
-	return t == term
-}
-
-func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
-	if maxIndex > l.committed && l.zeroTermOnErrCompacted(l.term(maxIndex)) == term {
-		l.commitTo(maxIndex)
-		return true
-	}
-	return false
-}
-
-func (l *raftLog) restore(s pb.Snapshot) {
-	l.logger.Infof("log [%s] starts to restore snapshot [index: %d, term: %d]", l, s.Metadata.Index, s.Metadata.Term)
-	l.committed = s.Metadata.Index
-	l.unstable.restore(s)
-}
-
-// slice returns a slice of log entries from lo through hi-1, inclusive.
-func (l *raftLog) slice(lo, hi, maxSize uint64) ([]pb.Entry, error) {
-	err := l.mustCheckOutOfBounds(lo, hi)
-	if err != nil {
-		return nil, err
-	}
-	if lo == hi {
-		return nil, nil
-	}
-	var ents []pb.Entry
-	if lo < l.unstable.offset {
-		storedEnts, err := l.storage.Entries(lo, min(hi, l.unstable.offset), maxSize)
-		if err == ErrCompacted {
-			return nil, err
-		} else if err == ErrUnavailable {
-			l.logger.Panicf("entries[%d:%d) is unavailable from storage", lo, min(hi, l.unstable.offset))
-		} else if err != nil {
-			panic(err) // TODO(bdarnell)
-		}
-
-		// check if ents has reached the size limitation
-		if uint64(len(storedEnts)) < min(hi, l.unstable.offset)-lo {
-			return storedEnts, nil
-		}
-
-		ents = storedEnts
-	}
-	if hi > l.unstable.offset {
-		unstable := l.unstable.slice(max(lo, l.unstable.offset), hi)
-		if len(ents) > 0 {
-			combined := make([]pb.Entry, len(ents)+len(unstable))
-			n := copy(combined, ents)
-			copy(combined[n:], unstable)
-			ents = combined
-		} else {
-			ents = unstable
-		}
-	}
-	return limitSize(ents, maxSize), nil
-}
-
-// l.firstIndex <= lo <= hi <= l.firstIndex + len(l.entries)
-func (l *raftLog) mustCheckOutOfBounds(lo, hi uint64) error {
-	if lo > hi {
-		l.logger.Panicf("invalid slice %d > %d", lo, hi)
-	}
-	fi := l.firstIndex()
-	if lo < fi {
-		return ErrCompacted
-	}
-
-	length := l.lastIndex() + 1 - fi
-	if lo < fi || hi > fi+length {
-		l.logger.Panicf("slice[%d,%d) out of bound [%d,%d]", lo, hi, fi, l.lastIndex())
-	}
-	return nil
-}
-
-func (l *raftLog) zeroTermOnErrCompacted(t uint64, err error) uint64 {
-	if err == nil {
-		return t
-	}
-	if err == ErrCompacted {
-		return 0
-	}
-	l.logger.Panicf("unexpected error (%v)", err)
-	return 0
-}
diff --git a/vendor/go.etcd.io/etcd/raft/log_unstable.go b/vendor/go.etcd.io/etcd/raft/log_unstable.go
deleted file mode 100644
index 1bff5a7..0000000
--- a/vendor/go.etcd.io/etcd/raft/log_unstable.go
+++ /dev/null
@@ -1,157 +0,0 @@
-// 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 raft
-
-import pb "go.etcd.io/etcd/raft/raftpb"
-
-// unstable.entries[i] has raft log position i+unstable.offset.
-// Note that unstable.offset may be less than the highest log
-// position in storage; this means that the next write to storage
-// might need to truncate the log before persisting unstable.entries.
-type unstable struct {
-	// the incoming unstable snapshot, if any.
-	snapshot *pb.Snapshot
-	// all entries that have not yet been written to storage.
-	entries []pb.Entry
-	offset  uint64
-
-	logger Logger
-}
-
-// maybeFirstIndex returns the index of the first possible entry in entries
-// if it has a snapshot.
-func (u *unstable) maybeFirstIndex() (uint64, bool) {
-	if u.snapshot != nil {
-		return u.snapshot.Metadata.Index + 1, true
-	}
-	return 0, false
-}
-
-// maybeLastIndex returns the last index if it has at least one
-// unstable entry or snapshot.
-func (u *unstable) maybeLastIndex() (uint64, bool) {
-	if l := len(u.entries); l != 0 {
-		return u.offset + uint64(l) - 1, true
-	}
-	if u.snapshot != nil {
-		return u.snapshot.Metadata.Index, true
-	}
-	return 0, false
-}
-
-// maybeTerm returns the term of the entry at index i, if there
-// is any.
-func (u *unstable) maybeTerm(i uint64) (uint64, bool) {
-	if i < u.offset {
-		if u.snapshot != nil && u.snapshot.Metadata.Index == i {
-			return u.snapshot.Metadata.Term, true
-		}
-		return 0, false
-	}
-
-	last, ok := u.maybeLastIndex()
-	if !ok {
-		return 0, false
-	}
-	if i > last {
-		return 0, false
-	}
-
-	return u.entries[i-u.offset].Term, true
-}
-
-func (u *unstable) stableTo(i, t uint64) {
-	gt, ok := u.maybeTerm(i)
-	if !ok {
-		return
-	}
-	// if i < offset, term is matched with the snapshot
-	// only update the unstable entries if term is matched with
-	// an unstable entry.
-	if gt == t && i >= u.offset {
-		u.entries = u.entries[i+1-u.offset:]
-		u.offset = i + 1
-		u.shrinkEntriesArray()
-	}
-}
-
-// shrinkEntriesArray discards the underlying array used by the entries slice
-// if most of it isn't being used. This avoids holding references to a bunch of
-// potentially large entries that aren't needed anymore. Simply clearing the
-// entries wouldn't be safe because clients might still be using them.
-func (u *unstable) shrinkEntriesArray() {
-	// We replace the array if we're using less than half of the space in
-	// it. This number is fairly arbitrary, chosen as an attempt to balance
-	// memory usage vs number of allocations. It could probably be improved
-	// with some focused tuning.
-	const lenMultiple = 2
-	if len(u.entries) == 0 {
-		u.entries = nil
-	} else if len(u.entries)*lenMultiple < cap(u.entries) {
-		newEntries := make([]pb.Entry, len(u.entries))
-		copy(newEntries, u.entries)
-		u.entries = newEntries
-	}
-}
-
-func (u *unstable) stableSnapTo(i uint64) {
-	if u.snapshot != nil && u.snapshot.Metadata.Index == i {
-		u.snapshot = nil
-	}
-}
-
-func (u *unstable) restore(s pb.Snapshot) {
-	u.offset = s.Metadata.Index + 1
-	u.entries = nil
-	u.snapshot = &s
-}
-
-func (u *unstable) truncateAndAppend(ents []pb.Entry) {
-	after := ents[0].Index
-	switch {
-	case after == u.offset+uint64(len(u.entries)):
-		// after is the next index in the u.entries
-		// directly append
-		u.entries = append(u.entries, ents...)
-	case after <= u.offset:
-		u.logger.Infof("replace the unstable entries from index %d", after)
-		// The log is being truncated to before our current offset
-		// portion, so set the offset and replace the entries
-		u.offset = after
-		u.entries = ents
-	default:
-		// truncate to after and copy to u.entries
-		// then append
-		u.logger.Infof("truncate the unstable entries before index %d", after)
-		u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...)
-		u.entries = append(u.entries, ents...)
-	}
-}
-
-func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry {
-	u.mustCheckOutOfBounds(lo, hi)
-	return u.entries[lo-u.offset : hi-u.offset]
-}
-
-// u.offset <= lo <= hi <= u.offset+len(u.entries)
-func (u *unstable) mustCheckOutOfBounds(lo, hi uint64) {
-	if lo > hi {
-		u.logger.Panicf("invalid unstable.slice %d > %d", lo, hi)
-	}
-	upper := u.offset + uint64(len(u.entries))
-	if lo < u.offset || hi > upper {
-		u.logger.Panicf("unstable.slice[%d,%d) out of bound [%d,%d]", lo, hi, u.offset, upper)
-	}
-}
diff --git a/vendor/go.etcd.io/etcd/raft/logger.go b/vendor/go.etcd.io/etcd/raft/logger.go
deleted file mode 100644
index 6d89629..0000000
--- a/vendor/go.etcd.io/etcd/raft/logger.go
+++ /dev/null
@@ -1,132 +0,0 @@
-// 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 raft
-
-import (
-	"fmt"
-	"io/ioutil"
-	"log"
-	"os"
-	"sync"
-)
-
-type Logger interface {
-	Debug(v ...interface{})
-	Debugf(format string, v ...interface{})
-
-	Error(v ...interface{})
-	Errorf(format string, v ...interface{})
-
-	Info(v ...interface{})
-	Infof(format string, v ...interface{})
-
-	Warning(v ...interface{})
-	Warningf(format string, v ...interface{})
-
-	Fatal(v ...interface{})
-	Fatalf(format string, v ...interface{})
-
-	Panic(v ...interface{})
-	Panicf(format string, v ...interface{})
-}
-
-func SetLogger(l Logger) {
-	raftLoggerMu.Lock()
-	raftLogger = l
-	raftLoggerMu.Unlock()
-}
-
-var (
-	defaultLogger = &DefaultLogger{Logger: log.New(os.Stderr, "raft", log.LstdFlags)}
-	discardLogger = &DefaultLogger{Logger: log.New(ioutil.Discard, "", 0)}
-	raftLoggerMu  sync.Mutex
-	raftLogger    = Logger(defaultLogger)
-)
-
-const (
-	calldepth = 2
-)
-
-// DefaultLogger is a default implementation of the Logger interface.
-type DefaultLogger struct {
-	*log.Logger
-	debug bool
-}
-
-func (l *DefaultLogger) EnableTimestamps() {
-	l.SetFlags(l.Flags() | log.Ldate | log.Ltime)
-}
-
-func (l *DefaultLogger) EnableDebug() {
-	l.debug = true
-}
-
-func (l *DefaultLogger) Debug(v ...interface{}) {
-	if l.debug {
-		l.Output(calldepth, header("DEBUG", fmt.Sprint(v...)))
-	}
-}
-
-func (l *DefaultLogger) Debugf(format string, v ...interface{}) {
-	if l.debug {
-		l.Output(calldepth, header("DEBUG", fmt.Sprintf(format, v...)))
-	}
-}
-
-func (l *DefaultLogger) Info(v ...interface{}) {
-	l.Output(calldepth, header("INFO", fmt.Sprint(v...)))
-}
-
-func (l *DefaultLogger) Infof(format string, v ...interface{}) {
-	l.Output(calldepth, header("INFO", fmt.Sprintf(format, v...)))
-}
-
-func (l *DefaultLogger) Error(v ...interface{}) {
-	l.Output(calldepth, header("ERROR", fmt.Sprint(v...)))
-}
-
-func (l *DefaultLogger) Errorf(format string, v ...interface{}) {
-	l.Output(calldepth, header("ERROR", fmt.Sprintf(format, v...)))
-}
-
-func (l *DefaultLogger) Warning(v ...interface{}) {
-	l.Output(calldepth, header("WARN", fmt.Sprint(v...)))
-}
-
-func (l *DefaultLogger) Warningf(format string, v ...interface{}) {
-	l.Output(calldepth, header("WARN", fmt.Sprintf(format, v...)))
-}
-
-func (l *DefaultLogger) Fatal(v ...interface{}) {
-	l.Output(calldepth, header("FATAL", fmt.Sprint(v...)))
-	os.Exit(1)
-}
-
-func (l *DefaultLogger) Fatalf(format string, v ...interface{}) {
-	l.Output(calldepth, header("FATAL", fmt.Sprintf(format, v...)))
-	os.Exit(1)
-}
-
-func (l *DefaultLogger) Panic(v ...interface{}) {
-	l.Logger.Panic(v...)
-}
-
-func (l *DefaultLogger) Panicf(format string, v ...interface{}) {
-	l.Logger.Panicf(format, v...)
-}
-
-func header(lvl, msg string) string {
-	return fmt.Sprintf("%s: %s", lvl, msg)
-}
diff --git a/vendor/go.etcd.io/etcd/raft/node.go b/vendor/go.etcd.io/etcd/raft/node.go
deleted file mode 100644
index ab6185b..0000000
--- a/vendor/go.etcd.io/etcd/raft/node.go
+++ /dev/null
@@ -1,584 +0,0 @@
-// 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 raft
-
-import (
-	"context"
-	"errors"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-type SnapshotStatus int
-
-const (
-	SnapshotFinish  SnapshotStatus = 1
-	SnapshotFailure SnapshotStatus = 2
-)
-
-var (
-	emptyState = pb.HardState{}
-
-	// ErrStopped is returned by methods on Nodes that have been stopped.
-	ErrStopped = errors.New("raft: stopped")
-)
-
-// SoftState provides state that is useful for logging and debugging.
-// The state is volatile and does not need to be persisted to the WAL.
-type SoftState struct {
-	Lead      uint64 // must use atomic operations to access; keep 64-bit aligned.
-	RaftState StateType
-}
-
-func (a *SoftState) equal(b *SoftState) bool {
-	return a.Lead == b.Lead && a.RaftState == b.RaftState
-}
-
-// Ready encapsulates the entries and messages that are ready to read,
-// be saved to stable storage, committed or sent to other peers.
-// All fields in Ready are read-only.
-type Ready struct {
-	// The current volatile state of a Node.
-	// SoftState will be nil if there is no update.
-	// It is not required to consume or store SoftState.
-	*SoftState
-
-	// The current state of a Node to be saved to stable storage BEFORE
-	// Messages are sent.
-	// HardState will be equal to empty state if there is no update.
-	pb.HardState
-
-	// ReadStates can be used for node to serve linearizable read requests locally
-	// when its applied index is greater than the index in ReadState.
-	// Note that the readState will be returned when raft receives msgReadIndex.
-	// The returned is only valid for the request that requested to read.
-	ReadStates []ReadState
-
-	// Entries specifies entries to be saved to stable storage BEFORE
-	// Messages are sent.
-	Entries []pb.Entry
-
-	// Snapshot specifies the snapshot to be saved to stable storage.
-	Snapshot pb.Snapshot
-
-	// CommittedEntries specifies entries to be committed to a
-	// store/state-machine. These have previously been committed to stable
-	// store.
-	CommittedEntries []pb.Entry
-
-	// Messages specifies outbound messages to be sent AFTER Entries are
-	// committed to stable storage.
-	// If it contains a MsgSnap message, the application MUST report back to raft
-	// when the snapshot has been received or has failed by calling ReportSnapshot.
-	Messages []pb.Message
-
-	// MustSync indicates whether the HardState and Entries must be synchronously
-	// written to disk or if an asynchronous write is permissible.
-	MustSync bool
-}
-
-func isHardStateEqual(a, b pb.HardState) bool {
-	return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit
-}
-
-// IsEmptyHardState returns true if the given HardState is empty.
-func IsEmptyHardState(st pb.HardState) bool {
-	return isHardStateEqual(st, emptyState)
-}
-
-// IsEmptySnap returns true if the given Snapshot is empty.
-func IsEmptySnap(sp pb.Snapshot) bool {
-	return sp.Metadata.Index == 0
-}
-
-func (rd Ready) containsUpdates() bool {
-	return rd.SoftState != nil || !IsEmptyHardState(rd.HardState) ||
-		!IsEmptySnap(rd.Snapshot) || len(rd.Entries) > 0 ||
-		len(rd.CommittedEntries) > 0 || len(rd.Messages) > 0 || len(rd.ReadStates) != 0
-}
-
-// appliedCursor extracts from the Ready the highest index the client has
-// applied (once the Ready is confirmed via Advance). If no information is
-// contained in the Ready, returns zero.
-func (rd Ready) appliedCursor() uint64 {
-	if n := len(rd.CommittedEntries); n > 0 {
-		return rd.CommittedEntries[n-1].Index
-	}
-	if index := rd.Snapshot.Metadata.Index; index > 0 {
-		return index
-	}
-	return 0
-}
-
-// Node represents a node in a raft cluster.
-type Node interface {
-	// Tick increments the internal logical clock for the Node by a single tick. Election
-	// timeouts and heartbeat timeouts are in units of ticks.
-	Tick()
-	// Campaign causes the Node to transition to candidate state and start campaigning to become leader.
-	Campaign(ctx context.Context) error
-	// Propose proposes that data be appended to the log. Note that proposals can be lost without
-	// notice, therefore it is user's job to ensure proposal retries.
-	Propose(ctx context.Context, data []byte) error
-	// ProposeConfChange proposes a configuration change. Like any proposal, the
-	// configuration change may be dropped with or without an error being
-	// returned. In particular, configuration changes are dropped unless the
-	// leader has certainty that there is no prior unapplied configuration
-	// change in its log.
-	//
-	// The method accepts either a pb.ConfChange (deprecated) or pb.ConfChangeV2
-	// message. The latter allows arbitrary configuration changes via joint
-	// consensus, notably including replacing a voter. Passing a ConfChangeV2
-	// message is only allowed if all Nodes participating in the cluster run a
-	// version of this library aware of the V2 API. See pb.ConfChangeV2 for
-	// usage details and semantics.
-	ProposeConfChange(ctx context.Context, cc pb.ConfChangeI) error
-
-	// Step advances the state machine using the given message. ctx.Err() will be returned, if any.
-	Step(ctx context.Context, msg pb.Message) error
-
-	// Ready returns a channel that returns the current point-in-time state.
-	// Users of the Node must call Advance after retrieving the state returned by Ready.
-	//
-	// NOTE: No committed entries from the next Ready may be applied until all committed entries
-	// and snapshots from the previous one have finished.
-	Ready() <-chan Ready
-
-	// Advance notifies the Node that the application has saved progress up to the last Ready.
-	// It prepares the node to return the next available Ready.
-	//
-	// The application should generally call Advance after it applies the entries in last Ready.
-	//
-	// However, as an optimization, the application may call Advance while it is applying the
-	// commands. For example. when the last Ready contains a snapshot, the application might take
-	// a long time to apply the snapshot data. To continue receiving Ready without blocking raft
-	// progress, it can call Advance before finishing applying the last ready.
-	Advance()
-	// ApplyConfChange applies a config change (previously passed to
-	// ProposeConfChange) to the node. This must be called whenever a config
-	// change is observed in Ready.CommittedEntries.
-	//
-	// Returns an opaque non-nil ConfState protobuf which must be recorded in
-	// snapshots.
-	ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState
-
-	// TransferLeadership attempts to transfer leadership to the given transferee.
-	TransferLeadership(ctx context.Context, lead, transferee uint64)
-
-	// ReadIndex request a read state. The read state will be set in the ready.
-	// Read state has a read index. Once the application advances further than the read
-	// index, any linearizable read requests issued before the read request can be
-	// processed safely. The read state will have the same rctx attached.
-	ReadIndex(ctx context.Context, rctx []byte) error
-
-	// Status returns the current status of the raft state machine.
-	Status() Status
-	// ReportUnreachable reports the given node is not reachable for the last send.
-	ReportUnreachable(id uint64)
-	// ReportSnapshot reports the status of the sent snapshot. The id is the raft ID of the follower
-	// who is meant to receive the snapshot, and the status is SnapshotFinish or SnapshotFailure.
-	// Calling ReportSnapshot with SnapshotFinish is a no-op. But, any failure in applying a
-	// snapshot (for e.g., while streaming it from leader to follower), should be reported to the
-	// leader with SnapshotFailure. When leader sends a snapshot to a follower, it pauses any raft
-	// log probes until the follower can apply the snapshot and advance its state. If the follower
-	// can't do that, for e.g., due to a crash, it could end up in a limbo, never getting any
-	// updates from the leader. Therefore, it is crucial that the application ensures that any
-	// failure in snapshot sending is caught and reported back to the leader; so it can resume raft
-	// log probing in the follower.
-	ReportSnapshot(id uint64, status SnapshotStatus)
-	// Stop performs any necessary termination of the Node.
-	Stop()
-}
-
-type Peer struct {
-	ID      uint64
-	Context []byte
-}
-
-// StartNode returns a new Node given configuration and a list of raft peers.
-// It appends a ConfChangeAddNode entry for each given peer to the initial log.
-//
-// Peers must not be zero length; call RestartNode in that case.
-func StartNode(c *Config, peers []Peer) Node {
-	if len(peers) == 0 {
-		panic("no peers given; use RestartNode instead")
-	}
-	rn, err := NewRawNode(c)
-	if err != nil {
-		panic(err)
-	}
-	rn.Bootstrap(peers)
-
-	n := newNode(rn)
-
-	go n.run()
-	return &n
-}
-
-// RestartNode is similar to StartNode but does not take a list of peers.
-// The current membership of the cluster will be restored from the Storage.
-// If the caller has an existing state machine, pass in the last log index that
-// has been applied to it; otherwise use zero.
-func RestartNode(c *Config) Node {
-	rn, err := NewRawNode(c)
-	if err != nil {
-		panic(err)
-	}
-	n := newNode(rn)
-	go n.run()
-	return &n
-}
-
-type msgWithResult struct {
-	m      pb.Message
-	result chan error
-}
-
-// node is the canonical implementation of the Node interface
-type node struct {
-	propc      chan msgWithResult
-	recvc      chan pb.Message
-	confc      chan pb.ConfChangeV2
-	confstatec chan pb.ConfState
-	readyc     chan Ready
-	advancec   chan struct{}
-	tickc      chan struct{}
-	done       chan struct{}
-	stop       chan struct{}
-	status     chan chan Status
-
-	rn *RawNode
-}
-
-func newNode(rn *RawNode) node {
-	return node{
-		propc:      make(chan msgWithResult),
-		recvc:      make(chan pb.Message),
-		confc:      make(chan pb.ConfChangeV2),
-		confstatec: make(chan pb.ConfState),
-		readyc:     make(chan Ready),
-		advancec:   make(chan struct{}),
-		// make tickc a buffered chan, so raft node can buffer some ticks when the node
-		// is busy processing raft messages. Raft node will resume process buffered
-		// ticks when it becomes idle.
-		tickc:  make(chan struct{}, 128),
-		done:   make(chan struct{}),
-		stop:   make(chan struct{}),
-		status: make(chan chan Status),
-		rn:     rn,
-	}
-}
-
-func (n *node) Stop() {
-	select {
-	case n.stop <- struct{}{}:
-		// Not already stopped, so trigger it
-	case <-n.done:
-		// Node has already been stopped - no need to do anything
-		return
-	}
-	// Block until the stop has been acknowledged by run()
-	<-n.done
-}
-
-func (n *node) run() {
-	var propc chan msgWithResult
-	var readyc chan Ready
-	var advancec chan struct{}
-	var rd Ready
-
-	r := n.rn.raft
-
-	lead := None
-
-	for {
-		if advancec != nil {
-			readyc = nil
-		} else if n.rn.HasReady() {
-			// Populate a Ready. Note that this Ready is not guaranteed to
-			// actually be handled. We will arm readyc, but there's no guarantee
-			// that we will actually send on it. It's possible that we will
-			// service another channel instead, loop around, and then populate
-			// the Ready again. We could instead force the previous Ready to be
-			// handled first, but it's generally good to emit larger Readys plus
-			// it simplifies testing (by emitting less frequently and more
-			// predictably).
-			rd = n.rn.readyWithoutAccept()
-			readyc = n.readyc
-		}
-
-		if lead != r.lead {
-			if r.hasLeader() {
-				if lead == None {
-					r.logger.Infof("raft.node: %x elected leader %x at term %d", r.id, r.lead, r.Term)
-				} else {
-					r.logger.Infof("raft.node: %x changed leader from %x to %x at term %d", r.id, lead, r.lead, r.Term)
-				}
-				propc = n.propc
-			} else {
-				r.logger.Infof("raft.node: %x lost leader %x at term %d", r.id, lead, r.Term)
-				propc = nil
-			}
-			lead = r.lead
-		}
-
-		select {
-		// TODO: maybe buffer the config propose if there exists one (the way
-		// described in raft dissertation)
-		// Currently it is dropped in Step silently.
-		case pm := <-propc:
-			m := pm.m
-			m.From = r.id
-			err := r.Step(m)
-			if pm.result != nil {
-				pm.result <- err
-				close(pm.result)
-			}
-		case m := <-n.recvc:
-			// filter out response message from unknown From.
-			if pr := r.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) {
-				r.Step(m)
-			}
-		case cc := <-n.confc:
-			_, okBefore := r.prs.Progress[r.id]
-			cs := r.applyConfChange(cc)
-			// If the node was removed, block incoming proposals. Note that we
-			// only do this if the node was in the config before. Nodes may be
-			// a member of the group without knowing this (when they're catching
-			// up on the log and don't have the latest config) and we don't want
-			// to block the proposal channel in that case.
-			//
-			// NB: propc is reset when the leader changes, which, if we learn
-			// about it, sort of implies that we got readded, maybe? This isn't
-			// very sound and likely has bugs.
-			if _, okAfter := r.prs.Progress[r.id]; okBefore && !okAfter {
-				var found bool
-				for _, sl := range [][]uint64{cs.Voters, cs.VotersOutgoing} {
-					for _, id := range sl {
-						if id == r.id {
-							found = true
-						}
-					}
-				}
-				if !found {
-					propc = nil
-				}
-			}
-			select {
-			case n.confstatec <- cs:
-			case <-n.done:
-			}
-		case <-n.tickc:
-			n.rn.Tick()
-		case readyc <- rd:
-			n.rn.acceptReady(rd)
-			advancec = n.advancec
-		case <-advancec:
-			n.rn.Advance(rd)
-			rd = Ready{}
-			advancec = nil
-		case c := <-n.status:
-			c <- getStatus(r)
-		case <-n.stop:
-			close(n.done)
-			return
-		}
-	}
-}
-
-// Tick increments the internal logical clock for this Node. Election timeouts
-// and heartbeat timeouts are in units of ticks.
-func (n *node) Tick() {
-	select {
-	case n.tickc <- struct{}{}:
-	case <-n.done:
-	default:
-		n.rn.raft.logger.Warningf("%x (leader %v) A tick missed to fire. Node blocks too long!", n.rn.raft.id, n.rn.raft.id == n.rn.raft.lead)
-	}
-}
-
-func (n *node) Campaign(ctx context.Context) error { return n.step(ctx, pb.Message{Type: pb.MsgHup}) }
-
-func (n *node) Propose(ctx context.Context, data []byte) error {
-	return n.stepWait(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Data: data}}})
-}
-
-func (n *node) Step(ctx context.Context, m pb.Message) error {
-	// ignore unexpected local messages receiving over network
-	if IsLocalMsg(m.Type) {
-		// TODO: return an error?
-		return nil
-	}
-	return n.step(ctx, m)
-}
-
-func confChangeToMsg(c pb.ConfChangeI) (pb.Message, error) {
-	typ, data, err := pb.MarshalConfChange(c)
-	if err != nil {
-		return pb.Message{}, err
-	}
-	return pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Type: typ, Data: data}}}, nil
-}
-
-func (n *node) ProposeConfChange(ctx context.Context, cc pb.ConfChangeI) error {
-	msg, err := confChangeToMsg(cc)
-	if err != nil {
-		return err
-	}
-	return n.Step(ctx, msg)
-}
-
-func (n *node) step(ctx context.Context, m pb.Message) error {
-	return n.stepWithWaitOption(ctx, m, false)
-}
-
-func (n *node) stepWait(ctx context.Context, m pb.Message) error {
-	return n.stepWithWaitOption(ctx, m, true)
-}
-
-// Step advances the state machine using msgs. The ctx.Err() will be returned,
-// if any.
-func (n *node) stepWithWaitOption(ctx context.Context, m pb.Message, wait bool) error {
-	if m.Type != pb.MsgProp {
-		select {
-		case n.recvc <- m:
-			return nil
-		case <-ctx.Done():
-			return ctx.Err()
-		case <-n.done:
-			return ErrStopped
-		}
-	}
-	ch := n.propc
-	pm := msgWithResult{m: m}
-	if wait {
-		pm.result = make(chan error, 1)
-	}
-	select {
-	case ch <- pm:
-		if !wait {
-			return nil
-		}
-	case <-ctx.Done():
-		return ctx.Err()
-	case <-n.done:
-		return ErrStopped
-	}
-	select {
-	case err := <-pm.result:
-		if err != nil {
-			return err
-		}
-	case <-ctx.Done():
-		return ctx.Err()
-	case <-n.done:
-		return ErrStopped
-	}
-	return nil
-}
-
-func (n *node) Ready() <-chan Ready { return n.readyc }
-
-func (n *node) Advance() {
-	select {
-	case n.advancec <- struct{}{}:
-	case <-n.done:
-	}
-}
-
-func (n *node) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState {
-	var cs pb.ConfState
-	select {
-	case n.confc <- cc.AsV2():
-	case <-n.done:
-	}
-	select {
-	case cs = <-n.confstatec:
-	case <-n.done:
-	}
-	return &cs
-}
-
-func (n *node) Status() Status {
-	c := make(chan Status)
-	select {
-	case n.status <- c:
-		return <-c
-	case <-n.done:
-		return Status{}
-	}
-}
-
-func (n *node) ReportUnreachable(id uint64) {
-	select {
-	case n.recvc <- pb.Message{Type: pb.MsgUnreachable, From: id}:
-	case <-n.done:
-	}
-}
-
-func (n *node) ReportSnapshot(id uint64, status SnapshotStatus) {
-	rej := status == SnapshotFailure
-
-	select {
-	case n.recvc <- pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej}:
-	case <-n.done:
-	}
-}
-
-func (n *node) TransferLeadership(ctx context.Context, lead, transferee uint64) {
-	select {
-	// manually set 'from' and 'to', so that leader can voluntarily transfers its leadership
-	case n.recvc <- pb.Message{Type: pb.MsgTransferLeader, From: transferee, To: lead}:
-	case <-n.done:
-	case <-ctx.Done():
-	}
-}
-
-func (n *node) ReadIndex(ctx context.Context, rctx []byte) error {
-	return n.step(ctx, pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
-}
-
-func newReady(r *raft, prevSoftSt *SoftState, prevHardSt pb.HardState) Ready {
-	rd := Ready{
-		Entries:          r.raftLog.unstableEntries(),
-		CommittedEntries: r.raftLog.nextEnts(),
-		Messages:         r.msgs,
-	}
-	if softSt := r.softState(); !softSt.equal(prevSoftSt) {
-		rd.SoftState = softSt
-	}
-	if hardSt := r.hardState(); !isHardStateEqual(hardSt, prevHardSt) {
-		rd.HardState = hardSt
-	}
-	if r.raftLog.unstable.snapshot != nil {
-		rd.Snapshot = *r.raftLog.unstable.snapshot
-	}
-	if len(r.readStates) != 0 {
-		rd.ReadStates = r.readStates
-	}
-	rd.MustSync = MustSync(r.hardState(), prevHardSt, len(rd.Entries))
-	return rd
-}
-
-// MustSync returns true if the hard state and count of Raft entries indicate
-// that a synchronous write to persistent storage is required.
-func MustSync(st, prevst pb.HardState, entsnum int) bool {
-	// Persistent state on all servers:
-	// (Updated on stable storage before responding to RPCs)
-	// currentTerm
-	// votedFor
-	// log entries[]
-	return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term
-}
diff --git a/vendor/go.etcd.io/etcd/raft/quorum/joint.go b/vendor/go.etcd.io/etcd/raft/quorum/joint.go
deleted file mode 100644
index e3741e0..0000000
--- a/vendor/go.etcd.io/etcd/raft/quorum/joint.go
+++ /dev/null
@@ -1,75 +0,0 @@
-// 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 quorum
-
-// JointConfig is a configuration of two groups of (possibly overlapping)
-// majority configurations. Decisions require the support of both majorities.
-type JointConfig [2]MajorityConfig
-
-func (c JointConfig) String() string {
-	if len(c[1]) > 0 {
-		return c[0].String() + "&&" + c[1].String()
-	}
-	return c[0].String()
-}
-
-// IDs returns a newly initialized map representing the set of voters present
-// in the joint configuration.
-func (c JointConfig) IDs() map[uint64]struct{} {
-	m := map[uint64]struct{}{}
-	for _, cc := range c {
-		for id := range cc {
-			m[id] = struct{}{}
-		}
-	}
-	return m
-}
-
-// Describe returns a (multi-line) representation of the commit indexes for the
-// given lookuper.
-func (c JointConfig) Describe(l AckedIndexer) string {
-	return MajorityConfig(c.IDs()).Describe(l)
-}
-
-// CommittedIndex returns the largest committed index for the given joint
-// quorum. An index is jointly committed if it is committed in both constituent
-// majorities.
-func (c JointConfig) CommittedIndex(l AckedIndexer) Index {
-	idx0 := c[0].CommittedIndex(l)
-	idx1 := c[1].CommittedIndex(l)
-	if idx0 < idx1 {
-		return idx0
-	}
-	return idx1
-}
-
-// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns
-// a result indicating whether the vote is pending, lost, or won. A joint quorum
-// requires both majority quorums to vote in favor.
-func (c JointConfig) VoteResult(votes map[uint64]bool) VoteResult {
-	r1 := c[0].VoteResult(votes)
-	r2 := c[1].VoteResult(votes)
-
-	if r1 == r2 {
-		// If they agree, return the agreed state.
-		return r1
-	}
-	if r1 == VoteLost || r2 == VoteLost {
-		// If either config has lost, loss is the only possible outcome.
-		return VoteLost
-	}
-	// One side won, the other one is pending, so the whole outcome is.
-	return VotePending
-}
diff --git a/vendor/go.etcd.io/etcd/raft/quorum/majority.go b/vendor/go.etcd.io/etcd/raft/quorum/majority.go
deleted file mode 100644
index 8858a36..0000000
--- a/vendor/go.etcd.io/etcd/raft/quorum/majority.go
+++ /dev/null
@@ -1,210 +0,0 @@
-// 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 quorum
-
-import (
-	"fmt"
-	"math"
-	"sort"
-	"strings"
-)
-
-// MajorityConfig is a set of IDs that uses majority quorums to make decisions.
-type MajorityConfig map[uint64]struct{}
-
-func (c MajorityConfig) String() string {
-	sl := make([]uint64, 0, len(c))
-	for id := range c {
-		sl = append(sl, id)
-	}
-	sort.Slice(sl, func(i, j int) bool { return sl[i] < sl[j] })
-	var buf strings.Builder
-	buf.WriteByte('(')
-	for i := range sl {
-		if i > 0 {
-			buf.WriteByte(' ')
-		}
-		fmt.Fprint(&buf, sl[i])
-	}
-	buf.WriteByte(')')
-	return buf.String()
-}
-
-// Describe returns a (multi-line) representation of the commit indexes for the
-// given lookuper.
-func (c MajorityConfig) Describe(l AckedIndexer) string {
-	if len(c) == 0 {
-		return "<empty majority quorum>"
-	}
-	type tup struct {
-		id  uint64
-		idx Index
-		ok  bool // idx found?
-		bar int  // length of bar displayed for this tup
-	}
-
-	// Below, populate .bar so that the i-th largest commit index has bar i (we
-	// plot this as sort of a progress bar). The actual code is a bit more
-	// complicated and also makes sure that equal index => equal bar.
-
-	n := len(c)
-	info := make([]tup, 0, n)
-	for id := range c {
-		idx, ok := l.AckedIndex(id)
-		info = append(info, tup{id: id, idx: idx, ok: ok})
-	}
-
-	// Sort by index
-	sort.Slice(info, func(i, j int) bool {
-		if info[i].idx == info[j].idx {
-			return info[i].id < info[j].id
-		}
-		return info[i].idx < info[j].idx
-	})
-
-	// Populate .bar.
-	for i := range info {
-		if i > 0 && info[i-1].idx < info[i].idx {
-			info[i].bar = i
-		}
-	}
-
-	// Sort by ID.
-	sort.Slice(info, func(i, j int) bool {
-		return info[i].id < info[j].id
-	})
-
-	var buf strings.Builder
-
-	// Print.
-	fmt.Fprint(&buf, strings.Repeat(" ", n)+"    idx\n")
-	for i := range info {
-		bar := info[i].bar
-		if !info[i].ok {
-			fmt.Fprint(&buf, "?"+strings.Repeat(" ", n))
-		} else {
-			fmt.Fprint(&buf, strings.Repeat("x", bar)+">"+strings.Repeat(" ", n-bar))
-		}
-		fmt.Fprintf(&buf, " %5d    (id=%d)\n", info[i].idx, info[i].id)
-	}
-	return buf.String()
-}
-
-// Slice returns the MajorityConfig as a sorted slice.
-func (c MajorityConfig) Slice() []uint64 {
-	var sl []uint64
-	for id := range c {
-		sl = append(sl, id)
-	}
-	sort.Slice(sl, func(i, j int) bool { return sl[i] < sl[j] })
-	return sl
-}
-
-func insertionSort(sl []uint64) {
-	a, b := 0, len(sl)
-	for i := a + 1; i < b; i++ {
-		for j := i; j > a && sl[j] < sl[j-1]; j-- {
-			sl[j], sl[j-1] = sl[j-1], sl[j]
-		}
-	}
-}
-
-// CommittedIndex computes the committed index from those supplied via the
-// provided AckedIndexer (for the active config).
-func (c MajorityConfig) CommittedIndex(l AckedIndexer) Index {
-	n := len(c)
-	if n == 0 {
-		// This plays well with joint quorums which, when one half is the zero
-		// MajorityConfig, should behave like the other half.
-		return math.MaxUint64
-	}
-
-	// Use an on-stack slice to collect the committed indexes when n <= 7
-	// (otherwise we alloc). The alternative is to stash a slice on
-	// MajorityConfig, but this impairs usability (as is, MajorityConfig is just
-	// a map, and that's nice). The assumption is that running with a
-	// replication factor of >7 is rare, and in cases in which it happens
-	// performance is a lesser concern (additionally the performance
-	// implications of an allocation here are far from drastic).
-	var stk [7]uint64
-	var srt []uint64
-	if len(stk) >= n {
-		srt = stk[:n]
-	} else {
-		srt = make([]uint64, n)
-	}
-
-	{
-		// Fill the slice with the indexes observed. Any unused slots will be
-		// left as zero; these correspond to voters that may report in, but
-		// haven't yet. We fill from the right (since the zeroes will end up on
-		// the left after sorting below anyway).
-		i := n - 1
-		for id := range c {
-			if idx, ok := l.AckedIndex(id); ok {
-				srt[i] = uint64(idx)
-				i--
-			}
-		}
-	}
-
-	// Sort by index. Use a bespoke algorithm (copied from the stdlib's sort
-	// package) to keep srt on the stack.
-	insertionSort(srt)
-
-	// The smallest index into the array for which the value is acked by a
-	// quorum. In other words, from the end of the slice, move n/2+1 to the
-	// left (accounting for zero-indexing).
-	pos := n - (n/2 + 1)
-	return Index(srt[pos])
-}
-
-// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns
-// a result indicating whether the vote is pending (i.e. neither a quorum of
-// yes/no has been reached), won (a quorum of yes has been reached), or lost (a
-// quorum of no has been reached).
-func (c MajorityConfig) VoteResult(votes map[uint64]bool) VoteResult {
-	if len(c) == 0 {
-		// By convention, the elections on an empty config win. This comes in
-		// handy with joint quorums because it'll make a half-populated joint
-		// quorum behave like a majority quorum.
-		return VoteWon
-	}
-
-	ny := [2]int{} // vote counts for no and yes, respectively
-
-	var missing int
-	for id := range c {
-		v, ok := votes[id]
-		if !ok {
-			missing++
-			continue
-		}
-		if v {
-			ny[1]++
-		} else {
-			ny[0]++
-		}
-	}
-
-	q := len(c)/2 + 1
-	if ny[1] >= q {
-		return VoteWon
-	}
-	if ny[1]+missing >= q {
-		return VotePending
-	}
-	return VoteLost
-}
diff --git a/vendor/go.etcd.io/etcd/raft/quorum/quorum.go b/vendor/go.etcd.io/etcd/raft/quorum/quorum.go
deleted file mode 100644
index 2899e46..0000000
--- a/vendor/go.etcd.io/etcd/raft/quorum/quorum.go
+++ /dev/null
@@ -1,58 +0,0 @@
-// 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 quorum
-
-import (
-	"math"
-	"strconv"
-)
-
-// Index is a Raft log position.
-type Index uint64
-
-func (i Index) String() string {
-	if i == math.MaxUint64 {
-		return "∞"
-	}
-	return strconv.FormatUint(uint64(i), 10)
-}
-
-// AckedIndexer allows looking up a commit index for a given ID of a voter
-// from a corresponding MajorityConfig.
-type AckedIndexer interface {
-	AckedIndex(voterID uint64) (idx Index, found bool)
-}
-
-type mapAckIndexer map[uint64]Index
-
-func (m mapAckIndexer) AckedIndex(id uint64) (Index, bool) {
-	idx, ok := m[id]
-	return idx, ok
-}
-
-// VoteResult indicates the outcome of a vote.
-//
-//go:generate stringer -type=VoteResult
-type VoteResult uint8
-
-const (
-	// VotePending indicates that the decision of the vote depends on future
-	// votes, i.e. neither "yes" or "no" has reached quorum yet.
-	VotePending VoteResult = 1 + iota
-	// VoteLost indicates that the quorum has voted "no".
-	VoteLost
-	// VoteWon indicates that the quorum has voted "yes".
-	VoteWon
-)
diff --git a/vendor/go.etcd.io/etcd/raft/quorum/voteresult_string.go b/vendor/go.etcd.io/etcd/raft/quorum/voteresult_string.go
deleted file mode 100644
index 9eca8fd..0000000
--- a/vendor/go.etcd.io/etcd/raft/quorum/voteresult_string.go
+++ /dev/null
@@ -1,26 +0,0 @@
-// Code generated by "stringer -type=VoteResult"; DO NOT EDIT.
-
-package quorum
-
-import "strconv"
-
-func _() {
-	// An "invalid array index" compiler error signifies that the constant values have changed.
-	// Re-run the stringer command to generate them again.
-	var x [1]struct{}
-	_ = x[VotePending-1]
-	_ = x[VoteLost-2]
-	_ = x[VoteWon-3]
-}
-
-const _VoteResult_name = "VotePendingVoteLostVoteWon"
-
-var _VoteResult_index = [...]uint8{0, 11, 19, 26}
-
-func (i VoteResult) String() string {
-	i -= 1
-	if i >= VoteResult(len(_VoteResult_index)-1) {
-		return "VoteResult(" + strconv.FormatInt(int64(i+1), 10) + ")"
-	}
-	return _VoteResult_name[_VoteResult_index[i]:_VoteResult_index[i+1]]
-}
diff --git a/vendor/go.etcd.io/etcd/raft/raft.go b/vendor/go.etcd.io/etcd/raft/raft.go
deleted file mode 100644
index cdcb43d..0000000
--- a/vendor/go.etcd.io/etcd/raft/raft.go
+++ /dev/null
@@ -1,1656 +0,0 @@
-// 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 raft
-
-import (
-	"bytes"
-	"errors"
-	"fmt"
-	"math"
-	"math/rand"
-	"sort"
-	"strings"
-	"sync"
-	"time"
-
-	"go.etcd.io/etcd/raft/confchange"
-	"go.etcd.io/etcd/raft/quorum"
-	pb "go.etcd.io/etcd/raft/raftpb"
-	"go.etcd.io/etcd/raft/tracker"
-)
-
-// None is a placeholder node ID used when there is no leader.
-const None uint64 = 0
-const noLimit = math.MaxUint64
-
-// Possible values for StateType.
-const (
-	StateFollower StateType = iota
-	StateCandidate
-	StateLeader
-	StatePreCandidate
-	numStates
-)
-
-type ReadOnlyOption int
-
-const (
-	// ReadOnlySafe guarantees the linearizability of the read only request by
-	// communicating with the quorum. It is the default and suggested option.
-	ReadOnlySafe ReadOnlyOption = iota
-	// ReadOnlyLeaseBased ensures linearizability of the read only request by
-	// relying on the leader lease. It can be affected by clock drift.
-	// If the clock drift is unbounded, leader might keep the lease longer than it
-	// should (clock can move backward/pause without any bound). ReadIndex is not safe
-	// in that case.
-	ReadOnlyLeaseBased
-)
-
-// Possible values for CampaignType
-const (
-	// campaignPreElection represents the first phase of a normal election when
-	// Config.PreVote is true.
-	campaignPreElection CampaignType = "CampaignPreElection"
-	// campaignElection represents a normal (time-based) election (the second phase
-	// of the election when Config.PreVote is true).
-	campaignElection CampaignType = "CampaignElection"
-	// campaignTransfer represents the type of leader transfer
-	campaignTransfer CampaignType = "CampaignTransfer"
-)
-
-// ErrProposalDropped is returned when the proposal is ignored by some cases,
-// so that the proposer can be notified and fail fast.
-var ErrProposalDropped = errors.New("raft proposal dropped")
-
-// lockedRand is a small wrapper around rand.Rand to provide
-// synchronization among multiple raft groups. Only the methods needed
-// by the code are exposed (e.g. Intn).
-type lockedRand struct {
-	mu   sync.Mutex
-	rand *rand.Rand
-}
-
-func (r *lockedRand) Intn(n int) int {
-	r.mu.Lock()
-	v := r.rand.Intn(n)
-	r.mu.Unlock()
-	return v
-}
-
-var globalRand = &lockedRand{
-	rand: rand.New(rand.NewSource(time.Now().UnixNano())),
-}
-
-// CampaignType represents the type of campaigning
-// the reason we use the type of string instead of uint64
-// is because it's simpler to compare and fill in raft entries
-type CampaignType string
-
-// StateType represents the role of a node in a cluster.
-type StateType uint64
-
-var stmap = [...]string{
-	"StateFollower",
-	"StateCandidate",
-	"StateLeader",
-	"StatePreCandidate",
-}
-
-func (st StateType) String() string {
-	return stmap[uint64(st)]
-}
-
-// Config contains the parameters to start a raft.
-type Config struct {
-	// ID is the identity of the local raft. ID cannot be 0.
-	ID uint64
-
-	// peers contains the IDs of all nodes (including self) in the raft cluster. It
-	// should only be set when starting a new raft cluster. Restarting raft from
-	// previous configuration will panic if peers is set. peer is private and only
-	// used for testing right now.
-	peers []uint64
-
-	// learners contains the IDs of all learner nodes (including self if the
-	// local node is a learner) in the raft cluster. learners only receives
-	// entries from the leader node. It does not vote or promote itself.
-	learners []uint64
-
-	// ElectionTick is the number of Node.Tick invocations that must pass between
-	// elections. That is, if a follower does not receive any message from the
-	// leader of current term before ElectionTick has elapsed, it will become
-	// candidate and start an election. ElectionTick must be greater than
-	// HeartbeatTick. We suggest ElectionTick = 10 * HeartbeatTick to avoid
-	// unnecessary leader switching.
-	ElectionTick int
-	// HeartbeatTick is the number of Node.Tick invocations that must pass between
-	// heartbeats. That is, a leader sends heartbeat messages to maintain its
-	// leadership every HeartbeatTick ticks.
-	HeartbeatTick int
-
-	// Storage is the storage for raft. raft generates entries and states to be
-	// stored in storage. raft reads the persisted entries and states out of
-	// Storage when it needs. raft reads out the previous state and configuration
-	// out of storage when restarting.
-	Storage Storage
-	// Applied is the last applied index. It should only be set when restarting
-	// raft. raft will not return entries to the application smaller or equal to
-	// Applied. If Applied is unset when restarting, raft might return previous
-	// applied entries. This is a very application dependent configuration.
-	Applied uint64
-
-	// MaxSizePerMsg limits the max byte size of each append message. Smaller
-	// value lowers the raft recovery cost(initial probing and message lost
-	// during normal operation). On the other side, it might affect the
-	// throughput during normal replication. Note: math.MaxUint64 for unlimited,
-	// 0 for at most one entry per message.
-	MaxSizePerMsg uint64
-	// MaxCommittedSizePerReady limits the size of the committed entries which
-	// can be applied.
-	MaxCommittedSizePerReady uint64
-	// MaxUncommittedEntriesSize limits the aggregate byte size of the
-	// uncommitted entries that may be appended to a leader's log. Once this
-	// limit is exceeded, proposals will begin to return ErrProposalDropped
-	// errors. Note: 0 for no limit.
-	MaxUncommittedEntriesSize uint64
-	// MaxInflightMsgs limits the max number of in-flight append messages during
-	// optimistic replication phase. The application transportation layer usually
-	// has its own sending buffer over TCP/UDP. Setting MaxInflightMsgs to avoid
-	// overflowing that sending buffer. TODO (xiangli): feedback to application to
-	// limit the proposal rate?
-	MaxInflightMsgs int
-
-	// CheckQuorum specifies if the leader should check quorum activity. Leader
-	// steps down when quorum is not active for an electionTimeout.
-	CheckQuorum bool
-
-	// PreVote enables the Pre-Vote algorithm described in raft thesis section
-	// 9.6. This prevents disruption when a node that has been partitioned away
-	// rejoins the cluster.
-	PreVote bool
-
-	// ReadOnlyOption specifies how the read only request is processed.
-	//
-	// ReadOnlySafe guarantees the linearizability of the read only request by
-	// communicating with the quorum. It is the default and suggested option.
-	//
-	// ReadOnlyLeaseBased ensures linearizability of the read only request by
-	// relying on the leader lease. It can be affected by clock drift.
-	// If the clock drift is unbounded, leader might keep the lease longer than it
-	// should (clock can move backward/pause without any bound). ReadIndex is not safe
-	// in that case.
-	// CheckQuorum MUST be enabled if ReadOnlyOption is ReadOnlyLeaseBased.
-	ReadOnlyOption ReadOnlyOption
-
-	// Logger is the logger used for raft log. For multinode which can host
-	// multiple raft group, each raft group can have its own logger
-	Logger Logger
-
-	// DisableProposalForwarding set to true means that followers will drop
-	// proposals, rather than forwarding them to the leader. One use case for
-	// this feature would be in a situation where the Raft leader is used to
-	// compute the data of a proposal, for example, adding a timestamp from a
-	// hybrid logical clock to data in a monotonically increasing way. Forwarding
-	// should be disabled to prevent a follower with an inaccurate hybrid
-	// logical clock from assigning the timestamp and then forwarding the data
-	// to the leader.
-	DisableProposalForwarding bool
-}
-
-func (c *Config) validate() error {
-	if c.ID == None {
-		return errors.New("cannot use none as id")
-	}
-
-	if c.HeartbeatTick <= 0 {
-		return errors.New("heartbeat tick must be greater than 0")
-	}
-
-	if c.ElectionTick <= c.HeartbeatTick {
-		return errors.New("election tick must be greater than heartbeat tick")
-	}
-
-	if c.Storage == nil {
-		return errors.New("storage cannot be nil")
-	}
-
-	if c.MaxUncommittedEntriesSize == 0 {
-		c.MaxUncommittedEntriesSize = noLimit
-	}
-
-	// default MaxCommittedSizePerReady to MaxSizePerMsg because they were
-	// previously the same parameter.
-	if c.MaxCommittedSizePerReady == 0 {
-		c.MaxCommittedSizePerReady = c.MaxSizePerMsg
-	}
-
-	if c.MaxInflightMsgs <= 0 {
-		return errors.New("max inflight messages must be greater than 0")
-	}
-
-	if c.Logger == nil {
-		c.Logger = raftLogger
-	}
-
-	if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum {
-		return errors.New("CheckQuorum must be enabled when ReadOnlyOption is ReadOnlyLeaseBased")
-	}
-
-	return nil
-}
-
-type raft struct {
-	id uint64
-
-	Term uint64
-	Vote uint64
-
-	readStates []ReadState
-
-	// the log
-	raftLog *raftLog
-
-	maxMsgSize         uint64
-	maxUncommittedSize uint64
-	// TODO(tbg): rename to trk.
-	prs tracker.ProgressTracker
-
-	state StateType
-
-	// isLearner is true if the local raft node is a learner.
-	isLearner bool
-
-	msgs []pb.Message
-
-	// the leader id
-	lead uint64
-	// leadTransferee is id of the leader transfer target when its value is not zero.
-	// Follow the procedure defined in raft thesis 3.10.
-	leadTransferee uint64
-	// Only one conf change may be pending (in the log, but not yet
-	// applied) at a time. This is enforced via pendingConfIndex, which
-	// is set to a value >= the log index of the latest pending
-	// configuration change (if any). Config changes are only allowed to
-	// be proposed if the leader's applied index is greater than this
-	// value.
-	pendingConfIndex uint64
-	// an estimate of the size of the uncommitted tail of the Raft log. Used to
-	// prevent unbounded log growth. Only maintained by the leader. Reset on
-	// term changes.
-	uncommittedSize uint64
-
-	readOnly *readOnly
-
-	// number of ticks since it reached last electionTimeout when it is leader
-	// or candidate.
-	// number of ticks since it reached last electionTimeout or received a
-	// valid message from current leader when it is a follower.
-	electionElapsed int
-
-	// number of ticks since it reached last heartbeatTimeout.
-	// only leader keeps heartbeatElapsed.
-	heartbeatElapsed int
-
-	checkQuorum bool
-	preVote     bool
-
-	heartbeatTimeout int
-	electionTimeout  int
-	// randomizedElectionTimeout is a random number between
-	// [electiontimeout, 2 * electiontimeout - 1]. It gets reset
-	// when raft changes its state to follower or candidate.
-	randomizedElectionTimeout int
-	disableProposalForwarding bool
-
-	tick func()
-	step stepFunc
-
-	logger Logger
-}
-
-func newRaft(c *Config) *raft {
-	if err := c.validate(); err != nil {
-		panic(err.Error())
-	}
-	raftlog := newLogWithSize(c.Storage, c.Logger, c.MaxCommittedSizePerReady)
-	hs, cs, err := c.Storage.InitialState()
-	if err != nil {
-		panic(err) // TODO(bdarnell)
-	}
-
-	if len(c.peers) > 0 || len(c.learners) > 0 {
-		if len(cs.Voters) > 0 || len(cs.Learners) > 0 {
-			// TODO(bdarnell): the peers argument is always nil except in
-			// tests; the argument should be removed and these tests should be
-			// updated to specify their nodes through a snapshot.
-			panic("cannot specify both newRaft(peers, learners) and ConfState.(Voters, Learners)")
-		}
-		cs.Voters = c.peers
-		cs.Learners = c.learners
-	}
-
-	r := &raft{
-		id:                        c.ID,
-		lead:                      None,
-		isLearner:                 false,
-		raftLog:                   raftlog,
-		maxMsgSize:                c.MaxSizePerMsg,
-		maxUncommittedSize:        c.MaxUncommittedEntriesSize,
-		prs:                       tracker.MakeProgressTracker(c.MaxInflightMsgs),
-		electionTimeout:           c.ElectionTick,
-		heartbeatTimeout:          c.HeartbeatTick,
-		logger:                    c.Logger,
-		checkQuorum:               c.CheckQuorum,
-		preVote:                   c.PreVote,
-		readOnly:                  newReadOnly(c.ReadOnlyOption),
-		disableProposalForwarding: c.DisableProposalForwarding,
-	}
-
-	cfg, prs, err := confchange.Restore(confchange.Changer{
-		Tracker:   r.prs,
-		LastIndex: raftlog.lastIndex(),
-	}, cs)
-	if err != nil {
-		panic(err)
-	}
-	assertConfStatesEquivalent(r.logger, cs, r.switchToConfig(cfg, prs))
-
-	if !IsEmptyHardState(hs) {
-		r.loadState(hs)
-	}
-	if c.Applied > 0 {
-		raftlog.appliedTo(c.Applied)
-	}
-	r.becomeFollower(r.Term, None)
-
-	var nodesStrs []string
-	for _, n := range r.prs.VoterNodes() {
-		nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
-	}
-
-	r.logger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
-		r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
-	return r
-}
-
-func (r *raft) hasLeader() bool { return r.lead != None }
-
-func (r *raft) softState() *SoftState { return &SoftState{Lead: r.lead, RaftState: r.state} }
-
-func (r *raft) hardState() pb.HardState {
-	return pb.HardState{
-		Term:   r.Term,
-		Vote:   r.Vote,
-		Commit: r.raftLog.committed,
-	}
-}
-
-// send persists state to stable storage and then sends to its mailbox.
-func (r *raft) send(m pb.Message) {
-	m.From = r.id
-	if m.Type == pb.MsgVote || m.Type == pb.MsgVoteResp || m.Type == pb.MsgPreVote || m.Type == pb.MsgPreVoteResp {
-		if m.Term == 0 {
-			// All {pre-,}campaign messages need to have the term set when
-			// sending.
-			// - MsgVote: m.Term is the term the node is campaigning for,
-			//   non-zero as we increment the term when campaigning.
-			// - MsgVoteResp: m.Term is the new r.Term if the MsgVote was
-			//   granted, non-zero for the same reason MsgVote is
-			// - MsgPreVote: m.Term is the term the node will campaign,
-			//   non-zero as we use m.Term to indicate the next term we'll be
-			//   campaigning for
-			// - MsgPreVoteResp: m.Term is the term received in the original
-			//   MsgPreVote if the pre-vote was granted, non-zero for the
-			//   same reasons MsgPreVote is
-			panic(fmt.Sprintf("term should be set when sending %s", m.Type))
-		}
-	} else {
-		if m.Term != 0 {
-			panic(fmt.Sprintf("term should not be set when sending %s (was %d)", m.Type, m.Term))
-		}
-		// do not attach term to MsgProp, MsgReadIndex
-		// proposals are a way to forward to the leader and
-		// should be treated as local message.
-		// MsgReadIndex is also forwarded to leader.
-		if m.Type != pb.MsgProp && m.Type != pb.MsgReadIndex {
-			m.Term = r.Term
-		}
-	}
-	r.msgs = append(r.msgs, m)
-}
-
-// sendAppend sends an append RPC with new entries (if any) and the
-// current commit index to the given peer.
-func (r *raft) sendAppend(to uint64) {
-	r.maybeSendAppend(to, true)
-}
-
-// maybeSendAppend sends an append RPC with new entries to the given peer,
-// if necessary. Returns true if a message was sent. The sendIfEmpty
-// argument controls whether messages with no entries will be sent
-// ("empty" messages are useful to convey updated Commit indexes, but
-// are undesirable when we're sending multiple messages in a batch).
-func (r *raft) maybeSendAppend(to uint64, sendIfEmpty bool) bool {
-	pr := r.prs.Progress[to]
-	if pr.IsPaused() {
-		return false
-	}
-	m := pb.Message{}
-	m.To = to
-
-	term, errt := r.raftLog.term(pr.Next - 1)
-	ents, erre := r.raftLog.entries(pr.Next, r.maxMsgSize)
-	if len(ents) == 0 && !sendIfEmpty {
-		return false
-	}
-
-	if errt != nil || erre != nil { // send snapshot if we failed to get term or entries
-		if !pr.RecentActive {
-			r.logger.Debugf("ignore sending snapshot to %x since it is not recently active", to)
-			return false
-		}
-
-		m.Type = pb.MsgSnap
-		snapshot, err := r.raftLog.snapshot()
-		if err != nil {
-			if err == ErrSnapshotTemporarilyUnavailable {
-				r.logger.Debugf("%x failed to send snapshot to %x because snapshot is temporarily unavailable", r.id, to)
-				return false
-			}
-			panic(err) // TODO(bdarnell)
-		}
-		if IsEmptySnap(snapshot) {
-			panic("need non-empty snapshot")
-		}
-		m.Snapshot = snapshot
-		sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
-		r.logger.Debugf("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
-			r.id, r.raftLog.firstIndex(), r.raftLog.committed, sindex, sterm, to, pr)
-		pr.BecomeSnapshot(sindex)
-		r.logger.Debugf("%x paused sending replication messages to %x [%s]", r.id, to, pr)
-	} else {
-		m.Type = pb.MsgApp
-		m.Index = pr.Next - 1
-		m.LogTerm = term
-		m.Entries = ents
-		m.Commit = r.raftLog.committed
-		if n := len(m.Entries); n != 0 {
-			switch pr.State {
-			// optimistically increase the next when in StateReplicate
-			case tracker.StateReplicate:
-				last := m.Entries[n-1].Index
-				pr.OptimisticUpdate(last)
-				pr.Inflights.Add(last)
-			case tracker.StateProbe:
-				pr.ProbeSent = true
-			default:
-				r.logger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
-			}
-		}
-	}
-	r.send(m)
-	return true
-}
-
-// sendHeartbeat sends a heartbeat RPC to the given peer.
-func (r *raft) sendHeartbeat(to uint64, ctx []byte) {
-	// Attach the commit as min(to.matched, r.committed).
-	// When the leader sends out heartbeat message,
-	// the receiver(follower) might not be matched with the leader
-	// or it might not have all the committed entries.
-	// The leader MUST NOT forward the follower's commit to
-	// an unmatched index.
-	commit := min(r.prs.Progress[to].Match, r.raftLog.committed)
-	m := pb.Message{
-		To:      to,
-		Type:    pb.MsgHeartbeat,
-		Commit:  commit,
-		Context: ctx,
-	}
-
-	r.send(m)
-}
-
-// bcastAppend sends RPC, with entries to all peers that are not up-to-date
-// according to the progress recorded in r.prs.
-func (r *raft) bcastAppend() {
-	r.prs.Visit(func(id uint64, _ *tracker.Progress) {
-		if id == r.id {
-			return
-		}
-		r.sendAppend(id)
-	})
-}
-
-// bcastHeartbeat sends RPC, without entries to all the peers.
-func (r *raft) bcastHeartbeat() {
-	lastCtx := r.readOnly.lastPendingRequestCtx()
-	if len(lastCtx) == 0 {
-		r.bcastHeartbeatWithCtx(nil)
-	} else {
-		r.bcastHeartbeatWithCtx([]byte(lastCtx))
-	}
-}
-
-func (r *raft) bcastHeartbeatWithCtx(ctx []byte) {
-	r.prs.Visit(func(id uint64, _ *tracker.Progress) {
-		if id == r.id {
-			return
-		}
-		r.sendHeartbeat(id, ctx)
-	})
-}
-
-func (r *raft) advance(rd Ready) {
-	// If entries were applied (or a snapshot), update our cursor for
-	// the next Ready. Note that if the current HardState contains a
-	// new Commit index, this does not mean that we're also applying
-	// all of the new entries due to commit pagination by size.
-	if index := rd.appliedCursor(); index > 0 {
-		r.raftLog.appliedTo(index)
-		if r.prs.Config.AutoLeave && index >= r.pendingConfIndex && r.state == StateLeader {
-			// If the current (and most recent, at least for this leader's term)
-			// configuration should be auto-left, initiate that now.
-			ccdata, err := (&pb.ConfChangeV2{}).Marshal()
-			if err != nil {
-				panic(err)
-			}
-			ent := pb.Entry{
-				Type: pb.EntryConfChangeV2,
-				Data: ccdata,
-			}
-			if !r.appendEntry(ent) {
-				// If we could not append the entry, bump the pending conf index
-				// so that we'll try again later.
-				//
-				// TODO(tbg): test this case.
-				r.pendingConfIndex = r.raftLog.lastIndex()
-			} else {
-				r.logger.Infof("initiating automatic transition out of joint configuration %s", r.prs.Config)
-			}
-		}
-	}
-	r.reduceUncommittedSize(rd.CommittedEntries)
-
-	if len(rd.Entries) > 0 {
-		e := rd.Entries[len(rd.Entries)-1]
-		r.raftLog.stableTo(e.Index, e.Term)
-	}
-	if !IsEmptySnap(rd.Snapshot) {
-		r.raftLog.stableSnapTo(rd.Snapshot.Metadata.Index)
-	}
-}
-
-// maybeCommit attempts to advance the commit index. Returns true if
-// the commit index changed (in which case the caller should call
-// r.bcastAppend).
-func (r *raft) maybeCommit() bool {
-	mci := r.prs.Committed()
-	return r.raftLog.maybeCommit(mci, r.Term)
-}
-
-func (r *raft) reset(term uint64) {
-	if r.Term != term {
-		r.Term = term
-		r.Vote = None
-	}
-	r.lead = None
-
-	r.electionElapsed = 0
-	r.heartbeatElapsed = 0
-	r.resetRandomizedElectionTimeout()
-
-	r.abortLeaderTransfer()
-
-	r.prs.ResetVotes()
-	r.prs.Visit(func(id uint64, pr *tracker.Progress) {
-		*pr = tracker.Progress{
-			Match:     0,
-			Next:      r.raftLog.lastIndex() + 1,
-			Inflights: tracker.NewInflights(r.prs.MaxInflight),
-			IsLearner: pr.IsLearner,
-		}
-		if id == r.id {
-			pr.Match = r.raftLog.lastIndex()
-		}
-	})
-
-	r.pendingConfIndex = 0
-	r.uncommittedSize = 0
-	r.readOnly = newReadOnly(r.readOnly.option)
-}
-
-func (r *raft) appendEntry(es ...pb.Entry) (accepted bool) {
-	li := r.raftLog.lastIndex()
-	for i := range es {
-		es[i].Term = r.Term
-		es[i].Index = li + 1 + uint64(i)
-	}
-	// Track the size of this uncommitted proposal.
-	if !r.increaseUncommittedSize(es) {
-		r.logger.Debugf(
-			"%x appending new entries to log would exceed uncommitted entry size limit; dropping proposal",
-			r.id,
-		)
-		// Drop the proposal.
-		return false
-	}
-	// use latest "last" index after truncate/append
-	li = r.raftLog.append(es...)
-	r.prs.Progress[r.id].MaybeUpdate(li)
-	// Regardless of maybeCommit's return, our caller will call bcastAppend.
-	r.maybeCommit()
-	return true
-}
-
-// tickElection is run by followers and candidates after r.electionTimeout.
-func (r *raft) tickElection() {
-	r.electionElapsed++
-
-	if r.promotable() && r.pastElectionTimeout() {
-		r.electionElapsed = 0
-		r.Step(pb.Message{From: r.id, Type: pb.MsgHup})
-	}
-}
-
-// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout.
-func (r *raft) tickHeartbeat() {
-	r.heartbeatElapsed++
-	r.electionElapsed++
-
-	if r.electionElapsed >= r.electionTimeout {
-		r.electionElapsed = 0
-		if r.checkQuorum {
-			r.Step(pb.Message{From: r.id, Type: pb.MsgCheckQuorum})
-		}
-		// If current leader cannot transfer leadership in electionTimeout, it becomes leader again.
-		if r.state == StateLeader && r.leadTransferee != None {
-			r.abortLeaderTransfer()
-		}
-	}
-
-	if r.state != StateLeader {
-		return
-	}
-
-	if r.heartbeatElapsed >= r.heartbeatTimeout {
-		r.heartbeatElapsed = 0
-		r.Step(pb.Message{From: r.id, Type: pb.MsgBeat})
-	}
-}
-
-func (r *raft) becomeFollower(term uint64, lead uint64) {
-	r.step = stepFollower
-	r.reset(term)
-	r.tick = r.tickElection
-	r.lead = lead
-	r.state = StateFollower
-	r.logger.Infof("%x became follower at term %d", r.id, r.Term)
-}
-
-func (r *raft) becomeCandidate() {
-	// TODO(xiangli) remove the panic when the raft implementation is stable
-	if r.state == StateLeader {
-		panic("invalid transition [leader -> candidate]")
-	}
-	r.step = stepCandidate
-	r.reset(r.Term + 1)
-	r.tick = r.tickElection
-	r.Vote = r.id
-	r.state = StateCandidate
-	r.logger.Infof("%x became candidate at term %d", r.id, r.Term)
-}
-
-func (r *raft) becomePreCandidate() {
-	// TODO(xiangli) remove the panic when the raft implementation is stable
-	if r.state == StateLeader {
-		panic("invalid transition [leader -> pre-candidate]")
-	}
-	// Becoming a pre-candidate changes our step functions and state,
-	// but doesn't change anything else. In particular it does not increase
-	// r.Term or change r.Vote.
-	r.step = stepCandidate
-	r.prs.ResetVotes()
-	r.tick = r.tickElection
-	r.lead = None
-	r.state = StatePreCandidate
-	r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term)
-}
-
-func (r *raft) becomeLeader() {
-	// TODO(xiangli) remove the panic when the raft implementation is stable
-	if r.state == StateFollower {
-		panic("invalid transition [follower -> leader]")
-	}
-	r.step = stepLeader
-	r.reset(r.Term)
-	r.tick = r.tickHeartbeat
-	r.lead = r.id
-	r.state = StateLeader
-	// Followers enter replicate mode when they've been successfully probed
-	// (perhaps after having received a snapshot as a result). The leader is
-	// trivially in this state. Note that r.reset() has initialized this
-	// progress with the last index already.
-	r.prs.Progress[r.id].BecomeReplicate()
-
-	// Conservatively set the pendingConfIndex to the last index in the
-	// log. There may or may not be a pending config change, but it's
-	// safe to delay any future proposals until we commit all our
-	// pending log entries, and scanning the entire tail of the log
-	// could be expensive.
-	r.pendingConfIndex = r.raftLog.lastIndex()
-
-	emptyEnt := pb.Entry{Data: nil}
-	if !r.appendEntry(emptyEnt) {
-		// This won't happen because we just called reset() above.
-		r.logger.Panic("empty entry was dropped")
-	}
-	// As a special case, don't count the initial empty entry towards the
-	// uncommitted log quota. This is because we want to preserve the
-	// behavior of allowing one entry larger than quota if the current
-	// usage is zero.
-	r.reduceUncommittedSize([]pb.Entry{emptyEnt})
-	r.logger.Infof("%x became leader at term %d", r.id, r.Term)
-}
-
-// campaign transitions the raft instance to candidate state. This must only be
-// called after verifying that this is a legitimate transition.
-func (r *raft) campaign(t CampaignType) {
-	if !r.promotable() {
-		// This path should not be hit (callers are supposed to check), but
-		// better safe than sorry.
-		r.logger.Warningf("%x is unpromotable; campaign() should have been called", r.id)
-	}
-	var term uint64
-	var voteMsg pb.MessageType
-	if t == campaignPreElection {
-		r.becomePreCandidate()
-		voteMsg = pb.MsgPreVote
-		// PreVote RPCs are sent for the next term before we've incremented r.Term.
-		term = r.Term + 1
-	} else {
-		r.becomeCandidate()
-		voteMsg = pb.MsgVote
-		term = r.Term
-	}
-	if _, _, res := r.poll(r.id, voteRespMsgType(voteMsg), true); res == quorum.VoteWon {
-		// We won the election after voting for ourselves (which must mean that
-		// this is a single-node cluster). Advance to the next state.
-		if t == campaignPreElection {
-			r.campaign(campaignElection)
-		} else {
-			r.becomeLeader()
-		}
-		return
-	}
-	var ids []uint64
-	{
-		idMap := r.prs.Voters.IDs()
-		ids = make([]uint64, 0, len(idMap))
-		for id := range idMap {
-			ids = append(ids, id)
-		}
-		sort.Slice(ids, func(i, j int) bool { return ids[i] < ids[j] })
-	}
-	for _, id := range ids {
-		if id == r.id {
-			continue
-		}
-		r.logger.Infof("%x [logterm: %d, index: %d] sent %s request to %x at term %d",
-			r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), voteMsg, id, r.Term)
-
-		var ctx []byte
-		if t == campaignTransfer {
-			ctx = []byte(t)
-		}
-		r.send(pb.Message{Term: term, To: id, Type: voteMsg, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm(), Context: ctx})
-	}
-}
-
-func (r *raft) poll(id uint64, t pb.MessageType, v bool) (granted int, rejected int, result quorum.VoteResult) {
-	if v {
-		r.logger.Infof("%x received %s from %x at term %d", r.id, t, id, r.Term)
-	} else {
-		r.logger.Infof("%x received %s rejection from %x at term %d", r.id, t, id, r.Term)
-	}
-	r.prs.RecordVote(id, v)
-	return r.prs.TallyVotes()
-}
-
-func (r *raft) Step(m pb.Message) error {
-	// Handle the message term, which may result in our stepping down to a follower.
-	switch {
-	case m.Term == 0:
-		// local message
-	case m.Term > r.Term:
-		if m.Type == pb.MsgVote || m.Type == pb.MsgPreVote {
-			force := bytes.Equal(m.Context, []byte(campaignTransfer))
-			inLease := r.checkQuorum && r.lead != None && r.electionElapsed < r.electionTimeout
-			if !force && inLease {
-				// If a server receives a RequestVote request within the minimum election timeout
-				// of hearing from a current leader, it does not update its term or grant its vote
-				r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] ignored %s from %x [logterm: %d, index: %d] at term %d: lease is not expired (remaining ticks: %d)",
-					r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term, r.electionTimeout-r.electionElapsed)
-				return nil
-			}
-		}
-		switch {
-		case m.Type == pb.MsgPreVote:
-			// Never change our term in response to a PreVote
-		case m.Type == pb.MsgPreVoteResp && !m.Reject:
-			// We send pre-vote requests with a term in our future. If the
-			// pre-vote is granted, we will increment our term when we get a
-			// quorum. If it is not, the term comes from the node that
-			// rejected our vote so we should become a follower at the new
-			// term.
-		default:
-			r.logger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
-				r.id, r.Term, m.Type, m.From, m.Term)
-			if m.Type == pb.MsgApp || m.Type == pb.MsgHeartbeat || m.Type == pb.MsgSnap {
-				r.becomeFollower(m.Term, m.From)
-			} else {
-				r.becomeFollower(m.Term, None)
-			}
-		}
-
-	case m.Term < r.Term:
-		if (r.checkQuorum || r.preVote) && (m.Type == pb.MsgHeartbeat || m.Type == pb.MsgApp) {
-			// We have received messages from a leader at a lower term. It is possible
-			// that these messages were simply delayed in the network, but this could
-			// also mean that this node has advanced its term number during a network
-			// partition, and it is now unable to either win an election or to rejoin
-			// the majority on the old term. If checkQuorum is false, this will be
-			// handled by incrementing term numbers in response to MsgVote with a
-			// higher term, but if checkQuorum is true we may not advance the term on
-			// MsgVote and must generate other messages to advance the term. The net
-			// result of these two features is to minimize the disruption caused by
-			// nodes that have been removed from the cluster's configuration: a
-			// removed node will send MsgVotes (or MsgPreVotes) which will be ignored,
-			// but it will not receive MsgApp or MsgHeartbeat, so it will not create
-			// disruptive term increases, by notifying leader of this node's activeness.
-			// The above comments also true for Pre-Vote
-			//
-			// When follower gets isolated, it soon starts an election ending
-			// up with a higher term than leader, although it won't receive enough
-			// votes to win the election. When it regains connectivity, this response
-			// with "pb.MsgAppResp" of higher term would force leader to step down.
-			// However, this disruption is inevitable to free this stuck node with
-			// fresh election. This can be prevented with Pre-Vote phase.
-			r.send(pb.Message{To: m.From, Type: pb.MsgAppResp})
-		} else if m.Type == pb.MsgPreVote {
-			// Before Pre-Vote enable, there may have candidate with higher term,
-			// but less log. After update to Pre-Vote, the cluster may deadlock if
-			// we drop messages with a lower term.
-			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
-				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
-			r.send(pb.Message{To: m.From, Term: r.Term, Type: pb.MsgPreVoteResp, Reject: true})
-		} else {
-			// ignore other cases
-			r.logger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
-				r.id, r.Term, m.Type, m.From, m.Term)
-		}
-		return nil
-	}
-
-	switch m.Type {
-	case pb.MsgHup:
-		if r.state != StateLeader {
-			if !r.promotable() {
-				r.logger.Warningf("%x is unpromotable and can not campaign; ignoring MsgHup", r.id)
-				return nil
-			}
-			ents, err := r.raftLog.slice(r.raftLog.applied+1, r.raftLog.committed+1, noLimit)
-			if err != nil {
-				r.logger.Panicf("unexpected error getting unapplied entries (%v)", err)
-			}
-			if n := numOfPendingConf(ents); n != 0 && r.raftLog.committed > r.raftLog.applied {
-				r.logger.Warningf("%x cannot campaign at term %d since there are still %d pending configuration changes to apply", r.id, r.Term, n)
-				return nil
-			}
-
-			r.logger.Infof("%x is starting a new election at term %d", r.id, r.Term)
-			if r.preVote {
-				r.campaign(campaignPreElection)
-			} else {
-				r.campaign(campaignElection)
-			}
-		} else {
-			r.logger.Debugf("%x ignoring MsgHup because already leader", r.id)
-		}
-
-	case pb.MsgVote, pb.MsgPreVote:
-		// We can vote if this is a repeat of a vote we've already cast...
-		canVote := r.Vote == m.From ||
-			// ...we haven't voted and we don't think there's a leader yet in this term...
-			(r.Vote == None && r.lead == None) ||
-			// ...or this is a PreVote for a future term...
-			(m.Type == pb.MsgPreVote && m.Term > r.Term)
-		// ...and we believe the candidate is up to date.
-		if canVote && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
-			// Note: it turns out that that learners must be allowed to cast votes.
-			// This seems counter- intuitive but is necessary in the situation in which
-			// a learner has been promoted (i.e. is now a voter) but has not learned
-			// about this yet.
-			// For example, consider a group in which id=1 is a learner and id=2 and
-			// id=3 are voters. A configuration change promoting 1 can be committed on
-			// the quorum `{2,3}` without the config change being appended to the
-			// learner's log. If the leader (say 2) fails, there are de facto two
-			// voters remaining. Only 3 can win an election (due to its log containing
-			// all committed entries), but to do so it will need 1 to vote. But 1
-			// considers itself a learner and will continue to do so until 3 has
-			// stepped up as leader, replicates the conf change to 1, and 1 applies it.
-			// Ultimately, by receiving a request to vote, the learner realizes that
-			// the candidate believes it to be a voter, and that it should act
-			// accordingly. The candidate's config may be stale, too; but in that case
-			// it won't win the election, at least in the absence of the bug discussed
-			// in:
-			// https://github.com/etcd-io/etcd/issues/7625#issuecomment-488798263.
-			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] cast %s for %x [logterm: %d, index: %d] at term %d",
-				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
-			// When responding to Msg{Pre,}Vote messages we include the term
-			// from the message, not the local term. To see why, consider the
-			// case where a single node was previously partitioned away and
-			// it's local term is now out of date. If we include the local term
-			// (recall that for pre-votes we don't update the local term), the
-			// (pre-)campaigning node on the other end will proceed to ignore
-			// the message (it ignores all out of date messages).
-			// The term in the original message and current local term are the
-			// same in the case of regular votes, but different for pre-votes.
-			r.send(pb.Message{To: m.From, Term: m.Term, Type: voteRespMsgType(m.Type)})
-			if m.Type == pb.MsgVote {
-				// Only record real votes.
-				r.electionElapsed = 0
-				r.Vote = m.From
-			}
-		} else {
-			r.logger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected %s from %x [logterm: %d, index: %d] at term %d",
-				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.Type, m.From, m.LogTerm, m.Index, r.Term)
-			r.send(pb.Message{To: m.From, Term: r.Term, Type: voteRespMsgType(m.Type), Reject: true})
-		}
-
-	default:
-		err := r.step(r, m)
-		if err != nil {
-			return err
-		}
-	}
-	return nil
-}
-
-type stepFunc func(r *raft, m pb.Message) error
-
-func stepLeader(r *raft, m pb.Message) error {
-	// These message types do not require any progress for m.From.
-	switch m.Type {
-	case pb.MsgBeat:
-		r.bcastHeartbeat()
-		return nil
-	case pb.MsgCheckQuorum:
-		// The leader should always see itself as active. As a precaution, handle
-		// the case in which the leader isn't in the configuration any more (for
-		// example if it just removed itself).
-		//
-		// TODO(tbg): I added a TODO in removeNode, it doesn't seem that the
-		// leader steps down when removing itself. I might be missing something.
-		if pr := r.prs.Progress[r.id]; pr != nil {
-			pr.RecentActive = true
-		}
-		if !r.prs.QuorumActive() {
-			r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
-			r.becomeFollower(r.Term, None)
-		}
-		// Mark everyone (but ourselves) as inactive in preparation for the next
-		// CheckQuorum.
-		r.prs.Visit(func(id uint64, pr *tracker.Progress) {
-			if id != r.id {
-				pr.RecentActive = false
-			}
-		})
-		return nil
-	case pb.MsgProp:
-		if len(m.Entries) == 0 {
-			r.logger.Panicf("%x stepped empty MsgProp", r.id)
-		}
-		if r.prs.Progress[r.id] == nil {
-			// If we are not currently a member of the range (i.e. this node
-			// was removed from the configuration while serving as leader),
-			// drop any new proposals.
-			return ErrProposalDropped
-		}
-		if r.leadTransferee != None {
-			r.logger.Debugf("%x [term %d] transfer leadership to %x is in progress; dropping proposal", r.id, r.Term, r.leadTransferee)
-			return ErrProposalDropped
-		}
-
-		for i := range m.Entries {
-			e := &m.Entries[i]
-			var cc pb.ConfChangeI
-			if e.Type == pb.EntryConfChange {
-				var ccc pb.ConfChange
-				if err := ccc.Unmarshal(e.Data); err != nil {
-					panic(err)
-				}
-				cc = ccc
-			} else if e.Type == pb.EntryConfChangeV2 {
-				var ccc pb.ConfChangeV2
-				if err := ccc.Unmarshal(e.Data); err != nil {
-					panic(err)
-				}
-				cc = ccc
-			}
-			if cc != nil {
-				alreadyPending := r.pendingConfIndex > r.raftLog.applied
-				alreadyJoint := len(r.prs.Config.Voters[1]) > 0
-				wantsLeaveJoint := len(cc.AsV2().Changes) == 0
-
-				var refused string
-				if alreadyPending {
-					refused = fmt.Sprintf("possible unapplied conf change at index %d (applied to %d)", r.pendingConfIndex, r.raftLog.applied)
-				} else if alreadyJoint && !wantsLeaveJoint {
-					refused = "must transition out of joint config first"
-				} else if !alreadyJoint && wantsLeaveJoint {
-					refused = "not in joint state; refusing empty conf change"
-				}
-
-				if refused != "" {
-					r.logger.Infof("%x ignoring conf change %v at config %s: %s", r.id, cc, r.prs.Config, refused)
-					m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
-				} else {
-					r.pendingConfIndex = r.raftLog.lastIndex() + uint64(i) + 1
-				}
-			}
-		}
-
-		if !r.appendEntry(m.Entries...) {
-			return ErrProposalDropped
-		}
-		r.bcastAppend()
-		return nil
-	case pb.MsgReadIndex:
-		// If more than the local vote is needed, go through a full broadcast,
-		// otherwise optimize.
-		if !r.prs.IsSingleton() {
-			if r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(r.raftLog.committed)) != r.Term {
-				// Reject read only request when this leader has not committed any log entry at its term.
-				return nil
-			}
-
-			// thinking: use an interally defined context instead of the user given context.
-			// We can express this in terms of the term and index instead of a user-supplied value.
-			// This would allow multiple reads to piggyback on the same message.
-			switch r.readOnly.option {
-			case ReadOnlySafe:
-				r.readOnly.addRequest(r.raftLog.committed, m)
-				// The local node automatically acks the request.
-				r.readOnly.recvAck(r.id, m.Entries[0].Data)
-				r.bcastHeartbeatWithCtx(m.Entries[0].Data)
-			case ReadOnlyLeaseBased:
-				ri := r.raftLog.committed
-				if m.From == None || m.From == r.id { // from local member
-					r.readStates = append(r.readStates, ReadState{Index: ri, RequestCtx: m.Entries[0].Data})
-				} else {
-					r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: ri, Entries: m.Entries})
-				}
-			}
-		} else { // only one voting member (the leader) in the cluster
-			if m.From == None || m.From == r.id { // from leader itself
-				r.readStates = append(r.readStates, ReadState{Index: r.raftLog.committed, RequestCtx: m.Entries[0].Data})
-			} else { // from learner member
-				r.send(pb.Message{To: m.From, Type: pb.MsgReadIndexResp, Index: r.raftLog.committed, Entries: m.Entries})
-			}
-		}
-
-		return nil
-	}
-
-	// All other message types require a progress for m.From (pr).
-	pr := r.prs.Progress[m.From]
-	if pr == nil {
-		r.logger.Debugf("%x no progress available for %x", r.id, m.From)
-		return nil
-	}
-	switch m.Type {
-	case pb.MsgAppResp:
-		pr.RecentActive = true
-
-		if m.Reject {
-			r.logger.Debugf("%x received MsgAppResp(MsgApp was rejected, lastindex: %d) from %x for index %d",
-				r.id, m.RejectHint, m.From, m.Index)
-			if pr.MaybeDecrTo(m.Index, m.RejectHint) {
-				r.logger.Debugf("%x decreased progress of %x to [%s]", r.id, m.From, pr)
-				if pr.State == tracker.StateReplicate {
-					pr.BecomeProbe()
-				}
-				r.sendAppend(m.From)
-			}
-		} else {
-			oldPaused := pr.IsPaused()
-			if pr.MaybeUpdate(m.Index) {
-				switch {
-				case pr.State == tracker.StateProbe:
-					pr.BecomeReplicate()
-				case pr.State == tracker.StateSnapshot && pr.Match >= pr.PendingSnapshot:
-					// TODO(tbg): we should also enter this branch if a snapshot is
-					// received that is below pr.PendingSnapshot but which makes it
-					// possible to use the log again.
-					r.logger.Debugf("%x recovered from needing snapshot, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
-					// Transition back to replicating state via probing state
-					// (which takes the snapshot into account). If we didn't
-					// move to replicating state, that would only happen with
-					// the next round of appends (but there may not be a next
-					// round for a while, exposing an inconsistent RaftStatus).
-					pr.BecomeProbe()
-					pr.BecomeReplicate()
-				case pr.State == tracker.StateReplicate:
-					pr.Inflights.FreeLE(m.Index)
-				}
-
-				if r.maybeCommit() {
-					r.bcastAppend()
-				} else if oldPaused {
-					// If we were paused before, this node may be missing the
-					// latest commit index, so send it.
-					r.sendAppend(m.From)
-				}
-				// We've updated flow control information above, which may
-				// allow us to send multiple (size-limited) in-flight messages
-				// at once (such as when transitioning from probe to
-				// replicate, or when freeTo() covers multiple messages). If
-				// we have more entries to send, send as many messages as we
-				// can (without sending empty messages for the commit index)
-				for r.maybeSendAppend(m.From, false) {
-				}
-				// Transfer leadership is in progress.
-				if m.From == r.leadTransferee && pr.Match == r.raftLog.lastIndex() {
-					r.logger.Infof("%x sent MsgTimeoutNow to %x after received MsgAppResp", r.id, m.From)
-					r.sendTimeoutNow(m.From)
-				}
-			}
-		}
-	case pb.MsgHeartbeatResp:
-		pr.RecentActive = true
-		pr.ProbeSent = false
-
-		// free one slot for the full inflights window to allow progress.
-		if pr.State == tracker.StateReplicate && pr.Inflights.Full() {
-			pr.Inflights.FreeFirstOne()
-		}
-		if pr.Match < r.raftLog.lastIndex() {
-			r.sendAppend(m.From)
-		}
-
-		if r.readOnly.option != ReadOnlySafe || len(m.Context) == 0 {
-			return nil
-		}
-
-		if r.prs.Voters.VoteResult(r.readOnly.recvAck(m.From, m.Context)) != quorum.VoteWon {
-			return nil
-		}
-
-		rss := r.readOnly.advance(m)
-		for _, rs := range rss {
-			req := rs.req
-			if req.From == None || req.From == r.id { // from local member
-				r.readStates = append(r.readStates, ReadState{Index: rs.index, RequestCtx: req.Entries[0].Data})
-			} else {
-				r.send(pb.Message{To: req.From, Type: pb.MsgReadIndexResp, Index: rs.index, Entries: req.Entries})
-			}
-		}
-	case pb.MsgSnapStatus:
-		if pr.State != tracker.StateSnapshot {
-			return nil
-		}
-		// TODO(tbg): this code is very similar to the snapshot handling in
-		// MsgAppResp above. In fact, the code there is more correct than the
-		// code here and should likely be updated to match (or even better, the
-		// logic pulled into a newly created Progress state machine handler).
-		if !m.Reject {
-			pr.BecomeProbe()
-			r.logger.Debugf("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
-		} else {
-			// NB: the order here matters or we'll be probing erroneously from
-			// the snapshot index, but the snapshot never applied.
-			pr.PendingSnapshot = 0
-			pr.BecomeProbe()
-			r.logger.Debugf("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
-		}
-		// If snapshot finish, wait for the MsgAppResp from the remote node before sending
-		// out the next MsgApp.
-		// If snapshot failure, wait for a heartbeat interval before next try
-		pr.ProbeSent = true
-	case pb.MsgUnreachable:
-		// During optimistic replication, if the remote becomes unreachable,
-		// there is huge probability that a MsgApp is lost.
-		if pr.State == tracker.StateReplicate {
-			pr.BecomeProbe()
-		}
-		r.logger.Debugf("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
-	case pb.MsgTransferLeader:
-		if pr.IsLearner {
-			r.logger.Debugf("%x is learner. Ignored transferring leadership", r.id)
-			return nil
-		}
-		leadTransferee := m.From
-		lastLeadTransferee := r.leadTransferee
-		if lastLeadTransferee != None {
-			if lastLeadTransferee == leadTransferee {
-				r.logger.Infof("%x [term %d] transfer leadership to %x is in progress, ignores request to same node %x",
-					r.id, r.Term, leadTransferee, leadTransferee)
-				return nil
-			}
-			r.abortLeaderTransfer()
-			r.logger.Infof("%x [term %d] abort previous transferring leadership to %x", r.id, r.Term, lastLeadTransferee)
-		}
-		if leadTransferee == r.id {
-			r.logger.Debugf("%x is already leader. Ignored transferring leadership to self", r.id)
-			return nil
-		}
-		// Transfer leadership to third party.
-		r.logger.Infof("%x [term %d] starts to transfer leadership to %x", r.id, r.Term, leadTransferee)
-		// Transfer leadership should be finished in one electionTimeout, so reset r.electionElapsed.
-		r.electionElapsed = 0
-		r.leadTransferee = leadTransferee
-		if pr.Match == r.raftLog.lastIndex() {
-			r.sendTimeoutNow(leadTransferee)
-			r.logger.Infof("%x sends MsgTimeoutNow to %x immediately as %x already has up-to-date log", r.id, leadTransferee, leadTransferee)
-		} else {
-			r.sendAppend(leadTransferee)
-		}
-	}
-	return nil
-}
-
-// stepCandidate is shared by StateCandidate and StatePreCandidate; the difference is
-// whether they respond to MsgVoteResp or MsgPreVoteResp.
-func stepCandidate(r *raft, m pb.Message) error {
-	// Only handle vote responses corresponding to our candidacy (while in
-	// StateCandidate, we may get stale MsgPreVoteResp messages in this term from
-	// our pre-candidate state).
-	var myVoteRespType pb.MessageType
-	if r.state == StatePreCandidate {
-		myVoteRespType = pb.MsgPreVoteResp
-	} else {
-		myVoteRespType = pb.MsgVoteResp
-	}
-	switch m.Type {
-	case pb.MsgProp:
-		r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
-		return ErrProposalDropped
-	case pb.MsgApp:
-		r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
-		r.handleAppendEntries(m)
-	case pb.MsgHeartbeat:
-		r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
-		r.handleHeartbeat(m)
-	case pb.MsgSnap:
-		r.becomeFollower(m.Term, m.From) // always m.Term == r.Term
-		r.handleSnapshot(m)
-	case myVoteRespType:
-		gr, rj, res := r.poll(m.From, m.Type, !m.Reject)
-		r.logger.Infof("%x has received %d %s votes and %d vote rejections", r.id, gr, m.Type, rj)
-		switch res {
-		case quorum.VoteWon:
-			if r.state == StatePreCandidate {
-				r.campaign(campaignElection)
-			} else {
-				r.becomeLeader()
-				r.bcastAppend()
-			}
-		case quorum.VoteLost:
-			// pb.MsgPreVoteResp contains future term of pre-candidate
-			// m.Term > r.Term; reuse r.Term
-			r.becomeFollower(r.Term, None)
-		}
-	case pb.MsgTimeoutNow:
-		r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From)
-	}
-	return nil
-}
-
-func stepFollower(r *raft, m pb.Message) error {
-	switch m.Type {
-	case pb.MsgProp:
-		if r.lead == None {
-			r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
-			return ErrProposalDropped
-		} else if r.disableProposalForwarding {
-			r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
-			return ErrProposalDropped
-		}
-		m.To = r.lead
-		r.send(m)
-	case pb.MsgApp:
-		r.electionElapsed = 0
-		r.lead = m.From
-		r.handleAppendEntries(m)
-	case pb.MsgHeartbeat:
-		r.electionElapsed = 0
-		r.lead = m.From
-		r.handleHeartbeat(m)
-	case pb.MsgSnap:
-		r.electionElapsed = 0
-		r.lead = m.From
-		r.handleSnapshot(m)
-	case pb.MsgTransferLeader:
-		if r.lead == None {
-			r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term)
-			return nil
-		}
-		m.To = r.lead
-		r.send(m)
-	case pb.MsgTimeoutNow:
-		if r.promotable() {
-			r.logger.Infof("%x [term %d] received MsgTimeoutNow from %x and starts an election to get leadership.", r.id, r.Term, m.From)
-			// Leadership transfers never use pre-vote even if r.preVote is true; we
-			// know we are not recovering from a partition so there is no need for the
-			// extra round trip.
-			r.campaign(campaignTransfer)
-		} else {
-			r.logger.Infof("%x received MsgTimeoutNow from %x but is not promotable", r.id, m.From)
-		}
-	case pb.MsgReadIndex:
-		if r.lead == None {
-			r.logger.Infof("%x no leader at term %d; dropping index reading msg", r.id, r.Term)
-			return nil
-		}
-		m.To = r.lead
-		r.send(m)
-	case pb.MsgReadIndexResp:
-		if len(m.Entries) != 1 {
-			r.logger.Errorf("%x invalid format of MsgReadIndexResp from %x, entries count: %d", r.id, m.From, len(m.Entries))
-			return nil
-		}
-		r.readStates = append(r.readStates, ReadState{Index: m.Index, RequestCtx: m.Entries[0].Data})
-	}
-	return nil
-}
-
-func (r *raft) handleAppendEntries(m pb.Message) {
-	if m.Index < r.raftLog.committed {
-		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
-		return
-	}
-
-	if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
-		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
-	} else {
-		r.logger.Debugf("%x [logterm: %d, index: %d] rejected MsgApp [logterm: %d, index: %d] from %x",
-			r.id, r.raftLog.zeroTermOnErrCompacted(r.raftLog.term(m.Index)), m.Index, m.LogTerm, m.Index, m.From)
-		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
-	}
-}
-
-func (r *raft) handleHeartbeat(m pb.Message) {
-	r.raftLog.commitTo(m.Commit)
-	r.send(pb.Message{To: m.From, Type: pb.MsgHeartbeatResp, Context: m.Context})
-}
-
-func (r *raft) handleSnapshot(m pb.Message) {
-	sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
-	if r.restore(m.Snapshot) {
-		r.logger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
-			r.id, r.raftLog.committed, sindex, sterm)
-		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
-	} else {
-		r.logger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
-			r.id, r.raftLog.committed, sindex, sterm)
-		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
-	}
-}
-
-// restore recovers the state machine from a snapshot. It restores the log and the
-// configuration of state machine. If this method returns false, the snapshot was
-// ignored, either because it was obsolete or because of an error.
-func (r *raft) restore(s pb.Snapshot) bool {
-	if s.Metadata.Index <= r.raftLog.committed {
-		return false
-	}
-	if r.state != StateFollower {
-		// This is defense-in-depth: if the leader somehow ended up applying a
-		// snapshot, it could move into a new term without moving into a
-		// follower state. This should never fire, but if it did, we'd have
-		// prevented damage by returning early, so log only a loud warning.
-		//
-		// At the time of writing, the instance is guaranteed to be in follower
-		// state when this method is called.
-		r.logger.Warningf("%x attempted to restore snapshot as leader; should never happen", r.id)
-		r.becomeFollower(r.Term+1, None)
-		return false
-	}
-
-	// More defense-in-depth: throw away snapshot if recipient is not in the
-	// config. This shouldn't ever happen (at the time of writing) but lots of
-	// code here and there assumes that r.id is in the progress tracker.
-	found := false
-	cs := s.Metadata.ConfState
-	for _, set := range [][]uint64{
-		cs.Voters,
-		cs.Learners,
-	} {
-		for _, id := range set {
-			if id == r.id {
-				found = true
-				break
-			}
-		}
-	}
-	if !found {
-		r.logger.Warningf(
-			"%x attempted to restore snapshot but it is not in the ConfState %v; should never happen",
-			r.id, cs,
-		)
-		return false
-	}
-
-	// Now go ahead and actually restore.
-
-	if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
-		r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
-			r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
-		r.raftLog.commitTo(s.Metadata.Index)
-		return false
-	}
-
-	r.raftLog.restore(s)
-
-	// Reset the configuration and add the (potentially updated) peers in anew.
-	r.prs = tracker.MakeProgressTracker(r.prs.MaxInflight)
-	cfg, prs, err := confchange.Restore(confchange.Changer{
-		Tracker:   r.prs,
-		LastIndex: r.raftLog.lastIndex(),
-	}, cs)
-
-	if err != nil {
-		// This should never happen. Either there's a bug in our config change
-		// handling or the client corrupted the conf change.
-		panic(fmt.Sprintf("unable to restore config %+v: %s", cs, err))
-	}
-
-	assertConfStatesEquivalent(r.logger, cs, r.switchToConfig(cfg, prs))
-
-	pr := r.prs.Progress[r.id]
-	pr.MaybeUpdate(pr.Next - 1) // TODO(tbg): this is untested and likely unneeded
-
-	r.logger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] restored snapshot [index: %d, term: %d]",
-		r.id, r.raftLog.committed, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
-	return true
-}
-
-// promotable indicates whether state machine can be promoted to leader,
-// which is true when its own id is in progress list.
-func (r *raft) promotable() bool {
-	pr := r.prs.Progress[r.id]
-	return pr != nil && !pr.IsLearner
-}
-
-func (r *raft) applyConfChange(cc pb.ConfChangeV2) pb.ConfState {
-	cfg, prs, err := func() (tracker.Config, tracker.ProgressMap, error) {
-		changer := confchange.Changer{
-			Tracker:   r.prs,
-			LastIndex: r.raftLog.lastIndex(),
-		}
-		if cc.LeaveJoint() {
-			return changer.LeaveJoint()
-		} else if autoLeave, ok := cc.EnterJoint(); ok {
-			return changer.EnterJoint(autoLeave, cc.Changes...)
-		}
-		return changer.Simple(cc.Changes...)
-	}()
-
-	if err != nil {
-		// TODO(tbg): return the error to the caller.
-		panic(err)
-	}
-
-	return r.switchToConfig(cfg, prs)
-}
-
-// switchToConfig reconfigures this node to use the provided configuration. It
-// updates the in-memory state and, when necessary, carries out additional
-// actions such as reacting to the removal of nodes or changed quorum
-// requirements.
-//
-// The inputs usually result from restoring a ConfState or applying a ConfChange.
-func (r *raft) switchToConfig(cfg tracker.Config, prs tracker.ProgressMap) pb.ConfState {
-	r.prs.Config = cfg
-	r.prs.Progress = prs
-
-	r.logger.Infof("%x switched to configuration %s", r.id, r.prs.Config)
-	cs := r.prs.ConfState()
-	pr, ok := r.prs.Progress[r.id]
-
-	// Update whether the node itself is a learner, resetting to false when the
-	// node is removed.
-	r.isLearner = ok && pr.IsLearner
-
-	if (!ok || r.isLearner) && r.state == StateLeader {
-		// This node is leader and was removed or demoted. We prevent demotions
-		// at the time writing but hypothetically we handle them the same way as
-		// removing the leader: stepping down into the next Term.
-		//
-		// TODO(tbg): step down (for sanity) and ask follower with largest Match
-		// to TimeoutNow (to avoid interruption). This might still drop some
-		// proposals but it's better than nothing.
-		//
-		// TODO(tbg): test this branch. It is untested at the time of writing.
-		return cs
-	}
-
-	// The remaining steps only make sense if this node is the leader and there
-	// are other nodes.
-	if r.state != StateLeader || len(cs.Voters) == 0 {
-		return cs
-	}
-
-	if r.maybeCommit() {
-		// If the configuration change means that more entries are committed now,
-		// broadcast/append to everyone in the updated config.
-		r.bcastAppend()
-	} else {
-		// Otherwise, still probe the newly added replicas; there's no reason to
-		// let them wait out a heartbeat interval (or the next incoming
-		// proposal).
-		r.prs.Visit(func(id uint64, pr *tracker.Progress) {
-			r.maybeSendAppend(id, false /* sendIfEmpty */)
-		})
-	}
-	// If the the leadTransferee was removed, abort the leadership transfer.
-	if _, tOK := r.prs.Progress[r.leadTransferee]; !tOK && r.leadTransferee != 0 {
-		r.abortLeaderTransfer()
-	}
-
-	return cs
-}
-
-func (r *raft) loadState(state pb.HardState) {
-	if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
-		r.logger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
-	}
-	r.raftLog.committed = state.Commit
-	r.Term = state.Term
-	r.Vote = state.Vote
-}
-
-// pastElectionTimeout returns true iff r.electionElapsed is greater
-// than or equal to the randomized election timeout in
-// [electiontimeout, 2 * electiontimeout - 1].
-func (r *raft) pastElectionTimeout() bool {
-	return r.electionElapsed >= r.randomizedElectionTimeout
-}
-
-func (r *raft) resetRandomizedElectionTimeout() {
-	r.randomizedElectionTimeout = r.electionTimeout + globalRand.Intn(r.electionTimeout)
-}
-
-func (r *raft) sendTimeoutNow(to uint64) {
-	r.send(pb.Message{To: to, Type: pb.MsgTimeoutNow})
-}
-
-func (r *raft) abortLeaderTransfer() {
-	r.leadTransferee = None
-}
-
-// increaseUncommittedSize computes the size of the proposed entries and
-// determines whether they would push leader over its maxUncommittedSize limit.
-// If the new entries would exceed the limit, the method returns false. If not,
-// the increase in uncommitted entry size is recorded and the method returns
-// true.
-func (r *raft) increaseUncommittedSize(ents []pb.Entry) bool {
-	var s uint64
-	for _, e := range ents {
-		s += uint64(PayloadSize(e))
-	}
-
-	if r.uncommittedSize > 0 && r.uncommittedSize+s > r.maxUncommittedSize {
-		// If the uncommitted tail of the Raft log is empty, allow any size
-		// proposal. Otherwise, limit the size of the uncommitted tail of the
-		// log and drop any proposal that would push the size over the limit.
-		return false
-	}
-	r.uncommittedSize += s
-	return true
-}
-
-// reduceUncommittedSize accounts for the newly committed entries by decreasing
-// the uncommitted entry size limit.
-func (r *raft) reduceUncommittedSize(ents []pb.Entry) {
-	if r.uncommittedSize == 0 {
-		// Fast-path for followers, who do not track or enforce the limit.
-		return
-	}
-
-	var s uint64
-	for _, e := range ents {
-		s += uint64(PayloadSize(e))
-	}
-	if s > r.uncommittedSize {
-		// uncommittedSize may underestimate the size of the uncommitted Raft
-		// log tail but will never overestimate it. Saturate at 0 instead of
-		// allowing overflow.
-		r.uncommittedSize = 0
-	} else {
-		r.uncommittedSize -= s
-	}
-}
-
-func numOfPendingConf(ents []pb.Entry) int {
-	n := 0
-	for i := range ents {
-		if ents[i].Type == pb.EntryConfChange {
-			n++
-		}
-	}
-	return n
-}
diff --git a/vendor/go.etcd.io/etcd/raft/raftpb/confchange.go b/vendor/go.etcd.io/etcd/raft/raftpb/confchange.go
deleted file mode 100644
index 46a7a70..0000000
--- a/vendor/go.etcd.io/etcd/raft/raftpb/confchange.go
+++ /dev/null
@@ -1,170 +0,0 @@
-// 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 raftpb
-
-import (
-	"fmt"
-	"strconv"
-	"strings"
-
-	"github.com/gogo/protobuf/proto"
-)
-
-// ConfChangeI abstracts over ConfChangeV2 and (legacy) ConfChange to allow
-// treating them in a unified manner.
-type ConfChangeI interface {
-	AsV2() ConfChangeV2
-	AsV1() (ConfChange, bool)
-}
-
-// MarshalConfChange calls Marshal on the underlying ConfChange or ConfChangeV2
-// and returns the result along with the corresponding EntryType.
-func MarshalConfChange(c ConfChangeI) (EntryType, []byte, error) {
-	var typ EntryType
-	var ccdata []byte
-	var err error
-	if ccv1, ok := c.AsV1(); ok {
-		typ = EntryConfChange
-		ccdata, err = ccv1.Marshal()
-	} else {
-		ccv2 := c.AsV2()
-		typ = EntryConfChangeV2
-		ccdata, err = ccv2.Marshal()
-	}
-	return typ, ccdata, err
-}
-
-// AsV2 returns a V2 configuration change carrying out the same operation.
-func (c ConfChange) AsV2() ConfChangeV2 {
-	return ConfChangeV2{
-		Changes: []ConfChangeSingle{{
-			Type:   c.Type,
-			NodeID: c.NodeID,
-		}},
-		Context: c.Context,
-	}
-}
-
-// AsV1 returns the ConfChange and true.
-func (c ConfChange) AsV1() (ConfChange, bool) {
-	return c, true
-}
-
-// AsV2 is the identity.
-func (c ConfChangeV2) AsV2() ConfChangeV2 { return c }
-
-// AsV1 returns ConfChange{} and false.
-func (c ConfChangeV2) AsV1() (ConfChange, bool) { return ConfChange{}, false }
-
-// EnterJoint returns two bools. The second bool is true if and only if this
-// config change will use Joint Consensus, which is the case if it contains more
-// than one change or if the use of Joint Consensus was requested explicitly.
-// The first bool can only be true if second one is, and indicates whether the
-// Joint State will be left automatically.
-func (c *ConfChangeV2) EnterJoint() (autoLeave bool, ok bool) {
-	// NB: in theory, more config changes could qualify for the "simple"
-	// protocol but it depends on the config on top of which the changes apply.
-	// For example, adding two learners is not OK if both nodes are part of the
-	// base config (i.e. two voters are turned into learners in the process of
-	// applying the conf change). In practice, these distinctions should not
-	// matter, so we keep it simple and use Joint Consensus liberally.
-	if c.Transition != ConfChangeTransitionAuto || len(c.Changes) > 1 {
-		// Use Joint Consensus.
-		var autoLeave bool
-		switch c.Transition {
-		case ConfChangeTransitionAuto:
-			autoLeave = true
-		case ConfChangeTransitionJointImplicit:
-			autoLeave = true
-		case ConfChangeTransitionJointExplicit:
-		default:
-			panic(fmt.Sprintf("unknown transition: %+v", c))
-		}
-		return autoLeave, true
-	}
-	return false, false
-}
-
-// LeaveJoint is true if the configuration change leaves a joint configuration.
-// This is the case if the ConfChangeV2 is zero, with the possible exception of
-// the Context field.
-func (c *ConfChangeV2) LeaveJoint() bool {
-	cpy := *c
-	cpy.Context = nil
-	return proto.Equal(&cpy, &ConfChangeV2{})
-}
-
-// ConfChangesFromString parses a Space-delimited sequence of operations into a
-// slice of ConfChangeSingle. The supported operations are:
-// - vn: make n a voter,
-// - ln: make n a learner,
-// - rn: remove n, and
-// - un: update n.
-func ConfChangesFromString(s string) ([]ConfChangeSingle, error) {
-	var ccs []ConfChangeSingle
-	toks := strings.Split(strings.TrimSpace(s), " ")
-	if toks[0] == "" {
-		toks = nil
-	}
-	for _, tok := range toks {
-		if len(tok) < 2 {
-			return nil, fmt.Errorf("unknown token %s", tok)
-		}
-		var cc ConfChangeSingle
-		switch tok[0] {
-		case 'v':
-			cc.Type = ConfChangeAddNode
-		case 'l':
-			cc.Type = ConfChangeAddLearnerNode
-		case 'r':
-			cc.Type = ConfChangeRemoveNode
-		case 'u':
-			cc.Type = ConfChangeUpdateNode
-		default:
-			return nil, fmt.Errorf("unknown input: %s", tok)
-		}
-		id, err := strconv.ParseUint(tok[1:], 10, 64)
-		if err != nil {
-			return nil, err
-		}
-		cc.NodeID = id
-		ccs = append(ccs, cc)
-	}
-	return ccs, nil
-}
-
-// ConfChangesToString is the inverse to ConfChangesFromString.
-func ConfChangesToString(ccs []ConfChangeSingle) string {
-	var buf strings.Builder
-	for i, cc := range ccs {
-		if i > 0 {
-			buf.WriteByte(' ')
-		}
-		switch cc.Type {
-		case ConfChangeAddNode:
-			buf.WriteByte('v')
-		case ConfChangeAddLearnerNode:
-			buf.WriteByte('l')
-		case ConfChangeRemoveNode:
-			buf.WriteByte('r')
-		case ConfChangeUpdateNode:
-			buf.WriteByte('u')
-		default:
-			buf.WriteString("unknown")
-		}
-		fmt.Fprintf(&buf, "%d", cc.NodeID)
-	}
-	return buf.String()
-}
diff --git a/vendor/go.etcd.io/etcd/raft/raftpb/confstate.go b/vendor/go.etcd.io/etcd/raft/raftpb/confstate.go
deleted file mode 100644
index 4bda932..0000000
--- a/vendor/go.etcd.io/etcd/raft/raftpb/confstate.go
+++ /dev/null
@@ -1,45 +0,0 @@
-// 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 raftpb
-
-import (
-	"fmt"
-	"reflect"
-	"sort"
-)
-
-// Equivalent returns a nil error if the inputs describe the same configuration.
-// On mismatch, returns a descriptive error showing the differences.
-func (cs ConfState) Equivalent(cs2 ConfState) error {
-	cs1 := cs
-	orig1, orig2 := cs1, cs2
-	s := func(sl *[]uint64) {
-		*sl = append([]uint64(nil), *sl...)
-		sort.Slice(*sl, func(i, j int) bool { return (*sl)[i] < (*sl)[j] })
-	}
-
-	for _, cs := range []*ConfState{&cs1, &cs2} {
-		s(&cs.Voters)
-		s(&cs.Learners)
-		s(&cs.VotersOutgoing)
-		s(&cs.LearnersNext)
-		cs.XXX_unrecognized = nil
-	}
-
-	if !reflect.DeepEqual(cs1, cs2) {
-		return fmt.Errorf("ConfStates not equivalent after sorting:\n%+#v\n%+#v\nInputs were:\n%+#v\n%+#v", cs1, cs2, orig1, orig2)
-	}
-	return nil
-}
diff --git a/vendor/go.etcd.io/etcd/raft/raftpb/raft.pb.go b/vendor/go.etcd.io/etcd/raft/raftpb/raft.pb.go
deleted file mode 100644
index fcf259c..0000000
--- a/vendor/go.etcd.io/etcd/raft/raftpb/raft.pb.go
+++ /dev/null
@@ -1,2646 +0,0 @@
-// Code generated by protoc-gen-gogo. DO NOT EDIT.
-// source: raft.proto
-
-/*
-	Package raftpb is a generated protocol buffer package.
-
-	It is generated from these files:
-		raft.proto
-
-	It has these top-level messages:
-		Entry
-		SnapshotMetadata
-		Snapshot
-		Message
-		HardState
-		ConfState
-		ConfChange
-		ConfChangeSingle
-		ConfChangeV2
-*/
-package raftpb
-
-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 EntryType int32
-
-const (
-	EntryNormal       EntryType = 0
-	EntryConfChange   EntryType = 1
-	EntryConfChangeV2 EntryType = 2
-)
-
-var EntryType_name = map[int32]string{
-	0: "EntryNormal",
-	1: "EntryConfChange",
-	2: "EntryConfChangeV2",
-}
-var EntryType_value = map[string]int32{
-	"EntryNormal":       0,
-	"EntryConfChange":   1,
-	"EntryConfChangeV2": 2,
-}
-
-func (x EntryType) Enum() *EntryType {
-	p := new(EntryType)
-	*p = x
-	return p
-}
-func (x EntryType) String() string {
-	return proto.EnumName(EntryType_name, int32(x))
-}
-func (x *EntryType) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(EntryType_value, data, "EntryType")
-	if err != nil {
-		return err
-	}
-	*x = EntryType(value)
-	return nil
-}
-func (EntryType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{0} }
-
-type MessageType int32
-
-const (
-	MsgHup            MessageType = 0
-	MsgBeat           MessageType = 1
-	MsgProp           MessageType = 2
-	MsgApp            MessageType = 3
-	MsgAppResp        MessageType = 4
-	MsgVote           MessageType = 5
-	MsgVoteResp       MessageType = 6
-	MsgSnap           MessageType = 7
-	MsgHeartbeat      MessageType = 8
-	MsgHeartbeatResp  MessageType = 9
-	MsgUnreachable    MessageType = 10
-	MsgSnapStatus     MessageType = 11
-	MsgCheckQuorum    MessageType = 12
-	MsgTransferLeader MessageType = 13
-	MsgTimeoutNow     MessageType = 14
-	MsgReadIndex      MessageType = 15
-	MsgReadIndexResp  MessageType = 16
-	MsgPreVote        MessageType = 17
-	MsgPreVoteResp    MessageType = 18
-)
-
-var MessageType_name = map[int32]string{
-	0:  "MsgHup",
-	1:  "MsgBeat",
-	2:  "MsgProp",
-	3:  "MsgApp",
-	4:  "MsgAppResp",
-	5:  "MsgVote",
-	6:  "MsgVoteResp",
-	7:  "MsgSnap",
-	8:  "MsgHeartbeat",
-	9:  "MsgHeartbeatResp",
-	10: "MsgUnreachable",
-	11: "MsgSnapStatus",
-	12: "MsgCheckQuorum",
-	13: "MsgTransferLeader",
-	14: "MsgTimeoutNow",
-	15: "MsgReadIndex",
-	16: "MsgReadIndexResp",
-	17: "MsgPreVote",
-	18: "MsgPreVoteResp",
-}
-var MessageType_value = map[string]int32{
-	"MsgHup":            0,
-	"MsgBeat":           1,
-	"MsgProp":           2,
-	"MsgApp":            3,
-	"MsgAppResp":        4,
-	"MsgVote":           5,
-	"MsgVoteResp":       6,
-	"MsgSnap":           7,
-	"MsgHeartbeat":      8,
-	"MsgHeartbeatResp":  9,
-	"MsgUnreachable":    10,
-	"MsgSnapStatus":     11,
-	"MsgCheckQuorum":    12,
-	"MsgTransferLeader": 13,
-	"MsgTimeoutNow":     14,
-	"MsgReadIndex":      15,
-	"MsgReadIndexResp":  16,
-	"MsgPreVote":        17,
-	"MsgPreVoteResp":    18,
-}
-
-func (x MessageType) Enum() *MessageType {
-	p := new(MessageType)
-	*p = x
-	return p
-}
-func (x MessageType) String() string {
-	return proto.EnumName(MessageType_name, int32(x))
-}
-func (x *MessageType) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(MessageType_value, data, "MessageType")
-	if err != nil {
-		return err
-	}
-	*x = MessageType(value)
-	return nil
-}
-func (MessageType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{1} }
-
-// ConfChangeTransition specifies the behavior of a configuration change with
-// respect to joint consensus.
-type ConfChangeTransition int32
-
-const (
-	// Automatically use the simple protocol if possible, otherwise fall back
-	// to ConfChangeJointImplicit. Most applications will want to use this.
-	ConfChangeTransitionAuto ConfChangeTransition = 0
-	// Use joint consensus unconditionally, and transition out of them
-	// automatically (by proposing a zero configuration change).
-	//
-	// This option is suitable for applications that want to minimize the time
-	// spent in the joint configuration and do not store the joint configuration
-	// in the state machine (outside of InitialState).
-	ConfChangeTransitionJointImplicit ConfChangeTransition = 1
-	// Use joint consensus and remain in the joint configuration until the
-	// application proposes a no-op configuration change. This is suitable for
-	// applications that want to explicitly control the transitions, for example
-	// to use a custom payload (via the Context field).
-	ConfChangeTransitionJointExplicit ConfChangeTransition = 2
-)
-
-var ConfChangeTransition_name = map[int32]string{
-	0: "ConfChangeTransitionAuto",
-	1: "ConfChangeTransitionJointImplicit",
-	2: "ConfChangeTransitionJointExplicit",
-}
-var ConfChangeTransition_value = map[string]int32{
-	"ConfChangeTransitionAuto":          0,
-	"ConfChangeTransitionJointImplicit": 1,
-	"ConfChangeTransitionJointExplicit": 2,
-}
-
-func (x ConfChangeTransition) Enum() *ConfChangeTransition {
-	p := new(ConfChangeTransition)
-	*p = x
-	return p
-}
-func (x ConfChangeTransition) String() string {
-	return proto.EnumName(ConfChangeTransition_name, int32(x))
-}
-func (x *ConfChangeTransition) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(ConfChangeTransition_value, data, "ConfChangeTransition")
-	if err != nil {
-		return err
-	}
-	*x = ConfChangeTransition(value)
-	return nil
-}
-func (ConfChangeTransition) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} }
-
-type ConfChangeType int32
-
-const (
-	ConfChangeAddNode        ConfChangeType = 0
-	ConfChangeRemoveNode     ConfChangeType = 1
-	ConfChangeUpdateNode     ConfChangeType = 2
-	ConfChangeAddLearnerNode ConfChangeType = 3
-)
-
-var ConfChangeType_name = map[int32]string{
-	0: "ConfChangeAddNode",
-	1: "ConfChangeRemoveNode",
-	2: "ConfChangeUpdateNode",
-	3: "ConfChangeAddLearnerNode",
-}
-var ConfChangeType_value = map[string]int32{
-	"ConfChangeAddNode":        0,
-	"ConfChangeRemoveNode":     1,
-	"ConfChangeUpdateNode":     2,
-	"ConfChangeAddLearnerNode": 3,
-}
-
-func (x ConfChangeType) Enum() *ConfChangeType {
-	p := new(ConfChangeType)
-	*p = x
-	return p
-}
-func (x ConfChangeType) String() string {
-	return proto.EnumName(ConfChangeType_name, int32(x))
-}
-func (x *ConfChangeType) UnmarshalJSON(data []byte) error {
-	value, err := proto.UnmarshalJSONEnum(ConfChangeType_value, data, "ConfChangeType")
-	if err != nil {
-		return err
-	}
-	*x = ConfChangeType(value)
-	return nil
-}
-func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{3} }
-
-type Entry struct {
-	Term             uint64    `protobuf:"varint,2,opt,name=Term" json:"Term"`
-	Index            uint64    `protobuf:"varint,3,opt,name=Index" json:"Index"`
-	Type             EntryType `protobuf:"varint,1,opt,name=Type,enum=raftpb.EntryType" json:"Type"`
-	Data             []byte    `protobuf:"bytes,4,opt,name=Data" json:"Data,omitempty"`
-	XXX_unrecognized []byte    `json:"-"`
-}
-
-func (m *Entry) Reset()                    { *m = Entry{} }
-func (m *Entry) String() string            { return proto.CompactTextString(m) }
-func (*Entry) ProtoMessage()               {}
-func (*Entry) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{0} }
-
-type SnapshotMetadata struct {
-	ConfState        ConfState `protobuf:"bytes,1,opt,name=conf_state,json=confState" json:"conf_state"`
-	Index            uint64    `protobuf:"varint,2,opt,name=index" json:"index"`
-	Term             uint64    `protobuf:"varint,3,opt,name=term" json:"term"`
-	XXX_unrecognized []byte    `json:"-"`
-}
-
-func (m *SnapshotMetadata) Reset()                    { *m = SnapshotMetadata{} }
-func (m *SnapshotMetadata) String() string            { return proto.CompactTextString(m) }
-func (*SnapshotMetadata) ProtoMessage()               {}
-func (*SnapshotMetadata) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{1} }
-
-type Snapshot struct {
-	Data             []byte           `protobuf:"bytes,1,opt,name=data" json:"data,omitempty"`
-	Metadata         SnapshotMetadata `protobuf:"bytes,2,opt,name=metadata" json:"metadata"`
-	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 fileDescriptorRaft, []int{2} }
-
-type Message struct {
-	Type             MessageType `protobuf:"varint,1,opt,name=type,enum=raftpb.MessageType" json:"type"`
-	To               uint64      `protobuf:"varint,2,opt,name=to" json:"to"`
-	From             uint64      `protobuf:"varint,3,opt,name=from" json:"from"`
-	Term             uint64      `protobuf:"varint,4,opt,name=term" json:"term"`
-	LogTerm          uint64      `protobuf:"varint,5,opt,name=logTerm" json:"logTerm"`
-	Index            uint64      `protobuf:"varint,6,opt,name=index" json:"index"`
-	Entries          []Entry     `protobuf:"bytes,7,rep,name=entries" json:"entries"`
-	Commit           uint64      `protobuf:"varint,8,opt,name=commit" json:"commit"`
-	Snapshot         Snapshot    `protobuf:"bytes,9,opt,name=snapshot" json:"snapshot"`
-	Reject           bool        `protobuf:"varint,10,opt,name=reject" json:"reject"`
-	RejectHint       uint64      `protobuf:"varint,11,opt,name=rejectHint" json:"rejectHint"`
-	Context          []byte      `protobuf:"bytes,12,opt,name=context" json:"context,omitempty"`
-	XXX_unrecognized []byte      `json:"-"`
-}
-
-func (m *Message) Reset()                    { *m = Message{} }
-func (m *Message) String() string            { return proto.CompactTextString(m) }
-func (*Message) ProtoMessage()               {}
-func (*Message) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{3} }
-
-type HardState struct {
-	Term             uint64 `protobuf:"varint,1,opt,name=term" json:"term"`
-	Vote             uint64 `protobuf:"varint,2,opt,name=vote" json:"vote"`
-	Commit           uint64 `protobuf:"varint,3,opt,name=commit" json:"commit"`
-	XXX_unrecognized []byte `json:"-"`
-}
-
-func (m *HardState) Reset()                    { *m = HardState{} }
-func (m *HardState) String() string            { return proto.CompactTextString(m) }
-func (*HardState) ProtoMessage()               {}
-func (*HardState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{4} }
-
-type ConfState struct {
-	// The voters in the incoming config. (If the configuration is not joint,
-	// then the outgoing config is empty).
-	Voters []uint64 `protobuf:"varint,1,rep,name=voters" json:"voters,omitempty"`
-	// The learners in the incoming config.
-	Learners []uint64 `protobuf:"varint,2,rep,name=learners" json:"learners,omitempty"`
-	// The voters in the outgoing config.
-	VotersOutgoing []uint64 `protobuf:"varint,3,rep,name=voters_outgoing,json=votersOutgoing" json:"voters_outgoing,omitempty"`
-	// The nodes that will become learners when the outgoing config is removed.
-	// These nodes are necessarily currently in nodes_joint (or they would have
-	// been added to the incoming config right away).
-	LearnersNext []uint64 `protobuf:"varint,4,rep,name=learners_next,json=learnersNext" json:"learners_next,omitempty"`
-	// If set, the config is joint and Raft will automatically transition into
-	// the final config (i.e. remove the outgoing config) when this is safe.
-	AutoLeave        bool   `protobuf:"varint,5,opt,name=auto_leave,json=autoLeave" json:"auto_leave"`
-	XXX_unrecognized []byte `json:"-"`
-}
-
-func (m *ConfState) Reset()                    { *m = ConfState{} }
-func (m *ConfState) String() string            { return proto.CompactTextString(m) }
-func (*ConfState) ProtoMessage()               {}
-func (*ConfState) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} }
-
-type ConfChange struct {
-	Type    ConfChangeType `protobuf:"varint,2,opt,name=type,enum=raftpb.ConfChangeType" json:"type"`
-	NodeID  uint64         `protobuf:"varint,3,opt,name=node_id,json=nodeId" json:"node_id"`
-	Context []byte         `protobuf:"bytes,4,opt,name=context" json:"context,omitempty"`
-	// NB: this is used only by etcd to thread through a unique identifier.
-	// Ideally it should really use the Context instead. No counterpart to
-	// this field exists in ConfChangeV2.
-	ID               uint64 `protobuf:"varint,1,opt,name=id" json:"id"`
-	XXX_unrecognized []byte `json:"-"`
-}
-
-func (m *ConfChange) Reset()                    { *m = ConfChange{} }
-func (m *ConfChange) String() string            { return proto.CompactTextString(m) }
-func (*ConfChange) ProtoMessage()               {}
-func (*ConfChange) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} }
-
-// ConfChangeSingle is an individual configuration change operation. Multiple
-// such operations can be carried out atomically via a ConfChangeV2.
-type ConfChangeSingle struct {
-	Type             ConfChangeType `protobuf:"varint,1,opt,name=type,enum=raftpb.ConfChangeType" json:"type"`
-	NodeID           uint64         `protobuf:"varint,2,opt,name=node_id,json=nodeId" json:"node_id"`
-	XXX_unrecognized []byte         `json:"-"`
-}
-
-func (m *ConfChangeSingle) Reset()                    { *m = ConfChangeSingle{} }
-func (m *ConfChangeSingle) String() string            { return proto.CompactTextString(m) }
-func (*ConfChangeSingle) ProtoMessage()               {}
-func (*ConfChangeSingle) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{7} }
-
-// ConfChangeV2 messages initiate configuration changes. They support both the
-// simple "one at a time" membership change protocol and full Joint Consensus
-// allowing for arbitrary changes in membership.
-//
-// The supplied context is treated as an opaque payload and can be used to
-// attach an action on the state machine to the application of the config change
-// proposal. Note that contrary to Joint Consensus as outlined in the Raft
-// paper[1], configuration changes become active when they are *applied* to the
-// state machine (not when they are appended to the log).
-//
-// The simple protocol can be used whenever only a single change is made.
-//
-// Non-simple changes require the use of Joint Consensus, for which two
-// configuration changes are run. The first configuration change specifies the
-// desired changes and transitions the Raft group into the joint configuration,
-// in which quorum requires a majority of both the pre-changes and post-changes
-// configuration. Joint Consensus avoids entering fragile intermediate
-// configurations that could compromise survivability. For example, without the
-// use of Joint Consensus and running across three availability zones with a
-// replication factor of three, it is not possible to replace a voter without
-// entering an intermediate configuration that does not survive the outage of
-// one availability zone.
-//
-// The provided ConfChangeTransition specifies how (and whether) Joint Consensus
-// is used, and assigns the task of leaving the joint configuration either to
-// Raft or the application. Leaving the joint configuration is accomplished by
-// proposing a ConfChangeV2 with only and optionally the Context field
-// populated.
-//
-// For details on Raft membership changes, see:
-//
-// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
-type ConfChangeV2 struct {
-	Transition       ConfChangeTransition `protobuf:"varint,1,opt,name=transition,enum=raftpb.ConfChangeTransition" json:"transition"`
-	Changes          []ConfChangeSingle   `protobuf:"bytes,2,rep,name=changes" json:"changes"`
-	Context          []byte               `protobuf:"bytes,3,opt,name=context" json:"context,omitempty"`
-	XXX_unrecognized []byte               `json:"-"`
-}
-
-func (m *ConfChangeV2) Reset()                    { *m = ConfChangeV2{} }
-func (m *ConfChangeV2) String() string            { return proto.CompactTextString(m) }
-func (*ConfChangeV2) ProtoMessage()               {}
-func (*ConfChangeV2) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{8} }
-
-func init() {
-	proto.RegisterType((*Entry)(nil), "raftpb.Entry")
-	proto.RegisterType((*SnapshotMetadata)(nil), "raftpb.SnapshotMetadata")
-	proto.RegisterType((*Snapshot)(nil), "raftpb.Snapshot")
-	proto.RegisterType((*Message)(nil), "raftpb.Message")
-	proto.RegisterType((*HardState)(nil), "raftpb.HardState")
-	proto.RegisterType((*ConfState)(nil), "raftpb.ConfState")
-	proto.RegisterType((*ConfChange)(nil), "raftpb.ConfChange")
-	proto.RegisterType((*ConfChangeSingle)(nil), "raftpb.ConfChangeSingle")
-	proto.RegisterType((*ConfChangeV2)(nil), "raftpb.ConfChangeV2")
-	proto.RegisterEnum("raftpb.EntryType", EntryType_name, EntryType_value)
-	proto.RegisterEnum("raftpb.MessageType", MessageType_name, MessageType_value)
-	proto.RegisterEnum("raftpb.ConfChangeTransition", ConfChangeTransition_name, ConfChangeTransition_value)
-	proto.RegisterEnum("raftpb.ConfChangeType", ConfChangeType_name, ConfChangeType_value)
-}
-func (m *Entry) 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 *Entry) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
-	dAtA[i] = 0x18
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
-	if m.Data != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRaft(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 *SnapshotMetadata) 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 *SnapshotMetadata) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0xa
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.ConfState.Size()))
-	n1, err := m.ConfState.MarshalTo(dAtA[i:])
-	if err != nil {
-		return 0, err
-	}
-	i += n1
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
-	dAtA[i] = 0x18
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
-	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
-	if m.Data != nil {
-		dAtA[i] = 0xa
-		i++
-		i = encodeVarintRaft(dAtA, i, uint64(len(m.Data)))
-		i += copy(dAtA[i:], m.Data)
-	}
-	dAtA[i] = 0x12
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Metadata.Size()))
-	n2, err := m.Metadata.MarshalTo(dAtA[i:])
-	if err != nil {
-		return 0, err
-	}
-	i += n2
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *Message) 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 *Message) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.To))
-	dAtA[i] = 0x18
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.From))
-	dAtA[i] = 0x20
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
-	dAtA[i] = 0x28
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.LogTerm))
-	dAtA[i] = 0x30
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Index))
-	if len(m.Entries) > 0 {
-		for _, msg := range m.Entries {
-			dAtA[i] = 0x3a
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	dAtA[i] = 0x40
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Commit))
-	dAtA[i] = 0x4a
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Snapshot.Size()))
-	n3, err := m.Snapshot.MarshalTo(dAtA[i:])
-	if err != nil {
-		return 0, err
-	}
-	i += n3
-	dAtA[i] = 0x50
-	i++
-	if m.Reject {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	dAtA[i] = 0x58
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.RejectHint))
-	if m.Context != nil {
-		dAtA[i] = 0x62
-		i++
-		i = encodeVarintRaft(dAtA, i, uint64(len(m.Context)))
-		i += copy(dAtA[i:], m.Context)
-	}
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *HardState) 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 *HardState) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Term))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Vote))
-	dAtA[i] = 0x18
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Commit))
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *ConfState) 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 *ConfState) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	if len(m.Voters) > 0 {
-		for _, num := range m.Voters {
-			dAtA[i] = 0x8
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(num))
-		}
-	}
-	if len(m.Learners) > 0 {
-		for _, num := range m.Learners {
-			dAtA[i] = 0x10
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(num))
-		}
-	}
-	if len(m.VotersOutgoing) > 0 {
-		for _, num := range m.VotersOutgoing {
-			dAtA[i] = 0x18
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(num))
-		}
-	}
-	if len(m.LearnersNext) > 0 {
-		for _, num := range m.LearnersNext {
-			dAtA[i] = 0x20
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(num))
-		}
-	}
-	dAtA[i] = 0x28
-	i++
-	if m.AutoLeave {
-		dAtA[i] = 1
-	} else {
-		dAtA[i] = 0
-	}
-	i++
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *ConfChange) 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 *ConfChange) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.ID))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
-	dAtA[i] = 0x18
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.NodeID))
-	if m.Context != nil {
-		dAtA[i] = 0x22
-		i++
-		i = encodeVarintRaft(dAtA, i, uint64(len(m.Context)))
-		i += copy(dAtA[i:], m.Context)
-	}
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *ConfChangeSingle) 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 *ConfChangeSingle) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Type))
-	dAtA[i] = 0x10
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.NodeID))
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func (m *ConfChangeV2) 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 *ConfChangeV2) MarshalTo(dAtA []byte) (int, error) {
-	var i int
-	_ = i
-	var l int
-	_ = l
-	dAtA[i] = 0x8
-	i++
-	i = encodeVarintRaft(dAtA, i, uint64(m.Transition))
-	if len(m.Changes) > 0 {
-		for _, msg := range m.Changes {
-			dAtA[i] = 0x12
-			i++
-			i = encodeVarintRaft(dAtA, i, uint64(msg.Size()))
-			n, err := msg.MarshalTo(dAtA[i:])
-			if err != nil {
-				return 0, err
-			}
-			i += n
-		}
-	}
-	if m.Context != nil {
-		dAtA[i] = 0x1a
-		i++
-		i = encodeVarintRaft(dAtA, i, uint64(len(m.Context)))
-		i += copy(dAtA[i:], m.Context)
-	}
-	if m.XXX_unrecognized != nil {
-		i += copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	return i, nil
-}
-
-func encodeVarintRaft(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 *Entry) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.Type))
-	n += 1 + sovRaft(uint64(m.Term))
-	n += 1 + sovRaft(uint64(m.Index))
-	if m.Data != nil {
-		l = len(m.Data)
-		n += 1 + l + sovRaft(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *SnapshotMetadata) Size() (n int) {
-	var l int
-	_ = l
-	l = m.ConfState.Size()
-	n += 1 + l + sovRaft(uint64(l))
-	n += 1 + sovRaft(uint64(m.Index))
-	n += 1 + sovRaft(uint64(m.Term))
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *Snapshot) Size() (n int) {
-	var l int
-	_ = l
-	if m.Data != nil {
-		l = len(m.Data)
-		n += 1 + l + sovRaft(uint64(l))
-	}
-	l = m.Metadata.Size()
-	n += 1 + l + sovRaft(uint64(l))
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *Message) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.Type))
-	n += 1 + sovRaft(uint64(m.To))
-	n += 1 + sovRaft(uint64(m.From))
-	n += 1 + sovRaft(uint64(m.Term))
-	n += 1 + sovRaft(uint64(m.LogTerm))
-	n += 1 + sovRaft(uint64(m.Index))
-	if len(m.Entries) > 0 {
-		for _, e := range m.Entries {
-			l = e.Size()
-			n += 1 + l + sovRaft(uint64(l))
-		}
-	}
-	n += 1 + sovRaft(uint64(m.Commit))
-	l = m.Snapshot.Size()
-	n += 1 + l + sovRaft(uint64(l))
-	n += 2
-	n += 1 + sovRaft(uint64(m.RejectHint))
-	if m.Context != nil {
-		l = len(m.Context)
-		n += 1 + l + sovRaft(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *HardState) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.Term))
-	n += 1 + sovRaft(uint64(m.Vote))
-	n += 1 + sovRaft(uint64(m.Commit))
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *ConfState) Size() (n int) {
-	var l int
-	_ = l
-	if len(m.Voters) > 0 {
-		for _, e := range m.Voters {
-			n += 1 + sovRaft(uint64(e))
-		}
-	}
-	if len(m.Learners) > 0 {
-		for _, e := range m.Learners {
-			n += 1 + sovRaft(uint64(e))
-		}
-	}
-	if len(m.VotersOutgoing) > 0 {
-		for _, e := range m.VotersOutgoing {
-			n += 1 + sovRaft(uint64(e))
-		}
-	}
-	if len(m.LearnersNext) > 0 {
-		for _, e := range m.LearnersNext {
-			n += 1 + sovRaft(uint64(e))
-		}
-	}
-	n += 2
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *ConfChange) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.ID))
-	n += 1 + sovRaft(uint64(m.Type))
-	n += 1 + sovRaft(uint64(m.NodeID))
-	if m.Context != nil {
-		l = len(m.Context)
-		n += 1 + l + sovRaft(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *ConfChangeSingle) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.Type))
-	n += 1 + sovRaft(uint64(m.NodeID))
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func (m *ConfChangeV2) Size() (n int) {
-	var l int
-	_ = l
-	n += 1 + sovRaft(uint64(m.Transition))
-	if len(m.Changes) > 0 {
-		for _, e := range m.Changes {
-			l = e.Size()
-			n += 1 + l + sovRaft(uint64(l))
-		}
-	}
-	if m.Context != nil {
-		l = len(m.Context)
-		n += 1 + l + sovRaft(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
-
-func sovRaft(x uint64) (n int) {
-	for {
-		n++
-		x >>= 7
-		if x == 0 {
-			break
-		}
-	}
-	return n
-}
-func sozRaft(x uint64) (n int) {
-	return sovRaft(uint64((x << 1) ^ uint64((int64(x) >> 63))))
-}
-func (m *Entry) 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 ErrIntOverflowRaft
-			}
-			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: Entry: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Entry: 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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Type |= (EntryType(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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Term |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Index |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *SnapshotMetadata) 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 ErrIntOverflowRaft
-			}
-			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: SnapshotMetadata: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: SnapshotMetadata: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field ConfState", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if err := m.ConfState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 2:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Index |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			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 ErrIntOverflowRaft
-				}
-				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 := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 ErrIntOverflowRaft
-			}
-			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 != 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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if err := m.Metadata.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *Message) 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 ErrIntOverflowRaft
-			}
-			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: Message: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Message: 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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Type |= (MessageType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field To", wireType)
-			}
-			m.To = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.To |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field From", wireType)
-			}
-			m.From = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.From |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Term |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field LogTerm", wireType)
-			}
-			m.LogTerm = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.LogTerm |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 6:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Index |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 7:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Entries", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Entries = append(m.Entries, Entry{})
-			if err := m.Entries[len(m.Entries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 8:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Commit", wireType)
-			}
-			m.Commit = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Commit |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 9:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			if err := m.Snapshot.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 10:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Reject", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.Reject = bool(v != 0)
-		case 11:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RejectHint", wireType)
-			}
-			m.RejectHint = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.RejectHint |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 12:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...)
-			if m.Context == nil {
-				m.Context = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *HardState) 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 ErrIntOverflowRaft
-			}
-			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: HardState: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: HardState: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Term |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Vote", wireType)
-			}
-			m.Vote = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Vote |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Commit", wireType)
-			}
-			m.Commit = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Commit |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *ConfState) 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 ErrIntOverflowRaft
-			}
-			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: ConfState: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ConfState: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType == 0 {
-				var v uint64
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= (uint64(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.Voters = append(m.Voters, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= (int(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthRaft
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				for iNdEx < postIndex {
-					var v uint64
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowRaft
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= (uint64(b) & 0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.Voters = append(m.Voters, v)
-				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Voters", wireType)
-			}
-		case 2:
-			if wireType == 0 {
-				var v uint64
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= (uint64(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.Learners = append(m.Learners, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= (int(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthRaft
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				for iNdEx < postIndex {
-					var v uint64
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowRaft
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= (uint64(b) & 0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.Learners = append(m.Learners, v)
-				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Learners", wireType)
-			}
-		case 3:
-			if wireType == 0 {
-				var v uint64
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= (uint64(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.VotersOutgoing = append(m.VotersOutgoing, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= (int(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthRaft
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				for iNdEx < postIndex {
-					var v uint64
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowRaft
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= (uint64(b) & 0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.VotersOutgoing = append(m.VotersOutgoing, v)
-				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field VotersOutgoing", wireType)
-			}
-		case 4:
-			if wireType == 0 {
-				var v uint64
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= (uint64(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.LearnersNext = append(m.LearnersNext, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowRaft
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= (int(b) & 0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthRaft
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				for iNdEx < postIndex {
-					var v uint64
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowRaft
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= (uint64(b) & 0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.LearnersNext = append(m.LearnersNext, v)
-				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field LearnersNext", wireType)
-			}
-		case 5:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field AutoLeave", wireType)
-			}
-			var v int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				v |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			m.AutoLeave = bool(v != 0)
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *ConfChange) 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 ErrIntOverflowRaft
-			}
-			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: ConfChange: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ConfChange: 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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.ID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Type |= (ConfChangeType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
-			}
-			m.NodeID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.NodeID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 4:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...)
-			if m.Context == nil {
-				m.Context = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *ConfChangeSingle) 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 ErrIntOverflowRaft
-			}
-			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: ConfChangeSingle: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ConfChangeSingle: 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 ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Type |= (ConfChangeType(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
-			}
-			m.NodeID = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.NodeID |= (uint64(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 *ConfChangeV2) 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 ErrIntOverflowRaft
-			}
-			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: ConfChangeV2: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: ConfChangeV2: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Transition", wireType)
-			}
-			m.Transition = 0
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Transition |= (ConfChangeTransition(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 2:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Changes", wireType)
-			}
-			var msglen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				msglen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if msglen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + msglen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Changes = append(m.Changes, ConfChangeSingle{})
-			if err := m.Changes[len(m.Changes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
-				return err
-			}
-			iNdEx = postIndex
-		case 3:
-			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
-			}
-			var byteLen int
-			for shift := uint(0); ; shift += 7 {
-				if shift >= 64 {
-					return ErrIntOverflowRaft
-				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				byteLen |= (int(b) & 0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-			if byteLen < 0 {
-				return ErrInvalidLengthRaft
-			}
-			postIndex := iNdEx + byteLen
-			if postIndex > l {
-				return io.ErrUnexpectedEOF
-			}
-			m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...)
-			if m.Context == nil {
-				m.Context = []byte{}
-			}
-			iNdEx = postIndex
-		default:
-			iNdEx = preIndex
-			skippy, err := skipRaft(dAtA[iNdEx:])
-			if err != nil {
-				return err
-			}
-			if skippy < 0 {
-				return ErrInvalidLengthRaft
-			}
-			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 skipRaft(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, ErrIntOverflowRaft
-			}
-			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, ErrIntOverflowRaft
-				}
-				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, ErrIntOverflowRaft
-				}
-				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, ErrInvalidLengthRaft
-			}
-			return iNdEx, nil
-		case 3:
-			for {
-				var innerWire uint64
-				var start int = iNdEx
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return 0, ErrIntOverflowRaft
-					}
-					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 := skipRaft(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 (
-	ErrInvalidLengthRaft = fmt.Errorf("proto: negative length found during unmarshaling")
-	ErrIntOverflowRaft   = fmt.Errorf("proto: integer overflow")
-)
-
-func init() { proto.RegisterFile("raft.proto", fileDescriptorRaft) }
-
-var fileDescriptorRaft = []byte{
-	// 1009 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xcd, 0x6e, 0xe3, 0x36,
-	0x17, 0xb5, 0x64, 0xc5, 0x3f, 0xd7, 0x8e, 0xc3, 0xdc, 0xc9, 0x37, 0x20, 0x82, 0xc0, 0xe3, 0xcf,
-	0xd3, 0x62, 0x8c, 0x14, 0x93, 0x16, 0x5e, 0x14, 0x45, 0x77, 0xf9, 0x19, 0x20, 0x29, 0xe2, 0x74,
-	0xea, 0x64, 0xb2, 0x28, 0x50, 0x04, 0x8c, 0x45, 0x2b, 0x6a, 0x2d, 0x51, 0xa0, 0xe8, 0x34, 0xd9,
-	0x14, 0x45, 0x9f, 0xa2, 0x9b, 0xd9, 0xf6, 0x01, 0xfa, 0x14, 0x59, 0x0e, 0xd0, 0xfd, 0xa0, 0x93,
-	0xbe, 0x48, 0x41, 0x8a, 0xb2, 0x65, 0x27, 0x98, 0x45, 0x77, 0xe4, 0x39, 0x87, 0xf7, 0x9e, 0x7b,
-	0x79, 0x45, 0x01, 0x48, 0x36, 0x56, 0x3b, 0x89, 0x14, 0x4a, 0x60, 0x45, 0xaf, 0x93, 0xcb, 0xcd,
-	0x8d, 0x40, 0x04, 0xc2, 0x40, 0x9f, 0xeb, 0x55, 0xc6, 0x76, 0x7f, 0x81, 0x95, 0x57, 0xb1, 0x92,
-	0xb7, 0xf8, 0x19, 0x78, 0x67, 0xb7, 0x09, 0xa7, 0x4e, 0xc7, 0xe9, 0xb5, 0xfa, 0xeb, 0x3b, 0xd9,
-	0xa9, 0x1d, 0x43, 0x6a, 0x62, 0xcf, 0xbb, 0x7b, 0xff, 0xac, 0x34, 0x34, 0x22, 0xa4, 0xe0, 0x9d,
-	0x71, 0x19, 0x51, 0xb7, 0xe3, 0xf4, 0xbc, 0x19, 0xc3, 0x65, 0x84, 0x9b, 0xb0, 0x72, 0x14, 0xfb,
-	0xfc, 0x86, 0x96, 0x0b, 0x54, 0x06, 0x21, 0x82, 0x77, 0xc0, 0x14, 0xa3, 0x5e, 0xc7, 0xe9, 0x35,
-	0x87, 0x66, 0xdd, 0xfd, 0xd5, 0x01, 0x72, 0x1a, 0xb3, 0x24, 0xbd, 0x12, 0x6a, 0xc0, 0x15, 0xf3,
-	0x99, 0x62, 0xf8, 0x25, 0xc0, 0x48, 0xc4, 0xe3, 0x8b, 0x54, 0x31, 0x95, 0x39, 0x6a, 0xcc, 0x1d,
-	0xed, 0x8b, 0x78, 0x7c, 0xaa, 0x09, 0x1b, 0xbc, 0x3e, 0xca, 0x01, 0x9d, 0x3c, 0x34, 0xc9, 0x8b,
-	0xbe, 0x32, 0x48, 0x5b, 0x56, 0xda, 0x72, 0xd1, 0x97, 0x41, 0xba, 0xdf, 0x43, 0x2d, 0x77, 0xa0,
-	0x2d, 0x6a, 0x07, 0x26, 0x67, 0x73, 0x68, 0xd6, 0xf8, 0x35, 0xd4, 0x22, 0xeb, 0xcc, 0x04, 0x6e,
-	0xf4, 0x69, 0xee, 0x65, 0xd9, 0xb9, 0x8d, 0x3b, 0xd3, 0x77, 0xdf, 0x96, 0xa1, 0x3a, 0xe0, 0x69,
-	0xca, 0x02, 0x8e, 0x2f, 0xc1, 0x53, 0xf3, 0x0e, 0x3f, 0xc9, 0x63, 0x58, 0xba, 0xd8, 0x63, 0x2d,
-	0xc3, 0x0d, 0x70, 0x95, 0x58, 0xa8, 0xc4, 0x55, 0x42, 0x97, 0x31, 0x96, 0x62, 0xa9, 0x0c, 0x8d,
-	0xcc, 0x0a, 0xf4, 0x96, 0x0b, 0xc4, 0x36, 0x54, 0x27, 0x22, 0x30, 0x17, 0xb6, 0x52, 0x20, 0x73,
-	0x70, 0xde, 0xb6, 0xca, 0xc3, 0xb6, 0xbd, 0x84, 0x2a, 0x8f, 0x95, 0x0c, 0x79, 0x4a, 0xab, 0x9d,
-	0x72, 0xaf, 0xd1, 0x5f, 0x5d, 0x98, 0x8c, 0x3c, 0x94, 0xd5, 0xe0, 0x16, 0x54, 0x46, 0x22, 0x8a,
-	0x42, 0x45, 0x6b, 0x85, 0x58, 0x16, 0xc3, 0x3e, 0xd4, 0x52, 0xdb, 0x31, 0x5a, 0x37, 0x9d, 0x24,
-	0xcb, 0x9d, 0xcc, 0x3b, 0x98, 0xeb, 0x74, 0x44, 0xc9, 0x7f, 0xe4, 0x23, 0x45, 0xa1, 0xe3, 0xf4,
-	0x6a, 0x79, 0xc4, 0x0c, 0xc3, 0x4f, 0x00, 0xb2, 0xd5, 0x61, 0x18, 0x2b, 0xda, 0x28, 0xe4, 0x2c,
-	0xe0, 0x48, 0xa1, 0x3a, 0x12, 0xb1, 0xe2, 0x37, 0x8a, 0x36, 0xcd, 0xc5, 0xe6, 0xdb, 0xee, 0x0f,
-	0x50, 0x3f, 0x64, 0xd2, 0xcf, 0xc6, 0x27, 0xef, 0xa0, 0xf3, 0xa0, 0x83, 0x14, 0xbc, 0x6b, 0xa1,
-	0xf8, 0xe2, 0xbc, 0x6b, 0xa4, 0x50, 0x70, 0xf9, 0x61, 0xc1, 0xdd, 0x3f, 0x1d, 0xa8, 0xcf, 0xe6,
-	0x15, 0x9f, 0x42, 0x45, 0x9f, 0x91, 0x29, 0x75, 0x3a, 0xe5, 0x9e, 0x37, 0xb4, 0x3b, 0xdc, 0x84,
-	0xda, 0x84, 0x33, 0x19, 0x6b, 0xc6, 0x35, 0xcc, 0x6c, 0x8f, 0x2f, 0x60, 0x2d, 0x53, 0x5d, 0x88,
-	0xa9, 0x0a, 0x44, 0x18, 0x07, 0xb4, 0x6c, 0x24, 0xad, 0x0c, 0xfe, 0xd6, 0xa2, 0xf8, 0x1c, 0x56,
-	0xf3, 0x43, 0x17, 0xb1, 0xae, 0xd4, 0x33, 0xb2, 0x66, 0x0e, 0x9e, 0xf0, 0x1b, 0x85, 0xcf, 0x01,
-	0xd8, 0x54, 0x89, 0x8b, 0x09, 0x67, 0xd7, 0xdc, 0x0c, 0x43, 0xde, 0xd0, 0xba, 0xc6, 0x8f, 0x35,
-	0xdc, 0x7d, 0xeb, 0x00, 0x68, 0xd3, 0xfb, 0x57, 0x2c, 0x0e, 0xf4, 0x47, 0xe5, 0x86, 0xbe, 0xed,
-	0x09, 0x68, 0xed, 0xfd, 0xfb, 0x67, 0xee, 0xd1, 0xc1, 0xd0, 0x0d, 0x7d, 0xfc, 0xc2, 0x8e, 0xb4,
-	0x6b, 0x46, 0xfa, 0x69, 0xf1, 0x13, 0xcd, 0x4e, 0x3f, 0x98, 0xea, 0x17, 0x50, 0x8d, 0x85, 0xcf,
-	0x2f, 0x42, 0xdf, 0x36, 0xac, 0x65, 0x43, 0x56, 0x4e, 0x84, 0xcf, 0x8f, 0x0e, 0x86, 0x15, 0x4d,
-	0x1f, 0xf9, 0xc5, 0x3b, 0xf3, 0x16, 0xef, 0x2c, 0x02, 0x32, 0x4f, 0x70, 0x1a, 0xc6, 0xc1, 0x84,
-	0xcf, 0x8c, 0x38, 0xff, 0xc5, 0x88, 0xfb, 0x31, 0x23, 0xdd, 0x3f, 0x1c, 0x68, 0xce, 0xe3, 0x9c,
-	0xf7, 0x71, 0x0f, 0x40, 0x49, 0x16, 0xa7, 0xa1, 0x0a, 0x45, 0x6c, 0x33, 0x6e, 0x3d, 0x92, 0x71,
-	0xa6, 0xc9, 0x27, 0x72, 0x7e, 0x0a, 0xbf, 0x82, 0xea, 0xc8, 0xa8, 0xb2, 0x1b, 0x2f, 0x3c, 0x29,
-	0xcb, 0xa5, 0xe5, 0x5f, 0x98, 0x95, 0x17, 0xfb, 0x52, 0x5e, 0xe8, 0xcb, 0xf6, 0x21, 0xd4, 0x67,
-	0xaf, 0x35, 0xae, 0x41, 0xc3, 0x6c, 0x4e, 0x84, 0x8c, 0xd8, 0x84, 0x94, 0xf0, 0x09, 0xac, 0x19,
-	0x60, 0x1e, 0x9f, 0x38, 0xf8, 0x3f, 0x58, 0x5f, 0x02, 0xcf, 0xfb, 0xc4, 0xdd, 0xfe, 0xcb, 0x85,
-	0x46, 0xe1, 0x59, 0x42, 0x80, 0xca, 0x20, 0x0d, 0x0e, 0xa7, 0x09, 0x29, 0x61, 0x03, 0xaa, 0x83,
-	0x34, 0xd8, 0xe3, 0x4c, 0x11, 0xc7, 0x6e, 0x5e, 0x4b, 0x91, 0x10, 0xd7, 0xaa, 0x76, 0x93, 0x84,
-	0x94, 0xb1, 0x05, 0x90, 0xad, 0x87, 0x3c, 0x4d, 0x88, 0x67, 0x85, 0xe7, 0x42, 0x71, 0xb2, 0xa2,
-	0xbd, 0xd9, 0x8d, 0x61, 0x2b, 0x96, 0xd5, 0x4f, 0x00, 0xa9, 0x22, 0x81, 0xa6, 0x4e, 0xc6, 0x99,
-	0x54, 0x97, 0x3a, 0x4b, 0x0d, 0x37, 0x80, 0x14, 0x11, 0x73, 0xa8, 0x8e, 0x08, 0xad, 0x41, 0x1a,
-	0xbc, 0x89, 0x25, 0x67, 0xa3, 0x2b, 0x76, 0x39, 0xe1, 0x04, 0x70, 0x1d, 0x56, 0x6d, 0x20, 0xfd,
-	0xc5, 0x4d, 0x53, 0xd2, 0xb0, 0xb2, 0xfd, 0x2b, 0x3e, 0xfa, 0xe9, 0xbb, 0xa9, 0x90, 0xd3, 0x88,
-	0x34, 0x75, 0xd9, 0x83, 0x34, 0x30, 0x17, 0x34, 0xe6, 0xf2, 0x98, 0x33, 0x9f, 0x4b, 0xb2, 0x6a,
-	0x4f, 0x9f, 0x85, 0x11, 0x17, 0x53, 0x75, 0x22, 0x7e, 0x26, 0x2d, 0x6b, 0x66, 0xc8, 0x99, 0x6f,
-	0x7e, 0x61, 0x64, 0xcd, 0x9a, 0x99, 0x21, 0xc6, 0x0c, 0xb1, 0xf5, 0xbe, 0x96, 0xdc, 0x94, 0xb8,
-	0x6e, 0xb3, 0xda, 0xbd, 0xd1, 0xe0, 0xf6, 0x6f, 0x0e, 0x6c, 0x3c, 0x36, 0x1e, 0xb8, 0x05, 0xf4,
-	0x31, 0x7c, 0x77, 0xaa, 0x04, 0x29, 0xe1, 0xa7, 0xf0, 0xff, 0xc7, 0xd8, 0x6f, 0x44, 0x18, 0xab,
-	0xa3, 0x28, 0x99, 0x84, 0xa3, 0x50, 0x5f, 0xc5, 0xc7, 0x64, 0xaf, 0x6e, 0xac, 0xcc, 0xdd, 0xbe,
-	0x85, 0xd6, 0xe2, 0x47, 0xa1, 0x9b, 0x31, 0x47, 0x76, 0x7d, 0x5f, 0x8f, 0x3f, 0x29, 0x21, 0x2d,
-	0x9a, 0x1d, 0xf2, 0x48, 0x5c, 0x73, 0xc3, 0x38, 0x8b, 0xcc, 0x9b, 0xc4, 0x67, 0x2a, 0x63, 0xdc,
-	0xc5, 0x42, 0x76, 0x7d, 0xff, 0x38, 0x7b, 0x7b, 0x0c, 0x5b, 0xde, 0xa3, 0x77, 0x1f, 0xda, 0xa5,
-	0x77, 0x1f, 0xda, 0xa5, 0xbb, 0xfb, 0xb6, 0xf3, 0xee, 0xbe, 0xed, 0xfc, 0x7d, 0xdf, 0x76, 0x7e,
-	0xff, 0xa7, 0x5d, 0xfa, 0x37, 0x00, 0x00, 0xff, 0xff, 0x87, 0x11, 0x6d, 0xd6, 0xaf, 0x08, 0x00,
-	0x00,
-}
diff --git a/vendor/go.etcd.io/etcd/raft/raftpb/raft.proto b/vendor/go.etcd.io/etcd/raft/raftpb/raft.proto
deleted file mode 100644
index 23d62ec..0000000
--- a/vendor/go.etcd.io/etcd/raft/raftpb/raft.proto
+++ /dev/null
@@ -1,177 +0,0 @@
-syntax = "proto2";
-package raftpb;
-
-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;
-option (gogoproto.goproto_enum_prefix_all) = false;
-
-enum EntryType {
-	EntryNormal       = 0;
-	EntryConfChange   = 1; // corresponds to pb.ConfChange
-	EntryConfChangeV2 = 2; // corresponds to pb.ConfChangeV2
-}
-
-message Entry {
-	optional uint64     Term  = 2 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
-	optional uint64     Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
-	optional EntryType  Type  = 1 [(gogoproto.nullable) = false];
-	optional bytes      Data  = 4;
-}
-
-message SnapshotMetadata {
-	optional ConfState conf_state = 1 [(gogoproto.nullable) = false];
-	optional uint64    index      = 2 [(gogoproto.nullable) = false];
-	optional uint64    term       = 3 [(gogoproto.nullable) = false];
-}
-
-message Snapshot {
-	optional bytes            data     = 1;
-	optional SnapshotMetadata metadata = 2 [(gogoproto.nullable) = false];
-}
-
-enum MessageType {
-	MsgHup             = 0;
-	MsgBeat            = 1;
-	MsgProp            = 2;
-	MsgApp             = 3;
-	MsgAppResp         = 4;
-	MsgVote            = 5;
-	MsgVoteResp        = 6;
-	MsgSnap            = 7;
-	MsgHeartbeat       = 8;
-	MsgHeartbeatResp   = 9;
-	MsgUnreachable     = 10;
-	MsgSnapStatus      = 11;
-	MsgCheckQuorum     = 12;
-	MsgTransferLeader  = 13;
-	MsgTimeoutNow      = 14;
-	MsgReadIndex       = 15;
-	MsgReadIndexResp   = 16;
-	MsgPreVote         = 17;
-	MsgPreVoteResp     = 18;
-}
-
-message Message {
-	optional MessageType type        = 1  [(gogoproto.nullable) = false];
-	optional uint64      to          = 2  [(gogoproto.nullable) = false];
-	optional uint64      from        = 3  [(gogoproto.nullable) = false];
-	optional uint64      term        = 4  [(gogoproto.nullable) = false];
-	optional uint64      logTerm     = 5  [(gogoproto.nullable) = false];
-	optional uint64      index       = 6  [(gogoproto.nullable) = false];
-	repeated Entry       entries     = 7  [(gogoproto.nullable) = false];
-	optional uint64      commit      = 8  [(gogoproto.nullable) = false];
-	optional Snapshot    snapshot    = 9  [(gogoproto.nullable) = false];
-	optional bool        reject      = 10 [(gogoproto.nullable) = false];
-	optional uint64      rejectHint  = 11 [(gogoproto.nullable) = false];
-	optional bytes       context     = 12;
-}
-
-message HardState {
-	optional uint64 term   = 1 [(gogoproto.nullable) = false];
-	optional uint64 vote   = 2 [(gogoproto.nullable) = false];
-	optional uint64 commit = 3 [(gogoproto.nullable) = false];
-}
-
-// ConfChangeTransition specifies the behavior of a configuration change with
-// respect to joint consensus.
-enum ConfChangeTransition {
-	// Automatically use the simple protocol if possible, otherwise fall back
-	// to ConfChangeJointImplicit. Most applications will want to use this.
-	ConfChangeTransitionAuto          = 0;
-	// Use joint consensus unconditionally, and transition out of them
-	// automatically (by proposing a zero configuration change).
-	//
-	// This option is suitable for applications that want to minimize the time
-	// spent in the joint configuration and do not store the joint configuration
-	// in the state machine (outside of InitialState).
-	ConfChangeTransitionJointImplicit = 1;
-	// Use joint consensus and remain in the joint configuration until the
-	// application proposes a no-op configuration change. This is suitable for
-	// applications that want to explicitly control the transitions, for example
-	// to use a custom payload (via the Context field).
-	ConfChangeTransitionJointExplicit = 2;
-}
-
-message ConfState {
-	// The voters in the incoming config. (If the configuration is not joint,
-	// then the outgoing config is empty).
-	repeated uint64 voters = 1;
-	// The learners in the incoming config.
-	repeated uint64 learners          = 2;
-	// The voters in the outgoing config.
-	repeated uint64 voters_outgoing   = 3;
-	// The nodes that will become learners when the outgoing config is removed.
-	// These nodes are necessarily currently in nodes_joint (or they would have
-	// been added to the incoming config right away).
-	repeated uint64 learners_next     = 4;
-	// If set, the config is joint and Raft will automatically transition into
-	// the final config (i.e. remove the outgoing config) when this is safe.
-	optional bool   auto_leave        = 5 [(gogoproto.nullable) = false];
-}
-
-enum ConfChangeType {
-	ConfChangeAddNode        = 0;
-	ConfChangeRemoveNode     = 1;
-	ConfChangeUpdateNode     = 2;
-	ConfChangeAddLearnerNode = 3;
-}
-
-message ConfChange {
-	optional ConfChangeType  type    = 2 [(gogoproto.nullable) = false];
-	optional uint64          node_id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "NodeID" ];
-	optional bytes           context = 4;
-
-	// NB: this is used only by etcd to thread through a unique identifier.
-	// Ideally it should really use the Context instead. No counterpart to
-	// this field exists in ConfChangeV2.
-	optional uint64          id      = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID" ];
-}
-
-// ConfChangeSingle is an individual configuration change operation. Multiple
-// such operations can be carried out atomically via a ConfChangeV2.
-message ConfChangeSingle {
-	optional ConfChangeType  type    = 1 [(gogoproto.nullable) = false];
-	optional uint64          node_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "NodeID"];
-}
-
-// ConfChangeV2 messages initiate configuration changes. They support both the
-// simple "one at a time" membership change protocol and full Joint Consensus
-// allowing for arbitrary changes in membership.
-//
-// The supplied context is treated as an opaque payload and can be used to
-// attach an action on the state machine to the application of the config change
-// proposal. Note that contrary to Joint Consensus as outlined in the Raft
-// paper[1], configuration changes become active when they are *applied* to the
-// state machine (not when they are appended to the log).
-//
-// The simple protocol can be used whenever only a single change is made.
-//
-// Non-simple changes require the use of Joint Consensus, for which two
-// configuration changes are run. The first configuration change specifies the
-// desired changes and transitions the Raft group into the joint configuration,
-// in which quorum requires a majority of both the pre-changes and post-changes
-// configuration. Joint Consensus avoids entering fragile intermediate
-// configurations that could compromise survivability. For example, without the
-// use of Joint Consensus and running across three availability zones with a
-// replication factor of three, it is not possible to replace a voter without
-// entering an intermediate configuration that does not survive the outage of
-// one availability zone.
-//
-// The provided ConfChangeTransition specifies how (and whether) Joint Consensus
-// is used, and assigns the task of leaving the joint configuration either to
-// Raft or the application. Leaving the joint configuration is accomplished by
-// proposing a ConfChangeV2 with only and optionally the Context field
-// populated.
-//
-// For details on Raft membership changes, see:
-//
-// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
-message ConfChangeV2 {
-	optional ConfChangeTransition transition = 1 [(gogoproto.nullable) = false];
-	repeated ConfChangeSingle     changes =    2 [(gogoproto.nullable) = false];
-	optional bytes                context =    3;
-}
diff --git a/vendor/go.etcd.io/etcd/raft/rawnode.go b/vendor/go.etcd.io/etcd/raft/rawnode.go
deleted file mode 100644
index 90eb694..0000000
--- a/vendor/go.etcd.io/etcd/raft/rawnode.go
+++ /dev/null
@@ -1,239 +0,0 @@
-// 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 raft
-
-import (
-	"errors"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-	"go.etcd.io/etcd/raft/tracker"
-)
-
-// ErrStepLocalMsg is returned when try to step a local raft message
-var ErrStepLocalMsg = errors.New("raft: cannot step raft local message")
-
-// ErrStepPeerNotFound is returned when try to step a response message
-// but there is no peer found in raft.prs for that node.
-var ErrStepPeerNotFound = errors.New("raft: cannot step as peer not found")
-
-// RawNode is a thread-unsafe Node.
-// The methods of this struct correspond to the methods of Node and are described
-// more fully there.
-type RawNode struct {
-	raft       *raft
-	prevSoftSt *SoftState
-	prevHardSt pb.HardState
-}
-
-// NewRawNode instantiates a RawNode from the given configuration.
-//
-// See Bootstrap() for bootstrapping an initial state; this replaces the former
-// 'peers' argument to this method (with identical behavior). However, It is
-// recommended that instead of calling Bootstrap, applications bootstrap their
-// state manually by setting up a Storage that has a first index > 1 and which
-// stores the desired ConfState as its InitialState.
-func NewRawNode(config *Config) (*RawNode, error) {
-	r := newRaft(config)
-	rn := &RawNode{
-		raft: r,
-	}
-	rn.prevSoftSt = r.softState()
-	rn.prevHardSt = r.hardState()
-	return rn, nil
-}
-
-// Tick advances the internal logical clock by a single tick.
-func (rn *RawNode) Tick() {
-	rn.raft.tick()
-}
-
-// TickQuiesced advances the internal logical clock by a single tick without
-// performing any other state machine processing. It allows the caller to avoid
-// periodic heartbeats and elections when all of the peers in a Raft group are
-// known to be at the same state. Expected usage is to periodically invoke Tick
-// or TickQuiesced depending on whether the group is "active" or "quiesced".
-//
-// WARNING: Be very careful about using this method as it subverts the Raft
-// state machine. You should probably be using Tick instead.
-func (rn *RawNode) TickQuiesced() {
-	rn.raft.electionElapsed++
-}
-
-// Campaign causes this RawNode to transition to candidate state.
-func (rn *RawNode) Campaign() error {
-	return rn.raft.Step(pb.Message{
-		Type: pb.MsgHup,
-	})
-}
-
-// Propose proposes data be appended to the raft log.
-func (rn *RawNode) Propose(data []byte) error {
-	return rn.raft.Step(pb.Message{
-		Type: pb.MsgProp,
-		From: rn.raft.id,
-		Entries: []pb.Entry{
-			{Data: data},
-		}})
-}
-
-// ProposeConfChange proposes a config change. See (Node).ProposeConfChange for
-// details.
-func (rn *RawNode) ProposeConfChange(cc pb.ConfChangeI) error {
-	m, err := confChangeToMsg(cc)
-	if err != nil {
-		return err
-	}
-	return rn.raft.Step(m)
-}
-
-// ApplyConfChange applies a config change to the local node.
-func (rn *RawNode) ApplyConfChange(cc pb.ConfChangeI) *pb.ConfState {
-	cs := rn.raft.applyConfChange(cc.AsV2())
-	return &cs
-}
-
-// Step advances the state machine using the given message.
-func (rn *RawNode) Step(m pb.Message) error {
-	// ignore unexpected local messages receiving over network
-	if IsLocalMsg(m.Type) {
-		return ErrStepLocalMsg
-	}
-	if pr := rn.raft.prs.Progress[m.From]; pr != nil || !IsResponseMsg(m.Type) {
-		return rn.raft.Step(m)
-	}
-	return ErrStepPeerNotFound
-}
-
-// Ready returns the outstanding work that the application needs to handle. This
-// includes appending and applying entries or a snapshot, updating the HardState,
-// and sending messages. The returned Ready() *must* be handled and subsequently
-// passed back via Advance().
-func (rn *RawNode) Ready() Ready {
-	rd := rn.readyWithoutAccept()
-	rn.acceptReady(rd)
-	return rd
-}
-
-// readyWithoutAccept returns a Ready. This is a read-only operation, i.e. there
-// is no obligation that the Ready must be handled.
-func (rn *RawNode) readyWithoutAccept() Ready {
-	return newReady(rn.raft, rn.prevSoftSt, rn.prevHardSt)
-}
-
-// acceptReady is called when the consumer of the RawNode has decided to go
-// ahead and handle a Ready. Nothing must alter the state of the RawNode between
-// this call and the prior call to Ready().
-func (rn *RawNode) acceptReady(rd Ready) {
-	if rd.SoftState != nil {
-		rn.prevSoftSt = rd.SoftState
-	}
-	if len(rd.ReadStates) != 0 {
-		rn.raft.readStates = nil
-	}
-	rn.raft.msgs = nil
-}
-
-// HasReady called when RawNode user need to check if any Ready pending.
-// Checking logic in this method should be consistent with Ready.containsUpdates().
-func (rn *RawNode) HasReady() bool {
-	r := rn.raft
-	if !r.softState().equal(rn.prevSoftSt) {
-		return true
-	}
-	if hardSt := r.hardState(); !IsEmptyHardState(hardSt) && !isHardStateEqual(hardSt, rn.prevHardSt) {
-		return true
-	}
-	if r.raftLog.unstable.snapshot != nil && !IsEmptySnap(*r.raftLog.unstable.snapshot) {
-		return true
-	}
-	if len(r.msgs) > 0 || len(r.raftLog.unstableEntries()) > 0 || r.raftLog.hasNextEnts() {
-		return true
-	}
-	if len(r.readStates) != 0 {
-		return true
-	}
-	return false
-}
-
-// Advance notifies the RawNode that the application has applied and saved progress in the
-// last Ready results.
-func (rn *RawNode) Advance(rd Ready) {
-	if !IsEmptyHardState(rd.HardState) {
-		rn.prevHardSt = rd.HardState
-	}
-	rn.raft.advance(rd)
-}
-
-// Status returns the current status of the given group. This allocates, see
-// BasicStatus and WithProgress for allocation-friendlier choices.
-func (rn *RawNode) Status() Status {
-	status := getStatus(rn.raft)
-	return status
-}
-
-// BasicStatus returns a BasicStatus. Notably this does not contain the
-// Progress map; see WithProgress for an allocation-free way to inspect it.
-func (rn *RawNode) BasicStatus() BasicStatus {
-	return getBasicStatus(rn.raft)
-}
-
-// ProgressType indicates the type of replica a Progress corresponds to.
-type ProgressType byte
-
-const (
-	// ProgressTypePeer accompanies a Progress for a regular peer replica.
-	ProgressTypePeer ProgressType = iota
-	// ProgressTypeLearner accompanies a Progress for a learner replica.
-	ProgressTypeLearner
-)
-
-// WithProgress is a helper to introspect the Progress for this node and its
-// peers.
-func (rn *RawNode) WithProgress(visitor func(id uint64, typ ProgressType, pr tracker.Progress)) {
-	rn.raft.prs.Visit(func(id uint64, pr *tracker.Progress) {
-		typ := ProgressTypePeer
-		if pr.IsLearner {
-			typ = ProgressTypeLearner
-		}
-		p := *pr
-		p.Inflights = nil
-		visitor(id, typ, p)
-	})
-}
-
-// ReportUnreachable reports the given node is not reachable for the last send.
-func (rn *RawNode) ReportUnreachable(id uint64) {
-	_ = rn.raft.Step(pb.Message{Type: pb.MsgUnreachable, From: id})
-}
-
-// ReportSnapshot reports the status of the sent snapshot.
-func (rn *RawNode) ReportSnapshot(id uint64, status SnapshotStatus) {
-	rej := status == SnapshotFailure
-
-	_ = rn.raft.Step(pb.Message{Type: pb.MsgSnapStatus, From: id, Reject: rej})
-}
-
-// TransferLeader tries to transfer leadership to the given transferee.
-func (rn *RawNode) TransferLeader(transferee uint64) {
-	_ = rn.raft.Step(pb.Message{Type: pb.MsgTransferLeader, From: transferee})
-}
-
-// ReadIndex requests a read state. The read state will be set in ready.
-// Read State has a read index. Once the application advances further than the read
-// index, any linearizable read requests issued before the read request can be
-// processed safely. The read state will have the same rctx attached.
-func (rn *RawNode) ReadIndex(rctx []byte) {
-	_ = rn.raft.Step(pb.Message{Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: rctx}}})
-}
diff --git a/vendor/go.etcd.io/etcd/raft/read_only.go b/vendor/go.etcd.io/etcd/raft/read_only.go
deleted file mode 100644
index 6987f1b..0000000
--- a/vendor/go.etcd.io/etcd/raft/read_only.go
+++ /dev/null
@@ -1,121 +0,0 @@
-// 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 raft
-
-import pb "go.etcd.io/etcd/raft/raftpb"
-
-// ReadState provides state for read only query.
-// It's caller's responsibility to call ReadIndex first before getting
-// this state from ready, it's also caller's duty to differentiate if this
-// state is what it requests through RequestCtx, eg. given a unique id as
-// RequestCtx
-type ReadState struct {
-	Index      uint64
-	RequestCtx []byte
-}
-
-type readIndexStatus struct {
-	req   pb.Message
-	index uint64
-	// NB: this never records 'false', but it's more convenient to use this
-	// instead of a map[uint64]struct{} due to the API of quorum.VoteResult. If
-	// this becomes performance sensitive enough (doubtful), quorum.VoteResult
-	// can change to an API that is closer to that of CommittedIndex.
-	acks map[uint64]bool
-}
-
-type readOnly struct {
-	option           ReadOnlyOption
-	pendingReadIndex map[string]*readIndexStatus
-	readIndexQueue   []string
-}
-
-func newReadOnly(option ReadOnlyOption) *readOnly {
-	return &readOnly{
-		option:           option,
-		pendingReadIndex: make(map[string]*readIndexStatus),
-	}
-}
-
-// addRequest adds a read only reuqest into readonly struct.
-// `index` is the commit index of the raft state machine when it received
-// the read only request.
-// `m` is the original read only request message from the local or remote node.
-func (ro *readOnly) addRequest(index uint64, m pb.Message) {
-	s := string(m.Entries[0].Data)
-	if _, ok := ro.pendingReadIndex[s]; ok {
-		return
-	}
-	ro.pendingReadIndex[s] = &readIndexStatus{index: index, req: m, acks: make(map[uint64]bool)}
-	ro.readIndexQueue = append(ro.readIndexQueue, s)
-}
-
-// recvAck notifies the readonly struct that the raft state machine received
-// an acknowledgment of the heartbeat that attached with the read only request
-// context.
-func (ro *readOnly) recvAck(id uint64, context []byte) map[uint64]bool {
-	rs, ok := ro.pendingReadIndex[string(context)]
-	if !ok {
-		return nil
-	}
-
-	rs.acks[id] = true
-	return rs.acks
-}
-
-// advance advances the read only request queue kept by the readonly struct.
-// It dequeues the requests until it finds the read only request that has
-// the same context as the given `m`.
-func (ro *readOnly) advance(m pb.Message) []*readIndexStatus {
-	var (
-		i     int
-		found bool
-	)
-
-	ctx := string(m.Context)
-	rss := []*readIndexStatus{}
-
-	for _, okctx := range ro.readIndexQueue {
-		i++
-		rs, ok := ro.pendingReadIndex[okctx]
-		if !ok {
-			panic("cannot find corresponding read state from pending map")
-		}
-		rss = append(rss, rs)
-		if okctx == ctx {
-			found = true
-			break
-		}
-	}
-
-	if found {
-		ro.readIndexQueue = ro.readIndexQueue[i:]
-		for _, rs := range rss {
-			delete(ro.pendingReadIndex, string(rs.req.Entries[0].Data))
-		}
-		return rss
-	}
-
-	return nil
-}
-
-// lastPendingRequestCtx returns the context of the last pending read only
-// request in readonly struct.
-func (ro *readOnly) lastPendingRequestCtx() string {
-	if len(ro.readIndexQueue) == 0 {
-		return ""
-	}
-	return ro.readIndexQueue[len(ro.readIndexQueue)-1]
-}
diff --git a/vendor/go.etcd.io/etcd/raft/status.go b/vendor/go.etcd.io/etcd/raft/status.go
deleted file mode 100644
index adc6048..0000000
--- a/vendor/go.etcd.io/etcd/raft/status.go
+++ /dev/null
@@ -1,106 +0,0 @@
-// 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 raft
-
-import (
-	"fmt"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-	"go.etcd.io/etcd/raft/tracker"
-)
-
-// Status contains information about this Raft peer and its view of the system.
-// The Progress is only populated on the leader.
-type Status struct {
-	BasicStatus
-	Config   tracker.Config
-	Progress map[uint64]tracker.Progress
-}
-
-// BasicStatus contains basic information about the Raft peer. It does not allocate.
-type BasicStatus struct {
-	ID uint64
-
-	pb.HardState
-	SoftState
-
-	Applied uint64
-
-	LeadTransferee uint64
-}
-
-func getProgressCopy(r *raft) map[uint64]tracker.Progress {
-	m := make(map[uint64]tracker.Progress)
-	r.prs.Visit(func(id uint64, pr *tracker.Progress) {
-		var p tracker.Progress
-		p = *pr
-		p.Inflights = pr.Inflights.Clone()
-		pr = nil
-
-		m[id] = p
-	})
-	return m
-}
-
-func getBasicStatus(r *raft) BasicStatus {
-	s := BasicStatus{
-		ID:             r.id,
-		LeadTransferee: r.leadTransferee,
-	}
-	s.HardState = r.hardState()
-	s.SoftState = *r.softState()
-	s.Applied = r.raftLog.applied
-	return s
-}
-
-// getStatus gets a copy of the current raft status.
-func getStatus(r *raft) Status {
-	var s Status
-	s.BasicStatus = getBasicStatus(r)
-	if s.RaftState == StateLeader {
-		s.Progress = getProgressCopy(r)
-	}
-	s.Config = r.prs.Config.Clone()
-	return s
-}
-
-// MarshalJSON translates the raft status into JSON.
-// TODO: try to simplify this by introducing ID type into raft
-func (s Status) MarshalJSON() ([]byte, error) {
-	j := fmt.Sprintf(`{"id":"%x","term":%d,"vote":"%x","commit":%d,"lead":"%x","raftState":%q,"applied":%d,"progress":{`,
-		s.ID, s.Term, s.Vote, s.Commit, s.Lead, s.RaftState, s.Applied)
-
-	if len(s.Progress) == 0 {
-		j += "},"
-	} else {
-		for k, v := range s.Progress {
-			subj := fmt.Sprintf(`"%x":{"match":%d,"next":%d,"state":%q},`, k, v.Match, v.Next, v.State)
-			j += subj
-		}
-		// remove the trailing ","
-		j = j[:len(j)-1] + "},"
-	}
-
-	j += fmt.Sprintf(`"leadtransferee":"%x"}`, s.LeadTransferee)
-	return []byte(j), nil
-}
-
-func (s Status) String() string {
-	b, err := s.MarshalJSON()
-	if err != nil {
-		raftLogger.Panicf("unexpected error: %v", err)
-	}
-	return string(b)
-}
diff --git a/vendor/go.etcd.io/etcd/raft/storage.go b/vendor/go.etcd.io/etcd/raft/storage.go
deleted file mode 100644
index 6be5745..0000000
--- a/vendor/go.etcd.io/etcd/raft/storage.go
+++ /dev/null
@@ -1,273 +0,0 @@
-// 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 raft
-
-import (
-	"errors"
-	"sync"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-// ErrCompacted is returned by Storage.Entries/Compact when a requested
-// index is unavailable because it predates the last snapshot.
-var ErrCompacted = errors.New("requested index is unavailable due to compaction")
-
-// ErrSnapOutOfDate is returned by Storage.CreateSnapshot when a requested
-// index is older than the existing snapshot.
-var ErrSnapOutOfDate = errors.New("requested index is older than the existing snapshot")
-
-// ErrUnavailable is returned by Storage interface when the requested log entries
-// are unavailable.
-var ErrUnavailable = errors.New("requested entry at index is unavailable")
-
-// ErrSnapshotTemporarilyUnavailable is returned by the Storage interface when the required
-// snapshot is temporarily unavailable.
-var ErrSnapshotTemporarilyUnavailable = errors.New("snapshot is temporarily unavailable")
-
-// Storage is an interface that may be implemented by the application
-// to retrieve log entries from storage.
-//
-// If any Storage method returns an error, the raft instance will
-// become inoperable and refuse to participate in elections; the
-// application is responsible for cleanup and recovery in this case.
-type Storage interface {
-	// TODO(tbg): split this into two interfaces, LogStorage and StateStorage.
-
-	// InitialState returns the saved HardState and ConfState information.
-	InitialState() (pb.HardState, pb.ConfState, error)
-	// Entries returns a slice of log entries in the range [lo,hi).
-	// MaxSize limits the total size of the log entries returned, but
-	// Entries returns at least one entry if any.
-	Entries(lo, hi, maxSize uint64) ([]pb.Entry, error)
-	// Term returns the term of entry i, which must be in the range
-	// [FirstIndex()-1, LastIndex()]. The term of the entry before
-	// FirstIndex is retained for matching purposes even though the
-	// rest of that entry may not be available.
-	Term(i uint64) (uint64, error)
-	// LastIndex returns the index of the last entry in the log.
-	LastIndex() (uint64, error)
-	// FirstIndex returns the index of the first log entry that is
-	// possibly available via Entries (older entries have been incorporated
-	// into the latest Snapshot; if storage only contains the dummy entry the
-	// first log entry is not available).
-	FirstIndex() (uint64, error)
-	// Snapshot returns the most recent snapshot.
-	// If snapshot is temporarily unavailable, it should return ErrSnapshotTemporarilyUnavailable,
-	// so raft state machine could know that Storage needs some time to prepare
-	// snapshot and call Snapshot later.
-	Snapshot() (pb.Snapshot, error)
-}
-
-// MemoryStorage implements the Storage interface backed by an
-// in-memory array.
-type MemoryStorage struct {
-	// Protects access to all fields. Most methods of MemoryStorage are
-	// run on the raft goroutine, but Append() is run on an application
-	// goroutine.
-	sync.Mutex
-
-	hardState pb.HardState
-	snapshot  pb.Snapshot
-	// ents[i] has raft log position i+snapshot.Metadata.Index
-	ents []pb.Entry
-}
-
-// NewMemoryStorage creates an empty MemoryStorage.
-func NewMemoryStorage() *MemoryStorage {
-	return &MemoryStorage{
-		// When starting from scratch populate the list with a dummy entry at term zero.
-		ents: make([]pb.Entry, 1),
-	}
-}
-
-// InitialState implements the Storage interface.
-func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) {
-	return ms.hardState, ms.snapshot.Metadata.ConfState, nil
-}
-
-// SetHardState saves the current HardState.
-func (ms *MemoryStorage) SetHardState(st pb.HardState) error {
-	ms.Lock()
-	defer ms.Unlock()
-	ms.hardState = st
-	return nil
-}
-
-// Entries implements the Storage interface.
-func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	offset := ms.ents[0].Index
-	if lo <= offset {
-		return nil, ErrCompacted
-	}
-	if hi > ms.lastIndex()+1 {
-		raftLogger.Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex())
-	}
-	// only contains dummy entries.
-	if len(ms.ents) == 1 {
-		return nil, ErrUnavailable
-	}
-
-	ents := ms.ents[lo-offset : hi-offset]
-	return limitSize(ents, maxSize), nil
-}
-
-// Term implements the Storage interface.
-func (ms *MemoryStorage) Term(i uint64) (uint64, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	offset := ms.ents[0].Index
-	if i < offset {
-		return 0, ErrCompacted
-	}
-	if int(i-offset) >= len(ms.ents) {
-		return 0, ErrUnavailable
-	}
-	return ms.ents[i-offset].Term, nil
-}
-
-// LastIndex implements the Storage interface.
-func (ms *MemoryStorage) LastIndex() (uint64, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	return ms.lastIndex(), nil
-}
-
-func (ms *MemoryStorage) lastIndex() uint64 {
-	return ms.ents[0].Index + uint64(len(ms.ents)) - 1
-}
-
-// FirstIndex implements the Storage interface.
-func (ms *MemoryStorage) FirstIndex() (uint64, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	return ms.firstIndex(), nil
-}
-
-func (ms *MemoryStorage) firstIndex() uint64 {
-	return ms.ents[0].Index + 1
-}
-
-// Snapshot implements the Storage interface.
-func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	return ms.snapshot, nil
-}
-
-// ApplySnapshot overwrites the contents of this Storage object with
-// those of the given snapshot.
-func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
-	ms.Lock()
-	defer ms.Unlock()
-
-	//handle check for old snapshot being applied
-	msIndex := ms.snapshot.Metadata.Index
-	snapIndex := snap.Metadata.Index
-	if msIndex >= snapIndex {
-		return ErrSnapOutOfDate
-	}
-
-	ms.snapshot = snap
-	ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}}
-	return nil
-}
-
-// CreateSnapshot makes a snapshot which can be retrieved with Snapshot() and
-// can be used to reconstruct the state at that point.
-// If any configuration changes have been made since the last compaction,
-// the result of the last ApplyConfChange must be passed in.
-func (ms *MemoryStorage) CreateSnapshot(i uint64, cs *pb.ConfState, data []byte) (pb.Snapshot, error) {
-	ms.Lock()
-	defer ms.Unlock()
-	if i <= ms.snapshot.Metadata.Index {
-		return pb.Snapshot{}, ErrSnapOutOfDate
-	}
-
-	offset := ms.ents[0].Index
-	if i > ms.lastIndex() {
-		raftLogger.Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex())
-	}
-
-	ms.snapshot.Metadata.Index = i
-	ms.snapshot.Metadata.Term = ms.ents[i-offset].Term
-	if cs != nil {
-		ms.snapshot.Metadata.ConfState = *cs
-	}
-	ms.snapshot.Data = data
-	return ms.snapshot, nil
-}
-
-// Compact discards all log entries prior to compactIndex.
-// It is the application's responsibility to not attempt to compact an index
-// greater than raftLog.applied.
-func (ms *MemoryStorage) Compact(compactIndex uint64) error {
-	ms.Lock()
-	defer ms.Unlock()
-	offset := ms.ents[0].Index
-	if compactIndex <= offset {
-		return ErrCompacted
-	}
-	if compactIndex > ms.lastIndex() {
-		raftLogger.Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex())
-	}
-
-	i := compactIndex - offset
-	ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i)
-	ents[0].Index = ms.ents[i].Index
-	ents[0].Term = ms.ents[i].Term
-	ents = append(ents, ms.ents[i+1:]...)
-	ms.ents = ents
-	return nil
-}
-
-// Append the new entries to storage.
-// TODO (xiangli): ensure the entries are continuous and
-// entries[0].Index > ms.entries[0].Index
-func (ms *MemoryStorage) Append(entries []pb.Entry) error {
-	if len(entries) == 0 {
-		return nil
-	}
-
-	ms.Lock()
-	defer ms.Unlock()
-
-	first := ms.firstIndex()
-	last := entries[0].Index + uint64(len(entries)) - 1
-
-	// shortcut if there is no new entry.
-	if last < first {
-		return nil
-	}
-	// truncate compacted entries
-	if first > entries[0].Index {
-		entries = entries[first-entries[0].Index:]
-	}
-
-	offset := entries[0].Index - ms.ents[0].Index
-	switch {
-	case uint64(len(ms.ents)) > offset:
-		ms.ents = append([]pb.Entry{}, ms.ents[:offset]...)
-		ms.ents = append(ms.ents, entries...)
-	case uint64(len(ms.ents)) == offset:
-		ms.ents = append(ms.ents, entries...)
-	default:
-		raftLogger.Panicf("missing log entry [last: %d, append at: %d]",
-			ms.lastIndex(), entries[0].Index)
-	}
-	return nil
-}
diff --git a/vendor/go.etcd.io/etcd/raft/tracker/inflights.go b/vendor/go.etcd.io/etcd/raft/tracker/inflights.go
deleted file mode 100644
index 1a05634..0000000
--- a/vendor/go.etcd.io/etcd/raft/tracker/inflights.go
+++ /dev/null
@@ -1,132 +0,0 @@
-// 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 tracker
-
-// Inflights limits the number of MsgApp (represented by the largest index
-// contained within) sent to followers but not yet acknowledged by them. Callers
-// use Full() to check whether more messages can be sent, call Add() whenever
-// they are sending a new append, and release "quota" via FreeLE() whenever an
-// ack is received.
-type Inflights struct {
-	// the starting index in the buffer
-	start int
-	// number of inflights in the buffer
-	count int
-
-	// the size of the buffer
-	size int
-
-	// buffer contains the index of the last entry
-	// inside one message.
-	buffer []uint64
-}
-
-// NewInflights sets up an Inflights that allows up to 'size' inflight messages.
-func NewInflights(size int) *Inflights {
-	return &Inflights{
-		size: size,
-	}
-}
-
-// Clone returns an *Inflights that is identical to but shares no memory with
-// the receiver.
-func (in *Inflights) Clone() *Inflights {
-	ins := *in
-	ins.buffer = append([]uint64(nil), in.buffer...)
-	return &ins
-}
-
-// Add notifies the Inflights that a new message with the given index is being
-// dispatched. Full() must be called prior to Add() to verify that there is room
-// for one more message, and consecutive calls to add Add() must provide a
-// monotonic sequence of indexes.
-func (in *Inflights) Add(inflight uint64) {
-	if in.Full() {
-		panic("cannot add into a Full inflights")
-	}
-	next := in.start + in.count
-	size := in.size
-	if next >= size {
-		next -= size
-	}
-	if next >= len(in.buffer) {
-		in.grow()
-	}
-	in.buffer[next] = inflight
-	in.count++
-}
-
-// grow the inflight buffer by doubling up to inflights.size. We grow on demand
-// instead of preallocating to inflights.size to handle systems which have
-// thousands of Raft groups per process.
-func (in *Inflights) grow() {
-	newSize := len(in.buffer) * 2
-	if newSize == 0 {
-		newSize = 1
-	} else if newSize > in.size {
-		newSize = in.size
-	}
-	newBuffer := make([]uint64, newSize)
-	copy(newBuffer, in.buffer)
-	in.buffer = newBuffer
-}
-
-// FreeLE frees the inflights smaller or equal to the given `to` flight.
-func (in *Inflights) FreeLE(to uint64) {
-	if in.count == 0 || to < in.buffer[in.start] {
-		// out of the left side of the window
-		return
-	}
-
-	idx := in.start
-	var i int
-	for i = 0; i < in.count; i++ {
-		if to < in.buffer[idx] { // found the first large inflight
-			break
-		}
-
-		// increase index and maybe rotate
-		size := in.size
-		if idx++; idx >= size {
-			idx -= size
-		}
-	}
-	// free i inflights and set new start index
-	in.count -= i
-	in.start = idx
-	if in.count == 0 {
-		// inflights is empty, reset the start index so that we don't grow the
-		// buffer unnecessarily.
-		in.start = 0
-	}
-}
-
-// FreeFirstOne releases the first inflight. This is a no-op if nothing is
-// inflight.
-func (in *Inflights) FreeFirstOne() { in.FreeLE(in.buffer[in.start]) }
-
-// Full returns true if no more messages can be sent at the moment.
-func (in *Inflights) Full() bool {
-	return in.count == in.size
-}
-
-// Count returns the number of inflight messages.
-func (in *Inflights) Count() int { return in.count }
-
-// reset frees all inflights.
-func (in *Inflights) reset() {
-	in.count = 0
-	in.start = 0
-}
diff --git a/vendor/go.etcd.io/etcd/raft/tracker/progress.go b/vendor/go.etcd.io/etcd/raft/tracker/progress.go
deleted file mode 100644
index 62c81f4..0000000
--- a/vendor/go.etcd.io/etcd/raft/tracker/progress.go
+++ /dev/null
@@ -1,259 +0,0 @@
-// 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 tracker
-
-import (
-	"fmt"
-	"sort"
-	"strings"
-)
-
-// Progress represents a follower’s progress in the view of the leader. Leader
-// maintains progresses of all followers, and sends entries to the follower
-// based on its progress.
-//
-// NB(tbg): Progress is basically a state machine whose transitions are mostly
-// strewn around `*raft.raft`. Additionally, some fields are only used when in a
-// certain State. All of this isn't ideal.
-type Progress struct {
-	Match, Next uint64
-	// State defines how the leader should interact with the follower.
-	//
-	// When in StateProbe, leader sends at most one replication message
-	// per heartbeat interval. It also probes actual progress of the follower.
-	//
-	// When in StateReplicate, leader optimistically increases next
-	// to the latest entry sent after sending replication message. This is
-	// an optimized state for fast replicating log entries to the follower.
-	//
-	// When in StateSnapshot, leader should have sent out snapshot
-	// before and stops sending any replication message.
-	State StateType
-
-	// PendingSnapshot is used in StateSnapshot.
-	// If there is a pending snapshot, the pendingSnapshot will be set to the
-	// index of the snapshot. If pendingSnapshot is set, the replication process of
-	// this Progress will be paused. raft will not resend snapshot until the pending one
-	// is reported to be failed.
-	PendingSnapshot uint64
-
-	// RecentActive is true if the progress is recently active. Receiving any messages
-	// from the corresponding follower indicates the progress is active.
-	// RecentActive can be reset to false after an election timeout.
-	//
-	// TODO(tbg): the leader should always have this set to true.
-	RecentActive bool
-
-	// ProbeSent is used while this follower is in StateProbe. When ProbeSent is
-	// true, raft should pause sending replication message to this peer until
-	// ProbeSent is reset. See ProbeAcked() and IsPaused().
-	ProbeSent bool
-
-	// Inflights is a sliding window for the inflight messages.
-	// Each inflight message contains one or more log entries.
-	// The max number of entries per message is defined in raft config as MaxSizePerMsg.
-	// Thus inflight effectively limits both the number of inflight messages
-	// and the bandwidth each Progress can use.
-	// When inflights is Full, no more message should be sent.
-	// When a leader sends out a message, the index of the last
-	// entry should be added to inflights. The index MUST be added
-	// into inflights in order.
-	// When a leader receives a reply, the previous inflights should
-	// be freed by calling inflights.FreeLE with the index of the last
-	// received entry.
-	Inflights *Inflights
-
-	// IsLearner is true if this progress is tracked for a learner.
-	IsLearner bool
-}
-
-// ResetState moves the Progress into the specified State, resetting ProbeSent,
-// PendingSnapshot, and Inflights.
-func (pr *Progress) ResetState(state StateType) {
-	pr.ProbeSent = false
-	pr.PendingSnapshot = 0
-	pr.State = state
-	pr.Inflights.reset()
-}
-
-func max(a, b uint64) uint64 {
-	if a > b {
-		return a
-	}
-	return b
-}
-
-func min(a, b uint64) uint64 {
-	if a > b {
-		return b
-	}
-	return a
-}
-
-// ProbeAcked is called when this peer has accepted an append. It resets
-// ProbeSent to signal that additional append messages should be sent without
-// further delay.
-func (pr *Progress) ProbeAcked() {
-	pr.ProbeSent = false
-}
-
-// BecomeProbe transitions into StateProbe. Next is reset to Match+1 or,
-// optionally and if larger, the index of the pending snapshot.
-func (pr *Progress) BecomeProbe() {
-	// If the original state is StateSnapshot, progress knows that
-	// the pending snapshot has been sent to this peer successfully, then
-	// probes from pendingSnapshot + 1.
-	if pr.State == StateSnapshot {
-		pendingSnapshot := pr.PendingSnapshot
-		pr.ResetState(StateProbe)
-		pr.Next = max(pr.Match+1, pendingSnapshot+1)
-	} else {
-		pr.ResetState(StateProbe)
-		pr.Next = pr.Match + 1
-	}
-}
-
-// BecomeReplicate transitions into StateReplicate, resetting Next to Match+1.
-func (pr *Progress) BecomeReplicate() {
-	pr.ResetState(StateReplicate)
-	pr.Next = pr.Match + 1
-}
-
-// BecomeSnapshot moves the Progress to StateSnapshot with the specified pending
-// snapshot index.
-func (pr *Progress) BecomeSnapshot(snapshoti uint64) {
-	pr.ResetState(StateSnapshot)
-	pr.PendingSnapshot = snapshoti
-}
-
-// MaybeUpdate is called when an MsgAppResp arrives from the follower, with the
-// index acked by it. The method returns false if the given n index comes from
-// an outdated message. Otherwise it updates the progress and returns true.
-func (pr *Progress) MaybeUpdate(n uint64) bool {
-	var updated bool
-	if pr.Match < n {
-		pr.Match = n
-		updated = true
-		pr.ProbeAcked()
-	}
-	if pr.Next < n+1 {
-		pr.Next = n + 1
-	}
-	return updated
-}
-
-// OptimisticUpdate signals that appends all the way up to and including index n
-// are in-flight. As a result, Next is increased to n+1.
-func (pr *Progress) OptimisticUpdate(n uint64) { pr.Next = n + 1 }
-
-// MaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The
-// arguments are the index the follower rejected to append to its log, and its
-// last index.
-//
-// Rejections can happen spuriously as messages are sent out of order or
-// duplicated. In such cases, the rejection pertains to an index that the
-// Progress already knows were previously acknowledged, and false is returned
-// without changing the Progress.
-//
-// If the rejection is genuine, Next is lowered sensibly, and the Progress is
-// cleared for sending log entries.
-func (pr *Progress) MaybeDecrTo(rejected, last uint64) bool {
-	if pr.State == StateReplicate {
-		// The rejection must be stale if the progress has matched and "rejected"
-		// is smaller than "match".
-		if rejected <= pr.Match {
-			return false
-		}
-		// Directly decrease next to match + 1.
-		//
-		// TODO(tbg): why not use last if it's larger?
-		pr.Next = pr.Match + 1
-		return true
-	}
-
-	// The rejection must be stale if "rejected" does not match next - 1. This
-	// is because non-replicating followers are probed one entry at a time.
-	if pr.Next-1 != rejected {
-		return false
-	}
-
-	if pr.Next = min(rejected, last+1); pr.Next < 1 {
-		pr.Next = 1
-	}
-	pr.ProbeSent = false
-	return true
-}
-
-// IsPaused returns whether sending log entries to this node has been throttled.
-// This is done when a node has rejected recent MsgApps, is currently waiting
-// for a snapshot, or has reached the MaxInflightMsgs limit. In normal
-// operation, this is false. A throttled node will be contacted less frequently
-// until it has reached a state in which it's able to accept a steady stream of
-// log entries again.
-func (pr *Progress) IsPaused() bool {
-	switch pr.State {
-	case StateProbe:
-		return pr.ProbeSent
-	case StateReplicate:
-		return pr.Inflights.Full()
-	case StateSnapshot:
-		return true
-	default:
-		panic("unexpected state")
-	}
-}
-
-func (pr *Progress) String() string {
-	var buf strings.Builder
-	fmt.Fprintf(&buf, "%s match=%d next=%d", pr.State, pr.Match, pr.Next)
-	if pr.IsLearner {
-		fmt.Fprint(&buf, " learner")
-	}
-	if pr.IsPaused() {
-		fmt.Fprint(&buf, " paused")
-	}
-	if pr.PendingSnapshot > 0 {
-		fmt.Fprintf(&buf, " pendingSnap=%d", pr.PendingSnapshot)
-	}
-	if !pr.RecentActive {
-		fmt.Fprintf(&buf, " inactive")
-	}
-	if n := pr.Inflights.Count(); n > 0 {
-		fmt.Fprintf(&buf, " inflight=%d", n)
-		if pr.Inflights.Full() {
-			fmt.Fprint(&buf, "[full]")
-		}
-	}
-	return buf.String()
-}
-
-// ProgressMap is a map of *Progress.
-type ProgressMap map[uint64]*Progress
-
-// String prints the ProgressMap in sorted key order, one Progress per line.
-func (m ProgressMap) String() string {
-	ids := make([]uint64, 0, len(m))
-	for k := range m {
-		ids = append(ids, k)
-	}
-	sort.Slice(ids, func(i, j int) bool {
-		return ids[i] < ids[j]
-	})
-	var buf strings.Builder
-	for _, id := range ids {
-		fmt.Fprintf(&buf, "%d: %s\n", id, m[id])
-	}
-	return buf.String()
-}
diff --git a/vendor/go.etcd.io/etcd/raft/tracker/state.go b/vendor/go.etcd.io/etcd/raft/tracker/state.go
deleted file mode 100644
index 285b4b8..0000000
--- a/vendor/go.etcd.io/etcd/raft/tracker/state.go
+++ /dev/null
@@ -1,42 +0,0 @@
-// 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 tracker
-
-// StateType is the state of a tracked follower.
-type StateType uint64
-
-const (
-	// StateProbe indicates a follower whose last index isn't known. Such a
-	// follower is "probed" (i.e. an append sent periodically) to narrow down
-	// its last index. In the ideal (and common) case, only one round of probing
-	// is necessary as the follower will react with a hint. Followers that are
-	// probed over extended periods of time are often offline.
-	StateProbe StateType = iota
-	// StateReplicate is the state steady in which a follower eagerly receives
-	// log entries to append to its log.
-	StateReplicate
-	// StateSnapshot indicates a follower that needs log entries not available
-	// from the leader's Raft log. Such a follower needs a full snapshot to
-	// return to StateReplicate.
-	StateSnapshot
-)
-
-var prstmap = [...]string{
-	"StateProbe",
-	"StateReplicate",
-	"StateSnapshot",
-}
-
-func (st StateType) String() string { return prstmap[uint64(st)] }
diff --git a/vendor/go.etcd.io/etcd/raft/tracker/tracker.go b/vendor/go.etcd.io/etcd/raft/tracker/tracker.go
deleted file mode 100644
index a458114..0000000
--- a/vendor/go.etcd.io/etcd/raft/tracker/tracker.go
+++ /dev/null
@@ -1,288 +0,0 @@
-// 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 tracker
-
-import (
-	"fmt"
-	"sort"
-	"strings"
-
-	"go.etcd.io/etcd/raft/quorum"
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-// Config reflects the configuration tracked in a ProgressTracker.
-type Config struct {
-	Voters quorum.JointConfig
-	// AutoLeave is true if the configuration is joint and a transition to the
-	// incoming configuration should be carried out automatically by Raft when
-	// this is possible. If false, the configuration will be joint until the
-	// application initiates the transition manually.
-	AutoLeave bool
-	// Learners is a set of IDs corresponding to the learners active in the
-	// current configuration.
-	//
-	// Invariant: Learners and Voters does not intersect, i.e. if a peer is in
-	// either half of the joint config, it can't be a learner; if it is a
-	// learner it can't be in either half of the joint config. This invariant
-	// simplifies the implementation since it allows peers to have clarity about
-	// its current role without taking into account joint consensus.
-	Learners map[uint64]struct{}
-	// When we turn a voter into a learner during a joint consensus transition,
-	// we cannot add the learner directly when entering the joint state. This is
-	// because this would violate the invariant that the intersection of
-	// voters and learners is empty. For example, assume a Voter is removed and
-	// immediately re-added as a learner (or in other words, it is demoted):
-	//
-	// Initially, the configuration will be
-	//
-	//   voters:   {1 2 3}
-	//   learners: {}
-	//
-	// and we want to demote 3. Entering the joint configuration, we naively get
-	//
-	//   voters:   {1 2} & {1 2 3}
-	//   learners: {3}
-	//
-	// but this violates the invariant (3 is both voter and learner). Instead,
-	// we get
-	//
-	//   voters:   {1 2} & {1 2 3}
-	//   learners: {}
-	//   next_learners: {3}
-	//
-	// Where 3 is now still purely a voter, but we are remembering the intention
-	// to make it a learner upon transitioning into the final configuration:
-	//
-	//   voters:   {1 2}
-	//   learners: {3}
-	//   next_learners: {}
-	//
-	// Note that next_learners is not used while adding a learner that is not
-	// also a voter in the joint config. In this case, the learner is added
-	// right away when entering the joint configuration, so that it is caught up
-	// as soon as possible.
-	LearnersNext map[uint64]struct{}
-}
-
-func (c Config) String() string {
-	var buf strings.Builder
-	fmt.Fprintf(&buf, "voters=%s", c.Voters)
-	if c.Learners != nil {
-		fmt.Fprintf(&buf, " learners=%s", quorum.MajorityConfig(c.Learners).String())
-	}
-	if c.LearnersNext != nil {
-		fmt.Fprintf(&buf, " learners_next=%s", quorum.MajorityConfig(c.LearnersNext).String())
-	}
-	if c.AutoLeave {
-		fmt.Fprintf(&buf, " autoleave")
-	}
-	return buf.String()
-}
-
-// Clone returns a copy of the Config that shares no memory with the original.
-func (c *Config) Clone() Config {
-	clone := func(m map[uint64]struct{}) map[uint64]struct{} {
-		if m == nil {
-			return nil
-		}
-		mm := make(map[uint64]struct{}, len(m))
-		for k := range m {
-			mm[k] = struct{}{}
-		}
-		return mm
-	}
-	return Config{
-		Voters:       quorum.JointConfig{clone(c.Voters[0]), clone(c.Voters[1])},
-		Learners:     clone(c.Learners),
-		LearnersNext: clone(c.LearnersNext),
-	}
-}
-
-// ProgressTracker tracks the currently active configuration and the information
-// known about the nodes and learners in it. In particular, it tracks the match
-// index for each peer which in turn allows reasoning about the committed index.
-type ProgressTracker struct {
-	Config
-
-	Progress ProgressMap
-
-	Votes map[uint64]bool
-
-	MaxInflight int
-}
-
-// MakeProgressTracker initializes a ProgressTracker.
-func MakeProgressTracker(maxInflight int) ProgressTracker {
-	p := ProgressTracker{
-		MaxInflight: maxInflight,
-		Config: Config{
-			Voters: quorum.JointConfig{
-				quorum.MajorityConfig{},
-				nil, // only populated when used
-			},
-			Learners:     nil, // only populated when used
-			LearnersNext: nil, // only populated when used
-		},
-		Votes:    map[uint64]bool{},
-		Progress: map[uint64]*Progress{},
-	}
-	return p
-}
-
-// ConfState returns a ConfState representing the active configuration.
-func (p *ProgressTracker) ConfState() pb.ConfState {
-	return pb.ConfState{
-		Voters:         p.Voters[0].Slice(),
-		VotersOutgoing: p.Voters[1].Slice(),
-		Learners:       quorum.MajorityConfig(p.Learners).Slice(),
-		LearnersNext:   quorum.MajorityConfig(p.LearnersNext).Slice(),
-		AutoLeave:      p.AutoLeave,
-	}
-}
-
-// IsSingleton returns true if (and only if) there is only one voting member
-// (i.e. the leader) in the current configuration.
-func (p *ProgressTracker) IsSingleton() bool {
-	return len(p.Voters[0]) == 1 && len(p.Voters[1]) == 0
-}
-
-type matchAckIndexer map[uint64]*Progress
-
-var _ quorum.AckedIndexer = matchAckIndexer(nil)
-
-// AckedIndex implements IndexLookuper.
-func (l matchAckIndexer) AckedIndex(id uint64) (quorum.Index, bool) {
-	pr, ok := l[id]
-	if !ok {
-		return 0, false
-	}
-	return quorum.Index(pr.Match), true
-}
-
-// Committed returns the largest log index known to be committed based on what
-// the voting members of the group have acknowledged.
-func (p *ProgressTracker) Committed() uint64 {
-	return uint64(p.Voters.CommittedIndex(matchAckIndexer(p.Progress)))
-}
-
-func insertionSort(sl []uint64) {
-	a, b := 0, len(sl)
-	for i := a + 1; i < b; i++ {
-		for j := i; j > a && sl[j] < sl[j-1]; j-- {
-			sl[j], sl[j-1] = sl[j-1], sl[j]
-		}
-	}
-}
-
-// Visit invokes the supplied closure for all tracked progresses in stable order.
-func (p *ProgressTracker) Visit(f func(id uint64, pr *Progress)) {
-	n := len(p.Progress)
-	// We need to sort the IDs and don't want to allocate since this is hot code.
-	// The optimization here mirrors that in `(MajorityConfig).CommittedIndex`,
-	// see there for details.
-	var sl [7]uint64
-	ids := sl[:]
-	if len(sl) >= n {
-		ids = sl[:n]
-	} else {
-		ids = make([]uint64, n)
-	}
-	for id := range p.Progress {
-		n--
-		ids[n] = id
-	}
-	insertionSort(ids)
-	for _, id := range ids {
-		f(id, p.Progress[id])
-	}
-}
-
-// QuorumActive returns true if the quorum is active from the view of the local
-// raft state machine. Otherwise, it returns false.
-func (p *ProgressTracker) QuorumActive() bool {
-	votes := map[uint64]bool{}
-	p.Visit(func(id uint64, pr *Progress) {
-		if pr.IsLearner {
-			return
-		}
-		votes[id] = pr.RecentActive
-	})
-
-	return p.Voters.VoteResult(votes) == quorum.VoteWon
-}
-
-// VoterNodes returns a sorted slice of voters.
-func (p *ProgressTracker) VoterNodes() []uint64 {
-	m := p.Voters.IDs()
-	nodes := make([]uint64, 0, len(m))
-	for id := range m {
-		nodes = append(nodes, id)
-	}
-	sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
-	return nodes
-}
-
-// LearnerNodes returns a sorted slice of learners.
-func (p *ProgressTracker) LearnerNodes() []uint64 {
-	if len(p.Learners) == 0 {
-		return nil
-	}
-	nodes := make([]uint64, 0, len(p.Learners))
-	for id := range p.Learners {
-		nodes = append(nodes, id)
-	}
-	sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
-	return nodes
-}
-
-// ResetVotes prepares for a new round of vote counting via recordVote.
-func (p *ProgressTracker) ResetVotes() {
-	p.Votes = map[uint64]bool{}
-}
-
-// RecordVote records that the node with the given id voted for this Raft
-// instance if v == true (and declined it otherwise).
-func (p *ProgressTracker) RecordVote(id uint64, v bool) {
-	_, ok := p.Votes[id]
-	if !ok {
-		p.Votes[id] = v
-	}
-}
-
-// TallyVotes returns the number of granted and rejected Votes, and whether the
-// election outcome is known.
-func (p *ProgressTracker) TallyVotes() (granted int, rejected int, _ quorum.VoteResult) {
-	// Make sure to populate granted/rejected correctly even if the Votes slice
-	// contains members no longer part of the configuration. This doesn't really
-	// matter in the way the numbers are used (they're informational), but might
-	// as well get it right.
-	for id, pr := range p.Progress {
-		if pr.IsLearner {
-			continue
-		}
-		v, voted := p.Votes[id]
-		if !voted {
-			continue
-		}
-		if v {
-			granted++
-		} else {
-			rejected++
-		}
-	}
-	result := p.Voters.VoteResult(p.Votes)
-	return granted, rejected, result
-}
diff --git a/vendor/go.etcd.io/etcd/raft/util.go b/vendor/go.etcd.io/etcd/raft/util.go
deleted file mode 100644
index 785cf73..0000000
--- a/vendor/go.etcd.io/etcd/raft/util.go
+++ /dev/null
@@ -1,233 +0,0 @@
-// 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 raft
-
-import (
-	"bytes"
-	"fmt"
-	"strings"
-
-	pb "go.etcd.io/etcd/raft/raftpb"
-)
-
-func (st StateType) MarshalJSON() ([]byte, error) {
-	return []byte(fmt.Sprintf("%q", st.String())), nil
-}
-
-func min(a, b uint64) uint64 {
-	if a > b {
-		return b
-	}
-	return a
-}
-
-func max(a, b uint64) uint64 {
-	if a > b {
-		return a
-	}
-	return b
-}
-
-func IsLocalMsg(msgt pb.MessageType) bool {
-	return msgt == pb.MsgHup || msgt == pb.MsgBeat || msgt == pb.MsgUnreachable ||
-		msgt == pb.MsgSnapStatus || msgt == pb.MsgCheckQuorum
-}
-
-func IsResponseMsg(msgt pb.MessageType) bool {
-	return msgt == pb.MsgAppResp || msgt == pb.MsgVoteResp || msgt == pb.MsgHeartbeatResp || msgt == pb.MsgUnreachable || msgt == pb.MsgPreVoteResp
-}
-
-// voteResponseType maps vote and prevote message types to their corresponding responses.
-func voteRespMsgType(msgt pb.MessageType) pb.MessageType {
-	switch msgt {
-	case pb.MsgVote:
-		return pb.MsgVoteResp
-	case pb.MsgPreVote:
-		return pb.MsgPreVoteResp
-	default:
-		panic(fmt.Sprintf("not a vote message: %s", msgt))
-	}
-}
-
-func DescribeHardState(hs pb.HardState) string {
-	var buf strings.Builder
-	fmt.Fprintf(&buf, "Term:%d", hs.Term)
-	if hs.Vote != 0 {
-		fmt.Fprintf(&buf, " Vote:%d", hs.Vote)
-	}
-	fmt.Fprintf(&buf, " Commit:%d", hs.Commit)
-	return buf.String()
-}
-
-func DescribeSoftState(ss SoftState) string {
-	return fmt.Sprintf("Lead:%d State:%s", ss.Lead, ss.RaftState)
-}
-
-func DescribeConfState(state pb.ConfState) string {
-	return fmt.Sprintf(
-		"Voters:%v VotersOutgoing:%v Learners:%v LearnersNext:%v AutoLeave:%v",
-		state.Voters, state.VotersOutgoing, state.Learners, state.LearnersNext, state.AutoLeave,
-	)
-}
-
-func DescribeSnapshot(snap pb.Snapshot) string {
-	m := snap.Metadata
-	return fmt.Sprintf("Index:%d Term:%d ConfState:%s", m.Index, m.Term, DescribeConfState(m.ConfState))
-}
-
-func DescribeReady(rd Ready, f EntryFormatter) string {
-	var buf strings.Builder
-	if rd.SoftState != nil {
-		fmt.Fprint(&buf, DescribeSoftState(*rd.SoftState))
-		buf.WriteByte('\n')
-	}
-	if !IsEmptyHardState(rd.HardState) {
-		fmt.Fprintf(&buf, "HardState %s", DescribeHardState(rd.HardState))
-		buf.WriteByte('\n')
-	}
-	if len(rd.ReadStates) > 0 {
-		fmt.Fprintf(&buf, "ReadStates %v\n", rd.ReadStates)
-	}
-	if len(rd.Entries) > 0 {
-		buf.WriteString("Entries:\n")
-		fmt.Fprint(&buf, DescribeEntries(rd.Entries, f))
-	}
-	if !IsEmptySnap(rd.Snapshot) {
-		fmt.Fprintf(&buf, "Snapshot %s\n", DescribeSnapshot(rd.Snapshot))
-	}
-	if len(rd.CommittedEntries) > 0 {
-		buf.WriteString("CommittedEntries:\n")
-		fmt.Fprint(&buf, DescribeEntries(rd.CommittedEntries, f))
-	}
-	if len(rd.Messages) > 0 {
-		buf.WriteString("Messages:\n")
-		for _, msg := range rd.Messages {
-			fmt.Fprint(&buf, DescribeMessage(msg, f))
-			buf.WriteByte('\n')
-		}
-	}
-	if buf.Len() > 0 {
-		return fmt.Sprintf("Ready MustSync=%t:\n%s", rd.MustSync, buf.String())
-	}
-	return "<empty Ready>"
-}
-
-// EntryFormatter can be implemented by the application to provide human-readable formatting
-// of entry data. Nil is a valid EntryFormatter and will use a default format.
-type EntryFormatter func([]byte) string
-
-// DescribeMessage returns a concise human-readable description of a
-// Message for debugging.
-func DescribeMessage(m pb.Message, f EntryFormatter) string {
-	var buf bytes.Buffer
-	fmt.Fprintf(&buf, "%x->%x %v Term:%d Log:%d/%d", m.From, m.To, m.Type, m.Term, m.LogTerm, m.Index)
-	if m.Reject {
-		fmt.Fprintf(&buf, " Rejected (Hint: %d)", m.RejectHint)
-	}
-	if m.Commit != 0 {
-		fmt.Fprintf(&buf, " Commit:%d", m.Commit)
-	}
-	if len(m.Entries) > 0 {
-		fmt.Fprintf(&buf, " Entries:[")
-		for i, e := range m.Entries {
-			if i != 0 {
-				buf.WriteString(", ")
-			}
-			buf.WriteString(DescribeEntry(e, f))
-		}
-		fmt.Fprintf(&buf, "]")
-	}
-	if !IsEmptySnap(m.Snapshot) {
-		fmt.Fprintf(&buf, " Snapshot: %s", DescribeSnapshot(m.Snapshot))
-	}
-	return buf.String()
-}
-
-// PayloadSize is the size of the payload of this Entry. Notably, it does not
-// depend on its Index or Term.
-func PayloadSize(e pb.Entry) int {
-	return len(e.Data)
-}
-
-// DescribeEntry returns a concise human-readable description of an
-// Entry for debugging.
-func DescribeEntry(e pb.Entry, f EntryFormatter) string {
-	if f == nil {
-		f = func(data []byte) string { return fmt.Sprintf("%q", data) }
-	}
-
-	formatConfChange := func(cc pb.ConfChangeI) string {
-		// TODO(tbg): give the EntryFormatter a type argument so that it gets
-		// a chance to expose the Context.
-		return pb.ConfChangesToString(cc.AsV2().Changes)
-	}
-
-	var formatted string
-	switch e.Type {
-	case pb.EntryNormal:
-		formatted = f(e.Data)
-	case pb.EntryConfChange:
-		var cc pb.ConfChange
-		if err := cc.Unmarshal(e.Data); err != nil {
-			formatted = err.Error()
-		} else {
-			formatted = formatConfChange(cc)
-		}
-	case pb.EntryConfChangeV2:
-		var cc pb.ConfChangeV2
-		if err := cc.Unmarshal(e.Data); err != nil {
-			formatted = err.Error()
-		} else {
-			formatted = formatConfChange(cc)
-		}
-	}
-	if formatted != "" {
-		formatted = " " + formatted
-	}
-	return fmt.Sprintf("%d/%d %s%s", e.Term, e.Index, e.Type, formatted)
-}
-
-// DescribeEntries calls DescribeEntry for each Entry, adding a newline to
-// each.
-func DescribeEntries(ents []pb.Entry, f EntryFormatter) string {
-	var buf bytes.Buffer
-	for _, e := range ents {
-		_, _ = buf.WriteString(DescribeEntry(e, f) + "\n")
-	}
-	return buf.String()
-}
-
-func limitSize(ents []pb.Entry, maxSize uint64) []pb.Entry {
-	if len(ents) == 0 {
-		return ents
-	}
-	size := ents[0].Size()
-	var limit int
-	for limit = 1; limit < len(ents); limit++ {
-		size += ents[limit].Size()
-		if uint64(size) > maxSize {
-			break
-		}
-	}
-	return ents[:limit]
-}
-
-func assertConfStatesEquivalent(l Logger, cs1, cs2 pb.ConfState) {
-	err := cs1.Equivalent(cs2)
-	if err == nil {
-		return
-	}
-	l.Panic(err)
-}
diff --git a/vendor/go.uber.org/atomic/.codecov.yml b/vendor/go.uber.org/atomic/.codecov.yml
index 6d4d1be..571116c 100644
--- a/vendor/go.uber.org/atomic/.codecov.yml
+++ b/vendor/go.uber.org/atomic/.codecov.yml
@@ -13,3 +13,7 @@
         if_not_found: success  # if parent is not found report status as success, error, or failure
         if_ci_failed: error    # if ci fails report status as success, error, or failure
 
+# Also update COVER_IGNORE_PKGS in the Makefile.
+ignore:
+  - /internal/gen-atomicint/
+  - /internal/gen-valuewrapper/
diff --git a/vendor/go.uber.org/atomic/.gitignore b/vendor/go.uber.org/atomic/.gitignore
index 0a4504f..c3fa253 100644
--- a/vendor/go.uber.org/atomic/.gitignore
+++ b/vendor/go.uber.org/atomic/.gitignore
@@ -1,6 +1,7 @@
+/bin
 .DS_Store
 /vendor
-/cover
+cover.html
 cover.out
 lint.log
 
diff --git a/vendor/go.uber.org/atomic/.travis.yml b/vendor/go.uber.org/atomic/.travis.yml
index 0f3769e..13d0a4f 100644
--- a/vendor/go.uber.org/atomic/.travis.yml
+++ b/vendor/go.uber.org/atomic/.travis.yml
@@ -2,26 +2,26 @@
 language: go
 go_import_path: go.uber.org/atomic
 
-go:
-  - 1.11.x
-  - 1.12.x
+env:
+  global:
+    - GO111MODULE=on
 
 matrix:
   include:
-  - go: 1.12.x
-    env: NO_TEST=yes LINT=yes
+  - go: oldstable
+  - go: stable
+    env: LINT=1
 
 cache:
   directories:
     - vendor
 
-install:
-  - make install_ci
+before_install:
+  - go version
 
 script:
-  - test -n "$NO_TEST" || make test_ci
-  - test -n "$NO_TEST" || scripts/test-ubergo.sh
-  - test -z "$LINT" || make install_lint lint
+  - test -z "$LINT" || make lint
+  - make cover
 
 after_success:
   - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/go.uber.org/atomic/CHANGELOG.md b/vendor/go.uber.org/atomic/CHANGELOG.md
new file mode 100644
index 0000000..24c0274
--- /dev/null
+++ b/vendor/go.uber.org/atomic/CHANGELOG.md
@@ -0,0 +1,76 @@
+# Changelog
+All notable changes to this project will be documented in this file.
+
+The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
+and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html).
+
+## [1.7.0] - 2020-09-14
+### Added
+- Support JSON serialization and deserialization of primitive atomic types.
+- Support Text marshalling and unmarshalling for string atomics.
+
+### Changed
+- Disallow incorrect comparison of atomic values in a non-atomic way.
+
+### Removed
+- Remove dependency on `golang.org/x/{lint, tools}`.
+
+## [1.6.0] - 2020-02-24
+### Changed
+- Drop library dependency on `golang.org/x/{lint, tools}`.
+
+## [1.5.1] - 2019-11-19
+- Fix bug where `Bool.CAS` and `Bool.Toggle` do work correctly together
+  causing `CAS` to fail even though the old value matches.
+
+## [1.5.0] - 2019-10-29
+### Changed
+- With Go modules, only the `go.uber.org/atomic` import path is supported now.
+  If you need to use the old import path, please add a `replace` directive to
+  your `go.mod`.
+
+## [1.4.0] - 2019-05-01
+### Added
+ - Add `atomic.Error` type for atomic operations on `error` values.
+
+## [1.3.2] - 2018-05-02
+### Added
+- Add `atomic.Duration` type for atomic operations on `time.Duration` values.
+
+## [1.3.1] - 2017-11-14
+### Fixed
+- Revert optimization for `atomic.String.Store("")` which caused data races.
+
+## [1.3.0] - 2017-11-13
+### Added
+- Add `atomic.Bool.CAS` for compare-and-swap semantics on bools.
+
+### Changed
+- Optimize `atomic.String.Store("")` by avoiding an allocation.
+
+## [1.2.0] - 2017-04-12
+### Added
+- Shadow `atomic.Value` from `sync/atomic`.
+
+## [1.1.0] - 2017-03-10
+### Added
+- Add atomic `Float64` type.
+
+### Changed
+- Support new `go.uber.org/atomic` import path.
+
+## [1.0.0] - 2016-07-18
+
+- Initial release.
+
+[1.7.0]: https://github.com/uber-go/atomic/compare/v1.6.0...v1.7.0
+[1.6.0]: https://github.com/uber-go/atomic/compare/v1.5.1...v1.6.0
+[1.5.1]: https://github.com/uber-go/atomic/compare/v1.5.0...v1.5.1
+[1.5.0]: https://github.com/uber-go/atomic/compare/v1.4.0...v1.5.0
+[1.4.0]: https://github.com/uber-go/atomic/compare/v1.3.2...v1.4.0
+[1.3.2]: https://github.com/uber-go/atomic/compare/v1.3.1...v1.3.2
+[1.3.1]: https://github.com/uber-go/atomic/compare/v1.3.0...v1.3.1
+[1.3.0]: https://github.com/uber-go/atomic/compare/v1.2.0...v1.3.0
+[1.2.0]: https://github.com/uber-go/atomic/compare/v1.1.0...v1.2.0
+[1.1.0]: https://github.com/uber-go/atomic/compare/v1.0.0...v1.1.0
+[1.0.0]: https://github.com/uber-go/atomic/releases/tag/v1.0.0
diff --git a/vendor/go.uber.org/atomic/Makefile b/vendor/go.uber.org/atomic/Makefile
index 1ef2630..1b1376d 100644
--- a/vendor/go.uber.org/atomic/Makefile
+++ b/vendor/go.uber.org/atomic/Makefile
@@ -1,51 +1,78 @@
-# Many Go tools take file globs or directories as arguments instead of packages.
-PACKAGE_FILES ?= *.go
+# Directory to place `go install`ed binaries into.
+export GOBIN ?= $(shell pwd)/bin
 
-# For pre go1.6
-export GO15VENDOREXPERIMENT=1
+GOLINT = $(GOBIN)/golint
+GEN_ATOMICINT = $(GOBIN)/gen-atomicint
+GEN_ATOMICWRAPPER = $(GOBIN)/gen-atomicwrapper
+STATICCHECK = $(GOBIN)/staticcheck
 
+GO_FILES ?= $(shell find . '(' -path .git -o -path vendor ')' -prune -o -name '*.go' -print)
+
+# Also update ignore section in .codecov.yml.
+COVER_IGNORE_PKGS = \
+	go.uber.org/atomic/internal/gen-atomicint \
+	go.uber.org/atomic/internal/gen-atomicwrapper
 
 .PHONY: build
 build:
-	go build -i ./...
-
-
-.PHONY: install
-install:
-	glide --version || go get github.com/Masterminds/glide
-	glide install
-
+	go build ./...
 
 .PHONY: test
 test:
-	go test -cover -race ./...
+	go test -race ./...
 
+.PHONY: gofmt
+gofmt:
+	$(eval FMT_LOG := $(shell mktemp -t gofmt.XXXXX))
+	gofmt -e -s -l $(GO_FILES) > $(FMT_LOG) || true
+	@[ ! -s "$(FMT_LOG)" ] || (echo "gofmt failed:" && cat $(FMT_LOG) && false)
 
-.PHONY: install_ci
-install_ci: install
-	go get github.com/wadey/gocovmerge
-	go get github.com/mattn/goveralls
-	go get golang.org/x/tools/cmd/cover
+$(GOLINT):
+	cd tools && go install golang.org/x/lint/golint
 
-.PHONY: install_lint
-install_lint:
-	go get golang.org/x/lint/golint
+$(STATICCHECK):
+	cd tools && go install honnef.co/go/tools/cmd/staticcheck
 
+$(GEN_ATOMICWRAPPER): $(wildcard ./internal/gen-atomicwrapper/*)
+	go build -o $@ ./internal/gen-atomicwrapper
+
+$(GEN_ATOMICINT): $(wildcard ./internal/gen-atomicint/*)
+	go build -o $@ ./internal/gen-atomicint
+
+.PHONY: golint
+golint: $(GOLINT)
+	$(GOLINT) ./...
+
+.PHONY: staticcheck
+staticcheck: $(STATICCHECK)
+	$(STATICCHECK) ./...
 
 .PHONY: lint
-lint:
-	@rm -rf lint.log
-	@echo "Checking formatting..."
-	@gofmt -d -s $(PACKAGE_FILES) 2>&1 | tee lint.log
-	@echo "Checking vet..."
-	@go vet ./... 2>&1 | tee -a lint.log;)
-	@echo "Checking lint..."
-	@golint $$(go list ./...) 2>&1 | tee -a lint.log
-	@echo "Checking for unresolved FIXMEs..."
-	@git grep -i fixme | grep -v -e vendor -e Makefile | tee -a lint.log
-	@[ ! -s lint.log ]
+lint: gofmt golint staticcheck generatenodirty
 
+# comma separated list of packages to consider for code coverage.
+COVER_PKG = $(shell \
+	go list -find ./... | \
+	grep -v $(foreach pkg,$(COVER_IGNORE_PKGS),-e "^$(pkg)$$") | \
+	paste -sd, -)
 
-.PHONY: test_ci
-test_ci: install_ci build
-	./scripts/cover.sh $(shell go list $(PACKAGES))
+.PHONY: cover
+cover:
+	go test -coverprofile=cover.out -coverpkg  $(COVER_PKG) -v ./...
+	go tool cover -html=cover.out -o cover.html
+
+.PHONY: generate
+generate: $(GEN_ATOMICINT) $(GEN_ATOMICWRAPPER)
+	go generate ./...
+
+.PHONY: generatenodirty
+generatenodirty:
+	@[ -z "$$(git status --porcelain)" ] || ( \
+		echo "Working tree is dirty. Commit your changes first."; \
+		exit 1 )
+	@make generate
+	@status=$$(git status --porcelain); \
+		[ -z "$$status" ] || ( \
+		echo "Working tree is dirty after `make generate`:"; \
+		echo "$$status"; \
+		echo "Please ensure that the generated code is up-to-date." )
diff --git a/vendor/go.uber.org/atomic/README.md b/vendor/go.uber.org/atomic/README.md
index 62eb8e5..ade0c20 100644
--- a/vendor/go.uber.org/atomic/README.md
+++ b/vendor/go.uber.org/atomic/README.md
@@ -3,9 +3,34 @@
 Simple wrappers for primitive types to enforce atomic access.
 
 ## Installation
-`go get -u go.uber.org/atomic`
+
+```shell
+$ go get -u go.uber.org/atomic@v1
+```
+
+### Legacy Import Path
+
+As of v1.5.0, the import path `go.uber.org/atomic` is the only supported way
+of using this package. If you are using Go modules, this package will fail to
+compile with the legacy import path path `github.com/uber-go/atomic`.
+
+We recommend migrating your code to the new import path but if you're unable
+to do so, or if your dependencies are still using the old import path, you
+will have to add a `replace` directive to your `go.mod` file downgrading the
+legacy import path to an older version.
+
+```
+replace github.com/uber-go/atomic => github.com/uber-go/atomic v1.4.0
+```
+
+You can do so automatically by running the following command.
+
+```shell
+$ go mod edit -replace github.com/uber-go/atomic=github.com/uber-go/atomic@v1.4.0
+```
 
 ## Usage
+
 The standard library's `sync/atomic` is powerful, but it's easy to forget which
 variables must be accessed atomically. `go.uber.org/atomic` preserves all the
 functionality of the standard library, but wraps the primitive types to
@@ -21,9 +46,11 @@
 See the [documentation][doc] for a complete API specification.
 
 ## Development Status
+
 Stable.
 
-___
+---
+
 Released under the [MIT License](LICENSE.txt).
 
 [doc-img]: https://godoc.org/github.com/uber-go/atomic?status.svg
diff --git a/vendor/go.uber.org/atomic/atomic.go b/vendor/go.uber.org/atomic/atomic.go
deleted file mode 100644
index 1db6849..0000000
--- a/vendor/go.uber.org/atomic/atomic.go
+++ /dev/null
@@ -1,351 +0,0 @@
-// Copyright (c) 2016 Uber Technologies, Inc.
-//
-// 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.
-
-// Package atomic provides simple wrappers around numerics to enforce atomic
-// access.
-package atomic
-
-import (
-	"math"
-	"sync/atomic"
-	"time"
-)
-
-// Int32 is an atomic wrapper around an int32.
-type Int32 struct{ v int32 }
-
-// NewInt32 creates an Int32.
-func NewInt32(i int32) *Int32 {
-	return &Int32{i}
-}
-
-// Load atomically loads the wrapped value.
-func (i *Int32) Load() int32 {
-	return atomic.LoadInt32(&i.v)
-}
-
-// Add atomically adds to the wrapped int32 and returns the new value.
-func (i *Int32) Add(n int32) int32 {
-	return atomic.AddInt32(&i.v, n)
-}
-
-// Sub atomically subtracts from the wrapped int32 and returns the new value.
-func (i *Int32) Sub(n int32) int32 {
-	return atomic.AddInt32(&i.v, -n)
-}
-
-// Inc atomically increments the wrapped int32 and returns the new value.
-func (i *Int32) Inc() int32 {
-	return i.Add(1)
-}
-
-// Dec atomically decrements the wrapped int32 and returns the new value.
-func (i *Int32) Dec() int32 {
-	return i.Sub(1)
-}
-
-// CAS is an atomic compare-and-swap.
-func (i *Int32) CAS(old, new int32) bool {
-	return atomic.CompareAndSwapInt32(&i.v, old, new)
-}
-
-// Store atomically stores the passed value.
-func (i *Int32) Store(n int32) {
-	atomic.StoreInt32(&i.v, n)
-}
-
-// Swap atomically swaps the wrapped int32 and returns the old value.
-func (i *Int32) Swap(n int32) int32 {
-	return atomic.SwapInt32(&i.v, n)
-}
-
-// Int64 is an atomic wrapper around an int64.
-type Int64 struct{ v int64 }
-
-// NewInt64 creates an Int64.
-func NewInt64(i int64) *Int64 {
-	return &Int64{i}
-}
-
-// Load atomically loads the wrapped value.
-func (i *Int64) Load() int64 {
-	return atomic.LoadInt64(&i.v)
-}
-
-// Add atomically adds to the wrapped int64 and returns the new value.
-func (i *Int64) Add(n int64) int64 {
-	return atomic.AddInt64(&i.v, n)
-}
-
-// Sub atomically subtracts from the wrapped int64 and returns the new value.
-func (i *Int64) Sub(n int64) int64 {
-	return atomic.AddInt64(&i.v, -n)
-}
-
-// Inc atomically increments the wrapped int64 and returns the new value.
-func (i *Int64) Inc() int64 {
-	return i.Add(1)
-}
-
-// Dec atomically decrements the wrapped int64 and returns the new value.
-func (i *Int64) Dec() int64 {
-	return i.Sub(1)
-}
-
-// CAS is an atomic compare-and-swap.
-func (i *Int64) CAS(old, new int64) bool {
-	return atomic.CompareAndSwapInt64(&i.v, old, new)
-}
-
-// Store atomically stores the passed value.
-func (i *Int64) Store(n int64) {
-	atomic.StoreInt64(&i.v, n)
-}
-
-// Swap atomically swaps the wrapped int64 and returns the old value.
-func (i *Int64) Swap(n int64) int64 {
-	return atomic.SwapInt64(&i.v, n)
-}
-
-// Uint32 is an atomic wrapper around an uint32.
-type Uint32 struct{ v uint32 }
-
-// NewUint32 creates a Uint32.
-func NewUint32(i uint32) *Uint32 {
-	return &Uint32{i}
-}
-
-// Load atomically loads the wrapped value.
-func (i *Uint32) Load() uint32 {
-	return atomic.LoadUint32(&i.v)
-}
-
-// Add atomically adds to the wrapped uint32 and returns the new value.
-func (i *Uint32) Add(n uint32) uint32 {
-	return atomic.AddUint32(&i.v, n)
-}
-
-// Sub atomically subtracts from the wrapped uint32 and returns the new value.
-func (i *Uint32) Sub(n uint32) uint32 {
-	return atomic.AddUint32(&i.v, ^(n - 1))
-}
-
-// Inc atomically increments the wrapped uint32 and returns the new value.
-func (i *Uint32) Inc() uint32 {
-	return i.Add(1)
-}
-
-// Dec atomically decrements the wrapped int32 and returns the new value.
-func (i *Uint32) Dec() uint32 {
-	return i.Sub(1)
-}
-
-// CAS is an atomic compare-and-swap.
-func (i *Uint32) CAS(old, new uint32) bool {
-	return atomic.CompareAndSwapUint32(&i.v, old, new)
-}
-
-// Store atomically stores the passed value.
-func (i *Uint32) Store(n uint32) {
-	atomic.StoreUint32(&i.v, n)
-}
-
-// Swap atomically swaps the wrapped uint32 and returns the old value.
-func (i *Uint32) Swap(n uint32) uint32 {
-	return atomic.SwapUint32(&i.v, n)
-}
-
-// Uint64 is an atomic wrapper around a uint64.
-type Uint64 struct{ v uint64 }
-
-// NewUint64 creates a Uint64.
-func NewUint64(i uint64) *Uint64 {
-	return &Uint64{i}
-}
-
-// Load atomically loads the wrapped value.
-func (i *Uint64) Load() uint64 {
-	return atomic.LoadUint64(&i.v)
-}
-
-// Add atomically adds to the wrapped uint64 and returns the new value.
-func (i *Uint64) Add(n uint64) uint64 {
-	return atomic.AddUint64(&i.v, n)
-}
-
-// Sub atomically subtracts from the wrapped uint64 and returns the new value.
-func (i *Uint64) Sub(n uint64) uint64 {
-	return atomic.AddUint64(&i.v, ^(n - 1))
-}
-
-// Inc atomically increments the wrapped uint64 and returns the new value.
-func (i *Uint64) Inc() uint64 {
-	return i.Add(1)
-}
-
-// Dec atomically decrements the wrapped uint64 and returns the new value.
-func (i *Uint64) Dec() uint64 {
-	return i.Sub(1)
-}
-
-// CAS is an atomic compare-and-swap.
-func (i *Uint64) CAS(old, new uint64) bool {
-	return atomic.CompareAndSwapUint64(&i.v, old, new)
-}
-
-// Store atomically stores the passed value.
-func (i *Uint64) Store(n uint64) {
-	atomic.StoreUint64(&i.v, n)
-}
-
-// Swap atomically swaps the wrapped uint64 and returns the old value.
-func (i *Uint64) Swap(n uint64) uint64 {
-	return atomic.SwapUint64(&i.v, n)
-}
-
-// Bool is an atomic Boolean.
-type Bool struct{ v uint32 }
-
-// NewBool creates a Bool.
-func NewBool(initial bool) *Bool {
-	return &Bool{boolToInt(initial)}
-}
-
-// Load atomically loads the Boolean.
-func (b *Bool) Load() bool {
-	return truthy(atomic.LoadUint32(&b.v))
-}
-
-// CAS is an atomic compare-and-swap.
-func (b *Bool) CAS(old, new bool) bool {
-	return atomic.CompareAndSwapUint32(&b.v, boolToInt(old), boolToInt(new))
-}
-
-// Store atomically stores the passed value.
-func (b *Bool) Store(new bool) {
-	atomic.StoreUint32(&b.v, boolToInt(new))
-}
-
-// Swap sets the given value and returns the previous value.
-func (b *Bool) Swap(new bool) bool {
-	return truthy(atomic.SwapUint32(&b.v, boolToInt(new)))
-}
-
-// Toggle atomically negates the Boolean and returns the previous value.
-func (b *Bool) Toggle() bool {
-	return truthy(atomic.AddUint32(&b.v, 1) - 1)
-}
-
-func truthy(n uint32) bool {
-	return n&1 == 1
-}
-
-func boolToInt(b bool) uint32 {
-	if b {
-		return 1
-	}
-	return 0
-}
-
-// Float64 is an atomic wrapper around float64.
-type Float64 struct {
-	v uint64
-}
-
-// NewFloat64 creates a Float64.
-func NewFloat64(f float64) *Float64 {
-	return &Float64{math.Float64bits(f)}
-}
-
-// Load atomically loads the wrapped value.
-func (f *Float64) Load() float64 {
-	return math.Float64frombits(atomic.LoadUint64(&f.v))
-}
-
-// Store atomically stores the passed value.
-func (f *Float64) Store(s float64) {
-	atomic.StoreUint64(&f.v, math.Float64bits(s))
-}
-
-// Add atomically adds to the wrapped float64 and returns the new value.
-func (f *Float64) Add(s float64) float64 {
-	for {
-		old := f.Load()
-		new := old + s
-		if f.CAS(old, new) {
-			return new
-		}
-	}
-}
-
-// Sub atomically subtracts from the wrapped float64 and returns the new value.
-func (f *Float64) Sub(s float64) float64 {
-	return f.Add(-s)
-}
-
-// CAS is an atomic compare-and-swap.
-func (f *Float64) CAS(old, new float64) bool {
-	return atomic.CompareAndSwapUint64(&f.v, math.Float64bits(old), math.Float64bits(new))
-}
-
-// Duration is an atomic wrapper around time.Duration
-// https://godoc.org/time#Duration
-type Duration struct {
-	v Int64
-}
-
-// NewDuration creates a Duration.
-func NewDuration(d time.Duration) *Duration {
-	return &Duration{v: *NewInt64(int64(d))}
-}
-
-// Load atomically loads the wrapped value.
-func (d *Duration) Load() time.Duration {
-	return time.Duration(d.v.Load())
-}
-
-// Store atomically stores the passed value.
-func (d *Duration) Store(n time.Duration) {
-	d.v.Store(int64(n))
-}
-
-// Add atomically adds to the wrapped time.Duration and returns the new value.
-func (d *Duration) Add(n time.Duration) time.Duration {
-	return time.Duration(d.v.Add(int64(n)))
-}
-
-// Sub atomically subtracts from the wrapped time.Duration and returns the new value.
-func (d *Duration) Sub(n time.Duration) time.Duration {
-	return time.Duration(d.v.Sub(int64(n)))
-}
-
-// Swap atomically swaps the wrapped time.Duration and returns the old value.
-func (d *Duration) Swap(n time.Duration) time.Duration {
-	return time.Duration(d.v.Swap(int64(n)))
-}
-
-// CAS is an atomic compare-and-swap.
-func (d *Duration) CAS(old, new time.Duration) bool {
-	return d.v.CAS(int64(old), int64(new))
-}
-
-// Value shadows the type of the same name from sync/atomic
-// https://godoc.org/sync/atomic#Value
-type Value struct{ atomic.Value }
diff --git a/vendor/go.uber.org/atomic/bool.go b/vendor/go.uber.org/atomic/bool.go
new file mode 100644
index 0000000..9cf1914
--- /dev/null
+++ b/vendor/go.uber.org/atomic/bool.go
@@ -0,0 +1,81 @@
+// @generated Code generated by gen-atomicwrapper.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+)
+
+// Bool is an atomic type-safe wrapper for bool values.
+type Bool struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v Uint32
+}
+
+var _zeroBool bool
+
+// NewBool creates a new Bool.
+func NewBool(v bool) *Bool {
+	x := &Bool{}
+	if v != _zeroBool {
+		x.Store(v)
+	}
+	return x
+}
+
+// Load atomically loads the wrapped bool.
+func (x *Bool) Load() bool {
+	return truthy(x.v.Load())
+}
+
+// Store atomically stores the passed bool.
+func (x *Bool) Store(v bool) {
+	x.v.Store(boolToInt(v))
+}
+
+// CAS is an atomic compare-and-swap for bool values.
+func (x *Bool) CAS(o, n bool) bool {
+	return x.v.CAS(boolToInt(o), boolToInt(n))
+}
+
+// Swap atomically stores the given bool and returns the old
+// value.
+func (x *Bool) Swap(o bool) bool {
+	return truthy(x.v.Swap(boolToInt(o)))
+}
+
+// MarshalJSON encodes the wrapped bool into JSON.
+func (x *Bool) MarshalJSON() ([]byte, error) {
+	return json.Marshal(x.Load())
+}
+
+// UnmarshalJSON decodes a bool from JSON.
+func (x *Bool) UnmarshalJSON(b []byte) error {
+	var v bool
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	x.Store(v)
+	return nil
+}
diff --git a/vendor/go.uber.org/atomic/bool_ext.go b/vendor/go.uber.org/atomic/bool_ext.go
new file mode 100644
index 0000000..c7bf7a8
--- /dev/null
+++ b/vendor/go.uber.org/atomic/bool_ext.go
@@ -0,0 +1,53 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"strconv"
+)
+
+//go:generate bin/gen-atomicwrapper -name=Bool -type=bool -wrapped=Uint32 -pack=boolToInt -unpack=truthy -cas -swap -json -file=bool.go
+
+func truthy(n uint32) bool {
+	return n == 1
+}
+
+func boolToInt(b bool) uint32 {
+	if b {
+		return 1
+	}
+	return 0
+}
+
+// Toggle atomically negates the Boolean and returns the previous value.
+func (b *Bool) Toggle() bool {
+	for {
+		old := b.Load()
+		if b.CAS(old, !old) {
+			return old
+		}
+	}
+}
+
+// String encodes the wrapped value as a string.
+func (b *Bool) String() string {
+	return strconv.FormatBool(b.Load())
+}
diff --git a/vendor/go.uber.org/atomic/doc.go b/vendor/go.uber.org/atomic/doc.go
new file mode 100644
index 0000000..ae7390e
--- /dev/null
+++ b/vendor/go.uber.org/atomic/doc.go
@@ -0,0 +1,23 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+// Package atomic provides simple wrappers around numerics to enforce atomic
+// access.
+package atomic
diff --git a/vendor/go.uber.org/atomic/duration.go b/vendor/go.uber.org/atomic/duration.go
new file mode 100644
index 0000000..027cfcb
--- /dev/null
+++ b/vendor/go.uber.org/atomic/duration.go
@@ -0,0 +1,82 @@
+// @generated Code generated by gen-atomicwrapper.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"time"
+)
+
+// Duration is an atomic type-safe wrapper for time.Duration values.
+type Duration struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v Int64
+}
+
+var _zeroDuration time.Duration
+
+// NewDuration creates a new Duration.
+func NewDuration(v time.Duration) *Duration {
+	x := &Duration{}
+	if v != _zeroDuration {
+		x.Store(v)
+	}
+	return x
+}
+
+// Load atomically loads the wrapped time.Duration.
+func (x *Duration) Load() time.Duration {
+	return time.Duration(x.v.Load())
+}
+
+// Store atomically stores the passed time.Duration.
+func (x *Duration) Store(v time.Duration) {
+	x.v.Store(int64(v))
+}
+
+// CAS is an atomic compare-and-swap for time.Duration values.
+func (x *Duration) CAS(o, n time.Duration) bool {
+	return x.v.CAS(int64(o), int64(n))
+}
+
+// Swap atomically stores the given time.Duration and returns the old
+// value.
+func (x *Duration) Swap(o time.Duration) time.Duration {
+	return time.Duration(x.v.Swap(int64(o)))
+}
+
+// MarshalJSON encodes the wrapped time.Duration into JSON.
+func (x *Duration) MarshalJSON() ([]byte, error) {
+	return json.Marshal(x.Load())
+}
+
+// UnmarshalJSON decodes a time.Duration from JSON.
+func (x *Duration) UnmarshalJSON(b []byte) error {
+	var v time.Duration
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	x.Store(v)
+	return nil
+}
diff --git a/vendor/go.uber.org/atomic/duration_ext.go b/vendor/go.uber.org/atomic/duration_ext.go
new file mode 100644
index 0000000..6273b66
--- /dev/null
+++ b/vendor/go.uber.org/atomic/duration_ext.go
@@ -0,0 +1,40 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import "time"
+
+//go:generate bin/gen-atomicwrapper -name=Duration -type=time.Duration -wrapped=Int64 -pack=int64 -unpack=time.Duration -cas -swap -json -imports time -file=duration.go
+
+// Add atomically adds to the wrapped time.Duration and returns the new value.
+func (d *Duration) Add(n time.Duration) time.Duration {
+	return time.Duration(d.v.Add(int64(n)))
+}
+
+// Sub atomically subtracts from the wrapped time.Duration and returns the new value.
+func (d *Duration) Sub(n time.Duration) time.Duration {
+	return time.Duration(d.v.Sub(int64(n)))
+}
+
+// String encodes the wrapped value as a string.
+func (d *Duration) String() string {
+	return d.Load().String()
+}
diff --git a/vendor/go.uber.org/atomic/error.go b/vendor/go.uber.org/atomic/error.go
index 0489d19..a6166fb 100644
--- a/vendor/go.uber.org/atomic/error.go
+++ b/vendor/go.uber.org/atomic/error.go
@@ -1,4 +1,6 @@
-// Copyright (c) 2016 Uber Technologies, Inc.
+// @generated Code generated by gen-atomicwrapper.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
 //
 // Permission is hereby granted, free of charge, to any person obtaining a copy
 // of this software and associated documentation files (the "Software"), to deal
@@ -20,36 +22,30 @@
 
 package atomic
 
-// Error is an atomic type-safe wrapper around Value for errors
-type Error struct{ v Value }
+// Error is an atomic type-safe wrapper for error values.
+type Error struct {
+	_ nocmp // disallow non-atomic comparison
 
-// errorHolder is non-nil holder for error object.
-// atomic.Value panics on saving nil object, so err object needs to be
-// wrapped with valid object first.
-type errorHolder struct{ err error }
-
-// NewError creates new atomic error object
-func NewError(err error) *Error {
-	e := &Error{}
-	if err != nil {
-		e.Store(err)
-	}
-	return e
+	v Value
 }
 
-// Load atomically loads the wrapped error
-func (e *Error) Load() error {
-	v := e.v.Load()
-	if v == nil {
-		return nil
-	}
+var _zeroError error
 
-	eh := v.(errorHolder)
-	return eh.err
+// NewError creates a new Error.
+func NewError(v error) *Error {
+	x := &Error{}
+	if v != _zeroError {
+		x.Store(v)
+	}
+	return x
 }
 
-// Store atomically stores error.
-// NOTE: a holder object is allocated on each Store call.
-func (e *Error) Store(err error) {
-	e.v.Store(errorHolder{err: err})
+// Load atomically loads the wrapped error.
+func (x *Error) Load() error {
+	return unpackError(x.v.Load())
+}
+
+// Store atomically stores the passed error.
+func (x *Error) Store(v error) {
+	x.v.Store(packError(v))
 }
diff --git a/vendor/go.uber.org/atomic/error_ext.go b/vendor/go.uber.org/atomic/error_ext.go
new file mode 100644
index 0000000..ffe0be2
--- /dev/null
+++ b/vendor/go.uber.org/atomic/error_ext.go
@@ -0,0 +1,39 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+// atomic.Value panics on nil inputs, or if the underlying type changes.
+// Stabilize by always storing a custom struct that we control.
+
+//go:generate bin/gen-atomicwrapper -name=Error -type=error -wrapped=Value -pack=packError -unpack=unpackError -file=error.go
+
+type packedError struct{ Value error }
+
+func packError(v error) interface{} {
+	return packedError{v}
+}
+
+func unpackError(v interface{}) error {
+	if err, ok := v.(packedError); ok {
+		return err.Value
+	}
+	return nil
+}
diff --git a/vendor/go.uber.org/atomic/float64.go b/vendor/go.uber.org/atomic/float64.go
new file mode 100644
index 0000000..0719060
--- /dev/null
+++ b/vendor/go.uber.org/atomic/float64.go
@@ -0,0 +1,76 @@
+// @generated Code generated by gen-atomicwrapper.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"math"
+)
+
+// Float64 is an atomic type-safe wrapper for float64 values.
+type Float64 struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v Uint64
+}
+
+var _zeroFloat64 float64
+
+// NewFloat64 creates a new Float64.
+func NewFloat64(v float64) *Float64 {
+	x := &Float64{}
+	if v != _zeroFloat64 {
+		x.Store(v)
+	}
+	return x
+}
+
+// Load atomically loads the wrapped float64.
+func (x *Float64) Load() float64 {
+	return math.Float64frombits(x.v.Load())
+}
+
+// Store atomically stores the passed float64.
+func (x *Float64) Store(v float64) {
+	x.v.Store(math.Float64bits(v))
+}
+
+// CAS is an atomic compare-and-swap for float64 values.
+func (x *Float64) CAS(o, n float64) bool {
+	return x.v.CAS(math.Float64bits(o), math.Float64bits(n))
+}
+
+// MarshalJSON encodes the wrapped float64 into JSON.
+func (x *Float64) MarshalJSON() ([]byte, error) {
+	return json.Marshal(x.Load())
+}
+
+// UnmarshalJSON decodes a float64 from JSON.
+func (x *Float64) UnmarshalJSON(b []byte) error {
+	var v float64
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	x.Store(v)
+	return nil
+}
diff --git a/vendor/go.uber.org/atomic/float64_ext.go b/vendor/go.uber.org/atomic/float64_ext.go
new file mode 100644
index 0000000..927b1ad
--- /dev/null
+++ b/vendor/go.uber.org/atomic/float64_ext.go
@@ -0,0 +1,47 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import "strconv"
+
+//go:generate bin/gen-atomicwrapper -name=Float64 -type=float64 -wrapped=Uint64 -pack=math.Float64bits -unpack=math.Float64frombits -cas -json -imports math -file=float64.go
+
+// Add atomically adds to the wrapped float64 and returns the new value.
+func (f *Float64) Add(s float64) float64 {
+	for {
+		old := f.Load()
+		new := old + s
+		if f.CAS(old, new) {
+			return new
+		}
+	}
+}
+
+// Sub atomically subtracts from the wrapped float64 and returns the new value.
+func (f *Float64) Sub(s float64) float64 {
+	return f.Add(-s)
+}
+
+// String encodes the wrapped value as a string.
+func (f *Float64) String() string {
+	// 'g' is the behavior for floats with %v.
+	return strconv.FormatFloat(f.Load(), 'g', -1, 64)
+}
diff --git a/vendor/go.uber.org/atomic/gen.go b/vendor/go.uber.org/atomic/gen.go
new file mode 100644
index 0000000..50d6b24
--- /dev/null
+++ b/vendor/go.uber.org/atomic/gen.go
@@ -0,0 +1,26 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+//go:generate bin/gen-atomicint -name=Int32 -wrapped=int32 -file=int32.go
+//go:generate bin/gen-atomicint -name=Int64 -wrapped=int64 -file=int64.go
+//go:generate bin/gen-atomicint -name=Uint32 -wrapped=uint32 -unsigned -file=uint32.go
+//go:generate bin/gen-atomicint -name=Uint64 -wrapped=uint64 -unsigned -file=uint64.go
diff --git a/vendor/go.uber.org/atomic/glide.lock b/vendor/go.uber.org/atomic/glide.lock
deleted file mode 100644
index 3c72c59..0000000
--- a/vendor/go.uber.org/atomic/glide.lock
+++ /dev/null
@@ -1,17 +0,0 @@
-hash: f14d51408e3e0e4f73b34e4039484c78059cd7fc5f4996fdd73db20dc8d24f53
-updated: 2016-10-27T00:10:51.16960137-07:00
-imports: []
-testImports:
-- name: github.com/davecgh/go-spew
-  version: 5215b55f46b2b919f50a1df0eaa5886afe4e3b3d
-  subpackages:
-  - spew
-- name: github.com/pmezard/go-difflib
-  version: d8ed2627bdf02c080bf22230dbb337003b7aba2d
-  subpackages:
-  - difflib
-- name: github.com/stretchr/testify
-  version: d77da356e56a7428ad25149ca77381849a6a5232
-  subpackages:
-  - assert
-  - require
diff --git a/vendor/go.uber.org/atomic/glide.yaml b/vendor/go.uber.org/atomic/glide.yaml
deleted file mode 100644
index 4cf608e..0000000
--- a/vendor/go.uber.org/atomic/glide.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-package: go.uber.org/atomic
-testImport:
-- package: github.com/stretchr/testify
-  subpackages:
-  - assert
-  - require
diff --git a/vendor/go.uber.org/atomic/go.mod b/vendor/go.uber.org/atomic/go.mod
new file mode 100644
index 0000000..daa7599
--- /dev/null
+++ b/vendor/go.uber.org/atomic/go.mod
@@ -0,0 +1,8 @@
+module go.uber.org/atomic
+
+require (
+	github.com/davecgh/go-spew v1.1.1 // indirect
+	github.com/stretchr/testify v1.3.0
+)
+
+go 1.13
diff --git a/vendor/go.uber.org/atomic/go.sum b/vendor/go.uber.org/atomic/go.sum
new file mode 100644
index 0000000..4f76e62
--- /dev/null
+++ b/vendor/go.uber.org/atomic/go.sum
@@ -0,0 +1,9 @@
+github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
+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/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/go.uber.org/atomic/int32.go b/vendor/go.uber.org/atomic/int32.go
new file mode 100644
index 0000000..18ae564
--- /dev/null
+++ b/vendor/go.uber.org/atomic/int32.go
@@ -0,0 +1,102 @@
+// @generated Code generated by gen-atomicint.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"strconv"
+	"sync/atomic"
+)
+
+// Int32 is an atomic wrapper around int32.
+type Int32 struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v int32
+}
+
+// NewInt32 creates a new Int32.
+func NewInt32(i int32) *Int32 {
+	return &Int32{v: i}
+}
+
+// Load atomically loads the wrapped value.
+func (i *Int32) Load() int32 {
+	return atomic.LoadInt32(&i.v)
+}
+
+// Add atomically adds to the wrapped int32 and returns the new value.
+func (i *Int32) Add(n int32) int32 {
+	return atomic.AddInt32(&i.v, n)
+}
+
+// Sub atomically subtracts from the wrapped int32 and returns the new value.
+func (i *Int32) Sub(n int32) int32 {
+	return atomic.AddInt32(&i.v, -n)
+}
+
+// Inc atomically increments the wrapped int32 and returns the new value.
+func (i *Int32) Inc() int32 {
+	return i.Add(1)
+}
+
+// Dec atomically decrements the wrapped int32 and returns the new value.
+func (i *Int32) Dec() int32 {
+	return i.Sub(1)
+}
+
+// CAS is an atomic compare-and-swap.
+func (i *Int32) CAS(old, new int32) bool {
+	return atomic.CompareAndSwapInt32(&i.v, old, new)
+}
+
+// Store atomically stores the passed value.
+func (i *Int32) Store(n int32) {
+	atomic.StoreInt32(&i.v, n)
+}
+
+// Swap atomically swaps the wrapped int32 and returns the old value.
+func (i *Int32) Swap(n int32) int32 {
+	return atomic.SwapInt32(&i.v, n)
+}
+
+// MarshalJSON encodes the wrapped int32 into JSON.
+func (i *Int32) MarshalJSON() ([]byte, error) {
+	return json.Marshal(i.Load())
+}
+
+// UnmarshalJSON decodes JSON into the wrapped int32.
+func (i *Int32) UnmarshalJSON(b []byte) error {
+	var v int32
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	i.Store(v)
+	return nil
+}
+
+// String encodes the wrapped value as a string.
+func (i *Int32) String() string {
+	v := i.Load()
+	return strconv.FormatInt(int64(v), 10)
+}
diff --git a/vendor/go.uber.org/atomic/int64.go b/vendor/go.uber.org/atomic/int64.go
new file mode 100644
index 0000000..2bcbbfa
--- /dev/null
+++ b/vendor/go.uber.org/atomic/int64.go
@@ -0,0 +1,102 @@
+// @generated Code generated by gen-atomicint.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"strconv"
+	"sync/atomic"
+)
+
+// Int64 is an atomic wrapper around int64.
+type Int64 struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v int64
+}
+
+// NewInt64 creates a new Int64.
+func NewInt64(i int64) *Int64 {
+	return &Int64{v: i}
+}
+
+// Load atomically loads the wrapped value.
+func (i *Int64) Load() int64 {
+	return atomic.LoadInt64(&i.v)
+}
+
+// Add atomically adds to the wrapped int64 and returns the new value.
+func (i *Int64) Add(n int64) int64 {
+	return atomic.AddInt64(&i.v, n)
+}
+
+// Sub atomically subtracts from the wrapped int64 and returns the new value.
+func (i *Int64) Sub(n int64) int64 {
+	return atomic.AddInt64(&i.v, -n)
+}
+
+// Inc atomically increments the wrapped int64 and returns the new value.
+func (i *Int64) Inc() int64 {
+	return i.Add(1)
+}
+
+// Dec atomically decrements the wrapped int64 and returns the new value.
+func (i *Int64) Dec() int64 {
+	return i.Sub(1)
+}
+
+// CAS is an atomic compare-and-swap.
+func (i *Int64) CAS(old, new int64) bool {
+	return atomic.CompareAndSwapInt64(&i.v, old, new)
+}
+
+// Store atomically stores the passed value.
+func (i *Int64) Store(n int64) {
+	atomic.StoreInt64(&i.v, n)
+}
+
+// Swap atomically swaps the wrapped int64 and returns the old value.
+func (i *Int64) Swap(n int64) int64 {
+	return atomic.SwapInt64(&i.v, n)
+}
+
+// MarshalJSON encodes the wrapped int64 into JSON.
+func (i *Int64) MarshalJSON() ([]byte, error) {
+	return json.Marshal(i.Load())
+}
+
+// UnmarshalJSON decodes JSON into the wrapped int64.
+func (i *Int64) UnmarshalJSON(b []byte) error {
+	var v int64
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	i.Store(v)
+	return nil
+}
+
+// String encodes the wrapped value as a string.
+func (i *Int64) String() string {
+	v := i.Load()
+	return strconv.FormatInt(int64(v), 10)
+}
diff --git a/vendor/go.uber.org/atomic/nocmp.go b/vendor/go.uber.org/atomic/nocmp.go
new file mode 100644
index 0000000..a8201cb
--- /dev/null
+++ b/vendor/go.uber.org/atomic/nocmp.go
@@ -0,0 +1,35 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+// nocmp is an uncomparable struct. Embed this inside another struct to make
+// it uncomparable.
+//
+//  type Foo struct {
+//    nocmp
+//    // ...
+//  }
+//
+// This DOES NOT:
+//
+//  - Disallow shallow copies of structs
+//  - Disallow comparison of pointers to uncomparable structs
+type nocmp [0]func()
diff --git a/vendor/go.uber.org/atomic/string.go b/vendor/go.uber.org/atomic/string.go
index ede8136..225b7a2 100644
--- a/vendor/go.uber.org/atomic/string.go
+++ b/vendor/go.uber.org/atomic/string.go
@@ -1,4 +1,6 @@
-// Copyright (c) 2016 Uber Technologies, Inc.
+// @generated Code generated by gen-atomicwrapper.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
 //
 // Permission is hereby granted, free of charge, to any person obtaining a copy
 // of this software and associated documentation files (the "Software"), to deal
@@ -20,30 +22,33 @@
 
 package atomic
 
-// String is an atomic type-safe wrapper around Value for strings.
-type String struct{ v Value }
+// String is an atomic type-safe wrapper for string values.
+type String struct {
+	_ nocmp // disallow non-atomic comparison
 
-// NewString creates a String.
-func NewString(str string) *String {
-	s := &String{}
-	if str != "" {
-		s.Store(str)
+	v Value
+}
+
+var _zeroString string
+
+// NewString creates a new String.
+func NewString(v string) *String {
+	x := &String{}
+	if v != _zeroString {
+		x.Store(v)
 	}
-	return s
+	return x
 }
 
 // Load atomically loads the wrapped string.
-func (s *String) Load() string {
-	v := s.v.Load()
-	if v == nil {
-		return ""
+func (x *String) Load() string {
+	if v := x.v.Load(); v != nil {
+		return v.(string)
 	}
-	return v.(string)
+	return _zeroString
 }
 
 // Store atomically stores the passed string.
-// Note: Converting the string to an interface{} to store in the Value
-// requires an allocation.
-func (s *String) Store(str string) {
-	s.v.Store(str)
+func (x *String) Store(v string) {
+	x.v.Store(v)
 }
diff --git a/vendor/go.uber.org/atomic/string_ext.go b/vendor/go.uber.org/atomic/string_ext.go
new file mode 100644
index 0000000..3a95582
--- /dev/null
+++ b/vendor/go.uber.org/atomic/string_ext.go
@@ -0,0 +1,43 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+//go:generate bin/gen-atomicwrapper -name=String -type=string -wrapped=Value -file=string.go
+
+// String returns the wrapped value.
+func (s *String) String() string {
+	return s.Load()
+}
+
+// MarshalText encodes the wrapped string into a textual form.
+//
+// This makes it encodable as JSON, YAML, XML, and more.
+func (s *String) MarshalText() ([]byte, error) {
+	return []byte(s.Load()), nil
+}
+
+// UnmarshalText decodes text and replaces the wrapped string with it.
+//
+// This makes it decodable from JSON, YAML, XML, and more.
+func (s *String) UnmarshalText(b []byte) error {
+	s.Store(string(b))
+	return nil
+}
diff --git a/vendor/go.uber.org/atomic/uint32.go b/vendor/go.uber.org/atomic/uint32.go
new file mode 100644
index 0000000..a973aba
--- /dev/null
+++ b/vendor/go.uber.org/atomic/uint32.go
@@ -0,0 +1,102 @@
+// @generated Code generated by gen-atomicint.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"strconv"
+	"sync/atomic"
+)
+
+// Uint32 is an atomic wrapper around uint32.
+type Uint32 struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v uint32
+}
+
+// NewUint32 creates a new Uint32.
+func NewUint32(i uint32) *Uint32 {
+	return &Uint32{v: i}
+}
+
+// Load atomically loads the wrapped value.
+func (i *Uint32) Load() uint32 {
+	return atomic.LoadUint32(&i.v)
+}
+
+// Add atomically adds to the wrapped uint32 and returns the new value.
+func (i *Uint32) Add(n uint32) uint32 {
+	return atomic.AddUint32(&i.v, n)
+}
+
+// Sub atomically subtracts from the wrapped uint32 and returns the new value.
+func (i *Uint32) Sub(n uint32) uint32 {
+	return atomic.AddUint32(&i.v, ^(n - 1))
+}
+
+// Inc atomically increments the wrapped uint32 and returns the new value.
+func (i *Uint32) Inc() uint32 {
+	return i.Add(1)
+}
+
+// Dec atomically decrements the wrapped uint32 and returns the new value.
+func (i *Uint32) Dec() uint32 {
+	return i.Sub(1)
+}
+
+// CAS is an atomic compare-and-swap.
+func (i *Uint32) CAS(old, new uint32) bool {
+	return atomic.CompareAndSwapUint32(&i.v, old, new)
+}
+
+// Store atomically stores the passed value.
+func (i *Uint32) Store(n uint32) {
+	atomic.StoreUint32(&i.v, n)
+}
+
+// Swap atomically swaps the wrapped uint32 and returns the old value.
+func (i *Uint32) Swap(n uint32) uint32 {
+	return atomic.SwapUint32(&i.v, n)
+}
+
+// MarshalJSON encodes the wrapped uint32 into JSON.
+func (i *Uint32) MarshalJSON() ([]byte, error) {
+	return json.Marshal(i.Load())
+}
+
+// UnmarshalJSON decodes JSON into the wrapped uint32.
+func (i *Uint32) UnmarshalJSON(b []byte) error {
+	var v uint32
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	i.Store(v)
+	return nil
+}
+
+// String encodes the wrapped value as a string.
+func (i *Uint32) String() string {
+	v := i.Load()
+	return strconv.FormatUint(uint64(v), 10)
+}
diff --git a/vendor/go.uber.org/atomic/uint64.go b/vendor/go.uber.org/atomic/uint64.go
new file mode 100644
index 0000000..3b6c71f
--- /dev/null
+++ b/vendor/go.uber.org/atomic/uint64.go
@@ -0,0 +1,102 @@
+// @generated Code generated by gen-atomicint.
+
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import (
+	"encoding/json"
+	"strconv"
+	"sync/atomic"
+)
+
+// Uint64 is an atomic wrapper around uint64.
+type Uint64 struct {
+	_ nocmp // disallow non-atomic comparison
+
+	v uint64
+}
+
+// NewUint64 creates a new Uint64.
+func NewUint64(i uint64) *Uint64 {
+	return &Uint64{v: i}
+}
+
+// Load atomically loads the wrapped value.
+func (i *Uint64) Load() uint64 {
+	return atomic.LoadUint64(&i.v)
+}
+
+// Add atomically adds to the wrapped uint64 and returns the new value.
+func (i *Uint64) Add(n uint64) uint64 {
+	return atomic.AddUint64(&i.v, n)
+}
+
+// Sub atomically subtracts from the wrapped uint64 and returns the new value.
+func (i *Uint64) Sub(n uint64) uint64 {
+	return atomic.AddUint64(&i.v, ^(n - 1))
+}
+
+// Inc atomically increments the wrapped uint64 and returns the new value.
+func (i *Uint64) Inc() uint64 {
+	return i.Add(1)
+}
+
+// Dec atomically decrements the wrapped uint64 and returns the new value.
+func (i *Uint64) Dec() uint64 {
+	return i.Sub(1)
+}
+
+// CAS is an atomic compare-and-swap.
+func (i *Uint64) CAS(old, new uint64) bool {
+	return atomic.CompareAndSwapUint64(&i.v, old, new)
+}
+
+// Store atomically stores the passed value.
+func (i *Uint64) Store(n uint64) {
+	atomic.StoreUint64(&i.v, n)
+}
+
+// Swap atomically swaps the wrapped uint64 and returns the old value.
+func (i *Uint64) Swap(n uint64) uint64 {
+	return atomic.SwapUint64(&i.v, n)
+}
+
+// MarshalJSON encodes the wrapped uint64 into JSON.
+func (i *Uint64) MarshalJSON() ([]byte, error) {
+	return json.Marshal(i.Load())
+}
+
+// UnmarshalJSON decodes JSON into the wrapped uint64.
+func (i *Uint64) UnmarshalJSON(b []byte) error {
+	var v uint64
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+	i.Store(v)
+	return nil
+}
+
+// String encodes the wrapped value as a string.
+func (i *Uint64) String() string {
+	v := i.Load()
+	return strconv.FormatUint(uint64(v), 10)
+}
diff --git a/vendor/go.uber.org/atomic/value.go b/vendor/go.uber.org/atomic/value.go
new file mode 100644
index 0000000..671f3a3
--- /dev/null
+++ b/vendor/go.uber.org/atomic/value.go
@@ -0,0 +1,31 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package atomic
+
+import "sync/atomic"
+
+// Value shadows the type of the same name from sync/atomic
+// https://godoc.org/sync/atomic#Value
+type Value struct {
+	atomic.Value
+
+	_ nocmp // disallow non-atomic comparison
+}
diff --git a/vendor/go.uber.org/multierr/.gitignore b/vendor/go.uber.org/multierr/.gitignore
index 61ead86..b9a05e3 100644
--- a/vendor/go.uber.org/multierr/.gitignore
+++ b/vendor/go.uber.org/multierr/.gitignore
@@ -1 +1,4 @@
 /vendor
+cover.html
+cover.out
+/bin
diff --git a/vendor/go.uber.org/multierr/.travis.yml b/vendor/go.uber.org/multierr/.travis.yml
index a6412b7..8636ab4 100644
--- a/vendor/go.uber.org/multierr/.travis.yml
+++ b/vendor/go.uber.org/multierr/.travis.yml
@@ -4,30 +4,20 @@
 
 env:
   global:
-    - GO15VENDOREXPERIMENT=1
+    - GO111MODULE=on
 
 go:
-  - 1.11.x
-  - 1.12.x
-  - 1.13.x
-
-cache:
-  directories:
-    - vendor
+  - oldstable
+  - stable
 
 before_install:
 - go version
 
-install:
-- |
-  set -e
-  make install_ci
-
 script:
 - |
   set -e
   make lint
-  make test_ci
+  make cover
 
 after_success:
 - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/go.uber.org/multierr/CHANGELOG.md b/vendor/go.uber.org/multierr/CHANGELOG.md
index f1b852c..6f1db9e 100644
--- a/vendor/go.uber.org/multierr/CHANGELOG.md
+++ b/vendor/go.uber.org/multierr/CHANGELOG.md
@@ -1,6 +1,31 @@
 Releases
 ========
 
+v1.6.0 (2020-09-14)
+===================
+
+-   Actually drop library dependency on development-time tooling.
+
+
+v1.5.0 (2020-02-24)
+===================
+
+-   Drop library dependency on development-time tooling.
+
+
+v1.4.0 (2019-11-04)
+===================
+
+-   Add `AppendInto` function to more ergonomically build errors inside a
+    loop.
+
+
+v1.3.0 (2019-10-29)
+===================
+
+-   Switch to Go modules.
+
+
 v1.2.0 (2019-09-26)
 ===================
 
diff --git a/vendor/go.uber.org/multierr/Makefile b/vendor/go.uber.org/multierr/Makefile
index b4bf73d..3160044 100644
--- a/vendor/go.uber.org/multierr/Makefile
+++ b/vendor/go.uber.org/multierr/Makefile
@@ -1,23 +1,17 @@
-export GO15VENDOREXPERIMENT=1
-
-PACKAGES := $(shell glide nv)
+# Directory to put `go install`ed binaries in.
+export GOBIN ?= $(shell pwd)/bin
 
 GO_FILES := $(shell \
 	find . '(' -path '*/.*' -o -path './vendor' ')' -prune \
 	-o -name '*.go' -print | cut -b3-)
 
-.PHONY: install
-install:
-	glide --version || go get github.com/Masterminds/glide
-	glide install
-
 .PHONY: build
 build:
-	go build -i $(PACKAGES)
+	go build ./...
 
 .PHONY: test
 test:
-	go test -cover -race $(PACKAGES)
+	go test -race ./...
 
 .PHONY: gofmt
 gofmt:
@@ -25,50 +19,24 @@
 	@gofmt -e -s -l $(GO_FILES) > $(FMT_LOG) || true
 	@[ ! -s "$(FMT_LOG)" ] || (echo "gofmt failed:" | cat - $(FMT_LOG) && false)
 
-.PHONY: govet
-govet:
-	$(eval VET_LOG := $(shell mktemp -t govet.XXXXX))
-	@go vet $(PACKAGES) 2>&1 \
-		| grep -v '^exit status' > $(VET_LOG) || true
-	@[ ! -s "$(VET_LOG)" ] || (echo "govet failed:" | cat - $(VET_LOG) && false)
-
 .PHONY: golint
 golint:
-	@go get golang.org/x/lint/golint
-	$(eval LINT_LOG := $(shell mktemp -t golint.XXXXX))
-	@cat /dev/null > $(LINT_LOG)
-	@$(foreach pkg, $(PACKAGES), golint $(pkg) >> $(LINT_LOG) || true;)
-	@[ ! -s "$(LINT_LOG)" ] || (echo "golint failed:" | cat - $(LINT_LOG) && false)
+	@cd tools && go install golang.org/x/lint/golint
+	@$(GOBIN)/golint ./...
 
 .PHONY: staticcheck
 staticcheck:
-	@go get honnef.co/go/tools/cmd/staticcheck
-	$(eval STATICCHECK_LOG := $(shell mktemp -t staticcheck.XXXXX))
-	@staticcheck $(PACKAGES) 2>&1 > $(STATICCHECK_LOG) || true
-	@[ ! -s "$(STATICCHECK_LOG)" ] || (echo "staticcheck failed:" | cat - $(STATICCHECK_LOG) && false)
+	@cd tools && go install honnef.co/go/tools/cmd/staticcheck
+	@$(GOBIN)/staticcheck ./...
 
 .PHONY: lint
-lint: gofmt govet golint staticcheck
+lint: gofmt golint staticcheck
 
 .PHONY: cover
 cover:
-	./scripts/cover.sh $(shell go list $(PACKAGES))
+	go test -coverprofile=cover.out -coverpkg=./... -v ./...
 	go tool cover -html=cover.out -o cover.html
 
 update-license:
-	@go get go.uber.org/tools/update-license
-	@update-license \
-		$(shell go list -json $(PACKAGES) | \
-			jq -r '.Dir + "/" + (.GoFiles | .[])')
-
-##############################################################################
-
-.PHONY: install_ci
-install_ci: install
-	go get github.com/wadey/gocovmerge
-	go get github.com/mattn/goveralls
-	go get golang.org/x/tools/cmd/cover
-
-.PHONY: test_ci
-test_ci: install_ci
-	./scripts/cover.sh $(shell go list $(PACKAGES))
+	@cd tools && go install go.uber.org/tools/update-license
+	@$(GOBIN)/update-license $(GO_FILES)
diff --git a/vendor/go.uber.org/multierr/error.go b/vendor/go.uber.org/multierr/error.go
index d4be183..5c9b67d 100644
--- a/vendor/go.uber.org/multierr/error.go
+++ b/vendor/go.uber.org/multierr/error.go
@@ -1,4 +1,4 @@
-// Copyright (c) 2017 Uber Technologies, Inc.
+// Copyright (c) 2019 Uber Technologies, Inc.
 //
 // Permission is hereby granted, free of charge, to any person obtaining a copy
 // of this software and associated documentation files (the "Software"), to deal
@@ -54,7 +54,7 @@
 //
 // 	errors := multierr.Errors(err)
 // 	if len(errors) > 0 {
-// 		fmt.Println("The following errors occurred:")
+// 		fmt.Println("The following errors occurred:", errors)
 // 	}
 //
 // Advanced Usage
@@ -130,7 +130,7 @@
 }
 
 // Errors returns a slice containing zero or more errors that the supplied
-// error is composed of. If the error is nil, the returned slice is empty.
+// error is composed of. If the error is nil, a nil slice is returned.
 //
 // 	err := multierr.Append(r.Close(), w.Close())
 // 	errors := multierr.Errors(err)
@@ -397,3 +397,53 @@
 	errors := [2]error{left, right}
 	return fromSlice(errors[0:])
 }
+
+// AppendInto appends an error into the destination of an error pointer and
+// returns whether the error being appended was non-nil.
+//
+// 	var err error
+// 	multierr.AppendInto(&err, r.Close())
+// 	multierr.AppendInto(&err, w.Close())
+//
+// The above is equivalent to,
+//
+// 	err := multierr.Append(r.Close(), w.Close())
+//
+// As AppendInto reports whether the provided error was non-nil, it may be
+// used to build a multierr error in a loop more ergonomically. For example:
+//
+// 	var err error
+// 	for line := range lines {
+// 		var item Item
+// 		if multierr.AppendInto(&err, parse(line, &item)) {
+// 			continue
+// 		}
+// 		items = append(items, item)
+// 	}
+//
+// Compare this with a verison that relies solely on Append:
+//
+// 	var err error
+// 	for line := range lines {
+// 		var item Item
+// 		if parseErr := parse(line, &item); parseErr != nil {
+// 			err = multierr.Append(err, parseErr)
+// 			continue
+// 		}
+// 		items = append(items, item)
+// 	}
+func AppendInto(into *error, err error) (errored bool) {
+	if into == nil {
+		// We panic if 'into' is nil. This is not documented above
+		// because suggesting that the pointer must be non-nil may
+		// confuse users into thinking that the error that it points
+		// to must be non-nil.
+		panic("misuse of multierr.AppendInto: into pointer must not be nil")
+	}
+
+	if err == nil {
+		return false
+	}
+	*into = Append(*into, err)
+	return true
+}
diff --git a/vendor/go.uber.org/multierr/glide.lock b/vendor/go.uber.org/multierr/glide.lock
deleted file mode 100644
index f9ea94c..0000000
--- a/vendor/go.uber.org/multierr/glide.lock
+++ /dev/null
@@ -1,19 +0,0 @@
-hash: b53b5e9a84b9cb3cc4b2d0499e23da2feca1eec318ce9bb717ecf35bf24bf221
-updated: 2017-04-10T13:34:45.671678062-07:00
-imports:
-- name: go.uber.org/atomic
-  version: 3b8db5e93c4c02efbc313e17b2e796b0914a01fb
-testImports:
-- name: github.com/davecgh/go-spew
-  version: 6d212800a42e8ab5c146b8ace3490ee17e5225f9
-  subpackages:
-  - spew
-- name: github.com/pmezard/go-difflib
-  version: d8ed2627bdf02c080bf22230dbb337003b7aba2d
-  subpackages:
-  - difflib
-- name: github.com/stretchr/testify
-  version: 69483b4bd14f5845b5a1e55bca19e954e827f1d0
-  subpackages:
-  - assert
-  - require
diff --git a/vendor/go.uber.org/multierr/go.mod b/vendor/go.uber.org/multierr/go.mod
new file mode 100644
index 0000000..ff8bdf9
--- /dev/null
+++ b/vendor/go.uber.org/multierr/go.mod
@@ -0,0 +1,8 @@
+module go.uber.org/multierr
+
+go 1.12
+
+require (
+	github.com/stretchr/testify v1.3.0
+	go.uber.org/atomic v1.7.0
+)
diff --git a/vendor/go.uber.org/multierr/go.sum b/vendor/go.uber.org/multierr/go.sum
new file mode 100644
index 0000000..ecfc286
--- /dev/null
+++ b/vendor/go.uber.org/multierr/go.sum
@@ -0,0 +1,11 @@
+github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
+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/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=
+go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
+go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
diff --git a/vendor/go.uber.org/zap/.gitignore b/vendor/go.uber.org/zap/.gitignore
index 08fbde6..da9d9d0 100644
--- a/vendor/go.uber.org/zap/.gitignore
+++ b/vendor/go.uber.org/zap/.gitignore
@@ -26,3 +26,7 @@
 *.pprof
 *.out
 *.log
+
+/bin
+cover.out
+cover.html
diff --git a/vendor/go.uber.org/zap/.readme.tmpl b/vendor/go.uber.org/zap/.readme.tmpl
index c6440db..3154a1e 100644
--- a/vendor/go.uber.org/zap/.readme.tmpl
+++ b/vendor/go.uber.org/zap/.readme.tmpl
@@ -100,9 +100,10 @@
 
 [doc-img]: https://godoc.org/go.uber.org/zap?status.svg
 [doc]: https://godoc.org/go.uber.org/zap
-[ci-img]: https://travis-ci.org/uber-go/zap.svg?branch=master
-[ci]: https://travis-ci.org/uber-go/zap
+[ci-img]: https://travis-ci.com/uber-go/zap.svg?branch=master
+[ci]: https://travis-ci.com/uber-go/zap
 [cov-img]: https://codecov.io/gh/uber-go/zap/branch/master/graph/badge.svg
 [cov]: https://codecov.io/gh/uber-go/zap
 [benchmarking suite]: https://github.com/uber-go/zap/tree/master/benchmarks
 [glide.lock]: https://github.com/uber-go/zap/blob/master/glide.lock
+
diff --git a/vendor/go.uber.org/zap/.travis.yml b/vendor/go.uber.org/zap/.travis.yml
deleted file mode 100644
index ada5ebd..0000000
--- a/vendor/go.uber.org/zap/.travis.yml
+++ /dev/null
@@ -1,21 +0,0 @@
-language: go
-sudo: false
-go:
-  - 1.11.x
-  - 1.12.x
-go_import_path: go.uber.org/zap
-env:
-  global:
-    - TEST_TIMEOUT_SCALE=10
-cache:
-  directories:
-    - vendor
-install:
-  - make dependencies
-script:
-  - make lint
-  - make test
-  - make bench
-after_success:
-  - make cover
-  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/go.uber.org/zap/CHANGELOG.md b/vendor/go.uber.org/zap/CHANGELOG.md
index 28d1067..6c32101 100644
--- a/vendor/go.uber.org/zap/CHANGELOG.md
+++ b/vendor/go.uber.org/zap/CHANGELOG.md
@@ -1,5 +1,136 @@
 # Changelog
 
+## 1.18.1 (28 Jun 2021)
+
+Bugfixes:
+* [#974][]: Fix nil dereference in logger constructed by `zap.NewNop`.
+
+[#974]: https://github.com/uber-go/zap/pull/974
+
+## 1.18.0 (28 Jun 2021)
+
+Enhancements:
+* [#961][]: Add `zapcore.BufferedWriteSyncer`, a new `WriteSyncer` that buffers
+  messages in-memory and flushes them periodically.
+* [#971][]: Add `zapio.Writer` to use a Zap logger as an `io.Writer`.
+* [#897][]: Add `zap.WithClock` option to control the source of time via the
+  new `zapcore.Clock` interface.
+* [#949][]: Avoid panicking in `zap.SugaredLogger` when arguments of `*w`
+  methods don't match expectations.
+* [#943][]: Add support for filtering by level or arbitrary matcher function to
+  `zaptest/observer`.
+* [#691][]: Comply with `io.StringWriter` and `io.ByteWriter` in Zap's
+  `buffer.Buffer`.
+
+Thanks to @atrn0, @ernado, @heyanfu, @hnlq715, @zchee
+for their contributions to this release.
+
+[#691]: https://github.com/uber-go/zap/pull/691
+[#897]: https://github.com/uber-go/zap/pull/897
+[#943]: https://github.com/uber-go/zap/pull/943
+[#949]: https://github.com/uber-go/zap/pull/949
+[#961]: https://github.com/uber-go/zap/pull/961
+[#971]: https://github.com/uber-go/zap/pull/971
+
+## 1.17.0 (25 May 2021)
+
+Bugfixes:
+* [#867][]: Encode `<nil>` for nil `error` instead of a panic.
+* [#931][], [#936][]: Update minimum version constraints to address
+  vulnerabilities in dependencies.
+
+Enhancements:
+* [#865][]: Improve alignment of fields of the Logger struct, reducing its
+  size from 96 to 80 bytes.
+* [#881][]: Support `grpclog.LoggerV2` in zapgrpc.
+* [#903][]: Support URL-encoded POST requests to the AtomicLevel HTTP handler
+  with the `application/x-www-form-urlencoded` content type.
+* [#912][]: Support multi-field encoding with `zap.Inline`.
+* [#913][]: Speed up SugaredLogger for calls with a single string.
+* [#928][]: Add support for filtering by field name to `zaptest/observer`.
+
+Thanks to @ash2k, @FMLS, @jimmystewpot, @Oncilla, @tsoslow, @tylitianrui, @withshubh, and @wziww for their contributions to this release.
+
+## 1.16.0 (1 Sep 2020)
+
+Bugfixes:
+* [#828][]: Fix missing newline in IncreaseLevel error messages.
+* [#835][]: Fix panic in JSON encoder when encoding times or durations
+  without specifying a time or duration encoder.
+* [#843][]: Honor CallerSkip when taking stack traces.
+* [#862][]: Fix the default file permissions to use `0666` and rely on the umask instead.
+* [#854][]: Encode `<nil>` for nil `Stringer` instead of a panic error log.
+
+Enhancements:
+* [#629][]: Added `zapcore.TimeEncoderOfLayout` to easily create time encoders
+  for custom layouts.
+* [#697][]: Added support for a configurable delimiter in the console encoder.
+* [#852][]: Optimize console encoder by pooling the underlying JSON encoder.
+* [#844][]: Add ability to include the calling function as part of logs.
+* [#843][]: Add `StackSkip` for including truncated stacks as a field.
+* [#861][]: Add options to customize Fatal behaviour for better testability.
+
+Thanks to @SteelPhase, @tmshn, @lixingwang, @wyxloading, @moul, @segevfiner, @andy-retailnext and @jcorbin for their contributions to this release.
+
+## 1.15.0 (23 Apr 2020)
+
+Bugfixes:
+* [#804][]: Fix handling of `Time` values out of `UnixNano` range.
+* [#812][]: Fix `IncreaseLevel` being reset after a call to `With`.
+
+Enhancements:
+* [#806][]: Add `WithCaller` option to supersede the `AddCaller` option. This
+  allows disabling annotation of log entries with caller information if
+  previously enabled with `AddCaller`.
+* [#813][]: Deprecate `NewSampler` constructor in favor of
+  `NewSamplerWithOptions` which supports a `SamplerHook` option. This option
+   adds support for monitoring sampling decisions through a hook.
+
+Thanks to @danielbprice for their contributions to this release.
+
+## 1.14.1 (14 Mar 2020)
+
+Bugfixes:
+* [#791][]: Fix panic on attempting to build a logger with an invalid Config.
+* [#795][]: Vendoring Zap with `go mod vendor` no longer includes Zap's
+  development-time dependencies.
+* [#799][]: Fix issue introduced in 1.14.0 that caused invalid JSON output to
+  be generated for arrays of `time.Time` objects when using string-based time
+  formats.
+
+Thanks to @YashishDua for their contributions to this release.
+
+## 1.14.0 (20 Feb 2020)
+
+Enhancements:
+* [#771][]: Optimize calls for disabled log levels.
+* [#773][]: Add millisecond duration encoder.
+* [#775][]: Add option to increase the level of a logger.
+* [#786][]: Optimize time formatters using `Time.AppendFormat` where possible.
+
+Thanks to @caibirdme for their contributions to this release.
+
+## 1.13.0 (13 Nov 2019)
+
+Enhancements:
+* [#758][]: Add `Intp`, `Stringp`, and other similar `*p` field constructors
+  to log pointers to primitives with support for `nil` values.
+
+Thanks to @jbizzle for their contributions to this release.
+
+## 1.12.0 (29 Oct 2019)
+
+Enhancements:
+* [#751][]: Migrate to Go modules.
+
+## 1.11.0 (21 Oct 2019)
+
+Enhancements:
+* [#725][]: Add `zapcore.OmitKey` to omit keys in an `EncoderConfig`.
+* [#736][]: Add `RFC3339` and `RFC3339Nano` time encoders.
+
+Thanks to @juicemia, @uhthomas for their contributions to this release.
+
 ## 1.10.0 (29 Apr 2019)
 
 Bugfixes:
@@ -325,3 +456,37 @@
 [#610]: https://github.com/uber-go/zap/pull/610
 [#675]: https://github.com/uber-go/zap/pull/675
 [#704]: https://github.com/uber-go/zap/pull/704
+[#725]: https://github.com/uber-go/zap/pull/725
+[#736]: https://github.com/uber-go/zap/pull/736
+[#751]: https://github.com/uber-go/zap/pull/751
+[#758]: https://github.com/uber-go/zap/pull/758
+[#771]: https://github.com/uber-go/zap/pull/771
+[#773]: https://github.com/uber-go/zap/pull/773
+[#775]: https://github.com/uber-go/zap/pull/775
+[#786]: https://github.com/uber-go/zap/pull/786
+[#791]: https://github.com/uber-go/zap/pull/791
+[#795]: https://github.com/uber-go/zap/pull/795
+[#799]: https://github.com/uber-go/zap/pull/799
+[#804]: https://github.com/uber-go/zap/pull/804
+[#812]: https://github.com/uber-go/zap/pull/812
+[#806]: https://github.com/uber-go/zap/pull/806
+[#813]: https://github.com/uber-go/zap/pull/813
+[#629]: https://github.com/uber-go/zap/pull/629
+[#697]: https://github.com/uber-go/zap/pull/697
+[#828]: https://github.com/uber-go/zap/pull/828
+[#835]: https://github.com/uber-go/zap/pull/835
+[#843]: https://github.com/uber-go/zap/pull/843
+[#844]: https://github.com/uber-go/zap/pull/844
+[#852]: https://github.com/uber-go/zap/pull/852
+[#854]: https://github.com/uber-go/zap/pull/854
+[#861]: https://github.com/uber-go/zap/pull/861
+[#862]: https://github.com/uber-go/zap/pull/862
+[#865]: https://github.com/uber-go/zap/pull/865
+[#867]: https://github.com/uber-go/zap/pull/867
+[#881]: https://github.com/uber-go/zap/pull/881
+[#903]: https://github.com/uber-go/zap/pull/903
+[#912]: https://github.com/uber-go/zap/pull/912
+[#913]: https://github.com/uber-go/zap/pull/913
+[#928]: https://github.com/uber-go/zap/pull/928
+[#931]: https://github.com/uber-go/zap/pull/931
+[#936]: https://github.com/uber-go/zap/pull/936
diff --git a/vendor/go.uber.org/zap/CONTRIBUTING.md b/vendor/go.uber.org/zap/CONTRIBUTING.md
index 9454bba..5cd9656 100644
--- a/vendor/go.uber.org/zap/CONTRIBUTING.md
+++ b/vendor/go.uber.org/zap/CONTRIBUTING.md
@@ -25,12 +25,6 @@
 git fetch upstream
 ```
 
-Install zap's dependencies:
-
-```
-make dependencies
-```
-
 Make sure that the tests and the linters pass:
 
 ```
diff --git a/vendor/go.uber.org/zap/FAQ.md b/vendor/go.uber.org/zap/FAQ.md
index 4256d35..b183b20 100644
--- a/vendor/go.uber.org/zap/FAQ.md
+++ b/vendor/go.uber.org/zap/FAQ.md
@@ -27,6 +27,13 @@
 Your applications should define and depend upon an interface that includes
 just the methods you use.
 
+### Why are some of my logs missing?
+
+Logs are dropped intentionally by zap when sampling is enabled. The production
+configuration (as returned by `NewProductionConfig()` enables sampling which will
+cause repeated logs within a second to be sampled. See more details on why sampling
+is enabled in [Why sample application logs](https://github.com/uber-go/zap/blob/master/FAQ.md#why-sample-application-logs).
+
 ### Why sample application logs?
 
 Applications often experience runs of errors, either because of a bug or
@@ -149,6 +156,8 @@
 | `github.com/tchap/zapext` | Sentry, syslog |
 | `github.com/fgrosse/zaptest` | Ginkgo |
 | `github.com/blendle/zapdriver` | Stackdriver |
+| `github.com/moul/zapgorm` | Gorm |
+| `github.com/moul/zapfilter` | Advanced filtering rules |
 
 [go-proverbs]: https://go-proverbs.github.io/
 [import-path]: https://golang.org/cmd/go/#hdr-Remote_import_paths
diff --git a/vendor/go.uber.org/zap/Makefile b/vendor/go.uber.org/zap/Makefile
index 073e9aa..9b1bc3b 100644
--- a/vendor/go.uber.org/zap/Makefile
+++ b/vendor/go.uber.org/zap/Makefile
@@ -1,76 +1,73 @@
-export GO15VENDOREXPERIMENT=1
+export GOBIN ?= $(shell pwd)/bin
 
+GOLINT = $(GOBIN)/golint
+STATICCHECK = $(GOBIN)/staticcheck
 BENCH_FLAGS ?= -cpuprofile=cpu.pprof -memprofile=mem.pprof -benchmem
-PKGS ?= $(shell glide novendor)
+
+# Directories containing independent Go modules.
+#
+# We track coverage only for the main module.
+MODULE_DIRS = . ./benchmarks ./zapgrpc/internal/test
+
 # Many Go tools take file globs or directories as arguments instead of packages.
-PKG_FILES ?= *.go zapcore benchmarks buffer zapgrpc zaptest zaptest/observer internal/bufferpool internal/exit internal/color internal/ztest
-
-# The linting tools evolve with each Go version, so run them only on the latest
-# stable release.
-GO_VERSION := $(shell go version | cut -d " " -f 3)
-GO_MINOR_VERSION := $(word 2,$(subst ., ,$(GO_VERSION)))
-LINTABLE_MINOR_VERSIONS := 12
-ifneq ($(filter $(LINTABLE_MINOR_VERSIONS),$(GO_MINOR_VERSION)),)
-SHOULD_LINT := true
-endif
-
+GO_FILES := $(shell \
+	find . '(' -path '*/.*' -o -path './vendor' ')' -prune \
+	-o -name '*.go' -print | cut -b3-)
 
 .PHONY: all
 all: lint test
 
-.PHONY: dependencies
-dependencies:
-	@echo "Installing Glide and locked dependencies..."
-	glide --version || go get -u -f github.com/Masterminds/glide
-	glide install
-	@echo "Installing test dependencies..."
-	go install ./vendor/github.com/axw/gocov/gocov
-	go install ./vendor/github.com/mattn/goveralls
-ifdef SHOULD_LINT
-	@echo "Installing golint..."
-	go install ./vendor/github.com/golang/lint/golint
-else
-	@echo "Not installing golint, since we don't expect to lint on" $(GO_VERSION)
-endif
-
-# Disable printf-like invocation checking due to testify.assert.Error()
-VET_RULES := -printf=false
-
 .PHONY: lint
-lint:
-ifdef SHOULD_LINT
+lint: $(GOLINT) $(STATICCHECK)
 	@rm -rf lint.log
 	@echo "Checking formatting..."
-	@gofmt -d -s $(PKG_FILES) 2>&1 | tee lint.log
-	@echo "Installing test dependencies for vet..."
-	@go test -i $(PKGS)
+	@gofmt -d -s $(GO_FILES) 2>&1 | tee lint.log
 	@echo "Checking vet..."
-	@go vet $(VET_RULES) $(PKGS) 2>&1 | tee -a lint.log
+	@$(foreach dir,$(MODULE_DIRS),(cd $(dir) && go vet ./... 2>&1) &&) true | tee -a lint.log
 	@echo "Checking lint..."
-	@$(foreach dir,$(PKGS),golint $(dir) 2>&1 | tee -a lint.log;)
+	@$(foreach dir,$(MODULE_DIRS),(cd $(dir) && $(GOLINT) ./... 2>&1) &&) true | tee -a lint.log
+	@echo "Checking staticcheck..."
+	@$(foreach dir,$(MODULE_DIRS),(cd $(dir) && $(STATICCHECK) ./... 2>&1) &&) true | tee -a lint.log
 	@echo "Checking for unresolved FIXMEs..."
-	@git grep -i fixme | grep -v -e vendor -e Makefile | tee -a lint.log
+	@git grep -i fixme | grep -v -e Makefile | tee -a lint.log
 	@echo "Checking for license headers..."
-	@./check_license.sh | tee -a lint.log
+	@./checklicense.sh | tee -a lint.log
 	@[ ! -s lint.log ]
-else
-	@echo "Skipping linters on" $(GO_VERSION)
-endif
+	@echo "Checking 'go mod tidy'..."
+	@make tidy
+	@if ! git diff --quiet; then \
+		echo "'go mod tidy' resulted in changes or working tree is dirty:"; \
+		git --no-pager diff; \
+	fi
+
+$(GOLINT):
+	cd tools && go install golang.org/x/lint/golint
+
+$(STATICCHECK):
+	cd tools && go install honnef.co/go/tools/cmd/staticcheck
 
 .PHONY: test
 test:
-	go test -race $(PKGS)
+	@$(foreach dir,$(MODULE_DIRS),(cd $(dir) && go test -race ./...) &&) true
 
 .PHONY: cover
 cover:
-	./scripts/cover.sh $(PKGS)
+	go test -race -coverprofile=cover.out -coverpkg=./... ./...
+	go tool cover -html=cover.out -o cover.html
 
 .PHONY: bench
 BENCH ?= .
 bench:
-	@$(foreach pkg,$(PKGS),go test -bench=$(BENCH) -run="^$$" $(BENCH_FLAGS) $(pkg);)
+	@$(foreach dir,$(MODULE_DIRS), ( \
+		cd $(dir) && \
+		go list ./... | xargs -n1 go test -bench=$(BENCH) -run="^$$" $(BENCH_FLAGS) \
+	) &&) true
 
 .PHONY: updatereadme
 updatereadme:
 	rm -f README.md
 	cat .readme.tmpl | go run internal/readme/readme.go > README.md
+
+.PHONY: tidy
+tidy:
+	@$(foreach dir,$(MODULE_DIRS),(cd $(dir) && go mod tidy) &&) true
diff --git a/vendor/go.uber.org/zap/README.md b/vendor/go.uber.org/zap/README.md
index f4fd1cb..1e64d6c 100644
--- a/vendor/go.uber.org/zap/README.md
+++ b/vendor/go.uber.org/zap/README.md
@@ -64,43 +64,40 @@
 
 Log a message and 10 fields:
 
-| Package | Time | Objects Allocated |
-| :--- | :---: | :---: |
-| :zap: zap | 3131 ns/op | 5 allocs/op |
-| :zap: zap (sugared) | 4173 ns/op | 21 allocs/op |
-| zerolog | 16154 ns/op | 90 allocs/op |
-| lion | 16341 ns/op | 111 allocs/op |
-| go-kit | 17049 ns/op | 126 allocs/op |
-| logrus | 23662 ns/op | 142 allocs/op |
-| log15 | 36351 ns/op | 149 allocs/op |
-| apex/log | 42530 ns/op | 126 allocs/op |
+| Package | Time | Time % to zap | Objects Allocated |
+| :------ | :--: | :-----------: | :---------------: |
+| :zap: zap | 862 ns/op | +0% | 5 allocs/op
+| :zap: zap (sugared) | 1250 ns/op | +45% | 11 allocs/op
+| zerolog | 4021 ns/op | +366% | 76 allocs/op
+| go-kit | 4542 ns/op | +427% | 105 allocs/op
+| apex/log | 26785 ns/op | +3007% | 115 allocs/op
+| logrus | 29501 ns/op | +3322% | 125 allocs/op
+| log15 | 29906 ns/op | +3369% | 122 allocs/op
 
 Log a message with a logger that already has 10 fields of context:
 
-| Package | Time | Objects Allocated |
-| :--- | :---: | :---: |
-| :zap: zap | 380 ns/op | 0 allocs/op |
-| :zap: zap (sugared) | 564 ns/op | 2 allocs/op |
-| zerolog | 321 ns/op | 0 allocs/op |
-| lion | 7092 ns/op | 39 allocs/op |
-| go-kit | 20226 ns/op | 115 allocs/op |
-| logrus | 22312 ns/op | 130 allocs/op |
-| log15 | 28788 ns/op | 79 allocs/op |
-| apex/log | 42063 ns/op | 115 allocs/op |
+| Package | Time | Time % to zap | Objects Allocated |
+| :------ | :--: | :-----------: | :---------------: |
+| :zap: zap | 126 ns/op | +0% | 0 allocs/op
+| :zap: zap (sugared) | 187 ns/op | +48% | 2 allocs/op
+| zerolog | 88 ns/op | -30% | 0 allocs/op
+| go-kit | 5087 ns/op | +3937% | 103 allocs/op
+| log15 | 18548 ns/op | +14621% | 73 allocs/op
+| apex/log | 26012 ns/op | +20544% | 104 allocs/op
+| logrus | 27236 ns/op | +21516% | 113 allocs/op
 
 Log a static string, without any context or `printf`-style templating:
 
-| Package | Time | Objects Allocated |
-| :--- | :---: | :---: |
-| :zap: zap | 361 ns/op | 0 allocs/op |
-| :zap: zap (sugared) | 534 ns/op | 2 allocs/op |
-| zerolog | 323 ns/op | 0 allocs/op |
-| standard library | 575 ns/op | 2 allocs/op |
-| go-kit | 922 ns/op | 13 allocs/op |
-| lion | 1413 ns/op | 10 allocs/op |
-| logrus | 2291 ns/op | 27 allocs/op |
-| apex/log | 3690 ns/op | 11 allocs/op |
-| log15 | 5954 ns/op | 26 allocs/op |
+| Package | Time | Time % to zap | Objects Allocated |
+| :------ | :--: | :-----------: | :---------------: |
+| :zap: zap | 118 ns/op | +0% | 0 allocs/op
+| :zap: zap (sugared) | 191 ns/op | +62% | 2 allocs/op
+| zerolog | 93 ns/op | -21% | 0 allocs/op
+| go-kit | 280 ns/op | +137% | 11 allocs/op
+| standard library | 499 ns/op | +323% | 2 allocs/op
+| apex/log | 1990 ns/op | +1586% | 10 allocs/op
+| logrus | 3129 ns/op | +2552% | 24 allocs/op
+| log15 | 3887 ns/op | +3194% | 23 allocs/op
 
 ## Development Status: Stable
 
@@ -124,13 +121,14 @@
 
 <sup id="footnote-versions">1</sup> In particular, keep in mind that we may be
 benchmarking against slightly older versions of other packages. Versions are
-pinned in zap's [glide.lock][] file. [↩](#anchor-versions)
+pinned in the [benchmarks/go.mod][] file. [↩](#anchor-versions)
 
-[doc-img]: https://godoc.org/go.uber.org/zap?status.svg
-[doc]: https://godoc.org/go.uber.org/zap
-[ci-img]: https://travis-ci.org/uber-go/zap.svg?branch=master
-[ci]: https://travis-ci.org/uber-go/zap
+[doc-img]: https://pkg.go.dev/badge/go.uber.org/zap
+[doc]: https://pkg.go.dev/go.uber.org/zap
+[ci-img]: https://github.com/uber-go/zap/actions/workflows/go.yml/badge.svg
+[ci]: https://github.com/uber-go/zap/actions/workflows/go.yml
 [cov-img]: https://codecov.io/gh/uber-go/zap/branch/master/graph/badge.svg
 [cov]: https://codecov.io/gh/uber-go/zap
 [benchmarking suite]: https://github.com/uber-go/zap/tree/master/benchmarks
-[glide.lock]: https://github.com/uber-go/zap/blob/master/glide.lock
+[benchmarks/go.mod]: https://github.com/uber-go/zap/blob/master/benchmarks/go.mod
+
diff --git a/vendor/go.uber.org/zap/buffer/buffer.go b/vendor/go.uber.org/zap/buffer/buffer.go
index 7592e8c..9e929cd 100644
--- a/vendor/go.uber.org/zap/buffer/buffer.go
+++ b/vendor/go.uber.org/zap/buffer/buffer.go
@@ -23,7 +23,10 @@
 // package's zero-allocation formatters.
 package buffer // import "go.uber.org/zap/buffer"
 
-import "strconv"
+import (
+	"strconv"
+	"time"
+)
 
 const _size = 1024 // by default, create 1 KiB buffers
 
@@ -49,6 +52,11 @@
 	b.bs = strconv.AppendInt(b.bs, i, 10)
 }
 
+// AppendTime appends the time formatted using the specified layout.
+func (b *Buffer) AppendTime(t time.Time, layout string) {
+	b.bs = t.AppendFormat(b.bs, layout)
+}
+
 // AppendUint appends an unsigned integer to the underlying buffer (assuming
 // base 10).
 func (b *Buffer) AppendUint(i uint64) {
@@ -98,6 +106,24 @@
 	return len(bs), nil
 }
 
+// WriteByte writes a single byte to the Buffer.
+//
+// Error returned is always nil, function signature is compatible
+// with bytes.Buffer and bufio.Writer
+func (b *Buffer) WriteByte(v byte) error {
+	b.AppendByte(v)
+	return nil
+}
+
+// WriteString writes a string to the Buffer.
+//
+// Error returned is always nil, function signature is compatible
+// with bytes.Buffer and bufio.Writer
+func (b *Buffer) WriteString(s string) (int, error) {
+	b.AppendString(s)
+	return len(s), nil
+}
+
 // TrimNewline trims any final "\n" byte from the end of the buffer.
 func (b *Buffer) TrimNewline() {
 	if i := len(b.bs) - 1; i >= 0 {
diff --git a/vendor/go.uber.org/zap/check_license.sh b/vendor/go.uber.org/zap/checklicense.sh
similarity index 100%
rename from vendor/go.uber.org/zap/check_license.sh
rename to vendor/go.uber.org/zap/checklicense.sh
diff --git a/vendor/go.uber.org/zap/config.go b/vendor/go.uber.org/zap/config.go
index 6fe17d9..55637fb 100644
--- a/vendor/go.uber.org/zap/config.go
+++ b/vendor/go.uber.org/zap/config.go
@@ -21,6 +21,7 @@
 package zap
 
 import (
+	"fmt"
 	"sort"
 	"time"
 
@@ -31,10 +32,14 @@
 // global CPU and I/O load that logging puts on your process while attempting
 // to preserve a representative subset of your logs.
 //
-// Values configured here are per-second. See zapcore.NewSampler for details.
+// If specified, the Sampler will invoke the Hook after each decision.
+//
+// Values configured here are per-second. See zapcore.NewSamplerWithOptions for
+// details.
 type SamplingConfig struct {
-	Initial    int `json:"initial" yaml:"initial"`
-	Thereafter int `json:"thereafter" yaml:"thereafter"`
+	Initial    int                                           `json:"initial" yaml:"initial"`
+	Thereafter int                                           `json:"thereafter" yaml:"thereafter"`
+	Hook       func(zapcore.Entry, zapcore.SamplingDecision) `json:"-" yaml:"-"`
 }
 
 // Config offers a declarative way to construct a logger. It doesn't do
@@ -96,6 +101,7 @@
 		LevelKey:       "level",
 		NameKey:        "logger",
 		CallerKey:      "caller",
+		FunctionKey:    zapcore.OmitKey,
 		MessageKey:     "msg",
 		StacktraceKey:  "stacktrace",
 		LineEnding:     zapcore.DefaultLineEnding,
@@ -135,6 +141,7 @@
 		LevelKey:       "L",
 		NameKey:        "N",
 		CallerKey:      "C",
+		FunctionKey:    zapcore.OmitKey,
 		MessageKey:     "M",
 		StacktraceKey:  "S",
 		LineEnding:     zapcore.DefaultLineEnding,
@@ -174,6 +181,10 @@
 		return nil, err
 	}
 
+	if cfg.Level == (AtomicLevel{}) {
+		return nil, fmt.Errorf("missing Level")
+	}
+
 	log := New(
 		zapcore.NewCore(enc, sink, cfg.Level),
 		cfg.buildOptions(errSink)...,
@@ -203,9 +214,19 @@
 		opts = append(opts, AddStacktrace(stackLevel))
 	}
 
-	if cfg.Sampling != nil {
+	if scfg := cfg.Sampling; scfg != nil {
 		opts = append(opts, WrapCore(func(core zapcore.Core) zapcore.Core {
-			return zapcore.NewSampler(core, time.Second, int(cfg.Sampling.Initial), int(cfg.Sampling.Thereafter))
+			var samplerOpts []zapcore.SamplerOption
+			if scfg.Hook != nil {
+				samplerOpts = append(samplerOpts, zapcore.SamplerHook(scfg.Hook))
+			}
+			return zapcore.NewSamplerWithOptions(
+				core,
+				time.Second,
+				cfg.Sampling.Initial,
+				cfg.Sampling.Thereafter,
+				samplerOpts...,
+			)
 		}))
 	}
 
diff --git a/vendor/go.uber.org/zap/encoder.go b/vendor/go.uber.org/zap/encoder.go
index 2e9d3c3..08ed833 100644
--- a/vendor/go.uber.org/zap/encoder.go
+++ b/vendor/go.uber.org/zap/encoder.go
@@ -62,6 +62,10 @@
 }
 
 func newEncoder(name string, encoderConfig zapcore.EncoderConfig) (zapcore.Encoder, error) {
+	if encoderConfig.TimeKey != "" && encoderConfig.EncodeTime == nil {
+		return nil, fmt.Errorf("missing EncodeTime in EncoderConfig")
+	}
+
 	_encoderMutex.RLock()
 	defer _encoderMutex.RUnlock()
 	if name == "" {
diff --git a/vendor/go.uber.org/zap/field.go b/vendor/go.uber.org/zap/field.go
index 5130e13..bbb745d 100644
--- a/vendor/go.uber.org/zap/field.go
+++ b/vendor/go.uber.org/zap/field.go
@@ -32,12 +32,23 @@
 // improves the navigability of this package's API documentation.
 type Field = zapcore.Field
 
+var (
+	_minTimeInt64 = time.Unix(0, math.MinInt64)
+	_maxTimeInt64 = time.Unix(0, math.MaxInt64)
+)
+
 // Skip constructs a no-op field, which is often useful when handling invalid
 // inputs in other Field constructors.
 func Skip() Field {
 	return Field{Type: zapcore.SkipType}
 }
 
+// nilField returns a field which will marshal explicitly as nil. See motivation
+// in https://github.com/uber-go/zap/issues/753 . If we ever make breaking
+// changes and add zapcore.NilType and zapcore.ObjectEncoder.AddNil, the
+// implementation here should be changed to reflect that.
+func nilField(key string) Field { return Reflect(key, nil) }
+
 // Binary constructs a field that carries an opaque binary blob.
 //
 // Binary data is serialized in an encoding-appropriate format. For example,
@@ -56,6 +67,15 @@
 	return Field{Key: key, Type: zapcore.BoolType, Integer: ival}
 }
 
+// Boolp constructs a field that carries a *bool. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Boolp(key string, val *bool) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Bool(key, *val)
+}
+
 // ByteString constructs a field that carries UTF-8 encoded text as a []byte.
 // To log opaque binary blobs (which aren't necessarily valid UTF-8), use
 // Binary.
@@ -70,6 +90,15 @@
 	return Field{Key: key, Type: zapcore.Complex128Type, Interface: val}
 }
 
+// Complex128p constructs a field that carries a *complex128. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Complex128p(key string, val *complex128) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Complex128(key, *val)
+}
+
 // Complex64 constructs a field that carries a complex number. Unlike most
 // numeric fields, this costs an allocation (to convert the complex64 to
 // interface{}).
@@ -77,6 +106,15 @@
 	return Field{Key: key, Type: zapcore.Complex64Type, Interface: val}
 }
 
+// Complex64p constructs a field that carries a *complex64. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Complex64p(key string, val *complex64) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Complex64(key, *val)
+}
+
 // Float64 constructs a field that carries a float64. The way the
 // floating-point value is represented is encoder-dependent, so marshaling is
 // necessarily lazy.
@@ -84,6 +122,15 @@
 	return Field{Key: key, Type: zapcore.Float64Type, Integer: int64(math.Float64bits(val))}
 }
 
+// Float64p constructs a field that carries a *float64. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Float64p(key string, val *float64) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Float64(key, *val)
+}
+
 // Float32 constructs a field that carries a float32. The way the
 // floating-point value is represented is encoder-dependent, so marshaling is
 // necessarily lazy.
@@ -91,66 +138,183 @@
 	return Field{Key: key, Type: zapcore.Float32Type, Integer: int64(math.Float32bits(val))}
 }
 
+// Float32p constructs a field that carries a *float32. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Float32p(key string, val *float32) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Float32(key, *val)
+}
+
 // Int constructs a field with the given key and value.
 func Int(key string, val int) Field {
 	return Int64(key, int64(val))
 }
 
+// Intp constructs a field that carries a *int. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Intp(key string, val *int) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Int(key, *val)
+}
+
 // Int64 constructs a field with the given key and value.
 func Int64(key string, val int64) Field {
 	return Field{Key: key, Type: zapcore.Int64Type, Integer: val}
 }
 
+// Int64p constructs a field that carries a *int64. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Int64p(key string, val *int64) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Int64(key, *val)
+}
+
 // Int32 constructs a field with the given key and value.
 func Int32(key string, val int32) Field {
 	return Field{Key: key, Type: zapcore.Int32Type, Integer: int64(val)}
 }
 
+// Int32p constructs a field that carries a *int32. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Int32p(key string, val *int32) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Int32(key, *val)
+}
+
 // Int16 constructs a field with the given key and value.
 func Int16(key string, val int16) Field {
 	return Field{Key: key, Type: zapcore.Int16Type, Integer: int64(val)}
 }
 
+// Int16p constructs a field that carries a *int16. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Int16p(key string, val *int16) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Int16(key, *val)
+}
+
 // Int8 constructs a field with the given key and value.
 func Int8(key string, val int8) Field {
 	return Field{Key: key, Type: zapcore.Int8Type, Integer: int64(val)}
 }
 
+// Int8p constructs a field that carries a *int8. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Int8p(key string, val *int8) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Int8(key, *val)
+}
+
 // String constructs a field with the given key and value.
 func String(key string, val string) Field {
 	return Field{Key: key, Type: zapcore.StringType, String: val}
 }
 
+// Stringp constructs a field that carries a *string. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Stringp(key string, val *string) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return String(key, *val)
+}
+
 // Uint constructs a field with the given key and value.
 func Uint(key string, val uint) Field {
 	return Uint64(key, uint64(val))
 }
 
+// Uintp constructs a field that carries a *uint. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uintp(key string, val *uint) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uint(key, *val)
+}
+
 // Uint64 constructs a field with the given key and value.
 func Uint64(key string, val uint64) Field {
 	return Field{Key: key, Type: zapcore.Uint64Type, Integer: int64(val)}
 }
 
+// Uint64p constructs a field that carries a *uint64. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uint64p(key string, val *uint64) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uint64(key, *val)
+}
+
 // Uint32 constructs a field with the given key and value.
 func Uint32(key string, val uint32) Field {
 	return Field{Key: key, Type: zapcore.Uint32Type, Integer: int64(val)}
 }
 
+// Uint32p constructs a field that carries a *uint32. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uint32p(key string, val *uint32) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uint32(key, *val)
+}
+
 // Uint16 constructs a field with the given key and value.
 func Uint16(key string, val uint16) Field {
 	return Field{Key: key, Type: zapcore.Uint16Type, Integer: int64(val)}
 }
 
+// Uint16p constructs a field that carries a *uint16. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uint16p(key string, val *uint16) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uint16(key, *val)
+}
+
 // Uint8 constructs a field with the given key and value.
 func Uint8(key string, val uint8) Field {
 	return Field{Key: key, Type: zapcore.Uint8Type, Integer: int64(val)}
 }
 
+// Uint8p constructs a field that carries a *uint8. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uint8p(key string, val *uint8) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uint8(key, *val)
+}
+
 // Uintptr constructs a field with the given key and value.
 func Uintptr(key string, val uintptr) Field {
 	return Field{Key: key, Type: zapcore.UintptrType, Integer: int64(val)}
 }
 
+// Uintptrp constructs a field that carries a *uintptr. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Uintptrp(key string, val *uintptr) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Uintptr(key, *val)
+}
+
 // Reflect constructs a field with the given key and an arbitrary object. It uses
 // an encoding-appropriate, reflection-based function to lazily serialize nearly
 // any object into the logging context, but it's relatively slow and
@@ -180,19 +344,37 @@
 // Time constructs a Field with the given key and value. The encoder
 // controls how the time is serialized.
 func Time(key string, val time.Time) Field {
+	if val.Before(_minTimeInt64) || val.After(_maxTimeInt64) {
+		return Field{Key: key, Type: zapcore.TimeFullType, Interface: val}
+	}
 	return Field{Key: key, Type: zapcore.TimeType, Integer: val.UnixNano(), Interface: val.Location()}
 }
 
+// Timep constructs a field that carries a *time.Time. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Timep(key string, val *time.Time) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Time(key, *val)
+}
+
 // Stack constructs a field that stores a stacktrace of the current goroutine
 // under provided key. Keep in mind that taking a stacktrace is eager and
 // expensive (relatively speaking); this function both makes an allocation and
 // takes about two microseconds.
 func Stack(key string) Field {
+	return StackSkip(key, 1) // skip Stack
+}
+
+// StackSkip constructs a field similarly to Stack, but also skips the given
+// number of frames from the top of the stacktrace.
+func StackSkip(key string, skip int) Field {
 	// Returning the stacktrace as a string costs an allocation, but saves us
 	// from expanding the zapcore.Field union struct to include a byte slice. Since
 	// taking a stacktrace is already so expensive (~10us), the extra allocation
 	// is okay.
-	return String(key, takeStacktrace())
+	return String(key, takeStacktrace(skip+1)) // skip StackSkip
 }
 
 // Duration constructs a field with the given key and value. The encoder
@@ -201,6 +383,15 @@
 	return Field{Key: key, Type: zapcore.DurationType, Integer: int64(val)}
 }
 
+// Durationp constructs a field that carries a *time.Duration. The returned Field will safely
+// and explicitly represent `nil` when appropriate.
+func Durationp(key string, val *time.Duration) Field {
+	if val == nil {
+		return nilField(key)
+	}
+	return Duration(key, *val)
+}
+
 // Object constructs a field with the given key and ObjectMarshaler. It
 // provides a flexible, but still type-safe and efficient, way to add map- or
 // struct-like user-defined types to the logging context. The struct's
@@ -209,6 +400,16 @@
 	return Field{Key: key, Type: zapcore.ObjectMarshalerType, Interface: val}
 }
 
+// Inline constructs a Field that is similar to Object, but it
+// will add the elements of the provided ObjectMarshaler to the
+// current namespace.
+func Inline(val zapcore.ObjectMarshaler) Field {
+	return zapcore.Field{
+		Type:      zapcore.InlineMarshalerType,
+		Interface: val,
+	}
+}
+
 // Any takes a key and an arbitrary value and chooses the best way to represent
 // them as a field, falling back to a reflection-based approach only if
 // necessary.
@@ -224,78 +425,116 @@
 		return Array(key, val)
 	case bool:
 		return Bool(key, val)
+	case *bool:
+		return Boolp(key, val)
 	case []bool:
 		return Bools(key, val)
 	case complex128:
 		return Complex128(key, val)
+	case *complex128:
+		return Complex128p(key, val)
 	case []complex128:
 		return Complex128s(key, val)
 	case complex64:
 		return Complex64(key, val)
+	case *complex64:
+		return Complex64p(key, val)
 	case []complex64:
 		return Complex64s(key, val)
 	case float64:
 		return Float64(key, val)
+	case *float64:
+		return Float64p(key, val)
 	case []float64:
 		return Float64s(key, val)
 	case float32:
 		return Float32(key, val)
+	case *float32:
+		return Float32p(key, val)
 	case []float32:
 		return Float32s(key, val)
 	case int:
 		return Int(key, val)
+	case *int:
+		return Intp(key, val)
 	case []int:
 		return Ints(key, val)
 	case int64:
 		return Int64(key, val)
+	case *int64:
+		return Int64p(key, val)
 	case []int64:
 		return Int64s(key, val)
 	case int32:
 		return Int32(key, val)
+	case *int32:
+		return Int32p(key, val)
 	case []int32:
 		return Int32s(key, val)
 	case int16:
 		return Int16(key, val)
+	case *int16:
+		return Int16p(key, val)
 	case []int16:
 		return Int16s(key, val)
 	case int8:
 		return Int8(key, val)
+	case *int8:
+		return Int8p(key, val)
 	case []int8:
 		return Int8s(key, val)
 	case string:
 		return String(key, val)
+	case *string:
+		return Stringp(key, val)
 	case []string:
 		return Strings(key, val)
 	case uint:
 		return Uint(key, val)
+	case *uint:
+		return Uintp(key, val)
 	case []uint:
 		return Uints(key, val)
 	case uint64:
 		return Uint64(key, val)
+	case *uint64:
+		return Uint64p(key, val)
 	case []uint64:
 		return Uint64s(key, val)
 	case uint32:
 		return Uint32(key, val)
+	case *uint32:
+		return Uint32p(key, val)
 	case []uint32:
 		return Uint32s(key, val)
 	case uint16:
 		return Uint16(key, val)
+	case *uint16:
+		return Uint16p(key, val)
 	case []uint16:
 		return Uint16s(key, val)
 	case uint8:
 		return Uint8(key, val)
+	case *uint8:
+		return Uint8p(key, val)
 	case []byte:
 		return Binary(key, val)
 	case uintptr:
 		return Uintptr(key, val)
+	case *uintptr:
+		return Uintptrp(key, val)
 	case []uintptr:
 		return Uintptrs(key, val)
 	case time.Time:
 		return Time(key, val)
+	case *time.Time:
+		return Timep(key, val)
 	case []time.Time:
 		return Times(key, val)
 	case time.Duration:
 		return Duration(key, val)
+	case *time.Duration:
+		return Durationp(key, val)
 	case []time.Duration:
 		return Durations(key, val)
 	case error:
diff --git a/vendor/go.uber.org/zap/glide.lock b/vendor/go.uber.org/zap/glide.lock
deleted file mode 100644
index 881b462..0000000
--- a/vendor/go.uber.org/zap/glide.lock
+++ /dev/null
@@ -1,76 +0,0 @@
-hash: f073ba522c06c88ea3075bde32a8aaf0969a840a66cab6318a0897d141ffee92
-updated: 2017-07-22T18:06:49.598185334-07:00
-imports:
-- name: go.uber.org/atomic
-  version: 4e336646b2ef9fc6e47be8e21594178f98e5ebcf
-- name: go.uber.org/multierr
-  version: 3c4937480c32f4c13a875a1829af76c98ca3d40a
-testImports:
-- name: github.com/apex/log
-  version: d9b960447bfa720077b2da653cc79e533455b499
-  subpackages:
-  - handlers/json
-- name: github.com/axw/gocov
-  version: 3a69a0d2a4ef1f263e2d92b041a69593d6964fe8
-  subpackages:
-  - gocov
-- name: github.com/davecgh/go-spew
-  version: 04cdfd42973bb9c8589fd6a731800cf222fde1a9
-  subpackages:
-  - spew
-- name: github.com/fatih/color
-  version: 62e9147c64a1ed519147b62a56a14e83e2be02c1
-- name: github.com/go-kit/kit
-  version: e10f5bf035be9af21fd5b2fb4469d5716c6ab07d
-  subpackages:
-  - log
-- name: github.com/go-logfmt/logfmt
-  version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5
-- name: github.com/go-stack/stack
-  version: 54be5f394ed2c3e19dac9134a40a95ba5a017f7b
-- name: github.com/golang/lint
-  version: c5fb716d6688a859aae56d26d3e6070808df29f7
-  subpackages:
-  - golint
-- name: github.com/kr/logfmt
-  version: b84e30acd515aadc4b783ad4ff83aff3299bdfe0
-- name: github.com/mattn/go-colorable
-  version: 3fa8c76f9daed4067e4a806fb7e4dc86455c6d6a
-- name: github.com/mattn/go-isatty
-  version: fc9e8d8ef48496124e79ae0df75490096eccf6fe
-- name: github.com/mattn/goveralls
-  version: 6efce81852ad1b7567c17ad71b03aeccc9dd9ae0
-- name: github.com/pborman/uuid
-  version: e790cca94e6cc75c7064b1332e63811d4aae1a53
-- name: github.com/pkg/errors
-  version: 645ef00459ed84a119197bfb8d8205042c6df63d
-- name: github.com/pmezard/go-difflib
-  version: d8ed2627bdf02c080bf22230dbb337003b7aba2d
-  subpackages:
-  - difflib
-- name: github.com/rs/zerolog
-  version: eed4c2b94d945e0b2456ad6aa518a443986b5f22
-- name: github.com/satori/go.uuid
-  version: 5bf94b69c6b68ee1b541973bb8e1144db23a194b
-- name: github.com/sirupsen/logrus
-  version: 7dd06bf38e1e13df288d471a57d5adbac106be9e
-- name: github.com/stretchr/testify
-  version: f6abca593680b2315d2075e0f5e2a9751e3f431a
-  subpackages:
-  - assert
-  - require
-- name: go.pedge.io/lion
-  version: 87958e8713f1fa138d993087133b97e976642159
-- name: golang.org/x/sys
-  version: c4489faa6e5ab84c0ef40d6ee878f7a030281f0f
-  subpackages:
-  - unix
-- name: golang.org/x/tools
-  version: 496819729719f9d07692195e0a94d6edd2251389
-  subpackages:
-  - cover
-- name: gopkg.in/inconshreveable/log15.v2
-  version: b105bd37f74e5d9dc7b6ad7806715c7a2b83fd3f
-  subpackages:
-  - stack
-  - term
diff --git a/vendor/go.uber.org/zap/glide.yaml b/vendor/go.uber.org/zap/glide.yaml
index 9441259..8e1d05e 100644
--- a/vendor/go.uber.org/zap/glide.yaml
+++ b/vendor/go.uber.org/zap/glide.yaml
@@ -22,12 +22,11 @@
 - package: github.com/mattn/goveralls
 - package: github.com/pborman/uuid
 - package: github.com/pkg/errors
-- package: go.pedge.io/lion
 - package: github.com/rs/zerolog
 - package: golang.org/x/tools
   subpackages:
   - cover
-- package: github.com/golang/lint
+- package: golang.org/x/lint
   subpackages:
   - golint
 - package: github.com/axw/gocov
diff --git a/vendor/go.uber.org/zap/go.mod b/vendor/go.uber.org/zap/go.mod
new file mode 100644
index 0000000..9455c99
--- /dev/null
+++ b/vendor/go.uber.org/zap/go.mod
@@ -0,0 +1,14 @@
+module go.uber.org/zap
+
+go 1.13
+
+require (
+	github.com/benbjohnson/clock v1.1.0
+	github.com/pkg/errors v0.8.1
+	github.com/stretchr/testify v1.7.0
+	go.uber.org/atomic v1.7.0
+	go.uber.org/goleak v1.1.10
+	go.uber.org/multierr v1.6.0
+	gopkg.in/yaml.v2 v2.2.8
+	gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect
+)
diff --git a/vendor/go.uber.org/zap/go.sum b/vendor/go.uber.org/zap/go.sum
new file mode 100644
index 0000000..b330071
--- /dev/null
+++ b/vendor/go.uber.org/zap/go.sum
@@ -0,0 +1,54 @@
+github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8=
+github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
+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/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
+github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
+github.com/kr/pty v1.1.1 h1:VkoXIwSboBpnk99O/KFauAEILuNHv5DVFKZMBN/gUgw=
+github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
+github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
+github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=
+github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+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 h1:4G4v2dO3VZwixGIRoQ5Lfboy6nUhCyYzaqnIAPPhYs4=
+github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY=
+github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
+go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
+go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0=
+go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
+go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4=
+go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI=
+golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
+golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU=
+golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=
+golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20191108193012-7d206e10da11 h1:Yq9t9jnGoR+dBuitxdo9l6Q7xh/zOyNnYUtDKaQ3x0E=
+golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7 h1:9zdDQZ7Thm29KFXgAX/+yaf3eVbP7djjWp/dXAppNCc=
+golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY=
+gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10=
+gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo=
+gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
diff --git a/vendor/go.uber.org/zap/http_handler.go b/vendor/go.uber.org/zap/http_handler.go
index 1b0ecac..1297c33 100644
--- a/vendor/go.uber.org/zap/http_handler.go
+++ b/vendor/go.uber.org/zap/http_handler.go
@@ -23,6 +23,7 @@
 import (
 	"encoding/json"
 	"fmt"
+	"io"
 	"net/http"
 
 	"go.uber.org/zap/zapcore"
@@ -31,47 +32,63 @@
 // ServeHTTP is a simple JSON endpoint that can report on or change the current
 // logging level.
 //
-// GET requests return a JSON description of the current logging level. PUT
-// requests change the logging level and expect a payload like:
+// GET
+//
+// The GET request returns a JSON description of the current logging level like:
 //   {"level":"info"}
 //
-// It's perfectly safe to change the logging level while a program is running.
+// PUT
+//
+// The PUT request changes the logging level. It is perfectly safe to change the
+// logging level while a program is running. Two content types are supported:
+//
+//    Content-Type: application/x-www-form-urlencoded
+//
+// With this content type, the level can be provided through the request body or
+// a query parameter. The log level is URL encoded like:
+//
+//    level=debug
+//
+// The request body takes precedence over the query parameter, if both are
+// specified.
+//
+// This content type is the default for a curl PUT request. Following are two
+// example curl requests that both set the logging level to debug.
+//
+//    curl -X PUT localhost:8080/log/level?level=debug
+//    curl -X PUT localhost:8080/log/level -d level=debug
+//
+// For any other content type, the payload is expected to be JSON encoded and
+// look like:
+//
+//   {"level":"info"}
+//
+// An example curl request could look like this:
+//
+//    curl -X PUT localhost:8080/log/level -H "Content-Type: application/json" -d '{"level":"debug"}'
+//
 func (lvl AtomicLevel) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	type errorResponse struct {
 		Error string `json:"error"`
 	}
 	type payload struct {
-		Level *zapcore.Level `json:"level"`
+		Level zapcore.Level `json:"level"`
 	}
 
 	enc := json.NewEncoder(w)
 
 	switch r.Method {
-
 	case http.MethodGet:
-		current := lvl.Level()
-		enc.Encode(payload{Level: &current})
-
+		enc.Encode(payload{Level: lvl.Level()})
 	case http.MethodPut:
-		var req payload
-
-		if errmess := func() string {
-			if err := json.NewDecoder(r.Body).Decode(&req); err != nil {
-				return fmt.Sprintf("Request body must be well-formed JSON: %v", err)
-			}
-			if req.Level == nil {
-				return "Must specify a logging level."
-			}
-			return ""
-		}(); errmess != "" {
+		requestedLvl, err := decodePutRequest(r.Header.Get("Content-Type"), r)
+		if err != nil {
 			w.WriteHeader(http.StatusBadRequest)
-			enc.Encode(errorResponse{Error: errmess})
+			enc.Encode(errorResponse{Error: err.Error()})
 			return
 		}
-
-		lvl.SetLevel(*req.Level)
-		enc.Encode(req)
-
+		lvl.SetLevel(requestedLvl)
+		enc.Encode(payload{Level: lvl.Level()})
 	default:
 		w.WriteHeader(http.StatusMethodNotAllowed)
 		enc.Encode(errorResponse{
@@ -79,3 +96,37 @@
 		})
 	}
 }
+
+// Decodes incoming PUT requests and returns the requested logging level.
+func decodePutRequest(contentType string, r *http.Request) (zapcore.Level, error) {
+	if contentType == "application/x-www-form-urlencoded" {
+		return decodePutURL(r)
+	}
+	return decodePutJSON(r.Body)
+}
+
+func decodePutURL(r *http.Request) (zapcore.Level, error) {
+	lvl := r.FormValue("level")
+	if lvl == "" {
+		return 0, fmt.Errorf("must specify logging level")
+	}
+	var l zapcore.Level
+	if err := l.UnmarshalText([]byte(lvl)); err != nil {
+		return 0, err
+	}
+	return l, nil
+}
+
+func decodePutJSON(body io.Reader) (zapcore.Level, error) {
+	var pld struct {
+		Level *zapcore.Level `json:"level"`
+	}
+	if err := json.NewDecoder(body).Decode(&pld); err != nil {
+		return 0, fmt.Errorf("malformed request body: %v", err)
+	}
+	if pld.Level == nil {
+		return 0, fmt.Errorf("must specify logging level")
+	}
+	return *pld.Level, nil
+
+}
diff --git a/vendor/go.uber.org/zap/logger.go b/vendor/go.uber.org/zap/logger.go
index dc8f6e3..f116bd9 100644
--- a/vendor/go.uber.org/zap/logger.go
+++ b/vendor/go.uber.org/zap/logger.go
@@ -26,7 +26,6 @@
 	"os"
 	"runtime"
 	"strings"
-	"time"
 
 	"go.uber.org/zap/zapcore"
 )
@@ -42,13 +41,17 @@
 	core zapcore.Core
 
 	development bool
+	addCaller   bool
+	onFatal     zapcore.CheckWriteAction // default is WriteThenFatal
+
 	name        string
 	errorOutput zapcore.WriteSyncer
 
-	addCaller bool
-	addStack  zapcore.LevelEnabler
+	addStack zapcore.LevelEnabler
 
 	callerSkip int
+
+	clock zapcore.Clock
 }
 
 // New constructs a new Logger from the provided zapcore.Core and Options. If
@@ -69,6 +72,7 @@
 		core:        core,
 		errorOutput: zapcore.Lock(os.Stderr),
 		addStack:    zapcore.FatalLevel + 1,
+		clock:       zapcore.DefaultClock,
 	}
 	return log.WithOptions(options...)
 }
@@ -83,6 +87,7 @@
 		core:        zapcore.NewNopCore(),
 		errorOutput: zapcore.AddSync(ioutil.Discard),
 		addStack:    zapcore.FatalLevel + 1,
+		clock:       zapcore.DefaultClock,
 	}
 }
 
@@ -258,11 +263,17 @@
 	// (e.g., Check, Info, Fatal).
 	const callerSkipOffset = 2
 
+	// Check the level first to reduce the cost of disabled log calls.
+	// Since Panic and higher may exit, we skip the optimization for those levels.
+	if lvl < zapcore.DPanicLevel && !log.core.Enabled(lvl) {
+		return nil
+	}
+
 	// Create basic checked entry thru the core; this will be non-nil if the
 	// log message will actually be written somewhere.
 	ent := zapcore.Entry{
 		LoggerName: log.name,
-		Time:       time.Now(),
+		Time:       log.clock.Now(),
 		Level:      lvl,
 		Message:    msg,
 	}
@@ -274,7 +285,13 @@
 	case zapcore.PanicLevel:
 		ce = ce.Should(ent, zapcore.WriteThenPanic)
 	case zapcore.FatalLevel:
-		ce = ce.Should(ent, zapcore.WriteThenFatal)
+		onFatal := log.onFatal
+		// Noop is the default value for CheckWriteAction, and it leads to
+		// continued execution after a Fatal which is unexpected.
+		if onFatal == zapcore.WriteThenNoop {
+			onFatal = zapcore.WriteThenFatal
+		}
+		ce = ce.Should(ent, onFatal)
 	case zapcore.DPanicLevel:
 		if log.development {
 			ce = ce.Should(ent, zapcore.WriteThenPanic)
@@ -291,15 +308,41 @@
 	// Thread the error output through to the CheckedEntry.
 	ce.ErrorOutput = log.errorOutput
 	if log.addCaller {
-		ce.Entry.Caller = zapcore.NewEntryCaller(runtime.Caller(log.callerSkip + callerSkipOffset))
-		if !ce.Entry.Caller.Defined {
-			fmt.Fprintf(log.errorOutput, "%v Logger.check error: failed to get caller\n", time.Now().UTC())
+		frame, defined := getCallerFrame(log.callerSkip + callerSkipOffset)
+		if !defined {
+			fmt.Fprintf(log.errorOutput, "%v Logger.check error: failed to get caller\n", ent.Time.UTC())
 			log.errorOutput.Sync()
 		}
+
+		ce.Entry.Caller = zapcore.EntryCaller{
+			Defined:  defined,
+			PC:       frame.PC,
+			File:     frame.File,
+			Line:     frame.Line,
+			Function: frame.Function,
+		}
 	}
 	if log.addStack.Enabled(ce.Entry.Level) {
-		ce.Entry.Stack = Stack("").String
+		ce.Entry.Stack = StackSkip("", log.callerSkip+callerSkipOffset).String
 	}
 
 	return ce
 }
+
+// getCallerFrame gets caller frame. The argument skip is the number of stack
+// frames to ascend, with 0 identifying the caller of getCallerFrame. The
+// boolean ok is false if it was not possible to recover the information.
+//
+// Note: This implementation is similar to runtime.Caller, but it returns the whole frame.
+func getCallerFrame(skip int) (frame runtime.Frame, ok bool) {
+	const skipOffset = 2 // skip getCallerFrame and Callers
+
+	pc := make([]uintptr, 1)
+	numFrames := runtime.Callers(skip+skipOffset, pc)
+	if numFrames < 1 {
+		return
+	}
+
+	frame, _ = runtime.CallersFrames(pc).Next()
+	return frame, frame.PC != 0
+}
diff --git a/vendor/go.uber.org/zap/options.go b/vendor/go.uber.org/zap/options.go
index 7a6b0fc..e9e6616 100644
--- a/vendor/go.uber.org/zap/options.go
+++ b/vendor/go.uber.org/zap/options.go
@@ -20,7 +20,11 @@
 
 package zap
 
-import "go.uber.org/zap/zapcore"
+import (
+	"fmt"
+
+	"go.uber.org/zap/zapcore"
+)
 
 // An Option configures a Logger.
 type Option interface {
@@ -82,11 +86,18 @@
 	})
 }
 
-// AddCaller configures the Logger to annotate each message with the filename
-// and line number of zap's caller.
+// AddCaller configures the Logger to annotate each message with the filename,
+// line number, and function name of zap's caller. See also WithCaller.
 func AddCaller() Option {
+	return WithCaller(true)
+}
+
+// WithCaller configures the Logger to annotate each message with the filename,
+// line number, and function name of zap's caller, or not, depending on the
+// value of enabled. This is a generalized form of AddCaller.
+func WithCaller(enabled bool) Option {
 	return optionFunc(func(log *Logger) {
-		log.addCaller = true
+		log.addCaller = enabled
 	})
 }
 
@@ -107,3 +118,31 @@
 		log.addStack = lvl
 	})
 }
+
+// IncreaseLevel increase the level of the logger. It has no effect if
+// the passed in level tries to decrease the level of the logger.
+func IncreaseLevel(lvl zapcore.LevelEnabler) Option {
+	return optionFunc(func(log *Logger) {
+		core, err := zapcore.NewIncreaseLevelCore(log.core, lvl)
+		if err != nil {
+			fmt.Fprintf(log.errorOutput, "failed to IncreaseLevel: %v\n", err)
+		} else {
+			log.core = core
+		}
+	})
+}
+
+// OnFatal sets the action to take on fatal logs.
+func OnFatal(action zapcore.CheckWriteAction) Option {
+	return optionFunc(func(log *Logger) {
+		log.onFatal = action
+	})
+}
+
+// WithClock specifies the clock used by the logger to determine the current
+// time for logged entries. Defaults to the system clock with time.Now.
+func WithClock(clock zapcore.Clock) Option {
+	return optionFunc(func(log *Logger) {
+		log.clock = clock
+	})
+}
diff --git a/vendor/go.uber.org/zap/sink.go b/vendor/go.uber.org/zap/sink.go
index ff0becf..df46fa8 100644
--- a/vendor/go.uber.org/zap/sink.go
+++ b/vendor/go.uber.org/zap/sink.go
@@ -136,7 +136,7 @@
 	case "stderr":
 		return nopCloserSink{os.Stderr}, nil
 	}
-	return os.OpenFile(u.Path, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0644)
+	return os.OpenFile(u.Path, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0666)
 }
 
 func normalizeScheme(s string) (string, error) {
diff --git a/vendor/go.uber.org/zap/stacktrace.go b/vendor/go.uber.org/zap/stacktrace.go
index 100fac2..0cf8c1d 100644
--- a/vendor/go.uber.org/zap/stacktrace.go
+++ b/vendor/go.uber.org/zap/stacktrace.go
@@ -22,28 +22,20 @@
 
 import (
 	"runtime"
-	"strings"
 	"sync"
 
 	"go.uber.org/zap/internal/bufferpool"
 )
 
-const _zapPackage = "go.uber.org/zap"
-
 var (
 	_stacktracePool = sync.Pool{
 		New: func() interface{} {
 			return newProgramCounters(64)
 		},
 	}
-
-	// We add "." and "/" suffixes to the package name to ensure we only match
-	// the exact package and not any package with the same prefix.
-	_zapStacktracePrefixes       = addPrefix(_zapPackage, ".", "/")
-	_zapStacktraceVendorContains = addPrefix("/vendor/", _zapStacktracePrefixes...)
 )
 
-func takeStacktrace() string {
+func takeStacktrace(skip int) string {
 	buffer := bufferpool.Get()
 	defer buffer.Free()
 	programCounters := _stacktracePool.Get().(*programCounters)
@@ -51,9 +43,9 @@
 
 	var numFrames int
 	for {
-		// Skip the call to runtime.Counters and takeStacktrace so that the
+		// Skip the call to runtime.Callers and takeStacktrace so that the
 		// program counters start at the caller of takeStacktrace.
-		numFrames = runtime.Callers(2, programCounters.pcs)
+		numFrames = runtime.Callers(skip+2, programCounters.pcs)
 		if numFrames < len(programCounters.pcs) {
 			break
 		}
@@ -63,19 +55,12 @@
 	}
 
 	i := 0
-	skipZapFrames := true // skip all consecutive zap frames at the beginning.
 	frames := runtime.CallersFrames(programCounters.pcs[:numFrames])
 
 	// Note: On the last iteration, frames.Next() returns false, with a valid
 	// frame, but we ignore this frame. The last frame is a a runtime frame which
 	// adds noise, since it's only either runtime.main or runtime.goexit.
 	for frame, more := frames.Next(); more; frame, more = frames.Next() {
-		if skipZapFrames && isZapFrame(frame.Function) {
-			continue
-		} else {
-			skipZapFrames = false
-		}
-
 		if i != 0 {
 			buffer.AppendByte('\n')
 		}
@@ -91,24 +76,6 @@
 	return buffer.String()
 }
 
-func isZapFrame(function string) bool {
-	for _, prefix := range _zapStacktracePrefixes {
-		if strings.HasPrefix(function, prefix) {
-			return true
-		}
-	}
-
-	// We can't use a prefix match here since the location of the vendor
-	// directory affects the prefix. Instead we do a contains match.
-	for _, contains := range _zapStacktraceVendorContains {
-		if strings.Contains(function, contains) {
-			return true
-		}
-	}
-
-	return false
-}
-
 type programCounters struct {
 	pcs []uintptr
 }
@@ -116,11 +83,3 @@
 func newProgramCounters(size int) *programCounters {
 	return &programCounters{make([]uintptr, size)}
 }
-
-func addPrefix(prefix string, ss ...string) []string {
-	withPrefix := make([]string, len(ss))
-	for i, s := range ss {
-		withPrefix[i] = prefix + s
-	}
-	return withPrefix
-}
diff --git a/vendor/go.uber.org/zap/sugar.go b/vendor/go.uber.org/zap/sugar.go
index 77ca227..0b96519 100644
--- a/vendor/go.uber.org/zap/sugar.go
+++ b/vendor/go.uber.org/zap/sugar.go
@@ -222,19 +222,30 @@
 		return
 	}
 
-	// Format with Sprint, Sprintf, or neither.
-	msg := template
-	if msg == "" && len(fmtArgs) > 0 {
-		msg = fmt.Sprint(fmtArgs...)
-	} else if msg != "" && len(fmtArgs) > 0 {
-		msg = fmt.Sprintf(template, fmtArgs...)
-	}
-
+	msg := getMessage(template, fmtArgs)
 	if ce := s.base.Check(lvl, msg); ce != nil {
 		ce.Write(s.sweetenFields(context)...)
 	}
 }
 
+// getMessage format with Sprint, Sprintf, or neither.
+func getMessage(template string, fmtArgs []interface{}) string {
+	if len(fmtArgs) == 0 {
+		return template
+	}
+
+	if template != "" {
+		return fmt.Sprintf(template, fmtArgs...)
+	}
+
+	if len(fmtArgs) == 1 {
+		if str, ok := fmtArgs[0].(string); ok {
+			return str
+		}
+	}
+	return fmt.Sprint(fmtArgs...)
+}
+
 func (s *SugaredLogger) sweetenFields(args []interface{}) []Field {
 	if len(args) == 0 {
 		return nil
@@ -255,7 +266,7 @@
 
 		// Make sure this element isn't a dangling key.
 		if i == len(args)-1 {
-			s.base.DPanic(_oddNumberErrMsg, Any("ignored", args[i]))
+			s.base.Error(_oddNumberErrMsg, Any("ignored", args[i]))
 			break
 		}
 
@@ -276,7 +287,7 @@
 
 	// If we encountered any invalid key-value pairs, log an error.
 	if len(invalid) > 0 {
-		s.base.DPanic(_nonStringKeyErrMsg, Array("invalid", invalid))
+		s.base.Error(_nonStringKeyErrMsg, Array("invalid", invalid))
 	}
 	return fields
 }
diff --git a/vendor/go.uber.org/zap/zapcore/buffered_write_syncer.go b/vendor/go.uber.org/zap/zapcore/buffered_write_syncer.go
new file mode 100644
index 0000000..0c1436f
--- /dev/null
+++ b/vendor/go.uber.org/zap/zapcore/buffered_write_syncer.go
@@ -0,0 +1,188 @@
+// Copyright (c) 2021 Uber Technologies, Inc.
+//
+// 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.
+
+package zapcore
+
+import (
+	"bufio"
+	"sync"
+	"time"
+
+	"go.uber.org/multierr"
+)
+
+const (
+	// _defaultBufferSize specifies the default size used by Buffer.
+	_defaultBufferSize = 256 * 1024 // 256 kB
+
+	// _defaultFlushInterval specifies the default flush interval for
+	// Buffer.
+	_defaultFlushInterval = 30 * time.Second
+)
+
+// A BufferedWriteSyncer is a WriteSyncer that buffers writes in-memory before
+// flushing them to a wrapped WriteSyncer after reaching some limit, or at some
+// fixed interval--whichever comes first.
+//
+// BufferedWriteSyncer is safe for concurrent use. You don't need to use
+// zapcore.Lock for WriteSyncers with BufferedWriteSyncer.
+type BufferedWriteSyncer struct {
+	// WS is the WriteSyncer around which BufferedWriteSyncer will buffer
+	// writes.
+	//
+	// This field is required.
+	WS WriteSyncer
+
+	// Size specifies the maximum amount of data the writer will buffered
+	// before flushing.
+	//
+	// Defaults to 256 kB if unspecified.
+	Size int
+
+	// FlushInterval specifies how often the writer should flush data if
+	// there have been no writes.
+	//
+	// Defaults to 30 seconds if unspecified.
+	FlushInterval time.Duration
+
+	// Clock, if specified, provides control of the source of time for the
+	// writer.
+	//
+	// Defaults to the system clock.
+	Clock Clock
+
+	// unexported fields for state
+	mu          sync.Mutex
+	initialized bool // whether initialize() has run
+	writer      *bufio.Writer
+	ticker      *time.Ticker
+	stop        chan struct{} // closed when flushLoop should stop
+	stopped     bool          // whether Stop() has run
+	done        chan struct{} // closed when flushLoop has stopped
+}
+
+func (s *BufferedWriteSyncer) initialize() {
+	size := s.Size
+	if size == 0 {
+		size = _defaultBufferSize
+	}
+
+	flushInterval := s.FlushInterval
+	if flushInterval == 0 {
+		flushInterval = _defaultFlushInterval
+	}
+
+	if s.Clock == nil {
+		s.Clock = DefaultClock
+	}
+
+	s.ticker = s.Clock.NewTicker(flushInterval)
+	s.writer = bufio.NewWriterSize(s.WS, size)
+	s.stop = make(chan struct{})
+	s.done = make(chan struct{})
+	s.initialized = true
+	go s.flushLoop()
+}
+
+// Write writes log data into buffer syncer directly, multiple Write calls will be batched,
+// and log data will be flushed to disk when the buffer is full or periodically.
+func (s *BufferedWriteSyncer) Write(bs []byte) (int, error) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	if !s.initialized {
+		s.initialize()
+	}
+
+	// To avoid partial writes from being flushed, we manually flush the existing buffer if:
+	// * The current write doesn't fit into the buffer fully, and
+	// * The buffer is not empty (since bufio will not split large writes when the buffer is empty)
+	if len(bs) > s.writer.Available() && s.writer.Buffered() > 0 {
+		if err := s.writer.Flush(); err != nil {
+			return 0, err
+		}
+	}
+
+	return s.writer.Write(bs)
+}
+
+// Sync flushes buffered log data into disk directly.
+func (s *BufferedWriteSyncer) Sync() error {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	var err error
+	if s.initialized {
+		err = s.writer.Flush()
+	}
+
+	return multierr.Append(err, s.WS.Sync())
+}
+
+// flushLoop flushes the buffer at the configured interval until Stop is
+// called.
+func (s *BufferedWriteSyncer) flushLoop() {
+	defer close(s.done)
+
+	for {
+		select {
+		case <-s.ticker.C:
+			// we just simply ignore error here
+			// because the underlying bufio writer stores any errors
+			// and we return any error from Sync() as part of the close
+			_ = s.Sync()
+		case <-s.stop:
+			return
+		}
+	}
+}
+
+// Stop closes the buffer, cleans up background goroutines, and flushes
+// remaining unwritten data.
+func (s *BufferedWriteSyncer) Stop() (err error) {
+	var stopped bool
+
+	// Critical section.
+	func() {
+		s.mu.Lock()
+		defer s.mu.Unlock()
+
+		if !s.initialized {
+			return
+		}
+
+		stopped = s.stopped
+		if stopped {
+			return
+		}
+		s.stopped = true
+
+		s.ticker.Stop()
+		close(s.stop) // tell flushLoop to stop
+		<-s.done      // and wait until it has
+	}()
+
+	// Don't call Sync on consecutive Stops.
+	if !stopped {
+		err = s.Sync()
+	}
+
+	return err
+}
diff --git a/vendor/go.uber.org/zap/zapcore/clock.go b/vendor/go.uber.org/zap/zapcore/clock.go
new file mode 100644
index 0000000..d2ea95b
--- /dev/null
+++ b/vendor/go.uber.org/zap/zapcore/clock.go
@@ -0,0 +1,50 @@
+// Copyright (c) 2021 Uber Technologies, Inc.
+//
+// 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.
+
+package zapcore
+
+import (
+	"time"
+)
+
+// DefaultClock is the default clock used by Zap in operations that require
+// time. This clock uses the system clock for all operations.
+var DefaultClock = systemClock{}
+
+// Clock is a source of time for logged entries.
+type Clock interface {
+	// Now returns the current local time.
+	Now() time.Time
+
+	// NewTicker returns *time.Ticker that holds a channel
+	// that delivers "ticks" of a clock.
+	NewTicker(time.Duration) *time.Ticker
+}
+
+// systemClock implements default Clock that uses system time.
+type systemClock struct{}
+
+func (systemClock) Now() time.Time {
+	return time.Now()
+}
+
+func (systemClock) NewTicker(duration time.Duration) *time.Ticker {
+	return time.NewTicker(duration)
+}
diff --git a/vendor/go.uber.org/zap/zapcore/console_encoder.go b/vendor/go.uber.org/zap/zapcore/console_encoder.go
index b787596..2307af4 100644
--- a/vendor/go.uber.org/zap/zapcore/console_encoder.go
+++ b/vendor/go.uber.org/zap/zapcore/console_encoder.go
@@ -56,6 +56,10 @@
 // encoder configuration, it will omit any element whose key is set to the empty
 // string.
 func NewConsoleEncoder(cfg EncoderConfig) Encoder {
+	if cfg.ConsoleSeparator == "" {
+		// Use a default delimiter of '\t' for backwards compatibility
+		cfg.ConsoleSeparator = "\t"
+	}
 	return consoleEncoder{newJSONEncoder(cfg, true)}
 }
 
@@ -89,12 +93,17 @@
 
 		nameEncoder(ent.LoggerName, arr)
 	}
-	if ent.Caller.Defined && c.CallerKey != "" && c.EncodeCaller != nil {
-		c.EncodeCaller(ent.Caller, arr)
+	if ent.Caller.Defined {
+		if c.CallerKey != "" && c.EncodeCaller != nil {
+			c.EncodeCaller(ent.Caller, arr)
+		}
+		if c.FunctionKey != "" {
+			arr.AppendString(ent.Caller.Function)
+		}
 	}
 	for i := range arr.elems {
 		if i > 0 {
-			line.AppendByte('\t')
+			line.AppendString(c.ConsoleSeparator)
 		}
 		fmt.Fprint(line, arr.elems[i])
 	}
@@ -102,7 +111,7 @@
 
 	// Add the message itself.
 	if c.MessageKey != "" {
-		c.addTabIfNecessary(line)
+		c.addSeparatorIfNecessary(line)
 		line.AppendString(ent.Message)
 	}
 
@@ -126,7 +135,12 @@
 
 func (c consoleEncoder) writeContext(line *buffer.Buffer, extra []Field) {
 	context := c.jsonEncoder.Clone().(*jsonEncoder)
-	defer context.buf.Free()
+	defer func() {
+		// putJSONEncoder assumes the buffer is still used, but we write out the buffer so
+		// we can free it.
+		context.buf.Free()
+		putJSONEncoder(context)
+	}()
 
 	addFields(context, extra)
 	context.closeOpenNamespaces()
@@ -134,14 +148,14 @@
 		return
 	}
 
-	c.addTabIfNecessary(line)
+	c.addSeparatorIfNecessary(line)
 	line.AppendByte('{')
 	line.Write(context.buf.Bytes())
 	line.AppendByte('}')
 }
 
-func (c consoleEncoder) addTabIfNecessary(line *buffer.Buffer) {
+func (c consoleEncoder) addSeparatorIfNecessary(line *buffer.Buffer) {
 	if line.Len() > 0 {
-		line.AppendByte('\t')
+		line.AppendString(c.ConsoleSeparator)
 	}
 }
diff --git a/vendor/go.uber.org/zap/zapcore/encoder.go b/vendor/go.uber.org/zap/zapcore/encoder.go
index f050952..6601ca1 100644
--- a/vendor/go.uber.org/zap/zapcore/encoder.go
+++ b/vendor/go.uber.org/zap/zapcore/encoder.go
@@ -21,6 +21,7 @@
 package zapcore
 
 import (
+	"encoding/json"
 	"time"
 
 	"go.uber.org/zap/buffer"
@@ -31,6 +32,9 @@
 // behavior.
 const DefaultLineEnding = "\n"
 
+// OmitKey defines the key to use when callers want to remove a key from log output.
+const OmitKey = ""
+
 // A LevelEncoder serializes a Level to a primitive type.
 type LevelEncoder func(Level, PrimitiveArrayEncoder)
 
@@ -109,17 +113,66 @@
 	enc.AppendInt64(t.UnixNano())
 }
 
-// ISO8601TimeEncoder serializes a time.Time to an ISO8601-formatted string
-// with millisecond precision.
-func ISO8601TimeEncoder(t time.Time, enc PrimitiveArrayEncoder) {
-	enc.AppendString(t.Format("2006-01-02T15:04:05.000Z0700"))
+func encodeTimeLayout(t time.Time, layout string, enc PrimitiveArrayEncoder) {
+	type appendTimeEncoder interface {
+		AppendTimeLayout(time.Time, string)
+	}
+
+	if enc, ok := enc.(appendTimeEncoder); ok {
+		enc.AppendTimeLayout(t, layout)
+		return
+	}
+
+	enc.AppendString(t.Format(layout))
 }
 
-// UnmarshalText unmarshals text to a TimeEncoder. "iso8601" and "ISO8601" are
-// unmarshaled to ISO8601TimeEncoder, "millis" is unmarshaled to
-// EpochMillisTimeEncoder, and anything else is unmarshaled to EpochTimeEncoder.
+// ISO8601TimeEncoder serializes a time.Time to an ISO8601-formatted string
+// with millisecond precision.
+//
+// If enc supports AppendTimeLayout(t time.Time,layout string), it's used
+// instead of appending a pre-formatted string value.
+func ISO8601TimeEncoder(t time.Time, enc PrimitiveArrayEncoder) {
+	encodeTimeLayout(t, "2006-01-02T15:04:05.000Z0700", enc)
+}
+
+// RFC3339TimeEncoder serializes a time.Time to an RFC3339-formatted string.
+//
+// If enc supports AppendTimeLayout(t time.Time,layout string), it's used
+// instead of appending a pre-formatted string value.
+func RFC3339TimeEncoder(t time.Time, enc PrimitiveArrayEncoder) {
+	encodeTimeLayout(t, time.RFC3339, enc)
+}
+
+// RFC3339NanoTimeEncoder serializes a time.Time to an RFC3339-formatted string
+// with nanosecond precision.
+//
+// If enc supports AppendTimeLayout(t time.Time,layout string), it's used
+// instead of appending a pre-formatted string value.
+func RFC3339NanoTimeEncoder(t time.Time, enc PrimitiveArrayEncoder) {
+	encodeTimeLayout(t, time.RFC3339Nano, enc)
+}
+
+// TimeEncoderOfLayout returns TimeEncoder which serializes a time.Time using
+// given layout.
+func TimeEncoderOfLayout(layout string) TimeEncoder {
+	return func(t time.Time, enc PrimitiveArrayEncoder) {
+		encodeTimeLayout(t, layout, enc)
+	}
+}
+
+// UnmarshalText unmarshals text to a TimeEncoder.
+// "rfc3339nano" and "RFC3339Nano" are unmarshaled to RFC3339NanoTimeEncoder.
+// "rfc3339" and "RFC3339" are unmarshaled to RFC3339TimeEncoder.
+// "iso8601" and "ISO8601" are unmarshaled to ISO8601TimeEncoder.
+// "millis" is unmarshaled to EpochMillisTimeEncoder.
+// "nanos" is unmarshaled to EpochNanosEncoder.
+// Anything else is unmarshaled to EpochTimeEncoder.
 func (e *TimeEncoder) UnmarshalText(text []byte) error {
 	switch string(text) {
+	case "rfc3339nano", "RFC3339Nano":
+		*e = RFC3339NanoTimeEncoder
+	case "rfc3339", "RFC3339":
+		*e = RFC3339TimeEncoder
 	case "iso8601", "ISO8601":
 		*e = ISO8601TimeEncoder
 	case "millis":
@@ -132,6 +185,35 @@
 	return nil
 }
 
+// UnmarshalYAML unmarshals YAML to a TimeEncoder.
+// If value is an object with a "layout" field, it will be unmarshaled to  TimeEncoder with given layout.
+//     timeEncoder:
+//       layout: 06/01/02 03:04pm
+// If value is string, it uses UnmarshalText.
+//     timeEncoder: iso8601
+func (e *TimeEncoder) UnmarshalYAML(unmarshal func(interface{}) error) error {
+	var o struct {
+		Layout string `json:"layout" yaml:"layout"`
+	}
+	if err := unmarshal(&o); err == nil {
+		*e = TimeEncoderOfLayout(o.Layout)
+		return nil
+	}
+
+	var s string
+	if err := unmarshal(&s); err != nil {
+		return err
+	}
+	return e.UnmarshalText([]byte(s))
+}
+
+// UnmarshalJSON unmarshals JSON to a TimeEncoder as same way UnmarshalYAML does.
+func (e *TimeEncoder) UnmarshalJSON(data []byte) error {
+	return e.UnmarshalYAML(func(v interface{}) error {
+		return json.Unmarshal(data, v)
+	})
+}
+
 // A DurationEncoder serializes a time.Duration to a primitive type.
 type DurationEncoder func(time.Duration, PrimitiveArrayEncoder)
 
@@ -146,6 +228,12 @@
 	enc.AppendInt64(int64(d))
 }
 
+// MillisDurationEncoder serializes a time.Duration to an integer number of
+// milliseconds elapsed.
+func MillisDurationEncoder(d time.Duration, enc PrimitiveArrayEncoder) {
+	enc.AppendInt64(d.Nanoseconds() / 1e6)
+}
+
 // StringDurationEncoder serializes a time.Duration using its built-in String
 // method.
 func StringDurationEncoder(d time.Duration, enc PrimitiveArrayEncoder) {
@@ -161,6 +249,8 @@
 		*e = StringDurationEncoder
 	case "nanos":
 		*e = NanosDurationEncoder
+	case "ms":
+		*e = MillisDurationEncoder
 	default:
 		*e = SecondsDurationEncoder
 	}
@@ -227,6 +317,7 @@
 	TimeKey       string `json:"timeKey" yaml:"timeKey"`
 	NameKey       string `json:"nameKey" yaml:"nameKey"`
 	CallerKey     string `json:"callerKey" yaml:"callerKey"`
+	FunctionKey   string `json:"functionKey" yaml:"functionKey"`
 	StacktraceKey string `json:"stacktraceKey" yaml:"stacktraceKey"`
 	LineEnding    string `json:"lineEnding" yaml:"lineEnding"`
 	// Configure the primitive representations of common complex types. For
@@ -239,6 +330,9 @@
 	// Unlike the other primitive type encoders, EncodeName is optional. The
 	// zero value falls back to FullNameEncoder.
 	EncodeName NameEncoder `json:"nameEncoder" yaml:"nameEncoder"`
+	// Configures the field separator used by the console encoder. Defaults
+	// to tab.
+	ConsoleSeparator string `json:"consoleSeparator" yaml:"consoleSeparator"`
 }
 
 // ObjectEncoder is a strongly-typed, encoding-agnostic interface for adding a
@@ -272,8 +366,8 @@
 	AddUint8(key string, value uint8)
 	AddUintptr(key string, value uintptr)
 
-	// AddReflected uses reflection to serialize arbitrary objects, so it's slow
-	// and allocation-heavy.
+	// AddReflected uses reflection to serialize arbitrary objects, so it can be
+	// slow and allocation-heavy.
 	AddReflected(key string, value interface{}) error
 	// OpenNamespace opens an isolated namespace where all subsequent fields will
 	// be added. Applications can use namespaces to prevent key collisions when
@@ -343,6 +437,7 @@
 	Clone() Encoder
 
 	// EncodeEntry encodes an entry and fields, along with any accumulated
-	// context, into a byte buffer and returns it.
+	// context, into a byte buffer and returns it. Any fields that are empty,
+	// including fields on the `Entry` type, should be omitted.
 	EncodeEntry(Entry, []Field) (*buffer.Buffer, error)
 }
diff --git a/vendor/go.uber.org/zap/zapcore/entry.go b/vendor/go.uber.org/zap/zapcore/entry.go
index 7d9893f..2d815fe 100644
--- a/vendor/go.uber.org/zap/zapcore/entry.go
+++ b/vendor/go.uber.org/zap/zapcore/entry.go
@@ -22,6 +22,7 @@
 
 import (
 	"fmt"
+	"runtime"
 	"strings"
 	"sync"
 	"time"
@@ -70,10 +71,11 @@
 
 // EntryCaller represents the caller of a logging function.
 type EntryCaller struct {
-	Defined bool
-	PC      uintptr
-	File    string
-	Line    int
+	Defined  bool
+	PC       uintptr
+	File     string
+	Line     int
+	Function string
 }
 
 // String returns the full path and line number of the caller.
@@ -136,7 +138,8 @@
 
 // An Entry represents a complete log message. The entry's structured context
 // is already serialized, but the log level, time, message, and call site
-// information are available for inspection and modification.
+// information are available for inspection and modification. Any fields left
+// empty will be omitted when encoding.
 //
 // Entries are pooled, so any functions that accept them MUST be careful not to
 // retain references to them.
@@ -157,6 +160,8 @@
 	// WriteThenNoop indicates that nothing special needs to be done. It's the
 	// default behavior.
 	WriteThenNoop CheckWriteAction = iota
+	// WriteThenGoexit runs runtime.Goexit after Write.
+	WriteThenGoexit
 	// WriteThenPanic causes a panic after Write.
 	WriteThenPanic
 	// WriteThenFatal causes a fatal os.Exit after Write.
@@ -203,7 +208,7 @@
 			// If the entry is dirty, log an internal error; because the
 			// CheckedEntry is being used after it was returned to the pool,
 			// the message may be an amalgamation from multiple call sites.
-			fmt.Fprintf(ce.ErrorOutput, "%v Unsafe CheckedEntry re-use near Entry %+v.\n", time.Now(), ce.Entry)
+			fmt.Fprintf(ce.ErrorOutput, "%v Unsafe CheckedEntry re-use near Entry %+v.\n", ce.Time, ce.Entry)
 			ce.ErrorOutput.Sync()
 		}
 		return
@@ -216,7 +221,7 @@
 	}
 	if ce.ErrorOutput != nil {
 		if err != nil {
-			fmt.Fprintf(ce.ErrorOutput, "%v write error: %v\n", time.Now(), err)
+			fmt.Fprintf(ce.ErrorOutput, "%v write error: %v\n", ce.Time, err)
 			ce.ErrorOutput.Sync()
 		}
 	}
@@ -229,6 +234,8 @@
 		panic(msg)
 	case WriteThenFatal:
 		exit.Exit()
+	case WriteThenGoexit:
+		runtime.Goexit()
 	}
 }
 
diff --git a/vendor/go.uber.org/zap/zapcore/error.go b/vendor/go.uber.org/zap/zapcore/error.go
index a67c7ba..f2a07d7 100644
--- a/vendor/go.uber.org/zap/zapcore/error.go
+++ b/vendor/go.uber.org/zap/zapcore/error.go
@@ -22,6 +22,7 @@
 
 import (
 	"fmt"
+	"reflect"
 	"sync"
 )
 
@@ -42,7 +43,23 @@
 //      ...
 //    ],
 //  }
-func encodeError(key string, err error, enc ObjectEncoder) error {
+func encodeError(key string, err error, enc ObjectEncoder) (retErr error) {
+	// Try to capture panics (from nil references or otherwise) when calling
+	// the Error() method
+	defer func() {
+		if rerr := recover(); rerr != nil {
+			// If it's a nil pointer, just say "<nil>". The likeliest causes are a
+			// error that fails to guard against nil or a nil pointer for a
+			// value receiver, and in either case, "<nil>" is a nice result.
+			if v := reflect.ValueOf(err); v.Kind() == reflect.Ptr && v.IsNil() {
+				enc.AddString(key, "<nil>")
+				return
+			}
+
+			retErr = fmt.Errorf("PANIC=%v", rerr)
+		}
+	}()
+
 	basic := err.Error()
 	enc.AddString(key, basic)
 
@@ -66,11 +83,6 @@
 	Errors() []error
 }
 
-type causer interface {
-	// Provides access to the error that caused this error.
-	Cause() error
-}
-
 // Note that errArry and errArrayElem are very similar to the version
 // implemented in the top-level error.go file. We can't re-use this because
 // that would require exporting errArray as part of the zapcore API.
diff --git a/vendor/go.uber.org/zap/zapcore/field.go b/vendor/go.uber.org/zap/zapcore/field.go
index ae772e4..95bdb0a 100644
--- a/vendor/go.uber.org/zap/zapcore/field.go
+++ b/vendor/go.uber.org/zap/zapcore/field.go
@@ -65,8 +65,11 @@
 	Int8Type
 	// StringType indicates that the field carries a string.
 	StringType
-	// TimeType indicates that the field carries a time.Time.
+	// TimeType indicates that the field carries a time.Time that is
+	// representable by a UnixNano() stored as an int64.
 	TimeType
+	// TimeFullType indicates that the field carries a time.Time stored as-is.
+	TimeFullType
 	// Uint64Type indicates that the field carries a uint64.
 	Uint64Type
 	// Uint32Type indicates that the field carries a uint32.
@@ -89,6 +92,10 @@
 	ErrorType
 	// SkipType indicates that the field is a no-op.
 	SkipType
+
+	// InlineMarshalerType indicates that the field carries an ObjectMarshaler
+	// that should be inlined.
+	InlineMarshalerType
 )
 
 // A Field is a marshaling operation used to add a key-value pair to a logger's
@@ -112,6 +119,8 @@
 		err = enc.AddArray(f.Key, f.Interface.(ArrayMarshaler))
 	case ObjectMarshalerType:
 		err = enc.AddObject(f.Key, f.Interface.(ObjectMarshaler))
+	case InlineMarshalerType:
+		err = f.Interface.(ObjectMarshaler).MarshalLogObject(enc)
 	case BinaryType:
 		enc.AddBinary(f.Key, f.Interface.([]byte))
 	case BoolType:
@@ -145,6 +154,8 @@
 			// Fall back to UTC if location is nil.
 			enc.AddTime(f.Key, time.Unix(0, f.Integer))
 		}
+	case TimeFullType:
+		enc.AddTime(f.Key, f.Interface.(time.Time))
 	case Uint64Type:
 		enc.AddUint64(f.Key, uint64(f.Integer))
 	case Uint32Type:
@@ -162,7 +173,7 @@
 	case StringerType:
 		err = encodeStringer(f.Key, f.Interface, enc)
 	case ErrorType:
-		encodeError(f.Key, f.Interface.(error), enc)
+		err = encodeError(f.Key, f.Interface.(error), enc)
 	case SkipType:
 		break
 	default:
@@ -200,13 +211,23 @@
 	}
 }
 
-func encodeStringer(key string, stringer interface{}, enc ObjectEncoder) (err error) {
+func encodeStringer(key string, stringer interface{}, enc ObjectEncoder) (retErr error) {
+	// Try to capture panics (from nil references or otherwise) when calling
+	// the String() method, similar to https://golang.org/src/fmt/print.go#L540
 	defer func() {
-		if v := recover(); v != nil {
-			err = fmt.Errorf("PANIC=%v", v)
+		if err := recover(); err != nil {
+			// If it's a nil pointer, just say "<nil>". The likeliest causes are a
+			// Stringer that fails to guard against nil or a nil pointer for a
+			// value receiver, and in either case, "<nil>" is a nice result.
+			if v := reflect.ValueOf(stringer); v.Kind() == reflect.Ptr && v.IsNil() {
+				enc.AddString(key, "<nil>")
+				return
+			}
+
+			retErr = fmt.Errorf("PANIC=%v", err)
 		}
 	}()
 
 	enc.AddString(key, stringer.(fmt.Stringer).String())
-	return
+	return nil
 }
diff --git a/vendor/go.uber.org/zap/zapcore/increase_level.go b/vendor/go.uber.org/zap/zapcore/increase_level.go
new file mode 100644
index 0000000..5a17492
--- /dev/null
+++ b/vendor/go.uber.org/zap/zapcore/increase_level.go
@@ -0,0 +1,66 @@
+// Copyright (c) 2020 Uber Technologies, Inc.
+//
+// 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.
+
+package zapcore
+
+import "fmt"
+
+type levelFilterCore struct {
+	core  Core
+	level LevelEnabler
+}
+
+// NewIncreaseLevelCore creates a core that can be used to increase the level of
+// an existing Core. It cannot be used to decrease the logging level, as it acts
+// as a filter before calling the underlying core. If level decreases the log level,
+// an error is returned.
+func NewIncreaseLevelCore(core Core, level LevelEnabler) (Core, error) {
+	for l := _maxLevel; l >= _minLevel; l-- {
+		if !core.Enabled(l) && level.Enabled(l) {
+			return nil, fmt.Errorf("invalid increase level, as level %q is allowed by increased level, but not by existing core", l)
+		}
+	}
+
+	return &levelFilterCore{core, level}, nil
+}
+
+func (c *levelFilterCore) Enabled(lvl Level) bool {
+	return c.level.Enabled(lvl)
+}
+
+func (c *levelFilterCore) With(fields []Field) Core {
+	return &levelFilterCore{c.core.With(fields), c.level}
+}
+
+func (c *levelFilterCore) Check(ent Entry, ce *CheckedEntry) *CheckedEntry {
+	if !c.Enabled(ent.Level) {
+		return ce
+	}
+
+	return c.core.Check(ent, ce)
+}
+
+func (c *levelFilterCore) Write(ent Entry, fields []Field) error {
+	return c.core.Write(ent, fields)
+}
+
+func (c *levelFilterCore) Sync() error {
+	return c.core.Sync()
+}
diff --git a/vendor/go.uber.org/zap/zapcore/json_encoder.go b/vendor/go.uber.org/zap/zapcore/json_encoder.go
index 9aec4ea..5cf7d91 100644
--- a/vendor/go.uber.org/zap/zapcore/json_encoder.go
+++ b/vendor/go.uber.org/zap/zapcore/json_encoder.go
@@ -145,15 +145,29 @@
 	}
 }
 
-func (enc *jsonEncoder) AddReflected(key string, obj interface{}) error {
+var nullLiteralBytes = []byte("null")
+
+// Only invoke the standard JSON encoder if there is actually something to
+// encode; otherwise write JSON null literal directly.
+func (enc *jsonEncoder) encodeReflected(obj interface{}) ([]byte, error) {
+	if obj == nil {
+		return nullLiteralBytes, nil
+	}
 	enc.resetReflectBuf()
-	err := enc.reflectEnc.Encode(obj)
+	if err := enc.reflectEnc.Encode(obj); err != nil {
+		return nil, err
+	}
+	enc.reflectBuf.TrimNewline()
+	return enc.reflectBuf.Bytes(), nil
+}
+
+func (enc *jsonEncoder) AddReflected(key string, obj interface{}) error {
+	valueBytes, err := enc.encodeReflected(obj)
 	if err != nil {
 		return err
 	}
-	enc.reflectBuf.TrimNewline()
 	enc.addKey(key)
-	_, err = enc.buf.Write(enc.reflectBuf.Bytes())
+	_, err = enc.buf.Write(valueBytes)
 	return err
 }
 
@@ -222,7 +236,9 @@
 
 func (enc *jsonEncoder) AppendDuration(val time.Duration) {
 	cur := enc.buf.Len()
-	enc.EncodeDuration(val, enc)
+	if e := enc.EncodeDuration; e != nil {
+		e(val, enc)
+	}
 	if cur == enc.buf.Len() {
 		// User-supplied EncodeDuration is a no-op. Fall back to nanoseconds to keep
 		// JSON valid.
@@ -236,14 +252,12 @@
 }
 
 func (enc *jsonEncoder) AppendReflected(val interface{}) error {
-	enc.resetReflectBuf()
-	err := enc.reflectEnc.Encode(val)
+	valueBytes, err := enc.encodeReflected(val)
 	if err != nil {
 		return err
 	}
-	enc.reflectBuf.TrimNewline()
 	enc.addElementSeparator()
-	_, err = enc.buf.Write(enc.reflectBuf.Bytes())
+	_, err = enc.buf.Write(valueBytes)
 	return err
 }
 
@@ -254,9 +268,18 @@
 	enc.buf.AppendByte('"')
 }
 
+func (enc *jsonEncoder) AppendTimeLayout(time time.Time, layout string) {
+	enc.addElementSeparator()
+	enc.buf.AppendByte('"')
+	enc.buf.AppendTime(time, layout)
+	enc.buf.AppendByte('"')
+}
+
 func (enc *jsonEncoder) AppendTime(val time.Time) {
 	cur := enc.buf.Len()
-	enc.EncodeTime(val, enc)
+	if e := enc.EncodeTime; e != nil {
+		e(val, enc)
+	}
 	if cur == enc.buf.Len() {
 		// User-supplied EncodeTime is a no-op. Fall back to nanos since epoch to keep
 		// output JSON valid.
@@ -343,14 +366,20 @@
 			final.AppendString(ent.LoggerName)
 		}
 	}
-	if ent.Caller.Defined && final.CallerKey != "" {
-		final.addKey(final.CallerKey)
-		cur := final.buf.Len()
-		final.EncodeCaller(ent.Caller, final)
-		if cur == final.buf.Len() {
-			// User-supplied EncodeCaller was a no-op. Fall back to strings to
-			// keep output JSON valid.
-			final.AppendString(ent.Caller.String())
+	if ent.Caller.Defined {
+		if final.CallerKey != "" {
+			final.addKey(final.CallerKey)
+			cur := final.buf.Len()
+			final.EncodeCaller(ent.Caller, final)
+			if cur == final.buf.Len() {
+				// User-supplied EncodeCaller was a no-op. Fall back to strings to
+				// keep output JSON valid.
+				final.AppendString(ent.Caller.String())
+			}
+		}
+		if final.FunctionKey != "" {
+			final.addKey(final.FunctionKey)
+			final.AppendString(ent.Caller.Function)
 		}
 	}
 	if final.MessageKey != "" {
diff --git a/vendor/go.uber.org/zap/zapcore/marshaler.go b/vendor/go.uber.org/zap/zapcore/marshaler.go
index 2627a65..c3c55ba 100644
--- a/vendor/go.uber.org/zap/zapcore/marshaler.go
+++ b/vendor/go.uber.org/zap/zapcore/marshaler.go
@@ -23,6 +23,10 @@
 // ObjectMarshaler allows user-defined types to efficiently add themselves to the
 // logging context, and to selectively omit information which shouldn't be
 // included in logs (e.g., passwords).
+//
+// Note: ObjectMarshaler is only used when zap.Object is used or when
+// passed directly to zap.Any. It is not used when reflection-based
+// encoding is used.
 type ObjectMarshaler interface {
 	MarshalLogObject(ObjectEncoder) error
 }
@@ -39,6 +43,10 @@
 // ArrayMarshaler allows user-defined types to efficiently add themselves to the
 // logging context, and to selectively omit information which shouldn't be
 // included in logs (e.g., passwords).
+//
+// Note: ArrayMarshaler is only used when zap.Array is used or when
+// passed directly to zap.Any. It is not used when reflection-based
+// encoding is used.
 type ArrayMarshaler interface {
 	MarshalLogArray(ArrayEncoder) error
 }
diff --git a/vendor/go.uber.org/zap/zapcore/sampler.go b/vendor/go.uber.org/zap/zapcore/sampler.go
index e316418..25f10ca 100644
--- a/vendor/go.uber.org/zap/zapcore/sampler.go
+++ b/vendor/go.uber.org/zap/zapcore/sampler.go
@@ -81,17 +81,92 @@
 	return 1
 }
 
+// SamplingDecision is a decision represented as a bit field made by sampler.
+// More decisions may be added in the future.
+type SamplingDecision uint32
+
+const (
+	// LogDropped indicates that the Sampler dropped a log entry.
+	LogDropped SamplingDecision = 1 << iota
+	// LogSampled indicates that the Sampler sampled a log entry.
+	LogSampled
+)
+
+// optionFunc wraps a func so it satisfies the SamplerOption interface.
+type optionFunc func(*sampler)
+
+func (f optionFunc) apply(s *sampler) {
+	f(s)
+}
+
+// SamplerOption configures a Sampler.
+type SamplerOption interface {
+	apply(*sampler)
+}
+
+// nopSamplingHook is the default hook used by sampler.
+func nopSamplingHook(Entry, SamplingDecision) {}
+
+// SamplerHook registers a function  which will be called when Sampler makes a
+// decision.
+//
+// This hook may be used to get visibility into the performance of the sampler.
+// For example, use it to track metrics of dropped versus sampled logs.
+//
+//  var dropped atomic.Int64
+//  zapcore.SamplerHook(func(ent zapcore.Entry, dec zapcore.SamplingDecision) {
+//    if dec&zapcore.LogDropped > 0 {
+//      dropped.Inc()
+//    }
+//  })
+func SamplerHook(hook func(entry Entry, dec SamplingDecision)) SamplerOption {
+	return optionFunc(func(s *sampler) {
+		s.hook = hook
+	})
+}
+
+// NewSamplerWithOptions creates a Core that samples incoming entries, which
+// caps the CPU and I/O load of logging while attempting to preserve a
+// representative subset of your logs.
+//
+// Zap samples by logging the first N entries with a given level and message
+// each tick. If more Entries with the same level and message are seen during
+// the same interval, every Mth message is logged and the rest are dropped.
+//
+// Sampler can be configured to report sampling decisions with the SamplerHook
+// option.
+//
+// Keep in mind that zap's sampling implementation is optimized for speed over
+// absolute precision; under load, each tick may be slightly over- or
+// under-sampled.
+func NewSamplerWithOptions(core Core, tick time.Duration, first, thereafter int, opts ...SamplerOption) Core {
+	s := &sampler{
+		Core:       core,
+		tick:       tick,
+		counts:     newCounters(),
+		first:      uint64(first),
+		thereafter: uint64(thereafter),
+		hook:       nopSamplingHook,
+	}
+	for _, opt := range opts {
+		opt.apply(s)
+	}
+
+	return s
+}
+
 type sampler struct {
 	Core
 
 	counts            *counters
 	tick              time.Duration
 	first, thereafter uint64
+	hook              func(Entry, SamplingDecision)
 }
 
-// NewSampler creates a Core that samples incoming entries, which caps the CPU
-// and I/O load of logging while attempting to preserve a representative subset
-// of your logs.
+// NewSampler creates a Core that samples incoming entries, which
+// caps the CPU and I/O load of logging while attempting to preserve a
+// representative subset of your logs.
 //
 // Zap samples by logging the first N entries with a given level and message
 // each tick. If more Entries with the same level and message are seen during
@@ -100,14 +175,10 @@
 // Keep in mind that zap's sampling implementation is optimized for speed over
 // absolute precision; under load, each tick may be slightly over- or
 // under-sampled.
+//
+// Deprecated: use NewSamplerWithOptions.
 func NewSampler(core Core, tick time.Duration, first, thereafter int) Core {
-	return &sampler{
-		Core:       core,
-		tick:       tick,
-		counts:     newCounters(),
-		first:      uint64(first),
-		thereafter: uint64(thereafter),
-	}
+	return NewSamplerWithOptions(core, tick, first, thereafter)
 }
 
 func (s *sampler) With(fields []Field) Core {
@@ -117,6 +188,7 @@
 		counts:     s.counts,
 		first:      s.first,
 		thereafter: s.thereafter,
+		hook:       s.hook,
 	}
 }
 
@@ -128,7 +200,9 @@
 	counter := s.counts.get(ent.Level, ent.Message)
 	n := counter.IncCheckReset(ent.Time, s.tick)
 	if n > s.first && (n-s.first)%s.thereafter != 0 {
+		s.hook(ent, LogDropped)
 		return ce
 	}
+	s.hook(ent, LogSampled)
 	return s.Core.Check(ent, ce)
 }
diff --git a/vendor/go.uber.org/zap/zapcore/write_syncer.go b/vendor/go.uber.org/zap/zapcore/write_syncer.go
index 209e25f..d4a1af3 100644
--- a/vendor/go.uber.org/zap/zapcore/write_syncer.go
+++ b/vendor/go.uber.org/zap/zapcore/write_syncer.go
@@ -91,8 +91,7 @@
 	if len(ws) == 1 {
 		return ws[0]
 	}
-	// Copy to protect against https://github.com/golang/go/issues/7809
-	return multiWriteSyncer(append([]WriteSyncer(nil), ws...))
+	return multiWriteSyncer(ws)
 }
 
 // See https://golang.org/src/io/multi.go
diff --git a/vendor/golang.org/x/net/context/context.go b/vendor/golang.org/x/net/context/context.go
deleted file mode 100644
index a3c021d..0000000
--- a/vendor/golang.org/x/net/context/context.go
+++ /dev/null
@@ -1,56 +0,0 @@
-// Copyright 2014 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 context defines the Context type, which carries deadlines,
-// cancelation signals, and other request-scoped values across API boundaries
-// and between processes.
-// As of Go 1.7 this package is available in the standard library under the
-// name context.  https://golang.org/pkg/context.
-//
-// Incoming requests to a server should create a Context, and outgoing calls to
-// servers should accept a Context. The chain of function calls between must
-// propagate the Context, optionally replacing it with a modified copy created
-// using WithDeadline, WithTimeout, WithCancel, or WithValue.
-//
-// Programs that use Contexts should follow these rules to keep interfaces
-// consistent across packages and enable static analysis tools to check context
-// propagation:
-//
-// Do not store Contexts inside a struct type; instead, pass a Context
-// explicitly to each function that needs it. The Context should be the first
-// parameter, typically named ctx:
-//
-// 	func DoSomething(ctx context.Context, arg Arg) error {
-// 		// ... use ctx ...
-// 	}
-//
-// Do not pass a nil Context, even if a function permits it. Pass context.TODO
-// if you are unsure about which Context to use.
-//
-// Use context Values only for request-scoped data that transits processes and
-// APIs, not for passing optional parameters to functions.
-//
-// The same Context may be passed to functions running in different goroutines;
-// Contexts are safe for simultaneous use by multiple goroutines.
-//
-// See http://blog.golang.org/context for example code for a server that uses
-// Contexts.
-package context // import "golang.org/x/net/context"
-
-// Background returns a non-nil, empty Context. It is never canceled, has no
-// values, and has no deadline. It is typically used by the main function,
-// initialization, and tests, and as the top-level Context for incoming
-// requests.
-func Background() Context {
-	return background
-}
-
-// TODO returns a non-nil, empty Context. Code should use context.TODO when
-// it's unclear which Context to use or it is not yet available (because the
-// surrounding function has not yet been extended to accept a Context
-// parameter).  TODO is recognized by static analysis tools that determine
-// whether Contexts are propagated correctly in a program.
-func TODO() Context {
-	return todo
-}
diff --git a/vendor/golang.org/x/net/context/go17.go b/vendor/golang.org/x/net/context/go17.go
deleted file mode 100644
index d20f52b..0000000
--- a/vendor/golang.org/x/net/context/go17.go
+++ /dev/null
@@ -1,72 +0,0 @@
-// 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.
-
-// +build go1.7
-
-package context
-
-import (
-	"context" // standard library's context, as of Go 1.7
-	"time"
-)
-
-var (
-	todo       = context.TODO()
-	background = context.Background()
-)
-
-// Canceled is the error returned by Context.Err when the context is canceled.
-var Canceled = context.Canceled
-
-// DeadlineExceeded is the error returned by Context.Err when the context's
-// deadline passes.
-var DeadlineExceeded = context.DeadlineExceeded
-
-// WithCancel returns a copy of parent with a new Done channel. The returned
-// context's Done channel is closed when the returned cancel function is called
-// or when the parent context's Done channel is closed, whichever happens first.
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete.
-func WithCancel(parent Context) (ctx Context, cancel CancelFunc) {
-	ctx, f := context.WithCancel(parent)
-	return ctx, CancelFunc(f)
-}
-
-// WithDeadline returns a copy of the parent context with the deadline adjusted
-// to be no later than d. If the parent's deadline is already earlier than d,
-// WithDeadline(parent, d) is semantically equivalent to parent. The returned
-// context's Done channel is closed when the deadline expires, when the returned
-// cancel function is called, or when the parent context's Done channel is
-// closed, whichever happens first.
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete.
-func WithDeadline(parent Context, deadline time.Time) (Context, CancelFunc) {
-	ctx, f := context.WithDeadline(parent, deadline)
-	return ctx, CancelFunc(f)
-}
-
-// WithTimeout returns WithDeadline(parent, time.Now().Add(timeout)).
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete:
-//
-// 	func slowOperationWithTimeout(ctx context.Context) (Result, error) {
-// 		ctx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
-// 		defer cancel()  // releases resources if slowOperation completes before timeout elapses
-// 		return slowOperation(ctx)
-// 	}
-func WithTimeout(parent Context, timeout time.Duration) (Context, CancelFunc) {
-	return WithDeadline(parent, time.Now().Add(timeout))
-}
-
-// WithValue returns a copy of parent in which the value associated with key is
-// val.
-//
-// Use context Values only for request-scoped data that transits processes and
-// APIs, not for passing optional parameters to functions.
-func WithValue(parent Context, key interface{}, val interface{}) Context {
-	return context.WithValue(parent, key, val)
-}
diff --git a/vendor/golang.org/x/net/context/go19.go b/vendor/golang.org/x/net/context/go19.go
deleted file mode 100644
index d88bd1d..0000000
--- a/vendor/golang.org/x/net/context/go19.go
+++ /dev/null
@@ -1,20 +0,0 @@
-// 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.9
-
-package context
-
-import "context" // standard library's context, as of Go 1.7
-
-// A Context carries a deadline, a cancelation signal, and other values across
-// API boundaries.
-//
-// Context's methods may be called by multiple goroutines simultaneously.
-type Context = context.Context
-
-// A CancelFunc tells an operation to abandon its work.
-// A CancelFunc does not wait for the work to stop.
-// After the first call, subsequent calls to a CancelFunc do nothing.
-type CancelFunc = context.CancelFunc
diff --git a/vendor/golang.org/x/net/context/pre_go17.go b/vendor/golang.org/x/net/context/pre_go17.go
deleted file mode 100644
index 0f35592..0000000
--- a/vendor/golang.org/x/net/context/pre_go17.go
+++ /dev/null
@@ -1,300 +0,0 @@
-// Copyright 2014 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 context
-
-import (
-	"errors"
-	"fmt"
-	"sync"
-	"time"
-)
-
-// An emptyCtx is never canceled, has no values, and has no deadline. It is not
-// struct{}, since vars of this type must have distinct addresses.
-type emptyCtx int
-
-func (*emptyCtx) Deadline() (deadline time.Time, ok bool) {
-	return
-}
-
-func (*emptyCtx) Done() <-chan struct{} {
-	return nil
-}
-
-func (*emptyCtx) Err() error {
-	return nil
-}
-
-func (*emptyCtx) Value(key interface{}) interface{} {
-	return nil
-}
-
-func (e *emptyCtx) String() string {
-	switch e {
-	case background:
-		return "context.Background"
-	case todo:
-		return "context.TODO"
-	}
-	return "unknown empty Context"
-}
-
-var (
-	background = new(emptyCtx)
-	todo       = new(emptyCtx)
-)
-
-// Canceled is the error returned by Context.Err when the context is canceled.
-var Canceled = errors.New("context canceled")
-
-// DeadlineExceeded is the error returned by Context.Err when the context's
-// deadline passes.
-var DeadlineExceeded = errors.New("context deadline exceeded")
-
-// WithCancel returns a copy of parent with a new Done channel. The returned
-// context's Done channel is closed when the returned cancel function is called
-// or when the parent context's Done channel is closed, whichever happens first.
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete.
-func WithCancel(parent Context) (ctx Context, cancel CancelFunc) {
-	c := newCancelCtx(parent)
-	propagateCancel(parent, c)
-	return c, func() { c.cancel(true, Canceled) }
-}
-
-// newCancelCtx returns an initialized cancelCtx.
-func newCancelCtx(parent Context) *cancelCtx {
-	return &cancelCtx{
-		Context: parent,
-		done:    make(chan struct{}),
-	}
-}
-
-// propagateCancel arranges for child to be canceled when parent is.
-func propagateCancel(parent Context, child canceler) {
-	if parent.Done() == nil {
-		return // parent is never canceled
-	}
-	if p, ok := parentCancelCtx(parent); ok {
-		p.mu.Lock()
-		if p.err != nil {
-			// parent has already been canceled
-			child.cancel(false, p.err)
-		} else {
-			if p.children == nil {
-				p.children = make(map[canceler]bool)
-			}
-			p.children[child] = true
-		}
-		p.mu.Unlock()
-	} else {
-		go func() {
-			select {
-			case <-parent.Done():
-				child.cancel(false, parent.Err())
-			case <-child.Done():
-			}
-		}()
-	}
-}
-
-// parentCancelCtx follows a chain of parent references until it finds a
-// *cancelCtx. This function understands how each of the concrete types in this
-// package represents its parent.
-func parentCancelCtx(parent Context) (*cancelCtx, bool) {
-	for {
-		switch c := parent.(type) {
-		case *cancelCtx:
-			return c, true
-		case *timerCtx:
-			return c.cancelCtx, true
-		case *valueCtx:
-			parent = c.Context
-		default:
-			return nil, false
-		}
-	}
-}
-
-// removeChild removes a context from its parent.
-func removeChild(parent Context, child canceler) {
-	p, ok := parentCancelCtx(parent)
-	if !ok {
-		return
-	}
-	p.mu.Lock()
-	if p.children != nil {
-		delete(p.children, child)
-	}
-	p.mu.Unlock()
-}
-
-// A canceler is a context type that can be canceled directly. The
-// implementations are *cancelCtx and *timerCtx.
-type canceler interface {
-	cancel(removeFromParent bool, err error)
-	Done() <-chan struct{}
-}
-
-// A cancelCtx can be canceled. When canceled, it also cancels any children
-// that implement canceler.
-type cancelCtx struct {
-	Context
-
-	done chan struct{} // closed by the first cancel call.
-
-	mu       sync.Mutex
-	children map[canceler]bool // set to nil by the first cancel call
-	err      error             // set to non-nil by the first cancel call
-}
-
-func (c *cancelCtx) Done() <-chan struct{} {
-	return c.done
-}
-
-func (c *cancelCtx) Err() error {
-	c.mu.Lock()
-	defer c.mu.Unlock()
-	return c.err
-}
-
-func (c *cancelCtx) String() string {
-	return fmt.Sprintf("%v.WithCancel", c.Context)
-}
-
-// cancel closes c.done, cancels each of c's children, and, if
-// removeFromParent is true, removes c from its parent's children.
-func (c *cancelCtx) cancel(removeFromParent bool, err error) {
-	if err == nil {
-		panic("context: internal error: missing cancel error")
-	}
-	c.mu.Lock()
-	if c.err != nil {
-		c.mu.Unlock()
-		return // already canceled
-	}
-	c.err = err
-	close(c.done)
-	for child := range c.children {
-		// NOTE: acquiring the child's lock while holding parent's lock.
-		child.cancel(false, err)
-	}
-	c.children = nil
-	c.mu.Unlock()
-
-	if removeFromParent {
-		removeChild(c.Context, c)
-	}
-}
-
-// WithDeadline returns a copy of the parent context with the deadline adjusted
-// to be no later than d. If the parent's deadline is already earlier than d,
-// WithDeadline(parent, d) is semantically equivalent to parent. The returned
-// context's Done channel is closed when the deadline expires, when the returned
-// cancel function is called, or when the parent context's Done channel is
-// closed, whichever happens first.
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete.
-func WithDeadline(parent Context, deadline time.Time) (Context, CancelFunc) {
-	if cur, ok := parent.Deadline(); ok && cur.Before(deadline) {
-		// The current deadline is already sooner than the new one.
-		return WithCancel(parent)
-	}
-	c := &timerCtx{
-		cancelCtx: newCancelCtx(parent),
-		deadline:  deadline,
-	}
-	propagateCancel(parent, c)
-	d := deadline.Sub(time.Now())
-	if d <= 0 {
-		c.cancel(true, DeadlineExceeded) // deadline has already passed
-		return c, func() { c.cancel(true, Canceled) }
-	}
-	c.mu.Lock()
-	defer c.mu.Unlock()
-	if c.err == nil {
-		c.timer = time.AfterFunc(d, func() {
-			c.cancel(true, DeadlineExceeded)
-		})
-	}
-	return c, func() { c.cancel(true, Canceled) }
-}
-
-// A timerCtx carries a timer and a deadline. It embeds a cancelCtx to
-// implement Done and Err. It implements cancel by stopping its timer then
-// delegating to cancelCtx.cancel.
-type timerCtx struct {
-	*cancelCtx
-	timer *time.Timer // Under cancelCtx.mu.
-
-	deadline time.Time
-}
-
-func (c *timerCtx) Deadline() (deadline time.Time, ok bool) {
-	return c.deadline, true
-}
-
-func (c *timerCtx) String() string {
-	return fmt.Sprintf("%v.WithDeadline(%s [%s])", c.cancelCtx.Context, c.deadline, c.deadline.Sub(time.Now()))
-}
-
-func (c *timerCtx) cancel(removeFromParent bool, err error) {
-	c.cancelCtx.cancel(false, err)
-	if removeFromParent {
-		// Remove this timerCtx from its parent cancelCtx's children.
-		removeChild(c.cancelCtx.Context, c)
-	}
-	c.mu.Lock()
-	if c.timer != nil {
-		c.timer.Stop()
-		c.timer = nil
-	}
-	c.mu.Unlock()
-}
-
-// WithTimeout returns WithDeadline(parent, time.Now().Add(timeout)).
-//
-// Canceling this context releases resources associated with it, so code should
-// call cancel as soon as the operations running in this Context complete:
-//
-// 	func slowOperationWithTimeout(ctx context.Context) (Result, error) {
-// 		ctx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
-// 		defer cancel()  // releases resources if slowOperation completes before timeout elapses
-// 		return slowOperation(ctx)
-// 	}
-func WithTimeout(parent Context, timeout time.Duration) (Context, CancelFunc) {
-	return WithDeadline(parent, time.Now().Add(timeout))
-}
-
-// WithValue returns a copy of parent in which the value associated with key is
-// val.
-//
-// Use context Values only for request-scoped data that transits processes and
-// APIs, not for passing optional parameters to functions.
-func WithValue(parent Context, key interface{}, val interface{}) Context {
-	return &valueCtx{parent, key, val}
-}
-
-// A valueCtx carries a key-value pair. It implements Value for that key and
-// delegates all other calls to the embedded Context.
-type valueCtx struct {
-	Context
-	key, val interface{}
-}
-
-func (c *valueCtx) String() string {
-	return fmt.Sprintf("%v.WithValue(%#v, %#v)", c.Context, c.key, c.val)
-}
-
-func (c *valueCtx) Value(key interface{}) interface{} {
-	if c.key == key {
-		return c.val
-	}
-	return c.Context.Value(key)
-}
diff --git a/vendor/golang.org/x/net/context/pre_go19.go b/vendor/golang.org/x/net/context/pre_go19.go
deleted file mode 100644
index b105f80..0000000
--- a/vendor/golang.org/x/net/context/pre_go19.go
+++ /dev/null
@@ -1,109 +0,0 @@
-// Copyright 2014 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.9
-
-package context
-
-import "time"
-
-// A Context carries a deadline, a cancelation signal, and other values across
-// API boundaries.
-//
-// Context's methods may be called by multiple goroutines simultaneously.
-type Context interface {
-	// Deadline returns the time when work done on behalf of this context
-	// should be canceled. Deadline returns ok==false when no deadline is
-	// set. Successive calls to Deadline return the same results.
-	Deadline() (deadline time.Time, ok bool)
-
-	// Done returns a channel that's closed when work done on behalf of this
-	// context should be canceled. Done may return nil if this context can
-	// never be canceled. Successive calls to Done return the same value.
-	//
-	// WithCancel arranges for Done to be closed when cancel is called;
-	// WithDeadline arranges for Done to be closed when the deadline
-	// expires; WithTimeout arranges for Done to be closed when the timeout
-	// elapses.
-	//
-	// Done is provided for use in select statements:
-	//
-	//  // Stream generates values with DoSomething and sends them to out
-	//  // until DoSomething returns an error or ctx.Done is closed.
-	//  func Stream(ctx context.Context, out chan<- Value) error {
-	//  	for {
-	//  		v, err := DoSomething(ctx)
-	//  		if err != nil {
-	//  			return err
-	//  		}
-	//  		select {
-	//  		case <-ctx.Done():
-	//  			return ctx.Err()
-	//  		case out <- v:
-	//  		}
-	//  	}
-	//  }
-	//
-	// See http://blog.golang.org/pipelines for more examples of how to use
-	// a Done channel for cancelation.
-	Done() <-chan struct{}
-
-	// Err returns a non-nil error value after Done is closed. Err returns
-	// Canceled if the context was canceled or DeadlineExceeded if the
-	// context's deadline passed. No other values for Err are defined.
-	// After Done is closed, successive calls to Err return the same value.
-	Err() error
-
-	// Value returns the value associated with this context for key, or nil
-	// if no value is associated with key. Successive calls to Value with
-	// the same key returns the same result.
-	//
-	// Use context values only for request-scoped data that transits
-	// processes and API boundaries, not for passing optional parameters to
-	// functions.
-	//
-	// A key identifies a specific value in a Context. Functions that wish
-	// to store values in Context typically allocate a key in a global
-	// variable then use that key as the argument to context.WithValue and
-	// Context.Value. A key can be any type that supports equality;
-	// packages should define keys as an unexported type to avoid
-	// collisions.
-	//
-	// Packages that define a Context key should provide type-safe accessors
-	// for the values stores using that key:
-	//
-	// 	// Package user defines a User type that's stored in Contexts.
-	// 	package user
-	//
-	// 	import "golang.org/x/net/context"
-	//
-	// 	// User is the type of value stored in the Contexts.
-	// 	type User struct {...}
-	//
-	// 	// key is an unexported type for keys defined in this package.
-	// 	// This prevents collisions with keys defined in other packages.
-	// 	type key int
-	//
-	// 	// userKey is the key for user.User values in Contexts. It is
-	// 	// unexported; clients use user.NewContext and user.FromContext
-	// 	// instead of using this key directly.
-	// 	var userKey key = 0
-	//
-	// 	// NewContext returns a new Context that carries value u.
-	// 	func NewContext(ctx context.Context, u *User) context.Context {
-	// 		return context.WithValue(ctx, userKey, u)
-	// 	}
-	//
-	// 	// FromContext returns the User value stored in ctx, if any.
-	// 	func FromContext(ctx context.Context) (*User, bool) {
-	// 		u, ok := ctx.Value(userKey).(*User)
-	// 		return u, ok
-	// 	}
-	Value(key interface{}) interface{}
-}
-
-// A CancelFunc tells an operation to abandon its work.
-// A CancelFunc does not wait for the work to stop.
-// After the first call, subsequent calls to a CancelFunc do nothing.
-type CancelFunc func()
diff --git a/vendor/golang.org/x/net/http/httpguts/httplex.go b/vendor/golang.org/x/net/http/httpguts/httplex.go
index e7de24e..c79aa73 100644
--- a/vendor/golang.org/x/net/http/httpguts/httplex.go
+++ b/vendor/golang.org/x/net/http/httpguts/httplex.go
@@ -137,11 +137,13 @@
 // contains token amongst its comma-separated tokens, ASCII
 // case-insensitively.
 func headerValueContainsToken(v string, token string) bool {
-	v = trimOWS(v)
-	if comma := strings.IndexByte(v, ','); comma != -1 {
-		return tokenEqual(trimOWS(v[:comma]), token) || headerValueContainsToken(v[comma+1:], token)
+	for comma := strings.IndexByte(v, ','); comma != -1; comma = strings.IndexByte(v, ',') {
+		if tokenEqual(trimOWS(v[:comma]), token) {
+			return true
+		}
+		v = v[comma+1:]
 	}
-	return tokenEqual(v, token)
+	return tokenEqual(trimOWS(v), token)
 }
 
 // lowerASCII returns the ASCII lowercase version of b.
diff --git a/vendor/golang.org/x/net/http2/Dockerfile b/vendor/golang.org/x/net/http2/Dockerfile
index 53fc525..8512245 100644
--- a/vendor/golang.org/x/net/http2/Dockerfile
+++ b/vendor/golang.org/x/net/http2/Dockerfile
@@ -38,7 +38,7 @@
 RUN make install
 
 WORKDIR /root
-RUN wget http://curl.haxx.se/download/curl-7.45.0.tar.gz
+RUN wget https://curl.se/download/curl-7.45.0.tar.gz
 RUN tar -zxvf curl-7.45.0.tar.gz
 WORKDIR /root/curl-7.45.0
 RUN ./configure --with-ssl --with-nghttp2=/usr/local
diff --git a/vendor/golang.org/x/net/http2/ascii.go b/vendor/golang.org/x/net/http2/ascii.go
new file mode 100644
index 0000000..17caa20
--- /dev/null
+++ b/vendor/golang.org/x/net/http2/ascii.go
@@ -0,0 +1,53 @@
+// Copyright 2021 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 http2
+
+import "strings"
+
+// The HTTP protocols are defined in terms of ASCII, not Unicode. This file
+// contains helper functions which may use Unicode-aware functions which would
+// otherwise be unsafe and could introduce vulnerabilities if used improperly.
+
+// asciiEqualFold is strings.EqualFold, ASCII only. It reports whether s and t
+// are equal, ASCII-case-insensitively.
+func asciiEqualFold(s, t string) bool {
+	if len(s) != len(t) {
+		return false
+	}
+	for i := 0; i < len(s); i++ {
+		if lower(s[i]) != lower(t[i]) {
+			return false
+		}
+	}
+	return true
+}
+
+// lower returns the ASCII lowercase version of b.
+func lower(b byte) byte {
+	if 'A' <= b && b <= 'Z' {
+		return b + ('a' - 'A')
+	}
+	return b
+}
+
+// isASCIIPrint returns whether s is ASCII and printable according to
+// https://tools.ietf.org/html/rfc20#section-4.2.
+func isASCIIPrint(s string) bool {
+	for i := 0; i < len(s); i++ {
+		if s[i] < ' ' || s[i] > '~' {
+			return false
+		}
+	}
+	return true
+}
+
+// asciiToLower returns the lowercase version of s if s is ASCII and printable,
+// and whether or not it was.
+func asciiToLower(s string) (lower string, ok bool) {
+	if !isASCIIPrint(s) {
+		return "", false
+	}
+	return strings.ToLower(s), true
+}
diff --git a/vendor/golang.org/x/net/http2/client_conn_pool.go b/vendor/golang.org/x/net/http2/client_conn_pool.go
index 3a67636..652bc11 100644
--- a/vendor/golang.org/x/net/http2/client_conn_pool.go
+++ b/vendor/golang.org/x/net/http2/client_conn_pool.go
@@ -7,7 +7,9 @@
 package http2
 
 import (
+	"context"
 	"crypto/tls"
+	"errors"
 	"net/http"
 	"sync"
 )
@@ -78,61 +80,69 @@
 		// It gets its own connection.
 		traceGetConn(req, addr)
 		const singleUse = true
-		cc, err := p.t.dialClientConn(addr, singleUse)
+		cc, err := p.t.dialClientConn(req.Context(), addr, singleUse)
 		if err != nil {
 			return nil, err
 		}
 		return cc, nil
 	}
-	p.mu.Lock()
-	for _, cc := range p.conns[addr] {
-		if st := cc.idleState(); st.canTakeNewRequest {
-			if p.shouldTraceGetConn(st) {
-				traceGetConn(req, addr)
+	for {
+		p.mu.Lock()
+		for _, cc := range p.conns[addr] {
+			if st := cc.idleState(); st.canTakeNewRequest {
+				if p.shouldTraceGetConn(st) {
+					traceGetConn(req, addr)
+				}
+				p.mu.Unlock()
+				return cc, nil
 			}
-			p.mu.Unlock()
-			return cc, nil
 		}
-	}
-	if !dialOnMiss {
+		if !dialOnMiss {
+			p.mu.Unlock()
+			return nil, ErrNoCachedConn
+		}
+		traceGetConn(req, addr)
+		call := p.getStartDialLocked(req.Context(), addr)
 		p.mu.Unlock()
-		return nil, ErrNoCachedConn
+		<-call.done
+		if shouldRetryDial(call, req) {
+			continue
+		}
+		return call.res, call.err
 	}
-	traceGetConn(req, addr)
-	call := p.getStartDialLocked(addr)
-	p.mu.Unlock()
-	<-call.done
-	return call.res, call.err
 }
 
 // dialCall is an in-flight Transport dial call to a host.
 type dialCall struct {
-	_    incomparable
-	p    *clientConnPool
+	_ incomparable
+	p *clientConnPool
+	// the context associated with the request
+	// that created this dialCall
+	ctx  context.Context
 	done chan struct{} // closed when done
 	res  *ClientConn   // valid after done is closed
 	err  error         // valid after done is closed
 }
 
 // requires p.mu is held.
-func (p *clientConnPool) getStartDialLocked(addr string) *dialCall {
+func (p *clientConnPool) getStartDialLocked(ctx context.Context, addr string) *dialCall {
 	if call, ok := p.dialing[addr]; ok {
 		// A dial is already in-flight. Don't start another.
 		return call
 	}
-	call := &dialCall{p: p, done: make(chan struct{})}
+	call := &dialCall{p: p, done: make(chan struct{}), ctx: ctx}
 	if p.dialing == nil {
 		p.dialing = make(map[string]*dialCall)
 	}
 	p.dialing[addr] = call
-	go call.dial(addr)
+	go call.dial(call.ctx, addr)
 	return call
 }
 
 // run in its own goroutine.
-func (c *dialCall) dial(addr string) {
+func (c *dialCall) dial(ctx context.Context, addr string) {
 	const singleUse = false // shared conn
-	c.res, c.err = c.p.t.dialClientConn(addr, singleUse)
+	c.res, c.err = c.p.t.dialClientConn(ctx, addr, singleUse)
 	close(c.done)
 
 	c.p.mu.Lock()
@@ -276,3 +286,28 @@
 func (p noDialClientConnPool) GetClientConn(req *http.Request, addr string) (*ClientConn, error) {
 	return p.getClientConn(req, addr, noDialOnMiss)
 }
+
+// shouldRetryDial reports whether the current request should
+// retry dialing after the call finished unsuccessfully, for example
+// if the dial was canceled because of a context cancellation or
+// deadline expiry.
+func shouldRetryDial(call *dialCall, req *http.Request) bool {
+	if call.err == nil {
+		// No error, no need to retry
+		return false
+	}
+	if call.ctx == req.Context() {
+		// If the call has the same context as the request, the dial
+		// should not be retried, since any cancellation will have come
+		// from this request.
+		return false
+	}
+	if !errors.Is(call.err, context.Canceled) && !errors.Is(call.err, context.DeadlineExceeded) {
+		// If the call error is not because of a context cancellation or a deadline expiry,
+		// the dial should not be retried.
+		return false
+	}
+	// Only retry if the error is a context cancellation error or deadline expiry
+	// and the context associated with the call was canceled or expired.
+	return call.ctx.Err() != nil
+}
diff --git a/vendor/golang.org/x/net/http2/go111.go b/vendor/golang.org/x/net/http2/go111.go
index 3a13101..5bf62b0 100644
--- a/vendor/golang.org/x/net/http2/go111.go
+++ b/vendor/golang.org/x/net/http2/go111.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build go1.11
 // +build go1.11
 
 package http2
diff --git a/vendor/golang.org/x/net/http2/go115.go b/vendor/golang.org/x/net/http2/go115.go
new file mode 100644
index 0000000..908af1a
--- /dev/null
+++ b/vendor/golang.org/x/net/http2/go115.go
@@ -0,0 +1,27 @@
+// Copyright 2021 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.
+
+//go:build go1.15
+// +build go1.15
+
+package http2
+
+import (
+	"context"
+	"crypto/tls"
+)
+
+// dialTLSWithContext uses tls.Dialer, added in Go 1.15, to open a TLS
+// connection.
+func (t *Transport) dialTLSWithContext(ctx context.Context, network, addr string, cfg *tls.Config) (*tls.Conn, error) {
+	dialer := &tls.Dialer{
+		Config: cfg,
+	}
+	cn, err := dialer.DialContext(ctx, network, addr)
+	if err != nil {
+		return nil, err
+	}
+	tlsCn := cn.(*tls.Conn) // DialContext comment promises this will always succeed
+	return tlsCn, nil
+}
diff --git a/vendor/golang.org/x/net/http2/headermap.go b/vendor/golang.org/x/net/http2/headermap.go
index c3ff3fa..9e12941 100644
--- a/vendor/golang.org/x/net/http2/headermap.go
+++ b/vendor/golang.org/x/net/http2/headermap.go
@@ -6,7 +6,6 @@
 
 import (
 	"net/http"
-	"strings"
 	"sync"
 )
 
@@ -79,10 +78,10 @@
 	}
 }
 
-func lowerHeader(v string) string {
+func lowerHeader(v string) (lower string, ascii bool) {
 	buildCommonHeaderMapsOnce()
 	if s, ok := commonLowerHeader[v]; ok {
-		return s
+		return s, true
 	}
-	return strings.ToLower(v)
+	return asciiToLower(v)
 }
diff --git a/vendor/golang.org/x/net/http2/not_go111.go b/vendor/golang.org/x/net/http2/not_go111.go
index 161bca7..cc0baa8 100644
--- a/vendor/golang.org/x/net/http2/not_go111.go
+++ b/vendor/golang.org/x/net/http2/not_go111.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build !go1.11
 // +build !go1.11
 
 package http2
diff --git a/vendor/golang.org/x/net/http2/not_go115.go b/vendor/golang.org/x/net/http2/not_go115.go
new file mode 100644
index 0000000..e6c04cf
--- /dev/null
+++ b/vendor/golang.org/x/net/http2/not_go115.go
@@ -0,0 +1,31 @@
+// Copyright 2021 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.
+
+//go:build !go1.15
+// +build !go1.15
+
+package http2
+
+import (
+	"context"
+	"crypto/tls"
+)
+
+// dialTLSWithContext opens a TLS connection.
+func (t *Transport) dialTLSWithContext(ctx context.Context, network, addr string, cfg *tls.Config) (*tls.Conn, error) {
+	cn, err := tls.Dial(network, addr, cfg)
+	if err != nil {
+		return nil, err
+	}
+	if err := cn.Handshake(); err != nil {
+		return nil, err
+	}
+	if cfg.InsecureSkipVerify {
+		return cn, nil
+	}
+	if err := cn.VerifyHostname(cfg.ServerName); err != nil {
+		return nil, err
+	}
+	return cn, nil
+}
diff --git a/vendor/golang.org/x/net/http2/server.go b/vendor/golang.org/x/net/http2/server.go
index 2aa859f..0ccbe9b 100644
--- a/vendor/golang.org/x/net/http2/server.go
+++ b/vendor/golang.org/x/net/http2/server.go
@@ -231,13 +231,12 @@
 
 	if s.TLSConfig == nil {
 		s.TLSConfig = new(tls.Config)
-	} else if s.TLSConfig.CipherSuites != nil {
-		// If they already provided a CipherSuite list, return
-		// an error if it has a bad order or is missing
-		// ECDHE_RSA_WITH_AES_128_GCM_SHA256 or ECDHE_ECDSA_WITH_AES_128_GCM_SHA256.
+	} else if s.TLSConfig.CipherSuites != nil && s.TLSConfig.MinVersion < tls.VersionTLS13 {
+		// If they already provided a TLS 1.0–1.2 CipherSuite list, return an
+		// error if it is missing ECDHE_RSA_WITH_AES_128_GCM_SHA256 or
+		// ECDHE_ECDSA_WITH_AES_128_GCM_SHA256.
 		haveRequired := false
-		sawBad := false
-		for i, cs := range s.TLSConfig.CipherSuites {
+		for _, cs := range s.TLSConfig.CipherSuites {
 			switch cs {
 			case tls.TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,
 				// Alternative MTI cipher to not discourage ECDSA-only servers.
@@ -245,14 +244,9 @@
 				tls.TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256:
 				haveRequired = true
 			}
-			if isBadCipher(cs) {
-				sawBad = true
-			} else if sawBad {
-				return fmt.Errorf("http2: TLSConfig.CipherSuites index %d contains an HTTP/2-approved cipher suite (%#04x), but it comes after unapproved cipher suites. With this configuration, clients that don't support previous, approved cipher suites may be given an unapproved one and reject the connection.", i, cs)
-			}
 		}
 		if !haveRequired {
-			return fmt.Errorf("http2: TLSConfig.CipherSuites is missing an HTTP/2-required AES_128_GCM_SHA256 cipher (need at least one of TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256 or TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256).")
+			return fmt.Errorf("http2: TLSConfig.CipherSuites is missing an HTTP/2-required AES_128_GCM_SHA256 cipher (need at least one of TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256 or TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256)")
 		}
 	}
 
@@ -265,16 +259,12 @@
 
 	s.TLSConfig.PreferServerCipherSuites = true
 
-	haveNPN := false
-	for _, p := range s.TLSConfig.NextProtos {
-		if p == NextProtoTLS {
-			haveNPN = true
-			break
-		}
-	}
-	if !haveNPN {
+	if !strSliceContains(s.TLSConfig.NextProtos, NextProtoTLS) {
 		s.TLSConfig.NextProtos = append(s.TLSConfig.NextProtos, NextProtoTLS)
 	}
+	if !strSliceContains(s.TLSConfig.NextProtos, "http/1.1") {
+		s.TLSConfig.NextProtos = append(s.TLSConfig.NextProtos, "http/1.1")
+	}
 
 	if s.TLSNextProto == nil {
 		s.TLSNextProto = map[string]func(*http.Server, *tls.Conn, http.Handler){}
@@ -1293,7 +1283,9 @@
 	sc.shutdownOnce.Do(func() { sc.sendServeMsg(gracefulShutdownMsg) })
 }
 
-// After sending GOAWAY, the connection will close after goAwayTimeout.
+// After sending GOAWAY with an error code (non-graceful shutdown), the
+// connection will close after goAwayTimeout.
+//
 // If we close the connection immediately after sending GOAWAY, there may
 // be unsent data in our kernel receive buffer, which will cause the kernel
 // to send a TCP RST on close() instead of a FIN. This RST will abort the
@@ -1629,23 +1621,37 @@
 
 func (sc *serverConn) processData(f *DataFrame) error {
 	sc.serveG.check()
-	if sc.inGoAway && sc.goAwayCode != ErrCodeNo {
+	id := f.Header().StreamID
+	if sc.inGoAway && (sc.goAwayCode != ErrCodeNo || id > sc.maxClientStreamID) {
+		// Discard all DATA frames if the GOAWAY is due to an
+		// error, or:
+		//
+		// Section 6.8: After sending a GOAWAY frame, the sender
+		// can discard frames for streams initiated by the
+		// receiver with identifiers higher than the identified
+		// last stream.
 		return nil
 	}
-	data := f.Data()
 
-	// "If a DATA frame is received whose stream is not in "open"
-	// or "half closed (local)" state, the recipient MUST respond
-	// with a stream error (Section 5.4.2) of type STREAM_CLOSED."
-	id := f.Header().StreamID
+	data := f.Data()
 	state, st := sc.state(id)
 	if id == 0 || state == stateIdle {
+		// Section 6.1: "DATA frames MUST be associated with a
+		// stream. If a DATA frame is received whose stream
+		// identifier field is 0x0, the recipient MUST respond
+		// with a connection error (Section 5.4.1) of type
+		// PROTOCOL_ERROR."
+		//
 		// Section 5.1: "Receiving any frame other than HEADERS
 		// or PRIORITY on a stream in this state MUST be
 		// treated as a connection error (Section 5.4.1) of
 		// type PROTOCOL_ERROR."
 		return ConnectionError(ErrCodeProtocol)
 	}
+
+	// "If a DATA frame is received whose stream is not in "open"
+	// or "half closed (local)" state, the recipient MUST respond
+	// with a stream error (Section 5.4.2) of type STREAM_CLOSED."
 	if st == nil || state != stateOpen || st.gotTrailerHeader || st.resetQueued {
 		// This includes sending a RST_STREAM if the stream is
 		// in stateHalfClosedLocal (which currently means that
@@ -2773,8 +2779,12 @@
 		// but PUSH_PROMISE requests cannot have a body.
 		// http://tools.ietf.org/html/rfc7540#section-8.2
 		// Also disallow Host, since the promised URL must be absolute.
-		switch strings.ToLower(k) {
-		case "content-length", "content-encoding", "trailer", "te", "expect", "host":
+		if asciiEqualFold(k, "content-length") ||
+			asciiEqualFold(k, "content-encoding") ||
+			asciiEqualFold(k, "trailer") ||
+			asciiEqualFold(k, "te") ||
+			asciiEqualFold(k, "expect") ||
+			asciiEqualFold(k, "host") {
 			return fmt.Errorf("promised request headers cannot include %q", k)
 		}
 	}
diff --git a/vendor/golang.org/x/net/http2/transport.go b/vendor/golang.org/x/net/http2/transport.go
index 4182f52..b97adff 100644
--- a/vendor/golang.org/x/net/http2/transport.go
+++ b/vendor/golang.org/x/net/http2/transport.go
@@ -154,12 +154,21 @@
 
 // ConfigureTransport configures a net/http HTTP/1 Transport to use HTTP/2.
 // It returns an error if t1 has already been HTTP/2-enabled.
+//
+// Use ConfigureTransports instead to configure the HTTP/2 Transport.
 func ConfigureTransport(t1 *http.Transport) error {
-	_, err := configureTransport(t1)
+	_, err := ConfigureTransports(t1)
 	return err
 }
 
-func configureTransport(t1 *http.Transport) (*Transport, error) {
+// ConfigureTransports configures a net/http HTTP/1 Transport to use HTTP/2.
+// It returns a new HTTP/2 Transport for further configuration.
+// It returns an error if t1 has already been HTTP/2-enabled.
+func ConfigureTransports(t1 *http.Transport) (*Transport, error) {
+	return configureTransports(t1)
+}
+
+func configureTransports(t1 *http.Transport) (*Transport, error) {
 	connPool := new(clientConnPool)
 	t2 := &Transport{
 		ConnPool: noDialClientConnPool{connPool},
@@ -255,9 +264,8 @@
 	peerMaxHeaderListSize uint64
 	initialWindowSize     uint32
 
-	hbuf    bytes.Buffer // HPACK encoder writes into this
-	henc    *hpack.Encoder
-	freeBuf [][]byte
+	hbuf bytes.Buffer // HPACK encoder writes into this
+	henc *hpack.Encoder
 
 	wmu  sync.Mutex // held while writing; acquire AFTER mu if holding both
 	werr error      // first write error that has occurred
@@ -555,12 +563,12 @@
 	return false
 }
 
-func (t *Transport) dialClientConn(addr string, singleUse bool) (*ClientConn, error) {
+func (t *Transport) dialClientConn(ctx context.Context, addr string, singleUse bool) (*ClientConn, error) {
 	host, _, err := net.SplitHostPort(addr)
 	if err != nil {
 		return nil, err
 	}
-	tconn, err := t.dialTLS()("tcp", addr, t.newTLSConfig(host))
+	tconn, err := t.dialTLS(ctx)("tcp", addr, t.newTLSConfig(host))
 	if err != nil {
 		return nil, err
 	}
@@ -581,34 +589,24 @@
 	return cfg
 }
 
-func (t *Transport) dialTLS() func(string, string, *tls.Config) (net.Conn, error) {
+func (t *Transport) dialTLS(ctx context.Context) func(string, string, *tls.Config) (net.Conn, error) {
 	if t.DialTLS != nil {
 		return t.DialTLS
 	}
-	return t.dialTLSDefault
-}
-
-func (t *Transport) dialTLSDefault(network, addr string, cfg *tls.Config) (net.Conn, error) {
-	cn, err := tls.Dial(network, addr, cfg)
-	if err != nil {
-		return nil, err
-	}
-	if err := cn.Handshake(); err != nil {
-		return nil, err
-	}
-	if !cfg.InsecureSkipVerify {
-		if err := cn.VerifyHostname(cfg.ServerName); err != nil {
+	return func(network, addr string, cfg *tls.Config) (net.Conn, error) {
+		tlsCn, err := t.dialTLSWithContext(ctx, network, addr, cfg)
+		if err != nil {
 			return nil, err
 		}
+		state := tlsCn.ConnectionState()
+		if p := state.NegotiatedProtocol; p != NextProtoTLS {
+			return nil, fmt.Errorf("http2: unexpected ALPN protocol %q; want %q", p, NextProtoTLS)
+		}
+		if !state.NegotiatedProtocolIsMutual {
+			return nil, errors.New("http2: could not negotiate protocol mutually")
+		}
+		return tlsCn, nil
 	}
-	state := cn.ConnectionState()
-	if p := state.NegotiatedProtocol; p != NextProtoTLS {
-		return nil, fmt.Errorf("http2: unexpected ALPN protocol %q; want %q", p, NextProtoTLS)
-	}
-	if !state.NegotiatedProtocolIsMutual {
-		return nil, errors.New("http2: could not negotiate protocol mutually")
-	}
-	return cn, nil
 }
 
 // disableKeepAlives reports whether connections should be closed as
@@ -914,46 +912,6 @@
 	return cc.closeForError(err)
 }
 
-const maxAllocFrameSize = 512 << 10
-
-// frameBuffer returns a scratch buffer suitable for writing DATA frames.
-// They're capped at the min of the peer's max frame size or 512KB
-// (kinda arbitrarily), but definitely capped so we don't allocate 4GB
-// bufers.
-func (cc *ClientConn) frameScratchBuffer() []byte {
-	cc.mu.Lock()
-	size := cc.maxFrameSize
-	if size > maxAllocFrameSize {
-		size = maxAllocFrameSize
-	}
-	for i, buf := range cc.freeBuf {
-		if len(buf) >= int(size) {
-			cc.freeBuf[i] = nil
-			cc.mu.Unlock()
-			return buf[:size]
-		}
-	}
-	cc.mu.Unlock()
-	return make([]byte, size)
-}
-
-func (cc *ClientConn) putFrameScratchBuffer(buf []byte) {
-	cc.mu.Lock()
-	defer cc.mu.Unlock()
-	const maxBufs = 4 // arbitrary; 4 concurrent requests per conn? investigate.
-	if len(cc.freeBuf) < maxBufs {
-		cc.freeBuf = append(cc.freeBuf, buf)
-		return
-	}
-	for i, old := range cc.freeBuf {
-		if old == nil {
-			cc.freeBuf[i] = buf
-			return
-		}
-	}
-	// forget about it.
-}
-
 // errRequestCanceled is a copy of net/http's errRequestCanceled because it's not
 // exported. At least they'll be DeepEqual for h1-vs-h2 comparisons tests.
 var errRequestCanceled = errors.New("net/http: request canceled")
@@ -996,7 +954,7 @@
 	if vv := req.Header["Transfer-Encoding"]; len(vv) > 0 && (len(vv) > 1 || vv[0] != "" && vv[0] != "chunked") {
 		return fmt.Errorf("http2: invalid Transfer-Encoding request header: %q", vv)
 	}
-	if vv := req.Header["Connection"]; len(vv) > 0 && (len(vv) > 1 || vv[0] != "" && !strings.EqualFold(vv[0], "close") && !strings.EqualFold(vv[0], "keep-alive")) {
+	if vv := req.Header["Connection"]; len(vv) > 0 && (len(vv) > 1 || vv[0] != "" && !asciiEqualFold(vv[0], "close") && !asciiEqualFold(vv[0], "keep-alive")) {
 		return fmt.Errorf("http2: invalid Connection request header: %q", vv)
 	}
 	return nil
@@ -1139,6 +1097,9 @@
 			// we can keep it.
 			bodyWriter.cancel()
 			cs.abortRequestBodyWrite(errStopReqBodyWrite)
+			if hasBody && !bodyWritten {
+				<-bodyWriter.resc
+			}
 		}
 		if re.err != nil {
 			cc.forgetStreamID(cs.ID)
@@ -1159,6 +1120,7 @@
 			} else {
 				bodyWriter.cancel()
 				cs.abortRequestBodyWrite(errStopReqBodyWriteAndCancel)
+				<-bodyWriter.resc
 			}
 			cc.forgetStreamID(cs.ID)
 			return nil, cs.getStartedWrite(), errTimeout
@@ -1168,6 +1130,7 @@
 			} else {
 				bodyWriter.cancel()
 				cs.abortRequestBodyWrite(errStopReqBodyWriteAndCancel)
+				<-bodyWriter.resc
 			}
 			cc.forgetStreamID(cs.ID)
 			return nil, cs.getStartedWrite(), ctx.Err()
@@ -1177,6 +1140,7 @@
 			} else {
 				bodyWriter.cancel()
 				cs.abortRequestBodyWrite(errStopReqBodyWriteAndCancel)
+				<-bodyWriter.resc
 			}
 			cc.forgetStreamID(cs.ID)
 			return nil, cs.getStartedWrite(), errRequestCanceled
@@ -1186,6 +1150,7 @@
 			// forgetStreamID.
 			return nil, cs.getStartedWrite(), cs.resetErr
 		case err := <-bodyWriter.resc:
+			bodyWritten = true
 			// Prefer the read loop's response, if available. Issue 16102.
 			select {
 			case re := <-readLoopResCh:
@@ -1196,7 +1161,6 @@
 				cc.forgetStreamID(cs.ID)
 				return nil, cs.getStartedWrite(), err
 			}
-			bodyWritten = true
 			if d := cc.responseHeaderTimeout(); d != 0 {
 				timer := time.NewTimer(d)
 				defer timer.Stop()
@@ -1290,11 +1254,35 @@
 	errReqBodyTooLong = errors.New("http2: request body larger than specified content length")
 )
 
+// frameScratchBufferLen returns the length of a buffer to use for
+// outgoing request bodies to read/write to/from.
+//
+// It returns max(1, min(peer's advertised max frame size,
+// Request.ContentLength+1, 512KB)).
+func (cs *clientStream) frameScratchBufferLen(maxFrameSize int) int {
+	const max = 512 << 10
+	n := int64(maxFrameSize)
+	if n > max {
+		n = max
+	}
+	if cl := actualContentLength(cs.req); cl != -1 && cl+1 < n {
+		// Add an extra byte past the declared content-length to
+		// give the caller's Request.Body io.Reader a chance to
+		// give us more bytes than they declared, so we can catch it
+		// early.
+		n = cl + 1
+	}
+	if n < 1 {
+		return 1
+	}
+	return int(n) // doesn't truncate; max is 512K
+}
+
+var bufPool sync.Pool // of *[]byte
+
 func (cs *clientStream) writeRequestBody(body io.Reader, bodyCloser io.Closer) (err error) {
 	cc := cs.cc
 	sentEnd := false // whether we sent the final DATA frame w/ END_STREAM
-	buf := cc.frameScratchBuffer()
-	defer cc.putFrameScratchBuffer(buf)
 
 	defer func() {
 		traceWroteRequest(cs.trace, err)
@@ -1313,9 +1301,24 @@
 	remainLen := actualContentLength(req)
 	hasContentLen := remainLen != -1
 
+	cc.mu.Lock()
+	maxFrameSize := int(cc.maxFrameSize)
+	cc.mu.Unlock()
+
+	// Scratch buffer for reading into & writing from.
+	scratchLen := cs.frameScratchBufferLen(maxFrameSize)
+	var buf []byte
+	if bp, ok := bufPool.Get().(*[]byte); ok && len(*bp) >= scratchLen {
+		defer bufPool.Put(bp)
+		buf = *bp
+	} else {
+		buf = make([]byte, scratchLen)
+		defer bufPool.Put(&buf)
+	}
+
 	var sawEOF bool
 	for !sawEOF {
-		n, err := body.Read(buf[:len(buf)-1])
+		n, err := body.Read(buf[:len(buf)])
 		if hasContentLen {
 			remainLen -= int64(n)
 			if remainLen == 0 && err == nil {
@@ -1326,8 +1329,9 @@
 				// to send the END_STREAM bit early, double-check that we're actually
 				// at EOF. Subsequent reads should return (0, EOF) at this point.
 				// If either value is different, we return an error in one of two ways below.
+				var scratch [1]byte
 				var n1 int
-				n1, err = body.Read(buf[n:])
+				n1, err = body.Read(scratch[:])
 				remainLen -= int64(n1)
 			}
 			if remainLen < 0 {
@@ -1397,10 +1401,6 @@
 		}
 	}
 
-	cc.mu.Lock()
-	maxFrameSize := int(cc.maxFrameSize)
-	cc.mu.Unlock()
-
 	cc.wmu.Lock()
 	defer cc.wmu.Unlock()
 
@@ -1516,19 +1516,21 @@
 
 		var didUA bool
 		for k, vv := range req.Header {
-			if strings.EqualFold(k, "host") || strings.EqualFold(k, "content-length") {
+			if asciiEqualFold(k, "host") || asciiEqualFold(k, "content-length") {
 				// Host is :authority, already sent.
 				// Content-Length is automatic, set below.
 				continue
-			} else if strings.EqualFold(k, "connection") || strings.EqualFold(k, "proxy-connection") ||
-				strings.EqualFold(k, "transfer-encoding") || strings.EqualFold(k, "upgrade") ||
-				strings.EqualFold(k, "keep-alive") {
+			} else if asciiEqualFold(k, "connection") ||
+				asciiEqualFold(k, "proxy-connection") ||
+				asciiEqualFold(k, "transfer-encoding") ||
+				asciiEqualFold(k, "upgrade") ||
+				asciiEqualFold(k, "keep-alive") {
 				// Per 8.1.2.2 Connection-Specific Header
 				// Fields, don't send connection-specific
 				// fields. We have already checked if any
 				// are error-worthy so just ignore the rest.
 				continue
-			} else if strings.EqualFold(k, "user-agent") {
+			} else if asciiEqualFold(k, "user-agent") {
 				// Match Go's http1 behavior: at most one
 				// User-Agent. If set to nil or empty string,
 				// then omit it. Otherwise if not mentioned,
@@ -1541,7 +1543,7 @@
 				if vv[0] == "" {
 					continue
 				}
-			} else if strings.EqualFold(k, "cookie") {
+			} else if asciiEqualFold(k, "cookie") {
 				// Per 8.1.2.5 To allow for better compression efficiency, the
 				// Cookie header field MAY be split into separate header fields,
 				// each with one or more cookie-pairs.
@@ -1600,7 +1602,12 @@
 
 	// Header list size is ok. Write the headers.
 	enumerateHeaders(func(name, value string) {
-		name = strings.ToLower(name)
+		name, ascii := asciiToLower(name)
+		if !ascii {
+			// Skip writing invalid headers. Per RFC 7540, Section 8.1.2, header
+			// field names have to be ASCII characters (just as in HTTP/1.x).
+			return
+		}
 		cc.writeHeader(name, value)
 		if traceHeaders {
 			traceWroteHeaderField(trace, name, value)
@@ -1648,9 +1655,14 @@
 	}
 
 	for k, vv := range req.Trailer {
+		lowKey, ascii := asciiToLower(k)
+		if !ascii {
+			// Skip writing invalid headers. Per RFC 7540, Section 8.1.2, header
+			// field names have to be ASCII characters (just as in HTTP/1.x).
+			continue
+		}
 		// Transfer-Encoding, etc.. have already been filtered at the
 		// start of RoundTrip
-		lowKey := strings.ToLower(k)
 		for _, v := range vv {
 			cc.writeHeader(lowKey, v)
 		}
@@ -2617,7 +2629,9 @@
 
 func (s bodyWriterState) cancel() {
 	if s.timer != nil {
-		s.timer.Stop()
+		if s.timer.Stop() {
+			s.resc <- nil
+		}
 	}
 }
 
diff --git a/vendor/golang.org/x/net/http2/write.go b/vendor/golang.org/x/net/http2/write.go
index 3849bc2..33f6139 100644
--- a/vendor/golang.org/x/net/http2/write.go
+++ b/vendor/golang.org/x/net/http2/write.go
@@ -341,7 +341,12 @@
 	}
 	for _, k := range keys {
 		vv := h[k]
-		k = lowerHeader(k)
+		k, ascii := lowerHeader(k)
+		if !ascii {
+			// Skip writing invalid headers. Per RFC 7540, Section 8.1.2, header
+			// field names have to be ASCII characters (just as in HTTP/1.x).
+			continue
+		}
 		if !validWireHeaderFieldName(k) {
 			// Skip it as backup paranoia. Per
 			// golang.org/issue/14048, these should
diff --git a/vendor/golang.org/x/net/idna/idna10.0.0.go b/vendor/golang.org/x/net/idna/idna10.0.0.go
index a98a31f..5208ba6 100644
--- a/vendor/golang.org/x/net/idna/idna10.0.0.go
+++ b/vendor/golang.org/x/net/idna/idna10.0.0.go
@@ -4,6 +4,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build go1.10
 // +build go1.10
 
 // Package idna implements IDNA2008 using the compatibility processing
@@ -66,15 +67,14 @@
 
 // VerifyDNSLength sets whether a Profile should fail if any of the IDN parts
 // are longer than allowed by the RFC.
+//
+// This option corresponds to the VerifyDnsLength flag in UTS #46.
 func VerifyDNSLength(verify bool) Option {
 	return func(o *options) { o.verifyDNSLength = verify }
 }
 
 // RemoveLeadingDots removes leading label separators. Leading runes that map to
 // dots, such as U+3002 IDEOGRAPHIC FULL STOP, are removed as well.
-//
-// This is the behavior suggested by the UTS #46 and is adopted by some
-// browsers.
 func RemoveLeadingDots(remove bool) Option {
 	return func(o *options) { o.removeLeadingDots = remove }
 }
@@ -82,6 +82,8 @@
 // ValidateLabels sets whether to check the mandatory label validation criteria
 // as defined in Section 5.4 of RFC 5891. This includes testing for correct use
 // of hyphens ('-'), normalization, validity of runes, and the context rules.
+// In particular, ValidateLabels also sets the CheckHyphens and CheckJoiners flags
+// in UTS #46.
 func ValidateLabels(enable bool) Option {
 	return func(o *options) {
 		// Don't override existing mappings, but set one that at least checks
@@ -90,25 +92,48 @@
 			o.mapping = normalize
 		}
 		o.trie = trie
-		o.validateLabels = enable
-		o.fromPuny = validateFromPunycode
+		o.checkJoiners = enable
+		o.checkHyphens = enable
+		if enable {
+			o.fromPuny = validateFromPunycode
+		} else {
+			o.fromPuny = nil
+		}
+	}
+}
+
+// CheckHyphens sets whether to check for correct use of hyphens ('-') in
+// labels. Most web browsers do not have this option set, since labels such as
+// "r3---sn-apo3qvuoxuxbt-j5pe" are in common use.
+//
+// This option corresponds to the CheckHyphens flag in UTS #46.
+func CheckHyphens(enable bool) Option {
+	return func(o *options) { o.checkHyphens = enable }
+}
+
+// CheckJoiners sets whether to check the ContextJ rules as defined in Appendix
+// A of RFC 5892, concerning the use of joiner runes.
+//
+// This option corresponds to the CheckJoiners flag in UTS #46.
+func CheckJoiners(enable bool) Option {
+	return func(o *options) {
+		o.trie = trie
+		o.checkJoiners = enable
 	}
 }
 
 // StrictDomainName limits the set of permissible ASCII characters to those
 // allowed in domain names as defined in RFC 1034 (A-Z, a-z, 0-9 and the
-// hyphen). This is set by default for MapForLookup and ValidateForRegistration.
+// hyphen). This is set by default for MapForLookup and ValidateForRegistration,
+// but is only useful if ValidateLabels is set.
 //
 // This option is useful, for instance, for browsers that allow characters
 // outside this range, for example a '_' (U+005F LOW LINE). See
-// http://www.rfc-editor.org/std/std3.txt for more details This option
-// corresponds to the UseSTD3ASCIIRules option in UTS #46.
+// http://www.rfc-editor.org/std/std3.txt for more details.
+//
+// This option corresponds to the UseSTD3ASCIIRules flag in UTS #46.
 func StrictDomainName(use bool) Option {
-	return func(o *options) {
-		o.trie = trie
-		o.useSTD3Rules = use
-		o.fromPuny = validateFromPunycode
-	}
+	return func(o *options) { o.useSTD3Rules = use }
 }
 
 // NOTE: the following options pull in tables. The tables should not be linked
@@ -116,6 +141,8 @@
 
 // BidiRule enables the Bidi rule as defined in RFC 5893. Any application
 // that relies on proper validation of labels should include this rule.
+//
+// This option corresponds to the CheckBidi flag in UTS #46.
 func BidiRule() Option {
 	return func(o *options) { o.bidirule = bidirule.ValidString }
 }
@@ -151,7 +178,8 @@
 type options struct {
 	transitional      bool
 	useSTD3Rules      bool
-	validateLabels    bool
+	checkHyphens      bool
+	checkJoiners      bool
 	verifyDNSLength   bool
 	removeLeadingDots bool
 
@@ -224,8 +252,11 @@
 	if p.useSTD3Rules {
 		s += ":UseSTD3Rules"
 	}
-	if p.validateLabels {
-		s += ":ValidateLabels"
+	if p.checkHyphens {
+		s += ":CheckHyphens"
+	}
+	if p.checkJoiners {
+		s += ":CheckJoiners"
 	}
 	if p.verifyDNSLength {
 		s += ":VerifyDNSLength"
@@ -253,26 +284,29 @@
 
 	punycode = &Profile{}
 	lookup   = &Profile{options{
-		transitional:   true,
-		useSTD3Rules:   true,
-		validateLabels: true,
-		trie:           trie,
-		fromPuny:       validateFromPunycode,
-		mapping:        validateAndMap,
-		bidirule:       bidirule.ValidString,
+		transitional: true,
+		useSTD3Rules: true,
+		checkHyphens: true,
+		checkJoiners: true,
+		trie:         trie,
+		fromPuny:     validateFromPunycode,
+		mapping:      validateAndMap,
+		bidirule:     bidirule.ValidString,
 	}}
 	display = &Profile{options{
-		useSTD3Rules:   true,
-		validateLabels: true,
-		trie:           trie,
-		fromPuny:       validateFromPunycode,
-		mapping:        validateAndMap,
-		bidirule:       bidirule.ValidString,
+		useSTD3Rules: true,
+		checkHyphens: true,
+		checkJoiners: true,
+		trie:         trie,
+		fromPuny:     validateFromPunycode,
+		mapping:      validateAndMap,
+		bidirule:     bidirule.ValidString,
 	}}
 	registration = &Profile{options{
 		useSTD3Rules:    true,
-		validateLabels:  true,
 		verifyDNSLength: true,
+		checkHyphens:    true,
+		checkJoiners:    true,
 		trie:            trie,
 		fromPuny:        validateFromPunycode,
 		mapping:         validateRegistration,
@@ -339,7 +373,7 @@
 			}
 			isBidi = isBidi || bidirule.DirectionString(u) != bidi.LeftToRight
 			labels.set(u)
-			if err == nil && p.validateLabels {
+			if err == nil && p.fromPuny != nil {
 				err = p.fromPuny(p, u)
 			}
 			if err == nil {
@@ -680,16 +714,18 @@
 		}
 		return nil
 	}
-	if !p.validateLabels {
+	if p.checkHyphens {
+		if len(s) > 4 && s[2] == '-' && s[3] == '-' {
+			return &labelError{s, "V2"}
+		}
+		if s[0] == '-' || s[len(s)-1] == '-' {
+			return &labelError{s, "V3"}
+		}
+	}
+	if !p.checkJoiners {
 		return nil
 	}
-	trie := p.trie // p.validateLabels is only set if trie is set.
-	if len(s) > 4 && s[2] == '-' && s[3] == '-' {
-		return &labelError{s, "V2"}
-	}
-	if s[0] == '-' || s[len(s)-1] == '-' {
-		return &labelError{s, "V3"}
-	}
+	trie := p.trie // p.checkJoiners is only set if trie is set.
 	// TODO: merge the use of this in the trie.
 	v, sz := trie.lookupString(s)
 	x := info(v)
diff --git a/vendor/golang.org/x/net/idna/idna9.0.0.go b/vendor/golang.org/x/net/idna/idna9.0.0.go
index 8842146..55f718f 100644
--- a/vendor/golang.org/x/net/idna/idna9.0.0.go
+++ b/vendor/golang.org/x/net/idna/idna9.0.0.go
@@ -4,6 +4,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build !go1.10
 // +build !go1.10
 
 // Package idna implements IDNA2008 using the compatibility processing
@@ -65,15 +66,14 @@
 
 // VerifyDNSLength sets whether a Profile should fail if any of the IDN parts
 // are longer than allowed by the RFC.
+//
+// This option corresponds to the VerifyDnsLength flag in UTS #46.
 func VerifyDNSLength(verify bool) Option {
 	return func(o *options) { o.verifyDNSLength = verify }
 }
 
 // RemoveLeadingDots removes leading label separators. Leading runes that map to
 // dots, such as U+3002 IDEOGRAPHIC FULL STOP, are removed as well.
-//
-// This is the behavior suggested by the UTS #46 and is adopted by some
-// browsers.
 func RemoveLeadingDots(remove bool) Option {
 	return func(o *options) { o.removeLeadingDots = remove }
 }
@@ -81,6 +81,8 @@
 // ValidateLabels sets whether to check the mandatory label validation criteria
 // as defined in Section 5.4 of RFC 5891. This includes testing for correct use
 // of hyphens ('-'), normalization, validity of runes, and the context rules.
+// In particular, ValidateLabels also sets the CheckHyphens and CheckJoiners flags
+// in UTS #46.
 func ValidateLabels(enable bool) Option {
 	return func(o *options) {
 		// Don't override existing mappings, but set one that at least checks
@@ -89,25 +91,48 @@
 			o.mapping = normalize
 		}
 		o.trie = trie
-		o.validateLabels = enable
-		o.fromPuny = validateFromPunycode
+		o.checkJoiners = enable
+		o.checkHyphens = enable
+		if enable {
+			o.fromPuny = validateFromPunycode
+		} else {
+			o.fromPuny = nil
+		}
+	}
+}
+
+// CheckHyphens sets whether to check for correct use of hyphens ('-') in
+// labels. Most web browsers do not have this option set, since labels such as
+// "r3---sn-apo3qvuoxuxbt-j5pe" are in common use.
+//
+// This option corresponds to the CheckHyphens flag in UTS #46.
+func CheckHyphens(enable bool) Option {
+	return func(o *options) { o.checkHyphens = enable }
+}
+
+// CheckJoiners sets whether to check the ContextJ rules as defined in Appendix
+// A of RFC 5892, concerning the use of joiner runes.
+//
+// This option corresponds to the CheckJoiners flag in UTS #46.
+func CheckJoiners(enable bool) Option {
+	return func(o *options) {
+		o.trie = trie
+		o.checkJoiners = enable
 	}
 }
 
 // StrictDomainName limits the set of permissable ASCII characters to those
 // allowed in domain names as defined in RFC 1034 (A-Z, a-z, 0-9 and the
-// hyphen). This is set by default for MapForLookup and ValidateForRegistration.
+// hyphen). This is set by default for MapForLookup and ValidateForRegistration,
+// but is only useful if ValidateLabels is set.
 //
 // This option is useful, for instance, for browsers that allow characters
 // outside this range, for example a '_' (U+005F LOW LINE). See
-// http://www.rfc-editor.org/std/std3.txt for more details This option
-// corresponds to the UseSTD3ASCIIRules option in UTS #46.
+// http://www.rfc-editor.org/std/std3.txt for more details.
+//
+// This option corresponds to the UseSTD3ASCIIRules flag in UTS #46.
 func StrictDomainName(use bool) Option {
-	return func(o *options) {
-		o.trie = trie
-		o.useSTD3Rules = use
-		o.fromPuny = validateFromPunycode
-	}
+	return func(o *options) { o.useSTD3Rules = use }
 }
 
 // NOTE: the following options pull in tables. The tables should not be linked
@@ -115,6 +140,8 @@
 
 // BidiRule enables the Bidi rule as defined in RFC 5893. Any application
 // that relies on proper validation of labels should include this rule.
+//
+// This option corresponds to the CheckBidi flag in UTS #46.
 func BidiRule() Option {
 	return func(o *options) { o.bidirule = bidirule.ValidString }
 }
@@ -151,7 +178,8 @@
 type options struct {
 	transitional      bool
 	useSTD3Rules      bool
-	validateLabels    bool
+	checkHyphens      bool
+	checkJoiners      bool
 	verifyDNSLength   bool
 	removeLeadingDots bool
 
@@ -224,8 +252,11 @@
 	if p.useSTD3Rules {
 		s += ":UseSTD3Rules"
 	}
-	if p.validateLabels {
-		s += ":ValidateLabels"
+	if p.checkHyphens {
+		s += ":CheckHyphens"
+	}
+	if p.checkJoiners {
+		s += ":CheckJoiners"
 	}
 	if p.verifyDNSLength {
 		s += ":VerifyDNSLength"
@@ -254,9 +285,10 @@
 	punycode = &Profile{}
 	lookup   = &Profile{options{
 		transitional:      true,
-		useSTD3Rules:      true,
-		validateLabels:    true,
 		removeLeadingDots: true,
+		useSTD3Rules:      true,
+		checkHyphens:      true,
+		checkJoiners:      true,
 		trie:              trie,
 		fromPuny:          validateFromPunycode,
 		mapping:           validateAndMap,
@@ -264,8 +296,9 @@
 	}}
 	display = &Profile{options{
 		useSTD3Rules:      true,
-		validateLabels:    true,
 		removeLeadingDots: true,
+		checkHyphens:      true,
+		checkJoiners:      true,
 		trie:              trie,
 		fromPuny:          validateFromPunycode,
 		mapping:           validateAndMap,
@@ -273,8 +306,9 @@
 	}}
 	registration = &Profile{options{
 		useSTD3Rules:    true,
-		validateLabels:  true,
 		verifyDNSLength: true,
+		checkHyphens:    true,
+		checkJoiners:    true,
 		trie:            trie,
 		fromPuny:        validateFromPunycode,
 		mapping:         validateRegistration,
@@ -338,7 +372,7 @@
 				continue
 			}
 			labels.set(u)
-			if err == nil && p.validateLabels {
+			if err == nil && p.fromPuny != nil {
 				err = p.fromPuny(p, u)
 			}
 			if err == nil {
@@ -628,16 +662,18 @@
 	if p.bidirule != nil && !p.bidirule(s) {
 		return &labelError{s, "B"}
 	}
-	if !p.validateLabels {
+	if p.checkHyphens {
+		if len(s) > 4 && s[2] == '-' && s[3] == '-' {
+			return &labelError{s, "V2"}
+		}
+		if s[0] == '-' || s[len(s)-1] == '-' {
+			return &labelError{s, "V3"}
+		}
+	}
+	if !p.checkJoiners {
 		return nil
 	}
-	trie := p.trie // p.validateLabels is only set if trie is set.
-	if len(s) > 4 && s[2] == '-' && s[3] == '-' {
-		return &labelError{s, "V2"}
-	}
-	if s[0] == '-' || s[len(s)-1] == '-' {
-		return &labelError{s, "V3"}
-	}
+	trie := p.trie // p.checkJoiners is only set if trie is set.
 	// TODO: merge the use of this in the trie.
 	v, sz := trie.lookupString(s)
 	x := info(v)
diff --git a/vendor/golang.org/x/net/idna/tables10.0.0.go b/vendor/golang.org/x/net/idna/tables10.0.0.go
index 54fddb4..d1d62ef 100644
--- a/vendor/golang.org/x/net/idna/tables10.0.0.go
+++ b/vendor/golang.org/x/net/idna/tables10.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.10 && !go1.13
 // +build go1.10,!go1.13
 
 package idna
diff --git a/vendor/golang.org/x/net/idna/tables11.0.0.go b/vendor/golang.org/x/net/idna/tables11.0.0.go
index 8ce0811..167efba 100644
--- a/vendor/golang.org/x/net/idna/tables11.0.0.go
+++ b/vendor/golang.org/x/net/idna/tables11.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.13 && !go1.14
 // +build go1.13,!go1.14
 
 package idna
diff --git a/vendor/golang.org/x/net/idna/tables12.0.0.go b/vendor/golang.org/x/net/idna/tables12.0.0.go
new file mode 100644
index 0000000..ab40f7b
--- /dev/null
+++ b/vendor/golang.org/x/net/idna/tables12.0.0.go
@@ -0,0 +1,4734 @@
+// Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
+
+//go:build go1.14 && !go1.16
+// +build go1.14,!go1.16
+
+package idna
+
+// UnicodeVersion is the Unicode version from which the tables in this package are derived.
+const UnicodeVersion = "12.0.0"
+
+var mappings string = "" + // Size: 8178 bytes
+	"\x00\x01 \x03 ̈\x01a\x03 ̄\x012\x013\x03 ́\x03 ̧\x011\x01o\x051⁄4\x051⁄2" +
+	"\x053⁄4\x03i̇\x03l·\x03ʼn\x01s\x03dž\x03ⱥ\x03ⱦ\x01h\x01j\x01r\x01w\x01y" +
+	"\x03 ̆\x03 ̇\x03 ̊\x03 ̨\x03 ̃\x03 ̋\x01l\x01x\x04̈́\x03 ι\x01;\x05 ̈́" +
+	"\x04եւ\x04اٴ\x04وٴ\x04ۇٴ\x04يٴ\x06क़\x06ख़\x06ग़\x06ज़\x06ड़\x06ढ़\x06फ़" +
+	"\x06य़\x06ড়\x06ঢ়\x06য়\x06ਲ਼\x06ਸ਼\x06ਖ਼\x06ਗ਼\x06ਜ਼\x06ਫ਼\x06ଡ଼\x06ଢ଼" +
+	"\x06ํา\x06ໍາ\x06ຫນ\x06ຫມ\x06གྷ\x06ཌྷ\x06དྷ\x06བྷ\x06ཛྷ\x06ཀྵ\x06ཱི\x06ཱུ" +
+	"\x06ྲྀ\x09ྲཱྀ\x06ླྀ\x09ླཱྀ\x06ཱྀ\x06ྒྷ\x06ྜྷ\x06ྡྷ\x06ྦྷ\x06ྫྷ\x06ྐྵ\x02" +
+	"в\x02д\x02о\x02с\x02т\x02ъ\x02ѣ\x02æ\x01b\x01d\x01e\x02ǝ\x01g\x01i\x01k" +
+	"\x01m\x01n\x02ȣ\x01p\x01t\x01u\x02ɐ\x02ɑ\x02ə\x02ɛ\x02ɜ\x02ŋ\x02ɔ\x02ɯ" +
+	"\x01v\x02β\x02γ\x02δ\x02φ\x02χ\x02ρ\x02н\x02ɒ\x01c\x02ɕ\x02ð\x01f\x02ɟ" +
+	"\x02ɡ\x02ɥ\x02ɨ\x02ɩ\x02ɪ\x02ʝ\x02ɭ\x02ʟ\x02ɱ\x02ɰ\x02ɲ\x02ɳ\x02ɴ\x02ɵ" +
+	"\x02ɸ\x02ʂ\x02ʃ\x02ƫ\x02ʉ\x02ʊ\x02ʋ\x02ʌ\x01z\x02ʐ\x02ʑ\x02ʒ\x02θ\x02ss" +
+	"\x02ά\x02έ\x02ή\x02ί\x02ό\x02ύ\x02ώ\x05ἀι\x05ἁι\x05ἂι\x05ἃι\x05ἄι\x05ἅι" +
+	"\x05ἆι\x05ἇι\x05ἠι\x05ἡι\x05ἢι\x05ἣι\x05ἤι\x05ἥι\x05ἦι\x05ἧι\x05ὠι\x05ὡι" +
+	"\x05ὢι\x05ὣι\x05ὤι\x05ὥι\x05ὦι\x05ὧι\x05ὰι\x04αι\x04άι\x05ᾶι\x02ι\x05 ̈͂" +
+	"\x05ὴι\x04ηι\x04ήι\x05ῆι\x05 ̓̀\x05 ̓́\x05 ̓͂\x02ΐ\x05 ̔̀\x05 ̔́\x05 ̔͂" +
+	"\x02ΰ\x05 ̈̀\x01`\x05ὼι\x04ωι\x04ώι\x05ῶι\x06′′\x09′′′\x06‵‵\x09‵‵‵\x02!" +
+	"!\x02??\x02?!\x02!?\x0c′′′′\x010\x014\x015\x016\x017\x018\x019\x01+\x01=" +
+	"\x01(\x01)\x02rs\x02ħ\x02no\x01q\x02sm\x02tm\x02ω\x02å\x02א\x02ב\x02ג" +
+	"\x02ד\x02π\x051⁄7\x051⁄9\x061⁄10\x051⁄3\x052⁄3\x051⁄5\x052⁄5\x053⁄5\x054" +
+	"⁄5\x051⁄6\x055⁄6\x051⁄8\x053⁄8\x055⁄8\x057⁄8\x041⁄\x02ii\x02iv\x02vi" +
+	"\x04viii\x02ix\x02xi\x050⁄3\x06∫∫\x09∫∫∫\x06∮∮\x09∮∮∮\x0210\x0211\x0212" +
+	"\x0213\x0214\x0215\x0216\x0217\x0218\x0219\x0220\x04(10)\x04(11)\x04(12)" +
+	"\x04(13)\x04(14)\x04(15)\x04(16)\x04(17)\x04(18)\x04(19)\x04(20)\x0c∫∫∫∫" +
+	"\x02==\x05⫝̸\x02ɫ\x02ɽ\x02ȿ\x02ɀ\x01.\x04 ゙\x04 ゚\x06より\x06コト\x05(ᄀ)\x05" +
+	"(ᄂ)\x05(ᄃ)\x05(ᄅ)\x05(ᄆ)\x05(ᄇ)\x05(ᄉ)\x05(ᄋ)\x05(ᄌ)\x05(ᄎ)\x05(ᄏ)\x05(ᄐ" +
+	")\x05(ᄑ)\x05(ᄒ)\x05(가)\x05(나)\x05(다)\x05(라)\x05(마)\x05(바)\x05(사)\x05(아)" +
+	"\x05(자)\x05(차)\x05(카)\x05(타)\x05(파)\x05(하)\x05(주)\x08(오전)\x08(오후)\x05(一)" +
+	"\x05(二)\x05(三)\x05(四)\x05(五)\x05(六)\x05(七)\x05(八)\x05(九)\x05(十)\x05(月)" +
+	"\x05(火)\x05(水)\x05(木)\x05(金)\x05(土)\x05(日)\x05(株)\x05(有)\x05(社)\x05(名)" +
+	"\x05(特)\x05(財)\x05(祝)\x05(労)\x05(代)\x05(呼)\x05(学)\x05(監)\x05(企)\x05(資)" +
+	"\x05(協)\x05(祭)\x05(休)\x05(自)\x05(至)\x0221\x0222\x0223\x0224\x0225\x0226" +
+	"\x0227\x0228\x0229\x0230\x0231\x0232\x0233\x0234\x0235\x06참고\x06주의\x0236" +
+	"\x0237\x0238\x0239\x0240\x0241\x0242\x0243\x0244\x0245\x0246\x0247\x0248" +
+	"\x0249\x0250\x041月\x042月\x043月\x044月\x045月\x046月\x047月\x048月\x049月\x0510" +
+	"月\x0511月\x0512月\x02hg\x02ev\x0cアパート\x0cアルファ\x0cアンペア\x09アール\x0cイニング\x09" +
+	"インチ\x09ウォン\x0fエスクード\x0cエーカー\x09オンス\x09オーム\x09カイリ\x0cカラット\x0cカロリー\x09ガロ" +
+	"ン\x09ガンマ\x06ギガ\x09ギニー\x0cキュリー\x0cギルダー\x06キロ\x0fキログラム\x12キロメートル\x0fキロワッ" +
+	"ト\x09グラム\x0fグラムトン\x0fクルゼイロ\x0cクローネ\x09ケース\x09コルナ\x09コーポ\x0cサイクル\x0fサンチ" +
+	"ーム\x0cシリング\x09センチ\x09セント\x09ダース\x06デシ\x06ドル\x06トン\x06ナノ\x09ノット\x09ハイツ" +
+	"\x0fパーセント\x09パーツ\x0cバーレル\x0fピアストル\x09ピクル\x06ピコ\x06ビル\x0fファラッド\x0cフィート" +
+	"\x0fブッシェル\x09フラン\x0fヘクタール\x06ペソ\x09ペニヒ\x09ヘルツ\x09ペンス\x09ページ\x09ベータ\x0cポイ" +
+	"ント\x09ボルト\x06ホン\x09ポンド\x09ホール\x09ホーン\x0cマイクロ\x09マイル\x09マッハ\x09マルク\x0fマ" +
+	"ンション\x0cミクロン\x06ミリ\x0fミリバール\x06メガ\x0cメガトン\x0cメートル\x09ヤード\x09ヤール\x09ユアン" +
+	"\x0cリットル\x06リラ\x09ルピー\x0cルーブル\x06レム\x0fレントゲン\x09ワット\x040点\x041点\x042点" +
+	"\x043点\x044点\x045点\x046点\x047点\x048点\x049点\x0510点\x0511点\x0512点\x0513点" +
+	"\x0514点\x0515点\x0516点\x0517点\x0518点\x0519点\x0520点\x0521点\x0522点\x0523点" +
+	"\x0524点\x02da\x02au\x02ov\x02pc\x02dm\x02iu\x06平成\x06昭和\x06大正\x06明治\x0c株" +
+	"式会社\x02pa\x02na\x02ma\x02ka\x02kb\x02mb\x02gb\x04kcal\x02pf\x02nf\x02m" +
+	"g\x02kg\x02hz\x02ml\x02dl\x02kl\x02fm\x02nm\x02mm\x02cm\x02km\x02m2\x02m" +
+	"3\x05m∕s\x06m∕s2\x07rad∕s\x08rad∕s2\x02ps\x02ns\x02ms\x02pv\x02nv\x02mv" +
+	"\x02kv\x02pw\x02nw\x02mw\x02kw\x02bq\x02cc\x02cd\x06c∕kg\x02db\x02gy\x02" +
+	"ha\x02hp\x02in\x02kk\x02kt\x02lm\x02ln\x02lx\x02ph\x02pr\x02sr\x02sv\x02" +
+	"wb\x05v∕m\x05a∕m\x041日\x042日\x043日\x044日\x045日\x046日\x047日\x048日\x049日" +
+	"\x0510日\x0511日\x0512日\x0513日\x0514日\x0515日\x0516日\x0517日\x0518日\x0519日" +
+	"\x0520日\x0521日\x0522日\x0523日\x0524日\x0525日\x0526日\x0527日\x0528日\x0529日" +
+	"\x0530日\x0531日\x02ь\x02ɦ\x02ɬ\x02ʞ\x02ʇ\x02œ\x04𤋮\x04𢡊\x04𢡄\x04𣏕\x04𥉉" +
+	"\x04𥳐\x04𧻓\x02ff\x02fi\x02fl\x02st\x04մն\x04մե\x04մի\x04վն\x04մխ\x04יִ" +
+	"\x04ײַ\x02ע\x02ה\x02כ\x02ל\x02ם\x02ר\x02ת\x04שׁ\x04שׂ\x06שּׁ\x06שּׂ\x04א" +
+	"ַ\x04אָ\x04אּ\x04בּ\x04גּ\x04דּ\x04הּ\x04וּ\x04זּ\x04טּ\x04יּ\x04ךּ\x04" +
+	"כּ\x04לּ\x04מּ\x04נּ\x04סּ\x04ףּ\x04פּ\x04צּ\x04קּ\x04רּ\x04שּ\x04תּ" +
+	"\x04וֹ\x04בֿ\x04כֿ\x04פֿ\x04אל\x02ٱ\x02ٻ\x02پ\x02ڀ\x02ٺ\x02ٿ\x02ٹ\x02ڤ" +
+	"\x02ڦ\x02ڄ\x02ڃ\x02چ\x02ڇ\x02ڍ\x02ڌ\x02ڎ\x02ڈ\x02ژ\x02ڑ\x02ک\x02گ\x02ڳ" +
+	"\x02ڱ\x02ں\x02ڻ\x02ۀ\x02ہ\x02ھ\x02ے\x02ۓ\x02ڭ\x02ۇ\x02ۆ\x02ۈ\x02ۋ\x02ۅ" +
+	"\x02ۉ\x02ې\x02ى\x04ئا\x04ئە\x04ئو\x04ئۇ\x04ئۆ\x04ئۈ\x04ئې\x04ئى\x02ی\x04" +
+	"ئج\x04ئح\x04ئم\x04ئي\x04بج\x04بح\x04بخ\x04بم\x04بى\x04بي\x04تج\x04تح" +
+	"\x04تخ\x04تم\x04تى\x04تي\x04ثج\x04ثم\x04ثى\x04ثي\x04جح\x04جم\x04حج\x04حم" +
+	"\x04خج\x04خح\x04خم\x04سج\x04سح\x04سخ\x04سم\x04صح\x04صم\x04ضج\x04ضح\x04ضخ" +
+	"\x04ضم\x04طح\x04طم\x04ظم\x04عج\x04عم\x04غج\x04غم\x04فج\x04فح\x04فخ\x04فم" +
+	"\x04فى\x04في\x04قح\x04قم\x04قى\x04قي\x04كا\x04كج\x04كح\x04كخ\x04كل\x04كم" +
+	"\x04كى\x04كي\x04لج\x04لح\x04لخ\x04لم\x04لى\x04لي\x04مج\x04مح\x04مخ\x04مم" +
+	"\x04مى\x04مي\x04نج\x04نح\x04نخ\x04نم\x04نى\x04ني\x04هج\x04هم\x04هى\x04هي" +
+	"\x04يج\x04يح\x04يخ\x04يم\x04يى\x04يي\x04ذٰ\x04رٰ\x04ىٰ\x05 ٌّ\x05 ٍّ\x05" +
+	" َّ\x05 ُّ\x05 ِّ\x05 ّٰ\x04ئر\x04ئز\x04ئن\x04بر\x04بز\x04بن\x04تر\x04تز" +
+	"\x04تن\x04ثر\x04ثز\x04ثن\x04ما\x04نر\x04نز\x04نن\x04ير\x04يز\x04ين\x04ئخ" +
+	"\x04ئه\x04به\x04ته\x04صخ\x04له\x04نه\x04هٰ\x04يه\x04ثه\x04سه\x04شم\x04شه" +
+	"\x06ـَّ\x06ـُّ\x06ـِّ\x04طى\x04طي\x04عى\x04عي\x04غى\x04غي\x04سى\x04سي" +
+	"\x04شى\x04شي\x04حى\x04حي\x04جى\x04جي\x04خى\x04خي\x04صى\x04صي\x04ضى\x04ضي" +
+	"\x04شج\x04شح\x04شخ\x04شر\x04سر\x04صر\x04ضر\x04اً\x06تجم\x06تحج\x06تحم" +
+	"\x06تخم\x06تمج\x06تمح\x06تمخ\x06جمح\x06حمي\x06حمى\x06سحج\x06سجح\x06سجى" +
+	"\x06سمح\x06سمج\x06سمم\x06صحح\x06صمم\x06شحم\x06شجي\x06شمخ\x06شمم\x06ضحى" +
+	"\x06ضخم\x06طمح\x06طمم\x06طمي\x06عجم\x06عمم\x06عمى\x06غمم\x06غمي\x06غمى" +
+	"\x06فخم\x06قمح\x06قمم\x06لحم\x06لحي\x06لحى\x06لجج\x06لخم\x06لمح\x06محج" +
+	"\x06محم\x06محي\x06مجح\x06مجم\x06مخج\x06مخم\x06مجخ\x06همج\x06همم\x06نحم" +
+	"\x06نحى\x06نجم\x06نجى\x06نمي\x06نمى\x06يمم\x06بخي\x06تجي\x06تجى\x06تخي" +
+	"\x06تخى\x06تمي\x06تمى\x06جمي\x06جحى\x06جمى\x06سخى\x06صحي\x06شحي\x06ضحي" +
+	"\x06لجي\x06لمي\x06يحي\x06يجي\x06يمي\x06ممي\x06قمي\x06نحي\x06عمي\x06كمي" +
+	"\x06نجح\x06مخي\x06لجم\x06كمم\x06جحي\x06حجي\x06مجي\x06فمي\x06بحي\x06سخي" +
+	"\x06نجي\x06صلے\x06قلے\x08الله\x08اكبر\x08محمد\x08صلعم\x08رسول\x08عليه" +
+	"\x08وسلم\x06صلى!صلى الله عليه وسلم\x0fجل جلاله\x08ریال\x01,\x01:\x01!" +
+	"\x01?\x01_\x01{\x01}\x01[\x01]\x01#\x01&\x01*\x01-\x01<\x01>\x01\\\x01$" +
+	"\x01%\x01@\x04ـً\x04ـَ\x04ـُ\x04ـِ\x04ـّ\x04ـْ\x02ء\x02آ\x02أ\x02ؤ\x02إ" +
+	"\x02ئ\x02ا\x02ب\x02ة\x02ت\x02ث\x02ج\x02ح\x02خ\x02د\x02ذ\x02ر\x02ز\x02س" +
+	"\x02ش\x02ص\x02ض\x02ط\x02ظ\x02ع\x02غ\x02ف\x02ق\x02ك\x02ل\x02م\x02ن\x02ه" +
+	"\x02و\x02ي\x04لآ\x04لأ\x04لإ\x04لا\x01\x22\x01'\x01/\x01^\x01|\x01~\x02¢" +
+	"\x02£\x02¬\x02¦\x02¥\x08𝅗𝅥\x08𝅘𝅥\x0c𝅘𝅥𝅮\x0c𝅘𝅥𝅯\x0c𝅘𝅥𝅰\x0c𝅘𝅥𝅱\x0c𝅘𝅥𝅲\x08𝆹" +
+	"𝅥\x08𝆺𝅥\x0c𝆹𝅥𝅮\x0c𝆺𝅥𝅮\x0c𝆹𝅥𝅯\x0c𝆺𝅥𝅯\x02ı\x02ȷ\x02α\x02ε\x02ζ\x02η\x02" +
+	"κ\x02λ\x02μ\x02ν\x02ξ\x02ο\x02σ\x02τ\x02υ\x02ψ\x03∇\x03∂\x02ϝ\x02ٮ\x02ڡ" +
+	"\x02ٯ\x020,\x021,\x022,\x023,\x024,\x025,\x026,\x027,\x028,\x029,\x03(a)" +
+	"\x03(b)\x03(c)\x03(d)\x03(e)\x03(f)\x03(g)\x03(h)\x03(i)\x03(j)\x03(k)" +
+	"\x03(l)\x03(m)\x03(n)\x03(o)\x03(p)\x03(q)\x03(r)\x03(s)\x03(t)\x03(u)" +
+	"\x03(v)\x03(w)\x03(x)\x03(y)\x03(z)\x07〔s〕\x02wz\x02hv\x02sd\x03ppv\x02w" +
+	"c\x02mc\x02md\x02mr\x02dj\x06ほか\x06ココ\x03サ\x03手\x03字\x03双\x03デ\x03二\x03多" +
+	"\x03解\x03天\x03交\x03映\x03無\x03料\x03前\x03後\x03再\x03新\x03初\x03終\x03生\x03販" +
+	"\x03声\x03吹\x03演\x03投\x03捕\x03一\x03三\x03遊\x03左\x03中\x03右\x03指\x03走\x03打" +
+	"\x03禁\x03空\x03合\x03満\x03有\x03月\x03申\x03割\x03営\x03配\x09〔本〕\x09〔三〕\x09〔二〕" +
+	"\x09〔安〕\x09〔点〕\x09〔打〕\x09〔盗〕\x09〔勝〕\x09〔敗〕\x03得\x03可\x03丽\x03丸\x03乁\x03你" +
+	"\x03侮\x03侻\x03倂\x03偺\x03備\x03僧\x03像\x03㒞\x03免\x03兔\x03兤\x03具\x03㒹\x03內" +
+	"\x03冗\x03冤\x03仌\x03冬\x03况\x03凵\x03刃\x03㓟\x03刻\x03剆\x03剷\x03㔕\x03勇\x03勉" +
+	"\x03勤\x03勺\x03包\x03匆\x03北\x03卉\x03卑\x03博\x03即\x03卽\x03卿\x03灰\x03及\x03叟" +
+	"\x03叫\x03叱\x03吆\x03咞\x03吸\x03呈\x03周\x03咢\x03哶\x03唐\x03啓\x03啣\x03善\x03喙" +
+	"\x03喫\x03喳\x03嗂\x03圖\x03嘆\x03圗\x03噑\x03噴\x03切\x03壮\x03城\x03埴\x03堍\x03型" +
+	"\x03堲\x03報\x03墬\x03売\x03壷\x03夆\x03夢\x03奢\x03姬\x03娛\x03娧\x03姘\x03婦\x03㛮" +
+	"\x03嬈\x03嬾\x03寃\x03寘\x03寧\x03寳\x03寿\x03将\x03尢\x03㞁\x03屠\x03屮\x03峀\x03岍" +
+	"\x03嵃\x03嵮\x03嵫\x03嵼\x03巡\x03巢\x03㠯\x03巽\x03帨\x03帽\x03幩\x03㡢\x03㡼\x03庰" +
+	"\x03庳\x03庶\x03廊\x03廾\x03舁\x03弢\x03㣇\x03形\x03彫\x03㣣\x03徚\x03忍\x03志\x03忹" +
+	"\x03悁\x03㤺\x03㤜\x03悔\x03惇\x03慈\x03慌\x03慎\x03慺\x03憎\x03憲\x03憤\x03憯\x03懞" +
+	"\x03懲\x03懶\x03成\x03戛\x03扝\x03抱\x03拔\x03捐\x03挽\x03拼\x03捨\x03掃\x03揤\x03搢" +
+	"\x03揅\x03掩\x03㨮\x03摩\x03摾\x03撝\x03摷\x03㩬\x03敏\x03敬\x03旣\x03書\x03晉\x03㬙" +
+	"\x03暑\x03㬈\x03㫤\x03冒\x03冕\x03最\x03暜\x03肭\x03䏙\x03朗\x03望\x03朡\x03杞\x03杓" +
+	"\x03㭉\x03柺\x03枅\x03桒\x03梅\x03梎\x03栟\x03椔\x03㮝\x03楂\x03榣\x03槪\x03檨\x03櫛" +
+	"\x03㰘\x03次\x03歔\x03㱎\x03歲\x03殟\x03殺\x03殻\x03汎\x03沿\x03泍\x03汧\x03洖\x03派" +
+	"\x03海\x03流\x03浩\x03浸\x03涅\x03洴\x03港\x03湮\x03㴳\x03滋\x03滇\x03淹\x03潮\x03濆" +
+	"\x03瀹\x03瀞\x03瀛\x03㶖\x03灊\x03災\x03灷\x03炭\x03煅\x03熜\x03爨\x03爵\x03牐\x03犀" +
+	"\x03犕\x03獺\x03王\x03㺬\x03玥\x03㺸\x03瑇\x03瑜\x03瑱\x03璅\x03瓊\x03㼛\x03甤\x03甾" +
+	"\x03異\x03瘐\x03㿼\x03䀈\x03直\x03眞\x03真\x03睊\x03䀹\x03瞋\x03䁆\x03䂖\x03硎\x03碌" +
+	"\x03磌\x03䃣\x03祖\x03福\x03秫\x03䄯\x03穀\x03穊\x03穏\x03䈂\x03篆\x03築\x03䈧\x03糒" +
+	"\x03䊠\x03糨\x03糣\x03紀\x03絣\x03䌁\x03緇\x03縂\x03繅\x03䌴\x03䍙\x03罺\x03羕\x03翺" +
+	"\x03者\x03聠\x03聰\x03䏕\x03育\x03脃\x03䐋\x03脾\x03媵\x03舄\x03辞\x03䑫\x03芑\x03芋" +
+	"\x03芝\x03劳\x03花\x03芳\x03芽\x03苦\x03若\x03茝\x03荣\x03莭\x03茣\x03莽\x03菧\x03著" +
+	"\x03荓\x03菊\x03菌\x03菜\x03䔫\x03蓱\x03蓳\x03蔖\x03蕤\x03䕝\x03䕡\x03䕫\x03虐\x03虜" +
+	"\x03虧\x03虩\x03蚩\x03蚈\x03蜎\x03蛢\x03蝹\x03蜨\x03蝫\x03螆\x03蟡\x03蠁\x03䗹\x03衠" +
+	"\x03衣\x03裗\x03裞\x03䘵\x03裺\x03㒻\x03䚾\x03䛇\x03誠\x03諭\x03變\x03豕\x03貫\x03賁" +
+	"\x03贛\x03起\x03跋\x03趼\x03跰\x03軔\x03輸\x03邔\x03郱\x03鄑\x03鄛\x03鈸\x03鋗\x03鋘" +
+	"\x03鉼\x03鏹\x03鐕\x03開\x03䦕\x03閷\x03䧦\x03雃\x03嶲\x03霣\x03䩮\x03䩶\x03韠\x03䪲" +
+	"\x03頋\x03頩\x03飢\x03䬳\x03餩\x03馧\x03駂\x03駾\x03䯎\x03鬒\x03鱀\x03鳽\x03䳎\x03䳭" +
+	"\x03鵧\x03䳸\x03麻\x03䵖\x03黹\x03黾\x03鼅\x03鼏\x03鼖\x03鼻"
+
+var xorData string = "" + // Size: 4862 bytes
+	"\x02\x0c\x09\x02\xb0\xec\x02\xad\xd8\x02\xad\xd9\x02\x06\x07\x02\x0f\x12" +
+	"\x02\x0f\x1f\x02\x0f\x1d\x02\x01\x13\x02\x0f\x16\x02\x0f\x0b\x02\x0f3" +
+	"\x02\x0f7\x02\x0f?\x02\x0f/\x02\x0f*\x02\x0c&\x02\x0c*\x02\x0c;\x02\x0c9" +
+	"\x02\x0c%\x02\xab\xed\x02\xab\xe2\x02\xab\xe3\x02\xa9\xe0\x02\xa9\xe1" +
+	"\x02\xa9\xe6\x02\xa3\xcb\x02\xa3\xc8\x02\xa3\xc9\x02\x01#\x02\x01\x08" +
+	"\x02\x0e>\x02\x0e'\x02\x0f\x03\x02\x03\x0d\x02\x03\x09\x02\x03\x17\x02" +
+	"\x03\x0e\x02\x02\x03\x02\x011\x02\x01\x00\x02\x01\x10\x02\x03<\x02\x07" +
+	"\x0d\x02\x02\x0c\x02\x0c0\x02\x01\x03\x02\x01\x01\x02\x01 \x02\x01\x22" +
+	"\x02\x01)\x02\x01\x0a\x02\x01\x0c\x02\x02\x06\x02\x02\x02\x02\x03\x10" +
+	"\x03\x037 \x03\x0b+\x03\x021\x00\x02\x01\x04\x02\x01\x02\x02\x019\x02" +
+	"\x03\x1c\x02\x02$\x03\x80p$\x02\x03:\x02\x03\x0a\x03\xc1r.\x03\xc1r,\x03" +
+	"\xc1r\x02\x02\x02:\x02\x02>\x02\x02,\x02\x02\x10\x02\x02\x00\x03\xc1s<" +
+	"\x03\xc1s*\x03\xc2L$\x03\xc2L;\x02\x09)\x02\x0a\x19\x03\x83\xab\xe3\x03" +
+	"\x83\xab\xf2\x03 4\xe0\x03\x81\xab\xea\x03\x81\xab\xf3\x03 4\xef\x03\x96" +
+	"\xe1\xcd\x03\x84\xe5\xc3\x02\x0d\x11\x03\x8b\xec\xcb\x03\x94\xec\xcf\x03" +
+	"\x9a\xec\xc2\x03\x8b\xec\xdb\x03\x94\xec\xdf\x03\x9a\xec\xd2\x03\x01\x0c" +
+	"!\x03\x01\x0c#\x03ʠ\x9d\x03ʣ\x9c\x03ʢ\x9f\x03ʥ\x9e\x03ʤ\x91\x03ʧ\x90\x03" +
+	"ʦ\x93\x03ʩ\x92\x03ʨ\x95\x03\xca\xf3\xb5\x03\xca\xf0\xb4\x03\xca\xf1\xb7" +
+	"\x03\xca\xf6\xb6\x03\xca\xf7\x89\x03\xca\xf4\x88\x03\xca\xf5\x8b\x03\xca" +
+	"\xfa\x8a\x03\xca\xfb\x8d\x03\xca\xf8\x8c\x03\xca\xf9\x8f\x03\xca\xfe\x8e" +
+	"\x03\xca\xff\x81\x03\xca\xfc\x80\x03\xca\xfd\x83\x03\xca\xe2\x82\x03\xca" +
+	"\xe3\x85\x03\xca\xe0\x84\x03\xca\xe1\x87\x03\xca\xe6\x86\x03\xca\xe7\x99" +
+	"\x03\xca\xe4\x98\x03\xca\xe5\x9b\x03\xca\xea\x9a\x03\xca\xeb\x9d\x03\xca" +
+	"\xe8\x9c\x03ؓ\x89\x03ߔ\x8b\x02\x010\x03\x03\x04\x1e\x03\x04\x15\x12\x03" +
+	"\x0b\x05,\x03\x06\x04\x00\x03\x06\x04)\x03\x06\x044\x03\x06\x04<\x03\x06" +
+	"\x05\x1d\x03\x06\x06\x00\x03\x06\x06\x0a\x03\x06\x06'\x03\x06\x062\x03" +
+	"\x0786\x03\x079/\x03\x079 \x03\x07:\x0e\x03\x07:\x1b\x03\x07:%\x03\x07;/" +
+	"\x03\x07;%\x03\x074\x11\x03\x076\x09\x03\x077*\x03\x070\x01\x03\x070\x0f" +
+	"\x03\x070.\x03\x071\x16\x03\x071\x04\x03\x0710\x03\x072\x18\x03\x072-" +
+	"\x03\x073\x14\x03\x073>\x03\x07'\x09\x03\x07 \x00\x03\x07\x1f\x0b\x03" +
+	"\x07\x18#\x03\x07\x18(\x03\x07\x186\x03\x07\x18\x03\x03\x07\x19\x16\x03" +
+	"\x07\x116\x03\x07\x12'\x03\x07\x13\x10\x03\x07\x0c&\x03\x07\x0c\x08\x03" +
+	"\x07\x0c\x13\x03\x07\x0d\x02\x03\x07\x0d\x1c\x03\x07\x0b5\x03\x07\x0b" +
+	"\x0a\x03\x07\x0b\x01\x03\x07\x0b\x0f\x03\x07\x05\x00\x03\x07\x05\x09\x03" +
+	"\x07\x05\x0b\x03\x07\x07\x01\x03\x07\x07\x08\x03\x07\x00<\x03\x07\x00+" +
+	"\x03\x07\x01)\x03\x07\x01\x1b\x03\x07\x01\x08\x03\x07\x03?\x03\x0445\x03" +
+	"\x044\x08\x03\x0454\x03\x04)/\x03\x04)5\x03\x04+\x05\x03\x04+\x14\x03" +
+	"\x04+ \x03\x04+<\x03\x04*&\x03\x04*\x22\x03\x04&8\x03\x04!\x01\x03\x04!" +
+	"\x22\x03\x04\x11+\x03\x04\x10.\x03\x04\x104\x03\x04\x13=\x03\x04\x12\x04" +
+	"\x03\x04\x12\x0a\x03\x04\x0d\x1d\x03\x04\x0d\x07\x03\x04\x0d \x03\x05<>" +
+	"\x03\x055<\x03\x055!\x03\x055#\x03\x055&\x03\x054\x1d\x03\x054\x02\x03" +
+	"\x054\x07\x03\x0571\x03\x053\x1a\x03\x053\x16\x03\x05.<\x03\x05.\x07\x03" +
+	"\x05):\x03\x05)<\x03\x05)\x0c\x03\x05)\x15\x03\x05+-\x03\x05+5\x03\x05$" +
+	"\x1e\x03\x05$\x14\x03\x05'\x04\x03\x05'\x14\x03\x05&\x02\x03\x05\x226" +
+	"\x03\x05\x22\x0c\x03\x05\x22\x1c\x03\x05\x19\x0a\x03\x05\x1b\x09\x03\x05" +
+	"\x1b\x0c\x03\x05\x14\x07\x03\x05\x16?\x03\x05\x16\x0c\x03\x05\x0c\x05" +
+	"\x03\x05\x0e\x0f\x03\x05\x01\x0e\x03\x05\x00(\x03\x05\x030\x03\x05\x03" +
+	"\x06\x03\x0a==\x03\x0a=1\x03\x0a=,\x03\x0a=\x0c\x03\x0a??\x03\x0a<\x08" +
+	"\x03\x0a9!\x03\x0a9)\x03\x0a97\x03\x0a99\x03\x0a6\x0a\x03\x0a6\x1c\x03" +
+	"\x0a6\x17\x03\x0a7'\x03\x0a78\x03\x0a73\x03\x0a'\x01\x03\x0a'&\x03\x0a" +
+	"\x1f\x0e\x03\x0a\x1f\x03\x03\x0a\x1f3\x03\x0a\x1b/\x03\x0a\x18\x19\x03" +
+	"\x0a\x19\x01\x03\x0a\x16\x14\x03\x0a\x0e\x22\x03\x0a\x0f\x10\x03\x0a\x0f" +
+	"\x02\x03\x0a\x0f \x03\x0a\x0c\x04\x03\x0a\x0b>\x03\x0a\x0b+\x03\x0a\x08/" +
+	"\x03\x0a\x046\x03\x0a\x05\x14\x03\x0a\x00\x04\x03\x0a\x00\x10\x03\x0a" +
+	"\x00\x14\x03\x0b<3\x03\x0b;*\x03\x0b9\x22\x03\x0b9)\x03\x0b97\x03\x0b+" +
+	"\x10\x03\x0b((\x03\x0b&5\x03\x0b$\x1c\x03\x0b$\x12\x03\x0b%\x04\x03\x0b#" +
+	"<\x03\x0b#0\x03\x0b#\x0d\x03\x0b#\x19\x03\x0b!:\x03\x0b!\x1f\x03\x0b!" +
+	"\x00\x03\x0b\x1e5\x03\x0b\x1c\x1d\x03\x0b\x1d-\x03\x0b\x1d(\x03\x0b\x18." +
+	"\x03\x0b\x18 \x03\x0b\x18\x16\x03\x0b\x14\x13\x03\x0b\x15$\x03\x0b\x15" +
+	"\x22\x03\x0b\x12\x1b\x03\x0b\x12\x10\x03\x0b\x132\x03\x0b\x13=\x03\x0b" +
+	"\x12\x18\x03\x0b\x0c&\x03\x0b\x061\x03\x0b\x06:\x03\x0b\x05#\x03\x0b\x05" +
+	"<\x03\x0b\x04\x0b\x03\x0b\x04\x04\x03\x0b\x04\x1b\x03\x0b\x042\x03\x0b" +
+	"\x041\x03\x0b\x03\x03\x03\x0b\x03\x1d\x03\x0b\x03/\x03\x0b\x03+\x03\x0b" +
+	"\x02\x1b\x03\x0b\x02\x00\x03\x0b\x01\x1e\x03\x0b\x01\x08\x03\x0b\x015" +
+	"\x03\x06\x0d9\x03\x06\x0d=\x03\x06\x0d?\x03\x02\x001\x03\x02\x003\x03" +
+	"\x02\x02\x19\x03\x02\x006\x03\x02\x02\x1b\x03\x02\x004\x03\x02\x00<\x03" +
+	"\x02\x02\x0a\x03\x02\x02\x0e\x03\x02\x01\x1a\x03\x02\x01\x07\x03\x02\x01" +
+	"\x05\x03\x02\x01\x0b\x03\x02\x01%\x03\x02\x01\x0c\x03\x02\x01\x04\x03" +
+	"\x02\x01\x1c\x03\x02\x00.\x03\x02\x002\x03\x02\x00>\x03\x02\x00\x12\x03" +
+	"\x02\x00\x16\x03\x02\x011\x03\x02\x013\x03\x02\x02 \x03\x02\x02%\x03\x02" +
+	"\x02$\x03\x02\x028\x03\x02\x02;\x03\x02\x024\x03\x02\x012\x03\x02\x022" +
+	"\x03\x02\x02/\x03\x02\x01,\x03\x02\x01\x13\x03\x02\x01\x16\x03\x02\x01" +
+	"\x11\x03\x02\x01\x1e\x03\x02\x01\x15\x03\x02\x01\x17\x03\x02\x01\x0f\x03" +
+	"\x02\x01\x08\x03\x02\x00?\x03\x02\x03\x07\x03\x02\x03\x0d\x03\x02\x03" +
+	"\x13\x03\x02\x03\x1d\x03\x02\x03\x1f\x03\x02\x00\x03\x03\x02\x00\x0d\x03" +
+	"\x02\x00\x01\x03\x02\x00\x1b\x03\x02\x00\x19\x03\x02\x00\x18\x03\x02\x00" +
+	"\x13\x03\x02\x00/\x03\x07>\x12\x03\x07<\x1f\x03\x07>\x1d\x03\x06\x1d\x0e" +
+	"\x03\x07>\x1c\x03\x07>:\x03\x07>\x13\x03\x04\x12+\x03\x07?\x03\x03\x07>" +
+	"\x02\x03\x06\x224\x03\x06\x1a.\x03\x07<%\x03\x06\x1c\x0b\x03\x0609\x03" +
+	"\x05\x1f\x01\x03\x04'\x08\x03\x93\xfd\xf5\x03\x02\x0d \x03\x02\x0d#\x03" +
+	"\x02\x0d!\x03\x02\x0d&\x03\x02\x0d\x22\x03\x02\x0d/\x03\x02\x0d,\x03\x02" +
+	"\x0d$\x03\x02\x0d'\x03\x02\x0d%\x03\x02\x0d;\x03\x02\x0d=\x03\x02\x0d?" +
+	"\x03\x099.\x03\x08\x0b7\x03\x08\x02\x14\x03\x08\x14\x0d\x03\x08.:\x03" +
+	"\x089'\x03\x0f\x0b\x18\x03\x0f\x1c1\x03\x0f\x17&\x03\x0f9\x1f\x03\x0f0" +
+	"\x0c\x03\x0e\x0a9\x03\x0e\x056\x03\x0e\x1c#\x03\x0f\x13\x0e\x03\x072\x00" +
+	"\x03\x070\x0d\x03\x072\x0b\x03\x06\x11\x18\x03\x070\x10\x03\x06\x0f(\x03" +
+	"\x072\x05\x03\x06\x0f,\x03\x073\x15\x03\x06\x07\x08\x03\x05\x16\x02\x03" +
+	"\x04\x0b \x03\x05:8\x03\x05\x16%\x03\x0a\x0d\x1f\x03\x06\x16\x10\x03\x05" +
+	"\x1d5\x03\x05*;\x03\x05\x16\x1b\x03\x04.-\x03\x06\x1a\x19\x03\x04\x03," +
+	"\x03\x0b87\x03\x04/\x0a\x03\x06\x00,\x03\x04-\x01\x03\x04\x1e-\x03\x06/(" +
+	"\x03\x0a\x0b5\x03\x06\x0e7\x03\x06\x07.\x03\x0597\x03\x0a*%\x03\x0760" +
+	"\x03\x06\x0c;\x03\x05'\x00\x03\x072.\x03\x072\x08\x03\x06=\x01\x03\x06" +
+	"\x05\x1b\x03\x06\x06\x12\x03\x06$=\x03\x06'\x0d\x03\x04\x11\x0f\x03\x076" +
+	",\x03\x06\x07;\x03\x06.,\x03\x86\xf9\xea\x03\x8f\xff\xeb\x02\x092\x02" +
+	"\x095\x02\x094\x02\x09;\x02\x09>\x02\x098\x02\x09*\x02\x09/\x02\x09,\x02" +
+	"\x09%\x02\x09&\x02\x09#\x02\x09 \x02\x08!\x02\x08%\x02\x08$\x02\x08+\x02" +
+	"\x08.\x02\x08*\x02\x08&\x02\x088\x02\x08>\x02\x084\x02\x086\x02\x080\x02" +
+	"\x08\x10\x02\x08\x17\x02\x08\x12\x02\x08\x1d\x02\x08\x1f\x02\x08\x13\x02" +
+	"\x08\x15\x02\x08\x14\x02\x08\x0c\x03\x8b\xfd\xd0\x03\x81\xec\xc6\x03\x87" +
+	"\xe0\x8a\x03-2\xe3\x03\x80\xef\xe4\x03-2\xea\x03\x88\xe6\xeb\x03\x8e\xe6" +
+	"\xe8\x03\x84\xe6\xe9\x03\x97\xe6\xee\x03-2\xf9\x03-2\xf6\x03\x8e\xe3\xad" +
+	"\x03\x80\xe3\x92\x03\x88\xe3\x90\x03\x8e\xe3\x90\x03\x80\xe3\x97\x03\x88" +
+	"\xe3\x95\x03\x88\xfe\xcb\x03\x8e\xfe\xca\x03\x84\xfe\xcd\x03\x91\xef\xc9" +
+	"\x03-2\xc1\x03-2\xc0\x03-2\xcb\x03\x88@\x09\x03\x8e@\x08\x03\x8f\xe0\xf5" +
+	"\x03\x8e\xe6\xf9\x03\x8e\xe0\xfa\x03\x93\xff\xf4\x03\x84\xee\xd3\x03\x0b" +
+	"(\x04\x023 \x03\x0b)\x08\x021;\x02\x01*\x03\x0b#\x10\x03\x0b 0\x03\x0b!" +
+	"\x10\x03\x0b!0\x03\x07\x15\x08\x03\x09?5\x03\x07\x1f\x08\x03\x07\x17\x0b" +
+	"\x03\x09\x1f\x15\x03\x0b\x1c7\x03\x0a+#\x03\x06\x1a\x1b\x03\x06\x1a\x14" +
+	"\x03\x0a\x01\x18\x03\x06#\x1b\x03\x0a2\x0c\x03\x0a\x01\x04\x03\x09#;\x03" +
+	"\x08='\x03\x08\x1a\x0a\x03\x07</\x03\x07:+\x03\x07\x07*\x03\x06&\x1c\x03" +
+	"\x09\x0c\x16\x03\x09\x10\x0e\x03\x08'\x0f\x03\x08+\x09\x03\x074%\x03\x06" +
+	"!3\x03\x06\x03+\x03\x0b\x1e\x19\x03\x0a))\x03\x09\x08\x19\x03\x08,\x05" +
+	"\x03\x07<2\x03\x06\x1c>\x03\x0a\x111\x03\x09\x1b\x09\x03\x073.\x03\x07" +
+	"\x01\x00\x03\x09/,\x03\x07#>\x03\x07\x048\x03\x0a\x1f\x22\x03\x098>\x03" +
+	"\x09\x11\x00\x03\x08/\x17\x03\x06'\x22\x03\x0b\x1a+\x03\x0a\x22\x19\x03" +
+	"\x0a/1\x03\x0974\x03\x09\x0f\x22\x03\x08,\x22\x03\x08?\x14\x03\x07$5\x03" +
+	"\x07<3\x03\x07=*\x03\x07\x13\x18\x03\x068\x0a\x03\x06\x09\x16\x03\x06" +
+	"\x13\x00\x03\x08\x067\x03\x08\x01\x03\x03\x08\x12\x1d\x03\x07+7\x03\x06(" +
+	";\x03\x06\x1c?\x03\x07\x0e\x17\x03\x0a\x06\x1d\x03\x0a\x19\x07\x03\x08" +
+	"\x14$\x03\x07$;\x03\x08,$\x03\x08\x06\x0d\x03\x07\x16\x0a\x03\x06>>\x03" +
+	"\x0a\x06\x12\x03\x0a\x14)\x03\x09\x0d\x1f\x03\x09\x12\x17\x03\x09\x19" +
+	"\x01\x03\x08\x11 \x03\x08\x1d'\x03\x06<\x1a\x03\x0a.\x00\x03\x07'\x18" +
+	"\x03\x0a\x22\x08\x03\x08\x0d\x0a\x03\x08\x13)\x03\x07*)\x03\x06<,\x03" +
+	"\x07\x0b\x1a\x03\x09.\x14\x03\x09\x0d\x1e\x03\x07\x0e#\x03\x0b\x1d'\x03" +
+	"\x0a\x0a8\x03\x09%2\x03\x08+&\x03\x080\x12\x03\x0a)4\x03\x08\x06\x1f\x03" +
+	"\x0b\x1b\x1a\x03\x0a\x1b\x0f\x03\x0b\x1d*\x03\x09\x16$\x03\x090\x11\x03" +
+	"\x08\x11\x08\x03\x0a*(\x03\x0a\x042\x03\x089,\x03\x074'\x03\x07\x0f\x05" +
+	"\x03\x09\x0b\x0a\x03\x07\x1b\x01\x03\x09\x17:\x03\x09.\x0d\x03\x07.\x11" +
+	"\x03\x09+\x15\x03\x080\x13\x03\x0b\x1f\x19\x03\x0a \x11\x03\x0a\x220\x03" +
+	"\x09\x07;\x03\x08\x16\x1c\x03\x07,\x13\x03\x07\x0e/\x03\x06\x221\x03\x0a" +
+	".\x0a\x03\x0a7\x02\x03\x0a\x032\x03\x0a\x1d.\x03\x091\x06\x03\x09\x19:" +
+	"\x03\x08\x02/\x03\x060+\x03\x06\x0f-\x03\x06\x1c\x1f\x03\x06\x1d\x07\x03" +
+	"\x0a,\x11\x03\x09=\x0d\x03\x09\x0b;\x03\x07\x1b/\x03\x0a\x1f:\x03\x09 " +
+	"\x1f\x03\x09.\x10\x03\x094\x0b\x03\x09\x1a1\x03\x08#\x1a\x03\x084\x1d" +
+	"\x03\x08\x01\x1f\x03\x08\x11\x22\x03\x07'8\x03\x07\x1a>\x03\x0757\x03" +
+	"\x06&9\x03\x06+\x11\x03\x0a.\x0b\x03\x0a,>\x03\x0a4#\x03\x08%\x17\x03" +
+	"\x07\x05\x22\x03\x07\x0c\x0b\x03\x0a\x1d+\x03\x0a\x19\x16\x03\x09+\x1f" +
+	"\x03\x09\x08\x0b\x03\x08\x16\x18\x03\x08+\x12\x03\x0b\x1d\x0c\x03\x0a=" +
+	"\x10\x03\x0a\x09\x0d\x03\x0a\x10\x11\x03\x09&0\x03\x08(\x1f\x03\x087\x07" +
+	"\x03\x08\x185\x03\x07'6\x03\x06.\x05\x03\x06=\x04\x03\x06;;\x03\x06\x06," +
+	"\x03\x0b\x18>\x03\x08\x00\x18\x03\x06 \x03\x03\x06<\x00\x03\x09%\x18\x03" +
+	"\x0b\x1c<\x03\x0a%!\x03\x0a\x09\x12\x03\x0a\x16\x02\x03\x090'\x03\x09" +
+	"\x0e=\x03\x08 \x0e\x03\x08>\x03\x03\x074>\x03\x06&?\x03\x06\x19\x09\x03" +
+	"\x06?(\x03\x0a-\x0e\x03\x09:3\x03\x098:\x03\x09\x12\x0b\x03\x09\x1d\x17" +
+	"\x03\x087\x05\x03\x082\x14\x03\x08\x06%\x03\x08\x13\x1f\x03\x06\x06\x0e" +
+	"\x03\x0a\x22<\x03\x09/<\x03\x06>+\x03\x0a'?\x03\x0a\x13\x0c\x03\x09\x10<" +
+	"\x03\x07\x1b=\x03\x0a\x19\x13\x03\x09\x22\x1d\x03\x09\x07\x0d\x03\x08)" +
+	"\x1c\x03\x06=\x1a\x03\x0a/4\x03\x0a7\x11\x03\x0a\x16:\x03\x09?3\x03\x09:" +
+	"/\x03\x09\x05\x0a\x03\x09\x14\x06\x03\x087\x22\x03\x080\x07\x03\x08\x1a" +
+	"\x1f\x03\x07\x04(\x03\x07\x04\x09\x03\x06 %\x03\x06<\x08\x03\x0a+\x14" +
+	"\x03\x09\x1d\x16\x03\x0a70\x03\x08 >\x03\x0857\x03\x070\x0a\x03\x06=\x12" +
+	"\x03\x06\x16%\x03\x06\x1d,\x03\x099#\x03\x09\x10>\x03\x07 \x1e\x03\x08" +
+	"\x0c<\x03\x08\x0b\x18\x03\x08\x15+\x03\x08,:\x03\x08%\x22\x03\x07\x0a$" +
+	"\x03\x0b\x1c=\x03\x07+\x08\x03\x0a/\x05\x03\x0a \x07\x03\x0a\x12'\x03" +
+	"\x09#\x11\x03\x08\x1b\x15\x03\x0a\x06\x01\x03\x09\x1c\x1b\x03\x0922\x03" +
+	"\x07\x14<\x03\x07\x09\x04\x03\x061\x04\x03\x07\x0e\x01\x03\x0a\x13\x18" +
+	"\x03\x0a-\x0c\x03\x0a?\x0d\x03\x0a\x09\x0a\x03\x091&\x03\x0a/\x0b\x03" +
+	"\x08$<\x03\x083\x1d\x03\x08\x0c$\x03\x08\x0d\x07\x03\x08\x0d?\x03\x08" +
+	"\x0e\x14\x03\x065\x0a\x03\x08\x1a#\x03\x08\x16#\x03\x0702\x03\x07\x03" +
+	"\x1a\x03\x06(\x1d\x03\x06+\x1b\x03\x06\x0b\x05\x03\x06\x0b\x17\x03\x06" +
+	"\x0c\x04\x03\x06\x1e\x19\x03\x06+0\x03\x062\x18\x03\x0b\x16\x1e\x03\x0a+" +
+	"\x16\x03\x0a-?\x03\x0a#:\x03\x0a#\x10\x03\x0a%$\x03\x0a>+\x03\x0a01\x03" +
+	"\x0a1\x10\x03\x0a\x099\x03\x0a\x0a\x12\x03\x0a\x19\x1f\x03\x0a\x19\x12" +
+	"\x03\x09*)\x03\x09-\x16\x03\x09.1\x03\x09.2\x03\x09<\x0e\x03\x09> \x03" +
+	"\x093\x12\x03\x09\x0b\x01\x03\x09\x1c2\x03\x09\x11\x1c\x03\x09\x15%\x03" +
+	"\x08,&\x03\x08!\x22\x03\x089(\x03\x08\x0b\x1a\x03\x08\x0d2\x03\x08\x0c" +
+	"\x04\x03\x08\x0c\x06\x03\x08\x0c\x1f\x03\x08\x0c\x0c\x03\x08\x0f\x1f\x03" +
+	"\x08\x0f\x1d\x03\x08\x00\x14\x03\x08\x03\x14\x03\x08\x06\x16\x03\x08\x1e" +
+	"#\x03\x08\x11\x11\x03\x08\x10\x18\x03\x08\x14(\x03\x07)\x1e\x03\x07.1" +
+	"\x03\x07 $\x03\x07 '\x03\x078\x08\x03\x07\x0d0\x03\x07\x0f7\x03\x07\x05#" +
+	"\x03\x07\x05\x1a\x03\x07\x1a7\x03\x07\x1d-\x03\x07\x17\x10\x03\x06)\x1f" +
+	"\x03\x062\x0b\x03\x066\x16\x03\x06\x09\x11\x03\x09(\x1e\x03\x07!5\x03" +
+	"\x0b\x11\x16\x03\x0a/\x04\x03\x0a,\x1a\x03\x0b\x173\x03\x0a,1\x03\x0a/5" +
+	"\x03\x0a\x221\x03\x0a\x22\x0d\x03\x0a?%\x03\x0a<,\x03\x0a?#\x03\x0a>\x19" +
+	"\x03\x0a\x08&\x03\x0a\x0b\x0e\x03\x0a\x0c:\x03\x0a\x0c+\x03\x0a\x03\x22" +
+	"\x03\x0a\x06)\x03\x0a\x11\x10\x03\x0a\x11\x1a\x03\x0a\x17-\x03\x0a\x14(" +
+	"\x03\x09)\x1e\x03\x09/\x09\x03\x09.\x00\x03\x09,\x07\x03\x09/*\x03\x09-9" +
+	"\x03\x09\x228\x03\x09%\x09\x03\x09:\x12\x03\x09;\x1d\x03\x09?\x06\x03" +
+	"\x093%\x03\x096\x05\x03\x096\x08\x03\x097\x02\x03\x09\x07,\x03\x09\x04," +
+	"\x03\x09\x1f\x16\x03\x09\x11\x03\x03\x09\x11\x12\x03\x09\x168\x03\x08*" +
+	"\x05\x03\x08/2\x03\x084:\x03\x08\x22+\x03\x08 0\x03\x08&\x0a\x03\x08;" +
+	"\x10\x03\x08>$\x03\x08>\x18\x03\x0829\x03\x082:\x03\x081,\x03\x081<\x03" +
+	"\x081\x1c\x03\x087#\x03\x087*\x03\x08\x09'\x03\x08\x00\x1d\x03\x08\x05-" +
+	"\x03\x08\x1f4\x03\x08\x1d\x04\x03\x08\x16\x0f\x03\x07*7\x03\x07'!\x03" +
+	"\x07%\x1b\x03\x077\x0c\x03\x07\x0c1\x03\x07\x0c.\x03\x07\x00\x06\x03\x07" +
+	"\x01\x02\x03\x07\x010\x03\x07\x06=\x03\x07\x01\x03\x03\x07\x01\x13\x03" +
+	"\x07\x06\x06\x03\x07\x05\x0a\x03\x07\x1f\x09\x03\x07\x17:\x03\x06*1\x03" +
+	"\x06-\x1d\x03\x06\x223\x03\x062:\x03\x060$\x03\x066\x1e\x03\x064\x12\x03" +
+	"\x0645\x03\x06\x0b\x00\x03\x06\x0b7\x03\x06\x07\x1f\x03\x06\x15\x12\x03" +
+	"\x0c\x05\x0f\x03\x0b+\x0b\x03\x0b+-\x03\x06\x16\x1b\x03\x06\x15\x17\x03" +
+	"\x89\xca\xea\x03\x89\xca\xe8\x03\x0c8\x10\x03\x0c8\x01\x03\x0c8\x0f\x03" +
+	"\x0d8%\x03\x0d8!\x03\x0c8-\x03\x0c8/\x03\x0c8+\x03\x0c87\x03\x0c85\x03" +
+	"\x0c9\x09\x03\x0c9\x0d\x03\x0c9\x0f\x03\x0c9\x0b\x03\xcfu\x0c\x03\xcfu" +
+	"\x0f\x03\xcfu\x0e\x03\xcfu\x09\x03\x0c9\x10\x03\x0d9\x0c\x03\xcf`;\x03" +
+	"\xcf`>\x03\xcf`9\x03\xcf`8\x03\xcf`7\x03\xcf`*\x03\xcf`-\x03\xcf`,\x03" +
+	"\x0d\x1b\x1a\x03\x0d\x1b&\x03\x0c=.\x03\x0c=%\x03\x0c>\x1e\x03\x0c>\x14" +
+	"\x03\x0c?\x06\x03\x0c?\x0b\x03\x0c?\x0c\x03\x0c?\x0d\x03\x0c?\x02\x03" +
+	"\x0c>\x0f\x03\x0c>\x08\x03\x0c>\x09\x03\x0c>,\x03\x0c>\x0c\x03\x0c?\x13" +
+	"\x03\x0c?\x16\x03\x0c?\x15\x03\x0c?\x1c\x03\x0c?\x1f\x03\x0c?\x1d\x03" +
+	"\x0c?\x1a\x03\x0c?\x17\x03\x0c?\x08\x03\x0c?\x09\x03\x0c?\x0e\x03\x0c?" +
+	"\x04\x03\x0c?\x05\x03\x0c<?\x03\x0c=\x00\x03\x0c=\x06\x03\x0c=\x05\x03" +
+	"\x0c=\x0c\x03\x0c=\x0f\x03\x0c=\x0d\x03\x0c=\x0b\x03\x0c=\x07\x03\x0c=" +
+	"\x19\x03\x0c=\x15\x03\x0c=\x11\x03\x0c=1\x03\x0c=3\x03\x0c=0\x03\x0c=>" +
+	"\x03\x0c=2\x03\x0c=6\x03\x0c<\x07\x03\x0c<\x05\x03\x0e:!\x03\x0e:#\x03" +
+	"\x0e8\x09\x03\x0e:&\x03\x0e8\x0b\x03\x0e:$\x03\x0e:,\x03\x0e8\x1a\x03" +
+	"\x0e8\x1e\x03\x0e:*\x03\x0e:7\x03\x0e:5\x03\x0e:;\x03\x0e:\x15\x03\x0e:<" +
+	"\x03\x0e:4\x03\x0e:'\x03\x0e:-\x03\x0e:%\x03\x0e:?\x03\x0e:=\x03\x0e:)" +
+	"\x03\x0e:/\x03\xcfs'\x03\x0d=\x0f\x03\x0d+*\x03\x0d99\x03\x0d9;\x03\x0d9" +
+	"?\x03\x0d)\x0d\x03\x0d(%\x02\x01\x18\x02\x01(\x02\x01\x1e\x03\x0f$!\x03" +
+	"\x0f87\x03\x0f4\x0e\x03\x0f5\x1d\x03\x06'\x03\x03\x0f\x08\x18\x03\x0f" +
+	"\x0d\x1b\x03\x0e2=\x03\x0e;\x08\x03\x0e:\x0b\x03\x0e\x06$\x03\x0e\x0d)" +
+	"\x03\x0e\x16\x1f\x03\x0e\x16\x1b\x03\x0d$\x0a\x03\x05,\x1d\x03\x0d. \x03" +
+	"\x0d.#\x03\x0c(/\x03\x09%\x02\x03\x0d90\x03\x0d\x0e4\x03\x0d\x0d\x0f\x03" +
+	"\x0c#\x00\x03\x0c,\x1e\x03\x0c2\x0e\x03\x0c\x01\x17\x03\x0c\x09:\x03\x0e" +
+	"\x173\x03\x0c\x08\x03\x03\x0c\x11\x07\x03\x0c\x10\x18\x03\x0c\x1f\x1c" +
+	"\x03\x0c\x19\x0e\x03\x0c\x1a\x1f\x03\x0f0>\x03\x0b->\x03\x0b<+\x03\x0b8" +
+	"\x13\x03\x0b\x043\x03\x0b\x14\x03\x03\x0b\x16%\x03\x0d\x22&\x03\x0b\x1a" +
+	"\x1a\x03\x0b\x1a\x04\x03\x0a%9\x03\x0a&2\x03\x0a&0\x03\x0a!\x1a\x03\x0a!" +
+	"7\x03\x0a5\x10\x03\x0a=4\x03\x0a?\x0e\x03\x0a>\x10\x03\x0a\x00 \x03\x0a" +
+	"\x0f:\x03\x0a\x0f9\x03\x0a\x0b\x0a\x03\x0a\x17%\x03\x0a\x1b-\x03\x09-" +
+	"\x1a\x03\x09,4\x03\x09.,\x03\x09)\x09\x03\x096!\x03\x091\x1f\x03\x093" +
+	"\x16\x03\x0c+\x1f\x03\x098 \x03\x098=\x03\x0c(\x1a\x03\x0c(\x16\x03\x09" +
+	"\x0a+\x03\x09\x16\x12\x03\x09\x13\x0e\x03\x09\x153\x03\x08)!\x03\x09\x1a" +
+	"\x01\x03\x09\x18\x01\x03\x08%#\x03\x08>\x22\x03\x08\x05%\x03\x08\x02*" +
+	"\x03\x08\x15;\x03\x08\x1b7\x03\x0f\x07\x1d\x03\x0f\x04\x03\x03\x070\x0c" +
+	"\x03\x07;\x0b\x03\x07\x08\x17\x03\x07\x12\x06\x03\x06/-\x03\x0671\x03" +
+	"\x065+\x03\x06>7\x03\x06\x049\x03\x05+\x1e\x03\x05,\x17\x03\x05 \x1d\x03" +
+	"\x05\x22\x05\x03\x050\x1d"
+
+// lookup returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *idnaTrie) lookup(s []byte) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return idnaValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = idnaIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *idnaTrie) lookupUnsafe(s []byte) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return idnaValues[c0]
+	}
+	i := idnaIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// lookupString returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *idnaTrie) lookupString(s string) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return idnaValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = idnaIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *idnaTrie) lookupStringUnsafe(s string) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return idnaValues[c0]
+	}
+	i := idnaIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// idnaTrie. Total size: 29708 bytes (29.01 KiB). Checksum: c3ecc76d8fffa6e6.
+type idnaTrie struct{}
+
+func newIdnaTrie(i int) *idnaTrie {
+	return &idnaTrie{}
+}
+
+// lookupValue determines the type of block n and looks up the value for b.
+func (t *idnaTrie) lookupValue(n uint32, b byte) uint16 {
+	switch {
+	case n < 125:
+		return uint16(idnaValues[n<<6+uint32(b)])
+	default:
+		n -= 125
+		return uint16(idnaSparse.lookup(n, b))
+	}
+}
+
+// idnaValues: 127 blocks, 8128 entries, 16256 bytes
+// The third block is the zero block.
+var idnaValues = [8128]uint16{
+	// Block 0x0, offset 0x0
+	0x00: 0x0080, 0x01: 0x0080, 0x02: 0x0080, 0x03: 0x0080, 0x04: 0x0080, 0x05: 0x0080,
+	0x06: 0x0080, 0x07: 0x0080, 0x08: 0x0080, 0x09: 0x0080, 0x0a: 0x0080, 0x0b: 0x0080,
+	0x0c: 0x0080, 0x0d: 0x0080, 0x0e: 0x0080, 0x0f: 0x0080, 0x10: 0x0080, 0x11: 0x0080,
+	0x12: 0x0080, 0x13: 0x0080, 0x14: 0x0080, 0x15: 0x0080, 0x16: 0x0080, 0x17: 0x0080,
+	0x18: 0x0080, 0x19: 0x0080, 0x1a: 0x0080, 0x1b: 0x0080, 0x1c: 0x0080, 0x1d: 0x0080,
+	0x1e: 0x0080, 0x1f: 0x0080, 0x20: 0x0080, 0x21: 0x0080, 0x22: 0x0080, 0x23: 0x0080,
+	0x24: 0x0080, 0x25: 0x0080, 0x26: 0x0080, 0x27: 0x0080, 0x28: 0x0080, 0x29: 0x0080,
+	0x2a: 0x0080, 0x2b: 0x0080, 0x2c: 0x0080, 0x2d: 0x0008, 0x2e: 0x0008, 0x2f: 0x0080,
+	0x30: 0x0008, 0x31: 0x0008, 0x32: 0x0008, 0x33: 0x0008, 0x34: 0x0008, 0x35: 0x0008,
+	0x36: 0x0008, 0x37: 0x0008, 0x38: 0x0008, 0x39: 0x0008, 0x3a: 0x0080, 0x3b: 0x0080,
+	0x3c: 0x0080, 0x3d: 0x0080, 0x3e: 0x0080, 0x3f: 0x0080,
+	// Block 0x1, offset 0x40
+	0x40: 0x0080, 0x41: 0xe105, 0x42: 0xe105, 0x43: 0xe105, 0x44: 0xe105, 0x45: 0xe105,
+	0x46: 0xe105, 0x47: 0xe105, 0x48: 0xe105, 0x49: 0xe105, 0x4a: 0xe105, 0x4b: 0xe105,
+	0x4c: 0xe105, 0x4d: 0xe105, 0x4e: 0xe105, 0x4f: 0xe105, 0x50: 0xe105, 0x51: 0xe105,
+	0x52: 0xe105, 0x53: 0xe105, 0x54: 0xe105, 0x55: 0xe105, 0x56: 0xe105, 0x57: 0xe105,
+	0x58: 0xe105, 0x59: 0xe105, 0x5a: 0xe105, 0x5b: 0x0080, 0x5c: 0x0080, 0x5d: 0x0080,
+	0x5e: 0x0080, 0x5f: 0x0080, 0x60: 0x0080, 0x61: 0x0008, 0x62: 0x0008, 0x63: 0x0008,
+	0x64: 0x0008, 0x65: 0x0008, 0x66: 0x0008, 0x67: 0x0008, 0x68: 0x0008, 0x69: 0x0008,
+	0x6a: 0x0008, 0x6b: 0x0008, 0x6c: 0x0008, 0x6d: 0x0008, 0x6e: 0x0008, 0x6f: 0x0008,
+	0x70: 0x0008, 0x71: 0x0008, 0x72: 0x0008, 0x73: 0x0008, 0x74: 0x0008, 0x75: 0x0008,
+	0x76: 0x0008, 0x77: 0x0008, 0x78: 0x0008, 0x79: 0x0008, 0x7a: 0x0008, 0x7b: 0x0080,
+	0x7c: 0x0080, 0x7d: 0x0080, 0x7e: 0x0080, 0x7f: 0x0080,
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc0: 0x0040, 0xc1: 0x0040, 0xc2: 0x0040, 0xc3: 0x0040, 0xc4: 0x0040, 0xc5: 0x0040,
+	0xc6: 0x0040, 0xc7: 0x0040, 0xc8: 0x0040, 0xc9: 0x0040, 0xca: 0x0040, 0xcb: 0x0040,
+	0xcc: 0x0040, 0xcd: 0x0040, 0xce: 0x0040, 0xcf: 0x0040, 0xd0: 0x0040, 0xd1: 0x0040,
+	0xd2: 0x0040, 0xd3: 0x0040, 0xd4: 0x0040, 0xd5: 0x0040, 0xd6: 0x0040, 0xd7: 0x0040,
+	0xd8: 0x0040, 0xd9: 0x0040, 0xda: 0x0040, 0xdb: 0x0040, 0xdc: 0x0040, 0xdd: 0x0040,
+	0xde: 0x0040, 0xdf: 0x0040, 0xe0: 0x000a, 0xe1: 0x0018, 0xe2: 0x0018, 0xe3: 0x0018,
+	0xe4: 0x0018, 0xe5: 0x0018, 0xe6: 0x0018, 0xe7: 0x0018, 0xe8: 0x001a, 0xe9: 0x0018,
+	0xea: 0x0039, 0xeb: 0x0018, 0xec: 0x0018, 0xed: 0x03c0, 0xee: 0x0018, 0xef: 0x004a,
+	0xf0: 0x0018, 0xf1: 0x0018, 0xf2: 0x0069, 0xf3: 0x0079, 0xf4: 0x008a, 0xf5: 0x0005,
+	0xf6: 0x0018, 0xf7: 0x0008, 0xf8: 0x00aa, 0xf9: 0x00c9, 0xfa: 0x00d9, 0xfb: 0x0018,
+	0xfc: 0x00e9, 0xfd: 0x0119, 0xfe: 0x0149, 0xff: 0x0018,
+	// Block 0x4, offset 0x100
+	0x100: 0xe00d, 0x101: 0x0008, 0x102: 0xe00d, 0x103: 0x0008, 0x104: 0xe00d, 0x105: 0x0008,
+	0x106: 0xe00d, 0x107: 0x0008, 0x108: 0xe00d, 0x109: 0x0008, 0x10a: 0xe00d, 0x10b: 0x0008,
+	0x10c: 0xe00d, 0x10d: 0x0008, 0x10e: 0xe00d, 0x10f: 0x0008, 0x110: 0xe00d, 0x111: 0x0008,
+	0x112: 0xe00d, 0x113: 0x0008, 0x114: 0xe00d, 0x115: 0x0008, 0x116: 0xe00d, 0x117: 0x0008,
+	0x118: 0xe00d, 0x119: 0x0008, 0x11a: 0xe00d, 0x11b: 0x0008, 0x11c: 0xe00d, 0x11d: 0x0008,
+	0x11e: 0xe00d, 0x11f: 0x0008, 0x120: 0xe00d, 0x121: 0x0008, 0x122: 0xe00d, 0x123: 0x0008,
+	0x124: 0xe00d, 0x125: 0x0008, 0x126: 0xe00d, 0x127: 0x0008, 0x128: 0xe00d, 0x129: 0x0008,
+	0x12a: 0xe00d, 0x12b: 0x0008, 0x12c: 0xe00d, 0x12d: 0x0008, 0x12e: 0xe00d, 0x12f: 0x0008,
+	0x130: 0x0179, 0x131: 0x0008, 0x132: 0x0035, 0x133: 0x004d, 0x134: 0xe00d, 0x135: 0x0008,
+	0x136: 0xe00d, 0x137: 0x0008, 0x138: 0x0008, 0x139: 0xe01d, 0x13a: 0x0008, 0x13b: 0xe03d,
+	0x13c: 0x0008, 0x13d: 0xe01d, 0x13e: 0x0008, 0x13f: 0x0199,
+	// Block 0x5, offset 0x140
+	0x140: 0x0199, 0x141: 0xe01d, 0x142: 0x0008, 0x143: 0xe03d, 0x144: 0x0008, 0x145: 0xe01d,
+	0x146: 0x0008, 0x147: 0xe07d, 0x148: 0x0008, 0x149: 0x01b9, 0x14a: 0xe00d, 0x14b: 0x0008,
+	0x14c: 0xe00d, 0x14d: 0x0008, 0x14e: 0xe00d, 0x14f: 0x0008, 0x150: 0xe00d, 0x151: 0x0008,
+	0x152: 0xe00d, 0x153: 0x0008, 0x154: 0xe00d, 0x155: 0x0008, 0x156: 0xe00d, 0x157: 0x0008,
+	0x158: 0xe00d, 0x159: 0x0008, 0x15a: 0xe00d, 0x15b: 0x0008, 0x15c: 0xe00d, 0x15d: 0x0008,
+	0x15e: 0xe00d, 0x15f: 0x0008, 0x160: 0xe00d, 0x161: 0x0008, 0x162: 0xe00d, 0x163: 0x0008,
+	0x164: 0xe00d, 0x165: 0x0008, 0x166: 0xe00d, 0x167: 0x0008, 0x168: 0xe00d, 0x169: 0x0008,
+	0x16a: 0xe00d, 0x16b: 0x0008, 0x16c: 0xe00d, 0x16d: 0x0008, 0x16e: 0xe00d, 0x16f: 0x0008,
+	0x170: 0xe00d, 0x171: 0x0008, 0x172: 0xe00d, 0x173: 0x0008, 0x174: 0xe00d, 0x175: 0x0008,
+	0x176: 0xe00d, 0x177: 0x0008, 0x178: 0x0065, 0x179: 0xe01d, 0x17a: 0x0008, 0x17b: 0xe03d,
+	0x17c: 0x0008, 0x17d: 0xe01d, 0x17e: 0x0008, 0x17f: 0x01d9,
+	// Block 0x6, offset 0x180
+	0x180: 0x0008, 0x181: 0x007d, 0x182: 0xe00d, 0x183: 0x0008, 0x184: 0xe00d, 0x185: 0x0008,
+	0x186: 0x007d, 0x187: 0xe07d, 0x188: 0x0008, 0x189: 0x0095, 0x18a: 0x00ad, 0x18b: 0xe03d,
+	0x18c: 0x0008, 0x18d: 0x0008, 0x18e: 0x00c5, 0x18f: 0x00dd, 0x190: 0x00f5, 0x191: 0xe01d,
+	0x192: 0x0008, 0x193: 0x010d, 0x194: 0x0125, 0x195: 0x0008, 0x196: 0x013d, 0x197: 0x013d,
+	0x198: 0xe00d, 0x199: 0x0008, 0x19a: 0x0008, 0x19b: 0x0008, 0x19c: 0x010d, 0x19d: 0x0155,
+	0x19e: 0x0008, 0x19f: 0x016d, 0x1a0: 0xe00d, 0x1a1: 0x0008, 0x1a2: 0xe00d, 0x1a3: 0x0008,
+	0x1a4: 0xe00d, 0x1a5: 0x0008, 0x1a6: 0x0185, 0x1a7: 0xe07d, 0x1a8: 0x0008, 0x1a9: 0x019d,
+	0x1aa: 0x0008, 0x1ab: 0x0008, 0x1ac: 0xe00d, 0x1ad: 0x0008, 0x1ae: 0x0185, 0x1af: 0xe0fd,
+	0x1b0: 0x0008, 0x1b1: 0x01b5, 0x1b2: 0x01cd, 0x1b3: 0xe03d, 0x1b4: 0x0008, 0x1b5: 0xe01d,
+	0x1b6: 0x0008, 0x1b7: 0x01e5, 0x1b8: 0xe00d, 0x1b9: 0x0008, 0x1ba: 0x0008, 0x1bb: 0x0008,
+	0x1bc: 0xe00d, 0x1bd: 0x0008, 0x1be: 0x0008, 0x1bf: 0x0008,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x0008, 0x1c1: 0x0008, 0x1c2: 0x0008, 0x1c3: 0x0008, 0x1c4: 0x01e9, 0x1c5: 0x01e9,
+	0x1c6: 0x01e9, 0x1c7: 0x01fd, 0x1c8: 0x0215, 0x1c9: 0x022d, 0x1ca: 0x0245, 0x1cb: 0x025d,
+	0x1cc: 0x0275, 0x1cd: 0xe01d, 0x1ce: 0x0008, 0x1cf: 0xe0fd, 0x1d0: 0x0008, 0x1d1: 0xe01d,
+	0x1d2: 0x0008, 0x1d3: 0xe03d, 0x1d4: 0x0008, 0x1d5: 0xe01d, 0x1d6: 0x0008, 0x1d7: 0xe07d,
+	0x1d8: 0x0008, 0x1d9: 0xe01d, 0x1da: 0x0008, 0x1db: 0xe03d, 0x1dc: 0x0008, 0x1dd: 0x0008,
+	0x1de: 0xe00d, 0x1df: 0x0008, 0x1e0: 0xe00d, 0x1e1: 0x0008, 0x1e2: 0xe00d, 0x1e3: 0x0008,
+	0x1e4: 0xe00d, 0x1e5: 0x0008, 0x1e6: 0xe00d, 0x1e7: 0x0008, 0x1e8: 0xe00d, 0x1e9: 0x0008,
+	0x1ea: 0xe00d, 0x1eb: 0x0008, 0x1ec: 0xe00d, 0x1ed: 0x0008, 0x1ee: 0xe00d, 0x1ef: 0x0008,
+	0x1f0: 0x0008, 0x1f1: 0x028d, 0x1f2: 0x02a5, 0x1f3: 0x02bd, 0x1f4: 0xe00d, 0x1f5: 0x0008,
+	0x1f6: 0x02d5, 0x1f7: 0x02ed, 0x1f8: 0xe00d, 0x1f9: 0x0008, 0x1fa: 0xe00d, 0x1fb: 0x0008,
+	0x1fc: 0xe00d, 0x1fd: 0x0008, 0x1fe: 0xe00d, 0x1ff: 0x0008,
+	// Block 0x8, offset 0x200
+	0x200: 0xe00d, 0x201: 0x0008, 0x202: 0xe00d, 0x203: 0x0008, 0x204: 0xe00d, 0x205: 0x0008,
+	0x206: 0xe00d, 0x207: 0x0008, 0x208: 0xe00d, 0x209: 0x0008, 0x20a: 0xe00d, 0x20b: 0x0008,
+	0x20c: 0xe00d, 0x20d: 0x0008, 0x20e: 0xe00d, 0x20f: 0x0008, 0x210: 0xe00d, 0x211: 0x0008,
+	0x212: 0xe00d, 0x213: 0x0008, 0x214: 0xe00d, 0x215: 0x0008, 0x216: 0xe00d, 0x217: 0x0008,
+	0x218: 0xe00d, 0x219: 0x0008, 0x21a: 0xe00d, 0x21b: 0x0008, 0x21c: 0xe00d, 0x21d: 0x0008,
+	0x21e: 0xe00d, 0x21f: 0x0008, 0x220: 0x0305, 0x221: 0x0008, 0x222: 0xe00d, 0x223: 0x0008,
+	0x224: 0xe00d, 0x225: 0x0008, 0x226: 0xe00d, 0x227: 0x0008, 0x228: 0xe00d, 0x229: 0x0008,
+	0x22a: 0xe00d, 0x22b: 0x0008, 0x22c: 0xe00d, 0x22d: 0x0008, 0x22e: 0xe00d, 0x22f: 0x0008,
+	0x230: 0xe00d, 0x231: 0x0008, 0x232: 0xe00d, 0x233: 0x0008, 0x234: 0x0008, 0x235: 0x0008,
+	0x236: 0x0008, 0x237: 0x0008, 0x238: 0x0008, 0x239: 0x0008, 0x23a: 0x0209, 0x23b: 0xe03d,
+	0x23c: 0x0008, 0x23d: 0x031d, 0x23e: 0x0229, 0x23f: 0x0008,
+	// Block 0x9, offset 0x240
+	0x240: 0x0008, 0x241: 0x0008, 0x242: 0x0018, 0x243: 0x0018, 0x244: 0x0018, 0x245: 0x0018,
+	0x246: 0x0008, 0x247: 0x0008, 0x248: 0x0008, 0x249: 0x0008, 0x24a: 0x0008, 0x24b: 0x0008,
+	0x24c: 0x0008, 0x24d: 0x0008, 0x24e: 0x0008, 0x24f: 0x0008, 0x250: 0x0008, 0x251: 0x0008,
+	0x252: 0x0018, 0x253: 0x0018, 0x254: 0x0018, 0x255: 0x0018, 0x256: 0x0018, 0x257: 0x0018,
+	0x258: 0x029a, 0x259: 0x02ba, 0x25a: 0x02da, 0x25b: 0x02fa, 0x25c: 0x031a, 0x25d: 0x033a,
+	0x25e: 0x0018, 0x25f: 0x0018, 0x260: 0x03ad, 0x261: 0x0359, 0x262: 0x01d9, 0x263: 0x0369,
+	0x264: 0x03c5, 0x265: 0x0018, 0x266: 0x0018, 0x267: 0x0018, 0x268: 0x0018, 0x269: 0x0018,
+	0x26a: 0x0018, 0x26b: 0x0018, 0x26c: 0x0008, 0x26d: 0x0018, 0x26e: 0x0008, 0x26f: 0x0018,
+	0x270: 0x0018, 0x271: 0x0018, 0x272: 0x0018, 0x273: 0x0018, 0x274: 0x0018, 0x275: 0x0018,
+	0x276: 0x0018, 0x277: 0x0018, 0x278: 0x0018, 0x279: 0x0018, 0x27a: 0x0018, 0x27b: 0x0018,
+	0x27c: 0x0018, 0x27d: 0x0018, 0x27e: 0x0018, 0x27f: 0x0018,
+	// Block 0xa, offset 0x280
+	0x280: 0x03dd, 0x281: 0x03dd, 0x282: 0x3308, 0x283: 0x03f5, 0x284: 0x0379, 0x285: 0x040d,
+	0x286: 0x3308, 0x287: 0x3308, 0x288: 0x3308, 0x289: 0x3308, 0x28a: 0x3308, 0x28b: 0x3308,
+	0x28c: 0x3308, 0x28d: 0x3308, 0x28e: 0x3308, 0x28f: 0x33c0, 0x290: 0x3308, 0x291: 0x3308,
+	0x292: 0x3308, 0x293: 0x3308, 0x294: 0x3308, 0x295: 0x3308, 0x296: 0x3308, 0x297: 0x3308,
+	0x298: 0x3308, 0x299: 0x3308, 0x29a: 0x3308, 0x29b: 0x3308, 0x29c: 0x3308, 0x29d: 0x3308,
+	0x29e: 0x3308, 0x29f: 0x3308, 0x2a0: 0x3308, 0x2a1: 0x3308, 0x2a2: 0x3308, 0x2a3: 0x3308,
+	0x2a4: 0x3308, 0x2a5: 0x3308, 0x2a6: 0x3308, 0x2a7: 0x3308, 0x2a8: 0x3308, 0x2a9: 0x3308,
+	0x2aa: 0x3308, 0x2ab: 0x3308, 0x2ac: 0x3308, 0x2ad: 0x3308, 0x2ae: 0x3308, 0x2af: 0x3308,
+	0x2b0: 0xe00d, 0x2b1: 0x0008, 0x2b2: 0xe00d, 0x2b3: 0x0008, 0x2b4: 0x0425, 0x2b5: 0x0008,
+	0x2b6: 0xe00d, 0x2b7: 0x0008, 0x2b8: 0x0040, 0x2b9: 0x0040, 0x2ba: 0x03a2, 0x2bb: 0x0008,
+	0x2bc: 0x0008, 0x2bd: 0x0008, 0x2be: 0x03c2, 0x2bf: 0x043d,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0x0040, 0x2c1: 0x0040, 0x2c2: 0x0040, 0x2c3: 0x0040, 0x2c4: 0x008a, 0x2c5: 0x03d2,
+	0x2c6: 0xe155, 0x2c7: 0x0455, 0x2c8: 0xe12d, 0x2c9: 0xe13d, 0x2ca: 0xe12d, 0x2cb: 0x0040,
+	0x2cc: 0x03dd, 0x2cd: 0x0040, 0x2ce: 0x046d, 0x2cf: 0x0485, 0x2d0: 0x0008, 0x2d1: 0xe105,
+	0x2d2: 0xe105, 0x2d3: 0xe105, 0x2d4: 0xe105, 0x2d5: 0xe105, 0x2d6: 0xe105, 0x2d7: 0xe105,
+	0x2d8: 0xe105, 0x2d9: 0xe105, 0x2da: 0xe105, 0x2db: 0xe105, 0x2dc: 0xe105, 0x2dd: 0xe105,
+	0x2de: 0xe105, 0x2df: 0xe105, 0x2e0: 0x049d, 0x2e1: 0x049d, 0x2e2: 0x0040, 0x2e3: 0x049d,
+	0x2e4: 0x049d, 0x2e5: 0x049d, 0x2e6: 0x049d, 0x2e7: 0x049d, 0x2e8: 0x049d, 0x2e9: 0x049d,
+	0x2ea: 0x049d, 0x2eb: 0x049d, 0x2ec: 0x0008, 0x2ed: 0x0008, 0x2ee: 0x0008, 0x2ef: 0x0008,
+	0x2f0: 0x0008, 0x2f1: 0x0008, 0x2f2: 0x0008, 0x2f3: 0x0008, 0x2f4: 0x0008, 0x2f5: 0x0008,
+	0x2f6: 0x0008, 0x2f7: 0x0008, 0x2f8: 0x0008, 0x2f9: 0x0008, 0x2fa: 0x0008, 0x2fb: 0x0008,
+	0x2fc: 0x0008, 0x2fd: 0x0008, 0x2fe: 0x0008, 0x2ff: 0x0008,
+	// Block 0xc, offset 0x300
+	0x300: 0x0008, 0x301: 0x0008, 0x302: 0xe00f, 0x303: 0x0008, 0x304: 0x0008, 0x305: 0x0008,
+	0x306: 0x0008, 0x307: 0x0008, 0x308: 0x0008, 0x309: 0x0008, 0x30a: 0x0008, 0x30b: 0x0008,
+	0x30c: 0x0008, 0x30d: 0x0008, 0x30e: 0x0008, 0x30f: 0xe0c5, 0x310: 0x04b5, 0x311: 0x04cd,
+	0x312: 0xe0bd, 0x313: 0xe0f5, 0x314: 0xe0fd, 0x315: 0xe09d, 0x316: 0xe0b5, 0x317: 0x0008,
+	0x318: 0xe00d, 0x319: 0x0008, 0x31a: 0xe00d, 0x31b: 0x0008, 0x31c: 0xe00d, 0x31d: 0x0008,
+	0x31e: 0xe00d, 0x31f: 0x0008, 0x320: 0xe00d, 0x321: 0x0008, 0x322: 0xe00d, 0x323: 0x0008,
+	0x324: 0xe00d, 0x325: 0x0008, 0x326: 0xe00d, 0x327: 0x0008, 0x328: 0xe00d, 0x329: 0x0008,
+	0x32a: 0xe00d, 0x32b: 0x0008, 0x32c: 0xe00d, 0x32d: 0x0008, 0x32e: 0xe00d, 0x32f: 0x0008,
+	0x330: 0x04e5, 0x331: 0xe185, 0x332: 0xe18d, 0x333: 0x0008, 0x334: 0x04fd, 0x335: 0x03dd,
+	0x336: 0x0018, 0x337: 0xe07d, 0x338: 0x0008, 0x339: 0xe1d5, 0x33a: 0xe00d, 0x33b: 0x0008,
+	0x33c: 0x0008, 0x33d: 0x0515, 0x33e: 0x052d, 0x33f: 0x052d,
+	// Block 0xd, offset 0x340
+	0x340: 0x0008, 0x341: 0x0008, 0x342: 0x0008, 0x343: 0x0008, 0x344: 0x0008, 0x345: 0x0008,
+	0x346: 0x0008, 0x347: 0x0008, 0x348: 0x0008, 0x349: 0x0008, 0x34a: 0x0008, 0x34b: 0x0008,
+	0x34c: 0x0008, 0x34d: 0x0008, 0x34e: 0x0008, 0x34f: 0x0008, 0x350: 0x0008, 0x351: 0x0008,
+	0x352: 0x0008, 0x353: 0x0008, 0x354: 0x0008, 0x355: 0x0008, 0x356: 0x0008, 0x357: 0x0008,
+	0x358: 0x0008, 0x359: 0x0008, 0x35a: 0x0008, 0x35b: 0x0008, 0x35c: 0x0008, 0x35d: 0x0008,
+	0x35e: 0x0008, 0x35f: 0x0008, 0x360: 0xe00d, 0x361: 0x0008, 0x362: 0xe00d, 0x363: 0x0008,
+	0x364: 0xe00d, 0x365: 0x0008, 0x366: 0xe00d, 0x367: 0x0008, 0x368: 0xe00d, 0x369: 0x0008,
+	0x36a: 0xe00d, 0x36b: 0x0008, 0x36c: 0xe00d, 0x36d: 0x0008, 0x36e: 0xe00d, 0x36f: 0x0008,
+	0x370: 0xe00d, 0x371: 0x0008, 0x372: 0xe00d, 0x373: 0x0008, 0x374: 0xe00d, 0x375: 0x0008,
+	0x376: 0xe00d, 0x377: 0x0008, 0x378: 0xe00d, 0x379: 0x0008, 0x37a: 0xe00d, 0x37b: 0x0008,
+	0x37c: 0xe00d, 0x37d: 0x0008, 0x37e: 0xe00d, 0x37f: 0x0008,
+	// Block 0xe, offset 0x380
+	0x380: 0xe00d, 0x381: 0x0008, 0x382: 0x0018, 0x383: 0x3308, 0x384: 0x3308, 0x385: 0x3308,
+	0x386: 0x3308, 0x387: 0x3308, 0x388: 0x3318, 0x389: 0x3318, 0x38a: 0xe00d, 0x38b: 0x0008,
+	0x38c: 0xe00d, 0x38d: 0x0008, 0x38e: 0xe00d, 0x38f: 0x0008, 0x390: 0xe00d, 0x391: 0x0008,
+	0x392: 0xe00d, 0x393: 0x0008, 0x394: 0xe00d, 0x395: 0x0008, 0x396: 0xe00d, 0x397: 0x0008,
+	0x398: 0xe00d, 0x399: 0x0008, 0x39a: 0xe00d, 0x39b: 0x0008, 0x39c: 0xe00d, 0x39d: 0x0008,
+	0x39e: 0xe00d, 0x39f: 0x0008, 0x3a0: 0xe00d, 0x3a1: 0x0008, 0x3a2: 0xe00d, 0x3a3: 0x0008,
+	0x3a4: 0xe00d, 0x3a5: 0x0008, 0x3a6: 0xe00d, 0x3a7: 0x0008, 0x3a8: 0xe00d, 0x3a9: 0x0008,
+	0x3aa: 0xe00d, 0x3ab: 0x0008, 0x3ac: 0xe00d, 0x3ad: 0x0008, 0x3ae: 0xe00d, 0x3af: 0x0008,
+	0x3b0: 0xe00d, 0x3b1: 0x0008, 0x3b2: 0xe00d, 0x3b3: 0x0008, 0x3b4: 0xe00d, 0x3b5: 0x0008,
+	0x3b6: 0xe00d, 0x3b7: 0x0008, 0x3b8: 0xe00d, 0x3b9: 0x0008, 0x3ba: 0xe00d, 0x3bb: 0x0008,
+	0x3bc: 0xe00d, 0x3bd: 0x0008, 0x3be: 0xe00d, 0x3bf: 0x0008,
+	// Block 0xf, offset 0x3c0
+	0x3c0: 0x0040, 0x3c1: 0xe01d, 0x3c2: 0x0008, 0x3c3: 0xe03d, 0x3c4: 0x0008, 0x3c5: 0xe01d,
+	0x3c6: 0x0008, 0x3c7: 0xe07d, 0x3c8: 0x0008, 0x3c9: 0xe01d, 0x3ca: 0x0008, 0x3cb: 0xe03d,
+	0x3cc: 0x0008, 0x3cd: 0xe01d, 0x3ce: 0x0008, 0x3cf: 0x0008, 0x3d0: 0xe00d, 0x3d1: 0x0008,
+	0x3d2: 0xe00d, 0x3d3: 0x0008, 0x3d4: 0xe00d, 0x3d5: 0x0008, 0x3d6: 0xe00d, 0x3d7: 0x0008,
+	0x3d8: 0xe00d, 0x3d9: 0x0008, 0x3da: 0xe00d, 0x3db: 0x0008, 0x3dc: 0xe00d, 0x3dd: 0x0008,
+	0x3de: 0xe00d, 0x3df: 0x0008, 0x3e0: 0xe00d, 0x3e1: 0x0008, 0x3e2: 0xe00d, 0x3e3: 0x0008,
+	0x3e4: 0xe00d, 0x3e5: 0x0008, 0x3e6: 0xe00d, 0x3e7: 0x0008, 0x3e8: 0xe00d, 0x3e9: 0x0008,
+	0x3ea: 0xe00d, 0x3eb: 0x0008, 0x3ec: 0xe00d, 0x3ed: 0x0008, 0x3ee: 0xe00d, 0x3ef: 0x0008,
+	0x3f0: 0xe00d, 0x3f1: 0x0008, 0x3f2: 0xe00d, 0x3f3: 0x0008, 0x3f4: 0xe00d, 0x3f5: 0x0008,
+	0x3f6: 0xe00d, 0x3f7: 0x0008, 0x3f8: 0xe00d, 0x3f9: 0x0008, 0x3fa: 0xe00d, 0x3fb: 0x0008,
+	0x3fc: 0xe00d, 0x3fd: 0x0008, 0x3fe: 0xe00d, 0x3ff: 0x0008,
+	// Block 0x10, offset 0x400
+	0x400: 0xe00d, 0x401: 0x0008, 0x402: 0xe00d, 0x403: 0x0008, 0x404: 0xe00d, 0x405: 0x0008,
+	0x406: 0xe00d, 0x407: 0x0008, 0x408: 0xe00d, 0x409: 0x0008, 0x40a: 0xe00d, 0x40b: 0x0008,
+	0x40c: 0xe00d, 0x40d: 0x0008, 0x40e: 0xe00d, 0x40f: 0x0008, 0x410: 0xe00d, 0x411: 0x0008,
+	0x412: 0xe00d, 0x413: 0x0008, 0x414: 0xe00d, 0x415: 0x0008, 0x416: 0xe00d, 0x417: 0x0008,
+	0x418: 0xe00d, 0x419: 0x0008, 0x41a: 0xe00d, 0x41b: 0x0008, 0x41c: 0xe00d, 0x41d: 0x0008,
+	0x41e: 0xe00d, 0x41f: 0x0008, 0x420: 0xe00d, 0x421: 0x0008, 0x422: 0xe00d, 0x423: 0x0008,
+	0x424: 0xe00d, 0x425: 0x0008, 0x426: 0xe00d, 0x427: 0x0008, 0x428: 0xe00d, 0x429: 0x0008,
+	0x42a: 0xe00d, 0x42b: 0x0008, 0x42c: 0xe00d, 0x42d: 0x0008, 0x42e: 0xe00d, 0x42f: 0x0008,
+	0x430: 0x0040, 0x431: 0x03f5, 0x432: 0x03f5, 0x433: 0x03f5, 0x434: 0x03f5, 0x435: 0x03f5,
+	0x436: 0x03f5, 0x437: 0x03f5, 0x438: 0x03f5, 0x439: 0x03f5, 0x43a: 0x03f5, 0x43b: 0x03f5,
+	0x43c: 0x03f5, 0x43d: 0x03f5, 0x43e: 0x03f5, 0x43f: 0x03f5,
+	// Block 0x11, offset 0x440
+	0x440: 0x0840, 0x441: 0x0840, 0x442: 0x0840, 0x443: 0x0840, 0x444: 0x0840, 0x445: 0x0840,
+	0x446: 0x0018, 0x447: 0x0018, 0x448: 0x0818, 0x449: 0x0018, 0x44a: 0x0018, 0x44b: 0x0818,
+	0x44c: 0x0018, 0x44d: 0x0818, 0x44e: 0x0018, 0x44f: 0x0018, 0x450: 0x3308, 0x451: 0x3308,
+	0x452: 0x3308, 0x453: 0x3308, 0x454: 0x3308, 0x455: 0x3308, 0x456: 0x3308, 0x457: 0x3308,
+	0x458: 0x3308, 0x459: 0x3308, 0x45a: 0x3308, 0x45b: 0x0818, 0x45c: 0x0b40, 0x45d: 0x0040,
+	0x45e: 0x0818, 0x45f: 0x0818, 0x460: 0x0a08, 0x461: 0x0808, 0x462: 0x0c08, 0x463: 0x0c08,
+	0x464: 0x0c08, 0x465: 0x0c08, 0x466: 0x0a08, 0x467: 0x0c08, 0x468: 0x0a08, 0x469: 0x0c08,
+	0x46a: 0x0a08, 0x46b: 0x0a08, 0x46c: 0x0a08, 0x46d: 0x0a08, 0x46e: 0x0a08, 0x46f: 0x0c08,
+	0x470: 0x0c08, 0x471: 0x0c08, 0x472: 0x0c08, 0x473: 0x0a08, 0x474: 0x0a08, 0x475: 0x0a08,
+	0x476: 0x0a08, 0x477: 0x0a08, 0x478: 0x0a08, 0x479: 0x0a08, 0x47a: 0x0a08, 0x47b: 0x0a08,
+	0x47c: 0x0a08, 0x47d: 0x0a08, 0x47e: 0x0a08, 0x47f: 0x0a08,
+	// Block 0x12, offset 0x480
+	0x480: 0x0818, 0x481: 0x0a08, 0x482: 0x0a08, 0x483: 0x0a08, 0x484: 0x0a08, 0x485: 0x0a08,
+	0x486: 0x0a08, 0x487: 0x0a08, 0x488: 0x0c08, 0x489: 0x0a08, 0x48a: 0x0a08, 0x48b: 0x3308,
+	0x48c: 0x3308, 0x48d: 0x3308, 0x48e: 0x3308, 0x48f: 0x3308, 0x490: 0x3308, 0x491: 0x3308,
+	0x492: 0x3308, 0x493: 0x3308, 0x494: 0x3308, 0x495: 0x3308, 0x496: 0x3308, 0x497: 0x3308,
+	0x498: 0x3308, 0x499: 0x3308, 0x49a: 0x3308, 0x49b: 0x3308, 0x49c: 0x3308, 0x49d: 0x3308,
+	0x49e: 0x3308, 0x49f: 0x3308, 0x4a0: 0x0808, 0x4a1: 0x0808, 0x4a2: 0x0808, 0x4a3: 0x0808,
+	0x4a4: 0x0808, 0x4a5: 0x0808, 0x4a6: 0x0808, 0x4a7: 0x0808, 0x4a8: 0x0808, 0x4a9: 0x0808,
+	0x4aa: 0x0018, 0x4ab: 0x0818, 0x4ac: 0x0818, 0x4ad: 0x0818, 0x4ae: 0x0a08, 0x4af: 0x0a08,
+	0x4b0: 0x3308, 0x4b1: 0x0c08, 0x4b2: 0x0c08, 0x4b3: 0x0c08, 0x4b4: 0x0808, 0x4b5: 0x0429,
+	0x4b6: 0x0451, 0x4b7: 0x0479, 0x4b8: 0x04a1, 0x4b9: 0x0a08, 0x4ba: 0x0a08, 0x4bb: 0x0a08,
+	0x4bc: 0x0a08, 0x4bd: 0x0a08, 0x4be: 0x0a08, 0x4bf: 0x0a08,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0x0c08, 0x4c1: 0x0a08, 0x4c2: 0x0a08, 0x4c3: 0x0c08, 0x4c4: 0x0c08, 0x4c5: 0x0c08,
+	0x4c6: 0x0c08, 0x4c7: 0x0c08, 0x4c8: 0x0c08, 0x4c9: 0x0c08, 0x4ca: 0x0c08, 0x4cb: 0x0c08,
+	0x4cc: 0x0a08, 0x4cd: 0x0c08, 0x4ce: 0x0a08, 0x4cf: 0x0c08, 0x4d0: 0x0a08, 0x4d1: 0x0a08,
+	0x4d2: 0x0c08, 0x4d3: 0x0c08, 0x4d4: 0x0818, 0x4d5: 0x0c08, 0x4d6: 0x3308, 0x4d7: 0x3308,
+	0x4d8: 0x3308, 0x4d9: 0x3308, 0x4da: 0x3308, 0x4db: 0x3308, 0x4dc: 0x3308, 0x4dd: 0x0840,
+	0x4de: 0x0018, 0x4df: 0x3308, 0x4e0: 0x3308, 0x4e1: 0x3308, 0x4e2: 0x3308, 0x4e3: 0x3308,
+	0x4e4: 0x3308, 0x4e5: 0x0808, 0x4e6: 0x0808, 0x4e7: 0x3308, 0x4e8: 0x3308, 0x4e9: 0x0018,
+	0x4ea: 0x3308, 0x4eb: 0x3308, 0x4ec: 0x3308, 0x4ed: 0x3308, 0x4ee: 0x0c08, 0x4ef: 0x0c08,
+	0x4f0: 0x0008, 0x4f1: 0x0008, 0x4f2: 0x0008, 0x4f3: 0x0008, 0x4f4: 0x0008, 0x4f5: 0x0008,
+	0x4f6: 0x0008, 0x4f7: 0x0008, 0x4f8: 0x0008, 0x4f9: 0x0008, 0x4fa: 0x0a08, 0x4fb: 0x0a08,
+	0x4fc: 0x0a08, 0x4fd: 0x0808, 0x4fe: 0x0808, 0x4ff: 0x0a08,
+	// Block 0x14, offset 0x500
+	0x500: 0x0818, 0x501: 0x0818, 0x502: 0x0818, 0x503: 0x0818, 0x504: 0x0818, 0x505: 0x0818,
+	0x506: 0x0818, 0x507: 0x0818, 0x508: 0x0818, 0x509: 0x0818, 0x50a: 0x0818, 0x50b: 0x0818,
+	0x50c: 0x0818, 0x50d: 0x0818, 0x50e: 0x0040, 0x50f: 0x0b40, 0x510: 0x0c08, 0x511: 0x3308,
+	0x512: 0x0a08, 0x513: 0x0a08, 0x514: 0x0a08, 0x515: 0x0c08, 0x516: 0x0c08, 0x517: 0x0c08,
+	0x518: 0x0c08, 0x519: 0x0c08, 0x51a: 0x0a08, 0x51b: 0x0a08, 0x51c: 0x0a08, 0x51d: 0x0a08,
+	0x51e: 0x0c08, 0x51f: 0x0a08, 0x520: 0x0a08, 0x521: 0x0a08, 0x522: 0x0a08, 0x523: 0x0a08,
+	0x524: 0x0a08, 0x525: 0x0a08, 0x526: 0x0a08, 0x527: 0x0a08, 0x528: 0x0c08, 0x529: 0x0a08,
+	0x52a: 0x0c08, 0x52b: 0x0a08, 0x52c: 0x0c08, 0x52d: 0x0a08, 0x52e: 0x0a08, 0x52f: 0x0c08,
+	0x530: 0x3308, 0x531: 0x3308, 0x532: 0x3308, 0x533: 0x3308, 0x534: 0x3308, 0x535: 0x3308,
+	0x536: 0x3308, 0x537: 0x3308, 0x538: 0x3308, 0x539: 0x3308, 0x53a: 0x3308, 0x53b: 0x3308,
+	0x53c: 0x3308, 0x53d: 0x3308, 0x53e: 0x3308, 0x53f: 0x3308,
+	// Block 0x15, offset 0x540
+	0x540: 0x0c08, 0x541: 0x0a08, 0x542: 0x0a08, 0x543: 0x0a08, 0x544: 0x0a08, 0x545: 0x0a08,
+	0x546: 0x0c08, 0x547: 0x0c08, 0x548: 0x0a08, 0x549: 0x0c08, 0x54a: 0x0a08, 0x54b: 0x0a08,
+	0x54c: 0x0a08, 0x54d: 0x0a08, 0x54e: 0x0a08, 0x54f: 0x0a08, 0x550: 0x0a08, 0x551: 0x0a08,
+	0x552: 0x0a08, 0x553: 0x0a08, 0x554: 0x0c08, 0x555: 0x0a08, 0x556: 0x0808, 0x557: 0x0808,
+	0x558: 0x0808, 0x559: 0x3308, 0x55a: 0x3308, 0x55b: 0x3308, 0x55c: 0x0040, 0x55d: 0x0040,
+	0x55e: 0x0818, 0x55f: 0x0040, 0x560: 0x0a08, 0x561: 0x0808, 0x562: 0x0a08, 0x563: 0x0a08,
+	0x564: 0x0a08, 0x565: 0x0a08, 0x566: 0x0808, 0x567: 0x0c08, 0x568: 0x0a08, 0x569: 0x0c08,
+	0x56a: 0x0c08, 0x56b: 0x0040, 0x56c: 0x0040, 0x56d: 0x0040, 0x56e: 0x0040, 0x56f: 0x0040,
+	0x570: 0x0040, 0x571: 0x0040, 0x572: 0x0040, 0x573: 0x0040, 0x574: 0x0040, 0x575: 0x0040,
+	0x576: 0x0040, 0x577: 0x0040, 0x578: 0x0040, 0x579: 0x0040, 0x57a: 0x0040, 0x57b: 0x0040,
+	0x57c: 0x0040, 0x57d: 0x0040, 0x57e: 0x0040, 0x57f: 0x0040,
+	// Block 0x16, offset 0x580
+	0x580: 0x3008, 0x581: 0x3308, 0x582: 0x3308, 0x583: 0x3308, 0x584: 0x3308, 0x585: 0x3308,
+	0x586: 0x3308, 0x587: 0x3308, 0x588: 0x3308, 0x589: 0x3008, 0x58a: 0x3008, 0x58b: 0x3008,
+	0x58c: 0x3008, 0x58d: 0x3b08, 0x58e: 0x3008, 0x58f: 0x3008, 0x590: 0x0008, 0x591: 0x3308,
+	0x592: 0x3308, 0x593: 0x3308, 0x594: 0x3308, 0x595: 0x3308, 0x596: 0x3308, 0x597: 0x3308,
+	0x598: 0x04c9, 0x599: 0x0501, 0x59a: 0x0539, 0x59b: 0x0571, 0x59c: 0x05a9, 0x59d: 0x05e1,
+	0x59e: 0x0619, 0x59f: 0x0651, 0x5a0: 0x0008, 0x5a1: 0x0008, 0x5a2: 0x3308, 0x5a3: 0x3308,
+	0x5a4: 0x0018, 0x5a5: 0x0018, 0x5a6: 0x0008, 0x5a7: 0x0008, 0x5a8: 0x0008, 0x5a9: 0x0008,
+	0x5aa: 0x0008, 0x5ab: 0x0008, 0x5ac: 0x0008, 0x5ad: 0x0008, 0x5ae: 0x0008, 0x5af: 0x0008,
+	0x5b0: 0x0018, 0x5b1: 0x0008, 0x5b2: 0x0008, 0x5b3: 0x0008, 0x5b4: 0x0008, 0x5b5: 0x0008,
+	0x5b6: 0x0008, 0x5b7: 0x0008, 0x5b8: 0x0008, 0x5b9: 0x0008, 0x5ba: 0x0008, 0x5bb: 0x0008,
+	0x5bc: 0x0008, 0x5bd: 0x0008, 0x5be: 0x0008, 0x5bf: 0x0008,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x0008, 0x5c1: 0x3308, 0x5c2: 0x3008, 0x5c3: 0x3008, 0x5c4: 0x0040, 0x5c5: 0x0008,
+	0x5c6: 0x0008, 0x5c7: 0x0008, 0x5c8: 0x0008, 0x5c9: 0x0008, 0x5ca: 0x0008, 0x5cb: 0x0008,
+	0x5cc: 0x0008, 0x5cd: 0x0040, 0x5ce: 0x0040, 0x5cf: 0x0008, 0x5d0: 0x0008, 0x5d1: 0x0040,
+	0x5d2: 0x0040, 0x5d3: 0x0008, 0x5d4: 0x0008, 0x5d5: 0x0008, 0x5d6: 0x0008, 0x5d7: 0x0008,
+	0x5d8: 0x0008, 0x5d9: 0x0008, 0x5da: 0x0008, 0x5db: 0x0008, 0x5dc: 0x0008, 0x5dd: 0x0008,
+	0x5de: 0x0008, 0x5df: 0x0008, 0x5e0: 0x0008, 0x5e1: 0x0008, 0x5e2: 0x0008, 0x5e3: 0x0008,
+	0x5e4: 0x0008, 0x5e5: 0x0008, 0x5e6: 0x0008, 0x5e7: 0x0008, 0x5e8: 0x0008, 0x5e9: 0x0040,
+	0x5ea: 0x0008, 0x5eb: 0x0008, 0x5ec: 0x0008, 0x5ed: 0x0008, 0x5ee: 0x0008, 0x5ef: 0x0008,
+	0x5f0: 0x0008, 0x5f1: 0x0040, 0x5f2: 0x0008, 0x5f3: 0x0040, 0x5f4: 0x0040, 0x5f5: 0x0040,
+	0x5f6: 0x0008, 0x5f7: 0x0008, 0x5f8: 0x0008, 0x5f9: 0x0008, 0x5fa: 0x0040, 0x5fb: 0x0040,
+	0x5fc: 0x3308, 0x5fd: 0x0008, 0x5fe: 0x3008, 0x5ff: 0x3008,
+	// Block 0x18, offset 0x600
+	0x600: 0x3008, 0x601: 0x3308, 0x602: 0x3308, 0x603: 0x3308, 0x604: 0x3308, 0x605: 0x0040,
+	0x606: 0x0040, 0x607: 0x3008, 0x608: 0x3008, 0x609: 0x0040, 0x60a: 0x0040, 0x60b: 0x3008,
+	0x60c: 0x3008, 0x60d: 0x3b08, 0x60e: 0x0008, 0x60f: 0x0040, 0x610: 0x0040, 0x611: 0x0040,
+	0x612: 0x0040, 0x613: 0x0040, 0x614: 0x0040, 0x615: 0x0040, 0x616: 0x0040, 0x617: 0x3008,
+	0x618: 0x0040, 0x619: 0x0040, 0x61a: 0x0040, 0x61b: 0x0040, 0x61c: 0x0689, 0x61d: 0x06c1,
+	0x61e: 0x0040, 0x61f: 0x06f9, 0x620: 0x0008, 0x621: 0x0008, 0x622: 0x3308, 0x623: 0x3308,
+	0x624: 0x0040, 0x625: 0x0040, 0x626: 0x0008, 0x627: 0x0008, 0x628: 0x0008, 0x629: 0x0008,
+	0x62a: 0x0008, 0x62b: 0x0008, 0x62c: 0x0008, 0x62d: 0x0008, 0x62e: 0x0008, 0x62f: 0x0008,
+	0x630: 0x0008, 0x631: 0x0008, 0x632: 0x0018, 0x633: 0x0018, 0x634: 0x0018, 0x635: 0x0018,
+	0x636: 0x0018, 0x637: 0x0018, 0x638: 0x0018, 0x639: 0x0018, 0x63a: 0x0018, 0x63b: 0x0018,
+	0x63c: 0x0008, 0x63d: 0x0018, 0x63e: 0x3308, 0x63f: 0x0040,
+	// Block 0x19, offset 0x640
+	0x640: 0x0040, 0x641: 0x3308, 0x642: 0x3308, 0x643: 0x3008, 0x644: 0x0040, 0x645: 0x0008,
+	0x646: 0x0008, 0x647: 0x0008, 0x648: 0x0008, 0x649: 0x0008, 0x64a: 0x0008, 0x64b: 0x0040,
+	0x64c: 0x0040, 0x64d: 0x0040, 0x64e: 0x0040, 0x64f: 0x0008, 0x650: 0x0008, 0x651: 0x0040,
+	0x652: 0x0040, 0x653: 0x0008, 0x654: 0x0008, 0x655: 0x0008, 0x656: 0x0008, 0x657: 0x0008,
+	0x658: 0x0008, 0x659: 0x0008, 0x65a: 0x0008, 0x65b: 0x0008, 0x65c: 0x0008, 0x65d: 0x0008,
+	0x65e: 0x0008, 0x65f: 0x0008, 0x660: 0x0008, 0x661: 0x0008, 0x662: 0x0008, 0x663: 0x0008,
+	0x664: 0x0008, 0x665: 0x0008, 0x666: 0x0008, 0x667: 0x0008, 0x668: 0x0008, 0x669: 0x0040,
+	0x66a: 0x0008, 0x66b: 0x0008, 0x66c: 0x0008, 0x66d: 0x0008, 0x66e: 0x0008, 0x66f: 0x0008,
+	0x670: 0x0008, 0x671: 0x0040, 0x672: 0x0008, 0x673: 0x0731, 0x674: 0x0040, 0x675: 0x0008,
+	0x676: 0x0769, 0x677: 0x0040, 0x678: 0x0008, 0x679: 0x0008, 0x67a: 0x0040, 0x67b: 0x0040,
+	0x67c: 0x3308, 0x67d: 0x0040, 0x67e: 0x3008, 0x67f: 0x3008,
+	// Block 0x1a, offset 0x680
+	0x680: 0x3008, 0x681: 0x3308, 0x682: 0x3308, 0x683: 0x0040, 0x684: 0x0040, 0x685: 0x0040,
+	0x686: 0x0040, 0x687: 0x3308, 0x688: 0x3308, 0x689: 0x0040, 0x68a: 0x0040, 0x68b: 0x3308,
+	0x68c: 0x3308, 0x68d: 0x3b08, 0x68e: 0x0040, 0x68f: 0x0040, 0x690: 0x0040, 0x691: 0x3308,
+	0x692: 0x0040, 0x693: 0x0040, 0x694: 0x0040, 0x695: 0x0040, 0x696: 0x0040, 0x697: 0x0040,
+	0x698: 0x0040, 0x699: 0x07a1, 0x69a: 0x07d9, 0x69b: 0x0811, 0x69c: 0x0008, 0x69d: 0x0040,
+	0x69e: 0x0849, 0x69f: 0x0040, 0x6a0: 0x0040, 0x6a1: 0x0040, 0x6a2: 0x0040, 0x6a3: 0x0040,
+	0x6a4: 0x0040, 0x6a5: 0x0040, 0x6a6: 0x0008, 0x6a7: 0x0008, 0x6a8: 0x0008, 0x6a9: 0x0008,
+	0x6aa: 0x0008, 0x6ab: 0x0008, 0x6ac: 0x0008, 0x6ad: 0x0008, 0x6ae: 0x0008, 0x6af: 0x0008,
+	0x6b0: 0x3308, 0x6b1: 0x3308, 0x6b2: 0x0008, 0x6b3: 0x0008, 0x6b4: 0x0008, 0x6b5: 0x3308,
+	0x6b6: 0x0018, 0x6b7: 0x0040, 0x6b8: 0x0040, 0x6b9: 0x0040, 0x6ba: 0x0040, 0x6bb: 0x0040,
+	0x6bc: 0x0040, 0x6bd: 0x0040, 0x6be: 0x0040, 0x6bf: 0x0040,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0x0040, 0x6c1: 0x3308, 0x6c2: 0x3308, 0x6c3: 0x3008, 0x6c4: 0x0040, 0x6c5: 0x0008,
+	0x6c6: 0x0008, 0x6c7: 0x0008, 0x6c8: 0x0008, 0x6c9: 0x0008, 0x6ca: 0x0008, 0x6cb: 0x0008,
+	0x6cc: 0x0008, 0x6cd: 0x0008, 0x6ce: 0x0040, 0x6cf: 0x0008, 0x6d0: 0x0008, 0x6d1: 0x0008,
+	0x6d2: 0x0040, 0x6d3: 0x0008, 0x6d4: 0x0008, 0x6d5: 0x0008, 0x6d6: 0x0008, 0x6d7: 0x0008,
+	0x6d8: 0x0008, 0x6d9: 0x0008, 0x6da: 0x0008, 0x6db: 0x0008, 0x6dc: 0x0008, 0x6dd: 0x0008,
+	0x6de: 0x0008, 0x6df: 0x0008, 0x6e0: 0x0008, 0x6e1: 0x0008, 0x6e2: 0x0008, 0x6e3: 0x0008,
+	0x6e4: 0x0008, 0x6e5: 0x0008, 0x6e6: 0x0008, 0x6e7: 0x0008, 0x6e8: 0x0008, 0x6e9: 0x0040,
+	0x6ea: 0x0008, 0x6eb: 0x0008, 0x6ec: 0x0008, 0x6ed: 0x0008, 0x6ee: 0x0008, 0x6ef: 0x0008,
+	0x6f0: 0x0008, 0x6f1: 0x0040, 0x6f2: 0x0008, 0x6f3: 0x0008, 0x6f4: 0x0040, 0x6f5: 0x0008,
+	0x6f6: 0x0008, 0x6f7: 0x0008, 0x6f8: 0x0008, 0x6f9: 0x0008, 0x6fa: 0x0040, 0x6fb: 0x0040,
+	0x6fc: 0x3308, 0x6fd: 0x0008, 0x6fe: 0x3008, 0x6ff: 0x3008,
+	// Block 0x1c, offset 0x700
+	0x700: 0x3008, 0x701: 0x3308, 0x702: 0x3308, 0x703: 0x3308, 0x704: 0x3308, 0x705: 0x3308,
+	0x706: 0x0040, 0x707: 0x3308, 0x708: 0x3308, 0x709: 0x3008, 0x70a: 0x0040, 0x70b: 0x3008,
+	0x70c: 0x3008, 0x70d: 0x3b08, 0x70e: 0x0040, 0x70f: 0x0040, 0x710: 0x0008, 0x711: 0x0040,
+	0x712: 0x0040, 0x713: 0x0040, 0x714: 0x0040, 0x715: 0x0040, 0x716: 0x0040, 0x717: 0x0040,
+	0x718: 0x0040, 0x719: 0x0040, 0x71a: 0x0040, 0x71b: 0x0040, 0x71c: 0x0040, 0x71d: 0x0040,
+	0x71e: 0x0040, 0x71f: 0x0040, 0x720: 0x0008, 0x721: 0x0008, 0x722: 0x3308, 0x723: 0x3308,
+	0x724: 0x0040, 0x725: 0x0040, 0x726: 0x0008, 0x727: 0x0008, 0x728: 0x0008, 0x729: 0x0008,
+	0x72a: 0x0008, 0x72b: 0x0008, 0x72c: 0x0008, 0x72d: 0x0008, 0x72e: 0x0008, 0x72f: 0x0008,
+	0x730: 0x0018, 0x731: 0x0018, 0x732: 0x0040, 0x733: 0x0040, 0x734: 0x0040, 0x735: 0x0040,
+	0x736: 0x0040, 0x737: 0x0040, 0x738: 0x0040, 0x739: 0x0008, 0x73a: 0x3308, 0x73b: 0x3308,
+	0x73c: 0x3308, 0x73d: 0x3308, 0x73e: 0x3308, 0x73f: 0x3308,
+	// Block 0x1d, offset 0x740
+	0x740: 0x0040, 0x741: 0x3308, 0x742: 0x3008, 0x743: 0x3008, 0x744: 0x0040, 0x745: 0x0008,
+	0x746: 0x0008, 0x747: 0x0008, 0x748: 0x0008, 0x749: 0x0008, 0x74a: 0x0008, 0x74b: 0x0008,
+	0x74c: 0x0008, 0x74d: 0x0040, 0x74e: 0x0040, 0x74f: 0x0008, 0x750: 0x0008, 0x751: 0x0040,
+	0x752: 0x0040, 0x753: 0x0008, 0x754: 0x0008, 0x755: 0x0008, 0x756: 0x0008, 0x757: 0x0008,
+	0x758: 0x0008, 0x759: 0x0008, 0x75a: 0x0008, 0x75b: 0x0008, 0x75c: 0x0008, 0x75d: 0x0008,
+	0x75e: 0x0008, 0x75f: 0x0008, 0x760: 0x0008, 0x761: 0x0008, 0x762: 0x0008, 0x763: 0x0008,
+	0x764: 0x0008, 0x765: 0x0008, 0x766: 0x0008, 0x767: 0x0008, 0x768: 0x0008, 0x769: 0x0040,
+	0x76a: 0x0008, 0x76b: 0x0008, 0x76c: 0x0008, 0x76d: 0x0008, 0x76e: 0x0008, 0x76f: 0x0008,
+	0x770: 0x0008, 0x771: 0x0040, 0x772: 0x0008, 0x773: 0x0008, 0x774: 0x0040, 0x775: 0x0008,
+	0x776: 0x0008, 0x777: 0x0008, 0x778: 0x0008, 0x779: 0x0008, 0x77a: 0x0040, 0x77b: 0x0040,
+	0x77c: 0x3308, 0x77d: 0x0008, 0x77e: 0x3008, 0x77f: 0x3308,
+	// Block 0x1e, offset 0x780
+	0x780: 0x3008, 0x781: 0x3308, 0x782: 0x3308, 0x783: 0x3308, 0x784: 0x3308, 0x785: 0x0040,
+	0x786: 0x0040, 0x787: 0x3008, 0x788: 0x3008, 0x789: 0x0040, 0x78a: 0x0040, 0x78b: 0x3008,
+	0x78c: 0x3008, 0x78d: 0x3b08, 0x78e: 0x0040, 0x78f: 0x0040, 0x790: 0x0040, 0x791: 0x0040,
+	0x792: 0x0040, 0x793: 0x0040, 0x794: 0x0040, 0x795: 0x0040, 0x796: 0x3308, 0x797: 0x3008,
+	0x798: 0x0040, 0x799: 0x0040, 0x79a: 0x0040, 0x79b: 0x0040, 0x79c: 0x0881, 0x79d: 0x08b9,
+	0x79e: 0x0040, 0x79f: 0x0008, 0x7a0: 0x0008, 0x7a1: 0x0008, 0x7a2: 0x3308, 0x7a3: 0x3308,
+	0x7a4: 0x0040, 0x7a5: 0x0040, 0x7a6: 0x0008, 0x7a7: 0x0008, 0x7a8: 0x0008, 0x7a9: 0x0008,
+	0x7aa: 0x0008, 0x7ab: 0x0008, 0x7ac: 0x0008, 0x7ad: 0x0008, 0x7ae: 0x0008, 0x7af: 0x0008,
+	0x7b0: 0x0018, 0x7b1: 0x0008, 0x7b2: 0x0018, 0x7b3: 0x0018, 0x7b4: 0x0018, 0x7b5: 0x0018,
+	0x7b6: 0x0018, 0x7b7: 0x0018, 0x7b8: 0x0040, 0x7b9: 0x0040, 0x7ba: 0x0040, 0x7bb: 0x0040,
+	0x7bc: 0x0040, 0x7bd: 0x0040, 0x7be: 0x0040, 0x7bf: 0x0040,
+	// Block 0x1f, offset 0x7c0
+	0x7c0: 0x0040, 0x7c1: 0x0040, 0x7c2: 0x3308, 0x7c3: 0x0008, 0x7c4: 0x0040, 0x7c5: 0x0008,
+	0x7c6: 0x0008, 0x7c7: 0x0008, 0x7c8: 0x0008, 0x7c9: 0x0008, 0x7ca: 0x0008, 0x7cb: 0x0040,
+	0x7cc: 0x0040, 0x7cd: 0x0040, 0x7ce: 0x0008, 0x7cf: 0x0008, 0x7d0: 0x0008, 0x7d1: 0x0040,
+	0x7d2: 0x0008, 0x7d3: 0x0008, 0x7d4: 0x0008, 0x7d5: 0x0008, 0x7d6: 0x0040, 0x7d7: 0x0040,
+	0x7d8: 0x0040, 0x7d9: 0x0008, 0x7da: 0x0008, 0x7db: 0x0040, 0x7dc: 0x0008, 0x7dd: 0x0040,
+	0x7de: 0x0008, 0x7df: 0x0008, 0x7e0: 0x0040, 0x7e1: 0x0040, 0x7e2: 0x0040, 0x7e3: 0x0008,
+	0x7e4: 0x0008, 0x7e5: 0x0040, 0x7e6: 0x0040, 0x7e7: 0x0040, 0x7e8: 0x0008, 0x7e9: 0x0008,
+	0x7ea: 0x0008, 0x7eb: 0x0040, 0x7ec: 0x0040, 0x7ed: 0x0040, 0x7ee: 0x0008, 0x7ef: 0x0008,
+	0x7f0: 0x0008, 0x7f1: 0x0008, 0x7f2: 0x0008, 0x7f3: 0x0008, 0x7f4: 0x0008, 0x7f5: 0x0008,
+	0x7f6: 0x0008, 0x7f7: 0x0008, 0x7f8: 0x0008, 0x7f9: 0x0008, 0x7fa: 0x0040, 0x7fb: 0x0040,
+	0x7fc: 0x0040, 0x7fd: 0x0040, 0x7fe: 0x3008, 0x7ff: 0x3008,
+	// Block 0x20, offset 0x800
+	0x800: 0x3308, 0x801: 0x3008, 0x802: 0x3008, 0x803: 0x3008, 0x804: 0x3008, 0x805: 0x0040,
+	0x806: 0x3308, 0x807: 0x3308, 0x808: 0x3308, 0x809: 0x0040, 0x80a: 0x3308, 0x80b: 0x3308,
+	0x80c: 0x3308, 0x80d: 0x3b08, 0x80e: 0x0040, 0x80f: 0x0040, 0x810: 0x0040, 0x811: 0x0040,
+	0x812: 0x0040, 0x813: 0x0040, 0x814: 0x0040, 0x815: 0x3308, 0x816: 0x3308, 0x817: 0x0040,
+	0x818: 0x0008, 0x819: 0x0008, 0x81a: 0x0008, 0x81b: 0x0040, 0x81c: 0x0040, 0x81d: 0x0040,
+	0x81e: 0x0040, 0x81f: 0x0040, 0x820: 0x0008, 0x821: 0x0008, 0x822: 0x3308, 0x823: 0x3308,
+	0x824: 0x0040, 0x825: 0x0040, 0x826: 0x0008, 0x827: 0x0008, 0x828: 0x0008, 0x829: 0x0008,
+	0x82a: 0x0008, 0x82b: 0x0008, 0x82c: 0x0008, 0x82d: 0x0008, 0x82e: 0x0008, 0x82f: 0x0008,
+	0x830: 0x0040, 0x831: 0x0040, 0x832: 0x0040, 0x833: 0x0040, 0x834: 0x0040, 0x835: 0x0040,
+	0x836: 0x0040, 0x837: 0x0018, 0x838: 0x0018, 0x839: 0x0018, 0x83a: 0x0018, 0x83b: 0x0018,
+	0x83c: 0x0018, 0x83d: 0x0018, 0x83e: 0x0018, 0x83f: 0x0018,
+	// Block 0x21, offset 0x840
+	0x840: 0x0008, 0x841: 0x3308, 0x842: 0x3008, 0x843: 0x3008, 0x844: 0x0018, 0x845: 0x0008,
+	0x846: 0x0008, 0x847: 0x0008, 0x848: 0x0008, 0x849: 0x0008, 0x84a: 0x0008, 0x84b: 0x0008,
+	0x84c: 0x0008, 0x84d: 0x0040, 0x84e: 0x0008, 0x84f: 0x0008, 0x850: 0x0008, 0x851: 0x0040,
+	0x852: 0x0008, 0x853: 0x0008, 0x854: 0x0008, 0x855: 0x0008, 0x856: 0x0008, 0x857: 0x0008,
+	0x858: 0x0008, 0x859: 0x0008, 0x85a: 0x0008, 0x85b: 0x0008, 0x85c: 0x0008, 0x85d: 0x0008,
+	0x85e: 0x0008, 0x85f: 0x0008, 0x860: 0x0008, 0x861: 0x0008, 0x862: 0x0008, 0x863: 0x0008,
+	0x864: 0x0008, 0x865: 0x0008, 0x866: 0x0008, 0x867: 0x0008, 0x868: 0x0008, 0x869: 0x0040,
+	0x86a: 0x0008, 0x86b: 0x0008, 0x86c: 0x0008, 0x86d: 0x0008, 0x86e: 0x0008, 0x86f: 0x0008,
+	0x870: 0x0008, 0x871: 0x0008, 0x872: 0x0008, 0x873: 0x0008, 0x874: 0x0040, 0x875: 0x0008,
+	0x876: 0x0008, 0x877: 0x0008, 0x878: 0x0008, 0x879: 0x0008, 0x87a: 0x0040, 0x87b: 0x0040,
+	0x87c: 0x3308, 0x87d: 0x0008, 0x87e: 0x3008, 0x87f: 0x3308,
+	// Block 0x22, offset 0x880
+	0x880: 0x3008, 0x881: 0x3008, 0x882: 0x3008, 0x883: 0x3008, 0x884: 0x3008, 0x885: 0x0040,
+	0x886: 0x3308, 0x887: 0x3008, 0x888: 0x3008, 0x889: 0x0040, 0x88a: 0x3008, 0x88b: 0x3008,
+	0x88c: 0x3308, 0x88d: 0x3b08, 0x88e: 0x0040, 0x88f: 0x0040, 0x890: 0x0040, 0x891: 0x0040,
+	0x892: 0x0040, 0x893: 0x0040, 0x894: 0x0040, 0x895: 0x3008, 0x896: 0x3008, 0x897: 0x0040,
+	0x898: 0x0040, 0x899: 0x0040, 0x89a: 0x0040, 0x89b: 0x0040, 0x89c: 0x0040, 0x89d: 0x0040,
+	0x89e: 0x0008, 0x89f: 0x0040, 0x8a0: 0x0008, 0x8a1: 0x0008, 0x8a2: 0x3308, 0x8a3: 0x3308,
+	0x8a4: 0x0040, 0x8a5: 0x0040, 0x8a6: 0x0008, 0x8a7: 0x0008, 0x8a8: 0x0008, 0x8a9: 0x0008,
+	0x8aa: 0x0008, 0x8ab: 0x0008, 0x8ac: 0x0008, 0x8ad: 0x0008, 0x8ae: 0x0008, 0x8af: 0x0008,
+	0x8b0: 0x0040, 0x8b1: 0x0008, 0x8b2: 0x0008, 0x8b3: 0x0040, 0x8b4: 0x0040, 0x8b5: 0x0040,
+	0x8b6: 0x0040, 0x8b7: 0x0040, 0x8b8: 0x0040, 0x8b9: 0x0040, 0x8ba: 0x0040, 0x8bb: 0x0040,
+	0x8bc: 0x0040, 0x8bd: 0x0040, 0x8be: 0x0040, 0x8bf: 0x0040,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x3008, 0x8c1: 0x3308, 0x8c2: 0x3308, 0x8c3: 0x3308, 0x8c4: 0x3308, 0x8c5: 0x0040,
+	0x8c6: 0x3008, 0x8c7: 0x3008, 0x8c8: 0x3008, 0x8c9: 0x0040, 0x8ca: 0x3008, 0x8cb: 0x3008,
+	0x8cc: 0x3008, 0x8cd: 0x3b08, 0x8ce: 0x0008, 0x8cf: 0x0018, 0x8d0: 0x0040, 0x8d1: 0x0040,
+	0x8d2: 0x0040, 0x8d3: 0x0040, 0x8d4: 0x0008, 0x8d5: 0x0008, 0x8d6: 0x0008, 0x8d7: 0x3008,
+	0x8d8: 0x0018, 0x8d9: 0x0018, 0x8da: 0x0018, 0x8db: 0x0018, 0x8dc: 0x0018, 0x8dd: 0x0018,
+	0x8de: 0x0018, 0x8df: 0x0008, 0x8e0: 0x0008, 0x8e1: 0x0008, 0x8e2: 0x3308, 0x8e3: 0x3308,
+	0x8e4: 0x0040, 0x8e5: 0x0040, 0x8e6: 0x0008, 0x8e7: 0x0008, 0x8e8: 0x0008, 0x8e9: 0x0008,
+	0x8ea: 0x0008, 0x8eb: 0x0008, 0x8ec: 0x0008, 0x8ed: 0x0008, 0x8ee: 0x0008, 0x8ef: 0x0008,
+	0x8f0: 0x0018, 0x8f1: 0x0018, 0x8f2: 0x0018, 0x8f3: 0x0018, 0x8f4: 0x0018, 0x8f5: 0x0018,
+	0x8f6: 0x0018, 0x8f7: 0x0018, 0x8f8: 0x0018, 0x8f9: 0x0018, 0x8fa: 0x0008, 0x8fb: 0x0008,
+	0x8fc: 0x0008, 0x8fd: 0x0008, 0x8fe: 0x0008, 0x8ff: 0x0008,
+	// Block 0x24, offset 0x900
+	0x900: 0x0040, 0x901: 0x0008, 0x902: 0x0008, 0x903: 0x0040, 0x904: 0x0008, 0x905: 0x0040,
+	0x906: 0x0008, 0x907: 0x0008, 0x908: 0x0008, 0x909: 0x0008, 0x90a: 0x0008, 0x90b: 0x0040,
+	0x90c: 0x0008, 0x90d: 0x0008, 0x90e: 0x0008, 0x90f: 0x0008, 0x910: 0x0008, 0x911: 0x0008,
+	0x912: 0x0008, 0x913: 0x0008, 0x914: 0x0008, 0x915: 0x0008, 0x916: 0x0008, 0x917: 0x0008,
+	0x918: 0x0008, 0x919: 0x0008, 0x91a: 0x0008, 0x91b: 0x0008, 0x91c: 0x0008, 0x91d: 0x0008,
+	0x91e: 0x0008, 0x91f: 0x0008, 0x920: 0x0008, 0x921: 0x0008, 0x922: 0x0008, 0x923: 0x0008,
+	0x924: 0x0040, 0x925: 0x0008, 0x926: 0x0040, 0x927: 0x0008, 0x928: 0x0008, 0x929: 0x0008,
+	0x92a: 0x0008, 0x92b: 0x0008, 0x92c: 0x0008, 0x92d: 0x0008, 0x92e: 0x0008, 0x92f: 0x0008,
+	0x930: 0x0008, 0x931: 0x3308, 0x932: 0x0008, 0x933: 0x0929, 0x934: 0x3308, 0x935: 0x3308,
+	0x936: 0x3308, 0x937: 0x3308, 0x938: 0x3308, 0x939: 0x3308, 0x93a: 0x3b08, 0x93b: 0x3308,
+	0x93c: 0x3308, 0x93d: 0x0008, 0x93e: 0x0040, 0x93f: 0x0040,
+	// Block 0x25, offset 0x940
+	0x940: 0x0008, 0x941: 0x0008, 0x942: 0x0008, 0x943: 0x09d1, 0x944: 0x0008, 0x945: 0x0008,
+	0x946: 0x0008, 0x947: 0x0008, 0x948: 0x0040, 0x949: 0x0008, 0x94a: 0x0008, 0x94b: 0x0008,
+	0x94c: 0x0008, 0x94d: 0x0a09, 0x94e: 0x0008, 0x94f: 0x0008, 0x950: 0x0008, 0x951: 0x0008,
+	0x952: 0x0a41, 0x953: 0x0008, 0x954: 0x0008, 0x955: 0x0008, 0x956: 0x0008, 0x957: 0x0a79,
+	0x958: 0x0008, 0x959: 0x0008, 0x95a: 0x0008, 0x95b: 0x0008, 0x95c: 0x0ab1, 0x95d: 0x0008,
+	0x95e: 0x0008, 0x95f: 0x0008, 0x960: 0x0008, 0x961: 0x0008, 0x962: 0x0008, 0x963: 0x0008,
+	0x964: 0x0008, 0x965: 0x0008, 0x966: 0x0008, 0x967: 0x0008, 0x968: 0x0008, 0x969: 0x0ae9,
+	0x96a: 0x0008, 0x96b: 0x0008, 0x96c: 0x0008, 0x96d: 0x0040, 0x96e: 0x0040, 0x96f: 0x0040,
+	0x970: 0x0040, 0x971: 0x3308, 0x972: 0x3308, 0x973: 0x0b21, 0x974: 0x3308, 0x975: 0x0b59,
+	0x976: 0x0b91, 0x977: 0x0bc9, 0x978: 0x0c19, 0x979: 0x0c51, 0x97a: 0x3308, 0x97b: 0x3308,
+	0x97c: 0x3308, 0x97d: 0x3308, 0x97e: 0x3308, 0x97f: 0x3008,
+	// Block 0x26, offset 0x980
+	0x980: 0x3308, 0x981: 0x0ca1, 0x982: 0x3308, 0x983: 0x3308, 0x984: 0x3b08, 0x985: 0x0018,
+	0x986: 0x3308, 0x987: 0x3308, 0x988: 0x0008, 0x989: 0x0008, 0x98a: 0x0008, 0x98b: 0x0008,
+	0x98c: 0x0008, 0x98d: 0x3308, 0x98e: 0x3308, 0x98f: 0x3308, 0x990: 0x3308, 0x991: 0x3308,
+	0x992: 0x3308, 0x993: 0x0cd9, 0x994: 0x3308, 0x995: 0x3308, 0x996: 0x3308, 0x997: 0x3308,
+	0x998: 0x0040, 0x999: 0x3308, 0x99a: 0x3308, 0x99b: 0x3308, 0x99c: 0x3308, 0x99d: 0x0d11,
+	0x99e: 0x3308, 0x99f: 0x3308, 0x9a0: 0x3308, 0x9a1: 0x3308, 0x9a2: 0x0d49, 0x9a3: 0x3308,
+	0x9a4: 0x3308, 0x9a5: 0x3308, 0x9a6: 0x3308, 0x9a7: 0x0d81, 0x9a8: 0x3308, 0x9a9: 0x3308,
+	0x9aa: 0x3308, 0x9ab: 0x3308, 0x9ac: 0x0db9, 0x9ad: 0x3308, 0x9ae: 0x3308, 0x9af: 0x3308,
+	0x9b0: 0x3308, 0x9b1: 0x3308, 0x9b2: 0x3308, 0x9b3: 0x3308, 0x9b4: 0x3308, 0x9b5: 0x3308,
+	0x9b6: 0x3308, 0x9b7: 0x3308, 0x9b8: 0x3308, 0x9b9: 0x0df1, 0x9ba: 0x3308, 0x9bb: 0x3308,
+	0x9bc: 0x3308, 0x9bd: 0x0040, 0x9be: 0x0018, 0x9bf: 0x0018,
+	// Block 0x27, offset 0x9c0
+	0x9c0: 0x0008, 0x9c1: 0x0008, 0x9c2: 0x0008, 0x9c3: 0x0008, 0x9c4: 0x0008, 0x9c5: 0x0008,
+	0x9c6: 0x0008, 0x9c7: 0x0008, 0x9c8: 0x0008, 0x9c9: 0x0008, 0x9ca: 0x0008, 0x9cb: 0x0008,
+	0x9cc: 0x0008, 0x9cd: 0x0008, 0x9ce: 0x0008, 0x9cf: 0x0008, 0x9d0: 0x0008, 0x9d1: 0x0008,
+	0x9d2: 0x0008, 0x9d3: 0x0008, 0x9d4: 0x0008, 0x9d5: 0x0008, 0x9d6: 0x0008, 0x9d7: 0x0008,
+	0x9d8: 0x0008, 0x9d9: 0x0008, 0x9da: 0x0008, 0x9db: 0x0008, 0x9dc: 0x0008, 0x9dd: 0x0008,
+	0x9de: 0x0008, 0x9df: 0x0008, 0x9e0: 0x0008, 0x9e1: 0x0008, 0x9e2: 0x0008, 0x9e3: 0x0008,
+	0x9e4: 0x0008, 0x9e5: 0x0008, 0x9e6: 0x0008, 0x9e7: 0x0008, 0x9e8: 0x0008, 0x9e9: 0x0008,
+	0x9ea: 0x0008, 0x9eb: 0x0008, 0x9ec: 0x0039, 0x9ed: 0x0ed1, 0x9ee: 0x0ee9, 0x9ef: 0x0008,
+	0x9f0: 0x0ef9, 0x9f1: 0x0f09, 0x9f2: 0x0f19, 0x9f3: 0x0f31, 0x9f4: 0x0249, 0x9f5: 0x0f41,
+	0x9f6: 0x0259, 0x9f7: 0x0f51, 0x9f8: 0x0359, 0x9f9: 0x0f61, 0x9fa: 0x0f71, 0x9fb: 0x0008,
+	0x9fc: 0x00d9, 0x9fd: 0x0f81, 0x9fe: 0x0f99, 0x9ff: 0x0269,
+	// Block 0x28, offset 0xa00
+	0xa00: 0x0fa9, 0xa01: 0x0fb9, 0xa02: 0x0279, 0xa03: 0x0039, 0xa04: 0x0fc9, 0xa05: 0x0fe1,
+	0xa06: 0x05b5, 0xa07: 0x0ee9, 0xa08: 0x0ef9, 0xa09: 0x0f09, 0xa0a: 0x0ff9, 0xa0b: 0x1011,
+	0xa0c: 0x1029, 0xa0d: 0x0f31, 0xa0e: 0x0008, 0xa0f: 0x0f51, 0xa10: 0x0f61, 0xa11: 0x1041,
+	0xa12: 0x00d9, 0xa13: 0x1059, 0xa14: 0x05cd, 0xa15: 0x05cd, 0xa16: 0x0f99, 0xa17: 0x0fa9,
+	0xa18: 0x0fb9, 0xa19: 0x05b5, 0xa1a: 0x1071, 0xa1b: 0x1089, 0xa1c: 0x05e5, 0xa1d: 0x1099,
+	0xa1e: 0x10b1, 0xa1f: 0x10c9, 0xa20: 0x10e1, 0xa21: 0x10f9, 0xa22: 0x0f41, 0xa23: 0x0269,
+	0xa24: 0x0fb9, 0xa25: 0x1089, 0xa26: 0x1099, 0xa27: 0x10b1, 0xa28: 0x1111, 0xa29: 0x10e1,
+	0xa2a: 0x10f9, 0xa2b: 0x0008, 0xa2c: 0x0008, 0xa2d: 0x0008, 0xa2e: 0x0008, 0xa2f: 0x0008,
+	0xa30: 0x0008, 0xa31: 0x0008, 0xa32: 0x0008, 0xa33: 0x0008, 0xa34: 0x0008, 0xa35: 0x0008,
+	0xa36: 0x0008, 0xa37: 0x0008, 0xa38: 0x1129, 0xa39: 0x0008, 0xa3a: 0x0008, 0xa3b: 0x0008,
+	0xa3c: 0x0008, 0xa3d: 0x0008, 0xa3e: 0x0008, 0xa3f: 0x0008,
+	// Block 0x29, offset 0xa40
+	0xa40: 0x0008, 0xa41: 0x0008, 0xa42: 0x0008, 0xa43: 0x0008, 0xa44: 0x0008, 0xa45: 0x0008,
+	0xa46: 0x0008, 0xa47: 0x0008, 0xa48: 0x0008, 0xa49: 0x0008, 0xa4a: 0x0008, 0xa4b: 0x0008,
+	0xa4c: 0x0008, 0xa4d: 0x0008, 0xa4e: 0x0008, 0xa4f: 0x0008, 0xa50: 0x0008, 0xa51: 0x0008,
+	0xa52: 0x0008, 0xa53: 0x0008, 0xa54: 0x0008, 0xa55: 0x0008, 0xa56: 0x0008, 0xa57: 0x0008,
+	0xa58: 0x0008, 0xa59: 0x0008, 0xa5a: 0x0008, 0xa5b: 0x1141, 0xa5c: 0x1159, 0xa5d: 0x1169,
+	0xa5e: 0x1181, 0xa5f: 0x1029, 0xa60: 0x1199, 0xa61: 0x11a9, 0xa62: 0x11c1, 0xa63: 0x11d9,
+	0xa64: 0x11f1, 0xa65: 0x1209, 0xa66: 0x1221, 0xa67: 0x05fd, 0xa68: 0x1239, 0xa69: 0x1251,
+	0xa6a: 0xe17d, 0xa6b: 0x1269, 0xa6c: 0x1281, 0xa6d: 0x1299, 0xa6e: 0x12b1, 0xa6f: 0x12c9,
+	0xa70: 0x12e1, 0xa71: 0x12f9, 0xa72: 0x1311, 0xa73: 0x1329, 0xa74: 0x1341, 0xa75: 0x1359,
+	0xa76: 0x1371, 0xa77: 0x1389, 0xa78: 0x0615, 0xa79: 0x13a1, 0xa7a: 0x13b9, 0xa7b: 0x13d1,
+	0xa7c: 0x13e1, 0xa7d: 0x13f9, 0xa7e: 0x1411, 0xa7f: 0x1429,
+	// Block 0x2a, offset 0xa80
+	0xa80: 0xe00d, 0xa81: 0x0008, 0xa82: 0xe00d, 0xa83: 0x0008, 0xa84: 0xe00d, 0xa85: 0x0008,
+	0xa86: 0xe00d, 0xa87: 0x0008, 0xa88: 0xe00d, 0xa89: 0x0008, 0xa8a: 0xe00d, 0xa8b: 0x0008,
+	0xa8c: 0xe00d, 0xa8d: 0x0008, 0xa8e: 0xe00d, 0xa8f: 0x0008, 0xa90: 0xe00d, 0xa91: 0x0008,
+	0xa92: 0xe00d, 0xa93: 0x0008, 0xa94: 0xe00d, 0xa95: 0x0008, 0xa96: 0xe00d, 0xa97: 0x0008,
+	0xa98: 0xe00d, 0xa99: 0x0008, 0xa9a: 0xe00d, 0xa9b: 0x0008, 0xa9c: 0xe00d, 0xa9d: 0x0008,
+	0xa9e: 0xe00d, 0xa9f: 0x0008, 0xaa0: 0xe00d, 0xaa1: 0x0008, 0xaa2: 0xe00d, 0xaa3: 0x0008,
+	0xaa4: 0xe00d, 0xaa5: 0x0008, 0xaa6: 0xe00d, 0xaa7: 0x0008, 0xaa8: 0xe00d, 0xaa9: 0x0008,
+	0xaaa: 0xe00d, 0xaab: 0x0008, 0xaac: 0xe00d, 0xaad: 0x0008, 0xaae: 0xe00d, 0xaaf: 0x0008,
+	0xab0: 0xe00d, 0xab1: 0x0008, 0xab2: 0xe00d, 0xab3: 0x0008, 0xab4: 0xe00d, 0xab5: 0x0008,
+	0xab6: 0xe00d, 0xab7: 0x0008, 0xab8: 0xe00d, 0xab9: 0x0008, 0xaba: 0xe00d, 0xabb: 0x0008,
+	0xabc: 0xe00d, 0xabd: 0x0008, 0xabe: 0xe00d, 0xabf: 0x0008,
+	// Block 0x2b, offset 0xac0
+	0xac0: 0xe00d, 0xac1: 0x0008, 0xac2: 0xe00d, 0xac3: 0x0008, 0xac4: 0xe00d, 0xac5: 0x0008,
+	0xac6: 0xe00d, 0xac7: 0x0008, 0xac8: 0xe00d, 0xac9: 0x0008, 0xaca: 0xe00d, 0xacb: 0x0008,
+	0xacc: 0xe00d, 0xacd: 0x0008, 0xace: 0xe00d, 0xacf: 0x0008, 0xad0: 0xe00d, 0xad1: 0x0008,
+	0xad2: 0xe00d, 0xad3: 0x0008, 0xad4: 0xe00d, 0xad5: 0x0008, 0xad6: 0x0008, 0xad7: 0x0008,
+	0xad8: 0x0008, 0xad9: 0x0008, 0xada: 0x062d, 0xadb: 0x064d, 0xadc: 0x0008, 0xadd: 0x0008,
+	0xade: 0x1441, 0xadf: 0x0008, 0xae0: 0xe00d, 0xae1: 0x0008, 0xae2: 0xe00d, 0xae3: 0x0008,
+	0xae4: 0xe00d, 0xae5: 0x0008, 0xae6: 0xe00d, 0xae7: 0x0008, 0xae8: 0xe00d, 0xae9: 0x0008,
+	0xaea: 0xe00d, 0xaeb: 0x0008, 0xaec: 0xe00d, 0xaed: 0x0008, 0xaee: 0xe00d, 0xaef: 0x0008,
+	0xaf0: 0xe00d, 0xaf1: 0x0008, 0xaf2: 0xe00d, 0xaf3: 0x0008, 0xaf4: 0xe00d, 0xaf5: 0x0008,
+	0xaf6: 0xe00d, 0xaf7: 0x0008, 0xaf8: 0xe00d, 0xaf9: 0x0008, 0xafa: 0xe00d, 0xafb: 0x0008,
+	0xafc: 0xe00d, 0xafd: 0x0008, 0xafe: 0xe00d, 0xaff: 0x0008,
+	// Block 0x2c, offset 0xb00
+	0xb00: 0x0008, 0xb01: 0x0008, 0xb02: 0x0008, 0xb03: 0x0008, 0xb04: 0x0008, 0xb05: 0x0008,
+	0xb06: 0x0040, 0xb07: 0x0040, 0xb08: 0xe045, 0xb09: 0xe045, 0xb0a: 0xe045, 0xb0b: 0xe045,
+	0xb0c: 0xe045, 0xb0d: 0xe045, 0xb0e: 0x0040, 0xb0f: 0x0040, 0xb10: 0x0008, 0xb11: 0x0008,
+	0xb12: 0x0008, 0xb13: 0x0008, 0xb14: 0x0008, 0xb15: 0x0008, 0xb16: 0x0008, 0xb17: 0x0008,
+	0xb18: 0x0040, 0xb19: 0xe045, 0xb1a: 0x0040, 0xb1b: 0xe045, 0xb1c: 0x0040, 0xb1d: 0xe045,
+	0xb1e: 0x0040, 0xb1f: 0xe045, 0xb20: 0x0008, 0xb21: 0x0008, 0xb22: 0x0008, 0xb23: 0x0008,
+	0xb24: 0x0008, 0xb25: 0x0008, 0xb26: 0x0008, 0xb27: 0x0008, 0xb28: 0xe045, 0xb29: 0xe045,
+	0xb2a: 0xe045, 0xb2b: 0xe045, 0xb2c: 0xe045, 0xb2d: 0xe045, 0xb2e: 0xe045, 0xb2f: 0xe045,
+	0xb30: 0x0008, 0xb31: 0x1459, 0xb32: 0x0008, 0xb33: 0x1471, 0xb34: 0x0008, 0xb35: 0x1489,
+	0xb36: 0x0008, 0xb37: 0x14a1, 0xb38: 0x0008, 0xb39: 0x14b9, 0xb3a: 0x0008, 0xb3b: 0x14d1,
+	0xb3c: 0x0008, 0xb3d: 0x14e9, 0xb3e: 0x0040, 0xb3f: 0x0040,
+	// Block 0x2d, offset 0xb40
+	0xb40: 0x1501, 0xb41: 0x1531, 0xb42: 0x1561, 0xb43: 0x1591, 0xb44: 0x15c1, 0xb45: 0x15f1,
+	0xb46: 0x1621, 0xb47: 0x1651, 0xb48: 0x1501, 0xb49: 0x1531, 0xb4a: 0x1561, 0xb4b: 0x1591,
+	0xb4c: 0x15c1, 0xb4d: 0x15f1, 0xb4e: 0x1621, 0xb4f: 0x1651, 0xb50: 0x1681, 0xb51: 0x16b1,
+	0xb52: 0x16e1, 0xb53: 0x1711, 0xb54: 0x1741, 0xb55: 0x1771, 0xb56: 0x17a1, 0xb57: 0x17d1,
+	0xb58: 0x1681, 0xb59: 0x16b1, 0xb5a: 0x16e1, 0xb5b: 0x1711, 0xb5c: 0x1741, 0xb5d: 0x1771,
+	0xb5e: 0x17a1, 0xb5f: 0x17d1, 0xb60: 0x1801, 0xb61: 0x1831, 0xb62: 0x1861, 0xb63: 0x1891,
+	0xb64: 0x18c1, 0xb65: 0x18f1, 0xb66: 0x1921, 0xb67: 0x1951, 0xb68: 0x1801, 0xb69: 0x1831,
+	0xb6a: 0x1861, 0xb6b: 0x1891, 0xb6c: 0x18c1, 0xb6d: 0x18f1, 0xb6e: 0x1921, 0xb6f: 0x1951,
+	0xb70: 0x0008, 0xb71: 0x0008, 0xb72: 0x1981, 0xb73: 0x19b1, 0xb74: 0x19d9, 0xb75: 0x0040,
+	0xb76: 0x0008, 0xb77: 0x1a01, 0xb78: 0xe045, 0xb79: 0xe045, 0xb7a: 0x0665, 0xb7b: 0x1459,
+	0xb7c: 0x19b1, 0xb7d: 0x067e, 0xb7e: 0x1a31, 0xb7f: 0x069e,
+	// Block 0x2e, offset 0xb80
+	0xb80: 0x06be, 0xb81: 0x1a4a, 0xb82: 0x1a79, 0xb83: 0x1aa9, 0xb84: 0x1ad1, 0xb85: 0x0040,
+	0xb86: 0x0008, 0xb87: 0x1af9, 0xb88: 0x06dd, 0xb89: 0x1471, 0xb8a: 0x06f5, 0xb8b: 0x1489,
+	0xb8c: 0x1aa9, 0xb8d: 0x1b2a, 0xb8e: 0x1b5a, 0xb8f: 0x1b8a, 0xb90: 0x0008, 0xb91: 0x0008,
+	0xb92: 0x0008, 0xb93: 0x1bb9, 0xb94: 0x0040, 0xb95: 0x0040, 0xb96: 0x0008, 0xb97: 0x0008,
+	0xb98: 0xe045, 0xb99: 0xe045, 0xb9a: 0x070d, 0xb9b: 0x14a1, 0xb9c: 0x0040, 0xb9d: 0x1bd2,
+	0xb9e: 0x1c02, 0xb9f: 0x1c32, 0xba0: 0x0008, 0xba1: 0x0008, 0xba2: 0x0008, 0xba3: 0x1c61,
+	0xba4: 0x0008, 0xba5: 0x0008, 0xba6: 0x0008, 0xba7: 0x0008, 0xba8: 0xe045, 0xba9: 0xe045,
+	0xbaa: 0x0725, 0xbab: 0x14d1, 0xbac: 0xe04d, 0xbad: 0x1c7a, 0xbae: 0x03d2, 0xbaf: 0x1caa,
+	0xbb0: 0x0040, 0xbb1: 0x0040, 0xbb2: 0x1cb9, 0xbb3: 0x1ce9, 0xbb4: 0x1d11, 0xbb5: 0x0040,
+	0xbb6: 0x0008, 0xbb7: 0x1d39, 0xbb8: 0x073d, 0xbb9: 0x14b9, 0xbba: 0x0515, 0xbbb: 0x14e9,
+	0xbbc: 0x1ce9, 0xbbd: 0x0756, 0xbbe: 0x0776, 0xbbf: 0x0040,
+	// Block 0x2f, offset 0xbc0
+	0xbc0: 0x000a, 0xbc1: 0x000a, 0xbc2: 0x000a, 0xbc3: 0x000a, 0xbc4: 0x000a, 0xbc5: 0x000a,
+	0xbc6: 0x000a, 0xbc7: 0x000a, 0xbc8: 0x000a, 0xbc9: 0x000a, 0xbca: 0x000a, 0xbcb: 0x03c0,
+	0xbcc: 0x0003, 0xbcd: 0x0003, 0xbce: 0x0340, 0xbcf: 0x0b40, 0xbd0: 0x0018, 0xbd1: 0xe00d,
+	0xbd2: 0x0018, 0xbd3: 0x0018, 0xbd4: 0x0018, 0xbd5: 0x0018, 0xbd6: 0x0018, 0xbd7: 0x0796,
+	0xbd8: 0x0018, 0xbd9: 0x0018, 0xbda: 0x0018, 0xbdb: 0x0018, 0xbdc: 0x0018, 0xbdd: 0x0018,
+	0xbde: 0x0018, 0xbdf: 0x0018, 0xbe0: 0x0018, 0xbe1: 0x0018, 0xbe2: 0x0018, 0xbe3: 0x0018,
+	0xbe4: 0x0040, 0xbe5: 0x0040, 0xbe6: 0x0040, 0xbe7: 0x0018, 0xbe8: 0x0040, 0xbe9: 0x0040,
+	0xbea: 0x0340, 0xbeb: 0x0340, 0xbec: 0x0340, 0xbed: 0x0340, 0xbee: 0x0340, 0xbef: 0x000a,
+	0xbf0: 0x0018, 0xbf1: 0x0018, 0xbf2: 0x0018, 0xbf3: 0x1d69, 0xbf4: 0x1da1, 0xbf5: 0x0018,
+	0xbf6: 0x1df1, 0xbf7: 0x1e29, 0xbf8: 0x0018, 0xbf9: 0x0018, 0xbfa: 0x0018, 0xbfb: 0x0018,
+	0xbfc: 0x1e7a, 0xbfd: 0x0018, 0xbfe: 0x07b6, 0xbff: 0x0018,
+	// Block 0x30, offset 0xc00
+	0xc00: 0x0018, 0xc01: 0x0018, 0xc02: 0x0018, 0xc03: 0x0018, 0xc04: 0x0018, 0xc05: 0x0018,
+	0xc06: 0x0018, 0xc07: 0x1e92, 0xc08: 0x1eaa, 0xc09: 0x1ec2, 0xc0a: 0x0018, 0xc0b: 0x0018,
+	0xc0c: 0x0018, 0xc0d: 0x0018, 0xc0e: 0x0018, 0xc0f: 0x0018, 0xc10: 0x0018, 0xc11: 0x0018,
+	0xc12: 0x0018, 0xc13: 0x0018, 0xc14: 0x0018, 0xc15: 0x0018, 0xc16: 0x0018, 0xc17: 0x1ed9,
+	0xc18: 0x0018, 0xc19: 0x0018, 0xc1a: 0x0018, 0xc1b: 0x0018, 0xc1c: 0x0018, 0xc1d: 0x0018,
+	0xc1e: 0x0018, 0xc1f: 0x000a, 0xc20: 0x03c0, 0xc21: 0x0340, 0xc22: 0x0340, 0xc23: 0x0340,
+	0xc24: 0x03c0, 0xc25: 0x0040, 0xc26: 0x0040, 0xc27: 0x0040, 0xc28: 0x0040, 0xc29: 0x0040,
+	0xc2a: 0x0340, 0xc2b: 0x0340, 0xc2c: 0x0340, 0xc2d: 0x0340, 0xc2e: 0x0340, 0xc2f: 0x0340,
+	0xc30: 0x1f41, 0xc31: 0x0f41, 0xc32: 0x0040, 0xc33: 0x0040, 0xc34: 0x1f51, 0xc35: 0x1f61,
+	0xc36: 0x1f71, 0xc37: 0x1f81, 0xc38: 0x1f91, 0xc39: 0x1fa1, 0xc3a: 0x1fb2, 0xc3b: 0x07d5,
+	0xc3c: 0x1fc2, 0xc3d: 0x1fd2, 0xc3e: 0x1fe2, 0xc3f: 0x0f71,
+	// Block 0x31, offset 0xc40
+	0xc40: 0x1f41, 0xc41: 0x00c9, 0xc42: 0x0069, 0xc43: 0x0079, 0xc44: 0x1f51, 0xc45: 0x1f61,
+	0xc46: 0x1f71, 0xc47: 0x1f81, 0xc48: 0x1f91, 0xc49: 0x1fa1, 0xc4a: 0x1fb2, 0xc4b: 0x07ed,
+	0xc4c: 0x1fc2, 0xc4d: 0x1fd2, 0xc4e: 0x1fe2, 0xc4f: 0x0040, 0xc50: 0x0039, 0xc51: 0x0f09,
+	0xc52: 0x00d9, 0xc53: 0x0369, 0xc54: 0x0ff9, 0xc55: 0x0249, 0xc56: 0x0f51, 0xc57: 0x0359,
+	0xc58: 0x0f61, 0xc59: 0x0f71, 0xc5a: 0x0f99, 0xc5b: 0x01d9, 0xc5c: 0x0fa9, 0xc5d: 0x0040,
+	0xc5e: 0x0040, 0xc5f: 0x0040, 0xc60: 0x0018, 0xc61: 0x0018, 0xc62: 0x0018, 0xc63: 0x0018,
+	0xc64: 0x0018, 0xc65: 0x0018, 0xc66: 0x0018, 0xc67: 0x0018, 0xc68: 0x1ff1, 0xc69: 0x0018,
+	0xc6a: 0x0018, 0xc6b: 0x0018, 0xc6c: 0x0018, 0xc6d: 0x0018, 0xc6e: 0x0018, 0xc6f: 0x0018,
+	0xc70: 0x0018, 0xc71: 0x0018, 0xc72: 0x0018, 0xc73: 0x0018, 0xc74: 0x0018, 0xc75: 0x0018,
+	0xc76: 0x0018, 0xc77: 0x0018, 0xc78: 0x0018, 0xc79: 0x0018, 0xc7a: 0x0018, 0xc7b: 0x0018,
+	0xc7c: 0x0018, 0xc7d: 0x0018, 0xc7e: 0x0018, 0xc7f: 0x0018,
+	// Block 0x32, offset 0xc80
+	0xc80: 0x0806, 0xc81: 0x0826, 0xc82: 0x1159, 0xc83: 0x0845, 0xc84: 0x0018, 0xc85: 0x0866,
+	0xc86: 0x0886, 0xc87: 0x1011, 0xc88: 0x0018, 0xc89: 0x08a5, 0xc8a: 0x0f31, 0xc8b: 0x0249,
+	0xc8c: 0x0249, 0xc8d: 0x0249, 0xc8e: 0x0249, 0xc8f: 0x2009, 0xc90: 0x0f41, 0xc91: 0x0f41,
+	0xc92: 0x0359, 0xc93: 0x0359, 0xc94: 0x0018, 0xc95: 0x0f71, 0xc96: 0x2021, 0xc97: 0x0018,
+	0xc98: 0x0018, 0xc99: 0x0f99, 0xc9a: 0x2039, 0xc9b: 0x0269, 0xc9c: 0x0269, 0xc9d: 0x0269,
+	0xc9e: 0x0018, 0xc9f: 0x0018, 0xca0: 0x2049, 0xca1: 0x08c5, 0xca2: 0x2061, 0xca3: 0x0018,
+	0xca4: 0x13d1, 0xca5: 0x0018, 0xca6: 0x2079, 0xca7: 0x0018, 0xca8: 0x13d1, 0xca9: 0x0018,
+	0xcaa: 0x0f51, 0xcab: 0x2091, 0xcac: 0x0ee9, 0xcad: 0x1159, 0xcae: 0x0018, 0xcaf: 0x0f09,
+	0xcb0: 0x0f09, 0xcb1: 0x1199, 0xcb2: 0x0040, 0xcb3: 0x0f61, 0xcb4: 0x00d9, 0xcb5: 0x20a9,
+	0xcb6: 0x20c1, 0xcb7: 0x20d9, 0xcb8: 0x20f1, 0xcb9: 0x0f41, 0xcba: 0x0018, 0xcbb: 0x08e5,
+	0xcbc: 0x2109, 0xcbd: 0x10b1, 0xcbe: 0x10b1, 0xcbf: 0x2109,
+	// Block 0x33, offset 0xcc0
+	0xcc0: 0x0905, 0xcc1: 0x0018, 0xcc2: 0x0018, 0xcc3: 0x0018, 0xcc4: 0x0018, 0xcc5: 0x0ef9,
+	0xcc6: 0x0ef9, 0xcc7: 0x0f09, 0xcc8: 0x0f41, 0xcc9: 0x0259, 0xcca: 0x0018, 0xccb: 0x0018,
+	0xccc: 0x0018, 0xccd: 0x0018, 0xcce: 0x0008, 0xccf: 0x0018, 0xcd0: 0x2121, 0xcd1: 0x2151,
+	0xcd2: 0x2181, 0xcd3: 0x21b9, 0xcd4: 0x21e9, 0xcd5: 0x2219, 0xcd6: 0x2249, 0xcd7: 0x2279,
+	0xcd8: 0x22a9, 0xcd9: 0x22d9, 0xcda: 0x2309, 0xcdb: 0x2339, 0xcdc: 0x2369, 0xcdd: 0x2399,
+	0xcde: 0x23c9, 0xcdf: 0x23f9, 0xce0: 0x0f41, 0xce1: 0x2421, 0xce2: 0x091d, 0xce3: 0x2439,
+	0xce4: 0x1089, 0xce5: 0x2451, 0xce6: 0x093d, 0xce7: 0x2469, 0xce8: 0x2491, 0xce9: 0x0369,
+	0xcea: 0x24a9, 0xceb: 0x095d, 0xcec: 0x0359, 0xced: 0x1159, 0xcee: 0x0ef9, 0xcef: 0x0f61,
+	0xcf0: 0x0f41, 0xcf1: 0x2421, 0xcf2: 0x097d, 0xcf3: 0x2439, 0xcf4: 0x1089, 0xcf5: 0x2451,
+	0xcf6: 0x099d, 0xcf7: 0x2469, 0xcf8: 0x2491, 0xcf9: 0x0369, 0xcfa: 0x24a9, 0xcfb: 0x09bd,
+	0xcfc: 0x0359, 0xcfd: 0x1159, 0xcfe: 0x0ef9, 0xcff: 0x0f61,
+	// Block 0x34, offset 0xd00
+	0xd00: 0x0018, 0xd01: 0x0018, 0xd02: 0x0018, 0xd03: 0x0018, 0xd04: 0x0018, 0xd05: 0x0018,
+	0xd06: 0x0018, 0xd07: 0x0018, 0xd08: 0x0018, 0xd09: 0x0018, 0xd0a: 0x0018, 0xd0b: 0x0040,
+	0xd0c: 0x0040, 0xd0d: 0x0040, 0xd0e: 0x0040, 0xd0f: 0x0040, 0xd10: 0x0040, 0xd11: 0x0040,
+	0xd12: 0x0040, 0xd13: 0x0040, 0xd14: 0x0040, 0xd15: 0x0040, 0xd16: 0x0040, 0xd17: 0x0040,
+	0xd18: 0x0040, 0xd19: 0x0040, 0xd1a: 0x0040, 0xd1b: 0x0040, 0xd1c: 0x0040, 0xd1d: 0x0040,
+	0xd1e: 0x0040, 0xd1f: 0x0040, 0xd20: 0x00c9, 0xd21: 0x0069, 0xd22: 0x0079, 0xd23: 0x1f51,
+	0xd24: 0x1f61, 0xd25: 0x1f71, 0xd26: 0x1f81, 0xd27: 0x1f91, 0xd28: 0x1fa1, 0xd29: 0x2601,
+	0xd2a: 0x2619, 0xd2b: 0x2631, 0xd2c: 0x2649, 0xd2d: 0x2661, 0xd2e: 0x2679, 0xd2f: 0x2691,
+	0xd30: 0x26a9, 0xd31: 0x26c1, 0xd32: 0x26d9, 0xd33: 0x26f1, 0xd34: 0x0a1e, 0xd35: 0x0a3e,
+	0xd36: 0x0a5e, 0xd37: 0x0a7e, 0xd38: 0x0a9e, 0xd39: 0x0abe, 0xd3a: 0x0ade, 0xd3b: 0x0afe,
+	0xd3c: 0x0b1e, 0xd3d: 0x270a, 0xd3e: 0x2732, 0xd3f: 0x275a,
+	// Block 0x35, offset 0xd40
+	0xd40: 0x2782, 0xd41: 0x27aa, 0xd42: 0x27d2, 0xd43: 0x27fa, 0xd44: 0x2822, 0xd45: 0x284a,
+	0xd46: 0x2872, 0xd47: 0x289a, 0xd48: 0x0040, 0xd49: 0x0040, 0xd4a: 0x0040, 0xd4b: 0x0040,
+	0xd4c: 0x0040, 0xd4d: 0x0040, 0xd4e: 0x0040, 0xd4f: 0x0040, 0xd50: 0x0040, 0xd51: 0x0040,
+	0xd52: 0x0040, 0xd53: 0x0040, 0xd54: 0x0040, 0xd55: 0x0040, 0xd56: 0x0040, 0xd57: 0x0040,
+	0xd58: 0x0040, 0xd59: 0x0040, 0xd5a: 0x0040, 0xd5b: 0x0040, 0xd5c: 0x0b3e, 0xd5d: 0x0b5e,
+	0xd5e: 0x0b7e, 0xd5f: 0x0b9e, 0xd60: 0x0bbe, 0xd61: 0x0bde, 0xd62: 0x0bfe, 0xd63: 0x0c1e,
+	0xd64: 0x0c3e, 0xd65: 0x0c5e, 0xd66: 0x0c7e, 0xd67: 0x0c9e, 0xd68: 0x0cbe, 0xd69: 0x0cde,
+	0xd6a: 0x0cfe, 0xd6b: 0x0d1e, 0xd6c: 0x0d3e, 0xd6d: 0x0d5e, 0xd6e: 0x0d7e, 0xd6f: 0x0d9e,
+	0xd70: 0x0dbe, 0xd71: 0x0dde, 0xd72: 0x0dfe, 0xd73: 0x0e1e, 0xd74: 0x0e3e, 0xd75: 0x0e5e,
+	0xd76: 0x0039, 0xd77: 0x0ee9, 0xd78: 0x1159, 0xd79: 0x0ef9, 0xd7a: 0x0f09, 0xd7b: 0x1199,
+	0xd7c: 0x0f31, 0xd7d: 0x0249, 0xd7e: 0x0f41, 0xd7f: 0x0259,
+	// Block 0x36, offset 0xd80
+	0xd80: 0x0f51, 0xd81: 0x0359, 0xd82: 0x0f61, 0xd83: 0x0f71, 0xd84: 0x00d9, 0xd85: 0x0f99,
+	0xd86: 0x2039, 0xd87: 0x0269, 0xd88: 0x01d9, 0xd89: 0x0fa9, 0xd8a: 0x0fb9, 0xd8b: 0x1089,
+	0xd8c: 0x0279, 0xd8d: 0x0369, 0xd8e: 0x0289, 0xd8f: 0x13d1, 0xd90: 0x0039, 0xd91: 0x0ee9,
+	0xd92: 0x1159, 0xd93: 0x0ef9, 0xd94: 0x0f09, 0xd95: 0x1199, 0xd96: 0x0f31, 0xd97: 0x0249,
+	0xd98: 0x0f41, 0xd99: 0x0259, 0xd9a: 0x0f51, 0xd9b: 0x0359, 0xd9c: 0x0f61, 0xd9d: 0x0f71,
+	0xd9e: 0x00d9, 0xd9f: 0x0f99, 0xda0: 0x2039, 0xda1: 0x0269, 0xda2: 0x01d9, 0xda3: 0x0fa9,
+	0xda4: 0x0fb9, 0xda5: 0x1089, 0xda6: 0x0279, 0xda7: 0x0369, 0xda8: 0x0289, 0xda9: 0x13d1,
+	0xdaa: 0x1f41, 0xdab: 0x0018, 0xdac: 0x0018, 0xdad: 0x0018, 0xdae: 0x0018, 0xdaf: 0x0018,
+	0xdb0: 0x0018, 0xdb1: 0x0018, 0xdb2: 0x0018, 0xdb3: 0x0018, 0xdb4: 0x0018, 0xdb5: 0x0018,
+	0xdb6: 0x0018, 0xdb7: 0x0018, 0xdb8: 0x0018, 0xdb9: 0x0018, 0xdba: 0x0018, 0xdbb: 0x0018,
+	0xdbc: 0x0018, 0xdbd: 0x0018, 0xdbe: 0x0018, 0xdbf: 0x0018,
+	// Block 0x37, offset 0xdc0
+	0xdc0: 0x0008, 0xdc1: 0x0008, 0xdc2: 0x0008, 0xdc3: 0x0008, 0xdc4: 0x0008, 0xdc5: 0x0008,
+	0xdc6: 0x0008, 0xdc7: 0x0008, 0xdc8: 0x0008, 0xdc9: 0x0008, 0xdca: 0x0008, 0xdcb: 0x0008,
+	0xdcc: 0x0008, 0xdcd: 0x0008, 0xdce: 0x0008, 0xdcf: 0x0008, 0xdd0: 0x0008, 0xdd1: 0x0008,
+	0xdd2: 0x0008, 0xdd3: 0x0008, 0xdd4: 0x0008, 0xdd5: 0x0008, 0xdd6: 0x0008, 0xdd7: 0x0008,
+	0xdd8: 0x0008, 0xdd9: 0x0008, 0xdda: 0x0008, 0xddb: 0x0008, 0xddc: 0x0008, 0xddd: 0x0008,
+	0xdde: 0x0008, 0xddf: 0x0040, 0xde0: 0xe00d, 0xde1: 0x0008, 0xde2: 0x2971, 0xde3: 0x0ed5,
+	0xde4: 0x2989, 0xde5: 0x0008, 0xde6: 0x0008, 0xde7: 0xe07d, 0xde8: 0x0008, 0xde9: 0xe01d,
+	0xdea: 0x0008, 0xdeb: 0xe03d, 0xdec: 0x0008, 0xded: 0x0fe1, 0xdee: 0x1281, 0xdef: 0x0fc9,
+	0xdf0: 0x1141, 0xdf1: 0x0008, 0xdf2: 0xe00d, 0xdf3: 0x0008, 0xdf4: 0x0008, 0xdf5: 0xe01d,
+	0xdf6: 0x0008, 0xdf7: 0x0008, 0xdf8: 0x0008, 0xdf9: 0x0008, 0xdfa: 0x0008, 0xdfb: 0x0008,
+	0xdfc: 0x0259, 0xdfd: 0x1089, 0xdfe: 0x29a1, 0xdff: 0x29b9,
+	// Block 0x38, offset 0xe00
+	0xe00: 0xe00d, 0xe01: 0x0008, 0xe02: 0xe00d, 0xe03: 0x0008, 0xe04: 0xe00d, 0xe05: 0x0008,
+	0xe06: 0xe00d, 0xe07: 0x0008, 0xe08: 0xe00d, 0xe09: 0x0008, 0xe0a: 0xe00d, 0xe0b: 0x0008,
+	0xe0c: 0xe00d, 0xe0d: 0x0008, 0xe0e: 0xe00d, 0xe0f: 0x0008, 0xe10: 0xe00d, 0xe11: 0x0008,
+	0xe12: 0xe00d, 0xe13: 0x0008, 0xe14: 0xe00d, 0xe15: 0x0008, 0xe16: 0xe00d, 0xe17: 0x0008,
+	0xe18: 0xe00d, 0xe19: 0x0008, 0xe1a: 0xe00d, 0xe1b: 0x0008, 0xe1c: 0xe00d, 0xe1d: 0x0008,
+	0xe1e: 0xe00d, 0xe1f: 0x0008, 0xe20: 0xe00d, 0xe21: 0x0008, 0xe22: 0xe00d, 0xe23: 0x0008,
+	0xe24: 0x0008, 0xe25: 0x0018, 0xe26: 0x0018, 0xe27: 0x0018, 0xe28: 0x0018, 0xe29: 0x0018,
+	0xe2a: 0x0018, 0xe2b: 0xe03d, 0xe2c: 0x0008, 0xe2d: 0xe01d, 0xe2e: 0x0008, 0xe2f: 0x3308,
+	0xe30: 0x3308, 0xe31: 0x3308, 0xe32: 0xe00d, 0xe33: 0x0008, 0xe34: 0x0040, 0xe35: 0x0040,
+	0xe36: 0x0040, 0xe37: 0x0040, 0xe38: 0x0040, 0xe39: 0x0018, 0xe3a: 0x0018, 0xe3b: 0x0018,
+	0xe3c: 0x0018, 0xe3d: 0x0018, 0xe3e: 0x0018, 0xe3f: 0x0018,
+	// Block 0x39, offset 0xe40
+	0xe40: 0x2715, 0xe41: 0x2735, 0xe42: 0x2755, 0xe43: 0x2775, 0xe44: 0x2795, 0xe45: 0x27b5,
+	0xe46: 0x27d5, 0xe47: 0x27f5, 0xe48: 0x2815, 0xe49: 0x2835, 0xe4a: 0x2855, 0xe4b: 0x2875,
+	0xe4c: 0x2895, 0xe4d: 0x28b5, 0xe4e: 0x28d5, 0xe4f: 0x28f5, 0xe50: 0x2915, 0xe51: 0x2935,
+	0xe52: 0x2955, 0xe53: 0x2975, 0xe54: 0x2995, 0xe55: 0x29b5, 0xe56: 0x0040, 0xe57: 0x0040,
+	0xe58: 0x0040, 0xe59: 0x0040, 0xe5a: 0x0040, 0xe5b: 0x0040, 0xe5c: 0x0040, 0xe5d: 0x0040,
+	0xe5e: 0x0040, 0xe5f: 0x0040, 0xe60: 0x0040, 0xe61: 0x0040, 0xe62: 0x0040, 0xe63: 0x0040,
+	0xe64: 0x0040, 0xe65: 0x0040, 0xe66: 0x0040, 0xe67: 0x0040, 0xe68: 0x0040, 0xe69: 0x0040,
+	0xe6a: 0x0040, 0xe6b: 0x0040, 0xe6c: 0x0040, 0xe6d: 0x0040, 0xe6e: 0x0040, 0xe6f: 0x0040,
+	0xe70: 0x0040, 0xe71: 0x0040, 0xe72: 0x0040, 0xe73: 0x0040, 0xe74: 0x0040, 0xe75: 0x0040,
+	0xe76: 0x0040, 0xe77: 0x0040, 0xe78: 0x0040, 0xe79: 0x0040, 0xe7a: 0x0040, 0xe7b: 0x0040,
+	0xe7c: 0x0040, 0xe7d: 0x0040, 0xe7e: 0x0040, 0xe7f: 0x0040,
+	// Block 0x3a, offset 0xe80
+	0xe80: 0x000a, 0xe81: 0x0018, 0xe82: 0x29d1, 0xe83: 0x0018, 0xe84: 0x0018, 0xe85: 0x0008,
+	0xe86: 0x0008, 0xe87: 0x0008, 0xe88: 0x0018, 0xe89: 0x0018, 0xe8a: 0x0018, 0xe8b: 0x0018,
+	0xe8c: 0x0018, 0xe8d: 0x0018, 0xe8e: 0x0018, 0xe8f: 0x0018, 0xe90: 0x0018, 0xe91: 0x0018,
+	0xe92: 0x0018, 0xe93: 0x0018, 0xe94: 0x0018, 0xe95: 0x0018, 0xe96: 0x0018, 0xe97: 0x0018,
+	0xe98: 0x0018, 0xe99: 0x0018, 0xe9a: 0x0018, 0xe9b: 0x0018, 0xe9c: 0x0018, 0xe9d: 0x0018,
+	0xe9e: 0x0018, 0xe9f: 0x0018, 0xea0: 0x0018, 0xea1: 0x0018, 0xea2: 0x0018, 0xea3: 0x0018,
+	0xea4: 0x0018, 0xea5: 0x0018, 0xea6: 0x0018, 0xea7: 0x0018, 0xea8: 0x0018, 0xea9: 0x0018,
+	0xeaa: 0x3308, 0xeab: 0x3308, 0xeac: 0x3308, 0xead: 0x3308, 0xeae: 0x3018, 0xeaf: 0x3018,
+	0xeb0: 0x0018, 0xeb1: 0x0018, 0xeb2: 0x0018, 0xeb3: 0x0018, 0xeb4: 0x0018, 0xeb5: 0x0018,
+	0xeb6: 0xe125, 0xeb7: 0x0018, 0xeb8: 0x29d5, 0xeb9: 0x29f5, 0xeba: 0x2a15, 0xebb: 0x0018,
+	0xebc: 0x0008, 0xebd: 0x0018, 0xebe: 0x0018, 0xebf: 0x0018,
+	// Block 0x3b, offset 0xec0
+	0xec0: 0x2b55, 0xec1: 0x2b75, 0xec2: 0x2b95, 0xec3: 0x2bb5, 0xec4: 0x2bd5, 0xec5: 0x2bf5,
+	0xec6: 0x2bf5, 0xec7: 0x2bf5, 0xec8: 0x2c15, 0xec9: 0x2c15, 0xeca: 0x2c15, 0xecb: 0x2c15,
+	0xecc: 0x2c35, 0xecd: 0x2c35, 0xece: 0x2c35, 0xecf: 0x2c55, 0xed0: 0x2c75, 0xed1: 0x2c75,
+	0xed2: 0x2a95, 0xed3: 0x2a95, 0xed4: 0x2c75, 0xed5: 0x2c75, 0xed6: 0x2c95, 0xed7: 0x2c95,
+	0xed8: 0x2c75, 0xed9: 0x2c75, 0xeda: 0x2a95, 0xedb: 0x2a95, 0xedc: 0x2c75, 0xedd: 0x2c75,
+	0xede: 0x2c55, 0xedf: 0x2c55, 0xee0: 0x2cb5, 0xee1: 0x2cb5, 0xee2: 0x2cd5, 0xee3: 0x2cd5,
+	0xee4: 0x0040, 0xee5: 0x2cf5, 0xee6: 0x2d15, 0xee7: 0x2d35, 0xee8: 0x2d35, 0xee9: 0x2d55,
+	0xeea: 0x2d75, 0xeeb: 0x2d95, 0xeec: 0x2db5, 0xeed: 0x2dd5, 0xeee: 0x2df5, 0xeef: 0x2e15,
+	0xef0: 0x2e35, 0xef1: 0x2e55, 0xef2: 0x2e55, 0xef3: 0x2e75, 0xef4: 0x2e95, 0xef5: 0x2e95,
+	0xef6: 0x2eb5, 0xef7: 0x2ed5, 0xef8: 0x2e75, 0xef9: 0x2ef5, 0xefa: 0x2f15, 0xefb: 0x2ef5,
+	0xefc: 0x2e75, 0xefd: 0x2f35, 0xefe: 0x2f55, 0xeff: 0x2f75,
+	// Block 0x3c, offset 0xf00
+	0xf00: 0x2f95, 0xf01: 0x2fb5, 0xf02: 0x2d15, 0xf03: 0x2cf5, 0xf04: 0x2fd5, 0xf05: 0x2ff5,
+	0xf06: 0x3015, 0xf07: 0x3035, 0xf08: 0x3055, 0xf09: 0x3075, 0xf0a: 0x3095, 0xf0b: 0x30b5,
+	0xf0c: 0x30d5, 0xf0d: 0x30f5, 0xf0e: 0x3115, 0xf0f: 0x0040, 0xf10: 0x0018, 0xf11: 0x0018,
+	0xf12: 0x3135, 0xf13: 0x3155, 0xf14: 0x3175, 0xf15: 0x3195, 0xf16: 0x31b5, 0xf17: 0x31d5,
+	0xf18: 0x31f5, 0xf19: 0x3215, 0xf1a: 0x3235, 0xf1b: 0x3255, 0xf1c: 0x3175, 0xf1d: 0x3275,
+	0xf1e: 0x3295, 0xf1f: 0x32b5, 0xf20: 0x0008, 0xf21: 0x0008, 0xf22: 0x0008, 0xf23: 0x0008,
+	0xf24: 0x0008, 0xf25: 0x0008, 0xf26: 0x0008, 0xf27: 0x0008, 0xf28: 0x0008, 0xf29: 0x0008,
+	0xf2a: 0x0008, 0xf2b: 0x0008, 0xf2c: 0x0008, 0xf2d: 0x0008, 0xf2e: 0x0008, 0xf2f: 0x0008,
+	0xf30: 0x0008, 0xf31: 0x0008, 0xf32: 0x0008, 0xf33: 0x0008, 0xf34: 0x0008, 0xf35: 0x0008,
+	0xf36: 0x0008, 0xf37: 0x0008, 0xf38: 0x0008, 0xf39: 0x0008, 0xf3a: 0x0008, 0xf3b: 0x0040,
+	0xf3c: 0x0040, 0xf3d: 0x0040, 0xf3e: 0x0040, 0xf3f: 0x0040,
+	// Block 0x3d, offset 0xf40
+	0xf40: 0x36a2, 0xf41: 0x36d2, 0xf42: 0x3702, 0xf43: 0x3732, 0xf44: 0x32d5, 0xf45: 0x32f5,
+	0xf46: 0x3315, 0xf47: 0x3335, 0xf48: 0x0018, 0xf49: 0x0018, 0xf4a: 0x0018, 0xf4b: 0x0018,
+	0xf4c: 0x0018, 0xf4d: 0x0018, 0xf4e: 0x0018, 0xf4f: 0x0018, 0xf50: 0x3355, 0xf51: 0x3761,
+	0xf52: 0x3779, 0xf53: 0x3791, 0xf54: 0x37a9, 0xf55: 0x37c1, 0xf56: 0x37d9, 0xf57: 0x37f1,
+	0xf58: 0x3809, 0xf59: 0x3821, 0xf5a: 0x3839, 0xf5b: 0x3851, 0xf5c: 0x3869, 0xf5d: 0x3881,
+	0xf5e: 0x3899, 0xf5f: 0x38b1, 0xf60: 0x3375, 0xf61: 0x3395, 0xf62: 0x33b5, 0xf63: 0x33d5,
+	0xf64: 0x33f5, 0xf65: 0x33f5, 0xf66: 0x3415, 0xf67: 0x3435, 0xf68: 0x3455, 0xf69: 0x3475,
+	0xf6a: 0x3495, 0xf6b: 0x34b5, 0xf6c: 0x34d5, 0xf6d: 0x34f5, 0xf6e: 0x3515, 0xf6f: 0x3535,
+	0xf70: 0x3555, 0xf71: 0x3575, 0xf72: 0x3595, 0xf73: 0x35b5, 0xf74: 0x35d5, 0xf75: 0x35f5,
+	0xf76: 0x3615, 0xf77: 0x3635, 0xf78: 0x3655, 0xf79: 0x3675, 0xf7a: 0x3695, 0xf7b: 0x36b5,
+	0xf7c: 0x38c9, 0xf7d: 0x3901, 0xf7e: 0x36d5, 0xf7f: 0x0018,
+	// Block 0x3e, offset 0xf80
+	0xf80: 0x36f5, 0xf81: 0x3715, 0xf82: 0x3735, 0xf83: 0x3755, 0xf84: 0x3775, 0xf85: 0x3795,
+	0xf86: 0x37b5, 0xf87: 0x37d5, 0xf88: 0x37f5, 0xf89: 0x3815, 0xf8a: 0x3835, 0xf8b: 0x3855,
+	0xf8c: 0x3875, 0xf8d: 0x3895, 0xf8e: 0x38b5, 0xf8f: 0x38d5, 0xf90: 0x38f5, 0xf91: 0x3915,
+	0xf92: 0x3935, 0xf93: 0x3955, 0xf94: 0x3975, 0xf95: 0x3995, 0xf96: 0x39b5, 0xf97: 0x39d5,
+	0xf98: 0x39f5, 0xf99: 0x3a15, 0xf9a: 0x3a35, 0xf9b: 0x3a55, 0xf9c: 0x3a75, 0xf9d: 0x3a95,
+	0xf9e: 0x3ab5, 0xf9f: 0x3ad5, 0xfa0: 0x3af5, 0xfa1: 0x3b15, 0xfa2: 0x3b35, 0xfa3: 0x3b55,
+	0xfa4: 0x3b75, 0xfa5: 0x3b95, 0xfa6: 0x1295, 0xfa7: 0x3bb5, 0xfa8: 0x3bd5, 0xfa9: 0x3bf5,
+	0xfaa: 0x3c15, 0xfab: 0x3c35, 0xfac: 0x3c55, 0xfad: 0x3c75, 0xfae: 0x23b5, 0xfaf: 0x3c95,
+	0xfb0: 0x3cb5, 0xfb1: 0x3939, 0xfb2: 0x3951, 0xfb3: 0x3969, 0xfb4: 0x3981, 0xfb5: 0x3999,
+	0xfb6: 0x39b1, 0xfb7: 0x39c9, 0xfb8: 0x39e1, 0xfb9: 0x39f9, 0xfba: 0x3a11, 0xfbb: 0x3a29,
+	0xfbc: 0x3a41, 0xfbd: 0x3a59, 0xfbe: 0x3a71, 0xfbf: 0x3a89,
+	// Block 0x3f, offset 0xfc0
+	0xfc0: 0x3aa1, 0xfc1: 0x3ac9, 0xfc2: 0x3af1, 0xfc3: 0x3b19, 0xfc4: 0x3b41, 0xfc5: 0x3b69,
+	0xfc6: 0x3b91, 0xfc7: 0x3bb9, 0xfc8: 0x3be1, 0xfc9: 0x3c09, 0xfca: 0x3c39, 0xfcb: 0x3c69,
+	0xfcc: 0x3c99, 0xfcd: 0x3cd5, 0xfce: 0x3cb1, 0xfcf: 0x3cf5, 0xfd0: 0x3d15, 0xfd1: 0x3d2d,
+	0xfd2: 0x3d45, 0xfd3: 0x3d5d, 0xfd4: 0x3d75, 0xfd5: 0x3d75, 0xfd6: 0x3d5d, 0xfd7: 0x3d8d,
+	0xfd8: 0x07d5, 0xfd9: 0x3da5, 0xfda: 0x3dbd, 0xfdb: 0x3dd5, 0xfdc: 0x3ded, 0xfdd: 0x3e05,
+	0xfde: 0x3e1d, 0xfdf: 0x3e35, 0xfe0: 0x3e4d, 0xfe1: 0x3e65, 0xfe2: 0x3e7d, 0xfe3: 0x3e95,
+	0xfe4: 0x3ead, 0xfe5: 0x3ead, 0xfe6: 0x3ec5, 0xfe7: 0x3ec5, 0xfe8: 0x3edd, 0xfe9: 0x3edd,
+	0xfea: 0x3ef5, 0xfeb: 0x3f0d, 0xfec: 0x3f25, 0xfed: 0x3f3d, 0xfee: 0x3f55, 0xfef: 0x3f55,
+	0xff0: 0x3f6d, 0xff1: 0x3f6d, 0xff2: 0x3f6d, 0xff3: 0x3f85, 0xff4: 0x3f9d, 0xff5: 0x3fb5,
+	0xff6: 0x3fcd, 0xff7: 0x3fb5, 0xff8: 0x3fe5, 0xff9: 0x3ffd, 0xffa: 0x3f85, 0xffb: 0x4015,
+	0xffc: 0x402d, 0xffd: 0x402d, 0xffe: 0x402d, 0xfff: 0x0040,
+	// Block 0x40, offset 0x1000
+	0x1000: 0x3cc9, 0x1001: 0x3d31, 0x1002: 0x3d99, 0x1003: 0x3e01, 0x1004: 0x3e51, 0x1005: 0x3eb9,
+	0x1006: 0x3f09, 0x1007: 0x3f59, 0x1008: 0x3fd9, 0x1009: 0x4041, 0x100a: 0x4091, 0x100b: 0x40e1,
+	0x100c: 0x4131, 0x100d: 0x4199, 0x100e: 0x4201, 0x100f: 0x4251, 0x1010: 0x42a1, 0x1011: 0x42d9,
+	0x1012: 0x4329, 0x1013: 0x4391, 0x1014: 0x43f9, 0x1015: 0x4431, 0x1016: 0x44b1, 0x1017: 0x4549,
+	0x1018: 0x45c9, 0x1019: 0x4619, 0x101a: 0x4699, 0x101b: 0x4719, 0x101c: 0x4781, 0x101d: 0x47d1,
+	0x101e: 0x4821, 0x101f: 0x4871, 0x1020: 0x48d9, 0x1021: 0x4959, 0x1022: 0x49c1, 0x1023: 0x4a11,
+	0x1024: 0x4a61, 0x1025: 0x4ab1, 0x1026: 0x4ae9, 0x1027: 0x4b21, 0x1028: 0x4b59, 0x1029: 0x4b91,
+	0x102a: 0x4be1, 0x102b: 0x4c31, 0x102c: 0x4cb1, 0x102d: 0x4d01, 0x102e: 0x4d69, 0x102f: 0x4de9,
+	0x1030: 0x4e39, 0x1031: 0x4e71, 0x1032: 0x4ea9, 0x1033: 0x4f29, 0x1034: 0x4f91, 0x1035: 0x5011,
+	0x1036: 0x5061, 0x1037: 0x50e1, 0x1038: 0x5119, 0x1039: 0x5169, 0x103a: 0x51b9, 0x103b: 0x5209,
+	0x103c: 0x5259, 0x103d: 0x52a9, 0x103e: 0x5311, 0x103f: 0x5361,
+	// Block 0x41, offset 0x1040
+	0x1040: 0x5399, 0x1041: 0x53e9, 0x1042: 0x5439, 0x1043: 0x5489, 0x1044: 0x54f1, 0x1045: 0x5541,
+	0x1046: 0x5591, 0x1047: 0x55e1, 0x1048: 0x5661, 0x1049: 0x56c9, 0x104a: 0x5701, 0x104b: 0x5781,
+	0x104c: 0x57b9, 0x104d: 0x5821, 0x104e: 0x5889, 0x104f: 0x58d9, 0x1050: 0x5929, 0x1051: 0x5979,
+	0x1052: 0x59e1, 0x1053: 0x5a19, 0x1054: 0x5a69, 0x1055: 0x5ad1, 0x1056: 0x5b09, 0x1057: 0x5b89,
+	0x1058: 0x5bd9, 0x1059: 0x5c01, 0x105a: 0x5c29, 0x105b: 0x5c51, 0x105c: 0x5c79, 0x105d: 0x5ca1,
+	0x105e: 0x5cc9, 0x105f: 0x5cf1, 0x1060: 0x5d19, 0x1061: 0x5d41, 0x1062: 0x5d69, 0x1063: 0x5d99,
+	0x1064: 0x5dc9, 0x1065: 0x5df9, 0x1066: 0x5e29, 0x1067: 0x5e59, 0x1068: 0x5e89, 0x1069: 0x5eb9,
+	0x106a: 0x5ee9, 0x106b: 0x5f19, 0x106c: 0x5f49, 0x106d: 0x5f79, 0x106e: 0x5fa9, 0x106f: 0x5fd9,
+	0x1070: 0x6009, 0x1071: 0x4045, 0x1072: 0x6039, 0x1073: 0x6051, 0x1074: 0x4065, 0x1075: 0x6069,
+	0x1076: 0x6081, 0x1077: 0x6099, 0x1078: 0x4085, 0x1079: 0x4085, 0x107a: 0x60b1, 0x107b: 0x60c9,
+	0x107c: 0x6101, 0x107d: 0x6139, 0x107e: 0x6171, 0x107f: 0x61a9,
+	// Block 0x42, offset 0x1080
+	0x1080: 0x6211, 0x1081: 0x6229, 0x1082: 0x40a5, 0x1083: 0x6241, 0x1084: 0x6259, 0x1085: 0x6271,
+	0x1086: 0x6289, 0x1087: 0x62a1, 0x1088: 0x40c5, 0x1089: 0x62b9, 0x108a: 0x62e1, 0x108b: 0x62f9,
+	0x108c: 0x40e5, 0x108d: 0x40e5, 0x108e: 0x6311, 0x108f: 0x6329, 0x1090: 0x6341, 0x1091: 0x4105,
+	0x1092: 0x4125, 0x1093: 0x4145, 0x1094: 0x4165, 0x1095: 0x4185, 0x1096: 0x6359, 0x1097: 0x6371,
+	0x1098: 0x6389, 0x1099: 0x63a1, 0x109a: 0x63b9, 0x109b: 0x41a5, 0x109c: 0x63d1, 0x109d: 0x63e9,
+	0x109e: 0x6401, 0x109f: 0x41c5, 0x10a0: 0x41e5, 0x10a1: 0x6419, 0x10a2: 0x4205, 0x10a3: 0x4225,
+	0x10a4: 0x4245, 0x10a5: 0x6431, 0x10a6: 0x4265, 0x10a7: 0x6449, 0x10a8: 0x6479, 0x10a9: 0x6211,
+	0x10aa: 0x4285, 0x10ab: 0x42a5, 0x10ac: 0x42c5, 0x10ad: 0x42e5, 0x10ae: 0x64b1, 0x10af: 0x64f1,
+	0x10b0: 0x6539, 0x10b1: 0x6551, 0x10b2: 0x4305, 0x10b3: 0x6569, 0x10b4: 0x6581, 0x10b5: 0x6599,
+	0x10b6: 0x4325, 0x10b7: 0x65b1, 0x10b8: 0x65c9, 0x10b9: 0x65b1, 0x10ba: 0x65e1, 0x10bb: 0x65f9,
+	0x10bc: 0x4345, 0x10bd: 0x6611, 0x10be: 0x6629, 0x10bf: 0x6611,
+	// Block 0x43, offset 0x10c0
+	0x10c0: 0x4365, 0x10c1: 0x4385, 0x10c2: 0x0040, 0x10c3: 0x6641, 0x10c4: 0x6659, 0x10c5: 0x6671,
+	0x10c6: 0x6689, 0x10c7: 0x0040, 0x10c8: 0x66c1, 0x10c9: 0x66d9, 0x10ca: 0x66f1, 0x10cb: 0x6709,
+	0x10cc: 0x6721, 0x10cd: 0x6739, 0x10ce: 0x6401, 0x10cf: 0x6751, 0x10d0: 0x6769, 0x10d1: 0x6781,
+	0x10d2: 0x43a5, 0x10d3: 0x6799, 0x10d4: 0x6289, 0x10d5: 0x43c5, 0x10d6: 0x43e5, 0x10d7: 0x67b1,
+	0x10d8: 0x0040, 0x10d9: 0x4405, 0x10da: 0x67c9, 0x10db: 0x67e1, 0x10dc: 0x67f9, 0x10dd: 0x6811,
+	0x10de: 0x6829, 0x10df: 0x6859, 0x10e0: 0x6889, 0x10e1: 0x68b1, 0x10e2: 0x68d9, 0x10e3: 0x6901,
+	0x10e4: 0x6929, 0x10e5: 0x6951, 0x10e6: 0x6979, 0x10e7: 0x69a1, 0x10e8: 0x69c9, 0x10e9: 0x69f1,
+	0x10ea: 0x6a21, 0x10eb: 0x6a51, 0x10ec: 0x6a81, 0x10ed: 0x6ab1, 0x10ee: 0x6ae1, 0x10ef: 0x6b11,
+	0x10f0: 0x6b41, 0x10f1: 0x6b71, 0x10f2: 0x6ba1, 0x10f3: 0x6bd1, 0x10f4: 0x6c01, 0x10f5: 0x6c31,
+	0x10f6: 0x6c61, 0x10f7: 0x6c91, 0x10f8: 0x6cc1, 0x10f9: 0x6cf1, 0x10fa: 0x6d21, 0x10fb: 0x6d51,
+	0x10fc: 0x6d81, 0x10fd: 0x6db1, 0x10fe: 0x6de1, 0x10ff: 0x4425,
+	// Block 0x44, offset 0x1100
+	0x1100: 0xe00d, 0x1101: 0x0008, 0x1102: 0xe00d, 0x1103: 0x0008, 0x1104: 0xe00d, 0x1105: 0x0008,
+	0x1106: 0xe00d, 0x1107: 0x0008, 0x1108: 0xe00d, 0x1109: 0x0008, 0x110a: 0xe00d, 0x110b: 0x0008,
+	0x110c: 0xe00d, 0x110d: 0x0008, 0x110e: 0xe00d, 0x110f: 0x0008, 0x1110: 0xe00d, 0x1111: 0x0008,
+	0x1112: 0xe00d, 0x1113: 0x0008, 0x1114: 0xe00d, 0x1115: 0x0008, 0x1116: 0xe00d, 0x1117: 0x0008,
+	0x1118: 0xe00d, 0x1119: 0x0008, 0x111a: 0xe00d, 0x111b: 0x0008, 0x111c: 0xe00d, 0x111d: 0x0008,
+	0x111e: 0xe00d, 0x111f: 0x0008, 0x1120: 0xe00d, 0x1121: 0x0008, 0x1122: 0xe00d, 0x1123: 0x0008,
+	0x1124: 0xe00d, 0x1125: 0x0008, 0x1126: 0xe00d, 0x1127: 0x0008, 0x1128: 0xe00d, 0x1129: 0x0008,
+	0x112a: 0xe00d, 0x112b: 0x0008, 0x112c: 0xe00d, 0x112d: 0x0008, 0x112e: 0x0008, 0x112f: 0x3308,
+	0x1130: 0x3318, 0x1131: 0x3318, 0x1132: 0x3318, 0x1133: 0x0018, 0x1134: 0x3308, 0x1135: 0x3308,
+	0x1136: 0x3308, 0x1137: 0x3308, 0x1138: 0x3308, 0x1139: 0x3308, 0x113a: 0x3308, 0x113b: 0x3308,
+	0x113c: 0x3308, 0x113d: 0x3308, 0x113e: 0x0018, 0x113f: 0x0008,
+	// Block 0x45, offset 0x1140
+	0x1140: 0xe00d, 0x1141: 0x0008, 0x1142: 0xe00d, 0x1143: 0x0008, 0x1144: 0xe00d, 0x1145: 0x0008,
+	0x1146: 0xe00d, 0x1147: 0x0008, 0x1148: 0xe00d, 0x1149: 0x0008, 0x114a: 0xe00d, 0x114b: 0x0008,
+	0x114c: 0xe00d, 0x114d: 0x0008, 0x114e: 0xe00d, 0x114f: 0x0008, 0x1150: 0xe00d, 0x1151: 0x0008,
+	0x1152: 0xe00d, 0x1153: 0x0008, 0x1154: 0xe00d, 0x1155: 0x0008, 0x1156: 0xe00d, 0x1157: 0x0008,
+	0x1158: 0xe00d, 0x1159: 0x0008, 0x115a: 0xe00d, 0x115b: 0x0008, 0x115c: 0x0ea1, 0x115d: 0x6e11,
+	0x115e: 0x3308, 0x115f: 0x3308, 0x1160: 0x0008, 0x1161: 0x0008, 0x1162: 0x0008, 0x1163: 0x0008,
+	0x1164: 0x0008, 0x1165: 0x0008, 0x1166: 0x0008, 0x1167: 0x0008, 0x1168: 0x0008, 0x1169: 0x0008,
+	0x116a: 0x0008, 0x116b: 0x0008, 0x116c: 0x0008, 0x116d: 0x0008, 0x116e: 0x0008, 0x116f: 0x0008,
+	0x1170: 0x0008, 0x1171: 0x0008, 0x1172: 0x0008, 0x1173: 0x0008, 0x1174: 0x0008, 0x1175: 0x0008,
+	0x1176: 0x0008, 0x1177: 0x0008, 0x1178: 0x0008, 0x1179: 0x0008, 0x117a: 0x0008, 0x117b: 0x0008,
+	0x117c: 0x0008, 0x117d: 0x0008, 0x117e: 0x0008, 0x117f: 0x0008,
+	// Block 0x46, offset 0x1180
+	0x1180: 0x0018, 0x1181: 0x0018, 0x1182: 0x0018, 0x1183: 0x0018, 0x1184: 0x0018, 0x1185: 0x0018,
+	0x1186: 0x0018, 0x1187: 0x0018, 0x1188: 0x0018, 0x1189: 0x0018, 0x118a: 0x0018, 0x118b: 0x0018,
+	0x118c: 0x0018, 0x118d: 0x0018, 0x118e: 0x0018, 0x118f: 0x0018, 0x1190: 0x0018, 0x1191: 0x0018,
+	0x1192: 0x0018, 0x1193: 0x0018, 0x1194: 0x0018, 0x1195: 0x0018, 0x1196: 0x0018, 0x1197: 0x0008,
+	0x1198: 0x0008, 0x1199: 0x0008, 0x119a: 0x0008, 0x119b: 0x0008, 0x119c: 0x0008, 0x119d: 0x0008,
+	0x119e: 0x0008, 0x119f: 0x0008, 0x11a0: 0x0018, 0x11a1: 0x0018, 0x11a2: 0xe00d, 0x11a3: 0x0008,
+	0x11a4: 0xe00d, 0x11a5: 0x0008, 0x11a6: 0xe00d, 0x11a7: 0x0008, 0x11a8: 0xe00d, 0x11a9: 0x0008,
+	0x11aa: 0xe00d, 0x11ab: 0x0008, 0x11ac: 0xe00d, 0x11ad: 0x0008, 0x11ae: 0xe00d, 0x11af: 0x0008,
+	0x11b0: 0x0008, 0x11b1: 0x0008, 0x11b2: 0xe00d, 0x11b3: 0x0008, 0x11b4: 0xe00d, 0x11b5: 0x0008,
+	0x11b6: 0xe00d, 0x11b7: 0x0008, 0x11b8: 0xe00d, 0x11b9: 0x0008, 0x11ba: 0xe00d, 0x11bb: 0x0008,
+	0x11bc: 0xe00d, 0x11bd: 0x0008, 0x11be: 0xe00d, 0x11bf: 0x0008,
+	// Block 0x47, offset 0x11c0
+	0x11c0: 0xe00d, 0x11c1: 0x0008, 0x11c2: 0xe00d, 0x11c3: 0x0008, 0x11c4: 0xe00d, 0x11c5: 0x0008,
+	0x11c6: 0xe00d, 0x11c7: 0x0008, 0x11c8: 0xe00d, 0x11c9: 0x0008, 0x11ca: 0xe00d, 0x11cb: 0x0008,
+	0x11cc: 0xe00d, 0x11cd: 0x0008, 0x11ce: 0xe00d, 0x11cf: 0x0008, 0x11d0: 0xe00d, 0x11d1: 0x0008,
+	0x11d2: 0xe00d, 0x11d3: 0x0008, 0x11d4: 0xe00d, 0x11d5: 0x0008, 0x11d6: 0xe00d, 0x11d7: 0x0008,
+	0x11d8: 0xe00d, 0x11d9: 0x0008, 0x11da: 0xe00d, 0x11db: 0x0008, 0x11dc: 0xe00d, 0x11dd: 0x0008,
+	0x11de: 0xe00d, 0x11df: 0x0008, 0x11e0: 0xe00d, 0x11e1: 0x0008, 0x11e2: 0xe00d, 0x11e3: 0x0008,
+	0x11e4: 0xe00d, 0x11e5: 0x0008, 0x11e6: 0xe00d, 0x11e7: 0x0008, 0x11e8: 0xe00d, 0x11e9: 0x0008,
+	0x11ea: 0xe00d, 0x11eb: 0x0008, 0x11ec: 0xe00d, 0x11ed: 0x0008, 0x11ee: 0xe00d, 0x11ef: 0x0008,
+	0x11f0: 0xe0fd, 0x11f1: 0x0008, 0x11f2: 0x0008, 0x11f3: 0x0008, 0x11f4: 0x0008, 0x11f5: 0x0008,
+	0x11f6: 0x0008, 0x11f7: 0x0008, 0x11f8: 0x0008, 0x11f9: 0xe01d, 0x11fa: 0x0008, 0x11fb: 0xe03d,
+	0x11fc: 0x0008, 0x11fd: 0x4445, 0x11fe: 0xe00d, 0x11ff: 0x0008,
+	// Block 0x48, offset 0x1200
+	0x1200: 0xe00d, 0x1201: 0x0008, 0x1202: 0xe00d, 0x1203: 0x0008, 0x1204: 0xe00d, 0x1205: 0x0008,
+	0x1206: 0xe00d, 0x1207: 0x0008, 0x1208: 0x0008, 0x1209: 0x0018, 0x120a: 0x0018, 0x120b: 0xe03d,
+	0x120c: 0x0008, 0x120d: 0x11d9, 0x120e: 0x0008, 0x120f: 0x0008, 0x1210: 0xe00d, 0x1211: 0x0008,
+	0x1212: 0xe00d, 0x1213: 0x0008, 0x1214: 0x0008, 0x1215: 0x0008, 0x1216: 0xe00d, 0x1217: 0x0008,
+	0x1218: 0xe00d, 0x1219: 0x0008, 0x121a: 0xe00d, 0x121b: 0x0008, 0x121c: 0xe00d, 0x121d: 0x0008,
+	0x121e: 0xe00d, 0x121f: 0x0008, 0x1220: 0xe00d, 0x1221: 0x0008, 0x1222: 0xe00d, 0x1223: 0x0008,
+	0x1224: 0xe00d, 0x1225: 0x0008, 0x1226: 0xe00d, 0x1227: 0x0008, 0x1228: 0xe00d, 0x1229: 0x0008,
+	0x122a: 0x6e29, 0x122b: 0x1029, 0x122c: 0x11c1, 0x122d: 0x6e41, 0x122e: 0x1221, 0x122f: 0x0008,
+	0x1230: 0x6e59, 0x1231: 0x6e71, 0x1232: 0x1239, 0x1233: 0x4465, 0x1234: 0xe00d, 0x1235: 0x0008,
+	0x1236: 0xe00d, 0x1237: 0x0008, 0x1238: 0xe00d, 0x1239: 0x0008, 0x123a: 0xe00d, 0x123b: 0x0008,
+	0x123c: 0xe00d, 0x123d: 0x0008, 0x123e: 0xe00d, 0x123f: 0x0008,
+	// Block 0x49, offset 0x1240
+	0x1240: 0x650d, 0x1241: 0x652d, 0x1242: 0x654d, 0x1243: 0x656d, 0x1244: 0x658d, 0x1245: 0x65ad,
+	0x1246: 0x65cd, 0x1247: 0x65ed, 0x1248: 0x660d, 0x1249: 0x662d, 0x124a: 0x664d, 0x124b: 0x666d,
+	0x124c: 0x668d, 0x124d: 0x66ad, 0x124e: 0x0008, 0x124f: 0x0008, 0x1250: 0x66cd, 0x1251: 0x0008,
+	0x1252: 0x66ed, 0x1253: 0x0008, 0x1254: 0x0008, 0x1255: 0x670d, 0x1256: 0x672d, 0x1257: 0x674d,
+	0x1258: 0x676d, 0x1259: 0x678d, 0x125a: 0x67ad, 0x125b: 0x67cd, 0x125c: 0x67ed, 0x125d: 0x680d,
+	0x125e: 0x682d, 0x125f: 0x0008, 0x1260: 0x684d, 0x1261: 0x0008, 0x1262: 0x686d, 0x1263: 0x0008,
+	0x1264: 0x0008, 0x1265: 0x688d, 0x1266: 0x68ad, 0x1267: 0x0008, 0x1268: 0x0008, 0x1269: 0x0008,
+	0x126a: 0x68cd, 0x126b: 0x68ed, 0x126c: 0x690d, 0x126d: 0x692d, 0x126e: 0x694d, 0x126f: 0x696d,
+	0x1270: 0x698d, 0x1271: 0x69ad, 0x1272: 0x69cd, 0x1273: 0x69ed, 0x1274: 0x6a0d, 0x1275: 0x6a2d,
+	0x1276: 0x6a4d, 0x1277: 0x6a6d, 0x1278: 0x6a8d, 0x1279: 0x6aad, 0x127a: 0x6acd, 0x127b: 0x6aed,
+	0x127c: 0x6b0d, 0x127d: 0x6b2d, 0x127e: 0x6b4d, 0x127f: 0x6b6d,
+	// Block 0x4a, offset 0x1280
+	0x1280: 0x7acd, 0x1281: 0x7aed, 0x1282: 0x7b0d, 0x1283: 0x7b2d, 0x1284: 0x7b4d, 0x1285: 0x7b6d,
+	0x1286: 0x7b8d, 0x1287: 0x7bad, 0x1288: 0x7bcd, 0x1289: 0x7bed, 0x128a: 0x7c0d, 0x128b: 0x7c2d,
+	0x128c: 0x7c4d, 0x128d: 0x7c6d, 0x128e: 0x7c8d, 0x128f: 0x6ec9, 0x1290: 0x6ef1, 0x1291: 0x6f19,
+	0x1292: 0x7cad, 0x1293: 0x7ccd, 0x1294: 0x7ced, 0x1295: 0x6f41, 0x1296: 0x6f69, 0x1297: 0x6f91,
+	0x1298: 0x7d0d, 0x1299: 0x7d2d, 0x129a: 0x0040, 0x129b: 0x0040, 0x129c: 0x0040, 0x129d: 0x0040,
+	0x129e: 0x0040, 0x129f: 0x0040, 0x12a0: 0x0040, 0x12a1: 0x0040, 0x12a2: 0x0040, 0x12a3: 0x0040,
+	0x12a4: 0x0040, 0x12a5: 0x0040, 0x12a6: 0x0040, 0x12a7: 0x0040, 0x12a8: 0x0040, 0x12a9: 0x0040,
+	0x12aa: 0x0040, 0x12ab: 0x0040, 0x12ac: 0x0040, 0x12ad: 0x0040, 0x12ae: 0x0040, 0x12af: 0x0040,
+	0x12b0: 0x0040, 0x12b1: 0x0040, 0x12b2: 0x0040, 0x12b3: 0x0040, 0x12b4: 0x0040, 0x12b5: 0x0040,
+	0x12b6: 0x0040, 0x12b7: 0x0040, 0x12b8: 0x0040, 0x12b9: 0x0040, 0x12ba: 0x0040, 0x12bb: 0x0040,
+	0x12bc: 0x0040, 0x12bd: 0x0040, 0x12be: 0x0040, 0x12bf: 0x0040,
+	// Block 0x4b, offset 0x12c0
+	0x12c0: 0x6fb9, 0x12c1: 0x6fd1, 0x12c2: 0x6fe9, 0x12c3: 0x7d4d, 0x12c4: 0x7d6d, 0x12c5: 0x7001,
+	0x12c6: 0x7001, 0x12c7: 0x0040, 0x12c8: 0x0040, 0x12c9: 0x0040, 0x12ca: 0x0040, 0x12cb: 0x0040,
+	0x12cc: 0x0040, 0x12cd: 0x0040, 0x12ce: 0x0040, 0x12cf: 0x0040, 0x12d0: 0x0040, 0x12d1: 0x0040,
+	0x12d2: 0x0040, 0x12d3: 0x7019, 0x12d4: 0x7041, 0x12d5: 0x7069, 0x12d6: 0x7091, 0x12d7: 0x70b9,
+	0x12d8: 0x0040, 0x12d9: 0x0040, 0x12da: 0x0040, 0x12db: 0x0040, 0x12dc: 0x0040, 0x12dd: 0x70e1,
+	0x12de: 0x3308, 0x12df: 0x7109, 0x12e0: 0x7131, 0x12e1: 0x20a9, 0x12e2: 0x20f1, 0x12e3: 0x7149,
+	0x12e4: 0x7161, 0x12e5: 0x7179, 0x12e6: 0x7191, 0x12e7: 0x71a9, 0x12e8: 0x71c1, 0x12e9: 0x1fb2,
+	0x12ea: 0x71d9, 0x12eb: 0x7201, 0x12ec: 0x7229, 0x12ed: 0x7261, 0x12ee: 0x7299, 0x12ef: 0x72c1,
+	0x12f0: 0x72e9, 0x12f1: 0x7311, 0x12f2: 0x7339, 0x12f3: 0x7361, 0x12f4: 0x7389, 0x12f5: 0x73b1,
+	0x12f6: 0x73d9, 0x12f7: 0x0040, 0x12f8: 0x7401, 0x12f9: 0x7429, 0x12fa: 0x7451, 0x12fb: 0x7479,
+	0x12fc: 0x74a1, 0x12fd: 0x0040, 0x12fe: 0x74c9, 0x12ff: 0x0040,
+	// Block 0x4c, offset 0x1300
+	0x1300: 0x74f1, 0x1301: 0x7519, 0x1302: 0x0040, 0x1303: 0x7541, 0x1304: 0x7569, 0x1305: 0x0040,
+	0x1306: 0x7591, 0x1307: 0x75b9, 0x1308: 0x75e1, 0x1309: 0x7609, 0x130a: 0x7631, 0x130b: 0x7659,
+	0x130c: 0x7681, 0x130d: 0x76a9, 0x130e: 0x76d1, 0x130f: 0x76f9, 0x1310: 0x7721, 0x1311: 0x7721,
+	0x1312: 0x7739, 0x1313: 0x7739, 0x1314: 0x7739, 0x1315: 0x7739, 0x1316: 0x7751, 0x1317: 0x7751,
+	0x1318: 0x7751, 0x1319: 0x7751, 0x131a: 0x7769, 0x131b: 0x7769, 0x131c: 0x7769, 0x131d: 0x7769,
+	0x131e: 0x7781, 0x131f: 0x7781, 0x1320: 0x7781, 0x1321: 0x7781, 0x1322: 0x7799, 0x1323: 0x7799,
+	0x1324: 0x7799, 0x1325: 0x7799, 0x1326: 0x77b1, 0x1327: 0x77b1, 0x1328: 0x77b1, 0x1329: 0x77b1,
+	0x132a: 0x77c9, 0x132b: 0x77c9, 0x132c: 0x77c9, 0x132d: 0x77c9, 0x132e: 0x77e1, 0x132f: 0x77e1,
+	0x1330: 0x77e1, 0x1331: 0x77e1, 0x1332: 0x77f9, 0x1333: 0x77f9, 0x1334: 0x77f9, 0x1335: 0x77f9,
+	0x1336: 0x7811, 0x1337: 0x7811, 0x1338: 0x7811, 0x1339: 0x7811, 0x133a: 0x7829, 0x133b: 0x7829,
+	0x133c: 0x7829, 0x133d: 0x7829, 0x133e: 0x7841, 0x133f: 0x7841,
+	// Block 0x4d, offset 0x1340
+	0x1340: 0x7841, 0x1341: 0x7841, 0x1342: 0x7859, 0x1343: 0x7859, 0x1344: 0x7871, 0x1345: 0x7871,
+	0x1346: 0x7889, 0x1347: 0x7889, 0x1348: 0x78a1, 0x1349: 0x78a1, 0x134a: 0x78b9, 0x134b: 0x78b9,
+	0x134c: 0x78d1, 0x134d: 0x78d1, 0x134e: 0x78e9, 0x134f: 0x78e9, 0x1350: 0x78e9, 0x1351: 0x78e9,
+	0x1352: 0x7901, 0x1353: 0x7901, 0x1354: 0x7901, 0x1355: 0x7901, 0x1356: 0x7919, 0x1357: 0x7919,
+	0x1358: 0x7919, 0x1359: 0x7919, 0x135a: 0x7931, 0x135b: 0x7931, 0x135c: 0x7931, 0x135d: 0x7931,
+	0x135e: 0x7949, 0x135f: 0x7949, 0x1360: 0x7961, 0x1361: 0x7961, 0x1362: 0x7961, 0x1363: 0x7961,
+	0x1364: 0x7979, 0x1365: 0x7979, 0x1366: 0x7991, 0x1367: 0x7991, 0x1368: 0x7991, 0x1369: 0x7991,
+	0x136a: 0x79a9, 0x136b: 0x79a9, 0x136c: 0x79a9, 0x136d: 0x79a9, 0x136e: 0x79c1, 0x136f: 0x79c1,
+	0x1370: 0x79d9, 0x1371: 0x79d9, 0x1372: 0x0818, 0x1373: 0x0818, 0x1374: 0x0818, 0x1375: 0x0818,
+	0x1376: 0x0818, 0x1377: 0x0818, 0x1378: 0x0818, 0x1379: 0x0818, 0x137a: 0x0818, 0x137b: 0x0818,
+	0x137c: 0x0818, 0x137d: 0x0818, 0x137e: 0x0818, 0x137f: 0x0818,
+	// Block 0x4e, offset 0x1380
+	0x1380: 0x0818, 0x1381: 0x0818, 0x1382: 0x0040, 0x1383: 0x0040, 0x1384: 0x0040, 0x1385: 0x0040,
+	0x1386: 0x0040, 0x1387: 0x0040, 0x1388: 0x0040, 0x1389: 0x0040, 0x138a: 0x0040, 0x138b: 0x0040,
+	0x138c: 0x0040, 0x138d: 0x0040, 0x138e: 0x0040, 0x138f: 0x0040, 0x1390: 0x0040, 0x1391: 0x0040,
+	0x1392: 0x0040, 0x1393: 0x79f1, 0x1394: 0x79f1, 0x1395: 0x79f1, 0x1396: 0x79f1, 0x1397: 0x7a09,
+	0x1398: 0x7a09, 0x1399: 0x7a21, 0x139a: 0x7a21, 0x139b: 0x7a39, 0x139c: 0x7a39, 0x139d: 0x0479,
+	0x139e: 0x7a51, 0x139f: 0x7a51, 0x13a0: 0x7a69, 0x13a1: 0x7a69, 0x13a2: 0x7a81, 0x13a3: 0x7a81,
+	0x13a4: 0x7a99, 0x13a5: 0x7a99, 0x13a6: 0x7a99, 0x13a7: 0x7a99, 0x13a8: 0x7ab1, 0x13a9: 0x7ab1,
+	0x13aa: 0x7ac9, 0x13ab: 0x7ac9, 0x13ac: 0x7af1, 0x13ad: 0x7af1, 0x13ae: 0x7b19, 0x13af: 0x7b19,
+	0x13b0: 0x7b41, 0x13b1: 0x7b41, 0x13b2: 0x7b69, 0x13b3: 0x7b69, 0x13b4: 0x7b91, 0x13b5: 0x7b91,
+	0x13b6: 0x7bb9, 0x13b7: 0x7bb9, 0x13b8: 0x7bb9, 0x13b9: 0x7be1, 0x13ba: 0x7be1, 0x13bb: 0x7be1,
+	0x13bc: 0x7c09, 0x13bd: 0x7c09, 0x13be: 0x7c09, 0x13bf: 0x7c09,
+	// Block 0x4f, offset 0x13c0
+	0x13c0: 0x85f9, 0x13c1: 0x8621, 0x13c2: 0x8649, 0x13c3: 0x8671, 0x13c4: 0x8699, 0x13c5: 0x86c1,
+	0x13c6: 0x86e9, 0x13c7: 0x8711, 0x13c8: 0x8739, 0x13c9: 0x8761, 0x13ca: 0x8789, 0x13cb: 0x87b1,
+	0x13cc: 0x87d9, 0x13cd: 0x8801, 0x13ce: 0x8829, 0x13cf: 0x8851, 0x13d0: 0x8879, 0x13d1: 0x88a1,
+	0x13d2: 0x88c9, 0x13d3: 0x88f1, 0x13d4: 0x8919, 0x13d5: 0x8941, 0x13d6: 0x8969, 0x13d7: 0x8991,
+	0x13d8: 0x89b9, 0x13d9: 0x89e1, 0x13da: 0x8a09, 0x13db: 0x8a31, 0x13dc: 0x8a59, 0x13dd: 0x8a81,
+	0x13de: 0x8aaa, 0x13df: 0x8ada, 0x13e0: 0x8b0a, 0x13e1: 0x8b3a, 0x13e2: 0x8b6a, 0x13e3: 0x8b9a,
+	0x13e4: 0x8bc9, 0x13e5: 0x8bf1, 0x13e6: 0x7c71, 0x13e7: 0x8c19, 0x13e8: 0x7be1, 0x13e9: 0x7c99,
+	0x13ea: 0x8c41, 0x13eb: 0x8c69, 0x13ec: 0x7d39, 0x13ed: 0x8c91, 0x13ee: 0x7d61, 0x13ef: 0x7d89,
+	0x13f0: 0x8cb9, 0x13f1: 0x8ce1, 0x13f2: 0x7e29, 0x13f3: 0x8d09, 0x13f4: 0x7e51, 0x13f5: 0x7e79,
+	0x13f6: 0x8d31, 0x13f7: 0x8d59, 0x13f8: 0x7ec9, 0x13f9: 0x8d81, 0x13fa: 0x7ef1, 0x13fb: 0x7f19,
+	0x13fc: 0x83a1, 0x13fd: 0x83c9, 0x13fe: 0x8441, 0x13ff: 0x8469,
+	// Block 0x50, offset 0x1400
+	0x1400: 0x8491, 0x1401: 0x8531, 0x1402: 0x8559, 0x1403: 0x8581, 0x1404: 0x85a9, 0x1405: 0x8649,
+	0x1406: 0x8671, 0x1407: 0x8699, 0x1408: 0x8da9, 0x1409: 0x8739, 0x140a: 0x8dd1, 0x140b: 0x8df9,
+	0x140c: 0x8829, 0x140d: 0x8e21, 0x140e: 0x8851, 0x140f: 0x8879, 0x1410: 0x8a81, 0x1411: 0x8e49,
+	0x1412: 0x8e71, 0x1413: 0x89b9, 0x1414: 0x8e99, 0x1415: 0x89e1, 0x1416: 0x8a09, 0x1417: 0x7c21,
+	0x1418: 0x7c49, 0x1419: 0x8ec1, 0x141a: 0x7c71, 0x141b: 0x8ee9, 0x141c: 0x7cc1, 0x141d: 0x7ce9,
+	0x141e: 0x7d11, 0x141f: 0x7d39, 0x1420: 0x8f11, 0x1421: 0x7db1, 0x1422: 0x7dd9, 0x1423: 0x7e01,
+	0x1424: 0x7e29, 0x1425: 0x8f39, 0x1426: 0x7ec9, 0x1427: 0x7f41, 0x1428: 0x7f69, 0x1429: 0x7f91,
+	0x142a: 0x7fb9, 0x142b: 0x7fe1, 0x142c: 0x8031, 0x142d: 0x8059, 0x142e: 0x8081, 0x142f: 0x80a9,
+	0x1430: 0x80d1, 0x1431: 0x80f9, 0x1432: 0x8f61, 0x1433: 0x8121, 0x1434: 0x8149, 0x1435: 0x8171,
+	0x1436: 0x8199, 0x1437: 0x81c1, 0x1438: 0x81e9, 0x1439: 0x8239, 0x143a: 0x8261, 0x143b: 0x8289,
+	0x143c: 0x82b1, 0x143d: 0x82d9, 0x143e: 0x8301, 0x143f: 0x8329,
+	// Block 0x51, offset 0x1440
+	0x1440: 0x8351, 0x1441: 0x8379, 0x1442: 0x83f1, 0x1443: 0x8419, 0x1444: 0x84b9, 0x1445: 0x84e1,
+	0x1446: 0x8509, 0x1447: 0x8531, 0x1448: 0x8559, 0x1449: 0x85d1, 0x144a: 0x85f9, 0x144b: 0x8621,
+	0x144c: 0x8649, 0x144d: 0x8f89, 0x144e: 0x86c1, 0x144f: 0x86e9, 0x1450: 0x8711, 0x1451: 0x8739,
+	0x1452: 0x87b1, 0x1453: 0x87d9, 0x1454: 0x8801, 0x1455: 0x8829, 0x1456: 0x8fb1, 0x1457: 0x88a1,
+	0x1458: 0x88c9, 0x1459: 0x8fd9, 0x145a: 0x8941, 0x145b: 0x8969, 0x145c: 0x8991, 0x145d: 0x89b9,
+	0x145e: 0x9001, 0x145f: 0x7c71, 0x1460: 0x8ee9, 0x1461: 0x7d39, 0x1462: 0x8f11, 0x1463: 0x7e29,
+	0x1464: 0x8f39, 0x1465: 0x7ec9, 0x1466: 0x9029, 0x1467: 0x80d1, 0x1468: 0x9051, 0x1469: 0x9079,
+	0x146a: 0x90a1, 0x146b: 0x8531, 0x146c: 0x8559, 0x146d: 0x8649, 0x146e: 0x8829, 0x146f: 0x8fb1,
+	0x1470: 0x89b9, 0x1471: 0x9001, 0x1472: 0x90c9, 0x1473: 0x9101, 0x1474: 0x9139, 0x1475: 0x9171,
+	0x1476: 0x9199, 0x1477: 0x91c1, 0x1478: 0x91e9, 0x1479: 0x9211, 0x147a: 0x9239, 0x147b: 0x9261,
+	0x147c: 0x9289, 0x147d: 0x92b1, 0x147e: 0x92d9, 0x147f: 0x9301,
+	// Block 0x52, offset 0x1480
+	0x1480: 0x9329, 0x1481: 0x9351, 0x1482: 0x9379, 0x1483: 0x93a1, 0x1484: 0x93c9, 0x1485: 0x93f1,
+	0x1486: 0x9419, 0x1487: 0x9441, 0x1488: 0x9469, 0x1489: 0x9491, 0x148a: 0x94b9, 0x148b: 0x94e1,
+	0x148c: 0x9079, 0x148d: 0x9509, 0x148e: 0x9531, 0x148f: 0x9559, 0x1490: 0x9581, 0x1491: 0x9171,
+	0x1492: 0x9199, 0x1493: 0x91c1, 0x1494: 0x91e9, 0x1495: 0x9211, 0x1496: 0x9239, 0x1497: 0x9261,
+	0x1498: 0x9289, 0x1499: 0x92b1, 0x149a: 0x92d9, 0x149b: 0x9301, 0x149c: 0x9329, 0x149d: 0x9351,
+	0x149e: 0x9379, 0x149f: 0x93a1, 0x14a0: 0x93c9, 0x14a1: 0x93f1, 0x14a2: 0x9419, 0x14a3: 0x9441,
+	0x14a4: 0x9469, 0x14a5: 0x9491, 0x14a6: 0x94b9, 0x14a7: 0x94e1, 0x14a8: 0x9079, 0x14a9: 0x9509,
+	0x14aa: 0x9531, 0x14ab: 0x9559, 0x14ac: 0x9581, 0x14ad: 0x9491, 0x14ae: 0x94b9, 0x14af: 0x94e1,
+	0x14b0: 0x9079, 0x14b1: 0x9051, 0x14b2: 0x90a1, 0x14b3: 0x8211, 0x14b4: 0x8059, 0x14b5: 0x8081,
+	0x14b6: 0x80a9, 0x14b7: 0x9491, 0x14b8: 0x94b9, 0x14b9: 0x94e1, 0x14ba: 0x8211, 0x14bb: 0x8239,
+	0x14bc: 0x95a9, 0x14bd: 0x95a9, 0x14be: 0x0018, 0x14bf: 0x0018,
+	// Block 0x53, offset 0x14c0
+	0x14c0: 0x0040, 0x14c1: 0x0040, 0x14c2: 0x0040, 0x14c3: 0x0040, 0x14c4: 0x0040, 0x14c5: 0x0040,
+	0x14c6: 0x0040, 0x14c7: 0x0040, 0x14c8: 0x0040, 0x14c9: 0x0040, 0x14ca: 0x0040, 0x14cb: 0x0040,
+	0x14cc: 0x0040, 0x14cd: 0x0040, 0x14ce: 0x0040, 0x14cf: 0x0040, 0x14d0: 0x95d1, 0x14d1: 0x9609,
+	0x14d2: 0x9609, 0x14d3: 0x9641, 0x14d4: 0x9679, 0x14d5: 0x96b1, 0x14d6: 0x96e9, 0x14d7: 0x9721,
+	0x14d8: 0x9759, 0x14d9: 0x9759, 0x14da: 0x9791, 0x14db: 0x97c9, 0x14dc: 0x9801, 0x14dd: 0x9839,
+	0x14de: 0x9871, 0x14df: 0x98a9, 0x14e0: 0x98a9, 0x14e1: 0x98e1, 0x14e2: 0x9919, 0x14e3: 0x9919,
+	0x14e4: 0x9951, 0x14e5: 0x9951, 0x14e6: 0x9989, 0x14e7: 0x99c1, 0x14e8: 0x99c1, 0x14e9: 0x99f9,
+	0x14ea: 0x9a31, 0x14eb: 0x9a31, 0x14ec: 0x9a69, 0x14ed: 0x9a69, 0x14ee: 0x9aa1, 0x14ef: 0x9ad9,
+	0x14f0: 0x9ad9, 0x14f1: 0x9b11, 0x14f2: 0x9b11, 0x14f3: 0x9b49, 0x14f4: 0x9b81, 0x14f5: 0x9bb9,
+	0x14f6: 0x9bf1, 0x14f7: 0x9bf1, 0x14f8: 0x9c29, 0x14f9: 0x9c61, 0x14fa: 0x9c99, 0x14fb: 0x9cd1,
+	0x14fc: 0x9d09, 0x14fd: 0x9d09, 0x14fe: 0x9d41, 0x14ff: 0x9d79,
+	// Block 0x54, offset 0x1500
+	0x1500: 0xa949, 0x1501: 0xa981, 0x1502: 0xa9b9, 0x1503: 0xa8a1, 0x1504: 0x9bb9, 0x1505: 0x9989,
+	0x1506: 0xa9f1, 0x1507: 0xaa29, 0x1508: 0x0040, 0x1509: 0x0040, 0x150a: 0x0040, 0x150b: 0x0040,
+	0x150c: 0x0040, 0x150d: 0x0040, 0x150e: 0x0040, 0x150f: 0x0040, 0x1510: 0x0040, 0x1511: 0x0040,
+	0x1512: 0x0040, 0x1513: 0x0040, 0x1514: 0x0040, 0x1515: 0x0040, 0x1516: 0x0040, 0x1517: 0x0040,
+	0x1518: 0x0040, 0x1519: 0x0040, 0x151a: 0x0040, 0x151b: 0x0040, 0x151c: 0x0040, 0x151d: 0x0040,
+	0x151e: 0x0040, 0x151f: 0x0040, 0x1520: 0x0040, 0x1521: 0x0040, 0x1522: 0x0040, 0x1523: 0x0040,
+	0x1524: 0x0040, 0x1525: 0x0040, 0x1526: 0x0040, 0x1527: 0x0040, 0x1528: 0x0040, 0x1529: 0x0040,
+	0x152a: 0x0040, 0x152b: 0x0040, 0x152c: 0x0040, 0x152d: 0x0040, 0x152e: 0x0040, 0x152f: 0x0040,
+	0x1530: 0xaa61, 0x1531: 0xaa99, 0x1532: 0xaad1, 0x1533: 0xab19, 0x1534: 0xab61, 0x1535: 0xaba9,
+	0x1536: 0xabf1, 0x1537: 0xac39, 0x1538: 0xac81, 0x1539: 0xacc9, 0x153a: 0xad02, 0x153b: 0xae12,
+	0x153c: 0xae91, 0x153d: 0x0018, 0x153e: 0x0040, 0x153f: 0x0040,
+	// Block 0x55, offset 0x1540
+	0x1540: 0x33c0, 0x1541: 0x33c0, 0x1542: 0x33c0, 0x1543: 0x33c0, 0x1544: 0x33c0, 0x1545: 0x33c0,
+	0x1546: 0x33c0, 0x1547: 0x33c0, 0x1548: 0x33c0, 0x1549: 0x33c0, 0x154a: 0x33c0, 0x154b: 0x33c0,
+	0x154c: 0x33c0, 0x154d: 0x33c0, 0x154e: 0x33c0, 0x154f: 0x33c0, 0x1550: 0xaeda, 0x1551: 0x7d8d,
+	0x1552: 0x0040, 0x1553: 0xaeea, 0x1554: 0x03c2, 0x1555: 0xaefa, 0x1556: 0xaf0a, 0x1557: 0x7dad,
+	0x1558: 0x7dcd, 0x1559: 0x0040, 0x155a: 0x0040, 0x155b: 0x0040, 0x155c: 0x0040, 0x155d: 0x0040,
+	0x155e: 0x0040, 0x155f: 0x0040, 0x1560: 0x3308, 0x1561: 0x3308, 0x1562: 0x3308, 0x1563: 0x3308,
+	0x1564: 0x3308, 0x1565: 0x3308, 0x1566: 0x3308, 0x1567: 0x3308, 0x1568: 0x3308, 0x1569: 0x3308,
+	0x156a: 0x3308, 0x156b: 0x3308, 0x156c: 0x3308, 0x156d: 0x3308, 0x156e: 0x3308, 0x156f: 0x3308,
+	0x1570: 0x0040, 0x1571: 0x7ded, 0x1572: 0x7e0d, 0x1573: 0xaf1a, 0x1574: 0xaf1a, 0x1575: 0x1fd2,
+	0x1576: 0x1fe2, 0x1577: 0xaf2a, 0x1578: 0xaf3a, 0x1579: 0x7e2d, 0x157a: 0x7e4d, 0x157b: 0x7e6d,
+	0x157c: 0x7e2d, 0x157d: 0x7e8d, 0x157e: 0x7ead, 0x157f: 0x7e8d,
+	// Block 0x56, offset 0x1580
+	0x1580: 0x7ecd, 0x1581: 0x7eed, 0x1582: 0x7f0d, 0x1583: 0x7eed, 0x1584: 0x7f2d, 0x1585: 0x0018,
+	0x1586: 0x0018, 0x1587: 0xaf4a, 0x1588: 0xaf5a, 0x1589: 0x7f4e, 0x158a: 0x7f6e, 0x158b: 0x7f8e,
+	0x158c: 0x7fae, 0x158d: 0xaf1a, 0x158e: 0xaf1a, 0x158f: 0xaf1a, 0x1590: 0xaeda, 0x1591: 0x7fcd,
+	0x1592: 0x0040, 0x1593: 0x0040, 0x1594: 0x03c2, 0x1595: 0xaeea, 0x1596: 0xaf0a, 0x1597: 0xaefa,
+	0x1598: 0x7fed, 0x1599: 0x1fd2, 0x159a: 0x1fe2, 0x159b: 0xaf2a, 0x159c: 0xaf3a, 0x159d: 0x7ecd,
+	0x159e: 0x7f2d, 0x159f: 0xaf6a, 0x15a0: 0xaf7a, 0x15a1: 0xaf8a, 0x15a2: 0x1fb2, 0x15a3: 0xaf99,
+	0x15a4: 0xafaa, 0x15a5: 0xafba, 0x15a6: 0x1fc2, 0x15a7: 0x0040, 0x15a8: 0xafca, 0x15a9: 0xafda,
+	0x15aa: 0xafea, 0x15ab: 0xaffa, 0x15ac: 0x0040, 0x15ad: 0x0040, 0x15ae: 0x0040, 0x15af: 0x0040,
+	0x15b0: 0x800e, 0x15b1: 0xb009, 0x15b2: 0x802e, 0x15b3: 0x0808, 0x15b4: 0x804e, 0x15b5: 0x0040,
+	0x15b6: 0x806e, 0x15b7: 0xb031, 0x15b8: 0x808e, 0x15b9: 0xb059, 0x15ba: 0x80ae, 0x15bb: 0xb081,
+	0x15bc: 0x80ce, 0x15bd: 0xb0a9, 0x15be: 0x80ee, 0x15bf: 0xb0d1,
+	// Block 0x57, offset 0x15c0
+	0x15c0: 0xb0f9, 0x15c1: 0xb111, 0x15c2: 0xb111, 0x15c3: 0xb129, 0x15c4: 0xb129, 0x15c5: 0xb141,
+	0x15c6: 0xb141, 0x15c7: 0xb159, 0x15c8: 0xb159, 0x15c9: 0xb171, 0x15ca: 0xb171, 0x15cb: 0xb171,
+	0x15cc: 0xb171, 0x15cd: 0xb189, 0x15ce: 0xb189, 0x15cf: 0xb1a1, 0x15d0: 0xb1a1, 0x15d1: 0xb1a1,
+	0x15d2: 0xb1a1, 0x15d3: 0xb1b9, 0x15d4: 0xb1b9, 0x15d5: 0xb1d1, 0x15d6: 0xb1d1, 0x15d7: 0xb1d1,
+	0x15d8: 0xb1d1, 0x15d9: 0xb1e9, 0x15da: 0xb1e9, 0x15db: 0xb1e9, 0x15dc: 0xb1e9, 0x15dd: 0xb201,
+	0x15de: 0xb201, 0x15df: 0xb201, 0x15e0: 0xb201, 0x15e1: 0xb219, 0x15e2: 0xb219, 0x15e3: 0xb219,
+	0x15e4: 0xb219, 0x15e5: 0xb231, 0x15e6: 0xb231, 0x15e7: 0xb231, 0x15e8: 0xb231, 0x15e9: 0xb249,
+	0x15ea: 0xb249, 0x15eb: 0xb261, 0x15ec: 0xb261, 0x15ed: 0xb279, 0x15ee: 0xb279, 0x15ef: 0xb291,
+	0x15f0: 0xb291, 0x15f1: 0xb2a9, 0x15f2: 0xb2a9, 0x15f3: 0xb2a9, 0x15f4: 0xb2a9, 0x15f5: 0xb2c1,
+	0x15f6: 0xb2c1, 0x15f7: 0xb2c1, 0x15f8: 0xb2c1, 0x15f9: 0xb2d9, 0x15fa: 0xb2d9, 0x15fb: 0xb2d9,
+	0x15fc: 0xb2d9, 0x15fd: 0xb2f1, 0x15fe: 0xb2f1, 0x15ff: 0xb2f1,
+	// Block 0x58, offset 0x1600
+	0x1600: 0xb2f1, 0x1601: 0xb309, 0x1602: 0xb309, 0x1603: 0xb309, 0x1604: 0xb309, 0x1605: 0xb321,
+	0x1606: 0xb321, 0x1607: 0xb321, 0x1608: 0xb321, 0x1609: 0xb339, 0x160a: 0xb339, 0x160b: 0xb339,
+	0x160c: 0xb339, 0x160d: 0xb351, 0x160e: 0xb351, 0x160f: 0xb351, 0x1610: 0xb351, 0x1611: 0xb369,
+	0x1612: 0xb369, 0x1613: 0xb369, 0x1614: 0xb369, 0x1615: 0xb381, 0x1616: 0xb381, 0x1617: 0xb381,
+	0x1618: 0xb381, 0x1619: 0xb399, 0x161a: 0xb399, 0x161b: 0xb399, 0x161c: 0xb399, 0x161d: 0xb3b1,
+	0x161e: 0xb3b1, 0x161f: 0xb3b1, 0x1620: 0xb3b1, 0x1621: 0xb3c9, 0x1622: 0xb3c9, 0x1623: 0xb3c9,
+	0x1624: 0xb3c9, 0x1625: 0xb3e1, 0x1626: 0xb3e1, 0x1627: 0xb3e1, 0x1628: 0xb3e1, 0x1629: 0xb3f9,
+	0x162a: 0xb3f9, 0x162b: 0xb3f9, 0x162c: 0xb3f9, 0x162d: 0xb411, 0x162e: 0xb411, 0x162f: 0x7ab1,
+	0x1630: 0x7ab1, 0x1631: 0xb429, 0x1632: 0xb429, 0x1633: 0xb429, 0x1634: 0xb429, 0x1635: 0xb441,
+	0x1636: 0xb441, 0x1637: 0xb469, 0x1638: 0xb469, 0x1639: 0xb491, 0x163a: 0xb491, 0x163b: 0xb4b9,
+	0x163c: 0xb4b9, 0x163d: 0x0040, 0x163e: 0x0040, 0x163f: 0x03c0,
+	// Block 0x59, offset 0x1640
+	0x1640: 0x0040, 0x1641: 0xaefa, 0x1642: 0xb4e2, 0x1643: 0xaf6a, 0x1644: 0xafda, 0x1645: 0xafea,
+	0x1646: 0xaf7a, 0x1647: 0xb4f2, 0x1648: 0x1fd2, 0x1649: 0x1fe2, 0x164a: 0xaf8a, 0x164b: 0x1fb2,
+	0x164c: 0xaeda, 0x164d: 0xaf99, 0x164e: 0x29d1, 0x164f: 0xb502, 0x1650: 0x1f41, 0x1651: 0x00c9,
+	0x1652: 0x0069, 0x1653: 0x0079, 0x1654: 0x1f51, 0x1655: 0x1f61, 0x1656: 0x1f71, 0x1657: 0x1f81,
+	0x1658: 0x1f91, 0x1659: 0x1fa1, 0x165a: 0xaeea, 0x165b: 0x03c2, 0x165c: 0xafaa, 0x165d: 0x1fc2,
+	0x165e: 0xafba, 0x165f: 0xaf0a, 0x1660: 0xaffa, 0x1661: 0x0039, 0x1662: 0x0ee9, 0x1663: 0x1159,
+	0x1664: 0x0ef9, 0x1665: 0x0f09, 0x1666: 0x1199, 0x1667: 0x0f31, 0x1668: 0x0249, 0x1669: 0x0f41,
+	0x166a: 0x0259, 0x166b: 0x0f51, 0x166c: 0x0359, 0x166d: 0x0f61, 0x166e: 0x0f71, 0x166f: 0x00d9,
+	0x1670: 0x0f99, 0x1671: 0x2039, 0x1672: 0x0269, 0x1673: 0x01d9, 0x1674: 0x0fa9, 0x1675: 0x0fb9,
+	0x1676: 0x1089, 0x1677: 0x0279, 0x1678: 0x0369, 0x1679: 0x0289, 0x167a: 0x13d1, 0x167b: 0xaf4a,
+	0x167c: 0xafca, 0x167d: 0xaf5a, 0x167e: 0xb512, 0x167f: 0xaf1a,
+	// Block 0x5a, offset 0x1680
+	0x1680: 0x1caa, 0x1681: 0x0039, 0x1682: 0x0ee9, 0x1683: 0x1159, 0x1684: 0x0ef9, 0x1685: 0x0f09,
+	0x1686: 0x1199, 0x1687: 0x0f31, 0x1688: 0x0249, 0x1689: 0x0f41, 0x168a: 0x0259, 0x168b: 0x0f51,
+	0x168c: 0x0359, 0x168d: 0x0f61, 0x168e: 0x0f71, 0x168f: 0x00d9, 0x1690: 0x0f99, 0x1691: 0x2039,
+	0x1692: 0x0269, 0x1693: 0x01d9, 0x1694: 0x0fa9, 0x1695: 0x0fb9, 0x1696: 0x1089, 0x1697: 0x0279,
+	0x1698: 0x0369, 0x1699: 0x0289, 0x169a: 0x13d1, 0x169b: 0xaf2a, 0x169c: 0xb522, 0x169d: 0xaf3a,
+	0x169e: 0xb532, 0x169f: 0x810d, 0x16a0: 0x812d, 0x16a1: 0x29d1, 0x16a2: 0x814d, 0x16a3: 0x814d,
+	0x16a4: 0x816d, 0x16a5: 0x818d, 0x16a6: 0x81ad, 0x16a7: 0x81cd, 0x16a8: 0x81ed, 0x16a9: 0x820d,
+	0x16aa: 0x822d, 0x16ab: 0x824d, 0x16ac: 0x826d, 0x16ad: 0x828d, 0x16ae: 0x82ad, 0x16af: 0x82cd,
+	0x16b0: 0x82ed, 0x16b1: 0x830d, 0x16b2: 0x832d, 0x16b3: 0x834d, 0x16b4: 0x836d, 0x16b5: 0x838d,
+	0x16b6: 0x83ad, 0x16b7: 0x83cd, 0x16b8: 0x83ed, 0x16b9: 0x840d, 0x16ba: 0x842d, 0x16bb: 0x844d,
+	0x16bc: 0x81ed, 0x16bd: 0x846d, 0x16be: 0x848d, 0x16bf: 0x824d,
+	// Block 0x5b, offset 0x16c0
+	0x16c0: 0x84ad, 0x16c1: 0x84cd, 0x16c2: 0x84ed, 0x16c3: 0x850d, 0x16c4: 0x852d, 0x16c5: 0x854d,
+	0x16c6: 0x856d, 0x16c7: 0x858d, 0x16c8: 0x850d, 0x16c9: 0x85ad, 0x16ca: 0x850d, 0x16cb: 0x85cd,
+	0x16cc: 0x85cd, 0x16cd: 0x85ed, 0x16ce: 0x85ed, 0x16cf: 0x860d, 0x16d0: 0x854d, 0x16d1: 0x862d,
+	0x16d2: 0x864d, 0x16d3: 0x862d, 0x16d4: 0x866d, 0x16d5: 0x864d, 0x16d6: 0x868d, 0x16d7: 0x868d,
+	0x16d8: 0x86ad, 0x16d9: 0x86ad, 0x16da: 0x86cd, 0x16db: 0x86cd, 0x16dc: 0x864d, 0x16dd: 0x814d,
+	0x16de: 0x86ed, 0x16df: 0x870d, 0x16e0: 0x0040, 0x16e1: 0x872d, 0x16e2: 0x874d, 0x16e3: 0x876d,
+	0x16e4: 0x878d, 0x16e5: 0x876d, 0x16e6: 0x87ad, 0x16e7: 0x87cd, 0x16e8: 0x87ed, 0x16e9: 0x87ed,
+	0x16ea: 0x880d, 0x16eb: 0x880d, 0x16ec: 0x882d, 0x16ed: 0x882d, 0x16ee: 0x880d, 0x16ef: 0x880d,
+	0x16f0: 0x884d, 0x16f1: 0x886d, 0x16f2: 0x888d, 0x16f3: 0x88ad, 0x16f4: 0x88cd, 0x16f5: 0x88ed,
+	0x16f6: 0x88ed, 0x16f7: 0x88ed, 0x16f8: 0x890d, 0x16f9: 0x890d, 0x16fa: 0x890d, 0x16fb: 0x890d,
+	0x16fc: 0x87ed, 0x16fd: 0x87ed, 0x16fe: 0x87ed, 0x16ff: 0x0040,
+	// Block 0x5c, offset 0x1700
+	0x1700: 0x0040, 0x1701: 0x0040, 0x1702: 0x874d, 0x1703: 0x872d, 0x1704: 0x892d, 0x1705: 0x872d,
+	0x1706: 0x874d, 0x1707: 0x872d, 0x1708: 0x0040, 0x1709: 0x0040, 0x170a: 0x894d, 0x170b: 0x874d,
+	0x170c: 0x896d, 0x170d: 0x892d, 0x170e: 0x896d, 0x170f: 0x874d, 0x1710: 0x0040, 0x1711: 0x0040,
+	0x1712: 0x898d, 0x1713: 0x89ad, 0x1714: 0x88ad, 0x1715: 0x896d, 0x1716: 0x892d, 0x1717: 0x896d,
+	0x1718: 0x0040, 0x1719: 0x0040, 0x171a: 0x89cd, 0x171b: 0x89ed, 0x171c: 0x89cd, 0x171d: 0x0040,
+	0x171e: 0x0040, 0x171f: 0x0040, 0x1720: 0xb541, 0x1721: 0xb559, 0x1722: 0xb571, 0x1723: 0x8a0e,
+	0x1724: 0xb589, 0x1725: 0xb5a1, 0x1726: 0x8a2d, 0x1727: 0x0040, 0x1728: 0x8a4d, 0x1729: 0x8a6d,
+	0x172a: 0x8a8d, 0x172b: 0x8a6d, 0x172c: 0x8aad, 0x172d: 0x8acd, 0x172e: 0x8aed, 0x172f: 0x0040,
+	0x1730: 0x0040, 0x1731: 0x0040, 0x1732: 0x0040, 0x1733: 0x0040, 0x1734: 0x0040, 0x1735: 0x0040,
+	0x1736: 0x0040, 0x1737: 0x0040, 0x1738: 0x0040, 0x1739: 0x0340, 0x173a: 0x0340, 0x173b: 0x0340,
+	0x173c: 0x0040, 0x173d: 0x0040, 0x173e: 0x0040, 0x173f: 0x0040,
+	// Block 0x5d, offset 0x1740
+	0x1740: 0x0a08, 0x1741: 0x0a08, 0x1742: 0x0a08, 0x1743: 0x0a08, 0x1744: 0x0a08, 0x1745: 0x0c08,
+	0x1746: 0x0808, 0x1747: 0x0c08, 0x1748: 0x0818, 0x1749: 0x0c08, 0x174a: 0x0c08, 0x174b: 0x0808,
+	0x174c: 0x0808, 0x174d: 0x0908, 0x174e: 0x0c08, 0x174f: 0x0c08, 0x1750: 0x0c08, 0x1751: 0x0c08,
+	0x1752: 0x0c08, 0x1753: 0x0a08, 0x1754: 0x0a08, 0x1755: 0x0a08, 0x1756: 0x0a08, 0x1757: 0x0908,
+	0x1758: 0x0a08, 0x1759: 0x0a08, 0x175a: 0x0a08, 0x175b: 0x0a08, 0x175c: 0x0a08, 0x175d: 0x0c08,
+	0x175e: 0x0a08, 0x175f: 0x0a08, 0x1760: 0x0a08, 0x1761: 0x0c08, 0x1762: 0x0808, 0x1763: 0x0808,
+	0x1764: 0x0c08, 0x1765: 0x3308, 0x1766: 0x3308, 0x1767: 0x0040, 0x1768: 0x0040, 0x1769: 0x0040,
+	0x176a: 0x0040, 0x176b: 0x0a18, 0x176c: 0x0a18, 0x176d: 0x0a18, 0x176e: 0x0a18, 0x176f: 0x0c18,
+	0x1770: 0x0818, 0x1771: 0x0818, 0x1772: 0x0818, 0x1773: 0x0818, 0x1774: 0x0818, 0x1775: 0x0818,
+	0x1776: 0x0818, 0x1777: 0x0040, 0x1778: 0x0040, 0x1779: 0x0040, 0x177a: 0x0040, 0x177b: 0x0040,
+	0x177c: 0x0040, 0x177d: 0x0040, 0x177e: 0x0040, 0x177f: 0x0040,
+	// Block 0x5e, offset 0x1780
+	0x1780: 0x0a08, 0x1781: 0x0c08, 0x1782: 0x0a08, 0x1783: 0x0c08, 0x1784: 0x0c08, 0x1785: 0x0c08,
+	0x1786: 0x0a08, 0x1787: 0x0a08, 0x1788: 0x0a08, 0x1789: 0x0c08, 0x178a: 0x0a08, 0x178b: 0x0a08,
+	0x178c: 0x0c08, 0x178d: 0x0a08, 0x178e: 0x0c08, 0x178f: 0x0c08, 0x1790: 0x0a08, 0x1791: 0x0c08,
+	0x1792: 0x0040, 0x1793: 0x0040, 0x1794: 0x0040, 0x1795: 0x0040, 0x1796: 0x0040, 0x1797: 0x0040,
+	0x1798: 0x0040, 0x1799: 0x0818, 0x179a: 0x0818, 0x179b: 0x0818, 0x179c: 0x0818, 0x179d: 0x0040,
+	0x179e: 0x0040, 0x179f: 0x0040, 0x17a0: 0x0040, 0x17a1: 0x0040, 0x17a2: 0x0040, 0x17a3: 0x0040,
+	0x17a4: 0x0040, 0x17a5: 0x0040, 0x17a6: 0x0040, 0x17a7: 0x0040, 0x17a8: 0x0040, 0x17a9: 0x0c18,
+	0x17aa: 0x0c18, 0x17ab: 0x0c18, 0x17ac: 0x0c18, 0x17ad: 0x0a18, 0x17ae: 0x0a18, 0x17af: 0x0818,
+	0x17b0: 0x0040, 0x17b1: 0x0040, 0x17b2: 0x0040, 0x17b3: 0x0040, 0x17b4: 0x0040, 0x17b5: 0x0040,
+	0x17b6: 0x0040, 0x17b7: 0x0040, 0x17b8: 0x0040, 0x17b9: 0x0040, 0x17ba: 0x0040, 0x17bb: 0x0040,
+	0x17bc: 0x0040, 0x17bd: 0x0040, 0x17be: 0x0040, 0x17bf: 0x0040,
+	// Block 0x5f, offset 0x17c0
+	0x17c0: 0x3308, 0x17c1: 0x3308, 0x17c2: 0x3008, 0x17c3: 0x3008, 0x17c4: 0x0040, 0x17c5: 0x0008,
+	0x17c6: 0x0008, 0x17c7: 0x0008, 0x17c8: 0x0008, 0x17c9: 0x0008, 0x17ca: 0x0008, 0x17cb: 0x0008,
+	0x17cc: 0x0008, 0x17cd: 0x0040, 0x17ce: 0x0040, 0x17cf: 0x0008, 0x17d0: 0x0008, 0x17d1: 0x0040,
+	0x17d2: 0x0040, 0x17d3: 0x0008, 0x17d4: 0x0008, 0x17d5: 0x0008, 0x17d6: 0x0008, 0x17d7: 0x0008,
+	0x17d8: 0x0008, 0x17d9: 0x0008, 0x17da: 0x0008, 0x17db: 0x0008, 0x17dc: 0x0008, 0x17dd: 0x0008,
+	0x17de: 0x0008, 0x17df: 0x0008, 0x17e0: 0x0008, 0x17e1: 0x0008, 0x17e2: 0x0008, 0x17e3: 0x0008,
+	0x17e4: 0x0008, 0x17e5: 0x0008, 0x17e6: 0x0008, 0x17e7: 0x0008, 0x17e8: 0x0008, 0x17e9: 0x0040,
+	0x17ea: 0x0008, 0x17eb: 0x0008, 0x17ec: 0x0008, 0x17ed: 0x0008, 0x17ee: 0x0008, 0x17ef: 0x0008,
+	0x17f0: 0x0008, 0x17f1: 0x0040, 0x17f2: 0x0008, 0x17f3: 0x0008, 0x17f4: 0x0040, 0x17f5: 0x0008,
+	0x17f6: 0x0008, 0x17f7: 0x0008, 0x17f8: 0x0008, 0x17f9: 0x0008, 0x17fa: 0x0040, 0x17fb: 0x3308,
+	0x17fc: 0x3308, 0x17fd: 0x0008, 0x17fe: 0x3008, 0x17ff: 0x3008,
+	// Block 0x60, offset 0x1800
+	0x1800: 0x3308, 0x1801: 0x3008, 0x1802: 0x3008, 0x1803: 0x3008, 0x1804: 0x3008, 0x1805: 0x0040,
+	0x1806: 0x0040, 0x1807: 0x3008, 0x1808: 0x3008, 0x1809: 0x0040, 0x180a: 0x0040, 0x180b: 0x3008,
+	0x180c: 0x3008, 0x180d: 0x3808, 0x180e: 0x0040, 0x180f: 0x0040, 0x1810: 0x0008, 0x1811: 0x0040,
+	0x1812: 0x0040, 0x1813: 0x0040, 0x1814: 0x0040, 0x1815: 0x0040, 0x1816: 0x0040, 0x1817: 0x3008,
+	0x1818: 0x0040, 0x1819: 0x0040, 0x181a: 0x0040, 0x181b: 0x0040, 0x181c: 0x0040, 0x181d: 0x0008,
+	0x181e: 0x0008, 0x181f: 0x0008, 0x1820: 0x0008, 0x1821: 0x0008, 0x1822: 0x3008, 0x1823: 0x3008,
+	0x1824: 0x0040, 0x1825: 0x0040, 0x1826: 0x3308, 0x1827: 0x3308, 0x1828: 0x3308, 0x1829: 0x3308,
+	0x182a: 0x3308, 0x182b: 0x3308, 0x182c: 0x3308, 0x182d: 0x0040, 0x182e: 0x0040, 0x182f: 0x0040,
+	0x1830: 0x3308, 0x1831: 0x3308, 0x1832: 0x3308, 0x1833: 0x3308, 0x1834: 0x3308, 0x1835: 0x0040,
+	0x1836: 0x0040, 0x1837: 0x0040, 0x1838: 0x0040, 0x1839: 0x0040, 0x183a: 0x0040, 0x183b: 0x0040,
+	0x183c: 0x0040, 0x183d: 0x0040, 0x183e: 0x0040, 0x183f: 0x0040,
+	// Block 0x61, offset 0x1840
+	0x1840: 0x0039, 0x1841: 0x0ee9, 0x1842: 0x1159, 0x1843: 0x0ef9, 0x1844: 0x0f09, 0x1845: 0x1199,
+	0x1846: 0x0f31, 0x1847: 0x0249, 0x1848: 0x0f41, 0x1849: 0x0259, 0x184a: 0x0f51, 0x184b: 0x0359,
+	0x184c: 0x0f61, 0x184d: 0x0f71, 0x184e: 0x00d9, 0x184f: 0x0f99, 0x1850: 0x2039, 0x1851: 0x0269,
+	0x1852: 0x01d9, 0x1853: 0x0fa9, 0x1854: 0x0fb9, 0x1855: 0x1089, 0x1856: 0x0279, 0x1857: 0x0369,
+	0x1858: 0x0289, 0x1859: 0x13d1, 0x185a: 0x0039, 0x185b: 0x0ee9, 0x185c: 0x1159, 0x185d: 0x0ef9,
+	0x185e: 0x0f09, 0x185f: 0x1199, 0x1860: 0x0f31, 0x1861: 0x0249, 0x1862: 0x0f41, 0x1863: 0x0259,
+	0x1864: 0x0f51, 0x1865: 0x0359, 0x1866: 0x0f61, 0x1867: 0x0f71, 0x1868: 0x00d9, 0x1869: 0x0f99,
+	0x186a: 0x2039, 0x186b: 0x0269, 0x186c: 0x01d9, 0x186d: 0x0fa9, 0x186e: 0x0fb9, 0x186f: 0x1089,
+	0x1870: 0x0279, 0x1871: 0x0369, 0x1872: 0x0289, 0x1873: 0x13d1, 0x1874: 0x0039, 0x1875: 0x0ee9,
+	0x1876: 0x1159, 0x1877: 0x0ef9, 0x1878: 0x0f09, 0x1879: 0x1199, 0x187a: 0x0f31, 0x187b: 0x0249,
+	0x187c: 0x0f41, 0x187d: 0x0259, 0x187e: 0x0f51, 0x187f: 0x0359,
+	// Block 0x62, offset 0x1880
+	0x1880: 0x0f61, 0x1881: 0x0f71, 0x1882: 0x00d9, 0x1883: 0x0f99, 0x1884: 0x2039, 0x1885: 0x0269,
+	0x1886: 0x01d9, 0x1887: 0x0fa9, 0x1888: 0x0fb9, 0x1889: 0x1089, 0x188a: 0x0279, 0x188b: 0x0369,
+	0x188c: 0x0289, 0x188d: 0x13d1, 0x188e: 0x0039, 0x188f: 0x0ee9, 0x1890: 0x1159, 0x1891: 0x0ef9,
+	0x1892: 0x0f09, 0x1893: 0x1199, 0x1894: 0x0f31, 0x1895: 0x0040, 0x1896: 0x0f41, 0x1897: 0x0259,
+	0x1898: 0x0f51, 0x1899: 0x0359, 0x189a: 0x0f61, 0x189b: 0x0f71, 0x189c: 0x00d9, 0x189d: 0x0f99,
+	0x189e: 0x2039, 0x189f: 0x0269, 0x18a0: 0x01d9, 0x18a1: 0x0fa9, 0x18a2: 0x0fb9, 0x18a3: 0x1089,
+	0x18a4: 0x0279, 0x18a5: 0x0369, 0x18a6: 0x0289, 0x18a7: 0x13d1, 0x18a8: 0x0039, 0x18a9: 0x0ee9,
+	0x18aa: 0x1159, 0x18ab: 0x0ef9, 0x18ac: 0x0f09, 0x18ad: 0x1199, 0x18ae: 0x0f31, 0x18af: 0x0249,
+	0x18b0: 0x0f41, 0x18b1: 0x0259, 0x18b2: 0x0f51, 0x18b3: 0x0359, 0x18b4: 0x0f61, 0x18b5: 0x0f71,
+	0x18b6: 0x00d9, 0x18b7: 0x0f99, 0x18b8: 0x2039, 0x18b9: 0x0269, 0x18ba: 0x01d9, 0x18bb: 0x0fa9,
+	0x18bc: 0x0fb9, 0x18bd: 0x1089, 0x18be: 0x0279, 0x18bf: 0x0369,
+	// Block 0x63, offset 0x18c0
+	0x18c0: 0x0289, 0x18c1: 0x13d1, 0x18c2: 0x0039, 0x18c3: 0x0ee9, 0x18c4: 0x1159, 0x18c5: 0x0ef9,
+	0x18c6: 0x0f09, 0x18c7: 0x1199, 0x18c8: 0x0f31, 0x18c9: 0x0249, 0x18ca: 0x0f41, 0x18cb: 0x0259,
+	0x18cc: 0x0f51, 0x18cd: 0x0359, 0x18ce: 0x0f61, 0x18cf: 0x0f71, 0x18d0: 0x00d9, 0x18d1: 0x0f99,
+	0x18d2: 0x2039, 0x18d3: 0x0269, 0x18d4: 0x01d9, 0x18d5: 0x0fa9, 0x18d6: 0x0fb9, 0x18d7: 0x1089,
+	0x18d8: 0x0279, 0x18d9: 0x0369, 0x18da: 0x0289, 0x18db: 0x13d1, 0x18dc: 0x0039, 0x18dd: 0x0040,
+	0x18de: 0x1159, 0x18df: 0x0ef9, 0x18e0: 0x0040, 0x18e1: 0x0040, 0x18e2: 0x0f31, 0x18e3: 0x0040,
+	0x18e4: 0x0040, 0x18e5: 0x0259, 0x18e6: 0x0f51, 0x18e7: 0x0040, 0x18e8: 0x0040, 0x18e9: 0x0f71,
+	0x18ea: 0x00d9, 0x18eb: 0x0f99, 0x18ec: 0x2039, 0x18ed: 0x0040, 0x18ee: 0x01d9, 0x18ef: 0x0fa9,
+	0x18f0: 0x0fb9, 0x18f1: 0x1089, 0x18f2: 0x0279, 0x18f3: 0x0369, 0x18f4: 0x0289, 0x18f5: 0x13d1,
+	0x18f6: 0x0039, 0x18f7: 0x0ee9, 0x18f8: 0x1159, 0x18f9: 0x0ef9, 0x18fa: 0x0040, 0x18fb: 0x1199,
+	0x18fc: 0x0040, 0x18fd: 0x0249, 0x18fe: 0x0f41, 0x18ff: 0x0259,
+	// Block 0x64, offset 0x1900
+	0x1900: 0x0f51, 0x1901: 0x0359, 0x1902: 0x0f61, 0x1903: 0x0f71, 0x1904: 0x0040, 0x1905: 0x0f99,
+	0x1906: 0x2039, 0x1907: 0x0269, 0x1908: 0x01d9, 0x1909: 0x0fa9, 0x190a: 0x0fb9, 0x190b: 0x1089,
+	0x190c: 0x0279, 0x190d: 0x0369, 0x190e: 0x0289, 0x190f: 0x13d1, 0x1910: 0x0039, 0x1911: 0x0ee9,
+	0x1912: 0x1159, 0x1913: 0x0ef9, 0x1914: 0x0f09, 0x1915: 0x1199, 0x1916: 0x0f31, 0x1917: 0x0249,
+	0x1918: 0x0f41, 0x1919: 0x0259, 0x191a: 0x0f51, 0x191b: 0x0359, 0x191c: 0x0f61, 0x191d: 0x0f71,
+	0x191e: 0x00d9, 0x191f: 0x0f99, 0x1920: 0x2039, 0x1921: 0x0269, 0x1922: 0x01d9, 0x1923: 0x0fa9,
+	0x1924: 0x0fb9, 0x1925: 0x1089, 0x1926: 0x0279, 0x1927: 0x0369, 0x1928: 0x0289, 0x1929: 0x13d1,
+	0x192a: 0x0039, 0x192b: 0x0ee9, 0x192c: 0x1159, 0x192d: 0x0ef9, 0x192e: 0x0f09, 0x192f: 0x1199,
+	0x1930: 0x0f31, 0x1931: 0x0249, 0x1932: 0x0f41, 0x1933: 0x0259, 0x1934: 0x0f51, 0x1935: 0x0359,
+	0x1936: 0x0f61, 0x1937: 0x0f71, 0x1938: 0x00d9, 0x1939: 0x0f99, 0x193a: 0x2039, 0x193b: 0x0269,
+	0x193c: 0x01d9, 0x193d: 0x0fa9, 0x193e: 0x0fb9, 0x193f: 0x1089,
+	// Block 0x65, offset 0x1940
+	0x1940: 0x0279, 0x1941: 0x0369, 0x1942: 0x0289, 0x1943: 0x13d1, 0x1944: 0x0039, 0x1945: 0x0ee9,
+	0x1946: 0x0040, 0x1947: 0x0ef9, 0x1948: 0x0f09, 0x1949: 0x1199, 0x194a: 0x0f31, 0x194b: 0x0040,
+	0x194c: 0x0040, 0x194d: 0x0259, 0x194e: 0x0f51, 0x194f: 0x0359, 0x1950: 0x0f61, 0x1951: 0x0f71,
+	0x1952: 0x00d9, 0x1953: 0x0f99, 0x1954: 0x2039, 0x1955: 0x0040, 0x1956: 0x01d9, 0x1957: 0x0fa9,
+	0x1958: 0x0fb9, 0x1959: 0x1089, 0x195a: 0x0279, 0x195b: 0x0369, 0x195c: 0x0289, 0x195d: 0x0040,
+	0x195e: 0x0039, 0x195f: 0x0ee9, 0x1960: 0x1159, 0x1961: 0x0ef9, 0x1962: 0x0f09, 0x1963: 0x1199,
+	0x1964: 0x0f31, 0x1965: 0x0249, 0x1966: 0x0f41, 0x1967: 0x0259, 0x1968: 0x0f51, 0x1969: 0x0359,
+	0x196a: 0x0f61, 0x196b: 0x0f71, 0x196c: 0x00d9, 0x196d: 0x0f99, 0x196e: 0x2039, 0x196f: 0x0269,
+	0x1970: 0x01d9, 0x1971: 0x0fa9, 0x1972: 0x0fb9, 0x1973: 0x1089, 0x1974: 0x0279, 0x1975: 0x0369,
+	0x1976: 0x0289, 0x1977: 0x13d1, 0x1978: 0x0039, 0x1979: 0x0ee9, 0x197a: 0x0040, 0x197b: 0x0ef9,
+	0x197c: 0x0f09, 0x197d: 0x1199, 0x197e: 0x0f31, 0x197f: 0x0040,
+	// Block 0x66, offset 0x1980
+	0x1980: 0x0f41, 0x1981: 0x0259, 0x1982: 0x0f51, 0x1983: 0x0359, 0x1984: 0x0f61, 0x1985: 0x0040,
+	0x1986: 0x00d9, 0x1987: 0x0040, 0x1988: 0x0040, 0x1989: 0x0040, 0x198a: 0x01d9, 0x198b: 0x0fa9,
+	0x198c: 0x0fb9, 0x198d: 0x1089, 0x198e: 0x0279, 0x198f: 0x0369, 0x1990: 0x0289, 0x1991: 0x0040,
+	0x1992: 0x0039, 0x1993: 0x0ee9, 0x1994: 0x1159, 0x1995: 0x0ef9, 0x1996: 0x0f09, 0x1997: 0x1199,
+	0x1998: 0x0f31, 0x1999: 0x0249, 0x199a: 0x0f41, 0x199b: 0x0259, 0x199c: 0x0f51, 0x199d: 0x0359,
+	0x199e: 0x0f61, 0x199f: 0x0f71, 0x19a0: 0x00d9, 0x19a1: 0x0f99, 0x19a2: 0x2039, 0x19a3: 0x0269,
+	0x19a4: 0x01d9, 0x19a5: 0x0fa9, 0x19a6: 0x0fb9, 0x19a7: 0x1089, 0x19a8: 0x0279, 0x19a9: 0x0369,
+	0x19aa: 0x0289, 0x19ab: 0x13d1, 0x19ac: 0x0039, 0x19ad: 0x0ee9, 0x19ae: 0x1159, 0x19af: 0x0ef9,
+	0x19b0: 0x0f09, 0x19b1: 0x1199, 0x19b2: 0x0f31, 0x19b3: 0x0249, 0x19b4: 0x0f41, 0x19b5: 0x0259,
+	0x19b6: 0x0f51, 0x19b7: 0x0359, 0x19b8: 0x0f61, 0x19b9: 0x0f71, 0x19ba: 0x00d9, 0x19bb: 0x0f99,
+	0x19bc: 0x2039, 0x19bd: 0x0269, 0x19be: 0x01d9, 0x19bf: 0x0fa9,
+	// Block 0x67, offset 0x19c0
+	0x19c0: 0x0fb9, 0x19c1: 0x1089, 0x19c2: 0x0279, 0x19c3: 0x0369, 0x19c4: 0x0289, 0x19c5: 0x13d1,
+	0x19c6: 0x0039, 0x19c7: 0x0ee9, 0x19c8: 0x1159, 0x19c9: 0x0ef9, 0x19ca: 0x0f09, 0x19cb: 0x1199,
+	0x19cc: 0x0f31, 0x19cd: 0x0249, 0x19ce: 0x0f41, 0x19cf: 0x0259, 0x19d0: 0x0f51, 0x19d1: 0x0359,
+	0x19d2: 0x0f61, 0x19d3: 0x0f71, 0x19d4: 0x00d9, 0x19d5: 0x0f99, 0x19d6: 0x2039, 0x19d7: 0x0269,
+	0x19d8: 0x01d9, 0x19d9: 0x0fa9, 0x19da: 0x0fb9, 0x19db: 0x1089, 0x19dc: 0x0279, 0x19dd: 0x0369,
+	0x19de: 0x0289, 0x19df: 0x13d1, 0x19e0: 0x0039, 0x19e1: 0x0ee9, 0x19e2: 0x1159, 0x19e3: 0x0ef9,
+	0x19e4: 0x0f09, 0x19e5: 0x1199, 0x19e6: 0x0f31, 0x19e7: 0x0249, 0x19e8: 0x0f41, 0x19e9: 0x0259,
+	0x19ea: 0x0f51, 0x19eb: 0x0359, 0x19ec: 0x0f61, 0x19ed: 0x0f71, 0x19ee: 0x00d9, 0x19ef: 0x0f99,
+	0x19f0: 0x2039, 0x19f1: 0x0269, 0x19f2: 0x01d9, 0x19f3: 0x0fa9, 0x19f4: 0x0fb9, 0x19f5: 0x1089,
+	0x19f6: 0x0279, 0x19f7: 0x0369, 0x19f8: 0x0289, 0x19f9: 0x13d1, 0x19fa: 0x0039, 0x19fb: 0x0ee9,
+	0x19fc: 0x1159, 0x19fd: 0x0ef9, 0x19fe: 0x0f09, 0x19ff: 0x1199,
+	// Block 0x68, offset 0x1a00
+	0x1a00: 0x0f31, 0x1a01: 0x0249, 0x1a02: 0x0f41, 0x1a03: 0x0259, 0x1a04: 0x0f51, 0x1a05: 0x0359,
+	0x1a06: 0x0f61, 0x1a07: 0x0f71, 0x1a08: 0x00d9, 0x1a09: 0x0f99, 0x1a0a: 0x2039, 0x1a0b: 0x0269,
+	0x1a0c: 0x01d9, 0x1a0d: 0x0fa9, 0x1a0e: 0x0fb9, 0x1a0f: 0x1089, 0x1a10: 0x0279, 0x1a11: 0x0369,
+	0x1a12: 0x0289, 0x1a13: 0x13d1, 0x1a14: 0x0039, 0x1a15: 0x0ee9, 0x1a16: 0x1159, 0x1a17: 0x0ef9,
+	0x1a18: 0x0f09, 0x1a19: 0x1199, 0x1a1a: 0x0f31, 0x1a1b: 0x0249, 0x1a1c: 0x0f41, 0x1a1d: 0x0259,
+	0x1a1e: 0x0f51, 0x1a1f: 0x0359, 0x1a20: 0x0f61, 0x1a21: 0x0f71, 0x1a22: 0x00d9, 0x1a23: 0x0f99,
+	0x1a24: 0x2039, 0x1a25: 0x0269, 0x1a26: 0x01d9, 0x1a27: 0x0fa9, 0x1a28: 0x0fb9, 0x1a29: 0x1089,
+	0x1a2a: 0x0279, 0x1a2b: 0x0369, 0x1a2c: 0x0289, 0x1a2d: 0x13d1, 0x1a2e: 0x0039, 0x1a2f: 0x0ee9,
+	0x1a30: 0x1159, 0x1a31: 0x0ef9, 0x1a32: 0x0f09, 0x1a33: 0x1199, 0x1a34: 0x0f31, 0x1a35: 0x0249,
+	0x1a36: 0x0f41, 0x1a37: 0x0259, 0x1a38: 0x0f51, 0x1a39: 0x0359, 0x1a3a: 0x0f61, 0x1a3b: 0x0f71,
+	0x1a3c: 0x00d9, 0x1a3d: 0x0f99, 0x1a3e: 0x2039, 0x1a3f: 0x0269,
+	// Block 0x69, offset 0x1a40
+	0x1a40: 0x01d9, 0x1a41: 0x0fa9, 0x1a42: 0x0fb9, 0x1a43: 0x1089, 0x1a44: 0x0279, 0x1a45: 0x0369,
+	0x1a46: 0x0289, 0x1a47: 0x13d1, 0x1a48: 0x0039, 0x1a49: 0x0ee9, 0x1a4a: 0x1159, 0x1a4b: 0x0ef9,
+	0x1a4c: 0x0f09, 0x1a4d: 0x1199, 0x1a4e: 0x0f31, 0x1a4f: 0x0249, 0x1a50: 0x0f41, 0x1a51: 0x0259,
+	0x1a52: 0x0f51, 0x1a53: 0x0359, 0x1a54: 0x0f61, 0x1a55: 0x0f71, 0x1a56: 0x00d9, 0x1a57: 0x0f99,
+	0x1a58: 0x2039, 0x1a59: 0x0269, 0x1a5a: 0x01d9, 0x1a5b: 0x0fa9, 0x1a5c: 0x0fb9, 0x1a5d: 0x1089,
+	0x1a5e: 0x0279, 0x1a5f: 0x0369, 0x1a60: 0x0289, 0x1a61: 0x13d1, 0x1a62: 0x0039, 0x1a63: 0x0ee9,
+	0x1a64: 0x1159, 0x1a65: 0x0ef9, 0x1a66: 0x0f09, 0x1a67: 0x1199, 0x1a68: 0x0f31, 0x1a69: 0x0249,
+	0x1a6a: 0x0f41, 0x1a6b: 0x0259, 0x1a6c: 0x0f51, 0x1a6d: 0x0359, 0x1a6e: 0x0f61, 0x1a6f: 0x0f71,
+	0x1a70: 0x00d9, 0x1a71: 0x0f99, 0x1a72: 0x2039, 0x1a73: 0x0269, 0x1a74: 0x01d9, 0x1a75: 0x0fa9,
+	0x1a76: 0x0fb9, 0x1a77: 0x1089, 0x1a78: 0x0279, 0x1a79: 0x0369, 0x1a7a: 0x0289, 0x1a7b: 0x13d1,
+	0x1a7c: 0x0039, 0x1a7d: 0x0ee9, 0x1a7e: 0x1159, 0x1a7f: 0x0ef9,
+	// Block 0x6a, offset 0x1a80
+	0x1a80: 0x0f09, 0x1a81: 0x1199, 0x1a82: 0x0f31, 0x1a83: 0x0249, 0x1a84: 0x0f41, 0x1a85: 0x0259,
+	0x1a86: 0x0f51, 0x1a87: 0x0359, 0x1a88: 0x0f61, 0x1a89: 0x0f71, 0x1a8a: 0x00d9, 0x1a8b: 0x0f99,
+	0x1a8c: 0x2039, 0x1a8d: 0x0269, 0x1a8e: 0x01d9, 0x1a8f: 0x0fa9, 0x1a90: 0x0fb9, 0x1a91: 0x1089,
+	0x1a92: 0x0279, 0x1a93: 0x0369, 0x1a94: 0x0289, 0x1a95: 0x13d1, 0x1a96: 0x0039, 0x1a97: 0x0ee9,
+	0x1a98: 0x1159, 0x1a99: 0x0ef9, 0x1a9a: 0x0f09, 0x1a9b: 0x1199, 0x1a9c: 0x0f31, 0x1a9d: 0x0249,
+	0x1a9e: 0x0f41, 0x1a9f: 0x0259, 0x1aa0: 0x0f51, 0x1aa1: 0x0359, 0x1aa2: 0x0f61, 0x1aa3: 0x0f71,
+	0x1aa4: 0x00d9, 0x1aa5: 0x0f99, 0x1aa6: 0x2039, 0x1aa7: 0x0269, 0x1aa8: 0x01d9, 0x1aa9: 0x0fa9,
+	0x1aaa: 0x0fb9, 0x1aab: 0x1089, 0x1aac: 0x0279, 0x1aad: 0x0369, 0x1aae: 0x0289, 0x1aaf: 0x13d1,
+	0x1ab0: 0x0039, 0x1ab1: 0x0ee9, 0x1ab2: 0x1159, 0x1ab3: 0x0ef9, 0x1ab4: 0x0f09, 0x1ab5: 0x1199,
+	0x1ab6: 0x0f31, 0x1ab7: 0x0249, 0x1ab8: 0x0f41, 0x1ab9: 0x0259, 0x1aba: 0x0f51, 0x1abb: 0x0359,
+	0x1abc: 0x0f61, 0x1abd: 0x0f71, 0x1abe: 0x00d9, 0x1abf: 0x0f99,
+	// Block 0x6b, offset 0x1ac0
+	0x1ac0: 0x2039, 0x1ac1: 0x0269, 0x1ac2: 0x01d9, 0x1ac3: 0x0fa9, 0x1ac4: 0x0fb9, 0x1ac5: 0x1089,
+	0x1ac6: 0x0279, 0x1ac7: 0x0369, 0x1ac8: 0x0289, 0x1ac9: 0x13d1, 0x1aca: 0x0039, 0x1acb: 0x0ee9,
+	0x1acc: 0x1159, 0x1acd: 0x0ef9, 0x1ace: 0x0f09, 0x1acf: 0x1199, 0x1ad0: 0x0f31, 0x1ad1: 0x0249,
+	0x1ad2: 0x0f41, 0x1ad3: 0x0259, 0x1ad4: 0x0f51, 0x1ad5: 0x0359, 0x1ad6: 0x0f61, 0x1ad7: 0x0f71,
+	0x1ad8: 0x00d9, 0x1ad9: 0x0f99, 0x1ada: 0x2039, 0x1adb: 0x0269, 0x1adc: 0x01d9, 0x1add: 0x0fa9,
+	0x1ade: 0x0fb9, 0x1adf: 0x1089, 0x1ae0: 0x0279, 0x1ae1: 0x0369, 0x1ae2: 0x0289, 0x1ae3: 0x13d1,
+	0x1ae4: 0xba81, 0x1ae5: 0xba99, 0x1ae6: 0x0040, 0x1ae7: 0x0040, 0x1ae8: 0xbab1, 0x1ae9: 0x1099,
+	0x1aea: 0x10b1, 0x1aeb: 0x10c9, 0x1aec: 0xbac9, 0x1aed: 0xbae1, 0x1aee: 0xbaf9, 0x1aef: 0x1429,
+	0x1af0: 0x1a31, 0x1af1: 0xbb11, 0x1af2: 0xbb29, 0x1af3: 0xbb41, 0x1af4: 0xbb59, 0x1af5: 0xbb71,
+	0x1af6: 0xbb89, 0x1af7: 0x2109, 0x1af8: 0x1111, 0x1af9: 0x1429, 0x1afa: 0xbba1, 0x1afb: 0xbbb9,
+	0x1afc: 0xbbd1, 0x1afd: 0x10e1, 0x1afe: 0x10f9, 0x1aff: 0xbbe9,
+	// Block 0x6c, offset 0x1b00
+	0x1b00: 0x2079, 0x1b01: 0xbc01, 0x1b02: 0xbab1, 0x1b03: 0x1099, 0x1b04: 0x10b1, 0x1b05: 0x10c9,
+	0x1b06: 0xbac9, 0x1b07: 0xbae1, 0x1b08: 0xbaf9, 0x1b09: 0x1429, 0x1b0a: 0x1a31, 0x1b0b: 0xbb11,
+	0x1b0c: 0xbb29, 0x1b0d: 0xbb41, 0x1b0e: 0xbb59, 0x1b0f: 0xbb71, 0x1b10: 0xbb89, 0x1b11: 0x2109,
+	0x1b12: 0x1111, 0x1b13: 0xbba1, 0x1b14: 0xbba1, 0x1b15: 0xbbb9, 0x1b16: 0xbbd1, 0x1b17: 0x10e1,
+	0x1b18: 0x10f9, 0x1b19: 0xbbe9, 0x1b1a: 0x2079, 0x1b1b: 0xbc21, 0x1b1c: 0xbac9, 0x1b1d: 0x1429,
+	0x1b1e: 0xbb11, 0x1b1f: 0x10e1, 0x1b20: 0x1111, 0x1b21: 0x2109, 0x1b22: 0xbab1, 0x1b23: 0x1099,
+	0x1b24: 0x10b1, 0x1b25: 0x10c9, 0x1b26: 0xbac9, 0x1b27: 0xbae1, 0x1b28: 0xbaf9, 0x1b29: 0x1429,
+	0x1b2a: 0x1a31, 0x1b2b: 0xbb11, 0x1b2c: 0xbb29, 0x1b2d: 0xbb41, 0x1b2e: 0xbb59, 0x1b2f: 0xbb71,
+	0x1b30: 0xbb89, 0x1b31: 0x2109, 0x1b32: 0x1111, 0x1b33: 0x1429, 0x1b34: 0xbba1, 0x1b35: 0xbbb9,
+	0x1b36: 0xbbd1, 0x1b37: 0x10e1, 0x1b38: 0x10f9, 0x1b39: 0xbbe9, 0x1b3a: 0x2079, 0x1b3b: 0xbc01,
+	0x1b3c: 0xbab1, 0x1b3d: 0x1099, 0x1b3e: 0x10b1, 0x1b3f: 0x10c9,
+	// Block 0x6d, offset 0x1b40
+	0x1b40: 0xbac9, 0x1b41: 0xbae1, 0x1b42: 0xbaf9, 0x1b43: 0x1429, 0x1b44: 0x1a31, 0x1b45: 0xbb11,
+	0x1b46: 0xbb29, 0x1b47: 0xbb41, 0x1b48: 0xbb59, 0x1b49: 0xbb71, 0x1b4a: 0xbb89, 0x1b4b: 0x2109,
+	0x1b4c: 0x1111, 0x1b4d: 0xbba1, 0x1b4e: 0xbba1, 0x1b4f: 0xbbb9, 0x1b50: 0xbbd1, 0x1b51: 0x10e1,
+	0x1b52: 0x10f9, 0x1b53: 0xbbe9, 0x1b54: 0x2079, 0x1b55: 0xbc21, 0x1b56: 0xbac9, 0x1b57: 0x1429,
+	0x1b58: 0xbb11, 0x1b59: 0x10e1, 0x1b5a: 0x1111, 0x1b5b: 0x2109, 0x1b5c: 0xbab1, 0x1b5d: 0x1099,
+	0x1b5e: 0x10b1, 0x1b5f: 0x10c9, 0x1b60: 0xbac9, 0x1b61: 0xbae1, 0x1b62: 0xbaf9, 0x1b63: 0x1429,
+	0x1b64: 0x1a31, 0x1b65: 0xbb11, 0x1b66: 0xbb29, 0x1b67: 0xbb41, 0x1b68: 0xbb59, 0x1b69: 0xbb71,
+	0x1b6a: 0xbb89, 0x1b6b: 0x2109, 0x1b6c: 0x1111, 0x1b6d: 0x1429, 0x1b6e: 0xbba1, 0x1b6f: 0xbbb9,
+	0x1b70: 0xbbd1, 0x1b71: 0x10e1, 0x1b72: 0x10f9, 0x1b73: 0xbbe9, 0x1b74: 0x2079, 0x1b75: 0xbc01,
+	0x1b76: 0xbab1, 0x1b77: 0x1099, 0x1b78: 0x10b1, 0x1b79: 0x10c9, 0x1b7a: 0xbac9, 0x1b7b: 0xbae1,
+	0x1b7c: 0xbaf9, 0x1b7d: 0x1429, 0x1b7e: 0x1a31, 0x1b7f: 0xbb11,
+	// Block 0x6e, offset 0x1b80
+	0x1b80: 0xbb29, 0x1b81: 0xbb41, 0x1b82: 0xbb59, 0x1b83: 0xbb71, 0x1b84: 0xbb89, 0x1b85: 0x2109,
+	0x1b86: 0x1111, 0x1b87: 0xbba1, 0x1b88: 0xbba1, 0x1b89: 0xbbb9, 0x1b8a: 0xbbd1, 0x1b8b: 0x10e1,
+	0x1b8c: 0x10f9, 0x1b8d: 0xbbe9, 0x1b8e: 0x2079, 0x1b8f: 0xbc21, 0x1b90: 0xbac9, 0x1b91: 0x1429,
+	0x1b92: 0xbb11, 0x1b93: 0x10e1, 0x1b94: 0x1111, 0x1b95: 0x2109, 0x1b96: 0xbab1, 0x1b97: 0x1099,
+	0x1b98: 0x10b1, 0x1b99: 0x10c9, 0x1b9a: 0xbac9, 0x1b9b: 0xbae1, 0x1b9c: 0xbaf9, 0x1b9d: 0x1429,
+	0x1b9e: 0x1a31, 0x1b9f: 0xbb11, 0x1ba0: 0xbb29, 0x1ba1: 0xbb41, 0x1ba2: 0xbb59, 0x1ba3: 0xbb71,
+	0x1ba4: 0xbb89, 0x1ba5: 0x2109, 0x1ba6: 0x1111, 0x1ba7: 0x1429, 0x1ba8: 0xbba1, 0x1ba9: 0xbbb9,
+	0x1baa: 0xbbd1, 0x1bab: 0x10e1, 0x1bac: 0x10f9, 0x1bad: 0xbbe9, 0x1bae: 0x2079, 0x1baf: 0xbc01,
+	0x1bb0: 0xbab1, 0x1bb1: 0x1099, 0x1bb2: 0x10b1, 0x1bb3: 0x10c9, 0x1bb4: 0xbac9, 0x1bb5: 0xbae1,
+	0x1bb6: 0xbaf9, 0x1bb7: 0x1429, 0x1bb8: 0x1a31, 0x1bb9: 0xbb11, 0x1bba: 0xbb29, 0x1bbb: 0xbb41,
+	0x1bbc: 0xbb59, 0x1bbd: 0xbb71, 0x1bbe: 0xbb89, 0x1bbf: 0x2109,
+	// Block 0x6f, offset 0x1bc0
+	0x1bc0: 0x1111, 0x1bc1: 0xbba1, 0x1bc2: 0xbba1, 0x1bc3: 0xbbb9, 0x1bc4: 0xbbd1, 0x1bc5: 0x10e1,
+	0x1bc6: 0x10f9, 0x1bc7: 0xbbe9, 0x1bc8: 0x2079, 0x1bc9: 0xbc21, 0x1bca: 0xbac9, 0x1bcb: 0x1429,
+	0x1bcc: 0xbb11, 0x1bcd: 0x10e1, 0x1bce: 0x1111, 0x1bcf: 0x2109, 0x1bd0: 0xbab1, 0x1bd1: 0x1099,
+	0x1bd2: 0x10b1, 0x1bd3: 0x10c9, 0x1bd4: 0xbac9, 0x1bd5: 0xbae1, 0x1bd6: 0xbaf9, 0x1bd7: 0x1429,
+	0x1bd8: 0x1a31, 0x1bd9: 0xbb11, 0x1bda: 0xbb29, 0x1bdb: 0xbb41, 0x1bdc: 0xbb59, 0x1bdd: 0xbb71,
+	0x1bde: 0xbb89, 0x1bdf: 0x2109, 0x1be0: 0x1111, 0x1be1: 0x1429, 0x1be2: 0xbba1, 0x1be3: 0xbbb9,
+	0x1be4: 0xbbd1, 0x1be5: 0x10e1, 0x1be6: 0x10f9, 0x1be7: 0xbbe9, 0x1be8: 0x2079, 0x1be9: 0xbc01,
+	0x1bea: 0xbab1, 0x1beb: 0x1099, 0x1bec: 0x10b1, 0x1bed: 0x10c9, 0x1bee: 0xbac9, 0x1bef: 0xbae1,
+	0x1bf0: 0xbaf9, 0x1bf1: 0x1429, 0x1bf2: 0x1a31, 0x1bf3: 0xbb11, 0x1bf4: 0xbb29, 0x1bf5: 0xbb41,
+	0x1bf6: 0xbb59, 0x1bf7: 0xbb71, 0x1bf8: 0xbb89, 0x1bf9: 0x2109, 0x1bfa: 0x1111, 0x1bfb: 0xbba1,
+	0x1bfc: 0xbba1, 0x1bfd: 0xbbb9, 0x1bfe: 0xbbd1, 0x1bff: 0x10e1,
+	// Block 0x70, offset 0x1c00
+	0x1c00: 0x10f9, 0x1c01: 0xbbe9, 0x1c02: 0x2079, 0x1c03: 0xbc21, 0x1c04: 0xbac9, 0x1c05: 0x1429,
+	0x1c06: 0xbb11, 0x1c07: 0x10e1, 0x1c08: 0x1111, 0x1c09: 0x2109, 0x1c0a: 0xbc41, 0x1c0b: 0xbc41,
+	0x1c0c: 0x0040, 0x1c0d: 0x0040, 0x1c0e: 0x1f41, 0x1c0f: 0x00c9, 0x1c10: 0x0069, 0x1c11: 0x0079,
+	0x1c12: 0x1f51, 0x1c13: 0x1f61, 0x1c14: 0x1f71, 0x1c15: 0x1f81, 0x1c16: 0x1f91, 0x1c17: 0x1fa1,
+	0x1c18: 0x1f41, 0x1c19: 0x00c9, 0x1c1a: 0x0069, 0x1c1b: 0x0079, 0x1c1c: 0x1f51, 0x1c1d: 0x1f61,
+	0x1c1e: 0x1f71, 0x1c1f: 0x1f81, 0x1c20: 0x1f91, 0x1c21: 0x1fa1, 0x1c22: 0x1f41, 0x1c23: 0x00c9,
+	0x1c24: 0x0069, 0x1c25: 0x0079, 0x1c26: 0x1f51, 0x1c27: 0x1f61, 0x1c28: 0x1f71, 0x1c29: 0x1f81,
+	0x1c2a: 0x1f91, 0x1c2b: 0x1fa1, 0x1c2c: 0x1f41, 0x1c2d: 0x00c9, 0x1c2e: 0x0069, 0x1c2f: 0x0079,
+	0x1c30: 0x1f51, 0x1c31: 0x1f61, 0x1c32: 0x1f71, 0x1c33: 0x1f81, 0x1c34: 0x1f91, 0x1c35: 0x1fa1,
+	0x1c36: 0x1f41, 0x1c37: 0x00c9, 0x1c38: 0x0069, 0x1c39: 0x0079, 0x1c3a: 0x1f51, 0x1c3b: 0x1f61,
+	0x1c3c: 0x1f71, 0x1c3d: 0x1f81, 0x1c3e: 0x1f91, 0x1c3f: 0x1fa1,
+	// Block 0x71, offset 0x1c40
+	0x1c40: 0xe115, 0x1c41: 0xe115, 0x1c42: 0xe135, 0x1c43: 0xe135, 0x1c44: 0xe115, 0x1c45: 0xe115,
+	0x1c46: 0xe175, 0x1c47: 0xe175, 0x1c48: 0xe115, 0x1c49: 0xe115, 0x1c4a: 0xe135, 0x1c4b: 0xe135,
+	0x1c4c: 0xe115, 0x1c4d: 0xe115, 0x1c4e: 0xe1f5, 0x1c4f: 0xe1f5, 0x1c50: 0xe115, 0x1c51: 0xe115,
+	0x1c52: 0xe135, 0x1c53: 0xe135, 0x1c54: 0xe115, 0x1c55: 0xe115, 0x1c56: 0xe175, 0x1c57: 0xe175,
+	0x1c58: 0xe115, 0x1c59: 0xe115, 0x1c5a: 0xe135, 0x1c5b: 0xe135, 0x1c5c: 0xe115, 0x1c5d: 0xe115,
+	0x1c5e: 0x8b3d, 0x1c5f: 0x8b3d, 0x1c60: 0x04b5, 0x1c61: 0x04b5, 0x1c62: 0x0a08, 0x1c63: 0x0a08,
+	0x1c64: 0x0a08, 0x1c65: 0x0a08, 0x1c66: 0x0a08, 0x1c67: 0x0a08, 0x1c68: 0x0a08, 0x1c69: 0x0a08,
+	0x1c6a: 0x0a08, 0x1c6b: 0x0a08, 0x1c6c: 0x0a08, 0x1c6d: 0x0a08, 0x1c6e: 0x0a08, 0x1c6f: 0x0a08,
+	0x1c70: 0x0a08, 0x1c71: 0x0a08, 0x1c72: 0x0a08, 0x1c73: 0x0a08, 0x1c74: 0x0a08, 0x1c75: 0x0a08,
+	0x1c76: 0x0a08, 0x1c77: 0x0a08, 0x1c78: 0x0a08, 0x1c79: 0x0a08, 0x1c7a: 0x0a08, 0x1c7b: 0x0a08,
+	0x1c7c: 0x0a08, 0x1c7d: 0x0a08, 0x1c7e: 0x0a08, 0x1c7f: 0x0a08,
+	// Block 0x72, offset 0x1c80
+	0x1c80: 0xb189, 0x1c81: 0xb1a1, 0x1c82: 0xb201, 0x1c83: 0xb249, 0x1c84: 0x0040, 0x1c85: 0xb411,
+	0x1c86: 0xb291, 0x1c87: 0xb219, 0x1c88: 0xb309, 0x1c89: 0xb429, 0x1c8a: 0xb399, 0x1c8b: 0xb3b1,
+	0x1c8c: 0xb3c9, 0x1c8d: 0xb3e1, 0x1c8e: 0xb2a9, 0x1c8f: 0xb339, 0x1c90: 0xb369, 0x1c91: 0xb2d9,
+	0x1c92: 0xb381, 0x1c93: 0xb279, 0x1c94: 0xb2c1, 0x1c95: 0xb1d1, 0x1c96: 0xb1e9, 0x1c97: 0xb231,
+	0x1c98: 0xb261, 0x1c99: 0xb2f1, 0x1c9a: 0xb321, 0x1c9b: 0xb351, 0x1c9c: 0xbc59, 0x1c9d: 0x7949,
+	0x1c9e: 0xbc71, 0x1c9f: 0xbc89, 0x1ca0: 0x0040, 0x1ca1: 0xb1a1, 0x1ca2: 0xb201, 0x1ca3: 0x0040,
+	0x1ca4: 0xb3f9, 0x1ca5: 0x0040, 0x1ca6: 0x0040, 0x1ca7: 0xb219, 0x1ca8: 0x0040, 0x1ca9: 0xb429,
+	0x1caa: 0xb399, 0x1cab: 0xb3b1, 0x1cac: 0xb3c9, 0x1cad: 0xb3e1, 0x1cae: 0xb2a9, 0x1caf: 0xb339,
+	0x1cb0: 0xb369, 0x1cb1: 0xb2d9, 0x1cb2: 0xb381, 0x1cb3: 0x0040, 0x1cb4: 0xb2c1, 0x1cb5: 0xb1d1,
+	0x1cb6: 0xb1e9, 0x1cb7: 0xb231, 0x1cb8: 0x0040, 0x1cb9: 0xb2f1, 0x1cba: 0x0040, 0x1cbb: 0xb351,
+	0x1cbc: 0x0040, 0x1cbd: 0x0040, 0x1cbe: 0x0040, 0x1cbf: 0x0040,
+	// Block 0x73, offset 0x1cc0
+	0x1cc0: 0x0040, 0x1cc1: 0x0040, 0x1cc2: 0xb201, 0x1cc3: 0x0040, 0x1cc4: 0x0040, 0x1cc5: 0x0040,
+	0x1cc6: 0x0040, 0x1cc7: 0xb219, 0x1cc8: 0x0040, 0x1cc9: 0xb429, 0x1cca: 0x0040, 0x1ccb: 0xb3b1,
+	0x1ccc: 0x0040, 0x1ccd: 0xb3e1, 0x1cce: 0xb2a9, 0x1ccf: 0xb339, 0x1cd0: 0x0040, 0x1cd1: 0xb2d9,
+	0x1cd2: 0xb381, 0x1cd3: 0x0040, 0x1cd4: 0xb2c1, 0x1cd5: 0x0040, 0x1cd6: 0x0040, 0x1cd7: 0xb231,
+	0x1cd8: 0x0040, 0x1cd9: 0xb2f1, 0x1cda: 0x0040, 0x1cdb: 0xb351, 0x1cdc: 0x0040, 0x1cdd: 0x7949,
+	0x1cde: 0x0040, 0x1cdf: 0xbc89, 0x1ce0: 0x0040, 0x1ce1: 0xb1a1, 0x1ce2: 0xb201, 0x1ce3: 0x0040,
+	0x1ce4: 0xb3f9, 0x1ce5: 0x0040, 0x1ce6: 0x0040, 0x1ce7: 0xb219, 0x1ce8: 0xb309, 0x1ce9: 0xb429,
+	0x1cea: 0xb399, 0x1ceb: 0x0040, 0x1cec: 0xb3c9, 0x1ced: 0xb3e1, 0x1cee: 0xb2a9, 0x1cef: 0xb339,
+	0x1cf0: 0xb369, 0x1cf1: 0xb2d9, 0x1cf2: 0xb381, 0x1cf3: 0x0040, 0x1cf4: 0xb2c1, 0x1cf5: 0xb1d1,
+	0x1cf6: 0xb1e9, 0x1cf7: 0xb231, 0x1cf8: 0x0040, 0x1cf9: 0xb2f1, 0x1cfa: 0xb321, 0x1cfb: 0xb351,
+	0x1cfc: 0xbc59, 0x1cfd: 0x0040, 0x1cfe: 0xbc71, 0x1cff: 0x0040,
+	// Block 0x74, offset 0x1d00
+	0x1d00: 0xb189, 0x1d01: 0xb1a1, 0x1d02: 0xb201, 0x1d03: 0xb249, 0x1d04: 0xb3f9, 0x1d05: 0xb411,
+	0x1d06: 0xb291, 0x1d07: 0xb219, 0x1d08: 0xb309, 0x1d09: 0xb429, 0x1d0a: 0x0040, 0x1d0b: 0xb3b1,
+	0x1d0c: 0xb3c9, 0x1d0d: 0xb3e1, 0x1d0e: 0xb2a9, 0x1d0f: 0xb339, 0x1d10: 0xb369, 0x1d11: 0xb2d9,
+	0x1d12: 0xb381, 0x1d13: 0xb279, 0x1d14: 0xb2c1, 0x1d15: 0xb1d1, 0x1d16: 0xb1e9, 0x1d17: 0xb231,
+	0x1d18: 0xb261, 0x1d19: 0xb2f1, 0x1d1a: 0xb321, 0x1d1b: 0xb351, 0x1d1c: 0x0040, 0x1d1d: 0x0040,
+	0x1d1e: 0x0040, 0x1d1f: 0x0040, 0x1d20: 0x0040, 0x1d21: 0xb1a1, 0x1d22: 0xb201, 0x1d23: 0xb249,
+	0x1d24: 0x0040, 0x1d25: 0xb411, 0x1d26: 0xb291, 0x1d27: 0xb219, 0x1d28: 0xb309, 0x1d29: 0xb429,
+	0x1d2a: 0x0040, 0x1d2b: 0xb3b1, 0x1d2c: 0xb3c9, 0x1d2d: 0xb3e1, 0x1d2e: 0xb2a9, 0x1d2f: 0xb339,
+	0x1d30: 0xb369, 0x1d31: 0xb2d9, 0x1d32: 0xb381, 0x1d33: 0xb279, 0x1d34: 0xb2c1, 0x1d35: 0xb1d1,
+	0x1d36: 0xb1e9, 0x1d37: 0xb231, 0x1d38: 0xb261, 0x1d39: 0xb2f1, 0x1d3a: 0xb321, 0x1d3b: 0xb351,
+	0x1d3c: 0x0040, 0x1d3d: 0x0040, 0x1d3e: 0x0040, 0x1d3f: 0x0040,
+	// Block 0x75, offset 0x1d40
+	0x1d40: 0x0040, 0x1d41: 0xbca2, 0x1d42: 0xbcba, 0x1d43: 0xbcd2, 0x1d44: 0xbcea, 0x1d45: 0xbd02,
+	0x1d46: 0xbd1a, 0x1d47: 0xbd32, 0x1d48: 0xbd4a, 0x1d49: 0xbd62, 0x1d4a: 0xbd7a, 0x1d4b: 0x0018,
+	0x1d4c: 0x0018, 0x1d4d: 0x0040, 0x1d4e: 0x0040, 0x1d4f: 0x0040, 0x1d50: 0xbd92, 0x1d51: 0xbdb2,
+	0x1d52: 0xbdd2, 0x1d53: 0xbdf2, 0x1d54: 0xbe12, 0x1d55: 0xbe32, 0x1d56: 0xbe52, 0x1d57: 0xbe72,
+	0x1d58: 0xbe92, 0x1d59: 0xbeb2, 0x1d5a: 0xbed2, 0x1d5b: 0xbef2, 0x1d5c: 0xbf12, 0x1d5d: 0xbf32,
+	0x1d5e: 0xbf52, 0x1d5f: 0xbf72, 0x1d60: 0xbf92, 0x1d61: 0xbfb2, 0x1d62: 0xbfd2, 0x1d63: 0xbff2,
+	0x1d64: 0xc012, 0x1d65: 0xc032, 0x1d66: 0xc052, 0x1d67: 0xc072, 0x1d68: 0xc092, 0x1d69: 0xc0b2,
+	0x1d6a: 0xc0d1, 0x1d6b: 0x1159, 0x1d6c: 0x0269, 0x1d6d: 0x6671, 0x1d6e: 0xc111, 0x1d6f: 0x0018,
+	0x1d70: 0x0039, 0x1d71: 0x0ee9, 0x1d72: 0x1159, 0x1d73: 0x0ef9, 0x1d74: 0x0f09, 0x1d75: 0x1199,
+	0x1d76: 0x0f31, 0x1d77: 0x0249, 0x1d78: 0x0f41, 0x1d79: 0x0259, 0x1d7a: 0x0f51, 0x1d7b: 0x0359,
+	0x1d7c: 0x0f61, 0x1d7d: 0x0f71, 0x1d7e: 0x00d9, 0x1d7f: 0x0f99,
+	// Block 0x76, offset 0x1d80
+	0x1d80: 0x2039, 0x1d81: 0x0269, 0x1d82: 0x01d9, 0x1d83: 0x0fa9, 0x1d84: 0x0fb9, 0x1d85: 0x1089,
+	0x1d86: 0x0279, 0x1d87: 0x0369, 0x1d88: 0x0289, 0x1d89: 0x13d1, 0x1d8a: 0xc129, 0x1d8b: 0x65b1,
+	0x1d8c: 0xc141, 0x1d8d: 0x1441, 0x1d8e: 0xc159, 0x1d8f: 0xc179, 0x1d90: 0x0018, 0x1d91: 0x0018,
+	0x1d92: 0x0018, 0x1d93: 0x0018, 0x1d94: 0x0018, 0x1d95: 0x0018, 0x1d96: 0x0018, 0x1d97: 0x0018,
+	0x1d98: 0x0018, 0x1d99: 0x0018, 0x1d9a: 0x0018, 0x1d9b: 0x0018, 0x1d9c: 0x0018, 0x1d9d: 0x0018,
+	0x1d9e: 0x0018, 0x1d9f: 0x0018, 0x1da0: 0x0018, 0x1da1: 0x0018, 0x1da2: 0x0018, 0x1da3: 0x0018,
+	0x1da4: 0x0018, 0x1da5: 0x0018, 0x1da6: 0x0018, 0x1da7: 0x0018, 0x1da8: 0x0018, 0x1da9: 0x0018,
+	0x1daa: 0xc191, 0x1dab: 0xc1a9, 0x1dac: 0xc1c1, 0x1dad: 0x0040, 0x1dae: 0x0040, 0x1daf: 0x0040,
+	0x1db0: 0x0018, 0x1db1: 0x0018, 0x1db2: 0x0018, 0x1db3: 0x0018, 0x1db4: 0x0018, 0x1db5: 0x0018,
+	0x1db6: 0x0018, 0x1db7: 0x0018, 0x1db8: 0x0018, 0x1db9: 0x0018, 0x1dba: 0x0018, 0x1dbb: 0x0018,
+	0x1dbc: 0x0018, 0x1dbd: 0x0018, 0x1dbe: 0x0018, 0x1dbf: 0x0018,
+	// Block 0x77, offset 0x1dc0
+	0x1dc0: 0xc1f1, 0x1dc1: 0xc229, 0x1dc2: 0xc261, 0x1dc3: 0x0040, 0x1dc4: 0x0040, 0x1dc5: 0x0040,
+	0x1dc6: 0x0040, 0x1dc7: 0x0040, 0x1dc8: 0x0040, 0x1dc9: 0x0040, 0x1dca: 0x0040, 0x1dcb: 0x0040,
+	0x1dcc: 0x0040, 0x1dcd: 0x0040, 0x1dce: 0x0040, 0x1dcf: 0x0040, 0x1dd0: 0xc281, 0x1dd1: 0xc2a1,
+	0x1dd2: 0xc2c1, 0x1dd3: 0xc2e1, 0x1dd4: 0xc301, 0x1dd5: 0xc321, 0x1dd6: 0xc341, 0x1dd7: 0xc361,
+	0x1dd8: 0xc381, 0x1dd9: 0xc3a1, 0x1dda: 0xc3c1, 0x1ddb: 0xc3e1, 0x1ddc: 0xc401, 0x1ddd: 0xc421,
+	0x1dde: 0xc441, 0x1ddf: 0xc461, 0x1de0: 0xc481, 0x1de1: 0xc4a1, 0x1de2: 0xc4c1, 0x1de3: 0xc4e1,
+	0x1de4: 0xc501, 0x1de5: 0xc521, 0x1de6: 0xc541, 0x1de7: 0xc561, 0x1de8: 0xc581, 0x1de9: 0xc5a1,
+	0x1dea: 0xc5c1, 0x1deb: 0xc5e1, 0x1dec: 0xc601, 0x1ded: 0xc621, 0x1dee: 0xc641, 0x1def: 0xc661,
+	0x1df0: 0xc681, 0x1df1: 0xc6a1, 0x1df2: 0xc6c1, 0x1df3: 0xc6e1, 0x1df4: 0xc701, 0x1df5: 0xc721,
+	0x1df6: 0xc741, 0x1df7: 0xc761, 0x1df8: 0xc781, 0x1df9: 0xc7a1, 0x1dfa: 0xc7c1, 0x1dfb: 0xc7e1,
+	0x1dfc: 0x0040, 0x1dfd: 0x0040, 0x1dfe: 0x0040, 0x1dff: 0x0040,
+	// Block 0x78, offset 0x1e00
+	0x1e00: 0xcb11, 0x1e01: 0xcb31, 0x1e02: 0xcb51, 0x1e03: 0x8b55, 0x1e04: 0xcb71, 0x1e05: 0xcb91,
+	0x1e06: 0xcbb1, 0x1e07: 0xcbd1, 0x1e08: 0xcbf1, 0x1e09: 0xcc11, 0x1e0a: 0xcc31, 0x1e0b: 0xcc51,
+	0x1e0c: 0xcc71, 0x1e0d: 0x8b75, 0x1e0e: 0xcc91, 0x1e0f: 0xccb1, 0x1e10: 0xccd1, 0x1e11: 0xccf1,
+	0x1e12: 0x8b95, 0x1e13: 0xcd11, 0x1e14: 0xcd31, 0x1e15: 0xc441, 0x1e16: 0x8bb5, 0x1e17: 0xcd51,
+	0x1e18: 0xcd71, 0x1e19: 0xcd91, 0x1e1a: 0xcdb1, 0x1e1b: 0xcdd1, 0x1e1c: 0x8bd5, 0x1e1d: 0xcdf1,
+	0x1e1e: 0xce11, 0x1e1f: 0xce31, 0x1e20: 0xce51, 0x1e21: 0xce71, 0x1e22: 0xc7a1, 0x1e23: 0xce91,
+	0x1e24: 0xceb1, 0x1e25: 0xced1, 0x1e26: 0xcef1, 0x1e27: 0xcf11, 0x1e28: 0xcf31, 0x1e29: 0xcf51,
+	0x1e2a: 0xcf71, 0x1e2b: 0xcf91, 0x1e2c: 0xcfb1, 0x1e2d: 0xcfd1, 0x1e2e: 0xcff1, 0x1e2f: 0xd011,
+	0x1e30: 0xd031, 0x1e31: 0xd051, 0x1e32: 0xd051, 0x1e33: 0xd051, 0x1e34: 0x8bf5, 0x1e35: 0xd071,
+	0x1e36: 0xd091, 0x1e37: 0xd0b1, 0x1e38: 0x8c15, 0x1e39: 0xd0d1, 0x1e3a: 0xd0f1, 0x1e3b: 0xd111,
+	0x1e3c: 0xd131, 0x1e3d: 0xd151, 0x1e3e: 0xd171, 0x1e3f: 0xd191,
+	// Block 0x79, offset 0x1e40
+	0x1e40: 0xd1b1, 0x1e41: 0xd1d1, 0x1e42: 0xd1f1, 0x1e43: 0xd211, 0x1e44: 0xd231, 0x1e45: 0xd251,
+	0x1e46: 0xd251, 0x1e47: 0xd271, 0x1e48: 0xd291, 0x1e49: 0xd2b1, 0x1e4a: 0xd2d1, 0x1e4b: 0xd2f1,
+	0x1e4c: 0xd311, 0x1e4d: 0xd331, 0x1e4e: 0xd351, 0x1e4f: 0xd371, 0x1e50: 0xd391, 0x1e51: 0xd3b1,
+	0x1e52: 0xd3d1, 0x1e53: 0xd3f1, 0x1e54: 0xd411, 0x1e55: 0xd431, 0x1e56: 0xd451, 0x1e57: 0xd471,
+	0x1e58: 0xd491, 0x1e59: 0x8c35, 0x1e5a: 0xd4b1, 0x1e5b: 0xd4d1, 0x1e5c: 0xd4f1, 0x1e5d: 0xc321,
+	0x1e5e: 0xd511, 0x1e5f: 0xd531, 0x1e60: 0x8c55, 0x1e61: 0x8c75, 0x1e62: 0xd551, 0x1e63: 0xd571,
+	0x1e64: 0xd591, 0x1e65: 0xd5b1, 0x1e66: 0xd5d1, 0x1e67: 0xd5f1, 0x1e68: 0x2040, 0x1e69: 0xd611,
+	0x1e6a: 0xd631, 0x1e6b: 0xd631, 0x1e6c: 0x8c95, 0x1e6d: 0xd651, 0x1e6e: 0xd671, 0x1e6f: 0xd691,
+	0x1e70: 0xd6b1, 0x1e71: 0x8cb5, 0x1e72: 0xd6d1, 0x1e73: 0xd6f1, 0x1e74: 0x2040, 0x1e75: 0xd711,
+	0x1e76: 0xd731, 0x1e77: 0xd751, 0x1e78: 0xd771, 0x1e79: 0xd791, 0x1e7a: 0xd7b1, 0x1e7b: 0x8cd5,
+	0x1e7c: 0xd7d1, 0x1e7d: 0x8cf5, 0x1e7e: 0xd7f1, 0x1e7f: 0xd811,
+	// Block 0x7a, offset 0x1e80
+	0x1e80: 0xd831, 0x1e81: 0xd851, 0x1e82: 0xd871, 0x1e83: 0xd891, 0x1e84: 0xd8b1, 0x1e85: 0xd8d1,
+	0x1e86: 0xd8f1, 0x1e87: 0xd911, 0x1e88: 0xd931, 0x1e89: 0x8d15, 0x1e8a: 0xd951, 0x1e8b: 0xd971,
+	0x1e8c: 0xd991, 0x1e8d: 0xd9b1, 0x1e8e: 0xd9d1, 0x1e8f: 0x8d35, 0x1e90: 0xd9f1, 0x1e91: 0x8d55,
+	0x1e92: 0x8d75, 0x1e93: 0xda11, 0x1e94: 0xda31, 0x1e95: 0xda31, 0x1e96: 0xda51, 0x1e97: 0x8d95,
+	0x1e98: 0x8db5, 0x1e99: 0xda71, 0x1e9a: 0xda91, 0x1e9b: 0xdab1, 0x1e9c: 0xdad1, 0x1e9d: 0xdaf1,
+	0x1e9e: 0xdb11, 0x1e9f: 0xdb31, 0x1ea0: 0xdb51, 0x1ea1: 0xdb71, 0x1ea2: 0xdb91, 0x1ea3: 0xdbb1,
+	0x1ea4: 0x8dd5, 0x1ea5: 0xdbd1, 0x1ea6: 0xdbf1, 0x1ea7: 0xdc11, 0x1ea8: 0xdc31, 0x1ea9: 0xdc11,
+	0x1eaa: 0xdc51, 0x1eab: 0xdc71, 0x1eac: 0xdc91, 0x1ead: 0xdcb1, 0x1eae: 0xdcd1, 0x1eaf: 0xdcf1,
+	0x1eb0: 0xdd11, 0x1eb1: 0xdd31, 0x1eb2: 0xdd51, 0x1eb3: 0xdd71, 0x1eb4: 0xdd91, 0x1eb5: 0xddb1,
+	0x1eb6: 0xddd1, 0x1eb7: 0xddf1, 0x1eb8: 0x8df5, 0x1eb9: 0xde11, 0x1eba: 0xde31, 0x1ebb: 0xde51,
+	0x1ebc: 0xde71, 0x1ebd: 0xde91, 0x1ebe: 0x8e15, 0x1ebf: 0xdeb1,
+	// Block 0x7b, offset 0x1ec0
+	0x1ec0: 0xe5b1, 0x1ec1: 0xe5d1, 0x1ec2: 0xe5f1, 0x1ec3: 0xe611, 0x1ec4: 0xe631, 0x1ec5: 0xe651,
+	0x1ec6: 0x8f35, 0x1ec7: 0xe671, 0x1ec8: 0xe691, 0x1ec9: 0xe6b1, 0x1eca: 0xe6d1, 0x1ecb: 0xe6f1,
+	0x1ecc: 0xe711, 0x1ecd: 0x8f55, 0x1ece: 0xe731, 0x1ecf: 0xe751, 0x1ed0: 0x8f75, 0x1ed1: 0x8f95,
+	0x1ed2: 0xe771, 0x1ed3: 0xe791, 0x1ed4: 0xe7b1, 0x1ed5: 0xe7d1, 0x1ed6: 0xe7f1, 0x1ed7: 0xe811,
+	0x1ed8: 0xe831, 0x1ed9: 0xe851, 0x1eda: 0xe871, 0x1edb: 0x8fb5, 0x1edc: 0xe891, 0x1edd: 0x8fd5,
+	0x1ede: 0xe8b1, 0x1edf: 0x2040, 0x1ee0: 0xe8d1, 0x1ee1: 0xe8f1, 0x1ee2: 0xe911, 0x1ee3: 0x8ff5,
+	0x1ee4: 0xe931, 0x1ee5: 0xe951, 0x1ee6: 0x9015, 0x1ee7: 0x9035, 0x1ee8: 0xe971, 0x1ee9: 0xe991,
+	0x1eea: 0xe9b1, 0x1eeb: 0xe9d1, 0x1eec: 0xe9f1, 0x1eed: 0xe9f1, 0x1eee: 0xea11, 0x1eef: 0xea31,
+	0x1ef0: 0xea51, 0x1ef1: 0xea71, 0x1ef2: 0xea91, 0x1ef3: 0xeab1, 0x1ef4: 0xead1, 0x1ef5: 0x9055,
+	0x1ef6: 0xeaf1, 0x1ef7: 0x9075, 0x1ef8: 0xeb11, 0x1ef9: 0x9095, 0x1efa: 0xeb31, 0x1efb: 0x90b5,
+	0x1efc: 0x90d5, 0x1efd: 0x90f5, 0x1efe: 0xeb51, 0x1eff: 0xeb71,
+	// Block 0x7c, offset 0x1f00
+	0x1f00: 0xeb91, 0x1f01: 0x9115, 0x1f02: 0x9135, 0x1f03: 0x9155, 0x1f04: 0x9175, 0x1f05: 0xebb1,
+	0x1f06: 0xebd1, 0x1f07: 0xebd1, 0x1f08: 0xebf1, 0x1f09: 0xec11, 0x1f0a: 0xec31, 0x1f0b: 0xec51,
+	0x1f0c: 0xec71, 0x1f0d: 0x9195, 0x1f0e: 0xec91, 0x1f0f: 0xecb1, 0x1f10: 0xecd1, 0x1f11: 0xecf1,
+	0x1f12: 0x91b5, 0x1f13: 0xed11, 0x1f14: 0x91d5, 0x1f15: 0x91f5, 0x1f16: 0xed31, 0x1f17: 0xed51,
+	0x1f18: 0xed71, 0x1f19: 0xed91, 0x1f1a: 0xedb1, 0x1f1b: 0xedd1, 0x1f1c: 0x9215, 0x1f1d: 0x9235,
+	0x1f1e: 0x9255, 0x1f1f: 0x2040, 0x1f20: 0xedf1, 0x1f21: 0x9275, 0x1f22: 0xee11, 0x1f23: 0xee31,
+	0x1f24: 0xee51, 0x1f25: 0x9295, 0x1f26: 0xee71, 0x1f27: 0xee91, 0x1f28: 0xeeb1, 0x1f29: 0xeed1,
+	0x1f2a: 0xeef1, 0x1f2b: 0x92b5, 0x1f2c: 0xef11, 0x1f2d: 0xef31, 0x1f2e: 0xef51, 0x1f2f: 0xef71,
+	0x1f30: 0xef91, 0x1f31: 0xefb1, 0x1f32: 0x92d5, 0x1f33: 0x92f5, 0x1f34: 0xefd1, 0x1f35: 0x9315,
+	0x1f36: 0xeff1, 0x1f37: 0x9335, 0x1f38: 0xf011, 0x1f39: 0xf031, 0x1f3a: 0xf051, 0x1f3b: 0x9355,
+	0x1f3c: 0x9375, 0x1f3d: 0xf071, 0x1f3e: 0x9395, 0x1f3f: 0xf091,
+	// Block 0x7d, offset 0x1f40
+	0x1f40: 0xf6d1, 0x1f41: 0xf6f1, 0x1f42: 0xf711, 0x1f43: 0xf731, 0x1f44: 0xf751, 0x1f45: 0x9555,
+	0x1f46: 0xf771, 0x1f47: 0xf791, 0x1f48: 0xf7b1, 0x1f49: 0xf7d1, 0x1f4a: 0xf7f1, 0x1f4b: 0x9575,
+	0x1f4c: 0x9595, 0x1f4d: 0xf811, 0x1f4e: 0xf831, 0x1f4f: 0xf851, 0x1f50: 0xf871, 0x1f51: 0xf891,
+	0x1f52: 0xf8b1, 0x1f53: 0x95b5, 0x1f54: 0xf8d1, 0x1f55: 0xf8f1, 0x1f56: 0xf911, 0x1f57: 0xf931,
+	0x1f58: 0x95d5, 0x1f59: 0x95f5, 0x1f5a: 0xf951, 0x1f5b: 0xf971, 0x1f5c: 0xf991, 0x1f5d: 0x9615,
+	0x1f5e: 0xf9b1, 0x1f5f: 0xf9d1, 0x1f60: 0x684d, 0x1f61: 0x9635, 0x1f62: 0xf9f1, 0x1f63: 0xfa11,
+	0x1f64: 0xfa31, 0x1f65: 0x9655, 0x1f66: 0xfa51, 0x1f67: 0xfa71, 0x1f68: 0xfa91, 0x1f69: 0xfab1,
+	0x1f6a: 0xfad1, 0x1f6b: 0xfaf1, 0x1f6c: 0xfb11, 0x1f6d: 0x9675, 0x1f6e: 0xfb31, 0x1f6f: 0xfb51,
+	0x1f70: 0xfb71, 0x1f71: 0x9695, 0x1f72: 0xfb91, 0x1f73: 0xfbb1, 0x1f74: 0xfbd1, 0x1f75: 0xfbf1,
+	0x1f76: 0x7b6d, 0x1f77: 0x96b5, 0x1f78: 0xfc11, 0x1f79: 0xfc31, 0x1f7a: 0xfc51, 0x1f7b: 0x96d5,
+	0x1f7c: 0xfc71, 0x1f7d: 0x96f5, 0x1f7e: 0xfc91, 0x1f7f: 0xfc91,
+	// Block 0x7e, offset 0x1f80
+	0x1f80: 0xfcb1, 0x1f81: 0x9715, 0x1f82: 0xfcd1, 0x1f83: 0xfcf1, 0x1f84: 0xfd11, 0x1f85: 0xfd31,
+	0x1f86: 0xfd51, 0x1f87: 0xfd71, 0x1f88: 0xfd91, 0x1f89: 0x9735, 0x1f8a: 0xfdb1, 0x1f8b: 0xfdd1,
+	0x1f8c: 0xfdf1, 0x1f8d: 0xfe11, 0x1f8e: 0xfe31, 0x1f8f: 0xfe51, 0x1f90: 0x9755, 0x1f91: 0xfe71,
+	0x1f92: 0x9775, 0x1f93: 0x9795, 0x1f94: 0x97b5, 0x1f95: 0xfe91, 0x1f96: 0xfeb1, 0x1f97: 0xfed1,
+	0x1f98: 0xfef1, 0x1f99: 0xff11, 0x1f9a: 0xff31, 0x1f9b: 0xff51, 0x1f9c: 0xff71, 0x1f9d: 0x97d5,
+	0x1f9e: 0x0040, 0x1f9f: 0x0040, 0x1fa0: 0x0040, 0x1fa1: 0x0040, 0x1fa2: 0x0040, 0x1fa3: 0x0040,
+	0x1fa4: 0x0040, 0x1fa5: 0x0040, 0x1fa6: 0x0040, 0x1fa7: 0x0040, 0x1fa8: 0x0040, 0x1fa9: 0x0040,
+	0x1faa: 0x0040, 0x1fab: 0x0040, 0x1fac: 0x0040, 0x1fad: 0x0040, 0x1fae: 0x0040, 0x1faf: 0x0040,
+	0x1fb0: 0x0040, 0x1fb1: 0x0040, 0x1fb2: 0x0040, 0x1fb3: 0x0040, 0x1fb4: 0x0040, 0x1fb5: 0x0040,
+	0x1fb6: 0x0040, 0x1fb7: 0x0040, 0x1fb8: 0x0040, 0x1fb9: 0x0040, 0x1fba: 0x0040, 0x1fbb: 0x0040,
+	0x1fbc: 0x0040, 0x1fbd: 0x0040, 0x1fbe: 0x0040, 0x1fbf: 0x0040,
+}
+
+// idnaIndex: 36 blocks, 2304 entries, 4608 bytes
+// Block 0 is the zero block.
+var idnaIndex = [2304]uint16{
+	// Block 0x0, offset 0x0
+	// Block 0x1, offset 0x40
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc2: 0x01, 0xc3: 0x7d, 0xc4: 0x02, 0xc5: 0x03, 0xc6: 0x04, 0xc7: 0x05,
+	0xc8: 0x06, 0xc9: 0x7e, 0xca: 0x7f, 0xcb: 0x07, 0xcc: 0x80, 0xcd: 0x08, 0xce: 0x09, 0xcf: 0x0a,
+	0xd0: 0x81, 0xd1: 0x0b, 0xd2: 0x0c, 0xd3: 0x0d, 0xd4: 0x0e, 0xd5: 0x82, 0xd6: 0x83, 0xd7: 0x84,
+	0xd8: 0x0f, 0xd9: 0x10, 0xda: 0x85, 0xdb: 0x11, 0xdc: 0x12, 0xdd: 0x86, 0xde: 0x87, 0xdf: 0x88,
+	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05, 0xe4: 0x06, 0xe5: 0x07, 0xe6: 0x07, 0xe7: 0x07,
+	0xe8: 0x07, 0xe9: 0x08, 0xea: 0x09, 0xeb: 0x07, 0xec: 0x07, 0xed: 0x0a, 0xee: 0x0b, 0xef: 0x0c,
+	0xf0: 0x1d, 0xf1: 0x1e, 0xf2: 0x1e, 0xf3: 0x20, 0xf4: 0x21,
+	// Block 0x4, offset 0x100
+	0x120: 0x89, 0x121: 0x13, 0x122: 0x8a, 0x123: 0x8b, 0x124: 0x8c, 0x125: 0x14, 0x126: 0x15, 0x127: 0x16,
+	0x128: 0x17, 0x129: 0x18, 0x12a: 0x19, 0x12b: 0x1a, 0x12c: 0x1b, 0x12d: 0x1c, 0x12e: 0x1d, 0x12f: 0x8d,
+	0x130: 0x8e, 0x131: 0x1e, 0x132: 0x1f, 0x133: 0x20, 0x134: 0x8f, 0x135: 0x21, 0x136: 0x90, 0x137: 0x91,
+	0x138: 0x92, 0x139: 0x93, 0x13a: 0x22, 0x13b: 0x94, 0x13c: 0x95, 0x13d: 0x23, 0x13e: 0x24, 0x13f: 0x96,
+	// Block 0x5, offset 0x140
+	0x140: 0x97, 0x141: 0x98, 0x142: 0x99, 0x143: 0x9a, 0x144: 0x9b, 0x145: 0x9c, 0x146: 0x9d, 0x147: 0x9e,
+	0x148: 0x9f, 0x149: 0xa0, 0x14a: 0xa1, 0x14b: 0xa2, 0x14c: 0xa3, 0x14d: 0xa4, 0x14e: 0xa5, 0x14f: 0xa6,
+	0x150: 0xa7, 0x151: 0x9f, 0x152: 0x9f, 0x153: 0x9f, 0x154: 0x9f, 0x155: 0x9f, 0x156: 0x9f, 0x157: 0x9f,
+	0x158: 0x9f, 0x159: 0xa8, 0x15a: 0xa9, 0x15b: 0xaa, 0x15c: 0xab, 0x15d: 0xac, 0x15e: 0xad, 0x15f: 0xae,
+	0x160: 0xaf, 0x161: 0xb0, 0x162: 0xb1, 0x163: 0xb2, 0x164: 0xb3, 0x165: 0xb4, 0x166: 0xb5, 0x167: 0xb6,
+	0x168: 0xb7, 0x169: 0xb8, 0x16a: 0xb9, 0x16b: 0xba, 0x16c: 0xbb, 0x16d: 0xbc, 0x16e: 0xbd, 0x16f: 0xbe,
+	0x170: 0xbf, 0x171: 0xc0, 0x172: 0xc1, 0x173: 0xc2, 0x174: 0x25, 0x175: 0x26, 0x176: 0x27, 0x177: 0xc3,
+	0x178: 0x28, 0x179: 0x28, 0x17a: 0x29, 0x17b: 0x28, 0x17c: 0xc4, 0x17d: 0x2a, 0x17e: 0x2b, 0x17f: 0x2c,
+	// Block 0x6, offset 0x180
+	0x180: 0x2d, 0x181: 0x2e, 0x182: 0x2f, 0x183: 0xc5, 0x184: 0x30, 0x185: 0x31, 0x186: 0xc6, 0x187: 0x9b,
+	0x188: 0xc7, 0x189: 0xc8, 0x18a: 0x9b, 0x18b: 0x9b, 0x18c: 0xc9, 0x18d: 0x9b, 0x18e: 0x9b, 0x18f: 0x9b,
+	0x190: 0xca, 0x191: 0x32, 0x192: 0x33, 0x193: 0x34, 0x194: 0x9b, 0x195: 0x9b, 0x196: 0x9b, 0x197: 0x9b,
+	0x198: 0x9b, 0x199: 0x9b, 0x19a: 0x9b, 0x19b: 0x9b, 0x19c: 0x9b, 0x19d: 0x9b, 0x19e: 0x9b, 0x19f: 0x9b,
+	0x1a0: 0x9b, 0x1a1: 0x9b, 0x1a2: 0x9b, 0x1a3: 0x9b, 0x1a4: 0x9b, 0x1a5: 0x9b, 0x1a6: 0x9b, 0x1a7: 0x9b,
+	0x1a8: 0xcb, 0x1a9: 0xcc, 0x1aa: 0x9b, 0x1ab: 0xcd, 0x1ac: 0x9b, 0x1ad: 0xce, 0x1ae: 0xcf, 0x1af: 0x9b,
+	0x1b0: 0xd0, 0x1b1: 0x35, 0x1b2: 0x28, 0x1b3: 0x36, 0x1b4: 0xd1, 0x1b5: 0xd2, 0x1b6: 0xd3, 0x1b7: 0xd4,
+	0x1b8: 0xd5, 0x1b9: 0xd6, 0x1ba: 0xd7, 0x1bb: 0xd8, 0x1bc: 0xd9, 0x1bd: 0xda, 0x1be: 0xdb, 0x1bf: 0x37,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x38, 0x1c1: 0xdc, 0x1c2: 0xdd, 0x1c3: 0xde, 0x1c4: 0xdf, 0x1c5: 0x39, 0x1c6: 0x3a, 0x1c7: 0xe0,
+	0x1c8: 0xe1, 0x1c9: 0x3b, 0x1ca: 0x3c, 0x1cb: 0x3d, 0x1cc: 0x3e, 0x1cd: 0x3f, 0x1ce: 0x40, 0x1cf: 0x41,
+	0x1d0: 0x9f, 0x1d1: 0x9f, 0x1d2: 0x9f, 0x1d3: 0x9f, 0x1d4: 0x9f, 0x1d5: 0x9f, 0x1d6: 0x9f, 0x1d7: 0x9f,
+	0x1d8: 0x9f, 0x1d9: 0x9f, 0x1da: 0x9f, 0x1db: 0x9f, 0x1dc: 0x9f, 0x1dd: 0x9f, 0x1de: 0x9f, 0x1df: 0x9f,
+	0x1e0: 0x9f, 0x1e1: 0x9f, 0x1e2: 0x9f, 0x1e3: 0x9f, 0x1e4: 0x9f, 0x1e5: 0x9f, 0x1e6: 0x9f, 0x1e7: 0x9f,
+	0x1e8: 0x9f, 0x1e9: 0x9f, 0x1ea: 0x9f, 0x1eb: 0x9f, 0x1ec: 0x9f, 0x1ed: 0x9f, 0x1ee: 0x9f, 0x1ef: 0x9f,
+	0x1f0: 0x9f, 0x1f1: 0x9f, 0x1f2: 0x9f, 0x1f3: 0x9f, 0x1f4: 0x9f, 0x1f5: 0x9f, 0x1f6: 0x9f, 0x1f7: 0x9f,
+	0x1f8: 0x9f, 0x1f9: 0x9f, 0x1fa: 0x9f, 0x1fb: 0x9f, 0x1fc: 0x9f, 0x1fd: 0x9f, 0x1fe: 0x9f, 0x1ff: 0x9f,
+	// Block 0x8, offset 0x200
+	0x200: 0x9f, 0x201: 0x9f, 0x202: 0x9f, 0x203: 0x9f, 0x204: 0x9f, 0x205: 0x9f, 0x206: 0x9f, 0x207: 0x9f,
+	0x208: 0x9f, 0x209: 0x9f, 0x20a: 0x9f, 0x20b: 0x9f, 0x20c: 0x9f, 0x20d: 0x9f, 0x20e: 0x9f, 0x20f: 0x9f,
+	0x210: 0x9f, 0x211: 0x9f, 0x212: 0x9f, 0x213: 0x9f, 0x214: 0x9f, 0x215: 0x9f, 0x216: 0x9f, 0x217: 0x9f,
+	0x218: 0x9f, 0x219: 0x9f, 0x21a: 0x9f, 0x21b: 0x9f, 0x21c: 0x9f, 0x21d: 0x9f, 0x21e: 0x9f, 0x21f: 0x9f,
+	0x220: 0x9f, 0x221: 0x9f, 0x222: 0x9f, 0x223: 0x9f, 0x224: 0x9f, 0x225: 0x9f, 0x226: 0x9f, 0x227: 0x9f,
+	0x228: 0x9f, 0x229: 0x9f, 0x22a: 0x9f, 0x22b: 0x9f, 0x22c: 0x9f, 0x22d: 0x9f, 0x22e: 0x9f, 0x22f: 0x9f,
+	0x230: 0x9f, 0x231: 0x9f, 0x232: 0x9f, 0x233: 0x9f, 0x234: 0x9f, 0x235: 0x9f, 0x236: 0xb2, 0x237: 0x9b,
+	0x238: 0x9f, 0x239: 0x9f, 0x23a: 0x9f, 0x23b: 0x9f, 0x23c: 0x9f, 0x23d: 0x9f, 0x23e: 0x9f, 0x23f: 0x9f,
+	// Block 0x9, offset 0x240
+	0x240: 0x9f, 0x241: 0x9f, 0x242: 0x9f, 0x243: 0x9f, 0x244: 0x9f, 0x245: 0x9f, 0x246: 0x9f, 0x247: 0x9f,
+	0x248: 0x9f, 0x249: 0x9f, 0x24a: 0x9f, 0x24b: 0x9f, 0x24c: 0x9f, 0x24d: 0x9f, 0x24e: 0x9f, 0x24f: 0x9f,
+	0x250: 0x9f, 0x251: 0x9f, 0x252: 0x9f, 0x253: 0x9f, 0x254: 0x9f, 0x255: 0x9f, 0x256: 0x9f, 0x257: 0x9f,
+	0x258: 0x9f, 0x259: 0x9f, 0x25a: 0x9f, 0x25b: 0x9f, 0x25c: 0x9f, 0x25d: 0x9f, 0x25e: 0x9f, 0x25f: 0x9f,
+	0x260: 0x9f, 0x261: 0x9f, 0x262: 0x9f, 0x263: 0x9f, 0x264: 0x9f, 0x265: 0x9f, 0x266: 0x9f, 0x267: 0x9f,
+	0x268: 0x9f, 0x269: 0x9f, 0x26a: 0x9f, 0x26b: 0x9f, 0x26c: 0x9f, 0x26d: 0x9f, 0x26e: 0x9f, 0x26f: 0x9f,
+	0x270: 0x9f, 0x271: 0x9f, 0x272: 0x9f, 0x273: 0x9f, 0x274: 0x9f, 0x275: 0x9f, 0x276: 0x9f, 0x277: 0x9f,
+	0x278: 0x9f, 0x279: 0x9f, 0x27a: 0x9f, 0x27b: 0x9f, 0x27c: 0x9f, 0x27d: 0x9f, 0x27e: 0x9f, 0x27f: 0x9f,
+	// Block 0xa, offset 0x280
+	0x280: 0x9f, 0x281: 0x9f, 0x282: 0x9f, 0x283: 0x9f, 0x284: 0x9f, 0x285: 0x9f, 0x286: 0x9f, 0x287: 0x9f,
+	0x288: 0x9f, 0x289: 0x9f, 0x28a: 0x9f, 0x28b: 0x9f, 0x28c: 0x9f, 0x28d: 0x9f, 0x28e: 0x9f, 0x28f: 0x9f,
+	0x290: 0x9f, 0x291: 0x9f, 0x292: 0x9f, 0x293: 0x9f, 0x294: 0x9f, 0x295: 0x9f, 0x296: 0x9f, 0x297: 0x9f,
+	0x298: 0x9f, 0x299: 0x9f, 0x29a: 0x9f, 0x29b: 0x9f, 0x29c: 0x9f, 0x29d: 0x9f, 0x29e: 0x9f, 0x29f: 0x9f,
+	0x2a0: 0x9f, 0x2a1: 0x9f, 0x2a2: 0x9f, 0x2a3: 0x9f, 0x2a4: 0x9f, 0x2a5: 0x9f, 0x2a6: 0x9f, 0x2a7: 0x9f,
+	0x2a8: 0x9f, 0x2a9: 0x9f, 0x2aa: 0x9f, 0x2ab: 0x9f, 0x2ac: 0x9f, 0x2ad: 0x9f, 0x2ae: 0x9f, 0x2af: 0x9f,
+	0x2b0: 0x9f, 0x2b1: 0x9f, 0x2b2: 0x9f, 0x2b3: 0x9f, 0x2b4: 0x9f, 0x2b5: 0x9f, 0x2b6: 0x9f, 0x2b7: 0x9f,
+	0x2b8: 0x9f, 0x2b9: 0x9f, 0x2ba: 0x9f, 0x2bb: 0x9f, 0x2bc: 0x9f, 0x2bd: 0x9f, 0x2be: 0x9f, 0x2bf: 0xe2,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0x9f, 0x2c1: 0x9f, 0x2c2: 0x9f, 0x2c3: 0x9f, 0x2c4: 0x9f, 0x2c5: 0x9f, 0x2c6: 0x9f, 0x2c7: 0x9f,
+	0x2c8: 0x9f, 0x2c9: 0x9f, 0x2ca: 0x9f, 0x2cb: 0x9f, 0x2cc: 0x9f, 0x2cd: 0x9f, 0x2ce: 0x9f, 0x2cf: 0x9f,
+	0x2d0: 0x9f, 0x2d1: 0x9f, 0x2d2: 0xe3, 0x2d3: 0xe4, 0x2d4: 0x9f, 0x2d5: 0x9f, 0x2d6: 0x9f, 0x2d7: 0x9f,
+	0x2d8: 0xe5, 0x2d9: 0x42, 0x2da: 0x43, 0x2db: 0xe6, 0x2dc: 0x44, 0x2dd: 0x45, 0x2de: 0x46, 0x2df: 0xe7,
+	0x2e0: 0xe8, 0x2e1: 0xe9, 0x2e2: 0xea, 0x2e3: 0xeb, 0x2e4: 0xec, 0x2e5: 0xed, 0x2e6: 0xee, 0x2e7: 0xef,
+	0x2e8: 0xf0, 0x2e9: 0xf1, 0x2ea: 0xf2, 0x2eb: 0xf3, 0x2ec: 0xf4, 0x2ed: 0xf5, 0x2ee: 0xf6, 0x2ef: 0xf7,
+	0x2f0: 0x9f, 0x2f1: 0x9f, 0x2f2: 0x9f, 0x2f3: 0x9f, 0x2f4: 0x9f, 0x2f5: 0x9f, 0x2f6: 0x9f, 0x2f7: 0x9f,
+	0x2f8: 0x9f, 0x2f9: 0x9f, 0x2fa: 0x9f, 0x2fb: 0x9f, 0x2fc: 0x9f, 0x2fd: 0x9f, 0x2fe: 0x9f, 0x2ff: 0x9f,
+	// Block 0xc, offset 0x300
+	0x300: 0x9f, 0x301: 0x9f, 0x302: 0x9f, 0x303: 0x9f, 0x304: 0x9f, 0x305: 0x9f, 0x306: 0x9f, 0x307: 0x9f,
+	0x308: 0x9f, 0x309: 0x9f, 0x30a: 0x9f, 0x30b: 0x9f, 0x30c: 0x9f, 0x30d: 0x9f, 0x30e: 0x9f, 0x30f: 0x9f,
+	0x310: 0x9f, 0x311: 0x9f, 0x312: 0x9f, 0x313: 0x9f, 0x314: 0x9f, 0x315: 0x9f, 0x316: 0x9f, 0x317: 0x9f,
+	0x318: 0x9f, 0x319: 0x9f, 0x31a: 0x9f, 0x31b: 0x9f, 0x31c: 0x9f, 0x31d: 0x9f, 0x31e: 0xf8, 0x31f: 0xf9,
+	// Block 0xd, offset 0x340
+	0x340: 0xba, 0x341: 0xba, 0x342: 0xba, 0x343: 0xba, 0x344: 0xba, 0x345: 0xba, 0x346: 0xba, 0x347: 0xba,
+	0x348: 0xba, 0x349: 0xba, 0x34a: 0xba, 0x34b: 0xba, 0x34c: 0xba, 0x34d: 0xba, 0x34e: 0xba, 0x34f: 0xba,
+	0x350: 0xba, 0x351: 0xba, 0x352: 0xba, 0x353: 0xba, 0x354: 0xba, 0x355: 0xba, 0x356: 0xba, 0x357: 0xba,
+	0x358: 0xba, 0x359: 0xba, 0x35a: 0xba, 0x35b: 0xba, 0x35c: 0xba, 0x35d: 0xba, 0x35e: 0xba, 0x35f: 0xba,
+	0x360: 0xba, 0x361: 0xba, 0x362: 0xba, 0x363: 0xba, 0x364: 0xba, 0x365: 0xba, 0x366: 0xba, 0x367: 0xba,
+	0x368: 0xba, 0x369: 0xba, 0x36a: 0xba, 0x36b: 0xba, 0x36c: 0xba, 0x36d: 0xba, 0x36e: 0xba, 0x36f: 0xba,
+	0x370: 0xba, 0x371: 0xba, 0x372: 0xba, 0x373: 0xba, 0x374: 0xba, 0x375: 0xba, 0x376: 0xba, 0x377: 0xba,
+	0x378: 0xba, 0x379: 0xba, 0x37a: 0xba, 0x37b: 0xba, 0x37c: 0xba, 0x37d: 0xba, 0x37e: 0xba, 0x37f: 0xba,
+	// Block 0xe, offset 0x380
+	0x380: 0xba, 0x381: 0xba, 0x382: 0xba, 0x383: 0xba, 0x384: 0xba, 0x385: 0xba, 0x386: 0xba, 0x387: 0xba,
+	0x388: 0xba, 0x389: 0xba, 0x38a: 0xba, 0x38b: 0xba, 0x38c: 0xba, 0x38d: 0xba, 0x38e: 0xba, 0x38f: 0xba,
+	0x390: 0xba, 0x391: 0xba, 0x392: 0xba, 0x393: 0xba, 0x394: 0xba, 0x395: 0xba, 0x396: 0xba, 0x397: 0xba,
+	0x398: 0xba, 0x399: 0xba, 0x39a: 0xba, 0x39b: 0xba, 0x39c: 0xba, 0x39d: 0xba, 0x39e: 0xba, 0x39f: 0xba,
+	0x3a0: 0xba, 0x3a1: 0xba, 0x3a2: 0xba, 0x3a3: 0xba, 0x3a4: 0xfa, 0x3a5: 0xfb, 0x3a6: 0xfc, 0x3a7: 0xfd,
+	0x3a8: 0x47, 0x3a9: 0xfe, 0x3aa: 0xff, 0x3ab: 0x48, 0x3ac: 0x49, 0x3ad: 0x4a, 0x3ae: 0x4b, 0x3af: 0x4c,
+	0x3b0: 0x100, 0x3b1: 0x4d, 0x3b2: 0x4e, 0x3b3: 0x4f, 0x3b4: 0x50, 0x3b5: 0x51, 0x3b6: 0x101, 0x3b7: 0x52,
+	0x3b8: 0x53, 0x3b9: 0x54, 0x3ba: 0x55, 0x3bb: 0x56, 0x3bc: 0x57, 0x3bd: 0x58, 0x3be: 0x59, 0x3bf: 0x5a,
+	// Block 0xf, offset 0x3c0
+	0x3c0: 0x102, 0x3c1: 0x103, 0x3c2: 0x9f, 0x3c3: 0x104, 0x3c4: 0x105, 0x3c5: 0x9b, 0x3c6: 0x106, 0x3c7: 0x107,
+	0x3c8: 0xba, 0x3c9: 0xba, 0x3ca: 0x108, 0x3cb: 0x109, 0x3cc: 0x10a, 0x3cd: 0x10b, 0x3ce: 0x10c, 0x3cf: 0x10d,
+	0x3d0: 0x10e, 0x3d1: 0x9f, 0x3d2: 0x10f, 0x3d3: 0x110, 0x3d4: 0x111, 0x3d5: 0x112, 0x3d6: 0xba, 0x3d7: 0xba,
+	0x3d8: 0x9f, 0x3d9: 0x9f, 0x3da: 0x9f, 0x3db: 0x9f, 0x3dc: 0x113, 0x3dd: 0x114, 0x3de: 0xba, 0x3df: 0xba,
+	0x3e0: 0x115, 0x3e1: 0x116, 0x3e2: 0x117, 0x3e3: 0x118, 0x3e4: 0x119, 0x3e5: 0xba, 0x3e6: 0x11a, 0x3e7: 0x11b,
+	0x3e8: 0x11c, 0x3e9: 0x11d, 0x3ea: 0x11e, 0x3eb: 0x5b, 0x3ec: 0x11f, 0x3ed: 0x120, 0x3ee: 0x5c, 0x3ef: 0xba,
+	0x3f0: 0x121, 0x3f1: 0x122, 0x3f2: 0x123, 0x3f3: 0x124, 0x3f4: 0x125, 0x3f5: 0xba, 0x3f6: 0xba, 0x3f7: 0xba,
+	0x3f8: 0xba, 0x3f9: 0x126, 0x3fa: 0xba, 0x3fb: 0xba, 0x3fc: 0x127, 0x3fd: 0x128, 0x3fe: 0xba, 0x3ff: 0x129,
+	// Block 0x10, offset 0x400
+	0x400: 0x12a, 0x401: 0x12b, 0x402: 0x12c, 0x403: 0x12d, 0x404: 0x12e, 0x405: 0x12f, 0x406: 0x130, 0x407: 0x131,
+	0x408: 0x132, 0x409: 0xba, 0x40a: 0x133, 0x40b: 0x134, 0x40c: 0x5d, 0x40d: 0x5e, 0x40e: 0xba, 0x40f: 0xba,
+	0x410: 0x135, 0x411: 0x136, 0x412: 0x137, 0x413: 0x138, 0x414: 0xba, 0x415: 0xba, 0x416: 0x139, 0x417: 0x13a,
+	0x418: 0x13b, 0x419: 0x13c, 0x41a: 0x13d, 0x41b: 0x13e, 0x41c: 0x13f, 0x41d: 0xba, 0x41e: 0xba, 0x41f: 0xba,
+	0x420: 0x140, 0x421: 0xba, 0x422: 0x141, 0x423: 0x142, 0x424: 0xba, 0x425: 0xba, 0x426: 0x143, 0x427: 0x144,
+	0x428: 0x145, 0x429: 0x146, 0x42a: 0x147, 0x42b: 0x148, 0x42c: 0xba, 0x42d: 0xba, 0x42e: 0xba, 0x42f: 0xba,
+	0x430: 0x149, 0x431: 0x14a, 0x432: 0x14b, 0x433: 0xba, 0x434: 0x14c, 0x435: 0x14d, 0x436: 0x14e, 0x437: 0xba,
+	0x438: 0xba, 0x439: 0xba, 0x43a: 0xba, 0x43b: 0x14f, 0x43c: 0xba, 0x43d: 0xba, 0x43e: 0xba, 0x43f: 0x150,
+	// Block 0x11, offset 0x440
+	0x440: 0x9f, 0x441: 0x9f, 0x442: 0x9f, 0x443: 0x9f, 0x444: 0x9f, 0x445: 0x9f, 0x446: 0x9f, 0x447: 0x9f,
+	0x448: 0x9f, 0x449: 0x9f, 0x44a: 0x9f, 0x44b: 0x9f, 0x44c: 0x9f, 0x44d: 0x9f, 0x44e: 0x151, 0x44f: 0xba,
+	0x450: 0x9b, 0x451: 0x152, 0x452: 0x9f, 0x453: 0x9f, 0x454: 0x9f, 0x455: 0x153, 0x456: 0xba, 0x457: 0xba,
+	0x458: 0xba, 0x459: 0xba, 0x45a: 0xba, 0x45b: 0xba, 0x45c: 0xba, 0x45d: 0xba, 0x45e: 0xba, 0x45f: 0xba,
+	0x460: 0xba, 0x461: 0xba, 0x462: 0xba, 0x463: 0xba, 0x464: 0xba, 0x465: 0xba, 0x466: 0xba, 0x467: 0xba,
+	0x468: 0xba, 0x469: 0xba, 0x46a: 0xba, 0x46b: 0xba, 0x46c: 0xba, 0x46d: 0xba, 0x46e: 0xba, 0x46f: 0xba,
+	0x470: 0xba, 0x471: 0xba, 0x472: 0xba, 0x473: 0xba, 0x474: 0xba, 0x475: 0xba, 0x476: 0xba, 0x477: 0xba,
+	0x478: 0xba, 0x479: 0xba, 0x47a: 0xba, 0x47b: 0xba, 0x47c: 0xba, 0x47d: 0xba, 0x47e: 0xba, 0x47f: 0xba,
+	// Block 0x12, offset 0x480
+	0x480: 0x9f, 0x481: 0x9f, 0x482: 0x9f, 0x483: 0x9f, 0x484: 0x9f, 0x485: 0x9f, 0x486: 0x9f, 0x487: 0x9f,
+	0x488: 0x9f, 0x489: 0x9f, 0x48a: 0x9f, 0x48b: 0x9f, 0x48c: 0x9f, 0x48d: 0x9f, 0x48e: 0x9f, 0x48f: 0x9f,
+	0x490: 0x154, 0x491: 0xba, 0x492: 0xba, 0x493: 0xba, 0x494: 0xba, 0x495: 0xba, 0x496: 0xba, 0x497: 0xba,
+	0x498: 0xba, 0x499: 0xba, 0x49a: 0xba, 0x49b: 0xba, 0x49c: 0xba, 0x49d: 0xba, 0x49e: 0xba, 0x49f: 0xba,
+	0x4a0: 0xba, 0x4a1: 0xba, 0x4a2: 0xba, 0x4a3: 0xba, 0x4a4: 0xba, 0x4a5: 0xba, 0x4a6: 0xba, 0x4a7: 0xba,
+	0x4a8: 0xba, 0x4a9: 0xba, 0x4aa: 0xba, 0x4ab: 0xba, 0x4ac: 0xba, 0x4ad: 0xba, 0x4ae: 0xba, 0x4af: 0xba,
+	0x4b0: 0xba, 0x4b1: 0xba, 0x4b2: 0xba, 0x4b3: 0xba, 0x4b4: 0xba, 0x4b5: 0xba, 0x4b6: 0xba, 0x4b7: 0xba,
+	0x4b8: 0xba, 0x4b9: 0xba, 0x4ba: 0xba, 0x4bb: 0xba, 0x4bc: 0xba, 0x4bd: 0xba, 0x4be: 0xba, 0x4bf: 0xba,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0xba, 0x4c1: 0xba, 0x4c2: 0xba, 0x4c3: 0xba, 0x4c4: 0xba, 0x4c5: 0xba, 0x4c6: 0xba, 0x4c7: 0xba,
+	0x4c8: 0xba, 0x4c9: 0xba, 0x4ca: 0xba, 0x4cb: 0xba, 0x4cc: 0xba, 0x4cd: 0xba, 0x4ce: 0xba, 0x4cf: 0xba,
+	0x4d0: 0x9f, 0x4d1: 0x9f, 0x4d2: 0x9f, 0x4d3: 0x9f, 0x4d4: 0x9f, 0x4d5: 0x9f, 0x4d6: 0x9f, 0x4d7: 0x9f,
+	0x4d8: 0x9f, 0x4d9: 0x155, 0x4da: 0xba, 0x4db: 0xba, 0x4dc: 0xba, 0x4dd: 0xba, 0x4de: 0xba, 0x4df: 0xba,
+	0x4e0: 0xba, 0x4e1: 0xba, 0x4e2: 0xba, 0x4e3: 0xba, 0x4e4: 0xba, 0x4e5: 0xba, 0x4e6: 0xba, 0x4e7: 0xba,
+	0x4e8: 0xba, 0x4e9: 0xba, 0x4ea: 0xba, 0x4eb: 0xba, 0x4ec: 0xba, 0x4ed: 0xba, 0x4ee: 0xba, 0x4ef: 0xba,
+	0x4f0: 0xba, 0x4f1: 0xba, 0x4f2: 0xba, 0x4f3: 0xba, 0x4f4: 0xba, 0x4f5: 0xba, 0x4f6: 0xba, 0x4f7: 0xba,
+	0x4f8: 0xba, 0x4f9: 0xba, 0x4fa: 0xba, 0x4fb: 0xba, 0x4fc: 0xba, 0x4fd: 0xba, 0x4fe: 0xba, 0x4ff: 0xba,
+	// Block 0x14, offset 0x500
+	0x500: 0xba, 0x501: 0xba, 0x502: 0xba, 0x503: 0xba, 0x504: 0xba, 0x505: 0xba, 0x506: 0xba, 0x507: 0xba,
+	0x508: 0xba, 0x509: 0xba, 0x50a: 0xba, 0x50b: 0xba, 0x50c: 0xba, 0x50d: 0xba, 0x50e: 0xba, 0x50f: 0xba,
+	0x510: 0xba, 0x511: 0xba, 0x512: 0xba, 0x513: 0xba, 0x514: 0xba, 0x515: 0xba, 0x516: 0xba, 0x517: 0xba,
+	0x518: 0xba, 0x519: 0xba, 0x51a: 0xba, 0x51b: 0xba, 0x51c: 0xba, 0x51d: 0xba, 0x51e: 0xba, 0x51f: 0xba,
+	0x520: 0x9f, 0x521: 0x9f, 0x522: 0x9f, 0x523: 0x9f, 0x524: 0x9f, 0x525: 0x9f, 0x526: 0x9f, 0x527: 0x9f,
+	0x528: 0x148, 0x529: 0x156, 0x52a: 0xba, 0x52b: 0x157, 0x52c: 0x158, 0x52d: 0x159, 0x52e: 0x15a, 0x52f: 0xba,
+	0x530: 0xba, 0x531: 0xba, 0x532: 0xba, 0x533: 0xba, 0x534: 0xba, 0x535: 0xba, 0x536: 0xba, 0x537: 0xba,
+	0x538: 0xba, 0x539: 0x15b, 0x53a: 0x15c, 0x53b: 0xba, 0x53c: 0x9f, 0x53d: 0x15d, 0x53e: 0x15e, 0x53f: 0x15f,
+	// Block 0x15, offset 0x540
+	0x540: 0x9f, 0x541: 0x9f, 0x542: 0x9f, 0x543: 0x9f, 0x544: 0x9f, 0x545: 0x9f, 0x546: 0x9f, 0x547: 0x9f,
+	0x548: 0x9f, 0x549: 0x9f, 0x54a: 0x9f, 0x54b: 0x9f, 0x54c: 0x9f, 0x54d: 0x9f, 0x54e: 0x9f, 0x54f: 0x9f,
+	0x550: 0x9f, 0x551: 0x9f, 0x552: 0x9f, 0x553: 0x9f, 0x554: 0x9f, 0x555: 0x9f, 0x556: 0x9f, 0x557: 0x9f,
+	0x558: 0x9f, 0x559: 0x9f, 0x55a: 0x9f, 0x55b: 0x9f, 0x55c: 0x9f, 0x55d: 0x9f, 0x55e: 0x9f, 0x55f: 0x160,
+	0x560: 0x9f, 0x561: 0x9f, 0x562: 0x9f, 0x563: 0x9f, 0x564: 0x9f, 0x565: 0x9f, 0x566: 0x9f, 0x567: 0x9f,
+	0x568: 0x9f, 0x569: 0x9f, 0x56a: 0x9f, 0x56b: 0x161, 0x56c: 0xba, 0x56d: 0xba, 0x56e: 0xba, 0x56f: 0xba,
+	0x570: 0xba, 0x571: 0xba, 0x572: 0xba, 0x573: 0xba, 0x574: 0xba, 0x575: 0xba, 0x576: 0xba, 0x577: 0xba,
+	0x578: 0xba, 0x579: 0xba, 0x57a: 0xba, 0x57b: 0xba, 0x57c: 0xba, 0x57d: 0xba, 0x57e: 0xba, 0x57f: 0xba,
+	// Block 0x16, offset 0x580
+	0x580: 0x9f, 0x581: 0x9f, 0x582: 0x9f, 0x583: 0x9f, 0x584: 0x162, 0x585: 0x163, 0x586: 0x9f, 0x587: 0x9f,
+	0x588: 0x9f, 0x589: 0x9f, 0x58a: 0x9f, 0x58b: 0x164, 0x58c: 0xba, 0x58d: 0xba, 0x58e: 0xba, 0x58f: 0xba,
+	0x590: 0xba, 0x591: 0xba, 0x592: 0xba, 0x593: 0xba, 0x594: 0xba, 0x595: 0xba, 0x596: 0xba, 0x597: 0xba,
+	0x598: 0xba, 0x599: 0xba, 0x59a: 0xba, 0x59b: 0xba, 0x59c: 0xba, 0x59d: 0xba, 0x59e: 0xba, 0x59f: 0xba,
+	0x5a0: 0xba, 0x5a1: 0xba, 0x5a2: 0xba, 0x5a3: 0xba, 0x5a4: 0xba, 0x5a5: 0xba, 0x5a6: 0xba, 0x5a7: 0xba,
+	0x5a8: 0xba, 0x5a9: 0xba, 0x5aa: 0xba, 0x5ab: 0xba, 0x5ac: 0xba, 0x5ad: 0xba, 0x5ae: 0xba, 0x5af: 0xba,
+	0x5b0: 0x9f, 0x5b1: 0x165, 0x5b2: 0x166, 0x5b3: 0xba, 0x5b4: 0xba, 0x5b5: 0xba, 0x5b6: 0xba, 0x5b7: 0xba,
+	0x5b8: 0xba, 0x5b9: 0xba, 0x5ba: 0xba, 0x5bb: 0xba, 0x5bc: 0xba, 0x5bd: 0xba, 0x5be: 0xba, 0x5bf: 0xba,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x9b, 0x5c1: 0x9b, 0x5c2: 0x9b, 0x5c3: 0x167, 0x5c4: 0x168, 0x5c5: 0x169, 0x5c6: 0x16a, 0x5c7: 0x16b,
+	0x5c8: 0x9b, 0x5c9: 0x16c, 0x5ca: 0xba, 0x5cb: 0x16d, 0x5cc: 0x9b, 0x5cd: 0x16e, 0x5ce: 0xba, 0x5cf: 0xba,
+	0x5d0: 0x5f, 0x5d1: 0x60, 0x5d2: 0x61, 0x5d3: 0x62, 0x5d4: 0x63, 0x5d5: 0x64, 0x5d6: 0x65, 0x5d7: 0x66,
+	0x5d8: 0x67, 0x5d9: 0x68, 0x5da: 0x69, 0x5db: 0x6a, 0x5dc: 0x6b, 0x5dd: 0x6c, 0x5de: 0x6d, 0x5df: 0x6e,
+	0x5e0: 0x9b, 0x5e1: 0x9b, 0x5e2: 0x9b, 0x5e3: 0x9b, 0x5e4: 0x9b, 0x5e5: 0x9b, 0x5e6: 0x9b, 0x5e7: 0x9b,
+	0x5e8: 0x16f, 0x5e9: 0x170, 0x5ea: 0x171, 0x5eb: 0xba, 0x5ec: 0xba, 0x5ed: 0xba, 0x5ee: 0xba, 0x5ef: 0xba,
+	0x5f0: 0xba, 0x5f1: 0xba, 0x5f2: 0xba, 0x5f3: 0xba, 0x5f4: 0xba, 0x5f5: 0xba, 0x5f6: 0xba, 0x5f7: 0xba,
+	0x5f8: 0xba, 0x5f9: 0xba, 0x5fa: 0xba, 0x5fb: 0xba, 0x5fc: 0xba, 0x5fd: 0xba, 0x5fe: 0xba, 0x5ff: 0xba,
+	// Block 0x18, offset 0x600
+	0x600: 0x172, 0x601: 0xba, 0x602: 0xba, 0x603: 0xba, 0x604: 0x173, 0x605: 0x174, 0x606: 0xba, 0x607: 0xba,
+	0x608: 0xba, 0x609: 0xba, 0x60a: 0xba, 0x60b: 0x175, 0x60c: 0xba, 0x60d: 0xba, 0x60e: 0xba, 0x60f: 0xba,
+	0x610: 0xba, 0x611: 0xba, 0x612: 0xba, 0x613: 0xba, 0x614: 0xba, 0x615: 0xba, 0x616: 0xba, 0x617: 0xba,
+	0x618: 0xba, 0x619: 0xba, 0x61a: 0xba, 0x61b: 0xba, 0x61c: 0xba, 0x61d: 0xba, 0x61e: 0xba, 0x61f: 0xba,
+	0x620: 0x121, 0x621: 0x121, 0x622: 0x121, 0x623: 0x176, 0x624: 0x6f, 0x625: 0x177, 0x626: 0xba, 0x627: 0xba,
+	0x628: 0xba, 0x629: 0xba, 0x62a: 0xba, 0x62b: 0xba, 0x62c: 0xba, 0x62d: 0xba, 0x62e: 0xba, 0x62f: 0xba,
+	0x630: 0xba, 0x631: 0x178, 0x632: 0x179, 0x633: 0xba, 0x634: 0x17a, 0x635: 0xba, 0x636: 0xba, 0x637: 0xba,
+	0x638: 0x70, 0x639: 0x71, 0x63a: 0x72, 0x63b: 0x17b, 0x63c: 0xba, 0x63d: 0xba, 0x63e: 0xba, 0x63f: 0xba,
+	// Block 0x19, offset 0x640
+	0x640: 0x17c, 0x641: 0x9b, 0x642: 0x17d, 0x643: 0x17e, 0x644: 0x73, 0x645: 0x74, 0x646: 0x17f, 0x647: 0x180,
+	0x648: 0x75, 0x649: 0x181, 0x64a: 0xba, 0x64b: 0xba, 0x64c: 0x9b, 0x64d: 0x9b, 0x64e: 0x9b, 0x64f: 0x9b,
+	0x650: 0x9b, 0x651: 0x9b, 0x652: 0x9b, 0x653: 0x9b, 0x654: 0x9b, 0x655: 0x9b, 0x656: 0x9b, 0x657: 0x9b,
+	0x658: 0x9b, 0x659: 0x9b, 0x65a: 0x9b, 0x65b: 0x182, 0x65c: 0x9b, 0x65d: 0x183, 0x65e: 0x9b, 0x65f: 0x184,
+	0x660: 0x185, 0x661: 0x186, 0x662: 0x187, 0x663: 0xba, 0x664: 0x188, 0x665: 0x189, 0x666: 0x18a, 0x667: 0x18b,
+	0x668: 0x9b, 0x669: 0x18c, 0x66a: 0x18d, 0x66b: 0xba, 0x66c: 0xba, 0x66d: 0xba, 0x66e: 0xba, 0x66f: 0xba,
+	0x670: 0xba, 0x671: 0xba, 0x672: 0xba, 0x673: 0xba, 0x674: 0xba, 0x675: 0xba, 0x676: 0xba, 0x677: 0xba,
+	0x678: 0xba, 0x679: 0xba, 0x67a: 0xba, 0x67b: 0xba, 0x67c: 0xba, 0x67d: 0xba, 0x67e: 0xba, 0x67f: 0xba,
+	// Block 0x1a, offset 0x680
+	0x680: 0x9f, 0x681: 0x9f, 0x682: 0x9f, 0x683: 0x9f, 0x684: 0x9f, 0x685: 0x9f, 0x686: 0x9f, 0x687: 0x9f,
+	0x688: 0x9f, 0x689: 0x9f, 0x68a: 0x9f, 0x68b: 0x9f, 0x68c: 0x9f, 0x68d: 0x9f, 0x68e: 0x9f, 0x68f: 0x9f,
+	0x690: 0x9f, 0x691: 0x9f, 0x692: 0x9f, 0x693: 0x9f, 0x694: 0x9f, 0x695: 0x9f, 0x696: 0x9f, 0x697: 0x9f,
+	0x698: 0x9f, 0x699: 0x9f, 0x69a: 0x9f, 0x69b: 0x18e, 0x69c: 0x9f, 0x69d: 0x9f, 0x69e: 0x9f, 0x69f: 0x9f,
+	0x6a0: 0x9f, 0x6a1: 0x9f, 0x6a2: 0x9f, 0x6a3: 0x9f, 0x6a4: 0x9f, 0x6a5: 0x9f, 0x6a6: 0x9f, 0x6a7: 0x9f,
+	0x6a8: 0x9f, 0x6a9: 0x9f, 0x6aa: 0x9f, 0x6ab: 0x9f, 0x6ac: 0x9f, 0x6ad: 0x9f, 0x6ae: 0x9f, 0x6af: 0x9f,
+	0x6b0: 0x9f, 0x6b1: 0x9f, 0x6b2: 0x9f, 0x6b3: 0x9f, 0x6b4: 0x9f, 0x6b5: 0x9f, 0x6b6: 0x9f, 0x6b7: 0x9f,
+	0x6b8: 0x9f, 0x6b9: 0x9f, 0x6ba: 0x9f, 0x6bb: 0x9f, 0x6bc: 0x9f, 0x6bd: 0x9f, 0x6be: 0x9f, 0x6bf: 0x9f,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0x9f, 0x6c1: 0x9f, 0x6c2: 0x9f, 0x6c3: 0x9f, 0x6c4: 0x9f, 0x6c5: 0x9f, 0x6c6: 0x9f, 0x6c7: 0x9f,
+	0x6c8: 0x9f, 0x6c9: 0x9f, 0x6ca: 0x9f, 0x6cb: 0x9f, 0x6cc: 0x9f, 0x6cd: 0x9f, 0x6ce: 0x9f, 0x6cf: 0x9f,
+	0x6d0: 0x9f, 0x6d1: 0x9f, 0x6d2: 0x9f, 0x6d3: 0x9f, 0x6d4: 0x9f, 0x6d5: 0x9f, 0x6d6: 0x9f, 0x6d7: 0x9f,
+	0x6d8: 0x9f, 0x6d9: 0x9f, 0x6da: 0x9f, 0x6db: 0x9f, 0x6dc: 0x18f, 0x6dd: 0x9f, 0x6de: 0x9f, 0x6df: 0x9f,
+	0x6e0: 0x190, 0x6e1: 0x9f, 0x6e2: 0x9f, 0x6e3: 0x9f, 0x6e4: 0x9f, 0x6e5: 0x9f, 0x6e6: 0x9f, 0x6e7: 0x9f,
+	0x6e8: 0x9f, 0x6e9: 0x9f, 0x6ea: 0x9f, 0x6eb: 0x9f, 0x6ec: 0x9f, 0x6ed: 0x9f, 0x6ee: 0x9f, 0x6ef: 0x9f,
+	0x6f0: 0x9f, 0x6f1: 0x9f, 0x6f2: 0x9f, 0x6f3: 0x9f, 0x6f4: 0x9f, 0x6f5: 0x9f, 0x6f6: 0x9f, 0x6f7: 0x9f,
+	0x6f8: 0x9f, 0x6f9: 0x9f, 0x6fa: 0x9f, 0x6fb: 0x9f, 0x6fc: 0x9f, 0x6fd: 0x9f, 0x6fe: 0x9f, 0x6ff: 0x9f,
+	// Block 0x1c, offset 0x700
+	0x700: 0x9f, 0x701: 0x9f, 0x702: 0x9f, 0x703: 0x9f, 0x704: 0x9f, 0x705: 0x9f, 0x706: 0x9f, 0x707: 0x9f,
+	0x708: 0x9f, 0x709: 0x9f, 0x70a: 0x9f, 0x70b: 0x9f, 0x70c: 0x9f, 0x70d: 0x9f, 0x70e: 0x9f, 0x70f: 0x9f,
+	0x710: 0x9f, 0x711: 0x9f, 0x712: 0x9f, 0x713: 0x9f, 0x714: 0x9f, 0x715: 0x9f, 0x716: 0x9f, 0x717: 0x9f,
+	0x718: 0x9f, 0x719: 0x9f, 0x71a: 0x9f, 0x71b: 0x9f, 0x71c: 0x9f, 0x71d: 0x9f, 0x71e: 0x9f, 0x71f: 0x9f,
+	0x720: 0x9f, 0x721: 0x9f, 0x722: 0x9f, 0x723: 0x9f, 0x724: 0x9f, 0x725: 0x9f, 0x726: 0x9f, 0x727: 0x9f,
+	0x728: 0x9f, 0x729: 0x9f, 0x72a: 0x9f, 0x72b: 0x9f, 0x72c: 0x9f, 0x72d: 0x9f, 0x72e: 0x9f, 0x72f: 0x9f,
+	0x730: 0x9f, 0x731: 0x9f, 0x732: 0x9f, 0x733: 0x9f, 0x734: 0x9f, 0x735: 0x9f, 0x736: 0x9f, 0x737: 0x9f,
+	0x738: 0x9f, 0x739: 0x9f, 0x73a: 0x191, 0x73b: 0x9f, 0x73c: 0x9f, 0x73d: 0x9f, 0x73e: 0x9f, 0x73f: 0x9f,
+	// Block 0x1d, offset 0x740
+	0x740: 0x9f, 0x741: 0x9f, 0x742: 0x9f, 0x743: 0x9f, 0x744: 0x9f, 0x745: 0x9f, 0x746: 0x9f, 0x747: 0x9f,
+	0x748: 0x9f, 0x749: 0x9f, 0x74a: 0x9f, 0x74b: 0x9f, 0x74c: 0x9f, 0x74d: 0x9f, 0x74e: 0x9f, 0x74f: 0x9f,
+	0x750: 0x9f, 0x751: 0x9f, 0x752: 0x9f, 0x753: 0x9f, 0x754: 0x9f, 0x755: 0x9f, 0x756: 0x9f, 0x757: 0x9f,
+	0x758: 0x9f, 0x759: 0x9f, 0x75a: 0x9f, 0x75b: 0x9f, 0x75c: 0x9f, 0x75d: 0x9f, 0x75e: 0x9f, 0x75f: 0x9f,
+	0x760: 0x9f, 0x761: 0x9f, 0x762: 0x9f, 0x763: 0x9f, 0x764: 0x9f, 0x765: 0x9f, 0x766: 0x9f, 0x767: 0x9f,
+	0x768: 0x9f, 0x769: 0x9f, 0x76a: 0x9f, 0x76b: 0x9f, 0x76c: 0x9f, 0x76d: 0x9f, 0x76e: 0x9f, 0x76f: 0x192,
+	0x770: 0xba, 0x771: 0xba, 0x772: 0xba, 0x773: 0xba, 0x774: 0xba, 0x775: 0xba, 0x776: 0xba, 0x777: 0xba,
+	0x778: 0xba, 0x779: 0xba, 0x77a: 0xba, 0x77b: 0xba, 0x77c: 0xba, 0x77d: 0xba, 0x77e: 0xba, 0x77f: 0xba,
+	// Block 0x1e, offset 0x780
+	0x780: 0xba, 0x781: 0xba, 0x782: 0xba, 0x783: 0xba, 0x784: 0xba, 0x785: 0xba, 0x786: 0xba, 0x787: 0xba,
+	0x788: 0xba, 0x789: 0xba, 0x78a: 0xba, 0x78b: 0xba, 0x78c: 0xba, 0x78d: 0xba, 0x78e: 0xba, 0x78f: 0xba,
+	0x790: 0xba, 0x791: 0xba, 0x792: 0xba, 0x793: 0xba, 0x794: 0xba, 0x795: 0xba, 0x796: 0xba, 0x797: 0xba,
+	0x798: 0xba, 0x799: 0xba, 0x79a: 0xba, 0x79b: 0xba, 0x79c: 0xba, 0x79d: 0xba, 0x79e: 0xba, 0x79f: 0xba,
+	0x7a0: 0x76, 0x7a1: 0x77, 0x7a2: 0x78, 0x7a3: 0x193, 0x7a4: 0x79, 0x7a5: 0x7a, 0x7a6: 0x194, 0x7a7: 0x7b,
+	0x7a8: 0x7c, 0x7a9: 0xba, 0x7aa: 0xba, 0x7ab: 0xba, 0x7ac: 0xba, 0x7ad: 0xba, 0x7ae: 0xba, 0x7af: 0xba,
+	0x7b0: 0xba, 0x7b1: 0xba, 0x7b2: 0xba, 0x7b3: 0xba, 0x7b4: 0xba, 0x7b5: 0xba, 0x7b6: 0xba, 0x7b7: 0xba,
+	0x7b8: 0xba, 0x7b9: 0xba, 0x7ba: 0xba, 0x7bb: 0xba, 0x7bc: 0xba, 0x7bd: 0xba, 0x7be: 0xba, 0x7bf: 0xba,
+	// Block 0x1f, offset 0x7c0
+	0x7d0: 0x0d, 0x7d1: 0x0e, 0x7d2: 0x0f, 0x7d3: 0x10, 0x7d4: 0x11, 0x7d5: 0x0b, 0x7d6: 0x12, 0x7d7: 0x07,
+	0x7d8: 0x13, 0x7d9: 0x0b, 0x7da: 0x0b, 0x7db: 0x14, 0x7dc: 0x0b, 0x7dd: 0x15, 0x7de: 0x16, 0x7df: 0x17,
+	0x7e0: 0x07, 0x7e1: 0x07, 0x7e2: 0x07, 0x7e3: 0x07, 0x7e4: 0x07, 0x7e5: 0x07, 0x7e6: 0x07, 0x7e7: 0x07,
+	0x7e8: 0x07, 0x7e9: 0x07, 0x7ea: 0x18, 0x7eb: 0x19, 0x7ec: 0x1a, 0x7ed: 0x07, 0x7ee: 0x1b, 0x7ef: 0x1c,
+	0x7f0: 0x0b, 0x7f1: 0x0b, 0x7f2: 0x0b, 0x7f3: 0x0b, 0x7f4: 0x0b, 0x7f5: 0x0b, 0x7f6: 0x0b, 0x7f7: 0x0b,
+	0x7f8: 0x0b, 0x7f9: 0x0b, 0x7fa: 0x0b, 0x7fb: 0x0b, 0x7fc: 0x0b, 0x7fd: 0x0b, 0x7fe: 0x0b, 0x7ff: 0x0b,
+	// Block 0x20, offset 0x800
+	0x800: 0x0b, 0x801: 0x0b, 0x802: 0x0b, 0x803: 0x0b, 0x804: 0x0b, 0x805: 0x0b, 0x806: 0x0b, 0x807: 0x0b,
+	0x808: 0x0b, 0x809: 0x0b, 0x80a: 0x0b, 0x80b: 0x0b, 0x80c: 0x0b, 0x80d: 0x0b, 0x80e: 0x0b, 0x80f: 0x0b,
+	0x810: 0x0b, 0x811: 0x0b, 0x812: 0x0b, 0x813: 0x0b, 0x814: 0x0b, 0x815: 0x0b, 0x816: 0x0b, 0x817: 0x0b,
+	0x818: 0x0b, 0x819: 0x0b, 0x81a: 0x0b, 0x81b: 0x0b, 0x81c: 0x0b, 0x81d: 0x0b, 0x81e: 0x0b, 0x81f: 0x0b,
+	0x820: 0x0b, 0x821: 0x0b, 0x822: 0x0b, 0x823: 0x0b, 0x824: 0x0b, 0x825: 0x0b, 0x826: 0x0b, 0x827: 0x0b,
+	0x828: 0x0b, 0x829: 0x0b, 0x82a: 0x0b, 0x82b: 0x0b, 0x82c: 0x0b, 0x82d: 0x0b, 0x82e: 0x0b, 0x82f: 0x0b,
+	0x830: 0x0b, 0x831: 0x0b, 0x832: 0x0b, 0x833: 0x0b, 0x834: 0x0b, 0x835: 0x0b, 0x836: 0x0b, 0x837: 0x0b,
+	0x838: 0x0b, 0x839: 0x0b, 0x83a: 0x0b, 0x83b: 0x0b, 0x83c: 0x0b, 0x83d: 0x0b, 0x83e: 0x0b, 0x83f: 0x0b,
+	// Block 0x21, offset 0x840
+	0x840: 0x195, 0x841: 0x196, 0x842: 0xba, 0x843: 0xba, 0x844: 0x197, 0x845: 0x197, 0x846: 0x197, 0x847: 0x198,
+	0x848: 0xba, 0x849: 0xba, 0x84a: 0xba, 0x84b: 0xba, 0x84c: 0xba, 0x84d: 0xba, 0x84e: 0xba, 0x84f: 0xba,
+	0x850: 0xba, 0x851: 0xba, 0x852: 0xba, 0x853: 0xba, 0x854: 0xba, 0x855: 0xba, 0x856: 0xba, 0x857: 0xba,
+	0x858: 0xba, 0x859: 0xba, 0x85a: 0xba, 0x85b: 0xba, 0x85c: 0xba, 0x85d: 0xba, 0x85e: 0xba, 0x85f: 0xba,
+	0x860: 0xba, 0x861: 0xba, 0x862: 0xba, 0x863: 0xba, 0x864: 0xba, 0x865: 0xba, 0x866: 0xba, 0x867: 0xba,
+	0x868: 0xba, 0x869: 0xba, 0x86a: 0xba, 0x86b: 0xba, 0x86c: 0xba, 0x86d: 0xba, 0x86e: 0xba, 0x86f: 0xba,
+	0x870: 0xba, 0x871: 0xba, 0x872: 0xba, 0x873: 0xba, 0x874: 0xba, 0x875: 0xba, 0x876: 0xba, 0x877: 0xba,
+	0x878: 0xba, 0x879: 0xba, 0x87a: 0xba, 0x87b: 0xba, 0x87c: 0xba, 0x87d: 0xba, 0x87e: 0xba, 0x87f: 0xba,
+	// Block 0x22, offset 0x880
+	0x880: 0x0b, 0x881: 0x0b, 0x882: 0x0b, 0x883: 0x0b, 0x884: 0x0b, 0x885: 0x0b, 0x886: 0x0b, 0x887: 0x0b,
+	0x888: 0x0b, 0x889: 0x0b, 0x88a: 0x0b, 0x88b: 0x0b, 0x88c: 0x0b, 0x88d: 0x0b, 0x88e: 0x0b, 0x88f: 0x0b,
+	0x890: 0x0b, 0x891: 0x0b, 0x892: 0x0b, 0x893: 0x0b, 0x894: 0x0b, 0x895: 0x0b, 0x896: 0x0b, 0x897: 0x0b,
+	0x898: 0x0b, 0x899: 0x0b, 0x89a: 0x0b, 0x89b: 0x0b, 0x89c: 0x0b, 0x89d: 0x0b, 0x89e: 0x0b, 0x89f: 0x0b,
+	0x8a0: 0x1f, 0x8a1: 0x0b, 0x8a2: 0x0b, 0x8a3: 0x0b, 0x8a4: 0x0b, 0x8a5: 0x0b, 0x8a6: 0x0b, 0x8a7: 0x0b,
+	0x8a8: 0x0b, 0x8a9: 0x0b, 0x8aa: 0x0b, 0x8ab: 0x0b, 0x8ac: 0x0b, 0x8ad: 0x0b, 0x8ae: 0x0b, 0x8af: 0x0b,
+	0x8b0: 0x0b, 0x8b1: 0x0b, 0x8b2: 0x0b, 0x8b3: 0x0b, 0x8b4: 0x0b, 0x8b5: 0x0b, 0x8b6: 0x0b, 0x8b7: 0x0b,
+	0x8b8: 0x0b, 0x8b9: 0x0b, 0x8ba: 0x0b, 0x8bb: 0x0b, 0x8bc: 0x0b, 0x8bd: 0x0b, 0x8be: 0x0b, 0x8bf: 0x0b,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x0b, 0x8c1: 0x0b, 0x8c2: 0x0b, 0x8c3: 0x0b, 0x8c4: 0x0b, 0x8c5: 0x0b, 0x8c6: 0x0b, 0x8c7: 0x0b,
+	0x8c8: 0x0b, 0x8c9: 0x0b, 0x8ca: 0x0b, 0x8cb: 0x0b, 0x8cc: 0x0b, 0x8cd: 0x0b, 0x8ce: 0x0b, 0x8cf: 0x0b,
+}
+
+// idnaSparseOffset: 284 entries, 568 bytes
+var idnaSparseOffset = []uint16{0x0, 0x8, 0x19, 0x25, 0x27, 0x2c, 0x33, 0x3e, 0x4a, 0x4e, 0x5d, 0x62, 0x6c, 0x78, 0x86, 0x8b, 0x94, 0xa4, 0xb2, 0xbe, 0xca, 0xdb, 0xe5, 0xec, 0xf9, 0x10a, 0x111, 0x11c, 0x12b, 0x139, 0x143, 0x145, 0x14a, 0x14d, 0x150, 0x152, 0x15e, 0x169, 0x171, 0x177, 0x17d, 0x182, 0x187, 0x18a, 0x18e, 0x194, 0x199, 0x1a5, 0x1af, 0x1b5, 0x1c6, 0x1d0, 0x1d3, 0x1db, 0x1de, 0x1eb, 0x1f3, 0x1f7, 0x1fe, 0x206, 0x216, 0x222, 0x224, 0x22e, 0x23a, 0x246, 0x252, 0x25a, 0x25f, 0x26c, 0x27d, 0x281, 0x28c, 0x290, 0x299, 0x2a1, 0x2a7, 0x2ac, 0x2af, 0x2b3, 0x2b9, 0x2bd, 0x2c1, 0x2c5, 0x2cb, 0x2d3, 0x2da, 0x2e5, 0x2ef, 0x2f3, 0x2f6, 0x2fc, 0x300, 0x302, 0x305, 0x307, 0x30a, 0x314, 0x317, 0x326, 0x32a, 0x32f, 0x332, 0x336, 0x33b, 0x340, 0x346, 0x352, 0x361, 0x367, 0x36b, 0x37a, 0x37f, 0x387, 0x391, 0x39c, 0x3a4, 0x3b5, 0x3be, 0x3ce, 0x3db, 0x3e5, 0x3ea, 0x3f7, 0x3fb, 0x400, 0x402, 0x406, 0x408, 0x40c, 0x415, 0x41b, 0x41f, 0x42f, 0x439, 0x43e, 0x441, 0x447, 0x44e, 0x453, 0x457, 0x45d, 0x462, 0x46b, 0x470, 0x476, 0x47d, 0x484, 0x48b, 0x48f, 0x494, 0x497, 0x49c, 0x4a8, 0x4ae, 0x4b3, 0x4ba, 0x4c2, 0x4c7, 0x4cb, 0x4db, 0x4e2, 0x4e6, 0x4ea, 0x4f1, 0x4f3, 0x4f6, 0x4f9, 0x4fd, 0x506, 0x50a, 0x512, 0x51a, 0x51e, 0x524, 0x52d, 0x539, 0x540, 0x549, 0x553, 0x55a, 0x568, 0x575, 0x582, 0x58b, 0x58f, 0x59f, 0x5a7, 0x5b2, 0x5bb, 0x5c1, 0x5c9, 0x5d2, 0x5dd, 0x5e0, 0x5ec, 0x5f5, 0x5f8, 0x5fd, 0x602, 0x60f, 0x61a, 0x623, 0x62d, 0x630, 0x63a, 0x643, 0x64f, 0x65c, 0x669, 0x677, 0x67e, 0x682, 0x685, 0x68a, 0x68d, 0x692, 0x695, 0x69c, 0x6a3, 0x6a7, 0x6b2, 0x6b5, 0x6b8, 0x6bb, 0x6c1, 0x6c7, 0x6cd, 0x6d0, 0x6d3, 0x6d6, 0x6dd, 0x6e0, 0x6e5, 0x6ef, 0x6f2, 0x6f6, 0x705, 0x711, 0x715, 0x71a, 0x71e, 0x723, 0x727, 0x72c, 0x735, 0x740, 0x746, 0x74c, 0x752, 0x758, 0x761, 0x764, 0x767, 0x76b, 0x76f, 0x773, 0x779, 0x77f, 0x784, 0x787, 0x797, 0x79e, 0x7a1, 0x7a6, 0x7aa, 0x7b0, 0x7b5, 0x7b9, 0x7bf, 0x7c5, 0x7c9, 0x7d2, 0x7d7, 0x7da, 0x7dd, 0x7e1, 0x7e5, 0x7e8, 0x7f8, 0x809, 0x80e, 0x810, 0x812}
+
+// idnaSparseValues: 2069 entries, 8276 bytes
+var idnaSparseValues = [2069]valueRange{
+	// Block 0x0, offset 0x0
+	{value: 0x0000, lo: 0x07},
+	{value: 0xe105, lo: 0x80, hi: 0x96},
+	{value: 0x0018, lo: 0x97, hi: 0x97},
+	{value: 0xe105, lo: 0x98, hi: 0x9e},
+	{value: 0x001f, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbf},
+	// Block 0x1, offset 0x8
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0xe01d, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x82},
+	{value: 0x0335, lo: 0x83, hi: 0x83},
+	{value: 0x034d, lo: 0x84, hi: 0x84},
+	{value: 0x0365, lo: 0x85, hi: 0x85},
+	{value: 0xe00d, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x87},
+	{value: 0xe00d, lo: 0x88, hi: 0x88},
+	{value: 0x0008, lo: 0x89, hi: 0x89},
+	{value: 0xe00d, lo: 0x8a, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0x8b},
+	{value: 0xe00d, lo: 0x8c, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0x8d},
+	{value: 0xe00d, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0xbf},
+	// Block 0x2, offset 0x19
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x0249, lo: 0xb0, hi: 0xb0},
+	{value: 0x037d, lo: 0xb1, hi: 0xb1},
+	{value: 0x0259, lo: 0xb2, hi: 0xb2},
+	{value: 0x0269, lo: 0xb3, hi: 0xb3},
+	{value: 0x034d, lo: 0xb4, hi: 0xb4},
+	{value: 0x0395, lo: 0xb5, hi: 0xb5},
+	{value: 0xe1bd, lo: 0xb6, hi: 0xb6},
+	{value: 0x0279, lo: 0xb7, hi: 0xb7},
+	{value: 0x0289, lo: 0xb8, hi: 0xb8},
+	{value: 0x0008, lo: 0xb9, hi: 0xbf},
+	// Block 0x3, offset 0x25
+	{value: 0x0000, lo: 0x01},
+	{value: 0x3308, lo: 0x80, hi: 0xbf},
+	// Block 0x4, offset 0x27
+	{value: 0x0000, lo: 0x04},
+	{value: 0x03f5, lo: 0x80, hi: 0x8f},
+	{value: 0xe105, lo: 0x90, hi: 0x9f},
+	{value: 0x049d, lo: 0xa0, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x5, offset 0x2c
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe185, lo: 0x80, hi: 0x8f},
+	{value: 0x0545, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x98},
+	{value: 0x0008, lo: 0x99, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x6, offset 0x33
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0401, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x88},
+	{value: 0x0018, lo: 0x89, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x3308, lo: 0x91, hi: 0xbd},
+	{value: 0x0818, lo: 0xbe, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x7, offset 0x3e
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0818, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x82},
+	{value: 0x0818, lo: 0x83, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x85},
+	{value: 0x0818, lo: 0x86, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xae},
+	{value: 0x0808, lo: 0xaf, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x8, offset 0x4a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0a08, lo: 0x80, hi: 0x87},
+	{value: 0x0c08, lo: 0x88, hi: 0x99},
+	{value: 0x0a08, lo: 0x9a, hi: 0xbf},
+	// Block 0x9, offset 0x4e
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3308, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8c},
+	{value: 0x0c08, lo: 0x8d, hi: 0x8d},
+	{value: 0x0a08, lo: 0x8e, hi: 0x98},
+	{value: 0x0c08, lo: 0x99, hi: 0x9b},
+	{value: 0x0a08, lo: 0x9c, hi: 0xaa},
+	{value: 0x0c08, lo: 0xab, hi: 0xac},
+	{value: 0x0a08, lo: 0xad, hi: 0xb0},
+	{value: 0x0c08, lo: 0xb1, hi: 0xb1},
+	{value: 0x0a08, lo: 0xb2, hi: 0xb2},
+	{value: 0x0c08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0a08, lo: 0xb5, hi: 0xb7},
+	{value: 0x0c08, lo: 0xb8, hi: 0xb9},
+	{value: 0x0a08, lo: 0xba, hi: 0xbf},
+	// Block 0xa, offset 0x5d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xb0},
+	{value: 0x0808, lo: 0xb1, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0xb, offset 0x62
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0808, lo: 0x80, hi: 0x89},
+	{value: 0x0a08, lo: 0x8a, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xb3},
+	{value: 0x0808, lo: 0xb4, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xb9},
+	{value: 0x0818, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x0818, lo: 0xbe, hi: 0xbf},
+	// Block 0xc, offset 0x6c
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x99},
+	{value: 0x0808, lo: 0x9a, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0xa3},
+	{value: 0x0808, lo: 0xa4, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa7},
+	{value: 0x0808, lo: 0xa8, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0818, lo: 0xb0, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xd, offset 0x78
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0a08, lo: 0xa0, hi: 0xa9},
+	{value: 0x0c08, lo: 0xaa, hi: 0xac},
+	{value: 0x0808, lo: 0xad, hi: 0xad},
+	{value: 0x0c08, lo: 0xae, hi: 0xae},
+	{value: 0x0a08, lo: 0xaf, hi: 0xb0},
+	{value: 0x0c08, lo: 0xb1, hi: 0xb2},
+	{value: 0x0a08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xb5},
+	{value: 0x0a08, lo: 0xb6, hi: 0xb8},
+	{value: 0x0c08, lo: 0xb9, hi: 0xb9},
+	{value: 0x0a08, lo: 0xba, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0xe, offset 0x86
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x92},
+	{value: 0x3308, lo: 0x93, hi: 0xa1},
+	{value: 0x0840, lo: 0xa2, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xbf},
+	// Block 0xf, offset 0x8b
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x10, offset 0x94
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x85},
+	{value: 0x3008, lo: 0x86, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x3008, lo: 0x8a, hi: 0x8c},
+	{value: 0x3b08, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x11, offset 0xa4
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbf},
+	// Block 0x12, offset 0xb2
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0xba},
+	{value: 0x3b08, lo: 0xbb, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x13, offset 0xbe
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0040, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xb2},
+	{value: 0x0008, lo: 0xb3, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x14, offset 0xca
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x89},
+	{value: 0x3b08, lo: 0x8a, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8e},
+	{value: 0x3008, lo: 0x8f, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x94},
+	{value: 0x0040, lo: 0x95, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x3008, lo: 0x98, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xb1},
+	{value: 0x3008, lo: 0xb2, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x15, offset 0xdb
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb2},
+	{value: 0x08f1, lo: 0xb3, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb9},
+	{value: 0x3b08, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0x16, offset 0xe5
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x8e},
+	{value: 0x0018, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0xbf},
+	// Block 0x17, offset 0xec
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x3308, lo: 0x88, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0961, lo: 0x9c, hi: 0x9c},
+	{value: 0x0999, lo: 0x9d, hi: 0x9d},
+	{value: 0x0008, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0x18, offset 0xf9
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0x8b},
+	{value: 0xe03d, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xb8},
+	{value: 0x3308, lo: 0xb9, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x19, offset 0x10a
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0018, lo: 0x8e, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0xbf},
+	// Block 0x1a, offset 0x111
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x3008, lo: 0xab, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xb0},
+	{value: 0x3008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0x1b, offset 0x11c
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x95},
+	{value: 0x3008, lo: 0x96, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0x9d},
+	{value: 0x3308, lo: 0x9e, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3008, lo: 0xa2, hi: 0xa4},
+	{value: 0x0008, lo: 0xa5, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xbf},
+	// Block 0x1c, offset 0x12b
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x3008, lo: 0x87, hi: 0x8c},
+	{value: 0x3308, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x8e},
+	{value: 0x3008, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x3008, lo: 0x9a, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0x1d, offset 0x139
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0040, lo: 0x80, hi: 0x86},
+	{value: 0x055d, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8c},
+	{value: 0x055d, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbb},
+	{value: 0xe105, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbf},
+	// Block 0x1e, offset 0x143
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0018, lo: 0x80, hi: 0xbf},
+	// Block 0x1f, offset 0x145
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xa0},
+	{value: 0x2018, lo: 0xa1, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0x20, offset 0x14a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xa7},
+	{value: 0x2018, lo: 0xa8, hi: 0xbf},
+	// Block 0x21, offset 0x14d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x2018, lo: 0x80, hi: 0x82},
+	{value: 0x0018, lo: 0x83, hi: 0xbf},
+	// Block 0x22, offset 0x150
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0008, lo: 0x80, hi: 0xbf},
+	// Block 0x23, offset 0x152
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x24, offset 0x15e
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x25, offset 0x169
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0040, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbf},
+	// Block 0x26, offset 0x171
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbf},
+	// Block 0x27, offset 0x177
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x28, offset 0x17d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x29, offset 0x182
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0xe045, lo: 0xb8, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x2a, offset 0x187
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xbf},
+	// Block 0x2b, offset 0x18a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xac},
+	{value: 0x0018, lo: 0xad, hi: 0xae},
+	{value: 0x0008, lo: 0xaf, hi: 0xbf},
+	// Block 0x2c, offset 0x18e
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9c},
+	{value: 0x0040, lo: 0x9d, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x2d, offset 0x194
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xb0},
+	{value: 0x0008, lo: 0xb1, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0x2e, offset 0x199
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x93},
+	{value: 0x3b08, lo: 0x94, hi: 0x94},
+	{value: 0x0040, lo: 0x95, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
+	{value: 0x0018, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x2f, offset 0x1a5
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x30, offset 0x1af
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xb3},
+	{value: 0x3340, lo: 0xb4, hi: 0xb5},
+	{value: 0x3008, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x31, offset 0x1b5
+	{value: 0x0000, lo: 0x10},
+	{value: 0x3008, lo: 0x80, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x3008, lo: 0x87, hi: 0x88},
+	{value: 0x3308, lo: 0x89, hi: 0x91},
+	{value: 0x3b08, lo: 0x92, hi: 0x92},
+	{value: 0x3308, lo: 0x93, hi: 0x93},
+	{value: 0x0018, lo: 0x94, hi: 0x96},
+	{value: 0x0008, lo: 0x97, hi: 0x97},
+	{value: 0x0018, lo: 0x98, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x32, offset 0x1c6
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0018, lo: 0x80, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x86},
+	{value: 0x0218, lo: 0x87, hi: 0x87},
+	{value: 0x0018, lo: 0x88, hi: 0x8a},
+	{value: 0x33c0, lo: 0x8b, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0208, lo: 0xa0, hi: 0xbf},
+	// Block 0x33, offset 0x1d0
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0208, lo: 0x80, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0x34, offset 0x1d3
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x0208, lo: 0x87, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xa9},
+	{value: 0x0208, lo: 0xaa, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x35, offset 0x1db
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0x36, offset 0x1de
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb8},
+	{value: 0x3308, lo: 0xb9, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x37, offset 0x1eb
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0018, lo: 0x80, hi: 0x80},
+	{value: 0x0040, lo: 0x81, hi: 0x83},
+	{value: 0x0018, lo: 0x84, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x38, offset 0x1f3
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x39, offset 0x1f7
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0028, lo: 0x9a, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0xbf},
+	// Block 0x3a, offset 0x1fe
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x3308, lo: 0x97, hi: 0x98},
+	{value: 0x3008, lo: 0x99, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x3b, offset 0x206
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0008, lo: 0x80, hi: 0x94},
+	{value: 0x3008, lo: 0x95, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xac},
+	{value: 0x3008, lo: 0xad, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x3c, offset 0x216
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0018, lo: 0xa8, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xbd},
+	{value: 0x3318, lo: 0xbe, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x3d, offset 0x222
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0040, lo: 0x80, hi: 0xbf},
+	// Block 0x3e, offset 0x224
+	{value: 0x0000, lo: 0x09},
+	{value: 0x3308, lo: 0x80, hi: 0x83},
+	{value: 0x3008, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x3008, lo: 0xbd, hi: 0xbf},
+	// Block 0x3f, offset 0x22e
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x3808, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x40, offset 0x23a
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa9},
+	{value: 0x3808, lo: 0xaa, hi: 0xaa},
+	{value: 0x3b08, lo: 0xab, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xbf},
+	// Block 0x41, offset 0x246
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa9},
+	{value: 0x3008, lo: 0xaa, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb1},
+	{value: 0x3808, lo: 0xb2, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbb},
+	{value: 0x0018, lo: 0xbc, hi: 0xbf},
+	// Block 0x42, offset 0x252
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x3008, lo: 0xa4, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbf},
+	// Block 0x43, offset 0x25a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0x44, offset 0x25f
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0e29, lo: 0x80, hi: 0x80},
+	{value: 0x0e41, lo: 0x81, hi: 0x81},
+	{value: 0x0e59, lo: 0x82, hi: 0x82},
+	{value: 0x0e71, lo: 0x83, hi: 0x83},
+	{value: 0x0e89, lo: 0x84, hi: 0x85},
+	{value: 0x0ea1, lo: 0x86, hi: 0x86},
+	{value: 0x0eb9, lo: 0x87, hi: 0x87},
+	{value: 0x057d, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x059d, lo: 0x90, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbc},
+	{value: 0x059d, lo: 0xbd, hi: 0xbf},
+	// Block 0x45, offset 0x26c
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x92},
+	{value: 0x0018, lo: 0x93, hi: 0x93},
+	{value: 0x3308, lo: 0x94, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa8},
+	{value: 0x0008, lo: 0xa9, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xb6},
+	{value: 0x3008, lo: 0xb7, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x46, offset 0x27d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3308, lo: 0x80, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbf},
+	// Block 0x47, offset 0x281
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x87},
+	{value: 0xe045, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0xe045, lo: 0x98, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0xe045, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb7},
+	{value: 0xe045, lo: 0xb8, hi: 0xbf},
+	// Block 0x48, offset 0x28c
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x3318, lo: 0x90, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xbf},
+	// Block 0x49, offset 0x290
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x88},
+	{value: 0x24c1, lo: 0x89, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x4a, offset 0x299
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0018, lo: 0x80, hi: 0xab},
+	{value: 0x24f1, lo: 0xac, hi: 0xac},
+	{value: 0x2529, lo: 0xad, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xae},
+	{value: 0x2579, lo: 0xaf, hi: 0xaf},
+	{value: 0x25b1, lo: 0xb0, hi: 0xb0},
+	{value: 0x0018, lo: 0xb1, hi: 0xbf},
+	// Block 0x4b, offset 0x2a1
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x9f},
+	{value: 0x0080, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xad},
+	{value: 0x0080, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x4c, offset 0x2a7
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0xa8},
+	{value: 0x09dd, lo: 0xa9, hi: 0xa9},
+	{value: 0x09fd, lo: 0xaa, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xbf},
+	// Block 0x4d, offset 0x2ac
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xbf},
+	// Block 0x4e, offset 0x2af
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x28c1, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0xbf},
+	// Block 0x4f, offset 0x2b3
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0e7e, lo: 0xb4, hi: 0xb4},
+	{value: 0x292a, lo: 0xb5, hi: 0xb5},
+	{value: 0x0e9e, lo: 0xb6, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0x50, offset 0x2b9
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x9b},
+	{value: 0x2941, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0xbf},
+	// Block 0x51, offset 0x2bd
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0x52, offset 0x2c1
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0x0018, lo: 0x98, hi: 0xbf},
+	// Block 0x53, offset 0x2c5
+	{value: 0x0000, lo: 0x05},
+	{value: 0xe185, lo: 0x80, hi: 0x8f},
+	{value: 0x03f5, lo: 0x90, hi: 0x9f},
+	{value: 0x0ebd, lo: 0xa0, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x54, offset 0x2cb
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xac},
+	{value: 0x0008, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x55, offset 0x2d3
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xae},
+	{value: 0xe075, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0x56, offset 0x2da
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x57, offset 0x2e5
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xbf},
+	// Block 0x58, offset 0x2ef
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xae},
+	{value: 0x0008, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x59, offset 0x2f3
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0xbf},
+	// Block 0x5a, offset 0x2f6
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9e},
+	{value: 0x0ef5, lo: 0x9f, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbf},
+	// Block 0x5b, offset 0x2fc
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb2},
+	{value: 0x0f15, lo: 0xb3, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x5c, offset 0x300
+	{value: 0x0020, lo: 0x01},
+	{value: 0x0f35, lo: 0x80, hi: 0xbf},
+	// Block 0x5d, offset 0x302
+	{value: 0x0020, lo: 0x02},
+	{value: 0x1735, lo: 0x80, hi: 0x8f},
+	{value: 0x1915, lo: 0x90, hi: 0xbf},
+	// Block 0x5e, offset 0x305
+	{value: 0x0020, lo: 0x01},
+	{value: 0x1f15, lo: 0x80, hi: 0xbf},
+	// Block 0x5f, offset 0x307
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xbf},
+	// Block 0x60, offset 0x30a
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x98},
+	{value: 0x3308, lo: 0x99, hi: 0x9a},
+	{value: 0x29e2, lo: 0x9b, hi: 0x9b},
+	{value: 0x2a0a, lo: 0x9c, hi: 0x9c},
+	{value: 0x0008, lo: 0x9d, hi: 0x9e},
+	{value: 0x2a31, lo: 0x9f, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xbf},
+	// Block 0x61, offset 0x314
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xbe},
+	{value: 0x2a69, lo: 0xbf, hi: 0xbf},
+	// Block 0x62, offset 0x317
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0040, lo: 0x80, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xb0},
+	{value: 0x2a35, lo: 0xb1, hi: 0xb1},
+	{value: 0x2a55, lo: 0xb2, hi: 0xb2},
+	{value: 0x2a75, lo: 0xb3, hi: 0xb3},
+	{value: 0x2a95, lo: 0xb4, hi: 0xb4},
+	{value: 0x2a75, lo: 0xb5, hi: 0xb5},
+	{value: 0x2ab5, lo: 0xb6, hi: 0xb6},
+	{value: 0x2ad5, lo: 0xb7, hi: 0xb7},
+	{value: 0x2af5, lo: 0xb8, hi: 0xb9},
+	{value: 0x2b15, lo: 0xba, hi: 0xbb},
+	{value: 0x2b35, lo: 0xbc, hi: 0xbd},
+	{value: 0x2b15, lo: 0xbe, hi: 0xbf},
+	// Block 0x63, offset 0x326
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x64, offset 0x32a
+	{value: 0x0030, lo: 0x04},
+	{value: 0x2aa2, lo: 0x80, hi: 0x9d},
+	{value: 0x305a, lo: 0x9e, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x30a2, lo: 0xa0, hi: 0xbf},
+	// Block 0x65, offset 0x32f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0x66, offset 0x332
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x67, offset 0x336
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0x68, offset 0x33b
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xbf},
+	// Block 0x69, offset 0x340
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x0018, lo: 0xa6, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb1},
+	{value: 0x0018, lo: 0xb2, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0x6a, offset 0x346
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0040, lo: 0x80, hi: 0x81},
+	{value: 0xe00d, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0x83},
+	{value: 0x03f5, lo: 0x84, hi: 0x84},
+	{value: 0x1329, lo: 0x85, hi: 0x85},
+	{value: 0x447d, lo: 0x86, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0xb6},
+	{value: 0x0008, lo: 0xb7, hi: 0xb7},
+	{value: 0x2009, lo: 0xb8, hi: 0xb8},
+	{value: 0x6e89, lo: 0xb9, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xbf},
+	// Block 0x6b, offset 0x352
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0x85},
+	{value: 0x3b08, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x8a},
+	{value: 0x3308, lo: 0x8b, hi: 0x8b},
+	{value: 0x0008, lo: 0x8c, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0018, lo: 0xa8, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x6c, offset 0x361
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0208, lo: 0x80, hi: 0xb1},
+	{value: 0x0108, lo: 0xb2, hi: 0xb2},
+	{value: 0x0008, lo: 0xb3, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0x6d, offset 0x367
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xbf},
+	// Block 0x6e, offset 0x36b
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3008, lo: 0x80, hi: 0x83},
+	{value: 0x3b08, lo: 0x84, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8d},
+	{value: 0x0018, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xba},
+	{value: 0x0008, lo: 0xbb, hi: 0xbb},
+	{value: 0x0018, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x6f, offset 0x37a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x70, offset 0x37f
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x91},
+	{value: 0x3008, lo: 0x92, hi: 0x92},
+	{value: 0x3808, lo: 0x93, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x71, offset 0x387
+	{value: 0x0000, lo: 0x09},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb9},
+	{value: 0x3008, lo: 0xba, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x72, offset 0x391
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3808, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x73, offset 0x39c
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x74, offset 0x3a4
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x8b},
+	{value: 0x3308, lo: 0x8c, hi: 0x8c},
+	{value: 0x3008, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0018, lo: 0x9c, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x3008, lo: 0xbd, hi: 0xbd},
+	{value: 0x0008, lo: 0xbe, hi: 0xbf},
+	// Block 0x75, offset 0x3b5
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb0},
+	{value: 0x0008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb8},
+	{value: 0x0008, lo: 0xb9, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbf},
+	// Block 0x76, offset 0x3be
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x9a},
+	{value: 0x0008, lo: 0x9b, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xaa},
+	{value: 0x3008, lo: 0xab, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb5},
+	{value: 0x3b08, lo: 0xb6, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x77, offset 0x3ce
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x88},
+	{value: 0x0008, lo: 0x89, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x90},
+	{value: 0x0008, lo: 0x91, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x78, offset 0x3db
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9b},
+	{value: 0x449d, lo: 0x9c, hi: 0x9c},
+	{value: 0x44b5, lo: 0x9d, hi: 0x9d},
+	{value: 0x2971, lo: 0x9e, hi: 0x9e},
+	{value: 0xe06d, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x44cd, lo: 0xb0, hi: 0xbf},
+	// Block 0x79, offset 0x3e5
+	{value: 0x0000, lo: 0x04},
+	{value: 0x44ed, lo: 0x80, hi: 0x8f},
+	{value: 0x450d, lo: 0x90, hi: 0x9f},
+	{value: 0x452d, lo: 0xa0, hi: 0xaf},
+	{value: 0x450d, lo: 0xb0, hi: 0xbf},
+	// Block 0x7a, offset 0x3ea
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3b08, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x7b, offset 0x3f7
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x7c, offset 0x3fb
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8a},
+	{value: 0x0018, lo: 0x8b, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x7d, offset 0x400
+	{value: 0x0020, lo: 0x01},
+	{value: 0x454d, lo: 0x80, hi: 0xbf},
+	// Block 0x7e, offset 0x402
+	{value: 0x0020, lo: 0x03},
+	{value: 0x4d4d, lo: 0x80, hi: 0x94},
+	{value: 0x4b0d, lo: 0x95, hi: 0x95},
+	{value: 0x4fed, lo: 0x96, hi: 0xbf},
+	// Block 0x7f, offset 0x406
+	{value: 0x0020, lo: 0x01},
+	{value: 0x552d, lo: 0x80, hi: 0xbf},
+	// Block 0x80, offset 0x408
+	{value: 0x0020, lo: 0x03},
+	{value: 0x5d2d, lo: 0x80, hi: 0x84},
+	{value: 0x568d, lo: 0x85, hi: 0x85},
+	{value: 0x5dcd, lo: 0x86, hi: 0xbf},
+	// Block 0x81, offset 0x40c
+	{value: 0x0020, lo: 0x08},
+	{value: 0x6b8d, lo: 0x80, hi: 0x8f},
+	{value: 0x6d4d, lo: 0x90, hi: 0x90},
+	{value: 0x6d8d, lo: 0x91, hi: 0xab},
+	{value: 0x6ea1, lo: 0xac, hi: 0xac},
+	{value: 0x70ed, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x710d, lo: 0xb0, hi: 0xbf},
+	// Block 0x82, offset 0x415
+	{value: 0x0020, lo: 0x05},
+	{value: 0x730d, lo: 0x80, hi: 0xad},
+	{value: 0x656d, lo: 0xae, hi: 0xae},
+	{value: 0x78cd, lo: 0xaf, hi: 0xb5},
+	{value: 0x6f8d, lo: 0xb6, hi: 0xb6},
+	{value: 0x79ad, lo: 0xb7, hi: 0xbf},
+	// Block 0x83, offset 0x41b
+	{value: 0x0028, lo: 0x03},
+	{value: 0x7c21, lo: 0x80, hi: 0x82},
+	{value: 0x7be1, lo: 0x83, hi: 0x83},
+	{value: 0x7c99, lo: 0x84, hi: 0xbf},
+	// Block 0x84, offset 0x41f
+	{value: 0x0038, lo: 0x0f},
+	{value: 0x9db1, lo: 0x80, hi: 0x83},
+	{value: 0x9e59, lo: 0x84, hi: 0x85},
+	{value: 0x9e91, lo: 0x86, hi: 0x87},
+	{value: 0x9ec9, lo: 0x88, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0xa089, lo: 0x92, hi: 0x97},
+	{value: 0xa1a1, lo: 0x98, hi: 0x9c},
+	{value: 0xa281, lo: 0x9d, hi: 0xb3},
+	{value: 0x9d41, lo: 0xb4, hi: 0xb4},
+	{value: 0x9db1, lo: 0xb5, hi: 0xb5},
+	{value: 0xa789, lo: 0xb6, hi: 0xbb},
+	{value: 0xa869, lo: 0xbc, hi: 0xbc},
+	{value: 0xa7f9, lo: 0xbd, hi: 0xbd},
+	{value: 0xa8d9, lo: 0xbe, hi: 0xbf},
+	// Block 0x85, offset 0x42f
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbb},
+	{value: 0x0008, lo: 0xbc, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0x86, offset 0x439
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0xbf},
+	// Block 0x87, offset 0x43e
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x88, offset 0x441
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0x89, offset 0x447
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa0},
+	{value: 0x0040, lo: 0xa1, hi: 0xbf},
+	// Block 0x8a, offset 0x44e
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x8b, offset 0x453
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x9c},
+	{value: 0x0040, lo: 0x9d, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x8c, offset 0x457
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x8d, offset 0x45d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xac},
+	{value: 0x0008, lo: 0xad, hi: 0xbf},
+	// Block 0x8e, offset 0x462
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x8f, offset 0x46b
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x90, offset 0x470
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0xbf},
+	// Block 0x91, offset 0x476
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe145, lo: 0x80, hi: 0x87},
+	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
+	{value: 0xe145, lo: 0x90, hi: 0x97},
+	{value: 0x8b0d, lo: 0x98, hi: 0x9f},
+	{value: 0x8b25, lo: 0xa0, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xbf},
+	// Block 0x92, offset 0x47d
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x8b25, lo: 0xb0, hi: 0xb7},
+	{value: 0x8b0d, lo: 0xb8, hi: 0xbf},
+	// Block 0x93, offset 0x484
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe145, lo: 0x80, hi: 0x87},
+	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
+	{value: 0xe145, lo: 0x90, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x94, offset 0x48b
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x95, offset 0x48f
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xae},
+	{value: 0x0018, lo: 0xaf, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0x96, offset 0x494
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x97, offset 0x497
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xbf},
+	// Block 0x98, offset 0x49c
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0808, lo: 0x80, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x87},
+	{value: 0x0808, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0808, lo: 0x8a, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb6},
+	{value: 0x0808, lo: 0xb7, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbb},
+	{value: 0x0808, lo: 0xbc, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbe},
+	{value: 0x0808, lo: 0xbf, hi: 0xbf},
+	// Block 0x99, offset 0x4a8
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x96},
+	{value: 0x0818, lo: 0x97, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb6},
+	{value: 0x0818, lo: 0xb7, hi: 0xbf},
+	// Block 0x9a, offset 0x4ae
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xa6},
+	{value: 0x0818, lo: 0xa7, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0x9b, offset 0x4b3
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb3},
+	{value: 0x0808, lo: 0xb4, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xba},
+	{value: 0x0818, lo: 0xbb, hi: 0xbf},
+	// Block 0x9c, offset 0x4ba
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0818, lo: 0x96, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbe},
+	{value: 0x0818, lo: 0xbf, hi: 0xbf},
+	// Block 0x9d, offset 0x4c2
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbb},
+	{value: 0x0818, lo: 0xbc, hi: 0xbd},
+	{value: 0x0808, lo: 0xbe, hi: 0xbf},
+	// Block 0x9e, offset 0x4c7
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0818, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x91},
+	{value: 0x0818, lo: 0x92, hi: 0xbf},
+	// Block 0x9f, offset 0x4cb
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0808, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8b},
+	{value: 0x3308, lo: 0x8c, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x94},
+	{value: 0x0808, lo: 0x95, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0x98},
+	{value: 0x0808, lo: 0x99, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xa0, offset 0x4db
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0818, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x0818, lo: 0x90, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xbc},
+	{value: 0x0818, lo: 0xbd, hi: 0xbf},
+	// Block 0xa1, offset 0x4e2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0x9c},
+	{value: 0x0818, lo: 0x9d, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xa2, offset 0x4e6
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb8},
+	{value: 0x0018, lo: 0xb9, hi: 0xbf},
+	// Block 0xa3, offset 0x4ea
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0x0818, lo: 0x98, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb7},
+	{value: 0x0818, lo: 0xb8, hi: 0xbf},
+	// Block 0xa4, offset 0x4f1
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0808, lo: 0x80, hi: 0xbf},
+	// Block 0xa5, offset 0x4f3
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0808, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0xbf},
+	// Block 0xa6, offset 0x4f6
+	{value: 0x0000, lo: 0x02},
+	{value: 0x03dd, lo: 0x80, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xbf},
+	// Block 0xa7, offset 0x4f9
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb9},
+	{value: 0x0818, lo: 0xba, hi: 0xbf},
+	// Block 0xa8, offset 0x4fd
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0908, lo: 0x80, hi: 0x80},
+	{value: 0x0a08, lo: 0x81, hi: 0xa1},
+	{value: 0x0c08, lo: 0xa2, hi: 0xa2},
+	{value: 0x0a08, lo: 0xa3, hi: 0xa3},
+	{value: 0x3308, lo: 0xa4, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0808, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xa9, offset 0x506
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0818, lo: 0xa0, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xaa, offset 0x50a
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0808, lo: 0x80, hi: 0x9c},
+	{value: 0x0818, lo: 0x9d, hi: 0xa6},
+	{value: 0x0808, lo: 0xa7, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0a08, lo: 0xb0, hi: 0xb2},
+	{value: 0x0c08, lo: 0xb3, hi: 0xb3},
+	{value: 0x0a08, lo: 0xb4, hi: 0xbf},
+	// Block 0xab, offset 0x512
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0a08, lo: 0x80, hi: 0x84},
+	{value: 0x0808, lo: 0x85, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x90},
+	{value: 0x0a18, lo: 0x91, hi: 0x93},
+	{value: 0x0c18, lo: 0x94, hi: 0x94},
+	{value: 0x0818, lo: 0x95, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xac, offset 0x51a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xad, offset 0x51e
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbf},
+	// Block 0xae, offset 0x524
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x85},
+	{value: 0x3b08, lo: 0x86, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x91},
+	{value: 0x0018, lo: 0x92, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xaf, offset 0x52d
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb6},
+	{value: 0x3008, lo: 0xb7, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0xb0, offset 0x539
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x81},
+	{value: 0x0040, lo: 0x82, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xb1, offset 0x540
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xb2},
+	{value: 0x3b08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xb5},
+	{value: 0x0008, lo: 0xb6, hi: 0xbf},
+	// Block 0xb2, offset 0x549
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0018, lo: 0x80, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x3008, lo: 0x85, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb5},
+	{value: 0x0008, lo: 0xb6, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xb3, offset 0x553
+	{value: 0x0000, lo: 0x06},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xbe},
+	{value: 0x3008, lo: 0xbf, hi: 0xbf},
+	// Block 0xb4, offset 0x55a
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x3808, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x88},
+	{value: 0x3308, lo: 0x89, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0xb5, offset 0x568
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x92},
+	{value: 0x0008, lo: 0x93, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb1},
+	{value: 0x3008, lo: 0xb2, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x3808, lo: 0xb5, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xb6, offset 0x575
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9e},
+	{value: 0x0008, lo: 0x9f, hi: 0xa8},
+	{value: 0x0018, lo: 0xa9, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0xb7, offset 0x582
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x3308, lo: 0x9f, hi: 0x9f},
+	{value: 0x3008, lo: 0xa0, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xa9},
+	{value: 0x3b08, lo: 0xaa, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xb8, offset 0x58b
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbf},
+	// Block 0xb9, offset 0x58f
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x3b08, lo: 0x82, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x84},
+	{value: 0x3008, lo: 0x85, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x8a},
+	{value: 0x0018, lo: 0x8b, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0x9d},
+	{value: 0x3308, lo: 0x9e, hi: 0x9e},
+	{value: 0x0008, lo: 0x9f, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xba, offset 0x59f
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb8},
+	{value: 0x3008, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0xbb, offset 0x5a7
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x81},
+	{value: 0x3b08, lo: 0x82, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x85},
+	{value: 0x0018, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xbc, offset 0x5b2
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xbd, offset 0x5bb
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x9b},
+	{value: 0x3308, lo: 0x9c, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0xbf},
+	// Block 0xbe, offset 0x5c1
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xbf, offset 0x5c9
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xbf},
+	// Block 0xc0, offset 0x5d2
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb5},
+	{value: 0x3808, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xc1, offset 0x5dd
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0xbf},
+	// Block 0xc2, offset 0x5e0
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9f},
+	{value: 0x3008, lo: 0xa0, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xaa},
+	{value: 0x3b08, lo: 0xab, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbf},
+	// Block 0xc3, offset 0x5ec
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0xc4, offset 0x5f5
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x049d, lo: 0xa0, hi: 0xbf},
+	// Block 0xc5, offset 0x5f8
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0xc6, offset 0x5fd
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xbf},
+	// Block 0xc7, offset 0x602
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x3008, lo: 0x91, hi: 0x93},
+	{value: 0x3308, lo: 0x94, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0x99},
+	{value: 0x3308, lo: 0x9a, hi: 0x9b},
+	{value: 0x3008, lo: 0x9c, hi: 0x9f},
+	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xa1},
+	{value: 0x0018, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xa3},
+	{value: 0x3008, lo: 0xa4, hi: 0xa4},
+	{value: 0x0040, lo: 0xa5, hi: 0xbf},
+	// Block 0xc8, offset 0x60f
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb8},
+	{value: 0x3008, lo: 0xb9, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xc9, offset 0x61a
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x3b08, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x3308, lo: 0x91, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x98},
+	{value: 0x3308, lo: 0x99, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0xbf},
+	// Block 0xca, offset 0x623
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x3308, lo: 0x8a, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x98},
+	{value: 0x3b08, lo: 0x99, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9c},
+	{value: 0x0008, lo: 0x9d, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0xa2},
+	{value: 0x0040, lo: 0xa3, hi: 0xbf},
+	// Block 0xcb, offset 0x62d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xcc, offset 0x630
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xcd, offset 0x63a
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xbf},
+	// Block 0xce, offset 0x643
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xa9},
+	{value: 0x3308, lo: 0xaa, hi: 0xb0},
+	{value: 0x3008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xcf, offset 0x64f
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0xd0, offset 0x65c
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x3308, lo: 0x80, hi: 0x83},
+	{value: 0x3b08, lo: 0x84, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xbf},
+	// Block 0xd1, offset 0x669
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x3008, lo: 0x8a, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x92},
+	{value: 0x3008, lo: 0x93, hi: 0x94},
+	{value: 0x3308, lo: 0x95, hi: 0x95},
+	{value: 0x3008, lo: 0x96, hi: 0x96},
+	{value: 0x3b08, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xbf},
+	// Block 0xd2, offset 0x677
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xd3, offset 0x67e
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xd4, offset 0x682
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xd5, offset 0x685
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0xd6, offset 0x68a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0xbf},
+	// Block 0xd7, offset 0x68d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0340, lo: 0xb0, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xd8, offset 0x692
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0xbf},
+	// Block 0xd9, offset 0x695
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0xda, offset 0x69c
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb4},
+	{value: 0x0018, lo: 0xb5, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0xdb, offset 0x6a3
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0xdc, offset 0x6a7
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0018, lo: 0x84, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbf},
+	// Block 0xdd, offset 0x6b2
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0xbf},
+	// Block 0xde, offset 0x6b5
+	{value: 0x0000, lo: 0x02},
+	{value: 0xe105, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0xdf, offset 0x6b8
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0xbf},
+	// Block 0xe0, offset 0x6bb
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8e},
+	{value: 0x3308, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x3008, lo: 0x91, hi: 0xbf},
+	// Block 0xe1, offset 0x6c1
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3008, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8e},
+	{value: 0x3308, lo: 0x8f, hi: 0x92},
+	{value: 0x0008, lo: 0x93, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xe2, offset 0x6c7
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa1},
+	{value: 0x0018, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xbf},
+	// Block 0xe3, offset 0x6cd
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0xe4, offset 0x6d0
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xbf},
+	// Block 0xe5, offset 0x6d3
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xbf},
+	// Block 0xe6, offset 0x6d6
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x92},
+	{value: 0x0040, lo: 0x93, hi: 0xa3},
+	{value: 0x0008, lo: 0xa4, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0xe7, offset 0x6dd
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0xe8, offset 0x6e0
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0xe9, offset 0x6e5
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0018, lo: 0x9c, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x03c0, lo: 0xa0, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xbf},
+	// Block 0xea, offset 0x6ef
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0xeb, offset 0x6f2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa8},
+	{value: 0x0018, lo: 0xa9, hi: 0xbf},
+	// Block 0xec, offset 0x6f6
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0018, lo: 0x80, hi: 0x9d},
+	{value: 0xb5b9, lo: 0x9e, hi: 0x9e},
+	{value: 0xb601, lo: 0x9f, hi: 0x9f},
+	{value: 0xb649, lo: 0xa0, hi: 0xa0},
+	{value: 0xb6b1, lo: 0xa1, hi: 0xa1},
+	{value: 0xb719, lo: 0xa2, hi: 0xa2},
+	{value: 0xb781, lo: 0xa3, hi: 0xa3},
+	{value: 0xb7e9, lo: 0xa4, hi: 0xa4},
+	{value: 0x3018, lo: 0xa5, hi: 0xa6},
+	{value: 0x3318, lo: 0xa7, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xac},
+	{value: 0x3018, lo: 0xad, hi: 0xb2},
+	{value: 0x0340, lo: 0xb3, hi: 0xba},
+	{value: 0x3318, lo: 0xbb, hi: 0xbf},
+	// Block 0xed, offset 0x705
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3318, lo: 0x80, hi: 0x82},
+	{value: 0x0018, lo: 0x83, hi: 0x84},
+	{value: 0x3318, lo: 0x85, hi: 0x8b},
+	{value: 0x0018, lo: 0x8c, hi: 0xa9},
+	{value: 0x3318, lo: 0xaa, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xba},
+	{value: 0xb851, lo: 0xbb, hi: 0xbb},
+	{value: 0xb899, lo: 0xbc, hi: 0xbc},
+	{value: 0xb8e1, lo: 0xbd, hi: 0xbd},
+	{value: 0xb949, lo: 0xbe, hi: 0xbe},
+	{value: 0xb9b1, lo: 0xbf, hi: 0xbf},
+	// Block 0xee, offset 0x711
+	{value: 0x0000, lo: 0x03},
+	{value: 0xba19, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xbf},
+	// Block 0xef, offset 0x715
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x81},
+	{value: 0x3318, lo: 0x82, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0xbf},
+	// Block 0xf0, offset 0x71a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0xf1, offset 0x71e
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xf2, offset 0x723
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3308, lo: 0x80, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbf},
+	// Block 0xf3, offset 0x727
+	{value: 0x0000, lo: 0x04},
+	{value: 0x3308, lo: 0x80, hi: 0xac},
+	{value: 0x0018, lo: 0xad, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0xf4, offset 0x72c
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa0},
+	{value: 0x3308, lo: 0xa1, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0xf5, offset 0x735
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3308, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x3308, lo: 0x88, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xa4},
+	{value: 0x0040, lo: 0xa5, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xbf},
+	// Block 0xf6, offset 0x740
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0008, lo: 0xb7, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0xf7, offset 0x746
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x8e},
+	{value: 0x0018, lo: 0x8f, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0xbf},
+	// Block 0xf8, offset 0x74c
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xf9, offset 0x752
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0808, lo: 0x80, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x86},
+	{value: 0x0818, lo: 0x87, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0xbf},
+	// Block 0xfa, offset 0x758
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0a08, lo: 0x80, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x8a},
+	{value: 0x0b08, lo: 0x8b, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9d},
+	{value: 0x0818, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xfb, offset 0x761
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0xb0},
+	{value: 0x0818, lo: 0xb1, hi: 0xbf},
+	// Block 0xfc, offset 0x764
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0818, lo: 0x80, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0xfd, offset 0x767
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0818, lo: 0x81, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0xfe, offset 0x76b
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0xff, offset 0x76f
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x100, offset 0x773
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xb0},
+	{value: 0x0018, lo: 0xb1, hi: 0xbf},
+	// Block 0x101, offset 0x779
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x0018, lo: 0x91, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0x102, offset 0x77f
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x8f},
+	{value: 0xc1d9, lo: 0x90, hi: 0x90},
+	{value: 0x0018, lo: 0x91, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xbf},
+	// Block 0x103, offset 0x784
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0xa5},
+	{value: 0x0018, lo: 0xa6, hi: 0xbf},
+	// Block 0x104, offset 0x787
+	{value: 0x0000, lo: 0x0f},
+	{value: 0xc801, lo: 0x80, hi: 0x80},
+	{value: 0xc851, lo: 0x81, hi: 0x81},
+	{value: 0xc8a1, lo: 0x82, hi: 0x82},
+	{value: 0xc8f1, lo: 0x83, hi: 0x83},
+	{value: 0xc941, lo: 0x84, hi: 0x84},
+	{value: 0xc991, lo: 0x85, hi: 0x85},
+	{value: 0xc9e1, lo: 0x86, hi: 0x86},
+	{value: 0xca31, lo: 0x87, hi: 0x87},
+	{value: 0xca81, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0xcad1, lo: 0x90, hi: 0x90},
+	{value: 0xcaf1, lo: 0x91, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xbf},
+	// Block 0x105, offset 0x797
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x106, offset 0x79e
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x107, offset 0x7a1
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xbf},
+	// Block 0x108, offset 0x7a6
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x109, offset 0x7aa
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbf},
+	// Block 0x10a, offset 0x7b0
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xbf},
+	// Block 0x10b, offset 0x7b5
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0xbf},
+	// Block 0x10c, offset 0x7b9
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xb2},
+	{value: 0x0018, lo: 0xb3, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbf},
+	// Block 0x10d, offset 0x7bf
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0xa2},
+	{value: 0x0040, lo: 0xa3, hi: 0xa4},
+	{value: 0x0018, lo: 0xa5, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xbf},
+	// Block 0x10e, offset 0x7c5
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0xbf},
+	// Block 0x10f, offset 0x7c9
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x110, offset 0x7d2
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0xbf},
+	// Block 0x111, offset 0x7d7
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0xbf},
+	// Block 0x112, offset 0x7da
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x113, offset 0x7dd
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x114, offset 0x7e1
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x115, offset 0x7e5
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xa0},
+	{value: 0x0040, lo: 0xa1, hi: 0xbf},
+	// Block 0x116, offset 0x7e8
+	{value: 0x0020, lo: 0x0f},
+	{value: 0xded1, lo: 0x80, hi: 0x89},
+	{value: 0x8e35, lo: 0x8a, hi: 0x8a},
+	{value: 0xe011, lo: 0x8b, hi: 0x9c},
+	{value: 0x8e55, lo: 0x9d, hi: 0x9d},
+	{value: 0xe251, lo: 0x9e, hi: 0xa2},
+	{value: 0x8e75, lo: 0xa3, hi: 0xa3},
+	{value: 0xe2f1, lo: 0xa4, hi: 0xab},
+	{value: 0x7f0d, lo: 0xac, hi: 0xac},
+	{value: 0xe3f1, lo: 0xad, hi: 0xaf},
+	{value: 0x8e95, lo: 0xb0, hi: 0xb0},
+	{value: 0xe451, lo: 0xb1, hi: 0xb6},
+	{value: 0x8eb5, lo: 0xb7, hi: 0xb9},
+	{value: 0xe511, lo: 0xba, hi: 0xba},
+	{value: 0x8f15, lo: 0xbb, hi: 0xbb},
+	{value: 0xe531, lo: 0xbc, hi: 0xbf},
+	// Block 0x117, offset 0x7f8
+	{value: 0x0020, lo: 0x10},
+	{value: 0x93b5, lo: 0x80, hi: 0x80},
+	{value: 0xf0b1, lo: 0x81, hi: 0x86},
+	{value: 0x93d5, lo: 0x87, hi: 0x8a},
+	{value: 0xda11, lo: 0x8b, hi: 0x8b},
+	{value: 0xf171, lo: 0x8c, hi: 0x96},
+	{value: 0x9455, lo: 0x97, hi: 0x97},
+	{value: 0xf2d1, lo: 0x98, hi: 0xa3},
+	{value: 0x9475, lo: 0xa4, hi: 0xa6},
+	{value: 0xf451, lo: 0xa7, hi: 0xaa},
+	{value: 0x94d5, lo: 0xab, hi: 0xab},
+	{value: 0xf4d1, lo: 0xac, hi: 0xac},
+	{value: 0x94f5, lo: 0xad, hi: 0xad},
+	{value: 0xf4f1, lo: 0xae, hi: 0xaf},
+	{value: 0x9515, lo: 0xb0, hi: 0xb1},
+	{value: 0xf531, lo: 0xb2, hi: 0xbe},
+	{value: 0x2040, lo: 0xbf, hi: 0xbf},
+	// Block 0x118, offset 0x809
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0340, lo: 0x81, hi: 0x81},
+	{value: 0x0040, lo: 0x82, hi: 0x9f},
+	{value: 0x0340, lo: 0xa0, hi: 0xbf},
+	// Block 0x119, offset 0x80e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0340, lo: 0x80, hi: 0xbf},
+	// Block 0x11a, offset 0x810
+	{value: 0x0000, lo: 0x01},
+	{value: 0x33c0, lo: 0x80, hi: 0xbf},
+	// Block 0x11b, offset 0x812
+	{value: 0x0000, lo: 0x02},
+	{value: 0x33c0, lo: 0x80, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+}
+
+// Total table size 42780 bytes (41KiB); checksum: 29936AB9
diff --git a/vendor/golang.org/x/net/idna/tables12.00.go b/vendor/golang.org/x/net/idna/tables12.00.go
deleted file mode 100644
index f4b8ea3..0000000
--- a/vendor/golang.org/x/net/idna/tables12.00.go
+++ /dev/null
@@ -1,4733 +0,0 @@
-// Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
-
-// +build go1.14
-
-package idna
-
-// UnicodeVersion is the Unicode version from which the tables in this package are derived.
-const UnicodeVersion = "12.0.0"
-
-var mappings string = "" + // Size: 8178 bytes
-	"\x00\x01 \x03 ̈\x01a\x03 ̄\x012\x013\x03 ́\x03 ̧\x011\x01o\x051⁄4\x051⁄2" +
-	"\x053⁄4\x03i̇\x03l·\x03ʼn\x01s\x03dž\x03ⱥ\x03ⱦ\x01h\x01j\x01r\x01w\x01y" +
-	"\x03 ̆\x03 ̇\x03 ̊\x03 ̨\x03 ̃\x03 ̋\x01l\x01x\x04̈́\x03 ι\x01;\x05 ̈́" +
-	"\x04եւ\x04اٴ\x04وٴ\x04ۇٴ\x04يٴ\x06क़\x06ख़\x06ग़\x06ज़\x06ड़\x06ढ़\x06फ़" +
-	"\x06य़\x06ড়\x06ঢ়\x06য়\x06ਲ਼\x06ਸ਼\x06ਖ਼\x06ਗ਼\x06ਜ਼\x06ਫ਼\x06ଡ଼\x06ଢ଼" +
-	"\x06ํา\x06ໍາ\x06ຫນ\x06ຫມ\x06གྷ\x06ཌྷ\x06དྷ\x06བྷ\x06ཛྷ\x06ཀྵ\x06ཱི\x06ཱུ" +
-	"\x06ྲྀ\x09ྲཱྀ\x06ླྀ\x09ླཱྀ\x06ཱྀ\x06ྒྷ\x06ྜྷ\x06ྡྷ\x06ྦྷ\x06ྫྷ\x06ྐྵ\x02" +
-	"в\x02д\x02о\x02с\x02т\x02ъ\x02ѣ\x02æ\x01b\x01d\x01e\x02ǝ\x01g\x01i\x01k" +
-	"\x01m\x01n\x02ȣ\x01p\x01t\x01u\x02ɐ\x02ɑ\x02ə\x02ɛ\x02ɜ\x02ŋ\x02ɔ\x02ɯ" +
-	"\x01v\x02β\x02γ\x02δ\x02φ\x02χ\x02ρ\x02н\x02ɒ\x01c\x02ɕ\x02ð\x01f\x02ɟ" +
-	"\x02ɡ\x02ɥ\x02ɨ\x02ɩ\x02ɪ\x02ʝ\x02ɭ\x02ʟ\x02ɱ\x02ɰ\x02ɲ\x02ɳ\x02ɴ\x02ɵ" +
-	"\x02ɸ\x02ʂ\x02ʃ\x02ƫ\x02ʉ\x02ʊ\x02ʋ\x02ʌ\x01z\x02ʐ\x02ʑ\x02ʒ\x02θ\x02ss" +
-	"\x02ά\x02έ\x02ή\x02ί\x02ό\x02ύ\x02ώ\x05ἀι\x05ἁι\x05ἂι\x05ἃι\x05ἄι\x05ἅι" +
-	"\x05ἆι\x05ἇι\x05ἠι\x05ἡι\x05ἢι\x05ἣι\x05ἤι\x05ἥι\x05ἦι\x05ἧι\x05ὠι\x05ὡι" +
-	"\x05ὢι\x05ὣι\x05ὤι\x05ὥι\x05ὦι\x05ὧι\x05ὰι\x04αι\x04άι\x05ᾶι\x02ι\x05 ̈͂" +
-	"\x05ὴι\x04ηι\x04ήι\x05ῆι\x05 ̓̀\x05 ̓́\x05 ̓͂\x02ΐ\x05 ̔̀\x05 ̔́\x05 ̔͂" +
-	"\x02ΰ\x05 ̈̀\x01`\x05ὼι\x04ωι\x04ώι\x05ῶι\x06′′\x09′′′\x06‵‵\x09‵‵‵\x02!" +
-	"!\x02??\x02?!\x02!?\x0c′′′′\x010\x014\x015\x016\x017\x018\x019\x01+\x01=" +
-	"\x01(\x01)\x02rs\x02ħ\x02no\x01q\x02sm\x02tm\x02ω\x02å\x02א\x02ב\x02ג" +
-	"\x02ד\x02π\x051⁄7\x051⁄9\x061⁄10\x051⁄3\x052⁄3\x051⁄5\x052⁄5\x053⁄5\x054" +
-	"⁄5\x051⁄6\x055⁄6\x051⁄8\x053⁄8\x055⁄8\x057⁄8\x041⁄\x02ii\x02iv\x02vi" +
-	"\x04viii\x02ix\x02xi\x050⁄3\x06∫∫\x09∫∫∫\x06∮∮\x09∮∮∮\x0210\x0211\x0212" +
-	"\x0213\x0214\x0215\x0216\x0217\x0218\x0219\x0220\x04(10)\x04(11)\x04(12)" +
-	"\x04(13)\x04(14)\x04(15)\x04(16)\x04(17)\x04(18)\x04(19)\x04(20)\x0c∫∫∫∫" +
-	"\x02==\x05⫝̸\x02ɫ\x02ɽ\x02ȿ\x02ɀ\x01.\x04 ゙\x04 ゚\x06より\x06コト\x05(ᄀ)\x05" +
-	"(ᄂ)\x05(ᄃ)\x05(ᄅ)\x05(ᄆ)\x05(ᄇ)\x05(ᄉ)\x05(ᄋ)\x05(ᄌ)\x05(ᄎ)\x05(ᄏ)\x05(ᄐ" +
-	")\x05(ᄑ)\x05(ᄒ)\x05(가)\x05(나)\x05(다)\x05(라)\x05(마)\x05(바)\x05(사)\x05(아)" +
-	"\x05(자)\x05(차)\x05(카)\x05(타)\x05(파)\x05(하)\x05(주)\x08(오전)\x08(오후)\x05(一)" +
-	"\x05(二)\x05(三)\x05(四)\x05(五)\x05(六)\x05(七)\x05(八)\x05(九)\x05(十)\x05(月)" +
-	"\x05(火)\x05(水)\x05(木)\x05(金)\x05(土)\x05(日)\x05(株)\x05(有)\x05(社)\x05(名)" +
-	"\x05(特)\x05(財)\x05(祝)\x05(労)\x05(代)\x05(呼)\x05(学)\x05(監)\x05(企)\x05(資)" +
-	"\x05(協)\x05(祭)\x05(休)\x05(自)\x05(至)\x0221\x0222\x0223\x0224\x0225\x0226" +
-	"\x0227\x0228\x0229\x0230\x0231\x0232\x0233\x0234\x0235\x06참고\x06주의\x0236" +
-	"\x0237\x0238\x0239\x0240\x0241\x0242\x0243\x0244\x0245\x0246\x0247\x0248" +
-	"\x0249\x0250\x041月\x042月\x043月\x044月\x045月\x046月\x047月\x048月\x049月\x0510" +
-	"月\x0511月\x0512月\x02hg\x02ev\x0cアパート\x0cアルファ\x0cアンペア\x09アール\x0cイニング\x09" +
-	"インチ\x09ウォン\x0fエスクード\x0cエーカー\x09オンス\x09オーム\x09カイリ\x0cカラット\x0cカロリー\x09ガロ" +
-	"ン\x09ガンマ\x06ギガ\x09ギニー\x0cキュリー\x0cギルダー\x06キロ\x0fキログラム\x12キロメートル\x0fキロワッ" +
-	"ト\x09グラム\x0fグラムトン\x0fクルゼイロ\x0cクローネ\x09ケース\x09コルナ\x09コーポ\x0cサイクル\x0fサンチ" +
-	"ーム\x0cシリング\x09センチ\x09セント\x09ダース\x06デシ\x06ドル\x06トン\x06ナノ\x09ノット\x09ハイツ" +
-	"\x0fパーセント\x09パーツ\x0cバーレル\x0fピアストル\x09ピクル\x06ピコ\x06ビル\x0fファラッド\x0cフィート" +
-	"\x0fブッシェル\x09フラン\x0fヘクタール\x06ペソ\x09ペニヒ\x09ヘルツ\x09ペンス\x09ページ\x09ベータ\x0cポイ" +
-	"ント\x09ボルト\x06ホン\x09ポンド\x09ホール\x09ホーン\x0cマイクロ\x09マイル\x09マッハ\x09マルク\x0fマ" +
-	"ンション\x0cミクロン\x06ミリ\x0fミリバール\x06メガ\x0cメガトン\x0cメートル\x09ヤード\x09ヤール\x09ユアン" +
-	"\x0cリットル\x06リラ\x09ルピー\x0cルーブル\x06レム\x0fレントゲン\x09ワット\x040点\x041点\x042点" +
-	"\x043点\x044点\x045点\x046点\x047点\x048点\x049点\x0510点\x0511点\x0512点\x0513点" +
-	"\x0514点\x0515点\x0516点\x0517点\x0518点\x0519点\x0520点\x0521点\x0522点\x0523点" +
-	"\x0524点\x02da\x02au\x02ov\x02pc\x02dm\x02iu\x06平成\x06昭和\x06大正\x06明治\x0c株" +
-	"式会社\x02pa\x02na\x02ma\x02ka\x02kb\x02mb\x02gb\x04kcal\x02pf\x02nf\x02m" +
-	"g\x02kg\x02hz\x02ml\x02dl\x02kl\x02fm\x02nm\x02mm\x02cm\x02km\x02m2\x02m" +
-	"3\x05m∕s\x06m∕s2\x07rad∕s\x08rad∕s2\x02ps\x02ns\x02ms\x02pv\x02nv\x02mv" +
-	"\x02kv\x02pw\x02nw\x02mw\x02kw\x02bq\x02cc\x02cd\x06c∕kg\x02db\x02gy\x02" +
-	"ha\x02hp\x02in\x02kk\x02kt\x02lm\x02ln\x02lx\x02ph\x02pr\x02sr\x02sv\x02" +
-	"wb\x05v∕m\x05a∕m\x041日\x042日\x043日\x044日\x045日\x046日\x047日\x048日\x049日" +
-	"\x0510日\x0511日\x0512日\x0513日\x0514日\x0515日\x0516日\x0517日\x0518日\x0519日" +
-	"\x0520日\x0521日\x0522日\x0523日\x0524日\x0525日\x0526日\x0527日\x0528日\x0529日" +
-	"\x0530日\x0531日\x02ь\x02ɦ\x02ɬ\x02ʞ\x02ʇ\x02œ\x04𤋮\x04𢡊\x04𢡄\x04𣏕\x04𥉉" +
-	"\x04𥳐\x04𧻓\x02ff\x02fi\x02fl\x02st\x04մն\x04մե\x04մի\x04վն\x04մխ\x04יִ" +
-	"\x04ײַ\x02ע\x02ה\x02כ\x02ל\x02ם\x02ר\x02ת\x04שׁ\x04שׂ\x06שּׁ\x06שּׂ\x04א" +
-	"ַ\x04אָ\x04אּ\x04בּ\x04גּ\x04דּ\x04הּ\x04וּ\x04זּ\x04טּ\x04יּ\x04ךּ\x04" +
-	"כּ\x04לּ\x04מּ\x04נּ\x04סּ\x04ףּ\x04פּ\x04צּ\x04קּ\x04רּ\x04שּ\x04תּ" +
-	"\x04וֹ\x04בֿ\x04כֿ\x04פֿ\x04אל\x02ٱ\x02ٻ\x02پ\x02ڀ\x02ٺ\x02ٿ\x02ٹ\x02ڤ" +
-	"\x02ڦ\x02ڄ\x02ڃ\x02چ\x02ڇ\x02ڍ\x02ڌ\x02ڎ\x02ڈ\x02ژ\x02ڑ\x02ک\x02گ\x02ڳ" +
-	"\x02ڱ\x02ں\x02ڻ\x02ۀ\x02ہ\x02ھ\x02ے\x02ۓ\x02ڭ\x02ۇ\x02ۆ\x02ۈ\x02ۋ\x02ۅ" +
-	"\x02ۉ\x02ې\x02ى\x04ئا\x04ئە\x04ئو\x04ئۇ\x04ئۆ\x04ئۈ\x04ئې\x04ئى\x02ی\x04" +
-	"ئج\x04ئح\x04ئم\x04ئي\x04بج\x04بح\x04بخ\x04بم\x04بى\x04بي\x04تج\x04تح" +
-	"\x04تخ\x04تم\x04تى\x04تي\x04ثج\x04ثم\x04ثى\x04ثي\x04جح\x04جم\x04حج\x04حم" +
-	"\x04خج\x04خح\x04خم\x04سج\x04سح\x04سخ\x04سم\x04صح\x04صم\x04ضج\x04ضح\x04ضخ" +
-	"\x04ضم\x04طح\x04طم\x04ظم\x04عج\x04عم\x04غج\x04غم\x04فج\x04فح\x04فخ\x04فم" +
-	"\x04فى\x04في\x04قح\x04قم\x04قى\x04قي\x04كا\x04كج\x04كح\x04كخ\x04كل\x04كم" +
-	"\x04كى\x04كي\x04لج\x04لح\x04لخ\x04لم\x04لى\x04لي\x04مج\x04مح\x04مخ\x04مم" +
-	"\x04مى\x04مي\x04نج\x04نح\x04نخ\x04نم\x04نى\x04ني\x04هج\x04هم\x04هى\x04هي" +
-	"\x04يج\x04يح\x04يخ\x04يم\x04يى\x04يي\x04ذٰ\x04رٰ\x04ىٰ\x05 ٌّ\x05 ٍّ\x05" +
-	" َّ\x05 ُّ\x05 ِّ\x05 ّٰ\x04ئر\x04ئز\x04ئن\x04بر\x04بز\x04بن\x04تر\x04تز" +
-	"\x04تن\x04ثر\x04ثز\x04ثن\x04ما\x04نر\x04نز\x04نن\x04ير\x04يز\x04ين\x04ئخ" +
-	"\x04ئه\x04به\x04ته\x04صخ\x04له\x04نه\x04هٰ\x04يه\x04ثه\x04سه\x04شم\x04شه" +
-	"\x06ـَّ\x06ـُّ\x06ـِّ\x04طى\x04طي\x04عى\x04عي\x04غى\x04غي\x04سى\x04سي" +
-	"\x04شى\x04شي\x04حى\x04حي\x04جى\x04جي\x04خى\x04خي\x04صى\x04صي\x04ضى\x04ضي" +
-	"\x04شج\x04شح\x04شخ\x04شر\x04سر\x04صر\x04ضر\x04اً\x06تجم\x06تحج\x06تحم" +
-	"\x06تخم\x06تمج\x06تمح\x06تمخ\x06جمح\x06حمي\x06حمى\x06سحج\x06سجح\x06سجى" +
-	"\x06سمح\x06سمج\x06سمم\x06صحح\x06صمم\x06شحم\x06شجي\x06شمخ\x06شمم\x06ضحى" +
-	"\x06ضخم\x06طمح\x06طمم\x06طمي\x06عجم\x06عمم\x06عمى\x06غمم\x06غمي\x06غمى" +
-	"\x06فخم\x06قمح\x06قمم\x06لحم\x06لحي\x06لحى\x06لجج\x06لخم\x06لمح\x06محج" +
-	"\x06محم\x06محي\x06مجح\x06مجم\x06مخج\x06مخم\x06مجخ\x06همج\x06همم\x06نحم" +
-	"\x06نحى\x06نجم\x06نجى\x06نمي\x06نمى\x06يمم\x06بخي\x06تجي\x06تجى\x06تخي" +
-	"\x06تخى\x06تمي\x06تمى\x06جمي\x06جحى\x06جمى\x06سخى\x06صحي\x06شحي\x06ضحي" +
-	"\x06لجي\x06لمي\x06يحي\x06يجي\x06يمي\x06ممي\x06قمي\x06نحي\x06عمي\x06كمي" +
-	"\x06نجح\x06مخي\x06لجم\x06كمم\x06جحي\x06حجي\x06مجي\x06فمي\x06بحي\x06سخي" +
-	"\x06نجي\x06صلے\x06قلے\x08الله\x08اكبر\x08محمد\x08صلعم\x08رسول\x08عليه" +
-	"\x08وسلم\x06صلى!صلى الله عليه وسلم\x0fجل جلاله\x08ریال\x01,\x01:\x01!" +
-	"\x01?\x01_\x01{\x01}\x01[\x01]\x01#\x01&\x01*\x01-\x01<\x01>\x01\\\x01$" +
-	"\x01%\x01@\x04ـً\x04ـَ\x04ـُ\x04ـِ\x04ـّ\x04ـْ\x02ء\x02آ\x02أ\x02ؤ\x02إ" +
-	"\x02ئ\x02ا\x02ب\x02ة\x02ت\x02ث\x02ج\x02ح\x02خ\x02د\x02ذ\x02ر\x02ز\x02س" +
-	"\x02ش\x02ص\x02ض\x02ط\x02ظ\x02ع\x02غ\x02ف\x02ق\x02ك\x02ل\x02م\x02ن\x02ه" +
-	"\x02و\x02ي\x04لآ\x04لأ\x04لإ\x04لا\x01\x22\x01'\x01/\x01^\x01|\x01~\x02¢" +
-	"\x02£\x02¬\x02¦\x02¥\x08𝅗𝅥\x08𝅘𝅥\x0c𝅘𝅥𝅮\x0c𝅘𝅥𝅯\x0c𝅘𝅥𝅰\x0c𝅘𝅥𝅱\x0c𝅘𝅥𝅲\x08𝆹" +
-	"𝅥\x08𝆺𝅥\x0c𝆹𝅥𝅮\x0c𝆺𝅥𝅮\x0c𝆹𝅥𝅯\x0c𝆺𝅥𝅯\x02ı\x02ȷ\x02α\x02ε\x02ζ\x02η\x02" +
-	"κ\x02λ\x02μ\x02ν\x02ξ\x02ο\x02σ\x02τ\x02υ\x02ψ\x03∇\x03∂\x02ϝ\x02ٮ\x02ڡ" +
-	"\x02ٯ\x020,\x021,\x022,\x023,\x024,\x025,\x026,\x027,\x028,\x029,\x03(a)" +
-	"\x03(b)\x03(c)\x03(d)\x03(e)\x03(f)\x03(g)\x03(h)\x03(i)\x03(j)\x03(k)" +
-	"\x03(l)\x03(m)\x03(n)\x03(o)\x03(p)\x03(q)\x03(r)\x03(s)\x03(t)\x03(u)" +
-	"\x03(v)\x03(w)\x03(x)\x03(y)\x03(z)\x07〔s〕\x02wz\x02hv\x02sd\x03ppv\x02w" +
-	"c\x02mc\x02md\x02mr\x02dj\x06ほか\x06ココ\x03サ\x03手\x03字\x03双\x03デ\x03二\x03多" +
-	"\x03解\x03天\x03交\x03映\x03無\x03料\x03前\x03後\x03再\x03新\x03初\x03終\x03生\x03販" +
-	"\x03声\x03吹\x03演\x03投\x03捕\x03一\x03三\x03遊\x03左\x03中\x03右\x03指\x03走\x03打" +
-	"\x03禁\x03空\x03合\x03満\x03有\x03月\x03申\x03割\x03営\x03配\x09〔本〕\x09〔三〕\x09〔二〕" +
-	"\x09〔安〕\x09〔点〕\x09〔打〕\x09〔盗〕\x09〔勝〕\x09〔敗〕\x03得\x03可\x03丽\x03丸\x03乁\x03你" +
-	"\x03侮\x03侻\x03倂\x03偺\x03備\x03僧\x03像\x03㒞\x03免\x03兔\x03兤\x03具\x03㒹\x03內" +
-	"\x03冗\x03冤\x03仌\x03冬\x03况\x03凵\x03刃\x03㓟\x03刻\x03剆\x03剷\x03㔕\x03勇\x03勉" +
-	"\x03勤\x03勺\x03包\x03匆\x03北\x03卉\x03卑\x03博\x03即\x03卽\x03卿\x03灰\x03及\x03叟" +
-	"\x03叫\x03叱\x03吆\x03咞\x03吸\x03呈\x03周\x03咢\x03哶\x03唐\x03啓\x03啣\x03善\x03喙" +
-	"\x03喫\x03喳\x03嗂\x03圖\x03嘆\x03圗\x03噑\x03噴\x03切\x03壮\x03城\x03埴\x03堍\x03型" +
-	"\x03堲\x03報\x03墬\x03売\x03壷\x03夆\x03夢\x03奢\x03姬\x03娛\x03娧\x03姘\x03婦\x03㛮" +
-	"\x03嬈\x03嬾\x03寃\x03寘\x03寧\x03寳\x03寿\x03将\x03尢\x03㞁\x03屠\x03屮\x03峀\x03岍" +
-	"\x03嵃\x03嵮\x03嵫\x03嵼\x03巡\x03巢\x03㠯\x03巽\x03帨\x03帽\x03幩\x03㡢\x03㡼\x03庰" +
-	"\x03庳\x03庶\x03廊\x03廾\x03舁\x03弢\x03㣇\x03形\x03彫\x03㣣\x03徚\x03忍\x03志\x03忹" +
-	"\x03悁\x03㤺\x03㤜\x03悔\x03惇\x03慈\x03慌\x03慎\x03慺\x03憎\x03憲\x03憤\x03憯\x03懞" +
-	"\x03懲\x03懶\x03成\x03戛\x03扝\x03抱\x03拔\x03捐\x03挽\x03拼\x03捨\x03掃\x03揤\x03搢" +
-	"\x03揅\x03掩\x03㨮\x03摩\x03摾\x03撝\x03摷\x03㩬\x03敏\x03敬\x03旣\x03書\x03晉\x03㬙" +
-	"\x03暑\x03㬈\x03㫤\x03冒\x03冕\x03最\x03暜\x03肭\x03䏙\x03朗\x03望\x03朡\x03杞\x03杓" +
-	"\x03㭉\x03柺\x03枅\x03桒\x03梅\x03梎\x03栟\x03椔\x03㮝\x03楂\x03榣\x03槪\x03檨\x03櫛" +
-	"\x03㰘\x03次\x03歔\x03㱎\x03歲\x03殟\x03殺\x03殻\x03汎\x03沿\x03泍\x03汧\x03洖\x03派" +
-	"\x03海\x03流\x03浩\x03浸\x03涅\x03洴\x03港\x03湮\x03㴳\x03滋\x03滇\x03淹\x03潮\x03濆" +
-	"\x03瀹\x03瀞\x03瀛\x03㶖\x03灊\x03災\x03灷\x03炭\x03煅\x03熜\x03爨\x03爵\x03牐\x03犀" +
-	"\x03犕\x03獺\x03王\x03㺬\x03玥\x03㺸\x03瑇\x03瑜\x03瑱\x03璅\x03瓊\x03㼛\x03甤\x03甾" +
-	"\x03異\x03瘐\x03㿼\x03䀈\x03直\x03眞\x03真\x03睊\x03䀹\x03瞋\x03䁆\x03䂖\x03硎\x03碌" +
-	"\x03磌\x03䃣\x03祖\x03福\x03秫\x03䄯\x03穀\x03穊\x03穏\x03䈂\x03篆\x03築\x03䈧\x03糒" +
-	"\x03䊠\x03糨\x03糣\x03紀\x03絣\x03䌁\x03緇\x03縂\x03繅\x03䌴\x03䍙\x03罺\x03羕\x03翺" +
-	"\x03者\x03聠\x03聰\x03䏕\x03育\x03脃\x03䐋\x03脾\x03媵\x03舄\x03辞\x03䑫\x03芑\x03芋" +
-	"\x03芝\x03劳\x03花\x03芳\x03芽\x03苦\x03若\x03茝\x03荣\x03莭\x03茣\x03莽\x03菧\x03著" +
-	"\x03荓\x03菊\x03菌\x03菜\x03䔫\x03蓱\x03蓳\x03蔖\x03蕤\x03䕝\x03䕡\x03䕫\x03虐\x03虜" +
-	"\x03虧\x03虩\x03蚩\x03蚈\x03蜎\x03蛢\x03蝹\x03蜨\x03蝫\x03螆\x03蟡\x03蠁\x03䗹\x03衠" +
-	"\x03衣\x03裗\x03裞\x03䘵\x03裺\x03㒻\x03䚾\x03䛇\x03誠\x03諭\x03變\x03豕\x03貫\x03賁" +
-	"\x03贛\x03起\x03跋\x03趼\x03跰\x03軔\x03輸\x03邔\x03郱\x03鄑\x03鄛\x03鈸\x03鋗\x03鋘" +
-	"\x03鉼\x03鏹\x03鐕\x03開\x03䦕\x03閷\x03䧦\x03雃\x03嶲\x03霣\x03䩮\x03䩶\x03韠\x03䪲" +
-	"\x03頋\x03頩\x03飢\x03䬳\x03餩\x03馧\x03駂\x03駾\x03䯎\x03鬒\x03鱀\x03鳽\x03䳎\x03䳭" +
-	"\x03鵧\x03䳸\x03麻\x03䵖\x03黹\x03黾\x03鼅\x03鼏\x03鼖\x03鼻"
-
-var xorData string = "" + // Size: 4862 bytes
-	"\x02\x0c\x09\x02\xb0\xec\x02\xad\xd8\x02\xad\xd9\x02\x06\x07\x02\x0f\x12" +
-	"\x02\x0f\x1f\x02\x0f\x1d\x02\x01\x13\x02\x0f\x16\x02\x0f\x0b\x02\x0f3" +
-	"\x02\x0f7\x02\x0f?\x02\x0f/\x02\x0f*\x02\x0c&\x02\x0c*\x02\x0c;\x02\x0c9" +
-	"\x02\x0c%\x02\xab\xed\x02\xab\xe2\x02\xab\xe3\x02\xa9\xe0\x02\xa9\xe1" +
-	"\x02\xa9\xe6\x02\xa3\xcb\x02\xa3\xc8\x02\xa3\xc9\x02\x01#\x02\x01\x08" +
-	"\x02\x0e>\x02\x0e'\x02\x0f\x03\x02\x03\x0d\x02\x03\x09\x02\x03\x17\x02" +
-	"\x03\x0e\x02\x02\x03\x02\x011\x02\x01\x00\x02\x01\x10\x02\x03<\x02\x07" +
-	"\x0d\x02\x02\x0c\x02\x0c0\x02\x01\x03\x02\x01\x01\x02\x01 \x02\x01\x22" +
-	"\x02\x01)\x02\x01\x0a\x02\x01\x0c\x02\x02\x06\x02\x02\x02\x02\x03\x10" +
-	"\x03\x037 \x03\x0b+\x03\x021\x00\x02\x01\x04\x02\x01\x02\x02\x019\x02" +
-	"\x03\x1c\x02\x02$\x03\x80p$\x02\x03:\x02\x03\x0a\x03\xc1r.\x03\xc1r,\x03" +
-	"\xc1r\x02\x02\x02:\x02\x02>\x02\x02,\x02\x02\x10\x02\x02\x00\x03\xc1s<" +
-	"\x03\xc1s*\x03\xc2L$\x03\xc2L;\x02\x09)\x02\x0a\x19\x03\x83\xab\xe3\x03" +
-	"\x83\xab\xf2\x03 4\xe0\x03\x81\xab\xea\x03\x81\xab\xf3\x03 4\xef\x03\x96" +
-	"\xe1\xcd\x03\x84\xe5\xc3\x02\x0d\x11\x03\x8b\xec\xcb\x03\x94\xec\xcf\x03" +
-	"\x9a\xec\xc2\x03\x8b\xec\xdb\x03\x94\xec\xdf\x03\x9a\xec\xd2\x03\x01\x0c" +
-	"!\x03\x01\x0c#\x03ʠ\x9d\x03ʣ\x9c\x03ʢ\x9f\x03ʥ\x9e\x03ʤ\x91\x03ʧ\x90\x03" +
-	"ʦ\x93\x03ʩ\x92\x03ʨ\x95\x03\xca\xf3\xb5\x03\xca\xf0\xb4\x03\xca\xf1\xb7" +
-	"\x03\xca\xf6\xb6\x03\xca\xf7\x89\x03\xca\xf4\x88\x03\xca\xf5\x8b\x03\xca" +
-	"\xfa\x8a\x03\xca\xfb\x8d\x03\xca\xf8\x8c\x03\xca\xf9\x8f\x03\xca\xfe\x8e" +
-	"\x03\xca\xff\x81\x03\xca\xfc\x80\x03\xca\xfd\x83\x03\xca\xe2\x82\x03\xca" +
-	"\xe3\x85\x03\xca\xe0\x84\x03\xca\xe1\x87\x03\xca\xe6\x86\x03\xca\xe7\x99" +
-	"\x03\xca\xe4\x98\x03\xca\xe5\x9b\x03\xca\xea\x9a\x03\xca\xeb\x9d\x03\xca" +
-	"\xe8\x9c\x03ؓ\x89\x03ߔ\x8b\x02\x010\x03\x03\x04\x1e\x03\x04\x15\x12\x03" +
-	"\x0b\x05,\x03\x06\x04\x00\x03\x06\x04)\x03\x06\x044\x03\x06\x04<\x03\x06" +
-	"\x05\x1d\x03\x06\x06\x00\x03\x06\x06\x0a\x03\x06\x06'\x03\x06\x062\x03" +
-	"\x0786\x03\x079/\x03\x079 \x03\x07:\x0e\x03\x07:\x1b\x03\x07:%\x03\x07;/" +
-	"\x03\x07;%\x03\x074\x11\x03\x076\x09\x03\x077*\x03\x070\x01\x03\x070\x0f" +
-	"\x03\x070.\x03\x071\x16\x03\x071\x04\x03\x0710\x03\x072\x18\x03\x072-" +
-	"\x03\x073\x14\x03\x073>\x03\x07'\x09\x03\x07 \x00\x03\x07\x1f\x0b\x03" +
-	"\x07\x18#\x03\x07\x18(\x03\x07\x186\x03\x07\x18\x03\x03\x07\x19\x16\x03" +
-	"\x07\x116\x03\x07\x12'\x03\x07\x13\x10\x03\x07\x0c&\x03\x07\x0c\x08\x03" +
-	"\x07\x0c\x13\x03\x07\x0d\x02\x03\x07\x0d\x1c\x03\x07\x0b5\x03\x07\x0b" +
-	"\x0a\x03\x07\x0b\x01\x03\x07\x0b\x0f\x03\x07\x05\x00\x03\x07\x05\x09\x03" +
-	"\x07\x05\x0b\x03\x07\x07\x01\x03\x07\x07\x08\x03\x07\x00<\x03\x07\x00+" +
-	"\x03\x07\x01)\x03\x07\x01\x1b\x03\x07\x01\x08\x03\x07\x03?\x03\x0445\x03" +
-	"\x044\x08\x03\x0454\x03\x04)/\x03\x04)5\x03\x04+\x05\x03\x04+\x14\x03" +
-	"\x04+ \x03\x04+<\x03\x04*&\x03\x04*\x22\x03\x04&8\x03\x04!\x01\x03\x04!" +
-	"\x22\x03\x04\x11+\x03\x04\x10.\x03\x04\x104\x03\x04\x13=\x03\x04\x12\x04" +
-	"\x03\x04\x12\x0a\x03\x04\x0d\x1d\x03\x04\x0d\x07\x03\x04\x0d \x03\x05<>" +
-	"\x03\x055<\x03\x055!\x03\x055#\x03\x055&\x03\x054\x1d\x03\x054\x02\x03" +
-	"\x054\x07\x03\x0571\x03\x053\x1a\x03\x053\x16\x03\x05.<\x03\x05.\x07\x03" +
-	"\x05):\x03\x05)<\x03\x05)\x0c\x03\x05)\x15\x03\x05+-\x03\x05+5\x03\x05$" +
-	"\x1e\x03\x05$\x14\x03\x05'\x04\x03\x05'\x14\x03\x05&\x02\x03\x05\x226" +
-	"\x03\x05\x22\x0c\x03\x05\x22\x1c\x03\x05\x19\x0a\x03\x05\x1b\x09\x03\x05" +
-	"\x1b\x0c\x03\x05\x14\x07\x03\x05\x16?\x03\x05\x16\x0c\x03\x05\x0c\x05" +
-	"\x03\x05\x0e\x0f\x03\x05\x01\x0e\x03\x05\x00(\x03\x05\x030\x03\x05\x03" +
-	"\x06\x03\x0a==\x03\x0a=1\x03\x0a=,\x03\x0a=\x0c\x03\x0a??\x03\x0a<\x08" +
-	"\x03\x0a9!\x03\x0a9)\x03\x0a97\x03\x0a99\x03\x0a6\x0a\x03\x0a6\x1c\x03" +
-	"\x0a6\x17\x03\x0a7'\x03\x0a78\x03\x0a73\x03\x0a'\x01\x03\x0a'&\x03\x0a" +
-	"\x1f\x0e\x03\x0a\x1f\x03\x03\x0a\x1f3\x03\x0a\x1b/\x03\x0a\x18\x19\x03" +
-	"\x0a\x19\x01\x03\x0a\x16\x14\x03\x0a\x0e\x22\x03\x0a\x0f\x10\x03\x0a\x0f" +
-	"\x02\x03\x0a\x0f \x03\x0a\x0c\x04\x03\x0a\x0b>\x03\x0a\x0b+\x03\x0a\x08/" +
-	"\x03\x0a\x046\x03\x0a\x05\x14\x03\x0a\x00\x04\x03\x0a\x00\x10\x03\x0a" +
-	"\x00\x14\x03\x0b<3\x03\x0b;*\x03\x0b9\x22\x03\x0b9)\x03\x0b97\x03\x0b+" +
-	"\x10\x03\x0b((\x03\x0b&5\x03\x0b$\x1c\x03\x0b$\x12\x03\x0b%\x04\x03\x0b#" +
-	"<\x03\x0b#0\x03\x0b#\x0d\x03\x0b#\x19\x03\x0b!:\x03\x0b!\x1f\x03\x0b!" +
-	"\x00\x03\x0b\x1e5\x03\x0b\x1c\x1d\x03\x0b\x1d-\x03\x0b\x1d(\x03\x0b\x18." +
-	"\x03\x0b\x18 \x03\x0b\x18\x16\x03\x0b\x14\x13\x03\x0b\x15$\x03\x0b\x15" +
-	"\x22\x03\x0b\x12\x1b\x03\x0b\x12\x10\x03\x0b\x132\x03\x0b\x13=\x03\x0b" +
-	"\x12\x18\x03\x0b\x0c&\x03\x0b\x061\x03\x0b\x06:\x03\x0b\x05#\x03\x0b\x05" +
-	"<\x03\x0b\x04\x0b\x03\x0b\x04\x04\x03\x0b\x04\x1b\x03\x0b\x042\x03\x0b" +
-	"\x041\x03\x0b\x03\x03\x03\x0b\x03\x1d\x03\x0b\x03/\x03\x0b\x03+\x03\x0b" +
-	"\x02\x1b\x03\x0b\x02\x00\x03\x0b\x01\x1e\x03\x0b\x01\x08\x03\x0b\x015" +
-	"\x03\x06\x0d9\x03\x06\x0d=\x03\x06\x0d?\x03\x02\x001\x03\x02\x003\x03" +
-	"\x02\x02\x19\x03\x02\x006\x03\x02\x02\x1b\x03\x02\x004\x03\x02\x00<\x03" +
-	"\x02\x02\x0a\x03\x02\x02\x0e\x03\x02\x01\x1a\x03\x02\x01\x07\x03\x02\x01" +
-	"\x05\x03\x02\x01\x0b\x03\x02\x01%\x03\x02\x01\x0c\x03\x02\x01\x04\x03" +
-	"\x02\x01\x1c\x03\x02\x00.\x03\x02\x002\x03\x02\x00>\x03\x02\x00\x12\x03" +
-	"\x02\x00\x16\x03\x02\x011\x03\x02\x013\x03\x02\x02 \x03\x02\x02%\x03\x02" +
-	"\x02$\x03\x02\x028\x03\x02\x02;\x03\x02\x024\x03\x02\x012\x03\x02\x022" +
-	"\x03\x02\x02/\x03\x02\x01,\x03\x02\x01\x13\x03\x02\x01\x16\x03\x02\x01" +
-	"\x11\x03\x02\x01\x1e\x03\x02\x01\x15\x03\x02\x01\x17\x03\x02\x01\x0f\x03" +
-	"\x02\x01\x08\x03\x02\x00?\x03\x02\x03\x07\x03\x02\x03\x0d\x03\x02\x03" +
-	"\x13\x03\x02\x03\x1d\x03\x02\x03\x1f\x03\x02\x00\x03\x03\x02\x00\x0d\x03" +
-	"\x02\x00\x01\x03\x02\x00\x1b\x03\x02\x00\x19\x03\x02\x00\x18\x03\x02\x00" +
-	"\x13\x03\x02\x00/\x03\x07>\x12\x03\x07<\x1f\x03\x07>\x1d\x03\x06\x1d\x0e" +
-	"\x03\x07>\x1c\x03\x07>:\x03\x07>\x13\x03\x04\x12+\x03\x07?\x03\x03\x07>" +
-	"\x02\x03\x06\x224\x03\x06\x1a.\x03\x07<%\x03\x06\x1c\x0b\x03\x0609\x03" +
-	"\x05\x1f\x01\x03\x04'\x08\x03\x93\xfd\xf5\x03\x02\x0d \x03\x02\x0d#\x03" +
-	"\x02\x0d!\x03\x02\x0d&\x03\x02\x0d\x22\x03\x02\x0d/\x03\x02\x0d,\x03\x02" +
-	"\x0d$\x03\x02\x0d'\x03\x02\x0d%\x03\x02\x0d;\x03\x02\x0d=\x03\x02\x0d?" +
-	"\x03\x099.\x03\x08\x0b7\x03\x08\x02\x14\x03\x08\x14\x0d\x03\x08.:\x03" +
-	"\x089'\x03\x0f\x0b\x18\x03\x0f\x1c1\x03\x0f\x17&\x03\x0f9\x1f\x03\x0f0" +
-	"\x0c\x03\x0e\x0a9\x03\x0e\x056\x03\x0e\x1c#\x03\x0f\x13\x0e\x03\x072\x00" +
-	"\x03\x070\x0d\x03\x072\x0b\x03\x06\x11\x18\x03\x070\x10\x03\x06\x0f(\x03" +
-	"\x072\x05\x03\x06\x0f,\x03\x073\x15\x03\x06\x07\x08\x03\x05\x16\x02\x03" +
-	"\x04\x0b \x03\x05:8\x03\x05\x16%\x03\x0a\x0d\x1f\x03\x06\x16\x10\x03\x05" +
-	"\x1d5\x03\x05*;\x03\x05\x16\x1b\x03\x04.-\x03\x06\x1a\x19\x03\x04\x03," +
-	"\x03\x0b87\x03\x04/\x0a\x03\x06\x00,\x03\x04-\x01\x03\x04\x1e-\x03\x06/(" +
-	"\x03\x0a\x0b5\x03\x06\x0e7\x03\x06\x07.\x03\x0597\x03\x0a*%\x03\x0760" +
-	"\x03\x06\x0c;\x03\x05'\x00\x03\x072.\x03\x072\x08\x03\x06=\x01\x03\x06" +
-	"\x05\x1b\x03\x06\x06\x12\x03\x06$=\x03\x06'\x0d\x03\x04\x11\x0f\x03\x076" +
-	",\x03\x06\x07;\x03\x06.,\x03\x86\xf9\xea\x03\x8f\xff\xeb\x02\x092\x02" +
-	"\x095\x02\x094\x02\x09;\x02\x09>\x02\x098\x02\x09*\x02\x09/\x02\x09,\x02" +
-	"\x09%\x02\x09&\x02\x09#\x02\x09 \x02\x08!\x02\x08%\x02\x08$\x02\x08+\x02" +
-	"\x08.\x02\x08*\x02\x08&\x02\x088\x02\x08>\x02\x084\x02\x086\x02\x080\x02" +
-	"\x08\x10\x02\x08\x17\x02\x08\x12\x02\x08\x1d\x02\x08\x1f\x02\x08\x13\x02" +
-	"\x08\x15\x02\x08\x14\x02\x08\x0c\x03\x8b\xfd\xd0\x03\x81\xec\xc6\x03\x87" +
-	"\xe0\x8a\x03-2\xe3\x03\x80\xef\xe4\x03-2\xea\x03\x88\xe6\xeb\x03\x8e\xe6" +
-	"\xe8\x03\x84\xe6\xe9\x03\x97\xe6\xee\x03-2\xf9\x03-2\xf6\x03\x8e\xe3\xad" +
-	"\x03\x80\xe3\x92\x03\x88\xe3\x90\x03\x8e\xe3\x90\x03\x80\xe3\x97\x03\x88" +
-	"\xe3\x95\x03\x88\xfe\xcb\x03\x8e\xfe\xca\x03\x84\xfe\xcd\x03\x91\xef\xc9" +
-	"\x03-2\xc1\x03-2\xc0\x03-2\xcb\x03\x88@\x09\x03\x8e@\x08\x03\x8f\xe0\xf5" +
-	"\x03\x8e\xe6\xf9\x03\x8e\xe0\xfa\x03\x93\xff\xf4\x03\x84\xee\xd3\x03\x0b" +
-	"(\x04\x023 \x03\x0b)\x08\x021;\x02\x01*\x03\x0b#\x10\x03\x0b 0\x03\x0b!" +
-	"\x10\x03\x0b!0\x03\x07\x15\x08\x03\x09?5\x03\x07\x1f\x08\x03\x07\x17\x0b" +
-	"\x03\x09\x1f\x15\x03\x0b\x1c7\x03\x0a+#\x03\x06\x1a\x1b\x03\x06\x1a\x14" +
-	"\x03\x0a\x01\x18\x03\x06#\x1b\x03\x0a2\x0c\x03\x0a\x01\x04\x03\x09#;\x03" +
-	"\x08='\x03\x08\x1a\x0a\x03\x07</\x03\x07:+\x03\x07\x07*\x03\x06&\x1c\x03" +
-	"\x09\x0c\x16\x03\x09\x10\x0e\x03\x08'\x0f\x03\x08+\x09\x03\x074%\x03\x06" +
-	"!3\x03\x06\x03+\x03\x0b\x1e\x19\x03\x0a))\x03\x09\x08\x19\x03\x08,\x05" +
-	"\x03\x07<2\x03\x06\x1c>\x03\x0a\x111\x03\x09\x1b\x09\x03\x073.\x03\x07" +
-	"\x01\x00\x03\x09/,\x03\x07#>\x03\x07\x048\x03\x0a\x1f\x22\x03\x098>\x03" +
-	"\x09\x11\x00\x03\x08/\x17\x03\x06'\x22\x03\x0b\x1a+\x03\x0a\x22\x19\x03" +
-	"\x0a/1\x03\x0974\x03\x09\x0f\x22\x03\x08,\x22\x03\x08?\x14\x03\x07$5\x03" +
-	"\x07<3\x03\x07=*\x03\x07\x13\x18\x03\x068\x0a\x03\x06\x09\x16\x03\x06" +
-	"\x13\x00\x03\x08\x067\x03\x08\x01\x03\x03\x08\x12\x1d\x03\x07+7\x03\x06(" +
-	";\x03\x06\x1c?\x03\x07\x0e\x17\x03\x0a\x06\x1d\x03\x0a\x19\x07\x03\x08" +
-	"\x14$\x03\x07$;\x03\x08,$\x03\x08\x06\x0d\x03\x07\x16\x0a\x03\x06>>\x03" +
-	"\x0a\x06\x12\x03\x0a\x14)\x03\x09\x0d\x1f\x03\x09\x12\x17\x03\x09\x19" +
-	"\x01\x03\x08\x11 \x03\x08\x1d'\x03\x06<\x1a\x03\x0a.\x00\x03\x07'\x18" +
-	"\x03\x0a\x22\x08\x03\x08\x0d\x0a\x03\x08\x13)\x03\x07*)\x03\x06<,\x03" +
-	"\x07\x0b\x1a\x03\x09.\x14\x03\x09\x0d\x1e\x03\x07\x0e#\x03\x0b\x1d'\x03" +
-	"\x0a\x0a8\x03\x09%2\x03\x08+&\x03\x080\x12\x03\x0a)4\x03\x08\x06\x1f\x03" +
-	"\x0b\x1b\x1a\x03\x0a\x1b\x0f\x03\x0b\x1d*\x03\x09\x16$\x03\x090\x11\x03" +
-	"\x08\x11\x08\x03\x0a*(\x03\x0a\x042\x03\x089,\x03\x074'\x03\x07\x0f\x05" +
-	"\x03\x09\x0b\x0a\x03\x07\x1b\x01\x03\x09\x17:\x03\x09.\x0d\x03\x07.\x11" +
-	"\x03\x09+\x15\x03\x080\x13\x03\x0b\x1f\x19\x03\x0a \x11\x03\x0a\x220\x03" +
-	"\x09\x07;\x03\x08\x16\x1c\x03\x07,\x13\x03\x07\x0e/\x03\x06\x221\x03\x0a" +
-	".\x0a\x03\x0a7\x02\x03\x0a\x032\x03\x0a\x1d.\x03\x091\x06\x03\x09\x19:" +
-	"\x03\x08\x02/\x03\x060+\x03\x06\x0f-\x03\x06\x1c\x1f\x03\x06\x1d\x07\x03" +
-	"\x0a,\x11\x03\x09=\x0d\x03\x09\x0b;\x03\x07\x1b/\x03\x0a\x1f:\x03\x09 " +
-	"\x1f\x03\x09.\x10\x03\x094\x0b\x03\x09\x1a1\x03\x08#\x1a\x03\x084\x1d" +
-	"\x03\x08\x01\x1f\x03\x08\x11\x22\x03\x07'8\x03\x07\x1a>\x03\x0757\x03" +
-	"\x06&9\x03\x06+\x11\x03\x0a.\x0b\x03\x0a,>\x03\x0a4#\x03\x08%\x17\x03" +
-	"\x07\x05\x22\x03\x07\x0c\x0b\x03\x0a\x1d+\x03\x0a\x19\x16\x03\x09+\x1f" +
-	"\x03\x09\x08\x0b\x03\x08\x16\x18\x03\x08+\x12\x03\x0b\x1d\x0c\x03\x0a=" +
-	"\x10\x03\x0a\x09\x0d\x03\x0a\x10\x11\x03\x09&0\x03\x08(\x1f\x03\x087\x07" +
-	"\x03\x08\x185\x03\x07'6\x03\x06.\x05\x03\x06=\x04\x03\x06;;\x03\x06\x06," +
-	"\x03\x0b\x18>\x03\x08\x00\x18\x03\x06 \x03\x03\x06<\x00\x03\x09%\x18\x03" +
-	"\x0b\x1c<\x03\x0a%!\x03\x0a\x09\x12\x03\x0a\x16\x02\x03\x090'\x03\x09" +
-	"\x0e=\x03\x08 \x0e\x03\x08>\x03\x03\x074>\x03\x06&?\x03\x06\x19\x09\x03" +
-	"\x06?(\x03\x0a-\x0e\x03\x09:3\x03\x098:\x03\x09\x12\x0b\x03\x09\x1d\x17" +
-	"\x03\x087\x05\x03\x082\x14\x03\x08\x06%\x03\x08\x13\x1f\x03\x06\x06\x0e" +
-	"\x03\x0a\x22<\x03\x09/<\x03\x06>+\x03\x0a'?\x03\x0a\x13\x0c\x03\x09\x10<" +
-	"\x03\x07\x1b=\x03\x0a\x19\x13\x03\x09\x22\x1d\x03\x09\x07\x0d\x03\x08)" +
-	"\x1c\x03\x06=\x1a\x03\x0a/4\x03\x0a7\x11\x03\x0a\x16:\x03\x09?3\x03\x09:" +
-	"/\x03\x09\x05\x0a\x03\x09\x14\x06\x03\x087\x22\x03\x080\x07\x03\x08\x1a" +
-	"\x1f\x03\x07\x04(\x03\x07\x04\x09\x03\x06 %\x03\x06<\x08\x03\x0a+\x14" +
-	"\x03\x09\x1d\x16\x03\x0a70\x03\x08 >\x03\x0857\x03\x070\x0a\x03\x06=\x12" +
-	"\x03\x06\x16%\x03\x06\x1d,\x03\x099#\x03\x09\x10>\x03\x07 \x1e\x03\x08" +
-	"\x0c<\x03\x08\x0b\x18\x03\x08\x15+\x03\x08,:\x03\x08%\x22\x03\x07\x0a$" +
-	"\x03\x0b\x1c=\x03\x07+\x08\x03\x0a/\x05\x03\x0a \x07\x03\x0a\x12'\x03" +
-	"\x09#\x11\x03\x08\x1b\x15\x03\x0a\x06\x01\x03\x09\x1c\x1b\x03\x0922\x03" +
-	"\x07\x14<\x03\x07\x09\x04\x03\x061\x04\x03\x07\x0e\x01\x03\x0a\x13\x18" +
-	"\x03\x0a-\x0c\x03\x0a?\x0d\x03\x0a\x09\x0a\x03\x091&\x03\x0a/\x0b\x03" +
-	"\x08$<\x03\x083\x1d\x03\x08\x0c$\x03\x08\x0d\x07\x03\x08\x0d?\x03\x08" +
-	"\x0e\x14\x03\x065\x0a\x03\x08\x1a#\x03\x08\x16#\x03\x0702\x03\x07\x03" +
-	"\x1a\x03\x06(\x1d\x03\x06+\x1b\x03\x06\x0b\x05\x03\x06\x0b\x17\x03\x06" +
-	"\x0c\x04\x03\x06\x1e\x19\x03\x06+0\x03\x062\x18\x03\x0b\x16\x1e\x03\x0a+" +
-	"\x16\x03\x0a-?\x03\x0a#:\x03\x0a#\x10\x03\x0a%$\x03\x0a>+\x03\x0a01\x03" +
-	"\x0a1\x10\x03\x0a\x099\x03\x0a\x0a\x12\x03\x0a\x19\x1f\x03\x0a\x19\x12" +
-	"\x03\x09*)\x03\x09-\x16\x03\x09.1\x03\x09.2\x03\x09<\x0e\x03\x09> \x03" +
-	"\x093\x12\x03\x09\x0b\x01\x03\x09\x1c2\x03\x09\x11\x1c\x03\x09\x15%\x03" +
-	"\x08,&\x03\x08!\x22\x03\x089(\x03\x08\x0b\x1a\x03\x08\x0d2\x03\x08\x0c" +
-	"\x04\x03\x08\x0c\x06\x03\x08\x0c\x1f\x03\x08\x0c\x0c\x03\x08\x0f\x1f\x03" +
-	"\x08\x0f\x1d\x03\x08\x00\x14\x03\x08\x03\x14\x03\x08\x06\x16\x03\x08\x1e" +
-	"#\x03\x08\x11\x11\x03\x08\x10\x18\x03\x08\x14(\x03\x07)\x1e\x03\x07.1" +
-	"\x03\x07 $\x03\x07 '\x03\x078\x08\x03\x07\x0d0\x03\x07\x0f7\x03\x07\x05#" +
-	"\x03\x07\x05\x1a\x03\x07\x1a7\x03\x07\x1d-\x03\x07\x17\x10\x03\x06)\x1f" +
-	"\x03\x062\x0b\x03\x066\x16\x03\x06\x09\x11\x03\x09(\x1e\x03\x07!5\x03" +
-	"\x0b\x11\x16\x03\x0a/\x04\x03\x0a,\x1a\x03\x0b\x173\x03\x0a,1\x03\x0a/5" +
-	"\x03\x0a\x221\x03\x0a\x22\x0d\x03\x0a?%\x03\x0a<,\x03\x0a?#\x03\x0a>\x19" +
-	"\x03\x0a\x08&\x03\x0a\x0b\x0e\x03\x0a\x0c:\x03\x0a\x0c+\x03\x0a\x03\x22" +
-	"\x03\x0a\x06)\x03\x0a\x11\x10\x03\x0a\x11\x1a\x03\x0a\x17-\x03\x0a\x14(" +
-	"\x03\x09)\x1e\x03\x09/\x09\x03\x09.\x00\x03\x09,\x07\x03\x09/*\x03\x09-9" +
-	"\x03\x09\x228\x03\x09%\x09\x03\x09:\x12\x03\x09;\x1d\x03\x09?\x06\x03" +
-	"\x093%\x03\x096\x05\x03\x096\x08\x03\x097\x02\x03\x09\x07,\x03\x09\x04," +
-	"\x03\x09\x1f\x16\x03\x09\x11\x03\x03\x09\x11\x12\x03\x09\x168\x03\x08*" +
-	"\x05\x03\x08/2\x03\x084:\x03\x08\x22+\x03\x08 0\x03\x08&\x0a\x03\x08;" +
-	"\x10\x03\x08>$\x03\x08>\x18\x03\x0829\x03\x082:\x03\x081,\x03\x081<\x03" +
-	"\x081\x1c\x03\x087#\x03\x087*\x03\x08\x09'\x03\x08\x00\x1d\x03\x08\x05-" +
-	"\x03\x08\x1f4\x03\x08\x1d\x04\x03\x08\x16\x0f\x03\x07*7\x03\x07'!\x03" +
-	"\x07%\x1b\x03\x077\x0c\x03\x07\x0c1\x03\x07\x0c.\x03\x07\x00\x06\x03\x07" +
-	"\x01\x02\x03\x07\x010\x03\x07\x06=\x03\x07\x01\x03\x03\x07\x01\x13\x03" +
-	"\x07\x06\x06\x03\x07\x05\x0a\x03\x07\x1f\x09\x03\x07\x17:\x03\x06*1\x03" +
-	"\x06-\x1d\x03\x06\x223\x03\x062:\x03\x060$\x03\x066\x1e\x03\x064\x12\x03" +
-	"\x0645\x03\x06\x0b\x00\x03\x06\x0b7\x03\x06\x07\x1f\x03\x06\x15\x12\x03" +
-	"\x0c\x05\x0f\x03\x0b+\x0b\x03\x0b+-\x03\x06\x16\x1b\x03\x06\x15\x17\x03" +
-	"\x89\xca\xea\x03\x89\xca\xe8\x03\x0c8\x10\x03\x0c8\x01\x03\x0c8\x0f\x03" +
-	"\x0d8%\x03\x0d8!\x03\x0c8-\x03\x0c8/\x03\x0c8+\x03\x0c87\x03\x0c85\x03" +
-	"\x0c9\x09\x03\x0c9\x0d\x03\x0c9\x0f\x03\x0c9\x0b\x03\xcfu\x0c\x03\xcfu" +
-	"\x0f\x03\xcfu\x0e\x03\xcfu\x09\x03\x0c9\x10\x03\x0d9\x0c\x03\xcf`;\x03" +
-	"\xcf`>\x03\xcf`9\x03\xcf`8\x03\xcf`7\x03\xcf`*\x03\xcf`-\x03\xcf`,\x03" +
-	"\x0d\x1b\x1a\x03\x0d\x1b&\x03\x0c=.\x03\x0c=%\x03\x0c>\x1e\x03\x0c>\x14" +
-	"\x03\x0c?\x06\x03\x0c?\x0b\x03\x0c?\x0c\x03\x0c?\x0d\x03\x0c?\x02\x03" +
-	"\x0c>\x0f\x03\x0c>\x08\x03\x0c>\x09\x03\x0c>,\x03\x0c>\x0c\x03\x0c?\x13" +
-	"\x03\x0c?\x16\x03\x0c?\x15\x03\x0c?\x1c\x03\x0c?\x1f\x03\x0c?\x1d\x03" +
-	"\x0c?\x1a\x03\x0c?\x17\x03\x0c?\x08\x03\x0c?\x09\x03\x0c?\x0e\x03\x0c?" +
-	"\x04\x03\x0c?\x05\x03\x0c<?\x03\x0c=\x00\x03\x0c=\x06\x03\x0c=\x05\x03" +
-	"\x0c=\x0c\x03\x0c=\x0f\x03\x0c=\x0d\x03\x0c=\x0b\x03\x0c=\x07\x03\x0c=" +
-	"\x19\x03\x0c=\x15\x03\x0c=\x11\x03\x0c=1\x03\x0c=3\x03\x0c=0\x03\x0c=>" +
-	"\x03\x0c=2\x03\x0c=6\x03\x0c<\x07\x03\x0c<\x05\x03\x0e:!\x03\x0e:#\x03" +
-	"\x0e8\x09\x03\x0e:&\x03\x0e8\x0b\x03\x0e:$\x03\x0e:,\x03\x0e8\x1a\x03" +
-	"\x0e8\x1e\x03\x0e:*\x03\x0e:7\x03\x0e:5\x03\x0e:;\x03\x0e:\x15\x03\x0e:<" +
-	"\x03\x0e:4\x03\x0e:'\x03\x0e:-\x03\x0e:%\x03\x0e:?\x03\x0e:=\x03\x0e:)" +
-	"\x03\x0e:/\x03\xcfs'\x03\x0d=\x0f\x03\x0d+*\x03\x0d99\x03\x0d9;\x03\x0d9" +
-	"?\x03\x0d)\x0d\x03\x0d(%\x02\x01\x18\x02\x01(\x02\x01\x1e\x03\x0f$!\x03" +
-	"\x0f87\x03\x0f4\x0e\x03\x0f5\x1d\x03\x06'\x03\x03\x0f\x08\x18\x03\x0f" +
-	"\x0d\x1b\x03\x0e2=\x03\x0e;\x08\x03\x0e:\x0b\x03\x0e\x06$\x03\x0e\x0d)" +
-	"\x03\x0e\x16\x1f\x03\x0e\x16\x1b\x03\x0d$\x0a\x03\x05,\x1d\x03\x0d. \x03" +
-	"\x0d.#\x03\x0c(/\x03\x09%\x02\x03\x0d90\x03\x0d\x0e4\x03\x0d\x0d\x0f\x03" +
-	"\x0c#\x00\x03\x0c,\x1e\x03\x0c2\x0e\x03\x0c\x01\x17\x03\x0c\x09:\x03\x0e" +
-	"\x173\x03\x0c\x08\x03\x03\x0c\x11\x07\x03\x0c\x10\x18\x03\x0c\x1f\x1c" +
-	"\x03\x0c\x19\x0e\x03\x0c\x1a\x1f\x03\x0f0>\x03\x0b->\x03\x0b<+\x03\x0b8" +
-	"\x13\x03\x0b\x043\x03\x0b\x14\x03\x03\x0b\x16%\x03\x0d\x22&\x03\x0b\x1a" +
-	"\x1a\x03\x0b\x1a\x04\x03\x0a%9\x03\x0a&2\x03\x0a&0\x03\x0a!\x1a\x03\x0a!" +
-	"7\x03\x0a5\x10\x03\x0a=4\x03\x0a?\x0e\x03\x0a>\x10\x03\x0a\x00 \x03\x0a" +
-	"\x0f:\x03\x0a\x0f9\x03\x0a\x0b\x0a\x03\x0a\x17%\x03\x0a\x1b-\x03\x09-" +
-	"\x1a\x03\x09,4\x03\x09.,\x03\x09)\x09\x03\x096!\x03\x091\x1f\x03\x093" +
-	"\x16\x03\x0c+\x1f\x03\x098 \x03\x098=\x03\x0c(\x1a\x03\x0c(\x16\x03\x09" +
-	"\x0a+\x03\x09\x16\x12\x03\x09\x13\x0e\x03\x09\x153\x03\x08)!\x03\x09\x1a" +
-	"\x01\x03\x09\x18\x01\x03\x08%#\x03\x08>\x22\x03\x08\x05%\x03\x08\x02*" +
-	"\x03\x08\x15;\x03\x08\x1b7\x03\x0f\x07\x1d\x03\x0f\x04\x03\x03\x070\x0c" +
-	"\x03\x07;\x0b\x03\x07\x08\x17\x03\x07\x12\x06\x03\x06/-\x03\x0671\x03" +
-	"\x065+\x03\x06>7\x03\x06\x049\x03\x05+\x1e\x03\x05,\x17\x03\x05 \x1d\x03" +
-	"\x05\x22\x05\x03\x050\x1d"
-
-// lookup returns the trie value for the first UTF-8 encoding in s and
-// the width in bytes of this encoding. The size will be 0 if s does not
-// hold enough bytes to complete the encoding. len(s) must be greater than 0.
-func (t *idnaTrie) lookup(s []byte) (v uint16, sz int) {
-	c0 := s[0]
-	switch {
-	case c0 < 0x80: // is ASCII
-		return idnaValues[c0], 1
-	case c0 < 0xC2:
-		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
-	case c0 < 0xE0: // 2-byte UTF-8
-		if len(s) < 2 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c1), 2
-	case c0 < 0xF0: // 3-byte UTF-8
-		if len(s) < 3 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		o := uint32(i)<<6 + uint32(c1)
-		i = idnaIndex[o]
-		c2 := s[2]
-		if c2 < 0x80 || 0xC0 <= c2 {
-			return 0, 2 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c2), 3
-	case c0 < 0xF8: // 4-byte UTF-8
-		if len(s) < 4 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		o := uint32(i)<<6 + uint32(c1)
-		i = idnaIndex[o]
-		c2 := s[2]
-		if c2 < 0x80 || 0xC0 <= c2 {
-			return 0, 2 // Illegal UTF-8: not a continuation byte.
-		}
-		o = uint32(i)<<6 + uint32(c2)
-		i = idnaIndex[o]
-		c3 := s[3]
-		if c3 < 0x80 || 0xC0 <= c3 {
-			return 0, 3 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c3), 4
-	}
-	// Illegal rune
-	return 0, 1
-}
-
-// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
-// s must start with a full and valid UTF-8 encoded rune.
-func (t *idnaTrie) lookupUnsafe(s []byte) uint16 {
-	c0 := s[0]
-	if c0 < 0x80 { // is ASCII
-		return idnaValues[c0]
-	}
-	i := idnaIndex[c0]
-	if c0 < 0xE0 { // 2-byte UTF-8
-		return t.lookupValue(uint32(i), s[1])
-	}
-	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
-	if c0 < 0xF0 { // 3-byte UTF-8
-		return t.lookupValue(uint32(i), s[2])
-	}
-	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
-	if c0 < 0xF8 { // 4-byte UTF-8
-		return t.lookupValue(uint32(i), s[3])
-	}
-	return 0
-}
-
-// lookupString returns the trie value for the first UTF-8 encoding in s and
-// the width in bytes of this encoding. The size will be 0 if s does not
-// hold enough bytes to complete the encoding. len(s) must be greater than 0.
-func (t *idnaTrie) lookupString(s string) (v uint16, sz int) {
-	c0 := s[0]
-	switch {
-	case c0 < 0x80: // is ASCII
-		return idnaValues[c0], 1
-	case c0 < 0xC2:
-		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
-	case c0 < 0xE0: // 2-byte UTF-8
-		if len(s) < 2 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c1), 2
-	case c0 < 0xF0: // 3-byte UTF-8
-		if len(s) < 3 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		o := uint32(i)<<6 + uint32(c1)
-		i = idnaIndex[o]
-		c2 := s[2]
-		if c2 < 0x80 || 0xC0 <= c2 {
-			return 0, 2 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c2), 3
-	case c0 < 0xF8: // 4-byte UTF-8
-		if len(s) < 4 {
-			return 0, 0
-		}
-		i := idnaIndex[c0]
-		c1 := s[1]
-		if c1 < 0x80 || 0xC0 <= c1 {
-			return 0, 1 // Illegal UTF-8: not a continuation byte.
-		}
-		o := uint32(i)<<6 + uint32(c1)
-		i = idnaIndex[o]
-		c2 := s[2]
-		if c2 < 0x80 || 0xC0 <= c2 {
-			return 0, 2 // Illegal UTF-8: not a continuation byte.
-		}
-		o = uint32(i)<<6 + uint32(c2)
-		i = idnaIndex[o]
-		c3 := s[3]
-		if c3 < 0x80 || 0xC0 <= c3 {
-			return 0, 3 // Illegal UTF-8: not a continuation byte.
-		}
-		return t.lookupValue(uint32(i), c3), 4
-	}
-	// Illegal rune
-	return 0, 1
-}
-
-// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
-// s must start with a full and valid UTF-8 encoded rune.
-func (t *idnaTrie) lookupStringUnsafe(s string) uint16 {
-	c0 := s[0]
-	if c0 < 0x80 { // is ASCII
-		return idnaValues[c0]
-	}
-	i := idnaIndex[c0]
-	if c0 < 0xE0 { // 2-byte UTF-8
-		return t.lookupValue(uint32(i), s[1])
-	}
-	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
-	if c0 < 0xF0 { // 3-byte UTF-8
-		return t.lookupValue(uint32(i), s[2])
-	}
-	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
-	if c0 < 0xF8 { // 4-byte UTF-8
-		return t.lookupValue(uint32(i), s[3])
-	}
-	return 0
-}
-
-// idnaTrie. Total size: 29708 bytes (29.01 KiB). Checksum: c3ecc76d8fffa6e6.
-type idnaTrie struct{}
-
-func newIdnaTrie(i int) *idnaTrie {
-	return &idnaTrie{}
-}
-
-// lookupValue determines the type of block n and looks up the value for b.
-func (t *idnaTrie) lookupValue(n uint32, b byte) uint16 {
-	switch {
-	case n < 125:
-		return uint16(idnaValues[n<<6+uint32(b)])
-	default:
-		n -= 125
-		return uint16(idnaSparse.lookup(n, b))
-	}
-}
-
-// idnaValues: 127 blocks, 8128 entries, 16256 bytes
-// The third block is the zero block.
-var idnaValues = [8128]uint16{
-	// Block 0x0, offset 0x0
-	0x00: 0x0080, 0x01: 0x0080, 0x02: 0x0080, 0x03: 0x0080, 0x04: 0x0080, 0x05: 0x0080,
-	0x06: 0x0080, 0x07: 0x0080, 0x08: 0x0080, 0x09: 0x0080, 0x0a: 0x0080, 0x0b: 0x0080,
-	0x0c: 0x0080, 0x0d: 0x0080, 0x0e: 0x0080, 0x0f: 0x0080, 0x10: 0x0080, 0x11: 0x0080,
-	0x12: 0x0080, 0x13: 0x0080, 0x14: 0x0080, 0x15: 0x0080, 0x16: 0x0080, 0x17: 0x0080,
-	0x18: 0x0080, 0x19: 0x0080, 0x1a: 0x0080, 0x1b: 0x0080, 0x1c: 0x0080, 0x1d: 0x0080,
-	0x1e: 0x0080, 0x1f: 0x0080, 0x20: 0x0080, 0x21: 0x0080, 0x22: 0x0080, 0x23: 0x0080,
-	0x24: 0x0080, 0x25: 0x0080, 0x26: 0x0080, 0x27: 0x0080, 0x28: 0x0080, 0x29: 0x0080,
-	0x2a: 0x0080, 0x2b: 0x0080, 0x2c: 0x0080, 0x2d: 0x0008, 0x2e: 0x0008, 0x2f: 0x0080,
-	0x30: 0x0008, 0x31: 0x0008, 0x32: 0x0008, 0x33: 0x0008, 0x34: 0x0008, 0x35: 0x0008,
-	0x36: 0x0008, 0x37: 0x0008, 0x38: 0x0008, 0x39: 0x0008, 0x3a: 0x0080, 0x3b: 0x0080,
-	0x3c: 0x0080, 0x3d: 0x0080, 0x3e: 0x0080, 0x3f: 0x0080,
-	// Block 0x1, offset 0x40
-	0x40: 0x0080, 0x41: 0xe105, 0x42: 0xe105, 0x43: 0xe105, 0x44: 0xe105, 0x45: 0xe105,
-	0x46: 0xe105, 0x47: 0xe105, 0x48: 0xe105, 0x49: 0xe105, 0x4a: 0xe105, 0x4b: 0xe105,
-	0x4c: 0xe105, 0x4d: 0xe105, 0x4e: 0xe105, 0x4f: 0xe105, 0x50: 0xe105, 0x51: 0xe105,
-	0x52: 0xe105, 0x53: 0xe105, 0x54: 0xe105, 0x55: 0xe105, 0x56: 0xe105, 0x57: 0xe105,
-	0x58: 0xe105, 0x59: 0xe105, 0x5a: 0xe105, 0x5b: 0x0080, 0x5c: 0x0080, 0x5d: 0x0080,
-	0x5e: 0x0080, 0x5f: 0x0080, 0x60: 0x0080, 0x61: 0x0008, 0x62: 0x0008, 0x63: 0x0008,
-	0x64: 0x0008, 0x65: 0x0008, 0x66: 0x0008, 0x67: 0x0008, 0x68: 0x0008, 0x69: 0x0008,
-	0x6a: 0x0008, 0x6b: 0x0008, 0x6c: 0x0008, 0x6d: 0x0008, 0x6e: 0x0008, 0x6f: 0x0008,
-	0x70: 0x0008, 0x71: 0x0008, 0x72: 0x0008, 0x73: 0x0008, 0x74: 0x0008, 0x75: 0x0008,
-	0x76: 0x0008, 0x77: 0x0008, 0x78: 0x0008, 0x79: 0x0008, 0x7a: 0x0008, 0x7b: 0x0080,
-	0x7c: 0x0080, 0x7d: 0x0080, 0x7e: 0x0080, 0x7f: 0x0080,
-	// Block 0x2, offset 0x80
-	// Block 0x3, offset 0xc0
-	0xc0: 0x0040, 0xc1: 0x0040, 0xc2: 0x0040, 0xc3: 0x0040, 0xc4: 0x0040, 0xc5: 0x0040,
-	0xc6: 0x0040, 0xc7: 0x0040, 0xc8: 0x0040, 0xc9: 0x0040, 0xca: 0x0040, 0xcb: 0x0040,
-	0xcc: 0x0040, 0xcd: 0x0040, 0xce: 0x0040, 0xcf: 0x0040, 0xd0: 0x0040, 0xd1: 0x0040,
-	0xd2: 0x0040, 0xd3: 0x0040, 0xd4: 0x0040, 0xd5: 0x0040, 0xd6: 0x0040, 0xd7: 0x0040,
-	0xd8: 0x0040, 0xd9: 0x0040, 0xda: 0x0040, 0xdb: 0x0040, 0xdc: 0x0040, 0xdd: 0x0040,
-	0xde: 0x0040, 0xdf: 0x0040, 0xe0: 0x000a, 0xe1: 0x0018, 0xe2: 0x0018, 0xe3: 0x0018,
-	0xe4: 0x0018, 0xe5: 0x0018, 0xe6: 0x0018, 0xe7: 0x0018, 0xe8: 0x001a, 0xe9: 0x0018,
-	0xea: 0x0039, 0xeb: 0x0018, 0xec: 0x0018, 0xed: 0x03c0, 0xee: 0x0018, 0xef: 0x004a,
-	0xf0: 0x0018, 0xf1: 0x0018, 0xf2: 0x0069, 0xf3: 0x0079, 0xf4: 0x008a, 0xf5: 0x0005,
-	0xf6: 0x0018, 0xf7: 0x0008, 0xf8: 0x00aa, 0xf9: 0x00c9, 0xfa: 0x00d9, 0xfb: 0x0018,
-	0xfc: 0x00e9, 0xfd: 0x0119, 0xfe: 0x0149, 0xff: 0x0018,
-	// Block 0x4, offset 0x100
-	0x100: 0xe00d, 0x101: 0x0008, 0x102: 0xe00d, 0x103: 0x0008, 0x104: 0xe00d, 0x105: 0x0008,
-	0x106: 0xe00d, 0x107: 0x0008, 0x108: 0xe00d, 0x109: 0x0008, 0x10a: 0xe00d, 0x10b: 0x0008,
-	0x10c: 0xe00d, 0x10d: 0x0008, 0x10e: 0xe00d, 0x10f: 0x0008, 0x110: 0xe00d, 0x111: 0x0008,
-	0x112: 0xe00d, 0x113: 0x0008, 0x114: 0xe00d, 0x115: 0x0008, 0x116: 0xe00d, 0x117: 0x0008,
-	0x118: 0xe00d, 0x119: 0x0008, 0x11a: 0xe00d, 0x11b: 0x0008, 0x11c: 0xe00d, 0x11d: 0x0008,
-	0x11e: 0xe00d, 0x11f: 0x0008, 0x120: 0xe00d, 0x121: 0x0008, 0x122: 0xe00d, 0x123: 0x0008,
-	0x124: 0xe00d, 0x125: 0x0008, 0x126: 0xe00d, 0x127: 0x0008, 0x128: 0xe00d, 0x129: 0x0008,
-	0x12a: 0xe00d, 0x12b: 0x0008, 0x12c: 0xe00d, 0x12d: 0x0008, 0x12e: 0xe00d, 0x12f: 0x0008,
-	0x130: 0x0179, 0x131: 0x0008, 0x132: 0x0035, 0x133: 0x004d, 0x134: 0xe00d, 0x135: 0x0008,
-	0x136: 0xe00d, 0x137: 0x0008, 0x138: 0x0008, 0x139: 0xe01d, 0x13a: 0x0008, 0x13b: 0xe03d,
-	0x13c: 0x0008, 0x13d: 0xe01d, 0x13e: 0x0008, 0x13f: 0x0199,
-	// Block 0x5, offset 0x140
-	0x140: 0x0199, 0x141: 0xe01d, 0x142: 0x0008, 0x143: 0xe03d, 0x144: 0x0008, 0x145: 0xe01d,
-	0x146: 0x0008, 0x147: 0xe07d, 0x148: 0x0008, 0x149: 0x01b9, 0x14a: 0xe00d, 0x14b: 0x0008,
-	0x14c: 0xe00d, 0x14d: 0x0008, 0x14e: 0xe00d, 0x14f: 0x0008, 0x150: 0xe00d, 0x151: 0x0008,
-	0x152: 0xe00d, 0x153: 0x0008, 0x154: 0xe00d, 0x155: 0x0008, 0x156: 0xe00d, 0x157: 0x0008,
-	0x158: 0xe00d, 0x159: 0x0008, 0x15a: 0xe00d, 0x15b: 0x0008, 0x15c: 0xe00d, 0x15d: 0x0008,
-	0x15e: 0xe00d, 0x15f: 0x0008, 0x160: 0xe00d, 0x161: 0x0008, 0x162: 0xe00d, 0x163: 0x0008,
-	0x164: 0xe00d, 0x165: 0x0008, 0x166: 0xe00d, 0x167: 0x0008, 0x168: 0xe00d, 0x169: 0x0008,
-	0x16a: 0xe00d, 0x16b: 0x0008, 0x16c: 0xe00d, 0x16d: 0x0008, 0x16e: 0xe00d, 0x16f: 0x0008,
-	0x170: 0xe00d, 0x171: 0x0008, 0x172: 0xe00d, 0x173: 0x0008, 0x174: 0xe00d, 0x175: 0x0008,
-	0x176: 0xe00d, 0x177: 0x0008, 0x178: 0x0065, 0x179: 0xe01d, 0x17a: 0x0008, 0x17b: 0xe03d,
-	0x17c: 0x0008, 0x17d: 0xe01d, 0x17e: 0x0008, 0x17f: 0x01d9,
-	// Block 0x6, offset 0x180
-	0x180: 0x0008, 0x181: 0x007d, 0x182: 0xe00d, 0x183: 0x0008, 0x184: 0xe00d, 0x185: 0x0008,
-	0x186: 0x007d, 0x187: 0xe07d, 0x188: 0x0008, 0x189: 0x0095, 0x18a: 0x00ad, 0x18b: 0xe03d,
-	0x18c: 0x0008, 0x18d: 0x0008, 0x18e: 0x00c5, 0x18f: 0x00dd, 0x190: 0x00f5, 0x191: 0xe01d,
-	0x192: 0x0008, 0x193: 0x010d, 0x194: 0x0125, 0x195: 0x0008, 0x196: 0x013d, 0x197: 0x013d,
-	0x198: 0xe00d, 0x199: 0x0008, 0x19a: 0x0008, 0x19b: 0x0008, 0x19c: 0x010d, 0x19d: 0x0155,
-	0x19e: 0x0008, 0x19f: 0x016d, 0x1a0: 0xe00d, 0x1a1: 0x0008, 0x1a2: 0xe00d, 0x1a3: 0x0008,
-	0x1a4: 0xe00d, 0x1a5: 0x0008, 0x1a6: 0x0185, 0x1a7: 0xe07d, 0x1a8: 0x0008, 0x1a9: 0x019d,
-	0x1aa: 0x0008, 0x1ab: 0x0008, 0x1ac: 0xe00d, 0x1ad: 0x0008, 0x1ae: 0x0185, 0x1af: 0xe0fd,
-	0x1b0: 0x0008, 0x1b1: 0x01b5, 0x1b2: 0x01cd, 0x1b3: 0xe03d, 0x1b4: 0x0008, 0x1b5: 0xe01d,
-	0x1b6: 0x0008, 0x1b7: 0x01e5, 0x1b8: 0xe00d, 0x1b9: 0x0008, 0x1ba: 0x0008, 0x1bb: 0x0008,
-	0x1bc: 0xe00d, 0x1bd: 0x0008, 0x1be: 0x0008, 0x1bf: 0x0008,
-	// Block 0x7, offset 0x1c0
-	0x1c0: 0x0008, 0x1c1: 0x0008, 0x1c2: 0x0008, 0x1c3: 0x0008, 0x1c4: 0x01e9, 0x1c5: 0x01e9,
-	0x1c6: 0x01e9, 0x1c7: 0x01fd, 0x1c8: 0x0215, 0x1c9: 0x022d, 0x1ca: 0x0245, 0x1cb: 0x025d,
-	0x1cc: 0x0275, 0x1cd: 0xe01d, 0x1ce: 0x0008, 0x1cf: 0xe0fd, 0x1d0: 0x0008, 0x1d1: 0xe01d,
-	0x1d2: 0x0008, 0x1d3: 0xe03d, 0x1d4: 0x0008, 0x1d5: 0xe01d, 0x1d6: 0x0008, 0x1d7: 0xe07d,
-	0x1d8: 0x0008, 0x1d9: 0xe01d, 0x1da: 0x0008, 0x1db: 0xe03d, 0x1dc: 0x0008, 0x1dd: 0x0008,
-	0x1de: 0xe00d, 0x1df: 0x0008, 0x1e0: 0xe00d, 0x1e1: 0x0008, 0x1e2: 0xe00d, 0x1e3: 0x0008,
-	0x1e4: 0xe00d, 0x1e5: 0x0008, 0x1e6: 0xe00d, 0x1e7: 0x0008, 0x1e8: 0xe00d, 0x1e9: 0x0008,
-	0x1ea: 0xe00d, 0x1eb: 0x0008, 0x1ec: 0xe00d, 0x1ed: 0x0008, 0x1ee: 0xe00d, 0x1ef: 0x0008,
-	0x1f0: 0x0008, 0x1f1: 0x028d, 0x1f2: 0x02a5, 0x1f3: 0x02bd, 0x1f4: 0xe00d, 0x1f5: 0x0008,
-	0x1f6: 0x02d5, 0x1f7: 0x02ed, 0x1f8: 0xe00d, 0x1f9: 0x0008, 0x1fa: 0xe00d, 0x1fb: 0x0008,
-	0x1fc: 0xe00d, 0x1fd: 0x0008, 0x1fe: 0xe00d, 0x1ff: 0x0008,
-	// Block 0x8, offset 0x200
-	0x200: 0xe00d, 0x201: 0x0008, 0x202: 0xe00d, 0x203: 0x0008, 0x204: 0xe00d, 0x205: 0x0008,
-	0x206: 0xe00d, 0x207: 0x0008, 0x208: 0xe00d, 0x209: 0x0008, 0x20a: 0xe00d, 0x20b: 0x0008,
-	0x20c: 0xe00d, 0x20d: 0x0008, 0x20e: 0xe00d, 0x20f: 0x0008, 0x210: 0xe00d, 0x211: 0x0008,
-	0x212: 0xe00d, 0x213: 0x0008, 0x214: 0xe00d, 0x215: 0x0008, 0x216: 0xe00d, 0x217: 0x0008,
-	0x218: 0xe00d, 0x219: 0x0008, 0x21a: 0xe00d, 0x21b: 0x0008, 0x21c: 0xe00d, 0x21d: 0x0008,
-	0x21e: 0xe00d, 0x21f: 0x0008, 0x220: 0x0305, 0x221: 0x0008, 0x222: 0xe00d, 0x223: 0x0008,
-	0x224: 0xe00d, 0x225: 0x0008, 0x226: 0xe00d, 0x227: 0x0008, 0x228: 0xe00d, 0x229: 0x0008,
-	0x22a: 0xe00d, 0x22b: 0x0008, 0x22c: 0xe00d, 0x22d: 0x0008, 0x22e: 0xe00d, 0x22f: 0x0008,
-	0x230: 0xe00d, 0x231: 0x0008, 0x232: 0xe00d, 0x233: 0x0008, 0x234: 0x0008, 0x235: 0x0008,
-	0x236: 0x0008, 0x237: 0x0008, 0x238: 0x0008, 0x239: 0x0008, 0x23a: 0x0209, 0x23b: 0xe03d,
-	0x23c: 0x0008, 0x23d: 0x031d, 0x23e: 0x0229, 0x23f: 0x0008,
-	// Block 0x9, offset 0x240
-	0x240: 0x0008, 0x241: 0x0008, 0x242: 0x0018, 0x243: 0x0018, 0x244: 0x0018, 0x245: 0x0018,
-	0x246: 0x0008, 0x247: 0x0008, 0x248: 0x0008, 0x249: 0x0008, 0x24a: 0x0008, 0x24b: 0x0008,
-	0x24c: 0x0008, 0x24d: 0x0008, 0x24e: 0x0008, 0x24f: 0x0008, 0x250: 0x0008, 0x251: 0x0008,
-	0x252: 0x0018, 0x253: 0x0018, 0x254: 0x0018, 0x255: 0x0018, 0x256: 0x0018, 0x257: 0x0018,
-	0x258: 0x029a, 0x259: 0x02ba, 0x25a: 0x02da, 0x25b: 0x02fa, 0x25c: 0x031a, 0x25d: 0x033a,
-	0x25e: 0x0018, 0x25f: 0x0018, 0x260: 0x03ad, 0x261: 0x0359, 0x262: 0x01d9, 0x263: 0x0369,
-	0x264: 0x03c5, 0x265: 0x0018, 0x266: 0x0018, 0x267: 0x0018, 0x268: 0x0018, 0x269: 0x0018,
-	0x26a: 0x0018, 0x26b: 0x0018, 0x26c: 0x0008, 0x26d: 0x0018, 0x26e: 0x0008, 0x26f: 0x0018,
-	0x270: 0x0018, 0x271: 0x0018, 0x272: 0x0018, 0x273: 0x0018, 0x274: 0x0018, 0x275: 0x0018,
-	0x276: 0x0018, 0x277: 0x0018, 0x278: 0x0018, 0x279: 0x0018, 0x27a: 0x0018, 0x27b: 0x0018,
-	0x27c: 0x0018, 0x27d: 0x0018, 0x27e: 0x0018, 0x27f: 0x0018,
-	// Block 0xa, offset 0x280
-	0x280: 0x03dd, 0x281: 0x03dd, 0x282: 0x3308, 0x283: 0x03f5, 0x284: 0x0379, 0x285: 0x040d,
-	0x286: 0x3308, 0x287: 0x3308, 0x288: 0x3308, 0x289: 0x3308, 0x28a: 0x3308, 0x28b: 0x3308,
-	0x28c: 0x3308, 0x28d: 0x3308, 0x28e: 0x3308, 0x28f: 0x33c0, 0x290: 0x3308, 0x291: 0x3308,
-	0x292: 0x3308, 0x293: 0x3308, 0x294: 0x3308, 0x295: 0x3308, 0x296: 0x3308, 0x297: 0x3308,
-	0x298: 0x3308, 0x299: 0x3308, 0x29a: 0x3308, 0x29b: 0x3308, 0x29c: 0x3308, 0x29d: 0x3308,
-	0x29e: 0x3308, 0x29f: 0x3308, 0x2a0: 0x3308, 0x2a1: 0x3308, 0x2a2: 0x3308, 0x2a3: 0x3308,
-	0x2a4: 0x3308, 0x2a5: 0x3308, 0x2a6: 0x3308, 0x2a7: 0x3308, 0x2a8: 0x3308, 0x2a9: 0x3308,
-	0x2aa: 0x3308, 0x2ab: 0x3308, 0x2ac: 0x3308, 0x2ad: 0x3308, 0x2ae: 0x3308, 0x2af: 0x3308,
-	0x2b0: 0xe00d, 0x2b1: 0x0008, 0x2b2: 0xe00d, 0x2b3: 0x0008, 0x2b4: 0x0425, 0x2b5: 0x0008,
-	0x2b6: 0xe00d, 0x2b7: 0x0008, 0x2b8: 0x0040, 0x2b9: 0x0040, 0x2ba: 0x03a2, 0x2bb: 0x0008,
-	0x2bc: 0x0008, 0x2bd: 0x0008, 0x2be: 0x03c2, 0x2bf: 0x043d,
-	// Block 0xb, offset 0x2c0
-	0x2c0: 0x0040, 0x2c1: 0x0040, 0x2c2: 0x0040, 0x2c3: 0x0040, 0x2c4: 0x008a, 0x2c5: 0x03d2,
-	0x2c6: 0xe155, 0x2c7: 0x0455, 0x2c8: 0xe12d, 0x2c9: 0xe13d, 0x2ca: 0xe12d, 0x2cb: 0x0040,
-	0x2cc: 0x03dd, 0x2cd: 0x0040, 0x2ce: 0x046d, 0x2cf: 0x0485, 0x2d0: 0x0008, 0x2d1: 0xe105,
-	0x2d2: 0xe105, 0x2d3: 0xe105, 0x2d4: 0xe105, 0x2d5: 0xe105, 0x2d6: 0xe105, 0x2d7: 0xe105,
-	0x2d8: 0xe105, 0x2d9: 0xe105, 0x2da: 0xe105, 0x2db: 0xe105, 0x2dc: 0xe105, 0x2dd: 0xe105,
-	0x2de: 0xe105, 0x2df: 0xe105, 0x2e0: 0x049d, 0x2e1: 0x049d, 0x2e2: 0x0040, 0x2e3: 0x049d,
-	0x2e4: 0x049d, 0x2e5: 0x049d, 0x2e6: 0x049d, 0x2e7: 0x049d, 0x2e8: 0x049d, 0x2e9: 0x049d,
-	0x2ea: 0x049d, 0x2eb: 0x049d, 0x2ec: 0x0008, 0x2ed: 0x0008, 0x2ee: 0x0008, 0x2ef: 0x0008,
-	0x2f0: 0x0008, 0x2f1: 0x0008, 0x2f2: 0x0008, 0x2f3: 0x0008, 0x2f4: 0x0008, 0x2f5: 0x0008,
-	0x2f6: 0x0008, 0x2f7: 0x0008, 0x2f8: 0x0008, 0x2f9: 0x0008, 0x2fa: 0x0008, 0x2fb: 0x0008,
-	0x2fc: 0x0008, 0x2fd: 0x0008, 0x2fe: 0x0008, 0x2ff: 0x0008,
-	// Block 0xc, offset 0x300
-	0x300: 0x0008, 0x301: 0x0008, 0x302: 0xe00f, 0x303: 0x0008, 0x304: 0x0008, 0x305: 0x0008,
-	0x306: 0x0008, 0x307: 0x0008, 0x308: 0x0008, 0x309: 0x0008, 0x30a: 0x0008, 0x30b: 0x0008,
-	0x30c: 0x0008, 0x30d: 0x0008, 0x30e: 0x0008, 0x30f: 0xe0c5, 0x310: 0x04b5, 0x311: 0x04cd,
-	0x312: 0xe0bd, 0x313: 0xe0f5, 0x314: 0xe0fd, 0x315: 0xe09d, 0x316: 0xe0b5, 0x317: 0x0008,
-	0x318: 0xe00d, 0x319: 0x0008, 0x31a: 0xe00d, 0x31b: 0x0008, 0x31c: 0xe00d, 0x31d: 0x0008,
-	0x31e: 0xe00d, 0x31f: 0x0008, 0x320: 0xe00d, 0x321: 0x0008, 0x322: 0xe00d, 0x323: 0x0008,
-	0x324: 0xe00d, 0x325: 0x0008, 0x326: 0xe00d, 0x327: 0x0008, 0x328: 0xe00d, 0x329: 0x0008,
-	0x32a: 0xe00d, 0x32b: 0x0008, 0x32c: 0xe00d, 0x32d: 0x0008, 0x32e: 0xe00d, 0x32f: 0x0008,
-	0x330: 0x04e5, 0x331: 0xe185, 0x332: 0xe18d, 0x333: 0x0008, 0x334: 0x04fd, 0x335: 0x03dd,
-	0x336: 0x0018, 0x337: 0xe07d, 0x338: 0x0008, 0x339: 0xe1d5, 0x33a: 0xe00d, 0x33b: 0x0008,
-	0x33c: 0x0008, 0x33d: 0x0515, 0x33e: 0x052d, 0x33f: 0x052d,
-	// Block 0xd, offset 0x340
-	0x340: 0x0008, 0x341: 0x0008, 0x342: 0x0008, 0x343: 0x0008, 0x344: 0x0008, 0x345: 0x0008,
-	0x346: 0x0008, 0x347: 0x0008, 0x348: 0x0008, 0x349: 0x0008, 0x34a: 0x0008, 0x34b: 0x0008,
-	0x34c: 0x0008, 0x34d: 0x0008, 0x34e: 0x0008, 0x34f: 0x0008, 0x350: 0x0008, 0x351: 0x0008,
-	0x352: 0x0008, 0x353: 0x0008, 0x354: 0x0008, 0x355: 0x0008, 0x356: 0x0008, 0x357: 0x0008,
-	0x358: 0x0008, 0x359: 0x0008, 0x35a: 0x0008, 0x35b: 0x0008, 0x35c: 0x0008, 0x35d: 0x0008,
-	0x35e: 0x0008, 0x35f: 0x0008, 0x360: 0xe00d, 0x361: 0x0008, 0x362: 0xe00d, 0x363: 0x0008,
-	0x364: 0xe00d, 0x365: 0x0008, 0x366: 0xe00d, 0x367: 0x0008, 0x368: 0xe00d, 0x369: 0x0008,
-	0x36a: 0xe00d, 0x36b: 0x0008, 0x36c: 0xe00d, 0x36d: 0x0008, 0x36e: 0xe00d, 0x36f: 0x0008,
-	0x370: 0xe00d, 0x371: 0x0008, 0x372: 0xe00d, 0x373: 0x0008, 0x374: 0xe00d, 0x375: 0x0008,
-	0x376: 0xe00d, 0x377: 0x0008, 0x378: 0xe00d, 0x379: 0x0008, 0x37a: 0xe00d, 0x37b: 0x0008,
-	0x37c: 0xe00d, 0x37d: 0x0008, 0x37e: 0xe00d, 0x37f: 0x0008,
-	// Block 0xe, offset 0x380
-	0x380: 0xe00d, 0x381: 0x0008, 0x382: 0x0018, 0x383: 0x3308, 0x384: 0x3308, 0x385: 0x3308,
-	0x386: 0x3308, 0x387: 0x3308, 0x388: 0x3318, 0x389: 0x3318, 0x38a: 0xe00d, 0x38b: 0x0008,
-	0x38c: 0xe00d, 0x38d: 0x0008, 0x38e: 0xe00d, 0x38f: 0x0008, 0x390: 0xe00d, 0x391: 0x0008,
-	0x392: 0xe00d, 0x393: 0x0008, 0x394: 0xe00d, 0x395: 0x0008, 0x396: 0xe00d, 0x397: 0x0008,
-	0x398: 0xe00d, 0x399: 0x0008, 0x39a: 0xe00d, 0x39b: 0x0008, 0x39c: 0xe00d, 0x39d: 0x0008,
-	0x39e: 0xe00d, 0x39f: 0x0008, 0x3a0: 0xe00d, 0x3a1: 0x0008, 0x3a2: 0xe00d, 0x3a3: 0x0008,
-	0x3a4: 0xe00d, 0x3a5: 0x0008, 0x3a6: 0xe00d, 0x3a7: 0x0008, 0x3a8: 0xe00d, 0x3a9: 0x0008,
-	0x3aa: 0xe00d, 0x3ab: 0x0008, 0x3ac: 0xe00d, 0x3ad: 0x0008, 0x3ae: 0xe00d, 0x3af: 0x0008,
-	0x3b0: 0xe00d, 0x3b1: 0x0008, 0x3b2: 0xe00d, 0x3b3: 0x0008, 0x3b4: 0xe00d, 0x3b5: 0x0008,
-	0x3b6: 0xe00d, 0x3b7: 0x0008, 0x3b8: 0xe00d, 0x3b9: 0x0008, 0x3ba: 0xe00d, 0x3bb: 0x0008,
-	0x3bc: 0xe00d, 0x3bd: 0x0008, 0x3be: 0xe00d, 0x3bf: 0x0008,
-	// Block 0xf, offset 0x3c0
-	0x3c0: 0x0040, 0x3c1: 0xe01d, 0x3c2: 0x0008, 0x3c3: 0xe03d, 0x3c4: 0x0008, 0x3c5: 0xe01d,
-	0x3c6: 0x0008, 0x3c7: 0xe07d, 0x3c8: 0x0008, 0x3c9: 0xe01d, 0x3ca: 0x0008, 0x3cb: 0xe03d,
-	0x3cc: 0x0008, 0x3cd: 0xe01d, 0x3ce: 0x0008, 0x3cf: 0x0008, 0x3d0: 0xe00d, 0x3d1: 0x0008,
-	0x3d2: 0xe00d, 0x3d3: 0x0008, 0x3d4: 0xe00d, 0x3d5: 0x0008, 0x3d6: 0xe00d, 0x3d7: 0x0008,
-	0x3d8: 0xe00d, 0x3d9: 0x0008, 0x3da: 0xe00d, 0x3db: 0x0008, 0x3dc: 0xe00d, 0x3dd: 0x0008,
-	0x3de: 0xe00d, 0x3df: 0x0008, 0x3e0: 0xe00d, 0x3e1: 0x0008, 0x3e2: 0xe00d, 0x3e3: 0x0008,
-	0x3e4: 0xe00d, 0x3e5: 0x0008, 0x3e6: 0xe00d, 0x3e7: 0x0008, 0x3e8: 0xe00d, 0x3e9: 0x0008,
-	0x3ea: 0xe00d, 0x3eb: 0x0008, 0x3ec: 0xe00d, 0x3ed: 0x0008, 0x3ee: 0xe00d, 0x3ef: 0x0008,
-	0x3f0: 0xe00d, 0x3f1: 0x0008, 0x3f2: 0xe00d, 0x3f3: 0x0008, 0x3f4: 0xe00d, 0x3f5: 0x0008,
-	0x3f6: 0xe00d, 0x3f7: 0x0008, 0x3f8: 0xe00d, 0x3f9: 0x0008, 0x3fa: 0xe00d, 0x3fb: 0x0008,
-	0x3fc: 0xe00d, 0x3fd: 0x0008, 0x3fe: 0xe00d, 0x3ff: 0x0008,
-	// Block 0x10, offset 0x400
-	0x400: 0xe00d, 0x401: 0x0008, 0x402: 0xe00d, 0x403: 0x0008, 0x404: 0xe00d, 0x405: 0x0008,
-	0x406: 0xe00d, 0x407: 0x0008, 0x408: 0xe00d, 0x409: 0x0008, 0x40a: 0xe00d, 0x40b: 0x0008,
-	0x40c: 0xe00d, 0x40d: 0x0008, 0x40e: 0xe00d, 0x40f: 0x0008, 0x410: 0xe00d, 0x411: 0x0008,
-	0x412: 0xe00d, 0x413: 0x0008, 0x414: 0xe00d, 0x415: 0x0008, 0x416: 0xe00d, 0x417: 0x0008,
-	0x418: 0xe00d, 0x419: 0x0008, 0x41a: 0xe00d, 0x41b: 0x0008, 0x41c: 0xe00d, 0x41d: 0x0008,
-	0x41e: 0xe00d, 0x41f: 0x0008, 0x420: 0xe00d, 0x421: 0x0008, 0x422: 0xe00d, 0x423: 0x0008,
-	0x424: 0xe00d, 0x425: 0x0008, 0x426: 0xe00d, 0x427: 0x0008, 0x428: 0xe00d, 0x429: 0x0008,
-	0x42a: 0xe00d, 0x42b: 0x0008, 0x42c: 0xe00d, 0x42d: 0x0008, 0x42e: 0xe00d, 0x42f: 0x0008,
-	0x430: 0x0040, 0x431: 0x03f5, 0x432: 0x03f5, 0x433: 0x03f5, 0x434: 0x03f5, 0x435: 0x03f5,
-	0x436: 0x03f5, 0x437: 0x03f5, 0x438: 0x03f5, 0x439: 0x03f5, 0x43a: 0x03f5, 0x43b: 0x03f5,
-	0x43c: 0x03f5, 0x43d: 0x03f5, 0x43e: 0x03f5, 0x43f: 0x03f5,
-	// Block 0x11, offset 0x440
-	0x440: 0x0840, 0x441: 0x0840, 0x442: 0x0840, 0x443: 0x0840, 0x444: 0x0840, 0x445: 0x0840,
-	0x446: 0x0018, 0x447: 0x0018, 0x448: 0x0818, 0x449: 0x0018, 0x44a: 0x0018, 0x44b: 0x0818,
-	0x44c: 0x0018, 0x44d: 0x0818, 0x44e: 0x0018, 0x44f: 0x0018, 0x450: 0x3308, 0x451: 0x3308,
-	0x452: 0x3308, 0x453: 0x3308, 0x454: 0x3308, 0x455: 0x3308, 0x456: 0x3308, 0x457: 0x3308,
-	0x458: 0x3308, 0x459: 0x3308, 0x45a: 0x3308, 0x45b: 0x0818, 0x45c: 0x0b40, 0x45d: 0x0040,
-	0x45e: 0x0818, 0x45f: 0x0818, 0x460: 0x0a08, 0x461: 0x0808, 0x462: 0x0c08, 0x463: 0x0c08,
-	0x464: 0x0c08, 0x465: 0x0c08, 0x466: 0x0a08, 0x467: 0x0c08, 0x468: 0x0a08, 0x469: 0x0c08,
-	0x46a: 0x0a08, 0x46b: 0x0a08, 0x46c: 0x0a08, 0x46d: 0x0a08, 0x46e: 0x0a08, 0x46f: 0x0c08,
-	0x470: 0x0c08, 0x471: 0x0c08, 0x472: 0x0c08, 0x473: 0x0a08, 0x474: 0x0a08, 0x475: 0x0a08,
-	0x476: 0x0a08, 0x477: 0x0a08, 0x478: 0x0a08, 0x479: 0x0a08, 0x47a: 0x0a08, 0x47b: 0x0a08,
-	0x47c: 0x0a08, 0x47d: 0x0a08, 0x47e: 0x0a08, 0x47f: 0x0a08,
-	// Block 0x12, offset 0x480
-	0x480: 0x0818, 0x481: 0x0a08, 0x482: 0x0a08, 0x483: 0x0a08, 0x484: 0x0a08, 0x485: 0x0a08,
-	0x486: 0x0a08, 0x487: 0x0a08, 0x488: 0x0c08, 0x489: 0x0a08, 0x48a: 0x0a08, 0x48b: 0x3308,
-	0x48c: 0x3308, 0x48d: 0x3308, 0x48e: 0x3308, 0x48f: 0x3308, 0x490: 0x3308, 0x491: 0x3308,
-	0x492: 0x3308, 0x493: 0x3308, 0x494: 0x3308, 0x495: 0x3308, 0x496: 0x3308, 0x497: 0x3308,
-	0x498: 0x3308, 0x499: 0x3308, 0x49a: 0x3308, 0x49b: 0x3308, 0x49c: 0x3308, 0x49d: 0x3308,
-	0x49e: 0x3308, 0x49f: 0x3308, 0x4a0: 0x0808, 0x4a1: 0x0808, 0x4a2: 0x0808, 0x4a3: 0x0808,
-	0x4a4: 0x0808, 0x4a5: 0x0808, 0x4a6: 0x0808, 0x4a7: 0x0808, 0x4a8: 0x0808, 0x4a9: 0x0808,
-	0x4aa: 0x0018, 0x4ab: 0x0818, 0x4ac: 0x0818, 0x4ad: 0x0818, 0x4ae: 0x0a08, 0x4af: 0x0a08,
-	0x4b0: 0x3308, 0x4b1: 0x0c08, 0x4b2: 0x0c08, 0x4b3: 0x0c08, 0x4b4: 0x0808, 0x4b5: 0x0429,
-	0x4b6: 0x0451, 0x4b7: 0x0479, 0x4b8: 0x04a1, 0x4b9: 0x0a08, 0x4ba: 0x0a08, 0x4bb: 0x0a08,
-	0x4bc: 0x0a08, 0x4bd: 0x0a08, 0x4be: 0x0a08, 0x4bf: 0x0a08,
-	// Block 0x13, offset 0x4c0
-	0x4c0: 0x0c08, 0x4c1: 0x0a08, 0x4c2: 0x0a08, 0x4c3: 0x0c08, 0x4c4: 0x0c08, 0x4c5: 0x0c08,
-	0x4c6: 0x0c08, 0x4c7: 0x0c08, 0x4c8: 0x0c08, 0x4c9: 0x0c08, 0x4ca: 0x0c08, 0x4cb: 0x0c08,
-	0x4cc: 0x0a08, 0x4cd: 0x0c08, 0x4ce: 0x0a08, 0x4cf: 0x0c08, 0x4d0: 0x0a08, 0x4d1: 0x0a08,
-	0x4d2: 0x0c08, 0x4d3: 0x0c08, 0x4d4: 0x0818, 0x4d5: 0x0c08, 0x4d6: 0x3308, 0x4d7: 0x3308,
-	0x4d8: 0x3308, 0x4d9: 0x3308, 0x4da: 0x3308, 0x4db: 0x3308, 0x4dc: 0x3308, 0x4dd: 0x0840,
-	0x4de: 0x0018, 0x4df: 0x3308, 0x4e0: 0x3308, 0x4e1: 0x3308, 0x4e2: 0x3308, 0x4e3: 0x3308,
-	0x4e4: 0x3308, 0x4e5: 0x0808, 0x4e6: 0x0808, 0x4e7: 0x3308, 0x4e8: 0x3308, 0x4e9: 0x0018,
-	0x4ea: 0x3308, 0x4eb: 0x3308, 0x4ec: 0x3308, 0x4ed: 0x3308, 0x4ee: 0x0c08, 0x4ef: 0x0c08,
-	0x4f0: 0x0008, 0x4f1: 0x0008, 0x4f2: 0x0008, 0x4f3: 0x0008, 0x4f4: 0x0008, 0x4f5: 0x0008,
-	0x4f6: 0x0008, 0x4f7: 0x0008, 0x4f8: 0x0008, 0x4f9: 0x0008, 0x4fa: 0x0a08, 0x4fb: 0x0a08,
-	0x4fc: 0x0a08, 0x4fd: 0x0808, 0x4fe: 0x0808, 0x4ff: 0x0a08,
-	// Block 0x14, offset 0x500
-	0x500: 0x0818, 0x501: 0x0818, 0x502: 0x0818, 0x503: 0x0818, 0x504: 0x0818, 0x505: 0x0818,
-	0x506: 0x0818, 0x507: 0x0818, 0x508: 0x0818, 0x509: 0x0818, 0x50a: 0x0818, 0x50b: 0x0818,
-	0x50c: 0x0818, 0x50d: 0x0818, 0x50e: 0x0040, 0x50f: 0x0b40, 0x510: 0x0c08, 0x511: 0x3308,
-	0x512: 0x0a08, 0x513: 0x0a08, 0x514: 0x0a08, 0x515: 0x0c08, 0x516: 0x0c08, 0x517: 0x0c08,
-	0x518: 0x0c08, 0x519: 0x0c08, 0x51a: 0x0a08, 0x51b: 0x0a08, 0x51c: 0x0a08, 0x51d: 0x0a08,
-	0x51e: 0x0c08, 0x51f: 0x0a08, 0x520: 0x0a08, 0x521: 0x0a08, 0x522: 0x0a08, 0x523: 0x0a08,
-	0x524: 0x0a08, 0x525: 0x0a08, 0x526: 0x0a08, 0x527: 0x0a08, 0x528: 0x0c08, 0x529: 0x0a08,
-	0x52a: 0x0c08, 0x52b: 0x0a08, 0x52c: 0x0c08, 0x52d: 0x0a08, 0x52e: 0x0a08, 0x52f: 0x0c08,
-	0x530: 0x3308, 0x531: 0x3308, 0x532: 0x3308, 0x533: 0x3308, 0x534: 0x3308, 0x535: 0x3308,
-	0x536: 0x3308, 0x537: 0x3308, 0x538: 0x3308, 0x539: 0x3308, 0x53a: 0x3308, 0x53b: 0x3308,
-	0x53c: 0x3308, 0x53d: 0x3308, 0x53e: 0x3308, 0x53f: 0x3308,
-	// Block 0x15, offset 0x540
-	0x540: 0x0c08, 0x541: 0x0a08, 0x542: 0x0a08, 0x543: 0x0a08, 0x544: 0x0a08, 0x545: 0x0a08,
-	0x546: 0x0c08, 0x547: 0x0c08, 0x548: 0x0a08, 0x549: 0x0c08, 0x54a: 0x0a08, 0x54b: 0x0a08,
-	0x54c: 0x0a08, 0x54d: 0x0a08, 0x54e: 0x0a08, 0x54f: 0x0a08, 0x550: 0x0a08, 0x551: 0x0a08,
-	0x552: 0x0a08, 0x553: 0x0a08, 0x554: 0x0c08, 0x555: 0x0a08, 0x556: 0x0808, 0x557: 0x0808,
-	0x558: 0x0808, 0x559: 0x3308, 0x55a: 0x3308, 0x55b: 0x3308, 0x55c: 0x0040, 0x55d: 0x0040,
-	0x55e: 0x0818, 0x55f: 0x0040, 0x560: 0x0a08, 0x561: 0x0808, 0x562: 0x0a08, 0x563: 0x0a08,
-	0x564: 0x0a08, 0x565: 0x0a08, 0x566: 0x0808, 0x567: 0x0c08, 0x568: 0x0a08, 0x569: 0x0c08,
-	0x56a: 0x0c08, 0x56b: 0x0040, 0x56c: 0x0040, 0x56d: 0x0040, 0x56e: 0x0040, 0x56f: 0x0040,
-	0x570: 0x0040, 0x571: 0x0040, 0x572: 0x0040, 0x573: 0x0040, 0x574: 0x0040, 0x575: 0x0040,
-	0x576: 0x0040, 0x577: 0x0040, 0x578: 0x0040, 0x579: 0x0040, 0x57a: 0x0040, 0x57b: 0x0040,
-	0x57c: 0x0040, 0x57d: 0x0040, 0x57e: 0x0040, 0x57f: 0x0040,
-	// Block 0x16, offset 0x580
-	0x580: 0x3008, 0x581: 0x3308, 0x582: 0x3308, 0x583: 0x3308, 0x584: 0x3308, 0x585: 0x3308,
-	0x586: 0x3308, 0x587: 0x3308, 0x588: 0x3308, 0x589: 0x3008, 0x58a: 0x3008, 0x58b: 0x3008,
-	0x58c: 0x3008, 0x58d: 0x3b08, 0x58e: 0x3008, 0x58f: 0x3008, 0x590: 0x0008, 0x591: 0x3308,
-	0x592: 0x3308, 0x593: 0x3308, 0x594: 0x3308, 0x595: 0x3308, 0x596: 0x3308, 0x597: 0x3308,
-	0x598: 0x04c9, 0x599: 0x0501, 0x59a: 0x0539, 0x59b: 0x0571, 0x59c: 0x05a9, 0x59d: 0x05e1,
-	0x59e: 0x0619, 0x59f: 0x0651, 0x5a0: 0x0008, 0x5a1: 0x0008, 0x5a2: 0x3308, 0x5a3: 0x3308,
-	0x5a4: 0x0018, 0x5a5: 0x0018, 0x5a6: 0x0008, 0x5a7: 0x0008, 0x5a8: 0x0008, 0x5a9: 0x0008,
-	0x5aa: 0x0008, 0x5ab: 0x0008, 0x5ac: 0x0008, 0x5ad: 0x0008, 0x5ae: 0x0008, 0x5af: 0x0008,
-	0x5b0: 0x0018, 0x5b1: 0x0008, 0x5b2: 0x0008, 0x5b3: 0x0008, 0x5b4: 0x0008, 0x5b5: 0x0008,
-	0x5b6: 0x0008, 0x5b7: 0x0008, 0x5b8: 0x0008, 0x5b9: 0x0008, 0x5ba: 0x0008, 0x5bb: 0x0008,
-	0x5bc: 0x0008, 0x5bd: 0x0008, 0x5be: 0x0008, 0x5bf: 0x0008,
-	// Block 0x17, offset 0x5c0
-	0x5c0: 0x0008, 0x5c1: 0x3308, 0x5c2: 0x3008, 0x5c3: 0x3008, 0x5c4: 0x0040, 0x5c5: 0x0008,
-	0x5c6: 0x0008, 0x5c7: 0x0008, 0x5c8: 0x0008, 0x5c9: 0x0008, 0x5ca: 0x0008, 0x5cb: 0x0008,
-	0x5cc: 0x0008, 0x5cd: 0x0040, 0x5ce: 0x0040, 0x5cf: 0x0008, 0x5d0: 0x0008, 0x5d1: 0x0040,
-	0x5d2: 0x0040, 0x5d3: 0x0008, 0x5d4: 0x0008, 0x5d5: 0x0008, 0x5d6: 0x0008, 0x5d7: 0x0008,
-	0x5d8: 0x0008, 0x5d9: 0x0008, 0x5da: 0x0008, 0x5db: 0x0008, 0x5dc: 0x0008, 0x5dd: 0x0008,
-	0x5de: 0x0008, 0x5df: 0x0008, 0x5e0: 0x0008, 0x5e1: 0x0008, 0x5e2: 0x0008, 0x5e3: 0x0008,
-	0x5e4: 0x0008, 0x5e5: 0x0008, 0x5e6: 0x0008, 0x5e7: 0x0008, 0x5e8: 0x0008, 0x5e9: 0x0040,
-	0x5ea: 0x0008, 0x5eb: 0x0008, 0x5ec: 0x0008, 0x5ed: 0x0008, 0x5ee: 0x0008, 0x5ef: 0x0008,
-	0x5f0: 0x0008, 0x5f1: 0x0040, 0x5f2: 0x0008, 0x5f3: 0x0040, 0x5f4: 0x0040, 0x5f5: 0x0040,
-	0x5f6: 0x0008, 0x5f7: 0x0008, 0x5f8: 0x0008, 0x5f9: 0x0008, 0x5fa: 0x0040, 0x5fb: 0x0040,
-	0x5fc: 0x3308, 0x5fd: 0x0008, 0x5fe: 0x3008, 0x5ff: 0x3008,
-	// Block 0x18, offset 0x600
-	0x600: 0x3008, 0x601: 0x3308, 0x602: 0x3308, 0x603: 0x3308, 0x604: 0x3308, 0x605: 0x0040,
-	0x606: 0x0040, 0x607: 0x3008, 0x608: 0x3008, 0x609: 0x0040, 0x60a: 0x0040, 0x60b: 0x3008,
-	0x60c: 0x3008, 0x60d: 0x3b08, 0x60e: 0x0008, 0x60f: 0x0040, 0x610: 0x0040, 0x611: 0x0040,
-	0x612: 0x0040, 0x613: 0x0040, 0x614: 0x0040, 0x615: 0x0040, 0x616: 0x0040, 0x617: 0x3008,
-	0x618: 0x0040, 0x619: 0x0040, 0x61a: 0x0040, 0x61b: 0x0040, 0x61c: 0x0689, 0x61d: 0x06c1,
-	0x61e: 0x0040, 0x61f: 0x06f9, 0x620: 0x0008, 0x621: 0x0008, 0x622: 0x3308, 0x623: 0x3308,
-	0x624: 0x0040, 0x625: 0x0040, 0x626: 0x0008, 0x627: 0x0008, 0x628: 0x0008, 0x629: 0x0008,
-	0x62a: 0x0008, 0x62b: 0x0008, 0x62c: 0x0008, 0x62d: 0x0008, 0x62e: 0x0008, 0x62f: 0x0008,
-	0x630: 0x0008, 0x631: 0x0008, 0x632: 0x0018, 0x633: 0x0018, 0x634: 0x0018, 0x635: 0x0018,
-	0x636: 0x0018, 0x637: 0x0018, 0x638: 0x0018, 0x639: 0x0018, 0x63a: 0x0018, 0x63b: 0x0018,
-	0x63c: 0x0008, 0x63d: 0x0018, 0x63e: 0x3308, 0x63f: 0x0040,
-	// Block 0x19, offset 0x640
-	0x640: 0x0040, 0x641: 0x3308, 0x642: 0x3308, 0x643: 0x3008, 0x644: 0x0040, 0x645: 0x0008,
-	0x646: 0x0008, 0x647: 0x0008, 0x648: 0x0008, 0x649: 0x0008, 0x64a: 0x0008, 0x64b: 0x0040,
-	0x64c: 0x0040, 0x64d: 0x0040, 0x64e: 0x0040, 0x64f: 0x0008, 0x650: 0x0008, 0x651: 0x0040,
-	0x652: 0x0040, 0x653: 0x0008, 0x654: 0x0008, 0x655: 0x0008, 0x656: 0x0008, 0x657: 0x0008,
-	0x658: 0x0008, 0x659: 0x0008, 0x65a: 0x0008, 0x65b: 0x0008, 0x65c: 0x0008, 0x65d: 0x0008,
-	0x65e: 0x0008, 0x65f: 0x0008, 0x660: 0x0008, 0x661: 0x0008, 0x662: 0x0008, 0x663: 0x0008,
-	0x664: 0x0008, 0x665: 0x0008, 0x666: 0x0008, 0x667: 0x0008, 0x668: 0x0008, 0x669: 0x0040,
-	0x66a: 0x0008, 0x66b: 0x0008, 0x66c: 0x0008, 0x66d: 0x0008, 0x66e: 0x0008, 0x66f: 0x0008,
-	0x670: 0x0008, 0x671: 0x0040, 0x672: 0x0008, 0x673: 0x0731, 0x674: 0x0040, 0x675: 0x0008,
-	0x676: 0x0769, 0x677: 0x0040, 0x678: 0x0008, 0x679: 0x0008, 0x67a: 0x0040, 0x67b: 0x0040,
-	0x67c: 0x3308, 0x67d: 0x0040, 0x67e: 0x3008, 0x67f: 0x3008,
-	// Block 0x1a, offset 0x680
-	0x680: 0x3008, 0x681: 0x3308, 0x682: 0x3308, 0x683: 0x0040, 0x684: 0x0040, 0x685: 0x0040,
-	0x686: 0x0040, 0x687: 0x3308, 0x688: 0x3308, 0x689: 0x0040, 0x68a: 0x0040, 0x68b: 0x3308,
-	0x68c: 0x3308, 0x68d: 0x3b08, 0x68e: 0x0040, 0x68f: 0x0040, 0x690: 0x0040, 0x691: 0x3308,
-	0x692: 0x0040, 0x693: 0x0040, 0x694: 0x0040, 0x695: 0x0040, 0x696: 0x0040, 0x697: 0x0040,
-	0x698: 0x0040, 0x699: 0x07a1, 0x69a: 0x07d9, 0x69b: 0x0811, 0x69c: 0x0008, 0x69d: 0x0040,
-	0x69e: 0x0849, 0x69f: 0x0040, 0x6a0: 0x0040, 0x6a1: 0x0040, 0x6a2: 0x0040, 0x6a3: 0x0040,
-	0x6a4: 0x0040, 0x6a5: 0x0040, 0x6a6: 0x0008, 0x6a7: 0x0008, 0x6a8: 0x0008, 0x6a9: 0x0008,
-	0x6aa: 0x0008, 0x6ab: 0x0008, 0x6ac: 0x0008, 0x6ad: 0x0008, 0x6ae: 0x0008, 0x6af: 0x0008,
-	0x6b0: 0x3308, 0x6b1: 0x3308, 0x6b2: 0x0008, 0x6b3: 0x0008, 0x6b4: 0x0008, 0x6b5: 0x3308,
-	0x6b6: 0x0018, 0x6b7: 0x0040, 0x6b8: 0x0040, 0x6b9: 0x0040, 0x6ba: 0x0040, 0x6bb: 0x0040,
-	0x6bc: 0x0040, 0x6bd: 0x0040, 0x6be: 0x0040, 0x6bf: 0x0040,
-	// Block 0x1b, offset 0x6c0
-	0x6c0: 0x0040, 0x6c1: 0x3308, 0x6c2: 0x3308, 0x6c3: 0x3008, 0x6c4: 0x0040, 0x6c5: 0x0008,
-	0x6c6: 0x0008, 0x6c7: 0x0008, 0x6c8: 0x0008, 0x6c9: 0x0008, 0x6ca: 0x0008, 0x6cb: 0x0008,
-	0x6cc: 0x0008, 0x6cd: 0x0008, 0x6ce: 0x0040, 0x6cf: 0x0008, 0x6d0: 0x0008, 0x6d1: 0x0008,
-	0x6d2: 0x0040, 0x6d3: 0x0008, 0x6d4: 0x0008, 0x6d5: 0x0008, 0x6d6: 0x0008, 0x6d7: 0x0008,
-	0x6d8: 0x0008, 0x6d9: 0x0008, 0x6da: 0x0008, 0x6db: 0x0008, 0x6dc: 0x0008, 0x6dd: 0x0008,
-	0x6de: 0x0008, 0x6df: 0x0008, 0x6e0: 0x0008, 0x6e1: 0x0008, 0x6e2: 0x0008, 0x6e3: 0x0008,
-	0x6e4: 0x0008, 0x6e5: 0x0008, 0x6e6: 0x0008, 0x6e7: 0x0008, 0x6e8: 0x0008, 0x6e9: 0x0040,
-	0x6ea: 0x0008, 0x6eb: 0x0008, 0x6ec: 0x0008, 0x6ed: 0x0008, 0x6ee: 0x0008, 0x6ef: 0x0008,
-	0x6f0: 0x0008, 0x6f1: 0x0040, 0x6f2: 0x0008, 0x6f3: 0x0008, 0x6f4: 0x0040, 0x6f5: 0x0008,
-	0x6f6: 0x0008, 0x6f7: 0x0008, 0x6f8: 0x0008, 0x6f9: 0x0008, 0x6fa: 0x0040, 0x6fb: 0x0040,
-	0x6fc: 0x3308, 0x6fd: 0x0008, 0x6fe: 0x3008, 0x6ff: 0x3008,
-	// Block 0x1c, offset 0x700
-	0x700: 0x3008, 0x701: 0x3308, 0x702: 0x3308, 0x703: 0x3308, 0x704: 0x3308, 0x705: 0x3308,
-	0x706: 0x0040, 0x707: 0x3308, 0x708: 0x3308, 0x709: 0x3008, 0x70a: 0x0040, 0x70b: 0x3008,
-	0x70c: 0x3008, 0x70d: 0x3b08, 0x70e: 0x0040, 0x70f: 0x0040, 0x710: 0x0008, 0x711: 0x0040,
-	0x712: 0x0040, 0x713: 0x0040, 0x714: 0x0040, 0x715: 0x0040, 0x716: 0x0040, 0x717: 0x0040,
-	0x718: 0x0040, 0x719: 0x0040, 0x71a: 0x0040, 0x71b: 0x0040, 0x71c: 0x0040, 0x71d: 0x0040,
-	0x71e: 0x0040, 0x71f: 0x0040, 0x720: 0x0008, 0x721: 0x0008, 0x722: 0x3308, 0x723: 0x3308,
-	0x724: 0x0040, 0x725: 0x0040, 0x726: 0x0008, 0x727: 0x0008, 0x728: 0x0008, 0x729: 0x0008,
-	0x72a: 0x0008, 0x72b: 0x0008, 0x72c: 0x0008, 0x72d: 0x0008, 0x72e: 0x0008, 0x72f: 0x0008,
-	0x730: 0x0018, 0x731: 0x0018, 0x732: 0x0040, 0x733: 0x0040, 0x734: 0x0040, 0x735: 0x0040,
-	0x736: 0x0040, 0x737: 0x0040, 0x738: 0x0040, 0x739: 0x0008, 0x73a: 0x3308, 0x73b: 0x3308,
-	0x73c: 0x3308, 0x73d: 0x3308, 0x73e: 0x3308, 0x73f: 0x3308,
-	// Block 0x1d, offset 0x740
-	0x740: 0x0040, 0x741: 0x3308, 0x742: 0x3008, 0x743: 0x3008, 0x744: 0x0040, 0x745: 0x0008,
-	0x746: 0x0008, 0x747: 0x0008, 0x748: 0x0008, 0x749: 0x0008, 0x74a: 0x0008, 0x74b: 0x0008,
-	0x74c: 0x0008, 0x74d: 0x0040, 0x74e: 0x0040, 0x74f: 0x0008, 0x750: 0x0008, 0x751: 0x0040,
-	0x752: 0x0040, 0x753: 0x0008, 0x754: 0x0008, 0x755: 0x0008, 0x756: 0x0008, 0x757: 0x0008,
-	0x758: 0x0008, 0x759: 0x0008, 0x75a: 0x0008, 0x75b: 0x0008, 0x75c: 0x0008, 0x75d: 0x0008,
-	0x75e: 0x0008, 0x75f: 0x0008, 0x760: 0x0008, 0x761: 0x0008, 0x762: 0x0008, 0x763: 0x0008,
-	0x764: 0x0008, 0x765: 0x0008, 0x766: 0x0008, 0x767: 0x0008, 0x768: 0x0008, 0x769: 0x0040,
-	0x76a: 0x0008, 0x76b: 0x0008, 0x76c: 0x0008, 0x76d: 0x0008, 0x76e: 0x0008, 0x76f: 0x0008,
-	0x770: 0x0008, 0x771: 0x0040, 0x772: 0x0008, 0x773: 0x0008, 0x774: 0x0040, 0x775: 0x0008,
-	0x776: 0x0008, 0x777: 0x0008, 0x778: 0x0008, 0x779: 0x0008, 0x77a: 0x0040, 0x77b: 0x0040,
-	0x77c: 0x3308, 0x77d: 0x0008, 0x77e: 0x3008, 0x77f: 0x3308,
-	// Block 0x1e, offset 0x780
-	0x780: 0x3008, 0x781: 0x3308, 0x782: 0x3308, 0x783: 0x3308, 0x784: 0x3308, 0x785: 0x0040,
-	0x786: 0x0040, 0x787: 0x3008, 0x788: 0x3008, 0x789: 0x0040, 0x78a: 0x0040, 0x78b: 0x3008,
-	0x78c: 0x3008, 0x78d: 0x3b08, 0x78e: 0x0040, 0x78f: 0x0040, 0x790: 0x0040, 0x791: 0x0040,
-	0x792: 0x0040, 0x793: 0x0040, 0x794: 0x0040, 0x795: 0x0040, 0x796: 0x3308, 0x797: 0x3008,
-	0x798: 0x0040, 0x799: 0x0040, 0x79a: 0x0040, 0x79b: 0x0040, 0x79c: 0x0881, 0x79d: 0x08b9,
-	0x79e: 0x0040, 0x79f: 0x0008, 0x7a0: 0x0008, 0x7a1: 0x0008, 0x7a2: 0x3308, 0x7a3: 0x3308,
-	0x7a4: 0x0040, 0x7a5: 0x0040, 0x7a6: 0x0008, 0x7a7: 0x0008, 0x7a8: 0x0008, 0x7a9: 0x0008,
-	0x7aa: 0x0008, 0x7ab: 0x0008, 0x7ac: 0x0008, 0x7ad: 0x0008, 0x7ae: 0x0008, 0x7af: 0x0008,
-	0x7b0: 0x0018, 0x7b1: 0x0008, 0x7b2: 0x0018, 0x7b3: 0x0018, 0x7b4: 0x0018, 0x7b5: 0x0018,
-	0x7b6: 0x0018, 0x7b7: 0x0018, 0x7b8: 0x0040, 0x7b9: 0x0040, 0x7ba: 0x0040, 0x7bb: 0x0040,
-	0x7bc: 0x0040, 0x7bd: 0x0040, 0x7be: 0x0040, 0x7bf: 0x0040,
-	// Block 0x1f, offset 0x7c0
-	0x7c0: 0x0040, 0x7c1: 0x0040, 0x7c2: 0x3308, 0x7c3: 0x0008, 0x7c4: 0x0040, 0x7c5: 0x0008,
-	0x7c6: 0x0008, 0x7c7: 0x0008, 0x7c8: 0x0008, 0x7c9: 0x0008, 0x7ca: 0x0008, 0x7cb: 0x0040,
-	0x7cc: 0x0040, 0x7cd: 0x0040, 0x7ce: 0x0008, 0x7cf: 0x0008, 0x7d0: 0x0008, 0x7d1: 0x0040,
-	0x7d2: 0x0008, 0x7d3: 0x0008, 0x7d4: 0x0008, 0x7d5: 0x0008, 0x7d6: 0x0040, 0x7d7: 0x0040,
-	0x7d8: 0x0040, 0x7d9: 0x0008, 0x7da: 0x0008, 0x7db: 0x0040, 0x7dc: 0x0008, 0x7dd: 0x0040,
-	0x7de: 0x0008, 0x7df: 0x0008, 0x7e0: 0x0040, 0x7e1: 0x0040, 0x7e2: 0x0040, 0x7e3: 0x0008,
-	0x7e4: 0x0008, 0x7e5: 0x0040, 0x7e6: 0x0040, 0x7e7: 0x0040, 0x7e8: 0x0008, 0x7e9: 0x0008,
-	0x7ea: 0x0008, 0x7eb: 0x0040, 0x7ec: 0x0040, 0x7ed: 0x0040, 0x7ee: 0x0008, 0x7ef: 0x0008,
-	0x7f0: 0x0008, 0x7f1: 0x0008, 0x7f2: 0x0008, 0x7f3: 0x0008, 0x7f4: 0x0008, 0x7f5: 0x0008,
-	0x7f6: 0x0008, 0x7f7: 0x0008, 0x7f8: 0x0008, 0x7f9: 0x0008, 0x7fa: 0x0040, 0x7fb: 0x0040,
-	0x7fc: 0x0040, 0x7fd: 0x0040, 0x7fe: 0x3008, 0x7ff: 0x3008,
-	// Block 0x20, offset 0x800
-	0x800: 0x3308, 0x801: 0x3008, 0x802: 0x3008, 0x803: 0x3008, 0x804: 0x3008, 0x805: 0x0040,
-	0x806: 0x3308, 0x807: 0x3308, 0x808: 0x3308, 0x809: 0x0040, 0x80a: 0x3308, 0x80b: 0x3308,
-	0x80c: 0x3308, 0x80d: 0x3b08, 0x80e: 0x0040, 0x80f: 0x0040, 0x810: 0x0040, 0x811: 0x0040,
-	0x812: 0x0040, 0x813: 0x0040, 0x814: 0x0040, 0x815: 0x3308, 0x816: 0x3308, 0x817: 0x0040,
-	0x818: 0x0008, 0x819: 0x0008, 0x81a: 0x0008, 0x81b: 0x0040, 0x81c: 0x0040, 0x81d: 0x0040,
-	0x81e: 0x0040, 0x81f: 0x0040, 0x820: 0x0008, 0x821: 0x0008, 0x822: 0x3308, 0x823: 0x3308,
-	0x824: 0x0040, 0x825: 0x0040, 0x826: 0x0008, 0x827: 0x0008, 0x828: 0x0008, 0x829: 0x0008,
-	0x82a: 0x0008, 0x82b: 0x0008, 0x82c: 0x0008, 0x82d: 0x0008, 0x82e: 0x0008, 0x82f: 0x0008,
-	0x830: 0x0040, 0x831: 0x0040, 0x832: 0x0040, 0x833: 0x0040, 0x834: 0x0040, 0x835: 0x0040,
-	0x836: 0x0040, 0x837: 0x0018, 0x838: 0x0018, 0x839: 0x0018, 0x83a: 0x0018, 0x83b: 0x0018,
-	0x83c: 0x0018, 0x83d: 0x0018, 0x83e: 0x0018, 0x83f: 0x0018,
-	// Block 0x21, offset 0x840
-	0x840: 0x0008, 0x841: 0x3308, 0x842: 0x3008, 0x843: 0x3008, 0x844: 0x0018, 0x845: 0x0008,
-	0x846: 0x0008, 0x847: 0x0008, 0x848: 0x0008, 0x849: 0x0008, 0x84a: 0x0008, 0x84b: 0x0008,
-	0x84c: 0x0008, 0x84d: 0x0040, 0x84e: 0x0008, 0x84f: 0x0008, 0x850: 0x0008, 0x851: 0x0040,
-	0x852: 0x0008, 0x853: 0x0008, 0x854: 0x0008, 0x855: 0x0008, 0x856: 0x0008, 0x857: 0x0008,
-	0x858: 0x0008, 0x859: 0x0008, 0x85a: 0x0008, 0x85b: 0x0008, 0x85c: 0x0008, 0x85d: 0x0008,
-	0x85e: 0x0008, 0x85f: 0x0008, 0x860: 0x0008, 0x861: 0x0008, 0x862: 0x0008, 0x863: 0x0008,
-	0x864: 0x0008, 0x865: 0x0008, 0x866: 0x0008, 0x867: 0x0008, 0x868: 0x0008, 0x869: 0x0040,
-	0x86a: 0x0008, 0x86b: 0x0008, 0x86c: 0x0008, 0x86d: 0x0008, 0x86e: 0x0008, 0x86f: 0x0008,
-	0x870: 0x0008, 0x871: 0x0008, 0x872: 0x0008, 0x873: 0x0008, 0x874: 0x0040, 0x875: 0x0008,
-	0x876: 0x0008, 0x877: 0x0008, 0x878: 0x0008, 0x879: 0x0008, 0x87a: 0x0040, 0x87b: 0x0040,
-	0x87c: 0x3308, 0x87d: 0x0008, 0x87e: 0x3008, 0x87f: 0x3308,
-	// Block 0x22, offset 0x880
-	0x880: 0x3008, 0x881: 0x3008, 0x882: 0x3008, 0x883: 0x3008, 0x884: 0x3008, 0x885: 0x0040,
-	0x886: 0x3308, 0x887: 0x3008, 0x888: 0x3008, 0x889: 0x0040, 0x88a: 0x3008, 0x88b: 0x3008,
-	0x88c: 0x3308, 0x88d: 0x3b08, 0x88e: 0x0040, 0x88f: 0x0040, 0x890: 0x0040, 0x891: 0x0040,
-	0x892: 0x0040, 0x893: 0x0040, 0x894: 0x0040, 0x895: 0x3008, 0x896: 0x3008, 0x897: 0x0040,
-	0x898: 0x0040, 0x899: 0x0040, 0x89a: 0x0040, 0x89b: 0x0040, 0x89c: 0x0040, 0x89d: 0x0040,
-	0x89e: 0x0008, 0x89f: 0x0040, 0x8a0: 0x0008, 0x8a1: 0x0008, 0x8a2: 0x3308, 0x8a3: 0x3308,
-	0x8a4: 0x0040, 0x8a5: 0x0040, 0x8a6: 0x0008, 0x8a7: 0x0008, 0x8a8: 0x0008, 0x8a9: 0x0008,
-	0x8aa: 0x0008, 0x8ab: 0x0008, 0x8ac: 0x0008, 0x8ad: 0x0008, 0x8ae: 0x0008, 0x8af: 0x0008,
-	0x8b0: 0x0040, 0x8b1: 0x0008, 0x8b2: 0x0008, 0x8b3: 0x0040, 0x8b4: 0x0040, 0x8b5: 0x0040,
-	0x8b6: 0x0040, 0x8b7: 0x0040, 0x8b8: 0x0040, 0x8b9: 0x0040, 0x8ba: 0x0040, 0x8bb: 0x0040,
-	0x8bc: 0x0040, 0x8bd: 0x0040, 0x8be: 0x0040, 0x8bf: 0x0040,
-	// Block 0x23, offset 0x8c0
-	0x8c0: 0x3008, 0x8c1: 0x3308, 0x8c2: 0x3308, 0x8c3: 0x3308, 0x8c4: 0x3308, 0x8c5: 0x0040,
-	0x8c6: 0x3008, 0x8c7: 0x3008, 0x8c8: 0x3008, 0x8c9: 0x0040, 0x8ca: 0x3008, 0x8cb: 0x3008,
-	0x8cc: 0x3008, 0x8cd: 0x3b08, 0x8ce: 0x0008, 0x8cf: 0x0018, 0x8d0: 0x0040, 0x8d1: 0x0040,
-	0x8d2: 0x0040, 0x8d3: 0x0040, 0x8d4: 0x0008, 0x8d5: 0x0008, 0x8d6: 0x0008, 0x8d7: 0x3008,
-	0x8d8: 0x0018, 0x8d9: 0x0018, 0x8da: 0x0018, 0x8db: 0x0018, 0x8dc: 0x0018, 0x8dd: 0x0018,
-	0x8de: 0x0018, 0x8df: 0x0008, 0x8e0: 0x0008, 0x8e1: 0x0008, 0x8e2: 0x3308, 0x8e3: 0x3308,
-	0x8e4: 0x0040, 0x8e5: 0x0040, 0x8e6: 0x0008, 0x8e7: 0x0008, 0x8e8: 0x0008, 0x8e9: 0x0008,
-	0x8ea: 0x0008, 0x8eb: 0x0008, 0x8ec: 0x0008, 0x8ed: 0x0008, 0x8ee: 0x0008, 0x8ef: 0x0008,
-	0x8f0: 0x0018, 0x8f1: 0x0018, 0x8f2: 0x0018, 0x8f3: 0x0018, 0x8f4: 0x0018, 0x8f5: 0x0018,
-	0x8f6: 0x0018, 0x8f7: 0x0018, 0x8f8: 0x0018, 0x8f9: 0x0018, 0x8fa: 0x0008, 0x8fb: 0x0008,
-	0x8fc: 0x0008, 0x8fd: 0x0008, 0x8fe: 0x0008, 0x8ff: 0x0008,
-	// Block 0x24, offset 0x900
-	0x900: 0x0040, 0x901: 0x0008, 0x902: 0x0008, 0x903: 0x0040, 0x904: 0x0008, 0x905: 0x0040,
-	0x906: 0x0008, 0x907: 0x0008, 0x908: 0x0008, 0x909: 0x0008, 0x90a: 0x0008, 0x90b: 0x0040,
-	0x90c: 0x0008, 0x90d: 0x0008, 0x90e: 0x0008, 0x90f: 0x0008, 0x910: 0x0008, 0x911: 0x0008,
-	0x912: 0x0008, 0x913: 0x0008, 0x914: 0x0008, 0x915: 0x0008, 0x916: 0x0008, 0x917: 0x0008,
-	0x918: 0x0008, 0x919: 0x0008, 0x91a: 0x0008, 0x91b: 0x0008, 0x91c: 0x0008, 0x91d: 0x0008,
-	0x91e: 0x0008, 0x91f: 0x0008, 0x920: 0x0008, 0x921: 0x0008, 0x922: 0x0008, 0x923: 0x0008,
-	0x924: 0x0040, 0x925: 0x0008, 0x926: 0x0040, 0x927: 0x0008, 0x928: 0x0008, 0x929: 0x0008,
-	0x92a: 0x0008, 0x92b: 0x0008, 0x92c: 0x0008, 0x92d: 0x0008, 0x92e: 0x0008, 0x92f: 0x0008,
-	0x930: 0x0008, 0x931: 0x3308, 0x932: 0x0008, 0x933: 0x0929, 0x934: 0x3308, 0x935: 0x3308,
-	0x936: 0x3308, 0x937: 0x3308, 0x938: 0x3308, 0x939: 0x3308, 0x93a: 0x3b08, 0x93b: 0x3308,
-	0x93c: 0x3308, 0x93d: 0x0008, 0x93e: 0x0040, 0x93f: 0x0040,
-	// Block 0x25, offset 0x940
-	0x940: 0x0008, 0x941: 0x0008, 0x942: 0x0008, 0x943: 0x09d1, 0x944: 0x0008, 0x945: 0x0008,
-	0x946: 0x0008, 0x947: 0x0008, 0x948: 0x0040, 0x949: 0x0008, 0x94a: 0x0008, 0x94b: 0x0008,
-	0x94c: 0x0008, 0x94d: 0x0a09, 0x94e: 0x0008, 0x94f: 0x0008, 0x950: 0x0008, 0x951: 0x0008,
-	0x952: 0x0a41, 0x953: 0x0008, 0x954: 0x0008, 0x955: 0x0008, 0x956: 0x0008, 0x957: 0x0a79,
-	0x958: 0x0008, 0x959: 0x0008, 0x95a: 0x0008, 0x95b: 0x0008, 0x95c: 0x0ab1, 0x95d: 0x0008,
-	0x95e: 0x0008, 0x95f: 0x0008, 0x960: 0x0008, 0x961: 0x0008, 0x962: 0x0008, 0x963: 0x0008,
-	0x964: 0x0008, 0x965: 0x0008, 0x966: 0x0008, 0x967: 0x0008, 0x968: 0x0008, 0x969: 0x0ae9,
-	0x96a: 0x0008, 0x96b: 0x0008, 0x96c: 0x0008, 0x96d: 0x0040, 0x96e: 0x0040, 0x96f: 0x0040,
-	0x970: 0x0040, 0x971: 0x3308, 0x972: 0x3308, 0x973: 0x0b21, 0x974: 0x3308, 0x975: 0x0b59,
-	0x976: 0x0b91, 0x977: 0x0bc9, 0x978: 0x0c19, 0x979: 0x0c51, 0x97a: 0x3308, 0x97b: 0x3308,
-	0x97c: 0x3308, 0x97d: 0x3308, 0x97e: 0x3308, 0x97f: 0x3008,
-	// Block 0x26, offset 0x980
-	0x980: 0x3308, 0x981: 0x0ca1, 0x982: 0x3308, 0x983: 0x3308, 0x984: 0x3b08, 0x985: 0x0018,
-	0x986: 0x3308, 0x987: 0x3308, 0x988: 0x0008, 0x989: 0x0008, 0x98a: 0x0008, 0x98b: 0x0008,
-	0x98c: 0x0008, 0x98d: 0x3308, 0x98e: 0x3308, 0x98f: 0x3308, 0x990: 0x3308, 0x991: 0x3308,
-	0x992: 0x3308, 0x993: 0x0cd9, 0x994: 0x3308, 0x995: 0x3308, 0x996: 0x3308, 0x997: 0x3308,
-	0x998: 0x0040, 0x999: 0x3308, 0x99a: 0x3308, 0x99b: 0x3308, 0x99c: 0x3308, 0x99d: 0x0d11,
-	0x99e: 0x3308, 0x99f: 0x3308, 0x9a0: 0x3308, 0x9a1: 0x3308, 0x9a2: 0x0d49, 0x9a3: 0x3308,
-	0x9a4: 0x3308, 0x9a5: 0x3308, 0x9a6: 0x3308, 0x9a7: 0x0d81, 0x9a8: 0x3308, 0x9a9: 0x3308,
-	0x9aa: 0x3308, 0x9ab: 0x3308, 0x9ac: 0x0db9, 0x9ad: 0x3308, 0x9ae: 0x3308, 0x9af: 0x3308,
-	0x9b0: 0x3308, 0x9b1: 0x3308, 0x9b2: 0x3308, 0x9b3: 0x3308, 0x9b4: 0x3308, 0x9b5: 0x3308,
-	0x9b6: 0x3308, 0x9b7: 0x3308, 0x9b8: 0x3308, 0x9b9: 0x0df1, 0x9ba: 0x3308, 0x9bb: 0x3308,
-	0x9bc: 0x3308, 0x9bd: 0x0040, 0x9be: 0x0018, 0x9bf: 0x0018,
-	// Block 0x27, offset 0x9c0
-	0x9c0: 0x0008, 0x9c1: 0x0008, 0x9c2: 0x0008, 0x9c3: 0x0008, 0x9c4: 0x0008, 0x9c5: 0x0008,
-	0x9c6: 0x0008, 0x9c7: 0x0008, 0x9c8: 0x0008, 0x9c9: 0x0008, 0x9ca: 0x0008, 0x9cb: 0x0008,
-	0x9cc: 0x0008, 0x9cd: 0x0008, 0x9ce: 0x0008, 0x9cf: 0x0008, 0x9d0: 0x0008, 0x9d1: 0x0008,
-	0x9d2: 0x0008, 0x9d3: 0x0008, 0x9d4: 0x0008, 0x9d5: 0x0008, 0x9d6: 0x0008, 0x9d7: 0x0008,
-	0x9d8: 0x0008, 0x9d9: 0x0008, 0x9da: 0x0008, 0x9db: 0x0008, 0x9dc: 0x0008, 0x9dd: 0x0008,
-	0x9de: 0x0008, 0x9df: 0x0008, 0x9e0: 0x0008, 0x9e1: 0x0008, 0x9e2: 0x0008, 0x9e3: 0x0008,
-	0x9e4: 0x0008, 0x9e5: 0x0008, 0x9e6: 0x0008, 0x9e7: 0x0008, 0x9e8: 0x0008, 0x9e9: 0x0008,
-	0x9ea: 0x0008, 0x9eb: 0x0008, 0x9ec: 0x0039, 0x9ed: 0x0ed1, 0x9ee: 0x0ee9, 0x9ef: 0x0008,
-	0x9f0: 0x0ef9, 0x9f1: 0x0f09, 0x9f2: 0x0f19, 0x9f3: 0x0f31, 0x9f4: 0x0249, 0x9f5: 0x0f41,
-	0x9f6: 0x0259, 0x9f7: 0x0f51, 0x9f8: 0x0359, 0x9f9: 0x0f61, 0x9fa: 0x0f71, 0x9fb: 0x0008,
-	0x9fc: 0x00d9, 0x9fd: 0x0f81, 0x9fe: 0x0f99, 0x9ff: 0x0269,
-	// Block 0x28, offset 0xa00
-	0xa00: 0x0fa9, 0xa01: 0x0fb9, 0xa02: 0x0279, 0xa03: 0x0039, 0xa04: 0x0fc9, 0xa05: 0x0fe1,
-	0xa06: 0x05b5, 0xa07: 0x0ee9, 0xa08: 0x0ef9, 0xa09: 0x0f09, 0xa0a: 0x0ff9, 0xa0b: 0x1011,
-	0xa0c: 0x1029, 0xa0d: 0x0f31, 0xa0e: 0x0008, 0xa0f: 0x0f51, 0xa10: 0x0f61, 0xa11: 0x1041,
-	0xa12: 0x00d9, 0xa13: 0x1059, 0xa14: 0x05cd, 0xa15: 0x05cd, 0xa16: 0x0f99, 0xa17: 0x0fa9,
-	0xa18: 0x0fb9, 0xa19: 0x05b5, 0xa1a: 0x1071, 0xa1b: 0x1089, 0xa1c: 0x05e5, 0xa1d: 0x1099,
-	0xa1e: 0x10b1, 0xa1f: 0x10c9, 0xa20: 0x10e1, 0xa21: 0x10f9, 0xa22: 0x0f41, 0xa23: 0x0269,
-	0xa24: 0x0fb9, 0xa25: 0x1089, 0xa26: 0x1099, 0xa27: 0x10b1, 0xa28: 0x1111, 0xa29: 0x10e1,
-	0xa2a: 0x10f9, 0xa2b: 0x0008, 0xa2c: 0x0008, 0xa2d: 0x0008, 0xa2e: 0x0008, 0xa2f: 0x0008,
-	0xa30: 0x0008, 0xa31: 0x0008, 0xa32: 0x0008, 0xa33: 0x0008, 0xa34: 0x0008, 0xa35: 0x0008,
-	0xa36: 0x0008, 0xa37: 0x0008, 0xa38: 0x1129, 0xa39: 0x0008, 0xa3a: 0x0008, 0xa3b: 0x0008,
-	0xa3c: 0x0008, 0xa3d: 0x0008, 0xa3e: 0x0008, 0xa3f: 0x0008,
-	// Block 0x29, offset 0xa40
-	0xa40: 0x0008, 0xa41: 0x0008, 0xa42: 0x0008, 0xa43: 0x0008, 0xa44: 0x0008, 0xa45: 0x0008,
-	0xa46: 0x0008, 0xa47: 0x0008, 0xa48: 0x0008, 0xa49: 0x0008, 0xa4a: 0x0008, 0xa4b: 0x0008,
-	0xa4c: 0x0008, 0xa4d: 0x0008, 0xa4e: 0x0008, 0xa4f: 0x0008, 0xa50: 0x0008, 0xa51: 0x0008,
-	0xa52: 0x0008, 0xa53: 0x0008, 0xa54: 0x0008, 0xa55: 0x0008, 0xa56: 0x0008, 0xa57: 0x0008,
-	0xa58: 0x0008, 0xa59: 0x0008, 0xa5a: 0x0008, 0xa5b: 0x1141, 0xa5c: 0x1159, 0xa5d: 0x1169,
-	0xa5e: 0x1181, 0xa5f: 0x1029, 0xa60: 0x1199, 0xa61: 0x11a9, 0xa62: 0x11c1, 0xa63: 0x11d9,
-	0xa64: 0x11f1, 0xa65: 0x1209, 0xa66: 0x1221, 0xa67: 0x05fd, 0xa68: 0x1239, 0xa69: 0x1251,
-	0xa6a: 0xe17d, 0xa6b: 0x1269, 0xa6c: 0x1281, 0xa6d: 0x1299, 0xa6e: 0x12b1, 0xa6f: 0x12c9,
-	0xa70: 0x12e1, 0xa71: 0x12f9, 0xa72: 0x1311, 0xa73: 0x1329, 0xa74: 0x1341, 0xa75: 0x1359,
-	0xa76: 0x1371, 0xa77: 0x1389, 0xa78: 0x0615, 0xa79: 0x13a1, 0xa7a: 0x13b9, 0xa7b: 0x13d1,
-	0xa7c: 0x13e1, 0xa7d: 0x13f9, 0xa7e: 0x1411, 0xa7f: 0x1429,
-	// Block 0x2a, offset 0xa80
-	0xa80: 0xe00d, 0xa81: 0x0008, 0xa82: 0xe00d, 0xa83: 0x0008, 0xa84: 0xe00d, 0xa85: 0x0008,
-	0xa86: 0xe00d, 0xa87: 0x0008, 0xa88: 0xe00d, 0xa89: 0x0008, 0xa8a: 0xe00d, 0xa8b: 0x0008,
-	0xa8c: 0xe00d, 0xa8d: 0x0008, 0xa8e: 0xe00d, 0xa8f: 0x0008, 0xa90: 0xe00d, 0xa91: 0x0008,
-	0xa92: 0xe00d, 0xa93: 0x0008, 0xa94: 0xe00d, 0xa95: 0x0008, 0xa96: 0xe00d, 0xa97: 0x0008,
-	0xa98: 0xe00d, 0xa99: 0x0008, 0xa9a: 0xe00d, 0xa9b: 0x0008, 0xa9c: 0xe00d, 0xa9d: 0x0008,
-	0xa9e: 0xe00d, 0xa9f: 0x0008, 0xaa0: 0xe00d, 0xaa1: 0x0008, 0xaa2: 0xe00d, 0xaa3: 0x0008,
-	0xaa4: 0xe00d, 0xaa5: 0x0008, 0xaa6: 0xe00d, 0xaa7: 0x0008, 0xaa8: 0xe00d, 0xaa9: 0x0008,
-	0xaaa: 0xe00d, 0xaab: 0x0008, 0xaac: 0xe00d, 0xaad: 0x0008, 0xaae: 0xe00d, 0xaaf: 0x0008,
-	0xab0: 0xe00d, 0xab1: 0x0008, 0xab2: 0xe00d, 0xab3: 0x0008, 0xab4: 0xe00d, 0xab5: 0x0008,
-	0xab6: 0xe00d, 0xab7: 0x0008, 0xab8: 0xe00d, 0xab9: 0x0008, 0xaba: 0xe00d, 0xabb: 0x0008,
-	0xabc: 0xe00d, 0xabd: 0x0008, 0xabe: 0xe00d, 0xabf: 0x0008,
-	// Block 0x2b, offset 0xac0
-	0xac0: 0xe00d, 0xac1: 0x0008, 0xac2: 0xe00d, 0xac3: 0x0008, 0xac4: 0xe00d, 0xac5: 0x0008,
-	0xac6: 0xe00d, 0xac7: 0x0008, 0xac8: 0xe00d, 0xac9: 0x0008, 0xaca: 0xe00d, 0xacb: 0x0008,
-	0xacc: 0xe00d, 0xacd: 0x0008, 0xace: 0xe00d, 0xacf: 0x0008, 0xad0: 0xe00d, 0xad1: 0x0008,
-	0xad2: 0xe00d, 0xad3: 0x0008, 0xad4: 0xe00d, 0xad5: 0x0008, 0xad6: 0x0008, 0xad7: 0x0008,
-	0xad8: 0x0008, 0xad9: 0x0008, 0xada: 0x062d, 0xadb: 0x064d, 0xadc: 0x0008, 0xadd: 0x0008,
-	0xade: 0x1441, 0xadf: 0x0008, 0xae0: 0xe00d, 0xae1: 0x0008, 0xae2: 0xe00d, 0xae3: 0x0008,
-	0xae4: 0xe00d, 0xae5: 0x0008, 0xae6: 0xe00d, 0xae7: 0x0008, 0xae8: 0xe00d, 0xae9: 0x0008,
-	0xaea: 0xe00d, 0xaeb: 0x0008, 0xaec: 0xe00d, 0xaed: 0x0008, 0xaee: 0xe00d, 0xaef: 0x0008,
-	0xaf0: 0xe00d, 0xaf1: 0x0008, 0xaf2: 0xe00d, 0xaf3: 0x0008, 0xaf4: 0xe00d, 0xaf5: 0x0008,
-	0xaf6: 0xe00d, 0xaf7: 0x0008, 0xaf8: 0xe00d, 0xaf9: 0x0008, 0xafa: 0xe00d, 0xafb: 0x0008,
-	0xafc: 0xe00d, 0xafd: 0x0008, 0xafe: 0xe00d, 0xaff: 0x0008,
-	// Block 0x2c, offset 0xb00
-	0xb00: 0x0008, 0xb01: 0x0008, 0xb02: 0x0008, 0xb03: 0x0008, 0xb04: 0x0008, 0xb05: 0x0008,
-	0xb06: 0x0040, 0xb07: 0x0040, 0xb08: 0xe045, 0xb09: 0xe045, 0xb0a: 0xe045, 0xb0b: 0xe045,
-	0xb0c: 0xe045, 0xb0d: 0xe045, 0xb0e: 0x0040, 0xb0f: 0x0040, 0xb10: 0x0008, 0xb11: 0x0008,
-	0xb12: 0x0008, 0xb13: 0x0008, 0xb14: 0x0008, 0xb15: 0x0008, 0xb16: 0x0008, 0xb17: 0x0008,
-	0xb18: 0x0040, 0xb19: 0xe045, 0xb1a: 0x0040, 0xb1b: 0xe045, 0xb1c: 0x0040, 0xb1d: 0xe045,
-	0xb1e: 0x0040, 0xb1f: 0xe045, 0xb20: 0x0008, 0xb21: 0x0008, 0xb22: 0x0008, 0xb23: 0x0008,
-	0xb24: 0x0008, 0xb25: 0x0008, 0xb26: 0x0008, 0xb27: 0x0008, 0xb28: 0xe045, 0xb29: 0xe045,
-	0xb2a: 0xe045, 0xb2b: 0xe045, 0xb2c: 0xe045, 0xb2d: 0xe045, 0xb2e: 0xe045, 0xb2f: 0xe045,
-	0xb30: 0x0008, 0xb31: 0x1459, 0xb32: 0x0008, 0xb33: 0x1471, 0xb34: 0x0008, 0xb35: 0x1489,
-	0xb36: 0x0008, 0xb37: 0x14a1, 0xb38: 0x0008, 0xb39: 0x14b9, 0xb3a: 0x0008, 0xb3b: 0x14d1,
-	0xb3c: 0x0008, 0xb3d: 0x14e9, 0xb3e: 0x0040, 0xb3f: 0x0040,
-	// Block 0x2d, offset 0xb40
-	0xb40: 0x1501, 0xb41: 0x1531, 0xb42: 0x1561, 0xb43: 0x1591, 0xb44: 0x15c1, 0xb45: 0x15f1,
-	0xb46: 0x1621, 0xb47: 0x1651, 0xb48: 0x1501, 0xb49: 0x1531, 0xb4a: 0x1561, 0xb4b: 0x1591,
-	0xb4c: 0x15c1, 0xb4d: 0x15f1, 0xb4e: 0x1621, 0xb4f: 0x1651, 0xb50: 0x1681, 0xb51: 0x16b1,
-	0xb52: 0x16e1, 0xb53: 0x1711, 0xb54: 0x1741, 0xb55: 0x1771, 0xb56: 0x17a1, 0xb57: 0x17d1,
-	0xb58: 0x1681, 0xb59: 0x16b1, 0xb5a: 0x16e1, 0xb5b: 0x1711, 0xb5c: 0x1741, 0xb5d: 0x1771,
-	0xb5e: 0x17a1, 0xb5f: 0x17d1, 0xb60: 0x1801, 0xb61: 0x1831, 0xb62: 0x1861, 0xb63: 0x1891,
-	0xb64: 0x18c1, 0xb65: 0x18f1, 0xb66: 0x1921, 0xb67: 0x1951, 0xb68: 0x1801, 0xb69: 0x1831,
-	0xb6a: 0x1861, 0xb6b: 0x1891, 0xb6c: 0x18c1, 0xb6d: 0x18f1, 0xb6e: 0x1921, 0xb6f: 0x1951,
-	0xb70: 0x0008, 0xb71: 0x0008, 0xb72: 0x1981, 0xb73: 0x19b1, 0xb74: 0x19d9, 0xb75: 0x0040,
-	0xb76: 0x0008, 0xb77: 0x1a01, 0xb78: 0xe045, 0xb79: 0xe045, 0xb7a: 0x0665, 0xb7b: 0x1459,
-	0xb7c: 0x19b1, 0xb7d: 0x067e, 0xb7e: 0x1a31, 0xb7f: 0x069e,
-	// Block 0x2e, offset 0xb80
-	0xb80: 0x06be, 0xb81: 0x1a4a, 0xb82: 0x1a79, 0xb83: 0x1aa9, 0xb84: 0x1ad1, 0xb85: 0x0040,
-	0xb86: 0x0008, 0xb87: 0x1af9, 0xb88: 0x06dd, 0xb89: 0x1471, 0xb8a: 0x06f5, 0xb8b: 0x1489,
-	0xb8c: 0x1aa9, 0xb8d: 0x1b2a, 0xb8e: 0x1b5a, 0xb8f: 0x1b8a, 0xb90: 0x0008, 0xb91: 0x0008,
-	0xb92: 0x0008, 0xb93: 0x1bb9, 0xb94: 0x0040, 0xb95: 0x0040, 0xb96: 0x0008, 0xb97: 0x0008,
-	0xb98: 0xe045, 0xb99: 0xe045, 0xb9a: 0x070d, 0xb9b: 0x14a1, 0xb9c: 0x0040, 0xb9d: 0x1bd2,
-	0xb9e: 0x1c02, 0xb9f: 0x1c32, 0xba0: 0x0008, 0xba1: 0x0008, 0xba2: 0x0008, 0xba3: 0x1c61,
-	0xba4: 0x0008, 0xba5: 0x0008, 0xba6: 0x0008, 0xba7: 0x0008, 0xba8: 0xe045, 0xba9: 0xe045,
-	0xbaa: 0x0725, 0xbab: 0x14d1, 0xbac: 0xe04d, 0xbad: 0x1c7a, 0xbae: 0x03d2, 0xbaf: 0x1caa,
-	0xbb0: 0x0040, 0xbb1: 0x0040, 0xbb2: 0x1cb9, 0xbb3: 0x1ce9, 0xbb4: 0x1d11, 0xbb5: 0x0040,
-	0xbb6: 0x0008, 0xbb7: 0x1d39, 0xbb8: 0x073d, 0xbb9: 0x14b9, 0xbba: 0x0515, 0xbbb: 0x14e9,
-	0xbbc: 0x1ce9, 0xbbd: 0x0756, 0xbbe: 0x0776, 0xbbf: 0x0040,
-	// Block 0x2f, offset 0xbc0
-	0xbc0: 0x000a, 0xbc1: 0x000a, 0xbc2: 0x000a, 0xbc3: 0x000a, 0xbc4: 0x000a, 0xbc5: 0x000a,
-	0xbc6: 0x000a, 0xbc7: 0x000a, 0xbc8: 0x000a, 0xbc9: 0x000a, 0xbca: 0x000a, 0xbcb: 0x03c0,
-	0xbcc: 0x0003, 0xbcd: 0x0003, 0xbce: 0x0340, 0xbcf: 0x0b40, 0xbd0: 0x0018, 0xbd1: 0xe00d,
-	0xbd2: 0x0018, 0xbd3: 0x0018, 0xbd4: 0x0018, 0xbd5: 0x0018, 0xbd6: 0x0018, 0xbd7: 0x0796,
-	0xbd8: 0x0018, 0xbd9: 0x0018, 0xbda: 0x0018, 0xbdb: 0x0018, 0xbdc: 0x0018, 0xbdd: 0x0018,
-	0xbde: 0x0018, 0xbdf: 0x0018, 0xbe0: 0x0018, 0xbe1: 0x0018, 0xbe2: 0x0018, 0xbe3: 0x0018,
-	0xbe4: 0x0040, 0xbe5: 0x0040, 0xbe6: 0x0040, 0xbe7: 0x0018, 0xbe8: 0x0040, 0xbe9: 0x0040,
-	0xbea: 0x0340, 0xbeb: 0x0340, 0xbec: 0x0340, 0xbed: 0x0340, 0xbee: 0x0340, 0xbef: 0x000a,
-	0xbf0: 0x0018, 0xbf1: 0x0018, 0xbf2: 0x0018, 0xbf3: 0x1d69, 0xbf4: 0x1da1, 0xbf5: 0x0018,
-	0xbf6: 0x1df1, 0xbf7: 0x1e29, 0xbf8: 0x0018, 0xbf9: 0x0018, 0xbfa: 0x0018, 0xbfb: 0x0018,
-	0xbfc: 0x1e7a, 0xbfd: 0x0018, 0xbfe: 0x07b6, 0xbff: 0x0018,
-	// Block 0x30, offset 0xc00
-	0xc00: 0x0018, 0xc01: 0x0018, 0xc02: 0x0018, 0xc03: 0x0018, 0xc04: 0x0018, 0xc05: 0x0018,
-	0xc06: 0x0018, 0xc07: 0x1e92, 0xc08: 0x1eaa, 0xc09: 0x1ec2, 0xc0a: 0x0018, 0xc0b: 0x0018,
-	0xc0c: 0x0018, 0xc0d: 0x0018, 0xc0e: 0x0018, 0xc0f: 0x0018, 0xc10: 0x0018, 0xc11: 0x0018,
-	0xc12: 0x0018, 0xc13: 0x0018, 0xc14: 0x0018, 0xc15: 0x0018, 0xc16: 0x0018, 0xc17: 0x1ed9,
-	0xc18: 0x0018, 0xc19: 0x0018, 0xc1a: 0x0018, 0xc1b: 0x0018, 0xc1c: 0x0018, 0xc1d: 0x0018,
-	0xc1e: 0x0018, 0xc1f: 0x000a, 0xc20: 0x03c0, 0xc21: 0x0340, 0xc22: 0x0340, 0xc23: 0x0340,
-	0xc24: 0x03c0, 0xc25: 0x0040, 0xc26: 0x0040, 0xc27: 0x0040, 0xc28: 0x0040, 0xc29: 0x0040,
-	0xc2a: 0x0340, 0xc2b: 0x0340, 0xc2c: 0x0340, 0xc2d: 0x0340, 0xc2e: 0x0340, 0xc2f: 0x0340,
-	0xc30: 0x1f41, 0xc31: 0x0f41, 0xc32: 0x0040, 0xc33: 0x0040, 0xc34: 0x1f51, 0xc35: 0x1f61,
-	0xc36: 0x1f71, 0xc37: 0x1f81, 0xc38: 0x1f91, 0xc39: 0x1fa1, 0xc3a: 0x1fb2, 0xc3b: 0x07d5,
-	0xc3c: 0x1fc2, 0xc3d: 0x1fd2, 0xc3e: 0x1fe2, 0xc3f: 0x0f71,
-	// Block 0x31, offset 0xc40
-	0xc40: 0x1f41, 0xc41: 0x00c9, 0xc42: 0x0069, 0xc43: 0x0079, 0xc44: 0x1f51, 0xc45: 0x1f61,
-	0xc46: 0x1f71, 0xc47: 0x1f81, 0xc48: 0x1f91, 0xc49: 0x1fa1, 0xc4a: 0x1fb2, 0xc4b: 0x07ed,
-	0xc4c: 0x1fc2, 0xc4d: 0x1fd2, 0xc4e: 0x1fe2, 0xc4f: 0x0040, 0xc50: 0x0039, 0xc51: 0x0f09,
-	0xc52: 0x00d9, 0xc53: 0x0369, 0xc54: 0x0ff9, 0xc55: 0x0249, 0xc56: 0x0f51, 0xc57: 0x0359,
-	0xc58: 0x0f61, 0xc59: 0x0f71, 0xc5a: 0x0f99, 0xc5b: 0x01d9, 0xc5c: 0x0fa9, 0xc5d: 0x0040,
-	0xc5e: 0x0040, 0xc5f: 0x0040, 0xc60: 0x0018, 0xc61: 0x0018, 0xc62: 0x0018, 0xc63: 0x0018,
-	0xc64: 0x0018, 0xc65: 0x0018, 0xc66: 0x0018, 0xc67: 0x0018, 0xc68: 0x1ff1, 0xc69: 0x0018,
-	0xc6a: 0x0018, 0xc6b: 0x0018, 0xc6c: 0x0018, 0xc6d: 0x0018, 0xc6e: 0x0018, 0xc6f: 0x0018,
-	0xc70: 0x0018, 0xc71: 0x0018, 0xc72: 0x0018, 0xc73: 0x0018, 0xc74: 0x0018, 0xc75: 0x0018,
-	0xc76: 0x0018, 0xc77: 0x0018, 0xc78: 0x0018, 0xc79: 0x0018, 0xc7a: 0x0018, 0xc7b: 0x0018,
-	0xc7c: 0x0018, 0xc7d: 0x0018, 0xc7e: 0x0018, 0xc7f: 0x0018,
-	// Block 0x32, offset 0xc80
-	0xc80: 0x0806, 0xc81: 0x0826, 0xc82: 0x1159, 0xc83: 0x0845, 0xc84: 0x0018, 0xc85: 0x0866,
-	0xc86: 0x0886, 0xc87: 0x1011, 0xc88: 0x0018, 0xc89: 0x08a5, 0xc8a: 0x0f31, 0xc8b: 0x0249,
-	0xc8c: 0x0249, 0xc8d: 0x0249, 0xc8e: 0x0249, 0xc8f: 0x2009, 0xc90: 0x0f41, 0xc91: 0x0f41,
-	0xc92: 0x0359, 0xc93: 0x0359, 0xc94: 0x0018, 0xc95: 0x0f71, 0xc96: 0x2021, 0xc97: 0x0018,
-	0xc98: 0x0018, 0xc99: 0x0f99, 0xc9a: 0x2039, 0xc9b: 0x0269, 0xc9c: 0x0269, 0xc9d: 0x0269,
-	0xc9e: 0x0018, 0xc9f: 0x0018, 0xca0: 0x2049, 0xca1: 0x08c5, 0xca2: 0x2061, 0xca3: 0x0018,
-	0xca4: 0x13d1, 0xca5: 0x0018, 0xca6: 0x2079, 0xca7: 0x0018, 0xca8: 0x13d1, 0xca9: 0x0018,
-	0xcaa: 0x0f51, 0xcab: 0x2091, 0xcac: 0x0ee9, 0xcad: 0x1159, 0xcae: 0x0018, 0xcaf: 0x0f09,
-	0xcb0: 0x0f09, 0xcb1: 0x1199, 0xcb2: 0x0040, 0xcb3: 0x0f61, 0xcb4: 0x00d9, 0xcb5: 0x20a9,
-	0xcb6: 0x20c1, 0xcb7: 0x20d9, 0xcb8: 0x20f1, 0xcb9: 0x0f41, 0xcba: 0x0018, 0xcbb: 0x08e5,
-	0xcbc: 0x2109, 0xcbd: 0x10b1, 0xcbe: 0x10b1, 0xcbf: 0x2109,
-	// Block 0x33, offset 0xcc0
-	0xcc0: 0x0905, 0xcc1: 0x0018, 0xcc2: 0x0018, 0xcc3: 0x0018, 0xcc4: 0x0018, 0xcc5: 0x0ef9,
-	0xcc6: 0x0ef9, 0xcc7: 0x0f09, 0xcc8: 0x0f41, 0xcc9: 0x0259, 0xcca: 0x0018, 0xccb: 0x0018,
-	0xccc: 0x0018, 0xccd: 0x0018, 0xcce: 0x0008, 0xccf: 0x0018, 0xcd0: 0x2121, 0xcd1: 0x2151,
-	0xcd2: 0x2181, 0xcd3: 0x21b9, 0xcd4: 0x21e9, 0xcd5: 0x2219, 0xcd6: 0x2249, 0xcd7: 0x2279,
-	0xcd8: 0x22a9, 0xcd9: 0x22d9, 0xcda: 0x2309, 0xcdb: 0x2339, 0xcdc: 0x2369, 0xcdd: 0x2399,
-	0xcde: 0x23c9, 0xcdf: 0x23f9, 0xce0: 0x0f41, 0xce1: 0x2421, 0xce2: 0x091d, 0xce3: 0x2439,
-	0xce4: 0x1089, 0xce5: 0x2451, 0xce6: 0x093d, 0xce7: 0x2469, 0xce8: 0x2491, 0xce9: 0x0369,
-	0xcea: 0x24a9, 0xceb: 0x095d, 0xcec: 0x0359, 0xced: 0x1159, 0xcee: 0x0ef9, 0xcef: 0x0f61,
-	0xcf0: 0x0f41, 0xcf1: 0x2421, 0xcf2: 0x097d, 0xcf3: 0x2439, 0xcf4: 0x1089, 0xcf5: 0x2451,
-	0xcf6: 0x099d, 0xcf7: 0x2469, 0xcf8: 0x2491, 0xcf9: 0x0369, 0xcfa: 0x24a9, 0xcfb: 0x09bd,
-	0xcfc: 0x0359, 0xcfd: 0x1159, 0xcfe: 0x0ef9, 0xcff: 0x0f61,
-	// Block 0x34, offset 0xd00
-	0xd00: 0x0018, 0xd01: 0x0018, 0xd02: 0x0018, 0xd03: 0x0018, 0xd04: 0x0018, 0xd05: 0x0018,
-	0xd06: 0x0018, 0xd07: 0x0018, 0xd08: 0x0018, 0xd09: 0x0018, 0xd0a: 0x0018, 0xd0b: 0x0040,
-	0xd0c: 0x0040, 0xd0d: 0x0040, 0xd0e: 0x0040, 0xd0f: 0x0040, 0xd10: 0x0040, 0xd11: 0x0040,
-	0xd12: 0x0040, 0xd13: 0x0040, 0xd14: 0x0040, 0xd15: 0x0040, 0xd16: 0x0040, 0xd17: 0x0040,
-	0xd18: 0x0040, 0xd19: 0x0040, 0xd1a: 0x0040, 0xd1b: 0x0040, 0xd1c: 0x0040, 0xd1d: 0x0040,
-	0xd1e: 0x0040, 0xd1f: 0x0040, 0xd20: 0x00c9, 0xd21: 0x0069, 0xd22: 0x0079, 0xd23: 0x1f51,
-	0xd24: 0x1f61, 0xd25: 0x1f71, 0xd26: 0x1f81, 0xd27: 0x1f91, 0xd28: 0x1fa1, 0xd29: 0x2601,
-	0xd2a: 0x2619, 0xd2b: 0x2631, 0xd2c: 0x2649, 0xd2d: 0x2661, 0xd2e: 0x2679, 0xd2f: 0x2691,
-	0xd30: 0x26a9, 0xd31: 0x26c1, 0xd32: 0x26d9, 0xd33: 0x26f1, 0xd34: 0x0a1e, 0xd35: 0x0a3e,
-	0xd36: 0x0a5e, 0xd37: 0x0a7e, 0xd38: 0x0a9e, 0xd39: 0x0abe, 0xd3a: 0x0ade, 0xd3b: 0x0afe,
-	0xd3c: 0x0b1e, 0xd3d: 0x270a, 0xd3e: 0x2732, 0xd3f: 0x275a,
-	// Block 0x35, offset 0xd40
-	0xd40: 0x2782, 0xd41: 0x27aa, 0xd42: 0x27d2, 0xd43: 0x27fa, 0xd44: 0x2822, 0xd45: 0x284a,
-	0xd46: 0x2872, 0xd47: 0x289a, 0xd48: 0x0040, 0xd49: 0x0040, 0xd4a: 0x0040, 0xd4b: 0x0040,
-	0xd4c: 0x0040, 0xd4d: 0x0040, 0xd4e: 0x0040, 0xd4f: 0x0040, 0xd50: 0x0040, 0xd51: 0x0040,
-	0xd52: 0x0040, 0xd53: 0x0040, 0xd54: 0x0040, 0xd55: 0x0040, 0xd56: 0x0040, 0xd57: 0x0040,
-	0xd58: 0x0040, 0xd59: 0x0040, 0xd5a: 0x0040, 0xd5b: 0x0040, 0xd5c: 0x0b3e, 0xd5d: 0x0b5e,
-	0xd5e: 0x0b7e, 0xd5f: 0x0b9e, 0xd60: 0x0bbe, 0xd61: 0x0bde, 0xd62: 0x0bfe, 0xd63: 0x0c1e,
-	0xd64: 0x0c3e, 0xd65: 0x0c5e, 0xd66: 0x0c7e, 0xd67: 0x0c9e, 0xd68: 0x0cbe, 0xd69: 0x0cde,
-	0xd6a: 0x0cfe, 0xd6b: 0x0d1e, 0xd6c: 0x0d3e, 0xd6d: 0x0d5e, 0xd6e: 0x0d7e, 0xd6f: 0x0d9e,
-	0xd70: 0x0dbe, 0xd71: 0x0dde, 0xd72: 0x0dfe, 0xd73: 0x0e1e, 0xd74: 0x0e3e, 0xd75: 0x0e5e,
-	0xd76: 0x0039, 0xd77: 0x0ee9, 0xd78: 0x1159, 0xd79: 0x0ef9, 0xd7a: 0x0f09, 0xd7b: 0x1199,
-	0xd7c: 0x0f31, 0xd7d: 0x0249, 0xd7e: 0x0f41, 0xd7f: 0x0259,
-	// Block 0x36, offset 0xd80
-	0xd80: 0x0f51, 0xd81: 0x0359, 0xd82: 0x0f61, 0xd83: 0x0f71, 0xd84: 0x00d9, 0xd85: 0x0f99,
-	0xd86: 0x2039, 0xd87: 0x0269, 0xd88: 0x01d9, 0xd89: 0x0fa9, 0xd8a: 0x0fb9, 0xd8b: 0x1089,
-	0xd8c: 0x0279, 0xd8d: 0x0369, 0xd8e: 0x0289, 0xd8f: 0x13d1, 0xd90: 0x0039, 0xd91: 0x0ee9,
-	0xd92: 0x1159, 0xd93: 0x0ef9, 0xd94: 0x0f09, 0xd95: 0x1199, 0xd96: 0x0f31, 0xd97: 0x0249,
-	0xd98: 0x0f41, 0xd99: 0x0259, 0xd9a: 0x0f51, 0xd9b: 0x0359, 0xd9c: 0x0f61, 0xd9d: 0x0f71,
-	0xd9e: 0x00d9, 0xd9f: 0x0f99, 0xda0: 0x2039, 0xda1: 0x0269, 0xda2: 0x01d9, 0xda3: 0x0fa9,
-	0xda4: 0x0fb9, 0xda5: 0x1089, 0xda6: 0x0279, 0xda7: 0x0369, 0xda8: 0x0289, 0xda9: 0x13d1,
-	0xdaa: 0x1f41, 0xdab: 0x0018, 0xdac: 0x0018, 0xdad: 0x0018, 0xdae: 0x0018, 0xdaf: 0x0018,
-	0xdb0: 0x0018, 0xdb1: 0x0018, 0xdb2: 0x0018, 0xdb3: 0x0018, 0xdb4: 0x0018, 0xdb5: 0x0018,
-	0xdb6: 0x0018, 0xdb7: 0x0018, 0xdb8: 0x0018, 0xdb9: 0x0018, 0xdba: 0x0018, 0xdbb: 0x0018,
-	0xdbc: 0x0018, 0xdbd: 0x0018, 0xdbe: 0x0018, 0xdbf: 0x0018,
-	// Block 0x37, offset 0xdc0
-	0xdc0: 0x0008, 0xdc1: 0x0008, 0xdc2: 0x0008, 0xdc3: 0x0008, 0xdc4: 0x0008, 0xdc5: 0x0008,
-	0xdc6: 0x0008, 0xdc7: 0x0008, 0xdc8: 0x0008, 0xdc9: 0x0008, 0xdca: 0x0008, 0xdcb: 0x0008,
-	0xdcc: 0x0008, 0xdcd: 0x0008, 0xdce: 0x0008, 0xdcf: 0x0008, 0xdd0: 0x0008, 0xdd1: 0x0008,
-	0xdd2: 0x0008, 0xdd3: 0x0008, 0xdd4: 0x0008, 0xdd5: 0x0008, 0xdd6: 0x0008, 0xdd7: 0x0008,
-	0xdd8: 0x0008, 0xdd9: 0x0008, 0xdda: 0x0008, 0xddb: 0x0008, 0xddc: 0x0008, 0xddd: 0x0008,
-	0xdde: 0x0008, 0xddf: 0x0040, 0xde0: 0xe00d, 0xde1: 0x0008, 0xde2: 0x2971, 0xde3: 0x0ed5,
-	0xde4: 0x2989, 0xde5: 0x0008, 0xde6: 0x0008, 0xde7: 0xe07d, 0xde8: 0x0008, 0xde9: 0xe01d,
-	0xdea: 0x0008, 0xdeb: 0xe03d, 0xdec: 0x0008, 0xded: 0x0fe1, 0xdee: 0x1281, 0xdef: 0x0fc9,
-	0xdf0: 0x1141, 0xdf1: 0x0008, 0xdf2: 0xe00d, 0xdf3: 0x0008, 0xdf4: 0x0008, 0xdf5: 0xe01d,
-	0xdf6: 0x0008, 0xdf7: 0x0008, 0xdf8: 0x0008, 0xdf9: 0x0008, 0xdfa: 0x0008, 0xdfb: 0x0008,
-	0xdfc: 0x0259, 0xdfd: 0x1089, 0xdfe: 0x29a1, 0xdff: 0x29b9,
-	// Block 0x38, offset 0xe00
-	0xe00: 0xe00d, 0xe01: 0x0008, 0xe02: 0xe00d, 0xe03: 0x0008, 0xe04: 0xe00d, 0xe05: 0x0008,
-	0xe06: 0xe00d, 0xe07: 0x0008, 0xe08: 0xe00d, 0xe09: 0x0008, 0xe0a: 0xe00d, 0xe0b: 0x0008,
-	0xe0c: 0xe00d, 0xe0d: 0x0008, 0xe0e: 0xe00d, 0xe0f: 0x0008, 0xe10: 0xe00d, 0xe11: 0x0008,
-	0xe12: 0xe00d, 0xe13: 0x0008, 0xe14: 0xe00d, 0xe15: 0x0008, 0xe16: 0xe00d, 0xe17: 0x0008,
-	0xe18: 0xe00d, 0xe19: 0x0008, 0xe1a: 0xe00d, 0xe1b: 0x0008, 0xe1c: 0xe00d, 0xe1d: 0x0008,
-	0xe1e: 0xe00d, 0xe1f: 0x0008, 0xe20: 0xe00d, 0xe21: 0x0008, 0xe22: 0xe00d, 0xe23: 0x0008,
-	0xe24: 0x0008, 0xe25: 0x0018, 0xe26: 0x0018, 0xe27: 0x0018, 0xe28: 0x0018, 0xe29: 0x0018,
-	0xe2a: 0x0018, 0xe2b: 0xe03d, 0xe2c: 0x0008, 0xe2d: 0xe01d, 0xe2e: 0x0008, 0xe2f: 0x3308,
-	0xe30: 0x3308, 0xe31: 0x3308, 0xe32: 0xe00d, 0xe33: 0x0008, 0xe34: 0x0040, 0xe35: 0x0040,
-	0xe36: 0x0040, 0xe37: 0x0040, 0xe38: 0x0040, 0xe39: 0x0018, 0xe3a: 0x0018, 0xe3b: 0x0018,
-	0xe3c: 0x0018, 0xe3d: 0x0018, 0xe3e: 0x0018, 0xe3f: 0x0018,
-	// Block 0x39, offset 0xe40
-	0xe40: 0x2715, 0xe41: 0x2735, 0xe42: 0x2755, 0xe43: 0x2775, 0xe44: 0x2795, 0xe45: 0x27b5,
-	0xe46: 0x27d5, 0xe47: 0x27f5, 0xe48: 0x2815, 0xe49: 0x2835, 0xe4a: 0x2855, 0xe4b: 0x2875,
-	0xe4c: 0x2895, 0xe4d: 0x28b5, 0xe4e: 0x28d5, 0xe4f: 0x28f5, 0xe50: 0x2915, 0xe51: 0x2935,
-	0xe52: 0x2955, 0xe53: 0x2975, 0xe54: 0x2995, 0xe55: 0x29b5, 0xe56: 0x0040, 0xe57: 0x0040,
-	0xe58: 0x0040, 0xe59: 0x0040, 0xe5a: 0x0040, 0xe5b: 0x0040, 0xe5c: 0x0040, 0xe5d: 0x0040,
-	0xe5e: 0x0040, 0xe5f: 0x0040, 0xe60: 0x0040, 0xe61: 0x0040, 0xe62: 0x0040, 0xe63: 0x0040,
-	0xe64: 0x0040, 0xe65: 0x0040, 0xe66: 0x0040, 0xe67: 0x0040, 0xe68: 0x0040, 0xe69: 0x0040,
-	0xe6a: 0x0040, 0xe6b: 0x0040, 0xe6c: 0x0040, 0xe6d: 0x0040, 0xe6e: 0x0040, 0xe6f: 0x0040,
-	0xe70: 0x0040, 0xe71: 0x0040, 0xe72: 0x0040, 0xe73: 0x0040, 0xe74: 0x0040, 0xe75: 0x0040,
-	0xe76: 0x0040, 0xe77: 0x0040, 0xe78: 0x0040, 0xe79: 0x0040, 0xe7a: 0x0040, 0xe7b: 0x0040,
-	0xe7c: 0x0040, 0xe7d: 0x0040, 0xe7e: 0x0040, 0xe7f: 0x0040,
-	// Block 0x3a, offset 0xe80
-	0xe80: 0x000a, 0xe81: 0x0018, 0xe82: 0x29d1, 0xe83: 0x0018, 0xe84: 0x0018, 0xe85: 0x0008,
-	0xe86: 0x0008, 0xe87: 0x0008, 0xe88: 0x0018, 0xe89: 0x0018, 0xe8a: 0x0018, 0xe8b: 0x0018,
-	0xe8c: 0x0018, 0xe8d: 0x0018, 0xe8e: 0x0018, 0xe8f: 0x0018, 0xe90: 0x0018, 0xe91: 0x0018,
-	0xe92: 0x0018, 0xe93: 0x0018, 0xe94: 0x0018, 0xe95: 0x0018, 0xe96: 0x0018, 0xe97: 0x0018,
-	0xe98: 0x0018, 0xe99: 0x0018, 0xe9a: 0x0018, 0xe9b: 0x0018, 0xe9c: 0x0018, 0xe9d: 0x0018,
-	0xe9e: 0x0018, 0xe9f: 0x0018, 0xea0: 0x0018, 0xea1: 0x0018, 0xea2: 0x0018, 0xea3: 0x0018,
-	0xea4: 0x0018, 0xea5: 0x0018, 0xea6: 0x0018, 0xea7: 0x0018, 0xea8: 0x0018, 0xea9: 0x0018,
-	0xeaa: 0x3308, 0xeab: 0x3308, 0xeac: 0x3308, 0xead: 0x3308, 0xeae: 0x3018, 0xeaf: 0x3018,
-	0xeb0: 0x0018, 0xeb1: 0x0018, 0xeb2: 0x0018, 0xeb3: 0x0018, 0xeb4: 0x0018, 0xeb5: 0x0018,
-	0xeb6: 0xe125, 0xeb7: 0x0018, 0xeb8: 0x29d5, 0xeb9: 0x29f5, 0xeba: 0x2a15, 0xebb: 0x0018,
-	0xebc: 0x0008, 0xebd: 0x0018, 0xebe: 0x0018, 0xebf: 0x0018,
-	// Block 0x3b, offset 0xec0
-	0xec0: 0x2b55, 0xec1: 0x2b75, 0xec2: 0x2b95, 0xec3: 0x2bb5, 0xec4: 0x2bd5, 0xec5: 0x2bf5,
-	0xec6: 0x2bf5, 0xec7: 0x2bf5, 0xec8: 0x2c15, 0xec9: 0x2c15, 0xeca: 0x2c15, 0xecb: 0x2c15,
-	0xecc: 0x2c35, 0xecd: 0x2c35, 0xece: 0x2c35, 0xecf: 0x2c55, 0xed0: 0x2c75, 0xed1: 0x2c75,
-	0xed2: 0x2a95, 0xed3: 0x2a95, 0xed4: 0x2c75, 0xed5: 0x2c75, 0xed6: 0x2c95, 0xed7: 0x2c95,
-	0xed8: 0x2c75, 0xed9: 0x2c75, 0xeda: 0x2a95, 0xedb: 0x2a95, 0xedc: 0x2c75, 0xedd: 0x2c75,
-	0xede: 0x2c55, 0xedf: 0x2c55, 0xee0: 0x2cb5, 0xee1: 0x2cb5, 0xee2: 0x2cd5, 0xee3: 0x2cd5,
-	0xee4: 0x0040, 0xee5: 0x2cf5, 0xee6: 0x2d15, 0xee7: 0x2d35, 0xee8: 0x2d35, 0xee9: 0x2d55,
-	0xeea: 0x2d75, 0xeeb: 0x2d95, 0xeec: 0x2db5, 0xeed: 0x2dd5, 0xeee: 0x2df5, 0xeef: 0x2e15,
-	0xef0: 0x2e35, 0xef1: 0x2e55, 0xef2: 0x2e55, 0xef3: 0x2e75, 0xef4: 0x2e95, 0xef5: 0x2e95,
-	0xef6: 0x2eb5, 0xef7: 0x2ed5, 0xef8: 0x2e75, 0xef9: 0x2ef5, 0xefa: 0x2f15, 0xefb: 0x2ef5,
-	0xefc: 0x2e75, 0xefd: 0x2f35, 0xefe: 0x2f55, 0xeff: 0x2f75,
-	// Block 0x3c, offset 0xf00
-	0xf00: 0x2f95, 0xf01: 0x2fb5, 0xf02: 0x2d15, 0xf03: 0x2cf5, 0xf04: 0x2fd5, 0xf05: 0x2ff5,
-	0xf06: 0x3015, 0xf07: 0x3035, 0xf08: 0x3055, 0xf09: 0x3075, 0xf0a: 0x3095, 0xf0b: 0x30b5,
-	0xf0c: 0x30d5, 0xf0d: 0x30f5, 0xf0e: 0x3115, 0xf0f: 0x0040, 0xf10: 0x0018, 0xf11: 0x0018,
-	0xf12: 0x3135, 0xf13: 0x3155, 0xf14: 0x3175, 0xf15: 0x3195, 0xf16: 0x31b5, 0xf17: 0x31d5,
-	0xf18: 0x31f5, 0xf19: 0x3215, 0xf1a: 0x3235, 0xf1b: 0x3255, 0xf1c: 0x3175, 0xf1d: 0x3275,
-	0xf1e: 0x3295, 0xf1f: 0x32b5, 0xf20: 0x0008, 0xf21: 0x0008, 0xf22: 0x0008, 0xf23: 0x0008,
-	0xf24: 0x0008, 0xf25: 0x0008, 0xf26: 0x0008, 0xf27: 0x0008, 0xf28: 0x0008, 0xf29: 0x0008,
-	0xf2a: 0x0008, 0xf2b: 0x0008, 0xf2c: 0x0008, 0xf2d: 0x0008, 0xf2e: 0x0008, 0xf2f: 0x0008,
-	0xf30: 0x0008, 0xf31: 0x0008, 0xf32: 0x0008, 0xf33: 0x0008, 0xf34: 0x0008, 0xf35: 0x0008,
-	0xf36: 0x0008, 0xf37: 0x0008, 0xf38: 0x0008, 0xf39: 0x0008, 0xf3a: 0x0008, 0xf3b: 0x0040,
-	0xf3c: 0x0040, 0xf3d: 0x0040, 0xf3e: 0x0040, 0xf3f: 0x0040,
-	// Block 0x3d, offset 0xf40
-	0xf40: 0x36a2, 0xf41: 0x36d2, 0xf42: 0x3702, 0xf43: 0x3732, 0xf44: 0x32d5, 0xf45: 0x32f5,
-	0xf46: 0x3315, 0xf47: 0x3335, 0xf48: 0x0018, 0xf49: 0x0018, 0xf4a: 0x0018, 0xf4b: 0x0018,
-	0xf4c: 0x0018, 0xf4d: 0x0018, 0xf4e: 0x0018, 0xf4f: 0x0018, 0xf50: 0x3355, 0xf51: 0x3761,
-	0xf52: 0x3779, 0xf53: 0x3791, 0xf54: 0x37a9, 0xf55: 0x37c1, 0xf56: 0x37d9, 0xf57: 0x37f1,
-	0xf58: 0x3809, 0xf59: 0x3821, 0xf5a: 0x3839, 0xf5b: 0x3851, 0xf5c: 0x3869, 0xf5d: 0x3881,
-	0xf5e: 0x3899, 0xf5f: 0x38b1, 0xf60: 0x3375, 0xf61: 0x3395, 0xf62: 0x33b5, 0xf63: 0x33d5,
-	0xf64: 0x33f5, 0xf65: 0x33f5, 0xf66: 0x3415, 0xf67: 0x3435, 0xf68: 0x3455, 0xf69: 0x3475,
-	0xf6a: 0x3495, 0xf6b: 0x34b5, 0xf6c: 0x34d5, 0xf6d: 0x34f5, 0xf6e: 0x3515, 0xf6f: 0x3535,
-	0xf70: 0x3555, 0xf71: 0x3575, 0xf72: 0x3595, 0xf73: 0x35b5, 0xf74: 0x35d5, 0xf75: 0x35f5,
-	0xf76: 0x3615, 0xf77: 0x3635, 0xf78: 0x3655, 0xf79: 0x3675, 0xf7a: 0x3695, 0xf7b: 0x36b5,
-	0xf7c: 0x38c9, 0xf7d: 0x3901, 0xf7e: 0x36d5, 0xf7f: 0x0018,
-	// Block 0x3e, offset 0xf80
-	0xf80: 0x36f5, 0xf81: 0x3715, 0xf82: 0x3735, 0xf83: 0x3755, 0xf84: 0x3775, 0xf85: 0x3795,
-	0xf86: 0x37b5, 0xf87: 0x37d5, 0xf88: 0x37f5, 0xf89: 0x3815, 0xf8a: 0x3835, 0xf8b: 0x3855,
-	0xf8c: 0x3875, 0xf8d: 0x3895, 0xf8e: 0x38b5, 0xf8f: 0x38d5, 0xf90: 0x38f5, 0xf91: 0x3915,
-	0xf92: 0x3935, 0xf93: 0x3955, 0xf94: 0x3975, 0xf95: 0x3995, 0xf96: 0x39b5, 0xf97: 0x39d5,
-	0xf98: 0x39f5, 0xf99: 0x3a15, 0xf9a: 0x3a35, 0xf9b: 0x3a55, 0xf9c: 0x3a75, 0xf9d: 0x3a95,
-	0xf9e: 0x3ab5, 0xf9f: 0x3ad5, 0xfa0: 0x3af5, 0xfa1: 0x3b15, 0xfa2: 0x3b35, 0xfa3: 0x3b55,
-	0xfa4: 0x3b75, 0xfa5: 0x3b95, 0xfa6: 0x1295, 0xfa7: 0x3bb5, 0xfa8: 0x3bd5, 0xfa9: 0x3bf5,
-	0xfaa: 0x3c15, 0xfab: 0x3c35, 0xfac: 0x3c55, 0xfad: 0x3c75, 0xfae: 0x23b5, 0xfaf: 0x3c95,
-	0xfb0: 0x3cb5, 0xfb1: 0x3939, 0xfb2: 0x3951, 0xfb3: 0x3969, 0xfb4: 0x3981, 0xfb5: 0x3999,
-	0xfb6: 0x39b1, 0xfb7: 0x39c9, 0xfb8: 0x39e1, 0xfb9: 0x39f9, 0xfba: 0x3a11, 0xfbb: 0x3a29,
-	0xfbc: 0x3a41, 0xfbd: 0x3a59, 0xfbe: 0x3a71, 0xfbf: 0x3a89,
-	// Block 0x3f, offset 0xfc0
-	0xfc0: 0x3aa1, 0xfc1: 0x3ac9, 0xfc2: 0x3af1, 0xfc3: 0x3b19, 0xfc4: 0x3b41, 0xfc5: 0x3b69,
-	0xfc6: 0x3b91, 0xfc7: 0x3bb9, 0xfc8: 0x3be1, 0xfc9: 0x3c09, 0xfca: 0x3c39, 0xfcb: 0x3c69,
-	0xfcc: 0x3c99, 0xfcd: 0x3cd5, 0xfce: 0x3cb1, 0xfcf: 0x3cf5, 0xfd0: 0x3d15, 0xfd1: 0x3d2d,
-	0xfd2: 0x3d45, 0xfd3: 0x3d5d, 0xfd4: 0x3d75, 0xfd5: 0x3d75, 0xfd6: 0x3d5d, 0xfd7: 0x3d8d,
-	0xfd8: 0x07d5, 0xfd9: 0x3da5, 0xfda: 0x3dbd, 0xfdb: 0x3dd5, 0xfdc: 0x3ded, 0xfdd: 0x3e05,
-	0xfde: 0x3e1d, 0xfdf: 0x3e35, 0xfe0: 0x3e4d, 0xfe1: 0x3e65, 0xfe2: 0x3e7d, 0xfe3: 0x3e95,
-	0xfe4: 0x3ead, 0xfe5: 0x3ead, 0xfe6: 0x3ec5, 0xfe7: 0x3ec5, 0xfe8: 0x3edd, 0xfe9: 0x3edd,
-	0xfea: 0x3ef5, 0xfeb: 0x3f0d, 0xfec: 0x3f25, 0xfed: 0x3f3d, 0xfee: 0x3f55, 0xfef: 0x3f55,
-	0xff0: 0x3f6d, 0xff1: 0x3f6d, 0xff2: 0x3f6d, 0xff3: 0x3f85, 0xff4: 0x3f9d, 0xff5: 0x3fb5,
-	0xff6: 0x3fcd, 0xff7: 0x3fb5, 0xff8: 0x3fe5, 0xff9: 0x3ffd, 0xffa: 0x3f85, 0xffb: 0x4015,
-	0xffc: 0x402d, 0xffd: 0x402d, 0xffe: 0x402d, 0xfff: 0x0040,
-	// Block 0x40, offset 0x1000
-	0x1000: 0x3cc9, 0x1001: 0x3d31, 0x1002: 0x3d99, 0x1003: 0x3e01, 0x1004: 0x3e51, 0x1005: 0x3eb9,
-	0x1006: 0x3f09, 0x1007: 0x3f59, 0x1008: 0x3fd9, 0x1009: 0x4041, 0x100a: 0x4091, 0x100b: 0x40e1,
-	0x100c: 0x4131, 0x100d: 0x4199, 0x100e: 0x4201, 0x100f: 0x4251, 0x1010: 0x42a1, 0x1011: 0x42d9,
-	0x1012: 0x4329, 0x1013: 0x4391, 0x1014: 0x43f9, 0x1015: 0x4431, 0x1016: 0x44b1, 0x1017: 0x4549,
-	0x1018: 0x45c9, 0x1019: 0x4619, 0x101a: 0x4699, 0x101b: 0x4719, 0x101c: 0x4781, 0x101d: 0x47d1,
-	0x101e: 0x4821, 0x101f: 0x4871, 0x1020: 0x48d9, 0x1021: 0x4959, 0x1022: 0x49c1, 0x1023: 0x4a11,
-	0x1024: 0x4a61, 0x1025: 0x4ab1, 0x1026: 0x4ae9, 0x1027: 0x4b21, 0x1028: 0x4b59, 0x1029: 0x4b91,
-	0x102a: 0x4be1, 0x102b: 0x4c31, 0x102c: 0x4cb1, 0x102d: 0x4d01, 0x102e: 0x4d69, 0x102f: 0x4de9,
-	0x1030: 0x4e39, 0x1031: 0x4e71, 0x1032: 0x4ea9, 0x1033: 0x4f29, 0x1034: 0x4f91, 0x1035: 0x5011,
-	0x1036: 0x5061, 0x1037: 0x50e1, 0x1038: 0x5119, 0x1039: 0x5169, 0x103a: 0x51b9, 0x103b: 0x5209,
-	0x103c: 0x5259, 0x103d: 0x52a9, 0x103e: 0x5311, 0x103f: 0x5361,
-	// Block 0x41, offset 0x1040
-	0x1040: 0x5399, 0x1041: 0x53e9, 0x1042: 0x5439, 0x1043: 0x5489, 0x1044: 0x54f1, 0x1045: 0x5541,
-	0x1046: 0x5591, 0x1047: 0x55e1, 0x1048: 0x5661, 0x1049: 0x56c9, 0x104a: 0x5701, 0x104b: 0x5781,
-	0x104c: 0x57b9, 0x104d: 0x5821, 0x104e: 0x5889, 0x104f: 0x58d9, 0x1050: 0x5929, 0x1051: 0x5979,
-	0x1052: 0x59e1, 0x1053: 0x5a19, 0x1054: 0x5a69, 0x1055: 0x5ad1, 0x1056: 0x5b09, 0x1057: 0x5b89,
-	0x1058: 0x5bd9, 0x1059: 0x5c01, 0x105a: 0x5c29, 0x105b: 0x5c51, 0x105c: 0x5c79, 0x105d: 0x5ca1,
-	0x105e: 0x5cc9, 0x105f: 0x5cf1, 0x1060: 0x5d19, 0x1061: 0x5d41, 0x1062: 0x5d69, 0x1063: 0x5d99,
-	0x1064: 0x5dc9, 0x1065: 0x5df9, 0x1066: 0x5e29, 0x1067: 0x5e59, 0x1068: 0x5e89, 0x1069: 0x5eb9,
-	0x106a: 0x5ee9, 0x106b: 0x5f19, 0x106c: 0x5f49, 0x106d: 0x5f79, 0x106e: 0x5fa9, 0x106f: 0x5fd9,
-	0x1070: 0x6009, 0x1071: 0x4045, 0x1072: 0x6039, 0x1073: 0x6051, 0x1074: 0x4065, 0x1075: 0x6069,
-	0x1076: 0x6081, 0x1077: 0x6099, 0x1078: 0x4085, 0x1079: 0x4085, 0x107a: 0x60b1, 0x107b: 0x60c9,
-	0x107c: 0x6101, 0x107d: 0x6139, 0x107e: 0x6171, 0x107f: 0x61a9,
-	// Block 0x42, offset 0x1080
-	0x1080: 0x6211, 0x1081: 0x6229, 0x1082: 0x40a5, 0x1083: 0x6241, 0x1084: 0x6259, 0x1085: 0x6271,
-	0x1086: 0x6289, 0x1087: 0x62a1, 0x1088: 0x40c5, 0x1089: 0x62b9, 0x108a: 0x62e1, 0x108b: 0x62f9,
-	0x108c: 0x40e5, 0x108d: 0x40e5, 0x108e: 0x6311, 0x108f: 0x6329, 0x1090: 0x6341, 0x1091: 0x4105,
-	0x1092: 0x4125, 0x1093: 0x4145, 0x1094: 0x4165, 0x1095: 0x4185, 0x1096: 0x6359, 0x1097: 0x6371,
-	0x1098: 0x6389, 0x1099: 0x63a1, 0x109a: 0x63b9, 0x109b: 0x41a5, 0x109c: 0x63d1, 0x109d: 0x63e9,
-	0x109e: 0x6401, 0x109f: 0x41c5, 0x10a0: 0x41e5, 0x10a1: 0x6419, 0x10a2: 0x4205, 0x10a3: 0x4225,
-	0x10a4: 0x4245, 0x10a5: 0x6431, 0x10a6: 0x4265, 0x10a7: 0x6449, 0x10a8: 0x6479, 0x10a9: 0x6211,
-	0x10aa: 0x4285, 0x10ab: 0x42a5, 0x10ac: 0x42c5, 0x10ad: 0x42e5, 0x10ae: 0x64b1, 0x10af: 0x64f1,
-	0x10b0: 0x6539, 0x10b1: 0x6551, 0x10b2: 0x4305, 0x10b3: 0x6569, 0x10b4: 0x6581, 0x10b5: 0x6599,
-	0x10b6: 0x4325, 0x10b7: 0x65b1, 0x10b8: 0x65c9, 0x10b9: 0x65b1, 0x10ba: 0x65e1, 0x10bb: 0x65f9,
-	0x10bc: 0x4345, 0x10bd: 0x6611, 0x10be: 0x6629, 0x10bf: 0x6611,
-	// Block 0x43, offset 0x10c0
-	0x10c0: 0x4365, 0x10c1: 0x4385, 0x10c2: 0x0040, 0x10c3: 0x6641, 0x10c4: 0x6659, 0x10c5: 0x6671,
-	0x10c6: 0x6689, 0x10c7: 0x0040, 0x10c8: 0x66c1, 0x10c9: 0x66d9, 0x10ca: 0x66f1, 0x10cb: 0x6709,
-	0x10cc: 0x6721, 0x10cd: 0x6739, 0x10ce: 0x6401, 0x10cf: 0x6751, 0x10d0: 0x6769, 0x10d1: 0x6781,
-	0x10d2: 0x43a5, 0x10d3: 0x6799, 0x10d4: 0x6289, 0x10d5: 0x43c5, 0x10d6: 0x43e5, 0x10d7: 0x67b1,
-	0x10d8: 0x0040, 0x10d9: 0x4405, 0x10da: 0x67c9, 0x10db: 0x67e1, 0x10dc: 0x67f9, 0x10dd: 0x6811,
-	0x10de: 0x6829, 0x10df: 0x6859, 0x10e0: 0x6889, 0x10e1: 0x68b1, 0x10e2: 0x68d9, 0x10e3: 0x6901,
-	0x10e4: 0x6929, 0x10e5: 0x6951, 0x10e6: 0x6979, 0x10e7: 0x69a1, 0x10e8: 0x69c9, 0x10e9: 0x69f1,
-	0x10ea: 0x6a21, 0x10eb: 0x6a51, 0x10ec: 0x6a81, 0x10ed: 0x6ab1, 0x10ee: 0x6ae1, 0x10ef: 0x6b11,
-	0x10f0: 0x6b41, 0x10f1: 0x6b71, 0x10f2: 0x6ba1, 0x10f3: 0x6bd1, 0x10f4: 0x6c01, 0x10f5: 0x6c31,
-	0x10f6: 0x6c61, 0x10f7: 0x6c91, 0x10f8: 0x6cc1, 0x10f9: 0x6cf1, 0x10fa: 0x6d21, 0x10fb: 0x6d51,
-	0x10fc: 0x6d81, 0x10fd: 0x6db1, 0x10fe: 0x6de1, 0x10ff: 0x4425,
-	// Block 0x44, offset 0x1100
-	0x1100: 0xe00d, 0x1101: 0x0008, 0x1102: 0xe00d, 0x1103: 0x0008, 0x1104: 0xe00d, 0x1105: 0x0008,
-	0x1106: 0xe00d, 0x1107: 0x0008, 0x1108: 0xe00d, 0x1109: 0x0008, 0x110a: 0xe00d, 0x110b: 0x0008,
-	0x110c: 0xe00d, 0x110d: 0x0008, 0x110e: 0xe00d, 0x110f: 0x0008, 0x1110: 0xe00d, 0x1111: 0x0008,
-	0x1112: 0xe00d, 0x1113: 0x0008, 0x1114: 0xe00d, 0x1115: 0x0008, 0x1116: 0xe00d, 0x1117: 0x0008,
-	0x1118: 0xe00d, 0x1119: 0x0008, 0x111a: 0xe00d, 0x111b: 0x0008, 0x111c: 0xe00d, 0x111d: 0x0008,
-	0x111e: 0xe00d, 0x111f: 0x0008, 0x1120: 0xe00d, 0x1121: 0x0008, 0x1122: 0xe00d, 0x1123: 0x0008,
-	0x1124: 0xe00d, 0x1125: 0x0008, 0x1126: 0xe00d, 0x1127: 0x0008, 0x1128: 0xe00d, 0x1129: 0x0008,
-	0x112a: 0xe00d, 0x112b: 0x0008, 0x112c: 0xe00d, 0x112d: 0x0008, 0x112e: 0x0008, 0x112f: 0x3308,
-	0x1130: 0x3318, 0x1131: 0x3318, 0x1132: 0x3318, 0x1133: 0x0018, 0x1134: 0x3308, 0x1135: 0x3308,
-	0x1136: 0x3308, 0x1137: 0x3308, 0x1138: 0x3308, 0x1139: 0x3308, 0x113a: 0x3308, 0x113b: 0x3308,
-	0x113c: 0x3308, 0x113d: 0x3308, 0x113e: 0x0018, 0x113f: 0x0008,
-	// Block 0x45, offset 0x1140
-	0x1140: 0xe00d, 0x1141: 0x0008, 0x1142: 0xe00d, 0x1143: 0x0008, 0x1144: 0xe00d, 0x1145: 0x0008,
-	0x1146: 0xe00d, 0x1147: 0x0008, 0x1148: 0xe00d, 0x1149: 0x0008, 0x114a: 0xe00d, 0x114b: 0x0008,
-	0x114c: 0xe00d, 0x114d: 0x0008, 0x114e: 0xe00d, 0x114f: 0x0008, 0x1150: 0xe00d, 0x1151: 0x0008,
-	0x1152: 0xe00d, 0x1153: 0x0008, 0x1154: 0xe00d, 0x1155: 0x0008, 0x1156: 0xe00d, 0x1157: 0x0008,
-	0x1158: 0xe00d, 0x1159: 0x0008, 0x115a: 0xe00d, 0x115b: 0x0008, 0x115c: 0x0ea1, 0x115d: 0x6e11,
-	0x115e: 0x3308, 0x115f: 0x3308, 0x1160: 0x0008, 0x1161: 0x0008, 0x1162: 0x0008, 0x1163: 0x0008,
-	0x1164: 0x0008, 0x1165: 0x0008, 0x1166: 0x0008, 0x1167: 0x0008, 0x1168: 0x0008, 0x1169: 0x0008,
-	0x116a: 0x0008, 0x116b: 0x0008, 0x116c: 0x0008, 0x116d: 0x0008, 0x116e: 0x0008, 0x116f: 0x0008,
-	0x1170: 0x0008, 0x1171: 0x0008, 0x1172: 0x0008, 0x1173: 0x0008, 0x1174: 0x0008, 0x1175: 0x0008,
-	0x1176: 0x0008, 0x1177: 0x0008, 0x1178: 0x0008, 0x1179: 0x0008, 0x117a: 0x0008, 0x117b: 0x0008,
-	0x117c: 0x0008, 0x117d: 0x0008, 0x117e: 0x0008, 0x117f: 0x0008,
-	// Block 0x46, offset 0x1180
-	0x1180: 0x0018, 0x1181: 0x0018, 0x1182: 0x0018, 0x1183: 0x0018, 0x1184: 0x0018, 0x1185: 0x0018,
-	0x1186: 0x0018, 0x1187: 0x0018, 0x1188: 0x0018, 0x1189: 0x0018, 0x118a: 0x0018, 0x118b: 0x0018,
-	0x118c: 0x0018, 0x118d: 0x0018, 0x118e: 0x0018, 0x118f: 0x0018, 0x1190: 0x0018, 0x1191: 0x0018,
-	0x1192: 0x0018, 0x1193: 0x0018, 0x1194: 0x0018, 0x1195: 0x0018, 0x1196: 0x0018, 0x1197: 0x0008,
-	0x1198: 0x0008, 0x1199: 0x0008, 0x119a: 0x0008, 0x119b: 0x0008, 0x119c: 0x0008, 0x119d: 0x0008,
-	0x119e: 0x0008, 0x119f: 0x0008, 0x11a0: 0x0018, 0x11a1: 0x0018, 0x11a2: 0xe00d, 0x11a3: 0x0008,
-	0x11a4: 0xe00d, 0x11a5: 0x0008, 0x11a6: 0xe00d, 0x11a7: 0x0008, 0x11a8: 0xe00d, 0x11a9: 0x0008,
-	0x11aa: 0xe00d, 0x11ab: 0x0008, 0x11ac: 0xe00d, 0x11ad: 0x0008, 0x11ae: 0xe00d, 0x11af: 0x0008,
-	0x11b0: 0x0008, 0x11b1: 0x0008, 0x11b2: 0xe00d, 0x11b3: 0x0008, 0x11b4: 0xe00d, 0x11b5: 0x0008,
-	0x11b6: 0xe00d, 0x11b7: 0x0008, 0x11b8: 0xe00d, 0x11b9: 0x0008, 0x11ba: 0xe00d, 0x11bb: 0x0008,
-	0x11bc: 0xe00d, 0x11bd: 0x0008, 0x11be: 0xe00d, 0x11bf: 0x0008,
-	// Block 0x47, offset 0x11c0
-	0x11c0: 0xe00d, 0x11c1: 0x0008, 0x11c2: 0xe00d, 0x11c3: 0x0008, 0x11c4: 0xe00d, 0x11c5: 0x0008,
-	0x11c6: 0xe00d, 0x11c7: 0x0008, 0x11c8: 0xe00d, 0x11c9: 0x0008, 0x11ca: 0xe00d, 0x11cb: 0x0008,
-	0x11cc: 0xe00d, 0x11cd: 0x0008, 0x11ce: 0xe00d, 0x11cf: 0x0008, 0x11d0: 0xe00d, 0x11d1: 0x0008,
-	0x11d2: 0xe00d, 0x11d3: 0x0008, 0x11d4: 0xe00d, 0x11d5: 0x0008, 0x11d6: 0xe00d, 0x11d7: 0x0008,
-	0x11d8: 0xe00d, 0x11d9: 0x0008, 0x11da: 0xe00d, 0x11db: 0x0008, 0x11dc: 0xe00d, 0x11dd: 0x0008,
-	0x11de: 0xe00d, 0x11df: 0x0008, 0x11e0: 0xe00d, 0x11e1: 0x0008, 0x11e2: 0xe00d, 0x11e3: 0x0008,
-	0x11e4: 0xe00d, 0x11e5: 0x0008, 0x11e6: 0xe00d, 0x11e7: 0x0008, 0x11e8: 0xe00d, 0x11e9: 0x0008,
-	0x11ea: 0xe00d, 0x11eb: 0x0008, 0x11ec: 0xe00d, 0x11ed: 0x0008, 0x11ee: 0xe00d, 0x11ef: 0x0008,
-	0x11f0: 0xe0fd, 0x11f1: 0x0008, 0x11f2: 0x0008, 0x11f3: 0x0008, 0x11f4: 0x0008, 0x11f5: 0x0008,
-	0x11f6: 0x0008, 0x11f7: 0x0008, 0x11f8: 0x0008, 0x11f9: 0xe01d, 0x11fa: 0x0008, 0x11fb: 0xe03d,
-	0x11fc: 0x0008, 0x11fd: 0x4445, 0x11fe: 0xe00d, 0x11ff: 0x0008,
-	// Block 0x48, offset 0x1200
-	0x1200: 0xe00d, 0x1201: 0x0008, 0x1202: 0xe00d, 0x1203: 0x0008, 0x1204: 0xe00d, 0x1205: 0x0008,
-	0x1206: 0xe00d, 0x1207: 0x0008, 0x1208: 0x0008, 0x1209: 0x0018, 0x120a: 0x0018, 0x120b: 0xe03d,
-	0x120c: 0x0008, 0x120d: 0x11d9, 0x120e: 0x0008, 0x120f: 0x0008, 0x1210: 0xe00d, 0x1211: 0x0008,
-	0x1212: 0xe00d, 0x1213: 0x0008, 0x1214: 0x0008, 0x1215: 0x0008, 0x1216: 0xe00d, 0x1217: 0x0008,
-	0x1218: 0xe00d, 0x1219: 0x0008, 0x121a: 0xe00d, 0x121b: 0x0008, 0x121c: 0xe00d, 0x121d: 0x0008,
-	0x121e: 0xe00d, 0x121f: 0x0008, 0x1220: 0xe00d, 0x1221: 0x0008, 0x1222: 0xe00d, 0x1223: 0x0008,
-	0x1224: 0xe00d, 0x1225: 0x0008, 0x1226: 0xe00d, 0x1227: 0x0008, 0x1228: 0xe00d, 0x1229: 0x0008,
-	0x122a: 0x6e29, 0x122b: 0x1029, 0x122c: 0x11c1, 0x122d: 0x6e41, 0x122e: 0x1221, 0x122f: 0x0008,
-	0x1230: 0x6e59, 0x1231: 0x6e71, 0x1232: 0x1239, 0x1233: 0x4465, 0x1234: 0xe00d, 0x1235: 0x0008,
-	0x1236: 0xe00d, 0x1237: 0x0008, 0x1238: 0xe00d, 0x1239: 0x0008, 0x123a: 0xe00d, 0x123b: 0x0008,
-	0x123c: 0xe00d, 0x123d: 0x0008, 0x123e: 0xe00d, 0x123f: 0x0008,
-	// Block 0x49, offset 0x1240
-	0x1240: 0x650d, 0x1241: 0x652d, 0x1242: 0x654d, 0x1243: 0x656d, 0x1244: 0x658d, 0x1245: 0x65ad,
-	0x1246: 0x65cd, 0x1247: 0x65ed, 0x1248: 0x660d, 0x1249: 0x662d, 0x124a: 0x664d, 0x124b: 0x666d,
-	0x124c: 0x668d, 0x124d: 0x66ad, 0x124e: 0x0008, 0x124f: 0x0008, 0x1250: 0x66cd, 0x1251: 0x0008,
-	0x1252: 0x66ed, 0x1253: 0x0008, 0x1254: 0x0008, 0x1255: 0x670d, 0x1256: 0x672d, 0x1257: 0x674d,
-	0x1258: 0x676d, 0x1259: 0x678d, 0x125a: 0x67ad, 0x125b: 0x67cd, 0x125c: 0x67ed, 0x125d: 0x680d,
-	0x125e: 0x682d, 0x125f: 0x0008, 0x1260: 0x684d, 0x1261: 0x0008, 0x1262: 0x686d, 0x1263: 0x0008,
-	0x1264: 0x0008, 0x1265: 0x688d, 0x1266: 0x68ad, 0x1267: 0x0008, 0x1268: 0x0008, 0x1269: 0x0008,
-	0x126a: 0x68cd, 0x126b: 0x68ed, 0x126c: 0x690d, 0x126d: 0x692d, 0x126e: 0x694d, 0x126f: 0x696d,
-	0x1270: 0x698d, 0x1271: 0x69ad, 0x1272: 0x69cd, 0x1273: 0x69ed, 0x1274: 0x6a0d, 0x1275: 0x6a2d,
-	0x1276: 0x6a4d, 0x1277: 0x6a6d, 0x1278: 0x6a8d, 0x1279: 0x6aad, 0x127a: 0x6acd, 0x127b: 0x6aed,
-	0x127c: 0x6b0d, 0x127d: 0x6b2d, 0x127e: 0x6b4d, 0x127f: 0x6b6d,
-	// Block 0x4a, offset 0x1280
-	0x1280: 0x7acd, 0x1281: 0x7aed, 0x1282: 0x7b0d, 0x1283: 0x7b2d, 0x1284: 0x7b4d, 0x1285: 0x7b6d,
-	0x1286: 0x7b8d, 0x1287: 0x7bad, 0x1288: 0x7bcd, 0x1289: 0x7bed, 0x128a: 0x7c0d, 0x128b: 0x7c2d,
-	0x128c: 0x7c4d, 0x128d: 0x7c6d, 0x128e: 0x7c8d, 0x128f: 0x6ec9, 0x1290: 0x6ef1, 0x1291: 0x6f19,
-	0x1292: 0x7cad, 0x1293: 0x7ccd, 0x1294: 0x7ced, 0x1295: 0x6f41, 0x1296: 0x6f69, 0x1297: 0x6f91,
-	0x1298: 0x7d0d, 0x1299: 0x7d2d, 0x129a: 0x0040, 0x129b: 0x0040, 0x129c: 0x0040, 0x129d: 0x0040,
-	0x129e: 0x0040, 0x129f: 0x0040, 0x12a0: 0x0040, 0x12a1: 0x0040, 0x12a2: 0x0040, 0x12a3: 0x0040,
-	0x12a4: 0x0040, 0x12a5: 0x0040, 0x12a6: 0x0040, 0x12a7: 0x0040, 0x12a8: 0x0040, 0x12a9: 0x0040,
-	0x12aa: 0x0040, 0x12ab: 0x0040, 0x12ac: 0x0040, 0x12ad: 0x0040, 0x12ae: 0x0040, 0x12af: 0x0040,
-	0x12b0: 0x0040, 0x12b1: 0x0040, 0x12b2: 0x0040, 0x12b3: 0x0040, 0x12b4: 0x0040, 0x12b5: 0x0040,
-	0x12b6: 0x0040, 0x12b7: 0x0040, 0x12b8: 0x0040, 0x12b9: 0x0040, 0x12ba: 0x0040, 0x12bb: 0x0040,
-	0x12bc: 0x0040, 0x12bd: 0x0040, 0x12be: 0x0040, 0x12bf: 0x0040,
-	// Block 0x4b, offset 0x12c0
-	0x12c0: 0x6fb9, 0x12c1: 0x6fd1, 0x12c2: 0x6fe9, 0x12c3: 0x7d4d, 0x12c4: 0x7d6d, 0x12c5: 0x7001,
-	0x12c6: 0x7001, 0x12c7: 0x0040, 0x12c8: 0x0040, 0x12c9: 0x0040, 0x12ca: 0x0040, 0x12cb: 0x0040,
-	0x12cc: 0x0040, 0x12cd: 0x0040, 0x12ce: 0x0040, 0x12cf: 0x0040, 0x12d0: 0x0040, 0x12d1: 0x0040,
-	0x12d2: 0x0040, 0x12d3: 0x7019, 0x12d4: 0x7041, 0x12d5: 0x7069, 0x12d6: 0x7091, 0x12d7: 0x70b9,
-	0x12d8: 0x0040, 0x12d9: 0x0040, 0x12da: 0x0040, 0x12db: 0x0040, 0x12dc: 0x0040, 0x12dd: 0x70e1,
-	0x12de: 0x3308, 0x12df: 0x7109, 0x12e0: 0x7131, 0x12e1: 0x20a9, 0x12e2: 0x20f1, 0x12e3: 0x7149,
-	0x12e4: 0x7161, 0x12e5: 0x7179, 0x12e6: 0x7191, 0x12e7: 0x71a9, 0x12e8: 0x71c1, 0x12e9: 0x1fb2,
-	0x12ea: 0x71d9, 0x12eb: 0x7201, 0x12ec: 0x7229, 0x12ed: 0x7261, 0x12ee: 0x7299, 0x12ef: 0x72c1,
-	0x12f0: 0x72e9, 0x12f1: 0x7311, 0x12f2: 0x7339, 0x12f3: 0x7361, 0x12f4: 0x7389, 0x12f5: 0x73b1,
-	0x12f6: 0x73d9, 0x12f7: 0x0040, 0x12f8: 0x7401, 0x12f9: 0x7429, 0x12fa: 0x7451, 0x12fb: 0x7479,
-	0x12fc: 0x74a1, 0x12fd: 0x0040, 0x12fe: 0x74c9, 0x12ff: 0x0040,
-	// Block 0x4c, offset 0x1300
-	0x1300: 0x74f1, 0x1301: 0x7519, 0x1302: 0x0040, 0x1303: 0x7541, 0x1304: 0x7569, 0x1305: 0x0040,
-	0x1306: 0x7591, 0x1307: 0x75b9, 0x1308: 0x75e1, 0x1309: 0x7609, 0x130a: 0x7631, 0x130b: 0x7659,
-	0x130c: 0x7681, 0x130d: 0x76a9, 0x130e: 0x76d1, 0x130f: 0x76f9, 0x1310: 0x7721, 0x1311: 0x7721,
-	0x1312: 0x7739, 0x1313: 0x7739, 0x1314: 0x7739, 0x1315: 0x7739, 0x1316: 0x7751, 0x1317: 0x7751,
-	0x1318: 0x7751, 0x1319: 0x7751, 0x131a: 0x7769, 0x131b: 0x7769, 0x131c: 0x7769, 0x131d: 0x7769,
-	0x131e: 0x7781, 0x131f: 0x7781, 0x1320: 0x7781, 0x1321: 0x7781, 0x1322: 0x7799, 0x1323: 0x7799,
-	0x1324: 0x7799, 0x1325: 0x7799, 0x1326: 0x77b1, 0x1327: 0x77b1, 0x1328: 0x77b1, 0x1329: 0x77b1,
-	0x132a: 0x77c9, 0x132b: 0x77c9, 0x132c: 0x77c9, 0x132d: 0x77c9, 0x132e: 0x77e1, 0x132f: 0x77e1,
-	0x1330: 0x77e1, 0x1331: 0x77e1, 0x1332: 0x77f9, 0x1333: 0x77f9, 0x1334: 0x77f9, 0x1335: 0x77f9,
-	0x1336: 0x7811, 0x1337: 0x7811, 0x1338: 0x7811, 0x1339: 0x7811, 0x133a: 0x7829, 0x133b: 0x7829,
-	0x133c: 0x7829, 0x133d: 0x7829, 0x133e: 0x7841, 0x133f: 0x7841,
-	// Block 0x4d, offset 0x1340
-	0x1340: 0x7841, 0x1341: 0x7841, 0x1342: 0x7859, 0x1343: 0x7859, 0x1344: 0x7871, 0x1345: 0x7871,
-	0x1346: 0x7889, 0x1347: 0x7889, 0x1348: 0x78a1, 0x1349: 0x78a1, 0x134a: 0x78b9, 0x134b: 0x78b9,
-	0x134c: 0x78d1, 0x134d: 0x78d1, 0x134e: 0x78e9, 0x134f: 0x78e9, 0x1350: 0x78e9, 0x1351: 0x78e9,
-	0x1352: 0x7901, 0x1353: 0x7901, 0x1354: 0x7901, 0x1355: 0x7901, 0x1356: 0x7919, 0x1357: 0x7919,
-	0x1358: 0x7919, 0x1359: 0x7919, 0x135a: 0x7931, 0x135b: 0x7931, 0x135c: 0x7931, 0x135d: 0x7931,
-	0x135e: 0x7949, 0x135f: 0x7949, 0x1360: 0x7961, 0x1361: 0x7961, 0x1362: 0x7961, 0x1363: 0x7961,
-	0x1364: 0x7979, 0x1365: 0x7979, 0x1366: 0x7991, 0x1367: 0x7991, 0x1368: 0x7991, 0x1369: 0x7991,
-	0x136a: 0x79a9, 0x136b: 0x79a9, 0x136c: 0x79a9, 0x136d: 0x79a9, 0x136e: 0x79c1, 0x136f: 0x79c1,
-	0x1370: 0x79d9, 0x1371: 0x79d9, 0x1372: 0x0818, 0x1373: 0x0818, 0x1374: 0x0818, 0x1375: 0x0818,
-	0x1376: 0x0818, 0x1377: 0x0818, 0x1378: 0x0818, 0x1379: 0x0818, 0x137a: 0x0818, 0x137b: 0x0818,
-	0x137c: 0x0818, 0x137d: 0x0818, 0x137e: 0x0818, 0x137f: 0x0818,
-	// Block 0x4e, offset 0x1380
-	0x1380: 0x0818, 0x1381: 0x0818, 0x1382: 0x0040, 0x1383: 0x0040, 0x1384: 0x0040, 0x1385: 0x0040,
-	0x1386: 0x0040, 0x1387: 0x0040, 0x1388: 0x0040, 0x1389: 0x0040, 0x138a: 0x0040, 0x138b: 0x0040,
-	0x138c: 0x0040, 0x138d: 0x0040, 0x138e: 0x0040, 0x138f: 0x0040, 0x1390: 0x0040, 0x1391: 0x0040,
-	0x1392: 0x0040, 0x1393: 0x79f1, 0x1394: 0x79f1, 0x1395: 0x79f1, 0x1396: 0x79f1, 0x1397: 0x7a09,
-	0x1398: 0x7a09, 0x1399: 0x7a21, 0x139a: 0x7a21, 0x139b: 0x7a39, 0x139c: 0x7a39, 0x139d: 0x0479,
-	0x139e: 0x7a51, 0x139f: 0x7a51, 0x13a0: 0x7a69, 0x13a1: 0x7a69, 0x13a2: 0x7a81, 0x13a3: 0x7a81,
-	0x13a4: 0x7a99, 0x13a5: 0x7a99, 0x13a6: 0x7a99, 0x13a7: 0x7a99, 0x13a8: 0x7ab1, 0x13a9: 0x7ab1,
-	0x13aa: 0x7ac9, 0x13ab: 0x7ac9, 0x13ac: 0x7af1, 0x13ad: 0x7af1, 0x13ae: 0x7b19, 0x13af: 0x7b19,
-	0x13b0: 0x7b41, 0x13b1: 0x7b41, 0x13b2: 0x7b69, 0x13b3: 0x7b69, 0x13b4: 0x7b91, 0x13b5: 0x7b91,
-	0x13b6: 0x7bb9, 0x13b7: 0x7bb9, 0x13b8: 0x7bb9, 0x13b9: 0x7be1, 0x13ba: 0x7be1, 0x13bb: 0x7be1,
-	0x13bc: 0x7c09, 0x13bd: 0x7c09, 0x13be: 0x7c09, 0x13bf: 0x7c09,
-	// Block 0x4f, offset 0x13c0
-	0x13c0: 0x85f9, 0x13c1: 0x8621, 0x13c2: 0x8649, 0x13c3: 0x8671, 0x13c4: 0x8699, 0x13c5: 0x86c1,
-	0x13c6: 0x86e9, 0x13c7: 0x8711, 0x13c8: 0x8739, 0x13c9: 0x8761, 0x13ca: 0x8789, 0x13cb: 0x87b1,
-	0x13cc: 0x87d9, 0x13cd: 0x8801, 0x13ce: 0x8829, 0x13cf: 0x8851, 0x13d0: 0x8879, 0x13d1: 0x88a1,
-	0x13d2: 0x88c9, 0x13d3: 0x88f1, 0x13d4: 0x8919, 0x13d5: 0x8941, 0x13d6: 0x8969, 0x13d7: 0x8991,
-	0x13d8: 0x89b9, 0x13d9: 0x89e1, 0x13da: 0x8a09, 0x13db: 0x8a31, 0x13dc: 0x8a59, 0x13dd: 0x8a81,
-	0x13de: 0x8aaa, 0x13df: 0x8ada, 0x13e0: 0x8b0a, 0x13e1: 0x8b3a, 0x13e2: 0x8b6a, 0x13e3: 0x8b9a,
-	0x13e4: 0x8bc9, 0x13e5: 0x8bf1, 0x13e6: 0x7c71, 0x13e7: 0x8c19, 0x13e8: 0x7be1, 0x13e9: 0x7c99,
-	0x13ea: 0x8c41, 0x13eb: 0x8c69, 0x13ec: 0x7d39, 0x13ed: 0x8c91, 0x13ee: 0x7d61, 0x13ef: 0x7d89,
-	0x13f0: 0x8cb9, 0x13f1: 0x8ce1, 0x13f2: 0x7e29, 0x13f3: 0x8d09, 0x13f4: 0x7e51, 0x13f5: 0x7e79,
-	0x13f6: 0x8d31, 0x13f7: 0x8d59, 0x13f8: 0x7ec9, 0x13f9: 0x8d81, 0x13fa: 0x7ef1, 0x13fb: 0x7f19,
-	0x13fc: 0x83a1, 0x13fd: 0x83c9, 0x13fe: 0x8441, 0x13ff: 0x8469,
-	// Block 0x50, offset 0x1400
-	0x1400: 0x8491, 0x1401: 0x8531, 0x1402: 0x8559, 0x1403: 0x8581, 0x1404: 0x85a9, 0x1405: 0x8649,
-	0x1406: 0x8671, 0x1407: 0x8699, 0x1408: 0x8da9, 0x1409: 0x8739, 0x140a: 0x8dd1, 0x140b: 0x8df9,
-	0x140c: 0x8829, 0x140d: 0x8e21, 0x140e: 0x8851, 0x140f: 0x8879, 0x1410: 0x8a81, 0x1411: 0x8e49,
-	0x1412: 0x8e71, 0x1413: 0x89b9, 0x1414: 0x8e99, 0x1415: 0x89e1, 0x1416: 0x8a09, 0x1417: 0x7c21,
-	0x1418: 0x7c49, 0x1419: 0x8ec1, 0x141a: 0x7c71, 0x141b: 0x8ee9, 0x141c: 0x7cc1, 0x141d: 0x7ce9,
-	0x141e: 0x7d11, 0x141f: 0x7d39, 0x1420: 0x8f11, 0x1421: 0x7db1, 0x1422: 0x7dd9, 0x1423: 0x7e01,
-	0x1424: 0x7e29, 0x1425: 0x8f39, 0x1426: 0x7ec9, 0x1427: 0x7f41, 0x1428: 0x7f69, 0x1429: 0x7f91,
-	0x142a: 0x7fb9, 0x142b: 0x7fe1, 0x142c: 0x8031, 0x142d: 0x8059, 0x142e: 0x8081, 0x142f: 0x80a9,
-	0x1430: 0x80d1, 0x1431: 0x80f9, 0x1432: 0x8f61, 0x1433: 0x8121, 0x1434: 0x8149, 0x1435: 0x8171,
-	0x1436: 0x8199, 0x1437: 0x81c1, 0x1438: 0x81e9, 0x1439: 0x8239, 0x143a: 0x8261, 0x143b: 0x8289,
-	0x143c: 0x82b1, 0x143d: 0x82d9, 0x143e: 0x8301, 0x143f: 0x8329,
-	// Block 0x51, offset 0x1440
-	0x1440: 0x8351, 0x1441: 0x8379, 0x1442: 0x83f1, 0x1443: 0x8419, 0x1444: 0x84b9, 0x1445: 0x84e1,
-	0x1446: 0x8509, 0x1447: 0x8531, 0x1448: 0x8559, 0x1449: 0x85d1, 0x144a: 0x85f9, 0x144b: 0x8621,
-	0x144c: 0x8649, 0x144d: 0x8f89, 0x144e: 0x86c1, 0x144f: 0x86e9, 0x1450: 0x8711, 0x1451: 0x8739,
-	0x1452: 0x87b1, 0x1453: 0x87d9, 0x1454: 0x8801, 0x1455: 0x8829, 0x1456: 0x8fb1, 0x1457: 0x88a1,
-	0x1458: 0x88c9, 0x1459: 0x8fd9, 0x145a: 0x8941, 0x145b: 0x8969, 0x145c: 0x8991, 0x145d: 0x89b9,
-	0x145e: 0x9001, 0x145f: 0x7c71, 0x1460: 0x8ee9, 0x1461: 0x7d39, 0x1462: 0x8f11, 0x1463: 0x7e29,
-	0x1464: 0x8f39, 0x1465: 0x7ec9, 0x1466: 0x9029, 0x1467: 0x80d1, 0x1468: 0x9051, 0x1469: 0x9079,
-	0x146a: 0x90a1, 0x146b: 0x8531, 0x146c: 0x8559, 0x146d: 0x8649, 0x146e: 0x8829, 0x146f: 0x8fb1,
-	0x1470: 0x89b9, 0x1471: 0x9001, 0x1472: 0x90c9, 0x1473: 0x9101, 0x1474: 0x9139, 0x1475: 0x9171,
-	0x1476: 0x9199, 0x1477: 0x91c1, 0x1478: 0x91e9, 0x1479: 0x9211, 0x147a: 0x9239, 0x147b: 0x9261,
-	0x147c: 0x9289, 0x147d: 0x92b1, 0x147e: 0x92d9, 0x147f: 0x9301,
-	// Block 0x52, offset 0x1480
-	0x1480: 0x9329, 0x1481: 0x9351, 0x1482: 0x9379, 0x1483: 0x93a1, 0x1484: 0x93c9, 0x1485: 0x93f1,
-	0x1486: 0x9419, 0x1487: 0x9441, 0x1488: 0x9469, 0x1489: 0x9491, 0x148a: 0x94b9, 0x148b: 0x94e1,
-	0x148c: 0x9079, 0x148d: 0x9509, 0x148e: 0x9531, 0x148f: 0x9559, 0x1490: 0x9581, 0x1491: 0x9171,
-	0x1492: 0x9199, 0x1493: 0x91c1, 0x1494: 0x91e9, 0x1495: 0x9211, 0x1496: 0x9239, 0x1497: 0x9261,
-	0x1498: 0x9289, 0x1499: 0x92b1, 0x149a: 0x92d9, 0x149b: 0x9301, 0x149c: 0x9329, 0x149d: 0x9351,
-	0x149e: 0x9379, 0x149f: 0x93a1, 0x14a0: 0x93c9, 0x14a1: 0x93f1, 0x14a2: 0x9419, 0x14a3: 0x9441,
-	0x14a4: 0x9469, 0x14a5: 0x9491, 0x14a6: 0x94b9, 0x14a7: 0x94e1, 0x14a8: 0x9079, 0x14a9: 0x9509,
-	0x14aa: 0x9531, 0x14ab: 0x9559, 0x14ac: 0x9581, 0x14ad: 0x9491, 0x14ae: 0x94b9, 0x14af: 0x94e1,
-	0x14b0: 0x9079, 0x14b1: 0x9051, 0x14b2: 0x90a1, 0x14b3: 0x8211, 0x14b4: 0x8059, 0x14b5: 0x8081,
-	0x14b6: 0x80a9, 0x14b7: 0x9491, 0x14b8: 0x94b9, 0x14b9: 0x94e1, 0x14ba: 0x8211, 0x14bb: 0x8239,
-	0x14bc: 0x95a9, 0x14bd: 0x95a9, 0x14be: 0x0018, 0x14bf: 0x0018,
-	// Block 0x53, offset 0x14c0
-	0x14c0: 0x0040, 0x14c1: 0x0040, 0x14c2: 0x0040, 0x14c3: 0x0040, 0x14c4: 0x0040, 0x14c5: 0x0040,
-	0x14c6: 0x0040, 0x14c7: 0x0040, 0x14c8: 0x0040, 0x14c9: 0x0040, 0x14ca: 0x0040, 0x14cb: 0x0040,
-	0x14cc: 0x0040, 0x14cd: 0x0040, 0x14ce: 0x0040, 0x14cf: 0x0040, 0x14d0: 0x95d1, 0x14d1: 0x9609,
-	0x14d2: 0x9609, 0x14d3: 0x9641, 0x14d4: 0x9679, 0x14d5: 0x96b1, 0x14d6: 0x96e9, 0x14d7: 0x9721,
-	0x14d8: 0x9759, 0x14d9: 0x9759, 0x14da: 0x9791, 0x14db: 0x97c9, 0x14dc: 0x9801, 0x14dd: 0x9839,
-	0x14de: 0x9871, 0x14df: 0x98a9, 0x14e0: 0x98a9, 0x14e1: 0x98e1, 0x14e2: 0x9919, 0x14e3: 0x9919,
-	0x14e4: 0x9951, 0x14e5: 0x9951, 0x14e6: 0x9989, 0x14e7: 0x99c1, 0x14e8: 0x99c1, 0x14e9: 0x99f9,
-	0x14ea: 0x9a31, 0x14eb: 0x9a31, 0x14ec: 0x9a69, 0x14ed: 0x9a69, 0x14ee: 0x9aa1, 0x14ef: 0x9ad9,
-	0x14f0: 0x9ad9, 0x14f1: 0x9b11, 0x14f2: 0x9b11, 0x14f3: 0x9b49, 0x14f4: 0x9b81, 0x14f5: 0x9bb9,
-	0x14f6: 0x9bf1, 0x14f7: 0x9bf1, 0x14f8: 0x9c29, 0x14f9: 0x9c61, 0x14fa: 0x9c99, 0x14fb: 0x9cd1,
-	0x14fc: 0x9d09, 0x14fd: 0x9d09, 0x14fe: 0x9d41, 0x14ff: 0x9d79,
-	// Block 0x54, offset 0x1500
-	0x1500: 0xa949, 0x1501: 0xa981, 0x1502: 0xa9b9, 0x1503: 0xa8a1, 0x1504: 0x9bb9, 0x1505: 0x9989,
-	0x1506: 0xa9f1, 0x1507: 0xaa29, 0x1508: 0x0040, 0x1509: 0x0040, 0x150a: 0x0040, 0x150b: 0x0040,
-	0x150c: 0x0040, 0x150d: 0x0040, 0x150e: 0x0040, 0x150f: 0x0040, 0x1510: 0x0040, 0x1511: 0x0040,
-	0x1512: 0x0040, 0x1513: 0x0040, 0x1514: 0x0040, 0x1515: 0x0040, 0x1516: 0x0040, 0x1517: 0x0040,
-	0x1518: 0x0040, 0x1519: 0x0040, 0x151a: 0x0040, 0x151b: 0x0040, 0x151c: 0x0040, 0x151d: 0x0040,
-	0x151e: 0x0040, 0x151f: 0x0040, 0x1520: 0x0040, 0x1521: 0x0040, 0x1522: 0x0040, 0x1523: 0x0040,
-	0x1524: 0x0040, 0x1525: 0x0040, 0x1526: 0x0040, 0x1527: 0x0040, 0x1528: 0x0040, 0x1529: 0x0040,
-	0x152a: 0x0040, 0x152b: 0x0040, 0x152c: 0x0040, 0x152d: 0x0040, 0x152e: 0x0040, 0x152f: 0x0040,
-	0x1530: 0xaa61, 0x1531: 0xaa99, 0x1532: 0xaad1, 0x1533: 0xab19, 0x1534: 0xab61, 0x1535: 0xaba9,
-	0x1536: 0xabf1, 0x1537: 0xac39, 0x1538: 0xac81, 0x1539: 0xacc9, 0x153a: 0xad02, 0x153b: 0xae12,
-	0x153c: 0xae91, 0x153d: 0x0018, 0x153e: 0x0040, 0x153f: 0x0040,
-	// Block 0x55, offset 0x1540
-	0x1540: 0x33c0, 0x1541: 0x33c0, 0x1542: 0x33c0, 0x1543: 0x33c0, 0x1544: 0x33c0, 0x1545: 0x33c0,
-	0x1546: 0x33c0, 0x1547: 0x33c0, 0x1548: 0x33c0, 0x1549: 0x33c0, 0x154a: 0x33c0, 0x154b: 0x33c0,
-	0x154c: 0x33c0, 0x154d: 0x33c0, 0x154e: 0x33c0, 0x154f: 0x33c0, 0x1550: 0xaeda, 0x1551: 0x7d8d,
-	0x1552: 0x0040, 0x1553: 0xaeea, 0x1554: 0x03c2, 0x1555: 0xaefa, 0x1556: 0xaf0a, 0x1557: 0x7dad,
-	0x1558: 0x7dcd, 0x1559: 0x0040, 0x155a: 0x0040, 0x155b: 0x0040, 0x155c: 0x0040, 0x155d: 0x0040,
-	0x155e: 0x0040, 0x155f: 0x0040, 0x1560: 0x3308, 0x1561: 0x3308, 0x1562: 0x3308, 0x1563: 0x3308,
-	0x1564: 0x3308, 0x1565: 0x3308, 0x1566: 0x3308, 0x1567: 0x3308, 0x1568: 0x3308, 0x1569: 0x3308,
-	0x156a: 0x3308, 0x156b: 0x3308, 0x156c: 0x3308, 0x156d: 0x3308, 0x156e: 0x3308, 0x156f: 0x3308,
-	0x1570: 0x0040, 0x1571: 0x7ded, 0x1572: 0x7e0d, 0x1573: 0xaf1a, 0x1574: 0xaf1a, 0x1575: 0x1fd2,
-	0x1576: 0x1fe2, 0x1577: 0xaf2a, 0x1578: 0xaf3a, 0x1579: 0x7e2d, 0x157a: 0x7e4d, 0x157b: 0x7e6d,
-	0x157c: 0x7e2d, 0x157d: 0x7e8d, 0x157e: 0x7ead, 0x157f: 0x7e8d,
-	// Block 0x56, offset 0x1580
-	0x1580: 0x7ecd, 0x1581: 0x7eed, 0x1582: 0x7f0d, 0x1583: 0x7eed, 0x1584: 0x7f2d, 0x1585: 0x0018,
-	0x1586: 0x0018, 0x1587: 0xaf4a, 0x1588: 0xaf5a, 0x1589: 0x7f4e, 0x158a: 0x7f6e, 0x158b: 0x7f8e,
-	0x158c: 0x7fae, 0x158d: 0xaf1a, 0x158e: 0xaf1a, 0x158f: 0xaf1a, 0x1590: 0xaeda, 0x1591: 0x7fcd,
-	0x1592: 0x0040, 0x1593: 0x0040, 0x1594: 0x03c2, 0x1595: 0xaeea, 0x1596: 0xaf0a, 0x1597: 0xaefa,
-	0x1598: 0x7fed, 0x1599: 0x1fd2, 0x159a: 0x1fe2, 0x159b: 0xaf2a, 0x159c: 0xaf3a, 0x159d: 0x7ecd,
-	0x159e: 0x7f2d, 0x159f: 0xaf6a, 0x15a0: 0xaf7a, 0x15a1: 0xaf8a, 0x15a2: 0x1fb2, 0x15a3: 0xaf99,
-	0x15a4: 0xafaa, 0x15a5: 0xafba, 0x15a6: 0x1fc2, 0x15a7: 0x0040, 0x15a8: 0xafca, 0x15a9: 0xafda,
-	0x15aa: 0xafea, 0x15ab: 0xaffa, 0x15ac: 0x0040, 0x15ad: 0x0040, 0x15ae: 0x0040, 0x15af: 0x0040,
-	0x15b0: 0x800e, 0x15b1: 0xb009, 0x15b2: 0x802e, 0x15b3: 0x0808, 0x15b4: 0x804e, 0x15b5: 0x0040,
-	0x15b6: 0x806e, 0x15b7: 0xb031, 0x15b8: 0x808e, 0x15b9: 0xb059, 0x15ba: 0x80ae, 0x15bb: 0xb081,
-	0x15bc: 0x80ce, 0x15bd: 0xb0a9, 0x15be: 0x80ee, 0x15bf: 0xb0d1,
-	// Block 0x57, offset 0x15c0
-	0x15c0: 0xb0f9, 0x15c1: 0xb111, 0x15c2: 0xb111, 0x15c3: 0xb129, 0x15c4: 0xb129, 0x15c5: 0xb141,
-	0x15c6: 0xb141, 0x15c7: 0xb159, 0x15c8: 0xb159, 0x15c9: 0xb171, 0x15ca: 0xb171, 0x15cb: 0xb171,
-	0x15cc: 0xb171, 0x15cd: 0xb189, 0x15ce: 0xb189, 0x15cf: 0xb1a1, 0x15d0: 0xb1a1, 0x15d1: 0xb1a1,
-	0x15d2: 0xb1a1, 0x15d3: 0xb1b9, 0x15d4: 0xb1b9, 0x15d5: 0xb1d1, 0x15d6: 0xb1d1, 0x15d7: 0xb1d1,
-	0x15d8: 0xb1d1, 0x15d9: 0xb1e9, 0x15da: 0xb1e9, 0x15db: 0xb1e9, 0x15dc: 0xb1e9, 0x15dd: 0xb201,
-	0x15de: 0xb201, 0x15df: 0xb201, 0x15e0: 0xb201, 0x15e1: 0xb219, 0x15e2: 0xb219, 0x15e3: 0xb219,
-	0x15e4: 0xb219, 0x15e5: 0xb231, 0x15e6: 0xb231, 0x15e7: 0xb231, 0x15e8: 0xb231, 0x15e9: 0xb249,
-	0x15ea: 0xb249, 0x15eb: 0xb261, 0x15ec: 0xb261, 0x15ed: 0xb279, 0x15ee: 0xb279, 0x15ef: 0xb291,
-	0x15f0: 0xb291, 0x15f1: 0xb2a9, 0x15f2: 0xb2a9, 0x15f3: 0xb2a9, 0x15f4: 0xb2a9, 0x15f5: 0xb2c1,
-	0x15f6: 0xb2c1, 0x15f7: 0xb2c1, 0x15f8: 0xb2c1, 0x15f9: 0xb2d9, 0x15fa: 0xb2d9, 0x15fb: 0xb2d9,
-	0x15fc: 0xb2d9, 0x15fd: 0xb2f1, 0x15fe: 0xb2f1, 0x15ff: 0xb2f1,
-	// Block 0x58, offset 0x1600
-	0x1600: 0xb2f1, 0x1601: 0xb309, 0x1602: 0xb309, 0x1603: 0xb309, 0x1604: 0xb309, 0x1605: 0xb321,
-	0x1606: 0xb321, 0x1607: 0xb321, 0x1608: 0xb321, 0x1609: 0xb339, 0x160a: 0xb339, 0x160b: 0xb339,
-	0x160c: 0xb339, 0x160d: 0xb351, 0x160e: 0xb351, 0x160f: 0xb351, 0x1610: 0xb351, 0x1611: 0xb369,
-	0x1612: 0xb369, 0x1613: 0xb369, 0x1614: 0xb369, 0x1615: 0xb381, 0x1616: 0xb381, 0x1617: 0xb381,
-	0x1618: 0xb381, 0x1619: 0xb399, 0x161a: 0xb399, 0x161b: 0xb399, 0x161c: 0xb399, 0x161d: 0xb3b1,
-	0x161e: 0xb3b1, 0x161f: 0xb3b1, 0x1620: 0xb3b1, 0x1621: 0xb3c9, 0x1622: 0xb3c9, 0x1623: 0xb3c9,
-	0x1624: 0xb3c9, 0x1625: 0xb3e1, 0x1626: 0xb3e1, 0x1627: 0xb3e1, 0x1628: 0xb3e1, 0x1629: 0xb3f9,
-	0x162a: 0xb3f9, 0x162b: 0xb3f9, 0x162c: 0xb3f9, 0x162d: 0xb411, 0x162e: 0xb411, 0x162f: 0x7ab1,
-	0x1630: 0x7ab1, 0x1631: 0xb429, 0x1632: 0xb429, 0x1633: 0xb429, 0x1634: 0xb429, 0x1635: 0xb441,
-	0x1636: 0xb441, 0x1637: 0xb469, 0x1638: 0xb469, 0x1639: 0xb491, 0x163a: 0xb491, 0x163b: 0xb4b9,
-	0x163c: 0xb4b9, 0x163d: 0x0040, 0x163e: 0x0040, 0x163f: 0x03c0,
-	// Block 0x59, offset 0x1640
-	0x1640: 0x0040, 0x1641: 0xaefa, 0x1642: 0xb4e2, 0x1643: 0xaf6a, 0x1644: 0xafda, 0x1645: 0xafea,
-	0x1646: 0xaf7a, 0x1647: 0xb4f2, 0x1648: 0x1fd2, 0x1649: 0x1fe2, 0x164a: 0xaf8a, 0x164b: 0x1fb2,
-	0x164c: 0xaeda, 0x164d: 0xaf99, 0x164e: 0x29d1, 0x164f: 0xb502, 0x1650: 0x1f41, 0x1651: 0x00c9,
-	0x1652: 0x0069, 0x1653: 0x0079, 0x1654: 0x1f51, 0x1655: 0x1f61, 0x1656: 0x1f71, 0x1657: 0x1f81,
-	0x1658: 0x1f91, 0x1659: 0x1fa1, 0x165a: 0xaeea, 0x165b: 0x03c2, 0x165c: 0xafaa, 0x165d: 0x1fc2,
-	0x165e: 0xafba, 0x165f: 0xaf0a, 0x1660: 0xaffa, 0x1661: 0x0039, 0x1662: 0x0ee9, 0x1663: 0x1159,
-	0x1664: 0x0ef9, 0x1665: 0x0f09, 0x1666: 0x1199, 0x1667: 0x0f31, 0x1668: 0x0249, 0x1669: 0x0f41,
-	0x166a: 0x0259, 0x166b: 0x0f51, 0x166c: 0x0359, 0x166d: 0x0f61, 0x166e: 0x0f71, 0x166f: 0x00d9,
-	0x1670: 0x0f99, 0x1671: 0x2039, 0x1672: 0x0269, 0x1673: 0x01d9, 0x1674: 0x0fa9, 0x1675: 0x0fb9,
-	0x1676: 0x1089, 0x1677: 0x0279, 0x1678: 0x0369, 0x1679: 0x0289, 0x167a: 0x13d1, 0x167b: 0xaf4a,
-	0x167c: 0xafca, 0x167d: 0xaf5a, 0x167e: 0xb512, 0x167f: 0xaf1a,
-	// Block 0x5a, offset 0x1680
-	0x1680: 0x1caa, 0x1681: 0x0039, 0x1682: 0x0ee9, 0x1683: 0x1159, 0x1684: 0x0ef9, 0x1685: 0x0f09,
-	0x1686: 0x1199, 0x1687: 0x0f31, 0x1688: 0x0249, 0x1689: 0x0f41, 0x168a: 0x0259, 0x168b: 0x0f51,
-	0x168c: 0x0359, 0x168d: 0x0f61, 0x168e: 0x0f71, 0x168f: 0x00d9, 0x1690: 0x0f99, 0x1691: 0x2039,
-	0x1692: 0x0269, 0x1693: 0x01d9, 0x1694: 0x0fa9, 0x1695: 0x0fb9, 0x1696: 0x1089, 0x1697: 0x0279,
-	0x1698: 0x0369, 0x1699: 0x0289, 0x169a: 0x13d1, 0x169b: 0xaf2a, 0x169c: 0xb522, 0x169d: 0xaf3a,
-	0x169e: 0xb532, 0x169f: 0x810d, 0x16a0: 0x812d, 0x16a1: 0x29d1, 0x16a2: 0x814d, 0x16a3: 0x814d,
-	0x16a4: 0x816d, 0x16a5: 0x818d, 0x16a6: 0x81ad, 0x16a7: 0x81cd, 0x16a8: 0x81ed, 0x16a9: 0x820d,
-	0x16aa: 0x822d, 0x16ab: 0x824d, 0x16ac: 0x826d, 0x16ad: 0x828d, 0x16ae: 0x82ad, 0x16af: 0x82cd,
-	0x16b0: 0x82ed, 0x16b1: 0x830d, 0x16b2: 0x832d, 0x16b3: 0x834d, 0x16b4: 0x836d, 0x16b5: 0x838d,
-	0x16b6: 0x83ad, 0x16b7: 0x83cd, 0x16b8: 0x83ed, 0x16b9: 0x840d, 0x16ba: 0x842d, 0x16bb: 0x844d,
-	0x16bc: 0x81ed, 0x16bd: 0x846d, 0x16be: 0x848d, 0x16bf: 0x824d,
-	// Block 0x5b, offset 0x16c0
-	0x16c0: 0x84ad, 0x16c1: 0x84cd, 0x16c2: 0x84ed, 0x16c3: 0x850d, 0x16c4: 0x852d, 0x16c5: 0x854d,
-	0x16c6: 0x856d, 0x16c7: 0x858d, 0x16c8: 0x850d, 0x16c9: 0x85ad, 0x16ca: 0x850d, 0x16cb: 0x85cd,
-	0x16cc: 0x85cd, 0x16cd: 0x85ed, 0x16ce: 0x85ed, 0x16cf: 0x860d, 0x16d0: 0x854d, 0x16d1: 0x862d,
-	0x16d2: 0x864d, 0x16d3: 0x862d, 0x16d4: 0x866d, 0x16d5: 0x864d, 0x16d6: 0x868d, 0x16d7: 0x868d,
-	0x16d8: 0x86ad, 0x16d9: 0x86ad, 0x16da: 0x86cd, 0x16db: 0x86cd, 0x16dc: 0x864d, 0x16dd: 0x814d,
-	0x16de: 0x86ed, 0x16df: 0x870d, 0x16e0: 0x0040, 0x16e1: 0x872d, 0x16e2: 0x874d, 0x16e3: 0x876d,
-	0x16e4: 0x878d, 0x16e5: 0x876d, 0x16e6: 0x87ad, 0x16e7: 0x87cd, 0x16e8: 0x87ed, 0x16e9: 0x87ed,
-	0x16ea: 0x880d, 0x16eb: 0x880d, 0x16ec: 0x882d, 0x16ed: 0x882d, 0x16ee: 0x880d, 0x16ef: 0x880d,
-	0x16f0: 0x884d, 0x16f1: 0x886d, 0x16f2: 0x888d, 0x16f3: 0x88ad, 0x16f4: 0x88cd, 0x16f5: 0x88ed,
-	0x16f6: 0x88ed, 0x16f7: 0x88ed, 0x16f8: 0x890d, 0x16f9: 0x890d, 0x16fa: 0x890d, 0x16fb: 0x890d,
-	0x16fc: 0x87ed, 0x16fd: 0x87ed, 0x16fe: 0x87ed, 0x16ff: 0x0040,
-	// Block 0x5c, offset 0x1700
-	0x1700: 0x0040, 0x1701: 0x0040, 0x1702: 0x874d, 0x1703: 0x872d, 0x1704: 0x892d, 0x1705: 0x872d,
-	0x1706: 0x874d, 0x1707: 0x872d, 0x1708: 0x0040, 0x1709: 0x0040, 0x170a: 0x894d, 0x170b: 0x874d,
-	0x170c: 0x896d, 0x170d: 0x892d, 0x170e: 0x896d, 0x170f: 0x874d, 0x1710: 0x0040, 0x1711: 0x0040,
-	0x1712: 0x898d, 0x1713: 0x89ad, 0x1714: 0x88ad, 0x1715: 0x896d, 0x1716: 0x892d, 0x1717: 0x896d,
-	0x1718: 0x0040, 0x1719: 0x0040, 0x171a: 0x89cd, 0x171b: 0x89ed, 0x171c: 0x89cd, 0x171d: 0x0040,
-	0x171e: 0x0040, 0x171f: 0x0040, 0x1720: 0xb541, 0x1721: 0xb559, 0x1722: 0xb571, 0x1723: 0x8a0e,
-	0x1724: 0xb589, 0x1725: 0xb5a1, 0x1726: 0x8a2d, 0x1727: 0x0040, 0x1728: 0x8a4d, 0x1729: 0x8a6d,
-	0x172a: 0x8a8d, 0x172b: 0x8a6d, 0x172c: 0x8aad, 0x172d: 0x8acd, 0x172e: 0x8aed, 0x172f: 0x0040,
-	0x1730: 0x0040, 0x1731: 0x0040, 0x1732: 0x0040, 0x1733: 0x0040, 0x1734: 0x0040, 0x1735: 0x0040,
-	0x1736: 0x0040, 0x1737: 0x0040, 0x1738: 0x0040, 0x1739: 0x0340, 0x173a: 0x0340, 0x173b: 0x0340,
-	0x173c: 0x0040, 0x173d: 0x0040, 0x173e: 0x0040, 0x173f: 0x0040,
-	// Block 0x5d, offset 0x1740
-	0x1740: 0x0a08, 0x1741: 0x0a08, 0x1742: 0x0a08, 0x1743: 0x0a08, 0x1744: 0x0a08, 0x1745: 0x0c08,
-	0x1746: 0x0808, 0x1747: 0x0c08, 0x1748: 0x0818, 0x1749: 0x0c08, 0x174a: 0x0c08, 0x174b: 0x0808,
-	0x174c: 0x0808, 0x174d: 0x0908, 0x174e: 0x0c08, 0x174f: 0x0c08, 0x1750: 0x0c08, 0x1751: 0x0c08,
-	0x1752: 0x0c08, 0x1753: 0x0a08, 0x1754: 0x0a08, 0x1755: 0x0a08, 0x1756: 0x0a08, 0x1757: 0x0908,
-	0x1758: 0x0a08, 0x1759: 0x0a08, 0x175a: 0x0a08, 0x175b: 0x0a08, 0x175c: 0x0a08, 0x175d: 0x0c08,
-	0x175e: 0x0a08, 0x175f: 0x0a08, 0x1760: 0x0a08, 0x1761: 0x0c08, 0x1762: 0x0808, 0x1763: 0x0808,
-	0x1764: 0x0c08, 0x1765: 0x3308, 0x1766: 0x3308, 0x1767: 0x0040, 0x1768: 0x0040, 0x1769: 0x0040,
-	0x176a: 0x0040, 0x176b: 0x0a18, 0x176c: 0x0a18, 0x176d: 0x0a18, 0x176e: 0x0a18, 0x176f: 0x0c18,
-	0x1770: 0x0818, 0x1771: 0x0818, 0x1772: 0x0818, 0x1773: 0x0818, 0x1774: 0x0818, 0x1775: 0x0818,
-	0x1776: 0x0818, 0x1777: 0x0040, 0x1778: 0x0040, 0x1779: 0x0040, 0x177a: 0x0040, 0x177b: 0x0040,
-	0x177c: 0x0040, 0x177d: 0x0040, 0x177e: 0x0040, 0x177f: 0x0040,
-	// Block 0x5e, offset 0x1780
-	0x1780: 0x0a08, 0x1781: 0x0c08, 0x1782: 0x0a08, 0x1783: 0x0c08, 0x1784: 0x0c08, 0x1785: 0x0c08,
-	0x1786: 0x0a08, 0x1787: 0x0a08, 0x1788: 0x0a08, 0x1789: 0x0c08, 0x178a: 0x0a08, 0x178b: 0x0a08,
-	0x178c: 0x0c08, 0x178d: 0x0a08, 0x178e: 0x0c08, 0x178f: 0x0c08, 0x1790: 0x0a08, 0x1791: 0x0c08,
-	0x1792: 0x0040, 0x1793: 0x0040, 0x1794: 0x0040, 0x1795: 0x0040, 0x1796: 0x0040, 0x1797: 0x0040,
-	0x1798: 0x0040, 0x1799: 0x0818, 0x179a: 0x0818, 0x179b: 0x0818, 0x179c: 0x0818, 0x179d: 0x0040,
-	0x179e: 0x0040, 0x179f: 0x0040, 0x17a0: 0x0040, 0x17a1: 0x0040, 0x17a2: 0x0040, 0x17a3: 0x0040,
-	0x17a4: 0x0040, 0x17a5: 0x0040, 0x17a6: 0x0040, 0x17a7: 0x0040, 0x17a8: 0x0040, 0x17a9: 0x0c18,
-	0x17aa: 0x0c18, 0x17ab: 0x0c18, 0x17ac: 0x0c18, 0x17ad: 0x0a18, 0x17ae: 0x0a18, 0x17af: 0x0818,
-	0x17b0: 0x0040, 0x17b1: 0x0040, 0x17b2: 0x0040, 0x17b3: 0x0040, 0x17b4: 0x0040, 0x17b5: 0x0040,
-	0x17b6: 0x0040, 0x17b7: 0x0040, 0x17b8: 0x0040, 0x17b9: 0x0040, 0x17ba: 0x0040, 0x17bb: 0x0040,
-	0x17bc: 0x0040, 0x17bd: 0x0040, 0x17be: 0x0040, 0x17bf: 0x0040,
-	// Block 0x5f, offset 0x17c0
-	0x17c0: 0x3308, 0x17c1: 0x3308, 0x17c2: 0x3008, 0x17c3: 0x3008, 0x17c4: 0x0040, 0x17c5: 0x0008,
-	0x17c6: 0x0008, 0x17c7: 0x0008, 0x17c8: 0x0008, 0x17c9: 0x0008, 0x17ca: 0x0008, 0x17cb: 0x0008,
-	0x17cc: 0x0008, 0x17cd: 0x0040, 0x17ce: 0x0040, 0x17cf: 0x0008, 0x17d0: 0x0008, 0x17d1: 0x0040,
-	0x17d2: 0x0040, 0x17d3: 0x0008, 0x17d4: 0x0008, 0x17d5: 0x0008, 0x17d6: 0x0008, 0x17d7: 0x0008,
-	0x17d8: 0x0008, 0x17d9: 0x0008, 0x17da: 0x0008, 0x17db: 0x0008, 0x17dc: 0x0008, 0x17dd: 0x0008,
-	0x17de: 0x0008, 0x17df: 0x0008, 0x17e0: 0x0008, 0x17e1: 0x0008, 0x17e2: 0x0008, 0x17e3: 0x0008,
-	0x17e4: 0x0008, 0x17e5: 0x0008, 0x17e6: 0x0008, 0x17e7: 0x0008, 0x17e8: 0x0008, 0x17e9: 0x0040,
-	0x17ea: 0x0008, 0x17eb: 0x0008, 0x17ec: 0x0008, 0x17ed: 0x0008, 0x17ee: 0x0008, 0x17ef: 0x0008,
-	0x17f0: 0x0008, 0x17f1: 0x0040, 0x17f2: 0x0008, 0x17f3: 0x0008, 0x17f4: 0x0040, 0x17f5: 0x0008,
-	0x17f6: 0x0008, 0x17f7: 0x0008, 0x17f8: 0x0008, 0x17f9: 0x0008, 0x17fa: 0x0040, 0x17fb: 0x3308,
-	0x17fc: 0x3308, 0x17fd: 0x0008, 0x17fe: 0x3008, 0x17ff: 0x3008,
-	// Block 0x60, offset 0x1800
-	0x1800: 0x3308, 0x1801: 0x3008, 0x1802: 0x3008, 0x1803: 0x3008, 0x1804: 0x3008, 0x1805: 0x0040,
-	0x1806: 0x0040, 0x1807: 0x3008, 0x1808: 0x3008, 0x1809: 0x0040, 0x180a: 0x0040, 0x180b: 0x3008,
-	0x180c: 0x3008, 0x180d: 0x3808, 0x180e: 0x0040, 0x180f: 0x0040, 0x1810: 0x0008, 0x1811: 0x0040,
-	0x1812: 0x0040, 0x1813: 0x0040, 0x1814: 0x0040, 0x1815: 0x0040, 0x1816: 0x0040, 0x1817: 0x3008,
-	0x1818: 0x0040, 0x1819: 0x0040, 0x181a: 0x0040, 0x181b: 0x0040, 0x181c: 0x0040, 0x181d: 0x0008,
-	0x181e: 0x0008, 0x181f: 0x0008, 0x1820: 0x0008, 0x1821: 0x0008, 0x1822: 0x3008, 0x1823: 0x3008,
-	0x1824: 0x0040, 0x1825: 0x0040, 0x1826: 0x3308, 0x1827: 0x3308, 0x1828: 0x3308, 0x1829: 0x3308,
-	0x182a: 0x3308, 0x182b: 0x3308, 0x182c: 0x3308, 0x182d: 0x0040, 0x182e: 0x0040, 0x182f: 0x0040,
-	0x1830: 0x3308, 0x1831: 0x3308, 0x1832: 0x3308, 0x1833: 0x3308, 0x1834: 0x3308, 0x1835: 0x0040,
-	0x1836: 0x0040, 0x1837: 0x0040, 0x1838: 0x0040, 0x1839: 0x0040, 0x183a: 0x0040, 0x183b: 0x0040,
-	0x183c: 0x0040, 0x183d: 0x0040, 0x183e: 0x0040, 0x183f: 0x0040,
-	// Block 0x61, offset 0x1840
-	0x1840: 0x0039, 0x1841: 0x0ee9, 0x1842: 0x1159, 0x1843: 0x0ef9, 0x1844: 0x0f09, 0x1845: 0x1199,
-	0x1846: 0x0f31, 0x1847: 0x0249, 0x1848: 0x0f41, 0x1849: 0x0259, 0x184a: 0x0f51, 0x184b: 0x0359,
-	0x184c: 0x0f61, 0x184d: 0x0f71, 0x184e: 0x00d9, 0x184f: 0x0f99, 0x1850: 0x2039, 0x1851: 0x0269,
-	0x1852: 0x01d9, 0x1853: 0x0fa9, 0x1854: 0x0fb9, 0x1855: 0x1089, 0x1856: 0x0279, 0x1857: 0x0369,
-	0x1858: 0x0289, 0x1859: 0x13d1, 0x185a: 0x0039, 0x185b: 0x0ee9, 0x185c: 0x1159, 0x185d: 0x0ef9,
-	0x185e: 0x0f09, 0x185f: 0x1199, 0x1860: 0x0f31, 0x1861: 0x0249, 0x1862: 0x0f41, 0x1863: 0x0259,
-	0x1864: 0x0f51, 0x1865: 0x0359, 0x1866: 0x0f61, 0x1867: 0x0f71, 0x1868: 0x00d9, 0x1869: 0x0f99,
-	0x186a: 0x2039, 0x186b: 0x0269, 0x186c: 0x01d9, 0x186d: 0x0fa9, 0x186e: 0x0fb9, 0x186f: 0x1089,
-	0x1870: 0x0279, 0x1871: 0x0369, 0x1872: 0x0289, 0x1873: 0x13d1, 0x1874: 0x0039, 0x1875: 0x0ee9,
-	0x1876: 0x1159, 0x1877: 0x0ef9, 0x1878: 0x0f09, 0x1879: 0x1199, 0x187a: 0x0f31, 0x187b: 0x0249,
-	0x187c: 0x0f41, 0x187d: 0x0259, 0x187e: 0x0f51, 0x187f: 0x0359,
-	// Block 0x62, offset 0x1880
-	0x1880: 0x0f61, 0x1881: 0x0f71, 0x1882: 0x00d9, 0x1883: 0x0f99, 0x1884: 0x2039, 0x1885: 0x0269,
-	0x1886: 0x01d9, 0x1887: 0x0fa9, 0x1888: 0x0fb9, 0x1889: 0x1089, 0x188a: 0x0279, 0x188b: 0x0369,
-	0x188c: 0x0289, 0x188d: 0x13d1, 0x188e: 0x0039, 0x188f: 0x0ee9, 0x1890: 0x1159, 0x1891: 0x0ef9,
-	0x1892: 0x0f09, 0x1893: 0x1199, 0x1894: 0x0f31, 0x1895: 0x0040, 0x1896: 0x0f41, 0x1897: 0x0259,
-	0x1898: 0x0f51, 0x1899: 0x0359, 0x189a: 0x0f61, 0x189b: 0x0f71, 0x189c: 0x00d9, 0x189d: 0x0f99,
-	0x189e: 0x2039, 0x189f: 0x0269, 0x18a0: 0x01d9, 0x18a1: 0x0fa9, 0x18a2: 0x0fb9, 0x18a3: 0x1089,
-	0x18a4: 0x0279, 0x18a5: 0x0369, 0x18a6: 0x0289, 0x18a7: 0x13d1, 0x18a8: 0x0039, 0x18a9: 0x0ee9,
-	0x18aa: 0x1159, 0x18ab: 0x0ef9, 0x18ac: 0x0f09, 0x18ad: 0x1199, 0x18ae: 0x0f31, 0x18af: 0x0249,
-	0x18b0: 0x0f41, 0x18b1: 0x0259, 0x18b2: 0x0f51, 0x18b3: 0x0359, 0x18b4: 0x0f61, 0x18b5: 0x0f71,
-	0x18b6: 0x00d9, 0x18b7: 0x0f99, 0x18b8: 0x2039, 0x18b9: 0x0269, 0x18ba: 0x01d9, 0x18bb: 0x0fa9,
-	0x18bc: 0x0fb9, 0x18bd: 0x1089, 0x18be: 0x0279, 0x18bf: 0x0369,
-	// Block 0x63, offset 0x18c0
-	0x18c0: 0x0289, 0x18c1: 0x13d1, 0x18c2: 0x0039, 0x18c3: 0x0ee9, 0x18c4: 0x1159, 0x18c5: 0x0ef9,
-	0x18c6: 0x0f09, 0x18c7: 0x1199, 0x18c8: 0x0f31, 0x18c9: 0x0249, 0x18ca: 0x0f41, 0x18cb: 0x0259,
-	0x18cc: 0x0f51, 0x18cd: 0x0359, 0x18ce: 0x0f61, 0x18cf: 0x0f71, 0x18d0: 0x00d9, 0x18d1: 0x0f99,
-	0x18d2: 0x2039, 0x18d3: 0x0269, 0x18d4: 0x01d9, 0x18d5: 0x0fa9, 0x18d6: 0x0fb9, 0x18d7: 0x1089,
-	0x18d8: 0x0279, 0x18d9: 0x0369, 0x18da: 0x0289, 0x18db: 0x13d1, 0x18dc: 0x0039, 0x18dd: 0x0040,
-	0x18de: 0x1159, 0x18df: 0x0ef9, 0x18e0: 0x0040, 0x18e1: 0x0040, 0x18e2: 0x0f31, 0x18e3: 0x0040,
-	0x18e4: 0x0040, 0x18e5: 0x0259, 0x18e6: 0x0f51, 0x18e7: 0x0040, 0x18e8: 0x0040, 0x18e9: 0x0f71,
-	0x18ea: 0x00d9, 0x18eb: 0x0f99, 0x18ec: 0x2039, 0x18ed: 0x0040, 0x18ee: 0x01d9, 0x18ef: 0x0fa9,
-	0x18f0: 0x0fb9, 0x18f1: 0x1089, 0x18f2: 0x0279, 0x18f3: 0x0369, 0x18f4: 0x0289, 0x18f5: 0x13d1,
-	0x18f6: 0x0039, 0x18f7: 0x0ee9, 0x18f8: 0x1159, 0x18f9: 0x0ef9, 0x18fa: 0x0040, 0x18fb: 0x1199,
-	0x18fc: 0x0040, 0x18fd: 0x0249, 0x18fe: 0x0f41, 0x18ff: 0x0259,
-	// Block 0x64, offset 0x1900
-	0x1900: 0x0f51, 0x1901: 0x0359, 0x1902: 0x0f61, 0x1903: 0x0f71, 0x1904: 0x0040, 0x1905: 0x0f99,
-	0x1906: 0x2039, 0x1907: 0x0269, 0x1908: 0x01d9, 0x1909: 0x0fa9, 0x190a: 0x0fb9, 0x190b: 0x1089,
-	0x190c: 0x0279, 0x190d: 0x0369, 0x190e: 0x0289, 0x190f: 0x13d1, 0x1910: 0x0039, 0x1911: 0x0ee9,
-	0x1912: 0x1159, 0x1913: 0x0ef9, 0x1914: 0x0f09, 0x1915: 0x1199, 0x1916: 0x0f31, 0x1917: 0x0249,
-	0x1918: 0x0f41, 0x1919: 0x0259, 0x191a: 0x0f51, 0x191b: 0x0359, 0x191c: 0x0f61, 0x191d: 0x0f71,
-	0x191e: 0x00d9, 0x191f: 0x0f99, 0x1920: 0x2039, 0x1921: 0x0269, 0x1922: 0x01d9, 0x1923: 0x0fa9,
-	0x1924: 0x0fb9, 0x1925: 0x1089, 0x1926: 0x0279, 0x1927: 0x0369, 0x1928: 0x0289, 0x1929: 0x13d1,
-	0x192a: 0x0039, 0x192b: 0x0ee9, 0x192c: 0x1159, 0x192d: 0x0ef9, 0x192e: 0x0f09, 0x192f: 0x1199,
-	0x1930: 0x0f31, 0x1931: 0x0249, 0x1932: 0x0f41, 0x1933: 0x0259, 0x1934: 0x0f51, 0x1935: 0x0359,
-	0x1936: 0x0f61, 0x1937: 0x0f71, 0x1938: 0x00d9, 0x1939: 0x0f99, 0x193a: 0x2039, 0x193b: 0x0269,
-	0x193c: 0x01d9, 0x193d: 0x0fa9, 0x193e: 0x0fb9, 0x193f: 0x1089,
-	// Block 0x65, offset 0x1940
-	0x1940: 0x0279, 0x1941: 0x0369, 0x1942: 0x0289, 0x1943: 0x13d1, 0x1944: 0x0039, 0x1945: 0x0ee9,
-	0x1946: 0x0040, 0x1947: 0x0ef9, 0x1948: 0x0f09, 0x1949: 0x1199, 0x194a: 0x0f31, 0x194b: 0x0040,
-	0x194c: 0x0040, 0x194d: 0x0259, 0x194e: 0x0f51, 0x194f: 0x0359, 0x1950: 0x0f61, 0x1951: 0x0f71,
-	0x1952: 0x00d9, 0x1953: 0x0f99, 0x1954: 0x2039, 0x1955: 0x0040, 0x1956: 0x01d9, 0x1957: 0x0fa9,
-	0x1958: 0x0fb9, 0x1959: 0x1089, 0x195a: 0x0279, 0x195b: 0x0369, 0x195c: 0x0289, 0x195d: 0x0040,
-	0x195e: 0x0039, 0x195f: 0x0ee9, 0x1960: 0x1159, 0x1961: 0x0ef9, 0x1962: 0x0f09, 0x1963: 0x1199,
-	0x1964: 0x0f31, 0x1965: 0x0249, 0x1966: 0x0f41, 0x1967: 0x0259, 0x1968: 0x0f51, 0x1969: 0x0359,
-	0x196a: 0x0f61, 0x196b: 0x0f71, 0x196c: 0x00d9, 0x196d: 0x0f99, 0x196e: 0x2039, 0x196f: 0x0269,
-	0x1970: 0x01d9, 0x1971: 0x0fa9, 0x1972: 0x0fb9, 0x1973: 0x1089, 0x1974: 0x0279, 0x1975: 0x0369,
-	0x1976: 0x0289, 0x1977: 0x13d1, 0x1978: 0x0039, 0x1979: 0x0ee9, 0x197a: 0x0040, 0x197b: 0x0ef9,
-	0x197c: 0x0f09, 0x197d: 0x1199, 0x197e: 0x0f31, 0x197f: 0x0040,
-	// Block 0x66, offset 0x1980
-	0x1980: 0x0f41, 0x1981: 0x0259, 0x1982: 0x0f51, 0x1983: 0x0359, 0x1984: 0x0f61, 0x1985: 0x0040,
-	0x1986: 0x00d9, 0x1987: 0x0040, 0x1988: 0x0040, 0x1989: 0x0040, 0x198a: 0x01d9, 0x198b: 0x0fa9,
-	0x198c: 0x0fb9, 0x198d: 0x1089, 0x198e: 0x0279, 0x198f: 0x0369, 0x1990: 0x0289, 0x1991: 0x0040,
-	0x1992: 0x0039, 0x1993: 0x0ee9, 0x1994: 0x1159, 0x1995: 0x0ef9, 0x1996: 0x0f09, 0x1997: 0x1199,
-	0x1998: 0x0f31, 0x1999: 0x0249, 0x199a: 0x0f41, 0x199b: 0x0259, 0x199c: 0x0f51, 0x199d: 0x0359,
-	0x199e: 0x0f61, 0x199f: 0x0f71, 0x19a0: 0x00d9, 0x19a1: 0x0f99, 0x19a2: 0x2039, 0x19a3: 0x0269,
-	0x19a4: 0x01d9, 0x19a5: 0x0fa9, 0x19a6: 0x0fb9, 0x19a7: 0x1089, 0x19a8: 0x0279, 0x19a9: 0x0369,
-	0x19aa: 0x0289, 0x19ab: 0x13d1, 0x19ac: 0x0039, 0x19ad: 0x0ee9, 0x19ae: 0x1159, 0x19af: 0x0ef9,
-	0x19b0: 0x0f09, 0x19b1: 0x1199, 0x19b2: 0x0f31, 0x19b3: 0x0249, 0x19b4: 0x0f41, 0x19b5: 0x0259,
-	0x19b6: 0x0f51, 0x19b7: 0x0359, 0x19b8: 0x0f61, 0x19b9: 0x0f71, 0x19ba: 0x00d9, 0x19bb: 0x0f99,
-	0x19bc: 0x2039, 0x19bd: 0x0269, 0x19be: 0x01d9, 0x19bf: 0x0fa9,
-	// Block 0x67, offset 0x19c0
-	0x19c0: 0x0fb9, 0x19c1: 0x1089, 0x19c2: 0x0279, 0x19c3: 0x0369, 0x19c4: 0x0289, 0x19c5: 0x13d1,
-	0x19c6: 0x0039, 0x19c7: 0x0ee9, 0x19c8: 0x1159, 0x19c9: 0x0ef9, 0x19ca: 0x0f09, 0x19cb: 0x1199,
-	0x19cc: 0x0f31, 0x19cd: 0x0249, 0x19ce: 0x0f41, 0x19cf: 0x0259, 0x19d0: 0x0f51, 0x19d1: 0x0359,
-	0x19d2: 0x0f61, 0x19d3: 0x0f71, 0x19d4: 0x00d9, 0x19d5: 0x0f99, 0x19d6: 0x2039, 0x19d7: 0x0269,
-	0x19d8: 0x01d9, 0x19d9: 0x0fa9, 0x19da: 0x0fb9, 0x19db: 0x1089, 0x19dc: 0x0279, 0x19dd: 0x0369,
-	0x19de: 0x0289, 0x19df: 0x13d1, 0x19e0: 0x0039, 0x19e1: 0x0ee9, 0x19e2: 0x1159, 0x19e3: 0x0ef9,
-	0x19e4: 0x0f09, 0x19e5: 0x1199, 0x19e6: 0x0f31, 0x19e7: 0x0249, 0x19e8: 0x0f41, 0x19e9: 0x0259,
-	0x19ea: 0x0f51, 0x19eb: 0x0359, 0x19ec: 0x0f61, 0x19ed: 0x0f71, 0x19ee: 0x00d9, 0x19ef: 0x0f99,
-	0x19f0: 0x2039, 0x19f1: 0x0269, 0x19f2: 0x01d9, 0x19f3: 0x0fa9, 0x19f4: 0x0fb9, 0x19f5: 0x1089,
-	0x19f6: 0x0279, 0x19f7: 0x0369, 0x19f8: 0x0289, 0x19f9: 0x13d1, 0x19fa: 0x0039, 0x19fb: 0x0ee9,
-	0x19fc: 0x1159, 0x19fd: 0x0ef9, 0x19fe: 0x0f09, 0x19ff: 0x1199,
-	// Block 0x68, offset 0x1a00
-	0x1a00: 0x0f31, 0x1a01: 0x0249, 0x1a02: 0x0f41, 0x1a03: 0x0259, 0x1a04: 0x0f51, 0x1a05: 0x0359,
-	0x1a06: 0x0f61, 0x1a07: 0x0f71, 0x1a08: 0x00d9, 0x1a09: 0x0f99, 0x1a0a: 0x2039, 0x1a0b: 0x0269,
-	0x1a0c: 0x01d9, 0x1a0d: 0x0fa9, 0x1a0e: 0x0fb9, 0x1a0f: 0x1089, 0x1a10: 0x0279, 0x1a11: 0x0369,
-	0x1a12: 0x0289, 0x1a13: 0x13d1, 0x1a14: 0x0039, 0x1a15: 0x0ee9, 0x1a16: 0x1159, 0x1a17: 0x0ef9,
-	0x1a18: 0x0f09, 0x1a19: 0x1199, 0x1a1a: 0x0f31, 0x1a1b: 0x0249, 0x1a1c: 0x0f41, 0x1a1d: 0x0259,
-	0x1a1e: 0x0f51, 0x1a1f: 0x0359, 0x1a20: 0x0f61, 0x1a21: 0x0f71, 0x1a22: 0x00d9, 0x1a23: 0x0f99,
-	0x1a24: 0x2039, 0x1a25: 0x0269, 0x1a26: 0x01d9, 0x1a27: 0x0fa9, 0x1a28: 0x0fb9, 0x1a29: 0x1089,
-	0x1a2a: 0x0279, 0x1a2b: 0x0369, 0x1a2c: 0x0289, 0x1a2d: 0x13d1, 0x1a2e: 0x0039, 0x1a2f: 0x0ee9,
-	0x1a30: 0x1159, 0x1a31: 0x0ef9, 0x1a32: 0x0f09, 0x1a33: 0x1199, 0x1a34: 0x0f31, 0x1a35: 0x0249,
-	0x1a36: 0x0f41, 0x1a37: 0x0259, 0x1a38: 0x0f51, 0x1a39: 0x0359, 0x1a3a: 0x0f61, 0x1a3b: 0x0f71,
-	0x1a3c: 0x00d9, 0x1a3d: 0x0f99, 0x1a3e: 0x2039, 0x1a3f: 0x0269,
-	// Block 0x69, offset 0x1a40
-	0x1a40: 0x01d9, 0x1a41: 0x0fa9, 0x1a42: 0x0fb9, 0x1a43: 0x1089, 0x1a44: 0x0279, 0x1a45: 0x0369,
-	0x1a46: 0x0289, 0x1a47: 0x13d1, 0x1a48: 0x0039, 0x1a49: 0x0ee9, 0x1a4a: 0x1159, 0x1a4b: 0x0ef9,
-	0x1a4c: 0x0f09, 0x1a4d: 0x1199, 0x1a4e: 0x0f31, 0x1a4f: 0x0249, 0x1a50: 0x0f41, 0x1a51: 0x0259,
-	0x1a52: 0x0f51, 0x1a53: 0x0359, 0x1a54: 0x0f61, 0x1a55: 0x0f71, 0x1a56: 0x00d9, 0x1a57: 0x0f99,
-	0x1a58: 0x2039, 0x1a59: 0x0269, 0x1a5a: 0x01d9, 0x1a5b: 0x0fa9, 0x1a5c: 0x0fb9, 0x1a5d: 0x1089,
-	0x1a5e: 0x0279, 0x1a5f: 0x0369, 0x1a60: 0x0289, 0x1a61: 0x13d1, 0x1a62: 0x0039, 0x1a63: 0x0ee9,
-	0x1a64: 0x1159, 0x1a65: 0x0ef9, 0x1a66: 0x0f09, 0x1a67: 0x1199, 0x1a68: 0x0f31, 0x1a69: 0x0249,
-	0x1a6a: 0x0f41, 0x1a6b: 0x0259, 0x1a6c: 0x0f51, 0x1a6d: 0x0359, 0x1a6e: 0x0f61, 0x1a6f: 0x0f71,
-	0x1a70: 0x00d9, 0x1a71: 0x0f99, 0x1a72: 0x2039, 0x1a73: 0x0269, 0x1a74: 0x01d9, 0x1a75: 0x0fa9,
-	0x1a76: 0x0fb9, 0x1a77: 0x1089, 0x1a78: 0x0279, 0x1a79: 0x0369, 0x1a7a: 0x0289, 0x1a7b: 0x13d1,
-	0x1a7c: 0x0039, 0x1a7d: 0x0ee9, 0x1a7e: 0x1159, 0x1a7f: 0x0ef9,
-	// Block 0x6a, offset 0x1a80
-	0x1a80: 0x0f09, 0x1a81: 0x1199, 0x1a82: 0x0f31, 0x1a83: 0x0249, 0x1a84: 0x0f41, 0x1a85: 0x0259,
-	0x1a86: 0x0f51, 0x1a87: 0x0359, 0x1a88: 0x0f61, 0x1a89: 0x0f71, 0x1a8a: 0x00d9, 0x1a8b: 0x0f99,
-	0x1a8c: 0x2039, 0x1a8d: 0x0269, 0x1a8e: 0x01d9, 0x1a8f: 0x0fa9, 0x1a90: 0x0fb9, 0x1a91: 0x1089,
-	0x1a92: 0x0279, 0x1a93: 0x0369, 0x1a94: 0x0289, 0x1a95: 0x13d1, 0x1a96: 0x0039, 0x1a97: 0x0ee9,
-	0x1a98: 0x1159, 0x1a99: 0x0ef9, 0x1a9a: 0x0f09, 0x1a9b: 0x1199, 0x1a9c: 0x0f31, 0x1a9d: 0x0249,
-	0x1a9e: 0x0f41, 0x1a9f: 0x0259, 0x1aa0: 0x0f51, 0x1aa1: 0x0359, 0x1aa2: 0x0f61, 0x1aa3: 0x0f71,
-	0x1aa4: 0x00d9, 0x1aa5: 0x0f99, 0x1aa6: 0x2039, 0x1aa7: 0x0269, 0x1aa8: 0x01d9, 0x1aa9: 0x0fa9,
-	0x1aaa: 0x0fb9, 0x1aab: 0x1089, 0x1aac: 0x0279, 0x1aad: 0x0369, 0x1aae: 0x0289, 0x1aaf: 0x13d1,
-	0x1ab0: 0x0039, 0x1ab1: 0x0ee9, 0x1ab2: 0x1159, 0x1ab3: 0x0ef9, 0x1ab4: 0x0f09, 0x1ab5: 0x1199,
-	0x1ab6: 0x0f31, 0x1ab7: 0x0249, 0x1ab8: 0x0f41, 0x1ab9: 0x0259, 0x1aba: 0x0f51, 0x1abb: 0x0359,
-	0x1abc: 0x0f61, 0x1abd: 0x0f71, 0x1abe: 0x00d9, 0x1abf: 0x0f99,
-	// Block 0x6b, offset 0x1ac0
-	0x1ac0: 0x2039, 0x1ac1: 0x0269, 0x1ac2: 0x01d9, 0x1ac3: 0x0fa9, 0x1ac4: 0x0fb9, 0x1ac5: 0x1089,
-	0x1ac6: 0x0279, 0x1ac7: 0x0369, 0x1ac8: 0x0289, 0x1ac9: 0x13d1, 0x1aca: 0x0039, 0x1acb: 0x0ee9,
-	0x1acc: 0x1159, 0x1acd: 0x0ef9, 0x1ace: 0x0f09, 0x1acf: 0x1199, 0x1ad0: 0x0f31, 0x1ad1: 0x0249,
-	0x1ad2: 0x0f41, 0x1ad3: 0x0259, 0x1ad4: 0x0f51, 0x1ad5: 0x0359, 0x1ad6: 0x0f61, 0x1ad7: 0x0f71,
-	0x1ad8: 0x00d9, 0x1ad9: 0x0f99, 0x1ada: 0x2039, 0x1adb: 0x0269, 0x1adc: 0x01d9, 0x1add: 0x0fa9,
-	0x1ade: 0x0fb9, 0x1adf: 0x1089, 0x1ae0: 0x0279, 0x1ae1: 0x0369, 0x1ae2: 0x0289, 0x1ae3: 0x13d1,
-	0x1ae4: 0xba81, 0x1ae5: 0xba99, 0x1ae6: 0x0040, 0x1ae7: 0x0040, 0x1ae8: 0xbab1, 0x1ae9: 0x1099,
-	0x1aea: 0x10b1, 0x1aeb: 0x10c9, 0x1aec: 0xbac9, 0x1aed: 0xbae1, 0x1aee: 0xbaf9, 0x1aef: 0x1429,
-	0x1af0: 0x1a31, 0x1af1: 0xbb11, 0x1af2: 0xbb29, 0x1af3: 0xbb41, 0x1af4: 0xbb59, 0x1af5: 0xbb71,
-	0x1af6: 0xbb89, 0x1af7: 0x2109, 0x1af8: 0x1111, 0x1af9: 0x1429, 0x1afa: 0xbba1, 0x1afb: 0xbbb9,
-	0x1afc: 0xbbd1, 0x1afd: 0x10e1, 0x1afe: 0x10f9, 0x1aff: 0xbbe9,
-	// Block 0x6c, offset 0x1b00
-	0x1b00: 0x2079, 0x1b01: 0xbc01, 0x1b02: 0xbab1, 0x1b03: 0x1099, 0x1b04: 0x10b1, 0x1b05: 0x10c9,
-	0x1b06: 0xbac9, 0x1b07: 0xbae1, 0x1b08: 0xbaf9, 0x1b09: 0x1429, 0x1b0a: 0x1a31, 0x1b0b: 0xbb11,
-	0x1b0c: 0xbb29, 0x1b0d: 0xbb41, 0x1b0e: 0xbb59, 0x1b0f: 0xbb71, 0x1b10: 0xbb89, 0x1b11: 0x2109,
-	0x1b12: 0x1111, 0x1b13: 0xbba1, 0x1b14: 0xbba1, 0x1b15: 0xbbb9, 0x1b16: 0xbbd1, 0x1b17: 0x10e1,
-	0x1b18: 0x10f9, 0x1b19: 0xbbe9, 0x1b1a: 0x2079, 0x1b1b: 0xbc21, 0x1b1c: 0xbac9, 0x1b1d: 0x1429,
-	0x1b1e: 0xbb11, 0x1b1f: 0x10e1, 0x1b20: 0x1111, 0x1b21: 0x2109, 0x1b22: 0xbab1, 0x1b23: 0x1099,
-	0x1b24: 0x10b1, 0x1b25: 0x10c9, 0x1b26: 0xbac9, 0x1b27: 0xbae1, 0x1b28: 0xbaf9, 0x1b29: 0x1429,
-	0x1b2a: 0x1a31, 0x1b2b: 0xbb11, 0x1b2c: 0xbb29, 0x1b2d: 0xbb41, 0x1b2e: 0xbb59, 0x1b2f: 0xbb71,
-	0x1b30: 0xbb89, 0x1b31: 0x2109, 0x1b32: 0x1111, 0x1b33: 0x1429, 0x1b34: 0xbba1, 0x1b35: 0xbbb9,
-	0x1b36: 0xbbd1, 0x1b37: 0x10e1, 0x1b38: 0x10f9, 0x1b39: 0xbbe9, 0x1b3a: 0x2079, 0x1b3b: 0xbc01,
-	0x1b3c: 0xbab1, 0x1b3d: 0x1099, 0x1b3e: 0x10b1, 0x1b3f: 0x10c9,
-	// Block 0x6d, offset 0x1b40
-	0x1b40: 0xbac9, 0x1b41: 0xbae1, 0x1b42: 0xbaf9, 0x1b43: 0x1429, 0x1b44: 0x1a31, 0x1b45: 0xbb11,
-	0x1b46: 0xbb29, 0x1b47: 0xbb41, 0x1b48: 0xbb59, 0x1b49: 0xbb71, 0x1b4a: 0xbb89, 0x1b4b: 0x2109,
-	0x1b4c: 0x1111, 0x1b4d: 0xbba1, 0x1b4e: 0xbba1, 0x1b4f: 0xbbb9, 0x1b50: 0xbbd1, 0x1b51: 0x10e1,
-	0x1b52: 0x10f9, 0x1b53: 0xbbe9, 0x1b54: 0x2079, 0x1b55: 0xbc21, 0x1b56: 0xbac9, 0x1b57: 0x1429,
-	0x1b58: 0xbb11, 0x1b59: 0x10e1, 0x1b5a: 0x1111, 0x1b5b: 0x2109, 0x1b5c: 0xbab1, 0x1b5d: 0x1099,
-	0x1b5e: 0x10b1, 0x1b5f: 0x10c9, 0x1b60: 0xbac9, 0x1b61: 0xbae1, 0x1b62: 0xbaf9, 0x1b63: 0x1429,
-	0x1b64: 0x1a31, 0x1b65: 0xbb11, 0x1b66: 0xbb29, 0x1b67: 0xbb41, 0x1b68: 0xbb59, 0x1b69: 0xbb71,
-	0x1b6a: 0xbb89, 0x1b6b: 0x2109, 0x1b6c: 0x1111, 0x1b6d: 0x1429, 0x1b6e: 0xbba1, 0x1b6f: 0xbbb9,
-	0x1b70: 0xbbd1, 0x1b71: 0x10e1, 0x1b72: 0x10f9, 0x1b73: 0xbbe9, 0x1b74: 0x2079, 0x1b75: 0xbc01,
-	0x1b76: 0xbab1, 0x1b77: 0x1099, 0x1b78: 0x10b1, 0x1b79: 0x10c9, 0x1b7a: 0xbac9, 0x1b7b: 0xbae1,
-	0x1b7c: 0xbaf9, 0x1b7d: 0x1429, 0x1b7e: 0x1a31, 0x1b7f: 0xbb11,
-	// Block 0x6e, offset 0x1b80
-	0x1b80: 0xbb29, 0x1b81: 0xbb41, 0x1b82: 0xbb59, 0x1b83: 0xbb71, 0x1b84: 0xbb89, 0x1b85: 0x2109,
-	0x1b86: 0x1111, 0x1b87: 0xbba1, 0x1b88: 0xbba1, 0x1b89: 0xbbb9, 0x1b8a: 0xbbd1, 0x1b8b: 0x10e1,
-	0x1b8c: 0x10f9, 0x1b8d: 0xbbe9, 0x1b8e: 0x2079, 0x1b8f: 0xbc21, 0x1b90: 0xbac9, 0x1b91: 0x1429,
-	0x1b92: 0xbb11, 0x1b93: 0x10e1, 0x1b94: 0x1111, 0x1b95: 0x2109, 0x1b96: 0xbab1, 0x1b97: 0x1099,
-	0x1b98: 0x10b1, 0x1b99: 0x10c9, 0x1b9a: 0xbac9, 0x1b9b: 0xbae1, 0x1b9c: 0xbaf9, 0x1b9d: 0x1429,
-	0x1b9e: 0x1a31, 0x1b9f: 0xbb11, 0x1ba0: 0xbb29, 0x1ba1: 0xbb41, 0x1ba2: 0xbb59, 0x1ba3: 0xbb71,
-	0x1ba4: 0xbb89, 0x1ba5: 0x2109, 0x1ba6: 0x1111, 0x1ba7: 0x1429, 0x1ba8: 0xbba1, 0x1ba9: 0xbbb9,
-	0x1baa: 0xbbd1, 0x1bab: 0x10e1, 0x1bac: 0x10f9, 0x1bad: 0xbbe9, 0x1bae: 0x2079, 0x1baf: 0xbc01,
-	0x1bb0: 0xbab1, 0x1bb1: 0x1099, 0x1bb2: 0x10b1, 0x1bb3: 0x10c9, 0x1bb4: 0xbac9, 0x1bb5: 0xbae1,
-	0x1bb6: 0xbaf9, 0x1bb7: 0x1429, 0x1bb8: 0x1a31, 0x1bb9: 0xbb11, 0x1bba: 0xbb29, 0x1bbb: 0xbb41,
-	0x1bbc: 0xbb59, 0x1bbd: 0xbb71, 0x1bbe: 0xbb89, 0x1bbf: 0x2109,
-	// Block 0x6f, offset 0x1bc0
-	0x1bc0: 0x1111, 0x1bc1: 0xbba1, 0x1bc2: 0xbba1, 0x1bc3: 0xbbb9, 0x1bc4: 0xbbd1, 0x1bc5: 0x10e1,
-	0x1bc6: 0x10f9, 0x1bc7: 0xbbe9, 0x1bc8: 0x2079, 0x1bc9: 0xbc21, 0x1bca: 0xbac9, 0x1bcb: 0x1429,
-	0x1bcc: 0xbb11, 0x1bcd: 0x10e1, 0x1bce: 0x1111, 0x1bcf: 0x2109, 0x1bd0: 0xbab1, 0x1bd1: 0x1099,
-	0x1bd2: 0x10b1, 0x1bd3: 0x10c9, 0x1bd4: 0xbac9, 0x1bd5: 0xbae1, 0x1bd6: 0xbaf9, 0x1bd7: 0x1429,
-	0x1bd8: 0x1a31, 0x1bd9: 0xbb11, 0x1bda: 0xbb29, 0x1bdb: 0xbb41, 0x1bdc: 0xbb59, 0x1bdd: 0xbb71,
-	0x1bde: 0xbb89, 0x1bdf: 0x2109, 0x1be0: 0x1111, 0x1be1: 0x1429, 0x1be2: 0xbba1, 0x1be3: 0xbbb9,
-	0x1be4: 0xbbd1, 0x1be5: 0x10e1, 0x1be6: 0x10f9, 0x1be7: 0xbbe9, 0x1be8: 0x2079, 0x1be9: 0xbc01,
-	0x1bea: 0xbab1, 0x1beb: 0x1099, 0x1bec: 0x10b1, 0x1bed: 0x10c9, 0x1bee: 0xbac9, 0x1bef: 0xbae1,
-	0x1bf0: 0xbaf9, 0x1bf1: 0x1429, 0x1bf2: 0x1a31, 0x1bf3: 0xbb11, 0x1bf4: 0xbb29, 0x1bf5: 0xbb41,
-	0x1bf6: 0xbb59, 0x1bf7: 0xbb71, 0x1bf8: 0xbb89, 0x1bf9: 0x2109, 0x1bfa: 0x1111, 0x1bfb: 0xbba1,
-	0x1bfc: 0xbba1, 0x1bfd: 0xbbb9, 0x1bfe: 0xbbd1, 0x1bff: 0x10e1,
-	// Block 0x70, offset 0x1c00
-	0x1c00: 0x10f9, 0x1c01: 0xbbe9, 0x1c02: 0x2079, 0x1c03: 0xbc21, 0x1c04: 0xbac9, 0x1c05: 0x1429,
-	0x1c06: 0xbb11, 0x1c07: 0x10e1, 0x1c08: 0x1111, 0x1c09: 0x2109, 0x1c0a: 0xbc41, 0x1c0b: 0xbc41,
-	0x1c0c: 0x0040, 0x1c0d: 0x0040, 0x1c0e: 0x1f41, 0x1c0f: 0x00c9, 0x1c10: 0x0069, 0x1c11: 0x0079,
-	0x1c12: 0x1f51, 0x1c13: 0x1f61, 0x1c14: 0x1f71, 0x1c15: 0x1f81, 0x1c16: 0x1f91, 0x1c17: 0x1fa1,
-	0x1c18: 0x1f41, 0x1c19: 0x00c9, 0x1c1a: 0x0069, 0x1c1b: 0x0079, 0x1c1c: 0x1f51, 0x1c1d: 0x1f61,
-	0x1c1e: 0x1f71, 0x1c1f: 0x1f81, 0x1c20: 0x1f91, 0x1c21: 0x1fa1, 0x1c22: 0x1f41, 0x1c23: 0x00c9,
-	0x1c24: 0x0069, 0x1c25: 0x0079, 0x1c26: 0x1f51, 0x1c27: 0x1f61, 0x1c28: 0x1f71, 0x1c29: 0x1f81,
-	0x1c2a: 0x1f91, 0x1c2b: 0x1fa1, 0x1c2c: 0x1f41, 0x1c2d: 0x00c9, 0x1c2e: 0x0069, 0x1c2f: 0x0079,
-	0x1c30: 0x1f51, 0x1c31: 0x1f61, 0x1c32: 0x1f71, 0x1c33: 0x1f81, 0x1c34: 0x1f91, 0x1c35: 0x1fa1,
-	0x1c36: 0x1f41, 0x1c37: 0x00c9, 0x1c38: 0x0069, 0x1c39: 0x0079, 0x1c3a: 0x1f51, 0x1c3b: 0x1f61,
-	0x1c3c: 0x1f71, 0x1c3d: 0x1f81, 0x1c3e: 0x1f91, 0x1c3f: 0x1fa1,
-	// Block 0x71, offset 0x1c40
-	0x1c40: 0xe115, 0x1c41: 0xe115, 0x1c42: 0xe135, 0x1c43: 0xe135, 0x1c44: 0xe115, 0x1c45: 0xe115,
-	0x1c46: 0xe175, 0x1c47: 0xe175, 0x1c48: 0xe115, 0x1c49: 0xe115, 0x1c4a: 0xe135, 0x1c4b: 0xe135,
-	0x1c4c: 0xe115, 0x1c4d: 0xe115, 0x1c4e: 0xe1f5, 0x1c4f: 0xe1f5, 0x1c50: 0xe115, 0x1c51: 0xe115,
-	0x1c52: 0xe135, 0x1c53: 0xe135, 0x1c54: 0xe115, 0x1c55: 0xe115, 0x1c56: 0xe175, 0x1c57: 0xe175,
-	0x1c58: 0xe115, 0x1c59: 0xe115, 0x1c5a: 0xe135, 0x1c5b: 0xe135, 0x1c5c: 0xe115, 0x1c5d: 0xe115,
-	0x1c5e: 0x8b3d, 0x1c5f: 0x8b3d, 0x1c60: 0x04b5, 0x1c61: 0x04b5, 0x1c62: 0x0a08, 0x1c63: 0x0a08,
-	0x1c64: 0x0a08, 0x1c65: 0x0a08, 0x1c66: 0x0a08, 0x1c67: 0x0a08, 0x1c68: 0x0a08, 0x1c69: 0x0a08,
-	0x1c6a: 0x0a08, 0x1c6b: 0x0a08, 0x1c6c: 0x0a08, 0x1c6d: 0x0a08, 0x1c6e: 0x0a08, 0x1c6f: 0x0a08,
-	0x1c70: 0x0a08, 0x1c71: 0x0a08, 0x1c72: 0x0a08, 0x1c73: 0x0a08, 0x1c74: 0x0a08, 0x1c75: 0x0a08,
-	0x1c76: 0x0a08, 0x1c77: 0x0a08, 0x1c78: 0x0a08, 0x1c79: 0x0a08, 0x1c7a: 0x0a08, 0x1c7b: 0x0a08,
-	0x1c7c: 0x0a08, 0x1c7d: 0x0a08, 0x1c7e: 0x0a08, 0x1c7f: 0x0a08,
-	// Block 0x72, offset 0x1c80
-	0x1c80: 0xb189, 0x1c81: 0xb1a1, 0x1c82: 0xb201, 0x1c83: 0xb249, 0x1c84: 0x0040, 0x1c85: 0xb411,
-	0x1c86: 0xb291, 0x1c87: 0xb219, 0x1c88: 0xb309, 0x1c89: 0xb429, 0x1c8a: 0xb399, 0x1c8b: 0xb3b1,
-	0x1c8c: 0xb3c9, 0x1c8d: 0xb3e1, 0x1c8e: 0xb2a9, 0x1c8f: 0xb339, 0x1c90: 0xb369, 0x1c91: 0xb2d9,
-	0x1c92: 0xb381, 0x1c93: 0xb279, 0x1c94: 0xb2c1, 0x1c95: 0xb1d1, 0x1c96: 0xb1e9, 0x1c97: 0xb231,
-	0x1c98: 0xb261, 0x1c99: 0xb2f1, 0x1c9a: 0xb321, 0x1c9b: 0xb351, 0x1c9c: 0xbc59, 0x1c9d: 0x7949,
-	0x1c9e: 0xbc71, 0x1c9f: 0xbc89, 0x1ca0: 0x0040, 0x1ca1: 0xb1a1, 0x1ca2: 0xb201, 0x1ca3: 0x0040,
-	0x1ca4: 0xb3f9, 0x1ca5: 0x0040, 0x1ca6: 0x0040, 0x1ca7: 0xb219, 0x1ca8: 0x0040, 0x1ca9: 0xb429,
-	0x1caa: 0xb399, 0x1cab: 0xb3b1, 0x1cac: 0xb3c9, 0x1cad: 0xb3e1, 0x1cae: 0xb2a9, 0x1caf: 0xb339,
-	0x1cb0: 0xb369, 0x1cb1: 0xb2d9, 0x1cb2: 0xb381, 0x1cb3: 0x0040, 0x1cb4: 0xb2c1, 0x1cb5: 0xb1d1,
-	0x1cb6: 0xb1e9, 0x1cb7: 0xb231, 0x1cb8: 0x0040, 0x1cb9: 0xb2f1, 0x1cba: 0x0040, 0x1cbb: 0xb351,
-	0x1cbc: 0x0040, 0x1cbd: 0x0040, 0x1cbe: 0x0040, 0x1cbf: 0x0040,
-	// Block 0x73, offset 0x1cc0
-	0x1cc0: 0x0040, 0x1cc1: 0x0040, 0x1cc2: 0xb201, 0x1cc3: 0x0040, 0x1cc4: 0x0040, 0x1cc5: 0x0040,
-	0x1cc6: 0x0040, 0x1cc7: 0xb219, 0x1cc8: 0x0040, 0x1cc9: 0xb429, 0x1cca: 0x0040, 0x1ccb: 0xb3b1,
-	0x1ccc: 0x0040, 0x1ccd: 0xb3e1, 0x1cce: 0xb2a9, 0x1ccf: 0xb339, 0x1cd0: 0x0040, 0x1cd1: 0xb2d9,
-	0x1cd2: 0xb381, 0x1cd3: 0x0040, 0x1cd4: 0xb2c1, 0x1cd5: 0x0040, 0x1cd6: 0x0040, 0x1cd7: 0xb231,
-	0x1cd8: 0x0040, 0x1cd9: 0xb2f1, 0x1cda: 0x0040, 0x1cdb: 0xb351, 0x1cdc: 0x0040, 0x1cdd: 0x7949,
-	0x1cde: 0x0040, 0x1cdf: 0xbc89, 0x1ce0: 0x0040, 0x1ce1: 0xb1a1, 0x1ce2: 0xb201, 0x1ce3: 0x0040,
-	0x1ce4: 0xb3f9, 0x1ce5: 0x0040, 0x1ce6: 0x0040, 0x1ce7: 0xb219, 0x1ce8: 0xb309, 0x1ce9: 0xb429,
-	0x1cea: 0xb399, 0x1ceb: 0x0040, 0x1cec: 0xb3c9, 0x1ced: 0xb3e1, 0x1cee: 0xb2a9, 0x1cef: 0xb339,
-	0x1cf0: 0xb369, 0x1cf1: 0xb2d9, 0x1cf2: 0xb381, 0x1cf3: 0x0040, 0x1cf4: 0xb2c1, 0x1cf5: 0xb1d1,
-	0x1cf6: 0xb1e9, 0x1cf7: 0xb231, 0x1cf8: 0x0040, 0x1cf9: 0xb2f1, 0x1cfa: 0xb321, 0x1cfb: 0xb351,
-	0x1cfc: 0xbc59, 0x1cfd: 0x0040, 0x1cfe: 0xbc71, 0x1cff: 0x0040,
-	// Block 0x74, offset 0x1d00
-	0x1d00: 0xb189, 0x1d01: 0xb1a1, 0x1d02: 0xb201, 0x1d03: 0xb249, 0x1d04: 0xb3f9, 0x1d05: 0xb411,
-	0x1d06: 0xb291, 0x1d07: 0xb219, 0x1d08: 0xb309, 0x1d09: 0xb429, 0x1d0a: 0x0040, 0x1d0b: 0xb3b1,
-	0x1d0c: 0xb3c9, 0x1d0d: 0xb3e1, 0x1d0e: 0xb2a9, 0x1d0f: 0xb339, 0x1d10: 0xb369, 0x1d11: 0xb2d9,
-	0x1d12: 0xb381, 0x1d13: 0xb279, 0x1d14: 0xb2c1, 0x1d15: 0xb1d1, 0x1d16: 0xb1e9, 0x1d17: 0xb231,
-	0x1d18: 0xb261, 0x1d19: 0xb2f1, 0x1d1a: 0xb321, 0x1d1b: 0xb351, 0x1d1c: 0x0040, 0x1d1d: 0x0040,
-	0x1d1e: 0x0040, 0x1d1f: 0x0040, 0x1d20: 0x0040, 0x1d21: 0xb1a1, 0x1d22: 0xb201, 0x1d23: 0xb249,
-	0x1d24: 0x0040, 0x1d25: 0xb411, 0x1d26: 0xb291, 0x1d27: 0xb219, 0x1d28: 0xb309, 0x1d29: 0xb429,
-	0x1d2a: 0x0040, 0x1d2b: 0xb3b1, 0x1d2c: 0xb3c9, 0x1d2d: 0xb3e1, 0x1d2e: 0xb2a9, 0x1d2f: 0xb339,
-	0x1d30: 0xb369, 0x1d31: 0xb2d9, 0x1d32: 0xb381, 0x1d33: 0xb279, 0x1d34: 0xb2c1, 0x1d35: 0xb1d1,
-	0x1d36: 0xb1e9, 0x1d37: 0xb231, 0x1d38: 0xb261, 0x1d39: 0xb2f1, 0x1d3a: 0xb321, 0x1d3b: 0xb351,
-	0x1d3c: 0x0040, 0x1d3d: 0x0040, 0x1d3e: 0x0040, 0x1d3f: 0x0040,
-	// Block 0x75, offset 0x1d40
-	0x1d40: 0x0040, 0x1d41: 0xbca2, 0x1d42: 0xbcba, 0x1d43: 0xbcd2, 0x1d44: 0xbcea, 0x1d45: 0xbd02,
-	0x1d46: 0xbd1a, 0x1d47: 0xbd32, 0x1d48: 0xbd4a, 0x1d49: 0xbd62, 0x1d4a: 0xbd7a, 0x1d4b: 0x0018,
-	0x1d4c: 0x0018, 0x1d4d: 0x0040, 0x1d4e: 0x0040, 0x1d4f: 0x0040, 0x1d50: 0xbd92, 0x1d51: 0xbdb2,
-	0x1d52: 0xbdd2, 0x1d53: 0xbdf2, 0x1d54: 0xbe12, 0x1d55: 0xbe32, 0x1d56: 0xbe52, 0x1d57: 0xbe72,
-	0x1d58: 0xbe92, 0x1d59: 0xbeb2, 0x1d5a: 0xbed2, 0x1d5b: 0xbef2, 0x1d5c: 0xbf12, 0x1d5d: 0xbf32,
-	0x1d5e: 0xbf52, 0x1d5f: 0xbf72, 0x1d60: 0xbf92, 0x1d61: 0xbfb2, 0x1d62: 0xbfd2, 0x1d63: 0xbff2,
-	0x1d64: 0xc012, 0x1d65: 0xc032, 0x1d66: 0xc052, 0x1d67: 0xc072, 0x1d68: 0xc092, 0x1d69: 0xc0b2,
-	0x1d6a: 0xc0d1, 0x1d6b: 0x1159, 0x1d6c: 0x0269, 0x1d6d: 0x6671, 0x1d6e: 0xc111, 0x1d6f: 0x0018,
-	0x1d70: 0x0039, 0x1d71: 0x0ee9, 0x1d72: 0x1159, 0x1d73: 0x0ef9, 0x1d74: 0x0f09, 0x1d75: 0x1199,
-	0x1d76: 0x0f31, 0x1d77: 0x0249, 0x1d78: 0x0f41, 0x1d79: 0x0259, 0x1d7a: 0x0f51, 0x1d7b: 0x0359,
-	0x1d7c: 0x0f61, 0x1d7d: 0x0f71, 0x1d7e: 0x00d9, 0x1d7f: 0x0f99,
-	// Block 0x76, offset 0x1d80
-	0x1d80: 0x2039, 0x1d81: 0x0269, 0x1d82: 0x01d9, 0x1d83: 0x0fa9, 0x1d84: 0x0fb9, 0x1d85: 0x1089,
-	0x1d86: 0x0279, 0x1d87: 0x0369, 0x1d88: 0x0289, 0x1d89: 0x13d1, 0x1d8a: 0xc129, 0x1d8b: 0x65b1,
-	0x1d8c: 0xc141, 0x1d8d: 0x1441, 0x1d8e: 0xc159, 0x1d8f: 0xc179, 0x1d90: 0x0018, 0x1d91: 0x0018,
-	0x1d92: 0x0018, 0x1d93: 0x0018, 0x1d94: 0x0018, 0x1d95: 0x0018, 0x1d96: 0x0018, 0x1d97: 0x0018,
-	0x1d98: 0x0018, 0x1d99: 0x0018, 0x1d9a: 0x0018, 0x1d9b: 0x0018, 0x1d9c: 0x0018, 0x1d9d: 0x0018,
-	0x1d9e: 0x0018, 0x1d9f: 0x0018, 0x1da0: 0x0018, 0x1da1: 0x0018, 0x1da2: 0x0018, 0x1da3: 0x0018,
-	0x1da4: 0x0018, 0x1da5: 0x0018, 0x1da6: 0x0018, 0x1da7: 0x0018, 0x1da8: 0x0018, 0x1da9: 0x0018,
-	0x1daa: 0xc191, 0x1dab: 0xc1a9, 0x1dac: 0xc1c1, 0x1dad: 0x0040, 0x1dae: 0x0040, 0x1daf: 0x0040,
-	0x1db0: 0x0018, 0x1db1: 0x0018, 0x1db2: 0x0018, 0x1db3: 0x0018, 0x1db4: 0x0018, 0x1db5: 0x0018,
-	0x1db6: 0x0018, 0x1db7: 0x0018, 0x1db8: 0x0018, 0x1db9: 0x0018, 0x1dba: 0x0018, 0x1dbb: 0x0018,
-	0x1dbc: 0x0018, 0x1dbd: 0x0018, 0x1dbe: 0x0018, 0x1dbf: 0x0018,
-	// Block 0x77, offset 0x1dc0
-	0x1dc0: 0xc1f1, 0x1dc1: 0xc229, 0x1dc2: 0xc261, 0x1dc3: 0x0040, 0x1dc4: 0x0040, 0x1dc5: 0x0040,
-	0x1dc6: 0x0040, 0x1dc7: 0x0040, 0x1dc8: 0x0040, 0x1dc9: 0x0040, 0x1dca: 0x0040, 0x1dcb: 0x0040,
-	0x1dcc: 0x0040, 0x1dcd: 0x0040, 0x1dce: 0x0040, 0x1dcf: 0x0040, 0x1dd0: 0xc281, 0x1dd1: 0xc2a1,
-	0x1dd2: 0xc2c1, 0x1dd3: 0xc2e1, 0x1dd4: 0xc301, 0x1dd5: 0xc321, 0x1dd6: 0xc341, 0x1dd7: 0xc361,
-	0x1dd8: 0xc381, 0x1dd9: 0xc3a1, 0x1dda: 0xc3c1, 0x1ddb: 0xc3e1, 0x1ddc: 0xc401, 0x1ddd: 0xc421,
-	0x1dde: 0xc441, 0x1ddf: 0xc461, 0x1de0: 0xc481, 0x1de1: 0xc4a1, 0x1de2: 0xc4c1, 0x1de3: 0xc4e1,
-	0x1de4: 0xc501, 0x1de5: 0xc521, 0x1de6: 0xc541, 0x1de7: 0xc561, 0x1de8: 0xc581, 0x1de9: 0xc5a1,
-	0x1dea: 0xc5c1, 0x1deb: 0xc5e1, 0x1dec: 0xc601, 0x1ded: 0xc621, 0x1dee: 0xc641, 0x1def: 0xc661,
-	0x1df0: 0xc681, 0x1df1: 0xc6a1, 0x1df2: 0xc6c1, 0x1df3: 0xc6e1, 0x1df4: 0xc701, 0x1df5: 0xc721,
-	0x1df6: 0xc741, 0x1df7: 0xc761, 0x1df8: 0xc781, 0x1df9: 0xc7a1, 0x1dfa: 0xc7c1, 0x1dfb: 0xc7e1,
-	0x1dfc: 0x0040, 0x1dfd: 0x0040, 0x1dfe: 0x0040, 0x1dff: 0x0040,
-	// Block 0x78, offset 0x1e00
-	0x1e00: 0xcb11, 0x1e01: 0xcb31, 0x1e02: 0xcb51, 0x1e03: 0x8b55, 0x1e04: 0xcb71, 0x1e05: 0xcb91,
-	0x1e06: 0xcbb1, 0x1e07: 0xcbd1, 0x1e08: 0xcbf1, 0x1e09: 0xcc11, 0x1e0a: 0xcc31, 0x1e0b: 0xcc51,
-	0x1e0c: 0xcc71, 0x1e0d: 0x8b75, 0x1e0e: 0xcc91, 0x1e0f: 0xccb1, 0x1e10: 0xccd1, 0x1e11: 0xccf1,
-	0x1e12: 0x8b95, 0x1e13: 0xcd11, 0x1e14: 0xcd31, 0x1e15: 0xc441, 0x1e16: 0x8bb5, 0x1e17: 0xcd51,
-	0x1e18: 0xcd71, 0x1e19: 0xcd91, 0x1e1a: 0xcdb1, 0x1e1b: 0xcdd1, 0x1e1c: 0x8bd5, 0x1e1d: 0xcdf1,
-	0x1e1e: 0xce11, 0x1e1f: 0xce31, 0x1e20: 0xce51, 0x1e21: 0xce71, 0x1e22: 0xc7a1, 0x1e23: 0xce91,
-	0x1e24: 0xceb1, 0x1e25: 0xced1, 0x1e26: 0xcef1, 0x1e27: 0xcf11, 0x1e28: 0xcf31, 0x1e29: 0xcf51,
-	0x1e2a: 0xcf71, 0x1e2b: 0xcf91, 0x1e2c: 0xcfb1, 0x1e2d: 0xcfd1, 0x1e2e: 0xcff1, 0x1e2f: 0xd011,
-	0x1e30: 0xd031, 0x1e31: 0xd051, 0x1e32: 0xd051, 0x1e33: 0xd051, 0x1e34: 0x8bf5, 0x1e35: 0xd071,
-	0x1e36: 0xd091, 0x1e37: 0xd0b1, 0x1e38: 0x8c15, 0x1e39: 0xd0d1, 0x1e3a: 0xd0f1, 0x1e3b: 0xd111,
-	0x1e3c: 0xd131, 0x1e3d: 0xd151, 0x1e3e: 0xd171, 0x1e3f: 0xd191,
-	// Block 0x79, offset 0x1e40
-	0x1e40: 0xd1b1, 0x1e41: 0xd1d1, 0x1e42: 0xd1f1, 0x1e43: 0xd211, 0x1e44: 0xd231, 0x1e45: 0xd251,
-	0x1e46: 0xd251, 0x1e47: 0xd271, 0x1e48: 0xd291, 0x1e49: 0xd2b1, 0x1e4a: 0xd2d1, 0x1e4b: 0xd2f1,
-	0x1e4c: 0xd311, 0x1e4d: 0xd331, 0x1e4e: 0xd351, 0x1e4f: 0xd371, 0x1e50: 0xd391, 0x1e51: 0xd3b1,
-	0x1e52: 0xd3d1, 0x1e53: 0xd3f1, 0x1e54: 0xd411, 0x1e55: 0xd431, 0x1e56: 0xd451, 0x1e57: 0xd471,
-	0x1e58: 0xd491, 0x1e59: 0x8c35, 0x1e5a: 0xd4b1, 0x1e5b: 0xd4d1, 0x1e5c: 0xd4f1, 0x1e5d: 0xc321,
-	0x1e5e: 0xd511, 0x1e5f: 0xd531, 0x1e60: 0x8c55, 0x1e61: 0x8c75, 0x1e62: 0xd551, 0x1e63: 0xd571,
-	0x1e64: 0xd591, 0x1e65: 0xd5b1, 0x1e66: 0xd5d1, 0x1e67: 0xd5f1, 0x1e68: 0x2040, 0x1e69: 0xd611,
-	0x1e6a: 0xd631, 0x1e6b: 0xd631, 0x1e6c: 0x8c95, 0x1e6d: 0xd651, 0x1e6e: 0xd671, 0x1e6f: 0xd691,
-	0x1e70: 0xd6b1, 0x1e71: 0x8cb5, 0x1e72: 0xd6d1, 0x1e73: 0xd6f1, 0x1e74: 0x2040, 0x1e75: 0xd711,
-	0x1e76: 0xd731, 0x1e77: 0xd751, 0x1e78: 0xd771, 0x1e79: 0xd791, 0x1e7a: 0xd7b1, 0x1e7b: 0x8cd5,
-	0x1e7c: 0xd7d1, 0x1e7d: 0x8cf5, 0x1e7e: 0xd7f1, 0x1e7f: 0xd811,
-	// Block 0x7a, offset 0x1e80
-	0x1e80: 0xd831, 0x1e81: 0xd851, 0x1e82: 0xd871, 0x1e83: 0xd891, 0x1e84: 0xd8b1, 0x1e85: 0xd8d1,
-	0x1e86: 0xd8f1, 0x1e87: 0xd911, 0x1e88: 0xd931, 0x1e89: 0x8d15, 0x1e8a: 0xd951, 0x1e8b: 0xd971,
-	0x1e8c: 0xd991, 0x1e8d: 0xd9b1, 0x1e8e: 0xd9d1, 0x1e8f: 0x8d35, 0x1e90: 0xd9f1, 0x1e91: 0x8d55,
-	0x1e92: 0x8d75, 0x1e93: 0xda11, 0x1e94: 0xda31, 0x1e95: 0xda31, 0x1e96: 0xda51, 0x1e97: 0x8d95,
-	0x1e98: 0x8db5, 0x1e99: 0xda71, 0x1e9a: 0xda91, 0x1e9b: 0xdab1, 0x1e9c: 0xdad1, 0x1e9d: 0xdaf1,
-	0x1e9e: 0xdb11, 0x1e9f: 0xdb31, 0x1ea0: 0xdb51, 0x1ea1: 0xdb71, 0x1ea2: 0xdb91, 0x1ea3: 0xdbb1,
-	0x1ea4: 0x8dd5, 0x1ea5: 0xdbd1, 0x1ea6: 0xdbf1, 0x1ea7: 0xdc11, 0x1ea8: 0xdc31, 0x1ea9: 0xdc11,
-	0x1eaa: 0xdc51, 0x1eab: 0xdc71, 0x1eac: 0xdc91, 0x1ead: 0xdcb1, 0x1eae: 0xdcd1, 0x1eaf: 0xdcf1,
-	0x1eb0: 0xdd11, 0x1eb1: 0xdd31, 0x1eb2: 0xdd51, 0x1eb3: 0xdd71, 0x1eb4: 0xdd91, 0x1eb5: 0xddb1,
-	0x1eb6: 0xddd1, 0x1eb7: 0xddf1, 0x1eb8: 0x8df5, 0x1eb9: 0xde11, 0x1eba: 0xde31, 0x1ebb: 0xde51,
-	0x1ebc: 0xde71, 0x1ebd: 0xde91, 0x1ebe: 0x8e15, 0x1ebf: 0xdeb1,
-	// Block 0x7b, offset 0x1ec0
-	0x1ec0: 0xe5b1, 0x1ec1: 0xe5d1, 0x1ec2: 0xe5f1, 0x1ec3: 0xe611, 0x1ec4: 0xe631, 0x1ec5: 0xe651,
-	0x1ec6: 0x8f35, 0x1ec7: 0xe671, 0x1ec8: 0xe691, 0x1ec9: 0xe6b1, 0x1eca: 0xe6d1, 0x1ecb: 0xe6f1,
-	0x1ecc: 0xe711, 0x1ecd: 0x8f55, 0x1ece: 0xe731, 0x1ecf: 0xe751, 0x1ed0: 0x8f75, 0x1ed1: 0x8f95,
-	0x1ed2: 0xe771, 0x1ed3: 0xe791, 0x1ed4: 0xe7b1, 0x1ed5: 0xe7d1, 0x1ed6: 0xe7f1, 0x1ed7: 0xe811,
-	0x1ed8: 0xe831, 0x1ed9: 0xe851, 0x1eda: 0xe871, 0x1edb: 0x8fb5, 0x1edc: 0xe891, 0x1edd: 0x8fd5,
-	0x1ede: 0xe8b1, 0x1edf: 0x2040, 0x1ee0: 0xe8d1, 0x1ee1: 0xe8f1, 0x1ee2: 0xe911, 0x1ee3: 0x8ff5,
-	0x1ee4: 0xe931, 0x1ee5: 0xe951, 0x1ee6: 0x9015, 0x1ee7: 0x9035, 0x1ee8: 0xe971, 0x1ee9: 0xe991,
-	0x1eea: 0xe9b1, 0x1eeb: 0xe9d1, 0x1eec: 0xe9f1, 0x1eed: 0xe9f1, 0x1eee: 0xea11, 0x1eef: 0xea31,
-	0x1ef0: 0xea51, 0x1ef1: 0xea71, 0x1ef2: 0xea91, 0x1ef3: 0xeab1, 0x1ef4: 0xead1, 0x1ef5: 0x9055,
-	0x1ef6: 0xeaf1, 0x1ef7: 0x9075, 0x1ef8: 0xeb11, 0x1ef9: 0x9095, 0x1efa: 0xeb31, 0x1efb: 0x90b5,
-	0x1efc: 0x90d5, 0x1efd: 0x90f5, 0x1efe: 0xeb51, 0x1eff: 0xeb71,
-	// Block 0x7c, offset 0x1f00
-	0x1f00: 0xeb91, 0x1f01: 0x9115, 0x1f02: 0x9135, 0x1f03: 0x9155, 0x1f04: 0x9175, 0x1f05: 0xebb1,
-	0x1f06: 0xebd1, 0x1f07: 0xebd1, 0x1f08: 0xebf1, 0x1f09: 0xec11, 0x1f0a: 0xec31, 0x1f0b: 0xec51,
-	0x1f0c: 0xec71, 0x1f0d: 0x9195, 0x1f0e: 0xec91, 0x1f0f: 0xecb1, 0x1f10: 0xecd1, 0x1f11: 0xecf1,
-	0x1f12: 0x91b5, 0x1f13: 0xed11, 0x1f14: 0x91d5, 0x1f15: 0x91f5, 0x1f16: 0xed31, 0x1f17: 0xed51,
-	0x1f18: 0xed71, 0x1f19: 0xed91, 0x1f1a: 0xedb1, 0x1f1b: 0xedd1, 0x1f1c: 0x9215, 0x1f1d: 0x9235,
-	0x1f1e: 0x9255, 0x1f1f: 0x2040, 0x1f20: 0xedf1, 0x1f21: 0x9275, 0x1f22: 0xee11, 0x1f23: 0xee31,
-	0x1f24: 0xee51, 0x1f25: 0x9295, 0x1f26: 0xee71, 0x1f27: 0xee91, 0x1f28: 0xeeb1, 0x1f29: 0xeed1,
-	0x1f2a: 0xeef1, 0x1f2b: 0x92b5, 0x1f2c: 0xef11, 0x1f2d: 0xef31, 0x1f2e: 0xef51, 0x1f2f: 0xef71,
-	0x1f30: 0xef91, 0x1f31: 0xefb1, 0x1f32: 0x92d5, 0x1f33: 0x92f5, 0x1f34: 0xefd1, 0x1f35: 0x9315,
-	0x1f36: 0xeff1, 0x1f37: 0x9335, 0x1f38: 0xf011, 0x1f39: 0xf031, 0x1f3a: 0xf051, 0x1f3b: 0x9355,
-	0x1f3c: 0x9375, 0x1f3d: 0xf071, 0x1f3e: 0x9395, 0x1f3f: 0xf091,
-	// Block 0x7d, offset 0x1f40
-	0x1f40: 0xf6d1, 0x1f41: 0xf6f1, 0x1f42: 0xf711, 0x1f43: 0xf731, 0x1f44: 0xf751, 0x1f45: 0x9555,
-	0x1f46: 0xf771, 0x1f47: 0xf791, 0x1f48: 0xf7b1, 0x1f49: 0xf7d1, 0x1f4a: 0xf7f1, 0x1f4b: 0x9575,
-	0x1f4c: 0x9595, 0x1f4d: 0xf811, 0x1f4e: 0xf831, 0x1f4f: 0xf851, 0x1f50: 0xf871, 0x1f51: 0xf891,
-	0x1f52: 0xf8b1, 0x1f53: 0x95b5, 0x1f54: 0xf8d1, 0x1f55: 0xf8f1, 0x1f56: 0xf911, 0x1f57: 0xf931,
-	0x1f58: 0x95d5, 0x1f59: 0x95f5, 0x1f5a: 0xf951, 0x1f5b: 0xf971, 0x1f5c: 0xf991, 0x1f5d: 0x9615,
-	0x1f5e: 0xf9b1, 0x1f5f: 0xf9d1, 0x1f60: 0x684d, 0x1f61: 0x9635, 0x1f62: 0xf9f1, 0x1f63: 0xfa11,
-	0x1f64: 0xfa31, 0x1f65: 0x9655, 0x1f66: 0xfa51, 0x1f67: 0xfa71, 0x1f68: 0xfa91, 0x1f69: 0xfab1,
-	0x1f6a: 0xfad1, 0x1f6b: 0xfaf1, 0x1f6c: 0xfb11, 0x1f6d: 0x9675, 0x1f6e: 0xfb31, 0x1f6f: 0xfb51,
-	0x1f70: 0xfb71, 0x1f71: 0x9695, 0x1f72: 0xfb91, 0x1f73: 0xfbb1, 0x1f74: 0xfbd1, 0x1f75: 0xfbf1,
-	0x1f76: 0x7b6d, 0x1f77: 0x96b5, 0x1f78: 0xfc11, 0x1f79: 0xfc31, 0x1f7a: 0xfc51, 0x1f7b: 0x96d5,
-	0x1f7c: 0xfc71, 0x1f7d: 0x96f5, 0x1f7e: 0xfc91, 0x1f7f: 0xfc91,
-	// Block 0x7e, offset 0x1f80
-	0x1f80: 0xfcb1, 0x1f81: 0x9715, 0x1f82: 0xfcd1, 0x1f83: 0xfcf1, 0x1f84: 0xfd11, 0x1f85: 0xfd31,
-	0x1f86: 0xfd51, 0x1f87: 0xfd71, 0x1f88: 0xfd91, 0x1f89: 0x9735, 0x1f8a: 0xfdb1, 0x1f8b: 0xfdd1,
-	0x1f8c: 0xfdf1, 0x1f8d: 0xfe11, 0x1f8e: 0xfe31, 0x1f8f: 0xfe51, 0x1f90: 0x9755, 0x1f91: 0xfe71,
-	0x1f92: 0x9775, 0x1f93: 0x9795, 0x1f94: 0x97b5, 0x1f95: 0xfe91, 0x1f96: 0xfeb1, 0x1f97: 0xfed1,
-	0x1f98: 0xfef1, 0x1f99: 0xff11, 0x1f9a: 0xff31, 0x1f9b: 0xff51, 0x1f9c: 0xff71, 0x1f9d: 0x97d5,
-	0x1f9e: 0x0040, 0x1f9f: 0x0040, 0x1fa0: 0x0040, 0x1fa1: 0x0040, 0x1fa2: 0x0040, 0x1fa3: 0x0040,
-	0x1fa4: 0x0040, 0x1fa5: 0x0040, 0x1fa6: 0x0040, 0x1fa7: 0x0040, 0x1fa8: 0x0040, 0x1fa9: 0x0040,
-	0x1faa: 0x0040, 0x1fab: 0x0040, 0x1fac: 0x0040, 0x1fad: 0x0040, 0x1fae: 0x0040, 0x1faf: 0x0040,
-	0x1fb0: 0x0040, 0x1fb1: 0x0040, 0x1fb2: 0x0040, 0x1fb3: 0x0040, 0x1fb4: 0x0040, 0x1fb5: 0x0040,
-	0x1fb6: 0x0040, 0x1fb7: 0x0040, 0x1fb8: 0x0040, 0x1fb9: 0x0040, 0x1fba: 0x0040, 0x1fbb: 0x0040,
-	0x1fbc: 0x0040, 0x1fbd: 0x0040, 0x1fbe: 0x0040, 0x1fbf: 0x0040,
-}
-
-// idnaIndex: 36 blocks, 2304 entries, 4608 bytes
-// Block 0 is the zero block.
-var idnaIndex = [2304]uint16{
-	// Block 0x0, offset 0x0
-	// Block 0x1, offset 0x40
-	// Block 0x2, offset 0x80
-	// Block 0x3, offset 0xc0
-	0xc2: 0x01, 0xc3: 0x7d, 0xc4: 0x02, 0xc5: 0x03, 0xc6: 0x04, 0xc7: 0x05,
-	0xc8: 0x06, 0xc9: 0x7e, 0xca: 0x7f, 0xcb: 0x07, 0xcc: 0x80, 0xcd: 0x08, 0xce: 0x09, 0xcf: 0x0a,
-	0xd0: 0x81, 0xd1: 0x0b, 0xd2: 0x0c, 0xd3: 0x0d, 0xd4: 0x0e, 0xd5: 0x82, 0xd6: 0x83, 0xd7: 0x84,
-	0xd8: 0x0f, 0xd9: 0x10, 0xda: 0x85, 0xdb: 0x11, 0xdc: 0x12, 0xdd: 0x86, 0xde: 0x87, 0xdf: 0x88,
-	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05, 0xe4: 0x06, 0xe5: 0x07, 0xe6: 0x07, 0xe7: 0x07,
-	0xe8: 0x07, 0xe9: 0x08, 0xea: 0x09, 0xeb: 0x07, 0xec: 0x07, 0xed: 0x0a, 0xee: 0x0b, 0xef: 0x0c,
-	0xf0: 0x1d, 0xf1: 0x1e, 0xf2: 0x1e, 0xf3: 0x20, 0xf4: 0x21,
-	// Block 0x4, offset 0x100
-	0x120: 0x89, 0x121: 0x13, 0x122: 0x8a, 0x123: 0x8b, 0x124: 0x8c, 0x125: 0x14, 0x126: 0x15, 0x127: 0x16,
-	0x128: 0x17, 0x129: 0x18, 0x12a: 0x19, 0x12b: 0x1a, 0x12c: 0x1b, 0x12d: 0x1c, 0x12e: 0x1d, 0x12f: 0x8d,
-	0x130: 0x8e, 0x131: 0x1e, 0x132: 0x1f, 0x133: 0x20, 0x134: 0x8f, 0x135: 0x21, 0x136: 0x90, 0x137: 0x91,
-	0x138: 0x92, 0x139: 0x93, 0x13a: 0x22, 0x13b: 0x94, 0x13c: 0x95, 0x13d: 0x23, 0x13e: 0x24, 0x13f: 0x96,
-	// Block 0x5, offset 0x140
-	0x140: 0x97, 0x141: 0x98, 0x142: 0x99, 0x143: 0x9a, 0x144: 0x9b, 0x145: 0x9c, 0x146: 0x9d, 0x147: 0x9e,
-	0x148: 0x9f, 0x149: 0xa0, 0x14a: 0xa1, 0x14b: 0xa2, 0x14c: 0xa3, 0x14d: 0xa4, 0x14e: 0xa5, 0x14f: 0xa6,
-	0x150: 0xa7, 0x151: 0x9f, 0x152: 0x9f, 0x153: 0x9f, 0x154: 0x9f, 0x155: 0x9f, 0x156: 0x9f, 0x157: 0x9f,
-	0x158: 0x9f, 0x159: 0xa8, 0x15a: 0xa9, 0x15b: 0xaa, 0x15c: 0xab, 0x15d: 0xac, 0x15e: 0xad, 0x15f: 0xae,
-	0x160: 0xaf, 0x161: 0xb0, 0x162: 0xb1, 0x163: 0xb2, 0x164: 0xb3, 0x165: 0xb4, 0x166: 0xb5, 0x167: 0xb6,
-	0x168: 0xb7, 0x169: 0xb8, 0x16a: 0xb9, 0x16b: 0xba, 0x16c: 0xbb, 0x16d: 0xbc, 0x16e: 0xbd, 0x16f: 0xbe,
-	0x170: 0xbf, 0x171: 0xc0, 0x172: 0xc1, 0x173: 0xc2, 0x174: 0x25, 0x175: 0x26, 0x176: 0x27, 0x177: 0xc3,
-	0x178: 0x28, 0x179: 0x28, 0x17a: 0x29, 0x17b: 0x28, 0x17c: 0xc4, 0x17d: 0x2a, 0x17e: 0x2b, 0x17f: 0x2c,
-	// Block 0x6, offset 0x180
-	0x180: 0x2d, 0x181: 0x2e, 0x182: 0x2f, 0x183: 0xc5, 0x184: 0x30, 0x185: 0x31, 0x186: 0xc6, 0x187: 0x9b,
-	0x188: 0xc7, 0x189: 0xc8, 0x18a: 0x9b, 0x18b: 0x9b, 0x18c: 0xc9, 0x18d: 0x9b, 0x18e: 0x9b, 0x18f: 0x9b,
-	0x190: 0xca, 0x191: 0x32, 0x192: 0x33, 0x193: 0x34, 0x194: 0x9b, 0x195: 0x9b, 0x196: 0x9b, 0x197: 0x9b,
-	0x198: 0x9b, 0x199: 0x9b, 0x19a: 0x9b, 0x19b: 0x9b, 0x19c: 0x9b, 0x19d: 0x9b, 0x19e: 0x9b, 0x19f: 0x9b,
-	0x1a0: 0x9b, 0x1a1: 0x9b, 0x1a2: 0x9b, 0x1a3: 0x9b, 0x1a4: 0x9b, 0x1a5: 0x9b, 0x1a6: 0x9b, 0x1a7: 0x9b,
-	0x1a8: 0xcb, 0x1a9: 0xcc, 0x1aa: 0x9b, 0x1ab: 0xcd, 0x1ac: 0x9b, 0x1ad: 0xce, 0x1ae: 0xcf, 0x1af: 0x9b,
-	0x1b0: 0xd0, 0x1b1: 0x35, 0x1b2: 0x28, 0x1b3: 0x36, 0x1b4: 0xd1, 0x1b5: 0xd2, 0x1b6: 0xd3, 0x1b7: 0xd4,
-	0x1b8: 0xd5, 0x1b9: 0xd6, 0x1ba: 0xd7, 0x1bb: 0xd8, 0x1bc: 0xd9, 0x1bd: 0xda, 0x1be: 0xdb, 0x1bf: 0x37,
-	// Block 0x7, offset 0x1c0
-	0x1c0: 0x38, 0x1c1: 0xdc, 0x1c2: 0xdd, 0x1c3: 0xde, 0x1c4: 0xdf, 0x1c5: 0x39, 0x1c6: 0x3a, 0x1c7: 0xe0,
-	0x1c8: 0xe1, 0x1c9: 0x3b, 0x1ca: 0x3c, 0x1cb: 0x3d, 0x1cc: 0x3e, 0x1cd: 0x3f, 0x1ce: 0x40, 0x1cf: 0x41,
-	0x1d0: 0x9f, 0x1d1: 0x9f, 0x1d2: 0x9f, 0x1d3: 0x9f, 0x1d4: 0x9f, 0x1d5: 0x9f, 0x1d6: 0x9f, 0x1d7: 0x9f,
-	0x1d8: 0x9f, 0x1d9: 0x9f, 0x1da: 0x9f, 0x1db: 0x9f, 0x1dc: 0x9f, 0x1dd: 0x9f, 0x1de: 0x9f, 0x1df: 0x9f,
-	0x1e0: 0x9f, 0x1e1: 0x9f, 0x1e2: 0x9f, 0x1e3: 0x9f, 0x1e4: 0x9f, 0x1e5: 0x9f, 0x1e6: 0x9f, 0x1e7: 0x9f,
-	0x1e8: 0x9f, 0x1e9: 0x9f, 0x1ea: 0x9f, 0x1eb: 0x9f, 0x1ec: 0x9f, 0x1ed: 0x9f, 0x1ee: 0x9f, 0x1ef: 0x9f,
-	0x1f0: 0x9f, 0x1f1: 0x9f, 0x1f2: 0x9f, 0x1f3: 0x9f, 0x1f4: 0x9f, 0x1f5: 0x9f, 0x1f6: 0x9f, 0x1f7: 0x9f,
-	0x1f8: 0x9f, 0x1f9: 0x9f, 0x1fa: 0x9f, 0x1fb: 0x9f, 0x1fc: 0x9f, 0x1fd: 0x9f, 0x1fe: 0x9f, 0x1ff: 0x9f,
-	// Block 0x8, offset 0x200
-	0x200: 0x9f, 0x201: 0x9f, 0x202: 0x9f, 0x203: 0x9f, 0x204: 0x9f, 0x205: 0x9f, 0x206: 0x9f, 0x207: 0x9f,
-	0x208: 0x9f, 0x209: 0x9f, 0x20a: 0x9f, 0x20b: 0x9f, 0x20c: 0x9f, 0x20d: 0x9f, 0x20e: 0x9f, 0x20f: 0x9f,
-	0x210: 0x9f, 0x211: 0x9f, 0x212: 0x9f, 0x213: 0x9f, 0x214: 0x9f, 0x215: 0x9f, 0x216: 0x9f, 0x217: 0x9f,
-	0x218: 0x9f, 0x219: 0x9f, 0x21a: 0x9f, 0x21b: 0x9f, 0x21c: 0x9f, 0x21d: 0x9f, 0x21e: 0x9f, 0x21f: 0x9f,
-	0x220: 0x9f, 0x221: 0x9f, 0x222: 0x9f, 0x223: 0x9f, 0x224: 0x9f, 0x225: 0x9f, 0x226: 0x9f, 0x227: 0x9f,
-	0x228: 0x9f, 0x229: 0x9f, 0x22a: 0x9f, 0x22b: 0x9f, 0x22c: 0x9f, 0x22d: 0x9f, 0x22e: 0x9f, 0x22f: 0x9f,
-	0x230: 0x9f, 0x231: 0x9f, 0x232: 0x9f, 0x233: 0x9f, 0x234: 0x9f, 0x235: 0x9f, 0x236: 0xb2, 0x237: 0x9b,
-	0x238: 0x9f, 0x239: 0x9f, 0x23a: 0x9f, 0x23b: 0x9f, 0x23c: 0x9f, 0x23d: 0x9f, 0x23e: 0x9f, 0x23f: 0x9f,
-	// Block 0x9, offset 0x240
-	0x240: 0x9f, 0x241: 0x9f, 0x242: 0x9f, 0x243: 0x9f, 0x244: 0x9f, 0x245: 0x9f, 0x246: 0x9f, 0x247: 0x9f,
-	0x248: 0x9f, 0x249: 0x9f, 0x24a: 0x9f, 0x24b: 0x9f, 0x24c: 0x9f, 0x24d: 0x9f, 0x24e: 0x9f, 0x24f: 0x9f,
-	0x250: 0x9f, 0x251: 0x9f, 0x252: 0x9f, 0x253: 0x9f, 0x254: 0x9f, 0x255: 0x9f, 0x256: 0x9f, 0x257: 0x9f,
-	0x258: 0x9f, 0x259: 0x9f, 0x25a: 0x9f, 0x25b: 0x9f, 0x25c: 0x9f, 0x25d: 0x9f, 0x25e: 0x9f, 0x25f: 0x9f,
-	0x260: 0x9f, 0x261: 0x9f, 0x262: 0x9f, 0x263: 0x9f, 0x264: 0x9f, 0x265: 0x9f, 0x266: 0x9f, 0x267: 0x9f,
-	0x268: 0x9f, 0x269: 0x9f, 0x26a: 0x9f, 0x26b: 0x9f, 0x26c: 0x9f, 0x26d: 0x9f, 0x26e: 0x9f, 0x26f: 0x9f,
-	0x270: 0x9f, 0x271: 0x9f, 0x272: 0x9f, 0x273: 0x9f, 0x274: 0x9f, 0x275: 0x9f, 0x276: 0x9f, 0x277: 0x9f,
-	0x278: 0x9f, 0x279: 0x9f, 0x27a: 0x9f, 0x27b: 0x9f, 0x27c: 0x9f, 0x27d: 0x9f, 0x27e: 0x9f, 0x27f: 0x9f,
-	// Block 0xa, offset 0x280
-	0x280: 0x9f, 0x281: 0x9f, 0x282: 0x9f, 0x283: 0x9f, 0x284: 0x9f, 0x285: 0x9f, 0x286: 0x9f, 0x287: 0x9f,
-	0x288: 0x9f, 0x289: 0x9f, 0x28a: 0x9f, 0x28b: 0x9f, 0x28c: 0x9f, 0x28d: 0x9f, 0x28e: 0x9f, 0x28f: 0x9f,
-	0x290: 0x9f, 0x291: 0x9f, 0x292: 0x9f, 0x293: 0x9f, 0x294: 0x9f, 0x295: 0x9f, 0x296: 0x9f, 0x297: 0x9f,
-	0x298: 0x9f, 0x299: 0x9f, 0x29a: 0x9f, 0x29b: 0x9f, 0x29c: 0x9f, 0x29d: 0x9f, 0x29e: 0x9f, 0x29f: 0x9f,
-	0x2a0: 0x9f, 0x2a1: 0x9f, 0x2a2: 0x9f, 0x2a3: 0x9f, 0x2a4: 0x9f, 0x2a5: 0x9f, 0x2a6: 0x9f, 0x2a7: 0x9f,
-	0x2a8: 0x9f, 0x2a9: 0x9f, 0x2aa: 0x9f, 0x2ab: 0x9f, 0x2ac: 0x9f, 0x2ad: 0x9f, 0x2ae: 0x9f, 0x2af: 0x9f,
-	0x2b0: 0x9f, 0x2b1: 0x9f, 0x2b2: 0x9f, 0x2b3: 0x9f, 0x2b4: 0x9f, 0x2b5: 0x9f, 0x2b6: 0x9f, 0x2b7: 0x9f,
-	0x2b8: 0x9f, 0x2b9: 0x9f, 0x2ba: 0x9f, 0x2bb: 0x9f, 0x2bc: 0x9f, 0x2bd: 0x9f, 0x2be: 0x9f, 0x2bf: 0xe2,
-	// Block 0xb, offset 0x2c0
-	0x2c0: 0x9f, 0x2c1: 0x9f, 0x2c2: 0x9f, 0x2c3: 0x9f, 0x2c4: 0x9f, 0x2c5: 0x9f, 0x2c6: 0x9f, 0x2c7: 0x9f,
-	0x2c8: 0x9f, 0x2c9: 0x9f, 0x2ca: 0x9f, 0x2cb: 0x9f, 0x2cc: 0x9f, 0x2cd: 0x9f, 0x2ce: 0x9f, 0x2cf: 0x9f,
-	0x2d0: 0x9f, 0x2d1: 0x9f, 0x2d2: 0xe3, 0x2d3: 0xe4, 0x2d4: 0x9f, 0x2d5: 0x9f, 0x2d6: 0x9f, 0x2d7: 0x9f,
-	0x2d8: 0xe5, 0x2d9: 0x42, 0x2da: 0x43, 0x2db: 0xe6, 0x2dc: 0x44, 0x2dd: 0x45, 0x2de: 0x46, 0x2df: 0xe7,
-	0x2e0: 0xe8, 0x2e1: 0xe9, 0x2e2: 0xea, 0x2e3: 0xeb, 0x2e4: 0xec, 0x2e5: 0xed, 0x2e6: 0xee, 0x2e7: 0xef,
-	0x2e8: 0xf0, 0x2e9: 0xf1, 0x2ea: 0xf2, 0x2eb: 0xf3, 0x2ec: 0xf4, 0x2ed: 0xf5, 0x2ee: 0xf6, 0x2ef: 0xf7,
-	0x2f0: 0x9f, 0x2f1: 0x9f, 0x2f2: 0x9f, 0x2f3: 0x9f, 0x2f4: 0x9f, 0x2f5: 0x9f, 0x2f6: 0x9f, 0x2f7: 0x9f,
-	0x2f8: 0x9f, 0x2f9: 0x9f, 0x2fa: 0x9f, 0x2fb: 0x9f, 0x2fc: 0x9f, 0x2fd: 0x9f, 0x2fe: 0x9f, 0x2ff: 0x9f,
-	// Block 0xc, offset 0x300
-	0x300: 0x9f, 0x301: 0x9f, 0x302: 0x9f, 0x303: 0x9f, 0x304: 0x9f, 0x305: 0x9f, 0x306: 0x9f, 0x307: 0x9f,
-	0x308: 0x9f, 0x309: 0x9f, 0x30a: 0x9f, 0x30b: 0x9f, 0x30c: 0x9f, 0x30d: 0x9f, 0x30e: 0x9f, 0x30f: 0x9f,
-	0x310: 0x9f, 0x311: 0x9f, 0x312: 0x9f, 0x313: 0x9f, 0x314: 0x9f, 0x315: 0x9f, 0x316: 0x9f, 0x317: 0x9f,
-	0x318: 0x9f, 0x319: 0x9f, 0x31a: 0x9f, 0x31b: 0x9f, 0x31c: 0x9f, 0x31d: 0x9f, 0x31e: 0xf8, 0x31f: 0xf9,
-	// Block 0xd, offset 0x340
-	0x340: 0xba, 0x341: 0xba, 0x342: 0xba, 0x343: 0xba, 0x344: 0xba, 0x345: 0xba, 0x346: 0xba, 0x347: 0xba,
-	0x348: 0xba, 0x349: 0xba, 0x34a: 0xba, 0x34b: 0xba, 0x34c: 0xba, 0x34d: 0xba, 0x34e: 0xba, 0x34f: 0xba,
-	0x350: 0xba, 0x351: 0xba, 0x352: 0xba, 0x353: 0xba, 0x354: 0xba, 0x355: 0xba, 0x356: 0xba, 0x357: 0xba,
-	0x358: 0xba, 0x359: 0xba, 0x35a: 0xba, 0x35b: 0xba, 0x35c: 0xba, 0x35d: 0xba, 0x35e: 0xba, 0x35f: 0xba,
-	0x360: 0xba, 0x361: 0xba, 0x362: 0xba, 0x363: 0xba, 0x364: 0xba, 0x365: 0xba, 0x366: 0xba, 0x367: 0xba,
-	0x368: 0xba, 0x369: 0xba, 0x36a: 0xba, 0x36b: 0xba, 0x36c: 0xba, 0x36d: 0xba, 0x36e: 0xba, 0x36f: 0xba,
-	0x370: 0xba, 0x371: 0xba, 0x372: 0xba, 0x373: 0xba, 0x374: 0xba, 0x375: 0xba, 0x376: 0xba, 0x377: 0xba,
-	0x378: 0xba, 0x379: 0xba, 0x37a: 0xba, 0x37b: 0xba, 0x37c: 0xba, 0x37d: 0xba, 0x37e: 0xba, 0x37f: 0xba,
-	// Block 0xe, offset 0x380
-	0x380: 0xba, 0x381: 0xba, 0x382: 0xba, 0x383: 0xba, 0x384: 0xba, 0x385: 0xba, 0x386: 0xba, 0x387: 0xba,
-	0x388: 0xba, 0x389: 0xba, 0x38a: 0xba, 0x38b: 0xba, 0x38c: 0xba, 0x38d: 0xba, 0x38e: 0xba, 0x38f: 0xba,
-	0x390: 0xba, 0x391: 0xba, 0x392: 0xba, 0x393: 0xba, 0x394: 0xba, 0x395: 0xba, 0x396: 0xba, 0x397: 0xba,
-	0x398: 0xba, 0x399: 0xba, 0x39a: 0xba, 0x39b: 0xba, 0x39c: 0xba, 0x39d: 0xba, 0x39e: 0xba, 0x39f: 0xba,
-	0x3a0: 0xba, 0x3a1: 0xba, 0x3a2: 0xba, 0x3a3: 0xba, 0x3a4: 0xfa, 0x3a5: 0xfb, 0x3a6: 0xfc, 0x3a7: 0xfd,
-	0x3a8: 0x47, 0x3a9: 0xfe, 0x3aa: 0xff, 0x3ab: 0x48, 0x3ac: 0x49, 0x3ad: 0x4a, 0x3ae: 0x4b, 0x3af: 0x4c,
-	0x3b0: 0x100, 0x3b1: 0x4d, 0x3b2: 0x4e, 0x3b3: 0x4f, 0x3b4: 0x50, 0x3b5: 0x51, 0x3b6: 0x101, 0x3b7: 0x52,
-	0x3b8: 0x53, 0x3b9: 0x54, 0x3ba: 0x55, 0x3bb: 0x56, 0x3bc: 0x57, 0x3bd: 0x58, 0x3be: 0x59, 0x3bf: 0x5a,
-	// Block 0xf, offset 0x3c0
-	0x3c0: 0x102, 0x3c1: 0x103, 0x3c2: 0x9f, 0x3c3: 0x104, 0x3c4: 0x105, 0x3c5: 0x9b, 0x3c6: 0x106, 0x3c7: 0x107,
-	0x3c8: 0xba, 0x3c9: 0xba, 0x3ca: 0x108, 0x3cb: 0x109, 0x3cc: 0x10a, 0x3cd: 0x10b, 0x3ce: 0x10c, 0x3cf: 0x10d,
-	0x3d0: 0x10e, 0x3d1: 0x9f, 0x3d2: 0x10f, 0x3d3: 0x110, 0x3d4: 0x111, 0x3d5: 0x112, 0x3d6: 0xba, 0x3d7: 0xba,
-	0x3d8: 0x9f, 0x3d9: 0x9f, 0x3da: 0x9f, 0x3db: 0x9f, 0x3dc: 0x113, 0x3dd: 0x114, 0x3de: 0xba, 0x3df: 0xba,
-	0x3e0: 0x115, 0x3e1: 0x116, 0x3e2: 0x117, 0x3e3: 0x118, 0x3e4: 0x119, 0x3e5: 0xba, 0x3e6: 0x11a, 0x3e7: 0x11b,
-	0x3e8: 0x11c, 0x3e9: 0x11d, 0x3ea: 0x11e, 0x3eb: 0x5b, 0x3ec: 0x11f, 0x3ed: 0x120, 0x3ee: 0x5c, 0x3ef: 0xba,
-	0x3f0: 0x121, 0x3f1: 0x122, 0x3f2: 0x123, 0x3f3: 0x124, 0x3f4: 0x125, 0x3f5: 0xba, 0x3f6: 0xba, 0x3f7: 0xba,
-	0x3f8: 0xba, 0x3f9: 0x126, 0x3fa: 0xba, 0x3fb: 0xba, 0x3fc: 0x127, 0x3fd: 0x128, 0x3fe: 0xba, 0x3ff: 0x129,
-	// Block 0x10, offset 0x400
-	0x400: 0x12a, 0x401: 0x12b, 0x402: 0x12c, 0x403: 0x12d, 0x404: 0x12e, 0x405: 0x12f, 0x406: 0x130, 0x407: 0x131,
-	0x408: 0x132, 0x409: 0xba, 0x40a: 0x133, 0x40b: 0x134, 0x40c: 0x5d, 0x40d: 0x5e, 0x40e: 0xba, 0x40f: 0xba,
-	0x410: 0x135, 0x411: 0x136, 0x412: 0x137, 0x413: 0x138, 0x414: 0xba, 0x415: 0xba, 0x416: 0x139, 0x417: 0x13a,
-	0x418: 0x13b, 0x419: 0x13c, 0x41a: 0x13d, 0x41b: 0x13e, 0x41c: 0x13f, 0x41d: 0xba, 0x41e: 0xba, 0x41f: 0xba,
-	0x420: 0x140, 0x421: 0xba, 0x422: 0x141, 0x423: 0x142, 0x424: 0xba, 0x425: 0xba, 0x426: 0x143, 0x427: 0x144,
-	0x428: 0x145, 0x429: 0x146, 0x42a: 0x147, 0x42b: 0x148, 0x42c: 0xba, 0x42d: 0xba, 0x42e: 0xba, 0x42f: 0xba,
-	0x430: 0x149, 0x431: 0x14a, 0x432: 0x14b, 0x433: 0xba, 0x434: 0x14c, 0x435: 0x14d, 0x436: 0x14e, 0x437: 0xba,
-	0x438: 0xba, 0x439: 0xba, 0x43a: 0xba, 0x43b: 0x14f, 0x43c: 0xba, 0x43d: 0xba, 0x43e: 0xba, 0x43f: 0x150,
-	// Block 0x11, offset 0x440
-	0x440: 0x9f, 0x441: 0x9f, 0x442: 0x9f, 0x443: 0x9f, 0x444: 0x9f, 0x445: 0x9f, 0x446: 0x9f, 0x447: 0x9f,
-	0x448: 0x9f, 0x449: 0x9f, 0x44a: 0x9f, 0x44b: 0x9f, 0x44c: 0x9f, 0x44d: 0x9f, 0x44e: 0x151, 0x44f: 0xba,
-	0x450: 0x9b, 0x451: 0x152, 0x452: 0x9f, 0x453: 0x9f, 0x454: 0x9f, 0x455: 0x153, 0x456: 0xba, 0x457: 0xba,
-	0x458: 0xba, 0x459: 0xba, 0x45a: 0xba, 0x45b: 0xba, 0x45c: 0xba, 0x45d: 0xba, 0x45e: 0xba, 0x45f: 0xba,
-	0x460: 0xba, 0x461: 0xba, 0x462: 0xba, 0x463: 0xba, 0x464: 0xba, 0x465: 0xba, 0x466: 0xba, 0x467: 0xba,
-	0x468: 0xba, 0x469: 0xba, 0x46a: 0xba, 0x46b: 0xba, 0x46c: 0xba, 0x46d: 0xba, 0x46e: 0xba, 0x46f: 0xba,
-	0x470: 0xba, 0x471: 0xba, 0x472: 0xba, 0x473: 0xba, 0x474: 0xba, 0x475: 0xba, 0x476: 0xba, 0x477: 0xba,
-	0x478: 0xba, 0x479: 0xba, 0x47a: 0xba, 0x47b: 0xba, 0x47c: 0xba, 0x47d: 0xba, 0x47e: 0xba, 0x47f: 0xba,
-	// Block 0x12, offset 0x480
-	0x480: 0x9f, 0x481: 0x9f, 0x482: 0x9f, 0x483: 0x9f, 0x484: 0x9f, 0x485: 0x9f, 0x486: 0x9f, 0x487: 0x9f,
-	0x488: 0x9f, 0x489: 0x9f, 0x48a: 0x9f, 0x48b: 0x9f, 0x48c: 0x9f, 0x48d: 0x9f, 0x48e: 0x9f, 0x48f: 0x9f,
-	0x490: 0x154, 0x491: 0xba, 0x492: 0xba, 0x493: 0xba, 0x494: 0xba, 0x495: 0xba, 0x496: 0xba, 0x497: 0xba,
-	0x498: 0xba, 0x499: 0xba, 0x49a: 0xba, 0x49b: 0xba, 0x49c: 0xba, 0x49d: 0xba, 0x49e: 0xba, 0x49f: 0xba,
-	0x4a0: 0xba, 0x4a1: 0xba, 0x4a2: 0xba, 0x4a3: 0xba, 0x4a4: 0xba, 0x4a5: 0xba, 0x4a6: 0xba, 0x4a7: 0xba,
-	0x4a8: 0xba, 0x4a9: 0xba, 0x4aa: 0xba, 0x4ab: 0xba, 0x4ac: 0xba, 0x4ad: 0xba, 0x4ae: 0xba, 0x4af: 0xba,
-	0x4b0: 0xba, 0x4b1: 0xba, 0x4b2: 0xba, 0x4b3: 0xba, 0x4b4: 0xba, 0x4b5: 0xba, 0x4b6: 0xba, 0x4b7: 0xba,
-	0x4b8: 0xba, 0x4b9: 0xba, 0x4ba: 0xba, 0x4bb: 0xba, 0x4bc: 0xba, 0x4bd: 0xba, 0x4be: 0xba, 0x4bf: 0xba,
-	// Block 0x13, offset 0x4c0
-	0x4c0: 0xba, 0x4c1: 0xba, 0x4c2: 0xba, 0x4c3: 0xba, 0x4c4: 0xba, 0x4c5: 0xba, 0x4c6: 0xba, 0x4c7: 0xba,
-	0x4c8: 0xba, 0x4c9: 0xba, 0x4ca: 0xba, 0x4cb: 0xba, 0x4cc: 0xba, 0x4cd: 0xba, 0x4ce: 0xba, 0x4cf: 0xba,
-	0x4d0: 0x9f, 0x4d1: 0x9f, 0x4d2: 0x9f, 0x4d3: 0x9f, 0x4d4: 0x9f, 0x4d5: 0x9f, 0x4d6: 0x9f, 0x4d7: 0x9f,
-	0x4d8: 0x9f, 0x4d9: 0x155, 0x4da: 0xba, 0x4db: 0xba, 0x4dc: 0xba, 0x4dd: 0xba, 0x4de: 0xba, 0x4df: 0xba,
-	0x4e0: 0xba, 0x4e1: 0xba, 0x4e2: 0xba, 0x4e3: 0xba, 0x4e4: 0xba, 0x4e5: 0xba, 0x4e6: 0xba, 0x4e7: 0xba,
-	0x4e8: 0xba, 0x4e9: 0xba, 0x4ea: 0xba, 0x4eb: 0xba, 0x4ec: 0xba, 0x4ed: 0xba, 0x4ee: 0xba, 0x4ef: 0xba,
-	0x4f0: 0xba, 0x4f1: 0xba, 0x4f2: 0xba, 0x4f3: 0xba, 0x4f4: 0xba, 0x4f5: 0xba, 0x4f6: 0xba, 0x4f7: 0xba,
-	0x4f8: 0xba, 0x4f9: 0xba, 0x4fa: 0xba, 0x4fb: 0xba, 0x4fc: 0xba, 0x4fd: 0xba, 0x4fe: 0xba, 0x4ff: 0xba,
-	// Block 0x14, offset 0x500
-	0x500: 0xba, 0x501: 0xba, 0x502: 0xba, 0x503: 0xba, 0x504: 0xba, 0x505: 0xba, 0x506: 0xba, 0x507: 0xba,
-	0x508: 0xba, 0x509: 0xba, 0x50a: 0xba, 0x50b: 0xba, 0x50c: 0xba, 0x50d: 0xba, 0x50e: 0xba, 0x50f: 0xba,
-	0x510: 0xba, 0x511: 0xba, 0x512: 0xba, 0x513: 0xba, 0x514: 0xba, 0x515: 0xba, 0x516: 0xba, 0x517: 0xba,
-	0x518: 0xba, 0x519: 0xba, 0x51a: 0xba, 0x51b: 0xba, 0x51c: 0xba, 0x51d: 0xba, 0x51e: 0xba, 0x51f: 0xba,
-	0x520: 0x9f, 0x521: 0x9f, 0x522: 0x9f, 0x523: 0x9f, 0x524: 0x9f, 0x525: 0x9f, 0x526: 0x9f, 0x527: 0x9f,
-	0x528: 0x148, 0x529: 0x156, 0x52a: 0xba, 0x52b: 0x157, 0x52c: 0x158, 0x52d: 0x159, 0x52e: 0x15a, 0x52f: 0xba,
-	0x530: 0xba, 0x531: 0xba, 0x532: 0xba, 0x533: 0xba, 0x534: 0xba, 0x535: 0xba, 0x536: 0xba, 0x537: 0xba,
-	0x538: 0xba, 0x539: 0x15b, 0x53a: 0x15c, 0x53b: 0xba, 0x53c: 0x9f, 0x53d: 0x15d, 0x53e: 0x15e, 0x53f: 0x15f,
-	// Block 0x15, offset 0x540
-	0x540: 0x9f, 0x541: 0x9f, 0x542: 0x9f, 0x543: 0x9f, 0x544: 0x9f, 0x545: 0x9f, 0x546: 0x9f, 0x547: 0x9f,
-	0x548: 0x9f, 0x549: 0x9f, 0x54a: 0x9f, 0x54b: 0x9f, 0x54c: 0x9f, 0x54d: 0x9f, 0x54e: 0x9f, 0x54f: 0x9f,
-	0x550: 0x9f, 0x551: 0x9f, 0x552: 0x9f, 0x553: 0x9f, 0x554: 0x9f, 0x555: 0x9f, 0x556: 0x9f, 0x557: 0x9f,
-	0x558: 0x9f, 0x559: 0x9f, 0x55a: 0x9f, 0x55b: 0x9f, 0x55c: 0x9f, 0x55d: 0x9f, 0x55e: 0x9f, 0x55f: 0x160,
-	0x560: 0x9f, 0x561: 0x9f, 0x562: 0x9f, 0x563: 0x9f, 0x564: 0x9f, 0x565: 0x9f, 0x566: 0x9f, 0x567: 0x9f,
-	0x568: 0x9f, 0x569: 0x9f, 0x56a: 0x9f, 0x56b: 0x161, 0x56c: 0xba, 0x56d: 0xba, 0x56e: 0xba, 0x56f: 0xba,
-	0x570: 0xba, 0x571: 0xba, 0x572: 0xba, 0x573: 0xba, 0x574: 0xba, 0x575: 0xba, 0x576: 0xba, 0x577: 0xba,
-	0x578: 0xba, 0x579: 0xba, 0x57a: 0xba, 0x57b: 0xba, 0x57c: 0xba, 0x57d: 0xba, 0x57e: 0xba, 0x57f: 0xba,
-	// Block 0x16, offset 0x580
-	0x580: 0x9f, 0x581: 0x9f, 0x582: 0x9f, 0x583: 0x9f, 0x584: 0x162, 0x585: 0x163, 0x586: 0x9f, 0x587: 0x9f,
-	0x588: 0x9f, 0x589: 0x9f, 0x58a: 0x9f, 0x58b: 0x164, 0x58c: 0xba, 0x58d: 0xba, 0x58e: 0xba, 0x58f: 0xba,
-	0x590: 0xba, 0x591: 0xba, 0x592: 0xba, 0x593: 0xba, 0x594: 0xba, 0x595: 0xba, 0x596: 0xba, 0x597: 0xba,
-	0x598: 0xba, 0x599: 0xba, 0x59a: 0xba, 0x59b: 0xba, 0x59c: 0xba, 0x59d: 0xba, 0x59e: 0xba, 0x59f: 0xba,
-	0x5a0: 0xba, 0x5a1: 0xba, 0x5a2: 0xba, 0x5a3: 0xba, 0x5a4: 0xba, 0x5a5: 0xba, 0x5a6: 0xba, 0x5a7: 0xba,
-	0x5a8: 0xba, 0x5a9: 0xba, 0x5aa: 0xba, 0x5ab: 0xba, 0x5ac: 0xba, 0x5ad: 0xba, 0x5ae: 0xba, 0x5af: 0xba,
-	0x5b0: 0x9f, 0x5b1: 0x165, 0x5b2: 0x166, 0x5b3: 0xba, 0x5b4: 0xba, 0x5b5: 0xba, 0x5b6: 0xba, 0x5b7: 0xba,
-	0x5b8: 0xba, 0x5b9: 0xba, 0x5ba: 0xba, 0x5bb: 0xba, 0x5bc: 0xba, 0x5bd: 0xba, 0x5be: 0xba, 0x5bf: 0xba,
-	// Block 0x17, offset 0x5c0
-	0x5c0: 0x9b, 0x5c1: 0x9b, 0x5c2: 0x9b, 0x5c3: 0x167, 0x5c4: 0x168, 0x5c5: 0x169, 0x5c6: 0x16a, 0x5c7: 0x16b,
-	0x5c8: 0x9b, 0x5c9: 0x16c, 0x5ca: 0xba, 0x5cb: 0x16d, 0x5cc: 0x9b, 0x5cd: 0x16e, 0x5ce: 0xba, 0x5cf: 0xba,
-	0x5d0: 0x5f, 0x5d1: 0x60, 0x5d2: 0x61, 0x5d3: 0x62, 0x5d4: 0x63, 0x5d5: 0x64, 0x5d6: 0x65, 0x5d7: 0x66,
-	0x5d8: 0x67, 0x5d9: 0x68, 0x5da: 0x69, 0x5db: 0x6a, 0x5dc: 0x6b, 0x5dd: 0x6c, 0x5de: 0x6d, 0x5df: 0x6e,
-	0x5e0: 0x9b, 0x5e1: 0x9b, 0x5e2: 0x9b, 0x5e3: 0x9b, 0x5e4: 0x9b, 0x5e5: 0x9b, 0x5e6: 0x9b, 0x5e7: 0x9b,
-	0x5e8: 0x16f, 0x5e9: 0x170, 0x5ea: 0x171, 0x5eb: 0xba, 0x5ec: 0xba, 0x5ed: 0xba, 0x5ee: 0xba, 0x5ef: 0xba,
-	0x5f0: 0xba, 0x5f1: 0xba, 0x5f2: 0xba, 0x5f3: 0xba, 0x5f4: 0xba, 0x5f5: 0xba, 0x5f6: 0xba, 0x5f7: 0xba,
-	0x5f8: 0xba, 0x5f9: 0xba, 0x5fa: 0xba, 0x5fb: 0xba, 0x5fc: 0xba, 0x5fd: 0xba, 0x5fe: 0xba, 0x5ff: 0xba,
-	// Block 0x18, offset 0x600
-	0x600: 0x172, 0x601: 0xba, 0x602: 0xba, 0x603: 0xba, 0x604: 0x173, 0x605: 0x174, 0x606: 0xba, 0x607: 0xba,
-	0x608: 0xba, 0x609: 0xba, 0x60a: 0xba, 0x60b: 0x175, 0x60c: 0xba, 0x60d: 0xba, 0x60e: 0xba, 0x60f: 0xba,
-	0x610: 0xba, 0x611: 0xba, 0x612: 0xba, 0x613: 0xba, 0x614: 0xba, 0x615: 0xba, 0x616: 0xba, 0x617: 0xba,
-	0x618: 0xba, 0x619: 0xba, 0x61a: 0xba, 0x61b: 0xba, 0x61c: 0xba, 0x61d: 0xba, 0x61e: 0xba, 0x61f: 0xba,
-	0x620: 0x121, 0x621: 0x121, 0x622: 0x121, 0x623: 0x176, 0x624: 0x6f, 0x625: 0x177, 0x626: 0xba, 0x627: 0xba,
-	0x628: 0xba, 0x629: 0xba, 0x62a: 0xba, 0x62b: 0xba, 0x62c: 0xba, 0x62d: 0xba, 0x62e: 0xba, 0x62f: 0xba,
-	0x630: 0xba, 0x631: 0x178, 0x632: 0x179, 0x633: 0xba, 0x634: 0x17a, 0x635: 0xba, 0x636: 0xba, 0x637: 0xba,
-	0x638: 0x70, 0x639: 0x71, 0x63a: 0x72, 0x63b: 0x17b, 0x63c: 0xba, 0x63d: 0xba, 0x63e: 0xba, 0x63f: 0xba,
-	// Block 0x19, offset 0x640
-	0x640: 0x17c, 0x641: 0x9b, 0x642: 0x17d, 0x643: 0x17e, 0x644: 0x73, 0x645: 0x74, 0x646: 0x17f, 0x647: 0x180,
-	0x648: 0x75, 0x649: 0x181, 0x64a: 0xba, 0x64b: 0xba, 0x64c: 0x9b, 0x64d: 0x9b, 0x64e: 0x9b, 0x64f: 0x9b,
-	0x650: 0x9b, 0x651: 0x9b, 0x652: 0x9b, 0x653: 0x9b, 0x654: 0x9b, 0x655: 0x9b, 0x656: 0x9b, 0x657: 0x9b,
-	0x658: 0x9b, 0x659: 0x9b, 0x65a: 0x9b, 0x65b: 0x182, 0x65c: 0x9b, 0x65d: 0x183, 0x65e: 0x9b, 0x65f: 0x184,
-	0x660: 0x185, 0x661: 0x186, 0x662: 0x187, 0x663: 0xba, 0x664: 0x188, 0x665: 0x189, 0x666: 0x18a, 0x667: 0x18b,
-	0x668: 0x9b, 0x669: 0x18c, 0x66a: 0x18d, 0x66b: 0xba, 0x66c: 0xba, 0x66d: 0xba, 0x66e: 0xba, 0x66f: 0xba,
-	0x670: 0xba, 0x671: 0xba, 0x672: 0xba, 0x673: 0xba, 0x674: 0xba, 0x675: 0xba, 0x676: 0xba, 0x677: 0xba,
-	0x678: 0xba, 0x679: 0xba, 0x67a: 0xba, 0x67b: 0xba, 0x67c: 0xba, 0x67d: 0xba, 0x67e: 0xba, 0x67f: 0xba,
-	// Block 0x1a, offset 0x680
-	0x680: 0x9f, 0x681: 0x9f, 0x682: 0x9f, 0x683: 0x9f, 0x684: 0x9f, 0x685: 0x9f, 0x686: 0x9f, 0x687: 0x9f,
-	0x688: 0x9f, 0x689: 0x9f, 0x68a: 0x9f, 0x68b: 0x9f, 0x68c: 0x9f, 0x68d: 0x9f, 0x68e: 0x9f, 0x68f: 0x9f,
-	0x690: 0x9f, 0x691: 0x9f, 0x692: 0x9f, 0x693: 0x9f, 0x694: 0x9f, 0x695: 0x9f, 0x696: 0x9f, 0x697: 0x9f,
-	0x698: 0x9f, 0x699: 0x9f, 0x69a: 0x9f, 0x69b: 0x18e, 0x69c: 0x9f, 0x69d: 0x9f, 0x69e: 0x9f, 0x69f: 0x9f,
-	0x6a0: 0x9f, 0x6a1: 0x9f, 0x6a2: 0x9f, 0x6a3: 0x9f, 0x6a4: 0x9f, 0x6a5: 0x9f, 0x6a6: 0x9f, 0x6a7: 0x9f,
-	0x6a8: 0x9f, 0x6a9: 0x9f, 0x6aa: 0x9f, 0x6ab: 0x9f, 0x6ac: 0x9f, 0x6ad: 0x9f, 0x6ae: 0x9f, 0x6af: 0x9f,
-	0x6b0: 0x9f, 0x6b1: 0x9f, 0x6b2: 0x9f, 0x6b3: 0x9f, 0x6b4: 0x9f, 0x6b5: 0x9f, 0x6b6: 0x9f, 0x6b7: 0x9f,
-	0x6b8: 0x9f, 0x6b9: 0x9f, 0x6ba: 0x9f, 0x6bb: 0x9f, 0x6bc: 0x9f, 0x6bd: 0x9f, 0x6be: 0x9f, 0x6bf: 0x9f,
-	// Block 0x1b, offset 0x6c0
-	0x6c0: 0x9f, 0x6c1: 0x9f, 0x6c2: 0x9f, 0x6c3: 0x9f, 0x6c4: 0x9f, 0x6c5: 0x9f, 0x6c6: 0x9f, 0x6c7: 0x9f,
-	0x6c8: 0x9f, 0x6c9: 0x9f, 0x6ca: 0x9f, 0x6cb: 0x9f, 0x6cc: 0x9f, 0x6cd: 0x9f, 0x6ce: 0x9f, 0x6cf: 0x9f,
-	0x6d0: 0x9f, 0x6d1: 0x9f, 0x6d2: 0x9f, 0x6d3: 0x9f, 0x6d4: 0x9f, 0x6d5: 0x9f, 0x6d6: 0x9f, 0x6d7: 0x9f,
-	0x6d8: 0x9f, 0x6d9: 0x9f, 0x6da: 0x9f, 0x6db: 0x9f, 0x6dc: 0x18f, 0x6dd: 0x9f, 0x6de: 0x9f, 0x6df: 0x9f,
-	0x6e0: 0x190, 0x6e1: 0x9f, 0x6e2: 0x9f, 0x6e3: 0x9f, 0x6e4: 0x9f, 0x6e5: 0x9f, 0x6e6: 0x9f, 0x6e7: 0x9f,
-	0x6e8: 0x9f, 0x6e9: 0x9f, 0x6ea: 0x9f, 0x6eb: 0x9f, 0x6ec: 0x9f, 0x6ed: 0x9f, 0x6ee: 0x9f, 0x6ef: 0x9f,
-	0x6f0: 0x9f, 0x6f1: 0x9f, 0x6f2: 0x9f, 0x6f3: 0x9f, 0x6f4: 0x9f, 0x6f5: 0x9f, 0x6f6: 0x9f, 0x6f7: 0x9f,
-	0x6f8: 0x9f, 0x6f9: 0x9f, 0x6fa: 0x9f, 0x6fb: 0x9f, 0x6fc: 0x9f, 0x6fd: 0x9f, 0x6fe: 0x9f, 0x6ff: 0x9f,
-	// Block 0x1c, offset 0x700
-	0x700: 0x9f, 0x701: 0x9f, 0x702: 0x9f, 0x703: 0x9f, 0x704: 0x9f, 0x705: 0x9f, 0x706: 0x9f, 0x707: 0x9f,
-	0x708: 0x9f, 0x709: 0x9f, 0x70a: 0x9f, 0x70b: 0x9f, 0x70c: 0x9f, 0x70d: 0x9f, 0x70e: 0x9f, 0x70f: 0x9f,
-	0x710: 0x9f, 0x711: 0x9f, 0x712: 0x9f, 0x713: 0x9f, 0x714: 0x9f, 0x715: 0x9f, 0x716: 0x9f, 0x717: 0x9f,
-	0x718: 0x9f, 0x719: 0x9f, 0x71a: 0x9f, 0x71b: 0x9f, 0x71c: 0x9f, 0x71d: 0x9f, 0x71e: 0x9f, 0x71f: 0x9f,
-	0x720: 0x9f, 0x721: 0x9f, 0x722: 0x9f, 0x723: 0x9f, 0x724: 0x9f, 0x725: 0x9f, 0x726: 0x9f, 0x727: 0x9f,
-	0x728: 0x9f, 0x729: 0x9f, 0x72a: 0x9f, 0x72b: 0x9f, 0x72c: 0x9f, 0x72d: 0x9f, 0x72e: 0x9f, 0x72f: 0x9f,
-	0x730: 0x9f, 0x731: 0x9f, 0x732: 0x9f, 0x733: 0x9f, 0x734: 0x9f, 0x735: 0x9f, 0x736: 0x9f, 0x737: 0x9f,
-	0x738: 0x9f, 0x739: 0x9f, 0x73a: 0x191, 0x73b: 0x9f, 0x73c: 0x9f, 0x73d: 0x9f, 0x73e: 0x9f, 0x73f: 0x9f,
-	// Block 0x1d, offset 0x740
-	0x740: 0x9f, 0x741: 0x9f, 0x742: 0x9f, 0x743: 0x9f, 0x744: 0x9f, 0x745: 0x9f, 0x746: 0x9f, 0x747: 0x9f,
-	0x748: 0x9f, 0x749: 0x9f, 0x74a: 0x9f, 0x74b: 0x9f, 0x74c: 0x9f, 0x74d: 0x9f, 0x74e: 0x9f, 0x74f: 0x9f,
-	0x750: 0x9f, 0x751: 0x9f, 0x752: 0x9f, 0x753: 0x9f, 0x754: 0x9f, 0x755: 0x9f, 0x756: 0x9f, 0x757: 0x9f,
-	0x758: 0x9f, 0x759: 0x9f, 0x75a: 0x9f, 0x75b: 0x9f, 0x75c: 0x9f, 0x75d: 0x9f, 0x75e: 0x9f, 0x75f: 0x9f,
-	0x760: 0x9f, 0x761: 0x9f, 0x762: 0x9f, 0x763: 0x9f, 0x764: 0x9f, 0x765: 0x9f, 0x766: 0x9f, 0x767: 0x9f,
-	0x768: 0x9f, 0x769: 0x9f, 0x76a: 0x9f, 0x76b: 0x9f, 0x76c: 0x9f, 0x76d: 0x9f, 0x76e: 0x9f, 0x76f: 0x192,
-	0x770: 0xba, 0x771: 0xba, 0x772: 0xba, 0x773: 0xba, 0x774: 0xba, 0x775: 0xba, 0x776: 0xba, 0x777: 0xba,
-	0x778: 0xba, 0x779: 0xba, 0x77a: 0xba, 0x77b: 0xba, 0x77c: 0xba, 0x77d: 0xba, 0x77e: 0xba, 0x77f: 0xba,
-	// Block 0x1e, offset 0x780
-	0x780: 0xba, 0x781: 0xba, 0x782: 0xba, 0x783: 0xba, 0x784: 0xba, 0x785: 0xba, 0x786: 0xba, 0x787: 0xba,
-	0x788: 0xba, 0x789: 0xba, 0x78a: 0xba, 0x78b: 0xba, 0x78c: 0xba, 0x78d: 0xba, 0x78e: 0xba, 0x78f: 0xba,
-	0x790: 0xba, 0x791: 0xba, 0x792: 0xba, 0x793: 0xba, 0x794: 0xba, 0x795: 0xba, 0x796: 0xba, 0x797: 0xba,
-	0x798: 0xba, 0x799: 0xba, 0x79a: 0xba, 0x79b: 0xba, 0x79c: 0xba, 0x79d: 0xba, 0x79e: 0xba, 0x79f: 0xba,
-	0x7a0: 0x76, 0x7a1: 0x77, 0x7a2: 0x78, 0x7a3: 0x193, 0x7a4: 0x79, 0x7a5: 0x7a, 0x7a6: 0x194, 0x7a7: 0x7b,
-	0x7a8: 0x7c, 0x7a9: 0xba, 0x7aa: 0xba, 0x7ab: 0xba, 0x7ac: 0xba, 0x7ad: 0xba, 0x7ae: 0xba, 0x7af: 0xba,
-	0x7b0: 0xba, 0x7b1: 0xba, 0x7b2: 0xba, 0x7b3: 0xba, 0x7b4: 0xba, 0x7b5: 0xba, 0x7b6: 0xba, 0x7b7: 0xba,
-	0x7b8: 0xba, 0x7b9: 0xba, 0x7ba: 0xba, 0x7bb: 0xba, 0x7bc: 0xba, 0x7bd: 0xba, 0x7be: 0xba, 0x7bf: 0xba,
-	// Block 0x1f, offset 0x7c0
-	0x7d0: 0x0d, 0x7d1: 0x0e, 0x7d2: 0x0f, 0x7d3: 0x10, 0x7d4: 0x11, 0x7d5: 0x0b, 0x7d6: 0x12, 0x7d7: 0x07,
-	0x7d8: 0x13, 0x7d9: 0x0b, 0x7da: 0x0b, 0x7db: 0x14, 0x7dc: 0x0b, 0x7dd: 0x15, 0x7de: 0x16, 0x7df: 0x17,
-	0x7e0: 0x07, 0x7e1: 0x07, 0x7e2: 0x07, 0x7e3: 0x07, 0x7e4: 0x07, 0x7e5: 0x07, 0x7e6: 0x07, 0x7e7: 0x07,
-	0x7e8: 0x07, 0x7e9: 0x07, 0x7ea: 0x18, 0x7eb: 0x19, 0x7ec: 0x1a, 0x7ed: 0x07, 0x7ee: 0x1b, 0x7ef: 0x1c,
-	0x7f0: 0x0b, 0x7f1: 0x0b, 0x7f2: 0x0b, 0x7f3: 0x0b, 0x7f4: 0x0b, 0x7f5: 0x0b, 0x7f6: 0x0b, 0x7f7: 0x0b,
-	0x7f8: 0x0b, 0x7f9: 0x0b, 0x7fa: 0x0b, 0x7fb: 0x0b, 0x7fc: 0x0b, 0x7fd: 0x0b, 0x7fe: 0x0b, 0x7ff: 0x0b,
-	// Block 0x20, offset 0x800
-	0x800: 0x0b, 0x801: 0x0b, 0x802: 0x0b, 0x803: 0x0b, 0x804: 0x0b, 0x805: 0x0b, 0x806: 0x0b, 0x807: 0x0b,
-	0x808: 0x0b, 0x809: 0x0b, 0x80a: 0x0b, 0x80b: 0x0b, 0x80c: 0x0b, 0x80d: 0x0b, 0x80e: 0x0b, 0x80f: 0x0b,
-	0x810: 0x0b, 0x811: 0x0b, 0x812: 0x0b, 0x813: 0x0b, 0x814: 0x0b, 0x815: 0x0b, 0x816: 0x0b, 0x817: 0x0b,
-	0x818: 0x0b, 0x819: 0x0b, 0x81a: 0x0b, 0x81b: 0x0b, 0x81c: 0x0b, 0x81d: 0x0b, 0x81e: 0x0b, 0x81f: 0x0b,
-	0x820: 0x0b, 0x821: 0x0b, 0x822: 0x0b, 0x823: 0x0b, 0x824: 0x0b, 0x825: 0x0b, 0x826: 0x0b, 0x827: 0x0b,
-	0x828: 0x0b, 0x829: 0x0b, 0x82a: 0x0b, 0x82b: 0x0b, 0x82c: 0x0b, 0x82d: 0x0b, 0x82e: 0x0b, 0x82f: 0x0b,
-	0x830: 0x0b, 0x831: 0x0b, 0x832: 0x0b, 0x833: 0x0b, 0x834: 0x0b, 0x835: 0x0b, 0x836: 0x0b, 0x837: 0x0b,
-	0x838: 0x0b, 0x839: 0x0b, 0x83a: 0x0b, 0x83b: 0x0b, 0x83c: 0x0b, 0x83d: 0x0b, 0x83e: 0x0b, 0x83f: 0x0b,
-	// Block 0x21, offset 0x840
-	0x840: 0x195, 0x841: 0x196, 0x842: 0xba, 0x843: 0xba, 0x844: 0x197, 0x845: 0x197, 0x846: 0x197, 0x847: 0x198,
-	0x848: 0xba, 0x849: 0xba, 0x84a: 0xba, 0x84b: 0xba, 0x84c: 0xba, 0x84d: 0xba, 0x84e: 0xba, 0x84f: 0xba,
-	0x850: 0xba, 0x851: 0xba, 0x852: 0xba, 0x853: 0xba, 0x854: 0xba, 0x855: 0xba, 0x856: 0xba, 0x857: 0xba,
-	0x858: 0xba, 0x859: 0xba, 0x85a: 0xba, 0x85b: 0xba, 0x85c: 0xba, 0x85d: 0xba, 0x85e: 0xba, 0x85f: 0xba,
-	0x860: 0xba, 0x861: 0xba, 0x862: 0xba, 0x863: 0xba, 0x864: 0xba, 0x865: 0xba, 0x866: 0xba, 0x867: 0xba,
-	0x868: 0xba, 0x869: 0xba, 0x86a: 0xba, 0x86b: 0xba, 0x86c: 0xba, 0x86d: 0xba, 0x86e: 0xba, 0x86f: 0xba,
-	0x870: 0xba, 0x871: 0xba, 0x872: 0xba, 0x873: 0xba, 0x874: 0xba, 0x875: 0xba, 0x876: 0xba, 0x877: 0xba,
-	0x878: 0xba, 0x879: 0xba, 0x87a: 0xba, 0x87b: 0xba, 0x87c: 0xba, 0x87d: 0xba, 0x87e: 0xba, 0x87f: 0xba,
-	// Block 0x22, offset 0x880
-	0x880: 0x0b, 0x881: 0x0b, 0x882: 0x0b, 0x883: 0x0b, 0x884: 0x0b, 0x885: 0x0b, 0x886: 0x0b, 0x887: 0x0b,
-	0x888: 0x0b, 0x889: 0x0b, 0x88a: 0x0b, 0x88b: 0x0b, 0x88c: 0x0b, 0x88d: 0x0b, 0x88e: 0x0b, 0x88f: 0x0b,
-	0x890: 0x0b, 0x891: 0x0b, 0x892: 0x0b, 0x893: 0x0b, 0x894: 0x0b, 0x895: 0x0b, 0x896: 0x0b, 0x897: 0x0b,
-	0x898: 0x0b, 0x899: 0x0b, 0x89a: 0x0b, 0x89b: 0x0b, 0x89c: 0x0b, 0x89d: 0x0b, 0x89e: 0x0b, 0x89f: 0x0b,
-	0x8a0: 0x1f, 0x8a1: 0x0b, 0x8a2: 0x0b, 0x8a3: 0x0b, 0x8a4: 0x0b, 0x8a5: 0x0b, 0x8a6: 0x0b, 0x8a7: 0x0b,
-	0x8a8: 0x0b, 0x8a9: 0x0b, 0x8aa: 0x0b, 0x8ab: 0x0b, 0x8ac: 0x0b, 0x8ad: 0x0b, 0x8ae: 0x0b, 0x8af: 0x0b,
-	0x8b0: 0x0b, 0x8b1: 0x0b, 0x8b2: 0x0b, 0x8b3: 0x0b, 0x8b4: 0x0b, 0x8b5: 0x0b, 0x8b6: 0x0b, 0x8b7: 0x0b,
-	0x8b8: 0x0b, 0x8b9: 0x0b, 0x8ba: 0x0b, 0x8bb: 0x0b, 0x8bc: 0x0b, 0x8bd: 0x0b, 0x8be: 0x0b, 0x8bf: 0x0b,
-	// Block 0x23, offset 0x8c0
-	0x8c0: 0x0b, 0x8c1: 0x0b, 0x8c2: 0x0b, 0x8c3: 0x0b, 0x8c4: 0x0b, 0x8c5: 0x0b, 0x8c6: 0x0b, 0x8c7: 0x0b,
-	0x8c8: 0x0b, 0x8c9: 0x0b, 0x8ca: 0x0b, 0x8cb: 0x0b, 0x8cc: 0x0b, 0x8cd: 0x0b, 0x8ce: 0x0b, 0x8cf: 0x0b,
-}
-
-// idnaSparseOffset: 284 entries, 568 bytes
-var idnaSparseOffset = []uint16{0x0, 0x8, 0x19, 0x25, 0x27, 0x2c, 0x33, 0x3e, 0x4a, 0x4e, 0x5d, 0x62, 0x6c, 0x78, 0x86, 0x8b, 0x94, 0xa4, 0xb2, 0xbe, 0xca, 0xdb, 0xe5, 0xec, 0xf9, 0x10a, 0x111, 0x11c, 0x12b, 0x139, 0x143, 0x145, 0x14a, 0x14d, 0x150, 0x152, 0x15e, 0x169, 0x171, 0x177, 0x17d, 0x182, 0x187, 0x18a, 0x18e, 0x194, 0x199, 0x1a5, 0x1af, 0x1b5, 0x1c6, 0x1d0, 0x1d3, 0x1db, 0x1de, 0x1eb, 0x1f3, 0x1f7, 0x1fe, 0x206, 0x216, 0x222, 0x224, 0x22e, 0x23a, 0x246, 0x252, 0x25a, 0x25f, 0x26c, 0x27d, 0x281, 0x28c, 0x290, 0x299, 0x2a1, 0x2a7, 0x2ac, 0x2af, 0x2b3, 0x2b9, 0x2bd, 0x2c1, 0x2c5, 0x2cb, 0x2d3, 0x2da, 0x2e5, 0x2ef, 0x2f3, 0x2f6, 0x2fc, 0x300, 0x302, 0x305, 0x307, 0x30a, 0x314, 0x317, 0x326, 0x32a, 0x32f, 0x332, 0x336, 0x33b, 0x340, 0x346, 0x352, 0x361, 0x367, 0x36b, 0x37a, 0x37f, 0x387, 0x391, 0x39c, 0x3a4, 0x3b5, 0x3be, 0x3ce, 0x3db, 0x3e5, 0x3ea, 0x3f7, 0x3fb, 0x400, 0x402, 0x406, 0x408, 0x40c, 0x415, 0x41b, 0x41f, 0x42f, 0x439, 0x43e, 0x441, 0x447, 0x44e, 0x453, 0x457, 0x45d, 0x462, 0x46b, 0x470, 0x476, 0x47d, 0x484, 0x48b, 0x48f, 0x494, 0x497, 0x49c, 0x4a8, 0x4ae, 0x4b3, 0x4ba, 0x4c2, 0x4c7, 0x4cb, 0x4db, 0x4e2, 0x4e6, 0x4ea, 0x4f1, 0x4f3, 0x4f6, 0x4f9, 0x4fd, 0x506, 0x50a, 0x512, 0x51a, 0x51e, 0x524, 0x52d, 0x539, 0x540, 0x549, 0x553, 0x55a, 0x568, 0x575, 0x582, 0x58b, 0x58f, 0x59f, 0x5a7, 0x5b2, 0x5bb, 0x5c1, 0x5c9, 0x5d2, 0x5dd, 0x5e0, 0x5ec, 0x5f5, 0x5f8, 0x5fd, 0x602, 0x60f, 0x61a, 0x623, 0x62d, 0x630, 0x63a, 0x643, 0x64f, 0x65c, 0x669, 0x677, 0x67e, 0x682, 0x685, 0x68a, 0x68d, 0x692, 0x695, 0x69c, 0x6a3, 0x6a7, 0x6b2, 0x6b5, 0x6b8, 0x6bb, 0x6c1, 0x6c7, 0x6cd, 0x6d0, 0x6d3, 0x6d6, 0x6dd, 0x6e0, 0x6e5, 0x6ef, 0x6f2, 0x6f6, 0x705, 0x711, 0x715, 0x71a, 0x71e, 0x723, 0x727, 0x72c, 0x735, 0x740, 0x746, 0x74c, 0x752, 0x758, 0x761, 0x764, 0x767, 0x76b, 0x76f, 0x773, 0x779, 0x77f, 0x784, 0x787, 0x797, 0x79e, 0x7a1, 0x7a6, 0x7aa, 0x7b0, 0x7b5, 0x7b9, 0x7bf, 0x7c5, 0x7c9, 0x7d2, 0x7d7, 0x7da, 0x7dd, 0x7e1, 0x7e5, 0x7e8, 0x7f8, 0x809, 0x80e, 0x810, 0x812}
-
-// idnaSparseValues: 2069 entries, 8276 bytes
-var idnaSparseValues = [2069]valueRange{
-	// Block 0x0, offset 0x0
-	{value: 0x0000, lo: 0x07},
-	{value: 0xe105, lo: 0x80, hi: 0x96},
-	{value: 0x0018, lo: 0x97, hi: 0x97},
-	{value: 0xe105, lo: 0x98, hi: 0x9e},
-	{value: 0x001f, lo: 0x9f, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xb7},
-	{value: 0x0008, lo: 0xb8, hi: 0xbf},
-	// Block 0x1, offset 0x8
-	{value: 0x0000, lo: 0x10},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0xe01d, lo: 0x81, hi: 0x81},
-	{value: 0x0008, lo: 0x82, hi: 0x82},
-	{value: 0x0335, lo: 0x83, hi: 0x83},
-	{value: 0x034d, lo: 0x84, hi: 0x84},
-	{value: 0x0365, lo: 0x85, hi: 0x85},
-	{value: 0xe00d, lo: 0x86, hi: 0x86},
-	{value: 0x0008, lo: 0x87, hi: 0x87},
-	{value: 0xe00d, lo: 0x88, hi: 0x88},
-	{value: 0x0008, lo: 0x89, hi: 0x89},
-	{value: 0xe00d, lo: 0x8a, hi: 0x8a},
-	{value: 0x0008, lo: 0x8b, hi: 0x8b},
-	{value: 0xe00d, lo: 0x8c, hi: 0x8c},
-	{value: 0x0008, lo: 0x8d, hi: 0x8d},
-	{value: 0xe00d, lo: 0x8e, hi: 0x8e},
-	{value: 0x0008, lo: 0x8f, hi: 0xbf},
-	// Block 0x2, offset 0x19
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x0249, lo: 0xb0, hi: 0xb0},
-	{value: 0x037d, lo: 0xb1, hi: 0xb1},
-	{value: 0x0259, lo: 0xb2, hi: 0xb2},
-	{value: 0x0269, lo: 0xb3, hi: 0xb3},
-	{value: 0x034d, lo: 0xb4, hi: 0xb4},
-	{value: 0x0395, lo: 0xb5, hi: 0xb5},
-	{value: 0xe1bd, lo: 0xb6, hi: 0xb6},
-	{value: 0x0279, lo: 0xb7, hi: 0xb7},
-	{value: 0x0289, lo: 0xb8, hi: 0xb8},
-	{value: 0x0008, lo: 0xb9, hi: 0xbf},
-	// Block 0x3, offset 0x25
-	{value: 0x0000, lo: 0x01},
-	{value: 0x3308, lo: 0x80, hi: 0xbf},
-	// Block 0x4, offset 0x27
-	{value: 0x0000, lo: 0x04},
-	{value: 0x03f5, lo: 0x80, hi: 0x8f},
-	{value: 0xe105, lo: 0x90, hi: 0x9f},
-	{value: 0x049d, lo: 0xa0, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x5, offset 0x2c
-	{value: 0x0000, lo: 0x06},
-	{value: 0xe185, lo: 0x80, hi: 0x8f},
-	{value: 0x0545, lo: 0x90, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x98},
-	{value: 0x0008, lo: 0x99, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x6, offset 0x33
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0401, lo: 0x87, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x88},
-	{value: 0x0018, lo: 0x89, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0x90},
-	{value: 0x3308, lo: 0x91, hi: 0xbd},
-	{value: 0x0818, lo: 0xbe, hi: 0xbe},
-	{value: 0x3308, lo: 0xbf, hi: 0xbf},
-	// Block 0x7, offset 0x3e
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0818, lo: 0x80, hi: 0x80},
-	{value: 0x3308, lo: 0x81, hi: 0x82},
-	{value: 0x0818, lo: 0x83, hi: 0x83},
-	{value: 0x3308, lo: 0x84, hi: 0x85},
-	{value: 0x0818, lo: 0x86, hi: 0x86},
-	{value: 0x3308, lo: 0x87, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0808, lo: 0x90, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xae},
-	{value: 0x0808, lo: 0xaf, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0x8, offset 0x4a
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0a08, lo: 0x80, hi: 0x87},
-	{value: 0x0c08, lo: 0x88, hi: 0x99},
-	{value: 0x0a08, lo: 0x9a, hi: 0xbf},
-	// Block 0x9, offset 0x4e
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x3308, lo: 0x80, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8c},
-	{value: 0x0c08, lo: 0x8d, hi: 0x8d},
-	{value: 0x0a08, lo: 0x8e, hi: 0x98},
-	{value: 0x0c08, lo: 0x99, hi: 0x9b},
-	{value: 0x0a08, lo: 0x9c, hi: 0xaa},
-	{value: 0x0c08, lo: 0xab, hi: 0xac},
-	{value: 0x0a08, lo: 0xad, hi: 0xb0},
-	{value: 0x0c08, lo: 0xb1, hi: 0xb1},
-	{value: 0x0a08, lo: 0xb2, hi: 0xb2},
-	{value: 0x0c08, lo: 0xb3, hi: 0xb4},
-	{value: 0x0a08, lo: 0xb5, hi: 0xb7},
-	{value: 0x0c08, lo: 0xb8, hi: 0xb9},
-	{value: 0x0a08, lo: 0xba, hi: 0xbf},
-	// Block 0xa, offset 0x5d
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0808, lo: 0x80, hi: 0xa5},
-	{value: 0x3308, lo: 0xa6, hi: 0xb0},
-	{value: 0x0808, lo: 0xb1, hi: 0xb1},
-	{value: 0x0040, lo: 0xb2, hi: 0xbf},
-	// Block 0xb, offset 0x62
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0808, lo: 0x80, hi: 0x89},
-	{value: 0x0a08, lo: 0x8a, hi: 0xaa},
-	{value: 0x3308, lo: 0xab, hi: 0xb3},
-	{value: 0x0808, lo: 0xb4, hi: 0xb5},
-	{value: 0x0018, lo: 0xb6, hi: 0xb9},
-	{value: 0x0818, lo: 0xba, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbc},
-	{value: 0x3308, lo: 0xbd, hi: 0xbd},
-	{value: 0x0818, lo: 0xbe, hi: 0xbf},
-	// Block 0xc, offset 0x6c
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0808, lo: 0x80, hi: 0x95},
-	{value: 0x3308, lo: 0x96, hi: 0x99},
-	{value: 0x0808, lo: 0x9a, hi: 0x9a},
-	{value: 0x3308, lo: 0x9b, hi: 0xa3},
-	{value: 0x0808, lo: 0xa4, hi: 0xa4},
-	{value: 0x3308, lo: 0xa5, hi: 0xa7},
-	{value: 0x0808, lo: 0xa8, hi: 0xa8},
-	{value: 0x3308, lo: 0xa9, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x0818, lo: 0xb0, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0xd, offset 0x78
-	{value: 0x0000, lo: 0x0d},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0a08, lo: 0xa0, hi: 0xa9},
-	{value: 0x0c08, lo: 0xaa, hi: 0xac},
-	{value: 0x0808, lo: 0xad, hi: 0xad},
-	{value: 0x0c08, lo: 0xae, hi: 0xae},
-	{value: 0x0a08, lo: 0xaf, hi: 0xb0},
-	{value: 0x0c08, lo: 0xb1, hi: 0xb2},
-	{value: 0x0a08, lo: 0xb3, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xb5},
-	{value: 0x0a08, lo: 0xb6, hi: 0xb8},
-	{value: 0x0c08, lo: 0xb9, hi: 0xb9},
-	{value: 0x0a08, lo: 0xba, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0xe, offset 0x86
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0040, lo: 0x80, hi: 0x92},
-	{value: 0x3308, lo: 0x93, hi: 0xa1},
-	{value: 0x0840, lo: 0xa2, hi: 0xa2},
-	{value: 0x3308, lo: 0xa3, hi: 0xbf},
-	// Block 0xf, offset 0x8b
-	{value: 0x0000, lo: 0x08},
-	{value: 0x3308, lo: 0x80, hi: 0x82},
-	{value: 0x3008, lo: 0x83, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0xb9},
-	{value: 0x3308, lo: 0xba, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbf},
-	// Block 0x10, offset 0x94
-	{value: 0x0000, lo: 0x0f},
-	{value: 0x3308, lo: 0x80, hi: 0x80},
-	{value: 0x3008, lo: 0x81, hi: 0x82},
-	{value: 0x0040, lo: 0x83, hi: 0x85},
-	{value: 0x3008, lo: 0x86, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x3008, lo: 0x8a, hi: 0x8c},
-	{value: 0x3b08, lo: 0x8d, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x96},
-	{value: 0x3008, lo: 0x97, hi: 0x97},
-	{value: 0x0040, lo: 0x98, hi: 0xa5},
-	{value: 0x0008, lo: 0xa6, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x11, offset 0xa4
-	{value: 0x0000, lo: 0x0d},
-	{value: 0x3308, lo: 0x80, hi: 0x80},
-	{value: 0x3008, lo: 0x81, hi: 0x83},
-	{value: 0x3308, lo: 0x84, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0x8c},
-	{value: 0x0040, lo: 0x8d, hi: 0x8d},
-	{value: 0x0008, lo: 0x8e, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x91},
-	{value: 0x0008, lo: 0x92, hi: 0xa8},
-	{value: 0x0040, lo: 0xa9, hi: 0xa9},
-	{value: 0x0008, lo: 0xaa, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbd},
-	{value: 0x3308, lo: 0xbe, hi: 0xbf},
-	// Block 0x12, offset 0xb2
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x3308, lo: 0x80, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x83},
-	{value: 0x0040, lo: 0x84, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0x8c},
-	{value: 0x0040, lo: 0x8d, hi: 0x8d},
-	{value: 0x0008, lo: 0x8e, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x91},
-	{value: 0x0008, lo: 0x92, hi: 0xba},
-	{value: 0x3b08, lo: 0xbb, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbf},
-	// Block 0x13, offset 0xbe
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0040, lo: 0x80, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x83},
-	{value: 0x0040, lo: 0x84, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x99},
-	{value: 0x0008, lo: 0x9a, hi: 0xb1},
-	{value: 0x0040, lo: 0xb2, hi: 0xb2},
-	{value: 0x0008, lo: 0xb3, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0x14, offset 0xca
-	{value: 0x0000, lo: 0x10},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x89},
-	{value: 0x3b08, lo: 0x8a, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8e},
-	{value: 0x3008, lo: 0x8f, hi: 0x91},
-	{value: 0x3308, lo: 0x92, hi: 0x94},
-	{value: 0x0040, lo: 0x95, hi: 0x95},
-	{value: 0x3308, lo: 0x96, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x97},
-	{value: 0x3008, lo: 0x98, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xa5},
-	{value: 0x0008, lo: 0xa6, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xb1},
-	{value: 0x3008, lo: 0xb2, hi: 0xb3},
-	{value: 0x0018, lo: 0xb4, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0x15, offset 0xdb
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0xb0},
-	{value: 0x3308, lo: 0xb1, hi: 0xb1},
-	{value: 0x0008, lo: 0xb2, hi: 0xb2},
-	{value: 0x08f1, lo: 0xb3, hi: 0xb3},
-	{value: 0x3308, lo: 0xb4, hi: 0xb9},
-	{value: 0x3b08, lo: 0xba, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbe},
-	{value: 0x0018, lo: 0xbf, hi: 0xbf},
-	// Block 0x16, offset 0xe5
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x3308, lo: 0x87, hi: 0x8e},
-	{value: 0x0018, lo: 0x8f, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0x9b},
-	{value: 0x0040, lo: 0x9c, hi: 0xbf},
-	// Block 0x17, offset 0xec
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x84},
-	{value: 0x0040, lo: 0x85, hi: 0x85},
-	{value: 0x0008, lo: 0x86, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x87},
-	{value: 0x3308, lo: 0x88, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9b},
-	{value: 0x0961, lo: 0x9c, hi: 0x9c},
-	{value: 0x0999, lo: 0x9d, hi: 0x9d},
-	{value: 0x0008, lo: 0x9e, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0x18, offset 0xf9
-	{value: 0x0000, lo: 0x10},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x8a},
-	{value: 0x0008, lo: 0x8b, hi: 0x8b},
-	{value: 0xe03d, lo: 0x8c, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0x97},
-	{value: 0x3308, lo: 0x98, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa9},
-	{value: 0x0018, lo: 0xaa, hi: 0xb4},
-	{value: 0x3308, lo: 0xb5, hi: 0xb5},
-	{value: 0x0018, lo: 0xb6, hi: 0xb6},
-	{value: 0x3308, lo: 0xb7, hi: 0xb7},
-	{value: 0x0018, lo: 0xb8, hi: 0xb8},
-	{value: 0x3308, lo: 0xb9, hi: 0xb9},
-	{value: 0x0018, lo: 0xba, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbf},
-	// Block 0x19, offset 0x10a
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0018, lo: 0x80, hi: 0x85},
-	{value: 0x3308, lo: 0x86, hi: 0x86},
-	{value: 0x0018, lo: 0x87, hi: 0x8c},
-	{value: 0x0040, lo: 0x8d, hi: 0x8d},
-	{value: 0x0018, lo: 0x8e, hi: 0x9a},
-	{value: 0x0040, lo: 0x9b, hi: 0xbf},
-	// Block 0x1a, offset 0x111
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0xaa},
-	{value: 0x3008, lo: 0xab, hi: 0xac},
-	{value: 0x3308, lo: 0xad, hi: 0xb0},
-	{value: 0x3008, lo: 0xb1, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb7},
-	{value: 0x3008, lo: 0xb8, hi: 0xb8},
-	{value: 0x3b08, lo: 0xb9, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbc},
-	{value: 0x3308, lo: 0xbd, hi: 0xbe},
-	{value: 0x0008, lo: 0xbf, hi: 0xbf},
-	// Block 0x1b, offset 0x11c
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0018, lo: 0x8a, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x95},
-	{value: 0x3008, lo: 0x96, hi: 0x97},
-	{value: 0x3308, lo: 0x98, hi: 0x99},
-	{value: 0x0008, lo: 0x9a, hi: 0x9d},
-	{value: 0x3308, lo: 0x9e, hi: 0xa0},
-	{value: 0x0008, lo: 0xa1, hi: 0xa1},
-	{value: 0x3008, lo: 0xa2, hi: 0xa4},
-	{value: 0x0008, lo: 0xa5, hi: 0xa6},
-	{value: 0x3008, lo: 0xa7, hi: 0xad},
-	{value: 0x0008, lo: 0xae, hi: 0xb0},
-	{value: 0x3308, lo: 0xb1, hi: 0xb4},
-	{value: 0x0008, lo: 0xb5, hi: 0xbf},
-	// Block 0x1c, offset 0x12b
-	{value: 0x0000, lo: 0x0d},
-	{value: 0x0008, lo: 0x80, hi: 0x81},
-	{value: 0x3308, lo: 0x82, hi: 0x82},
-	{value: 0x3008, lo: 0x83, hi: 0x84},
-	{value: 0x3308, lo: 0x85, hi: 0x86},
-	{value: 0x3008, lo: 0x87, hi: 0x8c},
-	{value: 0x3308, lo: 0x8d, hi: 0x8d},
-	{value: 0x0008, lo: 0x8e, hi: 0x8e},
-	{value: 0x3008, lo: 0x8f, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x3008, lo: 0x9a, hi: 0x9c},
-	{value: 0x3308, lo: 0x9d, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0x1d, offset 0x139
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0040, lo: 0x80, hi: 0x86},
-	{value: 0x055d, lo: 0x87, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8c},
-	{value: 0x055d, lo: 0x8d, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xba},
-	{value: 0x0018, lo: 0xbb, hi: 0xbb},
-	{value: 0xe105, lo: 0xbc, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbf},
-	// Block 0x1e, offset 0x143
-	{value: 0x0000, lo: 0x01},
-	{value: 0x0018, lo: 0x80, hi: 0xbf},
-	// Block 0x1f, offset 0x145
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0xa0},
-	{value: 0x2018, lo: 0xa1, hi: 0xb5},
-	{value: 0x0018, lo: 0xb6, hi: 0xbf},
-	// Block 0x20, offset 0x14a
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0xa7},
-	{value: 0x2018, lo: 0xa8, hi: 0xbf},
-	// Block 0x21, offset 0x14d
-	{value: 0x0000, lo: 0x02},
-	{value: 0x2018, lo: 0x80, hi: 0x82},
-	{value: 0x0018, lo: 0x83, hi: 0xbf},
-	// Block 0x22, offset 0x150
-	{value: 0x0000, lo: 0x01},
-	{value: 0x0008, lo: 0x80, hi: 0xbf},
-	// Block 0x23, offset 0x152
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x0008, lo: 0x8a, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0x98},
-	{value: 0x0040, lo: 0x99, hi: 0x99},
-	{value: 0x0008, lo: 0x9a, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x24, offset 0x15e
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x0008, lo: 0x8a, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xb0},
-	{value: 0x0040, lo: 0xb1, hi: 0xb1},
-	{value: 0x0008, lo: 0xb2, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb7},
-	{value: 0x0008, lo: 0xb8, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0x25, offset 0x169
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x0040, lo: 0x81, hi: 0x81},
-	{value: 0x0008, lo: 0x82, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0xbf},
-	// Block 0x26, offset 0x171
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x91},
-	{value: 0x0008, lo: 0x92, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0xbf},
-	// Block 0x27, offset 0x177
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x9a},
-	{value: 0x0040, lo: 0x9b, hi: 0x9c},
-	{value: 0x3308, lo: 0x9d, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbf},
-	// Block 0x28, offset 0x17d
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x29, offset 0x182
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb7},
-	{value: 0xe045, lo: 0xb8, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0x2a, offset 0x187
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0xbf},
-	// Block 0x2b, offset 0x18a
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xac},
-	{value: 0x0018, lo: 0xad, hi: 0xae},
-	{value: 0x0008, lo: 0xaf, hi: 0xbf},
-	// Block 0x2c, offset 0x18e
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0x9c},
-	{value: 0x0040, lo: 0x9d, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x2d, offset 0x194
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xaa},
-	{value: 0x0018, lo: 0xab, hi: 0xb0},
-	{value: 0x0008, lo: 0xb1, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0x2e, offset 0x199
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0x8c},
-	{value: 0x0040, lo: 0x8d, hi: 0x8d},
-	{value: 0x0008, lo: 0x8e, hi: 0x91},
-	{value: 0x3308, lo: 0x92, hi: 0x93},
-	{value: 0x3b08, lo: 0x94, hi: 0x94},
-	{value: 0x0040, lo: 0x95, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb3},
-	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
-	{value: 0x0018, lo: 0xb5, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0x2f, offset 0x1a5
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x91},
-	{value: 0x3308, lo: 0x92, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xad},
-	{value: 0x0008, lo: 0xae, hi: 0xb0},
-	{value: 0x0040, lo: 0xb1, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xbf},
-	// Block 0x30, offset 0x1af
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0xb3},
-	{value: 0x3340, lo: 0xb4, hi: 0xb5},
-	{value: 0x3008, lo: 0xb6, hi: 0xb6},
-	{value: 0x3308, lo: 0xb7, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbf},
-	// Block 0x31, offset 0x1b5
-	{value: 0x0000, lo: 0x10},
-	{value: 0x3008, lo: 0x80, hi: 0x85},
-	{value: 0x3308, lo: 0x86, hi: 0x86},
-	{value: 0x3008, lo: 0x87, hi: 0x88},
-	{value: 0x3308, lo: 0x89, hi: 0x91},
-	{value: 0x3b08, lo: 0x92, hi: 0x92},
-	{value: 0x3308, lo: 0x93, hi: 0x93},
-	{value: 0x0018, lo: 0x94, hi: 0x96},
-	{value: 0x0008, lo: 0x97, hi: 0x97},
-	{value: 0x0018, lo: 0x98, hi: 0x9b},
-	{value: 0x0008, lo: 0x9c, hi: 0x9c},
-	{value: 0x3308, lo: 0x9d, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa9},
-	{value: 0x0040, lo: 0xaa, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0x32, offset 0x1c6
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0018, lo: 0x80, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x86},
-	{value: 0x0218, lo: 0x87, hi: 0x87},
-	{value: 0x0018, lo: 0x88, hi: 0x8a},
-	{value: 0x33c0, lo: 0x8b, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0208, lo: 0xa0, hi: 0xbf},
-	// Block 0x33, offset 0x1d0
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0208, lo: 0x80, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0x34, offset 0x1d3
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0x84},
-	{value: 0x3308, lo: 0x85, hi: 0x86},
-	{value: 0x0208, lo: 0x87, hi: 0xa8},
-	{value: 0x3308, lo: 0xa9, hi: 0xa9},
-	{value: 0x0208, lo: 0xaa, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x35, offset 0x1db
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xbf},
-	// Block 0x36, offset 0x1de
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0x9f},
-	{value: 0x3308, lo: 0xa0, hi: 0xa2},
-	{value: 0x3008, lo: 0xa3, hi: 0xa6},
-	{value: 0x3308, lo: 0xa7, hi: 0xa8},
-	{value: 0x3008, lo: 0xa9, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xaf},
-	{value: 0x3008, lo: 0xb0, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb2},
-	{value: 0x3008, lo: 0xb3, hi: 0xb8},
-	{value: 0x3308, lo: 0xb9, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0x37, offset 0x1eb
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0018, lo: 0x80, hi: 0x80},
-	{value: 0x0040, lo: 0x81, hi: 0x83},
-	{value: 0x0018, lo: 0x84, hi: 0x85},
-	{value: 0x0008, lo: 0x86, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0x38, offset 0x1f3
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x39, offset 0x1f7
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0028, lo: 0x9a, hi: 0x9a},
-	{value: 0x0040, lo: 0x9b, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0xbf},
-	// Block 0x3a, offset 0x1fe
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0x96},
-	{value: 0x3308, lo: 0x97, hi: 0x98},
-	{value: 0x3008, lo: 0x99, hi: 0x9a},
-	{value: 0x3308, lo: 0x9b, hi: 0x9b},
-	{value: 0x0040, lo: 0x9c, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x3b, offset 0x206
-	{value: 0x0000, lo: 0x0f},
-	{value: 0x0008, lo: 0x80, hi: 0x94},
-	{value: 0x3008, lo: 0x95, hi: 0x95},
-	{value: 0x3308, lo: 0x96, hi: 0x96},
-	{value: 0x3008, lo: 0x97, hi: 0x97},
-	{value: 0x3308, lo: 0x98, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0x9f},
-	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
-	{value: 0x3008, lo: 0xa1, hi: 0xa1},
-	{value: 0x3308, lo: 0xa2, hi: 0xa2},
-	{value: 0x3008, lo: 0xa3, hi: 0xa4},
-	{value: 0x3308, lo: 0xa5, hi: 0xac},
-	{value: 0x3008, lo: 0xad, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbe},
-	{value: 0x3308, lo: 0xbf, hi: 0xbf},
-	// Block 0x3c, offset 0x216
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xa6},
-	{value: 0x0008, lo: 0xa7, hi: 0xa7},
-	{value: 0x0018, lo: 0xa8, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xbd},
-	{value: 0x3318, lo: 0xbe, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0x3d, offset 0x222
-	{value: 0x0000, lo: 0x01},
-	{value: 0x0040, lo: 0x80, hi: 0xbf},
-	// Block 0x3e, offset 0x224
-	{value: 0x0000, lo: 0x09},
-	{value: 0x3308, lo: 0x80, hi: 0x83},
-	{value: 0x3008, lo: 0x84, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0xb3},
-	{value: 0x3308, lo: 0xb4, hi: 0xb4},
-	{value: 0x3008, lo: 0xb5, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbc},
-	{value: 0x3008, lo: 0xbd, hi: 0xbf},
-	// Block 0x3f, offset 0x22e
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x3008, lo: 0x80, hi: 0x81},
-	{value: 0x3308, lo: 0x82, hi: 0x82},
-	{value: 0x3008, lo: 0x83, hi: 0x83},
-	{value: 0x3808, lo: 0x84, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0xaa},
-	{value: 0x3308, lo: 0xab, hi: 0xb3},
-	{value: 0x0018, lo: 0xb4, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbf},
-	// Block 0x40, offset 0x23a
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x3308, lo: 0x80, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0xa0},
-	{value: 0x3008, lo: 0xa1, hi: 0xa1},
-	{value: 0x3308, lo: 0xa2, hi: 0xa5},
-	{value: 0x3008, lo: 0xa6, hi: 0xa7},
-	{value: 0x3308, lo: 0xa8, hi: 0xa9},
-	{value: 0x3808, lo: 0xaa, hi: 0xaa},
-	{value: 0x3b08, lo: 0xab, hi: 0xab},
-	{value: 0x3308, lo: 0xac, hi: 0xad},
-	{value: 0x0008, lo: 0xae, hi: 0xbf},
-	// Block 0x41, offset 0x246
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0xa5},
-	{value: 0x3308, lo: 0xa6, hi: 0xa6},
-	{value: 0x3008, lo: 0xa7, hi: 0xa7},
-	{value: 0x3308, lo: 0xa8, hi: 0xa9},
-	{value: 0x3008, lo: 0xaa, hi: 0xac},
-	{value: 0x3308, lo: 0xad, hi: 0xad},
-	{value: 0x3008, lo: 0xae, hi: 0xae},
-	{value: 0x3308, lo: 0xaf, hi: 0xb1},
-	{value: 0x3808, lo: 0xb2, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xbb},
-	{value: 0x0018, lo: 0xbc, hi: 0xbf},
-	// Block 0x42, offset 0x252
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0xa3},
-	{value: 0x3008, lo: 0xa4, hi: 0xab},
-	{value: 0x3308, lo: 0xac, hi: 0xb3},
-	{value: 0x3008, lo: 0xb4, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xba},
-	{value: 0x0018, lo: 0xbb, hi: 0xbf},
-	// Block 0x43, offset 0x25a
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0x8c},
-	{value: 0x0008, lo: 0x8d, hi: 0xbd},
-	{value: 0x0018, lo: 0xbe, hi: 0xbf},
-	// Block 0x44, offset 0x25f
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0e29, lo: 0x80, hi: 0x80},
-	{value: 0x0e41, lo: 0x81, hi: 0x81},
-	{value: 0x0e59, lo: 0x82, hi: 0x82},
-	{value: 0x0e71, lo: 0x83, hi: 0x83},
-	{value: 0x0e89, lo: 0x84, hi: 0x85},
-	{value: 0x0ea1, lo: 0x86, hi: 0x86},
-	{value: 0x0eb9, lo: 0x87, hi: 0x87},
-	{value: 0x057d, lo: 0x88, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x8f},
-	{value: 0x059d, lo: 0x90, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbc},
-	{value: 0x059d, lo: 0xbd, hi: 0xbf},
-	// Block 0x45, offset 0x26c
-	{value: 0x0000, lo: 0x10},
-	{value: 0x0018, lo: 0x80, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x3308, lo: 0x90, hi: 0x92},
-	{value: 0x0018, lo: 0x93, hi: 0x93},
-	{value: 0x3308, lo: 0x94, hi: 0xa0},
-	{value: 0x3008, lo: 0xa1, hi: 0xa1},
-	{value: 0x3308, lo: 0xa2, hi: 0xa8},
-	{value: 0x0008, lo: 0xa9, hi: 0xac},
-	{value: 0x3308, lo: 0xad, hi: 0xad},
-	{value: 0x0008, lo: 0xae, hi: 0xb3},
-	{value: 0x3308, lo: 0xb4, hi: 0xb4},
-	{value: 0x0008, lo: 0xb5, hi: 0xb6},
-	{value: 0x3008, lo: 0xb7, hi: 0xb7},
-	{value: 0x3308, lo: 0xb8, hi: 0xb9},
-	{value: 0x0008, lo: 0xba, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x46, offset 0x27d
-	{value: 0x0000, lo: 0x03},
-	{value: 0x3308, lo: 0x80, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xba},
-	{value: 0x3308, lo: 0xbb, hi: 0xbf},
-	// Block 0x47, offset 0x281
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x87},
-	{value: 0xe045, lo: 0x88, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x97},
-	{value: 0xe045, lo: 0x98, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa7},
-	{value: 0xe045, lo: 0xa8, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb7},
-	{value: 0xe045, lo: 0xb8, hi: 0xbf},
-	// Block 0x48, offset 0x28c
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0x8f},
-	{value: 0x3318, lo: 0x90, hi: 0xb0},
-	{value: 0x0040, lo: 0xb1, hi: 0xbf},
-	// Block 0x49, offset 0x290
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0018, lo: 0x80, hi: 0x82},
-	{value: 0x0040, lo: 0x83, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0x84},
-	{value: 0x0018, lo: 0x85, hi: 0x88},
-	{value: 0x24c1, lo: 0x89, hi: 0x89},
-	{value: 0x0018, lo: 0x8a, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0xbf},
-	// Block 0x4a, offset 0x299
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0018, lo: 0x80, hi: 0xab},
-	{value: 0x24f1, lo: 0xac, hi: 0xac},
-	{value: 0x2529, lo: 0xad, hi: 0xad},
-	{value: 0x0018, lo: 0xae, hi: 0xae},
-	{value: 0x2579, lo: 0xaf, hi: 0xaf},
-	{value: 0x25b1, lo: 0xb0, hi: 0xb0},
-	{value: 0x0018, lo: 0xb1, hi: 0xbf},
-	// Block 0x4b, offset 0x2a1
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0x9f},
-	{value: 0x0080, lo: 0xa0, hi: 0xa0},
-	{value: 0x0018, lo: 0xa1, hi: 0xad},
-	{value: 0x0080, lo: 0xae, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xbf},
-	// Block 0x4c, offset 0x2a7
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0xa8},
-	{value: 0x09dd, lo: 0xa9, hi: 0xa9},
-	{value: 0x09fd, lo: 0xaa, hi: 0xaa},
-	{value: 0x0018, lo: 0xab, hi: 0xbf},
-	// Block 0x4d, offset 0x2ac
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0xa6},
-	{value: 0x0040, lo: 0xa7, hi: 0xbf},
-	// Block 0x4e, offset 0x2af
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x8b},
-	{value: 0x28c1, lo: 0x8c, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0xbf},
-	// Block 0x4f, offset 0x2b3
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0xb3},
-	{value: 0x0e7e, lo: 0xb4, hi: 0xb4},
-	{value: 0x292a, lo: 0xb5, hi: 0xb5},
-	{value: 0x0e9e, lo: 0xb6, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xbf},
-	// Block 0x50, offset 0x2b9
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x9b},
-	{value: 0x2941, lo: 0x9c, hi: 0x9c},
-	{value: 0x0018, lo: 0x9d, hi: 0xbf},
-	// Block 0x51, offset 0x2bd
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xb5},
-	{value: 0x0018, lo: 0xb6, hi: 0xbf},
-	// Block 0x52, offset 0x2c1
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x97},
-	{value: 0x0018, lo: 0x98, hi: 0xbf},
-	// Block 0x53, offset 0x2c5
-	{value: 0x0000, lo: 0x05},
-	{value: 0xe185, lo: 0x80, hi: 0x8f},
-	{value: 0x03f5, lo: 0x90, hi: 0x9f},
-	{value: 0x0ebd, lo: 0xa0, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x54, offset 0x2cb
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0xa5},
-	{value: 0x0040, lo: 0xa6, hi: 0xa6},
-	{value: 0x0008, lo: 0xa7, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xac},
-	{value: 0x0008, lo: 0xad, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x55, offset 0x2d3
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0008, lo: 0x80, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xae},
-	{value: 0xe075, lo: 0xaf, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb0},
-	{value: 0x0040, lo: 0xb1, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0x56, offset 0x2da
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa6},
-	{value: 0x0040, lo: 0xa7, hi: 0xa7},
-	{value: 0x0008, lo: 0xa8, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xb7},
-	{value: 0x0008, lo: 0xb8, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0x57, offset 0x2e5
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x8e},
-	{value: 0x0040, lo: 0x8f, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0x9f},
-	{value: 0x3308, lo: 0xa0, hi: 0xbf},
-	// Block 0x58, offset 0x2ef
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xae},
-	{value: 0x0008, lo: 0xaf, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xbf},
-	// Block 0x59, offset 0x2f3
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0xbf},
-	// Block 0x5a, offset 0x2f6
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0x9e},
-	{value: 0x0ef5, lo: 0x9f, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xbf},
-	// Block 0x5b, offset 0x2fc
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xb2},
-	{value: 0x0f15, lo: 0xb3, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xbf},
-	// Block 0x5c, offset 0x300
-	{value: 0x0020, lo: 0x01},
-	{value: 0x0f35, lo: 0x80, hi: 0xbf},
-	// Block 0x5d, offset 0x302
-	{value: 0x0020, lo: 0x02},
-	{value: 0x1735, lo: 0x80, hi: 0x8f},
-	{value: 0x1915, lo: 0x90, hi: 0xbf},
-	// Block 0x5e, offset 0x305
-	{value: 0x0020, lo: 0x01},
-	{value: 0x1f15, lo: 0x80, hi: 0xbf},
-	// Block 0x5f, offset 0x307
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0xbf},
-	// Block 0x60, offset 0x30a
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x98},
-	{value: 0x3308, lo: 0x99, hi: 0x9a},
-	{value: 0x29e2, lo: 0x9b, hi: 0x9b},
-	{value: 0x2a0a, lo: 0x9c, hi: 0x9c},
-	{value: 0x0008, lo: 0x9d, hi: 0x9e},
-	{value: 0x2a31, lo: 0x9f, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xa0},
-	{value: 0x0008, lo: 0xa1, hi: 0xbf},
-	// Block 0x61, offset 0x314
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xbe},
-	{value: 0x2a69, lo: 0xbf, hi: 0xbf},
-	// Block 0x62, offset 0x317
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x0040, lo: 0x80, hi: 0x84},
-	{value: 0x0008, lo: 0x85, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xb0},
-	{value: 0x2a35, lo: 0xb1, hi: 0xb1},
-	{value: 0x2a55, lo: 0xb2, hi: 0xb2},
-	{value: 0x2a75, lo: 0xb3, hi: 0xb3},
-	{value: 0x2a95, lo: 0xb4, hi: 0xb4},
-	{value: 0x2a75, lo: 0xb5, hi: 0xb5},
-	{value: 0x2ab5, lo: 0xb6, hi: 0xb6},
-	{value: 0x2ad5, lo: 0xb7, hi: 0xb7},
-	{value: 0x2af5, lo: 0xb8, hi: 0xb9},
-	{value: 0x2b15, lo: 0xba, hi: 0xbb},
-	{value: 0x2b35, lo: 0xbc, hi: 0xbd},
-	{value: 0x2b15, lo: 0xbe, hi: 0xbf},
-	// Block 0x63, offset 0x326
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x64, offset 0x32a
-	{value: 0x0030, lo: 0x04},
-	{value: 0x2aa2, lo: 0x80, hi: 0x9d},
-	{value: 0x305a, lo: 0x9e, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0x9f},
-	{value: 0x30a2, lo: 0xa0, hi: 0xbf},
-	// Block 0x65, offset 0x32f
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-	// Block 0x66, offset 0x332
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0x8c},
-	{value: 0x0040, lo: 0x8d, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0xbf},
-	// Block 0x67, offset 0x336
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xbd},
-	{value: 0x0018, lo: 0xbe, hi: 0xbf},
-	// Block 0x68, offset 0x33b
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xbf},
-	// Block 0x69, offset 0x340
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0xa5},
-	{value: 0x0018, lo: 0xa6, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb1},
-	{value: 0x0018, lo: 0xb2, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xbf},
-	// Block 0x6a, offset 0x346
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0040, lo: 0x80, hi: 0x81},
-	{value: 0xe00d, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0x83},
-	{value: 0x03f5, lo: 0x84, hi: 0x84},
-	{value: 0x1329, lo: 0x85, hi: 0x85},
-	{value: 0x447d, lo: 0x86, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0xb6},
-	{value: 0x0008, lo: 0xb7, hi: 0xb7},
-	{value: 0x2009, lo: 0xb8, hi: 0xb8},
-	{value: 0x6e89, lo: 0xb9, hi: 0xb9},
-	{value: 0x0008, lo: 0xba, hi: 0xbf},
-	// Block 0x6b, offset 0x352
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x0008, lo: 0x80, hi: 0x81},
-	{value: 0x3308, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0x85},
-	{value: 0x3b08, lo: 0x86, hi: 0x86},
-	{value: 0x0008, lo: 0x87, hi: 0x8a},
-	{value: 0x3308, lo: 0x8b, hi: 0x8b},
-	{value: 0x0008, lo: 0x8c, hi: 0xa2},
-	{value: 0x3008, lo: 0xa3, hi: 0xa4},
-	{value: 0x3308, lo: 0xa5, hi: 0xa6},
-	{value: 0x3008, lo: 0xa7, hi: 0xa7},
-	{value: 0x0018, lo: 0xa8, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0x6c, offset 0x361
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0208, lo: 0x80, hi: 0xb1},
-	{value: 0x0108, lo: 0xb2, hi: 0xb2},
-	{value: 0x0008, lo: 0xb3, hi: 0xb3},
-	{value: 0x0018, lo: 0xb4, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xbf},
-	// Block 0x6d, offset 0x367
-	{value: 0x0000, lo: 0x03},
-	{value: 0x3008, lo: 0x80, hi: 0x81},
-	{value: 0x0008, lo: 0x82, hi: 0xb3},
-	{value: 0x3008, lo: 0xb4, hi: 0xbf},
-	// Block 0x6e, offset 0x36b
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x3008, lo: 0x80, hi: 0x83},
-	{value: 0x3b08, lo: 0x84, hi: 0x84},
-	{value: 0x3308, lo: 0x85, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x8d},
-	{value: 0x0018, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x3308, lo: 0xa0, hi: 0xb1},
-	{value: 0x0008, lo: 0xb2, hi: 0xb7},
-	{value: 0x0018, lo: 0xb8, hi: 0xba},
-	{value: 0x0008, lo: 0xbb, hi: 0xbb},
-	{value: 0x0018, lo: 0xbc, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbe},
-	{value: 0x3308, lo: 0xbf, hi: 0xbf},
-	// Block 0x6f, offset 0x37a
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xa5},
-	{value: 0x3308, lo: 0xa6, hi: 0xad},
-	{value: 0x0018, lo: 0xae, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x70, offset 0x37f
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x3308, lo: 0x87, hi: 0x91},
-	{value: 0x3008, lo: 0x92, hi: 0x92},
-	{value: 0x3808, lo: 0x93, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x9e},
-	{value: 0x0018, lo: 0x9f, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbf},
-	// Block 0x71, offset 0x387
-	{value: 0x0000, lo: 0x09},
-	{value: 0x3308, lo: 0x80, hi: 0x82},
-	{value: 0x3008, lo: 0x83, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb3},
-	{value: 0x3008, lo: 0xb4, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xb9},
-	{value: 0x3008, lo: 0xba, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbf},
-	// Block 0x72, offset 0x391
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x3808, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8e},
-	{value: 0x0008, lo: 0x8f, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa4},
-	{value: 0x3308, lo: 0xa5, hi: 0xa5},
-	{value: 0x0008, lo: 0xa6, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0x73, offset 0x39c
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0xa8},
-	{value: 0x3308, lo: 0xa9, hi: 0xae},
-	{value: 0x3008, lo: 0xaf, hi: 0xb0},
-	{value: 0x3308, lo: 0xb1, hi: 0xb2},
-	{value: 0x3008, lo: 0xb3, hi: 0xb4},
-	{value: 0x3308, lo: 0xb5, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0x74, offset 0x3a4
-	{value: 0x0000, lo: 0x10},
-	{value: 0x0008, lo: 0x80, hi: 0x82},
-	{value: 0x3308, lo: 0x83, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0x8b},
-	{value: 0x3308, lo: 0x8c, hi: 0x8c},
-	{value: 0x3008, lo: 0x8d, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9b},
-	{value: 0x0018, lo: 0x9c, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xb9},
-	{value: 0x0008, lo: 0xba, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbc},
-	{value: 0x3008, lo: 0xbd, hi: 0xbd},
-	{value: 0x0008, lo: 0xbe, hi: 0xbf},
-	// Block 0x75, offset 0x3b5
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb0},
-	{value: 0x0008, lo: 0xb1, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb4},
-	{value: 0x0008, lo: 0xb5, hi: 0xb6},
-	{value: 0x3308, lo: 0xb7, hi: 0xb8},
-	{value: 0x0008, lo: 0xb9, hi: 0xbd},
-	{value: 0x3308, lo: 0xbe, hi: 0xbf},
-	// Block 0x76, offset 0x3be
-	{value: 0x0000, lo: 0x0f},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x3308, lo: 0x81, hi: 0x81},
-	{value: 0x0008, lo: 0x82, hi: 0x82},
-	{value: 0x0040, lo: 0x83, hi: 0x9a},
-	{value: 0x0008, lo: 0x9b, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xaa},
-	{value: 0x3008, lo: 0xab, hi: 0xab},
-	{value: 0x3308, lo: 0xac, hi: 0xad},
-	{value: 0x3008, lo: 0xae, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb1},
-	{value: 0x0008, lo: 0xb2, hi: 0xb4},
-	{value: 0x3008, lo: 0xb5, hi: 0xb5},
-	{value: 0x3b08, lo: 0xb6, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0x77, offset 0x3ce
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x88},
-	{value: 0x0008, lo: 0x89, hi: 0x8e},
-	{value: 0x0040, lo: 0x8f, hi: 0x90},
-	{value: 0x0008, lo: 0x91, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa6},
-	{value: 0x0040, lo: 0xa7, hi: 0xa7},
-	{value: 0x0008, lo: 0xa8, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x78, offset 0x3db
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0x9b},
-	{value: 0x449d, lo: 0x9c, hi: 0x9c},
-	{value: 0x44b5, lo: 0x9d, hi: 0x9d},
-	{value: 0x2971, lo: 0x9e, hi: 0x9e},
-	{value: 0xe06d, lo: 0x9f, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xaf},
-	{value: 0x44cd, lo: 0xb0, hi: 0xbf},
-	// Block 0x79, offset 0x3e5
-	{value: 0x0000, lo: 0x04},
-	{value: 0x44ed, lo: 0x80, hi: 0x8f},
-	{value: 0x450d, lo: 0x90, hi: 0x9f},
-	{value: 0x452d, lo: 0xa0, hi: 0xaf},
-	{value: 0x450d, lo: 0xb0, hi: 0xbf},
-	// Block 0x7a, offset 0x3ea
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0xa2},
-	{value: 0x3008, lo: 0xa3, hi: 0xa4},
-	{value: 0x3308, lo: 0xa5, hi: 0xa5},
-	{value: 0x3008, lo: 0xa6, hi: 0xa7},
-	{value: 0x3308, lo: 0xa8, hi: 0xa8},
-	{value: 0x3008, lo: 0xa9, hi: 0xaa},
-	{value: 0x0018, lo: 0xab, hi: 0xab},
-	{value: 0x3008, lo: 0xac, hi: 0xac},
-	{value: 0x3b08, lo: 0xad, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0x7b, offset 0x3f7
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xbf},
-	// Block 0x7c, offset 0x3fb
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x8a},
-	{value: 0x0018, lo: 0x8b, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0x7d, offset 0x400
-	{value: 0x0020, lo: 0x01},
-	{value: 0x454d, lo: 0x80, hi: 0xbf},
-	// Block 0x7e, offset 0x402
-	{value: 0x0020, lo: 0x03},
-	{value: 0x4d4d, lo: 0x80, hi: 0x94},
-	{value: 0x4b0d, lo: 0x95, hi: 0x95},
-	{value: 0x4fed, lo: 0x96, hi: 0xbf},
-	// Block 0x7f, offset 0x406
-	{value: 0x0020, lo: 0x01},
-	{value: 0x552d, lo: 0x80, hi: 0xbf},
-	// Block 0x80, offset 0x408
-	{value: 0x0020, lo: 0x03},
-	{value: 0x5d2d, lo: 0x80, hi: 0x84},
-	{value: 0x568d, lo: 0x85, hi: 0x85},
-	{value: 0x5dcd, lo: 0x86, hi: 0xbf},
-	// Block 0x81, offset 0x40c
-	{value: 0x0020, lo: 0x08},
-	{value: 0x6b8d, lo: 0x80, hi: 0x8f},
-	{value: 0x6d4d, lo: 0x90, hi: 0x90},
-	{value: 0x6d8d, lo: 0x91, hi: 0xab},
-	{value: 0x6ea1, lo: 0xac, hi: 0xac},
-	{value: 0x70ed, lo: 0xad, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x710d, lo: 0xb0, hi: 0xbf},
-	// Block 0x82, offset 0x415
-	{value: 0x0020, lo: 0x05},
-	{value: 0x730d, lo: 0x80, hi: 0xad},
-	{value: 0x656d, lo: 0xae, hi: 0xae},
-	{value: 0x78cd, lo: 0xaf, hi: 0xb5},
-	{value: 0x6f8d, lo: 0xb6, hi: 0xb6},
-	{value: 0x79ad, lo: 0xb7, hi: 0xbf},
-	// Block 0x83, offset 0x41b
-	{value: 0x0028, lo: 0x03},
-	{value: 0x7c21, lo: 0x80, hi: 0x82},
-	{value: 0x7be1, lo: 0x83, hi: 0x83},
-	{value: 0x7c99, lo: 0x84, hi: 0xbf},
-	// Block 0x84, offset 0x41f
-	{value: 0x0038, lo: 0x0f},
-	{value: 0x9db1, lo: 0x80, hi: 0x83},
-	{value: 0x9e59, lo: 0x84, hi: 0x85},
-	{value: 0x9e91, lo: 0x86, hi: 0x87},
-	{value: 0x9ec9, lo: 0x88, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x91},
-	{value: 0xa089, lo: 0x92, hi: 0x97},
-	{value: 0xa1a1, lo: 0x98, hi: 0x9c},
-	{value: 0xa281, lo: 0x9d, hi: 0xb3},
-	{value: 0x9d41, lo: 0xb4, hi: 0xb4},
-	{value: 0x9db1, lo: 0xb5, hi: 0xb5},
-	{value: 0xa789, lo: 0xb6, hi: 0xbb},
-	{value: 0xa869, lo: 0xbc, hi: 0xbc},
-	{value: 0xa7f9, lo: 0xbd, hi: 0xbd},
-	{value: 0xa8d9, lo: 0xbe, hi: 0xbf},
-	// Block 0x85, offset 0x42f
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8c},
-	{value: 0x0008, lo: 0x8d, hi: 0xa6},
-	{value: 0x0040, lo: 0xa7, hi: 0xa7},
-	{value: 0x0008, lo: 0xa8, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbb},
-	{value: 0x0008, lo: 0xbc, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbe},
-	{value: 0x0008, lo: 0xbf, hi: 0xbf},
-	// Block 0x86, offset 0x439
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0xbf},
-	// Block 0x87, offset 0x43e
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x88, offset 0x441
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0x82},
-	{value: 0x0040, lo: 0x83, hi: 0x86},
-	{value: 0x0018, lo: 0x87, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xbf},
-	// Block 0x89, offset 0x447
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0018, lo: 0x80, hi: 0x8e},
-	{value: 0x0040, lo: 0x8f, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0x9b},
-	{value: 0x0040, lo: 0x9c, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xa0},
-	{value: 0x0040, lo: 0xa1, hi: 0xbf},
-	// Block 0x8a, offset 0x44e
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0040, lo: 0x80, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0xbc},
-	{value: 0x3308, lo: 0xbd, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0x8b, offset 0x453
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0x9c},
-	{value: 0x0040, lo: 0x9d, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x8c, offset 0x457
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x90},
-	{value: 0x0040, lo: 0x91, hi: 0x9f},
-	{value: 0x3308, lo: 0xa0, hi: 0xa0},
-	{value: 0x0018, lo: 0xa1, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0x8d, offset 0x45d
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xac},
-	{value: 0x0008, lo: 0xad, hi: 0xbf},
-	// Block 0x8e, offset 0x462
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x81},
-	{value: 0x0008, lo: 0x82, hi: 0x89},
-	{value: 0x0018, lo: 0x8a, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x8f, offset 0x46b
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9e},
-	{value: 0x0018, lo: 0x9f, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x90, offset 0x470
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x83},
-	{value: 0x0040, lo: 0x84, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0xbf},
-	// Block 0x91, offset 0x476
-	{value: 0x0000, lo: 0x06},
-	{value: 0xe145, lo: 0x80, hi: 0x87},
-	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
-	{value: 0xe145, lo: 0x90, hi: 0x97},
-	{value: 0x8b0d, lo: 0x98, hi: 0x9f},
-	{value: 0x8b25, lo: 0xa0, hi: 0xa7},
-	{value: 0x0008, lo: 0xa8, hi: 0xbf},
-	// Block 0x92, offset 0x47d
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0008, lo: 0x80, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa9},
-	{value: 0x0040, lo: 0xaa, hi: 0xaf},
-	{value: 0x8b25, lo: 0xb0, hi: 0xb7},
-	{value: 0x8b0d, lo: 0xb8, hi: 0xbf},
-	// Block 0x93, offset 0x484
-	{value: 0x0000, lo: 0x06},
-	{value: 0xe145, lo: 0x80, hi: 0x87},
-	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
-	{value: 0xe145, lo: 0x90, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0x94, offset 0x48b
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x95, offset 0x48f
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xae},
-	{value: 0x0018, lo: 0xaf, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-	// Block 0x96, offset 0x494
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0x97, offset 0x497
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xbf},
-	// Block 0x98, offset 0x49c
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0808, lo: 0x80, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x87},
-	{value: 0x0808, lo: 0x88, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x0808, lo: 0x8a, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb6},
-	{value: 0x0808, lo: 0xb7, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbb},
-	{value: 0x0808, lo: 0xbc, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbe},
-	{value: 0x0808, lo: 0xbf, hi: 0xbf},
-	// Block 0x99, offset 0x4a8
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0808, lo: 0x80, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x96},
-	{value: 0x0818, lo: 0x97, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xb6},
-	{value: 0x0818, lo: 0xb7, hi: 0xbf},
-	// Block 0x9a, offset 0x4ae
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0808, lo: 0x80, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0xa6},
-	{value: 0x0818, lo: 0xa7, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-	// Block 0x9b, offset 0x4b3
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xb3},
-	{value: 0x0808, lo: 0xb4, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xba},
-	{value: 0x0818, lo: 0xbb, hi: 0xbf},
-	// Block 0x9c, offset 0x4ba
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0808, lo: 0x80, hi: 0x95},
-	{value: 0x0818, lo: 0x96, hi: 0x9b},
-	{value: 0x0040, lo: 0x9c, hi: 0x9e},
-	{value: 0x0018, lo: 0x9f, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbe},
-	{value: 0x0818, lo: 0xbf, hi: 0xbf},
-	// Block 0x9d, offset 0x4c2
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0808, lo: 0x80, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xbb},
-	{value: 0x0818, lo: 0xbc, hi: 0xbd},
-	{value: 0x0808, lo: 0xbe, hi: 0xbf},
-	// Block 0x9e, offset 0x4c7
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0818, lo: 0x80, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0x91},
-	{value: 0x0818, lo: 0x92, hi: 0xbf},
-	// Block 0x9f, offset 0x4cb
-	{value: 0x0000, lo: 0x0f},
-	{value: 0x0808, lo: 0x80, hi: 0x80},
-	{value: 0x3308, lo: 0x81, hi: 0x83},
-	{value: 0x0040, lo: 0x84, hi: 0x84},
-	{value: 0x3308, lo: 0x85, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x8b},
-	{value: 0x3308, lo: 0x8c, hi: 0x8f},
-	{value: 0x0808, lo: 0x90, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x94},
-	{value: 0x0808, lo: 0x95, hi: 0x97},
-	{value: 0x0040, lo: 0x98, hi: 0x98},
-	{value: 0x0808, lo: 0x99, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb7},
-	{value: 0x3308, lo: 0xb8, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0xa0, offset 0x4db
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0818, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x8f},
-	{value: 0x0818, lo: 0x90, hi: 0x98},
-	{value: 0x0040, lo: 0x99, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xbc},
-	{value: 0x0818, lo: 0xbd, hi: 0xbf},
-	// Block 0xa1, offset 0x4e2
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0808, lo: 0x80, hi: 0x9c},
-	{value: 0x0818, lo: 0x9d, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0xa2, offset 0x4e6
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0808, lo: 0x80, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb8},
-	{value: 0x0018, lo: 0xb9, hi: 0xbf},
-	// Block 0xa3, offset 0x4ea
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0808, lo: 0x80, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x97},
-	{value: 0x0818, lo: 0x98, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xb7},
-	{value: 0x0818, lo: 0xb8, hi: 0xbf},
-	// Block 0xa4, offset 0x4f1
-	{value: 0x0000, lo: 0x01},
-	{value: 0x0808, lo: 0x80, hi: 0xbf},
-	// Block 0xa5, offset 0x4f3
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0808, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0xbf},
-	// Block 0xa6, offset 0x4f6
-	{value: 0x0000, lo: 0x02},
-	{value: 0x03dd, lo: 0x80, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xbf},
-	// Block 0xa7, offset 0x4f9
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0808, lo: 0x80, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xb9},
-	{value: 0x0818, lo: 0xba, hi: 0xbf},
-	// Block 0xa8, offset 0x4fd
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0908, lo: 0x80, hi: 0x80},
-	{value: 0x0a08, lo: 0x81, hi: 0xa1},
-	{value: 0x0c08, lo: 0xa2, hi: 0xa2},
-	{value: 0x0a08, lo: 0xa3, hi: 0xa3},
-	{value: 0x3308, lo: 0xa4, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xaf},
-	{value: 0x0808, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0xa9, offset 0x506
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0818, lo: 0xa0, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0xaa, offset 0x50a
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0808, lo: 0x80, hi: 0x9c},
-	{value: 0x0818, lo: 0x9d, hi: 0xa6},
-	{value: 0x0808, lo: 0xa7, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xaf},
-	{value: 0x0a08, lo: 0xb0, hi: 0xb2},
-	{value: 0x0c08, lo: 0xb3, hi: 0xb3},
-	{value: 0x0a08, lo: 0xb4, hi: 0xbf},
-	// Block 0xab, offset 0x512
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0a08, lo: 0x80, hi: 0x84},
-	{value: 0x0808, lo: 0x85, hi: 0x85},
-	{value: 0x3308, lo: 0x86, hi: 0x90},
-	{value: 0x0a18, lo: 0x91, hi: 0x93},
-	{value: 0x0c18, lo: 0x94, hi: 0x94},
-	{value: 0x0818, lo: 0x95, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0xbf},
-	// Block 0xac, offset 0x51a
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0808, lo: 0xa0, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0xad, offset 0x51e
-	{value: 0x0000, lo: 0x05},
-	{value: 0x3008, lo: 0x80, hi: 0x80},
-	{value: 0x3308, lo: 0x81, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0xb7},
-	{value: 0x3308, lo: 0xb8, hi: 0xbf},
-	// Block 0xae, offset 0x524
-	{value: 0x0000, lo: 0x08},
-	{value: 0x3308, lo: 0x80, hi: 0x85},
-	{value: 0x3b08, lo: 0x86, hi: 0x86},
-	{value: 0x0018, lo: 0x87, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x91},
-	{value: 0x0018, lo: 0x92, hi: 0xa5},
-	{value: 0x0008, lo: 0xa6, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0xaf, offset 0x52d
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x3308, lo: 0x80, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0xaf},
-	{value: 0x3008, lo: 0xb0, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb6},
-	{value: 0x3008, lo: 0xb7, hi: 0xb8},
-	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
-	{value: 0x3308, lo: 0xba, hi: 0xba},
-	{value: 0x0018, lo: 0xbb, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbd},
-	{value: 0x0018, lo: 0xbe, hi: 0xbf},
-	// Block 0xb0, offset 0x539
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0018, lo: 0x80, hi: 0x81},
-	{value: 0x0040, lo: 0x82, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xa8},
-	{value: 0x0040, lo: 0xa9, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0xb1, offset 0x540
-	{value: 0x0000, lo: 0x08},
-	{value: 0x3308, lo: 0x80, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0xa6},
-	{value: 0x3308, lo: 0xa7, hi: 0xab},
-	{value: 0x3008, lo: 0xac, hi: 0xac},
-	{value: 0x3308, lo: 0xad, hi: 0xb2},
-	{value: 0x3b08, lo: 0xb3, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xb5},
-	{value: 0x0008, lo: 0xb6, hi: 0xbf},
-	// Block 0xb2, offset 0x549
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0018, lo: 0x80, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0x84},
-	{value: 0x3008, lo: 0x85, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb3},
-	{value: 0x0018, lo: 0xb4, hi: 0xb5},
-	{value: 0x0008, lo: 0xb6, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0xb3, offset 0x553
-	{value: 0x0000, lo: 0x06},
-	{value: 0x3308, lo: 0x80, hi: 0x81},
-	{value: 0x3008, lo: 0x82, hi: 0x82},
-	{value: 0x0008, lo: 0x83, hi: 0xb2},
-	{value: 0x3008, lo: 0xb3, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xbe},
-	{value: 0x3008, lo: 0xbf, hi: 0xbf},
-	// Block 0xb4, offset 0x55a
-	{value: 0x0000, lo: 0x0d},
-	{value: 0x3808, lo: 0x80, hi: 0x80},
-	{value: 0x0008, lo: 0x81, hi: 0x84},
-	{value: 0x0018, lo: 0x85, hi: 0x88},
-	{value: 0x3308, lo: 0x89, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0x9b},
-	{value: 0x0008, lo: 0x9c, hi: 0x9c},
-	{value: 0x0018, lo: 0x9d, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xa0},
-	{value: 0x0018, lo: 0xa1, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0xb5, offset 0x568
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x91},
-	{value: 0x0040, lo: 0x92, hi: 0x92},
-	{value: 0x0008, lo: 0x93, hi: 0xab},
-	{value: 0x3008, lo: 0xac, hi: 0xae},
-	{value: 0x3308, lo: 0xaf, hi: 0xb1},
-	{value: 0x3008, lo: 0xb2, hi: 0xb3},
-	{value: 0x3308, lo: 0xb4, hi: 0xb4},
-	{value: 0x3808, lo: 0xb5, hi: 0xb5},
-	{value: 0x3308, lo: 0xb6, hi: 0xb7},
-	{value: 0x0018, lo: 0xb8, hi: 0xbd},
-	{value: 0x3308, lo: 0xbe, hi: 0xbe},
-	{value: 0x0040, lo: 0xbf, hi: 0xbf},
-	// Block 0xb6, offset 0x575
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x0008, lo: 0x8a, hi: 0x8d},
-	{value: 0x0040, lo: 0x8e, hi: 0x8e},
-	{value: 0x0008, lo: 0x8f, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9e},
-	{value: 0x0008, lo: 0x9f, hi: 0xa8},
-	{value: 0x0018, lo: 0xa9, hi: 0xa9},
-	{value: 0x0040, lo: 0xaa, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0xb7, offset 0x582
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0x9e},
-	{value: 0x3308, lo: 0x9f, hi: 0x9f},
-	{value: 0x3008, lo: 0xa0, hi: 0xa2},
-	{value: 0x3308, lo: 0xa3, hi: 0xa9},
-	{value: 0x3b08, lo: 0xaa, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbf},
-	// Block 0xb8, offset 0x58b
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xb4},
-	{value: 0x3008, lo: 0xb5, hi: 0xb7},
-	{value: 0x3308, lo: 0xb8, hi: 0xbf},
-	// Block 0xb9, offset 0x58f
-	{value: 0x0000, lo: 0x0f},
-	{value: 0x3008, lo: 0x80, hi: 0x81},
-	{value: 0x3b08, lo: 0x82, hi: 0x82},
-	{value: 0x3308, lo: 0x83, hi: 0x84},
-	{value: 0x3008, lo: 0x85, hi: 0x85},
-	{value: 0x3308, lo: 0x86, hi: 0x86},
-	{value: 0x0008, lo: 0x87, hi: 0x8a},
-	{value: 0x0018, lo: 0x8b, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0x9b},
-	{value: 0x0040, lo: 0x9c, hi: 0x9c},
-	{value: 0x0018, lo: 0x9d, hi: 0x9d},
-	{value: 0x3308, lo: 0x9e, hi: 0x9e},
-	{value: 0x0008, lo: 0x9f, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0xba, offset 0x59f
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x3008, lo: 0xb0, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb8},
-	{value: 0x3008, lo: 0xb9, hi: 0xb9},
-	{value: 0x3308, lo: 0xba, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbe},
-	{value: 0x3308, lo: 0xbf, hi: 0xbf},
-	// Block 0xbb, offset 0x5a7
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x3308, lo: 0x80, hi: 0x80},
-	{value: 0x3008, lo: 0x81, hi: 0x81},
-	{value: 0x3b08, lo: 0x82, hi: 0x82},
-	{value: 0x3308, lo: 0x83, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0x85},
-	{value: 0x0018, lo: 0x86, hi: 0x86},
-	{value: 0x0008, lo: 0x87, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0xbf},
-	// Block 0xbc, offset 0x5b2
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0xae},
-	{value: 0x3008, lo: 0xaf, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xb7},
-	{value: 0x3008, lo: 0xb8, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0xbd, offset 0x5bb
-	{value: 0x0000, lo: 0x05},
-	{value: 0x3308, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0x9b},
-	{value: 0x3308, lo: 0x9c, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0xbf},
-	// Block 0xbe, offset 0x5c1
-	{value: 0x0000, lo: 0x07},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x3008, lo: 0xb0, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xba},
-	{value: 0x3008, lo: 0xbb, hi: 0xbc},
-	{value: 0x3308, lo: 0xbd, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0xbf, offset 0x5c9
-	{value: 0x0000, lo: 0x08},
-	{value: 0x3308, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x83},
-	{value: 0x0008, lo: 0x84, hi: 0x84},
-	{value: 0x0040, lo: 0x85, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xbf},
-	// Block 0xc0, offset 0x5d2
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0xaa},
-	{value: 0x3308, lo: 0xab, hi: 0xab},
-	{value: 0x3008, lo: 0xac, hi: 0xac},
-	{value: 0x3308, lo: 0xad, hi: 0xad},
-	{value: 0x3008, lo: 0xae, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb5},
-	{value: 0x3808, lo: 0xb6, hi: 0xb6},
-	{value: 0x3308, lo: 0xb7, hi: 0xb7},
-	{value: 0x0008, lo: 0xb8, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0xc1, offset 0x5dd
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0xbf},
-	// Block 0xc2, offset 0x5e0
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0x9a},
-	{value: 0x0040, lo: 0x9b, hi: 0x9c},
-	{value: 0x3308, lo: 0x9d, hi: 0x9f},
-	{value: 0x3008, lo: 0xa0, hi: 0xa1},
-	{value: 0x3308, lo: 0xa2, hi: 0xa5},
-	{value: 0x3008, lo: 0xa6, hi: 0xa6},
-	{value: 0x3308, lo: 0xa7, hi: 0xaa},
-	{value: 0x3b08, lo: 0xab, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb9},
-	{value: 0x0018, lo: 0xba, hi: 0xbf},
-	// Block 0xc3, offset 0x5ec
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0xab},
-	{value: 0x3008, lo: 0xac, hi: 0xae},
-	{value: 0x3308, lo: 0xaf, hi: 0xb7},
-	{value: 0x3008, lo: 0xb8, hi: 0xb8},
-	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
-	{value: 0x3308, lo: 0xba, hi: 0xba},
-	{value: 0x0018, lo: 0xbb, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0xc4, offset 0x5f5
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x049d, lo: 0xa0, hi: 0xbf},
-	// Block 0xc5, offset 0x5f8
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xa9},
-	{value: 0x0018, lo: 0xaa, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xbe},
-	{value: 0x0008, lo: 0xbf, hi: 0xbf},
-	// Block 0xc6, offset 0x5fd
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xa9},
-	{value: 0x0008, lo: 0xaa, hi: 0xbf},
-	// Block 0xc7, offset 0x602
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x90},
-	{value: 0x3008, lo: 0x91, hi: 0x93},
-	{value: 0x3308, lo: 0x94, hi: 0x97},
-	{value: 0x0040, lo: 0x98, hi: 0x99},
-	{value: 0x3308, lo: 0x9a, hi: 0x9b},
-	{value: 0x3008, lo: 0x9c, hi: 0x9f},
-	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
-	{value: 0x0008, lo: 0xa1, hi: 0xa1},
-	{value: 0x0018, lo: 0xa2, hi: 0xa2},
-	{value: 0x0008, lo: 0xa3, hi: 0xa3},
-	{value: 0x3008, lo: 0xa4, hi: 0xa4},
-	{value: 0x0040, lo: 0xa5, hi: 0xbf},
-	// Block 0xc8, offset 0x60f
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x3308, lo: 0x81, hi: 0x8a},
-	{value: 0x0008, lo: 0x8b, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb3},
-	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
-	{value: 0x3308, lo: 0xb5, hi: 0xb8},
-	{value: 0x3008, lo: 0xb9, hi: 0xb9},
-	{value: 0x0008, lo: 0xba, hi: 0xba},
-	{value: 0x3308, lo: 0xbb, hi: 0xbe},
-	{value: 0x0018, lo: 0xbf, hi: 0xbf},
-	// Block 0xc9, offset 0x61a
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0018, lo: 0x80, hi: 0x86},
-	{value: 0x3b08, lo: 0x87, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x90},
-	{value: 0x3308, lo: 0x91, hi: 0x96},
-	{value: 0x3008, lo: 0x97, hi: 0x98},
-	{value: 0x3308, lo: 0x99, hi: 0x9b},
-	{value: 0x0008, lo: 0x9c, hi: 0xbf},
-	// Block 0xca, offset 0x623
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x3308, lo: 0x8a, hi: 0x96},
-	{value: 0x3008, lo: 0x97, hi: 0x97},
-	{value: 0x3308, lo: 0x98, hi: 0x98},
-	{value: 0x3b08, lo: 0x99, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0x9c},
-	{value: 0x0008, lo: 0x9d, hi: 0x9d},
-	{value: 0x0018, lo: 0x9e, hi: 0xa2},
-	{value: 0x0040, lo: 0xa3, hi: 0xbf},
-	// Block 0xcb, offset 0x62d
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0xcc, offset 0x630
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x89},
-	{value: 0x0008, lo: 0x8a, hi: 0xae},
-	{value: 0x3008, lo: 0xaf, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xb7},
-	{value: 0x3308, lo: 0xb8, hi: 0xbd},
-	{value: 0x3008, lo: 0xbe, hi: 0xbe},
-	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
-	// Block 0xcd, offset 0x63a
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0008, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0018, lo: 0x9a, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb1},
-	{value: 0x0008, lo: 0xb2, hi: 0xbf},
-	// Block 0xce, offset 0x643
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x0008, lo: 0x80, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0x91},
-	{value: 0x3308, lo: 0x92, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xa8},
-	{value: 0x3008, lo: 0xa9, hi: 0xa9},
-	{value: 0x3308, lo: 0xaa, hi: 0xb0},
-	{value: 0x3008, lo: 0xb1, hi: 0xb1},
-	{value: 0x3308, lo: 0xb2, hi: 0xb3},
-	{value: 0x3008, lo: 0xb4, hi: 0xb4},
-	{value: 0x3308, lo: 0xb5, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xbf},
-	// Block 0xcf, offset 0x64f
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x87},
-	{value: 0x0008, lo: 0x88, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0x8a},
-	{value: 0x0008, lo: 0x8b, hi: 0xb0},
-	{value: 0x3308, lo: 0xb1, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xb9},
-	{value: 0x3308, lo: 0xba, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbb},
-	{value: 0x3308, lo: 0xbc, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbe},
-	{value: 0x3308, lo: 0xbf, hi: 0xbf},
-	// Block 0xd0, offset 0x65c
-	{value: 0x0000, lo: 0x0c},
-	{value: 0x3308, lo: 0x80, hi: 0x83},
-	{value: 0x3b08, lo: 0x84, hi: 0x85},
-	{value: 0x0008, lo: 0x86, hi: 0x86},
-	{value: 0x3308, lo: 0x87, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa5},
-	{value: 0x0040, lo: 0xa6, hi: 0xa6},
-	{value: 0x0008, lo: 0xa7, hi: 0xa8},
-	{value: 0x0040, lo: 0xa9, hi: 0xa9},
-	{value: 0x0008, lo: 0xaa, hi: 0xbf},
-	// Block 0xd1, offset 0x669
-	{value: 0x0000, lo: 0x0d},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x3008, lo: 0x8a, hi: 0x8e},
-	{value: 0x0040, lo: 0x8f, hi: 0x8f},
-	{value: 0x3308, lo: 0x90, hi: 0x91},
-	{value: 0x0040, lo: 0x92, hi: 0x92},
-	{value: 0x3008, lo: 0x93, hi: 0x94},
-	{value: 0x3308, lo: 0x95, hi: 0x95},
-	{value: 0x3008, lo: 0x96, hi: 0x96},
-	{value: 0x3b08, lo: 0x97, hi: 0x97},
-	{value: 0x0008, lo: 0x98, hi: 0x98},
-	{value: 0x0040, lo: 0x99, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa9},
-	{value: 0x0040, lo: 0xaa, hi: 0xbf},
-	// Block 0xd2, offset 0x677
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xb2},
-	{value: 0x3308, lo: 0xb3, hi: 0xb4},
-	{value: 0x3008, lo: 0xb5, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0xd3, offset 0x67e
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xb1},
-	{value: 0x0040, lo: 0xb2, hi: 0xbe},
-	{value: 0x0018, lo: 0xbf, hi: 0xbf},
-	// Block 0xd4, offset 0x682
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0xbf},
-	// Block 0xd5, offset 0x685
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0xd6, offset 0x68a
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x83},
-	{value: 0x0040, lo: 0x84, hi: 0xbf},
-	// Block 0xd7, offset 0x68d
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xaf},
-	{value: 0x0340, lo: 0xb0, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0xd8, offset 0x692
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0xbf},
-	// Block 0xd9, offset 0x695
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0008, lo: 0x80, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa9},
-	{value: 0x0040, lo: 0xaa, hi: 0xad},
-	{value: 0x0018, lo: 0xae, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-	// Block 0xda, offset 0x69c
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0040, lo: 0x80, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb4},
-	{value: 0x0018, lo: 0xb5, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xbf},
-	// Block 0xdb, offset 0x6a3
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xbf},
-	// Block 0xdc, offset 0x6a7
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x0008, lo: 0x80, hi: 0x83},
-	{value: 0x0018, lo: 0x84, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9a},
-	{value: 0x0018, lo: 0x9b, hi: 0xa1},
-	{value: 0x0040, lo: 0xa2, hi: 0xa2},
-	{value: 0x0008, lo: 0xa3, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xbc},
-	{value: 0x0008, lo: 0xbd, hi: 0xbf},
-	// Block 0xdd, offset 0x6b2
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0xbf},
-	// Block 0xde, offset 0x6b5
-	{value: 0x0000, lo: 0x02},
-	{value: 0xe105, lo: 0x80, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0xdf, offset 0x6b8
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0x9a},
-	{value: 0x0040, lo: 0x9b, hi: 0xbf},
-	// Block 0xe0, offset 0x6bb
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8e},
-	{value: 0x3308, lo: 0x8f, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x90},
-	{value: 0x3008, lo: 0x91, hi: 0xbf},
-	// Block 0xe1, offset 0x6c1
-	{value: 0x0000, lo: 0x05},
-	{value: 0x3008, lo: 0x80, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8e},
-	{value: 0x3308, lo: 0x8f, hi: 0x92},
-	{value: 0x0008, lo: 0x93, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0xe2, offset 0x6c7
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xa1},
-	{value: 0x0018, lo: 0xa2, hi: 0xa2},
-	{value: 0x0008, lo: 0xa3, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xbf},
-	// Block 0xe3, offset 0x6cd
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb7},
-	{value: 0x0040, lo: 0xb8, hi: 0xbf},
-	// Block 0xe4, offset 0x6d0
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb2},
-	{value: 0x0040, lo: 0xb3, hi: 0xbf},
-	// Block 0xe5, offset 0x6d3
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x9e},
-	{value: 0x0040, lo: 0x9f, hi: 0xbf},
-	// Block 0xe6, offset 0x6d6
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0040, lo: 0x80, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x92},
-	{value: 0x0040, lo: 0x93, hi: 0xa3},
-	{value: 0x0008, lo: 0xa4, hi: 0xa7},
-	{value: 0x0040, lo: 0xa8, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0xe7, offset 0x6dd
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xbb},
-	{value: 0x0040, lo: 0xbc, hi: 0xbf},
-	// Block 0xe8, offset 0x6e0
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0008, lo: 0x80, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbc},
-	{value: 0x0040, lo: 0xbd, hi: 0xbf},
-	// Block 0xe9, offset 0x6e5
-	{value: 0x0000, lo: 0x09},
-	{value: 0x0008, lo: 0x80, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x8f},
-	{value: 0x0008, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9b},
-	{value: 0x0018, lo: 0x9c, hi: 0x9c},
-	{value: 0x3308, lo: 0x9d, hi: 0x9e},
-	{value: 0x0018, lo: 0x9f, hi: 0x9f},
-	{value: 0x03c0, lo: 0xa0, hi: 0xa3},
-	{value: 0x0040, lo: 0xa4, hi: 0xbf},
-	// Block 0xea, offset 0x6ef
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xbf},
-	// Block 0xeb, offset 0x6f2
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xa6},
-	{value: 0x0040, lo: 0xa7, hi: 0xa8},
-	{value: 0x0018, lo: 0xa9, hi: 0xbf},
-	// Block 0xec, offset 0x6f6
-	{value: 0x0000, lo: 0x0e},
-	{value: 0x0018, lo: 0x80, hi: 0x9d},
-	{value: 0xb5b9, lo: 0x9e, hi: 0x9e},
-	{value: 0xb601, lo: 0x9f, hi: 0x9f},
-	{value: 0xb649, lo: 0xa0, hi: 0xa0},
-	{value: 0xb6b1, lo: 0xa1, hi: 0xa1},
-	{value: 0xb719, lo: 0xa2, hi: 0xa2},
-	{value: 0xb781, lo: 0xa3, hi: 0xa3},
-	{value: 0xb7e9, lo: 0xa4, hi: 0xa4},
-	{value: 0x3018, lo: 0xa5, hi: 0xa6},
-	{value: 0x3318, lo: 0xa7, hi: 0xa9},
-	{value: 0x0018, lo: 0xaa, hi: 0xac},
-	{value: 0x3018, lo: 0xad, hi: 0xb2},
-	{value: 0x0340, lo: 0xb3, hi: 0xba},
-	{value: 0x3318, lo: 0xbb, hi: 0xbf},
-	// Block 0xed, offset 0x705
-	{value: 0x0000, lo: 0x0b},
-	{value: 0x3318, lo: 0x80, hi: 0x82},
-	{value: 0x0018, lo: 0x83, hi: 0x84},
-	{value: 0x3318, lo: 0x85, hi: 0x8b},
-	{value: 0x0018, lo: 0x8c, hi: 0xa9},
-	{value: 0x3318, lo: 0xaa, hi: 0xad},
-	{value: 0x0018, lo: 0xae, hi: 0xba},
-	{value: 0xb851, lo: 0xbb, hi: 0xbb},
-	{value: 0xb899, lo: 0xbc, hi: 0xbc},
-	{value: 0xb8e1, lo: 0xbd, hi: 0xbd},
-	{value: 0xb949, lo: 0xbe, hi: 0xbe},
-	{value: 0xb9b1, lo: 0xbf, hi: 0xbf},
-	// Block 0xee, offset 0x711
-	{value: 0x0000, lo: 0x03},
-	{value: 0xba19, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0xa8},
-	{value: 0x0040, lo: 0xa9, hi: 0xbf},
-	// Block 0xef, offset 0x715
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x81},
-	{value: 0x3318, lo: 0x82, hi: 0x84},
-	{value: 0x0018, lo: 0x85, hi: 0x85},
-	{value: 0x0040, lo: 0x86, hi: 0xbf},
-	// Block 0xf0, offset 0x71a
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xbf},
-	// Block 0xf1, offset 0x71e
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xb8},
-	{value: 0x0040, lo: 0xb9, hi: 0xbf},
-	// Block 0xf2, offset 0x723
-	{value: 0x0000, lo: 0x03},
-	{value: 0x3308, lo: 0x80, hi: 0xb6},
-	{value: 0x0018, lo: 0xb7, hi: 0xba},
-	{value: 0x3308, lo: 0xbb, hi: 0xbf},
-	// Block 0xf3, offset 0x727
-	{value: 0x0000, lo: 0x04},
-	{value: 0x3308, lo: 0x80, hi: 0xac},
-	{value: 0x0018, lo: 0xad, hi: 0xb4},
-	{value: 0x3308, lo: 0xb5, hi: 0xb5},
-	{value: 0x0018, lo: 0xb6, hi: 0xbf},
-	// Block 0xf4, offset 0x72c
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0018, lo: 0x80, hi: 0x83},
-	{value: 0x3308, lo: 0x84, hi: 0x84},
-	{value: 0x0018, lo: 0x85, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x9a},
-	{value: 0x3308, lo: 0x9b, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xa0},
-	{value: 0x3308, lo: 0xa1, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-	// Block 0xf5, offset 0x735
-	{value: 0x0000, lo: 0x0a},
-	{value: 0x3308, lo: 0x80, hi: 0x86},
-	{value: 0x0040, lo: 0x87, hi: 0x87},
-	{value: 0x3308, lo: 0x88, hi: 0x98},
-	{value: 0x0040, lo: 0x99, hi: 0x9a},
-	{value: 0x3308, lo: 0x9b, hi: 0xa1},
-	{value: 0x0040, lo: 0xa2, hi: 0xa2},
-	{value: 0x3308, lo: 0xa3, hi: 0xa4},
-	{value: 0x0040, lo: 0xa5, hi: 0xa5},
-	{value: 0x3308, lo: 0xa6, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xbf},
-	// Block 0xf6, offset 0x740
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xaf},
-	{value: 0x3308, lo: 0xb0, hi: 0xb6},
-	{value: 0x0008, lo: 0xb7, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0xf7, offset 0x746
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0x89},
-	{value: 0x0040, lo: 0x8a, hi: 0x8d},
-	{value: 0x0008, lo: 0x8e, hi: 0x8e},
-	{value: 0x0018, lo: 0x8f, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0xbf},
-	// Block 0xf8, offset 0x74c
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0008, lo: 0x80, hi: 0xab},
-	{value: 0x3308, lo: 0xac, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xb9},
-	{value: 0x0040, lo: 0xba, hi: 0xbe},
-	{value: 0x0018, lo: 0xbf, hi: 0xbf},
-	// Block 0xf9, offset 0x752
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0808, lo: 0x80, hi: 0x84},
-	{value: 0x0040, lo: 0x85, hi: 0x86},
-	{value: 0x0818, lo: 0x87, hi: 0x8f},
-	{value: 0x3308, lo: 0x90, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0xbf},
-	// Block 0xfa, offset 0x758
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0a08, lo: 0x80, hi: 0x83},
-	{value: 0x3308, lo: 0x84, hi: 0x8a},
-	{value: 0x0b08, lo: 0x8b, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8f},
-	{value: 0x0808, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9d},
-	{value: 0x0818, lo: 0x9e, hi: 0x9f},
-	{value: 0x0040, lo: 0xa0, hi: 0xbf},
-	// Block 0xfb, offset 0x761
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0040, lo: 0x80, hi: 0xb0},
-	{value: 0x0818, lo: 0xb1, hi: 0xbf},
-	// Block 0xfc, offset 0x764
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0818, lo: 0x80, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0xfd, offset 0x767
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0818, lo: 0x81, hi: 0xbd},
-	{value: 0x0040, lo: 0xbe, hi: 0xbf},
-	// Block 0xfe, offset 0x76b
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0040, lo: 0x80, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb1},
-	{value: 0x0040, lo: 0xb2, hi: 0xbf},
-	// Block 0xff, offset 0x76f
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xbf},
-	// Block 0x100, offset 0x773
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xae},
-	{value: 0x0040, lo: 0xaf, hi: 0xb0},
-	{value: 0x0018, lo: 0xb1, hi: 0xbf},
-	// Block 0x101, offset 0x779
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0018, lo: 0x81, hi: 0x8f},
-	{value: 0x0040, lo: 0x90, hi: 0x90},
-	{value: 0x0018, lo: 0x91, hi: 0xb5},
-	{value: 0x0040, lo: 0xb6, hi: 0xbf},
-	// Block 0x102, offset 0x77f
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x8f},
-	{value: 0xc1d9, lo: 0x90, hi: 0x90},
-	{value: 0x0018, lo: 0x91, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xbf},
-	// Block 0x103, offset 0x784
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0040, lo: 0x80, hi: 0xa5},
-	{value: 0x0018, lo: 0xa6, hi: 0xbf},
-	// Block 0x104, offset 0x787
-	{value: 0x0000, lo: 0x0f},
-	{value: 0xc801, lo: 0x80, hi: 0x80},
-	{value: 0xc851, lo: 0x81, hi: 0x81},
-	{value: 0xc8a1, lo: 0x82, hi: 0x82},
-	{value: 0xc8f1, lo: 0x83, hi: 0x83},
-	{value: 0xc941, lo: 0x84, hi: 0x84},
-	{value: 0xc991, lo: 0x85, hi: 0x85},
-	{value: 0xc9e1, lo: 0x86, hi: 0x86},
-	{value: 0xca31, lo: 0x87, hi: 0x87},
-	{value: 0xca81, lo: 0x88, hi: 0x88},
-	{value: 0x0040, lo: 0x89, hi: 0x8f},
-	{value: 0xcad1, lo: 0x90, hi: 0x90},
-	{value: 0xcaf1, lo: 0x91, hi: 0x91},
-	{value: 0x0040, lo: 0x92, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xa5},
-	{value: 0x0040, lo: 0xa6, hi: 0xbf},
-	// Block 0x105, offset 0x797
-	{value: 0x0000, lo: 0x06},
-	{value: 0x0018, lo: 0x80, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xac},
-	{value: 0x0040, lo: 0xad, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x106, offset 0x79e
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0018, lo: 0x80, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xbf},
-	// Block 0x107, offset 0x7a1
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x98},
-	{value: 0x0040, lo: 0x99, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xab},
-	{value: 0x0040, lo: 0xac, hi: 0xbf},
-	// Block 0x108, offset 0x7a6
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0xbf},
-	// Block 0x109, offset 0x7aa
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0x99},
-	{value: 0x0040, lo: 0x9a, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xbf},
-	// Block 0x10a, offset 0x7b0
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x87},
-	{value: 0x0040, lo: 0x88, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xbf},
-	// Block 0x10b, offset 0x7b5
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x8b},
-	{value: 0x0040, lo: 0x8c, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0xbf},
-	// Block 0x10c, offset 0x7b9
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0xb1},
-	{value: 0x0040, lo: 0xb2, hi: 0xb2},
-	{value: 0x0018, lo: 0xb3, hi: 0xb6},
-	{value: 0x0040, lo: 0xb7, hi: 0xb9},
-	{value: 0x0018, lo: 0xba, hi: 0xbf},
-	// Block 0x10d, offset 0x7bf
-	{value: 0x0000, lo: 0x05},
-	{value: 0x0018, lo: 0x80, hi: 0xa2},
-	{value: 0x0040, lo: 0xa3, hi: 0xa4},
-	{value: 0x0018, lo: 0xa5, hi: 0xaa},
-	{value: 0x0040, lo: 0xab, hi: 0xad},
-	{value: 0x0018, lo: 0xae, hi: 0xbf},
-	// Block 0x10e, offset 0x7c5
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0018, lo: 0x80, hi: 0x8a},
-	{value: 0x0040, lo: 0x8b, hi: 0x8c},
-	{value: 0x0018, lo: 0x8d, hi: 0xbf},
-	// Block 0x10f, offset 0x7c9
-	{value: 0x0000, lo: 0x08},
-	{value: 0x0018, lo: 0x80, hi: 0x93},
-	{value: 0x0040, lo: 0x94, hi: 0x9f},
-	{value: 0x0018, lo: 0xa0, hi: 0xad},
-	{value: 0x0040, lo: 0xae, hi: 0xaf},
-	{value: 0x0018, lo: 0xb0, hi: 0xb3},
-	{value: 0x0040, lo: 0xb4, hi: 0xb7},
-	{value: 0x0018, lo: 0xb8, hi: 0xba},
-	{value: 0x0040, lo: 0xbb, hi: 0xbf},
-	// Block 0x110, offset 0x7d2
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0018, lo: 0x80, hi: 0x82},
-	{value: 0x0040, lo: 0x83, hi: 0x8f},
-	{value: 0x0018, lo: 0x90, hi: 0x95},
-	{value: 0x0040, lo: 0x96, hi: 0xbf},
-	// Block 0x111, offset 0x7d7
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0x96},
-	{value: 0x0040, lo: 0x97, hi: 0xbf},
-	// Block 0x112, offset 0x7da
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xb4},
-	{value: 0x0040, lo: 0xb5, hi: 0xbf},
-	// Block 0x113, offset 0x7dd
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0x9d},
-	{value: 0x0040, lo: 0x9e, hi: 0x9f},
-	{value: 0x0008, lo: 0xa0, hi: 0xbf},
-	// Block 0x114, offset 0x7e1
-	{value: 0x0000, lo: 0x03},
-	{value: 0x0008, lo: 0x80, hi: 0xa1},
-	{value: 0x0040, lo: 0xa2, hi: 0xaf},
-	{value: 0x0008, lo: 0xb0, hi: 0xbf},
-	// Block 0x115, offset 0x7e5
-	{value: 0x0000, lo: 0x02},
-	{value: 0x0008, lo: 0x80, hi: 0xa0},
-	{value: 0x0040, lo: 0xa1, hi: 0xbf},
-	// Block 0x116, offset 0x7e8
-	{value: 0x0020, lo: 0x0f},
-	{value: 0xded1, lo: 0x80, hi: 0x89},
-	{value: 0x8e35, lo: 0x8a, hi: 0x8a},
-	{value: 0xe011, lo: 0x8b, hi: 0x9c},
-	{value: 0x8e55, lo: 0x9d, hi: 0x9d},
-	{value: 0xe251, lo: 0x9e, hi: 0xa2},
-	{value: 0x8e75, lo: 0xa3, hi: 0xa3},
-	{value: 0xe2f1, lo: 0xa4, hi: 0xab},
-	{value: 0x7f0d, lo: 0xac, hi: 0xac},
-	{value: 0xe3f1, lo: 0xad, hi: 0xaf},
-	{value: 0x8e95, lo: 0xb0, hi: 0xb0},
-	{value: 0xe451, lo: 0xb1, hi: 0xb6},
-	{value: 0x8eb5, lo: 0xb7, hi: 0xb9},
-	{value: 0xe511, lo: 0xba, hi: 0xba},
-	{value: 0x8f15, lo: 0xbb, hi: 0xbb},
-	{value: 0xe531, lo: 0xbc, hi: 0xbf},
-	// Block 0x117, offset 0x7f8
-	{value: 0x0020, lo: 0x10},
-	{value: 0x93b5, lo: 0x80, hi: 0x80},
-	{value: 0xf0b1, lo: 0x81, hi: 0x86},
-	{value: 0x93d5, lo: 0x87, hi: 0x8a},
-	{value: 0xda11, lo: 0x8b, hi: 0x8b},
-	{value: 0xf171, lo: 0x8c, hi: 0x96},
-	{value: 0x9455, lo: 0x97, hi: 0x97},
-	{value: 0xf2d1, lo: 0x98, hi: 0xa3},
-	{value: 0x9475, lo: 0xa4, hi: 0xa6},
-	{value: 0xf451, lo: 0xa7, hi: 0xaa},
-	{value: 0x94d5, lo: 0xab, hi: 0xab},
-	{value: 0xf4d1, lo: 0xac, hi: 0xac},
-	{value: 0x94f5, lo: 0xad, hi: 0xad},
-	{value: 0xf4f1, lo: 0xae, hi: 0xaf},
-	{value: 0x9515, lo: 0xb0, hi: 0xb1},
-	{value: 0xf531, lo: 0xb2, hi: 0xbe},
-	{value: 0x2040, lo: 0xbf, hi: 0xbf},
-	// Block 0x118, offset 0x809
-	{value: 0x0000, lo: 0x04},
-	{value: 0x0040, lo: 0x80, hi: 0x80},
-	{value: 0x0340, lo: 0x81, hi: 0x81},
-	{value: 0x0040, lo: 0x82, hi: 0x9f},
-	{value: 0x0340, lo: 0xa0, hi: 0xbf},
-	// Block 0x119, offset 0x80e
-	{value: 0x0000, lo: 0x01},
-	{value: 0x0340, lo: 0x80, hi: 0xbf},
-	// Block 0x11a, offset 0x810
-	{value: 0x0000, lo: 0x01},
-	{value: 0x33c0, lo: 0x80, hi: 0xbf},
-	// Block 0x11b, offset 0x812
-	{value: 0x0000, lo: 0x02},
-	{value: 0x33c0, lo: 0x80, hi: 0xaf},
-	{value: 0x0040, lo: 0xb0, hi: 0xbf},
-}
-
-// Total table size 42780 bytes (41KiB); checksum: 29936AB9
diff --git a/vendor/golang.org/x/net/idna/tables13.0.0.go b/vendor/golang.org/x/net/idna/tables13.0.0.go
new file mode 100644
index 0000000..390c5e5
--- /dev/null
+++ b/vendor/golang.org/x/net/idna/tables13.0.0.go
@@ -0,0 +1,4840 @@
+// Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
+
+//go:build go1.16
+// +build go1.16
+
+package idna
+
+// UnicodeVersion is the Unicode version from which the tables in this package are derived.
+const UnicodeVersion = "13.0.0"
+
+var mappings string = "" + // Size: 8188 bytes
+	"\x00\x01 \x03 ̈\x01a\x03 ̄\x012\x013\x03 ́\x03 ̧\x011\x01o\x051⁄4\x051⁄2" +
+	"\x053⁄4\x03i̇\x03l·\x03ʼn\x01s\x03dž\x03ⱥ\x03ⱦ\x01h\x01j\x01r\x01w\x01y" +
+	"\x03 ̆\x03 ̇\x03 ̊\x03 ̨\x03 ̃\x03 ̋\x01l\x01x\x04̈́\x03 ι\x01;\x05 ̈́" +
+	"\x04եւ\x04اٴ\x04وٴ\x04ۇٴ\x04يٴ\x06क़\x06ख़\x06ग़\x06ज़\x06ड़\x06ढ़\x06फ़" +
+	"\x06य़\x06ড়\x06ঢ়\x06য়\x06ਲ਼\x06ਸ਼\x06ਖ਼\x06ਗ਼\x06ਜ਼\x06ਫ਼\x06ଡ଼\x06ଢ଼" +
+	"\x06ํา\x06ໍາ\x06ຫນ\x06ຫມ\x06གྷ\x06ཌྷ\x06དྷ\x06བྷ\x06ཛྷ\x06ཀྵ\x06ཱི\x06ཱུ" +
+	"\x06ྲྀ\x09ྲཱྀ\x06ླྀ\x09ླཱྀ\x06ཱྀ\x06ྒྷ\x06ྜྷ\x06ྡྷ\x06ྦྷ\x06ྫྷ\x06ྐྵ\x02" +
+	"в\x02д\x02о\x02с\x02т\x02ъ\x02ѣ\x02æ\x01b\x01d\x01e\x02ǝ\x01g\x01i\x01k" +
+	"\x01m\x01n\x02ȣ\x01p\x01t\x01u\x02ɐ\x02ɑ\x02ə\x02ɛ\x02ɜ\x02ŋ\x02ɔ\x02ɯ" +
+	"\x01v\x02β\x02γ\x02δ\x02φ\x02χ\x02ρ\x02н\x02ɒ\x01c\x02ɕ\x02ð\x01f\x02ɟ" +
+	"\x02ɡ\x02ɥ\x02ɨ\x02ɩ\x02ɪ\x02ʝ\x02ɭ\x02ʟ\x02ɱ\x02ɰ\x02ɲ\x02ɳ\x02ɴ\x02ɵ" +
+	"\x02ɸ\x02ʂ\x02ʃ\x02ƫ\x02ʉ\x02ʊ\x02ʋ\x02ʌ\x01z\x02ʐ\x02ʑ\x02ʒ\x02θ\x02ss" +
+	"\x02ά\x02έ\x02ή\x02ί\x02ό\x02ύ\x02ώ\x05ἀι\x05ἁι\x05ἂι\x05ἃι\x05ἄι\x05ἅι" +
+	"\x05ἆι\x05ἇι\x05ἠι\x05ἡι\x05ἢι\x05ἣι\x05ἤι\x05ἥι\x05ἦι\x05ἧι\x05ὠι\x05ὡι" +
+	"\x05ὢι\x05ὣι\x05ὤι\x05ὥι\x05ὦι\x05ὧι\x05ὰι\x04αι\x04άι\x05ᾶι\x02ι\x05 ̈͂" +
+	"\x05ὴι\x04ηι\x04ήι\x05ῆι\x05 ̓̀\x05 ̓́\x05 ̓͂\x02ΐ\x05 ̔̀\x05 ̔́\x05 ̔͂" +
+	"\x02ΰ\x05 ̈̀\x01`\x05ὼι\x04ωι\x04ώι\x05ῶι\x06′′\x09′′′\x06‵‵\x09‵‵‵\x02!" +
+	"!\x02??\x02?!\x02!?\x0c′′′′\x010\x014\x015\x016\x017\x018\x019\x01+\x01=" +
+	"\x01(\x01)\x02rs\x02ħ\x02no\x01q\x02sm\x02tm\x02ω\x02å\x02א\x02ב\x02ג" +
+	"\x02ד\x02π\x051⁄7\x051⁄9\x061⁄10\x051⁄3\x052⁄3\x051⁄5\x052⁄5\x053⁄5\x054" +
+	"⁄5\x051⁄6\x055⁄6\x051⁄8\x053⁄8\x055⁄8\x057⁄8\x041⁄\x02ii\x02iv\x02vi" +
+	"\x04viii\x02ix\x02xi\x050⁄3\x06∫∫\x09∫∫∫\x06∮∮\x09∮∮∮\x0210\x0211\x0212" +
+	"\x0213\x0214\x0215\x0216\x0217\x0218\x0219\x0220\x04(10)\x04(11)\x04(12)" +
+	"\x04(13)\x04(14)\x04(15)\x04(16)\x04(17)\x04(18)\x04(19)\x04(20)\x0c∫∫∫∫" +
+	"\x02==\x05⫝̸\x02ɫ\x02ɽ\x02ȿ\x02ɀ\x01.\x04 ゙\x04 ゚\x06より\x06コト\x05(ᄀ)\x05" +
+	"(ᄂ)\x05(ᄃ)\x05(ᄅ)\x05(ᄆ)\x05(ᄇ)\x05(ᄉ)\x05(ᄋ)\x05(ᄌ)\x05(ᄎ)\x05(ᄏ)\x05(ᄐ" +
+	")\x05(ᄑ)\x05(ᄒ)\x05(가)\x05(나)\x05(다)\x05(라)\x05(마)\x05(바)\x05(사)\x05(아)" +
+	"\x05(자)\x05(차)\x05(카)\x05(타)\x05(파)\x05(하)\x05(주)\x08(오전)\x08(오후)\x05(一)" +
+	"\x05(二)\x05(三)\x05(四)\x05(五)\x05(六)\x05(七)\x05(八)\x05(九)\x05(十)\x05(月)" +
+	"\x05(火)\x05(水)\x05(木)\x05(金)\x05(土)\x05(日)\x05(株)\x05(有)\x05(社)\x05(名)" +
+	"\x05(特)\x05(財)\x05(祝)\x05(労)\x05(代)\x05(呼)\x05(学)\x05(監)\x05(企)\x05(資)" +
+	"\x05(協)\x05(祭)\x05(休)\x05(自)\x05(至)\x0221\x0222\x0223\x0224\x0225\x0226" +
+	"\x0227\x0228\x0229\x0230\x0231\x0232\x0233\x0234\x0235\x06참고\x06주의\x0236" +
+	"\x0237\x0238\x0239\x0240\x0241\x0242\x0243\x0244\x0245\x0246\x0247\x0248" +
+	"\x0249\x0250\x041月\x042月\x043月\x044月\x045月\x046月\x047月\x048月\x049月\x0510" +
+	"月\x0511月\x0512月\x02hg\x02ev\x06令和\x0cアパート\x0cアルファ\x0cアンペア\x09アール\x0cイニ" +
+	"ング\x09インチ\x09ウォン\x0fエスクード\x0cエーカー\x09オンス\x09オーム\x09カイリ\x0cカラット\x0cカロリー" +
+	"\x09ガロン\x09ガンマ\x06ギガ\x09ギニー\x0cキュリー\x0cギルダー\x06キロ\x0fキログラム\x12キロメートル\x0f" +
+	"キロワット\x09グラム\x0fグラムトン\x0fクルゼイロ\x0cクローネ\x09ケース\x09コルナ\x09コーポ\x0cサイクル" +
+	"\x0fサンチーム\x0cシリング\x09センチ\x09セント\x09ダース\x06デシ\x06ドル\x06トン\x06ナノ\x09ノット" +
+	"\x09ハイツ\x0fパーセント\x09パーツ\x0cバーレル\x0fピアストル\x09ピクル\x06ピコ\x06ビル\x0fファラッド\x0c" +
+	"フィート\x0fブッシェル\x09フラン\x0fヘクタール\x06ペソ\x09ペニヒ\x09ヘルツ\x09ペンス\x09ページ\x09ベータ" +
+	"\x0cポイント\x09ボルト\x06ホン\x09ポンド\x09ホール\x09ホーン\x0cマイクロ\x09マイル\x09マッハ\x09マルク" +
+	"\x0fマンション\x0cミクロン\x06ミリ\x0fミリバール\x06メガ\x0cメガトン\x0cメートル\x09ヤード\x09ヤール\x09" +
+	"ユアン\x0cリットル\x06リラ\x09ルピー\x0cルーブル\x06レム\x0fレントゲン\x09ワット\x040点\x041点\x04" +
+	"2点\x043点\x044点\x045点\x046点\x047点\x048点\x049点\x0510点\x0511点\x0512点\x0513点" +
+	"\x0514点\x0515点\x0516点\x0517点\x0518点\x0519点\x0520点\x0521点\x0522点\x0523点" +
+	"\x0524点\x02da\x02au\x02ov\x02pc\x02dm\x02iu\x06平成\x06昭和\x06大正\x06明治\x0c株" +
+	"式会社\x02pa\x02na\x02ma\x02ka\x02kb\x02mb\x02gb\x04kcal\x02pf\x02nf\x02m" +
+	"g\x02kg\x02hz\x02ml\x02dl\x02kl\x02fm\x02nm\x02mm\x02cm\x02km\x02m2\x02m" +
+	"3\x05m∕s\x06m∕s2\x07rad∕s\x08rad∕s2\x02ps\x02ns\x02ms\x02pv\x02nv\x02mv" +
+	"\x02kv\x02pw\x02nw\x02mw\x02kw\x02bq\x02cc\x02cd\x06c∕kg\x02db\x02gy\x02" +
+	"ha\x02hp\x02in\x02kk\x02kt\x02lm\x02ln\x02lx\x02ph\x02pr\x02sr\x02sv\x02" +
+	"wb\x05v∕m\x05a∕m\x041日\x042日\x043日\x044日\x045日\x046日\x047日\x048日\x049日" +
+	"\x0510日\x0511日\x0512日\x0513日\x0514日\x0515日\x0516日\x0517日\x0518日\x0519日" +
+	"\x0520日\x0521日\x0522日\x0523日\x0524日\x0525日\x0526日\x0527日\x0528日\x0529日" +
+	"\x0530日\x0531日\x02ь\x02ɦ\x02ɬ\x02ʞ\x02ʇ\x02œ\x02ʍ\x04𤋮\x04𢡊\x04𢡄\x04𣏕" +
+	"\x04𥉉\x04𥳐\x04𧻓\x02ff\x02fi\x02fl\x02st\x04մն\x04մե\x04մի\x04վն\x04մխ" +
+	"\x04יִ\x04ײַ\x02ע\x02ה\x02כ\x02ל\x02ם\x02ר\x02ת\x04שׁ\x04שׂ\x06שּׁ\x06שּ" +
+	"ׂ\x04אַ\x04אָ\x04אּ\x04בּ\x04גּ\x04דּ\x04הּ\x04וּ\x04זּ\x04טּ\x04יּ\x04" +
+	"ךּ\x04כּ\x04לּ\x04מּ\x04נּ\x04סּ\x04ףּ\x04פּ\x04צּ\x04קּ\x04רּ\x04שּ" +
+	"\x04תּ\x04וֹ\x04בֿ\x04כֿ\x04פֿ\x04אל\x02ٱ\x02ٻ\x02پ\x02ڀ\x02ٺ\x02ٿ\x02ٹ" +
+	"\x02ڤ\x02ڦ\x02ڄ\x02ڃ\x02چ\x02ڇ\x02ڍ\x02ڌ\x02ڎ\x02ڈ\x02ژ\x02ڑ\x02ک\x02گ" +
+	"\x02ڳ\x02ڱ\x02ں\x02ڻ\x02ۀ\x02ہ\x02ھ\x02ے\x02ۓ\x02ڭ\x02ۇ\x02ۆ\x02ۈ\x02ۋ" +
+	"\x02ۅ\x02ۉ\x02ې\x02ى\x04ئا\x04ئە\x04ئو\x04ئۇ\x04ئۆ\x04ئۈ\x04ئې\x04ئى\x02" +
+	"ی\x04ئج\x04ئح\x04ئم\x04ئي\x04بج\x04بح\x04بخ\x04بم\x04بى\x04بي\x04تج\x04" +
+	"تح\x04تخ\x04تم\x04تى\x04تي\x04ثج\x04ثم\x04ثى\x04ثي\x04جح\x04جم\x04حج" +
+	"\x04حم\x04خج\x04خح\x04خم\x04سج\x04سح\x04سخ\x04سم\x04صح\x04صم\x04ضج\x04ضح" +
+	"\x04ضخ\x04ضم\x04طح\x04طم\x04ظم\x04عج\x04عم\x04غج\x04غم\x04فج\x04فح\x04فخ" +
+	"\x04فم\x04فى\x04في\x04قح\x04قم\x04قى\x04قي\x04كا\x04كج\x04كح\x04كخ\x04كل" +
+	"\x04كم\x04كى\x04كي\x04لج\x04لح\x04لخ\x04لم\x04لى\x04لي\x04مج\x04مح\x04مخ" +
+	"\x04مم\x04مى\x04مي\x04نج\x04نح\x04نخ\x04نم\x04نى\x04ني\x04هج\x04هم\x04هى" +
+	"\x04هي\x04يج\x04يح\x04يخ\x04يم\x04يى\x04يي\x04ذٰ\x04رٰ\x04ىٰ\x05 ٌّ\x05 " +
+	"ٍّ\x05 َّ\x05 ُّ\x05 ِّ\x05 ّٰ\x04ئر\x04ئز\x04ئن\x04بر\x04بز\x04بن\x04ت" +
+	"ر\x04تز\x04تن\x04ثر\x04ثز\x04ثن\x04ما\x04نر\x04نز\x04نن\x04ير\x04يز\x04" +
+	"ين\x04ئخ\x04ئه\x04به\x04ته\x04صخ\x04له\x04نه\x04هٰ\x04يه\x04ثه\x04سه" +
+	"\x04شم\x04شه\x06ـَّ\x06ـُّ\x06ـِّ\x04طى\x04طي\x04عى\x04عي\x04غى\x04غي" +
+	"\x04سى\x04سي\x04شى\x04شي\x04حى\x04حي\x04جى\x04جي\x04خى\x04خي\x04صى\x04صي" +
+	"\x04ضى\x04ضي\x04شج\x04شح\x04شخ\x04شر\x04سر\x04صر\x04ضر\x04اً\x06تجم\x06ت" +
+	"حج\x06تحم\x06تخم\x06تمج\x06تمح\x06تمخ\x06جمح\x06حمي\x06حمى\x06سحج\x06سج" +
+	"ح\x06سجى\x06سمح\x06سمج\x06سمم\x06صحح\x06صمم\x06شحم\x06شجي\x06شمخ\x06شمم" +
+	"\x06ضحى\x06ضخم\x06طمح\x06طمم\x06طمي\x06عجم\x06عمم\x06عمى\x06غمم\x06غمي" +
+	"\x06غمى\x06فخم\x06قمح\x06قمم\x06لحم\x06لحي\x06لحى\x06لجج\x06لخم\x06لمح" +
+	"\x06محج\x06محم\x06محي\x06مجح\x06مجم\x06مخج\x06مخم\x06مجخ\x06همج\x06همم" +
+	"\x06نحم\x06نحى\x06نجم\x06نجى\x06نمي\x06نمى\x06يمم\x06بخي\x06تجي\x06تجى" +
+	"\x06تخي\x06تخى\x06تمي\x06تمى\x06جمي\x06جحى\x06جمى\x06سخى\x06صحي\x06شحي" +
+	"\x06ضحي\x06لجي\x06لمي\x06يحي\x06يجي\x06يمي\x06ممي\x06قمي\x06نحي\x06عمي" +
+	"\x06كمي\x06نجح\x06مخي\x06لجم\x06كمم\x06جحي\x06حجي\x06مجي\x06فمي\x06بحي" +
+	"\x06سخي\x06نجي\x06صلے\x06قلے\x08الله\x08اكبر\x08محمد\x08صلعم\x08رسول\x08" +
+	"عليه\x08وسلم\x06صلى!صلى الله عليه وسلم\x0fجل جلاله\x08ریال\x01,\x01:" +
+	"\x01!\x01?\x01_\x01{\x01}\x01[\x01]\x01#\x01&\x01*\x01-\x01<\x01>\x01\\" +
+	"\x01$\x01%\x01@\x04ـً\x04ـَ\x04ـُ\x04ـِ\x04ـّ\x04ـْ\x02ء\x02آ\x02أ\x02ؤ" +
+	"\x02إ\x02ئ\x02ا\x02ب\x02ة\x02ت\x02ث\x02ج\x02ح\x02خ\x02د\x02ذ\x02ر\x02ز" +
+	"\x02س\x02ش\x02ص\x02ض\x02ط\x02ظ\x02ع\x02غ\x02ف\x02ق\x02ك\x02ل\x02م\x02ن" +
+	"\x02ه\x02و\x02ي\x04لآ\x04لأ\x04لإ\x04لا\x01\x22\x01'\x01/\x01^\x01|\x01~" +
+	"\x02¢\x02£\x02¬\x02¦\x02¥\x08𝅗𝅥\x08𝅘𝅥\x0c𝅘𝅥𝅮\x0c𝅘𝅥𝅯\x0c𝅘𝅥𝅰\x0c𝅘𝅥𝅱\x0c𝅘𝅥𝅲" +
+	"\x08𝆹𝅥\x08𝆺𝅥\x0c𝆹𝅥𝅮\x0c𝆺𝅥𝅮\x0c𝆹𝅥𝅯\x0c𝆺𝅥𝅯\x02ı\x02ȷ\x02α\x02ε\x02ζ\x02η" +
+	"\x02κ\x02λ\x02μ\x02ν\x02ξ\x02ο\x02σ\x02τ\x02υ\x02ψ\x03∇\x03∂\x02ϝ\x02ٮ" +
+	"\x02ڡ\x02ٯ\x020,\x021,\x022,\x023,\x024,\x025,\x026,\x027,\x028,\x029," +
+	"\x03(a)\x03(b)\x03(c)\x03(d)\x03(e)\x03(f)\x03(g)\x03(h)\x03(i)\x03(j)" +
+	"\x03(k)\x03(l)\x03(m)\x03(n)\x03(o)\x03(p)\x03(q)\x03(r)\x03(s)\x03(t)" +
+	"\x03(u)\x03(v)\x03(w)\x03(x)\x03(y)\x03(z)\x07〔s〕\x02wz\x02hv\x02sd\x03p" +
+	"pv\x02wc\x02mc\x02md\x02mr\x02dj\x06ほか\x06ココ\x03サ\x03手\x03字\x03双\x03デ" +
+	"\x03二\x03多\x03解\x03天\x03交\x03映\x03無\x03料\x03前\x03後\x03再\x03新\x03初\x03終" +
+	"\x03生\x03販\x03声\x03吹\x03演\x03投\x03捕\x03一\x03三\x03遊\x03左\x03中\x03右\x03指" +
+	"\x03走\x03打\x03禁\x03空\x03合\x03満\x03有\x03月\x03申\x03割\x03営\x03配\x09〔本〕\x09〔" +
+	"三〕\x09〔二〕\x09〔安〕\x09〔点〕\x09〔打〕\x09〔盗〕\x09〔勝〕\x09〔敗〕\x03得\x03可\x03丽\x03" +
+	"丸\x03乁\x03你\x03侮\x03侻\x03倂\x03偺\x03備\x03僧\x03像\x03㒞\x03免\x03兔\x03兤\x03" +
+	"具\x03㒹\x03內\x03冗\x03冤\x03仌\x03冬\x03况\x03凵\x03刃\x03㓟\x03刻\x03剆\x03剷\x03" +
+	"㔕\x03勇\x03勉\x03勤\x03勺\x03包\x03匆\x03北\x03卉\x03卑\x03博\x03即\x03卽\x03卿\x03" +
+	"灰\x03及\x03叟\x03叫\x03叱\x03吆\x03咞\x03吸\x03呈\x03周\x03咢\x03哶\x03唐\x03啓\x03" +
+	"啣\x03善\x03喙\x03喫\x03喳\x03嗂\x03圖\x03嘆\x03圗\x03噑\x03噴\x03切\x03壮\x03城\x03" +
+	"埴\x03堍\x03型\x03堲\x03報\x03墬\x03売\x03壷\x03夆\x03夢\x03奢\x03姬\x03娛\x03娧\x03" +
+	"姘\x03婦\x03㛮\x03嬈\x03嬾\x03寃\x03寘\x03寧\x03寳\x03寿\x03将\x03尢\x03㞁\x03屠\x03" +
+	"屮\x03峀\x03岍\x03嵃\x03嵮\x03嵫\x03嵼\x03巡\x03巢\x03㠯\x03巽\x03帨\x03帽\x03幩\x03" +
+	"㡢\x03㡼\x03庰\x03庳\x03庶\x03廊\x03廾\x03舁\x03弢\x03㣇\x03形\x03彫\x03㣣\x03徚\x03" +
+	"忍\x03志\x03忹\x03悁\x03㤺\x03㤜\x03悔\x03惇\x03慈\x03慌\x03慎\x03慺\x03憎\x03憲\x03" +
+	"憤\x03憯\x03懞\x03懲\x03懶\x03成\x03戛\x03扝\x03抱\x03拔\x03捐\x03挽\x03拼\x03捨\x03" +
+	"掃\x03揤\x03搢\x03揅\x03掩\x03㨮\x03摩\x03摾\x03撝\x03摷\x03㩬\x03敏\x03敬\x03旣\x03" +
+	"書\x03晉\x03㬙\x03暑\x03㬈\x03㫤\x03冒\x03冕\x03最\x03暜\x03肭\x03䏙\x03朗\x03望\x03" +
+	"朡\x03杞\x03杓\x03㭉\x03柺\x03枅\x03桒\x03梅\x03梎\x03栟\x03椔\x03㮝\x03楂\x03榣\x03" +
+	"槪\x03檨\x03櫛\x03㰘\x03次\x03歔\x03㱎\x03歲\x03殟\x03殺\x03殻\x03汎\x03沿\x03泍\x03" +
+	"汧\x03洖\x03派\x03海\x03流\x03浩\x03浸\x03涅\x03洴\x03港\x03湮\x03㴳\x03滋\x03滇\x03" +
+	"淹\x03潮\x03濆\x03瀹\x03瀞\x03瀛\x03㶖\x03灊\x03災\x03灷\x03炭\x03煅\x03熜\x03爨\x03" +
+	"爵\x03牐\x03犀\x03犕\x03獺\x03王\x03㺬\x03玥\x03㺸\x03瑇\x03瑜\x03瑱\x03璅\x03瓊\x03" +
+	"㼛\x03甤\x03甾\x03異\x03瘐\x03㿼\x03䀈\x03直\x03眞\x03真\x03睊\x03䀹\x03瞋\x03䁆\x03" +
+	"䂖\x03硎\x03碌\x03磌\x03䃣\x03祖\x03福\x03秫\x03䄯\x03穀\x03穊\x03穏\x03䈂\x03篆\x03" +
+	"築\x03䈧\x03糒\x03䊠\x03糨\x03糣\x03紀\x03絣\x03䌁\x03緇\x03縂\x03繅\x03䌴\x03䍙\x03" +
+	"罺\x03羕\x03翺\x03者\x03聠\x03聰\x03䏕\x03育\x03脃\x03䐋\x03脾\x03媵\x03舄\x03辞\x03" +
+	"䑫\x03芑\x03芋\x03芝\x03劳\x03花\x03芳\x03芽\x03苦\x03若\x03茝\x03荣\x03莭\x03茣\x03" +
+	"莽\x03菧\x03著\x03荓\x03菊\x03菌\x03菜\x03䔫\x03蓱\x03蓳\x03蔖\x03蕤\x03䕝\x03䕡\x03" +
+	"䕫\x03虐\x03虜\x03虧\x03虩\x03蚩\x03蚈\x03蜎\x03蛢\x03蝹\x03蜨\x03蝫\x03螆\x03蟡\x03" +
+	"蠁\x03䗹\x03衠\x03衣\x03裗\x03裞\x03䘵\x03裺\x03㒻\x03䚾\x03䛇\x03誠\x03諭\x03變\x03" +
+	"豕\x03貫\x03賁\x03贛\x03起\x03跋\x03趼\x03跰\x03軔\x03輸\x03邔\x03郱\x03鄑\x03鄛\x03" +
+	"鈸\x03鋗\x03鋘\x03鉼\x03鏹\x03鐕\x03開\x03䦕\x03閷\x03䧦\x03雃\x03嶲\x03霣\x03䩮\x03" +
+	"䩶\x03韠\x03䪲\x03頋\x03頩\x03飢\x03䬳\x03餩\x03馧\x03駂\x03駾\x03䯎\x03鬒\x03鱀\x03" +
+	"鳽\x03䳎\x03䳭\x03鵧\x03䳸\x03麻\x03䵖\x03黹\x03黾\x03鼅\x03鼏\x03鼖\x03鼻"
+
+var xorData string = "" + // Size: 4862 bytes
+	"\x02\x0c\x09\x02\xb0\xec\x02\xad\xd8\x02\xad\xd9\x02\x06\x07\x02\x0f\x12" +
+	"\x02\x0f\x1f\x02\x0f\x1d\x02\x01\x13\x02\x0f\x16\x02\x0f\x0b\x02\x0f3" +
+	"\x02\x0f7\x02\x0f?\x02\x0f/\x02\x0f*\x02\x0c&\x02\x0c*\x02\x0c;\x02\x0c9" +
+	"\x02\x0c%\x02\xab\xed\x02\xab\xe2\x02\xab\xe3\x02\xa9\xe0\x02\xa9\xe1" +
+	"\x02\xa9\xe6\x02\xa3\xcb\x02\xa3\xc8\x02\xa3\xc9\x02\x01#\x02\x01\x08" +
+	"\x02\x0e>\x02\x0e'\x02\x0f\x03\x02\x03\x0d\x02\x03\x09\x02\x03\x17\x02" +
+	"\x03\x0e\x02\x02\x03\x02\x011\x02\x01\x00\x02\x01\x10\x02\x03<\x02\x07" +
+	"\x0d\x02\x02\x0c\x02\x0c0\x02\x01\x03\x02\x01\x01\x02\x01 \x02\x01\x22" +
+	"\x02\x01)\x02\x01\x0a\x02\x01\x0c\x02\x02\x06\x02\x02\x02\x02\x03\x10" +
+	"\x03\x037 \x03\x0b+\x03\x021\x00\x02\x01\x04\x02\x01\x02\x02\x019\x02" +
+	"\x03\x1c\x02\x02$\x03\x80p$\x02\x03:\x02\x03\x0a\x03\xc1r.\x03\xc1r,\x03" +
+	"\xc1r\x02\x02\x02:\x02\x02>\x02\x02,\x02\x02\x10\x02\x02\x00\x03\xc1s<" +
+	"\x03\xc1s*\x03\xc2L$\x03\xc2L;\x02\x09)\x02\x0a\x19\x03\x83\xab\xe3\x03" +
+	"\x83\xab\xf2\x03 4\xe0\x03\x81\xab\xea\x03\x81\xab\xf3\x03 4\xef\x03\x96" +
+	"\xe1\xcd\x03\x84\xe5\xc3\x02\x0d\x11\x03\x8b\xec\xcb\x03\x94\xec\xcf\x03" +
+	"\x9a\xec\xc2\x03\x8b\xec\xdb\x03\x94\xec\xdf\x03\x9a\xec\xd2\x03\x01\x0c" +
+	"!\x03\x01\x0c#\x03ʠ\x9d\x03ʣ\x9c\x03ʢ\x9f\x03ʥ\x9e\x03ʤ\x91\x03ʧ\x90\x03" +
+	"ʦ\x93\x03ʩ\x92\x03ʨ\x95\x03\xca\xf3\xb5\x03\xca\xf0\xb4\x03\xca\xf1\xb7" +
+	"\x03\xca\xf6\xb6\x03\xca\xf7\x89\x03\xca\xf4\x88\x03\xca\xf5\x8b\x03\xca" +
+	"\xfa\x8a\x03\xca\xfb\x8d\x03\xca\xf8\x8c\x03\xca\xf9\x8f\x03\xca\xfe\x8e" +
+	"\x03\xca\xff\x81\x03\xca\xfc\x80\x03\xca\xfd\x83\x03\xca\xe2\x82\x03\xca" +
+	"\xe3\x85\x03\xca\xe0\x84\x03\xca\xe1\x87\x03\xca\xe6\x86\x03\xca\xe7\x99" +
+	"\x03\xca\xe4\x98\x03\xca\xe5\x9b\x03\xca\xea\x9a\x03\xca\xeb\x9d\x03\xca" +
+	"\xe8\x9c\x03ؓ\x89\x03ߔ\x8b\x02\x010\x03\x03\x04\x1e\x03\x04\x15\x12\x03" +
+	"\x0b\x05,\x03\x06\x04\x00\x03\x06\x04)\x03\x06\x044\x03\x06\x04<\x03\x06" +
+	"\x05\x1d\x03\x06\x06\x00\x03\x06\x06\x0a\x03\x06\x06'\x03\x06\x062\x03" +
+	"\x0786\x03\x079/\x03\x079 \x03\x07:\x0e\x03\x07:\x1b\x03\x07:%\x03\x07;/" +
+	"\x03\x07;%\x03\x074\x11\x03\x076\x09\x03\x077*\x03\x070\x01\x03\x070\x0f" +
+	"\x03\x070.\x03\x071\x16\x03\x071\x04\x03\x0710\x03\x072\x18\x03\x072-" +
+	"\x03\x073\x14\x03\x073>\x03\x07'\x09\x03\x07 \x00\x03\x07\x1f\x0b\x03" +
+	"\x07\x18#\x03\x07\x18(\x03\x07\x186\x03\x07\x18\x03\x03\x07\x19\x16\x03" +
+	"\x07\x116\x03\x07\x12'\x03\x07\x13\x10\x03\x07\x0c&\x03\x07\x0c\x08\x03" +
+	"\x07\x0c\x13\x03\x07\x0d\x02\x03\x07\x0d\x1c\x03\x07\x0b5\x03\x07\x0b" +
+	"\x0a\x03\x07\x0b\x01\x03\x07\x0b\x0f\x03\x07\x05\x00\x03\x07\x05\x09\x03" +
+	"\x07\x05\x0b\x03\x07\x07\x01\x03\x07\x07\x08\x03\x07\x00<\x03\x07\x00+" +
+	"\x03\x07\x01)\x03\x07\x01\x1b\x03\x07\x01\x08\x03\x07\x03?\x03\x0445\x03" +
+	"\x044\x08\x03\x0454\x03\x04)/\x03\x04)5\x03\x04+\x05\x03\x04+\x14\x03" +
+	"\x04+ \x03\x04+<\x03\x04*&\x03\x04*\x22\x03\x04&8\x03\x04!\x01\x03\x04!" +
+	"\x22\x03\x04\x11+\x03\x04\x10.\x03\x04\x104\x03\x04\x13=\x03\x04\x12\x04" +
+	"\x03\x04\x12\x0a\x03\x04\x0d\x1d\x03\x04\x0d\x07\x03\x04\x0d \x03\x05<>" +
+	"\x03\x055<\x03\x055!\x03\x055#\x03\x055&\x03\x054\x1d\x03\x054\x02\x03" +
+	"\x054\x07\x03\x0571\x03\x053\x1a\x03\x053\x16\x03\x05.<\x03\x05.\x07\x03" +
+	"\x05):\x03\x05)<\x03\x05)\x0c\x03\x05)\x15\x03\x05+-\x03\x05+5\x03\x05$" +
+	"\x1e\x03\x05$\x14\x03\x05'\x04\x03\x05'\x14\x03\x05&\x02\x03\x05\x226" +
+	"\x03\x05\x22\x0c\x03\x05\x22\x1c\x03\x05\x19\x0a\x03\x05\x1b\x09\x03\x05" +
+	"\x1b\x0c\x03\x05\x14\x07\x03\x05\x16?\x03\x05\x16\x0c\x03\x05\x0c\x05" +
+	"\x03\x05\x0e\x0f\x03\x05\x01\x0e\x03\x05\x00(\x03\x05\x030\x03\x05\x03" +
+	"\x06\x03\x0a==\x03\x0a=1\x03\x0a=,\x03\x0a=\x0c\x03\x0a??\x03\x0a<\x08" +
+	"\x03\x0a9!\x03\x0a9)\x03\x0a97\x03\x0a99\x03\x0a6\x0a\x03\x0a6\x1c\x03" +
+	"\x0a6\x17\x03\x0a7'\x03\x0a78\x03\x0a73\x03\x0a'\x01\x03\x0a'&\x03\x0a" +
+	"\x1f\x0e\x03\x0a\x1f\x03\x03\x0a\x1f3\x03\x0a\x1b/\x03\x0a\x18\x19\x03" +
+	"\x0a\x19\x01\x03\x0a\x16\x14\x03\x0a\x0e\x22\x03\x0a\x0f\x10\x03\x0a\x0f" +
+	"\x02\x03\x0a\x0f \x03\x0a\x0c\x04\x03\x0a\x0b>\x03\x0a\x0b+\x03\x0a\x08/" +
+	"\x03\x0a\x046\x03\x0a\x05\x14\x03\x0a\x00\x04\x03\x0a\x00\x10\x03\x0a" +
+	"\x00\x14\x03\x0b<3\x03\x0b;*\x03\x0b9\x22\x03\x0b9)\x03\x0b97\x03\x0b+" +
+	"\x10\x03\x0b((\x03\x0b&5\x03\x0b$\x1c\x03\x0b$\x12\x03\x0b%\x04\x03\x0b#" +
+	"<\x03\x0b#0\x03\x0b#\x0d\x03\x0b#\x19\x03\x0b!:\x03\x0b!\x1f\x03\x0b!" +
+	"\x00\x03\x0b\x1e5\x03\x0b\x1c\x1d\x03\x0b\x1d-\x03\x0b\x1d(\x03\x0b\x18." +
+	"\x03\x0b\x18 \x03\x0b\x18\x16\x03\x0b\x14\x13\x03\x0b\x15$\x03\x0b\x15" +
+	"\x22\x03\x0b\x12\x1b\x03\x0b\x12\x10\x03\x0b\x132\x03\x0b\x13=\x03\x0b" +
+	"\x12\x18\x03\x0b\x0c&\x03\x0b\x061\x03\x0b\x06:\x03\x0b\x05#\x03\x0b\x05" +
+	"<\x03\x0b\x04\x0b\x03\x0b\x04\x04\x03\x0b\x04\x1b\x03\x0b\x042\x03\x0b" +
+	"\x041\x03\x0b\x03\x03\x03\x0b\x03\x1d\x03\x0b\x03/\x03\x0b\x03+\x03\x0b" +
+	"\x02\x1b\x03\x0b\x02\x00\x03\x0b\x01\x1e\x03\x0b\x01\x08\x03\x0b\x015" +
+	"\x03\x06\x0d9\x03\x06\x0d=\x03\x06\x0d?\x03\x02\x001\x03\x02\x003\x03" +
+	"\x02\x02\x19\x03\x02\x006\x03\x02\x02\x1b\x03\x02\x004\x03\x02\x00<\x03" +
+	"\x02\x02\x0a\x03\x02\x02\x0e\x03\x02\x01\x1a\x03\x02\x01\x07\x03\x02\x01" +
+	"\x05\x03\x02\x01\x0b\x03\x02\x01%\x03\x02\x01\x0c\x03\x02\x01\x04\x03" +
+	"\x02\x01\x1c\x03\x02\x00.\x03\x02\x002\x03\x02\x00>\x03\x02\x00\x12\x03" +
+	"\x02\x00\x16\x03\x02\x011\x03\x02\x013\x03\x02\x02 \x03\x02\x02%\x03\x02" +
+	"\x02$\x03\x02\x028\x03\x02\x02;\x03\x02\x024\x03\x02\x012\x03\x02\x022" +
+	"\x03\x02\x02/\x03\x02\x01,\x03\x02\x01\x13\x03\x02\x01\x16\x03\x02\x01" +
+	"\x11\x03\x02\x01\x1e\x03\x02\x01\x15\x03\x02\x01\x17\x03\x02\x01\x0f\x03" +
+	"\x02\x01\x08\x03\x02\x00?\x03\x02\x03\x07\x03\x02\x03\x0d\x03\x02\x03" +
+	"\x13\x03\x02\x03\x1d\x03\x02\x03\x1f\x03\x02\x00\x03\x03\x02\x00\x0d\x03" +
+	"\x02\x00\x01\x03\x02\x00\x1b\x03\x02\x00\x19\x03\x02\x00\x18\x03\x02\x00" +
+	"\x13\x03\x02\x00/\x03\x07>\x12\x03\x07<\x1f\x03\x07>\x1d\x03\x06\x1d\x0e" +
+	"\x03\x07>\x1c\x03\x07>:\x03\x07>\x13\x03\x04\x12+\x03\x07?\x03\x03\x07>" +
+	"\x02\x03\x06\x224\x03\x06\x1a.\x03\x07<%\x03\x06\x1c\x0b\x03\x0609\x03" +
+	"\x05\x1f\x01\x03\x04'\x08\x03\x93\xfd\xf5\x03\x02\x0d \x03\x02\x0d#\x03" +
+	"\x02\x0d!\x03\x02\x0d&\x03\x02\x0d\x22\x03\x02\x0d/\x03\x02\x0d,\x03\x02" +
+	"\x0d$\x03\x02\x0d'\x03\x02\x0d%\x03\x02\x0d;\x03\x02\x0d=\x03\x02\x0d?" +
+	"\x03\x099.\x03\x08\x0b7\x03\x08\x02\x14\x03\x08\x14\x0d\x03\x08.:\x03" +
+	"\x089'\x03\x0f\x0b\x18\x03\x0f\x1c1\x03\x0f\x17&\x03\x0f9\x1f\x03\x0f0" +
+	"\x0c\x03\x0e\x0a9\x03\x0e\x056\x03\x0e\x1c#\x03\x0f\x13\x0e\x03\x072\x00" +
+	"\x03\x070\x0d\x03\x072\x0b\x03\x06\x11\x18\x03\x070\x10\x03\x06\x0f(\x03" +
+	"\x072\x05\x03\x06\x0f,\x03\x073\x15\x03\x06\x07\x08\x03\x05\x16\x02\x03" +
+	"\x04\x0b \x03\x05:8\x03\x05\x16%\x03\x0a\x0d\x1f\x03\x06\x16\x10\x03\x05" +
+	"\x1d5\x03\x05*;\x03\x05\x16\x1b\x03\x04.-\x03\x06\x1a\x19\x03\x04\x03," +
+	"\x03\x0b87\x03\x04/\x0a\x03\x06\x00,\x03\x04-\x01\x03\x04\x1e-\x03\x06/(" +
+	"\x03\x0a\x0b5\x03\x06\x0e7\x03\x06\x07.\x03\x0597\x03\x0a*%\x03\x0760" +
+	"\x03\x06\x0c;\x03\x05'\x00\x03\x072.\x03\x072\x08\x03\x06=\x01\x03\x06" +
+	"\x05\x1b\x03\x06\x06\x12\x03\x06$=\x03\x06'\x0d\x03\x04\x11\x0f\x03\x076" +
+	",\x03\x06\x07;\x03\x06.,\x03\x86\xf9\xea\x03\x8f\xff\xeb\x02\x092\x02" +
+	"\x095\x02\x094\x02\x09;\x02\x09>\x02\x098\x02\x09*\x02\x09/\x02\x09,\x02" +
+	"\x09%\x02\x09&\x02\x09#\x02\x09 \x02\x08!\x02\x08%\x02\x08$\x02\x08+\x02" +
+	"\x08.\x02\x08*\x02\x08&\x02\x088\x02\x08>\x02\x084\x02\x086\x02\x080\x02" +
+	"\x08\x10\x02\x08\x17\x02\x08\x12\x02\x08\x1d\x02\x08\x1f\x02\x08\x13\x02" +
+	"\x08\x15\x02\x08\x14\x02\x08\x0c\x03\x8b\xfd\xd0\x03\x81\xec\xc6\x03\x87" +
+	"\xe0\x8a\x03-2\xe3\x03\x80\xef\xe4\x03-2\xea\x03\x88\xe6\xeb\x03\x8e\xe6" +
+	"\xe8\x03\x84\xe6\xe9\x03\x97\xe6\xee\x03-2\xf9\x03-2\xf6\x03\x8e\xe3\xad" +
+	"\x03\x80\xe3\x92\x03\x88\xe3\x90\x03\x8e\xe3\x90\x03\x80\xe3\x97\x03\x88" +
+	"\xe3\x95\x03\x88\xfe\xcb\x03\x8e\xfe\xca\x03\x84\xfe\xcd\x03\x91\xef\xc9" +
+	"\x03-2\xc1\x03-2\xc0\x03-2\xcb\x03\x88@\x09\x03\x8e@\x08\x03\x8f\xe0\xf5" +
+	"\x03\x8e\xe6\xf9\x03\x8e\xe0\xfa\x03\x93\xff\xf4\x03\x84\xee\xd3\x03\x0b" +
+	"(\x04\x023 \x03\x0b)\x08\x021;\x02\x01*\x03\x0b#\x10\x03\x0b 0\x03\x0b!" +
+	"\x10\x03\x0b!0\x03\x07\x15\x08\x03\x09?5\x03\x07\x1f\x08\x03\x07\x17\x0b" +
+	"\x03\x09\x1f\x15\x03\x0b\x1c7\x03\x0a+#\x03\x06\x1a\x1b\x03\x06\x1a\x14" +
+	"\x03\x0a\x01\x18\x03\x06#\x1b\x03\x0a2\x0c\x03\x0a\x01\x04\x03\x09#;\x03" +
+	"\x08='\x03\x08\x1a\x0a\x03\x07</\x03\x07:+\x03\x07\x07*\x03\x06&\x1c\x03" +
+	"\x09\x0c\x16\x03\x09\x10\x0e\x03\x08'\x0f\x03\x08+\x09\x03\x074%\x03\x06" +
+	"!3\x03\x06\x03+\x03\x0b\x1e\x19\x03\x0a))\x03\x09\x08\x19\x03\x08,\x05" +
+	"\x03\x07<2\x03\x06\x1c>\x03\x0a\x111\x03\x09\x1b\x09\x03\x073.\x03\x07" +
+	"\x01\x00\x03\x09/,\x03\x07#>\x03\x07\x048\x03\x0a\x1f\x22\x03\x098>\x03" +
+	"\x09\x11\x00\x03\x08/\x17\x03\x06'\x22\x03\x0b\x1a+\x03\x0a\x22\x19\x03" +
+	"\x0a/1\x03\x0974\x03\x09\x0f\x22\x03\x08,\x22\x03\x08?\x14\x03\x07$5\x03" +
+	"\x07<3\x03\x07=*\x03\x07\x13\x18\x03\x068\x0a\x03\x06\x09\x16\x03\x06" +
+	"\x13\x00\x03\x08\x067\x03\x08\x01\x03\x03\x08\x12\x1d\x03\x07+7\x03\x06(" +
+	";\x03\x06\x1c?\x03\x07\x0e\x17\x03\x0a\x06\x1d\x03\x0a\x19\x07\x03\x08" +
+	"\x14$\x03\x07$;\x03\x08,$\x03\x08\x06\x0d\x03\x07\x16\x0a\x03\x06>>\x03" +
+	"\x0a\x06\x12\x03\x0a\x14)\x03\x09\x0d\x1f\x03\x09\x12\x17\x03\x09\x19" +
+	"\x01\x03\x08\x11 \x03\x08\x1d'\x03\x06<\x1a\x03\x0a.\x00\x03\x07'\x18" +
+	"\x03\x0a\x22\x08\x03\x08\x0d\x0a\x03\x08\x13)\x03\x07*)\x03\x06<,\x03" +
+	"\x07\x0b\x1a\x03\x09.\x14\x03\x09\x0d\x1e\x03\x07\x0e#\x03\x0b\x1d'\x03" +
+	"\x0a\x0a8\x03\x09%2\x03\x08+&\x03\x080\x12\x03\x0a)4\x03\x08\x06\x1f\x03" +
+	"\x0b\x1b\x1a\x03\x0a\x1b\x0f\x03\x0b\x1d*\x03\x09\x16$\x03\x090\x11\x03" +
+	"\x08\x11\x08\x03\x0a*(\x03\x0a\x042\x03\x089,\x03\x074'\x03\x07\x0f\x05" +
+	"\x03\x09\x0b\x0a\x03\x07\x1b\x01\x03\x09\x17:\x03\x09.\x0d\x03\x07.\x11" +
+	"\x03\x09+\x15\x03\x080\x13\x03\x0b\x1f\x19\x03\x0a \x11\x03\x0a\x220\x03" +
+	"\x09\x07;\x03\x08\x16\x1c\x03\x07,\x13\x03\x07\x0e/\x03\x06\x221\x03\x0a" +
+	".\x0a\x03\x0a7\x02\x03\x0a\x032\x03\x0a\x1d.\x03\x091\x06\x03\x09\x19:" +
+	"\x03\x08\x02/\x03\x060+\x03\x06\x0f-\x03\x06\x1c\x1f\x03\x06\x1d\x07\x03" +
+	"\x0a,\x11\x03\x09=\x0d\x03\x09\x0b;\x03\x07\x1b/\x03\x0a\x1f:\x03\x09 " +
+	"\x1f\x03\x09.\x10\x03\x094\x0b\x03\x09\x1a1\x03\x08#\x1a\x03\x084\x1d" +
+	"\x03\x08\x01\x1f\x03\x08\x11\x22\x03\x07'8\x03\x07\x1a>\x03\x0757\x03" +
+	"\x06&9\x03\x06+\x11\x03\x0a.\x0b\x03\x0a,>\x03\x0a4#\x03\x08%\x17\x03" +
+	"\x07\x05\x22\x03\x07\x0c\x0b\x03\x0a\x1d+\x03\x0a\x19\x16\x03\x09+\x1f" +
+	"\x03\x09\x08\x0b\x03\x08\x16\x18\x03\x08+\x12\x03\x0b\x1d\x0c\x03\x0a=" +
+	"\x10\x03\x0a\x09\x0d\x03\x0a\x10\x11\x03\x09&0\x03\x08(\x1f\x03\x087\x07" +
+	"\x03\x08\x185\x03\x07'6\x03\x06.\x05\x03\x06=\x04\x03\x06;;\x03\x06\x06," +
+	"\x03\x0b\x18>\x03\x08\x00\x18\x03\x06 \x03\x03\x06<\x00\x03\x09%\x18\x03" +
+	"\x0b\x1c<\x03\x0a%!\x03\x0a\x09\x12\x03\x0a\x16\x02\x03\x090'\x03\x09" +
+	"\x0e=\x03\x08 \x0e\x03\x08>\x03\x03\x074>\x03\x06&?\x03\x06\x19\x09\x03" +
+	"\x06?(\x03\x0a-\x0e\x03\x09:3\x03\x098:\x03\x09\x12\x0b\x03\x09\x1d\x17" +
+	"\x03\x087\x05\x03\x082\x14\x03\x08\x06%\x03\x08\x13\x1f\x03\x06\x06\x0e" +
+	"\x03\x0a\x22<\x03\x09/<\x03\x06>+\x03\x0a'?\x03\x0a\x13\x0c\x03\x09\x10<" +
+	"\x03\x07\x1b=\x03\x0a\x19\x13\x03\x09\x22\x1d\x03\x09\x07\x0d\x03\x08)" +
+	"\x1c\x03\x06=\x1a\x03\x0a/4\x03\x0a7\x11\x03\x0a\x16:\x03\x09?3\x03\x09:" +
+	"/\x03\x09\x05\x0a\x03\x09\x14\x06\x03\x087\x22\x03\x080\x07\x03\x08\x1a" +
+	"\x1f\x03\x07\x04(\x03\x07\x04\x09\x03\x06 %\x03\x06<\x08\x03\x0a+\x14" +
+	"\x03\x09\x1d\x16\x03\x0a70\x03\x08 >\x03\x0857\x03\x070\x0a\x03\x06=\x12" +
+	"\x03\x06\x16%\x03\x06\x1d,\x03\x099#\x03\x09\x10>\x03\x07 \x1e\x03\x08" +
+	"\x0c<\x03\x08\x0b\x18\x03\x08\x15+\x03\x08,:\x03\x08%\x22\x03\x07\x0a$" +
+	"\x03\x0b\x1c=\x03\x07+\x08\x03\x0a/\x05\x03\x0a \x07\x03\x0a\x12'\x03" +
+	"\x09#\x11\x03\x08\x1b\x15\x03\x0a\x06\x01\x03\x09\x1c\x1b\x03\x0922\x03" +
+	"\x07\x14<\x03\x07\x09\x04\x03\x061\x04\x03\x07\x0e\x01\x03\x0a\x13\x18" +
+	"\x03\x0a-\x0c\x03\x0a?\x0d\x03\x0a\x09\x0a\x03\x091&\x03\x0a/\x0b\x03" +
+	"\x08$<\x03\x083\x1d\x03\x08\x0c$\x03\x08\x0d\x07\x03\x08\x0d?\x03\x08" +
+	"\x0e\x14\x03\x065\x0a\x03\x08\x1a#\x03\x08\x16#\x03\x0702\x03\x07\x03" +
+	"\x1a\x03\x06(\x1d\x03\x06+\x1b\x03\x06\x0b\x05\x03\x06\x0b\x17\x03\x06" +
+	"\x0c\x04\x03\x06\x1e\x19\x03\x06+0\x03\x062\x18\x03\x0b\x16\x1e\x03\x0a+" +
+	"\x16\x03\x0a-?\x03\x0a#:\x03\x0a#\x10\x03\x0a%$\x03\x0a>+\x03\x0a01\x03" +
+	"\x0a1\x10\x03\x0a\x099\x03\x0a\x0a\x12\x03\x0a\x19\x1f\x03\x0a\x19\x12" +
+	"\x03\x09*)\x03\x09-\x16\x03\x09.1\x03\x09.2\x03\x09<\x0e\x03\x09> \x03" +
+	"\x093\x12\x03\x09\x0b\x01\x03\x09\x1c2\x03\x09\x11\x1c\x03\x09\x15%\x03" +
+	"\x08,&\x03\x08!\x22\x03\x089(\x03\x08\x0b\x1a\x03\x08\x0d2\x03\x08\x0c" +
+	"\x04\x03\x08\x0c\x06\x03\x08\x0c\x1f\x03\x08\x0c\x0c\x03\x08\x0f\x1f\x03" +
+	"\x08\x0f\x1d\x03\x08\x00\x14\x03\x08\x03\x14\x03\x08\x06\x16\x03\x08\x1e" +
+	"#\x03\x08\x11\x11\x03\x08\x10\x18\x03\x08\x14(\x03\x07)\x1e\x03\x07.1" +
+	"\x03\x07 $\x03\x07 '\x03\x078\x08\x03\x07\x0d0\x03\x07\x0f7\x03\x07\x05#" +
+	"\x03\x07\x05\x1a\x03\x07\x1a7\x03\x07\x1d-\x03\x07\x17\x10\x03\x06)\x1f" +
+	"\x03\x062\x0b\x03\x066\x16\x03\x06\x09\x11\x03\x09(\x1e\x03\x07!5\x03" +
+	"\x0b\x11\x16\x03\x0a/\x04\x03\x0a,\x1a\x03\x0b\x173\x03\x0a,1\x03\x0a/5" +
+	"\x03\x0a\x221\x03\x0a\x22\x0d\x03\x0a?%\x03\x0a<,\x03\x0a?#\x03\x0a>\x19" +
+	"\x03\x0a\x08&\x03\x0a\x0b\x0e\x03\x0a\x0c:\x03\x0a\x0c+\x03\x0a\x03\x22" +
+	"\x03\x0a\x06)\x03\x0a\x11\x10\x03\x0a\x11\x1a\x03\x0a\x17-\x03\x0a\x14(" +
+	"\x03\x09)\x1e\x03\x09/\x09\x03\x09.\x00\x03\x09,\x07\x03\x09/*\x03\x09-9" +
+	"\x03\x09\x228\x03\x09%\x09\x03\x09:\x12\x03\x09;\x1d\x03\x09?\x06\x03" +
+	"\x093%\x03\x096\x05\x03\x096\x08\x03\x097\x02\x03\x09\x07,\x03\x09\x04," +
+	"\x03\x09\x1f\x16\x03\x09\x11\x03\x03\x09\x11\x12\x03\x09\x168\x03\x08*" +
+	"\x05\x03\x08/2\x03\x084:\x03\x08\x22+\x03\x08 0\x03\x08&\x0a\x03\x08;" +
+	"\x10\x03\x08>$\x03\x08>\x18\x03\x0829\x03\x082:\x03\x081,\x03\x081<\x03" +
+	"\x081\x1c\x03\x087#\x03\x087*\x03\x08\x09'\x03\x08\x00\x1d\x03\x08\x05-" +
+	"\x03\x08\x1f4\x03\x08\x1d\x04\x03\x08\x16\x0f\x03\x07*7\x03\x07'!\x03" +
+	"\x07%\x1b\x03\x077\x0c\x03\x07\x0c1\x03\x07\x0c.\x03\x07\x00\x06\x03\x07" +
+	"\x01\x02\x03\x07\x010\x03\x07\x06=\x03\x07\x01\x03\x03\x07\x01\x13\x03" +
+	"\x07\x06\x06\x03\x07\x05\x0a\x03\x07\x1f\x09\x03\x07\x17:\x03\x06*1\x03" +
+	"\x06-\x1d\x03\x06\x223\x03\x062:\x03\x060$\x03\x066\x1e\x03\x064\x12\x03" +
+	"\x0645\x03\x06\x0b\x00\x03\x06\x0b7\x03\x06\x07\x1f\x03\x06\x15\x12\x03" +
+	"\x0c\x05\x0f\x03\x0b+\x0b\x03\x0b+-\x03\x06\x16\x1b\x03\x06\x15\x17\x03" +
+	"\x89\xca\xea\x03\x89\xca\xe8\x03\x0c8\x10\x03\x0c8\x01\x03\x0c8\x0f\x03" +
+	"\x0d8%\x03\x0d8!\x03\x0c8-\x03\x0c8/\x03\x0c8+\x03\x0c87\x03\x0c85\x03" +
+	"\x0c9\x09\x03\x0c9\x0d\x03\x0c9\x0f\x03\x0c9\x0b\x03\xcfu\x0c\x03\xcfu" +
+	"\x0f\x03\xcfu\x0e\x03\xcfu\x09\x03\x0c9\x10\x03\x0d9\x0c\x03\xcf`;\x03" +
+	"\xcf`>\x03\xcf`9\x03\xcf`8\x03\xcf`7\x03\xcf`*\x03\xcf`-\x03\xcf`,\x03" +
+	"\x0d\x1b\x1a\x03\x0d\x1b&\x03\x0c=.\x03\x0c=%\x03\x0c>\x1e\x03\x0c>\x14" +
+	"\x03\x0c?\x06\x03\x0c?\x0b\x03\x0c?\x0c\x03\x0c?\x0d\x03\x0c?\x02\x03" +
+	"\x0c>\x0f\x03\x0c>\x08\x03\x0c>\x09\x03\x0c>,\x03\x0c>\x0c\x03\x0c?\x13" +
+	"\x03\x0c?\x16\x03\x0c?\x15\x03\x0c?\x1c\x03\x0c?\x1f\x03\x0c?\x1d\x03" +
+	"\x0c?\x1a\x03\x0c?\x17\x03\x0c?\x08\x03\x0c?\x09\x03\x0c?\x0e\x03\x0c?" +
+	"\x04\x03\x0c?\x05\x03\x0c<?\x03\x0c=\x00\x03\x0c=\x06\x03\x0c=\x05\x03" +
+	"\x0c=\x0c\x03\x0c=\x0f\x03\x0c=\x0d\x03\x0c=\x0b\x03\x0c=\x07\x03\x0c=" +
+	"\x19\x03\x0c=\x15\x03\x0c=\x11\x03\x0c=1\x03\x0c=3\x03\x0c=0\x03\x0c=>" +
+	"\x03\x0c=2\x03\x0c=6\x03\x0c<\x07\x03\x0c<\x05\x03\x0e:!\x03\x0e:#\x03" +
+	"\x0e8\x09\x03\x0e:&\x03\x0e8\x0b\x03\x0e:$\x03\x0e:,\x03\x0e8\x1a\x03" +
+	"\x0e8\x1e\x03\x0e:*\x03\x0e:7\x03\x0e:5\x03\x0e:;\x03\x0e:\x15\x03\x0e:<" +
+	"\x03\x0e:4\x03\x0e:'\x03\x0e:-\x03\x0e:%\x03\x0e:?\x03\x0e:=\x03\x0e:)" +
+	"\x03\x0e:/\x03\xcfs'\x03\x0d=\x0f\x03\x0d+*\x03\x0d99\x03\x0d9;\x03\x0d9" +
+	"?\x03\x0d)\x0d\x03\x0d(%\x02\x01\x18\x02\x01(\x02\x01\x1e\x03\x0f$!\x03" +
+	"\x0f87\x03\x0f4\x0e\x03\x0f5\x1d\x03\x06'\x03\x03\x0f\x08\x18\x03\x0f" +
+	"\x0d\x1b\x03\x0e2=\x03\x0e;\x08\x03\x0e:\x0b\x03\x0e\x06$\x03\x0e\x0d)" +
+	"\x03\x0e\x16\x1f\x03\x0e\x16\x1b\x03\x0d$\x0a\x03\x05,\x1d\x03\x0d. \x03" +
+	"\x0d.#\x03\x0c(/\x03\x09%\x02\x03\x0d90\x03\x0d\x0e4\x03\x0d\x0d\x0f\x03" +
+	"\x0c#\x00\x03\x0c,\x1e\x03\x0c2\x0e\x03\x0c\x01\x17\x03\x0c\x09:\x03\x0e" +
+	"\x173\x03\x0c\x08\x03\x03\x0c\x11\x07\x03\x0c\x10\x18\x03\x0c\x1f\x1c" +
+	"\x03\x0c\x19\x0e\x03\x0c\x1a\x1f\x03\x0f0>\x03\x0b->\x03\x0b<+\x03\x0b8" +
+	"\x13\x03\x0b\x043\x03\x0b\x14\x03\x03\x0b\x16%\x03\x0d\x22&\x03\x0b\x1a" +
+	"\x1a\x03\x0b\x1a\x04\x03\x0a%9\x03\x0a&2\x03\x0a&0\x03\x0a!\x1a\x03\x0a!" +
+	"7\x03\x0a5\x10\x03\x0a=4\x03\x0a?\x0e\x03\x0a>\x10\x03\x0a\x00 \x03\x0a" +
+	"\x0f:\x03\x0a\x0f9\x03\x0a\x0b\x0a\x03\x0a\x17%\x03\x0a\x1b-\x03\x09-" +
+	"\x1a\x03\x09,4\x03\x09.,\x03\x09)\x09\x03\x096!\x03\x091\x1f\x03\x093" +
+	"\x16\x03\x0c+\x1f\x03\x098 \x03\x098=\x03\x0c(\x1a\x03\x0c(\x16\x03\x09" +
+	"\x0a+\x03\x09\x16\x12\x03\x09\x13\x0e\x03\x09\x153\x03\x08)!\x03\x09\x1a" +
+	"\x01\x03\x09\x18\x01\x03\x08%#\x03\x08>\x22\x03\x08\x05%\x03\x08\x02*" +
+	"\x03\x08\x15;\x03\x08\x1b7\x03\x0f\x07\x1d\x03\x0f\x04\x03\x03\x070\x0c" +
+	"\x03\x07;\x0b\x03\x07\x08\x17\x03\x07\x12\x06\x03\x06/-\x03\x0671\x03" +
+	"\x065+\x03\x06>7\x03\x06\x049\x03\x05+\x1e\x03\x05,\x17\x03\x05 \x1d\x03" +
+	"\x05\x22\x05\x03\x050\x1d"
+
+// lookup returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *idnaTrie) lookup(s []byte) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return idnaValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = idnaIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *idnaTrie) lookupUnsafe(s []byte) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return idnaValues[c0]
+	}
+	i := idnaIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// lookupString returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *idnaTrie) lookupString(s string) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return idnaValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := idnaIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = idnaIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = idnaIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *idnaTrie) lookupStringUnsafe(s string) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return idnaValues[c0]
+	}
+	i := idnaIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = idnaIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// idnaTrie. Total size: 30288 bytes (29.58 KiB). Checksum: c0cd84404a2f6f19.
+type idnaTrie struct{}
+
+func newIdnaTrie(i int) *idnaTrie {
+	return &idnaTrie{}
+}
+
+// lookupValue determines the type of block n and looks up the value for b.
+func (t *idnaTrie) lookupValue(n uint32, b byte) uint16 {
+	switch {
+	case n < 126:
+		return uint16(idnaValues[n<<6+uint32(b)])
+	default:
+		n -= 126
+		return uint16(idnaSparse.lookup(n, b))
+	}
+}
+
+// idnaValues: 128 blocks, 8192 entries, 16384 bytes
+// The third block is the zero block.
+var idnaValues = [8192]uint16{
+	// Block 0x0, offset 0x0
+	0x00: 0x0080, 0x01: 0x0080, 0x02: 0x0080, 0x03: 0x0080, 0x04: 0x0080, 0x05: 0x0080,
+	0x06: 0x0080, 0x07: 0x0080, 0x08: 0x0080, 0x09: 0x0080, 0x0a: 0x0080, 0x0b: 0x0080,
+	0x0c: 0x0080, 0x0d: 0x0080, 0x0e: 0x0080, 0x0f: 0x0080, 0x10: 0x0080, 0x11: 0x0080,
+	0x12: 0x0080, 0x13: 0x0080, 0x14: 0x0080, 0x15: 0x0080, 0x16: 0x0080, 0x17: 0x0080,
+	0x18: 0x0080, 0x19: 0x0080, 0x1a: 0x0080, 0x1b: 0x0080, 0x1c: 0x0080, 0x1d: 0x0080,
+	0x1e: 0x0080, 0x1f: 0x0080, 0x20: 0x0080, 0x21: 0x0080, 0x22: 0x0080, 0x23: 0x0080,
+	0x24: 0x0080, 0x25: 0x0080, 0x26: 0x0080, 0x27: 0x0080, 0x28: 0x0080, 0x29: 0x0080,
+	0x2a: 0x0080, 0x2b: 0x0080, 0x2c: 0x0080, 0x2d: 0x0008, 0x2e: 0x0008, 0x2f: 0x0080,
+	0x30: 0x0008, 0x31: 0x0008, 0x32: 0x0008, 0x33: 0x0008, 0x34: 0x0008, 0x35: 0x0008,
+	0x36: 0x0008, 0x37: 0x0008, 0x38: 0x0008, 0x39: 0x0008, 0x3a: 0x0080, 0x3b: 0x0080,
+	0x3c: 0x0080, 0x3d: 0x0080, 0x3e: 0x0080, 0x3f: 0x0080,
+	// Block 0x1, offset 0x40
+	0x40: 0x0080, 0x41: 0xe105, 0x42: 0xe105, 0x43: 0xe105, 0x44: 0xe105, 0x45: 0xe105,
+	0x46: 0xe105, 0x47: 0xe105, 0x48: 0xe105, 0x49: 0xe105, 0x4a: 0xe105, 0x4b: 0xe105,
+	0x4c: 0xe105, 0x4d: 0xe105, 0x4e: 0xe105, 0x4f: 0xe105, 0x50: 0xe105, 0x51: 0xe105,
+	0x52: 0xe105, 0x53: 0xe105, 0x54: 0xe105, 0x55: 0xe105, 0x56: 0xe105, 0x57: 0xe105,
+	0x58: 0xe105, 0x59: 0xe105, 0x5a: 0xe105, 0x5b: 0x0080, 0x5c: 0x0080, 0x5d: 0x0080,
+	0x5e: 0x0080, 0x5f: 0x0080, 0x60: 0x0080, 0x61: 0x0008, 0x62: 0x0008, 0x63: 0x0008,
+	0x64: 0x0008, 0x65: 0x0008, 0x66: 0x0008, 0x67: 0x0008, 0x68: 0x0008, 0x69: 0x0008,
+	0x6a: 0x0008, 0x6b: 0x0008, 0x6c: 0x0008, 0x6d: 0x0008, 0x6e: 0x0008, 0x6f: 0x0008,
+	0x70: 0x0008, 0x71: 0x0008, 0x72: 0x0008, 0x73: 0x0008, 0x74: 0x0008, 0x75: 0x0008,
+	0x76: 0x0008, 0x77: 0x0008, 0x78: 0x0008, 0x79: 0x0008, 0x7a: 0x0008, 0x7b: 0x0080,
+	0x7c: 0x0080, 0x7d: 0x0080, 0x7e: 0x0080, 0x7f: 0x0080,
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc0: 0x0040, 0xc1: 0x0040, 0xc2: 0x0040, 0xc3: 0x0040, 0xc4: 0x0040, 0xc5: 0x0040,
+	0xc6: 0x0040, 0xc7: 0x0040, 0xc8: 0x0040, 0xc9: 0x0040, 0xca: 0x0040, 0xcb: 0x0040,
+	0xcc: 0x0040, 0xcd: 0x0040, 0xce: 0x0040, 0xcf: 0x0040, 0xd0: 0x0040, 0xd1: 0x0040,
+	0xd2: 0x0040, 0xd3: 0x0040, 0xd4: 0x0040, 0xd5: 0x0040, 0xd6: 0x0040, 0xd7: 0x0040,
+	0xd8: 0x0040, 0xd9: 0x0040, 0xda: 0x0040, 0xdb: 0x0040, 0xdc: 0x0040, 0xdd: 0x0040,
+	0xde: 0x0040, 0xdf: 0x0040, 0xe0: 0x000a, 0xe1: 0x0018, 0xe2: 0x0018, 0xe3: 0x0018,
+	0xe4: 0x0018, 0xe5: 0x0018, 0xe6: 0x0018, 0xe7: 0x0018, 0xe8: 0x001a, 0xe9: 0x0018,
+	0xea: 0x0039, 0xeb: 0x0018, 0xec: 0x0018, 0xed: 0x03c0, 0xee: 0x0018, 0xef: 0x004a,
+	0xf0: 0x0018, 0xf1: 0x0018, 0xf2: 0x0069, 0xf3: 0x0079, 0xf4: 0x008a, 0xf5: 0x0005,
+	0xf6: 0x0018, 0xf7: 0x0008, 0xf8: 0x00aa, 0xf9: 0x00c9, 0xfa: 0x00d9, 0xfb: 0x0018,
+	0xfc: 0x00e9, 0xfd: 0x0119, 0xfe: 0x0149, 0xff: 0x0018,
+	// Block 0x4, offset 0x100
+	0x100: 0xe00d, 0x101: 0x0008, 0x102: 0xe00d, 0x103: 0x0008, 0x104: 0xe00d, 0x105: 0x0008,
+	0x106: 0xe00d, 0x107: 0x0008, 0x108: 0xe00d, 0x109: 0x0008, 0x10a: 0xe00d, 0x10b: 0x0008,
+	0x10c: 0xe00d, 0x10d: 0x0008, 0x10e: 0xe00d, 0x10f: 0x0008, 0x110: 0xe00d, 0x111: 0x0008,
+	0x112: 0xe00d, 0x113: 0x0008, 0x114: 0xe00d, 0x115: 0x0008, 0x116: 0xe00d, 0x117: 0x0008,
+	0x118: 0xe00d, 0x119: 0x0008, 0x11a: 0xe00d, 0x11b: 0x0008, 0x11c: 0xe00d, 0x11d: 0x0008,
+	0x11e: 0xe00d, 0x11f: 0x0008, 0x120: 0xe00d, 0x121: 0x0008, 0x122: 0xe00d, 0x123: 0x0008,
+	0x124: 0xe00d, 0x125: 0x0008, 0x126: 0xe00d, 0x127: 0x0008, 0x128: 0xe00d, 0x129: 0x0008,
+	0x12a: 0xe00d, 0x12b: 0x0008, 0x12c: 0xe00d, 0x12d: 0x0008, 0x12e: 0xe00d, 0x12f: 0x0008,
+	0x130: 0x0179, 0x131: 0x0008, 0x132: 0x0035, 0x133: 0x004d, 0x134: 0xe00d, 0x135: 0x0008,
+	0x136: 0xe00d, 0x137: 0x0008, 0x138: 0x0008, 0x139: 0xe01d, 0x13a: 0x0008, 0x13b: 0xe03d,
+	0x13c: 0x0008, 0x13d: 0xe01d, 0x13e: 0x0008, 0x13f: 0x0199,
+	// Block 0x5, offset 0x140
+	0x140: 0x0199, 0x141: 0xe01d, 0x142: 0x0008, 0x143: 0xe03d, 0x144: 0x0008, 0x145: 0xe01d,
+	0x146: 0x0008, 0x147: 0xe07d, 0x148: 0x0008, 0x149: 0x01b9, 0x14a: 0xe00d, 0x14b: 0x0008,
+	0x14c: 0xe00d, 0x14d: 0x0008, 0x14e: 0xe00d, 0x14f: 0x0008, 0x150: 0xe00d, 0x151: 0x0008,
+	0x152: 0xe00d, 0x153: 0x0008, 0x154: 0xe00d, 0x155: 0x0008, 0x156: 0xe00d, 0x157: 0x0008,
+	0x158: 0xe00d, 0x159: 0x0008, 0x15a: 0xe00d, 0x15b: 0x0008, 0x15c: 0xe00d, 0x15d: 0x0008,
+	0x15e: 0xe00d, 0x15f: 0x0008, 0x160: 0xe00d, 0x161: 0x0008, 0x162: 0xe00d, 0x163: 0x0008,
+	0x164: 0xe00d, 0x165: 0x0008, 0x166: 0xe00d, 0x167: 0x0008, 0x168: 0xe00d, 0x169: 0x0008,
+	0x16a: 0xe00d, 0x16b: 0x0008, 0x16c: 0xe00d, 0x16d: 0x0008, 0x16e: 0xe00d, 0x16f: 0x0008,
+	0x170: 0xe00d, 0x171: 0x0008, 0x172: 0xe00d, 0x173: 0x0008, 0x174: 0xe00d, 0x175: 0x0008,
+	0x176: 0xe00d, 0x177: 0x0008, 0x178: 0x0065, 0x179: 0xe01d, 0x17a: 0x0008, 0x17b: 0xe03d,
+	0x17c: 0x0008, 0x17d: 0xe01d, 0x17e: 0x0008, 0x17f: 0x01d9,
+	// Block 0x6, offset 0x180
+	0x180: 0x0008, 0x181: 0x007d, 0x182: 0xe00d, 0x183: 0x0008, 0x184: 0xe00d, 0x185: 0x0008,
+	0x186: 0x007d, 0x187: 0xe07d, 0x188: 0x0008, 0x189: 0x0095, 0x18a: 0x00ad, 0x18b: 0xe03d,
+	0x18c: 0x0008, 0x18d: 0x0008, 0x18e: 0x00c5, 0x18f: 0x00dd, 0x190: 0x00f5, 0x191: 0xe01d,
+	0x192: 0x0008, 0x193: 0x010d, 0x194: 0x0125, 0x195: 0x0008, 0x196: 0x013d, 0x197: 0x013d,
+	0x198: 0xe00d, 0x199: 0x0008, 0x19a: 0x0008, 0x19b: 0x0008, 0x19c: 0x010d, 0x19d: 0x0155,
+	0x19e: 0x0008, 0x19f: 0x016d, 0x1a0: 0xe00d, 0x1a1: 0x0008, 0x1a2: 0xe00d, 0x1a3: 0x0008,
+	0x1a4: 0xe00d, 0x1a5: 0x0008, 0x1a6: 0x0185, 0x1a7: 0xe07d, 0x1a8: 0x0008, 0x1a9: 0x019d,
+	0x1aa: 0x0008, 0x1ab: 0x0008, 0x1ac: 0xe00d, 0x1ad: 0x0008, 0x1ae: 0x0185, 0x1af: 0xe0fd,
+	0x1b0: 0x0008, 0x1b1: 0x01b5, 0x1b2: 0x01cd, 0x1b3: 0xe03d, 0x1b4: 0x0008, 0x1b5: 0xe01d,
+	0x1b6: 0x0008, 0x1b7: 0x01e5, 0x1b8: 0xe00d, 0x1b9: 0x0008, 0x1ba: 0x0008, 0x1bb: 0x0008,
+	0x1bc: 0xe00d, 0x1bd: 0x0008, 0x1be: 0x0008, 0x1bf: 0x0008,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x0008, 0x1c1: 0x0008, 0x1c2: 0x0008, 0x1c3: 0x0008, 0x1c4: 0x01e9, 0x1c5: 0x01e9,
+	0x1c6: 0x01e9, 0x1c7: 0x01fd, 0x1c8: 0x0215, 0x1c9: 0x022d, 0x1ca: 0x0245, 0x1cb: 0x025d,
+	0x1cc: 0x0275, 0x1cd: 0xe01d, 0x1ce: 0x0008, 0x1cf: 0xe0fd, 0x1d0: 0x0008, 0x1d1: 0xe01d,
+	0x1d2: 0x0008, 0x1d3: 0xe03d, 0x1d4: 0x0008, 0x1d5: 0xe01d, 0x1d6: 0x0008, 0x1d7: 0xe07d,
+	0x1d8: 0x0008, 0x1d9: 0xe01d, 0x1da: 0x0008, 0x1db: 0xe03d, 0x1dc: 0x0008, 0x1dd: 0x0008,
+	0x1de: 0xe00d, 0x1df: 0x0008, 0x1e0: 0xe00d, 0x1e1: 0x0008, 0x1e2: 0xe00d, 0x1e3: 0x0008,
+	0x1e4: 0xe00d, 0x1e5: 0x0008, 0x1e6: 0xe00d, 0x1e7: 0x0008, 0x1e8: 0xe00d, 0x1e9: 0x0008,
+	0x1ea: 0xe00d, 0x1eb: 0x0008, 0x1ec: 0xe00d, 0x1ed: 0x0008, 0x1ee: 0xe00d, 0x1ef: 0x0008,
+	0x1f0: 0x0008, 0x1f1: 0x028d, 0x1f2: 0x02a5, 0x1f3: 0x02bd, 0x1f4: 0xe00d, 0x1f5: 0x0008,
+	0x1f6: 0x02d5, 0x1f7: 0x02ed, 0x1f8: 0xe00d, 0x1f9: 0x0008, 0x1fa: 0xe00d, 0x1fb: 0x0008,
+	0x1fc: 0xe00d, 0x1fd: 0x0008, 0x1fe: 0xe00d, 0x1ff: 0x0008,
+	// Block 0x8, offset 0x200
+	0x200: 0xe00d, 0x201: 0x0008, 0x202: 0xe00d, 0x203: 0x0008, 0x204: 0xe00d, 0x205: 0x0008,
+	0x206: 0xe00d, 0x207: 0x0008, 0x208: 0xe00d, 0x209: 0x0008, 0x20a: 0xe00d, 0x20b: 0x0008,
+	0x20c: 0xe00d, 0x20d: 0x0008, 0x20e: 0xe00d, 0x20f: 0x0008, 0x210: 0xe00d, 0x211: 0x0008,
+	0x212: 0xe00d, 0x213: 0x0008, 0x214: 0xe00d, 0x215: 0x0008, 0x216: 0xe00d, 0x217: 0x0008,
+	0x218: 0xe00d, 0x219: 0x0008, 0x21a: 0xe00d, 0x21b: 0x0008, 0x21c: 0xe00d, 0x21d: 0x0008,
+	0x21e: 0xe00d, 0x21f: 0x0008, 0x220: 0x0305, 0x221: 0x0008, 0x222: 0xe00d, 0x223: 0x0008,
+	0x224: 0xe00d, 0x225: 0x0008, 0x226: 0xe00d, 0x227: 0x0008, 0x228: 0xe00d, 0x229: 0x0008,
+	0x22a: 0xe00d, 0x22b: 0x0008, 0x22c: 0xe00d, 0x22d: 0x0008, 0x22e: 0xe00d, 0x22f: 0x0008,
+	0x230: 0xe00d, 0x231: 0x0008, 0x232: 0xe00d, 0x233: 0x0008, 0x234: 0x0008, 0x235: 0x0008,
+	0x236: 0x0008, 0x237: 0x0008, 0x238: 0x0008, 0x239: 0x0008, 0x23a: 0x0209, 0x23b: 0xe03d,
+	0x23c: 0x0008, 0x23d: 0x031d, 0x23e: 0x0229, 0x23f: 0x0008,
+	// Block 0x9, offset 0x240
+	0x240: 0x0008, 0x241: 0x0008, 0x242: 0x0018, 0x243: 0x0018, 0x244: 0x0018, 0x245: 0x0018,
+	0x246: 0x0008, 0x247: 0x0008, 0x248: 0x0008, 0x249: 0x0008, 0x24a: 0x0008, 0x24b: 0x0008,
+	0x24c: 0x0008, 0x24d: 0x0008, 0x24e: 0x0008, 0x24f: 0x0008, 0x250: 0x0008, 0x251: 0x0008,
+	0x252: 0x0018, 0x253: 0x0018, 0x254: 0x0018, 0x255: 0x0018, 0x256: 0x0018, 0x257: 0x0018,
+	0x258: 0x029a, 0x259: 0x02ba, 0x25a: 0x02da, 0x25b: 0x02fa, 0x25c: 0x031a, 0x25d: 0x033a,
+	0x25e: 0x0018, 0x25f: 0x0018, 0x260: 0x03ad, 0x261: 0x0359, 0x262: 0x01d9, 0x263: 0x0369,
+	0x264: 0x03c5, 0x265: 0x0018, 0x266: 0x0018, 0x267: 0x0018, 0x268: 0x0018, 0x269: 0x0018,
+	0x26a: 0x0018, 0x26b: 0x0018, 0x26c: 0x0008, 0x26d: 0x0018, 0x26e: 0x0008, 0x26f: 0x0018,
+	0x270: 0x0018, 0x271: 0x0018, 0x272: 0x0018, 0x273: 0x0018, 0x274: 0x0018, 0x275: 0x0018,
+	0x276: 0x0018, 0x277: 0x0018, 0x278: 0x0018, 0x279: 0x0018, 0x27a: 0x0018, 0x27b: 0x0018,
+	0x27c: 0x0018, 0x27d: 0x0018, 0x27e: 0x0018, 0x27f: 0x0018,
+	// Block 0xa, offset 0x280
+	0x280: 0x03dd, 0x281: 0x03dd, 0x282: 0x3308, 0x283: 0x03f5, 0x284: 0x0379, 0x285: 0x040d,
+	0x286: 0x3308, 0x287: 0x3308, 0x288: 0x3308, 0x289: 0x3308, 0x28a: 0x3308, 0x28b: 0x3308,
+	0x28c: 0x3308, 0x28d: 0x3308, 0x28e: 0x3308, 0x28f: 0x33c0, 0x290: 0x3308, 0x291: 0x3308,
+	0x292: 0x3308, 0x293: 0x3308, 0x294: 0x3308, 0x295: 0x3308, 0x296: 0x3308, 0x297: 0x3308,
+	0x298: 0x3308, 0x299: 0x3308, 0x29a: 0x3308, 0x29b: 0x3308, 0x29c: 0x3308, 0x29d: 0x3308,
+	0x29e: 0x3308, 0x29f: 0x3308, 0x2a0: 0x3308, 0x2a1: 0x3308, 0x2a2: 0x3308, 0x2a3: 0x3308,
+	0x2a4: 0x3308, 0x2a5: 0x3308, 0x2a6: 0x3308, 0x2a7: 0x3308, 0x2a8: 0x3308, 0x2a9: 0x3308,
+	0x2aa: 0x3308, 0x2ab: 0x3308, 0x2ac: 0x3308, 0x2ad: 0x3308, 0x2ae: 0x3308, 0x2af: 0x3308,
+	0x2b0: 0xe00d, 0x2b1: 0x0008, 0x2b2: 0xe00d, 0x2b3: 0x0008, 0x2b4: 0x0425, 0x2b5: 0x0008,
+	0x2b6: 0xe00d, 0x2b7: 0x0008, 0x2b8: 0x0040, 0x2b9: 0x0040, 0x2ba: 0x03a2, 0x2bb: 0x0008,
+	0x2bc: 0x0008, 0x2bd: 0x0008, 0x2be: 0x03c2, 0x2bf: 0x043d,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0x0040, 0x2c1: 0x0040, 0x2c2: 0x0040, 0x2c3: 0x0040, 0x2c4: 0x008a, 0x2c5: 0x03d2,
+	0x2c6: 0xe155, 0x2c7: 0x0455, 0x2c8: 0xe12d, 0x2c9: 0xe13d, 0x2ca: 0xe12d, 0x2cb: 0x0040,
+	0x2cc: 0x03dd, 0x2cd: 0x0040, 0x2ce: 0x046d, 0x2cf: 0x0485, 0x2d0: 0x0008, 0x2d1: 0xe105,
+	0x2d2: 0xe105, 0x2d3: 0xe105, 0x2d4: 0xe105, 0x2d5: 0xe105, 0x2d6: 0xe105, 0x2d7: 0xe105,
+	0x2d8: 0xe105, 0x2d9: 0xe105, 0x2da: 0xe105, 0x2db: 0xe105, 0x2dc: 0xe105, 0x2dd: 0xe105,
+	0x2de: 0xe105, 0x2df: 0xe105, 0x2e0: 0x049d, 0x2e1: 0x049d, 0x2e2: 0x0040, 0x2e3: 0x049d,
+	0x2e4: 0x049d, 0x2e5: 0x049d, 0x2e6: 0x049d, 0x2e7: 0x049d, 0x2e8: 0x049d, 0x2e9: 0x049d,
+	0x2ea: 0x049d, 0x2eb: 0x049d, 0x2ec: 0x0008, 0x2ed: 0x0008, 0x2ee: 0x0008, 0x2ef: 0x0008,
+	0x2f0: 0x0008, 0x2f1: 0x0008, 0x2f2: 0x0008, 0x2f3: 0x0008, 0x2f4: 0x0008, 0x2f5: 0x0008,
+	0x2f6: 0x0008, 0x2f7: 0x0008, 0x2f8: 0x0008, 0x2f9: 0x0008, 0x2fa: 0x0008, 0x2fb: 0x0008,
+	0x2fc: 0x0008, 0x2fd: 0x0008, 0x2fe: 0x0008, 0x2ff: 0x0008,
+	// Block 0xc, offset 0x300
+	0x300: 0x0008, 0x301: 0x0008, 0x302: 0xe00f, 0x303: 0x0008, 0x304: 0x0008, 0x305: 0x0008,
+	0x306: 0x0008, 0x307: 0x0008, 0x308: 0x0008, 0x309: 0x0008, 0x30a: 0x0008, 0x30b: 0x0008,
+	0x30c: 0x0008, 0x30d: 0x0008, 0x30e: 0x0008, 0x30f: 0xe0c5, 0x310: 0x04b5, 0x311: 0x04cd,
+	0x312: 0xe0bd, 0x313: 0xe0f5, 0x314: 0xe0fd, 0x315: 0xe09d, 0x316: 0xe0b5, 0x317: 0x0008,
+	0x318: 0xe00d, 0x319: 0x0008, 0x31a: 0xe00d, 0x31b: 0x0008, 0x31c: 0xe00d, 0x31d: 0x0008,
+	0x31e: 0xe00d, 0x31f: 0x0008, 0x320: 0xe00d, 0x321: 0x0008, 0x322: 0xe00d, 0x323: 0x0008,
+	0x324: 0xe00d, 0x325: 0x0008, 0x326: 0xe00d, 0x327: 0x0008, 0x328: 0xe00d, 0x329: 0x0008,
+	0x32a: 0xe00d, 0x32b: 0x0008, 0x32c: 0xe00d, 0x32d: 0x0008, 0x32e: 0xe00d, 0x32f: 0x0008,
+	0x330: 0x04e5, 0x331: 0xe185, 0x332: 0xe18d, 0x333: 0x0008, 0x334: 0x04fd, 0x335: 0x03dd,
+	0x336: 0x0018, 0x337: 0xe07d, 0x338: 0x0008, 0x339: 0xe1d5, 0x33a: 0xe00d, 0x33b: 0x0008,
+	0x33c: 0x0008, 0x33d: 0x0515, 0x33e: 0x052d, 0x33f: 0x052d,
+	// Block 0xd, offset 0x340
+	0x340: 0x0008, 0x341: 0x0008, 0x342: 0x0008, 0x343: 0x0008, 0x344: 0x0008, 0x345: 0x0008,
+	0x346: 0x0008, 0x347: 0x0008, 0x348: 0x0008, 0x349: 0x0008, 0x34a: 0x0008, 0x34b: 0x0008,
+	0x34c: 0x0008, 0x34d: 0x0008, 0x34e: 0x0008, 0x34f: 0x0008, 0x350: 0x0008, 0x351: 0x0008,
+	0x352: 0x0008, 0x353: 0x0008, 0x354: 0x0008, 0x355: 0x0008, 0x356: 0x0008, 0x357: 0x0008,
+	0x358: 0x0008, 0x359: 0x0008, 0x35a: 0x0008, 0x35b: 0x0008, 0x35c: 0x0008, 0x35d: 0x0008,
+	0x35e: 0x0008, 0x35f: 0x0008, 0x360: 0xe00d, 0x361: 0x0008, 0x362: 0xe00d, 0x363: 0x0008,
+	0x364: 0xe00d, 0x365: 0x0008, 0x366: 0xe00d, 0x367: 0x0008, 0x368: 0xe00d, 0x369: 0x0008,
+	0x36a: 0xe00d, 0x36b: 0x0008, 0x36c: 0xe00d, 0x36d: 0x0008, 0x36e: 0xe00d, 0x36f: 0x0008,
+	0x370: 0xe00d, 0x371: 0x0008, 0x372: 0xe00d, 0x373: 0x0008, 0x374: 0xe00d, 0x375: 0x0008,
+	0x376: 0xe00d, 0x377: 0x0008, 0x378: 0xe00d, 0x379: 0x0008, 0x37a: 0xe00d, 0x37b: 0x0008,
+	0x37c: 0xe00d, 0x37d: 0x0008, 0x37e: 0xe00d, 0x37f: 0x0008,
+	// Block 0xe, offset 0x380
+	0x380: 0xe00d, 0x381: 0x0008, 0x382: 0x0018, 0x383: 0x3308, 0x384: 0x3308, 0x385: 0x3308,
+	0x386: 0x3308, 0x387: 0x3308, 0x388: 0x3318, 0x389: 0x3318, 0x38a: 0xe00d, 0x38b: 0x0008,
+	0x38c: 0xe00d, 0x38d: 0x0008, 0x38e: 0xe00d, 0x38f: 0x0008, 0x390: 0xe00d, 0x391: 0x0008,
+	0x392: 0xe00d, 0x393: 0x0008, 0x394: 0xe00d, 0x395: 0x0008, 0x396: 0xe00d, 0x397: 0x0008,
+	0x398: 0xe00d, 0x399: 0x0008, 0x39a: 0xe00d, 0x39b: 0x0008, 0x39c: 0xe00d, 0x39d: 0x0008,
+	0x39e: 0xe00d, 0x39f: 0x0008, 0x3a0: 0xe00d, 0x3a1: 0x0008, 0x3a2: 0xe00d, 0x3a3: 0x0008,
+	0x3a4: 0xe00d, 0x3a5: 0x0008, 0x3a6: 0xe00d, 0x3a7: 0x0008, 0x3a8: 0xe00d, 0x3a9: 0x0008,
+	0x3aa: 0xe00d, 0x3ab: 0x0008, 0x3ac: 0xe00d, 0x3ad: 0x0008, 0x3ae: 0xe00d, 0x3af: 0x0008,
+	0x3b0: 0xe00d, 0x3b1: 0x0008, 0x3b2: 0xe00d, 0x3b3: 0x0008, 0x3b4: 0xe00d, 0x3b5: 0x0008,
+	0x3b6: 0xe00d, 0x3b7: 0x0008, 0x3b8: 0xe00d, 0x3b9: 0x0008, 0x3ba: 0xe00d, 0x3bb: 0x0008,
+	0x3bc: 0xe00d, 0x3bd: 0x0008, 0x3be: 0xe00d, 0x3bf: 0x0008,
+	// Block 0xf, offset 0x3c0
+	0x3c0: 0x0040, 0x3c1: 0xe01d, 0x3c2: 0x0008, 0x3c3: 0xe03d, 0x3c4: 0x0008, 0x3c5: 0xe01d,
+	0x3c6: 0x0008, 0x3c7: 0xe07d, 0x3c8: 0x0008, 0x3c9: 0xe01d, 0x3ca: 0x0008, 0x3cb: 0xe03d,
+	0x3cc: 0x0008, 0x3cd: 0xe01d, 0x3ce: 0x0008, 0x3cf: 0x0008, 0x3d0: 0xe00d, 0x3d1: 0x0008,
+	0x3d2: 0xe00d, 0x3d3: 0x0008, 0x3d4: 0xe00d, 0x3d5: 0x0008, 0x3d6: 0xe00d, 0x3d7: 0x0008,
+	0x3d8: 0xe00d, 0x3d9: 0x0008, 0x3da: 0xe00d, 0x3db: 0x0008, 0x3dc: 0xe00d, 0x3dd: 0x0008,
+	0x3de: 0xe00d, 0x3df: 0x0008, 0x3e0: 0xe00d, 0x3e1: 0x0008, 0x3e2: 0xe00d, 0x3e3: 0x0008,
+	0x3e4: 0xe00d, 0x3e5: 0x0008, 0x3e6: 0xe00d, 0x3e7: 0x0008, 0x3e8: 0xe00d, 0x3e9: 0x0008,
+	0x3ea: 0xe00d, 0x3eb: 0x0008, 0x3ec: 0xe00d, 0x3ed: 0x0008, 0x3ee: 0xe00d, 0x3ef: 0x0008,
+	0x3f0: 0xe00d, 0x3f1: 0x0008, 0x3f2: 0xe00d, 0x3f3: 0x0008, 0x3f4: 0xe00d, 0x3f5: 0x0008,
+	0x3f6: 0xe00d, 0x3f7: 0x0008, 0x3f8: 0xe00d, 0x3f9: 0x0008, 0x3fa: 0xe00d, 0x3fb: 0x0008,
+	0x3fc: 0xe00d, 0x3fd: 0x0008, 0x3fe: 0xe00d, 0x3ff: 0x0008,
+	// Block 0x10, offset 0x400
+	0x400: 0xe00d, 0x401: 0x0008, 0x402: 0xe00d, 0x403: 0x0008, 0x404: 0xe00d, 0x405: 0x0008,
+	0x406: 0xe00d, 0x407: 0x0008, 0x408: 0xe00d, 0x409: 0x0008, 0x40a: 0xe00d, 0x40b: 0x0008,
+	0x40c: 0xe00d, 0x40d: 0x0008, 0x40e: 0xe00d, 0x40f: 0x0008, 0x410: 0xe00d, 0x411: 0x0008,
+	0x412: 0xe00d, 0x413: 0x0008, 0x414: 0xe00d, 0x415: 0x0008, 0x416: 0xe00d, 0x417: 0x0008,
+	0x418: 0xe00d, 0x419: 0x0008, 0x41a: 0xe00d, 0x41b: 0x0008, 0x41c: 0xe00d, 0x41d: 0x0008,
+	0x41e: 0xe00d, 0x41f: 0x0008, 0x420: 0xe00d, 0x421: 0x0008, 0x422: 0xe00d, 0x423: 0x0008,
+	0x424: 0xe00d, 0x425: 0x0008, 0x426: 0xe00d, 0x427: 0x0008, 0x428: 0xe00d, 0x429: 0x0008,
+	0x42a: 0xe00d, 0x42b: 0x0008, 0x42c: 0xe00d, 0x42d: 0x0008, 0x42e: 0xe00d, 0x42f: 0x0008,
+	0x430: 0x0040, 0x431: 0x03f5, 0x432: 0x03f5, 0x433: 0x03f5, 0x434: 0x03f5, 0x435: 0x03f5,
+	0x436: 0x03f5, 0x437: 0x03f5, 0x438: 0x03f5, 0x439: 0x03f5, 0x43a: 0x03f5, 0x43b: 0x03f5,
+	0x43c: 0x03f5, 0x43d: 0x03f5, 0x43e: 0x03f5, 0x43f: 0x03f5,
+	// Block 0x11, offset 0x440
+	0x440: 0x0840, 0x441: 0x0840, 0x442: 0x0840, 0x443: 0x0840, 0x444: 0x0840, 0x445: 0x0840,
+	0x446: 0x0018, 0x447: 0x0018, 0x448: 0x0818, 0x449: 0x0018, 0x44a: 0x0018, 0x44b: 0x0818,
+	0x44c: 0x0018, 0x44d: 0x0818, 0x44e: 0x0018, 0x44f: 0x0018, 0x450: 0x3308, 0x451: 0x3308,
+	0x452: 0x3308, 0x453: 0x3308, 0x454: 0x3308, 0x455: 0x3308, 0x456: 0x3308, 0x457: 0x3308,
+	0x458: 0x3308, 0x459: 0x3308, 0x45a: 0x3308, 0x45b: 0x0818, 0x45c: 0x0b40, 0x45d: 0x0040,
+	0x45e: 0x0818, 0x45f: 0x0818, 0x460: 0x0a08, 0x461: 0x0808, 0x462: 0x0c08, 0x463: 0x0c08,
+	0x464: 0x0c08, 0x465: 0x0c08, 0x466: 0x0a08, 0x467: 0x0c08, 0x468: 0x0a08, 0x469: 0x0c08,
+	0x46a: 0x0a08, 0x46b: 0x0a08, 0x46c: 0x0a08, 0x46d: 0x0a08, 0x46e: 0x0a08, 0x46f: 0x0c08,
+	0x470: 0x0c08, 0x471: 0x0c08, 0x472: 0x0c08, 0x473: 0x0a08, 0x474: 0x0a08, 0x475: 0x0a08,
+	0x476: 0x0a08, 0x477: 0x0a08, 0x478: 0x0a08, 0x479: 0x0a08, 0x47a: 0x0a08, 0x47b: 0x0a08,
+	0x47c: 0x0a08, 0x47d: 0x0a08, 0x47e: 0x0a08, 0x47f: 0x0a08,
+	// Block 0x12, offset 0x480
+	0x480: 0x0818, 0x481: 0x0a08, 0x482: 0x0a08, 0x483: 0x0a08, 0x484: 0x0a08, 0x485: 0x0a08,
+	0x486: 0x0a08, 0x487: 0x0a08, 0x488: 0x0c08, 0x489: 0x0a08, 0x48a: 0x0a08, 0x48b: 0x3308,
+	0x48c: 0x3308, 0x48d: 0x3308, 0x48e: 0x3308, 0x48f: 0x3308, 0x490: 0x3308, 0x491: 0x3308,
+	0x492: 0x3308, 0x493: 0x3308, 0x494: 0x3308, 0x495: 0x3308, 0x496: 0x3308, 0x497: 0x3308,
+	0x498: 0x3308, 0x499: 0x3308, 0x49a: 0x3308, 0x49b: 0x3308, 0x49c: 0x3308, 0x49d: 0x3308,
+	0x49e: 0x3308, 0x49f: 0x3308, 0x4a0: 0x0808, 0x4a1: 0x0808, 0x4a2: 0x0808, 0x4a3: 0x0808,
+	0x4a4: 0x0808, 0x4a5: 0x0808, 0x4a6: 0x0808, 0x4a7: 0x0808, 0x4a8: 0x0808, 0x4a9: 0x0808,
+	0x4aa: 0x0018, 0x4ab: 0x0818, 0x4ac: 0x0818, 0x4ad: 0x0818, 0x4ae: 0x0a08, 0x4af: 0x0a08,
+	0x4b0: 0x3308, 0x4b1: 0x0c08, 0x4b2: 0x0c08, 0x4b3: 0x0c08, 0x4b4: 0x0808, 0x4b5: 0x0429,
+	0x4b6: 0x0451, 0x4b7: 0x0479, 0x4b8: 0x04a1, 0x4b9: 0x0a08, 0x4ba: 0x0a08, 0x4bb: 0x0a08,
+	0x4bc: 0x0a08, 0x4bd: 0x0a08, 0x4be: 0x0a08, 0x4bf: 0x0a08,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0x0c08, 0x4c1: 0x0a08, 0x4c2: 0x0a08, 0x4c3: 0x0c08, 0x4c4: 0x0c08, 0x4c5: 0x0c08,
+	0x4c6: 0x0c08, 0x4c7: 0x0c08, 0x4c8: 0x0c08, 0x4c9: 0x0c08, 0x4ca: 0x0c08, 0x4cb: 0x0c08,
+	0x4cc: 0x0a08, 0x4cd: 0x0c08, 0x4ce: 0x0a08, 0x4cf: 0x0c08, 0x4d0: 0x0a08, 0x4d1: 0x0a08,
+	0x4d2: 0x0c08, 0x4d3: 0x0c08, 0x4d4: 0x0818, 0x4d5: 0x0c08, 0x4d6: 0x3308, 0x4d7: 0x3308,
+	0x4d8: 0x3308, 0x4d9: 0x3308, 0x4da: 0x3308, 0x4db: 0x3308, 0x4dc: 0x3308, 0x4dd: 0x0840,
+	0x4de: 0x0018, 0x4df: 0x3308, 0x4e0: 0x3308, 0x4e1: 0x3308, 0x4e2: 0x3308, 0x4e3: 0x3308,
+	0x4e4: 0x3308, 0x4e5: 0x0808, 0x4e6: 0x0808, 0x4e7: 0x3308, 0x4e8: 0x3308, 0x4e9: 0x0018,
+	0x4ea: 0x3308, 0x4eb: 0x3308, 0x4ec: 0x3308, 0x4ed: 0x3308, 0x4ee: 0x0c08, 0x4ef: 0x0c08,
+	0x4f0: 0x0008, 0x4f1: 0x0008, 0x4f2: 0x0008, 0x4f3: 0x0008, 0x4f4: 0x0008, 0x4f5: 0x0008,
+	0x4f6: 0x0008, 0x4f7: 0x0008, 0x4f8: 0x0008, 0x4f9: 0x0008, 0x4fa: 0x0a08, 0x4fb: 0x0a08,
+	0x4fc: 0x0a08, 0x4fd: 0x0808, 0x4fe: 0x0808, 0x4ff: 0x0a08,
+	// Block 0x14, offset 0x500
+	0x500: 0x0818, 0x501: 0x0818, 0x502: 0x0818, 0x503: 0x0818, 0x504: 0x0818, 0x505: 0x0818,
+	0x506: 0x0818, 0x507: 0x0818, 0x508: 0x0818, 0x509: 0x0818, 0x50a: 0x0818, 0x50b: 0x0818,
+	0x50c: 0x0818, 0x50d: 0x0818, 0x50e: 0x0040, 0x50f: 0x0b40, 0x510: 0x0c08, 0x511: 0x3308,
+	0x512: 0x0a08, 0x513: 0x0a08, 0x514: 0x0a08, 0x515: 0x0c08, 0x516: 0x0c08, 0x517: 0x0c08,
+	0x518: 0x0c08, 0x519: 0x0c08, 0x51a: 0x0a08, 0x51b: 0x0a08, 0x51c: 0x0a08, 0x51d: 0x0a08,
+	0x51e: 0x0c08, 0x51f: 0x0a08, 0x520: 0x0a08, 0x521: 0x0a08, 0x522: 0x0a08, 0x523: 0x0a08,
+	0x524: 0x0a08, 0x525: 0x0a08, 0x526: 0x0a08, 0x527: 0x0a08, 0x528: 0x0c08, 0x529: 0x0a08,
+	0x52a: 0x0c08, 0x52b: 0x0a08, 0x52c: 0x0c08, 0x52d: 0x0a08, 0x52e: 0x0a08, 0x52f: 0x0c08,
+	0x530: 0x3308, 0x531: 0x3308, 0x532: 0x3308, 0x533: 0x3308, 0x534: 0x3308, 0x535: 0x3308,
+	0x536: 0x3308, 0x537: 0x3308, 0x538: 0x3308, 0x539: 0x3308, 0x53a: 0x3308, 0x53b: 0x3308,
+	0x53c: 0x3308, 0x53d: 0x3308, 0x53e: 0x3308, 0x53f: 0x3308,
+	// Block 0x15, offset 0x540
+	0x540: 0x0c08, 0x541: 0x0a08, 0x542: 0x0a08, 0x543: 0x0a08, 0x544: 0x0a08, 0x545: 0x0a08,
+	0x546: 0x0c08, 0x547: 0x0c08, 0x548: 0x0a08, 0x549: 0x0c08, 0x54a: 0x0a08, 0x54b: 0x0a08,
+	0x54c: 0x0a08, 0x54d: 0x0a08, 0x54e: 0x0a08, 0x54f: 0x0a08, 0x550: 0x0a08, 0x551: 0x0a08,
+	0x552: 0x0a08, 0x553: 0x0a08, 0x554: 0x0c08, 0x555: 0x0a08, 0x556: 0x0c08, 0x557: 0x0c08,
+	0x558: 0x0c08, 0x559: 0x3308, 0x55a: 0x3308, 0x55b: 0x3308, 0x55c: 0x0040, 0x55d: 0x0040,
+	0x55e: 0x0818, 0x55f: 0x0040, 0x560: 0x0a08, 0x561: 0x0808, 0x562: 0x0a08, 0x563: 0x0a08,
+	0x564: 0x0a08, 0x565: 0x0a08, 0x566: 0x0808, 0x567: 0x0c08, 0x568: 0x0a08, 0x569: 0x0c08,
+	0x56a: 0x0c08, 0x56b: 0x0040, 0x56c: 0x0040, 0x56d: 0x0040, 0x56e: 0x0040, 0x56f: 0x0040,
+	0x570: 0x0040, 0x571: 0x0040, 0x572: 0x0040, 0x573: 0x0040, 0x574: 0x0040, 0x575: 0x0040,
+	0x576: 0x0040, 0x577: 0x0040, 0x578: 0x0040, 0x579: 0x0040, 0x57a: 0x0040, 0x57b: 0x0040,
+	0x57c: 0x0040, 0x57d: 0x0040, 0x57e: 0x0040, 0x57f: 0x0040,
+	// Block 0x16, offset 0x580
+	0x580: 0x3008, 0x581: 0x3308, 0x582: 0x3308, 0x583: 0x3308, 0x584: 0x3308, 0x585: 0x3308,
+	0x586: 0x3308, 0x587: 0x3308, 0x588: 0x3308, 0x589: 0x3008, 0x58a: 0x3008, 0x58b: 0x3008,
+	0x58c: 0x3008, 0x58d: 0x3b08, 0x58e: 0x3008, 0x58f: 0x3008, 0x590: 0x0008, 0x591: 0x3308,
+	0x592: 0x3308, 0x593: 0x3308, 0x594: 0x3308, 0x595: 0x3308, 0x596: 0x3308, 0x597: 0x3308,
+	0x598: 0x04c9, 0x599: 0x0501, 0x59a: 0x0539, 0x59b: 0x0571, 0x59c: 0x05a9, 0x59d: 0x05e1,
+	0x59e: 0x0619, 0x59f: 0x0651, 0x5a0: 0x0008, 0x5a1: 0x0008, 0x5a2: 0x3308, 0x5a3: 0x3308,
+	0x5a4: 0x0018, 0x5a5: 0x0018, 0x5a6: 0x0008, 0x5a7: 0x0008, 0x5a8: 0x0008, 0x5a9: 0x0008,
+	0x5aa: 0x0008, 0x5ab: 0x0008, 0x5ac: 0x0008, 0x5ad: 0x0008, 0x5ae: 0x0008, 0x5af: 0x0008,
+	0x5b0: 0x0018, 0x5b1: 0x0008, 0x5b2: 0x0008, 0x5b3: 0x0008, 0x5b4: 0x0008, 0x5b5: 0x0008,
+	0x5b6: 0x0008, 0x5b7: 0x0008, 0x5b8: 0x0008, 0x5b9: 0x0008, 0x5ba: 0x0008, 0x5bb: 0x0008,
+	0x5bc: 0x0008, 0x5bd: 0x0008, 0x5be: 0x0008, 0x5bf: 0x0008,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x0008, 0x5c1: 0x3308, 0x5c2: 0x3008, 0x5c3: 0x3008, 0x5c4: 0x0040, 0x5c5: 0x0008,
+	0x5c6: 0x0008, 0x5c7: 0x0008, 0x5c8: 0x0008, 0x5c9: 0x0008, 0x5ca: 0x0008, 0x5cb: 0x0008,
+	0x5cc: 0x0008, 0x5cd: 0x0040, 0x5ce: 0x0040, 0x5cf: 0x0008, 0x5d0: 0x0008, 0x5d1: 0x0040,
+	0x5d2: 0x0040, 0x5d3: 0x0008, 0x5d4: 0x0008, 0x5d5: 0x0008, 0x5d6: 0x0008, 0x5d7: 0x0008,
+	0x5d8: 0x0008, 0x5d9: 0x0008, 0x5da: 0x0008, 0x5db: 0x0008, 0x5dc: 0x0008, 0x5dd: 0x0008,
+	0x5de: 0x0008, 0x5df: 0x0008, 0x5e0: 0x0008, 0x5e1: 0x0008, 0x5e2: 0x0008, 0x5e3: 0x0008,
+	0x5e4: 0x0008, 0x5e5: 0x0008, 0x5e6: 0x0008, 0x5e7: 0x0008, 0x5e8: 0x0008, 0x5e9: 0x0040,
+	0x5ea: 0x0008, 0x5eb: 0x0008, 0x5ec: 0x0008, 0x5ed: 0x0008, 0x5ee: 0x0008, 0x5ef: 0x0008,
+	0x5f0: 0x0008, 0x5f1: 0x0040, 0x5f2: 0x0008, 0x5f3: 0x0040, 0x5f4: 0x0040, 0x5f5: 0x0040,
+	0x5f6: 0x0008, 0x5f7: 0x0008, 0x5f8: 0x0008, 0x5f9: 0x0008, 0x5fa: 0x0040, 0x5fb: 0x0040,
+	0x5fc: 0x3308, 0x5fd: 0x0008, 0x5fe: 0x3008, 0x5ff: 0x3008,
+	// Block 0x18, offset 0x600
+	0x600: 0x3008, 0x601: 0x3308, 0x602: 0x3308, 0x603: 0x3308, 0x604: 0x3308, 0x605: 0x0040,
+	0x606: 0x0040, 0x607: 0x3008, 0x608: 0x3008, 0x609: 0x0040, 0x60a: 0x0040, 0x60b: 0x3008,
+	0x60c: 0x3008, 0x60d: 0x3b08, 0x60e: 0x0008, 0x60f: 0x0040, 0x610: 0x0040, 0x611: 0x0040,
+	0x612: 0x0040, 0x613: 0x0040, 0x614: 0x0040, 0x615: 0x0040, 0x616: 0x0040, 0x617: 0x3008,
+	0x618: 0x0040, 0x619: 0x0040, 0x61a: 0x0040, 0x61b: 0x0040, 0x61c: 0x0689, 0x61d: 0x06c1,
+	0x61e: 0x0040, 0x61f: 0x06f9, 0x620: 0x0008, 0x621: 0x0008, 0x622: 0x3308, 0x623: 0x3308,
+	0x624: 0x0040, 0x625: 0x0040, 0x626: 0x0008, 0x627: 0x0008, 0x628: 0x0008, 0x629: 0x0008,
+	0x62a: 0x0008, 0x62b: 0x0008, 0x62c: 0x0008, 0x62d: 0x0008, 0x62e: 0x0008, 0x62f: 0x0008,
+	0x630: 0x0008, 0x631: 0x0008, 0x632: 0x0018, 0x633: 0x0018, 0x634: 0x0018, 0x635: 0x0018,
+	0x636: 0x0018, 0x637: 0x0018, 0x638: 0x0018, 0x639: 0x0018, 0x63a: 0x0018, 0x63b: 0x0018,
+	0x63c: 0x0008, 0x63d: 0x0018, 0x63e: 0x3308, 0x63f: 0x0040,
+	// Block 0x19, offset 0x640
+	0x640: 0x0040, 0x641: 0x3308, 0x642: 0x3308, 0x643: 0x3008, 0x644: 0x0040, 0x645: 0x0008,
+	0x646: 0x0008, 0x647: 0x0008, 0x648: 0x0008, 0x649: 0x0008, 0x64a: 0x0008, 0x64b: 0x0040,
+	0x64c: 0x0040, 0x64d: 0x0040, 0x64e: 0x0040, 0x64f: 0x0008, 0x650: 0x0008, 0x651: 0x0040,
+	0x652: 0x0040, 0x653: 0x0008, 0x654: 0x0008, 0x655: 0x0008, 0x656: 0x0008, 0x657: 0x0008,
+	0x658: 0x0008, 0x659: 0x0008, 0x65a: 0x0008, 0x65b: 0x0008, 0x65c: 0x0008, 0x65d: 0x0008,
+	0x65e: 0x0008, 0x65f: 0x0008, 0x660: 0x0008, 0x661: 0x0008, 0x662: 0x0008, 0x663: 0x0008,
+	0x664: 0x0008, 0x665: 0x0008, 0x666: 0x0008, 0x667: 0x0008, 0x668: 0x0008, 0x669: 0x0040,
+	0x66a: 0x0008, 0x66b: 0x0008, 0x66c: 0x0008, 0x66d: 0x0008, 0x66e: 0x0008, 0x66f: 0x0008,
+	0x670: 0x0008, 0x671: 0x0040, 0x672: 0x0008, 0x673: 0x0731, 0x674: 0x0040, 0x675: 0x0008,
+	0x676: 0x0769, 0x677: 0x0040, 0x678: 0x0008, 0x679: 0x0008, 0x67a: 0x0040, 0x67b: 0x0040,
+	0x67c: 0x3308, 0x67d: 0x0040, 0x67e: 0x3008, 0x67f: 0x3008,
+	// Block 0x1a, offset 0x680
+	0x680: 0x3008, 0x681: 0x3308, 0x682: 0x3308, 0x683: 0x0040, 0x684: 0x0040, 0x685: 0x0040,
+	0x686: 0x0040, 0x687: 0x3308, 0x688: 0x3308, 0x689: 0x0040, 0x68a: 0x0040, 0x68b: 0x3308,
+	0x68c: 0x3308, 0x68d: 0x3b08, 0x68e: 0x0040, 0x68f: 0x0040, 0x690: 0x0040, 0x691: 0x3308,
+	0x692: 0x0040, 0x693: 0x0040, 0x694: 0x0040, 0x695: 0x0040, 0x696: 0x0040, 0x697: 0x0040,
+	0x698: 0x0040, 0x699: 0x07a1, 0x69a: 0x07d9, 0x69b: 0x0811, 0x69c: 0x0008, 0x69d: 0x0040,
+	0x69e: 0x0849, 0x69f: 0x0040, 0x6a0: 0x0040, 0x6a1: 0x0040, 0x6a2: 0x0040, 0x6a3: 0x0040,
+	0x6a4: 0x0040, 0x6a5: 0x0040, 0x6a6: 0x0008, 0x6a7: 0x0008, 0x6a8: 0x0008, 0x6a9: 0x0008,
+	0x6aa: 0x0008, 0x6ab: 0x0008, 0x6ac: 0x0008, 0x6ad: 0x0008, 0x6ae: 0x0008, 0x6af: 0x0008,
+	0x6b0: 0x3308, 0x6b1: 0x3308, 0x6b2: 0x0008, 0x6b3: 0x0008, 0x6b4: 0x0008, 0x6b5: 0x3308,
+	0x6b6: 0x0018, 0x6b7: 0x0040, 0x6b8: 0x0040, 0x6b9: 0x0040, 0x6ba: 0x0040, 0x6bb: 0x0040,
+	0x6bc: 0x0040, 0x6bd: 0x0040, 0x6be: 0x0040, 0x6bf: 0x0040,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0x0040, 0x6c1: 0x3308, 0x6c2: 0x3308, 0x6c3: 0x3008, 0x6c4: 0x0040, 0x6c5: 0x0008,
+	0x6c6: 0x0008, 0x6c7: 0x0008, 0x6c8: 0x0008, 0x6c9: 0x0008, 0x6ca: 0x0008, 0x6cb: 0x0008,
+	0x6cc: 0x0008, 0x6cd: 0x0008, 0x6ce: 0x0040, 0x6cf: 0x0008, 0x6d0: 0x0008, 0x6d1: 0x0008,
+	0x6d2: 0x0040, 0x6d3: 0x0008, 0x6d4: 0x0008, 0x6d5: 0x0008, 0x6d6: 0x0008, 0x6d7: 0x0008,
+	0x6d8: 0x0008, 0x6d9: 0x0008, 0x6da: 0x0008, 0x6db: 0x0008, 0x6dc: 0x0008, 0x6dd: 0x0008,
+	0x6de: 0x0008, 0x6df: 0x0008, 0x6e0: 0x0008, 0x6e1: 0x0008, 0x6e2: 0x0008, 0x6e3: 0x0008,
+	0x6e4: 0x0008, 0x6e5: 0x0008, 0x6e6: 0x0008, 0x6e7: 0x0008, 0x6e8: 0x0008, 0x6e9: 0x0040,
+	0x6ea: 0x0008, 0x6eb: 0x0008, 0x6ec: 0x0008, 0x6ed: 0x0008, 0x6ee: 0x0008, 0x6ef: 0x0008,
+	0x6f0: 0x0008, 0x6f1: 0x0040, 0x6f2: 0x0008, 0x6f3: 0x0008, 0x6f4: 0x0040, 0x6f5: 0x0008,
+	0x6f6: 0x0008, 0x6f7: 0x0008, 0x6f8: 0x0008, 0x6f9: 0x0008, 0x6fa: 0x0040, 0x6fb: 0x0040,
+	0x6fc: 0x3308, 0x6fd: 0x0008, 0x6fe: 0x3008, 0x6ff: 0x3008,
+	// Block 0x1c, offset 0x700
+	0x700: 0x3008, 0x701: 0x3308, 0x702: 0x3308, 0x703: 0x3308, 0x704: 0x3308, 0x705: 0x3308,
+	0x706: 0x0040, 0x707: 0x3308, 0x708: 0x3308, 0x709: 0x3008, 0x70a: 0x0040, 0x70b: 0x3008,
+	0x70c: 0x3008, 0x70d: 0x3b08, 0x70e: 0x0040, 0x70f: 0x0040, 0x710: 0x0008, 0x711: 0x0040,
+	0x712: 0x0040, 0x713: 0x0040, 0x714: 0x0040, 0x715: 0x0040, 0x716: 0x0040, 0x717: 0x0040,
+	0x718: 0x0040, 0x719: 0x0040, 0x71a: 0x0040, 0x71b: 0x0040, 0x71c: 0x0040, 0x71d: 0x0040,
+	0x71e: 0x0040, 0x71f: 0x0040, 0x720: 0x0008, 0x721: 0x0008, 0x722: 0x3308, 0x723: 0x3308,
+	0x724: 0x0040, 0x725: 0x0040, 0x726: 0x0008, 0x727: 0x0008, 0x728: 0x0008, 0x729: 0x0008,
+	0x72a: 0x0008, 0x72b: 0x0008, 0x72c: 0x0008, 0x72d: 0x0008, 0x72e: 0x0008, 0x72f: 0x0008,
+	0x730: 0x0018, 0x731: 0x0018, 0x732: 0x0040, 0x733: 0x0040, 0x734: 0x0040, 0x735: 0x0040,
+	0x736: 0x0040, 0x737: 0x0040, 0x738: 0x0040, 0x739: 0x0008, 0x73a: 0x3308, 0x73b: 0x3308,
+	0x73c: 0x3308, 0x73d: 0x3308, 0x73e: 0x3308, 0x73f: 0x3308,
+	// Block 0x1d, offset 0x740
+	0x740: 0x0040, 0x741: 0x3308, 0x742: 0x3008, 0x743: 0x3008, 0x744: 0x0040, 0x745: 0x0008,
+	0x746: 0x0008, 0x747: 0x0008, 0x748: 0x0008, 0x749: 0x0008, 0x74a: 0x0008, 0x74b: 0x0008,
+	0x74c: 0x0008, 0x74d: 0x0040, 0x74e: 0x0040, 0x74f: 0x0008, 0x750: 0x0008, 0x751: 0x0040,
+	0x752: 0x0040, 0x753: 0x0008, 0x754: 0x0008, 0x755: 0x0008, 0x756: 0x0008, 0x757: 0x0008,
+	0x758: 0x0008, 0x759: 0x0008, 0x75a: 0x0008, 0x75b: 0x0008, 0x75c: 0x0008, 0x75d: 0x0008,
+	0x75e: 0x0008, 0x75f: 0x0008, 0x760: 0x0008, 0x761: 0x0008, 0x762: 0x0008, 0x763: 0x0008,
+	0x764: 0x0008, 0x765: 0x0008, 0x766: 0x0008, 0x767: 0x0008, 0x768: 0x0008, 0x769: 0x0040,
+	0x76a: 0x0008, 0x76b: 0x0008, 0x76c: 0x0008, 0x76d: 0x0008, 0x76e: 0x0008, 0x76f: 0x0008,
+	0x770: 0x0008, 0x771: 0x0040, 0x772: 0x0008, 0x773: 0x0008, 0x774: 0x0040, 0x775: 0x0008,
+	0x776: 0x0008, 0x777: 0x0008, 0x778: 0x0008, 0x779: 0x0008, 0x77a: 0x0040, 0x77b: 0x0040,
+	0x77c: 0x3308, 0x77d: 0x0008, 0x77e: 0x3008, 0x77f: 0x3308,
+	// Block 0x1e, offset 0x780
+	0x780: 0x3008, 0x781: 0x3308, 0x782: 0x3308, 0x783: 0x3308, 0x784: 0x3308, 0x785: 0x0040,
+	0x786: 0x0040, 0x787: 0x3008, 0x788: 0x3008, 0x789: 0x0040, 0x78a: 0x0040, 0x78b: 0x3008,
+	0x78c: 0x3008, 0x78d: 0x3b08, 0x78e: 0x0040, 0x78f: 0x0040, 0x790: 0x0040, 0x791: 0x0040,
+	0x792: 0x0040, 0x793: 0x0040, 0x794: 0x0040, 0x795: 0x3308, 0x796: 0x3308, 0x797: 0x3008,
+	0x798: 0x0040, 0x799: 0x0040, 0x79a: 0x0040, 0x79b: 0x0040, 0x79c: 0x0881, 0x79d: 0x08b9,
+	0x79e: 0x0040, 0x79f: 0x0008, 0x7a0: 0x0008, 0x7a1: 0x0008, 0x7a2: 0x3308, 0x7a3: 0x3308,
+	0x7a4: 0x0040, 0x7a5: 0x0040, 0x7a6: 0x0008, 0x7a7: 0x0008, 0x7a8: 0x0008, 0x7a9: 0x0008,
+	0x7aa: 0x0008, 0x7ab: 0x0008, 0x7ac: 0x0008, 0x7ad: 0x0008, 0x7ae: 0x0008, 0x7af: 0x0008,
+	0x7b0: 0x0018, 0x7b1: 0x0008, 0x7b2: 0x0018, 0x7b3: 0x0018, 0x7b4: 0x0018, 0x7b5: 0x0018,
+	0x7b6: 0x0018, 0x7b7: 0x0018, 0x7b8: 0x0040, 0x7b9: 0x0040, 0x7ba: 0x0040, 0x7bb: 0x0040,
+	0x7bc: 0x0040, 0x7bd: 0x0040, 0x7be: 0x0040, 0x7bf: 0x0040,
+	// Block 0x1f, offset 0x7c0
+	0x7c0: 0x0040, 0x7c1: 0x0040, 0x7c2: 0x3308, 0x7c3: 0x0008, 0x7c4: 0x0040, 0x7c5: 0x0008,
+	0x7c6: 0x0008, 0x7c7: 0x0008, 0x7c8: 0x0008, 0x7c9: 0x0008, 0x7ca: 0x0008, 0x7cb: 0x0040,
+	0x7cc: 0x0040, 0x7cd: 0x0040, 0x7ce: 0x0008, 0x7cf: 0x0008, 0x7d0: 0x0008, 0x7d1: 0x0040,
+	0x7d2: 0x0008, 0x7d3: 0x0008, 0x7d4: 0x0008, 0x7d5: 0x0008, 0x7d6: 0x0040, 0x7d7: 0x0040,
+	0x7d8: 0x0040, 0x7d9: 0x0008, 0x7da: 0x0008, 0x7db: 0x0040, 0x7dc: 0x0008, 0x7dd: 0x0040,
+	0x7de: 0x0008, 0x7df: 0x0008, 0x7e0: 0x0040, 0x7e1: 0x0040, 0x7e2: 0x0040, 0x7e3: 0x0008,
+	0x7e4: 0x0008, 0x7e5: 0x0040, 0x7e6: 0x0040, 0x7e7: 0x0040, 0x7e8: 0x0008, 0x7e9: 0x0008,
+	0x7ea: 0x0008, 0x7eb: 0x0040, 0x7ec: 0x0040, 0x7ed: 0x0040, 0x7ee: 0x0008, 0x7ef: 0x0008,
+	0x7f0: 0x0008, 0x7f1: 0x0008, 0x7f2: 0x0008, 0x7f3: 0x0008, 0x7f4: 0x0008, 0x7f5: 0x0008,
+	0x7f6: 0x0008, 0x7f7: 0x0008, 0x7f8: 0x0008, 0x7f9: 0x0008, 0x7fa: 0x0040, 0x7fb: 0x0040,
+	0x7fc: 0x0040, 0x7fd: 0x0040, 0x7fe: 0x3008, 0x7ff: 0x3008,
+	// Block 0x20, offset 0x800
+	0x800: 0x3308, 0x801: 0x3008, 0x802: 0x3008, 0x803: 0x3008, 0x804: 0x3008, 0x805: 0x0040,
+	0x806: 0x3308, 0x807: 0x3308, 0x808: 0x3308, 0x809: 0x0040, 0x80a: 0x3308, 0x80b: 0x3308,
+	0x80c: 0x3308, 0x80d: 0x3b08, 0x80e: 0x0040, 0x80f: 0x0040, 0x810: 0x0040, 0x811: 0x0040,
+	0x812: 0x0040, 0x813: 0x0040, 0x814: 0x0040, 0x815: 0x3308, 0x816: 0x3308, 0x817: 0x0040,
+	0x818: 0x0008, 0x819: 0x0008, 0x81a: 0x0008, 0x81b: 0x0040, 0x81c: 0x0040, 0x81d: 0x0040,
+	0x81e: 0x0040, 0x81f: 0x0040, 0x820: 0x0008, 0x821: 0x0008, 0x822: 0x3308, 0x823: 0x3308,
+	0x824: 0x0040, 0x825: 0x0040, 0x826: 0x0008, 0x827: 0x0008, 0x828: 0x0008, 0x829: 0x0008,
+	0x82a: 0x0008, 0x82b: 0x0008, 0x82c: 0x0008, 0x82d: 0x0008, 0x82e: 0x0008, 0x82f: 0x0008,
+	0x830: 0x0040, 0x831: 0x0040, 0x832: 0x0040, 0x833: 0x0040, 0x834: 0x0040, 0x835: 0x0040,
+	0x836: 0x0040, 0x837: 0x0018, 0x838: 0x0018, 0x839: 0x0018, 0x83a: 0x0018, 0x83b: 0x0018,
+	0x83c: 0x0018, 0x83d: 0x0018, 0x83e: 0x0018, 0x83f: 0x0018,
+	// Block 0x21, offset 0x840
+	0x840: 0x0008, 0x841: 0x3308, 0x842: 0x3008, 0x843: 0x3008, 0x844: 0x0018, 0x845: 0x0008,
+	0x846: 0x0008, 0x847: 0x0008, 0x848: 0x0008, 0x849: 0x0008, 0x84a: 0x0008, 0x84b: 0x0008,
+	0x84c: 0x0008, 0x84d: 0x0040, 0x84e: 0x0008, 0x84f: 0x0008, 0x850: 0x0008, 0x851: 0x0040,
+	0x852: 0x0008, 0x853: 0x0008, 0x854: 0x0008, 0x855: 0x0008, 0x856: 0x0008, 0x857: 0x0008,
+	0x858: 0x0008, 0x859: 0x0008, 0x85a: 0x0008, 0x85b: 0x0008, 0x85c: 0x0008, 0x85d: 0x0008,
+	0x85e: 0x0008, 0x85f: 0x0008, 0x860: 0x0008, 0x861: 0x0008, 0x862: 0x0008, 0x863: 0x0008,
+	0x864: 0x0008, 0x865: 0x0008, 0x866: 0x0008, 0x867: 0x0008, 0x868: 0x0008, 0x869: 0x0040,
+	0x86a: 0x0008, 0x86b: 0x0008, 0x86c: 0x0008, 0x86d: 0x0008, 0x86e: 0x0008, 0x86f: 0x0008,
+	0x870: 0x0008, 0x871: 0x0008, 0x872: 0x0008, 0x873: 0x0008, 0x874: 0x0040, 0x875: 0x0008,
+	0x876: 0x0008, 0x877: 0x0008, 0x878: 0x0008, 0x879: 0x0008, 0x87a: 0x0040, 0x87b: 0x0040,
+	0x87c: 0x3308, 0x87d: 0x0008, 0x87e: 0x3008, 0x87f: 0x3308,
+	// Block 0x22, offset 0x880
+	0x880: 0x3008, 0x881: 0x3008, 0x882: 0x3008, 0x883: 0x3008, 0x884: 0x3008, 0x885: 0x0040,
+	0x886: 0x3308, 0x887: 0x3008, 0x888: 0x3008, 0x889: 0x0040, 0x88a: 0x3008, 0x88b: 0x3008,
+	0x88c: 0x3308, 0x88d: 0x3b08, 0x88e: 0x0040, 0x88f: 0x0040, 0x890: 0x0040, 0x891: 0x0040,
+	0x892: 0x0040, 0x893: 0x0040, 0x894: 0x0040, 0x895: 0x3008, 0x896: 0x3008, 0x897: 0x0040,
+	0x898: 0x0040, 0x899: 0x0040, 0x89a: 0x0040, 0x89b: 0x0040, 0x89c: 0x0040, 0x89d: 0x0040,
+	0x89e: 0x0008, 0x89f: 0x0040, 0x8a0: 0x0008, 0x8a1: 0x0008, 0x8a2: 0x3308, 0x8a3: 0x3308,
+	0x8a4: 0x0040, 0x8a5: 0x0040, 0x8a6: 0x0008, 0x8a7: 0x0008, 0x8a8: 0x0008, 0x8a9: 0x0008,
+	0x8aa: 0x0008, 0x8ab: 0x0008, 0x8ac: 0x0008, 0x8ad: 0x0008, 0x8ae: 0x0008, 0x8af: 0x0008,
+	0x8b0: 0x0040, 0x8b1: 0x0008, 0x8b2: 0x0008, 0x8b3: 0x0040, 0x8b4: 0x0040, 0x8b5: 0x0040,
+	0x8b6: 0x0040, 0x8b7: 0x0040, 0x8b8: 0x0040, 0x8b9: 0x0040, 0x8ba: 0x0040, 0x8bb: 0x0040,
+	0x8bc: 0x0040, 0x8bd: 0x0040, 0x8be: 0x0040, 0x8bf: 0x0040,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x3008, 0x8c1: 0x3308, 0x8c2: 0x3308, 0x8c3: 0x3308, 0x8c4: 0x3308, 0x8c5: 0x0040,
+	0x8c6: 0x3008, 0x8c7: 0x3008, 0x8c8: 0x3008, 0x8c9: 0x0040, 0x8ca: 0x3008, 0x8cb: 0x3008,
+	0x8cc: 0x3008, 0x8cd: 0x3b08, 0x8ce: 0x0008, 0x8cf: 0x0018, 0x8d0: 0x0040, 0x8d1: 0x0040,
+	0x8d2: 0x0040, 0x8d3: 0x0040, 0x8d4: 0x0008, 0x8d5: 0x0008, 0x8d6: 0x0008, 0x8d7: 0x3008,
+	0x8d8: 0x0018, 0x8d9: 0x0018, 0x8da: 0x0018, 0x8db: 0x0018, 0x8dc: 0x0018, 0x8dd: 0x0018,
+	0x8de: 0x0018, 0x8df: 0x0008, 0x8e0: 0x0008, 0x8e1: 0x0008, 0x8e2: 0x3308, 0x8e3: 0x3308,
+	0x8e4: 0x0040, 0x8e5: 0x0040, 0x8e6: 0x0008, 0x8e7: 0x0008, 0x8e8: 0x0008, 0x8e9: 0x0008,
+	0x8ea: 0x0008, 0x8eb: 0x0008, 0x8ec: 0x0008, 0x8ed: 0x0008, 0x8ee: 0x0008, 0x8ef: 0x0008,
+	0x8f0: 0x0018, 0x8f1: 0x0018, 0x8f2: 0x0018, 0x8f3: 0x0018, 0x8f4: 0x0018, 0x8f5: 0x0018,
+	0x8f6: 0x0018, 0x8f7: 0x0018, 0x8f8: 0x0018, 0x8f9: 0x0018, 0x8fa: 0x0008, 0x8fb: 0x0008,
+	0x8fc: 0x0008, 0x8fd: 0x0008, 0x8fe: 0x0008, 0x8ff: 0x0008,
+	// Block 0x24, offset 0x900
+	0x900: 0x0040, 0x901: 0x0008, 0x902: 0x0008, 0x903: 0x0040, 0x904: 0x0008, 0x905: 0x0040,
+	0x906: 0x0008, 0x907: 0x0008, 0x908: 0x0008, 0x909: 0x0008, 0x90a: 0x0008, 0x90b: 0x0040,
+	0x90c: 0x0008, 0x90d: 0x0008, 0x90e: 0x0008, 0x90f: 0x0008, 0x910: 0x0008, 0x911: 0x0008,
+	0x912: 0x0008, 0x913: 0x0008, 0x914: 0x0008, 0x915: 0x0008, 0x916: 0x0008, 0x917: 0x0008,
+	0x918: 0x0008, 0x919: 0x0008, 0x91a: 0x0008, 0x91b: 0x0008, 0x91c: 0x0008, 0x91d: 0x0008,
+	0x91e: 0x0008, 0x91f: 0x0008, 0x920: 0x0008, 0x921: 0x0008, 0x922: 0x0008, 0x923: 0x0008,
+	0x924: 0x0040, 0x925: 0x0008, 0x926: 0x0040, 0x927: 0x0008, 0x928: 0x0008, 0x929: 0x0008,
+	0x92a: 0x0008, 0x92b: 0x0008, 0x92c: 0x0008, 0x92d: 0x0008, 0x92e: 0x0008, 0x92f: 0x0008,
+	0x930: 0x0008, 0x931: 0x3308, 0x932: 0x0008, 0x933: 0x0929, 0x934: 0x3308, 0x935: 0x3308,
+	0x936: 0x3308, 0x937: 0x3308, 0x938: 0x3308, 0x939: 0x3308, 0x93a: 0x3b08, 0x93b: 0x3308,
+	0x93c: 0x3308, 0x93d: 0x0008, 0x93e: 0x0040, 0x93f: 0x0040,
+	// Block 0x25, offset 0x940
+	0x940: 0x0008, 0x941: 0x0008, 0x942: 0x0008, 0x943: 0x09d1, 0x944: 0x0008, 0x945: 0x0008,
+	0x946: 0x0008, 0x947: 0x0008, 0x948: 0x0040, 0x949: 0x0008, 0x94a: 0x0008, 0x94b: 0x0008,
+	0x94c: 0x0008, 0x94d: 0x0a09, 0x94e: 0x0008, 0x94f: 0x0008, 0x950: 0x0008, 0x951: 0x0008,
+	0x952: 0x0a41, 0x953: 0x0008, 0x954: 0x0008, 0x955: 0x0008, 0x956: 0x0008, 0x957: 0x0a79,
+	0x958: 0x0008, 0x959: 0x0008, 0x95a: 0x0008, 0x95b: 0x0008, 0x95c: 0x0ab1, 0x95d: 0x0008,
+	0x95e: 0x0008, 0x95f: 0x0008, 0x960: 0x0008, 0x961: 0x0008, 0x962: 0x0008, 0x963: 0x0008,
+	0x964: 0x0008, 0x965: 0x0008, 0x966: 0x0008, 0x967: 0x0008, 0x968: 0x0008, 0x969: 0x0ae9,
+	0x96a: 0x0008, 0x96b: 0x0008, 0x96c: 0x0008, 0x96d: 0x0040, 0x96e: 0x0040, 0x96f: 0x0040,
+	0x970: 0x0040, 0x971: 0x3308, 0x972: 0x3308, 0x973: 0x0b21, 0x974: 0x3308, 0x975: 0x0b59,
+	0x976: 0x0b91, 0x977: 0x0bc9, 0x978: 0x0c19, 0x979: 0x0c51, 0x97a: 0x3308, 0x97b: 0x3308,
+	0x97c: 0x3308, 0x97d: 0x3308, 0x97e: 0x3308, 0x97f: 0x3008,
+	// Block 0x26, offset 0x980
+	0x980: 0x3308, 0x981: 0x0ca1, 0x982: 0x3308, 0x983: 0x3308, 0x984: 0x3b08, 0x985: 0x0018,
+	0x986: 0x3308, 0x987: 0x3308, 0x988: 0x0008, 0x989: 0x0008, 0x98a: 0x0008, 0x98b: 0x0008,
+	0x98c: 0x0008, 0x98d: 0x3308, 0x98e: 0x3308, 0x98f: 0x3308, 0x990: 0x3308, 0x991: 0x3308,
+	0x992: 0x3308, 0x993: 0x0cd9, 0x994: 0x3308, 0x995: 0x3308, 0x996: 0x3308, 0x997: 0x3308,
+	0x998: 0x0040, 0x999: 0x3308, 0x99a: 0x3308, 0x99b: 0x3308, 0x99c: 0x3308, 0x99d: 0x0d11,
+	0x99e: 0x3308, 0x99f: 0x3308, 0x9a0: 0x3308, 0x9a1: 0x3308, 0x9a2: 0x0d49, 0x9a3: 0x3308,
+	0x9a4: 0x3308, 0x9a5: 0x3308, 0x9a6: 0x3308, 0x9a7: 0x0d81, 0x9a8: 0x3308, 0x9a9: 0x3308,
+	0x9aa: 0x3308, 0x9ab: 0x3308, 0x9ac: 0x0db9, 0x9ad: 0x3308, 0x9ae: 0x3308, 0x9af: 0x3308,
+	0x9b0: 0x3308, 0x9b1: 0x3308, 0x9b2: 0x3308, 0x9b3: 0x3308, 0x9b4: 0x3308, 0x9b5: 0x3308,
+	0x9b6: 0x3308, 0x9b7: 0x3308, 0x9b8: 0x3308, 0x9b9: 0x0df1, 0x9ba: 0x3308, 0x9bb: 0x3308,
+	0x9bc: 0x3308, 0x9bd: 0x0040, 0x9be: 0x0018, 0x9bf: 0x0018,
+	// Block 0x27, offset 0x9c0
+	0x9c0: 0x0008, 0x9c1: 0x0008, 0x9c2: 0x0008, 0x9c3: 0x0008, 0x9c4: 0x0008, 0x9c5: 0x0008,
+	0x9c6: 0x0008, 0x9c7: 0x0008, 0x9c8: 0x0008, 0x9c9: 0x0008, 0x9ca: 0x0008, 0x9cb: 0x0008,
+	0x9cc: 0x0008, 0x9cd: 0x0008, 0x9ce: 0x0008, 0x9cf: 0x0008, 0x9d0: 0x0008, 0x9d1: 0x0008,
+	0x9d2: 0x0008, 0x9d3: 0x0008, 0x9d4: 0x0008, 0x9d5: 0x0008, 0x9d6: 0x0008, 0x9d7: 0x0008,
+	0x9d8: 0x0008, 0x9d9: 0x0008, 0x9da: 0x0008, 0x9db: 0x0008, 0x9dc: 0x0008, 0x9dd: 0x0008,
+	0x9de: 0x0008, 0x9df: 0x0008, 0x9e0: 0x0008, 0x9e1: 0x0008, 0x9e2: 0x0008, 0x9e3: 0x0008,
+	0x9e4: 0x0008, 0x9e5: 0x0008, 0x9e6: 0x0008, 0x9e7: 0x0008, 0x9e8: 0x0008, 0x9e9: 0x0008,
+	0x9ea: 0x0008, 0x9eb: 0x0008, 0x9ec: 0x0039, 0x9ed: 0x0ed1, 0x9ee: 0x0ee9, 0x9ef: 0x0008,
+	0x9f0: 0x0ef9, 0x9f1: 0x0f09, 0x9f2: 0x0f19, 0x9f3: 0x0f31, 0x9f4: 0x0249, 0x9f5: 0x0f41,
+	0x9f6: 0x0259, 0x9f7: 0x0f51, 0x9f8: 0x0359, 0x9f9: 0x0f61, 0x9fa: 0x0f71, 0x9fb: 0x0008,
+	0x9fc: 0x00d9, 0x9fd: 0x0f81, 0x9fe: 0x0f99, 0x9ff: 0x0269,
+	// Block 0x28, offset 0xa00
+	0xa00: 0x0fa9, 0xa01: 0x0fb9, 0xa02: 0x0279, 0xa03: 0x0039, 0xa04: 0x0fc9, 0xa05: 0x0fe1,
+	0xa06: 0x05b5, 0xa07: 0x0ee9, 0xa08: 0x0ef9, 0xa09: 0x0f09, 0xa0a: 0x0ff9, 0xa0b: 0x1011,
+	0xa0c: 0x1029, 0xa0d: 0x0f31, 0xa0e: 0x0008, 0xa0f: 0x0f51, 0xa10: 0x0f61, 0xa11: 0x1041,
+	0xa12: 0x00d9, 0xa13: 0x1059, 0xa14: 0x05cd, 0xa15: 0x05cd, 0xa16: 0x0f99, 0xa17: 0x0fa9,
+	0xa18: 0x0fb9, 0xa19: 0x05b5, 0xa1a: 0x1071, 0xa1b: 0x1089, 0xa1c: 0x05e5, 0xa1d: 0x1099,
+	0xa1e: 0x10b1, 0xa1f: 0x10c9, 0xa20: 0x10e1, 0xa21: 0x10f9, 0xa22: 0x0f41, 0xa23: 0x0269,
+	0xa24: 0x0fb9, 0xa25: 0x1089, 0xa26: 0x1099, 0xa27: 0x10b1, 0xa28: 0x1111, 0xa29: 0x10e1,
+	0xa2a: 0x10f9, 0xa2b: 0x0008, 0xa2c: 0x0008, 0xa2d: 0x0008, 0xa2e: 0x0008, 0xa2f: 0x0008,
+	0xa30: 0x0008, 0xa31: 0x0008, 0xa32: 0x0008, 0xa33: 0x0008, 0xa34: 0x0008, 0xa35: 0x0008,
+	0xa36: 0x0008, 0xa37: 0x0008, 0xa38: 0x1129, 0xa39: 0x0008, 0xa3a: 0x0008, 0xa3b: 0x0008,
+	0xa3c: 0x0008, 0xa3d: 0x0008, 0xa3e: 0x0008, 0xa3f: 0x0008,
+	// Block 0x29, offset 0xa40
+	0xa40: 0x0008, 0xa41: 0x0008, 0xa42: 0x0008, 0xa43: 0x0008, 0xa44: 0x0008, 0xa45: 0x0008,
+	0xa46: 0x0008, 0xa47: 0x0008, 0xa48: 0x0008, 0xa49: 0x0008, 0xa4a: 0x0008, 0xa4b: 0x0008,
+	0xa4c: 0x0008, 0xa4d: 0x0008, 0xa4e: 0x0008, 0xa4f: 0x0008, 0xa50: 0x0008, 0xa51: 0x0008,
+	0xa52: 0x0008, 0xa53: 0x0008, 0xa54: 0x0008, 0xa55: 0x0008, 0xa56: 0x0008, 0xa57: 0x0008,
+	0xa58: 0x0008, 0xa59: 0x0008, 0xa5a: 0x0008, 0xa5b: 0x1141, 0xa5c: 0x1159, 0xa5d: 0x1169,
+	0xa5e: 0x1181, 0xa5f: 0x1029, 0xa60: 0x1199, 0xa61: 0x11a9, 0xa62: 0x11c1, 0xa63: 0x11d9,
+	0xa64: 0x11f1, 0xa65: 0x1209, 0xa66: 0x1221, 0xa67: 0x05fd, 0xa68: 0x1239, 0xa69: 0x1251,
+	0xa6a: 0xe17d, 0xa6b: 0x1269, 0xa6c: 0x1281, 0xa6d: 0x1299, 0xa6e: 0x12b1, 0xa6f: 0x12c9,
+	0xa70: 0x12e1, 0xa71: 0x12f9, 0xa72: 0x1311, 0xa73: 0x1329, 0xa74: 0x1341, 0xa75: 0x1359,
+	0xa76: 0x1371, 0xa77: 0x1389, 0xa78: 0x0615, 0xa79: 0x13a1, 0xa7a: 0x13b9, 0xa7b: 0x13d1,
+	0xa7c: 0x13e1, 0xa7d: 0x13f9, 0xa7e: 0x1411, 0xa7f: 0x1429,
+	// Block 0x2a, offset 0xa80
+	0xa80: 0xe00d, 0xa81: 0x0008, 0xa82: 0xe00d, 0xa83: 0x0008, 0xa84: 0xe00d, 0xa85: 0x0008,
+	0xa86: 0xe00d, 0xa87: 0x0008, 0xa88: 0xe00d, 0xa89: 0x0008, 0xa8a: 0xe00d, 0xa8b: 0x0008,
+	0xa8c: 0xe00d, 0xa8d: 0x0008, 0xa8e: 0xe00d, 0xa8f: 0x0008, 0xa90: 0xe00d, 0xa91: 0x0008,
+	0xa92: 0xe00d, 0xa93: 0x0008, 0xa94: 0xe00d, 0xa95: 0x0008, 0xa96: 0xe00d, 0xa97: 0x0008,
+	0xa98: 0xe00d, 0xa99: 0x0008, 0xa9a: 0xe00d, 0xa9b: 0x0008, 0xa9c: 0xe00d, 0xa9d: 0x0008,
+	0xa9e: 0xe00d, 0xa9f: 0x0008, 0xaa0: 0xe00d, 0xaa1: 0x0008, 0xaa2: 0xe00d, 0xaa3: 0x0008,
+	0xaa4: 0xe00d, 0xaa5: 0x0008, 0xaa6: 0xe00d, 0xaa7: 0x0008, 0xaa8: 0xe00d, 0xaa9: 0x0008,
+	0xaaa: 0xe00d, 0xaab: 0x0008, 0xaac: 0xe00d, 0xaad: 0x0008, 0xaae: 0xe00d, 0xaaf: 0x0008,
+	0xab0: 0xe00d, 0xab1: 0x0008, 0xab2: 0xe00d, 0xab3: 0x0008, 0xab4: 0xe00d, 0xab5: 0x0008,
+	0xab6: 0xe00d, 0xab7: 0x0008, 0xab8: 0xe00d, 0xab9: 0x0008, 0xaba: 0xe00d, 0xabb: 0x0008,
+	0xabc: 0xe00d, 0xabd: 0x0008, 0xabe: 0xe00d, 0xabf: 0x0008,
+	// Block 0x2b, offset 0xac0
+	0xac0: 0xe00d, 0xac1: 0x0008, 0xac2: 0xe00d, 0xac3: 0x0008, 0xac4: 0xe00d, 0xac5: 0x0008,
+	0xac6: 0xe00d, 0xac7: 0x0008, 0xac8: 0xe00d, 0xac9: 0x0008, 0xaca: 0xe00d, 0xacb: 0x0008,
+	0xacc: 0xe00d, 0xacd: 0x0008, 0xace: 0xe00d, 0xacf: 0x0008, 0xad0: 0xe00d, 0xad1: 0x0008,
+	0xad2: 0xe00d, 0xad3: 0x0008, 0xad4: 0xe00d, 0xad5: 0x0008, 0xad6: 0x0008, 0xad7: 0x0008,
+	0xad8: 0x0008, 0xad9: 0x0008, 0xada: 0x062d, 0xadb: 0x064d, 0xadc: 0x0008, 0xadd: 0x0008,
+	0xade: 0x1441, 0xadf: 0x0008, 0xae0: 0xe00d, 0xae1: 0x0008, 0xae2: 0xe00d, 0xae3: 0x0008,
+	0xae4: 0xe00d, 0xae5: 0x0008, 0xae6: 0xe00d, 0xae7: 0x0008, 0xae8: 0xe00d, 0xae9: 0x0008,
+	0xaea: 0xe00d, 0xaeb: 0x0008, 0xaec: 0xe00d, 0xaed: 0x0008, 0xaee: 0xe00d, 0xaef: 0x0008,
+	0xaf0: 0xe00d, 0xaf1: 0x0008, 0xaf2: 0xe00d, 0xaf3: 0x0008, 0xaf4: 0xe00d, 0xaf5: 0x0008,
+	0xaf6: 0xe00d, 0xaf7: 0x0008, 0xaf8: 0xe00d, 0xaf9: 0x0008, 0xafa: 0xe00d, 0xafb: 0x0008,
+	0xafc: 0xe00d, 0xafd: 0x0008, 0xafe: 0xe00d, 0xaff: 0x0008,
+	// Block 0x2c, offset 0xb00
+	0xb00: 0x0008, 0xb01: 0x0008, 0xb02: 0x0008, 0xb03: 0x0008, 0xb04: 0x0008, 0xb05: 0x0008,
+	0xb06: 0x0040, 0xb07: 0x0040, 0xb08: 0xe045, 0xb09: 0xe045, 0xb0a: 0xe045, 0xb0b: 0xe045,
+	0xb0c: 0xe045, 0xb0d: 0xe045, 0xb0e: 0x0040, 0xb0f: 0x0040, 0xb10: 0x0008, 0xb11: 0x0008,
+	0xb12: 0x0008, 0xb13: 0x0008, 0xb14: 0x0008, 0xb15: 0x0008, 0xb16: 0x0008, 0xb17: 0x0008,
+	0xb18: 0x0040, 0xb19: 0xe045, 0xb1a: 0x0040, 0xb1b: 0xe045, 0xb1c: 0x0040, 0xb1d: 0xe045,
+	0xb1e: 0x0040, 0xb1f: 0xe045, 0xb20: 0x0008, 0xb21: 0x0008, 0xb22: 0x0008, 0xb23: 0x0008,
+	0xb24: 0x0008, 0xb25: 0x0008, 0xb26: 0x0008, 0xb27: 0x0008, 0xb28: 0xe045, 0xb29: 0xe045,
+	0xb2a: 0xe045, 0xb2b: 0xe045, 0xb2c: 0xe045, 0xb2d: 0xe045, 0xb2e: 0xe045, 0xb2f: 0xe045,
+	0xb30: 0x0008, 0xb31: 0x1459, 0xb32: 0x0008, 0xb33: 0x1471, 0xb34: 0x0008, 0xb35: 0x1489,
+	0xb36: 0x0008, 0xb37: 0x14a1, 0xb38: 0x0008, 0xb39: 0x14b9, 0xb3a: 0x0008, 0xb3b: 0x14d1,
+	0xb3c: 0x0008, 0xb3d: 0x14e9, 0xb3e: 0x0040, 0xb3f: 0x0040,
+	// Block 0x2d, offset 0xb40
+	0xb40: 0x1501, 0xb41: 0x1531, 0xb42: 0x1561, 0xb43: 0x1591, 0xb44: 0x15c1, 0xb45: 0x15f1,
+	0xb46: 0x1621, 0xb47: 0x1651, 0xb48: 0x1501, 0xb49: 0x1531, 0xb4a: 0x1561, 0xb4b: 0x1591,
+	0xb4c: 0x15c1, 0xb4d: 0x15f1, 0xb4e: 0x1621, 0xb4f: 0x1651, 0xb50: 0x1681, 0xb51: 0x16b1,
+	0xb52: 0x16e1, 0xb53: 0x1711, 0xb54: 0x1741, 0xb55: 0x1771, 0xb56: 0x17a1, 0xb57: 0x17d1,
+	0xb58: 0x1681, 0xb59: 0x16b1, 0xb5a: 0x16e1, 0xb5b: 0x1711, 0xb5c: 0x1741, 0xb5d: 0x1771,
+	0xb5e: 0x17a1, 0xb5f: 0x17d1, 0xb60: 0x1801, 0xb61: 0x1831, 0xb62: 0x1861, 0xb63: 0x1891,
+	0xb64: 0x18c1, 0xb65: 0x18f1, 0xb66: 0x1921, 0xb67: 0x1951, 0xb68: 0x1801, 0xb69: 0x1831,
+	0xb6a: 0x1861, 0xb6b: 0x1891, 0xb6c: 0x18c1, 0xb6d: 0x18f1, 0xb6e: 0x1921, 0xb6f: 0x1951,
+	0xb70: 0x0008, 0xb71: 0x0008, 0xb72: 0x1981, 0xb73: 0x19b1, 0xb74: 0x19d9, 0xb75: 0x0040,
+	0xb76: 0x0008, 0xb77: 0x1a01, 0xb78: 0xe045, 0xb79: 0xe045, 0xb7a: 0x0665, 0xb7b: 0x1459,
+	0xb7c: 0x19b1, 0xb7d: 0x067e, 0xb7e: 0x1a31, 0xb7f: 0x069e,
+	// Block 0x2e, offset 0xb80
+	0xb80: 0x06be, 0xb81: 0x1a4a, 0xb82: 0x1a79, 0xb83: 0x1aa9, 0xb84: 0x1ad1, 0xb85: 0x0040,
+	0xb86: 0x0008, 0xb87: 0x1af9, 0xb88: 0x06dd, 0xb89: 0x1471, 0xb8a: 0x06f5, 0xb8b: 0x1489,
+	0xb8c: 0x1aa9, 0xb8d: 0x1b2a, 0xb8e: 0x1b5a, 0xb8f: 0x1b8a, 0xb90: 0x0008, 0xb91: 0x0008,
+	0xb92: 0x0008, 0xb93: 0x1bb9, 0xb94: 0x0040, 0xb95: 0x0040, 0xb96: 0x0008, 0xb97: 0x0008,
+	0xb98: 0xe045, 0xb99: 0xe045, 0xb9a: 0x070d, 0xb9b: 0x14a1, 0xb9c: 0x0040, 0xb9d: 0x1bd2,
+	0xb9e: 0x1c02, 0xb9f: 0x1c32, 0xba0: 0x0008, 0xba1: 0x0008, 0xba2: 0x0008, 0xba3: 0x1c61,
+	0xba4: 0x0008, 0xba5: 0x0008, 0xba6: 0x0008, 0xba7: 0x0008, 0xba8: 0xe045, 0xba9: 0xe045,
+	0xbaa: 0x0725, 0xbab: 0x14d1, 0xbac: 0xe04d, 0xbad: 0x1c7a, 0xbae: 0x03d2, 0xbaf: 0x1caa,
+	0xbb0: 0x0040, 0xbb1: 0x0040, 0xbb2: 0x1cb9, 0xbb3: 0x1ce9, 0xbb4: 0x1d11, 0xbb5: 0x0040,
+	0xbb6: 0x0008, 0xbb7: 0x1d39, 0xbb8: 0x073d, 0xbb9: 0x14b9, 0xbba: 0x0515, 0xbbb: 0x14e9,
+	0xbbc: 0x1ce9, 0xbbd: 0x0756, 0xbbe: 0x0776, 0xbbf: 0x0040,
+	// Block 0x2f, offset 0xbc0
+	0xbc0: 0x000a, 0xbc1: 0x000a, 0xbc2: 0x000a, 0xbc3: 0x000a, 0xbc4: 0x000a, 0xbc5: 0x000a,
+	0xbc6: 0x000a, 0xbc7: 0x000a, 0xbc8: 0x000a, 0xbc9: 0x000a, 0xbca: 0x000a, 0xbcb: 0x03c0,
+	0xbcc: 0x0003, 0xbcd: 0x0003, 0xbce: 0x0340, 0xbcf: 0x0b40, 0xbd0: 0x0018, 0xbd1: 0xe00d,
+	0xbd2: 0x0018, 0xbd3: 0x0018, 0xbd4: 0x0018, 0xbd5: 0x0018, 0xbd6: 0x0018, 0xbd7: 0x0796,
+	0xbd8: 0x0018, 0xbd9: 0x0018, 0xbda: 0x0018, 0xbdb: 0x0018, 0xbdc: 0x0018, 0xbdd: 0x0018,
+	0xbde: 0x0018, 0xbdf: 0x0018, 0xbe0: 0x0018, 0xbe1: 0x0018, 0xbe2: 0x0018, 0xbe3: 0x0018,
+	0xbe4: 0x0040, 0xbe5: 0x0040, 0xbe6: 0x0040, 0xbe7: 0x0018, 0xbe8: 0x0040, 0xbe9: 0x0040,
+	0xbea: 0x0340, 0xbeb: 0x0340, 0xbec: 0x0340, 0xbed: 0x0340, 0xbee: 0x0340, 0xbef: 0x000a,
+	0xbf0: 0x0018, 0xbf1: 0x0018, 0xbf2: 0x0018, 0xbf3: 0x1d69, 0xbf4: 0x1da1, 0xbf5: 0x0018,
+	0xbf6: 0x1df1, 0xbf7: 0x1e29, 0xbf8: 0x0018, 0xbf9: 0x0018, 0xbfa: 0x0018, 0xbfb: 0x0018,
+	0xbfc: 0x1e7a, 0xbfd: 0x0018, 0xbfe: 0x07b6, 0xbff: 0x0018,
+	// Block 0x30, offset 0xc00
+	0xc00: 0x0018, 0xc01: 0x0018, 0xc02: 0x0018, 0xc03: 0x0018, 0xc04: 0x0018, 0xc05: 0x0018,
+	0xc06: 0x0018, 0xc07: 0x1e92, 0xc08: 0x1eaa, 0xc09: 0x1ec2, 0xc0a: 0x0018, 0xc0b: 0x0018,
+	0xc0c: 0x0018, 0xc0d: 0x0018, 0xc0e: 0x0018, 0xc0f: 0x0018, 0xc10: 0x0018, 0xc11: 0x0018,
+	0xc12: 0x0018, 0xc13: 0x0018, 0xc14: 0x0018, 0xc15: 0x0018, 0xc16: 0x0018, 0xc17: 0x1ed9,
+	0xc18: 0x0018, 0xc19: 0x0018, 0xc1a: 0x0018, 0xc1b: 0x0018, 0xc1c: 0x0018, 0xc1d: 0x0018,
+	0xc1e: 0x0018, 0xc1f: 0x000a, 0xc20: 0x03c0, 0xc21: 0x0340, 0xc22: 0x0340, 0xc23: 0x0340,
+	0xc24: 0x03c0, 0xc25: 0x0040, 0xc26: 0x0040, 0xc27: 0x0040, 0xc28: 0x0040, 0xc29: 0x0040,
+	0xc2a: 0x0340, 0xc2b: 0x0340, 0xc2c: 0x0340, 0xc2d: 0x0340, 0xc2e: 0x0340, 0xc2f: 0x0340,
+	0xc30: 0x1f41, 0xc31: 0x0f41, 0xc32: 0x0040, 0xc33: 0x0040, 0xc34: 0x1f51, 0xc35: 0x1f61,
+	0xc36: 0x1f71, 0xc37: 0x1f81, 0xc38: 0x1f91, 0xc39: 0x1fa1, 0xc3a: 0x1fb2, 0xc3b: 0x07d5,
+	0xc3c: 0x1fc2, 0xc3d: 0x1fd2, 0xc3e: 0x1fe2, 0xc3f: 0x0f71,
+	// Block 0x31, offset 0xc40
+	0xc40: 0x1f41, 0xc41: 0x00c9, 0xc42: 0x0069, 0xc43: 0x0079, 0xc44: 0x1f51, 0xc45: 0x1f61,
+	0xc46: 0x1f71, 0xc47: 0x1f81, 0xc48: 0x1f91, 0xc49: 0x1fa1, 0xc4a: 0x1fb2, 0xc4b: 0x07ed,
+	0xc4c: 0x1fc2, 0xc4d: 0x1fd2, 0xc4e: 0x1fe2, 0xc4f: 0x0040, 0xc50: 0x0039, 0xc51: 0x0f09,
+	0xc52: 0x00d9, 0xc53: 0x0369, 0xc54: 0x0ff9, 0xc55: 0x0249, 0xc56: 0x0f51, 0xc57: 0x0359,
+	0xc58: 0x0f61, 0xc59: 0x0f71, 0xc5a: 0x0f99, 0xc5b: 0x01d9, 0xc5c: 0x0fa9, 0xc5d: 0x0040,
+	0xc5e: 0x0040, 0xc5f: 0x0040, 0xc60: 0x0018, 0xc61: 0x0018, 0xc62: 0x0018, 0xc63: 0x0018,
+	0xc64: 0x0018, 0xc65: 0x0018, 0xc66: 0x0018, 0xc67: 0x0018, 0xc68: 0x1ff1, 0xc69: 0x0018,
+	0xc6a: 0x0018, 0xc6b: 0x0018, 0xc6c: 0x0018, 0xc6d: 0x0018, 0xc6e: 0x0018, 0xc6f: 0x0018,
+	0xc70: 0x0018, 0xc71: 0x0018, 0xc72: 0x0018, 0xc73: 0x0018, 0xc74: 0x0018, 0xc75: 0x0018,
+	0xc76: 0x0018, 0xc77: 0x0018, 0xc78: 0x0018, 0xc79: 0x0018, 0xc7a: 0x0018, 0xc7b: 0x0018,
+	0xc7c: 0x0018, 0xc7d: 0x0018, 0xc7e: 0x0018, 0xc7f: 0x0018,
+	// Block 0x32, offset 0xc80
+	0xc80: 0x0806, 0xc81: 0x0826, 0xc82: 0x1159, 0xc83: 0x0845, 0xc84: 0x0018, 0xc85: 0x0866,
+	0xc86: 0x0886, 0xc87: 0x1011, 0xc88: 0x0018, 0xc89: 0x08a5, 0xc8a: 0x0f31, 0xc8b: 0x0249,
+	0xc8c: 0x0249, 0xc8d: 0x0249, 0xc8e: 0x0249, 0xc8f: 0x2009, 0xc90: 0x0f41, 0xc91: 0x0f41,
+	0xc92: 0x0359, 0xc93: 0x0359, 0xc94: 0x0018, 0xc95: 0x0f71, 0xc96: 0x2021, 0xc97: 0x0018,
+	0xc98: 0x0018, 0xc99: 0x0f99, 0xc9a: 0x2039, 0xc9b: 0x0269, 0xc9c: 0x0269, 0xc9d: 0x0269,
+	0xc9e: 0x0018, 0xc9f: 0x0018, 0xca0: 0x2049, 0xca1: 0x08c5, 0xca2: 0x2061, 0xca3: 0x0018,
+	0xca4: 0x13d1, 0xca5: 0x0018, 0xca6: 0x2079, 0xca7: 0x0018, 0xca8: 0x13d1, 0xca9: 0x0018,
+	0xcaa: 0x0f51, 0xcab: 0x2091, 0xcac: 0x0ee9, 0xcad: 0x1159, 0xcae: 0x0018, 0xcaf: 0x0f09,
+	0xcb0: 0x0f09, 0xcb1: 0x1199, 0xcb2: 0x0040, 0xcb3: 0x0f61, 0xcb4: 0x00d9, 0xcb5: 0x20a9,
+	0xcb6: 0x20c1, 0xcb7: 0x20d9, 0xcb8: 0x20f1, 0xcb9: 0x0f41, 0xcba: 0x0018, 0xcbb: 0x08e5,
+	0xcbc: 0x2109, 0xcbd: 0x10b1, 0xcbe: 0x10b1, 0xcbf: 0x2109,
+	// Block 0x33, offset 0xcc0
+	0xcc0: 0x0905, 0xcc1: 0x0018, 0xcc2: 0x0018, 0xcc3: 0x0018, 0xcc4: 0x0018, 0xcc5: 0x0ef9,
+	0xcc6: 0x0ef9, 0xcc7: 0x0f09, 0xcc8: 0x0f41, 0xcc9: 0x0259, 0xcca: 0x0018, 0xccb: 0x0018,
+	0xccc: 0x0018, 0xccd: 0x0018, 0xcce: 0x0008, 0xccf: 0x0018, 0xcd0: 0x2121, 0xcd1: 0x2151,
+	0xcd2: 0x2181, 0xcd3: 0x21b9, 0xcd4: 0x21e9, 0xcd5: 0x2219, 0xcd6: 0x2249, 0xcd7: 0x2279,
+	0xcd8: 0x22a9, 0xcd9: 0x22d9, 0xcda: 0x2309, 0xcdb: 0x2339, 0xcdc: 0x2369, 0xcdd: 0x2399,
+	0xcde: 0x23c9, 0xcdf: 0x23f9, 0xce0: 0x0f41, 0xce1: 0x2421, 0xce2: 0x091d, 0xce3: 0x2439,
+	0xce4: 0x1089, 0xce5: 0x2451, 0xce6: 0x093d, 0xce7: 0x2469, 0xce8: 0x2491, 0xce9: 0x0369,
+	0xcea: 0x24a9, 0xceb: 0x095d, 0xcec: 0x0359, 0xced: 0x1159, 0xcee: 0x0ef9, 0xcef: 0x0f61,
+	0xcf0: 0x0f41, 0xcf1: 0x2421, 0xcf2: 0x097d, 0xcf3: 0x2439, 0xcf4: 0x1089, 0xcf5: 0x2451,
+	0xcf6: 0x099d, 0xcf7: 0x2469, 0xcf8: 0x2491, 0xcf9: 0x0369, 0xcfa: 0x24a9, 0xcfb: 0x09bd,
+	0xcfc: 0x0359, 0xcfd: 0x1159, 0xcfe: 0x0ef9, 0xcff: 0x0f61,
+	// Block 0x34, offset 0xd00
+	0xd00: 0x0018, 0xd01: 0x0018, 0xd02: 0x0018, 0xd03: 0x0018, 0xd04: 0x0018, 0xd05: 0x0018,
+	0xd06: 0x0018, 0xd07: 0x0018, 0xd08: 0x0018, 0xd09: 0x0018, 0xd0a: 0x0018, 0xd0b: 0x0040,
+	0xd0c: 0x0040, 0xd0d: 0x0040, 0xd0e: 0x0040, 0xd0f: 0x0040, 0xd10: 0x0040, 0xd11: 0x0040,
+	0xd12: 0x0040, 0xd13: 0x0040, 0xd14: 0x0040, 0xd15: 0x0040, 0xd16: 0x0040, 0xd17: 0x0040,
+	0xd18: 0x0040, 0xd19: 0x0040, 0xd1a: 0x0040, 0xd1b: 0x0040, 0xd1c: 0x0040, 0xd1d: 0x0040,
+	0xd1e: 0x0040, 0xd1f: 0x0040, 0xd20: 0x00c9, 0xd21: 0x0069, 0xd22: 0x0079, 0xd23: 0x1f51,
+	0xd24: 0x1f61, 0xd25: 0x1f71, 0xd26: 0x1f81, 0xd27: 0x1f91, 0xd28: 0x1fa1, 0xd29: 0x2601,
+	0xd2a: 0x2619, 0xd2b: 0x2631, 0xd2c: 0x2649, 0xd2d: 0x2661, 0xd2e: 0x2679, 0xd2f: 0x2691,
+	0xd30: 0x26a9, 0xd31: 0x26c1, 0xd32: 0x26d9, 0xd33: 0x26f1, 0xd34: 0x0a1e, 0xd35: 0x0a3e,
+	0xd36: 0x0a5e, 0xd37: 0x0a7e, 0xd38: 0x0a9e, 0xd39: 0x0abe, 0xd3a: 0x0ade, 0xd3b: 0x0afe,
+	0xd3c: 0x0b1e, 0xd3d: 0x270a, 0xd3e: 0x2732, 0xd3f: 0x275a,
+	// Block 0x35, offset 0xd40
+	0xd40: 0x2782, 0xd41: 0x27aa, 0xd42: 0x27d2, 0xd43: 0x27fa, 0xd44: 0x2822, 0xd45: 0x284a,
+	0xd46: 0x2872, 0xd47: 0x289a, 0xd48: 0x0040, 0xd49: 0x0040, 0xd4a: 0x0040, 0xd4b: 0x0040,
+	0xd4c: 0x0040, 0xd4d: 0x0040, 0xd4e: 0x0040, 0xd4f: 0x0040, 0xd50: 0x0040, 0xd51: 0x0040,
+	0xd52: 0x0040, 0xd53: 0x0040, 0xd54: 0x0040, 0xd55: 0x0040, 0xd56: 0x0040, 0xd57: 0x0040,
+	0xd58: 0x0040, 0xd59: 0x0040, 0xd5a: 0x0040, 0xd5b: 0x0040, 0xd5c: 0x0b3e, 0xd5d: 0x0b5e,
+	0xd5e: 0x0b7e, 0xd5f: 0x0b9e, 0xd60: 0x0bbe, 0xd61: 0x0bde, 0xd62: 0x0bfe, 0xd63: 0x0c1e,
+	0xd64: 0x0c3e, 0xd65: 0x0c5e, 0xd66: 0x0c7e, 0xd67: 0x0c9e, 0xd68: 0x0cbe, 0xd69: 0x0cde,
+	0xd6a: 0x0cfe, 0xd6b: 0x0d1e, 0xd6c: 0x0d3e, 0xd6d: 0x0d5e, 0xd6e: 0x0d7e, 0xd6f: 0x0d9e,
+	0xd70: 0x0dbe, 0xd71: 0x0dde, 0xd72: 0x0dfe, 0xd73: 0x0e1e, 0xd74: 0x0e3e, 0xd75: 0x0e5e,
+	0xd76: 0x0039, 0xd77: 0x0ee9, 0xd78: 0x1159, 0xd79: 0x0ef9, 0xd7a: 0x0f09, 0xd7b: 0x1199,
+	0xd7c: 0x0f31, 0xd7d: 0x0249, 0xd7e: 0x0f41, 0xd7f: 0x0259,
+	// Block 0x36, offset 0xd80
+	0xd80: 0x0f51, 0xd81: 0x0359, 0xd82: 0x0f61, 0xd83: 0x0f71, 0xd84: 0x00d9, 0xd85: 0x0f99,
+	0xd86: 0x2039, 0xd87: 0x0269, 0xd88: 0x01d9, 0xd89: 0x0fa9, 0xd8a: 0x0fb9, 0xd8b: 0x1089,
+	0xd8c: 0x0279, 0xd8d: 0x0369, 0xd8e: 0x0289, 0xd8f: 0x13d1, 0xd90: 0x0039, 0xd91: 0x0ee9,
+	0xd92: 0x1159, 0xd93: 0x0ef9, 0xd94: 0x0f09, 0xd95: 0x1199, 0xd96: 0x0f31, 0xd97: 0x0249,
+	0xd98: 0x0f41, 0xd99: 0x0259, 0xd9a: 0x0f51, 0xd9b: 0x0359, 0xd9c: 0x0f61, 0xd9d: 0x0f71,
+	0xd9e: 0x00d9, 0xd9f: 0x0f99, 0xda0: 0x2039, 0xda1: 0x0269, 0xda2: 0x01d9, 0xda3: 0x0fa9,
+	0xda4: 0x0fb9, 0xda5: 0x1089, 0xda6: 0x0279, 0xda7: 0x0369, 0xda8: 0x0289, 0xda9: 0x13d1,
+	0xdaa: 0x1f41, 0xdab: 0x0018, 0xdac: 0x0018, 0xdad: 0x0018, 0xdae: 0x0018, 0xdaf: 0x0018,
+	0xdb0: 0x0018, 0xdb1: 0x0018, 0xdb2: 0x0018, 0xdb3: 0x0018, 0xdb4: 0x0018, 0xdb5: 0x0018,
+	0xdb6: 0x0018, 0xdb7: 0x0018, 0xdb8: 0x0018, 0xdb9: 0x0018, 0xdba: 0x0018, 0xdbb: 0x0018,
+	0xdbc: 0x0018, 0xdbd: 0x0018, 0xdbe: 0x0018, 0xdbf: 0x0018,
+	// Block 0x37, offset 0xdc0
+	0xdc0: 0x0008, 0xdc1: 0x0008, 0xdc2: 0x0008, 0xdc3: 0x0008, 0xdc4: 0x0008, 0xdc5: 0x0008,
+	0xdc6: 0x0008, 0xdc7: 0x0008, 0xdc8: 0x0008, 0xdc9: 0x0008, 0xdca: 0x0008, 0xdcb: 0x0008,
+	0xdcc: 0x0008, 0xdcd: 0x0008, 0xdce: 0x0008, 0xdcf: 0x0008, 0xdd0: 0x0008, 0xdd1: 0x0008,
+	0xdd2: 0x0008, 0xdd3: 0x0008, 0xdd4: 0x0008, 0xdd5: 0x0008, 0xdd6: 0x0008, 0xdd7: 0x0008,
+	0xdd8: 0x0008, 0xdd9: 0x0008, 0xdda: 0x0008, 0xddb: 0x0008, 0xddc: 0x0008, 0xddd: 0x0008,
+	0xdde: 0x0008, 0xddf: 0x0040, 0xde0: 0xe00d, 0xde1: 0x0008, 0xde2: 0x2971, 0xde3: 0x0ed5,
+	0xde4: 0x2989, 0xde5: 0x0008, 0xde6: 0x0008, 0xde7: 0xe07d, 0xde8: 0x0008, 0xde9: 0xe01d,
+	0xdea: 0x0008, 0xdeb: 0xe03d, 0xdec: 0x0008, 0xded: 0x0fe1, 0xdee: 0x1281, 0xdef: 0x0fc9,
+	0xdf0: 0x1141, 0xdf1: 0x0008, 0xdf2: 0xe00d, 0xdf3: 0x0008, 0xdf4: 0x0008, 0xdf5: 0xe01d,
+	0xdf6: 0x0008, 0xdf7: 0x0008, 0xdf8: 0x0008, 0xdf9: 0x0008, 0xdfa: 0x0008, 0xdfb: 0x0008,
+	0xdfc: 0x0259, 0xdfd: 0x1089, 0xdfe: 0x29a1, 0xdff: 0x29b9,
+	// Block 0x38, offset 0xe00
+	0xe00: 0xe00d, 0xe01: 0x0008, 0xe02: 0xe00d, 0xe03: 0x0008, 0xe04: 0xe00d, 0xe05: 0x0008,
+	0xe06: 0xe00d, 0xe07: 0x0008, 0xe08: 0xe00d, 0xe09: 0x0008, 0xe0a: 0xe00d, 0xe0b: 0x0008,
+	0xe0c: 0xe00d, 0xe0d: 0x0008, 0xe0e: 0xe00d, 0xe0f: 0x0008, 0xe10: 0xe00d, 0xe11: 0x0008,
+	0xe12: 0xe00d, 0xe13: 0x0008, 0xe14: 0xe00d, 0xe15: 0x0008, 0xe16: 0xe00d, 0xe17: 0x0008,
+	0xe18: 0xe00d, 0xe19: 0x0008, 0xe1a: 0xe00d, 0xe1b: 0x0008, 0xe1c: 0xe00d, 0xe1d: 0x0008,
+	0xe1e: 0xe00d, 0xe1f: 0x0008, 0xe20: 0xe00d, 0xe21: 0x0008, 0xe22: 0xe00d, 0xe23: 0x0008,
+	0xe24: 0x0008, 0xe25: 0x0018, 0xe26: 0x0018, 0xe27: 0x0018, 0xe28: 0x0018, 0xe29: 0x0018,
+	0xe2a: 0x0018, 0xe2b: 0xe03d, 0xe2c: 0x0008, 0xe2d: 0xe01d, 0xe2e: 0x0008, 0xe2f: 0x3308,
+	0xe30: 0x3308, 0xe31: 0x3308, 0xe32: 0xe00d, 0xe33: 0x0008, 0xe34: 0x0040, 0xe35: 0x0040,
+	0xe36: 0x0040, 0xe37: 0x0040, 0xe38: 0x0040, 0xe39: 0x0018, 0xe3a: 0x0018, 0xe3b: 0x0018,
+	0xe3c: 0x0018, 0xe3d: 0x0018, 0xe3e: 0x0018, 0xe3f: 0x0018,
+	// Block 0x39, offset 0xe40
+	0xe40: 0x2715, 0xe41: 0x2735, 0xe42: 0x2755, 0xe43: 0x2775, 0xe44: 0x2795, 0xe45: 0x27b5,
+	0xe46: 0x27d5, 0xe47: 0x27f5, 0xe48: 0x2815, 0xe49: 0x2835, 0xe4a: 0x2855, 0xe4b: 0x2875,
+	0xe4c: 0x2895, 0xe4d: 0x28b5, 0xe4e: 0x28d5, 0xe4f: 0x28f5, 0xe50: 0x2915, 0xe51: 0x2935,
+	0xe52: 0x2955, 0xe53: 0x2975, 0xe54: 0x2995, 0xe55: 0x29b5, 0xe56: 0x0040, 0xe57: 0x0040,
+	0xe58: 0x0040, 0xe59: 0x0040, 0xe5a: 0x0040, 0xe5b: 0x0040, 0xe5c: 0x0040, 0xe5d: 0x0040,
+	0xe5e: 0x0040, 0xe5f: 0x0040, 0xe60: 0x0040, 0xe61: 0x0040, 0xe62: 0x0040, 0xe63: 0x0040,
+	0xe64: 0x0040, 0xe65: 0x0040, 0xe66: 0x0040, 0xe67: 0x0040, 0xe68: 0x0040, 0xe69: 0x0040,
+	0xe6a: 0x0040, 0xe6b: 0x0040, 0xe6c: 0x0040, 0xe6d: 0x0040, 0xe6e: 0x0040, 0xe6f: 0x0040,
+	0xe70: 0x0040, 0xe71: 0x0040, 0xe72: 0x0040, 0xe73: 0x0040, 0xe74: 0x0040, 0xe75: 0x0040,
+	0xe76: 0x0040, 0xe77: 0x0040, 0xe78: 0x0040, 0xe79: 0x0040, 0xe7a: 0x0040, 0xe7b: 0x0040,
+	0xe7c: 0x0040, 0xe7d: 0x0040, 0xe7e: 0x0040, 0xe7f: 0x0040,
+	// Block 0x3a, offset 0xe80
+	0xe80: 0x000a, 0xe81: 0x0018, 0xe82: 0x29d1, 0xe83: 0x0018, 0xe84: 0x0018, 0xe85: 0x0008,
+	0xe86: 0x0008, 0xe87: 0x0008, 0xe88: 0x0018, 0xe89: 0x0018, 0xe8a: 0x0018, 0xe8b: 0x0018,
+	0xe8c: 0x0018, 0xe8d: 0x0018, 0xe8e: 0x0018, 0xe8f: 0x0018, 0xe90: 0x0018, 0xe91: 0x0018,
+	0xe92: 0x0018, 0xe93: 0x0018, 0xe94: 0x0018, 0xe95: 0x0018, 0xe96: 0x0018, 0xe97: 0x0018,
+	0xe98: 0x0018, 0xe99: 0x0018, 0xe9a: 0x0018, 0xe9b: 0x0018, 0xe9c: 0x0018, 0xe9d: 0x0018,
+	0xe9e: 0x0018, 0xe9f: 0x0018, 0xea0: 0x0018, 0xea1: 0x0018, 0xea2: 0x0018, 0xea3: 0x0018,
+	0xea4: 0x0018, 0xea5: 0x0018, 0xea6: 0x0018, 0xea7: 0x0018, 0xea8: 0x0018, 0xea9: 0x0018,
+	0xeaa: 0x3308, 0xeab: 0x3308, 0xeac: 0x3308, 0xead: 0x3308, 0xeae: 0x3018, 0xeaf: 0x3018,
+	0xeb0: 0x0018, 0xeb1: 0x0018, 0xeb2: 0x0018, 0xeb3: 0x0018, 0xeb4: 0x0018, 0xeb5: 0x0018,
+	0xeb6: 0xe125, 0xeb7: 0x0018, 0xeb8: 0x29d5, 0xeb9: 0x29f5, 0xeba: 0x2a15, 0xebb: 0x0018,
+	0xebc: 0x0008, 0xebd: 0x0018, 0xebe: 0x0018, 0xebf: 0x0018,
+	// Block 0x3b, offset 0xec0
+	0xec0: 0x2b55, 0xec1: 0x2b75, 0xec2: 0x2b95, 0xec3: 0x2bb5, 0xec4: 0x2bd5, 0xec5: 0x2bf5,
+	0xec6: 0x2bf5, 0xec7: 0x2bf5, 0xec8: 0x2c15, 0xec9: 0x2c15, 0xeca: 0x2c15, 0xecb: 0x2c15,
+	0xecc: 0x2c35, 0xecd: 0x2c35, 0xece: 0x2c35, 0xecf: 0x2c55, 0xed0: 0x2c75, 0xed1: 0x2c75,
+	0xed2: 0x2a95, 0xed3: 0x2a95, 0xed4: 0x2c75, 0xed5: 0x2c75, 0xed6: 0x2c95, 0xed7: 0x2c95,
+	0xed8: 0x2c75, 0xed9: 0x2c75, 0xeda: 0x2a95, 0xedb: 0x2a95, 0xedc: 0x2c75, 0xedd: 0x2c75,
+	0xede: 0x2c55, 0xedf: 0x2c55, 0xee0: 0x2cb5, 0xee1: 0x2cb5, 0xee2: 0x2cd5, 0xee3: 0x2cd5,
+	0xee4: 0x0040, 0xee5: 0x2cf5, 0xee6: 0x2d15, 0xee7: 0x2d35, 0xee8: 0x2d35, 0xee9: 0x2d55,
+	0xeea: 0x2d75, 0xeeb: 0x2d95, 0xeec: 0x2db5, 0xeed: 0x2dd5, 0xeee: 0x2df5, 0xeef: 0x2e15,
+	0xef0: 0x2e35, 0xef1: 0x2e55, 0xef2: 0x2e55, 0xef3: 0x2e75, 0xef4: 0x2e95, 0xef5: 0x2e95,
+	0xef6: 0x2eb5, 0xef7: 0x2ed5, 0xef8: 0x2e75, 0xef9: 0x2ef5, 0xefa: 0x2f15, 0xefb: 0x2ef5,
+	0xefc: 0x2e75, 0xefd: 0x2f35, 0xefe: 0x2f55, 0xeff: 0x2f75,
+	// Block 0x3c, offset 0xf00
+	0xf00: 0x2f95, 0xf01: 0x2fb5, 0xf02: 0x2d15, 0xf03: 0x2cf5, 0xf04: 0x2fd5, 0xf05: 0x2ff5,
+	0xf06: 0x3015, 0xf07: 0x3035, 0xf08: 0x3055, 0xf09: 0x3075, 0xf0a: 0x3095, 0xf0b: 0x30b5,
+	0xf0c: 0x30d5, 0xf0d: 0x30f5, 0xf0e: 0x3115, 0xf0f: 0x0040, 0xf10: 0x0018, 0xf11: 0x0018,
+	0xf12: 0x3135, 0xf13: 0x3155, 0xf14: 0x3175, 0xf15: 0x3195, 0xf16: 0x31b5, 0xf17: 0x31d5,
+	0xf18: 0x31f5, 0xf19: 0x3215, 0xf1a: 0x3235, 0xf1b: 0x3255, 0xf1c: 0x3175, 0xf1d: 0x3275,
+	0xf1e: 0x3295, 0xf1f: 0x32b5, 0xf20: 0x0008, 0xf21: 0x0008, 0xf22: 0x0008, 0xf23: 0x0008,
+	0xf24: 0x0008, 0xf25: 0x0008, 0xf26: 0x0008, 0xf27: 0x0008, 0xf28: 0x0008, 0xf29: 0x0008,
+	0xf2a: 0x0008, 0xf2b: 0x0008, 0xf2c: 0x0008, 0xf2d: 0x0008, 0xf2e: 0x0008, 0xf2f: 0x0008,
+	0xf30: 0x0008, 0xf31: 0x0008, 0xf32: 0x0008, 0xf33: 0x0008, 0xf34: 0x0008, 0xf35: 0x0008,
+	0xf36: 0x0008, 0xf37: 0x0008, 0xf38: 0x0008, 0xf39: 0x0008, 0xf3a: 0x0008, 0xf3b: 0x0008,
+	0xf3c: 0x0008, 0xf3d: 0x0008, 0xf3e: 0x0008, 0xf3f: 0x0008,
+	// Block 0x3d, offset 0xf40
+	0xf40: 0x36a2, 0xf41: 0x36d2, 0xf42: 0x3702, 0xf43: 0x3732, 0xf44: 0x32d5, 0xf45: 0x32f5,
+	0xf46: 0x3315, 0xf47: 0x3335, 0xf48: 0x0018, 0xf49: 0x0018, 0xf4a: 0x0018, 0xf4b: 0x0018,
+	0xf4c: 0x0018, 0xf4d: 0x0018, 0xf4e: 0x0018, 0xf4f: 0x0018, 0xf50: 0x3355, 0xf51: 0x3761,
+	0xf52: 0x3779, 0xf53: 0x3791, 0xf54: 0x37a9, 0xf55: 0x37c1, 0xf56: 0x37d9, 0xf57: 0x37f1,
+	0xf58: 0x3809, 0xf59: 0x3821, 0xf5a: 0x3839, 0xf5b: 0x3851, 0xf5c: 0x3869, 0xf5d: 0x3881,
+	0xf5e: 0x3899, 0xf5f: 0x38b1, 0xf60: 0x3375, 0xf61: 0x3395, 0xf62: 0x33b5, 0xf63: 0x33d5,
+	0xf64: 0x33f5, 0xf65: 0x33f5, 0xf66: 0x3415, 0xf67: 0x3435, 0xf68: 0x3455, 0xf69: 0x3475,
+	0xf6a: 0x3495, 0xf6b: 0x34b5, 0xf6c: 0x34d5, 0xf6d: 0x34f5, 0xf6e: 0x3515, 0xf6f: 0x3535,
+	0xf70: 0x3555, 0xf71: 0x3575, 0xf72: 0x3595, 0xf73: 0x35b5, 0xf74: 0x35d5, 0xf75: 0x35f5,
+	0xf76: 0x3615, 0xf77: 0x3635, 0xf78: 0x3655, 0xf79: 0x3675, 0xf7a: 0x3695, 0xf7b: 0x36b5,
+	0xf7c: 0x38c9, 0xf7d: 0x3901, 0xf7e: 0x36d5, 0xf7f: 0x0018,
+	// Block 0x3e, offset 0xf80
+	0xf80: 0x36f5, 0xf81: 0x3715, 0xf82: 0x3735, 0xf83: 0x3755, 0xf84: 0x3775, 0xf85: 0x3795,
+	0xf86: 0x37b5, 0xf87: 0x37d5, 0xf88: 0x37f5, 0xf89: 0x3815, 0xf8a: 0x3835, 0xf8b: 0x3855,
+	0xf8c: 0x3875, 0xf8d: 0x3895, 0xf8e: 0x38b5, 0xf8f: 0x38d5, 0xf90: 0x38f5, 0xf91: 0x3915,
+	0xf92: 0x3935, 0xf93: 0x3955, 0xf94: 0x3975, 0xf95: 0x3995, 0xf96: 0x39b5, 0xf97: 0x39d5,
+	0xf98: 0x39f5, 0xf99: 0x3a15, 0xf9a: 0x3a35, 0xf9b: 0x3a55, 0xf9c: 0x3a75, 0xf9d: 0x3a95,
+	0xf9e: 0x3ab5, 0xf9f: 0x3ad5, 0xfa0: 0x3af5, 0xfa1: 0x3b15, 0xfa2: 0x3b35, 0xfa3: 0x3b55,
+	0xfa4: 0x3b75, 0xfa5: 0x3b95, 0xfa6: 0x1295, 0xfa7: 0x3bb5, 0xfa8: 0x3bd5, 0xfa9: 0x3bf5,
+	0xfaa: 0x3c15, 0xfab: 0x3c35, 0xfac: 0x3c55, 0xfad: 0x3c75, 0xfae: 0x23b5, 0xfaf: 0x3c95,
+	0xfb0: 0x3cb5, 0xfb1: 0x3939, 0xfb2: 0x3951, 0xfb3: 0x3969, 0xfb4: 0x3981, 0xfb5: 0x3999,
+	0xfb6: 0x39b1, 0xfb7: 0x39c9, 0xfb8: 0x39e1, 0xfb9: 0x39f9, 0xfba: 0x3a11, 0xfbb: 0x3a29,
+	0xfbc: 0x3a41, 0xfbd: 0x3a59, 0xfbe: 0x3a71, 0xfbf: 0x3a89,
+	// Block 0x3f, offset 0xfc0
+	0xfc0: 0x3aa1, 0xfc1: 0x3ac9, 0xfc2: 0x3af1, 0xfc3: 0x3b19, 0xfc4: 0x3b41, 0xfc5: 0x3b69,
+	0xfc6: 0x3b91, 0xfc7: 0x3bb9, 0xfc8: 0x3be1, 0xfc9: 0x3c09, 0xfca: 0x3c39, 0xfcb: 0x3c69,
+	0xfcc: 0x3c99, 0xfcd: 0x3cd5, 0xfce: 0x3cb1, 0xfcf: 0x3cf5, 0xfd0: 0x3d15, 0xfd1: 0x3d2d,
+	0xfd2: 0x3d45, 0xfd3: 0x3d5d, 0xfd4: 0x3d75, 0xfd5: 0x3d75, 0xfd6: 0x3d5d, 0xfd7: 0x3d8d,
+	0xfd8: 0x07d5, 0xfd9: 0x3da5, 0xfda: 0x3dbd, 0xfdb: 0x3dd5, 0xfdc: 0x3ded, 0xfdd: 0x3e05,
+	0xfde: 0x3e1d, 0xfdf: 0x3e35, 0xfe0: 0x3e4d, 0xfe1: 0x3e65, 0xfe2: 0x3e7d, 0xfe3: 0x3e95,
+	0xfe4: 0x3ead, 0xfe5: 0x3ead, 0xfe6: 0x3ec5, 0xfe7: 0x3ec5, 0xfe8: 0x3edd, 0xfe9: 0x3edd,
+	0xfea: 0x3ef5, 0xfeb: 0x3f0d, 0xfec: 0x3f25, 0xfed: 0x3f3d, 0xfee: 0x3f55, 0xfef: 0x3f55,
+	0xff0: 0x3f6d, 0xff1: 0x3f6d, 0xff2: 0x3f6d, 0xff3: 0x3f85, 0xff4: 0x3f9d, 0xff5: 0x3fb5,
+	0xff6: 0x3fcd, 0xff7: 0x3fb5, 0xff8: 0x3fe5, 0xff9: 0x3ffd, 0xffa: 0x3f85, 0xffb: 0x4015,
+	0xffc: 0x402d, 0xffd: 0x402d, 0xffe: 0x402d, 0xfff: 0x3cc9,
+	// Block 0x40, offset 0x1000
+	0x1000: 0x3d01, 0x1001: 0x3d69, 0x1002: 0x3dd1, 0x1003: 0x3e39, 0x1004: 0x3e89, 0x1005: 0x3ef1,
+	0x1006: 0x3f41, 0x1007: 0x3f91, 0x1008: 0x4011, 0x1009: 0x4079, 0x100a: 0x40c9, 0x100b: 0x4119,
+	0x100c: 0x4169, 0x100d: 0x41d1, 0x100e: 0x4239, 0x100f: 0x4289, 0x1010: 0x42d9, 0x1011: 0x4311,
+	0x1012: 0x4361, 0x1013: 0x43c9, 0x1014: 0x4431, 0x1015: 0x4469, 0x1016: 0x44e9, 0x1017: 0x4581,
+	0x1018: 0x4601, 0x1019: 0x4651, 0x101a: 0x46d1, 0x101b: 0x4751, 0x101c: 0x47b9, 0x101d: 0x4809,
+	0x101e: 0x4859, 0x101f: 0x48a9, 0x1020: 0x4911, 0x1021: 0x4991, 0x1022: 0x49f9, 0x1023: 0x4a49,
+	0x1024: 0x4a99, 0x1025: 0x4ae9, 0x1026: 0x4b21, 0x1027: 0x4b59, 0x1028: 0x4b91, 0x1029: 0x4bc9,
+	0x102a: 0x4c19, 0x102b: 0x4c69, 0x102c: 0x4ce9, 0x102d: 0x4d39, 0x102e: 0x4da1, 0x102f: 0x4e21,
+	0x1030: 0x4e71, 0x1031: 0x4ea9, 0x1032: 0x4ee1, 0x1033: 0x4f61, 0x1034: 0x4fc9, 0x1035: 0x5049,
+	0x1036: 0x5099, 0x1037: 0x5119, 0x1038: 0x5151, 0x1039: 0x51a1, 0x103a: 0x51f1, 0x103b: 0x5241,
+	0x103c: 0x5291, 0x103d: 0x52e1, 0x103e: 0x5349, 0x103f: 0x5399,
+	// Block 0x41, offset 0x1040
+	0x1040: 0x53d1, 0x1041: 0x5421, 0x1042: 0x5471, 0x1043: 0x54c1, 0x1044: 0x5529, 0x1045: 0x5579,
+	0x1046: 0x55c9, 0x1047: 0x5619, 0x1048: 0x5699, 0x1049: 0x5701, 0x104a: 0x5739, 0x104b: 0x57b9,
+	0x104c: 0x57f1, 0x104d: 0x5859, 0x104e: 0x58c1, 0x104f: 0x5911, 0x1050: 0x5961, 0x1051: 0x59b1,
+	0x1052: 0x5a19, 0x1053: 0x5a51, 0x1054: 0x5aa1, 0x1055: 0x5b09, 0x1056: 0x5b41, 0x1057: 0x5bc1,
+	0x1058: 0x5c11, 0x1059: 0x5c39, 0x105a: 0x5c61, 0x105b: 0x5c89, 0x105c: 0x5cb1, 0x105d: 0x5cd9,
+	0x105e: 0x5d01, 0x105f: 0x5d29, 0x1060: 0x5d51, 0x1061: 0x5d79, 0x1062: 0x5da1, 0x1063: 0x5dd1,
+	0x1064: 0x5e01, 0x1065: 0x5e31, 0x1066: 0x5e61, 0x1067: 0x5e91, 0x1068: 0x5ec1, 0x1069: 0x5ef1,
+	0x106a: 0x5f21, 0x106b: 0x5f51, 0x106c: 0x5f81, 0x106d: 0x5fb1, 0x106e: 0x5fe1, 0x106f: 0x6011,
+	0x1070: 0x6041, 0x1071: 0x4045, 0x1072: 0x6071, 0x1073: 0x6089, 0x1074: 0x4065, 0x1075: 0x60a1,
+	0x1076: 0x60b9, 0x1077: 0x60d1, 0x1078: 0x4085, 0x1079: 0x4085, 0x107a: 0x60e9, 0x107b: 0x6101,
+	0x107c: 0x6139, 0x107d: 0x6171, 0x107e: 0x61a9, 0x107f: 0x61e1,
+	// Block 0x42, offset 0x1080
+	0x1080: 0x6249, 0x1081: 0x6261, 0x1082: 0x40a5, 0x1083: 0x6279, 0x1084: 0x6291, 0x1085: 0x62a9,
+	0x1086: 0x62c1, 0x1087: 0x62d9, 0x1088: 0x40c5, 0x1089: 0x62f1, 0x108a: 0x6319, 0x108b: 0x6331,
+	0x108c: 0x40e5, 0x108d: 0x40e5, 0x108e: 0x6349, 0x108f: 0x6361, 0x1090: 0x6379, 0x1091: 0x4105,
+	0x1092: 0x4125, 0x1093: 0x4145, 0x1094: 0x4165, 0x1095: 0x4185, 0x1096: 0x6391, 0x1097: 0x63a9,
+	0x1098: 0x63c1, 0x1099: 0x63d9, 0x109a: 0x63f1, 0x109b: 0x41a5, 0x109c: 0x6409, 0x109d: 0x6421,
+	0x109e: 0x6439, 0x109f: 0x41c5, 0x10a0: 0x41e5, 0x10a1: 0x6451, 0x10a2: 0x4205, 0x10a3: 0x4225,
+	0x10a4: 0x4245, 0x10a5: 0x6469, 0x10a6: 0x4265, 0x10a7: 0x6481, 0x10a8: 0x64b1, 0x10a9: 0x6249,
+	0x10aa: 0x4285, 0x10ab: 0x42a5, 0x10ac: 0x42c5, 0x10ad: 0x42e5, 0x10ae: 0x64e9, 0x10af: 0x6529,
+	0x10b0: 0x6571, 0x10b1: 0x6589, 0x10b2: 0x4305, 0x10b3: 0x65a1, 0x10b4: 0x65b9, 0x10b5: 0x65d1,
+	0x10b6: 0x4325, 0x10b7: 0x65e9, 0x10b8: 0x6601, 0x10b9: 0x65e9, 0x10ba: 0x6619, 0x10bb: 0x6631,
+	0x10bc: 0x4345, 0x10bd: 0x6649, 0x10be: 0x6661, 0x10bf: 0x6649,
+	// Block 0x43, offset 0x10c0
+	0x10c0: 0x4365, 0x10c1: 0x4385, 0x10c2: 0x0040, 0x10c3: 0x6679, 0x10c4: 0x6691, 0x10c5: 0x66a9,
+	0x10c6: 0x66c1, 0x10c7: 0x0040, 0x10c8: 0x66f9, 0x10c9: 0x6711, 0x10ca: 0x6729, 0x10cb: 0x6741,
+	0x10cc: 0x6759, 0x10cd: 0x6771, 0x10ce: 0x6439, 0x10cf: 0x6789, 0x10d0: 0x67a1, 0x10d1: 0x67b9,
+	0x10d2: 0x43a5, 0x10d3: 0x67d1, 0x10d4: 0x62c1, 0x10d5: 0x43c5, 0x10d6: 0x43e5, 0x10d7: 0x67e9,
+	0x10d8: 0x0040, 0x10d9: 0x4405, 0x10da: 0x6801, 0x10db: 0x6819, 0x10dc: 0x6831, 0x10dd: 0x6849,
+	0x10de: 0x6861, 0x10df: 0x6891, 0x10e0: 0x68c1, 0x10e1: 0x68e9, 0x10e2: 0x6911, 0x10e3: 0x6939,
+	0x10e4: 0x6961, 0x10e5: 0x6989, 0x10e6: 0x69b1, 0x10e7: 0x69d9, 0x10e8: 0x6a01, 0x10e9: 0x6a29,
+	0x10ea: 0x6a59, 0x10eb: 0x6a89, 0x10ec: 0x6ab9, 0x10ed: 0x6ae9, 0x10ee: 0x6b19, 0x10ef: 0x6b49,
+	0x10f0: 0x6b79, 0x10f1: 0x6ba9, 0x10f2: 0x6bd9, 0x10f3: 0x6c09, 0x10f4: 0x6c39, 0x10f5: 0x6c69,
+	0x10f6: 0x6c99, 0x10f7: 0x6cc9, 0x10f8: 0x6cf9, 0x10f9: 0x6d29, 0x10fa: 0x6d59, 0x10fb: 0x6d89,
+	0x10fc: 0x6db9, 0x10fd: 0x6de9, 0x10fe: 0x6e19, 0x10ff: 0x4425,
+	// Block 0x44, offset 0x1100
+	0x1100: 0xe00d, 0x1101: 0x0008, 0x1102: 0xe00d, 0x1103: 0x0008, 0x1104: 0xe00d, 0x1105: 0x0008,
+	0x1106: 0xe00d, 0x1107: 0x0008, 0x1108: 0xe00d, 0x1109: 0x0008, 0x110a: 0xe00d, 0x110b: 0x0008,
+	0x110c: 0xe00d, 0x110d: 0x0008, 0x110e: 0xe00d, 0x110f: 0x0008, 0x1110: 0xe00d, 0x1111: 0x0008,
+	0x1112: 0xe00d, 0x1113: 0x0008, 0x1114: 0xe00d, 0x1115: 0x0008, 0x1116: 0xe00d, 0x1117: 0x0008,
+	0x1118: 0xe00d, 0x1119: 0x0008, 0x111a: 0xe00d, 0x111b: 0x0008, 0x111c: 0xe00d, 0x111d: 0x0008,
+	0x111e: 0xe00d, 0x111f: 0x0008, 0x1120: 0xe00d, 0x1121: 0x0008, 0x1122: 0xe00d, 0x1123: 0x0008,
+	0x1124: 0xe00d, 0x1125: 0x0008, 0x1126: 0xe00d, 0x1127: 0x0008, 0x1128: 0xe00d, 0x1129: 0x0008,
+	0x112a: 0xe00d, 0x112b: 0x0008, 0x112c: 0xe00d, 0x112d: 0x0008, 0x112e: 0x0008, 0x112f: 0x3308,
+	0x1130: 0x3318, 0x1131: 0x3318, 0x1132: 0x3318, 0x1133: 0x0018, 0x1134: 0x3308, 0x1135: 0x3308,
+	0x1136: 0x3308, 0x1137: 0x3308, 0x1138: 0x3308, 0x1139: 0x3308, 0x113a: 0x3308, 0x113b: 0x3308,
+	0x113c: 0x3308, 0x113d: 0x3308, 0x113e: 0x0018, 0x113f: 0x0008,
+	// Block 0x45, offset 0x1140
+	0x1140: 0xe00d, 0x1141: 0x0008, 0x1142: 0xe00d, 0x1143: 0x0008, 0x1144: 0xe00d, 0x1145: 0x0008,
+	0x1146: 0xe00d, 0x1147: 0x0008, 0x1148: 0xe00d, 0x1149: 0x0008, 0x114a: 0xe00d, 0x114b: 0x0008,
+	0x114c: 0xe00d, 0x114d: 0x0008, 0x114e: 0xe00d, 0x114f: 0x0008, 0x1150: 0xe00d, 0x1151: 0x0008,
+	0x1152: 0xe00d, 0x1153: 0x0008, 0x1154: 0xe00d, 0x1155: 0x0008, 0x1156: 0xe00d, 0x1157: 0x0008,
+	0x1158: 0xe00d, 0x1159: 0x0008, 0x115a: 0xe00d, 0x115b: 0x0008, 0x115c: 0x0ea1, 0x115d: 0x6e49,
+	0x115e: 0x3308, 0x115f: 0x3308, 0x1160: 0x0008, 0x1161: 0x0008, 0x1162: 0x0008, 0x1163: 0x0008,
+	0x1164: 0x0008, 0x1165: 0x0008, 0x1166: 0x0008, 0x1167: 0x0008, 0x1168: 0x0008, 0x1169: 0x0008,
+	0x116a: 0x0008, 0x116b: 0x0008, 0x116c: 0x0008, 0x116d: 0x0008, 0x116e: 0x0008, 0x116f: 0x0008,
+	0x1170: 0x0008, 0x1171: 0x0008, 0x1172: 0x0008, 0x1173: 0x0008, 0x1174: 0x0008, 0x1175: 0x0008,
+	0x1176: 0x0008, 0x1177: 0x0008, 0x1178: 0x0008, 0x1179: 0x0008, 0x117a: 0x0008, 0x117b: 0x0008,
+	0x117c: 0x0008, 0x117d: 0x0008, 0x117e: 0x0008, 0x117f: 0x0008,
+	// Block 0x46, offset 0x1180
+	0x1180: 0x0018, 0x1181: 0x0018, 0x1182: 0x0018, 0x1183: 0x0018, 0x1184: 0x0018, 0x1185: 0x0018,
+	0x1186: 0x0018, 0x1187: 0x0018, 0x1188: 0x0018, 0x1189: 0x0018, 0x118a: 0x0018, 0x118b: 0x0018,
+	0x118c: 0x0018, 0x118d: 0x0018, 0x118e: 0x0018, 0x118f: 0x0018, 0x1190: 0x0018, 0x1191: 0x0018,
+	0x1192: 0x0018, 0x1193: 0x0018, 0x1194: 0x0018, 0x1195: 0x0018, 0x1196: 0x0018, 0x1197: 0x0008,
+	0x1198: 0x0008, 0x1199: 0x0008, 0x119a: 0x0008, 0x119b: 0x0008, 0x119c: 0x0008, 0x119d: 0x0008,
+	0x119e: 0x0008, 0x119f: 0x0008, 0x11a0: 0x0018, 0x11a1: 0x0018, 0x11a2: 0xe00d, 0x11a3: 0x0008,
+	0x11a4: 0xe00d, 0x11a5: 0x0008, 0x11a6: 0xe00d, 0x11a7: 0x0008, 0x11a8: 0xe00d, 0x11a9: 0x0008,
+	0x11aa: 0xe00d, 0x11ab: 0x0008, 0x11ac: 0xe00d, 0x11ad: 0x0008, 0x11ae: 0xe00d, 0x11af: 0x0008,
+	0x11b0: 0x0008, 0x11b1: 0x0008, 0x11b2: 0xe00d, 0x11b3: 0x0008, 0x11b4: 0xe00d, 0x11b5: 0x0008,
+	0x11b6: 0xe00d, 0x11b7: 0x0008, 0x11b8: 0xe00d, 0x11b9: 0x0008, 0x11ba: 0xe00d, 0x11bb: 0x0008,
+	0x11bc: 0xe00d, 0x11bd: 0x0008, 0x11be: 0xe00d, 0x11bf: 0x0008,
+	// Block 0x47, offset 0x11c0
+	0x11c0: 0xe00d, 0x11c1: 0x0008, 0x11c2: 0xe00d, 0x11c3: 0x0008, 0x11c4: 0xe00d, 0x11c5: 0x0008,
+	0x11c6: 0xe00d, 0x11c7: 0x0008, 0x11c8: 0xe00d, 0x11c9: 0x0008, 0x11ca: 0xe00d, 0x11cb: 0x0008,
+	0x11cc: 0xe00d, 0x11cd: 0x0008, 0x11ce: 0xe00d, 0x11cf: 0x0008, 0x11d0: 0xe00d, 0x11d1: 0x0008,
+	0x11d2: 0xe00d, 0x11d3: 0x0008, 0x11d4: 0xe00d, 0x11d5: 0x0008, 0x11d6: 0xe00d, 0x11d7: 0x0008,
+	0x11d8: 0xe00d, 0x11d9: 0x0008, 0x11da: 0xe00d, 0x11db: 0x0008, 0x11dc: 0xe00d, 0x11dd: 0x0008,
+	0x11de: 0xe00d, 0x11df: 0x0008, 0x11e0: 0xe00d, 0x11e1: 0x0008, 0x11e2: 0xe00d, 0x11e3: 0x0008,
+	0x11e4: 0xe00d, 0x11e5: 0x0008, 0x11e6: 0xe00d, 0x11e7: 0x0008, 0x11e8: 0xe00d, 0x11e9: 0x0008,
+	0x11ea: 0xe00d, 0x11eb: 0x0008, 0x11ec: 0xe00d, 0x11ed: 0x0008, 0x11ee: 0xe00d, 0x11ef: 0x0008,
+	0x11f0: 0xe0fd, 0x11f1: 0x0008, 0x11f2: 0x0008, 0x11f3: 0x0008, 0x11f4: 0x0008, 0x11f5: 0x0008,
+	0x11f6: 0x0008, 0x11f7: 0x0008, 0x11f8: 0x0008, 0x11f9: 0xe01d, 0x11fa: 0x0008, 0x11fb: 0xe03d,
+	0x11fc: 0x0008, 0x11fd: 0x4445, 0x11fe: 0xe00d, 0x11ff: 0x0008,
+	// Block 0x48, offset 0x1200
+	0x1200: 0xe00d, 0x1201: 0x0008, 0x1202: 0xe00d, 0x1203: 0x0008, 0x1204: 0xe00d, 0x1205: 0x0008,
+	0x1206: 0xe00d, 0x1207: 0x0008, 0x1208: 0x0008, 0x1209: 0x0018, 0x120a: 0x0018, 0x120b: 0xe03d,
+	0x120c: 0x0008, 0x120d: 0x11d9, 0x120e: 0x0008, 0x120f: 0x0008, 0x1210: 0xe00d, 0x1211: 0x0008,
+	0x1212: 0xe00d, 0x1213: 0x0008, 0x1214: 0x0008, 0x1215: 0x0008, 0x1216: 0xe00d, 0x1217: 0x0008,
+	0x1218: 0xe00d, 0x1219: 0x0008, 0x121a: 0xe00d, 0x121b: 0x0008, 0x121c: 0xe00d, 0x121d: 0x0008,
+	0x121e: 0xe00d, 0x121f: 0x0008, 0x1220: 0xe00d, 0x1221: 0x0008, 0x1222: 0xe00d, 0x1223: 0x0008,
+	0x1224: 0xe00d, 0x1225: 0x0008, 0x1226: 0xe00d, 0x1227: 0x0008, 0x1228: 0xe00d, 0x1229: 0x0008,
+	0x122a: 0x6e61, 0x122b: 0x1029, 0x122c: 0x11c1, 0x122d: 0x6e79, 0x122e: 0x1221, 0x122f: 0x0008,
+	0x1230: 0x6e91, 0x1231: 0x6ea9, 0x1232: 0x1239, 0x1233: 0x4465, 0x1234: 0xe00d, 0x1235: 0x0008,
+	0x1236: 0xe00d, 0x1237: 0x0008, 0x1238: 0xe00d, 0x1239: 0x0008, 0x123a: 0xe00d, 0x123b: 0x0008,
+	0x123c: 0xe00d, 0x123d: 0x0008, 0x123e: 0xe00d, 0x123f: 0x0008,
+	// Block 0x49, offset 0x1240
+	0x1240: 0x650d, 0x1241: 0x652d, 0x1242: 0x654d, 0x1243: 0x656d, 0x1244: 0x658d, 0x1245: 0x65ad,
+	0x1246: 0x65cd, 0x1247: 0x65ed, 0x1248: 0x660d, 0x1249: 0x662d, 0x124a: 0x664d, 0x124b: 0x666d,
+	0x124c: 0x668d, 0x124d: 0x66ad, 0x124e: 0x0008, 0x124f: 0x0008, 0x1250: 0x66cd, 0x1251: 0x0008,
+	0x1252: 0x66ed, 0x1253: 0x0008, 0x1254: 0x0008, 0x1255: 0x670d, 0x1256: 0x672d, 0x1257: 0x674d,
+	0x1258: 0x676d, 0x1259: 0x678d, 0x125a: 0x67ad, 0x125b: 0x67cd, 0x125c: 0x67ed, 0x125d: 0x680d,
+	0x125e: 0x682d, 0x125f: 0x0008, 0x1260: 0x684d, 0x1261: 0x0008, 0x1262: 0x686d, 0x1263: 0x0008,
+	0x1264: 0x0008, 0x1265: 0x688d, 0x1266: 0x68ad, 0x1267: 0x0008, 0x1268: 0x0008, 0x1269: 0x0008,
+	0x126a: 0x68cd, 0x126b: 0x68ed, 0x126c: 0x690d, 0x126d: 0x692d, 0x126e: 0x694d, 0x126f: 0x696d,
+	0x1270: 0x698d, 0x1271: 0x69ad, 0x1272: 0x69cd, 0x1273: 0x69ed, 0x1274: 0x6a0d, 0x1275: 0x6a2d,
+	0x1276: 0x6a4d, 0x1277: 0x6a6d, 0x1278: 0x6a8d, 0x1279: 0x6aad, 0x127a: 0x6acd, 0x127b: 0x6aed,
+	0x127c: 0x6b0d, 0x127d: 0x6b2d, 0x127e: 0x6b4d, 0x127f: 0x6b6d,
+	// Block 0x4a, offset 0x1280
+	0x1280: 0x7acd, 0x1281: 0x7aed, 0x1282: 0x7b0d, 0x1283: 0x7b2d, 0x1284: 0x7b4d, 0x1285: 0x7b6d,
+	0x1286: 0x7b8d, 0x1287: 0x7bad, 0x1288: 0x7bcd, 0x1289: 0x7bed, 0x128a: 0x7c0d, 0x128b: 0x7c2d,
+	0x128c: 0x7c4d, 0x128d: 0x7c6d, 0x128e: 0x7c8d, 0x128f: 0x6f19, 0x1290: 0x6f41, 0x1291: 0x6f69,
+	0x1292: 0x7cad, 0x1293: 0x7ccd, 0x1294: 0x7ced, 0x1295: 0x6f91, 0x1296: 0x6fb9, 0x1297: 0x6fe1,
+	0x1298: 0x7d0d, 0x1299: 0x7d2d, 0x129a: 0x0040, 0x129b: 0x0040, 0x129c: 0x0040, 0x129d: 0x0040,
+	0x129e: 0x0040, 0x129f: 0x0040, 0x12a0: 0x0040, 0x12a1: 0x0040, 0x12a2: 0x0040, 0x12a3: 0x0040,
+	0x12a4: 0x0040, 0x12a5: 0x0040, 0x12a6: 0x0040, 0x12a7: 0x0040, 0x12a8: 0x0040, 0x12a9: 0x0040,
+	0x12aa: 0x0040, 0x12ab: 0x0040, 0x12ac: 0x0040, 0x12ad: 0x0040, 0x12ae: 0x0040, 0x12af: 0x0040,
+	0x12b0: 0x0040, 0x12b1: 0x0040, 0x12b2: 0x0040, 0x12b3: 0x0040, 0x12b4: 0x0040, 0x12b5: 0x0040,
+	0x12b6: 0x0040, 0x12b7: 0x0040, 0x12b8: 0x0040, 0x12b9: 0x0040, 0x12ba: 0x0040, 0x12bb: 0x0040,
+	0x12bc: 0x0040, 0x12bd: 0x0040, 0x12be: 0x0040, 0x12bf: 0x0040,
+	// Block 0x4b, offset 0x12c0
+	0x12c0: 0x7009, 0x12c1: 0x7021, 0x12c2: 0x7039, 0x12c3: 0x7d4d, 0x12c4: 0x7d6d, 0x12c5: 0x7051,
+	0x12c6: 0x7051, 0x12c7: 0x0040, 0x12c8: 0x0040, 0x12c9: 0x0040, 0x12ca: 0x0040, 0x12cb: 0x0040,
+	0x12cc: 0x0040, 0x12cd: 0x0040, 0x12ce: 0x0040, 0x12cf: 0x0040, 0x12d0: 0x0040, 0x12d1: 0x0040,
+	0x12d2: 0x0040, 0x12d3: 0x7069, 0x12d4: 0x7091, 0x12d5: 0x70b9, 0x12d6: 0x70e1, 0x12d7: 0x7109,
+	0x12d8: 0x0040, 0x12d9: 0x0040, 0x12da: 0x0040, 0x12db: 0x0040, 0x12dc: 0x0040, 0x12dd: 0x7131,
+	0x12de: 0x3308, 0x12df: 0x7159, 0x12e0: 0x7181, 0x12e1: 0x20a9, 0x12e2: 0x20f1, 0x12e3: 0x7199,
+	0x12e4: 0x71b1, 0x12e5: 0x71c9, 0x12e6: 0x71e1, 0x12e7: 0x71f9, 0x12e8: 0x7211, 0x12e9: 0x1fb2,
+	0x12ea: 0x7229, 0x12eb: 0x7251, 0x12ec: 0x7279, 0x12ed: 0x72b1, 0x12ee: 0x72e9, 0x12ef: 0x7311,
+	0x12f0: 0x7339, 0x12f1: 0x7361, 0x12f2: 0x7389, 0x12f3: 0x73b1, 0x12f4: 0x73d9, 0x12f5: 0x7401,
+	0x12f6: 0x7429, 0x12f7: 0x0040, 0x12f8: 0x7451, 0x12f9: 0x7479, 0x12fa: 0x74a1, 0x12fb: 0x74c9,
+	0x12fc: 0x74f1, 0x12fd: 0x0040, 0x12fe: 0x7519, 0x12ff: 0x0040,
+	// Block 0x4c, offset 0x1300
+	0x1300: 0x7541, 0x1301: 0x7569, 0x1302: 0x0040, 0x1303: 0x7591, 0x1304: 0x75b9, 0x1305: 0x0040,
+	0x1306: 0x75e1, 0x1307: 0x7609, 0x1308: 0x7631, 0x1309: 0x7659, 0x130a: 0x7681, 0x130b: 0x76a9,
+	0x130c: 0x76d1, 0x130d: 0x76f9, 0x130e: 0x7721, 0x130f: 0x7749, 0x1310: 0x7771, 0x1311: 0x7771,
+	0x1312: 0x7789, 0x1313: 0x7789, 0x1314: 0x7789, 0x1315: 0x7789, 0x1316: 0x77a1, 0x1317: 0x77a1,
+	0x1318: 0x77a1, 0x1319: 0x77a1, 0x131a: 0x77b9, 0x131b: 0x77b9, 0x131c: 0x77b9, 0x131d: 0x77b9,
+	0x131e: 0x77d1, 0x131f: 0x77d1, 0x1320: 0x77d1, 0x1321: 0x77d1, 0x1322: 0x77e9, 0x1323: 0x77e9,
+	0x1324: 0x77e9, 0x1325: 0x77e9, 0x1326: 0x7801, 0x1327: 0x7801, 0x1328: 0x7801, 0x1329: 0x7801,
+	0x132a: 0x7819, 0x132b: 0x7819, 0x132c: 0x7819, 0x132d: 0x7819, 0x132e: 0x7831, 0x132f: 0x7831,
+	0x1330: 0x7831, 0x1331: 0x7831, 0x1332: 0x7849, 0x1333: 0x7849, 0x1334: 0x7849, 0x1335: 0x7849,
+	0x1336: 0x7861, 0x1337: 0x7861, 0x1338: 0x7861, 0x1339: 0x7861, 0x133a: 0x7879, 0x133b: 0x7879,
+	0x133c: 0x7879, 0x133d: 0x7879, 0x133e: 0x7891, 0x133f: 0x7891,
+	// Block 0x4d, offset 0x1340
+	0x1340: 0x7891, 0x1341: 0x7891, 0x1342: 0x78a9, 0x1343: 0x78a9, 0x1344: 0x78c1, 0x1345: 0x78c1,
+	0x1346: 0x78d9, 0x1347: 0x78d9, 0x1348: 0x78f1, 0x1349: 0x78f1, 0x134a: 0x7909, 0x134b: 0x7909,
+	0x134c: 0x7921, 0x134d: 0x7921, 0x134e: 0x7939, 0x134f: 0x7939, 0x1350: 0x7939, 0x1351: 0x7939,
+	0x1352: 0x7951, 0x1353: 0x7951, 0x1354: 0x7951, 0x1355: 0x7951, 0x1356: 0x7969, 0x1357: 0x7969,
+	0x1358: 0x7969, 0x1359: 0x7969, 0x135a: 0x7981, 0x135b: 0x7981, 0x135c: 0x7981, 0x135d: 0x7981,
+	0x135e: 0x7999, 0x135f: 0x7999, 0x1360: 0x79b1, 0x1361: 0x79b1, 0x1362: 0x79b1, 0x1363: 0x79b1,
+	0x1364: 0x79c9, 0x1365: 0x79c9, 0x1366: 0x79e1, 0x1367: 0x79e1, 0x1368: 0x79e1, 0x1369: 0x79e1,
+	0x136a: 0x79f9, 0x136b: 0x79f9, 0x136c: 0x79f9, 0x136d: 0x79f9, 0x136e: 0x7a11, 0x136f: 0x7a11,
+	0x1370: 0x7a29, 0x1371: 0x7a29, 0x1372: 0x0818, 0x1373: 0x0818, 0x1374: 0x0818, 0x1375: 0x0818,
+	0x1376: 0x0818, 0x1377: 0x0818, 0x1378: 0x0818, 0x1379: 0x0818, 0x137a: 0x0818, 0x137b: 0x0818,
+	0x137c: 0x0818, 0x137d: 0x0818, 0x137e: 0x0818, 0x137f: 0x0818,
+	// Block 0x4e, offset 0x1380
+	0x1380: 0x0818, 0x1381: 0x0818, 0x1382: 0x0040, 0x1383: 0x0040, 0x1384: 0x0040, 0x1385: 0x0040,
+	0x1386: 0x0040, 0x1387: 0x0040, 0x1388: 0x0040, 0x1389: 0x0040, 0x138a: 0x0040, 0x138b: 0x0040,
+	0x138c: 0x0040, 0x138d: 0x0040, 0x138e: 0x0040, 0x138f: 0x0040, 0x1390: 0x0040, 0x1391: 0x0040,
+	0x1392: 0x0040, 0x1393: 0x7a41, 0x1394: 0x7a41, 0x1395: 0x7a41, 0x1396: 0x7a41, 0x1397: 0x7a59,
+	0x1398: 0x7a59, 0x1399: 0x7a71, 0x139a: 0x7a71, 0x139b: 0x7a89, 0x139c: 0x7a89, 0x139d: 0x0479,
+	0x139e: 0x7aa1, 0x139f: 0x7aa1, 0x13a0: 0x7ab9, 0x13a1: 0x7ab9, 0x13a2: 0x7ad1, 0x13a3: 0x7ad1,
+	0x13a4: 0x7ae9, 0x13a5: 0x7ae9, 0x13a6: 0x7ae9, 0x13a7: 0x7ae9, 0x13a8: 0x7b01, 0x13a9: 0x7b01,
+	0x13aa: 0x7b19, 0x13ab: 0x7b19, 0x13ac: 0x7b41, 0x13ad: 0x7b41, 0x13ae: 0x7b69, 0x13af: 0x7b69,
+	0x13b0: 0x7b91, 0x13b1: 0x7b91, 0x13b2: 0x7bb9, 0x13b3: 0x7bb9, 0x13b4: 0x7be1, 0x13b5: 0x7be1,
+	0x13b6: 0x7c09, 0x13b7: 0x7c09, 0x13b8: 0x7c09, 0x13b9: 0x7c31, 0x13ba: 0x7c31, 0x13bb: 0x7c31,
+	0x13bc: 0x7c59, 0x13bd: 0x7c59, 0x13be: 0x7c59, 0x13bf: 0x7c59,
+	// Block 0x4f, offset 0x13c0
+	0x13c0: 0x8649, 0x13c1: 0x8671, 0x13c2: 0x8699, 0x13c3: 0x86c1, 0x13c4: 0x86e9, 0x13c5: 0x8711,
+	0x13c6: 0x8739, 0x13c7: 0x8761, 0x13c8: 0x8789, 0x13c9: 0x87b1, 0x13ca: 0x87d9, 0x13cb: 0x8801,
+	0x13cc: 0x8829, 0x13cd: 0x8851, 0x13ce: 0x8879, 0x13cf: 0x88a1, 0x13d0: 0x88c9, 0x13d1: 0x88f1,
+	0x13d2: 0x8919, 0x13d3: 0x8941, 0x13d4: 0x8969, 0x13d5: 0x8991, 0x13d6: 0x89b9, 0x13d7: 0x89e1,
+	0x13d8: 0x8a09, 0x13d9: 0x8a31, 0x13da: 0x8a59, 0x13db: 0x8a81, 0x13dc: 0x8aa9, 0x13dd: 0x8ad1,
+	0x13de: 0x8afa, 0x13df: 0x8b2a, 0x13e0: 0x8b5a, 0x13e1: 0x8b8a, 0x13e2: 0x8bba, 0x13e3: 0x8bea,
+	0x13e4: 0x8c19, 0x13e5: 0x8c41, 0x13e6: 0x7cc1, 0x13e7: 0x8c69, 0x13e8: 0x7c31, 0x13e9: 0x7ce9,
+	0x13ea: 0x8c91, 0x13eb: 0x8cb9, 0x13ec: 0x7d89, 0x13ed: 0x8ce1, 0x13ee: 0x7db1, 0x13ef: 0x7dd9,
+	0x13f0: 0x8d09, 0x13f1: 0x8d31, 0x13f2: 0x7e79, 0x13f3: 0x8d59, 0x13f4: 0x7ea1, 0x13f5: 0x7ec9,
+	0x13f6: 0x8d81, 0x13f7: 0x8da9, 0x13f8: 0x7f19, 0x13f9: 0x8dd1, 0x13fa: 0x7f41, 0x13fb: 0x7f69,
+	0x13fc: 0x83f1, 0x13fd: 0x8419, 0x13fe: 0x8491, 0x13ff: 0x84b9,
+	// Block 0x50, offset 0x1400
+	0x1400: 0x84e1, 0x1401: 0x8581, 0x1402: 0x85a9, 0x1403: 0x85d1, 0x1404: 0x85f9, 0x1405: 0x8699,
+	0x1406: 0x86c1, 0x1407: 0x86e9, 0x1408: 0x8df9, 0x1409: 0x8789, 0x140a: 0x8e21, 0x140b: 0x8e49,
+	0x140c: 0x8879, 0x140d: 0x8e71, 0x140e: 0x88a1, 0x140f: 0x88c9, 0x1410: 0x8ad1, 0x1411: 0x8e99,
+	0x1412: 0x8ec1, 0x1413: 0x8a09, 0x1414: 0x8ee9, 0x1415: 0x8a31, 0x1416: 0x8a59, 0x1417: 0x7c71,
+	0x1418: 0x7c99, 0x1419: 0x8f11, 0x141a: 0x7cc1, 0x141b: 0x8f39, 0x141c: 0x7d11, 0x141d: 0x7d39,
+	0x141e: 0x7d61, 0x141f: 0x7d89, 0x1420: 0x8f61, 0x1421: 0x7e01, 0x1422: 0x7e29, 0x1423: 0x7e51,
+	0x1424: 0x7e79, 0x1425: 0x8f89, 0x1426: 0x7f19, 0x1427: 0x7f91, 0x1428: 0x7fb9, 0x1429: 0x7fe1,
+	0x142a: 0x8009, 0x142b: 0x8031, 0x142c: 0x8081, 0x142d: 0x80a9, 0x142e: 0x80d1, 0x142f: 0x80f9,
+	0x1430: 0x8121, 0x1431: 0x8149, 0x1432: 0x8fb1, 0x1433: 0x8171, 0x1434: 0x8199, 0x1435: 0x81c1,
+	0x1436: 0x81e9, 0x1437: 0x8211, 0x1438: 0x8239, 0x1439: 0x8289, 0x143a: 0x82b1, 0x143b: 0x82d9,
+	0x143c: 0x8301, 0x143d: 0x8329, 0x143e: 0x8351, 0x143f: 0x8379,
+	// Block 0x51, offset 0x1440
+	0x1440: 0x83a1, 0x1441: 0x83c9, 0x1442: 0x8441, 0x1443: 0x8469, 0x1444: 0x8509, 0x1445: 0x8531,
+	0x1446: 0x8559, 0x1447: 0x8581, 0x1448: 0x85a9, 0x1449: 0x8621, 0x144a: 0x8649, 0x144b: 0x8671,
+	0x144c: 0x8699, 0x144d: 0x8fd9, 0x144e: 0x8711, 0x144f: 0x8739, 0x1450: 0x8761, 0x1451: 0x8789,
+	0x1452: 0x8801, 0x1453: 0x8829, 0x1454: 0x8851, 0x1455: 0x8879, 0x1456: 0x9001, 0x1457: 0x88f1,
+	0x1458: 0x8919, 0x1459: 0x9029, 0x145a: 0x8991, 0x145b: 0x89b9, 0x145c: 0x89e1, 0x145d: 0x8a09,
+	0x145e: 0x9051, 0x145f: 0x7cc1, 0x1460: 0x8f39, 0x1461: 0x7d89, 0x1462: 0x8f61, 0x1463: 0x7e79,
+	0x1464: 0x8f89, 0x1465: 0x7f19, 0x1466: 0x9079, 0x1467: 0x8121, 0x1468: 0x90a1, 0x1469: 0x90c9,
+	0x146a: 0x90f1, 0x146b: 0x8581, 0x146c: 0x85a9, 0x146d: 0x8699, 0x146e: 0x8879, 0x146f: 0x9001,
+	0x1470: 0x8a09, 0x1471: 0x9051, 0x1472: 0x9119, 0x1473: 0x9151, 0x1474: 0x9189, 0x1475: 0x91c1,
+	0x1476: 0x91e9, 0x1477: 0x9211, 0x1478: 0x9239, 0x1479: 0x9261, 0x147a: 0x9289, 0x147b: 0x92b1,
+	0x147c: 0x92d9, 0x147d: 0x9301, 0x147e: 0x9329, 0x147f: 0x9351,
+	// Block 0x52, offset 0x1480
+	0x1480: 0x9379, 0x1481: 0x93a1, 0x1482: 0x93c9, 0x1483: 0x93f1, 0x1484: 0x9419, 0x1485: 0x9441,
+	0x1486: 0x9469, 0x1487: 0x9491, 0x1488: 0x94b9, 0x1489: 0x94e1, 0x148a: 0x9509, 0x148b: 0x9531,
+	0x148c: 0x90c9, 0x148d: 0x9559, 0x148e: 0x9581, 0x148f: 0x95a9, 0x1490: 0x95d1, 0x1491: 0x91c1,
+	0x1492: 0x91e9, 0x1493: 0x9211, 0x1494: 0x9239, 0x1495: 0x9261, 0x1496: 0x9289, 0x1497: 0x92b1,
+	0x1498: 0x92d9, 0x1499: 0x9301, 0x149a: 0x9329, 0x149b: 0x9351, 0x149c: 0x9379, 0x149d: 0x93a1,
+	0x149e: 0x93c9, 0x149f: 0x93f1, 0x14a0: 0x9419, 0x14a1: 0x9441, 0x14a2: 0x9469, 0x14a3: 0x9491,
+	0x14a4: 0x94b9, 0x14a5: 0x94e1, 0x14a6: 0x9509, 0x14a7: 0x9531, 0x14a8: 0x90c9, 0x14a9: 0x9559,
+	0x14aa: 0x9581, 0x14ab: 0x95a9, 0x14ac: 0x95d1, 0x14ad: 0x94e1, 0x14ae: 0x9509, 0x14af: 0x9531,
+	0x14b0: 0x90c9, 0x14b1: 0x90a1, 0x14b2: 0x90f1, 0x14b3: 0x8261, 0x14b4: 0x80a9, 0x14b5: 0x80d1,
+	0x14b6: 0x80f9, 0x14b7: 0x94e1, 0x14b8: 0x9509, 0x14b9: 0x9531, 0x14ba: 0x8261, 0x14bb: 0x8289,
+	0x14bc: 0x95f9, 0x14bd: 0x95f9, 0x14be: 0x0018, 0x14bf: 0x0018,
+	// Block 0x53, offset 0x14c0
+	0x14c0: 0x0040, 0x14c1: 0x0040, 0x14c2: 0x0040, 0x14c3: 0x0040, 0x14c4: 0x0040, 0x14c5: 0x0040,
+	0x14c6: 0x0040, 0x14c7: 0x0040, 0x14c8: 0x0040, 0x14c9: 0x0040, 0x14ca: 0x0040, 0x14cb: 0x0040,
+	0x14cc: 0x0040, 0x14cd: 0x0040, 0x14ce: 0x0040, 0x14cf: 0x0040, 0x14d0: 0x9621, 0x14d1: 0x9659,
+	0x14d2: 0x9659, 0x14d3: 0x9691, 0x14d4: 0x96c9, 0x14d5: 0x9701, 0x14d6: 0x9739, 0x14d7: 0x9771,
+	0x14d8: 0x97a9, 0x14d9: 0x97a9, 0x14da: 0x97e1, 0x14db: 0x9819, 0x14dc: 0x9851, 0x14dd: 0x9889,
+	0x14de: 0x98c1, 0x14df: 0x98f9, 0x14e0: 0x98f9, 0x14e1: 0x9931, 0x14e2: 0x9969, 0x14e3: 0x9969,
+	0x14e4: 0x99a1, 0x14e5: 0x99a1, 0x14e6: 0x99d9, 0x14e7: 0x9a11, 0x14e8: 0x9a11, 0x14e9: 0x9a49,
+	0x14ea: 0x9a81, 0x14eb: 0x9a81, 0x14ec: 0x9ab9, 0x14ed: 0x9ab9, 0x14ee: 0x9af1, 0x14ef: 0x9b29,
+	0x14f0: 0x9b29, 0x14f1: 0x9b61, 0x14f2: 0x9b61, 0x14f3: 0x9b99, 0x14f4: 0x9bd1, 0x14f5: 0x9c09,
+	0x14f6: 0x9c41, 0x14f7: 0x9c41, 0x14f8: 0x9c79, 0x14f9: 0x9cb1, 0x14fa: 0x9ce9, 0x14fb: 0x9d21,
+	0x14fc: 0x9d59, 0x14fd: 0x9d59, 0x14fe: 0x9d91, 0x14ff: 0x9dc9,
+	// Block 0x54, offset 0x1500
+	0x1500: 0xa999, 0x1501: 0xa9d1, 0x1502: 0xaa09, 0x1503: 0xa8f1, 0x1504: 0x9c09, 0x1505: 0x99d9,
+	0x1506: 0xaa41, 0x1507: 0xaa79, 0x1508: 0x0040, 0x1509: 0x0040, 0x150a: 0x0040, 0x150b: 0x0040,
+	0x150c: 0x0040, 0x150d: 0x0040, 0x150e: 0x0040, 0x150f: 0x0040, 0x1510: 0x0040, 0x1511: 0x0040,
+	0x1512: 0x0040, 0x1513: 0x0040, 0x1514: 0x0040, 0x1515: 0x0040, 0x1516: 0x0040, 0x1517: 0x0040,
+	0x1518: 0x0040, 0x1519: 0x0040, 0x151a: 0x0040, 0x151b: 0x0040, 0x151c: 0x0040, 0x151d: 0x0040,
+	0x151e: 0x0040, 0x151f: 0x0040, 0x1520: 0x0040, 0x1521: 0x0040, 0x1522: 0x0040, 0x1523: 0x0040,
+	0x1524: 0x0040, 0x1525: 0x0040, 0x1526: 0x0040, 0x1527: 0x0040, 0x1528: 0x0040, 0x1529: 0x0040,
+	0x152a: 0x0040, 0x152b: 0x0040, 0x152c: 0x0040, 0x152d: 0x0040, 0x152e: 0x0040, 0x152f: 0x0040,
+	0x1530: 0xaab1, 0x1531: 0xaae9, 0x1532: 0xab21, 0x1533: 0xab69, 0x1534: 0xabb1, 0x1535: 0xabf9,
+	0x1536: 0xac41, 0x1537: 0xac89, 0x1538: 0xacd1, 0x1539: 0xad19, 0x153a: 0xad52, 0x153b: 0xae62,
+	0x153c: 0xaee1, 0x153d: 0x0018, 0x153e: 0x0040, 0x153f: 0x0040,
+	// Block 0x55, offset 0x1540
+	0x1540: 0x33c0, 0x1541: 0x33c0, 0x1542: 0x33c0, 0x1543: 0x33c0, 0x1544: 0x33c0, 0x1545: 0x33c0,
+	0x1546: 0x33c0, 0x1547: 0x33c0, 0x1548: 0x33c0, 0x1549: 0x33c0, 0x154a: 0x33c0, 0x154b: 0x33c0,
+	0x154c: 0x33c0, 0x154d: 0x33c0, 0x154e: 0x33c0, 0x154f: 0x33c0, 0x1550: 0xaf2a, 0x1551: 0x7d8d,
+	0x1552: 0x0040, 0x1553: 0xaf3a, 0x1554: 0x03c2, 0x1555: 0xaf4a, 0x1556: 0xaf5a, 0x1557: 0x7dad,
+	0x1558: 0x7dcd, 0x1559: 0x0040, 0x155a: 0x0040, 0x155b: 0x0040, 0x155c: 0x0040, 0x155d: 0x0040,
+	0x155e: 0x0040, 0x155f: 0x0040, 0x1560: 0x3308, 0x1561: 0x3308, 0x1562: 0x3308, 0x1563: 0x3308,
+	0x1564: 0x3308, 0x1565: 0x3308, 0x1566: 0x3308, 0x1567: 0x3308, 0x1568: 0x3308, 0x1569: 0x3308,
+	0x156a: 0x3308, 0x156b: 0x3308, 0x156c: 0x3308, 0x156d: 0x3308, 0x156e: 0x3308, 0x156f: 0x3308,
+	0x1570: 0x0040, 0x1571: 0x7ded, 0x1572: 0x7e0d, 0x1573: 0xaf6a, 0x1574: 0xaf6a, 0x1575: 0x1fd2,
+	0x1576: 0x1fe2, 0x1577: 0xaf7a, 0x1578: 0xaf8a, 0x1579: 0x7e2d, 0x157a: 0x7e4d, 0x157b: 0x7e6d,
+	0x157c: 0x7e2d, 0x157d: 0x7e8d, 0x157e: 0x7ead, 0x157f: 0x7e8d,
+	// Block 0x56, offset 0x1580
+	0x1580: 0x7ecd, 0x1581: 0x7eed, 0x1582: 0x7f0d, 0x1583: 0x7eed, 0x1584: 0x7f2d, 0x1585: 0x0018,
+	0x1586: 0x0018, 0x1587: 0xaf9a, 0x1588: 0xafaa, 0x1589: 0x7f4e, 0x158a: 0x7f6e, 0x158b: 0x7f8e,
+	0x158c: 0x7fae, 0x158d: 0xaf6a, 0x158e: 0xaf6a, 0x158f: 0xaf6a, 0x1590: 0xaf2a, 0x1591: 0x7fcd,
+	0x1592: 0x0040, 0x1593: 0x0040, 0x1594: 0x03c2, 0x1595: 0xaf3a, 0x1596: 0xaf5a, 0x1597: 0xaf4a,
+	0x1598: 0x7fed, 0x1599: 0x1fd2, 0x159a: 0x1fe2, 0x159b: 0xaf7a, 0x159c: 0xaf8a, 0x159d: 0x7ecd,
+	0x159e: 0x7f2d, 0x159f: 0xafba, 0x15a0: 0xafca, 0x15a1: 0xafda, 0x15a2: 0x1fb2, 0x15a3: 0xafe9,
+	0x15a4: 0xaffa, 0x15a5: 0xb00a, 0x15a6: 0x1fc2, 0x15a7: 0x0040, 0x15a8: 0xb01a, 0x15a9: 0xb02a,
+	0x15aa: 0xb03a, 0x15ab: 0xb04a, 0x15ac: 0x0040, 0x15ad: 0x0040, 0x15ae: 0x0040, 0x15af: 0x0040,
+	0x15b0: 0x800e, 0x15b1: 0xb059, 0x15b2: 0x802e, 0x15b3: 0x0808, 0x15b4: 0x804e, 0x15b5: 0x0040,
+	0x15b6: 0x806e, 0x15b7: 0xb081, 0x15b8: 0x808e, 0x15b9: 0xb0a9, 0x15ba: 0x80ae, 0x15bb: 0xb0d1,
+	0x15bc: 0x80ce, 0x15bd: 0xb0f9, 0x15be: 0x80ee, 0x15bf: 0xb121,
+	// Block 0x57, offset 0x15c0
+	0x15c0: 0xb149, 0x15c1: 0xb161, 0x15c2: 0xb161, 0x15c3: 0xb179, 0x15c4: 0xb179, 0x15c5: 0xb191,
+	0x15c6: 0xb191, 0x15c7: 0xb1a9, 0x15c8: 0xb1a9, 0x15c9: 0xb1c1, 0x15ca: 0xb1c1, 0x15cb: 0xb1c1,
+	0x15cc: 0xb1c1, 0x15cd: 0xb1d9, 0x15ce: 0xb1d9, 0x15cf: 0xb1f1, 0x15d0: 0xb1f1, 0x15d1: 0xb1f1,
+	0x15d2: 0xb1f1, 0x15d3: 0xb209, 0x15d4: 0xb209, 0x15d5: 0xb221, 0x15d6: 0xb221, 0x15d7: 0xb221,
+	0x15d8: 0xb221, 0x15d9: 0xb239, 0x15da: 0xb239, 0x15db: 0xb239, 0x15dc: 0xb239, 0x15dd: 0xb251,
+	0x15de: 0xb251, 0x15df: 0xb251, 0x15e0: 0xb251, 0x15e1: 0xb269, 0x15e2: 0xb269, 0x15e3: 0xb269,
+	0x15e4: 0xb269, 0x15e5: 0xb281, 0x15e6: 0xb281, 0x15e7: 0xb281, 0x15e8: 0xb281, 0x15e9: 0xb299,
+	0x15ea: 0xb299, 0x15eb: 0xb2b1, 0x15ec: 0xb2b1, 0x15ed: 0xb2c9, 0x15ee: 0xb2c9, 0x15ef: 0xb2e1,
+	0x15f0: 0xb2e1, 0x15f1: 0xb2f9, 0x15f2: 0xb2f9, 0x15f3: 0xb2f9, 0x15f4: 0xb2f9, 0x15f5: 0xb311,
+	0x15f6: 0xb311, 0x15f7: 0xb311, 0x15f8: 0xb311, 0x15f9: 0xb329, 0x15fa: 0xb329, 0x15fb: 0xb329,
+	0x15fc: 0xb329, 0x15fd: 0xb341, 0x15fe: 0xb341, 0x15ff: 0xb341,
+	// Block 0x58, offset 0x1600
+	0x1600: 0xb341, 0x1601: 0xb359, 0x1602: 0xb359, 0x1603: 0xb359, 0x1604: 0xb359, 0x1605: 0xb371,
+	0x1606: 0xb371, 0x1607: 0xb371, 0x1608: 0xb371, 0x1609: 0xb389, 0x160a: 0xb389, 0x160b: 0xb389,
+	0x160c: 0xb389, 0x160d: 0xb3a1, 0x160e: 0xb3a1, 0x160f: 0xb3a1, 0x1610: 0xb3a1, 0x1611: 0xb3b9,
+	0x1612: 0xb3b9, 0x1613: 0xb3b9, 0x1614: 0xb3b9, 0x1615: 0xb3d1, 0x1616: 0xb3d1, 0x1617: 0xb3d1,
+	0x1618: 0xb3d1, 0x1619: 0xb3e9, 0x161a: 0xb3e9, 0x161b: 0xb3e9, 0x161c: 0xb3e9, 0x161d: 0xb401,
+	0x161e: 0xb401, 0x161f: 0xb401, 0x1620: 0xb401, 0x1621: 0xb419, 0x1622: 0xb419, 0x1623: 0xb419,
+	0x1624: 0xb419, 0x1625: 0xb431, 0x1626: 0xb431, 0x1627: 0xb431, 0x1628: 0xb431, 0x1629: 0xb449,
+	0x162a: 0xb449, 0x162b: 0xb449, 0x162c: 0xb449, 0x162d: 0xb461, 0x162e: 0xb461, 0x162f: 0x7b01,
+	0x1630: 0x7b01, 0x1631: 0xb479, 0x1632: 0xb479, 0x1633: 0xb479, 0x1634: 0xb479, 0x1635: 0xb491,
+	0x1636: 0xb491, 0x1637: 0xb4b9, 0x1638: 0xb4b9, 0x1639: 0xb4e1, 0x163a: 0xb4e1, 0x163b: 0xb509,
+	0x163c: 0xb509, 0x163d: 0x0040, 0x163e: 0x0040, 0x163f: 0x03c0,
+	// Block 0x59, offset 0x1640
+	0x1640: 0x0040, 0x1641: 0xaf4a, 0x1642: 0xb532, 0x1643: 0xafba, 0x1644: 0xb02a, 0x1645: 0xb03a,
+	0x1646: 0xafca, 0x1647: 0xb542, 0x1648: 0x1fd2, 0x1649: 0x1fe2, 0x164a: 0xafda, 0x164b: 0x1fb2,
+	0x164c: 0xaf2a, 0x164d: 0xafe9, 0x164e: 0x29d1, 0x164f: 0xb552, 0x1650: 0x1f41, 0x1651: 0x00c9,
+	0x1652: 0x0069, 0x1653: 0x0079, 0x1654: 0x1f51, 0x1655: 0x1f61, 0x1656: 0x1f71, 0x1657: 0x1f81,
+	0x1658: 0x1f91, 0x1659: 0x1fa1, 0x165a: 0xaf3a, 0x165b: 0x03c2, 0x165c: 0xaffa, 0x165d: 0x1fc2,
+	0x165e: 0xb00a, 0x165f: 0xaf5a, 0x1660: 0xb04a, 0x1661: 0x0039, 0x1662: 0x0ee9, 0x1663: 0x1159,
+	0x1664: 0x0ef9, 0x1665: 0x0f09, 0x1666: 0x1199, 0x1667: 0x0f31, 0x1668: 0x0249, 0x1669: 0x0f41,
+	0x166a: 0x0259, 0x166b: 0x0f51, 0x166c: 0x0359, 0x166d: 0x0f61, 0x166e: 0x0f71, 0x166f: 0x00d9,
+	0x1670: 0x0f99, 0x1671: 0x2039, 0x1672: 0x0269, 0x1673: 0x01d9, 0x1674: 0x0fa9, 0x1675: 0x0fb9,
+	0x1676: 0x1089, 0x1677: 0x0279, 0x1678: 0x0369, 0x1679: 0x0289, 0x167a: 0x13d1, 0x167b: 0xaf9a,
+	0x167c: 0xb01a, 0x167d: 0xafaa, 0x167e: 0xb562, 0x167f: 0xaf6a,
+	// Block 0x5a, offset 0x1680
+	0x1680: 0x1caa, 0x1681: 0x0039, 0x1682: 0x0ee9, 0x1683: 0x1159, 0x1684: 0x0ef9, 0x1685: 0x0f09,
+	0x1686: 0x1199, 0x1687: 0x0f31, 0x1688: 0x0249, 0x1689: 0x0f41, 0x168a: 0x0259, 0x168b: 0x0f51,
+	0x168c: 0x0359, 0x168d: 0x0f61, 0x168e: 0x0f71, 0x168f: 0x00d9, 0x1690: 0x0f99, 0x1691: 0x2039,
+	0x1692: 0x0269, 0x1693: 0x01d9, 0x1694: 0x0fa9, 0x1695: 0x0fb9, 0x1696: 0x1089, 0x1697: 0x0279,
+	0x1698: 0x0369, 0x1699: 0x0289, 0x169a: 0x13d1, 0x169b: 0xaf7a, 0x169c: 0xb572, 0x169d: 0xaf8a,
+	0x169e: 0xb582, 0x169f: 0x810d, 0x16a0: 0x812d, 0x16a1: 0x29d1, 0x16a2: 0x814d, 0x16a3: 0x814d,
+	0x16a4: 0x816d, 0x16a5: 0x818d, 0x16a6: 0x81ad, 0x16a7: 0x81cd, 0x16a8: 0x81ed, 0x16a9: 0x820d,
+	0x16aa: 0x822d, 0x16ab: 0x824d, 0x16ac: 0x826d, 0x16ad: 0x828d, 0x16ae: 0x82ad, 0x16af: 0x82cd,
+	0x16b0: 0x82ed, 0x16b1: 0x830d, 0x16b2: 0x832d, 0x16b3: 0x834d, 0x16b4: 0x836d, 0x16b5: 0x838d,
+	0x16b6: 0x83ad, 0x16b7: 0x83cd, 0x16b8: 0x83ed, 0x16b9: 0x840d, 0x16ba: 0x842d, 0x16bb: 0x844d,
+	0x16bc: 0x81ed, 0x16bd: 0x846d, 0x16be: 0x848d, 0x16bf: 0x824d,
+	// Block 0x5b, offset 0x16c0
+	0x16c0: 0x84ad, 0x16c1: 0x84cd, 0x16c2: 0x84ed, 0x16c3: 0x850d, 0x16c4: 0x852d, 0x16c5: 0x854d,
+	0x16c6: 0x856d, 0x16c7: 0x858d, 0x16c8: 0x850d, 0x16c9: 0x85ad, 0x16ca: 0x850d, 0x16cb: 0x85cd,
+	0x16cc: 0x85cd, 0x16cd: 0x85ed, 0x16ce: 0x85ed, 0x16cf: 0x860d, 0x16d0: 0x854d, 0x16d1: 0x862d,
+	0x16d2: 0x864d, 0x16d3: 0x862d, 0x16d4: 0x866d, 0x16d5: 0x864d, 0x16d6: 0x868d, 0x16d7: 0x868d,
+	0x16d8: 0x86ad, 0x16d9: 0x86ad, 0x16da: 0x86cd, 0x16db: 0x86cd, 0x16dc: 0x864d, 0x16dd: 0x814d,
+	0x16de: 0x86ed, 0x16df: 0x870d, 0x16e0: 0x0040, 0x16e1: 0x872d, 0x16e2: 0x874d, 0x16e3: 0x876d,
+	0x16e4: 0x878d, 0x16e5: 0x876d, 0x16e6: 0x87ad, 0x16e7: 0x87cd, 0x16e8: 0x87ed, 0x16e9: 0x87ed,
+	0x16ea: 0x880d, 0x16eb: 0x880d, 0x16ec: 0x882d, 0x16ed: 0x882d, 0x16ee: 0x880d, 0x16ef: 0x880d,
+	0x16f0: 0x884d, 0x16f1: 0x886d, 0x16f2: 0x888d, 0x16f3: 0x88ad, 0x16f4: 0x88cd, 0x16f5: 0x88ed,
+	0x16f6: 0x88ed, 0x16f7: 0x88ed, 0x16f8: 0x890d, 0x16f9: 0x890d, 0x16fa: 0x890d, 0x16fb: 0x890d,
+	0x16fc: 0x87ed, 0x16fd: 0x87ed, 0x16fe: 0x87ed, 0x16ff: 0x0040,
+	// Block 0x5c, offset 0x1700
+	0x1700: 0x0040, 0x1701: 0x0040, 0x1702: 0x874d, 0x1703: 0x872d, 0x1704: 0x892d, 0x1705: 0x872d,
+	0x1706: 0x874d, 0x1707: 0x872d, 0x1708: 0x0040, 0x1709: 0x0040, 0x170a: 0x894d, 0x170b: 0x874d,
+	0x170c: 0x896d, 0x170d: 0x892d, 0x170e: 0x896d, 0x170f: 0x874d, 0x1710: 0x0040, 0x1711: 0x0040,
+	0x1712: 0x898d, 0x1713: 0x89ad, 0x1714: 0x88ad, 0x1715: 0x896d, 0x1716: 0x892d, 0x1717: 0x896d,
+	0x1718: 0x0040, 0x1719: 0x0040, 0x171a: 0x89cd, 0x171b: 0x89ed, 0x171c: 0x89cd, 0x171d: 0x0040,
+	0x171e: 0x0040, 0x171f: 0x0040, 0x1720: 0xb591, 0x1721: 0xb5a9, 0x1722: 0xb5c1, 0x1723: 0x8a0e,
+	0x1724: 0xb5d9, 0x1725: 0xb5f1, 0x1726: 0x8a2d, 0x1727: 0x0040, 0x1728: 0x8a4d, 0x1729: 0x8a6d,
+	0x172a: 0x8a8d, 0x172b: 0x8a6d, 0x172c: 0x8aad, 0x172d: 0x8acd, 0x172e: 0x8aed, 0x172f: 0x0040,
+	0x1730: 0x0040, 0x1731: 0x0040, 0x1732: 0x0040, 0x1733: 0x0040, 0x1734: 0x0040, 0x1735: 0x0040,
+	0x1736: 0x0040, 0x1737: 0x0040, 0x1738: 0x0040, 0x1739: 0x0340, 0x173a: 0x0340, 0x173b: 0x0340,
+	0x173c: 0x0040, 0x173d: 0x0040, 0x173e: 0x0040, 0x173f: 0x0040,
+	// Block 0x5d, offset 0x1740
+	0x1740: 0x0a08, 0x1741: 0x0a08, 0x1742: 0x0a08, 0x1743: 0x0a08, 0x1744: 0x0a08, 0x1745: 0x0c08,
+	0x1746: 0x0808, 0x1747: 0x0c08, 0x1748: 0x0818, 0x1749: 0x0c08, 0x174a: 0x0c08, 0x174b: 0x0808,
+	0x174c: 0x0808, 0x174d: 0x0908, 0x174e: 0x0c08, 0x174f: 0x0c08, 0x1750: 0x0c08, 0x1751: 0x0c08,
+	0x1752: 0x0c08, 0x1753: 0x0a08, 0x1754: 0x0a08, 0x1755: 0x0a08, 0x1756: 0x0a08, 0x1757: 0x0908,
+	0x1758: 0x0a08, 0x1759: 0x0a08, 0x175a: 0x0a08, 0x175b: 0x0a08, 0x175c: 0x0a08, 0x175d: 0x0c08,
+	0x175e: 0x0a08, 0x175f: 0x0a08, 0x1760: 0x0a08, 0x1761: 0x0c08, 0x1762: 0x0808, 0x1763: 0x0808,
+	0x1764: 0x0c08, 0x1765: 0x3308, 0x1766: 0x3308, 0x1767: 0x0040, 0x1768: 0x0040, 0x1769: 0x0040,
+	0x176a: 0x0040, 0x176b: 0x0a18, 0x176c: 0x0a18, 0x176d: 0x0a18, 0x176e: 0x0a18, 0x176f: 0x0c18,
+	0x1770: 0x0818, 0x1771: 0x0818, 0x1772: 0x0818, 0x1773: 0x0818, 0x1774: 0x0818, 0x1775: 0x0818,
+	0x1776: 0x0818, 0x1777: 0x0040, 0x1778: 0x0040, 0x1779: 0x0040, 0x177a: 0x0040, 0x177b: 0x0040,
+	0x177c: 0x0040, 0x177d: 0x0040, 0x177e: 0x0040, 0x177f: 0x0040,
+	// Block 0x5e, offset 0x1780
+	0x1780: 0x0a08, 0x1781: 0x0c08, 0x1782: 0x0a08, 0x1783: 0x0c08, 0x1784: 0x0c08, 0x1785: 0x0c08,
+	0x1786: 0x0a08, 0x1787: 0x0a08, 0x1788: 0x0a08, 0x1789: 0x0c08, 0x178a: 0x0a08, 0x178b: 0x0a08,
+	0x178c: 0x0c08, 0x178d: 0x0a08, 0x178e: 0x0c08, 0x178f: 0x0c08, 0x1790: 0x0a08, 0x1791: 0x0c08,
+	0x1792: 0x0040, 0x1793: 0x0040, 0x1794: 0x0040, 0x1795: 0x0040, 0x1796: 0x0040, 0x1797: 0x0040,
+	0x1798: 0x0040, 0x1799: 0x0818, 0x179a: 0x0818, 0x179b: 0x0818, 0x179c: 0x0818, 0x179d: 0x0040,
+	0x179e: 0x0040, 0x179f: 0x0040, 0x17a0: 0x0040, 0x17a1: 0x0040, 0x17a2: 0x0040, 0x17a3: 0x0040,
+	0x17a4: 0x0040, 0x17a5: 0x0040, 0x17a6: 0x0040, 0x17a7: 0x0040, 0x17a8: 0x0040, 0x17a9: 0x0c18,
+	0x17aa: 0x0c18, 0x17ab: 0x0c18, 0x17ac: 0x0c18, 0x17ad: 0x0a18, 0x17ae: 0x0a18, 0x17af: 0x0818,
+	0x17b0: 0x0040, 0x17b1: 0x0040, 0x17b2: 0x0040, 0x17b3: 0x0040, 0x17b4: 0x0040, 0x17b5: 0x0040,
+	0x17b6: 0x0040, 0x17b7: 0x0040, 0x17b8: 0x0040, 0x17b9: 0x0040, 0x17ba: 0x0040, 0x17bb: 0x0040,
+	0x17bc: 0x0040, 0x17bd: 0x0040, 0x17be: 0x0040, 0x17bf: 0x0040,
+	// Block 0x5f, offset 0x17c0
+	0x17c0: 0x3308, 0x17c1: 0x3308, 0x17c2: 0x3008, 0x17c3: 0x3008, 0x17c4: 0x0040, 0x17c5: 0x0008,
+	0x17c6: 0x0008, 0x17c7: 0x0008, 0x17c8: 0x0008, 0x17c9: 0x0008, 0x17ca: 0x0008, 0x17cb: 0x0008,
+	0x17cc: 0x0008, 0x17cd: 0x0040, 0x17ce: 0x0040, 0x17cf: 0x0008, 0x17d0: 0x0008, 0x17d1: 0x0040,
+	0x17d2: 0x0040, 0x17d3: 0x0008, 0x17d4: 0x0008, 0x17d5: 0x0008, 0x17d6: 0x0008, 0x17d7: 0x0008,
+	0x17d8: 0x0008, 0x17d9: 0x0008, 0x17da: 0x0008, 0x17db: 0x0008, 0x17dc: 0x0008, 0x17dd: 0x0008,
+	0x17de: 0x0008, 0x17df: 0x0008, 0x17e0: 0x0008, 0x17e1: 0x0008, 0x17e2: 0x0008, 0x17e3: 0x0008,
+	0x17e4: 0x0008, 0x17e5: 0x0008, 0x17e6: 0x0008, 0x17e7: 0x0008, 0x17e8: 0x0008, 0x17e9: 0x0040,
+	0x17ea: 0x0008, 0x17eb: 0x0008, 0x17ec: 0x0008, 0x17ed: 0x0008, 0x17ee: 0x0008, 0x17ef: 0x0008,
+	0x17f0: 0x0008, 0x17f1: 0x0040, 0x17f2: 0x0008, 0x17f3: 0x0008, 0x17f4: 0x0040, 0x17f5: 0x0008,
+	0x17f6: 0x0008, 0x17f7: 0x0008, 0x17f8: 0x0008, 0x17f9: 0x0008, 0x17fa: 0x0040, 0x17fb: 0x3308,
+	0x17fc: 0x3308, 0x17fd: 0x0008, 0x17fe: 0x3008, 0x17ff: 0x3008,
+	// Block 0x60, offset 0x1800
+	0x1800: 0x3308, 0x1801: 0x3008, 0x1802: 0x3008, 0x1803: 0x3008, 0x1804: 0x3008, 0x1805: 0x0040,
+	0x1806: 0x0040, 0x1807: 0x3008, 0x1808: 0x3008, 0x1809: 0x0040, 0x180a: 0x0040, 0x180b: 0x3008,
+	0x180c: 0x3008, 0x180d: 0x3808, 0x180e: 0x0040, 0x180f: 0x0040, 0x1810: 0x0008, 0x1811: 0x0040,
+	0x1812: 0x0040, 0x1813: 0x0040, 0x1814: 0x0040, 0x1815: 0x0040, 0x1816: 0x0040, 0x1817: 0x3008,
+	0x1818: 0x0040, 0x1819: 0x0040, 0x181a: 0x0040, 0x181b: 0x0040, 0x181c: 0x0040, 0x181d: 0x0008,
+	0x181e: 0x0008, 0x181f: 0x0008, 0x1820: 0x0008, 0x1821: 0x0008, 0x1822: 0x3008, 0x1823: 0x3008,
+	0x1824: 0x0040, 0x1825: 0x0040, 0x1826: 0x3308, 0x1827: 0x3308, 0x1828: 0x3308, 0x1829: 0x3308,
+	0x182a: 0x3308, 0x182b: 0x3308, 0x182c: 0x3308, 0x182d: 0x0040, 0x182e: 0x0040, 0x182f: 0x0040,
+	0x1830: 0x3308, 0x1831: 0x3308, 0x1832: 0x3308, 0x1833: 0x3308, 0x1834: 0x3308, 0x1835: 0x0040,
+	0x1836: 0x0040, 0x1837: 0x0040, 0x1838: 0x0040, 0x1839: 0x0040, 0x183a: 0x0040, 0x183b: 0x0040,
+	0x183c: 0x0040, 0x183d: 0x0040, 0x183e: 0x0040, 0x183f: 0x0040,
+	// Block 0x61, offset 0x1840
+	0x1840: 0x0008, 0x1841: 0x0008, 0x1842: 0x0008, 0x1843: 0x0008, 0x1844: 0x0008, 0x1845: 0x0008,
+	0x1846: 0x0008, 0x1847: 0x0040, 0x1848: 0x0040, 0x1849: 0x0008, 0x184a: 0x0040, 0x184b: 0x0040,
+	0x184c: 0x0008, 0x184d: 0x0008, 0x184e: 0x0008, 0x184f: 0x0008, 0x1850: 0x0008, 0x1851: 0x0008,
+	0x1852: 0x0008, 0x1853: 0x0008, 0x1854: 0x0040, 0x1855: 0x0008, 0x1856: 0x0008, 0x1857: 0x0040,
+	0x1858: 0x0008, 0x1859: 0x0008, 0x185a: 0x0008, 0x185b: 0x0008, 0x185c: 0x0008, 0x185d: 0x0008,
+	0x185e: 0x0008, 0x185f: 0x0008, 0x1860: 0x0008, 0x1861: 0x0008, 0x1862: 0x0008, 0x1863: 0x0008,
+	0x1864: 0x0008, 0x1865: 0x0008, 0x1866: 0x0008, 0x1867: 0x0008, 0x1868: 0x0008, 0x1869: 0x0008,
+	0x186a: 0x0008, 0x186b: 0x0008, 0x186c: 0x0008, 0x186d: 0x0008, 0x186e: 0x0008, 0x186f: 0x0008,
+	0x1870: 0x3008, 0x1871: 0x3008, 0x1872: 0x3008, 0x1873: 0x3008, 0x1874: 0x3008, 0x1875: 0x3008,
+	0x1876: 0x0040, 0x1877: 0x3008, 0x1878: 0x3008, 0x1879: 0x0040, 0x187a: 0x0040, 0x187b: 0x3308,
+	0x187c: 0x3308, 0x187d: 0x3808, 0x187e: 0x3b08, 0x187f: 0x0008,
+	// Block 0x62, offset 0x1880
+	0x1880: 0x0039, 0x1881: 0x0ee9, 0x1882: 0x1159, 0x1883: 0x0ef9, 0x1884: 0x0f09, 0x1885: 0x1199,
+	0x1886: 0x0f31, 0x1887: 0x0249, 0x1888: 0x0f41, 0x1889: 0x0259, 0x188a: 0x0f51, 0x188b: 0x0359,
+	0x188c: 0x0f61, 0x188d: 0x0f71, 0x188e: 0x00d9, 0x188f: 0x0f99, 0x1890: 0x2039, 0x1891: 0x0269,
+	0x1892: 0x01d9, 0x1893: 0x0fa9, 0x1894: 0x0fb9, 0x1895: 0x1089, 0x1896: 0x0279, 0x1897: 0x0369,
+	0x1898: 0x0289, 0x1899: 0x13d1, 0x189a: 0x0039, 0x189b: 0x0ee9, 0x189c: 0x1159, 0x189d: 0x0ef9,
+	0x189e: 0x0f09, 0x189f: 0x1199, 0x18a0: 0x0f31, 0x18a1: 0x0249, 0x18a2: 0x0f41, 0x18a3: 0x0259,
+	0x18a4: 0x0f51, 0x18a5: 0x0359, 0x18a6: 0x0f61, 0x18a7: 0x0f71, 0x18a8: 0x00d9, 0x18a9: 0x0f99,
+	0x18aa: 0x2039, 0x18ab: 0x0269, 0x18ac: 0x01d9, 0x18ad: 0x0fa9, 0x18ae: 0x0fb9, 0x18af: 0x1089,
+	0x18b0: 0x0279, 0x18b1: 0x0369, 0x18b2: 0x0289, 0x18b3: 0x13d1, 0x18b4: 0x0039, 0x18b5: 0x0ee9,
+	0x18b6: 0x1159, 0x18b7: 0x0ef9, 0x18b8: 0x0f09, 0x18b9: 0x1199, 0x18ba: 0x0f31, 0x18bb: 0x0249,
+	0x18bc: 0x0f41, 0x18bd: 0x0259, 0x18be: 0x0f51, 0x18bf: 0x0359,
+	// Block 0x63, offset 0x18c0
+	0x18c0: 0x0f61, 0x18c1: 0x0f71, 0x18c2: 0x00d9, 0x18c3: 0x0f99, 0x18c4: 0x2039, 0x18c5: 0x0269,
+	0x18c6: 0x01d9, 0x18c7: 0x0fa9, 0x18c8: 0x0fb9, 0x18c9: 0x1089, 0x18ca: 0x0279, 0x18cb: 0x0369,
+	0x18cc: 0x0289, 0x18cd: 0x13d1, 0x18ce: 0x0039, 0x18cf: 0x0ee9, 0x18d0: 0x1159, 0x18d1: 0x0ef9,
+	0x18d2: 0x0f09, 0x18d3: 0x1199, 0x18d4: 0x0f31, 0x18d5: 0x0040, 0x18d6: 0x0f41, 0x18d7: 0x0259,
+	0x18d8: 0x0f51, 0x18d9: 0x0359, 0x18da: 0x0f61, 0x18db: 0x0f71, 0x18dc: 0x00d9, 0x18dd: 0x0f99,
+	0x18de: 0x2039, 0x18df: 0x0269, 0x18e0: 0x01d9, 0x18e1: 0x0fa9, 0x18e2: 0x0fb9, 0x18e3: 0x1089,
+	0x18e4: 0x0279, 0x18e5: 0x0369, 0x18e6: 0x0289, 0x18e7: 0x13d1, 0x18e8: 0x0039, 0x18e9: 0x0ee9,
+	0x18ea: 0x1159, 0x18eb: 0x0ef9, 0x18ec: 0x0f09, 0x18ed: 0x1199, 0x18ee: 0x0f31, 0x18ef: 0x0249,
+	0x18f0: 0x0f41, 0x18f1: 0x0259, 0x18f2: 0x0f51, 0x18f3: 0x0359, 0x18f4: 0x0f61, 0x18f5: 0x0f71,
+	0x18f6: 0x00d9, 0x18f7: 0x0f99, 0x18f8: 0x2039, 0x18f9: 0x0269, 0x18fa: 0x01d9, 0x18fb: 0x0fa9,
+	0x18fc: 0x0fb9, 0x18fd: 0x1089, 0x18fe: 0x0279, 0x18ff: 0x0369,
+	// Block 0x64, offset 0x1900
+	0x1900: 0x0289, 0x1901: 0x13d1, 0x1902: 0x0039, 0x1903: 0x0ee9, 0x1904: 0x1159, 0x1905: 0x0ef9,
+	0x1906: 0x0f09, 0x1907: 0x1199, 0x1908: 0x0f31, 0x1909: 0x0249, 0x190a: 0x0f41, 0x190b: 0x0259,
+	0x190c: 0x0f51, 0x190d: 0x0359, 0x190e: 0x0f61, 0x190f: 0x0f71, 0x1910: 0x00d9, 0x1911: 0x0f99,
+	0x1912: 0x2039, 0x1913: 0x0269, 0x1914: 0x01d9, 0x1915: 0x0fa9, 0x1916: 0x0fb9, 0x1917: 0x1089,
+	0x1918: 0x0279, 0x1919: 0x0369, 0x191a: 0x0289, 0x191b: 0x13d1, 0x191c: 0x0039, 0x191d: 0x0040,
+	0x191e: 0x1159, 0x191f: 0x0ef9, 0x1920: 0x0040, 0x1921: 0x0040, 0x1922: 0x0f31, 0x1923: 0x0040,
+	0x1924: 0x0040, 0x1925: 0x0259, 0x1926: 0x0f51, 0x1927: 0x0040, 0x1928: 0x0040, 0x1929: 0x0f71,
+	0x192a: 0x00d9, 0x192b: 0x0f99, 0x192c: 0x2039, 0x192d: 0x0040, 0x192e: 0x01d9, 0x192f: 0x0fa9,
+	0x1930: 0x0fb9, 0x1931: 0x1089, 0x1932: 0x0279, 0x1933: 0x0369, 0x1934: 0x0289, 0x1935: 0x13d1,
+	0x1936: 0x0039, 0x1937: 0x0ee9, 0x1938: 0x1159, 0x1939: 0x0ef9, 0x193a: 0x0040, 0x193b: 0x1199,
+	0x193c: 0x0040, 0x193d: 0x0249, 0x193e: 0x0f41, 0x193f: 0x0259,
+	// Block 0x65, offset 0x1940
+	0x1940: 0x0f51, 0x1941: 0x0359, 0x1942: 0x0f61, 0x1943: 0x0f71, 0x1944: 0x0040, 0x1945: 0x0f99,
+	0x1946: 0x2039, 0x1947: 0x0269, 0x1948: 0x01d9, 0x1949: 0x0fa9, 0x194a: 0x0fb9, 0x194b: 0x1089,
+	0x194c: 0x0279, 0x194d: 0x0369, 0x194e: 0x0289, 0x194f: 0x13d1, 0x1950: 0x0039, 0x1951: 0x0ee9,
+	0x1952: 0x1159, 0x1953: 0x0ef9, 0x1954: 0x0f09, 0x1955: 0x1199, 0x1956: 0x0f31, 0x1957: 0x0249,
+	0x1958: 0x0f41, 0x1959: 0x0259, 0x195a: 0x0f51, 0x195b: 0x0359, 0x195c: 0x0f61, 0x195d: 0x0f71,
+	0x195e: 0x00d9, 0x195f: 0x0f99, 0x1960: 0x2039, 0x1961: 0x0269, 0x1962: 0x01d9, 0x1963: 0x0fa9,
+	0x1964: 0x0fb9, 0x1965: 0x1089, 0x1966: 0x0279, 0x1967: 0x0369, 0x1968: 0x0289, 0x1969: 0x13d1,
+	0x196a: 0x0039, 0x196b: 0x0ee9, 0x196c: 0x1159, 0x196d: 0x0ef9, 0x196e: 0x0f09, 0x196f: 0x1199,
+	0x1970: 0x0f31, 0x1971: 0x0249, 0x1972: 0x0f41, 0x1973: 0x0259, 0x1974: 0x0f51, 0x1975: 0x0359,
+	0x1976: 0x0f61, 0x1977: 0x0f71, 0x1978: 0x00d9, 0x1979: 0x0f99, 0x197a: 0x2039, 0x197b: 0x0269,
+	0x197c: 0x01d9, 0x197d: 0x0fa9, 0x197e: 0x0fb9, 0x197f: 0x1089,
+	// Block 0x66, offset 0x1980
+	0x1980: 0x0279, 0x1981: 0x0369, 0x1982: 0x0289, 0x1983: 0x13d1, 0x1984: 0x0039, 0x1985: 0x0ee9,
+	0x1986: 0x0040, 0x1987: 0x0ef9, 0x1988: 0x0f09, 0x1989: 0x1199, 0x198a: 0x0f31, 0x198b: 0x0040,
+	0x198c: 0x0040, 0x198d: 0x0259, 0x198e: 0x0f51, 0x198f: 0x0359, 0x1990: 0x0f61, 0x1991: 0x0f71,
+	0x1992: 0x00d9, 0x1993: 0x0f99, 0x1994: 0x2039, 0x1995: 0x0040, 0x1996: 0x01d9, 0x1997: 0x0fa9,
+	0x1998: 0x0fb9, 0x1999: 0x1089, 0x199a: 0x0279, 0x199b: 0x0369, 0x199c: 0x0289, 0x199d: 0x0040,
+	0x199e: 0x0039, 0x199f: 0x0ee9, 0x19a0: 0x1159, 0x19a1: 0x0ef9, 0x19a2: 0x0f09, 0x19a3: 0x1199,
+	0x19a4: 0x0f31, 0x19a5: 0x0249, 0x19a6: 0x0f41, 0x19a7: 0x0259, 0x19a8: 0x0f51, 0x19a9: 0x0359,
+	0x19aa: 0x0f61, 0x19ab: 0x0f71, 0x19ac: 0x00d9, 0x19ad: 0x0f99, 0x19ae: 0x2039, 0x19af: 0x0269,
+	0x19b0: 0x01d9, 0x19b1: 0x0fa9, 0x19b2: 0x0fb9, 0x19b3: 0x1089, 0x19b4: 0x0279, 0x19b5: 0x0369,
+	0x19b6: 0x0289, 0x19b7: 0x13d1, 0x19b8: 0x0039, 0x19b9: 0x0ee9, 0x19ba: 0x0040, 0x19bb: 0x0ef9,
+	0x19bc: 0x0f09, 0x19bd: 0x1199, 0x19be: 0x0f31, 0x19bf: 0x0040,
+	// Block 0x67, offset 0x19c0
+	0x19c0: 0x0f41, 0x19c1: 0x0259, 0x19c2: 0x0f51, 0x19c3: 0x0359, 0x19c4: 0x0f61, 0x19c5: 0x0040,
+	0x19c6: 0x00d9, 0x19c7: 0x0040, 0x19c8: 0x0040, 0x19c9: 0x0040, 0x19ca: 0x01d9, 0x19cb: 0x0fa9,
+	0x19cc: 0x0fb9, 0x19cd: 0x1089, 0x19ce: 0x0279, 0x19cf: 0x0369, 0x19d0: 0x0289, 0x19d1: 0x0040,
+	0x19d2: 0x0039, 0x19d3: 0x0ee9, 0x19d4: 0x1159, 0x19d5: 0x0ef9, 0x19d6: 0x0f09, 0x19d7: 0x1199,
+	0x19d8: 0x0f31, 0x19d9: 0x0249, 0x19da: 0x0f41, 0x19db: 0x0259, 0x19dc: 0x0f51, 0x19dd: 0x0359,
+	0x19de: 0x0f61, 0x19df: 0x0f71, 0x19e0: 0x00d9, 0x19e1: 0x0f99, 0x19e2: 0x2039, 0x19e3: 0x0269,
+	0x19e4: 0x01d9, 0x19e5: 0x0fa9, 0x19e6: 0x0fb9, 0x19e7: 0x1089, 0x19e8: 0x0279, 0x19e9: 0x0369,
+	0x19ea: 0x0289, 0x19eb: 0x13d1, 0x19ec: 0x0039, 0x19ed: 0x0ee9, 0x19ee: 0x1159, 0x19ef: 0x0ef9,
+	0x19f0: 0x0f09, 0x19f1: 0x1199, 0x19f2: 0x0f31, 0x19f3: 0x0249, 0x19f4: 0x0f41, 0x19f5: 0x0259,
+	0x19f6: 0x0f51, 0x19f7: 0x0359, 0x19f8: 0x0f61, 0x19f9: 0x0f71, 0x19fa: 0x00d9, 0x19fb: 0x0f99,
+	0x19fc: 0x2039, 0x19fd: 0x0269, 0x19fe: 0x01d9, 0x19ff: 0x0fa9,
+	// Block 0x68, offset 0x1a00
+	0x1a00: 0x0fb9, 0x1a01: 0x1089, 0x1a02: 0x0279, 0x1a03: 0x0369, 0x1a04: 0x0289, 0x1a05: 0x13d1,
+	0x1a06: 0x0039, 0x1a07: 0x0ee9, 0x1a08: 0x1159, 0x1a09: 0x0ef9, 0x1a0a: 0x0f09, 0x1a0b: 0x1199,
+	0x1a0c: 0x0f31, 0x1a0d: 0x0249, 0x1a0e: 0x0f41, 0x1a0f: 0x0259, 0x1a10: 0x0f51, 0x1a11: 0x0359,
+	0x1a12: 0x0f61, 0x1a13: 0x0f71, 0x1a14: 0x00d9, 0x1a15: 0x0f99, 0x1a16: 0x2039, 0x1a17: 0x0269,
+	0x1a18: 0x01d9, 0x1a19: 0x0fa9, 0x1a1a: 0x0fb9, 0x1a1b: 0x1089, 0x1a1c: 0x0279, 0x1a1d: 0x0369,
+	0x1a1e: 0x0289, 0x1a1f: 0x13d1, 0x1a20: 0x0039, 0x1a21: 0x0ee9, 0x1a22: 0x1159, 0x1a23: 0x0ef9,
+	0x1a24: 0x0f09, 0x1a25: 0x1199, 0x1a26: 0x0f31, 0x1a27: 0x0249, 0x1a28: 0x0f41, 0x1a29: 0x0259,
+	0x1a2a: 0x0f51, 0x1a2b: 0x0359, 0x1a2c: 0x0f61, 0x1a2d: 0x0f71, 0x1a2e: 0x00d9, 0x1a2f: 0x0f99,
+	0x1a30: 0x2039, 0x1a31: 0x0269, 0x1a32: 0x01d9, 0x1a33: 0x0fa9, 0x1a34: 0x0fb9, 0x1a35: 0x1089,
+	0x1a36: 0x0279, 0x1a37: 0x0369, 0x1a38: 0x0289, 0x1a39: 0x13d1, 0x1a3a: 0x0039, 0x1a3b: 0x0ee9,
+	0x1a3c: 0x1159, 0x1a3d: 0x0ef9, 0x1a3e: 0x0f09, 0x1a3f: 0x1199,
+	// Block 0x69, offset 0x1a40
+	0x1a40: 0x0f31, 0x1a41: 0x0249, 0x1a42: 0x0f41, 0x1a43: 0x0259, 0x1a44: 0x0f51, 0x1a45: 0x0359,
+	0x1a46: 0x0f61, 0x1a47: 0x0f71, 0x1a48: 0x00d9, 0x1a49: 0x0f99, 0x1a4a: 0x2039, 0x1a4b: 0x0269,
+	0x1a4c: 0x01d9, 0x1a4d: 0x0fa9, 0x1a4e: 0x0fb9, 0x1a4f: 0x1089, 0x1a50: 0x0279, 0x1a51: 0x0369,
+	0x1a52: 0x0289, 0x1a53: 0x13d1, 0x1a54: 0x0039, 0x1a55: 0x0ee9, 0x1a56: 0x1159, 0x1a57: 0x0ef9,
+	0x1a58: 0x0f09, 0x1a59: 0x1199, 0x1a5a: 0x0f31, 0x1a5b: 0x0249, 0x1a5c: 0x0f41, 0x1a5d: 0x0259,
+	0x1a5e: 0x0f51, 0x1a5f: 0x0359, 0x1a60: 0x0f61, 0x1a61: 0x0f71, 0x1a62: 0x00d9, 0x1a63: 0x0f99,
+	0x1a64: 0x2039, 0x1a65: 0x0269, 0x1a66: 0x01d9, 0x1a67: 0x0fa9, 0x1a68: 0x0fb9, 0x1a69: 0x1089,
+	0x1a6a: 0x0279, 0x1a6b: 0x0369, 0x1a6c: 0x0289, 0x1a6d: 0x13d1, 0x1a6e: 0x0039, 0x1a6f: 0x0ee9,
+	0x1a70: 0x1159, 0x1a71: 0x0ef9, 0x1a72: 0x0f09, 0x1a73: 0x1199, 0x1a74: 0x0f31, 0x1a75: 0x0249,
+	0x1a76: 0x0f41, 0x1a77: 0x0259, 0x1a78: 0x0f51, 0x1a79: 0x0359, 0x1a7a: 0x0f61, 0x1a7b: 0x0f71,
+	0x1a7c: 0x00d9, 0x1a7d: 0x0f99, 0x1a7e: 0x2039, 0x1a7f: 0x0269,
+	// Block 0x6a, offset 0x1a80
+	0x1a80: 0x01d9, 0x1a81: 0x0fa9, 0x1a82: 0x0fb9, 0x1a83: 0x1089, 0x1a84: 0x0279, 0x1a85: 0x0369,
+	0x1a86: 0x0289, 0x1a87: 0x13d1, 0x1a88: 0x0039, 0x1a89: 0x0ee9, 0x1a8a: 0x1159, 0x1a8b: 0x0ef9,
+	0x1a8c: 0x0f09, 0x1a8d: 0x1199, 0x1a8e: 0x0f31, 0x1a8f: 0x0249, 0x1a90: 0x0f41, 0x1a91: 0x0259,
+	0x1a92: 0x0f51, 0x1a93: 0x0359, 0x1a94: 0x0f61, 0x1a95: 0x0f71, 0x1a96: 0x00d9, 0x1a97: 0x0f99,
+	0x1a98: 0x2039, 0x1a99: 0x0269, 0x1a9a: 0x01d9, 0x1a9b: 0x0fa9, 0x1a9c: 0x0fb9, 0x1a9d: 0x1089,
+	0x1a9e: 0x0279, 0x1a9f: 0x0369, 0x1aa0: 0x0289, 0x1aa1: 0x13d1, 0x1aa2: 0x0039, 0x1aa3: 0x0ee9,
+	0x1aa4: 0x1159, 0x1aa5: 0x0ef9, 0x1aa6: 0x0f09, 0x1aa7: 0x1199, 0x1aa8: 0x0f31, 0x1aa9: 0x0249,
+	0x1aaa: 0x0f41, 0x1aab: 0x0259, 0x1aac: 0x0f51, 0x1aad: 0x0359, 0x1aae: 0x0f61, 0x1aaf: 0x0f71,
+	0x1ab0: 0x00d9, 0x1ab1: 0x0f99, 0x1ab2: 0x2039, 0x1ab3: 0x0269, 0x1ab4: 0x01d9, 0x1ab5: 0x0fa9,
+	0x1ab6: 0x0fb9, 0x1ab7: 0x1089, 0x1ab8: 0x0279, 0x1ab9: 0x0369, 0x1aba: 0x0289, 0x1abb: 0x13d1,
+	0x1abc: 0x0039, 0x1abd: 0x0ee9, 0x1abe: 0x1159, 0x1abf: 0x0ef9,
+	// Block 0x6b, offset 0x1ac0
+	0x1ac0: 0x0f09, 0x1ac1: 0x1199, 0x1ac2: 0x0f31, 0x1ac3: 0x0249, 0x1ac4: 0x0f41, 0x1ac5: 0x0259,
+	0x1ac6: 0x0f51, 0x1ac7: 0x0359, 0x1ac8: 0x0f61, 0x1ac9: 0x0f71, 0x1aca: 0x00d9, 0x1acb: 0x0f99,
+	0x1acc: 0x2039, 0x1acd: 0x0269, 0x1ace: 0x01d9, 0x1acf: 0x0fa9, 0x1ad0: 0x0fb9, 0x1ad1: 0x1089,
+	0x1ad2: 0x0279, 0x1ad3: 0x0369, 0x1ad4: 0x0289, 0x1ad5: 0x13d1, 0x1ad6: 0x0039, 0x1ad7: 0x0ee9,
+	0x1ad8: 0x1159, 0x1ad9: 0x0ef9, 0x1ada: 0x0f09, 0x1adb: 0x1199, 0x1adc: 0x0f31, 0x1add: 0x0249,
+	0x1ade: 0x0f41, 0x1adf: 0x0259, 0x1ae0: 0x0f51, 0x1ae1: 0x0359, 0x1ae2: 0x0f61, 0x1ae3: 0x0f71,
+	0x1ae4: 0x00d9, 0x1ae5: 0x0f99, 0x1ae6: 0x2039, 0x1ae7: 0x0269, 0x1ae8: 0x01d9, 0x1ae9: 0x0fa9,
+	0x1aea: 0x0fb9, 0x1aeb: 0x1089, 0x1aec: 0x0279, 0x1aed: 0x0369, 0x1aee: 0x0289, 0x1aef: 0x13d1,
+	0x1af0: 0x0039, 0x1af1: 0x0ee9, 0x1af2: 0x1159, 0x1af3: 0x0ef9, 0x1af4: 0x0f09, 0x1af5: 0x1199,
+	0x1af6: 0x0f31, 0x1af7: 0x0249, 0x1af8: 0x0f41, 0x1af9: 0x0259, 0x1afa: 0x0f51, 0x1afb: 0x0359,
+	0x1afc: 0x0f61, 0x1afd: 0x0f71, 0x1afe: 0x00d9, 0x1aff: 0x0f99,
+	// Block 0x6c, offset 0x1b00
+	0x1b00: 0x2039, 0x1b01: 0x0269, 0x1b02: 0x01d9, 0x1b03: 0x0fa9, 0x1b04: 0x0fb9, 0x1b05: 0x1089,
+	0x1b06: 0x0279, 0x1b07: 0x0369, 0x1b08: 0x0289, 0x1b09: 0x13d1, 0x1b0a: 0x0039, 0x1b0b: 0x0ee9,
+	0x1b0c: 0x1159, 0x1b0d: 0x0ef9, 0x1b0e: 0x0f09, 0x1b0f: 0x1199, 0x1b10: 0x0f31, 0x1b11: 0x0249,
+	0x1b12: 0x0f41, 0x1b13: 0x0259, 0x1b14: 0x0f51, 0x1b15: 0x0359, 0x1b16: 0x0f61, 0x1b17: 0x0f71,
+	0x1b18: 0x00d9, 0x1b19: 0x0f99, 0x1b1a: 0x2039, 0x1b1b: 0x0269, 0x1b1c: 0x01d9, 0x1b1d: 0x0fa9,
+	0x1b1e: 0x0fb9, 0x1b1f: 0x1089, 0x1b20: 0x0279, 0x1b21: 0x0369, 0x1b22: 0x0289, 0x1b23: 0x13d1,
+	0x1b24: 0xbad1, 0x1b25: 0xbae9, 0x1b26: 0x0040, 0x1b27: 0x0040, 0x1b28: 0xbb01, 0x1b29: 0x1099,
+	0x1b2a: 0x10b1, 0x1b2b: 0x10c9, 0x1b2c: 0xbb19, 0x1b2d: 0xbb31, 0x1b2e: 0xbb49, 0x1b2f: 0x1429,
+	0x1b30: 0x1a31, 0x1b31: 0xbb61, 0x1b32: 0xbb79, 0x1b33: 0xbb91, 0x1b34: 0xbba9, 0x1b35: 0xbbc1,
+	0x1b36: 0xbbd9, 0x1b37: 0x2109, 0x1b38: 0x1111, 0x1b39: 0x1429, 0x1b3a: 0xbbf1, 0x1b3b: 0xbc09,
+	0x1b3c: 0xbc21, 0x1b3d: 0x10e1, 0x1b3e: 0x10f9, 0x1b3f: 0xbc39,
+	// Block 0x6d, offset 0x1b40
+	0x1b40: 0x2079, 0x1b41: 0xbc51, 0x1b42: 0xbb01, 0x1b43: 0x1099, 0x1b44: 0x10b1, 0x1b45: 0x10c9,
+	0x1b46: 0xbb19, 0x1b47: 0xbb31, 0x1b48: 0xbb49, 0x1b49: 0x1429, 0x1b4a: 0x1a31, 0x1b4b: 0xbb61,
+	0x1b4c: 0xbb79, 0x1b4d: 0xbb91, 0x1b4e: 0xbba9, 0x1b4f: 0xbbc1, 0x1b50: 0xbbd9, 0x1b51: 0x2109,
+	0x1b52: 0x1111, 0x1b53: 0xbbf1, 0x1b54: 0xbbf1, 0x1b55: 0xbc09, 0x1b56: 0xbc21, 0x1b57: 0x10e1,
+	0x1b58: 0x10f9, 0x1b59: 0xbc39, 0x1b5a: 0x2079, 0x1b5b: 0xbc71, 0x1b5c: 0xbb19, 0x1b5d: 0x1429,
+	0x1b5e: 0xbb61, 0x1b5f: 0x10e1, 0x1b60: 0x1111, 0x1b61: 0x2109, 0x1b62: 0xbb01, 0x1b63: 0x1099,
+	0x1b64: 0x10b1, 0x1b65: 0x10c9, 0x1b66: 0xbb19, 0x1b67: 0xbb31, 0x1b68: 0xbb49, 0x1b69: 0x1429,
+	0x1b6a: 0x1a31, 0x1b6b: 0xbb61, 0x1b6c: 0xbb79, 0x1b6d: 0xbb91, 0x1b6e: 0xbba9, 0x1b6f: 0xbbc1,
+	0x1b70: 0xbbd9, 0x1b71: 0x2109, 0x1b72: 0x1111, 0x1b73: 0x1429, 0x1b74: 0xbbf1, 0x1b75: 0xbc09,
+	0x1b76: 0xbc21, 0x1b77: 0x10e1, 0x1b78: 0x10f9, 0x1b79: 0xbc39, 0x1b7a: 0x2079, 0x1b7b: 0xbc51,
+	0x1b7c: 0xbb01, 0x1b7d: 0x1099, 0x1b7e: 0x10b1, 0x1b7f: 0x10c9,
+	// Block 0x6e, offset 0x1b80
+	0x1b80: 0xbb19, 0x1b81: 0xbb31, 0x1b82: 0xbb49, 0x1b83: 0x1429, 0x1b84: 0x1a31, 0x1b85: 0xbb61,
+	0x1b86: 0xbb79, 0x1b87: 0xbb91, 0x1b88: 0xbba9, 0x1b89: 0xbbc1, 0x1b8a: 0xbbd9, 0x1b8b: 0x2109,
+	0x1b8c: 0x1111, 0x1b8d: 0xbbf1, 0x1b8e: 0xbbf1, 0x1b8f: 0xbc09, 0x1b90: 0xbc21, 0x1b91: 0x10e1,
+	0x1b92: 0x10f9, 0x1b93: 0xbc39, 0x1b94: 0x2079, 0x1b95: 0xbc71, 0x1b96: 0xbb19, 0x1b97: 0x1429,
+	0x1b98: 0xbb61, 0x1b99: 0x10e1, 0x1b9a: 0x1111, 0x1b9b: 0x2109, 0x1b9c: 0xbb01, 0x1b9d: 0x1099,
+	0x1b9e: 0x10b1, 0x1b9f: 0x10c9, 0x1ba0: 0xbb19, 0x1ba1: 0xbb31, 0x1ba2: 0xbb49, 0x1ba3: 0x1429,
+	0x1ba4: 0x1a31, 0x1ba5: 0xbb61, 0x1ba6: 0xbb79, 0x1ba7: 0xbb91, 0x1ba8: 0xbba9, 0x1ba9: 0xbbc1,
+	0x1baa: 0xbbd9, 0x1bab: 0x2109, 0x1bac: 0x1111, 0x1bad: 0x1429, 0x1bae: 0xbbf1, 0x1baf: 0xbc09,
+	0x1bb0: 0xbc21, 0x1bb1: 0x10e1, 0x1bb2: 0x10f9, 0x1bb3: 0xbc39, 0x1bb4: 0x2079, 0x1bb5: 0xbc51,
+	0x1bb6: 0xbb01, 0x1bb7: 0x1099, 0x1bb8: 0x10b1, 0x1bb9: 0x10c9, 0x1bba: 0xbb19, 0x1bbb: 0xbb31,
+	0x1bbc: 0xbb49, 0x1bbd: 0x1429, 0x1bbe: 0x1a31, 0x1bbf: 0xbb61,
+	// Block 0x6f, offset 0x1bc0
+	0x1bc0: 0xbb79, 0x1bc1: 0xbb91, 0x1bc2: 0xbba9, 0x1bc3: 0xbbc1, 0x1bc4: 0xbbd9, 0x1bc5: 0x2109,
+	0x1bc6: 0x1111, 0x1bc7: 0xbbf1, 0x1bc8: 0xbbf1, 0x1bc9: 0xbc09, 0x1bca: 0xbc21, 0x1bcb: 0x10e1,
+	0x1bcc: 0x10f9, 0x1bcd: 0xbc39, 0x1bce: 0x2079, 0x1bcf: 0xbc71, 0x1bd0: 0xbb19, 0x1bd1: 0x1429,
+	0x1bd2: 0xbb61, 0x1bd3: 0x10e1, 0x1bd4: 0x1111, 0x1bd5: 0x2109, 0x1bd6: 0xbb01, 0x1bd7: 0x1099,
+	0x1bd8: 0x10b1, 0x1bd9: 0x10c9, 0x1bda: 0xbb19, 0x1bdb: 0xbb31, 0x1bdc: 0xbb49, 0x1bdd: 0x1429,
+	0x1bde: 0x1a31, 0x1bdf: 0xbb61, 0x1be0: 0xbb79, 0x1be1: 0xbb91, 0x1be2: 0xbba9, 0x1be3: 0xbbc1,
+	0x1be4: 0xbbd9, 0x1be5: 0x2109, 0x1be6: 0x1111, 0x1be7: 0x1429, 0x1be8: 0xbbf1, 0x1be9: 0xbc09,
+	0x1bea: 0xbc21, 0x1beb: 0x10e1, 0x1bec: 0x10f9, 0x1bed: 0xbc39, 0x1bee: 0x2079, 0x1bef: 0xbc51,
+	0x1bf0: 0xbb01, 0x1bf1: 0x1099, 0x1bf2: 0x10b1, 0x1bf3: 0x10c9, 0x1bf4: 0xbb19, 0x1bf5: 0xbb31,
+	0x1bf6: 0xbb49, 0x1bf7: 0x1429, 0x1bf8: 0x1a31, 0x1bf9: 0xbb61, 0x1bfa: 0xbb79, 0x1bfb: 0xbb91,
+	0x1bfc: 0xbba9, 0x1bfd: 0xbbc1, 0x1bfe: 0xbbd9, 0x1bff: 0x2109,
+	// Block 0x70, offset 0x1c00
+	0x1c00: 0x1111, 0x1c01: 0xbbf1, 0x1c02: 0xbbf1, 0x1c03: 0xbc09, 0x1c04: 0xbc21, 0x1c05: 0x10e1,
+	0x1c06: 0x10f9, 0x1c07: 0xbc39, 0x1c08: 0x2079, 0x1c09: 0xbc71, 0x1c0a: 0xbb19, 0x1c0b: 0x1429,
+	0x1c0c: 0xbb61, 0x1c0d: 0x10e1, 0x1c0e: 0x1111, 0x1c0f: 0x2109, 0x1c10: 0xbb01, 0x1c11: 0x1099,
+	0x1c12: 0x10b1, 0x1c13: 0x10c9, 0x1c14: 0xbb19, 0x1c15: 0xbb31, 0x1c16: 0xbb49, 0x1c17: 0x1429,
+	0x1c18: 0x1a31, 0x1c19: 0xbb61, 0x1c1a: 0xbb79, 0x1c1b: 0xbb91, 0x1c1c: 0xbba9, 0x1c1d: 0xbbc1,
+	0x1c1e: 0xbbd9, 0x1c1f: 0x2109, 0x1c20: 0x1111, 0x1c21: 0x1429, 0x1c22: 0xbbf1, 0x1c23: 0xbc09,
+	0x1c24: 0xbc21, 0x1c25: 0x10e1, 0x1c26: 0x10f9, 0x1c27: 0xbc39, 0x1c28: 0x2079, 0x1c29: 0xbc51,
+	0x1c2a: 0xbb01, 0x1c2b: 0x1099, 0x1c2c: 0x10b1, 0x1c2d: 0x10c9, 0x1c2e: 0xbb19, 0x1c2f: 0xbb31,
+	0x1c30: 0xbb49, 0x1c31: 0x1429, 0x1c32: 0x1a31, 0x1c33: 0xbb61, 0x1c34: 0xbb79, 0x1c35: 0xbb91,
+	0x1c36: 0xbba9, 0x1c37: 0xbbc1, 0x1c38: 0xbbd9, 0x1c39: 0x2109, 0x1c3a: 0x1111, 0x1c3b: 0xbbf1,
+	0x1c3c: 0xbbf1, 0x1c3d: 0xbc09, 0x1c3e: 0xbc21, 0x1c3f: 0x10e1,
+	// Block 0x71, offset 0x1c40
+	0x1c40: 0x10f9, 0x1c41: 0xbc39, 0x1c42: 0x2079, 0x1c43: 0xbc71, 0x1c44: 0xbb19, 0x1c45: 0x1429,
+	0x1c46: 0xbb61, 0x1c47: 0x10e1, 0x1c48: 0x1111, 0x1c49: 0x2109, 0x1c4a: 0xbc91, 0x1c4b: 0xbc91,
+	0x1c4c: 0x0040, 0x1c4d: 0x0040, 0x1c4e: 0x1f41, 0x1c4f: 0x00c9, 0x1c50: 0x0069, 0x1c51: 0x0079,
+	0x1c52: 0x1f51, 0x1c53: 0x1f61, 0x1c54: 0x1f71, 0x1c55: 0x1f81, 0x1c56: 0x1f91, 0x1c57: 0x1fa1,
+	0x1c58: 0x1f41, 0x1c59: 0x00c9, 0x1c5a: 0x0069, 0x1c5b: 0x0079, 0x1c5c: 0x1f51, 0x1c5d: 0x1f61,
+	0x1c5e: 0x1f71, 0x1c5f: 0x1f81, 0x1c60: 0x1f91, 0x1c61: 0x1fa1, 0x1c62: 0x1f41, 0x1c63: 0x00c9,
+	0x1c64: 0x0069, 0x1c65: 0x0079, 0x1c66: 0x1f51, 0x1c67: 0x1f61, 0x1c68: 0x1f71, 0x1c69: 0x1f81,
+	0x1c6a: 0x1f91, 0x1c6b: 0x1fa1, 0x1c6c: 0x1f41, 0x1c6d: 0x00c9, 0x1c6e: 0x0069, 0x1c6f: 0x0079,
+	0x1c70: 0x1f51, 0x1c71: 0x1f61, 0x1c72: 0x1f71, 0x1c73: 0x1f81, 0x1c74: 0x1f91, 0x1c75: 0x1fa1,
+	0x1c76: 0x1f41, 0x1c77: 0x00c9, 0x1c78: 0x0069, 0x1c79: 0x0079, 0x1c7a: 0x1f51, 0x1c7b: 0x1f61,
+	0x1c7c: 0x1f71, 0x1c7d: 0x1f81, 0x1c7e: 0x1f91, 0x1c7f: 0x1fa1,
+	// Block 0x72, offset 0x1c80
+	0x1c80: 0xe115, 0x1c81: 0xe115, 0x1c82: 0xe135, 0x1c83: 0xe135, 0x1c84: 0xe115, 0x1c85: 0xe115,
+	0x1c86: 0xe175, 0x1c87: 0xe175, 0x1c88: 0xe115, 0x1c89: 0xe115, 0x1c8a: 0xe135, 0x1c8b: 0xe135,
+	0x1c8c: 0xe115, 0x1c8d: 0xe115, 0x1c8e: 0xe1f5, 0x1c8f: 0xe1f5, 0x1c90: 0xe115, 0x1c91: 0xe115,
+	0x1c92: 0xe135, 0x1c93: 0xe135, 0x1c94: 0xe115, 0x1c95: 0xe115, 0x1c96: 0xe175, 0x1c97: 0xe175,
+	0x1c98: 0xe115, 0x1c99: 0xe115, 0x1c9a: 0xe135, 0x1c9b: 0xe135, 0x1c9c: 0xe115, 0x1c9d: 0xe115,
+	0x1c9e: 0x8b3d, 0x1c9f: 0x8b3d, 0x1ca0: 0x04b5, 0x1ca1: 0x04b5, 0x1ca2: 0x0a08, 0x1ca3: 0x0a08,
+	0x1ca4: 0x0a08, 0x1ca5: 0x0a08, 0x1ca6: 0x0a08, 0x1ca7: 0x0a08, 0x1ca8: 0x0a08, 0x1ca9: 0x0a08,
+	0x1caa: 0x0a08, 0x1cab: 0x0a08, 0x1cac: 0x0a08, 0x1cad: 0x0a08, 0x1cae: 0x0a08, 0x1caf: 0x0a08,
+	0x1cb0: 0x0a08, 0x1cb1: 0x0a08, 0x1cb2: 0x0a08, 0x1cb3: 0x0a08, 0x1cb4: 0x0a08, 0x1cb5: 0x0a08,
+	0x1cb6: 0x0a08, 0x1cb7: 0x0a08, 0x1cb8: 0x0a08, 0x1cb9: 0x0a08, 0x1cba: 0x0a08, 0x1cbb: 0x0a08,
+	0x1cbc: 0x0a08, 0x1cbd: 0x0a08, 0x1cbe: 0x0a08, 0x1cbf: 0x0a08,
+	// Block 0x73, offset 0x1cc0
+	0x1cc0: 0xb1d9, 0x1cc1: 0xb1f1, 0x1cc2: 0xb251, 0x1cc3: 0xb299, 0x1cc4: 0x0040, 0x1cc5: 0xb461,
+	0x1cc6: 0xb2e1, 0x1cc7: 0xb269, 0x1cc8: 0xb359, 0x1cc9: 0xb479, 0x1cca: 0xb3e9, 0x1ccb: 0xb401,
+	0x1ccc: 0xb419, 0x1ccd: 0xb431, 0x1cce: 0xb2f9, 0x1ccf: 0xb389, 0x1cd0: 0xb3b9, 0x1cd1: 0xb329,
+	0x1cd2: 0xb3d1, 0x1cd3: 0xb2c9, 0x1cd4: 0xb311, 0x1cd5: 0xb221, 0x1cd6: 0xb239, 0x1cd7: 0xb281,
+	0x1cd8: 0xb2b1, 0x1cd9: 0xb341, 0x1cda: 0xb371, 0x1cdb: 0xb3a1, 0x1cdc: 0xbca9, 0x1cdd: 0x7999,
+	0x1cde: 0xbcc1, 0x1cdf: 0xbcd9, 0x1ce0: 0x0040, 0x1ce1: 0xb1f1, 0x1ce2: 0xb251, 0x1ce3: 0x0040,
+	0x1ce4: 0xb449, 0x1ce5: 0x0040, 0x1ce6: 0x0040, 0x1ce7: 0xb269, 0x1ce8: 0x0040, 0x1ce9: 0xb479,
+	0x1cea: 0xb3e9, 0x1ceb: 0xb401, 0x1cec: 0xb419, 0x1ced: 0xb431, 0x1cee: 0xb2f9, 0x1cef: 0xb389,
+	0x1cf0: 0xb3b9, 0x1cf1: 0xb329, 0x1cf2: 0xb3d1, 0x1cf3: 0x0040, 0x1cf4: 0xb311, 0x1cf5: 0xb221,
+	0x1cf6: 0xb239, 0x1cf7: 0xb281, 0x1cf8: 0x0040, 0x1cf9: 0xb341, 0x1cfa: 0x0040, 0x1cfb: 0xb3a1,
+	0x1cfc: 0x0040, 0x1cfd: 0x0040, 0x1cfe: 0x0040, 0x1cff: 0x0040,
+	// Block 0x74, offset 0x1d00
+	0x1d00: 0x0040, 0x1d01: 0x0040, 0x1d02: 0xb251, 0x1d03: 0x0040, 0x1d04: 0x0040, 0x1d05: 0x0040,
+	0x1d06: 0x0040, 0x1d07: 0xb269, 0x1d08: 0x0040, 0x1d09: 0xb479, 0x1d0a: 0x0040, 0x1d0b: 0xb401,
+	0x1d0c: 0x0040, 0x1d0d: 0xb431, 0x1d0e: 0xb2f9, 0x1d0f: 0xb389, 0x1d10: 0x0040, 0x1d11: 0xb329,
+	0x1d12: 0xb3d1, 0x1d13: 0x0040, 0x1d14: 0xb311, 0x1d15: 0x0040, 0x1d16: 0x0040, 0x1d17: 0xb281,
+	0x1d18: 0x0040, 0x1d19: 0xb341, 0x1d1a: 0x0040, 0x1d1b: 0xb3a1, 0x1d1c: 0x0040, 0x1d1d: 0x7999,
+	0x1d1e: 0x0040, 0x1d1f: 0xbcd9, 0x1d20: 0x0040, 0x1d21: 0xb1f1, 0x1d22: 0xb251, 0x1d23: 0x0040,
+	0x1d24: 0xb449, 0x1d25: 0x0040, 0x1d26: 0x0040, 0x1d27: 0xb269, 0x1d28: 0xb359, 0x1d29: 0xb479,
+	0x1d2a: 0xb3e9, 0x1d2b: 0x0040, 0x1d2c: 0xb419, 0x1d2d: 0xb431, 0x1d2e: 0xb2f9, 0x1d2f: 0xb389,
+	0x1d30: 0xb3b9, 0x1d31: 0xb329, 0x1d32: 0xb3d1, 0x1d33: 0x0040, 0x1d34: 0xb311, 0x1d35: 0xb221,
+	0x1d36: 0xb239, 0x1d37: 0xb281, 0x1d38: 0x0040, 0x1d39: 0xb341, 0x1d3a: 0xb371, 0x1d3b: 0xb3a1,
+	0x1d3c: 0xbca9, 0x1d3d: 0x0040, 0x1d3e: 0xbcc1, 0x1d3f: 0x0040,
+	// Block 0x75, offset 0x1d40
+	0x1d40: 0xb1d9, 0x1d41: 0xb1f1, 0x1d42: 0xb251, 0x1d43: 0xb299, 0x1d44: 0xb449, 0x1d45: 0xb461,
+	0x1d46: 0xb2e1, 0x1d47: 0xb269, 0x1d48: 0xb359, 0x1d49: 0xb479, 0x1d4a: 0x0040, 0x1d4b: 0xb401,
+	0x1d4c: 0xb419, 0x1d4d: 0xb431, 0x1d4e: 0xb2f9, 0x1d4f: 0xb389, 0x1d50: 0xb3b9, 0x1d51: 0xb329,
+	0x1d52: 0xb3d1, 0x1d53: 0xb2c9, 0x1d54: 0xb311, 0x1d55: 0xb221, 0x1d56: 0xb239, 0x1d57: 0xb281,
+	0x1d58: 0xb2b1, 0x1d59: 0xb341, 0x1d5a: 0xb371, 0x1d5b: 0xb3a1, 0x1d5c: 0x0040, 0x1d5d: 0x0040,
+	0x1d5e: 0x0040, 0x1d5f: 0x0040, 0x1d60: 0x0040, 0x1d61: 0xb1f1, 0x1d62: 0xb251, 0x1d63: 0xb299,
+	0x1d64: 0x0040, 0x1d65: 0xb461, 0x1d66: 0xb2e1, 0x1d67: 0xb269, 0x1d68: 0xb359, 0x1d69: 0xb479,
+	0x1d6a: 0x0040, 0x1d6b: 0xb401, 0x1d6c: 0xb419, 0x1d6d: 0xb431, 0x1d6e: 0xb2f9, 0x1d6f: 0xb389,
+	0x1d70: 0xb3b9, 0x1d71: 0xb329, 0x1d72: 0xb3d1, 0x1d73: 0xb2c9, 0x1d74: 0xb311, 0x1d75: 0xb221,
+	0x1d76: 0xb239, 0x1d77: 0xb281, 0x1d78: 0xb2b1, 0x1d79: 0xb341, 0x1d7a: 0xb371, 0x1d7b: 0xb3a1,
+	0x1d7c: 0x0040, 0x1d7d: 0x0040, 0x1d7e: 0x0040, 0x1d7f: 0x0040,
+	// Block 0x76, offset 0x1d80
+	0x1d80: 0x0040, 0x1d81: 0xbcf2, 0x1d82: 0xbd0a, 0x1d83: 0xbd22, 0x1d84: 0xbd3a, 0x1d85: 0xbd52,
+	0x1d86: 0xbd6a, 0x1d87: 0xbd82, 0x1d88: 0xbd9a, 0x1d89: 0xbdb2, 0x1d8a: 0xbdca, 0x1d8b: 0x0018,
+	0x1d8c: 0x0018, 0x1d8d: 0x0018, 0x1d8e: 0x0018, 0x1d8f: 0x0018, 0x1d90: 0xbde2, 0x1d91: 0xbe02,
+	0x1d92: 0xbe22, 0x1d93: 0xbe42, 0x1d94: 0xbe62, 0x1d95: 0xbe82, 0x1d96: 0xbea2, 0x1d97: 0xbec2,
+	0x1d98: 0xbee2, 0x1d99: 0xbf02, 0x1d9a: 0xbf22, 0x1d9b: 0xbf42, 0x1d9c: 0xbf62, 0x1d9d: 0xbf82,
+	0x1d9e: 0xbfa2, 0x1d9f: 0xbfc2, 0x1da0: 0xbfe2, 0x1da1: 0xc002, 0x1da2: 0xc022, 0x1da3: 0xc042,
+	0x1da4: 0xc062, 0x1da5: 0xc082, 0x1da6: 0xc0a2, 0x1da7: 0xc0c2, 0x1da8: 0xc0e2, 0x1da9: 0xc102,
+	0x1daa: 0xc121, 0x1dab: 0x1159, 0x1dac: 0x0269, 0x1dad: 0x66a9, 0x1dae: 0xc161, 0x1daf: 0x0018,
+	0x1db0: 0x0039, 0x1db1: 0x0ee9, 0x1db2: 0x1159, 0x1db3: 0x0ef9, 0x1db4: 0x0f09, 0x1db5: 0x1199,
+	0x1db6: 0x0f31, 0x1db7: 0x0249, 0x1db8: 0x0f41, 0x1db9: 0x0259, 0x1dba: 0x0f51, 0x1dbb: 0x0359,
+	0x1dbc: 0x0f61, 0x1dbd: 0x0f71, 0x1dbe: 0x00d9, 0x1dbf: 0x0f99,
+	// Block 0x77, offset 0x1dc0
+	0x1dc0: 0x2039, 0x1dc1: 0x0269, 0x1dc2: 0x01d9, 0x1dc3: 0x0fa9, 0x1dc4: 0x0fb9, 0x1dc5: 0x1089,
+	0x1dc6: 0x0279, 0x1dc7: 0x0369, 0x1dc8: 0x0289, 0x1dc9: 0x13d1, 0x1dca: 0xc179, 0x1dcb: 0x65e9,
+	0x1dcc: 0xc191, 0x1dcd: 0x1441, 0x1dce: 0xc1a9, 0x1dcf: 0xc1c9, 0x1dd0: 0x0018, 0x1dd1: 0x0018,
+	0x1dd2: 0x0018, 0x1dd3: 0x0018, 0x1dd4: 0x0018, 0x1dd5: 0x0018, 0x1dd6: 0x0018, 0x1dd7: 0x0018,
+	0x1dd8: 0x0018, 0x1dd9: 0x0018, 0x1dda: 0x0018, 0x1ddb: 0x0018, 0x1ddc: 0x0018, 0x1ddd: 0x0018,
+	0x1dde: 0x0018, 0x1ddf: 0x0018, 0x1de0: 0x0018, 0x1de1: 0x0018, 0x1de2: 0x0018, 0x1de3: 0x0018,
+	0x1de4: 0x0018, 0x1de5: 0x0018, 0x1de6: 0x0018, 0x1de7: 0x0018, 0x1de8: 0x0018, 0x1de9: 0x0018,
+	0x1dea: 0xc1e1, 0x1deb: 0xc1f9, 0x1dec: 0xc211, 0x1ded: 0x0018, 0x1dee: 0x0018, 0x1def: 0x0018,
+	0x1df0: 0x0018, 0x1df1: 0x0018, 0x1df2: 0x0018, 0x1df3: 0x0018, 0x1df4: 0x0018, 0x1df5: 0x0018,
+	0x1df6: 0x0018, 0x1df7: 0x0018, 0x1df8: 0x0018, 0x1df9: 0x0018, 0x1dfa: 0x0018, 0x1dfb: 0x0018,
+	0x1dfc: 0x0018, 0x1dfd: 0x0018, 0x1dfe: 0x0018, 0x1dff: 0x0018,
+	// Block 0x78, offset 0x1e00
+	0x1e00: 0xc241, 0x1e01: 0xc279, 0x1e02: 0xc2b1, 0x1e03: 0x0040, 0x1e04: 0x0040, 0x1e05: 0x0040,
+	0x1e06: 0x0040, 0x1e07: 0x0040, 0x1e08: 0x0040, 0x1e09: 0x0040, 0x1e0a: 0x0040, 0x1e0b: 0x0040,
+	0x1e0c: 0x0040, 0x1e0d: 0x0040, 0x1e0e: 0x0040, 0x1e0f: 0x0040, 0x1e10: 0xc2d1, 0x1e11: 0xc2f1,
+	0x1e12: 0xc311, 0x1e13: 0xc331, 0x1e14: 0xc351, 0x1e15: 0xc371, 0x1e16: 0xc391, 0x1e17: 0xc3b1,
+	0x1e18: 0xc3d1, 0x1e19: 0xc3f1, 0x1e1a: 0xc411, 0x1e1b: 0xc431, 0x1e1c: 0xc451, 0x1e1d: 0xc471,
+	0x1e1e: 0xc491, 0x1e1f: 0xc4b1, 0x1e20: 0xc4d1, 0x1e21: 0xc4f1, 0x1e22: 0xc511, 0x1e23: 0xc531,
+	0x1e24: 0xc551, 0x1e25: 0xc571, 0x1e26: 0xc591, 0x1e27: 0xc5b1, 0x1e28: 0xc5d1, 0x1e29: 0xc5f1,
+	0x1e2a: 0xc611, 0x1e2b: 0xc631, 0x1e2c: 0xc651, 0x1e2d: 0xc671, 0x1e2e: 0xc691, 0x1e2f: 0xc6b1,
+	0x1e30: 0xc6d1, 0x1e31: 0xc6f1, 0x1e32: 0xc711, 0x1e33: 0xc731, 0x1e34: 0xc751, 0x1e35: 0xc771,
+	0x1e36: 0xc791, 0x1e37: 0xc7b1, 0x1e38: 0xc7d1, 0x1e39: 0xc7f1, 0x1e3a: 0xc811, 0x1e3b: 0xc831,
+	0x1e3c: 0x0040, 0x1e3d: 0x0040, 0x1e3e: 0x0040, 0x1e3f: 0x0040,
+	// Block 0x79, offset 0x1e40
+	0x1e40: 0xcb61, 0x1e41: 0xcb81, 0x1e42: 0xcba1, 0x1e43: 0x8b55, 0x1e44: 0xcbc1, 0x1e45: 0xcbe1,
+	0x1e46: 0xcc01, 0x1e47: 0xcc21, 0x1e48: 0xcc41, 0x1e49: 0xcc61, 0x1e4a: 0xcc81, 0x1e4b: 0xcca1,
+	0x1e4c: 0xccc1, 0x1e4d: 0x8b75, 0x1e4e: 0xcce1, 0x1e4f: 0xcd01, 0x1e50: 0xcd21, 0x1e51: 0xcd41,
+	0x1e52: 0x8b95, 0x1e53: 0xcd61, 0x1e54: 0xcd81, 0x1e55: 0xc491, 0x1e56: 0x8bb5, 0x1e57: 0xcda1,
+	0x1e58: 0xcdc1, 0x1e59: 0xcde1, 0x1e5a: 0xce01, 0x1e5b: 0xce21, 0x1e5c: 0x8bd5, 0x1e5d: 0xce41,
+	0x1e5e: 0xce61, 0x1e5f: 0xce81, 0x1e60: 0xcea1, 0x1e61: 0xcec1, 0x1e62: 0xc7f1, 0x1e63: 0xcee1,
+	0x1e64: 0xcf01, 0x1e65: 0xcf21, 0x1e66: 0xcf41, 0x1e67: 0xcf61, 0x1e68: 0xcf81, 0x1e69: 0xcfa1,
+	0x1e6a: 0xcfc1, 0x1e6b: 0xcfe1, 0x1e6c: 0xd001, 0x1e6d: 0xd021, 0x1e6e: 0xd041, 0x1e6f: 0xd061,
+	0x1e70: 0xd081, 0x1e71: 0xd0a1, 0x1e72: 0xd0a1, 0x1e73: 0xd0a1, 0x1e74: 0x8bf5, 0x1e75: 0xd0c1,
+	0x1e76: 0xd0e1, 0x1e77: 0xd101, 0x1e78: 0x8c15, 0x1e79: 0xd121, 0x1e7a: 0xd141, 0x1e7b: 0xd161,
+	0x1e7c: 0xd181, 0x1e7d: 0xd1a1, 0x1e7e: 0xd1c1, 0x1e7f: 0xd1e1,
+	// Block 0x7a, offset 0x1e80
+	0x1e80: 0xd201, 0x1e81: 0xd221, 0x1e82: 0xd241, 0x1e83: 0xd261, 0x1e84: 0xd281, 0x1e85: 0xd2a1,
+	0x1e86: 0xd2a1, 0x1e87: 0xd2c1, 0x1e88: 0xd2e1, 0x1e89: 0xd301, 0x1e8a: 0xd321, 0x1e8b: 0xd341,
+	0x1e8c: 0xd361, 0x1e8d: 0xd381, 0x1e8e: 0xd3a1, 0x1e8f: 0xd3c1, 0x1e90: 0xd3e1, 0x1e91: 0xd401,
+	0x1e92: 0xd421, 0x1e93: 0xd441, 0x1e94: 0xd461, 0x1e95: 0xd481, 0x1e96: 0xd4a1, 0x1e97: 0xd4c1,
+	0x1e98: 0xd4e1, 0x1e99: 0x8c35, 0x1e9a: 0xd501, 0x1e9b: 0xd521, 0x1e9c: 0xd541, 0x1e9d: 0xc371,
+	0x1e9e: 0xd561, 0x1e9f: 0xd581, 0x1ea0: 0x8c55, 0x1ea1: 0x8c75, 0x1ea2: 0xd5a1, 0x1ea3: 0xd5c1,
+	0x1ea4: 0xd5e1, 0x1ea5: 0xd601, 0x1ea6: 0xd621, 0x1ea7: 0xd641, 0x1ea8: 0x2040, 0x1ea9: 0xd661,
+	0x1eaa: 0xd681, 0x1eab: 0xd681, 0x1eac: 0x8c95, 0x1ead: 0xd6a1, 0x1eae: 0xd6c1, 0x1eaf: 0xd6e1,
+	0x1eb0: 0xd701, 0x1eb1: 0x8cb5, 0x1eb2: 0xd721, 0x1eb3: 0xd741, 0x1eb4: 0x2040, 0x1eb5: 0xd761,
+	0x1eb6: 0xd781, 0x1eb7: 0xd7a1, 0x1eb8: 0xd7c1, 0x1eb9: 0xd7e1, 0x1eba: 0xd801, 0x1ebb: 0x8cd5,
+	0x1ebc: 0xd821, 0x1ebd: 0x8cf5, 0x1ebe: 0xd841, 0x1ebf: 0xd861,
+	// Block 0x7b, offset 0x1ec0
+	0x1ec0: 0xd881, 0x1ec1: 0xd8a1, 0x1ec2: 0xd8c1, 0x1ec3: 0xd8e1, 0x1ec4: 0xd901, 0x1ec5: 0xd921,
+	0x1ec6: 0xd941, 0x1ec7: 0xd961, 0x1ec8: 0xd981, 0x1ec9: 0x8d15, 0x1eca: 0xd9a1, 0x1ecb: 0xd9c1,
+	0x1ecc: 0xd9e1, 0x1ecd: 0xda01, 0x1ece: 0xda21, 0x1ecf: 0x8d35, 0x1ed0: 0xda41, 0x1ed1: 0x8d55,
+	0x1ed2: 0x8d75, 0x1ed3: 0xda61, 0x1ed4: 0xda81, 0x1ed5: 0xda81, 0x1ed6: 0xdaa1, 0x1ed7: 0x8d95,
+	0x1ed8: 0x8db5, 0x1ed9: 0xdac1, 0x1eda: 0xdae1, 0x1edb: 0xdb01, 0x1edc: 0xdb21, 0x1edd: 0xdb41,
+	0x1ede: 0xdb61, 0x1edf: 0xdb81, 0x1ee0: 0xdba1, 0x1ee1: 0xdbc1, 0x1ee2: 0xdbe1, 0x1ee3: 0xdc01,
+	0x1ee4: 0x8dd5, 0x1ee5: 0xdc21, 0x1ee6: 0xdc41, 0x1ee7: 0xdc61, 0x1ee8: 0xdc81, 0x1ee9: 0xdc61,
+	0x1eea: 0xdca1, 0x1eeb: 0xdcc1, 0x1eec: 0xdce1, 0x1eed: 0xdd01, 0x1eee: 0xdd21, 0x1eef: 0xdd41,
+	0x1ef0: 0xdd61, 0x1ef1: 0xdd81, 0x1ef2: 0xdda1, 0x1ef3: 0xddc1, 0x1ef4: 0xdde1, 0x1ef5: 0xde01,
+	0x1ef6: 0xde21, 0x1ef7: 0xde41, 0x1ef8: 0x8df5, 0x1ef9: 0xde61, 0x1efa: 0xde81, 0x1efb: 0xdea1,
+	0x1efc: 0xdec1, 0x1efd: 0xdee1, 0x1efe: 0x8e15, 0x1eff: 0xdf01,
+	// Block 0x7c, offset 0x1f00
+	0x1f00: 0xe601, 0x1f01: 0xe621, 0x1f02: 0xe641, 0x1f03: 0xe661, 0x1f04: 0xe681, 0x1f05: 0xe6a1,
+	0x1f06: 0x8f35, 0x1f07: 0xe6c1, 0x1f08: 0xe6e1, 0x1f09: 0xe701, 0x1f0a: 0xe721, 0x1f0b: 0xe741,
+	0x1f0c: 0xe761, 0x1f0d: 0x8f55, 0x1f0e: 0xe781, 0x1f0f: 0xe7a1, 0x1f10: 0x8f75, 0x1f11: 0x8f95,
+	0x1f12: 0xe7c1, 0x1f13: 0xe7e1, 0x1f14: 0xe801, 0x1f15: 0xe821, 0x1f16: 0xe841, 0x1f17: 0xe861,
+	0x1f18: 0xe881, 0x1f19: 0xe8a1, 0x1f1a: 0xe8c1, 0x1f1b: 0x8fb5, 0x1f1c: 0xe8e1, 0x1f1d: 0x8fd5,
+	0x1f1e: 0xe901, 0x1f1f: 0x2040, 0x1f20: 0xe921, 0x1f21: 0xe941, 0x1f22: 0xe961, 0x1f23: 0x8ff5,
+	0x1f24: 0xe981, 0x1f25: 0xe9a1, 0x1f26: 0x9015, 0x1f27: 0x9035, 0x1f28: 0xe9c1, 0x1f29: 0xe9e1,
+	0x1f2a: 0xea01, 0x1f2b: 0xea21, 0x1f2c: 0xea41, 0x1f2d: 0xea41, 0x1f2e: 0xea61, 0x1f2f: 0xea81,
+	0x1f30: 0xeaa1, 0x1f31: 0xeac1, 0x1f32: 0xeae1, 0x1f33: 0xeb01, 0x1f34: 0xeb21, 0x1f35: 0x9055,
+	0x1f36: 0xeb41, 0x1f37: 0x9075, 0x1f38: 0xeb61, 0x1f39: 0x9095, 0x1f3a: 0xeb81, 0x1f3b: 0x90b5,
+	0x1f3c: 0x90d5, 0x1f3d: 0x90f5, 0x1f3e: 0xeba1, 0x1f3f: 0xebc1,
+	// Block 0x7d, offset 0x1f40
+	0x1f40: 0xebe1, 0x1f41: 0x9115, 0x1f42: 0x9135, 0x1f43: 0x9155, 0x1f44: 0x9175, 0x1f45: 0xec01,
+	0x1f46: 0xec21, 0x1f47: 0xec21, 0x1f48: 0xec41, 0x1f49: 0xec61, 0x1f4a: 0xec81, 0x1f4b: 0xeca1,
+	0x1f4c: 0xecc1, 0x1f4d: 0x9195, 0x1f4e: 0xece1, 0x1f4f: 0xed01, 0x1f50: 0xed21, 0x1f51: 0xed41,
+	0x1f52: 0x91b5, 0x1f53: 0xed61, 0x1f54: 0x91d5, 0x1f55: 0x91f5, 0x1f56: 0xed81, 0x1f57: 0xeda1,
+	0x1f58: 0xedc1, 0x1f59: 0xede1, 0x1f5a: 0xee01, 0x1f5b: 0xee21, 0x1f5c: 0x9215, 0x1f5d: 0x9235,
+	0x1f5e: 0x9255, 0x1f5f: 0x2040, 0x1f60: 0xee41, 0x1f61: 0x9275, 0x1f62: 0xee61, 0x1f63: 0xee81,
+	0x1f64: 0xeea1, 0x1f65: 0x9295, 0x1f66: 0xeec1, 0x1f67: 0xeee1, 0x1f68: 0xef01, 0x1f69: 0xef21,
+	0x1f6a: 0xef41, 0x1f6b: 0x92b5, 0x1f6c: 0xef61, 0x1f6d: 0xef81, 0x1f6e: 0xefa1, 0x1f6f: 0xefc1,
+	0x1f70: 0xefe1, 0x1f71: 0xf001, 0x1f72: 0x92d5, 0x1f73: 0x92f5, 0x1f74: 0xf021, 0x1f75: 0x9315,
+	0x1f76: 0xf041, 0x1f77: 0x9335, 0x1f78: 0xf061, 0x1f79: 0xf081, 0x1f7a: 0xf0a1, 0x1f7b: 0x9355,
+	0x1f7c: 0x9375, 0x1f7d: 0xf0c1, 0x1f7e: 0x9395, 0x1f7f: 0xf0e1,
+	// Block 0x7e, offset 0x1f80
+	0x1f80: 0xf721, 0x1f81: 0xf741, 0x1f82: 0xf761, 0x1f83: 0xf781, 0x1f84: 0xf7a1, 0x1f85: 0x9555,
+	0x1f86: 0xf7c1, 0x1f87: 0xf7e1, 0x1f88: 0xf801, 0x1f89: 0xf821, 0x1f8a: 0xf841, 0x1f8b: 0x9575,
+	0x1f8c: 0x9595, 0x1f8d: 0xf861, 0x1f8e: 0xf881, 0x1f8f: 0xf8a1, 0x1f90: 0xf8c1, 0x1f91: 0xf8e1,
+	0x1f92: 0xf901, 0x1f93: 0x95b5, 0x1f94: 0xf921, 0x1f95: 0xf941, 0x1f96: 0xf961, 0x1f97: 0xf981,
+	0x1f98: 0x95d5, 0x1f99: 0x95f5, 0x1f9a: 0xf9a1, 0x1f9b: 0xf9c1, 0x1f9c: 0xf9e1, 0x1f9d: 0x9615,
+	0x1f9e: 0xfa01, 0x1f9f: 0xfa21, 0x1fa0: 0x684d, 0x1fa1: 0x9635, 0x1fa2: 0xfa41, 0x1fa3: 0xfa61,
+	0x1fa4: 0xfa81, 0x1fa5: 0x9655, 0x1fa6: 0xfaa1, 0x1fa7: 0xfac1, 0x1fa8: 0xfae1, 0x1fa9: 0xfb01,
+	0x1faa: 0xfb21, 0x1fab: 0xfb41, 0x1fac: 0xfb61, 0x1fad: 0x9675, 0x1fae: 0xfb81, 0x1faf: 0xfba1,
+	0x1fb0: 0xfbc1, 0x1fb1: 0x9695, 0x1fb2: 0xfbe1, 0x1fb3: 0xfc01, 0x1fb4: 0xfc21, 0x1fb5: 0xfc41,
+	0x1fb6: 0x7b6d, 0x1fb7: 0x96b5, 0x1fb8: 0xfc61, 0x1fb9: 0xfc81, 0x1fba: 0xfca1, 0x1fbb: 0x96d5,
+	0x1fbc: 0xfcc1, 0x1fbd: 0x96f5, 0x1fbe: 0xfce1, 0x1fbf: 0xfce1,
+	// Block 0x7f, offset 0x1fc0
+	0x1fc0: 0xfd01, 0x1fc1: 0x9715, 0x1fc2: 0xfd21, 0x1fc3: 0xfd41, 0x1fc4: 0xfd61, 0x1fc5: 0xfd81,
+	0x1fc6: 0xfda1, 0x1fc7: 0xfdc1, 0x1fc8: 0xfde1, 0x1fc9: 0x9735, 0x1fca: 0xfe01, 0x1fcb: 0xfe21,
+	0x1fcc: 0xfe41, 0x1fcd: 0xfe61, 0x1fce: 0xfe81, 0x1fcf: 0xfea1, 0x1fd0: 0x9755, 0x1fd1: 0xfec1,
+	0x1fd2: 0x9775, 0x1fd3: 0x9795, 0x1fd4: 0x97b5, 0x1fd5: 0xfee1, 0x1fd6: 0xff01, 0x1fd7: 0xff21,
+	0x1fd8: 0xff41, 0x1fd9: 0xff61, 0x1fda: 0xff81, 0x1fdb: 0xffa1, 0x1fdc: 0xffc1, 0x1fdd: 0x97d5,
+	0x1fde: 0x0040, 0x1fdf: 0x0040, 0x1fe0: 0x0040, 0x1fe1: 0x0040, 0x1fe2: 0x0040, 0x1fe3: 0x0040,
+	0x1fe4: 0x0040, 0x1fe5: 0x0040, 0x1fe6: 0x0040, 0x1fe7: 0x0040, 0x1fe8: 0x0040, 0x1fe9: 0x0040,
+	0x1fea: 0x0040, 0x1feb: 0x0040, 0x1fec: 0x0040, 0x1fed: 0x0040, 0x1fee: 0x0040, 0x1fef: 0x0040,
+	0x1ff0: 0x0040, 0x1ff1: 0x0040, 0x1ff2: 0x0040, 0x1ff3: 0x0040, 0x1ff4: 0x0040, 0x1ff5: 0x0040,
+	0x1ff6: 0x0040, 0x1ff7: 0x0040, 0x1ff8: 0x0040, 0x1ff9: 0x0040, 0x1ffa: 0x0040, 0x1ffb: 0x0040,
+	0x1ffc: 0x0040, 0x1ffd: 0x0040, 0x1ffe: 0x0040, 0x1fff: 0x0040,
+}
+
+// idnaIndex: 37 blocks, 2368 entries, 4736 bytes
+// Block 0 is the zero block.
+var idnaIndex = [2368]uint16{
+	// Block 0x0, offset 0x0
+	// Block 0x1, offset 0x40
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc2: 0x01, 0xc3: 0x7e, 0xc4: 0x02, 0xc5: 0x03, 0xc6: 0x04, 0xc7: 0x05,
+	0xc8: 0x06, 0xc9: 0x7f, 0xca: 0x80, 0xcb: 0x07, 0xcc: 0x81, 0xcd: 0x08, 0xce: 0x09, 0xcf: 0x0a,
+	0xd0: 0x82, 0xd1: 0x0b, 0xd2: 0x0c, 0xd3: 0x0d, 0xd4: 0x0e, 0xd5: 0x83, 0xd6: 0x84, 0xd7: 0x85,
+	0xd8: 0x0f, 0xd9: 0x10, 0xda: 0x86, 0xdb: 0x11, 0xdc: 0x12, 0xdd: 0x87, 0xde: 0x88, 0xdf: 0x89,
+	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05, 0xe4: 0x06, 0xe5: 0x07, 0xe6: 0x07, 0xe7: 0x07,
+	0xe8: 0x07, 0xe9: 0x08, 0xea: 0x09, 0xeb: 0x07, 0xec: 0x07, 0xed: 0x0a, 0xee: 0x0b, 0xef: 0x0c,
+	0xf0: 0x1e, 0xf1: 0x1f, 0xf2: 0x1f, 0xf3: 0x21, 0xf4: 0x22,
+	// Block 0x4, offset 0x100
+	0x120: 0x8a, 0x121: 0x13, 0x122: 0x8b, 0x123: 0x8c, 0x124: 0x8d, 0x125: 0x14, 0x126: 0x15, 0x127: 0x16,
+	0x128: 0x17, 0x129: 0x18, 0x12a: 0x19, 0x12b: 0x1a, 0x12c: 0x1b, 0x12d: 0x1c, 0x12e: 0x1d, 0x12f: 0x8e,
+	0x130: 0x8f, 0x131: 0x1e, 0x132: 0x1f, 0x133: 0x20, 0x134: 0x90, 0x135: 0x21, 0x136: 0x91, 0x137: 0x92,
+	0x138: 0x93, 0x139: 0x94, 0x13a: 0x22, 0x13b: 0x95, 0x13c: 0x96, 0x13d: 0x23, 0x13e: 0x24, 0x13f: 0x97,
+	// Block 0x5, offset 0x140
+	0x140: 0x98, 0x141: 0x99, 0x142: 0x9a, 0x143: 0x9b, 0x144: 0x9c, 0x145: 0x9d, 0x146: 0x9e, 0x147: 0x9f,
+	0x148: 0xa0, 0x149: 0xa1, 0x14a: 0xa2, 0x14b: 0xa3, 0x14c: 0xa4, 0x14d: 0xa5, 0x14e: 0xa6, 0x14f: 0xa7,
+	0x150: 0xa8, 0x151: 0xa0, 0x152: 0xa0, 0x153: 0xa0, 0x154: 0xa0, 0x155: 0xa0, 0x156: 0xa0, 0x157: 0xa0,
+	0x158: 0xa0, 0x159: 0xa9, 0x15a: 0xaa, 0x15b: 0xab, 0x15c: 0xac, 0x15d: 0xad, 0x15e: 0xae, 0x15f: 0xaf,
+	0x160: 0xb0, 0x161: 0xb1, 0x162: 0xb2, 0x163: 0xb3, 0x164: 0xb4, 0x165: 0xb5, 0x166: 0xb6, 0x167: 0xb7,
+	0x168: 0xb8, 0x169: 0xb9, 0x16a: 0xba, 0x16b: 0xbb, 0x16c: 0xbc, 0x16d: 0xbd, 0x16e: 0xbe, 0x16f: 0xbf,
+	0x170: 0xc0, 0x171: 0xc1, 0x172: 0xc2, 0x173: 0xc3, 0x174: 0x25, 0x175: 0x26, 0x176: 0x27, 0x177: 0xc4,
+	0x178: 0x28, 0x179: 0x28, 0x17a: 0x29, 0x17b: 0x28, 0x17c: 0xc5, 0x17d: 0x2a, 0x17e: 0x2b, 0x17f: 0x2c,
+	// Block 0x6, offset 0x180
+	0x180: 0x2d, 0x181: 0x2e, 0x182: 0x2f, 0x183: 0xc6, 0x184: 0x30, 0x185: 0x31, 0x186: 0xc7, 0x187: 0x9c,
+	0x188: 0xc8, 0x189: 0xc9, 0x18a: 0x9c, 0x18b: 0x9c, 0x18c: 0xca, 0x18d: 0x9c, 0x18e: 0x9c, 0x18f: 0x9c,
+	0x190: 0xcb, 0x191: 0x32, 0x192: 0x33, 0x193: 0x34, 0x194: 0x9c, 0x195: 0x9c, 0x196: 0x9c, 0x197: 0x9c,
+	0x198: 0x9c, 0x199: 0x9c, 0x19a: 0x9c, 0x19b: 0x9c, 0x19c: 0x9c, 0x19d: 0x9c, 0x19e: 0x9c, 0x19f: 0x9c,
+	0x1a0: 0x9c, 0x1a1: 0x9c, 0x1a2: 0x9c, 0x1a3: 0x9c, 0x1a4: 0x9c, 0x1a5: 0x9c, 0x1a6: 0x9c, 0x1a7: 0x9c,
+	0x1a8: 0xcc, 0x1a9: 0xcd, 0x1aa: 0x9c, 0x1ab: 0xce, 0x1ac: 0x9c, 0x1ad: 0xcf, 0x1ae: 0xd0, 0x1af: 0x9c,
+	0x1b0: 0xd1, 0x1b1: 0x35, 0x1b2: 0x28, 0x1b3: 0x36, 0x1b4: 0xd2, 0x1b5: 0xd3, 0x1b6: 0xd4, 0x1b7: 0xd5,
+	0x1b8: 0xd6, 0x1b9: 0xd7, 0x1ba: 0xd8, 0x1bb: 0xd9, 0x1bc: 0xda, 0x1bd: 0xdb, 0x1be: 0xdc, 0x1bf: 0x37,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x38, 0x1c1: 0xdd, 0x1c2: 0xde, 0x1c3: 0xdf, 0x1c4: 0xe0, 0x1c5: 0x39, 0x1c6: 0x3a, 0x1c7: 0xe1,
+	0x1c8: 0xe2, 0x1c9: 0x3b, 0x1ca: 0x3c, 0x1cb: 0x3d, 0x1cc: 0x3e, 0x1cd: 0x3f, 0x1ce: 0x40, 0x1cf: 0x41,
+	0x1d0: 0xa0, 0x1d1: 0xa0, 0x1d2: 0xa0, 0x1d3: 0xa0, 0x1d4: 0xa0, 0x1d5: 0xa0, 0x1d6: 0xa0, 0x1d7: 0xa0,
+	0x1d8: 0xa0, 0x1d9: 0xa0, 0x1da: 0xa0, 0x1db: 0xa0, 0x1dc: 0xa0, 0x1dd: 0xa0, 0x1de: 0xa0, 0x1df: 0xa0,
+	0x1e0: 0xa0, 0x1e1: 0xa0, 0x1e2: 0xa0, 0x1e3: 0xa0, 0x1e4: 0xa0, 0x1e5: 0xa0, 0x1e6: 0xa0, 0x1e7: 0xa0,
+	0x1e8: 0xa0, 0x1e9: 0xa0, 0x1ea: 0xa0, 0x1eb: 0xa0, 0x1ec: 0xa0, 0x1ed: 0xa0, 0x1ee: 0xa0, 0x1ef: 0xa0,
+	0x1f0: 0xa0, 0x1f1: 0xa0, 0x1f2: 0xa0, 0x1f3: 0xa0, 0x1f4: 0xa0, 0x1f5: 0xa0, 0x1f6: 0xa0, 0x1f7: 0xa0,
+	0x1f8: 0xa0, 0x1f9: 0xa0, 0x1fa: 0xa0, 0x1fb: 0xa0, 0x1fc: 0xa0, 0x1fd: 0xa0, 0x1fe: 0xa0, 0x1ff: 0xa0,
+	// Block 0x8, offset 0x200
+	0x200: 0xa0, 0x201: 0xa0, 0x202: 0xa0, 0x203: 0xa0, 0x204: 0xa0, 0x205: 0xa0, 0x206: 0xa0, 0x207: 0xa0,
+	0x208: 0xa0, 0x209: 0xa0, 0x20a: 0xa0, 0x20b: 0xa0, 0x20c: 0xa0, 0x20d: 0xa0, 0x20e: 0xa0, 0x20f: 0xa0,
+	0x210: 0xa0, 0x211: 0xa0, 0x212: 0xa0, 0x213: 0xa0, 0x214: 0xa0, 0x215: 0xa0, 0x216: 0xa0, 0x217: 0xa0,
+	0x218: 0xa0, 0x219: 0xa0, 0x21a: 0xa0, 0x21b: 0xa0, 0x21c: 0xa0, 0x21d: 0xa0, 0x21e: 0xa0, 0x21f: 0xa0,
+	0x220: 0xa0, 0x221: 0xa0, 0x222: 0xa0, 0x223: 0xa0, 0x224: 0xa0, 0x225: 0xa0, 0x226: 0xa0, 0x227: 0xa0,
+	0x228: 0xa0, 0x229: 0xa0, 0x22a: 0xa0, 0x22b: 0xa0, 0x22c: 0xa0, 0x22d: 0xa0, 0x22e: 0xa0, 0x22f: 0xa0,
+	0x230: 0xa0, 0x231: 0xa0, 0x232: 0xa0, 0x233: 0xa0, 0x234: 0xa0, 0x235: 0xa0, 0x236: 0xa0, 0x237: 0x9c,
+	0x238: 0xa0, 0x239: 0xa0, 0x23a: 0xa0, 0x23b: 0xa0, 0x23c: 0xa0, 0x23d: 0xa0, 0x23e: 0xa0, 0x23f: 0xa0,
+	// Block 0x9, offset 0x240
+	0x240: 0xa0, 0x241: 0xa0, 0x242: 0xa0, 0x243: 0xa0, 0x244: 0xa0, 0x245: 0xa0, 0x246: 0xa0, 0x247: 0xa0,
+	0x248: 0xa0, 0x249: 0xa0, 0x24a: 0xa0, 0x24b: 0xa0, 0x24c: 0xa0, 0x24d: 0xa0, 0x24e: 0xa0, 0x24f: 0xa0,
+	0x250: 0xa0, 0x251: 0xa0, 0x252: 0xa0, 0x253: 0xa0, 0x254: 0xa0, 0x255: 0xa0, 0x256: 0xa0, 0x257: 0xa0,
+	0x258: 0xa0, 0x259: 0xa0, 0x25a: 0xa0, 0x25b: 0xa0, 0x25c: 0xa0, 0x25d: 0xa0, 0x25e: 0xa0, 0x25f: 0xa0,
+	0x260: 0xa0, 0x261: 0xa0, 0x262: 0xa0, 0x263: 0xa0, 0x264: 0xa0, 0x265: 0xa0, 0x266: 0xa0, 0x267: 0xa0,
+	0x268: 0xa0, 0x269: 0xa0, 0x26a: 0xa0, 0x26b: 0xa0, 0x26c: 0xa0, 0x26d: 0xa0, 0x26e: 0xa0, 0x26f: 0xa0,
+	0x270: 0xa0, 0x271: 0xa0, 0x272: 0xa0, 0x273: 0xa0, 0x274: 0xa0, 0x275: 0xa0, 0x276: 0xa0, 0x277: 0xa0,
+	0x278: 0xa0, 0x279: 0xa0, 0x27a: 0xa0, 0x27b: 0xa0, 0x27c: 0xa0, 0x27d: 0xa0, 0x27e: 0xa0, 0x27f: 0xa0,
+	// Block 0xa, offset 0x280
+	0x280: 0xa0, 0x281: 0xa0, 0x282: 0xa0, 0x283: 0xa0, 0x284: 0xa0, 0x285: 0xa0, 0x286: 0xa0, 0x287: 0xa0,
+	0x288: 0xa0, 0x289: 0xa0, 0x28a: 0xa0, 0x28b: 0xa0, 0x28c: 0xa0, 0x28d: 0xa0, 0x28e: 0xa0, 0x28f: 0xa0,
+	0x290: 0xa0, 0x291: 0xa0, 0x292: 0xa0, 0x293: 0xa0, 0x294: 0xa0, 0x295: 0xa0, 0x296: 0xa0, 0x297: 0xa0,
+	0x298: 0xa0, 0x299: 0xa0, 0x29a: 0xa0, 0x29b: 0xa0, 0x29c: 0xa0, 0x29d: 0xa0, 0x29e: 0xa0, 0x29f: 0xa0,
+	0x2a0: 0xa0, 0x2a1: 0xa0, 0x2a2: 0xa0, 0x2a3: 0xa0, 0x2a4: 0xa0, 0x2a5: 0xa0, 0x2a6: 0xa0, 0x2a7: 0xa0,
+	0x2a8: 0xa0, 0x2a9: 0xa0, 0x2aa: 0xa0, 0x2ab: 0xa0, 0x2ac: 0xa0, 0x2ad: 0xa0, 0x2ae: 0xa0, 0x2af: 0xa0,
+	0x2b0: 0xa0, 0x2b1: 0xa0, 0x2b2: 0xa0, 0x2b3: 0xa0, 0x2b4: 0xa0, 0x2b5: 0xa0, 0x2b6: 0xa0, 0x2b7: 0xa0,
+	0x2b8: 0xa0, 0x2b9: 0xa0, 0x2ba: 0xa0, 0x2bb: 0xa0, 0x2bc: 0xa0, 0x2bd: 0xa0, 0x2be: 0xa0, 0x2bf: 0xe3,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0xa0, 0x2c1: 0xa0, 0x2c2: 0xa0, 0x2c3: 0xa0, 0x2c4: 0xa0, 0x2c5: 0xa0, 0x2c6: 0xa0, 0x2c7: 0xa0,
+	0x2c8: 0xa0, 0x2c9: 0xa0, 0x2ca: 0xa0, 0x2cb: 0xa0, 0x2cc: 0xa0, 0x2cd: 0xa0, 0x2ce: 0xa0, 0x2cf: 0xa0,
+	0x2d0: 0xa0, 0x2d1: 0xa0, 0x2d2: 0xe4, 0x2d3: 0xe5, 0x2d4: 0xa0, 0x2d5: 0xa0, 0x2d6: 0xa0, 0x2d7: 0xa0,
+	0x2d8: 0xe6, 0x2d9: 0x42, 0x2da: 0x43, 0x2db: 0xe7, 0x2dc: 0x44, 0x2dd: 0x45, 0x2de: 0x46, 0x2df: 0xe8,
+	0x2e0: 0xe9, 0x2e1: 0xea, 0x2e2: 0xeb, 0x2e3: 0xec, 0x2e4: 0xed, 0x2e5: 0xee, 0x2e6: 0xef, 0x2e7: 0xf0,
+	0x2e8: 0xf1, 0x2e9: 0xf2, 0x2ea: 0xf3, 0x2eb: 0xf4, 0x2ec: 0xf5, 0x2ed: 0xf6, 0x2ee: 0xf7, 0x2ef: 0xf8,
+	0x2f0: 0xa0, 0x2f1: 0xa0, 0x2f2: 0xa0, 0x2f3: 0xa0, 0x2f4: 0xa0, 0x2f5: 0xa0, 0x2f6: 0xa0, 0x2f7: 0xa0,
+	0x2f8: 0xa0, 0x2f9: 0xa0, 0x2fa: 0xa0, 0x2fb: 0xa0, 0x2fc: 0xa0, 0x2fd: 0xa0, 0x2fe: 0xa0, 0x2ff: 0xa0,
+	// Block 0xc, offset 0x300
+	0x300: 0xa0, 0x301: 0xa0, 0x302: 0xa0, 0x303: 0xa0, 0x304: 0xa0, 0x305: 0xa0, 0x306: 0xa0, 0x307: 0xa0,
+	0x308: 0xa0, 0x309: 0xa0, 0x30a: 0xa0, 0x30b: 0xa0, 0x30c: 0xa0, 0x30d: 0xa0, 0x30e: 0xa0, 0x30f: 0xa0,
+	0x310: 0xa0, 0x311: 0xa0, 0x312: 0xa0, 0x313: 0xa0, 0x314: 0xa0, 0x315: 0xa0, 0x316: 0xa0, 0x317: 0xa0,
+	0x318: 0xa0, 0x319: 0xa0, 0x31a: 0xa0, 0x31b: 0xa0, 0x31c: 0xa0, 0x31d: 0xa0, 0x31e: 0xf9, 0x31f: 0xfa,
+	// Block 0xd, offset 0x340
+	0x340: 0xfb, 0x341: 0xfb, 0x342: 0xfb, 0x343: 0xfb, 0x344: 0xfb, 0x345: 0xfb, 0x346: 0xfb, 0x347: 0xfb,
+	0x348: 0xfb, 0x349: 0xfb, 0x34a: 0xfb, 0x34b: 0xfb, 0x34c: 0xfb, 0x34d: 0xfb, 0x34e: 0xfb, 0x34f: 0xfb,
+	0x350: 0xfb, 0x351: 0xfb, 0x352: 0xfb, 0x353: 0xfb, 0x354: 0xfb, 0x355: 0xfb, 0x356: 0xfb, 0x357: 0xfb,
+	0x358: 0xfb, 0x359: 0xfb, 0x35a: 0xfb, 0x35b: 0xfb, 0x35c: 0xfb, 0x35d: 0xfb, 0x35e: 0xfb, 0x35f: 0xfb,
+	0x360: 0xfb, 0x361: 0xfb, 0x362: 0xfb, 0x363: 0xfb, 0x364: 0xfb, 0x365: 0xfb, 0x366: 0xfb, 0x367: 0xfb,
+	0x368: 0xfb, 0x369: 0xfb, 0x36a: 0xfb, 0x36b: 0xfb, 0x36c: 0xfb, 0x36d: 0xfb, 0x36e: 0xfb, 0x36f: 0xfb,
+	0x370: 0xfb, 0x371: 0xfb, 0x372: 0xfb, 0x373: 0xfb, 0x374: 0xfb, 0x375: 0xfb, 0x376: 0xfb, 0x377: 0xfb,
+	0x378: 0xfb, 0x379: 0xfb, 0x37a: 0xfb, 0x37b: 0xfb, 0x37c: 0xfb, 0x37d: 0xfb, 0x37e: 0xfb, 0x37f: 0xfb,
+	// Block 0xe, offset 0x380
+	0x380: 0xfb, 0x381: 0xfb, 0x382: 0xfb, 0x383: 0xfb, 0x384: 0xfb, 0x385: 0xfb, 0x386: 0xfb, 0x387: 0xfb,
+	0x388: 0xfb, 0x389: 0xfb, 0x38a: 0xfb, 0x38b: 0xfb, 0x38c: 0xfb, 0x38d: 0xfb, 0x38e: 0xfb, 0x38f: 0xfb,
+	0x390: 0xfb, 0x391: 0xfb, 0x392: 0xfb, 0x393: 0xfb, 0x394: 0xfb, 0x395: 0xfb, 0x396: 0xfb, 0x397: 0xfb,
+	0x398: 0xfb, 0x399: 0xfb, 0x39a: 0xfb, 0x39b: 0xfb, 0x39c: 0xfb, 0x39d: 0xfb, 0x39e: 0xfb, 0x39f: 0xfb,
+	0x3a0: 0xfb, 0x3a1: 0xfb, 0x3a2: 0xfb, 0x3a3: 0xfb, 0x3a4: 0xfc, 0x3a5: 0xfd, 0x3a6: 0xfe, 0x3a7: 0xff,
+	0x3a8: 0x47, 0x3a9: 0x100, 0x3aa: 0x101, 0x3ab: 0x48, 0x3ac: 0x49, 0x3ad: 0x4a, 0x3ae: 0x4b, 0x3af: 0x4c,
+	0x3b0: 0x102, 0x3b1: 0x4d, 0x3b2: 0x4e, 0x3b3: 0x4f, 0x3b4: 0x50, 0x3b5: 0x51, 0x3b6: 0x103, 0x3b7: 0x52,
+	0x3b8: 0x53, 0x3b9: 0x54, 0x3ba: 0x55, 0x3bb: 0x56, 0x3bc: 0x57, 0x3bd: 0x58, 0x3be: 0x59, 0x3bf: 0x5a,
+	// Block 0xf, offset 0x3c0
+	0x3c0: 0x104, 0x3c1: 0x105, 0x3c2: 0xa0, 0x3c3: 0x106, 0x3c4: 0x107, 0x3c5: 0x9c, 0x3c6: 0x108, 0x3c7: 0x109,
+	0x3c8: 0xfb, 0x3c9: 0xfb, 0x3ca: 0x10a, 0x3cb: 0x10b, 0x3cc: 0x10c, 0x3cd: 0x10d, 0x3ce: 0x10e, 0x3cf: 0x10f,
+	0x3d0: 0x110, 0x3d1: 0xa0, 0x3d2: 0x111, 0x3d3: 0x112, 0x3d4: 0x113, 0x3d5: 0x114, 0x3d6: 0xfb, 0x3d7: 0xfb,
+	0x3d8: 0xa0, 0x3d9: 0xa0, 0x3da: 0xa0, 0x3db: 0xa0, 0x3dc: 0x115, 0x3dd: 0x116, 0x3de: 0xfb, 0x3df: 0xfb,
+	0x3e0: 0x117, 0x3e1: 0x118, 0x3e2: 0x119, 0x3e3: 0x11a, 0x3e4: 0x11b, 0x3e5: 0xfb, 0x3e6: 0x11c, 0x3e7: 0x11d,
+	0x3e8: 0x11e, 0x3e9: 0x11f, 0x3ea: 0x120, 0x3eb: 0x5b, 0x3ec: 0x121, 0x3ed: 0x122, 0x3ee: 0x5c, 0x3ef: 0xfb,
+	0x3f0: 0x123, 0x3f1: 0x124, 0x3f2: 0x125, 0x3f3: 0x126, 0x3f4: 0x127, 0x3f5: 0xfb, 0x3f6: 0xfb, 0x3f7: 0xfb,
+	0x3f8: 0xfb, 0x3f9: 0x128, 0x3fa: 0x129, 0x3fb: 0xfb, 0x3fc: 0x12a, 0x3fd: 0x12b, 0x3fe: 0x12c, 0x3ff: 0x12d,
+	// Block 0x10, offset 0x400
+	0x400: 0x12e, 0x401: 0x12f, 0x402: 0x130, 0x403: 0x131, 0x404: 0x132, 0x405: 0x133, 0x406: 0x134, 0x407: 0x135,
+	0x408: 0x136, 0x409: 0xfb, 0x40a: 0x137, 0x40b: 0x138, 0x40c: 0x5d, 0x40d: 0x5e, 0x40e: 0xfb, 0x40f: 0xfb,
+	0x410: 0x139, 0x411: 0x13a, 0x412: 0x13b, 0x413: 0x13c, 0x414: 0xfb, 0x415: 0xfb, 0x416: 0x13d, 0x417: 0x13e,
+	0x418: 0x13f, 0x419: 0x140, 0x41a: 0x141, 0x41b: 0x142, 0x41c: 0x143, 0x41d: 0xfb, 0x41e: 0xfb, 0x41f: 0xfb,
+	0x420: 0x144, 0x421: 0xfb, 0x422: 0x145, 0x423: 0x146, 0x424: 0x5f, 0x425: 0x147, 0x426: 0x148, 0x427: 0x149,
+	0x428: 0x14a, 0x429: 0x14b, 0x42a: 0x14c, 0x42b: 0x14d, 0x42c: 0xfb, 0x42d: 0xfb, 0x42e: 0xfb, 0x42f: 0xfb,
+	0x430: 0x14e, 0x431: 0x14f, 0x432: 0x150, 0x433: 0xfb, 0x434: 0x151, 0x435: 0x152, 0x436: 0x153, 0x437: 0xfb,
+	0x438: 0xfb, 0x439: 0xfb, 0x43a: 0xfb, 0x43b: 0x154, 0x43c: 0xfb, 0x43d: 0xfb, 0x43e: 0x155, 0x43f: 0x156,
+	// Block 0x11, offset 0x440
+	0x440: 0xa0, 0x441: 0xa0, 0x442: 0xa0, 0x443: 0xa0, 0x444: 0xa0, 0x445: 0xa0, 0x446: 0xa0, 0x447: 0xa0,
+	0x448: 0xa0, 0x449: 0xa0, 0x44a: 0xa0, 0x44b: 0xa0, 0x44c: 0xa0, 0x44d: 0xa0, 0x44e: 0x157, 0x44f: 0xfb,
+	0x450: 0x9c, 0x451: 0x158, 0x452: 0xa0, 0x453: 0xa0, 0x454: 0xa0, 0x455: 0x159, 0x456: 0xfb, 0x457: 0xfb,
+	0x458: 0xfb, 0x459: 0xfb, 0x45a: 0xfb, 0x45b: 0xfb, 0x45c: 0xfb, 0x45d: 0xfb, 0x45e: 0xfb, 0x45f: 0xfb,
+	0x460: 0xfb, 0x461: 0xfb, 0x462: 0xfb, 0x463: 0xfb, 0x464: 0xfb, 0x465: 0xfb, 0x466: 0xfb, 0x467: 0xfb,
+	0x468: 0xfb, 0x469: 0xfb, 0x46a: 0xfb, 0x46b: 0xfb, 0x46c: 0xfb, 0x46d: 0xfb, 0x46e: 0xfb, 0x46f: 0xfb,
+	0x470: 0xfb, 0x471: 0xfb, 0x472: 0xfb, 0x473: 0xfb, 0x474: 0xfb, 0x475: 0xfb, 0x476: 0xfb, 0x477: 0xfb,
+	0x478: 0xfb, 0x479: 0xfb, 0x47a: 0xfb, 0x47b: 0xfb, 0x47c: 0xfb, 0x47d: 0xfb, 0x47e: 0xfb, 0x47f: 0xfb,
+	// Block 0x12, offset 0x480
+	0x480: 0xa0, 0x481: 0xa0, 0x482: 0xa0, 0x483: 0xa0, 0x484: 0xa0, 0x485: 0xa0, 0x486: 0xa0, 0x487: 0xa0,
+	0x488: 0xa0, 0x489: 0xa0, 0x48a: 0xa0, 0x48b: 0xa0, 0x48c: 0xa0, 0x48d: 0xa0, 0x48e: 0xa0, 0x48f: 0xa0,
+	0x490: 0x15a, 0x491: 0xfb, 0x492: 0xfb, 0x493: 0xfb, 0x494: 0xfb, 0x495: 0xfb, 0x496: 0xfb, 0x497: 0xfb,
+	0x498: 0xfb, 0x499: 0xfb, 0x49a: 0xfb, 0x49b: 0xfb, 0x49c: 0xfb, 0x49d: 0xfb, 0x49e: 0xfb, 0x49f: 0xfb,
+	0x4a0: 0xfb, 0x4a1: 0xfb, 0x4a2: 0xfb, 0x4a3: 0xfb, 0x4a4: 0xfb, 0x4a5: 0xfb, 0x4a6: 0xfb, 0x4a7: 0xfb,
+	0x4a8: 0xfb, 0x4a9: 0xfb, 0x4aa: 0xfb, 0x4ab: 0xfb, 0x4ac: 0xfb, 0x4ad: 0xfb, 0x4ae: 0xfb, 0x4af: 0xfb,
+	0x4b0: 0xfb, 0x4b1: 0xfb, 0x4b2: 0xfb, 0x4b3: 0xfb, 0x4b4: 0xfb, 0x4b5: 0xfb, 0x4b6: 0xfb, 0x4b7: 0xfb,
+	0x4b8: 0xfb, 0x4b9: 0xfb, 0x4ba: 0xfb, 0x4bb: 0xfb, 0x4bc: 0xfb, 0x4bd: 0xfb, 0x4be: 0xfb, 0x4bf: 0xfb,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0xfb, 0x4c1: 0xfb, 0x4c2: 0xfb, 0x4c3: 0xfb, 0x4c4: 0xfb, 0x4c5: 0xfb, 0x4c6: 0xfb, 0x4c7: 0xfb,
+	0x4c8: 0xfb, 0x4c9: 0xfb, 0x4ca: 0xfb, 0x4cb: 0xfb, 0x4cc: 0xfb, 0x4cd: 0xfb, 0x4ce: 0xfb, 0x4cf: 0xfb,
+	0x4d0: 0xa0, 0x4d1: 0xa0, 0x4d2: 0xa0, 0x4d3: 0xa0, 0x4d4: 0xa0, 0x4d5: 0xa0, 0x4d6: 0xa0, 0x4d7: 0xa0,
+	0x4d8: 0xa0, 0x4d9: 0x15b, 0x4da: 0xfb, 0x4db: 0xfb, 0x4dc: 0xfb, 0x4dd: 0xfb, 0x4de: 0xfb, 0x4df: 0xfb,
+	0x4e0: 0xfb, 0x4e1: 0xfb, 0x4e2: 0xfb, 0x4e3: 0xfb, 0x4e4: 0xfb, 0x4e5: 0xfb, 0x4e6: 0xfb, 0x4e7: 0xfb,
+	0x4e8: 0xfb, 0x4e9: 0xfb, 0x4ea: 0xfb, 0x4eb: 0xfb, 0x4ec: 0xfb, 0x4ed: 0xfb, 0x4ee: 0xfb, 0x4ef: 0xfb,
+	0x4f0: 0xfb, 0x4f1: 0xfb, 0x4f2: 0xfb, 0x4f3: 0xfb, 0x4f4: 0xfb, 0x4f5: 0xfb, 0x4f6: 0xfb, 0x4f7: 0xfb,
+	0x4f8: 0xfb, 0x4f9: 0xfb, 0x4fa: 0xfb, 0x4fb: 0xfb, 0x4fc: 0xfb, 0x4fd: 0xfb, 0x4fe: 0xfb, 0x4ff: 0xfb,
+	// Block 0x14, offset 0x500
+	0x500: 0xfb, 0x501: 0xfb, 0x502: 0xfb, 0x503: 0xfb, 0x504: 0xfb, 0x505: 0xfb, 0x506: 0xfb, 0x507: 0xfb,
+	0x508: 0xfb, 0x509: 0xfb, 0x50a: 0xfb, 0x50b: 0xfb, 0x50c: 0xfb, 0x50d: 0xfb, 0x50e: 0xfb, 0x50f: 0xfb,
+	0x510: 0xfb, 0x511: 0xfb, 0x512: 0xfb, 0x513: 0xfb, 0x514: 0xfb, 0x515: 0xfb, 0x516: 0xfb, 0x517: 0xfb,
+	0x518: 0xfb, 0x519: 0xfb, 0x51a: 0xfb, 0x51b: 0xfb, 0x51c: 0xfb, 0x51d: 0xfb, 0x51e: 0xfb, 0x51f: 0xfb,
+	0x520: 0xa0, 0x521: 0xa0, 0x522: 0xa0, 0x523: 0xa0, 0x524: 0xa0, 0x525: 0xa0, 0x526: 0xa0, 0x527: 0xa0,
+	0x528: 0x14d, 0x529: 0x15c, 0x52a: 0xfb, 0x52b: 0x15d, 0x52c: 0x15e, 0x52d: 0x15f, 0x52e: 0x160, 0x52f: 0xfb,
+	0x530: 0xfb, 0x531: 0xfb, 0x532: 0xfb, 0x533: 0xfb, 0x534: 0xfb, 0x535: 0xfb, 0x536: 0xfb, 0x537: 0xfb,
+	0x538: 0xfb, 0x539: 0x161, 0x53a: 0x162, 0x53b: 0xfb, 0x53c: 0xa0, 0x53d: 0x163, 0x53e: 0x164, 0x53f: 0x165,
+	// Block 0x15, offset 0x540
+	0x540: 0xa0, 0x541: 0xa0, 0x542: 0xa0, 0x543: 0xa0, 0x544: 0xa0, 0x545: 0xa0, 0x546: 0xa0, 0x547: 0xa0,
+	0x548: 0xa0, 0x549: 0xa0, 0x54a: 0xa0, 0x54b: 0xa0, 0x54c: 0xa0, 0x54d: 0xa0, 0x54e: 0xa0, 0x54f: 0xa0,
+	0x550: 0xa0, 0x551: 0xa0, 0x552: 0xa0, 0x553: 0xa0, 0x554: 0xa0, 0x555: 0xa0, 0x556: 0xa0, 0x557: 0xa0,
+	0x558: 0xa0, 0x559: 0xa0, 0x55a: 0xa0, 0x55b: 0xa0, 0x55c: 0xa0, 0x55d: 0xa0, 0x55e: 0xa0, 0x55f: 0x166,
+	0x560: 0xa0, 0x561: 0xa0, 0x562: 0xa0, 0x563: 0xa0, 0x564: 0xa0, 0x565: 0xa0, 0x566: 0xa0, 0x567: 0xa0,
+	0x568: 0xa0, 0x569: 0xa0, 0x56a: 0xa0, 0x56b: 0xa0, 0x56c: 0xa0, 0x56d: 0xa0, 0x56e: 0xa0, 0x56f: 0xa0,
+	0x570: 0xa0, 0x571: 0xa0, 0x572: 0xa0, 0x573: 0x167, 0x574: 0x168, 0x575: 0xfb, 0x576: 0xfb, 0x577: 0xfb,
+	0x578: 0xfb, 0x579: 0xfb, 0x57a: 0xfb, 0x57b: 0xfb, 0x57c: 0xfb, 0x57d: 0xfb, 0x57e: 0xfb, 0x57f: 0xfb,
+	// Block 0x16, offset 0x580
+	0x580: 0xa0, 0x581: 0xa0, 0x582: 0xa0, 0x583: 0xa0, 0x584: 0x169, 0x585: 0x16a, 0x586: 0xa0, 0x587: 0xa0,
+	0x588: 0xa0, 0x589: 0xa0, 0x58a: 0xa0, 0x58b: 0x16b, 0x58c: 0xfb, 0x58d: 0xfb, 0x58e: 0xfb, 0x58f: 0xfb,
+	0x590: 0xfb, 0x591: 0xfb, 0x592: 0xfb, 0x593: 0xfb, 0x594: 0xfb, 0x595: 0xfb, 0x596: 0xfb, 0x597: 0xfb,
+	0x598: 0xfb, 0x599: 0xfb, 0x59a: 0xfb, 0x59b: 0xfb, 0x59c: 0xfb, 0x59d: 0xfb, 0x59e: 0xfb, 0x59f: 0xfb,
+	0x5a0: 0xfb, 0x5a1: 0xfb, 0x5a2: 0xfb, 0x5a3: 0xfb, 0x5a4: 0xfb, 0x5a5: 0xfb, 0x5a6: 0xfb, 0x5a7: 0xfb,
+	0x5a8: 0xfb, 0x5a9: 0xfb, 0x5aa: 0xfb, 0x5ab: 0xfb, 0x5ac: 0xfb, 0x5ad: 0xfb, 0x5ae: 0xfb, 0x5af: 0xfb,
+	0x5b0: 0xa0, 0x5b1: 0x16c, 0x5b2: 0x16d, 0x5b3: 0xfb, 0x5b4: 0xfb, 0x5b5: 0xfb, 0x5b6: 0xfb, 0x5b7: 0xfb,
+	0x5b8: 0xfb, 0x5b9: 0xfb, 0x5ba: 0xfb, 0x5bb: 0xfb, 0x5bc: 0xfb, 0x5bd: 0xfb, 0x5be: 0xfb, 0x5bf: 0xfb,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x9c, 0x5c1: 0x9c, 0x5c2: 0x9c, 0x5c3: 0x16e, 0x5c4: 0x16f, 0x5c5: 0x170, 0x5c6: 0x171, 0x5c7: 0x172,
+	0x5c8: 0x9c, 0x5c9: 0x173, 0x5ca: 0xfb, 0x5cb: 0x174, 0x5cc: 0x9c, 0x5cd: 0x175, 0x5ce: 0xfb, 0x5cf: 0xfb,
+	0x5d0: 0x60, 0x5d1: 0x61, 0x5d2: 0x62, 0x5d3: 0x63, 0x5d4: 0x64, 0x5d5: 0x65, 0x5d6: 0x66, 0x5d7: 0x67,
+	0x5d8: 0x68, 0x5d9: 0x69, 0x5da: 0x6a, 0x5db: 0x6b, 0x5dc: 0x6c, 0x5dd: 0x6d, 0x5de: 0x6e, 0x5df: 0x6f,
+	0x5e0: 0x9c, 0x5e1: 0x9c, 0x5e2: 0x9c, 0x5e3: 0x9c, 0x5e4: 0x9c, 0x5e5: 0x9c, 0x5e6: 0x9c, 0x5e7: 0x9c,
+	0x5e8: 0x176, 0x5e9: 0x177, 0x5ea: 0x178, 0x5eb: 0xfb, 0x5ec: 0xfb, 0x5ed: 0xfb, 0x5ee: 0xfb, 0x5ef: 0xfb,
+	0x5f0: 0xfb, 0x5f1: 0xfb, 0x5f2: 0xfb, 0x5f3: 0xfb, 0x5f4: 0xfb, 0x5f5: 0xfb, 0x5f6: 0xfb, 0x5f7: 0xfb,
+	0x5f8: 0xfb, 0x5f9: 0xfb, 0x5fa: 0xfb, 0x5fb: 0xfb, 0x5fc: 0xfb, 0x5fd: 0xfb, 0x5fe: 0xfb, 0x5ff: 0xfb,
+	// Block 0x18, offset 0x600
+	0x600: 0x179, 0x601: 0xfb, 0x602: 0xfb, 0x603: 0xfb, 0x604: 0x17a, 0x605: 0x17b, 0x606: 0xfb, 0x607: 0xfb,
+	0x608: 0xfb, 0x609: 0xfb, 0x60a: 0xfb, 0x60b: 0x17c, 0x60c: 0xfb, 0x60d: 0xfb, 0x60e: 0xfb, 0x60f: 0xfb,
+	0x610: 0xfb, 0x611: 0xfb, 0x612: 0xfb, 0x613: 0xfb, 0x614: 0xfb, 0x615: 0xfb, 0x616: 0xfb, 0x617: 0xfb,
+	0x618: 0xfb, 0x619: 0xfb, 0x61a: 0xfb, 0x61b: 0xfb, 0x61c: 0xfb, 0x61d: 0xfb, 0x61e: 0xfb, 0x61f: 0xfb,
+	0x620: 0x123, 0x621: 0x123, 0x622: 0x123, 0x623: 0x17d, 0x624: 0x70, 0x625: 0x17e, 0x626: 0xfb, 0x627: 0xfb,
+	0x628: 0xfb, 0x629: 0xfb, 0x62a: 0xfb, 0x62b: 0xfb, 0x62c: 0xfb, 0x62d: 0xfb, 0x62e: 0xfb, 0x62f: 0xfb,
+	0x630: 0xfb, 0x631: 0x17f, 0x632: 0x180, 0x633: 0xfb, 0x634: 0x181, 0x635: 0xfb, 0x636: 0xfb, 0x637: 0xfb,
+	0x638: 0x71, 0x639: 0x72, 0x63a: 0x73, 0x63b: 0x182, 0x63c: 0xfb, 0x63d: 0xfb, 0x63e: 0xfb, 0x63f: 0xfb,
+	// Block 0x19, offset 0x640
+	0x640: 0x183, 0x641: 0x9c, 0x642: 0x184, 0x643: 0x185, 0x644: 0x74, 0x645: 0x75, 0x646: 0x186, 0x647: 0x187,
+	0x648: 0x76, 0x649: 0x188, 0x64a: 0xfb, 0x64b: 0xfb, 0x64c: 0x9c, 0x64d: 0x9c, 0x64e: 0x9c, 0x64f: 0x9c,
+	0x650: 0x9c, 0x651: 0x9c, 0x652: 0x9c, 0x653: 0x9c, 0x654: 0x9c, 0x655: 0x9c, 0x656: 0x9c, 0x657: 0x9c,
+	0x658: 0x9c, 0x659: 0x9c, 0x65a: 0x9c, 0x65b: 0x189, 0x65c: 0x9c, 0x65d: 0x18a, 0x65e: 0x9c, 0x65f: 0x18b,
+	0x660: 0x18c, 0x661: 0x18d, 0x662: 0x18e, 0x663: 0xfb, 0x664: 0x9c, 0x665: 0x18f, 0x666: 0x9c, 0x667: 0x190,
+	0x668: 0x9c, 0x669: 0x191, 0x66a: 0x192, 0x66b: 0x193, 0x66c: 0x9c, 0x66d: 0x9c, 0x66e: 0x194, 0x66f: 0x195,
+	0x670: 0xfb, 0x671: 0xfb, 0x672: 0xfb, 0x673: 0xfb, 0x674: 0xfb, 0x675: 0xfb, 0x676: 0xfb, 0x677: 0xfb,
+	0x678: 0xfb, 0x679: 0xfb, 0x67a: 0xfb, 0x67b: 0xfb, 0x67c: 0xfb, 0x67d: 0xfb, 0x67e: 0xfb, 0x67f: 0xfb,
+	// Block 0x1a, offset 0x680
+	0x680: 0xa0, 0x681: 0xa0, 0x682: 0xa0, 0x683: 0xa0, 0x684: 0xa0, 0x685: 0xa0, 0x686: 0xa0, 0x687: 0xa0,
+	0x688: 0xa0, 0x689: 0xa0, 0x68a: 0xa0, 0x68b: 0xa0, 0x68c: 0xa0, 0x68d: 0xa0, 0x68e: 0xa0, 0x68f: 0xa0,
+	0x690: 0xa0, 0x691: 0xa0, 0x692: 0xa0, 0x693: 0xa0, 0x694: 0xa0, 0x695: 0xa0, 0x696: 0xa0, 0x697: 0xa0,
+	0x698: 0xa0, 0x699: 0xa0, 0x69a: 0xa0, 0x69b: 0x196, 0x69c: 0xa0, 0x69d: 0xa0, 0x69e: 0xa0, 0x69f: 0xa0,
+	0x6a0: 0xa0, 0x6a1: 0xa0, 0x6a2: 0xa0, 0x6a3: 0xa0, 0x6a4: 0xa0, 0x6a5: 0xa0, 0x6a6: 0xa0, 0x6a7: 0xa0,
+	0x6a8: 0xa0, 0x6a9: 0xa0, 0x6aa: 0xa0, 0x6ab: 0xa0, 0x6ac: 0xa0, 0x6ad: 0xa0, 0x6ae: 0xa0, 0x6af: 0xa0,
+	0x6b0: 0xa0, 0x6b1: 0xa0, 0x6b2: 0xa0, 0x6b3: 0xa0, 0x6b4: 0xa0, 0x6b5: 0xa0, 0x6b6: 0xa0, 0x6b7: 0xa0,
+	0x6b8: 0xa0, 0x6b9: 0xa0, 0x6ba: 0xa0, 0x6bb: 0xa0, 0x6bc: 0xa0, 0x6bd: 0xa0, 0x6be: 0xa0, 0x6bf: 0xa0,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0xa0, 0x6c1: 0xa0, 0x6c2: 0xa0, 0x6c3: 0xa0, 0x6c4: 0xa0, 0x6c5: 0xa0, 0x6c6: 0xa0, 0x6c7: 0xa0,
+	0x6c8: 0xa0, 0x6c9: 0xa0, 0x6ca: 0xa0, 0x6cb: 0xa0, 0x6cc: 0xa0, 0x6cd: 0xa0, 0x6ce: 0xa0, 0x6cf: 0xa0,
+	0x6d0: 0xa0, 0x6d1: 0xa0, 0x6d2: 0xa0, 0x6d3: 0xa0, 0x6d4: 0xa0, 0x6d5: 0xa0, 0x6d6: 0xa0, 0x6d7: 0xa0,
+	0x6d8: 0xa0, 0x6d9: 0xa0, 0x6da: 0xa0, 0x6db: 0xa0, 0x6dc: 0x197, 0x6dd: 0xa0, 0x6de: 0xa0, 0x6df: 0xa0,
+	0x6e0: 0x198, 0x6e1: 0xa0, 0x6e2: 0xa0, 0x6e3: 0xa0, 0x6e4: 0xa0, 0x6e5: 0xa0, 0x6e6: 0xa0, 0x6e7: 0xa0,
+	0x6e8: 0xa0, 0x6e9: 0xa0, 0x6ea: 0xa0, 0x6eb: 0xa0, 0x6ec: 0xa0, 0x6ed: 0xa0, 0x6ee: 0xa0, 0x6ef: 0xa0,
+	0x6f0: 0xa0, 0x6f1: 0xa0, 0x6f2: 0xa0, 0x6f3: 0xa0, 0x6f4: 0xa0, 0x6f5: 0xa0, 0x6f6: 0xa0, 0x6f7: 0xa0,
+	0x6f8: 0xa0, 0x6f9: 0xa0, 0x6fa: 0xa0, 0x6fb: 0xa0, 0x6fc: 0xa0, 0x6fd: 0xa0, 0x6fe: 0xa0, 0x6ff: 0xa0,
+	// Block 0x1c, offset 0x700
+	0x700: 0xa0, 0x701: 0xa0, 0x702: 0xa0, 0x703: 0xa0, 0x704: 0xa0, 0x705: 0xa0, 0x706: 0xa0, 0x707: 0xa0,
+	0x708: 0xa0, 0x709: 0xa0, 0x70a: 0xa0, 0x70b: 0xa0, 0x70c: 0xa0, 0x70d: 0xa0, 0x70e: 0xa0, 0x70f: 0xa0,
+	0x710: 0xa0, 0x711: 0xa0, 0x712: 0xa0, 0x713: 0xa0, 0x714: 0xa0, 0x715: 0xa0, 0x716: 0xa0, 0x717: 0xa0,
+	0x718: 0xa0, 0x719: 0xa0, 0x71a: 0xa0, 0x71b: 0xa0, 0x71c: 0xa0, 0x71d: 0xa0, 0x71e: 0xa0, 0x71f: 0xa0,
+	0x720: 0xa0, 0x721: 0xa0, 0x722: 0xa0, 0x723: 0xa0, 0x724: 0xa0, 0x725: 0xa0, 0x726: 0xa0, 0x727: 0xa0,
+	0x728: 0xa0, 0x729: 0xa0, 0x72a: 0xa0, 0x72b: 0xa0, 0x72c: 0xa0, 0x72d: 0xa0, 0x72e: 0xa0, 0x72f: 0xa0,
+	0x730: 0xa0, 0x731: 0xa0, 0x732: 0xa0, 0x733: 0xa0, 0x734: 0xa0, 0x735: 0xa0, 0x736: 0xa0, 0x737: 0xa0,
+	0x738: 0xa0, 0x739: 0xa0, 0x73a: 0x199, 0x73b: 0xa0, 0x73c: 0xa0, 0x73d: 0xa0, 0x73e: 0xa0, 0x73f: 0xa0,
+	// Block 0x1d, offset 0x740
+	0x740: 0xa0, 0x741: 0xa0, 0x742: 0xa0, 0x743: 0xa0, 0x744: 0xa0, 0x745: 0xa0, 0x746: 0xa0, 0x747: 0xa0,
+	0x748: 0xa0, 0x749: 0xa0, 0x74a: 0xa0, 0x74b: 0xa0, 0x74c: 0xa0, 0x74d: 0xa0, 0x74e: 0xa0, 0x74f: 0xa0,
+	0x750: 0xa0, 0x751: 0xa0, 0x752: 0xa0, 0x753: 0xa0, 0x754: 0xa0, 0x755: 0xa0, 0x756: 0xa0, 0x757: 0xa0,
+	0x758: 0xa0, 0x759: 0xa0, 0x75a: 0xa0, 0x75b: 0xa0, 0x75c: 0xa0, 0x75d: 0xa0, 0x75e: 0xa0, 0x75f: 0xa0,
+	0x760: 0xa0, 0x761: 0xa0, 0x762: 0xa0, 0x763: 0xa0, 0x764: 0xa0, 0x765: 0xa0, 0x766: 0xa0, 0x767: 0xa0,
+	0x768: 0xa0, 0x769: 0xa0, 0x76a: 0xa0, 0x76b: 0xa0, 0x76c: 0xa0, 0x76d: 0xa0, 0x76e: 0xa0, 0x76f: 0x19a,
+	0x770: 0xfb, 0x771: 0xfb, 0x772: 0xfb, 0x773: 0xfb, 0x774: 0xfb, 0x775: 0xfb, 0x776: 0xfb, 0x777: 0xfb,
+	0x778: 0xfb, 0x779: 0xfb, 0x77a: 0xfb, 0x77b: 0xfb, 0x77c: 0xfb, 0x77d: 0xfb, 0x77e: 0xfb, 0x77f: 0xfb,
+	// Block 0x1e, offset 0x780
+	0x780: 0xfb, 0x781: 0xfb, 0x782: 0xfb, 0x783: 0xfb, 0x784: 0xfb, 0x785: 0xfb, 0x786: 0xfb, 0x787: 0xfb,
+	0x788: 0xfb, 0x789: 0xfb, 0x78a: 0xfb, 0x78b: 0xfb, 0x78c: 0xfb, 0x78d: 0xfb, 0x78e: 0xfb, 0x78f: 0xfb,
+	0x790: 0xfb, 0x791: 0xfb, 0x792: 0xfb, 0x793: 0xfb, 0x794: 0xfb, 0x795: 0xfb, 0x796: 0xfb, 0x797: 0xfb,
+	0x798: 0xfb, 0x799: 0xfb, 0x79a: 0xfb, 0x79b: 0xfb, 0x79c: 0xfb, 0x79d: 0xfb, 0x79e: 0xfb, 0x79f: 0xfb,
+	0x7a0: 0x77, 0x7a1: 0x78, 0x7a2: 0x79, 0x7a3: 0x19b, 0x7a4: 0x7a, 0x7a5: 0x7b, 0x7a6: 0x19c, 0x7a7: 0x7c,
+	0x7a8: 0x7d, 0x7a9: 0xfb, 0x7aa: 0xfb, 0x7ab: 0xfb, 0x7ac: 0xfb, 0x7ad: 0xfb, 0x7ae: 0xfb, 0x7af: 0xfb,
+	0x7b0: 0xfb, 0x7b1: 0xfb, 0x7b2: 0xfb, 0x7b3: 0xfb, 0x7b4: 0xfb, 0x7b5: 0xfb, 0x7b6: 0xfb, 0x7b7: 0xfb,
+	0x7b8: 0xfb, 0x7b9: 0xfb, 0x7ba: 0xfb, 0x7bb: 0xfb, 0x7bc: 0xfb, 0x7bd: 0xfb, 0x7be: 0xfb, 0x7bf: 0xfb,
+	// Block 0x1f, offset 0x7c0
+	0x7c0: 0xa0, 0x7c1: 0xa0, 0x7c2: 0xa0, 0x7c3: 0xa0, 0x7c4: 0xa0, 0x7c5: 0xa0, 0x7c6: 0xa0, 0x7c7: 0xa0,
+	0x7c8: 0xa0, 0x7c9: 0xa0, 0x7ca: 0xa0, 0x7cb: 0xa0, 0x7cc: 0xa0, 0x7cd: 0x19d, 0x7ce: 0xfb, 0x7cf: 0xfb,
+	0x7d0: 0xfb, 0x7d1: 0xfb, 0x7d2: 0xfb, 0x7d3: 0xfb, 0x7d4: 0xfb, 0x7d5: 0xfb, 0x7d6: 0xfb, 0x7d7: 0xfb,
+	0x7d8: 0xfb, 0x7d9: 0xfb, 0x7da: 0xfb, 0x7db: 0xfb, 0x7dc: 0xfb, 0x7dd: 0xfb, 0x7de: 0xfb, 0x7df: 0xfb,
+	0x7e0: 0xfb, 0x7e1: 0xfb, 0x7e2: 0xfb, 0x7e3: 0xfb, 0x7e4: 0xfb, 0x7e5: 0xfb, 0x7e6: 0xfb, 0x7e7: 0xfb,
+	0x7e8: 0xfb, 0x7e9: 0xfb, 0x7ea: 0xfb, 0x7eb: 0xfb, 0x7ec: 0xfb, 0x7ed: 0xfb, 0x7ee: 0xfb, 0x7ef: 0xfb,
+	0x7f0: 0xfb, 0x7f1: 0xfb, 0x7f2: 0xfb, 0x7f3: 0xfb, 0x7f4: 0xfb, 0x7f5: 0xfb, 0x7f6: 0xfb, 0x7f7: 0xfb,
+	0x7f8: 0xfb, 0x7f9: 0xfb, 0x7fa: 0xfb, 0x7fb: 0xfb, 0x7fc: 0xfb, 0x7fd: 0xfb, 0x7fe: 0xfb, 0x7ff: 0xfb,
+	// Block 0x20, offset 0x800
+	0x810: 0x0d, 0x811: 0x0e, 0x812: 0x0f, 0x813: 0x10, 0x814: 0x11, 0x815: 0x0b, 0x816: 0x12, 0x817: 0x07,
+	0x818: 0x13, 0x819: 0x0b, 0x81a: 0x0b, 0x81b: 0x14, 0x81c: 0x0b, 0x81d: 0x15, 0x81e: 0x16, 0x81f: 0x17,
+	0x820: 0x07, 0x821: 0x07, 0x822: 0x07, 0x823: 0x07, 0x824: 0x07, 0x825: 0x07, 0x826: 0x07, 0x827: 0x07,
+	0x828: 0x07, 0x829: 0x07, 0x82a: 0x18, 0x82b: 0x19, 0x82c: 0x1a, 0x82d: 0x07, 0x82e: 0x1b, 0x82f: 0x1c,
+	0x830: 0x07, 0x831: 0x1d, 0x832: 0x0b, 0x833: 0x0b, 0x834: 0x0b, 0x835: 0x0b, 0x836: 0x0b, 0x837: 0x0b,
+	0x838: 0x0b, 0x839: 0x0b, 0x83a: 0x0b, 0x83b: 0x0b, 0x83c: 0x0b, 0x83d: 0x0b, 0x83e: 0x0b, 0x83f: 0x0b,
+	// Block 0x21, offset 0x840
+	0x840: 0x0b, 0x841: 0x0b, 0x842: 0x0b, 0x843: 0x0b, 0x844: 0x0b, 0x845: 0x0b, 0x846: 0x0b, 0x847: 0x0b,
+	0x848: 0x0b, 0x849: 0x0b, 0x84a: 0x0b, 0x84b: 0x0b, 0x84c: 0x0b, 0x84d: 0x0b, 0x84e: 0x0b, 0x84f: 0x0b,
+	0x850: 0x0b, 0x851: 0x0b, 0x852: 0x0b, 0x853: 0x0b, 0x854: 0x0b, 0x855: 0x0b, 0x856: 0x0b, 0x857: 0x0b,
+	0x858: 0x0b, 0x859: 0x0b, 0x85a: 0x0b, 0x85b: 0x0b, 0x85c: 0x0b, 0x85d: 0x0b, 0x85e: 0x0b, 0x85f: 0x0b,
+	0x860: 0x0b, 0x861: 0x0b, 0x862: 0x0b, 0x863: 0x0b, 0x864: 0x0b, 0x865: 0x0b, 0x866: 0x0b, 0x867: 0x0b,
+	0x868: 0x0b, 0x869: 0x0b, 0x86a: 0x0b, 0x86b: 0x0b, 0x86c: 0x0b, 0x86d: 0x0b, 0x86e: 0x0b, 0x86f: 0x0b,
+	0x870: 0x0b, 0x871: 0x0b, 0x872: 0x0b, 0x873: 0x0b, 0x874: 0x0b, 0x875: 0x0b, 0x876: 0x0b, 0x877: 0x0b,
+	0x878: 0x0b, 0x879: 0x0b, 0x87a: 0x0b, 0x87b: 0x0b, 0x87c: 0x0b, 0x87d: 0x0b, 0x87e: 0x0b, 0x87f: 0x0b,
+	// Block 0x22, offset 0x880
+	0x880: 0x19e, 0x881: 0x19f, 0x882: 0xfb, 0x883: 0xfb, 0x884: 0x1a0, 0x885: 0x1a0, 0x886: 0x1a0, 0x887: 0x1a1,
+	0x888: 0xfb, 0x889: 0xfb, 0x88a: 0xfb, 0x88b: 0xfb, 0x88c: 0xfb, 0x88d: 0xfb, 0x88e: 0xfb, 0x88f: 0xfb,
+	0x890: 0xfb, 0x891: 0xfb, 0x892: 0xfb, 0x893: 0xfb, 0x894: 0xfb, 0x895: 0xfb, 0x896: 0xfb, 0x897: 0xfb,
+	0x898: 0xfb, 0x899: 0xfb, 0x89a: 0xfb, 0x89b: 0xfb, 0x89c: 0xfb, 0x89d: 0xfb, 0x89e: 0xfb, 0x89f: 0xfb,
+	0x8a0: 0xfb, 0x8a1: 0xfb, 0x8a2: 0xfb, 0x8a3: 0xfb, 0x8a4: 0xfb, 0x8a5: 0xfb, 0x8a6: 0xfb, 0x8a7: 0xfb,
+	0x8a8: 0xfb, 0x8a9: 0xfb, 0x8aa: 0xfb, 0x8ab: 0xfb, 0x8ac: 0xfb, 0x8ad: 0xfb, 0x8ae: 0xfb, 0x8af: 0xfb,
+	0x8b0: 0xfb, 0x8b1: 0xfb, 0x8b2: 0xfb, 0x8b3: 0xfb, 0x8b4: 0xfb, 0x8b5: 0xfb, 0x8b6: 0xfb, 0x8b7: 0xfb,
+	0x8b8: 0xfb, 0x8b9: 0xfb, 0x8ba: 0xfb, 0x8bb: 0xfb, 0x8bc: 0xfb, 0x8bd: 0xfb, 0x8be: 0xfb, 0x8bf: 0xfb,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x0b, 0x8c1: 0x0b, 0x8c2: 0x0b, 0x8c3: 0x0b, 0x8c4: 0x0b, 0x8c5: 0x0b, 0x8c6: 0x0b, 0x8c7: 0x0b,
+	0x8c8: 0x0b, 0x8c9: 0x0b, 0x8ca: 0x0b, 0x8cb: 0x0b, 0x8cc: 0x0b, 0x8cd: 0x0b, 0x8ce: 0x0b, 0x8cf: 0x0b,
+	0x8d0: 0x0b, 0x8d1: 0x0b, 0x8d2: 0x0b, 0x8d3: 0x0b, 0x8d4: 0x0b, 0x8d5: 0x0b, 0x8d6: 0x0b, 0x8d7: 0x0b,
+	0x8d8: 0x0b, 0x8d9: 0x0b, 0x8da: 0x0b, 0x8db: 0x0b, 0x8dc: 0x0b, 0x8dd: 0x0b, 0x8de: 0x0b, 0x8df: 0x0b,
+	0x8e0: 0x20, 0x8e1: 0x0b, 0x8e2: 0x0b, 0x8e3: 0x0b, 0x8e4: 0x0b, 0x8e5: 0x0b, 0x8e6: 0x0b, 0x8e7: 0x0b,
+	0x8e8: 0x0b, 0x8e9: 0x0b, 0x8ea: 0x0b, 0x8eb: 0x0b, 0x8ec: 0x0b, 0x8ed: 0x0b, 0x8ee: 0x0b, 0x8ef: 0x0b,
+	0x8f0: 0x0b, 0x8f1: 0x0b, 0x8f2: 0x0b, 0x8f3: 0x0b, 0x8f4: 0x0b, 0x8f5: 0x0b, 0x8f6: 0x0b, 0x8f7: 0x0b,
+	0x8f8: 0x0b, 0x8f9: 0x0b, 0x8fa: 0x0b, 0x8fb: 0x0b, 0x8fc: 0x0b, 0x8fd: 0x0b, 0x8fe: 0x0b, 0x8ff: 0x0b,
+	// Block 0x24, offset 0x900
+	0x900: 0x0b, 0x901: 0x0b, 0x902: 0x0b, 0x903: 0x0b, 0x904: 0x0b, 0x905: 0x0b, 0x906: 0x0b, 0x907: 0x0b,
+	0x908: 0x0b, 0x909: 0x0b, 0x90a: 0x0b, 0x90b: 0x0b, 0x90c: 0x0b, 0x90d: 0x0b, 0x90e: 0x0b, 0x90f: 0x0b,
+}
+
+// idnaSparseOffset: 292 entries, 584 bytes
+var idnaSparseOffset = []uint16{0x0, 0x8, 0x19, 0x25, 0x27, 0x2c, 0x33, 0x3e, 0x4a, 0x4e, 0x5d, 0x62, 0x6c, 0x78, 0x85, 0x8b, 0x94, 0xa4, 0xb2, 0xbd, 0xca, 0xdb, 0xe5, 0xec, 0xf9, 0x10a, 0x111, 0x11c, 0x12b, 0x139, 0x143, 0x145, 0x14a, 0x14d, 0x150, 0x152, 0x15e, 0x169, 0x171, 0x177, 0x17d, 0x182, 0x187, 0x18a, 0x18e, 0x194, 0x199, 0x1a5, 0x1af, 0x1b5, 0x1c6, 0x1d0, 0x1d3, 0x1db, 0x1de, 0x1eb, 0x1f3, 0x1f7, 0x1fe, 0x206, 0x216, 0x222, 0x225, 0x22f, 0x23b, 0x247, 0x253, 0x25b, 0x260, 0x26d, 0x27e, 0x282, 0x28d, 0x291, 0x29a, 0x2a2, 0x2a8, 0x2ad, 0x2b0, 0x2b4, 0x2ba, 0x2be, 0x2c2, 0x2c6, 0x2cc, 0x2d4, 0x2db, 0x2e6, 0x2f0, 0x2f4, 0x2f7, 0x2fd, 0x301, 0x303, 0x306, 0x308, 0x30b, 0x315, 0x318, 0x327, 0x32b, 0x330, 0x333, 0x337, 0x33c, 0x341, 0x347, 0x358, 0x368, 0x36e, 0x372, 0x381, 0x386, 0x38e, 0x398, 0x3a3, 0x3ab, 0x3bc, 0x3c5, 0x3d5, 0x3e2, 0x3ee, 0x3f3, 0x400, 0x404, 0x409, 0x40b, 0x40d, 0x411, 0x413, 0x417, 0x420, 0x426, 0x42a, 0x43a, 0x444, 0x449, 0x44c, 0x452, 0x459, 0x45e, 0x462, 0x468, 0x46d, 0x476, 0x47b, 0x481, 0x488, 0x48f, 0x496, 0x49a, 0x49f, 0x4a2, 0x4a7, 0x4b3, 0x4b9, 0x4be, 0x4c5, 0x4cd, 0x4d2, 0x4d6, 0x4e6, 0x4ed, 0x4f1, 0x4f5, 0x4fc, 0x4fe, 0x501, 0x504, 0x508, 0x511, 0x515, 0x51d, 0x525, 0x52d, 0x539, 0x545, 0x54b, 0x554, 0x560, 0x567, 0x570, 0x57b, 0x582, 0x591, 0x59e, 0x5ab, 0x5b4, 0x5b8, 0x5c7, 0x5cf, 0x5da, 0x5e3, 0x5e9, 0x5f1, 0x5fa, 0x605, 0x608, 0x614, 0x61d, 0x620, 0x625, 0x62e, 0x633, 0x640, 0x64b, 0x654, 0x65e, 0x661, 0x66b, 0x674, 0x680, 0x68d, 0x69a, 0x6a8, 0x6af, 0x6b3, 0x6b7, 0x6ba, 0x6bf, 0x6c2, 0x6c7, 0x6ca, 0x6d1, 0x6d8, 0x6dc, 0x6e7, 0x6ea, 0x6ed, 0x6f0, 0x6f6, 0x6fc, 0x705, 0x708, 0x70b, 0x70e, 0x711, 0x718, 0x71b, 0x720, 0x72a, 0x72d, 0x731, 0x740, 0x74c, 0x750, 0x755, 0x759, 0x75e, 0x762, 0x767, 0x770, 0x77b, 0x781, 0x787, 0x78d, 0x793, 0x79c, 0x79f, 0x7a2, 0x7a6, 0x7aa, 0x7ae, 0x7b4, 0x7ba, 0x7bf, 0x7c2, 0x7d2, 0x7d9, 0x7dc, 0x7e1, 0x7e5, 0x7eb, 0x7f2, 0x7f6, 0x7fa, 0x803, 0x80a, 0x80f, 0x813, 0x821, 0x824, 0x827, 0x82b, 0x82f, 0x832, 0x842, 0x853, 0x856, 0x85b, 0x85d, 0x85f}
+
+// idnaSparseValues: 2146 entries, 8584 bytes
+var idnaSparseValues = [2146]valueRange{
+	// Block 0x0, offset 0x0
+	{value: 0x0000, lo: 0x07},
+	{value: 0xe105, lo: 0x80, hi: 0x96},
+	{value: 0x0018, lo: 0x97, hi: 0x97},
+	{value: 0xe105, lo: 0x98, hi: 0x9e},
+	{value: 0x001f, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbf},
+	// Block 0x1, offset 0x8
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0xe01d, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x82},
+	{value: 0x0335, lo: 0x83, hi: 0x83},
+	{value: 0x034d, lo: 0x84, hi: 0x84},
+	{value: 0x0365, lo: 0x85, hi: 0x85},
+	{value: 0xe00d, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x87},
+	{value: 0xe00d, lo: 0x88, hi: 0x88},
+	{value: 0x0008, lo: 0x89, hi: 0x89},
+	{value: 0xe00d, lo: 0x8a, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0x8b},
+	{value: 0xe00d, lo: 0x8c, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0x8d},
+	{value: 0xe00d, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0xbf},
+	// Block 0x2, offset 0x19
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x0249, lo: 0xb0, hi: 0xb0},
+	{value: 0x037d, lo: 0xb1, hi: 0xb1},
+	{value: 0x0259, lo: 0xb2, hi: 0xb2},
+	{value: 0x0269, lo: 0xb3, hi: 0xb3},
+	{value: 0x034d, lo: 0xb4, hi: 0xb4},
+	{value: 0x0395, lo: 0xb5, hi: 0xb5},
+	{value: 0xe1bd, lo: 0xb6, hi: 0xb6},
+	{value: 0x0279, lo: 0xb7, hi: 0xb7},
+	{value: 0x0289, lo: 0xb8, hi: 0xb8},
+	{value: 0x0008, lo: 0xb9, hi: 0xbf},
+	// Block 0x3, offset 0x25
+	{value: 0x0000, lo: 0x01},
+	{value: 0x3308, lo: 0x80, hi: 0xbf},
+	// Block 0x4, offset 0x27
+	{value: 0x0000, lo: 0x04},
+	{value: 0x03f5, lo: 0x80, hi: 0x8f},
+	{value: 0xe105, lo: 0x90, hi: 0x9f},
+	{value: 0x049d, lo: 0xa0, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x5, offset 0x2c
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe185, lo: 0x80, hi: 0x8f},
+	{value: 0x0545, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x98},
+	{value: 0x0008, lo: 0x99, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x6, offset 0x33
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0401, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x88},
+	{value: 0x0018, lo: 0x89, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x3308, lo: 0x91, hi: 0xbd},
+	{value: 0x0818, lo: 0xbe, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x7, offset 0x3e
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0818, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x82},
+	{value: 0x0818, lo: 0x83, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x85},
+	{value: 0x0818, lo: 0x86, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xae},
+	{value: 0x0808, lo: 0xaf, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x8, offset 0x4a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0a08, lo: 0x80, hi: 0x87},
+	{value: 0x0c08, lo: 0x88, hi: 0x99},
+	{value: 0x0a08, lo: 0x9a, hi: 0xbf},
+	// Block 0x9, offset 0x4e
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3308, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8c},
+	{value: 0x0c08, lo: 0x8d, hi: 0x8d},
+	{value: 0x0a08, lo: 0x8e, hi: 0x98},
+	{value: 0x0c08, lo: 0x99, hi: 0x9b},
+	{value: 0x0a08, lo: 0x9c, hi: 0xaa},
+	{value: 0x0c08, lo: 0xab, hi: 0xac},
+	{value: 0x0a08, lo: 0xad, hi: 0xb0},
+	{value: 0x0c08, lo: 0xb1, hi: 0xb1},
+	{value: 0x0a08, lo: 0xb2, hi: 0xb2},
+	{value: 0x0c08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0a08, lo: 0xb5, hi: 0xb7},
+	{value: 0x0c08, lo: 0xb8, hi: 0xb9},
+	{value: 0x0a08, lo: 0xba, hi: 0xbf},
+	// Block 0xa, offset 0x5d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xb0},
+	{value: 0x0808, lo: 0xb1, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0xb, offset 0x62
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0808, lo: 0x80, hi: 0x89},
+	{value: 0x0a08, lo: 0x8a, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xb3},
+	{value: 0x0808, lo: 0xb4, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xb9},
+	{value: 0x0818, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x0818, lo: 0xbe, hi: 0xbf},
+	// Block 0xc, offset 0x6c
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x99},
+	{value: 0x0808, lo: 0x9a, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0xa3},
+	{value: 0x0808, lo: 0xa4, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa7},
+	{value: 0x0808, lo: 0xa8, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0818, lo: 0xb0, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xd, offset 0x78
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0a08, lo: 0xa0, hi: 0xa9},
+	{value: 0x0c08, lo: 0xaa, hi: 0xac},
+	{value: 0x0808, lo: 0xad, hi: 0xad},
+	{value: 0x0c08, lo: 0xae, hi: 0xae},
+	{value: 0x0a08, lo: 0xaf, hi: 0xb0},
+	{value: 0x0c08, lo: 0xb1, hi: 0xb2},
+	{value: 0x0a08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xb5},
+	{value: 0x0a08, lo: 0xb6, hi: 0xb8},
+	{value: 0x0c08, lo: 0xb9, hi: 0xb9},
+	{value: 0x0a08, lo: 0xba, hi: 0xbf},
+	// Block 0xe, offset 0x85
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0a08, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x92},
+	{value: 0x3308, lo: 0x93, hi: 0xa1},
+	{value: 0x0840, lo: 0xa2, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xbf},
+	// Block 0xf, offset 0x8b
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x10, offset 0x94
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x85},
+	{value: 0x3008, lo: 0x86, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x3008, lo: 0x8a, hi: 0x8c},
+	{value: 0x3b08, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x11, offset 0xa4
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbf},
+	// Block 0x12, offset 0xb2
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0xba},
+	{value: 0x3b08, lo: 0xbb, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x13, offset 0xbd
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xb2},
+	{value: 0x0008, lo: 0xb3, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x14, offset 0xca
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x89},
+	{value: 0x3b08, lo: 0x8a, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8e},
+	{value: 0x3008, lo: 0x8f, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x94},
+	{value: 0x0040, lo: 0x95, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x3008, lo: 0x98, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xb1},
+	{value: 0x3008, lo: 0xb2, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x15, offset 0xdb
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb2},
+	{value: 0x08f1, lo: 0xb3, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb9},
+	{value: 0x3b08, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0x16, offset 0xe5
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x8e},
+	{value: 0x0018, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0xbf},
+	// Block 0x17, offset 0xec
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x3308, lo: 0x88, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0961, lo: 0x9c, hi: 0x9c},
+	{value: 0x0999, lo: 0x9d, hi: 0x9d},
+	{value: 0x0008, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0x18, offset 0xf9
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0x8b},
+	{value: 0xe03d, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xb8},
+	{value: 0x3308, lo: 0xb9, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x19, offset 0x10a
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0018, lo: 0x8e, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0xbf},
+	// Block 0x1a, offset 0x111
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x3008, lo: 0xab, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xb0},
+	{value: 0x3008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0x1b, offset 0x11c
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x95},
+	{value: 0x3008, lo: 0x96, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0x9d},
+	{value: 0x3308, lo: 0x9e, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3008, lo: 0xa2, hi: 0xa4},
+	{value: 0x0008, lo: 0xa5, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xbf},
+	// Block 0x1c, offset 0x12b
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x3008, lo: 0x87, hi: 0x8c},
+	{value: 0x3308, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x8e},
+	{value: 0x3008, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x3008, lo: 0x9a, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0x1d, offset 0x139
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0040, lo: 0x80, hi: 0x86},
+	{value: 0x055d, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8c},
+	{value: 0x055d, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbb},
+	{value: 0xe105, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbf},
+	// Block 0x1e, offset 0x143
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0018, lo: 0x80, hi: 0xbf},
+	// Block 0x1f, offset 0x145
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xa0},
+	{value: 0x2018, lo: 0xa1, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0x20, offset 0x14a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xa7},
+	{value: 0x2018, lo: 0xa8, hi: 0xbf},
+	// Block 0x21, offset 0x14d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x2018, lo: 0x80, hi: 0x82},
+	{value: 0x0018, lo: 0x83, hi: 0xbf},
+	// Block 0x22, offset 0x150
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0008, lo: 0x80, hi: 0xbf},
+	// Block 0x23, offset 0x152
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x99},
+	{value: 0x0008, lo: 0x9a, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x24, offset 0x15e
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x25, offset 0x169
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0040, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbf},
+	// Block 0x26, offset 0x171
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0x0008, lo: 0x92, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbf},
+	// Block 0x27, offset 0x177
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x28, offset 0x17d
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x29, offset 0x182
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0xe045, lo: 0xb8, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x2a, offset 0x187
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xbf},
+	// Block 0x2b, offset 0x18a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xac},
+	{value: 0x0018, lo: 0xad, hi: 0xae},
+	{value: 0x0008, lo: 0xaf, hi: 0xbf},
+	// Block 0x2c, offset 0x18e
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9c},
+	{value: 0x0040, lo: 0x9d, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x2d, offset 0x194
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xb0},
+	{value: 0x0008, lo: 0xb1, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0x2e, offset 0x199
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x93},
+	{value: 0x3b08, lo: 0x94, hi: 0x94},
+	{value: 0x0040, lo: 0x95, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
+	{value: 0x0018, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x2f, offset 0x1a5
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x30, offset 0x1af
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xb3},
+	{value: 0x3340, lo: 0xb4, hi: 0xb5},
+	{value: 0x3008, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x31, offset 0x1b5
+	{value: 0x0000, lo: 0x10},
+	{value: 0x3008, lo: 0x80, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x3008, lo: 0x87, hi: 0x88},
+	{value: 0x3308, lo: 0x89, hi: 0x91},
+	{value: 0x3b08, lo: 0x92, hi: 0x92},
+	{value: 0x3308, lo: 0x93, hi: 0x93},
+	{value: 0x0018, lo: 0x94, hi: 0x96},
+	{value: 0x0008, lo: 0x97, hi: 0x97},
+	{value: 0x0018, lo: 0x98, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x32, offset 0x1c6
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0018, lo: 0x80, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x86},
+	{value: 0x0218, lo: 0x87, hi: 0x87},
+	{value: 0x0018, lo: 0x88, hi: 0x8a},
+	{value: 0x33c0, lo: 0x8b, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0208, lo: 0xa0, hi: 0xbf},
+	// Block 0x33, offset 0x1d0
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0208, lo: 0x80, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0x34, offset 0x1d3
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x0208, lo: 0x87, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xa9},
+	{value: 0x0208, lo: 0xaa, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x35, offset 0x1db
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0x36, offset 0x1de
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb8},
+	{value: 0x3308, lo: 0xb9, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x37, offset 0x1eb
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0018, lo: 0x80, hi: 0x80},
+	{value: 0x0040, lo: 0x81, hi: 0x83},
+	{value: 0x0018, lo: 0x84, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x38, offset 0x1f3
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x39, offset 0x1f7
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0028, lo: 0x9a, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0xbf},
+	// Block 0x3a, offset 0x1fe
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x3308, lo: 0x97, hi: 0x98},
+	{value: 0x3008, lo: 0x99, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x3b, offset 0x206
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0008, lo: 0x80, hi: 0x94},
+	{value: 0x3008, lo: 0x95, hi: 0x95},
+	{value: 0x3308, lo: 0x96, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xac},
+	{value: 0x3008, lo: 0xad, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x3c, offset 0x216
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0018, lo: 0xa8, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xbd},
+	{value: 0x3318, lo: 0xbe, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x3d, offset 0x222
+	{value: 0x0000, lo: 0x02},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x0040, lo: 0x81, hi: 0xbf},
+	// Block 0x3e, offset 0x225
+	{value: 0x0000, lo: 0x09},
+	{value: 0x3308, lo: 0x80, hi: 0x83},
+	{value: 0x3008, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x3008, lo: 0xbd, hi: 0xbf},
+	// Block 0x3f, offset 0x22f
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x3808, lo: 0x84, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x40, offset 0x23b
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa9},
+	{value: 0x3808, lo: 0xaa, hi: 0xaa},
+	{value: 0x3b08, lo: 0xab, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xbf},
+	// Block 0x41, offset 0x247
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa9},
+	{value: 0x3008, lo: 0xaa, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb1},
+	{value: 0x3808, lo: 0xb2, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbb},
+	{value: 0x0018, lo: 0xbc, hi: 0xbf},
+	// Block 0x42, offset 0x253
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x3008, lo: 0xa4, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbf},
+	// Block 0x43, offset 0x25b
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0x44, offset 0x260
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0e29, lo: 0x80, hi: 0x80},
+	{value: 0x0e41, lo: 0x81, hi: 0x81},
+	{value: 0x0e59, lo: 0x82, hi: 0x82},
+	{value: 0x0e71, lo: 0x83, hi: 0x83},
+	{value: 0x0e89, lo: 0x84, hi: 0x85},
+	{value: 0x0ea1, lo: 0x86, hi: 0x86},
+	{value: 0x0eb9, lo: 0x87, hi: 0x87},
+	{value: 0x057d, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x059d, lo: 0x90, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbc},
+	{value: 0x059d, lo: 0xbd, hi: 0xbf},
+	// Block 0x45, offset 0x26d
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x92},
+	{value: 0x0018, lo: 0x93, hi: 0x93},
+	{value: 0x3308, lo: 0x94, hi: 0xa0},
+	{value: 0x3008, lo: 0xa1, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa8},
+	{value: 0x0008, lo: 0xa9, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x0008, lo: 0xae, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xb6},
+	{value: 0x3008, lo: 0xb7, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x46, offset 0x27e
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3308, lo: 0x80, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbf},
+	// Block 0x47, offset 0x282
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x87},
+	{value: 0xe045, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0xe045, lo: 0x98, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0xe045, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb7},
+	{value: 0xe045, lo: 0xb8, hi: 0xbf},
+	// Block 0x48, offset 0x28d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x3318, lo: 0x90, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xbf},
+	// Block 0x49, offset 0x291
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x88},
+	{value: 0x24c1, lo: 0x89, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x4a, offset 0x29a
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0018, lo: 0x80, hi: 0xab},
+	{value: 0x24f1, lo: 0xac, hi: 0xac},
+	{value: 0x2529, lo: 0xad, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xae},
+	{value: 0x2579, lo: 0xaf, hi: 0xaf},
+	{value: 0x25b1, lo: 0xb0, hi: 0xb0},
+	{value: 0x0018, lo: 0xb1, hi: 0xbf},
+	// Block 0x4b, offset 0x2a2
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x9f},
+	{value: 0x0080, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xad},
+	{value: 0x0080, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x4c, offset 0x2a8
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0xa8},
+	{value: 0x09dd, lo: 0xa9, hi: 0xa9},
+	{value: 0x09fd, lo: 0xaa, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xbf},
+	// Block 0x4d, offset 0x2ad
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xbf},
+	// Block 0x4e, offset 0x2b0
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x28c1, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0xbf},
+	// Block 0x4f, offset 0x2b4
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0e7e, lo: 0xb4, hi: 0xb4},
+	{value: 0x292a, lo: 0xb5, hi: 0xb5},
+	{value: 0x0e9e, lo: 0xb6, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0x50, offset 0x2ba
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x9b},
+	{value: 0x2941, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0xbf},
+	// Block 0x51, offset 0x2be
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0x52, offset 0x2c2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x96},
+	{value: 0x0018, lo: 0x97, hi: 0xbf},
+	// Block 0x53, offset 0x2c6
+	{value: 0x0000, lo: 0x05},
+	{value: 0xe185, lo: 0x80, hi: 0x8f},
+	{value: 0x03f5, lo: 0x90, hi: 0x9f},
+	{value: 0x0ebd, lo: 0xa0, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x54, offset 0x2cc
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xac},
+	{value: 0x0008, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x55, offset 0x2d4
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xae},
+	{value: 0xe075, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0x56, offset 0x2db
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x57, offset 0x2e6
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xbf},
+	// Block 0x58, offset 0x2f0
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xae},
+	{value: 0x0008, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x59, offset 0x2f4
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x92},
+	{value: 0x0040, lo: 0x93, hi: 0xbf},
+	// Block 0x5a, offset 0x2f7
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9e},
+	{value: 0x0ef5, lo: 0x9f, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbf},
+	// Block 0x5b, offset 0x2fd
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb2},
+	{value: 0x0f15, lo: 0xb3, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x5c, offset 0x301
+	{value: 0x0020, lo: 0x01},
+	{value: 0x0f35, lo: 0x80, hi: 0xbf},
+	// Block 0x5d, offset 0x303
+	{value: 0x0020, lo: 0x02},
+	{value: 0x1735, lo: 0x80, hi: 0x8f},
+	{value: 0x1915, lo: 0x90, hi: 0xbf},
+	// Block 0x5e, offset 0x306
+	{value: 0x0020, lo: 0x01},
+	{value: 0x1f15, lo: 0x80, hi: 0xbf},
+	// Block 0x5f, offset 0x308
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0xbf},
+	// Block 0x60, offset 0x30b
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x98},
+	{value: 0x3308, lo: 0x99, hi: 0x9a},
+	{value: 0x29e2, lo: 0x9b, hi: 0x9b},
+	{value: 0x2a0a, lo: 0x9c, hi: 0x9c},
+	{value: 0x0008, lo: 0x9d, hi: 0x9e},
+	{value: 0x2a31, lo: 0x9f, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xbf},
+	// Block 0x61, offset 0x315
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xbe},
+	{value: 0x2a69, lo: 0xbf, hi: 0xbf},
+	// Block 0x62, offset 0x318
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0040, lo: 0x80, hi: 0x84},
+	{value: 0x0008, lo: 0x85, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xb0},
+	{value: 0x2a35, lo: 0xb1, hi: 0xb1},
+	{value: 0x2a55, lo: 0xb2, hi: 0xb2},
+	{value: 0x2a75, lo: 0xb3, hi: 0xb3},
+	{value: 0x2a95, lo: 0xb4, hi: 0xb4},
+	{value: 0x2a75, lo: 0xb5, hi: 0xb5},
+	{value: 0x2ab5, lo: 0xb6, hi: 0xb6},
+	{value: 0x2ad5, lo: 0xb7, hi: 0xb7},
+	{value: 0x2af5, lo: 0xb8, hi: 0xb9},
+	{value: 0x2b15, lo: 0xba, hi: 0xbb},
+	{value: 0x2b35, lo: 0xbc, hi: 0xbd},
+	{value: 0x2b15, lo: 0xbe, hi: 0xbf},
+	// Block 0x63, offset 0x327
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x64, offset 0x32b
+	{value: 0x0030, lo: 0x04},
+	{value: 0x2aa2, lo: 0x80, hi: 0x9d},
+	{value: 0x305a, lo: 0x9e, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x30a2, lo: 0xa0, hi: 0xbf},
+	// Block 0x65, offset 0x330
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x66, offset 0x333
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0040, lo: 0x8d, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x67, offset 0x337
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0x68, offset 0x33c
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xbf},
+	// Block 0x69, offset 0x341
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x0018, lo: 0xa6, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb1},
+	{value: 0x0018, lo: 0xb2, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0x6a, offset 0x347
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0040, lo: 0x80, hi: 0x81},
+	{value: 0xe00d, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0x83},
+	{value: 0x03f5, lo: 0x84, hi: 0x84},
+	{value: 0x1329, lo: 0x85, hi: 0x85},
+	{value: 0x447d, lo: 0x86, hi: 0x86},
+	{value: 0xe07d, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x88},
+	{value: 0xe01d, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0xb4},
+	{value: 0xe01d, lo: 0xb5, hi: 0xb5},
+	{value: 0x0008, lo: 0xb6, hi: 0xb7},
+	{value: 0x2009, lo: 0xb8, hi: 0xb8},
+	{value: 0x6ec1, lo: 0xb9, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xbf},
+	// Block 0x6b, offset 0x358
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0008, lo: 0x80, hi: 0x81},
+	{value: 0x3308, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0x85},
+	{value: 0x3b08, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x8a},
+	{value: 0x3308, lo: 0x8b, hi: 0x8b},
+	{value: 0x0008, lo: 0x8c, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa6},
+	{value: 0x3008, lo: 0xa7, hi: 0xa7},
+	{value: 0x0018, lo: 0xa8, hi: 0xab},
+	{value: 0x3b08, lo: 0xac, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x6c, offset 0x368
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0208, lo: 0x80, hi: 0xb1},
+	{value: 0x0108, lo: 0xb2, hi: 0xb2},
+	{value: 0x0008, lo: 0xb3, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0x6d, offset 0x36e
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xbf},
+	// Block 0x6e, offset 0x372
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3008, lo: 0x80, hi: 0x83},
+	{value: 0x3b08, lo: 0x84, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8d},
+	{value: 0x0018, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xba},
+	{value: 0x0008, lo: 0xbb, hi: 0xbb},
+	{value: 0x0018, lo: 0xbc, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0x6f, offset 0x381
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x70, offset 0x386
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x91},
+	{value: 0x3008, lo: 0x92, hi: 0x92},
+	{value: 0x3808, lo: 0x93, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x71, offset 0x38e
+	{value: 0x0000, lo: 0x09},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x3008, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb9},
+	{value: 0x3008, lo: 0xba, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbf},
+	// Block 0x72, offset 0x398
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3808, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0x73, offset 0x3a3
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xa8},
+	{value: 0x3308, lo: 0xa9, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x74, offset 0x3ab
+	{value: 0x0000, lo: 0x10},
+	{value: 0x0008, lo: 0x80, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x8b},
+	{value: 0x3308, lo: 0x8c, hi: 0x8c},
+	{value: 0x3008, lo: 0x8d, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0018, lo: 0x9c, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbc},
+	{value: 0x3008, lo: 0xbd, hi: 0xbd},
+	{value: 0x0008, lo: 0xbe, hi: 0xbf},
+	// Block 0x75, offset 0x3bc
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb0},
+	{value: 0x0008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb4},
+	{value: 0x0008, lo: 0xb5, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb8},
+	{value: 0x0008, lo: 0xb9, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbf},
+	// Block 0x76, offset 0x3c5
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x9a},
+	{value: 0x0008, lo: 0x9b, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xaa},
+	{value: 0x3008, lo: 0xab, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb5},
+	{value: 0x3b08, lo: 0xb6, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x77, offset 0x3d5
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x88},
+	{value: 0x0008, lo: 0x89, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x90},
+	{value: 0x0008, lo: 0x91, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x78, offset 0x3e2
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9b},
+	{value: 0x449d, lo: 0x9c, hi: 0x9c},
+	{value: 0x44b5, lo: 0x9d, hi: 0x9d},
+	{value: 0x2971, lo: 0x9e, hi: 0x9e},
+	{value: 0xe06d, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa8},
+	{value: 0x6ed9, lo: 0xa9, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x44cd, lo: 0xb0, hi: 0xbf},
+	// Block 0x79, offset 0x3ee
+	{value: 0x0000, lo: 0x04},
+	{value: 0x44ed, lo: 0x80, hi: 0x8f},
+	{value: 0x450d, lo: 0x90, hi: 0x9f},
+	{value: 0x452d, lo: 0xa0, hi: 0xaf},
+	{value: 0x450d, lo: 0xb0, hi: 0xbf},
+	// Block 0x7a, offset 0x3f3
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0xa2},
+	{value: 0x3008, lo: 0xa3, hi: 0xa4},
+	{value: 0x3308, lo: 0xa5, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa7},
+	{value: 0x3308, lo: 0xa8, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xaa},
+	{value: 0x0018, lo: 0xab, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3b08, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x7b, offset 0x400
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x7c, offset 0x404
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8a},
+	{value: 0x0018, lo: 0x8b, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x7d, offset 0x409
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0040, lo: 0x80, hi: 0xbf},
+	// Block 0x7e, offset 0x40b
+	{value: 0x0020, lo: 0x01},
+	{value: 0x454d, lo: 0x80, hi: 0xbf},
+	// Block 0x7f, offset 0x40d
+	{value: 0x0020, lo: 0x03},
+	{value: 0x4d4d, lo: 0x80, hi: 0x94},
+	{value: 0x4b0d, lo: 0x95, hi: 0x95},
+	{value: 0x4fed, lo: 0x96, hi: 0xbf},
+	// Block 0x80, offset 0x411
+	{value: 0x0020, lo: 0x01},
+	{value: 0x552d, lo: 0x80, hi: 0xbf},
+	// Block 0x81, offset 0x413
+	{value: 0x0020, lo: 0x03},
+	{value: 0x5d2d, lo: 0x80, hi: 0x84},
+	{value: 0x568d, lo: 0x85, hi: 0x85},
+	{value: 0x5dcd, lo: 0x86, hi: 0xbf},
+	// Block 0x82, offset 0x417
+	{value: 0x0020, lo: 0x08},
+	{value: 0x6b8d, lo: 0x80, hi: 0x8f},
+	{value: 0x6d4d, lo: 0x90, hi: 0x90},
+	{value: 0x6d8d, lo: 0x91, hi: 0xab},
+	{value: 0x6ef1, lo: 0xac, hi: 0xac},
+	{value: 0x70ed, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x710d, lo: 0xb0, hi: 0xbf},
+	// Block 0x83, offset 0x420
+	{value: 0x0020, lo: 0x05},
+	{value: 0x730d, lo: 0x80, hi: 0xad},
+	{value: 0x656d, lo: 0xae, hi: 0xae},
+	{value: 0x78cd, lo: 0xaf, hi: 0xb5},
+	{value: 0x6f8d, lo: 0xb6, hi: 0xb6},
+	{value: 0x79ad, lo: 0xb7, hi: 0xbf},
+	// Block 0x84, offset 0x426
+	{value: 0x0028, lo: 0x03},
+	{value: 0x7c71, lo: 0x80, hi: 0x82},
+	{value: 0x7c31, lo: 0x83, hi: 0x83},
+	{value: 0x7ce9, lo: 0x84, hi: 0xbf},
+	// Block 0x85, offset 0x42a
+	{value: 0x0038, lo: 0x0f},
+	{value: 0x9e01, lo: 0x80, hi: 0x83},
+	{value: 0x9ea9, lo: 0x84, hi: 0x85},
+	{value: 0x9ee1, lo: 0x86, hi: 0x87},
+	{value: 0x9f19, lo: 0x88, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x91},
+	{value: 0xa0d9, lo: 0x92, hi: 0x97},
+	{value: 0xa1f1, lo: 0x98, hi: 0x9c},
+	{value: 0xa2d1, lo: 0x9d, hi: 0xb3},
+	{value: 0x9d91, lo: 0xb4, hi: 0xb4},
+	{value: 0x9e01, lo: 0xb5, hi: 0xb5},
+	{value: 0xa7d9, lo: 0xb6, hi: 0xbb},
+	{value: 0xa8b9, lo: 0xbc, hi: 0xbc},
+	{value: 0xa849, lo: 0xbd, hi: 0xbd},
+	{value: 0xa929, lo: 0xbe, hi: 0xbf},
+	// Block 0x86, offset 0x43a
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8c},
+	{value: 0x0008, lo: 0x8d, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbb},
+	{value: 0x0008, lo: 0xbc, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0x87, offset 0x444
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0xbf},
+	// Block 0x88, offset 0x449
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x89, offset 0x44c
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0x8a, offset 0x452
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x9c},
+	{value: 0x0040, lo: 0x9d, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa0},
+	{value: 0x0040, lo: 0xa1, hi: 0xbf},
+	// Block 0x8b, offset 0x459
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x8c, offset 0x45e
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x9c},
+	{value: 0x0040, lo: 0x9d, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x8d, offset 0x462
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x0040, lo: 0x91, hi: 0x9f},
+	{value: 0x3308, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x8e, offset 0x468
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xac},
+	{value: 0x0008, lo: 0xad, hi: 0xbf},
+	// Block 0x8f, offset 0x46d
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x81},
+	{value: 0x0008, lo: 0x82, hi: 0x89},
+	{value: 0x0018, lo: 0x8a, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x90, offset 0x476
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x91, offset 0x47b
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0xbf},
+	// Block 0x92, offset 0x481
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe145, lo: 0x80, hi: 0x87},
+	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
+	{value: 0xe145, lo: 0x90, hi: 0x97},
+	{value: 0x8b0d, lo: 0x98, hi: 0x9f},
+	{value: 0x8b25, lo: 0xa0, hi: 0xa7},
+	{value: 0x0008, lo: 0xa8, hi: 0xbf},
+	// Block 0x93, offset 0x488
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x8b25, lo: 0xb0, hi: 0xb7},
+	{value: 0x8b0d, lo: 0xb8, hi: 0xbf},
+	// Block 0x94, offset 0x48f
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe145, lo: 0x80, hi: 0x87},
+	{value: 0xe1c5, lo: 0x88, hi: 0x8f},
+	{value: 0xe145, lo: 0x90, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0x95, offset 0x496
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x96, offset 0x49a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xae},
+	{value: 0x0018, lo: 0xaf, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0x97, offset 0x49f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x98, offset 0x4a2
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xbf},
+	// Block 0x99, offset 0x4a7
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0808, lo: 0x80, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x87},
+	{value: 0x0808, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0808, lo: 0x8a, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb6},
+	{value: 0x0808, lo: 0xb7, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbb},
+	{value: 0x0808, lo: 0xbc, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbe},
+	{value: 0x0808, lo: 0xbf, hi: 0xbf},
+	// Block 0x9a, offset 0x4b3
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x96},
+	{value: 0x0818, lo: 0x97, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb6},
+	{value: 0x0818, lo: 0xb7, hi: 0xbf},
+	// Block 0x9b, offset 0x4b9
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xa6},
+	{value: 0x0818, lo: 0xa7, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0x9c, offset 0x4be
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb3},
+	{value: 0x0808, lo: 0xb4, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xba},
+	{value: 0x0818, lo: 0xbb, hi: 0xbf},
+	// Block 0x9d, offset 0x4c5
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0818, lo: 0x96, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbe},
+	{value: 0x0818, lo: 0xbf, hi: 0xbf},
+	// Block 0x9e, offset 0x4cd
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0808, lo: 0x80, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbb},
+	{value: 0x0818, lo: 0xbc, hi: 0xbd},
+	{value: 0x0808, lo: 0xbe, hi: 0xbf},
+	// Block 0x9f, offset 0x4d2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0818, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x91},
+	{value: 0x0818, lo: 0x92, hi: 0xbf},
+	// Block 0xa0, offset 0x4d6
+	{value: 0x0000, lo: 0x0f},
+	{value: 0x0808, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0x84},
+	{value: 0x3308, lo: 0x85, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8b},
+	{value: 0x3308, lo: 0x8c, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x94},
+	{value: 0x0808, lo: 0x95, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0x98},
+	{value: 0x0808, lo: 0x99, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xa1, offset 0x4e6
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0818, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x0818, lo: 0x90, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xbc},
+	{value: 0x0818, lo: 0xbd, hi: 0xbf},
+	// Block 0xa2, offset 0x4ed
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0x9c},
+	{value: 0x0818, lo: 0x9d, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xa3, offset 0x4f1
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb8},
+	{value: 0x0018, lo: 0xb9, hi: 0xbf},
+	// Block 0xa4, offset 0x4f5
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0808, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0x97},
+	{value: 0x0818, lo: 0x98, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb7},
+	{value: 0x0818, lo: 0xb8, hi: 0xbf},
+	// Block 0xa5, offset 0x4fc
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0808, lo: 0x80, hi: 0xbf},
+	// Block 0xa6, offset 0x4fe
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0808, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0xbf},
+	// Block 0xa7, offset 0x501
+	{value: 0x0000, lo: 0x02},
+	{value: 0x03dd, lo: 0x80, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xbf},
+	// Block 0xa8, offset 0x504
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0808, lo: 0x80, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xb9},
+	{value: 0x0818, lo: 0xba, hi: 0xbf},
+	// Block 0xa9, offset 0x508
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0908, lo: 0x80, hi: 0x80},
+	{value: 0x0a08, lo: 0x81, hi: 0xa1},
+	{value: 0x0c08, lo: 0xa2, hi: 0xa2},
+	{value: 0x0a08, lo: 0xa3, hi: 0xa3},
+	{value: 0x3308, lo: 0xa4, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0808, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xaa, offset 0x511
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0818, lo: 0xa0, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xab, offset 0x515
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0808, lo: 0x80, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xac},
+	{value: 0x0818, lo: 0xad, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0808, lo: 0xb0, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0xac, offset 0x51d
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0808, lo: 0x80, hi: 0x9c},
+	{value: 0x0818, lo: 0x9d, hi: 0xa6},
+	{value: 0x0808, lo: 0xa7, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0a08, lo: 0xb0, hi: 0xb2},
+	{value: 0x0c08, lo: 0xb3, hi: 0xb3},
+	{value: 0x0a08, lo: 0xb4, hi: 0xbf},
+	// Block 0xad, offset 0x525
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0a08, lo: 0x80, hi: 0x84},
+	{value: 0x0808, lo: 0x85, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x90},
+	{value: 0x0a18, lo: 0x91, hi: 0x93},
+	{value: 0x0c18, lo: 0x94, hi: 0x94},
+	{value: 0x0818, lo: 0x95, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xae, offset 0x52d
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0040, lo: 0x80, hi: 0xaf},
+	{value: 0x0a08, lo: 0xb0, hi: 0xb0},
+	{value: 0x0808, lo: 0xb1, hi: 0xb1},
+	{value: 0x0a08, lo: 0xb2, hi: 0xb3},
+	{value: 0x0c08, lo: 0xb4, hi: 0xb6},
+	{value: 0x0808, lo: 0xb7, hi: 0xb7},
+	{value: 0x0a08, lo: 0xb8, hi: 0xb8},
+	{value: 0x0c08, lo: 0xb9, hi: 0xba},
+	{value: 0x0a08, lo: 0xbb, hi: 0xbc},
+	{value: 0x0c08, lo: 0xbd, hi: 0xbd},
+	{value: 0x0a08, lo: 0xbe, hi: 0xbf},
+	// Block 0xaf, offset 0x539
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0808, lo: 0x80, hi: 0x80},
+	{value: 0x0a08, lo: 0x81, hi: 0x81},
+	{value: 0x0c08, lo: 0x82, hi: 0x83},
+	{value: 0x0a08, lo: 0x84, hi: 0x84},
+	{value: 0x0818, lo: 0x85, hi: 0x88},
+	{value: 0x0c18, lo: 0x89, hi: 0x89},
+	{value: 0x0a18, lo: 0x8a, hi: 0x8a},
+	{value: 0x0918, lo: 0x8b, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x9f},
+	{value: 0x0808, lo: 0xa0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xb0, offset 0x545
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbf},
+	// Block 0xb1, offset 0x54b
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x85},
+	{value: 0x3b08, lo: 0x86, hi: 0x86},
+	{value: 0x0018, lo: 0x87, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x91},
+	{value: 0x0018, lo: 0x92, hi: 0xa5},
+	{value: 0x0008, lo: 0xa6, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xb2, offset 0x554
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb6},
+	{value: 0x3008, lo: 0xb7, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbd},
+	{value: 0x0018, lo: 0xbe, hi: 0xbf},
+	// Block 0xb3, offset 0x560
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x81},
+	{value: 0x0040, lo: 0x82, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xb4, offset 0x567
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xb2},
+	{value: 0x3b08, lo: 0xb3, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xb5},
+	{value: 0x0008, lo: 0xb6, hi: 0xbf},
+	// Block 0xb5, offset 0x570
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0018, lo: 0x80, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x3008, lo: 0x85, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x0018, lo: 0xb4, hi: 0xb5},
+	{value: 0x0008, lo: 0xb6, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xb6, offset 0x57b
+	{value: 0x0000, lo: 0x06},
+	{value: 0x3308, lo: 0x80, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x0008, lo: 0x83, hi: 0xb2},
+	{value: 0x3008, lo: 0xb3, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xbe},
+	{value: 0x3008, lo: 0xbf, hi: 0xbf},
+	// Block 0xb7, offset 0x582
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3808, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x88},
+	{value: 0x3308, lo: 0x89, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0x8d},
+	{value: 0x3008, lo: 0x8e, hi: 0x8e},
+	{value: 0x3308, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa0},
+	{value: 0x0018, lo: 0xa1, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0xb8, offset 0x591
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x92},
+	{value: 0x0008, lo: 0x93, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb1},
+	{value: 0x3008, lo: 0xb2, hi: 0xb3},
+	{value: 0x3308, lo: 0xb4, hi: 0xb4},
+	{value: 0x3808, lo: 0xb5, hi: 0xb5},
+	{value: 0x3308, lo: 0xb6, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xbd},
+	{value: 0x3308, lo: 0xbe, hi: 0xbe},
+	{value: 0x0040, lo: 0xbf, hi: 0xbf},
+	// Block 0xb9, offset 0x59e
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0x8d},
+	{value: 0x0040, lo: 0x8e, hi: 0x8e},
+	{value: 0x0008, lo: 0x8f, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9e},
+	{value: 0x0008, lo: 0x9f, hi: 0xa8},
+	{value: 0x0018, lo: 0xa9, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0xba, offset 0x5ab
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x3308, lo: 0x9f, hi: 0x9f},
+	{value: 0x3008, lo: 0xa0, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xa9},
+	{value: 0x3b08, lo: 0xaa, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0xbb, offset 0x5b4
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbf},
+	// Block 0xbc, offset 0x5b8
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3008, lo: 0x80, hi: 0x81},
+	{value: 0x3b08, lo: 0x82, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x84},
+	{value: 0x3008, lo: 0x85, hi: 0x85},
+	{value: 0x3308, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x8a},
+	{value: 0x0018, lo: 0x8b, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9b},
+	{value: 0x0040, lo: 0x9c, hi: 0x9c},
+	{value: 0x0018, lo: 0x9d, hi: 0x9d},
+	{value: 0x3308, lo: 0x9e, hi: 0x9e},
+	{value: 0x0008, lo: 0x9f, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xbf},
+	// Block 0xbd, offset 0x5c7
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb8},
+	{value: 0x3008, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0xbe, offset 0x5cf
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x3008, lo: 0x81, hi: 0x81},
+	{value: 0x3b08, lo: 0x82, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x85},
+	{value: 0x0018, lo: 0x86, hi: 0x86},
+	{value: 0x0008, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xbf, offset 0x5da
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xc0, offset 0x5e3
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x9b},
+	{value: 0x3308, lo: 0x9c, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0xbf},
+	// Block 0xc1, offset 0x5e9
+	{value: 0x0000, lo: 0x07},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xba},
+	{value: 0x3008, lo: 0xbb, hi: 0xbc},
+	{value: 0x3308, lo: 0xbd, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xc2, offset 0x5f1
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3308, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x83},
+	{value: 0x0008, lo: 0x84, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xbf},
+	// Block 0xc3, offset 0x5fa
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x3308, lo: 0xab, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xac},
+	{value: 0x3308, lo: 0xad, hi: 0xad},
+	{value: 0x3008, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb5},
+	{value: 0x3808, lo: 0xb6, hi: 0xb6},
+	{value: 0x3308, lo: 0xb7, hi: 0xb7},
+	{value: 0x0008, lo: 0xb8, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xc4, offset 0x605
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0xbf},
+	// Block 0xc5, offset 0x608
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9f},
+	{value: 0x3008, lo: 0xa0, hi: 0xa1},
+	{value: 0x3308, lo: 0xa2, hi: 0xa5},
+	{value: 0x3008, lo: 0xa6, hi: 0xa6},
+	{value: 0x3308, lo: 0xa7, hi: 0xaa},
+	{value: 0x3b08, lo: 0xab, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbf},
+	// Block 0xc6, offset 0x614
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x3008, lo: 0xac, hi: 0xae},
+	{value: 0x3308, lo: 0xaf, hi: 0xb7},
+	{value: 0x3008, lo: 0xb8, hi: 0xb8},
+	{value: 0x3b08, lo: 0xb9, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0018, lo: 0xbb, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0xc7, offset 0x61d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x049d, lo: 0xa0, hi: 0xbf},
+	// Block 0xc8, offset 0x620
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xb2},
+	{value: 0x0040, lo: 0xb3, hi: 0xbe},
+	{value: 0x0008, lo: 0xbf, hi: 0xbf},
+	// Block 0xc9, offset 0x625
+	{value: 0x0000, lo: 0x08},
+	{value: 0x3008, lo: 0x80, hi: 0x80},
+	{value: 0x0008, lo: 0x81, hi: 0x81},
+	{value: 0x3008, lo: 0x82, hi: 0x82},
+	{value: 0x3308, lo: 0x83, hi: 0x83},
+	{value: 0x0018, lo: 0x84, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xca, offset 0x62e
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xbf},
+	// Block 0xcb, offset 0x633
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x90},
+	{value: 0x3008, lo: 0x91, hi: 0x93},
+	{value: 0x3308, lo: 0x94, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0x99},
+	{value: 0x3308, lo: 0x9a, hi: 0x9b},
+	{value: 0x3008, lo: 0x9c, hi: 0x9f},
+	{value: 0x3b08, lo: 0xa0, hi: 0xa0},
+	{value: 0x0008, lo: 0xa1, hi: 0xa1},
+	{value: 0x0018, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xa3},
+	{value: 0x3008, lo: 0xa4, hi: 0xa4},
+	{value: 0x0040, lo: 0xa5, hi: 0xbf},
+	// Block 0xcc, offset 0x640
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x3308, lo: 0x81, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb3},
+	{value: 0x3b08, lo: 0xb4, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb8},
+	{value: 0x3008, lo: 0xb9, hi: 0xb9},
+	{value: 0x0008, lo: 0xba, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xcd, offset 0x64b
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x3b08, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x3308, lo: 0x91, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x98},
+	{value: 0x3308, lo: 0x99, hi: 0x9b},
+	{value: 0x0008, lo: 0x9c, hi: 0xbf},
+	// Block 0xce, offset 0x654
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x3308, lo: 0x8a, hi: 0x96},
+	{value: 0x3008, lo: 0x97, hi: 0x97},
+	{value: 0x3308, lo: 0x98, hi: 0x98},
+	{value: 0x3b08, lo: 0x99, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0x9c},
+	{value: 0x0008, lo: 0x9d, hi: 0x9d},
+	{value: 0x0018, lo: 0x9e, hi: 0xa2},
+	{value: 0x0040, lo: 0xa3, hi: 0xbf},
+	// Block 0xcf, offset 0x65e
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xd0, offset 0x661
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x89},
+	{value: 0x0008, lo: 0x8a, hi: 0xae},
+	{value: 0x3008, lo: 0xaf, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb7},
+	{value: 0x3308, lo: 0xb8, hi: 0xbd},
+	{value: 0x3008, lo: 0xbe, hi: 0xbe},
+	{value: 0x3b08, lo: 0xbf, hi: 0xbf},
+	// Block 0xd1, offset 0x66b
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0008, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0018, lo: 0x9a, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0008, lo: 0xb2, hi: 0xbf},
+	// Block 0xd2, offset 0x674
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x91},
+	{value: 0x3308, lo: 0x92, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xa8},
+	{value: 0x3008, lo: 0xa9, hi: 0xa9},
+	{value: 0x3308, lo: 0xaa, hi: 0xb0},
+	{value: 0x3008, lo: 0xb1, hi: 0xb1},
+	{value: 0x3308, lo: 0xb2, hi: 0xb3},
+	{value: 0x3008, lo: 0xb4, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0xd3, offset 0x680
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x0008, lo: 0x88, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8a},
+	{value: 0x0008, lo: 0x8b, hi: 0xb0},
+	{value: 0x3308, lo: 0xb1, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xb9},
+	{value: 0x3308, lo: 0xba, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbb},
+	{value: 0x3308, lo: 0xbc, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbe},
+	{value: 0x3308, lo: 0xbf, hi: 0xbf},
+	// Block 0xd4, offset 0x68d
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x3308, lo: 0x80, hi: 0x83},
+	{value: 0x3b08, lo: 0x84, hi: 0x85},
+	{value: 0x0008, lo: 0x86, hi: 0x86},
+	{value: 0x3308, lo: 0x87, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xa6},
+	{value: 0x0008, lo: 0xa7, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xa9},
+	{value: 0x0008, lo: 0xaa, hi: 0xbf},
+	// Block 0xd5, offset 0x69a
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x3008, lo: 0x8a, hi: 0x8e},
+	{value: 0x0040, lo: 0x8f, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x92},
+	{value: 0x3008, lo: 0x93, hi: 0x94},
+	{value: 0x3308, lo: 0x95, hi: 0x95},
+	{value: 0x3008, lo: 0x96, hi: 0x96},
+	{value: 0x3b08, lo: 0x97, hi: 0x97},
+	{value: 0x0008, lo: 0x98, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xbf},
+	// Block 0xd6, offset 0x6a8
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xb2},
+	{value: 0x3308, lo: 0xb3, hi: 0xb4},
+	{value: 0x3008, lo: 0xb5, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xd7, offset 0x6af
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb0},
+	{value: 0x0040, lo: 0xb1, hi: 0xbf},
+	// Block 0xd8, offset 0x6b3
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xd9, offset 0x6b7
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0xbf},
+	// Block 0xda, offset 0x6ba
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0xdb, offset 0x6bf
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0040, lo: 0x84, hi: 0xbf},
+	// Block 0xdc, offset 0x6c2
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xaf},
+	{value: 0x0340, lo: 0xb0, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xdd, offset 0x6c7
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0xbf},
+	// Block 0xde, offset 0x6ca
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa9},
+	{value: 0x0040, lo: 0xaa, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0xdf, offset 0x6d1
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb4},
+	{value: 0x0018, lo: 0xb5, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0xe0, offset 0x6d8
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xbf},
+	// Block 0xe1, offset 0x6dc
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x0008, lo: 0x80, hi: 0x83},
+	{value: 0x0018, lo: 0x84, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9a},
+	{value: 0x0018, lo: 0x9b, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbc},
+	{value: 0x0008, lo: 0xbd, hi: 0xbf},
+	// Block 0xe2, offset 0x6e7
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0xbf},
+	// Block 0xe3, offset 0x6ea
+	{value: 0x0000, lo: 0x02},
+	{value: 0xe105, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0xe4, offset 0x6ed
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0x9a},
+	{value: 0x0040, lo: 0x9b, hi: 0xbf},
+	// Block 0xe5, offset 0x6f0
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0x8e},
+	{value: 0x3308, lo: 0x8f, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x90},
+	{value: 0x3008, lo: 0x91, hi: 0xbf},
+	// Block 0xe6, offset 0x6f6
+	{value: 0x0000, lo: 0x05},
+	{value: 0x3008, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8e},
+	{value: 0x3308, lo: 0x8f, hi: 0x92},
+	{value: 0x0008, lo: 0x93, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0xe7, offset 0x6fc
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xa1},
+	{value: 0x0018, lo: 0xa2, hi: 0xa2},
+	{value: 0x0008, lo: 0xa3, hi: 0xa3},
+	{value: 0x3308, lo: 0xa4, hi: 0xa4},
+	{value: 0x0040, lo: 0xa5, hi: 0xaf},
+	{value: 0x3008, lo: 0xb0, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0xe8, offset 0x705
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb7},
+	{value: 0x0040, lo: 0xb8, hi: 0xbf},
+	// Block 0xe9, offset 0x708
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x95},
+	{value: 0x0040, lo: 0x96, hi: 0xbf},
+	// Block 0xea, offset 0x70b
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0xbf},
+	// Block 0xeb, offset 0x70e
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x9e},
+	{value: 0x0040, lo: 0x9f, hi: 0xbf},
+	// Block 0xec, offset 0x711
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0040, lo: 0x80, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x92},
+	{value: 0x0040, lo: 0x93, hi: 0xa3},
+	{value: 0x0008, lo: 0xa4, hi: 0xa7},
+	{value: 0x0040, lo: 0xa8, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0xed, offset 0x718
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xbb},
+	{value: 0x0040, lo: 0xbc, hi: 0xbf},
+	// Block 0xee, offset 0x71b
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0008, lo: 0x80, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0xef, offset 0x720
+	{value: 0x0000, lo: 0x09},
+	{value: 0x0008, lo: 0x80, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0x0008, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9b},
+	{value: 0x0018, lo: 0x9c, hi: 0x9c},
+	{value: 0x3308, lo: 0x9d, hi: 0x9e},
+	{value: 0x0018, lo: 0x9f, hi: 0x9f},
+	{value: 0x03c0, lo: 0xa0, hi: 0xa3},
+	{value: 0x0040, lo: 0xa4, hi: 0xbf},
+	// Block 0xf0, offset 0x72a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0xf1, offset 0x72d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xa6},
+	{value: 0x0040, lo: 0xa7, hi: 0xa8},
+	{value: 0x0018, lo: 0xa9, hi: 0xbf},
+	// Block 0xf2, offset 0x731
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x0018, lo: 0x80, hi: 0x9d},
+	{value: 0xb609, lo: 0x9e, hi: 0x9e},
+	{value: 0xb651, lo: 0x9f, hi: 0x9f},
+	{value: 0xb699, lo: 0xa0, hi: 0xa0},
+	{value: 0xb701, lo: 0xa1, hi: 0xa1},
+	{value: 0xb769, lo: 0xa2, hi: 0xa2},
+	{value: 0xb7d1, lo: 0xa3, hi: 0xa3},
+	{value: 0xb839, lo: 0xa4, hi: 0xa4},
+	{value: 0x3018, lo: 0xa5, hi: 0xa6},
+	{value: 0x3318, lo: 0xa7, hi: 0xa9},
+	{value: 0x0018, lo: 0xaa, hi: 0xac},
+	{value: 0x3018, lo: 0xad, hi: 0xb2},
+	{value: 0x0340, lo: 0xb3, hi: 0xba},
+	{value: 0x3318, lo: 0xbb, hi: 0xbf},
+	// Block 0xf3, offset 0x740
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x3318, lo: 0x80, hi: 0x82},
+	{value: 0x0018, lo: 0x83, hi: 0x84},
+	{value: 0x3318, lo: 0x85, hi: 0x8b},
+	{value: 0x0018, lo: 0x8c, hi: 0xa9},
+	{value: 0x3318, lo: 0xaa, hi: 0xad},
+	{value: 0x0018, lo: 0xae, hi: 0xba},
+	{value: 0xb8a1, lo: 0xbb, hi: 0xbb},
+	{value: 0xb8e9, lo: 0xbc, hi: 0xbc},
+	{value: 0xb931, lo: 0xbd, hi: 0xbd},
+	{value: 0xb999, lo: 0xbe, hi: 0xbe},
+	{value: 0xba01, lo: 0xbf, hi: 0xbf},
+	// Block 0xf4, offset 0x74c
+	{value: 0x0000, lo: 0x03},
+	{value: 0xba69, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xbf},
+	// Block 0xf5, offset 0x750
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x81},
+	{value: 0x3318, lo: 0x82, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x85},
+	{value: 0x0040, lo: 0x86, hi: 0xbf},
+	// Block 0xf6, offset 0x755
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0xf7, offset 0x759
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xbf},
+	// Block 0xf8, offset 0x75e
+	{value: 0x0000, lo: 0x03},
+	{value: 0x3308, lo: 0x80, hi: 0xb6},
+	{value: 0x0018, lo: 0xb7, hi: 0xba},
+	{value: 0x3308, lo: 0xbb, hi: 0xbf},
+	// Block 0xf9, offset 0x762
+	{value: 0x0000, lo: 0x04},
+	{value: 0x3308, lo: 0x80, hi: 0xac},
+	{value: 0x0018, lo: 0xad, hi: 0xb4},
+	{value: 0x3308, lo: 0xb5, hi: 0xb5},
+	{value: 0x0018, lo: 0xb6, hi: 0xbf},
+	// Block 0xfa, offset 0x767
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x84},
+	{value: 0x0018, lo: 0x85, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xa0},
+	{value: 0x3308, lo: 0xa1, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+	// Block 0xfb, offset 0x770
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x3308, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x87},
+	{value: 0x3308, lo: 0x88, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9a},
+	{value: 0x3308, lo: 0x9b, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xa2},
+	{value: 0x3308, lo: 0xa3, hi: 0xa4},
+	{value: 0x0040, lo: 0xa5, hi: 0xa5},
+	{value: 0x3308, lo: 0xa6, hi: 0xaa},
+	{value: 0x0040, lo: 0xab, hi: 0xbf},
+	// Block 0xfc, offset 0x77b
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x3308, lo: 0xb0, hi: 0xb6},
+	{value: 0x0008, lo: 0xb7, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0xfd, offset 0x781
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0x89},
+	{value: 0x0040, lo: 0x8a, hi: 0x8d},
+	{value: 0x0008, lo: 0x8e, hi: 0x8e},
+	{value: 0x0018, lo: 0x8f, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0xbf},
+	// Block 0xfe, offset 0x787
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0008, lo: 0x80, hi: 0xab},
+	{value: 0x3308, lo: 0xac, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbe},
+	{value: 0x0018, lo: 0xbf, hi: 0xbf},
+	// Block 0xff, offset 0x78d
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0808, lo: 0x80, hi: 0x84},
+	{value: 0x0040, lo: 0x85, hi: 0x86},
+	{value: 0x0818, lo: 0x87, hi: 0x8f},
+	{value: 0x3308, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0xbf},
+	// Block 0x100, offset 0x793
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0a08, lo: 0x80, hi: 0x83},
+	{value: 0x3308, lo: 0x84, hi: 0x8a},
+	{value: 0x0b08, lo: 0x8b, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0808, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9d},
+	{value: 0x0818, lo: 0x9e, hi: 0x9f},
+	{value: 0x0040, lo: 0xa0, hi: 0xbf},
+	// Block 0x101, offset 0x79c
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0xb0},
+	{value: 0x0818, lo: 0xb1, hi: 0xbf},
+	// Block 0x102, offset 0x79f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0818, lo: 0x80, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x103, offset 0x7a2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0818, lo: 0x81, hi: 0xbd},
+	{value: 0x0040, lo: 0xbe, hi: 0xbf},
+	// Block 0x104, offset 0x7a6
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0040, lo: 0x80, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0x105, offset 0x7aa
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbf},
+	// Block 0x106, offset 0x7ae
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xae},
+	{value: 0x0040, lo: 0xaf, hi: 0xb0},
+	{value: 0x0018, lo: 0xb1, hi: 0xbf},
+	// Block 0x107, offset 0x7b4
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0018, lo: 0x81, hi: 0x8f},
+	{value: 0x0040, lo: 0x90, hi: 0x90},
+	{value: 0x0018, lo: 0x91, hi: 0xb5},
+	{value: 0x0040, lo: 0xb6, hi: 0xbf},
+	// Block 0x108, offset 0x7ba
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x8f},
+	{value: 0xc229, lo: 0x90, hi: 0x90},
+	{value: 0x0018, lo: 0x91, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xbf},
+	// Block 0x109, offset 0x7bf
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0040, lo: 0x80, hi: 0xa5},
+	{value: 0x0018, lo: 0xa6, hi: 0xbf},
+	// Block 0x10a, offset 0x7c2
+	{value: 0x0000, lo: 0x0f},
+	{value: 0xc851, lo: 0x80, hi: 0x80},
+	{value: 0xc8a1, lo: 0x81, hi: 0x81},
+	{value: 0xc8f1, lo: 0x82, hi: 0x82},
+	{value: 0xc941, lo: 0x83, hi: 0x83},
+	{value: 0xc991, lo: 0x84, hi: 0x84},
+	{value: 0xc9e1, lo: 0x85, hi: 0x85},
+	{value: 0xca31, lo: 0x86, hi: 0x86},
+	{value: 0xca81, lo: 0x87, hi: 0x87},
+	{value: 0xcad1, lo: 0x88, hi: 0x88},
+	{value: 0x0040, lo: 0x89, hi: 0x8f},
+	{value: 0xcb21, lo: 0x90, hi: 0x90},
+	{value: 0xcb41, lo: 0x91, hi: 0x91},
+	{value: 0x0040, lo: 0x92, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xa5},
+	{value: 0x0040, lo: 0xa6, hi: 0xbf},
+	// Block 0x10b, offset 0x7d2
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x97},
+	{value: 0x0040, lo: 0x98, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xac},
+	{value: 0x0040, lo: 0xad, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xbc},
+	{value: 0x0040, lo: 0xbd, hi: 0xbf},
+	// Block 0x10c, offset 0x7d9
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0018, lo: 0x80, hi: 0xb3},
+	{value: 0x0040, lo: 0xb4, hi: 0xbf},
+	// Block 0x10d, offset 0x7dc
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x98},
+	{value: 0x0040, lo: 0x99, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xab},
+	{value: 0x0040, lo: 0xac, hi: 0xbf},
+	// Block 0x10e, offset 0x7e1
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xbf},
+	// Block 0x10f, offset 0x7e5
+	{value: 0x0000, lo: 0x05},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x99},
+	{value: 0x0040, lo: 0x9a, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xbf},
+	// Block 0x110, offset 0x7eb
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x87},
+	{value: 0x0040, lo: 0x88, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb1},
+	{value: 0x0040, lo: 0xb2, hi: 0xbf},
+	// Block 0x111, offset 0x7f2
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0xb8},
+	{value: 0x0040, lo: 0xb9, hi: 0xb9},
+	{value: 0x0018, lo: 0xba, hi: 0xbf},
+	// Block 0x112, offset 0x7f6
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x8b},
+	{value: 0x0040, lo: 0x8c, hi: 0x8c},
+	{value: 0x0018, lo: 0x8d, hi: 0xbf},
+	// Block 0x113, offset 0x7fa
+	{value: 0x0000, lo: 0x08},
+	{value: 0x0018, lo: 0x80, hi: 0x93},
+	{value: 0x0040, lo: 0x94, hi: 0x9f},
+	{value: 0x0018, lo: 0xa0, hi: 0xad},
+	{value: 0x0040, lo: 0xae, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xb7},
+	{value: 0x0018, lo: 0xb8, hi: 0xba},
+	{value: 0x0040, lo: 0xbb, hi: 0xbf},
+	// Block 0x114, offset 0x803
+	{value: 0x0000, lo: 0x06},
+	{value: 0x0018, lo: 0x80, hi: 0x86},
+	{value: 0x0040, lo: 0x87, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0xa8},
+	{value: 0x0040, lo: 0xa9, hi: 0xaf},
+	{value: 0x0018, lo: 0xb0, hi: 0xb6},
+	{value: 0x0040, lo: 0xb7, hi: 0xbf},
+	// Block 0x115, offset 0x80a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0018, lo: 0x80, hi: 0x82},
+	{value: 0x0040, lo: 0x83, hi: 0x8f},
+	{value: 0x0018, lo: 0x90, hi: 0x96},
+	{value: 0x0040, lo: 0x97, hi: 0xbf},
+	// Block 0x116, offset 0x80f
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0018, lo: 0x80, hi: 0x92},
+	{value: 0x0040, lo: 0x93, hi: 0x93},
+	{value: 0x0018, lo: 0x94, hi: 0xbf},
+	// Block 0x117, offset 0x813
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0018, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0xaf},
+	{value: 0x1f41, lo: 0xb0, hi: 0xb0},
+	{value: 0x00c9, lo: 0xb1, hi: 0xb1},
+	{value: 0x0069, lo: 0xb2, hi: 0xb2},
+	{value: 0x0079, lo: 0xb3, hi: 0xb3},
+	{value: 0x1f51, lo: 0xb4, hi: 0xb4},
+	{value: 0x1f61, lo: 0xb5, hi: 0xb5},
+	{value: 0x1f71, lo: 0xb6, hi: 0xb6},
+	{value: 0x1f81, lo: 0xb7, hi: 0xb7},
+	{value: 0x1f91, lo: 0xb8, hi: 0xb8},
+	{value: 0x1fa1, lo: 0xb9, hi: 0xb9},
+	{value: 0x0040, lo: 0xba, hi: 0xbf},
+	// Block 0x118, offset 0x821
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0xbf},
+	// Block 0x119, offset 0x824
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xb4},
+	{value: 0x0040, lo: 0xb5, hi: 0xbf},
+	// Block 0x11a, offset 0x827
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0x9d},
+	{value: 0x0040, lo: 0x9e, hi: 0x9f},
+	{value: 0x0008, lo: 0xa0, hi: 0xbf},
+	// Block 0x11b, offset 0x82b
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0008, lo: 0x80, hi: 0xa1},
+	{value: 0x0040, lo: 0xa2, hi: 0xaf},
+	{value: 0x0008, lo: 0xb0, hi: 0xbf},
+	// Block 0x11c, offset 0x82f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0xa0},
+	{value: 0x0040, lo: 0xa1, hi: 0xbf},
+	// Block 0x11d, offset 0x832
+	{value: 0x0020, lo: 0x0f},
+	{value: 0xdf21, lo: 0x80, hi: 0x89},
+	{value: 0x8e35, lo: 0x8a, hi: 0x8a},
+	{value: 0xe061, lo: 0x8b, hi: 0x9c},
+	{value: 0x8e55, lo: 0x9d, hi: 0x9d},
+	{value: 0xe2a1, lo: 0x9e, hi: 0xa2},
+	{value: 0x8e75, lo: 0xa3, hi: 0xa3},
+	{value: 0xe341, lo: 0xa4, hi: 0xab},
+	{value: 0x7f0d, lo: 0xac, hi: 0xac},
+	{value: 0xe441, lo: 0xad, hi: 0xaf},
+	{value: 0x8e95, lo: 0xb0, hi: 0xb0},
+	{value: 0xe4a1, lo: 0xb1, hi: 0xb6},
+	{value: 0x8eb5, lo: 0xb7, hi: 0xb9},
+	{value: 0xe561, lo: 0xba, hi: 0xba},
+	{value: 0x8f15, lo: 0xbb, hi: 0xbb},
+	{value: 0xe581, lo: 0xbc, hi: 0xbf},
+	// Block 0x11e, offset 0x842
+	{value: 0x0020, lo: 0x10},
+	{value: 0x93b5, lo: 0x80, hi: 0x80},
+	{value: 0xf101, lo: 0x81, hi: 0x86},
+	{value: 0x93d5, lo: 0x87, hi: 0x8a},
+	{value: 0xda61, lo: 0x8b, hi: 0x8b},
+	{value: 0xf1c1, lo: 0x8c, hi: 0x96},
+	{value: 0x9455, lo: 0x97, hi: 0x97},
+	{value: 0xf321, lo: 0x98, hi: 0xa3},
+	{value: 0x9475, lo: 0xa4, hi: 0xa6},
+	{value: 0xf4a1, lo: 0xa7, hi: 0xaa},
+	{value: 0x94d5, lo: 0xab, hi: 0xab},
+	{value: 0xf521, lo: 0xac, hi: 0xac},
+	{value: 0x94f5, lo: 0xad, hi: 0xad},
+	{value: 0xf541, lo: 0xae, hi: 0xaf},
+	{value: 0x9515, lo: 0xb0, hi: 0xb1},
+	{value: 0xf581, lo: 0xb2, hi: 0xbe},
+	{value: 0x2040, lo: 0xbf, hi: 0xbf},
+	// Block 0x11f, offset 0x853
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0008, lo: 0x80, hi: 0x8a},
+	{value: 0x0040, lo: 0x8b, hi: 0xbf},
+	// Block 0x120, offset 0x856
+	{value: 0x0000, lo: 0x04},
+	{value: 0x0040, lo: 0x80, hi: 0x80},
+	{value: 0x0340, lo: 0x81, hi: 0x81},
+	{value: 0x0040, lo: 0x82, hi: 0x9f},
+	{value: 0x0340, lo: 0xa0, hi: 0xbf},
+	// Block 0x121, offset 0x85b
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0340, lo: 0x80, hi: 0xbf},
+	// Block 0x122, offset 0x85d
+	{value: 0x0000, lo: 0x01},
+	{value: 0x33c0, lo: 0x80, hi: 0xbf},
+	// Block 0x123, offset 0x85f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x33c0, lo: 0x80, hi: 0xaf},
+	{value: 0x0040, lo: 0xb0, hi: 0xbf},
+}
+
+// Total table size 43370 bytes (42KiB); checksum: EBD909C0
diff --git a/vendor/golang.org/x/net/idna/tables9.0.0.go b/vendor/golang.org/x/net/idna/tables9.0.0.go
index 8b65fa1..4074b53 100644
--- a/vendor/golang.org/x/net/idna/tables9.0.0.go
+++ b/vendor/golang.org/x/net/idna/tables9.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build !go1.10
 // +build !go1.10
 
 package idna
diff --git a/vendor/golang.org/x/sys/unix/README.md b/vendor/golang.org/x/sys/unix/README.md
index 579d2d7..474efad 100644
--- a/vendor/golang.org/x/sys/unix/README.md
+++ b/vendor/golang.org/x/sys/unix/README.md
@@ -76,7 +76,7 @@
 ForkExec wrapper. Unlike the first two, it does not call into the scheduler to
 let it know that a system call is running.
 
-When porting Go to an new architecture/OS, this file must be implemented for
+When porting Go to a new architecture/OS, this file must be implemented for
 each GOOS/GOARCH pair.
 
 ### mksysnum
@@ -107,7 +107,7 @@
 Adding a new syscall often just requires adding a new `//sys` function prototype
 with the desired arguments and a capitalized name so it is exported. However, if
 you want the interface to the syscall to be different, often one will make an
-unexported `//sys` prototype, an then write a custom wrapper in
+unexported `//sys` prototype, and then write a custom wrapper in
 `syscall_${GOOS}.go`.
 
 ### types files
@@ -137,7 +137,7 @@
 
 This script is used to generate the system's various constants. This doesn't
 just include the error numbers and error strings, but also the signal numbers
-an a wide variety of miscellaneous constants. The constants come from the list
+and a wide variety of miscellaneous constants. The constants come from the list
 of include files in the `includes_${uname}` variable. A regex then picks out
 the desired `#define` statements, and generates the corresponding Go constants.
 The error numbers and strings are generated from `#include <errno.h>`, and the
diff --git a/vendor/golang.org/x/sys/unix/aliases.go b/vendor/golang.org/x/sys/unix/aliases.go
index 951fce4..abc89c1 100644
--- a/vendor/golang.org/x/sys/unix/aliases.go
+++ b/vendor/golang.org/x/sys/unix/aliases.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build (aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos) && go1.9
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 // +build go1.9
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/asm_aix_ppc64.s b/vendor/golang.org/x/sys/unix/asm_aix_ppc64.s
index 06f84b8..db9171c 100644
--- a/vendor/golang.org/x/sys/unix/asm_aix_ppc64.s
+++ b/vendor/golang.org/x/sys/unix/asm_aix_ppc64.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_bsd_386.s b/vendor/golang.org/x/sys/unix/asm_bsd_386.s
new file mode 100644
index 0000000..e0fcd9b
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/asm_bsd_386.s
@@ -0,0 +1,29 @@
+// Copyright 2021 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.
+
+//go:build (freebsd || netbsd || openbsd) && gc
+// +build freebsd netbsd openbsd
+// +build gc
+
+#include "textflag.h"
+
+// System call support for 386 BSD
+
+// Just jump to package syscall's implementation for all these functions.
+// The runtime may know about them.
+
+TEXT	·Syscall(SB),NOSPLIT,$0-28
+	JMP	syscall·Syscall(SB)
+
+TEXT	·Syscall6(SB),NOSPLIT,$0-40
+	JMP	syscall·Syscall6(SB)
+
+TEXT	·Syscall9(SB),NOSPLIT,$0-52
+	JMP	syscall·Syscall9(SB)
+
+TEXT	·RawSyscall(SB),NOSPLIT,$0-28
+	JMP	syscall·RawSyscall(SB)
+
+TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
+	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_bsd_amd64.s b/vendor/golang.org/x/sys/unix/asm_bsd_amd64.s
new file mode 100644
index 0000000..2b99c34
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/asm_bsd_amd64.s
@@ -0,0 +1,29 @@
+// Copyright 2021 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.
+
+//go:build (darwin || dragonfly || freebsd || netbsd || openbsd) && gc
+// +build darwin dragonfly freebsd netbsd openbsd
+// +build gc
+
+#include "textflag.h"
+
+// System call support for AMD64 BSD
+
+// Just jump to package syscall's implementation for all these functions.
+// The runtime may know about them.
+
+TEXT	·Syscall(SB),NOSPLIT,$0-56
+	JMP	syscall·Syscall(SB)
+
+TEXT	·Syscall6(SB),NOSPLIT,$0-80
+	JMP	syscall·Syscall6(SB)
+
+TEXT	·Syscall9(SB),NOSPLIT,$0-104
+	JMP	syscall·Syscall9(SB)
+
+TEXT	·RawSyscall(SB),NOSPLIT,$0-56
+	JMP	syscall·RawSyscall(SB)
+
+TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
+	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_bsd_arm.s b/vendor/golang.org/x/sys/unix/asm_bsd_arm.s
new file mode 100644
index 0000000..d702d4a
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/asm_bsd_arm.s
@@ -0,0 +1,29 @@
+// Copyright 2021 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.
+
+//go:build (freebsd || netbsd || openbsd) && gc
+// +build freebsd netbsd openbsd
+// +build gc
+
+#include "textflag.h"
+
+// System call support for ARM BSD
+
+// Just jump to package syscall's implementation for all these functions.
+// The runtime may know about them.
+
+TEXT	·Syscall(SB),NOSPLIT,$0-28
+	B	syscall·Syscall(SB)
+
+TEXT	·Syscall6(SB),NOSPLIT,$0-40
+	B	syscall·Syscall6(SB)
+
+TEXT	·Syscall9(SB),NOSPLIT,$0-52
+	B	syscall·Syscall9(SB)
+
+TEXT	·RawSyscall(SB),NOSPLIT,$0-28
+	B	syscall·RawSyscall(SB)
+
+TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
+	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_bsd_arm64.s b/vendor/golang.org/x/sys/unix/asm_bsd_arm64.s
new file mode 100644
index 0000000..fe36a73
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/asm_bsd_arm64.s
@@ -0,0 +1,29 @@
+// Copyright 2021 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.
+
+//go:build (darwin || freebsd || netbsd || openbsd) && gc
+// +build darwin freebsd netbsd openbsd
+// +build gc
+
+#include "textflag.h"
+
+// System call support for ARM64 BSD
+
+// Just jump to package syscall's implementation for all these functions.
+// The runtime may know about them.
+
+TEXT	·Syscall(SB),NOSPLIT,$0-56
+	JMP	syscall·Syscall(SB)
+
+TEXT	·Syscall6(SB),NOSPLIT,$0-80
+	JMP	syscall·Syscall6(SB)
+
+TEXT	·Syscall9(SB),NOSPLIT,$0-104
+	JMP	syscall·Syscall9(SB)
+
+TEXT	·RawSyscall(SB),NOSPLIT,$0-56
+	JMP	syscall·RawSyscall(SB)
+
+TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
+	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_darwin_386.s b/vendor/golang.org/x/sys/unix/asm_darwin_386.s
deleted file mode 100644
index 8a72783..0000000
--- a/vendor/golang.org/x/sys/unix/asm_darwin_386.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for 386, Darwin
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-28
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_darwin_amd64.s b/vendor/golang.org/x/sys/unix/asm_darwin_amd64.s
deleted file mode 100644
index 6321421..0000000
--- a/vendor/golang.org/x/sys/unix/asm_darwin_amd64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, Darwin
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_darwin_arm.s b/vendor/golang.org/x/sys/unix/asm_darwin_arm.s
deleted file mode 100644
index 333242d..0000000
--- a/vendor/golang.org/x/sys/unix/asm_darwin_arm.s
+++ /dev/null
@@ -1,30 +0,0 @@
-// 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 !gccgo
-// +build arm,darwin
-
-#include "textflag.h"
-
-//
-// System call support for ARM, Darwin
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-28
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_darwin_arm64.s b/vendor/golang.org/x/sys/unix/asm_darwin_arm64.s
deleted file mode 100644
index 97e0174..0000000
--- a/vendor/golang.org/x/sys/unix/asm_darwin_arm64.s
+++ /dev/null
@@ -1,30 +0,0 @@
-// 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 !gccgo
-// +build arm64,darwin
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, Darwin
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_dragonfly_amd64.s b/vendor/golang.org/x/sys/unix/asm_dragonfly_amd64.s
deleted file mode 100644
index 603dd57..0000000
--- a/vendor/golang.org/x/sys/unix/asm_dragonfly_amd64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, DragonFly
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_freebsd_386.s b/vendor/golang.org/x/sys/unix/asm_freebsd_386.s
deleted file mode 100644
index c9a0a26..0000000
--- a/vendor/golang.org/x/sys/unix/asm_freebsd_386.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for 386, FreeBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-28
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_freebsd_amd64.s b/vendor/golang.org/x/sys/unix/asm_freebsd_amd64.s
deleted file mode 100644
index 3517247..0000000
--- a/vendor/golang.org/x/sys/unix/asm_freebsd_amd64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, FreeBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_freebsd_arm.s b/vendor/golang.org/x/sys/unix/asm_freebsd_arm.s
deleted file mode 100644
index 9227c87..0000000
--- a/vendor/golang.org/x/sys/unix/asm_freebsd_arm.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for ARM, FreeBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-28
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_freebsd_arm64.s b/vendor/golang.org/x/sys/unix/asm_freebsd_arm64.s
deleted file mode 100644
index d9318cb..0000000
--- a/vendor/golang.org/x/sys/unix/asm_freebsd_arm64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for ARM64, FreeBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_386.s b/vendor/golang.org/x/sys/unix/asm_linux_386.s
index 448bebb..8fd101d 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_386.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_386.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_amd64.s b/vendor/golang.org/x/sys/unix/asm_linux_amd64.s
index c6468a9..7ed38e4 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_amd64.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_amd64.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_arm.s b/vendor/golang.org/x/sys/unix/asm_linux_arm.s
index cf0f357..8ef1d51 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_arm.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_arm.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_arm64.s b/vendor/golang.org/x/sys/unix/asm_linux_arm64.s
index afe6fdf..98ae027 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_arm64.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_arm64.s
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && arm64 && gc
 // +build linux
 // +build arm64
-// +build !gccgo
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_mips64x.s b/vendor/golang.org/x/sys/unix/asm_linux_mips64x.s
index ab9d638..21231d2 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_mips64x.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_mips64x.s
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (mips64 || mips64le) && gc
 // +build linux
 // +build mips64 mips64le
-// +build !gccgo
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_mipsx.s b/vendor/golang.org/x/sys/unix/asm_linux_mipsx.s
index 99e5399..6783b26 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_mipsx.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_mipsx.s
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (mips || mipsle) && gc
 // +build linux
 // +build mips mipsle
-// +build !gccgo
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_ppc64x.s b/vendor/golang.org/x/sys/unix/asm_linux_ppc64x.s
index 88f7125..19d4989 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_ppc64x.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_ppc64x.s
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (ppc64 || ppc64le) && gc
 // +build linux
 // +build ppc64 ppc64le
-// +build !gccgo
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_riscv64.s b/vendor/golang.org/x/sys/unix/asm_linux_riscv64.s
index 3cfefed..e42eb81 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_riscv64.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_riscv64.s
@@ -2,7 +2,9 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build riscv64,!gccgo
+//go:build riscv64 && gc
+// +build riscv64
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_linux_s390x.s b/vendor/golang.org/x/sys/unix/asm_linux_s390x.s
index a5a863c..c46aab3 100644
--- a/vendor/golang.org/x/sys/unix/asm_linux_s390x.s
+++ b/vendor/golang.org/x/sys/unix/asm_linux_s390x.s
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build s390x
+//go:build linux && s390x && gc
 // +build linux
-// +build !gccgo
+// +build s390x
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_netbsd_386.s b/vendor/golang.org/x/sys/unix/asm_netbsd_386.s
deleted file mode 100644
index 48bdcd7..0000000
--- a/vendor/golang.org/x/sys/unix/asm_netbsd_386.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for 386, NetBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-28
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_netbsd_amd64.s b/vendor/golang.org/x/sys/unix/asm_netbsd_amd64.s
deleted file mode 100644
index 2ede05c..0000000
--- a/vendor/golang.org/x/sys/unix/asm_netbsd_amd64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, NetBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_netbsd_arm.s b/vendor/golang.org/x/sys/unix/asm_netbsd_arm.s
deleted file mode 100644
index e892857..0000000
--- a/vendor/golang.org/x/sys/unix/asm_netbsd_arm.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for ARM, NetBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-28
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_netbsd_arm64.s b/vendor/golang.org/x/sys/unix/asm_netbsd_arm64.s
deleted file mode 100644
index 6f98ba5..0000000
--- a/vendor/golang.org/x/sys/unix/asm_netbsd_arm64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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.
-
-// +build !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for ARM64, NetBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_openbsd_386.s b/vendor/golang.org/x/sys/unix/asm_openbsd_386.s
deleted file mode 100644
index 00576f3..0000000
--- a/vendor/golang.org/x/sys/unix/asm_openbsd_386.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for 386, OpenBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	JMP	syscall·Syscall9(SB)
-
-TEXT ·RawSyscall(SB),NOSPLIT,$0-28
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_openbsd_amd64.s b/vendor/golang.org/x/sys/unix/asm_openbsd_amd64.s
deleted file mode 100644
index 790ef77..0000000
--- a/vendor/golang.org/x/sys/unix/asm_openbsd_amd64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for AMD64, OpenBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_openbsd_arm.s b/vendor/golang.org/x/sys/unix/asm_openbsd_arm.s
deleted file mode 100644
index 469bfa1..0000000
--- a/vendor/golang.org/x/sys/unix/asm_openbsd_arm.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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 !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for ARM, OpenBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-28
-	B	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-40
-	B	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-52
-	B	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-28
-	B	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-40
-	B	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_openbsd_arm64.s b/vendor/golang.org/x/sys/unix/asm_openbsd_arm64.s
deleted file mode 100644
index 0cedea3..0000000
--- a/vendor/golang.org/x/sys/unix/asm_openbsd_arm64.s
+++ /dev/null
@@ -1,29 +0,0 @@
-// 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.
-
-// +build !gccgo
-
-#include "textflag.h"
-
-//
-// System call support for arm64, OpenBSD
-//
-
-// Just jump to package syscall's implementation for all these functions.
-// The runtime may know about them.
-
-TEXT	·Syscall(SB),NOSPLIT,$0-56
-	JMP	syscall·Syscall(SB)
-
-TEXT	·Syscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·Syscall6(SB)
-
-TEXT	·Syscall9(SB),NOSPLIT,$0-104
-	JMP	syscall·Syscall9(SB)
-
-TEXT	·RawSyscall(SB),NOSPLIT,$0-56
-	JMP	syscall·RawSyscall(SB)
-
-TEXT	·RawSyscall6(SB),NOSPLIT,$0-80
-	JMP	syscall·RawSyscall6(SB)
diff --git a/vendor/golang.org/x/sys/unix/asm_openbsd_mips64.s b/vendor/golang.org/x/sys/unix/asm_openbsd_mips64.s
index 567a476..5e7a116 100644
--- a/vendor/golang.org/x/sys/unix/asm_openbsd_mips64.s
+++ b/vendor/golang.org/x/sys/unix/asm_openbsd_mips64.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_solaris_amd64.s b/vendor/golang.org/x/sys/unix/asm_solaris_amd64.s
index ded8260..f8c5394 100644
--- a/vendor/golang.org/x/sys/unix/asm_solaris_amd64.s
+++ b/vendor/golang.org/x/sys/unix/asm_solaris_amd64.s
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build !gccgo
+//go:build gc
+// +build gc
 
 #include "textflag.h"
 
diff --git a/vendor/golang.org/x/sys/unix/asm_zos_s390x.s b/vendor/golang.org/x/sys/unix/asm_zos_s390x.s
new file mode 100644
index 0000000..3b54e18
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/asm_zos_s390x.s
@@ -0,0 +1,426 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x && gc
+// +build zos
+// +build s390x
+// +build gc
+
+#include "textflag.h"
+
+#define PSALAA            1208(R0)
+#define GTAB64(x)           80(x)
+#define LCA64(x)            88(x)
+#define CAA(x)               8(x)
+#define EDCHPXV(x)        1016(x)       // in the CAA
+#define SAVSTACK_ASYNC(x)  336(x)       // in the LCA
+
+// SS_*, where x=SAVSTACK_ASYNC
+#define SS_LE(x)             0(x)
+#define SS_GO(x)             8(x)
+#define SS_ERRNO(x)         16(x)
+#define SS_ERRNOJR(x)       20(x)
+
+#define LE_CALL BYTE $0x0D; BYTE $0x76; // BL R7, R6
+
+TEXT ·clearErrno(SB),NOSPLIT,$0-0
+	BL	addrerrno<>(SB)
+	MOVD	$0, 0(R3)
+	RET
+
+// Returns the address of errno in R3.
+TEXT addrerrno<>(SB),NOSPLIT|NOFRAME,$0-0
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get __errno FuncDesc.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	ADD	$(0x156*16), R9
+	LMG	0(R9), R5, R6
+
+	// Switch to saved LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Call __errno function.
+	LE_CALL
+	NOPH
+
+	// Switch back to Go stack.
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+	RET
+
+TEXT ·syscall_syscall(SB),NOSPLIT,$0-56
+	BL	runtime·entersyscall(SB)
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+32(FP)
+	MOVD	R0, r2+40(FP)
+	MOVD	R0, err+48(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	addrerrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+48(FP)
+done:
+	BL	runtime·exitsyscall(SB)
+	RET
+
+TEXT ·syscall_rawsyscall(SB),NOSPLIT,$0-56
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+32(FP)
+	MOVD	R0, r2+40(FP)
+	MOVD	R0, err+48(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	addrerrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+48(FP)
+done:
+	RET
+
+TEXT ·syscall_syscall6(SB),NOSPLIT,$0-80
+	BL	runtime·entersyscall(SB)
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Fill in parameter list.
+	MOVD	a4+32(FP), R12
+	MOVD	R12, (2176+24)(R4)
+	MOVD	a5+40(FP), R12
+	MOVD	R12, (2176+32)(R4)
+	MOVD	a6+48(FP), R12
+	MOVD	R12, (2176+40)(R4)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+56(FP)
+	MOVD	R0, r2+64(FP)
+	MOVD	R0, err+72(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	addrerrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+72(FP)
+done:
+	BL	runtime·exitsyscall(SB)
+	RET
+
+TEXT ·syscall_rawsyscall6(SB),NOSPLIT,$0-80
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Fill in parameter list.
+	MOVD	a4+32(FP), R12
+	MOVD	R12, (2176+24)(R4)
+	MOVD	a5+40(FP), R12
+	MOVD	R12, (2176+32)(R4)
+	MOVD	a6+48(FP), R12
+	MOVD	R12, (2176+40)(R4)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+56(FP)
+	MOVD	R0, r2+64(FP)
+	MOVD	R0, err+72(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	·rrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+72(FP)
+done:
+	RET
+
+TEXT ·syscall_syscall9(SB),NOSPLIT,$0
+	BL	runtime·entersyscall(SB)
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Fill in parameter list.
+	MOVD	a4+32(FP), R12
+	MOVD	R12, (2176+24)(R4)
+	MOVD	a5+40(FP), R12
+	MOVD	R12, (2176+32)(R4)
+	MOVD	a6+48(FP), R12
+	MOVD	R12, (2176+40)(R4)
+	MOVD	a7+56(FP), R12
+	MOVD	R12, (2176+48)(R4)
+	MOVD	a8+64(FP), R12
+	MOVD	R12, (2176+56)(R4)
+	MOVD	a9+72(FP), R12
+	MOVD	R12, (2176+64)(R4)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+80(FP)
+	MOVD	R0, r2+88(FP)
+	MOVD	R0, err+96(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	addrerrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+96(FP)
+done:
+        BL	runtime·exitsyscall(SB)
+        RET
+
+TEXT ·syscall_rawsyscall9(SB),NOSPLIT,$0
+	MOVD	a1+8(FP), R1
+	MOVD	a2+16(FP), R2
+	MOVD	a3+24(FP), R3
+
+	// Get library control area (LCA).
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+
+	// Get function.
+	MOVD	CAA(R8), R9
+	MOVD	EDCHPXV(R9), R9
+	MOVD	trap+0(FP), R5
+	SLD	$4, R5
+	ADD	R5, R9
+	LMG	0(R9), R5, R6
+
+	// Restore LE stack.
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R4
+	MOVD	$0, 0(R9)
+
+	// Fill in parameter list.
+	MOVD	a4+32(FP), R12
+	MOVD	R12, (2176+24)(R4)
+	MOVD	a5+40(FP), R12
+	MOVD	R12, (2176+32)(R4)
+	MOVD	a6+48(FP), R12
+	MOVD	R12, (2176+40)(R4)
+	MOVD	a7+56(FP), R12
+	MOVD	R12, (2176+48)(R4)
+	MOVD	a8+64(FP), R12
+	MOVD	R12, (2176+56)(R4)
+	MOVD	a9+72(FP), R12
+	MOVD	R12, (2176+64)(R4)
+
+	// Call function.
+	LE_CALL
+	NOPH
+	XOR	R0, R0      // Restore R0 to $0.
+	MOVD	R4, 0(R9)   // Save stack pointer.
+
+	MOVD	R3, r1+80(FP)
+	MOVD	R0, r2+88(FP)
+	MOVD	R0, err+96(FP)
+	MOVW	R3, R4
+	CMP	R4, $-1
+	BNE	done
+	BL	addrerrno<>(SB)
+	MOVWZ	0(R3), R3
+	MOVD	R3, err+96(FP)
+done:
+	RET
+
+// func svcCall(fnptr unsafe.Pointer, argv *unsafe.Pointer, dsa *uint64)
+TEXT ·svcCall(SB),NOSPLIT,$0
+	BL	runtime·save_g(SB)   // Save g and stack pointer
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	R15, 0(R9)
+
+	MOVD	argv+8(FP), R1       // Move function arguments into registers
+	MOVD	dsa+16(FP), g
+	MOVD	fnptr+0(FP), R15
+
+	BYTE	$0x0D                // Branch to function
+	BYTE	$0xEF
+
+	BL	runtime·load_g(SB)   // Restore g and stack pointer
+	MOVW	PSALAA, R8
+	MOVD	LCA64(R8), R8
+	MOVD	SAVSTACK_ASYNC(R8), R9
+	MOVD	0(R9), R15
+
+	RET
+
+// func svcLoad(name *byte) unsafe.Pointer
+TEXT ·svcLoad(SB),NOSPLIT,$0
+	MOVD	R15, R2          // Save go stack pointer
+	MOVD	name+0(FP), R0   // Move SVC args into registers
+	MOVD	$0x80000000, R1
+	MOVD	$0, R15
+	BYTE	$0x0A            // SVC 08 LOAD
+	BYTE	$0x08
+	MOVW	R15, R3          // Save return code from SVC
+	MOVD	R2, R15          // Restore go stack pointer
+	CMP	R3, $0           // Check SVC return code
+	BNE	error
+
+	MOVD	$-2, R3          // Reset last bit of entry point to zero
+	AND	R0, R3
+	MOVD	R3, addr+8(FP)   // Return entry point returned by SVC
+	CMP	R0, R3           // Check if last bit of entry point was set
+	BNE	done
+
+	MOVD	R15, R2          // Save go stack pointer
+	MOVD	$0, R15          // Move SVC args into registers (entry point still in r0 from SVC 08)
+	BYTE	$0x0A            // SVC 09 DELETE
+	BYTE	$0x09
+	MOVD	R2, R15          // Restore go stack pointer
+
+error:
+	MOVD	$0, addr+8(FP)   // Return 0 on failure
+done:
+	XOR	R0, R0           // Reset r0 to 0
+	RET
+
+// func svcUnload(name *byte, fnptr unsafe.Pointer) int64
+TEXT ·svcUnload(SB),NOSPLIT,$0
+	MOVD	R15, R2          // Save go stack pointer
+	MOVD	name+0(FP), R0   // Move SVC args into registers
+	MOVD	addr+8(FP), R15
+	BYTE	$0x0A            // SVC 09
+	BYTE	$0x09
+	XOR	R0, R0           // Reset r0 to 0
+	MOVD	R15, R1          // Save SVC return code
+	MOVD	R2, R15          // Restore go stack pointer
+	MOVD	R1, rc+0(FP)     // Return SVC return code
+	RET
+
+// func gettid() uint64
+TEXT ·gettid(SB), NOSPLIT, $0
+	// Get library control area (LCA).
+	MOVW PSALAA, R8
+	MOVD LCA64(R8), R8
+
+	// Get CEECAATHDID
+	MOVD CAA(R8), R9
+	MOVD 0x3D0(R9), R9
+	MOVD R9, ret+0(FP)
+
+	RET
diff --git a/vendor/golang.org/x/sys/unix/cap_freebsd.go b/vendor/golang.org/x/sys/unix/cap_freebsd.go
index df52048..0b7c6ad 100644
--- a/vendor/golang.org/x/sys/unix/cap_freebsd.go
+++ b/vendor/golang.org/x/sys/unix/cap_freebsd.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build freebsd
 // +build freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/constants.go b/vendor/golang.org/x/sys/unix/constants.go
index 3a6ac64..394a396 100644
--- a/vendor/golang.org/x/sys/unix/constants.go
+++ b/vendor/golang.org/x/sys/unix/constants.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/dev_aix_ppc.go b/vendor/golang.org/x/sys/unix/dev_aix_ppc.go
index 5e5fb45..65a9985 100644
--- a/vendor/golang.org/x/sys/unix/dev_aix_ppc.go
+++ b/vendor/golang.org/x/sys/unix/dev_aix_ppc.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix
-// +build ppc
+//go:build aix && ppc
+// +build aix,ppc
 
 // Functions to access/create device major and minor numbers matching the
 // encoding used by AIX.
diff --git a/vendor/golang.org/x/sys/unix/dev_aix_ppc64.go b/vendor/golang.org/x/sys/unix/dev_aix_ppc64.go
index 8b40124..8fc08ad 100644
--- a/vendor/golang.org/x/sys/unix/dev_aix_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/dev_aix_ppc64.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix
-// +build ppc64
+//go:build aix && ppc64
+// +build aix,ppc64
 
 // Functions to access/create device major and minor numbers matching the
 // encoding used AIX.
diff --git a/vendor/golang.org/x/sys/unix/dev_zos.go b/vendor/golang.org/x/sys/unix/dev_zos.go
new file mode 100644
index 0000000..a388e59
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/dev_zos.go
@@ -0,0 +1,29 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+// Functions to access/create device major and minor numbers matching the
+// encoding used by z/OS.
+//
+// The information below is extracted and adapted from <sys/stat.h> macros.
+
+package unix
+
+// Major returns the major component of a z/OS device number.
+func Major(dev uint64) uint32 {
+	return uint32((dev >> 16) & 0x0000FFFF)
+}
+
+// Minor returns the minor component of a z/OS device number.
+func Minor(dev uint64) uint32 {
+	return uint32(dev & 0x0000FFFF)
+}
+
+// Mkdev returns a z/OS device number generated from the given major and minor
+// components.
+func Mkdev(major, minor uint32) uint64 {
+	return (uint64(major) << 16) | uint64(minor)
+}
diff --git a/vendor/golang.org/x/sys/unix/dirent.go b/vendor/golang.org/x/sys/unix/dirent.go
index 304016b..e74e5ea 100644
--- a/vendor/golang.org/x/sys/unix/dirent.go
+++ b/vendor/golang.org/x/sys/unix/dirent.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris
 // +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/endian_big.go b/vendor/golang.org/x/sys/unix/endian_big.go
index 5e92690..a520265 100644
--- a/vendor/golang.org/x/sys/unix/endian_big.go
+++ b/vendor/golang.org/x/sys/unix/endian_big.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 //
-// +build ppc64 s390x mips mips64
+//go:build armbe || arm64be || m68k || mips || mips64 || mips64p32 || ppc || ppc64 || s390 || s390x || shbe || sparc || sparc64
+// +build armbe arm64be m68k mips mips64 mips64p32 ppc ppc64 s390 s390x shbe sparc sparc64
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/endian_little.go b/vendor/golang.org/x/sys/unix/endian_little.go
index bcdb5d3..4362f47 100644
--- a/vendor/golang.org/x/sys/unix/endian_little.go
+++ b/vendor/golang.org/x/sys/unix/endian_little.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 //
-// +build 386 amd64 amd64p32 arm arm64 ppc64le mipsle mips64le riscv64
+//go:build 386 || amd64 || amd64p32 || alpha || arm || arm64 || mipsle || mips64le || mips64p32le || nios2 || ppc64le || riscv || riscv64 || sh
+// +build 386 amd64 amd64p32 alpha arm arm64 mipsle mips64le mips64p32le nios2 ppc64le riscv riscv64 sh
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/env_unix.go b/vendor/golang.org/x/sys/unix/env_unix.go
index 84178b0..29ccc4d 100644
--- a/vendor/golang.org/x/sys/unix/env_unix.go
+++ b/vendor/golang.org/x/sys/unix/env_unix.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 // Unix environment variables.
 
diff --git a/vendor/golang.org/x/sys/unix/epoll_zos.go b/vendor/golang.org/x/sys/unix/epoll_zos.go
new file mode 100644
index 0000000..cedaf7e
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/epoll_zos.go
@@ -0,0 +1,221 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+import (
+	"sync"
+)
+
+// This file simulates epoll on z/OS using poll.
+
+// Analogous to epoll_event on Linux.
+// TODO(neeilan): Pad is because the Linux kernel expects a 96-bit struct. We never pass this to the kernel; remove?
+type EpollEvent struct {
+	Events uint32
+	Fd     int32
+	Pad    int32
+}
+
+const (
+	EPOLLERR      = 0x8
+	EPOLLHUP      = 0x10
+	EPOLLIN       = 0x1
+	EPOLLMSG      = 0x400
+	EPOLLOUT      = 0x4
+	EPOLLPRI      = 0x2
+	EPOLLRDBAND   = 0x80
+	EPOLLRDNORM   = 0x40
+	EPOLLWRBAND   = 0x200
+	EPOLLWRNORM   = 0x100
+	EPOLL_CTL_ADD = 0x1
+	EPOLL_CTL_DEL = 0x2
+	EPOLL_CTL_MOD = 0x3
+	// The following constants are part of the epoll API, but represent
+	// currently unsupported functionality on z/OS.
+	// EPOLL_CLOEXEC  = 0x80000
+	// EPOLLET        = 0x80000000
+	// EPOLLONESHOT   = 0x40000000
+	// EPOLLRDHUP     = 0x2000     // Typically used with edge-triggered notis
+	// EPOLLEXCLUSIVE = 0x10000000 // Exclusive wake-up mode
+	// EPOLLWAKEUP    = 0x20000000 // Relies on Linux's BLOCK_SUSPEND capability
+)
+
+// TODO(neeilan): We can eliminate these epToPoll / pToEpoll calls by using identical mask values for POLL/EPOLL
+// constants where possible The lower 16 bits of epoll events (uint32) can fit any system poll event (int16).
+
+// epToPollEvt converts epoll event field to poll equivalent.
+// In epoll, Events is a 32-bit field, while poll uses 16 bits.
+func epToPollEvt(events uint32) int16 {
+	var ep2p = map[uint32]int16{
+		EPOLLIN:  POLLIN,
+		EPOLLOUT: POLLOUT,
+		EPOLLHUP: POLLHUP,
+		EPOLLPRI: POLLPRI,
+		EPOLLERR: POLLERR,
+	}
+
+	var pollEvts int16 = 0
+	for epEvt, pEvt := range ep2p {
+		if (events & epEvt) != 0 {
+			pollEvts |= pEvt
+		}
+	}
+
+	return pollEvts
+}
+
+// pToEpollEvt converts 16 bit poll event bitfields to 32-bit epoll event fields.
+func pToEpollEvt(revents int16) uint32 {
+	var p2ep = map[int16]uint32{
+		POLLIN:  EPOLLIN,
+		POLLOUT: EPOLLOUT,
+		POLLHUP: EPOLLHUP,
+		POLLPRI: EPOLLPRI,
+		POLLERR: EPOLLERR,
+	}
+
+	var epollEvts uint32 = 0
+	for pEvt, epEvt := range p2ep {
+		if (revents & pEvt) != 0 {
+			epollEvts |= epEvt
+		}
+	}
+
+	return epollEvts
+}
+
+// Per-process epoll implementation.
+type epollImpl struct {
+	mu       sync.Mutex
+	epfd2ep  map[int]*eventPoll
+	nextEpfd int
+}
+
+// eventPoll holds a set of file descriptors being watched by the process. A process can have multiple epoll instances.
+// On Linux, this is an in-kernel data structure accessed through a fd.
+type eventPoll struct {
+	mu  sync.Mutex
+	fds map[int]*EpollEvent
+}
+
+// epoll impl for this process.
+var impl epollImpl = epollImpl{
+	epfd2ep:  make(map[int]*eventPoll),
+	nextEpfd: 0,
+}
+
+func (e *epollImpl) epollcreate(size int) (epfd int, err error) {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+	epfd = e.nextEpfd
+	e.nextEpfd++
+
+	e.epfd2ep[epfd] = &eventPoll{
+		fds: make(map[int]*EpollEvent),
+	}
+	return epfd, nil
+}
+
+func (e *epollImpl) epollcreate1(flag int) (fd int, err error) {
+	return e.epollcreate(4)
+}
+
+func (e *epollImpl) epollctl(epfd int, op int, fd int, event *EpollEvent) (err error) {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+
+	ep, ok := e.epfd2ep[epfd]
+	if !ok {
+
+		return EBADF
+	}
+
+	switch op {
+	case EPOLL_CTL_ADD:
+		// TODO(neeilan): When we make epfds and fds disjoint, detect epoll
+		// loops here (instances watching each other) and return ELOOP.
+		if _, ok := ep.fds[fd]; ok {
+			return EEXIST
+		}
+		ep.fds[fd] = event
+	case EPOLL_CTL_MOD:
+		if _, ok := ep.fds[fd]; !ok {
+			return ENOENT
+		}
+		ep.fds[fd] = event
+	case EPOLL_CTL_DEL:
+		if _, ok := ep.fds[fd]; !ok {
+			return ENOENT
+		}
+		delete(ep.fds, fd)
+
+	}
+	return nil
+}
+
+// Must be called while holding ep.mu
+func (ep *eventPoll) getFds() []int {
+	fds := make([]int, len(ep.fds))
+	for fd := range ep.fds {
+		fds = append(fds, fd)
+	}
+	return fds
+}
+
+func (e *epollImpl) epollwait(epfd int, events []EpollEvent, msec int) (n int, err error) {
+	e.mu.Lock() // in [rare] case of concurrent epollcreate + epollwait
+	ep, ok := e.epfd2ep[epfd]
+
+	if !ok {
+		e.mu.Unlock()
+		return 0, EBADF
+	}
+
+	pollfds := make([]PollFd, 4)
+	for fd, epollevt := range ep.fds {
+		pollfds = append(pollfds, PollFd{Fd: int32(fd), Events: epToPollEvt(epollevt.Events)})
+	}
+	e.mu.Unlock()
+
+	n, err = Poll(pollfds, msec)
+	if err != nil {
+		return n, err
+	}
+
+	i := 0
+	for _, pFd := range pollfds {
+		if pFd.Revents != 0 {
+			events[i] = EpollEvent{Fd: pFd.Fd, Events: pToEpollEvt(pFd.Revents)}
+			i++
+		}
+
+		if i == n {
+			break
+		}
+	}
+
+	return n, nil
+}
+
+func EpollCreate(size int) (fd int, err error) {
+	return impl.epollcreate(size)
+}
+
+func EpollCreate1(flag int) (fd int, err error) {
+	return impl.epollcreate1(flag)
+}
+
+func EpollCtl(epfd int, op int, fd int, event *EpollEvent) (err error) {
+	return impl.epollctl(epfd, op, fd, event)
+}
+
+// Because EpollWait mutates events, the caller is expected to coordinate
+// concurrent access if calling with the same epfd from multiple goroutines.
+func EpollWait(epfd int, events []EpollEvent, msec int) (n int, err error) {
+	return impl.epollwait(epfd, events, msec)
+}
diff --git a/vendor/golang.org/x/sys/unix/fcntl.go b/vendor/golang.org/x/sys/unix/fcntl.go
index 4dc5348..e9b9912 100644
--- a/vendor/golang.org/x/sys/unix/fcntl.go
+++ b/vendor/golang.org/x/sys/unix/fcntl.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build dragonfly || freebsd || linux || netbsd || openbsd
 // +build dragonfly freebsd linux netbsd openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/fcntl_linux_32bit.go b/vendor/golang.org/x/sys/unix/fcntl_linux_32bit.go
index fc0e50e..29d4480 100644
--- a/vendor/golang.org/x/sys/unix/fcntl_linux_32bit.go
+++ b/vendor/golang.org/x/sys/unix/fcntl_linux_32bit.go
@@ -1,9 +1,10 @@
-// +build linux,386 linux,arm linux,mips linux,mipsle
-
 // Copyright 2014 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.
 
+//go:build (linux && 386) || (linux && arm) || (linux && mips) || (linux && mipsle) || (linux && ppc)
+// +build linux,386 linux,arm linux,mips linux,mipsle linux,ppc
+
 package unix
 
 func init() {
diff --git a/vendor/golang.org/x/sys/unix/fdset.go b/vendor/golang.org/x/sys/unix/fdset.go
index b27be0a..a8068f9 100644
--- a/vendor/golang.org/x/sys/unix/fdset.go
+++ b/vendor/golang.org/x/sys/unix/fdset.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/fstatfs_zos.go b/vendor/golang.org/x/sys/unix/fstatfs_zos.go
new file mode 100644
index 0000000..e377cc9
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/fstatfs_zos.go
@@ -0,0 +1,164 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+import (
+	"unsafe"
+)
+
+// This file simulates fstatfs on z/OS using fstatvfs and w_getmntent.
+
+func Fstatfs(fd int, stat *Statfs_t) (err error) {
+	var stat_v Statvfs_t
+	err = Fstatvfs(fd, &stat_v)
+	if err == nil {
+		// populate stat
+		stat.Type = 0
+		stat.Bsize = stat_v.Bsize
+		stat.Blocks = stat_v.Blocks
+		stat.Bfree = stat_v.Bfree
+		stat.Bavail = stat_v.Bavail
+		stat.Files = stat_v.Files
+		stat.Ffree = stat_v.Ffree
+		stat.Fsid = stat_v.Fsid
+		stat.Namelen = stat_v.Namemax
+		stat.Frsize = stat_v.Frsize
+		stat.Flags = stat_v.Flag
+		for passn := 0; passn < 5; passn++ {
+			switch passn {
+			case 0:
+				err = tryGetmntent64(stat)
+				break
+			case 1:
+				err = tryGetmntent128(stat)
+				break
+			case 2:
+				err = tryGetmntent256(stat)
+				break
+			case 3:
+				err = tryGetmntent512(stat)
+				break
+			case 4:
+				err = tryGetmntent1024(stat)
+				break
+			default:
+				break
+			}
+			//proceed to return if: err is nil (found), err is nonnil but not ERANGE (another error occurred)
+			if err == nil || err != nil && err != ERANGE {
+				break
+			}
+		}
+	}
+	return err
+}
+
+func tryGetmntent64(stat *Statfs_t) (err error) {
+	var mnt_ent_buffer struct {
+		header       W_Mnth
+		filesys_info [64]W_Mntent
+	}
+	var buffer_size int = int(unsafe.Sizeof(mnt_ent_buffer))
+	fs_count, err := W_Getmntent((*byte)(unsafe.Pointer(&mnt_ent_buffer)), buffer_size)
+	if err != nil {
+		return err
+	}
+	err = ERANGE //return ERANGE if no match is found in this batch
+	for i := 0; i < fs_count; i++ {
+		if stat.Fsid == uint64(mnt_ent_buffer.filesys_info[i].Dev) {
+			stat.Type = uint32(mnt_ent_buffer.filesys_info[i].Fstname[0])
+			err = nil
+			break
+		}
+	}
+	return err
+}
+
+func tryGetmntent128(stat *Statfs_t) (err error) {
+	var mnt_ent_buffer struct {
+		header       W_Mnth
+		filesys_info [128]W_Mntent
+	}
+	var buffer_size int = int(unsafe.Sizeof(mnt_ent_buffer))
+	fs_count, err := W_Getmntent((*byte)(unsafe.Pointer(&mnt_ent_buffer)), buffer_size)
+	if err != nil {
+		return err
+	}
+	err = ERANGE //return ERANGE if no match is found in this batch
+	for i := 0; i < fs_count; i++ {
+		if stat.Fsid == uint64(mnt_ent_buffer.filesys_info[i].Dev) {
+			stat.Type = uint32(mnt_ent_buffer.filesys_info[i].Fstname[0])
+			err = nil
+			break
+		}
+	}
+	return err
+}
+
+func tryGetmntent256(stat *Statfs_t) (err error) {
+	var mnt_ent_buffer struct {
+		header       W_Mnth
+		filesys_info [256]W_Mntent
+	}
+	var buffer_size int = int(unsafe.Sizeof(mnt_ent_buffer))
+	fs_count, err := W_Getmntent((*byte)(unsafe.Pointer(&mnt_ent_buffer)), buffer_size)
+	if err != nil {
+		return err
+	}
+	err = ERANGE //return ERANGE if no match is found in this batch
+	for i := 0; i < fs_count; i++ {
+		if stat.Fsid == uint64(mnt_ent_buffer.filesys_info[i].Dev) {
+			stat.Type = uint32(mnt_ent_buffer.filesys_info[i].Fstname[0])
+			err = nil
+			break
+		}
+	}
+	return err
+}
+
+func tryGetmntent512(stat *Statfs_t) (err error) {
+	var mnt_ent_buffer struct {
+		header       W_Mnth
+		filesys_info [512]W_Mntent
+	}
+	var buffer_size int = int(unsafe.Sizeof(mnt_ent_buffer))
+	fs_count, err := W_Getmntent((*byte)(unsafe.Pointer(&mnt_ent_buffer)), buffer_size)
+	if err != nil {
+		return err
+	}
+	err = ERANGE //return ERANGE if no match is found in this batch
+	for i := 0; i < fs_count; i++ {
+		if stat.Fsid == uint64(mnt_ent_buffer.filesys_info[i].Dev) {
+			stat.Type = uint32(mnt_ent_buffer.filesys_info[i].Fstname[0])
+			err = nil
+			break
+		}
+	}
+	return err
+}
+
+func tryGetmntent1024(stat *Statfs_t) (err error) {
+	var mnt_ent_buffer struct {
+		header       W_Mnth
+		filesys_info [1024]W_Mntent
+	}
+	var buffer_size int = int(unsafe.Sizeof(mnt_ent_buffer))
+	fs_count, err := W_Getmntent((*byte)(unsafe.Pointer(&mnt_ent_buffer)), buffer_size)
+	if err != nil {
+		return err
+	}
+	err = ERANGE //return ERANGE if no match is found in this batch
+	for i := 0; i < fs_count; i++ {
+		if stat.Fsid == uint64(mnt_ent_buffer.filesys_info[i].Dev) {
+			stat.Type = uint32(mnt_ent_buffer.filesys_info[i].Fstname[0])
+			err = nil
+			break
+		}
+	}
+	return err
+}
diff --git a/vendor/golang.org/x/sys/unix/gccgo.go b/vendor/golang.org/x/sys/unix/gccgo.go
index 86032c1..0dee232 100644
--- a/vendor/golang.org/x/sys/unix/gccgo.go
+++ b/vendor/golang.org/x/sys/unix/gccgo.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build gccgo
-// +build !aix
+//go:build gccgo && !aix
+// +build gccgo,!aix
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/gccgo_linux_amd64.go b/vendor/golang.org/x/sys/unix/gccgo_linux_amd64.go
index 251a977..e60e49a 100644
--- a/vendor/golang.org/x/sys/unix/gccgo_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/gccgo_linux_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build gccgo && linux && amd64
 // +build gccgo,linux,amd64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/ioctl.go b/vendor/golang.org/x/sys/unix/ioctl.go
index 5641678..6c7ad05 100644
--- a/vendor/golang.org/x/sys/unix/ioctl.go
+++ b/vendor/golang.org/x/sys/unix/ioctl.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris
 // +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/ioctl_linux.go b/vendor/golang.org/x/sys/unix/ioctl_linux.go
new file mode 100644
index 0000000..48773f7
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ioctl_linux.go
@@ -0,0 +1,196 @@
+// Copyright 2021 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 unix
+
+import (
+	"runtime"
+	"unsafe"
+)
+
+// IoctlRetInt performs an ioctl operation specified by req on a device
+// associated with opened file descriptor fd, and returns a non-negative
+// integer that is returned by the ioctl syscall.
+func IoctlRetInt(fd int, req uint) (int, error) {
+	ret, _, err := Syscall(SYS_IOCTL, uintptr(fd), uintptr(req), 0)
+	if err != 0 {
+		return 0, err
+	}
+	return int(ret), nil
+}
+
+func IoctlGetUint32(fd int, req uint) (uint32, error) {
+	var value uint32
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(&value)))
+	return value, err
+}
+
+func IoctlGetRTCTime(fd int) (*RTCTime, error) {
+	var value RTCTime
+	err := ioctl(fd, RTC_RD_TIME, uintptr(unsafe.Pointer(&value)))
+	return &value, err
+}
+
+func IoctlSetRTCTime(fd int, value *RTCTime) error {
+	err := ioctl(fd, RTC_SET_TIME, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
+}
+
+func IoctlGetRTCWkAlrm(fd int) (*RTCWkAlrm, error) {
+	var value RTCWkAlrm
+	err := ioctl(fd, RTC_WKALM_RD, uintptr(unsafe.Pointer(&value)))
+	return &value, err
+}
+
+func IoctlSetRTCWkAlrm(fd int, value *RTCWkAlrm) error {
+	err := ioctl(fd, RTC_WKALM_SET, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
+}
+
+type ifreqEthtool struct {
+	name [IFNAMSIZ]byte
+	data unsafe.Pointer
+}
+
+// IoctlGetEthtoolDrvinfo fetches ethtool driver information for the network
+// device specified by ifname.
+func IoctlGetEthtoolDrvinfo(fd int, ifname string) (*EthtoolDrvinfo, error) {
+	// Leave room for terminating NULL byte.
+	if len(ifname) >= IFNAMSIZ {
+		return nil, EINVAL
+	}
+
+	value := EthtoolDrvinfo{
+		Cmd: ETHTOOL_GDRVINFO,
+	}
+	ifreq := ifreqEthtool{
+		data: unsafe.Pointer(&value),
+	}
+	copy(ifreq.name[:], ifname)
+	err := ioctl(fd, SIOCETHTOOL, uintptr(unsafe.Pointer(&ifreq)))
+	runtime.KeepAlive(ifreq)
+	return &value, err
+}
+
+// IoctlGetWatchdogInfo fetches information about a watchdog device from the
+// Linux watchdog API. For more information, see:
+// https://www.kernel.org/doc/html/latest/watchdog/watchdog-api.html.
+func IoctlGetWatchdogInfo(fd int) (*WatchdogInfo, error) {
+	var value WatchdogInfo
+	err := ioctl(fd, WDIOC_GETSUPPORT, uintptr(unsafe.Pointer(&value)))
+	return &value, err
+}
+
+// IoctlWatchdogKeepalive issues a keepalive ioctl to a watchdog device. For
+// more information, see:
+// https://www.kernel.org/doc/html/latest/watchdog/watchdog-api.html.
+func IoctlWatchdogKeepalive(fd int) error {
+	return ioctl(fd, WDIOC_KEEPALIVE, 0)
+}
+
+// IoctlFileCloneRange performs an FICLONERANGE ioctl operation to clone the
+// range of data conveyed in value to the file associated with the file
+// descriptor destFd. See the ioctl_ficlonerange(2) man page for details.
+func IoctlFileCloneRange(destFd int, value *FileCloneRange) error {
+	err := ioctl(destFd, FICLONERANGE, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
+}
+
+// IoctlFileClone performs an FICLONE ioctl operation to clone the entire file
+// associated with the file description srcFd to the file associated with the
+// file descriptor destFd. See the ioctl_ficlone(2) man page for details.
+func IoctlFileClone(destFd, srcFd int) error {
+	return ioctl(destFd, FICLONE, uintptr(srcFd))
+}
+
+type FileDedupeRange struct {
+	Src_offset uint64
+	Src_length uint64
+	Reserved1  uint16
+	Reserved2  uint32
+	Info       []FileDedupeRangeInfo
+}
+
+type FileDedupeRangeInfo struct {
+	Dest_fd       int64
+	Dest_offset   uint64
+	Bytes_deduped uint64
+	Status        int32
+	Reserved      uint32
+}
+
+// IoctlFileDedupeRange performs an FIDEDUPERANGE ioctl operation to share the
+// range of data conveyed in value from the file associated with the file
+// descriptor srcFd to the value.Info destinations. See the
+// ioctl_fideduperange(2) man page for details.
+func IoctlFileDedupeRange(srcFd int, value *FileDedupeRange) error {
+	buf := make([]byte, SizeofRawFileDedupeRange+
+		len(value.Info)*SizeofRawFileDedupeRangeInfo)
+	rawrange := (*RawFileDedupeRange)(unsafe.Pointer(&buf[0]))
+	rawrange.Src_offset = value.Src_offset
+	rawrange.Src_length = value.Src_length
+	rawrange.Dest_count = uint16(len(value.Info))
+	rawrange.Reserved1 = value.Reserved1
+	rawrange.Reserved2 = value.Reserved2
+
+	for i := range value.Info {
+		rawinfo := (*RawFileDedupeRangeInfo)(unsafe.Pointer(
+			uintptr(unsafe.Pointer(&buf[0])) + uintptr(SizeofRawFileDedupeRange) +
+				uintptr(i*SizeofRawFileDedupeRangeInfo)))
+		rawinfo.Dest_fd = value.Info[i].Dest_fd
+		rawinfo.Dest_offset = value.Info[i].Dest_offset
+		rawinfo.Bytes_deduped = value.Info[i].Bytes_deduped
+		rawinfo.Status = value.Info[i].Status
+		rawinfo.Reserved = value.Info[i].Reserved
+	}
+
+	err := ioctl(srcFd, FIDEDUPERANGE, uintptr(unsafe.Pointer(&buf[0])))
+
+	// Output
+	for i := range value.Info {
+		rawinfo := (*RawFileDedupeRangeInfo)(unsafe.Pointer(
+			uintptr(unsafe.Pointer(&buf[0])) + uintptr(SizeofRawFileDedupeRange) +
+				uintptr(i*SizeofRawFileDedupeRangeInfo)))
+		value.Info[i].Dest_fd = rawinfo.Dest_fd
+		value.Info[i].Dest_offset = rawinfo.Dest_offset
+		value.Info[i].Bytes_deduped = rawinfo.Bytes_deduped
+		value.Info[i].Status = rawinfo.Status
+		value.Info[i].Reserved = rawinfo.Reserved
+	}
+
+	return err
+}
+
+func IoctlHIDGetDesc(fd int, value *HIDRawReportDescriptor) error {
+	err := ioctl(fd, HIDIOCGRDESC, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
+}
+
+func IoctlHIDGetRawInfo(fd int) (*HIDRawDevInfo, error) {
+	var value HIDRawDevInfo
+	err := ioctl(fd, HIDIOCGRAWINFO, uintptr(unsafe.Pointer(&value)))
+	return &value, err
+}
+
+func IoctlHIDGetRawName(fd int) (string, error) {
+	var value [_HIDIOCGRAWNAME_LEN]byte
+	err := ioctl(fd, _HIDIOCGRAWNAME, uintptr(unsafe.Pointer(&value[0])))
+	return ByteSliceToString(value[:]), err
+}
+
+func IoctlHIDGetRawPhys(fd int) (string, error) {
+	var value [_HIDIOCGRAWPHYS_LEN]byte
+	err := ioctl(fd, _HIDIOCGRAWPHYS, uintptr(unsafe.Pointer(&value[0])))
+	return ByteSliceToString(value[:]), err
+}
+
+func IoctlHIDGetRawUniq(fd int) (string, error) {
+	var value [_HIDIOCGRAWUNIQ_LEN]byte
+	err := ioctl(fd, _HIDIOCGRAWUNIQ, uintptr(unsafe.Pointer(&value[0])))
+	return ByteSliceToString(value[:]), err
+}
diff --git a/vendor/golang.org/x/sys/unix/ioctl_zos.go b/vendor/golang.org/x/sys/unix/ioctl_zos.go
new file mode 100644
index 0000000..5384e7d
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ioctl_zos.go
@@ -0,0 +1,74 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+import (
+	"runtime"
+	"unsafe"
+)
+
+// ioctl itself should not be exposed directly, but additional get/set
+// functions for specific types are permissible.
+
+// IoctlSetInt performs an ioctl operation which sets an integer value
+// on fd, using the specified request number.
+func IoctlSetInt(fd int, req uint, value int) error {
+	return ioctl(fd, req, uintptr(value))
+}
+
+// IoctlSetWinsize performs an ioctl on fd with a *Winsize argument.
+//
+// To change fd's window size, the req argument should be TIOCSWINSZ.
+func IoctlSetWinsize(fd int, req uint, value *Winsize) error {
+	// TODO: if we get the chance, remove the req parameter and
+	// hardcode TIOCSWINSZ.
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
+}
+
+// IoctlSetTermios performs an ioctl on fd with a *Termios.
+//
+// The req value is expected to be TCSETS, TCSETSW, or TCSETSF
+func IoctlSetTermios(fd int, req uint, value *Termios) error {
+	if (req != TCSETS) && (req != TCSETSW) && (req != TCSETSF) {
+		return ENOSYS
+	}
+	err := Tcsetattr(fd, int(req), value)
+	runtime.KeepAlive(value)
+	return err
+}
+
+// IoctlGetInt performs an ioctl operation which gets an integer value
+// from fd, using the specified request number.
+//
+// A few ioctl requests use the return value as an output parameter;
+// for those, IoctlRetInt should be used instead of this function.
+func IoctlGetInt(fd int, req uint) (int, error) {
+	var value int
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(&value)))
+	return value, err
+}
+
+func IoctlGetWinsize(fd int, req uint) (*Winsize, error) {
+	var value Winsize
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(&value)))
+	return &value, err
+}
+
+// IoctlGetTermios performs an ioctl on fd with a *Termios.
+//
+// The req value is expected to be TCGETS
+func IoctlGetTermios(fd int, req uint) (*Termios, error) {
+	var value Termios
+	if req != TCGETS {
+		return &value, ENOSYS
+	}
+	err := Tcgetattr(fd, &value)
+	return &value, err
+}
diff --git a/vendor/golang.org/x/sys/unix/mkall.sh b/vendor/golang.org/x/sys/unix/mkall.sh
index d257fac..396aadf 100644
--- a/vendor/golang.org/x/sys/unix/mkall.sh
+++ b/vendor/golang.org/x/sys/unix/mkall.sh
@@ -70,23 +70,11 @@
 	mksyscall="go run mksyscall_aix_ppc64.go -aix"
 	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
 	;;
-darwin_386)
-	mkerrors="$mkerrors -m32"
-	mksyscall="go run mksyscall.go -l32"
-	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
-	mkasm="go run mkasm_darwin.go"
-	;;
 darwin_amd64)
 	mkerrors="$mkerrors -m64"
 	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
 	mkasm="go run mkasm_darwin.go"
 	;;
-darwin_arm)
-	mkerrors="$mkerrors"
-	mksyscall="go run mksyscall.go -l32"
-	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
-	mkasm="go run mkasm_darwin.go"
-	;;
 darwin_arm64)
 	mkerrors="$mkerrors -m64"
 	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
@@ -199,7 +187,7 @@
         mksyscall="go run mksyscall_solaris.go"
 	mkerrors=
 	mksysnum=
-	mktypes=
+	mktypes="GOARCH=$GOARCH go tool cgo -godefs"
 	;;
 *)
 	echo 'unrecognized $GOOS_$GOARCH: ' "$GOOSARCH" 1>&2
diff --git a/vendor/golang.org/x/sys/unix/mkerrors.sh b/vendor/golang.org/x/sys/unix/mkerrors.sh
index 1bef714..3f670fa 100644
--- a/vendor/golang.org/x/sys/unix/mkerrors.sh
+++ b/vendor/golang.org/x/sys/unix/mkerrors.sh
@@ -56,15 +56,19 @@
 #define _DARWIN_C_SOURCE
 #define KERNEL
 #define _DARWIN_USE_64_BIT_INODE
+#define __APPLE_USE_RFC_3542
 #include <stdint.h>
 #include <sys/attr.h>
 #include <sys/clonefile.h>
+#include <sys/kern_control.h>
 #include <sys/types.h>
 #include <sys/event.h>
 #include <sys/ptrace.h>
 #include <sys/select.h>
 #include <sys/socket.h>
+#include <sys/un.h>
 #include <sys/sockio.h>
+#include <sys/sys_domain.h>
 #include <sys/sysctl.h>
 #include <sys/mman.h>
 #include <sys/mount.h>
@@ -94,6 +98,7 @@
 #include <sys/ioctl.h>
 #include <net/bpf.h>
 #include <net/if.h>
+#include <net/if_clone.h>
 #include <net/if_types.h>
 #include <net/route.h>
 #include <netinet/in.h>
@@ -111,6 +116,7 @@
 #include <sys/sched.h>
 #include <sys/select.h>
 #include <sys/socket.h>
+#include <sys/un.h>
 #include <sys/sockio.h>
 #include <sys/stat.h>
 #include <sys/sysctl.h>
@@ -201,6 +207,7 @@
 #include <linux/devlink.h>
 #include <linux/dm-ioctl.h>
 #include <linux/errqueue.h>
+#include <linux/ethtool_netlink.h>
 #include <linux/falloc.h>
 #include <linux/fanotify.h>
 #include <linux/filter.h>
@@ -209,6 +216,8 @@
 #include <linux/fsverity.h>
 #include <linux/genetlink.h>
 #include <linux/hdreg.h>
+#include <linux/hidraw.h>
+#include <linux/icmp.h>
 #include <linux/icmpv6.h>
 #include <linux/if.h>
 #include <linux/if_addr.h>
@@ -219,17 +228,21 @@
 #include <linux/if_tun.h>
 #include <linux/if_packet.h>
 #include <linux/if_xdp.h>
+#include <linux/input.h>
 #include <linux/kexec.h>
 #include <linux/keyctl.h>
 #include <linux/loop.h>
+#include <linux/lwtunnel.h>
 #include <linux/magic.h>
 #include <linux/memfd.h>
 #include <linux/module.h>
 #include <linux/netfilter/nfnetlink.h>
 #include <linux/netlink.h>
 #include <linux/net_namespace.h>
+#include <linux/nfc.h>
 #include <linux/nsfs.h>
 #include <linux/perf_event.h>
+#include <linux/pps.h>
 #include <linux/ptrace.h>
 #include <linux/random.h>
 #include <linux/reboot.h>
@@ -246,6 +259,7 @@
 #include <linux/watchdog.h>
 
 #include <mtd/ubi-user.h>
+#include <mtd/mtd-user.h>
 #include <net/route.h>
 
 #if defined(__sparc__)
@@ -293,6 +307,17 @@
 // Including linux/l2tp.h here causes conflicts between linux/in.h
 // and netinet/in.h included via net/route.h above.
 #define IPPROTO_L2TP		115
+
+// Copied from linux/hid.h.
+// Keep in sync with the size of the referenced fields.
+#define _HIDIOCGRAWNAME_LEN	128 // sizeof_field(struct hid_device, name)
+#define _HIDIOCGRAWPHYS_LEN	64  // sizeof_field(struct hid_device, phys)
+#define _HIDIOCGRAWUNIQ_LEN	64  // sizeof_field(struct hid_device, uniq)
+
+#define _HIDIOCGRAWNAME		HIDIOCGRAWNAME(_HIDIOCGRAWNAME_LEN)
+#define _HIDIOCGRAWPHYS		HIDIOCGRAWPHYS(_HIDIOCGRAWPHYS_LEN)
+#define _HIDIOCGRAWUNIQ		HIDIOCGRAWUNIQ(_HIDIOCGRAWUNIQ_LEN)
+
 '
 
 includes_NetBSD='
@@ -372,6 +397,7 @@
 #include <sys/socket.h>
 #include <sys/sockio.h>
 #include <sys/stat.h>
+#include <sys/stream.h>
 #include <sys/mman.h>
 #include <sys/wait.h>
 #include <sys/ioctl.h>
@@ -381,10 +407,11 @@
 #include <net/if_arp.h>
 #include <net/if_types.h>
 #include <net/route.h>
+#include <netinet/icmp6.h>
 #include <netinet/in.h>
-#include <termios.h>
 #include <netinet/ip.h>
 #include <netinet/ip_mroute.h>
+#include <termios.h>
 '
 
 
@@ -439,6 +466,8 @@
 		$2 !~ /^EPROC_/ &&
 		$2 !~ /^EQUIV_/ &&
 		$2 !~ /^EXPR_/ &&
+		$2 !~ /^EVIOC/ &&
+		$2 !~ /^EV_/ &&
 		$2 ~ /^E[A-Z0-9_]+$/ ||
 		$2 ~ /^B[0-9_]+$/ ||
 		$2 ~ /^(OLD|NEW)DEV$/ ||
@@ -473,10 +502,13 @@
 		$2 ~ /^LOCK_(SH|EX|NB|UN)$/ ||
 		$2 ~ /^LO_(KEY|NAME)_SIZE$/ ||
 		$2 ~ /^LOOP_(CLR|CTL|GET|SET)_/ ||
-		$2 ~ /^(AF|SOCK|SO|SOL|IPPROTO|IP|IPV6|ICMP6|TCP|MCAST|EVFILT|NOTE|EV|SHUT|PROT|MAP|MFD|T?PACKET|MSG|SCM|MCL|DT|MADV|PR)_/ ||
+		$2 ~ /^(AF|SOCK|SO|SOL|IPPROTO|IP|IPV6|TCP|MCAST|EVFILT|NOTE|SHUT|PROT|MAP|MFD|T?PACKET|MSG|SCM|MCL|DT|MADV|PR|LOCAL)_/ ||
+		$2 ~ /^NFC_(GENL|PROTO|COMM|RF|SE|DIRECTION|LLCP|SOCKPROTO)_/ ||
+		$2 ~ /^NFC_.*_(MAX)?SIZE$/ ||
+		$2 ~ /^RAW_PAYLOAD_/ ||
 		$2 ~ /^TP_STATUS_/ ||
 		$2 ~ /^FALLOC_/ ||
-		$2 == "ICMPV6_FILTER" ||
+		$2 ~ /^ICMPV?6?_(FILTER|SEC)/ ||
 		$2 == "SOMAXCONN" ||
 		$2 == "NAME_MAX" ||
 		$2 == "IFNAMSIZ" ||
@@ -496,6 +528,7 @@
 		$2 !~ "NLA_TYPE_MASK" &&
 		$2 !~ /^RTC_VL_(ACCURACY|BACKUP|DATA)/ &&
 		$2 ~ /^(NETLINK|NLM|NLMSG|NLA|IFA|IFAN|RT|RTC|RTCF|RTN|RTPROT|RTNH|ARPHRD|ETH_P|NETNSA)_/ ||
+		$2 ~ /^FIORDCHK$/ ||
 		$2 ~ /^SIOC/ ||
 		$2 ~ /^TIOC/ ||
 		$2 ~ /^TCGET/ ||
@@ -516,6 +549,7 @@
 		$2 ~ /^CAP_/ ||
 		$2 ~ /^CP_/ ||
 		$2 ~ /^CPUSTATES$/ ||
+		$2 ~ /^CTLIOCGINFO$/ ||
 		$2 ~ /^ALG_/ ||
 		$2 ~ /^FI(CLONE|DEDUPERANGE)/ ||
 		$2 ~ /^FS_(POLICY_FLAGS|KEY_DESC|ENCRYPTION_MODE|[A-Z0-9_]+_KEY_SIZE)/ ||
@@ -527,7 +561,7 @@
 		$2 ~ /^RND/ ||
 		$2 ~ /^KEY_(SPEC|REQKEY_DEFL)_/ ||
 		$2 ~ /^KEYCTL_/ ||
-		$2 ~ /^PERF_EVENT_IOC_/ ||
+		$2 ~ /^PERF_/ ||
 		$2 ~ /^SECCOMP_MODE_/ ||
 		$2 ~ /^SPLICE_/ ||
 		$2 ~ /^SYNC_FILE_RANGE_/ ||
@@ -546,18 +580,27 @@
 		$2 ~ /^XATTR_(CREATE|REPLACE|NO(DEFAULT|FOLLOW|SECURITY)|SHOWCOMPRESSION)/ ||
 		$2 ~ /^ATTR_(BIT_MAP_COUNT|(CMN|VOL|FILE)_)/ ||
 		$2 ~ /^FSOPT_/ ||
-		$2 ~ /^WDIOC_/ ||
+		$2 ~ /^WDIO[CFS]_/ ||
 		$2 ~ /^NFN/ ||
 		$2 ~ /^XDP_/ ||
 		$2 ~ /^RWF_/ ||
 		$2 ~ /^(HDIO|WIN|SMART)_/ ||
 		$2 ~ /^CRYPTO_/ ||
 		$2 ~ /^TIPC_/ ||
+		$2 !~  "DEVLINK_RELOAD_LIMITS_VALID_MASK" &&
 		$2 ~ /^DEVLINK_/ ||
+		$2 ~ /^ETHTOOL_/ ||
+		$2 ~ /^LWTUNNEL_IP/ ||
 		$2 !~ "WMESGLEN" &&
 		$2 ~ /^W[A-Z0-9]+$/ ||
 		$2 ~/^PPPIOC/ ||
 		$2 ~ /^FAN_|FANOTIFY_/ ||
+		$2 == "HID_MAX_DESCRIPTOR_SIZE" ||
+		$2 ~ /^_?HIDIOC/ ||
+		$2 ~ /^BUS_(USB|HIL|BLUETOOTH|VIRTUAL)$/ ||
+		$2 ~ /^MTD/ ||
+		$2 ~ /^OTP/ ||
+		$2 ~ /^MEM/ ||
 		$2 ~ /^BLK[A-Z]*(GET$|SET$|BUF$|PART$|SIZE)/ {printf("\t%s = C.%s\n", $2, $2)}
 		$2 ~ /^__WCOREFLAG$/ {next}
 		$2 ~ /^__W[A-Z0-9]+$/ {printf("\t%s = C.%s\n", substr($2,3), $2)}
@@ -595,6 +638,7 @@
 echo '// mkerrors.sh' "$@"
 echo '// Code generated by the command above; see README.md. DO NOT EDIT.'
 echo
+echo "//go:build ${GOARCH} && ${GOOS}"
 echo "// +build ${GOARCH},${GOOS}"
 echo
 go tool cgo -godefs -- "$@" _const.go >_error.out
diff --git a/vendor/golang.org/x/sys/unix/pagesize_unix.go b/vendor/golang.org/x/sys/unix/pagesize_unix.go
index bc2f362..53f1b4c 100644
--- a/vendor/golang.org/x/sys/unix/pagesize_unix.go
+++ b/vendor/golang.org/x/sys/unix/pagesize_unix.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris
 // +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
 
 // For Unix, get the pagesize from the runtime.
diff --git a/vendor/golang.org/x/sys/unix/ptrace_darwin.go b/vendor/golang.org/x/sys/unix/ptrace_darwin.go
new file mode 100644
index 0000000..463c3ef
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ptrace_darwin.go
@@ -0,0 +1,12 @@
+// Copyright 2020 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.
+
+//go:build darwin && !ios
+// +build darwin,!ios
+
+package unix
+
+func ptrace(request int, pid int, addr uintptr, data uintptr) error {
+	return ptrace1(request, pid, addr, data)
+}
diff --git a/vendor/golang.org/x/sys/unix/ptrace_ios.go b/vendor/golang.org/x/sys/unix/ptrace_ios.go
new file mode 100644
index 0000000..ed0509a
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ptrace_ios.go
@@ -0,0 +1,12 @@
+// Copyright 2020 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.
+
+//go:build ios
+// +build ios
+
+package unix
+
+func ptrace(request int, pid int, addr uintptr, data uintptr) (err error) {
+	return ENOTSUP
+}
diff --git a/vendor/golang.org/x/sys/unix/race.go b/vendor/golang.org/x/sys/unix/race.go
index 61712b5..6f6c5fe 100644
--- a/vendor/golang.org/x/sys/unix/race.go
+++ b/vendor/golang.org/x/sys/unix/race.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build (darwin && race) || (linux && race) || (freebsd && race)
 // +build darwin,race linux,race freebsd,race
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/race0.go b/vendor/golang.org/x/sys/unix/race0.go
index ad02667..706e132 100644
--- a/vendor/golang.org/x/sys/unix/race0.go
+++ b/vendor/golang.org/x/sys/unix/race0.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix darwin,!race linux,!race freebsd,!race netbsd openbsd solaris dragonfly
+//go:build aix || (darwin && !race) || (linux && !race) || (freebsd && !race) || netbsd || openbsd || solaris || dragonfly || zos
+// +build aix darwin,!race linux,!race freebsd,!race netbsd openbsd solaris dragonfly zos
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/readdirent_getdents.go b/vendor/golang.org/x/sys/unix/readdirent_getdents.go
index 3a90aa6..4d62575 100644
--- a/vendor/golang.org/x/sys/unix/readdirent_getdents.go
+++ b/vendor/golang.org/x/sys/unix/readdirent_getdents.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || dragonfly || freebsd || linux || netbsd || openbsd
 // +build aix dragonfly freebsd linux netbsd openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/readdirent_getdirentries.go b/vendor/golang.org/x/sys/unix/readdirent_getdirentries.go
index 5fdae40..2a4ba47 100644
--- a/vendor/golang.org/x/sys/unix/readdirent_getdirentries.go
+++ b/vendor/golang.org/x/sys/unix/readdirent_getdirentries.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build darwin
 // +build darwin
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/sockcmsg_unix.go b/vendor/golang.org/x/sys/unix/sockcmsg_unix.go
index 003916e..453a942 100644
--- a/vendor/golang.org/x/sys/unix/sockcmsg_unix.go
+++ b/vendor/golang.org/x/sys/unix/sockcmsg_unix.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 // Socket control messages
 
diff --git a/vendor/golang.org/x/sys/unix/sockcmsg_unix_other.go b/vendor/golang.org/x/sys/unix/sockcmsg_unix_other.go
index 57a0021..0840fe4 100644
--- a/vendor/golang.org/x/sys/unix/sockcmsg_unix_other.go
+++ b/vendor/golang.org/x/sys/unix/sockcmsg_unix_other.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix darwin freebsd linux netbsd openbsd solaris
+//go:build aix || darwin || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin freebsd linux netbsd openbsd solaris zos
 
 package unix
 
@@ -36,6 +37,10 @@
 		if runtime.GOOS == "netbsd" && runtime.GOARCH == "arm64" {
 			salign = 16
 		}
+	case "zos":
+		// z/OS socket macros use [32-bit] sizeof(int) alignment,
+		// not pointer width.
+		salign = SizeofInt
 	}
 
 	return (salen + salign - 1) & ^(salign - 1)
diff --git a/vendor/golang.org/x/sys/unix/str.go b/vendor/golang.org/x/sys/unix/str.go
index 17fb698..8ba89ed 100644
--- a/vendor/golang.org/x/sys/unix/str.go
+++ b/vendor/golang.org/x/sys/unix/str.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris
 // +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall.go b/vendor/golang.org/x/sys/unix/syscall.go
index fd4ee8e..649fa87 100644
--- a/vendor/golang.org/x/sys/unix/syscall.go
+++ b/vendor/golang.org/x/sys/unix/syscall.go
@@ -2,7 +2,8 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 // Package unix contains an interface to the low-level operating system
 // primitives. OS details vary depending on the underlying system, and
@@ -24,7 +25,13 @@
 // holds a value of type syscall.Errno.
 package unix // import "golang.org/x/sys/unix"
 
-import "strings"
+import (
+	"bytes"
+	"strings"
+	"unsafe"
+
+	"golang.org/x/sys/internal/unsafeheader"
+)
 
 // ByteSliceFromString returns a NUL-terminated slice of bytes
 // containing the text of s. If s contains a NUL byte at any
@@ -49,5 +56,40 @@
 	return &a[0], nil
 }
 
+// ByteSliceToString returns a string form of the text represented by the slice s, with a terminating NUL and any
+// bytes after the NUL removed.
+func ByteSliceToString(s []byte) string {
+	if i := bytes.IndexByte(s, 0); i != -1 {
+		s = s[:i]
+	}
+	return string(s)
+}
+
+// BytePtrToString takes a pointer to a sequence of text and returns the corresponding string.
+// If the pointer is nil, it returns the empty string. It assumes that the text sequence is terminated
+// at a zero byte; if the zero byte is not present, the program may crash.
+func BytePtrToString(p *byte) string {
+	if p == nil {
+		return ""
+	}
+	if *p == 0 {
+		return ""
+	}
+
+	// Find NUL terminator.
+	n := 0
+	for ptr := unsafe.Pointer(p); *(*byte)(ptr) != 0; n++ {
+		ptr = unsafe.Pointer(uintptr(ptr) + 1)
+	}
+
+	var s []byte
+	h := (*unsafeheader.Slice)(unsafe.Pointer(&s))
+	h.Data = unsafe.Pointer(p)
+	h.Len = n
+	h.Cap = n
+
+	return string(s)
+}
+
 // Single-word zero for use when we need a valid pointer to 0 bytes.
 var _zero uintptr
diff --git a/vendor/golang.org/x/sys/unix/syscall_aix.go b/vendor/golang.org/x/sys/unix/syscall_aix.go
index 9ad8a0d..d8efb71 100644
--- a/vendor/golang.org/x/sys/unix/syscall_aix.go
+++ b/vendor/golang.org/x/sys/unix/syscall_aix.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix
 // +build aix
 
 // Aix system calls.
@@ -19,6 +20,22 @@
  * Wrapped
  */
 
+func Access(path string, mode uint32) (err error) {
+	return Faccessat(AT_FDCWD, path, mode, 0)
+}
+
+func Chmod(path string, mode uint32) (err error) {
+	return Fchmodat(AT_FDCWD, path, mode, 0)
+}
+
+func Chown(path string, uid int, gid int) (err error) {
+	return Fchownat(AT_FDCWD, path, uid, gid, 0)
+}
+
+func Creat(path string, mode uint32) (fd int, err error) {
+	return Open(path, O_CREAT|O_WRONLY|O_TRUNC, mode)
+}
+
 //sys	utimes(path string, times *[2]Timeval) (err error)
 func Utimes(path string, tv []Timeval) error {
 	if len(tv) != 2 {
@@ -235,7 +252,7 @@
 			}
 		}
 
-		bytes := (*[10000]byte)(unsafe.Pointer(&pp.Path[0]))[0:n]
+		bytes := (*[len(pp.Path)]byte)(unsafe.Pointer(&pp.Path[0]))[0:n]
 		sa.Name = string(bytes)
 		return sa, nil
 
@@ -403,8 +420,8 @@
 //sys	Mknod(path string, mode uint32, dev int) (err error)
 //sys	Mknodat(dirfd int, path string, mode uint32, dev int) (err error)
 //sys	Nanosleep(time *Timespec, leftover *Timespec) (err error)
-//sys   Open(path string, mode int, perm uint32) (fd int, err error) = open64
-//sys   Openat(dirfd int, path string, flags int, mode uint32) (fd int, err error)
+//sys	Open(path string, mode int, perm uint32) (fd int, err error) = open64
+//sys	Openat(dirfd int, path string, flags int, mode uint32) (fd int, err error)
 //sys	read(fd int, p []byte) (n int, err error)
 //sys	Readlink(path string, buf []byte) (n int, err error)
 //sys	Renameat(olddirfd int, oldpath string, newdirfd int, newpath string) (err error)
@@ -423,8 +440,8 @@
 //sysnb	Times(tms *Tms) (ticks uintptr, err error)
 //sysnb	Umask(mask int) (oldmask int)
 //sysnb	Uname(buf *Utsname) (err error)
-//sys   Unlink(path string) (err error)
-//sys   Unlinkat(dirfd int, path string, flags int) (err error)
+//sys	Unlink(path string) (err error)
+//sys	Unlinkat(dirfd int, path string, flags int) (err error)
 //sys	Ustat(dev int, ubuf *Ustat_t) (err error)
 //sys	write(fd int, p []byte) (n int, err error)
 //sys	readlen(fd int, p *byte, np int) (n int, err error) = read
@@ -498,7 +515,7 @@
 //sys	Munlock(b []byte) (err error)
 //sys	Munlockall() (err error)
 
-//sysnb pipe(p *[2]_C_int) (err error)
+//sysnb	pipe(p *[2]_C_int) (err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
diff --git a/vendor/golang.org/x/sys/unix/syscall_aix_ppc.go b/vendor/golang.org/x/sys/unix/syscall_aix_ppc.go
index b3c8e33..e92a0be 100644
--- a/vendor/golang.org/x/sys/unix/syscall_aix_ppc.go
+++ b/vendor/golang.org/x/sys/unix/syscall_aix_ppc.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix
-// +build ppc
+//go:build aix && ppc
+// +build aix,ppc
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_aix_ppc64.go b/vendor/golang.org/x/sys/unix/syscall_aix_ppc64.go
index 9a6e024..16eed17 100644
--- a/vendor/golang.org/x/sys/unix/syscall_aix_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_aix_ppc64.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build aix
-// +build ppc64
+//go:build aix && ppc64
+// +build aix,ppc64
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_bsd.go b/vendor/golang.org/x/sys/unix/syscall_bsd.go
index 123536a..95ac394 100644
--- a/vendor/golang.org/x/sys/unix/syscall_bsd.go
+++ b/vendor/golang.org/x/sys/unix/syscall_bsd.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build darwin || dragonfly || freebsd || netbsd || openbsd
 // +build darwin dragonfly freebsd netbsd openbsd
 
 // BSD system call wrappers shared by *BSD based systems
@@ -277,7 +278,7 @@
 		}
 		return sa, nil
 	}
-	return nil, EAFNOSUPPORT
+	return anyToSockaddrGOOS(fd, rsa)
 }
 
 func Accept(fd int) (nfd int, sa Sockaddr, err error) {
@@ -318,7 +319,7 @@
 	return anyToSockaddr(fd, &rsa)
 }
 
-//sysnb socketpair(domain int, typ int, proto int, fd *[2]int32) (err error)
+//sysnb	socketpair(domain int, typ int, proto int, fd *[2]int32) (err error)
 
 // GetsockoptString returns the string value of the socket option opt for the
 // socket associated with fd at the given socket level.
@@ -332,8 +333,8 @@
 	return string(buf[:vallen-1]), nil
 }
 
-//sys   recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error)
-//sys   sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error)
+//sys	recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error)
+//sys	sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error)
 //sys	recvmsg(s int, msg *Msghdr, flags int) (n int, err error)
 
 func Recvmsg(fd int, p, oob []byte, flags int) (n, oobn int, recvflags int, from Sockaddr, err error) {
@@ -626,7 +627,7 @@
 	return futimes(fd, (*[2]Timeval)(unsafe.Pointer(&tv[0])))
 }
 
-//sys   poll(fds *PollFd, nfds int, timeout int) (n int, err error)
+//sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
 	if len(fds) == 0 {
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin.1_12.go b/vendor/golang.org/x/sys/unix/syscall_darwin.1_12.go
index b31ef03..b009860 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin.1_12.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin.1_12.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build darwin && go1.12 && !go1.13
 // +build darwin,go1.12,!go1.13
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin.1_13.go b/vendor/golang.org/x/sys/unix/syscall_darwin.1_13.go
index dc0befe..1596426 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin.1_13.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin.1_13.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build darwin && go1.13
 // +build darwin,go1.13
 
 package unix
@@ -16,7 +17,7 @@
 //sys	readdir_r(dir uintptr, entry *Dirent, result **Dirent) (res Errno)
 
 func fdopendir(fd int) (dir uintptr, err error) {
-	r0, _, e1 := syscall_syscallPtr(funcPC(libc_fdopendir_trampoline), uintptr(fd), 0, 0)
+	r0, _, e1 := syscall_syscallPtr(libc_fdopendir_trampoline_addr, uintptr(fd), 0, 0)
 	dir = uintptr(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -24,9 +25,8 @@
 	return
 }
 
-func libc_fdopendir_trampoline()
+var libc_fdopendir_trampoline_addr uintptr
 
-//go:linkname libc_fdopendir libc_fdopendir
 //go:cgo_import_dynamic libc_fdopendir fdopendir "/usr/lib/libSystem.B.dylib"
 
 func Getdirentries(fd int, buf []byte, basep *uintptr) (n int, err error) {
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin.go b/vendor/golang.org/x/sys/unix/syscall_darwin.go
index 21b8092..9945e5f 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin.go
@@ -13,6 +13,7 @@
 package unix
 
 import (
+	"runtime"
 	"syscall"
 	"unsafe"
 )
@@ -30,10 +31,40 @@
 	raw    RawSockaddrDatalink
 }
 
+// SockaddrCtl implements the Sockaddr interface for AF_SYSTEM type sockets.
+type SockaddrCtl struct {
+	ID   uint32
+	Unit uint32
+	raw  RawSockaddrCtl
+}
+
+func (sa *SockaddrCtl) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	sa.raw.Sc_len = SizeofSockaddrCtl
+	sa.raw.Sc_family = AF_SYSTEM
+	sa.raw.Ss_sysaddr = AF_SYS_CONTROL
+	sa.raw.Sc_id = sa.ID
+	sa.raw.Sc_unit = sa.Unit
+	return unsafe.Pointer(&sa.raw), SizeofSockaddrCtl, nil
+}
+
+func anyToSockaddrGOOS(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	switch rsa.Addr.Family {
+	case AF_SYSTEM:
+		pp := (*RawSockaddrCtl)(unsafe.Pointer(rsa))
+		if pp.Ss_sysaddr == AF_SYS_CONTROL {
+			sa := new(SockaddrCtl)
+			sa.ID = pp.Sc_id
+			sa.Unit = pp.Sc_unit
+			return sa, nil
+		}
+	}
+	return nil, EAFNOSUPPORT
+}
+
 // Some external packages rely on SYS___SYSCTL being defined to implement their
 // own sysctl wrappers. Provide it here, even though direct syscalls are no
 // longer supported on darwin.
-const SYS___SYSCTL = 202
+const SYS___SYSCTL = SYS_SYSCTL
 
 // Translate "kern.hostname" to []_C_int{0,1,2,3}.
 func nametomib(name string) (mib []_C_int, err error) {
@@ -88,13 +119,16 @@
 	Forkattr    uint32
 }
 
-//sysnb pipe() (r int, w int, err error)
+//sysnb	pipe(p *[2]int32) (err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
 		return EINVAL
 	}
-	p[0], p[1], err = pipe()
+	var x [2]int32
+	err = pipe(&x)
+	p[0] = int(x[0])
+	p[1] = int(x[1])
 	return
 }
 
@@ -238,7 +272,7 @@
 		options)
 }
 
-//sys setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error)
+//sys	setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error)
 
 func utimensat(dirfd int, path string, times *[2]Timespec, flags int) error {
 	// Darwin doesn't support SYS_UTIMENSAT
@@ -257,7 +291,36 @@
 
 //sys	ioctl(fd int, req uint, arg uintptr) (err error)
 
-//sys   sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS_SYSCTL
+func IoctlCtlInfo(fd int, ctlInfo *CtlInfo) error {
+	err := ioctl(fd, CTLIOCGINFO, uintptr(unsafe.Pointer(ctlInfo)))
+	runtime.KeepAlive(ctlInfo)
+	return err
+}
+
+// IfreqMTU is struct ifreq used to get or set a network device's MTU.
+type IfreqMTU struct {
+	Name [IFNAMSIZ]byte
+	MTU  int32
+}
+
+// IoctlGetIfreqMTU performs the SIOCGIFMTU ioctl operation on fd to get the MTU
+// of the network device specified by ifname.
+func IoctlGetIfreqMTU(fd int, ifname string) (*IfreqMTU, error) {
+	var ifreq IfreqMTU
+	copy(ifreq.Name[:], ifname)
+	err := ioctl(fd, SIOCGIFMTU, uintptr(unsafe.Pointer(&ifreq)))
+	return &ifreq, err
+}
+
+// IoctlSetIfreqMTU performs the SIOCSIFMTU ioctl operation on fd to set the MTU
+// of the network device specified by ifreq.Name.
+func IoctlSetIfreqMTU(fd int, ifreq *IfreqMTU) error {
+	err := ioctl(fd, SIOCSIFMTU, uintptr(unsafe.Pointer(ifreq)))
+	runtime.KeepAlive(ifreq)
+	return err
+}
+
+//sys	sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS_SYSCTL
 
 func Uname(uname *Utsname) error {
 	mib := []_C_int{CTL_KERN, KERN_OSTYPE}
@@ -315,6 +378,26 @@
 	return
 }
 
+func GetsockoptIPMreqn(fd, level, opt int) (*IPMreqn, error) {
+	var value IPMreqn
+	vallen := _Socklen(SizeofIPMreqn)
+	errno := getsockopt(fd, level, opt, unsafe.Pointer(&value), &vallen)
+	return &value, errno
+}
+
+func SetsockoptIPMreqn(fd, level, opt int, mreq *IPMreqn) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(mreq), unsafe.Sizeof(*mreq))
+}
+
+// GetsockoptXucred is a getsockopt wrapper that returns an Xucred struct.
+// The usual level and opt are SOL_LOCAL and LOCAL_PEERCRED, respectively.
+func GetsockoptXucred(fd, level, opt int) (*Xucred, error) {
+	x := new(Xucred)
+	vallen := _Socklen(SizeofXucred)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(x), &vallen)
+	return x, err
+}
+
 //sys	sendfile(infd int, outfd int, offset int64, len *int64, hdtr unsafe.Pointer, flags int) (err error)
 
 /*
@@ -409,8 +492,8 @@
 //sys	Unlinkat(dirfd int, path string, flags int) (err error)
 //sys	Unmount(path string, flags int) (err error)
 //sys	write(fd int, p []byte) (n int, err error)
-//sys   mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
-//sys   munmap(addr uintptr, length uintptr) (err error)
+//sys	mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
+//sys	munmap(addr uintptr, length uintptr) (err error)
 //sys	readlen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_READ
 //sys	writelen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_WRITE
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin_386.go b/vendor/golang.org/x/sys/unix/syscall_darwin_386.go
deleted file mode 100644
index ea0be1e..0000000
--- a/vendor/golang.org/x/sys/unix/syscall_darwin_386.go
+++ /dev/null
@@ -1,53 +0,0 @@
-// 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 386,darwin
-
-package unix
-
-import (
-	"syscall"
-)
-
-//sys   ptrace(request int, pid int, addr uintptr, data uintptr) (err error)
-
-func setTimespec(sec, nsec int64) Timespec {
-	return Timespec{Sec: int32(sec), Nsec: int32(nsec)}
-}
-
-func setTimeval(sec, usec int64) Timeval {
-	return Timeval{Sec: int32(sec), Usec: int32(usec)}
-}
-
-func SetKevent(k *Kevent_t, fd, mode, flags int) {
-	k.Ident = uint32(fd)
-	k.Filter = int16(mode)
-	k.Flags = uint16(flags)
-}
-
-func (iov *Iovec) SetLen(length int) {
-	iov.Len = uint32(length)
-}
-
-func (msghdr *Msghdr) SetControllen(length int) {
-	msghdr.Controllen = uint32(length)
-}
-
-func (msghdr *Msghdr) SetIovlen(length int) {
-	msghdr.Iovlen = int32(length)
-}
-
-func (cmsg *Cmsghdr) SetLen(length int) {
-	cmsg.Len = uint32(length)
-}
-
-func Syscall9(num, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err syscall.Errno)
-
-//sys	Fstat(fd int, stat *Stat_t) (err error) = SYS_FSTAT64
-//sys	Fstatat(fd int, path string, stat *Stat_t, flags int) (err error) = SYS_FSTATAT64
-//sys	Fstatfs(fd int, stat *Statfs_t) (err error) = SYS_FSTATFS64
-//sys	getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) = SYS_GETFSSTAT64
-//sys	Lstat(path string, stat *Stat_t) (err error) = SYS_LSTAT64
-//sys	Stat(path string, stat *Stat_t) (err error) = SYS_STAT64
-//sys	Statfs(path string, stat *Statfs_t) (err error) = SYS_STATFS64
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin_amd64.go b/vendor/golang.org/x/sys/unix/syscall_darwin_amd64.go
index 5862404..b37310c 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin_amd64.go
@@ -2,15 +2,12 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && darwin
 // +build amd64,darwin
 
 package unix
 
-import (
-	"syscall"
-)
-
-//sys   ptrace(request int, pid int, addr uintptr, data uintptr) (err error)
+import "syscall"
 
 func setTimespec(sec, nsec int64) Timespec {
 	return Timespec{Sec: sec, Nsec: nsec}
@@ -49,5 +46,6 @@
 //sys	Fstatfs(fd int, stat *Statfs_t) (err error) = SYS_FSTATFS64
 //sys	getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) = SYS_GETFSSTAT64
 //sys	Lstat(path string, stat *Stat_t) (err error) = SYS_LSTAT64
+//sys	ptrace1(request int, pid int, addr uintptr, data uintptr) (err error) = SYS_ptrace
 //sys	Stat(path string, stat *Stat_t) (err error) = SYS_STAT64
 //sys	Statfs(path string, stat *Statfs_t) (err error) = SYS_STATFS64
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin_arm.go b/vendor/golang.org/x/sys/unix/syscall_darwin_arm.go
deleted file mode 100644
index b8b3141..0000000
--- a/vendor/golang.org/x/sys/unix/syscall_darwin_arm.go
+++ /dev/null
@@ -1,53 +0,0 @@
-// 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 unix
-
-import (
-	"syscall"
-)
-
-func ptrace(request int, pid int, addr uintptr, data uintptr) error {
-	return ENOTSUP
-}
-
-func setTimespec(sec, nsec int64) Timespec {
-	return Timespec{Sec: int32(sec), Nsec: int32(nsec)}
-}
-
-func setTimeval(sec, usec int64) Timeval {
-	return Timeval{Sec: int32(sec), Usec: int32(usec)}
-}
-
-func SetKevent(k *Kevent_t, fd, mode, flags int) {
-	k.Ident = uint32(fd)
-	k.Filter = int16(mode)
-	k.Flags = uint16(flags)
-}
-
-func (iov *Iovec) SetLen(length int) {
-	iov.Len = uint32(length)
-}
-
-func (msghdr *Msghdr) SetControllen(length int) {
-	msghdr.Controllen = uint32(length)
-}
-
-func (msghdr *Msghdr) SetIovlen(length int) {
-	msghdr.Iovlen = int32(length)
-}
-
-func (cmsg *Cmsghdr) SetLen(length int) {
-	cmsg.Len = uint32(length)
-}
-
-func Syscall9(num, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err syscall.Errno) // sic
-
-//sys	Fstat(fd int, stat *Stat_t) (err error)
-//sys	Fstatat(fd int, path string, stat *Stat_t, flags int) (err error)
-//sys	Fstatfs(fd int, stat *Statfs_t) (err error)
-//sys	getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) = SYS_GETFSSTAT
-//sys	Lstat(path string, stat *Stat_t) (err error)
-//sys	Stat(path string, stat *Stat_t) (err error)
-//sys	Statfs(path string, stat *Statfs_t) (err error)
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin_arm64.go b/vendor/golang.org/x/sys/unix/syscall_darwin_arm64.go
index 6741398..d51ec99 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin_arm64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin_arm64.go
@@ -2,17 +2,12 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm64 && darwin
 // +build arm64,darwin
 
 package unix
 
-import (
-	"syscall"
-)
-
-func ptrace(request int, pid int, addr uintptr, data uintptr) error {
-	return ENOTSUP
-}
+import "syscall"
 
 func setTimespec(sec, nsec int64) Timespec {
 	return Timespec{Sec: sec, Nsec: nsec}
@@ -51,5 +46,6 @@
 //sys	Fstatfs(fd int, stat *Statfs_t) (err error)
 //sys	getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) = SYS_GETFSSTAT
 //sys	Lstat(path string, stat *Stat_t) (err error)
+//sys	ptrace1(request int, pid int, addr uintptr, data uintptr) (err error) = SYS_ptrace
 //sys	Stat(path string, stat *Stat_t) (err error)
 //sys	Statfs(path string, stat *Statfs_t) (err error)
diff --git a/vendor/golang.org/x/sys/unix/syscall_darwin_libSystem.go b/vendor/golang.org/x/sys/unix/syscall_darwin_libSystem.go
index f34c86c..53c9664 100644
--- a/vendor/golang.org/x/sys/unix/syscall_darwin_libSystem.go
+++ b/vendor/golang.org/x/sys/unix/syscall_darwin_libSystem.go
@@ -2,11 +2,12 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build darwin && go1.12
 // +build darwin,go1.12
 
 package unix
 
-import "unsafe"
+import _ "unsafe"
 
 // Implemented in the runtime package (runtime/sys_darwin.go)
 func syscall_syscall(fn, a1, a2, a3 uintptr) (r1, r2 uintptr, err Errno)
@@ -24,10 +25,3 @@
 //go:linkname syscall_rawSyscall syscall.rawSyscall
 //go:linkname syscall_rawSyscall6 syscall.rawSyscall6
 //go:linkname syscall_syscallPtr syscall.syscallPtr
-
-// Find the entry point for f. See comments in runtime/proc.go for the
-// function of the same name.
-//go:nosplit
-func funcPC(f func()) uintptr {
-	return **(**uintptr)(unsafe.Pointer(&f))
-}
diff --git a/vendor/golang.org/x/sys/unix/syscall_dragonfly.go b/vendor/golang.org/x/sys/unix/syscall_dragonfly.go
index bed7dcf..5af108a 100644
--- a/vendor/golang.org/x/sys/unix/syscall_dragonfly.go
+++ b/vendor/golang.org/x/sys/unix/syscall_dragonfly.go
@@ -47,6 +47,10 @@
 	raw    RawSockaddrDatalink
 }
 
+func anyToSockaddrGOOS(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	return nil, EAFNOSUPPORT
+}
+
 // Translate "kern.hostname" to []_C_int{0,1,2,3}.
 func nametomib(name string) (mib []_C_int, err error) {
 	const siz = unsafe.Sizeof(mib[0])
@@ -91,7 +95,7 @@
 	return readInt(buf, unsafe.Offsetof(Dirent{}.Namlen), unsafe.Sizeof(Dirent{}.Namlen))
 }
 
-//sysnb pipe() (r int, w int, err error)
+//sysnb	pipe() (r int, w int, err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
@@ -101,6 +105,19 @@
 	return
 }
 
+//sysnb	pipe2(p *[2]_C_int, flags int) (r int, w int, err error)
+
+func Pipe2(p []int, flags int) (err error) {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	// pipe2 on dragonfly takes an fds array as an argument, but still
+	// returns the file descriptors.
+	p[0], p[1], err = pipe2(&pp, flags)
+	return err
+}
+
 //sys	extpread(fd int, p []byte, flags int, offset int64) (n int, err error)
 func Pread(fd int, p []byte, offset int64) (n int, err error) {
 	return extpread(fd, p, 0, offset)
@@ -153,7 +170,7 @@
 
 //sys	ioctl(fd int, req uint, arg uintptr) (err error)
 
-//sys   sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
+//sys	sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
 
 func sysctlUname(mib []_C_int, old *byte, oldlen *uintptr) error {
 	err := sysctl(mib, old, oldlen, nil, 0)
@@ -320,8 +337,8 @@
 //sys	Unlinkat(dirfd int, path string, flags int) (err error)
 //sys	Unmount(path string, flags int) (err error)
 //sys	write(fd int, p []byte) (n int, err error)
-//sys   mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
-//sys   munmap(addr uintptr, length uintptr) (err error)
+//sys	mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
+//sys	munmap(addr uintptr, length uintptr) (err error)
 //sys	readlen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_READ
 //sys	writelen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_WRITE
 //sys	accept4(fd int, rsa *RawSockaddrAny, addrlen *_Socklen, flags int) (nfd int, err error)
diff --git a/vendor/golang.org/x/sys/unix/syscall_dragonfly_amd64.go b/vendor/golang.org/x/sys/unix/syscall_dragonfly_amd64.go
index a6b4830..4e2d321 100644
--- a/vendor/golang.org/x/sys/unix/syscall_dragonfly_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_dragonfly_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && dragonfly
 // +build amd64,dragonfly
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_freebsd.go b/vendor/golang.org/x/sys/unix/syscall_freebsd.go
index f6db02a..18c392c 100644
--- a/vendor/golang.org/x/sys/unix/syscall_freebsd.go
+++ b/vendor/golang.org/x/sys/unix/syscall_freebsd.go
@@ -54,6 +54,10 @@
 	raw    RawSockaddrDatalink
 }
 
+func anyToSockaddrGOOS(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	return nil, EAFNOSUPPORT
+}
+
 // Translate "kern.hostname" to []_C_int{0,1,2,3}.
 func nametomib(name string) (mib []_C_int, err error) {
 	const siz = unsafe.Sizeof(mib[0])
@@ -122,6 +126,15 @@
 	return setsockopt(fd, level, opt, unsafe.Pointer(mreq), unsafe.Sizeof(*mreq))
 }
 
+// GetsockoptXucred is a getsockopt wrapper that returns an Xucred struct.
+// The usual level and opt are SOL_LOCAL and LOCAL_PEERCRED, respectively.
+func GetsockoptXucred(fd, level, opt int) (*Xucred, error) {
+	x := new(Xucred)
+	vallen := _Socklen(SizeofXucred)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(x), &vallen)
+	return x, err
+}
+
 func Accept4(fd, flags int) (nfd int, sa Sockaddr, err error) {
 	var rsa RawSockaddrAny
 	var len _Socklen = SizeofSockaddrAny
@@ -184,9 +197,9 @@
 	return ENOSYS
 }
 
-//sys   ioctl(fd int, req uint, arg uintptr) (err error)
+//sys	ioctl(fd int, req uint, arg uintptr) (err error)
 
-//sys   sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
+//sys	sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
 
 func Uname(uname *Utsname) error {
 	mib := []_C_int{CTL_KERN, KERN_OSTYPE}
@@ -661,8 +674,8 @@
 //sys	Unlinkat(dirfd int, path string, flags int) (err error)
 //sys	Unmount(path string, flags int) (err error)
 //sys	write(fd int, p []byte) (n int, err error)
-//sys   mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
-//sys   munmap(addr uintptr, length uintptr) (err error)
+//sys	mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error)
+//sys	munmap(addr uintptr, length uintptr) (err error)
 //sys	readlen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_READ
 //sys	writelen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_WRITE
 //sys	accept4(fd int, rsa *RawSockaddrAny, addrlen *_Socklen, flags int) (nfd int, err error)
diff --git a/vendor/golang.org/x/sys/unix/syscall_freebsd_386.go b/vendor/golang.org/x/sys/unix/syscall_freebsd_386.go
index 72a506d..342fc32 100644
--- a/vendor/golang.org/x/sys/unix/syscall_freebsd_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_freebsd_386.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build 386 && freebsd
 // +build 386,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_freebsd_amd64.go b/vendor/golang.org/x/sys/unix/syscall_freebsd_amd64.go
index d5e376a..a32d5aa 100644
--- a/vendor/golang.org/x/sys/unix/syscall_freebsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_freebsd_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && freebsd
 // +build amd64,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_freebsd_arm.go b/vendor/golang.org/x/sys/unix/syscall_freebsd_arm.go
index 4ea45bc..1e36d39 100644
--- a/vendor/golang.org/x/sys/unix/syscall_freebsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_freebsd_arm.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm && freebsd
 // +build arm,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_freebsd_arm64.go b/vendor/golang.org/x/sys/unix/syscall_freebsd_arm64.go
index aa5326d..a09a153 100644
--- a/vendor/golang.org/x/sys/unix/syscall_freebsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_freebsd_arm64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm64 && freebsd
 // +build arm64,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_illumos.go b/vendor/golang.org/x/sys/unix/syscall_illumos.go
index bbc4f3e..8c53576 100644
--- a/vendor/golang.org/x/sys/unix/syscall_illumos.go
+++ b/vendor/golang.org/x/sys/unix/syscall_illumos.go
@@ -1,14 +1,19 @@
-// Copyright 2009 The Go Authors. All rights reserved.
+// Copyright 2021 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.
 
 // illumos system calls not present on Solaris.
 
+//go:build amd64 && illumos
 // +build amd64,illumos
 
 package unix
 
-import "unsafe"
+import (
+	"fmt"
+	"runtime"
+	"unsafe"
+)
 
 func bytes2iovec(bs [][]byte) []Iovec {
 	iovecs := make([]Iovec, len(bs))
@@ -76,15 +81,98 @@
 	return
 }
 
-//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
+//sys	putmsg(fd int, clptr *strbuf, dataptr *strbuf, flags int) (err error)
 
-func Pipe2(p []int, flags int) error {
-	if len(p) != 2 {
-		return EINVAL
+func Putmsg(fd int, cl []byte, data []byte, flags int) (err error) {
+	var clp, datap *strbuf
+	if len(cl) > 0 {
+		clp = &strbuf{
+			Len: int32(len(cl)),
+			Buf: (*int8)(unsafe.Pointer(&cl[0])),
+		}
 	}
-	var pp [2]_C_int
-	err := pipe2(&pp, flags)
-	p[0] = int(pp[0])
-	p[1] = int(pp[1])
+	if len(data) > 0 {
+		datap = &strbuf{
+			Len: int32(len(data)),
+			Buf: (*int8)(unsafe.Pointer(&data[0])),
+		}
+	}
+	return putmsg(fd, clp, datap, flags)
+}
+
+//sys	getmsg(fd int, clptr *strbuf, dataptr *strbuf, flags *int) (err error)
+
+func Getmsg(fd int, cl []byte, data []byte) (retCl []byte, retData []byte, flags int, err error) {
+	var clp, datap *strbuf
+	if len(cl) > 0 {
+		clp = &strbuf{
+			Maxlen: int32(len(cl)),
+			Buf:    (*int8)(unsafe.Pointer(&cl[0])),
+		}
+	}
+	if len(data) > 0 {
+		datap = &strbuf{
+			Maxlen: int32(len(data)),
+			Buf:    (*int8)(unsafe.Pointer(&data[0])),
+		}
+	}
+
+	if err = getmsg(fd, clp, datap, &flags); err != nil {
+		return nil, nil, 0, err
+	}
+
+	if len(cl) > 0 {
+		retCl = cl[:clp.Len]
+	}
+	if len(data) > 0 {
+		retData = data[:datap.Len]
+	}
+	return retCl, retData, flags, nil
+}
+
+func IoctlSetIntRetInt(fd int, req uint, arg int) (int, error) {
+	return ioctlRet(fd, req, uintptr(arg))
+}
+
+func IoctlSetString(fd int, req uint, val string) error {
+	bs := make([]byte, len(val)+1)
+	copy(bs[:len(bs)-1], val)
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(&bs[0])))
+	runtime.KeepAlive(&bs[0])
 	return err
 }
+
+// Lifreq Helpers
+
+func (l *Lifreq) SetName(name string) error {
+	if len(name) >= len(l.Name) {
+		return fmt.Errorf("name cannot be more than %d characters", len(l.Name)-1)
+	}
+	for i := range name {
+		l.Name[i] = int8(name[i])
+	}
+	return nil
+}
+
+func (l *Lifreq) SetLifruInt(d int) {
+	*(*int)(unsafe.Pointer(&l.Lifru[0])) = d
+}
+
+func (l *Lifreq) GetLifruInt() int {
+	return *(*int)(unsafe.Pointer(&l.Lifru[0]))
+}
+
+func IoctlLifreq(fd int, req uint, l *Lifreq) error {
+	return ioctl(fd, req, uintptr(unsafe.Pointer(l)))
+}
+
+// Strioctl Helpers
+
+func (s *Strioctl) SetInt(i int) {
+	s.Len = int32(unsafe.Sizeof(i))
+	s.Dp = (*int8)(unsafe.Pointer(&i))
+}
+
+func IoctlSetStrioctlRetInt(fd int, req uint, s *Strioctl) (int, error) {
+	return ioctlRet(fd, req, uintptr(unsafe.Pointer(s)))
+}
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux.go b/vendor/golang.org/x/sys/unix/syscall_linux.go
index 94dafa4..41b91fd 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux.go
@@ -70,72 +70,7 @@
 
 // ioctl itself should not be exposed directly, but additional get/set
 // functions for specific types are permissible.
-
-// IoctlRetInt performs an ioctl operation specified by req on a device
-// associated with opened file descriptor fd, and returns a non-negative
-// integer that is returned by the ioctl syscall.
-func IoctlRetInt(fd int, req uint) (int, error) {
-	ret, _, err := Syscall(SYS_IOCTL, uintptr(fd), uintptr(req), 0)
-	if err != 0 {
-		return 0, err
-	}
-	return int(ret), nil
-}
-
-func IoctlSetRTCTime(fd int, value *RTCTime) error {
-	err := ioctl(fd, RTC_SET_TIME, uintptr(unsafe.Pointer(value)))
-	runtime.KeepAlive(value)
-	return err
-}
-
-func IoctlSetRTCWkAlrm(fd int, value *RTCWkAlrm) error {
-	err := ioctl(fd, RTC_WKALM_SET, uintptr(unsafe.Pointer(value)))
-	runtime.KeepAlive(value)
-	return err
-}
-
-func IoctlGetUint32(fd int, req uint) (uint32, error) {
-	var value uint32
-	err := ioctl(fd, req, uintptr(unsafe.Pointer(&value)))
-	return value, err
-}
-
-func IoctlGetRTCTime(fd int) (*RTCTime, error) {
-	var value RTCTime
-	err := ioctl(fd, RTC_RD_TIME, uintptr(unsafe.Pointer(&value)))
-	return &value, err
-}
-
-func IoctlGetRTCWkAlrm(fd int) (*RTCWkAlrm, error) {
-	var value RTCWkAlrm
-	err := ioctl(fd, RTC_WKALM_RD, uintptr(unsafe.Pointer(&value)))
-	return &value, err
-}
-
-// IoctlFileClone performs an FICLONERANGE ioctl operation to clone the range of
-// data conveyed in value to the file associated with the file descriptor
-// destFd. See the ioctl_ficlonerange(2) man page for details.
-func IoctlFileCloneRange(destFd int, value *FileCloneRange) error {
-	err := ioctl(destFd, FICLONERANGE, uintptr(unsafe.Pointer(value)))
-	runtime.KeepAlive(value)
-	return err
-}
-
-// IoctlFileClone performs an FICLONE ioctl operation to clone the entire file
-// associated with the file description srcFd to the file associated with the
-// file descriptor destFd. See the ioctl_ficlone(2) man page for details.
-func IoctlFileClone(destFd, srcFd int) error {
-	return ioctl(destFd, FICLONE, uintptr(srcFd))
-}
-
-// IoctlFileClone performs an FIDEDUPERANGE ioctl operation to share the range of
-// data conveyed in value with the file associated with the file descriptor
-// destFd. See the ioctl_fideduperange(2) man page for details.
-func IoctlFileDedupeRange(destFd int, value *FileDedupeRange) error {
-	err := ioctl(destFd, FIDEDUPERANGE, uintptr(unsafe.Pointer(value)))
-	runtime.KeepAlive(value)
-	return err
-}
+// These are defined in ioctl.go and ioctl_linux.go.
 
 //sys	Linkat(olddirfd int, oldpath string, newdirfd int, newpath string, flags int) (err error)
 
@@ -625,6 +560,36 @@
 	return unsafe.Pointer(&sa.raw), SizeofSockaddrCAN, nil
 }
 
+// SockaddrCANJ1939 implements the Sockaddr interface for AF_CAN using J1939
+// protocol (https://en.wikipedia.org/wiki/SAE_J1939). For more information
+// on the purposes of the fields, check the official linux kernel documentation
+// available here: https://www.kernel.org/doc/Documentation/networking/j1939.rst
+type SockaddrCANJ1939 struct {
+	Ifindex int
+	Name    uint64
+	PGN     uint32
+	Addr    uint8
+	raw     RawSockaddrCAN
+}
+
+func (sa *SockaddrCANJ1939) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	if sa.Ifindex < 0 || sa.Ifindex > 0x7fffffff {
+		return nil, 0, EINVAL
+	}
+	sa.raw.Family = AF_CAN
+	sa.raw.Ifindex = int32(sa.Ifindex)
+	n := (*[8]byte)(unsafe.Pointer(&sa.Name))
+	for i := 0; i < 8; i++ {
+		sa.raw.Addr[i] = n[i]
+	}
+	p := (*[4]byte)(unsafe.Pointer(&sa.PGN))
+	for i := 0; i < 4; i++ {
+		sa.raw.Addr[i+8] = p[i]
+	}
+	sa.raw.Addr[12] = sa.Addr
+	return unsafe.Pointer(&sa.raw), SizeofSockaddrCAN, nil
+}
+
 // SockaddrALG implements the Sockaddr interface for AF_ALG type sockets.
 // SockaddrALG enables userspace access to the Linux kernel's cryptography
 // subsystem. The Type and Name fields specify which type of hash or cipher
@@ -732,16 +697,19 @@
 	// CID and Port specify a context ID and port address for a VM socket.
 	// Guests have a unique CID, and hosts may have a well-known CID of:
 	//  - VMADDR_CID_HYPERVISOR: refers to the hypervisor process.
+	//  - VMADDR_CID_LOCAL: refers to local communication (loopback).
 	//  - VMADDR_CID_HOST: refers to other processes on the host.
-	CID  uint32
-	Port uint32
-	raw  RawSockaddrVM
+	CID   uint32
+	Port  uint32
+	Flags uint8
+	raw   RawSockaddrVM
 }
 
 func (sa *SockaddrVM) sockaddr() (unsafe.Pointer, _Socklen, error) {
 	sa.raw.Family = AF_VSOCK
 	sa.raw.Port = sa.Port
 	sa.raw.Cid = sa.CID
+	sa.raw.Flags = sa.Flags
 
 	return unsafe.Pointer(&sa.raw), SizeofSockaddrVM, nil
 }
@@ -936,6 +904,50 @@
 	return unsafe.Pointer(&sa.raw), SizeofSockaddrIUCV, nil
 }
 
+type SockaddrNFC struct {
+	DeviceIdx   uint32
+	TargetIdx   uint32
+	NFCProtocol uint32
+	raw         RawSockaddrNFC
+}
+
+func (sa *SockaddrNFC) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	sa.raw.Sa_family = AF_NFC
+	sa.raw.Dev_idx = sa.DeviceIdx
+	sa.raw.Target_idx = sa.TargetIdx
+	sa.raw.Nfc_protocol = sa.NFCProtocol
+	return unsafe.Pointer(&sa.raw), SizeofSockaddrNFC, nil
+}
+
+type SockaddrNFCLLCP struct {
+	DeviceIdx      uint32
+	TargetIdx      uint32
+	NFCProtocol    uint32
+	DestinationSAP uint8
+	SourceSAP      uint8
+	ServiceName    string
+	raw            RawSockaddrNFCLLCP
+}
+
+func (sa *SockaddrNFCLLCP) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	sa.raw.Sa_family = AF_NFC
+	sa.raw.Dev_idx = sa.DeviceIdx
+	sa.raw.Target_idx = sa.TargetIdx
+	sa.raw.Nfc_protocol = sa.NFCProtocol
+	sa.raw.Dsap = sa.DestinationSAP
+	sa.raw.Ssap = sa.SourceSAP
+	if len(sa.ServiceName) > len(sa.raw.Service_name) {
+		return nil, 0, EINVAL
+	}
+	copy(sa.raw.Service_name[:], sa.ServiceName)
+	sa.raw.SetServiceNameLen(len(sa.ServiceName))
+	return unsafe.Pointer(&sa.raw), SizeofSockaddrNFCLLCP, nil
+}
+
+var socketProtocol = func(fd int) (int, error) {
+	return GetsockoptInt(fd, SOL_SOCKET, SO_PROTOCOL)
+}
+
 func anyToSockaddr(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
 	switch rsa.Addr.Family {
 	case AF_NETLINK:
@@ -986,7 +998,7 @@
 		return sa, nil
 
 	case AF_INET:
-		proto, err := GetsockoptInt(fd, SOL_SOCKET, SO_PROTOCOL)
+		proto, err := socketProtocol(fd)
 		if err != nil {
 			return nil, err
 		}
@@ -1012,7 +1024,7 @@
 		}
 
 	case AF_INET6:
-		proto, err := GetsockoptInt(fd, SOL_SOCKET, SO_PROTOCOL)
+		proto, err := socketProtocol(fd)
 		if err != nil {
 			return nil, err
 		}
@@ -1042,12 +1054,13 @@
 	case AF_VSOCK:
 		pp := (*RawSockaddrVM)(unsafe.Pointer(rsa))
 		sa := &SockaddrVM{
-			CID:  pp.Cid,
-			Port: pp.Port,
+			CID:   pp.Cid,
+			Port:  pp.Port,
+			Flags: pp.Flags,
 		}
 		return sa, nil
 	case AF_BLUETOOTH:
-		proto, err := GetsockoptInt(fd, SOL_SOCKET, SO_PROTOCOL)
+		proto, err := socketProtocol(fd)
 		if err != nil {
 			return nil, err
 		}
@@ -1134,20 +1147,74 @@
 		return sa, nil
 
 	case AF_CAN:
-		pp := (*RawSockaddrCAN)(unsafe.Pointer(rsa))
-		sa := &SockaddrCAN{
-			Ifindex: int(pp.Ifindex),
+		proto, err := socketProtocol(fd)
+		if err != nil {
+			return nil, err
 		}
-		rx := (*[4]byte)(unsafe.Pointer(&sa.RxID))
-		for i := 0; i < 4; i++ {
-			rx[i] = pp.Addr[i]
-		}
-		tx := (*[4]byte)(unsafe.Pointer(&sa.TxID))
-		for i := 0; i < 4; i++ {
-			tx[i] = pp.Addr[i+4]
-		}
-		return sa, nil
 
+		pp := (*RawSockaddrCAN)(unsafe.Pointer(rsa))
+
+		switch proto {
+		case CAN_J1939:
+			sa := &SockaddrCANJ1939{
+				Ifindex: int(pp.Ifindex),
+			}
+			name := (*[8]byte)(unsafe.Pointer(&sa.Name))
+			for i := 0; i < 8; i++ {
+				name[i] = pp.Addr[i]
+			}
+			pgn := (*[4]byte)(unsafe.Pointer(&sa.PGN))
+			for i := 0; i < 4; i++ {
+				pgn[i] = pp.Addr[i+8]
+			}
+			addr := (*[1]byte)(unsafe.Pointer(&sa.Addr))
+			addr[0] = pp.Addr[12]
+			return sa, nil
+		default:
+			sa := &SockaddrCAN{
+				Ifindex: int(pp.Ifindex),
+			}
+			rx := (*[4]byte)(unsafe.Pointer(&sa.RxID))
+			for i := 0; i < 4; i++ {
+				rx[i] = pp.Addr[i]
+			}
+			tx := (*[4]byte)(unsafe.Pointer(&sa.TxID))
+			for i := 0; i < 4; i++ {
+				tx[i] = pp.Addr[i+4]
+			}
+			return sa, nil
+		}
+	case AF_NFC:
+		proto, err := socketProtocol(fd)
+		if err != nil {
+			return nil, err
+		}
+		switch proto {
+		case NFC_SOCKPROTO_RAW:
+			pp := (*RawSockaddrNFC)(unsafe.Pointer(rsa))
+			sa := &SockaddrNFC{
+				DeviceIdx:   pp.Dev_idx,
+				TargetIdx:   pp.Target_idx,
+				NFCProtocol: pp.Nfc_protocol,
+			}
+			return sa, nil
+		case NFC_SOCKPROTO_LLCP:
+			pp := (*RawSockaddrNFCLLCP)(unsafe.Pointer(rsa))
+			if uint64(pp.Service_name_len) > uint64(len(pp.Service_name)) {
+				return nil, EINVAL
+			}
+			sa := &SockaddrNFCLLCP{
+				DeviceIdx:      pp.Dev_idx,
+				TargetIdx:      pp.Target_idx,
+				NFCProtocol:    pp.Nfc_protocol,
+				DestinationSAP: pp.Dsap,
+				SourceSAP:      pp.Ssap,
+				ServiceName:    string(pp.Service_name[:pp.Service_name_len]),
+			}
+			return sa, nil
+		default:
+			return nil, EINVAL
+		}
 	}
 	return nil, EAFNOSUPPORT
 }
@@ -1155,7 +1222,11 @@
 func Accept(fd int) (nfd int, sa Sockaddr, err error) {
 	var rsa RawSockaddrAny
 	var len _Socklen = SizeofSockaddrAny
-	nfd, err = accept(fd, &rsa, &len)
+	// Try accept4 first for Android, then try accept for kernel older than 2.6.28
+	nfd, err = accept4(fd, &rsa, &len, 0)
+	if err == ENOSYS {
+		nfd, err = accept(fd, &rsa, &len)
+	}
 	if err != nil {
 		return
 	}
@@ -1409,8 +1480,8 @@
 	return keyctlRestrictKeyringByType(KEYCTL_RESTRICT_KEYRING, ringid, keyType, restriction)
 }
 
-//sys keyctlRestrictKeyringByType(cmd int, arg2 int, keyType string, restriction string) (err error) = SYS_KEYCTL
-//sys keyctlRestrictKeyring(cmd int, arg2 int) (err error) = SYS_KEYCTL
+//sys	keyctlRestrictKeyringByType(cmd int, arg2 int, keyType string, restriction string) (err error) = SYS_KEYCTL
+//sys	keyctlRestrictKeyring(cmd int, arg2 int) (err error) = SYS_KEYCTL
 
 func Recvmsg(fd int, p, oob []byte, flags int) (n, oobn int, recvflags int, from Sockaddr, err error) {
 	var msg Msghdr
@@ -1725,6 +1796,7 @@
 //sys	ClockGettime(clockid int32, time *Timespec) (err error)
 //sys	ClockNanosleep(clockid int32, flags int, request *Timespec, remain *Timespec) (err error)
 //sys	Close(fd int) (err error)
+//sys	CloseRange(first uint, last uint, flags uint) (err error)
 //sys	CopyFileRange(rfd int, roff *int64, wfd int, woff *int64, len int, flags int) (n int, err error)
 //sys	DeleteModule(name string, flags int) (err error)
 //sys	Dup(oldfd int) (fd int, err error)
@@ -1787,8 +1859,8 @@
 //sys	Nanosleep(time *Timespec, leftover *Timespec) (err error)
 //sys	PerfEventOpen(attr *PerfEventAttr, pid int, cpu int, groupFd int, flags int) (fd int, err error)
 //sys	PivotRoot(newroot string, putold string) (err error) = SYS_PIVOT_ROOT
-//sysnb prlimit(pid int, resource int, newlimit *Rlimit, old *Rlimit) (err error) = SYS_PRLIMIT64
-//sys   Prctl(option int, arg2 uintptr, arg3 uintptr, arg4 uintptr, arg5 uintptr) (err error)
+//sysnb	prlimit(pid int, resource int, newlimit *Rlimit, old *Rlimit) (err error) = SYS_PRLIMIT64
+//sys	Prctl(option int, arg2 uintptr, arg3 uintptr, arg4 uintptr, arg5 uintptr) (err error)
 //sys	Pselect(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timespec, sigmask *Sigset_t) (n int, err error) = SYS_PSELECT6
 //sys	read(fd int, p []byte) (n int, err error)
 //sys	Removexattr(path string, attr string) (err error)
@@ -1861,9 +1933,9 @@
 //sys	Syncfs(fd int) (err error)
 //sysnb	Sysinfo(info *Sysinfo_t) (err error)
 //sys	Tee(rfd int, wfd int, len int, flags int) (n int64, err error)
-//sysnb TimerfdCreate(clockid int, flags int) (fd int, err error)
-//sysnb TimerfdGettime(fd int, currValue *ItimerSpec) (err error)
-//sysnb TimerfdSettime(fd int, flags int, newValue *ItimerSpec, oldValue *ItimerSpec) (err error)
+//sysnb	TimerfdCreate(clockid int, flags int) (fd int, err error)
+//sysnb	TimerfdGettime(fd int, currValue *ItimerSpec) (err error)
+//sysnb	TimerfdSettime(fd int, flags int, newValue *ItimerSpec, oldValue *ItimerSpec) (err error)
 //sysnb	Tgkill(tgid int, tid int, sig syscall.Signal) (err error)
 //sysnb	Times(tms *Tms) (ticks uintptr, err error)
 //sysnb	Umask(mask int) (oldmask int)
@@ -2123,8 +2195,8 @@
 	return EACCES
 }
 
-//sys nameToHandleAt(dirFD int, pathname string, fh *fileHandle, mountID *_C_int, flags int) (err error) = SYS_NAME_TO_HANDLE_AT
-//sys openByHandleAt(mountFD int, fh *fileHandle, flags int) (fd int, err error) = SYS_OPEN_BY_HANDLE_AT
+//sys	nameToHandleAt(dirFD int, pathname string, fh *fileHandle, mountID *_C_int, flags int) (err error) = SYS_NAME_TO_HANDLE_AT
+//sys	openByHandleAt(mountFD int, fh *fileHandle, flags int) (fd int, err error) = SYS_OPEN_BY_HANDLE_AT
 
 // fileHandle is the argument to nameToHandleAt and openByHandleAt. We
 // originally tried to generate it via unix/linux/types.go with "type
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_386.go b/vendor/golang.org/x/sys/unix/syscall_linux_386.go
index 048d18e..b430536 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_386.go
@@ -2,9 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// TODO(rsc): Rewrite all nn(SP) references into name+(nn-8)(FP)
-// so that go vet can check that they are correct.
-
+//go:build 386 && linux
 // +build 386,linux
 
 package unix
@@ -34,7 +32,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -101,7 +99,7 @@
 	Max uint32
 }
 
-//sysnb getrlimit(resource int, rlim *rlimit32) (err error) = SYS_GETRLIMIT
+//sysnb	getrlimit(resource int, rlim *rlimit32) (err error) = SYS_GETRLIMIT
 
 const rlimInf32 = ^uint32(0)
 const rlimInf64 = ^uint64(0)
@@ -132,7 +130,7 @@
 	return
 }
 
-//sysnb setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
+//sysnb	setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
 
 func Setrlimit(resource int, rlim *Rlimit) (err error) {
 	err = prlimit(0, resource, rlim, nil)
@@ -380,6 +378,10 @@
 	cmsg.Len = uint32(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint32(length)
+}
+
 //sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_amd64.go b/vendor/golang.org/x/sys/unix/syscall_linux_amd64.go
index 72efe86..85cd97d 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && linux
 // +build amd64,linux
 
 package unix
@@ -138,7 +139,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -171,6 +172,10 @@
 	cmsg.Len = uint64(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
 //sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_amd64_gc.go b/vendor/golang.org/x/sys/unix/syscall_linux_amd64_gc.go
index 21a4946..8b0f0f3 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_amd64_gc.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_amd64_gc.go
@@ -2,8 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build amd64,linux
-// +build !gccgo
+//go:build amd64 && linux && gc
+// +build amd64,linux,gc
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_arm.go b/vendor/golang.org/x/sys/unix/syscall_linux_arm.go
index 496837b..39a864d 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_arm.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm && linux
 // +build arm,linux
 
 package unix
@@ -35,7 +36,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -129,8 +130,8 @@
 
 //sys	utimes(path string, times *[2]Timeval) (err error)
 
-//sys   Pread(fd int, p []byte, offset int64) (n int, err error) = SYS_PREAD64
-//sys   Pwrite(fd int, p []byte, offset int64) (n int, err error) = SYS_PWRITE64
+//sys	Pread(fd int, p []byte, offset int64) (n int, err error) = SYS_PREAD64
+//sys	Pwrite(fd int, p []byte, offset int64) (n int, err error) = SYS_PWRITE64
 //sys	Truncate(path string, length int64) (err error) = SYS_TRUNCATE64
 //sys	Ftruncate(fd int, length int64) (err error) = SYS_FTRUNCATE64
 
@@ -177,7 +178,7 @@
 	Max uint32
 }
 
-//sysnb getrlimit(resource int, rlim *rlimit32) (err error) = SYS_UGETRLIMIT
+//sysnb	getrlimit(resource int, rlim *rlimit32) (err error) = SYS_UGETRLIMIT
 
 const rlimInf32 = ^uint32(0)
 const rlimInf64 = ^uint64(0)
@@ -208,7 +209,7 @@
 	return
 }
 
-//sysnb setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
+//sysnb	setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
 
 func Setrlimit(resource int, rlim *Rlimit) (err error) {
 	err = prlimit(0, resource, rlim, nil)
@@ -255,6 +256,10 @@
 	cmsg.Len = uint32(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint32(length)
+}
+
 //sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_arm64.go b/vendor/golang.org/x/sys/unix/syscall_linux_arm64.go
index c6de6b9..7f27ebf 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_arm64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_arm64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm64 && linux
 // +build arm64,linux
 
 package unix
@@ -155,7 +156,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -206,6 +207,10 @@
 	cmsg.Len = uint64(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
 func InotifyInit() (fd int, err error) {
 	return InotifyInit1(0)
 }
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_gc.go b/vendor/golang.org/x/sys/unix/syscall_linux_gc.go
index c26e6ec..2b1168d 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_gc.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_gc.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build linux,!gccgo
+//go:build linux && gc
+// +build linux,gc
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_gc_386.go b/vendor/golang.org/x/sys/unix/syscall_linux_gc_386.go
index 070bd38..9843fb4 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_gc_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_gc_386.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build linux,!gccgo,386
+//go:build linux && gc && 386
+// +build linux,gc,386
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_gc_arm.go b/vendor/golang.org/x/sys/unix/syscall_linux_gc_arm.go
index 8c514c9..a6008fc 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_gc_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_gc_arm.go
@@ -2,7 +2,8 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// +build arm,!gccgo,linux
+//go:build arm && gc && linux
+// +build arm,gc,linux
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_386.go b/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_386.go
index 308eb7a..7740af2 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_386.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && gccgo && 386
 // +build linux,gccgo,386
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_arm.go b/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_arm.go
index aa7fc9e..e16a122 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_gccgo_arm.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && gccgo && arm
 // +build linux,gccgo,arm
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_mips64x.go b/vendor/golang.org/x/sys/unix/syscall_linux_mips64x.go
index f028747..27aee81 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_mips64x.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_mips64x.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (mips64 || mips64le)
 // +build linux
 // +build mips64 mips64le
 
@@ -104,7 +105,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -216,6 +217,10 @@
 	cmsg.Len = uint64(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
 func InotifyInit() (fd int, err error) {
 	return InotifyInit1(0)
 }
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_mipsx.go b/vendor/golang.org/x/sys/unix/syscall_linux_mipsx.go
index c113281..3a5621e 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_mipsx.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_mipsx.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (mips || mipsle)
 // +build linux
 // +build mips mipsle
 
@@ -112,7 +113,7 @@
 	return Timeval{Sec: int32(sec), Usec: int32(usec)}
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -125,7 +126,7 @@
 	return
 }
 
-//sysnb pipe() (p1 int, p2 int, err error)
+//sysnb	pipe() (p1 int, p2 int, err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
@@ -153,7 +154,7 @@
 	Max uint32
 }
 
-//sysnb getrlimit(resource int, rlim *rlimit32) (err error) = SYS_GETRLIMIT
+//sysnb	getrlimit(resource int, rlim *rlimit32) (err error) = SYS_GETRLIMIT
 
 func Getrlimit(resource int, rlim *Rlimit) (err error) {
 	err = prlimit(0, resource, nil, rlim)
@@ -181,7 +182,7 @@
 	return
 }
 
-//sysnb setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
+//sysnb	setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
 
 func Setrlimit(resource int, rlim *Rlimit) (err error) {
 	err = prlimit(0, resource, rlim, nil)
@@ -228,6 +229,10 @@
 	cmsg.Len = uint32(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint32(length)
+}
+
 //sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_ppc.go b/vendor/golang.org/x/sys/unix/syscall_linux_ppc.go
new file mode 100644
index 0000000..cf0d36f
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_ppc.go
@@ -0,0 +1,276 @@
+// Copyright 2021 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.
+
+//go:build linux && ppc
+// +build linux
+// +build ppc
+
+package unix
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+//sys	dup2(oldfd int, newfd int) (err error)
+//sysnb	EpollCreate(size int) (fd int, err error)
+//sys	EpollWait(epfd int, events []EpollEvent, msec int) (n int, err error)
+//sys	Fchown(fd int, uid int, gid int) (err error)
+//sys	Fstat(fd int, stat *Stat_t) (err error) = SYS_FSTAT64
+//sys	Fstatat(dirfd int, path string, stat *Stat_t, flags int) (err error) = SYS_FSTATAT64
+//sys	Ftruncate(fd int, length int64) (err error) = SYS_FTRUNCATE64
+//sysnb	Getegid() (egid int)
+//sysnb	Geteuid() (euid int)
+//sysnb	Getgid() (gid int)
+//sysnb	Getuid() (uid int)
+//sysnb	InotifyInit() (fd int, err error)
+//sys	Ioperm(from int, num int, on int) (err error)
+//sys	Iopl(level int) (err error)
+//sys	Lchown(path string, uid int, gid int) (err error)
+//sys	Listen(s int, n int) (err error)
+//sys	Lstat(path string, stat *Stat_t) (err error) = SYS_LSTAT64
+//sys	Pause() (err error)
+//sys	Pread(fd int, p []byte, offset int64) (n int, err error) = SYS_PREAD64
+//sys	Pwrite(fd int, p []byte, offset int64) (n int, err error) = SYS_PWRITE64
+//sys	Renameat(olddirfd int, oldpath string, newdirfd int, newpath string) (err error)
+//sys	Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) = SYS__NEWSELECT
+//sys	sendfile(outfd int, infd int, offset *int64, count int) (written int, err error) = SYS_SENDFILE64
+//sys	setfsgid(gid int) (prev int, err error)
+//sys	setfsuid(uid int) (prev int, err error)
+//sysnb	Setregid(rgid int, egid int) (err error)
+//sysnb	Setresgid(rgid int, egid int, sgid int) (err error)
+//sysnb	Setresuid(ruid int, euid int, suid int) (err error)
+//sysnb	Setreuid(ruid int, euid int) (err error)
+//sys	Shutdown(fd int, how int) (err error)
+//sys	Splice(rfd int, roff *int64, wfd int, woff *int64, len int, flags int) (n int, err error)
+//sys	Stat(path string, stat *Stat_t) (err error) = SYS_STAT64
+//sys	Truncate(path string, length int64) (err error) = SYS_TRUNCATE64
+//sys	Ustat(dev int, ubuf *Ustat_t) (err error)
+//sys	accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error)
+//sys	accept4(s int, rsa *RawSockaddrAny, addrlen *_Socklen, flags int) (fd int, err error)
+//sys	bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error)
+//sys	connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error)
+//sysnb	getgroups(n int, list *_Gid_t) (nn int, err error)
+//sysnb	setgroups(n int, list *_Gid_t) (err error)
+//sys	getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error)
+//sys	setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error)
+//sysnb	socket(domain int, typ int, proto int) (fd int, err error)
+//sysnb	socketpair(domain int, typ int, proto int, fd *[2]int32) (err error)
+//sysnb	getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error)
+//sysnb	getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error)
+//sys	recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error)
+//sys	sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error)
+//sys	recvmsg(s int, msg *Msghdr, flags int) (n int, err error)
+//sys	sendmsg(s int, msg *Msghdr, flags int) (n int, err error)
+
+//sys	futimesat(dirfd int, path string, times *[2]Timeval) (err error)
+//sysnb	Gettimeofday(tv *Timeval) (err error)
+//sysnb	Time(t *Time_t) (tt Time_t, err error)
+//sys	Utime(path string, buf *Utimbuf) (err error)
+//sys	utimes(path string, times *[2]Timeval) (err error)
+
+func Fadvise(fd int, offset int64, length int64, advice int) (err error) {
+	_, _, e1 := Syscall6(SYS_FADVISE64_64, uintptr(fd), uintptr(advice), uintptr(offset>>32), uintptr(offset), uintptr(length>>32), uintptr(length))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func seek(fd int, offset int64, whence int) (int64, syscall.Errno) {
+	var newoffset int64
+	offsetLow := uint32(offset & 0xffffffff)
+	offsetHigh := uint32((offset >> 32) & 0xffffffff)
+	_, _, err := Syscall6(SYS__LLSEEK, uintptr(fd), uintptr(offsetHigh), uintptr(offsetLow), uintptr(unsafe.Pointer(&newoffset)), uintptr(whence), 0)
+	return newoffset, err
+}
+
+func Seek(fd int, offset int64, whence int) (newoffset int64, err error) {
+	newoffset, errno := seek(fd, offset, whence)
+	if errno != 0 {
+		return 0, errno
+	}
+	return newoffset, nil
+}
+
+func Fstatfs(fd int, buf *Statfs_t) (err error) {
+	_, _, e := Syscall(SYS_FSTATFS64, uintptr(fd), unsafe.Sizeof(*buf), uintptr(unsafe.Pointer(buf)))
+	if e != 0 {
+		err = e
+	}
+	return
+}
+
+func Statfs(path string, buf *Statfs_t) (err error) {
+	pathp, err := BytePtrFromString(path)
+	if err != nil {
+		return err
+	}
+	_, _, e := Syscall(SYS_STATFS64, uintptr(unsafe.Pointer(pathp)), unsafe.Sizeof(*buf), uintptr(unsafe.Pointer(buf)))
+	if e != 0 {
+		err = e
+	}
+	return
+}
+
+//sys	mmap2(addr uintptr, length uintptr, prot int, flags int, fd int, pageOffset uintptr) (xaddr uintptr, err error)
+
+func mmap(addr uintptr, length uintptr, prot int, flags int, fd int, offset int64) (xaddr uintptr, err error) {
+	page := uintptr(offset / 4096)
+	if offset != int64(page)*4096 {
+		return 0, EINVAL
+	}
+	return mmap2(addr, length, prot, flags, fd, page)
+}
+
+func setTimespec(sec, nsec int64) Timespec {
+	return Timespec{Sec: int32(sec), Nsec: int32(nsec)}
+}
+
+func setTimeval(sec, usec int64) Timeval {
+	return Timeval{Sec: int32(sec), Usec: int32(usec)}
+}
+
+type rlimit32 struct {
+	Cur uint32
+	Max uint32
+}
+
+//sysnb	getrlimit(resource int, rlim *rlimit32) (err error) = SYS_UGETRLIMIT
+
+const rlimInf32 = ^uint32(0)
+const rlimInf64 = ^uint64(0)
+
+func Getrlimit(resource int, rlim *Rlimit) (err error) {
+	err = prlimit(0, resource, nil, rlim)
+	if err != ENOSYS {
+		return err
+	}
+
+	rl := rlimit32{}
+	err = getrlimit(resource, &rl)
+	if err != nil {
+		return
+	}
+
+	if rl.Cur == rlimInf32 {
+		rlim.Cur = rlimInf64
+	} else {
+		rlim.Cur = uint64(rl.Cur)
+	}
+
+	if rl.Max == rlimInf32 {
+		rlim.Max = rlimInf64
+	} else {
+		rlim.Max = uint64(rl.Max)
+	}
+	return
+}
+
+//sysnb	setrlimit(resource int, rlim *rlimit32) (err error) = SYS_SETRLIMIT
+
+func Setrlimit(resource int, rlim *Rlimit) (err error) {
+	err = prlimit(0, resource, rlim, nil)
+	if err != ENOSYS {
+		return err
+	}
+
+	rl := rlimit32{}
+	if rlim.Cur == rlimInf64 {
+		rl.Cur = rlimInf32
+	} else if rlim.Cur < uint64(rlimInf32) {
+		rl.Cur = uint32(rlim.Cur)
+	} else {
+		return EINVAL
+	}
+	if rlim.Max == rlimInf64 {
+		rl.Max = rlimInf32
+	} else if rlim.Max < uint64(rlimInf32) {
+		rl.Max = uint32(rlim.Max)
+	} else {
+		return EINVAL
+	}
+
+	return setrlimit(resource, &rl)
+}
+
+func (r *PtraceRegs) PC() uint32 { return r.Nip }
+
+func (r *PtraceRegs) SetPC(pc uint32) { r.Nip = pc }
+
+func (iov *Iovec) SetLen(length int) {
+	iov.Len = uint32(length)
+}
+
+func (msghdr *Msghdr) SetControllen(length int) {
+	msghdr.Controllen = uint32(length)
+}
+
+func (msghdr *Msghdr) SetIovlen(length int) {
+	msghdr.Iovlen = uint32(length)
+}
+
+func (cmsg *Cmsghdr) SetLen(length int) {
+	cmsg.Len = uint32(length)
+}
+
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint32(length)
+}
+
+//sysnb	pipe(p *[2]_C_int) (err error)
+
+func Pipe(p []int) (err error) {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	err = pipe(&pp)
+	p[0] = int(pp[0])
+	p[1] = int(pp[1])
+	return
+}
+
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
+
+func Pipe2(p []int, flags int) (err error) {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	err = pipe2(&pp, flags)
+	p[0] = int(pp[0])
+	p[1] = int(pp[1])
+	return
+}
+
+//sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
+
+func Poll(fds []PollFd, timeout int) (n int, err error) {
+	if len(fds) == 0 {
+		return poll(nil, 0, timeout)
+	}
+	return poll(&fds[0], len(fds), timeout)
+}
+
+//sys	syncFileRange2(fd int, flags int, off int64, n int64) (err error) = SYS_SYNC_FILE_RANGE2
+
+func SyncFileRange(fd int, off int64, n int64, flags int) error {
+	// The sync_file_range and sync_file_range2 syscalls differ only in the
+	// order of their arguments.
+	return syncFileRange2(fd, flags, off, n)
+}
+
+//sys	kexecFileLoad(kernelFd int, initrdFd int, cmdlineLen int, cmdline string, flags int) (err error)
+
+func KexecFileLoad(kernelFd int, initrdFd int, cmdline string, flags int) error {
+	cmdlineLen := len(cmdline)
+	if cmdlineLen > 0 {
+		// Account for the additional NULL byte added by
+		// BytePtrFromString in kexecFileLoad. The kexec_file_load
+		// syscall expects a NULL-terminated string.
+		cmdlineLen++
+	}
+	return kexecFileLoad(kernelFd, initrdFd, cmdlineLen, cmdline, flags)
+}
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_ppc64x.go b/vendor/golang.org/x/sys/unix/syscall_linux_ppc64x.go
index 3493744..5259a5f 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_ppc64x.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_ppc64x.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (ppc64 || ppc64le)
 // +build linux
 // +build ppc64 ppc64le
 
@@ -99,7 +100,11 @@
 	cmsg.Len = uint64(length)
 }
 
-//sysnb pipe(p *[2]_C_int) (err error)
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
+//sysnb	pipe(p *[2]_C_int) (err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
@@ -112,7 +117,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_riscv64.go b/vendor/golang.org/x/sys/unix/syscall_linux_riscv64.go
index b0b1505..8ef821e 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_riscv64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_riscv64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build riscv64 && linux
 // +build riscv64,linux
 
 package unix
@@ -154,7 +155,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
@@ -187,6 +188,10 @@
 	cmsg.Len = uint64(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
 func InotifyInit() (fd int, err error) {
 	return InotifyInit1(0)
 }
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_s390x.go b/vendor/golang.org/x/sys/unix/syscall_linux_s390x.go
index 2363f74..a1c0574 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_s390x.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_s390x.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build s390x && linux
 // +build s390x,linux
 
 package unix
@@ -76,7 +77,7 @@
 	return Timeval{Sec: sec, Usec: usec}
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
@@ -128,6 +129,10 @@
 	cmsg.Len = uint64(length)
 }
 
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
 // Linux on s390x uses the old mmap interface, which requires arguments to be passed in a struct.
 // mmap2 also requires arguments to be passed in a struct; it is currently not exposed in <asm/unistd.h>.
 func mmap(addr uintptr, length uintptr, prot int, flags int, fd int, offset int64) (xaddr uintptr, err error) {
@@ -249,7 +254,7 @@
 }
 
 func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) error {
-	args := [4]uintptr{uintptr(s), uintptr(level), uintptr(name), uintptr(val)}
+	args := [5]uintptr{uintptr(s), uintptr(level), uintptr(name), uintptr(val), vallen}
 	_, _, err := Syscall(SYS_SOCKETCALL, netSetSockOpt, uintptr(unsafe.Pointer(&args)), 0)
 	if err != 0 {
 		return err
diff --git a/vendor/golang.org/x/sys/unix/syscall_linux_sparc64.go b/vendor/golang.org/x/sys/unix/syscall_linux_sparc64.go
index d389f15..de14b88 100644
--- a/vendor/golang.org/x/sys/unix/syscall_linux_sparc64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_linux_sparc64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build sparc64 && linux
 // +build sparc64,linux
 
 package unix
@@ -115,7 +116,11 @@
 	cmsg.Len = uint64(length)
 }
 
-//sysnb pipe(p *[2]_C_int) (err error)
+func (rsa *RawSockaddrNFCLLCP) SetServiceNameLen(length int) {
+	rsa.Service_name_len = uint64(length)
+}
+
+//sysnb	pipe(p *[2]_C_int) (err error)
 
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
@@ -128,7 +133,7 @@
 	return
 }
 
-//sysnb pipe2(p *[2]_C_int, flags int) (err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
 
 func Pipe2(p []int, flags int) (err error) {
 	if len(p) != 2 {
diff --git a/vendor/golang.org/x/sys/unix/syscall_netbsd.go b/vendor/golang.org/x/sys/unix/syscall_netbsd.go
index dbd5e03..853d5f0 100644
--- a/vendor/golang.org/x/sys/unix/syscall_netbsd.go
+++ b/vendor/golang.org/x/sys/unix/syscall_netbsd.go
@@ -31,6 +31,10 @@
 	raw    RawSockaddrDatalink
 }
 
+func anyToSockaddrGOOS(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	return nil, EAFNOSUPPORT
+}
+
 func Syscall9(trap, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err syscall.Errno)
 
 func sysctlNodes(mib []_C_int) (nodes []Sysctlnode, err error) {
@@ -106,7 +110,8 @@
 	return readInt(buf, unsafe.Offsetof(Dirent{}.Namlen), unsafe.Sizeof(Dirent{}.Namlen))
 }
 
-//sysnb pipe() (fd1 int, fd2 int, err error)
+//sysnb	pipe() (fd1 int, fd2 int, err error)
+
 func Pipe(p []int) (err error) {
 	if len(p) != 2 {
 		return EINVAL
@@ -115,7 +120,21 @@
 	return
 }
 
-//sys Getdents(fd int, buf []byte) (n int, err error)
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
+
+func Pipe2(p []int, flags int) error {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	err := pipe2(&pp, flags)
+	p[0] = int(pp[0])
+	p[1] = int(pp[1])
+	return err
+}
+
+//sys	Getdents(fd int, buf []byte) (n int, err error)
+
 func Getdirentries(fd int, buf []byte, basep *uintptr) (n int, err error) {
 	n, err = Getdents(fd, buf)
 	if err != nil || basep == nil {
@@ -155,7 +174,7 @@
 
 //sys	ioctl(fd int, req uint, arg uintptr) (err error)
 
-//sys   sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
+//sys	sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
 
 func IoctlGetPtmget(fd int, req uint) (*Ptmget, error) {
 	var value Ptmget
diff --git a/vendor/golang.org/x/sys/unix/syscall_netbsd_386.go b/vendor/golang.org/x/sys/unix/syscall_netbsd_386.go
index 24da8b5..5199d28 100644
--- a/vendor/golang.org/x/sys/unix/syscall_netbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_netbsd_386.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build 386 && netbsd
 // +build 386,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_netbsd_amd64.go b/vendor/golang.org/x/sys/unix/syscall_netbsd_amd64.go
index 25a0ac8..70a9c52 100644
--- a/vendor/golang.org/x/sys/unix/syscall_netbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_netbsd_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && netbsd
 // +build amd64,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_netbsd_arm.go b/vendor/golang.org/x/sys/unix/syscall_netbsd_arm.go
index 21591ec..3eb5942 100644
--- a/vendor/golang.org/x/sys/unix/syscall_netbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_netbsd_arm.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm && netbsd
 // +build arm,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_netbsd_arm64.go b/vendor/golang.org/x/sys/unix/syscall_netbsd_arm64.go
index 8047496..fc6ccfd 100644
--- a/vendor/golang.org/x/sys/unix/syscall_netbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_netbsd_arm64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm64 && netbsd
 // +build arm64,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd.go b/vendor/golang.org/x/sys/unix/syscall_openbsd.go
index 2c1f46e..22b5503 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd.go
@@ -31,6 +31,10 @@
 	raw    RawSockaddrDatalink
 }
 
+func anyToSockaddrGOOS(fd int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	return nil, EAFNOSUPPORT
+}
+
 func Syscall9(trap, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err syscall.Errno)
 
 func nametomib(name string) (mib []_C_int, err error) {
@@ -88,7 +92,7 @@
 	return err
 }
 
-//sys Getdents(fd int, buf []byte) (n int, err error)
+//sys	Getdents(fd int, buf []byte) (n int, err error)
 func Getdirentries(fd int, buf []byte, basep *uintptr) (n int, err error) {
 	n, err = Getdents(fd, buf)
 	if err != nil || basep == nil {
@@ -150,7 +154,7 @@
 
 //sys	ioctl(fd int, req uint, arg uintptr) (err error)
 
-//sys   sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
+//sys	sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) = SYS___SYSCTL
 
 //sys	ppoll(fds *PollFd, nfds int, timeout *Timespec, sigmask *Sigset_t) (n int, err error)
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd_386.go b/vendor/golang.org/x/sys/unix/syscall_openbsd_386.go
index 42b5a0e..6baabcd 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd_386.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build 386 && openbsd
 // +build 386,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/syscall_openbsd_amd64.go
index 6ea4b48..bab2536 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && openbsd
 // +build amd64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd_arm.go b/vendor/golang.org/x/sys/unix/syscall_openbsd_arm.go
index 1c3d26f..8eed3c4 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd_arm.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm && openbsd
 // +build arm,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/syscall_openbsd_arm64.go
index a8c458c..483dde9 100644
--- a/vendor/golang.org/x/sys/unix/syscall_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_openbsd_arm64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build arm64 && openbsd
 // +build arm64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_solaris.go b/vendor/golang.org/x/sys/unix/syscall_solaris.go
index 0e2a696..77fcde7 100644
--- a/vendor/golang.org/x/sys/unix/syscall_solaris.go
+++ b/vendor/golang.org/x/sys/unix/syscall_solaris.go
@@ -13,6 +13,7 @@
 package unix
 
 import (
+	"runtime"
 	"syscall"
 	"unsafe"
 )
@@ -67,6 +68,19 @@
 	return nil
 }
 
+//sysnb	pipe2(p *[2]_C_int, flags int) (err error)
+
+func Pipe2(p []int, flags int) error {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	err := pipe2(&pp, flags)
+	p[0] = int(pp[0])
+	p[1] = int(pp[1])
+	return err
+}
+
 func (sa *SockaddrInet4) sockaddr() (unsafe.Pointer, _Socklen, error) {
 	if sa.Port < 0 || sa.Port > 0xFFFF {
 		return nil, 0, EINVAL
@@ -551,10 +565,17 @@
  * Expose the ioctl function
  */
 
-//sys	ioctl(fd int, req uint, arg uintptr) (err error)
+//sys	ioctlRet(fd int, req uint, arg uintptr) (ret int, err error) = libc.ioctl
 
-func IoctlSetTermio(fd int, req uint, value *Termio) (err error) {
-	return ioctl(fd, req, uintptr(unsafe.Pointer(value)))
+func ioctl(fd int, req uint, arg uintptr) (err error) {
+	_, err = ioctlRet(fd, req, arg)
+	return err
+}
+
+func IoctlSetTermio(fd int, req uint, value *Termio) error {
+	err := ioctl(fd, req, uintptr(unsafe.Pointer(value)))
+	runtime.KeepAlive(value)
+	return err
 }
 
 func IoctlGetTermio(fd int, req uint) (*Termio, error) {
@@ -563,7 +584,7 @@
 	return &value, err
 }
 
-//sys   poll(fds *PollFd, nfds int, timeout int) (n int, err error)
+//sys	poll(fds *PollFd, nfds int, timeout int) (n int, err error)
 
 func Poll(fds []PollFd, timeout int) (n int, err error) {
 	if len(fds) == 0 {
@@ -666,6 +687,7 @@
 //sys	Statvfs(path string, vfsstat *Statvfs_t) (err error)
 //sys	Symlink(path string, link string) (err error)
 //sys	Sync() (err error)
+//sys	Sysconf(which int) (n int64, err error)
 //sysnb	Times(tms *Tms) (ticks uintptr, err error)
 //sys	Truncate(path string, length int64) (err error)
 //sys	Fsync(fd int) (err error)
diff --git a/vendor/golang.org/x/sys/unix/syscall_solaris_amd64.go b/vendor/golang.org/x/sys/unix/syscall_solaris_amd64.go
index b22a34d..0bd25ef 100644
--- a/vendor/golang.org/x/sys/unix/syscall_solaris_amd64.go
+++ b/vendor/golang.org/x/sys/unix/syscall_solaris_amd64.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build amd64 && solaris
 // +build amd64,solaris
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_unix.go b/vendor/golang.org/x/sys/unix/syscall_unix.go
index 400ba9f..a7618ce 100644
--- a/vendor/golang.org/x/sys/unix/syscall_unix.go
+++ b/vendor/golang.org/x/sys/unix/syscall_unix.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris
 // +build aix darwin dragonfly freebsd linux netbsd openbsd solaris
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/syscall_unix_gc.go b/vendor/golang.org/x/sys/unix/syscall_unix_gc.go
index 1c70d1b..5898e9a 100644
--- a/vendor/golang.org/x/sys/unix/syscall_unix_gc.go
+++ b/vendor/golang.org/x/sys/unix/syscall_unix_gc.go
@@ -2,8 +2,11 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build (darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris) && gc && !ppc64le && !ppc64
 // +build darwin dragonfly freebsd linux netbsd openbsd solaris
-// +build !gccgo,!ppc64le,!ppc64
+// +build gc
+// +build !ppc64le
+// +build !ppc64
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_unix_gc_ppc64x.go b/vendor/golang.org/x/sys/unix/syscall_unix_gc_ppc64x.go
index 86dc765..f6f707a 100644
--- a/vendor/golang.org/x/sys/unix/syscall_unix_gc_ppc64x.go
+++ b/vendor/golang.org/x/sys/unix/syscall_unix_gc_ppc64x.go
@@ -2,9 +2,10 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build linux && (ppc64le || ppc64) && gc
 // +build linux
 // +build ppc64le ppc64
-// +build !gccgo
+// +build gc
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/syscall_zos_s390x.go b/vendor/golang.org/x/sys/unix/syscall_zos_s390x.go
new file mode 100644
index 0000000..1ffd8bf
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/syscall_zos_s390x.go
@@ -0,0 +1,1829 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+import (
+	"bytes"
+	"runtime"
+	"sort"
+	"sync"
+	"syscall"
+	"unsafe"
+)
+
+const (
+	O_CLOEXEC = 0       // Dummy value (not supported).
+	AF_LOCAL  = AF_UNIX // AF_LOCAL is an alias for AF_UNIX
+)
+
+func syscall_syscall(trap, a1, a2, a3 uintptr) (r1, r2 uintptr, err Errno)
+func syscall_rawsyscall(trap, a1, a2, a3 uintptr) (r1, r2 uintptr, err Errno)
+func syscall_syscall6(trap, a1, a2, a3, a4, a5, a6 uintptr) (r1, r2 uintptr, err Errno)
+func syscall_rawsyscall6(trap, a1, a2, a3, a4, a5, a6 uintptr) (r1, r2 uintptr, err Errno)
+func syscall_syscall9(trap, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err Errno)
+func syscall_rawsyscall9(trap, a1, a2, a3, a4, a5, a6, a7, a8, a9 uintptr) (r1, r2 uintptr, err Errno)
+
+func copyStat(stat *Stat_t, statLE *Stat_LE_t) {
+	stat.Dev = uint64(statLE.Dev)
+	stat.Ino = uint64(statLE.Ino)
+	stat.Nlink = uint64(statLE.Nlink)
+	stat.Mode = uint32(statLE.Mode)
+	stat.Uid = uint32(statLE.Uid)
+	stat.Gid = uint32(statLE.Gid)
+	stat.Rdev = uint64(statLE.Rdev)
+	stat.Size = statLE.Size
+	stat.Atim.Sec = int64(statLE.Atim)
+	stat.Atim.Nsec = 0 //zos doesn't return nanoseconds
+	stat.Mtim.Sec = int64(statLE.Mtim)
+	stat.Mtim.Nsec = 0 //zos doesn't return nanoseconds
+	stat.Ctim.Sec = int64(statLE.Ctim)
+	stat.Ctim.Nsec = 0 //zos doesn't return nanoseconds
+	stat.Blksize = int64(statLE.Blksize)
+	stat.Blocks = statLE.Blocks
+}
+
+func svcCall(fnptr unsafe.Pointer, argv *unsafe.Pointer, dsa *uint64)
+func svcLoad(name *byte) unsafe.Pointer
+func svcUnload(name *byte, fnptr unsafe.Pointer) int64
+
+func (d *Dirent) NameString() string {
+	if d == nil {
+		return ""
+	}
+	return string(d.Name[:d.Namlen])
+}
+
+func (sa *SockaddrInet4) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	if sa.Port < 0 || sa.Port > 0xFFFF {
+		return nil, 0, EINVAL
+	}
+	sa.raw.Len = SizeofSockaddrInet4
+	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), _Socklen(sa.raw.Len), nil
+}
+
+func (sa *SockaddrInet6) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	if sa.Port < 0 || sa.Port > 0xFFFF {
+		return nil, 0, EINVAL
+	}
+	sa.raw.Len = SizeofSockaddrInet6
+	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), _Socklen(sa.raw.Len), nil
+}
+
+func (sa *SockaddrUnix) sockaddr() (unsafe.Pointer, _Socklen, error) {
+	name := sa.Name
+	n := len(name)
+	if n >= len(sa.raw.Path) || n == 0 {
+		return nil, 0, EINVAL
+	}
+	sa.raw.Len = byte(3 + n) // 2 for Family, Len; 1 for NUL
+	sa.raw.Family = AF_UNIX
+	for i := 0; i < n; i++ {
+		sa.raw.Path[i] = int8(name[i])
+	}
+	return unsafe.Pointer(&sa.raw), _Socklen(sa.raw.Len), nil
+}
+
+func anyToSockaddr(_ int, rsa *RawSockaddrAny) (Sockaddr, error) {
+	// TODO(neeilan): Implement use of first param (fd)
+	switch rsa.Addr.Family {
+	case AF_UNIX:
+		pp := (*RawSockaddrUnix)(unsafe.Pointer(rsa))
+		sa := new(SockaddrUnix)
+		// For z/OS, only replace NUL with @ when the
+		// length is not zero.
+		if pp.Len != 0 && 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.
+		//
+		// For z/OS, the length of the name is a field
+		// in the structure. To be on the safe side, we
+		// will still scan the name for a NUL but only
+		// to the length provided in the structure.
+		//
+		// 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 < int(pp.Len) && pp.Path[n] != 0 {
+			n++
+		}
+		bytes := (*[len(pp.Path)]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, EAFNOSUPPORT
+}
+
+func Accept(fd int) (nfd int, sa Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	var len _Socklen = SizeofSockaddrAny
+	nfd, err = accept(fd, &rsa, &len)
+	if err != nil {
+		return
+	}
+	// TODO(neeilan): Remove 0 in call
+	sa, err = anyToSockaddr(0, &rsa)
+	if err != nil {
+		Close(nfd)
+		nfd = 0
+	}
+	return
+}
+
+func (iov *Iovec) SetLen(length int) {
+	iov.Len = uint64(length)
+}
+
+func (msghdr *Msghdr) SetControllen(length int) {
+	msghdr.Controllen = int32(length)
+}
+
+func (cmsg *Cmsghdr) SetLen(length int) {
+	cmsg.Len = int32(length)
+}
+
+//sys   fcntl(fd int, cmd int, arg int) (val int, err error)
+//sys	read(fd int, p []byte) (n int, err error)
+//sys   readlen(fd int, buf *byte, nbuf int) (n int, err error) = SYS_READ
+//sys	write(fd int, p []byte) (n int, err error)
+
+//sys	accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) = SYS___ACCEPT_A
+//sys	bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) = SYS___BIND_A
+//sys	connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) = SYS___CONNECT_A
+//sysnb	getgroups(n int, list *_Gid_t) (nn int, err error)
+//sysnb	setgroups(n int, list *_Gid_t) (err error)
+//sys	getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error)
+//sys	setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error)
+//sysnb	socket(domain int, typ int, proto int) (fd int, err error)
+//sysnb	socketpair(domain int, typ int, proto int, fd *[2]int32) (err error)
+//sysnb	getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) = SYS___GETPEERNAME_A
+//sysnb	getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) = SYS___GETSOCKNAME_A
+//sys	recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error) = SYS___RECVFROM_A
+//sys	sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error) = SYS___SENDTO_A
+//sys	recvmsg(s int, msg *Msghdr, flags int) (n int, err error) = SYS___RECVMSG_A
+//sys	sendmsg(s int, msg *Msghdr, flags int) (n int, err error) = SYS___SENDMSG_A
+//sys   mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) = SYS_MMAP
+//sys   munmap(addr uintptr, length uintptr) (err error) = SYS_MUNMAP
+//sys   ioctl(fd int, req uint, arg uintptr) (err error) = SYS_IOCTL
+
+//sys   Access(path string, mode uint32) (err error) = SYS___ACCESS_A
+//sys   Chdir(path string) (err error) = SYS___CHDIR_A
+//sys	Chown(path string, uid int, gid int) (err error) = SYS___CHOWN_A
+//sys	Chmod(path string, mode uint32) (err error) = SYS___CHMOD_A
+//sys   Creat(path string, mode uint32) (fd int, err error) = SYS___CREAT_A
+//sys	Dup(oldfd int) (fd int, err error)
+//sys	Dup2(oldfd int, newfd int) (err error)
+//sys	Errno2() (er2 int) = SYS___ERRNO2
+//sys	Err2ad() (eadd *int) = SYS___ERR2AD
+//sys	Exit(code int)
+//sys	Fchdir(fd int) (err error)
+//sys	Fchmod(fd int, mode uint32) (err error)
+//sys	Fchown(fd int, uid int, gid int) (err error)
+//sys	FcntlInt(fd uintptr, cmd int, arg int) (retval int, err error) = SYS_FCNTL
+//sys	fstat(fd int, stat *Stat_LE_t) (err error)
+
+func Fstat(fd int, stat *Stat_t) (err error) {
+	var statLE Stat_LE_t
+	err = fstat(fd, &statLE)
+	copyStat(stat, &statLE)
+	return
+}
+
+//sys	Fstatvfs(fd int, stat *Statvfs_t) (err error) = SYS_FSTATVFS
+//sys	Fsync(fd int) (err error)
+//sys	Ftruncate(fd int, length int64) (err error)
+//sys   Getpagesize() (pgsize int) = SYS_GETPAGESIZE
+//sys   Mprotect(b []byte, prot int) (err error) = SYS_MPROTECT
+//sys   Msync(b []byte, flags int) (err error) = SYS_MSYNC
+//sys   Poll(fds []PollFd, timeout int) (n int, err error) = SYS_POLL
+//sys   Times(tms *Tms) (ticks uintptr, err error) = SYS_TIMES
+//sys   W_Getmntent(buff *byte, size int) (lastsys int, err error) = SYS_W_GETMNTENT
+//sys   W_Getmntent_A(buff *byte, size int) (lastsys int, err error) = SYS___W_GETMNTENT_A
+
+//sys   mount_LE(path string, filesystem string, fstype string, mtm uint32, parmlen int32, parm string) (err error) = SYS___MOUNT_A
+//sys   unmount(filesystem string, mtm int) (err error) = SYS___UMOUNT_A
+//sys   Chroot(path string) (err error) = SYS___CHROOT_A
+//sys   Select(nmsgsfds int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (ret int, err error) = SYS_SELECT
+//sysnb Uname(buf *Utsname) (err error) = SYS___UNAME_A
+
+func Ptsname(fd int) (name string, err error) {
+	r0, _, e1 := syscall_syscall(SYS___PTSNAME_A, uintptr(fd), 0, 0)
+	name = u2s(unsafe.Pointer(r0))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func u2s(cstr unsafe.Pointer) string {
+	str := (*[1024]uint8)(cstr)
+	i := 0
+	for str[i] != 0 {
+		i++
+	}
+	return string(str[:i])
+}
+
+func Close(fd int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_CLOSE, uintptr(fd), 0, 0)
+	for i := 0; e1 == EAGAIN && i < 10; i++ {
+		_, _, _ = syscall_syscall(SYS_USLEEP, uintptr(10), 0, 0)
+		_, _, e1 = syscall_syscall(SYS_CLOSE, uintptr(fd), 0, 0)
+	}
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+var mapper = &mmapper{
+	active: make(map[*byte][]byte),
+	mmap:   mmap,
+	munmap: munmap,
+}
+
+// Dummy function: there are no semantics for Madvise on z/OS
+func Madvise(b []byte, advice int) (err error) {
+	return
+}
+
+func Mmap(fd int, offset int64, length int, prot int, flags int) (data []byte, err error) {
+	return mapper.Mmap(fd, offset, length, prot, flags)
+}
+
+func Munmap(b []byte) (err error) {
+	return mapper.Munmap(b)
+}
+
+//sys   Gethostname(buf []byte) (err error) = SYS___GETHOSTNAME_A
+//sysnb	Getegid() (egid int)
+//sysnb	Geteuid() (uid int)
+//sysnb	Getgid() (gid int)
+//sysnb	Getpid() (pid int)
+//sysnb	Getpgid(pid int) (pgid int, err error) = SYS_GETPGID
+
+func Getpgrp() (pid int) {
+	pid, _ = Getpgid(0)
+	return
+}
+
+//sysnb	Getppid() (pid int)
+//sys	Getpriority(which int, who int) (prio int, err error)
+//sysnb	Getrlimit(resource int, rlim *Rlimit) (err error) = SYS_GETRLIMIT
+
+//sysnb getrusage(who int, rusage *rusage_zos) (err error) = SYS_GETRUSAGE
+
+func Getrusage(who int, rusage *Rusage) (err error) {
+	var ruz rusage_zos
+	err = getrusage(who, &ruz)
+	//Only the first two fields of Rusage are set
+	rusage.Utime.Sec = ruz.Utime.Sec
+	rusage.Utime.Usec = int64(ruz.Utime.Usec)
+	rusage.Stime.Sec = ruz.Stime.Sec
+	rusage.Stime.Usec = int64(ruz.Stime.Usec)
+	return
+}
+
+//sysnb Getsid(pid int) (sid int, err error) = SYS_GETSID
+//sysnb	Getuid() (uid int)
+//sysnb	Kill(pid int, sig Signal) (err error)
+//sys	Lchown(path string, uid int, gid int) (err error) = SYS___LCHOWN_A
+//sys	Link(path string, link string) (err error) = SYS___LINK_A
+//sys	Listen(s int, n int) (err error)
+//sys	lstat(path string, stat *Stat_LE_t) (err error) = SYS___LSTAT_A
+
+func Lstat(path string, stat *Stat_t) (err error) {
+	var statLE Stat_LE_t
+	err = lstat(path, &statLE)
+	copyStat(stat, &statLE)
+	return
+}
+
+//sys	Mkdir(path string, mode uint32) (err error) = SYS___MKDIR_A
+//sys   Mkfifo(path string, mode uint32) (err error) = SYS___MKFIFO_A
+//sys	Mknod(path string, mode uint32, dev int) (err error) = SYS___MKNOD_A
+//sys	Pread(fd int, p []byte, offset int64) (n int, err error)
+//sys	Pwrite(fd int, p []byte, offset int64) (n int, err error)
+//sys	Readlink(path string, buf []byte) (n int, err error) = SYS___READLINK_A
+//sys	Rename(from string, to string) (err error) = SYS___RENAME_A
+//sys	Rmdir(path string) (err error) = SYS___RMDIR_A
+//sys   Seek(fd int, offset int64, whence int) (off int64, err error) = SYS_LSEEK
+//sys	Setpriority(which int, who int, prio int) (err error)
+//sysnb	Setpgid(pid int, pgid int) (err error) = SYS_SETPGID
+//sysnb	Setrlimit(resource int, lim *Rlimit) (err error)
+//sysnb	Setregid(rgid int, egid int) (err error) = SYS_SETREGID
+//sysnb	Setreuid(ruid int, euid int) (err error) = SYS_SETREUID
+//sysnb	Setsid() (pid int, err error) = SYS_SETSID
+//sys	Setuid(uid int) (err error) = SYS_SETUID
+//sys	Setgid(uid int) (err error) = SYS_SETGID
+//sys	Shutdown(fd int, how int) (err error)
+//sys	stat(path string, statLE *Stat_LE_t) (err error) = SYS___STAT_A
+
+func Stat(path string, sta *Stat_t) (err error) {
+	var statLE Stat_LE_t
+	err = stat(path, &statLE)
+	copyStat(sta, &statLE)
+	return
+}
+
+//sys	Symlink(path string, link string) (err error) = SYS___SYMLINK_A
+//sys	Sync() = SYS_SYNC
+//sys	Truncate(path string, length int64) (err error) = SYS___TRUNCATE_A
+//sys	Tcgetattr(fildes int, termptr *Termios) (err error) = SYS_TCGETATTR
+//sys	Tcsetattr(fildes int, when int, termptr *Termios) (err error) = SYS_TCSETATTR
+//sys	Umask(mask int) (oldmask int)
+//sys	Unlink(path string) (err error) = SYS___UNLINK_A
+//sys	Utime(path string, utim *Utimbuf) (err error) = SYS___UTIME_A
+
+//sys	open(path string, mode int, perm uint32) (fd int, err error) = SYS___OPEN_A
+
+func Open(path string, mode int, perm uint32) (fd int, err error) {
+	return open(path, mode, perm)
+}
+
+func Mkfifoat(dirfd int, path string, mode uint32) (err error) {
+	wd, err := Getwd()
+	if err != nil {
+		return err
+	}
+
+	if err := Fchdir(dirfd); err != nil {
+		return err
+	}
+	defer Chdir(wd)
+
+	return Mkfifo(path, mode)
+}
+
+//sys	remove(path string) (err error)
+
+func Remove(path string) error {
+	return remove(path)
+}
+
+const ImplementsGetwd = true
+
+func Getcwd(buf []byte) (n int, err error) {
+	var p unsafe.Pointer
+	if len(buf) > 0 {
+		p = unsafe.Pointer(&buf[0])
+	} else {
+		p = unsafe.Pointer(&_zero)
+	}
+	_, _, e := syscall_syscall(SYS___GETCWD_A, uintptr(p), uintptr(len(buf)), 0)
+	n = clen(buf) + 1
+	if e != 0 {
+		err = errnoErr(e)
+	}
+	return
+}
+
+func Getwd() (wd string, err error) {
+	var buf [PathMax]byte
+	n, err := Getcwd(buf[0:])
+	if err != nil {
+		return "", err
+	}
+	// Getcwd returns the number of bytes written to buf, including the NUL.
+	if n < 1 || n > len(buf) || buf[n-1] != 0 {
+		return "", EINVAL
+	}
+	return string(buf[0 : n-1]), nil
+}
+
+func Getgroups() (gids []int, err error) {
+	n, err := getgroups(0, nil)
+	if err != nil {
+		return nil, err
+	}
+	if n == 0 {
+		return nil, nil
+	}
+
+	// Sanity check group count.  Max is 1<<16 on Linux.
+	if n < 0 || n > 1<<20 {
+		return nil, EINVAL
+	}
+
+	a := make([]_Gid_t, n)
+	n, err = getgroups(n, &a[0])
+	if err != nil {
+		return nil, err
+	}
+	gids = make([]int, n)
+	for i, v := range a[0:n] {
+		gids[i] = int(v)
+	}
+	return
+}
+
+func Setgroups(gids []int) (err error) {
+	if len(gids) == 0 {
+		return setgroups(0, nil)
+	}
+
+	a := make([]_Gid_t, len(gids))
+	for i, v := range gids {
+		a[i] = _Gid_t(v)
+	}
+	return setgroups(len(a), &a[0])
+}
+
+func gettid() uint64
+
+func Gettid() (tid int) {
+	return int(gettid())
+}
+
+type WaitStatus uint32
+
+// Wait status is 7 bits at bottom, either 0 (exited),
+// 0x7F (stopped), or a signal number that caused an exit.
+// The 0x80 bit is whether there was a core dump.
+// An extra number (exit code, signal causing a stop)
+// is in the high bits.  At least that's the idea.
+// There are various irregularities.  For example, the
+// "continued" status is 0xFFFF, distinguishing itself
+// from stopped via the core dump bit.
+
+const (
+	mask    = 0x7F
+	core    = 0x80
+	exited  = 0x00
+	stopped = 0x7F
+	shift   = 8
+)
+
+func (w WaitStatus) Exited() bool { return w&mask == exited }
+
+func (w WaitStatus) Signaled() bool { return w&mask != stopped && w&mask != exited }
+
+func (w WaitStatus) Stopped() bool { return w&0xFF == stopped }
+
+func (w WaitStatus) Continued() bool { return w == 0xFFFF }
+
+func (w WaitStatus) CoreDump() bool { return w.Signaled() && w&core != 0 }
+
+func (w WaitStatus) ExitStatus() int {
+	if !w.Exited() {
+		return -1
+	}
+	return int(w>>shift) & 0xFF
+}
+
+func (w WaitStatus) Signal() Signal {
+	if !w.Signaled() {
+		return -1
+	}
+	return Signal(w & mask)
+}
+
+func (w WaitStatus) StopSignal() Signal {
+	if !w.Stopped() {
+		return -1
+	}
+	return Signal(w>>shift) & 0xFF
+}
+
+func (w WaitStatus) TrapCause() int { return -1 }
+
+//sys	waitpid(pid int, wstatus *_C_int, options int) (wpid int, err error)
+
+func Wait4(pid int, wstatus *WaitStatus, options int, rusage *Rusage) (wpid int, err error) {
+	// TODO(mundaym): z/OS doesn't have wait4. I don't think getrusage does what we want.
+	// At the moment rusage will not be touched.
+	var status _C_int
+	wpid, err = waitpid(pid, &status, options)
+	if wstatus != nil {
+		*wstatus = WaitStatus(status)
+	}
+	return
+}
+
+//sysnb	gettimeofday(tv *timeval_zos) (err error)
+
+func Gettimeofday(tv *Timeval) (err error) {
+	var tvz timeval_zos
+	err = gettimeofday(&tvz)
+	tv.Sec = tvz.Sec
+	tv.Usec = int64(tvz.Usec)
+	return
+}
+
+func Time(t *Time_t) (tt Time_t, err error) {
+	var tv Timeval
+	err = Gettimeofday(&tv)
+	if err != nil {
+		return 0, err
+	}
+	if t != nil {
+		*t = Time_t(tv.Sec)
+	}
+	return Time_t(tv.Sec), nil
+}
+
+func setTimespec(sec, nsec int64) Timespec {
+	return Timespec{Sec: sec, Nsec: nsec}
+}
+
+func setTimeval(sec, usec int64) Timeval { //fix
+	return Timeval{Sec: sec, Usec: usec}
+}
+
+//sysnb pipe(p *[2]_C_int) (err error)
+
+func Pipe(p []int) (err error) {
+	if len(p) != 2 {
+		return EINVAL
+	}
+	var pp [2]_C_int
+	err = pipe(&pp)
+	p[0] = int(pp[0])
+	p[1] = int(pp[1])
+	return
+}
+
+//sys	utimes(path string, timeval *[2]Timeval) (err error) = SYS___UTIMES_A
+
+func Utimes(path string, tv []Timeval) (err error) {
+	if len(tv) != 2 {
+		return EINVAL
+	}
+	return utimes(path, (*[2]Timeval)(unsafe.Pointer(&tv[0])))
+}
+
+func UtimesNano(path string, ts []Timespec) error {
+	if len(ts) != 2 {
+		return EINVAL
+	}
+	// Not as efficient as it could be because Timespec and
+	// Timeval have different types in the different OSes
+	tv := [2]Timeval{
+		NsecToTimeval(TimespecToNsec(ts[0])),
+		NsecToTimeval(TimespecToNsec(ts[1])),
+	}
+	return utimes(path, (*[2]Timeval)(unsafe.Pointer(&tv[0])))
+}
+
+func Getsockname(fd int) (sa Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	var len _Socklen = SizeofSockaddrAny
+	if err = getsockname(fd, &rsa, &len); err != nil {
+		return
+	}
+	// TODO(neeilan) : Remove this 0 ( added to get sys/unix compiling on z/OS )
+	return anyToSockaddr(0, &rsa)
+}
+
+const (
+	// identifier constants
+	nwmHeaderIdentifier    = 0xd5e6d4c8
+	nwmFilterIdentifier    = 0xd5e6d4c6
+	nwmTCPConnIdentifier   = 0xd5e6d4c3
+	nwmRecHeaderIdentifier = 0xd5e6d4d9
+	nwmIPStatsIdentifier   = 0xd5e6d4c9d7e2e340
+	nwmIPGStatsIdentifier  = 0xd5e6d4c9d7c7e2e3
+	nwmTCPStatsIdentifier  = 0xd5e6d4e3c3d7e2e3
+	nwmUDPStatsIdentifier  = 0xd5e6d4e4c4d7e2e3
+	nwmICMPGStatsEntry     = 0xd5e6d4c9c3d4d7c7
+	nwmICMPTStatsEntry     = 0xd5e6d4c9c3d4d7e3
+
+	// nwmHeader constants
+	nwmVersion1   = 1
+	nwmVersion2   = 2
+	nwmCurrentVer = 2
+
+	nwmTCPConnType     = 1
+	nwmGlobalStatsType = 14
+
+	// nwmFilter constants
+	nwmFilterLclAddrMask = 0x20000000 // Local address
+	nwmFilterSrcAddrMask = 0x20000000 // Source address
+	nwmFilterLclPortMask = 0x10000000 // Local port
+	nwmFilterSrcPortMask = 0x10000000 // Source port
+
+	// nwmConnEntry constants
+	nwmTCPStateClosed   = 1
+	nwmTCPStateListen   = 2
+	nwmTCPStateSynSent  = 3
+	nwmTCPStateSynRcvd  = 4
+	nwmTCPStateEstab    = 5
+	nwmTCPStateFinWait1 = 6
+	nwmTCPStateFinWait2 = 7
+	nwmTCPStateClosWait = 8
+	nwmTCPStateLastAck  = 9
+	nwmTCPStateClosing  = 10
+	nwmTCPStateTimeWait = 11
+	nwmTCPStateDeletTCB = 12
+
+	// Existing constants on linux
+	BPF_TCP_CLOSE        = 1
+	BPF_TCP_LISTEN       = 2
+	BPF_TCP_SYN_SENT     = 3
+	BPF_TCP_SYN_RECV     = 4
+	BPF_TCP_ESTABLISHED  = 5
+	BPF_TCP_FIN_WAIT1    = 6
+	BPF_TCP_FIN_WAIT2    = 7
+	BPF_TCP_CLOSE_WAIT   = 8
+	BPF_TCP_LAST_ACK     = 9
+	BPF_TCP_CLOSING      = 10
+	BPF_TCP_TIME_WAIT    = 11
+	BPF_TCP_NEW_SYN_RECV = -1
+	BPF_TCP_MAX_STATES   = -2
+)
+
+type nwmTriplet struct {
+	offset uint32
+	length uint32
+	number uint32
+}
+
+type nwmQuadruplet struct {
+	offset uint32
+	length uint32
+	number uint32
+	match  uint32
+}
+
+type nwmHeader struct {
+	ident       uint32
+	length      uint32
+	version     uint16
+	nwmType     uint16
+	bytesNeeded uint32
+	options     uint32
+	_           [16]byte
+	inputDesc   nwmTriplet
+	outputDesc  nwmQuadruplet
+}
+
+type nwmFilter struct {
+	ident         uint32
+	flags         uint32
+	resourceName  [8]byte
+	resourceId    uint32
+	listenerId    uint32
+	local         [28]byte // union of sockaddr4 and sockaddr6
+	remote        [28]byte // union of sockaddr4 and sockaddr6
+	_             uint16
+	_             uint16
+	asid          uint16
+	_             [2]byte
+	tnLuName      [8]byte
+	tnMonGrp      uint32
+	tnAppl        [8]byte
+	applData      [40]byte
+	nInterface    [16]byte
+	dVipa         [16]byte
+	dVipaPfx      uint16
+	dVipaPort     uint16
+	dVipaFamily   byte
+	_             [3]byte
+	destXCF       [16]byte
+	destXCFPfx    uint16
+	destXCFFamily byte
+	_             [1]byte
+	targIP        [16]byte
+	targIPPfx     uint16
+	targIPFamily  byte
+	_             [1]byte
+	_             [20]byte
+}
+
+type nwmRecHeader struct {
+	ident  uint32
+	length uint32
+	number byte
+	_      [3]byte
+}
+
+type nwmTCPStatsEntry struct {
+	ident             uint64
+	currEstab         uint32
+	activeOpened      uint32
+	passiveOpened     uint32
+	connClosed        uint32
+	estabResets       uint32
+	attemptFails      uint32
+	passiveDrops      uint32
+	timeWaitReused    uint32
+	inSegs            uint64
+	predictAck        uint32
+	predictData       uint32
+	inDupAck          uint32
+	inBadSum          uint32
+	inBadLen          uint32
+	inShort           uint32
+	inDiscOldTime     uint32
+	inAllBeforeWin    uint32
+	inSomeBeforeWin   uint32
+	inAllAfterWin     uint32
+	inSomeAfterWin    uint32
+	inOutOfOrder      uint32
+	inAfterClose      uint32
+	inWinProbes       uint32
+	inWinUpdates      uint32
+	outWinUpdates     uint32
+	outSegs           uint64
+	outDelayAcks      uint32
+	outRsts           uint32
+	retransSegs       uint32
+	retransTimeouts   uint32
+	retransDrops      uint32
+	pmtuRetrans       uint32
+	pmtuErrors        uint32
+	outWinProbes      uint32
+	probeDrops        uint32
+	keepAliveProbes   uint32
+	keepAliveDrops    uint32
+	finwait2Drops     uint32
+	acceptCount       uint64
+	inBulkQSegs       uint64
+	inDiscards        uint64
+	connFloods        uint32
+	connStalls        uint32
+	cfgEphemDef       uint16
+	ephemInUse        uint16
+	ephemHiWater      uint16
+	flags             byte
+	_                 [1]byte
+	ephemExhaust      uint32
+	smcRCurrEstabLnks uint32
+	smcRLnkActTimeOut uint32
+	smcRActLnkOpened  uint32
+	smcRPasLnkOpened  uint32
+	smcRLnksClosed    uint32
+	smcRCurrEstab     uint32
+	smcRActiveOpened  uint32
+	smcRPassiveOpened uint32
+	smcRConnClosed    uint32
+	smcRInSegs        uint64
+	smcROutSegs       uint64
+	smcRInRsts        uint32
+	smcROutRsts       uint32
+	smcDCurrEstabLnks uint32
+	smcDActLnkOpened  uint32
+	smcDPasLnkOpened  uint32
+	smcDLnksClosed    uint32
+	smcDCurrEstab     uint32
+	smcDActiveOpened  uint32
+	smcDPassiveOpened uint32
+	smcDConnClosed    uint32
+	smcDInSegs        uint64
+	smcDOutSegs       uint64
+	smcDInRsts        uint32
+	smcDOutRsts       uint32
+}
+
+type nwmConnEntry struct {
+	ident             uint32
+	local             [28]byte // union of sockaddr4 and sockaddr6
+	remote            [28]byte // union of sockaddr4 and sockaddr6
+	startTime         [8]byte  // uint64, changed to prevent padding from being inserted
+	lastActivity      [8]byte  // uint64
+	bytesIn           [8]byte  // uint64
+	bytesOut          [8]byte  // uint64
+	inSegs            [8]byte  // uint64
+	outSegs           [8]byte  // uint64
+	state             uint16
+	activeOpen        byte
+	flag01            byte
+	outBuffered       uint32
+	inBuffered        uint32
+	maxSndWnd         uint32
+	reXmtCount        uint32
+	congestionWnd     uint32
+	ssThresh          uint32
+	roundTripTime     uint32
+	roundTripVar      uint32
+	sendMSS           uint32
+	sndWnd            uint32
+	rcvBufSize        uint32
+	sndBufSize        uint32
+	outOfOrderCount   uint32
+	lcl0WindowCount   uint32
+	rmt0WindowCount   uint32
+	dupacks           uint32
+	flag02            byte
+	sockOpt6Cont      byte
+	asid              uint16
+	resourceName      [8]byte
+	resourceId        uint32
+	subtask           uint32
+	sockOpt           byte
+	sockOpt6          byte
+	clusterConnFlag   byte
+	proto             byte
+	targetAppl        [8]byte
+	luName            [8]byte
+	clientUserId      [8]byte
+	logMode           [8]byte
+	timeStamp         uint32
+	timeStampAge      uint32
+	serverResourceId  uint32
+	intfName          [16]byte
+	ttlsStatPol       byte
+	ttlsStatConn      byte
+	ttlsSSLProt       uint16
+	ttlsNegCiph       [2]byte
+	ttlsSecType       byte
+	ttlsFIPS140Mode   byte
+	ttlsUserID        [8]byte
+	applData          [40]byte
+	inOldestTime      [8]byte // uint64
+	outOldestTime     [8]byte // uint64
+	tcpTrustedPartner byte
+	_                 [3]byte
+	bulkDataIntfName  [16]byte
+	ttlsNegCiph4      [4]byte
+	smcReason         uint32
+	lclSMCLinkId      uint32
+	rmtSMCLinkId      uint32
+	smcStatus         byte
+	smcFlags          byte
+	_                 [2]byte
+	rcvWnd            uint32
+	lclSMCBufSz       uint32
+	rmtSMCBufSz       uint32
+	ttlsSessID        [32]byte
+	ttlsSessIDLen     int16
+	_                 [1]byte
+	smcDStatus        byte
+	smcDReason        uint32
+}
+
+var svcNameTable [][]byte = [][]byte{
+	[]byte("\xc5\xe9\xc2\xd5\xd4\xc9\xc6\xf4"), // svc_EZBNMIF4
+}
+
+const (
+	svc_EZBNMIF4 = 0
+)
+
+func GetsockoptTCPInfo(fd, level, opt int) (*TCPInfo, error) {
+	jobname := []byte("\x5c\x40\x40\x40\x40\x40\x40\x40") // "*"
+	responseBuffer := [4096]byte{0}
+	var bufferAlet, reasonCode uint32 = 0, 0
+	var bufferLen, returnValue, returnCode int32 = 4096, 0, 0
+
+	dsa := [18]uint64{0}
+	var argv [7]unsafe.Pointer
+	argv[0] = unsafe.Pointer(&jobname[0])
+	argv[1] = unsafe.Pointer(&responseBuffer[0])
+	argv[2] = unsafe.Pointer(&bufferAlet)
+	argv[3] = unsafe.Pointer(&bufferLen)
+	argv[4] = unsafe.Pointer(&returnValue)
+	argv[5] = unsafe.Pointer(&returnCode)
+	argv[6] = unsafe.Pointer(&reasonCode)
+
+	request := (*struct {
+		header nwmHeader
+		filter nwmFilter
+	})(unsafe.Pointer(&responseBuffer[0]))
+
+	EZBNMIF4 := svcLoad(&svcNameTable[svc_EZBNMIF4][0])
+	if EZBNMIF4 == nil {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// GetGlobalStats EZBNMIF4 call
+	request.header.ident = nwmHeaderIdentifier
+	request.header.length = uint32(unsafe.Sizeof(request.header))
+	request.header.version = nwmCurrentVer
+	request.header.nwmType = nwmGlobalStatsType
+	request.header.options = 0x80000000
+
+	svcCall(EZBNMIF4, &argv[0], &dsa[0])
+
+	// outputDesc field is filled by EZBNMIF4 on success
+	if returnCode != 0 || request.header.outputDesc.offset == 0 {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// Check that EZBNMIF4 returned a nwmRecHeader
+	recHeader := (*nwmRecHeader)(unsafe.Pointer(&responseBuffer[request.header.outputDesc.offset]))
+	if recHeader.ident != nwmRecHeaderIdentifier {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// Parse nwmTriplets to get offsets of returned entries
+	var sections []*uint64
+	var sectionDesc *nwmTriplet = (*nwmTriplet)(unsafe.Pointer(&responseBuffer[0]))
+	for i := uint32(0); i < uint32(recHeader.number); i++ {
+		offset := request.header.outputDesc.offset + uint32(unsafe.Sizeof(*recHeader)) + i*uint32(unsafe.Sizeof(*sectionDesc))
+		sectionDesc = (*nwmTriplet)(unsafe.Pointer(&responseBuffer[offset]))
+		for j := uint32(0); j < sectionDesc.number; j++ {
+			offset = request.header.outputDesc.offset + sectionDesc.offset + j*sectionDesc.length
+			sections = append(sections, (*uint64)(unsafe.Pointer(&responseBuffer[offset])))
+		}
+	}
+
+	// Find nwmTCPStatsEntry in returned entries
+	var tcpStats *nwmTCPStatsEntry = nil
+	for _, ptr := range sections {
+		switch *ptr {
+		case nwmTCPStatsIdentifier:
+			if tcpStats != nil {
+				return nil, errnoErr(EINVAL)
+			}
+			tcpStats = (*nwmTCPStatsEntry)(unsafe.Pointer(ptr))
+		case nwmIPStatsIdentifier:
+		case nwmIPGStatsIdentifier:
+		case nwmUDPStatsIdentifier:
+		case nwmICMPGStatsEntry:
+		case nwmICMPTStatsEntry:
+		default:
+			return nil, errnoErr(EINVAL)
+		}
+	}
+	if tcpStats == nil {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// GetConnectionDetail EZBNMIF4 call
+	responseBuffer = [4096]byte{0}
+	dsa = [18]uint64{0}
+	bufferAlet, reasonCode = 0, 0
+	bufferLen, returnValue, returnCode = 4096, 0, 0
+	nameptr := (*uint32)(unsafe.Pointer(uintptr(0x21c))) // Get jobname of current process
+	nameptr = (*uint32)(unsafe.Pointer(uintptr(*nameptr + 12)))
+	argv[0] = unsafe.Pointer(uintptr(*nameptr))
+
+	request.header.ident = nwmHeaderIdentifier
+	request.header.length = uint32(unsafe.Sizeof(request.header))
+	request.header.version = nwmCurrentVer
+	request.header.nwmType = nwmTCPConnType
+	request.header.options = 0x80000000
+
+	request.filter.ident = nwmFilterIdentifier
+
+	var localSockaddr RawSockaddrAny
+	socklen := _Socklen(SizeofSockaddrAny)
+	err := getsockname(fd, &localSockaddr, &socklen)
+	if err != nil {
+		return nil, errnoErr(EINVAL)
+	}
+	if localSockaddr.Addr.Family == AF_INET {
+		localSockaddr := (*RawSockaddrInet4)(unsafe.Pointer(&localSockaddr.Addr))
+		localSockFilter := (*RawSockaddrInet4)(unsafe.Pointer(&request.filter.local[0]))
+		localSockFilter.Family = AF_INET
+		var i int
+		for i = 0; i < 4; i++ {
+			if localSockaddr.Addr[i] != 0 {
+				break
+			}
+		}
+		if i != 4 {
+			request.filter.flags |= nwmFilterLclAddrMask
+			for i = 0; i < 4; i++ {
+				localSockFilter.Addr[i] = localSockaddr.Addr[i]
+			}
+		}
+		if localSockaddr.Port != 0 {
+			request.filter.flags |= nwmFilterLclPortMask
+			localSockFilter.Port = localSockaddr.Port
+		}
+	} else if localSockaddr.Addr.Family == AF_INET6 {
+		localSockaddr := (*RawSockaddrInet6)(unsafe.Pointer(&localSockaddr.Addr))
+		localSockFilter := (*RawSockaddrInet6)(unsafe.Pointer(&request.filter.local[0]))
+		localSockFilter.Family = AF_INET6
+		var i int
+		for i = 0; i < 16; i++ {
+			if localSockaddr.Addr[i] != 0 {
+				break
+			}
+		}
+		if i != 16 {
+			request.filter.flags |= nwmFilterLclAddrMask
+			for i = 0; i < 16; i++ {
+				localSockFilter.Addr[i] = localSockaddr.Addr[i]
+			}
+		}
+		if localSockaddr.Port != 0 {
+			request.filter.flags |= nwmFilterLclPortMask
+			localSockFilter.Port = localSockaddr.Port
+		}
+	}
+
+	svcCall(EZBNMIF4, &argv[0], &dsa[0])
+
+	// outputDesc field is filled by EZBNMIF4 on success
+	if returnCode != 0 || request.header.outputDesc.offset == 0 {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// Check that EZBNMIF4 returned a nwmConnEntry
+	conn := (*nwmConnEntry)(unsafe.Pointer(&responseBuffer[request.header.outputDesc.offset]))
+	if conn.ident != nwmTCPConnIdentifier {
+		return nil, errnoErr(EINVAL)
+	}
+
+	// Copy data from the returned data structures into tcpInfo
+	// Stats from nwmConnEntry are specific to that connection.
+	// Stats from nwmTCPStatsEntry are global (to the interface?)
+	// Fields may not be an exact match. Some fields have no equivalent.
+	var tcpinfo TCPInfo
+	tcpinfo.State = uint8(conn.state)
+	tcpinfo.Ca_state = 0 // dummy
+	tcpinfo.Retransmits = uint8(tcpStats.retransSegs)
+	tcpinfo.Probes = uint8(tcpStats.outWinProbes)
+	tcpinfo.Backoff = 0 // dummy
+	tcpinfo.Options = 0 // dummy
+	tcpinfo.Rto = tcpStats.retransTimeouts
+	tcpinfo.Ato = tcpStats.outDelayAcks
+	tcpinfo.Snd_mss = conn.sendMSS
+	tcpinfo.Rcv_mss = conn.sendMSS // dummy
+	tcpinfo.Unacked = 0            // dummy
+	tcpinfo.Sacked = 0             // dummy
+	tcpinfo.Lost = 0               // dummy
+	tcpinfo.Retrans = conn.reXmtCount
+	tcpinfo.Fackets = 0 // dummy
+	tcpinfo.Last_data_sent = uint32(*(*uint64)(unsafe.Pointer(&conn.lastActivity[0])))
+	tcpinfo.Last_ack_sent = uint32(*(*uint64)(unsafe.Pointer(&conn.outOldestTime[0])))
+	tcpinfo.Last_data_recv = uint32(*(*uint64)(unsafe.Pointer(&conn.inOldestTime[0])))
+	tcpinfo.Last_ack_recv = uint32(*(*uint64)(unsafe.Pointer(&conn.inOldestTime[0])))
+	tcpinfo.Pmtu = conn.sendMSS // dummy, NWMIfRouteMtu is a candidate
+	tcpinfo.Rcv_ssthresh = conn.ssThresh
+	tcpinfo.Rtt = conn.roundTripTime
+	tcpinfo.Rttvar = conn.roundTripVar
+	tcpinfo.Snd_ssthresh = conn.ssThresh // dummy
+	tcpinfo.Snd_cwnd = conn.congestionWnd
+	tcpinfo.Advmss = conn.sendMSS        // dummy
+	tcpinfo.Reordering = 0               // dummy
+	tcpinfo.Rcv_rtt = conn.roundTripTime // dummy
+	tcpinfo.Rcv_space = conn.sendMSS     // dummy
+	tcpinfo.Total_retrans = conn.reXmtCount
+
+	svcUnload(&svcNameTable[svc_EZBNMIF4][0], EZBNMIF4)
+
+	return &tcpinfo, nil
+}
+
+// GetsockoptString returns the string value of the socket option opt for the
+// socket associated with fd at the given socket level.
+func GetsockoptString(fd, level, opt int) (string, error) {
+	buf := make([]byte, 256)
+	vallen := _Socklen(len(buf))
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&buf[0]), &vallen)
+	if err != nil {
+		return "", err
+	}
+
+	return string(buf[:vallen-1]), nil
+}
+
+func Recvmsg(fd int, p, oob []byte, flags int) (n, oobn int, recvflags int, from Sockaddr, err error) {
+	var msg Msghdr
+	var rsa RawSockaddrAny
+	msg.Name = (*byte)(unsafe.Pointer(&rsa))
+	msg.Namelen = SizeofSockaddrAny
+	var iov Iovec
+	if len(p) > 0 {
+		iov.Base = (*byte)(unsafe.Pointer(&p[0]))
+		iov.SetLen(len(p))
+	}
+	var dummy byte
+	if len(oob) > 0 {
+		// receive at least one normal byte
+		if len(p) == 0 {
+			iov.Base = &dummy
+			iov.SetLen(1)
+		}
+		msg.Control = (*byte)(unsafe.Pointer(&oob[0]))
+		msg.SetControllen(len(oob))
+	}
+	msg.Iov = &iov
+	msg.Iovlen = 1
+	if n, err = recvmsg(fd, &msg, flags); err != nil {
+		return
+	}
+	oobn = int(msg.Controllen)
+	recvflags = int(msg.Flags)
+	// source address is only specified if the socket is unconnected
+	if rsa.Addr.Family != AF_UNSPEC {
+		// TODO(neeilan): Remove 0 arg added to get this compiling on z/OS
+		from, err = anyToSockaddr(0, &rsa)
+	}
+	return
+}
+
+func Sendmsg(fd int, p, oob []byte, to Sockaddr, flags int) (err error) {
+	_, err = SendmsgN(fd, p, oob, to, flags)
+	return
+}
+
+func SendmsgN(fd int, p, oob []byte, to Sockaddr, flags int) (n int, err error) {
+	var ptr unsafe.Pointer
+	var salen _Socklen
+	if to != nil {
+		var err error
+		ptr, salen, err = to.sockaddr()
+		if err != nil {
+			return 0, err
+		}
+	}
+	var msg Msghdr
+	msg.Name = (*byte)(unsafe.Pointer(ptr))
+	msg.Namelen = int32(salen)
+	var iov Iovec
+	if len(p) > 0 {
+		iov.Base = (*byte)(unsafe.Pointer(&p[0]))
+		iov.SetLen(len(p))
+	}
+	var dummy byte
+	if len(oob) > 0 {
+		// send at least one normal byte
+		if len(p) == 0 {
+			iov.Base = &dummy
+			iov.SetLen(1)
+		}
+		msg.Control = (*byte)(unsafe.Pointer(&oob[0]))
+		msg.SetControllen(len(oob))
+	}
+	msg.Iov = &iov
+	msg.Iovlen = 1
+	if n, err = sendmsg(fd, &msg, flags); err != nil {
+		return 0, err
+	}
+	if len(oob) > 0 && len(p) == 0 {
+		n = 0
+	}
+	return n, nil
+}
+
+func Opendir(name string) (uintptr, error) {
+	p, err := BytePtrFromString(name)
+	if err != nil {
+		return 0, err
+	}
+	dir, _, e := syscall_syscall(SYS___OPENDIR_A, uintptr(unsafe.Pointer(p)), 0, 0)
+	runtime.KeepAlive(unsafe.Pointer(p))
+	if e != 0 {
+		err = errnoErr(e)
+	}
+	return dir, err
+}
+
+// clearsyscall.Errno resets the errno value to 0.
+func clearErrno()
+
+func Readdir(dir uintptr) (*Dirent, error) {
+	var ent Dirent
+	var res uintptr
+	// __readdir_r_a returns errno at the end of the directory stream, rather than 0.
+	// Therefore to avoid false positives we clear errno before calling it.
+
+	// TODO(neeilan): Commented this out to get sys/unix compiling on z/OS. Uncomment and fix. Error: "undefined: clearsyscall"
+	//clearsyscall.Errno() // TODO(mundaym): check pre-emption rules.
+
+	e, _, _ := syscall_syscall(SYS___READDIR_R_A, dir, uintptr(unsafe.Pointer(&ent)), uintptr(unsafe.Pointer(&res)))
+	var err error
+	if e != 0 {
+		err = errnoErr(Errno(e))
+	}
+	if res == 0 {
+		return nil, err
+	}
+	return &ent, err
+}
+
+func Closedir(dir uintptr) error {
+	_, _, e := syscall_syscall(SYS_CLOSEDIR, dir, 0, 0)
+	if e != 0 {
+		return errnoErr(e)
+	}
+	return nil
+}
+
+func Seekdir(dir uintptr, pos int) {
+	_, _, _ = syscall_syscall(SYS_SEEKDIR, dir, uintptr(pos), 0)
+}
+
+func Telldir(dir uintptr) (int, error) {
+	p, _, e := syscall_syscall(SYS_TELLDIR, dir, 0, 0)
+	pos := int(p)
+	if pos == -1 {
+		return pos, errnoErr(e)
+	}
+	return pos, nil
+}
+
+// FcntlFlock performs a fcntl syscall for the F_GETLK, F_SETLK or F_SETLKW command.
+func FcntlFlock(fd uintptr, cmd int, lk *Flock_t) error {
+	// struct flock is packed on z/OS. We can't emulate that in Go so
+	// instead we pack it here.
+	var flock [24]byte
+	*(*int16)(unsafe.Pointer(&flock[0])) = lk.Type
+	*(*int16)(unsafe.Pointer(&flock[2])) = lk.Whence
+	*(*int64)(unsafe.Pointer(&flock[4])) = lk.Start
+	*(*int64)(unsafe.Pointer(&flock[12])) = lk.Len
+	*(*int32)(unsafe.Pointer(&flock[20])) = lk.Pid
+	_, _, errno := syscall_syscall(SYS_FCNTL, fd, uintptr(cmd), uintptr(unsafe.Pointer(&flock)))
+	lk.Type = *(*int16)(unsafe.Pointer(&flock[0]))
+	lk.Whence = *(*int16)(unsafe.Pointer(&flock[2]))
+	lk.Start = *(*int64)(unsafe.Pointer(&flock[4]))
+	lk.Len = *(*int64)(unsafe.Pointer(&flock[12]))
+	lk.Pid = *(*int32)(unsafe.Pointer(&flock[20]))
+	if errno == 0 {
+		return nil
+	}
+	return errno
+}
+
+func Flock(fd int, how int) error {
+
+	var flock_type int16
+	var fcntl_cmd int
+
+	switch how {
+	case LOCK_SH | LOCK_NB:
+		flock_type = F_RDLCK
+		fcntl_cmd = F_SETLK
+	case LOCK_EX | LOCK_NB:
+		flock_type = F_WRLCK
+		fcntl_cmd = F_SETLK
+	case LOCK_EX:
+		flock_type = F_WRLCK
+		fcntl_cmd = F_SETLKW
+	case LOCK_UN:
+		flock_type = F_UNLCK
+		fcntl_cmd = F_SETLKW
+	default:
+	}
+
+	flock := Flock_t{
+		Type:   int16(flock_type),
+		Whence: int16(0),
+		Start:  int64(0),
+		Len:    int64(0),
+		Pid:    int32(Getppid()),
+	}
+
+	err := FcntlFlock(uintptr(fd), fcntl_cmd, &flock)
+	return err
+}
+
+func Mlock(b []byte) (err error) {
+	_, _, e1 := syscall_syscall(SYS___MLOCKALL, _BPX_NONSWAP, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func Mlock2(b []byte, flags int) (err error) {
+	_, _, e1 := syscall_syscall(SYS___MLOCKALL, _BPX_NONSWAP, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func Mlockall(flags int) (err error) {
+	_, _, e1 := syscall_syscall(SYS___MLOCKALL, _BPX_NONSWAP, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func Munlock(b []byte) (err error) {
+	_, _, e1 := syscall_syscall(SYS___MLOCKALL, _BPX_SWAP, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func Munlockall() (err error) {
+	_, _, e1 := syscall_syscall(SYS___MLOCKALL, _BPX_SWAP, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+func ClockGettime(clockid int32, ts *Timespec) error {
+
+	var ticks_per_sec uint32 = 100 //TODO(kenan): value is currently hardcoded; need sysconf() call otherwise
+	var nsec_per_sec int64 = 1000000000
+
+	if ts == nil {
+		return EFAULT
+	}
+	if clockid == CLOCK_REALTIME || clockid == CLOCK_MONOTONIC {
+		var nanotime int64 = runtime.Nanotime1()
+		ts.Sec = nanotime / nsec_per_sec
+		ts.Nsec = nanotime % nsec_per_sec
+	} else if clockid == CLOCK_PROCESS_CPUTIME_ID || clockid == CLOCK_THREAD_CPUTIME_ID {
+		var tm Tms
+		_, err := Times(&tm)
+		if err != nil {
+			return EFAULT
+		}
+		ts.Sec = int64(tm.Utime / ticks_per_sec)
+		ts.Nsec = int64(tm.Utime) * nsec_per_sec / int64(ticks_per_sec)
+	} else {
+		return EINVAL
+	}
+	return nil
+}
+
+func Statfs(path string, stat *Statfs_t) (err error) {
+	fd, err := open(path, O_RDONLY, 0)
+	defer Close(fd)
+	if err != nil {
+		return err
+	}
+	return Fstatfs(fd, stat)
+}
+
+var (
+	Stdin  = 0
+	Stdout = 1
+	Stderr = 2
+)
+
+// Do the interface allocations only once for common
+// Errno values.
+var (
+	errEAGAIN error = syscall.EAGAIN
+	errEINVAL error = syscall.EINVAL
+	errENOENT error = syscall.ENOENT
+)
+
+var (
+	signalNameMapOnce sync.Once
+	signalNameMap     map[string]syscall.Signal
+)
+
+// errnoErr returns common boxed Errno values, to prevent
+// allocations at runtime.
+func errnoErr(e Errno) error {
+	switch e {
+	case 0:
+		return nil
+	case EAGAIN:
+		return errEAGAIN
+	case EINVAL:
+		return errEINVAL
+	case ENOENT:
+		return errENOENT
+	}
+	return e
+}
+
+// ErrnoName returns the error name for error number e.
+func ErrnoName(e Errno) string {
+	i := sort.Search(len(errorList), func(i int) bool {
+		return errorList[i].num >= e
+	})
+	if i < len(errorList) && errorList[i].num == e {
+		return errorList[i].name
+	}
+	return ""
+}
+
+// SignalName returns the signal name for signal number s.
+func SignalName(s syscall.Signal) string {
+	i := sort.Search(len(signalList), func(i int) bool {
+		return signalList[i].num >= s
+	})
+	if i < len(signalList) && signalList[i].num == s {
+		return signalList[i].name
+	}
+	return ""
+}
+
+// SignalNum returns the syscall.Signal for signal named s,
+// or 0 if a signal with such name is not found.
+// The signal name should start with "SIG".
+func SignalNum(s string) syscall.Signal {
+	signalNameMapOnce.Do(func() {
+		signalNameMap = make(map[string]syscall.Signal, len(signalList))
+		for _, signal := range signalList {
+			signalNameMap[signal.name] = signal.num
+		}
+	})
+	return signalNameMap[s]
+}
+
+// clen returns the index of the first NULL byte in n or len(n) if n contains no NULL byte.
+func clen(n []byte) int {
+	i := bytes.IndexByte(n, 0)
+	if i == -1 {
+		i = len(n)
+	}
+	return i
+}
+
+// Mmap manager, for use by operating system-specific implementations.
+
+type mmapper struct {
+	sync.Mutex
+	active map[*byte][]byte // active mappings; key is last byte in mapping
+	mmap   func(addr, length uintptr, prot, flags, fd int, offset int64) (uintptr, error)
+	munmap func(addr uintptr, length uintptr) error
+}
+
+func (m *mmapper) Mmap(fd int, offset int64, length int, prot int, flags int) (data []byte, err error) {
+	if length <= 0 {
+		return nil, EINVAL
+	}
+
+	// Map the requested memory.
+	addr, errno := m.mmap(0, uintptr(length), prot, flags, fd, offset)
+	if errno != nil {
+		return nil, errno
+	}
+
+	// Slice memory layout
+	var sl = struct {
+		addr uintptr
+		len  int
+		cap  int
+	}{addr, length, length}
+
+	// Use unsafe to turn sl into a []byte.
+	b := *(*[]byte)(unsafe.Pointer(&sl))
+
+	// Register mapping in m and return it.
+	p := &b[cap(b)-1]
+	m.Lock()
+	defer m.Unlock()
+	m.active[p] = b
+	return b, nil
+}
+
+func (m *mmapper) Munmap(data []byte) (err error) {
+	if len(data) == 0 || len(data) != cap(data) {
+		return EINVAL
+	}
+
+	// Find the base of the mapping.
+	p := &data[cap(data)-1]
+	m.Lock()
+	defer m.Unlock()
+	b := m.active[p]
+	if b == nil || &b[0] != &data[0] {
+		return EINVAL
+	}
+
+	// Unmap the memory and update m.
+	if errno := m.munmap(uintptr(unsafe.Pointer(&b[0])), uintptr(len(b))); errno != nil {
+		return errno
+	}
+	delete(m.active, p)
+	return nil
+}
+
+func Read(fd int, p []byte) (n int, err error) {
+	n, err = read(fd, p)
+	if raceenabled {
+		if n > 0 {
+			raceWriteRange(unsafe.Pointer(&p[0]), n)
+		}
+		if err == nil {
+			raceAcquire(unsafe.Pointer(&ioSync))
+		}
+	}
+	return
+}
+
+func Write(fd int, p []byte) (n int, err error) {
+	if raceenabled {
+		raceReleaseMerge(unsafe.Pointer(&ioSync))
+	}
+	n, err = write(fd, p)
+	if raceenabled && n > 0 {
+		raceReadRange(unsafe.Pointer(&p[0]), n)
+	}
+	return
+}
+
+// For testing: clients can set this flag to force
+// creation of IPv6 sockets to return EAFNOSUPPORT.
+var SocketDisableIPv6 bool
+
+// Sockaddr represents a socket address.
+type Sockaddr interface {
+	sockaddr() (ptr unsafe.Pointer, len _Socklen, err error) // lowercase; only we can define Sockaddrs
+}
+
+// SockaddrInet4 implements the Sockaddr interface for AF_INET type sockets.
+type SockaddrInet4 struct {
+	Port int
+	Addr [4]byte
+	raw  RawSockaddrInet4
+}
+
+// SockaddrInet6 implements the Sockaddr interface for AF_INET6 type sockets.
+type SockaddrInet6 struct {
+	Port   int
+	ZoneId uint32
+	Addr   [16]byte
+	raw    RawSockaddrInet6
+}
+
+// SockaddrUnix implements the Sockaddr interface for AF_UNIX type sockets.
+type SockaddrUnix struct {
+	Name string
+	raw  RawSockaddrUnix
+}
+
+func Bind(fd int, sa Sockaddr) (err error) {
+	ptr, n, err := sa.sockaddr()
+	if err != nil {
+		return err
+	}
+	return bind(fd, ptr, n)
+}
+
+func Connect(fd int, sa Sockaddr) (err error) {
+	ptr, n, err := sa.sockaddr()
+	if err != nil {
+		return err
+	}
+	return connect(fd, ptr, n)
+}
+
+func Getpeername(fd int) (sa Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	var len _Socklen = SizeofSockaddrAny
+	if err = getpeername(fd, &rsa, &len); err != nil {
+		return
+	}
+	return anyToSockaddr(fd, &rsa)
+}
+
+func GetsockoptByte(fd, level, opt int) (value byte, err error) {
+	var n byte
+	vallen := _Socklen(1)
+	err = getsockopt(fd, level, opt, unsafe.Pointer(&n), &vallen)
+	return n, err
+}
+
+func GetsockoptInt(fd, level, opt int) (value int, err error) {
+	var n int32
+	vallen := _Socklen(4)
+	err = getsockopt(fd, level, opt, unsafe.Pointer(&n), &vallen)
+	return int(n), err
+}
+
+func GetsockoptInet4Addr(fd, level, opt int) (value [4]byte, err error) {
+	vallen := _Socklen(4)
+	err = getsockopt(fd, level, opt, unsafe.Pointer(&value[0]), &vallen)
+	return value, err
+}
+
+func GetsockoptIPMreq(fd, level, opt int) (*IPMreq, error) {
+	var value IPMreq
+	vallen := _Socklen(SizeofIPMreq)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&value), &vallen)
+	return &value, err
+}
+
+func GetsockoptIPv6Mreq(fd, level, opt int) (*IPv6Mreq, error) {
+	var value IPv6Mreq
+	vallen := _Socklen(SizeofIPv6Mreq)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&value), &vallen)
+	return &value, err
+}
+
+func GetsockoptIPv6MTUInfo(fd, level, opt int) (*IPv6MTUInfo, error) {
+	var value IPv6MTUInfo
+	vallen := _Socklen(SizeofIPv6MTUInfo)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&value), &vallen)
+	return &value, err
+}
+
+func GetsockoptICMPv6Filter(fd, level, opt int) (*ICMPv6Filter, error) {
+	var value ICMPv6Filter
+	vallen := _Socklen(SizeofICMPv6Filter)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&value), &vallen)
+	return &value, err
+}
+
+func GetsockoptLinger(fd, level, opt int) (*Linger, error) {
+	var linger Linger
+	vallen := _Socklen(SizeofLinger)
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&linger), &vallen)
+	return &linger, err
+}
+
+func GetsockoptTimeval(fd, level, opt int) (*Timeval, error) {
+	var tv Timeval
+	vallen := _Socklen(unsafe.Sizeof(tv))
+	err := getsockopt(fd, level, opt, unsafe.Pointer(&tv), &vallen)
+	return &tv, err
+}
+
+func GetsockoptUint64(fd, level, opt int) (value uint64, err error) {
+	var n uint64
+	vallen := _Socklen(8)
+	err = getsockopt(fd, level, opt, unsafe.Pointer(&n), &vallen)
+	return n, err
+}
+
+func Recvfrom(fd int, p []byte, flags int) (n int, from Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	var len _Socklen = SizeofSockaddrAny
+	if n, err = recvfrom(fd, p, flags, &rsa, &len); err != nil {
+		return
+	}
+	if rsa.Addr.Family != AF_UNSPEC {
+		from, err = anyToSockaddr(fd, &rsa)
+	}
+	return
+}
+
+func Sendto(fd int, p []byte, flags int, to Sockaddr) (err error) {
+	ptr, n, err := to.sockaddr()
+	if err != nil {
+		return err
+	}
+	return sendto(fd, p, flags, ptr, n)
+}
+
+func SetsockoptByte(fd, level, opt int, value byte) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(&value), 1)
+}
+
+func SetsockoptInt(fd, level, opt int, value int) (err error) {
+	var n = int32(value)
+	return setsockopt(fd, level, opt, unsafe.Pointer(&n), 4)
+}
+
+func SetsockoptInet4Addr(fd, level, opt int, value [4]byte) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(&value[0]), 4)
+}
+
+func SetsockoptIPMreq(fd, level, opt int, mreq *IPMreq) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(mreq), SizeofIPMreq)
+}
+
+func SetsockoptIPv6Mreq(fd, level, opt int, mreq *IPv6Mreq) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(mreq), SizeofIPv6Mreq)
+}
+
+func SetsockoptICMPv6Filter(fd, level, opt int, filter *ICMPv6Filter) error {
+	return setsockopt(fd, level, opt, unsafe.Pointer(filter), SizeofICMPv6Filter)
+}
+
+func SetsockoptLinger(fd, level, opt int, l *Linger) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(l), SizeofLinger)
+}
+
+func SetsockoptString(fd, level, opt int, s string) (err error) {
+	var p unsafe.Pointer
+	if len(s) > 0 {
+		p = unsafe.Pointer(&[]byte(s)[0])
+	}
+	return setsockopt(fd, level, opt, p, uintptr(len(s)))
+}
+
+func SetsockoptTimeval(fd, level, opt int, tv *Timeval) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(tv), unsafe.Sizeof(*tv))
+}
+
+func SetsockoptUint64(fd, level, opt int, value uint64) (err error) {
+	return setsockopt(fd, level, opt, unsafe.Pointer(&value), 8)
+}
+
+func Socket(domain, typ, proto int) (fd int, err error) {
+	if domain == AF_INET6 && SocketDisableIPv6 {
+		return -1, EAFNOSUPPORT
+	}
+	fd, err = socket(domain, typ, proto)
+	return
+}
+
+func Socketpair(domain, typ, proto int) (fd [2]int, err error) {
+	var fdx [2]int32
+	err = socketpair(domain, typ, proto, &fdx)
+	if err == nil {
+		fd[0] = int(fdx[0])
+		fd[1] = int(fdx[1])
+	}
+	return
+}
+
+var ioSync int64
+
+func CloseOnExec(fd int) { fcntl(fd, F_SETFD, FD_CLOEXEC) }
+
+func SetNonblock(fd int, nonblocking bool) (err error) {
+	flag, err := fcntl(fd, F_GETFL, 0)
+	if err != nil {
+		return err
+	}
+	if nonblocking {
+		flag |= O_NONBLOCK
+	} else {
+		flag &= ^O_NONBLOCK
+	}
+	_, err = fcntl(fd, F_SETFL, flag)
+	return err
+}
+
+// Exec calls execve(2), which replaces the calling executable in the process
+// tree. argv0 should be the full path to an executable ("/bin/ls") and the
+// executable name should also be the first argument in argv (["ls", "-l"]).
+// envv are the environment variables that should be passed to the new
+// process (["USER=go", "PWD=/tmp"]).
+func Exec(argv0 string, argv []string, envv []string) error {
+	return syscall.Exec(argv0, argv, envv)
+}
+
+func Mount(source string, target string, fstype string, flags uintptr, data string) (err error) {
+	if needspace := 8 - len(fstype); needspace <= 0 {
+		fstype = fstype[:8]
+	} else {
+		fstype += "        "[:needspace]
+	}
+	return mount_LE(target, source, fstype, uint32(flags), int32(len(data)), data)
+}
+
+func Unmount(name string, mtm int) (err error) {
+	// mountpoint is always a full path and starts with a '/'
+	// check if input string is not a mountpoint but a filesystem name
+	if name[0] != '/' {
+		return unmount(name, mtm)
+	}
+	// treat name as mountpoint
+	b2s := func(arr []byte) string {
+		nulli := bytes.IndexByte(arr, 0)
+		if nulli == -1 {
+			return string(arr)
+		} else {
+			return string(arr[:nulli])
+		}
+	}
+	var buffer struct {
+		header W_Mnth
+		fsinfo [64]W_Mntent
+	}
+	fsCount, err := W_Getmntent_A((*byte)(unsafe.Pointer(&buffer)), int(unsafe.Sizeof(buffer)))
+	if err != nil {
+		return err
+	}
+	if fsCount == 0 {
+		return EINVAL
+	}
+	for i := 0; i < fsCount; i++ {
+		if b2s(buffer.fsinfo[i].Mountpoint[:]) == name {
+			err = unmount(b2s(buffer.fsinfo[i].Fsname[:]), mtm)
+			break
+		}
+	}
+	return err
+}
diff --git a/vendor/golang.org/x/sys/unix/timestruct.go b/vendor/golang.org/x/sys/unix/timestruct.go
index 4a672f5..3d89304 100644
--- a/vendor/golang.org/x/sys/unix/timestruct.go
+++ b/vendor/golang.org/x/sys/unix/timestruct.go
@@ -2,18 +2,17 @@
 // 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 netbsd openbsd solaris
+//go:build aix || darwin || dragonfly || freebsd || linux || netbsd || openbsd || solaris || zos
+// +build aix darwin dragonfly freebsd linux netbsd openbsd solaris zos
 
 package unix
 
 import "time"
 
-// TimespecToNsec converts a Timespec value into a number of
-// nanoseconds since the Unix epoch.
-func TimespecToNsec(ts Timespec) int64 { return int64(ts.Sec)*1e9 + int64(ts.Nsec) }
+// TimespecToNSec returns the time stored in ts as nanoseconds.
+func TimespecToNsec(ts Timespec) int64 { return ts.Nano() }
 
-// NsecToTimespec takes a number of nanoseconds since the Unix epoch
-// and returns the corresponding Timespec value.
+// NsecToTimespec converts a number of nanoseconds into a Timespec.
 func NsecToTimespec(nsec int64) Timespec {
 	sec := nsec / 1e9
 	nsec = nsec % 1e9
@@ -42,12 +41,10 @@
 	return ts, nil
 }
 
-// TimevalToNsec converts a Timeval value into a number of nanoseconds
-// since the Unix epoch.
-func TimevalToNsec(tv Timeval) int64 { return int64(tv.Sec)*1e9 + int64(tv.Usec)*1e3 }
+// TimevalToNsec returns the time stored in tv as nanoseconds.
+func TimevalToNsec(tv Timeval) int64 { return tv.Nano() }
 
-// NsecToTimeval takes a number of nanoseconds since the Unix epoch
-// and returns the corresponding Timeval value.
+// NsecToTimeval converts a number of nanoseconds into a Timeval.
 func NsecToTimeval(nsec int64) Timeval {
 	nsec += 999 // round up to microsecond
 	usec := nsec % 1e9 / 1e3
@@ -59,24 +56,22 @@
 	return setTimeval(sec, usec)
 }
 
-// Unix returns ts as the number of seconds and nanoseconds elapsed since the
-// Unix epoch.
+// Unix returns the time stored in ts as seconds plus nanoseconds.
 func (ts *Timespec) Unix() (sec int64, nsec int64) {
 	return int64(ts.Sec), int64(ts.Nsec)
 }
 
-// Unix returns tv as the number of seconds and nanoseconds elapsed since the
-// Unix epoch.
+// Unix returns the time stored in tv as seconds plus nanoseconds.
 func (tv *Timeval) Unix() (sec int64, nsec int64) {
 	return int64(tv.Sec), int64(tv.Usec) * 1000
 }
 
-// Nano returns ts as the number of nanoseconds elapsed since the Unix epoch.
+// Nano returns the time stored in ts as nanoseconds.
 func (ts *Timespec) Nano() int64 {
 	return int64(ts.Sec)*1e9 + int64(ts.Nsec)
 }
 
-// Nano returns tv as the number of nanoseconds elapsed since the Unix epoch.
+// Nano returns the time stored in tv as nanoseconds.
 func (tv *Timeval) Nano() int64 {
 	return int64(tv.Sec)*1e9 + int64(tv.Usec)*1000
 }
diff --git a/vendor/golang.org/x/sys/unix/xattr_bsd.go b/vendor/golang.org/x/sys/unix/xattr_bsd.go
index 30c1d71..25df1e3 100644
--- a/vendor/golang.org/x/sys/unix/xattr_bsd.go
+++ b/vendor/golang.org/x/sys/unix/xattr_bsd.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build freebsd || netbsd
 // +build freebsd netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zerrors_aix_ppc.go b/vendor/golang.org/x/sys/unix/zerrors_aix_ppc.go
index 104994b..ca9799b 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_aix_ppc.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_aix_ppc.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -maix32
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc && aix
 // +build ppc,aix
 
 // Created by cgo -godefs - DO NOT EDIT
diff --git a/vendor/golang.org/x/sys/unix/zerrors_aix_ppc64.go b/vendor/golang.org/x/sys/unix/zerrors_aix_ppc64.go
index 4fc8d30..200c8c2 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_aix_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_aix_ppc64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -maix64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64 && aix
 // +build ppc64,aix
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_darwin_386.go b/vendor/golang.org/x/sys/unix/zerrors_darwin_386.go
deleted file mode 100644
index 6f33359..0000000
--- a/vendor/golang.org/x/sys/unix/zerrors_darwin_386.go
+++ /dev/null
@@ -1,1786 +0,0 @@
-// mkerrors.sh -m32
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build 386,darwin
-
-// Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -m32 _const.go
-
-package unix
-
-import "syscall"
-
-const (
-	AF_APPLETALK                      = 0x10
-	AF_CCITT                          = 0xa
-	AF_CHAOS                          = 0x5
-	AF_CNT                            = 0x15
-	AF_COIP                           = 0x14
-	AF_DATAKIT                        = 0x9
-	AF_DECnet                         = 0xc
-	AF_DLI                            = 0xd
-	AF_E164                           = 0x1c
-	AF_ECMA                           = 0x8
-	AF_HYLINK                         = 0xf
-	AF_IEEE80211                      = 0x25
-	AF_IMPLINK                        = 0x3
-	AF_INET                           = 0x2
-	AF_INET6                          = 0x1e
-	AF_IPX                            = 0x17
-	AF_ISDN                           = 0x1c
-	AF_ISO                            = 0x7
-	AF_LAT                            = 0xe
-	AF_LINK                           = 0x12
-	AF_LOCAL                          = 0x1
-	AF_MAX                            = 0x28
-	AF_NATM                           = 0x1f
-	AF_NDRV                           = 0x1b
-	AF_NETBIOS                        = 0x21
-	AF_NS                             = 0x6
-	AF_OSI                            = 0x7
-	AF_PPP                            = 0x22
-	AF_PUP                            = 0x4
-	AF_RESERVED_36                    = 0x24
-	AF_ROUTE                          = 0x11
-	AF_SIP                            = 0x18
-	AF_SNA                            = 0xb
-	AF_SYSTEM                         = 0x20
-	AF_UNIX                           = 0x1
-	AF_UNSPEC                         = 0x0
-	AF_UTUN                           = 0x26
-	ALTWERASE                         = 0x200
-	ATTR_BIT_MAP_COUNT                = 0x5
-	ATTR_CMN_ACCESSMASK               = 0x20000
-	ATTR_CMN_ACCTIME                  = 0x1000
-	ATTR_CMN_ADDEDTIME                = 0x10000000
-	ATTR_CMN_BKUPTIME                 = 0x2000
-	ATTR_CMN_CHGTIME                  = 0x800
-	ATTR_CMN_CRTIME                   = 0x200
-	ATTR_CMN_DATA_PROTECT_FLAGS       = 0x40000000
-	ATTR_CMN_DEVID                    = 0x2
-	ATTR_CMN_DOCUMENT_ID              = 0x100000
-	ATTR_CMN_ERROR                    = 0x20000000
-	ATTR_CMN_EXTENDED_SECURITY        = 0x400000
-	ATTR_CMN_FILEID                   = 0x2000000
-	ATTR_CMN_FLAGS                    = 0x40000
-	ATTR_CMN_FNDRINFO                 = 0x4000
-	ATTR_CMN_FSID                     = 0x4
-	ATTR_CMN_FULLPATH                 = 0x8000000
-	ATTR_CMN_GEN_COUNT                = 0x80000
-	ATTR_CMN_GRPID                    = 0x10000
-	ATTR_CMN_GRPUUID                  = 0x1000000
-	ATTR_CMN_MODTIME                  = 0x400
-	ATTR_CMN_NAME                     = 0x1
-	ATTR_CMN_NAMEDATTRCOUNT           = 0x80000
-	ATTR_CMN_NAMEDATTRLIST            = 0x100000
-	ATTR_CMN_OBJID                    = 0x20
-	ATTR_CMN_OBJPERMANENTID           = 0x40
-	ATTR_CMN_OBJTAG                   = 0x10
-	ATTR_CMN_OBJTYPE                  = 0x8
-	ATTR_CMN_OWNERID                  = 0x8000
-	ATTR_CMN_PARENTID                 = 0x4000000
-	ATTR_CMN_PAROBJID                 = 0x80
-	ATTR_CMN_RETURNED_ATTRS           = 0x80000000
-	ATTR_CMN_SCRIPT                   = 0x100
-	ATTR_CMN_SETMASK                  = 0x41c7ff00
-	ATTR_CMN_USERACCESS               = 0x200000
-	ATTR_CMN_UUID                     = 0x800000
-	ATTR_CMN_VALIDMASK                = 0xffffffff
-	ATTR_CMN_VOLSETMASK               = 0x6700
-	ATTR_FILE_ALLOCSIZE               = 0x4
-	ATTR_FILE_CLUMPSIZE               = 0x10
-	ATTR_FILE_DATAALLOCSIZE           = 0x400
-	ATTR_FILE_DATAEXTENTS             = 0x800
-	ATTR_FILE_DATALENGTH              = 0x200
-	ATTR_FILE_DEVTYPE                 = 0x20
-	ATTR_FILE_FILETYPE                = 0x40
-	ATTR_FILE_FORKCOUNT               = 0x80
-	ATTR_FILE_FORKLIST                = 0x100
-	ATTR_FILE_IOBLOCKSIZE             = 0x8
-	ATTR_FILE_LINKCOUNT               = 0x1
-	ATTR_FILE_RSRCALLOCSIZE           = 0x2000
-	ATTR_FILE_RSRCEXTENTS             = 0x4000
-	ATTR_FILE_RSRCLENGTH              = 0x1000
-	ATTR_FILE_SETMASK                 = 0x20
-	ATTR_FILE_TOTALSIZE               = 0x2
-	ATTR_FILE_VALIDMASK               = 0x37ff
-	ATTR_VOL_ALLOCATIONCLUMP          = 0x40
-	ATTR_VOL_ATTRIBUTES               = 0x40000000
-	ATTR_VOL_CAPABILITIES             = 0x20000
-	ATTR_VOL_DIRCOUNT                 = 0x400
-	ATTR_VOL_ENCODINGSUSED            = 0x10000
-	ATTR_VOL_FILECOUNT                = 0x200
-	ATTR_VOL_FSTYPE                   = 0x1
-	ATTR_VOL_INFO                     = 0x80000000
-	ATTR_VOL_IOBLOCKSIZE              = 0x80
-	ATTR_VOL_MAXOBJCOUNT              = 0x800
-	ATTR_VOL_MINALLOCATION            = 0x20
-	ATTR_VOL_MOUNTEDDEVICE            = 0x8000
-	ATTR_VOL_MOUNTFLAGS               = 0x4000
-	ATTR_VOL_MOUNTPOINT               = 0x1000
-	ATTR_VOL_NAME                     = 0x2000
-	ATTR_VOL_OBJCOUNT                 = 0x100
-	ATTR_VOL_QUOTA_SIZE               = 0x10000000
-	ATTR_VOL_RESERVED_SIZE            = 0x20000000
-	ATTR_VOL_SETMASK                  = 0x80002000
-	ATTR_VOL_SIGNATURE                = 0x2
-	ATTR_VOL_SIZE                     = 0x4
-	ATTR_VOL_SPACEAVAIL               = 0x10
-	ATTR_VOL_SPACEFREE                = 0x8
-	ATTR_VOL_UUID                     = 0x40000
-	ATTR_VOL_VALIDMASK                = 0xf007ffff
-	B0                                = 0x0
-	B110                              = 0x6e
-	B115200                           = 0x1c200
-	B1200                             = 0x4b0
-	B134                              = 0x86
-	B14400                            = 0x3840
-	B150                              = 0x96
-	B1800                             = 0x708
-	B19200                            = 0x4b00
-	B200                              = 0xc8
-	B230400                           = 0x38400
-	B2400                             = 0x960
-	B28800                            = 0x7080
-	B300                              = 0x12c
-	B38400                            = 0x9600
-	B4800                             = 0x12c0
-	B50                               = 0x32
-	B57600                            = 0xe100
-	B600                              = 0x258
-	B7200                             = 0x1c20
-	B75                               = 0x4b
-	B76800                            = 0x12c00
-	B9600                             = 0x2580
-	BIOCFLUSH                         = 0x20004268
-	BIOCGBLEN                         = 0x40044266
-	BIOCGDLT                          = 0x4004426a
-	BIOCGDLTLIST                      = 0xc00c4279
-	BIOCGETIF                         = 0x4020426b
-	BIOCGHDRCMPLT                     = 0x40044274
-	BIOCGRSIG                         = 0x40044272
-	BIOCGRTIMEOUT                     = 0x4008426e
-	BIOCGSEESENT                      = 0x40044276
-	BIOCGSTATS                        = 0x4008426f
-	BIOCIMMEDIATE                     = 0x80044270
-	BIOCPROMISC                       = 0x20004269
-	BIOCSBLEN                         = 0xc0044266
-	BIOCSDLT                          = 0x80044278
-	BIOCSETF                          = 0x80084267
-	BIOCSETFNR                        = 0x8008427e
-	BIOCSETIF                         = 0x8020426c
-	BIOCSHDRCMPLT                     = 0x80044275
-	BIOCSRSIG                         = 0x80044273
-	BIOCSRTIMEOUT                     = 0x8008426d
-	BIOCSSEESENT                      = 0x80044277
-	BIOCVERSION                       = 0x40044271
-	BPF_A                             = 0x10
-	BPF_ABS                           = 0x20
-	BPF_ADD                           = 0x0
-	BPF_ALIGNMENT                     = 0x4
-	BPF_ALU                           = 0x4
-	BPF_AND                           = 0x50
-	BPF_B                             = 0x10
-	BPF_DIV                           = 0x30
-	BPF_H                             = 0x8
-	BPF_IMM                           = 0x0
-	BPF_IND                           = 0x40
-	BPF_JA                            = 0x0
-	BPF_JEQ                           = 0x10
-	BPF_JGE                           = 0x30
-	BPF_JGT                           = 0x20
-	BPF_JMP                           = 0x5
-	BPF_JSET                          = 0x40
-	BPF_K                             = 0x0
-	BPF_LD                            = 0x0
-	BPF_LDX                           = 0x1
-	BPF_LEN                           = 0x80
-	BPF_LSH                           = 0x60
-	BPF_MAJOR_VERSION                 = 0x1
-	BPF_MAXBUFSIZE                    = 0x80000
-	BPF_MAXINSNS                      = 0x200
-	BPF_MEM                           = 0x60
-	BPF_MEMWORDS                      = 0x10
-	BPF_MINBUFSIZE                    = 0x20
-	BPF_MINOR_VERSION                 = 0x1
-	BPF_MISC                          = 0x7
-	BPF_MSH                           = 0xa0
-	BPF_MUL                           = 0x20
-	BPF_NEG                           = 0x80
-	BPF_OR                            = 0x40
-	BPF_RELEASE                       = 0x30bb6
-	BPF_RET                           = 0x6
-	BPF_RSH                           = 0x70
-	BPF_ST                            = 0x2
-	BPF_STX                           = 0x3
-	BPF_SUB                           = 0x10
-	BPF_TAX                           = 0x0
-	BPF_TXA                           = 0x80
-	BPF_W                             = 0x0
-	BPF_X                             = 0x8
-	BRKINT                            = 0x2
-	BS0                               = 0x0
-	BS1                               = 0x8000
-	BSDLY                             = 0x8000
-	CFLUSH                            = 0xf
-	CLOCAL                            = 0x8000
-	CLOCK_MONOTONIC                   = 0x6
-	CLOCK_MONOTONIC_RAW               = 0x4
-	CLOCK_MONOTONIC_RAW_APPROX        = 0x5
-	CLOCK_PROCESS_CPUTIME_ID          = 0xc
-	CLOCK_REALTIME                    = 0x0
-	CLOCK_THREAD_CPUTIME_ID           = 0x10
-	CLOCK_UPTIME_RAW                  = 0x8
-	CLOCK_UPTIME_RAW_APPROX           = 0x9
-	CLONE_NOFOLLOW                    = 0x1
-	CLONE_NOOWNERCOPY                 = 0x2
-	CR0                               = 0x0
-	CR1                               = 0x1000
-	CR2                               = 0x2000
-	CR3                               = 0x3000
-	CRDLY                             = 0x3000
-	CREAD                             = 0x800
-	CRTSCTS                           = 0x30000
-	CS5                               = 0x0
-	CS6                               = 0x100
-	CS7                               = 0x200
-	CS8                               = 0x300
-	CSIZE                             = 0x300
-	CSTART                            = 0x11
-	CSTATUS                           = 0x14
-	CSTOP                             = 0x13
-	CSTOPB                            = 0x400
-	CSUSP                             = 0x1a
-	CTL_HW                            = 0x6
-	CTL_KERN                          = 0x1
-	CTL_MAXNAME                       = 0xc
-	CTL_NET                           = 0x4
-	DLT_A429                          = 0xb8
-	DLT_A653_ICM                      = 0xb9
-	DLT_AIRONET_HEADER                = 0x78
-	DLT_AOS                           = 0xde
-	DLT_APPLE_IP_OVER_IEEE1394        = 0x8a
-	DLT_ARCNET                        = 0x7
-	DLT_ARCNET_LINUX                  = 0x81
-	DLT_ATM_CLIP                      = 0x13
-	DLT_ATM_RFC1483                   = 0xb
-	DLT_AURORA                        = 0x7e
-	DLT_AX25                          = 0x3
-	DLT_AX25_KISS                     = 0xca
-	DLT_BACNET_MS_TP                  = 0xa5
-	DLT_BLUETOOTH_HCI_H4              = 0xbb
-	DLT_BLUETOOTH_HCI_H4_WITH_PHDR    = 0xc9
-	DLT_CAN20B                        = 0xbe
-	DLT_CAN_SOCKETCAN                 = 0xe3
-	DLT_CHAOS                         = 0x5
-	DLT_CHDLC                         = 0x68
-	DLT_CISCO_IOS                     = 0x76
-	DLT_C_HDLC                        = 0x68
-	DLT_C_HDLC_WITH_DIR               = 0xcd
-	DLT_DBUS                          = 0xe7
-	DLT_DECT                          = 0xdd
-	DLT_DOCSIS                        = 0x8f
-	DLT_DVB_CI                        = 0xeb
-	DLT_ECONET                        = 0x73
-	DLT_EN10MB                        = 0x1
-	DLT_EN3MB                         = 0x2
-	DLT_ENC                           = 0x6d
-	DLT_ERF                           = 0xc5
-	DLT_ERF_ETH                       = 0xaf
-	DLT_ERF_POS                       = 0xb0
-	DLT_FC_2                          = 0xe0
-	DLT_FC_2_WITH_FRAME_DELIMS        = 0xe1
-	DLT_FDDI                          = 0xa
-	DLT_FLEXRAY                       = 0xd2
-	DLT_FRELAY                        = 0x6b
-	DLT_FRELAY_WITH_DIR               = 0xce
-	DLT_GCOM_SERIAL                   = 0xad
-	DLT_GCOM_T1E1                     = 0xac
-	DLT_GPF_F                         = 0xab
-	DLT_GPF_T                         = 0xaa
-	DLT_GPRS_LLC                      = 0xa9
-	DLT_GSMTAP_ABIS                   = 0xda
-	DLT_GSMTAP_UM                     = 0xd9
-	DLT_HHDLC                         = 0x79
-	DLT_IBM_SN                        = 0x92
-	DLT_IBM_SP                        = 0x91
-	DLT_IEEE802                       = 0x6
-	DLT_IEEE802_11                    = 0x69
-	DLT_IEEE802_11_RADIO              = 0x7f
-	DLT_IEEE802_11_RADIO_AVS          = 0xa3
-	DLT_IEEE802_15_4                  = 0xc3
-	DLT_IEEE802_15_4_LINUX            = 0xbf
-	DLT_IEEE802_15_4_NOFCS            = 0xe6
-	DLT_IEEE802_15_4_NONASK_PHY       = 0xd7
-	DLT_IEEE802_16_MAC_CPS            = 0xbc
-	DLT_IEEE802_16_MAC_CPS_RADIO      = 0xc1
-	DLT_IPFILTER                      = 0x74
-	DLT_IPMB                          = 0xc7
-	DLT_IPMB_LINUX                    = 0xd1
-	DLT_IPNET                         = 0xe2
-	DLT_IPOIB                         = 0xf2
-	DLT_IPV4                          = 0xe4
-	DLT_IPV6                          = 0xe5
-	DLT_IP_OVER_FC                    = 0x7a
-	DLT_JUNIPER_ATM1                  = 0x89
-	DLT_JUNIPER_ATM2                  = 0x87
-	DLT_JUNIPER_ATM_CEMIC             = 0xee
-	DLT_JUNIPER_CHDLC                 = 0xb5
-	DLT_JUNIPER_ES                    = 0x84
-	DLT_JUNIPER_ETHER                 = 0xb2
-	DLT_JUNIPER_FIBRECHANNEL          = 0xea
-	DLT_JUNIPER_FRELAY                = 0xb4
-	DLT_JUNIPER_GGSN                  = 0x85
-	DLT_JUNIPER_ISM                   = 0xc2
-	DLT_JUNIPER_MFR                   = 0x86
-	DLT_JUNIPER_MLFR                  = 0x83
-	DLT_JUNIPER_MLPPP                 = 0x82
-	DLT_JUNIPER_MONITOR               = 0xa4
-	DLT_JUNIPER_PIC_PEER              = 0xae
-	DLT_JUNIPER_PPP                   = 0xb3
-	DLT_JUNIPER_PPPOE                 = 0xa7
-	DLT_JUNIPER_PPPOE_ATM             = 0xa8
-	DLT_JUNIPER_SERVICES              = 0x88
-	DLT_JUNIPER_SRX_E2E               = 0xe9
-	DLT_JUNIPER_ST                    = 0xc8
-	DLT_JUNIPER_VP                    = 0xb7
-	DLT_JUNIPER_VS                    = 0xe8
-	DLT_LAPB_WITH_DIR                 = 0xcf
-	DLT_LAPD                          = 0xcb
-	DLT_LIN                           = 0xd4
-	DLT_LINUX_EVDEV                   = 0xd8
-	DLT_LINUX_IRDA                    = 0x90
-	DLT_LINUX_LAPD                    = 0xb1
-	DLT_LINUX_PPP_WITHDIRECTION       = 0xa6
-	DLT_LINUX_SLL                     = 0x71
-	DLT_LOOP                          = 0x6c
-	DLT_LTALK                         = 0x72
-	DLT_MATCHING_MAX                  = 0xf5
-	DLT_MATCHING_MIN                  = 0x68
-	DLT_MFR                           = 0xb6
-	DLT_MOST                          = 0xd3
-	DLT_MPEG_2_TS                     = 0xf3
-	DLT_MPLS                          = 0xdb
-	DLT_MTP2                          = 0x8c
-	DLT_MTP2_WITH_PHDR                = 0x8b
-	DLT_MTP3                          = 0x8d
-	DLT_MUX27010                      = 0xec
-	DLT_NETANALYZER                   = 0xf0
-	DLT_NETANALYZER_TRANSPARENT       = 0xf1
-	DLT_NFC_LLCP                      = 0xf5
-	DLT_NFLOG                         = 0xef
-	DLT_NG40                          = 0xf4
-	DLT_NULL                          = 0x0
-	DLT_PCI_EXP                       = 0x7d
-	DLT_PFLOG                         = 0x75
-	DLT_PFSYNC                        = 0x12
-	DLT_PPI                           = 0xc0
-	DLT_PPP                           = 0x9
-	DLT_PPP_BSDOS                     = 0x10
-	DLT_PPP_ETHER                     = 0x33
-	DLT_PPP_PPPD                      = 0xa6
-	DLT_PPP_SERIAL                    = 0x32
-	DLT_PPP_WITH_DIR                  = 0xcc
-	DLT_PPP_WITH_DIRECTION            = 0xa6
-	DLT_PRISM_HEADER                  = 0x77
-	DLT_PRONET                        = 0x4
-	DLT_RAIF1                         = 0xc6
-	DLT_RAW                           = 0xc
-	DLT_RIO                           = 0x7c
-	DLT_SCCP                          = 0x8e
-	DLT_SITA                          = 0xc4
-	DLT_SLIP                          = 0x8
-	DLT_SLIP_BSDOS                    = 0xf
-	DLT_STANAG_5066_D_PDU             = 0xed
-	DLT_SUNATM                        = 0x7b
-	DLT_SYMANTEC_FIREWALL             = 0x63
-	DLT_TZSP                          = 0x80
-	DLT_USB                           = 0xba
-	DLT_USB_LINUX                     = 0xbd
-	DLT_USB_LINUX_MMAPPED             = 0xdc
-	DLT_USER0                         = 0x93
-	DLT_USER1                         = 0x94
-	DLT_USER10                        = 0x9d
-	DLT_USER11                        = 0x9e
-	DLT_USER12                        = 0x9f
-	DLT_USER13                        = 0xa0
-	DLT_USER14                        = 0xa1
-	DLT_USER15                        = 0xa2
-	DLT_USER2                         = 0x95
-	DLT_USER3                         = 0x96
-	DLT_USER4                         = 0x97
-	DLT_USER5                         = 0x98
-	DLT_USER6                         = 0x99
-	DLT_USER7                         = 0x9a
-	DLT_USER8                         = 0x9b
-	DLT_USER9                         = 0x9c
-	DLT_WIHART                        = 0xdf
-	DLT_X2E_SERIAL                    = 0xd5
-	DLT_X2E_XORAYA                    = 0xd6
-	DT_BLK                            = 0x6
-	DT_CHR                            = 0x2
-	DT_DIR                            = 0x4
-	DT_FIFO                           = 0x1
-	DT_LNK                            = 0xa
-	DT_REG                            = 0x8
-	DT_SOCK                           = 0xc
-	DT_UNKNOWN                        = 0x0
-	DT_WHT                            = 0xe
-	ECHO                              = 0x8
-	ECHOCTL                           = 0x40
-	ECHOE                             = 0x2
-	ECHOK                             = 0x4
-	ECHOKE                            = 0x1
-	ECHONL                            = 0x10
-	ECHOPRT                           = 0x20
-	EVFILT_AIO                        = -0x3
-	EVFILT_EXCEPT                     = -0xf
-	EVFILT_FS                         = -0x9
-	EVFILT_MACHPORT                   = -0x8
-	EVFILT_PROC                       = -0x5
-	EVFILT_READ                       = -0x1
-	EVFILT_SIGNAL                     = -0x6
-	EVFILT_SYSCOUNT                   = 0xf
-	EVFILT_THREADMARKER               = 0xf
-	EVFILT_TIMER                      = -0x7
-	EVFILT_USER                       = -0xa
-	EVFILT_VM                         = -0xc
-	EVFILT_VNODE                      = -0x4
-	EVFILT_WRITE                      = -0x2
-	EV_ADD                            = 0x1
-	EV_CLEAR                          = 0x20
-	EV_DELETE                         = 0x2
-	EV_DISABLE                        = 0x8
-	EV_DISPATCH                       = 0x80
-	EV_DISPATCH2                      = 0x180
-	EV_ENABLE                         = 0x4
-	EV_EOF                            = 0x8000
-	EV_ERROR                          = 0x4000
-	EV_FLAG0                          = 0x1000
-	EV_FLAG1                          = 0x2000
-	EV_ONESHOT                        = 0x10
-	EV_OOBAND                         = 0x2000
-	EV_POLL                           = 0x1000
-	EV_RECEIPT                        = 0x40
-	EV_SYSFLAGS                       = 0xf000
-	EV_UDATA_SPECIFIC                 = 0x100
-	EV_VANISHED                       = 0x200
-	EXTA                              = 0x4b00
-	EXTB                              = 0x9600
-	EXTPROC                           = 0x800
-	FD_CLOEXEC                        = 0x1
-	FD_SETSIZE                        = 0x400
-	FF0                               = 0x0
-	FF1                               = 0x4000
-	FFDLY                             = 0x4000
-	FLUSHO                            = 0x800000
-	FSOPT_ATTR_CMN_EXTENDED           = 0x20
-	FSOPT_NOFOLLOW                    = 0x1
-	FSOPT_NOINMEMUPDATE               = 0x2
-	FSOPT_PACK_INVAL_ATTRS            = 0x8
-	FSOPT_REPORT_FULLSIZE             = 0x4
-	F_ADDFILESIGS                     = 0x3d
-	F_ADDFILESIGS_FOR_DYLD_SIM        = 0x53
-	F_ADDFILESIGS_RETURN              = 0x61
-	F_ADDSIGS                         = 0x3b
-	F_ALLOCATEALL                     = 0x4
-	F_ALLOCATECONTIG                  = 0x2
-	F_BARRIERFSYNC                    = 0x55
-	F_CHECK_LV                        = 0x62
-	F_CHKCLEAN                        = 0x29
-	F_DUPFD                           = 0x0
-	F_DUPFD_CLOEXEC                   = 0x43
-	F_FINDSIGS                        = 0x4e
-	F_FLUSH_DATA                      = 0x28
-	F_FREEZE_FS                       = 0x35
-	F_FULLFSYNC                       = 0x33
-	F_GETCODEDIR                      = 0x48
-	F_GETFD                           = 0x1
-	F_GETFL                           = 0x3
-	F_GETLK                           = 0x7
-	F_GETLKPID                        = 0x42
-	F_GETNOSIGPIPE                    = 0x4a
-	F_GETOWN                          = 0x5
-	F_GETPATH                         = 0x32
-	F_GETPATH_MTMINFO                 = 0x47
-	F_GETPROTECTIONCLASS              = 0x3f
-	F_GETPROTECTIONLEVEL              = 0x4d
-	F_GLOBAL_NOCACHE                  = 0x37
-	F_LOG2PHYS                        = 0x31
-	F_LOG2PHYS_EXT                    = 0x41
-	F_NOCACHE                         = 0x30
-	F_NODIRECT                        = 0x3e
-	F_OK                              = 0x0
-	F_PATHPKG_CHECK                   = 0x34
-	F_PEOFPOSMODE                     = 0x3
-	F_PREALLOCATE                     = 0x2a
-	F_PUNCHHOLE                       = 0x63
-	F_RDADVISE                        = 0x2c
-	F_RDAHEAD                         = 0x2d
-	F_RDLCK                           = 0x1
-	F_SETBACKINGSTORE                 = 0x46
-	F_SETFD                           = 0x2
-	F_SETFL                           = 0x4
-	F_SETLK                           = 0x8
-	F_SETLKW                          = 0x9
-	F_SETLKWTIMEOUT                   = 0xa
-	F_SETNOSIGPIPE                    = 0x49
-	F_SETOWN                          = 0x6
-	F_SETPROTECTIONCLASS              = 0x40
-	F_SETSIZE                         = 0x2b
-	F_SINGLE_WRITER                   = 0x4c
-	F_THAW_FS                         = 0x36
-	F_TRANSCODEKEY                    = 0x4b
-	F_TRIM_ACTIVE_FILE                = 0x64
-	F_UNLCK                           = 0x2
-	F_VOLPOSMODE                      = 0x4
-	F_WRLCK                           = 0x3
-	HUPCL                             = 0x4000
-	HW_MACHINE                        = 0x1
-	ICANON                            = 0x100
-	ICMP6_FILTER                      = 0x12
-	ICRNL                             = 0x100
-	IEXTEN                            = 0x400
-	IFF_ALLMULTI                      = 0x200
-	IFF_ALTPHYS                       = 0x4000
-	IFF_BROADCAST                     = 0x2
-	IFF_DEBUG                         = 0x4
-	IFF_LINK0                         = 0x1000
-	IFF_LINK1                         = 0x2000
-	IFF_LINK2                         = 0x4000
-	IFF_LOOPBACK                      = 0x8
-	IFF_MULTICAST                     = 0x8000
-	IFF_NOARP                         = 0x80
-	IFF_NOTRAILERS                    = 0x20
-	IFF_OACTIVE                       = 0x400
-	IFF_POINTOPOINT                   = 0x10
-	IFF_PROMISC                       = 0x100
-	IFF_RUNNING                       = 0x40
-	IFF_SIMPLEX                       = 0x800
-	IFF_UP                            = 0x1
-	IFNAMSIZ                          = 0x10
-	IFT_1822                          = 0x2
-	IFT_AAL5                          = 0x31
-	IFT_ARCNET                        = 0x23
-	IFT_ARCNETPLUS                    = 0x24
-	IFT_ATM                           = 0x25
-	IFT_BRIDGE                        = 0xd1
-	IFT_CARP                          = 0xf8
-	IFT_CELLULAR                      = 0xff
-	IFT_CEPT                          = 0x13
-	IFT_DS3                           = 0x1e
-	IFT_ENC                           = 0xf4
-	IFT_EON                           = 0x19
-	IFT_ETHER                         = 0x6
-	IFT_FAITH                         = 0x38
-	IFT_FDDI                          = 0xf
-	IFT_FRELAY                        = 0x20
-	IFT_FRELAYDCE                     = 0x2c
-	IFT_GIF                           = 0x37
-	IFT_HDH1822                       = 0x3
-	IFT_HIPPI                         = 0x2f
-	IFT_HSSI                          = 0x2e
-	IFT_HY                            = 0xe
-	IFT_IEEE1394                      = 0x90
-	IFT_IEEE8023ADLAG                 = 0x88
-	IFT_ISDNBASIC                     = 0x14
-	IFT_ISDNPRIMARY                   = 0x15
-	IFT_ISO88022LLC                   = 0x29
-	IFT_ISO88023                      = 0x7
-	IFT_ISO88024                      = 0x8
-	IFT_ISO88025                      = 0x9
-	IFT_ISO88026                      = 0xa
-	IFT_L2VLAN                        = 0x87
-	IFT_LAPB                          = 0x10
-	IFT_LOCALTALK                     = 0x2a
-	IFT_LOOP                          = 0x18
-	IFT_MIOX25                        = 0x26
-	IFT_MODEM                         = 0x30
-	IFT_NSIP                          = 0x1b
-	IFT_OTHER                         = 0x1
-	IFT_P10                           = 0xc
-	IFT_P80                           = 0xd
-	IFT_PARA                          = 0x22
-	IFT_PDP                           = 0xff
-	IFT_PFLOG                         = 0xf5
-	IFT_PFSYNC                        = 0xf6
-	IFT_PKTAP                         = 0xfe
-	IFT_PPP                           = 0x17
-	IFT_PROPMUX                       = 0x36
-	IFT_PROPVIRTUAL                   = 0x35
-	IFT_PTPSERIAL                     = 0x16
-	IFT_RS232                         = 0x21
-	IFT_SDLC                          = 0x11
-	IFT_SIP                           = 0x1f
-	IFT_SLIP                          = 0x1c
-	IFT_SMDSDXI                       = 0x2b
-	IFT_SMDSICIP                      = 0x34
-	IFT_SONET                         = 0x27
-	IFT_SONETPATH                     = 0x32
-	IFT_SONETVT                       = 0x33
-	IFT_STARLAN                       = 0xb
-	IFT_STF                           = 0x39
-	IFT_T1                            = 0x12
-	IFT_ULTRA                         = 0x1d
-	IFT_V35                           = 0x2d
-	IFT_X25                           = 0x5
-	IFT_X25DDN                        = 0x4
-	IFT_X25PLE                        = 0x28
-	IFT_XETHER                        = 0x1a
-	IGNBRK                            = 0x1
-	IGNCR                             = 0x80
-	IGNPAR                            = 0x4
-	IMAXBEL                           = 0x2000
-	INLCR                             = 0x40
-	INPCK                             = 0x10
-	IN_CLASSA_HOST                    = 0xffffff
-	IN_CLASSA_MAX                     = 0x80
-	IN_CLASSA_NET                     = 0xff000000
-	IN_CLASSA_NSHIFT                  = 0x18
-	IN_CLASSB_HOST                    = 0xffff
-	IN_CLASSB_MAX                     = 0x10000
-	IN_CLASSB_NET                     = 0xffff0000
-	IN_CLASSB_NSHIFT                  = 0x10
-	IN_CLASSC_HOST                    = 0xff
-	IN_CLASSC_NET                     = 0xffffff00
-	IN_CLASSC_NSHIFT                  = 0x8
-	IN_CLASSD_HOST                    = 0xfffffff
-	IN_CLASSD_NET                     = 0xf0000000
-	IN_CLASSD_NSHIFT                  = 0x1c
-	IN_LINKLOCALNETNUM                = 0xa9fe0000
-	IN_LOOPBACKNET                    = 0x7f
-	IPPROTO_3PC                       = 0x22
-	IPPROTO_ADFS                      = 0x44
-	IPPROTO_AH                        = 0x33
-	IPPROTO_AHIP                      = 0x3d
-	IPPROTO_APES                      = 0x63
-	IPPROTO_ARGUS                     = 0xd
-	IPPROTO_AX25                      = 0x5d
-	IPPROTO_BHA                       = 0x31
-	IPPROTO_BLT                       = 0x1e
-	IPPROTO_BRSATMON                  = 0x4c
-	IPPROTO_CFTP                      = 0x3e
-	IPPROTO_CHAOS                     = 0x10
-	IPPROTO_CMTP                      = 0x26
-	IPPROTO_CPHB                      = 0x49
-	IPPROTO_CPNX                      = 0x48
-	IPPROTO_DDP                       = 0x25
-	IPPROTO_DGP                       = 0x56
-	IPPROTO_DIVERT                    = 0xfe
-	IPPROTO_DONE                      = 0x101
-	IPPROTO_DSTOPTS                   = 0x3c
-	IPPROTO_EGP                       = 0x8
-	IPPROTO_EMCON                     = 0xe
-	IPPROTO_ENCAP                     = 0x62
-	IPPROTO_EON                       = 0x50
-	IPPROTO_ESP                       = 0x32
-	IPPROTO_ETHERIP                   = 0x61
-	IPPROTO_FRAGMENT                  = 0x2c
-	IPPROTO_GGP                       = 0x3
-	IPPROTO_GMTP                      = 0x64
-	IPPROTO_GRE                       = 0x2f
-	IPPROTO_HELLO                     = 0x3f
-	IPPROTO_HMP                       = 0x14
-	IPPROTO_HOPOPTS                   = 0x0
-	IPPROTO_ICMP                      = 0x1
-	IPPROTO_ICMPV6                    = 0x3a
-	IPPROTO_IDP                       = 0x16
-	IPPROTO_IDPR                      = 0x23
-	IPPROTO_IDRP                      = 0x2d
-	IPPROTO_IGMP                      = 0x2
-	IPPROTO_IGP                       = 0x55
-	IPPROTO_IGRP                      = 0x58
-	IPPROTO_IL                        = 0x28
-	IPPROTO_INLSP                     = 0x34
-	IPPROTO_INP                       = 0x20
-	IPPROTO_IP                        = 0x0
-	IPPROTO_IPCOMP                    = 0x6c
-	IPPROTO_IPCV                      = 0x47
-	IPPROTO_IPEIP                     = 0x5e
-	IPPROTO_IPIP                      = 0x4
-	IPPROTO_IPPC                      = 0x43
-	IPPROTO_IPV4                      = 0x4
-	IPPROTO_IPV6                      = 0x29
-	IPPROTO_IRTP                      = 0x1c
-	IPPROTO_KRYPTOLAN                 = 0x41
-	IPPROTO_LARP                      = 0x5b
-	IPPROTO_LEAF1                     = 0x19
-	IPPROTO_LEAF2                     = 0x1a
-	IPPROTO_MAX                       = 0x100
-	IPPROTO_MAXID                     = 0x34
-	IPPROTO_MEAS                      = 0x13
-	IPPROTO_MHRP                      = 0x30
-	IPPROTO_MICP                      = 0x5f
-	IPPROTO_MTP                       = 0x5c
-	IPPROTO_MUX                       = 0x12
-	IPPROTO_ND                        = 0x4d
-	IPPROTO_NHRP                      = 0x36
-	IPPROTO_NONE                      = 0x3b
-	IPPROTO_NSP                       = 0x1f
-	IPPROTO_NVPII                     = 0xb
-	IPPROTO_OSPFIGP                   = 0x59
-	IPPROTO_PGM                       = 0x71
-	IPPROTO_PIGP                      = 0x9
-	IPPROTO_PIM                       = 0x67
-	IPPROTO_PRM                       = 0x15
-	IPPROTO_PUP                       = 0xc
-	IPPROTO_PVP                       = 0x4b
-	IPPROTO_RAW                       = 0xff
-	IPPROTO_RCCMON                    = 0xa
-	IPPROTO_RDP                       = 0x1b
-	IPPROTO_ROUTING                   = 0x2b
-	IPPROTO_RSVP                      = 0x2e
-	IPPROTO_RVD                       = 0x42
-	IPPROTO_SATEXPAK                  = 0x40
-	IPPROTO_SATMON                    = 0x45
-	IPPROTO_SCCSP                     = 0x60
-	IPPROTO_SCTP                      = 0x84
-	IPPROTO_SDRP                      = 0x2a
-	IPPROTO_SEP                       = 0x21
-	IPPROTO_SRPC                      = 0x5a
-	IPPROTO_ST                        = 0x7
-	IPPROTO_SVMTP                     = 0x52
-	IPPROTO_SWIPE                     = 0x35
-	IPPROTO_TCF                       = 0x57
-	IPPROTO_TCP                       = 0x6
-	IPPROTO_TP                        = 0x1d
-	IPPROTO_TPXX                      = 0x27
-	IPPROTO_TRUNK1                    = 0x17
-	IPPROTO_TRUNK2                    = 0x18
-	IPPROTO_TTP                       = 0x54
-	IPPROTO_UDP                       = 0x11
-	IPPROTO_VINES                     = 0x53
-	IPPROTO_VISA                      = 0x46
-	IPPROTO_VMTP                      = 0x51
-	IPPROTO_WBEXPAK                   = 0x4f
-	IPPROTO_WBMON                     = 0x4e
-	IPPROTO_WSN                       = 0x4a
-	IPPROTO_XNET                      = 0xf
-	IPPROTO_XTP                       = 0x24
-	IPV6_2292DSTOPTS                  = 0x17
-	IPV6_2292HOPLIMIT                 = 0x14
-	IPV6_2292HOPOPTS                  = 0x16
-	IPV6_2292NEXTHOP                  = 0x15
-	IPV6_2292PKTINFO                  = 0x13
-	IPV6_2292PKTOPTIONS               = 0x19
-	IPV6_2292RTHDR                    = 0x18
-	IPV6_BINDV6ONLY                   = 0x1b
-	IPV6_BOUND_IF                     = 0x7d
-	IPV6_CHECKSUM                     = 0x1a
-	IPV6_DEFAULT_MULTICAST_HOPS       = 0x1
-	IPV6_DEFAULT_MULTICAST_LOOP       = 0x1
-	IPV6_DEFHLIM                      = 0x40
-	IPV6_FAITH                        = 0x1d
-	IPV6_FLOWINFO_MASK                = 0xffffff0f
-	IPV6_FLOWLABEL_MASK               = 0xffff0f00
-	IPV6_FLOW_ECN_MASK                = 0x300
-	IPV6_FRAGTTL                      = 0x3c
-	IPV6_FW_ADD                       = 0x1e
-	IPV6_FW_DEL                       = 0x1f
-	IPV6_FW_FLUSH                     = 0x20
-	IPV6_FW_GET                       = 0x22
-	IPV6_FW_ZERO                      = 0x21
-	IPV6_HLIMDEC                      = 0x1
-	IPV6_IPSEC_POLICY                 = 0x1c
-	IPV6_JOIN_GROUP                   = 0xc
-	IPV6_LEAVE_GROUP                  = 0xd
-	IPV6_MAXHLIM                      = 0xff
-	IPV6_MAXOPTHDR                    = 0x800
-	IPV6_MAXPACKET                    = 0xffff
-	IPV6_MAX_GROUP_SRC_FILTER         = 0x200
-	IPV6_MAX_MEMBERSHIPS              = 0xfff
-	IPV6_MAX_SOCK_SRC_FILTER          = 0x80
-	IPV6_MIN_MEMBERSHIPS              = 0x1f
-	IPV6_MMTU                         = 0x500
-	IPV6_MULTICAST_HOPS               = 0xa
-	IPV6_MULTICAST_IF                 = 0x9
-	IPV6_MULTICAST_LOOP               = 0xb
-	IPV6_PORTRANGE                    = 0xe
-	IPV6_PORTRANGE_DEFAULT            = 0x0
-	IPV6_PORTRANGE_HIGH               = 0x1
-	IPV6_PORTRANGE_LOW                = 0x2
-	IPV6_RECVTCLASS                   = 0x23
-	IPV6_RTHDR_LOOSE                  = 0x0
-	IPV6_RTHDR_STRICT                 = 0x1
-	IPV6_RTHDR_TYPE_0                 = 0x0
-	IPV6_SOCKOPT_RESERVED1            = 0x3
-	IPV6_TCLASS                       = 0x24
-	IPV6_UNICAST_HOPS                 = 0x4
-	IPV6_V6ONLY                       = 0x1b
-	IPV6_VERSION                      = 0x60
-	IPV6_VERSION_MASK                 = 0xf0
-	IP_ADD_MEMBERSHIP                 = 0xc
-	IP_ADD_SOURCE_MEMBERSHIP          = 0x46
-	IP_BLOCK_SOURCE                   = 0x48
-	IP_BOUND_IF                       = 0x19
-	IP_DEFAULT_MULTICAST_LOOP         = 0x1
-	IP_DEFAULT_MULTICAST_TTL          = 0x1
-	IP_DF                             = 0x4000
-	IP_DROP_MEMBERSHIP                = 0xd
-	IP_DROP_SOURCE_MEMBERSHIP         = 0x47
-	IP_DUMMYNET_CONFIGURE             = 0x3c
-	IP_DUMMYNET_DEL                   = 0x3d
-	IP_DUMMYNET_FLUSH                 = 0x3e
-	IP_DUMMYNET_GET                   = 0x40
-	IP_FAITH                          = 0x16
-	IP_FW_ADD                         = 0x28
-	IP_FW_DEL                         = 0x29
-	IP_FW_FLUSH                       = 0x2a
-	IP_FW_GET                         = 0x2c
-	IP_FW_RESETLOG                    = 0x2d
-	IP_FW_ZERO                        = 0x2b
-	IP_HDRINCL                        = 0x2
-	IP_IPSEC_POLICY                   = 0x15
-	IP_MAXPACKET                      = 0xffff
-	IP_MAX_GROUP_SRC_FILTER           = 0x200
-	IP_MAX_MEMBERSHIPS                = 0xfff
-	IP_MAX_SOCK_MUTE_FILTER           = 0x80
-	IP_MAX_SOCK_SRC_FILTER            = 0x80
-	IP_MF                             = 0x2000
-	IP_MIN_MEMBERSHIPS                = 0x1f
-	IP_MSFILTER                       = 0x4a
-	IP_MSS                            = 0x240
-	IP_MULTICAST_IF                   = 0x9
-	IP_MULTICAST_IFINDEX              = 0x42
-	IP_MULTICAST_LOOP                 = 0xb
-	IP_MULTICAST_TTL                  = 0xa
-	IP_MULTICAST_VIF                  = 0xe
-	IP_NAT__XXX                       = 0x37
-	IP_OFFMASK                        = 0x1fff
-	IP_OLD_FW_ADD                     = 0x32
-	IP_OLD_FW_DEL                     = 0x33
-	IP_OLD_FW_FLUSH                   = 0x34
-	IP_OLD_FW_GET                     = 0x36
-	IP_OLD_FW_RESETLOG                = 0x38
-	IP_OLD_FW_ZERO                    = 0x35
-	IP_OPTIONS                        = 0x1
-	IP_PKTINFO                        = 0x1a
-	IP_PORTRANGE                      = 0x13
-	IP_PORTRANGE_DEFAULT              = 0x0
-	IP_PORTRANGE_HIGH                 = 0x1
-	IP_PORTRANGE_LOW                  = 0x2
-	IP_RECVDSTADDR                    = 0x7
-	IP_RECVIF                         = 0x14
-	IP_RECVOPTS                       = 0x5
-	IP_RECVPKTINFO                    = 0x1a
-	IP_RECVRETOPTS                    = 0x6
-	IP_RECVTOS                        = 0x1b
-	IP_RECVTTL                        = 0x18
-	IP_RETOPTS                        = 0x8
-	IP_RF                             = 0x8000
-	IP_RSVP_OFF                       = 0x10
-	IP_RSVP_ON                        = 0xf
-	IP_RSVP_VIF_OFF                   = 0x12
-	IP_RSVP_VIF_ON                    = 0x11
-	IP_STRIPHDR                       = 0x17
-	IP_TOS                            = 0x3
-	IP_TRAFFIC_MGT_BACKGROUND         = 0x41
-	IP_TTL                            = 0x4
-	IP_UNBLOCK_SOURCE                 = 0x49
-	ISIG                              = 0x80
-	ISTRIP                            = 0x20
-	IUTF8                             = 0x4000
-	IXANY                             = 0x800
-	IXOFF                             = 0x400
-	IXON                              = 0x200
-	KERN_HOSTNAME                     = 0xa
-	KERN_OSRELEASE                    = 0x2
-	KERN_OSTYPE                       = 0x1
-	KERN_VERSION                      = 0x4
-	LOCK_EX                           = 0x2
-	LOCK_NB                           = 0x4
-	LOCK_SH                           = 0x1
-	LOCK_UN                           = 0x8
-	MADV_CAN_REUSE                    = 0x9
-	MADV_DONTNEED                     = 0x4
-	MADV_FREE                         = 0x5
-	MADV_FREE_REUSABLE                = 0x7
-	MADV_FREE_REUSE                   = 0x8
-	MADV_NORMAL                       = 0x0
-	MADV_PAGEOUT                      = 0xa
-	MADV_RANDOM                       = 0x1
-	MADV_SEQUENTIAL                   = 0x2
-	MADV_WILLNEED                     = 0x3
-	MADV_ZERO_WIRED_PAGES             = 0x6
-	MAP_ANON                          = 0x1000
-	MAP_ANONYMOUS                     = 0x1000
-	MAP_COPY                          = 0x2
-	MAP_FILE                          = 0x0
-	MAP_FIXED                         = 0x10
-	MAP_HASSEMAPHORE                  = 0x200
-	MAP_JIT                           = 0x800
-	MAP_NOCACHE                       = 0x400
-	MAP_NOEXTEND                      = 0x100
-	MAP_NORESERVE                     = 0x40
-	MAP_PRIVATE                       = 0x2
-	MAP_RENAME                        = 0x20
-	MAP_RESERVED0080                  = 0x80
-	MAP_RESILIENT_CODESIGN            = 0x2000
-	MAP_RESILIENT_MEDIA               = 0x4000
-	MAP_SHARED                        = 0x1
-	MCL_CURRENT                       = 0x1
-	MCL_FUTURE                        = 0x2
-	MNT_ASYNC                         = 0x40
-	MNT_AUTOMOUNTED                   = 0x400000
-	MNT_CMDFLAGS                      = 0xf0000
-	MNT_CPROTECT                      = 0x80
-	MNT_DEFWRITE                      = 0x2000000
-	MNT_DONTBROWSE                    = 0x100000
-	MNT_DOVOLFS                       = 0x8000
-	MNT_DWAIT                         = 0x4
-	MNT_EXPORTED                      = 0x100
-	MNT_FORCE                         = 0x80000
-	MNT_IGNORE_OWNERSHIP              = 0x200000
-	MNT_JOURNALED                     = 0x800000
-	MNT_LOCAL                         = 0x1000
-	MNT_MULTILABEL                    = 0x4000000
-	MNT_NOATIME                       = 0x10000000
-	MNT_NOBLOCK                       = 0x20000
-	MNT_NODEV                         = 0x10
-	MNT_NOEXEC                        = 0x4
-	MNT_NOSUID                        = 0x8
-	MNT_NOUSERXATTR                   = 0x1000000
-	MNT_NOWAIT                        = 0x2
-	MNT_QUARANTINE                    = 0x400
-	MNT_QUOTA                         = 0x2000
-	MNT_RDONLY                        = 0x1
-	MNT_RELOAD                        = 0x40000
-	MNT_ROOTFS                        = 0x4000
-	MNT_SYNCHRONOUS                   = 0x2
-	MNT_UNION                         = 0x20
-	MNT_UNKNOWNPERMISSIONS            = 0x200000
-	MNT_UPDATE                        = 0x10000
-	MNT_VISFLAGMASK                   = 0x17f0f5ff
-	MNT_WAIT                          = 0x1
-	MSG_CTRUNC                        = 0x20
-	MSG_DONTROUTE                     = 0x4
-	MSG_DONTWAIT                      = 0x80
-	MSG_EOF                           = 0x100
-	MSG_EOR                           = 0x8
-	MSG_FLUSH                         = 0x400
-	MSG_HAVEMORE                      = 0x2000
-	MSG_HOLD                          = 0x800
-	MSG_NEEDSA                        = 0x10000
-	MSG_OOB                           = 0x1
-	MSG_PEEK                          = 0x2
-	MSG_RCVMORE                       = 0x4000
-	MSG_SEND                          = 0x1000
-	MSG_TRUNC                         = 0x10
-	MSG_WAITALL                       = 0x40
-	MSG_WAITSTREAM                    = 0x200
-	MS_ASYNC                          = 0x1
-	MS_DEACTIVATE                     = 0x8
-	MS_INVALIDATE                     = 0x2
-	MS_KILLPAGES                      = 0x4
-	MS_SYNC                           = 0x10
-	NAME_MAX                          = 0xff
-	NET_RT_DUMP                       = 0x1
-	NET_RT_DUMP2                      = 0x7
-	NET_RT_FLAGS                      = 0x2
-	NET_RT_IFLIST                     = 0x3
-	NET_RT_IFLIST2                    = 0x6
-	NET_RT_MAXID                      = 0xa
-	NET_RT_STAT                       = 0x4
-	NET_RT_TRASH                      = 0x5
-	NFDBITS                           = 0x20
-	NL0                               = 0x0
-	NL1                               = 0x100
-	NL2                               = 0x200
-	NL3                               = 0x300
-	NLDLY                             = 0x300
-	NOFLSH                            = 0x80000000
-	NOKERNINFO                        = 0x2000000
-	NOTE_ABSOLUTE                     = 0x8
-	NOTE_ATTRIB                       = 0x8
-	NOTE_BACKGROUND                   = 0x40
-	NOTE_CHILD                        = 0x4
-	NOTE_CRITICAL                     = 0x20
-	NOTE_DELETE                       = 0x1
-	NOTE_EXEC                         = 0x20000000
-	NOTE_EXIT                         = 0x80000000
-	NOTE_EXITSTATUS                   = 0x4000000
-	NOTE_EXIT_CSERROR                 = 0x40000
-	NOTE_EXIT_DECRYPTFAIL             = 0x10000
-	NOTE_EXIT_DETAIL                  = 0x2000000
-	NOTE_EXIT_DETAIL_MASK             = 0x70000
-	NOTE_EXIT_MEMORY                  = 0x20000
-	NOTE_EXIT_REPARENTED              = 0x80000
-	NOTE_EXTEND                       = 0x4
-	NOTE_FFAND                        = 0x40000000
-	NOTE_FFCOPY                       = 0xc0000000
-	NOTE_FFCTRLMASK                   = 0xc0000000
-	NOTE_FFLAGSMASK                   = 0xffffff
-	NOTE_FFNOP                        = 0x0
-	NOTE_FFOR                         = 0x80000000
-	NOTE_FORK                         = 0x40000000
-	NOTE_FUNLOCK                      = 0x100
-	NOTE_LEEWAY                       = 0x10
-	NOTE_LINK                         = 0x10
-	NOTE_LOWAT                        = 0x1
-	NOTE_MACH_CONTINUOUS_TIME         = 0x80
-	NOTE_NONE                         = 0x80
-	NOTE_NSECONDS                     = 0x4
-	NOTE_OOB                          = 0x2
-	NOTE_PCTRLMASK                    = -0x100000
-	NOTE_PDATAMASK                    = 0xfffff
-	NOTE_REAP                         = 0x10000000
-	NOTE_RENAME                       = 0x20
-	NOTE_REVOKE                       = 0x40
-	NOTE_SECONDS                      = 0x1
-	NOTE_SIGNAL                       = 0x8000000
-	NOTE_TRACK                        = 0x1
-	NOTE_TRACKERR                     = 0x2
-	NOTE_TRIGGER                      = 0x1000000
-	NOTE_USECONDS                     = 0x2
-	NOTE_VM_ERROR                     = 0x10000000
-	NOTE_VM_PRESSURE                  = 0x80000000
-	NOTE_VM_PRESSURE_SUDDEN_TERMINATE = 0x20000000
-	NOTE_VM_PRESSURE_TERMINATE        = 0x40000000
-	NOTE_WRITE                        = 0x2
-	OCRNL                             = 0x10
-	OFDEL                             = 0x20000
-	OFILL                             = 0x80
-	ONLCR                             = 0x2
-	ONLRET                            = 0x40
-	ONOCR                             = 0x20
-	ONOEOT                            = 0x8
-	OPOST                             = 0x1
-	OXTABS                            = 0x4
-	O_ACCMODE                         = 0x3
-	O_ALERT                           = 0x20000000
-	O_APPEND                          = 0x8
-	O_ASYNC                           = 0x40
-	O_CLOEXEC                         = 0x1000000
-	O_CREAT                           = 0x200
-	O_DIRECTORY                       = 0x100000
-	O_DP_GETRAWENCRYPTED              = 0x1
-	O_DP_GETRAWUNENCRYPTED            = 0x2
-	O_DSYNC                           = 0x400000
-	O_EVTONLY                         = 0x8000
-	O_EXCL                            = 0x800
-	O_EXLOCK                          = 0x20
-	O_FSYNC                           = 0x80
-	O_NDELAY                          = 0x4
-	O_NOCTTY                          = 0x20000
-	O_NOFOLLOW                        = 0x100
-	O_NONBLOCK                        = 0x4
-	O_POPUP                           = 0x80000000
-	O_RDONLY                          = 0x0
-	O_RDWR                            = 0x2
-	O_SHLOCK                          = 0x10
-	O_SYMLINK                         = 0x200000
-	O_SYNC                            = 0x80
-	O_TRUNC                           = 0x400
-	O_WRONLY                          = 0x1
-	PARENB                            = 0x1000
-	PARMRK                            = 0x8
-	PARODD                            = 0x2000
-	PENDIN                            = 0x20000000
-	PRIO_PGRP                         = 0x1
-	PRIO_PROCESS                      = 0x0
-	PRIO_USER                         = 0x2
-	PROT_EXEC                         = 0x4
-	PROT_NONE                         = 0x0
-	PROT_READ                         = 0x1
-	PROT_WRITE                        = 0x2
-	PT_ATTACH                         = 0xa
-	PT_ATTACHEXC                      = 0xe
-	PT_CONTINUE                       = 0x7
-	PT_DENY_ATTACH                    = 0x1f
-	PT_DETACH                         = 0xb
-	PT_FIRSTMACH                      = 0x20
-	PT_FORCEQUOTA                     = 0x1e
-	PT_KILL                           = 0x8
-	PT_READ_D                         = 0x2
-	PT_READ_I                         = 0x1
-	PT_READ_U                         = 0x3
-	PT_SIGEXC                         = 0xc
-	PT_STEP                           = 0x9
-	PT_THUPDATE                       = 0xd
-	PT_TRACE_ME                       = 0x0
-	PT_WRITE_D                        = 0x5
-	PT_WRITE_I                        = 0x4
-	PT_WRITE_U                        = 0x6
-	RLIMIT_AS                         = 0x5
-	RLIMIT_CORE                       = 0x4
-	RLIMIT_CPU                        = 0x0
-	RLIMIT_CPU_USAGE_MONITOR          = 0x2
-	RLIMIT_DATA                       = 0x2
-	RLIMIT_FSIZE                      = 0x1
-	RLIMIT_MEMLOCK                    = 0x6
-	RLIMIT_NOFILE                     = 0x8
-	RLIMIT_NPROC                      = 0x7
-	RLIMIT_RSS                        = 0x5
-	RLIMIT_STACK                      = 0x3
-	RLIM_INFINITY                     = 0x7fffffffffffffff
-	RTAX_AUTHOR                       = 0x6
-	RTAX_BRD                          = 0x7
-	RTAX_DST                          = 0x0
-	RTAX_GATEWAY                      = 0x1
-	RTAX_GENMASK                      = 0x3
-	RTAX_IFA                          = 0x5
-	RTAX_IFP                          = 0x4
-	RTAX_MAX                          = 0x8
-	RTAX_NETMASK                      = 0x2
-	RTA_AUTHOR                        = 0x40
-	RTA_BRD                           = 0x80
-	RTA_DST                           = 0x1
-	RTA_GATEWAY                       = 0x2
-	RTA_GENMASK                       = 0x8
-	RTA_IFA                           = 0x20
-	RTA_IFP                           = 0x10
-	RTA_NETMASK                       = 0x4
-	RTF_BLACKHOLE                     = 0x1000
-	RTF_BROADCAST                     = 0x400000
-	RTF_CLONING                       = 0x100
-	RTF_CONDEMNED                     = 0x2000000
-	RTF_DELCLONE                      = 0x80
-	RTF_DONE                          = 0x40
-	RTF_DYNAMIC                       = 0x10
-	RTF_GATEWAY                       = 0x2
-	RTF_HOST                          = 0x4
-	RTF_IFREF                         = 0x4000000
-	RTF_IFSCOPE                       = 0x1000000
-	RTF_LLINFO                        = 0x400
-	RTF_LOCAL                         = 0x200000
-	RTF_MODIFIED                      = 0x20
-	RTF_MULTICAST                     = 0x800000
-	RTF_NOIFREF                       = 0x2000
-	RTF_PINNED                        = 0x100000
-	RTF_PRCLONING                     = 0x10000
-	RTF_PROTO1                        = 0x8000
-	RTF_PROTO2                        = 0x4000
-	RTF_PROTO3                        = 0x40000
-	RTF_PROXY                         = 0x8000000
-	RTF_REJECT                        = 0x8
-	RTF_ROUTER                        = 0x10000000
-	RTF_STATIC                        = 0x800
-	RTF_UP                            = 0x1
-	RTF_WASCLONED                     = 0x20000
-	RTF_XRESOLVE                      = 0x200
-	RTM_ADD                           = 0x1
-	RTM_CHANGE                        = 0x3
-	RTM_DELADDR                       = 0xd
-	RTM_DELETE                        = 0x2
-	RTM_DELMADDR                      = 0x10
-	RTM_GET                           = 0x4
-	RTM_GET2                          = 0x14
-	RTM_IFINFO                        = 0xe
-	RTM_IFINFO2                       = 0x12
-	RTM_LOCK                          = 0x8
-	RTM_LOSING                        = 0x5
-	RTM_MISS                          = 0x7
-	RTM_NEWADDR                       = 0xc
-	RTM_NEWMADDR                      = 0xf
-	RTM_NEWMADDR2                     = 0x13
-	RTM_OLDADD                        = 0x9
-	RTM_OLDDEL                        = 0xa
-	RTM_REDIRECT                      = 0x6
-	RTM_RESOLVE                       = 0xb
-	RTM_RTTUNIT                       = 0xf4240
-	RTM_VERSION                       = 0x5
-	RTV_EXPIRE                        = 0x4
-	RTV_HOPCOUNT                      = 0x2
-	RTV_MTU                           = 0x1
-	RTV_RPIPE                         = 0x8
-	RTV_RTT                           = 0x40
-	RTV_RTTVAR                        = 0x80
-	RTV_SPIPE                         = 0x10
-	RTV_SSTHRESH                      = 0x20
-	RUSAGE_CHILDREN                   = -0x1
-	RUSAGE_SELF                       = 0x0
-	SCM_CREDS                         = 0x3
-	SCM_RIGHTS                        = 0x1
-	SCM_TIMESTAMP                     = 0x2
-	SCM_TIMESTAMP_MONOTONIC           = 0x4
-	SHUT_RD                           = 0x0
-	SHUT_RDWR                         = 0x2
-	SHUT_WR                           = 0x1
-	SIOCADDMULTI                      = 0x80206931
-	SIOCAIFADDR                       = 0x8040691a
-	SIOCARPIPLL                       = 0xc0206928
-	SIOCATMARK                        = 0x40047307
-	SIOCAUTOADDR                      = 0xc0206926
-	SIOCAUTONETMASK                   = 0x80206927
-	SIOCDELMULTI                      = 0x80206932
-	SIOCDIFADDR                       = 0x80206919
-	SIOCDIFPHYADDR                    = 0x80206941
-	SIOCGDRVSPEC                      = 0xc01c697b
-	SIOCGETVLAN                       = 0xc020697f
-	SIOCGHIWAT                        = 0x40047301
-	SIOCGIFADDR                       = 0xc0206921
-	SIOCGIFALTMTU                     = 0xc0206948
-	SIOCGIFASYNCMAP                   = 0xc020697c
-	SIOCGIFBOND                       = 0xc0206947
-	SIOCGIFBRDADDR                    = 0xc0206923
-	SIOCGIFCAP                        = 0xc020695b
-	SIOCGIFCONF                       = 0xc0086924
-	SIOCGIFDEVMTU                     = 0xc0206944
-	SIOCGIFDSTADDR                    = 0xc0206922
-	SIOCGIFFLAGS                      = 0xc0206911
-	SIOCGIFGENERIC                    = 0xc020693a
-	SIOCGIFKPI                        = 0xc0206987
-	SIOCGIFMAC                        = 0xc0206982
-	SIOCGIFMEDIA                      = 0xc0286938
-	SIOCGIFMETRIC                     = 0xc0206917
-	SIOCGIFMTU                        = 0xc0206933
-	SIOCGIFNETMASK                    = 0xc0206925
-	SIOCGIFPDSTADDR                   = 0xc0206940
-	SIOCGIFPHYS                       = 0xc0206935
-	SIOCGIFPSRCADDR                   = 0xc020693f
-	SIOCGIFSTATUS                     = 0xc331693d
-	SIOCGIFVLAN                       = 0xc020697f
-	SIOCGIFWAKEFLAGS                  = 0xc0206988
-	SIOCGLOWAT                        = 0x40047303
-	SIOCGPGRP                         = 0x40047309
-	SIOCIFCREATE                      = 0xc0206978
-	SIOCIFCREATE2                     = 0xc020697a
-	SIOCIFDESTROY                     = 0x80206979
-	SIOCIFGCLONERS                    = 0xc00c6981
-	SIOCRSLVMULTI                     = 0xc008693b
-	SIOCSDRVSPEC                      = 0x801c697b
-	SIOCSETVLAN                       = 0x8020697e
-	SIOCSHIWAT                        = 0x80047300
-	SIOCSIFADDR                       = 0x8020690c
-	SIOCSIFALTMTU                     = 0x80206945
-	SIOCSIFASYNCMAP                   = 0x8020697d
-	SIOCSIFBOND                       = 0x80206946
-	SIOCSIFBRDADDR                    = 0x80206913
-	SIOCSIFCAP                        = 0x8020695a
-	SIOCSIFDSTADDR                    = 0x8020690e
-	SIOCSIFFLAGS                      = 0x80206910
-	SIOCSIFGENERIC                    = 0x80206939
-	SIOCSIFKPI                        = 0x80206986
-	SIOCSIFLLADDR                     = 0x8020693c
-	SIOCSIFMAC                        = 0x80206983
-	SIOCSIFMEDIA                      = 0xc0206937
-	SIOCSIFMETRIC                     = 0x80206918
-	SIOCSIFMTU                        = 0x80206934
-	SIOCSIFNETMASK                    = 0x80206916
-	SIOCSIFPHYADDR                    = 0x8040693e
-	SIOCSIFPHYS                       = 0x80206936
-	SIOCSIFVLAN                       = 0x8020697e
-	SIOCSLOWAT                        = 0x80047302
-	SIOCSPGRP                         = 0x80047308
-	SOCK_DGRAM                        = 0x2
-	SOCK_MAXADDRLEN                   = 0xff
-	SOCK_RAW                          = 0x3
-	SOCK_RDM                          = 0x4
-	SOCK_SEQPACKET                    = 0x5
-	SOCK_STREAM                       = 0x1
-	SOL_SOCKET                        = 0xffff
-	SOMAXCONN                         = 0x80
-	SO_ACCEPTCONN                     = 0x2
-	SO_BROADCAST                      = 0x20
-	SO_DEBUG                          = 0x1
-	SO_DONTROUTE                      = 0x10
-	SO_DONTTRUNC                      = 0x2000
-	SO_ERROR                          = 0x1007
-	SO_KEEPALIVE                      = 0x8
-	SO_LABEL                          = 0x1010
-	SO_LINGER                         = 0x80
-	SO_LINGER_SEC                     = 0x1080
-	SO_NETSVC_MARKING_LEVEL           = 0x1119
-	SO_NET_SERVICE_TYPE               = 0x1116
-	SO_NKE                            = 0x1021
-	SO_NOADDRERR                      = 0x1023
-	SO_NOSIGPIPE                      = 0x1022
-	SO_NOTIFYCONFLICT                 = 0x1026
-	SO_NP_EXTENSIONS                  = 0x1083
-	SO_NREAD                          = 0x1020
-	SO_NUMRCVPKT                      = 0x1112
-	SO_NWRITE                         = 0x1024
-	SO_OOBINLINE                      = 0x100
-	SO_PEERLABEL                      = 0x1011
-	SO_RANDOMPORT                     = 0x1082
-	SO_RCVBUF                         = 0x1002
-	SO_RCVLOWAT                       = 0x1004
-	SO_RCVTIMEO                       = 0x1006
-	SO_REUSEADDR                      = 0x4
-	SO_REUSEPORT                      = 0x200
-	SO_REUSESHAREUID                  = 0x1025
-	SO_SNDBUF                         = 0x1001
-	SO_SNDLOWAT                       = 0x1003
-	SO_SNDTIMEO                       = 0x1005
-	SO_TIMESTAMP                      = 0x400
-	SO_TIMESTAMP_MONOTONIC            = 0x800
-	SO_TYPE                           = 0x1008
-	SO_UPCALLCLOSEWAIT                = 0x1027
-	SO_USELOOPBACK                    = 0x40
-	SO_WANTMORE                       = 0x4000
-	SO_WANTOOBFLAG                    = 0x8000
-	S_IEXEC                           = 0x40
-	S_IFBLK                           = 0x6000
-	S_IFCHR                           = 0x2000
-	S_IFDIR                           = 0x4000
-	S_IFIFO                           = 0x1000
-	S_IFLNK                           = 0xa000
-	S_IFMT                            = 0xf000
-	S_IFREG                           = 0x8000
-	S_IFSOCK                          = 0xc000
-	S_IFWHT                           = 0xe000
-	S_IREAD                           = 0x100
-	S_IRGRP                           = 0x20
-	S_IROTH                           = 0x4
-	S_IRUSR                           = 0x100
-	S_IRWXG                           = 0x38
-	S_IRWXO                           = 0x7
-	S_IRWXU                           = 0x1c0
-	S_ISGID                           = 0x400
-	S_ISTXT                           = 0x200
-	S_ISUID                           = 0x800
-	S_ISVTX                           = 0x200
-	S_IWGRP                           = 0x10
-	S_IWOTH                           = 0x2
-	S_IWRITE                          = 0x80
-	S_IWUSR                           = 0x80
-	S_IXGRP                           = 0x8
-	S_IXOTH                           = 0x1
-	S_IXUSR                           = 0x40
-	TAB0                              = 0x0
-	TAB1                              = 0x400
-	TAB2                              = 0x800
-	TAB3                              = 0x4
-	TABDLY                            = 0xc04
-	TCIFLUSH                          = 0x1
-	TCIOFF                            = 0x3
-	TCIOFLUSH                         = 0x3
-	TCION                             = 0x4
-	TCOFLUSH                          = 0x2
-	TCOOFF                            = 0x1
-	TCOON                             = 0x2
-	TCP_CONNECTIONTIMEOUT             = 0x20
-	TCP_CONNECTION_INFO               = 0x106
-	TCP_ENABLE_ECN                    = 0x104
-	TCP_FASTOPEN                      = 0x105
-	TCP_KEEPALIVE                     = 0x10
-	TCP_KEEPCNT                       = 0x102
-	TCP_KEEPINTVL                     = 0x101
-	TCP_MAXHLEN                       = 0x3c
-	TCP_MAXOLEN                       = 0x28
-	TCP_MAXSEG                        = 0x2
-	TCP_MAXWIN                        = 0xffff
-	TCP_MAX_SACK                      = 0x4
-	TCP_MAX_WINSHIFT                  = 0xe
-	TCP_MINMSS                        = 0xd8
-	TCP_MSS                           = 0x200
-	TCP_NODELAY                       = 0x1
-	TCP_NOOPT                         = 0x8
-	TCP_NOPUSH                        = 0x4
-	TCP_NOTSENT_LOWAT                 = 0x201
-	TCP_RXT_CONNDROPTIME              = 0x80
-	TCP_RXT_FINDROP                   = 0x100
-	TCP_SENDMOREACKS                  = 0x103
-	TCSAFLUSH                         = 0x2
-	TIOCCBRK                          = 0x2000747a
-	TIOCCDTR                          = 0x20007478
-	TIOCCONS                          = 0x80047462
-	TIOCDCDTIMESTAMP                  = 0x40087458
-	TIOCDRAIN                         = 0x2000745e
-	TIOCDSIMICROCODE                  = 0x20007455
-	TIOCEXCL                          = 0x2000740d
-	TIOCEXT                           = 0x80047460
-	TIOCFLUSH                         = 0x80047410
-	TIOCGDRAINWAIT                    = 0x40047456
-	TIOCGETA                          = 0x402c7413
-	TIOCGETD                          = 0x4004741a
-	TIOCGPGRP                         = 0x40047477
-	TIOCGWINSZ                        = 0x40087468
-	TIOCIXOFF                         = 0x20007480
-	TIOCIXON                          = 0x20007481
-	TIOCMBIC                          = 0x8004746b
-	TIOCMBIS                          = 0x8004746c
-	TIOCMGDTRWAIT                     = 0x4004745a
-	TIOCMGET                          = 0x4004746a
-	TIOCMODG                          = 0x40047403
-	TIOCMODS                          = 0x80047404
-	TIOCMSDTRWAIT                     = 0x8004745b
-	TIOCMSET                          = 0x8004746d
-	TIOCM_CAR                         = 0x40
-	TIOCM_CD                          = 0x40
-	TIOCM_CTS                         = 0x20
-	TIOCM_DSR                         = 0x100
-	TIOCM_DTR                         = 0x2
-	TIOCM_LE                          = 0x1
-	TIOCM_RI                          = 0x80
-	TIOCM_RNG                         = 0x80
-	TIOCM_RTS                         = 0x4
-	TIOCM_SR                          = 0x10
-	TIOCM_ST                          = 0x8
-	TIOCNOTTY                         = 0x20007471
-	TIOCNXCL                          = 0x2000740e
-	TIOCOUTQ                          = 0x40047473
-	TIOCPKT                           = 0x80047470
-	TIOCPKT_DATA                      = 0x0
-	TIOCPKT_DOSTOP                    = 0x20
-	TIOCPKT_FLUSHREAD                 = 0x1
-	TIOCPKT_FLUSHWRITE                = 0x2
-	TIOCPKT_IOCTL                     = 0x40
-	TIOCPKT_NOSTOP                    = 0x10
-	TIOCPKT_START                     = 0x8
-	TIOCPKT_STOP                      = 0x4
-	TIOCPTYGNAME                      = 0x40807453
-	TIOCPTYGRANT                      = 0x20007454
-	TIOCPTYUNLK                       = 0x20007452
-	TIOCREMOTE                        = 0x80047469
-	TIOCSBRK                          = 0x2000747b
-	TIOCSCONS                         = 0x20007463
-	TIOCSCTTY                         = 0x20007461
-	TIOCSDRAINWAIT                    = 0x80047457
-	TIOCSDTR                          = 0x20007479
-	TIOCSETA                          = 0x802c7414
-	TIOCSETAF                         = 0x802c7416
-	TIOCSETAW                         = 0x802c7415
-	TIOCSETD                          = 0x8004741b
-	TIOCSIG                           = 0x2000745f
-	TIOCSPGRP                         = 0x80047476
-	TIOCSTART                         = 0x2000746e
-	TIOCSTAT                          = 0x20007465
-	TIOCSTI                           = 0x80017472
-	TIOCSTOP                          = 0x2000746f
-	TIOCSWINSZ                        = 0x80087467
-	TIOCTIMESTAMP                     = 0x40087459
-	TIOCUCNTL                         = 0x80047466
-	TOSTOP                            = 0x400000
-	VDISCARD                          = 0xf
-	VDSUSP                            = 0xb
-	VEOF                              = 0x0
-	VEOL                              = 0x1
-	VEOL2                             = 0x2
-	VERASE                            = 0x3
-	VINTR                             = 0x8
-	VKILL                             = 0x5
-	VLNEXT                            = 0xe
-	VMIN                              = 0x10
-	VM_LOADAVG                        = 0x2
-	VM_MACHFACTOR                     = 0x4
-	VM_MAXID                          = 0x6
-	VM_METER                          = 0x1
-	VM_SWAPUSAGE                      = 0x5
-	VQUIT                             = 0x9
-	VREPRINT                          = 0x6
-	VSTART                            = 0xc
-	VSTATUS                           = 0x12
-	VSTOP                             = 0xd
-	VSUSP                             = 0xa
-	VT0                               = 0x0
-	VT1                               = 0x10000
-	VTDLY                             = 0x10000
-	VTIME                             = 0x11
-	VWERASE                           = 0x4
-	WCONTINUED                        = 0x10
-	WCOREFLAG                         = 0x80
-	WEXITED                           = 0x4
-	WNOHANG                           = 0x1
-	WNOWAIT                           = 0x20
-	WORDSIZE                          = 0x20
-	WSTOPPED                          = 0x8
-	WUNTRACED                         = 0x2
-	XATTR_CREATE                      = 0x2
-	XATTR_NODEFAULT                   = 0x10
-	XATTR_NOFOLLOW                    = 0x1
-	XATTR_NOSECURITY                  = 0x8
-	XATTR_REPLACE                     = 0x4
-	XATTR_SHOWCOMPRESSION             = 0x20
-)
-
-// Errors
-const (
-	E2BIG           = syscall.Errno(0x7)
-	EACCES          = syscall.Errno(0xd)
-	EADDRINUSE      = syscall.Errno(0x30)
-	EADDRNOTAVAIL   = syscall.Errno(0x31)
-	EAFNOSUPPORT    = syscall.Errno(0x2f)
-	EAGAIN          = syscall.Errno(0x23)
-	EALREADY        = syscall.Errno(0x25)
-	EAUTH           = syscall.Errno(0x50)
-	EBADARCH        = syscall.Errno(0x56)
-	EBADEXEC        = syscall.Errno(0x55)
-	EBADF           = syscall.Errno(0x9)
-	EBADMACHO       = syscall.Errno(0x58)
-	EBADMSG         = syscall.Errno(0x5e)
-	EBADRPC         = syscall.Errno(0x48)
-	EBUSY           = syscall.Errno(0x10)
-	ECANCELED       = syscall.Errno(0x59)
-	ECHILD          = syscall.Errno(0xa)
-	ECONNABORTED    = syscall.Errno(0x35)
-	ECONNREFUSED    = syscall.Errno(0x3d)
-	ECONNRESET      = syscall.Errno(0x36)
-	EDEADLK         = syscall.Errno(0xb)
-	EDESTADDRREQ    = syscall.Errno(0x27)
-	EDEVERR         = syscall.Errno(0x53)
-	EDOM            = syscall.Errno(0x21)
-	EDQUOT          = syscall.Errno(0x45)
-	EEXIST          = syscall.Errno(0x11)
-	EFAULT          = syscall.Errno(0xe)
-	EFBIG           = syscall.Errno(0x1b)
-	EFTYPE          = syscall.Errno(0x4f)
-	EHOSTDOWN       = syscall.Errno(0x40)
-	EHOSTUNREACH    = syscall.Errno(0x41)
-	EIDRM           = syscall.Errno(0x5a)
-	EILSEQ          = syscall.Errno(0x5c)
-	EINPROGRESS     = syscall.Errno(0x24)
-	EINTR           = syscall.Errno(0x4)
-	EINVAL          = syscall.Errno(0x16)
-	EIO             = syscall.Errno(0x5)
-	EISCONN         = syscall.Errno(0x38)
-	EISDIR          = syscall.Errno(0x15)
-	ELAST           = syscall.Errno(0x6a)
-	ELOOP           = syscall.Errno(0x3e)
-	EMFILE          = syscall.Errno(0x18)
-	EMLINK          = syscall.Errno(0x1f)
-	EMSGSIZE        = syscall.Errno(0x28)
-	EMULTIHOP       = syscall.Errno(0x5f)
-	ENAMETOOLONG    = syscall.Errno(0x3f)
-	ENEEDAUTH       = syscall.Errno(0x51)
-	ENETDOWN        = syscall.Errno(0x32)
-	ENETRESET       = syscall.Errno(0x34)
-	ENETUNREACH     = syscall.Errno(0x33)
-	ENFILE          = syscall.Errno(0x17)
-	ENOATTR         = syscall.Errno(0x5d)
-	ENOBUFS         = syscall.Errno(0x37)
-	ENODATA         = syscall.Errno(0x60)
-	ENODEV          = syscall.Errno(0x13)
-	ENOENT          = syscall.Errno(0x2)
-	ENOEXEC         = syscall.Errno(0x8)
-	ENOLCK          = syscall.Errno(0x4d)
-	ENOLINK         = syscall.Errno(0x61)
-	ENOMEM          = syscall.Errno(0xc)
-	ENOMSG          = syscall.Errno(0x5b)
-	ENOPOLICY       = syscall.Errno(0x67)
-	ENOPROTOOPT     = syscall.Errno(0x2a)
-	ENOSPC          = syscall.Errno(0x1c)
-	ENOSR           = syscall.Errno(0x62)
-	ENOSTR          = syscall.Errno(0x63)
-	ENOSYS          = syscall.Errno(0x4e)
-	ENOTBLK         = syscall.Errno(0xf)
-	ENOTCONN        = syscall.Errno(0x39)
-	ENOTDIR         = syscall.Errno(0x14)
-	ENOTEMPTY       = syscall.Errno(0x42)
-	ENOTRECOVERABLE = syscall.Errno(0x68)
-	ENOTSOCK        = syscall.Errno(0x26)
-	ENOTSUP         = syscall.Errno(0x2d)
-	ENOTTY          = syscall.Errno(0x19)
-	ENXIO           = syscall.Errno(0x6)
-	EOPNOTSUPP      = syscall.Errno(0x66)
-	EOVERFLOW       = syscall.Errno(0x54)
-	EOWNERDEAD      = syscall.Errno(0x69)
-	EPERM           = syscall.Errno(0x1)
-	EPFNOSUPPORT    = syscall.Errno(0x2e)
-	EPIPE           = syscall.Errno(0x20)
-	EPROCLIM        = syscall.Errno(0x43)
-	EPROCUNAVAIL    = syscall.Errno(0x4c)
-	EPROGMISMATCH   = syscall.Errno(0x4b)
-	EPROGUNAVAIL    = syscall.Errno(0x4a)
-	EPROTO          = syscall.Errno(0x64)
-	EPROTONOSUPPORT = syscall.Errno(0x2b)
-	EPROTOTYPE      = syscall.Errno(0x29)
-	EPWROFF         = syscall.Errno(0x52)
-	EQFULL          = syscall.Errno(0x6a)
-	ERANGE          = syscall.Errno(0x22)
-	EREMOTE         = syscall.Errno(0x47)
-	EROFS           = syscall.Errno(0x1e)
-	ERPCMISMATCH    = syscall.Errno(0x49)
-	ESHLIBVERS      = syscall.Errno(0x57)
-	ESHUTDOWN       = syscall.Errno(0x3a)
-	ESOCKTNOSUPPORT = syscall.Errno(0x2c)
-	ESPIPE          = syscall.Errno(0x1d)
-	ESRCH           = syscall.Errno(0x3)
-	ESTALE          = syscall.Errno(0x46)
-	ETIME           = syscall.Errno(0x65)
-	ETIMEDOUT       = syscall.Errno(0x3c)
-	ETOOMANYREFS    = syscall.Errno(0x3b)
-	ETXTBSY         = syscall.Errno(0x1a)
-	EUSERS          = syscall.Errno(0x44)
-	EWOULDBLOCK     = syscall.Errno(0x23)
-	EXDEV           = syscall.Errno(0x12)
-)
-
-// Signals
-const (
-	SIGABRT   = syscall.Signal(0x6)
-	SIGALRM   = syscall.Signal(0xe)
-	SIGBUS    = syscall.Signal(0xa)
-	SIGCHLD   = syscall.Signal(0x14)
-	SIGCONT   = syscall.Signal(0x13)
-	SIGEMT    = syscall.Signal(0x7)
-	SIGFPE    = syscall.Signal(0x8)
-	SIGHUP    = syscall.Signal(0x1)
-	SIGILL    = syscall.Signal(0x4)
-	SIGINFO   = syscall.Signal(0x1d)
-	SIGINT    = syscall.Signal(0x2)
-	SIGIO     = syscall.Signal(0x17)
-	SIGIOT    = syscall.Signal(0x6)
-	SIGKILL   = syscall.Signal(0x9)
-	SIGPIPE   = syscall.Signal(0xd)
-	SIGPROF   = syscall.Signal(0x1b)
-	SIGQUIT   = syscall.Signal(0x3)
-	SIGSEGV   = syscall.Signal(0xb)
-	SIGSTOP   = syscall.Signal(0x11)
-	SIGSYS    = syscall.Signal(0xc)
-	SIGTERM   = syscall.Signal(0xf)
-	SIGTRAP   = syscall.Signal(0x5)
-	SIGTSTP   = syscall.Signal(0x12)
-	SIGTTIN   = syscall.Signal(0x15)
-	SIGTTOU   = syscall.Signal(0x16)
-	SIGURG    = syscall.Signal(0x10)
-	SIGUSR1   = syscall.Signal(0x1e)
-	SIGUSR2   = syscall.Signal(0x1f)
-	SIGVTALRM = syscall.Signal(0x1a)
-	SIGWINCH  = syscall.Signal(0x1c)
-	SIGXCPU   = syscall.Signal(0x18)
-	SIGXFSZ   = syscall.Signal(0x19)
-)
-
-// Error table
-var errorList = [...]struct {
-	num  syscall.Errno
-	name string
-	desc string
-}{
-	{1, "EPERM", "operation not permitted"},
-	{2, "ENOENT", "no such file or directory"},
-	{3, "ESRCH", "no such process"},
-	{4, "EINTR", "interrupted system call"},
-	{5, "EIO", "input/output error"},
-	{6, "ENXIO", "device not configured"},
-	{7, "E2BIG", "argument list too long"},
-	{8, "ENOEXEC", "exec format error"},
-	{9, "EBADF", "bad file descriptor"},
-	{10, "ECHILD", "no child processes"},
-	{11, "EDEADLK", "resource deadlock avoided"},
-	{12, "ENOMEM", "cannot allocate memory"},
-	{13, "EACCES", "permission denied"},
-	{14, "EFAULT", "bad address"},
-	{15, "ENOTBLK", "block device required"},
-	{16, "EBUSY", "resource busy"},
-	{17, "EEXIST", "file exists"},
-	{18, "EXDEV", "cross-device link"},
-	{19, "ENODEV", "operation not supported by device"},
-	{20, "ENOTDIR", "not a directory"},
-	{21, "EISDIR", "is a directory"},
-	{22, "EINVAL", "invalid argument"},
-	{23, "ENFILE", "too many open files in system"},
-	{24, "EMFILE", "too many open files"},
-	{25, "ENOTTY", "inappropriate ioctl for device"},
-	{26, "ETXTBSY", "text file busy"},
-	{27, "EFBIG", "file too large"},
-	{28, "ENOSPC", "no space left on device"},
-	{29, "ESPIPE", "illegal seek"},
-	{30, "EROFS", "read-only file system"},
-	{31, "EMLINK", "too many links"},
-	{32, "EPIPE", "broken pipe"},
-	{33, "EDOM", "numerical argument out of domain"},
-	{34, "ERANGE", "result too large"},
-	{35, "EAGAIN", "resource temporarily unavailable"},
-	{36, "EINPROGRESS", "operation now in progress"},
-	{37, "EALREADY", "operation already in progress"},
-	{38, "ENOTSOCK", "socket operation on non-socket"},
-	{39, "EDESTADDRREQ", "destination address required"},
-	{40, "EMSGSIZE", "message too long"},
-	{41, "EPROTOTYPE", "protocol wrong type for socket"},
-	{42, "ENOPROTOOPT", "protocol not available"},
-	{43, "EPROTONOSUPPORT", "protocol not supported"},
-	{44, "ESOCKTNOSUPPORT", "socket type not supported"},
-	{45, "ENOTSUP", "operation not supported"},
-	{46, "EPFNOSUPPORT", "protocol family not supported"},
-	{47, "EAFNOSUPPORT", "address family not supported by protocol family"},
-	{48, "EADDRINUSE", "address already in use"},
-	{49, "EADDRNOTAVAIL", "can't assign requested address"},
-	{50, "ENETDOWN", "network is down"},
-	{51, "ENETUNREACH", "network is unreachable"},
-	{52, "ENETRESET", "network dropped connection on reset"},
-	{53, "ECONNABORTED", "software caused connection abort"},
-	{54, "ECONNRESET", "connection reset by peer"},
-	{55, "ENOBUFS", "no buffer space available"},
-	{56, "EISCONN", "socket is already connected"},
-	{57, "ENOTCONN", "socket is not connected"},
-	{58, "ESHUTDOWN", "can't send after socket shutdown"},
-	{59, "ETOOMANYREFS", "too many references: can't splice"},
-	{60, "ETIMEDOUT", "operation timed out"},
-	{61, "ECONNREFUSED", "connection refused"},
-	{62, "ELOOP", "too many levels of symbolic links"},
-	{63, "ENAMETOOLONG", "file name too long"},
-	{64, "EHOSTDOWN", "host is down"},
-	{65, "EHOSTUNREACH", "no route to host"},
-	{66, "ENOTEMPTY", "directory not empty"},
-	{67, "EPROCLIM", "too many processes"},
-	{68, "EUSERS", "too many users"},
-	{69, "EDQUOT", "disc quota exceeded"},
-	{70, "ESTALE", "stale NFS file handle"},
-	{71, "EREMOTE", "too many levels of remote in path"},
-	{72, "EBADRPC", "RPC struct is bad"},
-	{73, "ERPCMISMATCH", "RPC version wrong"},
-	{74, "EPROGUNAVAIL", "RPC prog. not avail"},
-	{75, "EPROGMISMATCH", "program version wrong"},
-	{76, "EPROCUNAVAIL", "bad procedure for program"},
-	{77, "ENOLCK", "no locks available"},
-	{78, "ENOSYS", "function not implemented"},
-	{79, "EFTYPE", "inappropriate file type or format"},
-	{80, "EAUTH", "authentication error"},
-	{81, "ENEEDAUTH", "need authenticator"},
-	{82, "EPWROFF", "device power is off"},
-	{83, "EDEVERR", "device error"},
-	{84, "EOVERFLOW", "value too large to be stored in data type"},
-	{85, "EBADEXEC", "bad executable (or shared library)"},
-	{86, "EBADARCH", "bad CPU type in executable"},
-	{87, "ESHLIBVERS", "shared library version mismatch"},
-	{88, "EBADMACHO", "malformed Mach-o file"},
-	{89, "ECANCELED", "operation canceled"},
-	{90, "EIDRM", "identifier removed"},
-	{91, "ENOMSG", "no message of desired type"},
-	{92, "EILSEQ", "illegal byte sequence"},
-	{93, "ENOATTR", "attribute not found"},
-	{94, "EBADMSG", "bad message"},
-	{95, "EMULTIHOP", "EMULTIHOP (Reserved)"},
-	{96, "ENODATA", "no message available on STREAM"},
-	{97, "ENOLINK", "ENOLINK (Reserved)"},
-	{98, "ENOSR", "no STREAM resources"},
-	{99, "ENOSTR", "not a STREAM"},
-	{100, "EPROTO", "protocol error"},
-	{101, "ETIME", "STREAM ioctl timeout"},
-	{102, "EOPNOTSUPP", "operation not supported on socket"},
-	{103, "ENOPOLICY", "policy not found"},
-	{104, "ENOTRECOVERABLE", "state not recoverable"},
-	{105, "EOWNERDEAD", "previous owner died"},
-	{106, "EQFULL", "interface output queue is full"},
-}
-
-// Signal table
-var signalList = [...]struct {
-	num  syscall.Signal
-	name string
-	desc string
-}{
-	{1, "SIGHUP", "hangup"},
-	{2, "SIGINT", "interrupt"},
-	{3, "SIGQUIT", "quit"},
-	{4, "SIGILL", "illegal instruction"},
-	{5, "SIGTRAP", "trace/BPT trap"},
-	{6, "SIGABRT", "abort trap"},
-	{7, "SIGEMT", "EMT trap"},
-	{8, "SIGFPE", "floating point exception"},
-	{9, "SIGKILL", "killed"},
-	{10, "SIGBUS", "bus error"},
-	{11, "SIGSEGV", "segmentation fault"},
-	{12, "SIGSYS", "bad system call"},
-	{13, "SIGPIPE", "broken pipe"},
-	{14, "SIGALRM", "alarm clock"},
-	{15, "SIGTERM", "terminated"},
-	{16, "SIGURG", "urgent I/O condition"},
-	{17, "SIGSTOP", "suspended (signal)"},
-	{18, "SIGTSTP", "suspended"},
-	{19, "SIGCONT", "continued"},
-	{20, "SIGCHLD", "child exited"},
-	{21, "SIGTTIN", "stopped (tty input)"},
-	{22, "SIGTTOU", "stopped (tty output)"},
-	{23, "SIGIO", "I/O possible"},
-	{24, "SIGXCPU", "cputime limit exceeded"},
-	{25, "SIGXFSZ", "filesize limit exceeded"},
-	{26, "SIGVTALRM", "virtual timer expired"},
-	{27, "SIGPROF", "profiling timer expired"},
-	{28, "SIGWINCH", "window size changes"},
-	{29, "SIGINFO", "information request"},
-	{30, "SIGUSR1", "user defined signal 1"},
-	{31, "SIGUSR2", "user defined signal 2"},
-}
diff --git a/vendor/golang.org/x/sys/unix/zerrors_darwin_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_darwin_amd64.go
index db767eb..991996b 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_darwin_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_darwin_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && darwin
 // +build amd64,darwin
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -32,7 +33,7 @@
 	AF_LAT                            = 0xe
 	AF_LINK                           = 0x12
 	AF_LOCAL                          = 0x1
-	AF_MAX                            = 0x28
+	AF_MAX                            = 0x29
 	AF_NATM                           = 0x1f
 	AF_NDRV                           = 0x1b
 	AF_NETBIOS                        = 0x21
@@ -45,9 +46,11 @@
 	AF_SIP                            = 0x18
 	AF_SNA                            = 0xb
 	AF_SYSTEM                         = 0x20
+	AF_SYS_CONTROL                    = 0x2
 	AF_UNIX                           = 0x1
 	AF_UNSPEC                         = 0x0
 	AF_UTUN                           = 0x26
+	AF_VSOCK                          = 0x28
 	ALTWERASE                         = 0x200
 	ATTR_BIT_MAP_COUNT                = 0x5
 	ATTR_CMN_ACCESSMASK               = 0x20000
@@ -82,7 +85,7 @@
 	ATTR_CMN_PAROBJID                 = 0x80
 	ATTR_CMN_RETURNED_ATTRS           = 0x80000000
 	ATTR_CMN_SCRIPT                   = 0x100
-	ATTR_CMN_SETMASK                  = 0x41c7ff00
+	ATTR_CMN_SETMASK                  = 0x51c7ff00
 	ATTR_CMN_USERACCESS               = 0x200000
 	ATTR_CMN_UUID                     = 0x800000
 	ATTR_CMN_VALIDMASK                = 0xffffffff
@@ -251,6 +254,7 @@
 	CSTOP                             = 0x13
 	CSTOPB                            = 0x400
 	CSUSP                             = 0x1a
+	CTLIOCGINFO                       = 0xc0644e03
 	CTL_HW                            = 0x6
 	CTL_KERN                          = 0x1
 	CTL_MAXNAME                       = 0xc
@@ -355,7 +359,7 @@
 	DLT_LINUX_SLL                     = 0x71
 	DLT_LOOP                          = 0x6c
 	DLT_LTALK                         = 0x72
-	DLT_MATCHING_MAX                  = 0xf5
+	DLT_MATCHING_MAX                  = 0x10a
 	DLT_MATCHING_MIN                  = 0x68
 	DLT_MFR                           = 0xb6
 	DLT_MOST                          = 0xd3
@@ -396,6 +400,7 @@
 	DLT_SYMANTEC_FIREWALL             = 0x63
 	DLT_TZSP                          = 0x80
 	DLT_USB                           = 0xba
+	DLT_USB_DARWIN                    = 0x10a
 	DLT_USB_LINUX                     = 0xbd
 	DLT_USB_LINUX_MMAPPED             = 0xdc
 	DLT_USER0                         = 0x93
@@ -440,8 +445,8 @@
 	EVFILT_PROC                       = -0x5
 	EVFILT_READ                       = -0x1
 	EVFILT_SIGNAL                     = -0x6
-	EVFILT_SYSCOUNT                   = 0xf
-	EVFILT_THREADMARKER               = 0xf
+	EVFILT_SYSCOUNT                   = 0x11
+	EVFILT_THREADMARKER               = 0x11
 	EVFILT_TIMER                      = -0x7
 	EVFILT_USER                       = -0xa
 	EVFILT_VM                         = -0xc
@@ -479,9 +484,12 @@
 	FSOPT_NOINMEMUPDATE               = 0x2
 	FSOPT_PACK_INVAL_ATTRS            = 0x8
 	FSOPT_REPORT_FULLSIZE             = 0x4
+	FSOPT_RETURN_REALDEV              = 0x200
 	F_ADDFILESIGS                     = 0x3d
 	F_ADDFILESIGS_FOR_DYLD_SIM        = 0x53
+	F_ADDFILESIGS_INFO                = 0x67
 	F_ADDFILESIGS_RETURN              = 0x61
+	F_ADDFILESUPPL                    = 0x68
 	F_ADDSIGS                         = 0x3b
 	F_ALLOCATEALL                     = 0x4
 	F_ALLOCATECONTIG                  = 0x2
@@ -503,8 +511,10 @@
 	F_GETOWN                          = 0x5
 	F_GETPATH                         = 0x32
 	F_GETPATH_MTMINFO                 = 0x47
+	F_GETPATH_NOFIRMLINK              = 0x66
 	F_GETPROTECTIONCLASS              = 0x3f
 	F_GETPROTECTIONLEVEL              = 0x4d
+	F_GETSIGSINFO                     = 0x69
 	F_GLOBAL_NOCACHE                  = 0x37
 	F_LOG2PHYS                        = 0x31
 	F_LOG2PHYS_EXT                    = 0x41
@@ -529,6 +539,7 @@
 	F_SETPROTECTIONCLASS              = 0x40
 	F_SETSIZE                         = 0x2b
 	F_SINGLE_WRITER                   = 0x4c
+	F_SPECULATIVE_READ                = 0x65
 	F_THAW_FS                         = 0x36
 	F_TRANSCODEKEY                    = 0x4b
 	F_TRIM_ACTIVE_FILE                = 0x64
@@ -560,6 +571,7 @@
 	IFF_UP                            = 0x1
 	IFNAMSIZ                          = 0x10
 	IFT_1822                          = 0x2
+	IFT_6LOWPAN                       = 0x40
 	IFT_AAL5                          = 0x31
 	IFT_ARCNET                        = 0x23
 	IFT_ARCNETPLUS                    = 0x24
@@ -764,16 +776,28 @@
 	IPV6_2292PKTINFO                  = 0x13
 	IPV6_2292PKTOPTIONS               = 0x19
 	IPV6_2292RTHDR                    = 0x18
+	IPV6_3542DSTOPTS                  = 0x32
+	IPV6_3542HOPLIMIT                 = 0x2f
+	IPV6_3542HOPOPTS                  = 0x31
+	IPV6_3542NEXTHOP                  = 0x30
+	IPV6_3542PKTINFO                  = 0x2e
+	IPV6_3542RTHDR                    = 0x33
+	IPV6_ADDR_MC_FLAGS_PREFIX         = 0x20
+	IPV6_ADDR_MC_FLAGS_TRANSIENT      = 0x10
+	IPV6_ADDR_MC_FLAGS_UNICAST_BASED  = 0x30
+	IPV6_AUTOFLOWLABEL                = 0x3b
 	IPV6_BINDV6ONLY                   = 0x1b
 	IPV6_BOUND_IF                     = 0x7d
 	IPV6_CHECKSUM                     = 0x1a
 	IPV6_DEFAULT_MULTICAST_HOPS       = 0x1
 	IPV6_DEFAULT_MULTICAST_LOOP       = 0x1
 	IPV6_DEFHLIM                      = 0x40
+	IPV6_DONTFRAG                     = 0x3e
+	IPV6_DSTOPTS                      = 0x32
 	IPV6_FAITH                        = 0x1d
 	IPV6_FLOWINFO_MASK                = 0xffffff0f
 	IPV6_FLOWLABEL_MASK               = 0xffff0f00
-	IPV6_FLOW_ECN_MASK                = 0x300
+	IPV6_FLOW_ECN_MASK                = 0x3000
 	IPV6_FRAGTTL                      = 0x3c
 	IPV6_FW_ADD                       = 0x1e
 	IPV6_FW_DEL                       = 0x1f
@@ -781,6 +805,8 @@
 	IPV6_FW_GET                       = 0x22
 	IPV6_FW_ZERO                      = 0x21
 	IPV6_HLIMDEC                      = 0x1
+	IPV6_HOPLIMIT                     = 0x2f
+	IPV6_HOPOPTS                      = 0x31
 	IPV6_IPSEC_POLICY                 = 0x1c
 	IPV6_JOIN_GROUP                   = 0xc
 	IPV6_LEAVE_GROUP                  = 0xd
@@ -792,20 +818,34 @@
 	IPV6_MAX_SOCK_SRC_FILTER          = 0x80
 	IPV6_MIN_MEMBERSHIPS              = 0x1f
 	IPV6_MMTU                         = 0x500
+	IPV6_MSFILTER                     = 0x4a
 	IPV6_MULTICAST_HOPS               = 0xa
 	IPV6_MULTICAST_IF                 = 0x9
 	IPV6_MULTICAST_LOOP               = 0xb
+	IPV6_NEXTHOP                      = 0x30
+	IPV6_PATHMTU                      = 0x2c
+	IPV6_PKTINFO                      = 0x2e
 	IPV6_PORTRANGE                    = 0xe
 	IPV6_PORTRANGE_DEFAULT            = 0x0
 	IPV6_PORTRANGE_HIGH               = 0x1
 	IPV6_PORTRANGE_LOW                = 0x2
+	IPV6_PREFER_TEMPADDR              = 0x3f
+	IPV6_RECVDSTOPTS                  = 0x28
+	IPV6_RECVHOPLIMIT                 = 0x25
+	IPV6_RECVHOPOPTS                  = 0x27
+	IPV6_RECVPATHMTU                  = 0x2b
+	IPV6_RECVPKTINFO                  = 0x3d
+	IPV6_RECVRTHDR                    = 0x26
 	IPV6_RECVTCLASS                   = 0x23
+	IPV6_RTHDR                        = 0x33
+	IPV6_RTHDRDSTOPTS                 = 0x39
 	IPV6_RTHDR_LOOSE                  = 0x0
 	IPV6_RTHDR_STRICT                 = 0x1
 	IPV6_RTHDR_TYPE_0                 = 0x0
 	IPV6_SOCKOPT_RESERVED1            = 0x3
 	IPV6_TCLASS                       = 0x24
 	IPV6_UNICAST_HOPS                 = 0x4
+	IPV6_USE_MIN_MTU                  = 0x2a
 	IPV6_V6ONLY                       = 0x1b
 	IPV6_VERSION                      = 0x60
 	IPV6_VERSION_MASK                 = 0xf0
@@ -816,6 +856,7 @@
 	IP_DEFAULT_MULTICAST_LOOP         = 0x1
 	IP_DEFAULT_MULTICAST_TTL          = 0x1
 	IP_DF                             = 0x4000
+	IP_DONTFRAG                       = 0x1c
 	IP_DROP_MEMBERSHIP                = 0xd
 	IP_DROP_SOURCE_MEMBERSHIP         = 0x47
 	IP_DUMMYNET_CONFIGURE             = 0x3c
@@ -887,6 +928,12 @@
 	KERN_OSRELEASE                    = 0x2
 	KERN_OSTYPE                       = 0x1
 	KERN_VERSION                      = 0x4
+	LOCAL_PEERCRED                    = 0x1
+	LOCAL_PEEREPID                    = 0x3
+	LOCAL_PEEREUUID                   = 0x5
+	LOCAL_PEERPID                     = 0x2
+	LOCAL_PEERTOKEN                   = 0x6
+	LOCAL_PEERUUID                    = 0x4
 	LOCK_EX                           = 0x2
 	LOCK_NB                           = 0x4
 	LOCK_SH                           = 0x1
@@ -902,6 +949,7 @@
 	MADV_SEQUENTIAL                   = 0x2
 	MADV_WILLNEED                     = 0x3
 	MADV_ZERO_WIRED_PAGES             = 0x6
+	MAP_32BIT                         = 0x8000
 	MAP_ANON                          = 0x1000
 	MAP_ANONYMOUS                     = 0x1000
 	MAP_COPY                          = 0x2
@@ -918,6 +966,17 @@
 	MAP_RESILIENT_CODESIGN            = 0x2000
 	MAP_RESILIENT_MEDIA               = 0x4000
 	MAP_SHARED                        = 0x1
+	MAP_TRANSLATED_ALLOW_EXECUTE      = 0x20000
+	MAP_UNIX03                        = 0x40000
+	MCAST_BLOCK_SOURCE                = 0x54
+	MCAST_EXCLUDE                     = 0x2
+	MCAST_INCLUDE                     = 0x1
+	MCAST_JOIN_GROUP                  = 0x50
+	MCAST_JOIN_SOURCE_GROUP           = 0x52
+	MCAST_LEAVE_GROUP                 = 0x51
+	MCAST_LEAVE_SOURCE_GROUP          = 0x53
+	MCAST_UNBLOCK_SOURCE              = 0x55
+	MCAST_UNDEFINED                   = 0x0
 	MCL_CURRENT                       = 0x1
 	MCL_FUTURE                        = 0x2
 	MNT_ASYNC                         = 0x40
@@ -929,6 +988,7 @@
 	MNT_DOVOLFS                       = 0x8000
 	MNT_DWAIT                         = 0x4
 	MNT_EXPORTED                      = 0x100
+	MNT_EXT_ROOT_DATA_VOL             = 0x1
 	MNT_FORCE                         = 0x80000
 	MNT_IGNORE_OWNERSHIP              = 0x200000
 	MNT_JOURNALED                     = 0x800000
@@ -945,12 +1005,15 @@
 	MNT_QUOTA                         = 0x2000
 	MNT_RDONLY                        = 0x1
 	MNT_RELOAD                        = 0x40000
+	MNT_REMOVABLE                     = 0x200
 	MNT_ROOTFS                        = 0x4000
+	MNT_SNAPSHOT                      = 0x40000000
+	MNT_STRICTATIME                   = 0x80000000
 	MNT_SYNCHRONOUS                   = 0x2
 	MNT_UNION                         = 0x20
 	MNT_UNKNOWNPERMISSIONS            = 0x200000
 	MNT_UPDATE                        = 0x10000
-	MNT_VISFLAGMASK                   = 0x17f0f5ff
+	MNT_VISFLAGMASK                   = 0xd7f0f7ff
 	MNT_WAIT                          = 0x1
 	MSG_CTRUNC                        = 0x20
 	MSG_DONTROUTE                     = 0x4
@@ -961,6 +1024,7 @@
 	MSG_HAVEMORE                      = 0x2000
 	MSG_HOLD                          = 0x800
 	MSG_NEEDSA                        = 0x10000
+	MSG_NOSIGNAL                      = 0x80000
 	MSG_OOB                           = 0x1
 	MSG_PEEK                          = 0x2
 	MSG_RCVMORE                       = 0x4000
@@ -977,9 +1041,10 @@
 	NET_RT_DUMP                       = 0x1
 	NET_RT_DUMP2                      = 0x7
 	NET_RT_FLAGS                      = 0x2
+	NET_RT_FLAGS_PRIV                 = 0xa
 	NET_RT_IFLIST                     = 0x3
 	NET_RT_IFLIST2                    = 0x6
-	NET_RT_MAXID                      = 0xa
+	NET_RT_MAXID                      = 0xb
 	NET_RT_STAT                       = 0x4
 	NET_RT_TRASH                      = 0x5
 	NFDBITS                           = 0x20
@@ -1017,6 +1082,7 @@
 	NOTE_LEEWAY                       = 0x10
 	NOTE_LINK                         = 0x10
 	NOTE_LOWAT                        = 0x1
+	NOTE_MACHTIME                     = 0x100
 	NOTE_MACH_CONTINUOUS_TIME         = 0x80
 	NOTE_NONE                         = 0x80
 	NOTE_NSECONDS                     = 0x4
@@ -1063,6 +1129,7 @@
 	O_NDELAY                          = 0x4
 	O_NOCTTY                          = 0x20000
 	O_NOFOLLOW                        = 0x100
+	O_NOFOLLOW_ANY                    = 0x20000000
 	O_NONBLOCK                        = 0x4
 	O_POPUP                           = 0x80000000
 	O_RDONLY                          = 0x0
@@ -1134,6 +1201,7 @@
 	RTF_BROADCAST                     = 0x400000
 	RTF_CLONING                       = 0x100
 	RTF_CONDEMNED                     = 0x2000000
+	RTF_DEAD                          = 0x20000000
 	RTF_DELCLONE                      = 0x80
 	RTF_DONE                          = 0x40
 	RTF_DYNAMIC                       = 0x10
@@ -1141,6 +1209,7 @@
 	RTF_HOST                          = 0x4
 	RTF_IFREF                         = 0x4000000
 	RTF_IFSCOPE                       = 0x1000000
+	RTF_LLDATA                        = 0x400
 	RTF_LLINFO                        = 0x400
 	RTF_LOCAL                         = 0x200000
 	RTF_MODIFIED                      = 0x20
@@ -1208,6 +1277,7 @@
 	SIOCGDRVSPEC                      = 0xc028697b
 	SIOCGETVLAN                       = 0xc020697f
 	SIOCGHIWAT                        = 0x40047301
+	SIOCGIF6LOWPAN                    = 0xc02069c5
 	SIOCGIFADDR                       = 0xc0206921
 	SIOCGIFALTMTU                     = 0xc0206948
 	SIOCGIFASYNCMAP                   = 0xc020697c
@@ -1218,6 +1288,7 @@
 	SIOCGIFDEVMTU                     = 0xc0206944
 	SIOCGIFDSTADDR                    = 0xc0206922
 	SIOCGIFFLAGS                      = 0xc0206911
+	SIOCGIFFUNCTIONALTYPE             = 0xc02069ad
 	SIOCGIFGENERIC                    = 0xc020693a
 	SIOCGIFKPI                        = 0xc0206987
 	SIOCGIFMAC                        = 0xc0206982
@@ -1231,6 +1302,7 @@
 	SIOCGIFSTATUS                     = 0xc331693d
 	SIOCGIFVLAN                       = 0xc020697f
 	SIOCGIFWAKEFLAGS                  = 0xc0206988
+	SIOCGIFXMEDIA                     = 0xc02c6948
 	SIOCGLOWAT                        = 0x40047303
 	SIOCGPGRP                         = 0x40047309
 	SIOCIFCREATE                      = 0xc0206978
@@ -1241,6 +1313,7 @@
 	SIOCSDRVSPEC                      = 0x8028697b
 	SIOCSETVLAN                       = 0x8020697e
 	SIOCSHIWAT                        = 0x80047300
+	SIOCSIF6LOWPAN                    = 0x802069c4
 	SIOCSIFADDR                       = 0x8020690c
 	SIOCSIFALTMTU                     = 0x80206945
 	SIOCSIFASYNCMAP                   = 0x8020697d
@@ -1268,6 +1341,7 @@
 	SOCK_RDM                          = 0x4
 	SOCK_SEQPACKET                    = 0x5
 	SOCK_STREAM                       = 0x1
+	SOL_LOCAL                         = 0x0
 	SOL_SOCKET                        = 0xffff
 	SOMAXCONN                         = 0x80
 	SO_ACCEPTCONN                     = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_darwin_arm.go b/vendor/golang.org/x/sys/unix/zerrors_darwin_arm.go
deleted file mode 100644
index ddc5d00..0000000
--- a/vendor/golang.org/x/sys/unix/zerrors_darwin_arm.go
+++ /dev/null
@@ -1,1786 +0,0 @@
-// mkerrors.sh
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build arm,darwin
-
-// Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- _const.go
-
-package unix
-
-import "syscall"
-
-const (
-	AF_APPLETALK                      = 0x10
-	AF_CCITT                          = 0xa
-	AF_CHAOS                          = 0x5
-	AF_CNT                            = 0x15
-	AF_COIP                           = 0x14
-	AF_DATAKIT                        = 0x9
-	AF_DECnet                         = 0xc
-	AF_DLI                            = 0xd
-	AF_E164                           = 0x1c
-	AF_ECMA                           = 0x8
-	AF_HYLINK                         = 0xf
-	AF_IEEE80211                      = 0x25
-	AF_IMPLINK                        = 0x3
-	AF_INET                           = 0x2
-	AF_INET6                          = 0x1e
-	AF_IPX                            = 0x17
-	AF_ISDN                           = 0x1c
-	AF_ISO                            = 0x7
-	AF_LAT                            = 0xe
-	AF_LINK                           = 0x12
-	AF_LOCAL                          = 0x1
-	AF_MAX                            = 0x28
-	AF_NATM                           = 0x1f
-	AF_NDRV                           = 0x1b
-	AF_NETBIOS                        = 0x21
-	AF_NS                             = 0x6
-	AF_OSI                            = 0x7
-	AF_PPP                            = 0x22
-	AF_PUP                            = 0x4
-	AF_RESERVED_36                    = 0x24
-	AF_ROUTE                          = 0x11
-	AF_SIP                            = 0x18
-	AF_SNA                            = 0xb
-	AF_SYSTEM                         = 0x20
-	AF_UNIX                           = 0x1
-	AF_UNSPEC                         = 0x0
-	AF_UTUN                           = 0x26
-	ALTWERASE                         = 0x200
-	ATTR_BIT_MAP_COUNT                = 0x5
-	ATTR_CMN_ACCESSMASK               = 0x20000
-	ATTR_CMN_ACCTIME                  = 0x1000
-	ATTR_CMN_ADDEDTIME                = 0x10000000
-	ATTR_CMN_BKUPTIME                 = 0x2000
-	ATTR_CMN_CHGTIME                  = 0x800
-	ATTR_CMN_CRTIME                   = 0x200
-	ATTR_CMN_DATA_PROTECT_FLAGS       = 0x40000000
-	ATTR_CMN_DEVID                    = 0x2
-	ATTR_CMN_DOCUMENT_ID              = 0x100000
-	ATTR_CMN_ERROR                    = 0x20000000
-	ATTR_CMN_EXTENDED_SECURITY        = 0x400000
-	ATTR_CMN_FILEID                   = 0x2000000
-	ATTR_CMN_FLAGS                    = 0x40000
-	ATTR_CMN_FNDRINFO                 = 0x4000
-	ATTR_CMN_FSID                     = 0x4
-	ATTR_CMN_FULLPATH                 = 0x8000000
-	ATTR_CMN_GEN_COUNT                = 0x80000
-	ATTR_CMN_GRPID                    = 0x10000
-	ATTR_CMN_GRPUUID                  = 0x1000000
-	ATTR_CMN_MODTIME                  = 0x400
-	ATTR_CMN_NAME                     = 0x1
-	ATTR_CMN_NAMEDATTRCOUNT           = 0x80000
-	ATTR_CMN_NAMEDATTRLIST            = 0x100000
-	ATTR_CMN_OBJID                    = 0x20
-	ATTR_CMN_OBJPERMANENTID           = 0x40
-	ATTR_CMN_OBJTAG                   = 0x10
-	ATTR_CMN_OBJTYPE                  = 0x8
-	ATTR_CMN_OWNERID                  = 0x8000
-	ATTR_CMN_PARENTID                 = 0x4000000
-	ATTR_CMN_PAROBJID                 = 0x80
-	ATTR_CMN_RETURNED_ATTRS           = 0x80000000
-	ATTR_CMN_SCRIPT                   = 0x100
-	ATTR_CMN_SETMASK                  = 0x41c7ff00
-	ATTR_CMN_USERACCESS               = 0x200000
-	ATTR_CMN_UUID                     = 0x800000
-	ATTR_CMN_VALIDMASK                = 0xffffffff
-	ATTR_CMN_VOLSETMASK               = 0x6700
-	ATTR_FILE_ALLOCSIZE               = 0x4
-	ATTR_FILE_CLUMPSIZE               = 0x10
-	ATTR_FILE_DATAALLOCSIZE           = 0x400
-	ATTR_FILE_DATAEXTENTS             = 0x800
-	ATTR_FILE_DATALENGTH              = 0x200
-	ATTR_FILE_DEVTYPE                 = 0x20
-	ATTR_FILE_FILETYPE                = 0x40
-	ATTR_FILE_FORKCOUNT               = 0x80
-	ATTR_FILE_FORKLIST                = 0x100
-	ATTR_FILE_IOBLOCKSIZE             = 0x8
-	ATTR_FILE_LINKCOUNT               = 0x1
-	ATTR_FILE_RSRCALLOCSIZE           = 0x2000
-	ATTR_FILE_RSRCEXTENTS             = 0x4000
-	ATTR_FILE_RSRCLENGTH              = 0x1000
-	ATTR_FILE_SETMASK                 = 0x20
-	ATTR_FILE_TOTALSIZE               = 0x2
-	ATTR_FILE_VALIDMASK               = 0x37ff
-	ATTR_VOL_ALLOCATIONCLUMP          = 0x40
-	ATTR_VOL_ATTRIBUTES               = 0x40000000
-	ATTR_VOL_CAPABILITIES             = 0x20000
-	ATTR_VOL_DIRCOUNT                 = 0x400
-	ATTR_VOL_ENCODINGSUSED            = 0x10000
-	ATTR_VOL_FILECOUNT                = 0x200
-	ATTR_VOL_FSTYPE                   = 0x1
-	ATTR_VOL_INFO                     = 0x80000000
-	ATTR_VOL_IOBLOCKSIZE              = 0x80
-	ATTR_VOL_MAXOBJCOUNT              = 0x800
-	ATTR_VOL_MINALLOCATION            = 0x20
-	ATTR_VOL_MOUNTEDDEVICE            = 0x8000
-	ATTR_VOL_MOUNTFLAGS               = 0x4000
-	ATTR_VOL_MOUNTPOINT               = 0x1000
-	ATTR_VOL_NAME                     = 0x2000
-	ATTR_VOL_OBJCOUNT                 = 0x100
-	ATTR_VOL_QUOTA_SIZE               = 0x10000000
-	ATTR_VOL_RESERVED_SIZE            = 0x20000000
-	ATTR_VOL_SETMASK                  = 0x80002000
-	ATTR_VOL_SIGNATURE                = 0x2
-	ATTR_VOL_SIZE                     = 0x4
-	ATTR_VOL_SPACEAVAIL               = 0x10
-	ATTR_VOL_SPACEFREE                = 0x8
-	ATTR_VOL_UUID                     = 0x40000
-	ATTR_VOL_VALIDMASK                = 0xf007ffff
-	B0                                = 0x0
-	B110                              = 0x6e
-	B115200                           = 0x1c200
-	B1200                             = 0x4b0
-	B134                              = 0x86
-	B14400                            = 0x3840
-	B150                              = 0x96
-	B1800                             = 0x708
-	B19200                            = 0x4b00
-	B200                              = 0xc8
-	B230400                           = 0x38400
-	B2400                             = 0x960
-	B28800                            = 0x7080
-	B300                              = 0x12c
-	B38400                            = 0x9600
-	B4800                             = 0x12c0
-	B50                               = 0x32
-	B57600                            = 0xe100
-	B600                              = 0x258
-	B7200                             = 0x1c20
-	B75                               = 0x4b
-	B76800                            = 0x12c00
-	B9600                             = 0x2580
-	BIOCFLUSH                         = 0x20004268
-	BIOCGBLEN                         = 0x40044266
-	BIOCGDLT                          = 0x4004426a
-	BIOCGDLTLIST                      = 0xc00c4279
-	BIOCGETIF                         = 0x4020426b
-	BIOCGHDRCMPLT                     = 0x40044274
-	BIOCGRSIG                         = 0x40044272
-	BIOCGRTIMEOUT                     = 0x4010426e
-	BIOCGSEESENT                      = 0x40044276
-	BIOCGSTATS                        = 0x4008426f
-	BIOCIMMEDIATE                     = 0x80044270
-	BIOCPROMISC                       = 0x20004269
-	BIOCSBLEN                         = 0xc0044266
-	BIOCSDLT                          = 0x80044278
-	BIOCSETF                          = 0x80104267
-	BIOCSETFNR                        = 0x8010427e
-	BIOCSETIF                         = 0x8020426c
-	BIOCSHDRCMPLT                     = 0x80044275
-	BIOCSRSIG                         = 0x80044273
-	BIOCSRTIMEOUT                     = 0x8010426d
-	BIOCSSEESENT                      = 0x80044277
-	BIOCVERSION                       = 0x40044271
-	BPF_A                             = 0x10
-	BPF_ABS                           = 0x20
-	BPF_ADD                           = 0x0
-	BPF_ALIGNMENT                     = 0x4
-	BPF_ALU                           = 0x4
-	BPF_AND                           = 0x50
-	BPF_B                             = 0x10
-	BPF_DIV                           = 0x30
-	BPF_H                             = 0x8
-	BPF_IMM                           = 0x0
-	BPF_IND                           = 0x40
-	BPF_JA                            = 0x0
-	BPF_JEQ                           = 0x10
-	BPF_JGE                           = 0x30
-	BPF_JGT                           = 0x20
-	BPF_JMP                           = 0x5
-	BPF_JSET                          = 0x40
-	BPF_K                             = 0x0
-	BPF_LD                            = 0x0
-	BPF_LDX                           = 0x1
-	BPF_LEN                           = 0x80
-	BPF_LSH                           = 0x60
-	BPF_MAJOR_VERSION                 = 0x1
-	BPF_MAXBUFSIZE                    = 0x80000
-	BPF_MAXINSNS                      = 0x200
-	BPF_MEM                           = 0x60
-	BPF_MEMWORDS                      = 0x10
-	BPF_MINBUFSIZE                    = 0x20
-	BPF_MINOR_VERSION                 = 0x1
-	BPF_MISC                          = 0x7
-	BPF_MSH                           = 0xa0
-	BPF_MUL                           = 0x20
-	BPF_NEG                           = 0x80
-	BPF_OR                            = 0x40
-	BPF_RELEASE                       = 0x30bb6
-	BPF_RET                           = 0x6
-	BPF_RSH                           = 0x70
-	BPF_ST                            = 0x2
-	BPF_STX                           = 0x3
-	BPF_SUB                           = 0x10
-	BPF_TAX                           = 0x0
-	BPF_TXA                           = 0x80
-	BPF_W                             = 0x0
-	BPF_X                             = 0x8
-	BRKINT                            = 0x2
-	BS0                               = 0x0
-	BS1                               = 0x8000
-	BSDLY                             = 0x8000
-	CFLUSH                            = 0xf
-	CLOCAL                            = 0x8000
-	CLOCK_MONOTONIC                   = 0x6
-	CLOCK_MONOTONIC_RAW               = 0x4
-	CLOCK_MONOTONIC_RAW_APPROX        = 0x5
-	CLOCK_PROCESS_CPUTIME_ID          = 0xc
-	CLOCK_REALTIME                    = 0x0
-	CLOCK_THREAD_CPUTIME_ID           = 0x10
-	CLOCK_UPTIME_RAW                  = 0x8
-	CLOCK_UPTIME_RAW_APPROX           = 0x9
-	CLONE_NOFOLLOW                    = 0x1
-	CLONE_NOOWNERCOPY                 = 0x2
-	CR0                               = 0x0
-	CR1                               = 0x1000
-	CR2                               = 0x2000
-	CR3                               = 0x3000
-	CRDLY                             = 0x3000
-	CREAD                             = 0x800
-	CRTSCTS                           = 0x30000
-	CS5                               = 0x0
-	CS6                               = 0x100
-	CS7                               = 0x200
-	CS8                               = 0x300
-	CSIZE                             = 0x300
-	CSTART                            = 0x11
-	CSTATUS                           = 0x14
-	CSTOP                             = 0x13
-	CSTOPB                            = 0x400
-	CSUSP                             = 0x1a
-	CTL_HW                            = 0x6
-	CTL_KERN                          = 0x1
-	CTL_MAXNAME                       = 0xc
-	CTL_NET                           = 0x4
-	DLT_A429                          = 0xb8
-	DLT_A653_ICM                      = 0xb9
-	DLT_AIRONET_HEADER                = 0x78
-	DLT_AOS                           = 0xde
-	DLT_APPLE_IP_OVER_IEEE1394        = 0x8a
-	DLT_ARCNET                        = 0x7
-	DLT_ARCNET_LINUX                  = 0x81
-	DLT_ATM_CLIP                      = 0x13
-	DLT_ATM_RFC1483                   = 0xb
-	DLT_AURORA                        = 0x7e
-	DLT_AX25                          = 0x3
-	DLT_AX25_KISS                     = 0xca
-	DLT_BACNET_MS_TP                  = 0xa5
-	DLT_BLUETOOTH_HCI_H4              = 0xbb
-	DLT_BLUETOOTH_HCI_H4_WITH_PHDR    = 0xc9
-	DLT_CAN20B                        = 0xbe
-	DLT_CAN_SOCKETCAN                 = 0xe3
-	DLT_CHAOS                         = 0x5
-	DLT_CHDLC                         = 0x68
-	DLT_CISCO_IOS                     = 0x76
-	DLT_C_HDLC                        = 0x68
-	DLT_C_HDLC_WITH_DIR               = 0xcd
-	DLT_DBUS                          = 0xe7
-	DLT_DECT                          = 0xdd
-	DLT_DOCSIS                        = 0x8f
-	DLT_DVB_CI                        = 0xeb
-	DLT_ECONET                        = 0x73
-	DLT_EN10MB                        = 0x1
-	DLT_EN3MB                         = 0x2
-	DLT_ENC                           = 0x6d
-	DLT_ERF                           = 0xc5
-	DLT_ERF_ETH                       = 0xaf
-	DLT_ERF_POS                       = 0xb0
-	DLT_FC_2                          = 0xe0
-	DLT_FC_2_WITH_FRAME_DELIMS        = 0xe1
-	DLT_FDDI                          = 0xa
-	DLT_FLEXRAY                       = 0xd2
-	DLT_FRELAY                        = 0x6b
-	DLT_FRELAY_WITH_DIR               = 0xce
-	DLT_GCOM_SERIAL                   = 0xad
-	DLT_GCOM_T1E1                     = 0xac
-	DLT_GPF_F                         = 0xab
-	DLT_GPF_T                         = 0xaa
-	DLT_GPRS_LLC                      = 0xa9
-	DLT_GSMTAP_ABIS                   = 0xda
-	DLT_GSMTAP_UM                     = 0xd9
-	DLT_HHDLC                         = 0x79
-	DLT_IBM_SN                        = 0x92
-	DLT_IBM_SP                        = 0x91
-	DLT_IEEE802                       = 0x6
-	DLT_IEEE802_11                    = 0x69
-	DLT_IEEE802_11_RADIO              = 0x7f
-	DLT_IEEE802_11_RADIO_AVS          = 0xa3
-	DLT_IEEE802_15_4                  = 0xc3
-	DLT_IEEE802_15_4_LINUX            = 0xbf
-	DLT_IEEE802_15_4_NOFCS            = 0xe6
-	DLT_IEEE802_15_4_NONASK_PHY       = 0xd7
-	DLT_IEEE802_16_MAC_CPS            = 0xbc
-	DLT_IEEE802_16_MAC_CPS_RADIO      = 0xc1
-	DLT_IPFILTER                      = 0x74
-	DLT_IPMB                          = 0xc7
-	DLT_IPMB_LINUX                    = 0xd1
-	DLT_IPNET                         = 0xe2
-	DLT_IPOIB                         = 0xf2
-	DLT_IPV4                          = 0xe4
-	DLT_IPV6                          = 0xe5
-	DLT_IP_OVER_FC                    = 0x7a
-	DLT_JUNIPER_ATM1                  = 0x89
-	DLT_JUNIPER_ATM2                  = 0x87
-	DLT_JUNIPER_ATM_CEMIC             = 0xee
-	DLT_JUNIPER_CHDLC                 = 0xb5
-	DLT_JUNIPER_ES                    = 0x84
-	DLT_JUNIPER_ETHER                 = 0xb2
-	DLT_JUNIPER_FIBRECHANNEL          = 0xea
-	DLT_JUNIPER_FRELAY                = 0xb4
-	DLT_JUNIPER_GGSN                  = 0x85
-	DLT_JUNIPER_ISM                   = 0xc2
-	DLT_JUNIPER_MFR                   = 0x86
-	DLT_JUNIPER_MLFR                  = 0x83
-	DLT_JUNIPER_MLPPP                 = 0x82
-	DLT_JUNIPER_MONITOR               = 0xa4
-	DLT_JUNIPER_PIC_PEER              = 0xae
-	DLT_JUNIPER_PPP                   = 0xb3
-	DLT_JUNIPER_PPPOE                 = 0xa7
-	DLT_JUNIPER_PPPOE_ATM             = 0xa8
-	DLT_JUNIPER_SERVICES              = 0x88
-	DLT_JUNIPER_SRX_E2E               = 0xe9
-	DLT_JUNIPER_ST                    = 0xc8
-	DLT_JUNIPER_VP                    = 0xb7
-	DLT_JUNIPER_VS                    = 0xe8
-	DLT_LAPB_WITH_DIR                 = 0xcf
-	DLT_LAPD                          = 0xcb
-	DLT_LIN                           = 0xd4
-	DLT_LINUX_EVDEV                   = 0xd8
-	DLT_LINUX_IRDA                    = 0x90
-	DLT_LINUX_LAPD                    = 0xb1
-	DLT_LINUX_PPP_WITHDIRECTION       = 0xa6
-	DLT_LINUX_SLL                     = 0x71
-	DLT_LOOP                          = 0x6c
-	DLT_LTALK                         = 0x72
-	DLT_MATCHING_MAX                  = 0xf5
-	DLT_MATCHING_MIN                  = 0x68
-	DLT_MFR                           = 0xb6
-	DLT_MOST                          = 0xd3
-	DLT_MPEG_2_TS                     = 0xf3
-	DLT_MPLS                          = 0xdb
-	DLT_MTP2                          = 0x8c
-	DLT_MTP2_WITH_PHDR                = 0x8b
-	DLT_MTP3                          = 0x8d
-	DLT_MUX27010                      = 0xec
-	DLT_NETANALYZER                   = 0xf0
-	DLT_NETANALYZER_TRANSPARENT       = 0xf1
-	DLT_NFC_LLCP                      = 0xf5
-	DLT_NFLOG                         = 0xef
-	DLT_NG40                          = 0xf4
-	DLT_NULL                          = 0x0
-	DLT_PCI_EXP                       = 0x7d
-	DLT_PFLOG                         = 0x75
-	DLT_PFSYNC                        = 0x12
-	DLT_PPI                           = 0xc0
-	DLT_PPP                           = 0x9
-	DLT_PPP_BSDOS                     = 0x10
-	DLT_PPP_ETHER                     = 0x33
-	DLT_PPP_PPPD                      = 0xa6
-	DLT_PPP_SERIAL                    = 0x32
-	DLT_PPP_WITH_DIR                  = 0xcc
-	DLT_PPP_WITH_DIRECTION            = 0xa6
-	DLT_PRISM_HEADER                  = 0x77
-	DLT_PRONET                        = 0x4
-	DLT_RAIF1                         = 0xc6
-	DLT_RAW                           = 0xc
-	DLT_RIO                           = 0x7c
-	DLT_SCCP                          = 0x8e
-	DLT_SITA                          = 0xc4
-	DLT_SLIP                          = 0x8
-	DLT_SLIP_BSDOS                    = 0xf
-	DLT_STANAG_5066_D_PDU             = 0xed
-	DLT_SUNATM                        = 0x7b
-	DLT_SYMANTEC_FIREWALL             = 0x63
-	DLT_TZSP                          = 0x80
-	DLT_USB                           = 0xba
-	DLT_USB_LINUX                     = 0xbd
-	DLT_USB_LINUX_MMAPPED             = 0xdc
-	DLT_USER0                         = 0x93
-	DLT_USER1                         = 0x94
-	DLT_USER10                        = 0x9d
-	DLT_USER11                        = 0x9e
-	DLT_USER12                        = 0x9f
-	DLT_USER13                        = 0xa0
-	DLT_USER14                        = 0xa1
-	DLT_USER15                        = 0xa2
-	DLT_USER2                         = 0x95
-	DLT_USER3                         = 0x96
-	DLT_USER4                         = 0x97
-	DLT_USER5                         = 0x98
-	DLT_USER6                         = 0x99
-	DLT_USER7                         = 0x9a
-	DLT_USER8                         = 0x9b
-	DLT_USER9                         = 0x9c
-	DLT_WIHART                        = 0xdf
-	DLT_X2E_SERIAL                    = 0xd5
-	DLT_X2E_XORAYA                    = 0xd6
-	DT_BLK                            = 0x6
-	DT_CHR                            = 0x2
-	DT_DIR                            = 0x4
-	DT_FIFO                           = 0x1
-	DT_LNK                            = 0xa
-	DT_REG                            = 0x8
-	DT_SOCK                           = 0xc
-	DT_UNKNOWN                        = 0x0
-	DT_WHT                            = 0xe
-	ECHO                              = 0x8
-	ECHOCTL                           = 0x40
-	ECHOE                             = 0x2
-	ECHOK                             = 0x4
-	ECHOKE                            = 0x1
-	ECHONL                            = 0x10
-	ECHOPRT                           = 0x20
-	EVFILT_AIO                        = -0x3
-	EVFILT_EXCEPT                     = -0xf
-	EVFILT_FS                         = -0x9
-	EVFILT_MACHPORT                   = -0x8
-	EVFILT_PROC                       = -0x5
-	EVFILT_READ                       = -0x1
-	EVFILT_SIGNAL                     = -0x6
-	EVFILT_SYSCOUNT                   = 0xf
-	EVFILT_THREADMARKER               = 0xf
-	EVFILT_TIMER                      = -0x7
-	EVFILT_USER                       = -0xa
-	EVFILT_VM                         = -0xc
-	EVFILT_VNODE                      = -0x4
-	EVFILT_WRITE                      = -0x2
-	EV_ADD                            = 0x1
-	EV_CLEAR                          = 0x20
-	EV_DELETE                         = 0x2
-	EV_DISABLE                        = 0x8
-	EV_DISPATCH                       = 0x80
-	EV_DISPATCH2                      = 0x180
-	EV_ENABLE                         = 0x4
-	EV_EOF                            = 0x8000
-	EV_ERROR                          = 0x4000
-	EV_FLAG0                          = 0x1000
-	EV_FLAG1                          = 0x2000
-	EV_ONESHOT                        = 0x10
-	EV_OOBAND                         = 0x2000
-	EV_POLL                           = 0x1000
-	EV_RECEIPT                        = 0x40
-	EV_SYSFLAGS                       = 0xf000
-	EV_UDATA_SPECIFIC                 = 0x100
-	EV_VANISHED                       = 0x200
-	EXTA                              = 0x4b00
-	EXTB                              = 0x9600
-	EXTPROC                           = 0x800
-	FD_CLOEXEC                        = 0x1
-	FD_SETSIZE                        = 0x400
-	FF0                               = 0x0
-	FF1                               = 0x4000
-	FFDLY                             = 0x4000
-	FLUSHO                            = 0x800000
-	FSOPT_ATTR_CMN_EXTENDED           = 0x20
-	FSOPT_NOFOLLOW                    = 0x1
-	FSOPT_NOINMEMUPDATE               = 0x2
-	FSOPT_PACK_INVAL_ATTRS            = 0x8
-	FSOPT_REPORT_FULLSIZE             = 0x4
-	F_ADDFILESIGS                     = 0x3d
-	F_ADDFILESIGS_FOR_DYLD_SIM        = 0x53
-	F_ADDFILESIGS_RETURN              = 0x61
-	F_ADDSIGS                         = 0x3b
-	F_ALLOCATEALL                     = 0x4
-	F_ALLOCATECONTIG                  = 0x2
-	F_BARRIERFSYNC                    = 0x55
-	F_CHECK_LV                        = 0x62
-	F_CHKCLEAN                        = 0x29
-	F_DUPFD                           = 0x0
-	F_DUPFD_CLOEXEC                   = 0x43
-	F_FINDSIGS                        = 0x4e
-	F_FLUSH_DATA                      = 0x28
-	F_FREEZE_FS                       = 0x35
-	F_FULLFSYNC                       = 0x33
-	F_GETCODEDIR                      = 0x48
-	F_GETFD                           = 0x1
-	F_GETFL                           = 0x3
-	F_GETLK                           = 0x7
-	F_GETLKPID                        = 0x42
-	F_GETNOSIGPIPE                    = 0x4a
-	F_GETOWN                          = 0x5
-	F_GETPATH                         = 0x32
-	F_GETPATH_MTMINFO                 = 0x47
-	F_GETPROTECTIONCLASS              = 0x3f
-	F_GETPROTECTIONLEVEL              = 0x4d
-	F_GLOBAL_NOCACHE                  = 0x37
-	F_LOG2PHYS                        = 0x31
-	F_LOG2PHYS_EXT                    = 0x41
-	F_NOCACHE                         = 0x30
-	F_NODIRECT                        = 0x3e
-	F_OK                              = 0x0
-	F_PATHPKG_CHECK                   = 0x34
-	F_PEOFPOSMODE                     = 0x3
-	F_PREALLOCATE                     = 0x2a
-	F_PUNCHHOLE                       = 0x63
-	F_RDADVISE                        = 0x2c
-	F_RDAHEAD                         = 0x2d
-	F_RDLCK                           = 0x1
-	F_SETBACKINGSTORE                 = 0x46
-	F_SETFD                           = 0x2
-	F_SETFL                           = 0x4
-	F_SETLK                           = 0x8
-	F_SETLKW                          = 0x9
-	F_SETLKWTIMEOUT                   = 0xa
-	F_SETNOSIGPIPE                    = 0x49
-	F_SETOWN                          = 0x6
-	F_SETPROTECTIONCLASS              = 0x40
-	F_SETSIZE                         = 0x2b
-	F_SINGLE_WRITER                   = 0x4c
-	F_THAW_FS                         = 0x36
-	F_TRANSCODEKEY                    = 0x4b
-	F_TRIM_ACTIVE_FILE                = 0x64
-	F_UNLCK                           = 0x2
-	F_VOLPOSMODE                      = 0x4
-	F_WRLCK                           = 0x3
-	HUPCL                             = 0x4000
-	HW_MACHINE                        = 0x1
-	ICANON                            = 0x100
-	ICMP6_FILTER                      = 0x12
-	ICRNL                             = 0x100
-	IEXTEN                            = 0x400
-	IFF_ALLMULTI                      = 0x200
-	IFF_ALTPHYS                       = 0x4000
-	IFF_BROADCAST                     = 0x2
-	IFF_DEBUG                         = 0x4
-	IFF_LINK0                         = 0x1000
-	IFF_LINK1                         = 0x2000
-	IFF_LINK2                         = 0x4000
-	IFF_LOOPBACK                      = 0x8
-	IFF_MULTICAST                     = 0x8000
-	IFF_NOARP                         = 0x80
-	IFF_NOTRAILERS                    = 0x20
-	IFF_OACTIVE                       = 0x400
-	IFF_POINTOPOINT                   = 0x10
-	IFF_PROMISC                       = 0x100
-	IFF_RUNNING                       = 0x40
-	IFF_SIMPLEX                       = 0x800
-	IFF_UP                            = 0x1
-	IFNAMSIZ                          = 0x10
-	IFT_1822                          = 0x2
-	IFT_AAL5                          = 0x31
-	IFT_ARCNET                        = 0x23
-	IFT_ARCNETPLUS                    = 0x24
-	IFT_ATM                           = 0x25
-	IFT_BRIDGE                        = 0xd1
-	IFT_CARP                          = 0xf8
-	IFT_CELLULAR                      = 0xff
-	IFT_CEPT                          = 0x13
-	IFT_DS3                           = 0x1e
-	IFT_ENC                           = 0xf4
-	IFT_EON                           = 0x19
-	IFT_ETHER                         = 0x6
-	IFT_FAITH                         = 0x38
-	IFT_FDDI                          = 0xf
-	IFT_FRELAY                        = 0x20
-	IFT_FRELAYDCE                     = 0x2c
-	IFT_GIF                           = 0x37
-	IFT_HDH1822                       = 0x3
-	IFT_HIPPI                         = 0x2f
-	IFT_HSSI                          = 0x2e
-	IFT_HY                            = 0xe
-	IFT_IEEE1394                      = 0x90
-	IFT_IEEE8023ADLAG                 = 0x88
-	IFT_ISDNBASIC                     = 0x14
-	IFT_ISDNPRIMARY                   = 0x15
-	IFT_ISO88022LLC                   = 0x29
-	IFT_ISO88023                      = 0x7
-	IFT_ISO88024                      = 0x8
-	IFT_ISO88025                      = 0x9
-	IFT_ISO88026                      = 0xa
-	IFT_L2VLAN                        = 0x87
-	IFT_LAPB                          = 0x10
-	IFT_LOCALTALK                     = 0x2a
-	IFT_LOOP                          = 0x18
-	IFT_MIOX25                        = 0x26
-	IFT_MODEM                         = 0x30
-	IFT_NSIP                          = 0x1b
-	IFT_OTHER                         = 0x1
-	IFT_P10                           = 0xc
-	IFT_P80                           = 0xd
-	IFT_PARA                          = 0x22
-	IFT_PDP                           = 0xff
-	IFT_PFLOG                         = 0xf5
-	IFT_PFSYNC                        = 0xf6
-	IFT_PKTAP                         = 0xfe
-	IFT_PPP                           = 0x17
-	IFT_PROPMUX                       = 0x36
-	IFT_PROPVIRTUAL                   = 0x35
-	IFT_PTPSERIAL                     = 0x16
-	IFT_RS232                         = 0x21
-	IFT_SDLC                          = 0x11
-	IFT_SIP                           = 0x1f
-	IFT_SLIP                          = 0x1c
-	IFT_SMDSDXI                       = 0x2b
-	IFT_SMDSICIP                      = 0x34
-	IFT_SONET                         = 0x27
-	IFT_SONETPATH                     = 0x32
-	IFT_SONETVT                       = 0x33
-	IFT_STARLAN                       = 0xb
-	IFT_STF                           = 0x39
-	IFT_T1                            = 0x12
-	IFT_ULTRA                         = 0x1d
-	IFT_V35                           = 0x2d
-	IFT_X25                           = 0x5
-	IFT_X25DDN                        = 0x4
-	IFT_X25PLE                        = 0x28
-	IFT_XETHER                        = 0x1a
-	IGNBRK                            = 0x1
-	IGNCR                             = 0x80
-	IGNPAR                            = 0x4
-	IMAXBEL                           = 0x2000
-	INLCR                             = 0x40
-	INPCK                             = 0x10
-	IN_CLASSA_HOST                    = 0xffffff
-	IN_CLASSA_MAX                     = 0x80
-	IN_CLASSA_NET                     = 0xff000000
-	IN_CLASSA_NSHIFT                  = 0x18
-	IN_CLASSB_HOST                    = 0xffff
-	IN_CLASSB_MAX                     = 0x10000
-	IN_CLASSB_NET                     = 0xffff0000
-	IN_CLASSB_NSHIFT                  = 0x10
-	IN_CLASSC_HOST                    = 0xff
-	IN_CLASSC_NET                     = 0xffffff00
-	IN_CLASSC_NSHIFT                  = 0x8
-	IN_CLASSD_HOST                    = 0xfffffff
-	IN_CLASSD_NET                     = 0xf0000000
-	IN_CLASSD_NSHIFT                  = 0x1c
-	IN_LINKLOCALNETNUM                = 0xa9fe0000
-	IN_LOOPBACKNET                    = 0x7f
-	IPPROTO_3PC                       = 0x22
-	IPPROTO_ADFS                      = 0x44
-	IPPROTO_AH                        = 0x33
-	IPPROTO_AHIP                      = 0x3d
-	IPPROTO_APES                      = 0x63
-	IPPROTO_ARGUS                     = 0xd
-	IPPROTO_AX25                      = 0x5d
-	IPPROTO_BHA                       = 0x31
-	IPPROTO_BLT                       = 0x1e
-	IPPROTO_BRSATMON                  = 0x4c
-	IPPROTO_CFTP                      = 0x3e
-	IPPROTO_CHAOS                     = 0x10
-	IPPROTO_CMTP                      = 0x26
-	IPPROTO_CPHB                      = 0x49
-	IPPROTO_CPNX                      = 0x48
-	IPPROTO_DDP                       = 0x25
-	IPPROTO_DGP                       = 0x56
-	IPPROTO_DIVERT                    = 0xfe
-	IPPROTO_DONE                      = 0x101
-	IPPROTO_DSTOPTS                   = 0x3c
-	IPPROTO_EGP                       = 0x8
-	IPPROTO_EMCON                     = 0xe
-	IPPROTO_ENCAP                     = 0x62
-	IPPROTO_EON                       = 0x50
-	IPPROTO_ESP                       = 0x32
-	IPPROTO_ETHERIP                   = 0x61
-	IPPROTO_FRAGMENT                  = 0x2c
-	IPPROTO_GGP                       = 0x3
-	IPPROTO_GMTP                      = 0x64
-	IPPROTO_GRE                       = 0x2f
-	IPPROTO_HELLO                     = 0x3f
-	IPPROTO_HMP                       = 0x14
-	IPPROTO_HOPOPTS                   = 0x0
-	IPPROTO_ICMP                      = 0x1
-	IPPROTO_ICMPV6                    = 0x3a
-	IPPROTO_IDP                       = 0x16
-	IPPROTO_IDPR                      = 0x23
-	IPPROTO_IDRP                      = 0x2d
-	IPPROTO_IGMP                      = 0x2
-	IPPROTO_IGP                       = 0x55
-	IPPROTO_IGRP                      = 0x58
-	IPPROTO_IL                        = 0x28
-	IPPROTO_INLSP                     = 0x34
-	IPPROTO_INP                       = 0x20
-	IPPROTO_IP                        = 0x0
-	IPPROTO_IPCOMP                    = 0x6c
-	IPPROTO_IPCV                      = 0x47
-	IPPROTO_IPEIP                     = 0x5e
-	IPPROTO_IPIP                      = 0x4
-	IPPROTO_IPPC                      = 0x43
-	IPPROTO_IPV4                      = 0x4
-	IPPROTO_IPV6                      = 0x29
-	IPPROTO_IRTP                      = 0x1c
-	IPPROTO_KRYPTOLAN                 = 0x41
-	IPPROTO_LARP                      = 0x5b
-	IPPROTO_LEAF1                     = 0x19
-	IPPROTO_LEAF2                     = 0x1a
-	IPPROTO_MAX                       = 0x100
-	IPPROTO_MAXID                     = 0x34
-	IPPROTO_MEAS                      = 0x13
-	IPPROTO_MHRP                      = 0x30
-	IPPROTO_MICP                      = 0x5f
-	IPPROTO_MTP                       = 0x5c
-	IPPROTO_MUX                       = 0x12
-	IPPROTO_ND                        = 0x4d
-	IPPROTO_NHRP                      = 0x36
-	IPPROTO_NONE                      = 0x3b
-	IPPROTO_NSP                       = 0x1f
-	IPPROTO_NVPII                     = 0xb
-	IPPROTO_OSPFIGP                   = 0x59
-	IPPROTO_PGM                       = 0x71
-	IPPROTO_PIGP                      = 0x9
-	IPPROTO_PIM                       = 0x67
-	IPPROTO_PRM                       = 0x15
-	IPPROTO_PUP                       = 0xc
-	IPPROTO_PVP                       = 0x4b
-	IPPROTO_RAW                       = 0xff
-	IPPROTO_RCCMON                    = 0xa
-	IPPROTO_RDP                       = 0x1b
-	IPPROTO_ROUTING                   = 0x2b
-	IPPROTO_RSVP                      = 0x2e
-	IPPROTO_RVD                       = 0x42
-	IPPROTO_SATEXPAK                  = 0x40
-	IPPROTO_SATMON                    = 0x45
-	IPPROTO_SCCSP                     = 0x60
-	IPPROTO_SCTP                      = 0x84
-	IPPROTO_SDRP                      = 0x2a
-	IPPROTO_SEP                       = 0x21
-	IPPROTO_SRPC                      = 0x5a
-	IPPROTO_ST                        = 0x7
-	IPPROTO_SVMTP                     = 0x52
-	IPPROTO_SWIPE                     = 0x35
-	IPPROTO_TCF                       = 0x57
-	IPPROTO_TCP                       = 0x6
-	IPPROTO_TP                        = 0x1d
-	IPPROTO_TPXX                      = 0x27
-	IPPROTO_TRUNK1                    = 0x17
-	IPPROTO_TRUNK2                    = 0x18
-	IPPROTO_TTP                       = 0x54
-	IPPROTO_UDP                       = 0x11
-	IPPROTO_VINES                     = 0x53
-	IPPROTO_VISA                      = 0x46
-	IPPROTO_VMTP                      = 0x51
-	IPPROTO_WBEXPAK                   = 0x4f
-	IPPROTO_WBMON                     = 0x4e
-	IPPROTO_WSN                       = 0x4a
-	IPPROTO_XNET                      = 0xf
-	IPPROTO_XTP                       = 0x24
-	IPV6_2292DSTOPTS                  = 0x17
-	IPV6_2292HOPLIMIT                 = 0x14
-	IPV6_2292HOPOPTS                  = 0x16
-	IPV6_2292NEXTHOP                  = 0x15
-	IPV6_2292PKTINFO                  = 0x13
-	IPV6_2292PKTOPTIONS               = 0x19
-	IPV6_2292RTHDR                    = 0x18
-	IPV6_BINDV6ONLY                   = 0x1b
-	IPV6_BOUND_IF                     = 0x7d
-	IPV6_CHECKSUM                     = 0x1a
-	IPV6_DEFAULT_MULTICAST_HOPS       = 0x1
-	IPV6_DEFAULT_MULTICAST_LOOP       = 0x1
-	IPV6_DEFHLIM                      = 0x40
-	IPV6_FAITH                        = 0x1d
-	IPV6_FLOWINFO_MASK                = 0xffffff0f
-	IPV6_FLOWLABEL_MASK               = 0xffff0f00
-	IPV6_FLOW_ECN_MASK                = 0x300
-	IPV6_FRAGTTL                      = 0x3c
-	IPV6_FW_ADD                       = 0x1e
-	IPV6_FW_DEL                       = 0x1f
-	IPV6_FW_FLUSH                     = 0x20
-	IPV6_FW_GET                       = 0x22
-	IPV6_FW_ZERO                      = 0x21
-	IPV6_HLIMDEC                      = 0x1
-	IPV6_IPSEC_POLICY                 = 0x1c
-	IPV6_JOIN_GROUP                   = 0xc
-	IPV6_LEAVE_GROUP                  = 0xd
-	IPV6_MAXHLIM                      = 0xff
-	IPV6_MAXOPTHDR                    = 0x800
-	IPV6_MAXPACKET                    = 0xffff
-	IPV6_MAX_GROUP_SRC_FILTER         = 0x200
-	IPV6_MAX_MEMBERSHIPS              = 0xfff
-	IPV6_MAX_SOCK_SRC_FILTER          = 0x80
-	IPV6_MIN_MEMBERSHIPS              = 0x1f
-	IPV6_MMTU                         = 0x500
-	IPV6_MULTICAST_HOPS               = 0xa
-	IPV6_MULTICAST_IF                 = 0x9
-	IPV6_MULTICAST_LOOP               = 0xb
-	IPV6_PORTRANGE                    = 0xe
-	IPV6_PORTRANGE_DEFAULT            = 0x0
-	IPV6_PORTRANGE_HIGH               = 0x1
-	IPV6_PORTRANGE_LOW                = 0x2
-	IPV6_RECVTCLASS                   = 0x23
-	IPV6_RTHDR_LOOSE                  = 0x0
-	IPV6_RTHDR_STRICT                 = 0x1
-	IPV6_RTHDR_TYPE_0                 = 0x0
-	IPV6_SOCKOPT_RESERVED1            = 0x3
-	IPV6_TCLASS                       = 0x24
-	IPV6_UNICAST_HOPS                 = 0x4
-	IPV6_V6ONLY                       = 0x1b
-	IPV6_VERSION                      = 0x60
-	IPV6_VERSION_MASK                 = 0xf0
-	IP_ADD_MEMBERSHIP                 = 0xc
-	IP_ADD_SOURCE_MEMBERSHIP          = 0x46
-	IP_BLOCK_SOURCE                   = 0x48
-	IP_BOUND_IF                       = 0x19
-	IP_DEFAULT_MULTICAST_LOOP         = 0x1
-	IP_DEFAULT_MULTICAST_TTL          = 0x1
-	IP_DF                             = 0x4000
-	IP_DROP_MEMBERSHIP                = 0xd
-	IP_DROP_SOURCE_MEMBERSHIP         = 0x47
-	IP_DUMMYNET_CONFIGURE             = 0x3c
-	IP_DUMMYNET_DEL                   = 0x3d
-	IP_DUMMYNET_FLUSH                 = 0x3e
-	IP_DUMMYNET_GET                   = 0x40
-	IP_FAITH                          = 0x16
-	IP_FW_ADD                         = 0x28
-	IP_FW_DEL                         = 0x29
-	IP_FW_FLUSH                       = 0x2a
-	IP_FW_GET                         = 0x2c
-	IP_FW_RESETLOG                    = 0x2d
-	IP_FW_ZERO                        = 0x2b
-	IP_HDRINCL                        = 0x2
-	IP_IPSEC_POLICY                   = 0x15
-	IP_MAXPACKET                      = 0xffff
-	IP_MAX_GROUP_SRC_FILTER           = 0x200
-	IP_MAX_MEMBERSHIPS                = 0xfff
-	IP_MAX_SOCK_MUTE_FILTER           = 0x80
-	IP_MAX_SOCK_SRC_FILTER            = 0x80
-	IP_MF                             = 0x2000
-	IP_MIN_MEMBERSHIPS                = 0x1f
-	IP_MSFILTER                       = 0x4a
-	IP_MSS                            = 0x240
-	IP_MULTICAST_IF                   = 0x9
-	IP_MULTICAST_IFINDEX              = 0x42
-	IP_MULTICAST_LOOP                 = 0xb
-	IP_MULTICAST_TTL                  = 0xa
-	IP_MULTICAST_VIF                  = 0xe
-	IP_NAT__XXX                       = 0x37
-	IP_OFFMASK                        = 0x1fff
-	IP_OLD_FW_ADD                     = 0x32
-	IP_OLD_FW_DEL                     = 0x33
-	IP_OLD_FW_FLUSH                   = 0x34
-	IP_OLD_FW_GET                     = 0x36
-	IP_OLD_FW_RESETLOG                = 0x38
-	IP_OLD_FW_ZERO                    = 0x35
-	IP_OPTIONS                        = 0x1
-	IP_PKTINFO                        = 0x1a
-	IP_PORTRANGE                      = 0x13
-	IP_PORTRANGE_DEFAULT              = 0x0
-	IP_PORTRANGE_HIGH                 = 0x1
-	IP_PORTRANGE_LOW                  = 0x2
-	IP_RECVDSTADDR                    = 0x7
-	IP_RECVIF                         = 0x14
-	IP_RECVOPTS                       = 0x5
-	IP_RECVPKTINFO                    = 0x1a
-	IP_RECVRETOPTS                    = 0x6
-	IP_RECVTOS                        = 0x1b
-	IP_RECVTTL                        = 0x18
-	IP_RETOPTS                        = 0x8
-	IP_RF                             = 0x8000
-	IP_RSVP_OFF                       = 0x10
-	IP_RSVP_ON                        = 0xf
-	IP_RSVP_VIF_OFF                   = 0x12
-	IP_RSVP_VIF_ON                    = 0x11
-	IP_STRIPHDR                       = 0x17
-	IP_TOS                            = 0x3
-	IP_TRAFFIC_MGT_BACKGROUND         = 0x41
-	IP_TTL                            = 0x4
-	IP_UNBLOCK_SOURCE                 = 0x49
-	ISIG                              = 0x80
-	ISTRIP                            = 0x20
-	IUTF8                             = 0x4000
-	IXANY                             = 0x800
-	IXOFF                             = 0x400
-	IXON                              = 0x200
-	KERN_HOSTNAME                     = 0xa
-	KERN_OSRELEASE                    = 0x2
-	KERN_OSTYPE                       = 0x1
-	KERN_VERSION                      = 0x4
-	LOCK_EX                           = 0x2
-	LOCK_NB                           = 0x4
-	LOCK_SH                           = 0x1
-	LOCK_UN                           = 0x8
-	MADV_CAN_REUSE                    = 0x9
-	MADV_DONTNEED                     = 0x4
-	MADV_FREE                         = 0x5
-	MADV_FREE_REUSABLE                = 0x7
-	MADV_FREE_REUSE                   = 0x8
-	MADV_NORMAL                       = 0x0
-	MADV_PAGEOUT                      = 0xa
-	MADV_RANDOM                       = 0x1
-	MADV_SEQUENTIAL                   = 0x2
-	MADV_WILLNEED                     = 0x3
-	MADV_ZERO_WIRED_PAGES             = 0x6
-	MAP_ANON                          = 0x1000
-	MAP_ANONYMOUS                     = 0x1000
-	MAP_COPY                          = 0x2
-	MAP_FILE                          = 0x0
-	MAP_FIXED                         = 0x10
-	MAP_HASSEMAPHORE                  = 0x200
-	MAP_JIT                           = 0x800
-	MAP_NOCACHE                       = 0x400
-	MAP_NOEXTEND                      = 0x100
-	MAP_NORESERVE                     = 0x40
-	MAP_PRIVATE                       = 0x2
-	MAP_RENAME                        = 0x20
-	MAP_RESERVED0080                  = 0x80
-	MAP_RESILIENT_CODESIGN            = 0x2000
-	MAP_RESILIENT_MEDIA               = 0x4000
-	MAP_SHARED                        = 0x1
-	MCL_CURRENT                       = 0x1
-	MCL_FUTURE                        = 0x2
-	MNT_ASYNC                         = 0x40
-	MNT_AUTOMOUNTED                   = 0x400000
-	MNT_CMDFLAGS                      = 0xf0000
-	MNT_CPROTECT                      = 0x80
-	MNT_DEFWRITE                      = 0x2000000
-	MNT_DONTBROWSE                    = 0x100000
-	MNT_DOVOLFS                       = 0x8000
-	MNT_DWAIT                         = 0x4
-	MNT_EXPORTED                      = 0x100
-	MNT_FORCE                         = 0x80000
-	MNT_IGNORE_OWNERSHIP              = 0x200000
-	MNT_JOURNALED                     = 0x800000
-	MNT_LOCAL                         = 0x1000
-	MNT_MULTILABEL                    = 0x4000000
-	MNT_NOATIME                       = 0x10000000
-	MNT_NOBLOCK                       = 0x20000
-	MNT_NODEV                         = 0x10
-	MNT_NOEXEC                        = 0x4
-	MNT_NOSUID                        = 0x8
-	MNT_NOUSERXATTR                   = 0x1000000
-	MNT_NOWAIT                        = 0x2
-	MNT_QUARANTINE                    = 0x400
-	MNT_QUOTA                         = 0x2000
-	MNT_RDONLY                        = 0x1
-	MNT_RELOAD                        = 0x40000
-	MNT_ROOTFS                        = 0x4000
-	MNT_SYNCHRONOUS                   = 0x2
-	MNT_UNION                         = 0x20
-	MNT_UNKNOWNPERMISSIONS            = 0x200000
-	MNT_UPDATE                        = 0x10000
-	MNT_VISFLAGMASK                   = 0x17f0f5ff
-	MNT_WAIT                          = 0x1
-	MSG_CTRUNC                        = 0x20
-	MSG_DONTROUTE                     = 0x4
-	MSG_DONTWAIT                      = 0x80
-	MSG_EOF                           = 0x100
-	MSG_EOR                           = 0x8
-	MSG_FLUSH                         = 0x400
-	MSG_HAVEMORE                      = 0x2000
-	MSG_HOLD                          = 0x800
-	MSG_NEEDSA                        = 0x10000
-	MSG_OOB                           = 0x1
-	MSG_PEEK                          = 0x2
-	MSG_RCVMORE                       = 0x4000
-	MSG_SEND                          = 0x1000
-	MSG_TRUNC                         = 0x10
-	MSG_WAITALL                       = 0x40
-	MSG_WAITSTREAM                    = 0x200
-	MS_ASYNC                          = 0x1
-	MS_DEACTIVATE                     = 0x8
-	MS_INVALIDATE                     = 0x2
-	MS_KILLPAGES                      = 0x4
-	MS_SYNC                           = 0x10
-	NAME_MAX                          = 0xff
-	NET_RT_DUMP                       = 0x1
-	NET_RT_DUMP2                      = 0x7
-	NET_RT_FLAGS                      = 0x2
-	NET_RT_IFLIST                     = 0x3
-	NET_RT_IFLIST2                    = 0x6
-	NET_RT_MAXID                      = 0xa
-	NET_RT_STAT                       = 0x4
-	NET_RT_TRASH                      = 0x5
-	NFDBITS                           = 0x20
-	NL0                               = 0x0
-	NL1                               = 0x100
-	NL2                               = 0x200
-	NL3                               = 0x300
-	NLDLY                             = 0x300
-	NOFLSH                            = 0x80000000
-	NOKERNINFO                        = 0x2000000
-	NOTE_ABSOLUTE                     = 0x8
-	NOTE_ATTRIB                       = 0x8
-	NOTE_BACKGROUND                   = 0x40
-	NOTE_CHILD                        = 0x4
-	NOTE_CRITICAL                     = 0x20
-	NOTE_DELETE                       = 0x1
-	NOTE_EXEC                         = 0x20000000
-	NOTE_EXIT                         = 0x80000000
-	NOTE_EXITSTATUS                   = 0x4000000
-	NOTE_EXIT_CSERROR                 = 0x40000
-	NOTE_EXIT_DECRYPTFAIL             = 0x10000
-	NOTE_EXIT_DETAIL                  = 0x2000000
-	NOTE_EXIT_DETAIL_MASK             = 0x70000
-	NOTE_EXIT_MEMORY                  = 0x20000
-	NOTE_EXIT_REPARENTED              = 0x80000
-	NOTE_EXTEND                       = 0x4
-	NOTE_FFAND                        = 0x40000000
-	NOTE_FFCOPY                       = 0xc0000000
-	NOTE_FFCTRLMASK                   = 0xc0000000
-	NOTE_FFLAGSMASK                   = 0xffffff
-	NOTE_FFNOP                        = 0x0
-	NOTE_FFOR                         = 0x80000000
-	NOTE_FORK                         = 0x40000000
-	NOTE_FUNLOCK                      = 0x100
-	NOTE_LEEWAY                       = 0x10
-	NOTE_LINK                         = 0x10
-	NOTE_LOWAT                        = 0x1
-	NOTE_MACH_CONTINUOUS_TIME         = 0x80
-	NOTE_NONE                         = 0x80
-	NOTE_NSECONDS                     = 0x4
-	NOTE_OOB                          = 0x2
-	NOTE_PCTRLMASK                    = -0x100000
-	NOTE_PDATAMASK                    = 0xfffff
-	NOTE_REAP                         = 0x10000000
-	NOTE_RENAME                       = 0x20
-	NOTE_REVOKE                       = 0x40
-	NOTE_SECONDS                      = 0x1
-	NOTE_SIGNAL                       = 0x8000000
-	NOTE_TRACK                        = 0x1
-	NOTE_TRACKERR                     = 0x2
-	NOTE_TRIGGER                      = 0x1000000
-	NOTE_USECONDS                     = 0x2
-	NOTE_VM_ERROR                     = 0x10000000
-	NOTE_VM_PRESSURE                  = 0x80000000
-	NOTE_VM_PRESSURE_SUDDEN_TERMINATE = 0x20000000
-	NOTE_VM_PRESSURE_TERMINATE        = 0x40000000
-	NOTE_WRITE                        = 0x2
-	OCRNL                             = 0x10
-	OFDEL                             = 0x20000
-	OFILL                             = 0x80
-	ONLCR                             = 0x2
-	ONLRET                            = 0x40
-	ONOCR                             = 0x20
-	ONOEOT                            = 0x8
-	OPOST                             = 0x1
-	OXTABS                            = 0x4
-	O_ACCMODE                         = 0x3
-	O_ALERT                           = 0x20000000
-	O_APPEND                          = 0x8
-	O_ASYNC                           = 0x40
-	O_CLOEXEC                         = 0x1000000
-	O_CREAT                           = 0x200
-	O_DIRECTORY                       = 0x100000
-	O_DP_GETRAWENCRYPTED              = 0x1
-	O_DP_GETRAWUNENCRYPTED            = 0x2
-	O_DSYNC                           = 0x400000
-	O_EVTONLY                         = 0x8000
-	O_EXCL                            = 0x800
-	O_EXLOCK                          = 0x20
-	O_FSYNC                           = 0x80
-	O_NDELAY                          = 0x4
-	O_NOCTTY                          = 0x20000
-	O_NOFOLLOW                        = 0x100
-	O_NONBLOCK                        = 0x4
-	O_POPUP                           = 0x80000000
-	O_RDONLY                          = 0x0
-	O_RDWR                            = 0x2
-	O_SHLOCK                          = 0x10
-	O_SYMLINK                         = 0x200000
-	O_SYNC                            = 0x80
-	O_TRUNC                           = 0x400
-	O_WRONLY                          = 0x1
-	PARENB                            = 0x1000
-	PARMRK                            = 0x8
-	PARODD                            = 0x2000
-	PENDIN                            = 0x20000000
-	PRIO_PGRP                         = 0x1
-	PRIO_PROCESS                      = 0x0
-	PRIO_USER                         = 0x2
-	PROT_EXEC                         = 0x4
-	PROT_NONE                         = 0x0
-	PROT_READ                         = 0x1
-	PROT_WRITE                        = 0x2
-	PT_ATTACH                         = 0xa
-	PT_ATTACHEXC                      = 0xe
-	PT_CONTINUE                       = 0x7
-	PT_DENY_ATTACH                    = 0x1f
-	PT_DETACH                         = 0xb
-	PT_FIRSTMACH                      = 0x20
-	PT_FORCEQUOTA                     = 0x1e
-	PT_KILL                           = 0x8
-	PT_READ_D                         = 0x2
-	PT_READ_I                         = 0x1
-	PT_READ_U                         = 0x3
-	PT_SIGEXC                         = 0xc
-	PT_STEP                           = 0x9
-	PT_THUPDATE                       = 0xd
-	PT_TRACE_ME                       = 0x0
-	PT_WRITE_D                        = 0x5
-	PT_WRITE_I                        = 0x4
-	PT_WRITE_U                        = 0x6
-	RLIMIT_AS                         = 0x5
-	RLIMIT_CORE                       = 0x4
-	RLIMIT_CPU                        = 0x0
-	RLIMIT_CPU_USAGE_MONITOR          = 0x2
-	RLIMIT_DATA                       = 0x2
-	RLIMIT_FSIZE                      = 0x1
-	RLIMIT_MEMLOCK                    = 0x6
-	RLIMIT_NOFILE                     = 0x8
-	RLIMIT_NPROC                      = 0x7
-	RLIMIT_RSS                        = 0x5
-	RLIMIT_STACK                      = 0x3
-	RLIM_INFINITY                     = 0x7fffffffffffffff
-	RTAX_AUTHOR                       = 0x6
-	RTAX_BRD                          = 0x7
-	RTAX_DST                          = 0x0
-	RTAX_GATEWAY                      = 0x1
-	RTAX_GENMASK                      = 0x3
-	RTAX_IFA                          = 0x5
-	RTAX_IFP                          = 0x4
-	RTAX_MAX                          = 0x8
-	RTAX_NETMASK                      = 0x2
-	RTA_AUTHOR                        = 0x40
-	RTA_BRD                           = 0x80
-	RTA_DST                           = 0x1
-	RTA_GATEWAY                       = 0x2
-	RTA_GENMASK                       = 0x8
-	RTA_IFA                           = 0x20
-	RTA_IFP                           = 0x10
-	RTA_NETMASK                       = 0x4
-	RTF_BLACKHOLE                     = 0x1000
-	RTF_BROADCAST                     = 0x400000
-	RTF_CLONING                       = 0x100
-	RTF_CONDEMNED                     = 0x2000000
-	RTF_DELCLONE                      = 0x80
-	RTF_DONE                          = 0x40
-	RTF_DYNAMIC                       = 0x10
-	RTF_GATEWAY                       = 0x2
-	RTF_HOST                          = 0x4
-	RTF_IFREF                         = 0x4000000
-	RTF_IFSCOPE                       = 0x1000000
-	RTF_LLINFO                        = 0x400
-	RTF_LOCAL                         = 0x200000
-	RTF_MODIFIED                      = 0x20
-	RTF_MULTICAST                     = 0x800000
-	RTF_NOIFREF                       = 0x2000
-	RTF_PINNED                        = 0x100000
-	RTF_PRCLONING                     = 0x10000
-	RTF_PROTO1                        = 0x8000
-	RTF_PROTO2                        = 0x4000
-	RTF_PROTO3                        = 0x40000
-	RTF_PROXY                         = 0x8000000
-	RTF_REJECT                        = 0x8
-	RTF_ROUTER                        = 0x10000000
-	RTF_STATIC                        = 0x800
-	RTF_UP                            = 0x1
-	RTF_WASCLONED                     = 0x20000
-	RTF_XRESOLVE                      = 0x200
-	RTM_ADD                           = 0x1
-	RTM_CHANGE                        = 0x3
-	RTM_DELADDR                       = 0xd
-	RTM_DELETE                        = 0x2
-	RTM_DELMADDR                      = 0x10
-	RTM_GET                           = 0x4
-	RTM_GET2                          = 0x14
-	RTM_IFINFO                        = 0xe
-	RTM_IFINFO2                       = 0x12
-	RTM_LOCK                          = 0x8
-	RTM_LOSING                        = 0x5
-	RTM_MISS                          = 0x7
-	RTM_NEWADDR                       = 0xc
-	RTM_NEWMADDR                      = 0xf
-	RTM_NEWMADDR2                     = 0x13
-	RTM_OLDADD                        = 0x9
-	RTM_OLDDEL                        = 0xa
-	RTM_REDIRECT                      = 0x6
-	RTM_RESOLVE                       = 0xb
-	RTM_RTTUNIT                       = 0xf4240
-	RTM_VERSION                       = 0x5
-	RTV_EXPIRE                        = 0x4
-	RTV_HOPCOUNT                      = 0x2
-	RTV_MTU                           = 0x1
-	RTV_RPIPE                         = 0x8
-	RTV_RTT                           = 0x40
-	RTV_RTTVAR                        = 0x80
-	RTV_SPIPE                         = 0x10
-	RTV_SSTHRESH                      = 0x20
-	RUSAGE_CHILDREN                   = -0x1
-	RUSAGE_SELF                       = 0x0
-	SCM_CREDS                         = 0x3
-	SCM_RIGHTS                        = 0x1
-	SCM_TIMESTAMP                     = 0x2
-	SCM_TIMESTAMP_MONOTONIC           = 0x4
-	SHUT_RD                           = 0x0
-	SHUT_RDWR                         = 0x2
-	SHUT_WR                           = 0x1
-	SIOCADDMULTI                      = 0x80206931
-	SIOCAIFADDR                       = 0x8040691a
-	SIOCARPIPLL                       = 0xc0206928
-	SIOCATMARK                        = 0x40047307
-	SIOCAUTOADDR                      = 0xc0206926
-	SIOCAUTONETMASK                   = 0x80206927
-	SIOCDELMULTI                      = 0x80206932
-	SIOCDIFADDR                       = 0x80206919
-	SIOCDIFPHYADDR                    = 0x80206941
-	SIOCGDRVSPEC                      = 0xc028697b
-	SIOCGETVLAN                       = 0xc020697f
-	SIOCGHIWAT                        = 0x40047301
-	SIOCGIFADDR                       = 0xc0206921
-	SIOCGIFALTMTU                     = 0xc0206948
-	SIOCGIFASYNCMAP                   = 0xc020697c
-	SIOCGIFBOND                       = 0xc0206947
-	SIOCGIFBRDADDR                    = 0xc0206923
-	SIOCGIFCAP                        = 0xc020695b
-	SIOCGIFCONF                       = 0xc00c6924
-	SIOCGIFDEVMTU                     = 0xc0206944
-	SIOCGIFDSTADDR                    = 0xc0206922
-	SIOCGIFFLAGS                      = 0xc0206911
-	SIOCGIFGENERIC                    = 0xc020693a
-	SIOCGIFKPI                        = 0xc0206987
-	SIOCGIFMAC                        = 0xc0206982
-	SIOCGIFMEDIA                      = 0xc02c6938
-	SIOCGIFMETRIC                     = 0xc0206917
-	SIOCGIFMTU                        = 0xc0206933
-	SIOCGIFNETMASK                    = 0xc0206925
-	SIOCGIFPDSTADDR                   = 0xc0206940
-	SIOCGIFPHYS                       = 0xc0206935
-	SIOCGIFPSRCADDR                   = 0xc020693f
-	SIOCGIFSTATUS                     = 0xc331693d
-	SIOCGIFVLAN                       = 0xc020697f
-	SIOCGIFWAKEFLAGS                  = 0xc0206988
-	SIOCGLOWAT                        = 0x40047303
-	SIOCGPGRP                         = 0x40047309
-	SIOCIFCREATE                      = 0xc0206978
-	SIOCIFCREATE2                     = 0xc020697a
-	SIOCIFDESTROY                     = 0x80206979
-	SIOCIFGCLONERS                    = 0xc0106981
-	SIOCRSLVMULTI                     = 0xc010693b
-	SIOCSDRVSPEC                      = 0x8028697b
-	SIOCSETVLAN                       = 0x8020697e
-	SIOCSHIWAT                        = 0x80047300
-	SIOCSIFADDR                       = 0x8020690c
-	SIOCSIFALTMTU                     = 0x80206945
-	SIOCSIFASYNCMAP                   = 0x8020697d
-	SIOCSIFBOND                       = 0x80206946
-	SIOCSIFBRDADDR                    = 0x80206913
-	SIOCSIFCAP                        = 0x8020695a
-	SIOCSIFDSTADDR                    = 0x8020690e
-	SIOCSIFFLAGS                      = 0x80206910
-	SIOCSIFGENERIC                    = 0x80206939
-	SIOCSIFKPI                        = 0x80206986
-	SIOCSIFLLADDR                     = 0x8020693c
-	SIOCSIFMAC                        = 0x80206983
-	SIOCSIFMEDIA                      = 0xc0206937
-	SIOCSIFMETRIC                     = 0x80206918
-	SIOCSIFMTU                        = 0x80206934
-	SIOCSIFNETMASK                    = 0x80206916
-	SIOCSIFPHYADDR                    = 0x8040693e
-	SIOCSIFPHYS                       = 0x80206936
-	SIOCSIFVLAN                       = 0x8020697e
-	SIOCSLOWAT                        = 0x80047302
-	SIOCSPGRP                         = 0x80047308
-	SOCK_DGRAM                        = 0x2
-	SOCK_MAXADDRLEN                   = 0xff
-	SOCK_RAW                          = 0x3
-	SOCK_RDM                          = 0x4
-	SOCK_SEQPACKET                    = 0x5
-	SOCK_STREAM                       = 0x1
-	SOL_SOCKET                        = 0xffff
-	SOMAXCONN                         = 0x80
-	SO_ACCEPTCONN                     = 0x2
-	SO_BROADCAST                      = 0x20
-	SO_DEBUG                          = 0x1
-	SO_DONTROUTE                      = 0x10
-	SO_DONTTRUNC                      = 0x2000
-	SO_ERROR                          = 0x1007
-	SO_KEEPALIVE                      = 0x8
-	SO_LABEL                          = 0x1010
-	SO_LINGER                         = 0x80
-	SO_LINGER_SEC                     = 0x1080
-	SO_NETSVC_MARKING_LEVEL           = 0x1119
-	SO_NET_SERVICE_TYPE               = 0x1116
-	SO_NKE                            = 0x1021
-	SO_NOADDRERR                      = 0x1023
-	SO_NOSIGPIPE                      = 0x1022
-	SO_NOTIFYCONFLICT                 = 0x1026
-	SO_NP_EXTENSIONS                  = 0x1083
-	SO_NREAD                          = 0x1020
-	SO_NUMRCVPKT                      = 0x1112
-	SO_NWRITE                         = 0x1024
-	SO_OOBINLINE                      = 0x100
-	SO_PEERLABEL                      = 0x1011
-	SO_RANDOMPORT                     = 0x1082
-	SO_RCVBUF                         = 0x1002
-	SO_RCVLOWAT                       = 0x1004
-	SO_RCVTIMEO                       = 0x1006
-	SO_REUSEADDR                      = 0x4
-	SO_REUSEPORT                      = 0x200
-	SO_REUSESHAREUID                  = 0x1025
-	SO_SNDBUF                         = 0x1001
-	SO_SNDLOWAT                       = 0x1003
-	SO_SNDTIMEO                       = 0x1005
-	SO_TIMESTAMP                      = 0x400
-	SO_TIMESTAMP_MONOTONIC            = 0x800
-	SO_TYPE                           = 0x1008
-	SO_UPCALLCLOSEWAIT                = 0x1027
-	SO_USELOOPBACK                    = 0x40
-	SO_WANTMORE                       = 0x4000
-	SO_WANTOOBFLAG                    = 0x8000
-	S_IEXEC                           = 0x40
-	S_IFBLK                           = 0x6000
-	S_IFCHR                           = 0x2000
-	S_IFDIR                           = 0x4000
-	S_IFIFO                           = 0x1000
-	S_IFLNK                           = 0xa000
-	S_IFMT                            = 0xf000
-	S_IFREG                           = 0x8000
-	S_IFSOCK                          = 0xc000
-	S_IFWHT                           = 0xe000
-	S_IREAD                           = 0x100
-	S_IRGRP                           = 0x20
-	S_IROTH                           = 0x4
-	S_IRUSR                           = 0x100
-	S_IRWXG                           = 0x38
-	S_IRWXO                           = 0x7
-	S_IRWXU                           = 0x1c0
-	S_ISGID                           = 0x400
-	S_ISTXT                           = 0x200
-	S_ISUID                           = 0x800
-	S_ISVTX                           = 0x200
-	S_IWGRP                           = 0x10
-	S_IWOTH                           = 0x2
-	S_IWRITE                          = 0x80
-	S_IWUSR                           = 0x80
-	S_IXGRP                           = 0x8
-	S_IXOTH                           = 0x1
-	S_IXUSR                           = 0x40
-	TAB0                              = 0x0
-	TAB1                              = 0x400
-	TAB2                              = 0x800
-	TAB3                              = 0x4
-	TABDLY                            = 0xc04
-	TCIFLUSH                          = 0x1
-	TCIOFF                            = 0x3
-	TCIOFLUSH                         = 0x3
-	TCION                             = 0x4
-	TCOFLUSH                          = 0x2
-	TCOOFF                            = 0x1
-	TCOON                             = 0x2
-	TCP_CONNECTIONTIMEOUT             = 0x20
-	TCP_CONNECTION_INFO               = 0x106
-	TCP_ENABLE_ECN                    = 0x104
-	TCP_FASTOPEN                      = 0x105
-	TCP_KEEPALIVE                     = 0x10
-	TCP_KEEPCNT                       = 0x102
-	TCP_KEEPINTVL                     = 0x101
-	TCP_MAXHLEN                       = 0x3c
-	TCP_MAXOLEN                       = 0x28
-	TCP_MAXSEG                        = 0x2
-	TCP_MAXWIN                        = 0xffff
-	TCP_MAX_SACK                      = 0x4
-	TCP_MAX_WINSHIFT                  = 0xe
-	TCP_MINMSS                        = 0xd8
-	TCP_MSS                           = 0x200
-	TCP_NODELAY                       = 0x1
-	TCP_NOOPT                         = 0x8
-	TCP_NOPUSH                        = 0x4
-	TCP_NOTSENT_LOWAT                 = 0x201
-	TCP_RXT_CONNDROPTIME              = 0x80
-	TCP_RXT_FINDROP                   = 0x100
-	TCP_SENDMOREACKS                  = 0x103
-	TCSAFLUSH                         = 0x2
-	TIOCCBRK                          = 0x2000747a
-	TIOCCDTR                          = 0x20007478
-	TIOCCONS                          = 0x80047462
-	TIOCDCDTIMESTAMP                  = 0x40107458
-	TIOCDRAIN                         = 0x2000745e
-	TIOCDSIMICROCODE                  = 0x20007455
-	TIOCEXCL                          = 0x2000740d
-	TIOCEXT                           = 0x80047460
-	TIOCFLUSH                         = 0x80047410
-	TIOCGDRAINWAIT                    = 0x40047456
-	TIOCGETA                          = 0x40487413
-	TIOCGETD                          = 0x4004741a
-	TIOCGPGRP                         = 0x40047477
-	TIOCGWINSZ                        = 0x40087468
-	TIOCIXOFF                         = 0x20007480
-	TIOCIXON                          = 0x20007481
-	TIOCMBIC                          = 0x8004746b
-	TIOCMBIS                          = 0x8004746c
-	TIOCMGDTRWAIT                     = 0x4004745a
-	TIOCMGET                          = 0x4004746a
-	TIOCMODG                          = 0x40047403
-	TIOCMODS                          = 0x80047404
-	TIOCMSDTRWAIT                     = 0x8004745b
-	TIOCMSET                          = 0x8004746d
-	TIOCM_CAR                         = 0x40
-	TIOCM_CD                          = 0x40
-	TIOCM_CTS                         = 0x20
-	TIOCM_DSR                         = 0x100
-	TIOCM_DTR                         = 0x2
-	TIOCM_LE                          = 0x1
-	TIOCM_RI                          = 0x80
-	TIOCM_RNG                         = 0x80
-	TIOCM_RTS                         = 0x4
-	TIOCM_SR                          = 0x10
-	TIOCM_ST                          = 0x8
-	TIOCNOTTY                         = 0x20007471
-	TIOCNXCL                          = 0x2000740e
-	TIOCOUTQ                          = 0x40047473
-	TIOCPKT                           = 0x80047470
-	TIOCPKT_DATA                      = 0x0
-	TIOCPKT_DOSTOP                    = 0x20
-	TIOCPKT_FLUSHREAD                 = 0x1
-	TIOCPKT_FLUSHWRITE                = 0x2
-	TIOCPKT_IOCTL                     = 0x40
-	TIOCPKT_NOSTOP                    = 0x10
-	TIOCPKT_START                     = 0x8
-	TIOCPKT_STOP                      = 0x4
-	TIOCPTYGNAME                      = 0x40807453
-	TIOCPTYGRANT                      = 0x20007454
-	TIOCPTYUNLK                       = 0x20007452
-	TIOCREMOTE                        = 0x80047469
-	TIOCSBRK                          = 0x2000747b
-	TIOCSCONS                         = 0x20007463
-	TIOCSCTTY                         = 0x20007461
-	TIOCSDRAINWAIT                    = 0x80047457
-	TIOCSDTR                          = 0x20007479
-	TIOCSETA                          = 0x80487414
-	TIOCSETAF                         = 0x80487416
-	TIOCSETAW                         = 0x80487415
-	TIOCSETD                          = 0x8004741b
-	TIOCSIG                           = 0x2000745f
-	TIOCSPGRP                         = 0x80047476
-	TIOCSTART                         = 0x2000746e
-	TIOCSTAT                          = 0x20007465
-	TIOCSTI                           = 0x80017472
-	TIOCSTOP                          = 0x2000746f
-	TIOCSWINSZ                        = 0x80087467
-	TIOCTIMESTAMP                     = 0x40107459
-	TIOCUCNTL                         = 0x80047466
-	TOSTOP                            = 0x400000
-	VDISCARD                          = 0xf
-	VDSUSP                            = 0xb
-	VEOF                              = 0x0
-	VEOL                              = 0x1
-	VEOL2                             = 0x2
-	VERASE                            = 0x3
-	VINTR                             = 0x8
-	VKILL                             = 0x5
-	VLNEXT                            = 0xe
-	VMIN                              = 0x10
-	VM_LOADAVG                        = 0x2
-	VM_MACHFACTOR                     = 0x4
-	VM_MAXID                          = 0x6
-	VM_METER                          = 0x1
-	VM_SWAPUSAGE                      = 0x5
-	VQUIT                             = 0x9
-	VREPRINT                          = 0x6
-	VSTART                            = 0xc
-	VSTATUS                           = 0x12
-	VSTOP                             = 0xd
-	VSUSP                             = 0xa
-	VT0                               = 0x0
-	VT1                               = 0x10000
-	VTDLY                             = 0x10000
-	VTIME                             = 0x11
-	VWERASE                           = 0x4
-	WCONTINUED                        = 0x10
-	WCOREFLAG                         = 0x80
-	WEXITED                           = 0x4
-	WNOHANG                           = 0x1
-	WNOWAIT                           = 0x20
-	WORDSIZE                          = 0x40
-	WSTOPPED                          = 0x8
-	WUNTRACED                         = 0x2
-	XATTR_CREATE                      = 0x2
-	XATTR_NODEFAULT                   = 0x10
-	XATTR_NOFOLLOW                    = 0x1
-	XATTR_NOSECURITY                  = 0x8
-	XATTR_REPLACE                     = 0x4
-	XATTR_SHOWCOMPRESSION             = 0x20
-)
-
-// Errors
-const (
-	E2BIG           = syscall.Errno(0x7)
-	EACCES          = syscall.Errno(0xd)
-	EADDRINUSE      = syscall.Errno(0x30)
-	EADDRNOTAVAIL   = syscall.Errno(0x31)
-	EAFNOSUPPORT    = syscall.Errno(0x2f)
-	EAGAIN          = syscall.Errno(0x23)
-	EALREADY        = syscall.Errno(0x25)
-	EAUTH           = syscall.Errno(0x50)
-	EBADARCH        = syscall.Errno(0x56)
-	EBADEXEC        = syscall.Errno(0x55)
-	EBADF           = syscall.Errno(0x9)
-	EBADMACHO       = syscall.Errno(0x58)
-	EBADMSG         = syscall.Errno(0x5e)
-	EBADRPC         = syscall.Errno(0x48)
-	EBUSY           = syscall.Errno(0x10)
-	ECANCELED       = syscall.Errno(0x59)
-	ECHILD          = syscall.Errno(0xa)
-	ECONNABORTED    = syscall.Errno(0x35)
-	ECONNREFUSED    = syscall.Errno(0x3d)
-	ECONNRESET      = syscall.Errno(0x36)
-	EDEADLK         = syscall.Errno(0xb)
-	EDESTADDRREQ    = syscall.Errno(0x27)
-	EDEVERR         = syscall.Errno(0x53)
-	EDOM            = syscall.Errno(0x21)
-	EDQUOT          = syscall.Errno(0x45)
-	EEXIST          = syscall.Errno(0x11)
-	EFAULT          = syscall.Errno(0xe)
-	EFBIG           = syscall.Errno(0x1b)
-	EFTYPE          = syscall.Errno(0x4f)
-	EHOSTDOWN       = syscall.Errno(0x40)
-	EHOSTUNREACH    = syscall.Errno(0x41)
-	EIDRM           = syscall.Errno(0x5a)
-	EILSEQ          = syscall.Errno(0x5c)
-	EINPROGRESS     = syscall.Errno(0x24)
-	EINTR           = syscall.Errno(0x4)
-	EINVAL          = syscall.Errno(0x16)
-	EIO             = syscall.Errno(0x5)
-	EISCONN         = syscall.Errno(0x38)
-	EISDIR          = syscall.Errno(0x15)
-	ELAST           = syscall.Errno(0x6a)
-	ELOOP           = syscall.Errno(0x3e)
-	EMFILE          = syscall.Errno(0x18)
-	EMLINK          = syscall.Errno(0x1f)
-	EMSGSIZE        = syscall.Errno(0x28)
-	EMULTIHOP       = syscall.Errno(0x5f)
-	ENAMETOOLONG    = syscall.Errno(0x3f)
-	ENEEDAUTH       = syscall.Errno(0x51)
-	ENETDOWN        = syscall.Errno(0x32)
-	ENETRESET       = syscall.Errno(0x34)
-	ENETUNREACH     = syscall.Errno(0x33)
-	ENFILE          = syscall.Errno(0x17)
-	ENOATTR         = syscall.Errno(0x5d)
-	ENOBUFS         = syscall.Errno(0x37)
-	ENODATA         = syscall.Errno(0x60)
-	ENODEV          = syscall.Errno(0x13)
-	ENOENT          = syscall.Errno(0x2)
-	ENOEXEC         = syscall.Errno(0x8)
-	ENOLCK          = syscall.Errno(0x4d)
-	ENOLINK         = syscall.Errno(0x61)
-	ENOMEM          = syscall.Errno(0xc)
-	ENOMSG          = syscall.Errno(0x5b)
-	ENOPOLICY       = syscall.Errno(0x67)
-	ENOPROTOOPT     = syscall.Errno(0x2a)
-	ENOSPC          = syscall.Errno(0x1c)
-	ENOSR           = syscall.Errno(0x62)
-	ENOSTR          = syscall.Errno(0x63)
-	ENOSYS          = syscall.Errno(0x4e)
-	ENOTBLK         = syscall.Errno(0xf)
-	ENOTCONN        = syscall.Errno(0x39)
-	ENOTDIR         = syscall.Errno(0x14)
-	ENOTEMPTY       = syscall.Errno(0x42)
-	ENOTRECOVERABLE = syscall.Errno(0x68)
-	ENOTSOCK        = syscall.Errno(0x26)
-	ENOTSUP         = syscall.Errno(0x2d)
-	ENOTTY          = syscall.Errno(0x19)
-	ENXIO           = syscall.Errno(0x6)
-	EOPNOTSUPP      = syscall.Errno(0x66)
-	EOVERFLOW       = syscall.Errno(0x54)
-	EOWNERDEAD      = syscall.Errno(0x69)
-	EPERM           = syscall.Errno(0x1)
-	EPFNOSUPPORT    = syscall.Errno(0x2e)
-	EPIPE           = syscall.Errno(0x20)
-	EPROCLIM        = syscall.Errno(0x43)
-	EPROCUNAVAIL    = syscall.Errno(0x4c)
-	EPROGMISMATCH   = syscall.Errno(0x4b)
-	EPROGUNAVAIL    = syscall.Errno(0x4a)
-	EPROTO          = syscall.Errno(0x64)
-	EPROTONOSUPPORT = syscall.Errno(0x2b)
-	EPROTOTYPE      = syscall.Errno(0x29)
-	EPWROFF         = syscall.Errno(0x52)
-	EQFULL          = syscall.Errno(0x6a)
-	ERANGE          = syscall.Errno(0x22)
-	EREMOTE         = syscall.Errno(0x47)
-	EROFS           = syscall.Errno(0x1e)
-	ERPCMISMATCH    = syscall.Errno(0x49)
-	ESHLIBVERS      = syscall.Errno(0x57)
-	ESHUTDOWN       = syscall.Errno(0x3a)
-	ESOCKTNOSUPPORT = syscall.Errno(0x2c)
-	ESPIPE          = syscall.Errno(0x1d)
-	ESRCH           = syscall.Errno(0x3)
-	ESTALE          = syscall.Errno(0x46)
-	ETIME           = syscall.Errno(0x65)
-	ETIMEDOUT       = syscall.Errno(0x3c)
-	ETOOMANYREFS    = syscall.Errno(0x3b)
-	ETXTBSY         = syscall.Errno(0x1a)
-	EUSERS          = syscall.Errno(0x44)
-	EWOULDBLOCK     = syscall.Errno(0x23)
-	EXDEV           = syscall.Errno(0x12)
-)
-
-// Signals
-const (
-	SIGABRT   = syscall.Signal(0x6)
-	SIGALRM   = syscall.Signal(0xe)
-	SIGBUS    = syscall.Signal(0xa)
-	SIGCHLD   = syscall.Signal(0x14)
-	SIGCONT   = syscall.Signal(0x13)
-	SIGEMT    = syscall.Signal(0x7)
-	SIGFPE    = syscall.Signal(0x8)
-	SIGHUP    = syscall.Signal(0x1)
-	SIGILL    = syscall.Signal(0x4)
-	SIGINFO   = syscall.Signal(0x1d)
-	SIGINT    = syscall.Signal(0x2)
-	SIGIO     = syscall.Signal(0x17)
-	SIGIOT    = syscall.Signal(0x6)
-	SIGKILL   = syscall.Signal(0x9)
-	SIGPIPE   = syscall.Signal(0xd)
-	SIGPROF   = syscall.Signal(0x1b)
-	SIGQUIT   = syscall.Signal(0x3)
-	SIGSEGV   = syscall.Signal(0xb)
-	SIGSTOP   = syscall.Signal(0x11)
-	SIGSYS    = syscall.Signal(0xc)
-	SIGTERM   = syscall.Signal(0xf)
-	SIGTRAP   = syscall.Signal(0x5)
-	SIGTSTP   = syscall.Signal(0x12)
-	SIGTTIN   = syscall.Signal(0x15)
-	SIGTTOU   = syscall.Signal(0x16)
-	SIGURG    = syscall.Signal(0x10)
-	SIGUSR1   = syscall.Signal(0x1e)
-	SIGUSR2   = syscall.Signal(0x1f)
-	SIGVTALRM = syscall.Signal(0x1a)
-	SIGWINCH  = syscall.Signal(0x1c)
-	SIGXCPU   = syscall.Signal(0x18)
-	SIGXFSZ   = syscall.Signal(0x19)
-)
-
-// Error table
-var errorList = [...]struct {
-	num  syscall.Errno
-	name string
-	desc string
-}{
-	{1, "EPERM", "operation not permitted"},
-	{2, "ENOENT", "no such file or directory"},
-	{3, "ESRCH", "no such process"},
-	{4, "EINTR", "interrupted system call"},
-	{5, "EIO", "input/output error"},
-	{6, "ENXIO", "device not configured"},
-	{7, "E2BIG", "argument list too long"},
-	{8, "ENOEXEC", "exec format error"},
-	{9, "EBADF", "bad file descriptor"},
-	{10, "ECHILD", "no child processes"},
-	{11, "EDEADLK", "resource deadlock avoided"},
-	{12, "ENOMEM", "cannot allocate memory"},
-	{13, "EACCES", "permission denied"},
-	{14, "EFAULT", "bad address"},
-	{15, "ENOTBLK", "block device required"},
-	{16, "EBUSY", "resource busy"},
-	{17, "EEXIST", "file exists"},
-	{18, "EXDEV", "cross-device link"},
-	{19, "ENODEV", "operation not supported by device"},
-	{20, "ENOTDIR", "not a directory"},
-	{21, "EISDIR", "is a directory"},
-	{22, "EINVAL", "invalid argument"},
-	{23, "ENFILE", "too many open files in system"},
-	{24, "EMFILE", "too many open files"},
-	{25, "ENOTTY", "inappropriate ioctl for device"},
-	{26, "ETXTBSY", "text file busy"},
-	{27, "EFBIG", "file too large"},
-	{28, "ENOSPC", "no space left on device"},
-	{29, "ESPIPE", "illegal seek"},
-	{30, "EROFS", "read-only file system"},
-	{31, "EMLINK", "too many links"},
-	{32, "EPIPE", "broken pipe"},
-	{33, "EDOM", "numerical argument out of domain"},
-	{34, "ERANGE", "result too large"},
-	{35, "EAGAIN", "resource temporarily unavailable"},
-	{36, "EINPROGRESS", "operation now in progress"},
-	{37, "EALREADY", "operation already in progress"},
-	{38, "ENOTSOCK", "socket operation on non-socket"},
-	{39, "EDESTADDRREQ", "destination address required"},
-	{40, "EMSGSIZE", "message too long"},
-	{41, "EPROTOTYPE", "protocol wrong type for socket"},
-	{42, "ENOPROTOOPT", "protocol not available"},
-	{43, "EPROTONOSUPPORT", "protocol not supported"},
-	{44, "ESOCKTNOSUPPORT", "socket type not supported"},
-	{45, "ENOTSUP", "operation not supported"},
-	{46, "EPFNOSUPPORT", "protocol family not supported"},
-	{47, "EAFNOSUPPORT", "address family not supported by protocol family"},
-	{48, "EADDRINUSE", "address already in use"},
-	{49, "EADDRNOTAVAIL", "can't assign requested address"},
-	{50, "ENETDOWN", "network is down"},
-	{51, "ENETUNREACH", "network is unreachable"},
-	{52, "ENETRESET", "network dropped connection on reset"},
-	{53, "ECONNABORTED", "software caused connection abort"},
-	{54, "ECONNRESET", "connection reset by peer"},
-	{55, "ENOBUFS", "no buffer space available"},
-	{56, "EISCONN", "socket is already connected"},
-	{57, "ENOTCONN", "socket is not connected"},
-	{58, "ESHUTDOWN", "can't send after socket shutdown"},
-	{59, "ETOOMANYREFS", "too many references: can't splice"},
-	{60, "ETIMEDOUT", "operation timed out"},
-	{61, "ECONNREFUSED", "connection refused"},
-	{62, "ELOOP", "too many levels of symbolic links"},
-	{63, "ENAMETOOLONG", "file name too long"},
-	{64, "EHOSTDOWN", "host is down"},
-	{65, "EHOSTUNREACH", "no route to host"},
-	{66, "ENOTEMPTY", "directory not empty"},
-	{67, "EPROCLIM", "too many processes"},
-	{68, "EUSERS", "too many users"},
-	{69, "EDQUOT", "disc quota exceeded"},
-	{70, "ESTALE", "stale NFS file handle"},
-	{71, "EREMOTE", "too many levels of remote in path"},
-	{72, "EBADRPC", "RPC struct is bad"},
-	{73, "ERPCMISMATCH", "RPC version wrong"},
-	{74, "EPROGUNAVAIL", "RPC prog. not avail"},
-	{75, "EPROGMISMATCH", "program version wrong"},
-	{76, "EPROCUNAVAIL", "bad procedure for program"},
-	{77, "ENOLCK", "no locks available"},
-	{78, "ENOSYS", "function not implemented"},
-	{79, "EFTYPE", "inappropriate file type or format"},
-	{80, "EAUTH", "authentication error"},
-	{81, "ENEEDAUTH", "need authenticator"},
-	{82, "EPWROFF", "device power is off"},
-	{83, "EDEVERR", "device error"},
-	{84, "EOVERFLOW", "value too large to be stored in data type"},
-	{85, "EBADEXEC", "bad executable (or shared library)"},
-	{86, "EBADARCH", "bad CPU type in executable"},
-	{87, "ESHLIBVERS", "shared library version mismatch"},
-	{88, "EBADMACHO", "malformed Mach-o file"},
-	{89, "ECANCELED", "operation canceled"},
-	{90, "EIDRM", "identifier removed"},
-	{91, "ENOMSG", "no message of desired type"},
-	{92, "EILSEQ", "illegal byte sequence"},
-	{93, "ENOATTR", "attribute not found"},
-	{94, "EBADMSG", "bad message"},
-	{95, "EMULTIHOP", "EMULTIHOP (Reserved)"},
-	{96, "ENODATA", "no message available on STREAM"},
-	{97, "ENOLINK", "ENOLINK (Reserved)"},
-	{98, "ENOSR", "no STREAM resources"},
-	{99, "ENOSTR", "not a STREAM"},
-	{100, "EPROTO", "protocol error"},
-	{101, "ETIME", "STREAM ioctl timeout"},
-	{102, "EOPNOTSUPP", "operation not supported on socket"},
-	{103, "ENOPOLICY", "policy not found"},
-	{104, "ENOTRECOVERABLE", "state not recoverable"},
-	{105, "EOWNERDEAD", "previous owner died"},
-	{106, "EQFULL", "interface output queue is full"},
-}
-
-// Signal table
-var signalList = [...]struct {
-	num  syscall.Signal
-	name string
-	desc string
-}{
-	{1, "SIGHUP", "hangup"},
-	{2, "SIGINT", "interrupt"},
-	{3, "SIGQUIT", "quit"},
-	{4, "SIGILL", "illegal instruction"},
-	{5, "SIGTRAP", "trace/BPT trap"},
-	{6, "SIGABRT", "abort trap"},
-	{7, "SIGEMT", "EMT trap"},
-	{8, "SIGFPE", "floating point exception"},
-	{9, "SIGKILL", "killed"},
-	{10, "SIGBUS", "bus error"},
-	{11, "SIGSEGV", "segmentation fault"},
-	{12, "SIGSYS", "bad system call"},
-	{13, "SIGPIPE", "broken pipe"},
-	{14, "SIGALRM", "alarm clock"},
-	{15, "SIGTERM", "terminated"},
-	{16, "SIGURG", "urgent I/O condition"},
-	{17, "SIGSTOP", "suspended (signal)"},
-	{18, "SIGTSTP", "suspended"},
-	{19, "SIGCONT", "continued"},
-	{20, "SIGCHLD", "child exited"},
-	{21, "SIGTTIN", "stopped (tty input)"},
-	{22, "SIGTTOU", "stopped (tty output)"},
-	{23, "SIGIO", "I/O possible"},
-	{24, "SIGXCPU", "cputime limit exceeded"},
-	{25, "SIGXFSZ", "filesize limit exceeded"},
-	{26, "SIGVTALRM", "virtual timer expired"},
-	{27, "SIGPROF", "profiling timer expired"},
-	{28, "SIGWINCH", "window size changes"},
-	{29, "SIGINFO", "information request"},
-	{30, "SIGUSR1", "user defined signal 1"},
-	{31, "SIGUSR2", "user defined signal 2"},
-}
diff --git a/vendor/golang.org/x/sys/unix/zerrors_darwin_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_darwin_arm64.go
index 0614d26..e644eaf 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_darwin_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_darwin_arm64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && darwin
 // +build arm64,darwin
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -32,7 +33,7 @@
 	AF_LAT                            = 0xe
 	AF_LINK                           = 0x12
 	AF_LOCAL                          = 0x1
-	AF_MAX                            = 0x28
+	AF_MAX                            = 0x29
 	AF_NATM                           = 0x1f
 	AF_NDRV                           = 0x1b
 	AF_NETBIOS                        = 0x21
@@ -45,9 +46,11 @@
 	AF_SIP                            = 0x18
 	AF_SNA                            = 0xb
 	AF_SYSTEM                         = 0x20
+	AF_SYS_CONTROL                    = 0x2
 	AF_UNIX                           = 0x1
 	AF_UNSPEC                         = 0x0
 	AF_UTUN                           = 0x26
+	AF_VSOCK                          = 0x28
 	ALTWERASE                         = 0x200
 	ATTR_BIT_MAP_COUNT                = 0x5
 	ATTR_CMN_ACCESSMASK               = 0x20000
@@ -82,7 +85,7 @@
 	ATTR_CMN_PAROBJID                 = 0x80
 	ATTR_CMN_RETURNED_ATTRS           = 0x80000000
 	ATTR_CMN_SCRIPT                   = 0x100
-	ATTR_CMN_SETMASK                  = 0x41c7ff00
+	ATTR_CMN_SETMASK                  = 0x51c7ff00
 	ATTR_CMN_USERACCESS               = 0x200000
 	ATTR_CMN_UUID                     = 0x800000
 	ATTR_CMN_VALIDMASK                = 0xffffffff
@@ -251,6 +254,7 @@
 	CSTOP                             = 0x13
 	CSTOPB                            = 0x400
 	CSUSP                             = 0x1a
+	CTLIOCGINFO                       = 0xc0644e03
 	CTL_HW                            = 0x6
 	CTL_KERN                          = 0x1
 	CTL_MAXNAME                       = 0xc
@@ -355,7 +359,7 @@
 	DLT_LINUX_SLL                     = 0x71
 	DLT_LOOP                          = 0x6c
 	DLT_LTALK                         = 0x72
-	DLT_MATCHING_MAX                  = 0xf5
+	DLT_MATCHING_MAX                  = 0x10a
 	DLT_MATCHING_MIN                  = 0x68
 	DLT_MFR                           = 0xb6
 	DLT_MOST                          = 0xd3
@@ -396,6 +400,7 @@
 	DLT_SYMANTEC_FIREWALL             = 0x63
 	DLT_TZSP                          = 0x80
 	DLT_USB                           = 0xba
+	DLT_USB_DARWIN                    = 0x10a
 	DLT_USB_LINUX                     = 0xbd
 	DLT_USB_LINUX_MMAPPED             = 0xdc
 	DLT_USER0                         = 0x93
@@ -440,8 +445,8 @@
 	EVFILT_PROC                       = -0x5
 	EVFILT_READ                       = -0x1
 	EVFILT_SIGNAL                     = -0x6
-	EVFILT_SYSCOUNT                   = 0xf
-	EVFILT_THREADMARKER               = 0xf
+	EVFILT_SYSCOUNT                   = 0x11
+	EVFILT_THREADMARKER               = 0x11
 	EVFILT_TIMER                      = -0x7
 	EVFILT_USER                       = -0xa
 	EVFILT_VM                         = -0xc
@@ -479,9 +484,12 @@
 	FSOPT_NOINMEMUPDATE               = 0x2
 	FSOPT_PACK_INVAL_ATTRS            = 0x8
 	FSOPT_REPORT_FULLSIZE             = 0x4
+	FSOPT_RETURN_REALDEV              = 0x200
 	F_ADDFILESIGS                     = 0x3d
 	F_ADDFILESIGS_FOR_DYLD_SIM        = 0x53
+	F_ADDFILESIGS_INFO                = 0x67
 	F_ADDFILESIGS_RETURN              = 0x61
+	F_ADDFILESUPPL                    = 0x68
 	F_ADDSIGS                         = 0x3b
 	F_ALLOCATEALL                     = 0x4
 	F_ALLOCATECONTIG                  = 0x2
@@ -503,8 +511,10 @@
 	F_GETOWN                          = 0x5
 	F_GETPATH                         = 0x32
 	F_GETPATH_MTMINFO                 = 0x47
+	F_GETPATH_NOFIRMLINK              = 0x66
 	F_GETPROTECTIONCLASS              = 0x3f
 	F_GETPROTECTIONLEVEL              = 0x4d
+	F_GETSIGSINFO                     = 0x69
 	F_GLOBAL_NOCACHE                  = 0x37
 	F_LOG2PHYS                        = 0x31
 	F_LOG2PHYS_EXT                    = 0x41
@@ -529,6 +539,7 @@
 	F_SETPROTECTIONCLASS              = 0x40
 	F_SETSIZE                         = 0x2b
 	F_SINGLE_WRITER                   = 0x4c
+	F_SPECULATIVE_READ                = 0x65
 	F_THAW_FS                         = 0x36
 	F_TRANSCODEKEY                    = 0x4b
 	F_TRIM_ACTIVE_FILE                = 0x64
@@ -560,6 +571,7 @@
 	IFF_UP                            = 0x1
 	IFNAMSIZ                          = 0x10
 	IFT_1822                          = 0x2
+	IFT_6LOWPAN                       = 0x40
 	IFT_AAL5                          = 0x31
 	IFT_ARCNET                        = 0x23
 	IFT_ARCNETPLUS                    = 0x24
@@ -764,16 +776,28 @@
 	IPV6_2292PKTINFO                  = 0x13
 	IPV6_2292PKTOPTIONS               = 0x19
 	IPV6_2292RTHDR                    = 0x18
+	IPV6_3542DSTOPTS                  = 0x32
+	IPV6_3542HOPLIMIT                 = 0x2f
+	IPV6_3542HOPOPTS                  = 0x31
+	IPV6_3542NEXTHOP                  = 0x30
+	IPV6_3542PKTINFO                  = 0x2e
+	IPV6_3542RTHDR                    = 0x33
+	IPV6_ADDR_MC_FLAGS_PREFIX         = 0x20
+	IPV6_ADDR_MC_FLAGS_TRANSIENT      = 0x10
+	IPV6_ADDR_MC_FLAGS_UNICAST_BASED  = 0x30
+	IPV6_AUTOFLOWLABEL                = 0x3b
 	IPV6_BINDV6ONLY                   = 0x1b
 	IPV6_BOUND_IF                     = 0x7d
 	IPV6_CHECKSUM                     = 0x1a
 	IPV6_DEFAULT_MULTICAST_HOPS       = 0x1
 	IPV6_DEFAULT_MULTICAST_LOOP       = 0x1
 	IPV6_DEFHLIM                      = 0x40
+	IPV6_DONTFRAG                     = 0x3e
+	IPV6_DSTOPTS                      = 0x32
 	IPV6_FAITH                        = 0x1d
 	IPV6_FLOWINFO_MASK                = 0xffffff0f
 	IPV6_FLOWLABEL_MASK               = 0xffff0f00
-	IPV6_FLOW_ECN_MASK                = 0x300
+	IPV6_FLOW_ECN_MASK                = 0x3000
 	IPV6_FRAGTTL                      = 0x3c
 	IPV6_FW_ADD                       = 0x1e
 	IPV6_FW_DEL                       = 0x1f
@@ -781,6 +805,8 @@
 	IPV6_FW_GET                       = 0x22
 	IPV6_FW_ZERO                      = 0x21
 	IPV6_HLIMDEC                      = 0x1
+	IPV6_HOPLIMIT                     = 0x2f
+	IPV6_HOPOPTS                      = 0x31
 	IPV6_IPSEC_POLICY                 = 0x1c
 	IPV6_JOIN_GROUP                   = 0xc
 	IPV6_LEAVE_GROUP                  = 0xd
@@ -792,20 +818,34 @@
 	IPV6_MAX_SOCK_SRC_FILTER          = 0x80
 	IPV6_MIN_MEMBERSHIPS              = 0x1f
 	IPV6_MMTU                         = 0x500
+	IPV6_MSFILTER                     = 0x4a
 	IPV6_MULTICAST_HOPS               = 0xa
 	IPV6_MULTICAST_IF                 = 0x9
 	IPV6_MULTICAST_LOOP               = 0xb
+	IPV6_NEXTHOP                      = 0x30
+	IPV6_PATHMTU                      = 0x2c
+	IPV6_PKTINFO                      = 0x2e
 	IPV6_PORTRANGE                    = 0xe
 	IPV6_PORTRANGE_DEFAULT            = 0x0
 	IPV6_PORTRANGE_HIGH               = 0x1
 	IPV6_PORTRANGE_LOW                = 0x2
+	IPV6_PREFER_TEMPADDR              = 0x3f
+	IPV6_RECVDSTOPTS                  = 0x28
+	IPV6_RECVHOPLIMIT                 = 0x25
+	IPV6_RECVHOPOPTS                  = 0x27
+	IPV6_RECVPATHMTU                  = 0x2b
+	IPV6_RECVPKTINFO                  = 0x3d
+	IPV6_RECVRTHDR                    = 0x26
 	IPV6_RECVTCLASS                   = 0x23
+	IPV6_RTHDR                        = 0x33
+	IPV6_RTHDRDSTOPTS                 = 0x39
 	IPV6_RTHDR_LOOSE                  = 0x0
 	IPV6_RTHDR_STRICT                 = 0x1
 	IPV6_RTHDR_TYPE_0                 = 0x0
 	IPV6_SOCKOPT_RESERVED1            = 0x3
 	IPV6_TCLASS                       = 0x24
 	IPV6_UNICAST_HOPS                 = 0x4
+	IPV6_USE_MIN_MTU                  = 0x2a
 	IPV6_V6ONLY                       = 0x1b
 	IPV6_VERSION                      = 0x60
 	IPV6_VERSION_MASK                 = 0xf0
@@ -816,6 +856,7 @@
 	IP_DEFAULT_MULTICAST_LOOP         = 0x1
 	IP_DEFAULT_MULTICAST_TTL          = 0x1
 	IP_DF                             = 0x4000
+	IP_DONTFRAG                       = 0x1c
 	IP_DROP_MEMBERSHIP                = 0xd
 	IP_DROP_SOURCE_MEMBERSHIP         = 0x47
 	IP_DUMMYNET_CONFIGURE             = 0x3c
@@ -887,6 +928,12 @@
 	KERN_OSRELEASE                    = 0x2
 	KERN_OSTYPE                       = 0x1
 	KERN_VERSION                      = 0x4
+	LOCAL_PEERCRED                    = 0x1
+	LOCAL_PEEREPID                    = 0x3
+	LOCAL_PEEREUUID                   = 0x5
+	LOCAL_PEERPID                     = 0x2
+	LOCAL_PEERTOKEN                   = 0x6
+	LOCAL_PEERUUID                    = 0x4
 	LOCK_EX                           = 0x2
 	LOCK_NB                           = 0x4
 	LOCK_SH                           = 0x1
@@ -902,6 +949,7 @@
 	MADV_SEQUENTIAL                   = 0x2
 	MADV_WILLNEED                     = 0x3
 	MADV_ZERO_WIRED_PAGES             = 0x6
+	MAP_32BIT                         = 0x8000
 	MAP_ANON                          = 0x1000
 	MAP_ANONYMOUS                     = 0x1000
 	MAP_COPY                          = 0x2
@@ -918,6 +966,17 @@
 	MAP_RESILIENT_CODESIGN            = 0x2000
 	MAP_RESILIENT_MEDIA               = 0x4000
 	MAP_SHARED                        = 0x1
+	MAP_TRANSLATED_ALLOW_EXECUTE      = 0x20000
+	MAP_UNIX03                        = 0x40000
+	MCAST_BLOCK_SOURCE                = 0x54
+	MCAST_EXCLUDE                     = 0x2
+	MCAST_INCLUDE                     = 0x1
+	MCAST_JOIN_GROUP                  = 0x50
+	MCAST_JOIN_SOURCE_GROUP           = 0x52
+	MCAST_LEAVE_GROUP                 = 0x51
+	MCAST_LEAVE_SOURCE_GROUP          = 0x53
+	MCAST_UNBLOCK_SOURCE              = 0x55
+	MCAST_UNDEFINED                   = 0x0
 	MCL_CURRENT                       = 0x1
 	MCL_FUTURE                        = 0x2
 	MNT_ASYNC                         = 0x40
@@ -929,6 +988,7 @@
 	MNT_DOVOLFS                       = 0x8000
 	MNT_DWAIT                         = 0x4
 	MNT_EXPORTED                      = 0x100
+	MNT_EXT_ROOT_DATA_VOL             = 0x1
 	MNT_FORCE                         = 0x80000
 	MNT_IGNORE_OWNERSHIP              = 0x200000
 	MNT_JOURNALED                     = 0x800000
@@ -945,12 +1005,15 @@
 	MNT_QUOTA                         = 0x2000
 	MNT_RDONLY                        = 0x1
 	MNT_RELOAD                        = 0x40000
+	MNT_REMOVABLE                     = 0x200
 	MNT_ROOTFS                        = 0x4000
+	MNT_SNAPSHOT                      = 0x40000000
+	MNT_STRICTATIME                   = 0x80000000
 	MNT_SYNCHRONOUS                   = 0x2
 	MNT_UNION                         = 0x20
 	MNT_UNKNOWNPERMISSIONS            = 0x200000
 	MNT_UPDATE                        = 0x10000
-	MNT_VISFLAGMASK                   = 0x17f0f5ff
+	MNT_VISFLAGMASK                   = 0xd7f0f7ff
 	MNT_WAIT                          = 0x1
 	MSG_CTRUNC                        = 0x20
 	MSG_DONTROUTE                     = 0x4
@@ -961,6 +1024,7 @@
 	MSG_HAVEMORE                      = 0x2000
 	MSG_HOLD                          = 0x800
 	MSG_NEEDSA                        = 0x10000
+	MSG_NOSIGNAL                      = 0x80000
 	MSG_OOB                           = 0x1
 	MSG_PEEK                          = 0x2
 	MSG_RCVMORE                       = 0x4000
@@ -977,9 +1041,10 @@
 	NET_RT_DUMP                       = 0x1
 	NET_RT_DUMP2                      = 0x7
 	NET_RT_FLAGS                      = 0x2
+	NET_RT_FLAGS_PRIV                 = 0xa
 	NET_RT_IFLIST                     = 0x3
 	NET_RT_IFLIST2                    = 0x6
-	NET_RT_MAXID                      = 0xa
+	NET_RT_MAXID                      = 0xb
 	NET_RT_STAT                       = 0x4
 	NET_RT_TRASH                      = 0x5
 	NFDBITS                           = 0x20
@@ -1017,6 +1082,7 @@
 	NOTE_LEEWAY                       = 0x10
 	NOTE_LINK                         = 0x10
 	NOTE_LOWAT                        = 0x1
+	NOTE_MACHTIME                     = 0x100
 	NOTE_MACH_CONTINUOUS_TIME         = 0x80
 	NOTE_NONE                         = 0x80
 	NOTE_NSECONDS                     = 0x4
@@ -1063,6 +1129,7 @@
 	O_NDELAY                          = 0x4
 	O_NOCTTY                          = 0x20000
 	O_NOFOLLOW                        = 0x100
+	O_NOFOLLOW_ANY                    = 0x20000000
 	O_NONBLOCK                        = 0x4
 	O_POPUP                           = 0x80000000
 	O_RDONLY                          = 0x0
@@ -1134,6 +1201,7 @@
 	RTF_BROADCAST                     = 0x400000
 	RTF_CLONING                       = 0x100
 	RTF_CONDEMNED                     = 0x2000000
+	RTF_DEAD                          = 0x20000000
 	RTF_DELCLONE                      = 0x80
 	RTF_DONE                          = 0x40
 	RTF_DYNAMIC                       = 0x10
@@ -1141,6 +1209,7 @@
 	RTF_HOST                          = 0x4
 	RTF_IFREF                         = 0x4000000
 	RTF_IFSCOPE                       = 0x1000000
+	RTF_LLDATA                        = 0x400
 	RTF_LLINFO                        = 0x400
 	RTF_LOCAL                         = 0x200000
 	RTF_MODIFIED                      = 0x20
@@ -1208,6 +1277,7 @@
 	SIOCGDRVSPEC                      = 0xc028697b
 	SIOCGETVLAN                       = 0xc020697f
 	SIOCGHIWAT                        = 0x40047301
+	SIOCGIF6LOWPAN                    = 0xc02069c5
 	SIOCGIFADDR                       = 0xc0206921
 	SIOCGIFALTMTU                     = 0xc0206948
 	SIOCGIFASYNCMAP                   = 0xc020697c
@@ -1218,6 +1288,7 @@
 	SIOCGIFDEVMTU                     = 0xc0206944
 	SIOCGIFDSTADDR                    = 0xc0206922
 	SIOCGIFFLAGS                      = 0xc0206911
+	SIOCGIFFUNCTIONALTYPE             = 0xc02069ad
 	SIOCGIFGENERIC                    = 0xc020693a
 	SIOCGIFKPI                        = 0xc0206987
 	SIOCGIFMAC                        = 0xc0206982
@@ -1231,6 +1302,7 @@
 	SIOCGIFSTATUS                     = 0xc331693d
 	SIOCGIFVLAN                       = 0xc020697f
 	SIOCGIFWAKEFLAGS                  = 0xc0206988
+	SIOCGIFXMEDIA                     = 0xc02c6948
 	SIOCGLOWAT                        = 0x40047303
 	SIOCGPGRP                         = 0x40047309
 	SIOCIFCREATE                      = 0xc0206978
@@ -1241,6 +1313,7 @@
 	SIOCSDRVSPEC                      = 0x8028697b
 	SIOCSETVLAN                       = 0x8020697e
 	SIOCSHIWAT                        = 0x80047300
+	SIOCSIF6LOWPAN                    = 0x802069c4
 	SIOCSIFADDR                       = 0x8020690c
 	SIOCSIFALTMTU                     = 0x80206945
 	SIOCSIFASYNCMAP                   = 0x8020697d
@@ -1268,6 +1341,7 @@
 	SOCK_RDM                          = 0x4
 	SOCK_SEQPACKET                    = 0x5
 	SOCK_STREAM                       = 0x1
+	SOL_LOCAL                         = 0x0
 	SOL_SOCKET                        = 0xffff
 	SOMAXCONN                         = 0x80
 	SO_ACCEPTCONN                     = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_dragonfly_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_dragonfly_amd64.go
index 6130471..17bba0e 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_dragonfly_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_dragonfly_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && dragonfly
 // +build amd64,dragonfly
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -62,6 +63,7 @@
 	B28800                            = 0x7080
 	B300                              = 0x12c
 	B38400                            = 0x9600
+	B460800                           = 0x70800
 	B4800                             = 0x12c0
 	B50                               = 0x32
 	B57600                            = 0xe100
@@ -69,12 +71,15 @@
 	B7200                             = 0x1c20
 	B75                               = 0x4b
 	B76800                            = 0x12c00
+	B921600                           = 0xe1000
 	B9600                             = 0x2580
+	BIOCFEEDBACK                      = 0x8004427d
 	BIOCFLUSH                         = 0x20004268
 	BIOCGBLEN                         = 0x40044266
 	BIOCGDLT                          = 0x4004426a
 	BIOCGDLTLIST                      = 0xc0104279
 	BIOCGETIF                         = 0x4020426b
+	BIOCGFEEDBACK                     = 0x4004427c
 	BIOCGHDRCMPLT                     = 0x40044274
 	BIOCGRSIG                         = 0x40044272
 	BIOCGRTIMEOUT                     = 0x4010426e
@@ -88,6 +93,7 @@
 	BIOCSETF                          = 0x80104267
 	BIOCSETIF                         = 0x8020426c
 	BIOCSETWF                         = 0x8010427b
+	BIOCSFEEDBACK                     = 0x8004427d
 	BIOCSHDRCMPLT                     = 0x80044275
 	BIOCSRSIG                         = 0x80044273
 	BIOCSRTIMEOUT                     = 0x8010426d
@@ -125,6 +131,7 @@
 	BPF_MINBUFSIZE                    = 0x20
 	BPF_MINOR_VERSION                 = 0x1
 	BPF_MISC                          = 0x7
+	BPF_MOD                           = 0x90
 	BPF_MSH                           = 0xa0
 	BPF_MUL                           = 0x20
 	BPF_NEG                           = 0x80
@@ -139,6 +146,7 @@
 	BPF_TXA                           = 0x80
 	BPF_W                             = 0x0
 	BPF_X                             = 0x8
+	BPF_XOR                           = 0xa0
 	BRKINT                            = 0x2
 	CFLUSH                            = 0xf
 	CLOCAL                            = 0x8000
@@ -156,6 +164,12 @@
 	CLOCK_UPTIME_FAST                 = 0x8
 	CLOCK_UPTIME_PRECISE              = 0x7
 	CLOCK_VIRTUAL                     = 0x1
+	CPUSTATES                         = 0x5
+	CP_IDLE                           = 0x4
+	CP_INTR                           = 0x3
+	CP_NICE                           = 0x1
+	CP_SYS                            = 0x2
+	CP_USER                           = 0x0
 	CREAD                             = 0x800
 	CRTSCTS                           = 0x30000
 	CS5                               = 0x0
@@ -175,6 +189,7 @@
 	DLT_A429                          = 0xb8
 	DLT_A653_ICM                      = 0xb9
 	DLT_AIRONET_HEADER                = 0x78
+	DLT_AOS                           = 0xde
 	DLT_APPLE_IP_OVER_IEEE1394        = 0x8a
 	DLT_ARCNET                        = 0x7
 	DLT_ARCNET_LINUX                  = 0x81
@@ -184,22 +199,33 @@
 	DLT_AX25                          = 0x3
 	DLT_AX25_KISS                     = 0xca
 	DLT_BACNET_MS_TP                  = 0xa5
+	DLT_BLUETOOTH_BREDR_BB            = 0xff
 	DLT_BLUETOOTH_HCI_H4              = 0xbb
 	DLT_BLUETOOTH_HCI_H4_WITH_PHDR    = 0xc9
+	DLT_BLUETOOTH_LE_LL               = 0xfb
+	DLT_BLUETOOTH_LE_LL_WITH_PHDR     = 0x100
+	DLT_BLUETOOTH_LINUX_MONITOR       = 0xfe
 	DLT_CAN20B                        = 0xbe
+	DLT_CAN_SOCKETCAN                 = 0xe3
 	DLT_CHAOS                         = 0x5
 	DLT_CHDLC                         = 0x68
 	DLT_CISCO_IOS                     = 0x76
 	DLT_C_HDLC                        = 0x68
 	DLT_C_HDLC_WITH_DIR               = 0xcd
+	DLT_DBUS                          = 0xe7
+	DLT_DECT                          = 0xdd
 	DLT_DOCSIS                        = 0x8f
+	DLT_DVB_CI                        = 0xeb
 	DLT_ECONET                        = 0x73
 	DLT_EN10MB                        = 0x1
 	DLT_EN3MB                         = 0x2
 	DLT_ENC                           = 0x6d
+	DLT_EPON                          = 0x103
 	DLT_ERF                           = 0xc5
 	DLT_ERF_ETH                       = 0xaf
 	DLT_ERF_POS                       = 0xb0
+	DLT_FC_2                          = 0xe0
+	DLT_FC_2_WITH_FRAME_DELIMS        = 0xe1
 	DLT_FDDI                          = 0xa
 	DLT_FLEXRAY                       = 0xd2
 	DLT_FRELAY                        = 0x6b
@@ -209,6 +235,8 @@
 	DLT_GPF_F                         = 0xab
 	DLT_GPF_T                         = 0xaa
 	DLT_GPRS_LLC                      = 0xa9
+	DLT_GSMTAP_ABIS                   = 0xda
+	DLT_GSMTAP_UM                     = 0xd9
 	DLT_HHDLC                         = 0x79
 	DLT_IBM_SN                        = 0x92
 	DLT_IBM_SP                        = 0x91
@@ -218,18 +246,28 @@
 	DLT_IEEE802_11_RADIO_AVS          = 0xa3
 	DLT_IEEE802_15_4                  = 0xc3
 	DLT_IEEE802_15_4_LINUX            = 0xbf
+	DLT_IEEE802_15_4_NOFCS            = 0xe6
 	DLT_IEEE802_15_4_NONASK_PHY       = 0xd7
 	DLT_IEEE802_16_MAC_CPS            = 0xbc
 	DLT_IEEE802_16_MAC_CPS_RADIO      = 0xc1
+	DLT_INFINIBAND                    = 0xf7
 	DLT_IPFILTER                      = 0x74
 	DLT_IPMB                          = 0xc7
 	DLT_IPMB_LINUX                    = 0xd1
+	DLT_IPMI_HPM_2                    = 0x104
+	DLT_IPNET                         = 0xe2
+	DLT_IPOIB                         = 0xf2
+	DLT_IPV4                          = 0xe4
+	DLT_IPV6                          = 0xe5
 	DLT_IP_OVER_FC                    = 0x7a
+	DLT_ISO_14443                     = 0x108
 	DLT_JUNIPER_ATM1                  = 0x89
 	DLT_JUNIPER_ATM2                  = 0x87
+	DLT_JUNIPER_ATM_CEMIC             = 0xee
 	DLT_JUNIPER_CHDLC                 = 0xb5
 	DLT_JUNIPER_ES                    = 0x84
 	DLT_JUNIPER_ETHER                 = 0xb2
+	DLT_JUNIPER_FIBRECHANNEL          = 0xea
 	DLT_JUNIPER_FRELAY                = 0xb4
 	DLT_JUNIPER_GGSN                  = 0x85
 	DLT_JUNIPER_ISM                   = 0xc2
@@ -242,25 +280,40 @@
 	DLT_JUNIPER_PPPOE                 = 0xa7
 	DLT_JUNIPER_PPPOE_ATM             = 0xa8
 	DLT_JUNIPER_SERVICES              = 0x88
+	DLT_JUNIPER_SRX_E2E               = 0xe9
 	DLT_JUNIPER_ST                    = 0xc8
 	DLT_JUNIPER_VP                    = 0xb7
+	DLT_JUNIPER_VS                    = 0xe8
 	DLT_LAPB_WITH_DIR                 = 0xcf
 	DLT_LAPD                          = 0xcb
 	DLT_LIN                           = 0xd4
+	DLT_LINUX_EVDEV                   = 0xd8
 	DLT_LINUX_IRDA                    = 0x90
 	DLT_LINUX_LAPD                    = 0xb1
 	DLT_LINUX_SLL                     = 0x71
 	DLT_LOOP                          = 0x6c
 	DLT_LTALK                         = 0x72
+	DLT_MATCHING_MAX                  = 0x109
+	DLT_MATCHING_MIN                  = 0x68
 	DLT_MFR                           = 0xb6
 	DLT_MOST                          = 0xd3
+	DLT_MPEG_2_TS                     = 0xf3
+	DLT_MPLS                          = 0xdb
 	DLT_MTP2                          = 0x8c
 	DLT_MTP2_WITH_PHDR                = 0x8b
 	DLT_MTP3                          = 0x8d
+	DLT_MUX27010                      = 0xec
+	DLT_NETANALYZER                   = 0xf0
+	DLT_NETANALYZER_TRANSPARENT       = 0xf1
+	DLT_NETLINK                       = 0xfd
+	DLT_NFC_LLCP                      = 0xf5
+	DLT_NFLOG                         = 0xef
+	DLT_NG40                          = 0xf4
 	DLT_NULL                          = 0x0
 	DLT_PCI_EXP                       = 0x7d
 	DLT_PFLOG                         = 0x75
 	DLT_PFSYNC                        = 0x12
+	DLT_PKTAP                         = 0x102
 	DLT_PPI                           = 0xc0
 	DLT_PPP                           = 0x9
 	DLT_PPP_BSDOS                     = 0x10
@@ -269,22 +322,51 @@
 	DLT_PPP_SERIAL                    = 0x32
 	DLT_PPP_WITH_DIR                  = 0xcc
 	DLT_PRISM_HEADER                  = 0x77
+	DLT_PROFIBUS_DL                   = 0x101
 	DLT_PRONET                        = 0x4
 	DLT_RAIF1                         = 0xc6
 	DLT_RAW                           = 0xc
+	DLT_RDS                           = 0x109
 	DLT_REDBACK_SMARTEDGE             = 0x20
 	DLT_RIO                           = 0x7c
+	DLT_RTAC_SERIAL                   = 0xfa
 	DLT_SCCP                          = 0x8e
+	DLT_SCTP                          = 0xf8
 	DLT_SITA                          = 0xc4
 	DLT_SLIP                          = 0x8
 	DLT_SLIP_BSDOS                    = 0xf
+	DLT_STANAG_5066_D_PDU             = 0xed
 	DLT_SUNATM                        = 0x7b
 	DLT_SYMANTEC_FIREWALL             = 0x63
 	DLT_TZSP                          = 0x80
 	DLT_USB                           = 0xba
+	DLT_USBPCAP                       = 0xf9
+	DLT_USB_FREEBSD                   = 0xba
 	DLT_USB_LINUX                     = 0xbd
+	DLT_USB_LINUX_MMAPPED             = 0xdc
+	DLT_USER0                         = 0x93
+	DLT_USER1                         = 0x94
+	DLT_USER10                        = 0x9d
+	DLT_USER11                        = 0x9e
+	DLT_USER12                        = 0x9f
+	DLT_USER13                        = 0xa0
+	DLT_USER14                        = 0xa1
+	DLT_USER15                        = 0xa2
+	DLT_USER2                         = 0x95
+	DLT_USER3                         = 0x96
+	DLT_USER4                         = 0x97
+	DLT_USER5                         = 0x98
+	DLT_USER6                         = 0x99
+	DLT_USER7                         = 0x9a
+	DLT_USER8                         = 0x9b
+	DLT_USER9                         = 0x9c
+	DLT_WATTSTOPPER_DLM               = 0x107
+	DLT_WIHART                        = 0xdf
+	DLT_WIRESHARK_UPPER_PDU           = 0xfc
 	DLT_X2E_SERIAL                    = 0xd5
 	DLT_X2E_XORAYA                    = 0xd6
+	DLT_ZWAVE_R1_R2                   = 0x105
+	DLT_ZWAVE_R3                      = 0x106
 	DT_BLK                            = 0x6
 	DT_CHR                            = 0x2
 	DT_DBF                            = 0xf
@@ -323,10 +405,11 @@
 	EV_EOF                            = 0x8000
 	EV_ERROR                          = 0x4000
 	EV_FLAG1                          = 0x2000
+	EV_HUP                            = 0x800
 	EV_NODATA                         = 0x1000
 	EV_ONESHOT                        = 0x10
 	EV_RECEIPT                        = 0x40
-	EV_SYSFLAGS                       = 0xf000
+	EV_SYSFLAGS                       = 0xf800
 	EXTA                              = 0x4b00
 	EXTB                              = 0x9600
 	EXTEXIT_LWP                       = 0x10000
@@ -365,8 +448,9 @@
 	IFF_ALLMULTI                      = 0x200
 	IFF_ALTPHYS                       = 0x4000
 	IFF_BROADCAST                     = 0x2
-	IFF_CANTCHANGE                    = 0x118e72
+	IFF_CANTCHANGE                    = 0x318e72
 	IFF_DEBUG                         = 0x4
+	IFF_IDIRECT                       = 0x200000
 	IFF_LINK0                         = 0x1000
 	IFF_LINK1                         = 0x2000
 	IFF_LINK2                         = 0x4000
@@ -441,7 +525,6 @@
 	IFT_EPLRS                         = 0x57
 	IFT_ESCON                         = 0x49
 	IFT_ETHER                         = 0x6
-	IFT_FAITH                         = 0xf2
 	IFT_FAST                          = 0x7d
 	IFT_FASTETHER                     = 0x3e
 	IFT_FASTETHERFX                   = 0x45
@@ -614,6 +697,7 @@
 	IN_CLASSD_NET                     = 0xf0000000
 	IN_CLASSD_NSHIFT                  = 0x1c
 	IN_LOOPBACKNET                    = 0x7f
+	IN_RFC3021_MASK                   = 0xfffffffe
 	IPPROTO_3PC                       = 0x22
 	IPPROTO_ADFS                      = 0x44
 	IPPROTO_AH                        = 0x33
@@ -735,7 +819,6 @@
 	IPV6_DEFHLIM                      = 0x40
 	IPV6_DONTFRAG                     = 0x3e
 	IPV6_DSTOPTS                      = 0x32
-	IPV6_FAITH                        = 0x1d
 	IPV6_FLOWINFO_MASK                = 0xffffff0f
 	IPV6_FLOWLABEL_MASK               = 0xffff0f00
 	IPV6_FRAGTTL                      = 0x78
@@ -747,7 +830,6 @@
 	IPV6_HLIMDEC                      = 0x1
 	IPV6_HOPLIMIT                     = 0x2f
 	IPV6_HOPOPTS                      = 0x31
-	IPV6_IPSEC_POLICY                 = 0x1c
 	IPV6_JOIN_GROUP                   = 0xc
 	IPV6_LEAVE_GROUP                  = 0xd
 	IPV6_MAXHLIM                      = 0xff
@@ -795,16 +877,22 @@
 	IP_DUMMYNET_DEL                   = 0x3d
 	IP_DUMMYNET_FLUSH                 = 0x3e
 	IP_DUMMYNET_GET                   = 0x40
-	IP_FAITH                          = 0x16
 	IP_FW_ADD                         = 0x32
 	IP_FW_DEL                         = 0x33
 	IP_FW_FLUSH                       = 0x34
 	IP_FW_GET                         = 0x36
 	IP_FW_RESETLOG                    = 0x37
+	IP_FW_TBL_ADD                     = 0x2a
+	IP_FW_TBL_CREATE                  = 0x28
+	IP_FW_TBL_DEL                     = 0x2b
+	IP_FW_TBL_DESTROY                 = 0x29
+	IP_FW_TBL_EXPIRE                  = 0x2f
+	IP_FW_TBL_FLUSH                   = 0x2c
+	IP_FW_TBL_GET                     = 0x2d
+	IP_FW_TBL_ZERO                    = 0x2e
 	IP_FW_X                           = 0x31
 	IP_FW_ZERO                        = 0x35
 	IP_HDRINCL                        = 0x2
-	IP_IPSEC_POLICY                   = 0x15
 	IP_MAXPACKET                      = 0xffff
 	IP_MAX_MEMBERSHIPS                = 0x14
 	IP_MF                             = 0x2000
@@ -1080,12 +1168,10 @@
 	RTM_MISS                          = 0x7
 	RTM_NEWADDR                       = 0xc
 	RTM_NEWMADDR                      = 0xf
-	RTM_OLDADD                        = 0x9
-	RTM_OLDDEL                        = 0xa
 	RTM_REDIRECT                      = 0x6
 	RTM_RESOLVE                       = 0xb
 	RTM_RTTUNIT                       = 0xf4240
-	RTM_VERSION                       = 0x6
+	RTM_VERSION                       = 0x7
 	RTV_EXPIRE                        = 0x4
 	RTV_HOPCOUNT                      = 0x2
 	RTV_IWCAPSEGS                     = 0x400
@@ -1106,13 +1192,13 @@
 	SHUT_RDWR                         = 0x2
 	SHUT_WR                           = 0x1
 	SIOCADDMULTI                      = 0x80206931
-	SIOCADDRT                         = 0x8040720a
 	SIOCAIFADDR                       = 0x8040691a
+	SIOCAIFGROUP                      = 0x80286987
 	SIOCALIFADDR                      = 0x8118691b
 	SIOCATMARK                        = 0x40047307
 	SIOCDELMULTI                      = 0x80206932
-	SIOCDELRT                         = 0x8040720b
 	SIOCDIFADDR                       = 0x80206919
+	SIOCDIFGROUP                      = 0x80286989
 	SIOCDIFPHYADDR                    = 0x80206949
 	SIOCDLIFADDR                      = 0x8118691d
 	SIOCGDRVSPEC                      = 0xc028697b
@@ -1120,6 +1206,7 @@
 	SIOCGETVIFCNT                     = 0xc028720f
 	SIOCGHIWAT                        = 0x40047301
 	SIOCGIFADDR                       = 0xc0206921
+	SIOCGIFALIAS                      = 0xc0406929
 	SIOCGIFBRDADDR                    = 0xc0206923
 	SIOCGIFCAP                        = 0xc020691f
 	SIOCGIFCONF                       = 0xc0106924
@@ -1128,6 +1215,7 @@
 	SIOCGIFFLAGS                      = 0xc0206911
 	SIOCGIFGENERIC                    = 0xc020693a
 	SIOCGIFGMEMB                      = 0xc028698a
+	SIOCGIFGROUP                      = 0xc0286988
 	SIOCGIFINDEX                      = 0xc0206920
 	SIOCGIFMEDIA                      = 0xc0306938
 	SIOCGIFMETRIC                     = 0xc0206917
@@ -1194,6 +1282,7 @@
 	SO_RCVBUF                         = 0x1002
 	SO_RCVLOWAT                       = 0x1004
 	SO_RCVTIMEO                       = 0x1006
+	SO_RERROR                         = 0x2000
 	SO_REUSEADDR                      = 0x4
 	SO_REUSEPORT                      = 0x200
 	SO_SNDBUF                         = 0x1001
@@ -1233,6 +1322,9 @@
 	S_IXGRP                           = 0x8
 	S_IXOTH                           = 0x1
 	S_IXUSR                           = 0x40
+	TAB0                              = 0x0
+	TAB3                              = 0x4
+	TABDLY                            = 0x4
 	TCIFLUSH                          = 0x1
 	TCIOFF                            = 0x3
 	TCIOFLUSH                         = 0x3
@@ -1259,6 +1351,8 @@
 	TCP_NOPUSH                        = 0x4
 	TCP_SIGNATURE_ENABLE              = 0x10
 	TCSAFLUSH                         = 0x2
+	TIMER_ABSTIME                     = 0x1
+	TIMER_RELTIME                     = 0x0
 	TIOCCBRK                          = 0x2000747a
 	TIOCCDTR                          = 0x20007478
 	TIOCCONS                          = 0x80047462
@@ -1272,7 +1366,6 @@
 	TIOCGETD                          = 0x4004741a
 	TIOCGPGRP                         = 0x40047477
 	TIOCGSID                          = 0x40047463
-	TIOCGSIZE                         = 0x40087468
 	TIOCGWINSZ                        = 0x40087468
 	TIOCISPTMASTER                    = 0x20007455
 	TIOCMBIC                          = 0x8004746b
@@ -1317,7 +1410,6 @@
 	TIOCSETD                          = 0x8004741b
 	TIOCSIG                           = 0x2000745f
 	TIOCSPGRP                         = 0x80047476
-	TIOCSSIZE                         = 0x80087467
 	TIOCSTART                         = 0x2000746e
 	TIOCSTAT                          = 0x20007465
 	TIOCSTI                           = 0x80017472
@@ -1326,6 +1418,8 @@
 	TIOCTIMESTAMP                     = 0x40107459
 	TIOCUCNTL                         = 0x80047466
 	TOSTOP                            = 0x400000
+	UTIME_NOW                         = -0x1
+	UTIME_OMIT                        = -0x2
 	VCHECKPT                          = 0x13
 	VDISCARD                          = 0xf
 	VDSUSP                            = 0xb
@@ -1350,9 +1444,12 @@
 	VWERASE                           = 0x4
 	WCONTINUED                        = 0x4
 	WCOREFLAG                         = 0x80
+	WEXITED                           = 0x10
 	WLINUXCLONE                       = 0x80000000
 	WNOHANG                           = 0x1
-	WSTOPPED                          = 0x7f
+	WNOWAIT                           = 0x8
+	WSTOPPED                          = 0x2
+	WTRAPPED                          = 0x20
 	WUNTRACED                         = 0x2
 )
 
@@ -1452,11 +1549,6 @@
 	ETIMEDOUT       = syscall.Errno(0x3c)
 	ETOOMANYREFS    = syscall.Errno(0x3b)
 	ETXTBSY         = syscall.Errno(0x1a)
-	EUNUSED94       = syscall.Errno(0x5e)
-	EUNUSED95       = syscall.Errno(0x5f)
-	EUNUSED96       = syscall.Errno(0x60)
-	EUNUSED97       = syscall.Errno(0x61)
-	EUNUSED98       = syscall.Errno(0x62)
 	EUSERS          = syscall.Errno(0x44)
 	EWOULDBLOCK     = syscall.Errno(0x23)
 	EXDEV           = syscall.Errno(0x12)
@@ -1600,12 +1692,7 @@
 	{91, "ENOLINK", "link has been severed"},
 	{92, "EPROTO", "protocol error"},
 	{93, "ENOMEDIUM", "no medium found"},
-	{94, "EUNUSED94", "unknown error: 94"},
-	{95, "EUNUSED95", "unknown error: 95"},
-	{96, "EUNUSED96", "unknown error: 96"},
-	{97, "EUNUSED97", "unknown error: 97"},
-	{98, "EUNUSED98", "unknown error: 98"},
-	{99, "ELAST", "unknown error: 99"},
+	{99, "EASYNC", "unknown error: 99"},
 }
 
 // Signal table
diff --git a/vendor/golang.org/x/sys/unix/zerrors_freebsd_386.go b/vendor/golang.org/x/sys/unix/zerrors_freebsd_386.go
index 3689c80..9c7c5e1 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_freebsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_freebsd_386.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m32
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && freebsd
 // +build 386,freebsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -997,6 +998,11 @@
 	KERN_OSRELEASE                 = 0x2
 	KERN_OSTYPE                    = 0x1
 	KERN_VERSION                   = 0x4
+	LOCAL_CONNWAIT                 = 0x4
+	LOCAL_CREDS                    = 0x2
+	LOCAL_CREDS_PERSISTENT         = 0x3
+	LOCAL_PEERCRED                 = 0x1
+	LOCAL_VENDOR                   = 0x80000000
 	LOCK_EX                        = 0x2
 	LOCK_NB                        = 0x4
 	LOCK_SH                        = 0x1
@@ -1375,6 +1381,7 @@
 	SOCK_RDM                       = 0x4
 	SOCK_SEQPACKET                 = 0x5
 	SOCK_STREAM                    = 0x1
+	SOL_LOCAL                      = 0x0
 	SOL_SOCKET                     = 0xffff
 	SOMAXCONN                      = 0x80
 	SO_ACCEPTCONN                  = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_freebsd_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_freebsd_amd64.go
index b8f7c3c..b265abb 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_freebsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_freebsd_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && freebsd
 // +build amd64,freebsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -997,6 +998,11 @@
 	KERN_OSRELEASE                 = 0x2
 	KERN_OSTYPE                    = 0x1
 	KERN_VERSION                   = 0x4
+	LOCAL_CONNWAIT                 = 0x4
+	LOCAL_CREDS                    = 0x2
+	LOCAL_CREDS_PERSISTENT         = 0x3
+	LOCAL_PEERCRED                 = 0x1
+	LOCAL_VENDOR                   = 0x80000000
 	LOCK_EX                        = 0x2
 	LOCK_NB                        = 0x4
 	LOCK_SH                        = 0x1
@@ -1376,6 +1382,7 @@
 	SOCK_RDM                       = 0x4
 	SOCK_SEQPACKET                 = 0x5
 	SOCK_STREAM                    = 0x1
+	SOL_LOCAL                      = 0x0
 	SOL_SOCKET                     = 0xffff
 	SOMAXCONN                      = 0x80
 	SO_ACCEPTCONN                  = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm.go b/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm.go
index be14bb1..3df99f2 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm.go
@@ -1,6 +1,7 @@
 // mkerrors.sh
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && freebsd
 // +build arm,freebsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -980,6 +981,11 @@
 	KERN_OSRELEASE                 = 0x2
 	KERN_OSTYPE                    = 0x1
 	KERN_VERSION                   = 0x4
+	LOCAL_CONNWAIT                 = 0x4
+	LOCAL_CREDS                    = 0x2
+	LOCAL_CREDS_PERSISTENT         = 0x3
+	LOCAL_PEERCRED                 = 0x1
+	LOCAL_VENDOR                   = 0x80000000
 	LOCK_EX                        = 0x2
 	LOCK_NB                        = 0x4
 	LOCK_SH                        = 0x1
@@ -1016,6 +1022,15 @@
 	MAP_RESERVED0100               = 0x100
 	MAP_SHARED                     = 0x1
 	MAP_STACK                      = 0x400
+	MCAST_BLOCK_SOURCE             = 0x54
+	MCAST_EXCLUDE                  = 0x2
+	MCAST_INCLUDE                  = 0x1
+	MCAST_JOIN_GROUP               = 0x50
+	MCAST_JOIN_SOURCE_GROUP        = 0x52
+	MCAST_LEAVE_GROUP              = 0x51
+	MCAST_LEAVE_SOURCE_GROUP       = 0x53
+	MCAST_UNBLOCK_SOURCE           = 0x55
+	MCAST_UNDEFINED                = 0x0
 	MCL_CURRENT                    = 0x1
 	MCL_FUTURE                     = 0x2
 	MNT_ACLS                       = 0x8000000
@@ -1341,6 +1356,7 @@
 	SOCK_RDM                       = 0x4
 	SOCK_SEQPACKET                 = 0x5
 	SOCK_STREAM                    = 0x1
+	SOL_LOCAL                      = 0x0
 	SOL_SOCKET                     = 0xffff
 	SOMAXCONN                      = 0x80
 	SO_ACCEPTCONN                  = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm64.go
index 7ce9c00..218d399 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_freebsd_arm64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && freebsd
 // +build arm64,freebsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -997,6 +998,11 @@
 	KERN_OSRELEASE                 = 0x2
 	KERN_OSTYPE                    = 0x1
 	KERN_VERSION                   = 0x4
+	LOCAL_CONNWAIT                 = 0x4
+	LOCAL_CREDS                    = 0x2
+	LOCAL_CREDS_PERSISTENT         = 0x3
+	LOCAL_PEERCRED                 = 0x1
+	LOCAL_VENDOR                   = 0x80000000
 	LOCK_EX                        = 0x2
 	LOCK_NB                        = 0x4
 	LOCK_SH                        = 0x1
@@ -1376,6 +1382,7 @@
 	SOCK_RDM                       = 0x4
 	SOCK_SEQPACKET                 = 0x5
 	SOCK_STREAM                    = 0x1
+	SOL_LOCAL                      = 0x0
 	SOL_SOCKET                     = 0xffff
 	SOMAXCONN                      = 0x80
 	SO_ACCEPTCONN                  = 0x2
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux.go b/vendor/golang.org/x/sys/unix/zerrors_linux.go
index 79e032f..c3fa224 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux.go
@@ -1,5 +1,6 @@
 // Code generated by mkmerge.go; DO NOT EDIT.
 
+//go:build linux
 // +build linux
 
 package unix
@@ -65,6 +66,7 @@
 	ALG_OP_ENCRYPT                              = 0x1
 	ALG_SET_AEAD_ASSOCLEN                       = 0x4
 	ALG_SET_AEAD_AUTHSIZE                       = 0x5
+	ALG_SET_DRBG_ENTROPY                        = 0x6
 	ALG_SET_IV                                  = 0x2
 	ALG_SET_KEY                                 = 0x1
 	ALG_SET_OP                                  = 0x3
@@ -164,13 +166,16 @@
 	BPF_ALU64                                   = 0x7
 	BPF_AND                                     = 0x50
 	BPF_ARSH                                    = 0xc0
+	BPF_ATOMIC                                  = 0xc0
 	BPF_B                                       = 0x10
 	BPF_BUILD_ID_SIZE                           = 0x14
 	BPF_CALL                                    = 0x80
+	BPF_CMPXCHG                                 = 0xf1
 	BPF_DIV                                     = 0x30
 	BPF_DW                                      = 0x18
 	BPF_END                                     = 0xd0
 	BPF_EXIT                                    = 0x90
+	BPF_FETCH                                   = 0x1
 	BPF_FROM_BE                                 = 0x8
 	BPF_FROM_LE                                 = 0x0
 	BPF_FS_MAGIC                                = 0xcafe4a11
@@ -179,8 +184,10 @@
 	BPF_F_ANY_ALIGNMENT                         = 0x2
 	BPF_F_QUERY_EFFECTIVE                       = 0x1
 	BPF_F_REPLACE                               = 0x4
+	BPF_F_SLEEPABLE                             = 0x10
 	BPF_F_STRICT_ALIGNMENT                      = 0x1
 	BPF_F_TEST_RND_HI32                         = 0x4
+	BPF_F_TEST_RUN_ON_CPU                       = 0x1
 	BPF_F_TEST_STATE_FREQ                       = 0x8
 	BPF_H                                       = 0x8
 	BPF_IMM                                     = 0x0
@@ -219,6 +226,7 @@
 	BPF_NET_OFF                                 = -0x100000
 	BPF_OBJ_NAME_LEN                            = 0x10
 	BPF_OR                                      = 0x40
+	BPF_PSEUDO_BTF_ID                           = 0x3
 	BPF_PSEUDO_CALL                             = 0x1
 	BPF_PSEUDO_MAP_FD                           = 0x1
 	BPF_PSEUDO_MAP_VALUE                        = 0x2
@@ -235,11 +243,16 @@
 	BPF_W                                       = 0x0
 	BPF_X                                       = 0x8
 	BPF_XADD                                    = 0xc0
+	BPF_XCHG                                    = 0xe1
 	BPF_XOR                                     = 0xa0
 	BRKINT                                      = 0x2
 	BS0                                         = 0x0
 	BTRFS_SUPER_MAGIC                           = 0x9123683e
 	BTRFS_TEST_MAGIC                            = 0x73727279
+	BUS_BLUETOOTH                               = 0x5
+	BUS_HIL                                     = 0x4
+	BUS_USB                                     = 0x3
+	BUS_VIRTUAL                                 = 0x6
 	CAN_BCM                                     = 0x2
 	CAN_EFF_FLAG                                = 0x80000000
 	CAN_EFF_ID_BITS                             = 0x1d
@@ -309,6 +322,7 @@
 	CAN_J1939                                   = 0x7
 	CAN_MAX_DLC                                 = 0x8
 	CAN_MAX_DLEN                                = 0x8
+	CAN_MAX_RAW_DLC                             = 0xf
 	CAN_MCNET                                   = 0x5
 	CAN_MTU                                     = 0x10
 	CAN_NPROTO                                  = 0x8
@@ -324,6 +338,7 @@
 	CAP_AUDIT_WRITE                             = 0x1d
 	CAP_BLOCK_SUSPEND                           = 0x24
 	CAP_BPF                                     = 0x27
+	CAP_CHECKPOINT_RESTORE                      = 0x28
 	CAP_CHOWN                                   = 0x0
 	CAP_DAC_OVERRIDE                            = 0x1
 	CAP_DAC_READ_SEARCH                         = 0x2
@@ -332,7 +347,7 @@
 	CAP_IPC_LOCK                                = 0xe
 	CAP_IPC_OWNER                               = 0xf
 	CAP_KILL                                    = 0x5
-	CAP_LAST_CAP                                = 0x27
+	CAP_LAST_CAP                                = 0x28
 	CAP_LEASE                                   = 0x1c
 	CAP_LINUX_IMMUTABLE                         = 0x9
 	CAP_MAC_ADMIN                               = 0x21
@@ -428,10 +443,13 @@
 	DEBUGFS_MAGIC                               = 0x64626720
 	DEVLINK_CMD_ESWITCH_MODE_GET                = 0x1d
 	DEVLINK_CMD_ESWITCH_MODE_SET                = 0x1e
+	DEVLINK_FLASH_OVERWRITE_IDENTIFIERS         = 0x2
+	DEVLINK_FLASH_OVERWRITE_SETTINGS            = 0x1
 	DEVLINK_GENL_MCGRP_CONFIG_NAME              = "config"
 	DEVLINK_GENL_NAME                           = "devlink"
 	DEVLINK_GENL_VERSION                        = 0x1
 	DEVLINK_SB_THRESHOLD_TO_ALPHA_MAX           = 0x14
+	DEVLINK_SUPPORTED_FLASH_OVERWRITE_SECTIONS  = 0x3
 	DEVMEM_MAGIC                                = 0x454d444d
 	DEVPTS_SUPER_MAGIC                          = 0x1cd1
 	DMA_BUF_MAGIC                               = 0x444d4142
@@ -476,9 +494,9 @@
 	DM_UUID_FLAG                                = 0x4000
 	DM_UUID_LEN                                 = 0x81
 	DM_VERSION                                  = 0xc138fd00
-	DM_VERSION_EXTRA                            = "-ioctl (2020-02-27)"
+	DM_VERSION_EXTRA                            = "-ioctl (2021-02-01)"
 	DM_VERSION_MAJOR                            = 0x4
-	DM_VERSION_MINOR                            = 0x2a
+	DM_VERSION_MINOR                            = 0x2c
 	DM_VERSION_PATCHLEVEL                       = 0x0
 	DT_BLK                                      = 0x6
 	DT_CHR                                      = 0x2
@@ -519,6 +537,119 @@
 	EPOLL_CTL_DEL                               = 0x2
 	EPOLL_CTL_MOD                               = 0x3
 	EROFS_SUPER_MAGIC_V1                        = 0xe0f5e1e2
+	ESP_V4_FLOW                                 = 0xa
+	ESP_V6_FLOW                                 = 0xc
+	ETHER_FLOW                                  = 0x12
+	ETHTOOL_BUSINFO_LEN                         = 0x20
+	ETHTOOL_EROMVERS_LEN                        = 0x20
+	ETHTOOL_FEC_AUTO                            = 0x2
+	ETHTOOL_FEC_BASER                           = 0x10
+	ETHTOOL_FEC_LLRS                            = 0x20
+	ETHTOOL_FEC_NONE                            = 0x1
+	ETHTOOL_FEC_OFF                             = 0x4
+	ETHTOOL_FEC_RS                              = 0x8
+	ETHTOOL_FLAG_ALL                            = 0x7
+	ETHTOOL_FLAG_COMPACT_BITSETS                = 0x1
+	ETHTOOL_FLAG_OMIT_REPLY                     = 0x2
+	ETHTOOL_FLAG_STATS                          = 0x4
+	ETHTOOL_FLASHDEV                            = 0x33
+	ETHTOOL_FLASH_MAX_FILENAME                  = 0x80
+	ETHTOOL_FWVERS_LEN                          = 0x20
+	ETHTOOL_F_COMPAT                            = 0x4
+	ETHTOOL_F_UNSUPPORTED                       = 0x1
+	ETHTOOL_F_WISH                              = 0x2
+	ETHTOOL_GCHANNELS                           = 0x3c
+	ETHTOOL_GCOALESCE                           = 0xe
+	ETHTOOL_GDRVINFO                            = 0x3
+	ETHTOOL_GEEE                                = 0x44
+	ETHTOOL_GEEPROM                             = 0xb
+	ETHTOOL_GENL_NAME                           = "ethtool"
+	ETHTOOL_GENL_VERSION                        = 0x1
+	ETHTOOL_GET_DUMP_DATA                       = 0x40
+	ETHTOOL_GET_DUMP_FLAG                       = 0x3f
+	ETHTOOL_GET_TS_INFO                         = 0x41
+	ETHTOOL_GFEATURES                           = 0x3a
+	ETHTOOL_GFECPARAM                           = 0x50
+	ETHTOOL_GFLAGS                              = 0x25
+	ETHTOOL_GGRO                                = 0x2b
+	ETHTOOL_GGSO                                = 0x23
+	ETHTOOL_GLINK                               = 0xa
+	ETHTOOL_GLINKSETTINGS                       = 0x4c
+	ETHTOOL_GMODULEEEPROM                       = 0x43
+	ETHTOOL_GMODULEINFO                         = 0x42
+	ETHTOOL_GMSGLVL                             = 0x7
+	ETHTOOL_GPAUSEPARAM                         = 0x12
+	ETHTOOL_GPERMADDR                           = 0x20
+	ETHTOOL_GPFLAGS                             = 0x27
+	ETHTOOL_GPHYSTATS                           = 0x4a
+	ETHTOOL_GREGS                               = 0x4
+	ETHTOOL_GRINGPARAM                          = 0x10
+	ETHTOOL_GRSSH                               = 0x46
+	ETHTOOL_GRXCLSRLALL                         = 0x30
+	ETHTOOL_GRXCLSRLCNT                         = 0x2e
+	ETHTOOL_GRXCLSRULE                          = 0x2f
+	ETHTOOL_GRXCSUM                             = 0x14
+	ETHTOOL_GRXFH                               = 0x29
+	ETHTOOL_GRXFHINDIR                          = 0x38
+	ETHTOOL_GRXNTUPLE                           = 0x36
+	ETHTOOL_GRXRINGS                            = 0x2d
+	ETHTOOL_GSET                                = 0x1
+	ETHTOOL_GSG                                 = 0x18
+	ETHTOOL_GSSET_INFO                          = 0x37
+	ETHTOOL_GSTATS                              = 0x1d
+	ETHTOOL_GSTRINGS                            = 0x1b
+	ETHTOOL_GTSO                                = 0x1e
+	ETHTOOL_GTUNABLE                            = 0x48
+	ETHTOOL_GTXCSUM                             = 0x16
+	ETHTOOL_GUFO                                = 0x21
+	ETHTOOL_GWOL                                = 0x5
+	ETHTOOL_MCGRP_MONITOR_NAME                  = "monitor"
+	ETHTOOL_NWAY_RST                            = 0x9
+	ETHTOOL_PERQUEUE                            = 0x4b
+	ETHTOOL_PHYS_ID                             = 0x1c
+	ETHTOOL_PHY_EDPD_DFLT_TX_MSECS              = 0xffff
+	ETHTOOL_PHY_EDPD_DISABLE                    = 0x0
+	ETHTOOL_PHY_EDPD_NO_TX                      = 0xfffe
+	ETHTOOL_PHY_FAST_LINK_DOWN_OFF              = 0xff
+	ETHTOOL_PHY_FAST_LINK_DOWN_ON               = 0x0
+	ETHTOOL_PHY_GTUNABLE                        = 0x4e
+	ETHTOOL_PHY_STUNABLE                        = 0x4f
+	ETHTOOL_RESET                               = 0x34
+	ETHTOOL_RXNTUPLE_ACTION_CLEAR               = -0x2
+	ETHTOOL_RXNTUPLE_ACTION_DROP                = -0x1
+	ETHTOOL_RX_FLOW_SPEC_RING                   = 0xffffffff
+	ETHTOOL_RX_FLOW_SPEC_RING_VF                = 0xff00000000
+	ETHTOOL_RX_FLOW_SPEC_RING_VF_OFF            = 0x20
+	ETHTOOL_SCHANNELS                           = 0x3d
+	ETHTOOL_SCOALESCE                           = 0xf
+	ETHTOOL_SEEE                                = 0x45
+	ETHTOOL_SEEPROM                             = 0xc
+	ETHTOOL_SET_DUMP                            = 0x3e
+	ETHTOOL_SFEATURES                           = 0x3b
+	ETHTOOL_SFECPARAM                           = 0x51
+	ETHTOOL_SFLAGS                              = 0x26
+	ETHTOOL_SGRO                                = 0x2c
+	ETHTOOL_SGSO                                = 0x24
+	ETHTOOL_SLINKSETTINGS                       = 0x4d
+	ETHTOOL_SMSGLVL                             = 0x8
+	ETHTOOL_SPAUSEPARAM                         = 0x13
+	ETHTOOL_SPFLAGS                             = 0x28
+	ETHTOOL_SRINGPARAM                          = 0x11
+	ETHTOOL_SRSSH                               = 0x47
+	ETHTOOL_SRXCLSRLDEL                         = 0x31
+	ETHTOOL_SRXCLSRLINS                         = 0x32
+	ETHTOOL_SRXCSUM                             = 0x15
+	ETHTOOL_SRXFH                               = 0x2a
+	ETHTOOL_SRXFHINDIR                          = 0x39
+	ETHTOOL_SRXNTUPLE                           = 0x35
+	ETHTOOL_SSET                                = 0x2
+	ETHTOOL_SSG                                 = 0x19
+	ETHTOOL_STSO                                = 0x1f
+	ETHTOOL_STUNABLE                            = 0x49
+	ETHTOOL_STXCSUM                             = 0x17
+	ETHTOOL_SUFO                                = 0x22
+	ETHTOOL_SWOL                                = 0x6
+	ETHTOOL_TEST                                = 0x1a
 	ETH_P_1588                                  = 0x88f7
 	ETH_P_8021AD                                = 0x88a8
 	ETH_P_8021AH                                = 0x88e7
@@ -543,6 +674,7 @@
 	ETH_P_CAIF                                  = 0xf7
 	ETH_P_CAN                                   = 0xc
 	ETH_P_CANFD                                 = 0xd
+	ETH_P_CFM                                   = 0x8902
 	ETH_P_CONTROL                               = 0x16
 	ETH_P_CUST                                  = 0x6006
 	ETH_P_DDCMP                                 = 0x6
@@ -650,8 +782,8 @@
 	FAN_DELETE                                  = 0x200
 	FAN_DELETE_SELF                             = 0x400
 	FAN_DENY                                    = 0x2
-	FAN_DIR_MODIFY                              = 0x80000
 	FAN_ENABLE_AUDIT                            = 0x40
+	FAN_EVENT_INFO_TYPE_DFID                    = 0x3
 	FAN_EVENT_INFO_TYPE_DFID_NAME               = 0x2
 	FAN_EVENT_INFO_TYPE_FID                     = 0x1
 	FAN_EVENT_METADATA_LEN                      = 0x18
@@ -679,7 +811,10 @@
 	FAN_OPEN_EXEC_PERM                          = 0x40000
 	FAN_OPEN_PERM                               = 0x10000
 	FAN_Q_OVERFLOW                              = 0x4000
+	FAN_REPORT_DFID_NAME                        = 0xc00
+	FAN_REPORT_DIR_FID                          = 0x400
 	FAN_REPORT_FID                              = 0x200
+	FAN_REPORT_NAME                             = 0x800
 	FAN_REPORT_TID                              = 0x100
 	FAN_UNLIMITED_MARKS                         = 0x20
 	FAN_UNLIMITED_QUEUE                         = 0x10
@@ -710,7 +845,6 @@
 	FSCRYPT_POLICY_FLAGS_PAD_4                  = 0x0
 	FSCRYPT_POLICY_FLAGS_PAD_8                  = 0x1
 	FSCRYPT_POLICY_FLAGS_PAD_MASK               = 0x3
-	FSCRYPT_POLICY_FLAGS_VALID                  = 0x1f
 	FSCRYPT_POLICY_FLAG_DIRECT_KEY              = 0x4
 	FSCRYPT_POLICY_FLAG_IV_INO_LBLK_32          = 0x10
 	FSCRYPT_POLICY_FLAG_IV_INO_LBLK_64          = 0x8
@@ -730,6 +864,7 @@
 	FS_IOC_GET_ENCRYPTION_KEY_STATUS            = 0xc080661a
 	FS_IOC_GET_ENCRYPTION_POLICY_EX             = 0xc0096616
 	FS_IOC_MEASURE_VERITY                       = 0xc0046686
+	FS_IOC_READ_VERITY_METADATA                 = 0xc0286687
 	FS_IOC_REMOVE_ENCRYPTION_KEY                = 0xc0406618
 	FS_IOC_REMOVE_ENCRYPTION_KEY_ALL_USERS      = 0xc0406619
 	FS_KEY_DESCRIPTOR_SIZE                      = 0x8
@@ -741,10 +876,13 @@
 	FS_POLICY_FLAGS_PAD_4                       = 0x0
 	FS_POLICY_FLAGS_PAD_8                       = 0x1
 	FS_POLICY_FLAGS_PAD_MASK                    = 0x3
-	FS_POLICY_FLAGS_VALID                       = 0x1f
+	FS_POLICY_FLAGS_VALID                       = 0x7
 	FS_VERITY_FL                                = 0x100000
 	FS_VERITY_HASH_ALG_SHA256                   = 0x1
 	FS_VERITY_HASH_ALG_SHA512                   = 0x2
+	FS_VERITY_METADATA_TYPE_DESCRIPTOR          = 0x2
+	FS_VERITY_METADATA_TYPE_MERKLE_TREE         = 0x1
+	FS_VERITY_METADATA_TYPE_SIGNATURE           = 0x3
 	FUTEXFS_SUPER_MAGIC                         = 0xbad1dea
 	F_ADD_SEALS                                 = 0x409
 	F_DUPFD                                     = 0x0
@@ -838,11 +976,17 @@
 	HDIO_SET_XFER                               = 0x306
 	HDIO_TRISTATE_HWIF                          = 0x31b
 	HDIO_UNREGISTER_HWIF                        = 0x32a
+	HID_MAX_DESCRIPTOR_SIZE                     = 0x1000
 	HOSTFS_SUPER_MAGIC                          = 0xc0ffee
 	HPFS_SUPER_MAGIC                            = 0xf995e849
 	HUGETLBFS_MAGIC                             = 0x958458f6
 	IBSHIFT                                     = 0x10
 	ICMPV6_FILTER                               = 0x1
+	ICMPV6_FILTER_BLOCK                         = 0x1
+	ICMPV6_FILTER_BLOCKOTHERS                   = 0x3
+	ICMPV6_FILTER_PASS                          = 0x2
+	ICMPV6_FILTER_PASSONLY                      = 0x4
+	ICMP_FILTER                                 = 0x1
 	ICRNL                                       = 0x100
 	IFA_F_DADFAILED                             = 0x8
 	IFA_F_DEPRECATED                            = 0x20
@@ -985,6 +1129,7 @@
 	IPV6_DONTFRAG                               = 0x3e
 	IPV6_DROP_MEMBERSHIP                        = 0x15
 	IPV6_DSTOPTS                                = 0x3b
+	IPV6_FLOW                                   = 0x11
 	IPV6_FREEBIND                               = 0x4e
 	IPV6_HDRINCL                                = 0x24
 	IPV6_HOPLIMIT                               = 0x34
@@ -1013,6 +1158,7 @@
 	IPV6_PMTUDISC_WANT                          = 0x1
 	IPV6_RECVDSTOPTS                            = 0x3a
 	IPV6_RECVERR                                = 0x19
+	IPV6_RECVERR_RFC4884                        = 0x1f
 	IPV6_RECVFRAGSIZE                           = 0x4d
 	IPV6_RECVHOPLIMIT                           = 0x33
 	IPV6_RECVHOPOPTS                            = 0x35
@@ -1034,6 +1180,7 @@
 	IPV6_TRANSPARENT                            = 0x4b
 	IPV6_UNICAST_HOPS                           = 0x10
 	IPV6_UNICAST_IF                             = 0x4c
+	IPV6_USER_FLOW                              = 0xe
 	IPV6_V6ONLY                                 = 0x1a
 	IPV6_XFRM_POLICY                            = 0x23
 	IP_ADD_MEMBERSHIP                           = 0x23
@@ -1076,6 +1223,7 @@
 	IP_PMTUDISC_PROBE                           = 0x3
 	IP_PMTUDISC_WANT                            = 0x1
 	IP_RECVERR                                  = 0xb
+	IP_RECVERR_RFC4884                          = 0x1a
 	IP_RECVFRAGSIZE                             = 0x19
 	IP_RECVOPTS                                 = 0x6
 	IP_RECVORIGDSTADDR                          = 0x14
@@ -1090,6 +1238,7 @@
 	IP_TTL                                      = 0x2
 	IP_UNBLOCK_SOURCE                           = 0x25
 	IP_UNICAST_IF                               = 0x32
+	IP_USER_FLOW                                = 0xd
 	IP_XFRM_POLICY                              = 0x11
 	ISOFS_SUPER_MAGIC                           = 0x9660
 	ISTRIP                                      = 0x20
@@ -1213,6 +1362,12 @@
 	LOOP_SET_STATUS_SETTABLE_FLAGS              = 0xc
 	LO_KEY_SIZE                                 = 0x20
 	LO_NAME_SIZE                                = 0x40
+	LWTUNNEL_IP6_MAX                            = 0x8
+	LWTUNNEL_IP_MAX                             = 0x8
+	LWTUNNEL_IP_OPTS_MAX                        = 0x3
+	LWTUNNEL_IP_OPT_ERSPAN_MAX                  = 0x4
+	LWTUNNEL_IP_OPT_GENEVE_MAX                  = 0x3
+	LWTUNNEL_IP_OPT_VXLAN_MAX                   = 0x1
 	MADV_COLD                                   = 0x14
 	MADV_DODUMP                                 = 0x11
 	MADV_DOFORK                                 = 0xb
@@ -1251,6 +1406,10 @@
 	MCAST_LEAVE_SOURCE_GROUP                    = 0x2f
 	MCAST_MSFILTER                              = 0x30
 	MCAST_UNBLOCK_SOURCE                        = 0x2c
+	MEMGETREGIONINFO                            = 0xc0104d08
+	MEMREADOOB64                                = 0xc0184d16
+	MEMWRITE                                    = 0xc0304d18
+	MEMWRITEOOB64                               = 0xc0184d15
 	MFD_ALLOW_SEALING                           = 0x2
 	MFD_CLOEXEC                                 = 0x1
 	MFD_HUGETLB                                 = 0x4
@@ -1321,6 +1480,7 @@
 	MS_NOREMOTELOCK                             = 0x8000000
 	MS_NOSEC                                    = 0x10000000
 	MS_NOSUID                                   = 0x2
+	MS_NOSYMFOLLOW                              = 0x100
 	MS_NOUSER                                   = -0x80000000
 	MS_POSIXACL                                 = 0x10000
 	MS_PRIVATE                                  = 0x40000
@@ -1338,7 +1498,35 @@
 	MS_SYNCHRONOUS                              = 0x10
 	MS_UNBINDABLE                               = 0x20000
 	MS_VERBOSE                                  = 0x8000
+	MTD_ABSENT                                  = 0x0
+	MTD_BIT_WRITEABLE                           = 0x800
+	MTD_CAP_NANDFLASH                           = 0x400
+	MTD_CAP_NORFLASH                            = 0xc00
+	MTD_CAP_NVRAM                               = 0x1c00
+	MTD_CAP_RAM                                 = 0x1c00
+	MTD_CAP_ROM                                 = 0x0
+	MTD_DATAFLASH                               = 0x6
 	MTD_INODE_FS_MAGIC                          = 0x11307854
+	MTD_MAX_ECCPOS_ENTRIES                      = 0x40
+	MTD_MAX_OOBFREE_ENTRIES                     = 0x8
+	MTD_MLCNANDFLASH                            = 0x8
+	MTD_NANDECC_AUTOPLACE                       = 0x2
+	MTD_NANDECC_AUTOPL_USR                      = 0x4
+	MTD_NANDECC_OFF                             = 0x0
+	MTD_NANDECC_PLACE                           = 0x1
+	MTD_NANDECC_PLACEONLY                       = 0x3
+	MTD_NANDFLASH                               = 0x4
+	MTD_NORFLASH                                = 0x3
+	MTD_NO_ERASE                                = 0x1000
+	MTD_OTP_FACTORY                             = 0x1
+	MTD_OTP_OFF                                 = 0x0
+	MTD_OTP_USER                                = 0x2
+	MTD_POWERUP_LOCK                            = 0x2000
+	MTD_RAM                                     = 0x1
+	MTD_ROM                                     = 0x2
+	MTD_SLC_ON_MLC_EMULATION                    = 0x4000
+	MTD_UBIVOLUME                               = 0x7
+	MTD_WRITEABLE                               = 0x400
 	NAME_MAX                                    = 0xff
 	NCP_SUPER_MAGIC                             = 0x564c
 	NETLINK_ADD_MEMBERSHIP                      = 0x1
@@ -1378,6 +1566,59 @@
 	NETLINK_XFRM                                = 0x6
 	NETNSA_MAX                                  = 0x5
 	NETNSA_NSID_NOT_ASSIGNED                    = -0x1
+	NFC_ATR_REQ_GB_MAXSIZE                      = 0x30
+	NFC_ATR_REQ_MAXSIZE                         = 0x40
+	NFC_ATR_RES_GB_MAXSIZE                      = 0x2f
+	NFC_ATR_RES_MAXSIZE                         = 0x40
+	NFC_COMM_ACTIVE                             = 0x0
+	NFC_COMM_PASSIVE                            = 0x1
+	NFC_DEVICE_NAME_MAXSIZE                     = 0x8
+	NFC_DIRECTION_RX                            = 0x0
+	NFC_DIRECTION_TX                            = 0x1
+	NFC_FIRMWARE_NAME_MAXSIZE                   = 0x20
+	NFC_GB_MAXSIZE                              = 0x30
+	NFC_GENL_MCAST_EVENT_NAME                   = "events"
+	NFC_GENL_NAME                               = "nfc"
+	NFC_GENL_VERSION                            = 0x1
+	NFC_HEADER_SIZE                             = 0x1
+	NFC_ISO15693_UID_MAXSIZE                    = 0x8
+	NFC_LLCP_MAX_SERVICE_NAME                   = 0x3f
+	NFC_LLCP_MIUX                               = 0x1
+	NFC_LLCP_REMOTE_LTO                         = 0x3
+	NFC_LLCP_REMOTE_MIU                         = 0x2
+	NFC_LLCP_REMOTE_RW                          = 0x4
+	NFC_LLCP_RW                                 = 0x0
+	NFC_NFCID1_MAXSIZE                          = 0xa
+	NFC_NFCID2_MAXSIZE                          = 0x8
+	NFC_NFCID3_MAXSIZE                          = 0xa
+	NFC_PROTO_FELICA                            = 0x3
+	NFC_PROTO_FELICA_MASK                       = 0x8
+	NFC_PROTO_ISO14443                          = 0x4
+	NFC_PROTO_ISO14443_B                        = 0x6
+	NFC_PROTO_ISO14443_B_MASK                   = 0x40
+	NFC_PROTO_ISO14443_MASK                     = 0x10
+	NFC_PROTO_ISO15693                          = 0x7
+	NFC_PROTO_ISO15693_MASK                     = 0x80
+	NFC_PROTO_JEWEL                             = 0x1
+	NFC_PROTO_JEWEL_MASK                        = 0x2
+	NFC_PROTO_MAX                               = 0x8
+	NFC_PROTO_MIFARE                            = 0x2
+	NFC_PROTO_MIFARE_MASK                       = 0x4
+	NFC_PROTO_NFC_DEP                           = 0x5
+	NFC_PROTO_NFC_DEP_MASK                      = 0x20
+	NFC_RAW_HEADER_SIZE                         = 0x2
+	NFC_RF_INITIATOR                            = 0x0
+	NFC_RF_NONE                                 = 0x2
+	NFC_RF_TARGET                               = 0x1
+	NFC_SENSB_RES_MAXSIZE                       = 0xc
+	NFC_SENSF_RES_MAXSIZE                       = 0x12
+	NFC_SE_DISABLED                             = 0x0
+	NFC_SE_EMBEDDED                             = 0x2
+	NFC_SE_ENABLED                              = 0x1
+	NFC_SE_UICC                                 = 0x1
+	NFC_SOCKPROTO_LLCP                          = 0x1
+	NFC_SOCKPROTO_MAX                           = 0x2
+	NFC_SOCKPROTO_RAW                           = 0x0
 	NFNETLINK_V0                                = 0x0
 	NFNLGRP_ACCT_QUOTA                          = 0x8
 	NFNLGRP_CONNTRACK_DESTROY                   = 0x3
@@ -1508,6 +1749,98 @@
 	PARITY_DEFAULT                              = 0x0
 	PARITY_NONE                                 = 0x1
 	PARMRK                                      = 0x8
+	PERF_ATTR_SIZE_VER0                         = 0x40
+	PERF_ATTR_SIZE_VER1                         = 0x48
+	PERF_ATTR_SIZE_VER2                         = 0x50
+	PERF_ATTR_SIZE_VER3                         = 0x60
+	PERF_ATTR_SIZE_VER4                         = 0x68
+	PERF_ATTR_SIZE_VER5                         = 0x70
+	PERF_ATTR_SIZE_VER6                         = 0x78
+	PERF_AUX_FLAG_COLLISION                     = 0x8
+	PERF_AUX_FLAG_OVERWRITE                     = 0x2
+	PERF_AUX_FLAG_PARTIAL                       = 0x4
+	PERF_AUX_FLAG_TRUNCATED                     = 0x1
+	PERF_FLAG_FD_CLOEXEC                        = 0x8
+	PERF_FLAG_FD_NO_GROUP                       = 0x1
+	PERF_FLAG_FD_OUTPUT                         = 0x2
+	PERF_FLAG_PID_CGROUP                        = 0x4
+	PERF_MAX_CONTEXTS_PER_STACK                 = 0x8
+	PERF_MAX_STACK_DEPTH                        = 0x7f
+	PERF_MEM_BLK_ADDR                           = 0x4
+	PERF_MEM_BLK_DATA                           = 0x2
+	PERF_MEM_BLK_NA                             = 0x1
+	PERF_MEM_BLK_SHIFT                          = 0x28
+	PERF_MEM_LOCK_LOCKED                        = 0x2
+	PERF_MEM_LOCK_NA                            = 0x1
+	PERF_MEM_LOCK_SHIFT                         = 0x18
+	PERF_MEM_LVLNUM_ANY_CACHE                   = 0xb
+	PERF_MEM_LVLNUM_L1                          = 0x1
+	PERF_MEM_LVLNUM_L2                          = 0x2
+	PERF_MEM_LVLNUM_L3                          = 0x3
+	PERF_MEM_LVLNUM_L4                          = 0x4
+	PERF_MEM_LVLNUM_LFB                         = 0xc
+	PERF_MEM_LVLNUM_NA                          = 0xf
+	PERF_MEM_LVLNUM_PMEM                        = 0xe
+	PERF_MEM_LVLNUM_RAM                         = 0xd
+	PERF_MEM_LVLNUM_SHIFT                       = 0x21
+	PERF_MEM_LVL_HIT                            = 0x2
+	PERF_MEM_LVL_IO                             = 0x1000
+	PERF_MEM_LVL_L1                             = 0x8
+	PERF_MEM_LVL_L2                             = 0x20
+	PERF_MEM_LVL_L3                             = 0x40
+	PERF_MEM_LVL_LFB                            = 0x10
+	PERF_MEM_LVL_LOC_RAM                        = 0x80
+	PERF_MEM_LVL_MISS                           = 0x4
+	PERF_MEM_LVL_NA                             = 0x1
+	PERF_MEM_LVL_REM_CCE1                       = 0x400
+	PERF_MEM_LVL_REM_CCE2                       = 0x800
+	PERF_MEM_LVL_REM_RAM1                       = 0x100
+	PERF_MEM_LVL_REM_RAM2                       = 0x200
+	PERF_MEM_LVL_SHIFT                          = 0x5
+	PERF_MEM_LVL_UNC                            = 0x2000
+	PERF_MEM_OP_EXEC                            = 0x10
+	PERF_MEM_OP_LOAD                            = 0x2
+	PERF_MEM_OP_NA                              = 0x1
+	PERF_MEM_OP_PFETCH                          = 0x8
+	PERF_MEM_OP_SHIFT                           = 0x0
+	PERF_MEM_OP_STORE                           = 0x4
+	PERF_MEM_REMOTE_REMOTE                      = 0x1
+	PERF_MEM_REMOTE_SHIFT                       = 0x25
+	PERF_MEM_SNOOPX_FWD                         = 0x1
+	PERF_MEM_SNOOPX_SHIFT                       = 0x26
+	PERF_MEM_SNOOP_HIT                          = 0x4
+	PERF_MEM_SNOOP_HITM                         = 0x10
+	PERF_MEM_SNOOP_MISS                         = 0x8
+	PERF_MEM_SNOOP_NA                           = 0x1
+	PERF_MEM_SNOOP_NONE                         = 0x2
+	PERF_MEM_SNOOP_SHIFT                        = 0x13
+	PERF_MEM_TLB_HIT                            = 0x2
+	PERF_MEM_TLB_L1                             = 0x8
+	PERF_MEM_TLB_L2                             = 0x10
+	PERF_MEM_TLB_MISS                           = 0x4
+	PERF_MEM_TLB_NA                             = 0x1
+	PERF_MEM_TLB_OS                             = 0x40
+	PERF_MEM_TLB_SHIFT                          = 0x1a
+	PERF_MEM_TLB_WK                             = 0x20
+	PERF_RECORD_KSYMBOL_FLAGS_UNREGISTER        = 0x1
+	PERF_RECORD_MISC_COMM_EXEC                  = 0x2000
+	PERF_RECORD_MISC_CPUMODE_MASK               = 0x7
+	PERF_RECORD_MISC_CPUMODE_UNKNOWN            = 0x0
+	PERF_RECORD_MISC_EXACT_IP                   = 0x4000
+	PERF_RECORD_MISC_EXT_RESERVED               = 0x8000
+	PERF_RECORD_MISC_FORK_EXEC                  = 0x2000
+	PERF_RECORD_MISC_GUEST_KERNEL               = 0x4
+	PERF_RECORD_MISC_GUEST_USER                 = 0x5
+	PERF_RECORD_MISC_HYPERVISOR                 = 0x3
+	PERF_RECORD_MISC_KERNEL                     = 0x1
+	PERF_RECORD_MISC_MMAP_BUILD_ID              = 0x4000
+	PERF_RECORD_MISC_MMAP_DATA                  = 0x2000
+	PERF_RECORD_MISC_PROC_MAP_PARSE_TIMEOUT     = 0x1000
+	PERF_RECORD_MISC_SWITCH_OUT                 = 0x2000
+	PERF_RECORD_MISC_SWITCH_OUT_PREEMPT         = 0x4000
+	PERF_RECORD_MISC_USER                       = 0x2
+	PERF_SAMPLE_BRANCH_PLM_ALL                  = 0x7
+	PERF_SAMPLE_WEIGHT_TYPE                     = 0x1004000
 	PIPEFS_MAGIC                                = 0x50495045
 	PPC_CMM_MAGIC                               = 0xc7571590
 	PPPIOCGNPMODE                               = 0xc008744c
@@ -1576,6 +1909,13 @@
 	PR_MCE_KILL_SET                             = 0x1
 	PR_MPX_DISABLE_MANAGEMENT                   = 0x2c
 	PR_MPX_ENABLE_MANAGEMENT                    = 0x2b
+	PR_MTE_TAG_MASK                             = 0x7fff8
+	PR_MTE_TAG_SHIFT                            = 0x3
+	PR_MTE_TCF_ASYNC                            = 0x4
+	PR_MTE_TCF_MASK                             = 0x6
+	PR_MTE_TCF_NONE                             = 0x0
+	PR_MTE_TCF_SHIFT                            = 0x1
+	PR_MTE_TCF_SYNC                             = 0x2
 	PR_PAC_APDAKEY                              = 0x4
 	PR_PAC_APDBKEY                              = 0x8
 	PR_PAC_APGAKEY                              = 0x10
@@ -1613,6 +1953,7 @@
 	PR_SET_SECCOMP                              = 0x16
 	PR_SET_SECUREBITS                           = 0x1c
 	PR_SET_SPECULATION_CTRL                     = 0x35
+	PR_SET_SYSCALL_USER_DISPATCH                = 0x3b
 	PR_SET_TAGGED_ADDR_CTRL                     = 0x37
 	PR_SET_THP_DISABLE                          = 0x29
 	PR_SET_TIMERSLACK                           = 0x1d
@@ -1632,6 +1973,8 @@
 	PR_SVE_SET_VL_ONEXEC                        = 0x40000
 	PR_SVE_VL_INHERIT                           = 0x20000
 	PR_SVE_VL_LEN_MASK                          = 0xffff
+	PR_SYS_DISPATCH_OFF                         = 0x0
+	PR_SYS_DISPATCH_ON                          = 0x1
 	PR_TAGGED_ADDR_ENABLE                       = 0x1
 	PR_TASK_PERF_EVENTS_DISABLE                 = 0x1f
 	PR_TASK_PERF_EVENTS_ENABLE                  = 0x20
@@ -1701,6 +2044,11 @@
 	QNX4_SUPER_MAGIC                            = 0x2f
 	QNX6_SUPER_MAGIC                            = 0x68191122
 	RAMFS_MAGIC                                 = 0x858458f6
+	RAW_PAYLOAD_DIGITAL                         = 0x3
+	RAW_PAYLOAD_HCI                             = 0x2
+	RAW_PAYLOAD_LLCP                            = 0x0
+	RAW_PAYLOAD_NCI                             = 0x1
+	RAW_PAYLOAD_PROPRIETARY                     = 0x4
 	RDTGROUP_SUPER_MAGIC                        = 0x7655821
 	REISERFS_SUPER_MAGIC                        = 0x52654973
 	RENAME_EXCHANGE                             = 0x2
@@ -1751,6 +2099,10 @@
 	RTCF_NAT                                    = 0x800000
 	RTCF_VALVE                                  = 0x200000
 	RTC_AF                                      = 0x20
+	RTC_FEATURE_ALARM                           = 0x0
+	RTC_FEATURE_ALARM_RES_MINUTE                = 0x1
+	RTC_FEATURE_CNT                             = 0x3
+	RTC_FEATURE_NEED_WEEK_DAY                   = 0x2
 	RTC_IRQF                                    = 0x80
 	RTC_MAX_FREQ                                = 0x2000
 	RTC_PF                                      = 0x40
@@ -1824,6 +2176,7 @@
 	RTM_F_LOOKUP_TABLE                          = 0x1000
 	RTM_F_NOTIFY                                = 0x100
 	RTM_F_OFFLOAD                               = 0x4000
+	RTM_F_OFFLOAD_FAILED                        = 0x20000000
 	RTM_F_PREFIX                                = 0x800
 	RTM_F_TRAP                                  = 0x8000
 	RTM_GETACTION                               = 0x32
@@ -1877,12 +2230,13 @@
 	RTM_SETLINK                                 = 0x13
 	RTM_SETNEIGHTBL                             = 0x43
 	RTNH_ALIGNTO                                = 0x4
-	RTNH_COMPARE_MASK                           = 0x19
+	RTNH_COMPARE_MASK                           = 0x59
 	RTNH_F_DEAD                                 = 0x1
 	RTNH_F_LINKDOWN                             = 0x10
 	RTNH_F_OFFLOAD                              = 0x8
 	RTNH_F_ONLINK                               = 0x4
 	RTNH_F_PERVASIVE                            = 0x2
+	RTNH_F_TRAP                                 = 0x40
 	RTNH_F_UNRESOLVED                           = 0x20
 	RTN_MAX                                     = 0xb
 	RTPROT_BABEL                                = 0x2a
@@ -1894,6 +2248,7 @@
 	RTPROT_EIGRP                                = 0xc0
 	RTPROT_GATED                                = 0x8
 	RTPROT_ISIS                                 = 0xbb
+	RTPROT_KEEPALIVED                           = 0x12
 	RTPROT_KERNEL                               = 0x2
 	RTPROT_MROUTED                              = 0x11
 	RTPROT_MRT                                  = 0xa
@@ -2084,6 +2439,7 @@
 	SO_EE_ORIGIN_TXSTATUS                       = 0x4
 	SO_EE_ORIGIN_TXTIME                         = 0x6
 	SO_EE_ORIGIN_ZEROCOPY                       = 0x5
+	SO_EE_RFC4884_FLAG_INVALID                  = 0x1
 	SO_GET_FILTER                               = 0x1a
 	SO_NO_CHECK                                 = 0xb
 	SO_PEERNAME                                 = 0x1c
@@ -2108,7 +2464,7 @@
 	STATX_ATTR_APPEND                           = 0x20
 	STATX_ATTR_AUTOMOUNT                        = 0x1000
 	STATX_ATTR_COMPRESSED                       = 0x4
-	STATX_ATTR_DAX                              = 0x2000
+	STATX_ATTR_DAX                              = 0x200000
 	STATX_ATTR_ENCRYPTED                        = 0x800
 	STATX_ATTR_IMMUTABLE                        = 0x10
 	STATX_ATTR_MOUNT_ROOT                       = 0x2000
@@ -2227,6 +2583,8 @@
 	TCP_TX_DELAY                                = 0x25
 	TCP_ULP                                     = 0x1f
 	TCP_USER_TIMEOUT                            = 0x12
+	TCP_V4_FLOW                                 = 0x1
+	TCP_V6_FLOW                                 = 0x5
 	TCP_WINDOW_CLAMP                            = 0xa
 	TCP_ZEROCOPY_RECEIVE                        = 0x23
 	TFD_TIMER_ABSTIME                           = 0x1
@@ -2292,6 +2650,7 @@
 	TIPC_NODE_STATE                             = 0x0
 	TIPC_OK                                     = 0x0
 	TIPC_PUBLISHED                              = 0x1
+	TIPC_REKEYING_NOW                           = 0xffffffff
 	TIPC_RESERVED_TYPES                         = 0x40
 	TIPC_RETDATA                                = 0x2
 	TIPC_SERVICE_ADDR                           = 0x2
@@ -2348,15 +2707,34 @@
 	VMADDR_CID_HOST                             = 0x2
 	VMADDR_CID_HYPERVISOR                       = 0x0
 	VMADDR_CID_LOCAL                            = 0x1
+	VMADDR_FLAG_TO_HOST                         = 0x1
 	VMADDR_PORT_ANY                             = 0xffffffff
 	VM_SOCKETS_INVALID_VERSION                  = 0xffffffff
 	VQUIT                                       = 0x1
 	VT0                                         = 0x0
+	WAKE_MAGIC                                  = 0x20
 	WALL                                        = 0x40000000
 	WCLONE                                      = 0x80000000
 	WCONTINUED                                  = 0x8
 	WDIOC_SETPRETIMEOUT                         = 0xc0045708
 	WDIOC_SETTIMEOUT                            = 0xc0045706
+	WDIOF_ALARMONLY                             = 0x400
+	WDIOF_CARDRESET                             = 0x20
+	WDIOF_EXTERN1                               = 0x4
+	WDIOF_EXTERN2                               = 0x8
+	WDIOF_FANFAULT                              = 0x2
+	WDIOF_KEEPALIVEPING                         = 0x8000
+	WDIOF_MAGICCLOSE                            = 0x100
+	WDIOF_OVERHEAT                              = 0x1
+	WDIOF_POWEROVER                             = 0x40
+	WDIOF_POWERUNDER                            = 0x10
+	WDIOF_PRETIMEOUT                            = 0x200
+	WDIOF_SETTIMEOUT                            = 0x80
+	WDIOF_UNKNOWN                               = -0x1
+	WDIOS_DISABLECARD                           = 0x1
+	WDIOS_ENABLECARD                            = 0x2
+	WDIOS_TEMPPANIC                             = 0x4
+	WDIOS_UNKNOWN                               = -0x1
 	WEXITED                                     = 0x4
 	WIN_ACKMEDIACHANGE                          = 0xdb
 	WIN_CHECKPOWERMODE1                         = 0xe5
@@ -2477,6 +2855,9 @@
 	Z3FOLD_MAGIC                                = 0x33
 	ZONEFS_MAGIC                                = 0x5a4f4653
 	ZSMALLOC_MAGIC                              = 0x58295829
+	_HIDIOCGRAWNAME_LEN                         = 0x80
+	_HIDIOCGRAWPHYS_LEN                         = 0x40
+	_HIDIOCGRAWUNIQ_LEN                         = 0x40
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_386.go b/vendor/golang.org/x/sys/unix/zerrors_linux_386.go
index dd282c0..09fc559 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_386.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_386.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include -m32
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && linux
 // +build 386,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m32 _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m32 /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -93,6 +96,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -119,6 +125,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc00c4d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc00c4d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x20
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -128,6 +147,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -165,6 +188,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -192,6 +216,7 @@
 	PPPIOCSPASS                      = 0x40087447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PR_SET_PTRACER_ANY               = 0xffffffff
 	PTRACE_GETFPREGS                 = 0xe
@@ -268,6 +293,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -290,6 +316,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -481,6 +508,9 @@
 	X86_FXSR_MAGIC                   = 0x0
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go
index 82fc93c..75730cc 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_amd64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && linux
 // +build amd64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m64 _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m64 /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -93,6 +96,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -119,6 +125,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -128,6 +147,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -165,6 +188,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -192,6 +216,7 @@
 	PPPIOCSPASS                      = 0x40107447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	PTRACE_ARCH_PRCTL                = 0x1e
@@ -269,6 +294,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -291,6 +317,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -481,6 +508,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go b/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go
index fe7094f..127cf17 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_arm.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && linux
 // +build arm,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc00c4d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc00c4d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x20
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x40087447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PR_SET_PTRACER_ANY               = 0xffffffff
 	PTRACE_GETCRUNCHREGS             = 0x19
@@ -275,6 +300,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -297,6 +323,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -487,6 +514,9 @@
 	WORDSIZE                         = 0x20
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go
index 3b6cc58..957ca1f 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_arm64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include -fsigned-char
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && linux
 // +build arm64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -95,6 +98,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -120,6 +126,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -129,6 +148,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -166,6 +189,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -193,9 +217,13 @@
 	PPPIOCSPASS                      = 0x40107447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PROT_BTI                         = 0x10
+	PROT_MTE                         = 0x20
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
+	PTRACE_PEEKMTETAGS               = 0x21
+	PTRACE_POKEMTETAGS               = 0x22
 	PTRACE_SYSEMU                    = 0x1f
 	PTRACE_SYSEMU_SINGLESTEP         = 0x20
 	RLIMIT_AS                        = 0x9
@@ -262,6 +290,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -284,6 +313,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -475,6 +505,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go
index ce3d9ae..314a205 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips && linux
 // +build mips,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x18
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x100
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc00c4d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc00c4d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x20
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x8
 	O_ASYNC                          = 0x1000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x80087447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PR_SET_PTRACER_ANY               = 0xffffffff
 	PTRACE_GETFPREGS                 = 0xe
@@ -268,6 +293,7 @@
 	SO_BROADCAST                     = 0x20
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -290,6 +316,7 @@
 	SO_PEERCRED                      = 0x12
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1e
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x1028
 	SO_RCVBUF                        = 0x1002
 	SO_RCVBUFFORCE                   = 0x21
@@ -483,6 +510,9 @@
 	WORDSIZE                         = 0x20
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go
index 7a85215..457e8de 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && linux
 // +build mips64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x18
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x100
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x8
 	O_ASYNC                          = 0x1000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x80107447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	PTRACE_GETFPREGS                 = 0xe
@@ -268,6 +293,7 @@
 	SO_BROADCAST                     = 0x20
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -290,6 +316,7 @@
 	SO_PEERCRED                      = 0x12
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1e
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x1028
 	SO_RCVBUF                        = 0x1002
 	SO_RCVBUFFORCE                   = 0x21
@@ -483,6 +510,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go
index 07d4cc1..33cd28f 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mips64le.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64le && linux
 // +build mips64le,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x18
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x100
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x8
 	O_ASYNC                          = 0x1000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x80107447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	PTRACE_GETFPREGS                 = 0xe
@@ -268,6 +293,7 @@
 	SO_BROADCAST                     = 0x20
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -290,6 +316,7 @@
 	SO_PEERCRED                      = 0x12
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1e
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x1028
 	SO_RCVBUF                        = 0x1002
 	SO_RCVBUFFORCE                   = 0x21
@@ -483,6 +510,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go b/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go
index d4842ba..0e085ba 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_mipsle.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mipsle && linux
 // +build mipsle,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x18
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x100
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc00c4d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc00c4d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x20
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x8
 	O_ASYNC                          = 0x1000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x80087447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PR_SET_PTRACER_ANY               = 0xffffffff
 	PTRACE_GETFPREGS                 = 0xe
@@ -268,6 +293,7 @@
 	SO_BROADCAST                     = 0x20
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -290,6 +316,7 @@
 	SO_PEERCRED                      = 0x12
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1e
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x1028
 	SO_RCVBUF                        = 0x1002
 	SO_RCVBUFFORCE                   = 0x21
@@ -483,6 +510,9 @@
 	WORDSIZE                         = 0x20
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go
new file mode 100644
index 0000000..1b5928c
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc.go
@@ -0,0 +1,879 @@
+// mkerrors.sh -Wall -Werror -static -I/tmp/include
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build ppc && linux
+// +build ppc,linux
+
+// Code generated by cmd/cgo -godefs; DO NOT EDIT.
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
+
+package unix
+
+import "syscall"
+
+const (
+	B1000000                         = 0x17
+	B115200                          = 0x11
+	B1152000                         = 0x18
+	B1500000                         = 0x19
+	B2000000                         = 0x1a
+	B230400                          = 0x12
+	B2500000                         = 0x1b
+	B3000000                         = 0x1c
+	B3500000                         = 0x1d
+	B4000000                         = 0x1e
+	B460800                          = 0x13
+	B500000                          = 0x14
+	B57600                           = 0x10
+	B576000                          = 0x15
+	B921600                          = 0x16
+	BLKBSZGET                        = 0x40041270
+	BLKBSZSET                        = 0x80041271
+	BLKFLSBUF                        = 0x20001261
+	BLKFRAGET                        = 0x20001265
+	BLKFRASET                        = 0x20001264
+	BLKGETSIZE                       = 0x20001260
+	BLKGETSIZE64                     = 0x40041272
+	BLKPBSZGET                       = 0x2000127b
+	BLKRAGET                         = 0x20001263
+	BLKRASET                         = 0x20001262
+	BLKROGET                         = 0x2000125e
+	BLKROSET                         = 0x2000125d
+	BLKRRPART                        = 0x2000125f
+	BLKSECTGET                       = 0x20001267
+	BLKSECTSET                       = 0x20001266
+	BLKSSZGET                        = 0x20001268
+	BOTHER                           = 0x1f
+	BS1                              = 0x8000
+	BSDLY                            = 0x8000
+	CBAUD                            = 0xff
+	CBAUDEX                          = 0x0
+	CIBAUD                           = 0xff0000
+	CLOCAL                           = 0x8000
+	CR1                              = 0x1000
+	CR2                              = 0x2000
+	CR3                              = 0x3000
+	CRDLY                            = 0x3000
+	CREAD                            = 0x800
+	CS6                              = 0x100
+	CS7                              = 0x200
+	CS8                              = 0x300
+	CSIZE                            = 0x300
+	CSTOPB                           = 0x400
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
+	ECHOCTL                          = 0x40
+	ECHOE                            = 0x2
+	ECHOK                            = 0x4
+	ECHOKE                           = 0x1
+	ECHONL                           = 0x10
+	ECHOPRT                          = 0x20
+	EFD_CLOEXEC                      = 0x80000
+	EFD_NONBLOCK                     = 0x800
+	EPOLL_CLOEXEC                    = 0x80000
+	EXTPROC                          = 0x10000000
+	FF1                              = 0x4000
+	FFDLY                            = 0x4000
+	FICLONE                          = 0x80049409
+	FICLONERANGE                     = 0x8020940d
+	FLUSHO                           = 0x800000
+	FS_IOC_ENABLE_VERITY             = 0x80806685
+	FS_IOC_GETFLAGS                  = 0x40046601
+	FS_IOC_GET_ENCRYPTION_NONCE      = 0x4010661b
+	FS_IOC_GET_ENCRYPTION_POLICY     = 0x800c6615
+	FS_IOC_GET_ENCRYPTION_PWSALT     = 0x80106614
+	FS_IOC_SETFLAGS                  = 0x80046602
+	FS_IOC_SET_ENCRYPTION_POLICY     = 0x400c6613
+	F_GETLK                          = 0xc
+	F_GETLK64                        = 0xc
+	F_GETOWN                         = 0x9
+	F_RDLCK                          = 0x0
+	F_SETLK                          = 0xd
+	F_SETLK64                        = 0xd
+	F_SETLKW                         = 0xe
+	F_SETLKW64                       = 0xe
+	F_SETOWN                         = 0x8
+	F_UNLCK                          = 0x2
+	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
+	HUPCL                            = 0x4000
+	ICANON                           = 0x100
+	IEXTEN                           = 0x400
+	IN_CLOEXEC                       = 0x80000
+	IN_NONBLOCK                      = 0x800
+	IOCTL_VM_SOCKETS_GET_LOCAL_CID   = 0x200007b9
+	ISIG                             = 0x80
+	IUCLC                            = 0x1000
+	IXOFF                            = 0x400
+	IXON                             = 0x200
+	MAP_ANON                         = 0x20
+	MAP_ANONYMOUS                    = 0x20
+	MAP_DENYWRITE                    = 0x800
+	MAP_EXECUTABLE                   = 0x1000
+	MAP_GROWSDOWN                    = 0x100
+	MAP_HUGETLB                      = 0x40000
+	MAP_LOCKED                       = 0x80
+	MAP_NONBLOCK                     = 0x10000
+	MAP_NORESERVE                    = 0x40
+	MAP_POPULATE                     = 0x8000
+	MAP_STACK                        = 0x20000
+	MAP_SYNC                         = 0x80000
+	MCL_CURRENT                      = 0x2000
+	MCL_FUTURE                       = 0x4000
+	MCL_ONFAULT                      = 0x8000
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc00c4d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc00c4d03
+	MTDFILEMODE                      = 0x20004d13
+	NFDBITS                          = 0x20
+	NL2                              = 0x200
+	NL3                              = 0x300
+	NLDLY                            = 0x300
+	NOFLSH                           = 0x80000000
+	NS_GET_NSTYPE                    = 0x2000b703
+	NS_GET_OWNER_UID                 = 0x2000b704
+	NS_GET_PARENT                    = 0x2000b702
+	NS_GET_USERNS                    = 0x2000b701
+	OLCUC                            = 0x4
+	ONLCR                            = 0x2
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
+	O_APPEND                         = 0x400
+	O_ASYNC                          = 0x2000
+	O_CLOEXEC                        = 0x80000
+	O_CREAT                          = 0x40
+	O_DIRECT                         = 0x20000
+	O_DIRECTORY                      = 0x4000
+	O_DSYNC                          = 0x1000
+	O_EXCL                           = 0x80
+	O_FSYNC                          = 0x101000
+	O_LARGEFILE                      = 0x10000
+	O_NDELAY                         = 0x800
+	O_NOATIME                        = 0x40000
+	O_NOCTTY                         = 0x100
+	O_NOFOLLOW                       = 0x8000
+	O_NONBLOCK                       = 0x800
+	O_PATH                           = 0x200000
+	O_RSYNC                          = 0x101000
+	O_SYNC                           = 0x101000
+	O_TMPFILE                        = 0x404000
+	O_TRUNC                          = 0x200
+	PARENB                           = 0x1000
+	PARODD                           = 0x2000
+	PENDIN                           = 0x20000000
+	PERF_EVENT_IOC_DISABLE           = 0x20002401
+	PERF_EVENT_IOC_ENABLE            = 0x20002400
+	PERF_EVENT_IOC_ID                = 0x40042407
+	PERF_EVENT_IOC_MODIFY_ATTRIBUTES = 0x8004240b
+	PERF_EVENT_IOC_PAUSE_OUTPUT      = 0x80042409
+	PERF_EVENT_IOC_PERIOD            = 0x80082404
+	PERF_EVENT_IOC_QUERY_BPF         = 0xc004240a
+	PERF_EVENT_IOC_REFRESH           = 0x20002402
+	PERF_EVENT_IOC_RESET             = 0x20002403
+	PERF_EVENT_IOC_SET_BPF           = 0x80042408
+	PERF_EVENT_IOC_SET_FILTER        = 0x80042406
+	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
+	PPPIOCATTACH                     = 0x8004743d
+	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
+	PPPIOCCONNECT                    = 0x8004743a
+	PPPIOCDETACH                     = 0x8004743c
+	PPPIOCDISCONN                    = 0x20007439
+	PPPIOCGASYNCMAP                  = 0x40047458
+	PPPIOCGCHAN                      = 0x40047437
+	PPPIOCGDEBUG                     = 0x40047441
+	PPPIOCGFLAGS                     = 0x4004745a
+	PPPIOCGIDLE                      = 0x4008743f
+	PPPIOCGIDLE32                    = 0x4008743f
+	PPPIOCGIDLE64                    = 0x4010743f
+	PPPIOCGL2TPSTATS                 = 0x40487436
+	PPPIOCGMRU                       = 0x40047453
+	PPPIOCGRASYNCMAP                 = 0x40047455
+	PPPIOCGUNIT                      = 0x40047456
+	PPPIOCGXASYNCMAP                 = 0x40207450
+	PPPIOCSACTIVE                    = 0x80087446
+	PPPIOCSASYNCMAP                  = 0x80047457
+	PPPIOCSCOMPRESS                  = 0x800c744d
+	PPPIOCSDEBUG                     = 0x80047440
+	PPPIOCSFLAGS                     = 0x80047459
+	PPPIOCSMAXCID                    = 0x80047451
+	PPPIOCSMRRU                      = 0x8004743b
+	PPPIOCSMRU                       = 0x80047452
+	PPPIOCSNPMODE                    = 0x8008744b
+	PPPIOCSPASS                      = 0x80087447
+	PPPIOCSRASYNCMAP                 = 0x80047454
+	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
+	PPPIOCXFERUNIT                   = 0x2000744e
+	PROT_SAO                         = 0x10
+	PR_SET_PTRACER_ANY               = 0xffffffff
+	PTRACE_GETEVRREGS                = 0x14
+	PTRACE_GETFPREGS                 = 0xe
+	PTRACE_GETREGS64                 = 0x16
+	PTRACE_GETVRREGS                 = 0x12
+	PTRACE_GETVSRREGS                = 0x1b
+	PTRACE_GET_DEBUGREG              = 0x19
+	PTRACE_SETEVRREGS                = 0x15
+	PTRACE_SETFPREGS                 = 0xf
+	PTRACE_SETREGS64                 = 0x17
+	PTRACE_SETVRREGS                 = 0x13
+	PTRACE_SETVSRREGS                = 0x1c
+	PTRACE_SET_DEBUGREG              = 0x1a
+	PTRACE_SINGLEBLOCK               = 0x100
+	PTRACE_SYSEMU                    = 0x1d
+	PTRACE_SYSEMU_SINGLESTEP         = 0x1e
+	PT_CCR                           = 0x26
+	PT_CTR                           = 0x23
+	PT_DAR                           = 0x29
+	PT_DSCR                          = 0x2c
+	PT_DSISR                         = 0x2a
+	PT_FPR0                          = 0x30
+	PT_FPR31                         = 0x6e
+	PT_FPSCR                         = 0x71
+	PT_LNK                           = 0x24
+	PT_MQ                            = 0x27
+	PT_MSR                           = 0x21
+	PT_NIP                           = 0x20
+	PT_ORIG_R3                       = 0x22
+	PT_R0                            = 0x0
+	PT_R1                            = 0x1
+	PT_R10                           = 0xa
+	PT_R11                           = 0xb
+	PT_R12                           = 0xc
+	PT_R13                           = 0xd
+	PT_R14                           = 0xe
+	PT_R15                           = 0xf
+	PT_R16                           = 0x10
+	PT_R17                           = 0x11
+	PT_R18                           = 0x12
+	PT_R19                           = 0x13
+	PT_R2                            = 0x2
+	PT_R20                           = 0x14
+	PT_R21                           = 0x15
+	PT_R22                           = 0x16
+	PT_R23                           = 0x17
+	PT_R24                           = 0x18
+	PT_R25                           = 0x19
+	PT_R26                           = 0x1a
+	PT_R27                           = 0x1b
+	PT_R28                           = 0x1c
+	PT_R29                           = 0x1d
+	PT_R3                            = 0x3
+	PT_R30                           = 0x1e
+	PT_R31                           = 0x1f
+	PT_R4                            = 0x4
+	PT_R5                            = 0x5
+	PT_R6                            = 0x6
+	PT_R7                            = 0x7
+	PT_R8                            = 0x8
+	PT_R9                            = 0x9
+	PT_REGS_COUNT                    = 0x2c
+	PT_RESULT                        = 0x2b
+	PT_TRAP                          = 0x28
+	PT_XER                           = 0x25
+	RLIMIT_AS                        = 0x9
+	RLIMIT_MEMLOCK                   = 0x8
+	RLIMIT_NOFILE                    = 0x7
+	RLIMIT_NPROC                     = 0x6
+	RLIMIT_RSS                       = 0x5
+	RNDADDENTROPY                    = 0x80085203
+	RNDADDTOENTCNT                   = 0x80045201
+	RNDCLEARPOOL                     = 0x20005206
+	RNDGETENTCNT                     = 0x40045200
+	RNDGETPOOL                       = 0x40085202
+	RNDRESEEDCRNG                    = 0x20005207
+	RNDZAPENTCNT                     = 0x20005204
+	RTC_AIE_OFF                      = 0x20007002
+	RTC_AIE_ON                       = 0x20007001
+	RTC_ALM_READ                     = 0x40247008
+	RTC_ALM_SET                      = 0x80247007
+	RTC_EPOCH_READ                   = 0x4004700d
+	RTC_EPOCH_SET                    = 0x8004700e
+	RTC_IRQP_READ                    = 0x4004700b
+	RTC_IRQP_SET                     = 0x8004700c
+	RTC_PIE_OFF                      = 0x20007006
+	RTC_PIE_ON                       = 0x20007005
+	RTC_PLL_GET                      = 0x401c7011
+	RTC_PLL_SET                      = 0x801c7012
+	RTC_RD_TIME                      = 0x40247009
+	RTC_SET_TIME                     = 0x8024700a
+	RTC_UIE_OFF                      = 0x20007004
+	RTC_UIE_ON                       = 0x20007003
+	RTC_VL_CLR                       = 0x20007014
+	RTC_VL_READ                      = 0x40047013
+	RTC_WIE_OFF                      = 0x20007010
+	RTC_WIE_ON                       = 0x2000700f
+	RTC_WKALM_RD                     = 0x40287010
+	RTC_WKALM_SET                    = 0x8028700f
+	SCM_TIMESTAMPING                 = 0x25
+	SCM_TIMESTAMPING_OPT_STATS       = 0x36
+	SCM_TIMESTAMPING_PKTINFO         = 0x3a
+	SCM_TIMESTAMPNS                  = 0x23
+	SCM_TXTIME                       = 0x3d
+	SCM_WIFI_STATUS                  = 0x29
+	SFD_CLOEXEC                      = 0x80000
+	SFD_NONBLOCK                     = 0x800
+	SIOCATMARK                       = 0x8905
+	SIOCGPGRP                        = 0x8904
+	SIOCGSTAMPNS_NEW                 = 0x40108907
+	SIOCGSTAMP_NEW                   = 0x40108906
+	SIOCINQ                          = 0x4004667f
+	SIOCOUTQ                         = 0x40047473
+	SIOCSPGRP                        = 0x8902
+	SOCK_CLOEXEC                     = 0x80000
+	SOCK_DGRAM                       = 0x2
+	SOCK_NONBLOCK                    = 0x800
+	SOCK_STREAM                      = 0x1
+	SOL_SOCKET                       = 0x1
+	SO_ACCEPTCONN                    = 0x1e
+	SO_ATTACH_BPF                    = 0x32
+	SO_ATTACH_REUSEPORT_CBPF         = 0x33
+	SO_ATTACH_REUSEPORT_EBPF         = 0x34
+	SO_BINDTODEVICE                  = 0x19
+	SO_BINDTOIFINDEX                 = 0x3e
+	SO_BPF_EXTENSIONS                = 0x30
+	SO_BROADCAST                     = 0x6
+	SO_BSDCOMPAT                     = 0xe
+	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
+	SO_CNX_ADVICE                    = 0x35
+	SO_COOKIE                        = 0x39
+	SO_DETACH_REUSEPORT_BPF          = 0x44
+	SO_DOMAIN                        = 0x27
+	SO_DONTROUTE                     = 0x5
+	SO_ERROR                         = 0x4
+	SO_INCOMING_CPU                  = 0x31
+	SO_INCOMING_NAPI_ID              = 0x38
+	SO_KEEPALIVE                     = 0x9
+	SO_LINGER                        = 0xd
+	SO_LOCK_FILTER                   = 0x2c
+	SO_MARK                          = 0x24
+	SO_MAX_PACING_RATE               = 0x2f
+	SO_MEMINFO                       = 0x37
+	SO_NOFCS                         = 0x2b
+	SO_OOBINLINE                     = 0xa
+	SO_PASSCRED                      = 0x14
+	SO_PASSSEC                       = 0x22
+	SO_PEEK_OFF                      = 0x2a
+	SO_PEERCRED                      = 0x15
+	SO_PEERGROUPS                    = 0x3b
+	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
+	SO_PROTOCOL                      = 0x26
+	SO_RCVBUF                        = 0x8
+	SO_RCVBUFFORCE                   = 0x21
+	SO_RCVLOWAT                      = 0x10
+	SO_RCVTIMEO                      = 0x12
+	SO_RCVTIMEO_NEW                  = 0x42
+	SO_RCVTIMEO_OLD                  = 0x12
+	SO_REUSEADDR                     = 0x2
+	SO_REUSEPORT                     = 0xf
+	SO_RXQ_OVFL                      = 0x28
+	SO_SECURITY_AUTHENTICATION       = 0x16
+	SO_SECURITY_ENCRYPTION_NETWORK   = 0x18
+	SO_SECURITY_ENCRYPTION_TRANSPORT = 0x17
+	SO_SELECT_ERR_QUEUE              = 0x2d
+	SO_SNDBUF                        = 0x7
+	SO_SNDBUFFORCE                   = 0x20
+	SO_SNDLOWAT                      = 0x11
+	SO_SNDTIMEO                      = 0x13
+	SO_SNDTIMEO_NEW                  = 0x43
+	SO_SNDTIMEO_OLD                  = 0x13
+	SO_TIMESTAMPING                  = 0x25
+	SO_TIMESTAMPING_NEW              = 0x41
+	SO_TIMESTAMPING_OLD              = 0x25
+	SO_TIMESTAMPNS                   = 0x23
+	SO_TIMESTAMPNS_NEW               = 0x40
+	SO_TIMESTAMPNS_OLD               = 0x23
+	SO_TIMESTAMP_NEW                 = 0x3f
+	SO_TXTIME                        = 0x3d
+	SO_TYPE                          = 0x3
+	SO_WIFI_STATUS                   = 0x29
+	SO_ZEROCOPY                      = 0x3c
+	TAB1                             = 0x400
+	TAB2                             = 0x800
+	TAB3                             = 0xc00
+	TABDLY                           = 0xc00
+	TCFLSH                           = 0x2000741f
+	TCGETA                           = 0x40147417
+	TCGETS                           = 0x402c7413
+	TCSAFLUSH                        = 0x2
+	TCSBRK                           = 0x2000741d
+	TCSBRKP                          = 0x5425
+	TCSETA                           = 0x80147418
+	TCSETAF                          = 0x8014741c
+	TCSETAW                          = 0x80147419
+	TCSETS                           = 0x802c7414
+	TCSETSF                          = 0x802c7416
+	TCSETSW                          = 0x802c7415
+	TCXONC                           = 0x2000741e
+	TFD_CLOEXEC                      = 0x80000
+	TFD_NONBLOCK                     = 0x800
+	TIOCCBRK                         = 0x5428
+	TIOCCONS                         = 0x541d
+	TIOCEXCL                         = 0x540c
+	TIOCGDEV                         = 0x40045432
+	TIOCGETC                         = 0x40067412
+	TIOCGETD                         = 0x5424
+	TIOCGETP                         = 0x40067408
+	TIOCGEXCL                        = 0x40045440
+	TIOCGICOUNT                      = 0x545d
+	TIOCGISO7816                     = 0x40285442
+	TIOCGLCKTRMIOS                   = 0x5456
+	TIOCGLTC                         = 0x40067474
+	TIOCGPGRP                        = 0x40047477
+	TIOCGPKT                         = 0x40045438
+	TIOCGPTLCK                       = 0x40045439
+	TIOCGPTN                         = 0x40045430
+	TIOCGPTPEER                      = 0x20005441
+	TIOCGRS485                       = 0x542e
+	TIOCGSERIAL                      = 0x541e
+	TIOCGSID                         = 0x5429
+	TIOCGSOFTCAR                     = 0x5419
+	TIOCGWINSZ                       = 0x40087468
+	TIOCINQ                          = 0x4004667f
+	TIOCLINUX                        = 0x541c
+	TIOCMBIC                         = 0x5417
+	TIOCMBIS                         = 0x5416
+	TIOCMGET                         = 0x5415
+	TIOCMIWAIT                       = 0x545c
+	TIOCMSET                         = 0x5418
+	TIOCM_CAR                        = 0x40
+	TIOCM_CD                         = 0x40
+	TIOCM_CTS                        = 0x20
+	TIOCM_DSR                        = 0x100
+	TIOCM_LOOP                       = 0x8000
+	TIOCM_OUT1                       = 0x2000
+	TIOCM_OUT2                       = 0x4000
+	TIOCM_RI                         = 0x80
+	TIOCM_RNG                        = 0x80
+	TIOCM_SR                         = 0x10
+	TIOCM_ST                         = 0x8
+	TIOCNOTTY                        = 0x5422
+	TIOCNXCL                         = 0x540d
+	TIOCOUTQ                         = 0x40047473
+	TIOCPKT                          = 0x5420
+	TIOCSBRK                         = 0x5427
+	TIOCSCTTY                        = 0x540e
+	TIOCSERCONFIG                    = 0x5453
+	TIOCSERGETLSR                    = 0x5459
+	TIOCSERGETMULTI                  = 0x545a
+	TIOCSERGSTRUCT                   = 0x5458
+	TIOCSERGWILD                     = 0x5454
+	TIOCSERSETMULTI                  = 0x545b
+	TIOCSERSWILD                     = 0x5455
+	TIOCSER_TEMT                     = 0x1
+	TIOCSETC                         = 0x80067411
+	TIOCSETD                         = 0x5423
+	TIOCSETN                         = 0x8006740a
+	TIOCSETP                         = 0x80067409
+	TIOCSIG                          = 0x80045436
+	TIOCSISO7816                     = 0xc0285443
+	TIOCSLCKTRMIOS                   = 0x5457
+	TIOCSLTC                         = 0x80067475
+	TIOCSPGRP                        = 0x80047476
+	TIOCSPTLCK                       = 0x80045431
+	TIOCSRS485                       = 0x542f
+	TIOCSSERIAL                      = 0x541f
+	TIOCSSOFTCAR                     = 0x541a
+	TIOCSTART                        = 0x2000746e
+	TIOCSTI                          = 0x5412
+	TIOCSTOP                         = 0x2000746f
+	TIOCSWINSZ                       = 0x80087467
+	TIOCVHANGUP                      = 0x5437
+	TOSTOP                           = 0x400000
+	TUNATTACHFILTER                  = 0x800854d5
+	TUNDETACHFILTER                  = 0x800854d6
+	TUNGETDEVNETNS                   = 0x200054e3
+	TUNGETFEATURES                   = 0x400454cf
+	TUNGETFILTER                     = 0x400854db
+	TUNGETIFF                        = 0x400454d2
+	TUNGETSNDBUF                     = 0x400454d3
+	TUNGETVNETBE                     = 0x400454df
+	TUNGETVNETHDRSZ                  = 0x400454d7
+	TUNGETVNETLE                     = 0x400454dd
+	TUNSETCARRIER                    = 0x800454e2
+	TUNSETDEBUG                      = 0x800454c9
+	TUNSETFILTEREBPF                 = 0x400454e1
+	TUNSETGROUP                      = 0x800454ce
+	TUNSETIFF                        = 0x800454ca
+	TUNSETIFINDEX                    = 0x800454da
+	TUNSETLINK                       = 0x800454cd
+	TUNSETNOCSUM                     = 0x800454c8
+	TUNSETOFFLOAD                    = 0x800454d0
+	TUNSETOWNER                      = 0x800454cc
+	TUNSETPERSIST                    = 0x800454cb
+	TUNSETQUEUE                      = 0x800454d9
+	TUNSETSNDBUF                     = 0x800454d4
+	TUNSETSTEERINGEBPF               = 0x400454e0
+	TUNSETTXFILTER                   = 0x800454d1
+	TUNSETVNETBE                     = 0x800454de
+	TUNSETVNETHDRSZ                  = 0x800454d8
+	TUNSETVNETLE                     = 0x800454dc
+	UBI_IOCATT                       = 0x80186f40
+	UBI_IOCDET                       = 0x80046f41
+	UBI_IOCEBCH                      = 0x80044f02
+	UBI_IOCEBER                      = 0x80044f01
+	UBI_IOCEBISMAP                   = 0x40044f05
+	UBI_IOCEBMAP                     = 0x80084f03
+	UBI_IOCEBUNMAP                   = 0x80044f04
+	UBI_IOCMKVOL                     = 0x80986f00
+	UBI_IOCRMVOL                     = 0x80046f01
+	UBI_IOCRNVOL                     = 0x91106f03
+	UBI_IOCRPEB                      = 0x80046f04
+	UBI_IOCRSVOL                     = 0x800c6f02
+	UBI_IOCSETVOLPROP                = 0x80104f06
+	UBI_IOCSPEB                      = 0x80046f05
+	UBI_IOCVOLCRBLK                  = 0x80804f07
+	UBI_IOCVOLRMBLK                  = 0x20004f08
+	UBI_IOCVOLUP                     = 0x80084f00
+	VDISCARD                         = 0x10
+	VEOF                             = 0x4
+	VEOL                             = 0x6
+	VEOL2                            = 0x8
+	VMIN                             = 0x5
+	VREPRINT                         = 0xb
+	VSTART                           = 0xd
+	VSTOP                            = 0xe
+	VSUSP                            = 0xc
+	VSWTC                            = 0x9
+	VT1                              = 0x10000
+	VTDLY                            = 0x10000
+	VTIME                            = 0x7
+	VWERASE                          = 0xa
+	WDIOC_GETBOOTSTATUS              = 0x40045702
+	WDIOC_GETPRETIMEOUT              = 0x40045709
+	WDIOC_GETSTATUS                  = 0x40045701
+	WDIOC_GETSUPPORT                 = 0x40285700
+	WDIOC_GETTEMP                    = 0x40045703
+	WDIOC_GETTIMELEFT                = 0x4004570a
+	WDIOC_GETTIMEOUT                 = 0x40045707
+	WDIOC_KEEPALIVE                  = 0x40045705
+	WDIOC_SETOPTIONS                 = 0x40045704
+	WORDSIZE                         = 0x20
+	XCASE                            = 0x4000
+	XTABS                            = 0xc00
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
+)
+
+// Errors
+const (
+	EADDRINUSE      = syscall.Errno(0x62)
+	EADDRNOTAVAIL   = syscall.Errno(0x63)
+	EADV            = syscall.Errno(0x44)
+	EAFNOSUPPORT    = syscall.Errno(0x61)
+	EALREADY        = syscall.Errno(0x72)
+	EBADE           = syscall.Errno(0x34)
+	EBADFD          = syscall.Errno(0x4d)
+	EBADMSG         = syscall.Errno(0x4a)
+	EBADR           = syscall.Errno(0x35)
+	EBADRQC         = syscall.Errno(0x38)
+	EBADSLT         = syscall.Errno(0x39)
+	EBFONT          = syscall.Errno(0x3b)
+	ECANCELED       = syscall.Errno(0x7d)
+	ECHRNG          = syscall.Errno(0x2c)
+	ECOMM           = syscall.Errno(0x46)
+	ECONNABORTED    = syscall.Errno(0x67)
+	ECONNREFUSED    = syscall.Errno(0x6f)
+	ECONNRESET      = syscall.Errno(0x68)
+	EDEADLK         = syscall.Errno(0x23)
+	EDEADLOCK       = syscall.Errno(0x3a)
+	EDESTADDRREQ    = syscall.Errno(0x59)
+	EDOTDOT         = syscall.Errno(0x49)
+	EDQUOT          = syscall.Errno(0x7a)
+	EHOSTDOWN       = syscall.Errno(0x70)
+	EHOSTUNREACH    = syscall.Errno(0x71)
+	EHWPOISON       = syscall.Errno(0x85)
+	EIDRM           = syscall.Errno(0x2b)
+	EILSEQ          = syscall.Errno(0x54)
+	EINPROGRESS     = syscall.Errno(0x73)
+	EISCONN         = syscall.Errno(0x6a)
+	EISNAM          = syscall.Errno(0x78)
+	EKEYEXPIRED     = syscall.Errno(0x7f)
+	EKEYREJECTED    = syscall.Errno(0x81)
+	EKEYREVOKED     = syscall.Errno(0x80)
+	EL2HLT          = syscall.Errno(0x33)
+	EL2NSYNC        = syscall.Errno(0x2d)
+	EL3HLT          = syscall.Errno(0x2e)
+	EL3RST          = syscall.Errno(0x2f)
+	ELIBACC         = syscall.Errno(0x4f)
+	ELIBBAD         = syscall.Errno(0x50)
+	ELIBEXEC        = syscall.Errno(0x53)
+	ELIBMAX         = syscall.Errno(0x52)
+	ELIBSCN         = syscall.Errno(0x51)
+	ELNRNG          = syscall.Errno(0x30)
+	ELOOP           = syscall.Errno(0x28)
+	EMEDIUMTYPE     = syscall.Errno(0x7c)
+	EMSGSIZE        = syscall.Errno(0x5a)
+	EMULTIHOP       = syscall.Errno(0x48)
+	ENAMETOOLONG    = syscall.Errno(0x24)
+	ENAVAIL         = syscall.Errno(0x77)
+	ENETDOWN        = syscall.Errno(0x64)
+	ENETRESET       = syscall.Errno(0x66)
+	ENETUNREACH     = syscall.Errno(0x65)
+	ENOANO          = syscall.Errno(0x37)
+	ENOBUFS         = syscall.Errno(0x69)
+	ENOCSI          = syscall.Errno(0x32)
+	ENODATA         = syscall.Errno(0x3d)
+	ENOKEY          = syscall.Errno(0x7e)
+	ENOLCK          = syscall.Errno(0x25)
+	ENOLINK         = syscall.Errno(0x43)
+	ENOMEDIUM       = syscall.Errno(0x7b)
+	ENOMSG          = syscall.Errno(0x2a)
+	ENONET          = syscall.Errno(0x40)
+	ENOPKG          = syscall.Errno(0x41)
+	ENOPROTOOPT     = syscall.Errno(0x5c)
+	ENOSR           = syscall.Errno(0x3f)
+	ENOSTR          = syscall.Errno(0x3c)
+	ENOSYS          = syscall.Errno(0x26)
+	ENOTCONN        = syscall.Errno(0x6b)
+	ENOTEMPTY       = syscall.Errno(0x27)
+	ENOTNAM         = syscall.Errno(0x76)
+	ENOTRECOVERABLE = syscall.Errno(0x83)
+	ENOTSOCK        = syscall.Errno(0x58)
+	ENOTSUP         = syscall.Errno(0x5f)
+	ENOTUNIQ        = syscall.Errno(0x4c)
+	EOPNOTSUPP      = syscall.Errno(0x5f)
+	EOVERFLOW       = syscall.Errno(0x4b)
+	EOWNERDEAD      = syscall.Errno(0x82)
+	EPFNOSUPPORT    = syscall.Errno(0x60)
+	EPROTO          = syscall.Errno(0x47)
+	EPROTONOSUPPORT = syscall.Errno(0x5d)
+	EPROTOTYPE      = syscall.Errno(0x5b)
+	EREMCHG         = syscall.Errno(0x4e)
+	EREMOTE         = syscall.Errno(0x42)
+	EREMOTEIO       = syscall.Errno(0x79)
+	ERESTART        = syscall.Errno(0x55)
+	ERFKILL         = syscall.Errno(0x84)
+	ESHUTDOWN       = syscall.Errno(0x6c)
+	ESOCKTNOSUPPORT = syscall.Errno(0x5e)
+	ESRMNT          = syscall.Errno(0x45)
+	ESTALE          = syscall.Errno(0x74)
+	ESTRPIPE        = syscall.Errno(0x56)
+	ETIME           = syscall.Errno(0x3e)
+	ETIMEDOUT       = syscall.Errno(0x6e)
+	ETOOMANYREFS    = syscall.Errno(0x6d)
+	EUCLEAN         = syscall.Errno(0x75)
+	EUNATCH         = syscall.Errno(0x31)
+	EUSERS          = syscall.Errno(0x57)
+	EXFULL          = syscall.Errno(0x36)
+)
+
+// Signals
+const (
+	SIGBUS    = syscall.Signal(0x7)
+	SIGCHLD   = syscall.Signal(0x11)
+	SIGCLD    = syscall.Signal(0x11)
+	SIGCONT   = syscall.Signal(0x12)
+	SIGIO     = syscall.Signal(0x1d)
+	SIGPOLL   = syscall.Signal(0x1d)
+	SIGPROF   = syscall.Signal(0x1b)
+	SIGPWR    = syscall.Signal(0x1e)
+	SIGSTKFLT = syscall.Signal(0x10)
+	SIGSTOP   = syscall.Signal(0x13)
+	SIGSYS    = syscall.Signal(0x1f)
+	SIGTSTP   = syscall.Signal(0x14)
+	SIGTTIN   = syscall.Signal(0x15)
+	SIGTTOU   = syscall.Signal(0x16)
+	SIGURG    = syscall.Signal(0x17)
+	SIGUSR1   = syscall.Signal(0xa)
+	SIGUSR2   = syscall.Signal(0xc)
+	SIGVTALRM = syscall.Signal(0x1a)
+	SIGWINCH  = syscall.Signal(0x1c)
+	SIGXCPU   = syscall.Signal(0x18)
+	SIGXFSZ   = syscall.Signal(0x19)
+)
+
+// Error table
+var errorList = [...]struct {
+	num  syscall.Errno
+	name string
+	desc string
+}{
+	{1, "EPERM", "operation not permitted"},
+	{2, "ENOENT", "no such file or directory"},
+	{3, "ESRCH", "no such process"},
+	{4, "EINTR", "interrupted system call"},
+	{5, "EIO", "input/output error"},
+	{6, "ENXIO", "no such device or address"},
+	{7, "E2BIG", "argument list too long"},
+	{8, "ENOEXEC", "exec format error"},
+	{9, "EBADF", "bad file descriptor"},
+	{10, "ECHILD", "no child processes"},
+	{11, "EAGAIN", "resource temporarily unavailable"},
+	{12, "ENOMEM", "cannot allocate memory"},
+	{13, "EACCES", "permission denied"},
+	{14, "EFAULT", "bad address"},
+	{15, "ENOTBLK", "block device required"},
+	{16, "EBUSY", "device or resource busy"},
+	{17, "EEXIST", "file exists"},
+	{18, "EXDEV", "invalid cross-device link"},
+	{19, "ENODEV", "no such device"},
+	{20, "ENOTDIR", "not a directory"},
+	{21, "EISDIR", "is a directory"},
+	{22, "EINVAL", "invalid argument"},
+	{23, "ENFILE", "too many open files in system"},
+	{24, "EMFILE", "too many open files"},
+	{25, "ENOTTY", "inappropriate ioctl for device"},
+	{26, "ETXTBSY", "text file busy"},
+	{27, "EFBIG", "file too large"},
+	{28, "ENOSPC", "no space left on device"},
+	{29, "ESPIPE", "illegal seek"},
+	{30, "EROFS", "read-only file system"},
+	{31, "EMLINK", "too many links"},
+	{32, "EPIPE", "broken pipe"},
+	{33, "EDOM", "numerical argument out of domain"},
+	{34, "ERANGE", "numerical result out of range"},
+	{35, "EDEADLK", "resource deadlock avoided"},
+	{36, "ENAMETOOLONG", "file name too long"},
+	{37, "ENOLCK", "no locks available"},
+	{38, "ENOSYS", "function not implemented"},
+	{39, "ENOTEMPTY", "directory not empty"},
+	{40, "ELOOP", "too many levels of symbolic links"},
+	{42, "ENOMSG", "no message of desired type"},
+	{43, "EIDRM", "identifier removed"},
+	{44, "ECHRNG", "channel number out of range"},
+	{45, "EL2NSYNC", "level 2 not synchronized"},
+	{46, "EL3HLT", "level 3 halted"},
+	{47, "EL3RST", "level 3 reset"},
+	{48, "ELNRNG", "link number out of range"},
+	{49, "EUNATCH", "protocol driver not attached"},
+	{50, "ENOCSI", "no CSI structure available"},
+	{51, "EL2HLT", "level 2 halted"},
+	{52, "EBADE", "invalid exchange"},
+	{53, "EBADR", "invalid request descriptor"},
+	{54, "EXFULL", "exchange full"},
+	{55, "ENOANO", "no anode"},
+	{56, "EBADRQC", "invalid request code"},
+	{57, "EBADSLT", "invalid slot"},
+	{58, "EDEADLOCK", "file locking deadlock error"},
+	{59, "EBFONT", "bad font file format"},
+	{60, "ENOSTR", "device not a stream"},
+	{61, "ENODATA", "no data available"},
+	{62, "ETIME", "timer expired"},
+	{63, "ENOSR", "out of streams resources"},
+	{64, "ENONET", "machine is not on the network"},
+	{65, "ENOPKG", "package not installed"},
+	{66, "EREMOTE", "object is remote"},
+	{67, "ENOLINK", "link has been severed"},
+	{68, "EADV", "advertise error"},
+	{69, "ESRMNT", "srmount error"},
+	{70, "ECOMM", "communication error on send"},
+	{71, "EPROTO", "protocol error"},
+	{72, "EMULTIHOP", "multihop attempted"},
+	{73, "EDOTDOT", "RFS specific error"},
+	{74, "EBADMSG", "bad message"},
+	{75, "EOVERFLOW", "value too large for defined data type"},
+	{76, "ENOTUNIQ", "name not unique on network"},
+	{77, "EBADFD", "file descriptor in bad state"},
+	{78, "EREMCHG", "remote address changed"},
+	{79, "ELIBACC", "can not access a needed shared library"},
+	{80, "ELIBBAD", "accessing a corrupted shared library"},
+	{81, "ELIBSCN", ".lib section in a.out corrupted"},
+	{82, "ELIBMAX", "attempting to link in too many shared libraries"},
+	{83, "ELIBEXEC", "cannot exec a shared library directly"},
+	{84, "EILSEQ", "invalid or incomplete multibyte or wide character"},
+	{85, "ERESTART", "interrupted system call should be restarted"},
+	{86, "ESTRPIPE", "streams pipe error"},
+	{87, "EUSERS", "too many users"},
+	{88, "ENOTSOCK", "socket operation on non-socket"},
+	{89, "EDESTADDRREQ", "destination address required"},
+	{90, "EMSGSIZE", "message too long"},
+	{91, "EPROTOTYPE", "protocol wrong type for socket"},
+	{92, "ENOPROTOOPT", "protocol not available"},
+	{93, "EPROTONOSUPPORT", "protocol not supported"},
+	{94, "ESOCKTNOSUPPORT", "socket type not supported"},
+	{95, "ENOTSUP", "operation not supported"},
+	{96, "EPFNOSUPPORT", "protocol family not supported"},
+	{97, "EAFNOSUPPORT", "address family not supported by protocol"},
+	{98, "EADDRINUSE", "address already in use"},
+	{99, "EADDRNOTAVAIL", "cannot assign requested address"},
+	{100, "ENETDOWN", "network is down"},
+	{101, "ENETUNREACH", "network is unreachable"},
+	{102, "ENETRESET", "network dropped connection on reset"},
+	{103, "ECONNABORTED", "software caused connection abort"},
+	{104, "ECONNRESET", "connection reset by peer"},
+	{105, "ENOBUFS", "no buffer space available"},
+	{106, "EISCONN", "transport endpoint is already connected"},
+	{107, "ENOTCONN", "transport endpoint is not connected"},
+	{108, "ESHUTDOWN", "cannot send after transport endpoint shutdown"},
+	{109, "ETOOMANYREFS", "too many references: cannot splice"},
+	{110, "ETIMEDOUT", "connection timed out"},
+	{111, "ECONNREFUSED", "connection refused"},
+	{112, "EHOSTDOWN", "host is down"},
+	{113, "EHOSTUNREACH", "no route to host"},
+	{114, "EALREADY", "operation already in progress"},
+	{115, "EINPROGRESS", "operation now in progress"},
+	{116, "ESTALE", "stale file handle"},
+	{117, "EUCLEAN", "structure needs cleaning"},
+	{118, "ENOTNAM", "not a XENIX named type file"},
+	{119, "ENAVAIL", "no XENIX semaphores available"},
+	{120, "EISNAM", "is a named type file"},
+	{121, "EREMOTEIO", "remote I/O error"},
+	{122, "EDQUOT", "disk quota exceeded"},
+	{123, "ENOMEDIUM", "no medium found"},
+	{124, "EMEDIUMTYPE", "wrong medium type"},
+	{125, "ECANCELED", "operation canceled"},
+	{126, "ENOKEY", "required key not available"},
+	{127, "EKEYEXPIRED", "key has expired"},
+	{128, "EKEYREVOKED", "key has been revoked"},
+	{129, "EKEYREJECTED", "key was rejected by service"},
+	{130, "EOWNERDEAD", "owner died"},
+	{131, "ENOTRECOVERABLE", "state not recoverable"},
+	{132, "ERFKILL", "operation not possible due to RF-kill"},
+	{133, "EHWPOISON", "memory page has hardware error"},
+}
+
+// Signal table
+var signalList = [...]struct {
+	num  syscall.Signal
+	name string
+	desc string
+}{
+	{1, "SIGHUP", "hangup"},
+	{2, "SIGINT", "interrupt"},
+	{3, "SIGQUIT", "quit"},
+	{4, "SIGILL", "illegal instruction"},
+	{5, "SIGTRAP", "trace/breakpoint trap"},
+	{6, "SIGABRT", "aborted"},
+	{7, "SIGBUS", "bus error"},
+	{8, "SIGFPE", "floating point exception"},
+	{9, "SIGKILL", "killed"},
+	{10, "SIGUSR1", "user defined signal 1"},
+	{11, "SIGSEGV", "segmentation fault"},
+	{12, "SIGUSR2", "user defined signal 2"},
+	{13, "SIGPIPE", "broken pipe"},
+	{14, "SIGALRM", "alarm clock"},
+	{15, "SIGTERM", "terminated"},
+	{16, "SIGSTKFLT", "stack fault"},
+	{17, "SIGCHLD", "child exited"},
+	{18, "SIGCONT", "continued"},
+	{19, "SIGSTOP", "stopped (signal)"},
+	{20, "SIGTSTP", "stopped"},
+	{21, "SIGTTIN", "stopped (tty input)"},
+	{22, "SIGTTOU", "stopped (tty output)"},
+	{23, "SIGURG", "urgent I/O condition"},
+	{24, "SIGXCPU", "CPU time limit exceeded"},
+	{25, "SIGXFSZ", "file size limit exceeded"},
+	{26, "SIGVTALRM", "virtual timer expired"},
+	{27, "SIGPROF", "profiling timer expired"},
+	{28, "SIGWINCH", "window changed"},
+	{29, "SIGIO", "I/O possible"},
+	{30, "SIGPWR", "power failure"},
+	{31, "SIGSYS", "bad system call"},
+}
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go
index 941e20d..f3a41d6 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64 && linux
 // +build ppc64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x300
 	CSIZE                            = 0x300
 	CSTOPB                           = 0x400
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x40
 	ECHOE                            = 0x2
 	ECHOK                            = 0x4
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x4000
 	ICANON                           = 0x100
 	IEXTEN                           = 0x400
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x2000
 	MCL_FUTURE                       = 0x4000
 	MCL_ONFAULT                      = 0x8000
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x40
 	NL2                              = 0x200
 	NL3                              = 0x300
@@ -128,6 +147,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x4
 	ONLCR                            = 0x2
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -165,6 +188,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -192,6 +216,7 @@
 	PPPIOCSPASS                      = 0x80107447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PROT_SAO                         = 0x10
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
@@ -327,6 +352,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -349,6 +375,7 @@
 	SO_PEERCRED                      = 0x15
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -543,6 +570,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4000
 	XTABS                            = 0xc00
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go
index 63d3bc5..6a5a555 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_ppc64le.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64le && linux
 // +build ppc64le,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x300
 	CSIZE                            = 0x300
 	CSTOPB                           = 0x400
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x40
 	ECHOE                            = 0x2
 	ECHOK                            = 0x4
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x4000
 	ICANON                           = 0x100
 	IEXTEN                           = 0x400
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x2000
 	MCL_FUTURE                       = 0x4000
 	MCL_ONFAULT                      = 0x8000
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x40
 	NL2                              = 0x200
 	NL3                              = 0x300
@@ -128,6 +147,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x4
 	ONLCR                            = 0x2
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -165,6 +188,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -192,6 +216,7 @@
 	PPPIOCSPASS                      = 0x80107447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PROT_SAO                         = 0x10
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
@@ -327,6 +352,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -349,6 +375,7 @@
 	SO_PEERCRED                      = 0x15
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -543,6 +570,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4000
 	XTABS                            = 0xc00
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go
index 490bee1..a4da67e 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_riscv64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build riscv64 && linux
 // +build riscv64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x40107447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	RLIMIT_AS                        = 0x9
@@ -256,6 +281,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -278,6 +304,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -468,6 +495,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go b/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go
index 467b821..a7028e0 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_s390x.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include -fsigned-char
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build s390x && linux
 // +build s390x,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char /build/_const.go
 
 package unix
 
@@ -59,6 +60,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x81484d11
+	ECCGETSTATS                      = 0x80104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -92,6 +95,9 @@
 	F_SETOWN                         = 0x8
 	F_UNLCK                          = 0x2
 	F_WRLCK                          = 0x1
+	HIDIOCGRAWINFO                   = 0x80084803
+	HIDIOCGRDESC                     = 0x90044802
+	HIDIOCGRDESCSIZE                 = 0x80044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -117,6 +123,19 @@
 	MCL_CURRENT                      = 0x1
 	MCL_FUTURE                       = 0x2
 	MCL_ONFAULT                      = 0x4
+	MEMERASE                         = 0x40084d02
+	MEMERASE64                       = 0x40104d14
+	MEMGETBADBLOCK                   = 0x40084d0b
+	MEMGETINFO                       = 0x80204d01
+	MEMGETOOBSEL                     = 0x80c84d0a
+	MEMGETREGIONCOUNT                = 0x80044d07
+	MEMISLOCKED                      = 0x80084d17
+	MEMLOCK                          = 0x40084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x40084d0c
+	MEMUNLOCK                        = 0x40084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x4d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -126,6 +145,10 @@
 	NS_GET_USERNS                    = 0xb701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x40044d0e
+	OTPGETREGIONINFO                 = 0x400c4d0f
+	OTPLOCK                          = 0x800c4d10
+	OTPSELECT                        = 0x80044d0d
 	O_APPEND                         = 0x400
 	O_ASYNC                          = 0x2000
 	O_CLOEXEC                        = 0x80000
@@ -163,6 +186,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x2405
 	PPPIOCATTACH                     = 0x4004743d
 	PPPIOCATTCHAN                    = 0x40047438
+	PPPIOCBRIDGECHAN                 = 0x40047435
 	PPPIOCCONNECT                    = 0x4004743a
 	PPPIOCDETACH                     = 0x4004743c
 	PPPIOCDISCONN                    = 0x7439
@@ -190,6 +214,7 @@
 	PPPIOCSPASS                      = 0x40107447
 	PPPIOCSRASYNCMAP                 = 0x40047454
 	PPPIOCSXASYNCMAP                 = 0x4020744f
+	PPPIOCUNBRIDGECHAN               = 0x7434
 	PPPIOCXFERUNIT                   = 0x744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	PTRACE_DISABLE_TE                = 0x5010
@@ -206,6 +231,8 @@
 	PTRACE_POKE_SYSTEM_CALL          = 0x5008
 	PTRACE_PROT                      = 0x15
 	PTRACE_SINGLEBLOCK               = 0xc
+	PTRACE_SYSEMU                    = 0x1f
+	PTRACE_SYSEMU_SINGLESTEP         = 0x20
 	PTRACE_TE_ABORT_RAND             = 0x5011
 	PT_ACR0                          = 0x90
 	PT_ACR1                          = 0x94
@@ -329,6 +356,7 @@
 	SO_BROADCAST                     = 0x6
 	SO_BSDCOMPAT                     = 0xe
 	SO_BUSY_POLL                     = 0x2e
+	SO_BUSY_POLL_BUDGET              = 0x46
 	SO_CNX_ADVICE                    = 0x35
 	SO_COOKIE                        = 0x39
 	SO_DETACH_REUSEPORT_BPF          = 0x44
@@ -351,6 +379,7 @@
 	SO_PEERCRED                      = 0x11
 	SO_PEERGROUPS                    = 0x3b
 	SO_PEERSEC                       = 0x1f
+	SO_PREFER_BUSY_POLL              = 0x45
 	SO_PROTOCOL                      = 0x26
 	SO_RCVBUF                        = 0x8
 	SO_RCVBUFFORCE                   = 0x21
@@ -541,6 +570,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x80804804
+	_HIDIOCGRAWPHYS                  = 0x80404805
+	_HIDIOCGRAWUNIQ                  = 0x80404808
 )
 
 // Errors
diff --git a/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go b/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go
index 79fbafb..ed3b328 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_linux_sparc64.go
@@ -1,10 +1,11 @@
 // mkerrors.sh -Wall -Werror -static -I/tmp/include
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build sparc64 && linux
 // +build sparc64,linux
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include _const.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/_const.go
 
 package unix
 
@@ -62,6 +63,8 @@
 	CS8                              = 0x30
 	CSIZE                            = 0x30
 	CSTOPB                           = 0x40
+	ECCGETLAYOUT                     = 0x41484d11
+	ECCGETSTATS                      = 0x40104d12
 	ECHOCTL                          = 0x200
 	ECHOE                            = 0x10
 	ECHOK                            = 0x20
@@ -96,6 +99,9 @@
 	F_SETOWN                         = 0x6
 	F_UNLCK                          = 0x3
 	F_WRLCK                          = 0x2
+	HIDIOCGRAWINFO                   = 0x40084803
+	HIDIOCGRDESC                     = 0x50044802
+	HIDIOCGRDESCSIZE                 = 0x40044801
 	HUPCL                            = 0x400
 	ICANON                           = 0x2
 	IEXTEN                           = 0x8000
@@ -122,6 +128,19 @@
 	MCL_CURRENT                      = 0x2000
 	MCL_FUTURE                       = 0x4000
 	MCL_ONFAULT                      = 0x8000
+	MEMERASE                         = 0x80084d02
+	MEMERASE64                       = 0x80104d14
+	MEMGETBADBLOCK                   = 0x80084d0b
+	MEMGETINFO                       = 0x40204d01
+	MEMGETOOBSEL                     = 0x40c84d0a
+	MEMGETREGIONCOUNT                = 0x40044d07
+	MEMISLOCKED                      = 0x40084d17
+	MEMLOCK                          = 0x80084d05
+	MEMREADOOB                       = 0xc0104d04
+	MEMSETBADBLOCK                   = 0x80084d0c
+	MEMUNLOCK                        = 0x80084d06
+	MEMWRITEOOB                      = 0xc0104d03
+	MTDFILEMODE                      = 0x20004d13
 	NFDBITS                          = 0x40
 	NLDLY                            = 0x100
 	NOFLSH                           = 0x80
@@ -131,6 +150,10 @@
 	NS_GET_USERNS                    = 0x2000b701
 	OLCUC                            = 0x2
 	ONLCR                            = 0x4
+	OTPGETREGIONCOUNT                = 0x80044d0e
+	OTPGETREGIONINFO                 = 0x800c4d0f
+	OTPLOCK                          = 0x400c4d10
+	OTPSELECT                        = 0x40044d0d
 	O_APPEND                         = 0x8
 	O_ASYNC                          = 0x40
 	O_CLOEXEC                        = 0x400000
@@ -168,6 +191,7 @@
 	PERF_EVENT_IOC_SET_OUTPUT        = 0x20002405
 	PPPIOCATTACH                     = 0x8004743d
 	PPPIOCATTCHAN                    = 0x80047438
+	PPPIOCBRIDGECHAN                 = 0x80047435
 	PPPIOCCONNECT                    = 0x8004743a
 	PPPIOCDETACH                     = 0x8004743c
 	PPPIOCDISCONN                    = 0x20007439
@@ -195,6 +219,7 @@
 	PPPIOCSPASS                      = 0x80107447
 	PPPIOCSRASYNCMAP                 = 0x80047454
 	PPPIOCSXASYNCMAP                 = 0x8020744f
+	PPPIOCUNBRIDGECHAN               = 0x20007434
 	PPPIOCXFERUNIT                   = 0x2000744e
 	PR_SET_PTRACER_ANY               = 0xffffffffffffffff
 	PTRACE_GETFPAREGS                = 0x14
@@ -322,6 +347,7 @@
 	SO_BROADCAST                     = 0x20
 	SO_BSDCOMPAT                     = 0x400
 	SO_BUSY_POLL                     = 0x30
+	SO_BUSY_POLL_BUDGET              = 0x49
 	SO_CNX_ADVICE                    = 0x37
 	SO_COOKIE                        = 0x3b
 	SO_DETACH_REUSEPORT_BPF          = 0x47
@@ -344,6 +370,7 @@
 	SO_PEERCRED                      = 0x40
 	SO_PEERGROUPS                    = 0x3d
 	SO_PEERSEC                       = 0x1e
+	SO_PREFER_BUSY_POLL              = 0x48
 	SO_PROTOCOL                      = 0x1028
 	SO_RCVBUF                        = 0x1002
 	SO_RCVBUFFORCE                   = 0x100b
@@ -531,6 +558,9 @@
 	WORDSIZE                         = 0x40
 	XCASE                            = 0x4
 	XTABS                            = 0x1800
+	_HIDIOCGRAWNAME                  = 0x40804804
+	_HIDIOCGRAWPHYS                  = 0x40404805
+	_HIDIOCGRAWUNIQ                  = 0x40404808
 	__TIOCFLUSH                      = 0x80047410
 )
 
diff --git a/vendor/golang.org/x/sys/unix/zerrors_netbsd_386.go b/vendor/golang.org/x/sys/unix/zerrors_netbsd_386.go
index 20f3a57..72f7420 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_netbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_netbsd_386.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m32
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && netbsd
 // +build 386,netbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_netbsd_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_netbsd_amd64.go
index 90b8fcd..8d4eb0c 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_netbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_netbsd_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && netbsd
 // +build amd64,netbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm.go b/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm.go
index c5c0399..9eef974 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -marm
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && netbsd
 // +build arm,netbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm64.go
index 14dd3c1..3b62ba1 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_netbsd_arm64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && netbsd
 // +build arm64,netbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_openbsd_386.go b/vendor/golang.org/x/sys/unix/zerrors_openbsd_386.go
index c865a10..593cc0f 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_openbsd_386.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m32
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && openbsd
 // +build 386,openbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_openbsd_amd64.go
index 9db6b2f..25cb609 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_openbsd_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && openbsd
 // +build amd64,openbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm.go b/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm.go
index 7072526..a4e4c22 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm.go
@@ -1,6 +1,7 @@
 // mkerrors.sh
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && openbsd
 // +build arm,openbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm64.go
index ac5efbe..90de7df 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_openbsd_arm64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && openbsd
 // +build arm64,openbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/zerrors_openbsd_mips64.go
index a74639a..f1154ff 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_openbsd_mips64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && openbsd
 // +build mips64,openbsd
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
diff --git a/vendor/golang.org/x/sys/unix/zerrors_solaris_amd64.go b/vendor/golang.org/x/sys/unix/zerrors_solaris_amd64.go
index 46e054c..1afee6a 100644
--- a/vendor/golang.org/x/sys/unix/zerrors_solaris_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zerrors_solaris_amd64.go
@@ -1,6 +1,7 @@
 // mkerrors.sh -m64
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && solaris
 // +build amd64,solaris
 
 // Code generated by cmd/cgo -godefs; DO NOT EDIT.
@@ -192,6 +193,12 @@
 	CSTOPB                        = 0x40
 	CSUSP                         = 0x1a
 	CSWTCH                        = 0x1a
+	DIOC                          = 0x6400
+	DIOCGETB                      = 0x6402
+	DIOCGETC                      = 0x6401
+	DIOCGETP                      = 0x6408
+	DIOCSETE                      = 0x6403
+	DIOCSETP                      = 0x6409
 	DLT_AIRONET_HEADER            = 0x78
 	DLT_APPLE_IP_OVER_IEEE1394    = 0x8a
 	DLT_ARCNET                    = 0x7
@@ -290,6 +297,7 @@
 	FF0                           = 0x0
 	FF1                           = 0x8000
 	FFDLY                         = 0x8000
+	FIORDCHK                      = 0x6603
 	FLUSHALL                      = 0x1
 	FLUSHDATA                     = 0x0
 	FLUSHO                        = 0x2000
@@ -358,6 +366,7 @@
 	HUPCL                         = 0x400
 	IBSHIFT                       = 0x10
 	ICANON                        = 0x2
+	ICMP6_FILTER                  = 0x1
 	ICRNL                         = 0x100
 	IEXTEN                        = 0x8000
 	IFF_ADDRCONF                  = 0x80000
@@ -604,6 +613,7 @@
 	IP_RECVPKTINFO                = 0x1a
 	IP_RECVRETOPTS                = 0x6
 	IP_RECVSLLA                   = 0xa
+	IP_RECVTOS                    = 0xc
 	IP_RECVTTL                    = 0xb
 	IP_RETOPTS                    = 0x8
 	IP_REUSEADDR                  = 0x104
@@ -645,6 +655,14 @@
 	MAP_SHARED                    = 0x1
 	MAP_TEXT                      = 0x400
 	MAP_TYPE                      = 0xf
+	MCAST_BLOCK_SOURCE            = 0x2b
+	MCAST_EXCLUDE                 = 0x2
+	MCAST_INCLUDE                 = 0x1
+	MCAST_JOIN_GROUP              = 0x29
+	MCAST_JOIN_SOURCE_GROUP       = 0x2d
+	MCAST_LEAVE_GROUP             = 0x2a
+	MCAST_LEAVE_SOURCE_GROUP      = 0x2e
+	MCAST_UNBLOCK_SOURCE          = 0x2c
 	MCL_CURRENT                   = 0x1
 	MCL_FUTURE                    = 0x2
 	MSG_CTRUNC                    = 0x10
@@ -653,6 +671,7 @@
 	MSG_DUPCTRL                   = 0x800
 	MSG_EOR                       = 0x8
 	MSG_MAXIOVLEN                 = 0x10
+	MSG_NOSIGNAL                  = 0x200
 	MSG_NOTIFICATION              = 0x100
 	MSG_OOB                       = 0x1
 	MSG_PEEK                      = 0x2
@@ -687,6 +706,8 @@
 	O_APPEND                      = 0x8
 	O_CLOEXEC                     = 0x800000
 	O_CREAT                       = 0x100
+	O_DIRECT                      = 0x2000000
+	O_DIRECTORY                   = 0x1000000
 	O_DSYNC                       = 0x40
 	O_EXCL                        = 0x400
 	O_EXEC                        = 0x400000
@@ -725,7 +746,7 @@
 	RLIMIT_FSIZE                  = 0x1
 	RLIMIT_NOFILE                 = 0x5
 	RLIMIT_STACK                  = 0x3
-	RLIM_INFINITY                 = -0x3
+	RLIM_INFINITY                 = 0xfffffffffffffffd
 	RTAX_AUTHOR                   = 0x6
 	RTAX_BRD                      = 0x7
 	RTAX_DST                      = 0x0
@@ -1047,6 +1068,7 @@
 	TCOON                         = 0x1
 	TCP_ABORT_THRESHOLD           = 0x11
 	TCP_ANONPRIVBIND              = 0x20
+	TCP_CONGESTION                = 0x25
 	TCP_CONN_ABORT_THRESHOLD      = 0x13
 	TCP_CONN_NOTIFY_THRESHOLD     = 0x12
 	TCP_CORK                      = 0x18
@@ -1076,6 +1098,8 @@
 	TCSETSF                       = 0x5410
 	TCSETSW                       = 0x540f
 	TCXONC                        = 0x5406
+	TIMER_ABSTIME                 = 0x1
+	TIMER_RELTIME                 = 0x0
 	TIOC                          = 0x5400
 	TIOCCBRK                      = 0x747a
 	TIOCCDTR                      = 0x7478
diff --git a/vendor/golang.org/x/sys/unix/zerrors_zos_s390x.go b/vendor/golang.org/x/sys/unix/zerrors_zos_s390x.go
new file mode 100644
index 0000000..fc7d050
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zerrors_zos_s390x.go
@@ -0,0 +1,860 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+// Hand edited based on zerrors_linux_s390x.go
+// TODO: auto-generate.
+
+package unix
+
+const (
+	BRKINT                          = 0x0001
+	CLOCK_MONOTONIC                 = 0x1
+	CLOCK_PROCESS_CPUTIME_ID        = 0x2
+	CLOCK_REALTIME                  = 0x0
+	CLOCK_THREAD_CPUTIME_ID         = 0x3
+	CS8                             = 0x0030
+	CSIZE                           = 0x0030
+	ECHO                            = 0x00000008
+	ECHONL                          = 0x00000001
+	FD_CLOEXEC                      = 0x01
+	FD_CLOFORK                      = 0x02
+	FNDELAY                         = 0x04
+	F_CLOSFD                        = 9
+	F_CONTROL_CVT                   = 13
+	F_DUPFD                         = 0
+	F_DUPFD2                        = 8
+	F_GETFD                         = 1
+	F_GETFL                         = 259
+	F_GETLK                         = 5
+	F_GETOWN                        = 10
+	F_OK                            = 0x0
+	F_RDLCK                         = 1
+	F_SETFD                         = 2
+	F_SETFL                         = 4
+	F_SETLK                         = 6
+	F_SETLKW                        = 7
+	F_SETOWN                        = 11
+	F_SETTAG                        = 12
+	F_UNLCK                         = 3
+	F_WRLCK                         = 2
+	FSTYPE_ZFS                      = 0xe9 //"Z"
+	FSTYPE_HFS                      = 0xc8 //"H"
+	FSTYPE_NFS                      = 0xd5 //"N"
+	FSTYPE_TFS                      = 0xe3 //"T"
+	FSTYPE_AUTOMOUNT                = 0xc1 //"A"
+	IP6F_MORE_FRAG                  = 0x0001
+	IP6F_OFF_MASK                   = 0xfff8
+	IP6F_RESERVED_MASK              = 0x0006
+	IP6OPT_JUMBO                    = 0xc2
+	IP6OPT_JUMBO_LEN                = 6
+	IP6OPT_MUTABLE                  = 0x20
+	IP6OPT_NSAP_ADDR                = 0xc3
+	IP6OPT_PAD1                     = 0x00
+	IP6OPT_PADN                     = 0x01
+	IP6OPT_ROUTER_ALERT             = 0x05
+	IP6OPT_TUNNEL_LIMIT             = 0x04
+	IP6OPT_TYPE_DISCARD             = 0x40
+	IP6OPT_TYPE_FORCEICMP           = 0x80
+	IP6OPT_TYPE_ICMP                = 0xc0
+	IP6OPT_TYPE_SKIP                = 0x00
+	IP6_ALERT_AN                    = 0x0002
+	IP6_ALERT_MLD                   = 0x0000
+	IP6_ALERT_RSVP                  = 0x0001
+	IPPORT_RESERVED                 = 1024
+	IPPORT_USERRESERVED             = 5000
+	IPPROTO_AH                      = 51
+	SOL_AH                          = 51
+	IPPROTO_DSTOPTS                 = 60
+	SOL_DSTOPTS                     = 60
+	IPPROTO_EGP                     = 8
+	SOL_EGP                         = 8
+	IPPROTO_ESP                     = 50
+	SOL_ESP                         = 50
+	IPPROTO_FRAGMENT                = 44
+	SOL_FRAGMENT                    = 44
+	IPPROTO_GGP                     = 2
+	SOL_GGP                         = 2
+	IPPROTO_HOPOPTS                 = 0
+	SOL_HOPOPTS                     = 0
+	IPPROTO_ICMP                    = 1
+	SOL_ICMP                        = 1
+	IPPROTO_ICMPV6                  = 58
+	SOL_ICMPV6                      = 58
+	IPPROTO_IDP                     = 22
+	SOL_IDP                         = 22
+	IPPROTO_IP                      = 0
+	SOL_IP                          = 0
+	IPPROTO_IPV6                    = 41
+	SOL_IPV6                        = 41
+	IPPROTO_MAX                     = 256
+	SOL_MAX                         = 256
+	IPPROTO_NONE                    = 59
+	SOL_NONE                        = 59
+	IPPROTO_PUP                     = 12
+	SOL_PUP                         = 12
+	IPPROTO_RAW                     = 255
+	SOL_RAW                         = 255
+	IPPROTO_ROUTING                 = 43
+	SOL_ROUTING                     = 43
+	IPPROTO_TCP                     = 6
+	SOL_TCP                         = 6
+	IPPROTO_UDP                     = 17
+	SOL_UDP                         = 17
+	IPV6_ADDR_PREFERENCES           = 32
+	IPV6_CHECKSUM                   = 19
+	IPV6_DONTFRAG                   = 29
+	IPV6_DSTOPTS                    = 23
+	IPV6_HOPLIMIT                   = 11
+	IPV6_HOPOPTS                    = 22
+	IPV6_JOIN_GROUP                 = 5
+	IPV6_LEAVE_GROUP                = 6
+	IPV6_MULTICAST_HOPS             = 9
+	IPV6_MULTICAST_IF               = 7
+	IPV6_MULTICAST_LOOP             = 4
+	IPV6_NEXTHOP                    = 20
+	IPV6_PATHMTU                    = 12
+	IPV6_PKTINFO                    = 13
+	IPV6_PREFER_SRC_CGA             = 0x10
+	IPV6_PREFER_SRC_COA             = 0x02
+	IPV6_PREFER_SRC_HOME            = 0x01
+	IPV6_PREFER_SRC_NONCGA          = 0x20
+	IPV6_PREFER_SRC_PUBLIC          = 0x08
+	IPV6_PREFER_SRC_TMP             = 0x04
+	IPV6_RECVDSTOPTS                = 28
+	IPV6_RECVHOPLIMIT               = 14
+	IPV6_RECVHOPOPTS                = 26
+	IPV6_RECVPATHMTU                = 16
+	IPV6_RECVPKTINFO                = 15
+	IPV6_RECVRTHDR                  = 25
+	IPV6_RECVTCLASS                 = 31
+	IPV6_RTHDR                      = 21
+	IPV6_RTHDRDSTOPTS               = 24
+	IPV6_RTHDR_TYPE_0               = 0
+	IPV6_TCLASS                     = 30
+	IPV6_UNICAST_HOPS               = 3
+	IPV6_USE_MIN_MTU                = 18
+	IPV6_V6ONLY                     = 10
+	IP_ADD_MEMBERSHIP               = 5
+	IP_ADD_SOURCE_MEMBERSHIP        = 12
+	IP_BLOCK_SOURCE                 = 10
+	IP_DEFAULT_MULTICAST_LOOP       = 1
+	IP_DEFAULT_MULTICAST_TTL        = 1
+	IP_DROP_MEMBERSHIP              = 6
+	IP_DROP_SOURCE_MEMBERSHIP       = 13
+	IP_MAX_MEMBERSHIPS              = 20
+	IP_MULTICAST_IF                 = 7
+	IP_MULTICAST_LOOP               = 4
+	IP_MULTICAST_TTL                = 3
+	IP_OPTIONS                      = 1
+	IP_PKTINFO                      = 101
+	IP_RECVPKTINFO                  = 102
+	IP_TOS                          = 2
+	IP_TTL                          = 3
+	IP_UNBLOCK_SOURCE               = 11
+	ICANON                          = 0x0010
+	ICMP6_FILTER                    = 0x26
+	ICRNL                           = 0x0002
+	IEXTEN                          = 0x0020
+	IGNBRK                          = 0x0004
+	IGNCR                           = 0x0008
+	INLCR                           = 0x0020
+	ISIG                            = 0x0040
+	ISTRIP                          = 0x0080
+	IXON                            = 0x0200
+	IXOFF                           = 0x0100
+	LOCK_SH                         = 0x1 // Not exist on zOS
+	LOCK_EX                         = 0x2 // Not exist on zOS
+	LOCK_NB                         = 0x4 // Not exist on zOS
+	LOCK_UN                         = 0x8 // Not exist on zOS
+	POLLIN                          = 0x0003
+	POLLOUT                         = 0x0004
+	POLLPRI                         = 0x0010
+	POLLERR                         = 0x0020
+	POLLHUP                         = 0x0040
+	POLLNVAL                        = 0x0080
+	PROT_READ                       = 0x1 // mmap - page can be read
+	PROT_WRITE                      = 0x2 // page can be written
+	PROT_NONE                       = 0x4 // can't be accessed
+	PROT_EXEC                       = 0x8 // can be executed
+	MAP_PRIVATE                     = 0x1 // changes are private
+	MAP_SHARED                      = 0x2 // changes are shared
+	MAP_FIXED                       = 0x4 // place exactly
+	MCAST_JOIN_GROUP                = 40
+	MCAST_LEAVE_GROUP               = 41
+	MCAST_JOIN_SOURCE_GROUP         = 42
+	MCAST_LEAVE_SOURCE_GROUP        = 43
+	MCAST_BLOCK_SOURCE              = 44
+	MCAST_UNBLOCK_SOURCE            = 45
+	MS_SYNC                         = 0x1 // msync - synchronous writes
+	MS_ASYNC                        = 0x2 // asynchronous writes
+	MS_INVALIDATE                   = 0x4 // invalidate mappings
+	MTM_RDONLY                      = 0x80000000
+	MTM_RDWR                        = 0x40000000
+	MTM_UMOUNT                      = 0x10000000
+	MTM_IMMED                       = 0x08000000
+	MTM_FORCE                       = 0x04000000
+	MTM_DRAIN                       = 0x02000000
+	MTM_RESET                       = 0x01000000
+	MTM_SAMEMODE                    = 0x00100000
+	MTM_UNQSEFORCE                  = 0x00040000
+	MTM_NOSUID                      = 0x00000400
+	MTM_SYNCHONLY                   = 0x00000200
+	MTM_REMOUNT                     = 0x00000100
+	MTM_NOSECURITY                  = 0x00000080
+	NFDBITS                         = 0x20
+	O_ACCMODE                       = 0x03
+	O_APPEND                        = 0x08
+	O_ASYNCSIG                      = 0x0200
+	O_CREAT                         = 0x80
+	O_EXCL                          = 0x40
+	O_GETFL                         = 0x0F
+	O_LARGEFILE                     = 0x0400
+	O_NONBLOCK                      = 0x04
+	O_RDONLY                        = 0x02
+	O_RDWR                          = 0x03
+	O_SYNC                          = 0x0100
+	O_TRUNC                         = 0x10
+	O_WRONLY                        = 0x01
+	O_NOCTTY                        = 0x20
+	OPOST                           = 0x0001
+	ONLCR                           = 0x0004
+	PARENB                          = 0x0200
+	PARMRK                          = 0x0400
+	QUERYCVT                        = 3
+	RUSAGE_CHILDREN                 = -0x1
+	RUSAGE_SELF                     = 0x0 // RUSAGE_THREAD unsupported on z/OS
+	SEEK_CUR                        = 1
+	SEEK_END                        = 2
+	SEEK_SET                        = 0
+	SETAUTOCVTALL                   = 5
+	SETAUTOCVTON                    = 2
+	SETCVTALL                       = 4
+	SETCVTOFF                       = 0
+	SETCVTON                        = 1
+	AF_APPLETALK                    = 16
+	AF_CCITT                        = 10
+	AF_CHAOS                        = 5
+	AF_DATAKIT                      = 9
+	AF_DLI                          = 13
+	AF_ECMA                         = 8
+	AF_HYLINK                       = 15
+	AF_IMPLINK                      = 3
+	AF_INET                         = 2
+	AF_INET6                        = 19
+	AF_INTF                         = 20
+	AF_IUCV                         = 17
+	AF_LAT                          = 14
+	AF_LINK                         = 18
+	AF_MAX                          = 30
+	AF_NBS                          = 7
+	AF_NDD                          = 23
+	AF_NETWARE                      = 22
+	AF_NS                           = 6
+	AF_PUP                          = 4
+	AF_RIF                          = 21
+	AF_ROUTE                        = 20
+	AF_SNA                          = 11
+	AF_UNIX                         = 1
+	AF_UNSPEC                       = 0
+	IBMTCP_IMAGE                    = 1
+	MSG_ACK_EXPECTED                = 0x10
+	MSG_ACK_GEN                     = 0x40
+	MSG_ACK_TIMEOUT                 = 0x20
+	MSG_CONNTERM                    = 0x80
+	MSG_CTRUNC                      = 0x20
+	MSG_DONTROUTE                   = 0x4
+	MSG_EOF                         = 0x8000
+	MSG_EOR                         = 0x8
+	MSG_MAXIOVLEN                   = 16
+	MSG_NONBLOCK                    = 0x4000
+	MSG_OOB                         = 0x1
+	MSG_PEEK                        = 0x2
+	MSG_TRUNC                       = 0x10
+	MSG_WAITALL                     = 0x40
+	PRIO_PROCESS                    = 1
+	PRIO_PGRP                       = 2
+	PRIO_USER                       = 3
+	RLIMIT_CPU                      = 0
+	RLIMIT_FSIZE                    = 1
+	RLIMIT_DATA                     = 2
+	RLIMIT_STACK                    = 3
+	RLIMIT_CORE                     = 4
+	RLIMIT_AS                       = 5
+	RLIMIT_NOFILE                   = 6
+	RLIMIT_MEMLIMIT                 = 7
+	RLIM_INFINITY                   = 2147483647
+	SCM_RIGHTS                      = 0x01
+	SF_CLOSE                        = 0x00000002
+	SF_REUSE                        = 0x00000001
+	SHUT_RD                         = 0
+	SHUT_RDWR                       = 2
+	SHUT_WR                         = 1
+	SOCK_CONN_DGRAM                 = 6
+	SOCK_DGRAM                      = 2
+	SOCK_RAW                        = 3
+	SOCK_RDM                        = 4
+	SOCK_SEQPACKET                  = 5
+	SOCK_STREAM                     = 1
+	SOL_SOCKET                      = 0xffff
+	SOMAXCONN                       = 10
+	SO_ACCEPTCONN                   = 0x0002
+	SO_ACCEPTECONNABORTED           = 0x0006
+	SO_ACKNOW                       = 0x7700
+	SO_BROADCAST                    = 0x0020
+	SO_BULKMODE                     = 0x8000
+	SO_CKSUMRECV                    = 0x0800
+	SO_CLOSE                        = 0x01
+	SO_CLUSTERCONNTYPE              = 0x00004001
+	SO_CLUSTERCONNTYPE_INTERNAL     = 8
+	SO_CLUSTERCONNTYPE_NOCONN       = 0
+	SO_CLUSTERCONNTYPE_NONE         = 1
+	SO_CLUSTERCONNTYPE_SAME_CLUSTER = 2
+	SO_CLUSTERCONNTYPE_SAME_IMAGE   = 4
+	SO_DEBUG                        = 0x0001
+	SO_DONTROUTE                    = 0x0010
+	SO_ERROR                        = 0x1007
+	SO_IGNOREINCOMINGPUSH           = 0x1
+	SO_IGNORESOURCEVIPA             = 0x0002
+	SO_KEEPALIVE                    = 0x0008
+	SO_LINGER                       = 0x0080
+	SO_NONBLOCKLOCAL                = 0x8001
+	SO_NOREUSEADDR                  = 0x1000
+	SO_OOBINLINE                    = 0x0100
+	SO_OPTACK                       = 0x8004
+	SO_OPTMSS                       = 0x8003
+	SO_RCVBUF                       = 0x1002
+	SO_RCVLOWAT                     = 0x1004
+	SO_RCVTIMEO                     = 0x1006
+	SO_REUSEADDR                    = 0x0004
+	SO_REUSEPORT                    = 0x0200
+	SO_SECINFO                      = 0x00004002
+	SO_SET                          = 0x0200
+	SO_SNDBUF                       = 0x1001
+	SO_SNDLOWAT                     = 0x1003
+	SO_SNDTIMEO                     = 0x1005
+	SO_TYPE                         = 0x1008
+	SO_UNSET                        = 0x0400
+	SO_USELOOPBACK                  = 0x0040
+	SO_USE_IFBUFS                   = 0x0400
+	S_ISUID                         = 0x0800
+	S_ISGID                         = 0x0400
+	S_ISVTX                         = 0x0200
+	S_IRUSR                         = 0x0100
+	S_IWUSR                         = 0x0080
+	S_IXUSR                         = 0x0040
+	S_IRWXU                         = 0x01C0
+	S_IRGRP                         = 0x0020
+	S_IWGRP                         = 0x0010
+	S_IXGRP                         = 0x0008
+	S_IRWXG                         = 0x0038
+	S_IROTH                         = 0x0004
+	S_IWOTH                         = 0x0002
+	S_IXOTH                         = 0x0001
+	S_IRWXO                         = 0x0007
+	S_IREAD                         = S_IRUSR
+	S_IWRITE                        = S_IWUSR
+	S_IEXEC                         = S_IXUSR
+	S_IFDIR                         = 0x01000000
+	S_IFCHR                         = 0x02000000
+	S_IFREG                         = 0x03000000
+	S_IFFIFO                        = 0x04000000
+	S_IFIFO                         = 0x04000000
+	S_IFLNK                         = 0x05000000
+	S_IFBLK                         = 0x06000000
+	S_IFSOCK                        = 0x07000000
+	S_IFVMEXTL                      = 0xFE000000
+	S_IFVMEXTL_EXEC                 = 0x00010000
+	S_IFVMEXTL_DATA                 = 0x00020000
+	S_IFVMEXTL_MEL                  = 0x00030000
+	S_IFEXTL                        = 0x00000001
+	S_IFPROGCTL                     = 0x00000002
+	S_IFAPFCTL                      = 0x00000004
+	S_IFNOSHARE                     = 0x00000008
+	S_IFSHARELIB                    = 0x00000010
+	S_IFMT                          = 0xFF000000
+	S_IFMST                         = 0x00FF0000
+	TCP_KEEPALIVE                   = 0x8
+	TCP_NODELAY                     = 0x1
+	TCP_INFO                        = 0xb
+	TCP_USER_TIMEOUT                = 0x1
+	TIOCGWINSZ                      = 0x4008a368
+	TIOCSWINSZ                      = 0x8008a367
+	TIOCSBRK                        = 0x2000a77b
+	TIOCCBRK                        = 0x2000a77a
+	TIOCSTI                         = 0x8001a772
+	TIOCGPGRP                       = 0x4004a777 // _IOR(167, 119, int)
+	TCSANOW                         = 0
+	TCSETS                          = 0 // equivalent to TCSANOW for tcsetattr
+	TCSADRAIN                       = 1
+	TCSETSW                         = 1 // equivalent to TCSADRAIN for tcsetattr
+	TCSAFLUSH                       = 2
+	TCSETSF                         = 2 // equivalent to TCSAFLUSH for tcsetattr
+	TCGETS                          = 3 // not defined in ioctl.h -- zos golang only
+	TCIFLUSH                        = 0
+	TCOFLUSH                        = 1
+	TCIOFLUSH                       = 2
+	TCOOFF                          = 0
+	TCOON                           = 1
+	TCIOFF                          = 2
+	TCION                           = 3
+	TIOCSPGRP                       = 0x8004a776
+	TIOCNOTTY                       = 0x2000a771
+	TIOCEXCL                        = 0x2000a70d
+	TIOCNXCL                        = 0x2000a70e
+	TIOCGETD                        = 0x4004a700
+	TIOCSETD                        = 0x8004a701
+	TIOCPKT                         = 0x8004a770
+	TIOCSTOP                        = 0x2000a76f
+	TIOCSTART                       = 0x2000a76e
+	TIOCUCNTL                       = 0x8004a766
+	TIOCREMOTE                      = 0x8004a769
+	TIOCMGET                        = 0x4004a76a
+	TIOCMSET                        = 0x8004a76d
+	TIOCMBIC                        = 0x8004a76b
+	TIOCMBIS                        = 0x8004a76c
+	VINTR                           = 0
+	VQUIT                           = 1
+	VERASE                          = 2
+	VKILL                           = 3
+	VEOF                            = 4
+	VEOL                            = 5
+	VMIN                            = 6
+	VSTART                          = 7
+	VSTOP                           = 8
+	VSUSP                           = 9
+	VTIME                           = 10
+	WCONTINUED                      = 0x4
+	WNOHANG                         = 0x1
+	WUNTRACED                       = 0x2
+	_BPX_SWAP                       = 1
+	_BPX_NONSWAP                    = 2
+	MCL_CURRENT                     = 1  // for Linux compatibility -- no zos semantics
+	MCL_FUTURE                      = 2  // for Linux compatibility -- no zos semantics
+	MCL_ONFAULT                     = 3  // for Linux compatibility -- no zos semantics
+	MADV_NORMAL                     = 0  // for Linux compatibility -- no zos semantics
+	MADV_RANDOM                     = 1  // for Linux compatibility -- no zos semantics
+	MADV_SEQUENTIAL                 = 2  // for Linux compatibility -- no zos semantics
+	MADV_WILLNEED                   = 3  // for Linux compatibility -- no zos semantics
+	MADV_REMOVE                     = 4  // for Linux compatibility -- no zos semantics
+	MADV_DONTFORK                   = 5  // for Linux compatibility -- no zos semantics
+	MADV_DOFORK                     = 6  // for Linux compatibility -- no zos semantics
+	MADV_HWPOISON                   = 7  // for Linux compatibility -- no zos semantics
+	MADV_MERGEABLE                  = 8  // for Linux compatibility -- no zos semantics
+	MADV_UNMERGEABLE                = 9  // for Linux compatibility -- no zos semantics
+	MADV_SOFT_OFFLINE               = 10 // for Linux compatibility -- no zos semantics
+	MADV_HUGEPAGE                   = 11 // for Linux compatibility -- no zos semantics
+	MADV_NOHUGEPAGE                 = 12 // for Linux compatibility -- no zos semantics
+	MADV_DONTDUMP                   = 13 // for Linux compatibility -- no zos semantics
+	MADV_DODUMP                     = 14 // for Linux compatibility -- no zos semantics
+	MADV_FREE                       = 15 // for Linux compatibility -- no zos semantics
+	MADV_WIPEONFORK                 = 16 // for Linux compatibility -- no zos semantics
+	MADV_KEEPONFORK                 = 17 // for Linux compatibility -- no zos semantics
+	AT_SYMLINK_NOFOLLOW             = 1  // for Unix  compatibility -- no zos semantics
+	AT_FDCWD                        = 2  // for Unix  compatibility -- no zos semantics
+)
+
+const (
+	EDOM               = Errno(1)
+	ERANGE             = Errno(2)
+	EACCES             = Errno(111)
+	EAGAIN             = Errno(112)
+	EBADF              = Errno(113)
+	EBUSY              = Errno(114)
+	ECHILD             = Errno(115)
+	EDEADLK            = Errno(116)
+	EEXIST             = Errno(117)
+	EFAULT             = Errno(118)
+	EFBIG              = Errno(119)
+	EINTR              = Errno(120)
+	EINVAL             = Errno(121)
+	EIO                = Errno(122)
+	EISDIR             = Errno(123)
+	EMFILE             = Errno(124)
+	EMLINK             = Errno(125)
+	ENAMETOOLONG       = Errno(126)
+	ENFILE             = Errno(127)
+	ENODEV             = Errno(128)
+	ENOENT             = Errno(129)
+	ENOEXEC            = Errno(130)
+	ENOLCK             = Errno(131)
+	ENOMEM             = Errno(132)
+	ENOSPC             = Errno(133)
+	ENOSYS             = Errno(134)
+	ENOTDIR            = Errno(135)
+	ENOTEMPTY          = Errno(136)
+	ENOTTY             = Errno(137)
+	ENXIO              = Errno(138)
+	EPERM              = Errno(139)
+	EPIPE              = Errno(140)
+	EROFS              = Errno(141)
+	ESPIPE             = Errno(142)
+	ESRCH              = Errno(143)
+	EXDEV              = Errno(144)
+	E2BIG              = Errno(145)
+	ELOOP              = Errno(146)
+	EILSEQ             = Errno(147)
+	ENODATA            = Errno(148)
+	EOVERFLOW          = Errno(149)
+	EMVSNOTUP          = Errno(150)
+	ECMSSTORAGE        = Errno(151)
+	EMVSDYNALC         = Errno(151)
+	EMVSCVAF           = Errno(152)
+	EMVSCATLG          = Errno(153)
+	ECMSINITIAL        = Errno(156)
+	EMVSINITIAL        = Errno(156)
+	ECMSERR            = Errno(157)
+	EMVSERR            = Errno(157)
+	EMVSPARM           = Errno(158)
+	ECMSPFSFILE        = Errno(159)
+	EMVSPFSFILE        = Errno(159)
+	EMVSBADCHAR        = Errno(160)
+	ECMSPFSPERM        = Errno(162)
+	EMVSPFSPERM        = Errno(162)
+	EMVSSAFEXTRERR     = Errno(163)
+	EMVSSAF2ERR        = Errno(164)
+	EMVSTODNOTSET      = Errno(165)
+	EMVSPATHOPTS       = Errno(166)
+	EMVSNORTL          = Errno(167)
+	EMVSEXPIRE         = Errno(168)
+	EMVSPASSWORD       = Errno(169)
+	EMVSWLMERROR       = Errno(170)
+	EMVSCPLERROR       = Errno(171)
+	EMVSARMERROR       = Errno(172)
+	ELENOFORK          = Errno(200)
+	ELEMSGERR          = Errno(201)
+	EFPMASKINV         = Errno(202)
+	EFPMODEINV         = Errno(203)
+	EBUFLEN            = Errno(227)
+	EEXTLINK           = Errno(228)
+	ENODD              = Errno(229)
+	ECMSESMERR         = Errno(230)
+	ECPERR             = Errno(231)
+	ELEMULTITHREAD     = Errno(232)
+	ELEFENCE           = Errno(244)
+	EBADDATA           = Errno(245)
+	EUNKNOWN           = Errno(246)
+	ENOTSUP            = Errno(247)
+	EBADNAME           = Errno(248)
+	ENOTSAFE           = Errno(249)
+	ELEMULTITHREADFORK = Errno(257)
+	ECUNNOENV          = Errno(258)
+	ECUNNOCONV         = Errno(259)
+	ECUNNOTALIGNED     = Errno(260)
+	ECUNERR            = Errno(262)
+	EIBMBADCALL        = Errno(1000)
+	EIBMBADPARM        = Errno(1001)
+	EIBMSOCKOUTOFRANGE = Errno(1002)
+	EIBMSOCKINUSE      = Errno(1003)
+	EIBMIUCVERR        = Errno(1004)
+	EOFFLOADboxERROR   = Errno(1005)
+	EOFFLOADboxRESTART = Errno(1006)
+	EOFFLOADboxDOWN    = Errno(1007)
+	EIBMCONFLICT       = Errno(1008)
+	EIBMCANCELLED      = Errno(1009)
+	EIBMBADTCPNAME     = Errno(1011)
+	ENOTBLK            = Errno(1100)
+	ETXTBSY            = Errno(1101)
+	EWOULDBLOCK        = Errno(1102)
+	EINPROGRESS        = Errno(1103)
+	EALREADY           = Errno(1104)
+	ENOTSOCK           = Errno(1105)
+	EDESTADDRREQ       = Errno(1106)
+	EMSGSIZE           = Errno(1107)
+	EPROTOTYPE         = Errno(1108)
+	ENOPROTOOPT        = Errno(1109)
+	EPROTONOSUPPORT    = Errno(1110)
+	ESOCKTNOSUPPORT    = Errno(1111)
+	EOPNOTSUPP         = Errno(1112)
+	EPFNOSUPPORT       = Errno(1113)
+	EAFNOSUPPORT       = Errno(1114)
+	EADDRINUSE         = Errno(1115)
+	EADDRNOTAVAIL      = Errno(1116)
+	ENETDOWN           = Errno(1117)
+	ENETUNREACH        = Errno(1118)
+	ENETRESET          = Errno(1119)
+	ECONNABORTED       = Errno(1120)
+	ECONNRESET         = Errno(1121)
+	ENOBUFS            = Errno(1122)
+	EISCONN            = Errno(1123)
+	ENOTCONN           = Errno(1124)
+	ESHUTDOWN          = Errno(1125)
+	ETOOMANYREFS       = Errno(1126)
+	ETIMEDOUT          = Errno(1127)
+	ECONNREFUSED       = Errno(1128)
+	EHOSTDOWN          = Errno(1129)
+	EHOSTUNREACH       = Errno(1130)
+	EPROCLIM           = Errno(1131)
+	EUSERS             = Errno(1132)
+	EDQUOT             = Errno(1133)
+	ESTALE             = Errno(1134)
+	EREMOTE            = Errno(1135)
+	ENOSTR             = Errno(1136)
+	ETIME              = Errno(1137)
+	ENOSR              = Errno(1138)
+	ENOMSG             = Errno(1139)
+	EBADMSG            = Errno(1140)
+	EIDRM              = Errno(1141)
+	ENONET             = Errno(1142)
+	ERREMOTE           = Errno(1143)
+	ENOLINK            = Errno(1144)
+	EADV               = Errno(1145)
+	ESRMNT             = Errno(1146)
+	ECOMM              = Errno(1147)
+	EPROTO             = Errno(1148)
+	EMULTIHOP          = Errno(1149)
+	EDOTDOT            = Errno(1150)
+	EREMCHG            = Errno(1151)
+	ECANCELED          = Errno(1152)
+	EINTRNODATA        = Errno(1159)
+	ENOREUSE           = Errno(1160)
+	ENOMOVE            = Errno(1161)
+)
+
+// Signals
+const (
+	SIGHUP    = Signal(1)
+	SIGINT    = Signal(2)
+	SIGABRT   = Signal(3)
+	SIGILL    = Signal(4)
+	SIGPOLL   = Signal(5)
+	SIGURG    = Signal(6)
+	SIGSTOP   = Signal(7)
+	SIGFPE    = Signal(8)
+	SIGKILL   = Signal(9)
+	SIGBUS    = Signal(10)
+	SIGSEGV   = Signal(11)
+	SIGSYS    = Signal(12)
+	SIGPIPE   = Signal(13)
+	SIGALRM   = Signal(14)
+	SIGTERM   = Signal(15)
+	SIGUSR1   = Signal(16)
+	SIGUSR2   = Signal(17)
+	SIGABND   = Signal(18)
+	SIGCONT   = Signal(19)
+	SIGCHLD   = Signal(20)
+	SIGTTIN   = Signal(21)
+	SIGTTOU   = Signal(22)
+	SIGIO     = Signal(23)
+	SIGQUIT   = Signal(24)
+	SIGTSTP   = Signal(25)
+	SIGTRAP   = Signal(26)
+	SIGIOERR  = Signal(27)
+	SIGWINCH  = Signal(28)
+	SIGXCPU   = Signal(29)
+	SIGXFSZ   = Signal(30)
+	SIGVTALRM = Signal(31)
+	SIGPROF   = Signal(32)
+	SIGDANGER = Signal(33)
+	SIGTHSTOP = Signal(34)
+	SIGTHCONT = Signal(35)
+	SIGTRACE  = Signal(37)
+	SIGDCE    = Signal(38)
+	SIGDUMP   = Signal(39)
+)
+
+// Error table
+var errorList = [...]struct {
+	num  Errno
+	name string
+	desc string
+}{
+	{1, "EDC5001I", "A domain error occurred."},
+	{2, "EDC5002I", "A range error occurred."},
+	{111, "EDC5111I", "Permission denied."},
+	{112, "EDC5112I", "Resource temporarily unavailable."},
+	{113, "EDC5113I", "Bad file descriptor."},
+	{114, "EDC5114I", "Resource busy."},
+	{115, "EDC5115I", "No child processes."},
+	{116, "EDC5116I", "Resource deadlock avoided."},
+	{117, "EDC5117I", "File exists."},
+	{118, "EDC5118I", "Incorrect address."},
+	{119, "EDC5119I", "File too large."},
+	{120, "EDC5120I", "Interrupted function call."},
+	{121, "EDC5121I", "Invalid argument."},
+	{122, "EDC5122I", "Input/output error."},
+	{123, "EDC5123I", "Is a directory."},
+	{124, "EDC5124I", "Too many open files."},
+	{125, "EDC5125I", "Too many links."},
+	{126, "EDC5126I", "Filename too long."},
+	{127, "EDC5127I", "Too many open files in system."},
+	{128, "EDC5128I", "No such device."},
+	{129, "EDC5129I", "No such file or directory."},
+	{130, "EDC5130I", "Exec format error."},
+	{131, "EDC5131I", "No locks available."},
+	{132, "EDC5132I", "Not enough memory."},
+	{133, "EDC5133I", "No space left on device."},
+	{134, "EDC5134I", "Function not implemented."},
+	{135, "EDC5135I", "Not a directory."},
+	{136, "EDC5136I", "Directory not empty."},
+	{137, "EDC5137I", "Inappropriate I/O control operation."},
+	{138, "EDC5138I", "No such device or address."},
+	{139, "EDC5139I", "Operation not permitted."},
+	{140, "EDC5140I", "Broken pipe."},
+	{141, "EDC5141I", "Read-only file system."},
+	{142, "EDC5142I", "Invalid seek."},
+	{143, "EDC5143I", "No such process."},
+	{144, "EDC5144I", "Improper link."},
+	{145, "EDC5145I", "The parameter list is too long, or the message to receive was too large for the buffer."},
+	{146, "EDC5146I", "Too many levels of symbolic links."},
+	{147, "EDC5147I", "Illegal byte sequence."},
+	{148, "", ""},
+	{149, "EDC5149I", "Value Overflow Error."},
+	{150, "EDC5150I", "UNIX System Services is not active."},
+	{151, "EDC5151I", "Dynamic allocation error."},
+	{152, "EDC5152I", "Common VTOC access facility (CVAF) error."},
+	{153, "EDC5153I", "Catalog obtain error."},
+	{156, "EDC5156I", "Process initialization error."},
+	{157, "EDC5157I", "An internal error has occurred."},
+	{158, "EDC5158I", "Bad parameters were passed to the service."},
+	{159, "EDC5159I", "The Physical File System encountered a permanent file error."},
+	{160, "EDC5160I", "Bad character in environment variable name."},
+	{162, "EDC5162I", "The Physical File System encountered a system error."},
+	{163, "EDC5163I", "SAF/RACF extract error."},
+	{164, "EDC5164I", "SAF/RACF error."},
+	{165, "EDC5165I", "System TOD clock not set."},
+	{166, "EDC5166I", "Access mode argument on function call conflicts with PATHOPTS parameter on JCL DD statement."},
+	{167, "EDC5167I", "Access to the UNIX System Services version of the C RTL is denied."},
+	{168, "EDC5168I", "Password has expired."},
+	{169, "EDC5169I", "Password is invalid."},
+	{170, "EDC5170I", "An error was encountered with WLM."},
+	{171, "EDC5171I", "An error was encountered with CPL."},
+	{172, "EDC5172I", "An error was encountered with Application Response Measurement (ARM) component."},
+	{200, "EDC5200I", "The application contains a Language Environment member language that cannot tolerate a fork()."},
+	{201, "EDC5201I", "The Language Environment message file was not found in the hierarchical file system."},
+	{202, "EDC5202E", "DLL facilities are not supported under SPC environment."},
+	{203, "EDC5203E", "DLL facilities are not supported under POSIX environment."},
+	{227, "EDC5227I", "Buffer is not long enough to contain a path definition"},
+	{228, "EDC5228I", "The file referred to is an external link"},
+	{229, "EDC5229I", "No path definition for ddname in effect"},
+	{230, "EDC5230I", "ESM error."},
+	{231, "EDC5231I", "CP or the external security manager had an error"},
+	{232, "EDC5232I", "The function failed because it was invoked from a multithread environment."},
+	{244, "EDC5244I", "The program, module or DLL is not supported in this environment."},
+	{245, "EDC5245I", "Data is not valid."},
+	{246, "EDC5246I", "Unknown system state."},
+	{247, "EDC5247I", "Operation not supported."},
+	{248, "EDC5248I", "The object name specified is not correct."},
+	{249, "EDC5249I", "The function is not allowed."},
+	{257, "EDC5257I", "Function cannot be called in the child process of a fork() from a multithreaded process until exec() is called."},
+	{258, "EDC5258I", "A CUN_RS_NO_UNI_ENV error was issued by Unicode Services."},
+	{259, "EDC5259I", "A CUN_RS_NO_CONVERSION error was issued by Unicode Services."},
+	{260, "EDC5260I", "A CUN_RS_TABLE_NOT_ALIGNED error was issued by Unicode Services."},
+	{262, "EDC5262I", "An iconv() function encountered an unexpected error while using Unicode Services."},
+	{1000, "EDC8000I", "A bad socket-call constant was found in the IUCV header."},
+	{1001, "EDC8001I", "An error was found in the IUCV header."},
+	{1002, "EDC8002I", "A socket descriptor is out of range."},
+	{1003, "EDC8003I", "A socket descriptor is in use."},
+	{1004, "EDC8004I", "Request failed because of an IUCV error."},
+	{1005, "EDC8005I", "Offload box error."},
+	{1006, "EDC8006I", "Offload box restarted."},
+	{1007, "EDC8007I", "Offload box down."},
+	{1008, "EDC8008I", "Already a conflicting call outstanding on socket."},
+	{1009, "EDC8009I", "Request cancelled using a SOCKcallCANCEL request."},
+	{1011, "EDC8011I", "A name of a PFS was specified that either is not configured or is not a Sockets PFS."},
+	{1100, "EDC8100I", "Block device required."},
+	{1101, "EDC8101I", "Text file busy."},
+	{1102, "EDC8102I", "Operation would block."},
+	{1103, "EDC8103I", "Operation now in progress."},
+	{1104, "EDC8104I", "Connection already in progress."},
+	{1105, "EDC8105I", "Socket operation on non-socket."},
+	{1106, "EDC8106I", "Destination address required."},
+	{1107, "EDC8107I", "Message too long."},
+	{1108, "EDC8108I", "Protocol wrong type for socket."},
+	{1109, "EDC8109I", "Protocol not available."},
+	{1110, "EDC8110I", "Protocol not supported."},
+	{1111, "EDC8111I", "Socket type not supported."},
+	{1112, "EDC8112I", "Operation not supported on socket."},
+	{1113, "EDC8113I", "Protocol family not supported."},
+	{1114, "EDC8114I", "Address family not supported."},
+	{1115, "EDC8115I", "Address already in use."},
+	{1116, "EDC8116I", "Address not available."},
+	{1117, "EDC8117I", "Network is down."},
+	{1118, "EDC8118I", "Network is unreachable."},
+	{1119, "EDC8119I", "Network dropped connection on reset."},
+	{1120, "EDC8120I", "Connection ended abnormally."},
+	{1121, "EDC8121I", "Connection reset."},
+	{1122, "EDC8122I", "No buffer space available."},
+	{1123, "EDC8123I", "Socket already connected."},
+	{1124, "EDC8124I", "Socket not connected."},
+	{1125, "EDC8125I", "Can't send after socket shutdown."},
+	{1126, "EDC8126I", "Too many references; can't splice."},
+	{1127, "EDC8127I", "Connection timed out."},
+	{1128, "EDC8128I", "Connection refused."},
+	{1129, "EDC8129I", "Host is not available."},
+	{1130, "EDC8130I", "Host cannot be reached."},
+	{1131, "EDC8131I", "Too many processes."},
+	{1132, "EDC8132I", "Too many users."},
+	{1133, "EDC8133I", "Disk quota exceeded."},
+	{1134, "EDC8134I", "Stale file handle."},
+	{1135, "", ""},
+	{1136, "EDC8136I", "File is not a STREAM."},
+	{1137, "EDC8137I", "STREAMS ioctl() timeout."},
+	{1138, "EDC8138I", "No STREAMS resources."},
+	{1139, "EDC8139I", "The message identified by set_id and msg_id is not in the message catalog."},
+	{1140, "EDC8140I", "Bad message."},
+	{1141, "EDC8141I", "Identifier removed."},
+	{1142, "", ""},
+	{1143, "", ""},
+	{1144, "EDC8144I", "The link has been severed."},
+	{1145, "", ""},
+	{1146, "", ""},
+	{1147, "", ""},
+	{1148, "EDC8148I", "Protocol error."},
+	{1149, "EDC8149I", "Multihop not allowed."},
+	{1150, "", ""},
+	{1151, "", ""},
+	{1152, "EDC8152I", "The asynchronous I/O request has been canceled."},
+	{1159, "EDC8159I", "Function call was interrupted before any data was received."},
+	{1160, "EDC8160I", "Socket reuse is not supported."},
+	{1161, "EDC8161I", "The file system cannot currently be moved."},
+}
+
+// Signal table
+var signalList = [...]struct {
+	num  Signal
+	name string
+	desc string
+}{
+	{1, "SIGHUP", "hangup"},
+	{2, "SIGINT", "interrupt"},
+	{3, "SIGABT", "aborted"},
+	{4, "SIGILL", "illegal instruction"},
+	{5, "SIGPOLL", "pollable event"},
+	{6, "SIGURG", "urgent I/O condition"},
+	{7, "SIGSTOP", "stop process"},
+	{8, "SIGFPE", "floating point exception"},
+	{9, "SIGKILL", "killed"},
+	{10, "SIGBUS", "bus error"},
+	{11, "SIGSEGV", "segmentation fault"},
+	{12, "SIGSYS", "bad argument to routine"},
+	{13, "SIGPIPE", "broken pipe"},
+	{14, "SIGALRM", "alarm clock"},
+	{15, "SIGTERM", "terminated"},
+	{16, "SIGUSR1", "user defined signal 1"},
+	{17, "SIGUSR2", "user defined signal 2"},
+	{18, "SIGABND", "abend"},
+	{19, "SIGCONT", "continued"},
+	{20, "SIGCHLD", "child exited"},
+	{21, "SIGTTIN", "stopped (tty input)"},
+	{22, "SIGTTOU", "stopped (tty output)"},
+	{23, "SIGIO", "I/O possible"},
+	{24, "SIGQUIT", "quit"},
+	{25, "SIGTSTP", "stopped"},
+	{26, "SIGTRAP", "trace/breakpoint trap"},
+	{27, "SIGIOER", "I/O error"},
+	{28, "SIGWINCH", "window changed"},
+	{29, "SIGXCPU", "CPU time limit exceeded"},
+	{30, "SIGXFSZ", "file size limit exceeded"},
+	{31, "SIGVTALRM", "virtual timer expired"},
+	{32, "SIGPROF", "profiling timer expired"},
+	{33, "SIGDANGER", "danger"},
+	{34, "SIGTHSTOP", "stop thread"},
+	{35, "SIGTHCONT", "continue thread"},
+	{37, "SIGTRACE", "trace"},
+	{38, "", "DCE"},
+	{39, "SIGDUMP", "dump"},
+}
diff --git a/vendor/golang.org/x/sys/unix/zptrace_armnn_linux.go b/vendor/golang.org/x/sys/unix/zptrace_armnn_linux.go
index 89c5920..bd001a6 100644
--- a/vendor/golang.org/x/sys/unix/zptrace_armnn_linux.go
+++ b/vendor/golang.org/x/sys/unix/zptrace_armnn_linux.go
@@ -1,5 +1,6 @@
 // Code generated by linux/mkall.go generatePtracePair("arm", "arm64"). DO NOT EDIT.
 
+//go:build linux && (arm || arm64)
 // +build linux
 // +build arm arm64
 
diff --git a/vendor/golang.org/x/sys/unix/zptrace_mipsnn_linux.go b/vendor/golang.org/x/sys/unix/zptrace_mipsnn_linux.go
index 24b841e..c34d063 100644
--- a/vendor/golang.org/x/sys/unix/zptrace_mipsnn_linux.go
+++ b/vendor/golang.org/x/sys/unix/zptrace_mipsnn_linux.go
@@ -1,5 +1,6 @@
 // Code generated by linux/mkall.go generatePtracePair("mips", "mips64"). DO NOT EDIT.
 
+//go:build linux && (mips || mips64)
 // +build linux
 // +build mips mips64
 
diff --git a/vendor/golang.org/x/sys/unix/zptrace_mipsnnle_linux.go b/vendor/golang.org/x/sys/unix/zptrace_mipsnnle_linux.go
index 47b0489..3ccf0c0 100644
--- a/vendor/golang.org/x/sys/unix/zptrace_mipsnnle_linux.go
+++ b/vendor/golang.org/x/sys/unix/zptrace_mipsnnle_linux.go
@@ -1,5 +1,6 @@
 // Code generated by linux/mkall.go generatePtracePair("mipsle", "mips64le"). DO NOT EDIT.
 
+//go:build linux && (mipsle || mips64le)
 // +build linux
 // +build mipsle mips64le
 
diff --git a/vendor/golang.org/x/sys/unix/zptrace_x86_linux.go b/vendor/golang.org/x/sys/unix/zptrace_x86_linux.go
index ea5d9cb..7d65857 100644
--- a/vendor/golang.org/x/sys/unix/zptrace_x86_linux.go
+++ b/vendor/golang.org/x/sys/unix/zptrace_x86_linux.go
@@ -1,5 +1,6 @@
 // Code generated by linux/mkall.go generatePtracePair("386", "amd64"). DO NOT EDIT.
 
+//go:build linux && (386 || amd64)
 // +build linux
 // +build 386 amd64
 
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc.go b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc.go
index ed657ff..91a23cc 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc.go
@@ -1,6 +1,7 @@
 // go run mksyscall_aix_ppc.go -aix -tags aix,ppc syscall_aix.go syscall_aix_ppc.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build aix && ppc
 // +build aix,ppc
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64.go b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64.go
index 664b293..33c2609 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64.go
@@ -1,6 +1,7 @@
 // go run mksyscall_aix_ppc64.go -aix -tags aix,ppc64 syscall_aix.go syscall_aix_ppc64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build aix && ppc64
 // +build aix,ppc64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gc.go b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gc.go
index 4b3a8ad..8b737fa 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gc.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gc.go
@@ -1,8 +1,8 @@
 // go run mksyscall_aix_ppc64.go -aix -tags aix,ppc64 syscall_aix.go syscall_aix_ppc64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
-// +build aix,ppc64
-// +build !gccgo
+//go:build aix && ppc64 && gc
+// +build aix,ppc64,gc
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gccgo.go b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gccgo.go
index cde4dbc..3c26091 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gccgo.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_aix_ppc64_gccgo.go
@@ -1,8 +1,8 @@
 // go run mksyscall_aix_ppc64.go -aix -tags aix,ppc64 syscall_aix.go syscall_aix_ppc64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
-// +build aix,ppc64
-// +build gccgo
+//go:build aix && ppc64 && gccgo
+// +build aix,ppc64,gccgo
 
 package unix
 
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.go
deleted file mode 100644
index e263fbd..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.go
+++ /dev/null
@@ -1,41 +0,0 @@
-// go run mksyscall.go -l32 -tags darwin,386,go1.13 syscall_darwin.1_13.go
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build darwin,386,go1.13
-
-package unix
-
-import (
-	"syscall"
-	"unsafe"
-)
-
-var _ syscall.Errno
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func closedir(dir uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_closedir_trampoline), uintptr(dir), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_closedir_trampoline()
-
-//go:linkname libc_closedir libc_closedir
-//go:cgo_import_dynamic libc_closedir closedir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func readdir_r(dir uintptr, entry *Dirent, result **Dirent) (res Errno) {
-	r0, _, _ := syscall_syscall(funcPC(libc_readdir_r_trampoline), uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
-	res = Errno(r0)
-	return
-}
-
-func libc_readdir_r_trampoline()
-
-//go:linkname libc_readdir_r libc_readdir_r
-//go:cgo_import_dynamic libc_readdir_r readdir_r "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.s
deleted file mode 100644
index 00da1eb..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.1_13.s
+++ /dev/null
@@ -1,12 +0,0 @@
-// go run mkasm_darwin.go 386
-// Code generated by the command above; DO NOT EDIT.
-
-// +build go1.13
-
-#include "textflag.h"
-TEXT ·libc_fdopendir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fdopendir(SB)
-TEXT ·libc_closedir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_closedir(SB)
-TEXT ·libc_readdir_r_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readdir_r(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.go
deleted file mode 100644
index bd13b38..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.go
+++ /dev/null
@@ -1,2574 +0,0 @@
-// go run mksyscall.go -l32 -tags darwin,386,go1.12 syscall_bsd.go syscall_darwin.go syscall_darwin_386.go
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build darwin,386,go1.12
-
-package unix
-
-import (
-	"syscall"
-	"unsafe"
-)
-
-var _ syscall.Errno
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getgroups(ngid int, gid *_Gid_t) (n int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getgroups_trampoline()
-
-//go:linkname libc_getgroups libc_getgroups
-//go:cgo_import_dynamic libc_getgroups getgroups "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setgroups(ngid int, gid *_Gid_t) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setgroups_trampoline()
-
-//go:linkname libc_setgroups libc_setgroups
-//go:cgo_import_dynamic libc_setgroups setgroups "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func wait4(pid int, wstatus *_C_int, options int, rusage *Rusage) (wpid int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_wait4_trampoline), uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
-	wpid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_wait4_trampoline()
-
-//go:linkname libc_wait4 libc_wait4
-//go:cgo_import_dynamic libc_wait4 wait4 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_accept_trampoline), uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_accept_trampoline()
-
-//go:linkname libc_accept libc_accept
-//go:cgo_import_dynamic libc_accept accept "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_bind_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_bind_trampoline()
-
-//go:linkname libc_bind libc_bind
-//go:cgo_import_dynamic libc_bind bind "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_connect_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_connect_trampoline()
-
-//go:linkname libc_connect libc_connect
-//go:cgo_import_dynamic libc_connect connect "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func socket(domain int, typ int, proto int) (fd int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_socket_trampoline), uintptr(domain), uintptr(typ), uintptr(proto))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_socket_trampoline()
-
-//go:linkname libc_socket libc_socket
-//go:cgo_import_dynamic libc_socket socket "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_getsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsockopt_trampoline()
-
-//go:linkname libc_getsockopt libc_getsockopt
-//go:cgo_import_dynamic libc_getsockopt getsockopt "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setsockopt_trampoline()
-
-//go:linkname libc_setsockopt libc_setsockopt
-//go:cgo_import_dynamic libc_setsockopt setsockopt "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getpeername_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpeername_trampoline()
-
-//go:linkname libc_getpeername libc_getpeername
-//go:cgo_import_dynamic libc_getpeername getpeername "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getsockname_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsockname_trampoline()
-
-//go:linkname libc_getsockname libc_getsockname
-//go:cgo_import_dynamic libc_getsockname getsockname "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Shutdown(s int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_shutdown_trampoline), uintptr(s), uintptr(how), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_shutdown_trampoline()
-
-//go:linkname libc_shutdown libc_shutdown
-//go:cgo_import_dynamic libc_shutdown shutdown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
-	_, _, e1 := syscall_rawSyscall6(funcPC(libc_socketpair_trampoline), uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_socketpair_trampoline()
-
-//go:linkname libc_socketpair libc_socketpair
-//go:cgo_import_dynamic libc_socketpair socketpair "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_recvfrom_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_recvfrom_trampoline()
-
-//go:linkname libc_recvfrom libc_recvfrom
-//go:cgo_import_dynamic libc_recvfrom recvfrom "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error) {
-	var _p0 unsafe.Pointer
-	if len(buf) > 0 {
-		_p0 = unsafe.Pointer(&buf[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendto_trampoline), uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendto_trampoline()
-
-//go:linkname libc_sendto libc_sendto
-//go:cgo_import_dynamic libc_sendto sendto "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_recvmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_recvmsg_trampoline()
-
-//go:linkname libc_recvmsg libc_recvmsg
-//go:cgo_import_dynamic libc_recvmsg recvmsg "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_sendmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendmsg_trampoline()
-
-//go:linkname libc_sendmsg libc_sendmsg
-//go:cgo_import_dynamic libc_sendmsg sendmsg "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func kevent(kq int, change unsafe.Pointer, nchange int, event unsafe.Pointer, nevent int, timeout *Timespec) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_kevent_trampoline), uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kevent_trampoline()
-
-//go:linkname libc_kevent libc_kevent
-//go:cgo_import_dynamic libc_kevent kevent "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func utimes(path string, timeval *[2]Timeval) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_utimes_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_utimes_trampoline()
-
-//go:linkname libc_utimes libc_utimes
-//go:cgo_import_dynamic libc_utimes utimes "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func futimes(fd int, timeval *[2]Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_futimes_trampoline), uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_futimes_trampoline()
-
-//go:linkname libc_futimes libc_futimes
-//go:cgo_import_dynamic libc_futimes futimes "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func poll(fds *PollFd, nfds int, timeout int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_poll_trampoline), uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_poll_trampoline()
-
-//go:linkname libc_poll libc_poll
-//go:cgo_import_dynamic libc_poll poll "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Madvise(b []byte, behav int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_madvise_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(behav))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_madvise_trampoline()
-
-//go:linkname libc_madvise libc_madvise
-//go:cgo_import_dynamic libc_madvise madvise "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mlock(b []byte) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mlock_trampoline()
-
-//go:linkname libc_mlock libc_mlock
-//go:cgo_import_dynamic libc_mlock mlock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mlockall(flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_mlockall_trampoline), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mlockall_trampoline()
-
-//go:linkname libc_mlockall libc_mlockall
-//go:cgo_import_dynamic libc_mlockall mlockall "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mprotect(b []byte, prot int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mprotect_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(prot))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mprotect_trampoline()
-
-//go:linkname libc_mprotect libc_mprotect
-//go:cgo_import_dynamic libc_mprotect mprotect "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Msync(b []byte, flags int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_msync_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_msync_trampoline()
-
-//go:linkname libc_msync libc_msync
-//go:cgo_import_dynamic libc_msync msync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Munlock(b []byte) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_munlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munlock_trampoline()
-
-//go:linkname libc_munlock libc_munlock
-//go:cgo_import_dynamic libc_munlock munlock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Munlockall() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munlockall_trampoline), 0, 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munlockall_trampoline()
-
-//go:linkname libc_munlockall libc_munlockall
-//go:cgo_import_dynamic libc_munlockall munlockall "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func pipe() (r int, w int, err error) {
-	r0, r1, e1 := syscall_rawSyscall(funcPC(libc_pipe_trampoline), 0, 0, 0)
-	r = int(r0)
-	w = int(r1)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pipe_trampoline()
-
-//go:linkname libc_pipe libc_pipe
-//go:cgo_import_dynamic libc_pipe pipe "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getxattr(path string, attr string, dest *byte, size int, position uint32, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_getxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getxattr_trampoline()
-
-//go:linkname libc_getxattr libc_getxattr
-//go:cgo_import_dynamic libc_getxattr getxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fgetxattr(fd int, attr string, dest *byte, size int, position uint32, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_fgetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fgetxattr_trampoline()
-
-//go:linkname libc_fgetxattr libc_fgetxattr
-//go:cgo_import_dynamic libc_fgetxattr fgetxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setxattr(path string, attr string, data *byte, size int, position uint32, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_setxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setxattr_trampoline()
-
-//go:linkname libc_setxattr libc_setxattr
-//go:cgo_import_dynamic libc_setxattr setxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fsetxattr(fd int, attr string, data *byte, size int, position uint32, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fsetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fsetxattr_trampoline()
-
-//go:linkname libc_fsetxattr libc_fsetxattr
-//go:cgo_import_dynamic libc_fsetxattr fsetxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func removexattr(path string, attr string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_removexattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_removexattr_trampoline()
-
-//go:linkname libc_removexattr libc_removexattr
-//go:cgo_import_dynamic libc_removexattr removexattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fremovexattr(fd int, attr string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_fremovexattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fremovexattr_trampoline()
-
-//go:linkname libc_fremovexattr libc_fremovexattr
-//go:cgo_import_dynamic libc_fremovexattr fremovexattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func listxattr(path string, dest *byte, size int, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_listxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_listxattr_trampoline()
-
-//go:linkname libc_listxattr libc_listxattr
-//go:cgo_import_dynamic libc_listxattr listxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func flistxattr(fd int, dest *byte, size int, options int) (sz int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_flistxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_flistxattr_trampoline()
-
-//go:linkname libc_flistxattr libc_flistxattr
-//go:cgo_import_dynamic libc_flistxattr flistxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setattrlist_trampoline), uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setattrlist_trampoline()
-
-//go:linkname libc_setattrlist libc_setattrlist
-//go:cgo_import_dynamic libc_setattrlist setattrlist "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fcntl(fd int, cmd int, arg int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fcntl_trampoline), uintptr(fd), uintptr(cmd), uintptr(arg))
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fcntl_trampoline()
-
-//go:linkname libc_fcntl libc_fcntl
-//go:cgo_import_dynamic libc_fcntl fcntl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func kill(pid int, signum int, posix int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_kill_trampoline), uintptr(pid), uintptr(signum), uintptr(posix))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kill_trampoline()
-
-//go:linkname libc_kill libc_kill
-//go:cgo_import_dynamic libc_kill kill "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func ioctl(fd int, req uint, arg uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ioctl_trampoline), uintptr(fd), uintptr(req), uintptr(arg))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ioctl_trampoline()
-
-//go:linkname libc_ioctl libc_ioctl
-//go:cgo_import_dynamic libc_ioctl ioctl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) {
-	var _p0 unsafe.Pointer
-	if len(mib) > 0 {
-		_p0 = unsafe.Pointer(&mib[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sysctl_trampoline), uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sysctl_trampoline()
-
-//go:linkname libc_sysctl libc_sysctl
-//go:cgo_import_dynamic libc_sysctl sysctl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendfile(infd int, outfd int, offset int64, len *int64, hdtr unsafe.Pointer, flags int) (err error) {
-	_, _, e1 := syscall_syscall9(funcPC(libc_sendfile_trampoline), uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(offset>>32), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendfile_trampoline()
-
-//go:linkname libc_sendfile libc_sendfile
-//go:cgo_import_dynamic libc_sendfile sendfile "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Access(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_access_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_access_trampoline()
-
-//go:linkname libc_access libc_access
-//go:cgo_import_dynamic libc_access access "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Adjtime(delta *Timeval, olddelta *Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_adjtime_trampoline), uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_adjtime_trampoline()
-
-//go:linkname libc_adjtime libc_adjtime
-//go:cgo_import_dynamic libc_adjtime adjtime "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chdir(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chdir_trampoline()
-
-//go:linkname libc_chdir libc_chdir
-//go:cgo_import_dynamic libc_chdir chdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chflags(path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chflags_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chflags_trampoline()
-
-//go:linkname libc_chflags libc_chflags
-//go:cgo_import_dynamic libc_chflags chflags "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chmod(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chmod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chmod_trampoline()
-
-//go:linkname libc_chmod libc_chmod
-//go:cgo_import_dynamic libc_chmod chmod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chown(path string, uid int, gid int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chown_trampoline()
-
-//go:linkname libc_chown libc_chown
-//go:cgo_import_dynamic libc_chown chown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chroot(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chroot_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chroot_trampoline()
-
-//go:linkname libc_chroot libc_chroot
-//go:cgo_import_dynamic libc_chroot chroot "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func ClockGettime(clockid int32, time *Timespec) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_clock_gettime_trampoline), uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clock_gettime_trampoline()
-
-//go:linkname libc_clock_gettime libc_clock_gettime
-//go:cgo_import_dynamic libc_clock_gettime clock_gettime "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Close(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_close_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_close_trampoline()
-
-//go:linkname libc_close libc_close
-//go:cgo_import_dynamic libc_close close "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Clonefile(src string, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(src)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_clonefile_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clonefile_trampoline()
-
-//go:linkname libc_clonefile libc_clonefile
-//go:cgo_import_dynamic libc_clonefile clonefile "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Clonefileat(srcDirfd int, src string, dstDirfd int, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(src)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_clonefileat_trampoline), uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clonefileat_trampoline()
-
-//go:linkname libc_clonefileat libc_clonefileat
-//go:cgo_import_dynamic libc_clonefileat clonefileat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Dup(fd int) (nfd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_dup_trampoline), uintptr(fd), 0, 0)
-	nfd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_dup_trampoline()
-
-//go:linkname libc_dup libc_dup
-//go:cgo_import_dynamic libc_dup dup "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Dup2(from int, to int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_dup2_trampoline), uintptr(from), uintptr(to), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_dup2_trampoline()
-
-//go:linkname libc_dup2 libc_dup2
-//go:cgo_import_dynamic libc_dup2 dup2 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Exchangedata(path1 string, path2 string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path1)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(path2)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_exchangedata_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_exchangedata_trampoline()
-
-//go:linkname libc_exchangedata libc_exchangedata
-//go:cgo_import_dynamic libc_exchangedata exchangedata "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Exit(code int) {
-	syscall_syscall(funcPC(libc_exit_trampoline), uintptr(code), 0, 0)
-	return
-}
-
-func libc_exit_trampoline()
-
-//go:linkname libc_exit libc_exit
-//go:cgo_import_dynamic libc_exit exit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Faccessat(dirfd int, path string, mode uint32, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_faccessat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_faccessat_trampoline()
-
-//go:linkname libc_faccessat libc_faccessat
-//go:cgo_import_dynamic libc_faccessat faccessat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchdir(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchdir_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchdir_trampoline()
-
-//go:linkname libc_fchdir libc_fchdir
-//go:cgo_import_dynamic libc_fchdir fchdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchflags(fd int, flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchflags_trampoline), uintptr(fd), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchflags_trampoline()
-
-//go:linkname libc_fchflags libc_fchflags
-//go:cgo_import_dynamic libc_fchflags fchflags "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchmod(fd int, mode uint32) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchmod_trampoline), uintptr(fd), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchmod_trampoline()
-
-//go:linkname libc_fchmod libc_fchmod
-//go:cgo_import_dynamic libc_fchmod fchmod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchmodat(dirfd int, path string, mode uint32, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchmodat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchmodat_trampoline()
-
-//go:linkname libc_fchmodat libc_fchmodat
-//go:cgo_import_dynamic libc_fchmodat fchmodat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchown(fd int, uid int, gid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchown_trampoline), uintptr(fd), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchown_trampoline()
-
-//go:linkname libc_fchown libc_fchown
-//go:cgo_import_dynamic libc_fchown fchown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchownat(dirfd int, path string, uid int, gid int, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchownat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchownat_trampoline()
-
-//go:linkname libc_fchownat libc_fchownat
-//go:cgo_import_dynamic libc_fchownat fchownat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fclonefileat(srcDirfd int, dstDirfd int, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fclonefileat_trampoline), uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fclonefileat_trampoline()
-
-//go:linkname libc_fclonefileat libc_fclonefileat
-//go:cgo_import_dynamic libc_fclonefileat fclonefileat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Flock(fd int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_flock_trampoline), uintptr(fd), uintptr(how), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_flock_trampoline()
-
-//go:linkname libc_flock libc_flock
-//go:cgo_import_dynamic libc_flock flock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fpathconf(fd int, name int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fpathconf_trampoline), uintptr(fd), uintptr(name), 0)
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fpathconf_trampoline()
-
-//go:linkname libc_fpathconf libc_fpathconf
-//go:cgo_import_dynamic libc_fpathconf fpathconf "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fsync(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fsync_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fsync_trampoline()
-
-//go:linkname libc_fsync libc_fsync
-//go:cgo_import_dynamic libc_fsync fsync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Ftruncate(fd int, length int64) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ftruncate_trampoline), uintptr(fd), uintptr(length), uintptr(length>>32))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ftruncate_trampoline()
-
-//go:linkname libc_ftruncate libc_ftruncate
-//go:cgo_import_dynamic libc_ftruncate ftruncate "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getcwd(buf []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(buf) > 0 {
-		_p0 = unsafe.Pointer(&buf[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_getcwd_trampoline), uintptr(_p0), uintptr(len(buf)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getcwd_trampoline()
-
-//go:linkname libc_getcwd libc_getcwd
-//go:cgo_import_dynamic libc_getcwd getcwd "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getdtablesize() (size int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_getdtablesize_trampoline), 0, 0, 0)
-	size = int(r0)
-	return
-}
-
-func libc_getdtablesize_trampoline()
-
-//go:linkname libc_getdtablesize libc_getdtablesize
-//go:cgo_import_dynamic libc_getdtablesize getdtablesize "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getegid() (egid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getegid_trampoline), 0, 0, 0)
-	egid = int(r0)
-	return
-}
-
-func libc_getegid_trampoline()
-
-//go:linkname libc_getegid libc_getegid
-//go:cgo_import_dynamic libc_getegid getegid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Geteuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_geteuid_trampoline), 0, 0, 0)
-	uid = int(r0)
-	return
-}
-
-func libc_geteuid_trampoline()
-
-//go:linkname libc_geteuid libc_geteuid
-//go:cgo_import_dynamic libc_geteuid geteuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getgid() (gid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getgid_trampoline), 0, 0, 0)
-	gid = int(r0)
-	return
-}
-
-func libc_getgid_trampoline()
-
-//go:linkname libc_getgid libc_getgid
-//go:cgo_import_dynamic libc_getgid getgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpgid(pid int) (pgid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getpgid_trampoline), uintptr(pid), 0, 0)
-	pgid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpgid_trampoline()
-
-//go:linkname libc_getpgid libc_getpgid
-//go:cgo_import_dynamic libc_getpgid getpgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpgrp() (pgrp int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpgrp_trampoline), 0, 0, 0)
-	pgrp = int(r0)
-	return
-}
-
-func libc_getpgrp_trampoline()
-
-//go:linkname libc_getpgrp libc_getpgrp
-//go:cgo_import_dynamic libc_getpgrp getpgrp "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpid() (pid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpid_trampoline), 0, 0, 0)
-	pid = int(r0)
-	return
-}
-
-func libc_getpid_trampoline()
-
-//go:linkname libc_getpid libc_getpid
-//go:cgo_import_dynamic libc_getpid getpid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getppid() (ppid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getppid_trampoline), 0, 0, 0)
-	ppid = int(r0)
-	return
-}
-
-func libc_getppid_trampoline()
-
-//go:linkname libc_getppid libc_getppid
-//go:cgo_import_dynamic libc_getppid getppid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpriority(which int, who int) (prio int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getpriority_trampoline), uintptr(which), uintptr(who), 0)
-	prio = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpriority_trampoline()
-
-//go:linkname libc_getpriority libc_getpriority
-//go:cgo_import_dynamic libc_getpriority getpriority "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getrlimit_trampoline()
-
-//go:linkname libc_getrlimit libc_getrlimit
-//go:cgo_import_dynamic libc_getrlimit getrlimit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getrusage(who int, rusage *Rusage) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrusage_trampoline), uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getrusage_trampoline()
-
-//go:linkname libc_getrusage libc_getrusage
-//go:cgo_import_dynamic libc_getrusage getrusage "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getsid(pid int) (sid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getsid_trampoline), uintptr(pid), 0, 0)
-	sid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsid_trampoline()
-
-//go:linkname libc_getsid libc_getsid
-//go:cgo_import_dynamic libc_getsid getsid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Gettimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_gettimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_gettimeofday_trampoline()
-
-//go:linkname libc_gettimeofday libc_gettimeofday
-//go:cgo_import_dynamic libc_gettimeofday gettimeofday "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getuid_trampoline), 0, 0, 0)
-	uid = int(r0)
-	return
-}
-
-func libc_getuid_trampoline()
-
-//go:linkname libc_getuid libc_getuid
-//go:cgo_import_dynamic libc_getuid getuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Issetugid() (tainted bool) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_issetugid_trampoline), 0, 0, 0)
-	tainted = bool(r0 != 0)
-	return
-}
-
-func libc_issetugid_trampoline()
-
-//go:linkname libc_issetugid libc_issetugid
-//go:cgo_import_dynamic libc_issetugid issetugid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Kqueue() (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_kqueue_trampoline), 0, 0, 0)
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kqueue_trampoline()
-
-//go:linkname libc_kqueue libc_kqueue
-//go:cgo_import_dynamic libc_kqueue kqueue "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Lchown(path string, uid int, gid int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lchown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lchown_trampoline()
-
-//go:linkname libc_lchown libc_lchown
-//go:cgo_import_dynamic libc_lchown lchown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Link(path string, link string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_link_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_link_trampoline()
-
-//go:linkname libc_link libc_link
-//go:cgo_import_dynamic libc_link link "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Linkat(pathfd int, path string, linkfd int, link string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_linkat_trampoline), uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_linkat_trampoline()
-
-//go:linkname libc_linkat libc_linkat
-//go:cgo_import_dynamic libc_linkat linkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Listen(s int, backlog int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_listen_trampoline), uintptr(s), uintptr(backlog), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_listen_trampoline()
-
-//go:linkname libc_listen libc_listen
-//go:cgo_import_dynamic libc_listen listen "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkdir(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdir_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkdir_trampoline()
-
-//go:linkname libc_mkdir libc_mkdir
-//go:cgo_import_dynamic libc_mkdir mkdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkdirat(dirfd int, path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdirat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkdirat_trampoline()
-
-//go:linkname libc_mkdirat libc_mkdirat
-//go:cgo_import_dynamic libc_mkdirat mkdirat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkfifo(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkfifo_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkfifo_trampoline()
-
-//go:linkname libc_mkfifo libc_mkfifo
-//go:cgo_import_dynamic libc_mkfifo mkfifo "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mknod(path string, mode uint32, dev int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mknod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mknod_trampoline()
-
-//go:linkname libc_mknod libc_mknod
-//go:cgo_import_dynamic libc_mknod mknod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Open(path string, mode int, perm uint32) (fd int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_open_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_open_trampoline()
-
-//go:linkname libc_open libc_open
-//go:cgo_import_dynamic libc_open open "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Openat(dirfd int, path string, mode int, perm uint32) (fd int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_openat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_openat_trampoline()
-
-//go:linkname libc_openat libc_openat
-//go:cgo_import_dynamic libc_openat openat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pathconf(path string, name int) (val int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_pathconf_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pathconf_trampoline()
-
-//go:linkname libc_pathconf libc_pathconf
-//go:cgo_import_dynamic libc_pathconf pathconf "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pread(fd int, p []byte, offset int64) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pread_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), uintptr(offset>>32), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pread_trampoline()
-
-//go:linkname libc_pread libc_pread
-//go:cgo_import_dynamic libc_pread pread "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pwrite(fd int, p []byte, offset int64) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pwrite_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), uintptr(offset>>32), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pwrite_trampoline()
-
-//go:linkname libc_pwrite libc_pwrite
-//go:cgo_import_dynamic libc_pwrite pwrite "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func read(fd int, p []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_read_trampoline()
-
-//go:linkname libc_read libc_read
-//go:cgo_import_dynamic libc_read read "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Readlink(path string, buf []byte) (n int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 unsafe.Pointer
-	if len(buf) > 0 {
-		_p1 = unsafe.Pointer(&buf[0])
-	} else {
-		_p1 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_readlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_readlink_trampoline()
-
-//go:linkname libc_readlink libc_readlink
-//go:cgo_import_dynamic libc_readlink readlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Readlinkat(dirfd int, path string, buf []byte) (n int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 unsafe.Pointer
-	if len(buf) > 0 {
-		_p1 = unsafe.Pointer(&buf[0])
-	} else {
-		_p1 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_readlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_readlinkat_trampoline()
-
-//go:linkname libc_readlinkat libc_readlinkat
-//go:cgo_import_dynamic libc_readlinkat readlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Rename(from string, to string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(from)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(to)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rename_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_rename_trampoline()
-
-//go:linkname libc_rename libc_rename
-//go:cgo_import_dynamic libc_rename rename "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Renameat(fromfd int, from string, tofd int, to string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(from)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(to)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_renameat_trampoline), uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_renameat_trampoline()
-
-//go:linkname libc_renameat libc_renameat
-//go:cgo_import_dynamic libc_renameat renameat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Revoke(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_revoke_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_revoke_trampoline()
-
-//go:linkname libc_revoke libc_revoke
-//go:cgo_import_dynamic libc_revoke revoke "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Rmdir(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rmdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_rmdir_trampoline()
-
-//go:linkname libc_rmdir libc_rmdir
-//go:cgo_import_dynamic libc_rmdir rmdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Seek(fd int, offset int64, whence int) (newoffset int64, err error) {
-	r0, r1, e1 := syscall_syscall6(funcPC(libc_lseek_trampoline), uintptr(fd), uintptr(offset), uintptr(offset>>32), uintptr(whence), 0, 0)
-	newoffset = int64(int64(r1)<<32 | int64(r0))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lseek_trampoline()
-
-//go:linkname libc_lseek libc_lseek
-//go:cgo_import_dynamic libc_lseek lseek "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_select_trampoline), uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_select_trampoline()
-
-//go:linkname libc_select libc_select
-//go:cgo_import_dynamic libc_select select "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setegid(egid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setegid_trampoline), uintptr(egid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setegid_trampoline()
-
-//go:linkname libc_setegid libc_setegid
-//go:cgo_import_dynamic libc_setegid setegid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Seteuid(euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_seteuid_trampoline), uintptr(euid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_seteuid_trampoline()
-
-//go:linkname libc_seteuid libc_seteuid
-//go:cgo_import_dynamic libc_seteuid seteuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setgid(gid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgid_trampoline), uintptr(gid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setgid_trampoline()
-
-//go:linkname libc_setgid libc_setgid
-//go:cgo_import_dynamic libc_setgid setgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setlogin(name string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(name)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_setlogin_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setlogin_trampoline()
-
-//go:linkname libc_setlogin libc_setlogin
-//go:cgo_import_dynamic libc_setlogin setlogin "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setpgid(pid int, pgid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setpgid_trampoline), uintptr(pid), uintptr(pgid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setpgid_trampoline()
-
-//go:linkname libc_setpgid libc_setpgid
-//go:cgo_import_dynamic libc_setpgid setpgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setpriority(which int, who int, prio int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setpriority_trampoline), uintptr(which), uintptr(who), uintptr(prio))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setpriority_trampoline()
-
-//go:linkname libc_setpriority libc_setpriority
-//go:cgo_import_dynamic libc_setpriority setpriority "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setprivexec(flag int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setprivexec_trampoline), uintptr(flag), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setprivexec_trampoline()
-
-//go:linkname libc_setprivexec libc_setprivexec
-//go:cgo_import_dynamic libc_setprivexec setprivexec "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setregid(rgid int, egid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setregid_trampoline), uintptr(rgid), uintptr(egid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setregid_trampoline()
-
-//go:linkname libc_setregid libc_setregid
-//go:cgo_import_dynamic libc_setregid setregid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setreuid(ruid int, euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setreuid_trampoline), uintptr(ruid), uintptr(euid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setreuid_trampoline()
-
-//go:linkname libc_setreuid libc_setreuid
-//go:cgo_import_dynamic libc_setreuid setreuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setrlimit_trampoline()
-
-//go:linkname libc_setrlimit libc_setrlimit
-//go:cgo_import_dynamic libc_setrlimit setrlimit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setsid() (pid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_setsid_trampoline), 0, 0, 0)
-	pid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setsid_trampoline()
-
-//go:linkname libc_setsid libc_setsid
-//go:cgo_import_dynamic libc_setsid setsid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Settimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_settimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_settimeofday_trampoline()
-
-//go:linkname libc_settimeofday libc_settimeofday
-//go:cgo_import_dynamic libc_settimeofday settimeofday "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setuid(uid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setuid_trampoline), uintptr(uid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setuid_trampoline()
-
-//go:linkname libc_setuid libc_setuid
-//go:cgo_import_dynamic libc_setuid setuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Symlink(path string, link string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_symlink_trampoline()
-
-//go:linkname libc_symlink libc_symlink
-//go:cgo_import_dynamic libc_symlink symlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Symlinkat(oldpath string, newdirfd int, newpath string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(oldpath)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(newpath)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlinkat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_symlinkat_trampoline()
-
-//go:linkname libc_symlinkat libc_symlinkat
-//go:cgo_import_dynamic libc_symlinkat symlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Sync() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_sync_trampoline), 0, 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sync_trampoline()
-
-//go:linkname libc_sync libc_sync
-//go:cgo_import_dynamic libc_sync sync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Truncate(path string, length int64) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_truncate_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(length), uintptr(length>>32))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_truncate_trampoline()
-
-//go:linkname libc_truncate libc_truncate
-//go:cgo_import_dynamic libc_truncate truncate "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Umask(newmask int) (oldmask int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_umask_trampoline), uintptr(newmask), 0, 0)
-	oldmask = int(r0)
-	return
-}
-
-func libc_umask_trampoline()
-
-//go:linkname libc_umask libc_umask
-//go:cgo_import_dynamic libc_umask umask "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Undelete(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_undelete_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_undelete_trampoline()
-
-//go:linkname libc_undelete libc_undelete
-//go:cgo_import_dynamic libc_undelete undelete "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unlink(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlink_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unlink_trampoline()
-
-//go:linkname libc_unlink libc_unlink
-//go:cgo_import_dynamic libc_unlink unlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unlinkat(dirfd int, path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unlinkat_trampoline()
-
-//go:linkname libc_unlinkat libc_unlinkat
-//go:cgo_import_dynamic libc_unlinkat unlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unmount(path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unmount_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unmount_trampoline()
-
-//go:linkname libc_unmount libc_unmount
-//go:cgo_import_dynamic libc_unmount unmount "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func write(fd int, p []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_write_trampoline()
-
-//go:linkname libc_write libc_write
-//go:cgo_import_dynamic libc_write write "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) {
-	r0, _, e1 := syscall_syscall9(funcPC(libc_mmap_trampoline), uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos), uintptr(pos>>32), 0, 0)
-	ret = uintptr(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mmap_trampoline()
-
-//go:linkname libc_mmap libc_mmap
-//go:cgo_import_dynamic libc_mmap mmap "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func munmap(addr uintptr, length uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munmap_trampoline), uintptr(addr), uintptr(length), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munmap_trampoline()
-
-//go:linkname libc_munmap libc_munmap
-//go:cgo_import_dynamic libc_munmap munmap "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func readlen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func writelen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func ptrace(request int, pid int, addr uintptr, data uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_ptrace_trampoline), uintptr(request), uintptr(pid), uintptr(addr), uintptr(data), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ptrace_trampoline()
-
-//go:linkname libc_ptrace libc_ptrace
-//go:cgo_import_dynamic libc_ptrace ptrace "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstat(fd int, stat *Stat_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstat64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstat64_trampoline()
-
-//go:linkname libc_fstat64 libc_fstat64
-//go:cgo_import_dynamic libc_fstat64 fstat64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstatat(fd int, path string, stat *Stat_t, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fstatat64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstatat64_trampoline()
-
-//go:linkname libc_fstatat64 libc_fstatat64
-//go:cgo_import_dynamic libc_fstatat64 fstatat64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstatfs(fd int, stat *Statfs_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstatfs64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstatfs64_trampoline()
-
-//go:linkname libc_fstatfs64 libc_fstatfs64
-//go:cgo_import_dynamic libc_fstatfs64 fstatfs64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getfsstat64_trampoline), uintptr(buf), uintptr(size), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getfsstat64_trampoline()
-
-//go:linkname libc_getfsstat64 libc_getfsstat64
-//go:cgo_import_dynamic libc_getfsstat64 getfsstat64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Lstat(path string, stat *Stat_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lstat64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lstat64_trampoline()
-
-//go:linkname libc_lstat64 libc_lstat64
-//go:cgo_import_dynamic libc_lstat64 lstat64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Stat(path string, stat *Stat_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_stat64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_stat64_trampoline()
-
-//go:linkname libc_stat64 libc_stat64
-//go:cgo_import_dynamic libc_stat64 stat64 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Statfs(path string, stat *Statfs_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_statfs64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_statfs64_trampoline()
-
-//go:linkname libc_statfs64 libc_statfs64
-//go:cgo_import_dynamic libc_statfs64 statfs64 "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.s
deleted file mode 100644
index d5fb53f..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_386.s
+++ /dev/null
@@ -1,290 +0,0 @@
-// go run mkasm_darwin.go 386
-// Code generated by the command above; DO NOT EDIT.
-
-// +build go1.12
-
-#include "textflag.h"
-TEXT ·libc_getgroups_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getgroups(SB)
-TEXT ·libc_setgroups_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setgroups(SB)
-TEXT ·libc_wait4_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_wait4(SB)
-TEXT ·libc_accept_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_accept(SB)
-TEXT ·libc_bind_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_bind(SB)
-TEXT ·libc_connect_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_connect(SB)
-TEXT ·libc_socket_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_socket(SB)
-TEXT ·libc_getsockopt_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsockopt(SB)
-TEXT ·libc_setsockopt_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setsockopt(SB)
-TEXT ·libc_getpeername_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpeername(SB)
-TEXT ·libc_getsockname_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsockname(SB)
-TEXT ·libc_shutdown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_shutdown(SB)
-TEXT ·libc_socketpair_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_socketpair(SB)
-TEXT ·libc_recvfrom_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_recvfrom(SB)
-TEXT ·libc_sendto_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendto(SB)
-TEXT ·libc_recvmsg_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_recvmsg(SB)
-TEXT ·libc_sendmsg_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendmsg(SB)
-TEXT ·libc_kevent_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kevent(SB)
-TEXT ·libc_utimes_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_utimes(SB)
-TEXT ·libc_futimes_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_futimes(SB)
-TEXT ·libc_poll_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_poll(SB)
-TEXT ·libc_madvise_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_madvise(SB)
-TEXT ·libc_mlock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mlock(SB)
-TEXT ·libc_mlockall_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mlockall(SB)
-TEXT ·libc_mprotect_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mprotect(SB)
-TEXT ·libc_msync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_msync(SB)
-TEXT ·libc_munlock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munlock(SB)
-TEXT ·libc_munlockall_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munlockall(SB)
-TEXT ·libc_pipe_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pipe(SB)
-TEXT ·libc_getxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getxattr(SB)
-TEXT ·libc_fgetxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fgetxattr(SB)
-TEXT ·libc_setxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setxattr(SB)
-TEXT ·libc_fsetxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fsetxattr(SB)
-TEXT ·libc_removexattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_removexattr(SB)
-TEXT ·libc_fremovexattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fremovexattr(SB)
-TEXT ·libc_listxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_listxattr(SB)
-TEXT ·libc_flistxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_flistxattr(SB)
-TEXT ·libc_setattrlist_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setattrlist(SB)
-TEXT ·libc_fcntl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fcntl(SB)
-TEXT ·libc_kill_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kill(SB)
-TEXT ·libc_ioctl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ioctl(SB)
-TEXT ·libc_sysctl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sysctl(SB)
-TEXT ·libc_sendfile_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendfile(SB)
-TEXT ·libc_access_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_access(SB)
-TEXT ·libc_adjtime_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_adjtime(SB)
-TEXT ·libc_chdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chdir(SB)
-TEXT ·libc_chflags_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chflags(SB)
-TEXT ·libc_chmod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chmod(SB)
-TEXT ·libc_chown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chown(SB)
-TEXT ·libc_chroot_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chroot(SB)
-TEXT ·libc_clock_gettime_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clock_gettime(SB)
-TEXT ·libc_close_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_close(SB)
-TEXT ·libc_clonefile_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clonefile(SB)
-TEXT ·libc_clonefileat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clonefileat(SB)
-TEXT ·libc_dup_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_dup(SB)
-TEXT ·libc_dup2_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_dup2(SB)
-TEXT ·libc_exchangedata_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_exchangedata(SB)
-TEXT ·libc_exit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_exit(SB)
-TEXT ·libc_faccessat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_faccessat(SB)
-TEXT ·libc_fchdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchdir(SB)
-TEXT ·libc_fchflags_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchflags(SB)
-TEXT ·libc_fchmod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchmod(SB)
-TEXT ·libc_fchmodat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchmodat(SB)
-TEXT ·libc_fchown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchown(SB)
-TEXT ·libc_fchownat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchownat(SB)
-TEXT ·libc_fclonefileat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fclonefileat(SB)
-TEXT ·libc_flock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_flock(SB)
-TEXT ·libc_fpathconf_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fpathconf(SB)
-TEXT ·libc_fsync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fsync(SB)
-TEXT ·libc_ftruncate_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ftruncate(SB)
-TEXT ·libc_getcwd_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getcwd(SB)
-TEXT ·libc_getdtablesize_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getdtablesize(SB)
-TEXT ·libc_getegid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getegid(SB)
-TEXT ·libc_geteuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_geteuid(SB)
-TEXT ·libc_getgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getgid(SB)
-TEXT ·libc_getpgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpgid(SB)
-TEXT ·libc_getpgrp_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpgrp(SB)
-TEXT ·libc_getpid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpid(SB)
-TEXT ·libc_getppid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getppid(SB)
-TEXT ·libc_getpriority_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpriority(SB)
-TEXT ·libc_getrlimit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getrlimit(SB)
-TEXT ·libc_getrusage_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getrusage(SB)
-TEXT ·libc_getsid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsid(SB)
-TEXT ·libc_gettimeofday_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_gettimeofday(SB)
-TEXT ·libc_getuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getuid(SB)
-TEXT ·libc_issetugid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_issetugid(SB)
-TEXT ·libc_kqueue_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kqueue(SB)
-TEXT ·libc_lchown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lchown(SB)
-TEXT ·libc_link_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_link(SB)
-TEXT ·libc_linkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_linkat(SB)
-TEXT ·libc_listen_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_listen(SB)
-TEXT ·libc_mkdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkdir(SB)
-TEXT ·libc_mkdirat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkdirat(SB)
-TEXT ·libc_mkfifo_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkfifo(SB)
-TEXT ·libc_mknod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mknod(SB)
-TEXT ·libc_open_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_open(SB)
-TEXT ·libc_openat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_openat(SB)
-TEXT ·libc_pathconf_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pathconf(SB)
-TEXT ·libc_pread_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pread(SB)
-TEXT ·libc_pwrite_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pwrite(SB)
-TEXT ·libc_read_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_read(SB)
-TEXT ·libc_readlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readlink(SB)
-TEXT ·libc_readlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readlinkat(SB)
-TEXT ·libc_rename_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_rename(SB)
-TEXT ·libc_renameat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_renameat(SB)
-TEXT ·libc_revoke_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_revoke(SB)
-TEXT ·libc_rmdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_rmdir(SB)
-TEXT ·libc_lseek_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lseek(SB)
-TEXT ·libc_select_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_select(SB)
-TEXT ·libc_setegid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setegid(SB)
-TEXT ·libc_seteuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_seteuid(SB)
-TEXT ·libc_setgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setgid(SB)
-TEXT ·libc_setlogin_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setlogin(SB)
-TEXT ·libc_setpgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setpgid(SB)
-TEXT ·libc_setpriority_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setpriority(SB)
-TEXT ·libc_setprivexec_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setprivexec(SB)
-TEXT ·libc_setregid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setregid(SB)
-TEXT ·libc_setreuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setreuid(SB)
-TEXT ·libc_setrlimit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setrlimit(SB)
-TEXT ·libc_setsid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setsid(SB)
-TEXT ·libc_settimeofday_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_settimeofday(SB)
-TEXT ·libc_setuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setuid(SB)
-TEXT ·libc_symlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_symlink(SB)
-TEXT ·libc_symlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_symlinkat(SB)
-TEXT ·libc_sync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sync(SB)
-TEXT ·libc_truncate_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_truncate(SB)
-TEXT ·libc_umask_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_umask(SB)
-TEXT ·libc_undelete_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_undelete(SB)
-TEXT ·libc_unlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unlink(SB)
-TEXT ·libc_unlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unlinkat(SB)
-TEXT ·libc_unmount_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unmount(SB)
-TEXT ·libc_write_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_write(SB)
-TEXT ·libc_mmap_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mmap(SB)
-TEXT ·libc_munmap_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munmap(SB)
-TEXT ·libc_ptrace_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ptrace(SB)
-TEXT ·libc_fstat64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstat64(SB)
-TEXT ·libc_fstatat64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstatat64(SB)
-TEXT ·libc_fstatfs64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstatfs64(SB)
-TEXT ·libc_getfsstat64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getfsstat64(SB)
-TEXT ·libc_lstat64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lstat64(SB)
-TEXT ·libc_stat64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_stat64(SB)
-TEXT ·libc_statfs64_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_statfs64(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.go
index 314042a..a06eb09 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags darwin,amd64,go1.13 syscall_darwin.1_13.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build darwin && amd64 && go1.13
 // +build darwin,amd64,go1.13
 
 package unix
@@ -15,27 +16,25 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func closedir(dir uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_closedir_trampoline), uintptr(dir), 0, 0)
+	_, _, e1 := syscall_syscall(libc_closedir_trampoline_addr, uintptr(dir), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_closedir_trampoline()
+var libc_closedir_trampoline_addr uintptr
 
-//go:linkname libc_closedir libc_closedir
 //go:cgo_import_dynamic libc_closedir closedir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func readdir_r(dir uintptr, entry *Dirent, result **Dirent) (res Errno) {
-	r0, _, _ := syscall_syscall(funcPC(libc_readdir_r_trampoline), uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
+	r0, _, _ := syscall_syscall(libc_readdir_r_trampoline_addr, uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
 	res = Errno(r0)
 	return
 }
 
-func libc_readdir_r_trampoline()
+var libc_readdir_r_trampoline_addr uintptr
 
-//go:linkname libc_readdir_r libc_readdir_r
 //go:cgo_import_dynamic libc_readdir_r readdir_r "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.s
index d671e83..d6c3e25 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.s
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.1_13.s
@@ -1,12 +1,25 @@
 // go run mkasm_darwin.go amd64
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build go1.13
 // +build go1.13
 
 #include "textflag.h"
-TEXT ·libc_fdopendir_trampoline(SB),NOSPLIT,$0-0
+
+TEXT libc_fdopendir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fdopendir(SB)
-TEXT ·libc_closedir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fdopendir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fdopendir_trampoline_addr(SB)/8, $libc_fdopendir_trampoline<>(SB)
+
+TEXT libc_closedir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_closedir(SB)
-TEXT ·libc_readdir_r_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_closedir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_closedir_trampoline_addr(SB)/8, $libc_closedir_trampoline<>(SB)
+
+TEXT libc_readdir_r_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readdir_r(SB)
+
+GLOBL	·libc_readdir_r_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readdir_r_trampoline_addr(SB)/8, $libc_readdir_r_trampoline<>(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.go
index d81696f..d4efe8d 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags darwin,amd64,go1.12 syscall_bsd.go syscall_darwin.go syscall_darwin_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build darwin && amd64 && go1.12
 // +build darwin,amd64,go1.12
 
 package unix
@@ -15,7 +16,7 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getgroups(ngid int, gid *_Gid_t) (n int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getgroups_trampoline_addr, uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -23,30 +24,28 @@
 	return
 }
 
-func libc_getgroups_trampoline()
+var libc_getgroups_trampoline_addr uintptr
 
-//go:linkname libc_getgroups libc_getgroups
 //go:cgo_import_dynamic libc_getgroups getgroups "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setgroups(ngid int, gid *_Gid_t) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setgroups_trampoline_addr, uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setgroups_trampoline()
+var libc_setgroups_trampoline_addr uintptr
 
-//go:linkname libc_setgroups libc_setgroups
 //go:cgo_import_dynamic libc_setgroups setgroups "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func wait4(pid int, wstatus *_C_int, options int, rusage *Rusage) (wpid int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_wait4_trampoline), uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_wait4_trampoline_addr, uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
 	wpid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -54,15 +53,14 @@
 	return
 }
 
-func libc_wait4_trampoline()
+var libc_wait4_trampoline_addr uintptr
 
-//go:linkname libc_wait4 libc_wait4
 //go:cgo_import_dynamic libc_wait4 wait4 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_accept_trampoline), uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	r0, _, e1 := syscall_syscall(libc_accept_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -70,45 +68,42 @@
 	return
 }
 
-func libc_accept_trampoline()
+var libc_accept_trampoline_addr uintptr
 
-//go:linkname libc_accept libc_accept
 //go:cgo_import_dynamic libc_accept accept "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_bind_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
+	_, _, e1 := syscall_syscall(libc_bind_trampoline_addr, uintptr(s), uintptr(addr), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_bind_trampoline()
+var libc_bind_trampoline_addr uintptr
 
-//go:linkname libc_bind libc_bind
 //go:cgo_import_dynamic libc_bind bind "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_connect_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
+	_, _, e1 := syscall_syscall(libc_connect_trampoline_addr, uintptr(s), uintptr(addr), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_connect_trampoline()
+var libc_connect_trampoline_addr uintptr
 
-//go:linkname libc_connect libc_connect
 //go:cgo_import_dynamic libc_connect connect "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func socket(domain int, typ int, proto int) (fd int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_socket_trampoline), uintptr(domain), uintptr(typ), uintptr(proto))
+	r0, _, e1 := syscall_rawSyscall(libc_socket_trampoline_addr, uintptr(domain), uintptr(typ), uintptr(proto))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -116,99 +111,92 @@
 	return
 }
 
-func libc_socket_trampoline()
+var libc_socket_trampoline_addr uintptr
 
-//go:linkname libc_socket libc_socket
 //go:cgo_import_dynamic libc_socket socket "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_getsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
+	_, _, e1 := syscall_syscall6(libc_getsockopt_trampoline_addr, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getsockopt_trampoline()
+var libc_getsockopt_trampoline_addr uintptr
 
-//go:linkname libc_getsockopt libc_getsockopt
 //go:cgo_import_dynamic libc_getsockopt getsockopt "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
+	_, _, e1 := syscall_syscall6(libc_setsockopt_trampoline_addr, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setsockopt_trampoline()
+var libc_setsockopt_trampoline_addr uintptr
 
-//go:linkname libc_setsockopt libc_setsockopt
 //go:cgo_import_dynamic libc_setsockopt setsockopt "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getpeername_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	_, _, e1 := syscall_rawSyscall(libc_getpeername_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getpeername_trampoline()
+var libc_getpeername_trampoline_addr uintptr
 
-//go:linkname libc_getpeername libc_getpeername
 //go:cgo_import_dynamic libc_getpeername getpeername "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getsockname_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	_, _, e1 := syscall_rawSyscall(libc_getsockname_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getsockname_trampoline()
+var libc_getsockname_trampoline_addr uintptr
 
-//go:linkname libc_getsockname libc_getsockname
 //go:cgo_import_dynamic libc_getsockname getsockname "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Shutdown(s int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_shutdown_trampoline), uintptr(s), uintptr(how), 0)
+	_, _, e1 := syscall_syscall(libc_shutdown_trampoline_addr, uintptr(s), uintptr(how), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_shutdown_trampoline()
+var libc_shutdown_trampoline_addr uintptr
 
-//go:linkname libc_shutdown libc_shutdown
 //go:cgo_import_dynamic libc_shutdown shutdown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
-	_, _, e1 := syscall_rawSyscall6(funcPC(libc_socketpair_trampoline), uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
+	_, _, e1 := syscall_rawSyscall6(libc_socketpair_trampoline_addr, uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_socketpair_trampoline()
+var libc_socketpair_trampoline_addr uintptr
 
-//go:linkname libc_socketpair libc_socketpair
 //go:cgo_import_dynamic libc_socketpair socketpair "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -220,7 +208,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_recvfrom_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
+	r0, _, e1 := syscall_syscall6(libc_recvfrom_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -228,9 +216,8 @@
 	return
 }
 
-func libc_recvfrom_trampoline()
+var libc_recvfrom_trampoline_addr uintptr
 
-//go:linkname libc_recvfrom libc_recvfrom
 //go:cgo_import_dynamic libc_recvfrom recvfrom "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -242,22 +229,21 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendto_trampoline), uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
+	_, _, e1 := syscall_syscall6(libc_sendto_trampoline_addr, uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sendto_trampoline()
+var libc_sendto_trampoline_addr uintptr
 
-//go:linkname libc_sendto libc_sendto
 //go:cgo_import_dynamic libc_sendto sendto "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_recvmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_recvmsg_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -265,15 +251,14 @@
 	return
 }
 
-func libc_recvmsg_trampoline()
+var libc_recvmsg_trampoline_addr uintptr
 
-//go:linkname libc_recvmsg libc_recvmsg
 //go:cgo_import_dynamic libc_recvmsg recvmsg "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_sendmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_sendmsg_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -281,15 +266,14 @@
 	return
 }
 
-func libc_sendmsg_trampoline()
+var libc_sendmsg_trampoline_addr uintptr
 
-//go:linkname libc_sendmsg libc_sendmsg
 //go:cgo_import_dynamic libc_sendmsg sendmsg "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func kevent(kq int, change unsafe.Pointer, nchange int, event unsafe.Pointer, nevent int, timeout *Timespec) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_kevent_trampoline), uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
+	r0, _, e1 := syscall_syscall6(libc_kevent_trampoline_addr, uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -297,9 +281,8 @@
 	return
 }
 
-func libc_kevent_trampoline()
+var libc_kevent_trampoline_addr uintptr
 
-//go:linkname libc_kevent libc_kevent
 //go:cgo_import_dynamic libc_kevent kevent "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -310,37 +293,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_utimes_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
+	_, _, e1 := syscall_syscall(libc_utimes_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_utimes_trampoline()
+var libc_utimes_trampoline_addr uintptr
 
-//go:linkname libc_utimes libc_utimes
 //go:cgo_import_dynamic libc_utimes utimes "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func futimes(fd int, timeval *[2]Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_futimes_trampoline), uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
+	_, _, e1 := syscall_syscall(libc_futimes_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_futimes_trampoline()
+var libc_futimes_trampoline_addr uintptr
 
-//go:linkname libc_futimes libc_futimes
 //go:cgo_import_dynamic libc_futimes futimes "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func poll(fds *PollFd, nfds int, timeout int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_poll_trampoline), uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
+	r0, _, e1 := syscall_syscall(libc_poll_trampoline_addr, uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -348,9 +329,8 @@
 	return
 }
 
-func libc_poll_trampoline()
+var libc_poll_trampoline_addr uintptr
 
-//go:linkname libc_poll libc_poll
 //go:cgo_import_dynamic libc_poll poll "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -362,16 +342,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_madvise_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(behav))
+	_, _, e1 := syscall_syscall(libc_madvise_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(behav))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_madvise_trampoline()
+var libc_madvise_trampoline_addr uintptr
 
-//go:linkname libc_madvise libc_madvise
 //go:cgo_import_dynamic libc_madvise madvise "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -383,31 +362,29 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
+	_, _, e1 := syscall_syscall(libc_mlock_trampoline_addr, uintptr(_p0), uintptr(len(b)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mlock_trampoline()
+var libc_mlock_trampoline_addr uintptr
 
-//go:linkname libc_mlock libc_mlock
 //go:cgo_import_dynamic libc_mlock mlock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Mlockall(flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_mlockall_trampoline), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall(libc_mlockall_trampoline_addr, uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mlockall_trampoline()
+var libc_mlockall_trampoline_addr uintptr
 
-//go:linkname libc_mlockall libc_mlockall
 //go:cgo_import_dynamic libc_mlockall mlockall "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -419,16 +396,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mprotect_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(prot))
+	_, _, e1 := syscall_syscall(libc_mprotect_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(prot))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mprotect_trampoline()
+var libc_mprotect_trampoline_addr uintptr
 
-//go:linkname libc_mprotect libc_mprotect
 //go:cgo_import_dynamic libc_mprotect mprotect "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -440,16 +416,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_msync_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_msync_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_msync_trampoline()
+var libc_msync_trampoline_addr uintptr
 
-//go:linkname libc_msync libc_msync
 //go:cgo_import_dynamic libc_msync msync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -461,48 +436,43 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_munlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
+	_, _, e1 := syscall_syscall(libc_munlock_trampoline_addr, uintptr(_p0), uintptr(len(b)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munlock_trampoline()
+var libc_munlock_trampoline_addr uintptr
 
-//go:linkname libc_munlock libc_munlock
 //go:cgo_import_dynamic libc_munlock munlock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Munlockall() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munlockall_trampoline), 0, 0, 0)
+	_, _, e1 := syscall_syscall(libc_munlockall_trampoline_addr, 0, 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munlockall_trampoline()
+var libc_munlockall_trampoline_addr uintptr
 
-//go:linkname libc_munlockall libc_munlockall
 //go:cgo_import_dynamic libc_munlockall munlockall "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func pipe() (r int, w int, err error) {
-	r0, r1, e1 := syscall_rawSyscall(funcPC(libc_pipe_trampoline), 0, 0, 0)
-	r = int(r0)
-	w = int(r1)
+func pipe(p *[2]int32) (err error) {
+	_, _, e1 := syscall_rawSyscall(libc_pipe_trampoline_addr, uintptr(unsafe.Pointer(p)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_pipe_trampoline()
+var libc_pipe_trampoline_addr uintptr
 
-//go:linkname libc_pipe libc_pipe
 //go:cgo_import_dynamic libc_pipe pipe "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -518,7 +488,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_getxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
+	r0, _, e1 := syscall_syscall6(libc_getxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -526,9 +496,8 @@
 	return
 }
 
-func libc_getxattr_trampoline()
+var libc_getxattr_trampoline_addr uintptr
 
-//go:linkname libc_getxattr libc_getxattr
 //go:cgo_import_dynamic libc_getxattr getxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -539,7 +508,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_fgetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
+	r0, _, e1 := syscall_syscall6(libc_fgetxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -547,9 +516,8 @@
 	return
 }
 
-func libc_fgetxattr_trampoline()
+var libc_fgetxattr_trampoline_addr uintptr
 
-//go:linkname libc_fgetxattr libc_fgetxattr
 //go:cgo_import_dynamic libc_fgetxattr fgetxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -565,16 +533,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_setxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
+	_, _, e1 := syscall_syscall6(libc_setxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setxattr_trampoline()
+var libc_setxattr_trampoline_addr uintptr
 
-//go:linkname libc_setxattr libc_setxattr
 //go:cgo_import_dynamic libc_setxattr setxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -585,16 +552,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fsetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
+	_, _, e1 := syscall_syscall6(libc_fsetxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fsetxattr_trampoline()
+var libc_fsetxattr_trampoline_addr uintptr
 
-//go:linkname libc_fsetxattr libc_fsetxattr
 //go:cgo_import_dynamic libc_fsetxattr fsetxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -610,16 +576,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_removexattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_removexattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_removexattr_trampoline()
+var libc_removexattr_trampoline_addr uintptr
 
-//go:linkname libc_removexattr libc_removexattr
 //go:cgo_import_dynamic libc_removexattr removexattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -630,16 +595,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_fremovexattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_fremovexattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fremovexattr_trampoline()
+var libc_fremovexattr_trampoline_addr uintptr
 
-//go:linkname libc_fremovexattr libc_fremovexattr
 //go:cgo_import_dynamic libc_fremovexattr fremovexattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -650,7 +614,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_listxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_listxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -658,15 +622,14 @@
 	return
 }
 
-func libc_listxattr_trampoline()
+var libc_listxattr_trampoline_addr uintptr
 
-//go:linkname libc_listxattr libc_listxattr
 //go:cgo_import_dynamic libc_listxattr listxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func flistxattr(fd int, dest *byte, size int, options int) (sz int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_flistxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_flistxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -674,30 +637,28 @@
 	return
 }
 
-func libc_flistxattr_trampoline()
+var libc_flistxattr_trampoline_addr uintptr
 
-//go:linkname libc_flistxattr libc_flistxattr
 //go:cgo_import_dynamic libc_flistxattr flistxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setattrlist_trampoline), uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
+	_, _, e1 := syscall_syscall6(libc_setattrlist_trampoline_addr, uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setattrlist_trampoline()
+var libc_setattrlist_trampoline_addr uintptr
 
-//go:linkname libc_setattrlist libc_setattrlist
 //go:cgo_import_dynamic libc_setattrlist setattrlist "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func fcntl(fd int, cmd int, arg int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fcntl_trampoline), uintptr(fd), uintptr(cmd), uintptr(arg))
+	r0, _, e1 := syscall_syscall(libc_fcntl_trampoline_addr, uintptr(fd), uintptr(cmd), uintptr(arg))
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -705,39 +666,36 @@
 	return
 }
 
-func libc_fcntl_trampoline()
+var libc_fcntl_trampoline_addr uintptr
 
-//go:linkname libc_fcntl libc_fcntl
 //go:cgo_import_dynamic libc_fcntl fcntl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func kill(pid int, signum int, posix int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_kill_trampoline), uintptr(pid), uintptr(signum), uintptr(posix))
+	_, _, e1 := syscall_syscall(libc_kill_trampoline_addr, uintptr(pid), uintptr(signum), uintptr(posix))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_kill_trampoline()
+var libc_kill_trampoline_addr uintptr
 
-//go:linkname libc_kill libc_kill
 //go:cgo_import_dynamic libc_kill kill "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func ioctl(fd int, req uint, arg uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ioctl_trampoline), uintptr(fd), uintptr(req), uintptr(arg))
+	_, _, e1 := syscall_syscall(libc_ioctl_trampoline_addr, uintptr(fd), uintptr(req), uintptr(arg))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_ioctl_trampoline()
+var libc_ioctl_trampoline_addr uintptr
 
-//go:linkname libc_ioctl libc_ioctl
 //go:cgo_import_dynamic libc_ioctl ioctl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -749,31 +707,29 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sysctl_trampoline), uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
+	_, _, e1 := syscall_syscall6(libc_sysctl_trampoline_addr, uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sysctl_trampoline()
+var libc_sysctl_trampoline_addr uintptr
 
-//go:linkname libc_sysctl libc_sysctl
 //go:cgo_import_dynamic libc_sysctl sysctl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func sendfile(infd int, outfd int, offset int64, len *int64, hdtr unsafe.Pointer, flags int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendfile_trampoline), uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags))
+	_, _, e1 := syscall_syscall6(libc_sendfile_trampoline_addr, uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sendfile_trampoline()
+var libc_sendfile_trampoline_addr uintptr
 
-//go:linkname libc_sendfile libc_sendfile
 //go:cgo_import_dynamic libc_sendfile sendfile "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -784,31 +740,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_access_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_access_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_access_trampoline()
+var libc_access_trampoline_addr uintptr
 
-//go:linkname libc_access libc_access
 //go:cgo_import_dynamic libc_access access "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Adjtime(delta *Timeval, olddelta *Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_adjtime_trampoline), uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
+	_, _, e1 := syscall_syscall(libc_adjtime_trampoline_addr, uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_adjtime_trampoline()
+var libc_adjtime_trampoline_addr uintptr
 
-//go:linkname libc_adjtime libc_adjtime
 //go:cgo_import_dynamic libc_adjtime adjtime "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -819,16 +773,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_chdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chdir_trampoline()
+var libc_chdir_trampoline_addr uintptr
 
-//go:linkname libc_chdir libc_chdir
 //go:cgo_import_dynamic libc_chdir chdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -839,16 +792,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chflags_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_chflags_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chflags_trampoline()
+var libc_chflags_trampoline_addr uintptr
 
-//go:linkname libc_chflags libc_chflags
 //go:cgo_import_dynamic libc_chflags chflags "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -859,16 +811,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chmod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_chmod_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chmod_trampoline()
+var libc_chmod_trampoline_addr uintptr
 
-//go:linkname libc_chmod libc_chmod
 //go:cgo_import_dynamic libc_chmod chmod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -879,16 +830,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_chown_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chown_trampoline()
+var libc_chown_trampoline_addr uintptr
 
-//go:linkname libc_chown libc_chown
 //go:cgo_import_dynamic libc_chown chown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -899,46 +849,43 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chroot_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_chroot_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chroot_trampoline()
+var libc_chroot_trampoline_addr uintptr
 
-//go:linkname libc_chroot libc_chroot
 //go:cgo_import_dynamic libc_chroot chroot "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func ClockGettime(clockid int32, time *Timespec) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_clock_gettime_trampoline), uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
+	_, _, e1 := syscall_syscall(libc_clock_gettime_trampoline_addr, uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clock_gettime_trampoline()
+var libc_clock_gettime_trampoline_addr uintptr
 
-//go:linkname libc_clock_gettime libc_clock_gettime
 //go:cgo_import_dynamic libc_clock_gettime clock_gettime "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Close(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_close_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_close_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_close_trampoline()
+var libc_close_trampoline_addr uintptr
 
-//go:linkname libc_close libc_close
 //go:cgo_import_dynamic libc_close close "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -954,16 +901,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_clonefile_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_clonefile_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clonefile_trampoline()
+var libc_clonefile_trampoline_addr uintptr
 
-//go:linkname libc_clonefile libc_clonefile
 //go:cgo_import_dynamic libc_clonefile clonefile "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -979,22 +925,21 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_clonefileat_trampoline), uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_clonefileat_trampoline_addr, uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clonefileat_trampoline()
+var libc_clonefileat_trampoline_addr uintptr
 
-//go:linkname libc_clonefileat libc_clonefileat
 //go:cgo_import_dynamic libc_clonefileat clonefileat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Dup(fd int) (nfd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_dup_trampoline), uintptr(fd), 0, 0)
+	r0, _, e1 := syscall_syscall(libc_dup_trampoline_addr, uintptr(fd), 0, 0)
 	nfd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1002,24 +947,22 @@
 	return
 }
 
-func libc_dup_trampoline()
+var libc_dup_trampoline_addr uintptr
 
-//go:linkname libc_dup libc_dup
 //go:cgo_import_dynamic libc_dup dup "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Dup2(from int, to int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_dup2_trampoline), uintptr(from), uintptr(to), 0)
+	_, _, e1 := syscall_syscall(libc_dup2_trampoline_addr, uintptr(from), uintptr(to), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_dup2_trampoline()
+var libc_dup2_trampoline_addr uintptr
 
-//go:linkname libc_dup2 libc_dup2
 //go:cgo_import_dynamic libc_dup2 dup2 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1035,28 +978,26 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_exchangedata_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_exchangedata_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_exchangedata_trampoline()
+var libc_exchangedata_trampoline_addr uintptr
 
-//go:linkname libc_exchangedata libc_exchangedata
 //go:cgo_import_dynamic libc_exchangedata exchangedata "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Exit(code int) {
-	syscall_syscall(funcPC(libc_exit_trampoline), uintptr(code), 0, 0)
+	syscall_syscall(libc_exit_trampoline_addr, uintptr(code), 0, 0)
 	return
 }
 
-func libc_exit_trampoline()
+var libc_exit_trampoline_addr uintptr
 
-//go:linkname libc_exit libc_exit
 //go:cgo_import_dynamic libc_exit exit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1067,61 +1008,57 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_faccessat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_faccessat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_faccessat_trampoline()
+var libc_faccessat_trampoline_addr uintptr
 
-//go:linkname libc_faccessat libc_faccessat
 //go:cgo_import_dynamic libc_faccessat faccessat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchdir(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchdir_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_fchdir_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchdir_trampoline()
+var libc_fchdir_trampoline_addr uintptr
 
-//go:linkname libc_fchdir libc_fchdir
 //go:cgo_import_dynamic libc_fchdir fchdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchflags(fd int, flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchflags_trampoline), uintptr(fd), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_fchflags_trampoline_addr, uintptr(fd), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchflags_trampoline()
+var libc_fchflags_trampoline_addr uintptr
 
-//go:linkname libc_fchflags libc_fchflags
 //go:cgo_import_dynamic libc_fchflags fchflags "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchmod(fd int, mode uint32) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchmod_trampoline), uintptr(fd), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_fchmod_trampoline_addr, uintptr(fd), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchmod_trampoline()
+var libc_fchmod_trampoline_addr uintptr
 
-//go:linkname libc_fchmod libc_fchmod
 //go:cgo_import_dynamic libc_fchmod fchmod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1132,31 +1069,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchmodat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fchmodat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchmodat_trampoline()
+var libc_fchmodat_trampoline_addr uintptr
 
-//go:linkname libc_fchmodat libc_fchmodat
 //go:cgo_import_dynamic libc_fchmodat fchmodat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchown(fd int, uid int, gid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchown_trampoline), uintptr(fd), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_fchown_trampoline_addr, uintptr(fd), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchown_trampoline()
+var libc_fchown_trampoline_addr uintptr
 
-//go:linkname libc_fchown libc_fchown
 //go:cgo_import_dynamic libc_fchown fchown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1167,16 +1102,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchownat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_fchownat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchownat_trampoline()
+var libc_fchownat_trampoline_addr uintptr
 
-//go:linkname libc_fchownat libc_fchownat
 //go:cgo_import_dynamic libc_fchownat fchownat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1187,37 +1121,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fclonefileat_trampoline), uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fclonefileat_trampoline_addr, uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fclonefileat_trampoline()
+var libc_fclonefileat_trampoline_addr uintptr
 
-//go:linkname libc_fclonefileat libc_fclonefileat
 //go:cgo_import_dynamic libc_fclonefileat fclonefileat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Flock(fd int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_flock_trampoline), uintptr(fd), uintptr(how), 0)
+	_, _, e1 := syscall_syscall(libc_flock_trampoline_addr, uintptr(fd), uintptr(how), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_flock_trampoline()
+var libc_flock_trampoline_addr uintptr
 
-//go:linkname libc_flock libc_flock
 //go:cgo_import_dynamic libc_flock flock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fpathconf(fd int, name int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fpathconf_trampoline), uintptr(fd), uintptr(name), 0)
+	r0, _, e1 := syscall_syscall(libc_fpathconf_trampoline_addr, uintptr(fd), uintptr(name), 0)
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1225,39 +1157,36 @@
 	return
 }
 
-func libc_fpathconf_trampoline()
+var libc_fpathconf_trampoline_addr uintptr
 
-//go:linkname libc_fpathconf libc_fpathconf
 //go:cgo_import_dynamic libc_fpathconf fpathconf "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fsync(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fsync_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_fsync_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fsync_trampoline()
+var libc_fsync_trampoline_addr uintptr
 
-//go:linkname libc_fsync libc_fsync
 //go:cgo_import_dynamic libc_fsync fsync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Ftruncate(fd int, length int64) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ftruncate_trampoline), uintptr(fd), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_ftruncate_trampoline_addr, uintptr(fd), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_ftruncate_trampoline()
+var libc_ftruncate_trampoline_addr uintptr
 
-//go:linkname libc_ftruncate libc_ftruncate
 //go:cgo_import_dynamic libc_ftruncate ftruncate "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1269,7 +1198,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_getcwd_trampoline), uintptr(_p0), uintptr(len(buf)), 0)
+	r0, _, e1 := syscall_syscall(libc_getcwd_trampoline_addr, uintptr(_p0), uintptr(len(buf)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1277,67 +1206,62 @@
 	return
 }
 
-func libc_getcwd_trampoline()
+var libc_getcwd_trampoline_addr uintptr
 
-//go:linkname libc_getcwd libc_getcwd
 //go:cgo_import_dynamic libc_getcwd getcwd "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getdtablesize() (size int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_getdtablesize_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_syscall(libc_getdtablesize_trampoline_addr, 0, 0, 0)
 	size = int(r0)
 	return
 }
 
-func libc_getdtablesize_trampoline()
+var libc_getdtablesize_trampoline_addr uintptr
 
-//go:linkname libc_getdtablesize libc_getdtablesize
 //go:cgo_import_dynamic libc_getdtablesize getdtablesize "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getegid() (egid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getegid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getegid_trampoline_addr, 0, 0, 0)
 	egid = int(r0)
 	return
 }
 
-func libc_getegid_trampoline()
+var libc_getegid_trampoline_addr uintptr
 
-//go:linkname libc_getegid libc_getegid
 //go:cgo_import_dynamic libc_getegid getegid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Geteuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_geteuid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_geteuid_trampoline_addr, 0, 0, 0)
 	uid = int(r0)
 	return
 }
 
-func libc_geteuid_trampoline()
+var libc_geteuid_trampoline_addr uintptr
 
-//go:linkname libc_geteuid libc_geteuid
 //go:cgo_import_dynamic libc_geteuid geteuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getgid() (gid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getgid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getgid_trampoline_addr, 0, 0, 0)
 	gid = int(r0)
 	return
 }
 
-func libc_getgid_trampoline()
+var libc_getgid_trampoline_addr uintptr
 
-//go:linkname libc_getgid libc_getgid
 //go:cgo_import_dynamic libc_getgid getgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpgid(pid int) (pgid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getpgid_trampoline), uintptr(pid), 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getpgid_trampoline_addr, uintptr(pid), 0, 0)
 	pgid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1345,54 +1269,50 @@
 	return
 }
 
-func libc_getpgid_trampoline()
+var libc_getpgid_trampoline_addr uintptr
 
-//go:linkname libc_getpgid libc_getpgid
 //go:cgo_import_dynamic libc_getpgid getpgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpgrp() (pgrp int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpgrp_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getpgrp_trampoline_addr, 0, 0, 0)
 	pgrp = int(r0)
 	return
 }
 
-func libc_getpgrp_trampoline()
+var libc_getpgrp_trampoline_addr uintptr
 
-//go:linkname libc_getpgrp libc_getpgrp
 //go:cgo_import_dynamic libc_getpgrp getpgrp "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpid() (pid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getpid_trampoline_addr, 0, 0, 0)
 	pid = int(r0)
 	return
 }
 
-func libc_getpid_trampoline()
+var libc_getpid_trampoline_addr uintptr
 
-//go:linkname libc_getpid libc_getpid
 //go:cgo_import_dynamic libc_getpid getpid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getppid() (ppid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getppid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getppid_trampoline_addr, 0, 0, 0)
 	ppid = int(r0)
 	return
 }
 
-func libc_getppid_trampoline()
+var libc_getppid_trampoline_addr uintptr
 
-//go:linkname libc_getppid libc_getppid
 //go:cgo_import_dynamic libc_getppid getppid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpriority(which int, who int) (prio int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getpriority_trampoline), uintptr(which), uintptr(who), 0)
+	r0, _, e1 := syscall_syscall(libc_getpriority_trampoline_addr, uintptr(which), uintptr(who), 0)
 	prio = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1400,45 +1320,42 @@
 	return
 }
 
-func libc_getpriority_trampoline()
+var libc_getpriority_trampoline_addr uintptr
 
-//go:linkname libc_getpriority libc_getpriority
 //go:cgo_import_dynamic libc_getpriority getpriority "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_getrlimit_trampoline_addr, uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getrlimit_trampoline()
+var libc_getrlimit_trampoline_addr uintptr
 
-//go:linkname libc_getrlimit libc_getrlimit
 //go:cgo_import_dynamic libc_getrlimit getrlimit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getrusage(who int, rusage *Rusage) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrusage_trampoline), uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_getrusage_trampoline_addr, uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getrusage_trampoline()
+var libc_getrusage_trampoline_addr uintptr
 
-//go:linkname libc_getrusage libc_getrusage
 //go:cgo_import_dynamic libc_getrusage getrusage "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getsid(pid int) (sid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getsid_trampoline), uintptr(pid), 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getsid_trampoline_addr, uintptr(pid), 0, 0)
 	sid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1446,56 +1363,52 @@
 	return
 }
 
-func libc_getsid_trampoline()
+var libc_getsid_trampoline_addr uintptr
 
-//go:linkname libc_getsid libc_getsid
 //go:cgo_import_dynamic libc_getsid getsid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Gettimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_gettimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_gettimeofday_trampoline_addr, uintptr(unsafe.Pointer(tp)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_gettimeofday_trampoline()
+var libc_gettimeofday_trampoline_addr uintptr
 
-//go:linkname libc_gettimeofday libc_gettimeofday
 //go:cgo_import_dynamic libc_gettimeofday gettimeofday "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getuid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getuid_trampoline_addr, 0, 0, 0)
 	uid = int(r0)
 	return
 }
 
-func libc_getuid_trampoline()
+var libc_getuid_trampoline_addr uintptr
 
-//go:linkname libc_getuid libc_getuid
 //go:cgo_import_dynamic libc_getuid getuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Issetugid() (tainted bool) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_issetugid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_issetugid_trampoline_addr, 0, 0, 0)
 	tainted = bool(r0 != 0)
 	return
 }
 
-func libc_issetugid_trampoline()
+var libc_issetugid_trampoline_addr uintptr
 
-//go:linkname libc_issetugid libc_issetugid
 //go:cgo_import_dynamic libc_issetugid issetugid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Kqueue() (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_kqueue_trampoline), 0, 0, 0)
+	r0, _, e1 := syscall_syscall(libc_kqueue_trampoline_addr, 0, 0, 0)
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1503,9 +1416,8 @@
 	return
 }
 
-func libc_kqueue_trampoline()
+var libc_kqueue_trampoline_addr uintptr
 
-//go:linkname libc_kqueue libc_kqueue
 //go:cgo_import_dynamic libc_kqueue kqueue "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1516,16 +1428,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lchown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_lchown_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_lchown_trampoline()
+var libc_lchown_trampoline_addr uintptr
 
-//go:linkname libc_lchown libc_lchown
 //go:cgo_import_dynamic libc_lchown lchown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1541,16 +1452,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_link_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_link_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_link_trampoline()
+var libc_link_trampoline_addr uintptr
 
-//go:linkname libc_link libc_link
 //go:cgo_import_dynamic libc_link link "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1566,31 +1476,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_linkat_trampoline), uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_linkat_trampoline_addr, uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_linkat_trampoline()
+var libc_linkat_trampoline_addr uintptr
 
-//go:linkname libc_linkat libc_linkat
 //go:cgo_import_dynamic libc_linkat linkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Listen(s int, backlog int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_listen_trampoline), uintptr(s), uintptr(backlog), 0)
+	_, _, e1 := syscall_syscall(libc_listen_trampoline_addr, uintptr(s), uintptr(backlog), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_listen_trampoline()
+var libc_listen_trampoline_addr uintptr
 
-//go:linkname libc_listen libc_listen
 //go:cgo_import_dynamic libc_listen listen "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1601,16 +1509,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdir_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_mkdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkdir_trampoline()
+var libc_mkdir_trampoline_addr uintptr
 
-//go:linkname libc_mkdir libc_mkdir
 //go:cgo_import_dynamic libc_mkdir mkdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1621,16 +1528,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdirat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
+	_, _, e1 := syscall_syscall(libc_mkdirat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkdirat_trampoline()
+var libc_mkdirat_trampoline_addr uintptr
 
-//go:linkname libc_mkdirat libc_mkdirat
 //go:cgo_import_dynamic libc_mkdirat mkdirat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1641,16 +1547,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkfifo_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_mkfifo_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkfifo_trampoline()
+var libc_mkfifo_trampoline_addr uintptr
 
-//go:linkname libc_mkfifo libc_mkfifo
 //go:cgo_import_dynamic libc_mkfifo mkfifo "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1661,16 +1566,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mknod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
+	_, _, e1 := syscall_syscall(libc_mknod_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mknod_trampoline()
+var libc_mknod_trampoline_addr uintptr
 
-//go:linkname libc_mknod libc_mknod
 //go:cgo_import_dynamic libc_mknod mknod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1681,7 +1585,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_open_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
+	r0, _, e1 := syscall_syscall(libc_open_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1689,9 +1593,8 @@
 	return
 }
 
-func libc_open_trampoline()
+var libc_open_trampoline_addr uintptr
 
-//go:linkname libc_open libc_open
 //go:cgo_import_dynamic libc_open open "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1702,7 +1605,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_openat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_openat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1710,9 +1613,8 @@
 	return
 }
 
-func libc_openat_trampoline()
+var libc_openat_trampoline_addr uintptr
 
-//go:linkname libc_openat libc_openat
 //go:cgo_import_dynamic libc_openat openat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1723,7 +1625,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_pathconf_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
+	r0, _, e1 := syscall_syscall(libc_pathconf_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1731,9 +1633,8 @@
 	return
 }
 
-func libc_pathconf_trampoline()
+var libc_pathconf_trampoline_addr uintptr
 
-//go:linkname libc_pathconf libc_pathconf
 //go:cgo_import_dynamic libc_pathconf pathconf "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1745,7 +1646,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pread_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_pread_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1753,9 +1654,8 @@
 	return
 }
 
-func libc_pread_trampoline()
+var libc_pread_trampoline_addr uintptr
 
-//go:linkname libc_pread libc_pread
 //go:cgo_import_dynamic libc_pread pread "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1767,7 +1667,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pwrite_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_pwrite_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1775,9 +1675,8 @@
 	return
 }
 
-func libc_pwrite_trampoline()
+var libc_pwrite_trampoline_addr uintptr
 
-//go:linkname libc_pwrite libc_pwrite
 //go:cgo_import_dynamic libc_pwrite pwrite "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1789,7 +1688,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	r0, _, e1 := syscall_syscall(libc_read_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1797,9 +1696,8 @@
 	return
 }
 
-func libc_read_trampoline()
+var libc_read_trampoline_addr uintptr
 
-//go:linkname libc_read libc_read
 //go:cgo_import_dynamic libc_read read "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1816,7 +1714,7 @@
 	} else {
 		_p1 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_readlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
+	r0, _, e1 := syscall_syscall(libc_readlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1824,9 +1722,8 @@
 	return
 }
 
-func libc_readlink_trampoline()
+var libc_readlink_trampoline_addr uintptr
 
-//go:linkname libc_readlink libc_readlink
 //go:cgo_import_dynamic libc_readlink readlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1843,7 +1740,7 @@
 	} else {
 		_p1 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_readlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_readlinkat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1851,9 +1748,8 @@
 	return
 }
 
-func libc_readlinkat_trampoline()
+var libc_readlinkat_trampoline_addr uintptr
 
-//go:linkname libc_readlinkat libc_readlinkat
 //go:cgo_import_dynamic libc_readlinkat readlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1869,16 +1765,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rename_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_rename_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_rename_trampoline()
+var libc_rename_trampoline_addr uintptr
 
-//go:linkname libc_rename libc_rename
 //go:cgo_import_dynamic libc_rename rename "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1894,16 +1789,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_renameat_trampoline), uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_renameat_trampoline_addr, uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_renameat_trampoline()
+var libc_renameat_trampoline_addr uintptr
 
-//go:linkname libc_renameat libc_renameat
 //go:cgo_import_dynamic libc_renameat renameat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1914,16 +1808,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_revoke_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_revoke_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_revoke_trampoline()
+var libc_revoke_trampoline_addr uintptr
 
-//go:linkname libc_revoke libc_revoke
 //go:cgo_import_dynamic libc_revoke revoke "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1934,22 +1827,21 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rmdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_rmdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_rmdir_trampoline()
+var libc_rmdir_trampoline_addr uintptr
 
-//go:linkname libc_rmdir libc_rmdir
 //go:cgo_import_dynamic libc_rmdir rmdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Seek(fd int, offset int64, whence int) (newoffset int64, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_lseek_trampoline), uintptr(fd), uintptr(offset), uintptr(whence))
+	r0, _, e1 := syscall_syscall(libc_lseek_trampoline_addr, uintptr(fd), uintptr(offset), uintptr(whence))
 	newoffset = int64(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1957,15 +1849,14 @@
 	return
 }
 
-func libc_lseek_trampoline()
+var libc_lseek_trampoline_addr uintptr
 
-//go:linkname libc_lseek libc_lseek
 //go:cgo_import_dynamic libc_lseek lseek "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_select_trampoline), uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
+	r0, _, e1 := syscall_syscall6(libc_select_trampoline_addr, uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1973,54 +1864,50 @@
 	return
 }
 
-func libc_select_trampoline()
+var libc_select_trampoline_addr uintptr
 
-//go:linkname libc_select libc_select
 //go:cgo_import_dynamic libc_select select "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setegid(egid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setegid_trampoline), uintptr(egid), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setegid_trampoline_addr, uintptr(egid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setegid_trampoline()
+var libc_setegid_trampoline_addr uintptr
 
-//go:linkname libc_setegid libc_setegid
 //go:cgo_import_dynamic libc_setegid setegid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Seteuid(euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_seteuid_trampoline), uintptr(euid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_seteuid_trampoline_addr, uintptr(euid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_seteuid_trampoline()
+var libc_seteuid_trampoline_addr uintptr
 
-//go:linkname libc_seteuid libc_seteuid
 //go:cgo_import_dynamic libc_seteuid seteuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setgid(gid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgid_trampoline), uintptr(gid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_setgid_trampoline_addr, uintptr(gid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setgid_trampoline()
+var libc_setgid_trampoline_addr uintptr
 
-//go:linkname libc_setgid libc_setgid
 //go:cgo_import_dynamic libc_setgid setgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2031,112 +1918,105 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_setlogin_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setlogin_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setlogin_trampoline()
+var libc_setlogin_trampoline_addr uintptr
 
-//go:linkname libc_setlogin libc_setlogin
 //go:cgo_import_dynamic libc_setlogin setlogin "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setpgid(pid int, pgid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setpgid_trampoline), uintptr(pid), uintptr(pgid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setpgid_trampoline_addr, uintptr(pid), uintptr(pgid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setpgid_trampoline()
+var libc_setpgid_trampoline_addr uintptr
 
-//go:linkname libc_setpgid libc_setpgid
 //go:cgo_import_dynamic libc_setpgid setpgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setpriority(which int, who int, prio int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setpriority_trampoline), uintptr(which), uintptr(who), uintptr(prio))
+	_, _, e1 := syscall_syscall(libc_setpriority_trampoline_addr, uintptr(which), uintptr(who), uintptr(prio))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setpriority_trampoline()
+var libc_setpriority_trampoline_addr uintptr
 
-//go:linkname libc_setpriority libc_setpriority
 //go:cgo_import_dynamic libc_setpriority setpriority "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setprivexec(flag int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setprivexec_trampoline), uintptr(flag), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setprivexec_trampoline_addr, uintptr(flag), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setprivexec_trampoline()
+var libc_setprivexec_trampoline_addr uintptr
 
-//go:linkname libc_setprivexec libc_setprivexec
 //go:cgo_import_dynamic libc_setprivexec setprivexec "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setregid(rgid int, egid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setregid_trampoline), uintptr(rgid), uintptr(egid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setregid_trampoline_addr, uintptr(rgid), uintptr(egid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setregid_trampoline()
+var libc_setregid_trampoline_addr uintptr
 
-//go:linkname libc_setregid libc_setregid
 //go:cgo_import_dynamic libc_setregid setregid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setreuid(ruid int, euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setreuid_trampoline), uintptr(ruid), uintptr(euid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setreuid_trampoline_addr, uintptr(ruid), uintptr(euid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setreuid_trampoline()
+var libc_setreuid_trampoline_addr uintptr
 
-//go:linkname libc_setreuid libc_setreuid
 //go:cgo_import_dynamic libc_setreuid setreuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setrlimit_trampoline_addr, uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setrlimit_trampoline()
+var libc_setrlimit_trampoline_addr uintptr
 
-//go:linkname libc_setrlimit libc_setrlimit
 //go:cgo_import_dynamic libc_setrlimit setrlimit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setsid() (pid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_setsid_trampoline), 0, 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_setsid_trampoline_addr, 0, 0, 0)
 	pid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2144,39 +2024,36 @@
 	return
 }
 
-func libc_setsid_trampoline()
+var libc_setsid_trampoline_addr uintptr
 
-//go:linkname libc_setsid libc_setsid
 //go:cgo_import_dynamic libc_setsid setsid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Settimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_settimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_settimeofday_trampoline_addr, uintptr(unsafe.Pointer(tp)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_settimeofday_trampoline()
+var libc_settimeofday_trampoline_addr uintptr
 
-//go:linkname libc_settimeofday libc_settimeofday
 //go:cgo_import_dynamic libc_settimeofday settimeofday "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setuid(uid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setuid_trampoline), uintptr(uid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_setuid_trampoline_addr, uintptr(uid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setuid_trampoline()
+var libc_setuid_trampoline_addr uintptr
 
-//go:linkname libc_setuid libc_setuid
 //go:cgo_import_dynamic libc_setuid setuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2192,16 +2069,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_symlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_symlink_trampoline()
+var libc_symlink_trampoline_addr uintptr
 
-//go:linkname libc_symlink libc_symlink
 //go:cgo_import_dynamic libc_symlink symlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2217,31 +2093,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlinkat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
+	_, _, e1 := syscall_syscall(libc_symlinkat_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_symlinkat_trampoline()
+var libc_symlinkat_trampoline_addr uintptr
 
-//go:linkname libc_symlinkat libc_symlinkat
 //go:cgo_import_dynamic libc_symlinkat symlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Sync() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_sync_trampoline), 0, 0, 0)
+	_, _, e1 := syscall_syscall(libc_sync_trampoline_addr, 0, 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sync_trampoline()
+var libc_sync_trampoline_addr uintptr
 
-//go:linkname libc_sync libc_sync
 //go:cgo_import_dynamic libc_sync sync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2252,29 +2126,27 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_truncate_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_truncate_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_truncate_trampoline()
+var libc_truncate_trampoline_addr uintptr
 
-//go:linkname libc_truncate libc_truncate
 //go:cgo_import_dynamic libc_truncate truncate "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Umask(newmask int) (oldmask int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_umask_trampoline), uintptr(newmask), 0, 0)
+	r0, _, _ := syscall_syscall(libc_umask_trampoline_addr, uintptr(newmask), 0, 0)
 	oldmask = int(r0)
 	return
 }
 
-func libc_umask_trampoline()
+var libc_umask_trampoline_addr uintptr
 
-//go:linkname libc_umask libc_umask
 //go:cgo_import_dynamic libc_umask umask "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2285,16 +2157,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_undelete_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_undelete_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_undelete_trampoline()
+var libc_undelete_trampoline_addr uintptr
 
-//go:linkname libc_undelete libc_undelete
 //go:cgo_import_dynamic libc_undelete undelete "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2305,16 +2176,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlink_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_unlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unlink_trampoline()
+var libc_unlink_trampoline_addr uintptr
 
-//go:linkname libc_unlink libc_unlink
 //go:cgo_import_dynamic libc_unlink unlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2325,16 +2195,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_unlinkat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unlinkat_trampoline()
+var libc_unlinkat_trampoline_addr uintptr
 
-//go:linkname libc_unlinkat libc_unlinkat
 //go:cgo_import_dynamic libc_unlinkat unlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2345,16 +2214,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unmount_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_unmount_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unmount_trampoline()
+var libc_unmount_trampoline_addr uintptr
 
-//go:linkname libc_unmount libc_unmount
 //go:cgo_import_dynamic libc_unmount unmount "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2366,7 +2234,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	r0, _, e1 := syscall_syscall(libc_write_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2374,15 +2242,14 @@
 	return
 }
 
-func libc_write_trampoline()
+var libc_write_trampoline_addr uintptr
 
-//go:linkname libc_write libc_write
 //go:cgo_import_dynamic libc_write write "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_mmap_trampoline), uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos))
+	r0, _, e1 := syscall_syscall6(libc_mmap_trampoline_addr, uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos))
 	ret = uintptr(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2390,30 +2257,28 @@
 	return
 }
 
-func libc_mmap_trampoline()
+var libc_mmap_trampoline_addr uintptr
 
-//go:linkname libc_mmap libc_mmap
 //go:cgo_import_dynamic libc_mmap mmap "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func munmap(addr uintptr, length uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munmap_trampoline), uintptr(addr), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_munmap_trampoline_addr, uintptr(addr), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munmap_trampoline()
+var libc_munmap_trampoline_addr uintptr
 
-//go:linkname libc_munmap libc_munmap
 //go:cgo_import_dynamic libc_munmap munmap "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func readlen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
+	r0, _, e1 := syscall_syscall(libc_read_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2424,7 +2289,7 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func writelen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
+	r0, _, e1 := syscall_syscall(libc_write_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2434,32 +2299,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func ptrace(request int, pid int, addr uintptr, data uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_ptrace_trampoline), uintptr(request), uintptr(pid), uintptr(addr), uintptr(data), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ptrace_trampoline()
-
-//go:linkname libc_ptrace libc_ptrace
-//go:cgo_import_dynamic libc_ptrace ptrace "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
 func Fstat(fd int, stat *Stat_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstat64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_fstat64_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstat64_trampoline()
+var libc_fstat64_trampoline_addr uintptr
 
-//go:linkname libc_fstat64 libc_fstat64
 //go:cgo_import_dynamic libc_fstat64 fstat64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2470,37 +2319,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fstatat64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fstatat64_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstatat64_trampoline()
+var libc_fstatat64_trampoline_addr uintptr
 
-//go:linkname libc_fstatat64 libc_fstatat64
 //go:cgo_import_dynamic libc_fstatat64 fstatat64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fstatfs(fd int, stat *Statfs_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstatfs64_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_fstatfs64_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstatfs64_trampoline()
+var libc_fstatfs64_trampoline_addr uintptr
 
-//go:linkname libc_fstatfs64 libc_fstatfs64
 //go:cgo_import_dynamic libc_fstatfs64 fstatfs64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getfsstat64_trampoline), uintptr(buf), uintptr(size), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_getfsstat64_trampoline_addr, uintptr(buf), uintptr(size), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2508,9 +2355,8 @@
 	return
 }
 
-func libc_getfsstat64_trampoline()
+var libc_getfsstat64_trampoline_addr uintptr
 
-//go:linkname libc_getfsstat64 libc_getfsstat64
 //go:cgo_import_dynamic libc_getfsstat64 getfsstat64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2521,36 +2367,48 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lstat64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_lstat64_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_lstat64_trampoline()
+var libc_lstat64_trampoline_addr uintptr
 
-//go:linkname libc_lstat64 libc_lstat64
 //go:cgo_import_dynamic libc_lstat64 lstat64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func ptrace1(request int, pid int, addr uintptr, data uintptr) (err error) {
+	_, _, e1 := syscall_syscall6(libc_ptrace_trampoline_addr, uintptr(request), uintptr(pid), uintptr(addr), uintptr(data), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+var libc_ptrace_trampoline_addr uintptr
+
+//go:cgo_import_dynamic libc_ptrace ptrace "/usr/lib/libSystem.B.dylib"
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Stat(path string, stat *Stat_t) (err error) {
 	var _p0 *byte
 	_p0, err = BytePtrFromString(path)
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_stat64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_stat64_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_stat64_trampoline()
+var libc_stat64_trampoline_addr uintptr
 
-//go:linkname libc_stat64 libc_stat64
 //go:cgo_import_dynamic libc_stat64 stat64 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2561,14 +2419,13 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_statfs64_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_statfs64_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_statfs64_trampoline()
+var libc_statfs64_trampoline_addr uintptr
 
-//go:linkname libc_statfs64 libc_statfs64
 //go:cgo_import_dynamic libc_statfs64 statfs64 "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.s
index 887fd5f..bc169c2 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.s
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_amd64.s
@@ -1,290 +1,859 @@
 // go run mkasm_darwin.go amd64
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build go1.12
 // +build go1.12
 
 #include "textflag.h"
-TEXT ·libc_getgroups_trampoline(SB),NOSPLIT,$0-0
+
+TEXT libc_getgroups_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getgroups(SB)
-TEXT ·libc_setgroups_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getgroups_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getgroups_trampoline_addr(SB)/8, $libc_getgroups_trampoline<>(SB)
+
+TEXT libc_setgroups_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setgroups(SB)
-TEXT ·libc_wait4_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setgroups_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setgroups_trampoline_addr(SB)/8, $libc_setgroups_trampoline<>(SB)
+
+TEXT libc_wait4_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_wait4(SB)
-TEXT ·libc_accept_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_wait4_trampoline_addr(SB), RODATA, $8
+DATA	·libc_wait4_trampoline_addr(SB)/8, $libc_wait4_trampoline<>(SB)
+
+TEXT libc_accept_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_accept(SB)
-TEXT ·libc_bind_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_accept_trampoline_addr(SB), RODATA, $8
+DATA	·libc_accept_trampoline_addr(SB)/8, $libc_accept_trampoline<>(SB)
+
+TEXT libc_bind_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_bind(SB)
-TEXT ·libc_connect_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_bind_trampoline_addr(SB), RODATA, $8
+DATA	·libc_bind_trampoline_addr(SB)/8, $libc_bind_trampoline<>(SB)
+
+TEXT libc_connect_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_connect(SB)
-TEXT ·libc_socket_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_connect_trampoline_addr(SB), RODATA, $8
+DATA	·libc_connect_trampoline_addr(SB)/8, $libc_connect_trampoline<>(SB)
+
+TEXT libc_socket_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_socket(SB)
-TEXT ·libc_getsockopt_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_socket_trampoline_addr(SB), RODATA, $8
+DATA	·libc_socket_trampoline_addr(SB)/8, $libc_socket_trampoline<>(SB)
+
+TEXT libc_getsockopt_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsockopt(SB)
-TEXT ·libc_setsockopt_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsockopt_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsockopt_trampoline_addr(SB)/8, $libc_getsockopt_trampoline<>(SB)
+
+TEXT libc_setsockopt_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setsockopt(SB)
-TEXT ·libc_getpeername_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setsockopt_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setsockopt_trampoline_addr(SB)/8, $libc_setsockopt_trampoline<>(SB)
+
+TEXT libc_getpeername_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpeername(SB)
-TEXT ·libc_getsockname_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpeername_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpeername_trampoline_addr(SB)/8, $libc_getpeername_trampoline<>(SB)
+
+TEXT libc_getsockname_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsockname(SB)
-TEXT ·libc_shutdown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsockname_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsockname_trampoline_addr(SB)/8, $libc_getsockname_trampoline<>(SB)
+
+TEXT libc_shutdown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_shutdown(SB)
-TEXT ·libc_socketpair_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_shutdown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_shutdown_trampoline_addr(SB)/8, $libc_shutdown_trampoline<>(SB)
+
+TEXT libc_socketpair_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_socketpair(SB)
-TEXT ·libc_recvfrom_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_socketpair_trampoline_addr(SB), RODATA, $8
+DATA	·libc_socketpair_trampoline_addr(SB)/8, $libc_socketpair_trampoline<>(SB)
+
+TEXT libc_recvfrom_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_recvfrom(SB)
-TEXT ·libc_sendto_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_recvfrom_trampoline_addr(SB), RODATA, $8
+DATA	·libc_recvfrom_trampoline_addr(SB)/8, $libc_recvfrom_trampoline<>(SB)
+
+TEXT libc_sendto_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendto(SB)
-TEXT ·libc_recvmsg_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendto_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendto_trampoline_addr(SB)/8, $libc_sendto_trampoline<>(SB)
+
+TEXT libc_recvmsg_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_recvmsg(SB)
-TEXT ·libc_sendmsg_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_recvmsg_trampoline_addr(SB), RODATA, $8
+DATA	·libc_recvmsg_trampoline_addr(SB)/8, $libc_recvmsg_trampoline<>(SB)
+
+TEXT libc_sendmsg_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendmsg(SB)
-TEXT ·libc_kevent_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendmsg_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendmsg_trampoline_addr(SB)/8, $libc_sendmsg_trampoline<>(SB)
+
+TEXT libc_kevent_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kevent(SB)
-TEXT ·libc_utimes_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kevent_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kevent_trampoline_addr(SB)/8, $libc_kevent_trampoline<>(SB)
+
+TEXT libc_utimes_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_utimes(SB)
-TEXT ·libc_futimes_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_utimes_trampoline_addr(SB), RODATA, $8
+DATA	·libc_utimes_trampoline_addr(SB)/8, $libc_utimes_trampoline<>(SB)
+
+TEXT libc_futimes_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_futimes(SB)
-TEXT ·libc_poll_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_futimes_trampoline_addr(SB), RODATA, $8
+DATA	·libc_futimes_trampoline_addr(SB)/8, $libc_futimes_trampoline<>(SB)
+
+TEXT libc_poll_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_poll(SB)
-TEXT ·libc_madvise_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_poll_trampoline_addr(SB), RODATA, $8
+DATA	·libc_poll_trampoline_addr(SB)/8, $libc_poll_trampoline<>(SB)
+
+TEXT libc_madvise_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_madvise(SB)
-TEXT ·libc_mlock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_madvise_trampoline_addr(SB), RODATA, $8
+DATA	·libc_madvise_trampoline_addr(SB)/8, $libc_madvise_trampoline<>(SB)
+
+TEXT libc_mlock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mlock(SB)
-TEXT ·libc_mlockall_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mlock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mlock_trampoline_addr(SB)/8, $libc_mlock_trampoline<>(SB)
+
+TEXT libc_mlockall_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mlockall(SB)
-TEXT ·libc_mprotect_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mlockall_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mlockall_trampoline_addr(SB)/8, $libc_mlockall_trampoline<>(SB)
+
+TEXT libc_mprotect_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mprotect(SB)
-TEXT ·libc_msync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mprotect_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mprotect_trampoline_addr(SB)/8, $libc_mprotect_trampoline<>(SB)
+
+TEXT libc_msync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_msync(SB)
-TEXT ·libc_munlock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_msync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_msync_trampoline_addr(SB)/8, $libc_msync_trampoline<>(SB)
+
+TEXT libc_munlock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munlock(SB)
-TEXT ·libc_munlockall_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munlock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munlock_trampoline_addr(SB)/8, $libc_munlock_trampoline<>(SB)
+
+TEXT libc_munlockall_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munlockall(SB)
-TEXT ·libc_pipe_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munlockall_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munlockall_trampoline_addr(SB)/8, $libc_munlockall_trampoline<>(SB)
+
+TEXT libc_pipe_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pipe(SB)
-TEXT ·libc_getxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pipe_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pipe_trampoline_addr(SB)/8, $libc_pipe_trampoline<>(SB)
+
+TEXT libc_getxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getxattr(SB)
-TEXT ·libc_fgetxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getxattr_trampoline_addr(SB)/8, $libc_getxattr_trampoline<>(SB)
+
+TEXT libc_fgetxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fgetxattr(SB)
-TEXT ·libc_setxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fgetxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fgetxattr_trampoline_addr(SB)/8, $libc_fgetxattr_trampoline<>(SB)
+
+TEXT libc_setxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setxattr(SB)
-TEXT ·libc_fsetxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setxattr_trampoline_addr(SB)/8, $libc_setxattr_trampoline<>(SB)
+
+TEXT libc_fsetxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fsetxattr(SB)
-TEXT ·libc_removexattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fsetxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fsetxattr_trampoline_addr(SB)/8, $libc_fsetxattr_trampoline<>(SB)
+
+TEXT libc_removexattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_removexattr(SB)
-TEXT ·libc_fremovexattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_removexattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_removexattr_trampoline_addr(SB)/8, $libc_removexattr_trampoline<>(SB)
+
+TEXT libc_fremovexattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fremovexattr(SB)
-TEXT ·libc_listxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fremovexattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fremovexattr_trampoline_addr(SB)/8, $libc_fremovexattr_trampoline<>(SB)
+
+TEXT libc_listxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_listxattr(SB)
-TEXT ·libc_flistxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_listxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_listxattr_trampoline_addr(SB)/8, $libc_listxattr_trampoline<>(SB)
+
+TEXT libc_flistxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_flistxattr(SB)
-TEXT ·libc_setattrlist_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_flistxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_flistxattr_trampoline_addr(SB)/8, $libc_flistxattr_trampoline<>(SB)
+
+TEXT libc_setattrlist_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setattrlist(SB)
-TEXT ·libc_fcntl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setattrlist_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setattrlist_trampoline_addr(SB)/8, $libc_setattrlist_trampoline<>(SB)
+
+TEXT libc_fcntl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fcntl(SB)
-TEXT ·libc_kill_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fcntl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fcntl_trampoline_addr(SB)/8, $libc_fcntl_trampoline<>(SB)
+
+TEXT libc_kill_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kill(SB)
-TEXT ·libc_ioctl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kill_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kill_trampoline_addr(SB)/8, $libc_kill_trampoline<>(SB)
+
+TEXT libc_ioctl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_ioctl(SB)
-TEXT ·libc_sysctl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_ioctl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ioctl_trampoline_addr(SB)/8, $libc_ioctl_trampoline<>(SB)
+
+TEXT libc_sysctl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sysctl(SB)
-TEXT ·libc_sendfile_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sysctl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sysctl_trampoline_addr(SB)/8, $libc_sysctl_trampoline<>(SB)
+
+TEXT libc_sendfile_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendfile(SB)
-TEXT ·libc_access_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendfile_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendfile_trampoline_addr(SB)/8, $libc_sendfile_trampoline<>(SB)
+
+TEXT libc_access_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_access(SB)
-TEXT ·libc_adjtime_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_access_trampoline_addr(SB), RODATA, $8
+DATA	·libc_access_trampoline_addr(SB)/8, $libc_access_trampoline<>(SB)
+
+TEXT libc_adjtime_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_adjtime(SB)
-TEXT ·libc_chdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_adjtime_trampoline_addr(SB), RODATA, $8
+DATA	·libc_adjtime_trampoline_addr(SB)/8, $libc_adjtime_trampoline<>(SB)
+
+TEXT libc_chdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chdir(SB)
-TEXT ·libc_chflags_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chdir_trampoline_addr(SB)/8, $libc_chdir_trampoline<>(SB)
+
+TEXT libc_chflags_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chflags(SB)
-TEXT ·libc_chmod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chflags_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chflags_trampoline_addr(SB)/8, $libc_chflags_trampoline<>(SB)
+
+TEXT libc_chmod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chmod(SB)
-TEXT ·libc_chown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chmod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chmod_trampoline_addr(SB)/8, $libc_chmod_trampoline<>(SB)
+
+TEXT libc_chown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chown(SB)
-TEXT ·libc_chroot_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chown_trampoline_addr(SB)/8, $libc_chown_trampoline<>(SB)
+
+TEXT libc_chroot_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chroot(SB)
-TEXT ·libc_clock_gettime_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chroot_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chroot_trampoline_addr(SB)/8, $libc_chroot_trampoline<>(SB)
+
+TEXT libc_clock_gettime_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clock_gettime(SB)
-TEXT ·libc_close_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clock_gettime_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clock_gettime_trampoline_addr(SB)/8, $libc_clock_gettime_trampoline<>(SB)
+
+TEXT libc_close_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_close(SB)
-TEXT ·libc_clonefile_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_close_trampoline_addr(SB), RODATA, $8
+DATA	·libc_close_trampoline_addr(SB)/8, $libc_close_trampoline<>(SB)
+
+TEXT libc_clonefile_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clonefile(SB)
-TEXT ·libc_clonefileat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clonefile_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clonefile_trampoline_addr(SB)/8, $libc_clonefile_trampoline<>(SB)
+
+TEXT libc_clonefileat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clonefileat(SB)
-TEXT ·libc_dup_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clonefileat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clonefileat_trampoline_addr(SB)/8, $libc_clonefileat_trampoline<>(SB)
+
+TEXT libc_dup_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_dup(SB)
-TEXT ·libc_dup2_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_dup_trampoline_addr(SB), RODATA, $8
+DATA	·libc_dup_trampoline_addr(SB)/8, $libc_dup_trampoline<>(SB)
+
+TEXT libc_dup2_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_dup2(SB)
-TEXT ·libc_exchangedata_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_dup2_trampoline_addr(SB), RODATA, $8
+DATA	·libc_dup2_trampoline_addr(SB)/8, $libc_dup2_trampoline<>(SB)
+
+TEXT libc_exchangedata_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_exchangedata(SB)
-TEXT ·libc_exit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_exchangedata_trampoline_addr(SB), RODATA, $8
+DATA	·libc_exchangedata_trampoline_addr(SB)/8, $libc_exchangedata_trampoline<>(SB)
+
+TEXT libc_exit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_exit(SB)
-TEXT ·libc_faccessat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_exit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_exit_trampoline_addr(SB)/8, $libc_exit_trampoline<>(SB)
+
+TEXT libc_faccessat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_faccessat(SB)
-TEXT ·libc_fchdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_faccessat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_faccessat_trampoline_addr(SB)/8, $libc_faccessat_trampoline<>(SB)
+
+TEXT libc_fchdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchdir(SB)
-TEXT ·libc_fchflags_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchdir_trampoline_addr(SB)/8, $libc_fchdir_trampoline<>(SB)
+
+TEXT libc_fchflags_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchflags(SB)
-TEXT ·libc_fchmod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchflags_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchflags_trampoline_addr(SB)/8, $libc_fchflags_trampoline<>(SB)
+
+TEXT libc_fchmod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchmod(SB)
-TEXT ·libc_fchmodat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchmod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchmod_trampoline_addr(SB)/8, $libc_fchmod_trampoline<>(SB)
+
+TEXT libc_fchmodat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchmodat(SB)
-TEXT ·libc_fchown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchmodat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchmodat_trampoline_addr(SB)/8, $libc_fchmodat_trampoline<>(SB)
+
+TEXT libc_fchown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchown(SB)
-TEXT ·libc_fchownat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchown_trampoline_addr(SB)/8, $libc_fchown_trampoline<>(SB)
+
+TEXT libc_fchownat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchownat(SB)
-TEXT ·libc_fclonefileat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchownat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchownat_trampoline_addr(SB)/8, $libc_fchownat_trampoline<>(SB)
+
+TEXT libc_fclonefileat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fclonefileat(SB)
-TEXT ·libc_flock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fclonefileat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fclonefileat_trampoline_addr(SB)/8, $libc_fclonefileat_trampoline<>(SB)
+
+TEXT libc_flock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_flock(SB)
-TEXT ·libc_fpathconf_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_flock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_flock_trampoline_addr(SB)/8, $libc_flock_trampoline<>(SB)
+
+TEXT libc_fpathconf_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fpathconf(SB)
-TEXT ·libc_fsync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fpathconf_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fpathconf_trampoline_addr(SB)/8, $libc_fpathconf_trampoline<>(SB)
+
+TEXT libc_fsync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fsync(SB)
-TEXT ·libc_ftruncate_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fsync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fsync_trampoline_addr(SB)/8, $libc_fsync_trampoline<>(SB)
+
+TEXT libc_ftruncate_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_ftruncate(SB)
-TEXT ·libc_getcwd_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_ftruncate_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ftruncate_trampoline_addr(SB)/8, $libc_ftruncate_trampoline<>(SB)
+
+TEXT libc_getcwd_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getcwd(SB)
-TEXT ·libc_getdtablesize_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getcwd_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getcwd_trampoline_addr(SB)/8, $libc_getcwd_trampoline<>(SB)
+
+TEXT libc_getdtablesize_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getdtablesize(SB)
-TEXT ·libc_getegid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getdtablesize_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getdtablesize_trampoline_addr(SB)/8, $libc_getdtablesize_trampoline<>(SB)
+
+TEXT libc_getegid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getegid(SB)
-TEXT ·libc_geteuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getegid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getegid_trampoline_addr(SB)/8, $libc_getegid_trampoline<>(SB)
+
+TEXT libc_geteuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_geteuid(SB)
-TEXT ·libc_getgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_geteuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_geteuid_trampoline_addr(SB)/8, $libc_geteuid_trampoline<>(SB)
+
+TEXT libc_getgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getgid(SB)
-TEXT ·libc_getpgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getgid_trampoline_addr(SB)/8, $libc_getgid_trampoline<>(SB)
+
+TEXT libc_getpgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpgid(SB)
-TEXT ·libc_getpgrp_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpgid_trampoline_addr(SB)/8, $libc_getpgid_trampoline<>(SB)
+
+TEXT libc_getpgrp_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpgrp(SB)
-TEXT ·libc_getpid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpgrp_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpgrp_trampoline_addr(SB)/8, $libc_getpgrp_trampoline<>(SB)
+
+TEXT libc_getpid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpid(SB)
-TEXT ·libc_getppid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpid_trampoline_addr(SB)/8, $libc_getpid_trampoline<>(SB)
+
+TEXT libc_getppid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getppid(SB)
-TEXT ·libc_getpriority_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getppid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getppid_trampoline_addr(SB)/8, $libc_getppid_trampoline<>(SB)
+
+TEXT libc_getpriority_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpriority(SB)
-TEXT ·libc_getrlimit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpriority_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpriority_trampoline_addr(SB)/8, $libc_getpriority_trampoline<>(SB)
+
+TEXT libc_getrlimit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getrlimit(SB)
-TEXT ·libc_getrusage_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getrlimit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getrlimit_trampoline_addr(SB)/8, $libc_getrlimit_trampoline<>(SB)
+
+TEXT libc_getrusage_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getrusage(SB)
-TEXT ·libc_getsid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getrusage_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getrusage_trampoline_addr(SB)/8, $libc_getrusage_trampoline<>(SB)
+
+TEXT libc_getsid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsid(SB)
-TEXT ·libc_gettimeofday_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsid_trampoline_addr(SB)/8, $libc_getsid_trampoline<>(SB)
+
+TEXT libc_gettimeofday_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_gettimeofday(SB)
-TEXT ·libc_getuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_gettimeofday_trampoline_addr(SB), RODATA, $8
+DATA	·libc_gettimeofday_trampoline_addr(SB)/8, $libc_gettimeofday_trampoline<>(SB)
+
+TEXT libc_getuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getuid(SB)
-TEXT ·libc_issetugid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getuid_trampoline_addr(SB)/8, $libc_getuid_trampoline<>(SB)
+
+TEXT libc_issetugid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_issetugid(SB)
-TEXT ·libc_kqueue_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_issetugid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_issetugid_trampoline_addr(SB)/8, $libc_issetugid_trampoline<>(SB)
+
+TEXT libc_kqueue_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kqueue(SB)
-TEXT ·libc_lchown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kqueue_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kqueue_trampoline_addr(SB)/8, $libc_kqueue_trampoline<>(SB)
+
+TEXT libc_lchown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lchown(SB)
-TEXT ·libc_link_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lchown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lchown_trampoline_addr(SB)/8, $libc_lchown_trampoline<>(SB)
+
+TEXT libc_link_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_link(SB)
-TEXT ·libc_linkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_link_trampoline_addr(SB), RODATA, $8
+DATA	·libc_link_trampoline_addr(SB)/8, $libc_link_trampoline<>(SB)
+
+TEXT libc_linkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_linkat(SB)
-TEXT ·libc_listen_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_linkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_linkat_trampoline_addr(SB)/8, $libc_linkat_trampoline<>(SB)
+
+TEXT libc_listen_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_listen(SB)
-TEXT ·libc_mkdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_listen_trampoline_addr(SB), RODATA, $8
+DATA	·libc_listen_trampoline_addr(SB)/8, $libc_listen_trampoline<>(SB)
+
+TEXT libc_mkdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkdir(SB)
-TEXT ·libc_mkdirat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkdir_trampoline_addr(SB)/8, $libc_mkdir_trampoline<>(SB)
+
+TEXT libc_mkdirat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkdirat(SB)
-TEXT ·libc_mkfifo_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkdirat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkdirat_trampoline_addr(SB)/8, $libc_mkdirat_trampoline<>(SB)
+
+TEXT libc_mkfifo_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkfifo(SB)
-TEXT ·libc_mknod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkfifo_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkfifo_trampoline_addr(SB)/8, $libc_mkfifo_trampoline<>(SB)
+
+TEXT libc_mknod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mknod(SB)
-TEXT ·libc_open_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mknod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mknod_trampoline_addr(SB)/8, $libc_mknod_trampoline<>(SB)
+
+TEXT libc_open_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_open(SB)
-TEXT ·libc_openat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_open_trampoline_addr(SB), RODATA, $8
+DATA	·libc_open_trampoline_addr(SB)/8, $libc_open_trampoline<>(SB)
+
+TEXT libc_openat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_openat(SB)
-TEXT ·libc_pathconf_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_openat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_openat_trampoline_addr(SB)/8, $libc_openat_trampoline<>(SB)
+
+TEXT libc_pathconf_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pathconf(SB)
-TEXT ·libc_pread_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pathconf_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pathconf_trampoline_addr(SB)/8, $libc_pathconf_trampoline<>(SB)
+
+TEXT libc_pread_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pread(SB)
-TEXT ·libc_pwrite_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pread_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pread_trampoline_addr(SB)/8, $libc_pread_trampoline<>(SB)
+
+TEXT libc_pwrite_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pwrite(SB)
-TEXT ·libc_read_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pwrite_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pwrite_trampoline_addr(SB)/8, $libc_pwrite_trampoline<>(SB)
+
+TEXT libc_read_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_read(SB)
-TEXT ·libc_readlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_read_trampoline_addr(SB), RODATA, $8
+DATA	·libc_read_trampoline_addr(SB)/8, $libc_read_trampoline<>(SB)
+
+TEXT libc_readlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readlink(SB)
-TEXT ·libc_readlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_readlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readlink_trampoline_addr(SB)/8, $libc_readlink_trampoline<>(SB)
+
+TEXT libc_readlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readlinkat(SB)
-TEXT ·libc_rename_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_readlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readlinkat_trampoline_addr(SB)/8, $libc_readlinkat_trampoline<>(SB)
+
+TEXT libc_rename_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_rename(SB)
-TEXT ·libc_renameat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_rename_trampoline_addr(SB), RODATA, $8
+DATA	·libc_rename_trampoline_addr(SB)/8, $libc_rename_trampoline<>(SB)
+
+TEXT libc_renameat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_renameat(SB)
-TEXT ·libc_revoke_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_renameat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_renameat_trampoline_addr(SB)/8, $libc_renameat_trampoline<>(SB)
+
+TEXT libc_revoke_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_revoke(SB)
-TEXT ·libc_rmdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_revoke_trampoline_addr(SB), RODATA, $8
+DATA	·libc_revoke_trampoline_addr(SB)/8, $libc_revoke_trampoline<>(SB)
+
+TEXT libc_rmdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_rmdir(SB)
-TEXT ·libc_lseek_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_rmdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_rmdir_trampoline_addr(SB)/8, $libc_rmdir_trampoline<>(SB)
+
+TEXT libc_lseek_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lseek(SB)
-TEXT ·libc_select_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lseek_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lseek_trampoline_addr(SB)/8, $libc_lseek_trampoline<>(SB)
+
+TEXT libc_select_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_select(SB)
-TEXT ·libc_setegid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_select_trampoline_addr(SB), RODATA, $8
+DATA	·libc_select_trampoline_addr(SB)/8, $libc_select_trampoline<>(SB)
+
+TEXT libc_setegid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setegid(SB)
-TEXT ·libc_seteuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setegid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setegid_trampoline_addr(SB)/8, $libc_setegid_trampoline<>(SB)
+
+TEXT libc_seteuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_seteuid(SB)
-TEXT ·libc_setgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_seteuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_seteuid_trampoline_addr(SB)/8, $libc_seteuid_trampoline<>(SB)
+
+TEXT libc_setgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setgid(SB)
-TEXT ·libc_setlogin_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setgid_trampoline_addr(SB)/8, $libc_setgid_trampoline<>(SB)
+
+TEXT libc_setlogin_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setlogin(SB)
-TEXT ·libc_setpgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setlogin_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setlogin_trampoline_addr(SB)/8, $libc_setlogin_trampoline<>(SB)
+
+TEXT libc_setpgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setpgid(SB)
-TEXT ·libc_setpriority_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setpgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setpgid_trampoline_addr(SB)/8, $libc_setpgid_trampoline<>(SB)
+
+TEXT libc_setpriority_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setpriority(SB)
-TEXT ·libc_setprivexec_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setpriority_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setpriority_trampoline_addr(SB)/8, $libc_setpriority_trampoline<>(SB)
+
+TEXT libc_setprivexec_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setprivexec(SB)
-TEXT ·libc_setregid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setprivexec_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setprivexec_trampoline_addr(SB)/8, $libc_setprivexec_trampoline<>(SB)
+
+TEXT libc_setregid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setregid(SB)
-TEXT ·libc_setreuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setregid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setregid_trampoline_addr(SB)/8, $libc_setregid_trampoline<>(SB)
+
+TEXT libc_setreuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setreuid(SB)
-TEXT ·libc_setrlimit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setreuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setreuid_trampoline_addr(SB)/8, $libc_setreuid_trampoline<>(SB)
+
+TEXT libc_setrlimit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setrlimit(SB)
-TEXT ·libc_setsid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setrlimit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setrlimit_trampoline_addr(SB)/8, $libc_setrlimit_trampoline<>(SB)
+
+TEXT libc_setsid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setsid(SB)
-TEXT ·libc_settimeofday_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setsid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setsid_trampoline_addr(SB)/8, $libc_setsid_trampoline<>(SB)
+
+TEXT libc_settimeofday_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_settimeofday(SB)
-TEXT ·libc_setuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_settimeofday_trampoline_addr(SB), RODATA, $8
+DATA	·libc_settimeofday_trampoline_addr(SB)/8, $libc_settimeofday_trampoline<>(SB)
+
+TEXT libc_setuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setuid(SB)
-TEXT ·libc_symlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setuid_trampoline_addr(SB)/8, $libc_setuid_trampoline<>(SB)
+
+TEXT libc_symlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_symlink(SB)
-TEXT ·libc_symlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_symlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_symlink_trampoline_addr(SB)/8, $libc_symlink_trampoline<>(SB)
+
+TEXT libc_symlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_symlinkat(SB)
-TEXT ·libc_sync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_symlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_symlinkat_trampoline_addr(SB)/8, $libc_symlinkat_trampoline<>(SB)
+
+TEXT libc_sync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sync(SB)
-TEXT ·libc_truncate_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sync_trampoline_addr(SB)/8, $libc_sync_trampoline<>(SB)
+
+TEXT libc_truncate_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_truncate(SB)
-TEXT ·libc_umask_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_truncate_trampoline_addr(SB), RODATA, $8
+DATA	·libc_truncate_trampoline_addr(SB)/8, $libc_truncate_trampoline<>(SB)
+
+TEXT libc_umask_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_umask(SB)
-TEXT ·libc_undelete_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_umask_trampoline_addr(SB), RODATA, $8
+DATA	·libc_umask_trampoline_addr(SB)/8, $libc_umask_trampoline<>(SB)
+
+TEXT libc_undelete_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_undelete(SB)
-TEXT ·libc_unlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_undelete_trampoline_addr(SB), RODATA, $8
+DATA	·libc_undelete_trampoline_addr(SB)/8, $libc_undelete_trampoline<>(SB)
+
+TEXT libc_unlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unlink(SB)
-TEXT ·libc_unlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unlink_trampoline_addr(SB)/8, $libc_unlink_trampoline<>(SB)
+
+TEXT libc_unlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unlinkat(SB)
-TEXT ·libc_unmount_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unlinkat_trampoline_addr(SB)/8, $libc_unlinkat_trampoline<>(SB)
+
+TEXT libc_unmount_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unmount(SB)
-TEXT ·libc_write_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unmount_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unmount_trampoline_addr(SB)/8, $libc_unmount_trampoline<>(SB)
+
+TEXT libc_write_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_write(SB)
-TEXT ·libc_mmap_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_write_trampoline_addr(SB), RODATA, $8
+DATA	·libc_write_trampoline_addr(SB)/8, $libc_write_trampoline<>(SB)
+
+TEXT libc_mmap_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mmap(SB)
-TEXT ·libc_munmap_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mmap_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mmap_trampoline_addr(SB)/8, $libc_mmap_trampoline<>(SB)
+
+TEXT libc_munmap_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munmap(SB)
-TEXT ·libc_ptrace_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ptrace(SB)
-TEXT ·libc_fstat64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munmap_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munmap_trampoline_addr(SB)/8, $libc_munmap_trampoline<>(SB)
+
+TEXT libc_fstat64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstat64(SB)
-TEXT ·libc_fstatat64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstat64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstat64_trampoline_addr(SB)/8, $libc_fstat64_trampoline<>(SB)
+
+TEXT libc_fstatat64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstatat64(SB)
-TEXT ·libc_fstatfs64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstatat64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstatat64_trampoline_addr(SB)/8, $libc_fstatat64_trampoline<>(SB)
+
+TEXT libc_fstatfs64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstatfs64(SB)
-TEXT ·libc_getfsstat64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstatfs64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstatfs64_trampoline_addr(SB)/8, $libc_fstatfs64_trampoline<>(SB)
+
+TEXT libc_getfsstat64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getfsstat64(SB)
-TEXT ·libc_lstat64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getfsstat64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getfsstat64_trampoline_addr(SB)/8, $libc_getfsstat64_trampoline<>(SB)
+
+TEXT libc_lstat64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lstat64(SB)
-TEXT ·libc_stat64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lstat64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lstat64_trampoline_addr(SB)/8, $libc_lstat64_trampoline<>(SB)
+
+TEXT libc_ptrace_trampoline<>(SB),NOSPLIT,$0-0
+	JMP	libc_ptrace(SB)
+
+GLOBL	·libc_ptrace_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ptrace_trampoline_addr(SB)/8, $libc_ptrace_trampoline<>(SB)
+
+TEXT libc_stat64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_stat64(SB)
-TEXT ·libc_statfs64_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_stat64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_stat64_trampoline_addr(SB)/8, $libc_stat64_trampoline<>(SB)
+
+TEXT libc_statfs64_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_statfs64(SB)
+
+GLOBL	·libc_statfs64_trampoline_addr(SB), RODATA, $8
+DATA	·libc_statfs64_trampoline_addr(SB)/8, $libc_statfs64_trampoline<>(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.go
deleted file mode 100644
index f519ce9..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.go
+++ /dev/null
@@ -1,41 +0,0 @@
-// go run mksyscall.go -l32 -tags darwin,arm,go1.13 syscall_darwin.1_13.go
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build darwin,arm,go1.13
-
-package unix
-
-import (
-	"syscall"
-	"unsafe"
-)
-
-var _ syscall.Errno
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func closedir(dir uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_closedir_trampoline), uintptr(dir), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_closedir_trampoline()
-
-//go:linkname libc_closedir libc_closedir
-//go:cgo_import_dynamic libc_closedir closedir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func readdir_r(dir uintptr, entry *Dirent, result **Dirent) (res Errno) {
-	r0, _, _ := syscall_syscall(funcPC(libc_readdir_r_trampoline), uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
-	res = Errno(r0)
-	return
-}
-
-func libc_readdir_r_trampoline()
-
-//go:linkname libc_readdir_r libc_readdir_r
-//go:cgo_import_dynamic libc_readdir_r readdir_r "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.s
deleted file mode 100644
index 488e557..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.1_13.s
+++ /dev/null
@@ -1,12 +0,0 @@
-// go run mkasm_darwin.go arm
-// Code generated by the command above; DO NOT EDIT.
-
-// +build go1.13
-
-#include "textflag.h"
-TEXT ·libc_fdopendir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fdopendir(SB)
-TEXT ·libc_closedir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_closedir(SB)
-TEXT ·libc_readdir_r_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readdir_r(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.go
deleted file mode 100644
index d6b5249..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.go
+++ /dev/null
@@ -1,2559 +0,0 @@
-// go run mksyscall.go -l32 -tags darwin,arm,go1.12 syscall_bsd.go syscall_darwin.go syscall_darwin_arm.go
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build darwin,arm,go1.12
-
-package unix
-
-import (
-	"syscall"
-	"unsafe"
-)
-
-var _ syscall.Errno
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getgroups(ngid int, gid *_Gid_t) (n int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getgroups_trampoline()
-
-//go:linkname libc_getgroups libc_getgroups
-//go:cgo_import_dynamic libc_getgroups getgroups "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setgroups(ngid int, gid *_Gid_t) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setgroups_trampoline()
-
-//go:linkname libc_setgroups libc_setgroups
-//go:cgo_import_dynamic libc_setgroups setgroups "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func wait4(pid int, wstatus *_C_int, options int, rusage *Rusage) (wpid int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_wait4_trampoline), uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
-	wpid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_wait4_trampoline()
-
-//go:linkname libc_wait4 libc_wait4
-//go:cgo_import_dynamic libc_wait4 wait4 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_accept_trampoline), uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_accept_trampoline()
-
-//go:linkname libc_accept libc_accept
-//go:cgo_import_dynamic libc_accept accept "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_bind_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_bind_trampoline()
-
-//go:linkname libc_bind libc_bind
-//go:cgo_import_dynamic libc_bind bind "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_connect_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_connect_trampoline()
-
-//go:linkname libc_connect libc_connect
-//go:cgo_import_dynamic libc_connect connect "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func socket(domain int, typ int, proto int) (fd int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_socket_trampoline), uintptr(domain), uintptr(typ), uintptr(proto))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_socket_trampoline()
-
-//go:linkname libc_socket libc_socket
-//go:cgo_import_dynamic libc_socket socket "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_getsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsockopt_trampoline()
-
-//go:linkname libc_getsockopt libc_getsockopt
-//go:cgo_import_dynamic libc_getsockopt getsockopt "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setsockopt_trampoline()
-
-//go:linkname libc_setsockopt libc_setsockopt
-//go:cgo_import_dynamic libc_setsockopt setsockopt "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getpeername_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpeername_trampoline()
-
-//go:linkname libc_getpeername libc_getpeername
-//go:cgo_import_dynamic libc_getpeername getpeername "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getsockname_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsockname_trampoline()
-
-//go:linkname libc_getsockname libc_getsockname
-//go:cgo_import_dynamic libc_getsockname getsockname "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Shutdown(s int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_shutdown_trampoline), uintptr(s), uintptr(how), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_shutdown_trampoline()
-
-//go:linkname libc_shutdown libc_shutdown
-//go:cgo_import_dynamic libc_shutdown shutdown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
-	_, _, e1 := syscall_rawSyscall6(funcPC(libc_socketpair_trampoline), uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_socketpair_trampoline()
-
-//go:linkname libc_socketpair libc_socketpair
-//go:cgo_import_dynamic libc_socketpair socketpair "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_recvfrom_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_recvfrom_trampoline()
-
-//go:linkname libc_recvfrom libc_recvfrom
-//go:cgo_import_dynamic libc_recvfrom recvfrom "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error) {
-	var _p0 unsafe.Pointer
-	if len(buf) > 0 {
-		_p0 = unsafe.Pointer(&buf[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendto_trampoline), uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendto_trampoline()
-
-//go:linkname libc_sendto libc_sendto
-//go:cgo_import_dynamic libc_sendto sendto "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_recvmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_recvmsg_trampoline()
-
-//go:linkname libc_recvmsg libc_recvmsg
-//go:cgo_import_dynamic libc_recvmsg recvmsg "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_sendmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendmsg_trampoline()
-
-//go:linkname libc_sendmsg libc_sendmsg
-//go:cgo_import_dynamic libc_sendmsg sendmsg "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func kevent(kq int, change unsafe.Pointer, nchange int, event unsafe.Pointer, nevent int, timeout *Timespec) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_kevent_trampoline), uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kevent_trampoline()
-
-//go:linkname libc_kevent libc_kevent
-//go:cgo_import_dynamic libc_kevent kevent "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func utimes(path string, timeval *[2]Timeval) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_utimes_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_utimes_trampoline()
-
-//go:linkname libc_utimes libc_utimes
-//go:cgo_import_dynamic libc_utimes utimes "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func futimes(fd int, timeval *[2]Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_futimes_trampoline), uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_futimes_trampoline()
-
-//go:linkname libc_futimes libc_futimes
-//go:cgo_import_dynamic libc_futimes futimes "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func poll(fds *PollFd, nfds int, timeout int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_poll_trampoline), uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_poll_trampoline()
-
-//go:linkname libc_poll libc_poll
-//go:cgo_import_dynamic libc_poll poll "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Madvise(b []byte, behav int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_madvise_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(behav))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_madvise_trampoline()
-
-//go:linkname libc_madvise libc_madvise
-//go:cgo_import_dynamic libc_madvise madvise "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mlock(b []byte) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mlock_trampoline()
-
-//go:linkname libc_mlock libc_mlock
-//go:cgo_import_dynamic libc_mlock mlock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mlockall(flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_mlockall_trampoline), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mlockall_trampoline()
-
-//go:linkname libc_mlockall libc_mlockall
-//go:cgo_import_dynamic libc_mlockall mlockall "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mprotect(b []byte, prot int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mprotect_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(prot))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mprotect_trampoline()
-
-//go:linkname libc_mprotect libc_mprotect
-//go:cgo_import_dynamic libc_mprotect mprotect "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Msync(b []byte, flags int) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_msync_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_msync_trampoline()
-
-//go:linkname libc_msync libc_msync
-//go:cgo_import_dynamic libc_msync msync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Munlock(b []byte) (err error) {
-	var _p0 unsafe.Pointer
-	if len(b) > 0 {
-		_p0 = unsafe.Pointer(&b[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_munlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munlock_trampoline()
-
-//go:linkname libc_munlock libc_munlock
-//go:cgo_import_dynamic libc_munlock munlock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Munlockall() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munlockall_trampoline), 0, 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munlockall_trampoline()
-
-//go:linkname libc_munlockall libc_munlockall
-//go:cgo_import_dynamic libc_munlockall munlockall "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func pipe() (r int, w int, err error) {
-	r0, r1, e1 := syscall_rawSyscall(funcPC(libc_pipe_trampoline), 0, 0, 0)
-	r = int(r0)
-	w = int(r1)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pipe_trampoline()
-
-//go:linkname libc_pipe libc_pipe
-//go:cgo_import_dynamic libc_pipe pipe "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getxattr(path string, attr string, dest *byte, size int, position uint32, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_getxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getxattr_trampoline()
-
-//go:linkname libc_getxattr libc_getxattr
-//go:cgo_import_dynamic libc_getxattr getxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fgetxattr(fd int, attr string, dest *byte, size int, position uint32, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_fgetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fgetxattr_trampoline()
-
-//go:linkname libc_fgetxattr libc_fgetxattr
-//go:cgo_import_dynamic libc_fgetxattr fgetxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setxattr(path string, attr string, data *byte, size int, position uint32, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_setxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setxattr_trampoline()
-
-//go:linkname libc_setxattr libc_setxattr
-//go:cgo_import_dynamic libc_setxattr setxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fsetxattr(fd int, attr string, data *byte, size int, position uint32, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fsetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fsetxattr_trampoline()
-
-//go:linkname libc_fsetxattr libc_fsetxattr
-//go:cgo_import_dynamic libc_fsetxattr fsetxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func removexattr(path string, attr string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_removexattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_removexattr_trampoline()
-
-//go:linkname libc_removexattr libc_removexattr
-//go:cgo_import_dynamic libc_removexattr removexattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fremovexattr(fd int, attr string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(attr)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_fremovexattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fremovexattr_trampoline()
-
-//go:linkname libc_fremovexattr libc_fremovexattr
-//go:cgo_import_dynamic libc_fremovexattr fremovexattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func listxattr(path string, dest *byte, size int, options int) (sz int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_listxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_listxattr_trampoline()
-
-//go:linkname libc_listxattr libc_listxattr
-//go:cgo_import_dynamic libc_listxattr listxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func flistxattr(fd int, dest *byte, size int, options int) (sz int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_flistxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
-	sz = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_flistxattr_trampoline()
-
-//go:linkname libc_flistxattr libc_flistxattr
-//go:cgo_import_dynamic libc_flistxattr flistxattr "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setattrlist_trampoline), uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setattrlist_trampoline()
-
-//go:linkname libc_setattrlist libc_setattrlist
-//go:cgo_import_dynamic libc_setattrlist setattrlist "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func fcntl(fd int, cmd int, arg int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fcntl_trampoline), uintptr(fd), uintptr(cmd), uintptr(arg))
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fcntl_trampoline()
-
-//go:linkname libc_fcntl libc_fcntl
-//go:cgo_import_dynamic libc_fcntl fcntl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func kill(pid int, signum int, posix int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_kill_trampoline), uintptr(pid), uintptr(signum), uintptr(posix))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kill_trampoline()
-
-//go:linkname libc_kill libc_kill
-//go:cgo_import_dynamic libc_kill kill "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func ioctl(fd int, req uint, arg uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ioctl_trampoline), uintptr(fd), uintptr(req), uintptr(arg))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ioctl_trampoline()
-
-//go:linkname libc_ioctl libc_ioctl
-//go:cgo_import_dynamic libc_ioctl ioctl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) {
-	var _p0 unsafe.Pointer
-	if len(mib) > 0 {
-		_p0 = unsafe.Pointer(&mib[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sysctl_trampoline), uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sysctl_trampoline()
-
-//go:linkname libc_sysctl libc_sysctl
-//go:cgo_import_dynamic libc_sysctl sysctl "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func sendfile(infd int, outfd int, offset int64, len *int64, hdtr unsafe.Pointer, flags int) (err error) {
-	_, _, e1 := syscall_syscall9(funcPC(libc_sendfile_trampoline), uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(offset>>32), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sendfile_trampoline()
-
-//go:linkname libc_sendfile libc_sendfile
-//go:cgo_import_dynamic libc_sendfile sendfile "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Access(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_access_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_access_trampoline()
-
-//go:linkname libc_access libc_access
-//go:cgo_import_dynamic libc_access access "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Adjtime(delta *Timeval, olddelta *Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_adjtime_trampoline), uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_adjtime_trampoline()
-
-//go:linkname libc_adjtime libc_adjtime
-//go:cgo_import_dynamic libc_adjtime adjtime "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chdir(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chdir_trampoline()
-
-//go:linkname libc_chdir libc_chdir
-//go:cgo_import_dynamic libc_chdir chdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chflags(path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chflags_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chflags_trampoline()
-
-//go:linkname libc_chflags libc_chflags
-//go:cgo_import_dynamic libc_chflags chflags "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chmod(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chmod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chmod_trampoline()
-
-//go:linkname libc_chmod libc_chmod
-//go:cgo_import_dynamic libc_chmod chmod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chown(path string, uid int, gid int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chown_trampoline()
-
-//go:linkname libc_chown libc_chown
-//go:cgo_import_dynamic libc_chown chown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Chroot(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chroot_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_chroot_trampoline()
-
-//go:linkname libc_chroot libc_chroot
-//go:cgo_import_dynamic libc_chroot chroot "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func ClockGettime(clockid int32, time *Timespec) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_clock_gettime_trampoline), uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clock_gettime_trampoline()
-
-//go:linkname libc_clock_gettime libc_clock_gettime
-//go:cgo_import_dynamic libc_clock_gettime clock_gettime "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Close(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_close_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_close_trampoline()
-
-//go:linkname libc_close libc_close
-//go:cgo_import_dynamic libc_close close "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Clonefile(src string, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(src)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_clonefile_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clonefile_trampoline()
-
-//go:linkname libc_clonefile libc_clonefile
-//go:cgo_import_dynamic libc_clonefile clonefile "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Clonefileat(srcDirfd int, src string, dstDirfd int, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(src)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_clonefileat_trampoline), uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_clonefileat_trampoline()
-
-//go:linkname libc_clonefileat libc_clonefileat
-//go:cgo_import_dynamic libc_clonefileat clonefileat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Dup(fd int) (nfd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_dup_trampoline), uintptr(fd), 0, 0)
-	nfd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_dup_trampoline()
-
-//go:linkname libc_dup libc_dup
-//go:cgo_import_dynamic libc_dup dup "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Dup2(from int, to int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_dup2_trampoline), uintptr(from), uintptr(to), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_dup2_trampoline()
-
-//go:linkname libc_dup2 libc_dup2
-//go:cgo_import_dynamic libc_dup2 dup2 "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Exchangedata(path1 string, path2 string, options int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path1)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(path2)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_exchangedata_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_exchangedata_trampoline()
-
-//go:linkname libc_exchangedata libc_exchangedata
-//go:cgo_import_dynamic libc_exchangedata exchangedata "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Exit(code int) {
-	syscall_syscall(funcPC(libc_exit_trampoline), uintptr(code), 0, 0)
-	return
-}
-
-func libc_exit_trampoline()
-
-//go:linkname libc_exit libc_exit
-//go:cgo_import_dynamic libc_exit exit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Faccessat(dirfd int, path string, mode uint32, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_faccessat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_faccessat_trampoline()
-
-//go:linkname libc_faccessat libc_faccessat
-//go:cgo_import_dynamic libc_faccessat faccessat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchdir(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchdir_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchdir_trampoline()
-
-//go:linkname libc_fchdir libc_fchdir
-//go:cgo_import_dynamic libc_fchdir fchdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchflags(fd int, flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchflags_trampoline), uintptr(fd), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchflags_trampoline()
-
-//go:linkname libc_fchflags libc_fchflags
-//go:cgo_import_dynamic libc_fchflags fchflags "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchmod(fd int, mode uint32) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchmod_trampoline), uintptr(fd), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchmod_trampoline()
-
-//go:linkname libc_fchmod libc_fchmod
-//go:cgo_import_dynamic libc_fchmod fchmod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchmodat(dirfd int, path string, mode uint32, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchmodat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchmodat_trampoline()
-
-//go:linkname libc_fchmodat libc_fchmodat
-//go:cgo_import_dynamic libc_fchmodat fchmodat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchown(fd int, uid int, gid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchown_trampoline), uintptr(fd), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchown_trampoline()
-
-//go:linkname libc_fchown libc_fchown
-//go:cgo_import_dynamic libc_fchown fchown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fchownat(dirfd int, path string, uid int, gid int, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchownat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fchownat_trampoline()
-
-//go:linkname libc_fchownat libc_fchownat
-//go:cgo_import_dynamic libc_fchownat fchownat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fclonefileat(srcDirfd int, dstDirfd int, dst string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(dst)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fclonefileat_trampoline), uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fclonefileat_trampoline()
-
-//go:linkname libc_fclonefileat libc_fclonefileat
-//go:cgo_import_dynamic libc_fclonefileat fclonefileat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Flock(fd int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_flock_trampoline), uintptr(fd), uintptr(how), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_flock_trampoline()
-
-//go:linkname libc_flock libc_flock
-//go:cgo_import_dynamic libc_flock flock "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fpathconf(fd int, name int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fpathconf_trampoline), uintptr(fd), uintptr(name), 0)
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fpathconf_trampoline()
-
-//go:linkname libc_fpathconf libc_fpathconf
-//go:cgo_import_dynamic libc_fpathconf fpathconf "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fsync(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fsync_trampoline), uintptr(fd), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fsync_trampoline()
-
-//go:linkname libc_fsync libc_fsync
-//go:cgo_import_dynamic libc_fsync fsync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Ftruncate(fd int, length int64) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ftruncate_trampoline), uintptr(fd), uintptr(length), uintptr(length>>32))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_ftruncate_trampoline()
-
-//go:linkname libc_ftruncate libc_ftruncate
-//go:cgo_import_dynamic libc_ftruncate ftruncate "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getcwd(buf []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(buf) > 0 {
-		_p0 = unsafe.Pointer(&buf[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_getcwd_trampoline), uintptr(_p0), uintptr(len(buf)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getcwd_trampoline()
-
-//go:linkname libc_getcwd libc_getcwd
-//go:cgo_import_dynamic libc_getcwd getcwd "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getdtablesize() (size int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_getdtablesize_trampoline), 0, 0, 0)
-	size = int(r0)
-	return
-}
-
-func libc_getdtablesize_trampoline()
-
-//go:linkname libc_getdtablesize libc_getdtablesize
-//go:cgo_import_dynamic libc_getdtablesize getdtablesize "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getegid() (egid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getegid_trampoline), 0, 0, 0)
-	egid = int(r0)
-	return
-}
-
-func libc_getegid_trampoline()
-
-//go:linkname libc_getegid libc_getegid
-//go:cgo_import_dynamic libc_getegid getegid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Geteuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_geteuid_trampoline), 0, 0, 0)
-	uid = int(r0)
-	return
-}
-
-func libc_geteuid_trampoline()
-
-//go:linkname libc_geteuid libc_geteuid
-//go:cgo_import_dynamic libc_geteuid geteuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getgid() (gid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getgid_trampoline), 0, 0, 0)
-	gid = int(r0)
-	return
-}
-
-func libc_getgid_trampoline()
-
-//go:linkname libc_getgid libc_getgid
-//go:cgo_import_dynamic libc_getgid getgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpgid(pid int) (pgid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getpgid_trampoline), uintptr(pid), 0, 0)
-	pgid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpgid_trampoline()
-
-//go:linkname libc_getpgid libc_getpgid
-//go:cgo_import_dynamic libc_getpgid getpgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpgrp() (pgrp int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpgrp_trampoline), 0, 0, 0)
-	pgrp = int(r0)
-	return
-}
-
-func libc_getpgrp_trampoline()
-
-//go:linkname libc_getpgrp libc_getpgrp
-//go:cgo_import_dynamic libc_getpgrp getpgrp "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpid() (pid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpid_trampoline), 0, 0, 0)
-	pid = int(r0)
-	return
-}
-
-func libc_getpid_trampoline()
-
-//go:linkname libc_getpid libc_getpid
-//go:cgo_import_dynamic libc_getpid getpid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getppid() (ppid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getppid_trampoline), 0, 0, 0)
-	ppid = int(r0)
-	return
-}
-
-func libc_getppid_trampoline()
-
-//go:linkname libc_getppid libc_getppid
-//go:cgo_import_dynamic libc_getppid getppid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getpriority(which int, who int) (prio int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getpriority_trampoline), uintptr(which), uintptr(who), 0)
-	prio = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getpriority_trampoline()
-
-//go:linkname libc_getpriority libc_getpriority
-//go:cgo_import_dynamic libc_getpriority getpriority "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getrlimit_trampoline()
-
-//go:linkname libc_getrlimit libc_getrlimit
-//go:cgo_import_dynamic libc_getrlimit getrlimit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getrusage(who int, rusage *Rusage) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrusage_trampoline), uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getrusage_trampoline()
-
-//go:linkname libc_getrusage libc_getrusage
-//go:cgo_import_dynamic libc_getrusage getrusage "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getsid(pid int) (sid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getsid_trampoline), uintptr(pid), 0, 0)
-	sid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getsid_trampoline()
-
-//go:linkname libc_getsid libc_getsid
-//go:cgo_import_dynamic libc_getsid getsid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Gettimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_gettimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_gettimeofday_trampoline()
-
-//go:linkname libc_gettimeofday libc_gettimeofday
-//go:cgo_import_dynamic libc_gettimeofday gettimeofday "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Getuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getuid_trampoline), 0, 0, 0)
-	uid = int(r0)
-	return
-}
-
-func libc_getuid_trampoline()
-
-//go:linkname libc_getuid libc_getuid
-//go:cgo_import_dynamic libc_getuid getuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Issetugid() (tainted bool) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_issetugid_trampoline), 0, 0, 0)
-	tainted = bool(r0 != 0)
-	return
-}
-
-func libc_issetugid_trampoline()
-
-//go:linkname libc_issetugid libc_issetugid
-//go:cgo_import_dynamic libc_issetugid issetugid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Kqueue() (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_kqueue_trampoline), 0, 0, 0)
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_kqueue_trampoline()
-
-//go:linkname libc_kqueue libc_kqueue
-//go:cgo_import_dynamic libc_kqueue kqueue "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Lchown(path string, uid int, gid int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lchown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lchown_trampoline()
-
-//go:linkname libc_lchown libc_lchown
-//go:cgo_import_dynamic libc_lchown lchown "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Link(path string, link string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_link_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_link_trampoline()
-
-//go:linkname libc_link libc_link
-//go:cgo_import_dynamic libc_link link "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Linkat(pathfd int, path string, linkfd int, link string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_linkat_trampoline), uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_linkat_trampoline()
-
-//go:linkname libc_linkat libc_linkat
-//go:cgo_import_dynamic libc_linkat linkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Listen(s int, backlog int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_listen_trampoline), uintptr(s), uintptr(backlog), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_listen_trampoline()
-
-//go:linkname libc_listen libc_listen
-//go:cgo_import_dynamic libc_listen listen "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkdir(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdir_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkdir_trampoline()
-
-//go:linkname libc_mkdir libc_mkdir
-//go:cgo_import_dynamic libc_mkdir mkdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkdirat(dirfd int, path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdirat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkdirat_trampoline()
-
-//go:linkname libc_mkdirat libc_mkdirat
-//go:cgo_import_dynamic libc_mkdirat mkdirat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mkfifo(path string, mode uint32) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkfifo_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mkfifo_trampoline()
-
-//go:linkname libc_mkfifo libc_mkfifo
-//go:cgo_import_dynamic libc_mkfifo mkfifo "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Mknod(path string, mode uint32, dev int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mknod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mknod_trampoline()
-
-//go:linkname libc_mknod libc_mknod
-//go:cgo_import_dynamic libc_mknod mknod "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Open(path string, mode int, perm uint32) (fd int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_open_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_open_trampoline()
-
-//go:linkname libc_open libc_open
-//go:cgo_import_dynamic libc_open open "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Openat(dirfd int, path string, mode int, perm uint32) (fd int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_openat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
-	fd = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_openat_trampoline()
-
-//go:linkname libc_openat libc_openat
-//go:cgo_import_dynamic libc_openat openat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pathconf(path string, name int) (val int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_pathconf_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
-	val = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pathconf_trampoline()
-
-//go:linkname libc_pathconf libc_pathconf
-//go:cgo_import_dynamic libc_pathconf pathconf "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pread(fd int, p []byte, offset int64) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pread_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), uintptr(offset>>32), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pread_trampoline()
-
-//go:linkname libc_pread libc_pread
-//go:cgo_import_dynamic libc_pread pread "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Pwrite(fd int, p []byte, offset int64) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pwrite_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), uintptr(offset>>32), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_pwrite_trampoline()
-
-//go:linkname libc_pwrite libc_pwrite
-//go:cgo_import_dynamic libc_pwrite pwrite "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func read(fd int, p []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_read_trampoline()
-
-//go:linkname libc_read libc_read
-//go:cgo_import_dynamic libc_read read "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Readlink(path string, buf []byte) (n int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 unsafe.Pointer
-	if len(buf) > 0 {
-		_p1 = unsafe.Pointer(&buf[0])
-	} else {
-		_p1 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_readlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_readlink_trampoline()
-
-//go:linkname libc_readlink libc_readlink
-//go:cgo_import_dynamic libc_readlink readlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Readlinkat(dirfd int, path string, buf []byte) (n int, err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 unsafe.Pointer
-	if len(buf) > 0 {
-		_p1 = unsafe.Pointer(&buf[0])
-	} else {
-		_p1 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_readlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_readlinkat_trampoline()
-
-//go:linkname libc_readlinkat libc_readlinkat
-//go:cgo_import_dynamic libc_readlinkat readlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Rename(from string, to string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(from)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(to)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rename_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_rename_trampoline()
-
-//go:linkname libc_rename libc_rename
-//go:cgo_import_dynamic libc_rename rename "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Renameat(fromfd int, from string, tofd int, to string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(from)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(to)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_renameat_trampoline), uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_renameat_trampoline()
-
-//go:linkname libc_renameat libc_renameat
-//go:cgo_import_dynamic libc_renameat renameat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Revoke(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_revoke_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_revoke_trampoline()
-
-//go:linkname libc_revoke libc_revoke
-//go:cgo_import_dynamic libc_revoke revoke "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Rmdir(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rmdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_rmdir_trampoline()
-
-//go:linkname libc_rmdir libc_rmdir
-//go:cgo_import_dynamic libc_rmdir rmdir "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Seek(fd int, offset int64, whence int) (newoffset int64, err error) {
-	r0, r1, e1 := syscall_syscall6(funcPC(libc_lseek_trampoline), uintptr(fd), uintptr(offset), uintptr(offset>>32), uintptr(whence), 0, 0)
-	newoffset = int64(int64(r1)<<32 | int64(r0))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lseek_trampoline()
-
-//go:linkname libc_lseek libc_lseek
-//go:cgo_import_dynamic libc_lseek lseek "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_select_trampoline), uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_select_trampoline()
-
-//go:linkname libc_select libc_select
-//go:cgo_import_dynamic libc_select select "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setegid(egid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setegid_trampoline), uintptr(egid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setegid_trampoline()
-
-//go:linkname libc_setegid libc_setegid
-//go:cgo_import_dynamic libc_setegid setegid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Seteuid(euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_seteuid_trampoline), uintptr(euid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_seteuid_trampoline()
-
-//go:linkname libc_seteuid libc_seteuid
-//go:cgo_import_dynamic libc_seteuid seteuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setgid(gid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgid_trampoline), uintptr(gid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setgid_trampoline()
-
-//go:linkname libc_setgid libc_setgid
-//go:cgo_import_dynamic libc_setgid setgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setlogin(name string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(name)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_setlogin_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setlogin_trampoline()
-
-//go:linkname libc_setlogin libc_setlogin
-//go:cgo_import_dynamic libc_setlogin setlogin "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setpgid(pid int, pgid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setpgid_trampoline), uintptr(pid), uintptr(pgid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setpgid_trampoline()
-
-//go:linkname libc_setpgid libc_setpgid
-//go:cgo_import_dynamic libc_setpgid setpgid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setpriority(which int, who int, prio int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setpriority_trampoline), uintptr(which), uintptr(who), uintptr(prio))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setpriority_trampoline()
-
-//go:linkname libc_setpriority libc_setpriority
-//go:cgo_import_dynamic libc_setpriority setpriority "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setprivexec(flag int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setprivexec_trampoline), uintptr(flag), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setprivexec_trampoline()
-
-//go:linkname libc_setprivexec libc_setprivexec
-//go:cgo_import_dynamic libc_setprivexec setprivexec "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setregid(rgid int, egid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setregid_trampoline), uintptr(rgid), uintptr(egid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setregid_trampoline()
-
-//go:linkname libc_setregid libc_setregid
-//go:cgo_import_dynamic libc_setregid setregid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setreuid(ruid int, euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setreuid_trampoline), uintptr(ruid), uintptr(euid), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setreuid_trampoline()
-
-//go:linkname libc_setreuid libc_setreuid
-//go:cgo_import_dynamic libc_setreuid setreuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setrlimit_trampoline()
-
-//go:linkname libc_setrlimit libc_setrlimit
-//go:cgo_import_dynamic libc_setrlimit setrlimit "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setsid() (pid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_setsid_trampoline), 0, 0, 0)
-	pid = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setsid_trampoline()
-
-//go:linkname libc_setsid libc_setsid
-//go:cgo_import_dynamic libc_setsid setsid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Settimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_settimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_settimeofday_trampoline()
-
-//go:linkname libc_settimeofday libc_settimeofday
-//go:cgo_import_dynamic libc_settimeofday settimeofday "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Setuid(uid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setuid_trampoline), uintptr(uid), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_setuid_trampoline()
-
-//go:linkname libc_setuid libc_setuid
-//go:cgo_import_dynamic libc_setuid setuid "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Symlink(path string, link string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(link)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_symlink_trampoline()
-
-//go:linkname libc_symlink libc_symlink
-//go:cgo_import_dynamic libc_symlink symlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Symlinkat(oldpath string, newdirfd int, newpath string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(oldpath)
-	if err != nil {
-		return
-	}
-	var _p1 *byte
-	_p1, err = BytePtrFromString(newpath)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlinkat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_symlinkat_trampoline()
-
-//go:linkname libc_symlinkat libc_symlinkat
-//go:cgo_import_dynamic libc_symlinkat symlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Sync() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_sync_trampoline), 0, 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_sync_trampoline()
-
-//go:linkname libc_sync libc_sync
-//go:cgo_import_dynamic libc_sync sync "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Truncate(path string, length int64) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_truncate_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(length), uintptr(length>>32))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_truncate_trampoline()
-
-//go:linkname libc_truncate libc_truncate
-//go:cgo_import_dynamic libc_truncate truncate "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Umask(newmask int) (oldmask int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_umask_trampoline), uintptr(newmask), 0, 0)
-	oldmask = int(r0)
-	return
-}
-
-func libc_umask_trampoline()
-
-//go:linkname libc_umask libc_umask
-//go:cgo_import_dynamic libc_umask umask "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Undelete(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_undelete_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_undelete_trampoline()
-
-//go:linkname libc_undelete libc_undelete
-//go:cgo_import_dynamic libc_undelete undelete "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unlink(path string) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlink_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unlink_trampoline()
-
-//go:linkname libc_unlink libc_unlink
-//go:cgo_import_dynamic libc_unlink unlink "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unlinkat(dirfd int, path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unlinkat_trampoline()
-
-//go:linkname libc_unlinkat libc_unlinkat
-//go:cgo_import_dynamic libc_unlinkat unlinkat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Unmount(path string, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unmount_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_unmount_trampoline()
-
-//go:linkname libc_unmount libc_unmount
-//go:cgo_import_dynamic libc_unmount unmount "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func write(fd int, p []byte) (n int, err error) {
-	var _p0 unsafe.Pointer
-	if len(p) > 0 {
-		_p0 = unsafe.Pointer(&p[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_write_trampoline()
-
-//go:linkname libc_write libc_write
-//go:cgo_import_dynamic libc_write write "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) {
-	r0, _, e1 := syscall_syscall9(funcPC(libc_mmap_trampoline), uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos), uintptr(pos>>32), 0, 0)
-	ret = uintptr(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_mmap_trampoline()
-
-//go:linkname libc_mmap libc_mmap
-//go:cgo_import_dynamic libc_mmap mmap "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func munmap(addr uintptr, length uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munmap_trampoline), uintptr(addr), uintptr(length), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_munmap_trampoline()
-
-//go:linkname libc_munmap libc_munmap
-//go:cgo_import_dynamic libc_munmap munmap "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func readlen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func writelen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstat(fd int, stat *Stat_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstat_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstat_trampoline()
-
-//go:linkname libc_fstat libc_fstat
-//go:cgo_import_dynamic libc_fstat fstat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstatat(fd int, path string, stat *Stat_t, flags int) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fstatat_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstatat_trampoline()
-
-//go:linkname libc_fstatat libc_fstatat
-//go:cgo_import_dynamic libc_fstatat fstatat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Fstatfs(fd int, stat *Statfs_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstatfs_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_fstatfs_trampoline()
-
-//go:linkname libc_fstatfs libc_fstatfs
-//go:cgo_import_dynamic libc_fstatfs fstatfs "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getfsstat_trampoline), uintptr(buf), uintptr(size), uintptr(flags))
-	n = int(r0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_getfsstat_trampoline()
-
-//go:linkname libc_getfsstat libc_getfsstat
-//go:cgo_import_dynamic libc_getfsstat getfsstat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Lstat(path string, stat *Stat_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lstat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_lstat_trampoline()
-
-//go:linkname libc_lstat libc_lstat
-//go:cgo_import_dynamic libc_lstat lstat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Stat(path string, stat *Stat_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_stat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_stat_trampoline()
-
-//go:linkname libc_stat libc_stat
-//go:cgo_import_dynamic libc_stat stat "/usr/lib/libSystem.B.dylib"
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
-func Statfs(path string, stat *Statfs_t) (err error) {
-	var _p0 *byte
-	_p0, err = BytePtrFromString(path)
-	if err != nil {
-		return
-	}
-	_, _, e1 := syscall_syscall(funcPC(libc_statfs_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-func libc_statfs_trampoline()
-
-//go:linkname libc_statfs libc_statfs
-//go:cgo_import_dynamic libc_statfs statfs "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.s
deleted file mode 100644
index 5eec5f1..0000000
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm.s
+++ /dev/null
@@ -1,288 +0,0 @@
-// go run mkasm_darwin.go arm
-// Code generated by the command above; DO NOT EDIT.
-
-// +build go1.12
-
-#include "textflag.h"
-TEXT ·libc_getgroups_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getgroups(SB)
-TEXT ·libc_setgroups_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setgroups(SB)
-TEXT ·libc_wait4_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_wait4(SB)
-TEXT ·libc_accept_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_accept(SB)
-TEXT ·libc_bind_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_bind(SB)
-TEXT ·libc_connect_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_connect(SB)
-TEXT ·libc_socket_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_socket(SB)
-TEXT ·libc_getsockopt_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsockopt(SB)
-TEXT ·libc_setsockopt_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setsockopt(SB)
-TEXT ·libc_getpeername_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpeername(SB)
-TEXT ·libc_getsockname_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsockname(SB)
-TEXT ·libc_shutdown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_shutdown(SB)
-TEXT ·libc_socketpair_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_socketpair(SB)
-TEXT ·libc_recvfrom_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_recvfrom(SB)
-TEXT ·libc_sendto_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendto(SB)
-TEXT ·libc_recvmsg_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_recvmsg(SB)
-TEXT ·libc_sendmsg_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendmsg(SB)
-TEXT ·libc_kevent_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kevent(SB)
-TEXT ·libc_utimes_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_utimes(SB)
-TEXT ·libc_futimes_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_futimes(SB)
-TEXT ·libc_poll_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_poll(SB)
-TEXT ·libc_madvise_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_madvise(SB)
-TEXT ·libc_mlock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mlock(SB)
-TEXT ·libc_mlockall_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mlockall(SB)
-TEXT ·libc_mprotect_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mprotect(SB)
-TEXT ·libc_msync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_msync(SB)
-TEXT ·libc_munlock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munlock(SB)
-TEXT ·libc_munlockall_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munlockall(SB)
-TEXT ·libc_pipe_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pipe(SB)
-TEXT ·libc_getxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getxattr(SB)
-TEXT ·libc_fgetxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fgetxattr(SB)
-TEXT ·libc_setxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setxattr(SB)
-TEXT ·libc_fsetxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fsetxattr(SB)
-TEXT ·libc_removexattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_removexattr(SB)
-TEXT ·libc_fremovexattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fremovexattr(SB)
-TEXT ·libc_listxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_listxattr(SB)
-TEXT ·libc_flistxattr_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_flistxattr(SB)
-TEXT ·libc_setattrlist_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setattrlist(SB)
-TEXT ·libc_fcntl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fcntl(SB)
-TEXT ·libc_kill_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kill(SB)
-TEXT ·libc_ioctl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ioctl(SB)
-TEXT ·libc_sysctl_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sysctl(SB)
-TEXT ·libc_sendfile_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sendfile(SB)
-TEXT ·libc_access_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_access(SB)
-TEXT ·libc_adjtime_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_adjtime(SB)
-TEXT ·libc_chdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chdir(SB)
-TEXT ·libc_chflags_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chflags(SB)
-TEXT ·libc_chmod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chmod(SB)
-TEXT ·libc_chown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chown(SB)
-TEXT ·libc_chroot_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_chroot(SB)
-TEXT ·libc_clock_gettime_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clock_gettime(SB)
-TEXT ·libc_close_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_close(SB)
-TEXT ·libc_clonefile_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clonefile(SB)
-TEXT ·libc_clonefileat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_clonefileat(SB)
-TEXT ·libc_dup_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_dup(SB)
-TEXT ·libc_dup2_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_dup2(SB)
-TEXT ·libc_exchangedata_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_exchangedata(SB)
-TEXT ·libc_exit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_exit(SB)
-TEXT ·libc_faccessat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_faccessat(SB)
-TEXT ·libc_fchdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchdir(SB)
-TEXT ·libc_fchflags_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchflags(SB)
-TEXT ·libc_fchmod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchmod(SB)
-TEXT ·libc_fchmodat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchmodat(SB)
-TEXT ·libc_fchown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchown(SB)
-TEXT ·libc_fchownat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fchownat(SB)
-TEXT ·libc_fclonefileat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fclonefileat(SB)
-TEXT ·libc_flock_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_flock(SB)
-TEXT ·libc_fpathconf_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fpathconf(SB)
-TEXT ·libc_fsync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fsync(SB)
-TEXT ·libc_ftruncate_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_ftruncate(SB)
-TEXT ·libc_getcwd_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getcwd(SB)
-TEXT ·libc_getdtablesize_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getdtablesize(SB)
-TEXT ·libc_getegid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getegid(SB)
-TEXT ·libc_geteuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_geteuid(SB)
-TEXT ·libc_getgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getgid(SB)
-TEXT ·libc_getpgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpgid(SB)
-TEXT ·libc_getpgrp_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpgrp(SB)
-TEXT ·libc_getpid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpid(SB)
-TEXT ·libc_getppid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getppid(SB)
-TEXT ·libc_getpriority_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getpriority(SB)
-TEXT ·libc_getrlimit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getrlimit(SB)
-TEXT ·libc_getrusage_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getrusage(SB)
-TEXT ·libc_getsid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getsid(SB)
-TEXT ·libc_gettimeofday_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_gettimeofday(SB)
-TEXT ·libc_getuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getuid(SB)
-TEXT ·libc_issetugid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_issetugid(SB)
-TEXT ·libc_kqueue_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_kqueue(SB)
-TEXT ·libc_lchown_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lchown(SB)
-TEXT ·libc_link_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_link(SB)
-TEXT ·libc_linkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_linkat(SB)
-TEXT ·libc_listen_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_listen(SB)
-TEXT ·libc_mkdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkdir(SB)
-TEXT ·libc_mkdirat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkdirat(SB)
-TEXT ·libc_mkfifo_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mkfifo(SB)
-TEXT ·libc_mknod_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mknod(SB)
-TEXT ·libc_open_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_open(SB)
-TEXT ·libc_openat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_openat(SB)
-TEXT ·libc_pathconf_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pathconf(SB)
-TEXT ·libc_pread_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pread(SB)
-TEXT ·libc_pwrite_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_pwrite(SB)
-TEXT ·libc_read_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_read(SB)
-TEXT ·libc_readlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readlink(SB)
-TEXT ·libc_readlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_readlinkat(SB)
-TEXT ·libc_rename_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_rename(SB)
-TEXT ·libc_renameat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_renameat(SB)
-TEXT ·libc_revoke_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_revoke(SB)
-TEXT ·libc_rmdir_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_rmdir(SB)
-TEXT ·libc_lseek_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lseek(SB)
-TEXT ·libc_select_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_select(SB)
-TEXT ·libc_setegid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setegid(SB)
-TEXT ·libc_seteuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_seteuid(SB)
-TEXT ·libc_setgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setgid(SB)
-TEXT ·libc_setlogin_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setlogin(SB)
-TEXT ·libc_setpgid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setpgid(SB)
-TEXT ·libc_setpriority_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setpriority(SB)
-TEXT ·libc_setprivexec_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setprivexec(SB)
-TEXT ·libc_setregid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setregid(SB)
-TEXT ·libc_setreuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setreuid(SB)
-TEXT ·libc_setrlimit_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setrlimit(SB)
-TEXT ·libc_setsid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setsid(SB)
-TEXT ·libc_settimeofday_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_settimeofday(SB)
-TEXT ·libc_setuid_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_setuid(SB)
-TEXT ·libc_symlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_symlink(SB)
-TEXT ·libc_symlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_symlinkat(SB)
-TEXT ·libc_sync_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_sync(SB)
-TEXT ·libc_truncate_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_truncate(SB)
-TEXT ·libc_umask_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_umask(SB)
-TEXT ·libc_undelete_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_undelete(SB)
-TEXT ·libc_unlink_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unlink(SB)
-TEXT ·libc_unlinkat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unlinkat(SB)
-TEXT ·libc_unmount_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_unmount(SB)
-TEXT ·libc_write_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_write(SB)
-TEXT ·libc_mmap_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_mmap(SB)
-TEXT ·libc_munmap_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_munmap(SB)
-TEXT ·libc_fstat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstat(SB)
-TEXT ·libc_fstatat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstatat(SB)
-TEXT ·libc_fstatfs_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_fstatfs(SB)
-TEXT ·libc_getfsstat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_getfsstat(SB)
-TEXT ·libc_lstat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_lstat(SB)
-TEXT ·libc_stat_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_stat(SB)
-TEXT ·libc_statfs_trampoline(SB),NOSPLIT,$0-0
-	JMP	libc_statfs(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.go
index d64e6c8..cec595d 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags darwin,arm64,go1.13 syscall_darwin.1_13.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build darwin && arm64 && go1.13
 // +build darwin,arm64,go1.13
 
 package unix
@@ -15,27 +16,25 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func closedir(dir uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_closedir_trampoline), uintptr(dir), 0, 0)
+	_, _, e1 := syscall_syscall(libc_closedir_trampoline_addr, uintptr(dir), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_closedir_trampoline()
+var libc_closedir_trampoline_addr uintptr
 
-//go:linkname libc_closedir libc_closedir
 //go:cgo_import_dynamic libc_closedir closedir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func readdir_r(dir uintptr, entry *Dirent, result **Dirent) (res Errno) {
-	r0, _, _ := syscall_syscall(funcPC(libc_readdir_r_trampoline), uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
+	r0, _, _ := syscall_syscall(libc_readdir_r_trampoline_addr, uintptr(dir), uintptr(unsafe.Pointer(entry)), uintptr(unsafe.Pointer(result)))
 	res = Errno(r0)
 	return
 }
 
-func libc_readdir_r_trampoline()
+var libc_readdir_r_trampoline_addr uintptr
 
-//go:linkname libc_readdir_r libc_readdir_r
 //go:cgo_import_dynamic libc_readdir_r readdir_r "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.s
index b29dabb..3579897 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.s
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.1_13.s
@@ -1,12 +1,25 @@
 // go run mkasm_darwin.go arm64
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build go1.13
 // +build go1.13
 
 #include "textflag.h"
-TEXT ·libc_fdopendir_trampoline(SB),NOSPLIT,$0-0
+
+TEXT libc_fdopendir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fdopendir(SB)
-TEXT ·libc_closedir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fdopendir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fdopendir_trampoline_addr(SB)/8, $libc_fdopendir_trampoline<>(SB)
+
+TEXT libc_closedir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_closedir(SB)
-TEXT ·libc_readdir_r_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_closedir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_closedir_trampoline_addr(SB)/8, $libc_closedir_trampoline<>(SB)
+
+TEXT libc_readdir_r_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readdir_r(SB)
+
+GLOBL	·libc_readdir_r_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readdir_r_trampoline_addr(SB)/8, $libc_readdir_r_trampoline<>(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.go b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.go
index 0863843..f2ee2bd 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags darwin,arm64,go1.12 syscall_bsd.go syscall_darwin.go syscall_darwin_arm64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build darwin && arm64 && go1.12
 // +build darwin,arm64,go1.12
 
 package unix
@@ -15,7 +16,7 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getgroups(ngid int, gid *_Gid_t) (n int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getgroups_trampoline_addr, uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -23,30 +24,28 @@
 	return
 }
 
-func libc_getgroups_trampoline()
+var libc_getgroups_trampoline_addr uintptr
 
-//go:linkname libc_getgroups libc_getgroups
 //go:cgo_import_dynamic libc_getgroups getgroups "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setgroups(ngid int, gid *_Gid_t) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgroups_trampoline), uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setgroups_trampoline_addr, uintptr(ngid), uintptr(unsafe.Pointer(gid)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setgroups_trampoline()
+var libc_setgroups_trampoline_addr uintptr
 
-//go:linkname libc_setgroups libc_setgroups
 //go:cgo_import_dynamic libc_setgroups setgroups "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func wait4(pid int, wstatus *_C_int, options int, rusage *Rusage) (wpid int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_wait4_trampoline), uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_wait4_trampoline_addr, uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options), uintptr(unsafe.Pointer(rusage)), 0, 0)
 	wpid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -54,15 +53,14 @@
 	return
 }
 
-func libc_wait4_trampoline()
+var libc_wait4_trampoline_addr uintptr
 
-//go:linkname libc_wait4 libc_wait4
 //go:cgo_import_dynamic libc_wait4 wait4 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_accept_trampoline), uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	r0, _, e1 := syscall_syscall(libc_accept_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -70,45 +68,42 @@
 	return
 }
 
-func libc_accept_trampoline()
+var libc_accept_trampoline_addr uintptr
 
-//go:linkname libc_accept libc_accept
 //go:cgo_import_dynamic libc_accept accept "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_bind_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
+	_, _, e1 := syscall_syscall(libc_bind_trampoline_addr, uintptr(s), uintptr(addr), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_bind_trampoline()
+var libc_bind_trampoline_addr uintptr
 
-//go:linkname libc_bind libc_bind
 //go:cgo_import_dynamic libc_bind bind "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_connect_trampoline), uintptr(s), uintptr(addr), uintptr(addrlen))
+	_, _, e1 := syscall_syscall(libc_connect_trampoline_addr, uintptr(s), uintptr(addr), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_connect_trampoline()
+var libc_connect_trampoline_addr uintptr
 
-//go:linkname libc_connect libc_connect
 //go:cgo_import_dynamic libc_connect connect "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func socket(domain int, typ int, proto int) (fd int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_socket_trampoline), uintptr(domain), uintptr(typ), uintptr(proto))
+	r0, _, e1 := syscall_rawSyscall(libc_socket_trampoline_addr, uintptr(domain), uintptr(typ), uintptr(proto))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -116,99 +111,92 @@
 	return
 }
 
-func libc_socket_trampoline()
+var libc_socket_trampoline_addr uintptr
 
-//go:linkname libc_socket libc_socket
 //go:cgo_import_dynamic libc_socket socket "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_getsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
+	_, _, e1 := syscall_syscall6(libc_getsockopt_trampoline_addr, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getsockopt_trampoline()
+var libc_getsockopt_trampoline_addr uintptr
 
-//go:linkname libc_getsockopt libc_getsockopt
 //go:cgo_import_dynamic libc_getsockopt getsockopt "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setsockopt_trampoline), uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
+	_, _, e1 := syscall_syscall6(libc_setsockopt_trampoline_addr, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setsockopt_trampoline()
+var libc_setsockopt_trampoline_addr uintptr
 
-//go:linkname libc_setsockopt libc_setsockopt
 //go:cgo_import_dynamic libc_setsockopt setsockopt "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getpeername_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	_, _, e1 := syscall_rawSyscall(libc_getpeername_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getpeername_trampoline()
+var libc_getpeername_trampoline_addr uintptr
 
-//go:linkname libc_getpeername libc_getpeername
 //go:cgo_import_dynamic libc_getpeername getpeername "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getsockname_trampoline), uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	_, _, e1 := syscall_rawSyscall(libc_getsockname_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getsockname_trampoline()
+var libc_getsockname_trampoline_addr uintptr
 
-//go:linkname libc_getsockname libc_getsockname
 //go:cgo_import_dynamic libc_getsockname getsockname "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Shutdown(s int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_shutdown_trampoline), uintptr(s), uintptr(how), 0)
+	_, _, e1 := syscall_syscall(libc_shutdown_trampoline_addr, uintptr(s), uintptr(how), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_shutdown_trampoline()
+var libc_shutdown_trampoline_addr uintptr
 
-//go:linkname libc_shutdown libc_shutdown
 //go:cgo_import_dynamic libc_shutdown shutdown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
-	_, _, e1 := syscall_rawSyscall6(funcPC(libc_socketpair_trampoline), uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
+	_, _, e1 := syscall_rawSyscall6(libc_socketpair_trampoline_addr, uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_socketpair_trampoline()
+var libc_socketpair_trampoline_addr uintptr
 
-//go:linkname libc_socketpair libc_socketpair
 //go:cgo_import_dynamic libc_socketpair socketpair "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -220,7 +208,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_recvfrom_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
+	r0, _, e1 := syscall_syscall6(libc_recvfrom_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -228,9 +216,8 @@
 	return
 }
 
-func libc_recvfrom_trampoline()
+var libc_recvfrom_trampoline_addr uintptr
 
-//go:linkname libc_recvfrom libc_recvfrom
 //go:cgo_import_dynamic libc_recvfrom recvfrom "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -242,22 +229,21 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendto_trampoline), uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
+	_, _, e1 := syscall_syscall6(libc_sendto_trampoline_addr, uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sendto_trampoline()
+var libc_sendto_trampoline_addr uintptr
 
-//go:linkname libc_sendto libc_sendto
 //go:cgo_import_dynamic libc_sendto sendto "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_recvmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_recvmsg_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -265,15 +251,14 @@
 	return
 }
 
-func libc_recvmsg_trampoline()
+var libc_recvmsg_trampoline_addr uintptr
 
-//go:linkname libc_recvmsg libc_recvmsg
 //go:cgo_import_dynamic libc_recvmsg recvmsg "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_sendmsg_trampoline), uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_sendmsg_trampoline_addr, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -281,15 +266,14 @@
 	return
 }
 
-func libc_sendmsg_trampoline()
+var libc_sendmsg_trampoline_addr uintptr
 
-//go:linkname libc_sendmsg libc_sendmsg
 //go:cgo_import_dynamic libc_sendmsg sendmsg "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func kevent(kq int, change unsafe.Pointer, nchange int, event unsafe.Pointer, nevent int, timeout *Timespec) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_kevent_trampoline), uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
+	r0, _, e1 := syscall_syscall6(libc_kevent_trampoline_addr, uintptr(kq), uintptr(change), uintptr(nchange), uintptr(event), uintptr(nevent), uintptr(unsafe.Pointer(timeout)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -297,9 +281,8 @@
 	return
 }
 
-func libc_kevent_trampoline()
+var libc_kevent_trampoline_addr uintptr
 
-//go:linkname libc_kevent libc_kevent
 //go:cgo_import_dynamic libc_kevent kevent "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -310,37 +293,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_utimes_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
+	_, _, e1 := syscall_syscall(libc_utimes_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_utimes_trampoline()
+var libc_utimes_trampoline_addr uintptr
 
-//go:linkname libc_utimes libc_utimes
 //go:cgo_import_dynamic libc_utimes utimes "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func futimes(fd int, timeval *[2]Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_futimes_trampoline), uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
+	_, _, e1 := syscall_syscall(libc_futimes_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(timeval)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_futimes_trampoline()
+var libc_futimes_trampoline_addr uintptr
 
-//go:linkname libc_futimes libc_futimes
 //go:cgo_import_dynamic libc_futimes futimes "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func poll(fds *PollFd, nfds int, timeout int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_poll_trampoline), uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
+	r0, _, e1 := syscall_syscall(libc_poll_trampoline_addr, uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -348,9 +329,8 @@
 	return
 }
 
-func libc_poll_trampoline()
+var libc_poll_trampoline_addr uintptr
 
-//go:linkname libc_poll libc_poll
 //go:cgo_import_dynamic libc_poll poll "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -362,16 +342,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_madvise_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(behav))
+	_, _, e1 := syscall_syscall(libc_madvise_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(behav))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_madvise_trampoline()
+var libc_madvise_trampoline_addr uintptr
 
-//go:linkname libc_madvise libc_madvise
 //go:cgo_import_dynamic libc_madvise madvise "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -383,31 +362,29 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
+	_, _, e1 := syscall_syscall(libc_mlock_trampoline_addr, uintptr(_p0), uintptr(len(b)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mlock_trampoline()
+var libc_mlock_trampoline_addr uintptr
 
-//go:linkname libc_mlock libc_mlock
 //go:cgo_import_dynamic libc_mlock mlock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Mlockall(flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_mlockall_trampoline), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall(libc_mlockall_trampoline_addr, uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mlockall_trampoline()
+var libc_mlockall_trampoline_addr uintptr
 
-//go:linkname libc_mlockall libc_mlockall
 //go:cgo_import_dynamic libc_mlockall mlockall "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -419,16 +396,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mprotect_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(prot))
+	_, _, e1 := syscall_syscall(libc_mprotect_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(prot))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mprotect_trampoline()
+var libc_mprotect_trampoline_addr uintptr
 
-//go:linkname libc_mprotect libc_mprotect
 //go:cgo_import_dynamic libc_mprotect mprotect "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -440,16 +416,15 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_msync_trampoline), uintptr(_p0), uintptr(len(b)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_msync_trampoline_addr, uintptr(_p0), uintptr(len(b)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_msync_trampoline()
+var libc_msync_trampoline_addr uintptr
 
-//go:linkname libc_msync libc_msync
 //go:cgo_import_dynamic libc_msync msync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -461,48 +436,43 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_munlock_trampoline), uintptr(_p0), uintptr(len(b)), 0)
+	_, _, e1 := syscall_syscall(libc_munlock_trampoline_addr, uintptr(_p0), uintptr(len(b)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munlock_trampoline()
+var libc_munlock_trampoline_addr uintptr
 
-//go:linkname libc_munlock libc_munlock
 //go:cgo_import_dynamic libc_munlock munlock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Munlockall() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munlockall_trampoline), 0, 0, 0)
+	_, _, e1 := syscall_syscall(libc_munlockall_trampoline_addr, 0, 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munlockall_trampoline()
+var libc_munlockall_trampoline_addr uintptr
 
-//go:linkname libc_munlockall libc_munlockall
 //go:cgo_import_dynamic libc_munlockall munlockall "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func pipe() (r int, w int, err error) {
-	r0, r1, e1 := syscall_rawSyscall(funcPC(libc_pipe_trampoline), 0, 0, 0)
-	r = int(r0)
-	w = int(r1)
+func pipe(p *[2]int32) (err error) {
+	_, _, e1 := syscall_rawSyscall(libc_pipe_trampoline_addr, uintptr(unsafe.Pointer(p)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_pipe_trampoline()
+var libc_pipe_trampoline_addr uintptr
 
-//go:linkname libc_pipe libc_pipe
 //go:cgo_import_dynamic libc_pipe pipe "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -518,7 +488,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_getxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
+	r0, _, e1 := syscall_syscall6(libc_getxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -526,9 +496,8 @@
 	return
 }
 
-func libc_getxattr_trampoline()
+var libc_getxattr_trampoline_addr uintptr
 
-//go:linkname libc_getxattr libc_getxattr
 //go:cgo_import_dynamic libc_getxattr getxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -539,7 +508,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_fgetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
+	r0, _, e1 := syscall_syscall6(libc_fgetxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(position), uintptr(options))
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -547,9 +516,8 @@
 	return
 }
 
-func libc_fgetxattr_trampoline()
+var libc_fgetxattr_trampoline_addr uintptr
 
-//go:linkname libc_fgetxattr libc_fgetxattr
 //go:cgo_import_dynamic libc_fgetxattr fgetxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -565,16 +533,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_setxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
+	_, _, e1 := syscall_syscall6(libc_setxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setxattr_trampoline()
+var libc_setxattr_trampoline_addr uintptr
 
-//go:linkname libc_setxattr libc_setxattr
 //go:cgo_import_dynamic libc_setxattr setxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -585,16 +552,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fsetxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
+	_, _, e1 := syscall_syscall6(libc_fsetxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(data)), uintptr(size), uintptr(position), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fsetxattr_trampoline()
+var libc_fsetxattr_trampoline_addr uintptr
 
-//go:linkname libc_fsetxattr libc_fsetxattr
 //go:cgo_import_dynamic libc_fsetxattr fsetxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -610,16 +576,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_removexattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_removexattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_removexattr_trampoline()
+var libc_removexattr_trampoline_addr uintptr
 
-//go:linkname libc_removexattr libc_removexattr
 //go:cgo_import_dynamic libc_removexattr removexattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -630,16 +595,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_fremovexattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_fremovexattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fremovexattr_trampoline()
+var libc_fremovexattr_trampoline_addr uintptr
 
-//go:linkname libc_fremovexattr libc_fremovexattr
 //go:cgo_import_dynamic libc_fremovexattr fremovexattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -650,7 +614,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_listxattr_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_listxattr_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -658,15 +622,14 @@
 	return
 }
 
-func libc_listxattr_trampoline()
+var libc_listxattr_trampoline_addr uintptr
 
-//go:linkname libc_listxattr libc_listxattr
 //go:cgo_import_dynamic libc_listxattr listxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func flistxattr(fd int, dest *byte, size int, options int) (sz int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_flistxattr_trampoline), uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_flistxattr_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(dest)), uintptr(size), uintptr(options), 0, 0)
 	sz = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -674,30 +637,28 @@
 	return
 }
 
-func libc_flistxattr_trampoline()
+var libc_flistxattr_trampoline_addr uintptr
 
-//go:linkname libc_flistxattr libc_flistxattr
 //go:cgo_import_dynamic libc_flistxattr flistxattr "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func setattrlist(path *byte, list unsafe.Pointer, buf unsafe.Pointer, size uintptr, options int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_setattrlist_trampoline), uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
+	_, _, e1 := syscall_syscall6(libc_setattrlist_trampoline_addr, uintptr(unsafe.Pointer(path)), uintptr(list), uintptr(buf), uintptr(size), uintptr(options), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setattrlist_trampoline()
+var libc_setattrlist_trampoline_addr uintptr
 
-//go:linkname libc_setattrlist libc_setattrlist
 //go:cgo_import_dynamic libc_setattrlist setattrlist "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func fcntl(fd int, cmd int, arg int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fcntl_trampoline), uintptr(fd), uintptr(cmd), uintptr(arg))
+	r0, _, e1 := syscall_syscall(libc_fcntl_trampoline_addr, uintptr(fd), uintptr(cmd), uintptr(arg))
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -705,39 +666,36 @@
 	return
 }
 
-func libc_fcntl_trampoline()
+var libc_fcntl_trampoline_addr uintptr
 
-//go:linkname libc_fcntl libc_fcntl
 //go:cgo_import_dynamic libc_fcntl fcntl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func kill(pid int, signum int, posix int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_kill_trampoline), uintptr(pid), uintptr(signum), uintptr(posix))
+	_, _, e1 := syscall_syscall(libc_kill_trampoline_addr, uintptr(pid), uintptr(signum), uintptr(posix))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_kill_trampoline()
+var libc_kill_trampoline_addr uintptr
 
-//go:linkname libc_kill libc_kill
 //go:cgo_import_dynamic libc_kill kill "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func ioctl(fd int, req uint, arg uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ioctl_trampoline), uintptr(fd), uintptr(req), uintptr(arg))
+	_, _, e1 := syscall_syscall(libc_ioctl_trampoline_addr, uintptr(fd), uintptr(req), uintptr(arg))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_ioctl_trampoline()
+var libc_ioctl_trampoline_addr uintptr
 
-//go:linkname libc_ioctl libc_ioctl
 //go:cgo_import_dynamic libc_ioctl ioctl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -749,31 +707,29 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_sysctl_trampoline), uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
+	_, _, e1 := syscall_syscall6(libc_sysctl_trampoline_addr, uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sysctl_trampoline()
+var libc_sysctl_trampoline_addr uintptr
 
-//go:linkname libc_sysctl libc_sysctl
 //go:cgo_import_dynamic libc_sysctl sysctl "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func sendfile(infd int, outfd int, offset int64, len *int64, hdtr unsafe.Pointer, flags int) (err error) {
-	_, _, e1 := syscall_syscall6(funcPC(libc_sendfile_trampoline), uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags))
+	_, _, e1 := syscall_syscall6(libc_sendfile_trampoline_addr, uintptr(infd), uintptr(outfd), uintptr(offset), uintptr(unsafe.Pointer(len)), uintptr(hdtr), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sendfile_trampoline()
+var libc_sendfile_trampoline_addr uintptr
 
-//go:linkname libc_sendfile libc_sendfile
 //go:cgo_import_dynamic libc_sendfile sendfile "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -784,31 +740,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_access_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_access_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_access_trampoline()
+var libc_access_trampoline_addr uintptr
 
-//go:linkname libc_access libc_access
 //go:cgo_import_dynamic libc_access access "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Adjtime(delta *Timeval, olddelta *Timeval) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_adjtime_trampoline), uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
+	_, _, e1 := syscall_syscall(libc_adjtime_trampoline_addr, uintptr(unsafe.Pointer(delta)), uintptr(unsafe.Pointer(olddelta)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_adjtime_trampoline()
+var libc_adjtime_trampoline_addr uintptr
 
-//go:linkname libc_adjtime libc_adjtime
 //go:cgo_import_dynamic libc_adjtime adjtime "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -819,16 +773,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_chdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chdir_trampoline()
+var libc_chdir_trampoline_addr uintptr
 
-//go:linkname libc_chdir libc_chdir
 //go:cgo_import_dynamic libc_chdir chdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -839,16 +792,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chflags_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_chflags_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chflags_trampoline()
+var libc_chflags_trampoline_addr uintptr
 
-//go:linkname libc_chflags libc_chflags
 //go:cgo_import_dynamic libc_chflags chflags "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -859,16 +811,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chmod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_chmod_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chmod_trampoline()
+var libc_chmod_trampoline_addr uintptr
 
-//go:linkname libc_chmod libc_chmod
 //go:cgo_import_dynamic libc_chmod chmod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -879,16 +830,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_chown_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chown_trampoline()
+var libc_chown_trampoline_addr uintptr
 
-//go:linkname libc_chown libc_chown
 //go:cgo_import_dynamic libc_chown chown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -899,46 +849,43 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_chroot_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_chroot_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_chroot_trampoline()
+var libc_chroot_trampoline_addr uintptr
 
-//go:linkname libc_chroot libc_chroot
 //go:cgo_import_dynamic libc_chroot chroot "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func ClockGettime(clockid int32, time *Timespec) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_clock_gettime_trampoline), uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
+	_, _, e1 := syscall_syscall(libc_clock_gettime_trampoline_addr, uintptr(clockid), uintptr(unsafe.Pointer(time)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clock_gettime_trampoline()
+var libc_clock_gettime_trampoline_addr uintptr
 
-//go:linkname libc_clock_gettime libc_clock_gettime
 //go:cgo_import_dynamic libc_clock_gettime clock_gettime "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Close(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_close_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_close_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_close_trampoline()
+var libc_close_trampoline_addr uintptr
 
-//go:linkname libc_close libc_close
 //go:cgo_import_dynamic libc_close close "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -954,16 +901,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_clonefile_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_clonefile_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clonefile_trampoline()
+var libc_clonefile_trampoline_addr uintptr
 
-//go:linkname libc_clonefile libc_clonefile
 //go:cgo_import_dynamic libc_clonefile clonefile "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -979,22 +925,21 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_clonefileat_trampoline), uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_clonefileat_trampoline_addr, uintptr(srcDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_clonefileat_trampoline()
+var libc_clonefileat_trampoline_addr uintptr
 
-//go:linkname libc_clonefileat libc_clonefileat
 //go:cgo_import_dynamic libc_clonefileat clonefileat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Dup(fd int) (nfd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_dup_trampoline), uintptr(fd), 0, 0)
+	r0, _, e1 := syscall_syscall(libc_dup_trampoline_addr, uintptr(fd), 0, 0)
 	nfd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1002,24 +947,22 @@
 	return
 }
 
-func libc_dup_trampoline()
+var libc_dup_trampoline_addr uintptr
 
-//go:linkname libc_dup libc_dup
 //go:cgo_import_dynamic libc_dup dup "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Dup2(from int, to int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_dup2_trampoline), uintptr(from), uintptr(to), 0)
+	_, _, e1 := syscall_syscall(libc_dup2_trampoline_addr, uintptr(from), uintptr(to), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_dup2_trampoline()
+var libc_dup2_trampoline_addr uintptr
 
-//go:linkname libc_dup2 libc_dup2
 //go:cgo_import_dynamic libc_dup2 dup2 "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1035,28 +978,26 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_exchangedata_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
+	_, _, e1 := syscall_syscall(libc_exchangedata_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(options))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_exchangedata_trampoline()
+var libc_exchangedata_trampoline_addr uintptr
 
-//go:linkname libc_exchangedata libc_exchangedata
 //go:cgo_import_dynamic libc_exchangedata exchangedata "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Exit(code int) {
-	syscall_syscall(funcPC(libc_exit_trampoline), uintptr(code), 0, 0)
+	syscall_syscall(libc_exit_trampoline_addr, uintptr(code), 0, 0)
 	return
 }
 
-func libc_exit_trampoline()
+var libc_exit_trampoline_addr uintptr
 
-//go:linkname libc_exit libc_exit
 //go:cgo_import_dynamic libc_exit exit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1067,61 +1008,57 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_faccessat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_faccessat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_faccessat_trampoline()
+var libc_faccessat_trampoline_addr uintptr
 
-//go:linkname libc_faccessat libc_faccessat
 //go:cgo_import_dynamic libc_faccessat faccessat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchdir(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchdir_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_fchdir_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchdir_trampoline()
+var libc_fchdir_trampoline_addr uintptr
 
-//go:linkname libc_fchdir libc_fchdir
 //go:cgo_import_dynamic libc_fchdir fchdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchflags(fd int, flags int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchflags_trampoline), uintptr(fd), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_fchflags_trampoline_addr, uintptr(fd), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchflags_trampoline()
+var libc_fchflags_trampoline_addr uintptr
 
-//go:linkname libc_fchflags libc_fchflags
 //go:cgo_import_dynamic libc_fchflags fchflags "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchmod(fd int, mode uint32) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchmod_trampoline), uintptr(fd), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_fchmod_trampoline_addr, uintptr(fd), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchmod_trampoline()
+var libc_fchmod_trampoline_addr uintptr
 
-//go:linkname libc_fchmod libc_fchmod
 //go:cgo_import_dynamic libc_fchmod fchmod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1132,31 +1069,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchmodat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fchmodat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchmodat_trampoline()
+var libc_fchmodat_trampoline_addr uintptr
 
-//go:linkname libc_fchmodat libc_fchmodat
 //go:cgo_import_dynamic libc_fchmodat fchmodat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fchown(fd int, uid int, gid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fchown_trampoline), uintptr(fd), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_fchown_trampoline_addr, uintptr(fd), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchown_trampoline()
+var libc_fchown_trampoline_addr uintptr
 
-//go:linkname libc_fchown libc_fchown
 //go:cgo_import_dynamic libc_fchown fchown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1167,16 +1102,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fchownat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_fchownat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fchownat_trampoline()
+var libc_fchownat_trampoline_addr uintptr
 
-//go:linkname libc_fchownat libc_fchownat
 //go:cgo_import_dynamic libc_fchownat fchownat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1187,37 +1121,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fclonefileat_trampoline), uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fclonefileat_trampoline_addr, uintptr(srcDirfd), uintptr(dstDirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fclonefileat_trampoline()
+var libc_fclonefileat_trampoline_addr uintptr
 
-//go:linkname libc_fclonefileat libc_fclonefileat
 //go:cgo_import_dynamic libc_fclonefileat fclonefileat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Flock(fd int, how int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_flock_trampoline), uintptr(fd), uintptr(how), 0)
+	_, _, e1 := syscall_syscall(libc_flock_trampoline_addr, uintptr(fd), uintptr(how), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_flock_trampoline()
+var libc_flock_trampoline_addr uintptr
 
-//go:linkname libc_flock libc_flock
 //go:cgo_import_dynamic libc_flock flock "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fpathconf(fd int, name int) (val int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_fpathconf_trampoline), uintptr(fd), uintptr(name), 0)
+	r0, _, e1 := syscall_syscall(libc_fpathconf_trampoline_addr, uintptr(fd), uintptr(name), 0)
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1225,39 +1157,36 @@
 	return
 }
 
-func libc_fpathconf_trampoline()
+var libc_fpathconf_trampoline_addr uintptr
 
-//go:linkname libc_fpathconf libc_fpathconf
 //go:cgo_import_dynamic libc_fpathconf fpathconf "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fsync(fd int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fsync_trampoline), uintptr(fd), 0, 0)
+	_, _, e1 := syscall_syscall(libc_fsync_trampoline_addr, uintptr(fd), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fsync_trampoline()
+var libc_fsync_trampoline_addr uintptr
 
-//go:linkname libc_fsync libc_fsync
 //go:cgo_import_dynamic libc_fsync fsync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Ftruncate(fd int, length int64) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_ftruncate_trampoline), uintptr(fd), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_ftruncate_trampoline_addr, uintptr(fd), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_ftruncate_trampoline()
+var libc_ftruncate_trampoline_addr uintptr
 
-//go:linkname libc_ftruncate libc_ftruncate
 //go:cgo_import_dynamic libc_ftruncate ftruncate "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1269,7 +1198,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_getcwd_trampoline), uintptr(_p0), uintptr(len(buf)), 0)
+	r0, _, e1 := syscall_syscall(libc_getcwd_trampoline_addr, uintptr(_p0), uintptr(len(buf)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1277,67 +1206,62 @@
 	return
 }
 
-func libc_getcwd_trampoline()
+var libc_getcwd_trampoline_addr uintptr
 
-//go:linkname libc_getcwd libc_getcwd
 //go:cgo_import_dynamic libc_getcwd getcwd "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getdtablesize() (size int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_getdtablesize_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_syscall(libc_getdtablesize_trampoline_addr, 0, 0, 0)
 	size = int(r0)
 	return
 }
 
-func libc_getdtablesize_trampoline()
+var libc_getdtablesize_trampoline_addr uintptr
 
-//go:linkname libc_getdtablesize libc_getdtablesize
 //go:cgo_import_dynamic libc_getdtablesize getdtablesize "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getegid() (egid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getegid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getegid_trampoline_addr, 0, 0, 0)
 	egid = int(r0)
 	return
 }
 
-func libc_getegid_trampoline()
+var libc_getegid_trampoline_addr uintptr
 
-//go:linkname libc_getegid libc_getegid
 //go:cgo_import_dynamic libc_getegid getegid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Geteuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_geteuid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_geteuid_trampoline_addr, 0, 0, 0)
 	uid = int(r0)
 	return
 }
 
-func libc_geteuid_trampoline()
+var libc_geteuid_trampoline_addr uintptr
 
-//go:linkname libc_geteuid libc_geteuid
 //go:cgo_import_dynamic libc_geteuid geteuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getgid() (gid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getgid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getgid_trampoline_addr, 0, 0, 0)
 	gid = int(r0)
 	return
 }
 
-func libc_getgid_trampoline()
+var libc_getgid_trampoline_addr uintptr
 
-//go:linkname libc_getgid libc_getgid
 //go:cgo_import_dynamic libc_getgid getgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpgid(pid int) (pgid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getpgid_trampoline), uintptr(pid), 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getpgid_trampoline_addr, uintptr(pid), 0, 0)
 	pgid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1345,54 +1269,50 @@
 	return
 }
 
-func libc_getpgid_trampoline()
+var libc_getpgid_trampoline_addr uintptr
 
-//go:linkname libc_getpgid libc_getpgid
 //go:cgo_import_dynamic libc_getpgid getpgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpgrp() (pgrp int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpgrp_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getpgrp_trampoline_addr, 0, 0, 0)
 	pgrp = int(r0)
 	return
 }
 
-func libc_getpgrp_trampoline()
+var libc_getpgrp_trampoline_addr uintptr
 
-//go:linkname libc_getpgrp libc_getpgrp
 //go:cgo_import_dynamic libc_getpgrp getpgrp "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpid() (pid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getpid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getpid_trampoline_addr, 0, 0, 0)
 	pid = int(r0)
 	return
 }
 
-func libc_getpid_trampoline()
+var libc_getpid_trampoline_addr uintptr
 
-//go:linkname libc_getpid libc_getpid
 //go:cgo_import_dynamic libc_getpid getpid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getppid() (ppid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getppid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getppid_trampoline_addr, 0, 0, 0)
 	ppid = int(r0)
 	return
 }
 
-func libc_getppid_trampoline()
+var libc_getppid_trampoline_addr uintptr
 
-//go:linkname libc_getppid libc_getppid
 //go:cgo_import_dynamic libc_getppid getppid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getpriority(which int, who int) (prio int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getpriority_trampoline), uintptr(which), uintptr(who), 0)
+	r0, _, e1 := syscall_syscall(libc_getpriority_trampoline_addr, uintptr(which), uintptr(who), 0)
 	prio = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1400,45 +1320,42 @@
 	return
 }
 
-func libc_getpriority_trampoline()
+var libc_getpriority_trampoline_addr uintptr
 
-//go:linkname libc_getpriority libc_getpriority
 //go:cgo_import_dynamic libc_getpriority getpriority "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_getrlimit_trampoline_addr, uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getrlimit_trampoline()
+var libc_getrlimit_trampoline_addr uintptr
 
-//go:linkname libc_getrlimit libc_getrlimit
 //go:cgo_import_dynamic libc_getrlimit getrlimit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getrusage(who int, rusage *Rusage) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_getrusage_trampoline), uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_getrusage_trampoline_addr, uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_getrusage_trampoline()
+var libc_getrusage_trampoline_addr uintptr
 
-//go:linkname libc_getrusage libc_getrusage
 //go:cgo_import_dynamic libc_getrusage getrusage "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getsid(pid int) (sid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_getsid_trampoline), uintptr(pid), 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_getsid_trampoline_addr, uintptr(pid), 0, 0)
 	sid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1446,56 +1363,52 @@
 	return
 }
 
-func libc_getsid_trampoline()
+var libc_getsid_trampoline_addr uintptr
 
-//go:linkname libc_getsid libc_getsid
 //go:cgo_import_dynamic libc_getsid getsid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Gettimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_gettimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_gettimeofday_trampoline_addr, uintptr(unsafe.Pointer(tp)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_gettimeofday_trampoline()
+var libc_gettimeofday_trampoline_addr uintptr
 
-//go:linkname libc_gettimeofday libc_gettimeofday
 //go:cgo_import_dynamic libc_gettimeofday gettimeofday "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Getuid() (uid int) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_getuid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_getuid_trampoline_addr, 0, 0, 0)
 	uid = int(r0)
 	return
 }
 
-func libc_getuid_trampoline()
+var libc_getuid_trampoline_addr uintptr
 
-//go:linkname libc_getuid libc_getuid
 //go:cgo_import_dynamic libc_getuid getuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Issetugid() (tainted bool) {
-	r0, _, _ := syscall_rawSyscall(funcPC(libc_issetugid_trampoline), 0, 0, 0)
+	r0, _, _ := syscall_rawSyscall(libc_issetugid_trampoline_addr, 0, 0, 0)
 	tainted = bool(r0 != 0)
 	return
 }
 
-func libc_issetugid_trampoline()
+var libc_issetugid_trampoline_addr uintptr
 
-//go:linkname libc_issetugid libc_issetugid
 //go:cgo_import_dynamic libc_issetugid issetugid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Kqueue() (fd int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_kqueue_trampoline), 0, 0, 0)
+	r0, _, e1 := syscall_syscall(libc_kqueue_trampoline_addr, 0, 0, 0)
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1503,9 +1416,8 @@
 	return
 }
 
-func libc_kqueue_trampoline()
+var libc_kqueue_trampoline_addr uintptr
 
-//go:linkname libc_kqueue libc_kqueue
 //go:cgo_import_dynamic libc_kqueue kqueue "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1516,16 +1428,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lchown_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	_, _, e1 := syscall_syscall(libc_lchown_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_lchown_trampoline()
+var libc_lchown_trampoline_addr uintptr
 
-//go:linkname libc_lchown libc_lchown
 //go:cgo_import_dynamic libc_lchown lchown "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1541,16 +1452,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_link_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_link_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_link_trampoline()
+var libc_link_trampoline_addr uintptr
 
-//go:linkname libc_link libc_link
 //go:cgo_import_dynamic libc_link link "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1566,31 +1476,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_linkat_trampoline), uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall6(libc_linkat_trampoline_addr, uintptr(pathfd), uintptr(unsafe.Pointer(_p0)), uintptr(linkfd), uintptr(unsafe.Pointer(_p1)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_linkat_trampoline()
+var libc_linkat_trampoline_addr uintptr
 
-//go:linkname libc_linkat libc_linkat
 //go:cgo_import_dynamic libc_linkat linkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Listen(s int, backlog int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_listen_trampoline), uintptr(s), uintptr(backlog), 0)
+	_, _, e1 := syscall_syscall(libc_listen_trampoline_addr, uintptr(s), uintptr(backlog), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_listen_trampoline()
+var libc_listen_trampoline_addr uintptr
 
-//go:linkname libc_listen libc_listen
 //go:cgo_import_dynamic libc_listen listen "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1601,16 +1509,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdir_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_mkdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkdir_trampoline()
+var libc_mkdir_trampoline_addr uintptr
 
-//go:linkname libc_mkdir libc_mkdir
 //go:cgo_import_dynamic libc_mkdir mkdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1621,16 +1528,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkdirat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
+	_, _, e1 := syscall_syscall(libc_mkdirat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkdirat_trampoline()
+var libc_mkdirat_trampoline_addr uintptr
 
-//go:linkname libc_mkdirat libc_mkdirat
 //go:cgo_import_dynamic libc_mkdirat mkdirat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1641,16 +1547,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mkfifo_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	_, _, e1 := syscall_syscall(libc_mkfifo_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mkfifo_trampoline()
+var libc_mkfifo_trampoline_addr uintptr
 
-//go:linkname libc_mkfifo libc_mkfifo
 //go:cgo_import_dynamic libc_mkfifo mkfifo "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1661,16 +1566,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_mknod_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
+	_, _, e1 := syscall_syscall(libc_mknod_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_mknod_trampoline()
+var libc_mknod_trampoline_addr uintptr
 
-//go:linkname libc_mknod libc_mknod
 //go:cgo_import_dynamic libc_mknod mknod "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1681,7 +1585,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_open_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
+	r0, _, e1 := syscall_syscall(libc_open_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1689,9 +1593,8 @@
 	return
 }
 
-func libc_open_trampoline()
+var libc_open_trampoline_addr uintptr
 
-//go:linkname libc_open libc_open
 //go:cgo_import_dynamic libc_open open "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1702,7 +1605,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_openat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_openat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm), 0, 0)
 	fd = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1710,9 +1613,8 @@
 	return
 }
 
-func libc_openat_trampoline()
+var libc_openat_trampoline_addr uintptr
 
-//go:linkname libc_openat libc_openat
 //go:cgo_import_dynamic libc_openat openat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1723,7 +1625,7 @@
 	if err != nil {
 		return
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_pathconf_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
+	r0, _, e1 := syscall_syscall(libc_pathconf_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(name), 0)
 	val = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1731,9 +1633,8 @@
 	return
 }
 
-func libc_pathconf_trampoline()
+var libc_pathconf_trampoline_addr uintptr
 
-//go:linkname libc_pathconf libc_pathconf
 //go:cgo_import_dynamic libc_pathconf pathconf "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1745,7 +1646,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pread_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_pread_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1753,9 +1654,8 @@
 	return
 }
 
-func libc_pread_trampoline()
+var libc_pread_trampoline_addr uintptr
 
-//go:linkname libc_pread libc_pread
 //go:cgo_import_dynamic libc_pread pread "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1767,7 +1667,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_pwrite_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_pwrite_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1775,9 +1675,8 @@
 	return
 }
 
-func libc_pwrite_trampoline()
+var libc_pwrite_trampoline_addr uintptr
 
-//go:linkname libc_pwrite libc_pwrite
 //go:cgo_import_dynamic libc_pwrite pwrite "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1789,7 +1688,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	r0, _, e1 := syscall_syscall(libc_read_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1797,9 +1696,8 @@
 	return
 }
 
-func libc_read_trampoline()
+var libc_read_trampoline_addr uintptr
 
-//go:linkname libc_read libc_read
 //go:cgo_import_dynamic libc_read read "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1816,7 +1714,7 @@
 	} else {
 		_p1 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_readlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
+	r0, _, e1 := syscall_syscall(libc_readlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1824,9 +1722,8 @@
 	return
 }
 
-func libc_readlink_trampoline()
+var libc_readlink_trampoline_addr uintptr
 
-//go:linkname libc_readlink libc_readlink
 //go:cgo_import_dynamic libc_readlink readlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1843,7 +1740,7 @@
 	} else {
 		_p1 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall6(funcPC(libc_readlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
+	r0, _, e1 := syscall_syscall6(libc_readlinkat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)), 0, 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1851,9 +1748,8 @@
 	return
 }
 
-func libc_readlinkat_trampoline()
+var libc_readlinkat_trampoline_addr uintptr
 
-//go:linkname libc_readlinkat libc_readlinkat
 //go:cgo_import_dynamic libc_readlinkat readlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1869,16 +1765,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rename_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_rename_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_rename_trampoline()
+var libc_rename_trampoline_addr uintptr
 
-//go:linkname libc_rename libc_rename
 //go:cgo_import_dynamic libc_rename rename "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1894,16 +1789,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_renameat_trampoline), uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_renameat_trampoline_addr, uintptr(fromfd), uintptr(unsafe.Pointer(_p0)), uintptr(tofd), uintptr(unsafe.Pointer(_p1)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_renameat_trampoline()
+var libc_renameat_trampoline_addr uintptr
 
-//go:linkname libc_renameat libc_renameat
 //go:cgo_import_dynamic libc_renameat renameat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1914,16 +1808,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_revoke_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_revoke_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_revoke_trampoline()
+var libc_revoke_trampoline_addr uintptr
 
-//go:linkname libc_revoke libc_revoke
 //go:cgo_import_dynamic libc_revoke revoke "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -1934,22 +1827,21 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_rmdir_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_rmdir_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_rmdir_trampoline()
+var libc_rmdir_trampoline_addr uintptr
 
-//go:linkname libc_rmdir libc_rmdir
 //go:cgo_import_dynamic libc_rmdir rmdir "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Seek(fd int, offset int64, whence int) (newoffset int64, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_lseek_trampoline), uintptr(fd), uintptr(offset), uintptr(whence))
+	r0, _, e1 := syscall_syscall(libc_lseek_trampoline_addr, uintptr(fd), uintptr(offset), uintptr(whence))
 	newoffset = int64(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1957,15 +1849,14 @@
 	return
 }
 
-func libc_lseek_trampoline()
+var libc_lseek_trampoline_addr uintptr
 
-//go:linkname libc_lseek libc_lseek
 //go:cgo_import_dynamic libc_lseek lseek "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_select_trampoline), uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
+	r0, _, e1 := syscall_syscall6(libc_select_trampoline_addr, uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -1973,54 +1864,50 @@
 	return
 }
 
-func libc_select_trampoline()
+var libc_select_trampoline_addr uintptr
 
-//go:linkname libc_select libc_select
 //go:cgo_import_dynamic libc_select select "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setegid(egid int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setegid_trampoline), uintptr(egid), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setegid_trampoline_addr, uintptr(egid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setegid_trampoline()
+var libc_setegid_trampoline_addr uintptr
 
-//go:linkname libc_setegid libc_setegid
 //go:cgo_import_dynamic libc_setegid setegid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Seteuid(euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_seteuid_trampoline), uintptr(euid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_seteuid_trampoline_addr, uintptr(euid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_seteuid_trampoline()
+var libc_seteuid_trampoline_addr uintptr
 
-//go:linkname libc_seteuid libc_seteuid
 //go:cgo_import_dynamic libc_seteuid seteuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setgid(gid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setgid_trampoline), uintptr(gid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_setgid_trampoline_addr, uintptr(gid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setgid_trampoline()
+var libc_setgid_trampoline_addr uintptr
 
-//go:linkname libc_setgid libc_setgid
 //go:cgo_import_dynamic libc_setgid setgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2031,112 +1918,105 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_setlogin_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setlogin_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setlogin_trampoline()
+var libc_setlogin_trampoline_addr uintptr
 
-//go:linkname libc_setlogin libc_setlogin
 //go:cgo_import_dynamic libc_setlogin setlogin "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setpgid(pid int, pgid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setpgid_trampoline), uintptr(pid), uintptr(pgid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setpgid_trampoline_addr, uintptr(pid), uintptr(pgid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setpgid_trampoline()
+var libc_setpgid_trampoline_addr uintptr
 
-//go:linkname libc_setpgid libc_setpgid
 //go:cgo_import_dynamic libc_setpgid setpgid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setpriority(which int, who int, prio int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setpriority_trampoline), uintptr(which), uintptr(who), uintptr(prio))
+	_, _, e1 := syscall_syscall(libc_setpriority_trampoline_addr, uintptr(which), uintptr(who), uintptr(prio))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setpriority_trampoline()
+var libc_setpriority_trampoline_addr uintptr
 
-//go:linkname libc_setpriority libc_setpriority
 //go:cgo_import_dynamic libc_setpriority setpriority "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setprivexec(flag int) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_setprivexec_trampoline), uintptr(flag), 0, 0)
+	_, _, e1 := syscall_syscall(libc_setprivexec_trampoline_addr, uintptr(flag), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setprivexec_trampoline()
+var libc_setprivexec_trampoline_addr uintptr
 
-//go:linkname libc_setprivexec libc_setprivexec
 //go:cgo_import_dynamic libc_setprivexec setprivexec "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setregid(rgid int, egid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setregid_trampoline), uintptr(rgid), uintptr(egid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setregid_trampoline_addr, uintptr(rgid), uintptr(egid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setregid_trampoline()
+var libc_setregid_trampoline_addr uintptr
 
-//go:linkname libc_setregid libc_setregid
 //go:cgo_import_dynamic libc_setregid setregid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setreuid(ruid int, euid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setreuid_trampoline), uintptr(ruid), uintptr(euid), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setreuid_trampoline_addr, uintptr(ruid), uintptr(euid), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setreuid_trampoline()
+var libc_setreuid_trampoline_addr uintptr
 
-//go:linkname libc_setreuid libc_setreuid
 //go:cgo_import_dynamic libc_setreuid setreuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setrlimit(which int, lim *Rlimit) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setrlimit_trampoline), uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
+	_, _, e1 := syscall_rawSyscall(libc_setrlimit_trampoline_addr, uintptr(which), uintptr(unsafe.Pointer(lim)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setrlimit_trampoline()
+var libc_setrlimit_trampoline_addr uintptr
 
-//go:linkname libc_setrlimit libc_setrlimit
 //go:cgo_import_dynamic libc_setrlimit setrlimit "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setsid() (pid int, err error) {
-	r0, _, e1 := syscall_rawSyscall(funcPC(libc_setsid_trampoline), 0, 0, 0)
+	r0, _, e1 := syscall_rawSyscall(libc_setsid_trampoline_addr, 0, 0, 0)
 	pid = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2144,39 +2024,36 @@
 	return
 }
 
-func libc_setsid_trampoline()
+var libc_setsid_trampoline_addr uintptr
 
-//go:linkname libc_setsid libc_setsid
 //go:cgo_import_dynamic libc_setsid setsid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Settimeofday(tp *Timeval) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_settimeofday_trampoline), uintptr(unsafe.Pointer(tp)), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_settimeofday_trampoline_addr, uintptr(unsafe.Pointer(tp)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_settimeofday_trampoline()
+var libc_settimeofday_trampoline_addr uintptr
 
-//go:linkname libc_settimeofday libc_settimeofday
 //go:cgo_import_dynamic libc_settimeofday settimeofday "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Setuid(uid int) (err error) {
-	_, _, e1 := syscall_rawSyscall(funcPC(libc_setuid_trampoline), uintptr(uid), 0, 0)
+	_, _, e1 := syscall_rawSyscall(libc_setuid_trampoline_addr, uintptr(uid), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_setuid_trampoline()
+var libc_setuid_trampoline_addr uintptr
 
-//go:linkname libc_setuid libc_setuid
 //go:cgo_import_dynamic libc_setuid setuid "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2192,16 +2069,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlink_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	_, _, e1 := syscall_syscall(libc_symlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_symlink_trampoline()
+var libc_symlink_trampoline_addr uintptr
 
-//go:linkname libc_symlink libc_symlink
 //go:cgo_import_dynamic libc_symlink symlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2217,31 +2093,29 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_symlinkat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
+	_, _, e1 := syscall_syscall(libc_symlinkat_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_symlinkat_trampoline()
+var libc_symlinkat_trampoline_addr uintptr
 
-//go:linkname libc_symlinkat libc_symlinkat
 //go:cgo_import_dynamic libc_symlinkat symlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Sync() (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_sync_trampoline), 0, 0, 0)
+	_, _, e1 := syscall_syscall(libc_sync_trampoline_addr, 0, 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_sync_trampoline()
+var libc_sync_trampoline_addr uintptr
 
-//go:linkname libc_sync libc_sync
 //go:cgo_import_dynamic libc_sync sync "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2252,29 +2126,27 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_truncate_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_truncate_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_truncate_trampoline()
+var libc_truncate_trampoline_addr uintptr
 
-//go:linkname libc_truncate libc_truncate
 //go:cgo_import_dynamic libc_truncate truncate "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Umask(newmask int) (oldmask int) {
-	r0, _, _ := syscall_syscall(funcPC(libc_umask_trampoline), uintptr(newmask), 0, 0)
+	r0, _, _ := syscall_syscall(libc_umask_trampoline_addr, uintptr(newmask), 0, 0)
 	oldmask = int(r0)
 	return
 }
 
-func libc_umask_trampoline()
+var libc_umask_trampoline_addr uintptr
 
-//go:linkname libc_umask libc_umask
 //go:cgo_import_dynamic libc_umask umask "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2285,16 +2157,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_undelete_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_undelete_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_undelete_trampoline()
+var libc_undelete_trampoline_addr uintptr
 
-//go:linkname libc_undelete libc_undelete
 //go:cgo_import_dynamic libc_undelete undelete "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2305,16 +2176,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlink_trampoline), uintptr(unsafe.Pointer(_p0)), 0, 0)
+	_, _, e1 := syscall_syscall(libc_unlink_trampoline_addr, uintptr(unsafe.Pointer(_p0)), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unlink_trampoline()
+var libc_unlink_trampoline_addr uintptr
 
-//go:linkname libc_unlink libc_unlink
 //go:cgo_import_dynamic libc_unlink unlink "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2325,16 +2195,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unlinkat_trampoline), uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
+	_, _, e1 := syscall_syscall(libc_unlinkat_trampoline_addr, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(flags))
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unlinkat_trampoline()
+var libc_unlinkat_trampoline_addr uintptr
 
-//go:linkname libc_unlinkat libc_unlinkat
 //go:cgo_import_dynamic libc_unlinkat unlinkat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2345,16 +2214,15 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_unmount_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
+	_, _, e1 := syscall_syscall(libc_unmount_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_unmount_trampoline()
+var libc_unmount_trampoline_addr uintptr
 
-//go:linkname libc_unmount libc_unmount
 //go:cgo_import_dynamic libc_unmount unmount "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2366,7 +2234,7 @@
 	} else {
 		_p0 = unsafe.Pointer(&_zero)
 	}
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	r0, _, e1 := syscall_syscall(libc_write_trampoline_addr, uintptr(fd), uintptr(_p0), uintptr(len(p)))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2374,15 +2242,14 @@
 	return
 }
 
-func libc_write_trampoline()
+var libc_write_trampoline_addr uintptr
 
-//go:linkname libc_write libc_write
 //go:cgo_import_dynamic libc_write write "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) {
-	r0, _, e1 := syscall_syscall6(funcPC(libc_mmap_trampoline), uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos))
+	r0, _, e1 := syscall_syscall6(libc_mmap_trampoline_addr, uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos))
 	ret = uintptr(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2390,30 +2257,28 @@
 	return
 }
 
-func libc_mmap_trampoline()
+var libc_mmap_trampoline_addr uintptr
 
-//go:linkname libc_mmap libc_mmap
 //go:cgo_import_dynamic libc_mmap mmap "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func munmap(addr uintptr, length uintptr) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_munmap_trampoline), uintptr(addr), uintptr(length), 0)
+	_, _, e1 := syscall_syscall(libc_munmap_trampoline_addr, uintptr(addr), uintptr(length), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_munmap_trampoline()
+var libc_munmap_trampoline_addr uintptr
 
-//go:linkname libc_munmap libc_munmap
 //go:cgo_import_dynamic libc_munmap munmap "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func readlen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_read_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
+	r0, _, e1 := syscall_syscall(libc_read_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2424,7 +2289,7 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func writelen(fd int, buf *byte, nbuf int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_write_trampoline), uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
+	r0, _, e1 := syscall_syscall(libc_write_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2435,16 +2300,15 @@
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fstat(fd int, stat *Stat_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstat_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_fstat_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstat_trampoline()
+var libc_fstat_trampoline_addr uintptr
 
-//go:linkname libc_fstat libc_fstat
 //go:cgo_import_dynamic libc_fstat fstat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2455,37 +2319,35 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall6(funcPC(libc_fstatat_trampoline), uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
+	_, _, e1 := syscall_syscall6(libc_fstatat_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstatat_trampoline()
+var libc_fstatat_trampoline_addr uintptr
 
-//go:linkname libc_fstatat libc_fstatat
 //go:cgo_import_dynamic libc_fstatat fstatat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func Fstatfs(fd int, stat *Statfs_t) (err error) {
-	_, _, e1 := syscall_syscall(funcPC(libc_fstatfs_trampoline), uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_fstatfs_trampoline_addr, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_fstatfs_trampoline()
+var libc_fstatfs_trampoline_addr uintptr
 
-//go:linkname libc_fstatfs libc_fstatfs
 //go:cgo_import_dynamic libc_fstatfs fstatfs "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
 func getfsstat(buf unsafe.Pointer, size uintptr, flags int) (n int, err error) {
-	r0, _, e1 := syscall_syscall(funcPC(libc_getfsstat_trampoline), uintptr(buf), uintptr(size), uintptr(flags))
+	r0, _, e1 := syscall_syscall(libc_getfsstat_trampoline_addr, uintptr(buf), uintptr(size), uintptr(flags))
 	n = int(r0)
 	if e1 != 0 {
 		err = errnoErr(e1)
@@ -2493,9 +2355,8 @@
 	return
 }
 
-func libc_getfsstat_trampoline()
+var libc_getfsstat_trampoline_addr uintptr
 
-//go:linkname libc_getfsstat libc_getfsstat
 //go:cgo_import_dynamic libc_getfsstat getfsstat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2506,36 +2367,48 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_lstat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_lstat_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_lstat_trampoline()
+var libc_lstat_trampoline_addr uintptr
 
-//go:linkname libc_lstat libc_lstat
 //go:cgo_import_dynamic libc_lstat lstat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func ptrace1(request int, pid int, addr uintptr, data uintptr) (err error) {
+	_, _, e1 := syscall_syscall6(libc_ptrace_trampoline_addr, uintptr(request), uintptr(pid), uintptr(addr), uintptr(data), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+var libc_ptrace_trampoline_addr uintptr
+
+//go:cgo_import_dynamic libc_ptrace ptrace "/usr/lib/libSystem.B.dylib"
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Stat(path string, stat *Stat_t) (err error) {
 	var _p0 *byte
 	_p0, err = BytePtrFromString(path)
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_stat_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_stat_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_stat_trampoline()
+var libc_stat_trampoline_addr uintptr
 
-//go:linkname libc_stat libc_stat
 //go:cgo_import_dynamic libc_stat stat "/usr/lib/libSystem.B.dylib"
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -2546,14 +2419,13 @@
 	if err != nil {
 		return
 	}
-	_, _, e1 := syscall_syscall(funcPC(libc_statfs_trampoline), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	_, _, e1 := syscall_syscall(libc_statfs_trampoline_addr, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
 	if e1 != 0 {
 		err = errnoErr(e1)
 	}
 	return
 }
 
-func libc_statfs_trampoline()
+var libc_statfs_trampoline_addr uintptr
 
-//go:linkname libc_statfs libc_statfs
 //go:cgo_import_dynamic libc_statfs statfs "/usr/lib/libSystem.B.dylib"
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.s b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.s
index 16aebee..33e1977 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.s
+++ b/vendor/golang.org/x/sys/unix/zsyscall_darwin_arm64.s
@@ -1,288 +1,859 @@
 // go run mkasm_darwin.go arm64
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build go1.12
 // +build go1.12
 
 #include "textflag.h"
-TEXT ·libc_getgroups_trampoline(SB),NOSPLIT,$0-0
+
+TEXT libc_getgroups_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getgroups(SB)
-TEXT ·libc_setgroups_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getgroups_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getgroups_trampoline_addr(SB)/8, $libc_getgroups_trampoline<>(SB)
+
+TEXT libc_setgroups_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setgroups(SB)
-TEXT ·libc_wait4_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setgroups_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setgroups_trampoline_addr(SB)/8, $libc_setgroups_trampoline<>(SB)
+
+TEXT libc_wait4_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_wait4(SB)
-TEXT ·libc_accept_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_wait4_trampoline_addr(SB), RODATA, $8
+DATA	·libc_wait4_trampoline_addr(SB)/8, $libc_wait4_trampoline<>(SB)
+
+TEXT libc_accept_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_accept(SB)
-TEXT ·libc_bind_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_accept_trampoline_addr(SB), RODATA, $8
+DATA	·libc_accept_trampoline_addr(SB)/8, $libc_accept_trampoline<>(SB)
+
+TEXT libc_bind_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_bind(SB)
-TEXT ·libc_connect_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_bind_trampoline_addr(SB), RODATA, $8
+DATA	·libc_bind_trampoline_addr(SB)/8, $libc_bind_trampoline<>(SB)
+
+TEXT libc_connect_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_connect(SB)
-TEXT ·libc_socket_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_connect_trampoline_addr(SB), RODATA, $8
+DATA	·libc_connect_trampoline_addr(SB)/8, $libc_connect_trampoline<>(SB)
+
+TEXT libc_socket_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_socket(SB)
-TEXT ·libc_getsockopt_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_socket_trampoline_addr(SB), RODATA, $8
+DATA	·libc_socket_trampoline_addr(SB)/8, $libc_socket_trampoline<>(SB)
+
+TEXT libc_getsockopt_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsockopt(SB)
-TEXT ·libc_setsockopt_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsockopt_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsockopt_trampoline_addr(SB)/8, $libc_getsockopt_trampoline<>(SB)
+
+TEXT libc_setsockopt_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setsockopt(SB)
-TEXT ·libc_getpeername_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setsockopt_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setsockopt_trampoline_addr(SB)/8, $libc_setsockopt_trampoline<>(SB)
+
+TEXT libc_getpeername_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpeername(SB)
-TEXT ·libc_getsockname_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpeername_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpeername_trampoline_addr(SB)/8, $libc_getpeername_trampoline<>(SB)
+
+TEXT libc_getsockname_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsockname(SB)
-TEXT ·libc_shutdown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsockname_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsockname_trampoline_addr(SB)/8, $libc_getsockname_trampoline<>(SB)
+
+TEXT libc_shutdown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_shutdown(SB)
-TEXT ·libc_socketpair_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_shutdown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_shutdown_trampoline_addr(SB)/8, $libc_shutdown_trampoline<>(SB)
+
+TEXT libc_socketpair_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_socketpair(SB)
-TEXT ·libc_recvfrom_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_socketpair_trampoline_addr(SB), RODATA, $8
+DATA	·libc_socketpair_trampoline_addr(SB)/8, $libc_socketpair_trampoline<>(SB)
+
+TEXT libc_recvfrom_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_recvfrom(SB)
-TEXT ·libc_sendto_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_recvfrom_trampoline_addr(SB), RODATA, $8
+DATA	·libc_recvfrom_trampoline_addr(SB)/8, $libc_recvfrom_trampoline<>(SB)
+
+TEXT libc_sendto_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendto(SB)
-TEXT ·libc_recvmsg_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendto_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendto_trampoline_addr(SB)/8, $libc_sendto_trampoline<>(SB)
+
+TEXT libc_recvmsg_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_recvmsg(SB)
-TEXT ·libc_sendmsg_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_recvmsg_trampoline_addr(SB), RODATA, $8
+DATA	·libc_recvmsg_trampoline_addr(SB)/8, $libc_recvmsg_trampoline<>(SB)
+
+TEXT libc_sendmsg_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendmsg(SB)
-TEXT ·libc_kevent_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendmsg_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendmsg_trampoline_addr(SB)/8, $libc_sendmsg_trampoline<>(SB)
+
+TEXT libc_kevent_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kevent(SB)
-TEXT ·libc_utimes_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kevent_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kevent_trampoline_addr(SB)/8, $libc_kevent_trampoline<>(SB)
+
+TEXT libc_utimes_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_utimes(SB)
-TEXT ·libc_futimes_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_utimes_trampoline_addr(SB), RODATA, $8
+DATA	·libc_utimes_trampoline_addr(SB)/8, $libc_utimes_trampoline<>(SB)
+
+TEXT libc_futimes_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_futimes(SB)
-TEXT ·libc_poll_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_futimes_trampoline_addr(SB), RODATA, $8
+DATA	·libc_futimes_trampoline_addr(SB)/8, $libc_futimes_trampoline<>(SB)
+
+TEXT libc_poll_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_poll(SB)
-TEXT ·libc_madvise_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_poll_trampoline_addr(SB), RODATA, $8
+DATA	·libc_poll_trampoline_addr(SB)/8, $libc_poll_trampoline<>(SB)
+
+TEXT libc_madvise_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_madvise(SB)
-TEXT ·libc_mlock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_madvise_trampoline_addr(SB), RODATA, $8
+DATA	·libc_madvise_trampoline_addr(SB)/8, $libc_madvise_trampoline<>(SB)
+
+TEXT libc_mlock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mlock(SB)
-TEXT ·libc_mlockall_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mlock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mlock_trampoline_addr(SB)/8, $libc_mlock_trampoline<>(SB)
+
+TEXT libc_mlockall_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mlockall(SB)
-TEXT ·libc_mprotect_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mlockall_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mlockall_trampoline_addr(SB)/8, $libc_mlockall_trampoline<>(SB)
+
+TEXT libc_mprotect_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mprotect(SB)
-TEXT ·libc_msync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mprotect_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mprotect_trampoline_addr(SB)/8, $libc_mprotect_trampoline<>(SB)
+
+TEXT libc_msync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_msync(SB)
-TEXT ·libc_munlock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_msync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_msync_trampoline_addr(SB)/8, $libc_msync_trampoline<>(SB)
+
+TEXT libc_munlock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munlock(SB)
-TEXT ·libc_munlockall_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munlock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munlock_trampoline_addr(SB)/8, $libc_munlock_trampoline<>(SB)
+
+TEXT libc_munlockall_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munlockall(SB)
-TEXT ·libc_pipe_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munlockall_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munlockall_trampoline_addr(SB)/8, $libc_munlockall_trampoline<>(SB)
+
+TEXT libc_pipe_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pipe(SB)
-TEXT ·libc_getxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pipe_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pipe_trampoline_addr(SB)/8, $libc_pipe_trampoline<>(SB)
+
+TEXT libc_getxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getxattr(SB)
-TEXT ·libc_fgetxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getxattr_trampoline_addr(SB)/8, $libc_getxattr_trampoline<>(SB)
+
+TEXT libc_fgetxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fgetxattr(SB)
-TEXT ·libc_setxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fgetxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fgetxattr_trampoline_addr(SB)/8, $libc_fgetxattr_trampoline<>(SB)
+
+TEXT libc_setxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setxattr(SB)
-TEXT ·libc_fsetxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setxattr_trampoline_addr(SB)/8, $libc_setxattr_trampoline<>(SB)
+
+TEXT libc_fsetxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fsetxattr(SB)
-TEXT ·libc_removexattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fsetxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fsetxattr_trampoline_addr(SB)/8, $libc_fsetxattr_trampoline<>(SB)
+
+TEXT libc_removexattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_removexattr(SB)
-TEXT ·libc_fremovexattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_removexattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_removexattr_trampoline_addr(SB)/8, $libc_removexattr_trampoline<>(SB)
+
+TEXT libc_fremovexattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fremovexattr(SB)
-TEXT ·libc_listxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fremovexattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fremovexattr_trampoline_addr(SB)/8, $libc_fremovexattr_trampoline<>(SB)
+
+TEXT libc_listxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_listxattr(SB)
-TEXT ·libc_flistxattr_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_listxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_listxattr_trampoline_addr(SB)/8, $libc_listxattr_trampoline<>(SB)
+
+TEXT libc_flistxattr_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_flistxattr(SB)
-TEXT ·libc_setattrlist_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_flistxattr_trampoline_addr(SB), RODATA, $8
+DATA	·libc_flistxattr_trampoline_addr(SB)/8, $libc_flistxattr_trampoline<>(SB)
+
+TEXT libc_setattrlist_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setattrlist(SB)
-TEXT ·libc_fcntl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setattrlist_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setattrlist_trampoline_addr(SB)/8, $libc_setattrlist_trampoline<>(SB)
+
+TEXT libc_fcntl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fcntl(SB)
-TEXT ·libc_kill_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fcntl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fcntl_trampoline_addr(SB)/8, $libc_fcntl_trampoline<>(SB)
+
+TEXT libc_kill_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kill(SB)
-TEXT ·libc_ioctl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kill_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kill_trampoline_addr(SB)/8, $libc_kill_trampoline<>(SB)
+
+TEXT libc_ioctl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_ioctl(SB)
-TEXT ·libc_sysctl_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_ioctl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ioctl_trampoline_addr(SB)/8, $libc_ioctl_trampoline<>(SB)
+
+TEXT libc_sysctl_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sysctl(SB)
-TEXT ·libc_sendfile_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sysctl_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sysctl_trampoline_addr(SB)/8, $libc_sysctl_trampoline<>(SB)
+
+TEXT libc_sendfile_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sendfile(SB)
-TEXT ·libc_access_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sendfile_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sendfile_trampoline_addr(SB)/8, $libc_sendfile_trampoline<>(SB)
+
+TEXT libc_access_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_access(SB)
-TEXT ·libc_adjtime_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_access_trampoline_addr(SB), RODATA, $8
+DATA	·libc_access_trampoline_addr(SB)/8, $libc_access_trampoline<>(SB)
+
+TEXT libc_adjtime_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_adjtime(SB)
-TEXT ·libc_chdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_adjtime_trampoline_addr(SB), RODATA, $8
+DATA	·libc_adjtime_trampoline_addr(SB)/8, $libc_adjtime_trampoline<>(SB)
+
+TEXT libc_chdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chdir(SB)
-TEXT ·libc_chflags_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chdir_trampoline_addr(SB)/8, $libc_chdir_trampoline<>(SB)
+
+TEXT libc_chflags_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chflags(SB)
-TEXT ·libc_chmod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chflags_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chflags_trampoline_addr(SB)/8, $libc_chflags_trampoline<>(SB)
+
+TEXT libc_chmod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chmod(SB)
-TEXT ·libc_chown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chmod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chmod_trampoline_addr(SB)/8, $libc_chmod_trampoline<>(SB)
+
+TEXT libc_chown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chown(SB)
-TEXT ·libc_chroot_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chown_trampoline_addr(SB)/8, $libc_chown_trampoline<>(SB)
+
+TEXT libc_chroot_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_chroot(SB)
-TEXT ·libc_clock_gettime_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_chroot_trampoline_addr(SB), RODATA, $8
+DATA	·libc_chroot_trampoline_addr(SB)/8, $libc_chroot_trampoline<>(SB)
+
+TEXT libc_clock_gettime_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clock_gettime(SB)
-TEXT ·libc_close_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clock_gettime_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clock_gettime_trampoline_addr(SB)/8, $libc_clock_gettime_trampoline<>(SB)
+
+TEXT libc_close_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_close(SB)
-TEXT ·libc_clonefile_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_close_trampoline_addr(SB), RODATA, $8
+DATA	·libc_close_trampoline_addr(SB)/8, $libc_close_trampoline<>(SB)
+
+TEXT libc_clonefile_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clonefile(SB)
-TEXT ·libc_clonefileat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clonefile_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clonefile_trampoline_addr(SB)/8, $libc_clonefile_trampoline<>(SB)
+
+TEXT libc_clonefileat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_clonefileat(SB)
-TEXT ·libc_dup_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_clonefileat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_clonefileat_trampoline_addr(SB)/8, $libc_clonefileat_trampoline<>(SB)
+
+TEXT libc_dup_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_dup(SB)
-TEXT ·libc_dup2_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_dup_trampoline_addr(SB), RODATA, $8
+DATA	·libc_dup_trampoline_addr(SB)/8, $libc_dup_trampoline<>(SB)
+
+TEXT libc_dup2_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_dup2(SB)
-TEXT ·libc_exchangedata_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_dup2_trampoline_addr(SB), RODATA, $8
+DATA	·libc_dup2_trampoline_addr(SB)/8, $libc_dup2_trampoline<>(SB)
+
+TEXT libc_exchangedata_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_exchangedata(SB)
-TEXT ·libc_exit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_exchangedata_trampoline_addr(SB), RODATA, $8
+DATA	·libc_exchangedata_trampoline_addr(SB)/8, $libc_exchangedata_trampoline<>(SB)
+
+TEXT libc_exit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_exit(SB)
-TEXT ·libc_faccessat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_exit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_exit_trampoline_addr(SB)/8, $libc_exit_trampoline<>(SB)
+
+TEXT libc_faccessat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_faccessat(SB)
-TEXT ·libc_fchdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_faccessat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_faccessat_trampoline_addr(SB)/8, $libc_faccessat_trampoline<>(SB)
+
+TEXT libc_fchdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchdir(SB)
-TEXT ·libc_fchflags_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchdir_trampoline_addr(SB)/8, $libc_fchdir_trampoline<>(SB)
+
+TEXT libc_fchflags_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchflags(SB)
-TEXT ·libc_fchmod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchflags_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchflags_trampoline_addr(SB)/8, $libc_fchflags_trampoline<>(SB)
+
+TEXT libc_fchmod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchmod(SB)
-TEXT ·libc_fchmodat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchmod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchmod_trampoline_addr(SB)/8, $libc_fchmod_trampoline<>(SB)
+
+TEXT libc_fchmodat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchmodat(SB)
-TEXT ·libc_fchown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchmodat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchmodat_trampoline_addr(SB)/8, $libc_fchmodat_trampoline<>(SB)
+
+TEXT libc_fchown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchown(SB)
-TEXT ·libc_fchownat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchown_trampoline_addr(SB)/8, $libc_fchown_trampoline<>(SB)
+
+TEXT libc_fchownat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fchownat(SB)
-TEXT ·libc_fclonefileat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fchownat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fchownat_trampoline_addr(SB)/8, $libc_fchownat_trampoline<>(SB)
+
+TEXT libc_fclonefileat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fclonefileat(SB)
-TEXT ·libc_flock_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fclonefileat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fclonefileat_trampoline_addr(SB)/8, $libc_fclonefileat_trampoline<>(SB)
+
+TEXT libc_flock_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_flock(SB)
-TEXT ·libc_fpathconf_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_flock_trampoline_addr(SB), RODATA, $8
+DATA	·libc_flock_trampoline_addr(SB)/8, $libc_flock_trampoline<>(SB)
+
+TEXT libc_fpathconf_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fpathconf(SB)
-TEXT ·libc_fsync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fpathconf_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fpathconf_trampoline_addr(SB)/8, $libc_fpathconf_trampoline<>(SB)
+
+TEXT libc_fsync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fsync(SB)
-TEXT ·libc_ftruncate_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fsync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fsync_trampoline_addr(SB)/8, $libc_fsync_trampoline<>(SB)
+
+TEXT libc_ftruncate_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_ftruncate(SB)
-TEXT ·libc_getcwd_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_ftruncate_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ftruncate_trampoline_addr(SB)/8, $libc_ftruncate_trampoline<>(SB)
+
+TEXT libc_getcwd_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getcwd(SB)
-TEXT ·libc_getdtablesize_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getcwd_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getcwd_trampoline_addr(SB)/8, $libc_getcwd_trampoline<>(SB)
+
+TEXT libc_getdtablesize_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getdtablesize(SB)
-TEXT ·libc_getegid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getdtablesize_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getdtablesize_trampoline_addr(SB)/8, $libc_getdtablesize_trampoline<>(SB)
+
+TEXT libc_getegid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getegid(SB)
-TEXT ·libc_geteuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getegid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getegid_trampoline_addr(SB)/8, $libc_getegid_trampoline<>(SB)
+
+TEXT libc_geteuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_geteuid(SB)
-TEXT ·libc_getgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_geteuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_geteuid_trampoline_addr(SB)/8, $libc_geteuid_trampoline<>(SB)
+
+TEXT libc_getgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getgid(SB)
-TEXT ·libc_getpgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getgid_trampoline_addr(SB)/8, $libc_getgid_trampoline<>(SB)
+
+TEXT libc_getpgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpgid(SB)
-TEXT ·libc_getpgrp_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpgid_trampoline_addr(SB)/8, $libc_getpgid_trampoline<>(SB)
+
+TEXT libc_getpgrp_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpgrp(SB)
-TEXT ·libc_getpid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpgrp_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpgrp_trampoline_addr(SB)/8, $libc_getpgrp_trampoline<>(SB)
+
+TEXT libc_getpid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpid(SB)
-TEXT ·libc_getppid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpid_trampoline_addr(SB)/8, $libc_getpid_trampoline<>(SB)
+
+TEXT libc_getppid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getppid(SB)
-TEXT ·libc_getpriority_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getppid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getppid_trampoline_addr(SB)/8, $libc_getppid_trampoline<>(SB)
+
+TEXT libc_getpriority_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getpriority(SB)
-TEXT ·libc_getrlimit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getpriority_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getpriority_trampoline_addr(SB)/8, $libc_getpriority_trampoline<>(SB)
+
+TEXT libc_getrlimit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getrlimit(SB)
-TEXT ·libc_getrusage_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getrlimit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getrlimit_trampoline_addr(SB)/8, $libc_getrlimit_trampoline<>(SB)
+
+TEXT libc_getrusage_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getrusage(SB)
-TEXT ·libc_getsid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getrusage_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getrusage_trampoline_addr(SB)/8, $libc_getrusage_trampoline<>(SB)
+
+TEXT libc_getsid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getsid(SB)
-TEXT ·libc_gettimeofday_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getsid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getsid_trampoline_addr(SB)/8, $libc_getsid_trampoline<>(SB)
+
+TEXT libc_gettimeofday_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_gettimeofday(SB)
-TEXT ·libc_getuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_gettimeofday_trampoline_addr(SB), RODATA, $8
+DATA	·libc_gettimeofday_trampoline_addr(SB)/8, $libc_gettimeofday_trampoline<>(SB)
+
+TEXT libc_getuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getuid(SB)
-TEXT ·libc_issetugid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getuid_trampoline_addr(SB)/8, $libc_getuid_trampoline<>(SB)
+
+TEXT libc_issetugid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_issetugid(SB)
-TEXT ·libc_kqueue_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_issetugid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_issetugid_trampoline_addr(SB)/8, $libc_issetugid_trampoline<>(SB)
+
+TEXT libc_kqueue_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_kqueue(SB)
-TEXT ·libc_lchown_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_kqueue_trampoline_addr(SB), RODATA, $8
+DATA	·libc_kqueue_trampoline_addr(SB)/8, $libc_kqueue_trampoline<>(SB)
+
+TEXT libc_lchown_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lchown(SB)
-TEXT ·libc_link_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lchown_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lchown_trampoline_addr(SB)/8, $libc_lchown_trampoline<>(SB)
+
+TEXT libc_link_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_link(SB)
-TEXT ·libc_linkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_link_trampoline_addr(SB), RODATA, $8
+DATA	·libc_link_trampoline_addr(SB)/8, $libc_link_trampoline<>(SB)
+
+TEXT libc_linkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_linkat(SB)
-TEXT ·libc_listen_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_linkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_linkat_trampoline_addr(SB)/8, $libc_linkat_trampoline<>(SB)
+
+TEXT libc_listen_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_listen(SB)
-TEXT ·libc_mkdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_listen_trampoline_addr(SB), RODATA, $8
+DATA	·libc_listen_trampoline_addr(SB)/8, $libc_listen_trampoline<>(SB)
+
+TEXT libc_mkdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkdir(SB)
-TEXT ·libc_mkdirat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkdir_trampoline_addr(SB)/8, $libc_mkdir_trampoline<>(SB)
+
+TEXT libc_mkdirat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkdirat(SB)
-TEXT ·libc_mkfifo_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkdirat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkdirat_trampoline_addr(SB)/8, $libc_mkdirat_trampoline<>(SB)
+
+TEXT libc_mkfifo_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mkfifo(SB)
-TEXT ·libc_mknod_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mkfifo_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mkfifo_trampoline_addr(SB)/8, $libc_mkfifo_trampoline<>(SB)
+
+TEXT libc_mknod_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mknod(SB)
-TEXT ·libc_open_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mknod_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mknod_trampoline_addr(SB)/8, $libc_mknod_trampoline<>(SB)
+
+TEXT libc_open_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_open(SB)
-TEXT ·libc_openat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_open_trampoline_addr(SB), RODATA, $8
+DATA	·libc_open_trampoline_addr(SB)/8, $libc_open_trampoline<>(SB)
+
+TEXT libc_openat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_openat(SB)
-TEXT ·libc_pathconf_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_openat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_openat_trampoline_addr(SB)/8, $libc_openat_trampoline<>(SB)
+
+TEXT libc_pathconf_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pathconf(SB)
-TEXT ·libc_pread_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pathconf_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pathconf_trampoline_addr(SB)/8, $libc_pathconf_trampoline<>(SB)
+
+TEXT libc_pread_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pread(SB)
-TEXT ·libc_pwrite_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pread_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pread_trampoline_addr(SB)/8, $libc_pread_trampoline<>(SB)
+
+TEXT libc_pwrite_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_pwrite(SB)
-TEXT ·libc_read_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_pwrite_trampoline_addr(SB), RODATA, $8
+DATA	·libc_pwrite_trampoline_addr(SB)/8, $libc_pwrite_trampoline<>(SB)
+
+TEXT libc_read_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_read(SB)
-TEXT ·libc_readlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_read_trampoline_addr(SB), RODATA, $8
+DATA	·libc_read_trampoline_addr(SB)/8, $libc_read_trampoline<>(SB)
+
+TEXT libc_readlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readlink(SB)
-TEXT ·libc_readlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_readlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readlink_trampoline_addr(SB)/8, $libc_readlink_trampoline<>(SB)
+
+TEXT libc_readlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_readlinkat(SB)
-TEXT ·libc_rename_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_readlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_readlinkat_trampoline_addr(SB)/8, $libc_readlinkat_trampoline<>(SB)
+
+TEXT libc_rename_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_rename(SB)
-TEXT ·libc_renameat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_rename_trampoline_addr(SB), RODATA, $8
+DATA	·libc_rename_trampoline_addr(SB)/8, $libc_rename_trampoline<>(SB)
+
+TEXT libc_renameat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_renameat(SB)
-TEXT ·libc_revoke_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_renameat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_renameat_trampoline_addr(SB)/8, $libc_renameat_trampoline<>(SB)
+
+TEXT libc_revoke_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_revoke(SB)
-TEXT ·libc_rmdir_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_revoke_trampoline_addr(SB), RODATA, $8
+DATA	·libc_revoke_trampoline_addr(SB)/8, $libc_revoke_trampoline<>(SB)
+
+TEXT libc_rmdir_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_rmdir(SB)
-TEXT ·libc_lseek_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_rmdir_trampoline_addr(SB), RODATA, $8
+DATA	·libc_rmdir_trampoline_addr(SB)/8, $libc_rmdir_trampoline<>(SB)
+
+TEXT libc_lseek_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lseek(SB)
-TEXT ·libc_select_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lseek_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lseek_trampoline_addr(SB)/8, $libc_lseek_trampoline<>(SB)
+
+TEXT libc_select_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_select(SB)
-TEXT ·libc_setegid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_select_trampoline_addr(SB), RODATA, $8
+DATA	·libc_select_trampoline_addr(SB)/8, $libc_select_trampoline<>(SB)
+
+TEXT libc_setegid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setegid(SB)
-TEXT ·libc_seteuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setegid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setegid_trampoline_addr(SB)/8, $libc_setegid_trampoline<>(SB)
+
+TEXT libc_seteuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_seteuid(SB)
-TEXT ·libc_setgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_seteuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_seteuid_trampoline_addr(SB)/8, $libc_seteuid_trampoline<>(SB)
+
+TEXT libc_setgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setgid(SB)
-TEXT ·libc_setlogin_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setgid_trampoline_addr(SB)/8, $libc_setgid_trampoline<>(SB)
+
+TEXT libc_setlogin_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setlogin(SB)
-TEXT ·libc_setpgid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setlogin_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setlogin_trampoline_addr(SB)/8, $libc_setlogin_trampoline<>(SB)
+
+TEXT libc_setpgid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setpgid(SB)
-TEXT ·libc_setpriority_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setpgid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setpgid_trampoline_addr(SB)/8, $libc_setpgid_trampoline<>(SB)
+
+TEXT libc_setpriority_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setpriority(SB)
-TEXT ·libc_setprivexec_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setpriority_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setpriority_trampoline_addr(SB)/8, $libc_setpriority_trampoline<>(SB)
+
+TEXT libc_setprivexec_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setprivexec(SB)
-TEXT ·libc_setregid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setprivexec_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setprivexec_trampoline_addr(SB)/8, $libc_setprivexec_trampoline<>(SB)
+
+TEXT libc_setregid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setregid(SB)
-TEXT ·libc_setreuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setregid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setregid_trampoline_addr(SB)/8, $libc_setregid_trampoline<>(SB)
+
+TEXT libc_setreuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setreuid(SB)
-TEXT ·libc_setrlimit_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setreuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setreuid_trampoline_addr(SB)/8, $libc_setreuid_trampoline<>(SB)
+
+TEXT libc_setrlimit_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setrlimit(SB)
-TEXT ·libc_setsid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setrlimit_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setrlimit_trampoline_addr(SB)/8, $libc_setrlimit_trampoline<>(SB)
+
+TEXT libc_setsid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setsid(SB)
-TEXT ·libc_settimeofday_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setsid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setsid_trampoline_addr(SB)/8, $libc_setsid_trampoline<>(SB)
+
+TEXT libc_settimeofday_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_settimeofday(SB)
-TEXT ·libc_setuid_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_settimeofday_trampoline_addr(SB), RODATA, $8
+DATA	·libc_settimeofday_trampoline_addr(SB)/8, $libc_settimeofday_trampoline<>(SB)
+
+TEXT libc_setuid_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_setuid(SB)
-TEXT ·libc_symlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_setuid_trampoline_addr(SB), RODATA, $8
+DATA	·libc_setuid_trampoline_addr(SB)/8, $libc_setuid_trampoline<>(SB)
+
+TEXT libc_symlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_symlink(SB)
-TEXT ·libc_symlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_symlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_symlink_trampoline_addr(SB)/8, $libc_symlink_trampoline<>(SB)
+
+TEXT libc_symlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_symlinkat(SB)
-TEXT ·libc_sync_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_symlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_symlinkat_trampoline_addr(SB)/8, $libc_symlinkat_trampoline<>(SB)
+
+TEXT libc_sync_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_sync(SB)
-TEXT ·libc_truncate_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_sync_trampoline_addr(SB), RODATA, $8
+DATA	·libc_sync_trampoline_addr(SB)/8, $libc_sync_trampoline<>(SB)
+
+TEXT libc_truncate_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_truncate(SB)
-TEXT ·libc_umask_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_truncate_trampoline_addr(SB), RODATA, $8
+DATA	·libc_truncate_trampoline_addr(SB)/8, $libc_truncate_trampoline<>(SB)
+
+TEXT libc_umask_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_umask(SB)
-TEXT ·libc_undelete_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_umask_trampoline_addr(SB), RODATA, $8
+DATA	·libc_umask_trampoline_addr(SB)/8, $libc_umask_trampoline<>(SB)
+
+TEXT libc_undelete_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_undelete(SB)
-TEXT ·libc_unlink_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_undelete_trampoline_addr(SB), RODATA, $8
+DATA	·libc_undelete_trampoline_addr(SB)/8, $libc_undelete_trampoline<>(SB)
+
+TEXT libc_unlink_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unlink(SB)
-TEXT ·libc_unlinkat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unlink_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unlink_trampoline_addr(SB)/8, $libc_unlink_trampoline<>(SB)
+
+TEXT libc_unlinkat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unlinkat(SB)
-TEXT ·libc_unmount_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unlinkat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unlinkat_trampoline_addr(SB)/8, $libc_unlinkat_trampoline<>(SB)
+
+TEXT libc_unmount_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_unmount(SB)
-TEXT ·libc_write_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_unmount_trampoline_addr(SB), RODATA, $8
+DATA	·libc_unmount_trampoline_addr(SB)/8, $libc_unmount_trampoline<>(SB)
+
+TEXT libc_write_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_write(SB)
-TEXT ·libc_mmap_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_write_trampoline_addr(SB), RODATA, $8
+DATA	·libc_write_trampoline_addr(SB)/8, $libc_write_trampoline<>(SB)
+
+TEXT libc_mmap_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_mmap(SB)
-TEXT ·libc_munmap_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_mmap_trampoline_addr(SB), RODATA, $8
+DATA	·libc_mmap_trampoline_addr(SB)/8, $libc_mmap_trampoline<>(SB)
+
+TEXT libc_munmap_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_munmap(SB)
-TEXT ·libc_fstat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_munmap_trampoline_addr(SB), RODATA, $8
+DATA	·libc_munmap_trampoline_addr(SB)/8, $libc_munmap_trampoline<>(SB)
+
+TEXT libc_fstat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstat(SB)
-TEXT ·libc_fstatat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstat_trampoline_addr(SB)/8, $libc_fstat_trampoline<>(SB)
+
+TEXT libc_fstatat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstatat(SB)
-TEXT ·libc_fstatfs_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstatat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstatat_trampoline_addr(SB)/8, $libc_fstatat_trampoline<>(SB)
+
+TEXT libc_fstatfs_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_fstatfs(SB)
-TEXT ·libc_getfsstat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_fstatfs_trampoline_addr(SB), RODATA, $8
+DATA	·libc_fstatfs_trampoline_addr(SB)/8, $libc_fstatfs_trampoline<>(SB)
+
+TEXT libc_getfsstat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_getfsstat(SB)
-TEXT ·libc_lstat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_getfsstat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_getfsstat_trampoline_addr(SB)/8, $libc_getfsstat_trampoline<>(SB)
+
+TEXT libc_lstat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_lstat(SB)
-TEXT ·libc_stat_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_lstat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_lstat_trampoline_addr(SB)/8, $libc_lstat_trampoline<>(SB)
+
+TEXT libc_ptrace_trampoline<>(SB),NOSPLIT,$0-0
+	JMP	libc_ptrace(SB)
+
+GLOBL	·libc_ptrace_trampoline_addr(SB), RODATA, $8
+DATA	·libc_ptrace_trampoline_addr(SB)/8, $libc_ptrace_trampoline<>(SB)
+
+TEXT libc_stat_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_stat(SB)
-TEXT ·libc_statfs_trampoline(SB),NOSPLIT,$0-0
+
+GLOBL	·libc_stat_trampoline_addr(SB), RODATA, $8
+DATA	·libc_stat_trampoline_addr(SB)/8, $libc_stat_trampoline<>(SB)
+
+TEXT libc_statfs_trampoline<>(SB),NOSPLIT,$0-0
 	JMP	libc_statfs(SB)
+
+GLOBL	·libc_statfs_trampoline_addr(SB), RODATA, $8
+DATA	·libc_statfs_trampoline_addr(SB)/8, $libc_statfs_trampoline<>(SB)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_dragonfly_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_dragonfly_amd64.go
index fe1fdd7..1b6eedf 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_dragonfly_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_dragonfly_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -dragonfly -tags dragonfly,amd64 syscall_bsd.go syscall_dragonfly.go syscall_dragonfly_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build dragonfly && amd64
 // +build dragonfly,amd64
 
 package unix
@@ -214,22 +215,6 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) {
-	var _p0 unsafe.Pointer
-	if len(mib) > 0 {
-		_p0 = unsafe.Pointer(&mib[0])
-	} else {
-		_p0 = unsafe.Pointer(&_zero)
-	}
-	_, _, e1 := Syscall6(SYS___SYSCTL, uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
-	if e1 != 0 {
-		err = errnoErr(e1)
-	}
-	return
-}
-
-// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
-
 func utimes(path string, timeval *[2]Timeval) (err error) {
 	var _p0 *byte
 	_p0, err = BytePtrFromString(path)
@@ -378,6 +363,18 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (r int, w int, err error) {
+	r0, r1, e1 := RawSyscall(SYS_PIPE2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0)
+	r = int(r0)
+	w = int(r1)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func extpread(fd int, p []byte, flags int, offset int64) (n int, err error) {
 	var _p0 unsafe.Pointer
 	if len(p) > 0 {
@@ -439,6 +436,22 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func sysctl(mib []_C_int, old *byte, oldlen *uintptr, new *byte, newlen uintptr) (err error) {
+	var _p0 unsafe.Pointer
+	if len(mib) > 0 {
+		_p0 = unsafe.Pointer(&mib[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := Syscall6(SYS___SYSCTL, uintptr(_p0), uintptr(len(mib)), uintptr(unsafe.Pointer(old)), uintptr(unsafe.Pointer(oldlen)), uintptr(unsafe.Pointer(new)), uintptr(newlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Access(path string, mode uint32) (err error) {
 	var _p0 *byte
 	_p0, err = BytePtrFromString(path)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_386.go b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_386.go
index 600f1d2..3e9bddb 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_386.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -tags freebsd,386 syscall_bsd.go syscall_freebsd.go syscall_freebsd_386.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build freebsd && 386
 // +build freebsd,386
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_amd64.go
index 064934b..c72a462 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags freebsd,amd64 syscall_bsd.go syscall_freebsd.go syscall_freebsd_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build freebsd && amd64
 // +build freebsd,amd64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm.go b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm.go
index 31d2c46..530d5df 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -arm -tags freebsd,arm syscall_bsd.go syscall_freebsd.go syscall_freebsd_arm.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build freebsd && arm
 // +build freebsd,arm
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm64.go b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm64.go
index 4adaaa5..71e7df9 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_freebsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags freebsd,arm64 syscall_bsd.go syscall_freebsd.go syscall_freebsd_arm64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build freebsd && arm64
 // +build freebsd,arm64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_illumos_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_illumos_amd64.go
index d3af083..af5cb06 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_illumos_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_illumos_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall_solaris.go -illumos -tags illumos,amd64 syscall_illumos.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build illumos && amd64
 // +build illumos,amd64
 
 package unix
@@ -14,14 +15,16 @@
 //go:cgo_import_dynamic libc_writev writev "libc.so"
 //go:cgo_import_dynamic libc_pwritev pwritev "libc.so"
 //go:cgo_import_dynamic libc_accept4 accept4 "libsocket.so"
-//go:cgo_import_dynamic libc_pipe2 pipe2 "libc.so"
+//go:cgo_import_dynamic libc_putmsg putmsg "libc.so"
+//go:cgo_import_dynamic libc_getmsg getmsg "libc.so"
 
 //go:linkname procreadv libc_readv
 //go:linkname procpreadv libc_preadv
 //go:linkname procwritev libc_writev
 //go:linkname procpwritev libc_pwritev
 //go:linkname procaccept4 libc_accept4
-//go:linkname procpipe2 libc_pipe2
+//go:linkname procputmsg libc_putmsg
+//go:linkname procgetmsg libc_getmsg
 
 var (
 	procreadv,
@@ -29,7 +32,8 @@
 	procwritev,
 	procpwritev,
 	procaccept4,
-	procpipe2 syscallFunc
+	procputmsg,
+	procgetmsg syscallFunc
 )
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
@@ -105,8 +109,18 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func pipe2(p *[2]_C_int, flags int) (err error) {
-	_, _, e1 := rawSysvicall6(uintptr(unsafe.Pointer(&procpipe2)), 2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0, 0, 0, 0)
+func putmsg(fd int, clptr *strbuf, dataptr *strbuf, flags int) (err error) {
+	_, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procputmsg)), 4, uintptr(fd), uintptr(unsafe.Pointer(clptr)), uintptr(unsafe.Pointer(dataptr)), uintptr(flags), 0, 0)
+	if e1 != 0 {
+		err = e1
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getmsg(fd int, clptr *strbuf, dataptr *strbuf, flags *int) (err error) {
+	_, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procgetmsg)), 4, uintptr(fd), uintptr(unsafe.Pointer(clptr)), uintptr(unsafe.Pointer(dataptr)), uintptr(unsafe.Pointer(flags)), 0, 0)
 	if e1 != 0 {
 		err = e1
 	}
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux.go b/vendor/golang.org/x/sys/unix/zsyscall_linux.go
index 2fbbbe5..7305cc9 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux.go
@@ -1,5 +1,6 @@
 // Code generated by mkmerge.go; DO NOT EDIT.
 
+//go:build linux
 // +build linux
 
 package unix
@@ -531,6 +532,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func CloseRange(first uint, last uint, flags uint) (err error) {
+	_, _, e1 := Syscall(SYS_CLOSE_RANGE, uintptr(first), uintptr(last), uintptr(flags))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func CopyFileRange(rfd int, roff *int64, wfd int, woff *int64, len int, flags int) (n int, err error) {
 	r0, _, e1 := Syscall6(SYS_COPY_FILE_RANGE, uintptr(rfd), uintptr(unsafe.Pointer(roff)), uintptr(wfd), uintptr(unsafe.Pointer(woff)), uintptr(len), uintptr(flags))
 	n = int(r0)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_386.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_386.go
index 19ebd3f..e37096e 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_386.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_386.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -tags linux,386 syscall_linux.go syscall_linux_386.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && 386
 // +build linux,386
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_amd64.go
index 5c56218..9919d84 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,amd64 syscall_linux.go syscall_linux_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && amd64
 // +build linux,amd64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_arm.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_arm.go
index dc69d99..076754d 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_arm.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -arm -tags linux,arm syscall_linux.go syscall_linux_arm.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && arm
 // +build linux,arm
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_arm64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_arm64.go
index 1b897de..e893f98 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_arm64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,arm64 syscall_linux.go syscall_linux_arm64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && arm64
 // +build linux,arm64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips.go
index 4918684..4703cf3 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -b32 -arm -tags linux,mips syscall_linux.go syscall_linux_mipsx.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && mips
 // +build linux,mips
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64.go
index 9171d3b..a134f9a 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,mips64 syscall_linux.go syscall_linux_mips64x.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && mips64
 // +build linux,mips64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64le.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64le.go
index 82286f0..b1fff2d 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64le.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_mips64le.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,mips64le syscall_linux.go syscall_linux_mips64x.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && mips64le
 // +build linux,mips64le
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_mipsle.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_mipsle.go
index 1592062..d13d6da 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_mipsle.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_mipsle.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -arm -tags linux,mipsle syscall_linux.go syscall_linux_mipsx.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && mipsle
 // +build linux,mipsle
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc.go
new file mode 100644
index 0000000..927cf1a
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc.go
@@ -0,0 +1,762 @@
+// go run mksyscall.go -b32 -tags linux,ppc syscall_linux.go syscall_linux_ppc.go
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build linux && ppc
+// +build linux,ppc
+
+package unix
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+var _ syscall.Errno
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func fanotifyMark(fd int, flags uint, mask uint64, dirFd int, pathname *byte) (err error) {
+	_, _, e1 := Syscall6(SYS_FANOTIFY_MARK, uintptr(fd), uintptr(flags), uintptr(mask>>32), uintptr(mask), uintptr(dirFd), uintptr(unsafe.Pointer(pathname)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fallocate(fd int, mode uint32, off int64, len int64) (err error) {
+	_, _, e1 := Syscall6(SYS_FALLOCATE, uintptr(fd), uintptr(mode), uintptr(off>>32), uintptr(off), uintptr(len>>32), uintptr(len))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Tee(rfd int, wfd int, len int, flags int) (n int64, err error) {
+	r0, r1, e1 := Syscall6(SYS_TEE, uintptr(rfd), uintptr(wfd), uintptr(len), uintptr(flags), 0, 0)
+	n = int64(int64(r0)<<32 | int64(r1))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func dup2(oldfd int, newfd int) (err error) {
+	_, _, e1 := Syscall(SYS_DUP2, uintptr(oldfd), uintptr(newfd), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func EpollCreate(size int) (fd int, err error) {
+	r0, _, e1 := RawSyscall(SYS_EPOLL_CREATE, uintptr(size), 0, 0)
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func EpollWait(epfd int, events []EpollEvent, msec int) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(events) > 0 {
+		_p0 = unsafe.Pointer(&events[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := Syscall6(SYS_EPOLL_WAIT, uintptr(epfd), uintptr(_p0), uintptr(len(events)), uintptr(msec), 0, 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fchown(fd int, uid int, gid int) (err error) {
+	_, _, e1 := Syscall(SYS_FCHOWN, uintptr(fd), uintptr(uid), uintptr(gid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fstat(fd int, stat *Stat_t) (err error) {
+	_, _, e1 := Syscall(SYS_FSTAT64, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fstatat(dirfd int, path string, stat *Stat_t, flags int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall6(SYS_FSTATAT64, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), uintptr(flags), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Ftruncate(fd int, length int64) (err error) {
+	_, _, e1 := Syscall(SYS_FTRUNCATE64, uintptr(fd), uintptr(length>>32), uintptr(length))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getegid() (egid int) {
+	r0, _ := RawSyscallNoError(SYS_GETEGID, 0, 0, 0)
+	egid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Geteuid() (euid int) {
+	r0, _ := RawSyscallNoError(SYS_GETEUID, 0, 0, 0)
+	euid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getgid() (gid int) {
+	r0, _ := RawSyscallNoError(SYS_GETGID, 0, 0, 0)
+	gid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getuid() (uid int) {
+	r0, _ := RawSyscallNoError(SYS_GETUID, 0, 0, 0)
+	uid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func InotifyInit() (fd int, err error) {
+	r0, _, e1 := RawSyscall(SYS_INOTIFY_INIT, 0, 0, 0)
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Ioperm(from int, num int, on int) (err error) {
+	_, _, e1 := Syscall(SYS_IOPERM, uintptr(from), uintptr(num), uintptr(on))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Iopl(level int) (err error) {
+	_, _, e1 := Syscall(SYS_IOPL, uintptr(level), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Lchown(path string, uid int, gid int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_LCHOWN, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Listen(s int, n int) (err error) {
+	_, _, e1 := Syscall(SYS_LISTEN, uintptr(s), uintptr(n), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Lstat(path string, stat *Stat_t) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_LSTAT64, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Pause() (err error) {
+	_, _, e1 := Syscall(SYS_PAUSE, 0, 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Pread(fd int, p []byte, offset int64) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := Syscall6(SYS_PREAD64, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset>>32), uintptr(offset), 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Pwrite(fd int, p []byte, offset int64) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := Syscall6(SYS_PWRITE64, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset>>32), uintptr(offset), 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Renameat(olddirfd int, oldpath string, newdirfd int, newpath string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(oldpath)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = BytePtrFromString(newpath)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall6(SYS_RENAMEAT, uintptr(olddirfd), uintptr(unsafe.Pointer(_p0)), uintptr(newdirfd), uintptr(unsafe.Pointer(_p1)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Select(nfd int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (n int, err error) {
+	r0, _, e1 := Syscall6(SYS__NEWSELECT, uintptr(nfd), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func sendfile(outfd int, infd int, offset *int64, count int) (written int, err error) {
+	r0, _, e1 := Syscall6(SYS_SENDFILE64, uintptr(outfd), uintptr(infd), uintptr(unsafe.Pointer(offset)), uintptr(count), 0, 0)
+	written = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setfsgid(gid int) (prev int, err error) {
+	r0, _, e1 := Syscall(SYS_SETFSGID, uintptr(gid), 0, 0)
+	prev = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setfsuid(uid int) (prev int, err error) {
+	r0, _, e1 := Syscall(SYS_SETFSUID, uintptr(uid), 0, 0)
+	prev = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setregid(rgid int, egid int) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETREGID, uintptr(rgid), uintptr(egid), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setresgid(rgid int, egid int, sgid int) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETRESGID, uintptr(rgid), uintptr(egid), uintptr(sgid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setresuid(ruid int, euid int, suid int) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETRESUID, uintptr(ruid), uintptr(euid), uintptr(suid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setreuid(ruid int, euid int) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETREUID, uintptr(ruid), uintptr(euid), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Shutdown(fd int, how int) (err error) {
+	_, _, e1 := Syscall(SYS_SHUTDOWN, uintptr(fd), uintptr(how), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Splice(rfd int, roff *int64, wfd int, woff *int64, len int, flags int) (n int, err error) {
+	r0, _, e1 := Syscall6(SYS_SPLICE, uintptr(rfd), uintptr(unsafe.Pointer(roff)), uintptr(wfd), uintptr(unsafe.Pointer(woff)), uintptr(len), uintptr(flags))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Stat(path string, stat *Stat_t) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_STAT64, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Truncate(path string, length int64) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_TRUNCATE64, uintptr(unsafe.Pointer(_p0)), uintptr(length>>32), uintptr(length))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Ustat(dev int, ubuf *Ustat_t) (err error) {
+	_, _, e1 := Syscall(SYS_USTAT, uintptr(dev), uintptr(unsafe.Pointer(ubuf)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
+	r0, _, e1 := Syscall(SYS_ACCEPT, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func accept4(s int, rsa *RawSockaddrAny, addrlen *_Socklen, flags int) (fd int, err error) {
+	r0, _, e1 := Syscall6(SYS_ACCEPT4, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)), uintptr(flags), 0, 0)
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
+	_, _, e1 := Syscall(SYS_BIND, uintptr(s), uintptr(addr), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
+	_, _, e1 := Syscall(SYS_CONNECT, uintptr(s), uintptr(addr), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getgroups(n int, list *_Gid_t) (nn int, err error) {
+	r0, _, e1 := RawSyscall(SYS_GETGROUPS, uintptr(n), uintptr(unsafe.Pointer(list)), 0)
+	nn = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setgroups(n int, list *_Gid_t) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETGROUPS, uintptr(n), uintptr(unsafe.Pointer(list)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
+	_, _, e1 := Syscall6(SYS_GETSOCKOPT, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
+	_, _, e1 := Syscall6(SYS_SETSOCKOPT, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func socket(domain int, typ int, proto int) (fd int, err error) {
+	r0, _, e1 := RawSyscall(SYS_SOCKET, uintptr(domain), uintptr(typ), uintptr(proto))
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
+	_, _, e1 := RawSyscall6(SYS_SOCKETPAIR, uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
+	_, _, e1 := RawSyscall(SYS_GETPEERNAME, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
+	_, _, e1 := RawSyscall(SYS_GETSOCKNAME, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := Syscall6(SYS_RECVFROM, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error) {
+	var _p0 unsafe.Pointer
+	if len(buf) > 0 {
+		_p0 = unsafe.Pointer(&buf[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := Syscall6(SYS_SENDTO, uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
+	r0, _, e1 := Syscall(SYS_RECVMSG, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
+	r0, _, e1 := Syscall(SYS_SENDMSG, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func futimesat(dirfd int, path string, times *[2]Timeval) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_FUTIMESAT, uintptr(dirfd), uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(times)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Gettimeofday(tv *Timeval) (err error) {
+	_, _, e1 := RawSyscall(SYS_GETTIMEOFDAY, uintptr(unsafe.Pointer(tv)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Time(t *Time_t) (tt Time_t, err error) {
+	r0, _, e1 := RawSyscall(SYS_TIME, uintptr(unsafe.Pointer(t)), 0, 0)
+	tt = Time_t(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Utime(path string, buf *Utimbuf) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_UTIME, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(buf)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func utimes(path string, times *[2]Timeval) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall(SYS_UTIMES, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(times)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func mmap2(addr uintptr, length uintptr, prot int, flags int, fd int, pageOffset uintptr) (xaddr uintptr, err error) {
+	r0, _, e1 := Syscall6(SYS_MMAP2, uintptr(addr), uintptr(length), uintptr(prot), uintptr(flags), uintptr(fd), uintptr(pageOffset))
+	xaddr = uintptr(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getrlimit(resource int, rlim *rlimit32) (err error) {
+	_, _, e1 := RawSyscall(SYS_UGETRLIMIT, uintptr(resource), uintptr(unsafe.Pointer(rlim)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setrlimit(resource int, rlim *rlimit32) (err error) {
+	_, _, e1 := RawSyscall(SYS_SETRLIMIT, uintptr(resource), uintptr(unsafe.Pointer(rlim)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func pipe(p *[2]_C_int) (err error) {
+	_, _, e1 := RawSyscall(SYS_PIPE, uintptr(unsafe.Pointer(p)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func poll(fds *PollFd, nfds int, timeout int) (n int, err error) {
+	r0, _, e1 := Syscall(SYS_POLL, uintptr(unsafe.Pointer(fds)), uintptr(nfds), uintptr(timeout))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func syncFileRange2(fd int, flags int, off int64, n int64) (err error) {
+	_, _, e1 := Syscall6(SYS_SYNC_FILE_RANGE2, uintptr(fd), uintptr(flags), uintptr(off>>32), uintptr(off), uintptr(n>>32), uintptr(n))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func kexecFileLoad(kernelFd int, initrdFd int, cmdlineLen int, cmdline string, flags int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(cmdline)
+	if err != nil {
+		return
+	}
+	_, _, e1 := Syscall6(SYS_KEXEC_FILE_LOAD, uintptr(kernelFd), uintptr(initrdFd), uintptr(cmdlineLen), uintptr(unsafe.Pointer(_p0)), uintptr(flags), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64.go
index 73a42e2..da8ec03 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,ppc64 syscall_linux.go syscall_linux_ppc64x.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && ppc64
 // +build linux,ppc64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64le.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64le.go
index 6b85595..083f493 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64le.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_ppc64le.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,ppc64le syscall_linux.go syscall_linux_ppc64x.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && ppc64le
 // +build linux,ppc64le
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_riscv64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_riscv64.go
index b761334..63b393b 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_riscv64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_riscv64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,riscv64 syscall_linux.go syscall_linux_riscv64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && riscv64
 // +build linux,riscv64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_s390x.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_s390x.go
index d7032ab..bb34740 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_s390x.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_s390x.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,s390x syscall_linux.go syscall_linux_s390x.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && s390x
 // +build linux,s390x
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_linux_sparc64.go b/vendor/golang.org/x/sys/unix/zsyscall_linux_sparc64.go
index bcbbdd9..8edc517 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_linux_sparc64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_linux_sparc64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -tags linux,sparc64 syscall_linux.go syscall_linux_sparc64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build linux && sparc64
 // +build linux,sparc64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_386.go b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_386.go
index 3bbd9e3..4726ab3 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_386.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -netbsd -tags netbsd,386 syscall_bsd.go syscall_netbsd.go syscall_netbsd_386.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build netbsd && 386
 // +build netbsd,386
 
 package unix
@@ -362,6 +363,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (err error) {
+	_, _, e1 := RawSyscall(SYS_PIPE2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Getdents(fd int, buf []byte) (n int, err error) {
 	var _p0 unsafe.Pointer
 	if len(buf) > 0 {
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_amd64.go
index d8cf501..fe71456 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -netbsd -tags netbsd,amd64 syscall_bsd.go syscall_netbsd.go syscall_netbsd_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build netbsd && amd64
 // +build netbsd,amd64
 
 package unix
@@ -362,6 +363,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (err error) {
+	_, _, e1 := RawSyscall(SYS_PIPE2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Getdents(fd int, buf []byte) (n int, err error) {
 	var _p0 unsafe.Pointer
 	if len(buf) > 0 {
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm.go b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm.go
index 1153fe6..0b5b2f0 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -netbsd -arm -tags netbsd,arm syscall_bsd.go syscall_netbsd.go syscall_netbsd_arm.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build netbsd && arm
 // +build netbsd,arm
 
 package unix
@@ -362,6 +363,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (err error) {
+	_, _, e1 := RawSyscall(SYS_PIPE2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Getdents(fd int, buf []byte) (n int, err error) {
 	var _p0 unsafe.Pointer
 	if len(buf) > 0 {
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm64.go b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm64.go
index 24b4ebb..bfca286 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_netbsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -netbsd -tags netbsd,arm64 syscall_bsd.go syscall_netbsd.go syscall_netbsd_arm64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build netbsd && arm64
 // +build netbsd,arm64
 
 package unix
@@ -362,6 +363,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (err error) {
+	_, _, e1 := RawSyscall(SYS_PIPE2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Getdents(fd int, buf []byte) (n int, err error) {
 	var _p0 unsafe.Pointer
 	if len(buf) > 0 {
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_386.go b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_386.go
index b44b31a..8f80f4a 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_386.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -openbsd -tags openbsd,386 syscall_bsd.go syscall_openbsd.go syscall_openbsd_386.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build openbsd && 386
 // +build openbsd,386
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_amd64.go
index 67f93ee..3a47aca 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -openbsd -tags openbsd,amd64 syscall_bsd.go syscall_openbsd.go syscall_openbsd_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build openbsd && amd64
 // +build openbsd,amd64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm.go b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm.go
index d7c878b..883a9b4 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -l32 -openbsd -arm -tags openbsd,arm syscall_bsd.go syscall_openbsd.go syscall_openbsd_arm.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build openbsd && arm
 // +build openbsd,arm
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm64.go
index 8facd69..aac7fdc 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -openbsd -tags openbsd,arm64 syscall_bsd.go syscall_openbsd.go syscall_openbsd_arm64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build openbsd && arm64
 // +build openbsd,arm64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_mips64.go
index ec6bd5b..8776187 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_openbsd_mips64.go
@@ -1,6 +1,7 @@
 // go run mksyscall.go -openbsd -tags openbsd,mips64 syscall_bsd.go syscall_openbsd.go syscall_openbsd_mips64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build openbsd && mips64
 // +build openbsd,mips64
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_solaris_amd64.go b/vendor/golang.org/x/sys/unix/zsyscall_solaris_amd64.go
index a96165d..4e18d5c 100644
--- a/vendor/golang.org/x/sys/unix/zsyscall_solaris_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsyscall_solaris_amd64.go
@@ -1,6 +1,7 @@
 // go run mksyscall_solaris.go -tags solaris,amd64 syscall_solaris.go syscall_solaris_amd64.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build solaris && amd64
 // +build solaris,amd64
 
 package unix
@@ -11,6 +12,7 @@
 )
 
 //go:cgo_import_dynamic libc_pipe pipe "libc.so"
+//go:cgo_import_dynamic libc_pipe2 pipe2 "libc.so"
 //go:cgo_import_dynamic libc_getsockname getsockname "libsocket.so"
 //go:cgo_import_dynamic libc_getcwd getcwd "libc.so"
 //go:cgo_import_dynamic libc_getgroups getgroups "libc.so"
@@ -114,6 +116,7 @@
 //go:cgo_import_dynamic libc_statvfs statvfs "libc.so"
 //go:cgo_import_dynamic libc_symlink symlink "libc.so"
 //go:cgo_import_dynamic libc_sync sync "libc.so"
+//go:cgo_import_dynamic libc_sysconf sysconf "libc.so"
 //go:cgo_import_dynamic libc_times times "libc.so"
 //go:cgo_import_dynamic libc_truncate truncate "libc.so"
 //go:cgo_import_dynamic libc_fsync fsync "libc.so"
@@ -140,6 +143,7 @@
 //go:cgo_import_dynamic libc_recvfrom recvfrom "libsocket.so"
 
 //go:linkname procpipe libc_pipe
+//go:linkname procpipe2 libc_pipe2
 //go:linkname procgetsockname libc_getsockname
 //go:linkname procGetcwd libc_getcwd
 //go:linkname procgetgroups libc_getgroups
@@ -243,6 +247,7 @@
 //go:linkname procStatvfs libc_statvfs
 //go:linkname procSymlink libc_symlink
 //go:linkname procSync libc_sync
+//go:linkname procSysconf libc_sysconf
 //go:linkname procTimes libc_times
 //go:linkname procTruncate libc_truncate
 //go:linkname procFsync libc_fsync
@@ -270,6 +275,7 @@
 
 var (
 	procpipe,
+	procpipe2,
 	procgetsockname,
 	procGetcwd,
 	procgetgroups,
@@ -373,6 +379,7 @@
 	procStatvfs,
 	procSymlink,
 	procSync,
+	procSysconf,
 	procTimes,
 	procTruncate,
 	procFsync,
@@ -412,6 +419,16 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func pipe2(p *[2]_C_int, flags int) (err error) {
+	_, _, e1 := rawSysvicall6(uintptr(unsafe.Pointer(&procpipe2)), 2, uintptr(unsafe.Pointer(p)), uintptr(flags), 0, 0, 0, 0)
+	if e1 != 0 {
+		err = e1
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
 	_, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procgetsockname)), 3, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)), 0, 0, 0)
 	if e1 != 0 {
@@ -602,8 +619,9 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
-func ioctl(fd int, req uint, arg uintptr) (err error) {
-	_, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procioctl)), 3, uintptr(fd), uintptr(req), uintptr(arg), 0, 0, 0)
+func ioctlRet(fd int, req uint, arg uintptr) (ret int, err error) {
+	r0, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procioctl)), 3, uintptr(fd), uintptr(req), uintptr(arg), 0, 0, 0)
+	ret = int(r0)
 	if e1 != 0 {
 		err = e1
 	}
@@ -1674,6 +1692,17 @@
 
 // THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
 
+func Sysconf(which int) (n int64, err error) {
+	r0, _, e1 := sysvicall6(uintptr(unsafe.Pointer(&procSysconf)), 1, uintptr(which), 0, 0, 0, 0, 0)
+	n = int64(r0)
+	if e1 != 0 {
+		err = e1
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
 func Times(tms *Tms) (ticks uintptr, err error) {
 	r0, _, e1 := rawSysvicall6(uintptr(unsafe.Pointer(&procTimes)), 1, uintptr(unsafe.Pointer(tms)), 0, 0, 0, 0, 0)
 	ticks = uintptr(r0)
diff --git a/vendor/golang.org/x/sys/unix/zsyscall_zos_s390x.go b/vendor/golang.org/x/sys/unix/zsyscall_zos_s390x.go
new file mode 100644
index 0000000..f207945
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsyscall_zos_s390x.go
@@ -0,0 +1,1255 @@
+// go run mksyscall.go -tags zos,s390x syscall_zos_s390x.go
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+import (
+	"unsafe"
+)
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func fcntl(fd int, cmd int, arg int) (val int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_FCNTL, uintptr(fd), uintptr(cmd), uintptr(arg))
+	val = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func read(fd int, p []byte) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall(SYS_READ, uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func readlen(fd int, buf *byte, nbuf int) (n int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_READ, uintptr(fd), uintptr(unsafe.Pointer(buf)), uintptr(nbuf))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func write(fd int, p []byte) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall(SYS_WRITE, uintptr(fd), uintptr(_p0), uintptr(len(p)))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func accept(s int, rsa *RawSockaddrAny, addrlen *_Socklen) (fd int, err error) {
+	r0, _, e1 := syscall_syscall(SYS___ACCEPT_A, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func bind(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
+	_, _, e1 := syscall_syscall(SYS___BIND_A, uintptr(s), uintptr(addr), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func connect(s int, addr unsafe.Pointer, addrlen _Socklen) (err error) {
+	_, _, e1 := syscall_syscall(SYS___CONNECT_A, uintptr(s), uintptr(addr), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getgroups(n int, list *_Gid_t) (nn int, err error) {
+	r0, _, e1 := syscall_rawsyscall(SYS_GETGROUPS, uintptr(n), uintptr(unsafe.Pointer(list)), 0)
+	nn = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setgroups(n int, list *_Gid_t) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_SETGROUPS, uintptr(n), uintptr(unsafe.Pointer(list)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getsockopt(s int, level int, name int, val unsafe.Pointer, vallen *_Socklen) (err error) {
+	_, _, e1 := syscall_syscall6(SYS_GETSOCKOPT, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(unsafe.Pointer(vallen)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func setsockopt(s int, level int, name int, val unsafe.Pointer, vallen uintptr) (err error) {
+	_, _, e1 := syscall_syscall6(SYS_SETSOCKOPT, uintptr(s), uintptr(level), uintptr(name), uintptr(val), uintptr(vallen), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func socket(domain int, typ int, proto int) (fd int, err error) {
+	r0, _, e1 := syscall_rawsyscall(SYS_SOCKET, uintptr(domain), uintptr(typ), uintptr(proto))
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func socketpair(domain int, typ int, proto int, fd *[2]int32) (err error) {
+	_, _, e1 := syscall_rawsyscall6(SYS_SOCKETPAIR, uintptr(domain), uintptr(typ), uintptr(proto), uintptr(unsafe.Pointer(fd)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getpeername(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS___GETPEERNAME_A, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getsockname(fd int, rsa *RawSockaddrAny, addrlen *_Socklen) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS___GETSOCKNAME_A, uintptr(fd), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func recvfrom(fd int, p []byte, flags int, from *RawSockaddrAny, fromlen *_Socklen) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall6(SYS___RECVFROM_A, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(flags), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func sendto(s int, buf []byte, flags int, to unsafe.Pointer, addrlen _Socklen) (err error) {
+	var _p0 unsafe.Pointer
+	if len(buf) > 0 {
+		_p0 = unsafe.Pointer(&buf[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := syscall_syscall6(SYS___SENDTO_A, uintptr(s), uintptr(_p0), uintptr(len(buf)), uintptr(flags), uintptr(to), uintptr(addrlen))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func recvmsg(s int, msg *Msghdr, flags int) (n int, err error) {
+	r0, _, e1 := syscall_syscall(SYS___RECVMSG_A, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func sendmsg(s int, msg *Msghdr, flags int) (n int, err error) {
+	r0, _, e1 := syscall_syscall(SYS___SENDMSG_A, uintptr(s), uintptr(unsafe.Pointer(msg)), uintptr(flags))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func mmap(addr uintptr, length uintptr, prot int, flag int, fd int, pos int64) (ret uintptr, err error) {
+	r0, _, e1 := syscall_syscall6(SYS_MMAP, uintptr(addr), uintptr(length), uintptr(prot), uintptr(flag), uintptr(fd), uintptr(pos))
+	ret = uintptr(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func munmap(addr uintptr, length uintptr) (err error) {
+	_, _, e1 := syscall_syscall(SYS_MUNMAP, uintptr(addr), uintptr(length), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func ioctl(fd int, req uint, arg uintptr) (err error) {
+	_, _, e1 := syscall_syscall(SYS_IOCTL, uintptr(fd), uintptr(req), uintptr(arg))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Access(path string, mode uint32) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___ACCESS_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Chdir(path string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___CHDIR_A, uintptr(unsafe.Pointer(_p0)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Chown(path string, uid int, gid int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___CHOWN_A, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Chmod(path string, mode uint32) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___CHMOD_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Creat(path string, mode uint32) (fd int, err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	r0, _, e1 := syscall_syscall(SYS___CREAT_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Dup(oldfd int) (fd int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_DUP, uintptr(oldfd), 0, 0)
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Dup2(oldfd int, newfd int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_DUP2, uintptr(oldfd), uintptr(newfd), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Errno2() (er2 int) {
+	uer2, _, _ := syscall_syscall(SYS___ERRNO2, 0, 0, 0)
+	er2 = int(uer2)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Err2ad() (eadd *int) {
+	ueadd, _, _ := syscall_syscall(SYS___ERR2AD, 0, 0, 0)
+	eadd = (*int)(unsafe.Pointer(ueadd))
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Exit(code int) {
+	syscall_syscall(SYS_EXIT, uintptr(code), 0, 0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fchdir(fd int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FCHDIR, uintptr(fd), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fchmod(fd int, mode uint32) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FCHMOD, uintptr(fd), uintptr(mode), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fchown(fd int, uid int, gid int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FCHOWN, uintptr(fd), uintptr(uid), uintptr(gid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func FcntlInt(fd uintptr, cmd int, arg int) (retval int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_FCNTL, uintptr(fd), uintptr(cmd), uintptr(arg))
+	retval = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func fstat(fd int, stat *Stat_LE_t) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FSTAT, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fstatvfs(fd int, stat *Statvfs_t) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FSTATVFS, uintptr(fd), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Fsync(fd int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FSYNC, uintptr(fd), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Ftruncate(fd int, length int64) (err error) {
+	_, _, e1 := syscall_syscall(SYS_FTRUNCATE, uintptr(fd), uintptr(length), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getpagesize() (pgsize int) {
+	r0, _, _ := syscall_syscall(SYS_GETPAGESIZE, 0, 0, 0)
+	pgsize = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Mprotect(b []byte, prot int) (err error) {
+	var _p0 unsafe.Pointer
+	if len(b) > 0 {
+		_p0 = unsafe.Pointer(&b[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := syscall_syscall(SYS_MPROTECT, uintptr(_p0), uintptr(len(b)), uintptr(prot))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Msync(b []byte, flags int) (err error) {
+	var _p0 unsafe.Pointer
+	if len(b) > 0 {
+		_p0 = unsafe.Pointer(&b[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := syscall_syscall(SYS_MSYNC, uintptr(_p0), uintptr(len(b)), uintptr(flags))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Poll(fds []PollFd, timeout int) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(fds) > 0 {
+		_p0 = unsafe.Pointer(&fds[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall(SYS_POLL, uintptr(_p0), uintptr(len(fds)), uintptr(timeout))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Times(tms *Tms) (ticks uintptr, err error) {
+	r0, _, e1 := syscall_syscall(SYS_TIMES, uintptr(unsafe.Pointer(tms)), 0, 0)
+	ticks = uintptr(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func W_Getmntent(buff *byte, size int) (lastsys int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_W_GETMNTENT, uintptr(unsafe.Pointer(buff)), uintptr(size), 0)
+	lastsys = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func W_Getmntent_A(buff *byte, size int) (lastsys int, err error) {
+	r0, _, e1 := syscall_syscall(SYS___W_GETMNTENT_A, uintptr(unsafe.Pointer(buff)), uintptr(size), 0)
+	lastsys = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func mount_LE(path string, filesystem string, fstype string, mtm uint32, parmlen int32, parm string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = BytePtrFromString(filesystem)
+	if err != nil {
+		return
+	}
+	var _p2 *byte
+	_p2, err = BytePtrFromString(fstype)
+	if err != nil {
+		return
+	}
+	var _p3 *byte
+	_p3, err = BytePtrFromString(parm)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall6(SYS___MOUNT_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), uintptr(unsafe.Pointer(_p2)), uintptr(mtm), uintptr(parmlen), uintptr(unsafe.Pointer(_p3)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func unmount(filesystem string, mtm int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(filesystem)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___UMOUNT_A, uintptr(unsafe.Pointer(_p0)), uintptr(mtm), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Chroot(path string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___CHROOT_A, uintptr(unsafe.Pointer(_p0)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Uname(buf *Utsname) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS___UNAME_A, uintptr(unsafe.Pointer(buf)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Gethostname(buf []byte) (err error) {
+	var _p0 unsafe.Pointer
+	if len(buf) > 0 {
+		_p0 = unsafe.Pointer(&buf[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	_, _, e1 := syscall_syscall(SYS___GETHOSTNAME_A, uintptr(_p0), uintptr(len(buf)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getegid() (egid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETEGID, 0, 0, 0)
+	egid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Geteuid() (uid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETEUID, 0, 0, 0)
+	uid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getgid() (gid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETGID, 0, 0, 0)
+	gid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getpid() (pid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETPID, 0, 0, 0)
+	pid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getpgid(pid int) (pgid int, err error) {
+	r0, _, e1 := syscall_rawsyscall(SYS_GETPGID, uintptr(pid), 0, 0)
+	pgid = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getppid() (pid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETPPID, 0, 0, 0)
+	pid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getpriority(which int, who int) (prio int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_GETPRIORITY, uintptr(which), uintptr(who), 0)
+	prio = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getrlimit(resource int, rlim *Rlimit) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_GETRLIMIT, uintptr(resource), uintptr(unsafe.Pointer(rlim)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func getrusage(who int, rusage *rusage_zos) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_GETRUSAGE, uintptr(who), uintptr(unsafe.Pointer(rusage)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getsid(pid int) (sid int, err error) {
+	r0, _, e1 := syscall_rawsyscall(SYS_GETSID, uintptr(pid), 0, 0)
+	sid = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Getuid() (uid int) {
+	r0, _, _ := syscall_rawsyscall(SYS_GETUID, 0, 0, 0)
+	uid = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Kill(pid int, sig Signal) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_KILL, uintptr(pid), uintptr(sig), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Lchown(path string, uid int, gid int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___LCHOWN_A, uintptr(unsafe.Pointer(_p0)), uintptr(uid), uintptr(gid))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Link(path string, link string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = BytePtrFromString(link)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___LINK_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Listen(s int, n int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_LISTEN, uintptr(s), uintptr(n), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func lstat(path string, stat *Stat_LE_t) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___LSTAT_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(stat)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Mkdir(path string, mode uint32) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___MKDIR_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Mkfifo(path string, mode uint32) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___MKFIFO_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Mknod(path string, mode uint32, dev int) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___MKNOD_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(dev))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Pread(fd int, p []byte, offset int64) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall6(SYS_PREAD, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Pwrite(fd int, p []byte, offset int64) (n int, err error) {
+	var _p0 unsafe.Pointer
+	if len(p) > 0 {
+		_p0 = unsafe.Pointer(&p[0])
+	} else {
+		_p0 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall6(SYS_PWRITE, uintptr(fd), uintptr(_p0), uintptr(len(p)), uintptr(offset), 0, 0)
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Readlink(path string, buf []byte) (n int, err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	var _p1 unsafe.Pointer
+	if len(buf) > 0 {
+		_p1 = unsafe.Pointer(&buf[0])
+	} else {
+		_p1 = unsafe.Pointer(&_zero)
+	}
+	r0, _, e1 := syscall_syscall(SYS___READLINK_A, uintptr(unsafe.Pointer(_p0)), uintptr(_p1), uintptr(len(buf)))
+	n = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Rename(from string, to string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(from)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = BytePtrFromString(to)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___RENAME_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Rmdir(path string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___RMDIR_A, uintptr(unsafe.Pointer(_p0)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Seek(fd int, offset int64, whence int) (off int64, err error) {
+	r0, _, e1 := syscall_syscall(SYS_LSEEK, uintptr(fd), uintptr(offset), uintptr(whence))
+	off = int64(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setpriority(which int, who int, prio int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_SETPRIORITY, uintptr(which), uintptr(who), uintptr(prio))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setpgid(pid int, pgid int) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_SETPGID, uintptr(pid), uintptr(pgid), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setrlimit(resource int, lim *Rlimit) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_SETRLIMIT, uintptr(resource), uintptr(unsafe.Pointer(lim)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setregid(rgid int, egid int) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_SETREGID, uintptr(rgid), uintptr(egid), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setreuid(ruid int, euid int) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_SETREUID, uintptr(ruid), uintptr(euid), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setsid() (pid int, err error) {
+	r0, _, e1 := syscall_rawsyscall(SYS_SETSID, 0, 0, 0)
+	pid = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setuid(uid int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_SETUID, uintptr(uid), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Setgid(uid int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_SETGID, uintptr(uid), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Shutdown(fd int, how int) (err error) {
+	_, _, e1 := syscall_syscall(SYS_SHUTDOWN, uintptr(fd), uintptr(how), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func stat(path string, statLE *Stat_LE_t) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___STAT_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(statLE)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Symlink(path string, link string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = BytePtrFromString(link)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___SYMLINK_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(_p1)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Sync() {
+	syscall_syscall(SYS_SYNC, 0, 0, 0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Truncate(path string, length int64) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___TRUNCATE_A, uintptr(unsafe.Pointer(_p0)), uintptr(length), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Tcgetattr(fildes int, termptr *Termios) (err error) {
+	_, _, e1 := syscall_syscall(SYS_TCGETATTR, uintptr(fildes), uintptr(unsafe.Pointer(termptr)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Tcsetattr(fildes int, when int, termptr *Termios) (err error) {
+	_, _, e1 := syscall_syscall(SYS_TCSETATTR, uintptr(fildes), uintptr(when), uintptr(unsafe.Pointer(termptr)))
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Umask(mask int) (oldmask int) {
+	r0, _, _ := syscall_syscall(SYS_UMASK, uintptr(mask), 0, 0)
+	oldmask = int(r0)
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Unlink(path string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___UNLINK_A, uintptr(unsafe.Pointer(_p0)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Utime(path string, utim *Utimbuf) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___UTIME_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(utim)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func open(path string, mode int, perm uint32) (fd int, err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	r0, _, e1 := syscall_syscall(SYS___OPEN_A, uintptr(unsafe.Pointer(_p0)), uintptr(mode), uintptr(perm))
+	fd = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func remove(path string) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS_REMOVE, uintptr(unsafe.Pointer(_p0)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func waitpid(pid int, wstatus *_C_int, options int) (wpid int, err error) {
+	r0, _, e1 := syscall_syscall(SYS_WAITPID, uintptr(pid), uintptr(unsafe.Pointer(wstatus)), uintptr(options))
+	wpid = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func gettimeofday(tv *timeval_zos) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_GETTIMEOFDAY, uintptr(unsafe.Pointer(tv)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func pipe(p *[2]_C_int) (err error) {
+	_, _, e1 := syscall_rawsyscall(SYS_PIPE, uintptr(unsafe.Pointer(p)), 0, 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func utimes(path string, timeval *[2]Timeval) (err error) {
+	var _p0 *byte
+	_p0, err = BytePtrFromString(path)
+	if err != nil {
+		return
+	}
+	_, _, e1 := syscall_syscall(SYS___UTIMES_A, uintptr(unsafe.Pointer(_p0)), uintptr(unsafe.Pointer(timeval)), 0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
+
+// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT
+
+func Select(nmsgsfds int, r *FdSet, w *FdSet, e *FdSet, timeout *Timeval) (ret int, err error) {
+	r0, _, e1 := syscall_syscall6(SYS_SELECT, uintptr(nmsgsfds), uintptr(unsafe.Pointer(r)), uintptr(unsafe.Pointer(w)), uintptr(unsafe.Pointer(e)), uintptr(unsafe.Pointer(timeout)), 0)
+	ret = int(r0)
+	if e1 != 0 {
+		err = errnoErr(e1)
+	}
+	return
+}
diff --git a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_386.go b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_386.go
index 102f1ab..9e9d0b2 100644
--- a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_386.go
@@ -1,6 +1,7 @@
 // go run mksysctl_openbsd.go
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build 386 && openbsd
 // +build 386,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_amd64.go
index 4866fce..adecd09 100644
--- a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksysctl_openbsd.go
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build amd64 && openbsd
 // +build amd64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm.go b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm.go
index d3801eb..8ea52a4 100644
--- a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksysctl_openbsd.go
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build arm && openbsd
 // +build arm,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm64.go
index ba4304f..154b57a 100644
--- a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksysctl_openbsd.go
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build arm64 && openbsd
 // +build arm64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_mips64.go
index aca34b3..d96bb2b 100644
--- a/vendor/golang.org/x/sys/unix/zsysctl_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zsysctl_openbsd_mips64.go
@@ -1,6 +1,7 @@
 // go run mksysctl_openbsd.go
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build mips64 && openbsd
 // +build mips64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_darwin_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_darwin_amd64.go
new file mode 100644
index 0000000..f8298ff
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsysnum_darwin_amd64.go
@@ -0,0 +1,440 @@
+// go run mksysnum.go /Applications/Xcode.app/Contents/Developer/Platforms/MacOSX.platform/Developer/SDKs/MacOSX10.14.sdk/usr/include/sys/syscall.h
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build amd64 && darwin
+// +build amd64,darwin
+
+package unix
+
+// Deprecated: Use libSystem wrappers instead of direct syscalls.
+const (
+	SYS_SYSCALL                        = 0
+	SYS_EXIT                           = 1
+	SYS_FORK                           = 2
+	SYS_READ                           = 3
+	SYS_WRITE                          = 4
+	SYS_OPEN                           = 5
+	SYS_CLOSE                          = 6
+	SYS_WAIT4                          = 7
+	SYS_LINK                           = 9
+	SYS_UNLINK                         = 10
+	SYS_CHDIR                          = 12
+	SYS_FCHDIR                         = 13
+	SYS_MKNOD                          = 14
+	SYS_CHMOD                          = 15
+	SYS_CHOWN                          = 16
+	SYS_GETFSSTAT                      = 18
+	SYS_GETPID                         = 20
+	SYS_SETUID                         = 23
+	SYS_GETUID                         = 24
+	SYS_GETEUID                        = 25
+	SYS_PTRACE                         = 26
+	SYS_RECVMSG                        = 27
+	SYS_SENDMSG                        = 28
+	SYS_RECVFROM                       = 29
+	SYS_ACCEPT                         = 30
+	SYS_GETPEERNAME                    = 31
+	SYS_GETSOCKNAME                    = 32
+	SYS_ACCESS                         = 33
+	SYS_CHFLAGS                        = 34
+	SYS_FCHFLAGS                       = 35
+	SYS_SYNC                           = 36
+	SYS_KILL                           = 37
+	SYS_GETPPID                        = 39
+	SYS_DUP                            = 41
+	SYS_PIPE                           = 42
+	SYS_GETEGID                        = 43
+	SYS_SIGACTION                      = 46
+	SYS_GETGID                         = 47
+	SYS_SIGPROCMASK                    = 48
+	SYS_GETLOGIN                       = 49
+	SYS_SETLOGIN                       = 50
+	SYS_ACCT                           = 51
+	SYS_SIGPENDING                     = 52
+	SYS_SIGALTSTACK                    = 53
+	SYS_IOCTL                          = 54
+	SYS_REBOOT                         = 55
+	SYS_REVOKE                         = 56
+	SYS_SYMLINK                        = 57
+	SYS_READLINK                       = 58
+	SYS_EXECVE                         = 59
+	SYS_UMASK                          = 60
+	SYS_CHROOT                         = 61
+	SYS_MSYNC                          = 65
+	SYS_VFORK                          = 66
+	SYS_MUNMAP                         = 73
+	SYS_MPROTECT                       = 74
+	SYS_MADVISE                        = 75
+	SYS_MINCORE                        = 78
+	SYS_GETGROUPS                      = 79
+	SYS_SETGROUPS                      = 80
+	SYS_GETPGRP                        = 81
+	SYS_SETPGID                        = 82
+	SYS_SETITIMER                      = 83
+	SYS_SWAPON                         = 85
+	SYS_GETITIMER                      = 86
+	SYS_GETDTABLESIZE                  = 89
+	SYS_DUP2                           = 90
+	SYS_FCNTL                          = 92
+	SYS_SELECT                         = 93
+	SYS_FSYNC                          = 95
+	SYS_SETPRIORITY                    = 96
+	SYS_SOCKET                         = 97
+	SYS_CONNECT                        = 98
+	SYS_GETPRIORITY                    = 100
+	SYS_BIND                           = 104
+	SYS_SETSOCKOPT                     = 105
+	SYS_LISTEN                         = 106
+	SYS_SIGSUSPEND                     = 111
+	SYS_GETTIMEOFDAY                   = 116
+	SYS_GETRUSAGE                      = 117
+	SYS_GETSOCKOPT                     = 118
+	SYS_READV                          = 120
+	SYS_WRITEV                         = 121
+	SYS_SETTIMEOFDAY                   = 122
+	SYS_FCHOWN                         = 123
+	SYS_FCHMOD                         = 124
+	SYS_SETREUID                       = 126
+	SYS_SETREGID                       = 127
+	SYS_RENAME                         = 128
+	SYS_FLOCK                          = 131
+	SYS_MKFIFO                         = 132
+	SYS_SENDTO                         = 133
+	SYS_SHUTDOWN                       = 134
+	SYS_SOCKETPAIR                     = 135
+	SYS_MKDIR                          = 136
+	SYS_RMDIR                          = 137
+	SYS_UTIMES                         = 138
+	SYS_FUTIMES                        = 139
+	SYS_ADJTIME                        = 140
+	SYS_GETHOSTUUID                    = 142
+	SYS_SETSID                         = 147
+	SYS_GETPGID                        = 151
+	SYS_SETPRIVEXEC                    = 152
+	SYS_PREAD                          = 153
+	SYS_PWRITE                         = 154
+	SYS_NFSSVC                         = 155
+	SYS_STATFS                         = 157
+	SYS_FSTATFS                        = 158
+	SYS_UNMOUNT                        = 159
+	SYS_GETFH                          = 161
+	SYS_QUOTACTL                       = 165
+	SYS_MOUNT                          = 167
+	SYS_CSOPS                          = 169
+	SYS_CSOPS_AUDITTOKEN               = 170
+	SYS_WAITID                         = 173
+	SYS_KDEBUG_TYPEFILTER              = 177
+	SYS_KDEBUG_TRACE_STRING            = 178
+	SYS_KDEBUG_TRACE64                 = 179
+	SYS_KDEBUG_TRACE                   = 180
+	SYS_SETGID                         = 181
+	SYS_SETEGID                        = 182
+	SYS_SETEUID                        = 183
+	SYS_SIGRETURN                      = 184
+	SYS_THREAD_SELFCOUNTS              = 186
+	SYS_FDATASYNC                      = 187
+	SYS_STAT                           = 188
+	SYS_FSTAT                          = 189
+	SYS_LSTAT                          = 190
+	SYS_PATHCONF                       = 191
+	SYS_FPATHCONF                      = 192
+	SYS_GETRLIMIT                      = 194
+	SYS_SETRLIMIT                      = 195
+	SYS_GETDIRENTRIES                  = 196
+	SYS_MMAP                           = 197
+	SYS_LSEEK                          = 199
+	SYS_TRUNCATE                       = 200
+	SYS_FTRUNCATE                      = 201
+	SYS_SYSCTL                         = 202
+	SYS_MLOCK                          = 203
+	SYS_MUNLOCK                        = 204
+	SYS_UNDELETE                       = 205
+	SYS_OPEN_DPROTECTED_NP             = 216
+	SYS_GETATTRLIST                    = 220
+	SYS_SETATTRLIST                    = 221
+	SYS_GETDIRENTRIESATTR              = 222
+	SYS_EXCHANGEDATA                   = 223
+	SYS_SEARCHFS                       = 225
+	SYS_DELETE                         = 226
+	SYS_COPYFILE                       = 227
+	SYS_FGETATTRLIST                   = 228
+	SYS_FSETATTRLIST                   = 229
+	SYS_POLL                           = 230
+	SYS_WATCHEVENT                     = 231
+	SYS_WAITEVENT                      = 232
+	SYS_MODWATCH                       = 233
+	SYS_GETXATTR                       = 234
+	SYS_FGETXATTR                      = 235
+	SYS_SETXATTR                       = 236
+	SYS_FSETXATTR                      = 237
+	SYS_REMOVEXATTR                    = 238
+	SYS_FREMOVEXATTR                   = 239
+	SYS_LISTXATTR                      = 240
+	SYS_FLISTXATTR                     = 241
+	SYS_FSCTL                          = 242
+	SYS_INITGROUPS                     = 243
+	SYS_POSIX_SPAWN                    = 244
+	SYS_FFSCTL                         = 245
+	SYS_NFSCLNT                        = 247
+	SYS_FHOPEN                         = 248
+	SYS_MINHERIT                       = 250
+	SYS_SEMSYS                         = 251
+	SYS_MSGSYS                         = 252
+	SYS_SHMSYS                         = 253
+	SYS_SEMCTL                         = 254
+	SYS_SEMGET                         = 255
+	SYS_SEMOP                          = 256
+	SYS_MSGCTL                         = 258
+	SYS_MSGGET                         = 259
+	SYS_MSGSND                         = 260
+	SYS_MSGRCV                         = 261
+	SYS_SHMAT                          = 262
+	SYS_SHMCTL                         = 263
+	SYS_SHMDT                          = 264
+	SYS_SHMGET                         = 265
+	SYS_SHM_OPEN                       = 266
+	SYS_SHM_UNLINK                     = 267
+	SYS_SEM_OPEN                       = 268
+	SYS_SEM_CLOSE                      = 269
+	SYS_SEM_UNLINK                     = 270
+	SYS_SEM_WAIT                       = 271
+	SYS_SEM_TRYWAIT                    = 272
+	SYS_SEM_POST                       = 273
+	SYS_SYSCTLBYNAME                   = 274
+	SYS_OPEN_EXTENDED                  = 277
+	SYS_UMASK_EXTENDED                 = 278
+	SYS_STAT_EXTENDED                  = 279
+	SYS_LSTAT_EXTENDED                 = 280
+	SYS_FSTAT_EXTENDED                 = 281
+	SYS_CHMOD_EXTENDED                 = 282
+	SYS_FCHMOD_EXTENDED                = 283
+	SYS_ACCESS_EXTENDED                = 284
+	SYS_SETTID                         = 285
+	SYS_GETTID                         = 286
+	SYS_SETSGROUPS                     = 287
+	SYS_GETSGROUPS                     = 288
+	SYS_SETWGROUPS                     = 289
+	SYS_GETWGROUPS                     = 290
+	SYS_MKFIFO_EXTENDED                = 291
+	SYS_MKDIR_EXTENDED                 = 292
+	SYS_IDENTITYSVC                    = 293
+	SYS_SHARED_REGION_CHECK_NP         = 294
+	SYS_VM_PRESSURE_MONITOR            = 296
+	SYS_PSYNCH_RW_LONGRDLOCK           = 297
+	SYS_PSYNCH_RW_YIELDWRLOCK          = 298
+	SYS_PSYNCH_RW_DOWNGRADE            = 299
+	SYS_PSYNCH_RW_UPGRADE              = 300
+	SYS_PSYNCH_MUTEXWAIT               = 301
+	SYS_PSYNCH_MUTEXDROP               = 302
+	SYS_PSYNCH_CVBROAD                 = 303
+	SYS_PSYNCH_CVSIGNAL                = 304
+	SYS_PSYNCH_CVWAIT                  = 305
+	SYS_PSYNCH_RW_RDLOCK               = 306
+	SYS_PSYNCH_RW_WRLOCK               = 307
+	SYS_PSYNCH_RW_UNLOCK               = 308
+	SYS_PSYNCH_RW_UNLOCK2              = 309
+	SYS_GETSID                         = 310
+	SYS_SETTID_WITH_PID                = 311
+	SYS_PSYNCH_CVCLRPREPOST            = 312
+	SYS_AIO_FSYNC                      = 313
+	SYS_AIO_RETURN                     = 314
+	SYS_AIO_SUSPEND                    = 315
+	SYS_AIO_CANCEL                     = 316
+	SYS_AIO_ERROR                      = 317
+	SYS_AIO_READ                       = 318
+	SYS_AIO_WRITE                      = 319
+	SYS_LIO_LISTIO                     = 320
+	SYS_IOPOLICYSYS                    = 322
+	SYS_PROCESS_POLICY                 = 323
+	SYS_MLOCKALL                       = 324
+	SYS_MUNLOCKALL                     = 325
+	SYS_ISSETUGID                      = 327
+	SYS___PTHREAD_KILL                 = 328
+	SYS___PTHREAD_SIGMASK              = 329
+	SYS___SIGWAIT                      = 330
+	SYS___DISABLE_THREADSIGNAL         = 331
+	SYS___PTHREAD_MARKCANCEL           = 332
+	SYS___PTHREAD_CANCELED             = 333
+	SYS___SEMWAIT_SIGNAL               = 334
+	SYS_PROC_INFO                      = 336
+	SYS_SENDFILE                       = 337
+	SYS_STAT64                         = 338
+	SYS_FSTAT64                        = 339
+	SYS_LSTAT64                        = 340
+	SYS_STAT64_EXTENDED                = 341
+	SYS_LSTAT64_EXTENDED               = 342
+	SYS_FSTAT64_EXTENDED               = 343
+	SYS_GETDIRENTRIES64                = 344
+	SYS_STATFS64                       = 345
+	SYS_FSTATFS64                      = 346
+	SYS_GETFSSTAT64                    = 347
+	SYS___PTHREAD_CHDIR                = 348
+	SYS___PTHREAD_FCHDIR               = 349
+	SYS_AUDIT                          = 350
+	SYS_AUDITON                        = 351
+	SYS_GETAUID                        = 353
+	SYS_SETAUID                        = 354
+	SYS_GETAUDIT_ADDR                  = 357
+	SYS_SETAUDIT_ADDR                  = 358
+	SYS_AUDITCTL                       = 359
+	SYS_BSDTHREAD_CREATE               = 360
+	SYS_BSDTHREAD_TERMINATE            = 361
+	SYS_KQUEUE                         = 362
+	SYS_KEVENT                         = 363
+	SYS_LCHOWN                         = 364
+	SYS_BSDTHREAD_REGISTER             = 366
+	SYS_WORKQ_OPEN                     = 367
+	SYS_WORKQ_KERNRETURN               = 368
+	SYS_KEVENT64                       = 369
+	SYS___OLD_SEMWAIT_SIGNAL           = 370
+	SYS___OLD_SEMWAIT_SIGNAL_NOCANCEL  = 371
+	SYS_THREAD_SELFID                  = 372
+	SYS_LEDGER                         = 373
+	SYS_KEVENT_QOS                     = 374
+	SYS_KEVENT_ID                      = 375
+	SYS___MAC_EXECVE                   = 380
+	SYS___MAC_SYSCALL                  = 381
+	SYS___MAC_GET_FILE                 = 382
+	SYS___MAC_SET_FILE                 = 383
+	SYS___MAC_GET_LINK                 = 384
+	SYS___MAC_SET_LINK                 = 385
+	SYS___MAC_GET_PROC                 = 386
+	SYS___MAC_SET_PROC                 = 387
+	SYS___MAC_GET_FD                   = 388
+	SYS___MAC_SET_FD                   = 389
+	SYS___MAC_GET_PID                  = 390
+	SYS_PSELECT                        = 394
+	SYS_PSELECT_NOCANCEL               = 395
+	SYS_READ_NOCANCEL                  = 396
+	SYS_WRITE_NOCANCEL                 = 397
+	SYS_OPEN_NOCANCEL                  = 398
+	SYS_CLOSE_NOCANCEL                 = 399
+	SYS_WAIT4_NOCANCEL                 = 400
+	SYS_RECVMSG_NOCANCEL               = 401
+	SYS_SENDMSG_NOCANCEL               = 402
+	SYS_RECVFROM_NOCANCEL              = 403
+	SYS_ACCEPT_NOCANCEL                = 404
+	SYS_MSYNC_NOCANCEL                 = 405
+	SYS_FCNTL_NOCANCEL                 = 406
+	SYS_SELECT_NOCANCEL                = 407
+	SYS_FSYNC_NOCANCEL                 = 408
+	SYS_CONNECT_NOCANCEL               = 409
+	SYS_SIGSUSPEND_NOCANCEL            = 410
+	SYS_READV_NOCANCEL                 = 411
+	SYS_WRITEV_NOCANCEL                = 412
+	SYS_SENDTO_NOCANCEL                = 413
+	SYS_PREAD_NOCANCEL                 = 414
+	SYS_PWRITE_NOCANCEL                = 415
+	SYS_WAITID_NOCANCEL                = 416
+	SYS_POLL_NOCANCEL                  = 417
+	SYS_MSGSND_NOCANCEL                = 418
+	SYS_MSGRCV_NOCANCEL                = 419
+	SYS_SEM_WAIT_NOCANCEL              = 420
+	SYS_AIO_SUSPEND_NOCANCEL           = 421
+	SYS___SIGWAIT_NOCANCEL             = 422
+	SYS___SEMWAIT_SIGNAL_NOCANCEL      = 423
+	SYS___MAC_MOUNT                    = 424
+	SYS___MAC_GET_MOUNT                = 425
+	SYS___MAC_GETFSSTAT                = 426
+	SYS_FSGETPATH                      = 427
+	SYS_AUDIT_SESSION_SELF             = 428
+	SYS_AUDIT_SESSION_JOIN             = 429
+	SYS_FILEPORT_MAKEPORT              = 430
+	SYS_FILEPORT_MAKEFD                = 431
+	SYS_AUDIT_SESSION_PORT             = 432
+	SYS_PID_SUSPEND                    = 433
+	SYS_PID_RESUME                     = 434
+	SYS_PID_HIBERNATE                  = 435
+	SYS_PID_SHUTDOWN_SOCKETS           = 436
+	SYS_SHARED_REGION_MAP_AND_SLIDE_NP = 438
+	SYS_KAS_INFO                       = 439
+	SYS_MEMORYSTATUS_CONTROL           = 440
+	SYS_GUARDED_OPEN_NP                = 441
+	SYS_GUARDED_CLOSE_NP               = 442
+	SYS_GUARDED_KQUEUE_NP              = 443
+	SYS_CHANGE_FDGUARD_NP              = 444
+	SYS_USRCTL                         = 445
+	SYS_PROC_RLIMIT_CONTROL            = 446
+	SYS_CONNECTX                       = 447
+	SYS_DISCONNECTX                    = 448
+	SYS_PEELOFF                        = 449
+	SYS_SOCKET_DELEGATE                = 450
+	SYS_TELEMETRY                      = 451
+	SYS_PROC_UUID_POLICY               = 452
+	SYS_MEMORYSTATUS_GET_LEVEL         = 453
+	SYS_SYSTEM_OVERRIDE                = 454
+	SYS_VFS_PURGE                      = 455
+	SYS_SFI_CTL                        = 456
+	SYS_SFI_PIDCTL                     = 457
+	SYS_COALITION                      = 458
+	SYS_COALITION_INFO                 = 459
+	SYS_NECP_MATCH_POLICY              = 460
+	SYS_GETATTRLISTBULK                = 461
+	SYS_CLONEFILEAT                    = 462
+	SYS_OPENAT                         = 463
+	SYS_OPENAT_NOCANCEL                = 464
+	SYS_RENAMEAT                       = 465
+	SYS_FACCESSAT                      = 466
+	SYS_FCHMODAT                       = 467
+	SYS_FCHOWNAT                       = 468
+	SYS_FSTATAT                        = 469
+	SYS_FSTATAT64                      = 470
+	SYS_LINKAT                         = 471
+	SYS_UNLINKAT                       = 472
+	SYS_READLINKAT                     = 473
+	SYS_SYMLINKAT                      = 474
+	SYS_MKDIRAT                        = 475
+	SYS_GETATTRLISTAT                  = 476
+	SYS_PROC_TRACE_LOG                 = 477
+	SYS_BSDTHREAD_CTL                  = 478
+	SYS_OPENBYID_NP                    = 479
+	SYS_RECVMSG_X                      = 480
+	SYS_SENDMSG_X                      = 481
+	SYS_THREAD_SELFUSAGE               = 482
+	SYS_CSRCTL                         = 483
+	SYS_GUARDED_OPEN_DPROTECTED_NP     = 484
+	SYS_GUARDED_WRITE_NP               = 485
+	SYS_GUARDED_PWRITE_NP              = 486
+	SYS_GUARDED_WRITEV_NP              = 487
+	SYS_RENAMEATX_NP                   = 488
+	SYS_MREMAP_ENCRYPTED               = 489
+	SYS_NETAGENT_TRIGGER               = 490
+	SYS_STACK_SNAPSHOT_WITH_CONFIG     = 491
+	SYS_MICROSTACKSHOT                 = 492
+	SYS_GRAB_PGO_DATA                  = 493
+	SYS_PERSONA                        = 494
+	SYS_WORK_INTERVAL_CTL              = 499
+	SYS_GETENTROPY                     = 500
+	SYS_NECP_OPEN                      = 501
+	SYS_NECP_CLIENT_ACTION             = 502
+	SYS___NEXUS_OPEN                   = 503
+	SYS___NEXUS_REGISTER               = 504
+	SYS___NEXUS_DEREGISTER             = 505
+	SYS___NEXUS_CREATE                 = 506
+	SYS___NEXUS_DESTROY                = 507
+	SYS___NEXUS_GET_OPT                = 508
+	SYS___NEXUS_SET_OPT                = 509
+	SYS___CHANNEL_OPEN                 = 510
+	SYS___CHANNEL_GET_INFO             = 511
+	SYS___CHANNEL_SYNC                 = 512
+	SYS___CHANNEL_GET_OPT              = 513
+	SYS___CHANNEL_SET_OPT              = 514
+	SYS_ULOCK_WAIT                     = 515
+	SYS_ULOCK_WAKE                     = 516
+	SYS_FCLONEFILEAT                   = 517
+	SYS_FS_SNAPSHOT                    = 518
+	SYS_TERMINATE_WITH_PAYLOAD         = 520
+	SYS_ABORT_WITH_PAYLOAD             = 521
+	SYS_NECP_SESSION_OPEN              = 522
+	SYS_NECP_SESSION_ACTION            = 523
+	SYS_SETATTRLISTAT                  = 524
+	SYS_NET_QOS_GUIDELINE              = 525
+	SYS_FMOUNT                         = 526
+	SYS_NTP_ADJTIME                    = 527
+	SYS_NTP_GETTIME                    = 528
+	SYS_OS_FAULT_WITH_PAYLOAD          = 529
+	SYS_KQUEUE_WORKLOOP_CTL            = 530
+	SYS___MACH_BRIDGE_REMOTE_TIME      = 531
+	SYS_MAXSYSCALL                     = 532
+	SYS_INVALID                        = 63
+)
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_darwin_arm64.go b/vendor/golang.org/x/sys/unix/zsysnum_darwin_arm64.go
new file mode 100644
index 0000000..5eb433b
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsysnum_darwin_arm64.go
@@ -0,0 +1,438 @@
+// go run mksysnum.go /Applications/Xcode.app/Contents/Developer/Platforms/iPhoneOS.platform/Developer/SDKs/iPhoneOS11.1.sdk/usr/include/sys/syscall.h
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build arm64 && darwin
+// +build arm64,darwin
+
+package unix
+
+// Deprecated: Use libSystem wrappers instead of direct syscalls.
+const (
+	SYS_SYSCALL                        = 0
+	SYS_EXIT                           = 1
+	SYS_FORK                           = 2
+	SYS_READ                           = 3
+	SYS_WRITE                          = 4
+	SYS_OPEN                           = 5
+	SYS_CLOSE                          = 6
+	SYS_WAIT4                          = 7
+	SYS_LINK                           = 9
+	SYS_UNLINK                         = 10
+	SYS_CHDIR                          = 12
+	SYS_FCHDIR                         = 13
+	SYS_MKNOD                          = 14
+	SYS_CHMOD                          = 15
+	SYS_CHOWN                          = 16
+	SYS_GETFSSTAT                      = 18
+	SYS_GETPID                         = 20
+	SYS_SETUID                         = 23
+	SYS_GETUID                         = 24
+	SYS_GETEUID                        = 25
+	SYS_PTRACE                         = 26
+	SYS_RECVMSG                        = 27
+	SYS_SENDMSG                        = 28
+	SYS_RECVFROM                       = 29
+	SYS_ACCEPT                         = 30
+	SYS_GETPEERNAME                    = 31
+	SYS_GETSOCKNAME                    = 32
+	SYS_ACCESS                         = 33
+	SYS_CHFLAGS                        = 34
+	SYS_FCHFLAGS                       = 35
+	SYS_SYNC                           = 36
+	SYS_KILL                           = 37
+	SYS_GETPPID                        = 39
+	SYS_DUP                            = 41
+	SYS_PIPE                           = 42
+	SYS_GETEGID                        = 43
+	SYS_SIGACTION                      = 46
+	SYS_GETGID                         = 47
+	SYS_SIGPROCMASK                    = 48
+	SYS_GETLOGIN                       = 49
+	SYS_SETLOGIN                       = 50
+	SYS_ACCT                           = 51
+	SYS_SIGPENDING                     = 52
+	SYS_SIGALTSTACK                    = 53
+	SYS_IOCTL                          = 54
+	SYS_REBOOT                         = 55
+	SYS_REVOKE                         = 56
+	SYS_SYMLINK                        = 57
+	SYS_READLINK                       = 58
+	SYS_EXECVE                         = 59
+	SYS_UMASK                          = 60
+	SYS_CHROOT                         = 61
+	SYS_MSYNC                          = 65
+	SYS_VFORK                          = 66
+	SYS_MUNMAP                         = 73
+	SYS_MPROTECT                       = 74
+	SYS_MADVISE                        = 75
+	SYS_MINCORE                        = 78
+	SYS_GETGROUPS                      = 79
+	SYS_SETGROUPS                      = 80
+	SYS_GETPGRP                        = 81
+	SYS_SETPGID                        = 82
+	SYS_SETITIMER                      = 83
+	SYS_SWAPON                         = 85
+	SYS_GETITIMER                      = 86
+	SYS_GETDTABLESIZE                  = 89
+	SYS_DUP2                           = 90
+	SYS_FCNTL                          = 92
+	SYS_SELECT                         = 93
+	SYS_FSYNC                          = 95
+	SYS_SETPRIORITY                    = 96
+	SYS_SOCKET                         = 97
+	SYS_CONNECT                        = 98
+	SYS_GETPRIORITY                    = 100
+	SYS_BIND                           = 104
+	SYS_SETSOCKOPT                     = 105
+	SYS_LISTEN                         = 106
+	SYS_SIGSUSPEND                     = 111
+	SYS_GETTIMEOFDAY                   = 116
+	SYS_GETRUSAGE                      = 117
+	SYS_GETSOCKOPT                     = 118
+	SYS_READV                          = 120
+	SYS_WRITEV                         = 121
+	SYS_SETTIMEOFDAY                   = 122
+	SYS_FCHOWN                         = 123
+	SYS_FCHMOD                         = 124
+	SYS_SETREUID                       = 126
+	SYS_SETREGID                       = 127
+	SYS_RENAME                         = 128
+	SYS_FLOCK                          = 131
+	SYS_MKFIFO                         = 132
+	SYS_SENDTO                         = 133
+	SYS_SHUTDOWN                       = 134
+	SYS_SOCKETPAIR                     = 135
+	SYS_MKDIR                          = 136
+	SYS_RMDIR                          = 137
+	SYS_UTIMES                         = 138
+	SYS_FUTIMES                        = 139
+	SYS_ADJTIME                        = 140
+	SYS_GETHOSTUUID                    = 142
+	SYS_SETSID                         = 147
+	SYS_GETPGID                        = 151
+	SYS_SETPRIVEXEC                    = 152
+	SYS_PREAD                          = 153
+	SYS_PWRITE                         = 154
+	SYS_NFSSVC                         = 155
+	SYS_STATFS                         = 157
+	SYS_FSTATFS                        = 158
+	SYS_UNMOUNT                        = 159
+	SYS_GETFH                          = 161
+	SYS_QUOTACTL                       = 165
+	SYS_MOUNT                          = 167
+	SYS_CSOPS                          = 169
+	SYS_CSOPS_AUDITTOKEN               = 170
+	SYS_WAITID                         = 173
+	SYS_KDEBUG_TYPEFILTER              = 177
+	SYS_KDEBUG_TRACE_STRING            = 178
+	SYS_KDEBUG_TRACE64                 = 179
+	SYS_KDEBUG_TRACE                   = 180
+	SYS_SETGID                         = 181
+	SYS_SETEGID                        = 182
+	SYS_SETEUID                        = 183
+	SYS_SIGRETURN                      = 184
+	SYS_THREAD_SELFCOUNTS              = 186
+	SYS_FDATASYNC                      = 187
+	SYS_STAT                           = 188
+	SYS_FSTAT                          = 189
+	SYS_LSTAT                          = 190
+	SYS_PATHCONF                       = 191
+	SYS_FPATHCONF                      = 192
+	SYS_GETRLIMIT                      = 194
+	SYS_SETRLIMIT                      = 195
+	SYS_GETDIRENTRIES                  = 196
+	SYS_MMAP                           = 197
+	SYS_LSEEK                          = 199
+	SYS_TRUNCATE                       = 200
+	SYS_FTRUNCATE                      = 201
+	SYS_SYSCTL                         = 202
+	SYS_MLOCK                          = 203
+	SYS_MUNLOCK                        = 204
+	SYS_UNDELETE                       = 205
+	SYS_OPEN_DPROTECTED_NP             = 216
+	SYS_GETATTRLIST                    = 220
+	SYS_SETATTRLIST                    = 221
+	SYS_GETDIRENTRIESATTR              = 222
+	SYS_EXCHANGEDATA                   = 223
+	SYS_SEARCHFS                       = 225
+	SYS_DELETE                         = 226
+	SYS_COPYFILE                       = 227
+	SYS_FGETATTRLIST                   = 228
+	SYS_FSETATTRLIST                   = 229
+	SYS_POLL                           = 230
+	SYS_WATCHEVENT                     = 231
+	SYS_WAITEVENT                      = 232
+	SYS_MODWATCH                       = 233
+	SYS_GETXATTR                       = 234
+	SYS_FGETXATTR                      = 235
+	SYS_SETXATTR                       = 236
+	SYS_FSETXATTR                      = 237
+	SYS_REMOVEXATTR                    = 238
+	SYS_FREMOVEXATTR                   = 239
+	SYS_LISTXATTR                      = 240
+	SYS_FLISTXATTR                     = 241
+	SYS_FSCTL                          = 242
+	SYS_INITGROUPS                     = 243
+	SYS_POSIX_SPAWN                    = 244
+	SYS_FFSCTL                         = 245
+	SYS_NFSCLNT                        = 247
+	SYS_FHOPEN                         = 248
+	SYS_MINHERIT                       = 250
+	SYS_SEMSYS                         = 251
+	SYS_MSGSYS                         = 252
+	SYS_SHMSYS                         = 253
+	SYS_SEMCTL                         = 254
+	SYS_SEMGET                         = 255
+	SYS_SEMOP                          = 256
+	SYS_MSGCTL                         = 258
+	SYS_MSGGET                         = 259
+	SYS_MSGSND                         = 260
+	SYS_MSGRCV                         = 261
+	SYS_SHMAT                          = 262
+	SYS_SHMCTL                         = 263
+	SYS_SHMDT                          = 264
+	SYS_SHMGET                         = 265
+	SYS_SHM_OPEN                       = 266
+	SYS_SHM_UNLINK                     = 267
+	SYS_SEM_OPEN                       = 268
+	SYS_SEM_CLOSE                      = 269
+	SYS_SEM_UNLINK                     = 270
+	SYS_SEM_WAIT                       = 271
+	SYS_SEM_TRYWAIT                    = 272
+	SYS_SEM_POST                       = 273
+	SYS_SYSCTLBYNAME                   = 274
+	SYS_OPEN_EXTENDED                  = 277
+	SYS_UMASK_EXTENDED                 = 278
+	SYS_STAT_EXTENDED                  = 279
+	SYS_LSTAT_EXTENDED                 = 280
+	SYS_FSTAT_EXTENDED                 = 281
+	SYS_CHMOD_EXTENDED                 = 282
+	SYS_FCHMOD_EXTENDED                = 283
+	SYS_ACCESS_EXTENDED                = 284
+	SYS_SETTID                         = 285
+	SYS_GETTID                         = 286
+	SYS_SETSGROUPS                     = 287
+	SYS_GETSGROUPS                     = 288
+	SYS_SETWGROUPS                     = 289
+	SYS_GETWGROUPS                     = 290
+	SYS_MKFIFO_EXTENDED                = 291
+	SYS_MKDIR_EXTENDED                 = 292
+	SYS_IDENTITYSVC                    = 293
+	SYS_SHARED_REGION_CHECK_NP         = 294
+	SYS_VM_PRESSURE_MONITOR            = 296
+	SYS_PSYNCH_RW_LONGRDLOCK           = 297
+	SYS_PSYNCH_RW_YIELDWRLOCK          = 298
+	SYS_PSYNCH_RW_DOWNGRADE            = 299
+	SYS_PSYNCH_RW_UPGRADE              = 300
+	SYS_PSYNCH_MUTEXWAIT               = 301
+	SYS_PSYNCH_MUTEXDROP               = 302
+	SYS_PSYNCH_CVBROAD                 = 303
+	SYS_PSYNCH_CVSIGNAL                = 304
+	SYS_PSYNCH_CVWAIT                  = 305
+	SYS_PSYNCH_RW_RDLOCK               = 306
+	SYS_PSYNCH_RW_WRLOCK               = 307
+	SYS_PSYNCH_RW_UNLOCK               = 308
+	SYS_PSYNCH_RW_UNLOCK2              = 309
+	SYS_GETSID                         = 310
+	SYS_SETTID_WITH_PID                = 311
+	SYS_PSYNCH_CVCLRPREPOST            = 312
+	SYS_AIO_FSYNC                      = 313
+	SYS_AIO_RETURN                     = 314
+	SYS_AIO_SUSPEND                    = 315
+	SYS_AIO_CANCEL                     = 316
+	SYS_AIO_ERROR                      = 317
+	SYS_AIO_READ                       = 318
+	SYS_AIO_WRITE                      = 319
+	SYS_LIO_LISTIO                     = 320
+	SYS_IOPOLICYSYS                    = 322
+	SYS_PROCESS_POLICY                 = 323
+	SYS_MLOCKALL                       = 324
+	SYS_MUNLOCKALL                     = 325
+	SYS_ISSETUGID                      = 327
+	SYS___PTHREAD_KILL                 = 328
+	SYS___PTHREAD_SIGMASK              = 329
+	SYS___SIGWAIT                      = 330
+	SYS___DISABLE_THREADSIGNAL         = 331
+	SYS___PTHREAD_MARKCANCEL           = 332
+	SYS___PTHREAD_CANCELED             = 333
+	SYS___SEMWAIT_SIGNAL               = 334
+	SYS_PROC_INFO                      = 336
+	SYS_SENDFILE                       = 337
+	SYS_STAT64                         = 338
+	SYS_FSTAT64                        = 339
+	SYS_LSTAT64                        = 340
+	SYS_STAT64_EXTENDED                = 341
+	SYS_LSTAT64_EXTENDED               = 342
+	SYS_FSTAT64_EXTENDED               = 343
+	SYS_GETDIRENTRIES64                = 344
+	SYS_STATFS64                       = 345
+	SYS_FSTATFS64                      = 346
+	SYS_GETFSSTAT64                    = 347
+	SYS___PTHREAD_CHDIR                = 348
+	SYS___PTHREAD_FCHDIR               = 349
+	SYS_AUDIT                          = 350
+	SYS_AUDITON                        = 351
+	SYS_GETAUID                        = 353
+	SYS_SETAUID                        = 354
+	SYS_GETAUDIT_ADDR                  = 357
+	SYS_SETAUDIT_ADDR                  = 358
+	SYS_AUDITCTL                       = 359
+	SYS_BSDTHREAD_CREATE               = 360
+	SYS_BSDTHREAD_TERMINATE            = 361
+	SYS_KQUEUE                         = 362
+	SYS_KEVENT                         = 363
+	SYS_LCHOWN                         = 364
+	SYS_BSDTHREAD_REGISTER             = 366
+	SYS_WORKQ_OPEN                     = 367
+	SYS_WORKQ_KERNRETURN               = 368
+	SYS_KEVENT64                       = 369
+	SYS___OLD_SEMWAIT_SIGNAL           = 370
+	SYS___OLD_SEMWAIT_SIGNAL_NOCANCEL  = 371
+	SYS_THREAD_SELFID                  = 372
+	SYS_LEDGER                         = 373
+	SYS_KEVENT_QOS                     = 374
+	SYS_KEVENT_ID                      = 375
+	SYS___MAC_EXECVE                   = 380
+	SYS___MAC_SYSCALL                  = 381
+	SYS___MAC_GET_FILE                 = 382
+	SYS___MAC_SET_FILE                 = 383
+	SYS___MAC_GET_LINK                 = 384
+	SYS___MAC_SET_LINK                 = 385
+	SYS___MAC_GET_PROC                 = 386
+	SYS___MAC_SET_PROC                 = 387
+	SYS___MAC_GET_FD                   = 388
+	SYS___MAC_SET_FD                   = 389
+	SYS___MAC_GET_PID                  = 390
+	SYS_PSELECT                        = 394
+	SYS_PSELECT_NOCANCEL               = 395
+	SYS_READ_NOCANCEL                  = 396
+	SYS_WRITE_NOCANCEL                 = 397
+	SYS_OPEN_NOCANCEL                  = 398
+	SYS_CLOSE_NOCANCEL                 = 399
+	SYS_WAIT4_NOCANCEL                 = 400
+	SYS_RECVMSG_NOCANCEL               = 401
+	SYS_SENDMSG_NOCANCEL               = 402
+	SYS_RECVFROM_NOCANCEL              = 403
+	SYS_ACCEPT_NOCANCEL                = 404
+	SYS_MSYNC_NOCANCEL                 = 405
+	SYS_FCNTL_NOCANCEL                 = 406
+	SYS_SELECT_NOCANCEL                = 407
+	SYS_FSYNC_NOCANCEL                 = 408
+	SYS_CONNECT_NOCANCEL               = 409
+	SYS_SIGSUSPEND_NOCANCEL            = 410
+	SYS_READV_NOCANCEL                 = 411
+	SYS_WRITEV_NOCANCEL                = 412
+	SYS_SENDTO_NOCANCEL                = 413
+	SYS_PREAD_NOCANCEL                 = 414
+	SYS_PWRITE_NOCANCEL                = 415
+	SYS_WAITID_NOCANCEL                = 416
+	SYS_POLL_NOCANCEL                  = 417
+	SYS_MSGSND_NOCANCEL                = 418
+	SYS_MSGRCV_NOCANCEL                = 419
+	SYS_SEM_WAIT_NOCANCEL              = 420
+	SYS_AIO_SUSPEND_NOCANCEL           = 421
+	SYS___SIGWAIT_NOCANCEL             = 422
+	SYS___SEMWAIT_SIGNAL_NOCANCEL      = 423
+	SYS___MAC_MOUNT                    = 424
+	SYS___MAC_GET_MOUNT                = 425
+	SYS___MAC_GETFSSTAT                = 426
+	SYS_FSGETPATH                      = 427
+	SYS_AUDIT_SESSION_SELF             = 428
+	SYS_AUDIT_SESSION_JOIN             = 429
+	SYS_FILEPORT_MAKEPORT              = 430
+	SYS_FILEPORT_MAKEFD                = 431
+	SYS_AUDIT_SESSION_PORT             = 432
+	SYS_PID_SUSPEND                    = 433
+	SYS_PID_RESUME                     = 434
+	SYS_PID_HIBERNATE                  = 435
+	SYS_PID_SHUTDOWN_SOCKETS           = 436
+	SYS_SHARED_REGION_MAP_AND_SLIDE_NP = 438
+	SYS_KAS_INFO                       = 439
+	SYS_MEMORYSTATUS_CONTROL           = 440
+	SYS_GUARDED_OPEN_NP                = 441
+	SYS_GUARDED_CLOSE_NP               = 442
+	SYS_GUARDED_KQUEUE_NP              = 443
+	SYS_CHANGE_FDGUARD_NP              = 444
+	SYS_USRCTL                         = 445
+	SYS_PROC_RLIMIT_CONTROL            = 446
+	SYS_CONNECTX                       = 447
+	SYS_DISCONNECTX                    = 448
+	SYS_PEELOFF                        = 449
+	SYS_SOCKET_DELEGATE                = 450
+	SYS_TELEMETRY                      = 451
+	SYS_PROC_UUID_POLICY               = 452
+	SYS_MEMORYSTATUS_GET_LEVEL         = 453
+	SYS_SYSTEM_OVERRIDE                = 454
+	SYS_VFS_PURGE                      = 455
+	SYS_SFI_CTL                        = 456
+	SYS_SFI_PIDCTL                     = 457
+	SYS_COALITION                      = 458
+	SYS_COALITION_INFO                 = 459
+	SYS_NECP_MATCH_POLICY              = 460
+	SYS_GETATTRLISTBULK                = 461
+	SYS_CLONEFILEAT                    = 462
+	SYS_OPENAT                         = 463
+	SYS_OPENAT_NOCANCEL                = 464
+	SYS_RENAMEAT                       = 465
+	SYS_FACCESSAT                      = 466
+	SYS_FCHMODAT                       = 467
+	SYS_FCHOWNAT                       = 468
+	SYS_FSTATAT                        = 469
+	SYS_FSTATAT64                      = 470
+	SYS_LINKAT                         = 471
+	SYS_UNLINKAT                       = 472
+	SYS_READLINKAT                     = 473
+	SYS_SYMLINKAT                      = 474
+	SYS_MKDIRAT                        = 475
+	SYS_GETATTRLISTAT                  = 476
+	SYS_PROC_TRACE_LOG                 = 477
+	SYS_BSDTHREAD_CTL                  = 478
+	SYS_OPENBYID_NP                    = 479
+	SYS_RECVMSG_X                      = 480
+	SYS_SENDMSG_X                      = 481
+	SYS_THREAD_SELFUSAGE               = 482
+	SYS_CSRCTL                         = 483
+	SYS_GUARDED_OPEN_DPROTECTED_NP     = 484
+	SYS_GUARDED_WRITE_NP               = 485
+	SYS_GUARDED_PWRITE_NP              = 486
+	SYS_GUARDED_WRITEV_NP              = 487
+	SYS_RENAMEATX_NP                   = 488
+	SYS_MREMAP_ENCRYPTED               = 489
+	SYS_NETAGENT_TRIGGER               = 490
+	SYS_STACK_SNAPSHOT_WITH_CONFIG     = 491
+	SYS_MICROSTACKSHOT                 = 492
+	SYS_GRAB_PGO_DATA                  = 493
+	SYS_PERSONA                        = 494
+	SYS_WORK_INTERVAL_CTL              = 499
+	SYS_GETENTROPY                     = 500
+	SYS_NECP_OPEN                      = 501
+	SYS_NECP_CLIENT_ACTION             = 502
+	SYS___NEXUS_OPEN                   = 503
+	SYS___NEXUS_REGISTER               = 504
+	SYS___NEXUS_DEREGISTER             = 505
+	SYS___NEXUS_CREATE                 = 506
+	SYS___NEXUS_DESTROY                = 507
+	SYS___NEXUS_GET_OPT                = 508
+	SYS___NEXUS_SET_OPT                = 509
+	SYS___CHANNEL_OPEN                 = 510
+	SYS___CHANNEL_GET_INFO             = 511
+	SYS___CHANNEL_SYNC                 = 512
+	SYS___CHANNEL_GET_OPT              = 513
+	SYS___CHANNEL_SET_OPT              = 514
+	SYS_ULOCK_WAIT                     = 515
+	SYS_ULOCK_WAKE                     = 516
+	SYS_FCLONEFILEAT                   = 517
+	SYS_FS_SNAPSHOT                    = 518
+	SYS_TERMINATE_WITH_PAYLOAD         = 520
+	SYS_ABORT_WITH_PAYLOAD             = 521
+	SYS_NECP_SESSION_OPEN              = 522
+	SYS_NECP_SESSION_ACTION            = 523
+	SYS_SETATTRLISTAT                  = 524
+	SYS_NET_QOS_GUIDELINE              = 525
+	SYS_FMOUNT                         = 526
+	SYS_NTP_ADJTIME                    = 527
+	SYS_NTP_GETTIME                    = 528
+	SYS_OS_FAULT_WITH_PAYLOAD          = 529
+	SYS_MAXSYSCALL                     = 530
+	SYS_INVALID                        = 63
+)
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_dragonfly_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_dragonfly_amd64.go
index 464c9a9..703675c 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_dragonfly_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_dragonfly_amd64.go
@@ -1,134 +1,131 @@
 // go run mksysnum.go https://gitweb.dragonflybsd.org/dragonfly.git/blob_plain/HEAD:/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && dragonfly
 // +build amd64,dragonfly
 
 package unix
 
 const (
-	// SYS_NOSYS = 0;  // { int nosys(void); } syscall nosys_args int
-	SYS_EXIT          = 1   // { void exit(int rval); }
-	SYS_FORK          = 2   // { int fork(void); }
-	SYS_READ          = 3   // { ssize_t read(int fd, void *buf, size_t nbyte); }
-	SYS_WRITE         = 4   // { ssize_t write(int fd, const void *buf, size_t nbyte); }
-	SYS_OPEN          = 5   // { int open(char *path, int flags, int mode); }
-	SYS_CLOSE         = 6   // { int close(int fd); }
-	SYS_WAIT4         = 7   // { int wait4(int pid, int *status, int options, struct rusage *rusage); } wait4 wait_args int
-	SYS_LINK          = 9   // { int link(char *path, char *link); }
-	SYS_UNLINK        = 10  // { int unlink(char *path); }
-	SYS_CHDIR         = 12  // { int chdir(char *path); }
-	SYS_FCHDIR        = 13  // { int fchdir(int fd); }
-	SYS_MKNOD         = 14  // { int mknod(char *path, int mode, int dev); }
-	SYS_CHMOD         = 15  // { int chmod(char *path, int mode); }
-	SYS_CHOWN         = 16  // { int chown(char *path, int uid, int gid); }
-	SYS_OBREAK        = 17  // { int obreak(char *nsize); } break obreak_args int
-	SYS_GETFSSTAT     = 18  // { int getfsstat(struct statfs *buf, long bufsize, int flags); }
-	SYS_GETPID        = 20  // { pid_t getpid(void); }
-	SYS_MOUNT         = 21  // { int mount(char *type, char *path, int flags, caddr_t data); }
-	SYS_UNMOUNT       = 22  // { int unmount(char *path, int flags); }
-	SYS_SETUID        = 23  // { int setuid(uid_t uid); }
-	SYS_GETUID        = 24  // { uid_t getuid(void); }
-	SYS_GETEUID       = 25  // { uid_t geteuid(void); }
-	SYS_PTRACE        = 26  // { int ptrace(int req, pid_t pid, caddr_t addr, int data); }
-	SYS_RECVMSG       = 27  // { int recvmsg(int s, struct msghdr *msg, int flags); }
-	SYS_SENDMSG       = 28  // { int sendmsg(int s, caddr_t msg, int flags); }
-	SYS_RECVFROM      = 29  // { int recvfrom(int s, caddr_t buf, size_t len, int flags, caddr_t from, int *fromlenaddr); }
-	SYS_ACCEPT        = 30  // { int accept(int s, caddr_t name, int *anamelen); }
-	SYS_GETPEERNAME   = 31  // { int getpeername(int fdes, caddr_t asa, int *alen); }
-	SYS_GETSOCKNAME   = 32  // { int getsockname(int fdes, caddr_t asa, int *alen); }
-	SYS_ACCESS        = 33  // { int access(char *path, int flags); }
-	SYS_CHFLAGS       = 34  // { int chflags(char *path, int flags); }
-	SYS_FCHFLAGS      = 35  // { int fchflags(int fd, int flags); }
-	SYS_SYNC          = 36  // { int sync(void); }
-	SYS_KILL          = 37  // { int kill(int pid, int signum); }
-	SYS_GETPPID       = 39  // { pid_t getppid(void); }
-	SYS_DUP           = 41  // { int dup(int fd); }
-	SYS_PIPE          = 42  // { int pipe(void); }
-	SYS_GETEGID       = 43  // { gid_t getegid(void); }
-	SYS_PROFIL        = 44  // { int profil(caddr_t samples, size_t size, size_t offset, u_int scale); }
-	SYS_KTRACE        = 45  // { int ktrace(const char *fname, int ops, int facs, int pid); }
-	SYS_GETGID        = 47  // { gid_t getgid(void); }
-	SYS_GETLOGIN      = 49  // { int getlogin(char *namebuf, u_int namelen); }
-	SYS_SETLOGIN      = 50  // { int setlogin(char *namebuf); }
-	SYS_ACCT          = 51  // { int acct(char *path); }
-	SYS_SIGALTSTACK   = 53  // { int sigaltstack(stack_t *ss, stack_t *oss); }
-	SYS_IOCTL         = 54  // { int ioctl(int fd, u_long com, caddr_t data); }
-	SYS_REBOOT        = 55  // { int reboot(int opt); }
-	SYS_REVOKE        = 56  // { int revoke(char *path); }
-	SYS_SYMLINK       = 57  // { int symlink(char *path, char *link); }
-	SYS_READLINK      = 58  // { int readlink(char *path, char *buf, int count); }
-	SYS_EXECVE        = 59  // { int execve(char *fname, char **argv, char **envv); }
-	SYS_UMASK         = 60  // { int umask(int newmask); } umask umask_args int
-	SYS_CHROOT        = 61  // { int chroot(char *path); }
-	SYS_MSYNC         = 65  // { int msync(void *addr, size_t len, int flags); }
-	SYS_VFORK         = 66  // { pid_t vfork(void); }
-	SYS_SBRK          = 69  // { int sbrk(int incr); }
-	SYS_SSTK          = 70  // { int sstk(int incr); }
-	SYS_MUNMAP        = 73  // { int munmap(void *addr, size_t len); }
-	SYS_MPROTECT      = 74  // { int mprotect(void *addr, size_t len, int prot); }
-	SYS_MADVISE       = 75  // { int madvise(void *addr, size_t len, int behav); }
-	SYS_MINCORE       = 78  // { int mincore(const void *addr, size_t len, char *vec); }
-	SYS_GETGROUPS     = 79  // { int getgroups(u_int gidsetsize, gid_t *gidset); }
-	SYS_SETGROUPS     = 80  // { int setgroups(u_int gidsetsize, gid_t *gidset); }
-	SYS_GETPGRP       = 81  // { int getpgrp(void); }
-	SYS_SETPGID       = 82  // { int setpgid(int pid, int pgid); }
-	SYS_SETITIMER     = 83  // { int setitimer(u_int which, struct itimerval *itv, struct itimerval *oitv); }
-	SYS_SWAPON        = 85  // { int swapon(char *name); }
-	SYS_GETITIMER     = 86  // { int getitimer(u_int which, struct itimerval *itv); }
-	SYS_GETDTABLESIZE = 89  // { int getdtablesize(void); }
-	SYS_DUP2          = 90  // { int dup2(int from, int to); }
-	SYS_FCNTL         = 92  // { int fcntl(int fd, int cmd, long arg); }
-	SYS_SELECT        = 93  // { int select(int nd, fd_set *in, fd_set *ou, fd_set *ex, struct timeval *tv); }
-	SYS_FSYNC         = 95  // { int fsync(int fd); }
-	SYS_SETPRIORITY   = 96  // { int setpriority(int which, int who, int prio); }
-	SYS_SOCKET        = 97  // { int socket(int domain, int type, int protocol); }
-	SYS_CONNECT       = 98  // { int connect(int s, caddr_t name, int namelen); }
-	SYS_GETPRIORITY   = 100 // { int getpriority(int which, int who); }
-	SYS_BIND          = 104 // { int bind(int s, caddr_t name, int namelen); }
-	SYS_SETSOCKOPT    = 105 // { int setsockopt(int s, int level, int name, caddr_t val, int valsize); }
-	SYS_LISTEN        = 106 // { int listen(int s, int backlog); }
-	SYS_GETTIMEOFDAY  = 116 // { int gettimeofday(struct timeval *tp, struct timezone *tzp); }
-	SYS_GETRUSAGE     = 117 // { int getrusage(int who, struct rusage *rusage); }
-	SYS_GETSOCKOPT    = 118 // { int getsockopt(int s, int level, int name, caddr_t val, int *avalsize); }
-	SYS_READV         = 120 // { int readv(int fd, struct iovec *iovp, u_int iovcnt); }
-	SYS_WRITEV        = 121 // { int writev(int fd, struct iovec *iovp, u_int iovcnt); }
-	SYS_SETTIMEOFDAY  = 122 // { int settimeofday(struct timeval *tv, struct timezone *tzp); }
-	SYS_FCHOWN        = 123 // { int fchown(int fd, int uid, int gid); }
-	SYS_FCHMOD        = 124 // { int fchmod(int fd, int mode); }
-	SYS_SETREUID      = 126 // { int setreuid(int ruid, int euid); }
-	SYS_SETREGID      = 127 // { int setregid(int rgid, int egid); }
-	SYS_RENAME        = 128 // { int rename(char *from, char *to); }
-	SYS_FLOCK         = 131 // { int flock(int fd, int how); }
-	SYS_MKFIFO        = 132 // { int mkfifo(char *path, int mode); }
-	SYS_SENDTO        = 133 // { int sendto(int s, caddr_t buf, size_t len, int flags, caddr_t to, int tolen); }
-	SYS_SHUTDOWN      = 134 // { int shutdown(int s, int how); }
-	SYS_SOCKETPAIR    = 135 // { int socketpair(int domain, int type, int protocol, int *rsv); }
-	SYS_MKDIR         = 136 // { int mkdir(char *path, int mode); }
-	SYS_RMDIR         = 137 // { int rmdir(char *path); }
-	SYS_UTIMES        = 138 // { int utimes(char *path, struct timeval *tptr); }
-	SYS_ADJTIME       = 140 // { int adjtime(struct timeval *delta, struct timeval *olddelta); }
-	SYS_SETSID        = 147 // { int setsid(void); }
-	SYS_QUOTACTL      = 148 // { int quotactl(char *path, int cmd, int uid, caddr_t arg); }
-	SYS_STATFS        = 157 // { int statfs(char *path, struct statfs *buf); }
-	SYS_FSTATFS       = 158 // { int fstatfs(int fd, struct statfs *buf); }
-	SYS_GETFH         = 161 // { int getfh(char *fname, struct fhandle *fhp); }
-	SYS_GETDOMAINNAME = 162 // { int getdomainname(char *domainname, int len); }
-	SYS_SETDOMAINNAME = 163 // { int setdomainname(char *domainname, int len); }
-	SYS_UNAME         = 164 // { int uname(struct utsname *name); }
-	SYS_SYSARCH       = 165 // { int sysarch(int op, char *parms); }
-	SYS_RTPRIO        = 166 // { int rtprio(int function, pid_t pid, struct rtprio *rtp); }
-	SYS_EXTPREAD      = 173 // { ssize_t extpread(int fd, void *buf, size_t nbyte, int flags, off_t offset); }
-	SYS_EXTPWRITE     = 174 // { ssize_t extpwrite(int fd, const void *buf, size_t nbyte, int flags, off_t offset); }
-	SYS_NTP_ADJTIME   = 176 // { int ntp_adjtime(struct timex *tp); }
-	SYS_SETGID        = 181 // { int setgid(gid_t gid); }
-	SYS_SETEGID       = 182 // { int setegid(gid_t egid); }
-	SYS_SETEUID       = 183 // { int seteuid(uid_t euid); }
-	SYS_PATHCONF      = 191 // { int pathconf(char *path, int name); }
-	SYS_FPATHCONF     = 192 // { int fpathconf(int fd, int name); }
-	SYS_GETRLIMIT     = 194 // { int getrlimit(u_int which, struct rlimit *rlp); } getrlimit __getrlimit_args int
-	SYS_SETRLIMIT     = 195 // { int setrlimit(u_int which, struct rlimit *rlp); } setrlimit __setrlimit_args int
-	SYS_MMAP          = 197 // { caddr_t mmap(caddr_t addr, size_t len, int prot, int flags, int fd, int pad, off_t pos); }
-	// SYS_NOSYS = 198;  // { int nosys(void); } __syscall __syscall_args int
+	SYS_EXIT  = 1 // { void exit(int rval); }
+	SYS_FORK  = 2 // { int fork(void); }
+	SYS_READ  = 3 // { ssize_t read(int fd, void *buf, size_t nbyte); }
+	SYS_WRITE = 4 // { ssize_t write(int fd, const void *buf, size_t nbyte); }
+	SYS_OPEN  = 5 // { int open(char *path, int flags, int mode); }
+	SYS_CLOSE = 6 // { int close(int fd); }
+	SYS_WAIT4 = 7 // { int wait4(int pid, int *status, int options, struct rusage *rusage); } wait4 wait_args int
+	// SYS_NOSYS = 8;  // { int nosys(void); } __nosys nosys_args int
+	SYS_LINK                   = 9   // { int link(char *path, char *link); }
+	SYS_UNLINK                 = 10  // { int unlink(char *path); }
+	SYS_CHDIR                  = 12  // { int chdir(char *path); }
+	SYS_FCHDIR                 = 13  // { int fchdir(int fd); }
+	SYS_MKNOD                  = 14  // { int mknod(char *path, int mode, int dev); }
+	SYS_CHMOD                  = 15  // { int chmod(char *path, int mode); }
+	SYS_CHOWN                  = 16  // { int chown(char *path, int uid, int gid); }
+	SYS_OBREAK                 = 17  // { int obreak(char *nsize); } break obreak_args int
+	SYS_GETFSSTAT              = 18  // { int getfsstat(struct statfs *buf, long bufsize, int flags); }
+	SYS_GETPID                 = 20  // { pid_t getpid(void); }
+	SYS_MOUNT                  = 21  // { int mount(char *type, char *path, int flags, caddr_t data); }
+	SYS_UNMOUNT                = 22  // { int unmount(char *path, int flags); }
+	SYS_SETUID                 = 23  // { int setuid(uid_t uid); }
+	SYS_GETUID                 = 24  // { uid_t getuid(void); }
+	SYS_GETEUID                = 25  // { uid_t geteuid(void); }
+	SYS_PTRACE                 = 26  // { int ptrace(int req, pid_t pid, caddr_t addr, int data); }
+	SYS_RECVMSG                = 27  // { int recvmsg(int s, struct msghdr *msg, int flags); }
+	SYS_SENDMSG                = 28  // { int sendmsg(int s, caddr_t msg, int flags); }
+	SYS_RECVFROM               = 29  // { int recvfrom(int s, caddr_t buf, size_t len, int flags, caddr_t from, int *fromlenaddr); }
+	SYS_ACCEPT                 = 30  // { int accept(int s, caddr_t name, int *anamelen); }
+	SYS_GETPEERNAME            = 31  // { int getpeername(int fdes, caddr_t asa, int *alen); }
+	SYS_GETSOCKNAME            = 32  // { int getsockname(int fdes, caddr_t asa, int *alen); }
+	SYS_ACCESS                 = 33  // { int access(char *path, int flags); }
+	SYS_CHFLAGS                = 34  // { int chflags(const char *path, u_long flags); }
+	SYS_FCHFLAGS               = 35  // { int fchflags(int fd, u_long flags); }
+	SYS_SYNC                   = 36  // { int sync(void); }
+	SYS_KILL                   = 37  // { int kill(int pid, int signum); }
+	SYS_GETPPID                = 39  // { pid_t getppid(void); }
+	SYS_DUP                    = 41  // { int dup(int fd); }
+	SYS_PIPE                   = 42  // { int pipe(void); }
+	SYS_GETEGID                = 43  // { gid_t getegid(void); }
+	SYS_PROFIL                 = 44  // { int profil(caddr_t samples, size_t size, u_long offset, u_int scale); }
+	SYS_KTRACE                 = 45  // { int ktrace(const char *fname, int ops, int facs, int pid); }
+	SYS_GETGID                 = 47  // { gid_t getgid(void); }
+	SYS_GETLOGIN               = 49  // { int getlogin(char *namebuf, size_t namelen); }
+	SYS_SETLOGIN               = 50  // { int setlogin(char *namebuf); }
+	SYS_ACCT                   = 51  // { int acct(char *path); }
+	SYS_SIGALTSTACK            = 53  // { int sigaltstack(stack_t *ss, stack_t *oss); }
+	SYS_IOCTL                  = 54  // { int ioctl(int fd, u_long com, caddr_t data); }
+	SYS_REBOOT                 = 55  // { int reboot(int opt); }
+	SYS_REVOKE                 = 56  // { int revoke(char *path); }
+	SYS_SYMLINK                = 57  // { int symlink(char *path, char *link); }
+	SYS_READLINK               = 58  // { int readlink(char *path, char *buf, int count); }
+	SYS_EXECVE                 = 59  // { int execve(char *fname, char **argv, char **envv); }
+	SYS_UMASK                  = 60  // { int umask(int newmask); } umask umask_args int
+	SYS_CHROOT                 = 61  // { int chroot(char *path); }
+	SYS_MSYNC                  = 65  // { int msync(void *addr, size_t len, int flags); }
+	SYS_VFORK                  = 66  // { pid_t vfork(void); }
+	SYS_SBRK                   = 69  // { caddr_t sbrk(size_t incr); }
+	SYS_SSTK                   = 70  // { int sstk(size_t incr); }
+	SYS_MUNMAP                 = 73  // { int munmap(void *addr, size_t len); }
+	SYS_MPROTECT               = 74  // { int mprotect(void *addr, size_t len, int prot); }
+	SYS_MADVISE                = 75  // { int madvise(void *addr, size_t len, int behav); }
+	SYS_MINCORE                = 78  // { int mincore(const void *addr, size_t len, char *vec); }
+	SYS_GETGROUPS              = 79  // { int getgroups(u_int gidsetsize, gid_t *gidset); }
+	SYS_SETGROUPS              = 80  // { int setgroups(u_int gidsetsize, gid_t *gidset); }
+	SYS_GETPGRP                = 81  // { int getpgrp(void); }
+	SYS_SETPGID                = 82  // { int setpgid(int pid, int pgid); }
+	SYS_SETITIMER              = 83  // { int setitimer(u_int which, struct itimerval *itv, struct itimerval *oitv); }
+	SYS_SWAPON                 = 85  // { int swapon(char *name); }
+	SYS_GETITIMER              = 86  // { int getitimer(u_int which, struct itimerval *itv); }
+	SYS_GETDTABLESIZE          = 89  // { int getdtablesize(void); }
+	SYS_DUP2                   = 90  // { int dup2(int from, int to); }
+	SYS_FCNTL                  = 92  // { int fcntl(int fd, int cmd, long arg); }
+	SYS_SELECT                 = 93  // { int select(int nd, fd_set *in, fd_set *ou, fd_set *ex, struct timeval *tv); }
+	SYS_FSYNC                  = 95  // { int fsync(int fd); }
+	SYS_SETPRIORITY            = 96  // { int setpriority(int which, int who, int prio); }
+	SYS_SOCKET                 = 97  // { int socket(int domain, int type, int protocol); }
+	SYS_CONNECT                = 98  // { int connect(int s, caddr_t name, int namelen); }
+	SYS_GETPRIORITY            = 100 // { int getpriority(int which, int who); }
+	SYS_BIND                   = 104 // { int bind(int s, caddr_t name, int namelen); }
+	SYS_SETSOCKOPT             = 105 // { int setsockopt(int s, int level, int name, caddr_t val, int valsize); }
+	SYS_LISTEN                 = 106 // { int listen(int s, int backlog); }
+	SYS_GETTIMEOFDAY           = 116 // { int gettimeofday(struct timeval *tp, struct timezone *tzp); }
+	SYS_GETRUSAGE              = 117 // { int getrusage(int who, struct rusage *rusage); }
+	SYS_GETSOCKOPT             = 118 // { int getsockopt(int s, int level, int name, caddr_t val, int *avalsize); }
+	SYS_READV                  = 120 // { int readv(int fd, struct iovec *iovp, u_int iovcnt); }
+	SYS_WRITEV                 = 121 // { int writev(int fd, struct iovec *iovp, u_int iovcnt); }
+	SYS_SETTIMEOFDAY           = 122 // { int settimeofday(struct timeval *tv, struct timezone *tzp); }
+	SYS_FCHOWN                 = 123 // { int fchown(int fd, int uid, int gid); }
+	SYS_FCHMOD                 = 124 // { int fchmod(int fd, int mode); }
+	SYS_SETREUID               = 126 // { int setreuid(int ruid, int euid); }
+	SYS_SETREGID               = 127 // { int setregid(int rgid, int egid); }
+	SYS_RENAME                 = 128 // { int rename(char *from, char *to); }
+	SYS_FLOCK                  = 131 // { int flock(int fd, int how); }
+	SYS_MKFIFO                 = 132 // { int mkfifo(char *path, int mode); }
+	SYS_SENDTO                 = 133 // { int sendto(int s, caddr_t buf, size_t len, int flags, caddr_t to, int tolen); }
+	SYS_SHUTDOWN               = 134 // { int shutdown(int s, int how); }
+	SYS_SOCKETPAIR             = 135 // { int socketpair(int domain, int type, int protocol, int *rsv); }
+	SYS_MKDIR                  = 136 // { int mkdir(char *path, int mode); }
+	SYS_RMDIR                  = 137 // { int rmdir(char *path); }
+	SYS_UTIMES                 = 138 // { int utimes(char *path, struct timeval *tptr); }
+	SYS_ADJTIME                = 140 // { int adjtime(struct timeval *delta, struct timeval *olddelta); }
+	SYS_SETSID                 = 147 // { int setsid(void); }
+	SYS_QUOTACTL               = 148 // { int quotactl(char *path, int cmd, int uid, caddr_t arg); }
+	SYS_STATFS                 = 157 // { int statfs(char *path, struct statfs *buf); }
+	SYS_FSTATFS                = 158 // { int fstatfs(int fd, struct statfs *buf); }
+	SYS_GETFH                  = 161 // { int getfh(char *fname, struct fhandle *fhp); }
+	SYS_SYSARCH                = 165 // { int sysarch(int op, char *parms); }
+	SYS_RTPRIO                 = 166 // { int rtprio(int function, pid_t pid, struct rtprio *rtp); }
+	SYS_EXTPREAD               = 173 // { ssize_t extpread(int fd, void *buf, size_t nbyte, int flags, off_t offset); }
+	SYS_EXTPWRITE              = 174 // { ssize_t extpwrite(int fd, const void *buf, size_t nbyte, int flags, off_t offset); }
+	SYS_NTP_ADJTIME            = 176 // { int ntp_adjtime(struct timex *tp); }
+	SYS_SETGID                 = 181 // { int setgid(gid_t gid); }
+	SYS_SETEGID                = 182 // { int setegid(gid_t egid); }
+	SYS_SETEUID                = 183 // { int seteuid(uid_t euid); }
+	SYS_PATHCONF               = 191 // { int pathconf(char *path, int name); }
+	SYS_FPATHCONF              = 192 // { int fpathconf(int fd, int name); }
+	SYS_GETRLIMIT              = 194 // { int getrlimit(u_int which, struct rlimit *rlp); } getrlimit __getrlimit_args int
+	SYS_SETRLIMIT              = 195 // { int setrlimit(u_int which, struct rlimit *rlp); } setrlimit __setrlimit_args int
+	SYS_MMAP                   = 197 // { caddr_t mmap(caddr_t addr, size_t len, int prot, int flags, int fd, int pad, off_t pos); }
 	SYS_LSEEK                  = 199 // { off_t lseek(int fd, int pad, off_t offset, int whence); }
 	SYS_TRUNCATE               = 200 // { int truncate(char *path, int pad, off_t length); }
 	SYS_FTRUNCATE              = 201 // { int ftruncate(int fd, int pad, off_t length); }
@@ -161,8 +158,8 @@
 	SYS_LCHOWN                 = 254 // { int lchown(char *path, int uid, int gid); }
 	SYS_LCHMOD                 = 274 // { int lchmod(char *path, mode_t mode); }
 	SYS_LUTIMES                = 276 // { int lutimes(char *path, struct timeval *tptr); }
-	SYS_EXTPREADV              = 289 // { ssize_t extpreadv(int fd, struct iovec *iovp, u_int iovcnt, int flags, off_t offset); }
-	SYS_EXTPWRITEV             = 290 // { ssize_t extpwritev(int fd, struct iovec *iovp,u_int iovcnt, int flags, off_t offset); }
+	SYS_EXTPREADV              = 289 // { ssize_t extpreadv(int fd, const struct iovec *iovp, int iovcnt, int flags, off_t offset); }
+	SYS_EXTPWRITEV             = 290 // { ssize_t extpwritev(int fd, const struct iovec *iovp, int iovcnt, int flags, off_t offset); }
 	SYS_FHSTATFS               = 297 // { int fhstatfs(const struct fhandle *u_fhp, struct statfs *buf); }
 	SYS_FHOPEN                 = 298 // { int fhopen(const struct fhandle *u_fhp, int flags); }
 	SYS_MODNEXT                = 300 // { int modnext(int modid); }
@@ -225,7 +222,7 @@
 	SYS_KQUEUE                 = 362 // { int kqueue(void); }
 	SYS_KEVENT                 = 363 // { int kevent(int fd, const struct kevent *changelist, int nchanges, struct kevent *eventlist, int nevents, const struct timespec *timeout); }
 	SYS_KENV                   = 390 // { int kenv(int what, const char *name, char *value, int len); }
-	SYS_LCHFLAGS               = 391 // { int lchflags(char *path, int flags); }
+	SYS_LCHFLAGS               = 391 // { int lchflags(const char *path, u_long flags); }
 	SYS_UUIDGEN                = 392 // { int uuidgen(struct uuid *store, int count); }
 	SYS_SENDFILE               = 393 // { int sendfile(int fd, int s, off_t offset, size_t nbytes, struct sf_hdtr *hdtr, off_t *sbytes, int flags); }
 	SYS_VARSYM_SET             = 450 // { int varsym_set(int level, const char *name, const char *data); }
@@ -302,7 +299,7 @@
 	SYS_VMM_GUEST_CTL          = 534 // { int vmm_guest_ctl(int op, struct vmm_guest_options *options); }
 	SYS_VMM_GUEST_SYNC_ADDR    = 535 // { int vmm_guest_sync_addr(long *dstaddr, long *srcaddr); }
 	SYS_PROCCTL                = 536 // { int procctl(idtype_t idtype, id_t id, int cmd, void *data); }
-	SYS_CHFLAGSAT              = 537 // { int chflagsat(int fd, const char *path, int flags, int atflags);}
+	SYS_CHFLAGSAT              = 537 // { int chflagsat(int fd, const char *path, u_long flags, int atflags);}
 	SYS_PIPE2                  = 538 // { int pipe2(int *fildes, int flags); }
 	SYS_UTIMENSAT              = 539 // { int utimensat(int fd, const char *path, const struct timespec *ts, int flags); }
 	SYS_FUTIMENS               = 540 // { int futimens(int fd, const struct timespec *ts); }
@@ -312,4 +309,9 @@
 	SYS_LWP_SETAFFINITY        = 544 // { int lwp_setaffinity(pid_t pid, lwpid_t tid, const cpumask_t *mask); }
 	SYS_LWP_GETAFFINITY        = 545 // { int lwp_getaffinity(pid_t pid, lwpid_t tid, cpumask_t *mask); }
 	SYS_LWP_CREATE2            = 546 // { int lwp_create2(struct lwp_params *params, const cpumask_t *mask); }
+	SYS_GETCPUCLOCKID          = 547 // { int getcpuclockid(pid_t pid, lwpid_t lwp_id, clockid_t *clock_id); }
+	SYS_WAIT6                  = 548 // { int wait6(idtype_t idtype, id_t id, int *status, int options, struct __wrusage *wrusage, siginfo_t *info); }
+	SYS_LWP_GETNAME            = 549 // { int lwp_getname(lwpid_t tid, char *name, size_t len); }
+	SYS_GETRANDOM              = 550 // { ssize_t getrandom(void *buf, size_t len, unsigned flags); }
+	SYS___REALPATH             = 551 // { ssize_t __realpath(const char *path, char *buf, size_t len); }
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_386.go b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_386.go
index 9474974..59d5dfc 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_386.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://svn.freebsd.org/base/stable/11/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && freebsd
 // +build 386,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_amd64.go
index 48a7bea..342d471 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://svn.freebsd.org/base/stable/11/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && freebsd
 // +build amd64,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm.go b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm.go
index 4a6dfd4..e2e3d72 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://svn.freebsd.org/base/stable/11/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && freebsd
 // +build arm,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm64.go b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm64.go
index 3e51af8..61ad5ca 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_freebsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://svn.freebsd.org/base/stable/11/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && freebsd
 // +build arm64,freebsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_386.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_386.go
index a597e06..fbc59b7 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_386.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_386.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include -m32 /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && linux
 // +build 386,linux
 
 package unix
@@ -431,7 +432,11 @@
 	SYS_FSPICK                       = 433
 	SYS_PIDFD_OPEN                   = 434
 	SYS_CLONE3                       = 435
+	SYS_CLOSE_RANGE                  = 436
 	SYS_OPENAT2                      = 437
 	SYS_PIDFD_GETFD                  = 438
 	SYS_FACCESSAT2                   = 439
+	SYS_PROCESS_MADVISE              = 440
+	SYS_EPOLL_PWAIT2                 = 441
+	SYS_MOUNT_SETATTR                = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_amd64.go
index 8c102e5..04d16d7 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_amd64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include -m64 /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && linux
 // +build amd64,linux
 
 package unix
@@ -353,7 +354,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_arm.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_arm.go
index 98f9b68..3b1c105 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_arm.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && linux
 // +build arm,linux
 
 package unix
@@ -395,7 +396,11 @@
 	SYS_FSPICK                       = 433
 	SYS_PIDFD_OPEN                   = 434
 	SYS_CLONE3                       = 435
+	SYS_CLOSE_RANGE                  = 436
 	SYS_OPENAT2                      = 437
 	SYS_PIDFD_GETFD                  = 438
 	SYS_FACCESSAT2                   = 439
+	SYS_PROCESS_MADVISE              = 440
+	SYS_EPOLL_PWAIT2                 = 441
+	SYS_MOUNT_SETATTR                = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_arm64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_arm64.go
index 4dabc33..3198adc 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_arm64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include -fsigned-char /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && linux
 // +build arm64,linux
 
 package unix
@@ -298,7 +299,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips.go
index d5724e5..c877ec6 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips && linux
 // +build mips,linux
 
 package unix
@@ -416,7 +417,11 @@
 	SYS_FSPICK                       = 4433
 	SYS_PIDFD_OPEN                   = 4434
 	SYS_CLONE3                       = 4435
+	SYS_CLOSE_RANGE                  = 4436
 	SYS_OPENAT2                      = 4437
 	SYS_PIDFD_GETFD                  = 4438
 	SYS_FACCESSAT2                   = 4439
+	SYS_PROCESS_MADVISE              = 4440
+	SYS_EPOLL_PWAIT2                 = 4441
+	SYS_MOUNT_SETATTR                = 4442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64.go
index c1d824a..b5f2903 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && linux
 // +build mips64,linux
 
 package unix
@@ -346,7 +347,11 @@
 	SYS_FSPICK                 = 5433
 	SYS_PIDFD_OPEN             = 5434
 	SYS_CLONE3                 = 5435
+	SYS_CLOSE_RANGE            = 5436
 	SYS_OPENAT2                = 5437
 	SYS_PIDFD_GETFD            = 5438
 	SYS_FACCESSAT2             = 5439
+	SYS_PROCESS_MADVISE        = 5440
+	SYS_EPOLL_PWAIT2           = 5441
+	SYS_MOUNT_SETATTR          = 5442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64le.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64le.go
index 598dd5d..4607768 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64le.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_mips64le.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64le && linux
 // +build mips64le,linux
 
 package unix
@@ -346,7 +347,11 @@
 	SYS_FSPICK                 = 5433
 	SYS_PIDFD_OPEN             = 5434
 	SYS_CLONE3                 = 5435
+	SYS_CLOSE_RANGE            = 5436
 	SYS_OPENAT2                = 5437
 	SYS_PIDFD_GETFD            = 5438
 	SYS_FACCESSAT2             = 5439
+	SYS_PROCESS_MADVISE        = 5440
+	SYS_EPOLL_PWAIT2           = 5441
+	SYS_MOUNT_SETATTR          = 5442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_mipsle.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_mipsle.go
index c36782d..80e6696 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_mipsle.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_mipsle.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mipsle && linux
 // +build mipsle,linux
 
 package unix
@@ -416,7 +417,11 @@
 	SYS_FSPICK                       = 4433
 	SYS_PIDFD_OPEN                   = 4434
 	SYS_CLONE3                       = 4435
+	SYS_CLOSE_RANGE                  = 4436
 	SYS_OPENAT2                      = 4437
 	SYS_PIDFD_GETFD                  = 4438
 	SYS_FACCESSAT2                   = 4439
+	SYS_PROCESS_MADVISE              = 4440
+	SYS_EPOLL_PWAIT2                 = 4441
+	SYS_MOUNT_SETATTR                = 4442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc.go
new file mode 100644
index 0000000..b9d697f
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc.go
@@ -0,0 +1,434 @@
+// go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build ppc && linux
+// +build ppc,linux
+
+package unix
+
+const (
+	SYS_RESTART_SYSCALL              = 0
+	SYS_EXIT                         = 1
+	SYS_FORK                         = 2
+	SYS_READ                         = 3
+	SYS_WRITE                        = 4
+	SYS_OPEN                         = 5
+	SYS_CLOSE                        = 6
+	SYS_WAITPID                      = 7
+	SYS_CREAT                        = 8
+	SYS_LINK                         = 9
+	SYS_UNLINK                       = 10
+	SYS_EXECVE                       = 11
+	SYS_CHDIR                        = 12
+	SYS_TIME                         = 13
+	SYS_MKNOD                        = 14
+	SYS_CHMOD                        = 15
+	SYS_LCHOWN                       = 16
+	SYS_BREAK                        = 17
+	SYS_OLDSTAT                      = 18
+	SYS_LSEEK                        = 19
+	SYS_GETPID                       = 20
+	SYS_MOUNT                        = 21
+	SYS_UMOUNT                       = 22
+	SYS_SETUID                       = 23
+	SYS_GETUID                       = 24
+	SYS_STIME                        = 25
+	SYS_PTRACE                       = 26
+	SYS_ALARM                        = 27
+	SYS_OLDFSTAT                     = 28
+	SYS_PAUSE                        = 29
+	SYS_UTIME                        = 30
+	SYS_STTY                         = 31
+	SYS_GTTY                         = 32
+	SYS_ACCESS                       = 33
+	SYS_NICE                         = 34
+	SYS_FTIME                        = 35
+	SYS_SYNC                         = 36
+	SYS_KILL                         = 37
+	SYS_RENAME                       = 38
+	SYS_MKDIR                        = 39
+	SYS_RMDIR                        = 40
+	SYS_DUP                          = 41
+	SYS_PIPE                         = 42
+	SYS_TIMES                        = 43
+	SYS_PROF                         = 44
+	SYS_BRK                          = 45
+	SYS_SETGID                       = 46
+	SYS_GETGID                       = 47
+	SYS_SIGNAL                       = 48
+	SYS_GETEUID                      = 49
+	SYS_GETEGID                      = 50
+	SYS_ACCT                         = 51
+	SYS_UMOUNT2                      = 52
+	SYS_LOCK                         = 53
+	SYS_IOCTL                        = 54
+	SYS_FCNTL                        = 55
+	SYS_MPX                          = 56
+	SYS_SETPGID                      = 57
+	SYS_ULIMIT                       = 58
+	SYS_OLDOLDUNAME                  = 59
+	SYS_UMASK                        = 60
+	SYS_CHROOT                       = 61
+	SYS_USTAT                        = 62
+	SYS_DUP2                         = 63
+	SYS_GETPPID                      = 64
+	SYS_GETPGRP                      = 65
+	SYS_SETSID                       = 66
+	SYS_SIGACTION                    = 67
+	SYS_SGETMASK                     = 68
+	SYS_SSETMASK                     = 69
+	SYS_SETREUID                     = 70
+	SYS_SETREGID                     = 71
+	SYS_SIGSUSPEND                   = 72
+	SYS_SIGPENDING                   = 73
+	SYS_SETHOSTNAME                  = 74
+	SYS_SETRLIMIT                    = 75
+	SYS_GETRLIMIT                    = 76
+	SYS_GETRUSAGE                    = 77
+	SYS_GETTIMEOFDAY                 = 78
+	SYS_SETTIMEOFDAY                 = 79
+	SYS_GETGROUPS                    = 80
+	SYS_SETGROUPS                    = 81
+	SYS_SELECT                       = 82
+	SYS_SYMLINK                      = 83
+	SYS_OLDLSTAT                     = 84
+	SYS_READLINK                     = 85
+	SYS_USELIB                       = 86
+	SYS_SWAPON                       = 87
+	SYS_REBOOT                       = 88
+	SYS_READDIR                      = 89
+	SYS_MMAP                         = 90
+	SYS_MUNMAP                       = 91
+	SYS_TRUNCATE                     = 92
+	SYS_FTRUNCATE                    = 93
+	SYS_FCHMOD                       = 94
+	SYS_FCHOWN                       = 95
+	SYS_GETPRIORITY                  = 96
+	SYS_SETPRIORITY                  = 97
+	SYS_PROFIL                       = 98
+	SYS_STATFS                       = 99
+	SYS_FSTATFS                      = 100
+	SYS_IOPERM                       = 101
+	SYS_SOCKETCALL                   = 102
+	SYS_SYSLOG                       = 103
+	SYS_SETITIMER                    = 104
+	SYS_GETITIMER                    = 105
+	SYS_STAT                         = 106
+	SYS_LSTAT                        = 107
+	SYS_FSTAT                        = 108
+	SYS_OLDUNAME                     = 109
+	SYS_IOPL                         = 110
+	SYS_VHANGUP                      = 111
+	SYS_IDLE                         = 112
+	SYS_VM86                         = 113
+	SYS_WAIT4                        = 114
+	SYS_SWAPOFF                      = 115
+	SYS_SYSINFO                      = 116
+	SYS_IPC                          = 117
+	SYS_FSYNC                        = 118
+	SYS_SIGRETURN                    = 119
+	SYS_CLONE                        = 120
+	SYS_SETDOMAINNAME                = 121
+	SYS_UNAME                        = 122
+	SYS_MODIFY_LDT                   = 123
+	SYS_ADJTIMEX                     = 124
+	SYS_MPROTECT                     = 125
+	SYS_SIGPROCMASK                  = 126
+	SYS_CREATE_MODULE                = 127
+	SYS_INIT_MODULE                  = 128
+	SYS_DELETE_MODULE                = 129
+	SYS_GET_KERNEL_SYMS              = 130
+	SYS_QUOTACTL                     = 131
+	SYS_GETPGID                      = 132
+	SYS_FCHDIR                       = 133
+	SYS_BDFLUSH                      = 134
+	SYS_SYSFS                        = 135
+	SYS_PERSONALITY                  = 136
+	SYS_AFS_SYSCALL                  = 137
+	SYS_SETFSUID                     = 138
+	SYS_SETFSGID                     = 139
+	SYS__LLSEEK                      = 140
+	SYS_GETDENTS                     = 141
+	SYS__NEWSELECT                   = 142
+	SYS_FLOCK                        = 143
+	SYS_MSYNC                        = 144
+	SYS_READV                        = 145
+	SYS_WRITEV                       = 146
+	SYS_GETSID                       = 147
+	SYS_FDATASYNC                    = 148
+	SYS__SYSCTL                      = 149
+	SYS_MLOCK                        = 150
+	SYS_MUNLOCK                      = 151
+	SYS_MLOCKALL                     = 152
+	SYS_MUNLOCKALL                   = 153
+	SYS_SCHED_SETPARAM               = 154
+	SYS_SCHED_GETPARAM               = 155
+	SYS_SCHED_SETSCHEDULER           = 156
+	SYS_SCHED_GETSCHEDULER           = 157
+	SYS_SCHED_YIELD                  = 158
+	SYS_SCHED_GET_PRIORITY_MAX       = 159
+	SYS_SCHED_GET_PRIORITY_MIN       = 160
+	SYS_SCHED_RR_GET_INTERVAL        = 161
+	SYS_NANOSLEEP                    = 162
+	SYS_MREMAP                       = 163
+	SYS_SETRESUID                    = 164
+	SYS_GETRESUID                    = 165
+	SYS_QUERY_MODULE                 = 166
+	SYS_POLL                         = 167
+	SYS_NFSSERVCTL                   = 168
+	SYS_SETRESGID                    = 169
+	SYS_GETRESGID                    = 170
+	SYS_PRCTL                        = 171
+	SYS_RT_SIGRETURN                 = 172
+	SYS_RT_SIGACTION                 = 173
+	SYS_RT_SIGPROCMASK               = 174
+	SYS_RT_SIGPENDING                = 175
+	SYS_RT_SIGTIMEDWAIT              = 176
+	SYS_RT_SIGQUEUEINFO              = 177
+	SYS_RT_SIGSUSPEND                = 178
+	SYS_PREAD64                      = 179
+	SYS_PWRITE64                     = 180
+	SYS_CHOWN                        = 181
+	SYS_GETCWD                       = 182
+	SYS_CAPGET                       = 183
+	SYS_CAPSET                       = 184
+	SYS_SIGALTSTACK                  = 185
+	SYS_SENDFILE                     = 186
+	SYS_GETPMSG                      = 187
+	SYS_PUTPMSG                      = 188
+	SYS_VFORK                        = 189
+	SYS_UGETRLIMIT                   = 190
+	SYS_READAHEAD                    = 191
+	SYS_MMAP2                        = 192
+	SYS_TRUNCATE64                   = 193
+	SYS_FTRUNCATE64                  = 194
+	SYS_STAT64                       = 195
+	SYS_LSTAT64                      = 196
+	SYS_FSTAT64                      = 197
+	SYS_PCICONFIG_READ               = 198
+	SYS_PCICONFIG_WRITE              = 199
+	SYS_PCICONFIG_IOBASE             = 200
+	SYS_MULTIPLEXER                  = 201
+	SYS_GETDENTS64                   = 202
+	SYS_PIVOT_ROOT                   = 203
+	SYS_FCNTL64                      = 204
+	SYS_MADVISE                      = 205
+	SYS_MINCORE                      = 206
+	SYS_GETTID                       = 207
+	SYS_TKILL                        = 208
+	SYS_SETXATTR                     = 209
+	SYS_LSETXATTR                    = 210
+	SYS_FSETXATTR                    = 211
+	SYS_GETXATTR                     = 212
+	SYS_LGETXATTR                    = 213
+	SYS_FGETXATTR                    = 214
+	SYS_LISTXATTR                    = 215
+	SYS_LLISTXATTR                   = 216
+	SYS_FLISTXATTR                   = 217
+	SYS_REMOVEXATTR                  = 218
+	SYS_LREMOVEXATTR                 = 219
+	SYS_FREMOVEXATTR                 = 220
+	SYS_FUTEX                        = 221
+	SYS_SCHED_SETAFFINITY            = 222
+	SYS_SCHED_GETAFFINITY            = 223
+	SYS_TUXCALL                      = 225
+	SYS_SENDFILE64                   = 226
+	SYS_IO_SETUP                     = 227
+	SYS_IO_DESTROY                   = 228
+	SYS_IO_GETEVENTS                 = 229
+	SYS_IO_SUBMIT                    = 230
+	SYS_IO_CANCEL                    = 231
+	SYS_SET_TID_ADDRESS              = 232
+	SYS_FADVISE64                    = 233
+	SYS_EXIT_GROUP                   = 234
+	SYS_LOOKUP_DCOOKIE               = 235
+	SYS_EPOLL_CREATE                 = 236
+	SYS_EPOLL_CTL                    = 237
+	SYS_EPOLL_WAIT                   = 238
+	SYS_REMAP_FILE_PAGES             = 239
+	SYS_TIMER_CREATE                 = 240
+	SYS_TIMER_SETTIME                = 241
+	SYS_TIMER_GETTIME                = 242
+	SYS_TIMER_GETOVERRUN             = 243
+	SYS_TIMER_DELETE                 = 244
+	SYS_CLOCK_SETTIME                = 245
+	SYS_CLOCK_GETTIME                = 246
+	SYS_CLOCK_GETRES                 = 247
+	SYS_CLOCK_NANOSLEEP              = 248
+	SYS_SWAPCONTEXT                  = 249
+	SYS_TGKILL                       = 250
+	SYS_UTIMES                       = 251
+	SYS_STATFS64                     = 252
+	SYS_FSTATFS64                    = 253
+	SYS_FADVISE64_64                 = 254
+	SYS_RTAS                         = 255
+	SYS_SYS_DEBUG_SETCONTEXT         = 256
+	SYS_MIGRATE_PAGES                = 258
+	SYS_MBIND                        = 259
+	SYS_GET_MEMPOLICY                = 260
+	SYS_SET_MEMPOLICY                = 261
+	SYS_MQ_OPEN                      = 262
+	SYS_MQ_UNLINK                    = 263
+	SYS_MQ_TIMEDSEND                 = 264
+	SYS_MQ_TIMEDRECEIVE              = 265
+	SYS_MQ_NOTIFY                    = 266
+	SYS_MQ_GETSETATTR                = 267
+	SYS_KEXEC_LOAD                   = 268
+	SYS_ADD_KEY                      = 269
+	SYS_REQUEST_KEY                  = 270
+	SYS_KEYCTL                       = 271
+	SYS_WAITID                       = 272
+	SYS_IOPRIO_SET                   = 273
+	SYS_IOPRIO_GET                   = 274
+	SYS_INOTIFY_INIT                 = 275
+	SYS_INOTIFY_ADD_WATCH            = 276
+	SYS_INOTIFY_RM_WATCH             = 277
+	SYS_SPU_RUN                      = 278
+	SYS_SPU_CREATE                   = 279
+	SYS_PSELECT6                     = 280
+	SYS_PPOLL                        = 281
+	SYS_UNSHARE                      = 282
+	SYS_SPLICE                       = 283
+	SYS_TEE                          = 284
+	SYS_VMSPLICE                     = 285
+	SYS_OPENAT                       = 286
+	SYS_MKDIRAT                      = 287
+	SYS_MKNODAT                      = 288
+	SYS_FCHOWNAT                     = 289
+	SYS_FUTIMESAT                    = 290
+	SYS_FSTATAT64                    = 291
+	SYS_UNLINKAT                     = 292
+	SYS_RENAMEAT                     = 293
+	SYS_LINKAT                       = 294
+	SYS_SYMLINKAT                    = 295
+	SYS_READLINKAT                   = 296
+	SYS_FCHMODAT                     = 297
+	SYS_FACCESSAT                    = 298
+	SYS_GET_ROBUST_LIST              = 299
+	SYS_SET_ROBUST_LIST              = 300
+	SYS_MOVE_PAGES                   = 301
+	SYS_GETCPU                       = 302
+	SYS_EPOLL_PWAIT                  = 303
+	SYS_UTIMENSAT                    = 304
+	SYS_SIGNALFD                     = 305
+	SYS_TIMERFD_CREATE               = 306
+	SYS_EVENTFD                      = 307
+	SYS_SYNC_FILE_RANGE2             = 308
+	SYS_FALLOCATE                    = 309
+	SYS_SUBPAGE_PROT                 = 310
+	SYS_TIMERFD_SETTIME              = 311
+	SYS_TIMERFD_GETTIME              = 312
+	SYS_SIGNALFD4                    = 313
+	SYS_EVENTFD2                     = 314
+	SYS_EPOLL_CREATE1                = 315
+	SYS_DUP3                         = 316
+	SYS_PIPE2                        = 317
+	SYS_INOTIFY_INIT1                = 318
+	SYS_PERF_EVENT_OPEN              = 319
+	SYS_PREADV                       = 320
+	SYS_PWRITEV                      = 321
+	SYS_RT_TGSIGQUEUEINFO            = 322
+	SYS_FANOTIFY_INIT                = 323
+	SYS_FANOTIFY_MARK                = 324
+	SYS_PRLIMIT64                    = 325
+	SYS_SOCKET                       = 326
+	SYS_BIND                         = 327
+	SYS_CONNECT                      = 328
+	SYS_LISTEN                       = 329
+	SYS_ACCEPT                       = 330
+	SYS_GETSOCKNAME                  = 331
+	SYS_GETPEERNAME                  = 332
+	SYS_SOCKETPAIR                   = 333
+	SYS_SEND                         = 334
+	SYS_SENDTO                       = 335
+	SYS_RECV                         = 336
+	SYS_RECVFROM                     = 337
+	SYS_SHUTDOWN                     = 338
+	SYS_SETSOCKOPT                   = 339
+	SYS_GETSOCKOPT                   = 340
+	SYS_SENDMSG                      = 341
+	SYS_RECVMSG                      = 342
+	SYS_RECVMMSG                     = 343
+	SYS_ACCEPT4                      = 344
+	SYS_NAME_TO_HANDLE_AT            = 345
+	SYS_OPEN_BY_HANDLE_AT            = 346
+	SYS_CLOCK_ADJTIME                = 347
+	SYS_SYNCFS                       = 348
+	SYS_SENDMMSG                     = 349
+	SYS_SETNS                        = 350
+	SYS_PROCESS_VM_READV             = 351
+	SYS_PROCESS_VM_WRITEV            = 352
+	SYS_FINIT_MODULE                 = 353
+	SYS_KCMP                         = 354
+	SYS_SCHED_SETATTR                = 355
+	SYS_SCHED_GETATTR                = 356
+	SYS_RENAMEAT2                    = 357
+	SYS_SECCOMP                      = 358
+	SYS_GETRANDOM                    = 359
+	SYS_MEMFD_CREATE                 = 360
+	SYS_BPF                          = 361
+	SYS_EXECVEAT                     = 362
+	SYS_SWITCH_ENDIAN                = 363
+	SYS_USERFAULTFD                  = 364
+	SYS_MEMBARRIER                   = 365
+	SYS_MLOCK2                       = 378
+	SYS_COPY_FILE_RANGE              = 379
+	SYS_PREADV2                      = 380
+	SYS_PWRITEV2                     = 381
+	SYS_KEXEC_FILE_LOAD              = 382
+	SYS_STATX                        = 383
+	SYS_PKEY_ALLOC                   = 384
+	SYS_PKEY_FREE                    = 385
+	SYS_PKEY_MPROTECT                = 386
+	SYS_RSEQ                         = 387
+	SYS_IO_PGETEVENTS                = 388
+	SYS_SEMGET                       = 393
+	SYS_SEMCTL                       = 394
+	SYS_SHMGET                       = 395
+	SYS_SHMCTL                       = 396
+	SYS_SHMAT                        = 397
+	SYS_SHMDT                        = 398
+	SYS_MSGGET                       = 399
+	SYS_MSGSND                       = 400
+	SYS_MSGRCV                       = 401
+	SYS_MSGCTL                       = 402
+	SYS_CLOCK_GETTIME64              = 403
+	SYS_CLOCK_SETTIME64              = 404
+	SYS_CLOCK_ADJTIME64              = 405
+	SYS_CLOCK_GETRES_TIME64          = 406
+	SYS_CLOCK_NANOSLEEP_TIME64       = 407
+	SYS_TIMER_GETTIME64              = 408
+	SYS_TIMER_SETTIME64              = 409
+	SYS_TIMERFD_GETTIME64            = 410
+	SYS_TIMERFD_SETTIME64            = 411
+	SYS_UTIMENSAT_TIME64             = 412
+	SYS_PSELECT6_TIME64              = 413
+	SYS_PPOLL_TIME64                 = 414
+	SYS_IO_PGETEVENTS_TIME64         = 416
+	SYS_RECVMMSG_TIME64              = 417
+	SYS_MQ_TIMEDSEND_TIME64          = 418
+	SYS_MQ_TIMEDRECEIVE_TIME64       = 419
+	SYS_SEMTIMEDOP_TIME64            = 420
+	SYS_RT_SIGTIMEDWAIT_TIME64       = 421
+	SYS_FUTEX_TIME64                 = 422
+	SYS_SCHED_RR_GET_INTERVAL_TIME64 = 423
+	SYS_PIDFD_SEND_SIGNAL            = 424
+	SYS_IO_URING_SETUP               = 425
+	SYS_IO_URING_ENTER               = 426
+	SYS_IO_URING_REGISTER            = 427
+	SYS_OPEN_TREE                    = 428
+	SYS_MOVE_MOUNT                   = 429
+	SYS_FSOPEN                       = 430
+	SYS_FSCONFIG                     = 431
+	SYS_FSMOUNT                      = 432
+	SYS_FSPICK                       = 433
+	SYS_PIDFD_OPEN                   = 434
+	SYS_CLONE3                       = 435
+	SYS_CLOSE_RANGE                  = 436
+	SYS_OPENAT2                      = 437
+	SYS_PIDFD_GETFD                  = 438
+	SYS_FACCESSAT2                   = 439
+	SYS_PROCESS_MADVISE              = 440
+	SYS_EPOLL_PWAIT2                 = 441
+	SYS_MOUNT_SETATTR                = 442
+)
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64.go
index 9287538..08edc54 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64 && linux
 // +build ppc64,linux
 
 package unix
@@ -395,7 +396,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64le.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64le.go
index 4dafad8..33b33b0 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64le.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_ppc64le.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64le && linux
 // +build ppc64le,linux
 
 package unix
@@ -395,7 +396,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_riscv64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_riscv64.go
index 6642cfc..66c8a8e 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_riscv64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_riscv64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build riscv64 && linux
 // +build riscv64,linux
 
 package unix
@@ -297,7 +298,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_s390x.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_s390x.go
index 23367b9..aea5760 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_s390x.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_s390x.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include -fsigned-char /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build s390x && linux
 // +build s390x,linux
 
 package unix
@@ -360,7 +361,11 @@
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
 	SYS_CLONE3                 = 435
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_linux_sparc64.go b/vendor/golang.org/x/sys/unix/zsysnum_linux_sparc64.go
index 083aa02..488ca84 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_linux_sparc64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_linux_sparc64.go
@@ -1,6 +1,7 @@
 // go run linux/mksysnum.go -Wall -Werror -static -I/tmp/include /tmp/include/asm/unistd.h
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build sparc64 && linux
 // +build sparc64,linux
 
 package unix
@@ -374,7 +375,11 @@
 	SYS_FSMOUNT                = 432
 	SYS_FSPICK                 = 433
 	SYS_PIDFD_OPEN             = 434
+	SYS_CLOSE_RANGE            = 436
 	SYS_OPENAT2                = 437
 	SYS_PIDFD_GETFD            = 438
 	SYS_FACCESSAT2             = 439
+	SYS_PROCESS_MADVISE        = 440
+	SYS_EPOLL_PWAIT2           = 441
+	SYS_MOUNT_SETATTR          = 442
 )
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_386.go b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_386.go
index e66a8c9..3a6699e 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_386.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go http://cvsweb.netbsd.org/bsdweb.cgi/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && netbsd
 // +build 386,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_amd64.go
index 42c788f..5677cd4 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go http://cvsweb.netbsd.org/bsdweb.cgi/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && netbsd
 // +build amd64,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm.go b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm.go
index 0a07571..e784cb6 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go http://cvsweb.netbsd.org/bsdweb.cgi/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && netbsd
 // +build arm,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm64.go b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm64.go
index 0291c09..bd4952e 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_netbsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go http://cvsweb.netbsd.org/bsdweb.cgi/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; DO NOT EDIT.
 
+//go:build arm64 && netbsd
 // +build arm64,netbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_386.go b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_386.go
index b0207d1..817edbf 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_386.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://cvsweb.openbsd.org/cgi-bin/cvsweb/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && openbsd
 // +build 386,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_amd64.go
index f0dec6f..ea45361 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_amd64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://cvsweb.openbsd.org/cgi-bin/cvsweb/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && openbsd
 // +build amd64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm.go b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm.go
index 33d1dc5..467971e 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://cvsweb.openbsd.org/cgi-bin/cvsweb/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && openbsd
 // +build arm,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm64.go
index fe2b689..32eec5e 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_arm64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://cvsweb.openbsd.org/cgi-bin/cvsweb/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && openbsd
 // +build arm64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_mips64.go
index 5c08d57..a37f773 100644
--- a/vendor/golang.org/x/sys/unix/zsysnum_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/zsysnum_openbsd_mips64.go
@@ -1,6 +1,7 @@
 // go run mksysnum.go https://cvsweb.openbsd.org/cgi-bin/cvsweb/~checkout~/src/sys/kern/syscalls.master
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && openbsd
 // +build mips64,openbsd
 
 package unix
diff --git a/vendor/golang.org/x/sys/unix/zsysnum_zos_s390x.go b/vendor/golang.org/x/sys/unix/zsysnum_zos_s390x.go
new file mode 100644
index 0000000..073daad
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/zsysnum_zos_s390x.go
@@ -0,0 +1,2670 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+package unix
+
+// TODO: auto-generate.
+
+const (
+	SYS_ACOSD128                        = 0xB80
+	SYS_ACOSD32                         = 0xB7E
+	SYS_ACOSD64                         = 0xB7F
+	SYS_ACOSHD128                       = 0xB83
+	SYS_ACOSHD32                        = 0xB81
+	SYS_ACOSHD64                        = 0xB82
+	SYS_AIO_FSYNC                       = 0xC69
+	SYS_ASCTIME                         = 0x0AE
+	SYS_ASCTIME64                       = 0xCD7
+	SYS_ASCTIME64_R                     = 0xCD8
+	SYS_ASIND128                        = 0xB86
+	SYS_ASIND32                         = 0xB84
+	SYS_ASIND64                         = 0xB85
+	SYS_ASINHD128                       = 0xB89
+	SYS_ASINHD32                        = 0xB87
+	SYS_ASINHD64                        = 0xB88
+	SYS_ATAN2D128                       = 0xB8F
+	SYS_ATAN2D32                        = 0xB8D
+	SYS_ATAN2D64                        = 0xB8E
+	SYS_ATAND128                        = 0xB8C
+	SYS_ATAND32                         = 0xB8A
+	SYS_ATAND64                         = 0xB8B
+	SYS_ATANHD128                       = 0xB92
+	SYS_ATANHD32                        = 0xB90
+	SYS_ATANHD64                        = 0xB91
+	SYS_BIND2ADDRSEL                    = 0xD59
+	SYS_C16RTOMB                        = 0xD40
+	SYS_C32RTOMB                        = 0xD41
+	SYS_CBRTD128                        = 0xB95
+	SYS_CBRTD32                         = 0xB93
+	SYS_CBRTD64                         = 0xB94
+	SYS_CEILD128                        = 0xB98
+	SYS_CEILD32                         = 0xB96
+	SYS_CEILD64                         = 0xB97
+	SYS_CLEARENV                        = 0x0C9
+	SYS_CLEARERR_UNLOCKED               = 0xCA1
+	SYS_CLOCK                           = 0x0AA
+	SYS_CLOGL                           = 0xA00
+	SYS_CLRMEMF                         = 0x0BD
+	SYS_CONJ                            = 0xA03
+	SYS_CONJF                           = 0xA06
+	SYS_CONJL                           = 0xA09
+	SYS_COPYSIGND128                    = 0xB9E
+	SYS_COPYSIGND32                     = 0xB9C
+	SYS_COPYSIGND64                     = 0xB9D
+	SYS_COSD128                         = 0xBA1
+	SYS_COSD32                          = 0xB9F
+	SYS_COSD64                          = 0xBA0
+	SYS_COSHD128                        = 0xBA4
+	SYS_COSHD32                         = 0xBA2
+	SYS_COSHD64                         = 0xBA3
+	SYS_CPOW                            = 0xA0C
+	SYS_CPOWF                           = 0xA0F
+	SYS_CPOWL                           = 0xA12
+	SYS_CPROJ                           = 0xA15
+	SYS_CPROJF                          = 0xA18
+	SYS_CPROJL                          = 0xA1B
+	SYS_CREAL                           = 0xA1E
+	SYS_CREALF                          = 0xA21
+	SYS_CREALL                          = 0xA24
+	SYS_CSIN                            = 0xA27
+	SYS_CSINF                           = 0xA2A
+	SYS_CSINH                           = 0xA30
+	SYS_CSINHF                          = 0xA33
+	SYS_CSINHL                          = 0xA36
+	SYS_CSINL                           = 0xA2D
+	SYS_CSNAP                           = 0x0C5
+	SYS_CSQRT                           = 0xA39
+	SYS_CSQRTF                          = 0xA3C
+	SYS_CSQRTL                          = 0xA3F
+	SYS_CTAN                            = 0xA42
+	SYS_CTANF                           = 0xA45
+	SYS_CTANH                           = 0xA4B
+	SYS_CTANHF                          = 0xA4E
+	SYS_CTANHL                          = 0xA51
+	SYS_CTANL                           = 0xA48
+	SYS_CTIME                           = 0x0AB
+	SYS_CTIME64                         = 0xCD9
+	SYS_CTIME64_R                       = 0xCDA
+	SYS_CTRACE                          = 0x0C6
+	SYS_DIFFTIME                        = 0x0A7
+	SYS_DIFFTIME64                      = 0xCDB
+	SYS_DLADDR                          = 0xC82
+	SYS_DYNALLOC                        = 0x0C3
+	SYS_DYNFREE                         = 0x0C2
+	SYS_ERFCD128                        = 0xBAA
+	SYS_ERFCD32                         = 0xBA8
+	SYS_ERFCD64                         = 0xBA9
+	SYS_ERFD128                         = 0xBA7
+	SYS_ERFD32                          = 0xBA5
+	SYS_ERFD64                          = 0xBA6
+	SYS_EXP2D128                        = 0xBB0
+	SYS_EXP2D32                         = 0xBAE
+	SYS_EXP2D64                         = 0xBAF
+	SYS_EXPD128                         = 0xBAD
+	SYS_EXPD32                          = 0xBAB
+	SYS_EXPD64                          = 0xBAC
+	SYS_EXPM1D128                       = 0xBB3
+	SYS_EXPM1D32                        = 0xBB1
+	SYS_EXPM1D64                        = 0xBB2
+	SYS_FABSD128                        = 0xBB6
+	SYS_FABSD32                         = 0xBB4
+	SYS_FABSD64                         = 0xBB5
+	SYS_FDELREC_UNLOCKED                = 0xCA2
+	SYS_FDIMD128                        = 0xBB9
+	SYS_FDIMD32                         = 0xBB7
+	SYS_FDIMD64                         = 0xBB8
+	SYS_FDOPEN_UNLOCKED                 = 0xCFC
+	SYS_FECLEAREXCEPT                   = 0xAEA
+	SYS_FEGETENV                        = 0xAEB
+	SYS_FEGETEXCEPTFLAG                 = 0xAEC
+	SYS_FEGETROUND                      = 0xAED
+	SYS_FEHOLDEXCEPT                    = 0xAEE
+	SYS_FEOF_UNLOCKED                   = 0xCA3
+	SYS_FERAISEEXCEPT                   = 0xAEF
+	SYS_FERROR_UNLOCKED                 = 0xCA4
+	SYS_FESETENV                        = 0xAF0
+	SYS_FESETEXCEPTFLAG                 = 0xAF1
+	SYS_FESETROUND                      = 0xAF2
+	SYS_FETCHEP                         = 0x0BF
+	SYS_FETESTEXCEPT                    = 0xAF3
+	SYS_FEUPDATEENV                     = 0xAF4
+	SYS_FE_DEC_GETROUND                 = 0xBBA
+	SYS_FE_DEC_SETROUND                 = 0xBBB
+	SYS_FFLUSH_UNLOCKED                 = 0xCA5
+	SYS_FGETC_UNLOCKED                  = 0xC80
+	SYS_FGETPOS64                       = 0xCEE
+	SYS_FGETPOS64_UNLOCKED              = 0xCF4
+	SYS_FGETPOS_UNLOCKED                = 0xCA6
+	SYS_FGETS_UNLOCKED                  = 0xC7C
+	SYS_FGETWC_UNLOCKED                 = 0xCA7
+	SYS_FGETWS_UNLOCKED                 = 0xCA8
+	SYS_FILENO_UNLOCKED                 = 0xCA9
+	SYS_FLDATA                          = 0x0C1
+	SYS_FLDATA_UNLOCKED                 = 0xCAA
+	SYS_FLOCATE_UNLOCKED                = 0xCAB
+	SYS_FLOORD128                       = 0xBBE
+	SYS_FLOORD32                        = 0xBBC
+	SYS_FLOORD64                        = 0xBBD
+	SYS_FMA                             = 0xA63
+	SYS_FMAD128                         = 0xBC1
+	SYS_FMAD32                          = 0xBBF
+	SYS_FMAD64                          = 0xBC0
+	SYS_FMAF                            = 0xA66
+	SYS_FMAL                            = 0xA69
+	SYS_FMAX                            = 0xA6C
+	SYS_FMAXD128                        = 0xBC4
+	SYS_FMAXD32                         = 0xBC2
+	SYS_FMAXD64                         = 0xBC3
+	SYS_FMAXF                           = 0xA6F
+	SYS_FMAXL                           = 0xA72
+	SYS_FMIN                            = 0xA75
+	SYS_FMIND128                        = 0xBC7
+	SYS_FMIND32                         = 0xBC5
+	SYS_FMIND64                         = 0xBC6
+	SYS_FMINF                           = 0xA78
+	SYS_FMINL                           = 0xA7B
+	SYS_FMODD128                        = 0xBCA
+	SYS_FMODD32                         = 0xBC8
+	SYS_FMODD64                         = 0xBC9
+	SYS_FOPEN64                         = 0xD49
+	SYS_FOPEN64_UNLOCKED                = 0xD4A
+	SYS_FOPEN_UNLOCKED                  = 0xCFA
+	SYS_FPRINTF_UNLOCKED                = 0xCAC
+	SYS_FPUTC_UNLOCKED                  = 0xC81
+	SYS_FPUTS_UNLOCKED                  = 0xC7E
+	SYS_FPUTWC_UNLOCKED                 = 0xCAD
+	SYS_FPUTWS_UNLOCKED                 = 0xCAE
+	SYS_FREAD_NOUPDATE                  = 0xCEC
+	SYS_FREAD_NOUPDATE_UNLOCKED         = 0xCED
+	SYS_FREAD_UNLOCKED                  = 0xC7B
+	SYS_FREEIFADDRS                     = 0xCE6
+	SYS_FREOPEN64                       = 0xD4B
+	SYS_FREOPEN64_UNLOCKED              = 0xD4C
+	SYS_FREOPEN_UNLOCKED                = 0xCFB
+	SYS_FREXPD128                       = 0xBCE
+	SYS_FREXPD32                        = 0xBCC
+	SYS_FREXPD64                        = 0xBCD
+	SYS_FSCANF_UNLOCKED                 = 0xCAF
+	SYS_FSEEK64                         = 0xCEF
+	SYS_FSEEK64_UNLOCKED                = 0xCF5
+	SYS_FSEEKO64                        = 0xCF0
+	SYS_FSEEKO64_UNLOCKED               = 0xCF6
+	SYS_FSEEKO_UNLOCKED                 = 0xCB1
+	SYS_FSEEK_UNLOCKED                  = 0xCB0
+	SYS_FSETPOS64                       = 0xCF1
+	SYS_FSETPOS64_UNLOCKED              = 0xCF7
+	SYS_FSETPOS_UNLOCKED                = 0xCB3
+	SYS_FTELL64                         = 0xCF2
+	SYS_FTELL64_UNLOCKED                = 0xCF8
+	SYS_FTELLO64                        = 0xCF3
+	SYS_FTELLO64_UNLOCKED               = 0xCF9
+	SYS_FTELLO_UNLOCKED                 = 0xCB5
+	SYS_FTELL_UNLOCKED                  = 0xCB4
+	SYS_FUPDATE                         = 0x0B5
+	SYS_FUPDATE_UNLOCKED                = 0xCB7
+	SYS_FWIDE_UNLOCKED                  = 0xCB8
+	SYS_FWPRINTF_UNLOCKED               = 0xCB9
+	SYS_FWRITE_UNLOCKED                 = 0xC7A
+	SYS_FWSCANF_UNLOCKED                = 0xCBA
+	SYS_GETDATE64                       = 0xD4F
+	SYS_GETIFADDRS                      = 0xCE7
+	SYS_GETIPV4SOURCEFILTER             = 0xC77
+	SYS_GETSOURCEFILTER                 = 0xC79
+	SYS_GETSYNTX                        = 0x0FD
+	SYS_GETS_UNLOCKED                   = 0xC7D
+	SYS_GETTIMEOFDAY64                  = 0xD50
+	SYS_GETWCHAR_UNLOCKED               = 0xCBC
+	SYS_GETWC_UNLOCKED                  = 0xCBB
+	SYS_GMTIME                          = 0x0B0
+	SYS_GMTIME64                        = 0xCDC
+	SYS_GMTIME64_R                      = 0xCDD
+	SYS_HYPOTD128                       = 0xBD1
+	SYS_HYPOTD32                        = 0xBCF
+	SYS_HYPOTD64                        = 0xBD0
+	SYS_ILOGBD128                       = 0xBD4
+	SYS_ILOGBD32                        = 0xBD2
+	SYS_ILOGBD64                        = 0xBD3
+	SYS_ILOGBF                          = 0xA7E
+	SYS_ILOGBL                          = 0xA81
+	SYS_INET6_IS_SRCADDR                = 0xD5A
+	SYS_ISBLANK                         = 0x0FE
+	SYS_ISWALNUM                        = 0x0FF
+	SYS_LDEXPD128                       = 0xBD7
+	SYS_LDEXPD32                        = 0xBD5
+	SYS_LDEXPD64                        = 0xBD6
+	SYS_LGAMMAD128                      = 0xBDA
+	SYS_LGAMMAD32                       = 0xBD8
+	SYS_LGAMMAD64                       = 0xBD9
+	SYS_LIO_LISTIO                      = 0xC6A
+	SYS_LLRINT                          = 0xA84
+	SYS_LLRINTD128                      = 0xBDD
+	SYS_LLRINTD32                       = 0xBDB
+	SYS_LLRINTD64                       = 0xBDC
+	SYS_LLRINTF                         = 0xA87
+	SYS_LLRINTL                         = 0xA8A
+	SYS_LLROUND                         = 0xA8D
+	SYS_LLROUNDD128                     = 0xBE0
+	SYS_LLROUNDD32                      = 0xBDE
+	SYS_LLROUNDD64                      = 0xBDF
+	SYS_LLROUNDF                        = 0xA90
+	SYS_LLROUNDL                        = 0xA93
+	SYS_LOCALTIM                        = 0x0B1
+	SYS_LOCALTIME                       = 0x0B1
+	SYS_LOCALTIME64                     = 0xCDE
+	SYS_LOCALTIME64_R                   = 0xCDF
+	SYS_LOG10D128                       = 0xBE6
+	SYS_LOG10D32                        = 0xBE4
+	SYS_LOG10D64                        = 0xBE5
+	SYS_LOG1PD128                       = 0xBE9
+	SYS_LOG1PD32                        = 0xBE7
+	SYS_LOG1PD64                        = 0xBE8
+	SYS_LOG2D128                        = 0xBEC
+	SYS_LOG2D32                         = 0xBEA
+	SYS_LOG2D64                         = 0xBEB
+	SYS_LOGBD128                        = 0xBEF
+	SYS_LOGBD32                         = 0xBED
+	SYS_LOGBD64                         = 0xBEE
+	SYS_LOGBF                           = 0xA96
+	SYS_LOGBL                           = 0xA99
+	SYS_LOGD128                         = 0xBE3
+	SYS_LOGD32                          = 0xBE1
+	SYS_LOGD64                          = 0xBE2
+	SYS_LRINT                           = 0xA9C
+	SYS_LRINTD128                       = 0xBF2
+	SYS_LRINTD32                        = 0xBF0
+	SYS_LRINTD64                        = 0xBF1
+	SYS_LRINTF                          = 0xA9F
+	SYS_LRINTL                          = 0xAA2
+	SYS_LROUNDD128                      = 0xBF5
+	SYS_LROUNDD32                       = 0xBF3
+	SYS_LROUNDD64                       = 0xBF4
+	SYS_LROUNDL                         = 0xAA5
+	SYS_MBLEN                           = 0x0AF
+	SYS_MBRTOC16                        = 0xD42
+	SYS_MBRTOC32                        = 0xD43
+	SYS_MEMSET                          = 0x0A3
+	SYS_MKTIME                          = 0x0AC
+	SYS_MKTIME64                        = 0xCE0
+	SYS_MODFD128                        = 0xBF8
+	SYS_MODFD32                         = 0xBF6
+	SYS_MODFD64                         = 0xBF7
+	SYS_NAN                             = 0xAA8
+	SYS_NAND128                         = 0xBFB
+	SYS_NAND32                          = 0xBF9
+	SYS_NAND64                          = 0xBFA
+	SYS_NANF                            = 0xAAA
+	SYS_NANL                            = 0xAAC
+	SYS_NEARBYINT                       = 0xAAE
+	SYS_NEARBYINTD128                   = 0xBFE
+	SYS_NEARBYINTD32                    = 0xBFC
+	SYS_NEARBYINTD64                    = 0xBFD
+	SYS_NEARBYINTF                      = 0xAB1
+	SYS_NEARBYINTL                      = 0xAB4
+	SYS_NEXTAFTERD128                   = 0xC01
+	SYS_NEXTAFTERD32                    = 0xBFF
+	SYS_NEXTAFTERD64                    = 0xC00
+	SYS_NEXTAFTERF                      = 0xAB7
+	SYS_NEXTAFTERL                      = 0xABA
+	SYS_NEXTTOWARD                      = 0xABD
+	SYS_NEXTTOWARDD128                  = 0xC04
+	SYS_NEXTTOWARDD32                   = 0xC02
+	SYS_NEXTTOWARDD64                   = 0xC03
+	SYS_NEXTTOWARDF                     = 0xAC0
+	SYS_NEXTTOWARDL                     = 0xAC3
+	SYS_NL_LANGINFO                     = 0x0FC
+	SYS_PERROR_UNLOCKED                 = 0xCBD
+	SYS_POSIX_FALLOCATE                 = 0xCE8
+	SYS_POSIX_MEMALIGN                  = 0xCE9
+	SYS_POSIX_OPENPT                    = 0xC66
+	SYS_POWD128                         = 0xC07
+	SYS_POWD32                          = 0xC05
+	SYS_POWD64                          = 0xC06
+	SYS_PRINTF_UNLOCKED                 = 0xCBE
+	SYS_PSELECT                         = 0xC67
+	SYS_PTHREAD_ATTR_GETSTACK           = 0xB3E
+	SYS_PTHREAD_ATTR_SETSTACK           = 0xB3F
+	SYS_PTHREAD_SECURITY_APPLID_NP      = 0xCE4
+	SYS_PUTS_UNLOCKED                   = 0xC7F
+	SYS_PUTWCHAR_UNLOCKED               = 0xCC0
+	SYS_PUTWC_UNLOCKED                  = 0xCBF
+	SYS_QUANTEXPD128                    = 0xD46
+	SYS_QUANTEXPD32                     = 0xD44
+	SYS_QUANTEXPD64                     = 0xD45
+	SYS_QUANTIZED128                    = 0xC0A
+	SYS_QUANTIZED32                     = 0xC08
+	SYS_QUANTIZED64                     = 0xC09
+	SYS_REMAINDERD128                   = 0xC0D
+	SYS_REMAINDERD32                    = 0xC0B
+	SYS_REMAINDERD64                    = 0xC0C
+	SYS_RESIZE_ALLOC                    = 0xCEB
+	SYS_REWIND_UNLOCKED                 = 0xCC1
+	SYS_RINTD128                        = 0xC13
+	SYS_RINTD32                         = 0xC11
+	SYS_RINTD64                         = 0xC12
+	SYS_RINTF                           = 0xACB
+	SYS_RINTL                           = 0xACD
+	SYS_ROUND                           = 0xACF
+	SYS_ROUNDD128                       = 0xC16
+	SYS_ROUNDD32                        = 0xC14
+	SYS_ROUNDD64                        = 0xC15
+	SYS_ROUNDF                          = 0xAD2
+	SYS_ROUNDL                          = 0xAD5
+	SYS_SAMEQUANTUMD128                 = 0xC19
+	SYS_SAMEQUANTUMD32                  = 0xC17
+	SYS_SAMEQUANTUMD64                  = 0xC18
+	SYS_SCALBLN                         = 0xAD8
+	SYS_SCALBLND128                     = 0xC1C
+	SYS_SCALBLND32                      = 0xC1A
+	SYS_SCALBLND64                      = 0xC1B
+	SYS_SCALBLNF                        = 0xADB
+	SYS_SCALBLNL                        = 0xADE
+	SYS_SCALBND128                      = 0xC1F
+	SYS_SCALBND32                       = 0xC1D
+	SYS_SCALBND64                       = 0xC1E
+	SYS_SCALBNF                         = 0xAE3
+	SYS_SCALBNL                         = 0xAE6
+	SYS_SCANF_UNLOCKED                  = 0xCC2
+	SYS_SCHED_YIELD                     = 0xB32
+	SYS_SETENV                          = 0x0C8
+	SYS_SETIPV4SOURCEFILTER             = 0xC76
+	SYS_SETSOURCEFILTER                 = 0xC78
+	SYS_SHM_OPEN                        = 0xC8C
+	SYS_SHM_UNLINK                      = 0xC8D
+	SYS_SIND128                         = 0xC22
+	SYS_SIND32                          = 0xC20
+	SYS_SIND64                          = 0xC21
+	SYS_SINHD128                        = 0xC25
+	SYS_SINHD32                         = 0xC23
+	SYS_SINHD64                         = 0xC24
+	SYS_SIZEOF_ALLOC                    = 0xCEA
+	SYS_SOCKATMARK                      = 0xC68
+	SYS_SQRTD128                        = 0xC28
+	SYS_SQRTD32                         = 0xC26
+	SYS_SQRTD64                         = 0xC27
+	SYS_STRCHR                          = 0x0A0
+	SYS_STRCSPN                         = 0x0A1
+	SYS_STRERROR                        = 0x0A8
+	SYS_STRERROR_R                      = 0xB33
+	SYS_STRFTIME                        = 0x0B2
+	SYS_STRLEN                          = 0x0A9
+	SYS_STRPBRK                         = 0x0A2
+	SYS_STRSPN                          = 0x0A4
+	SYS_STRSTR                          = 0x0A5
+	SYS_STRTOD128                       = 0xC2B
+	SYS_STRTOD32                        = 0xC29
+	SYS_STRTOD64                        = 0xC2A
+	SYS_STRTOK                          = 0x0A6
+	SYS_TAND128                         = 0xC2E
+	SYS_TAND32                          = 0xC2C
+	SYS_TAND64                          = 0xC2D
+	SYS_TANHD128                        = 0xC31
+	SYS_TANHD32                         = 0xC2F
+	SYS_TANHD64                         = 0xC30
+	SYS_TGAMMAD128                      = 0xC34
+	SYS_TGAMMAD32                       = 0xC32
+	SYS_TGAMMAD64                       = 0xC33
+	SYS_TIME                            = 0x0AD
+	SYS_TIME64                          = 0xCE1
+	SYS_TMPFILE64                       = 0xD4D
+	SYS_TMPFILE64_UNLOCKED              = 0xD4E
+	SYS_TMPFILE_UNLOCKED                = 0xCFD
+	SYS_TRUNCD128                       = 0xC40
+	SYS_TRUNCD32                        = 0xC3E
+	SYS_TRUNCD64                        = 0xC3F
+	SYS_UNGETC_UNLOCKED                 = 0xCC3
+	SYS_UNGETWC_UNLOCKED                = 0xCC4
+	SYS_UNSETENV                        = 0xB34
+	SYS_VFPRINTF_UNLOCKED               = 0xCC5
+	SYS_VFSCANF_UNLOCKED                = 0xCC7
+	SYS_VFWPRINTF_UNLOCKED              = 0xCC9
+	SYS_VFWSCANF_UNLOCKED               = 0xCCB
+	SYS_VPRINTF_UNLOCKED                = 0xCCD
+	SYS_VSCANF_UNLOCKED                 = 0xCCF
+	SYS_VWPRINTF_UNLOCKED               = 0xCD1
+	SYS_VWSCANF_UNLOCKED                = 0xCD3
+	SYS_WCSTOD128                       = 0xC43
+	SYS_WCSTOD32                        = 0xC41
+	SYS_WCSTOD64                        = 0xC42
+	SYS_WPRINTF_UNLOCKED                = 0xCD5
+	SYS_WSCANF_UNLOCKED                 = 0xCD6
+	SYS__FLUSHLBF                       = 0xD68
+	SYS__FLUSHLBF_UNLOCKED              = 0xD6F
+	SYS___ACOSHF_H                      = 0xA54
+	SYS___ACOSHL_H                      = 0xA55
+	SYS___ASINHF_H                      = 0xA56
+	SYS___ASINHL_H                      = 0xA57
+	SYS___ATANPID128                    = 0xC6D
+	SYS___ATANPID32                     = 0xC6B
+	SYS___ATANPID64                     = 0xC6C
+	SYS___CBRTF_H                       = 0xA58
+	SYS___CBRTL_H                       = 0xA59
+	SYS___CDUMP                         = 0x0C4
+	SYS___CLASS                         = 0xAFA
+	SYS___CLASS2                        = 0xB99
+	SYS___CLASS2D128                    = 0xC99
+	SYS___CLASS2D32                     = 0xC97
+	SYS___CLASS2D64                     = 0xC98
+	SYS___CLASS2F                       = 0xC91
+	SYS___CLASS2F_B                     = 0xC93
+	SYS___CLASS2F_H                     = 0xC94
+	SYS___CLASS2L                       = 0xC92
+	SYS___CLASS2L_B                     = 0xC95
+	SYS___CLASS2L_H                     = 0xC96
+	SYS___CLASS2_B                      = 0xB9A
+	SYS___CLASS2_H                      = 0xB9B
+	SYS___CLASS_B                       = 0xAFB
+	SYS___CLASS_H                       = 0xAFC
+	SYS___CLOGL_B                       = 0xA01
+	SYS___CLOGL_H                       = 0xA02
+	SYS___CLRENV                        = 0x0C9
+	SYS___CLRMF                         = 0x0BD
+	SYS___CODEPAGE_INFO                 = 0xC64
+	SYS___CONJF_B                       = 0xA07
+	SYS___CONJF_H                       = 0xA08
+	SYS___CONJL_B                       = 0xA0A
+	SYS___CONJL_H                       = 0xA0B
+	SYS___CONJ_B                        = 0xA04
+	SYS___CONJ_H                        = 0xA05
+	SYS___COPYSIGN_B                    = 0xA5A
+	SYS___COPYSIGN_H                    = 0xAF5
+	SYS___COSPID128                     = 0xC70
+	SYS___COSPID32                      = 0xC6E
+	SYS___COSPID64                      = 0xC6F
+	SYS___CPOWF_B                       = 0xA10
+	SYS___CPOWF_H                       = 0xA11
+	SYS___CPOWL_B                       = 0xA13
+	SYS___CPOWL_H                       = 0xA14
+	SYS___CPOW_B                        = 0xA0D
+	SYS___CPOW_H                        = 0xA0E
+	SYS___CPROJF_B                      = 0xA19
+	SYS___CPROJF_H                      = 0xA1A
+	SYS___CPROJL_B                      = 0xA1C
+	SYS___CPROJL_H                      = 0xA1D
+	SYS___CPROJ_B                       = 0xA16
+	SYS___CPROJ_H                       = 0xA17
+	SYS___CREALF_B                      = 0xA22
+	SYS___CREALF_H                      = 0xA23
+	SYS___CREALL_B                      = 0xA25
+	SYS___CREALL_H                      = 0xA26
+	SYS___CREAL_B                       = 0xA1F
+	SYS___CREAL_H                       = 0xA20
+	SYS___CSINF_B                       = 0xA2B
+	SYS___CSINF_H                       = 0xA2C
+	SYS___CSINHF_B                      = 0xA34
+	SYS___CSINHF_H                      = 0xA35
+	SYS___CSINHL_B                      = 0xA37
+	SYS___CSINHL_H                      = 0xA38
+	SYS___CSINH_B                       = 0xA31
+	SYS___CSINH_H                       = 0xA32
+	SYS___CSINL_B                       = 0xA2E
+	SYS___CSINL_H                       = 0xA2F
+	SYS___CSIN_B                        = 0xA28
+	SYS___CSIN_H                        = 0xA29
+	SYS___CSNAP                         = 0x0C5
+	SYS___CSQRTF_B                      = 0xA3D
+	SYS___CSQRTF_H                      = 0xA3E
+	SYS___CSQRTL_B                      = 0xA40
+	SYS___CSQRTL_H                      = 0xA41
+	SYS___CSQRT_B                       = 0xA3A
+	SYS___CSQRT_H                       = 0xA3B
+	SYS___CTANF_B                       = 0xA46
+	SYS___CTANF_H                       = 0xA47
+	SYS___CTANHF_B                      = 0xA4F
+	SYS___CTANHF_H                      = 0xA50
+	SYS___CTANHL_B                      = 0xA52
+	SYS___CTANHL_H                      = 0xA53
+	SYS___CTANH_B                       = 0xA4C
+	SYS___CTANH_H                       = 0xA4D
+	SYS___CTANL_B                       = 0xA49
+	SYS___CTANL_H                       = 0xA4A
+	SYS___CTAN_B                        = 0xA43
+	SYS___CTAN_H                        = 0xA44
+	SYS___CTEST                         = 0x0C7
+	SYS___CTRACE                        = 0x0C6
+	SYS___D1TOP                         = 0xC9B
+	SYS___D2TOP                         = 0xC9C
+	SYS___D4TOP                         = 0xC9D
+	SYS___DYNALL                        = 0x0C3
+	SYS___DYNFRE                        = 0x0C2
+	SYS___EXP2F_H                       = 0xA5E
+	SYS___EXP2L_H                       = 0xA5F
+	SYS___EXP2_H                        = 0xA5D
+	SYS___EXPM1F_H                      = 0xA5B
+	SYS___EXPM1L_H                      = 0xA5C
+	SYS___FBUFSIZE                      = 0xD60
+	SYS___FLBF                          = 0xD62
+	SYS___FLDATA                        = 0x0C1
+	SYS___FMAF_B                        = 0xA67
+	SYS___FMAF_H                        = 0xA68
+	SYS___FMAL_B                        = 0xA6A
+	SYS___FMAL_H                        = 0xA6B
+	SYS___FMAXF_B                       = 0xA70
+	SYS___FMAXF_H                       = 0xA71
+	SYS___FMAXL_B                       = 0xA73
+	SYS___FMAXL_H                       = 0xA74
+	SYS___FMAX_B                        = 0xA6D
+	SYS___FMAX_H                        = 0xA6E
+	SYS___FMA_B                         = 0xA64
+	SYS___FMA_H                         = 0xA65
+	SYS___FMINF_B                       = 0xA79
+	SYS___FMINF_H                       = 0xA7A
+	SYS___FMINL_B                       = 0xA7C
+	SYS___FMINL_H                       = 0xA7D
+	SYS___FMIN_B                        = 0xA76
+	SYS___FMIN_H                        = 0xA77
+	SYS___FPENDING                      = 0xD61
+	SYS___FPENDING_UNLOCKED             = 0xD6C
+	SYS___FPURGE                        = 0xD69
+	SYS___FPURGE_UNLOCKED               = 0xD70
+	SYS___FP_CAST_D                     = 0xBCB
+	SYS___FREADABLE                     = 0xD63
+	SYS___FREADAHEAD                    = 0xD6A
+	SYS___FREADAHEAD_UNLOCKED           = 0xD71
+	SYS___FREADING                      = 0xD65
+	SYS___FREADING_UNLOCKED             = 0xD6D
+	SYS___FSEEK2                        = 0xB3C
+	SYS___FSETERR                       = 0xD6B
+	SYS___FSETLOCKING                   = 0xD67
+	SYS___FTCHEP                        = 0x0BF
+	SYS___FTELL2                        = 0xB3B
+	SYS___FUPDT                         = 0x0B5
+	SYS___FWRITABLE                     = 0xD64
+	SYS___FWRITING                      = 0xD66
+	SYS___FWRITING_UNLOCKED             = 0xD6E
+	SYS___GETCB                         = 0x0B4
+	SYS___GETGRGID1                     = 0xD5B
+	SYS___GETGRNAM1                     = 0xD5C
+	SYS___GETTHENT                      = 0xCE5
+	SYS___GETTOD                        = 0xD3E
+	SYS___HYPOTF_H                      = 0xAF6
+	SYS___HYPOTL_H                      = 0xAF7
+	SYS___ILOGBF_B                      = 0xA7F
+	SYS___ILOGBF_H                      = 0xA80
+	SYS___ILOGBL_B                      = 0xA82
+	SYS___ILOGBL_H                      = 0xA83
+	SYS___ISBLANK_A                     = 0xB2E
+	SYS___ISBLNK                        = 0x0FE
+	SYS___ISWBLANK_A                    = 0xB2F
+	SYS___LE_CEEGTJS                    = 0xD72
+	SYS___LE_TRACEBACK                  = 0xB7A
+	SYS___LGAMMAL_H                     = 0xA62
+	SYS___LGAMMA_B_C99                  = 0xB39
+	SYS___LGAMMA_H_C99                  = 0xB38
+	SYS___LGAMMA_R_C99                  = 0xB3A
+	SYS___LLRINTF_B                     = 0xA88
+	SYS___LLRINTF_H                     = 0xA89
+	SYS___LLRINTL_B                     = 0xA8B
+	SYS___LLRINTL_H                     = 0xA8C
+	SYS___LLRINT_B                      = 0xA85
+	SYS___LLRINT_H                      = 0xA86
+	SYS___LLROUNDF_B                    = 0xA91
+	SYS___LLROUNDF_H                    = 0xA92
+	SYS___LLROUNDL_B                    = 0xA94
+	SYS___LLROUNDL_H                    = 0xA95
+	SYS___LLROUND_B                     = 0xA8E
+	SYS___LLROUND_H                     = 0xA8F
+	SYS___LOCALE_CTL                    = 0xD47
+	SYS___LOG1PF_H                      = 0xA60
+	SYS___LOG1PL_H                      = 0xA61
+	SYS___LOGBF_B                       = 0xA97
+	SYS___LOGBF_H                       = 0xA98
+	SYS___LOGBL_B                       = 0xA9A
+	SYS___LOGBL_H                       = 0xA9B
+	SYS___LOGIN_APPLID                  = 0xCE2
+	SYS___LRINTF_B                      = 0xAA0
+	SYS___LRINTF_H                      = 0xAA1
+	SYS___LRINTL_B                      = 0xAA3
+	SYS___LRINTL_H                      = 0xAA4
+	SYS___LRINT_B                       = 0xA9D
+	SYS___LRINT_H                       = 0xA9E
+	SYS___LROUNDF_FIXUP                 = 0xB31
+	SYS___LROUNDL_B                     = 0xAA6
+	SYS___LROUNDL_H                     = 0xAA7
+	SYS___LROUND_FIXUP                  = 0xB30
+	SYS___MOSERVICES                    = 0xD3D
+	SYS___MUST_STAY_CLEAN               = 0xB7C
+	SYS___NANF_B                        = 0xAAB
+	SYS___NANL_B                        = 0xAAD
+	SYS___NAN_B                         = 0xAA9
+	SYS___NEARBYINTF_B                  = 0xAB2
+	SYS___NEARBYINTF_H                  = 0xAB3
+	SYS___NEARBYINTL_B                  = 0xAB5
+	SYS___NEARBYINTL_H                  = 0xAB6
+	SYS___NEARBYINT_B                   = 0xAAF
+	SYS___NEARBYINT_H                   = 0xAB0
+	SYS___NEXTAFTERF_B                  = 0xAB8
+	SYS___NEXTAFTERF_H                  = 0xAB9
+	SYS___NEXTAFTERL_B                  = 0xABB
+	SYS___NEXTAFTERL_H                  = 0xABC
+	SYS___NEXTTOWARDF_B                 = 0xAC1
+	SYS___NEXTTOWARDF_H                 = 0xAC2
+	SYS___NEXTTOWARDL_B                 = 0xAC4
+	SYS___NEXTTOWARDL_H                 = 0xAC5
+	SYS___NEXTTOWARD_B                  = 0xABE
+	SYS___NEXTTOWARD_H                  = 0xABF
+	SYS___O_ENV                         = 0xB7D
+	SYS___PASSWD_APPLID                 = 0xCE3
+	SYS___PTOD1                         = 0xC9E
+	SYS___PTOD2                         = 0xC9F
+	SYS___PTOD4                         = 0xCA0
+	SYS___REGCOMP_STD                   = 0x0EA
+	SYS___REMAINDERF_H                  = 0xAC6
+	SYS___REMAINDERL_H                  = 0xAC7
+	SYS___REMQUOD128                    = 0xC10
+	SYS___REMQUOD32                     = 0xC0E
+	SYS___REMQUOD64                     = 0xC0F
+	SYS___REMQUOF_H                     = 0xAC9
+	SYS___REMQUOL_H                     = 0xACA
+	SYS___REMQUO_H                      = 0xAC8
+	SYS___RINTF_B                       = 0xACC
+	SYS___RINTL_B                       = 0xACE
+	SYS___ROUNDF_B                      = 0xAD3
+	SYS___ROUNDF_H                      = 0xAD4
+	SYS___ROUNDL_B                      = 0xAD6
+	SYS___ROUNDL_H                      = 0xAD7
+	SYS___ROUND_B                       = 0xAD0
+	SYS___ROUND_H                       = 0xAD1
+	SYS___SCALBLNF_B                    = 0xADC
+	SYS___SCALBLNF_H                    = 0xADD
+	SYS___SCALBLNL_B                    = 0xADF
+	SYS___SCALBLNL_H                    = 0xAE0
+	SYS___SCALBLN_B                     = 0xAD9
+	SYS___SCALBLN_H                     = 0xADA
+	SYS___SCALBNF_B                     = 0xAE4
+	SYS___SCALBNF_H                     = 0xAE5
+	SYS___SCALBNL_B                     = 0xAE7
+	SYS___SCALBNL_H                     = 0xAE8
+	SYS___SCALBN_B                      = 0xAE1
+	SYS___SCALBN_H                      = 0xAE2
+	SYS___SETENV                        = 0x0C8
+	SYS___SINPID128                     = 0xC73
+	SYS___SINPID32                      = 0xC71
+	SYS___SINPID64                      = 0xC72
+	SYS___SMF_RECORD2                   = 0xD48
+	SYS___STATIC_REINIT                 = 0xB3D
+	SYS___TGAMMAF_H_C99                 = 0xB79
+	SYS___TGAMMAL_H                     = 0xAE9
+	SYS___TGAMMA_H_C99                  = 0xB78
+	SYS___TOCSNAME2                     = 0xC9A
+	SYS_CEIL                            = 0x01F
+	SYS_CHAUDIT                         = 0x1E0
+	SYS_EXP                             = 0x01A
+	SYS_FCHAUDIT                        = 0x1E1
+	SYS_FREXP                           = 0x01D
+	SYS_GETGROUPSBYNAME                 = 0x1E2
+	SYS_GETPWUID                        = 0x1A0
+	SYS_GETUID                          = 0x1A1
+	SYS_ISATTY                          = 0x1A3
+	SYS_KILL                            = 0x1A4
+	SYS_LDEXP                           = 0x01E
+	SYS_LINK                            = 0x1A5
+	SYS_LOG10                           = 0x01C
+	SYS_LSEEK                           = 0x1A6
+	SYS_LSTAT                           = 0x1A7
+	SYS_MKDIR                           = 0x1A8
+	SYS_MKFIFO                          = 0x1A9
+	SYS_MKNOD                           = 0x1AA
+	SYS_MODF                            = 0x01B
+	SYS_MOUNT                           = 0x1AB
+	SYS_OPEN                            = 0x1AC
+	SYS_OPENDIR                         = 0x1AD
+	SYS_PATHCONF                        = 0x1AE
+	SYS_PAUSE                           = 0x1AF
+	SYS_PIPE                            = 0x1B0
+	SYS_PTHREAD_ATTR_DESTROY            = 0x1E7
+	SYS_PTHREAD_ATTR_GETDETACHSTATE     = 0x1EB
+	SYS_PTHREAD_ATTR_GETSTACKSIZE       = 0x1E9
+	SYS_PTHREAD_ATTR_GETWEIGHT_NP       = 0x1ED
+	SYS_PTHREAD_ATTR_INIT               = 0x1E6
+	SYS_PTHREAD_ATTR_SETDETACHSTATE     = 0x1EA
+	SYS_PTHREAD_ATTR_SETSTACKSIZE       = 0x1E8
+	SYS_PTHREAD_ATTR_SETWEIGHT_NP       = 0x1EC
+	SYS_PTHREAD_CANCEL                  = 0x1EE
+	SYS_PTHREAD_CLEANUP_POP             = 0x1F0
+	SYS_PTHREAD_CLEANUP_PUSH            = 0x1EF
+	SYS_PTHREAD_CONDATTR_DESTROY        = 0x1F2
+	SYS_PTHREAD_CONDATTR_INIT           = 0x1F1
+	SYS_PTHREAD_COND_BROADCAST          = 0x1F6
+	SYS_PTHREAD_COND_DESTROY            = 0x1F4
+	SYS_PTHREAD_COND_INIT               = 0x1F3
+	SYS_PTHREAD_COND_SIGNAL             = 0x1F5
+	SYS_PTHREAD_COND_TIMEDWAIT          = 0x1F8
+	SYS_PTHREAD_COND_WAIT               = 0x1F7
+	SYS_PTHREAD_CREATE                  = 0x1F9
+	SYS_PTHREAD_DETACH                  = 0x1FA
+	SYS_PTHREAD_EQUAL                   = 0x1FB
+	SYS_PTHREAD_EXIT                    = 0x1E4
+	SYS_PTHREAD_GETSPECIFIC             = 0x1FC
+	SYS_PTHREAD_JOIN                    = 0x1FD
+	SYS_PTHREAD_KEY_CREATE              = 0x1FE
+	SYS_PTHREAD_KILL                    = 0x1E5
+	SYS_PTHREAD_MUTEXATTR_INIT          = 0x1FF
+	SYS_READ                            = 0x1B2
+	SYS_READDIR                         = 0x1B3
+	SYS_READLINK                        = 0x1B4
+	SYS_REWINDDIR                       = 0x1B5
+	SYS_RMDIR                           = 0x1B6
+	SYS_SETEGID                         = 0x1B7
+	SYS_SETEUID                         = 0x1B8
+	SYS_SETGID                          = 0x1B9
+	SYS_SETPGID                         = 0x1BA
+	SYS_SETSID                          = 0x1BB
+	SYS_SETUID                          = 0x1BC
+	SYS_SIGACTION                       = 0x1BD
+	SYS_SIGADDSET                       = 0x1BE
+	SYS_SIGDELSET                       = 0x1BF
+	SYS_SIGEMPTYSET                     = 0x1C0
+	SYS_SIGFILLSET                      = 0x1C1
+	SYS_SIGISMEMBER                     = 0x1C2
+	SYS_SIGLONGJMP                      = 0x1C3
+	SYS_SIGPENDING                      = 0x1C4
+	SYS_SIGPROCMASK                     = 0x1C5
+	SYS_SIGSETJMP                       = 0x1C6
+	SYS_SIGSUSPEND                      = 0x1C7
+	SYS_SIGWAIT                         = 0x1E3
+	SYS_SLEEP                           = 0x1C8
+	SYS_STAT                            = 0x1C9
+	SYS_SYMLINK                         = 0x1CB
+	SYS_SYSCONF                         = 0x1CC
+	SYS_TCDRAIN                         = 0x1CD
+	SYS_TCFLOW                          = 0x1CE
+	SYS_TCFLUSH                         = 0x1CF
+	SYS_TCGETATTR                       = 0x1D0
+	SYS_TCGETPGRP                       = 0x1D1
+	SYS_TCSENDBREAK                     = 0x1D2
+	SYS_TCSETATTR                       = 0x1D3
+	SYS_TCSETPGRP                       = 0x1D4
+	SYS_TIMES                           = 0x1D5
+	SYS_TTYNAME                         = 0x1D6
+	SYS_TZSET                           = 0x1D7
+	SYS_UMASK                           = 0x1D8
+	SYS_UMOUNT                          = 0x1D9
+	SYS_UNAME                           = 0x1DA
+	SYS_UNLINK                          = 0x1DB
+	SYS_UTIME                           = 0x1DC
+	SYS_WAIT                            = 0x1DD
+	SYS_WAITPID                         = 0x1DE
+	SYS_WRITE                           = 0x1DF
+	SYS_W_GETPSENT                      = 0x1B1
+	SYS_W_IOCTL                         = 0x1A2
+	SYS_W_STATFS                        = 0x1CA
+	SYS_A64L                            = 0x2EF
+	SYS_BCMP                            = 0x2B9
+	SYS_BCOPY                           = 0x2BA
+	SYS_BZERO                           = 0x2BB
+	SYS_CATCLOSE                        = 0x2B6
+	SYS_CATGETS                         = 0x2B7
+	SYS_CATOPEN                         = 0x2B8
+	SYS_CRYPT                           = 0x2AC
+	SYS_DBM_CLEARERR                    = 0x2F7
+	SYS_DBM_CLOSE                       = 0x2F8
+	SYS_DBM_DELETE                      = 0x2F9
+	SYS_DBM_ERROR                       = 0x2FA
+	SYS_DBM_FETCH                       = 0x2FB
+	SYS_DBM_FIRSTKEY                    = 0x2FC
+	SYS_DBM_NEXTKEY                     = 0x2FD
+	SYS_DBM_OPEN                        = 0x2FE
+	SYS_DBM_STORE                       = 0x2FF
+	SYS_DRAND48                         = 0x2B2
+	SYS_ENCRYPT                         = 0x2AD
+	SYS_ENDUTXENT                       = 0x2E1
+	SYS_ERAND48                         = 0x2B3
+	SYS_ERF                             = 0x02C
+	SYS_ERFC                            = 0x02D
+	SYS_FCHDIR                          = 0x2D9
+	SYS_FFS                             = 0x2BC
+	SYS_FMTMSG                          = 0x2E5
+	SYS_FSTATVFS                        = 0x2B4
+	SYS_FTIME                           = 0x2F5
+	SYS_GAMMA                           = 0x02E
+	SYS_GETDATE                         = 0x2A6
+	SYS_GETPAGESIZE                     = 0x2D8
+	SYS_GETTIMEOFDAY                    = 0x2F6
+	SYS_GETUTXENT                       = 0x2E0
+	SYS_GETUTXID                        = 0x2E2
+	SYS_GETUTXLINE                      = 0x2E3
+	SYS_HCREATE                         = 0x2C6
+	SYS_HDESTROY                        = 0x2C7
+	SYS_HSEARCH                         = 0x2C8
+	SYS_HYPOT                           = 0x02B
+	SYS_INDEX                           = 0x2BD
+	SYS_INITSTATE                       = 0x2C2
+	SYS_INSQUE                          = 0x2CF
+	SYS_ISASCII                         = 0x2ED
+	SYS_JRAND48                         = 0x2E6
+	SYS_L64A                            = 0x2F0
+	SYS_LCONG48                         = 0x2EA
+	SYS_LFIND                           = 0x2C9
+	SYS_LRAND48                         = 0x2E7
+	SYS_LSEARCH                         = 0x2CA
+	SYS_MEMCCPY                         = 0x2D4
+	SYS_MRAND48                         = 0x2E8
+	SYS_NRAND48                         = 0x2E9
+	SYS_PCLOSE                          = 0x2D2
+	SYS_POPEN                           = 0x2D1
+	SYS_PUTUTXLINE                      = 0x2E4
+	SYS_RANDOM                          = 0x2C4
+	SYS_REMQUE                          = 0x2D0
+	SYS_RINDEX                          = 0x2BE
+	SYS_SEED48                          = 0x2EC
+	SYS_SETKEY                          = 0x2AE
+	SYS_SETSTATE                        = 0x2C3
+	SYS_SETUTXENT                       = 0x2DF
+	SYS_SRAND48                         = 0x2EB
+	SYS_SRANDOM                         = 0x2C5
+	SYS_STATVFS                         = 0x2B5
+	SYS_STRCASECMP                      = 0x2BF
+	SYS_STRDUP                          = 0x2C0
+	SYS_STRNCASECMP                     = 0x2C1
+	SYS_SWAB                            = 0x2D3
+	SYS_TDELETE                         = 0x2CB
+	SYS_TFIND                           = 0x2CC
+	SYS_TOASCII                         = 0x2EE
+	SYS_TSEARCH                         = 0x2CD
+	SYS_TWALK                           = 0x2CE
+	SYS_UALARM                          = 0x2F1
+	SYS_USLEEP                          = 0x2F2
+	SYS_WAIT3                           = 0x2A7
+	SYS_WAITID                          = 0x2A8
+	SYS_Y1                              = 0x02A
+	SYS___ATOE                          = 0x2DB
+	SYS___ATOE_L                        = 0x2DC
+	SYS___CATTRM                        = 0x2A9
+	SYS___CNVBLK                        = 0x2AF
+	SYS___CRYTRM                        = 0x2B0
+	SYS___DLGHT                         = 0x2A1
+	SYS___ECRTRM                        = 0x2B1
+	SYS___ETOA                          = 0x2DD
+	SYS___ETOA_L                        = 0x2DE
+	SYS___GDTRM                         = 0x2AA
+	SYS___OCLCK                         = 0x2DA
+	SYS___OPARGF                        = 0x2A2
+	SYS___OPERRF                        = 0x2A5
+	SYS___OPINDF                        = 0x2A4
+	SYS___OPOPTF                        = 0x2A3
+	SYS___RNDTRM                        = 0x2AB
+	SYS___SRCTRM                        = 0x2F4
+	SYS___TZONE                         = 0x2A0
+	SYS___UTXTRM                        = 0x2F3
+	SYS_ASIN                            = 0x03E
+	SYS_ISXDIGIT                        = 0x03B
+	SYS_SETLOCAL                        = 0x03A
+	SYS_SETLOCALE                       = 0x03A
+	SYS_SIN                             = 0x03F
+	SYS_TOLOWER                         = 0x03C
+	SYS_TOUPPER                         = 0x03D
+	SYS_ACCEPT_AND_RECV                 = 0x4F7
+	SYS_ATOL                            = 0x04E
+	SYS_CHECKSCH                        = 0x4BC
+	SYS_CHECKSCHENV                     = 0x4BC
+	SYS_CLEARERR                        = 0x04C
+	SYS_CONNECTS                        = 0x4B5
+	SYS_CONNECTSERVER                   = 0x4B5
+	SYS_CONNECTW                        = 0x4B4
+	SYS_CONNECTWORKMGR                  = 0x4B4
+	SYS_CONTINUE                        = 0x4B3
+	SYS_CONTINUEWORKUNIT                = 0x4B3
+	SYS_COPYSIGN                        = 0x4C2
+	SYS_CREATEWO                        = 0x4B2
+	SYS_CREATEWORKUNIT                  = 0x4B2
+	SYS_DELETEWO                        = 0x4B9
+	SYS_DELETEWORKUNIT                  = 0x4B9
+	SYS_DISCONNE                        = 0x4B6
+	SYS_DISCONNECTSERVER                = 0x4B6
+	SYS_FEOF                            = 0x04D
+	SYS_FERROR                          = 0x04A
+	SYS_FINITE                          = 0x4C8
+	SYS_GAMMA_R                         = 0x4E2
+	SYS_JOINWORK                        = 0x4B7
+	SYS_JOINWORKUNIT                    = 0x4B7
+	SYS_LEAVEWOR                        = 0x4B8
+	SYS_LEAVEWORKUNIT                   = 0x4B8
+	SYS_LGAMMA_R                        = 0x4EB
+	SYS_MATHERR                         = 0x4D0
+	SYS_PERROR                          = 0x04F
+	SYS_QUERYMET                        = 0x4BA
+	SYS_QUERYMETRICS                    = 0x4BA
+	SYS_QUERYSCH                        = 0x4BB
+	SYS_QUERYSCHENV                     = 0x4BB
+	SYS_REWIND                          = 0x04B
+	SYS_SCALBN                          = 0x4D4
+	SYS_SIGNIFIC                        = 0x4D5
+	SYS_SIGNIFICAND                     = 0x4D5
+	SYS___ACOSH_B                       = 0x4DA
+	SYS___ACOS_B                        = 0x4D9
+	SYS___ASINH_B                       = 0x4BE
+	SYS___ASIN_B                        = 0x4DB
+	SYS___ATAN2_B                       = 0x4DC
+	SYS___ATANH_B                       = 0x4DD
+	SYS___ATAN_B                        = 0x4BF
+	SYS___CBRT_B                        = 0x4C0
+	SYS___CEIL_B                        = 0x4C1
+	SYS___COSH_B                        = 0x4DE
+	SYS___COS_B                         = 0x4C3
+	SYS___DGHT                          = 0x4A8
+	SYS___ENVN                          = 0x4B0
+	SYS___ERFC_B                        = 0x4C5
+	SYS___ERF_B                         = 0x4C4
+	SYS___EXPM1_B                       = 0x4C6
+	SYS___EXP_B                         = 0x4DF
+	SYS___FABS_B                        = 0x4C7
+	SYS___FLOOR_B                       = 0x4C9
+	SYS___FMOD_B                        = 0x4E0
+	SYS___FP_SETMODE                    = 0x4F8
+	SYS___FREXP_B                       = 0x4CA
+	SYS___GAMMA_B                       = 0x4E1
+	SYS___GDRR                          = 0x4A1
+	SYS___HRRNO                         = 0x4A2
+	SYS___HYPOT_B                       = 0x4E3
+	SYS___ILOGB_B                       = 0x4CB
+	SYS___ISNAN_B                       = 0x4CC
+	SYS___J0_B                          = 0x4E4
+	SYS___J1_B                          = 0x4E6
+	SYS___JN_B                          = 0x4E8
+	SYS___LDEXP_B                       = 0x4CD
+	SYS___LGAMMA_B                      = 0x4EA
+	SYS___LOG10_B                       = 0x4ED
+	SYS___LOG1P_B                       = 0x4CE
+	SYS___LOGB_B                        = 0x4CF
+	SYS___LOGIN                         = 0x4F5
+	SYS___LOG_B                         = 0x4EC
+	SYS___MLOCKALL                      = 0x4B1
+	SYS___MODF_B                        = 0x4D1
+	SYS___NEXTAFTER_B                   = 0x4D2
+	SYS___OPENDIR2                      = 0x4F3
+	SYS___OPEN_STAT                     = 0x4F6
+	SYS___OPND                          = 0x4A5
+	SYS___OPPT                          = 0x4A6
+	SYS___OPRG                          = 0x4A3
+	SYS___OPRR                          = 0x4A4
+	SYS___PID_AFFINITY                  = 0x4BD
+	SYS___POW_B                         = 0x4EE
+	SYS___READDIR2                      = 0x4F4
+	SYS___REMAINDER_B                   = 0x4EF
+	SYS___RINT_B                        = 0x4D3
+	SYS___SCALB_B                       = 0x4F0
+	SYS___SIGACTIONSET                  = 0x4FB
+	SYS___SIGGM                         = 0x4A7
+	SYS___SINH_B                        = 0x4F1
+	SYS___SIN_B                         = 0x4D6
+	SYS___SQRT_B                        = 0x4F2
+	SYS___TANH_B                        = 0x4D8
+	SYS___TAN_B                         = 0x4D7
+	SYS___TRRNO                         = 0x4AF
+	SYS___TZNE                          = 0x4A9
+	SYS___TZZN                          = 0x4AA
+	SYS___UCREATE                       = 0x4FC
+	SYS___UFREE                         = 0x4FE
+	SYS___UHEAPREPORT                   = 0x4FF
+	SYS___UMALLOC                       = 0x4FD
+	SYS___Y0_B                          = 0x4E5
+	SYS___Y1_B                          = 0x4E7
+	SYS___YN_B                          = 0x4E9
+	SYS_ABORT                           = 0x05C
+	SYS_ASCTIME_R                       = 0x5E0
+	SYS_ATEXIT                          = 0x05D
+	SYS_CONNECTE                        = 0x5AE
+	SYS_CONNECTEXPORTIMPORT             = 0x5AE
+	SYS_CTIME_R                         = 0x5E1
+	SYS_DN_COMP                         = 0x5DF
+	SYS_DN_EXPAND                       = 0x5DD
+	SYS_DN_SKIPNAME                     = 0x5DE
+	SYS_EXIT                            = 0x05A
+	SYS_EXPORTWO                        = 0x5A1
+	SYS_EXPORTWORKUNIT                  = 0x5A1
+	SYS_EXTRACTW                        = 0x5A5
+	SYS_EXTRACTWORKUNIT                 = 0x5A5
+	SYS_FSEEKO                          = 0x5C9
+	SYS_FTELLO                          = 0x5C8
+	SYS_GETGRGID_R                      = 0x5E7
+	SYS_GETGRNAM_R                      = 0x5E8
+	SYS_GETLOGIN_R                      = 0x5E9
+	SYS_GETPWNAM_R                      = 0x5EA
+	SYS_GETPWUID_R                      = 0x5EB
+	SYS_GMTIME_R                        = 0x5E2
+	SYS_IMPORTWO                        = 0x5A3
+	SYS_IMPORTWORKUNIT                  = 0x5A3
+	SYS_INET_NTOP                       = 0x5D3
+	SYS_INET_PTON                       = 0x5D4
+	SYS_LLABS                           = 0x5CE
+	SYS_LLDIV                           = 0x5CB
+	SYS_LOCALTIME_R                     = 0x5E3
+	SYS_PTHREAD_ATFORK                  = 0x5ED
+	SYS_PTHREAD_ATTR_GETDETACHSTATE_U98 = 0x5FB
+	SYS_PTHREAD_ATTR_GETGUARDSIZE       = 0x5EE
+	SYS_PTHREAD_ATTR_GETSCHEDPARAM      = 0x5F9
+	SYS_PTHREAD_ATTR_GETSTACKADDR       = 0x5EF
+	SYS_PTHREAD_ATTR_SETDETACHSTATE_U98 = 0x5FC
+	SYS_PTHREAD_ATTR_SETGUARDSIZE       = 0x5F0
+	SYS_PTHREAD_ATTR_SETSCHEDPARAM      = 0x5FA
+	SYS_PTHREAD_ATTR_SETSTACKADDR       = 0x5F1
+	SYS_PTHREAD_CONDATTR_GETPSHARED     = 0x5F2
+	SYS_PTHREAD_CONDATTR_SETPSHARED     = 0x5F3
+	SYS_PTHREAD_DETACH_U98              = 0x5FD
+	SYS_PTHREAD_GETCONCURRENCY          = 0x5F4
+	SYS_PTHREAD_GETSPECIFIC_U98         = 0x5FE
+	SYS_PTHREAD_KEY_DELETE              = 0x5F5
+	SYS_PTHREAD_SETCANCELSTATE          = 0x5FF
+	SYS_PTHREAD_SETCONCURRENCY          = 0x5F6
+	SYS_PTHREAD_SIGMASK                 = 0x5F7
+	SYS_QUERYENC                        = 0x5AD
+	SYS_QUERYWORKUNITCLASSIFICATION     = 0x5AD
+	SYS_RAISE                           = 0x05E
+	SYS_RAND_R                          = 0x5E4
+	SYS_READDIR_R                       = 0x5E6
+	SYS_REALLOC                         = 0x05B
+	SYS_RES_INIT                        = 0x5D8
+	SYS_RES_MKQUERY                     = 0x5D7
+	SYS_RES_QUERY                       = 0x5D9
+	SYS_RES_QUERYDOMAIN                 = 0x5DC
+	SYS_RES_SEARCH                      = 0x5DA
+	SYS_RES_SEND                        = 0x5DB
+	SYS_SETJMP                          = 0x05F
+	SYS_SIGQUEUE                        = 0x5A9
+	SYS_STRTOK_R                        = 0x5E5
+	SYS_STRTOLL                         = 0x5B0
+	SYS_STRTOULL                        = 0x5B1
+	SYS_TTYNAME_R                       = 0x5EC
+	SYS_UNDOEXPO                        = 0x5A2
+	SYS_UNDOEXPORTWORKUNIT              = 0x5A2
+	SYS_UNDOIMPO                        = 0x5A4
+	SYS_UNDOIMPORTWORKUNIT              = 0x5A4
+	SYS_WCSTOLL                         = 0x5CC
+	SYS_WCSTOULL                        = 0x5CD
+	SYS___ABORT                         = 0x05C
+	SYS___CONSOLE2                      = 0x5D2
+	SYS___CPL                           = 0x5A6
+	SYS___DISCARDDATA                   = 0x5F8
+	SYS___DSA_PREV                      = 0x5B2
+	SYS___EP_FIND                       = 0x5B3
+	SYS___FP_SWAPMODE                   = 0x5AF
+	SYS___GETUSERID                     = 0x5AB
+	SYS___GET_CPUID                     = 0x5B9
+	SYS___GET_SYSTEM_SETTINGS           = 0x5BA
+	SYS___IPDOMAINNAME                  = 0x5AC
+	SYS___MAP_INIT                      = 0x5A7
+	SYS___MAP_SERVICE                   = 0x5A8
+	SYS___MOUNT                         = 0x5AA
+	SYS___MSGRCV_TIMED                  = 0x5B7
+	SYS___RES                           = 0x5D6
+	SYS___SEMOP_TIMED                   = 0x5B8
+	SYS___SERVER_THREADS_QUERY          = 0x5B4
+	SYS_FPRINTF                         = 0x06D
+	SYS_FSCANF                          = 0x06A
+	SYS_PRINTF                          = 0x06F
+	SYS_SETBUF                          = 0x06B
+	SYS_SETVBUF                         = 0x06C
+	SYS_SSCANF                          = 0x06E
+	SYS___CATGETS_A                     = 0x6C0
+	SYS___CHAUDIT_A                     = 0x6F4
+	SYS___CHMOD_A                       = 0x6E8
+	SYS___COLLATE_INIT_A                = 0x6AC
+	SYS___CREAT_A                       = 0x6F6
+	SYS___CTYPE_INIT_A                  = 0x6AF
+	SYS___DLLLOAD_A                     = 0x6DF
+	SYS___DLLQUERYFN_A                  = 0x6E0
+	SYS___DLLQUERYVAR_A                 = 0x6E1
+	SYS___E2A_L                         = 0x6E3
+	SYS___EXECLE_A                      = 0x6A0
+	SYS___EXECLP_A                      = 0x6A4
+	SYS___EXECVE_A                      = 0x6C1
+	SYS___EXECVP_A                      = 0x6C2
+	SYS___EXECV_A                       = 0x6B1
+	SYS___FPRINTF_A                     = 0x6FA
+	SYS___GETADDRINFO_A                 = 0x6BF
+	SYS___GETNAMEINFO_A                 = 0x6C4
+	SYS___GET_WCTYPE_STD_A              = 0x6AE
+	SYS___ICONV_OPEN_A                  = 0x6DE
+	SYS___IF_INDEXTONAME_A              = 0x6DC
+	SYS___IF_NAMETOINDEX_A              = 0x6DB
+	SYS___ISWCTYPE_A                    = 0x6B0
+	SYS___IS_WCTYPE_STD_A               = 0x6B2
+	SYS___LOCALECONV_A                  = 0x6B8
+	SYS___LOCALECONV_STD_A              = 0x6B9
+	SYS___LOCALE_INIT_A                 = 0x6B7
+	SYS___LSTAT_A                       = 0x6EE
+	SYS___LSTAT_O_A                     = 0x6EF
+	SYS___MKDIR_A                       = 0x6E9
+	SYS___MKFIFO_A                      = 0x6EC
+	SYS___MKNOD_A                       = 0x6F0
+	SYS___MONETARY_INIT_A               = 0x6BC
+	SYS___MOUNT_A                       = 0x6F1
+	SYS___NL_CSINFO_A                   = 0x6D6
+	SYS___NL_LANGINFO_A                 = 0x6BA
+	SYS___NL_LNAGINFO_STD_A             = 0x6BB
+	SYS___NL_MONINFO_A                  = 0x6D7
+	SYS___NL_NUMINFO_A                  = 0x6D8
+	SYS___NL_RESPINFO_A                 = 0x6D9
+	SYS___NL_TIMINFO_A                  = 0x6DA
+	SYS___NUMERIC_INIT_A                = 0x6C6
+	SYS___OPEN_A                        = 0x6F7
+	SYS___PRINTF_A                      = 0x6DD
+	SYS___RESP_INIT_A                   = 0x6C7
+	SYS___RPMATCH_A                     = 0x6C8
+	SYS___RPMATCH_C_A                   = 0x6C9
+	SYS___RPMATCH_STD_A                 = 0x6CA
+	SYS___SETLOCALE_A                   = 0x6F9
+	SYS___SPAWNP_A                      = 0x6C5
+	SYS___SPAWN_A                       = 0x6C3
+	SYS___SPRINTF_A                     = 0x6FB
+	SYS___STAT_A                        = 0x6EA
+	SYS___STAT_O_A                      = 0x6EB
+	SYS___STRCOLL_STD_A                 = 0x6A1
+	SYS___STRFMON_A                     = 0x6BD
+	SYS___STRFMON_STD_A                 = 0x6BE
+	SYS___STRFTIME_A                    = 0x6CC
+	SYS___STRFTIME_STD_A                = 0x6CD
+	SYS___STRPTIME_A                    = 0x6CE
+	SYS___STRPTIME_STD_A                = 0x6CF
+	SYS___STRXFRM_A                     = 0x6A2
+	SYS___STRXFRM_C_A                   = 0x6A3
+	SYS___STRXFRM_STD_A                 = 0x6A5
+	SYS___SYNTAX_INIT_A                 = 0x6D4
+	SYS___TIME_INIT_A                   = 0x6CB
+	SYS___TOD_INIT_A                    = 0x6D5
+	SYS___TOWLOWER_A                    = 0x6B3
+	SYS___TOWLOWER_STD_A                = 0x6B4
+	SYS___TOWUPPER_A                    = 0x6B5
+	SYS___TOWUPPER_STD_A                = 0x6B6
+	SYS___UMOUNT_A                      = 0x6F2
+	SYS___VFPRINTF_A                    = 0x6FC
+	SYS___VPRINTF_A                     = 0x6FD
+	SYS___VSPRINTF_A                    = 0x6FE
+	SYS___VSWPRINTF_A                   = 0x6FF
+	SYS___WCSCOLL_A                     = 0x6A6
+	SYS___WCSCOLL_C_A                   = 0x6A7
+	SYS___WCSCOLL_STD_A                 = 0x6A8
+	SYS___WCSFTIME_A                    = 0x6D0
+	SYS___WCSFTIME_STD_A                = 0x6D1
+	SYS___WCSXFRM_A                     = 0x6A9
+	SYS___WCSXFRM_C_A                   = 0x6AA
+	SYS___WCSXFRM_STD_A                 = 0x6AB
+	SYS___WCTYPE_A                      = 0x6AD
+	SYS___W_GETMNTENT_A                 = 0x6F5
+	SYS_____CCSIDTYPE_A                 = 0x6E6
+	SYS_____CHATTR_A                    = 0x6E2
+	SYS_____CSNAMETYPE_A                = 0x6E7
+	SYS_____OPEN_STAT_A                 = 0x6ED
+	SYS_____SPAWN2_A                    = 0x6D2
+	SYS_____SPAWNP2_A                   = 0x6D3
+	SYS_____TOCCSID_A                   = 0x6E4
+	SYS_____TOCSNAME_A                  = 0x6E5
+	SYS_ACL_FREE                        = 0x7FF
+	SYS_ACL_INIT                        = 0x7FE
+	SYS_FWIDE                           = 0x7DF
+	SYS_FWPRINTF                        = 0x7D1
+	SYS_FWRITE                          = 0x07E
+	SYS_FWSCANF                         = 0x7D5
+	SYS_GETCHAR                         = 0x07B
+	SYS_GETS                            = 0x07C
+	SYS_M_CREATE_LAYOUT                 = 0x7C9
+	SYS_M_DESTROY_LAYOUT                = 0x7CA
+	SYS_M_GETVALUES_LAYOUT              = 0x7CB
+	SYS_M_SETVALUES_LAYOUT              = 0x7CC
+	SYS_M_TRANSFORM_LAYOUT              = 0x7CD
+	SYS_M_WTRANSFORM_LAYOUT             = 0x7CE
+	SYS_PREAD                           = 0x7C7
+	SYS_PUTC                            = 0x07D
+	SYS_PUTCHAR                         = 0x07A
+	SYS_PUTS                            = 0x07F
+	SYS_PWRITE                          = 0x7C8
+	SYS_TOWCTRAN                        = 0x7D8
+	SYS_TOWCTRANS                       = 0x7D8
+	SYS_UNATEXIT                        = 0x7B5
+	SYS_VFWPRINT                        = 0x7D3
+	SYS_VFWPRINTF                       = 0x7D3
+	SYS_VWPRINTF                        = 0x7D4
+	SYS_WCTRANS                         = 0x7D7
+	SYS_WPRINTF                         = 0x7D2
+	SYS_WSCANF                          = 0x7D6
+	SYS___ASCTIME_R_A                   = 0x7A1
+	SYS___BASENAME_A                    = 0x7DC
+	SYS___BTOWC_A                       = 0x7E4
+	SYS___CDUMP_A                       = 0x7B7
+	SYS___CEE3DMP_A                     = 0x7B6
+	SYS___CEILF_H                       = 0x7F4
+	SYS___CEILL_H                       = 0x7F5
+	SYS___CEIL_H                        = 0x7EA
+	SYS___CRYPT_A                       = 0x7BE
+	SYS___CSNAP_A                       = 0x7B8
+	SYS___CTEST_A                       = 0x7B9
+	SYS___CTIME_R_A                     = 0x7A2
+	SYS___CTRACE_A                      = 0x7BA
+	SYS___DBM_OPEN_A                    = 0x7E6
+	SYS___DIRNAME_A                     = 0x7DD
+	SYS___FABSF_H                       = 0x7FA
+	SYS___FABSL_H                       = 0x7FB
+	SYS___FABS_H                        = 0x7ED
+	SYS___FGETWC_A                      = 0x7AA
+	SYS___FGETWS_A                      = 0x7AD
+	SYS___FLOORF_H                      = 0x7F6
+	SYS___FLOORL_H                      = 0x7F7
+	SYS___FLOOR_H                       = 0x7EB
+	SYS___FPUTWC_A                      = 0x7A5
+	SYS___FPUTWS_A                      = 0x7A8
+	SYS___GETTIMEOFDAY_A                = 0x7AE
+	SYS___GETWCHAR_A                    = 0x7AC
+	SYS___GETWC_A                       = 0x7AB
+	SYS___GLOB_A                        = 0x7DE
+	SYS___GMTIME_A                      = 0x7AF
+	SYS___GMTIME_R_A                    = 0x7B0
+	SYS___INET_PTON_A                   = 0x7BC
+	SYS___J0_H                          = 0x7EE
+	SYS___J1_H                          = 0x7EF
+	SYS___JN_H                          = 0x7F0
+	SYS___LOCALTIME_A                   = 0x7B1
+	SYS___LOCALTIME_R_A                 = 0x7B2
+	SYS___MALLOC24                      = 0x7FC
+	SYS___MALLOC31                      = 0x7FD
+	SYS___MKTIME_A                      = 0x7B3
+	SYS___MODFF_H                       = 0x7F8
+	SYS___MODFL_H                       = 0x7F9
+	SYS___MODF_H                        = 0x7EC
+	SYS___OPENDIR_A                     = 0x7C2
+	SYS___OSNAME                        = 0x7E0
+	SYS___PUTWCHAR_A                    = 0x7A7
+	SYS___PUTWC_A                       = 0x7A6
+	SYS___READDIR_A                     = 0x7C3
+	SYS___STRTOLL_A                     = 0x7A3
+	SYS___STRTOULL_A                    = 0x7A4
+	SYS___SYSLOG_A                      = 0x7BD
+	SYS___TZZNA                         = 0x7B4
+	SYS___UNGETWC_A                     = 0x7A9
+	SYS___UTIME_A                       = 0x7A0
+	SYS___VFPRINTF2_A                   = 0x7E7
+	SYS___VPRINTF2_A                    = 0x7E8
+	SYS___VSPRINTF2_A                   = 0x7E9
+	SYS___VSWPRNTF2_A                   = 0x7BB
+	SYS___WCSTOD_A                      = 0x7D9
+	SYS___WCSTOL_A                      = 0x7DA
+	SYS___WCSTOUL_A                     = 0x7DB
+	SYS___WCTOB_A                       = 0x7E5
+	SYS___Y0_H                          = 0x7F1
+	SYS___Y1_H                          = 0x7F2
+	SYS___YN_H                          = 0x7F3
+	SYS_____OPENDIR2_A                  = 0x7BF
+	SYS_____OSNAME_A                    = 0x7E1
+	SYS_____READDIR2_A                  = 0x7C0
+	SYS_DLCLOSE                         = 0x8DF
+	SYS_DLERROR                         = 0x8E0
+	SYS_DLOPEN                          = 0x8DD
+	SYS_DLSYM                           = 0x8DE
+	SYS_FLOCKFILE                       = 0x8D3
+	SYS_FTRYLOCKFILE                    = 0x8D4
+	SYS_FUNLOCKFILE                     = 0x8D5
+	SYS_GETCHAR_UNLOCKED                = 0x8D7
+	SYS_GETC_UNLOCKED                   = 0x8D6
+	SYS_PUTCHAR_UNLOCKED                = 0x8D9
+	SYS_PUTC_UNLOCKED                   = 0x8D8
+	SYS_SNPRINTF                        = 0x8DA
+	SYS_VSNPRINTF                       = 0x8DB
+	SYS_WCSCSPN                         = 0x08B
+	SYS_WCSLEN                          = 0x08C
+	SYS_WCSNCAT                         = 0x08D
+	SYS_WCSNCMP                         = 0x08A
+	SYS_WCSNCPY                         = 0x08F
+	SYS_WCSSPN                          = 0x08E
+	SYS___ABSF_H                        = 0x8E7
+	SYS___ABSL_H                        = 0x8E8
+	SYS___ABS_H                         = 0x8E6
+	SYS___ACOSF_H                       = 0x8EA
+	SYS___ACOSH_H                       = 0x8EC
+	SYS___ACOSL_H                       = 0x8EB
+	SYS___ACOS_H                        = 0x8E9
+	SYS___ASINF_H                       = 0x8EE
+	SYS___ASINH_H                       = 0x8F0
+	SYS___ASINL_H                       = 0x8EF
+	SYS___ASIN_H                        = 0x8ED
+	SYS___ATAN2F_H                      = 0x8F8
+	SYS___ATAN2L_H                      = 0x8F9
+	SYS___ATAN2_H                       = 0x8F7
+	SYS___ATANF_H                       = 0x8F2
+	SYS___ATANHF_H                      = 0x8F5
+	SYS___ATANHL_H                      = 0x8F6
+	SYS___ATANH_H                       = 0x8F4
+	SYS___ATANL_H                       = 0x8F3
+	SYS___ATAN_H                        = 0x8F1
+	SYS___CBRT_H                        = 0x8FA
+	SYS___COPYSIGNF_H                   = 0x8FB
+	SYS___COPYSIGNL_H                   = 0x8FC
+	SYS___COSF_H                        = 0x8FE
+	SYS___COSL_H                        = 0x8FF
+	SYS___COS_H                         = 0x8FD
+	SYS___DLERROR_A                     = 0x8D2
+	SYS___DLOPEN_A                      = 0x8D0
+	SYS___DLSYM_A                       = 0x8D1
+	SYS___GETUTXENT_A                   = 0x8C6
+	SYS___GETUTXID_A                    = 0x8C7
+	SYS___GETUTXLINE_A                  = 0x8C8
+	SYS___ITOA                          = 0x8AA
+	SYS___ITOA_A                        = 0x8B0
+	SYS___LE_CONDITION_TOKEN_BUILD      = 0x8A5
+	SYS___LE_MSG_ADD_INSERT             = 0x8A6
+	SYS___LE_MSG_GET                    = 0x8A7
+	SYS___LE_MSG_GET_AND_WRITE          = 0x8A8
+	SYS___LE_MSG_WRITE                  = 0x8A9
+	SYS___LLTOA                         = 0x8AE
+	SYS___LLTOA_A                       = 0x8B4
+	SYS___LTOA                          = 0x8AC
+	SYS___LTOA_A                        = 0x8B2
+	SYS___PUTCHAR_UNLOCKED_A            = 0x8CC
+	SYS___PUTC_UNLOCKED_A               = 0x8CB
+	SYS___PUTUTXLINE_A                  = 0x8C9
+	SYS___RESET_EXCEPTION_HANDLER       = 0x8E3
+	SYS___REXEC_A                       = 0x8C4
+	SYS___REXEC_AF_A                    = 0x8C5
+	SYS___SET_EXCEPTION_HANDLER         = 0x8E2
+	SYS___SNPRINTF_A                    = 0x8CD
+	SYS___SUPERKILL                     = 0x8A4
+	SYS___TCGETATTR_A                   = 0x8A1
+	SYS___TCSETATTR_A                   = 0x8A2
+	SYS___ULLTOA                        = 0x8AF
+	SYS___ULLTOA_A                      = 0x8B5
+	SYS___ULTOA                         = 0x8AD
+	SYS___ULTOA_A                       = 0x8B3
+	SYS___UTOA                          = 0x8AB
+	SYS___UTOA_A                        = 0x8B1
+	SYS___VHM_EVENT                     = 0x8E4
+	SYS___VSNPRINTF_A                   = 0x8CE
+	SYS_____GETENV_A                    = 0x8C3
+	SYS_____UTMPXNAME_A                 = 0x8CA
+	SYS_CACOSH                          = 0x9A0
+	SYS_CACOSHF                         = 0x9A3
+	SYS_CACOSHL                         = 0x9A6
+	SYS_CARG                            = 0x9A9
+	SYS_CARGF                           = 0x9AC
+	SYS_CARGL                           = 0x9AF
+	SYS_CASIN                           = 0x9B2
+	SYS_CASINF                          = 0x9B5
+	SYS_CASINH                          = 0x9BB
+	SYS_CASINHF                         = 0x9BE
+	SYS_CASINHL                         = 0x9C1
+	SYS_CASINL                          = 0x9B8
+	SYS_CATAN                           = 0x9C4
+	SYS_CATANF                          = 0x9C7
+	SYS_CATANH                          = 0x9CD
+	SYS_CATANHF                         = 0x9D0
+	SYS_CATANHL                         = 0x9D3
+	SYS_CATANL                          = 0x9CA
+	SYS_CCOS                            = 0x9D6
+	SYS_CCOSF                           = 0x9D9
+	SYS_CCOSH                           = 0x9DF
+	SYS_CCOSHF                          = 0x9E2
+	SYS_CCOSHL                          = 0x9E5
+	SYS_CCOSL                           = 0x9DC
+	SYS_CEXP                            = 0x9E8
+	SYS_CEXPF                           = 0x9EB
+	SYS_CEXPL                           = 0x9EE
+	SYS_CIMAG                           = 0x9F1
+	SYS_CIMAGF                          = 0x9F4
+	SYS_CIMAGL                          = 0x9F7
+	SYS_CLOGF                           = 0x9FD
+	SYS_MEMCHR                          = 0x09B
+	SYS_MEMCMP                          = 0x09A
+	SYS_STRCOLL                         = 0x09C
+	SYS_STRNCMP                         = 0x09D
+	SYS_STRRCHR                         = 0x09F
+	SYS_STRXFRM                         = 0x09E
+	SYS___CACOSHF_B                     = 0x9A4
+	SYS___CACOSHF_H                     = 0x9A5
+	SYS___CACOSHL_B                     = 0x9A7
+	SYS___CACOSHL_H                     = 0x9A8
+	SYS___CACOSH_B                      = 0x9A1
+	SYS___CACOSH_H                      = 0x9A2
+	SYS___CARGF_B                       = 0x9AD
+	SYS___CARGF_H                       = 0x9AE
+	SYS___CARGL_B                       = 0x9B0
+	SYS___CARGL_H                       = 0x9B1
+	SYS___CARG_B                        = 0x9AA
+	SYS___CARG_H                        = 0x9AB
+	SYS___CASINF_B                      = 0x9B6
+	SYS___CASINF_H                      = 0x9B7
+	SYS___CASINHF_B                     = 0x9BF
+	SYS___CASINHF_H                     = 0x9C0
+	SYS___CASINHL_B                     = 0x9C2
+	SYS___CASINHL_H                     = 0x9C3
+	SYS___CASINH_B                      = 0x9BC
+	SYS___CASINH_H                      = 0x9BD
+	SYS___CASINL_B                      = 0x9B9
+	SYS___CASINL_H                      = 0x9BA
+	SYS___CASIN_B                       = 0x9B3
+	SYS___CASIN_H                       = 0x9B4
+	SYS___CATANF_B                      = 0x9C8
+	SYS___CATANF_H                      = 0x9C9
+	SYS___CATANHF_B                     = 0x9D1
+	SYS___CATANHF_H                     = 0x9D2
+	SYS___CATANHL_B                     = 0x9D4
+	SYS___CATANHL_H                     = 0x9D5
+	SYS___CATANH_B                      = 0x9CE
+	SYS___CATANH_H                      = 0x9CF
+	SYS___CATANL_B                      = 0x9CB
+	SYS___CATANL_H                      = 0x9CC
+	SYS___CATAN_B                       = 0x9C5
+	SYS___CATAN_H                       = 0x9C6
+	SYS___CCOSF_B                       = 0x9DA
+	SYS___CCOSF_H                       = 0x9DB
+	SYS___CCOSHF_B                      = 0x9E3
+	SYS___CCOSHF_H                      = 0x9E4
+	SYS___CCOSHL_B                      = 0x9E6
+	SYS___CCOSHL_H                      = 0x9E7
+	SYS___CCOSH_B                       = 0x9E0
+	SYS___CCOSH_H                       = 0x9E1
+	SYS___CCOSL_B                       = 0x9DD
+	SYS___CCOSL_H                       = 0x9DE
+	SYS___CCOS_B                        = 0x9D7
+	SYS___CCOS_H                        = 0x9D8
+	SYS___CEXPF_B                       = 0x9EC
+	SYS___CEXPF_H                       = 0x9ED
+	SYS___CEXPL_B                       = 0x9EF
+	SYS___CEXPL_H                       = 0x9F0
+	SYS___CEXP_B                        = 0x9E9
+	SYS___CEXP_H                        = 0x9EA
+	SYS___CIMAGF_B                      = 0x9F5
+	SYS___CIMAGF_H                      = 0x9F6
+	SYS___CIMAGL_B                      = 0x9F8
+	SYS___CIMAGL_H                      = 0x9F9
+	SYS___CIMAG_B                       = 0x9F2
+	SYS___CIMAG_H                       = 0x9F3
+	SYS___CLOG                          = 0x9FA
+	SYS___CLOGF_B                       = 0x9FE
+	SYS___CLOGF_H                       = 0x9FF
+	SYS___CLOG_B                        = 0x9FB
+	SYS___CLOG_H                        = 0x9FC
+	SYS_ISWCTYPE                        = 0x10C
+	SYS_ISWXDIGI                        = 0x10A
+	SYS_ISWXDIGIT                       = 0x10A
+	SYS_MBSINIT                         = 0x10F
+	SYS_TOWLOWER                        = 0x10D
+	SYS_TOWUPPER                        = 0x10E
+	SYS_WCTYPE                          = 0x10B
+	SYS_WCSSTR                          = 0x11B
+	SYS___RPMTCH                        = 0x11A
+	SYS_WCSTOD                          = 0x12E
+	SYS_WCSTOK                          = 0x12C
+	SYS_WCSTOL                          = 0x12D
+	SYS_WCSTOUL                         = 0x12F
+	SYS_FGETWC                          = 0x13C
+	SYS_FGETWS                          = 0x13D
+	SYS_FPUTWC                          = 0x13E
+	SYS_FPUTWS                          = 0x13F
+	SYS_REGERROR                        = 0x13B
+	SYS_REGFREE                         = 0x13A
+	SYS_COLLEQUIV                       = 0x14F
+	SYS_COLLTOSTR                       = 0x14E
+	SYS_ISMCCOLLEL                      = 0x14C
+	SYS_STRTOCOLL                       = 0x14D
+	SYS_DLLFREE                         = 0x16F
+	SYS_DLLQUERYFN                      = 0x16D
+	SYS_DLLQUERYVAR                     = 0x16E
+	SYS_GETMCCOLL                       = 0x16A
+	SYS_GETWMCCOLL                      = 0x16B
+	SYS___ERR2AD                        = 0x16C
+	SYS_CFSETOSPEED                     = 0x17A
+	SYS_CHDIR                           = 0x17B
+	SYS_CHMOD                           = 0x17C
+	SYS_CHOWN                           = 0x17D
+	SYS_CLOSE                           = 0x17E
+	SYS_CLOSEDIR                        = 0x17F
+	SYS_LOG                             = 0x017
+	SYS_COSH                            = 0x018
+	SYS_FCHMOD                          = 0x18A
+	SYS_FCHOWN                          = 0x18B
+	SYS_FCNTL                           = 0x18C
+	SYS_FILENO                          = 0x18D
+	SYS_FORK                            = 0x18E
+	SYS_FPATHCONF                       = 0x18F
+	SYS_GETLOGIN                        = 0x19A
+	SYS_GETPGRP                         = 0x19C
+	SYS_GETPID                          = 0x19D
+	SYS_GETPPID                         = 0x19E
+	SYS_GETPWNAM                        = 0x19F
+	SYS_TANH                            = 0x019
+	SYS_W_GETMNTENT                     = 0x19B
+	SYS_POW                             = 0x020
+	SYS_PTHREAD_SELF                    = 0x20A
+	SYS_PTHREAD_SETINTR                 = 0x20B
+	SYS_PTHREAD_SETINTRTYPE             = 0x20C
+	SYS_PTHREAD_SETSPECIFIC             = 0x20D
+	SYS_PTHREAD_TESTINTR                = 0x20E
+	SYS_PTHREAD_YIELD                   = 0x20F
+	SYS_SQRT                            = 0x021
+	SYS_FLOOR                           = 0x022
+	SYS_J1                              = 0x023
+	SYS_WCSPBRK                         = 0x23F
+	SYS_BSEARCH                         = 0x24C
+	SYS_FABS                            = 0x024
+	SYS_GETENV                          = 0x24A
+	SYS_LDIV                            = 0x24D
+	SYS_SYSTEM                          = 0x24B
+	SYS_FMOD                            = 0x025
+	SYS___RETHROW                       = 0x25F
+	SYS___THROW                         = 0x25E
+	SYS_J0                              = 0x026
+	SYS_PUTENV                          = 0x26A
+	SYS___GETENV                        = 0x26F
+	SYS_SEMCTL                          = 0x27A
+	SYS_SEMGET                          = 0x27B
+	SYS_SEMOP                           = 0x27C
+	SYS_SHMAT                           = 0x27D
+	SYS_SHMCTL                          = 0x27E
+	SYS_SHMDT                           = 0x27F
+	SYS_YN                              = 0x027
+	SYS_JN                              = 0x028
+	SYS_SIGALTSTACK                     = 0x28A
+	SYS_SIGHOLD                         = 0x28B
+	SYS_SIGIGNORE                       = 0x28C
+	SYS_SIGINTERRUPT                    = 0x28D
+	SYS_SIGPAUSE                        = 0x28E
+	SYS_SIGRELSE                        = 0x28F
+	SYS_GETOPT                          = 0x29A
+	SYS_GETSUBOPT                       = 0x29D
+	SYS_LCHOWN                          = 0x29B
+	SYS_SETPGRP                         = 0x29E
+	SYS_TRUNCATE                        = 0x29C
+	SYS_Y0                              = 0x029
+	SYS___GDERR                         = 0x29F
+	SYS_ISALPHA                         = 0x030
+	SYS_VFORK                           = 0x30F
+	SYS__LONGJMP                        = 0x30D
+	SYS__SETJMP                         = 0x30E
+	SYS_GLOB                            = 0x31A
+	SYS_GLOBFREE                        = 0x31B
+	SYS_ISALNUM                         = 0x031
+	SYS_PUTW                            = 0x31C
+	SYS_SEEKDIR                         = 0x31D
+	SYS_TELLDIR                         = 0x31E
+	SYS_TEMPNAM                         = 0x31F
+	SYS_GETTIMEOFDAY_R                  = 0x32E
+	SYS_ISLOWER                         = 0x032
+	SYS_LGAMMA                          = 0x32C
+	SYS_REMAINDER                       = 0x32A
+	SYS_SCALB                           = 0x32B
+	SYS_SYNC                            = 0x32F
+	SYS_TTYSLOT                         = 0x32D
+	SYS_ENDPROTOENT                     = 0x33A
+	SYS_ENDSERVENT                      = 0x33B
+	SYS_GETHOSTBYADDR                   = 0x33D
+	SYS_GETHOSTBYADDR_R                 = 0x33C
+	SYS_GETHOSTBYNAME                   = 0x33F
+	SYS_GETHOSTBYNAME_R                 = 0x33E
+	SYS_ISCNTRL                         = 0x033
+	SYS_GETSERVBYNAME                   = 0x34A
+	SYS_GETSERVBYPORT                   = 0x34B
+	SYS_GETSERVENT                      = 0x34C
+	SYS_GETSOCKNAME                     = 0x34D
+	SYS_GETSOCKOPT                      = 0x34E
+	SYS_INET_ADDR                       = 0x34F
+	SYS_ISDIGIT                         = 0x034
+	SYS_ISGRAPH                         = 0x035
+	SYS_SELECT                          = 0x35B
+	SYS_SELECTEX                        = 0x35C
+	SYS_SEND                            = 0x35D
+	SYS_SENDTO                          = 0x35F
+	SYS_CHROOT                          = 0x36A
+	SYS_ISNAN                           = 0x36D
+	SYS_ISUPPER                         = 0x036
+	SYS_ULIMIT                          = 0x36C
+	SYS_UTIMES                          = 0x36E
+	SYS_W_STATVFS                       = 0x36B
+	SYS___H_ERRNO                       = 0x36F
+	SYS_GRANTPT                         = 0x37A
+	SYS_ISPRINT                         = 0x037
+	SYS_TCGETSID                        = 0x37C
+	SYS_UNLOCKPT                        = 0x37B
+	SYS___TCGETCP                       = 0x37D
+	SYS___TCSETCP                       = 0x37E
+	SYS___TCSETTABLES                   = 0x37F
+	SYS_ISPUNCT                         = 0x038
+	SYS_NLIST                           = 0x38C
+	SYS___IPDBCS                        = 0x38D
+	SYS___IPDSPX                        = 0x38E
+	SYS___IPMSGC                        = 0x38F
+	SYS___STHOSTENT                     = 0x38B
+	SYS___STSERVENT                     = 0x38A
+	SYS_ISSPACE                         = 0x039
+	SYS_COS                             = 0x040
+	SYS_T_ALLOC                         = 0x40A
+	SYS_T_BIND                          = 0x40B
+	SYS_T_CLOSE                         = 0x40C
+	SYS_T_CONNECT                       = 0x40D
+	SYS_T_ERROR                         = 0x40E
+	SYS_T_FREE                          = 0x40F
+	SYS_TAN                             = 0x041
+	SYS_T_RCVREL                        = 0x41A
+	SYS_T_RCVUDATA                      = 0x41B
+	SYS_T_RCVUDERR                      = 0x41C
+	SYS_T_SND                           = 0x41D
+	SYS_T_SNDDIS                        = 0x41E
+	SYS_T_SNDREL                        = 0x41F
+	SYS_GETPMSG                         = 0x42A
+	SYS_ISASTREAM                       = 0x42B
+	SYS_PUTMSG                          = 0x42C
+	SYS_PUTPMSG                         = 0x42D
+	SYS_SINH                            = 0x042
+	SYS___ISPOSIXON                     = 0x42E
+	SYS___OPENMVSREL                    = 0x42F
+	SYS_ACOS                            = 0x043
+	SYS_ATAN                            = 0x044
+	SYS_ATAN2                           = 0x045
+	SYS_FTELL                           = 0x046
+	SYS_FGETPOS                         = 0x047
+	SYS_SOCK_DEBUG                      = 0x47A
+	SYS_SOCK_DO_TESTSTOR                = 0x47D
+	SYS_TAKESOCKET                      = 0x47E
+	SYS___SERVER_INIT                   = 0x47F
+	SYS_FSEEK                           = 0x048
+	SYS___IPHOST                        = 0x48B
+	SYS___IPNODE                        = 0x48C
+	SYS___SERVER_CLASSIFY_CREATE        = 0x48D
+	SYS___SERVER_CLASSIFY_DESTROY       = 0x48E
+	SYS___SERVER_CLASSIFY_RESET         = 0x48F
+	SYS___SMF_RECORD                    = 0x48A
+	SYS_FSETPOS                         = 0x049
+	SYS___FNWSA                         = 0x49B
+	SYS___SPAWN2                        = 0x49D
+	SYS___SPAWNP2                       = 0x49E
+	SYS_ATOF                            = 0x050
+	SYS_PTHREAD_MUTEXATTR_GETPSHARED    = 0x50A
+	SYS_PTHREAD_MUTEXATTR_SETPSHARED    = 0x50B
+	SYS_PTHREAD_RWLOCK_DESTROY          = 0x50C
+	SYS_PTHREAD_RWLOCK_INIT             = 0x50D
+	SYS_PTHREAD_RWLOCK_RDLOCK           = 0x50E
+	SYS_PTHREAD_RWLOCK_TRYRDLOCK        = 0x50F
+	SYS_ATOI                            = 0x051
+	SYS___FP_CLASS                      = 0x51D
+	SYS___FP_CLR_FLAG                   = 0x51A
+	SYS___FP_FINITE                     = 0x51E
+	SYS___FP_ISNAN                      = 0x51F
+	SYS___FP_RAISE_XCP                  = 0x51C
+	SYS___FP_READ_FLAG                  = 0x51B
+	SYS_RAND                            = 0x052
+	SYS_SIGTIMEDWAIT                    = 0x52D
+	SYS_SIGWAITINFO                     = 0x52E
+	SYS___CHKBFP                        = 0x52F
+	SYS___FPC_RS                        = 0x52C
+	SYS___FPC_RW                        = 0x52A
+	SYS___FPC_SM                        = 0x52B
+	SYS_STRTOD                          = 0x053
+	SYS_STRTOL                          = 0x054
+	SYS_STRTOUL                         = 0x055
+	SYS_MALLOC                          = 0x056
+	SYS_SRAND                           = 0x057
+	SYS_CALLOC                          = 0x058
+	SYS_FREE                            = 0x059
+	SYS___OSENV                         = 0x59F
+	SYS___W_PIOCTL                      = 0x59E
+	SYS_LONGJMP                         = 0x060
+	SYS___FLOORF_B                      = 0x60A
+	SYS___FLOORL_B                      = 0x60B
+	SYS___FREXPF_B                      = 0x60C
+	SYS___FREXPL_B                      = 0x60D
+	SYS___LDEXPF_B                      = 0x60E
+	SYS___LDEXPL_B                      = 0x60F
+	SYS_SIGNAL                          = 0x061
+	SYS___ATAN2F_B                      = 0x61A
+	SYS___ATAN2L_B                      = 0x61B
+	SYS___COSHF_B                       = 0x61C
+	SYS___COSHL_B                       = 0x61D
+	SYS___EXPF_B                        = 0x61E
+	SYS___EXPL_B                        = 0x61F
+	SYS_TMPNAM                          = 0x062
+	SYS___ABSF_B                        = 0x62A
+	SYS___ABSL_B                        = 0x62C
+	SYS___ABS_B                         = 0x62B
+	SYS___FMODF_B                       = 0x62D
+	SYS___FMODL_B                       = 0x62E
+	SYS___MODFF_B                       = 0x62F
+	SYS_ATANL                           = 0x63A
+	SYS_CEILF                           = 0x63B
+	SYS_CEILL                           = 0x63C
+	SYS_COSF                            = 0x63D
+	SYS_COSHF                           = 0x63F
+	SYS_COSL                            = 0x63E
+	SYS_REMOVE                          = 0x063
+	SYS_POWL                            = 0x64A
+	SYS_RENAME                          = 0x064
+	SYS_SINF                            = 0x64B
+	SYS_SINHF                           = 0x64F
+	SYS_SINL                            = 0x64C
+	SYS_SQRTF                           = 0x64D
+	SYS_SQRTL                           = 0x64E
+	SYS_BTOWC                           = 0x65F
+	SYS_FREXPL                          = 0x65A
+	SYS_LDEXPF                          = 0x65B
+	SYS_LDEXPL                          = 0x65C
+	SYS_MODFF                           = 0x65D
+	SYS_MODFL                           = 0x65E
+	SYS_TMPFILE                         = 0x065
+	SYS_FREOPEN                         = 0x066
+	SYS___CHARMAP_INIT_A                = 0x66E
+	SYS___GETHOSTBYADDR_R_A             = 0x66C
+	SYS___GETHOSTBYNAME_A               = 0x66A
+	SYS___GETHOSTBYNAME_R_A             = 0x66D
+	SYS___MBLEN_A                       = 0x66F
+	SYS___RES_INIT_A                    = 0x66B
+	SYS_FCLOSE                          = 0x067
+	SYS___GETGRGID_R_A                  = 0x67D
+	SYS___WCSTOMBS_A                    = 0x67A
+	SYS___WCSTOMBS_STD_A                = 0x67B
+	SYS___WCSWIDTH_A                    = 0x67C
+	SYS___WCSWIDTH_ASIA                 = 0x67F
+	SYS___WCSWIDTH_STD_A                = 0x67E
+	SYS_FFLUSH                          = 0x068
+	SYS___GETLOGIN_R_A                  = 0x68E
+	SYS___GETPWNAM_R_A                  = 0x68C
+	SYS___GETPWUID_R_A                  = 0x68D
+	SYS___TTYNAME_R_A                   = 0x68F
+	SYS___WCWIDTH_ASIA                  = 0x68B
+	SYS___WCWIDTH_STD_A                 = 0x68A
+	SYS_FOPEN                           = 0x069
+	SYS___REGEXEC_A                     = 0x69A
+	SYS___REGEXEC_STD_A                 = 0x69B
+	SYS___REGFREE_A                     = 0x69C
+	SYS___REGFREE_STD_A                 = 0x69D
+	SYS___STRCOLL_A                     = 0x69E
+	SYS___STRCOLL_C_A                   = 0x69F
+	SYS_SCANF                           = 0x070
+	SYS___A64L_A                        = 0x70C
+	SYS___ECVT_A                        = 0x70D
+	SYS___FCVT_A                        = 0x70E
+	SYS___GCVT_A                        = 0x70F
+	SYS___STRTOUL_A                     = 0x70A
+	SYS_____AE_CORRESTBL_QUERY_A        = 0x70B
+	SYS_SPRINTF                         = 0x071
+	SYS___ACCESS_A                      = 0x71F
+	SYS___CATOPEN_A                     = 0x71E
+	SYS___GETOPT_A                      = 0x71D
+	SYS___REALPATH_A                    = 0x71A
+	SYS___SETENV_A                      = 0x71B
+	SYS___SYSTEM_A                      = 0x71C
+	SYS_FGETC                           = 0x072
+	SYS___GAI_STRERROR_A                = 0x72F
+	SYS___RMDIR_A                       = 0x72A
+	SYS___STATVFS_A                     = 0x72B
+	SYS___SYMLINK_A                     = 0x72C
+	SYS___TRUNCATE_A                    = 0x72D
+	SYS___UNLINK_A                      = 0x72E
+	SYS_VFPRINTF                        = 0x073
+	SYS___ISSPACE_A                     = 0x73A
+	SYS___ISUPPER_A                     = 0x73B
+	SYS___ISWALNUM_A                    = 0x73F
+	SYS___ISXDIGIT_A                    = 0x73C
+	SYS___TOLOWER_A                     = 0x73D
+	SYS___TOUPPER_A                     = 0x73E
+	SYS_VPRINTF                         = 0x074
+	SYS___CONFSTR_A                     = 0x74B
+	SYS___FDOPEN_A                      = 0x74E
+	SYS___FLDATA_A                      = 0x74F
+	SYS___FTOK_A                        = 0x74C
+	SYS___ISWXDIGIT_A                   = 0x74A
+	SYS___MKTEMP_A                      = 0x74D
+	SYS_VSPRINTF                        = 0x075
+	SYS___GETGRGID_A                    = 0x75A
+	SYS___GETGRNAM_A                    = 0x75B
+	SYS___GETGROUPSBYNAME_A             = 0x75C
+	SYS___GETHOSTENT_A                  = 0x75D
+	SYS___GETHOSTNAME_A                 = 0x75E
+	SYS___GETLOGIN_A                    = 0x75F
+	SYS_GETC                            = 0x076
+	SYS___CREATEWORKUNIT_A              = 0x76A
+	SYS___CTERMID_A                     = 0x76B
+	SYS___FMTMSG_A                      = 0x76C
+	SYS___INITGROUPS_A                  = 0x76D
+	SYS___MSGRCV_A                      = 0x76F
+	SYS_____LOGIN_A                     = 0x76E
+	SYS_FGETS                           = 0x077
+	SYS___STRCASECMP_A                  = 0x77B
+	SYS___STRNCASECMP_A                 = 0x77C
+	SYS___TTYNAME_A                     = 0x77D
+	SYS___UNAME_A                       = 0x77E
+	SYS___UTIMES_A                      = 0x77F
+	SYS_____SERVER_PWU_A                = 0x77A
+	SYS_FPUTC                           = 0x078
+	SYS___CREAT_O_A                     = 0x78E
+	SYS___ENVNA                         = 0x78F
+	SYS___FREAD_A                       = 0x78A
+	SYS___FWRITE_A                      = 0x78B
+	SYS___ISASCII                       = 0x78D
+	SYS___OPEN_O_A                      = 0x78C
+	SYS_FPUTS                           = 0x079
+	SYS___ASCTIME_A                     = 0x79C
+	SYS___CTIME_A                       = 0x79D
+	SYS___GETDATE_A                     = 0x79E
+	SYS___GETSERVBYPORT_A               = 0x79A
+	SYS___GETSERVENT_A                  = 0x79B
+	SYS___TZSET_A                       = 0x79F
+	SYS_ACL_FROM_TEXT                   = 0x80C
+	SYS_ACL_SET_FD                      = 0x80A
+	SYS_ACL_SET_FILE                    = 0x80B
+	SYS_ACL_SORT                        = 0x80E
+	SYS_ACL_TO_TEXT                     = 0x80D
+	SYS_UNGETC                          = 0x080
+	SYS___SHUTDOWN_REGISTRATION         = 0x80F
+	SYS_FREAD                           = 0x081
+	SYS_FREEADDRINFO                    = 0x81A
+	SYS_GAI_STRERROR                    = 0x81B
+	SYS_REXEC_AF                        = 0x81C
+	SYS___DYNALLOC_A                    = 0x81F
+	SYS___POE                           = 0x81D
+	SYS_WCSTOMBS                        = 0x082
+	SYS___INET_ADDR_A                   = 0x82F
+	SYS___NLIST_A                       = 0x82A
+	SYS_____TCGETCP_A                   = 0x82B
+	SYS_____TCSETCP_A                   = 0x82C
+	SYS_____W_PIOCTL_A                  = 0x82E
+	SYS_MBTOWC                          = 0x083
+	SYS___CABEND                        = 0x83D
+	SYS___LE_CIB_GET                    = 0x83E
+	SYS___RECVMSG_A                     = 0x83B
+	SYS___SENDMSG_A                     = 0x83A
+	SYS___SET_LAA_FOR_JIT               = 0x83F
+	SYS_____LCHATTR_A                   = 0x83C
+	SYS_WCTOMB                          = 0x084
+	SYS___CBRTL_B                       = 0x84A
+	SYS___COPYSIGNF_B                   = 0x84B
+	SYS___COPYSIGNL_B                   = 0x84C
+	SYS___COTANF_B                      = 0x84D
+	SYS___COTANL_B                      = 0x84F
+	SYS___COTAN_B                       = 0x84E
+	SYS_MBSTOWCS                        = 0x085
+	SYS___LOG1PL_B                      = 0x85A
+	SYS___LOG2F_B                       = 0x85B
+	SYS___LOG2L_B                       = 0x85D
+	SYS___LOG2_B                        = 0x85C
+	SYS___REMAINDERF_B                  = 0x85E
+	SYS___REMAINDERL_B                  = 0x85F
+	SYS_ACOSHF                          = 0x86E
+	SYS_ACOSHL                          = 0x86F
+	SYS_WCSCPY                          = 0x086
+	SYS___ERFCF_B                       = 0x86D
+	SYS___ERFF_B                        = 0x86C
+	SYS___LROUNDF_B                     = 0x86A
+	SYS___LROUND_B                      = 0x86B
+	SYS_COTANL                          = 0x87A
+	SYS_EXP2F                           = 0x87B
+	SYS_EXP2L                           = 0x87C
+	SYS_EXPM1F                          = 0x87D
+	SYS_EXPM1L                          = 0x87E
+	SYS_FDIMF                           = 0x87F
+	SYS_WCSCAT                          = 0x087
+	SYS___COTANL                        = 0x87A
+	SYS_REMAINDERF                      = 0x88A
+	SYS_REMAINDERL                      = 0x88B
+	SYS_REMAINDF                        = 0x88A
+	SYS_REMAINDL                        = 0x88B
+	SYS_REMQUO                          = 0x88D
+	SYS_REMQUOF                         = 0x88C
+	SYS_REMQUOL                         = 0x88E
+	SYS_TGAMMAF                         = 0x88F
+	SYS_WCSCHR                          = 0x088
+	SYS_ERFCF                           = 0x89B
+	SYS_ERFCL                           = 0x89C
+	SYS_ERFL                            = 0x89A
+	SYS_EXP2                            = 0x89E
+	SYS_WCSCMP                          = 0x089
+	SYS___EXP2_B                        = 0x89D
+	SYS___FAR_JUMP                      = 0x89F
+	SYS_ABS                             = 0x090
+	SYS___ERFCL_H                       = 0x90A
+	SYS___EXPF_H                        = 0x90C
+	SYS___EXPL_H                        = 0x90D
+	SYS___EXPM1_H                       = 0x90E
+	SYS___EXP_H                         = 0x90B
+	SYS___FDIM_H                        = 0x90F
+	SYS_DIV                             = 0x091
+	SYS___LOG2F_H                       = 0x91F
+	SYS___LOG2_H                        = 0x91E
+	SYS___LOGB_H                        = 0x91D
+	SYS___LOGF_H                        = 0x91B
+	SYS___LOGL_H                        = 0x91C
+	SYS___LOG_H                         = 0x91A
+	SYS_LABS                            = 0x092
+	SYS___POWL_H                        = 0x92A
+	SYS___REMAINDER_H                   = 0x92B
+	SYS___RINT_H                        = 0x92C
+	SYS___SCALB_H                       = 0x92D
+	SYS___SINF_H                        = 0x92F
+	SYS___SIN_H                         = 0x92E
+	SYS_STRNCPY                         = 0x093
+	SYS___TANHF_H                       = 0x93B
+	SYS___TANHL_H                       = 0x93C
+	SYS___TANH_H                        = 0x93A
+	SYS___TGAMMAF_H                     = 0x93E
+	SYS___TGAMMA_H                      = 0x93D
+	SYS___TRUNC_H                       = 0x93F
+	SYS_MEMCPY                          = 0x094
+	SYS_VFWSCANF                        = 0x94A
+	SYS_VSWSCANF                        = 0x94E
+	SYS_VWSCANF                         = 0x94C
+	SYS_INET6_RTH_ADD                   = 0x95D
+	SYS_INET6_RTH_INIT                  = 0x95C
+	SYS_INET6_RTH_REVERSE               = 0x95E
+	SYS_INET6_RTH_SEGMENTS              = 0x95F
+	SYS_INET6_RTH_SPACE                 = 0x95B
+	SYS_MEMMOVE                         = 0x095
+	SYS_WCSTOLD                         = 0x95A
+	SYS_STRCPY                          = 0x096
+	SYS_STRCMP                          = 0x097
+	SYS_CABS                            = 0x98E
+	SYS_STRCAT                          = 0x098
+	SYS___CABS_B                        = 0x98F
+	SYS___POW_II                        = 0x98A
+	SYS___POW_II_B                      = 0x98B
+	SYS___POW_II_H                      = 0x98C
+	SYS_CACOSF                          = 0x99A
+	SYS_CACOSL                          = 0x99D
+	SYS_STRNCAT                         = 0x099
+	SYS___CACOSF_B                      = 0x99B
+	SYS___CACOSF_H                      = 0x99C
+	SYS___CACOSL_B                      = 0x99E
+	SYS___CACOSL_H                      = 0x99F
+	SYS_ISWALPHA                        = 0x100
+	SYS_ISWBLANK                        = 0x101
+	SYS___ISWBLK                        = 0x101
+	SYS_ISWCNTRL                        = 0x102
+	SYS_ISWDIGIT                        = 0x103
+	SYS_ISWGRAPH                        = 0x104
+	SYS_ISWLOWER                        = 0x105
+	SYS_ISWPRINT                        = 0x106
+	SYS_ISWPUNCT                        = 0x107
+	SYS_ISWSPACE                        = 0x108
+	SYS_ISWUPPER                        = 0x109
+	SYS_WCTOB                           = 0x110
+	SYS_MBRLEN                          = 0x111
+	SYS_MBRTOWC                         = 0x112
+	SYS_MBSRTOWC                        = 0x113
+	SYS_MBSRTOWCS                       = 0x113
+	SYS_WCRTOMB                         = 0x114
+	SYS_WCSRTOMB                        = 0x115
+	SYS_WCSRTOMBS                       = 0x115
+	SYS___CSID                          = 0x116
+	SYS___WCSID                         = 0x117
+	SYS_STRPTIME                        = 0x118
+	SYS___STRPTM                        = 0x118
+	SYS_STRFMON                         = 0x119
+	SYS_WCSCOLL                         = 0x130
+	SYS_WCSXFRM                         = 0x131
+	SYS_WCSWIDTH                        = 0x132
+	SYS_WCWIDTH                         = 0x133
+	SYS_WCSFTIME                        = 0x134
+	SYS_SWPRINTF                        = 0x135
+	SYS_VSWPRINT                        = 0x136
+	SYS_VSWPRINTF                       = 0x136
+	SYS_SWSCANF                         = 0x137
+	SYS_REGCOMP                         = 0x138
+	SYS_REGEXEC                         = 0x139
+	SYS_GETWC                           = 0x140
+	SYS_GETWCHAR                        = 0x141
+	SYS_PUTWC                           = 0x142
+	SYS_PUTWCHAR                        = 0x143
+	SYS_UNGETWC                         = 0x144
+	SYS_ICONV_OPEN                      = 0x145
+	SYS_ICONV                           = 0x146
+	SYS_ICONV_CLOSE                     = 0x147
+	SYS_COLLRANGE                       = 0x150
+	SYS_CCLASS                          = 0x151
+	SYS_COLLORDER                       = 0x152
+	SYS___DEMANGLE                      = 0x154
+	SYS_FDOPEN                          = 0x155
+	SYS___ERRNO                         = 0x156
+	SYS___ERRNO2                        = 0x157
+	SYS___TERROR                        = 0x158
+	SYS_MAXCOLL                         = 0x169
+	SYS_DLLLOAD                         = 0x170
+	SYS__EXIT                           = 0x174
+	SYS_ACCESS                          = 0x175
+	SYS_ALARM                           = 0x176
+	SYS_CFGETISPEED                     = 0x177
+	SYS_CFGETOSPEED                     = 0x178
+	SYS_CFSETISPEED                     = 0x179
+	SYS_CREAT                           = 0x180
+	SYS_CTERMID                         = 0x181
+	SYS_DUP                             = 0x182
+	SYS_DUP2                            = 0x183
+	SYS_EXECL                           = 0x184
+	SYS_EXECLE                          = 0x185
+	SYS_EXECLP                          = 0x186
+	SYS_EXECV                           = 0x187
+	SYS_EXECVE                          = 0x188
+	SYS_EXECVP                          = 0x189
+	SYS_FSTAT                           = 0x190
+	SYS_FSYNC                           = 0x191
+	SYS_FTRUNCATE                       = 0x192
+	SYS_GETCWD                          = 0x193
+	SYS_GETEGID                         = 0x194
+	SYS_GETEUID                         = 0x195
+	SYS_GETGID                          = 0x196
+	SYS_GETGRGID                        = 0x197
+	SYS_GETGRNAM                        = 0x198
+	SYS_GETGROUPS                       = 0x199
+	SYS_PTHREAD_MUTEXATTR_DESTROY       = 0x200
+	SYS_PTHREAD_MUTEXATTR_SETKIND_NP    = 0x201
+	SYS_PTHREAD_MUTEXATTR_GETKIND_NP    = 0x202
+	SYS_PTHREAD_MUTEX_INIT              = 0x203
+	SYS_PTHREAD_MUTEX_DESTROY           = 0x204
+	SYS_PTHREAD_MUTEX_LOCK              = 0x205
+	SYS_PTHREAD_MUTEX_TRYLOCK           = 0x206
+	SYS_PTHREAD_MUTEX_UNLOCK            = 0x207
+	SYS_PTHREAD_ONCE                    = 0x209
+	SYS_TW_OPEN                         = 0x210
+	SYS_TW_FCNTL                        = 0x211
+	SYS_PTHREAD_JOIN_D4_NP              = 0x212
+	SYS_PTHREAD_CONDATTR_SETKIND_NP     = 0x213
+	SYS_PTHREAD_CONDATTR_GETKIND_NP     = 0x214
+	SYS_EXTLINK_NP                      = 0x215
+	SYS___PASSWD                        = 0x216
+	SYS_SETGROUPS                       = 0x217
+	SYS_INITGROUPS                      = 0x218
+	SYS_WCSRCHR                         = 0x240
+	SYS_SVC99                           = 0x241
+	SYS___SVC99                         = 0x241
+	SYS_WCSWCS                          = 0x242
+	SYS_LOCALECO                        = 0x243
+	SYS_LOCALECONV                      = 0x243
+	SYS___LIBREL                        = 0x244
+	SYS_RELEASE                         = 0x245
+	SYS___RLSE                          = 0x245
+	SYS_FLOCATE                         = 0x246
+	SYS___FLOCT                         = 0x246
+	SYS_FDELREC                         = 0x247
+	SYS___FDLREC                        = 0x247
+	SYS_FETCH                           = 0x248
+	SYS___FETCH                         = 0x248
+	SYS_QSORT                           = 0x249
+	SYS___CLEANUPCATCH                  = 0x260
+	SYS___CATCHMATCH                    = 0x261
+	SYS___CLEAN2UPCATCH                 = 0x262
+	SYS_GETPRIORITY                     = 0x270
+	SYS_NICE                            = 0x271
+	SYS_SETPRIORITY                     = 0x272
+	SYS_GETITIMER                       = 0x273
+	SYS_SETITIMER                       = 0x274
+	SYS_MSGCTL                          = 0x275
+	SYS_MSGGET                          = 0x276
+	SYS_MSGRCV                          = 0x277
+	SYS_MSGSND                          = 0x278
+	SYS_MSGXRCV                         = 0x279
+	SYS___MSGXR                         = 0x279
+	SYS_SHMGET                          = 0x280
+	SYS___GETIPC                        = 0x281
+	SYS_SETGRENT                        = 0x282
+	SYS_GETGRENT                        = 0x283
+	SYS_ENDGRENT                        = 0x284
+	SYS_SETPWENT                        = 0x285
+	SYS_GETPWENT                        = 0x286
+	SYS_ENDPWENT                        = 0x287
+	SYS_BSD_SIGNAL                      = 0x288
+	SYS_KILLPG                          = 0x289
+	SYS_SIGSET                          = 0x290
+	SYS_SIGSTACK                        = 0x291
+	SYS_GETRLIMIT                       = 0x292
+	SYS_SETRLIMIT                       = 0x293
+	SYS_GETRUSAGE                       = 0x294
+	SYS_MMAP                            = 0x295
+	SYS_MPROTECT                        = 0x296
+	SYS_MSYNC                           = 0x297
+	SYS_MUNMAP                          = 0x298
+	SYS_CONFSTR                         = 0x299
+	SYS___NDMTRM                        = 0x300
+	SYS_FTOK                            = 0x301
+	SYS_BASENAME                        = 0x302
+	SYS_DIRNAME                         = 0x303
+	SYS_GETDTABLESIZE                   = 0x304
+	SYS_MKSTEMP                         = 0x305
+	SYS_MKTEMP                          = 0x306
+	SYS_NFTW                            = 0x307
+	SYS_GETWD                           = 0x308
+	SYS_LOCKF                           = 0x309
+	SYS_WORDEXP                         = 0x310
+	SYS_WORDFREE                        = 0x311
+	SYS_GETPGID                         = 0x312
+	SYS_GETSID                          = 0x313
+	SYS___UTMPXNAME                     = 0x314
+	SYS_CUSERID                         = 0x315
+	SYS_GETPASS                         = 0x316
+	SYS_FNMATCH                         = 0x317
+	SYS_FTW                             = 0x318
+	SYS_GETW                            = 0x319
+	SYS_ACOSH                           = 0x320
+	SYS_ASINH                           = 0x321
+	SYS_ATANH                           = 0x322
+	SYS_CBRT                            = 0x323
+	SYS_EXPM1                           = 0x324
+	SYS_ILOGB                           = 0x325
+	SYS_LOGB                            = 0x326
+	SYS_LOG1P                           = 0x327
+	SYS_NEXTAFTER                       = 0x328
+	SYS_RINT                            = 0x329
+	SYS_SPAWN                           = 0x330
+	SYS_SPAWNP                          = 0x331
+	SYS_GETLOGIN_UU                     = 0x332
+	SYS_ECVT                            = 0x333
+	SYS_FCVT                            = 0x334
+	SYS_GCVT                            = 0x335
+	SYS_ACCEPT                          = 0x336
+	SYS_BIND                            = 0x337
+	SYS_CONNECT                         = 0x338
+	SYS_ENDHOSTENT                      = 0x339
+	SYS_GETHOSTENT                      = 0x340
+	SYS_GETHOSTID                       = 0x341
+	SYS_GETHOSTNAME                     = 0x342
+	SYS_GETNETBYADDR                    = 0x343
+	SYS_GETNETBYNAME                    = 0x344
+	SYS_GETNETENT                       = 0x345
+	SYS_GETPEERNAME                     = 0x346
+	SYS_GETPROTOBYNAME                  = 0x347
+	SYS_GETPROTOBYNUMBER                = 0x348
+	SYS_GETPROTOENT                     = 0x349
+	SYS_INET_LNAOF                      = 0x350
+	SYS_INET_MAKEADDR                   = 0x351
+	SYS_INET_NETOF                      = 0x352
+	SYS_INET_NETWORK                    = 0x353
+	SYS_INET_NTOA                       = 0x354
+	SYS_IOCTL                           = 0x355
+	SYS_LISTEN                          = 0x356
+	SYS_READV                           = 0x357
+	SYS_RECV                            = 0x358
+	SYS_RECVFROM                        = 0x359
+	SYS_SETHOSTENT                      = 0x360
+	SYS_SETNETENT                       = 0x361
+	SYS_SETPEER                         = 0x362
+	SYS_SETPROTOENT                     = 0x363
+	SYS_SETSERVENT                      = 0x364
+	SYS_SETSOCKOPT                      = 0x365
+	SYS_SHUTDOWN                        = 0x366
+	SYS_SOCKET                          = 0x367
+	SYS_SOCKETPAIR                      = 0x368
+	SYS_WRITEV                          = 0x369
+	SYS_ENDNETENT                       = 0x370
+	SYS_CLOSELOG                        = 0x371
+	SYS_OPENLOG                         = 0x372
+	SYS_SETLOGMASK                      = 0x373
+	SYS_SYSLOG                          = 0x374
+	SYS_PTSNAME                         = 0x375
+	SYS_SETREUID                        = 0x376
+	SYS_SETREGID                        = 0x377
+	SYS_REALPATH                        = 0x378
+	SYS___SIGNGAM                       = 0x379
+	SYS_POLL                            = 0x380
+	SYS_REXEC                           = 0x381
+	SYS___ISASCII2                      = 0x382
+	SYS___TOASCII2                      = 0x383
+	SYS_CHPRIORITY                      = 0x384
+	SYS_PTHREAD_ATTR_SETSYNCTYPE_NP     = 0x385
+	SYS_PTHREAD_ATTR_GETSYNCTYPE_NP     = 0x386
+	SYS_PTHREAD_SET_LIMIT_NP            = 0x387
+	SYS___STNETENT                      = 0x388
+	SYS___STPROTOENT                    = 0x389
+	SYS___SELECT1                       = 0x390
+	SYS_PTHREAD_SECURITY_NP             = 0x391
+	SYS___CHECK_RESOURCE_AUTH_NP        = 0x392
+	SYS___CONVERT_ID_NP                 = 0x393
+	SYS___OPENVMREL                     = 0x394
+	SYS_WMEMCHR                         = 0x395
+	SYS_WMEMCMP                         = 0x396
+	SYS_WMEMCPY                         = 0x397
+	SYS_WMEMMOVE                        = 0x398
+	SYS_WMEMSET                         = 0x399
+	SYS___FPUTWC                        = 0x400
+	SYS___PUTWC                         = 0x401
+	SYS___PWCHAR                        = 0x402
+	SYS___WCSFTM                        = 0x403
+	SYS___WCSTOK                        = 0x404
+	SYS___WCWDTH                        = 0x405
+	SYS_T_ACCEPT                        = 0x409
+	SYS_T_GETINFO                       = 0x410
+	SYS_T_GETPROTADDR                   = 0x411
+	SYS_T_GETSTATE                      = 0x412
+	SYS_T_LISTEN                        = 0x413
+	SYS_T_LOOK                          = 0x414
+	SYS_T_OPEN                          = 0x415
+	SYS_T_OPTMGMT                       = 0x416
+	SYS_T_RCV                           = 0x417
+	SYS_T_RCVCONNECT                    = 0x418
+	SYS_T_RCVDIS                        = 0x419
+	SYS_T_SNDUDATA                      = 0x420
+	SYS_T_STRERROR                      = 0x421
+	SYS_T_SYNC                          = 0x422
+	SYS_T_UNBIND                        = 0x423
+	SYS___T_ERRNO                       = 0x424
+	SYS___RECVMSG2                      = 0x425
+	SYS___SENDMSG2                      = 0x426
+	SYS_FATTACH                         = 0x427
+	SYS_FDETACH                         = 0x428
+	SYS_GETMSG                          = 0x429
+	SYS_GETCONTEXT                      = 0x430
+	SYS_SETCONTEXT                      = 0x431
+	SYS_MAKECONTEXT                     = 0x432
+	SYS_SWAPCONTEXT                     = 0x433
+	SYS_PTHREAD_GETSPECIFIC_D8_NP       = 0x434
+	SYS_GETCLIENTID                     = 0x470
+	SYS___GETCLIENTID                   = 0x471
+	SYS_GETSTABLESIZE                   = 0x472
+	SYS_GETIBMOPT                       = 0x473
+	SYS_GETIBMSOCKOPT                   = 0x474
+	SYS_GIVESOCKET                      = 0x475
+	SYS_IBMSFLUSH                       = 0x476
+	SYS_MAXDESC                         = 0x477
+	SYS_SETIBMOPT                       = 0x478
+	SYS_SETIBMSOCKOPT                   = 0x479
+	SYS___SERVER_PWU                    = 0x480
+	SYS_PTHREAD_TAG_NP                  = 0x481
+	SYS___CONSOLE                       = 0x482
+	SYS___WSINIT                        = 0x483
+	SYS___IPTCPN                        = 0x489
+	SYS___SERVER_CLASSIFY               = 0x490
+	SYS___HEAPRPT                       = 0x496
+	SYS___ISBFP                         = 0x500
+	SYS___FP_CAST                       = 0x501
+	SYS___CERTIFICATE                   = 0x502
+	SYS_SEND_FILE                       = 0x503
+	SYS_AIO_CANCEL                      = 0x504
+	SYS_AIO_ERROR                       = 0x505
+	SYS_AIO_READ                        = 0x506
+	SYS_AIO_RETURN                      = 0x507
+	SYS_AIO_SUSPEND                     = 0x508
+	SYS_AIO_WRITE                       = 0x509
+	SYS_PTHREAD_RWLOCK_TRYWRLOCK        = 0x510
+	SYS_PTHREAD_RWLOCK_UNLOCK           = 0x511
+	SYS_PTHREAD_RWLOCK_WRLOCK           = 0x512
+	SYS_PTHREAD_RWLOCKATTR_GETPSHARED   = 0x513
+	SYS_PTHREAD_RWLOCKATTR_SETPSHARED   = 0x514
+	SYS_PTHREAD_RWLOCKATTR_INIT         = 0x515
+	SYS_PTHREAD_RWLOCKATTR_DESTROY      = 0x516
+	SYS___CTTBL                         = 0x517
+	SYS_PTHREAD_MUTEXATTR_SETTYPE       = 0x518
+	SYS_PTHREAD_MUTEXATTR_GETTYPE       = 0x519
+	SYS___FP_UNORDERED                  = 0x520
+	SYS___FP_READ_RND                   = 0x521
+	SYS___FP_READ_RND_B                 = 0x522
+	SYS___FP_SWAP_RND                   = 0x523
+	SYS___FP_SWAP_RND_B                 = 0x524
+	SYS___FP_LEVEL                      = 0x525
+	SYS___FP_BTOH                       = 0x526
+	SYS___FP_HTOB                       = 0x527
+	SYS___FPC_RD                        = 0x528
+	SYS___FPC_WR                        = 0x529
+	SYS_PTHREAD_SETCANCELTYPE           = 0x600
+	SYS_PTHREAD_TESTCANCEL              = 0x601
+	SYS___ATANF_B                       = 0x602
+	SYS___ATANL_B                       = 0x603
+	SYS___CEILF_B                       = 0x604
+	SYS___CEILL_B                       = 0x605
+	SYS___COSF_B                        = 0x606
+	SYS___COSL_B                        = 0x607
+	SYS___FABSF_B                       = 0x608
+	SYS___FABSL_B                       = 0x609
+	SYS___SINF_B                        = 0x610
+	SYS___SINL_B                        = 0x611
+	SYS___TANF_B                        = 0x612
+	SYS___TANL_B                        = 0x613
+	SYS___TANHF_B                       = 0x614
+	SYS___TANHL_B                       = 0x615
+	SYS___ACOSF_B                       = 0x616
+	SYS___ACOSL_B                       = 0x617
+	SYS___ASINF_B                       = 0x618
+	SYS___ASINL_B                       = 0x619
+	SYS___LOGF_B                        = 0x620
+	SYS___LOGL_B                        = 0x621
+	SYS___LOG10F_B                      = 0x622
+	SYS___LOG10L_B                      = 0x623
+	SYS___POWF_B                        = 0x624
+	SYS___POWL_B                        = 0x625
+	SYS___SINHF_B                       = 0x626
+	SYS___SINHL_B                       = 0x627
+	SYS___SQRTF_B                       = 0x628
+	SYS___SQRTL_B                       = 0x629
+	SYS___MODFL_B                       = 0x630
+	SYS_ABSF                            = 0x631
+	SYS_ABSL                            = 0x632
+	SYS_ACOSF                           = 0x633
+	SYS_ACOSL                           = 0x634
+	SYS_ASINF                           = 0x635
+	SYS_ASINL                           = 0x636
+	SYS_ATAN2F                          = 0x637
+	SYS_ATAN2L                          = 0x638
+	SYS_ATANF                           = 0x639
+	SYS_COSHL                           = 0x640
+	SYS_EXPF                            = 0x641
+	SYS_EXPL                            = 0x642
+	SYS_TANHF                           = 0x643
+	SYS_TANHL                           = 0x644
+	SYS_LOG10F                          = 0x645
+	SYS_LOG10L                          = 0x646
+	SYS_LOGF                            = 0x647
+	SYS_LOGL                            = 0x648
+	SYS_POWF                            = 0x649
+	SYS_SINHL                           = 0x650
+	SYS_TANF                            = 0x651
+	SYS_TANL                            = 0x652
+	SYS_FABSF                           = 0x653
+	SYS_FABSL                           = 0x654
+	SYS_FLOORF                          = 0x655
+	SYS_FLOORL                          = 0x656
+	SYS_FMODF                           = 0x657
+	SYS_FMODL                           = 0x658
+	SYS_FREXPF                          = 0x659
+	SYS___CHATTR                        = 0x660
+	SYS___FCHATTR                       = 0x661
+	SYS___TOCCSID                       = 0x662
+	SYS___CSNAMETYPE                    = 0x663
+	SYS___TOCSNAME                      = 0x664
+	SYS___CCSIDTYPE                     = 0x665
+	SYS___AE_CORRESTBL_QUERY            = 0x666
+	SYS___AE_AUTOCONVERT_STATE          = 0x667
+	SYS_DN_FIND                         = 0x668
+	SYS___GETHOSTBYADDR_A               = 0x669
+	SYS___MBLEN_SB_A                    = 0x670
+	SYS___MBLEN_STD_A                   = 0x671
+	SYS___MBLEN_UTF                     = 0x672
+	SYS___MBSTOWCS_A                    = 0x673
+	SYS___MBSTOWCS_STD_A                = 0x674
+	SYS___MBTOWC_A                      = 0x675
+	SYS___MBTOWC_ISO1                   = 0x676
+	SYS___MBTOWC_SBCS                   = 0x677
+	SYS___MBTOWC_MBCS                   = 0x678
+	SYS___MBTOWC_UTF                    = 0x679
+	SYS___CSID_A                        = 0x680
+	SYS___CSID_STD_A                    = 0x681
+	SYS___WCSID_A                       = 0x682
+	SYS___WCSID_STD_A                   = 0x683
+	SYS___WCTOMB_A                      = 0x684
+	SYS___WCTOMB_ISO1                   = 0x685
+	SYS___WCTOMB_STD_A                  = 0x686
+	SYS___WCTOMB_UTF                    = 0x687
+	SYS___WCWIDTH_A                     = 0x688
+	SYS___GETGRNAM_R_A                  = 0x689
+	SYS___READDIR_R_A                   = 0x690
+	SYS___E2A_S                         = 0x691
+	SYS___FNMATCH_A                     = 0x692
+	SYS___FNMATCH_C_A                   = 0x693
+	SYS___EXECL_A                       = 0x694
+	SYS___FNMATCH_STD_A                 = 0x695
+	SYS___REGCOMP_A                     = 0x696
+	SYS___REGCOMP_STD_A                 = 0x697
+	SYS___REGERROR_A                    = 0x698
+	SYS___REGERROR_STD_A                = 0x699
+	SYS___SWPRINTF_A                    = 0x700
+	SYS___FSCANF_A                      = 0x701
+	SYS___SCANF_A                       = 0x702
+	SYS___SSCANF_A                      = 0x703
+	SYS___SWSCANF_A                     = 0x704
+	SYS___ATOF_A                        = 0x705
+	SYS___ATOI_A                        = 0x706
+	SYS___ATOL_A                        = 0x707
+	SYS___STRTOD_A                      = 0x708
+	SYS___STRTOL_A                      = 0x709
+	SYS___L64A_A                        = 0x710
+	SYS___STRERROR_A                    = 0x711
+	SYS___PERROR_A                      = 0x712
+	SYS___FETCH_A                       = 0x713
+	SYS___GETENV_A                      = 0x714
+	SYS___MKSTEMP_A                     = 0x717
+	SYS___PTSNAME_A                     = 0x718
+	SYS___PUTENV_A                      = 0x719
+	SYS___CHDIR_A                       = 0x720
+	SYS___CHOWN_A                       = 0x721
+	SYS___CHROOT_A                      = 0x722
+	SYS___GETCWD_A                      = 0x723
+	SYS___GETWD_A                       = 0x724
+	SYS___LCHOWN_A                      = 0x725
+	SYS___LINK_A                        = 0x726
+	SYS___PATHCONF_A                    = 0x727
+	SYS___IF_NAMEINDEX_A                = 0x728
+	SYS___READLINK_A                    = 0x729
+	SYS___EXTLINK_NP_A                  = 0x730
+	SYS___ISALNUM_A                     = 0x731
+	SYS___ISALPHA_A                     = 0x732
+	SYS___A2E_S                         = 0x733
+	SYS___ISCNTRL_A                     = 0x734
+	SYS___ISDIGIT_A                     = 0x735
+	SYS___ISGRAPH_A                     = 0x736
+	SYS___ISLOWER_A                     = 0x737
+	SYS___ISPRINT_A                     = 0x738
+	SYS___ISPUNCT_A                     = 0x739
+	SYS___ISWALPHA_A                    = 0x740
+	SYS___A2E_L                         = 0x741
+	SYS___ISWCNTRL_A                    = 0x742
+	SYS___ISWDIGIT_A                    = 0x743
+	SYS___ISWGRAPH_A                    = 0x744
+	SYS___ISWLOWER_A                    = 0x745
+	SYS___ISWPRINT_A                    = 0x746
+	SYS___ISWPUNCT_A                    = 0x747
+	SYS___ISWSPACE_A                    = 0x748
+	SYS___ISWUPPER_A                    = 0x749
+	SYS___REMOVE_A                      = 0x750
+	SYS___RENAME_A                      = 0x751
+	SYS___TMPNAM_A                      = 0x752
+	SYS___FOPEN_A                       = 0x753
+	SYS___FREOPEN_A                     = 0x754
+	SYS___CUSERID_A                     = 0x755
+	SYS___POPEN_A                       = 0x756
+	SYS___TEMPNAM_A                     = 0x757
+	SYS___FTW_A                         = 0x758
+	SYS___GETGRENT_A                    = 0x759
+	SYS___INET_NTOP_A                   = 0x760
+	SYS___GETPASS_A                     = 0x761
+	SYS___GETPWENT_A                    = 0x762
+	SYS___GETPWNAM_A                    = 0x763
+	SYS___GETPWUID_A                    = 0x764
+	SYS_____CHECK_RESOURCE_AUTH_NP_A    = 0x765
+	SYS___CHECKSCHENV_A                 = 0x766
+	SYS___CONNECTSERVER_A               = 0x767
+	SYS___CONNECTWORKMGR_A              = 0x768
+	SYS_____CONSOLE_A                   = 0x769
+	SYS___MSGSND_A                      = 0x770
+	SYS___MSGXRCV_A                     = 0x771
+	SYS___NFTW_A                        = 0x772
+	SYS_____PASSWD_A                    = 0x773
+	SYS___PTHREAD_SECURITY_NP_A         = 0x774
+	SYS___QUERYMETRICS_A                = 0x775
+	SYS___QUERYSCHENV                   = 0x776
+	SYS___READV_A                       = 0x777
+	SYS_____SERVER_CLASSIFY_A           = 0x778
+	SYS_____SERVER_INIT_A               = 0x779
+	SYS___W_GETPSENT_A                  = 0x780
+	SYS___WRITEV_A                      = 0x781
+	SYS___W_STATFS_A                    = 0x782
+	SYS___W_STATVFS_A                   = 0x783
+	SYS___FPUTC_A                       = 0x784
+	SYS___PUTCHAR_A                     = 0x785
+	SYS___PUTS_A                        = 0x786
+	SYS___FGETS_A                       = 0x787
+	SYS___GETS_A                        = 0x788
+	SYS___FPUTS_A                       = 0x789
+	SYS___PUTC_A                        = 0x790
+	SYS___AE_THREAD_SETMODE             = 0x791
+	SYS___AE_THREAD_SWAPMODE            = 0x792
+	SYS___GETNETBYADDR_A                = 0x793
+	SYS___GETNETBYNAME_A                = 0x794
+	SYS___GETNETENT_A                   = 0x795
+	SYS___GETPROTOBYNAME_A              = 0x796
+	SYS___GETPROTOBYNUMBER_A            = 0x797
+	SYS___GETPROTOENT_A                 = 0x798
+	SYS___GETSERVBYNAME_A               = 0x799
+	SYS_ACL_FIRST_ENTRY                 = 0x800
+	SYS_ACL_GET_ENTRY                   = 0x801
+	SYS_ACL_VALID                       = 0x802
+	SYS_ACL_CREATE_ENTRY                = 0x803
+	SYS_ACL_DELETE_ENTRY                = 0x804
+	SYS_ACL_UPDATE_ENTRY                = 0x805
+	SYS_ACL_DELETE_FD                   = 0x806
+	SYS_ACL_DELETE_FILE                 = 0x807
+	SYS_ACL_GET_FD                      = 0x808
+	SYS_ACL_GET_FILE                    = 0x809
+	SYS___ERFL_B                        = 0x810
+	SYS___ERFCL_B                       = 0x811
+	SYS___LGAMMAL_B                     = 0x812
+	SYS___SETHOOKEVENTS                 = 0x813
+	SYS_IF_NAMETOINDEX                  = 0x814
+	SYS_IF_INDEXTONAME                  = 0x815
+	SYS_IF_NAMEINDEX                    = 0x816
+	SYS_IF_FREENAMEINDEX                = 0x817
+	SYS_GETADDRINFO                     = 0x818
+	SYS_GETNAMEINFO                     = 0x819
+	SYS___DYNFREE_A                     = 0x820
+	SYS___RES_QUERY_A                   = 0x821
+	SYS___RES_SEARCH_A                  = 0x822
+	SYS___RES_QUERYDOMAIN_A             = 0x823
+	SYS___RES_MKQUERY_A                 = 0x824
+	SYS___RES_SEND_A                    = 0x825
+	SYS___DN_EXPAND_A                   = 0x826
+	SYS___DN_SKIPNAME_A                 = 0x827
+	SYS___DN_COMP_A                     = 0x828
+	SYS___DN_FIND_A                     = 0x829
+	SYS___INET_NTOA_A                   = 0x830
+	SYS___INET_NETWORK_A                = 0x831
+	SYS___ACCEPT_A                      = 0x832
+	SYS___ACCEPT_AND_RECV_A             = 0x833
+	SYS___BIND_A                        = 0x834
+	SYS___CONNECT_A                     = 0x835
+	SYS___GETPEERNAME_A                 = 0x836
+	SYS___GETSOCKNAME_A                 = 0x837
+	SYS___RECVFROM_A                    = 0x838
+	SYS___SENDTO_A                      = 0x839
+	SYS___LCHATTR                       = 0x840
+	SYS___WRITEDOWN                     = 0x841
+	SYS_PTHREAD_MUTEX_INIT2             = 0x842
+	SYS___ACOSHF_B                      = 0x843
+	SYS___ACOSHL_B                      = 0x844
+	SYS___ASINHF_B                      = 0x845
+	SYS___ASINHL_B                      = 0x846
+	SYS___ATANHF_B                      = 0x847
+	SYS___ATANHL_B                      = 0x848
+	SYS___CBRTF_B                       = 0x849
+	SYS___EXP2F_B                       = 0x850
+	SYS___EXP2L_B                       = 0x851
+	SYS___EXPM1F_B                      = 0x852
+	SYS___EXPM1L_B                      = 0x853
+	SYS___FDIMF_B                       = 0x854
+	SYS___FDIM_B                        = 0x855
+	SYS___FDIML_B                       = 0x856
+	SYS___HYPOTF_B                      = 0x857
+	SYS___HYPOTL_B                      = 0x858
+	SYS___LOG1PF_B                      = 0x859
+	SYS___REMQUOF_B                     = 0x860
+	SYS___REMQUO_B                      = 0x861
+	SYS___REMQUOL_B                     = 0x862
+	SYS___TGAMMAF_B                     = 0x863
+	SYS___TGAMMA_B                      = 0x864
+	SYS___TGAMMAL_B                     = 0x865
+	SYS___TRUNCF_B                      = 0x866
+	SYS___TRUNC_B                       = 0x867
+	SYS___TRUNCL_B                      = 0x868
+	SYS___LGAMMAF_B                     = 0x869
+	SYS_ASINHF                          = 0x870
+	SYS_ASINHL                          = 0x871
+	SYS_ATANHF                          = 0x872
+	SYS_ATANHL                          = 0x873
+	SYS_CBRTF                           = 0x874
+	SYS_CBRTL                           = 0x875
+	SYS_COPYSIGNF                       = 0x876
+	SYS_CPYSIGNF                        = 0x876
+	SYS_COPYSIGNL                       = 0x877
+	SYS_CPYSIGNL                        = 0x877
+	SYS_COTANF                          = 0x878
+	SYS___COTANF                        = 0x878
+	SYS_COTAN                           = 0x879
+	SYS___COTAN                         = 0x879
+	SYS_FDIM                            = 0x881
+	SYS_FDIML                           = 0x882
+	SYS_HYPOTF                          = 0x883
+	SYS_HYPOTL                          = 0x884
+	SYS_LOG1PF                          = 0x885
+	SYS_LOG1PL                          = 0x886
+	SYS_LOG2F                           = 0x887
+	SYS_LOG2                            = 0x888
+	SYS_LOG2L                           = 0x889
+	SYS_TGAMMA                          = 0x890
+	SYS_TGAMMAL                         = 0x891
+	SYS_TRUNCF                          = 0x892
+	SYS_TRUNC                           = 0x893
+	SYS_TRUNCL                          = 0x894
+	SYS_LGAMMAF                         = 0x895
+	SYS_LGAMMAL                         = 0x896
+	SYS_LROUNDF                         = 0x897
+	SYS_LROUND                          = 0x898
+	SYS_ERFF                            = 0x899
+	SYS___COSHF_H                       = 0x900
+	SYS___COSHL_H                       = 0x901
+	SYS___COTAN_H                       = 0x902
+	SYS___COTANF_H                      = 0x903
+	SYS___COTANL_H                      = 0x904
+	SYS___ERF_H                         = 0x905
+	SYS___ERFF_H                        = 0x906
+	SYS___ERFL_H                        = 0x907
+	SYS___ERFC_H                        = 0x908
+	SYS___ERFCF_H                       = 0x909
+	SYS___FDIMF_H                       = 0x910
+	SYS___FDIML_H                       = 0x911
+	SYS___FMOD_H                        = 0x912
+	SYS___FMODF_H                       = 0x913
+	SYS___FMODL_H                       = 0x914
+	SYS___GAMMA_H                       = 0x915
+	SYS___HYPOT_H                       = 0x916
+	SYS___ILOGB_H                       = 0x917
+	SYS___LGAMMA_H                      = 0x918
+	SYS___LGAMMAF_H                     = 0x919
+	SYS___LOG2L_H                       = 0x920
+	SYS___LOG1P_H                       = 0x921
+	SYS___LOG10_H                       = 0x922
+	SYS___LOG10F_H                      = 0x923
+	SYS___LOG10L_H                      = 0x924
+	SYS___LROUND_H                      = 0x925
+	SYS___LROUNDF_H                     = 0x926
+	SYS___NEXTAFTER_H                   = 0x927
+	SYS___POW_H                         = 0x928
+	SYS___POWF_H                        = 0x929
+	SYS___SINL_H                        = 0x930
+	SYS___SINH_H                        = 0x931
+	SYS___SINHF_H                       = 0x932
+	SYS___SINHL_H                       = 0x933
+	SYS___SQRT_H                        = 0x934
+	SYS___SQRTF_H                       = 0x935
+	SYS___SQRTL_H                       = 0x936
+	SYS___TAN_H                         = 0x937
+	SYS___TANF_H                        = 0x938
+	SYS___TANL_H                        = 0x939
+	SYS___TRUNCF_H                      = 0x940
+	SYS___TRUNCL_H                      = 0x941
+	SYS___COSH_H                        = 0x942
+	SYS___LE_DEBUG_SET_RESUME_MCH       = 0x943
+	SYS_VFSCANF                         = 0x944
+	SYS_VSCANF                          = 0x946
+	SYS_VSSCANF                         = 0x948
+	SYS_IMAXABS                         = 0x950
+	SYS_IMAXDIV                         = 0x951
+	SYS_STRTOIMAX                       = 0x952
+	SYS_STRTOUMAX                       = 0x953
+	SYS_WCSTOIMAX                       = 0x954
+	SYS_WCSTOUMAX                       = 0x955
+	SYS_ATOLL                           = 0x956
+	SYS_STRTOF                          = 0x957
+	SYS_STRTOLD                         = 0x958
+	SYS_WCSTOF                          = 0x959
+	SYS_INET6_RTH_GETADDR               = 0x960
+	SYS_INET6_OPT_INIT                  = 0x961
+	SYS_INET6_OPT_APPEND                = 0x962
+	SYS_INET6_OPT_FINISH                = 0x963
+	SYS_INET6_OPT_SET_VAL               = 0x964
+	SYS_INET6_OPT_NEXT                  = 0x965
+	SYS_INET6_OPT_FIND                  = 0x966
+	SYS_INET6_OPT_GET_VAL               = 0x967
+	SYS___POW_I                         = 0x987
+	SYS___POW_I_B                       = 0x988
+	SYS___POW_I_H                       = 0x989
+	SYS___CABS_H                        = 0x990
+	SYS_CABSF                           = 0x991
+	SYS___CABSF_B                       = 0x992
+	SYS___CABSF_H                       = 0x993
+	SYS_CABSL                           = 0x994
+	SYS___CABSL_B                       = 0x995
+	SYS___CABSL_H                       = 0x996
+	SYS_CACOS                           = 0x997
+	SYS___CACOS_B                       = 0x998
+	SYS___CACOS_H                       = 0x999
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_aix_ppc.go b/vendor/golang.org/x/sys/unix/ztypes_aix_ppc.go
index 2c1f815..7a8161c 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_aix_ppc.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_aix_ppc.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_aix.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc && aix
 // +build ppc,aix
 
 package unix
@@ -219,6 +220,7 @@
 	SizeofSockaddrUnix     = 0x401
 	SizeofSockaddrDatalink = 0x80
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofIPv6MTUInfo      = 0x20
diff --git a/vendor/golang.org/x/sys/unix/ztypes_aix_ppc64.go b/vendor/golang.org/x/sys/unix/ztypes_aix_ppc64.go
index b4a069e..07ed733 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_aix_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_aix_ppc64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_aix.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64 && aix
 // +build ppc64,aix
 
 package unix
@@ -223,6 +224,7 @@
 	SizeofSockaddrUnix     = 0x401
 	SizeofSockaddrDatalink = 0x80
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofIPv6MTUInfo      = 0x20
diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_386.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_386.go
deleted file mode 100644
index 6103f21..0000000
--- a/vendor/golang.org/x/sys/unix/ztypes_darwin_386.go
+++ /dev/null
@@ -1,503 +0,0 @@
-// cgo -godefs types_darwin.go | go run mkpost.go
-// Code generated by the command above; see README.md. DO NOT EDIT.
-
-// +build 386,darwin
-
-package unix
-
-const (
-	SizeofPtr      = 0x4
-	SizeofShort    = 0x2
-	SizeofInt      = 0x4
-	SizeofLong     = 0x4
-	SizeofLongLong = 0x8
-)
-
-type (
-	_C_short     int16
-	_C_int       int32
-	_C_long      int32
-	_C_long_long int64
-)
-
-type Timespec struct {
-	Sec  int32
-	Nsec int32
-}
-
-type Timeval struct {
-	Sec  int32
-	Usec int32
-}
-
-type Timeval32 struct{}
-
-type Rusage struct {
-	Utime    Timeval
-	Stime    Timeval
-	Maxrss   int32
-	Ixrss    int32
-	Idrss    int32
-	Isrss    int32
-	Minflt   int32
-	Majflt   int32
-	Nswap    int32
-	Inblock  int32
-	Oublock  int32
-	Msgsnd   int32
-	Msgrcv   int32
-	Nsignals int32
-	Nvcsw    int32
-	Nivcsw   int32
-}
-
-type Rlimit struct {
-	Cur uint64
-	Max uint64
-}
-
-type _Gid_t uint32
-
-type Stat_t struct {
-	Dev     int32
-	Mode    uint16
-	Nlink   uint16
-	Ino     uint64
-	Uid     uint32
-	Gid     uint32
-	Rdev    int32
-	Atim    Timespec
-	Mtim    Timespec
-	Ctim    Timespec
-	Btim    Timespec
-	Size    int64
-	Blocks  int64
-	Blksize int32
-	Flags   uint32
-	Gen     uint32
-	Lspare  int32
-	Qspare  [2]int64
-}
-
-type Statfs_t struct {
-	Bsize       uint32
-	Iosize      int32
-	Blocks      uint64
-	Bfree       uint64
-	Bavail      uint64
-	Files       uint64
-	Ffree       uint64
-	Fsid        Fsid
-	Owner       uint32
-	Type        uint32
-	Flags       uint32
-	Fssubtype   uint32
-	Fstypename  [16]int8
-	Mntonname   [1024]int8
-	Mntfromname [1024]int8
-	Reserved    [8]uint32
-}
-
-type Flock_t struct {
-	Start  int64
-	Len    int64
-	Pid    int32
-	Type   int16
-	Whence int16
-}
-
-type Fstore_t struct {
-	Flags      uint32
-	Posmode    int32
-	Offset     int64
-	Length     int64
-	Bytesalloc int64
-}
-
-type Radvisory_t struct {
-	Offset int64
-	Count  int32
-}
-
-type Fbootstraptransfer_t struct {
-	Offset int64
-	Length uint32
-	Buffer *byte
-}
-
-type Log2phys_t struct {
-	Flags       uint32
-	Contigbytes int64
-	Devoffset   int64
-}
-
-type Fsid struct {
-	Val [2]int32
-}
-
-type Dirent struct {
-	Ino     uint64
-	Seekoff uint64
-	Reclen  uint16
-	Namlen  uint16
-	Type    uint8
-	Name    [1024]int8
-	_       [3]byte
-}
-
-const (
-	PathMax = 0x400
-)
-
-type RawSockaddrInet4 struct {
-	Len    uint8
-	Family uint8
-	Port   uint16
-	Addr   [4]byte /* in_addr */
-	Zero   [8]int8
-}
-
-type RawSockaddrInet6 struct {
-	Len      uint8
-	Family   uint8
-	Port     uint16
-	Flowinfo uint32
-	Addr     [16]byte /* in6_addr */
-	Scope_id uint32
-}
-
-type RawSockaddrUnix struct {
-	Len    uint8
-	Family uint8
-	Path   [104]int8
-}
-
-type RawSockaddrDatalink struct {
-	Len    uint8
-	Family uint8
-	Index  uint16
-	Type   uint8
-	Nlen   uint8
-	Alen   uint8
-	Slen   uint8
-	Data   [12]int8
-}
-
-type RawSockaddr struct {
-	Len    uint8
-	Family uint8
-	Data   [14]int8
-}
-
-type RawSockaddrAny struct {
-	Addr RawSockaddr
-	Pad  [92]int8
-}
-
-type _Socklen uint32
-
-type Linger struct {
-	Onoff  int32
-	Linger int32
-}
-
-type Iovec struct {
-	Base *byte
-	Len  uint32
-}
-
-type IPMreq struct {
-	Multiaddr [4]byte /* in_addr */
-	Interface [4]byte /* in_addr */
-}
-
-type IPv6Mreq struct {
-	Multiaddr [16]byte /* in6_addr */
-	Interface uint32
-}
-
-type Msghdr struct {
-	Name       *byte
-	Namelen    uint32
-	Iov        *Iovec
-	Iovlen     int32
-	Control    *byte
-	Controllen uint32
-	Flags      int32
-}
-
-type Cmsghdr struct {
-	Len   uint32
-	Level int32
-	Type  int32
-}
-
-type Inet4Pktinfo struct {
-	Ifindex  uint32
-	Spec_dst [4]byte /* in_addr */
-	Addr     [4]byte /* in_addr */
-}
-
-type Inet6Pktinfo struct {
-	Addr    [16]byte /* in6_addr */
-	Ifindex uint32
-}
-
-type IPv6MTUInfo struct {
-	Addr RawSockaddrInet6
-	Mtu  uint32
-}
-
-type ICMPv6Filter struct {
-	Filt [8]uint32
-}
-
-const (
-	SizeofSockaddrInet4    = 0x10
-	SizeofSockaddrInet6    = 0x1c
-	SizeofSockaddrAny      = 0x6c
-	SizeofSockaddrUnix     = 0x6a
-	SizeofSockaddrDatalink = 0x14
-	SizeofLinger           = 0x8
-	SizeofIPMreq           = 0x8
-	SizeofIPv6Mreq         = 0x14
-	SizeofMsghdr           = 0x1c
-	SizeofCmsghdr          = 0xc
-	SizeofInet4Pktinfo     = 0xc
-	SizeofInet6Pktinfo     = 0x14
-	SizeofIPv6MTUInfo      = 0x20
-	SizeofICMPv6Filter     = 0x20
-)
-
-const (
-	PTRACE_TRACEME = 0x0
-	PTRACE_CONT    = 0x7
-	PTRACE_KILL    = 0x8
-)
-
-type Kevent_t struct {
-	Ident  uint32
-	Filter int16
-	Flags  uint16
-	Fflags uint32
-	Data   int32
-	Udata  *byte
-}
-
-type FdSet struct {
-	Bits [32]int32
-}
-
-const (
-	SizeofIfMsghdr    = 0x70
-	SizeofIfData      = 0x60
-	SizeofIfaMsghdr   = 0x14
-	SizeofIfmaMsghdr  = 0x10
-	SizeofIfmaMsghdr2 = 0x14
-	SizeofRtMsghdr    = 0x5c
-	SizeofRtMetrics   = 0x38
-)
-
-type IfMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-	Data    IfData
-}
-
-type IfData struct {
-	Type       uint8
-	Typelen    uint8
-	Physical   uint8
-	Addrlen    uint8
-	Hdrlen     uint8
-	Recvquota  uint8
-	Xmitquota  uint8
-	Unused1    uint8
-	Mtu        uint32
-	Metric     uint32
-	Baudrate   uint32
-	Ipackets   uint32
-	Ierrors    uint32
-	Opackets   uint32
-	Oerrors    uint32
-	Collisions uint32
-	Ibytes     uint32
-	Obytes     uint32
-	Imcasts    uint32
-	Omcasts    uint32
-	Iqdrops    uint32
-	Noproto    uint32
-	Recvtiming uint32
-	Xmittiming uint32
-	Lastchange Timeval
-	Unused2    uint32
-	Hwassist   uint32
-	Reserved1  uint32
-	Reserved2  uint32
-}
-
-type IfaMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-	Metric  int32
-}
-
-type IfmaMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-}
-
-type IfmaMsghdr2 struct {
-	Msglen   uint16
-	Version  uint8
-	Type     uint8
-	Addrs    int32
-	Flags    int32
-	Index    uint16
-	_        [2]byte
-	Refcount int32
-}
-
-type RtMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Index   uint16
-	_       [2]byte
-	Flags   int32
-	Addrs   int32
-	Pid     int32
-	Seq     int32
-	Errno   int32
-	Use     int32
-	Inits   uint32
-	Rmx     RtMetrics
-}
-
-type RtMetrics struct {
-	Locks    uint32
-	Mtu      uint32
-	Hopcount uint32
-	Expire   int32
-	Recvpipe uint32
-	Sendpipe uint32
-	Ssthresh uint32
-	Rtt      uint32
-	Rttvar   uint32
-	Pksent   uint32
-	Filler   [4]uint32
-}
-
-const (
-	SizeofBpfVersion = 0x4
-	SizeofBpfStat    = 0x8
-	SizeofBpfProgram = 0x8
-	SizeofBpfInsn    = 0x8
-	SizeofBpfHdr     = 0x14
-)
-
-type BpfVersion struct {
-	Major uint16
-	Minor uint16
-}
-
-type BpfStat struct {
-	Recv uint32
-	Drop uint32
-}
-
-type BpfProgram struct {
-	Len   uint32
-	Insns *BpfInsn
-}
-
-type BpfInsn struct {
-	Code uint16
-	Jt   uint8
-	Jf   uint8
-	K    uint32
-}
-
-type BpfHdr struct {
-	Tstamp  Timeval
-	Caplen  uint32
-	Datalen uint32
-	Hdrlen  uint16
-	_       [2]byte
-}
-
-type Termios struct {
-	Iflag  uint32
-	Oflag  uint32
-	Cflag  uint32
-	Lflag  uint32
-	Cc     [20]uint8
-	Ispeed uint32
-	Ospeed uint32
-}
-
-type Winsize struct {
-	Row    uint16
-	Col    uint16
-	Xpixel uint16
-	Ypixel uint16
-}
-
-const (
-	AT_FDCWD            = -0x2
-	AT_REMOVEDIR        = 0x80
-	AT_SYMLINK_FOLLOW   = 0x40
-	AT_SYMLINK_NOFOLLOW = 0x20
-)
-
-type PollFd struct {
-	Fd      int32
-	Events  int16
-	Revents int16
-}
-
-const (
-	POLLERR    = 0x8
-	POLLHUP    = 0x10
-	POLLIN     = 0x1
-	POLLNVAL   = 0x20
-	POLLOUT    = 0x4
-	POLLPRI    = 0x2
-	POLLRDBAND = 0x80
-	POLLRDNORM = 0x40
-	POLLWRBAND = 0x100
-	POLLWRNORM = 0x4
-)
-
-type Utsname struct {
-	Sysname  [256]byte
-	Nodename [256]byte
-	Release  [256]byte
-	Version  [256]byte
-	Machine  [256]byte
-}
-
-const SizeofClockinfo = 0x14
-
-type Clockinfo struct {
-	Hz      int32
-	Tick    int32
-	Tickadj int32
-	Stathz  int32
-	Profhz  int32
-}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go
index e6576d1..2673e6c 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_darwin_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_darwin.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && darwin
 // +build amd64,darwin
 
 package unix
@@ -70,7 +71,6 @@
 	Uid     uint32
 	Gid     uint32
 	Rdev    int32
-	_       [4]byte
 	Atim    Timespec
 	Mtim    Timespec
 	Ctim    Timespec
@@ -97,10 +97,11 @@
 	Type        uint32
 	Flags       uint32
 	Fssubtype   uint32
-	Fstypename  [16]int8
-	Mntonname   [1024]int8
-	Mntfromname [1024]int8
-	Reserved    [8]uint32
+	Fstypename  [16]byte
+	Mntonname   [1024]byte
+	Mntfromname [1024]byte
+	Flags_ext   uint32
+	Reserved    [7]uint32
 }
 
 type Flock_t struct {
@@ -133,8 +134,7 @@
 
 type Log2phys_t struct {
 	Flags uint32
-	_     [8]byte
-	_     [8]byte
+	_     [16]byte
 }
 
 type Fsid struct {
@@ -200,8 +200,24 @@
 	Pad  [92]int8
 }
 
+type RawSockaddrCtl struct {
+	Sc_len      uint8
+	Sc_family   uint8
+	Ss_sysaddr  uint16
+	Sc_id       uint32
+	Sc_unit     uint32
+	Sc_reserved [5]uint32
+}
+
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -217,6 +233,12 @@
 	Interface [4]byte /* in_addr */
 }
 
+type IPMreqn struct {
+	Multiaddr [4]byte /* in_addr */
+	Address   [4]byte /* in_addr */
+	Ifindex   int32
+}
+
 type IPv6Mreq struct {
 	Multiaddr [16]byte /* in6_addr */
 	Interface uint32
@@ -225,10 +247,8 @@
 type Msghdr struct {
 	Name       *byte
 	Namelen    uint32
-	_          [4]byte
 	Iov        *Iovec
 	Iovlen     int32
-	_          [4]byte
 	Control    *byte
 	Controllen uint32
 	Flags      int32
@@ -266,8 +286,12 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
+	SizeofSockaddrCtl      = 0x20
+	SizeofXucred           = 0x4c
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
+	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
 	SizeofCmsghdr          = 0xc
@@ -313,7 +337,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Data    IfData
 }
 
@@ -356,7 +379,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Metric  int32
 }
 
@@ -377,7 +399,6 @@
 	Addrs    int32
 	Flags    int32
 	Index    uint16
-	_        [2]byte
 	Refcount int32
 }
 
@@ -386,7 +407,6 @@
 	Version uint8
 	Type    uint8
 	Index   uint16
-	_       [2]byte
 	Flags   int32
 	Addrs   int32
 	Pid     int32
@@ -408,7 +428,8 @@
 	Rtt      uint32
 	Rttvar   uint32
 	Pksent   uint32
-	Filler   [4]uint32
+	State    uint32
+	Filler   [3]uint32
 }
 
 const (
@@ -431,7 +452,6 @@
 
 type BpfProgram struct {
 	Len   uint32
-	_     [4]byte
 	Insns *BpfInsn
 }
 
@@ -456,7 +476,6 @@
 	Cflag  uint64
 	Lflag  uint64
 	Cc     [20]uint8
-	_      [4]byte
 	Ispeed uint64
 	Ospeed uint64
 }
@@ -511,3 +530,8 @@
 	Stathz  int32
 	Profhz  int32
 }
+
+type CtlInfo struct {
+	Id   uint32
+	Name [96]byte
+}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_arm.go
deleted file mode 100644
index af9560f..0000000
--- a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm.go
+++ /dev/null
@@ -1,504 +0,0 @@
-// NOTE: cgo can't generate struct Stat_t and struct Statfs_t yet
-// Created by cgo -godefs - DO NOT EDIT
-// cgo -godefs types_darwin.go
-
-// +build arm,darwin
-
-package unix
-
-const (
-	SizeofPtr      = 0x4
-	SizeofShort    = 0x2
-	SizeofInt      = 0x4
-	SizeofLong     = 0x4
-	SizeofLongLong = 0x8
-)
-
-type (
-	_C_short     int16
-	_C_int       int32
-	_C_long      int32
-	_C_long_long int64
-)
-
-type Timespec struct {
-	Sec  int32
-	Nsec int32
-}
-
-type Timeval struct {
-	Sec  int32
-	Usec int32
-}
-
-type Timeval32 [0]byte
-
-type Rusage struct {
-	Utime    Timeval
-	Stime    Timeval
-	Maxrss   int32
-	Ixrss    int32
-	Idrss    int32
-	Isrss    int32
-	Minflt   int32
-	Majflt   int32
-	Nswap    int32
-	Inblock  int32
-	Oublock  int32
-	Msgsnd   int32
-	Msgrcv   int32
-	Nsignals int32
-	Nvcsw    int32
-	Nivcsw   int32
-}
-
-type Rlimit struct {
-	Cur uint64
-	Max uint64
-}
-
-type _Gid_t uint32
-
-type Stat_t struct {
-	Dev     int32
-	Mode    uint16
-	Nlink   uint16
-	Ino     uint64
-	Uid     uint32
-	Gid     uint32
-	Rdev    int32
-	Atim    Timespec
-	Mtim    Timespec
-	Ctim    Timespec
-	Btim    Timespec
-	Size    int64
-	Blocks  int64
-	Blksize int32
-	Flags   uint32
-	Gen     uint32
-	Lspare  int32
-	Qspare  [2]int64
-}
-
-type Statfs_t struct {
-	Bsize       uint32
-	Iosize      int32
-	Blocks      uint64
-	Bfree       uint64
-	Bavail      uint64
-	Files       uint64
-	Ffree       uint64
-	Fsid        Fsid
-	Owner       uint32
-	Type        uint32
-	Flags       uint32
-	Fssubtype   uint32
-	Fstypename  [16]int8
-	Mntonname   [1024]int8
-	Mntfromname [1024]int8
-	Reserved    [8]uint32
-}
-
-type Flock_t struct {
-	Start  int64
-	Len    int64
-	Pid    int32
-	Type   int16
-	Whence int16
-}
-
-type Fstore_t struct {
-	Flags      uint32
-	Posmode    int32
-	Offset     int64
-	Length     int64
-	Bytesalloc int64
-}
-
-type Radvisory_t struct {
-	Offset int64
-	Count  int32
-}
-
-type Fbootstraptransfer_t struct {
-	Offset int64
-	Length uint32
-	Buffer *byte
-}
-
-type Log2phys_t struct {
-	Flags       uint32
-	Contigbytes int64
-	Devoffset   int64
-}
-
-type Fsid struct {
-	Val [2]int32
-}
-
-type Dirent struct {
-	Ino     uint64
-	Seekoff uint64
-	Reclen  uint16
-	Namlen  uint16
-	Type    uint8
-	Name    [1024]int8
-	_       [3]byte
-}
-
-const (
-	PathMax = 0x400
-)
-
-type RawSockaddrInet4 struct {
-	Len    uint8
-	Family uint8
-	Port   uint16
-	Addr   [4]byte /* in_addr */
-	Zero   [8]int8
-}
-
-type RawSockaddrInet6 struct {
-	Len      uint8
-	Family   uint8
-	Port     uint16
-	Flowinfo uint32
-	Addr     [16]byte /* in6_addr */
-	Scope_id uint32
-}
-
-type RawSockaddrUnix struct {
-	Len    uint8
-	Family uint8
-	Path   [104]int8
-}
-
-type RawSockaddrDatalink struct {
-	Len    uint8
-	Family uint8
-	Index  uint16
-	Type   uint8
-	Nlen   uint8
-	Alen   uint8
-	Slen   uint8
-	Data   [12]int8
-}
-
-type RawSockaddr struct {
-	Len    uint8
-	Family uint8
-	Data   [14]int8
-}
-
-type RawSockaddrAny struct {
-	Addr RawSockaddr
-	Pad  [92]int8
-}
-
-type _Socklen uint32
-
-type Linger struct {
-	Onoff  int32
-	Linger int32
-}
-
-type Iovec struct {
-	Base *byte
-	Len  uint32
-}
-
-type IPMreq struct {
-	Multiaddr [4]byte /* in_addr */
-	Interface [4]byte /* in_addr */
-}
-
-type IPv6Mreq struct {
-	Multiaddr [16]byte /* in6_addr */
-	Interface uint32
-}
-
-type Msghdr struct {
-	Name       *byte
-	Namelen    uint32
-	Iov        *Iovec
-	Iovlen     int32
-	Control    *byte
-	Controllen uint32
-	Flags      int32
-}
-
-type Cmsghdr struct {
-	Len   uint32
-	Level int32
-	Type  int32
-}
-
-type Inet4Pktinfo struct {
-	Ifindex  uint32
-	Spec_dst [4]byte /* in_addr */
-	Addr     [4]byte /* in_addr */
-}
-
-type Inet6Pktinfo struct {
-	Addr    [16]byte /* in6_addr */
-	Ifindex uint32
-}
-
-type IPv6MTUInfo struct {
-	Addr RawSockaddrInet6
-	Mtu  uint32
-}
-
-type ICMPv6Filter struct {
-	Filt [8]uint32
-}
-
-const (
-	SizeofSockaddrInet4    = 0x10
-	SizeofSockaddrInet6    = 0x1c
-	SizeofSockaddrAny      = 0x6c
-	SizeofSockaddrUnix     = 0x6a
-	SizeofSockaddrDatalink = 0x14
-	SizeofLinger           = 0x8
-	SizeofIPMreq           = 0x8
-	SizeofIPv6Mreq         = 0x14
-	SizeofMsghdr           = 0x1c
-	SizeofCmsghdr          = 0xc
-	SizeofInet4Pktinfo     = 0xc
-	SizeofInet6Pktinfo     = 0x14
-	SizeofIPv6MTUInfo      = 0x20
-	SizeofICMPv6Filter     = 0x20
-)
-
-const (
-	PTRACE_TRACEME = 0x0
-	PTRACE_CONT    = 0x7
-	PTRACE_KILL    = 0x8
-)
-
-type Kevent_t struct {
-	Ident  uint32
-	Filter int16
-	Flags  uint16
-	Fflags uint32
-	Data   int32
-	Udata  *byte
-}
-
-type FdSet struct {
-	Bits [32]int32
-}
-
-const (
-	SizeofIfMsghdr    = 0x70
-	SizeofIfData      = 0x60
-	SizeofIfaMsghdr   = 0x14
-	SizeofIfmaMsghdr  = 0x10
-	SizeofIfmaMsghdr2 = 0x14
-	SizeofRtMsghdr    = 0x5c
-	SizeofRtMetrics   = 0x38
-)
-
-type IfMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-	Data    IfData
-}
-
-type IfData struct {
-	Type       uint8
-	Typelen    uint8
-	Physical   uint8
-	Addrlen    uint8
-	Hdrlen     uint8
-	Recvquota  uint8
-	Xmitquota  uint8
-	Unused1    uint8
-	Mtu        uint32
-	Metric     uint32
-	Baudrate   uint32
-	Ipackets   uint32
-	Ierrors    uint32
-	Opackets   uint32
-	Oerrors    uint32
-	Collisions uint32
-	Ibytes     uint32
-	Obytes     uint32
-	Imcasts    uint32
-	Omcasts    uint32
-	Iqdrops    uint32
-	Noproto    uint32
-	Recvtiming uint32
-	Xmittiming uint32
-	Lastchange Timeval
-	Unused2    uint32
-	Hwassist   uint32
-	Reserved1  uint32
-	Reserved2  uint32
-}
-
-type IfaMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-	Metric  int32
-}
-
-type IfmaMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-}
-
-type IfmaMsghdr2 struct {
-	Msglen   uint16
-	Version  uint8
-	Type     uint8
-	Addrs    int32
-	Flags    int32
-	Index    uint16
-	_        [2]byte
-	Refcount int32
-}
-
-type RtMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Index   uint16
-	_       [2]byte
-	Flags   int32
-	Addrs   int32
-	Pid     int32
-	Seq     int32
-	Errno   int32
-	Use     int32
-	Inits   uint32
-	Rmx     RtMetrics
-}
-
-type RtMetrics struct {
-	Locks    uint32
-	Mtu      uint32
-	Hopcount uint32
-	Expire   int32
-	Recvpipe uint32
-	Sendpipe uint32
-	Ssthresh uint32
-	Rtt      uint32
-	Rttvar   uint32
-	Pksent   uint32
-	Filler   [4]uint32
-}
-
-const (
-	SizeofBpfVersion = 0x4
-	SizeofBpfStat    = 0x8
-	SizeofBpfProgram = 0x8
-	SizeofBpfInsn    = 0x8
-	SizeofBpfHdr     = 0x14
-)
-
-type BpfVersion struct {
-	Major uint16
-	Minor uint16
-}
-
-type BpfStat struct {
-	Recv uint32
-	Drop uint32
-}
-
-type BpfProgram struct {
-	Len   uint32
-	Insns *BpfInsn
-}
-
-type BpfInsn struct {
-	Code uint16
-	Jt   uint8
-	Jf   uint8
-	K    uint32
-}
-
-type BpfHdr struct {
-	Tstamp  Timeval
-	Caplen  uint32
-	Datalen uint32
-	Hdrlen  uint16
-	_       [2]byte
-}
-
-type Termios struct {
-	Iflag  uint32
-	Oflag  uint32
-	Cflag  uint32
-	Lflag  uint32
-	Cc     [20]uint8
-	Ispeed uint32
-	Ospeed uint32
-}
-
-type Winsize struct {
-	Row    uint16
-	Col    uint16
-	Xpixel uint16
-	Ypixel uint16
-}
-
-const (
-	AT_FDCWD            = -0x2
-	AT_REMOVEDIR        = 0x80
-	AT_SYMLINK_FOLLOW   = 0x40
-	AT_SYMLINK_NOFOLLOW = 0x20
-)
-
-type PollFd struct {
-	Fd      int32
-	Events  int16
-	Revents int16
-}
-
-const (
-	POLLERR    = 0x8
-	POLLHUP    = 0x10
-	POLLIN     = 0x1
-	POLLNVAL   = 0x20
-	POLLOUT    = 0x4
-	POLLPRI    = 0x2
-	POLLRDBAND = 0x80
-	POLLRDNORM = 0x40
-	POLLWRBAND = 0x100
-	POLLWRNORM = 0x4
-)
-
-type Utsname struct {
-	Sysname  [256]byte
-	Nodename [256]byte
-	Release  [256]byte
-	Version  [256]byte
-	Machine  [256]byte
-}
-
-const SizeofClockinfo = 0x14
-
-type Clockinfo struct {
-	Hz      int32
-	Tick    int32
-	Tickadj int32
-	Stathz  int32
-	Profhz  int32
-}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go
index a09c0f9..1465cbc 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_darwin_arm64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_darwin.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && darwin
 // +build arm64,darwin
 
 package unix
@@ -70,7 +71,6 @@
 	Uid     uint32
 	Gid     uint32
 	Rdev    int32
-	_       [4]byte
 	Atim    Timespec
 	Mtim    Timespec
 	Ctim    Timespec
@@ -97,10 +97,11 @@
 	Type        uint32
 	Flags       uint32
 	Fssubtype   uint32
-	Fstypename  [16]int8
-	Mntonname   [1024]int8
-	Mntfromname [1024]int8
-	Reserved    [8]uint32
+	Fstypename  [16]byte
+	Mntonname   [1024]byte
+	Mntfromname [1024]byte
+	Flags_ext   uint32
+	Reserved    [7]uint32
 }
 
 type Flock_t struct {
@@ -133,8 +134,7 @@
 
 type Log2phys_t struct {
 	Flags uint32
-	_     [8]byte
-	_     [8]byte
+	_     [16]byte
 }
 
 type Fsid struct {
@@ -200,8 +200,24 @@
 	Pad  [92]int8
 }
 
+type RawSockaddrCtl struct {
+	Sc_len      uint8
+	Sc_family   uint8
+	Ss_sysaddr  uint16
+	Sc_id       uint32
+	Sc_unit     uint32
+	Sc_reserved [5]uint32
+}
+
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -217,6 +233,12 @@
 	Interface [4]byte /* in_addr */
 }
 
+type IPMreqn struct {
+	Multiaddr [4]byte /* in_addr */
+	Address   [4]byte /* in_addr */
+	Ifindex   int32
+}
+
 type IPv6Mreq struct {
 	Multiaddr [16]byte /* in6_addr */
 	Interface uint32
@@ -225,10 +247,8 @@
 type Msghdr struct {
 	Name       *byte
 	Namelen    uint32
-	_          [4]byte
 	Iov        *Iovec
 	Iovlen     int32
-	_          [4]byte
 	Control    *byte
 	Controllen uint32
 	Flags      int32
@@ -266,8 +286,12 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
+	SizeofSockaddrCtl      = 0x20
+	SizeofXucred           = 0x4c
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
+	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
 	SizeofCmsghdr          = 0xc
@@ -313,7 +337,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Data    IfData
 }
 
@@ -356,7 +379,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Metric  int32
 }
 
@@ -377,7 +399,6 @@
 	Addrs    int32
 	Flags    int32
 	Index    uint16
-	_        [2]byte
 	Refcount int32
 }
 
@@ -386,7 +407,6 @@
 	Version uint8
 	Type    uint8
 	Index   uint16
-	_       [2]byte
 	Flags   int32
 	Addrs   int32
 	Pid     int32
@@ -408,7 +428,8 @@
 	Rtt      uint32
 	Rttvar   uint32
 	Pksent   uint32
-	Filler   [4]uint32
+	State    uint32
+	Filler   [3]uint32
 }
 
 const (
@@ -431,7 +452,6 @@
 
 type BpfProgram struct {
 	Len   uint32
-	_     [4]byte
 	Insns *BpfInsn
 }
 
@@ -456,7 +476,6 @@
 	Cflag  uint64
 	Lflag  uint64
 	Cc     [20]uint8
-	_      [4]byte
 	Ispeed uint64
 	Ospeed uint64
 }
@@ -511,3 +530,8 @@
 	Stathz  int32
 	Profhz  int32
 }
+
+type CtlInfo struct {
+	Id   uint32
+	Name [96]byte
+}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_dragonfly_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_dragonfly_amd64.go
index 71ea1d6..d0ba8e9 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_dragonfly_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_dragonfly_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_dragonfly.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && dragonfly
 // +build amd64,dragonfly
 
 package unix
@@ -70,11 +71,11 @@
 	Ctim    Timespec
 	Size    int64
 	Blocks  int64
-	Blksize uint32
+	_       uint32
 	Flags   uint32
 	Gen     uint32
 	Lspare  int32
-	Qspare1 int64
+	Blksize int64
 	Qspare2 int64
 }
 
@@ -91,17 +92,15 @@
 	Owner       uint32
 	Type        int32
 	Flags       int32
-	_           [4]byte
 	Syncwrites  int64
 	Asyncwrites int64
-	Fstypename  [16]int8
-	Mntonname   [80]int8
+	Fstypename  [16]byte
+	Mntonname   [80]byte
 	Syncreads   int64
 	Asyncreads  int64
 	Spares1     int16
-	Mntfromname [80]int8
+	Mntfromname [80]byte
 	Spares2     int16
-	_           [4]byte
 	Spare       [2]int64
 }
 
@@ -202,10 +201,8 @@
 type Msghdr struct {
 	Name       *byte
 	Namelen    uint32
-	_          [4]byte
 	Iov        *Iovec
 	Iovlen     int32
-	_          [4]byte
 	Control    *byte
 	Controllen uint32
 	Flags      int32
@@ -238,6 +235,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x36
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -269,7 +267,7 @@
 const (
 	SizeofIfMsghdr         = 0xb0
 	SizeofIfData           = 0xa0
-	SizeofIfaMsghdr        = 0x14
+	SizeofIfaMsghdr        = 0x18
 	SizeofIfmaMsghdr       = 0x10
 	SizeofIfAnnounceMsghdr = 0x18
 	SizeofRtMsghdr         = 0x98
@@ -280,10 +278,9 @@
 	Msglen  uint16
 	Version uint8
 	Type    uint8
-	Addrs   int32
-	Flags   int32
 	Index   uint16
-	_       [2]byte
+	Flags   int32
+	Addrs   int32
 	Data    IfData
 }
 
@@ -294,7 +291,6 @@
 	Hdrlen     uint8
 	Recvquota  uint8
 	Xmitquota  uint8
-	_          [2]byte
 	Mtu        uint64
 	Metric     uint64
 	Link_state uint64
@@ -316,24 +312,23 @@
 }
 
 type IfaMsghdr struct {
-	Msglen  uint16
-	Version uint8
-	Type    uint8
-	Addrs   int32
-	Flags   int32
-	Index   uint16
-	_       [2]byte
-	Metric  int32
+	Msglen    uint16
+	Version   uint8
+	Type      uint8
+	Index     uint16
+	Flags     int32
+	Addrs     int32
+	Addrflags int32
+	Metric    int32
 }
 
 type IfmaMsghdr struct {
 	Msglen  uint16
 	Version uint8
 	Type    uint8
-	Addrs   int32
-	Flags   int32
 	Index   uint16
-	_       [2]byte
+	Flags   int32
+	Addrs   int32
 }
 
 type IfAnnounceMsghdr struct {
@@ -350,7 +345,6 @@
 	Version uint8
 	Type    uint8
 	Index   uint16
-	_       [2]byte
 	Flags   int32
 	Addrs   int32
 	Pid     int32
@@ -374,7 +368,6 @@
 	Hopcount  uint64
 	Mssopt    uint16
 	Pad       uint16
-	_         [4]byte
 	Msl       uint64
 	Iwmaxsegs uint64
 	Iwcapsegs uint64
@@ -400,7 +393,6 @@
 
 type BpfProgram struct {
 	Len   uint32
-	_     [4]byte
 	Insns *BpfInsn
 }
 
@@ -439,6 +431,9 @@
 const (
 	AT_FDCWD            = 0xfffafdcd
 	AT_SYMLINK_NOFOLLOW = 0x1
+	AT_REMOVEDIR        = 0x2
+	AT_EACCESS          = 0x4
+	AT_SYMLINK_FOLLOW   = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_freebsd_386.go b/vendor/golang.org/x/sys/unix/ztypes_freebsd_386.go
index 2a3ec61..1f99c02 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_freebsd_386.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_freebsd_386.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_freebsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && freebsd
 // +build 386,freebsd
 
 package unix
@@ -250,6 +251,14 @@
 
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+	_       *byte
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -312,7 +321,9 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x36
+	SizeofXucred           = 0x50
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
@@ -661,9 +672,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_REMOVEDIR        = 0x800
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_freebsd_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_freebsd_amd64.go
index e11e954..ddf0305 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_freebsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_freebsd_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_freebsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && freebsd
 // +build amd64,freebsd
 
 package unix
@@ -246,6 +247,14 @@
 
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+	_       *byte
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -308,7 +317,9 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x36
+	SizeofXucred           = 0x58
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
@@ -664,9 +675,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_REMOVEDIR        = 0x800
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm.go b/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm.go
index b91c2ae..dce0a5c 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm.go
@@ -1,6 +1,7 @@
 // cgo -godefs -- -fsigned-char types_freebsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && freebsd
 // +build arm,freebsd
 
 package unix
@@ -248,6 +249,14 @@
 
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+	_       *byte
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -310,7 +319,9 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x36
+	SizeofXucred           = 0x50
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
@@ -645,9 +656,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_REMOVEDIR        = 0x800
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm64.go
index c6fe1d0..e232447 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_freebsd_arm64.go
@@ -1,6 +1,7 @@
 // cgo -godefs -- -fsigned-char types_freebsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && freebsd
 // +build arm64,freebsd
 
 package unix
@@ -246,6 +247,14 @@
 
 type _Socklen uint32
 
+type Xucred struct {
+	Version uint32
+	Uid     uint32
+	Ngroups int16
+	Groups  [16]uint32
+	_       *byte
+}
+
 type Linger struct {
 	Onoff  int32
 	Linger int32
@@ -308,7 +317,9 @@
 	SizeofSockaddrAny      = 0x6c
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x36
+	SizeofXucred           = 0x58
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPMreqn          = 0xc
 	SizeofIPv6Mreq         = 0x14
@@ -642,9 +653,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_REMOVEDIR        = 0x800
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_illumos_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_illumos_amd64.go
new file mode 100644
index 0000000..236f37e
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ztypes_illumos_amd64.go
@@ -0,0 +1,40 @@
+// cgo -godefs types_illumos.go | go run mkpost.go
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build amd64 && illumos
+// +build amd64,illumos
+
+package unix
+
+const (
+	TUNNEWPPA = 0x540001
+	TUNSETPPA = 0x540002
+
+	I_STR     = 0x5308
+	I_POP     = 0x5303
+	I_PUSH    = 0x5302
+	I_PLINK   = 0x5316
+	I_PUNLINK = 0x5317
+
+	IF_UNITSEL = -0x7ffb8cca
+)
+
+type strbuf struct {
+	Maxlen int32
+	Len    int32
+	Buf    *int8
+}
+
+type Strioctl struct {
+	Cmd    int32
+	Timout int32
+	Len    int32
+	Dp     *int8
+}
+
+type Lifreq struct {
+	Name   [32]int8
+	Lifru1 [4]byte
+	Type   uint32
+	Lifru  [336]byte
+}
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux.go b/vendor/golang.org/x/sys/unix/ztypes_linux.go
index a92a501..72887ab 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux.go
@@ -1,5 +1,6 @@
 // Code generated by mkmerge.go; DO NOT EDIT.
 
+//go:build linux
 // +build linux
 
 package unix
@@ -83,7 +84,7 @@
 	Dest_offset uint64
 }
 
-type FileDedupeRange struct {
+type RawFileDedupeRange struct {
 	Src_offset uint64
 	Src_length uint64
 	Dest_count uint16
@@ -91,6 +92,21 @@
 	Reserved2  uint32
 }
 
+type RawFileDedupeRangeInfo struct {
+	Dest_fd       int64
+	Dest_offset   uint64
+	Bytes_deduped uint64
+	Status        int32
+	Reserved      uint32
+}
+
+const (
+	SizeofRawFileDedupeRange     = 0x18
+	SizeofRawFileDedupeRangeInfo = 0x20
+	FILE_DEDUPE_RANGE_SAME       = 0x0
+	FILE_DEDUPE_RANGE_DIFFERS    = 0x1
+)
+
 type FscryptPolicy struct {
 	Version                   uint8
 	Contents_encryption_mode  uint8
@@ -288,7 +304,8 @@
 	Reserved1 uint16
 	Port      uint32
 	Cid       uint32
-	Zero      [4]uint8
+	Flags     uint8
+	Zero      [3]uint8
 }
 
 type RawSockaddrXDP struct {
@@ -334,6 +351,13 @@
 	Name    [8]int8
 }
 
+type RawSockaddrNFC struct {
+	Sa_family    uint16
+	Dev_idx      uint32
+	Target_idx   uint32
+	Nfc_protocol uint32
+}
+
 type _Socklen uint32
 
 type Linger struct {
@@ -447,6 +471,7 @@
 	SizeofSockaddrL2TPIP    = 0x10
 	SizeofSockaddrL2TPIP6   = 0x20
 	SizeofSockaddrIUCV      = 0x20
+	SizeofSockaddrNFC       = 0x10
 	SizeofLinger            = 0x8
 	SizeofIPMreq            = 0x8
 	SizeofIPMreqn           = 0xc
@@ -462,166 +487,107 @@
 )
 
 const (
-	NDA_UNSPEC              = 0x0
-	NDA_DST                 = 0x1
-	NDA_LLADDR              = 0x2
-	NDA_CACHEINFO           = 0x3
-	NDA_PROBES              = 0x4
-	NDA_VLAN                = 0x5
-	NDA_PORT                = 0x6
-	NDA_VNI                 = 0x7
-	NDA_IFINDEX             = 0x8
-	NDA_MASTER              = 0x9
-	NDA_LINK_NETNSID        = 0xa
-	NDA_SRC_VNI             = 0xb
-	NTF_USE                 = 0x1
-	NTF_SELF                = 0x2
-	NTF_MASTER              = 0x4
-	NTF_PROXY               = 0x8
-	NTF_EXT_LEARNED         = 0x10
-	NTF_OFFLOADED           = 0x20
-	NTF_ROUTER              = 0x80
-	NUD_INCOMPLETE          = 0x1
-	NUD_REACHABLE           = 0x2
-	NUD_STALE               = 0x4
-	NUD_DELAY               = 0x8
-	NUD_PROBE               = 0x10
-	NUD_FAILED              = 0x20
-	NUD_NOARP               = 0x40
-	NUD_PERMANENT           = 0x80
-	NUD_NONE                = 0x0
-	IFA_UNSPEC              = 0x0
-	IFA_ADDRESS             = 0x1
-	IFA_LOCAL               = 0x2
-	IFA_LABEL               = 0x3
-	IFA_BROADCAST           = 0x4
-	IFA_ANYCAST             = 0x5
-	IFA_CACHEINFO           = 0x6
-	IFA_MULTICAST           = 0x7
-	IFA_FLAGS               = 0x8
-	IFA_RT_PRIORITY         = 0x9
-	IFA_TARGET_NETNSID      = 0xa
-	IFLA_UNSPEC             = 0x0
-	IFLA_ADDRESS            = 0x1
-	IFLA_BROADCAST          = 0x2
-	IFLA_IFNAME             = 0x3
-	IFLA_MTU                = 0x4
-	IFLA_LINK               = 0x5
-	IFLA_QDISC              = 0x6
-	IFLA_STATS              = 0x7
-	IFLA_COST               = 0x8
-	IFLA_PRIORITY           = 0x9
-	IFLA_MASTER             = 0xa
-	IFLA_WIRELESS           = 0xb
-	IFLA_PROTINFO           = 0xc
-	IFLA_TXQLEN             = 0xd
-	IFLA_MAP                = 0xe
-	IFLA_WEIGHT             = 0xf
-	IFLA_OPERSTATE          = 0x10
-	IFLA_LINKMODE           = 0x11
-	IFLA_LINKINFO           = 0x12
-	IFLA_NET_NS_PID         = 0x13
-	IFLA_IFALIAS            = 0x14
-	IFLA_NUM_VF             = 0x15
-	IFLA_VFINFO_LIST        = 0x16
-	IFLA_STATS64            = 0x17
-	IFLA_VF_PORTS           = 0x18
-	IFLA_PORT_SELF          = 0x19
-	IFLA_AF_SPEC            = 0x1a
-	IFLA_GROUP              = 0x1b
-	IFLA_NET_NS_FD          = 0x1c
-	IFLA_EXT_MASK           = 0x1d
-	IFLA_PROMISCUITY        = 0x1e
-	IFLA_NUM_TX_QUEUES      = 0x1f
-	IFLA_NUM_RX_QUEUES      = 0x20
-	IFLA_CARRIER            = 0x21
-	IFLA_PHYS_PORT_ID       = 0x22
-	IFLA_CARRIER_CHANGES    = 0x23
-	IFLA_PHYS_SWITCH_ID     = 0x24
-	IFLA_LINK_NETNSID       = 0x25
-	IFLA_PHYS_PORT_NAME     = 0x26
-	IFLA_PROTO_DOWN         = 0x27
-	IFLA_GSO_MAX_SEGS       = 0x28
-	IFLA_GSO_MAX_SIZE       = 0x29
-	IFLA_PAD                = 0x2a
-	IFLA_XDP                = 0x2b
-	IFLA_EVENT              = 0x2c
-	IFLA_NEW_NETNSID        = 0x2d
-	IFLA_IF_NETNSID         = 0x2e
-	IFLA_TARGET_NETNSID     = 0x2e
-	IFLA_CARRIER_UP_COUNT   = 0x2f
-	IFLA_CARRIER_DOWN_COUNT = 0x30
-	IFLA_NEW_IFINDEX        = 0x31
-	IFLA_MIN_MTU            = 0x32
-	IFLA_MAX_MTU            = 0x33
-	IFLA_MAX                = 0x36
-	IFLA_INFO_KIND          = 0x1
-	IFLA_INFO_DATA          = 0x2
-	IFLA_INFO_XSTATS        = 0x3
-	IFLA_INFO_SLAVE_KIND    = 0x4
-	IFLA_INFO_SLAVE_DATA    = 0x5
-	RT_SCOPE_UNIVERSE       = 0x0
-	RT_SCOPE_SITE           = 0xc8
-	RT_SCOPE_LINK           = 0xfd
-	RT_SCOPE_HOST           = 0xfe
-	RT_SCOPE_NOWHERE        = 0xff
-	RT_TABLE_UNSPEC         = 0x0
-	RT_TABLE_COMPAT         = 0xfc
-	RT_TABLE_DEFAULT        = 0xfd
-	RT_TABLE_MAIN           = 0xfe
-	RT_TABLE_LOCAL          = 0xff
-	RT_TABLE_MAX            = 0xffffffff
-	RTA_UNSPEC              = 0x0
-	RTA_DST                 = 0x1
-	RTA_SRC                 = 0x2
-	RTA_IIF                 = 0x3
-	RTA_OIF                 = 0x4
-	RTA_GATEWAY             = 0x5
-	RTA_PRIORITY            = 0x6
-	RTA_PREFSRC             = 0x7
-	RTA_METRICS             = 0x8
-	RTA_MULTIPATH           = 0x9
-	RTA_FLOW                = 0xb
-	RTA_CACHEINFO           = 0xc
-	RTA_TABLE               = 0xf
-	RTA_MARK                = 0x10
-	RTA_MFC_STATS           = 0x11
-	RTA_VIA                 = 0x12
-	RTA_NEWDST              = 0x13
-	RTA_PREF                = 0x14
-	RTA_ENCAP_TYPE          = 0x15
-	RTA_ENCAP               = 0x16
-	RTA_EXPIRES             = 0x17
-	RTA_PAD                 = 0x18
-	RTA_UID                 = 0x19
-	RTA_TTL_PROPAGATE       = 0x1a
-	RTA_IP_PROTO            = 0x1b
-	RTA_SPORT               = 0x1c
-	RTA_DPORT               = 0x1d
-	RTN_UNSPEC              = 0x0
-	RTN_UNICAST             = 0x1
-	RTN_LOCAL               = 0x2
-	RTN_BROADCAST           = 0x3
-	RTN_ANYCAST             = 0x4
-	RTN_MULTICAST           = 0x5
-	RTN_BLACKHOLE           = 0x6
-	RTN_UNREACHABLE         = 0x7
-	RTN_PROHIBIT            = 0x8
-	RTN_THROW               = 0x9
-	RTN_NAT                 = 0xa
-	RTN_XRESOLVE            = 0xb
-	SizeofNlMsghdr          = 0x10
-	SizeofNlMsgerr          = 0x14
-	SizeofRtGenmsg          = 0x1
-	SizeofNlAttr            = 0x4
-	SizeofRtAttr            = 0x4
-	SizeofIfInfomsg         = 0x10
-	SizeofIfAddrmsg         = 0x8
-	SizeofIfaCacheinfo      = 0x10
-	SizeofRtMsg             = 0xc
-	SizeofRtNexthop         = 0x8
-	SizeofNdUseroptmsg      = 0x10
-	SizeofNdMsg             = 0xc
+	NDA_UNSPEC         = 0x0
+	NDA_DST            = 0x1
+	NDA_LLADDR         = 0x2
+	NDA_CACHEINFO      = 0x3
+	NDA_PROBES         = 0x4
+	NDA_VLAN           = 0x5
+	NDA_PORT           = 0x6
+	NDA_VNI            = 0x7
+	NDA_IFINDEX        = 0x8
+	NDA_MASTER         = 0x9
+	NDA_LINK_NETNSID   = 0xa
+	NDA_SRC_VNI        = 0xb
+	NTF_USE            = 0x1
+	NTF_SELF           = 0x2
+	NTF_MASTER         = 0x4
+	NTF_PROXY          = 0x8
+	NTF_EXT_LEARNED    = 0x10
+	NTF_OFFLOADED      = 0x20
+	NTF_ROUTER         = 0x80
+	NUD_INCOMPLETE     = 0x1
+	NUD_REACHABLE      = 0x2
+	NUD_STALE          = 0x4
+	NUD_DELAY          = 0x8
+	NUD_PROBE          = 0x10
+	NUD_FAILED         = 0x20
+	NUD_NOARP          = 0x40
+	NUD_PERMANENT      = 0x80
+	NUD_NONE           = 0x0
+	IFA_UNSPEC         = 0x0
+	IFA_ADDRESS        = 0x1
+	IFA_LOCAL          = 0x2
+	IFA_LABEL          = 0x3
+	IFA_BROADCAST      = 0x4
+	IFA_ANYCAST        = 0x5
+	IFA_CACHEINFO      = 0x6
+	IFA_MULTICAST      = 0x7
+	IFA_FLAGS          = 0x8
+	IFA_RT_PRIORITY    = 0x9
+	IFA_TARGET_NETNSID = 0xa
+	RT_SCOPE_UNIVERSE  = 0x0
+	RT_SCOPE_SITE      = 0xc8
+	RT_SCOPE_LINK      = 0xfd
+	RT_SCOPE_HOST      = 0xfe
+	RT_SCOPE_NOWHERE   = 0xff
+	RT_TABLE_UNSPEC    = 0x0
+	RT_TABLE_COMPAT    = 0xfc
+	RT_TABLE_DEFAULT   = 0xfd
+	RT_TABLE_MAIN      = 0xfe
+	RT_TABLE_LOCAL     = 0xff
+	RT_TABLE_MAX       = 0xffffffff
+	RTA_UNSPEC         = 0x0
+	RTA_DST            = 0x1
+	RTA_SRC            = 0x2
+	RTA_IIF            = 0x3
+	RTA_OIF            = 0x4
+	RTA_GATEWAY        = 0x5
+	RTA_PRIORITY       = 0x6
+	RTA_PREFSRC        = 0x7
+	RTA_METRICS        = 0x8
+	RTA_MULTIPATH      = 0x9
+	RTA_FLOW           = 0xb
+	RTA_CACHEINFO      = 0xc
+	RTA_TABLE          = 0xf
+	RTA_MARK           = 0x10
+	RTA_MFC_STATS      = 0x11
+	RTA_VIA            = 0x12
+	RTA_NEWDST         = 0x13
+	RTA_PREF           = 0x14
+	RTA_ENCAP_TYPE     = 0x15
+	RTA_ENCAP          = 0x16
+	RTA_EXPIRES        = 0x17
+	RTA_PAD            = 0x18
+	RTA_UID            = 0x19
+	RTA_TTL_PROPAGATE  = 0x1a
+	RTA_IP_PROTO       = 0x1b
+	RTA_SPORT          = 0x1c
+	RTA_DPORT          = 0x1d
+	RTN_UNSPEC         = 0x0
+	RTN_UNICAST        = 0x1
+	RTN_LOCAL          = 0x2
+	RTN_BROADCAST      = 0x3
+	RTN_ANYCAST        = 0x4
+	RTN_MULTICAST      = 0x5
+	RTN_BLACKHOLE      = 0x6
+	RTN_UNREACHABLE    = 0x7
+	RTN_PROHIBIT       = 0x8
+	RTN_THROW          = 0x9
+	RTN_NAT            = 0xa
+	RTN_XRESOLVE       = 0xb
+	SizeofNlMsghdr     = 0x10
+	SizeofNlMsgerr     = 0x14
+	SizeofRtGenmsg     = 0x1
+	SizeofNlAttr       = 0x4
+	SizeofRtAttr       = 0x4
+	SizeofIfInfomsg    = 0x10
+	SizeofIfAddrmsg    = 0x8
+	SizeofIfaCacheinfo = 0x10
+	SizeofRtMsg        = 0xc
+	SizeofRtNexthop    = 0x8
+	SizeofNdUseroptmsg = 0x10
+	SizeofNdMsg        = 0xc
 )
 
 type NlMsghdr struct {
@@ -823,8 +789,6 @@
 	_         [28]uint8
 }
 
-const PERF_IOC_FLAG_GROUP = 0x1
-
 type Winsize struct {
 	Row    uint16
 	Col    uint16
@@ -948,7 +912,10 @@
 	Time_offset    uint64
 	Time_zero      uint64
 	Size           uint32
-	_              [948]uint8
+	_              uint32
+	Time_cycles    uint64
+	Time_mask      uint64
+	_              [928]uint8
 	Data_head      uint64
 	Data_tail      uint64
 	Data_offset    uint64
@@ -990,13 +957,13 @@
 )
 
 const (
-	PERF_TYPE_HARDWARE   = 0x0
-	PERF_TYPE_SOFTWARE   = 0x1
-	PERF_TYPE_TRACEPOINT = 0x2
-	PERF_TYPE_HW_CACHE   = 0x3
-	PERF_TYPE_RAW        = 0x4
-	PERF_TYPE_BREAKPOINT = 0x5
-
+	PERF_TYPE_HARDWARE                    = 0x0
+	PERF_TYPE_SOFTWARE                    = 0x1
+	PERF_TYPE_TRACEPOINT                  = 0x2
+	PERF_TYPE_HW_CACHE                    = 0x3
+	PERF_TYPE_RAW                         = 0x4
+	PERF_TYPE_BREAKPOINT                  = 0x5
+	PERF_TYPE_MAX                         = 0x6
 	PERF_COUNT_HW_CPU_CYCLES              = 0x0
 	PERF_COUNT_HW_INSTRUCTIONS            = 0x1
 	PERF_COUNT_HW_CACHE_REFERENCES        = 0x2
@@ -1007,106 +974,166 @@
 	PERF_COUNT_HW_STALLED_CYCLES_FRONTEND = 0x7
 	PERF_COUNT_HW_STALLED_CYCLES_BACKEND  = 0x8
 	PERF_COUNT_HW_REF_CPU_CYCLES          = 0x9
-
-	PERF_COUNT_HW_CACHE_L1D  = 0x0
-	PERF_COUNT_HW_CACHE_L1I  = 0x1
-	PERF_COUNT_HW_CACHE_LL   = 0x2
-	PERF_COUNT_HW_CACHE_DTLB = 0x3
-	PERF_COUNT_HW_CACHE_ITLB = 0x4
-	PERF_COUNT_HW_CACHE_BPU  = 0x5
-	PERF_COUNT_HW_CACHE_NODE = 0x6
-
-	PERF_COUNT_HW_CACHE_OP_READ     = 0x0
-	PERF_COUNT_HW_CACHE_OP_WRITE    = 0x1
-	PERF_COUNT_HW_CACHE_OP_PREFETCH = 0x2
-
-	PERF_COUNT_HW_CACHE_RESULT_ACCESS = 0x0
-	PERF_COUNT_HW_CACHE_RESULT_MISS   = 0x1
-
-	PERF_COUNT_SW_CPU_CLOCK        = 0x0
-	PERF_COUNT_SW_TASK_CLOCK       = 0x1
-	PERF_COUNT_SW_PAGE_FAULTS      = 0x2
-	PERF_COUNT_SW_CONTEXT_SWITCHES = 0x3
-	PERF_COUNT_SW_CPU_MIGRATIONS   = 0x4
-	PERF_COUNT_SW_PAGE_FAULTS_MIN  = 0x5
-	PERF_COUNT_SW_PAGE_FAULTS_MAJ  = 0x6
-	PERF_COUNT_SW_ALIGNMENT_FAULTS = 0x7
-	PERF_COUNT_SW_EMULATION_FAULTS = 0x8
-	PERF_COUNT_SW_DUMMY            = 0x9
-	PERF_COUNT_SW_BPF_OUTPUT       = 0xa
-
-	PERF_SAMPLE_IP           = 0x1
-	PERF_SAMPLE_TID          = 0x2
-	PERF_SAMPLE_TIME         = 0x4
-	PERF_SAMPLE_ADDR         = 0x8
-	PERF_SAMPLE_READ         = 0x10
-	PERF_SAMPLE_CALLCHAIN    = 0x20
-	PERF_SAMPLE_ID           = 0x40
-	PERF_SAMPLE_CPU          = 0x80
-	PERF_SAMPLE_PERIOD       = 0x100
-	PERF_SAMPLE_STREAM_ID    = 0x200
-	PERF_SAMPLE_RAW          = 0x400
-	PERF_SAMPLE_BRANCH_STACK = 0x800
-	PERF_SAMPLE_REGS_USER    = 0x1000
-	PERF_SAMPLE_STACK_USER   = 0x2000
-	PERF_SAMPLE_WEIGHT       = 0x4000
-	PERF_SAMPLE_DATA_SRC     = 0x8000
-	PERF_SAMPLE_IDENTIFIER   = 0x10000
-	PERF_SAMPLE_TRANSACTION  = 0x20000
-	PERF_SAMPLE_REGS_INTR    = 0x40000
-
-	PERF_SAMPLE_BRANCH_USER       = 0x1
-	PERF_SAMPLE_BRANCH_KERNEL     = 0x2
-	PERF_SAMPLE_BRANCH_HV         = 0x4
-	PERF_SAMPLE_BRANCH_ANY        = 0x8
-	PERF_SAMPLE_BRANCH_ANY_CALL   = 0x10
-	PERF_SAMPLE_BRANCH_ANY_RETURN = 0x20
-	PERF_SAMPLE_BRANCH_IND_CALL   = 0x40
-	PERF_SAMPLE_BRANCH_ABORT_TX   = 0x80
-	PERF_SAMPLE_BRANCH_IN_TX      = 0x100
-	PERF_SAMPLE_BRANCH_NO_TX      = 0x200
-	PERF_SAMPLE_BRANCH_COND       = 0x400
-	PERF_SAMPLE_BRANCH_CALL_STACK = 0x800
-	PERF_SAMPLE_BRANCH_IND_JUMP   = 0x1000
-	PERF_SAMPLE_BRANCH_CALL       = 0x2000
-	PERF_SAMPLE_BRANCH_NO_FLAGS   = 0x4000
-	PERF_SAMPLE_BRANCH_NO_CYCLES  = 0x8000
-	PERF_SAMPLE_BRANCH_TYPE_SAVE  = 0x10000
-
-	PERF_FORMAT_TOTAL_TIME_ENABLED = 0x1
-	PERF_FORMAT_TOTAL_TIME_RUNNING = 0x2
-	PERF_FORMAT_ID                 = 0x4
-	PERF_FORMAT_GROUP              = 0x8
-
-	PERF_RECORD_MMAP            = 0x1
-	PERF_RECORD_LOST            = 0x2
-	PERF_RECORD_COMM            = 0x3
-	PERF_RECORD_EXIT            = 0x4
-	PERF_RECORD_THROTTLE        = 0x5
-	PERF_RECORD_UNTHROTTLE      = 0x6
-	PERF_RECORD_FORK            = 0x7
-	PERF_RECORD_READ            = 0x8
-	PERF_RECORD_SAMPLE          = 0x9
-	PERF_RECORD_MMAP2           = 0xa
-	PERF_RECORD_AUX             = 0xb
-	PERF_RECORD_ITRACE_START    = 0xc
-	PERF_RECORD_LOST_SAMPLES    = 0xd
-	PERF_RECORD_SWITCH          = 0xe
-	PERF_RECORD_SWITCH_CPU_WIDE = 0xf
-	PERF_RECORD_NAMESPACES      = 0x10
-
-	PERF_CONTEXT_HV     = -0x20
-	PERF_CONTEXT_KERNEL = -0x80
-	PERF_CONTEXT_USER   = -0x200
-
-	PERF_CONTEXT_GUEST        = -0x800
-	PERF_CONTEXT_GUEST_KERNEL = -0x880
-	PERF_CONTEXT_GUEST_USER   = -0xa00
-
-	PERF_FLAG_FD_NO_GROUP = 0x1
-	PERF_FLAG_FD_OUTPUT   = 0x2
-	PERF_FLAG_PID_CGROUP  = 0x4
-	PERF_FLAG_FD_CLOEXEC  = 0x8
+	PERF_COUNT_HW_MAX                     = 0xa
+	PERF_COUNT_HW_CACHE_L1D               = 0x0
+	PERF_COUNT_HW_CACHE_L1I               = 0x1
+	PERF_COUNT_HW_CACHE_LL                = 0x2
+	PERF_COUNT_HW_CACHE_DTLB              = 0x3
+	PERF_COUNT_HW_CACHE_ITLB              = 0x4
+	PERF_COUNT_HW_CACHE_BPU               = 0x5
+	PERF_COUNT_HW_CACHE_NODE              = 0x6
+	PERF_COUNT_HW_CACHE_MAX               = 0x7
+	PERF_COUNT_HW_CACHE_OP_READ           = 0x0
+	PERF_COUNT_HW_CACHE_OP_WRITE          = 0x1
+	PERF_COUNT_HW_CACHE_OP_PREFETCH       = 0x2
+	PERF_COUNT_HW_CACHE_OP_MAX            = 0x3
+	PERF_COUNT_HW_CACHE_RESULT_ACCESS     = 0x0
+	PERF_COUNT_HW_CACHE_RESULT_MISS       = 0x1
+	PERF_COUNT_HW_CACHE_RESULT_MAX        = 0x2
+	PERF_COUNT_SW_CPU_CLOCK               = 0x0
+	PERF_COUNT_SW_TASK_CLOCK              = 0x1
+	PERF_COUNT_SW_PAGE_FAULTS             = 0x2
+	PERF_COUNT_SW_CONTEXT_SWITCHES        = 0x3
+	PERF_COUNT_SW_CPU_MIGRATIONS          = 0x4
+	PERF_COUNT_SW_PAGE_FAULTS_MIN         = 0x5
+	PERF_COUNT_SW_PAGE_FAULTS_MAJ         = 0x6
+	PERF_COUNT_SW_ALIGNMENT_FAULTS        = 0x7
+	PERF_COUNT_SW_EMULATION_FAULTS        = 0x8
+	PERF_COUNT_SW_DUMMY                   = 0x9
+	PERF_COUNT_SW_BPF_OUTPUT              = 0xa
+	PERF_COUNT_SW_MAX                     = 0xb
+	PERF_SAMPLE_IP                        = 0x1
+	PERF_SAMPLE_TID                       = 0x2
+	PERF_SAMPLE_TIME                      = 0x4
+	PERF_SAMPLE_ADDR                      = 0x8
+	PERF_SAMPLE_READ                      = 0x10
+	PERF_SAMPLE_CALLCHAIN                 = 0x20
+	PERF_SAMPLE_ID                        = 0x40
+	PERF_SAMPLE_CPU                       = 0x80
+	PERF_SAMPLE_PERIOD                    = 0x100
+	PERF_SAMPLE_STREAM_ID                 = 0x200
+	PERF_SAMPLE_RAW                       = 0x400
+	PERF_SAMPLE_BRANCH_STACK              = 0x800
+	PERF_SAMPLE_REGS_USER                 = 0x1000
+	PERF_SAMPLE_STACK_USER                = 0x2000
+	PERF_SAMPLE_WEIGHT                    = 0x4000
+	PERF_SAMPLE_DATA_SRC                  = 0x8000
+	PERF_SAMPLE_IDENTIFIER                = 0x10000
+	PERF_SAMPLE_TRANSACTION               = 0x20000
+	PERF_SAMPLE_REGS_INTR                 = 0x40000
+	PERF_SAMPLE_PHYS_ADDR                 = 0x80000
+	PERF_SAMPLE_AUX                       = 0x100000
+	PERF_SAMPLE_CGROUP                    = 0x200000
+	PERF_SAMPLE_DATA_PAGE_SIZE            = 0x400000
+	PERF_SAMPLE_CODE_PAGE_SIZE            = 0x800000
+	PERF_SAMPLE_WEIGHT_STRUCT             = 0x1000000
+	PERF_SAMPLE_MAX                       = 0x2000000
+	PERF_SAMPLE_BRANCH_USER_SHIFT         = 0x0
+	PERF_SAMPLE_BRANCH_KERNEL_SHIFT       = 0x1
+	PERF_SAMPLE_BRANCH_HV_SHIFT           = 0x2
+	PERF_SAMPLE_BRANCH_ANY_SHIFT          = 0x3
+	PERF_SAMPLE_BRANCH_ANY_CALL_SHIFT     = 0x4
+	PERF_SAMPLE_BRANCH_ANY_RETURN_SHIFT   = 0x5
+	PERF_SAMPLE_BRANCH_IND_CALL_SHIFT     = 0x6
+	PERF_SAMPLE_BRANCH_ABORT_TX_SHIFT     = 0x7
+	PERF_SAMPLE_BRANCH_IN_TX_SHIFT        = 0x8
+	PERF_SAMPLE_BRANCH_NO_TX_SHIFT        = 0x9
+	PERF_SAMPLE_BRANCH_COND_SHIFT         = 0xa
+	PERF_SAMPLE_BRANCH_CALL_STACK_SHIFT   = 0xb
+	PERF_SAMPLE_BRANCH_IND_JUMP_SHIFT     = 0xc
+	PERF_SAMPLE_BRANCH_CALL_SHIFT         = 0xd
+	PERF_SAMPLE_BRANCH_NO_FLAGS_SHIFT     = 0xe
+	PERF_SAMPLE_BRANCH_NO_CYCLES_SHIFT    = 0xf
+	PERF_SAMPLE_BRANCH_TYPE_SAVE_SHIFT    = 0x10
+	PERF_SAMPLE_BRANCH_HW_INDEX_SHIFT     = 0x11
+	PERF_SAMPLE_BRANCH_MAX_SHIFT          = 0x12
+	PERF_SAMPLE_BRANCH_USER               = 0x1
+	PERF_SAMPLE_BRANCH_KERNEL             = 0x2
+	PERF_SAMPLE_BRANCH_HV                 = 0x4
+	PERF_SAMPLE_BRANCH_ANY                = 0x8
+	PERF_SAMPLE_BRANCH_ANY_CALL           = 0x10
+	PERF_SAMPLE_BRANCH_ANY_RETURN         = 0x20
+	PERF_SAMPLE_BRANCH_IND_CALL           = 0x40
+	PERF_SAMPLE_BRANCH_ABORT_TX           = 0x80
+	PERF_SAMPLE_BRANCH_IN_TX              = 0x100
+	PERF_SAMPLE_BRANCH_NO_TX              = 0x200
+	PERF_SAMPLE_BRANCH_COND               = 0x400
+	PERF_SAMPLE_BRANCH_CALL_STACK         = 0x800
+	PERF_SAMPLE_BRANCH_IND_JUMP           = 0x1000
+	PERF_SAMPLE_BRANCH_CALL               = 0x2000
+	PERF_SAMPLE_BRANCH_NO_FLAGS           = 0x4000
+	PERF_SAMPLE_BRANCH_NO_CYCLES          = 0x8000
+	PERF_SAMPLE_BRANCH_TYPE_SAVE          = 0x10000
+	PERF_SAMPLE_BRANCH_HW_INDEX           = 0x20000
+	PERF_SAMPLE_BRANCH_MAX                = 0x40000
+	PERF_BR_UNKNOWN                       = 0x0
+	PERF_BR_COND                          = 0x1
+	PERF_BR_UNCOND                        = 0x2
+	PERF_BR_IND                           = 0x3
+	PERF_BR_CALL                          = 0x4
+	PERF_BR_IND_CALL                      = 0x5
+	PERF_BR_RET                           = 0x6
+	PERF_BR_SYSCALL                       = 0x7
+	PERF_BR_SYSRET                        = 0x8
+	PERF_BR_COND_CALL                     = 0x9
+	PERF_BR_COND_RET                      = 0xa
+	PERF_BR_MAX                           = 0xb
+	PERF_SAMPLE_REGS_ABI_NONE             = 0x0
+	PERF_SAMPLE_REGS_ABI_32               = 0x1
+	PERF_SAMPLE_REGS_ABI_64               = 0x2
+	PERF_TXN_ELISION                      = 0x1
+	PERF_TXN_TRANSACTION                  = 0x2
+	PERF_TXN_SYNC                         = 0x4
+	PERF_TXN_ASYNC                        = 0x8
+	PERF_TXN_RETRY                        = 0x10
+	PERF_TXN_CONFLICT                     = 0x20
+	PERF_TXN_CAPACITY_WRITE               = 0x40
+	PERF_TXN_CAPACITY_READ                = 0x80
+	PERF_TXN_MAX                          = 0x100
+	PERF_TXN_ABORT_MASK                   = -0x100000000
+	PERF_TXN_ABORT_SHIFT                  = 0x20
+	PERF_FORMAT_TOTAL_TIME_ENABLED        = 0x1
+	PERF_FORMAT_TOTAL_TIME_RUNNING        = 0x2
+	PERF_FORMAT_ID                        = 0x4
+	PERF_FORMAT_GROUP                     = 0x8
+	PERF_FORMAT_MAX                       = 0x10
+	PERF_IOC_FLAG_GROUP                   = 0x1
+	PERF_RECORD_MMAP                      = 0x1
+	PERF_RECORD_LOST                      = 0x2
+	PERF_RECORD_COMM                      = 0x3
+	PERF_RECORD_EXIT                      = 0x4
+	PERF_RECORD_THROTTLE                  = 0x5
+	PERF_RECORD_UNTHROTTLE                = 0x6
+	PERF_RECORD_FORK                      = 0x7
+	PERF_RECORD_READ                      = 0x8
+	PERF_RECORD_SAMPLE                    = 0x9
+	PERF_RECORD_MMAP2                     = 0xa
+	PERF_RECORD_AUX                       = 0xb
+	PERF_RECORD_ITRACE_START              = 0xc
+	PERF_RECORD_LOST_SAMPLES              = 0xd
+	PERF_RECORD_SWITCH                    = 0xe
+	PERF_RECORD_SWITCH_CPU_WIDE           = 0xf
+	PERF_RECORD_NAMESPACES                = 0x10
+	PERF_RECORD_KSYMBOL                   = 0x11
+	PERF_RECORD_BPF_EVENT                 = 0x12
+	PERF_RECORD_CGROUP                    = 0x13
+	PERF_RECORD_TEXT_POKE                 = 0x14
+	PERF_RECORD_MAX                       = 0x15
+	PERF_RECORD_KSYMBOL_TYPE_UNKNOWN      = 0x0
+	PERF_RECORD_KSYMBOL_TYPE_BPF          = 0x1
+	PERF_RECORD_KSYMBOL_TYPE_OOL          = 0x2
+	PERF_RECORD_KSYMBOL_TYPE_MAX          = 0x3
+	PERF_BPF_EVENT_UNKNOWN                = 0x0
+	PERF_BPF_EVENT_PROG_LOAD              = 0x1
+	PERF_BPF_EVENT_PROG_UNLOAD            = 0x2
+	PERF_BPF_EVENT_MAX                    = 0x3
+	PERF_CONTEXT_HV                       = -0x20
+	PERF_CONTEXT_KERNEL                   = -0x80
+	PERF_CONTEXT_USER                     = -0x200
+	PERF_CONTEXT_GUEST                    = -0x800
+	PERF_CONTEXT_GUEST_KERNEL             = -0x880
+	PERF_CONTEXT_GUEST_USER               = -0xa00
+	PERF_CONTEXT_MAX                      = -0xfff
 )
 
 type TCPMD5Sig struct {
@@ -1326,6 +1353,401 @@
 )
 
 const (
+	IFLA_UNSPEC                                = 0x0
+	IFLA_ADDRESS                               = 0x1
+	IFLA_BROADCAST                             = 0x2
+	IFLA_IFNAME                                = 0x3
+	IFLA_MTU                                   = 0x4
+	IFLA_LINK                                  = 0x5
+	IFLA_QDISC                                 = 0x6
+	IFLA_STATS                                 = 0x7
+	IFLA_COST                                  = 0x8
+	IFLA_PRIORITY                              = 0x9
+	IFLA_MASTER                                = 0xa
+	IFLA_WIRELESS                              = 0xb
+	IFLA_PROTINFO                              = 0xc
+	IFLA_TXQLEN                                = 0xd
+	IFLA_MAP                                   = 0xe
+	IFLA_WEIGHT                                = 0xf
+	IFLA_OPERSTATE                             = 0x10
+	IFLA_LINKMODE                              = 0x11
+	IFLA_LINKINFO                              = 0x12
+	IFLA_NET_NS_PID                            = 0x13
+	IFLA_IFALIAS                               = 0x14
+	IFLA_NUM_VF                                = 0x15
+	IFLA_VFINFO_LIST                           = 0x16
+	IFLA_STATS64                               = 0x17
+	IFLA_VF_PORTS                              = 0x18
+	IFLA_PORT_SELF                             = 0x19
+	IFLA_AF_SPEC                               = 0x1a
+	IFLA_GROUP                                 = 0x1b
+	IFLA_NET_NS_FD                             = 0x1c
+	IFLA_EXT_MASK                              = 0x1d
+	IFLA_PROMISCUITY                           = 0x1e
+	IFLA_NUM_TX_QUEUES                         = 0x1f
+	IFLA_NUM_RX_QUEUES                         = 0x20
+	IFLA_CARRIER                               = 0x21
+	IFLA_PHYS_PORT_ID                          = 0x22
+	IFLA_CARRIER_CHANGES                       = 0x23
+	IFLA_PHYS_SWITCH_ID                        = 0x24
+	IFLA_LINK_NETNSID                          = 0x25
+	IFLA_PHYS_PORT_NAME                        = 0x26
+	IFLA_PROTO_DOWN                            = 0x27
+	IFLA_GSO_MAX_SEGS                          = 0x28
+	IFLA_GSO_MAX_SIZE                          = 0x29
+	IFLA_PAD                                   = 0x2a
+	IFLA_XDP                                   = 0x2b
+	IFLA_EVENT                                 = 0x2c
+	IFLA_NEW_NETNSID                           = 0x2d
+	IFLA_IF_NETNSID                            = 0x2e
+	IFLA_TARGET_NETNSID                        = 0x2e
+	IFLA_CARRIER_UP_COUNT                      = 0x2f
+	IFLA_CARRIER_DOWN_COUNT                    = 0x30
+	IFLA_NEW_IFINDEX                           = 0x31
+	IFLA_MIN_MTU                               = 0x32
+	IFLA_MAX_MTU                               = 0x33
+	IFLA_PROP_LIST                             = 0x34
+	IFLA_ALT_IFNAME                            = 0x35
+	IFLA_PERM_ADDRESS                          = 0x36
+	IFLA_PROTO_DOWN_REASON                     = 0x37
+	IFLA_PROTO_DOWN_REASON_UNSPEC              = 0x0
+	IFLA_PROTO_DOWN_REASON_MASK                = 0x1
+	IFLA_PROTO_DOWN_REASON_VALUE               = 0x2
+	IFLA_PROTO_DOWN_REASON_MAX                 = 0x2
+	IFLA_INET_UNSPEC                           = 0x0
+	IFLA_INET_CONF                             = 0x1
+	IFLA_INET6_UNSPEC                          = 0x0
+	IFLA_INET6_FLAGS                           = 0x1
+	IFLA_INET6_CONF                            = 0x2
+	IFLA_INET6_STATS                           = 0x3
+	IFLA_INET6_MCAST                           = 0x4
+	IFLA_INET6_CACHEINFO                       = 0x5
+	IFLA_INET6_ICMP6STATS                      = 0x6
+	IFLA_INET6_TOKEN                           = 0x7
+	IFLA_INET6_ADDR_GEN_MODE                   = 0x8
+	IFLA_BR_UNSPEC                             = 0x0
+	IFLA_BR_FORWARD_DELAY                      = 0x1
+	IFLA_BR_HELLO_TIME                         = 0x2
+	IFLA_BR_MAX_AGE                            = 0x3
+	IFLA_BR_AGEING_TIME                        = 0x4
+	IFLA_BR_STP_STATE                          = 0x5
+	IFLA_BR_PRIORITY                           = 0x6
+	IFLA_BR_VLAN_FILTERING                     = 0x7
+	IFLA_BR_VLAN_PROTOCOL                      = 0x8
+	IFLA_BR_GROUP_FWD_MASK                     = 0x9
+	IFLA_BR_ROOT_ID                            = 0xa
+	IFLA_BR_BRIDGE_ID                          = 0xb
+	IFLA_BR_ROOT_PORT                          = 0xc
+	IFLA_BR_ROOT_PATH_COST                     = 0xd
+	IFLA_BR_TOPOLOGY_CHANGE                    = 0xe
+	IFLA_BR_TOPOLOGY_CHANGE_DETECTED           = 0xf
+	IFLA_BR_HELLO_TIMER                        = 0x10
+	IFLA_BR_TCN_TIMER                          = 0x11
+	IFLA_BR_TOPOLOGY_CHANGE_TIMER              = 0x12
+	IFLA_BR_GC_TIMER                           = 0x13
+	IFLA_BR_GROUP_ADDR                         = 0x14
+	IFLA_BR_FDB_FLUSH                          = 0x15
+	IFLA_BR_MCAST_ROUTER                       = 0x16
+	IFLA_BR_MCAST_SNOOPING                     = 0x17
+	IFLA_BR_MCAST_QUERY_USE_IFADDR             = 0x18
+	IFLA_BR_MCAST_QUERIER                      = 0x19
+	IFLA_BR_MCAST_HASH_ELASTICITY              = 0x1a
+	IFLA_BR_MCAST_HASH_MAX                     = 0x1b
+	IFLA_BR_MCAST_LAST_MEMBER_CNT              = 0x1c
+	IFLA_BR_MCAST_STARTUP_QUERY_CNT            = 0x1d
+	IFLA_BR_MCAST_LAST_MEMBER_INTVL            = 0x1e
+	IFLA_BR_MCAST_MEMBERSHIP_INTVL             = 0x1f
+	IFLA_BR_MCAST_QUERIER_INTVL                = 0x20
+	IFLA_BR_MCAST_QUERY_INTVL                  = 0x21
+	IFLA_BR_MCAST_QUERY_RESPONSE_INTVL         = 0x22
+	IFLA_BR_MCAST_STARTUP_QUERY_INTVL          = 0x23
+	IFLA_BR_NF_CALL_IPTABLES                   = 0x24
+	IFLA_BR_NF_CALL_IP6TABLES                  = 0x25
+	IFLA_BR_NF_CALL_ARPTABLES                  = 0x26
+	IFLA_BR_VLAN_DEFAULT_PVID                  = 0x27
+	IFLA_BR_PAD                                = 0x28
+	IFLA_BR_VLAN_STATS_ENABLED                 = 0x29
+	IFLA_BR_MCAST_STATS_ENABLED                = 0x2a
+	IFLA_BR_MCAST_IGMP_VERSION                 = 0x2b
+	IFLA_BR_MCAST_MLD_VERSION                  = 0x2c
+	IFLA_BR_VLAN_STATS_PER_PORT                = 0x2d
+	IFLA_BR_MULTI_BOOLOPT                      = 0x2e
+	IFLA_BRPORT_UNSPEC                         = 0x0
+	IFLA_BRPORT_STATE                          = 0x1
+	IFLA_BRPORT_PRIORITY                       = 0x2
+	IFLA_BRPORT_COST                           = 0x3
+	IFLA_BRPORT_MODE                           = 0x4
+	IFLA_BRPORT_GUARD                          = 0x5
+	IFLA_BRPORT_PROTECT                        = 0x6
+	IFLA_BRPORT_FAST_LEAVE                     = 0x7
+	IFLA_BRPORT_LEARNING                       = 0x8
+	IFLA_BRPORT_UNICAST_FLOOD                  = 0x9
+	IFLA_BRPORT_PROXYARP                       = 0xa
+	IFLA_BRPORT_LEARNING_SYNC                  = 0xb
+	IFLA_BRPORT_PROXYARP_WIFI                  = 0xc
+	IFLA_BRPORT_ROOT_ID                        = 0xd
+	IFLA_BRPORT_BRIDGE_ID                      = 0xe
+	IFLA_BRPORT_DESIGNATED_PORT                = 0xf
+	IFLA_BRPORT_DESIGNATED_COST                = 0x10
+	IFLA_BRPORT_ID                             = 0x11
+	IFLA_BRPORT_NO                             = 0x12
+	IFLA_BRPORT_TOPOLOGY_CHANGE_ACK            = 0x13
+	IFLA_BRPORT_CONFIG_PENDING                 = 0x14
+	IFLA_BRPORT_MESSAGE_AGE_TIMER              = 0x15
+	IFLA_BRPORT_FORWARD_DELAY_TIMER            = 0x16
+	IFLA_BRPORT_HOLD_TIMER                     = 0x17
+	IFLA_BRPORT_FLUSH                          = 0x18
+	IFLA_BRPORT_MULTICAST_ROUTER               = 0x19
+	IFLA_BRPORT_PAD                            = 0x1a
+	IFLA_BRPORT_MCAST_FLOOD                    = 0x1b
+	IFLA_BRPORT_MCAST_TO_UCAST                 = 0x1c
+	IFLA_BRPORT_VLAN_TUNNEL                    = 0x1d
+	IFLA_BRPORT_BCAST_FLOOD                    = 0x1e
+	IFLA_BRPORT_GROUP_FWD_MASK                 = 0x1f
+	IFLA_BRPORT_NEIGH_SUPPRESS                 = 0x20
+	IFLA_BRPORT_ISOLATED                       = 0x21
+	IFLA_BRPORT_BACKUP_PORT                    = 0x22
+	IFLA_BRPORT_MRP_RING_OPEN                  = 0x23
+	IFLA_BRPORT_MRP_IN_OPEN                    = 0x24
+	IFLA_INFO_UNSPEC                           = 0x0
+	IFLA_INFO_KIND                             = 0x1
+	IFLA_INFO_DATA                             = 0x2
+	IFLA_INFO_XSTATS                           = 0x3
+	IFLA_INFO_SLAVE_KIND                       = 0x4
+	IFLA_INFO_SLAVE_DATA                       = 0x5
+	IFLA_VLAN_UNSPEC                           = 0x0
+	IFLA_VLAN_ID                               = 0x1
+	IFLA_VLAN_FLAGS                            = 0x2
+	IFLA_VLAN_EGRESS_QOS                       = 0x3
+	IFLA_VLAN_INGRESS_QOS                      = 0x4
+	IFLA_VLAN_PROTOCOL                         = 0x5
+	IFLA_VLAN_QOS_UNSPEC                       = 0x0
+	IFLA_VLAN_QOS_MAPPING                      = 0x1
+	IFLA_MACVLAN_UNSPEC                        = 0x0
+	IFLA_MACVLAN_MODE                          = 0x1
+	IFLA_MACVLAN_FLAGS                         = 0x2
+	IFLA_MACVLAN_MACADDR_MODE                  = 0x3
+	IFLA_MACVLAN_MACADDR                       = 0x4
+	IFLA_MACVLAN_MACADDR_DATA                  = 0x5
+	IFLA_MACVLAN_MACADDR_COUNT                 = 0x6
+	IFLA_VRF_UNSPEC                            = 0x0
+	IFLA_VRF_TABLE                             = 0x1
+	IFLA_VRF_PORT_UNSPEC                       = 0x0
+	IFLA_VRF_PORT_TABLE                        = 0x1
+	IFLA_MACSEC_UNSPEC                         = 0x0
+	IFLA_MACSEC_SCI                            = 0x1
+	IFLA_MACSEC_PORT                           = 0x2
+	IFLA_MACSEC_ICV_LEN                        = 0x3
+	IFLA_MACSEC_CIPHER_SUITE                   = 0x4
+	IFLA_MACSEC_WINDOW                         = 0x5
+	IFLA_MACSEC_ENCODING_SA                    = 0x6
+	IFLA_MACSEC_ENCRYPT                        = 0x7
+	IFLA_MACSEC_PROTECT                        = 0x8
+	IFLA_MACSEC_INC_SCI                        = 0x9
+	IFLA_MACSEC_ES                             = 0xa
+	IFLA_MACSEC_SCB                            = 0xb
+	IFLA_MACSEC_REPLAY_PROTECT                 = 0xc
+	IFLA_MACSEC_VALIDATION                     = 0xd
+	IFLA_MACSEC_PAD                            = 0xe
+	IFLA_MACSEC_OFFLOAD                        = 0xf
+	IFLA_XFRM_UNSPEC                           = 0x0
+	IFLA_XFRM_LINK                             = 0x1
+	IFLA_XFRM_IF_ID                            = 0x2
+	IFLA_IPVLAN_UNSPEC                         = 0x0
+	IFLA_IPVLAN_MODE                           = 0x1
+	IFLA_IPVLAN_FLAGS                          = 0x2
+	IFLA_VXLAN_UNSPEC                          = 0x0
+	IFLA_VXLAN_ID                              = 0x1
+	IFLA_VXLAN_GROUP                           = 0x2
+	IFLA_VXLAN_LINK                            = 0x3
+	IFLA_VXLAN_LOCAL                           = 0x4
+	IFLA_VXLAN_TTL                             = 0x5
+	IFLA_VXLAN_TOS                             = 0x6
+	IFLA_VXLAN_LEARNING                        = 0x7
+	IFLA_VXLAN_AGEING                          = 0x8
+	IFLA_VXLAN_LIMIT                           = 0x9
+	IFLA_VXLAN_PORT_RANGE                      = 0xa
+	IFLA_VXLAN_PROXY                           = 0xb
+	IFLA_VXLAN_RSC                             = 0xc
+	IFLA_VXLAN_L2MISS                          = 0xd
+	IFLA_VXLAN_L3MISS                          = 0xe
+	IFLA_VXLAN_PORT                            = 0xf
+	IFLA_VXLAN_GROUP6                          = 0x10
+	IFLA_VXLAN_LOCAL6                          = 0x11
+	IFLA_VXLAN_UDP_CSUM                        = 0x12
+	IFLA_VXLAN_UDP_ZERO_CSUM6_TX               = 0x13
+	IFLA_VXLAN_UDP_ZERO_CSUM6_RX               = 0x14
+	IFLA_VXLAN_REMCSUM_TX                      = 0x15
+	IFLA_VXLAN_REMCSUM_RX                      = 0x16
+	IFLA_VXLAN_GBP                             = 0x17
+	IFLA_VXLAN_REMCSUM_NOPARTIAL               = 0x18
+	IFLA_VXLAN_COLLECT_METADATA                = 0x19
+	IFLA_VXLAN_LABEL                           = 0x1a
+	IFLA_VXLAN_GPE                             = 0x1b
+	IFLA_VXLAN_TTL_INHERIT                     = 0x1c
+	IFLA_VXLAN_DF                              = 0x1d
+	IFLA_GENEVE_UNSPEC                         = 0x0
+	IFLA_GENEVE_ID                             = 0x1
+	IFLA_GENEVE_REMOTE                         = 0x2
+	IFLA_GENEVE_TTL                            = 0x3
+	IFLA_GENEVE_TOS                            = 0x4
+	IFLA_GENEVE_PORT                           = 0x5
+	IFLA_GENEVE_COLLECT_METADATA               = 0x6
+	IFLA_GENEVE_REMOTE6                        = 0x7
+	IFLA_GENEVE_UDP_CSUM                       = 0x8
+	IFLA_GENEVE_UDP_ZERO_CSUM6_TX              = 0x9
+	IFLA_GENEVE_UDP_ZERO_CSUM6_RX              = 0xa
+	IFLA_GENEVE_LABEL                          = 0xb
+	IFLA_GENEVE_TTL_INHERIT                    = 0xc
+	IFLA_GENEVE_DF                             = 0xd
+	IFLA_BAREUDP_UNSPEC                        = 0x0
+	IFLA_BAREUDP_PORT                          = 0x1
+	IFLA_BAREUDP_ETHERTYPE                     = 0x2
+	IFLA_BAREUDP_SRCPORT_MIN                   = 0x3
+	IFLA_BAREUDP_MULTIPROTO_MODE               = 0x4
+	IFLA_PPP_UNSPEC                            = 0x0
+	IFLA_PPP_DEV_FD                            = 0x1
+	IFLA_GTP_UNSPEC                            = 0x0
+	IFLA_GTP_FD0                               = 0x1
+	IFLA_GTP_FD1                               = 0x2
+	IFLA_GTP_PDP_HASHSIZE                      = 0x3
+	IFLA_GTP_ROLE                              = 0x4
+	IFLA_BOND_UNSPEC                           = 0x0
+	IFLA_BOND_MODE                             = 0x1
+	IFLA_BOND_ACTIVE_SLAVE                     = 0x2
+	IFLA_BOND_MIIMON                           = 0x3
+	IFLA_BOND_UPDELAY                          = 0x4
+	IFLA_BOND_DOWNDELAY                        = 0x5
+	IFLA_BOND_USE_CARRIER                      = 0x6
+	IFLA_BOND_ARP_INTERVAL                     = 0x7
+	IFLA_BOND_ARP_IP_TARGET                    = 0x8
+	IFLA_BOND_ARP_VALIDATE                     = 0x9
+	IFLA_BOND_ARP_ALL_TARGETS                  = 0xa
+	IFLA_BOND_PRIMARY                          = 0xb
+	IFLA_BOND_PRIMARY_RESELECT                 = 0xc
+	IFLA_BOND_FAIL_OVER_MAC                    = 0xd
+	IFLA_BOND_XMIT_HASH_POLICY                 = 0xe
+	IFLA_BOND_RESEND_IGMP                      = 0xf
+	IFLA_BOND_NUM_PEER_NOTIF                   = 0x10
+	IFLA_BOND_ALL_SLAVES_ACTIVE                = 0x11
+	IFLA_BOND_MIN_LINKS                        = 0x12
+	IFLA_BOND_LP_INTERVAL                      = 0x13
+	IFLA_BOND_PACKETS_PER_SLAVE                = 0x14
+	IFLA_BOND_AD_LACP_RATE                     = 0x15
+	IFLA_BOND_AD_SELECT                        = 0x16
+	IFLA_BOND_AD_INFO                          = 0x17
+	IFLA_BOND_AD_ACTOR_SYS_PRIO                = 0x18
+	IFLA_BOND_AD_USER_PORT_KEY                 = 0x19
+	IFLA_BOND_AD_ACTOR_SYSTEM                  = 0x1a
+	IFLA_BOND_TLB_DYNAMIC_LB                   = 0x1b
+	IFLA_BOND_PEER_NOTIF_DELAY                 = 0x1c
+	IFLA_BOND_AD_INFO_UNSPEC                   = 0x0
+	IFLA_BOND_AD_INFO_AGGREGATOR               = 0x1
+	IFLA_BOND_AD_INFO_NUM_PORTS                = 0x2
+	IFLA_BOND_AD_INFO_ACTOR_KEY                = 0x3
+	IFLA_BOND_AD_INFO_PARTNER_KEY              = 0x4
+	IFLA_BOND_AD_INFO_PARTNER_MAC              = 0x5
+	IFLA_BOND_SLAVE_UNSPEC                     = 0x0
+	IFLA_BOND_SLAVE_STATE                      = 0x1
+	IFLA_BOND_SLAVE_MII_STATUS                 = 0x2
+	IFLA_BOND_SLAVE_LINK_FAILURE_COUNT         = 0x3
+	IFLA_BOND_SLAVE_PERM_HWADDR                = 0x4
+	IFLA_BOND_SLAVE_QUEUE_ID                   = 0x5
+	IFLA_BOND_SLAVE_AD_AGGREGATOR_ID           = 0x6
+	IFLA_BOND_SLAVE_AD_ACTOR_OPER_PORT_STATE   = 0x7
+	IFLA_BOND_SLAVE_AD_PARTNER_OPER_PORT_STATE = 0x8
+	IFLA_VF_INFO_UNSPEC                        = 0x0
+	IFLA_VF_INFO                               = 0x1
+	IFLA_VF_UNSPEC                             = 0x0
+	IFLA_VF_MAC                                = 0x1
+	IFLA_VF_VLAN                               = 0x2
+	IFLA_VF_TX_RATE                            = 0x3
+	IFLA_VF_SPOOFCHK                           = 0x4
+	IFLA_VF_LINK_STATE                         = 0x5
+	IFLA_VF_RATE                               = 0x6
+	IFLA_VF_RSS_QUERY_EN                       = 0x7
+	IFLA_VF_STATS                              = 0x8
+	IFLA_VF_TRUST                              = 0x9
+	IFLA_VF_IB_NODE_GUID                       = 0xa
+	IFLA_VF_IB_PORT_GUID                       = 0xb
+	IFLA_VF_VLAN_LIST                          = 0xc
+	IFLA_VF_BROADCAST                          = 0xd
+	IFLA_VF_VLAN_INFO_UNSPEC                   = 0x0
+	IFLA_VF_VLAN_INFO                          = 0x1
+	IFLA_VF_LINK_STATE_AUTO                    = 0x0
+	IFLA_VF_LINK_STATE_ENABLE                  = 0x1
+	IFLA_VF_LINK_STATE_DISABLE                 = 0x2
+	IFLA_VF_STATS_RX_PACKETS                   = 0x0
+	IFLA_VF_STATS_TX_PACKETS                   = 0x1
+	IFLA_VF_STATS_RX_BYTES                     = 0x2
+	IFLA_VF_STATS_TX_BYTES                     = 0x3
+	IFLA_VF_STATS_BROADCAST                    = 0x4
+	IFLA_VF_STATS_MULTICAST                    = 0x5
+	IFLA_VF_STATS_PAD                          = 0x6
+	IFLA_VF_STATS_RX_DROPPED                   = 0x7
+	IFLA_VF_STATS_TX_DROPPED                   = 0x8
+	IFLA_VF_PORT_UNSPEC                        = 0x0
+	IFLA_VF_PORT                               = 0x1
+	IFLA_PORT_UNSPEC                           = 0x0
+	IFLA_PORT_VF                               = 0x1
+	IFLA_PORT_PROFILE                          = 0x2
+	IFLA_PORT_VSI_TYPE                         = 0x3
+	IFLA_PORT_INSTANCE_UUID                    = 0x4
+	IFLA_PORT_HOST_UUID                        = 0x5
+	IFLA_PORT_REQUEST                          = 0x6
+	IFLA_PORT_RESPONSE                         = 0x7
+	IFLA_IPOIB_UNSPEC                          = 0x0
+	IFLA_IPOIB_PKEY                            = 0x1
+	IFLA_IPOIB_MODE                            = 0x2
+	IFLA_IPOIB_UMCAST                          = 0x3
+	IFLA_HSR_UNSPEC                            = 0x0
+	IFLA_HSR_SLAVE1                            = 0x1
+	IFLA_HSR_SLAVE2                            = 0x2
+	IFLA_HSR_MULTICAST_SPEC                    = 0x3
+	IFLA_HSR_SUPERVISION_ADDR                  = 0x4
+	IFLA_HSR_SEQ_NR                            = 0x5
+	IFLA_HSR_VERSION                           = 0x6
+	IFLA_HSR_PROTOCOL                          = 0x7
+	IFLA_STATS_UNSPEC                          = 0x0
+	IFLA_STATS_LINK_64                         = 0x1
+	IFLA_STATS_LINK_XSTATS                     = 0x2
+	IFLA_STATS_LINK_XSTATS_SLAVE               = 0x3
+	IFLA_STATS_LINK_OFFLOAD_XSTATS             = 0x4
+	IFLA_STATS_AF_SPEC                         = 0x5
+	IFLA_OFFLOAD_XSTATS_UNSPEC                 = 0x0
+	IFLA_OFFLOAD_XSTATS_CPU_HIT                = 0x1
+	IFLA_XDP_UNSPEC                            = 0x0
+	IFLA_XDP_FD                                = 0x1
+	IFLA_XDP_ATTACHED                          = 0x2
+	IFLA_XDP_FLAGS                             = 0x3
+	IFLA_XDP_PROG_ID                           = 0x4
+	IFLA_XDP_DRV_PROG_ID                       = 0x5
+	IFLA_XDP_SKB_PROG_ID                       = 0x6
+	IFLA_XDP_HW_PROG_ID                        = 0x7
+	IFLA_XDP_EXPECTED_FD                       = 0x8
+	IFLA_EVENT_NONE                            = 0x0
+	IFLA_EVENT_REBOOT                          = 0x1
+	IFLA_EVENT_FEATURES                        = 0x2
+	IFLA_EVENT_BONDING_FAILOVER                = 0x3
+	IFLA_EVENT_NOTIFY_PEERS                    = 0x4
+	IFLA_EVENT_IGMP_RESEND                     = 0x5
+	IFLA_EVENT_BONDING_OPTIONS                 = 0x6
+	IFLA_TUN_UNSPEC                            = 0x0
+	IFLA_TUN_OWNER                             = 0x1
+	IFLA_TUN_GROUP                             = 0x2
+	IFLA_TUN_TYPE                              = 0x3
+	IFLA_TUN_PI                                = 0x4
+	IFLA_TUN_VNET_HDR                          = 0x5
+	IFLA_TUN_PERSIST                           = 0x6
+	IFLA_TUN_MULTI_QUEUE                       = 0x7
+	IFLA_TUN_NUM_QUEUES                        = 0x8
+	IFLA_TUN_NUM_DISABLED_QUEUES               = 0x9
+	IFLA_RMNET_UNSPEC                          = 0x0
+	IFLA_RMNET_MUX_ID                          = 0x1
+	IFLA_RMNET_FLAGS                           = 0x2
+)
+
+const (
 	NF_INET_PRE_ROUTING  = 0x0
 	NF_INET_LOCAL_IN     = 0x1
 	NF_INET_FORWARD      = 0x2
@@ -1412,7 +1834,7 @@
 	NFT_MSG_DELOBJ                    = 0x14
 	NFT_MSG_GETOBJ_RESET              = 0x15
 	NFT_MSG_MAX                       = 0x19
-	NFTA_LIST_UNPEC                   = 0x0
+	NFTA_LIST_UNSPEC                  = 0x0
 	NFTA_LIST_ELEM                    = 0x1
 	NFTA_HOOK_UNSPEC                  = 0x0
 	NFTA_HOOK_HOOKNUM                 = 0x1
@@ -1830,10 +2252,12 @@
 )
 
 const (
-	NETNSA_NONE = 0x0
-	NETNSA_NSID = 0x1
-	NETNSA_PID  = 0x2
-	NETNSA_FD   = 0x3
+	NETNSA_NONE         = 0x0
+	NETNSA_NSID         = 0x1
+	NETNSA_PID          = 0x2
+	NETNSA_FD           = 0x3
+	NETNSA_TARGET_NSID  = 0x4
+	NETNSA_CURRENT_NSID = 0x5
 )
 
 type XDPRingOffset struct {
@@ -1851,9 +2275,12 @@
 }
 
 type XDPStatistics struct {
-	Rx_dropped       uint64
-	Rx_invalid_descs uint64
-	Tx_invalid_descs uint64
+	Rx_dropped               uint64
+	Rx_invalid_descs         uint64
+	Tx_invalid_descs         uint64
+	Rx_ring_full             uint64
+	Rx_fill_ring_empty_descs uint64
+	Tx_ring_empty_descs      uint64
 }
 
 type XDPDesc struct {
@@ -1980,281 +2407,309 @@
 )
 
 const (
-	BPF_REG_0                               = 0x0
-	BPF_REG_1                               = 0x1
-	BPF_REG_2                               = 0x2
-	BPF_REG_3                               = 0x3
-	BPF_REG_4                               = 0x4
-	BPF_REG_5                               = 0x5
-	BPF_REG_6                               = 0x6
-	BPF_REG_7                               = 0x7
-	BPF_REG_8                               = 0x8
-	BPF_REG_9                               = 0x9
-	BPF_REG_10                              = 0xa
-	BPF_MAP_CREATE                          = 0x0
-	BPF_MAP_LOOKUP_ELEM                     = 0x1
-	BPF_MAP_UPDATE_ELEM                     = 0x2
-	BPF_MAP_DELETE_ELEM                     = 0x3
-	BPF_MAP_GET_NEXT_KEY                    = 0x4
-	BPF_PROG_LOAD                           = 0x5
-	BPF_OBJ_PIN                             = 0x6
-	BPF_OBJ_GET                             = 0x7
-	BPF_PROG_ATTACH                         = 0x8
-	BPF_PROG_DETACH                         = 0x9
-	BPF_PROG_TEST_RUN                       = 0xa
-	BPF_PROG_GET_NEXT_ID                    = 0xb
-	BPF_MAP_GET_NEXT_ID                     = 0xc
-	BPF_PROG_GET_FD_BY_ID                   = 0xd
-	BPF_MAP_GET_FD_BY_ID                    = 0xe
-	BPF_OBJ_GET_INFO_BY_FD                  = 0xf
-	BPF_PROG_QUERY                          = 0x10
-	BPF_RAW_TRACEPOINT_OPEN                 = 0x11
-	BPF_BTF_LOAD                            = 0x12
-	BPF_BTF_GET_FD_BY_ID                    = 0x13
-	BPF_TASK_FD_QUERY                       = 0x14
-	BPF_MAP_LOOKUP_AND_DELETE_ELEM          = 0x15
-	BPF_MAP_FREEZE                          = 0x16
-	BPF_BTF_GET_NEXT_ID                     = 0x17
-	BPF_MAP_LOOKUP_BATCH                    = 0x18
-	BPF_MAP_LOOKUP_AND_DELETE_BATCH         = 0x19
-	BPF_MAP_UPDATE_BATCH                    = 0x1a
-	BPF_MAP_DELETE_BATCH                    = 0x1b
-	BPF_LINK_CREATE                         = 0x1c
-	BPF_LINK_UPDATE                         = 0x1d
-	BPF_LINK_GET_FD_BY_ID                   = 0x1e
-	BPF_LINK_GET_NEXT_ID                    = 0x1f
-	BPF_ENABLE_STATS                        = 0x20
-	BPF_ITER_CREATE                         = 0x21
-	BPF_MAP_TYPE_UNSPEC                     = 0x0
-	BPF_MAP_TYPE_HASH                       = 0x1
-	BPF_MAP_TYPE_ARRAY                      = 0x2
-	BPF_MAP_TYPE_PROG_ARRAY                 = 0x3
-	BPF_MAP_TYPE_PERF_EVENT_ARRAY           = 0x4
-	BPF_MAP_TYPE_PERCPU_HASH                = 0x5
-	BPF_MAP_TYPE_PERCPU_ARRAY               = 0x6
-	BPF_MAP_TYPE_STACK_TRACE                = 0x7
-	BPF_MAP_TYPE_CGROUP_ARRAY               = 0x8
-	BPF_MAP_TYPE_LRU_HASH                   = 0x9
-	BPF_MAP_TYPE_LRU_PERCPU_HASH            = 0xa
-	BPF_MAP_TYPE_LPM_TRIE                   = 0xb
-	BPF_MAP_TYPE_ARRAY_OF_MAPS              = 0xc
-	BPF_MAP_TYPE_HASH_OF_MAPS               = 0xd
-	BPF_MAP_TYPE_DEVMAP                     = 0xe
-	BPF_MAP_TYPE_SOCKMAP                    = 0xf
-	BPF_MAP_TYPE_CPUMAP                     = 0x10
-	BPF_MAP_TYPE_XSKMAP                     = 0x11
-	BPF_MAP_TYPE_SOCKHASH                   = 0x12
-	BPF_MAP_TYPE_CGROUP_STORAGE             = 0x13
-	BPF_MAP_TYPE_REUSEPORT_SOCKARRAY        = 0x14
-	BPF_MAP_TYPE_PERCPU_CGROUP_STORAGE      = 0x15
-	BPF_MAP_TYPE_QUEUE                      = 0x16
-	BPF_MAP_TYPE_STACK                      = 0x17
-	BPF_MAP_TYPE_SK_STORAGE                 = 0x18
-	BPF_MAP_TYPE_DEVMAP_HASH                = 0x19
-	BPF_MAP_TYPE_STRUCT_OPS                 = 0x1a
-	BPF_MAP_TYPE_RINGBUF                    = 0x1b
-	BPF_PROG_TYPE_UNSPEC                    = 0x0
-	BPF_PROG_TYPE_SOCKET_FILTER             = 0x1
-	BPF_PROG_TYPE_KPROBE                    = 0x2
-	BPF_PROG_TYPE_SCHED_CLS                 = 0x3
-	BPF_PROG_TYPE_SCHED_ACT                 = 0x4
-	BPF_PROG_TYPE_TRACEPOINT                = 0x5
-	BPF_PROG_TYPE_XDP                       = 0x6
-	BPF_PROG_TYPE_PERF_EVENT                = 0x7
-	BPF_PROG_TYPE_CGROUP_SKB                = 0x8
-	BPF_PROG_TYPE_CGROUP_SOCK               = 0x9
-	BPF_PROG_TYPE_LWT_IN                    = 0xa
-	BPF_PROG_TYPE_LWT_OUT                   = 0xb
-	BPF_PROG_TYPE_LWT_XMIT                  = 0xc
-	BPF_PROG_TYPE_SOCK_OPS                  = 0xd
-	BPF_PROG_TYPE_SK_SKB                    = 0xe
-	BPF_PROG_TYPE_CGROUP_DEVICE             = 0xf
-	BPF_PROG_TYPE_SK_MSG                    = 0x10
-	BPF_PROG_TYPE_RAW_TRACEPOINT            = 0x11
-	BPF_PROG_TYPE_CGROUP_SOCK_ADDR          = 0x12
-	BPF_PROG_TYPE_LWT_SEG6LOCAL             = 0x13
-	BPF_PROG_TYPE_LIRC_MODE2                = 0x14
-	BPF_PROG_TYPE_SK_REUSEPORT              = 0x15
-	BPF_PROG_TYPE_FLOW_DISSECTOR            = 0x16
-	BPF_PROG_TYPE_CGROUP_SYSCTL             = 0x17
-	BPF_PROG_TYPE_RAW_TRACEPOINT_WRITABLE   = 0x18
-	BPF_PROG_TYPE_CGROUP_SOCKOPT            = 0x19
-	BPF_PROG_TYPE_TRACING                   = 0x1a
-	BPF_PROG_TYPE_STRUCT_OPS                = 0x1b
-	BPF_PROG_TYPE_EXT                       = 0x1c
-	BPF_PROG_TYPE_LSM                       = 0x1d
-	BPF_CGROUP_INET_INGRESS                 = 0x0
-	BPF_CGROUP_INET_EGRESS                  = 0x1
-	BPF_CGROUP_INET_SOCK_CREATE             = 0x2
-	BPF_CGROUP_SOCK_OPS                     = 0x3
-	BPF_SK_SKB_STREAM_PARSER                = 0x4
-	BPF_SK_SKB_STREAM_VERDICT               = 0x5
-	BPF_CGROUP_DEVICE                       = 0x6
-	BPF_SK_MSG_VERDICT                      = 0x7
-	BPF_CGROUP_INET4_BIND                   = 0x8
-	BPF_CGROUP_INET6_BIND                   = 0x9
-	BPF_CGROUP_INET4_CONNECT                = 0xa
-	BPF_CGROUP_INET6_CONNECT                = 0xb
-	BPF_CGROUP_INET4_POST_BIND              = 0xc
-	BPF_CGROUP_INET6_POST_BIND              = 0xd
-	BPF_CGROUP_UDP4_SENDMSG                 = 0xe
-	BPF_CGROUP_UDP6_SENDMSG                 = 0xf
-	BPF_LIRC_MODE2                          = 0x10
-	BPF_FLOW_DISSECTOR                      = 0x11
-	BPF_CGROUP_SYSCTL                       = 0x12
-	BPF_CGROUP_UDP4_RECVMSG                 = 0x13
-	BPF_CGROUP_UDP6_RECVMSG                 = 0x14
-	BPF_CGROUP_GETSOCKOPT                   = 0x15
-	BPF_CGROUP_SETSOCKOPT                   = 0x16
-	BPF_TRACE_RAW_TP                        = 0x17
-	BPF_TRACE_FENTRY                        = 0x18
-	BPF_TRACE_FEXIT                         = 0x19
-	BPF_MODIFY_RETURN                       = 0x1a
-	BPF_LSM_MAC                             = 0x1b
-	BPF_TRACE_ITER                          = 0x1c
-	BPF_CGROUP_INET4_GETPEERNAME            = 0x1d
-	BPF_CGROUP_INET6_GETPEERNAME            = 0x1e
-	BPF_CGROUP_INET4_GETSOCKNAME            = 0x1f
-	BPF_CGROUP_INET6_GETSOCKNAME            = 0x20
-	BPF_XDP_DEVMAP                          = 0x21
-	BPF_LINK_TYPE_UNSPEC                    = 0x0
-	BPF_LINK_TYPE_RAW_TRACEPOINT            = 0x1
-	BPF_LINK_TYPE_TRACING                   = 0x2
-	BPF_LINK_TYPE_CGROUP                    = 0x3
-	BPF_LINK_TYPE_ITER                      = 0x4
-	BPF_LINK_TYPE_NETNS                     = 0x5
-	BPF_ANY                                 = 0x0
-	BPF_NOEXIST                             = 0x1
-	BPF_EXIST                               = 0x2
-	BPF_F_LOCK                              = 0x4
-	BPF_F_NO_PREALLOC                       = 0x1
-	BPF_F_NO_COMMON_LRU                     = 0x2
-	BPF_F_NUMA_NODE                         = 0x4
-	BPF_F_RDONLY                            = 0x8
-	BPF_F_WRONLY                            = 0x10
-	BPF_F_STACK_BUILD_ID                    = 0x20
-	BPF_F_ZERO_SEED                         = 0x40
-	BPF_F_RDONLY_PROG                       = 0x80
-	BPF_F_WRONLY_PROG                       = 0x100
-	BPF_F_CLONE                             = 0x200
-	BPF_F_MMAPABLE                          = 0x400
-	BPF_STATS_RUN_TIME                      = 0x0
-	BPF_STACK_BUILD_ID_EMPTY                = 0x0
-	BPF_STACK_BUILD_ID_VALID                = 0x1
-	BPF_STACK_BUILD_ID_IP                   = 0x2
-	BPF_F_RECOMPUTE_CSUM                    = 0x1
-	BPF_F_INVALIDATE_HASH                   = 0x2
-	BPF_F_HDR_FIELD_MASK                    = 0xf
-	BPF_F_PSEUDO_HDR                        = 0x10
-	BPF_F_MARK_MANGLED_0                    = 0x20
-	BPF_F_MARK_ENFORCE                      = 0x40
-	BPF_F_INGRESS                           = 0x1
-	BPF_F_TUNINFO_IPV6                      = 0x1
-	BPF_F_SKIP_FIELD_MASK                   = 0xff
-	BPF_F_USER_STACK                        = 0x100
-	BPF_F_FAST_STACK_CMP                    = 0x200
-	BPF_F_REUSE_STACKID                     = 0x400
-	BPF_F_USER_BUILD_ID                     = 0x800
-	BPF_F_ZERO_CSUM_TX                      = 0x2
-	BPF_F_DONT_FRAGMENT                     = 0x4
-	BPF_F_SEQ_NUMBER                        = 0x8
-	BPF_F_INDEX_MASK                        = 0xffffffff
-	BPF_F_CURRENT_CPU                       = 0xffffffff
-	BPF_F_CTXLEN_MASK                       = 0xfffff00000000
-	BPF_F_CURRENT_NETNS                     = -0x1
-	BPF_CSUM_LEVEL_QUERY                    = 0x0
-	BPF_CSUM_LEVEL_INC                      = 0x1
-	BPF_CSUM_LEVEL_DEC                      = 0x2
-	BPF_CSUM_LEVEL_RESET                    = 0x3
-	BPF_F_ADJ_ROOM_FIXED_GSO                = 0x1
-	BPF_F_ADJ_ROOM_ENCAP_L3_IPV4            = 0x2
-	BPF_F_ADJ_ROOM_ENCAP_L3_IPV6            = 0x4
-	BPF_F_ADJ_ROOM_ENCAP_L4_GRE             = 0x8
-	BPF_F_ADJ_ROOM_ENCAP_L4_UDP             = 0x10
-	BPF_F_ADJ_ROOM_NO_CSUM_RESET            = 0x20
-	BPF_ADJ_ROOM_ENCAP_L2_MASK              = 0xff
-	BPF_ADJ_ROOM_ENCAP_L2_SHIFT             = 0x38
-	BPF_F_SYSCTL_BASE_NAME                  = 0x1
-	BPF_SK_STORAGE_GET_F_CREATE             = 0x1
-	BPF_F_GET_BRANCH_RECORDS_SIZE           = 0x1
-	BPF_RB_NO_WAKEUP                        = 0x1
-	BPF_RB_FORCE_WAKEUP                     = 0x2
-	BPF_RB_AVAIL_DATA                       = 0x0
-	BPF_RB_RING_SIZE                        = 0x1
-	BPF_RB_CONS_POS                         = 0x2
-	BPF_RB_PROD_POS                         = 0x3
-	BPF_RINGBUF_BUSY_BIT                    = 0x80000000
-	BPF_RINGBUF_DISCARD_BIT                 = 0x40000000
-	BPF_RINGBUF_HDR_SZ                      = 0x8
-	BPF_ADJ_ROOM_NET                        = 0x0
-	BPF_ADJ_ROOM_MAC                        = 0x1
-	BPF_HDR_START_MAC                       = 0x0
-	BPF_HDR_START_NET                       = 0x1
-	BPF_LWT_ENCAP_SEG6                      = 0x0
-	BPF_LWT_ENCAP_SEG6_INLINE               = 0x1
-	BPF_LWT_ENCAP_IP                        = 0x2
-	BPF_OK                                  = 0x0
-	BPF_DROP                                = 0x2
-	BPF_REDIRECT                            = 0x7
-	BPF_LWT_REROUTE                         = 0x80
-	BPF_SOCK_OPS_RTO_CB_FLAG                = 0x1
-	BPF_SOCK_OPS_RETRANS_CB_FLAG            = 0x2
-	BPF_SOCK_OPS_STATE_CB_FLAG              = 0x4
-	BPF_SOCK_OPS_RTT_CB_FLAG                = 0x8
-	BPF_SOCK_OPS_ALL_CB_FLAGS               = 0xf
-	BPF_SOCK_OPS_VOID                       = 0x0
-	BPF_SOCK_OPS_TIMEOUT_INIT               = 0x1
-	BPF_SOCK_OPS_RWND_INIT                  = 0x2
-	BPF_SOCK_OPS_TCP_CONNECT_CB             = 0x3
-	BPF_SOCK_OPS_ACTIVE_ESTABLISHED_CB      = 0x4
-	BPF_SOCK_OPS_PASSIVE_ESTABLISHED_CB     = 0x5
-	BPF_SOCK_OPS_NEEDS_ECN                  = 0x6
-	BPF_SOCK_OPS_BASE_RTT                   = 0x7
-	BPF_SOCK_OPS_RTO_CB                     = 0x8
-	BPF_SOCK_OPS_RETRANS_CB                 = 0x9
-	BPF_SOCK_OPS_STATE_CB                   = 0xa
-	BPF_SOCK_OPS_TCP_LISTEN_CB              = 0xb
-	BPF_SOCK_OPS_RTT_CB                     = 0xc
-	BPF_TCP_ESTABLISHED                     = 0x1
-	BPF_TCP_SYN_SENT                        = 0x2
-	BPF_TCP_SYN_RECV                        = 0x3
-	BPF_TCP_FIN_WAIT1                       = 0x4
-	BPF_TCP_FIN_WAIT2                       = 0x5
-	BPF_TCP_TIME_WAIT                       = 0x6
-	BPF_TCP_CLOSE                           = 0x7
-	BPF_TCP_CLOSE_WAIT                      = 0x8
-	BPF_TCP_LAST_ACK                        = 0x9
-	BPF_TCP_LISTEN                          = 0xa
-	BPF_TCP_CLOSING                         = 0xb
-	BPF_TCP_NEW_SYN_RECV                    = 0xc
-	BPF_TCP_MAX_STATES                      = 0xd
-	TCP_BPF_IW                              = 0x3e9
-	TCP_BPF_SNDCWND_CLAMP                   = 0x3ea
-	BPF_DEVCG_ACC_MKNOD                     = 0x1
-	BPF_DEVCG_ACC_READ                      = 0x2
-	BPF_DEVCG_ACC_WRITE                     = 0x4
-	BPF_DEVCG_DEV_BLOCK                     = 0x1
-	BPF_DEVCG_DEV_CHAR                      = 0x2
-	BPF_FIB_LOOKUP_DIRECT                   = 0x1
-	BPF_FIB_LOOKUP_OUTPUT                   = 0x2
-	BPF_FIB_LKUP_RET_SUCCESS                = 0x0
-	BPF_FIB_LKUP_RET_BLACKHOLE              = 0x1
-	BPF_FIB_LKUP_RET_UNREACHABLE            = 0x2
-	BPF_FIB_LKUP_RET_PROHIBIT               = 0x3
-	BPF_FIB_LKUP_RET_NOT_FWDED              = 0x4
-	BPF_FIB_LKUP_RET_FWD_DISABLED           = 0x5
-	BPF_FIB_LKUP_RET_UNSUPP_LWT             = 0x6
-	BPF_FIB_LKUP_RET_NO_NEIGH               = 0x7
-	BPF_FIB_LKUP_RET_FRAG_NEEDED            = 0x8
-	BPF_FD_TYPE_RAW_TRACEPOINT              = 0x0
-	BPF_FD_TYPE_TRACEPOINT                  = 0x1
-	BPF_FD_TYPE_KPROBE                      = 0x2
-	BPF_FD_TYPE_KRETPROBE                   = 0x3
-	BPF_FD_TYPE_UPROBE                      = 0x4
-	BPF_FD_TYPE_URETPROBE                   = 0x5
-	BPF_FLOW_DISSECTOR_F_PARSE_1ST_FRAG     = 0x1
-	BPF_FLOW_DISSECTOR_F_STOP_AT_FLOW_LABEL = 0x2
-	BPF_FLOW_DISSECTOR_F_STOP_AT_ENCAP      = 0x4
+	BPF_REG_0                                  = 0x0
+	BPF_REG_1                                  = 0x1
+	BPF_REG_2                                  = 0x2
+	BPF_REG_3                                  = 0x3
+	BPF_REG_4                                  = 0x4
+	BPF_REG_5                                  = 0x5
+	BPF_REG_6                                  = 0x6
+	BPF_REG_7                                  = 0x7
+	BPF_REG_8                                  = 0x8
+	BPF_REG_9                                  = 0x9
+	BPF_REG_10                                 = 0xa
+	BPF_MAP_CREATE                             = 0x0
+	BPF_MAP_LOOKUP_ELEM                        = 0x1
+	BPF_MAP_UPDATE_ELEM                        = 0x2
+	BPF_MAP_DELETE_ELEM                        = 0x3
+	BPF_MAP_GET_NEXT_KEY                       = 0x4
+	BPF_PROG_LOAD                              = 0x5
+	BPF_OBJ_PIN                                = 0x6
+	BPF_OBJ_GET                                = 0x7
+	BPF_PROG_ATTACH                            = 0x8
+	BPF_PROG_DETACH                            = 0x9
+	BPF_PROG_TEST_RUN                          = 0xa
+	BPF_PROG_GET_NEXT_ID                       = 0xb
+	BPF_MAP_GET_NEXT_ID                        = 0xc
+	BPF_PROG_GET_FD_BY_ID                      = 0xd
+	BPF_MAP_GET_FD_BY_ID                       = 0xe
+	BPF_OBJ_GET_INFO_BY_FD                     = 0xf
+	BPF_PROG_QUERY                             = 0x10
+	BPF_RAW_TRACEPOINT_OPEN                    = 0x11
+	BPF_BTF_LOAD                               = 0x12
+	BPF_BTF_GET_FD_BY_ID                       = 0x13
+	BPF_TASK_FD_QUERY                          = 0x14
+	BPF_MAP_LOOKUP_AND_DELETE_ELEM             = 0x15
+	BPF_MAP_FREEZE                             = 0x16
+	BPF_BTF_GET_NEXT_ID                        = 0x17
+	BPF_MAP_LOOKUP_BATCH                       = 0x18
+	BPF_MAP_LOOKUP_AND_DELETE_BATCH            = 0x19
+	BPF_MAP_UPDATE_BATCH                       = 0x1a
+	BPF_MAP_DELETE_BATCH                       = 0x1b
+	BPF_LINK_CREATE                            = 0x1c
+	BPF_LINK_UPDATE                            = 0x1d
+	BPF_LINK_GET_FD_BY_ID                      = 0x1e
+	BPF_LINK_GET_NEXT_ID                       = 0x1f
+	BPF_ENABLE_STATS                           = 0x20
+	BPF_ITER_CREATE                            = 0x21
+	BPF_LINK_DETACH                            = 0x22
+	BPF_PROG_BIND_MAP                          = 0x23
+	BPF_MAP_TYPE_UNSPEC                        = 0x0
+	BPF_MAP_TYPE_HASH                          = 0x1
+	BPF_MAP_TYPE_ARRAY                         = 0x2
+	BPF_MAP_TYPE_PROG_ARRAY                    = 0x3
+	BPF_MAP_TYPE_PERF_EVENT_ARRAY              = 0x4
+	BPF_MAP_TYPE_PERCPU_HASH                   = 0x5
+	BPF_MAP_TYPE_PERCPU_ARRAY                  = 0x6
+	BPF_MAP_TYPE_STACK_TRACE                   = 0x7
+	BPF_MAP_TYPE_CGROUP_ARRAY                  = 0x8
+	BPF_MAP_TYPE_LRU_HASH                      = 0x9
+	BPF_MAP_TYPE_LRU_PERCPU_HASH               = 0xa
+	BPF_MAP_TYPE_LPM_TRIE                      = 0xb
+	BPF_MAP_TYPE_ARRAY_OF_MAPS                 = 0xc
+	BPF_MAP_TYPE_HASH_OF_MAPS                  = 0xd
+	BPF_MAP_TYPE_DEVMAP                        = 0xe
+	BPF_MAP_TYPE_SOCKMAP                       = 0xf
+	BPF_MAP_TYPE_CPUMAP                        = 0x10
+	BPF_MAP_TYPE_XSKMAP                        = 0x11
+	BPF_MAP_TYPE_SOCKHASH                      = 0x12
+	BPF_MAP_TYPE_CGROUP_STORAGE                = 0x13
+	BPF_MAP_TYPE_REUSEPORT_SOCKARRAY           = 0x14
+	BPF_MAP_TYPE_PERCPU_CGROUP_STORAGE         = 0x15
+	BPF_MAP_TYPE_QUEUE                         = 0x16
+	BPF_MAP_TYPE_STACK                         = 0x17
+	BPF_MAP_TYPE_SK_STORAGE                    = 0x18
+	BPF_MAP_TYPE_DEVMAP_HASH                   = 0x19
+	BPF_MAP_TYPE_STRUCT_OPS                    = 0x1a
+	BPF_MAP_TYPE_RINGBUF                       = 0x1b
+	BPF_MAP_TYPE_INODE_STORAGE                 = 0x1c
+	BPF_PROG_TYPE_UNSPEC                       = 0x0
+	BPF_PROG_TYPE_SOCKET_FILTER                = 0x1
+	BPF_PROG_TYPE_KPROBE                       = 0x2
+	BPF_PROG_TYPE_SCHED_CLS                    = 0x3
+	BPF_PROG_TYPE_SCHED_ACT                    = 0x4
+	BPF_PROG_TYPE_TRACEPOINT                   = 0x5
+	BPF_PROG_TYPE_XDP                          = 0x6
+	BPF_PROG_TYPE_PERF_EVENT                   = 0x7
+	BPF_PROG_TYPE_CGROUP_SKB                   = 0x8
+	BPF_PROG_TYPE_CGROUP_SOCK                  = 0x9
+	BPF_PROG_TYPE_LWT_IN                       = 0xa
+	BPF_PROG_TYPE_LWT_OUT                      = 0xb
+	BPF_PROG_TYPE_LWT_XMIT                     = 0xc
+	BPF_PROG_TYPE_SOCK_OPS                     = 0xd
+	BPF_PROG_TYPE_SK_SKB                       = 0xe
+	BPF_PROG_TYPE_CGROUP_DEVICE                = 0xf
+	BPF_PROG_TYPE_SK_MSG                       = 0x10
+	BPF_PROG_TYPE_RAW_TRACEPOINT               = 0x11
+	BPF_PROG_TYPE_CGROUP_SOCK_ADDR             = 0x12
+	BPF_PROG_TYPE_LWT_SEG6LOCAL                = 0x13
+	BPF_PROG_TYPE_LIRC_MODE2                   = 0x14
+	BPF_PROG_TYPE_SK_REUSEPORT                 = 0x15
+	BPF_PROG_TYPE_FLOW_DISSECTOR               = 0x16
+	BPF_PROG_TYPE_CGROUP_SYSCTL                = 0x17
+	BPF_PROG_TYPE_RAW_TRACEPOINT_WRITABLE      = 0x18
+	BPF_PROG_TYPE_CGROUP_SOCKOPT               = 0x19
+	BPF_PROG_TYPE_TRACING                      = 0x1a
+	BPF_PROG_TYPE_STRUCT_OPS                   = 0x1b
+	BPF_PROG_TYPE_EXT                          = 0x1c
+	BPF_PROG_TYPE_LSM                          = 0x1d
+	BPF_PROG_TYPE_SK_LOOKUP                    = 0x1e
+	BPF_CGROUP_INET_INGRESS                    = 0x0
+	BPF_CGROUP_INET_EGRESS                     = 0x1
+	BPF_CGROUP_INET_SOCK_CREATE                = 0x2
+	BPF_CGROUP_SOCK_OPS                        = 0x3
+	BPF_SK_SKB_STREAM_PARSER                   = 0x4
+	BPF_SK_SKB_STREAM_VERDICT                  = 0x5
+	BPF_CGROUP_DEVICE                          = 0x6
+	BPF_SK_MSG_VERDICT                         = 0x7
+	BPF_CGROUP_INET4_BIND                      = 0x8
+	BPF_CGROUP_INET6_BIND                      = 0x9
+	BPF_CGROUP_INET4_CONNECT                   = 0xa
+	BPF_CGROUP_INET6_CONNECT                   = 0xb
+	BPF_CGROUP_INET4_POST_BIND                 = 0xc
+	BPF_CGROUP_INET6_POST_BIND                 = 0xd
+	BPF_CGROUP_UDP4_SENDMSG                    = 0xe
+	BPF_CGROUP_UDP6_SENDMSG                    = 0xf
+	BPF_LIRC_MODE2                             = 0x10
+	BPF_FLOW_DISSECTOR                         = 0x11
+	BPF_CGROUP_SYSCTL                          = 0x12
+	BPF_CGROUP_UDP4_RECVMSG                    = 0x13
+	BPF_CGROUP_UDP6_RECVMSG                    = 0x14
+	BPF_CGROUP_GETSOCKOPT                      = 0x15
+	BPF_CGROUP_SETSOCKOPT                      = 0x16
+	BPF_TRACE_RAW_TP                           = 0x17
+	BPF_TRACE_FENTRY                           = 0x18
+	BPF_TRACE_FEXIT                            = 0x19
+	BPF_MODIFY_RETURN                          = 0x1a
+	BPF_LSM_MAC                                = 0x1b
+	BPF_TRACE_ITER                             = 0x1c
+	BPF_CGROUP_INET4_GETPEERNAME               = 0x1d
+	BPF_CGROUP_INET6_GETPEERNAME               = 0x1e
+	BPF_CGROUP_INET4_GETSOCKNAME               = 0x1f
+	BPF_CGROUP_INET6_GETSOCKNAME               = 0x20
+	BPF_XDP_DEVMAP                             = 0x21
+	BPF_CGROUP_INET_SOCK_RELEASE               = 0x22
+	BPF_XDP_CPUMAP                             = 0x23
+	BPF_SK_LOOKUP                              = 0x24
+	BPF_XDP                                    = 0x25
+	BPF_LINK_TYPE_UNSPEC                       = 0x0
+	BPF_LINK_TYPE_RAW_TRACEPOINT               = 0x1
+	BPF_LINK_TYPE_TRACING                      = 0x2
+	BPF_LINK_TYPE_CGROUP                       = 0x3
+	BPF_LINK_TYPE_ITER                         = 0x4
+	BPF_LINK_TYPE_NETNS                        = 0x5
+	BPF_LINK_TYPE_XDP                          = 0x6
+	BPF_ANY                                    = 0x0
+	BPF_NOEXIST                                = 0x1
+	BPF_EXIST                                  = 0x2
+	BPF_F_LOCK                                 = 0x4
+	BPF_F_NO_PREALLOC                          = 0x1
+	BPF_F_NO_COMMON_LRU                        = 0x2
+	BPF_F_NUMA_NODE                            = 0x4
+	BPF_F_RDONLY                               = 0x8
+	BPF_F_WRONLY                               = 0x10
+	BPF_F_STACK_BUILD_ID                       = 0x20
+	BPF_F_ZERO_SEED                            = 0x40
+	BPF_F_RDONLY_PROG                          = 0x80
+	BPF_F_WRONLY_PROG                          = 0x100
+	BPF_F_CLONE                                = 0x200
+	BPF_F_MMAPABLE                             = 0x400
+	BPF_F_PRESERVE_ELEMS                       = 0x800
+	BPF_F_INNER_MAP                            = 0x1000
+	BPF_STATS_RUN_TIME                         = 0x0
+	BPF_STACK_BUILD_ID_EMPTY                   = 0x0
+	BPF_STACK_BUILD_ID_VALID                   = 0x1
+	BPF_STACK_BUILD_ID_IP                      = 0x2
+	BPF_F_RECOMPUTE_CSUM                       = 0x1
+	BPF_F_INVALIDATE_HASH                      = 0x2
+	BPF_F_HDR_FIELD_MASK                       = 0xf
+	BPF_F_PSEUDO_HDR                           = 0x10
+	BPF_F_MARK_MANGLED_0                       = 0x20
+	BPF_F_MARK_ENFORCE                         = 0x40
+	BPF_F_INGRESS                              = 0x1
+	BPF_F_TUNINFO_IPV6                         = 0x1
+	BPF_F_SKIP_FIELD_MASK                      = 0xff
+	BPF_F_USER_STACK                           = 0x100
+	BPF_F_FAST_STACK_CMP                       = 0x200
+	BPF_F_REUSE_STACKID                        = 0x400
+	BPF_F_USER_BUILD_ID                        = 0x800
+	BPF_F_ZERO_CSUM_TX                         = 0x2
+	BPF_F_DONT_FRAGMENT                        = 0x4
+	BPF_F_SEQ_NUMBER                           = 0x8
+	BPF_F_INDEX_MASK                           = 0xffffffff
+	BPF_F_CURRENT_CPU                          = 0xffffffff
+	BPF_F_CTXLEN_MASK                          = 0xfffff00000000
+	BPF_F_CURRENT_NETNS                        = -0x1
+	BPF_CSUM_LEVEL_QUERY                       = 0x0
+	BPF_CSUM_LEVEL_INC                         = 0x1
+	BPF_CSUM_LEVEL_DEC                         = 0x2
+	BPF_CSUM_LEVEL_RESET                       = 0x3
+	BPF_F_ADJ_ROOM_FIXED_GSO                   = 0x1
+	BPF_F_ADJ_ROOM_ENCAP_L3_IPV4               = 0x2
+	BPF_F_ADJ_ROOM_ENCAP_L3_IPV6               = 0x4
+	BPF_F_ADJ_ROOM_ENCAP_L4_GRE                = 0x8
+	BPF_F_ADJ_ROOM_ENCAP_L4_UDP                = 0x10
+	BPF_F_ADJ_ROOM_NO_CSUM_RESET               = 0x20
+	BPF_ADJ_ROOM_ENCAP_L2_MASK                 = 0xff
+	BPF_ADJ_ROOM_ENCAP_L2_SHIFT                = 0x38
+	BPF_F_SYSCTL_BASE_NAME                     = 0x1
+	BPF_LOCAL_STORAGE_GET_F_CREATE             = 0x1
+	BPF_SK_STORAGE_GET_F_CREATE                = 0x1
+	BPF_F_GET_BRANCH_RECORDS_SIZE              = 0x1
+	BPF_RB_NO_WAKEUP                           = 0x1
+	BPF_RB_FORCE_WAKEUP                        = 0x2
+	BPF_RB_AVAIL_DATA                          = 0x0
+	BPF_RB_RING_SIZE                           = 0x1
+	BPF_RB_CONS_POS                            = 0x2
+	BPF_RB_PROD_POS                            = 0x3
+	BPF_RINGBUF_BUSY_BIT                       = 0x80000000
+	BPF_RINGBUF_DISCARD_BIT                    = 0x40000000
+	BPF_RINGBUF_HDR_SZ                         = 0x8
+	BPF_SK_LOOKUP_F_REPLACE                    = 0x1
+	BPF_SK_LOOKUP_F_NO_REUSEPORT               = 0x2
+	BPF_ADJ_ROOM_NET                           = 0x0
+	BPF_ADJ_ROOM_MAC                           = 0x1
+	BPF_HDR_START_MAC                          = 0x0
+	BPF_HDR_START_NET                          = 0x1
+	BPF_LWT_ENCAP_SEG6                         = 0x0
+	BPF_LWT_ENCAP_SEG6_INLINE                  = 0x1
+	BPF_LWT_ENCAP_IP                           = 0x2
+	BPF_OK                                     = 0x0
+	BPF_DROP                                   = 0x2
+	BPF_REDIRECT                               = 0x7
+	BPF_LWT_REROUTE                            = 0x80
+	BPF_SOCK_OPS_RTO_CB_FLAG                   = 0x1
+	BPF_SOCK_OPS_RETRANS_CB_FLAG               = 0x2
+	BPF_SOCK_OPS_STATE_CB_FLAG                 = 0x4
+	BPF_SOCK_OPS_RTT_CB_FLAG                   = 0x8
+	BPF_SOCK_OPS_PARSE_ALL_HDR_OPT_CB_FLAG     = 0x10
+	BPF_SOCK_OPS_PARSE_UNKNOWN_HDR_OPT_CB_FLAG = 0x20
+	BPF_SOCK_OPS_WRITE_HDR_OPT_CB_FLAG         = 0x40
+	BPF_SOCK_OPS_ALL_CB_FLAGS                  = 0x7f
+	BPF_SOCK_OPS_VOID                          = 0x0
+	BPF_SOCK_OPS_TIMEOUT_INIT                  = 0x1
+	BPF_SOCK_OPS_RWND_INIT                     = 0x2
+	BPF_SOCK_OPS_TCP_CONNECT_CB                = 0x3
+	BPF_SOCK_OPS_ACTIVE_ESTABLISHED_CB         = 0x4
+	BPF_SOCK_OPS_PASSIVE_ESTABLISHED_CB        = 0x5
+	BPF_SOCK_OPS_NEEDS_ECN                     = 0x6
+	BPF_SOCK_OPS_BASE_RTT                      = 0x7
+	BPF_SOCK_OPS_RTO_CB                        = 0x8
+	BPF_SOCK_OPS_RETRANS_CB                    = 0x9
+	BPF_SOCK_OPS_STATE_CB                      = 0xa
+	BPF_SOCK_OPS_TCP_LISTEN_CB                 = 0xb
+	BPF_SOCK_OPS_RTT_CB                        = 0xc
+	BPF_SOCK_OPS_PARSE_HDR_OPT_CB              = 0xd
+	BPF_SOCK_OPS_HDR_OPT_LEN_CB                = 0xe
+	BPF_SOCK_OPS_WRITE_HDR_OPT_CB              = 0xf
+	BPF_TCP_ESTABLISHED                        = 0x1
+	BPF_TCP_SYN_SENT                           = 0x2
+	BPF_TCP_SYN_RECV                           = 0x3
+	BPF_TCP_FIN_WAIT1                          = 0x4
+	BPF_TCP_FIN_WAIT2                          = 0x5
+	BPF_TCP_TIME_WAIT                          = 0x6
+	BPF_TCP_CLOSE                              = 0x7
+	BPF_TCP_CLOSE_WAIT                         = 0x8
+	BPF_TCP_LAST_ACK                           = 0x9
+	BPF_TCP_LISTEN                             = 0xa
+	BPF_TCP_CLOSING                            = 0xb
+	BPF_TCP_NEW_SYN_RECV                       = 0xc
+	BPF_TCP_MAX_STATES                         = 0xd
+	TCP_BPF_IW                                 = 0x3e9
+	TCP_BPF_SNDCWND_CLAMP                      = 0x3ea
+	TCP_BPF_DELACK_MAX                         = 0x3eb
+	TCP_BPF_RTO_MIN                            = 0x3ec
+	TCP_BPF_SYN                                = 0x3ed
+	TCP_BPF_SYN_IP                             = 0x3ee
+	TCP_BPF_SYN_MAC                            = 0x3ef
+	BPF_LOAD_HDR_OPT_TCP_SYN                   = 0x1
+	BPF_WRITE_HDR_TCP_CURRENT_MSS              = 0x1
+	BPF_WRITE_HDR_TCP_SYNACK_COOKIE            = 0x2
+	BPF_DEVCG_ACC_MKNOD                        = 0x1
+	BPF_DEVCG_ACC_READ                         = 0x2
+	BPF_DEVCG_ACC_WRITE                        = 0x4
+	BPF_DEVCG_DEV_BLOCK                        = 0x1
+	BPF_DEVCG_DEV_CHAR                         = 0x2
+	BPF_FIB_LOOKUP_DIRECT                      = 0x1
+	BPF_FIB_LOOKUP_OUTPUT                      = 0x2
+	BPF_FIB_LKUP_RET_SUCCESS                   = 0x0
+	BPF_FIB_LKUP_RET_BLACKHOLE                 = 0x1
+	BPF_FIB_LKUP_RET_UNREACHABLE               = 0x2
+	BPF_FIB_LKUP_RET_PROHIBIT                  = 0x3
+	BPF_FIB_LKUP_RET_NOT_FWDED                 = 0x4
+	BPF_FIB_LKUP_RET_FWD_DISABLED              = 0x5
+	BPF_FIB_LKUP_RET_UNSUPP_LWT                = 0x6
+	BPF_FIB_LKUP_RET_NO_NEIGH                  = 0x7
+	BPF_FIB_LKUP_RET_FRAG_NEEDED               = 0x8
+	BPF_FD_TYPE_RAW_TRACEPOINT                 = 0x0
+	BPF_FD_TYPE_TRACEPOINT                     = 0x1
+	BPF_FD_TYPE_KPROBE                         = 0x2
+	BPF_FD_TYPE_KRETPROBE                      = 0x3
+	BPF_FD_TYPE_UPROBE                         = 0x4
+	BPF_FD_TYPE_URETPROBE                      = 0x5
+	BPF_FLOW_DISSECTOR_F_PARSE_1ST_FRAG        = 0x1
+	BPF_FLOW_DISSECTOR_F_STOP_AT_FLOW_LABEL    = 0x2
+	BPF_FLOW_DISSECTOR_F_STOP_AT_ENCAP         = 0x4
 )
 
 const (
@@ -2291,6 +2746,7 @@
 	RTNLGRP_IPV4_MROUTE_R = 0x1e
 	RTNLGRP_IPV6_MROUTE_R = 0x1f
 	RTNLGRP_NEXTHOP       = 0x20
+	RTNLGRP_BRVLAN        = 0x21
 )
 
 type CapUserHeader struct {
@@ -2385,132 +2841,320 @@
 )
 
 const (
-	DEVLINK_CMD_UNSPEC                        = 0x0
-	DEVLINK_CMD_GET                           = 0x1
-	DEVLINK_CMD_SET                           = 0x2
-	DEVLINK_CMD_NEW                           = 0x3
-	DEVLINK_CMD_DEL                           = 0x4
-	DEVLINK_CMD_PORT_GET                      = 0x5
-	DEVLINK_CMD_PORT_SET                      = 0x6
-	DEVLINK_CMD_PORT_NEW                      = 0x7
-	DEVLINK_CMD_PORT_DEL                      = 0x8
-	DEVLINK_CMD_PORT_SPLIT                    = 0x9
-	DEVLINK_CMD_PORT_UNSPLIT                  = 0xa
-	DEVLINK_CMD_SB_GET                        = 0xb
-	DEVLINK_CMD_SB_SET                        = 0xc
-	DEVLINK_CMD_SB_NEW                        = 0xd
-	DEVLINK_CMD_SB_DEL                        = 0xe
-	DEVLINK_CMD_SB_POOL_GET                   = 0xf
-	DEVLINK_CMD_SB_POOL_SET                   = 0x10
-	DEVLINK_CMD_SB_POOL_NEW                   = 0x11
-	DEVLINK_CMD_SB_POOL_DEL                   = 0x12
-	DEVLINK_CMD_SB_PORT_POOL_GET              = 0x13
-	DEVLINK_CMD_SB_PORT_POOL_SET              = 0x14
-	DEVLINK_CMD_SB_PORT_POOL_NEW              = 0x15
-	DEVLINK_CMD_SB_PORT_POOL_DEL              = 0x16
-	DEVLINK_CMD_SB_TC_POOL_BIND_GET           = 0x17
-	DEVLINK_CMD_SB_TC_POOL_BIND_SET           = 0x18
-	DEVLINK_CMD_SB_TC_POOL_BIND_NEW           = 0x19
-	DEVLINK_CMD_SB_TC_POOL_BIND_DEL           = 0x1a
-	DEVLINK_CMD_SB_OCC_SNAPSHOT               = 0x1b
-	DEVLINK_CMD_SB_OCC_MAX_CLEAR              = 0x1c
-	DEVLINK_CMD_ESWITCH_GET                   = 0x1d
-	DEVLINK_CMD_ESWITCH_SET                   = 0x1e
-	DEVLINK_CMD_DPIPE_TABLE_GET               = 0x1f
-	DEVLINK_CMD_DPIPE_ENTRIES_GET             = 0x20
-	DEVLINK_CMD_DPIPE_HEADERS_GET             = 0x21
-	DEVLINK_CMD_DPIPE_TABLE_COUNTERS_SET      = 0x22
-	DEVLINK_CMD_MAX                           = 0x48
-	DEVLINK_PORT_TYPE_NOTSET                  = 0x0
-	DEVLINK_PORT_TYPE_AUTO                    = 0x1
-	DEVLINK_PORT_TYPE_ETH                     = 0x2
-	DEVLINK_PORT_TYPE_IB                      = 0x3
-	DEVLINK_SB_POOL_TYPE_INGRESS              = 0x0
-	DEVLINK_SB_POOL_TYPE_EGRESS               = 0x1
-	DEVLINK_SB_THRESHOLD_TYPE_STATIC          = 0x0
-	DEVLINK_SB_THRESHOLD_TYPE_DYNAMIC         = 0x1
-	DEVLINK_ESWITCH_MODE_LEGACY               = 0x0
-	DEVLINK_ESWITCH_MODE_SWITCHDEV            = 0x1
-	DEVLINK_ESWITCH_INLINE_MODE_NONE          = 0x0
-	DEVLINK_ESWITCH_INLINE_MODE_LINK          = 0x1
-	DEVLINK_ESWITCH_INLINE_MODE_NETWORK       = 0x2
-	DEVLINK_ESWITCH_INLINE_MODE_TRANSPORT     = 0x3
-	DEVLINK_ESWITCH_ENCAP_MODE_NONE           = 0x0
-	DEVLINK_ESWITCH_ENCAP_MODE_BASIC          = 0x1
-	DEVLINK_ATTR_UNSPEC                       = 0x0
-	DEVLINK_ATTR_BUS_NAME                     = 0x1
-	DEVLINK_ATTR_DEV_NAME                     = 0x2
-	DEVLINK_ATTR_PORT_INDEX                   = 0x3
-	DEVLINK_ATTR_PORT_TYPE                    = 0x4
-	DEVLINK_ATTR_PORT_DESIRED_TYPE            = 0x5
-	DEVLINK_ATTR_PORT_NETDEV_IFINDEX          = 0x6
-	DEVLINK_ATTR_PORT_NETDEV_NAME             = 0x7
-	DEVLINK_ATTR_PORT_IBDEV_NAME              = 0x8
-	DEVLINK_ATTR_PORT_SPLIT_COUNT             = 0x9
-	DEVLINK_ATTR_PORT_SPLIT_GROUP             = 0xa
-	DEVLINK_ATTR_SB_INDEX                     = 0xb
-	DEVLINK_ATTR_SB_SIZE                      = 0xc
-	DEVLINK_ATTR_SB_INGRESS_POOL_COUNT        = 0xd
-	DEVLINK_ATTR_SB_EGRESS_POOL_COUNT         = 0xe
-	DEVLINK_ATTR_SB_INGRESS_TC_COUNT          = 0xf
-	DEVLINK_ATTR_SB_EGRESS_TC_COUNT           = 0x10
-	DEVLINK_ATTR_SB_POOL_INDEX                = 0x11
-	DEVLINK_ATTR_SB_POOL_TYPE                 = 0x12
-	DEVLINK_ATTR_SB_POOL_SIZE                 = 0x13
-	DEVLINK_ATTR_SB_POOL_THRESHOLD_TYPE       = 0x14
-	DEVLINK_ATTR_SB_THRESHOLD                 = 0x15
-	DEVLINK_ATTR_SB_TC_INDEX                  = 0x16
-	DEVLINK_ATTR_SB_OCC_CUR                   = 0x17
-	DEVLINK_ATTR_SB_OCC_MAX                   = 0x18
-	DEVLINK_ATTR_ESWITCH_MODE                 = 0x19
-	DEVLINK_ATTR_ESWITCH_INLINE_MODE          = 0x1a
-	DEVLINK_ATTR_DPIPE_TABLES                 = 0x1b
-	DEVLINK_ATTR_DPIPE_TABLE                  = 0x1c
-	DEVLINK_ATTR_DPIPE_TABLE_NAME             = 0x1d
-	DEVLINK_ATTR_DPIPE_TABLE_SIZE             = 0x1e
-	DEVLINK_ATTR_DPIPE_TABLE_MATCHES          = 0x1f
-	DEVLINK_ATTR_DPIPE_TABLE_ACTIONS          = 0x20
-	DEVLINK_ATTR_DPIPE_TABLE_COUNTERS_ENABLED = 0x21
-	DEVLINK_ATTR_DPIPE_ENTRIES                = 0x22
-	DEVLINK_ATTR_DPIPE_ENTRY                  = 0x23
-	DEVLINK_ATTR_DPIPE_ENTRY_INDEX            = 0x24
-	DEVLINK_ATTR_DPIPE_ENTRY_MATCH_VALUES     = 0x25
-	DEVLINK_ATTR_DPIPE_ENTRY_ACTION_VALUES    = 0x26
-	DEVLINK_ATTR_DPIPE_ENTRY_COUNTER          = 0x27
-	DEVLINK_ATTR_DPIPE_MATCH                  = 0x28
-	DEVLINK_ATTR_DPIPE_MATCH_VALUE            = 0x29
-	DEVLINK_ATTR_DPIPE_MATCH_TYPE             = 0x2a
-	DEVLINK_ATTR_DPIPE_ACTION                 = 0x2b
-	DEVLINK_ATTR_DPIPE_ACTION_VALUE           = 0x2c
-	DEVLINK_ATTR_DPIPE_ACTION_TYPE            = 0x2d
-	DEVLINK_ATTR_DPIPE_VALUE                  = 0x2e
-	DEVLINK_ATTR_DPIPE_VALUE_MASK             = 0x2f
-	DEVLINK_ATTR_DPIPE_VALUE_MAPPING          = 0x30
-	DEVLINK_ATTR_DPIPE_HEADERS                = 0x31
-	DEVLINK_ATTR_DPIPE_HEADER                 = 0x32
-	DEVLINK_ATTR_DPIPE_HEADER_NAME            = 0x33
-	DEVLINK_ATTR_DPIPE_HEADER_ID              = 0x34
-	DEVLINK_ATTR_DPIPE_HEADER_FIELDS          = 0x35
-	DEVLINK_ATTR_DPIPE_HEADER_GLOBAL          = 0x36
-	DEVLINK_ATTR_DPIPE_HEADER_INDEX           = 0x37
-	DEVLINK_ATTR_DPIPE_FIELD                  = 0x38
-	DEVLINK_ATTR_DPIPE_FIELD_NAME             = 0x39
-	DEVLINK_ATTR_DPIPE_FIELD_ID               = 0x3a
-	DEVLINK_ATTR_DPIPE_FIELD_BITWIDTH         = 0x3b
-	DEVLINK_ATTR_DPIPE_FIELD_MAPPING_TYPE     = 0x3c
-	DEVLINK_ATTR_PAD                          = 0x3d
-	DEVLINK_ATTR_ESWITCH_ENCAP_MODE           = 0x3e
-	DEVLINK_ATTR_MAX                          = 0x90
-	DEVLINK_DPIPE_FIELD_MAPPING_TYPE_NONE     = 0x0
-	DEVLINK_DPIPE_FIELD_MAPPING_TYPE_IFINDEX  = 0x1
-	DEVLINK_DPIPE_MATCH_TYPE_FIELD_EXACT      = 0x0
-	DEVLINK_DPIPE_ACTION_TYPE_FIELD_MODIFY    = 0x0
-	DEVLINK_DPIPE_FIELD_ETHERNET_DST_MAC      = 0x0
-	DEVLINK_DPIPE_FIELD_IPV4_DST_IP           = 0x0
-	DEVLINK_DPIPE_FIELD_IPV6_DST_IP           = 0x0
-	DEVLINK_DPIPE_HEADER_ETHERNET             = 0x0
-	DEVLINK_DPIPE_HEADER_IPV4                 = 0x1
-	DEVLINK_DPIPE_HEADER_IPV6                 = 0x2
+	DEVLINK_CMD_UNSPEC                                 = 0x0
+	DEVLINK_CMD_GET                                    = 0x1
+	DEVLINK_CMD_SET                                    = 0x2
+	DEVLINK_CMD_NEW                                    = 0x3
+	DEVLINK_CMD_DEL                                    = 0x4
+	DEVLINK_CMD_PORT_GET                               = 0x5
+	DEVLINK_CMD_PORT_SET                               = 0x6
+	DEVLINK_CMD_PORT_NEW                               = 0x7
+	DEVLINK_CMD_PORT_DEL                               = 0x8
+	DEVLINK_CMD_PORT_SPLIT                             = 0x9
+	DEVLINK_CMD_PORT_UNSPLIT                           = 0xa
+	DEVLINK_CMD_SB_GET                                 = 0xb
+	DEVLINK_CMD_SB_SET                                 = 0xc
+	DEVLINK_CMD_SB_NEW                                 = 0xd
+	DEVLINK_CMD_SB_DEL                                 = 0xe
+	DEVLINK_CMD_SB_POOL_GET                            = 0xf
+	DEVLINK_CMD_SB_POOL_SET                            = 0x10
+	DEVLINK_CMD_SB_POOL_NEW                            = 0x11
+	DEVLINK_CMD_SB_POOL_DEL                            = 0x12
+	DEVLINK_CMD_SB_PORT_POOL_GET                       = 0x13
+	DEVLINK_CMD_SB_PORT_POOL_SET                       = 0x14
+	DEVLINK_CMD_SB_PORT_POOL_NEW                       = 0x15
+	DEVLINK_CMD_SB_PORT_POOL_DEL                       = 0x16
+	DEVLINK_CMD_SB_TC_POOL_BIND_GET                    = 0x17
+	DEVLINK_CMD_SB_TC_POOL_BIND_SET                    = 0x18
+	DEVLINK_CMD_SB_TC_POOL_BIND_NEW                    = 0x19
+	DEVLINK_CMD_SB_TC_POOL_BIND_DEL                    = 0x1a
+	DEVLINK_CMD_SB_OCC_SNAPSHOT                        = 0x1b
+	DEVLINK_CMD_SB_OCC_MAX_CLEAR                       = 0x1c
+	DEVLINK_CMD_ESWITCH_GET                            = 0x1d
+	DEVLINK_CMD_ESWITCH_SET                            = 0x1e
+	DEVLINK_CMD_DPIPE_TABLE_GET                        = 0x1f
+	DEVLINK_CMD_DPIPE_ENTRIES_GET                      = 0x20
+	DEVLINK_CMD_DPIPE_HEADERS_GET                      = 0x21
+	DEVLINK_CMD_DPIPE_TABLE_COUNTERS_SET               = 0x22
+	DEVLINK_CMD_RESOURCE_SET                           = 0x23
+	DEVLINK_CMD_RESOURCE_DUMP                          = 0x24
+	DEVLINK_CMD_RELOAD                                 = 0x25
+	DEVLINK_CMD_PARAM_GET                              = 0x26
+	DEVLINK_CMD_PARAM_SET                              = 0x27
+	DEVLINK_CMD_PARAM_NEW                              = 0x28
+	DEVLINK_CMD_PARAM_DEL                              = 0x29
+	DEVLINK_CMD_REGION_GET                             = 0x2a
+	DEVLINK_CMD_REGION_SET                             = 0x2b
+	DEVLINK_CMD_REGION_NEW                             = 0x2c
+	DEVLINK_CMD_REGION_DEL                             = 0x2d
+	DEVLINK_CMD_REGION_READ                            = 0x2e
+	DEVLINK_CMD_PORT_PARAM_GET                         = 0x2f
+	DEVLINK_CMD_PORT_PARAM_SET                         = 0x30
+	DEVLINK_CMD_PORT_PARAM_NEW                         = 0x31
+	DEVLINK_CMD_PORT_PARAM_DEL                         = 0x32
+	DEVLINK_CMD_INFO_GET                               = 0x33
+	DEVLINK_CMD_HEALTH_REPORTER_GET                    = 0x34
+	DEVLINK_CMD_HEALTH_REPORTER_SET                    = 0x35
+	DEVLINK_CMD_HEALTH_REPORTER_RECOVER                = 0x36
+	DEVLINK_CMD_HEALTH_REPORTER_DIAGNOSE               = 0x37
+	DEVLINK_CMD_HEALTH_REPORTER_DUMP_GET               = 0x38
+	DEVLINK_CMD_HEALTH_REPORTER_DUMP_CLEAR             = 0x39
+	DEVLINK_CMD_FLASH_UPDATE                           = 0x3a
+	DEVLINK_CMD_FLASH_UPDATE_END                       = 0x3b
+	DEVLINK_CMD_FLASH_UPDATE_STATUS                    = 0x3c
+	DEVLINK_CMD_TRAP_GET                               = 0x3d
+	DEVLINK_CMD_TRAP_SET                               = 0x3e
+	DEVLINK_CMD_TRAP_NEW                               = 0x3f
+	DEVLINK_CMD_TRAP_DEL                               = 0x40
+	DEVLINK_CMD_TRAP_GROUP_GET                         = 0x41
+	DEVLINK_CMD_TRAP_GROUP_SET                         = 0x42
+	DEVLINK_CMD_TRAP_GROUP_NEW                         = 0x43
+	DEVLINK_CMD_TRAP_GROUP_DEL                         = 0x44
+	DEVLINK_CMD_TRAP_POLICER_GET                       = 0x45
+	DEVLINK_CMD_TRAP_POLICER_SET                       = 0x46
+	DEVLINK_CMD_TRAP_POLICER_NEW                       = 0x47
+	DEVLINK_CMD_TRAP_POLICER_DEL                       = 0x48
+	DEVLINK_CMD_HEALTH_REPORTER_TEST                   = 0x49
+	DEVLINK_CMD_MAX                                    = 0x49
+	DEVLINK_PORT_TYPE_NOTSET                           = 0x0
+	DEVLINK_PORT_TYPE_AUTO                             = 0x1
+	DEVLINK_PORT_TYPE_ETH                              = 0x2
+	DEVLINK_PORT_TYPE_IB                               = 0x3
+	DEVLINK_SB_POOL_TYPE_INGRESS                       = 0x0
+	DEVLINK_SB_POOL_TYPE_EGRESS                        = 0x1
+	DEVLINK_SB_THRESHOLD_TYPE_STATIC                   = 0x0
+	DEVLINK_SB_THRESHOLD_TYPE_DYNAMIC                  = 0x1
+	DEVLINK_ESWITCH_MODE_LEGACY                        = 0x0
+	DEVLINK_ESWITCH_MODE_SWITCHDEV                     = 0x1
+	DEVLINK_ESWITCH_INLINE_MODE_NONE                   = 0x0
+	DEVLINK_ESWITCH_INLINE_MODE_LINK                   = 0x1
+	DEVLINK_ESWITCH_INLINE_MODE_NETWORK                = 0x2
+	DEVLINK_ESWITCH_INLINE_MODE_TRANSPORT              = 0x3
+	DEVLINK_ESWITCH_ENCAP_MODE_NONE                    = 0x0
+	DEVLINK_ESWITCH_ENCAP_MODE_BASIC                   = 0x1
+	DEVLINK_PORT_FLAVOUR_PHYSICAL                      = 0x0
+	DEVLINK_PORT_FLAVOUR_CPU                           = 0x1
+	DEVLINK_PORT_FLAVOUR_DSA                           = 0x2
+	DEVLINK_PORT_FLAVOUR_PCI_PF                        = 0x3
+	DEVLINK_PORT_FLAVOUR_PCI_VF                        = 0x4
+	DEVLINK_PORT_FLAVOUR_VIRTUAL                       = 0x5
+	DEVLINK_PORT_FLAVOUR_UNUSED                        = 0x6
+	DEVLINK_PARAM_CMODE_RUNTIME                        = 0x0
+	DEVLINK_PARAM_CMODE_DRIVERINIT                     = 0x1
+	DEVLINK_PARAM_CMODE_PERMANENT                      = 0x2
+	DEVLINK_PARAM_CMODE_MAX                            = 0x2
+	DEVLINK_PARAM_FW_LOAD_POLICY_VALUE_DRIVER          = 0x0
+	DEVLINK_PARAM_FW_LOAD_POLICY_VALUE_FLASH           = 0x1
+	DEVLINK_PARAM_FW_LOAD_POLICY_VALUE_DISK            = 0x2
+	DEVLINK_PARAM_FW_LOAD_POLICY_VALUE_UNKNOWN         = 0x3
+	DEVLINK_PARAM_RESET_DEV_ON_DRV_PROBE_VALUE_UNKNOWN = 0x0
+	DEVLINK_PARAM_RESET_DEV_ON_DRV_PROBE_VALUE_ALWAYS  = 0x1
+	DEVLINK_PARAM_RESET_DEV_ON_DRV_PROBE_VALUE_NEVER   = 0x2
+	DEVLINK_PARAM_RESET_DEV_ON_DRV_PROBE_VALUE_DISK    = 0x3
+	DEVLINK_ATTR_STATS_RX_PACKETS                      = 0x0
+	DEVLINK_ATTR_STATS_RX_BYTES                        = 0x1
+	DEVLINK_ATTR_STATS_RX_DROPPED                      = 0x2
+	DEVLINK_ATTR_STATS_MAX                             = 0x2
+	DEVLINK_FLASH_OVERWRITE_SETTINGS_BIT               = 0x0
+	DEVLINK_FLASH_OVERWRITE_IDENTIFIERS_BIT            = 0x1
+	DEVLINK_FLASH_OVERWRITE_MAX_BIT                    = 0x1
+	DEVLINK_TRAP_ACTION_DROP                           = 0x0
+	DEVLINK_TRAP_ACTION_TRAP                           = 0x1
+	DEVLINK_TRAP_ACTION_MIRROR                         = 0x2
+	DEVLINK_TRAP_TYPE_DROP                             = 0x0
+	DEVLINK_TRAP_TYPE_EXCEPTION                        = 0x1
+	DEVLINK_TRAP_TYPE_CONTROL                          = 0x2
+	DEVLINK_ATTR_TRAP_METADATA_TYPE_IN_PORT            = 0x0
+	DEVLINK_ATTR_TRAP_METADATA_TYPE_FA_COOKIE          = 0x1
+	DEVLINK_RELOAD_ACTION_UNSPEC                       = 0x0
+	DEVLINK_RELOAD_ACTION_DRIVER_REINIT                = 0x1
+	DEVLINK_RELOAD_ACTION_FW_ACTIVATE                  = 0x2
+	DEVLINK_RELOAD_ACTION_MAX                          = 0x2
+	DEVLINK_RELOAD_LIMIT_UNSPEC                        = 0x0
+	DEVLINK_RELOAD_LIMIT_NO_RESET                      = 0x1
+	DEVLINK_RELOAD_LIMIT_MAX                           = 0x1
+	DEVLINK_ATTR_UNSPEC                                = 0x0
+	DEVLINK_ATTR_BUS_NAME                              = 0x1
+	DEVLINK_ATTR_DEV_NAME                              = 0x2
+	DEVLINK_ATTR_PORT_INDEX                            = 0x3
+	DEVLINK_ATTR_PORT_TYPE                             = 0x4
+	DEVLINK_ATTR_PORT_DESIRED_TYPE                     = 0x5
+	DEVLINK_ATTR_PORT_NETDEV_IFINDEX                   = 0x6
+	DEVLINK_ATTR_PORT_NETDEV_NAME                      = 0x7
+	DEVLINK_ATTR_PORT_IBDEV_NAME                       = 0x8
+	DEVLINK_ATTR_PORT_SPLIT_COUNT                      = 0x9
+	DEVLINK_ATTR_PORT_SPLIT_GROUP                      = 0xa
+	DEVLINK_ATTR_SB_INDEX                              = 0xb
+	DEVLINK_ATTR_SB_SIZE                               = 0xc
+	DEVLINK_ATTR_SB_INGRESS_POOL_COUNT                 = 0xd
+	DEVLINK_ATTR_SB_EGRESS_POOL_COUNT                  = 0xe
+	DEVLINK_ATTR_SB_INGRESS_TC_COUNT                   = 0xf
+	DEVLINK_ATTR_SB_EGRESS_TC_COUNT                    = 0x10
+	DEVLINK_ATTR_SB_POOL_INDEX                         = 0x11
+	DEVLINK_ATTR_SB_POOL_TYPE                          = 0x12
+	DEVLINK_ATTR_SB_POOL_SIZE                          = 0x13
+	DEVLINK_ATTR_SB_POOL_THRESHOLD_TYPE                = 0x14
+	DEVLINK_ATTR_SB_THRESHOLD                          = 0x15
+	DEVLINK_ATTR_SB_TC_INDEX                           = 0x16
+	DEVLINK_ATTR_SB_OCC_CUR                            = 0x17
+	DEVLINK_ATTR_SB_OCC_MAX                            = 0x18
+	DEVLINK_ATTR_ESWITCH_MODE                          = 0x19
+	DEVLINK_ATTR_ESWITCH_INLINE_MODE                   = 0x1a
+	DEVLINK_ATTR_DPIPE_TABLES                          = 0x1b
+	DEVLINK_ATTR_DPIPE_TABLE                           = 0x1c
+	DEVLINK_ATTR_DPIPE_TABLE_NAME                      = 0x1d
+	DEVLINK_ATTR_DPIPE_TABLE_SIZE                      = 0x1e
+	DEVLINK_ATTR_DPIPE_TABLE_MATCHES                   = 0x1f
+	DEVLINK_ATTR_DPIPE_TABLE_ACTIONS                   = 0x20
+	DEVLINK_ATTR_DPIPE_TABLE_COUNTERS_ENABLED          = 0x21
+	DEVLINK_ATTR_DPIPE_ENTRIES                         = 0x22
+	DEVLINK_ATTR_DPIPE_ENTRY                           = 0x23
+	DEVLINK_ATTR_DPIPE_ENTRY_INDEX                     = 0x24
+	DEVLINK_ATTR_DPIPE_ENTRY_MATCH_VALUES              = 0x25
+	DEVLINK_ATTR_DPIPE_ENTRY_ACTION_VALUES             = 0x26
+	DEVLINK_ATTR_DPIPE_ENTRY_COUNTER                   = 0x27
+	DEVLINK_ATTR_DPIPE_MATCH                           = 0x28
+	DEVLINK_ATTR_DPIPE_MATCH_VALUE                     = 0x29
+	DEVLINK_ATTR_DPIPE_MATCH_TYPE                      = 0x2a
+	DEVLINK_ATTR_DPIPE_ACTION                          = 0x2b
+	DEVLINK_ATTR_DPIPE_ACTION_VALUE                    = 0x2c
+	DEVLINK_ATTR_DPIPE_ACTION_TYPE                     = 0x2d
+	DEVLINK_ATTR_DPIPE_VALUE                           = 0x2e
+	DEVLINK_ATTR_DPIPE_VALUE_MASK                      = 0x2f
+	DEVLINK_ATTR_DPIPE_VALUE_MAPPING                   = 0x30
+	DEVLINK_ATTR_DPIPE_HEADERS                         = 0x31
+	DEVLINK_ATTR_DPIPE_HEADER                          = 0x32
+	DEVLINK_ATTR_DPIPE_HEADER_NAME                     = 0x33
+	DEVLINK_ATTR_DPIPE_HEADER_ID                       = 0x34
+	DEVLINK_ATTR_DPIPE_HEADER_FIELDS                   = 0x35
+	DEVLINK_ATTR_DPIPE_HEADER_GLOBAL                   = 0x36
+	DEVLINK_ATTR_DPIPE_HEADER_INDEX                    = 0x37
+	DEVLINK_ATTR_DPIPE_FIELD                           = 0x38
+	DEVLINK_ATTR_DPIPE_FIELD_NAME                      = 0x39
+	DEVLINK_ATTR_DPIPE_FIELD_ID                        = 0x3a
+	DEVLINK_ATTR_DPIPE_FIELD_BITWIDTH                  = 0x3b
+	DEVLINK_ATTR_DPIPE_FIELD_MAPPING_TYPE              = 0x3c
+	DEVLINK_ATTR_PAD                                   = 0x3d
+	DEVLINK_ATTR_ESWITCH_ENCAP_MODE                    = 0x3e
+	DEVLINK_ATTR_RESOURCE_LIST                         = 0x3f
+	DEVLINK_ATTR_RESOURCE                              = 0x40
+	DEVLINK_ATTR_RESOURCE_NAME                         = 0x41
+	DEVLINK_ATTR_RESOURCE_ID                           = 0x42
+	DEVLINK_ATTR_RESOURCE_SIZE                         = 0x43
+	DEVLINK_ATTR_RESOURCE_SIZE_NEW                     = 0x44
+	DEVLINK_ATTR_RESOURCE_SIZE_VALID                   = 0x45
+	DEVLINK_ATTR_RESOURCE_SIZE_MIN                     = 0x46
+	DEVLINK_ATTR_RESOURCE_SIZE_MAX                     = 0x47
+	DEVLINK_ATTR_RESOURCE_SIZE_GRAN                    = 0x48
+	DEVLINK_ATTR_RESOURCE_UNIT                         = 0x49
+	DEVLINK_ATTR_RESOURCE_OCC                          = 0x4a
+	DEVLINK_ATTR_DPIPE_TABLE_RESOURCE_ID               = 0x4b
+	DEVLINK_ATTR_DPIPE_TABLE_RESOURCE_UNITS            = 0x4c
+	DEVLINK_ATTR_PORT_FLAVOUR                          = 0x4d
+	DEVLINK_ATTR_PORT_NUMBER                           = 0x4e
+	DEVLINK_ATTR_PORT_SPLIT_SUBPORT_NUMBER             = 0x4f
+	DEVLINK_ATTR_PARAM                                 = 0x50
+	DEVLINK_ATTR_PARAM_NAME                            = 0x51
+	DEVLINK_ATTR_PARAM_GENERIC                         = 0x52
+	DEVLINK_ATTR_PARAM_TYPE                            = 0x53
+	DEVLINK_ATTR_PARAM_VALUES_LIST                     = 0x54
+	DEVLINK_ATTR_PARAM_VALUE                           = 0x55
+	DEVLINK_ATTR_PARAM_VALUE_DATA                      = 0x56
+	DEVLINK_ATTR_PARAM_VALUE_CMODE                     = 0x57
+	DEVLINK_ATTR_REGION_NAME                           = 0x58
+	DEVLINK_ATTR_REGION_SIZE                           = 0x59
+	DEVLINK_ATTR_REGION_SNAPSHOTS                      = 0x5a
+	DEVLINK_ATTR_REGION_SNAPSHOT                       = 0x5b
+	DEVLINK_ATTR_REGION_SNAPSHOT_ID                    = 0x5c
+	DEVLINK_ATTR_REGION_CHUNKS                         = 0x5d
+	DEVLINK_ATTR_REGION_CHUNK                          = 0x5e
+	DEVLINK_ATTR_REGION_CHUNK_DATA                     = 0x5f
+	DEVLINK_ATTR_REGION_CHUNK_ADDR                     = 0x60
+	DEVLINK_ATTR_REGION_CHUNK_LEN                      = 0x61
+	DEVLINK_ATTR_INFO_DRIVER_NAME                      = 0x62
+	DEVLINK_ATTR_INFO_SERIAL_NUMBER                    = 0x63
+	DEVLINK_ATTR_INFO_VERSION_FIXED                    = 0x64
+	DEVLINK_ATTR_INFO_VERSION_RUNNING                  = 0x65
+	DEVLINK_ATTR_INFO_VERSION_STORED                   = 0x66
+	DEVLINK_ATTR_INFO_VERSION_NAME                     = 0x67
+	DEVLINK_ATTR_INFO_VERSION_VALUE                    = 0x68
+	DEVLINK_ATTR_SB_POOL_CELL_SIZE                     = 0x69
+	DEVLINK_ATTR_FMSG                                  = 0x6a
+	DEVLINK_ATTR_FMSG_OBJ_NEST_START                   = 0x6b
+	DEVLINK_ATTR_FMSG_PAIR_NEST_START                  = 0x6c
+	DEVLINK_ATTR_FMSG_ARR_NEST_START                   = 0x6d
+	DEVLINK_ATTR_FMSG_NEST_END                         = 0x6e
+	DEVLINK_ATTR_FMSG_OBJ_NAME                         = 0x6f
+	DEVLINK_ATTR_FMSG_OBJ_VALUE_TYPE                   = 0x70
+	DEVLINK_ATTR_FMSG_OBJ_VALUE_DATA                   = 0x71
+	DEVLINK_ATTR_HEALTH_REPORTER                       = 0x72
+	DEVLINK_ATTR_HEALTH_REPORTER_NAME                  = 0x73
+	DEVLINK_ATTR_HEALTH_REPORTER_STATE                 = 0x74
+	DEVLINK_ATTR_HEALTH_REPORTER_ERR_COUNT             = 0x75
+	DEVLINK_ATTR_HEALTH_REPORTER_RECOVER_COUNT         = 0x76
+	DEVLINK_ATTR_HEALTH_REPORTER_DUMP_TS               = 0x77
+	DEVLINK_ATTR_HEALTH_REPORTER_GRACEFUL_PERIOD       = 0x78
+	DEVLINK_ATTR_HEALTH_REPORTER_AUTO_RECOVER          = 0x79
+	DEVLINK_ATTR_FLASH_UPDATE_FILE_NAME                = 0x7a
+	DEVLINK_ATTR_FLASH_UPDATE_COMPONENT                = 0x7b
+	DEVLINK_ATTR_FLASH_UPDATE_STATUS_MSG               = 0x7c
+	DEVLINK_ATTR_FLASH_UPDATE_STATUS_DONE              = 0x7d
+	DEVLINK_ATTR_FLASH_UPDATE_STATUS_TOTAL             = 0x7e
+	DEVLINK_ATTR_PORT_PCI_PF_NUMBER                    = 0x7f
+	DEVLINK_ATTR_PORT_PCI_VF_NUMBER                    = 0x80
+	DEVLINK_ATTR_STATS                                 = 0x81
+	DEVLINK_ATTR_TRAP_NAME                             = 0x82
+	DEVLINK_ATTR_TRAP_ACTION                           = 0x83
+	DEVLINK_ATTR_TRAP_TYPE                             = 0x84
+	DEVLINK_ATTR_TRAP_GENERIC                          = 0x85
+	DEVLINK_ATTR_TRAP_METADATA                         = 0x86
+	DEVLINK_ATTR_TRAP_GROUP_NAME                       = 0x87
+	DEVLINK_ATTR_RELOAD_FAILED                         = 0x88
+	DEVLINK_ATTR_HEALTH_REPORTER_DUMP_TS_NS            = 0x89
+	DEVLINK_ATTR_NETNS_FD                              = 0x8a
+	DEVLINK_ATTR_NETNS_PID                             = 0x8b
+	DEVLINK_ATTR_NETNS_ID                              = 0x8c
+	DEVLINK_ATTR_HEALTH_REPORTER_AUTO_DUMP             = 0x8d
+	DEVLINK_ATTR_TRAP_POLICER_ID                       = 0x8e
+	DEVLINK_ATTR_TRAP_POLICER_RATE                     = 0x8f
+	DEVLINK_ATTR_TRAP_POLICER_BURST                    = 0x90
+	DEVLINK_ATTR_PORT_FUNCTION                         = 0x91
+	DEVLINK_ATTR_INFO_BOARD_SERIAL_NUMBER              = 0x92
+	DEVLINK_ATTR_PORT_LANES                            = 0x93
+	DEVLINK_ATTR_PORT_SPLITTABLE                       = 0x94
+	DEVLINK_ATTR_PORT_EXTERNAL                         = 0x95
+	DEVLINK_ATTR_PORT_CONTROLLER_NUMBER                = 0x96
+	DEVLINK_ATTR_FLASH_UPDATE_STATUS_TIMEOUT           = 0x97
+	DEVLINK_ATTR_FLASH_UPDATE_OVERWRITE_MASK           = 0x98
+	DEVLINK_ATTR_RELOAD_ACTION                         = 0x99
+	DEVLINK_ATTR_RELOAD_ACTIONS_PERFORMED              = 0x9a
+	DEVLINK_ATTR_RELOAD_LIMITS                         = 0x9b
+	DEVLINK_ATTR_DEV_STATS                             = 0x9c
+	DEVLINK_ATTR_RELOAD_STATS                          = 0x9d
+	DEVLINK_ATTR_RELOAD_STATS_ENTRY                    = 0x9e
+	DEVLINK_ATTR_RELOAD_STATS_LIMIT                    = 0x9f
+	DEVLINK_ATTR_RELOAD_STATS_VALUE                    = 0xa0
+	DEVLINK_ATTR_REMOTE_RELOAD_STATS                   = 0xa1
+	DEVLINK_ATTR_RELOAD_ACTION_INFO                    = 0xa2
+	DEVLINK_ATTR_RELOAD_ACTION_STATS                   = 0xa3
+	DEVLINK_ATTR_PORT_PCI_SF_NUMBER                    = 0xa4
+	DEVLINK_ATTR_MAX                                   = 0xa4
+	DEVLINK_DPIPE_FIELD_MAPPING_TYPE_NONE              = 0x0
+	DEVLINK_DPIPE_FIELD_MAPPING_TYPE_IFINDEX           = 0x1
+	DEVLINK_DPIPE_MATCH_TYPE_FIELD_EXACT               = 0x0
+	DEVLINK_DPIPE_ACTION_TYPE_FIELD_MODIFY             = 0x0
+	DEVLINK_DPIPE_FIELD_ETHERNET_DST_MAC               = 0x0
+	DEVLINK_DPIPE_FIELD_IPV4_DST_IP                    = 0x0
+	DEVLINK_DPIPE_FIELD_IPV6_DST_IP                    = 0x0
+	DEVLINK_DPIPE_HEADER_ETHERNET                      = 0x0
+	DEVLINK_DPIPE_HEADER_IPV4                          = 0x1
+	DEVLINK_DPIPE_HEADER_IPV6                          = 0x2
+	DEVLINK_RESOURCE_UNIT_ENTRY                        = 0x0
+	DEVLINK_PORT_FUNCTION_ATTR_UNSPEC                  = 0x0
+	DEVLINK_PORT_FUNCTION_ATTR_HW_ADDR                 = 0x1
+	DEVLINK_PORT_FN_ATTR_STATE                         = 0x2
+	DEVLINK_PORT_FN_ATTR_OPSTATE                       = 0x3
+	DEVLINK_PORT_FUNCTION_ATTR_MAX                     = 0x3
 )
 
 type FsverityDigest struct {
@@ -2567,3 +3211,697 @@
 	CAN_RAW_FD_FRAMES     = 0x5
 	CAN_RAW_JOIN_FILTERS  = 0x6
 )
+
+type WatchdogInfo struct {
+	Options  uint32
+	Version  uint32
+	Identity [32]uint8
+}
+
+type PPSFData struct {
+	Info    PPSKInfo
+	Timeout PPSKTime
+}
+
+type PPSKParams struct {
+	Api_version   int32
+	Mode          int32
+	Assert_off_tu PPSKTime
+	Clear_off_tu  PPSKTime
+}
+
+type PPSKTime struct {
+	Sec   int64
+	Nsec  int32
+	Flags uint32
+}
+
+const (
+	LWTUNNEL_ENCAP_NONE       = 0x0
+	LWTUNNEL_ENCAP_MPLS       = 0x1
+	LWTUNNEL_ENCAP_IP         = 0x2
+	LWTUNNEL_ENCAP_ILA        = 0x3
+	LWTUNNEL_ENCAP_IP6        = 0x4
+	LWTUNNEL_ENCAP_SEG6       = 0x5
+	LWTUNNEL_ENCAP_BPF        = 0x6
+	LWTUNNEL_ENCAP_SEG6_LOCAL = 0x7
+	LWTUNNEL_ENCAP_RPL        = 0x8
+	LWTUNNEL_ENCAP_MAX        = 0x8
+
+	MPLS_IPTUNNEL_UNSPEC = 0x0
+	MPLS_IPTUNNEL_DST    = 0x1
+	MPLS_IPTUNNEL_TTL    = 0x2
+	MPLS_IPTUNNEL_MAX    = 0x2
+)
+
+const (
+	ETHTOOL_ID_UNSPEC                                                       = 0x0
+	ETHTOOL_RX_COPYBREAK                                                    = 0x1
+	ETHTOOL_TX_COPYBREAK                                                    = 0x2
+	ETHTOOL_PFC_PREVENTION_TOUT                                             = 0x3
+	ETHTOOL_TUNABLE_UNSPEC                                                  = 0x0
+	ETHTOOL_TUNABLE_U8                                                      = 0x1
+	ETHTOOL_TUNABLE_U16                                                     = 0x2
+	ETHTOOL_TUNABLE_U32                                                     = 0x3
+	ETHTOOL_TUNABLE_U64                                                     = 0x4
+	ETHTOOL_TUNABLE_STRING                                                  = 0x5
+	ETHTOOL_TUNABLE_S8                                                      = 0x6
+	ETHTOOL_TUNABLE_S16                                                     = 0x7
+	ETHTOOL_TUNABLE_S32                                                     = 0x8
+	ETHTOOL_TUNABLE_S64                                                     = 0x9
+	ETHTOOL_PHY_ID_UNSPEC                                                   = 0x0
+	ETHTOOL_PHY_DOWNSHIFT                                                   = 0x1
+	ETHTOOL_PHY_FAST_LINK_DOWN                                              = 0x2
+	ETHTOOL_PHY_EDPD                                                        = 0x3
+	ETHTOOL_LINK_EXT_STATE_AUTONEG                                          = 0x0
+	ETHTOOL_LINK_EXT_STATE_LINK_TRAINING_FAILURE                            = 0x1
+	ETHTOOL_LINK_EXT_STATE_LINK_LOGICAL_MISMATCH                            = 0x2
+	ETHTOOL_LINK_EXT_STATE_BAD_SIGNAL_INTEGRITY                             = 0x3
+	ETHTOOL_LINK_EXT_STATE_NO_CABLE                                         = 0x4
+	ETHTOOL_LINK_EXT_STATE_CABLE_ISSUE                                      = 0x5
+	ETHTOOL_LINK_EXT_STATE_EEPROM_ISSUE                                     = 0x6
+	ETHTOOL_LINK_EXT_STATE_CALIBRATION_FAILURE                              = 0x7
+	ETHTOOL_LINK_EXT_STATE_POWER_BUDGET_EXCEEDED                            = 0x8
+	ETHTOOL_LINK_EXT_STATE_OVERHEAT                                         = 0x9
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_NO_PARTNER_DETECTED                        = 0x1
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_ACK_NOT_RECEIVED                           = 0x2
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_NEXT_PAGE_EXCHANGE_FAILED                  = 0x3
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_NO_PARTNER_DETECTED_FORCE_MODE             = 0x4
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_FEC_MISMATCH_DURING_OVERRIDE               = 0x5
+	ETHTOOL_LINK_EXT_SUBSTATE_AN_NO_HCD                                     = 0x6
+	ETHTOOL_LINK_EXT_SUBSTATE_LT_KR_FRAME_LOCK_NOT_ACQUIRED                 = 0x1
+	ETHTOOL_LINK_EXT_SUBSTATE_LT_KR_LINK_INHIBIT_TIMEOUT                    = 0x2
+	ETHTOOL_LINK_EXT_SUBSTATE_LT_KR_LINK_PARTNER_DID_NOT_SET_RECEIVER_READY = 0x3
+	ETHTOOL_LINK_EXT_SUBSTATE_LT_REMOTE_FAULT                               = 0x4
+	ETHTOOL_LINK_EXT_SUBSTATE_LLM_PCS_DID_NOT_ACQUIRE_BLOCK_LOCK            = 0x1
+	ETHTOOL_LINK_EXT_SUBSTATE_LLM_PCS_DID_NOT_ACQUIRE_AM_LOCK               = 0x2
+	ETHTOOL_LINK_EXT_SUBSTATE_LLM_PCS_DID_NOT_GET_ALIGN_STATUS              = 0x3
+	ETHTOOL_LINK_EXT_SUBSTATE_LLM_FC_FEC_IS_NOT_LOCKED                      = 0x4
+	ETHTOOL_LINK_EXT_SUBSTATE_LLM_RS_FEC_IS_NOT_LOCKED                      = 0x5
+	ETHTOOL_LINK_EXT_SUBSTATE_BSI_LARGE_NUMBER_OF_PHYSICAL_ERRORS           = 0x1
+	ETHTOOL_LINK_EXT_SUBSTATE_BSI_UNSUPPORTED_RATE                          = 0x2
+	ETHTOOL_LINK_EXT_SUBSTATE_CI_UNSUPPORTED_CABLE                          = 0x1
+	ETHTOOL_LINK_EXT_SUBSTATE_CI_CABLE_TEST_FAILURE                         = 0x2
+	ETHTOOL_FLASH_ALL_REGIONS                                               = 0x0
+	ETHTOOL_F_UNSUPPORTED__BIT                                              = 0x0
+	ETHTOOL_F_WISH__BIT                                                     = 0x1
+	ETHTOOL_F_COMPAT__BIT                                                   = 0x2
+	ETHTOOL_FEC_NONE_BIT                                                    = 0x0
+	ETHTOOL_FEC_AUTO_BIT                                                    = 0x1
+	ETHTOOL_FEC_OFF_BIT                                                     = 0x2
+	ETHTOOL_FEC_RS_BIT                                                      = 0x3
+	ETHTOOL_FEC_BASER_BIT                                                   = 0x4
+	ETHTOOL_FEC_LLRS_BIT                                                    = 0x5
+	ETHTOOL_LINK_MODE_10baseT_Half_BIT                                      = 0x0
+	ETHTOOL_LINK_MODE_10baseT_Full_BIT                                      = 0x1
+	ETHTOOL_LINK_MODE_100baseT_Half_BIT                                     = 0x2
+	ETHTOOL_LINK_MODE_100baseT_Full_BIT                                     = 0x3
+	ETHTOOL_LINK_MODE_1000baseT_Half_BIT                                    = 0x4
+	ETHTOOL_LINK_MODE_1000baseT_Full_BIT                                    = 0x5
+	ETHTOOL_LINK_MODE_Autoneg_BIT                                           = 0x6
+	ETHTOOL_LINK_MODE_TP_BIT                                                = 0x7
+	ETHTOOL_LINK_MODE_AUI_BIT                                               = 0x8
+	ETHTOOL_LINK_MODE_MII_BIT                                               = 0x9
+	ETHTOOL_LINK_MODE_FIBRE_BIT                                             = 0xa
+	ETHTOOL_LINK_MODE_BNC_BIT                                               = 0xb
+	ETHTOOL_LINK_MODE_10000baseT_Full_BIT                                   = 0xc
+	ETHTOOL_LINK_MODE_Pause_BIT                                             = 0xd
+	ETHTOOL_LINK_MODE_Asym_Pause_BIT                                        = 0xe
+	ETHTOOL_LINK_MODE_2500baseX_Full_BIT                                    = 0xf
+	ETHTOOL_LINK_MODE_Backplane_BIT                                         = 0x10
+	ETHTOOL_LINK_MODE_1000baseKX_Full_BIT                                   = 0x11
+	ETHTOOL_LINK_MODE_10000baseKX4_Full_BIT                                 = 0x12
+	ETHTOOL_LINK_MODE_10000baseKR_Full_BIT                                  = 0x13
+	ETHTOOL_LINK_MODE_10000baseR_FEC_BIT                                    = 0x14
+	ETHTOOL_LINK_MODE_20000baseMLD2_Full_BIT                                = 0x15
+	ETHTOOL_LINK_MODE_20000baseKR2_Full_BIT                                 = 0x16
+	ETHTOOL_LINK_MODE_40000baseKR4_Full_BIT                                 = 0x17
+	ETHTOOL_LINK_MODE_40000baseCR4_Full_BIT                                 = 0x18
+	ETHTOOL_LINK_MODE_40000baseSR4_Full_BIT                                 = 0x19
+	ETHTOOL_LINK_MODE_40000baseLR4_Full_BIT                                 = 0x1a
+	ETHTOOL_LINK_MODE_56000baseKR4_Full_BIT                                 = 0x1b
+	ETHTOOL_LINK_MODE_56000baseCR4_Full_BIT                                 = 0x1c
+	ETHTOOL_LINK_MODE_56000baseSR4_Full_BIT                                 = 0x1d
+	ETHTOOL_LINK_MODE_56000baseLR4_Full_BIT                                 = 0x1e
+	ETHTOOL_LINK_MODE_25000baseCR_Full_BIT                                  = 0x1f
+	ETHTOOL_LINK_MODE_25000baseKR_Full_BIT                                  = 0x20
+	ETHTOOL_LINK_MODE_25000baseSR_Full_BIT                                  = 0x21
+	ETHTOOL_LINK_MODE_50000baseCR2_Full_BIT                                 = 0x22
+	ETHTOOL_LINK_MODE_50000baseKR2_Full_BIT                                 = 0x23
+	ETHTOOL_LINK_MODE_100000baseKR4_Full_BIT                                = 0x24
+	ETHTOOL_LINK_MODE_100000baseSR4_Full_BIT                                = 0x25
+	ETHTOOL_LINK_MODE_100000baseCR4_Full_BIT                                = 0x26
+	ETHTOOL_LINK_MODE_100000baseLR4_ER4_Full_BIT                            = 0x27
+	ETHTOOL_LINK_MODE_50000baseSR2_Full_BIT                                 = 0x28
+	ETHTOOL_LINK_MODE_1000baseX_Full_BIT                                    = 0x29
+	ETHTOOL_LINK_MODE_10000baseCR_Full_BIT                                  = 0x2a
+	ETHTOOL_LINK_MODE_10000baseSR_Full_BIT                                  = 0x2b
+	ETHTOOL_LINK_MODE_10000baseLR_Full_BIT                                  = 0x2c
+	ETHTOOL_LINK_MODE_10000baseLRM_Full_BIT                                 = 0x2d
+	ETHTOOL_LINK_MODE_10000baseER_Full_BIT                                  = 0x2e
+	ETHTOOL_LINK_MODE_2500baseT_Full_BIT                                    = 0x2f
+	ETHTOOL_LINK_MODE_5000baseT_Full_BIT                                    = 0x30
+	ETHTOOL_LINK_MODE_FEC_NONE_BIT                                          = 0x31
+	ETHTOOL_LINK_MODE_FEC_RS_BIT                                            = 0x32
+	ETHTOOL_LINK_MODE_FEC_BASER_BIT                                         = 0x33
+	ETHTOOL_LINK_MODE_50000baseKR_Full_BIT                                  = 0x34
+	ETHTOOL_LINK_MODE_50000baseSR_Full_BIT                                  = 0x35
+	ETHTOOL_LINK_MODE_50000baseCR_Full_BIT                                  = 0x36
+	ETHTOOL_LINK_MODE_50000baseLR_ER_FR_Full_BIT                            = 0x37
+	ETHTOOL_LINK_MODE_50000baseDR_Full_BIT                                  = 0x38
+	ETHTOOL_LINK_MODE_100000baseKR2_Full_BIT                                = 0x39
+	ETHTOOL_LINK_MODE_100000baseSR2_Full_BIT                                = 0x3a
+	ETHTOOL_LINK_MODE_100000baseCR2_Full_BIT                                = 0x3b
+	ETHTOOL_LINK_MODE_100000baseLR2_ER2_FR2_Full_BIT                        = 0x3c
+	ETHTOOL_LINK_MODE_100000baseDR2_Full_BIT                                = 0x3d
+	ETHTOOL_LINK_MODE_200000baseKR4_Full_BIT                                = 0x3e
+	ETHTOOL_LINK_MODE_200000baseSR4_Full_BIT                                = 0x3f
+	ETHTOOL_LINK_MODE_200000baseLR4_ER4_FR4_Full_BIT                        = 0x40
+	ETHTOOL_LINK_MODE_200000baseDR4_Full_BIT                                = 0x41
+	ETHTOOL_LINK_MODE_200000baseCR4_Full_BIT                                = 0x42
+	ETHTOOL_LINK_MODE_100baseT1_Full_BIT                                    = 0x43
+	ETHTOOL_LINK_MODE_1000baseT1_Full_BIT                                   = 0x44
+	ETHTOOL_LINK_MODE_400000baseKR8_Full_BIT                                = 0x45
+	ETHTOOL_LINK_MODE_400000baseSR8_Full_BIT                                = 0x46
+	ETHTOOL_LINK_MODE_400000baseLR8_ER8_FR8_Full_BIT                        = 0x47
+	ETHTOOL_LINK_MODE_400000baseDR8_Full_BIT                                = 0x48
+	ETHTOOL_LINK_MODE_400000baseCR8_Full_BIT                                = 0x49
+	ETHTOOL_LINK_MODE_FEC_LLRS_BIT                                          = 0x4a
+	ETHTOOL_LINK_MODE_100000baseKR_Full_BIT                                 = 0x4b
+	ETHTOOL_LINK_MODE_100000baseSR_Full_BIT                                 = 0x4c
+	ETHTOOL_LINK_MODE_100000baseLR_ER_FR_Full_BIT                           = 0x4d
+	ETHTOOL_LINK_MODE_100000baseCR_Full_BIT                                 = 0x4e
+	ETHTOOL_LINK_MODE_100000baseDR_Full_BIT                                 = 0x4f
+	ETHTOOL_LINK_MODE_200000baseKR2_Full_BIT                                = 0x50
+	ETHTOOL_LINK_MODE_200000baseSR2_Full_BIT                                = 0x51
+	ETHTOOL_LINK_MODE_200000baseLR2_ER2_FR2_Full_BIT                        = 0x52
+	ETHTOOL_LINK_MODE_200000baseDR2_Full_BIT                                = 0x53
+	ETHTOOL_LINK_MODE_200000baseCR2_Full_BIT                                = 0x54
+	ETHTOOL_LINK_MODE_400000baseKR4_Full_BIT                                = 0x55
+	ETHTOOL_LINK_MODE_400000baseSR4_Full_BIT                                = 0x56
+	ETHTOOL_LINK_MODE_400000baseLR4_ER4_FR4_Full_BIT                        = 0x57
+	ETHTOOL_LINK_MODE_400000baseDR4_Full_BIT                                = 0x58
+	ETHTOOL_LINK_MODE_400000baseCR4_Full_BIT                                = 0x59
+	ETHTOOL_LINK_MODE_100baseFX_Half_BIT                                    = 0x5a
+	ETHTOOL_LINK_MODE_100baseFX_Full_BIT                                    = 0x5b
+
+	ETHTOOL_MSG_USER_NONE                     = 0x0
+	ETHTOOL_MSG_STRSET_GET                    = 0x1
+	ETHTOOL_MSG_LINKINFO_GET                  = 0x2
+	ETHTOOL_MSG_LINKINFO_SET                  = 0x3
+	ETHTOOL_MSG_LINKMODES_GET                 = 0x4
+	ETHTOOL_MSG_LINKMODES_SET                 = 0x5
+	ETHTOOL_MSG_LINKSTATE_GET                 = 0x6
+	ETHTOOL_MSG_DEBUG_GET                     = 0x7
+	ETHTOOL_MSG_DEBUG_SET                     = 0x8
+	ETHTOOL_MSG_WOL_GET                       = 0x9
+	ETHTOOL_MSG_WOL_SET                       = 0xa
+	ETHTOOL_MSG_FEATURES_GET                  = 0xb
+	ETHTOOL_MSG_FEATURES_SET                  = 0xc
+	ETHTOOL_MSG_PRIVFLAGS_GET                 = 0xd
+	ETHTOOL_MSG_PRIVFLAGS_SET                 = 0xe
+	ETHTOOL_MSG_RINGS_GET                     = 0xf
+	ETHTOOL_MSG_RINGS_SET                     = 0x10
+	ETHTOOL_MSG_CHANNELS_GET                  = 0x11
+	ETHTOOL_MSG_CHANNELS_SET                  = 0x12
+	ETHTOOL_MSG_COALESCE_GET                  = 0x13
+	ETHTOOL_MSG_COALESCE_SET                  = 0x14
+	ETHTOOL_MSG_PAUSE_GET                     = 0x15
+	ETHTOOL_MSG_PAUSE_SET                     = 0x16
+	ETHTOOL_MSG_EEE_GET                       = 0x17
+	ETHTOOL_MSG_EEE_SET                       = 0x18
+	ETHTOOL_MSG_TSINFO_GET                    = 0x19
+	ETHTOOL_MSG_CABLE_TEST_ACT                = 0x1a
+	ETHTOOL_MSG_CABLE_TEST_TDR_ACT            = 0x1b
+	ETHTOOL_MSG_TUNNEL_INFO_GET               = 0x1c
+	ETHTOOL_MSG_USER_MAX                      = 0x1c
+	ETHTOOL_MSG_KERNEL_NONE                   = 0x0
+	ETHTOOL_MSG_STRSET_GET_REPLY              = 0x1
+	ETHTOOL_MSG_LINKINFO_GET_REPLY            = 0x2
+	ETHTOOL_MSG_LINKINFO_NTF                  = 0x3
+	ETHTOOL_MSG_LINKMODES_GET_REPLY           = 0x4
+	ETHTOOL_MSG_LINKMODES_NTF                 = 0x5
+	ETHTOOL_MSG_LINKSTATE_GET_REPLY           = 0x6
+	ETHTOOL_MSG_DEBUG_GET_REPLY               = 0x7
+	ETHTOOL_MSG_DEBUG_NTF                     = 0x8
+	ETHTOOL_MSG_WOL_GET_REPLY                 = 0x9
+	ETHTOOL_MSG_WOL_NTF                       = 0xa
+	ETHTOOL_MSG_FEATURES_GET_REPLY            = 0xb
+	ETHTOOL_MSG_FEATURES_SET_REPLY            = 0xc
+	ETHTOOL_MSG_FEATURES_NTF                  = 0xd
+	ETHTOOL_MSG_PRIVFLAGS_GET_REPLY           = 0xe
+	ETHTOOL_MSG_PRIVFLAGS_NTF                 = 0xf
+	ETHTOOL_MSG_RINGS_GET_REPLY               = 0x10
+	ETHTOOL_MSG_RINGS_NTF                     = 0x11
+	ETHTOOL_MSG_CHANNELS_GET_REPLY            = 0x12
+	ETHTOOL_MSG_CHANNELS_NTF                  = 0x13
+	ETHTOOL_MSG_COALESCE_GET_REPLY            = 0x14
+	ETHTOOL_MSG_COALESCE_NTF                  = 0x15
+	ETHTOOL_MSG_PAUSE_GET_REPLY               = 0x16
+	ETHTOOL_MSG_PAUSE_NTF                     = 0x17
+	ETHTOOL_MSG_EEE_GET_REPLY                 = 0x18
+	ETHTOOL_MSG_EEE_NTF                       = 0x19
+	ETHTOOL_MSG_TSINFO_GET_REPLY              = 0x1a
+	ETHTOOL_MSG_CABLE_TEST_NTF                = 0x1b
+	ETHTOOL_MSG_CABLE_TEST_TDR_NTF            = 0x1c
+	ETHTOOL_MSG_TUNNEL_INFO_GET_REPLY         = 0x1d
+	ETHTOOL_MSG_KERNEL_MAX                    = 0x1d
+	ETHTOOL_A_HEADER_UNSPEC                   = 0x0
+	ETHTOOL_A_HEADER_DEV_INDEX                = 0x1
+	ETHTOOL_A_HEADER_DEV_NAME                 = 0x2
+	ETHTOOL_A_HEADER_FLAGS                    = 0x3
+	ETHTOOL_A_HEADER_MAX                      = 0x3
+	ETHTOOL_A_BITSET_BIT_UNSPEC               = 0x0
+	ETHTOOL_A_BITSET_BIT_INDEX                = 0x1
+	ETHTOOL_A_BITSET_BIT_NAME                 = 0x2
+	ETHTOOL_A_BITSET_BIT_VALUE                = 0x3
+	ETHTOOL_A_BITSET_BIT_MAX                  = 0x3
+	ETHTOOL_A_BITSET_BITS_UNSPEC              = 0x0
+	ETHTOOL_A_BITSET_BITS_BIT                 = 0x1
+	ETHTOOL_A_BITSET_BITS_MAX                 = 0x1
+	ETHTOOL_A_BITSET_UNSPEC                   = 0x0
+	ETHTOOL_A_BITSET_NOMASK                   = 0x1
+	ETHTOOL_A_BITSET_SIZE                     = 0x2
+	ETHTOOL_A_BITSET_BITS                     = 0x3
+	ETHTOOL_A_BITSET_VALUE                    = 0x4
+	ETHTOOL_A_BITSET_MASK                     = 0x5
+	ETHTOOL_A_BITSET_MAX                      = 0x5
+	ETHTOOL_A_STRING_UNSPEC                   = 0x0
+	ETHTOOL_A_STRING_INDEX                    = 0x1
+	ETHTOOL_A_STRING_VALUE                    = 0x2
+	ETHTOOL_A_STRING_MAX                      = 0x2
+	ETHTOOL_A_STRINGS_UNSPEC                  = 0x0
+	ETHTOOL_A_STRINGS_STRING                  = 0x1
+	ETHTOOL_A_STRINGS_MAX                     = 0x1
+	ETHTOOL_A_STRINGSET_UNSPEC                = 0x0
+	ETHTOOL_A_STRINGSET_ID                    = 0x1
+	ETHTOOL_A_STRINGSET_COUNT                 = 0x2
+	ETHTOOL_A_STRINGSET_STRINGS               = 0x3
+	ETHTOOL_A_STRINGSET_MAX                   = 0x3
+	ETHTOOL_A_STRINGSETS_UNSPEC               = 0x0
+	ETHTOOL_A_STRINGSETS_STRINGSET            = 0x1
+	ETHTOOL_A_STRINGSETS_MAX                  = 0x1
+	ETHTOOL_A_STRSET_UNSPEC                   = 0x0
+	ETHTOOL_A_STRSET_HEADER                   = 0x1
+	ETHTOOL_A_STRSET_STRINGSETS               = 0x2
+	ETHTOOL_A_STRSET_COUNTS_ONLY              = 0x3
+	ETHTOOL_A_STRSET_MAX                      = 0x3
+	ETHTOOL_A_LINKINFO_UNSPEC                 = 0x0
+	ETHTOOL_A_LINKINFO_HEADER                 = 0x1
+	ETHTOOL_A_LINKINFO_PORT                   = 0x2
+	ETHTOOL_A_LINKINFO_PHYADDR                = 0x3
+	ETHTOOL_A_LINKINFO_TP_MDIX                = 0x4
+	ETHTOOL_A_LINKINFO_TP_MDIX_CTRL           = 0x5
+	ETHTOOL_A_LINKINFO_TRANSCEIVER            = 0x6
+	ETHTOOL_A_LINKINFO_MAX                    = 0x6
+	ETHTOOL_A_LINKMODES_UNSPEC                = 0x0
+	ETHTOOL_A_LINKMODES_HEADER                = 0x1
+	ETHTOOL_A_LINKMODES_AUTONEG               = 0x2
+	ETHTOOL_A_LINKMODES_OURS                  = 0x3
+	ETHTOOL_A_LINKMODES_PEER                  = 0x4
+	ETHTOOL_A_LINKMODES_SPEED                 = 0x5
+	ETHTOOL_A_LINKMODES_DUPLEX                = 0x6
+	ETHTOOL_A_LINKMODES_MASTER_SLAVE_CFG      = 0x7
+	ETHTOOL_A_LINKMODES_MASTER_SLAVE_STATE    = 0x8
+	ETHTOOL_A_LINKMODES_LANES                 = 0x9
+	ETHTOOL_A_LINKMODES_MAX                   = 0x9
+	ETHTOOL_A_LINKSTATE_UNSPEC                = 0x0
+	ETHTOOL_A_LINKSTATE_HEADER                = 0x1
+	ETHTOOL_A_LINKSTATE_LINK                  = 0x2
+	ETHTOOL_A_LINKSTATE_SQI                   = 0x3
+	ETHTOOL_A_LINKSTATE_SQI_MAX               = 0x4
+	ETHTOOL_A_LINKSTATE_EXT_STATE             = 0x5
+	ETHTOOL_A_LINKSTATE_EXT_SUBSTATE          = 0x6
+	ETHTOOL_A_LINKSTATE_MAX                   = 0x6
+	ETHTOOL_A_DEBUG_UNSPEC                    = 0x0
+	ETHTOOL_A_DEBUG_HEADER                    = 0x1
+	ETHTOOL_A_DEBUG_MSGMASK                   = 0x2
+	ETHTOOL_A_DEBUG_MAX                       = 0x2
+	ETHTOOL_A_WOL_UNSPEC                      = 0x0
+	ETHTOOL_A_WOL_HEADER                      = 0x1
+	ETHTOOL_A_WOL_MODES                       = 0x2
+	ETHTOOL_A_WOL_SOPASS                      = 0x3
+	ETHTOOL_A_WOL_MAX                         = 0x3
+	ETHTOOL_A_FEATURES_UNSPEC                 = 0x0
+	ETHTOOL_A_FEATURES_HEADER                 = 0x1
+	ETHTOOL_A_FEATURES_HW                     = 0x2
+	ETHTOOL_A_FEATURES_WANTED                 = 0x3
+	ETHTOOL_A_FEATURES_ACTIVE                 = 0x4
+	ETHTOOL_A_FEATURES_NOCHANGE               = 0x5
+	ETHTOOL_A_FEATURES_MAX                    = 0x5
+	ETHTOOL_A_PRIVFLAGS_UNSPEC                = 0x0
+	ETHTOOL_A_PRIVFLAGS_HEADER                = 0x1
+	ETHTOOL_A_PRIVFLAGS_FLAGS                 = 0x2
+	ETHTOOL_A_PRIVFLAGS_MAX                   = 0x2
+	ETHTOOL_A_RINGS_UNSPEC                    = 0x0
+	ETHTOOL_A_RINGS_HEADER                    = 0x1
+	ETHTOOL_A_RINGS_RX_MAX                    = 0x2
+	ETHTOOL_A_RINGS_RX_MINI_MAX               = 0x3
+	ETHTOOL_A_RINGS_RX_JUMBO_MAX              = 0x4
+	ETHTOOL_A_RINGS_TX_MAX                    = 0x5
+	ETHTOOL_A_RINGS_RX                        = 0x6
+	ETHTOOL_A_RINGS_RX_MINI                   = 0x7
+	ETHTOOL_A_RINGS_RX_JUMBO                  = 0x8
+	ETHTOOL_A_RINGS_TX                        = 0x9
+	ETHTOOL_A_RINGS_MAX                       = 0x9
+	ETHTOOL_A_CHANNELS_UNSPEC                 = 0x0
+	ETHTOOL_A_CHANNELS_HEADER                 = 0x1
+	ETHTOOL_A_CHANNELS_RX_MAX                 = 0x2
+	ETHTOOL_A_CHANNELS_TX_MAX                 = 0x3
+	ETHTOOL_A_CHANNELS_OTHER_MAX              = 0x4
+	ETHTOOL_A_CHANNELS_COMBINED_MAX           = 0x5
+	ETHTOOL_A_CHANNELS_RX_COUNT               = 0x6
+	ETHTOOL_A_CHANNELS_TX_COUNT               = 0x7
+	ETHTOOL_A_CHANNELS_OTHER_COUNT            = 0x8
+	ETHTOOL_A_CHANNELS_COMBINED_COUNT         = 0x9
+	ETHTOOL_A_CHANNELS_MAX                    = 0x9
+	ETHTOOL_A_COALESCE_UNSPEC                 = 0x0
+	ETHTOOL_A_COALESCE_HEADER                 = 0x1
+	ETHTOOL_A_COALESCE_RX_USECS               = 0x2
+	ETHTOOL_A_COALESCE_RX_MAX_FRAMES          = 0x3
+	ETHTOOL_A_COALESCE_RX_USECS_IRQ           = 0x4
+	ETHTOOL_A_COALESCE_RX_MAX_FRAMES_IRQ      = 0x5
+	ETHTOOL_A_COALESCE_TX_USECS               = 0x6
+	ETHTOOL_A_COALESCE_TX_MAX_FRAMES          = 0x7
+	ETHTOOL_A_COALESCE_TX_USECS_IRQ           = 0x8
+	ETHTOOL_A_COALESCE_TX_MAX_FRAMES_IRQ      = 0x9
+	ETHTOOL_A_COALESCE_STATS_BLOCK_USECS      = 0xa
+	ETHTOOL_A_COALESCE_USE_ADAPTIVE_RX        = 0xb
+	ETHTOOL_A_COALESCE_USE_ADAPTIVE_TX        = 0xc
+	ETHTOOL_A_COALESCE_PKT_RATE_LOW           = 0xd
+	ETHTOOL_A_COALESCE_RX_USECS_LOW           = 0xe
+	ETHTOOL_A_COALESCE_RX_MAX_FRAMES_LOW      = 0xf
+	ETHTOOL_A_COALESCE_TX_USECS_LOW           = 0x10
+	ETHTOOL_A_COALESCE_TX_MAX_FRAMES_LOW      = 0x11
+	ETHTOOL_A_COALESCE_PKT_RATE_HIGH          = 0x12
+	ETHTOOL_A_COALESCE_RX_USECS_HIGH          = 0x13
+	ETHTOOL_A_COALESCE_RX_MAX_FRAMES_HIGH     = 0x14
+	ETHTOOL_A_COALESCE_TX_USECS_HIGH          = 0x15
+	ETHTOOL_A_COALESCE_TX_MAX_FRAMES_HIGH     = 0x16
+	ETHTOOL_A_COALESCE_RATE_SAMPLE_INTERVAL   = 0x17
+	ETHTOOL_A_COALESCE_MAX                    = 0x17
+	ETHTOOL_A_PAUSE_UNSPEC                    = 0x0
+	ETHTOOL_A_PAUSE_HEADER                    = 0x1
+	ETHTOOL_A_PAUSE_AUTONEG                   = 0x2
+	ETHTOOL_A_PAUSE_RX                        = 0x3
+	ETHTOOL_A_PAUSE_TX                        = 0x4
+	ETHTOOL_A_PAUSE_STATS                     = 0x5
+	ETHTOOL_A_PAUSE_MAX                       = 0x5
+	ETHTOOL_A_PAUSE_STAT_UNSPEC               = 0x0
+	ETHTOOL_A_PAUSE_STAT_PAD                  = 0x1
+	ETHTOOL_A_PAUSE_STAT_TX_FRAMES            = 0x2
+	ETHTOOL_A_PAUSE_STAT_RX_FRAMES            = 0x3
+	ETHTOOL_A_PAUSE_STAT_MAX                  = 0x3
+	ETHTOOL_A_EEE_UNSPEC                      = 0x0
+	ETHTOOL_A_EEE_HEADER                      = 0x1
+	ETHTOOL_A_EEE_MODES_OURS                  = 0x2
+	ETHTOOL_A_EEE_MODES_PEER                  = 0x3
+	ETHTOOL_A_EEE_ACTIVE                      = 0x4
+	ETHTOOL_A_EEE_ENABLED                     = 0x5
+	ETHTOOL_A_EEE_TX_LPI_ENABLED              = 0x6
+	ETHTOOL_A_EEE_TX_LPI_TIMER                = 0x7
+	ETHTOOL_A_EEE_MAX                         = 0x7
+	ETHTOOL_A_TSINFO_UNSPEC                   = 0x0
+	ETHTOOL_A_TSINFO_HEADER                   = 0x1
+	ETHTOOL_A_TSINFO_TIMESTAMPING             = 0x2
+	ETHTOOL_A_TSINFO_TX_TYPES                 = 0x3
+	ETHTOOL_A_TSINFO_RX_FILTERS               = 0x4
+	ETHTOOL_A_TSINFO_PHC_INDEX                = 0x5
+	ETHTOOL_A_TSINFO_MAX                      = 0x5
+	ETHTOOL_A_CABLE_TEST_UNSPEC               = 0x0
+	ETHTOOL_A_CABLE_TEST_HEADER               = 0x1
+	ETHTOOL_A_CABLE_TEST_MAX                  = 0x1
+	ETHTOOL_A_CABLE_RESULT_CODE_UNSPEC        = 0x0
+	ETHTOOL_A_CABLE_RESULT_CODE_OK            = 0x1
+	ETHTOOL_A_CABLE_RESULT_CODE_OPEN          = 0x2
+	ETHTOOL_A_CABLE_RESULT_CODE_SAME_SHORT    = 0x3
+	ETHTOOL_A_CABLE_RESULT_CODE_CROSS_SHORT   = 0x4
+	ETHTOOL_A_CABLE_PAIR_A                    = 0x0
+	ETHTOOL_A_CABLE_PAIR_B                    = 0x1
+	ETHTOOL_A_CABLE_PAIR_C                    = 0x2
+	ETHTOOL_A_CABLE_PAIR_D                    = 0x3
+	ETHTOOL_A_CABLE_RESULT_UNSPEC             = 0x0
+	ETHTOOL_A_CABLE_RESULT_PAIR               = 0x1
+	ETHTOOL_A_CABLE_RESULT_CODE               = 0x2
+	ETHTOOL_A_CABLE_RESULT_MAX                = 0x2
+	ETHTOOL_A_CABLE_FAULT_LENGTH_UNSPEC       = 0x0
+	ETHTOOL_A_CABLE_FAULT_LENGTH_PAIR         = 0x1
+	ETHTOOL_A_CABLE_FAULT_LENGTH_CM           = 0x2
+	ETHTOOL_A_CABLE_FAULT_LENGTH_MAX          = 0x2
+	ETHTOOL_A_CABLE_TEST_NTF_STATUS_UNSPEC    = 0x0
+	ETHTOOL_A_CABLE_TEST_NTF_STATUS_STARTED   = 0x1
+	ETHTOOL_A_CABLE_TEST_NTF_STATUS_COMPLETED = 0x2
+	ETHTOOL_A_CABLE_NEST_UNSPEC               = 0x0
+	ETHTOOL_A_CABLE_NEST_RESULT               = 0x1
+	ETHTOOL_A_CABLE_NEST_FAULT_LENGTH         = 0x2
+	ETHTOOL_A_CABLE_NEST_MAX                  = 0x2
+	ETHTOOL_A_CABLE_TEST_NTF_UNSPEC           = 0x0
+	ETHTOOL_A_CABLE_TEST_NTF_HEADER           = 0x1
+	ETHTOOL_A_CABLE_TEST_NTF_STATUS           = 0x2
+	ETHTOOL_A_CABLE_TEST_NTF_NEST             = 0x3
+	ETHTOOL_A_CABLE_TEST_NTF_MAX              = 0x3
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_UNSPEC       = 0x0
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_FIRST        = 0x1
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_LAST         = 0x2
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_STEP         = 0x3
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_PAIR         = 0x4
+	ETHTOOL_A_CABLE_TEST_TDR_CFG_MAX          = 0x4
+	ETHTOOL_A_CABLE_TEST_TDR_UNSPEC           = 0x0
+	ETHTOOL_A_CABLE_TEST_TDR_HEADER           = 0x1
+	ETHTOOL_A_CABLE_TEST_TDR_CFG              = 0x2
+	ETHTOOL_A_CABLE_TEST_TDR_MAX              = 0x2
+	ETHTOOL_A_CABLE_AMPLITUDE_UNSPEC          = 0x0
+	ETHTOOL_A_CABLE_AMPLITUDE_PAIR            = 0x1
+	ETHTOOL_A_CABLE_AMPLITUDE_mV              = 0x2
+	ETHTOOL_A_CABLE_AMPLITUDE_MAX             = 0x2
+	ETHTOOL_A_CABLE_PULSE_UNSPEC              = 0x0
+	ETHTOOL_A_CABLE_PULSE_mV                  = 0x1
+	ETHTOOL_A_CABLE_PULSE_MAX                 = 0x1
+	ETHTOOL_A_CABLE_STEP_UNSPEC               = 0x0
+	ETHTOOL_A_CABLE_STEP_FIRST_DISTANCE       = 0x1
+	ETHTOOL_A_CABLE_STEP_LAST_DISTANCE        = 0x2
+	ETHTOOL_A_CABLE_STEP_STEP_DISTANCE        = 0x3
+	ETHTOOL_A_CABLE_STEP_MAX                  = 0x3
+	ETHTOOL_A_CABLE_TDR_NEST_UNSPEC           = 0x0
+	ETHTOOL_A_CABLE_TDR_NEST_STEP             = 0x1
+	ETHTOOL_A_CABLE_TDR_NEST_AMPLITUDE        = 0x2
+	ETHTOOL_A_CABLE_TDR_NEST_PULSE            = 0x3
+	ETHTOOL_A_CABLE_TDR_NEST_MAX              = 0x3
+	ETHTOOL_A_CABLE_TEST_TDR_NTF_UNSPEC       = 0x0
+	ETHTOOL_A_CABLE_TEST_TDR_NTF_HEADER       = 0x1
+	ETHTOOL_A_CABLE_TEST_TDR_NTF_STATUS       = 0x2
+	ETHTOOL_A_CABLE_TEST_TDR_NTF_NEST         = 0x3
+	ETHTOOL_A_CABLE_TEST_TDR_NTF_MAX          = 0x3
+	ETHTOOL_UDP_TUNNEL_TYPE_VXLAN             = 0x0
+	ETHTOOL_UDP_TUNNEL_TYPE_GENEVE            = 0x1
+	ETHTOOL_UDP_TUNNEL_TYPE_VXLAN_GPE         = 0x2
+	ETHTOOL_A_TUNNEL_UDP_ENTRY_UNSPEC         = 0x0
+	ETHTOOL_A_TUNNEL_UDP_ENTRY_PORT           = 0x1
+	ETHTOOL_A_TUNNEL_UDP_ENTRY_TYPE           = 0x2
+	ETHTOOL_A_TUNNEL_UDP_ENTRY_MAX            = 0x2
+	ETHTOOL_A_TUNNEL_UDP_TABLE_UNSPEC         = 0x0
+	ETHTOOL_A_TUNNEL_UDP_TABLE_SIZE           = 0x1
+	ETHTOOL_A_TUNNEL_UDP_TABLE_TYPES          = 0x2
+	ETHTOOL_A_TUNNEL_UDP_TABLE_ENTRY          = 0x3
+	ETHTOOL_A_TUNNEL_UDP_TABLE_MAX            = 0x3
+	ETHTOOL_A_TUNNEL_UDP_UNSPEC               = 0x0
+	ETHTOOL_A_TUNNEL_UDP_TABLE                = 0x1
+	ETHTOOL_A_TUNNEL_UDP_MAX                  = 0x1
+	ETHTOOL_A_TUNNEL_INFO_UNSPEC              = 0x0
+	ETHTOOL_A_TUNNEL_INFO_HEADER              = 0x1
+	ETHTOOL_A_TUNNEL_INFO_UDP_PORTS           = 0x2
+	ETHTOOL_A_TUNNEL_INFO_MAX                 = 0x2
+)
+
+type EthtoolDrvinfo struct {
+	Cmd          uint32
+	Driver       [32]byte
+	Version      [32]byte
+	Fw_version   [32]byte
+	Bus_info     [32]byte
+	Erom_version [32]byte
+	Reserved2    [12]byte
+	N_priv_flags uint32
+	N_stats      uint32
+	Testinfo_len uint32
+	Eedump_len   uint32
+	Regdump_len  uint32
+}
+
+type (
+	HIDRawReportDescriptor struct {
+		Size  uint32
+		Value [4096]uint8
+	}
+	HIDRawDevInfo struct {
+		Bustype uint32
+		Vendor  int16
+		Product int16
+	}
+)
+
+const (
+	CLOSE_RANGE_UNSHARE = 0x2
+	CLOSE_RANGE_CLOEXEC = 0x4
+)
+
+const (
+	NLMSGERR_ATTR_MSG    = 0x1
+	NLMSGERR_ATTR_OFFS   = 0x2
+	NLMSGERR_ATTR_COOKIE = 0x3
+)
+
+type (
+	EraseInfo struct {
+		Start  uint32
+		Length uint32
+	}
+	EraseInfo64 struct {
+		Start  uint64
+		Length uint64
+	}
+	MtdOobBuf struct {
+		Start  uint32
+		Length uint32
+		Ptr    *uint8
+	}
+	MtdOobBuf64 struct {
+		Start  uint64
+		Pad    uint32
+		Length uint32
+		Ptr    uint64
+	}
+	MtdWriteReq struct {
+		Start  uint64
+		Len    uint64
+		Ooblen uint64
+		Data   uint64
+		Oob    uint64
+		Mode   uint8
+		_      [7]uint8
+	}
+	MtdInfo struct {
+		Type      uint8
+		Flags     uint32
+		Size      uint32
+		Erasesize uint32
+		Writesize uint32
+		Oobsize   uint32
+		_         uint64
+	}
+	RegionInfo struct {
+		Offset      uint32
+		Erasesize   uint32
+		Numblocks   uint32
+		Regionindex uint32
+	}
+	OtpInfo struct {
+		Start  uint32
+		Length uint32
+		Locked uint32
+	}
+	NandOobinfo struct {
+		Useecc   uint32
+		Eccbytes uint32
+		Oobfree  [8][2]uint32
+		Eccpos   [32]uint32
+	}
+	NandOobfree struct {
+		Offset uint32
+		Length uint32
+	}
+	NandEcclayout struct {
+		Eccbytes uint32
+		Eccpos   [64]uint32
+		Oobavail uint32
+		Oobfree  [8]NandOobfree
+	}
+	MtdEccStats struct {
+		Corrected uint32
+		Failed    uint32
+		Badblocks uint32
+		Bbtblocks uint32
+	}
+)
+
+const (
+	MTD_OPS_PLACE_OOB = 0x0
+	MTD_OPS_AUTO_OOB  = 0x1
+	MTD_OPS_RAW       = 0x2
+)
+
+const (
+	MTD_FILE_MODE_NORMAL      = 0x0
+	MTD_FILE_MODE_OTP_FACTORY = 0x1
+	MTD_FILE_MODE_OTP_USER    = 0x2
+	MTD_FILE_MODE_RAW         = 0x3
+)
+
+const (
+	NFC_CMD_UNSPEC                    = 0x0
+	NFC_CMD_GET_DEVICE                = 0x1
+	NFC_CMD_DEV_UP                    = 0x2
+	NFC_CMD_DEV_DOWN                  = 0x3
+	NFC_CMD_DEP_LINK_UP               = 0x4
+	NFC_CMD_DEP_LINK_DOWN             = 0x5
+	NFC_CMD_START_POLL                = 0x6
+	NFC_CMD_STOP_POLL                 = 0x7
+	NFC_CMD_GET_TARGET                = 0x8
+	NFC_EVENT_TARGETS_FOUND           = 0x9
+	NFC_EVENT_DEVICE_ADDED            = 0xa
+	NFC_EVENT_DEVICE_REMOVED          = 0xb
+	NFC_EVENT_TARGET_LOST             = 0xc
+	NFC_EVENT_TM_ACTIVATED            = 0xd
+	NFC_EVENT_TM_DEACTIVATED          = 0xe
+	NFC_CMD_LLC_GET_PARAMS            = 0xf
+	NFC_CMD_LLC_SET_PARAMS            = 0x10
+	NFC_CMD_ENABLE_SE                 = 0x11
+	NFC_CMD_DISABLE_SE                = 0x12
+	NFC_CMD_LLC_SDREQ                 = 0x13
+	NFC_EVENT_LLC_SDRES               = 0x14
+	NFC_CMD_FW_DOWNLOAD               = 0x15
+	NFC_EVENT_SE_ADDED                = 0x16
+	NFC_EVENT_SE_REMOVED              = 0x17
+	NFC_EVENT_SE_CONNECTIVITY         = 0x18
+	NFC_EVENT_SE_TRANSACTION          = 0x19
+	NFC_CMD_GET_SE                    = 0x1a
+	NFC_CMD_SE_IO                     = 0x1b
+	NFC_CMD_ACTIVATE_TARGET           = 0x1c
+	NFC_CMD_VENDOR                    = 0x1d
+	NFC_CMD_DEACTIVATE_TARGET         = 0x1e
+	NFC_ATTR_UNSPEC                   = 0x0
+	NFC_ATTR_DEVICE_INDEX             = 0x1
+	NFC_ATTR_DEVICE_NAME              = 0x2
+	NFC_ATTR_PROTOCOLS                = 0x3
+	NFC_ATTR_TARGET_INDEX             = 0x4
+	NFC_ATTR_TARGET_SENS_RES          = 0x5
+	NFC_ATTR_TARGET_SEL_RES           = 0x6
+	NFC_ATTR_TARGET_NFCID1            = 0x7
+	NFC_ATTR_TARGET_SENSB_RES         = 0x8
+	NFC_ATTR_TARGET_SENSF_RES         = 0x9
+	NFC_ATTR_COMM_MODE                = 0xa
+	NFC_ATTR_RF_MODE                  = 0xb
+	NFC_ATTR_DEVICE_POWERED           = 0xc
+	NFC_ATTR_IM_PROTOCOLS             = 0xd
+	NFC_ATTR_TM_PROTOCOLS             = 0xe
+	NFC_ATTR_LLC_PARAM_LTO            = 0xf
+	NFC_ATTR_LLC_PARAM_RW             = 0x10
+	NFC_ATTR_LLC_PARAM_MIUX           = 0x11
+	NFC_ATTR_SE                       = 0x12
+	NFC_ATTR_LLC_SDP                  = 0x13
+	NFC_ATTR_FIRMWARE_NAME            = 0x14
+	NFC_ATTR_SE_INDEX                 = 0x15
+	NFC_ATTR_SE_TYPE                  = 0x16
+	NFC_ATTR_SE_AID                   = 0x17
+	NFC_ATTR_FIRMWARE_DOWNLOAD_STATUS = 0x18
+	NFC_ATTR_SE_APDU                  = 0x19
+	NFC_ATTR_TARGET_ISO15693_DSFID    = 0x1a
+	NFC_ATTR_TARGET_ISO15693_UID      = 0x1b
+	NFC_ATTR_SE_PARAMS                = 0x1c
+	NFC_ATTR_VENDOR_ID                = 0x1d
+	NFC_ATTR_VENDOR_SUBCMD            = 0x1e
+	NFC_ATTR_VENDOR_DATA              = 0x1f
+	NFC_SDP_ATTR_UNSPEC               = 0x0
+	NFC_SDP_ATTR_URI                  = 0x1
+	NFC_SDP_ATTR_SAP                  = 0x2
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_386.go b/vendor/golang.org/x/sys/unix/ztypes_linux_386.go
index 73509d8..235c62e 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_386.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_386.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m32 linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m32 /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && linux
 // +build 386,linux
 
 package unix
@@ -127,6 +128,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint32
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -159,9 +171,10 @@
 }
 
 const (
-	SizeofIovec   = 0x8
-	SizeofMsghdr  = 0x1c
-	SizeofCmsghdr = 0xc
+	SizeofSockaddrNFCLLCP = 0x58
+	SizeofIovec           = 0x8
+	SizeofMsghdr          = 0x1c
+	SizeofCmsghdr         = 0xc
 )
 
 const (
@@ -602,3 +615,18 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+}
+
+const (
+	PPS_GETPARAMS = 0x800470a1
+	PPS_SETPARAMS = 0x400470a2
+	PPS_GETCAP    = 0x800470a3
+	PPS_FETCH     = 0xc00470a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_amd64.go
index 45eb873..99b1e5b 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_amd64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m64 linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -m64 /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && linux
 // +build amd64,linux
 
 package unix
@@ -129,6 +130,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -162,9 +174,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -619,3 +632,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x800870a1
+	PPS_SETPARAMS = 0x400870a2
+	PPS_GETCAP    = 0x800870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_arm.go b/vendor/golang.org/x/sys/unix/ztypes_linux_arm.go
index 8f6b453..cc8bba7 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_arm.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_arm.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && linux
 // +build arm,linux
 
 package unix
@@ -133,6 +134,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint32
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]uint8
@@ -165,9 +177,10 @@
 }
 
 const (
-	SizeofIovec   = 0x8
-	SizeofMsghdr  = 0x1c
-	SizeofCmsghdr = 0xc
+	SizeofSockaddrNFCLLCP = 0x58
+	SizeofIovec           = 0x8
+	SizeofMsghdr          = 0x1c
+	SizeofCmsghdr         = 0xc
 )
 
 const (
@@ -596,3 +609,19 @@
 	Peer uint32
 	Id   [16]uint8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x800470a1
+	PPS_SETPARAMS = 0x400470a2
+	PPS_GETCAP    = 0x800470a3
+	PPS_FETCH     = 0xc00470a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_arm64.go
index b1e0c24..fa8fe3a 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_arm64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_arm64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && linux
 // +build arm64,linux
 
 package unix
@@ -130,6 +131,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -163,9 +175,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -598,3 +611,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x800870a1
+	PPS_SETPARAMS = 0x400870a2
+	PPS_GETCAP    = 0x800870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_mips.go b/vendor/golang.org/x/sys/unix/ztypes_linux_mips.go
index fb802c3..e7fb8d9 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_mips.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_mips.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips && linux
 // +build mips,linux
 
 package unix
@@ -132,6 +133,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint32
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -164,9 +176,10 @@
 }
 
 const (
-	SizeofIovec   = 0x8
-	SizeofMsghdr  = 0x1c
-	SizeofCmsghdr = 0xc
+	SizeofSockaddrNFCLLCP = 0x58
+	SizeofIovec           = 0x8
+	SizeofMsghdr          = 0x1c
+	SizeofCmsghdr         = 0xc
 )
 
 const (
@@ -602,3 +615,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400470a1
+	PPS_SETPARAMS = 0x800470a2
+	PPS_GETCAP    = 0x400470a3
+	PPS_FETCH     = 0xc00470a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_mips64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_mips64.go
index 30abcf3..2fa61d5 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_mips64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_mips64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && linux
 // +build mips64,linux
 
 package unix
@@ -130,6 +131,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -163,9 +175,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -601,3 +614,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400870a1
+	PPS_SETPARAMS = 0x800870a2
+	PPS_GETCAP    = 0x400870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_mips64le.go b/vendor/golang.org/x/sys/unix/ztypes_linux_mips64le.go
index 99761aa..7f36399 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_mips64le.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_mips64le.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64le && linux
 // +build mips64le,linux
 
 package unix
@@ -130,6 +131,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -163,9 +175,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -601,3 +614,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400870a1
+	PPS_SETPARAMS = 0x800870a2
+	PPS_GETCAP    = 0x400870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_mipsle.go b/vendor/golang.org/x/sys/unix/ztypes_linux_mipsle.go
index 2936903..f3c20cb 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_mipsle.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_mipsle.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mipsle && linux
 // +build mipsle,linux
 
 package unix
@@ -132,6 +133,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint32
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -164,9 +176,10 @@
 }
 
 const (
-	SizeofIovec   = 0x8
-	SizeofMsghdr  = 0x1c
-	SizeofCmsghdr = 0xc
+	SizeofSockaddrNFCLLCP = 0x58
+	SizeofIovec           = 0x8
+	SizeofMsghdr          = 0x1c
+	SizeofCmsghdr         = 0xc
 )
 
 const (
@@ -602,3 +615,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400470a1
+	PPS_SETPARAMS = 0x800470a2
+	PPS_GETCAP    = 0x400470a3
+	PPS_FETCH     = 0xc00470a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_ppc.go b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc.go
new file mode 100644
index 0000000..885d279
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc.go
@@ -0,0 +1,639 @@
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+//go:build ppc && linux
+// +build ppc,linux
+
+package unix
+
+const (
+	SizeofPtr  = 0x4
+	SizeofLong = 0x4
+)
+
+type (
+	_C_long int32
+)
+
+type Timespec struct {
+	Sec  int32
+	Nsec int32
+}
+
+type Timeval struct {
+	Sec  int32
+	Usec int32
+}
+
+type Timex struct {
+	Modes     uint32
+	Offset    int32
+	Freq      int32
+	Maxerror  int32
+	Esterror  int32
+	Status    int32
+	Constant  int32
+	Precision int32
+	Tolerance int32
+	Time      Timeval
+	Tick      int32
+	Ppsfreq   int32
+	Jitter    int32
+	Shift     int32
+	Stabil    int32
+	Jitcnt    int32
+	Calcnt    int32
+	Errcnt    int32
+	Stbcnt    int32
+	Tai       int32
+	_         [44]byte
+}
+
+type Time_t int32
+
+type Tms struct {
+	Utime  int32
+	Stime  int32
+	Cutime int32
+	Cstime int32
+}
+
+type Utimbuf struct {
+	Actime  int32
+	Modtime int32
+}
+
+type Rusage struct {
+	Utime    Timeval
+	Stime    Timeval
+	Maxrss   int32
+	Ixrss    int32
+	Idrss    int32
+	Isrss    int32
+	Minflt   int32
+	Majflt   int32
+	Nswap    int32
+	Inblock  int32
+	Oublock  int32
+	Msgsnd   int32
+	Msgrcv   int32
+	Nsignals int32
+	Nvcsw    int32
+	Nivcsw   int32
+}
+
+type Stat_t struct {
+	Dev     uint64
+	Ino     uint64
+	Mode    uint32
+	Nlink   uint32
+	Uid     uint32
+	Gid     uint32
+	Rdev    uint64
+	_       uint16
+	_       [4]byte
+	Size    int64
+	Blksize int32
+	_       [4]byte
+	Blocks  int64
+	Atim    Timespec
+	Mtim    Timespec
+	Ctim    Timespec
+	_       uint32
+	_       uint32
+}
+
+type Dirent struct {
+	Ino    uint64
+	Off    int64
+	Reclen uint16
+	Type   uint8
+	Name   [256]uint8
+	_      [5]byte
+}
+
+type Flock_t struct {
+	Type   int16
+	Whence int16
+	_      [4]byte
+	Start  int64
+	Len    int64
+	Pid    int32
+	_      [4]byte
+}
+
+type DmNameList struct {
+	Dev  uint64
+	Next uint32
+	Name [0]byte
+	_    [4]byte
+}
+
+const (
+	FADV_DONTNEED = 0x4
+	FADV_NOREUSE  = 0x5
+)
+
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint32
+}
+
+type RawSockaddr struct {
+	Family uint16
+	Data   [14]uint8
+}
+
+type RawSockaddrAny struct {
+	Addr RawSockaddr
+	Pad  [96]uint8
+}
+
+type Iovec struct {
+	Base *byte
+	Len  uint32
+}
+
+type Msghdr struct {
+	Name       *byte
+	Namelen    uint32
+	Iov        *Iovec
+	Iovlen     uint32
+	Control    *byte
+	Controllen uint32
+	Flags      int32
+}
+
+type Cmsghdr struct {
+	Len   uint32
+	Level int32
+	Type  int32
+}
+
+const (
+	SizeofSockaddrNFCLLCP = 0x58
+	SizeofIovec           = 0x8
+	SizeofMsghdr          = 0x1c
+	SizeofCmsghdr         = 0xc
+)
+
+const (
+	SizeofSockFprog = 0x8
+)
+
+type PtraceRegs struct {
+	Gpr       [32]uint32
+	Nip       uint32
+	Msr       uint32
+	Orig_gpr3 uint32
+	Ctr       uint32
+	Link      uint32
+	Xer       uint32
+	Ccr       uint32
+	Mq        uint32
+	Trap      uint32
+	Dar       uint32
+	Dsisr     uint32
+	Result    uint32
+}
+
+type FdSet struct {
+	Bits [32]int32
+}
+
+type Sysinfo_t struct {
+	Uptime    int32
+	Loads     [3]uint32
+	Totalram  uint32
+	Freeram   uint32
+	Sharedram uint32
+	Bufferram uint32
+	Totalswap uint32
+	Freeswap  uint32
+	Procs     uint16
+	Pad       uint16
+	Totalhigh uint32
+	Freehigh  uint32
+	Unit      uint32
+	_         [8]uint8
+}
+
+type Ustat_t struct {
+	Tfree  int32
+	Tinode uint32
+	Fname  [6]uint8
+	Fpack  [6]uint8
+}
+
+type EpollEvent struct {
+	Events uint32
+	_      int32
+	Fd     int32
+	Pad    int32
+}
+
+const (
+	POLLRDHUP = 0x2000
+)
+
+type Sigset_t struct {
+	Val [32]uint32
+}
+
+const _C__NSIG = 0x41
+
+type Termios struct {
+	Iflag  uint32
+	Oflag  uint32
+	Cflag  uint32
+	Lflag  uint32
+	Cc     [19]uint8
+	Line   uint8
+	Ispeed uint32
+	Ospeed uint32
+}
+
+type Taskstats struct {
+	Version                   uint16
+	Ac_exitcode               uint32
+	Ac_flag                   uint8
+	Ac_nice                   uint8
+	_                         [4]byte
+	Cpu_count                 uint64
+	Cpu_delay_total           uint64
+	Blkio_count               uint64
+	Blkio_delay_total         uint64
+	Swapin_count              uint64
+	Swapin_delay_total        uint64
+	Cpu_run_real_total        uint64
+	Cpu_run_virtual_total     uint64
+	Ac_comm                   [32]uint8
+	Ac_sched                  uint8
+	Ac_pad                    [3]uint8
+	_                         [4]byte
+	Ac_uid                    uint32
+	Ac_gid                    uint32
+	Ac_pid                    uint32
+	Ac_ppid                   uint32
+	Ac_btime                  uint32
+	_                         [4]byte
+	Ac_etime                  uint64
+	Ac_utime                  uint64
+	Ac_stime                  uint64
+	Ac_minflt                 uint64
+	Ac_majflt                 uint64
+	Coremem                   uint64
+	Virtmem                   uint64
+	Hiwater_rss               uint64
+	Hiwater_vm                uint64
+	Read_char                 uint64
+	Write_char                uint64
+	Read_syscalls             uint64
+	Write_syscalls            uint64
+	Read_bytes                uint64
+	Write_bytes               uint64
+	Cancelled_write_bytes     uint64
+	Nvcsw                     uint64
+	Nivcsw                    uint64
+	Ac_utimescaled            uint64
+	Ac_stimescaled            uint64
+	Cpu_scaled_run_real_total uint64
+	Freepages_count           uint64
+	Freepages_delay_total     uint64
+	Thrashing_count           uint64
+	Thrashing_delay_total     uint64
+	Ac_btime64                uint64
+}
+
+type cpuMask uint32
+
+const (
+	_NCPUBITS = 0x20
+)
+
+const (
+	CBitFieldMaskBit0  = 0x8000000000000000
+	CBitFieldMaskBit1  = 0x4000000000000000
+	CBitFieldMaskBit2  = 0x2000000000000000
+	CBitFieldMaskBit3  = 0x1000000000000000
+	CBitFieldMaskBit4  = 0x800000000000000
+	CBitFieldMaskBit5  = 0x400000000000000
+	CBitFieldMaskBit6  = 0x200000000000000
+	CBitFieldMaskBit7  = 0x100000000000000
+	CBitFieldMaskBit8  = 0x80000000000000
+	CBitFieldMaskBit9  = 0x40000000000000
+	CBitFieldMaskBit10 = 0x20000000000000
+	CBitFieldMaskBit11 = 0x10000000000000
+	CBitFieldMaskBit12 = 0x8000000000000
+	CBitFieldMaskBit13 = 0x4000000000000
+	CBitFieldMaskBit14 = 0x2000000000000
+	CBitFieldMaskBit15 = 0x1000000000000
+	CBitFieldMaskBit16 = 0x800000000000
+	CBitFieldMaskBit17 = 0x400000000000
+	CBitFieldMaskBit18 = 0x200000000000
+	CBitFieldMaskBit19 = 0x100000000000
+	CBitFieldMaskBit20 = 0x80000000000
+	CBitFieldMaskBit21 = 0x40000000000
+	CBitFieldMaskBit22 = 0x20000000000
+	CBitFieldMaskBit23 = 0x10000000000
+	CBitFieldMaskBit24 = 0x8000000000
+	CBitFieldMaskBit25 = 0x4000000000
+	CBitFieldMaskBit26 = 0x2000000000
+	CBitFieldMaskBit27 = 0x1000000000
+	CBitFieldMaskBit28 = 0x800000000
+	CBitFieldMaskBit29 = 0x400000000
+	CBitFieldMaskBit30 = 0x200000000
+	CBitFieldMaskBit31 = 0x100000000
+	CBitFieldMaskBit32 = 0x80000000
+	CBitFieldMaskBit33 = 0x40000000
+	CBitFieldMaskBit34 = 0x20000000
+	CBitFieldMaskBit35 = 0x10000000
+	CBitFieldMaskBit36 = 0x8000000
+	CBitFieldMaskBit37 = 0x4000000
+	CBitFieldMaskBit38 = 0x2000000
+	CBitFieldMaskBit39 = 0x1000000
+	CBitFieldMaskBit40 = 0x800000
+	CBitFieldMaskBit41 = 0x400000
+	CBitFieldMaskBit42 = 0x200000
+	CBitFieldMaskBit43 = 0x100000
+	CBitFieldMaskBit44 = 0x80000
+	CBitFieldMaskBit45 = 0x40000
+	CBitFieldMaskBit46 = 0x20000
+	CBitFieldMaskBit47 = 0x10000
+	CBitFieldMaskBit48 = 0x8000
+	CBitFieldMaskBit49 = 0x4000
+	CBitFieldMaskBit50 = 0x2000
+	CBitFieldMaskBit51 = 0x1000
+	CBitFieldMaskBit52 = 0x800
+	CBitFieldMaskBit53 = 0x400
+	CBitFieldMaskBit54 = 0x200
+	CBitFieldMaskBit55 = 0x100
+	CBitFieldMaskBit56 = 0x80
+	CBitFieldMaskBit57 = 0x40
+	CBitFieldMaskBit58 = 0x20
+	CBitFieldMaskBit59 = 0x10
+	CBitFieldMaskBit60 = 0x8
+	CBitFieldMaskBit61 = 0x4
+	CBitFieldMaskBit62 = 0x2
+	CBitFieldMaskBit63 = 0x1
+)
+
+type SockaddrStorage struct {
+	Family uint16
+	_      [122]uint8
+	_      uint32
+}
+
+type HDGeometry struct {
+	Heads     uint8
+	Sectors   uint8
+	Cylinders uint16
+	Start     uint32
+}
+
+type Statfs_t struct {
+	Type    int32
+	Bsize   int32
+	Blocks  uint64
+	Bfree   uint64
+	Bavail  uint64
+	Files   uint64
+	Ffree   uint64
+	Fsid    Fsid
+	Namelen int32
+	Frsize  int32
+	Flags   int32
+	Spare   [4]int32
+	_       [4]byte
+}
+
+type TpacketHdr struct {
+	Status  uint32
+	Len     uint32
+	Snaplen uint32
+	Mac     uint16
+	Net     uint16
+	Sec     uint32
+	Usec    uint32
+}
+
+const (
+	SizeofTpacketHdr = 0x18
+)
+
+type RTCPLLInfo struct {
+	Ctrl    int32
+	Value   int32
+	Max     int32
+	Min     int32
+	Posmult int32
+	Negmult int32
+	Clock   int32
+}
+
+type BlkpgPartition struct {
+	Start   int64
+	Length  int64
+	Pno     int32
+	Devname [64]uint8
+	Volname [64]uint8
+	_       [4]byte
+}
+
+const (
+	BLKPG = 0x20001269
+)
+
+type XDPUmemReg struct {
+	Addr     uint64
+	Len      uint64
+	Size     uint32
+	Headroom uint32
+	Flags    uint32
+	_        [4]byte
+}
+
+type CryptoUserAlg struct {
+	Name        [64]uint8
+	Driver_name [64]uint8
+	Module_name [64]uint8
+	Type        uint32
+	Mask        uint32
+	Refcnt      uint32
+	Flags       uint32
+}
+
+type CryptoStatAEAD struct {
+	Type         [64]uint8
+	Encrypt_cnt  uint64
+	Encrypt_tlen uint64
+	Decrypt_cnt  uint64
+	Decrypt_tlen uint64
+	Err_cnt      uint64
+}
+
+type CryptoStatAKCipher struct {
+	Type         [64]uint8
+	Encrypt_cnt  uint64
+	Encrypt_tlen uint64
+	Decrypt_cnt  uint64
+	Decrypt_tlen uint64
+	Verify_cnt   uint64
+	Sign_cnt     uint64
+	Err_cnt      uint64
+}
+
+type CryptoStatCipher struct {
+	Type         [64]uint8
+	Encrypt_cnt  uint64
+	Encrypt_tlen uint64
+	Decrypt_cnt  uint64
+	Decrypt_tlen uint64
+	Err_cnt      uint64
+}
+
+type CryptoStatCompress struct {
+	Type            [64]uint8
+	Compress_cnt    uint64
+	Compress_tlen   uint64
+	Decompress_cnt  uint64
+	Decompress_tlen uint64
+	Err_cnt         uint64
+}
+
+type CryptoStatHash struct {
+	Type      [64]uint8
+	Hash_cnt  uint64
+	Hash_tlen uint64
+	Err_cnt   uint64
+}
+
+type CryptoStatKPP struct {
+	Type                      [64]uint8
+	Setsecret_cnt             uint64
+	Generate_public_key_cnt   uint64
+	Compute_shared_secret_cnt uint64
+	Err_cnt                   uint64
+}
+
+type CryptoStatRNG struct {
+	Type          [64]uint8
+	Generate_cnt  uint64
+	Generate_tlen uint64
+	Seed_cnt      uint64
+	Err_cnt       uint64
+}
+
+type CryptoStatLarval struct {
+	Type [64]uint8
+}
+
+type CryptoReportLarval struct {
+	Type [64]uint8
+}
+
+type CryptoReportHash struct {
+	Type       [64]uint8
+	Blocksize  uint32
+	Digestsize uint32
+}
+
+type CryptoReportCipher struct {
+	Type        [64]uint8
+	Blocksize   uint32
+	Min_keysize uint32
+	Max_keysize uint32
+}
+
+type CryptoReportBlkCipher struct {
+	Type        [64]uint8
+	Geniv       [64]uint8
+	Blocksize   uint32
+	Min_keysize uint32
+	Max_keysize uint32
+	Ivsize      uint32
+}
+
+type CryptoReportAEAD struct {
+	Type        [64]uint8
+	Geniv       [64]uint8
+	Blocksize   uint32
+	Maxauthsize uint32
+	Ivsize      uint32
+}
+
+type CryptoReportComp struct {
+	Type [64]uint8
+}
+
+type CryptoReportRNG struct {
+	Type     [64]uint8
+	Seedsize uint32
+}
+
+type CryptoReportAKCipher struct {
+	Type [64]uint8
+}
+
+type CryptoReportKPP struct {
+	Type [64]uint8
+}
+
+type CryptoReportAcomp struct {
+	Type [64]uint8
+}
+
+type LoopInfo struct {
+	Number           int32
+	Device           uint32
+	Inode            uint32
+	Rdevice          uint32
+	Offset           int32
+	Encrypt_type     int32
+	Encrypt_key_size int32
+	Flags            int32
+	Name             [64]uint8
+	Encrypt_key      [32]uint8
+	Init             [2]uint32
+	Reserved         [4]uint8
+}
+
+type TIPCSubscr struct {
+	Seq     TIPCServiceRange
+	Timeout uint32
+	Filter  uint32
+	Handle  [8]uint8
+}
+
+type TIPCSIOCLNReq struct {
+	Peer     uint32
+	Id       uint32
+	Linkname [68]uint8
+}
+
+type TIPCSIOCNodeIDReq struct {
+	Peer uint32
+	Id   [16]uint8
+}
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400470a1
+	PPS_SETPARAMS = 0x800470a2
+	PPS_GETCAP    = 0x400470a3
+	PPS_FETCH     = 0xc00470a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64.go
index 0ca856e..a94eb8e 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64 && linux
 // +build ppc64,linux
 
 package unix
@@ -131,6 +132,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]uint8
@@ -164,9 +176,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -608,3 +621,19 @@
 	Peer uint32
 	Id   [16]uint8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400870a1
+	PPS_SETPARAMS = 0x800870a2
+	PPS_GETCAP    = 0x400870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64le.go b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64le.go
index f50f648..659e32e 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64le.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_ppc64le.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build ppc64le && linux
 // +build ppc64le,linux
 
 package unix
@@ -131,6 +132,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]uint8
@@ -164,9 +176,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -608,3 +621,19 @@
 	Peer uint32
 	Id   [16]uint8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400870a1
+	PPS_SETPARAMS = 0x800870a2
+	PPS_GETCAP    = 0x400870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_riscv64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_riscv64.go
index 4d3ac8d..ab8ec60 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_riscv64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_riscv64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build riscv64 && linux
 // +build riscv64,linux
 
 package unix
@@ -130,6 +131,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]uint8
@@ -163,9 +175,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -626,3 +639,19 @@
 	Peer uint32
 	Id   [16]uint8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x800870a1
+	PPS_SETPARAMS = 0x400870a2
+	PPS_GETCAP    = 0x800870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_s390x.go b/vendor/golang.org/x/sys/unix/ztypes_linux_s390x.go
index 349f483..3ec0823 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_s390x.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_s390x.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include -fsigned-char /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build s390x && linux
 // +build s390x,linux
 
 package unix
@@ -129,6 +130,17 @@
 	FADV_NOREUSE  = 0x7
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -162,9 +174,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -622,3 +635,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x800870a1
+	PPS_SETPARAMS = 0x400870a2
+	PPS_GETCAP    = 0x800870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_linux_sparc64.go b/vendor/golang.org/x/sys/unix/ztypes_linux_sparc64.go
index 80c73be..23d4744 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_linux_sparc64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_linux_sparc64.go
@@ -1,6 +1,7 @@
-// cgo -godefs -- -Wall -Werror -static -I/tmp/include linux/types.go | go run mkpost.go
+// cgo -godefs -- -Wall -Werror -static -I/tmp/include /build/linux/types.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build sparc64 && linux
 // +build sparc64,linux
 
 package unix
@@ -133,6 +134,17 @@
 	FADV_NOREUSE  = 0x5
 )
 
+type RawSockaddrNFCLLCP struct {
+	Sa_family        uint16
+	Dev_idx          uint32
+	Target_idx       uint32
+	Nfc_protocol     uint32
+	Dsap             uint8
+	Ssap             uint8
+	Service_name     [63]uint8
+	Service_name_len uint64
+}
+
 type RawSockaddr struct {
 	Family uint16
 	Data   [14]int8
@@ -166,9 +178,10 @@
 }
 
 const (
-	SizeofIovec   = 0x10
-	SizeofMsghdr  = 0x38
-	SizeofCmsghdr = 0x10
+	SizeofSockaddrNFCLLCP = 0x60
+	SizeofIovec           = 0x10
+	SizeofMsghdr          = 0x38
+	SizeofCmsghdr         = 0x10
 )
 
 const (
@@ -603,3 +616,19 @@
 	Peer uint32
 	Id   [16]int8
 }
+
+type PPSKInfo struct {
+	Assert_sequence uint32
+	Clear_sequence  uint32
+	Assert_tu       PPSKTime
+	Clear_tu        PPSKTime
+	Current_mode    int32
+	_               [4]byte
+}
+
+const (
+	PPS_GETPARAMS = 0x400870a1
+	PPS_SETPARAMS = 0x800870a2
+	PPS_GETCAP    = 0x400870a3
+	PPS_FETCH     = 0xc00870a4
+)
diff --git a/vendor/golang.org/x/sys/unix/ztypes_netbsd_386.go b/vendor/golang.org/x/sys/unix/ztypes_netbsd_386.go
index a89100c..2fd2060 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_netbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_netbsd_386.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_netbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && netbsd
 // +build 386,netbsd
 
 package unix
@@ -248,6 +249,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x1c
@@ -443,8 +445,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_netbsd_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_netbsd_amd64.go
index 289184e..6a5a1a8 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_netbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_netbsd_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_netbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && netbsd
 // +build amd64,netbsd
 
 package unix
@@ -255,6 +256,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -451,8 +453,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm.go b/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm.go
index 428c450..84cc8d0 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_netbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && netbsd
 // +build arm,netbsd
 
 package unix
@@ -253,6 +254,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x1c
@@ -448,8 +450,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm64.go
index 6f1f284..c844e70 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_netbsd_arm64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_netbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && netbsd
 // +build arm64,netbsd
 
 package unix
@@ -255,6 +256,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x14
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -451,8 +453,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x400
+	AT_EACCESS          = 0x100
 	AT_SYMLINK_NOFOLLOW = 0x200
+	AT_SYMLINK_FOLLOW   = 0x400
+	AT_REMOVEDIR        = 0x800
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_openbsd_386.go b/vendor/golang.org/x/sys/unix/ztypes_openbsd_386.go
index 61ea001..2a8b1e6 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_openbsd_386.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_openbsd_386.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_openbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build 386 && openbsd
 // +build 386,openbsd
 
 package unix
@@ -231,6 +232,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x20
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x1c
@@ -436,8 +438,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x4
+	AT_EACCESS          = 0x1
 	AT_SYMLINK_NOFOLLOW = 0x2
+	AT_SYMLINK_FOLLOW   = 0x4
+	AT_REMOVEDIR        = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_openbsd_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_openbsd_amd64.go
index 87a493f..b1759cf 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_openbsd_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_openbsd_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_openbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && openbsd
 // +build amd64,openbsd
 
 package unix
@@ -235,6 +236,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x20
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -436,8 +438,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x4
+	AT_EACCESS          = 0x1
 	AT_SYMLINK_NOFOLLOW = 0x2
+	AT_SYMLINK_FOLLOW   = 0x4
+	AT_REMOVEDIR        = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm.go b/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm.go
index d80836e..e807de2 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm.go
@@ -1,6 +1,7 @@
 // cgo -godefs -- -fsigned-char types_openbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm && openbsd
 // +build arm,openbsd
 
 package unix
@@ -235,6 +236,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x20
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x8
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x1c
@@ -437,8 +439,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x4
+	AT_EACCESS          = 0x1
 	AT_SYMLINK_NOFOLLOW = 0x2
+	AT_SYMLINK_FOLLOW   = 0x4
+	AT_REMOVEDIR        = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm64.go b/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm64.go
index 4e15874..ff3aeca 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_openbsd_arm64.go
@@ -1,6 +1,7 @@
 // cgo -godefs -- -fsigned-char types_openbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build arm64 && openbsd
 // +build arm64,openbsd
 
 package unix
@@ -231,6 +232,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x20
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -430,8 +432,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x4
+	AT_EACCESS          = 0x1
 	AT_SYMLINK_NOFOLLOW = 0x2
+	AT_SYMLINK_FOLLOW   = 0x4
+	AT_REMOVEDIR        = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_openbsd_mips64.go b/vendor/golang.org/x/sys/unix/ztypes_openbsd_mips64.go
index 992a1f8..9ecda69 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_openbsd_mips64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_openbsd_mips64.go
@@ -1,6 +1,7 @@
 // cgo -godefs -- -fsigned-char types_openbsd.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build mips64 && openbsd
 // +build mips64,openbsd
 
 package unix
@@ -231,6 +232,7 @@
 	SizeofSockaddrUnix     = 0x6a
 	SizeofSockaddrDatalink = 0x20
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -430,8 +432,10 @@
 
 const (
 	AT_FDCWD            = -0x64
-	AT_SYMLINK_FOLLOW   = 0x4
+	AT_EACCESS          = 0x1
 	AT_SYMLINK_NOFOLLOW = 0x2
+	AT_SYMLINK_FOLLOW   = 0x4
+	AT_REMOVEDIR        = 0x8
 )
 
 type PollFd struct {
diff --git a/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go b/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
index 23ed9fe..85effef 100644
--- a/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
+++ b/vendor/golang.org/x/sys/unix/ztypes_solaris_amd64.go
@@ -1,6 +1,7 @@
 // cgo -godefs types_solaris.go | go run mkpost.go
 // Code generated by the command above; see README.md. DO NOT EDIT.
 
+//go:build amd64 && solaris
 // +build amd64,solaris
 
 package unix
@@ -88,7 +89,6 @@
 	Mtim    Timespec
 	Ctim    Timespec
 	Blksize int32
-	_       [4]byte
 	Blocks  int64
 	Fstype  [16]int8
 }
@@ -96,7 +96,6 @@
 type Flock_t struct {
 	Type   int16
 	Whence int16
-	_      [4]byte
 	Start  int64
 	Len    int64
 	Sysid  int32
@@ -138,12 +137,12 @@
 }
 
 type RawSockaddrInet6 struct {
-	Family         uint16
-	Port           uint16
-	Flowinfo       uint32
-	Addr           [16]byte /* in6_addr */
-	Scope_id       uint32
-	X__sin6_src_id uint32
+	Family   uint16
+	Port     uint16
+	Flowinfo uint32
+	Addr     [16]byte /* in6_addr */
+	Scope_id uint32
+	_        uint32
 }
 
 type RawSockaddrUnix struct {
@@ -196,10 +195,8 @@
 type Msghdr struct {
 	Name         *byte
 	Namelen      uint32
-	_            [4]byte
 	Iov          *Iovec
 	Iovlen       int32
-	_            [4]byte
 	Accrights    *int8
 	Accrightslen int32
 	_            [4]byte
@@ -228,7 +225,7 @@
 }
 
 type ICMPv6Filter struct {
-	X__icmp6_filt [8]uint32
+	Filt [8]uint32
 }
 
 const (
@@ -238,6 +235,7 @@
 	SizeofSockaddrUnix     = 0x6e
 	SizeofSockaddrDatalink = 0xfc
 	SizeofLinger           = 0x8
+	SizeofIovec            = 0x10
 	SizeofIPMreq           = 0x8
 	SizeofIPv6Mreq         = 0x14
 	SizeofMsghdr           = 0x30
@@ -291,7 +289,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Data    IfData
 }
 
@@ -299,7 +296,6 @@
 	Type       uint8
 	Addrlen    uint8
 	Hdrlen     uint8
-	_          [1]byte
 	Mtu        uint32
 	Metric     uint32
 	Baudrate   uint32
@@ -324,7 +320,6 @@
 	Addrs   int32
 	Flags   int32
 	Index   uint16
-	_       [2]byte
 	Metric  int32
 }
 
@@ -333,7 +328,6 @@
 	Version uint8
 	Type    uint8
 	Index   uint16
-	_       [2]byte
 	Flags   int32
 	Addrs   int32
 	Pid     int32
@@ -371,15 +365,14 @@
 }
 
 type BpfStat struct {
-	Recv    uint64
-	Drop    uint64
-	Capt    uint64
-	Padding [13]uint64
+	Recv uint64
+	Drop uint64
+	Capt uint64
+	_    [13]uint64
 }
 
 type BpfProgram struct {
 	Len   uint32
-	_     [4]byte
 	Insns *BpfInsn
 }
 
diff --git a/vendor/golang.org/x/sys/unix/ztypes_zos_s390x.go b/vendor/golang.org/x/sys/unix/ztypes_zos_s390x.go
new file mode 100644
index 0000000..4ab638c
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/ztypes_zos_s390x.go
@@ -0,0 +1,406 @@
+// Copyright 2020 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.
+
+//go:build zos && s390x
+// +build zos,s390x
+
+// Hand edited based on ztypes_linux_s390x.go
+// TODO: auto-generate.
+
+package unix
+
+const (
+	SizeofPtr      = 0x8
+	SizeofShort    = 0x2
+	SizeofInt      = 0x4
+	SizeofLong     = 0x8
+	SizeofLongLong = 0x8
+	PathMax        = 0x1000
+)
+
+const (
+	SizeofSockaddrAny   = 128
+	SizeofCmsghdr       = 12
+	SizeofIPMreq        = 8
+	SizeofIPv6Mreq      = 20
+	SizeofICMPv6Filter  = 32
+	SizeofIPv6MTUInfo   = 32
+	SizeofLinger        = 8
+	SizeofSockaddrInet4 = 16
+	SizeofSockaddrInet6 = 28
+	SizeofTCPInfo       = 0x68
+)
+
+type (
+	_C_short     int16
+	_C_int       int32
+	_C_long      int64
+	_C_long_long int64
+)
+
+type Timespec struct {
+	Sec  int64
+	Nsec int64
+}
+
+type Timeval struct {
+	Sec  int64
+	Usec int64
+}
+
+type timeval_zos struct { //correct (with padding and all)
+	Sec  int64
+	_    [4]byte // pad
+	Usec int32
+}
+
+type Tms struct { //clock_t is 4-byte unsigned int in zos
+	Utime  uint32
+	Stime  uint32
+	Cutime uint32
+	Cstime uint32
+}
+
+type Time_t int64
+
+type Utimbuf struct {
+	Actime  int64
+	Modtime int64
+}
+
+type Utsname struct {
+	Sysname    [65]byte
+	Nodename   [65]byte
+	Release    [65]byte
+	Version    [65]byte
+	Machine    [65]byte
+	Domainname [65]byte
+}
+
+type RawSockaddrInet4 struct {
+	Len    uint8
+	Family uint8
+	Port   uint16
+	Addr   [4]byte /* in_addr */
+	Zero   [8]uint8
+}
+
+type RawSockaddrInet6 struct {
+	Len      uint8
+	Family   uint8
+	Port     uint16
+	Flowinfo uint32
+	Addr     [16]byte /* in6_addr */
+	Scope_id uint32
+}
+
+type RawSockaddrUnix struct {
+	Len    uint8
+	Family uint8
+	Path   [108]int8
+}
+
+type RawSockaddr struct {
+	Len    uint8
+	Family uint8
+	Data   [14]uint8
+}
+
+type RawSockaddrAny struct {
+	Addr RawSockaddr
+	_    [112]uint8 // pad
+}
+
+type _Socklen uint32
+
+type Linger struct {
+	Onoff  int32
+	Linger int32
+}
+
+type Iovec struct {
+	Base *byte
+	Len  uint64
+}
+
+type IPMreq struct {
+	Multiaddr [4]byte /* in_addr */
+	Interface [4]byte /* in_addr */
+}
+
+type IPv6Mreq struct {
+	Multiaddr [16]byte /* in6_addr */
+	Interface uint32
+}
+
+type Msghdr struct {
+	Name       *byte
+	Iov        *Iovec
+	Control    *byte
+	Flags      int32
+	Namelen    int32
+	Iovlen     int32
+	Controllen int32
+}
+
+type Cmsghdr struct {
+	Len   int32
+	Level int32
+	Type  int32
+}
+
+type Inet4Pktinfo struct {
+	Addr    [4]byte /* in_addr */
+	Ifindex uint32
+}
+
+type Inet6Pktinfo struct {
+	Addr    [16]byte /* in6_addr */
+	Ifindex uint32
+}
+
+type IPv6MTUInfo struct {
+	Addr RawSockaddrInet6
+	Mtu  uint32
+}
+
+type ICMPv6Filter struct {
+	Data [8]uint32
+}
+
+type TCPInfo struct {
+	State          uint8
+	Ca_state       uint8
+	Retransmits    uint8
+	Probes         uint8
+	Backoff        uint8
+	Options        uint8
+	Rto            uint32
+	Ato            uint32
+	Snd_mss        uint32
+	Rcv_mss        uint32
+	Unacked        uint32
+	Sacked         uint32
+	Lost           uint32
+	Retrans        uint32
+	Fackets        uint32
+	Last_data_sent uint32
+	Last_ack_sent  uint32
+	Last_data_recv uint32
+	Last_ack_recv  uint32
+	Pmtu           uint32
+	Rcv_ssthresh   uint32
+	Rtt            uint32
+	Rttvar         uint32
+	Snd_ssthresh   uint32
+	Snd_cwnd       uint32
+	Advmss         uint32
+	Reordering     uint32
+	Rcv_rtt        uint32
+	Rcv_space      uint32
+	Total_retrans  uint32
+}
+
+type _Gid_t uint32
+
+type rusage_zos struct {
+	Utime timeval_zos
+	Stime timeval_zos
+}
+
+type Rusage struct {
+	Utime    Timeval
+	Stime    Timeval
+	Maxrss   int64
+	Ixrss    int64
+	Idrss    int64
+	Isrss    int64
+	Minflt   int64
+	Majflt   int64
+	Nswap    int64
+	Inblock  int64
+	Oublock  int64
+	Msgsnd   int64
+	Msgrcv   int64
+	Nsignals int64
+	Nvcsw    int64
+	Nivcsw   int64
+}
+
+type Rlimit struct {
+	Cur uint64
+	Max uint64
+}
+
+// { int, short, short } in poll.h
+type PollFd struct {
+	Fd      int32
+	Events  int16
+	Revents int16
+}
+
+type Stat_t struct { //Linux Definition
+	Dev     uint64
+	Ino     uint64
+	Nlink   uint64
+	Mode    uint32
+	Uid     uint32
+	Gid     uint32
+	_       int32
+	Rdev    uint64
+	Size    int64
+	Atim    Timespec
+	Mtim    Timespec
+	Ctim    Timespec
+	Blksize int64
+	Blocks  int64
+	_       [3]int64
+}
+
+type Stat_LE_t struct {
+	_            [4]byte // eye catcher
+	Length       uint16
+	Version      uint16
+	Mode         int32
+	Ino          uint32
+	Dev          uint32
+	Nlink        int32
+	Uid          int32
+	Gid          int32
+	Size         int64
+	Atim31       [4]byte
+	Mtim31       [4]byte
+	Ctim31       [4]byte
+	Rdev         uint32
+	Auditoraudit uint32
+	Useraudit    uint32
+	Blksize      int32
+	Creatim31    [4]byte
+	AuditID      [16]byte
+	_            [4]byte // rsrvd1
+	File_tag     struct {
+		Ccsid   uint16
+		Txtflag uint16 // aggregating Txflag:1 deferred:1 rsvflags:14
+	}
+	CharsetID [8]byte
+	Blocks    int64
+	Genvalue  uint32
+	Reftim31  [4]byte
+	Fid       [8]byte
+	Filefmt   byte
+	Fspflag2  byte
+	_         [2]byte // rsrvd2
+	Ctimemsec int32
+	Seclabel  [8]byte
+	_         [4]byte // rsrvd3
+	_         [4]byte // rsrvd4
+	Atim      Time_t
+	Mtim      Time_t
+	Ctim      Time_t
+	Creatim   Time_t
+	Reftim    Time_t
+	_         [24]byte // rsrvd5
+}
+
+type Statvfs_t struct {
+	ID          [4]byte
+	Len         int32
+	Bsize       uint64
+	Blocks      uint64
+	Usedspace   uint64
+	Bavail      uint64
+	Flag        uint64
+	Maxfilesize int64
+	_           [16]byte
+	Frsize      uint64
+	Bfree       uint64
+	Files       uint32
+	Ffree       uint32
+	Favail      uint32
+	Namemax31   uint32
+	Invarsec    uint32
+	_           [4]byte
+	Fsid        uint64
+	Namemax     uint64
+}
+
+type Statfs_t struct {
+	Type    uint32
+	Bsize   uint64
+	Blocks  uint64
+	Bfree   uint64
+	Bavail  uint64
+	Files   uint32
+	Ffree   uint32
+	Fsid    uint64
+	Namelen uint64
+	Frsize  uint64
+	Flags   uint64
+}
+
+type Dirent struct {
+	Reclen uint16
+	Namlen uint16
+	Ino    uint32
+	Extra  uintptr
+	Name   [256]byte
+}
+
+type FdSet struct {
+	Bits [64]int32
+}
+
+// This struct is packed on z/OS so it can't be used directly.
+type Flock_t struct {
+	Type   int16
+	Whence int16
+	Start  int64
+	Len    int64
+	Pid    int32
+}
+
+type Termios struct {
+	Cflag uint32
+	Iflag uint32
+	Lflag uint32
+	Oflag uint32
+	Cc    [11]uint8
+}
+
+type Winsize struct {
+	Row    uint16
+	Col    uint16
+	Xpixel uint16
+	Ypixel uint16
+}
+
+type W_Mnth struct {
+	Hid   [4]byte
+	Size  int32
+	Cur1  int32 //32bit pointer
+	Cur2  int32 //^
+	Devno uint32
+	_     [4]byte
+}
+
+type W_Mntent struct {
+	Fstype       uint32
+	Mode         uint32
+	Dev          uint32
+	Parentdev    uint32
+	Rootino      uint32
+	Status       byte
+	Ddname       [9]byte
+	Fstname      [9]byte
+	Fsname       [45]byte
+	Pathlen      uint32
+	Mountpoint   [1024]byte
+	Jobname      [8]byte
+	PID          int32
+	Parmoffset   int32
+	Parmlen      int16
+	Owner        [8]byte
+	Quiesceowner [8]byte
+	_            [38]byte
+}
diff --git a/vendor/golang.org/x/text/secure/bidirule/bidirule10.0.0.go b/vendor/golang.org/x/text/secure/bidirule/bidirule10.0.0.go
index e4c6228..8a7392c 100644
--- a/vendor/golang.org/x/text/secure/bidirule/bidirule10.0.0.go
+++ b/vendor/golang.org/x/text/secure/bidirule/bidirule10.0.0.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build go1.10
 // +build go1.10
 
 package bidirule
diff --git a/vendor/golang.org/x/text/secure/bidirule/bidirule9.0.0.go b/vendor/golang.org/x/text/secure/bidirule/bidirule9.0.0.go
index 02b9e1e..bb0a920 100644
--- a/vendor/golang.org/x/text/secure/bidirule/bidirule9.0.0.go
+++ b/vendor/golang.org/x/text/secure/bidirule/bidirule9.0.0.go
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
+//go:build !go1.10
 // +build !go1.10
 
 package bidirule
diff --git a/vendor/golang.org/x/text/unicode/bidi/bidi.go b/vendor/golang.org/x/text/unicode/bidi/bidi.go
index e8edc54..fd05760 100644
--- a/vendor/golang.org/x/text/unicode/bidi/bidi.go
+++ b/vendor/golang.org/x/text/unicode/bidi/bidi.go
@@ -12,15 +12,14 @@
 // and without notice.
 package bidi // import "golang.org/x/text/unicode/bidi"
 
-// TODO:
-// The following functionality would not be hard to implement, but hinges on
-// the definition of a Segmenter interface. For now this is up to the user.
-// - Iterate over paragraphs
-// - Segmenter to iterate over runs directly from a given text.
-// Also:
+// TODO
 // - Transformer for reordering?
 // - Transformer (validator, really) for Bidi Rule.
 
+import (
+	"bytes"
+)
+
 // This API tries to avoid dealing with embedding levels for now. Under the hood
 // these will be computed, but the question is to which extent the user should
 // know they exist. We should at some point allow the user to specify an
@@ -49,7 +48,9 @@
 	Neutral
 )
 
-type options struct{}
+type options struct {
+	defaultDirection Direction
+}
 
 // An Option is an option for Bidi processing.
 type Option func(*options)
@@ -66,12 +67,62 @@
 // DefaultDirection sets the default direction for a Paragraph. The direction is
 // overridden if the text contains directional characters.
 func DefaultDirection(d Direction) Option {
-	panic("unimplemented")
+	return func(opts *options) {
+		opts.defaultDirection = d
+	}
 }
 
 // A Paragraph holds a single Paragraph for Bidi processing.
 type Paragraph struct {
-	// buffers
+	p          []byte
+	o          Ordering
+	opts       []Option
+	types      []Class
+	pairTypes  []bracketType
+	pairValues []rune
+	runes      []rune
+	options    options
+}
+
+// Initialize the p.pairTypes, p.pairValues and p.types from the input previously
+// set by p.SetBytes() or p.SetString(). Also limit the input up to (and including) a paragraph
+// separator (bidi class B).
+//
+// The function p.Order() needs these values to be set, so this preparation could be postponed.
+// But since the SetBytes and SetStrings functions return the length of the input up to the paragraph
+// separator, the whole input needs to be processed anyway and should not be done twice.
+//
+// The function has the same return values as SetBytes() / SetString()
+func (p *Paragraph) prepareInput() (n int, err error) {
+	p.runes = bytes.Runes(p.p)
+	bytecount := 0
+	// clear slices from previous SetString or SetBytes
+	p.pairTypes = nil
+	p.pairValues = nil
+	p.types = nil
+
+	for _, r := range p.runes {
+		props, i := LookupRune(r)
+		bytecount += i
+		cls := props.Class()
+		if cls == B {
+			return bytecount, nil
+		}
+		p.types = append(p.types, cls)
+		if props.IsOpeningBracket() {
+			p.pairTypes = append(p.pairTypes, bpOpen)
+			p.pairValues = append(p.pairValues, r)
+		} else if props.IsBracket() {
+			// this must be a closing bracket,
+			// since IsOpeningBracket is not true
+			p.pairTypes = append(p.pairTypes, bpClose)
+			p.pairValues = append(p.pairValues, r)
+		} else {
+			p.pairTypes = append(p.pairTypes, bpNone)
+			p.pairValues = append(p.pairValues, 0)
+		}
+	}
+	return bytecount, nil
 }
 
 // SetBytes configures p for the given paragraph text. It replaces text
@@ -80,70 +131,150 @@
 // consumed from b including this separator. Error may be non-nil if options are
 // given.
 func (p *Paragraph) SetBytes(b []byte, opts ...Option) (n int, err error) {
-	panic("unimplemented")
+	p.p = b
+	p.opts = opts
+	return p.prepareInput()
 }
 
-// SetString configures p for the given paragraph text. It replaces text
-// previously set by SetBytes or SetString. If b contains a paragraph separator
+// SetString configures s for the given paragraph text. It replaces text
+// previously set by SetBytes or SetString. If s contains a paragraph separator
 // it will only process the first paragraph and report the number of bytes
-// consumed from b including this separator. Error may be non-nil if options are
+// consumed from s including this separator. Error may be non-nil if options are
 // given.
 func (p *Paragraph) SetString(s string, opts ...Option) (n int, err error) {
-	panic("unimplemented")
+	p.p = []byte(s)
+	p.opts = opts
+	return p.prepareInput()
 }
 
 // IsLeftToRight reports whether the principle direction of rendering for this
 // paragraphs is left-to-right. If this returns false, the principle direction
 // of rendering is right-to-left.
 func (p *Paragraph) IsLeftToRight() bool {
-	panic("unimplemented")
+	return p.Direction() == LeftToRight
 }
 
 // Direction returns the direction of the text of this paragraph.
 //
 // The direction may be LeftToRight, RightToLeft, Mixed, or Neutral.
 func (p *Paragraph) Direction() Direction {
-	panic("unimplemented")
+	return p.o.Direction()
 }
 
+// TODO: what happens if the position is > len(input)? This should return an error.
+
 // RunAt reports the Run at the given position of the input text.
 //
 // This method can be used for computing line breaks on paragraphs.
 func (p *Paragraph) RunAt(pos int) Run {
-	panic("unimplemented")
+	c := 0
+	runNumber := 0
+	for i, r := range p.o.runes {
+		c += len(r)
+		if pos < c {
+			runNumber = i
+		}
+	}
+	return p.o.Run(runNumber)
+}
+
+func calculateOrdering(levels []level, runes []rune) Ordering {
+	var curDir Direction
+
+	prevDir := Neutral
+	prevI := 0
+
+	o := Ordering{}
+	// lvl = 0,2,4,...: left to right
+	// lvl = 1,3,5,...: right to left
+	for i, lvl := range levels {
+		if lvl%2 == 0 {
+			curDir = LeftToRight
+		} else {
+			curDir = RightToLeft
+		}
+		if curDir != prevDir {
+			if i > 0 {
+				o.runes = append(o.runes, runes[prevI:i])
+				o.directions = append(o.directions, prevDir)
+				o.startpos = append(o.startpos, prevI)
+			}
+			prevI = i
+			prevDir = curDir
+		}
+	}
+	o.runes = append(o.runes, runes[prevI:])
+	o.directions = append(o.directions, prevDir)
+	o.startpos = append(o.startpos, prevI)
+	return o
 }
 
 // Order computes the visual ordering of all the runs in a Paragraph.
 func (p *Paragraph) Order() (Ordering, error) {
-	panic("unimplemented")
+	if len(p.types) == 0 {
+		return Ordering{}, nil
+	}
+
+	for _, fn := range p.opts {
+		fn(&p.options)
+	}
+	lvl := level(-1)
+	if p.options.defaultDirection == RightToLeft {
+		lvl = 1
+	}
+	para, err := newParagraph(p.types, p.pairTypes, p.pairValues, lvl)
+	if err != nil {
+		return Ordering{}, err
+	}
+
+	levels := para.getLevels([]int{len(p.types)})
+
+	p.o = calculateOrdering(levels, p.runes)
+	return p.o, nil
 }
 
 // Line computes the visual ordering of runs for a single line starting and
 // ending at the given positions in the original text.
 func (p *Paragraph) Line(start, end int) (Ordering, error) {
-	panic("unimplemented")
+	lineTypes := p.types[start:end]
+	para, err := newParagraph(lineTypes, p.pairTypes[start:end], p.pairValues[start:end], -1)
+	if err != nil {
+		return Ordering{}, err
+	}
+	levels := para.getLevels([]int{len(lineTypes)})
+	o := calculateOrdering(levels, p.runes[start:end])
+	return o, nil
 }
 
 // An Ordering holds the computed visual order of runs of a Paragraph. Calling
 // SetBytes or SetString on the originating Paragraph invalidates an Ordering.
 // The methods of an Ordering should only be called by one goroutine at a time.
-type Ordering struct{}
+type Ordering struct {
+	runes      [][]rune
+	directions []Direction
+	startpos   []int
+}
 
 // Direction reports the directionality of the runs.
 //
 // The direction may be LeftToRight, RightToLeft, Mixed, or Neutral.
 func (o *Ordering) Direction() Direction {
-	panic("unimplemented")
+	return o.directions[0]
 }
 
 // NumRuns returns the number of runs.
 func (o *Ordering) NumRuns() int {
-	panic("unimplemented")
+	return len(o.runes)
 }
 
 // Run returns the ith run within the ordering.
 func (o *Ordering) Run(i int) Run {
-	panic("unimplemented")
+	r := Run{
+		runes:     o.runes[i],
+		direction: o.directions[i],
+		startpos:  o.startpos[i],
+	}
+	return r
 }
 
 // TODO: perhaps with options.
@@ -155,16 +286,19 @@
 
 // A Run is a continuous sequence of characters of a single direction.
 type Run struct {
+	runes     []rune
+	direction Direction
+	startpos  int
 }
 
 // String returns the text of the run in its original order.
 func (r *Run) String() string {
-	panic("unimplemented")
+	return string(r.runes)
 }
 
 // Bytes returns the text of the run in its original order.
 func (r *Run) Bytes() []byte {
-	panic("unimplemented")
+	return []byte(r.String())
 }
 
 // TODO: methods for
@@ -174,25 +308,52 @@
 
 // Direction reports the direction of the run.
 func (r *Run) Direction() Direction {
-	panic("unimplemented")
+	return r.direction
 }
 
-// Position of the Run within the text passed to SetBytes or SetString of the
+// Pos returns the position of the Run within the text passed to SetBytes or SetString of the
 // originating Paragraph value.
 func (r *Run) Pos() (start, end int) {
-	panic("unimplemented")
+	return r.startpos, r.startpos + len(r.runes) - 1
 }
 
 // AppendReverse reverses the order of characters of in, appends them to out,
 // and returns the result. Modifiers will still follow the runes they modify.
 // Brackets are replaced with their counterparts.
 func AppendReverse(out, in []byte) []byte {
-	panic("unimplemented")
+	ret := make([]byte, len(in)+len(out))
+	copy(ret, out)
+	inRunes := bytes.Runes(in)
+
+	for i, r := range inRunes {
+		prop, _ := LookupRune(r)
+		if prop.IsBracket() {
+			inRunes[i] = prop.reverseBracket(r)
+		}
+	}
+
+	for i, j := 0, len(inRunes)-1; i < j; i, j = i+1, j-1 {
+		inRunes[i], inRunes[j] = inRunes[j], inRunes[i]
+	}
+	copy(ret[len(out):], string(inRunes))
+
+	return ret
 }
 
 // ReverseString reverses the order of characters in s and returns a new string.
 // Modifiers will still follow the runes they modify. Brackets are replaced with
 // their counterparts.
 func ReverseString(s string) string {
-	panic("unimplemented")
+	input := []rune(s)
+	li := len(input)
+	ret := make([]rune, li)
+	for i, r := range input {
+		prop, _ := LookupRune(r)
+		if prop.IsBracket() {
+			ret[li-i-1] = prop.reverseBracket(r)
+		} else {
+			ret[li-i-1] = r
+		}
+	}
+	return string(ret)
 }
diff --git a/vendor/golang.org/x/text/unicode/bidi/core.go b/vendor/golang.org/x/text/unicode/bidi/core.go
index 50deb66..e4c0811 100644
--- a/vendor/golang.org/x/text/unicode/bidi/core.go
+++ b/vendor/golang.org/x/text/unicode/bidi/core.go
@@ -4,7 +4,10 @@
 
 package bidi
 
-import "log"
+import (
+	"fmt"
+	"log"
+)
 
 // This implementation is a port based on the reference implementation found at:
 // https://www.unicode.org/Public/PROGRAMS/BidiReferenceJava/
@@ -97,13 +100,20 @@
 // rune (suggested is the rune of the open bracket for opening and matching
 // close brackets, after normalization). The embedding levels are optional, but
 // may be supplied to encode embedding levels of styled text.
-//
-// TODO: return an error.
-func newParagraph(types []Class, pairTypes []bracketType, pairValues []rune, levels level) *paragraph {
-	validateTypes(types)
-	validatePbTypes(pairTypes)
-	validatePbValues(pairValues, pairTypes)
-	validateParagraphEmbeddingLevel(levels)
+func newParagraph(types []Class, pairTypes []bracketType, pairValues []rune, levels level) (*paragraph, error) {
+	var err error
+	if err = validateTypes(types); err != nil {
+		return nil, err
+	}
+	if err = validatePbTypes(pairTypes); err != nil {
+		return nil, err
+	}
+	if err = validatePbValues(pairValues, pairTypes); err != nil {
+		return nil, err
+	}
+	if err = validateParagraphEmbeddingLevel(levels); err != nil {
+		return nil, err
+	}
 
 	p := &paragraph{
 		initialTypes:   append([]Class(nil), types...),
@@ -115,7 +125,7 @@
 		resultTypes: append([]Class(nil), types...),
 	}
 	p.run()
-	return p
+	return p, nil
 }
 
 func (p *paragraph) Len() int { return len(p.initialTypes) }
@@ -1001,58 +1011,61 @@
 	return R
 }
 
-// TODO: change validation to not panic
-
-func validateTypes(types []Class) {
+func validateTypes(types []Class) error {
 	if len(types) == 0 {
-		log.Panic("types is null")
+		return fmt.Errorf("types is null")
 	}
 	for i, t := range types[:len(types)-1] {
 		if t == B {
-			log.Panicf("B type before end of paragraph at index: %d", i)
+			return fmt.Errorf("B type before end of paragraph at index: %d", i)
 		}
 	}
+	return nil
 }
 
-func validateParagraphEmbeddingLevel(embeddingLevel level) {
+func validateParagraphEmbeddingLevel(embeddingLevel level) error {
 	if embeddingLevel != implicitLevel &&
 		embeddingLevel != 0 &&
 		embeddingLevel != 1 {
-		log.Panicf("illegal paragraph embedding level: %d", embeddingLevel)
+		return fmt.Errorf("illegal paragraph embedding level: %d", embeddingLevel)
 	}
+	return nil
 }
 
-func validateLineBreaks(linebreaks []int, textLength int) {
+func validateLineBreaks(linebreaks []int, textLength int) error {
 	prev := 0
 	for i, next := range linebreaks {
 		if next <= prev {
-			log.Panicf("bad linebreak: %d at index: %d", next, i)
+			return fmt.Errorf("bad linebreak: %d at index: %d", next, i)
 		}
 		prev = next
 	}
 	if prev != textLength {
-		log.Panicf("last linebreak was %d, want %d", prev, textLength)
+		return fmt.Errorf("last linebreak was %d, want %d", prev, textLength)
 	}
+	return nil
 }
 
-func validatePbTypes(pairTypes []bracketType) {
+func validatePbTypes(pairTypes []bracketType) error {
 	if len(pairTypes) == 0 {
-		log.Panic("pairTypes is null")
+		return fmt.Errorf("pairTypes is null")
 	}
 	for i, pt := range pairTypes {
 		switch pt {
 		case bpNone, bpOpen, bpClose:
 		default:
-			log.Panicf("illegal pairType value at %d: %v", i, pairTypes[i])
+			return fmt.Errorf("illegal pairType value at %d: %v", i, pairTypes[i])
 		}
 	}
+	return nil
 }
 
-func validatePbValues(pairValues []rune, pairTypes []bracketType) {
+func validatePbValues(pairValues []rune, pairTypes []bracketType) error {
 	if pairValues == nil {
-		log.Panic("pairValues is null")
+		return fmt.Errorf("pairValues is null")
 	}
 	if len(pairTypes) != len(pairValues) {
-		log.Panic("pairTypes is different length from pairValues")
+		return fmt.Errorf("pairTypes is different length from pairValues")
 	}
+	return nil
 }
diff --git a/vendor/golang.org/x/text/unicode/bidi/tables10.0.0.go b/vendor/golang.org/x/text/unicode/bidi/tables10.0.0.go
index d8c94e1..42fa8d7 100644
--- a/vendor/golang.org/x/text/unicode/bidi/tables10.0.0.go
+++ b/vendor/golang.org/x/text/unicode/bidi/tables10.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.10 && !go1.13
 // +build go1.10,!go1.13
 
 package bidi
diff --git a/vendor/golang.org/x/text/unicode/bidi/tables11.0.0.go b/vendor/golang.org/x/text/unicode/bidi/tables11.0.0.go
index 16b11db..56a0e1e 100644
--- a/vendor/golang.org/x/text/unicode/bidi/tables11.0.0.go
+++ b/vendor/golang.org/x/text/unicode/bidi/tables11.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.13 && !go1.14
 // +build go1.13,!go1.14
 
 package bidi
diff --git a/vendor/golang.org/x/text/unicode/bidi/tables12.0.0.go b/vendor/golang.org/x/text/unicode/bidi/tables12.0.0.go
index 7ffa365..baacf32 100644
--- a/vendor/golang.org/x/text/unicode/bidi/tables12.0.0.go
+++ b/vendor/golang.org/x/text/unicode/bidi/tables12.0.0.go
@@ -1,6 +1,7 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
-// +build go1.14
+//go:build go1.14 && !go1.16
+// +build go1.14,!go1.16
 
 package bidi
 
diff --git a/vendor/golang.org/x/text/unicode/bidi/tables13.0.0.go b/vendor/golang.org/x/text/unicode/bidi/tables13.0.0.go
new file mode 100644
index 0000000..f248eff
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/bidi/tables13.0.0.go
@@ -0,0 +1,1956 @@
+// Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
+
+//go:build go1.16
+// +build go1.16
+
+package bidi
+
+// UnicodeVersion is the Unicode version from which the tables in this package are derived.
+const UnicodeVersion = "13.0.0"
+
+// xorMasks contains masks to be xor-ed with brackets to get the reverse
+// version.
+var xorMasks = []int32{ // 8 elements
+	0, 1, 6, 7, 3, 15, 29, 63,
+} // Size: 56 bytes
+
+// lookup returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *bidiTrie) lookup(s []byte) (v uint8, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return bidiValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = bidiIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = bidiIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = bidiIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *bidiTrie) lookupUnsafe(s []byte) uint8 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return bidiValues[c0]
+	}
+	i := bidiIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = bidiIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = bidiIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// lookupString returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *bidiTrie) lookupString(s string) (v uint8, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return bidiValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = bidiIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := bidiIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = bidiIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = bidiIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *bidiTrie) lookupStringUnsafe(s string) uint8 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return bidiValues[c0]
+	}
+	i := bidiIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = bidiIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = bidiIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// bidiTrie. Total size: 17408 bytes (17.00 KiB). Checksum: df85fcbfe9b8377f.
+type bidiTrie struct{}
+
+func newBidiTrie(i int) *bidiTrie {
+	return &bidiTrie{}
+}
+
+// lookupValue determines the type of block n and looks up the value for b.
+func (t *bidiTrie) lookupValue(n uint32, b byte) uint8 {
+	switch {
+	default:
+		return uint8(bidiValues[n<<6+uint32(b)])
+	}
+}
+
+// bidiValues: 248 blocks, 15872 entries, 15872 bytes
+// The third block is the zero block.
+var bidiValues = [15872]uint8{
+	// Block 0x0, offset 0x0
+	0x00: 0x000b, 0x01: 0x000b, 0x02: 0x000b, 0x03: 0x000b, 0x04: 0x000b, 0x05: 0x000b,
+	0x06: 0x000b, 0x07: 0x000b, 0x08: 0x000b, 0x09: 0x0008, 0x0a: 0x0007, 0x0b: 0x0008,
+	0x0c: 0x0009, 0x0d: 0x0007, 0x0e: 0x000b, 0x0f: 0x000b, 0x10: 0x000b, 0x11: 0x000b,
+	0x12: 0x000b, 0x13: 0x000b, 0x14: 0x000b, 0x15: 0x000b, 0x16: 0x000b, 0x17: 0x000b,
+	0x18: 0x000b, 0x19: 0x000b, 0x1a: 0x000b, 0x1b: 0x000b, 0x1c: 0x0007, 0x1d: 0x0007,
+	0x1e: 0x0007, 0x1f: 0x0008, 0x20: 0x0009, 0x21: 0x000a, 0x22: 0x000a, 0x23: 0x0004,
+	0x24: 0x0004, 0x25: 0x0004, 0x26: 0x000a, 0x27: 0x000a, 0x28: 0x003a, 0x29: 0x002a,
+	0x2a: 0x000a, 0x2b: 0x0003, 0x2c: 0x0006, 0x2d: 0x0003, 0x2e: 0x0006, 0x2f: 0x0006,
+	0x30: 0x0002, 0x31: 0x0002, 0x32: 0x0002, 0x33: 0x0002, 0x34: 0x0002, 0x35: 0x0002,
+	0x36: 0x0002, 0x37: 0x0002, 0x38: 0x0002, 0x39: 0x0002, 0x3a: 0x0006, 0x3b: 0x000a,
+	0x3c: 0x000a, 0x3d: 0x000a, 0x3e: 0x000a, 0x3f: 0x000a,
+	// Block 0x1, offset 0x40
+	0x40: 0x000a,
+	0x5b: 0x005a, 0x5c: 0x000a, 0x5d: 0x004a,
+	0x5e: 0x000a, 0x5f: 0x000a, 0x60: 0x000a,
+	0x7b: 0x005a,
+	0x7c: 0x000a, 0x7d: 0x004a, 0x7e: 0x000a, 0x7f: 0x000b,
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc0: 0x000b, 0xc1: 0x000b, 0xc2: 0x000b, 0xc3: 0x000b, 0xc4: 0x000b, 0xc5: 0x0007,
+	0xc6: 0x000b, 0xc7: 0x000b, 0xc8: 0x000b, 0xc9: 0x000b, 0xca: 0x000b, 0xcb: 0x000b,
+	0xcc: 0x000b, 0xcd: 0x000b, 0xce: 0x000b, 0xcf: 0x000b, 0xd0: 0x000b, 0xd1: 0x000b,
+	0xd2: 0x000b, 0xd3: 0x000b, 0xd4: 0x000b, 0xd5: 0x000b, 0xd6: 0x000b, 0xd7: 0x000b,
+	0xd8: 0x000b, 0xd9: 0x000b, 0xda: 0x000b, 0xdb: 0x000b, 0xdc: 0x000b, 0xdd: 0x000b,
+	0xde: 0x000b, 0xdf: 0x000b, 0xe0: 0x0006, 0xe1: 0x000a, 0xe2: 0x0004, 0xe3: 0x0004,
+	0xe4: 0x0004, 0xe5: 0x0004, 0xe6: 0x000a, 0xe7: 0x000a, 0xe8: 0x000a, 0xe9: 0x000a,
+	0xeb: 0x000a, 0xec: 0x000a, 0xed: 0x000b, 0xee: 0x000a, 0xef: 0x000a,
+	0xf0: 0x0004, 0xf1: 0x0004, 0xf2: 0x0002, 0xf3: 0x0002, 0xf4: 0x000a,
+	0xf6: 0x000a, 0xf7: 0x000a, 0xf8: 0x000a, 0xf9: 0x0002, 0xfb: 0x000a,
+	0xfc: 0x000a, 0xfd: 0x000a, 0xfe: 0x000a, 0xff: 0x000a,
+	// Block 0x4, offset 0x100
+	0x117: 0x000a,
+	0x137: 0x000a,
+	// Block 0x5, offset 0x140
+	0x179: 0x000a, 0x17a: 0x000a,
+	// Block 0x6, offset 0x180
+	0x182: 0x000a, 0x183: 0x000a, 0x184: 0x000a, 0x185: 0x000a,
+	0x186: 0x000a, 0x187: 0x000a, 0x188: 0x000a, 0x189: 0x000a, 0x18a: 0x000a, 0x18b: 0x000a,
+	0x18c: 0x000a, 0x18d: 0x000a, 0x18e: 0x000a, 0x18f: 0x000a,
+	0x192: 0x000a, 0x193: 0x000a, 0x194: 0x000a, 0x195: 0x000a, 0x196: 0x000a, 0x197: 0x000a,
+	0x198: 0x000a, 0x199: 0x000a, 0x19a: 0x000a, 0x19b: 0x000a, 0x19c: 0x000a, 0x19d: 0x000a,
+	0x19e: 0x000a, 0x19f: 0x000a,
+	0x1a5: 0x000a, 0x1a6: 0x000a, 0x1a7: 0x000a, 0x1a8: 0x000a, 0x1a9: 0x000a,
+	0x1aa: 0x000a, 0x1ab: 0x000a, 0x1ac: 0x000a, 0x1ad: 0x000a, 0x1af: 0x000a,
+	0x1b0: 0x000a, 0x1b1: 0x000a, 0x1b2: 0x000a, 0x1b3: 0x000a, 0x1b4: 0x000a, 0x1b5: 0x000a,
+	0x1b6: 0x000a, 0x1b7: 0x000a, 0x1b8: 0x000a, 0x1b9: 0x000a, 0x1ba: 0x000a, 0x1bb: 0x000a,
+	0x1bc: 0x000a, 0x1bd: 0x000a, 0x1be: 0x000a, 0x1bf: 0x000a,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x000c, 0x1c1: 0x000c, 0x1c2: 0x000c, 0x1c3: 0x000c, 0x1c4: 0x000c, 0x1c5: 0x000c,
+	0x1c6: 0x000c, 0x1c7: 0x000c, 0x1c8: 0x000c, 0x1c9: 0x000c, 0x1ca: 0x000c, 0x1cb: 0x000c,
+	0x1cc: 0x000c, 0x1cd: 0x000c, 0x1ce: 0x000c, 0x1cf: 0x000c, 0x1d0: 0x000c, 0x1d1: 0x000c,
+	0x1d2: 0x000c, 0x1d3: 0x000c, 0x1d4: 0x000c, 0x1d5: 0x000c, 0x1d6: 0x000c, 0x1d7: 0x000c,
+	0x1d8: 0x000c, 0x1d9: 0x000c, 0x1da: 0x000c, 0x1db: 0x000c, 0x1dc: 0x000c, 0x1dd: 0x000c,
+	0x1de: 0x000c, 0x1df: 0x000c, 0x1e0: 0x000c, 0x1e1: 0x000c, 0x1e2: 0x000c, 0x1e3: 0x000c,
+	0x1e4: 0x000c, 0x1e5: 0x000c, 0x1e6: 0x000c, 0x1e7: 0x000c, 0x1e8: 0x000c, 0x1e9: 0x000c,
+	0x1ea: 0x000c, 0x1eb: 0x000c, 0x1ec: 0x000c, 0x1ed: 0x000c, 0x1ee: 0x000c, 0x1ef: 0x000c,
+	0x1f0: 0x000c, 0x1f1: 0x000c, 0x1f2: 0x000c, 0x1f3: 0x000c, 0x1f4: 0x000c, 0x1f5: 0x000c,
+	0x1f6: 0x000c, 0x1f7: 0x000c, 0x1f8: 0x000c, 0x1f9: 0x000c, 0x1fa: 0x000c, 0x1fb: 0x000c,
+	0x1fc: 0x000c, 0x1fd: 0x000c, 0x1fe: 0x000c, 0x1ff: 0x000c,
+	// Block 0x8, offset 0x200
+	0x200: 0x000c, 0x201: 0x000c, 0x202: 0x000c, 0x203: 0x000c, 0x204: 0x000c, 0x205: 0x000c,
+	0x206: 0x000c, 0x207: 0x000c, 0x208: 0x000c, 0x209: 0x000c, 0x20a: 0x000c, 0x20b: 0x000c,
+	0x20c: 0x000c, 0x20d: 0x000c, 0x20e: 0x000c, 0x20f: 0x000c, 0x210: 0x000c, 0x211: 0x000c,
+	0x212: 0x000c, 0x213: 0x000c, 0x214: 0x000c, 0x215: 0x000c, 0x216: 0x000c, 0x217: 0x000c,
+	0x218: 0x000c, 0x219: 0x000c, 0x21a: 0x000c, 0x21b: 0x000c, 0x21c: 0x000c, 0x21d: 0x000c,
+	0x21e: 0x000c, 0x21f: 0x000c, 0x220: 0x000c, 0x221: 0x000c, 0x222: 0x000c, 0x223: 0x000c,
+	0x224: 0x000c, 0x225: 0x000c, 0x226: 0x000c, 0x227: 0x000c, 0x228: 0x000c, 0x229: 0x000c,
+	0x22a: 0x000c, 0x22b: 0x000c, 0x22c: 0x000c, 0x22d: 0x000c, 0x22e: 0x000c, 0x22f: 0x000c,
+	0x234: 0x000a, 0x235: 0x000a,
+	0x23e: 0x000a,
+	// Block 0x9, offset 0x240
+	0x244: 0x000a, 0x245: 0x000a,
+	0x247: 0x000a,
+	// Block 0xa, offset 0x280
+	0x2b6: 0x000a,
+	// Block 0xb, offset 0x2c0
+	0x2c3: 0x000c, 0x2c4: 0x000c, 0x2c5: 0x000c,
+	0x2c6: 0x000c, 0x2c7: 0x000c, 0x2c8: 0x000c, 0x2c9: 0x000c,
+	// Block 0xc, offset 0x300
+	0x30a: 0x000a,
+	0x30d: 0x000a, 0x30e: 0x000a, 0x30f: 0x0004, 0x310: 0x0001, 0x311: 0x000c,
+	0x312: 0x000c, 0x313: 0x000c, 0x314: 0x000c, 0x315: 0x000c, 0x316: 0x000c, 0x317: 0x000c,
+	0x318: 0x000c, 0x319: 0x000c, 0x31a: 0x000c, 0x31b: 0x000c, 0x31c: 0x000c, 0x31d: 0x000c,
+	0x31e: 0x000c, 0x31f: 0x000c, 0x320: 0x000c, 0x321: 0x000c, 0x322: 0x000c, 0x323: 0x000c,
+	0x324: 0x000c, 0x325: 0x000c, 0x326: 0x000c, 0x327: 0x000c, 0x328: 0x000c, 0x329: 0x000c,
+	0x32a: 0x000c, 0x32b: 0x000c, 0x32c: 0x000c, 0x32d: 0x000c, 0x32e: 0x000c, 0x32f: 0x000c,
+	0x330: 0x000c, 0x331: 0x000c, 0x332: 0x000c, 0x333: 0x000c, 0x334: 0x000c, 0x335: 0x000c,
+	0x336: 0x000c, 0x337: 0x000c, 0x338: 0x000c, 0x339: 0x000c, 0x33a: 0x000c, 0x33b: 0x000c,
+	0x33c: 0x000c, 0x33d: 0x000c, 0x33e: 0x0001, 0x33f: 0x000c,
+	// Block 0xd, offset 0x340
+	0x340: 0x0001, 0x341: 0x000c, 0x342: 0x000c, 0x343: 0x0001, 0x344: 0x000c, 0x345: 0x000c,
+	0x346: 0x0001, 0x347: 0x000c, 0x348: 0x0001, 0x349: 0x0001, 0x34a: 0x0001, 0x34b: 0x0001,
+	0x34c: 0x0001, 0x34d: 0x0001, 0x34e: 0x0001, 0x34f: 0x0001, 0x350: 0x0001, 0x351: 0x0001,
+	0x352: 0x0001, 0x353: 0x0001, 0x354: 0x0001, 0x355: 0x0001, 0x356: 0x0001, 0x357: 0x0001,
+	0x358: 0x0001, 0x359: 0x0001, 0x35a: 0x0001, 0x35b: 0x0001, 0x35c: 0x0001, 0x35d: 0x0001,
+	0x35e: 0x0001, 0x35f: 0x0001, 0x360: 0x0001, 0x361: 0x0001, 0x362: 0x0001, 0x363: 0x0001,
+	0x364: 0x0001, 0x365: 0x0001, 0x366: 0x0001, 0x367: 0x0001, 0x368: 0x0001, 0x369: 0x0001,
+	0x36a: 0x0001, 0x36b: 0x0001, 0x36c: 0x0001, 0x36d: 0x0001, 0x36e: 0x0001, 0x36f: 0x0001,
+	0x370: 0x0001, 0x371: 0x0001, 0x372: 0x0001, 0x373: 0x0001, 0x374: 0x0001, 0x375: 0x0001,
+	0x376: 0x0001, 0x377: 0x0001, 0x378: 0x0001, 0x379: 0x0001, 0x37a: 0x0001, 0x37b: 0x0001,
+	0x37c: 0x0001, 0x37d: 0x0001, 0x37e: 0x0001, 0x37f: 0x0001,
+	// Block 0xe, offset 0x380
+	0x380: 0x0005, 0x381: 0x0005, 0x382: 0x0005, 0x383: 0x0005, 0x384: 0x0005, 0x385: 0x0005,
+	0x386: 0x000a, 0x387: 0x000a, 0x388: 0x000d, 0x389: 0x0004, 0x38a: 0x0004, 0x38b: 0x000d,
+	0x38c: 0x0006, 0x38d: 0x000d, 0x38e: 0x000a, 0x38f: 0x000a, 0x390: 0x000c, 0x391: 0x000c,
+	0x392: 0x000c, 0x393: 0x000c, 0x394: 0x000c, 0x395: 0x000c, 0x396: 0x000c, 0x397: 0x000c,
+	0x398: 0x000c, 0x399: 0x000c, 0x39a: 0x000c, 0x39b: 0x000d, 0x39c: 0x000d, 0x39d: 0x000d,
+	0x39e: 0x000d, 0x39f: 0x000d, 0x3a0: 0x000d, 0x3a1: 0x000d, 0x3a2: 0x000d, 0x3a3: 0x000d,
+	0x3a4: 0x000d, 0x3a5: 0x000d, 0x3a6: 0x000d, 0x3a7: 0x000d, 0x3a8: 0x000d, 0x3a9: 0x000d,
+	0x3aa: 0x000d, 0x3ab: 0x000d, 0x3ac: 0x000d, 0x3ad: 0x000d, 0x3ae: 0x000d, 0x3af: 0x000d,
+	0x3b0: 0x000d, 0x3b1: 0x000d, 0x3b2: 0x000d, 0x3b3: 0x000d, 0x3b4: 0x000d, 0x3b5: 0x000d,
+	0x3b6: 0x000d, 0x3b7: 0x000d, 0x3b8: 0x000d, 0x3b9: 0x000d, 0x3ba: 0x000d, 0x3bb: 0x000d,
+	0x3bc: 0x000d, 0x3bd: 0x000d, 0x3be: 0x000d, 0x3bf: 0x000d,
+	// Block 0xf, offset 0x3c0
+	0x3c0: 0x000d, 0x3c1: 0x000d, 0x3c2: 0x000d, 0x3c3: 0x000d, 0x3c4: 0x000d, 0x3c5: 0x000d,
+	0x3c6: 0x000d, 0x3c7: 0x000d, 0x3c8: 0x000d, 0x3c9: 0x000d, 0x3ca: 0x000d, 0x3cb: 0x000c,
+	0x3cc: 0x000c, 0x3cd: 0x000c, 0x3ce: 0x000c, 0x3cf: 0x000c, 0x3d0: 0x000c, 0x3d1: 0x000c,
+	0x3d2: 0x000c, 0x3d3: 0x000c, 0x3d4: 0x000c, 0x3d5: 0x000c, 0x3d6: 0x000c, 0x3d7: 0x000c,
+	0x3d8: 0x000c, 0x3d9: 0x000c, 0x3da: 0x000c, 0x3db: 0x000c, 0x3dc: 0x000c, 0x3dd: 0x000c,
+	0x3de: 0x000c, 0x3df: 0x000c, 0x3e0: 0x0005, 0x3e1: 0x0005, 0x3e2: 0x0005, 0x3e3: 0x0005,
+	0x3e4: 0x0005, 0x3e5: 0x0005, 0x3e6: 0x0005, 0x3e7: 0x0005, 0x3e8: 0x0005, 0x3e9: 0x0005,
+	0x3ea: 0x0004, 0x3eb: 0x0005, 0x3ec: 0x0005, 0x3ed: 0x000d, 0x3ee: 0x000d, 0x3ef: 0x000d,
+	0x3f0: 0x000c, 0x3f1: 0x000d, 0x3f2: 0x000d, 0x3f3: 0x000d, 0x3f4: 0x000d, 0x3f5: 0x000d,
+	0x3f6: 0x000d, 0x3f7: 0x000d, 0x3f8: 0x000d, 0x3f9: 0x000d, 0x3fa: 0x000d, 0x3fb: 0x000d,
+	0x3fc: 0x000d, 0x3fd: 0x000d, 0x3fe: 0x000d, 0x3ff: 0x000d,
+	// Block 0x10, offset 0x400
+	0x400: 0x000d, 0x401: 0x000d, 0x402: 0x000d, 0x403: 0x000d, 0x404: 0x000d, 0x405: 0x000d,
+	0x406: 0x000d, 0x407: 0x000d, 0x408: 0x000d, 0x409: 0x000d, 0x40a: 0x000d, 0x40b: 0x000d,
+	0x40c: 0x000d, 0x40d: 0x000d, 0x40e: 0x000d, 0x40f: 0x000d, 0x410: 0x000d, 0x411: 0x000d,
+	0x412: 0x000d, 0x413: 0x000d, 0x414: 0x000d, 0x415: 0x000d, 0x416: 0x000d, 0x417: 0x000d,
+	0x418: 0x000d, 0x419: 0x000d, 0x41a: 0x000d, 0x41b: 0x000d, 0x41c: 0x000d, 0x41d: 0x000d,
+	0x41e: 0x000d, 0x41f: 0x000d, 0x420: 0x000d, 0x421: 0x000d, 0x422: 0x000d, 0x423: 0x000d,
+	0x424: 0x000d, 0x425: 0x000d, 0x426: 0x000d, 0x427: 0x000d, 0x428: 0x000d, 0x429: 0x000d,
+	0x42a: 0x000d, 0x42b: 0x000d, 0x42c: 0x000d, 0x42d: 0x000d, 0x42e: 0x000d, 0x42f: 0x000d,
+	0x430: 0x000d, 0x431: 0x000d, 0x432: 0x000d, 0x433: 0x000d, 0x434: 0x000d, 0x435: 0x000d,
+	0x436: 0x000d, 0x437: 0x000d, 0x438: 0x000d, 0x439: 0x000d, 0x43a: 0x000d, 0x43b: 0x000d,
+	0x43c: 0x000d, 0x43d: 0x000d, 0x43e: 0x000d, 0x43f: 0x000d,
+	// Block 0x11, offset 0x440
+	0x440: 0x000d, 0x441: 0x000d, 0x442: 0x000d, 0x443: 0x000d, 0x444: 0x000d, 0x445: 0x000d,
+	0x446: 0x000d, 0x447: 0x000d, 0x448: 0x000d, 0x449: 0x000d, 0x44a: 0x000d, 0x44b: 0x000d,
+	0x44c: 0x000d, 0x44d: 0x000d, 0x44e: 0x000d, 0x44f: 0x000d, 0x450: 0x000d, 0x451: 0x000d,
+	0x452: 0x000d, 0x453: 0x000d, 0x454: 0x000d, 0x455: 0x000d, 0x456: 0x000c, 0x457: 0x000c,
+	0x458: 0x000c, 0x459: 0x000c, 0x45a: 0x000c, 0x45b: 0x000c, 0x45c: 0x000c, 0x45d: 0x0005,
+	0x45e: 0x000a, 0x45f: 0x000c, 0x460: 0x000c, 0x461: 0x000c, 0x462: 0x000c, 0x463: 0x000c,
+	0x464: 0x000c, 0x465: 0x000d, 0x466: 0x000d, 0x467: 0x000c, 0x468: 0x000c, 0x469: 0x000a,
+	0x46a: 0x000c, 0x46b: 0x000c, 0x46c: 0x000c, 0x46d: 0x000c, 0x46e: 0x000d, 0x46f: 0x000d,
+	0x470: 0x0002, 0x471: 0x0002, 0x472: 0x0002, 0x473: 0x0002, 0x474: 0x0002, 0x475: 0x0002,
+	0x476: 0x0002, 0x477: 0x0002, 0x478: 0x0002, 0x479: 0x0002, 0x47a: 0x000d, 0x47b: 0x000d,
+	0x47c: 0x000d, 0x47d: 0x000d, 0x47e: 0x000d, 0x47f: 0x000d,
+	// Block 0x12, offset 0x480
+	0x480: 0x000d, 0x481: 0x000d, 0x482: 0x000d, 0x483: 0x000d, 0x484: 0x000d, 0x485: 0x000d,
+	0x486: 0x000d, 0x487: 0x000d, 0x488: 0x000d, 0x489: 0x000d, 0x48a: 0x000d, 0x48b: 0x000d,
+	0x48c: 0x000d, 0x48d: 0x000d, 0x48e: 0x000d, 0x48f: 0x000d, 0x490: 0x000d, 0x491: 0x000c,
+	0x492: 0x000d, 0x493: 0x000d, 0x494: 0x000d, 0x495: 0x000d, 0x496: 0x000d, 0x497: 0x000d,
+	0x498: 0x000d, 0x499: 0x000d, 0x49a: 0x000d, 0x49b: 0x000d, 0x49c: 0x000d, 0x49d: 0x000d,
+	0x49e: 0x000d, 0x49f: 0x000d, 0x4a0: 0x000d, 0x4a1: 0x000d, 0x4a2: 0x000d, 0x4a3: 0x000d,
+	0x4a4: 0x000d, 0x4a5: 0x000d, 0x4a6: 0x000d, 0x4a7: 0x000d, 0x4a8: 0x000d, 0x4a9: 0x000d,
+	0x4aa: 0x000d, 0x4ab: 0x000d, 0x4ac: 0x000d, 0x4ad: 0x000d, 0x4ae: 0x000d, 0x4af: 0x000d,
+	0x4b0: 0x000c, 0x4b1: 0x000c, 0x4b2: 0x000c, 0x4b3: 0x000c, 0x4b4: 0x000c, 0x4b5: 0x000c,
+	0x4b6: 0x000c, 0x4b7: 0x000c, 0x4b8: 0x000c, 0x4b9: 0x000c, 0x4ba: 0x000c, 0x4bb: 0x000c,
+	0x4bc: 0x000c, 0x4bd: 0x000c, 0x4be: 0x000c, 0x4bf: 0x000c,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0x000c, 0x4c1: 0x000c, 0x4c2: 0x000c, 0x4c3: 0x000c, 0x4c4: 0x000c, 0x4c5: 0x000c,
+	0x4c6: 0x000c, 0x4c7: 0x000c, 0x4c8: 0x000c, 0x4c9: 0x000c, 0x4ca: 0x000c, 0x4cb: 0x000d,
+	0x4cc: 0x000d, 0x4cd: 0x000d, 0x4ce: 0x000d, 0x4cf: 0x000d, 0x4d0: 0x000d, 0x4d1: 0x000d,
+	0x4d2: 0x000d, 0x4d3: 0x000d, 0x4d4: 0x000d, 0x4d5: 0x000d, 0x4d6: 0x000d, 0x4d7: 0x000d,
+	0x4d8: 0x000d, 0x4d9: 0x000d, 0x4da: 0x000d, 0x4db: 0x000d, 0x4dc: 0x000d, 0x4dd: 0x000d,
+	0x4de: 0x000d, 0x4df: 0x000d, 0x4e0: 0x000d, 0x4e1: 0x000d, 0x4e2: 0x000d, 0x4e3: 0x000d,
+	0x4e4: 0x000d, 0x4e5: 0x000d, 0x4e6: 0x000d, 0x4e7: 0x000d, 0x4e8: 0x000d, 0x4e9: 0x000d,
+	0x4ea: 0x000d, 0x4eb: 0x000d, 0x4ec: 0x000d, 0x4ed: 0x000d, 0x4ee: 0x000d, 0x4ef: 0x000d,
+	0x4f0: 0x000d, 0x4f1: 0x000d, 0x4f2: 0x000d, 0x4f3: 0x000d, 0x4f4: 0x000d, 0x4f5: 0x000d,
+	0x4f6: 0x000d, 0x4f7: 0x000d, 0x4f8: 0x000d, 0x4f9: 0x000d, 0x4fa: 0x000d, 0x4fb: 0x000d,
+	0x4fc: 0x000d, 0x4fd: 0x000d, 0x4fe: 0x000d, 0x4ff: 0x000d,
+	// Block 0x14, offset 0x500
+	0x500: 0x000d, 0x501: 0x000d, 0x502: 0x000d, 0x503: 0x000d, 0x504: 0x000d, 0x505: 0x000d,
+	0x506: 0x000d, 0x507: 0x000d, 0x508: 0x000d, 0x509: 0x000d, 0x50a: 0x000d, 0x50b: 0x000d,
+	0x50c: 0x000d, 0x50d: 0x000d, 0x50e: 0x000d, 0x50f: 0x000d, 0x510: 0x000d, 0x511: 0x000d,
+	0x512: 0x000d, 0x513: 0x000d, 0x514: 0x000d, 0x515: 0x000d, 0x516: 0x000d, 0x517: 0x000d,
+	0x518: 0x000d, 0x519: 0x000d, 0x51a: 0x000d, 0x51b: 0x000d, 0x51c: 0x000d, 0x51d: 0x000d,
+	0x51e: 0x000d, 0x51f: 0x000d, 0x520: 0x000d, 0x521: 0x000d, 0x522: 0x000d, 0x523: 0x000d,
+	0x524: 0x000d, 0x525: 0x000d, 0x526: 0x000c, 0x527: 0x000c, 0x528: 0x000c, 0x529: 0x000c,
+	0x52a: 0x000c, 0x52b: 0x000c, 0x52c: 0x000c, 0x52d: 0x000c, 0x52e: 0x000c, 0x52f: 0x000c,
+	0x530: 0x000c, 0x531: 0x000d, 0x532: 0x000d, 0x533: 0x000d, 0x534: 0x000d, 0x535: 0x000d,
+	0x536: 0x000d, 0x537: 0x000d, 0x538: 0x000d, 0x539: 0x000d, 0x53a: 0x000d, 0x53b: 0x000d,
+	0x53c: 0x000d, 0x53d: 0x000d, 0x53e: 0x000d, 0x53f: 0x000d,
+	// Block 0x15, offset 0x540
+	0x540: 0x0001, 0x541: 0x0001, 0x542: 0x0001, 0x543: 0x0001, 0x544: 0x0001, 0x545: 0x0001,
+	0x546: 0x0001, 0x547: 0x0001, 0x548: 0x0001, 0x549: 0x0001, 0x54a: 0x0001, 0x54b: 0x0001,
+	0x54c: 0x0001, 0x54d: 0x0001, 0x54e: 0x0001, 0x54f: 0x0001, 0x550: 0x0001, 0x551: 0x0001,
+	0x552: 0x0001, 0x553: 0x0001, 0x554: 0x0001, 0x555: 0x0001, 0x556: 0x0001, 0x557: 0x0001,
+	0x558: 0x0001, 0x559: 0x0001, 0x55a: 0x0001, 0x55b: 0x0001, 0x55c: 0x0001, 0x55d: 0x0001,
+	0x55e: 0x0001, 0x55f: 0x0001, 0x560: 0x0001, 0x561: 0x0001, 0x562: 0x0001, 0x563: 0x0001,
+	0x564: 0x0001, 0x565: 0x0001, 0x566: 0x0001, 0x567: 0x0001, 0x568: 0x0001, 0x569: 0x0001,
+	0x56a: 0x0001, 0x56b: 0x000c, 0x56c: 0x000c, 0x56d: 0x000c, 0x56e: 0x000c, 0x56f: 0x000c,
+	0x570: 0x000c, 0x571: 0x000c, 0x572: 0x000c, 0x573: 0x000c, 0x574: 0x0001, 0x575: 0x0001,
+	0x576: 0x000a, 0x577: 0x000a, 0x578: 0x000a, 0x579: 0x000a, 0x57a: 0x0001, 0x57b: 0x0001,
+	0x57c: 0x0001, 0x57d: 0x000c, 0x57e: 0x0001, 0x57f: 0x0001,
+	// Block 0x16, offset 0x580
+	0x580: 0x0001, 0x581: 0x0001, 0x582: 0x0001, 0x583: 0x0001, 0x584: 0x0001, 0x585: 0x0001,
+	0x586: 0x0001, 0x587: 0x0001, 0x588: 0x0001, 0x589: 0x0001, 0x58a: 0x0001, 0x58b: 0x0001,
+	0x58c: 0x0001, 0x58d: 0x0001, 0x58e: 0x0001, 0x58f: 0x0001, 0x590: 0x0001, 0x591: 0x0001,
+	0x592: 0x0001, 0x593: 0x0001, 0x594: 0x0001, 0x595: 0x0001, 0x596: 0x000c, 0x597: 0x000c,
+	0x598: 0x000c, 0x599: 0x000c, 0x59a: 0x0001, 0x59b: 0x000c, 0x59c: 0x000c, 0x59d: 0x000c,
+	0x59e: 0x000c, 0x59f: 0x000c, 0x5a0: 0x000c, 0x5a1: 0x000c, 0x5a2: 0x000c, 0x5a3: 0x000c,
+	0x5a4: 0x0001, 0x5a5: 0x000c, 0x5a6: 0x000c, 0x5a7: 0x000c, 0x5a8: 0x0001, 0x5a9: 0x000c,
+	0x5aa: 0x000c, 0x5ab: 0x000c, 0x5ac: 0x000c, 0x5ad: 0x000c, 0x5ae: 0x0001, 0x5af: 0x0001,
+	0x5b0: 0x0001, 0x5b1: 0x0001, 0x5b2: 0x0001, 0x5b3: 0x0001, 0x5b4: 0x0001, 0x5b5: 0x0001,
+	0x5b6: 0x0001, 0x5b7: 0x0001, 0x5b8: 0x0001, 0x5b9: 0x0001, 0x5ba: 0x0001, 0x5bb: 0x0001,
+	0x5bc: 0x0001, 0x5bd: 0x0001, 0x5be: 0x0001, 0x5bf: 0x0001,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x0001, 0x5c1: 0x0001, 0x5c2: 0x0001, 0x5c3: 0x0001, 0x5c4: 0x0001, 0x5c5: 0x0001,
+	0x5c6: 0x0001, 0x5c7: 0x0001, 0x5c8: 0x0001, 0x5c9: 0x0001, 0x5ca: 0x0001, 0x5cb: 0x0001,
+	0x5cc: 0x0001, 0x5cd: 0x0001, 0x5ce: 0x0001, 0x5cf: 0x0001, 0x5d0: 0x0001, 0x5d1: 0x0001,
+	0x5d2: 0x0001, 0x5d3: 0x0001, 0x5d4: 0x0001, 0x5d5: 0x0001, 0x5d6: 0x0001, 0x5d7: 0x0001,
+	0x5d8: 0x0001, 0x5d9: 0x000c, 0x5da: 0x000c, 0x5db: 0x000c, 0x5dc: 0x0001, 0x5dd: 0x0001,
+	0x5de: 0x0001, 0x5df: 0x0001, 0x5e0: 0x000d, 0x5e1: 0x000d, 0x5e2: 0x000d, 0x5e3: 0x000d,
+	0x5e4: 0x000d, 0x5e5: 0x000d, 0x5e6: 0x000d, 0x5e7: 0x000d, 0x5e8: 0x000d, 0x5e9: 0x000d,
+	0x5ea: 0x000d, 0x5eb: 0x000d, 0x5ec: 0x000d, 0x5ed: 0x000d, 0x5ee: 0x000d, 0x5ef: 0x000d,
+	0x5f0: 0x0001, 0x5f1: 0x0001, 0x5f2: 0x0001, 0x5f3: 0x0001, 0x5f4: 0x0001, 0x5f5: 0x0001,
+	0x5f6: 0x0001, 0x5f7: 0x0001, 0x5f8: 0x0001, 0x5f9: 0x0001, 0x5fa: 0x0001, 0x5fb: 0x0001,
+	0x5fc: 0x0001, 0x5fd: 0x0001, 0x5fe: 0x0001, 0x5ff: 0x0001,
+	// Block 0x18, offset 0x600
+	0x600: 0x0001, 0x601: 0x0001, 0x602: 0x0001, 0x603: 0x0001, 0x604: 0x0001, 0x605: 0x0001,
+	0x606: 0x0001, 0x607: 0x0001, 0x608: 0x0001, 0x609: 0x0001, 0x60a: 0x0001, 0x60b: 0x0001,
+	0x60c: 0x0001, 0x60d: 0x0001, 0x60e: 0x0001, 0x60f: 0x0001, 0x610: 0x0001, 0x611: 0x0001,
+	0x612: 0x0001, 0x613: 0x0001, 0x614: 0x0001, 0x615: 0x0001, 0x616: 0x0001, 0x617: 0x0001,
+	0x618: 0x0001, 0x619: 0x0001, 0x61a: 0x0001, 0x61b: 0x0001, 0x61c: 0x0001, 0x61d: 0x0001,
+	0x61e: 0x0001, 0x61f: 0x0001, 0x620: 0x000d, 0x621: 0x000d, 0x622: 0x000d, 0x623: 0x000d,
+	0x624: 0x000d, 0x625: 0x000d, 0x626: 0x000d, 0x627: 0x000d, 0x628: 0x000d, 0x629: 0x000d,
+	0x62a: 0x000d, 0x62b: 0x000d, 0x62c: 0x000d, 0x62d: 0x000d, 0x62e: 0x000d, 0x62f: 0x000d,
+	0x630: 0x000d, 0x631: 0x000d, 0x632: 0x000d, 0x633: 0x000d, 0x634: 0x000d, 0x635: 0x000d,
+	0x636: 0x000d, 0x637: 0x000d, 0x638: 0x000d, 0x639: 0x000d, 0x63a: 0x000d, 0x63b: 0x000d,
+	0x63c: 0x000d, 0x63d: 0x000d, 0x63e: 0x000d, 0x63f: 0x000d,
+	// Block 0x19, offset 0x640
+	0x640: 0x000d, 0x641: 0x000d, 0x642: 0x000d, 0x643: 0x000d, 0x644: 0x000d, 0x645: 0x000d,
+	0x646: 0x000d, 0x647: 0x000d, 0x648: 0x000d, 0x649: 0x000d, 0x64a: 0x000d, 0x64b: 0x000d,
+	0x64c: 0x000d, 0x64d: 0x000d, 0x64e: 0x000d, 0x64f: 0x000d, 0x650: 0x000d, 0x651: 0x000d,
+	0x652: 0x000d, 0x653: 0x000c, 0x654: 0x000c, 0x655: 0x000c, 0x656: 0x000c, 0x657: 0x000c,
+	0x658: 0x000c, 0x659: 0x000c, 0x65a: 0x000c, 0x65b: 0x000c, 0x65c: 0x000c, 0x65d: 0x000c,
+	0x65e: 0x000c, 0x65f: 0x000c, 0x660: 0x000c, 0x661: 0x000c, 0x662: 0x0005, 0x663: 0x000c,
+	0x664: 0x000c, 0x665: 0x000c, 0x666: 0x000c, 0x667: 0x000c, 0x668: 0x000c, 0x669: 0x000c,
+	0x66a: 0x000c, 0x66b: 0x000c, 0x66c: 0x000c, 0x66d: 0x000c, 0x66e: 0x000c, 0x66f: 0x000c,
+	0x670: 0x000c, 0x671: 0x000c, 0x672: 0x000c, 0x673: 0x000c, 0x674: 0x000c, 0x675: 0x000c,
+	0x676: 0x000c, 0x677: 0x000c, 0x678: 0x000c, 0x679: 0x000c, 0x67a: 0x000c, 0x67b: 0x000c,
+	0x67c: 0x000c, 0x67d: 0x000c, 0x67e: 0x000c, 0x67f: 0x000c,
+	// Block 0x1a, offset 0x680
+	0x680: 0x000c, 0x681: 0x000c, 0x682: 0x000c,
+	0x6ba: 0x000c,
+	0x6bc: 0x000c,
+	// Block 0x1b, offset 0x6c0
+	0x6c1: 0x000c, 0x6c2: 0x000c, 0x6c3: 0x000c, 0x6c4: 0x000c, 0x6c5: 0x000c,
+	0x6c6: 0x000c, 0x6c7: 0x000c, 0x6c8: 0x000c,
+	0x6cd: 0x000c, 0x6d1: 0x000c,
+	0x6d2: 0x000c, 0x6d3: 0x000c, 0x6d4: 0x000c, 0x6d5: 0x000c, 0x6d6: 0x000c, 0x6d7: 0x000c,
+	0x6e2: 0x000c, 0x6e3: 0x000c,
+	// Block 0x1c, offset 0x700
+	0x701: 0x000c,
+	0x73c: 0x000c,
+	// Block 0x1d, offset 0x740
+	0x741: 0x000c, 0x742: 0x000c, 0x743: 0x000c, 0x744: 0x000c,
+	0x74d: 0x000c,
+	0x762: 0x000c, 0x763: 0x000c,
+	0x772: 0x0004, 0x773: 0x0004,
+	0x77b: 0x0004,
+	0x77e: 0x000c,
+	// Block 0x1e, offset 0x780
+	0x781: 0x000c, 0x782: 0x000c,
+	0x7bc: 0x000c,
+	// Block 0x1f, offset 0x7c0
+	0x7c1: 0x000c, 0x7c2: 0x000c,
+	0x7c7: 0x000c, 0x7c8: 0x000c, 0x7cb: 0x000c,
+	0x7cc: 0x000c, 0x7cd: 0x000c, 0x7d1: 0x000c,
+	0x7f0: 0x000c, 0x7f1: 0x000c, 0x7f5: 0x000c,
+	// Block 0x20, offset 0x800
+	0x801: 0x000c, 0x802: 0x000c, 0x803: 0x000c, 0x804: 0x000c, 0x805: 0x000c,
+	0x807: 0x000c, 0x808: 0x000c,
+	0x80d: 0x000c,
+	0x822: 0x000c, 0x823: 0x000c,
+	0x831: 0x0004,
+	0x83a: 0x000c, 0x83b: 0x000c,
+	0x83c: 0x000c, 0x83d: 0x000c, 0x83e: 0x000c, 0x83f: 0x000c,
+	// Block 0x21, offset 0x840
+	0x841: 0x000c,
+	0x87c: 0x000c, 0x87f: 0x000c,
+	// Block 0x22, offset 0x880
+	0x881: 0x000c, 0x882: 0x000c, 0x883: 0x000c, 0x884: 0x000c,
+	0x88d: 0x000c,
+	0x895: 0x000c, 0x896: 0x000c,
+	0x8a2: 0x000c, 0x8a3: 0x000c,
+	// Block 0x23, offset 0x8c0
+	0x8c2: 0x000c,
+	// Block 0x24, offset 0x900
+	0x900: 0x000c,
+	0x90d: 0x000c,
+	0x933: 0x000a, 0x934: 0x000a, 0x935: 0x000a,
+	0x936: 0x000a, 0x937: 0x000a, 0x938: 0x000a, 0x939: 0x0004, 0x93a: 0x000a,
+	// Block 0x25, offset 0x940
+	0x940: 0x000c, 0x944: 0x000c,
+	0x97e: 0x000c, 0x97f: 0x000c,
+	// Block 0x26, offset 0x980
+	0x980: 0x000c,
+	0x986: 0x000c, 0x987: 0x000c, 0x988: 0x000c, 0x98a: 0x000c, 0x98b: 0x000c,
+	0x98c: 0x000c, 0x98d: 0x000c,
+	0x995: 0x000c, 0x996: 0x000c,
+	0x9a2: 0x000c, 0x9a3: 0x000c,
+	0x9b8: 0x000a, 0x9b9: 0x000a, 0x9ba: 0x000a, 0x9bb: 0x000a,
+	0x9bc: 0x000a, 0x9bd: 0x000a, 0x9be: 0x000a,
+	// Block 0x27, offset 0x9c0
+	0x9cc: 0x000c, 0x9cd: 0x000c,
+	0x9e2: 0x000c, 0x9e3: 0x000c,
+	// Block 0x28, offset 0xa00
+	0xa00: 0x000c, 0xa01: 0x000c,
+	0xa3b: 0x000c,
+	0xa3c: 0x000c,
+	// Block 0x29, offset 0xa40
+	0xa41: 0x000c, 0xa42: 0x000c, 0xa43: 0x000c, 0xa44: 0x000c,
+	0xa4d: 0x000c,
+	0xa62: 0x000c, 0xa63: 0x000c,
+	// Block 0x2a, offset 0xa80
+	0xa81: 0x000c,
+	// Block 0x2b, offset 0xac0
+	0xaca: 0x000c,
+	0xad2: 0x000c, 0xad3: 0x000c, 0xad4: 0x000c, 0xad6: 0x000c,
+	// Block 0x2c, offset 0xb00
+	0xb31: 0x000c, 0xb34: 0x000c, 0xb35: 0x000c,
+	0xb36: 0x000c, 0xb37: 0x000c, 0xb38: 0x000c, 0xb39: 0x000c, 0xb3a: 0x000c,
+	0xb3f: 0x0004,
+	// Block 0x2d, offset 0xb40
+	0xb47: 0x000c, 0xb48: 0x000c, 0xb49: 0x000c, 0xb4a: 0x000c, 0xb4b: 0x000c,
+	0xb4c: 0x000c, 0xb4d: 0x000c, 0xb4e: 0x000c,
+	// Block 0x2e, offset 0xb80
+	0xbb1: 0x000c, 0xbb4: 0x000c, 0xbb5: 0x000c,
+	0xbb6: 0x000c, 0xbb7: 0x000c, 0xbb8: 0x000c, 0xbb9: 0x000c, 0xbba: 0x000c, 0xbbb: 0x000c,
+	0xbbc: 0x000c,
+	// Block 0x2f, offset 0xbc0
+	0xbc8: 0x000c, 0xbc9: 0x000c, 0xbca: 0x000c, 0xbcb: 0x000c,
+	0xbcc: 0x000c, 0xbcd: 0x000c,
+	// Block 0x30, offset 0xc00
+	0xc18: 0x000c, 0xc19: 0x000c,
+	0xc35: 0x000c,
+	0xc37: 0x000c, 0xc39: 0x000c, 0xc3a: 0x003a, 0xc3b: 0x002a,
+	0xc3c: 0x003a, 0xc3d: 0x002a,
+	// Block 0x31, offset 0xc40
+	0xc71: 0x000c, 0xc72: 0x000c, 0xc73: 0x000c, 0xc74: 0x000c, 0xc75: 0x000c,
+	0xc76: 0x000c, 0xc77: 0x000c, 0xc78: 0x000c, 0xc79: 0x000c, 0xc7a: 0x000c, 0xc7b: 0x000c,
+	0xc7c: 0x000c, 0xc7d: 0x000c, 0xc7e: 0x000c,
+	// Block 0x32, offset 0xc80
+	0xc80: 0x000c, 0xc81: 0x000c, 0xc82: 0x000c, 0xc83: 0x000c, 0xc84: 0x000c,
+	0xc86: 0x000c, 0xc87: 0x000c,
+	0xc8d: 0x000c, 0xc8e: 0x000c, 0xc8f: 0x000c, 0xc90: 0x000c, 0xc91: 0x000c,
+	0xc92: 0x000c, 0xc93: 0x000c, 0xc94: 0x000c, 0xc95: 0x000c, 0xc96: 0x000c, 0xc97: 0x000c,
+	0xc99: 0x000c, 0xc9a: 0x000c, 0xc9b: 0x000c, 0xc9c: 0x000c, 0xc9d: 0x000c,
+	0xc9e: 0x000c, 0xc9f: 0x000c, 0xca0: 0x000c, 0xca1: 0x000c, 0xca2: 0x000c, 0xca3: 0x000c,
+	0xca4: 0x000c, 0xca5: 0x000c, 0xca6: 0x000c, 0xca7: 0x000c, 0xca8: 0x000c, 0xca9: 0x000c,
+	0xcaa: 0x000c, 0xcab: 0x000c, 0xcac: 0x000c, 0xcad: 0x000c, 0xcae: 0x000c, 0xcaf: 0x000c,
+	0xcb0: 0x000c, 0xcb1: 0x000c, 0xcb2: 0x000c, 0xcb3: 0x000c, 0xcb4: 0x000c, 0xcb5: 0x000c,
+	0xcb6: 0x000c, 0xcb7: 0x000c, 0xcb8: 0x000c, 0xcb9: 0x000c, 0xcba: 0x000c, 0xcbb: 0x000c,
+	0xcbc: 0x000c,
+	// Block 0x33, offset 0xcc0
+	0xcc6: 0x000c,
+	// Block 0x34, offset 0xd00
+	0xd2d: 0x000c, 0xd2e: 0x000c, 0xd2f: 0x000c,
+	0xd30: 0x000c, 0xd32: 0x000c, 0xd33: 0x000c, 0xd34: 0x000c, 0xd35: 0x000c,
+	0xd36: 0x000c, 0xd37: 0x000c, 0xd39: 0x000c, 0xd3a: 0x000c,
+	0xd3d: 0x000c, 0xd3e: 0x000c,
+	// Block 0x35, offset 0xd40
+	0xd58: 0x000c, 0xd59: 0x000c,
+	0xd5e: 0x000c, 0xd5f: 0x000c, 0xd60: 0x000c,
+	0xd71: 0x000c, 0xd72: 0x000c, 0xd73: 0x000c, 0xd74: 0x000c,
+	// Block 0x36, offset 0xd80
+	0xd82: 0x000c, 0xd85: 0x000c,
+	0xd86: 0x000c,
+	0xd8d: 0x000c,
+	0xd9d: 0x000c,
+	// Block 0x37, offset 0xdc0
+	0xddd: 0x000c,
+	0xdde: 0x000c, 0xddf: 0x000c,
+	// Block 0x38, offset 0xe00
+	0xe10: 0x000a, 0xe11: 0x000a,
+	0xe12: 0x000a, 0xe13: 0x000a, 0xe14: 0x000a, 0xe15: 0x000a, 0xe16: 0x000a, 0xe17: 0x000a,
+	0xe18: 0x000a, 0xe19: 0x000a,
+	// Block 0x39, offset 0xe40
+	0xe40: 0x000a,
+	// Block 0x3a, offset 0xe80
+	0xe80: 0x0009,
+	0xe9b: 0x007a, 0xe9c: 0x006a,
+	// Block 0x3b, offset 0xec0
+	0xed2: 0x000c, 0xed3: 0x000c, 0xed4: 0x000c,
+	0xef2: 0x000c, 0xef3: 0x000c, 0xef4: 0x000c,
+	// Block 0x3c, offset 0xf00
+	0xf12: 0x000c, 0xf13: 0x000c,
+	0xf32: 0x000c, 0xf33: 0x000c,
+	// Block 0x3d, offset 0xf40
+	0xf74: 0x000c, 0xf75: 0x000c,
+	0xf77: 0x000c, 0xf78: 0x000c, 0xf79: 0x000c, 0xf7a: 0x000c, 0xf7b: 0x000c,
+	0xf7c: 0x000c, 0xf7d: 0x000c,
+	// Block 0x3e, offset 0xf80
+	0xf86: 0x000c, 0xf89: 0x000c, 0xf8a: 0x000c, 0xf8b: 0x000c,
+	0xf8c: 0x000c, 0xf8d: 0x000c, 0xf8e: 0x000c, 0xf8f: 0x000c, 0xf90: 0x000c, 0xf91: 0x000c,
+	0xf92: 0x000c, 0xf93: 0x000c,
+	0xf9b: 0x0004, 0xf9d: 0x000c,
+	0xfb0: 0x000a, 0xfb1: 0x000a, 0xfb2: 0x000a, 0xfb3: 0x000a, 0xfb4: 0x000a, 0xfb5: 0x000a,
+	0xfb6: 0x000a, 0xfb7: 0x000a, 0xfb8: 0x000a, 0xfb9: 0x000a,
+	// Block 0x3f, offset 0xfc0
+	0xfc0: 0x000a, 0xfc1: 0x000a, 0xfc2: 0x000a, 0xfc3: 0x000a, 0xfc4: 0x000a, 0xfc5: 0x000a,
+	0xfc6: 0x000a, 0xfc7: 0x000a, 0xfc8: 0x000a, 0xfc9: 0x000a, 0xfca: 0x000a, 0xfcb: 0x000c,
+	0xfcc: 0x000c, 0xfcd: 0x000c, 0xfce: 0x000b,
+	// Block 0x40, offset 0x1000
+	0x1005: 0x000c,
+	0x1006: 0x000c,
+	0x1029: 0x000c,
+	// Block 0x41, offset 0x1040
+	0x1060: 0x000c, 0x1061: 0x000c, 0x1062: 0x000c,
+	0x1067: 0x000c, 0x1068: 0x000c,
+	0x1072: 0x000c,
+	0x1079: 0x000c, 0x107a: 0x000c, 0x107b: 0x000c,
+	// Block 0x42, offset 0x1080
+	0x1080: 0x000a, 0x1084: 0x000a, 0x1085: 0x000a,
+	// Block 0x43, offset 0x10c0
+	0x10de: 0x000a, 0x10df: 0x000a, 0x10e0: 0x000a, 0x10e1: 0x000a, 0x10e2: 0x000a, 0x10e3: 0x000a,
+	0x10e4: 0x000a, 0x10e5: 0x000a, 0x10e6: 0x000a, 0x10e7: 0x000a, 0x10e8: 0x000a, 0x10e9: 0x000a,
+	0x10ea: 0x000a, 0x10eb: 0x000a, 0x10ec: 0x000a, 0x10ed: 0x000a, 0x10ee: 0x000a, 0x10ef: 0x000a,
+	0x10f0: 0x000a, 0x10f1: 0x000a, 0x10f2: 0x000a, 0x10f3: 0x000a, 0x10f4: 0x000a, 0x10f5: 0x000a,
+	0x10f6: 0x000a, 0x10f7: 0x000a, 0x10f8: 0x000a, 0x10f9: 0x000a, 0x10fa: 0x000a, 0x10fb: 0x000a,
+	0x10fc: 0x000a, 0x10fd: 0x000a, 0x10fe: 0x000a, 0x10ff: 0x000a,
+	// Block 0x44, offset 0x1100
+	0x1117: 0x000c,
+	0x1118: 0x000c, 0x111b: 0x000c,
+	// Block 0x45, offset 0x1140
+	0x1156: 0x000c,
+	0x1158: 0x000c, 0x1159: 0x000c, 0x115a: 0x000c, 0x115b: 0x000c, 0x115c: 0x000c, 0x115d: 0x000c,
+	0x115e: 0x000c, 0x1160: 0x000c, 0x1162: 0x000c,
+	0x1165: 0x000c, 0x1166: 0x000c, 0x1167: 0x000c, 0x1168: 0x000c, 0x1169: 0x000c,
+	0x116a: 0x000c, 0x116b: 0x000c, 0x116c: 0x000c,
+	0x1173: 0x000c, 0x1174: 0x000c, 0x1175: 0x000c,
+	0x1176: 0x000c, 0x1177: 0x000c, 0x1178: 0x000c, 0x1179: 0x000c, 0x117a: 0x000c, 0x117b: 0x000c,
+	0x117c: 0x000c, 0x117f: 0x000c,
+	// Block 0x46, offset 0x1180
+	0x11b0: 0x000c, 0x11b1: 0x000c, 0x11b2: 0x000c, 0x11b3: 0x000c, 0x11b4: 0x000c, 0x11b5: 0x000c,
+	0x11b6: 0x000c, 0x11b7: 0x000c, 0x11b8: 0x000c, 0x11b9: 0x000c, 0x11ba: 0x000c, 0x11bb: 0x000c,
+	0x11bc: 0x000c, 0x11bd: 0x000c, 0x11be: 0x000c, 0x11bf: 0x000c,
+	// Block 0x47, offset 0x11c0
+	0x11c0: 0x000c,
+	// Block 0x48, offset 0x1200
+	0x1200: 0x000c, 0x1201: 0x000c, 0x1202: 0x000c, 0x1203: 0x000c,
+	0x1234: 0x000c,
+	0x1236: 0x000c, 0x1237: 0x000c, 0x1238: 0x000c, 0x1239: 0x000c, 0x123a: 0x000c,
+	0x123c: 0x000c,
+	// Block 0x49, offset 0x1240
+	0x1242: 0x000c,
+	0x126b: 0x000c, 0x126c: 0x000c, 0x126d: 0x000c, 0x126e: 0x000c, 0x126f: 0x000c,
+	0x1270: 0x000c, 0x1271: 0x000c, 0x1272: 0x000c, 0x1273: 0x000c,
+	// Block 0x4a, offset 0x1280
+	0x1280: 0x000c, 0x1281: 0x000c,
+	0x12a2: 0x000c, 0x12a3: 0x000c,
+	0x12a4: 0x000c, 0x12a5: 0x000c, 0x12a8: 0x000c, 0x12a9: 0x000c,
+	0x12ab: 0x000c, 0x12ac: 0x000c, 0x12ad: 0x000c,
+	// Block 0x4b, offset 0x12c0
+	0x12e6: 0x000c, 0x12e8: 0x000c, 0x12e9: 0x000c,
+	0x12ed: 0x000c, 0x12ef: 0x000c,
+	0x12f0: 0x000c, 0x12f1: 0x000c,
+	// Block 0x4c, offset 0x1300
+	0x132c: 0x000c, 0x132d: 0x000c, 0x132e: 0x000c, 0x132f: 0x000c,
+	0x1330: 0x000c, 0x1331: 0x000c, 0x1332: 0x000c, 0x1333: 0x000c,
+	0x1336: 0x000c, 0x1337: 0x000c,
+	// Block 0x4d, offset 0x1340
+	0x1350: 0x000c, 0x1351: 0x000c,
+	0x1352: 0x000c, 0x1354: 0x000c, 0x1355: 0x000c, 0x1356: 0x000c, 0x1357: 0x000c,
+	0x1358: 0x000c, 0x1359: 0x000c, 0x135a: 0x000c, 0x135b: 0x000c, 0x135c: 0x000c, 0x135d: 0x000c,
+	0x135e: 0x000c, 0x135f: 0x000c, 0x1360: 0x000c, 0x1362: 0x000c, 0x1363: 0x000c,
+	0x1364: 0x000c, 0x1365: 0x000c, 0x1366: 0x000c, 0x1367: 0x000c, 0x1368: 0x000c,
+	0x136d: 0x000c,
+	0x1374: 0x000c,
+	0x1378: 0x000c, 0x1379: 0x000c,
+	// Block 0x4e, offset 0x1380
+	0x1380: 0x000c, 0x1381: 0x000c, 0x1382: 0x000c, 0x1383: 0x000c, 0x1384: 0x000c, 0x1385: 0x000c,
+	0x1386: 0x000c, 0x1387: 0x000c, 0x1388: 0x000c, 0x1389: 0x000c, 0x138a: 0x000c, 0x138b: 0x000c,
+	0x138c: 0x000c, 0x138d: 0x000c, 0x138e: 0x000c, 0x138f: 0x000c, 0x1390: 0x000c, 0x1391: 0x000c,
+	0x1392: 0x000c, 0x1393: 0x000c, 0x1394: 0x000c, 0x1395: 0x000c, 0x1396: 0x000c, 0x1397: 0x000c,
+	0x1398: 0x000c, 0x1399: 0x000c, 0x139a: 0x000c, 0x139b: 0x000c, 0x139c: 0x000c, 0x139d: 0x000c,
+	0x139e: 0x000c, 0x139f: 0x000c, 0x13a0: 0x000c, 0x13a1: 0x000c, 0x13a2: 0x000c, 0x13a3: 0x000c,
+	0x13a4: 0x000c, 0x13a5: 0x000c, 0x13a6: 0x000c, 0x13a7: 0x000c, 0x13a8: 0x000c, 0x13a9: 0x000c,
+	0x13aa: 0x000c, 0x13ab: 0x000c, 0x13ac: 0x000c, 0x13ad: 0x000c, 0x13ae: 0x000c, 0x13af: 0x000c,
+	0x13b0: 0x000c, 0x13b1: 0x000c, 0x13b2: 0x000c, 0x13b3: 0x000c, 0x13b4: 0x000c, 0x13b5: 0x000c,
+	0x13b6: 0x000c, 0x13b7: 0x000c, 0x13b8: 0x000c, 0x13b9: 0x000c, 0x13bb: 0x000c,
+	0x13bc: 0x000c, 0x13bd: 0x000c, 0x13be: 0x000c, 0x13bf: 0x000c,
+	// Block 0x4f, offset 0x13c0
+	0x13fd: 0x000a, 0x13ff: 0x000a,
+	// Block 0x50, offset 0x1400
+	0x1400: 0x000a, 0x1401: 0x000a,
+	0x140d: 0x000a, 0x140e: 0x000a, 0x140f: 0x000a,
+	0x141d: 0x000a,
+	0x141e: 0x000a, 0x141f: 0x000a,
+	0x142d: 0x000a, 0x142e: 0x000a, 0x142f: 0x000a,
+	0x143d: 0x000a, 0x143e: 0x000a,
+	// Block 0x51, offset 0x1440
+	0x1440: 0x0009, 0x1441: 0x0009, 0x1442: 0x0009, 0x1443: 0x0009, 0x1444: 0x0009, 0x1445: 0x0009,
+	0x1446: 0x0009, 0x1447: 0x0009, 0x1448: 0x0009, 0x1449: 0x0009, 0x144a: 0x0009, 0x144b: 0x000b,
+	0x144c: 0x000b, 0x144d: 0x000b, 0x144f: 0x0001, 0x1450: 0x000a, 0x1451: 0x000a,
+	0x1452: 0x000a, 0x1453: 0x000a, 0x1454: 0x000a, 0x1455: 0x000a, 0x1456: 0x000a, 0x1457: 0x000a,
+	0x1458: 0x000a, 0x1459: 0x000a, 0x145a: 0x000a, 0x145b: 0x000a, 0x145c: 0x000a, 0x145d: 0x000a,
+	0x145e: 0x000a, 0x145f: 0x000a, 0x1460: 0x000a, 0x1461: 0x000a, 0x1462: 0x000a, 0x1463: 0x000a,
+	0x1464: 0x000a, 0x1465: 0x000a, 0x1466: 0x000a, 0x1467: 0x000a, 0x1468: 0x0009, 0x1469: 0x0007,
+	0x146a: 0x000e, 0x146b: 0x000e, 0x146c: 0x000e, 0x146d: 0x000e, 0x146e: 0x000e, 0x146f: 0x0006,
+	0x1470: 0x0004, 0x1471: 0x0004, 0x1472: 0x0004, 0x1473: 0x0004, 0x1474: 0x0004, 0x1475: 0x000a,
+	0x1476: 0x000a, 0x1477: 0x000a, 0x1478: 0x000a, 0x1479: 0x000a, 0x147a: 0x000a, 0x147b: 0x000a,
+	0x147c: 0x000a, 0x147d: 0x000a, 0x147e: 0x000a, 0x147f: 0x000a,
+	// Block 0x52, offset 0x1480
+	0x1480: 0x000a, 0x1481: 0x000a, 0x1482: 0x000a, 0x1483: 0x000a, 0x1484: 0x0006, 0x1485: 0x009a,
+	0x1486: 0x008a, 0x1487: 0x000a, 0x1488: 0x000a, 0x1489: 0x000a, 0x148a: 0x000a, 0x148b: 0x000a,
+	0x148c: 0x000a, 0x148d: 0x000a, 0x148e: 0x000a, 0x148f: 0x000a, 0x1490: 0x000a, 0x1491: 0x000a,
+	0x1492: 0x000a, 0x1493: 0x000a, 0x1494: 0x000a, 0x1495: 0x000a, 0x1496: 0x000a, 0x1497: 0x000a,
+	0x1498: 0x000a, 0x1499: 0x000a, 0x149a: 0x000a, 0x149b: 0x000a, 0x149c: 0x000a, 0x149d: 0x000a,
+	0x149e: 0x000a, 0x149f: 0x0009, 0x14a0: 0x000b, 0x14a1: 0x000b, 0x14a2: 0x000b, 0x14a3: 0x000b,
+	0x14a4: 0x000b, 0x14a5: 0x000b, 0x14a6: 0x000e, 0x14a7: 0x000e, 0x14a8: 0x000e, 0x14a9: 0x000e,
+	0x14aa: 0x000b, 0x14ab: 0x000b, 0x14ac: 0x000b, 0x14ad: 0x000b, 0x14ae: 0x000b, 0x14af: 0x000b,
+	0x14b0: 0x0002, 0x14b4: 0x0002, 0x14b5: 0x0002,
+	0x14b6: 0x0002, 0x14b7: 0x0002, 0x14b8: 0x0002, 0x14b9: 0x0002, 0x14ba: 0x0003, 0x14bb: 0x0003,
+	0x14bc: 0x000a, 0x14bd: 0x009a, 0x14be: 0x008a,
+	// Block 0x53, offset 0x14c0
+	0x14c0: 0x0002, 0x14c1: 0x0002, 0x14c2: 0x0002, 0x14c3: 0x0002, 0x14c4: 0x0002, 0x14c5: 0x0002,
+	0x14c6: 0x0002, 0x14c7: 0x0002, 0x14c8: 0x0002, 0x14c9: 0x0002, 0x14ca: 0x0003, 0x14cb: 0x0003,
+	0x14cc: 0x000a, 0x14cd: 0x009a, 0x14ce: 0x008a,
+	0x14e0: 0x0004, 0x14e1: 0x0004, 0x14e2: 0x0004, 0x14e3: 0x0004,
+	0x14e4: 0x0004, 0x14e5: 0x0004, 0x14e6: 0x0004, 0x14e7: 0x0004, 0x14e8: 0x0004, 0x14e9: 0x0004,
+	0x14ea: 0x0004, 0x14eb: 0x0004, 0x14ec: 0x0004, 0x14ed: 0x0004, 0x14ee: 0x0004, 0x14ef: 0x0004,
+	0x14f0: 0x0004, 0x14f1: 0x0004, 0x14f2: 0x0004, 0x14f3: 0x0004, 0x14f4: 0x0004, 0x14f5: 0x0004,
+	0x14f6: 0x0004, 0x14f7: 0x0004, 0x14f8: 0x0004, 0x14f9: 0x0004, 0x14fa: 0x0004, 0x14fb: 0x0004,
+	0x14fc: 0x0004, 0x14fd: 0x0004, 0x14fe: 0x0004, 0x14ff: 0x0004,
+	// Block 0x54, offset 0x1500
+	0x1500: 0x0004, 0x1501: 0x0004, 0x1502: 0x0004, 0x1503: 0x0004, 0x1504: 0x0004, 0x1505: 0x0004,
+	0x1506: 0x0004, 0x1507: 0x0004, 0x1508: 0x0004, 0x1509: 0x0004, 0x150a: 0x0004, 0x150b: 0x0004,
+	0x150c: 0x0004, 0x150d: 0x0004, 0x150e: 0x0004, 0x150f: 0x0004, 0x1510: 0x000c, 0x1511: 0x000c,
+	0x1512: 0x000c, 0x1513: 0x000c, 0x1514: 0x000c, 0x1515: 0x000c, 0x1516: 0x000c, 0x1517: 0x000c,
+	0x1518: 0x000c, 0x1519: 0x000c, 0x151a: 0x000c, 0x151b: 0x000c, 0x151c: 0x000c, 0x151d: 0x000c,
+	0x151e: 0x000c, 0x151f: 0x000c, 0x1520: 0x000c, 0x1521: 0x000c, 0x1522: 0x000c, 0x1523: 0x000c,
+	0x1524: 0x000c, 0x1525: 0x000c, 0x1526: 0x000c, 0x1527: 0x000c, 0x1528: 0x000c, 0x1529: 0x000c,
+	0x152a: 0x000c, 0x152b: 0x000c, 0x152c: 0x000c, 0x152d: 0x000c, 0x152e: 0x000c, 0x152f: 0x000c,
+	0x1530: 0x000c,
+	// Block 0x55, offset 0x1540
+	0x1540: 0x000a, 0x1541: 0x000a, 0x1543: 0x000a, 0x1544: 0x000a, 0x1545: 0x000a,
+	0x1546: 0x000a, 0x1548: 0x000a, 0x1549: 0x000a,
+	0x1554: 0x000a, 0x1556: 0x000a, 0x1557: 0x000a,
+	0x1558: 0x000a,
+	0x155e: 0x000a, 0x155f: 0x000a, 0x1560: 0x000a, 0x1561: 0x000a, 0x1562: 0x000a, 0x1563: 0x000a,
+	0x1565: 0x000a, 0x1567: 0x000a, 0x1569: 0x000a,
+	0x156e: 0x0004,
+	0x157a: 0x000a, 0x157b: 0x000a,
+	// Block 0x56, offset 0x1580
+	0x1580: 0x000a, 0x1581: 0x000a, 0x1582: 0x000a, 0x1583: 0x000a, 0x1584: 0x000a,
+	0x158a: 0x000a, 0x158b: 0x000a,
+	0x158c: 0x000a, 0x158d: 0x000a, 0x1590: 0x000a, 0x1591: 0x000a,
+	0x1592: 0x000a, 0x1593: 0x000a, 0x1594: 0x000a, 0x1595: 0x000a, 0x1596: 0x000a, 0x1597: 0x000a,
+	0x1598: 0x000a, 0x1599: 0x000a, 0x159a: 0x000a, 0x159b: 0x000a, 0x159c: 0x000a, 0x159d: 0x000a,
+	0x159e: 0x000a, 0x159f: 0x000a,
+	// Block 0x57, offset 0x15c0
+	0x15c9: 0x000a, 0x15ca: 0x000a, 0x15cb: 0x000a,
+	0x15d0: 0x000a, 0x15d1: 0x000a,
+	0x15d2: 0x000a, 0x15d3: 0x000a, 0x15d4: 0x000a, 0x15d5: 0x000a, 0x15d6: 0x000a, 0x15d7: 0x000a,
+	0x15d8: 0x000a, 0x15d9: 0x000a, 0x15da: 0x000a, 0x15db: 0x000a, 0x15dc: 0x000a, 0x15dd: 0x000a,
+	0x15de: 0x000a, 0x15df: 0x000a, 0x15e0: 0x000a, 0x15e1: 0x000a, 0x15e2: 0x000a, 0x15e3: 0x000a,
+	0x15e4: 0x000a, 0x15e5: 0x000a, 0x15e6: 0x000a, 0x15e7: 0x000a, 0x15e8: 0x000a, 0x15e9: 0x000a,
+	0x15ea: 0x000a, 0x15eb: 0x000a, 0x15ec: 0x000a, 0x15ed: 0x000a, 0x15ee: 0x000a, 0x15ef: 0x000a,
+	0x15f0: 0x000a, 0x15f1: 0x000a, 0x15f2: 0x000a, 0x15f3: 0x000a, 0x15f4: 0x000a, 0x15f5: 0x000a,
+	0x15f6: 0x000a, 0x15f7: 0x000a, 0x15f8: 0x000a, 0x15f9: 0x000a, 0x15fa: 0x000a, 0x15fb: 0x000a,
+	0x15fc: 0x000a, 0x15fd: 0x000a, 0x15fe: 0x000a, 0x15ff: 0x000a,
+	// Block 0x58, offset 0x1600
+	0x1600: 0x000a, 0x1601: 0x000a, 0x1602: 0x000a, 0x1603: 0x000a, 0x1604: 0x000a, 0x1605: 0x000a,
+	0x1606: 0x000a, 0x1607: 0x000a, 0x1608: 0x000a, 0x1609: 0x000a, 0x160a: 0x000a, 0x160b: 0x000a,
+	0x160c: 0x000a, 0x160d: 0x000a, 0x160e: 0x000a, 0x160f: 0x000a, 0x1610: 0x000a, 0x1611: 0x000a,
+	0x1612: 0x000a, 0x1613: 0x000a, 0x1614: 0x000a, 0x1615: 0x000a, 0x1616: 0x000a, 0x1617: 0x000a,
+	0x1618: 0x000a, 0x1619: 0x000a, 0x161a: 0x000a, 0x161b: 0x000a, 0x161c: 0x000a, 0x161d: 0x000a,
+	0x161e: 0x000a, 0x161f: 0x000a, 0x1620: 0x000a, 0x1621: 0x000a, 0x1622: 0x000a, 0x1623: 0x000a,
+	0x1624: 0x000a, 0x1625: 0x000a, 0x1626: 0x000a, 0x1627: 0x000a, 0x1628: 0x000a, 0x1629: 0x000a,
+	0x162a: 0x000a, 0x162b: 0x000a, 0x162c: 0x000a, 0x162d: 0x000a, 0x162e: 0x000a, 0x162f: 0x000a,
+	0x1630: 0x000a, 0x1631: 0x000a, 0x1632: 0x000a, 0x1633: 0x000a, 0x1634: 0x000a, 0x1635: 0x000a,
+	0x1636: 0x000a, 0x1637: 0x000a, 0x1638: 0x000a, 0x1639: 0x000a, 0x163a: 0x000a, 0x163b: 0x000a,
+	0x163c: 0x000a, 0x163d: 0x000a, 0x163e: 0x000a, 0x163f: 0x000a,
+	// Block 0x59, offset 0x1640
+	0x1640: 0x000a, 0x1641: 0x000a, 0x1642: 0x000a, 0x1643: 0x000a, 0x1644: 0x000a, 0x1645: 0x000a,
+	0x1646: 0x000a, 0x1647: 0x000a, 0x1648: 0x000a, 0x1649: 0x000a, 0x164a: 0x000a, 0x164b: 0x000a,
+	0x164c: 0x000a, 0x164d: 0x000a, 0x164e: 0x000a, 0x164f: 0x000a, 0x1650: 0x000a, 0x1651: 0x000a,
+	0x1652: 0x0003, 0x1653: 0x0004, 0x1654: 0x000a, 0x1655: 0x000a, 0x1656: 0x000a, 0x1657: 0x000a,
+	0x1658: 0x000a, 0x1659: 0x000a, 0x165a: 0x000a, 0x165b: 0x000a, 0x165c: 0x000a, 0x165d: 0x000a,
+	0x165e: 0x000a, 0x165f: 0x000a, 0x1660: 0x000a, 0x1661: 0x000a, 0x1662: 0x000a, 0x1663: 0x000a,
+	0x1664: 0x000a, 0x1665: 0x000a, 0x1666: 0x000a, 0x1667: 0x000a, 0x1668: 0x000a, 0x1669: 0x000a,
+	0x166a: 0x000a, 0x166b: 0x000a, 0x166c: 0x000a, 0x166d: 0x000a, 0x166e: 0x000a, 0x166f: 0x000a,
+	0x1670: 0x000a, 0x1671: 0x000a, 0x1672: 0x000a, 0x1673: 0x000a, 0x1674: 0x000a, 0x1675: 0x000a,
+	0x1676: 0x000a, 0x1677: 0x000a, 0x1678: 0x000a, 0x1679: 0x000a, 0x167a: 0x000a, 0x167b: 0x000a,
+	0x167c: 0x000a, 0x167d: 0x000a, 0x167e: 0x000a, 0x167f: 0x000a,
+	// Block 0x5a, offset 0x1680
+	0x1680: 0x000a, 0x1681: 0x000a, 0x1682: 0x000a, 0x1683: 0x000a, 0x1684: 0x000a, 0x1685: 0x000a,
+	0x1686: 0x000a, 0x1687: 0x000a, 0x1688: 0x003a, 0x1689: 0x002a, 0x168a: 0x003a, 0x168b: 0x002a,
+	0x168c: 0x000a, 0x168d: 0x000a, 0x168e: 0x000a, 0x168f: 0x000a, 0x1690: 0x000a, 0x1691: 0x000a,
+	0x1692: 0x000a, 0x1693: 0x000a, 0x1694: 0x000a, 0x1695: 0x000a, 0x1696: 0x000a, 0x1697: 0x000a,
+	0x1698: 0x000a, 0x1699: 0x000a, 0x169a: 0x000a, 0x169b: 0x000a, 0x169c: 0x000a, 0x169d: 0x000a,
+	0x169e: 0x000a, 0x169f: 0x000a, 0x16a0: 0x000a, 0x16a1: 0x000a, 0x16a2: 0x000a, 0x16a3: 0x000a,
+	0x16a4: 0x000a, 0x16a5: 0x000a, 0x16a6: 0x000a, 0x16a7: 0x000a, 0x16a8: 0x000a, 0x16a9: 0x009a,
+	0x16aa: 0x008a, 0x16ab: 0x000a, 0x16ac: 0x000a, 0x16ad: 0x000a, 0x16ae: 0x000a, 0x16af: 0x000a,
+	0x16b0: 0x000a, 0x16b1: 0x000a, 0x16b2: 0x000a, 0x16b3: 0x000a, 0x16b4: 0x000a, 0x16b5: 0x000a,
+	// Block 0x5b, offset 0x16c0
+	0x16fb: 0x000a,
+	0x16fc: 0x000a, 0x16fd: 0x000a, 0x16fe: 0x000a, 0x16ff: 0x000a,
+	// Block 0x5c, offset 0x1700
+	0x1700: 0x000a, 0x1701: 0x000a, 0x1702: 0x000a, 0x1703: 0x000a, 0x1704: 0x000a, 0x1705: 0x000a,
+	0x1706: 0x000a, 0x1707: 0x000a, 0x1708: 0x000a, 0x1709: 0x000a, 0x170a: 0x000a, 0x170b: 0x000a,
+	0x170c: 0x000a, 0x170d: 0x000a, 0x170e: 0x000a, 0x170f: 0x000a, 0x1710: 0x000a, 0x1711: 0x000a,
+	0x1712: 0x000a, 0x1713: 0x000a, 0x1714: 0x000a, 0x1716: 0x000a, 0x1717: 0x000a,
+	0x1718: 0x000a, 0x1719: 0x000a, 0x171a: 0x000a, 0x171b: 0x000a, 0x171c: 0x000a, 0x171d: 0x000a,
+	0x171e: 0x000a, 0x171f: 0x000a, 0x1720: 0x000a, 0x1721: 0x000a, 0x1722: 0x000a, 0x1723: 0x000a,
+	0x1724: 0x000a, 0x1725: 0x000a, 0x1726: 0x000a, 0x1727: 0x000a, 0x1728: 0x000a, 0x1729: 0x000a,
+	0x172a: 0x000a, 0x172b: 0x000a, 0x172c: 0x000a, 0x172d: 0x000a, 0x172e: 0x000a, 0x172f: 0x000a,
+	0x1730: 0x000a, 0x1731: 0x000a, 0x1732: 0x000a, 0x1733: 0x000a, 0x1734: 0x000a, 0x1735: 0x000a,
+	0x1736: 0x000a, 0x1737: 0x000a, 0x1738: 0x000a, 0x1739: 0x000a, 0x173a: 0x000a, 0x173b: 0x000a,
+	0x173c: 0x000a, 0x173d: 0x000a, 0x173e: 0x000a, 0x173f: 0x000a,
+	// Block 0x5d, offset 0x1740
+	0x1740: 0x000a, 0x1741: 0x000a, 0x1742: 0x000a, 0x1743: 0x000a, 0x1744: 0x000a, 0x1745: 0x000a,
+	0x1746: 0x000a, 0x1747: 0x000a, 0x1748: 0x000a, 0x1749: 0x000a, 0x174a: 0x000a, 0x174b: 0x000a,
+	0x174c: 0x000a, 0x174d: 0x000a, 0x174e: 0x000a, 0x174f: 0x000a, 0x1750: 0x000a, 0x1751: 0x000a,
+	0x1752: 0x000a, 0x1753: 0x000a, 0x1754: 0x000a, 0x1755: 0x000a, 0x1756: 0x000a, 0x1757: 0x000a,
+	0x1758: 0x000a, 0x1759: 0x000a, 0x175a: 0x000a, 0x175b: 0x000a, 0x175c: 0x000a, 0x175d: 0x000a,
+	0x175e: 0x000a, 0x175f: 0x000a, 0x1760: 0x000a, 0x1761: 0x000a, 0x1762: 0x000a, 0x1763: 0x000a,
+	0x1764: 0x000a, 0x1765: 0x000a, 0x1766: 0x000a,
+	// Block 0x5e, offset 0x1780
+	0x1780: 0x000a, 0x1781: 0x000a, 0x1782: 0x000a, 0x1783: 0x000a, 0x1784: 0x000a, 0x1785: 0x000a,
+	0x1786: 0x000a, 0x1787: 0x000a, 0x1788: 0x000a, 0x1789: 0x000a, 0x178a: 0x000a,
+	0x17a0: 0x000a, 0x17a1: 0x000a, 0x17a2: 0x000a, 0x17a3: 0x000a,
+	0x17a4: 0x000a, 0x17a5: 0x000a, 0x17a6: 0x000a, 0x17a7: 0x000a, 0x17a8: 0x000a, 0x17a9: 0x000a,
+	0x17aa: 0x000a, 0x17ab: 0x000a, 0x17ac: 0x000a, 0x17ad: 0x000a, 0x17ae: 0x000a, 0x17af: 0x000a,
+	0x17b0: 0x000a, 0x17b1: 0x000a, 0x17b2: 0x000a, 0x17b3: 0x000a, 0x17b4: 0x000a, 0x17b5: 0x000a,
+	0x17b6: 0x000a, 0x17b7: 0x000a, 0x17b8: 0x000a, 0x17b9: 0x000a, 0x17ba: 0x000a, 0x17bb: 0x000a,
+	0x17bc: 0x000a, 0x17bd: 0x000a, 0x17be: 0x000a, 0x17bf: 0x000a,
+	// Block 0x5f, offset 0x17c0
+	0x17c0: 0x000a, 0x17c1: 0x000a, 0x17c2: 0x000a, 0x17c3: 0x000a, 0x17c4: 0x000a, 0x17c5: 0x000a,
+	0x17c6: 0x000a, 0x17c7: 0x000a, 0x17c8: 0x0002, 0x17c9: 0x0002, 0x17ca: 0x0002, 0x17cb: 0x0002,
+	0x17cc: 0x0002, 0x17cd: 0x0002, 0x17ce: 0x0002, 0x17cf: 0x0002, 0x17d0: 0x0002, 0x17d1: 0x0002,
+	0x17d2: 0x0002, 0x17d3: 0x0002, 0x17d4: 0x0002, 0x17d5: 0x0002, 0x17d6: 0x0002, 0x17d7: 0x0002,
+	0x17d8: 0x0002, 0x17d9: 0x0002, 0x17da: 0x0002, 0x17db: 0x0002,
+	// Block 0x60, offset 0x1800
+	0x182a: 0x000a, 0x182b: 0x000a, 0x182c: 0x000a, 0x182d: 0x000a, 0x182e: 0x000a, 0x182f: 0x000a,
+	0x1830: 0x000a, 0x1831: 0x000a, 0x1832: 0x000a, 0x1833: 0x000a, 0x1834: 0x000a, 0x1835: 0x000a,
+	0x1836: 0x000a, 0x1837: 0x000a, 0x1838: 0x000a, 0x1839: 0x000a, 0x183a: 0x000a, 0x183b: 0x000a,
+	0x183c: 0x000a, 0x183d: 0x000a, 0x183e: 0x000a, 0x183f: 0x000a,
+	// Block 0x61, offset 0x1840
+	0x1840: 0x000a, 0x1841: 0x000a, 0x1842: 0x000a, 0x1843: 0x000a, 0x1844: 0x000a, 0x1845: 0x000a,
+	0x1846: 0x000a, 0x1847: 0x000a, 0x1848: 0x000a, 0x1849: 0x000a, 0x184a: 0x000a, 0x184b: 0x000a,
+	0x184c: 0x000a, 0x184d: 0x000a, 0x184e: 0x000a, 0x184f: 0x000a, 0x1850: 0x000a, 0x1851: 0x000a,
+	0x1852: 0x000a, 0x1853: 0x000a, 0x1854: 0x000a, 0x1855: 0x000a, 0x1856: 0x000a, 0x1857: 0x000a,
+	0x1858: 0x000a, 0x1859: 0x000a, 0x185a: 0x000a, 0x185b: 0x000a, 0x185c: 0x000a, 0x185d: 0x000a,
+	0x185e: 0x000a, 0x185f: 0x000a, 0x1860: 0x000a, 0x1861: 0x000a, 0x1862: 0x000a, 0x1863: 0x000a,
+	0x1864: 0x000a, 0x1865: 0x000a, 0x1866: 0x000a, 0x1867: 0x000a, 0x1868: 0x000a, 0x1869: 0x000a,
+	0x186a: 0x000a, 0x186b: 0x000a, 0x186d: 0x000a, 0x186e: 0x000a, 0x186f: 0x000a,
+	0x1870: 0x000a, 0x1871: 0x000a, 0x1872: 0x000a, 0x1873: 0x000a, 0x1874: 0x000a, 0x1875: 0x000a,
+	0x1876: 0x000a, 0x1877: 0x000a, 0x1878: 0x000a, 0x1879: 0x000a, 0x187a: 0x000a, 0x187b: 0x000a,
+	0x187c: 0x000a, 0x187d: 0x000a, 0x187e: 0x000a, 0x187f: 0x000a,
+	// Block 0x62, offset 0x1880
+	0x1880: 0x000a, 0x1881: 0x000a, 0x1882: 0x000a, 0x1883: 0x000a, 0x1884: 0x000a, 0x1885: 0x000a,
+	0x1886: 0x000a, 0x1887: 0x000a, 0x1888: 0x000a, 0x1889: 0x000a, 0x188a: 0x000a, 0x188b: 0x000a,
+	0x188c: 0x000a, 0x188d: 0x000a, 0x188e: 0x000a, 0x188f: 0x000a, 0x1890: 0x000a, 0x1891: 0x000a,
+	0x1892: 0x000a, 0x1893: 0x000a, 0x1894: 0x000a, 0x1895: 0x000a, 0x1896: 0x000a, 0x1897: 0x000a,
+	0x1898: 0x000a, 0x1899: 0x000a, 0x189a: 0x000a, 0x189b: 0x000a, 0x189c: 0x000a, 0x189d: 0x000a,
+	0x189e: 0x000a, 0x189f: 0x000a, 0x18a0: 0x000a, 0x18a1: 0x000a, 0x18a2: 0x000a, 0x18a3: 0x000a,
+	0x18a4: 0x000a, 0x18a5: 0x000a, 0x18a6: 0x000a, 0x18a7: 0x000a, 0x18a8: 0x003a, 0x18a9: 0x002a,
+	0x18aa: 0x003a, 0x18ab: 0x002a, 0x18ac: 0x003a, 0x18ad: 0x002a, 0x18ae: 0x003a, 0x18af: 0x002a,
+	0x18b0: 0x003a, 0x18b1: 0x002a, 0x18b2: 0x003a, 0x18b3: 0x002a, 0x18b4: 0x003a, 0x18b5: 0x002a,
+	0x18b6: 0x000a, 0x18b7: 0x000a, 0x18b8: 0x000a, 0x18b9: 0x000a, 0x18ba: 0x000a, 0x18bb: 0x000a,
+	0x18bc: 0x000a, 0x18bd: 0x000a, 0x18be: 0x000a, 0x18bf: 0x000a,
+	// Block 0x63, offset 0x18c0
+	0x18c0: 0x000a, 0x18c1: 0x000a, 0x18c2: 0x000a, 0x18c3: 0x000a, 0x18c4: 0x000a, 0x18c5: 0x009a,
+	0x18c6: 0x008a, 0x18c7: 0x000a, 0x18c8: 0x000a, 0x18c9: 0x000a, 0x18ca: 0x000a, 0x18cb: 0x000a,
+	0x18cc: 0x000a, 0x18cd: 0x000a, 0x18ce: 0x000a, 0x18cf: 0x000a, 0x18d0: 0x000a, 0x18d1: 0x000a,
+	0x18d2: 0x000a, 0x18d3: 0x000a, 0x18d4: 0x000a, 0x18d5: 0x000a, 0x18d6: 0x000a, 0x18d7: 0x000a,
+	0x18d8: 0x000a, 0x18d9: 0x000a, 0x18da: 0x000a, 0x18db: 0x000a, 0x18dc: 0x000a, 0x18dd: 0x000a,
+	0x18de: 0x000a, 0x18df: 0x000a, 0x18e0: 0x000a, 0x18e1: 0x000a, 0x18e2: 0x000a, 0x18e3: 0x000a,
+	0x18e4: 0x000a, 0x18e5: 0x000a, 0x18e6: 0x003a, 0x18e7: 0x002a, 0x18e8: 0x003a, 0x18e9: 0x002a,
+	0x18ea: 0x003a, 0x18eb: 0x002a, 0x18ec: 0x003a, 0x18ed: 0x002a, 0x18ee: 0x003a, 0x18ef: 0x002a,
+	0x18f0: 0x000a, 0x18f1: 0x000a, 0x18f2: 0x000a, 0x18f3: 0x000a, 0x18f4: 0x000a, 0x18f5: 0x000a,
+	0x18f6: 0x000a, 0x18f7: 0x000a, 0x18f8: 0x000a, 0x18f9: 0x000a, 0x18fa: 0x000a, 0x18fb: 0x000a,
+	0x18fc: 0x000a, 0x18fd: 0x000a, 0x18fe: 0x000a, 0x18ff: 0x000a,
+	// Block 0x64, offset 0x1900
+	0x1900: 0x000a, 0x1901: 0x000a, 0x1902: 0x000a, 0x1903: 0x007a, 0x1904: 0x006a, 0x1905: 0x009a,
+	0x1906: 0x008a, 0x1907: 0x00ba, 0x1908: 0x00aa, 0x1909: 0x009a, 0x190a: 0x008a, 0x190b: 0x007a,
+	0x190c: 0x006a, 0x190d: 0x00da, 0x190e: 0x002a, 0x190f: 0x003a, 0x1910: 0x00ca, 0x1911: 0x009a,
+	0x1912: 0x008a, 0x1913: 0x007a, 0x1914: 0x006a, 0x1915: 0x009a, 0x1916: 0x008a, 0x1917: 0x00ba,
+	0x1918: 0x00aa, 0x1919: 0x000a, 0x191a: 0x000a, 0x191b: 0x000a, 0x191c: 0x000a, 0x191d: 0x000a,
+	0x191e: 0x000a, 0x191f: 0x000a, 0x1920: 0x000a, 0x1921: 0x000a, 0x1922: 0x000a, 0x1923: 0x000a,
+	0x1924: 0x000a, 0x1925: 0x000a, 0x1926: 0x000a, 0x1927: 0x000a, 0x1928: 0x000a, 0x1929: 0x000a,
+	0x192a: 0x000a, 0x192b: 0x000a, 0x192c: 0x000a, 0x192d: 0x000a, 0x192e: 0x000a, 0x192f: 0x000a,
+	0x1930: 0x000a, 0x1931: 0x000a, 0x1932: 0x000a, 0x1933: 0x000a, 0x1934: 0x000a, 0x1935: 0x000a,
+	0x1936: 0x000a, 0x1937: 0x000a, 0x1938: 0x000a, 0x1939: 0x000a, 0x193a: 0x000a, 0x193b: 0x000a,
+	0x193c: 0x000a, 0x193d: 0x000a, 0x193e: 0x000a, 0x193f: 0x000a,
+	// Block 0x65, offset 0x1940
+	0x1940: 0x000a, 0x1941: 0x000a, 0x1942: 0x000a, 0x1943: 0x000a, 0x1944: 0x000a, 0x1945: 0x000a,
+	0x1946: 0x000a, 0x1947: 0x000a, 0x1948: 0x000a, 0x1949: 0x000a, 0x194a: 0x000a, 0x194b: 0x000a,
+	0x194c: 0x000a, 0x194d: 0x000a, 0x194e: 0x000a, 0x194f: 0x000a, 0x1950: 0x000a, 0x1951: 0x000a,
+	0x1952: 0x000a, 0x1953: 0x000a, 0x1954: 0x000a, 0x1955: 0x000a, 0x1956: 0x000a, 0x1957: 0x000a,
+	0x1958: 0x003a, 0x1959: 0x002a, 0x195a: 0x003a, 0x195b: 0x002a, 0x195c: 0x000a, 0x195d: 0x000a,
+	0x195e: 0x000a, 0x195f: 0x000a, 0x1960: 0x000a, 0x1961: 0x000a, 0x1962: 0x000a, 0x1963: 0x000a,
+	0x1964: 0x000a, 0x1965: 0x000a, 0x1966: 0x000a, 0x1967: 0x000a, 0x1968: 0x000a, 0x1969: 0x000a,
+	0x196a: 0x000a, 0x196b: 0x000a, 0x196c: 0x000a, 0x196d: 0x000a, 0x196e: 0x000a, 0x196f: 0x000a,
+	0x1970: 0x000a, 0x1971: 0x000a, 0x1972: 0x000a, 0x1973: 0x000a, 0x1974: 0x000a, 0x1975: 0x000a,
+	0x1976: 0x000a, 0x1977: 0x000a, 0x1978: 0x000a, 0x1979: 0x000a, 0x197a: 0x000a, 0x197b: 0x000a,
+	0x197c: 0x003a, 0x197d: 0x002a, 0x197e: 0x000a, 0x197f: 0x000a,
+	// Block 0x66, offset 0x1980
+	0x1980: 0x000a, 0x1981: 0x000a, 0x1982: 0x000a, 0x1983: 0x000a, 0x1984: 0x000a, 0x1985: 0x000a,
+	0x1986: 0x000a, 0x1987: 0x000a, 0x1988: 0x000a, 0x1989: 0x000a, 0x198a: 0x000a, 0x198b: 0x000a,
+	0x198c: 0x000a, 0x198d: 0x000a, 0x198e: 0x000a, 0x198f: 0x000a, 0x1990: 0x000a, 0x1991: 0x000a,
+	0x1992: 0x000a, 0x1993: 0x000a, 0x1994: 0x000a, 0x1995: 0x000a, 0x1996: 0x000a, 0x1997: 0x000a,
+	0x1998: 0x000a, 0x1999: 0x000a, 0x199a: 0x000a, 0x199b: 0x000a, 0x199c: 0x000a, 0x199d: 0x000a,
+	0x199e: 0x000a, 0x199f: 0x000a, 0x19a0: 0x000a, 0x19a1: 0x000a, 0x19a2: 0x000a, 0x19a3: 0x000a,
+	0x19a4: 0x000a, 0x19a5: 0x000a, 0x19a6: 0x000a, 0x19a7: 0x000a, 0x19a8: 0x000a, 0x19a9: 0x000a,
+	0x19aa: 0x000a, 0x19ab: 0x000a, 0x19ac: 0x000a, 0x19ad: 0x000a, 0x19ae: 0x000a, 0x19af: 0x000a,
+	0x19b0: 0x000a, 0x19b1: 0x000a, 0x19b2: 0x000a, 0x19b3: 0x000a,
+	0x19b6: 0x000a, 0x19b7: 0x000a, 0x19b8: 0x000a, 0x19b9: 0x000a, 0x19ba: 0x000a, 0x19bb: 0x000a,
+	0x19bc: 0x000a, 0x19bd: 0x000a, 0x19be: 0x000a, 0x19bf: 0x000a,
+	// Block 0x67, offset 0x19c0
+	0x19c0: 0x000a, 0x19c1: 0x000a, 0x19c2: 0x000a, 0x19c3: 0x000a, 0x19c4: 0x000a, 0x19c5: 0x000a,
+	0x19c6: 0x000a, 0x19c7: 0x000a, 0x19c8: 0x000a, 0x19c9: 0x000a, 0x19ca: 0x000a, 0x19cb: 0x000a,
+	0x19cc: 0x000a, 0x19cd: 0x000a, 0x19ce: 0x000a, 0x19cf: 0x000a, 0x19d0: 0x000a, 0x19d1: 0x000a,
+	0x19d2: 0x000a, 0x19d3: 0x000a, 0x19d4: 0x000a, 0x19d5: 0x000a, 0x19d7: 0x000a,
+	0x19d8: 0x000a, 0x19d9: 0x000a, 0x19da: 0x000a, 0x19db: 0x000a, 0x19dc: 0x000a, 0x19dd: 0x000a,
+	0x19de: 0x000a, 0x19df: 0x000a, 0x19e0: 0x000a, 0x19e1: 0x000a, 0x19e2: 0x000a, 0x19e3: 0x000a,
+	0x19e4: 0x000a, 0x19e5: 0x000a, 0x19e6: 0x000a, 0x19e7: 0x000a, 0x19e8: 0x000a, 0x19e9: 0x000a,
+	0x19ea: 0x000a, 0x19eb: 0x000a, 0x19ec: 0x000a, 0x19ed: 0x000a, 0x19ee: 0x000a, 0x19ef: 0x000a,
+	0x19f0: 0x000a, 0x19f1: 0x000a, 0x19f2: 0x000a, 0x19f3: 0x000a, 0x19f4: 0x000a, 0x19f5: 0x000a,
+	0x19f6: 0x000a, 0x19f7: 0x000a, 0x19f8: 0x000a, 0x19f9: 0x000a, 0x19fa: 0x000a, 0x19fb: 0x000a,
+	0x19fc: 0x000a, 0x19fd: 0x000a, 0x19fe: 0x000a, 0x19ff: 0x000a,
+	// Block 0x68, offset 0x1a00
+	0x1a25: 0x000a, 0x1a26: 0x000a, 0x1a27: 0x000a, 0x1a28: 0x000a, 0x1a29: 0x000a,
+	0x1a2a: 0x000a, 0x1a2f: 0x000c,
+	0x1a30: 0x000c, 0x1a31: 0x000c,
+	0x1a39: 0x000a, 0x1a3a: 0x000a, 0x1a3b: 0x000a,
+	0x1a3c: 0x000a, 0x1a3d: 0x000a, 0x1a3e: 0x000a, 0x1a3f: 0x000a,
+	// Block 0x69, offset 0x1a40
+	0x1a7f: 0x000c,
+	// Block 0x6a, offset 0x1a80
+	0x1aa0: 0x000c, 0x1aa1: 0x000c, 0x1aa2: 0x000c, 0x1aa3: 0x000c,
+	0x1aa4: 0x000c, 0x1aa5: 0x000c, 0x1aa6: 0x000c, 0x1aa7: 0x000c, 0x1aa8: 0x000c, 0x1aa9: 0x000c,
+	0x1aaa: 0x000c, 0x1aab: 0x000c, 0x1aac: 0x000c, 0x1aad: 0x000c, 0x1aae: 0x000c, 0x1aaf: 0x000c,
+	0x1ab0: 0x000c, 0x1ab1: 0x000c, 0x1ab2: 0x000c, 0x1ab3: 0x000c, 0x1ab4: 0x000c, 0x1ab5: 0x000c,
+	0x1ab6: 0x000c, 0x1ab7: 0x000c, 0x1ab8: 0x000c, 0x1ab9: 0x000c, 0x1aba: 0x000c, 0x1abb: 0x000c,
+	0x1abc: 0x000c, 0x1abd: 0x000c, 0x1abe: 0x000c, 0x1abf: 0x000c,
+	// Block 0x6b, offset 0x1ac0
+	0x1ac0: 0x000a, 0x1ac1: 0x000a, 0x1ac2: 0x000a, 0x1ac3: 0x000a, 0x1ac4: 0x000a, 0x1ac5: 0x000a,
+	0x1ac6: 0x000a, 0x1ac7: 0x000a, 0x1ac8: 0x000a, 0x1ac9: 0x000a, 0x1aca: 0x000a, 0x1acb: 0x000a,
+	0x1acc: 0x000a, 0x1acd: 0x000a, 0x1ace: 0x000a, 0x1acf: 0x000a, 0x1ad0: 0x000a, 0x1ad1: 0x000a,
+	0x1ad2: 0x000a, 0x1ad3: 0x000a, 0x1ad4: 0x000a, 0x1ad5: 0x000a, 0x1ad6: 0x000a, 0x1ad7: 0x000a,
+	0x1ad8: 0x000a, 0x1ad9: 0x000a, 0x1ada: 0x000a, 0x1adb: 0x000a, 0x1adc: 0x000a, 0x1add: 0x000a,
+	0x1ade: 0x000a, 0x1adf: 0x000a, 0x1ae0: 0x000a, 0x1ae1: 0x000a, 0x1ae2: 0x003a, 0x1ae3: 0x002a,
+	0x1ae4: 0x003a, 0x1ae5: 0x002a, 0x1ae6: 0x003a, 0x1ae7: 0x002a, 0x1ae8: 0x003a, 0x1ae9: 0x002a,
+	0x1aea: 0x000a, 0x1aeb: 0x000a, 0x1aec: 0x000a, 0x1aed: 0x000a, 0x1aee: 0x000a, 0x1aef: 0x000a,
+	0x1af0: 0x000a, 0x1af1: 0x000a, 0x1af2: 0x000a, 0x1af3: 0x000a, 0x1af4: 0x000a, 0x1af5: 0x000a,
+	0x1af6: 0x000a, 0x1af7: 0x000a, 0x1af8: 0x000a, 0x1af9: 0x000a, 0x1afa: 0x000a, 0x1afb: 0x000a,
+	0x1afc: 0x000a, 0x1afd: 0x000a, 0x1afe: 0x000a, 0x1aff: 0x000a,
+	// Block 0x6c, offset 0x1b00
+	0x1b00: 0x000a, 0x1b01: 0x000a, 0x1b02: 0x000a, 0x1b03: 0x000a, 0x1b04: 0x000a, 0x1b05: 0x000a,
+	0x1b06: 0x000a, 0x1b07: 0x000a, 0x1b08: 0x000a, 0x1b09: 0x000a, 0x1b0a: 0x000a, 0x1b0b: 0x000a,
+	0x1b0c: 0x000a, 0x1b0d: 0x000a, 0x1b0e: 0x000a, 0x1b0f: 0x000a, 0x1b10: 0x000a, 0x1b11: 0x000a,
+	0x1b12: 0x000a,
+	// Block 0x6d, offset 0x1b40
+	0x1b40: 0x000a, 0x1b41: 0x000a, 0x1b42: 0x000a, 0x1b43: 0x000a, 0x1b44: 0x000a, 0x1b45: 0x000a,
+	0x1b46: 0x000a, 0x1b47: 0x000a, 0x1b48: 0x000a, 0x1b49: 0x000a, 0x1b4a: 0x000a, 0x1b4b: 0x000a,
+	0x1b4c: 0x000a, 0x1b4d: 0x000a, 0x1b4e: 0x000a, 0x1b4f: 0x000a, 0x1b50: 0x000a, 0x1b51: 0x000a,
+	0x1b52: 0x000a, 0x1b53: 0x000a, 0x1b54: 0x000a, 0x1b55: 0x000a, 0x1b56: 0x000a, 0x1b57: 0x000a,
+	0x1b58: 0x000a, 0x1b59: 0x000a, 0x1b5b: 0x000a, 0x1b5c: 0x000a, 0x1b5d: 0x000a,
+	0x1b5e: 0x000a, 0x1b5f: 0x000a, 0x1b60: 0x000a, 0x1b61: 0x000a, 0x1b62: 0x000a, 0x1b63: 0x000a,
+	0x1b64: 0x000a, 0x1b65: 0x000a, 0x1b66: 0x000a, 0x1b67: 0x000a, 0x1b68: 0x000a, 0x1b69: 0x000a,
+	0x1b6a: 0x000a, 0x1b6b: 0x000a, 0x1b6c: 0x000a, 0x1b6d: 0x000a, 0x1b6e: 0x000a, 0x1b6f: 0x000a,
+	0x1b70: 0x000a, 0x1b71: 0x000a, 0x1b72: 0x000a, 0x1b73: 0x000a, 0x1b74: 0x000a, 0x1b75: 0x000a,
+	0x1b76: 0x000a, 0x1b77: 0x000a, 0x1b78: 0x000a, 0x1b79: 0x000a, 0x1b7a: 0x000a, 0x1b7b: 0x000a,
+	0x1b7c: 0x000a, 0x1b7d: 0x000a, 0x1b7e: 0x000a, 0x1b7f: 0x000a,
+	// Block 0x6e, offset 0x1b80
+	0x1b80: 0x000a, 0x1b81: 0x000a, 0x1b82: 0x000a, 0x1b83: 0x000a, 0x1b84: 0x000a, 0x1b85: 0x000a,
+	0x1b86: 0x000a, 0x1b87: 0x000a, 0x1b88: 0x000a, 0x1b89: 0x000a, 0x1b8a: 0x000a, 0x1b8b: 0x000a,
+	0x1b8c: 0x000a, 0x1b8d: 0x000a, 0x1b8e: 0x000a, 0x1b8f: 0x000a, 0x1b90: 0x000a, 0x1b91: 0x000a,
+	0x1b92: 0x000a, 0x1b93: 0x000a, 0x1b94: 0x000a, 0x1b95: 0x000a, 0x1b96: 0x000a, 0x1b97: 0x000a,
+	0x1b98: 0x000a, 0x1b99: 0x000a, 0x1b9a: 0x000a, 0x1b9b: 0x000a, 0x1b9c: 0x000a, 0x1b9d: 0x000a,
+	0x1b9e: 0x000a, 0x1b9f: 0x000a, 0x1ba0: 0x000a, 0x1ba1: 0x000a, 0x1ba2: 0x000a, 0x1ba3: 0x000a,
+	0x1ba4: 0x000a, 0x1ba5: 0x000a, 0x1ba6: 0x000a, 0x1ba7: 0x000a, 0x1ba8: 0x000a, 0x1ba9: 0x000a,
+	0x1baa: 0x000a, 0x1bab: 0x000a, 0x1bac: 0x000a, 0x1bad: 0x000a, 0x1bae: 0x000a, 0x1baf: 0x000a,
+	0x1bb0: 0x000a, 0x1bb1: 0x000a, 0x1bb2: 0x000a, 0x1bb3: 0x000a,
+	// Block 0x6f, offset 0x1bc0
+	0x1bc0: 0x000a, 0x1bc1: 0x000a, 0x1bc2: 0x000a, 0x1bc3: 0x000a, 0x1bc4: 0x000a, 0x1bc5: 0x000a,
+	0x1bc6: 0x000a, 0x1bc7: 0x000a, 0x1bc8: 0x000a, 0x1bc9: 0x000a, 0x1bca: 0x000a, 0x1bcb: 0x000a,
+	0x1bcc: 0x000a, 0x1bcd: 0x000a, 0x1bce: 0x000a, 0x1bcf: 0x000a, 0x1bd0: 0x000a, 0x1bd1: 0x000a,
+	0x1bd2: 0x000a, 0x1bd3: 0x000a, 0x1bd4: 0x000a, 0x1bd5: 0x000a,
+	0x1bf0: 0x000a, 0x1bf1: 0x000a, 0x1bf2: 0x000a, 0x1bf3: 0x000a, 0x1bf4: 0x000a, 0x1bf5: 0x000a,
+	0x1bf6: 0x000a, 0x1bf7: 0x000a, 0x1bf8: 0x000a, 0x1bf9: 0x000a, 0x1bfa: 0x000a, 0x1bfb: 0x000a,
+	// Block 0x70, offset 0x1c00
+	0x1c00: 0x0009, 0x1c01: 0x000a, 0x1c02: 0x000a, 0x1c03: 0x000a, 0x1c04: 0x000a,
+	0x1c08: 0x003a, 0x1c09: 0x002a, 0x1c0a: 0x003a, 0x1c0b: 0x002a,
+	0x1c0c: 0x003a, 0x1c0d: 0x002a, 0x1c0e: 0x003a, 0x1c0f: 0x002a, 0x1c10: 0x003a, 0x1c11: 0x002a,
+	0x1c12: 0x000a, 0x1c13: 0x000a, 0x1c14: 0x003a, 0x1c15: 0x002a, 0x1c16: 0x003a, 0x1c17: 0x002a,
+	0x1c18: 0x003a, 0x1c19: 0x002a, 0x1c1a: 0x003a, 0x1c1b: 0x002a, 0x1c1c: 0x000a, 0x1c1d: 0x000a,
+	0x1c1e: 0x000a, 0x1c1f: 0x000a, 0x1c20: 0x000a,
+	0x1c2a: 0x000c, 0x1c2b: 0x000c, 0x1c2c: 0x000c, 0x1c2d: 0x000c,
+	0x1c30: 0x000a,
+	0x1c36: 0x000a, 0x1c37: 0x000a,
+	0x1c3d: 0x000a, 0x1c3e: 0x000a, 0x1c3f: 0x000a,
+	// Block 0x71, offset 0x1c40
+	0x1c59: 0x000c, 0x1c5a: 0x000c, 0x1c5b: 0x000a, 0x1c5c: 0x000a,
+	0x1c60: 0x000a,
+	// Block 0x72, offset 0x1c80
+	0x1cbb: 0x000a,
+	// Block 0x73, offset 0x1cc0
+	0x1cc0: 0x000a, 0x1cc1: 0x000a, 0x1cc2: 0x000a, 0x1cc3: 0x000a, 0x1cc4: 0x000a, 0x1cc5: 0x000a,
+	0x1cc6: 0x000a, 0x1cc7: 0x000a, 0x1cc8: 0x000a, 0x1cc9: 0x000a, 0x1cca: 0x000a, 0x1ccb: 0x000a,
+	0x1ccc: 0x000a, 0x1ccd: 0x000a, 0x1cce: 0x000a, 0x1ccf: 0x000a, 0x1cd0: 0x000a, 0x1cd1: 0x000a,
+	0x1cd2: 0x000a, 0x1cd3: 0x000a, 0x1cd4: 0x000a, 0x1cd5: 0x000a, 0x1cd6: 0x000a, 0x1cd7: 0x000a,
+	0x1cd8: 0x000a, 0x1cd9: 0x000a, 0x1cda: 0x000a, 0x1cdb: 0x000a, 0x1cdc: 0x000a, 0x1cdd: 0x000a,
+	0x1cde: 0x000a, 0x1cdf: 0x000a, 0x1ce0: 0x000a, 0x1ce1: 0x000a, 0x1ce2: 0x000a, 0x1ce3: 0x000a,
+	// Block 0x74, offset 0x1d00
+	0x1d1d: 0x000a,
+	0x1d1e: 0x000a,
+	// Block 0x75, offset 0x1d40
+	0x1d50: 0x000a, 0x1d51: 0x000a,
+	0x1d52: 0x000a, 0x1d53: 0x000a, 0x1d54: 0x000a, 0x1d55: 0x000a, 0x1d56: 0x000a, 0x1d57: 0x000a,
+	0x1d58: 0x000a, 0x1d59: 0x000a, 0x1d5a: 0x000a, 0x1d5b: 0x000a, 0x1d5c: 0x000a, 0x1d5d: 0x000a,
+	0x1d5e: 0x000a, 0x1d5f: 0x000a,
+	0x1d7c: 0x000a, 0x1d7d: 0x000a, 0x1d7e: 0x000a,
+	// Block 0x76, offset 0x1d80
+	0x1db1: 0x000a, 0x1db2: 0x000a, 0x1db3: 0x000a, 0x1db4: 0x000a, 0x1db5: 0x000a,
+	0x1db6: 0x000a, 0x1db7: 0x000a, 0x1db8: 0x000a, 0x1db9: 0x000a, 0x1dba: 0x000a, 0x1dbb: 0x000a,
+	0x1dbc: 0x000a, 0x1dbd: 0x000a, 0x1dbe: 0x000a, 0x1dbf: 0x000a,
+	// Block 0x77, offset 0x1dc0
+	0x1dcc: 0x000a, 0x1dcd: 0x000a, 0x1dce: 0x000a, 0x1dcf: 0x000a,
+	// Block 0x78, offset 0x1e00
+	0x1e37: 0x000a, 0x1e38: 0x000a, 0x1e39: 0x000a, 0x1e3a: 0x000a,
+	// Block 0x79, offset 0x1e40
+	0x1e5e: 0x000a, 0x1e5f: 0x000a,
+	0x1e7f: 0x000a,
+	// Block 0x7a, offset 0x1e80
+	0x1e90: 0x000a, 0x1e91: 0x000a,
+	0x1e92: 0x000a, 0x1e93: 0x000a, 0x1e94: 0x000a, 0x1e95: 0x000a, 0x1e96: 0x000a, 0x1e97: 0x000a,
+	0x1e98: 0x000a, 0x1e99: 0x000a, 0x1e9a: 0x000a, 0x1e9b: 0x000a, 0x1e9c: 0x000a, 0x1e9d: 0x000a,
+	0x1e9e: 0x000a, 0x1e9f: 0x000a, 0x1ea0: 0x000a, 0x1ea1: 0x000a, 0x1ea2: 0x000a, 0x1ea3: 0x000a,
+	0x1ea4: 0x000a, 0x1ea5: 0x000a, 0x1ea6: 0x000a, 0x1ea7: 0x000a, 0x1ea8: 0x000a, 0x1ea9: 0x000a,
+	0x1eaa: 0x000a, 0x1eab: 0x000a, 0x1eac: 0x000a, 0x1ead: 0x000a, 0x1eae: 0x000a, 0x1eaf: 0x000a,
+	0x1eb0: 0x000a, 0x1eb1: 0x000a, 0x1eb2: 0x000a, 0x1eb3: 0x000a, 0x1eb4: 0x000a, 0x1eb5: 0x000a,
+	0x1eb6: 0x000a, 0x1eb7: 0x000a, 0x1eb8: 0x000a, 0x1eb9: 0x000a, 0x1eba: 0x000a, 0x1ebb: 0x000a,
+	0x1ebc: 0x000a, 0x1ebd: 0x000a, 0x1ebe: 0x000a, 0x1ebf: 0x000a,
+	// Block 0x7b, offset 0x1ec0
+	0x1ec0: 0x000a, 0x1ec1: 0x000a, 0x1ec2: 0x000a, 0x1ec3: 0x000a, 0x1ec4: 0x000a, 0x1ec5: 0x000a,
+	0x1ec6: 0x000a,
+	// Block 0x7c, offset 0x1f00
+	0x1f0d: 0x000a, 0x1f0e: 0x000a, 0x1f0f: 0x000a,
+	// Block 0x7d, offset 0x1f40
+	0x1f6f: 0x000c,
+	0x1f70: 0x000c, 0x1f71: 0x000c, 0x1f72: 0x000c, 0x1f73: 0x000a, 0x1f74: 0x000c, 0x1f75: 0x000c,
+	0x1f76: 0x000c, 0x1f77: 0x000c, 0x1f78: 0x000c, 0x1f79: 0x000c, 0x1f7a: 0x000c, 0x1f7b: 0x000c,
+	0x1f7c: 0x000c, 0x1f7d: 0x000c, 0x1f7e: 0x000a, 0x1f7f: 0x000a,
+	// Block 0x7e, offset 0x1f80
+	0x1f9e: 0x000c, 0x1f9f: 0x000c,
+	// Block 0x7f, offset 0x1fc0
+	0x1ff0: 0x000c, 0x1ff1: 0x000c,
+	// Block 0x80, offset 0x2000
+	0x2000: 0x000a, 0x2001: 0x000a, 0x2002: 0x000a, 0x2003: 0x000a, 0x2004: 0x000a, 0x2005: 0x000a,
+	0x2006: 0x000a, 0x2007: 0x000a, 0x2008: 0x000a, 0x2009: 0x000a, 0x200a: 0x000a, 0x200b: 0x000a,
+	0x200c: 0x000a, 0x200d: 0x000a, 0x200e: 0x000a, 0x200f: 0x000a, 0x2010: 0x000a, 0x2011: 0x000a,
+	0x2012: 0x000a, 0x2013: 0x000a, 0x2014: 0x000a, 0x2015: 0x000a, 0x2016: 0x000a, 0x2017: 0x000a,
+	0x2018: 0x000a, 0x2019: 0x000a, 0x201a: 0x000a, 0x201b: 0x000a, 0x201c: 0x000a, 0x201d: 0x000a,
+	0x201e: 0x000a, 0x201f: 0x000a, 0x2020: 0x000a, 0x2021: 0x000a,
+	// Block 0x81, offset 0x2040
+	0x2048: 0x000a,
+	// Block 0x82, offset 0x2080
+	0x2082: 0x000c,
+	0x2086: 0x000c, 0x208b: 0x000c,
+	0x20a5: 0x000c, 0x20a6: 0x000c, 0x20a8: 0x000a, 0x20a9: 0x000a,
+	0x20aa: 0x000a, 0x20ab: 0x000a, 0x20ac: 0x000c,
+	0x20b8: 0x0004, 0x20b9: 0x0004,
+	// Block 0x83, offset 0x20c0
+	0x20f4: 0x000a, 0x20f5: 0x000a,
+	0x20f6: 0x000a, 0x20f7: 0x000a,
+	// Block 0x84, offset 0x2100
+	0x2104: 0x000c, 0x2105: 0x000c,
+	0x2120: 0x000c, 0x2121: 0x000c, 0x2122: 0x000c, 0x2123: 0x000c,
+	0x2124: 0x000c, 0x2125: 0x000c, 0x2126: 0x000c, 0x2127: 0x000c, 0x2128: 0x000c, 0x2129: 0x000c,
+	0x212a: 0x000c, 0x212b: 0x000c, 0x212c: 0x000c, 0x212d: 0x000c, 0x212e: 0x000c, 0x212f: 0x000c,
+	0x2130: 0x000c, 0x2131: 0x000c,
+	0x213f: 0x000c,
+	// Block 0x85, offset 0x2140
+	0x2166: 0x000c, 0x2167: 0x000c, 0x2168: 0x000c, 0x2169: 0x000c,
+	0x216a: 0x000c, 0x216b: 0x000c, 0x216c: 0x000c, 0x216d: 0x000c,
+	// Block 0x86, offset 0x2180
+	0x2187: 0x000c, 0x2188: 0x000c, 0x2189: 0x000c, 0x218a: 0x000c, 0x218b: 0x000c,
+	0x218c: 0x000c, 0x218d: 0x000c, 0x218e: 0x000c, 0x218f: 0x000c, 0x2190: 0x000c, 0x2191: 0x000c,
+	// Block 0x87, offset 0x21c0
+	0x21c0: 0x000c, 0x21c1: 0x000c, 0x21c2: 0x000c,
+	0x21f3: 0x000c,
+	0x21f6: 0x000c, 0x21f7: 0x000c, 0x21f8: 0x000c, 0x21f9: 0x000c,
+	0x21fc: 0x000c, 0x21fd: 0x000c,
+	// Block 0x88, offset 0x2200
+	0x2225: 0x000c,
+	// Block 0x89, offset 0x2240
+	0x2269: 0x000c,
+	0x226a: 0x000c, 0x226b: 0x000c, 0x226c: 0x000c, 0x226d: 0x000c, 0x226e: 0x000c,
+	0x2271: 0x000c, 0x2272: 0x000c, 0x2275: 0x000c,
+	0x2276: 0x000c,
+	// Block 0x8a, offset 0x2280
+	0x2283: 0x000c,
+	0x228c: 0x000c,
+	0x22bc: 0x000c,
+	// Block 0x8b, offset 0x22c0
+	0x22f0: 0x000c, 0x22f2: 0x000c, 0x22f3: 0x000c, 0x22f4: 0x000c,
+	0x22f7: 0x000c, 0x22f8: 0x000c,
+	0x22fe: 0x000c, 0x22ff: 0x000c,
+	// Block 0x8c, offset 0x2300
+	0x2301: 0x000c,
+	0x232c: 0x000c, 0x232d: 0x000c,
+	0x2336: 0x000c,
+	// Block 0x8d, offset 0x2340
+	0x236a: 0x000a, 0x236b: 0x000a,
+	// Block 0x8e, offset 0x2380
+	0x23a5: 0x000c, 0x23a8: 0x000c,
+	0x23ad: 0x000c,
+	// Block 0x8f, offset 0x23c0
+	0x23dd: 0x0001,
+	0x23de: 0x000c, 0x23df: 0x0001, 0x23e0: 0x0001, 0x23e1: 0x0001, 0x23e2: 0x0001, 0x23e3: 0x0001,
+	0x23e4: 0x0001, 0x23e5: 0x0001, 0x23e6: 0x0001, 0x23e7: 0x0001, 0x23e8: 0x0001, 0x23e9: 0x0003,
+	0x23ea: 0x0001, 0x23eb: 0x0001, 0x23ec: 0x0001, 0x23ed: 0x0001, 0x23ee: 0x0001, 0x23ef: 0x0001,
+	0x23f0: 0x0001, 0x23f1: 0x0001, 0x23f2: 0x0001, 0x23f3: 0x0001, 0x23f4: 0x0001, 0x23f5: 0x0001,
+	0x23f6: 0x0001, 0x23f7: 0x0001, 0x23f8: 0x0001, 0x23f9: 0x0001, 0x23fa: 0x0001, 0x23fb: 0x0001,
+	0x23fc: 0x0001, 0x23fd: 0x0001, 0x23fe: 0x0001, 0x23ff: 0x0001,
+	// Block 0x90, offset 0x2400
+	0x2400: 0x0001, 0x2401: 0x0001, 0x2402: 0x0001, 0x2403: 0x0001, 0x2404: 0x0001, 0x2405: 0x0001,
+	0x2406: 0x0001, 0x2407: 0x0001, 0x2408: 0x0001, 0x2409: 0x0001, 0x240a: 0x0001, 0x240b: 0x0001,
+	0x240c: 0x0001, 0x240d: 0x0001, 0x240e: 0x0001, 0x240f: 0x0001, 0x2410: 0x000d, 0x2411: 0x000d,
+	0x2412: 0x000d, 0x2413: 0x000d, 0x2414: 0x000d, 0x2415: 0x000d, 0x2416: 0x000d, 0x2417: 0x000d,
+	0x2418: 0x000d, 0x2419: 0x000d, 0x241a: 0x000d, 0x241b: 0x000d, 0x241c: 0x000d, 0x241d: 0x000d,
+	0x241e: 0x000d, 0x241f: 0x000d, 0x2420: 0x000d, 0x2421: 0x000d, 0x2422: 0x000d, 0x2423: 0x000d,
+	0x2424: 0x000d, 0x2425: 0x000d, 0x2426: 0x000d, 0x2427: 0x000d, 0x2428: 0x000d, 0x2429: 0x000d,
+	0x242a: 0x000d, 0x242b: 0x000d, 0x242c: 0x000d, 0x242d: 0x000d, 0x242e: 0x000d, 0x242f: 0x000d,
+	0x2430: 0x000d, 0x2431: 0x000d, 0x2432: 0x000d, 0x2433: 0x000d, 0x2434: 0x000d, 0x2435: 0x000d,
+	0x2436: 0x000d, 0x2437: 0x000d, 0x2438: 0x000d, 0x2439: 0x000d, 0x243a: 0x000d, 0x243b: 0x000d,
+	0x243c: 0x000d, 0x243d: 0x000d, 0x243e: 0x000d, 0x243f: 0x000d,
+	// Block 0x91, offset 0x2440
+	0x2440: 0x000d, 0x2441: 0x000d, 0x2442: 0x000d, 0x2443: 0x000d, 0x2444: 0x000d, 0x2445: 0x000d,
+	0x2446: 0x000d, 0x2447: 0x000d, 0x2448: 0x000d, 0x2449: 0x000d, 0x244a: 0x000d, 0x244b: 0x000d,
+	0x244c: 0x000d, 0x244d: 0x000d, 0x244e: 0x000d, 0x244f: 0x000d, 0x2450: 0x000d, 0x2451: 0x000d,
+	0x2452: 0x000d, 0x2453: 0x000d, 0x2454: 0x000d, 0x2455: 0x000d, 0x2456: 0x000d, 0x2457: 0x000d,
+	0x2458: 0x000d, 0x2459: 0x000d, 0x245a: 0x000d, 0x245b: 0x000d, 0x245c: 0x000d, 0x245d: 0x000d,
+	0x245e: 0x000d, 0x245f: 0x000d, 0x2460: 0x000d, 0x2461: 0x000d, 0x2462: 0x000d, 0x2463: 0x000d,
+	0x2464: 0x000d, 0x2465: 0x000d, 0x2466: 0x000d, 0x2467: 0x000d, 0x2468: 0x000d, 0x2469: 0x000d,
+	0x246a: 0x000d, 0x246b: 0x000d, 0x246c: 0x000d, 0x246d: 0x000d, 0x246e: 0x000d, 0x246f: 0x000d,
+	0x2470: 0x000d, 0x2471: 0x000d, 0x2472: 0x000d, 0x2473: 0x000d, 0x2474: 0x000d, 0x2475: 0x000d,
+	0x2476: 0x000d, 0x2477: 0x000d, 0x2478: 0x000d, 0x2479: 0x000d, 0x247a: 0x000d, 0x247b: 0x000d,
+	0x247c: 0x000d, 0x247d: 0x000d, 0x247e: 0x000a, 0x247f: 0x000a,
+	// Block 0x92, offset 0x2480
+	0x2480: 0x000d, 0x2481: 0x000d, 0x2482: 0x000d, 0x2483: 0x000d, 0x2484: 0x000d, 0x2485: 0x000d,
+	0x2486: 0x000d, 0x2487: 0x000d, 0x2488: 0x000d, 0x2489: 0x000d, 0x248a: 0x000d, 0x248b: 0x000d,
+	0x248c: 0x000d, 0x248d: 0x000d, 0x248e: 0x000d, 0x248f: 0x000d, 0x2490: 0x000b, 0x2491: 0x000b,
+	0x2492: 0x000b, 0x2493: 0x000b, 0x2494: 0x000b, 0x2495: 0x000b, 0x2496: 0x000b, 0x2497: 0x000b,
+	0x2498: 0x000b, 0x2499: 0x000b, 0x249a: 0x000b, 0x249b: 0x000b, 0x249c: 0x000b, 0x249d: 0x000b,
+	0x249e: 0x000b, 0x249f: 0x000b, 0x24a0: 0x000b, 0x24a1: 0x000b, 0x24a2: 0x000b, 0x24a3: 0x000b,
+	0x24a4: 0x000b, 0x24a5: 0x000b, 0x24a6: 0x000b, 0x24a7: 0x000b, 0x24a8: 0x000b, 0x24a9: 0x000b,
+	0x24aa: 0x000b, 0x24ab: 0x000b, 0x24ac: 0x000b, 0x24ad: 0x000b, 0x24ae: 0x000b, 0x24af: 0x000b,
+	0x24b0: 0x000d, 0x24b1: 0x000d, 0x24b2: 0x000d, 0x24b3: 0x000d, 0x24b4: 0x000d, 0x24b5: 0x000d,
+	0x24b6: 0x000d, 0x24b7: 0x000d, 0x24b8: 0x000d, 0x24b9: 0x000d, 0x24ba: 0x000d, 0x24bb: 0x000d,
+	0x24bc: 0x000d, 0x24bd: 0x000a, 0x24be: 0x000d, 0x24bf: 0x000d,
+	// Block 0x93, offset 0x24c0
+	0x24c0: 0x000c, 0x24c1: 0x000c, 0x24c2: 0x000c, 0x24c3: 0x000c, 0x24c4: 0x000c, 0x24c5: 0x000c,
+	0x24c6: 0x000c, 0x24c7: 0x000c, 0x24c8: 0x000c, 0x24c9: 0x000c, 0x24ca: 0x000c, 0x24cb: 0x000c,
+	0x24cc: 0x000c, 0x24cd: 0x000c, 0x24ce: 0x000c, 0x24cf: 0x000c, 0x24d0: 0x000a, 0x24d1: 0x000a,
+	0x24d2: 0x000a, 0x24d3: 0x000a, 0x24d4: 0x000a, 0x24d5: 0x000a, 0x24d6: 0x000a, 0x24d7: 0x000a,
+	0x24d8: 0x000a, 0x24d9: 0x000a,
+	0x24e0: 0x000c, 0x24e1: 0x000c, 0x24e2: 0x000c, 0x24e3: 0x000c,
+	0x24e4: 0x000c, 0x24e5: 0x000c, 0x24e6: 0x000c, 0x24e7: 0x000c, 0x24e8: 0x000c, 0x24e9: 0x000c,
+	0x24ea: 0x000c, 0x24eb: 0x000c, 0x24ec: 0x000c, 0x24ed: 0x000c, 0x24ee: 0x000c, 0x24ef: 0x000c,
+	0x24f0: 0x000a, 0x24f1: 0x000a, 0x24f2: 0x000a, 0x24f3: 0x000a, 0x24f4: 0x000a, 0x24f5: 0x000a,
+	0x24f6: 0x000a, 0x24f7: 0x000a, 0x24f8: 0x000a, 0x24f9: 0x000a, 0x24fa: 0x000a, 0x24fb: 0x000a,
+	0x24fc: 0x000a, 0x24fd: 0x000a, 0x24fe: 0x000a, 0x24ff: 0x000a,
+	// Block 0x94, offset 0x2500
+	0x2500: 0x000a, 0x2501: 0x000a, 0x2502: 0x000a, 0x2503: 0x000a, 0x2504: 0x000a, 0x2505: 0x000a,
+	0x2506: 0x000a, 0x2507: 0x000a, 0x2508: 0x000a, 0x2509: 0x000a, 0x250a: 0x000a, 0x250b: 0x000a,
+	0x250c: 0x000a, 0x250d: 0x000a, 0x250e: 0x000a, 0x250f: 0x000a, 0x2510: 0x0006, 0x2511: 0x000a,
+	0x2512: 0x0006, 0x2514: 0x000a, 0x2515: 0x0006, 0x2516: 0x000a, 0x2517: 0x000a,
+	0x2518: 0x000a, 0x2519: 0x009a, 0x251a: 0x008a, 0x251b: 0x007a, 0x251c: 0x006a, 0x251d: 0x009a,
+	0x251e: 0x008a, 0x251f: 0x0004, 0x2520: 0x000a, 0x2521: 0x000a, 0x2522: 0x0003, 0x2523: 0x0003,
+	0x2524: 0x000a, 0x2525: 0x000a, 0x2526: 0x000a, 0x2528: 0x000a, 0x2529: 0x0004,
+	0x252a: 0x0004, 0x252b: 0x000a,
+	0x2530: 0x000d, 0x2531: 0x000d, 0x2532: 0x000d, 0x2533: 0x000d, 0x2534: 0x000d, 0x2535: 0x000d,
+	0x2536: 0x000d, 0x2537: 0x000d, 0x2538: 0x000d, 0x2539: 0x000d, 0x253a: 0x000d, 0x253b: 0x000d,
+	0x253c: 0x000d, 0x253d: 0x000d, 0x253e: 0x000d, 0x253f: 0x000d,
+	// Block 0x95, offset 0x2540
+	0x2540: 0x000d, 0x2541: 0x000d, 0x2542: 0x000d, 0x2543: 0x000d, 0x2544: 0x000d, 0x2545: 0x000d,
+	0x2546: 0x000d, 0x2547: 0x000d, 0x2548: 0x000d, 0x2549: 0x000d, 0x254a: 0x000d, 0x254b: 0x000d,
+	0x254c: 0x000d, 0x254d: 0x000d, 0x254e: 0x000d, 0x254f: 0x000d, 0x2550: 0x000d, 0x2551: 0x000d,
+	0x2552: 0x000d, 0x2553: 0x000d, 0x2554: 0x000d, 0x2555: 0x000d, 0x2556: 0x000d, 0x2557: 0x000d,
+	0x2558: 0x000d, 0x2559: 0x000d, 0x255a: 0x000d, 0x255b: 0x000d, 0x255c: 0x000d, 0x255d: 0x000d,
+	0x255e: 0x000d, 0x255f: 0x000d, 0x2560: 0x000d, 0x2561: 0x000d, 0x2562: 0x000d, 0x2563: 0x000d,
+	0x2564: 0x000d, 0x2565: 0x000d, 0x2566: 0x000d, 0x2567: 0x000d, 0x2568: 0x000d, 0x2569: 0x000d,
+	0x256a: 0x000d, 0x256b: 0x000d, 0x256c: 0x000d, 0x256d: 0x000d, 0x256e: 0x000d, 0x256f: 0x000d,
+	0x2570: 0x000d, 0x2571: 0x000d, 0x2572: 0x000d, 0x2573: 0x000d, 0x2574: 0x000d, 0x2575: 0x000d,
+	0x2576: 0x000d, 0x2577: 0x000d, 0x2578: 0x000d, 0x2579: 0x000d, 0x257a: 0x000d, 0x257b: 0x000d,
+	0x257c: 0x000d, 0x257d: 0x000d, 0x257e: 0x000d, 0x257f: 0x000b,
+	// Block 0x96, offset 0x2580
+	0x2581: 0x000a, 0x2582: 0x000a, 0x2583: 0x0004, 0x2584: 0x0004, 0x2585: 0x0004,
+	0x2586: 0x000a, 0x2587: 0x000a, 0x2588: 0x003a, 0x2589: 0x002a, 0x258a: 0x000a, 0x258b: 0x0003,
+	0x258c: 0x0006, 0x258d: 0x0003, 0x258e: 0x0006, 0x258f: 0x0006, 0x2590: 0x0002, 0x2591: 0x0002,
+	0x2592: 0x0002, 0x2593: 0x0002, 0x2594: 0x0002, 0x2595: 0x0002, 0x2596: 0x0002, 0x2597: 0x0002,
+	0x2598: 0x0002, 0x2599: 0x0002, 0x259a: 0x0006, 0x259b: 0x000a, 0x259c: 0x000a, 0x259d: 0x000a,
+	0x259e: 0x000a, 0x259f: 0x000a, 0x25a0: 0x000a,
+	0x25bb: 0x005a,
+	0x25bc: 0x000a, 0x25bd: 0x004a, 0x25be: 0x000a, 0x25bf: 0x000a,
+	// Block 0x97, offset 0x25c0
+	0x25c0: 0x000a,
+	0x25db: 0x005a, 0x25dc: 0x000a, 0x25dd: 0x004a,
+	0x25de: 0x000a, 0x25df: 0x00fa, 0x25e0: 0x00ea, 0x25e1: 0x000a, 0x25e2: 0x003a, 0x25e3: 0x002a,
+	0x25e4: 0x000a, 0x25e5: 0x000a,
+	// Block 0x98, offset 0x2600
+	0x2620: 0x0004, 0x2621: 0x0004, 0x2622: 0x000a, 0x2623: 0x000a,
+	0x2624: 0x000a, 0x2625: 0x0004, 0x2626: 0x0004, 0x2628: 0x000a, 0x2629: 0x000a,
+	0x262a: 0x000a, 0x262b: 0x000a, 0x262c: 0x000a, 0x262d: 0x000a, 0x262e: 0x000a,
+	0x2630: 0x000b, 0x2631: 0x000b, 0x2632: 0x000b, 0x2633: 0x000b, 0x2634: 0x000b, 0x2635: 0x000b,
+	0x2636: 0x000b, 0x2637: 0x000b, 0x2638: 0x000b, 0x2639: 0x000a, 0x263a: 0x000a, 0x263b: 0x000a,
+	0x263c: 0x000a, 0x263d: 0x000a, 0x263e: 0x000b, 0x263f: 0x000b,
+	// Block 0x99, offset 0x2640
+	0x2641: 0x000a,
+	// Block 0x9a, offset 0x2680
+	0x2680: 0x000a, 0x2681: 0x000a, 0x2682: 0x000a, 0x2683: 0x000a, 0x2684: 0x000a, 0x2685: 0x000a,
+	0x2686: 0x000a, 0x2687: 0x000a, 0x2688: 0x000a, 0x2689: 0x000a, 0x268a: 0x000a, 0x268b: 0x000a,
+	0x268c: 0x000a, 0x2690: 0x000a, 0x2691: 0x000a,
+	0x2692: 0x000a, 0x2693: 0x000a, 0x2694: 0x000a, 0x2695: 0x000a, 0x2696: 0x000a, 0x2697: 0x000a,
+	0x2698: 0x000a, 0x2699: 0x000a, 0x269a: 0x000a, 0x269b: 0x000a, 0x269c: 0x000a,
+	0x26a0: 0x000a,
+	// Block 0x9b, offset 0x26c0
+	0x26fd: 0x000c,
+	// Block 0x9c, offset 0x2700
+	0x2720: 0x000c, 0x2721: 0x0002, 0x2722: 0x0002, 0x2723: 0x0002,
+	0x2724: 0x0002, 0x2725: 0x0002, 0x2726: 0x0002, 0x2727: 0x0002, 0x2728: 0x0002, 0x2729: 0x0002,
+	0x272a: 0x0002, 0x272b: 0x0002, 0x272c: 0x0002, 0x272d: 0x0002, 0x272e: 0x0002, 0x272f: 0x0002,
+	0x2730: 0x0002, 0x2731: 0x0002, 0x2732: 0x0002, 0x2733: 0x0002, 0x2734: 0x0002, 0x2735: 0x0002,
+	0x2736: 0x0002, 0x2737: 0x0002, 0x2738: 0x0002, 0x2739: 0x0002, 0x273a: 0x0002, 0x273b: 0x0002,
+	// Block 0x9d, offset 0x2740
+	0x2776: 0x000c, 0x2777: 0x000c, 0x2778: 0x000c, 0x2779: 0x000c, 0x277a: 0x000c,
+	// Block 0x9e, offset 0x2780
+	0x2780: 0x0001, 0x2781: 0x0001, 0x2782: 0x0001, 0x2783: 0x0001, 0x2784: 0x0001, 0x2785: 0x0001,
+	0x2786: 0x0001, 0x2787: 0x0001, 0x2788: 0x0001, 0x2789: 0x0001, 0x278a: 0x0001, 0x278b: 0x0001,
+	0x278c: 0x0001, 0x278d: 0x0001, 0x278e: 0x0001, 0x278f: 0x0001, 0x2790: 0x0001, 0x2791: 0x0001,
+	0x2792: 0x0001, 0x2793: 0x0001, 0x2794: 0x0001, 0x2795: 0x0001, 0x2796: 0x0001, 0x2797: 0x0001,
+	0x2798: 0x0001, 0x2799: 0x0001, 0x279a: 0x0001, 0x279b: 0x0001, 0x279c: 0x0001, 0x279d: 0x0001,
+	0x279e: 0x0001, 0x279f: 0x0001, 0x27a0: 0x0001, 0x27a1: 0x0001, 0x27a2: 0x0001, 0x27a3: 0x0001,
+	0x27a4: 0x0001, 0x27a5: 0x0001, 0x27a6: 0x0001, 0x27a7: 0x0001, 0x27a8: 0x0001, 0x27a9: 0x0001,
+	0x27aa: 0x0001, 0x27ab: 0x0001, 0x27ac: 0x0001, 0x27ad: 0x0001, 0x27ae: 0x0001, 0x27af: 0x0001,
+	0x27b0: 0x0001, 0x27b1: 0x0001, 0x27b2: 0x0001, 0x27b3: 0x0001, 0x27b4: 0x0001, 0x27b5: 0x0001,
+	0x27b6: 0x0001, 0x27b7: 0x0001, 0x27b8: 0x0001, 0x27b9: 0x0001, 0x27ba: 0x0001, 0x27bb: 0x0001,
+	0x27bc: 0x0001, 0x27bd: 0x0001, 0x27be: 0x0001, 0x27bf: 0x0001,
+	// Block 0x9f, offset 0x27c0
+	0x27c0: 0x0001, 0x27c1: 0x0001, 0x27c2: 0x0001, 0x27c3: 0x0001, 0x27c4: 0x0001, 0x27c5: 0x0001,
+	0x27c6: 0x0001, 0x27c7: 0x0001, 0x27c8: 0x0001, 0x27c9: 0x0001, 0x27ca: 0x0001, 0x27cb: 0x0001,
+	0x27cc: 0x0001, 0x27cd: 0x0001, 0x27ce: 0x0001, 0x27cf: 0x0001, 0x27d0: 0x0001, 0x27d1: 0x0001,
+	0x27d2: 0x0001, 0x27d3: 0x0001, 0x27d4: 0x0001, 0x27d5: 0x0001, 0x27d6: 0x0001, 0x27d7: 0x0001,
+	0x27d8: 0x0001, 0x27d9: 0x0001, 0x27da: 0x0001, 0x27db: 0x0001, 0x27dc: 0x0001, 0x27dd: 0x0001,
+	0x27de: 0x0001, 0x27df: 0x000a, 0x27e0: 0x0001, 0x27e1: 0x0001, 0x27e2: 0x0001, 0x27e3: 0x0001,
+	0x27e4: 0x0001, 0x27e5: 0x0001, 0x27e6: 0x0001, 0x27e7: 0x0001, 0x27e8: 0x0001, 0x27e9: 0x0001,
+	0x27ea: 0x0001, 0x27eb: 0x0001, 0x27ec: 0x0001, 0x27ed: 0x0001, 0x27ee: 0x0001, 0x27ef: 0x0001,
+	0x27f0: 0x0001, 0x27f1: 0x0001, 0x27f2: 0x0001, 0x27f3: 0x0001, 0x27f4: 0x0001, 0x27f5: 0x0001,
+	0x27f6: 0x0001, 0x27f7: 0x0001, 0x27f8: 0x0001, 0x27f9: 0x0001, 0x27fa: 0x0001, 0x27fb: 0x0001,
+	0x27fc: 0x0001, 0x27fd: 0x0001, 0x27fe: 0x0001, 0x27ff: 0x0001,
+	// Block 0xa0, offset 0x2800
+	0x2800: 0x0001, 0x2801: 0x000c, 0x2802: 0x000c, 0x2803: 0x000c, 0x2804: 0x0001, 0x2805: 0x000c,
+	0x2806: 0x000c, 0x2807: 0x0001, 0x2808: 0x0001, 0x2809: 0x0001, 0x280a: 0x0001, 0x280b: 0x0001,
+	0x280c: 0x000c, 0x280d: 0x000c, 0x280e: 0x000c, 0x280f: 0x000c, 0x2810: 0x0001, 0x2811: 0x0001,
+	0x2812: 0x0001, 0x2813: 0x0001, 0x2814: 0x0001, 0x2815: 0x0001, 0x2816: 0x0001, 0x2817: 0x0001,
+	0x2818: 0x0001, 0x2819: 0x0001, 0x281a: 0x0001, 0x281b: 0x0001, 0x281c: 0x0001, 0x281d: 0x0001,
+	0x281e: 0x0001, 0x281f: 0x0001, 0x2820: 0x0001, 0x2821: 0x0001, 0x2822: 0x0001, 0x2823: 0x0001,
+	0x2824: 0x0001, 0x2825: 0x0001, 0x2826: 0x0001, 0x2827: 0x0001, 0x2828: 0x0001, 0x2829: 0x0001,
+	0x282a: 0x0001, 0x282b: 0x0001, 0x282c: 0x0001, 0x282d: 0x0001, 0x282e: 0x0001, 0x282f: 0x0001,
+	0x2830: 0x0001, 0x2831: 0x0001, 0x2832: 0x0001, 0x2833: 0x0001, 0x2834: 0x0001, 0x2835: 0x0001,
+	0x2836: 0x0001, 0x2837: 0x0001, 0x2838: 0x000c, 0x2839: 0x000c, 0x283a: 0x000c, 0x283b: 0x0001,
+	0x283c: 0x0001, 0x283d: 0x0001, 0x283e: 0x0001, 0x283f: 0x000c,
+	// Block 0xa1, offset 0x2840
+	0x2840: 0x0001, 0x2841: 0x0001, 0x2842: 0x0001, 0x2843: 0x0001, 0x2844: 0x0001, 0x2845: 0x0001,
+	0x2846: 0x0001, 0x2847: 0x0001, 0x2848: 0x0001, 0x2849: 0x0001, 0x284a: 0x0001, 0x284b: 0x0001,
+	0x284c: 0x0001, 0x284d: 0x0001, 0x284e: 0x0001, 0x284f: 0x0001, 0x2850: 0x0001, 0x2851: 0x0001,
+	0x2852: 0x0001, 0x2853: 0x0001, 0x2854: 0x0001, 0x2855: 0x0001, 0x2856: 0x0001, 0x2857: 0x0001,
+	0x2858: 0x0001, 0x2859: 0x0001, 0x285a: 0x0001, 0x285b: 0x0001, 0x285c: 0x0001, 0x285d: 0x0001,
+	0x285e: 0x0001, 0x285f: 0x0001, 0x2860: 0x0001, 0x2861: 0x0001, 0x2862: 0x0001, 0x2863: 0x0001,
+	0x2864: 0x0001, 0x2865: 0x000c, 0x2866: 0x000c, 0x2867: 0x0001, 0x2868: 0x0001, 0x2869: 0x0001,
+	0x286a: 0x0001, 0x286b: 0x0001, 0x286c: 0x0001, 0x286d: 0x0001, 0x286e: 0x0001, 0x286f: 0x0001,
+	0x2870: 0x0001, 0x2871: 0x0001, 0x2872: 0x0001, 0x2873: 0x0001, 0x2874: 0x0001, 0x2875: 0x0001,
+	0x2876: 0x0001, 0x2877: 0x0001, 0x2878: 0x0001, 0x2879: 0x0001, 0x287a: 0x0001, 0x287b: 0x0001,
+	0x287c: 0x0001, 0x287d: 0x0001, 0x287e: 0x0001, 0x287f: 0x0001,
+	// Block 0xa2, offset 0x2880
+	0x2880: 0x0001, 0x2881: 0x0001, 0x2882: 0x0001, 0x2883: 0x0001, 0x2884: 0x0001, 0x2885: 0x0001,
+	0x2886: 0x0001, 0x2887: 0x0001, 0x2888: 0x0001, 0x2889: 0x0001, 0x288a: 0x0001, 0x288b: 0x0001,
+	0x288c: 0x0001, 0x288d: 0x0001, 0x288e: 0x0001, 0x288f: 0x0001, 0x2890: 0x0001, 0x2891: 0x0001,
+	0x2892: 0x0001, 0x2893: 0x0001, 0x2894: 0x0001, 0x2895: 0x0001, 0x2896: 0x0001, 0x2897: 0x0001,
+	0x2898: 0x0001, 0x2899: 0x0001, 0x289a: 0x0001, 0x289b: 0x0001, 0x289c: 0x0001, 0x289d: 0x0001,
+	0x289e: 0x0001, 0x289f: 0x0001, 0x28a0: 0x0001, 0x28a1: 0x0001, 0x28a2: 0x0001, 0x28a3: 0x0001,
+	0x28a4: 0x0001, 0x28a5: 0x0001, 0x28a6: 0x0001, 0x28a7: 0x0001, 0x28a8: 0x0001, 0x28a9: 0x0001,
+	0x28aa: 0x0001, 0x28ab: 0x0001, 0x28ac: 0x0001, 0x28ad: 0x0001, 0x28ae: 0x0001, 0x28af: 0x0001,
+	0x28b0: 0x0001, 0x28b1: 0x0001, 0x28b2: 0x0001, 0x28b3: 0x0001, 0x28b4: 0x0001, 0x28b5: 0x0001,
+	0x28b6: 0x0001, 0x28b7: 0x0001, 0x28b8: 0x0001, 0x28b9: 0x000a, 0x28ba: 0x000a, 0x28bb: 0x000a,
+	0x28bc: 0x000a, 0x28bd: 0x000a, 0x28be: 0x000a, 0x28bf: 0x000a,
+	// Block 0xa3, offset 0x28c0
+	0x28c0: 0x000d, 0x28c1: 0x000d, 0x28c2: 0x000d, 0x28c3: 0x000d, 0x28c4: 0x000d, 0x28c5: 0x000d,
+	0x28c6: 0x000d, 0x28c7: 0x000d, 0x28c8: 0x000d, 0x28c9: 0x000d, 0x28ca: 0x000d, 0x28cb: 0x000d,
+	0x28cc: 0x000d, 0x28cd: 0x000d, 0x28ce: 0x000d, 0x28cf: 0x000d, 0x28d0: 0x000d, 0x28d1: 0x000d,
+	0x28d2: 0x000d, 0x28d3: 0x000d, 0x28d4: 0x000d, 0x28d5: 0x000d, 0x28d6: 0x000d, 0x28d7: 0x000d,
+	0x28d8: 0x000d, 0x28d9: 0x000d, 0x28da: 0x000d, 0x28db: 0x000d, 0x28dc: 0x000d, 0x28dd: 0x000d,
+	0x28de: 0x000d, 0x28df: 0x000d, 0x28e0: 0x000d, 0x28e1: 0x000d, 0x28e2: 0x000d, 0x28e3: 0x000d,
+	0x28e4: 0x000c, 0x28e5: 0x000c, 0x28e6: 0x000c, 0x28e7: 0x000c, 0x28e8: 0x000d, 0x28e9: 0x000d,
+	0x28ea: 0x000d, 0x28eb: 0x000d, 0x28ec: 0x000d, 0x28ed: 0x000d, 0x28ee: 0x000d, 0x28ef: 0x000d,
+	0x28f0: 0x0005, 0x28f1: 0x0005, 0x28f2: 0x0005, 0x28f3: 0x0005, 0x28f4: 0x0005, 0x28f5: 0x0005,
+	0x28f6: 0x0005, 0x28f7: 0x0005, 0x28f8: 0x0005, 0x28f9: 0x0005, 0x28fa: 0x000d, 0x28fb: 0x000d,
+	0x28fc: 0x000d, 0x28fd: 0x000d, 0x28fe: 0x000d, 0x28ff: 0x000d,
+	// Block 0xa4, offset 0x2900
+	0x2900: 0x0001, 0x2901: 0x0001, 0x2902: 0x0001, 0x2903: 0x0001, 0x2904: 0x0001, 0x2905: 0x0001,
+	0x2906: 0x0001, 0x2907: 0x0001, 0x2908: 0x0001, 0x2909: 0x0001, 0x290a: 0x0001, 0x290b: 0x0001,
+	0x290c: 0x0001, 0x290d: 0x0001, 0x290e: 0x0001, 0x290f: 0x0001, 0x2910: 0x0001, 0x2911: 0x0001,
+	0x2912: 0x0001, 0x2913: 0x0001, 0x2914: 0x0001, 0x2915: 0x0001, 0x2916: 0x0001, 0x2917: 0x0001,
+	0x2918: 0x0001, 0x2919: 0x0001, 0x291a: 0x0001, 0x291b: 0x0001, 0x291c: 0x0001, 0x291d: 0x0001,
+	0x291e: 0x0001, 0x291f: 0x0001, 0x2920: 0x0005, 0x2921: 0x0005, 0x2922: 0x0005, 0x2923: 0x0005,
+	0x2924: 0x0005, 0x2925: 0x0005, 0x2926: 0x0005, 0x2927: 0x0005, 0x2928: 0x0005, 0x2929: 0x0005,
+	0x292a: 0x0005, 0x292b: 0x0005, 0x292c: 0x0005, 0x292d: 0x0005, 0x292e: 0x0005, 0x292f: 0x0005,
+	0x2930: 0x0005, 0x2931: 0x0005, 0x2932: 0x0005, 0x2933: 0x0005, 0x2934: 0x0005, 0x2935: 0x0005,
+	0x2936: 0x0005, 0x2937: 0x0005, 0x2938: 0x0005, 0x2939: 0x0005, 0x293a: 0x0005, 0x293b: 0x0005,
+	0x293c: 0x0005, 0x293d: 0x0005, 0x293e: 0x0005, 0x293f: 0x0001,
+	// Block 0xa5, offset 0x2940
+	0x2940: 0x0001, 0x2941: 0x0001, 0x2942: 0x0001, 0x2943: 0x0001, 0x2944: 0x0001, 0x2945: 0x0001,
+	0x2946: 0x0001, 0x2947: 0x0001, 0x2948: 0x0001, 0x2949: 0x0001, 0x294a: 0x0001, 0x294b: 0x0001,
+	0x294c: 0x0001, 0x294d: 0x0001, 0x294e: 0x0001, 0x294f: 0x0001, 0x2950: 0x0001, 0x2951: 0x0001,
+	0x2952: 0x0001, 0x2953: 0x0001, 0x2954: 0x0001, 0x2955: 0x0001, 0x2956: 0x0001, 0x2957: 0x0001,
+	0x2958: 0x0001, 0x2959: 0x0001, 0x295a: 0x0001, 0x295b: 0x0001, 0x295c: 0x0001, 0x295d: 0x0001,
+	0x295e: 0x0001, 0x295f: 0x0001, 0x2960: 0x0001, 0x2961: 0x0001, 0x2962: 0x0001, 0x2963: 0x0001,
+	0x2964: 0x0001, 0x2965: 0x0001, 0x2966: 0x0001, 0x2967: 0x0001, 0x2968: 0x0001, 0x2969: 0x0001,
+	0x296a: 0x0001, 0x296b: 0x000c, 0x296c: 0x000c, 0x296d: 0x0001, 0x296e: 0x0001, 0x296f: 0x0001,
+	0x2970: 0x0001, 0x2971: 0x0001, 0x2972: 0x0001, 0x2973: 0x0001, 0x2974: 0x0001, 0x2975: 0x0001,
+	0x2976: 0x0001, 0x2977: 0x0001, 0x2978: 0x0001, 0x2979: 0x0001, 0x297a: 0x0001, 0x297b: 0x0001,
+	0x297c: 0x0001, 0x297d: 0x0001, 0x297e: 0x0001, 0x297f: 0x0001,
+	// Block 0xa6, offset 0x2980
+	0x2980: 0x0001, 0x2981: 0x0001, 0x2982: 0x0001, 0x2983: 0x0001, 0x2984: 0x0001, 0x2985: 0x0001,
+	0x2986: 0x0001, 0x2987: 0x0001, 0x2988: 0x0001, 0x2989: 0x0001, 0x298a: 0x0001, 0x298b: 0x0001,
+	0x298c: 0x0001, 0x298d: 0x0001, 0x298e: 0x0001, 0x298f: 0x0001, 0x2990: 0x0001, 0x2991: 0x0001,
+	0x2992: 0x0001, 0x2993: 0x0001, 0x2994: 0x0001, 0x2995: 0x0001, 0x2996: 0x0001, 0x2997: 0x0001,
+	0x2998: 0x0001, 0x2999: 0x0001, 0x299a: 0x0001, 0x299b: 0x0001, 0x299c: 0x0001, 0x299d: 0x0001,
+	0x299e: 0x0001, 0x299f: 0x0001, 0x29a0: 0x0001, 0x29a1: 0x0001, 0x29a2: 0x0001, 0x29a3: 0x0001,
+	0x29a4: 0x0001, 0x29a5: 0x0001, 0x29a6: 0x0001, 0x29a7: 0x0001, 0x29a8: 0x0001, 0x29a9: 0x0001,
+	0x29aa: 0x0001, 0x29ab: 0x0001, 0x29ac: 0x0001, 0x29ad: 0x0001, 0x29ae: 0x0001, 0x29af: 0x0001,
+	0x29b0: 0x000d, 0x29b1: 0x000d, 0x29b2: 0x000d, 0x29b3: 0x000d, 0x29b4: 0x000d, 0x29b5: 0x000d,
+	0x29b6: 0x000d, 0x29b7: 0x000d, 0x29b8: 0x000d, 0x29b9: 0x000d, 0x29ba: 0x000d, 0x29bb: 0x000d,
+	0x29bc: 0x000d, 0x29bd: 0x000d, 0x29be: 0x000d, 0x29bf: 0x000d,
+	// Block 0xa7, offset 0x29c0
+	0x29c0: 0x000d, 0x29c1: 0x000d, 0x29c2: 0x000d, 0x29c3: 0x000d, 0x29c4: 0x000d, 0x29c5: 0x000d,
+	0x29c6: 0x000c, 0x29c7: 0x000c, 0x29c8: 0x000c, 0x29c9: 0x000c, 0x29ca: 0x000c, 0x29cb: 0x000c,
+	0x29cc: 0x000c, 0x29cd: 0x000c, 0x29ce: 0x000c, 0x29cf: 0x000c, 0x29d0: 0x000c, 0x29d1: 0x000d,
+	0x29d2: 0x000d, 0x29d3: 0x000d, 0x29d4: 0x000d, 0x29d5: 0x000d, 0x29d6: 0x000d, 0x29d7: 0x000d,
+	0x29d8: 0x000d, 0x29d9: 0x000d, 0x29da: 0x000d, 0x29db: 0x000d, 0x29dc: 0x000d, 0x29dd: 0x000d,
+	0x29de: 0x000d, 0x29df: 0x000d, 0x29e0: 0x000d, 0x29e1: 0x000d, 0x29e2: 0x000d, 0x29e3: 0x000d,
+	0x29e4: 0x000d, 0x29e5: 0x000d, 0x29e6: 0x000d, 0x29e7: 0x000d, 0x29e8: 0x000d, 0x29e9: 0x000d,
+	0x29ea: 0x000d, 0x29eb: 0x000d, 0x29ec: 0x000d, 0x29ed: 0x000d, 0x29ee: 0x000d, 0x29ef: 0x000d,
+	0x29f0: 0x0001, 0x29f1: 0x0001, 0x29f2: 0x0001, 0x29f3: 0x0001, 0x29f4: 0x0001, 0x29f5: 0x0001,
+	0x29f6: 0x0001, 0x29f7: 0x0001, 0x29f8: 0x0001, 0x29f9: 0x0001, 0x29fa: 0x0001, 0x29fb: 0x0001,
+	0x29fc: 0x0001, 0x29fd: 0x0001, 0x29fe: 0x0001, 0x29ff: 0x0001,
+	// Block 0xa8, offset 0x2a00
+	0x2a01: 0x000c,
+	0x2a38: 0x000c, 0x2a39: 0x000c, 0x2a3a: 0x000c, 0x2a3b: 0x000c,
+	0x2a3c: 0x000c, 0x2a3d: 0x000c, 0x2a3e: 0x000c, 0x2a3f: 0x000c,
+	// Block 0xa9, offset 0x2a40
+	0x2a40: 0x000c, 0x2a41: 0x000c, 0x2a42: 0x000c, 0x2a43: 0x000c, 0x2a44: 0x000c, 0x2a45: 0x000c,
+	0x2a46: 0x000c,
+	0x2a52: 0x000a, 0x2a53: 0x000a, 0x2a54: 0x000a, 0x2a55: 0x000a, 0x2a56: 0x000a, 0x2a57: 0x000a,
+	0x2a58: 0x000a, 0x2a59: 0x000a, 0x2a5a: 0x000a, 0x2a5b: 0x000a, 0x2a5c: 0x000a, 0x2a5d: 0x000a,
+	0x2a5e: 0x000a, 0x2a5f: 0x000a, 0x2a60: 0x000a, 0x2a61: 0x000a, 0x2a62: 0x000a, 0x2a63: 0x000a,
+	0x2a64: 0x000a, 0x2a65: 0x000a,
+	0x2a7f: 0x000c,
+	// Block 0xaa, offset 0x2a80
+	0x2a80: 0x000c, 0x2a81: 0x000c,
+	0x2ab3: 0x000c, 0x2ab4: 0x000c, 0x2ab5: 0x000c,
+	0x2ab6: 0x000c, 0x2ab9: 0x000c, 0x2aba: 0x000c,
+	// Block 0xab, offset 0x2ac0
+	0x2ac0: 0x000c, 0x2ac1: 0x000c, 0x2ac2: 0x000c,
+	0x2ae7: 0x000c, 0x2ae8: 0x000c, 0x2ae9: 0x000c,
+	0x2aea: 0x000c, 0x2aeb: 0x000c, 0x2aed: 0x000c, 0x2aee: 0x000c, 0x2aef: 0x000c,
+	0x2af0: 0x000c, 0x2af1: 0x000c, 0x2af2: 0x000c, 0x2af3: 0x000c, 0x2af4: 0x000c,
+	// Block 0xac, offset 0x2b00
+	0x2b33: 0x000c,
+	// Block 0xad, offset 0x2b40
+	0x2b40: 0x000c, 0x2b41: 0x000c,
+	0x2b76: 0x000c, 0x2b77: 0x000c, 0x2b78: 0x000c, 0x2b79: 0x000c, 0x2b7a: 0x000c, 0x2b7b: 0x000c,
+	0x2b7c: 0x000c, 0x2b7d: 0x000c, 0x2b7e: 0x000c,
+	// Block 0xae, offset 0x2b80
+	0x2b89: 0x000c, 0x2b8a: 0x000c, 0x2b8b: 0x000c,
+	0x2b8c: 0x000c, 0x2b8f: 0x000c,
+	// Block 0xaf, offset 0x2bc0
+	0x2bef: 0x000c,
+	0x2bf0: 0x000c, 0x2bf1: 0x000c, 0x2bf4: 0x000c,
+	0x2bf6: 0x000c, 0x2bf7: 0x000c,
+	0x2bfe: 0x000c,
+	// Block 0xb0, offset 0x2c00
+	0x2c1f: 0x000c, 0x2c23: 0x000c,
+	0x2c24: 0x000c, 0x2c25: 0x000c, 0x2c26: 0x000c, 0x2c27: 0x000c, 0x2c28: 0x000c, 0x2c29: 0x000c,
+	0x2c2a: 0x000c,
+	// Block 0xb1, offset 0x2c40
+	0x2c40: 0x000c,
+	0x2c66: 0x000c, 0x2c67: 0x000c, 0x2c68: 0x000c, 0x2c69: 0x000c,
+	0x2c6a: 0x000c, 0x2c6b: 0x000c, 0x2c6c: 0x000c,
+	0x2c70: 0x000c, 0x2c71: 0x000c, 0x2c72: 0x000c, 0x2c73: 0x000c, 0x2c74: 0x000c,
+	// Block 0xb2, offset 0x2c80
+	0x2cb8: 0x000c, 0x2cb9: 0x000c, 0x2cba: 0x000c, 0x2cbb: 0x000c,
+	0x2cbc: 0x000c, 0x2cbd: 0x000c, 0x2cbe: 0x000c, 0x2cbf: 0x000c,
+	// Block 0xb3, offset 0x2cc0
+	0x2cc2: 0x000c, 0x2cc3: 0x000c, 0x2cc4: 0x000c,
+	0x2cc6: 0x000c,
+	0x2cde: 0x000c,
+	// Block 0xb4, offset 0x2d00
+	0x2d33: 0x000c, 0x2d34: 0x000c, 0x2d35: 0x000c,
+	0x2d36: 0x000c, 0x2d37: 0x000c, 0x2d38: 0x000c, 0x2d3a: 0x000c,
+	0x2d3f: 0x000c,
+	// Block 0xb5, offset 0x2d40
+	0x2d40: 0x000c, 0x2d42: 0x000c, 0x2d43: 0x000c,
+	// Block 0xb6, offset 0x2d80
+	0x2db2: 0x000c, 0x2db3: 0x000c, 0x2db4: 0x000c, 0x2db5: 0x000c,
+	0x2dbc: 0x000c, 0x2dbd: 0x000c, 0x2dbf: 0x000c,
+	// Block 0xb7, offset 0x2dc0
+	0x2dc0: 0x000c,
+	0x2ddc: 0x000c, 0x2ddd: 0x000c,
+	// Block 0xb8, offset 0x2e00
+	0x2e33: 0x000c, 0x2e34: 0x000c, 0x2e35: 0x000c,
+	0x2e36: 0x000c, 0x2e37: 0x000c, 0x2e38: 0x000c, 0x2e39: 0x000c, 0x2e3a: 0x000c,
+	0x2e3d: 0x000c, 0x2e3f: 0x000c,
+	// Block 0xb9, offset 0x2e40
+	0x2e40: 0x000c,
+	0x2e60: 0x000a, 0x2e61: 0x000a, 0x2e62: 0x000a, 0x2e63: 0x000a,
+	0x2e64: 0x000a, 0x2e65: 0x000a, 0x2e66: 0x000a, 0x2e67: 0x000a, 0x2e68: 0x000a, 0x2e69: 0x000a,
+	0x2e6a: 0x000a, 0x2e6b: 0x000a, 0x2e6c: 0x000a,
+	// Block 0xba, offset 0x2e80
+	0x2eab: 0x000c, 0x2ead: 0x000c,
+	0x2eb0: 0x000c, 0x2eb1: 0x000c, 0x2eb2: 0x000c, 0x2eb3: 0x000c, 0x2eb4: 0x000c, 0x2eb5: 0x000c,
+	0x2eb7: 0x000c,
+	// Block 0xbb, offset 0x2ec0
+	0x2edd: 0x000c,
+	0x2ede: 0x000c, 0x2edf: 0x000c, 0x2ee2: 0x000c, 0x2ee3: 0x000c,
+	0x2ee4: 0x000c, 0x2ee5: 0x000c, 0x2ee7: 0x000c, 0x2ee8: 0x000c, 0x2ee9: 0x000c,
+	0x2eea: 0x000c, 0x2eeb: 0x000c,
+	// Block 0xbc, offset 0x2f00
+	0x2f2f: 0x000c,
+	0x2f30: 0x000c, 0x2f31: 0x000c, 0x2f32: 0x000c, 0x2f33: 0x000c, 0x2f34: 0x000c, 0x2f35: 0x000c,
+	0x2f36: 0x000c, 0x2f37: 0x000c, 0x2f39: 0x000c, 0x2f3a: 0x000c,
+	// Block 0xbd, offset 0x2f40
+	0x2f7b: 0x000c,
+	0x2f7c: 0x000c, 0x2f7e: 0x000c,
+	// Block 0xbe, offset 0x2f80
+	0x2f83: 0x000c,
+	// Block 0xbf, offset 0x2fc0
+	0x2fd4: 0x000c, 0x2fd5: 0x000c, 0x2fd6: 0x000c, 0x2fd7: 0x000c,
+	0x2fda: 0x000c, 0x2fdb: 0x000c,
+	0x2fe0: 0x000c,
+	// Block 0xc0, offset 0x3000
+	0x3001: 0x000c, 0x3002: 0x000c, 0x3003: 0x000c, 0x3004: 0x000c, 0x3005: 0x000c,
+	0x3006: 0x000c, 0x3009: 0x000c, 0x300a: 0x000c,
+	0x3033: 0x000c, 0x3034: 0x000c, 0x3035: 0x000c,
+	0x3036: 0x000c, 0x3037: 0x000c, 0x3038: 0x000c, 0x303b: 0x000c,
+	0x303c: 0x000c, 0x303d: 0x000c, 0x303e: 0x000c,
+	// Block 0xc1, offset 0x3040
+	0x3047: 0x000c,
+	0x3051: 0x000c,
+	0x3052: 0x000c, 0x3053: 0x000c, 0x3054: 0x000c, 0x3055: 0x000c, 0x3056: 0x000c,
+	0x3059: 0x000c, 0x305a: 0x000c, 0x305b: 0x000c,
+	// Block 0xc2, offset 0x3080
+	0x308a: 0x000c, 0x308b: 0x000c,
+	0x308c: 0x000c, 0x308d: 0x000c, 0x308e: 0x000c, 0x308f: 0x000c, 0x3090: 0x000c, 0x3091: 0x000c,
+	0x3092: 0x000c, 0x3093: 0x000c, 0x3094: 0x000c, 0x3095: 0x000c, 0x3096: 0x000c,
+	0x3098: 0x000c, 0x3099: 0x000c,
+	// Block 0xc3, offset 0x30c0
+	0x30f0: 0x000c, 0x30f1: 0x000c, 0x30f2: 0x000c, 0x30f3: 0x000c, 0x30f4: 0x000c, 0x30f5: 0x000c,
+	0x30f6: 0x000c, 0x30f8: 0x000c, 0x30f9: 0x000c, 0x30fa: 0x000c, 0x30fb: 0x000c,
+	0x30fc: 0x000c, 0x30fd: 0x000c,
+	// Block 0xc4, offset 0x3100
+	0x3112: 0x000c, 0x3113: 0x000c, 0x3114: 0x000c, 0x3115: 0x000c, 0x3116: 0x000c, 0x3117: 0x000c,
+	0x3118: 0x000c, 0x3119: 0x000c, 0x311a: 0x000c, 0x311b: 0x000c, 0x311c: 0x000c, 0x311d: 0x000c,
+	0x311e: 0x000c, 0x311f: 0x000c, 0x3120: 0x000c, 0x3121: 0x000c, 0x3122: 0x000c, 0x3123: 0x000c,
+	0x3124: 0x000c, 0x3125: 0x000c, 0x3126: 0x000c, 0x3127: 0x000c,
+	0x312a: 0x000c, 0x312b: 0x000c, 0x312c: 0x000c, 0x312d: 0x000c, 0x312e: 0x000c, 0x312f: 0x000c,
+	0x3130: 0x000c, 0x3132: 0x000c, 0x3133: 0x000c, 0x3135: 0x000c,
+	0x3136: 0x000c,
+	// Block 0xc5, offset 0x3140
+	0x3171: 0x000c, 0x3172: 0x000c, 0x3173: 0x000c, 0x3174: 0x000c, 0x3175: 0x000c,
+	0x3176: 0x000c, 0x317a: 0x000c,
+	0x317c: 0x000c, 0x317d: 0x000c, 0x317f: 0x000c,
+	// Block 0xc6, offset 0x3180
+	0x3180: 0x000c, 0x3181: 0x000c, 0x3182: 0x000c, 0x3183: 0x000c, 0x3184: 0x000c, 0x3185: 0x000c,
+	0x3187: 0x000c,
+	// Block 0xc7, offset 0x31c0
+	0x31d0: 0x000c, 0x31d1: 0x000c,
+	0x31d5: 0x000c, 0x31d7: 0x000c,
+	// Block 0xc8, offset 0x3200
+	0x3233: 0x000c, 0x3234: 0x000c,
+	// Block 0xc9, offset 0x3240
+	0x3255: 0x000a, 0x3256: 0x000a, 0x3257: 0x000a,
+	0x3258: 0x000a, 0x3259: 0x000a, 0x325a: 0x000a, 0x325b: 0x000a, 0x325c: 0x000a, 0x325d: 0x0004,
+	0x325e: 0x0004, 0x325f: 0x0004, 0x3260: 0x0004, 0x3261: 0x000a, 0x3262: 0x000a, 0x3263: 0x000a,
+	0x3264: 0x000a, 0x3265: 0x000a, 0x3266: 0x000a, 0x3267: 0x000a, 0x3268: 0x000a, 0x3269: 0x000a,
+	0x326a: 0x000a, 0x326b: 0x000a, 0x326c: 0x000a, 0x326d: 0x000a, 0x326e: 0x000a, 0x326f: 0x000a,
+	0x3270: 0x000a, 0x3271: 0x000a,
+	// Block 0xca, offset 0x3280
+	0x32b0: 0x000c, 0x32b1: 0x000c, 0x32b2: 0x000c, 0x32b3: 0x000c, 0x32b4: 0x000c,
+	// Block 0xcb, offset 0x32c0
+	0x32f0: 0x000c, 0x32f1: 0x000c, 0x32f2: 0x000c, 0x32f3: 0x000c, 0x32f4: 0x000c, 0x32f5: 0x000c,
+	0x32f6: 0x000c,
+	// Block 0xcc, offset 0x3300
+	0x330f: 0x000c,
+	// Block 0xcd, offset 0x3340
+	0x334f: 0x000c, 0x3350: 0x000c, 0x3351: 0x000c,
+	0x3352: 0x000c,
+	// Block 0xce, offset 0x3380
+	0x33a2: 0x000a,
+	0x33a4: 0x000c,
+	// Block 0xcf, offset 0x33c0
+	0x33dd: 0x000c,
+	0x33de: 0x000c, 0x33e0: 0x000b, 0x33e1: 0x000b, 0x33e2: 0x000b, 0x33e3: 0x000b,
+	// Block 0xd0, offset 0x3400
+	0x3427: 0x000c, 0x3428: 0x000c, 0x3429: 0x000c,
+	0x3433: 0x000b, 0x3434: 0x000b, 0x3435: 0x000b,
+	0x3436: 0x000b, 0x3437: 0x000b, 0x3438: 0x000b, 0x3439: 0x000b, 0x343a: 0x000b, 0x343b: 0x000c,
+	0x343c: 0x000c, 0x343d: 0x000c, 0x343e: 0x000c, 0x343f: 0x000c,
+	// Block 0xd1, offset 0x3440
+	0x3440: 0x000c, 0x3441: 0x000c, 0x3442: 0x000c, 0x3445: 0x000c,
+	0x3446: 0x000c, 0x3447: 0x000c, 0x3448: 0x000c, 0x3449: 0x000c, 0x344a: 0x000c, 0x344b: 0x000c,
+	0x346a: 0x000c, 0x346b: 0x000c, 0x346c: 0x000c, 0x346d: 0x000c,
+	// Block 0xd2, offset 0x3480
+	0x3480: 0x000a, 0x3481: 0x000a, 0x3482: 0x000c, 0x3483: 0x000c, 0x3484: 0x000c, 0x3485: 0x000a,
+	// Block 0xd3, offset 0x34c0
+	0x34c0: 0x000a, 0x34c1: 0x000a, 0x34c2: 0x000a, 0x34c3: 0x000a, 0x34c4: 0x000a, 0x34c5: 0x000a,
+	0x34c6: 0x000a, 0x34c7: 0x000a, 0x34c8: 0x000a, 0x34c9: 0x000a, 0x34ca: 0x000a, 0x34cb: 0x000a,
+	0x34cc: 0x000a, 0x34cd: 0x000a, 0x34ce: 0x000a, 0x34cf: 0x000a, 0x34d0: 0x000a, 0x34d1: 0x000a,
+	0x34d2: 0x000a, 0x34d3: 0x000a, 0x34d4: 0x000a, 0x34d5: 0x000a, 0x34d6: 0x000a,
+	// Block 0xd4, offset 0x3500
+	0x351b: 0x000a,
+	// Block 0xd5, offset 0x3540
+	0x3555: 0x000a,
+	// Block 0xd6, offset 0x3580
+	0x358f: 0x000a,
+	// Block 0xd7, offset 0x35c0
+	0x35c9: 0x000a,
+	// Block 0xd8, offset 0x3600
+	0x3603: 0x000a,
+	0x360e: 0x0002, 0x360f: 0x0002, 0x3610: 0x0002, 0x3611: 0x0002,
+	0x3612: 0x0002, 0x3613: 0x0002, 0x3614: 0x0002, 0x3615: 0x0002, 0x3616: 0x0002, 0x3617: 0x0002,
+	0x3618: 0x0002, 0x3619: 0x0002, 0x361a: 0x0002, 0x361b: 0x0002, 0x361c: 0x0002, 0x361d: 0x0002,
+	0x361e: 0x0002, 0x361f: 0x0002, 0x3620: 0x0002, 0x3621: 0x0002, 0x3622: 0x0002, 0x3623: 0x0002,
+	0x3624: 0x0002, 0x3625: 0x0002, 0x3626: 0x0002, 0x3627: 0x0002, 0x3628: 0x0002, 0x3629: 0x0002,
+	0x362a: 0x0002, 0x362b: 0x0002, 0x362c: 0x0002, 0x362d: 0x0002, 0x362e: 0x0002, 0x362f: 0x0002,
+	0x3630: 0x0002, 0x3631: 0x0002, 0x3632: 0x0002, 0x3633: 0x0002, 0x3634: 0x0002, 0x3635: 0x0002,
+	0x3636: 0x0002, 0x3637: 0x0002, 0x3638: 0x0002, 0x3639: 0x0002, 0x363a: 0x0002, 0x363b: 0x0002,
+	0x363c: 0x0002, 0x363d: 0x0002, 0x363e: 0x0002, 0x363f: 0x0002,
+	// Block 0xd9, offset 0x3640
+	0x3640: 0x000c, 0x3641: 0x000c, 0x3642: 0x000c, 0x3643: 0x000c, 0x3644: 0x000c, 0x3645: 0x000c,
+	0x3646: 0x000c, 0x3647: 0x000c, 0x3648: 0x000c, 0x3649: 0x000c, 0x364a: 0x000c, 0x364b: 0x000c,
+	0x364c: 0x000c, 0x364d: 0x000c, 0x364e: 0x000c, 0x364f: 0x000c, 0x3650: 0x000c, 0x3651: 0x000c,
+	0x3652: 0x000c, 0x3653: 0x000c, 0x3654: 0x000c, 0x3655: 0x000c, 0x3656: 0x000c, 0x3657: 0x000c,
+	0x3658: 0x000c, 0x3659: 0x000c, 0x365a: 0x000c, 0x365b: 0x000c, 0x365c: 0x000c, 0x365d: 0x000c,
+	0x365e: 0x000c, 0x365f: 0x000c, 0x3660: 0x000c, 0x3661: 0x000c, 0x3662: 0x000c, 0x3663: 0x000c,
+	0x3664: 0x000c, 0x3665: 0x000c, 0x3666: 0x000c, 0x3667: 0x000c, 0x3668: 0x000c, 0x3669: 0x000c,
+	0x366a: 0x000c, 0x366b: 0x000c, 0x366c: 0x000c, 0x366d: 0x000c, 0x366e: 0x000c, 0x366f: 0x000c,
+	0x3670: 0x000c, 0x3671: 0x000c, 0x3672: 0x000c, 0x3673: 0x000c, 0x3674: 0x000c, 0x3675: 0x000c,
+	0x3676: 0x000c, 0x367b: 0x000c,
+	0x367c: 0x000c, 0x367d: 0x000c, 0x367e: 0x000c, 0x367f: 0x000c,
+	// Block 0xda, offset 0x3680
+	0x3680: 0x000c, 0x3681: 0x000c, 0x3682: 0x000c, 0x3683: 0x000c, 0x3684: 0x000c, 0x3685: 0x000c,
+	0x3686: 0x000c, 0x3687: 0x000c, 0x3688: 0x000c, 0x3689: 0x000c, 0x368a: 0x000c, 0x368b: 0x000c,
+	0x368c: 0x000c, 0x368d: 0x000c, 0x368e: 0x000c, 0x368f: 0x000c, 0x3690: 0x000c, 0x3691: 0x000c,
+	0x3692: 0x000c, 0x3693: 0x000c, 0x3694: 0x000c, 0x3695: 0x000c, 0x3696: 0x000c, 0x3697: 0x000c,
+	0x3698: 0x000c, 0x3699: 0x000c, 0x369a: 0x000c, 0x369b: 0x000c, 0x369c: 0x000c, 0x369d: 0x000c,
+	0x369e: 0x000c, 0x369f: 0x000c, 0x36a0: 0x000c, 0x36a1: 0x000c, 0x36a2: 0x000c, 0x36a3: 0x000c,
+	0x36a4: 0x000c, 0x36a5: 0x000c, 0x36a6: 0x000c, 0x36a7: 0x000c, 0x36a8: 0x000c, 0x36a9: 0x000c,
+	0x36aa: 0x000c, 0x36ab: 0x000c, 0x36ac: 0x000c,
+	0x36b5: 0x000c,
+	// Block 0xdb, offset 0x36c0
+	0x36c4: 0x000c,
+	0x36db: 0x000c, 0x36dc: 0x000c, 0x36dd: 0x000c,
+	0x36de: 0x000c, 0x36df: 0x000c, 0x36e1: 0x000c, 0x36e2: 0x000c, 0x36e3: 0x000c,
+	0x36e4: 0x000c, 0x36e5: 0x000c, 0x36e6: 0x000c, 0x36e7: 0x000c, 0x36e8: 0x000c, 0x36e9: 0x000c,
+	0x36ea: 0x000c, 0x36eb: 0x000c, 0x36ec: 0x000c, 0x36ed: 0x000c, 0x36ee: 0x000c, 0x36ef: 0x000c,
+	// Block 0xdc, offset 0x3700
+	0x3700: 0x000c, 0x3701: 0x000c, 0x3702: 0x000c, 0x3703: 0x000c, 0x3704: 0x000c, 0x3705: 0x000c,
+	0x3706: 0x000c, 0x3708: 0x000c, 0x3709: 0x000c, 0x370a: 0x000c, 0x370b: 0x000c,
+	0x370c: 0x000c, 0x370d: 0x000c, 0x370e: 0x000c, 0x370f: 0x000c, 0x3710: 0x000c, 0x3711: 0x000c,
+	0x3712: 0x000c, 0x3713: 0x000c, 0x3714: 0x000c, 0x3715: 0x000c, 0x3716: 0x000c, 0x3717: 0x000c,
+	0x3718: 0x000c, 0x371b: 0x000c, 0x371c: 0x000c, 0x371d: 0x000c,
+	0x371e: 0x000c, 0x371f: 0x000c, 0x3720: 0x000c, 0x3721: 0x000c, 0x3723: 0x000c,
+	0x3724: 0x000c, 0x3726: 0x000c, 0x3727: 0x000c, 0x3728: 0x000c, 0x3729: 0x000c,
+	0x372a: 0x000c,
+	// Block 0xdd, offset 0x3740
+	0x376c: 0x000c, 0x376d: 0x000c, 0x376e: 0x000c, 0x376f: 0x000c,
+	0x377f: 0x0004,
+	// Block 0xde, offset 0x3780
+	0x3780: 0x0001, 0x3781: 0x0001, 0x3782: 0x0001, 0x3783: 0x0001, 0x3784: 0x0001, 0x3785: 0x0001,
+	0x3786: 0x0001, 0x3787: 0x0001, 0x3788: 0x0001, 0x3789: 0x0001, 0x378a: 0x0001, 0x378b: 0x0001,
+	0x378c: 0x0001, 0x378d: 0x0001, 0x378e: 0x0001, 0x378f: 0x0001, 0x3790: 0x000c, 0x3791: 0x000c,
+	0x3792: 0x000c, 0x3793: 0x000c, 0x3794: 0x000c, 0x3795: 0x000c, 0x3796: 0x000c, 0x3797: 0x0001,
+	0x3798: 0x0001, 0x3799: 0x0001, 0x379a: 0x0001, 0x379b: 0x0001, 0x379c: 0x0001, 0x379d: 0x0001,
+	0x379e: 0x0001, 0x379f: 0x0001, 0x37a0: 0x0001, 0x37a1: 0x0001, 0x37a2: 0x0001, 0x37a3: 0x0001,
+	0x37a4: 0x0001, 0x37a5: 0x0001, 0x37a6: 0x0001, 0x37a7: 0x0001, 0x37a8: 0x0001, 0x37a9: 0x0001,
+	0x37aa: 0x0001, 0x37ab: 0x0001, 0x37ac: 0x0001, 0x37ad: 0x0001, 0x37ae: 0x0001, 0x37af: 0x0001,
+	0x37b0: 0x0001, 0x37b1: 0x0001, 0x37b2: 0x0001, 0x37b3: 0x0001, 0x37b4: 0x0001, 0x37b5: 0x0001,
+	0x37b6: 0x0001, 0x37b7: 0x0001, 0x37b8: 0x0001, 0x37b9: 0x0001, 0x37ba: 0x0001, 0x37bb: 0x0001,
+	0x37bc: 0x0001, 0x37bd: 0x0001, 0x37be: 0x0001, 0x37bf: 0x0001,
+	// Block 0xdf, offset 0x37c0
+	0x37c0: 0x0001, 0x37c1: 0x0001, 0x37c2: 0x0001, 0x37c3: 0x0001, 0x37c4: 0x000c, 0x37c5: 0x000c,
+	0x37c6: 0x000c, 0x37c7: 0x000c, 0x37c8: 0x000c, 0x37c9: 0x000c, 0x37ca: 0x000c, 0x37cb: 0x0001,
+	0x37cc: 0x0001, 0x37cd: 0x0001, 0x37ce: 0x0001, 0x37cf: 0x0001, 0x37d0: 0x0001, 0x37d1: 0x0001,
+	0x37d2: 0x0001, 0x37d3: 0x0001, 0x37d4: 0x0001, 0x37d5: 0x0001, 0x37d6: 0x0001, 0x37d7: 0x0001,
+	0x37d8: 0x0001, 0x37d9: 0x0001, 0x37da: 0x0001, 0x37db: 0x0001, 0x37dc: 0x0001, 0x37dd: 0x0001,
+	0x37de: 0x0001, 0x37df: 0x0001, 0x37e0: 0x0001, 0x37e1: 0x0001, 0x37e2: 0x0001, 0x37e3: 0x0001,
+	0x37e4: 0x0001, 0x37e5: 0x0001, 0x37e6: 0x0001, 0x37e7: 0x0001, 0x37e8: 0x0001, 0x37e9: 0x0001,
+	0x37ea: 0x0001, 0x37eb: 0x0001, 0x37ec: 0x0001, 0x37ed: 0x0001, 0x37ee: 0x0001, 0x37ef: 0x0001,
+	0x37f0: 0x0001, 0x37f1: 0x0001, 0x37f2: 0x0001, 0x37f3: 0x0001, 0x37f4: 0x0001, 0x37f5: 0x0001,
+	0x37f6: 0x0001, 0x37f7: 0x0001, 0x37f8: 0x0001, 0x37f9: 0x0001, 0x37fa: 0x0001, 0x37fb: 0x0001,
+	0x37fc: 0x0001, 0x37fd: 0x0001, 0x37fe: 0x0001, 0x37ff: 0x0001,
+	// Block 0xe0, offset 0x3800
+	0x3800: 0x000d, 0x3801: 0x000d, 0x3802: 0x000d, 0x3803: 0x000d, 0x3804: 0x000d, 0x3805: 0x000d,
+	0x3806: 0x000d, 0x3807: 0x000d, 0x3808: 0x000d, 0x3809: 0x000d, 0x380a: 0x000d, 0x380b: 0x000d,
+	0x380c: 0x000d, 0x380d: 0x000d, 0x380e: 0x000d, 0x380f: 0x000d, 0x3810: 0x0001, 0x3811: 0x0001,
+	0x3812: 0x0001, 0x3813: 0x0001, 0x3814: 0x0001, 0x3815: 0x0001, 0x3816: 0x0001, 0x3817: 0x0001,
+	0x3818: 0x0001, 0x3819: 0x0001, 0x381a: 0x0001, 0x381b: 0x0001, 0x381c: 0x0001, 0x381d: 0x0001,
+	0x381e: 0x0001, 0x381f: 0x0001, 0x3820: 0x0001, 0x3821: 0x0001, 0x3822: 0x0001, 0x3823: 0x0001,
+	0x3824: 0x0001, 0x3825: 0x0001, 0x3826: 0x0001, 0x3827: 0x0001, 0x3828: 0x0001, 0x3829: 0x0001,
+	0x382a: 0x0001, 0x382b: 0x0001, 0x382c: 0x0001, 0x382d: 0x0001, 0x382e: 0x0001, 0x382f: 0x0001,
+	0x3830: 0x0001, 0x3831: 0x0001, 0x3832: 0x0001, 0x3833: 0x0001, 0x3834: 0x0001, 0x3835: 0x0001,
+	0x3836: 0x0001, 0x3837: 0x0001, 0x3838: 0x0001, 0x3839: 0x0001, 0x383a: 0x0001, 0x383b: 0x0001,
+	0x383c: 0x0001, 0x383d: 0x0001, 0x383e: 0x0001, 0x383f: 0x0001,
+	// Block 0xe1, offset 0x3840
+	0x3840: 0x000d, 0x3841: 0x000d, 0x3842: 0x000d, 0x3843: 0x000d, 0x3844: 0x000d, 0x3845: 0x000d,
+	0x3846: 0x000d, 0x3847: 0x000d, 0x3848: 0x000d, 0x3849: 0x000d, 0x384a: 0x000d, 0x384b: 0x000d,
+	0x384c: 0x000d, 0x384d: 0x000d, 0x384e: 0x000d, 0x384f: 0x000d, 0x3850: 0x000d, 0x3851: 0x000d,
+	0x3852: 0x000d, 0x3853: 0x000d, 0x3854: 0x000d, 0x3855: 0x000d, 0x3856: 0x000d, 0x3857: 0x000d,
+	0x3858: 0x000d, 0x3859: 0x000d, 0x385a: 0x000d, 0x385b: 0x000d, 0x385c: 0x000d, 0x385d: 0x000d,
+	0x385e: 0x000d, 0x385f: 0x000d, 0x3860: 0x000d, 0x3861: 0x000d, 0x3862: 0x000d, 0x3863: 0x000d,
+	0x3864: 0x000d, 0x3865: 0x000d, 0x3866: 0x000d, 0x3867: 0x000d, 0x3868: 0x000d, 0x3869: 0x000d,
+	0x386a: 0x000d, 0x386b: 0x000d, 0x386c: 0x000d, 0x386d: 0x000d, 0x386e: 0x000d, 0x386f: 0x000d,
+	0x3870: 0x000a, 0x3871: 0x000a, 0x3872: 0x000d, 0x3873: 0x000d, 0x3874: 0x000d, 0x3875: 0x000d,
+	0x3876: 0x000d, 0x3877: 0x000d, 0x3878: 0x000d, 0x3879: 0x000d, 0x387a: 0x000d, 0x387b: 0x000d,
+	0x387c: 0x000d, 0x387d: 0x000d, 0x387e: 0x000d, 0x387f: 0x000d,
+	// Block 0xe2, offset 0x3880
+	0x3880: 0x000a, 0x3881: 0x000a, 0x3882: 0x000a, 0x3883: 0x000a, 0x3884: 0x000a, 0x3885: 0x000a,
+	0x3886: 0x000a, 0x3887: 0x000a, 0x3888: 0x000a, 0x3889: 0x000a, 0x388a: 0x000a, 0x388b: 0x000a,
+	0x388c: 0x000a, 0x388d: 0x000a, 0x388e: 0x000a, 0x388f: 0x000a, 0x3890: 0x000a, 0x3891: 0x000a,
+	0x3892: 0x000a, 0x3893: 0x000a, 0x3894: 0x000a, 0x3895: 0x000a, 0x3896: 0x000a, 0x3897: 0x000a,
+	0x3898: 0x000a, 0x3899: 0x000a, 0x389a: 0x000a, 0x389b: 0x000a, 0x389c: 0x000a, 0x389d: 0x000a,
+	0x389e: 0x000a, 0x389f: 0x000a, 0x38a0: 0x000a, 0x38a1: 0x000a, 0x38a2: 0x000a, 0x38a3: 0x000a,
+	0x38a4: 0x000a, 0x38a5: 0x000a, 0x38a6: 0x000a, 0x38a7: 0x000a, 0x38a8: 0x000a, 0x38a9: 0x000a,
+	0x38aa: 0x000a, 0x38ab: 0x000a,
+	0x38b0: 0x000a, 0x38b1: 0x000a, 0x38b2: 0x000a, 0x38b3: 0x000a, 0x38b4: 0x000a, 0x38b5: 0x000a,
+	0x38b6: 0x000a, 0x38b7: 0x000a, 0x38b8: 0x000a, 0x38b9: 0x000a, 0x38ba: 0x000a, 0x38bb: 0x000a,
+	0x38bc: 0x000a, 0x38bd: 0x000a, 0x38be: 0x000a, 0x38bf: 0x000a,
+	// Block 0xe3, offset 0x38c0
+	0x38c0: 0x000a, 0x38c1: 0x000a, 0x38c2: 0x000a, 0x38c3: 0x000a, 0x38c4: 0x000a, 0x38c5: 0x000a,
+	0x38c6: 0x000a, 0x38c7: 0x000a, 0x38c8: 0x000a, 0x38c9: 0x000a, 0x38ca: 0x000a, 0x38cb: 0x000a,
+	0x38cc: 0x000a, 0x38cd: 0x000a, 0x38ce: 0x000a, 0x38cf: 0x000a, 0x38d0: 0x000a, 0x38d1: 0x000a,
+	0x38d2: 0x000a, 0x38d3: 0x000a,
+	0x38e0: 0x000a, 0x38e1: 0x000a, 0x38e2: 0x000a, 0x38e3: 0x000a,
+	0x38e4: 0x000a, 0x38e5: 0x000a, 0x38e6: 0x000a, 0x38e7: 0x000a, 0x38e8: 0x000a, 0x38e9: 0x000a,
+	0x38ea: 0x000a, 0x38eb: 0x000a, 0x38ec: 0x000a, 0x38ed: 0x000a, 0x38ee: 0x000a,
+	0x38f1: 0x000a, 0x38f2: 0x000a, 0x38f3: 0x000a, 0x38f4: 0x000a, 0x38f5: 0x000a,
+	0x38f6: 0x000a, 0x38f7: 0x000a, 0x38f8: 0x000a, 0x38f9: 0x000a, 0x38fa: 0x000a, 0x38fb: 0x000a,
+	0x38fc: 0x000a, 0x38fd: 0x000a, 0x38fe: 0x000a, 0x38ff: 0x000a,
+	// Block 0xe4, offset 0x3900
+	0x3901: 0x000a, 0x3902: 0x000a, 0x3903: 0x000a, 0x3904: 0x000a, 0x3905: 0x000a,
+	0x3906: 0x000a, 0x3907: 0x000a, 0x3908: 0x000a, 0x3909: 0x000a, 0x390a: 0x000a, 0x390b: 0x000a,
+	0x390c: 0x000a, 0x390d: 0x000a, 0x390e: 0x000a, 0x390f: 0x000a, 0x3911: 0x000a,
+	0x3912: 0x000a, 0x3913: 0x000a, 0x3914: 0x000a, 0x3915: 0x000a, 0x3916: 0x000a, 0x3917: 0x000a,
+	0x3918: 0x000a, 0x3919: 0x000a, 0x391a: 0x000a, 0x391b: 0x000a, 0x391c: 0x000a, 0x391d: 0x000a,
+	0x391e: 0x000a, 0x391f: 0x000a, 0x3920: 0x000a, 0x3921: 0x000a, 0x3922: 0x000a, 0x3923: 0x000a,
+	0x3924: 0x000a, 0x3925: 0x000a, 0x3926: 0x000a, 0x3927: 0x000a, 0x3928: 0x000a, 0x3929: 0x000a,
+	0x392a: 0x000a, 0x392b: 0x000a, 0x392c: 0x000a, 0x392d: 0x000a, 0x392e: 0x000a, 0x392f: 0x000a,
+	0x3930: 0x000a, 0x3931: 0x000a, 0x3932: 0x000a, 0x3933: 0x000a, 0x3934: 0x000a, 0x3935: 0x000a,
+	// Block 0xe5, offset 0x3940
+	0x3940: 0x0002, 0x3941: 0x0002, 0x3942: 0x0002, 0x3943: 0x0002, 0x3944: 0x0002, 0x3945: 0x0002,
+	0x3946: 0x0002, 0x3947: 0x0002, 0x3948: 0x0002, 0x3949: 0x0002, 0x394a: 0x0002, 0x394b: 0x000a,
+	0x394c: 0x000a, 0x394d: 0x000a, 0x394e: 0x000a, 0x394f: 0x000a,
+	0x396f: 0x000a,
+	// Block 0xe6, offset 0x3980
+	0x39aa: 0x000a, 0x39ab: 0x000a, 0x39ac: 0x000a, 0x39ad: 0x000a, 0x39ae: 0x000a, 0x39af: 0x000a,
+	// Block 0xe7, offset 0x39c0
+	0x39ed: 0x000a,
+	// Block 0xe8, offset 0x3a00
+	0x3a20: 0x000a, 0x3a21: 0x000a, 0x3a22: 0x000a, 0x3a23: 0x000a,
+	0x3a24: 0x000a, 0x3a25: 0x000a,
+	// Block 0xe9, offset 0x3a40
+	0x3a40: 0x000a, 0x3a41: 0x000a, 0x3a42: 0x000a, 0x3a43: 0x000a, 0x3a44: 0x000a, 0x3a45: 0x000a,
+	0x3a46: 0x000a, 0x3a47: 0x000a, 0x3a48: 0x000a, 0x3a49: 0x000a, 0x3a4a: 0x000a, 0x3a4b: 0x000a,
+	0x3a4c: 0x000a, 0x3a4d: 0x000a, 0x3a4e: 0x000a, 0x3a4f: 0x000a, 0x3a50: 0x000a, 0x3a51: 0x000a,
+	0x3a52: 0x000a, 0x3a53: 0x000a, 0x3a54: 0x000a, 0x3a55: 0x000a, 0x3a56: 0x000a, 0x3a57: 0x000a,
+	0x3a60: 0x000a, 0x3a61: 0x000a, 0x3a62: 0x000a, 0x3a63: 0x000a,
+	0x3a64: 0x000a, 0x3a65: 0x000a, 0x3a66: 0x000a, 0x3a67: 0x000a, 0x3a68: 0x000a, 0x3a69: 0x000a,
+	0x3a6a: 0x000a, 0x3a6b: 0x000a, 0x3a6c: 0x000a,
+	0x3a70: 0x000a, 0x3a71: 0x000a, 0x3a72: 0x000a, 0x3a73: 0x000a, 0x3a74: 0x000a, 0x3a75: 0x000a,
+	0x3a76: 0x000a, 0x3a77: 0x000a, 0x3a78: 0x000a, 0x3a79: 0x000a, 0x3a7a: 0x000a, 0x3a7b: 0x000a,
+	0x3a7c: 0x000a,
+	// Block 0xea, offset 0x3a80
+	0x3a80: 0x000a, 0x3a81: 0x000a, 0x3a82: 0x000a, 0x3a83: 0x000a, 0x3a84: 0x000a, 0x3a85: 0x000a,
+	0x3a86: 0x000a, 0x3a87: 0x000a, 0x3a88: 0x000a, 0x3a89: 0x000a, 0x3a8a: 0x000a, 0x3a8b: 0x000a,
+	0x3a8c: 0x000a, 0x3a8d: 0x000a, 0x3a8e: 0x000a, 0x3a8f: 0x000a, 0x3a90: 0x000a, 0x3a91: 0x000a,
+	0x3a92: 0x000a, 0x3a93: 0x000a, 0x3a94: 0x000a, 0x3a95: 0x000a, 0x3a96: 0x000a, 0x3a97: 0x000a,
+	0x3a98: 0x000a,
+	0x3aa0: 0x000a, 0x3aa1: 0x000a, 0x3aa2: 0x000a, 0x3aa3: 0x000a,
+	0x3aa4: 0x000a, 0x3aa5: 0x000a, 0x3aa6: 0x000a, 0x3aa7: 0x000a, 0x3aa8: 0x000a, 0x3aa9: 0x000a,
+	0x3aaa: 0x000a, 0x3aab: 0x000a,
+	// Block 0xeb, offset 0x3ac0
+	0x3ac0: 0x000a, 0x3ac1: 0x000a, 0x3ac2: 0x000a, 0x3ac3: 0x000a, 0x3ac4: 0x000a, 0x3ac5: 0x000a,
+	0x3ac6: 0x000a, 0x3ac7: 0x000a, 0x3ac8: 0x000a, 0x3ac9: 0x000a, 0x3aca: 0x000a, 0x3acb: 0x000a,
+	0x3ad0: 0x000a, 0x3ad1: 0x000a,
+	0x3ad2: 0x000a, 0x3ad3: 0x000a, 0x3ad4: 0x000a, 0x3ad5: 0x000a, 0x3ad6: 0x000a, 0x3ad7: 0x000a,
+	0x3ad8: 0x000a, 0x3ad9: 0x000a, 0x3ada: 0x000a, 0x3adb: 0x000a, 0x3adc: 0x000a, 0x3add: 0x000a,
+	0x3ade: 0x000a, 0x3adf: 0x000a, 0x3ae0: 0x000a, 0x3ae1: 0x000a, 0x3ae2: 0x000a, 0x3ae3: 0x000a,
+	0x3ae4: 0x000a, 0x3ae5: 0x000a, 0x3ae6: 0x000a, 0x3ae7: 0x000a, 0x3ae8: 0x000a, 0x3ae9: 0x000a,
+	0x3aea: 0x000a, 0x3aeb: 0x000a, 0x3aec: 0x000a, 0x3aed: 0x000a, 0x3aee: 0x000a, 0x3aef: 0x000a,
+	0x3af0: 0x000a, 0x3af1: 0x000a, 0x3af2: 0x000a, 0x3af3: 0x000a, 0x3af4: 0x000a, 0x3af5: 0x000a,
+	0x3af6: 0x000a, 0x3af7: 0x000a, 0x3af8: 0x000a, 0x3af9: 0x000a, 0x3afa: 0x000a, 0x3afb: 0x000a,
+	0x3afc: 0x000a, 0x3afd: 0x000a, 0x3afe: 0x000a, 0x3aff: 0x000a,
+	// Block 0xec, offset 0x3b00
+	0x3b00: 0x000a, 0x3b01: 0x000a, 0x3b02: 0x000a, 0x3b03: 0x000a, 0x3b04: 0x000a, 0x3b05: 0x000a,
+	0x3b06: 0x000a, 0x3b07: 0x000a,
+	0x3b10: 0x000a, 0x3b11: 0x000a,
+	0x3b12: 0x000a, 0x3b13: 0x000a, 0x3b14: 0x000a, 0x3b15: 0x000a, 0x3b16: 0x000a, 0x3b17: 0x000a,
+	0x3b18: 0x000a, 0x3b19: 0x000a,
+	0x3b20: 0x000a, 0x3b21: 0x000a, 0x3b22: 0x000a, 0x3b23: 0x000a,
+	0x3b24: 0x000a, 0x3b25: 0x000a, 0x3b26: 0x000a, 0x3b27: 0x000a, 0x3b28: 0x000a, 0x3b29: 0x000a,
+	0x3b2a: 0x000a, 0x3b2b: 0x000a, 0x3b2c: 0x000a, 0x3b2d: 0x000a, 0x3b2e: 0x000a, 0x3b2f: 0x000a,
+	0x3b30: 0x000a, 0x3b31: 0x000a, 0x3b32: 0x000a, 0x3b33: 0x000a, 0x3b34: 0x000a, 0x3b35: 0x000a,
+	0x3b36: 0x000a, 0x3b37: 0x000a, 0x3b38: 0x000a, 0x3b39: 0x000a, 0x3b3a: 0x000a, 0x3b3b: 0x000a,
+	0x3b3c: 0x000a, 0x3b3d: 0x000a, 0x3b3e: 0x000a, 0x3b3f: 0x000a,
+	// Block 0xed, offset 0x3b40
+	0x3b40: 0x000a, 0x3b41: 0x000a, 0x3b42: 0x000a, 0x3b43: 0x000a, 0x3b44: 0x000a, 0x3b45: 0x000a,
+	0x3b46: 0x000a, 0x3b47: 0x000a,
+	0x3b50: 0x000a, 0x3b51: 0x000a,
+	0x3b52: 0x000a, 0x3b53: 0x000a, 0x3b54: 0x000a, 0x3b55: 0x000a, 0x3b56: 0x000a, 0x3b57: 0x000a,
+	0x3b58: 0x000a, 0x3b59: 0x000a, 0x3b5a: 0x000a, 0x3b5b: 0x000a, 0x3b5c: 0x000a, 0x3b5d: 0x000a,
+	0x3b5e: 0x000a, 0x3b5f: 0x000a, 0x3b60: 0x000a, 0x3b61: 0x000a, 0x3b62: 0x000a, 0x3b63: 0x000a,
+	0x3b64: 0x000a, 0x3b65: 0x000a, 0x3b66: 0x000a, 0x3b67: 0x000a, 0x3b68: 0x000a, 0x3b69: 0x000a,
+	0x3b6a: 0x000a, 0x3b6b: 0x000a, 0x3b6c: 0x000a, 0x3b6d: 0x000a,
+	0x3b70: 0x000a, 0x3b71: 0x000a,
+	// Block 0xee, offset 0x3b80
+	0x3b80: 0x000a, 0x3b81: 0x000a, 0x3b82: 0x000a, 0x3b83: 0x000a, 0x3b84: 0x000a, 0x3b85: 0x000a,
+	0x3b86: 0x000a, 0x3b87: 0x000a, 0x3b88: 0x000a, 0x3b89: 0x000a, 0x3b8a: 0x000a, 0x3b8b: 0x000a,
+	0x3b8c: 0x000a, 0x3b8d: 0x000a, 0x3b8e: 0x000a, 0x3b8f: 0x000a, 0x3b90: 0x000a, 0x3b91: 0x000a,
+	0x3b92: 0x000a, 0x3b93: 0x000a, 0x3b94: 0x000a, 0x3b95: 0x000a, 0x3b96: 0x000a, 0x3b97: 0x000a,
+	0x3b98: 0x000a, 0x3b99: 0x000a, 0x3b9a: 0x000a, 0x3b9b: 0x000a, 0x3b9c: 0x000a, 0x3b9d: 0x000a,
+	0x3b9e: 0x000a, 0x3b9f: 0x000a, 0x3ba0: 0x000a, 0x3ba1: 0x000a, 0x3ba2: 0x000a, 0x3ba3: 0x000a,
+	0x3ba4: 0x000a, 0x3ba5: 0x000a, 0x3ba6: 0x000a, 0x3ba7: 0x000a, 0x3ba8: 0x000a, 0x3ba9: 0x000a,
+	0x3baa: 0x000a, 0x3bab: 0x000a, 0x3bac: 0x000a, 0x3bad: 0x000a, 0x3bae: 0x000a, 0x3baf: 0x000a,
+	0x3bb0: 0x000a, 0x3bb1: 0x000a, 0x3bb2: 0x000a, 0x3bb3: 0x000a, 0x3bb4: 0x000a, 0x3bb5: 0x000a,
+	0x3bb6: 0x000a, 0x3bb7: 0x000a, 0x3bb8: 0x000a, 0x3bba: 0x000a, 0x3bbb: 0x000a,
+	0x3bbc: 0x000a, 0x3bbd: 0x000a, 0x3bbe: 0x000a, 0x3bbf: 0x000a,
+	// Block 0xef, offset 0x3bc0
+	0x3bc0: 0x000a, 0x3bc1: 0x000a, 0x3bc2: 0x000a, 0x3bc3: 0x000a, 0x3bc4: 0x000a, 0x3bc5: 0x000a,
+	0x3bc6: 0x000a, 0x3bc7: 0x000a, 0x3bc8: 0x000a, 0x3bc9: 0x000a, 0x3bca: 0x000a, 0x3bcb: 0x000a,
+	0x3bcd: 0x000a, 0x3bce: 0x000a, 0x3bcf: 0x000a, 0x3bd0: 0x000a, 0x3bd1: 0x000a,
+	0x3bd2: 0x000a, 0x3bd3: 0x000a, 0x3bd4: 0x000a, 0x3bd5: 0x000a, 0x3bd6: 0x000a, 0x3bd7: 0x000a,
+	0x3bd8: 0x000a, 0x3bd9: 0x000a, 0x3bda: 0x000a, 0x3bdb: 0x000a, 0x3bdc: 0x000a, 0x3bdd: 0x000a,
+	0x3bde: 0x000a, 0x3bdf: 0x000a, 0x3be0: 0x000a, 0x3be1: 0x000a, 0x3be2: 0x000a, 0x3be3: 0x000a,
+	0x3be4: 0x000a, 0x3be5: 0x000a, 0x3be6: 0x000a, 0x3be7: 0x000a, 0x3be8: 0x000a, 0x3be9: 0x000a,
+	0x3bea: 0x000a, 0x3beb: 0x000a, 0x3bec: 0x000a, 0x3bed: 0x000a, 0x3bee: 0x000a, 0x3bef: 0x000a,
+	0x3bf0: 0x000a, 0x3bf1: 0x000a, 0x3bf2: 0x000a, 0x3bf3: 0x000a, 0x3bf4: 0x000a, 0x3bf5: 0x000a,
+	0x3bf6: 0x000a, 0x3bf7: 0x000a, 0x3bf8: 0x000a, 0x3bf9: 0x000a, 0x3bfa: 0x000a, 0x3bfb: 0x000a,
+	0x3bfc: 0x000a, 0x3bfd: 0x000a, 0x3bfe: 0x000a, 0x3bff: 0x000a,
+	// Block 0xf0, offset 0x3c00
+	0x3c00: 0x000a, 0x3c01: 0x000a, 0x3c02: 0x000a, 0x3c03: 0x000a, 0x3c04: 0x000a, 0x3c05: 0x000a,
+	0x3c06: 0x000a, 0x3c07: 0x000a, 0x3c08: 0x000a, 0x3c09: 0x000a, 0x3c0a: 0x000a, 0x3c0b: 0x000a,
+	0x3c0c: 0x000a, 0x3c0d: 0x000a, 0x3c0e: 0x000a, 0x3c0f: 0x000a, 0x3c10: 0x000a, 0x3c11: 0x000a,
+	0x3c12: 0x000a, 0x3c13: 0x000a,
+	0x3c20: 0x000a, 0x3c21: 0x000a, 0x3c22: 0x000a, 0x3c23: 0x000a,
+	0x3c24: 0x000a, 0x3c25: 0x000a, 0x3c26: 0x000a, 0x3c27: 0x000a, 0x3c28: 0x000a, 0x3c29: 0x000a,
+	0x3c2a: 0x000a, 0x3c2b: 0x000a, 0x3c2c: 0x000a, 0x3c2d: 0x000a,
+	0x3c30: 0x000a, 0x3c31: 0x000a, 0x3c32: 0x000a, 0x3c33: 0x000a, 0x3c34: 0x000a,
+	0x3c38: 0x000a, 0x3c39: 0x000a, 0x3c3a: 0x000a,
+	// Block 0xf1, offset 0x3c40
+	0x3c40: 0x000a, 0x3c41: 0x000a, 0x3c42: 0x000a, 0x3c43: 0x000a, 0x3c44: 0x000a, 0x3c45: 0x000a,
+	0x3c46: 0x000a,
+	0x3c50: 0x000a, 0x3c51: 0x000a,
+	0x3c52: 0x000a, 0x3c53: 0x000a, 0x3c54: 0x000a, 0x3c55: 0x000a, 0x3c56: 0x000a, 0x3c57: 0x000a,
+	0x3c58: 0x000a, 0x3c59: 0x000a, 0x3c5a: 0x000a, 0x3c5b: 0x000a, 0x3c5c: 0x000a, 0x3c5d: 0x000a,
+	0x3c5e: 0x000a, 0x3c5f: 0x000a, 0x3c60: 0x000a, 0x3c61: 0x000a, 0x3c62: 0x000a, 0x3c63: 0x000a,
+	0x3c64: 0x000a, 0x3c65: 0x000a, 0x3c66: 0x000a, 0x3c67: 0x000a, 0x3c68: 0x000a,
+	0x3c70: 0x000a, 0x3c71: 0x000a, 0x3c72: 0x000a, 0x3c73: 0x000a, 0x3c74: 0x000a, 0x3c75: 0x000a,
+	0x3c76: 0x000a,
+	// Block 0xf2, offset 0x3c80
+	0x3c80: 0x000a, 0x3c81: 0x000a, 0x3c82: 0x000a,
+	0x3c90: 0x000a, 0x3c91: 0x000a,
+	0x3c92: 0x000a, 0x3c93: 0x000a, 0x3c94: 0x000a, 0x3c95: 0x000a, 0x3c96: 0x000a,
+	// Block 0xf3, offset 0x3cc0
+	0x3cc0: 0x000a, 0x3cc1: 0x000a, 0x3cc2: 0x000a, 0x3cc3: 0x000a, 0x3cc4: 0x000a, 0x3cc5: 0x000a,
+	0x3cc6: 0x000a, 0x3cc7: 0x000a, 0x3cc8: 0x000a, 0x3cc9: 0x000a, 0x3cca: 0x000a, 0x3ccb: 0x000a,
+	0x3ccc: 0x000a, 0x3ccd: 0x000a, 0x3cce: 0x000a, 0x3ccf: 0x000a, 0x3cd0: 0x000a, 0x3cd1: 0x000a,
+	0x3cd2: 0x000a, 0x3cd4: 0x000a, 0x3cd5: 0x000a, 0x3cd6: 0x000a, 0x3cd7: 0x000a,
+	0x3cd8: 0x000a, 0x3cd9: 0x000a, 0x3cda: 0x000a, 0x3cdb: 0x000a, 0x3cdc: 0x000a, 0x3cdd: 0x000a,
+	0x3cde: 0x000a, 0x3cdf: 0x000a, 0x3ce0: 0x000a, 0x3ce1: 0x000a, 0x3ce2: 0x000a, 0x3ce3: 0x000a,
+	0x3ce4: 0x000a, 0x3ce5: 0x000a, 0x3ce6: 0x000a, 0x3ce7: 0x000a, 0x3ce8: 0x000a, 0x3ce9: 0x000a,
+	0x3cea: 0x000a, 0x3ceb: 0x000a, 0x3cec: 0x000a, 0x3ced: 0x000a, 0x3cee: 0x000a, 0x3cef: 0x000a,
+	0x3cf0: 0x000a, 0x3cf1: 0x000a, 0x3cf2: 0x000a, 0x3cf3: 0x000a, 0x3cf4: 0x000a, 0x3cf5: 0x000a,
+	0x3cf6: 0x000a, 0x3cf7: 0x000a, 0x3cf8: 0x000a, 0x3cf9: 0x000a, 0x3cfa: 0x000a, 0x3cfb: 0x000a,
+	0x3cfc: 0x000a, 0x3cfd: 0x000a, 0x3cfe: 0x000a, 0x3cff: 0x000a,
+	// Block 0xf4, offset 0x3d00
+	0x3d00: 0x000a, 0x3d01: 0x000a, 0x3d02: 0x000a, 0x3d03: 0x000a, 0x3d04: 0x000a, 0x3d05: 0x000a,
+	0x3d06: 0x000a, 0x3d07: 0x000a, 0x3d08: 0x000a, 0x3d09: 0x000a, 0x3d0a: 0x000a,
+	0x3d30: 0x0002, 0x3d31: 0x0002, 0x3d32: 0x0002, 0x3d33: 0x0002, 0x3d34: 0x0002, 0x3d35: 0x0002,
+	0x3d36: 0x0002, 0x3d37: 0x0002, 0x3d38: 0x0002, 0x3d39: 0x0002,
+	// Block 0xf5, offset 0x3d40
+	0x3d7e: 0x000b, 0x3d7f: 0x000b,
+	// Block 0xf6, offset 0x3d80
+	0x3d80: 0x000b, 0x3d81: 0x000b, 0x3d82: 0x000b, 0x3d83: 0x000b, 0x3d84: 0x000b, 0x3d85: 0x000b,
+	0x3d86: 0x000b, 0x3d87: 0x000b, 0x3d88: 0x000b, 0x3d89: 0x000b, 0x3d8a: 0x000b, 0x3d8b: 0x000b,
+	0x3d8c: 0x000b, 0x3d8d: 0x000b, 0x3d8e: 0x000b, 0x3d8f: 0x000b, 0x3d90: 0x000b, 0x3d91: 0x000b,
+	0x3d92: 0x000b, 0x3d93: 0x000b, 0x3d94: 0x000b, 0x3d95: 0x000b, 0x3d96: 0x000b, 0x3d97: 0x000b,
+	0x3d98: 0x000b, 0x3d99: 0x000b, 0x3d9a: 0x000b, 0x3d9b: 0x000b, 0x3d9c: 0x000b, 0x3d9d: 0x000b,
+	0x3d9e: 0x000b, 0x3d9f: 0x000b, 0x3da0: 0x000b, 0x3da1: 0x000b, 0x3da2: 0x000b, 0x3da3: 0x000b,
+	0x3da4: 0x000b, 0x3da5: 0x000b, 0x3da6: 0x000b, 0x3da7: 0x000b, 0x3da8: 0x000b, 0x3da9: 0x000b,
+	0x3daa: 0x000b, 0x3dab: 0x000b, 0x3dac: 0x000b, 0x3dad: 0x000b, 0x3dae: 0x000b, 0x3daf: 0x000b,
+	0x3db0: 0x000b, 0x3db1: 0x000b, 0x3db2: 0x000b, 0x3db3: 0x000b, 0x3db4: 0x000b, 0x3db5: 0x000b,
+	0x3db6: 0x000b, 0x3db7: 0x000b, 0x3db8: 0x000b, 0x3db9: 0x000b, 0x3dba: 0x000b, 0x3dbb: 0x000b,
+	0x3dbc: 0x000b, 0x3dbd: 0x000b, 0x3dbe: 0x000b, 0x3dbf: 0x000b,
+	// Block 0xf7, offset 0x3dc0
+	0x3dc0: 0x000c, 0x3dc1: 0x000c, 0x3dc2: 0x000c, 0x3dc3: 0x000c, 0x3dc4: 0x000c, 0x3dc5: 0x000c,
+	0x3dc6: 0x000c, 0x3dc7: 0x000c, 0x3dc8: 0x000c, 0x3dc9: 0x000c, 0x3dca: 0x000c, 0x3dcb: 0x000c,
+	0x3dcc: 0x000c, 0x3dcd: 0x000c, 0x3dce: 0x000c, 0x3dcf: 0x000c, 0x3dd0: 0x000c, 0x3dd1: 0x000c,
+	0x3dd2: 0x000c, 0x3dd3: 0x000c, 0x3dd4: 0x000c, 0x3dd5: 0x000c, 0x3dd6: 0x000c, 0x3dd7: 0x000c,
+	0x3dd8: 0x000c, 0x3dd9: 0x000c, 0x3dda: 0x000c, 0x3ddb: 0x000c, 0x3ddc: 0x000c, 0x3ddd: 0x000c,
+	0x3dde: 0x000c, 0x3ddf: 0x000c, 0x3de0: 0x000c, 0x3de1: 0x000c, 0x3de2: 0x000c, 0x3de3: 0x000c,
+	0x3de4: 0x000c, 0x3de5: 0x000c, 0x3de6: 0x000c, 0x3de7: 0x000c, 0x3de8: 0x000c, 0x3de9: 0x000c,
+	0x3dea: 0x000c, 0x3deb: 0x000c, 0x3dec: 0x000c, 0x3ded: 0x000c, 0x3dee: 0x000c, 0x3def: 0x000c,
+	0x3df0: 0x000b, 0x3df1: 0x000b, 0x3df2: 0x000b, 0x3df3: 0x000b, 0x3df4: 0x000b, 0x3df5: 0x000b,
+	0x3df6: 0x000b, 0x3df7: 0x000b, 0x3df8: 0x000b, 0x3df9: 0x000b, 0x3dfa: 0x000b, 0x3dfb: 0x000b,
+	0x3dfc: 0x000b, 0x3dfd: 0x000b, 0x3dfe: 0x000b, 0x3dff: 0x000b,
+}
+
+// bidiIndex: 24 blocks, 1536 entries, 1536 bytes
+// Block 0 is the zero block.
+var bidiIndex = [1536]uint8{
+	// Block 0x0, offset 0x0
+	// Block 0x1, offset 0x40
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc2: 0x01, 0xc3: 0x02,
+	0xca: 0x03, 0xcb: 0x04, 0xcc: 0x05, 0xcd: 0x06, 0xce: 0x07, 0xcf: 0x08,
+	0xd2: 0x09, 0xd6: 0x0a, 0xd7: 0x0b,
+	0xd8: 0x0c, 0xd9: 0x0d, 0xda: 0x0e, 0xdb: 0x0f, 0xdc: 0x10, 0xdd: 0x11, 0xde: 0x12, 0xdf: 0x13,
+	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05, 0xe4: 0x06,
+	0xea: 0x07, 0xef: 0x08,
+	0xf0: 0x11, 0xf1: 0x12, 0xf2: 0x12, 0xf3: 0x14, 0xf4: 0x15,
+	// Block 0x4, offset 0x100
+	0x120: 0x14, 0x121: 0x15, 0x122: 0x16, 0x123: 0x17, 0x124: 0x18, 0x125: 0x19, 0x126: 0x1a, 0x127: 0x1b,
+	0x128: 0x1c, 0x129: 0x1d, 0x12a: 0x1c, 0x12b: 0x1e, 0x12c: 0x1f, 0x12d: 0x20, 0x12e: 0x21, 0x12f: 0x22,
+	0x130: 0x23, 0x131: 0x24, 0x132: 0x1a, 0x133: 0x25, 0x134: 0x26, 0x135: 0x27, 0x136: 0x28, 0x137: 0x29,
+	0x138: 0x2a, 0x139: 0x2b, 0x13a: 0x2c, 0x13b: 0x2d, 0x13c: 0x2e, 0x13d: 0x2f, 0x13e: 0x30, 0x13f: 0x31,
+	// Block 0x5, offset 0x140
+	0x140: 0x32, 0x141: 0x33, 0x142: 0x34,
+	0x14d: 0x35, 0x14e: 0x36,
+	0x150: 0x37,
+	0x15a: 0x38, 0x15c: 0x39, 0x15d: 0x3a, 0x15e: 0x3b, 0x15f: 0x3c,
+	0x160: 0x3d, 0x162: 0x3e, 0x164: 0x3f, 0x165: 0x40, 0x167: 0x41,
+	0x168: 0x42, 0x169: 0x43, 0x16a: 0x44, 0x16b: 0x45, 0x16c: 0x46, 0x16d: 0x47, 0x16e: 0x48, 0x16f: 0x49,
+	0x170: 0x4a, 0x173: 0x4b, 0x177: 0x4c,
+	0x17e: 0x4d, 0x17f: 0x4e,
+	// Block 0x6, offset 0x180
+	0x180: 0x4f, 0x181: 0x50, 0x182: 0x51, 0x183: 0x52, 0x184: 0x53, 0x185: 0x54, 0x186: 0x55, 0x187: 0x56,
+	0x188: 0x57, 0x189: 0x56, 0x18a: 0x56, 0x18b: 0x56, 0x18c: 0x58, 0x18d: 0x59, 0x18e: 0x5a, 0x18f: 0x56,
+	0x190: 0x5b, 0x191: 0x5c, 0x192: 0x5d, 0x193: 0x5e, 0x194: 0x56, 0x195: 0x56, 0x196: 0x56, 0x197: 0x56,
+	0x198: 0x56, 0x199: 0x56, 0x19a: 0x5f, 0x19b: 0x56, 0x19c: 0x56, 0x19d: 0x60, 0x19e: 0x56, 0x19f: 0x61,
+	0x1a4: 0x56, 0x1a5: 0x56, 0x1a6: 0x62, 0x1a7: 0x63,
+	0x1a8: 0x56, 0x1a9: 0x56, 0x1aa: 0x56, 0x1ab: 0x56, 0x1ac: 0x56, 0x1ad: 0x64, 0x1ae: 0x65, 0x1af: 0x56,
+	0x1b3: 0x66, 0x1b5: 0x67, 0x1b7: 0x68,
+	0x1b8: 0x69, 0x1b9: 0x6a, 0x1ba: 0x6b, 0x1bb: 0x6c, 0x1bc: 0x56, 0x1bd: 0x56, 0x1be: 0x56, 0x1bf: 0x6d,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x6e, 0x1c2: 0x6f, 0x1c3: 0x70, 0x1c7: 0x71,
+	0x1c8: 0x72, 0x1c9: 0x73, 0x1ca: 0x74, 0x1cb: 0x75, 0x1cd: 0x76, 0x1cf: 0x77,
+	// Block 0x8, offset 0x200
+	0x237: 0x56,
+	// Block 0x9, offset 0x240
+	0x252: 0x78, 0x253: 0x79,
+	0x258: 0x7a, 0x259: 0x7b, 0x25a: 0x7c, 0x25b: 0x7d, 0x25c: 0x7e, 0x25e: 0x7f,
+	0x260: 0x80, 0x261: 0x81, 0x263: 0x82, 0x264: 0x83, 0x265: 0x84, 0x266: 0x85, 0x267: 0x86,
+	0x268: 0x87, 0x269: 0x88, 0x26a: 0x89, 0x26b: 0x8a, 0x26d: 0x8b, 0x26f: 0x8c,
+	// Block 0xa, offset 0x280
+	0x2ac: 0x8d, 0x2ad: 0x8e, 0x2ae: 0x0e, 0x2af: 0x0e,
+	0x2b0: 0x0e, 0x2b1: 0x0e, 0x2b2: 0x0e, 0x2b3: 0x0e, 0x2b4: 0x8f, 0x2b5: 0x0e, 0x2b6: 0x0e, 0x2b7: 0x90,
+	0x2b8: 0x91, 0x2b9: 0x92, 0x2ba: 0x0e, 0x2bb: 0x93, 0x2bc: 0x94, 0x2bd: 0x95, 0x2bf: 0x96,
+	// Block 0xb, offset 0x2c0
+	0x2c4: 0x97, 0x2c5: 0x56, 0x2c6: 0x98, 0x2c7: 0x99,
+	0x2cb: 0x9a, 0x2cd: 0x9b,
+	0x2e0: 0x9c, 0x2e1: 0x9c, 0x2e2: 0x9c, 0x2e3: 0x9c, 0x2e4: 0x9d, 0x2e5: 0x9c, 0x2e6: 0x9c, 0x2e7: 0x9c,
+	0x2e8: 0x9e, 0x2e9: 0x9c, 0x2ea: 0x9c, 0x2eb: 0x9f, 0x2ec: 0xa0, 0x2ed: 0x9c, 0x2ee: 0x9c, 0x2ef: 0x9c,
+	0x2f0: 0x9c, 0x2f1: 0x9c, 0x2f2: 0x9c, 0x2f3: 0x9c, 0x2f4: 0xa1, 0x2f5: 0x9c, 0x2f6: 0x9c, 0x2f7: 0x9c,
+	0x2f8: 0x9c, 0x2f9: 0xa2, 0x2fa: 0xa3, 0x2fb: 0x9c, 0x2fc: 0xa4, 0x2fd: 0xa5, 0x2fe: 0x9c, 0x2ff: 0x9c,
+	// Block 0xc, offset 0x300
+	0x300: 0xa6, 0x301: 0xa7, 0x302: 0xa8, 0x304: 0xa9, 0x305: 0xaa, 0x306: 0xab, 0x307: 0xac,
+	0x308: 0xad, 0x30b: 0xae, 0x30c: 0x26, 0x30d: 0xaf,
+	0x310: 0xb0, 0x311: 0xb1, 0x312: 0xb2, 0x313: 0xb3, 0x316: 0xb4, 0x317: 0xb5,
+	0x318: 0xb6, 0x319: 0xb7, 0x31a: 0xb8, 0x31c: 0xb9,
+	0x320: 0xba, 0x324: 0xbb, 0x325: 0xbc, 0x327: 0xbd,
+	0x328: 0xbe, 0x329: 0xbf, 0x32a: 0xc0,
+	0x330: 0xc1, 0x332: 0xc2, 0x334: 0xc3, 0x335: 0xc4, 0x336: 0xc5,
+	0x33b: 0xc6, 0x33f: 0xc7,
+	// Block 0xd, offset 0x340
+	0x36b: 0xc8, 0x36c: 0xc9,
+	0x37d: 0xca, 0x37e: 0xcb, 0x37f: 0xcc,
+	// Block 0xe, offset 0x380
+	0x3b2: 0xcd,
+	// Block 0xf, offset 0x3c0
+	0x3c5: 0xce, 0x3c6: 0xcf,
+	0x3c8: 0x56, 0x3c9: 0xd0, 0x3cc: 0x56, 0x3cd: 0xd1,
+	0x3db: 0xd2, 0x3dc: 0xd3, 0x3dd: 0xd4, 0x3de: 0xd5, 0x3df: 0xd6,
+	0x3e8: 0xd7, 0x3e9: 0xd8, 0x3ea: 0xd9,
+	// Block 0x10, offset 0x400
+	0x400: 0xda, 0x404: 0xc9,
+	0x40b: 0xdb,
+	0x420: 0x9c, 0x421: 0x9c, 0x422: 0x9c, 0x423: 0xdc, 0x424: 0x9c, 0x425: 0xdd, 0x426: 0x9c, 0x427: 0x9c,
+	0x428: 0x9c, 0x429: 0x9c, 0x42a: 0x9c, 0x42b: 0x9c, 0x42c: 0x9c, 0x42d: 0x9c, 0x42e: 0x9c, 0x42f: 0x9c,
+	0x430: 0x9c, 0x431: 0xa4, 0x432: 0x0e, 0x433: 0x9c, 0x434: 0x0e, 0x435: 0xde, 0x436: 0x9c, 0x437: 0x9c,
+	0x438: 0x0e, 0x439: 0x0e, 0x43a: 0x0e, 0x43b: 0xdf, 0x43c: 0x9c, 0x43d: 0x9c, 0x43e: 0x9c, 0x43f: 0x9c,
+	// Block 0x11, offset 0x440
+	0x440: 0xe0, 0x441: 0x56, 0x442: 0xe1, 0x443: 0xe2, 0x444: 0xe3, 0x445: 0xe4, 0x446: 0xe5,
+	0x449: 0xe6, 0x44c: 0x56, 0x44d: 0x56, 0x44e: 0x56, 0x44f: 0x56,
+	0x450: 0x56, 0x451: 0x56, 0x452: 0x56, 0x453: 0x56, 0x454: 0x56, 0x455: 0x56, 0x456: 0x56, 0x457: 0x56,
+	0x458: 0x56, 0x459: 0x56, 0x45a: 0x56, 0x45b: 0xe7, 0x45c: 0x56, 0x45d: 0x6c, 0x45e: 0x56, 0x45f: 0xe8,
+	0x460: 0xe9, 0x461: 0xea, 0x462: 0xeb, 0x464: 0x56, 0x465: 0xec, 0x466: 0x56, 0x467: 0xed,
+	0x468: 0x56, 0x469: 0xee, 0x46a: 0xef, 0x46b: 0xf0, 0x46c: 0x56, 0x46d: 0x56, 0x46e: 0xf1, 0x46f: 0xf2,
+	0x47f: 0xf3,
+	// Block 0x12, offset 0x480
+	0x4bf: 0xf3,
+	// Block 0x13, offset 0x4c0
+	0x4d0: 0x09, 0x4d1: 0x0a, 0x4d6: 0x0b,
+	0x4db: 0x0c, 0x4dd: 0x0d, 0x4de: 0x0e, 0x4df: 0x0f,
+	0x4ef: 0x10,
+	0x4ff: 0x10,
+	// Block 0x14, offset 0x500
+	0x50f: 0x10,
+	0x51f: 0x10,
+	0x52f: 0x10,
+	0x53f: 0x10,
+	// Block 0x15, offset 0x540
+	0x540: 0xf4, 0x541: 0xf4, 0x542: 0xf4, 0x543: 0xf4, 0x544: 0x05, 0x545: 0x05, 0x546: 0x05, 0x547: 0xf5,
+	0x548: 0xf4, 0x549: 0xf4, 0x54a: 0xf4, 0x54b: 0xf4, 0x54c: 0xf4, 0x54d: 0xf4, 0x54e: 0xf4, 0x54f: 0xf4,
+	0x550: 0xf4, 0x551: 0xf4, 0x552: 0xf4, 0x553: 0xf4, 0x554: 0xf4, 0x555: 0xf4, 0x556: 0xf4, 0x557: 0xf4,
+	0x558: 0xf4, 0x559: 0xf4, 0x55a: 0xf4, 0x55b: 0xf4, 0x55c: 0xf4, 0x55d: 0xf4, 0x55e: 0xf4, 0x55f: 0xf4,
+	0x560: 0xf4, 0x561: 0xf4, 0x562: 0xf4, 0x563: 0xf4, 0x564: 0xf4, 0x565: 0xf4, 0x566: 0xf4, 0x567: 0xf4,
+	0x568: 0xf4, 0x569: 0xf4, 0x56a: 0xf4, 0x56b: 0xf4, 0x56c: 0xf4, 0x56d: 0xf4, 0x56e: 0xf4, 0x56f: 0xf4,
+	0x570: 0xf4, 0x571: 0xf4, 0x572: 0xf4, 0x573: 0xf4, 0x574: 0xf4, 0x575: 0xf4, 0x576: 0xf4, 0x577: 0xf4,
+	0x578: 0xf4, 0x579: 0xf4, 0x57a: 0xf4, 0x57b: 0xf4, 0x57c: 0xf4, 0x57d: 0xf4, 0x57e: 0xf4, 0x57f: 0xf4,
+	// Block 0x16, offset 0x580
+	0x58f: 0x10,
+	0x59f: 0x10,
+	0x5a0: 0x13,
+	0x5af: 0x10,
+	0x5bf: 0x10,
+	// Block 0x17, offset 0x5c0
+	0x5cf: 0x10,
+}
+
+// Total table size 17464 bytes (17KiB); checksum: F50EF68C
diff --git a/vendor/golang.org/x/text/unicode/bidi/tables9.0.0.go b/vendor/golang.org/x/text/unicode/bidi/tables9.0.0.go
index 0ca0193..f517fdb 100644
--- a/vendor/golang.org/x/text/unicode/bidi/tables9.0.0.go
+++ b/vendor/golang.org/x/text/unicode/bidi/tables9.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build !go1.10
 // +build !go1.10
 
 package bidi
diff --git a/vendor/golang.org/x/text/unicode/norm/tables10.0.0.go b/vendor/golang.org/x/text/unicode/norm/tables10.0.0.go
index 26fbd55..f5a0788 100644
--- a/vendor/golang.org/x/text/unicode/norm/tables10.0.0.go
+++ b/vendor/golang.org/x/text/unicode/norm/tables10.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.10 && !go1.13
 // +build go1.10,!go1.13
 
 package norm
diff --git a/vendor/golang.org/x/text/unicode/norm/tables11.0.0.go b/vendor/golang.org/x/text/unicode/norm/tables11.0.0.go
index 2c58f09..cb7239c 100644
--- a/vendor/golang.org/x/text/unicode/norm/tables11.0.0.go
+++ b/vendor/golang.org/x/text/unicode/norm/tables11.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build go1.13 && !go1.14
 // +build go1.13,!go1.14
 
 package norm
diff --git a/vendor/golang.org/x/text/unicode/norm/tables12.0.0.go b/vendor/golang.org/x/text/unicode/norm/tables12.0.0.go
index 10f5202..11b2733 100644
--- a/vendor/golang.org/x/text/unicode/norm/tables12.0.0.go
+++ b/vendor/golang.org/x/text/unicode/norm/tables12.0.0.go
@@ -1,6 +1,7 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
-// +build go1.14
+//go:build go1.14 && !go1.16
+// +build go1.14,!go1.16
 
 package norm
 
diff --git a/vendor/golang.org/x/text/unicode/norm/tables13.0.0.go b/vendor/golang.org/x/text/unicode/norm/tables13.0.0.go
new file mode 100644
index 0000000..96a130d
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/norm/tables13.0.0.go
@@ -0,0 +1,7761 @@
+// Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
+
+//go:build go1.16
+// +build go1.16
+
+package norm
+
+import "sync"
+
+const (
+	// Version is the Unicode edition from which the tables are derived.
+	Version = "13.0.0"
+
+	// MaxTransformChunkSize indicates the maximum number of bytes that Transform
+	// may need to write atomically for any Form. Making a destination buffer at
+	// least this size ensures that Transform can always make progress and that
+	// the user does not need to grow the buffer on an ErrShortDst.
+	MaxTransformChunkSize = 35 + maxNonStarters*4
+)
+
+var ccc = [56]uint8{
+	0, 1, 6, 7, 8, 9, 10, 11,
+	12, 13, 14, 15, 16, 17, 18, 19,
+	20, 21, 22, 23, 24, 25, 26, 27,
+	28, 29, 30, 31, 32, 33, 34, 35,
+	36, 84, 91, 103, 107, 118, 122, 129,
+	130, 132, 202, 214, 216, 218, 220, 222,
+	224, 226, 228, 230, 232, 233, 234, 240,
+}
+
+const (
+	firstMulti            = 0x1870
+	firstCCC              = 0x2CAB
+	endMulti              = 0x2F77
+	firstLeadingCCC       = 0x49C5
+	firstCCCZeroExcept    = 0x4A8F
+	firstStarterWithNLead = 0x4AB6
+	lastDecomp            = 0x4AB8
+	maxDecomp             = 0x8000
+)
+
+// decomps: 19128 bytes
+var decomps = [...]byte{
+	// Bytes 0 - 3f
+	0x00, 0x41, 0x20, 0x41, 0x21, 0x41, 0x22, 0x41,
+	0x23, 0x41, 0x24, 0x41, 0x25, 0x41, 0x26, 0x41,
+	0x27, 0x41, 0x28, 0x41, 0x29, 0x41, 0x2A, 0x41,
+	0x2B, 0x41, 0x2C, 0x41, 0x2D, 0x41, 0x2E, 0x41,
+	0x2F, 0x41, 0x30, 0x41, 0x31, 0x41, 0x32, 0x41,
+	0x33, 0x41, 0x34, 0x41, 0x35, 0x41, 0x36, 0x41,
+	0x37, 0x41, 0x38, 0x41, 0x39, 0x41, 0x3A, 0x41,
+	0x3B, 0x41, 0x3C, 0x41, 0x3D, 0x41, 0x3E, 0x41,
+	// Bytes 40 - 7f
+	0x3F, 0x41, 0x40, 0x41, 0x41, 0x41, 0x42, 0x41,
+	0x43, 0x41, 0x44, 0x41, 0x45, 0x41, 0x46, 0x41,
+	0x47, 0x41, 0x48, 0x41, 0x49, 0x41, 0x4A, 0x41,
+	0x4B, 0x41, 0x4C, 0x41, 0x4D, 0x41, 0x4E, 0x41,
+	0x4F, 0x41, 0x50, 0x41, 0x51, 0x41, 0x52, 0x41,
+	0x53, 0x41, 0x54, 0x41, 0x55, 0x41, 0x56, 0x41,
+	0x57, 0x41, 0x58, 0x41, 0x59, 0x41, 0x5A, 0x41,
+	0x5B, 0x41, 0x5C, 0x41, 0x5D, 0x41, 0x5E, 0x41,
+	// Bytes 80 - bf
+	0x5F, 0x41, 0x60, 0x41, 0x61, 0x41, 0x62, 0x41,
+	0x63, 0x41, 0x64, 0x41, 0x65, 0x41, 0x66, 0x41,
+	0x67, 0x41, 0x68, 0x41, 0x69, 0x41, 0x6A, 0x41,
+	0x6B, 0x41, 0x6C, 0x41, 0x6D, 0x41, 0x6E, 0x41,
+	0x6F, 0x41, 0x70, 0x41, 0x71, 0x41, 0x72, 0x41,
+	0x73, 0x41, 0x74, 0x41, 0x75, 0x41, 0x76, 0x41,
+	0x77, 0x41, 0x78, 0x41, 0x79, 0x41, 0x7A, 0x41,
+	0x7B, 0x41, 0x7C, 0x41, 0x7D, 0x41, 0x7E, 0x42,
+	// Bytes c0 - ff
+	0xC2, 0xA2, 0x42, 0xC2, 0xA3, 0x42, 0xC2, 0xA5,
+	0x42, 0xC2, 0xA6, 0x42, 0xC2, 0xAC, 0x42, 0xC2,
+	0xB7, 0x42, 0xC3, 0x86, 0x42, 0xC3, 0xB0, 0x42,
+	0xC4, 0xA6, 0x42, 0xC4, 0xA7, 0x42, 0xC4, 0xB1,
+	0x42, 0xC5, 0x8B, 0x42, 0xC5, 0x93, 0x42, 0xC6,
+	0x8E, 0x42, 0xC6, 0x90, 0x42, 0xC6, 0xAB, 0x42,
+	0xC8, 0xA2, 0x42, 0xC8, 0xB7, 0x42, 0xC9, 0x90,
+	0x42, 0xC9, 0x91, 0x42, 0xC9, 0x92, 0x42, 0xC9,
+	// Bytes 100 - 13f
+	0x94, 0x42, 0xC9, 0x95, 0x42, 0xC9, 0x99, 0x42,
+	0xC9, 0x9B, 0x42, 0xC9, 0x9C, 0x42, 0xC9, 0x9F,
+	0x42, 0xC9, 0xA1, 0x42, 0xC9, 0xA3, 0x42, 0xC9,
+	0xA5, 0x42, 0xC9, 0xA6, 0x42, 0xC9, 0xA8, 0x42,
+	0xC9, 0xA9, 0x42, 0xC9, 0xAA, 0x42, 0xC9, 0xAB,
+	0x42, 0xC9, 0xAD, 0x42, 0xC9, 0xAF, 0x42, 0xC9,
+	0xB0, 0x42, 0xC9, 0xB1, 0x42, 0xC9, 0xB2, 0x42,
+	0xC9, 0xB3, 0x42, 0xC9, 0xB4, 0x42, 0xC9, 0xB5,
+	// Bytes 140 - 17f
+	0x42, 0xC9, 0xB8, 0x42, 0xC9, 0xB9, 0x42, 0xC9,
+	0xBB, 0x42, 0xCA, 0x81, 0x42, 0xCA, 0x82, 0x42,
+	0xCA, 0x83, 0x42, 0xCA, 0x89, 0x42, 0xCA, 0x8A,
+	0x42, 0xCA, 0x8B, 0x42, 0xCA, 0x8C, 0x42, 0xCA,
+	0x8D, 0x42, 0xCA, 0x90, 0x42, 0xCA, 0x91, 0x42,
+	0xCA, 0x92, 0x42, 0xCA, 0x95, 0x42, 0xCA, 0x9D,
+	0x42, 0xCA, 0x9F, 0x42, 0xCA, 0xB9, 0x42, 0xCE,
+	0x91, 0x42, 0xCE, 0x92, 0x42, 0xCE, 0x93, 0x42,
+	// Bytes 180 - 1bf
+	0xCE, 0x94, 0x42, 0xCE, 0x95, 0x42, 0xCE, 0x96,
+	0x42, 0xCE, 0x97, 0x42, 0xCE, 0x98, 0x42, 0xCE,
+	0x99, 0x42, 0xCE, 0x9A, 0x42, 0xCE, 0x9B, 0x42,
+	0xCE, 0x9C, 0x42, 0xCE, 0x9D, 0x42, 0xCE, 0x9E,
+	0x42, 0xCE, 0x9F, 0x42, 0xCE, 0xA0, 0x42, 0xCE,
+	0xA1, 0x42, 0xCE, 0xA3, 0x42, 0xCE, 0xA4, 0x42,
+	0xCE, 0xA5, 0x42, 0xCE, 0xA6, 0x42, 0xCE, 0xA7,
+	0x42, 0xCE, 0xA8, 0x42, 0xCE, 0xA9, 0x42, 0xCE,
+	// Bytes 1c0 - 1ff
+	0xB1, 0x42, 0xCE, 0xB2, 0x42, 0xCE, 0xB3, 0x42,
+	0xCE, 0xB4, 0x42, 0xCE, 0xB5, 0x42, 0xCE, 0xB6,
+	0x42, 0xCE, 0xB7, 0x42, 0xCE, 0xB8, 0x42, 0xCE,
+	0xB9, 0x42, 0xCE, 0xBA, 0x42, 0xCE, 0xBB, 0x42,
+	0xCE, 0xBC, 0x42, 0xCE, 0xBD, 0x42, 0xCE, 0xBE,
+	0x42, 0xCE, 0xBF, 0x42, 0xCF, 0x80, 0x42, 0xCF,
+	0x81, 0x42, 0xCF, 0x82, 0x42, 0xCF, 0x83, 0x42,
+	0xCF, 0x84, 0x42, 0xCF, 0x85, 0x42, 0xCF, 0x86,
+	// Bytes 200 - 23f
+	0x42, 0xCF, 0x87, 0x42, 0xCF, 0x88, 0x42, 0xCF,
+	0x89, 0x42, 0xCF, 0x9C, 0x42, 0xCF, 0x9D, 0x42,
+	0xD0, 0xBD, 0x42, 0xD1, 0x8A, 0x42, 0xD1, 0x8C,
+	0x42, 0xD7, 0x90, 0x42, 0xD7, 0x91, 0x42, 0xD7,
+	0x92, 0x42, 0xD7, 0x93, 0x42, 0xD7, 0x94, 0x42,
+	0xD7, 0x9B, 0x42, 0xD7, 0x9C, 0x42, 0xD7, 0x9D,
+	0x42, 0xD7, 0xA2, 0x42, 0xD7, 0xA8, 0x42, 0xD7,
+	0xAA, 0x42, 0xD8, 0xA1, 0x42, 0xD8, 0xA7, 0x42,
+	// Bytes 240 - 27f
+	0xD8, 0xA8, 0x42, 0xD8, 0xA9, 0x42, 0xD8, 0xAA,
+	0x42, 0xD8, 0xAB, 0x42, 0xD8, 0xAC, 0x42, 0xD8,
+	0xAD, 0x42, 0xD8, 0xAE, 0x42, 0xD8, 0xAF, 0x42,
+	0xD8, 0xB0, 0x42, 0xD8, 0xB1, 0x42, 0xD8, 0xB2,
+	0x42, 0xD8, 0xB3, 0x42, 0xD8, 0xB4, 0x42, 0xD8,
+	0xB5, 0x42, 0xD8, 0xB6, 0x42, 0xD8, 0xB7, 0x42,
+	0xD8, 0xB8, 0x42, 0xD8, 0xB9, 0x42, 0xD8, 0xBA,
+	0x42, 0xD9, 0x81, 0x42, 0xD9, 0x82, 0x42, 0xD9,
+	// Bytes 280 - 2bf
+	0x83, 0x42, 0xD9, 0x84, 0x42, 0xD9, 0x85, 0x42,
+	0xD9, 0x86, 0x42, 0xD9, 0x87, 0x42, 0xD9, 0x88,
+	0x42, 0xD9, 0x89, 0x42, 0xD9, 0x8A, 0x42, 0xD9,
+	0xAE, 0x42, 0xD9, 0xAF, 0x42, 0xD9, 0xB1, 0x42,
+	0xD9, 0xB9, 0x42, 0xD9, 0xBA, 0x42, 0xD9, 0xBB,
+	0x42, 0xD9, 0xBE, 0x42, 0xD9, 0xBF, 0x42, 0xDA,
+	0x80, 0x42, 0xDA, 0x83, 0x42, 0xDA, 0x84, 0x42,
+	0xDA, 0x86, 0x42, 0xDA, 0x87, 0x42, 0xDA, 0x88,
+	// Bytes 2c0 - 2ff
+	0x42, 0xDA, 0x8C, 0x42, 0xDA, 0x8D, 0x42, 0xDA,
+	0x8E, 0x42, 0xDA, 0x91, 0x42, 0xDA, 0x98, 0x42,
+	0xDA, 0xA1, 0x42, 0xDA, 0xA4, 0x42, 0xDA, 0xA6,
+	0x42, 0xDA, 0xA9, 0x42, 0xDA, 0xAD, 0x42, 0xDA,
+	0xAF, 0x42, 0xDA, 0xB1, 0x42, 0xDA, 0xB3, 0x42,
+	0xDA, 0xBA, 0x42, 0xDA, 0xBB, 0x42, 0xDA, 0xBE,
+	0x42, 0xDB, 0x81, 0x42, 0xDB, 0x85, 0x42, 0xDB,
+	0x86, 0x42, 0xDB, 0x87, 0x42, 0xDB, 0x88, 0x42,
+	// Bytes 300 - 33f
+	0xDB, 0x89, 0x42, 0xDB, 0x8B, 0x42, 0xDB, 0x8C,
+	0x42, 0xDB, 0x90, 0x42, 0xDB, 0x92, 0x43, 0xE0,
+	0xBC, 0x8B, 0x43, 0xE1, 0x83, 0x9C, 0x43, 0xE1,
+	0x84, 0x80, 0x43, 0xE1, 0x84, 0x81, 0x43, 0xE1,
+	0x84, 0x82, 0x43, 0xE1, 0x84, 0x83, 0x43, 0xE1,
+	0x84, 0x84, 0x43, 0xE1, 0x84, 0x85, 0x43, 0xE1,
+	0x84, 0x86, 0x43, 0xE1, 0x84, 0x87, 0x43, 0xE1,
+	0x84, 0x88, 0x43, 0xE1, 0x84, 0x89, 0x43, 0xE1,
+	// Bytes 340 - 37f
+	0x84, 0x8A, 0x43, 0xE1, 0x84, 0x8B, 0x43, 0xE1,
+	0x84, 0x8C, 0x43, 0xE1, 0x84, 0x8D, 0x43, 0xE1,
+	0x84, 0x8E, 0x43, 0xE1, 0x84, 0x8F, 0x43, 0xE1,
+	0x84, 0x90, 0x43, 0xE1, 0x84, 0x91, 0x43, 0xE1,
+	0x84, 0x92, 0x43, 0xE1, 0x84, 0x94, 0x43, 0xE1,
+	0x84, 0x95, 0x43, 0xE1, 0x84, 0x9A, 0x43, 0xE1,
+	0x84, 0x9C, 0x43, 0xE1, 0x84, 0x9D, 0x43, 0xE1,
+	0x84, 0x9E, 0x43, 0xE1, 0x84, 0xA0, 0x43, 0xE1,
+	// Bytes 380 - 3bf
+	0x84, 0xA1, 0x43, 0xE1, 0x84, 0xA2, 0x43, 0xE1,
+	0x84, 0xA3, 0x43, 0xE1, 0x84, 0xA7, 0x43, 0xE1,
+	0x84, 0xA9, 0x43, 0xE1, 0x84, 0xAB, 0x43, 0xE1,
+	0x84, 0xAC, 0x43, 0xE1, 0x84, 0xAD, 0x43, 0xE1,
+	0x84, 0xAE, 0x43, 0xE1, 0x84, 0xAF, 0x43, 0xE1,
+	0x84, 0xB2, 0x43, 0xE1, 0x84, 0xB6, 0x43, 0xE1,
+	0x85, 0x80, 0x43, 0xE1, 0x85, 0x87, 0x43, 0xE1,
+	0x85, 0x8C, 0x43, 0xE1, 0x85, 0x97, 0x43, 0xE1,
+	// Bytes 3c0 - 3ff
+	0x85, 0x98, 0x43, 0xE1, 0x85, 0x99, 0x43, 0xE1,
+	0x85, 0xA0, 0x43, 0xE1, 0x86, 0x84, 0x43, 0xE1,
+	0x86, 0x85, 0x43, 0xE1, 0x86, 0x88, 0x43, 0xE1,
+	0x86, 0x91, 0x43, 0xE1, 0x86, 0x92, 0x43, 0xE1,
+	0x86, 0x94, 0x43, 0xE1, 0x86, 0x9E, 0x43, 0xE1,
+	0x86, 0xA1, 0x43, 0xE1, 0x87, 0x87, 0x43, 0xE1,
+	0x87, 0x88, 0x43, 0xE1, 0x87, 0x8C, 0x43, 0xE1,
+	0x87, 0x8E, 0x43, 0xE1, 0x87, 0x93, 0x43, 0xE1,
+	// Bytes 400 - 43f
+	0x87, 0x97, 0x43, 0xE1, 0x87, 0x99, 0x43, 0xE1,
+	0x87, 0x9D, 0x43, 0xE1, 0x87, 0x9F, 0x43, 0xE1,
+	0x87, 0xB1, 0x43, 0xE1, 0x87, 0xB2, 0x43, 0xE1,
+	0xB4, 0x82, 0x43, 0xE1, 0xB4, 0x96, 0x43, 0xE1,
+	0xB4, 0x97, 0x43, 0xE1, 0xB4, 0x9C, 0x43, 0xE1,
+	0xB4, 0x9D, 0x43, 0xE1, 0xB4, 0xA5, 0x43, 0xE1,
+	0xB5, 0xBB, 0x43, 0xE1, 0xB6, 0x85, 0x43, 0xE2,
+	0x80, 0x82, 0x43, 0xE2, 0x80, 0x83, 0x43, 0xE2,
+	// Bytes 440 - 47f
+	0x80, 0x90, 0x43, 0xE2, 0x80, 0x93, 0x43, 0xE2,
+	0x80, 0x94, 0x43, 0xE2, 0x82, 0xA9, 0x43, 0xE2,
+	0x86, 0x90, 0x43, 0xE2, 0x86, 0x91, 0x43, 0xE2,
+	0x86, 0x92, 0x43, 0xE2, 0x86, 0x93, 0x43, 0xE2,
+	0x88, 0x82, 0x43, 0xE2, 0x88, 0x87, 0x43, 0xE2,
+	0x88, 0x91, 0x43, 0xE2, 0x88, 0x92, 0x43, 0xE2,
+	0x94, 0x82, 0x43, 0xE2, 0x96, 0xA0, 0x43, 0xE2,
+	0x97, 0x8B, 0x43, 0xE2, 0xA6, 0x85, 0x43, 0xE2,
+	// Bytes 480 - 4bf
+	0xA6, 0x86, 0x43, 0xE2, 0xB5, 0xA1, 0x43, 0xE3,
+	0x80, 0x81, 0x43, 0xE3, 0x80, 0x82, 0x43, 0xE3,
+	0x80, 0x88, 0x43, 0xE3, 0x80, 0x89, 0x43, 0xE3,
+	0x80, 0x8A, 0x43, 0xE3, 0x80, 0x8B, 0x43, 0xE3,
+	0x80, 0x8C, 0x43, 0xE3, 0x80, 0x8D, 0x43, 0xE3,
+	0x80, 0x8E, 0x43, 0xE3, 0x80, 0x8F, 0x43, 0xE3,
+	0x80, 0x90, 0x43, 0xE3, 0x80, 0x91, 0x43, 0xE3,
+	0x80, 0x92, 0x43, 0xE3, 0x80, 0x94, 0x43, 0xE3,
+	// Bytes 4c0 - 4ff
+	0x80, 0x95, 0x43, 0xE3, 0x80, 0x96, 0x43, 0xE3,
+	0x80, 0x97, 0x43, 0xE3, 0x82, 0xA1, 0x43, 0xE3,
+	0x82, 0xA2, 0x43, 0xE3, 0x82, 0xA3, 0x43, 0xE3,
+	0x82, 0xA4, 0x43, 0xE3, 0x82, 0xA5, 0x43, 0xE3,
+	0x82, 0xA6, 0x43, 0xE3, 0x82, 0xA7, 0x43, 0xE3,
+	0x82, 0xA8, 0x43, 0xE3, 0x82, 0xA9, 0x43, 0xE3,
+	0x82, 0xAA, 0x43, 0xE3, 0x82, 0xAB, 0x43, 0xE3,
+	0x82, 0xAD, 0x43, 0xE3, 0x82, 0xAF, 0x43, 0xE3,
+	// Bytes 500 - 53f
+	0x82, 0xB1, 0x43, 0xE3, 0x82, 0xB3, 0x43, 0xE3,
+	0x82, 0xB5, 0x43, 0xE3, 0x82, 0xB7, 0x43, 0xE3,
+	0x82, 0xB9, 0x43, 0xE3, 0x82, 0xBB, 0x43, 0xE3,
+	0x82, 0xBD, 0x43, 0xE3, 0x82, 0xBF, 0x43, 0xE3,
+	0x83, 0x81, 0x43, 0xE3, 0x83, 0x83, 0x43, 0xE3,
+	0x83, 0x84, 0x43, 0xE3, 0x83, 0x86, 0x43, 0xE3,
+	0x83, 0x88, 0x43, 0xE3, 0x83, 0x8A, 0x43, 0xE3,
+	0x83, 0x8B, 0x43, 0xE3, 0x83, 0x8C, 0x43, 0xE3,
+	// Bytes 540 - 57f
+	0x83, 0x8D, 0x43, 0xE3, 0x83, 0x8E, 0x43, 0xE3,
+	0x83, 0x8F, 0x43, 0xE3, 0x83, 0x92, 0x43, 0xE3,
+	0x83, 0x95, 0x43, 0xE3, 0x83, 0x98, 0x43, 0xE3,
+	0x83, 0x9B, 0x43, 0xE3, 0x83, 0x9E, 0x43, 0xE3,
+	0x83, 0x9F, 0x43, 0xE3, 0x83, 0xA0, 0x43, 0xE3,
+	0x83, 0xA1, 0x43, 0xE3, 0x83, 0xA2, 0x43, 0xE3,
+	0x83, 0xA3, 0x43, 0xE3, 0x83, 0xA4, 0x43, 0xE3,
+	0x83, 0xA5, 0x43, 0xE3, 0x83, 0xA6, 0x43, 0xE3,
+	// Bytes 580 - 5bf
+	0x83, 0xA7, 0x43, 0xE3, 0x83, 0xA8, 0x43, 0xE3,
+	0x83, 0xA9, 0x43, 0xE3, 0x83, 0xAA, 0x43, 0xE3,
+	0x83, 0xAB, 0x43, 0xE3, 0x83, 0xAC, 0x43, 0xE3,
+	0x83, 0xAD, 0x43, 0xE3, 0x83, 0xAF, 0x43, 0xE3,
+	0x83, 0xB0, 0x43, 0xE3, 0x83, 0xB1, 0x43, 0xE3,
+	0x83, 0xB2, 0x43, 0xE3, 0x83, 0xB3, 0x43, 0xE3,
+	0x83, 0xBB, 0x43, 0xE3, 0x83, 0xBC, 0x43, 0xE3,
+	0x92, 0x9E, 0x43, 0xE3, 0x92, 0xB9, 0x43, 0xE3,
+	// Bytes 5c0 - 5ff
+	0x92, 0xBB, 0x43, 0xE3, 0x93, 0x9F, 0x43, 0xE3,
+	0x94, 0x95, 0x43, 0xE3, 0x9B, 0xAE, 0x43, 0xE3,
+	0x9B, 0xBC, 0x43, 0xE3, 0x9E, 0x81, 0x43, 0xE3,
+	0xA0, 0xAF, 0x43, 0xE3, 0xA1, 0xA2, 0x43, 0xE3,
+	0xA1, 0xBC, 0x43, 0xE3, 0xA3, 0x87, 0x43, 0xE3,
+	0xA3, 0xA3, 0x43, 0xE3, 0xA4, 0x9C, 0x43, 0xE3,
+	0xA4, 0xBA, 0x43, 0xE3, 0xA8, 0xAE, 0x43, 0xE3,
+	0xA9, 0xAC, 0x43, 0xE3, 0xAB, 0xA4, 0x43, 0xE3,
+	// Bytes 600 - 63f
+	0xAC, 0x88, 0x43, 0xE3, 0xAC, 0x99, 0x43, 0xE3,
+	0xAD, 0x89, 0x43, 0xE3, 0xAE, 0x9D, 0x43, 0xE3,
+	0xB0, 0x98, 0x43, 0xE3, 0xB1, 0x8E, 0x43, 0xE3,
+	0xB4, 0xB3, 0x43, 0xE3, 0xB6, 0x96, 0x43, 0xE3,
+	0xBA, 0xAC, 0x43, 0xE3, 0xBA, 0xB8, 0x43, 0xE3,
+	0xBC, 0x9B, 0x43, 0xE3, 0xBF, 0xBC, 0x43, 0xE4,
+	0x80, 0x88, 0x43, 0xE4, 0x80, 0x98, 0x43, 0xE4,
+	0x80, 0xB9, 0x43, 0xE4, 0x81, 0x86, 0x43, 0xE4,
+	// Bytes 640 - 67f
+	0x82, 0x96, 0x43, 0xE4, 0x83, 0xA3, 0x43, 0xE4,
+	0x84, 0xAF, 0x43, 0xE4, 0x88, 0x82, 0x43, 0xE4,
+	0x88, 0xA7, 0x43, 0xE4, 0x8A, 0xA0, 0x43, 0xE4,
+	0x8C, 0x81, 0x43, 0xE4, 0x8C, 0xB4, 0x43, 0xE4,
+	0x8D, 0x99, 0x43, 0xE4, 0x8F, 0x95, 0x43, 0xE4,
+	0x8F, 0x99, 0x43, 0xE4, 0x90, 0x8B, 0x43, 0xE4,
+	0x91, 0xAB, 0x43, 0xE4, 0x94, 0xAB, 0x43, 0xE4,
+	0x95, 0x9D, 0x43, 0xE4, 0x95, 0xA1, 0x43, 0xE4,
+	// Bytes 680 - 6bf
+	0x95, 0xAB, 0x43, 0xE4, 0x97, 0x97, 0x43, 0xE4,
+	0x97, 0xB9, 0x43, 0xE4, 0x98, 0xB5, 0x43, 0xE4,
+	0x9A, 0xBE, 0x43, 0xE4, 0x9B, 0x87, 0x43, 0xE4,
+	0xA6, 0x95, 0x43, 0xE4, 0xA7, 0xA6, 0x43, 0xE4,
+	0xA9, 0xAE, 0x43, 0xE4, 0xA9, 0xB6, 0x43, 0xE4,
+	0xAA, 0xB2, 0x43, 0xE4, 0xAC, 0xB3, 0x43, 0xE4,
+	0xAF, 0x8E, 0x43, 0xE4, 0xB3, 0x8E, 0x43, 0xE4,
+	0xB3, 0xAD, 0x43, 0xE4, 0xB3, 0xB8, 0x43, 0xE4,
+	// Bytes 6c0 - 6ff
+	0xB5, 0x96, 0x43, 0xE4, 0xB8, 0x80, 0x43, 0xE4,
+	0xB8, 0x81, 0x43, 0xE4, 0xB8, 0x83, 0x43, 0xE4,
+	0xB8, 0x89, 0x43, 0xE4, 0xB8, 0x8A, 0x43, 0xE4,
+	0xB8, 0x8B, 0x43, 0xE4, 0xB8, 0x8D, 0x43, 0xE4,
+	0xB8, 0x99, 0x43, 0xE4, 0xB8, 0xA6, 0x43, 0xE4,
+	0xB8, 0xA8, 0x43, 0xE4, 0xB8, 0xAD, 0x43, 0xE4,
+	0xB8, 0xB2, 0x43, 0xE4, 0xB8, 0xB6, 0x43, 0xE4,
+	0xB8, 0xB8, 0x43, 0xE4, 0xB8, 0xB9, 0x43, 0xE4,
+	// Bytes 700 - 73f
+	0xB8, 0xBD, 0x43, 0xE4, 0xB8, 0xBF, 0x43, 0xE4,
+	0xB9, 0x81, 0x43, 0xE4, 0xB9, 0x99, 0x43, 0xE4,
+	0xB9, 0x9D, 0x43, 0xE4, 0xBA, 0x82, 0x43, 0xE4,
+	0xBA, 0x85, 0x43, 0xE4, 0xBA, 0x86, 0x43, 0xE4,
+	0xBA, 0x8C, 0x43, 0xE4, 0xBA, 0x94, 0x43, 0xE4,
+	0xBA, 0xA0, 0x43, 0xE4, 0xBA, 0xA4, 0x43, 0xE4,
+	0xBA, 0xAE, 0x43, 0xE4, 0xBA, 0xBA, 0x43, 0xE4,
+	0xBB, 0x80, 0x43, 0xE4, 0xBB, 0x8C, 0x43, 0xE4,
+	// Bytes 740 - 77f
+	0xBB, 0xA4, 0x43, 0xE4, 0xBC, 0x81, 0x43, 0xE4,
+	0xBC, 0x91, 0x43, 0xE4, 0xBD, 0xA0, 0x43, 0xE4,
+	0xBE, 0x80, 0x43, 0xE4, 0xBE, 0x86, 0x43, 0xE4,
+	0xBE, 0x8B, 0x43, 0xE4, 0xBE, 0xAE, 0x43, 0xE4,
+	0xBE, 0xBB, 0x43, 0xE4, 0xBE, 0xBF, 0x43, 0xE5,
+	0x80, 0x82, 0x43, 0xE5, 0x80, 0xAB, 0x43, 0xE5,
+	0x81, 0xBA, 0x43, 0xE5, 0x82, 0x99, 0x43, 0xE5,
+	0x83, 0x8F, 0x43, 0xE5, 0x83, 0x9A, 0x43, 0xE5,
+	// Bytes 780 - 7bf
+	0x83, 0xA7, 0x43, 0xE5, 0x84, 0xAA, 0x43, 0xE5,
+	0x84, 0xBF, 0x43, 0xE5, 0x85, 0x80, 0x43, 0xE5,
+	0x85, 0x85, 0x43, 0xE5, 0x85, 0x8D, 0x43, 0xE5,
+	0x85, 0x94, 0x43, 0xE5, 0x85, 0xA4, 0x43, 0xE5,
+	0x85, 0xA5, 0x43, 0xE5, 0x85, 0xA7, 0x43, 0xE5,
+	0x85, 0xA8, 0x43, 0xE5, 0x85, 0xA9, 0x43, 0xE5,
+	0x85, 0xAB, 0x43, 0xE5, 0x85, 0xAD, 0x43, 0xE5,
+	0x85, 0xB7, 0x43, 0xE5, 0x86, 0x80, 0x43, 0xE5,
+	// Bytes 7c0 - 7ff
+	0x86, 0x82, 0x43, 0xE5, 0x86, 0x8D, 0x43, 0xE5,
+	0x86, 0x92, 0x43, 0xE5, 0x86, 0x95, 0x43, 0xE5,
+	0x86, 0x96, 0x43, 0xE5, 0x86, 0x97, 0x43, 0xE5,
+	0x86, 0x99, 0x43, 0xE5, 0x86, 0xA4, 0x43, 0xE5,
+	0x86, 0xAB, 0x43, 0xE5, 0x86, 0xAC, 0x43, 0xE5,
+	0x86, 0xB5, 0x43, 0xE5, 0x86, 0xB7, 0x43, 0xE5,
+	0x87, 0x89, 0x43, 0xE5, 0x87, 0x8C, 0x43, 0xE5,
+	0x87, 0x9C, 0x43, 0xE5, 0x87, 0x9E, 0x43, 0xE5,
+	// Bytes 800 - 83f
+	0x87, 0xA0, 0x43, 0xE5, 0x87, 0xB5, 0x43, 0xE5,
+	0x88, 0x80, 0x43, 0xE5, 0x88, 0x83, 0x43, 0xE5,
+	0x88, 0x87, 0x43, 0xE5, 0x88, 0x97, 0x43, 0xE5,
+	0x88, 0x9D, 0x43, 0xE5, 0x88, 0xA9, 0x43, 0xE5,
+	0x88, 0xBA, 0x43, 0xE5, 0x88, 0xBB, 0x43, 0xE5,
+	0x89, 0x86, 0x43, 0xE5, 0x89, 0x8D, 0x43, 0xE5,
+	0x89, 0xB2, 0x43, 0xE5, 0x89, 0xB7, 0x43, 0xE5,
+	0x8A, 0x89, 0x43, 0xE5, 0x8A, 0x9B, 0x43, 0xE5,
+	// Bytes 840 - 87f
+	0x8A, 0xA3, 0x43, 0xE5, 0x8A, 0xB3, 0x43, 0xE5,
+	0x8A, 0xB4, 0x43, 0xE5, 0x8B, 0x87, 0x43, 0xE5,
+	0x8B, 0x89, 0x43, 0xE5, 0x8B, 0x92, 0x43, 0xE5,
+	0x8B, 0x9E, 0x43, 0xE5, 0x8B, 0xA4, 0x43, 0xE5,
+	0x8B, 0xB5, 0x43, 0xE5, 0x8B, 0xB9, 0x43, 0xE5,
+	0x8B, 0xBA, 0x43, 0xE5, 0x8C, 0x85, 0x43, 0xE5,
+	0x8C, 0x86, 0x43, 0xE5, 0x8C, 0x95, 0x43, 0xE5,
+	0x8C, 0x97, 0x43, 0xE5, 0x8C, 0x9A, 0x43, 0xE5,
+	// Bytes 880 - 8bf
+	0x8C, 0xB8, 0x43, 0xE5, 0x8C, 0xBB, 0x43, 0xE5,
+	0x8C, 0xBF, 0x43, 0xE5, 0x8D, 0x81, 0x43, 0xE5,
+	0x8D, 0x84, 0x43, 0xE5, 0x8D, 0x85, 0x43, 0xE5,
+	0x8D, 0x89, 0x43, 0xE5, 0x8D, 0x91, 0x43, 0xE5,
+	0x8D, 0x94, 0x43, 0xE5, 0x8D, 0x9A, 0x43, 0xE5,
+	0x8D, 0x9C, 0x43, 0xE5, 0x8D, 0xA9, 0x43, 0xE5,
+	0x8D, 0xB0, 0x43, 0xE5, 0x8D, 0xB3, 0x43, 0xE5,
+	0x8D, 0xB5, 0x43, 0xE5, 0x8D, 0xBD, 0x43, 0xE5,
+	// Bytes 8c0 - 8ff
+	0x8D, 0xBF, 0x43, 0xE5, 0x8E, 0x82, 0x43, 0xE5,
+	0x8E, 0xB6, 0x43, 0xE5, 0x8F, 0x83, 0x43, 0xE5,
+	0x8F, 0x88, 0x43, 0xE5, 0x8F, 0x8A, 0x43, 0xE5,
+	0x8F, 0x8C, 0x43, 0xE5, 0x8F, 0x9F, 0x43, 0xE5,
+	0x8F, 0xA3, 0x43, 0xE5, 0x8F, 0xA5, 0x43, 0xE5,
+	0x8F, 0xAB, 0x43, 0xE5, 0x8F, 0xAF, 0x43, 0xE5,
+	0x8F, 0xB1, 0x43, 0xE5, 0x8F, 0xB3, 0x43, 0xE5,
+	0x90, 0x86, 0x43, 0xE5, 0x90, 0x88, 0x43, 0xE5,
+	// Bytes 900 - 93f
+	0x90, 0x8D, 0x43, 0xE5, 0x90, 0x8F, 0x43, 0xE5,
+	0x90, 0x9D, 0x43, 0xE5, 0x90, 0xB8, 0x43, 0xE5,
+	0x90, 0xB9, 0x43, 0xE5, 0x91, 0x82, 0x43, 0xE5,
+	0x91, 0x88, 0x43, 0xE5, 0x91, 0xA8, 0x43, 0xE5,
+	0x92, 0x9E, 0x43, 0xE5, 0x92, 0xA2, 0x43, 0xE5,
+	0x92, 0xBD, 0x43, 0xE5, 0x93, 0xB6, 0x43, 0xE5,
+	0x94, 0x90, 0x43, 0xE5, 0x95, 0x8F, 0x43, 0xE5,
+	0x95, 0x93, 0x43, 0xE5, 0x95, 0x95, 0x43, 0xE5,
+	// Bytes 940 - 97f
+	0x95, 0xA3, 0x43, 0xE5, 0x96, 0x84, 0x43, 0xE5,
+	0x96, 0x87, 0x43, 0xE5, 0x96, 0x99, 0x43, 0xE5,
+	0x96, 0x9D, 0x43, 0xE5, 0x96, 0xAB, 0x43, 0xE5,
+	0x96, 0xB3, 0x43, 0xE5, 0x96, 0xB6, 0x43, 0xE5,
+	0x97, 0x80, 0x43, 0xE5, 0x97, 0x82, 0x43, 0xE5,
+	0x97, 0xA2, 0x43, 0xE5, 0x98, 0x86, 0x43, 0xE5,
+	0x99, 0x91, 0x43, 0xE5, 0x99, 0xA8, 0x43, 0xE5,
+	0x99, 0xB4, 0x43, 0xE5, 0x9B, 0x97, 0x43, 0xE5,
+	// Bytes 980 - 9bf
+	0x9B, 0x9B, 0x43, 0xE5, 0x9B, 0xB9, 0x43, 0xE5,
+	0x9C, 0x96, 0x43, 0xE5, 0x9C, 0x97, 0x43, 0xE5,
+	0x9C, 0x9F, 0x43, 0xE5, 0x9C, 0xB0, 0x43, 0xE5,
+	0x9E, 0x8B, 0x43, 0xE5, 0x9F, 0x8E, 0x43, 0xE5,
+	0x9F, 0xB4, 0x43, 0xE5, 0xA0, 0x8D, 0x43, 0xE5,
+	0xA0, 0xB1, 0x43, 0xE5, 0xA0, 0xB2, 0x43, 0xE5,
+	0xA1, 0x80, 0x43, 0xE5, 0xA1, 0x9A, 0x43, 0xE5,
+	0xA1, 0x9E, 0x43, 0xE5, 0xA2, 0xA8, 0x43, 0xE5,
+	// Bytes 9c0 - 9ff
+	0xA2, 0xAC, 0x43, 0xE5, 0xA2, 0xB3, 0x43, 0xE5,
+	0xA3, 0x98, 0x43, 0xE5, 0xA3, 0x9F, 0x43, 0xE5,
+	0xA3, 0xAB, 0x43, 0xE5, 0xA3, 0xAE, 0x43, 0xE5,
+	0xA3, 0xB0, 0x43, 0xE5, 0xA3, 0xB2, 0x43, 0xE5,
+	0xA3, 0xB7, 0x43, 0xE5, 0xA4, 0x82, 0x43, 0xE5,
+	0xA4, 0x86, 0x43, 0xE5, 0xA4, 0x8A, 0x43, 0xE5,
+	0xA4, 0x95, 0x43, 0xE5, 0xA4, 0x9A, 0x43, 0xE5,
+	0xA4, 0x9C, 0x43, 0xE5, 0xA4, 0xA2, 0x43, 0xE5,
+	// Bytes a00 - a3f
+	0xA4, 0xA7, 0x43, 0xE5, 0xA4, 0xA9, 0x43, 0xE5,
+	0xA5, 0x84, 0x43, 0xE5, 0xA5, 0x88, 0x43, 0xE5,
+	0xA5, 0x91, 0x43, 0xE5, 0xA5, 0x94, 0x43, 0xE5,
+	0xA5, 0xA2, 0x43, 0xE5, 0xA5, 0xB3, 0x43, 0xE5,
+	0xA7, 0x98, 0x43, 0xE5, 0xA7, 0xAC, 0x43, 0xE5,
+	0xA8, 0x9B, 0x43, 0xE5, 0xA8, 0xA7, 0x43, 0xE5,
+	0xA9, 0xA2, 0x43, 0xE5, 0xA9, 0xA6, 0x43, 0xE5,
+	0xAA, 0xB5, 0x43, 0xE5, 0xAC, 0x88, 0x43, 0xE5,
+	// Bytes a40 - a7f
+	0xAC, 0xA8, 0x43, 0xE5, 0xAC, 0xBE, 0x43, 0xE5,
+	0xAD, 0x90, 0x43, 0xE5, 0xAD, 0x97, 0x43, 0xE5,
+	0xAD, 0xA6, 0x43, 0xE5, 0xAE, 0x80, 0x43, 0xE5,
+	0xAE, 0x85, 0x43, 0xE5, 0xAE, 0x97, 0x43, 0xE5,
+	0xAF, 0x83, 0x43, 0xE5, 0xAF, 0x98, 0x43, 0xE5,
+	0xAF, 0xA7, 0x43, 0xE5, 0xAF, 0xAE, 0x43, 0xE5,
+	0xAF, 0xB3, 0x43, 0xE5, 0xAF, 0xB8, 0x43, 0xE5,
+	0xAF, 0xBF, 0x43, 0xE5, 0xB0, 0x86, 0x43, 0xE5,
+	// Bytes a80 - abf
+	0xB0, 0x8F, 0x43, 0xE5, 0xB0, 0xA2, 0x43, 0xE5,
+	0xB0, 0xB8, 0x43, 0xE5, 0xB0, 0xBF, 0x43, 0xE5,
+	0xB1, 0xA0, 0x43, 0xE5, 0xB1, 0xA2, 0x43, 0xE5,
+	0xB1, 0xA4, 0x43, 0xE5, 0xB1, 0xA5, 0x43, 0xE5,
+	0xB1, 0xAE, 0x43, 0xE5, 0xB1, 0xB1, 0x43, 0xE5,
+	0xB2, 0x8D, 0x43, 0xE5, 0xB3, 0x80, 0x43, 0xE5,
+	0xB4, 0x99, 0x43, 0xE5, 0xB5, 0x83, 0x43, 0xE5,
+	0xB5, 0x90, 0x43, 0xE5, 0xB5, 0xAB, 0x43, 0xE5,
+	// Bytes ac0 - aff
+	0xB5, 0xAE, 0x43, 0xE5, 0xB5, 0xBC, 0x43, 0xE5,
+	0xB6, 0xB2, 0x43, 0xE5, 0xB6, 0xBA, 0x43, 0xE5,
+	0xB7, 0x9B, 0x43, 0xE5, 0xB7, 0xA1, 0x43, 0xE5,
+	0xB7, 0xA2, 0x43, 0xE5, 0xB7, 0xA5, 0x43, 0xE5,
+	0xB7, 0xA6, 0x43, 0xE5, 0xB7, 0xB1, 0x43, 0xE5,
+	0xB7, 0xBD, 0x43, 0xE5, 0xB7, 0xBE, 0x43, 0xE5,
+	0xB8, 0xA8, 0x43, 0xE5, 0xB8, 0xBD, 0x43, 0xE5,
+	0xB9, 0xA9, 0x43, 0xE5, 0xB9, 0xB2, 0x43, 0xE5,
+	// Bytes b00 - b3f
+	0xB9, 0xB4, 0x43, 0xE5, 0xB9, 0xBA, 0x43, 0xE5,
+	0xB9, 0xBC, 0x43, 0xE5, 0xB9, 0xBF, 0x43, 0xE5,
+	0xBA, 0xA6, 0x43, 0xE5, 0xBA, 0xB0, 0x43, 0xE5,
+	0xBA, 0xB3, 0x43, 0xE5, 0xBA, 0xB6, 0x43, 0xE5,
+	0xBB, 0x89, 0x43, 0xE5, 0xBB, 0x8A, 0x43, 0xE5,
+	0xBB, 0x92, 0x43, 0xE5, 0xBB, 0x93, 0x43, 0xE5,
+	0xBB, 0x99, 0x43, 0xE5, 0xBB, 0xAC, 0x43, 0xE5,
+	0xBB, 0xB4, 0x43, 0xE5, 0xBB, 0xBE, 0x43, 0xE5,
+	// Bytes b40 - b7f
+	0xBC, 0x84, 0x43, 0xE5, 0xBC, 0x8B, 0x43, 0xE5,
+	0xBC, 0x93, 0x43, 0xE5, 0xBC, 0xA2, 0x43, 0xE5,
+	0xBD, 0x90, 0x43, 0xE5, 0xBD, 0x93, 0x43, 0xE5,
+	0xBD, 0xA1, 0x43, 0xE5, 0xBD, 0xA2, 0x43, 0xE5,
+	0xBD, 0xA9, 0x43, 0xE5, 0xBD, 0xAB, 0x43, 0xE5,
+	0xBD, 0xB3, 0x43, 0xE5, 0xBE, 0x8B, 0x43, 0xE5,
+	0xBE, 0x8C, 0x43, 0xE5, 0xBE, 0x97, 0x43, 0xE5,
+	0xBE, 0x9A, 0x43, 0xE5, 0xBE, 0xA9, 0x43, 0xE5,
+	// Bytes b80 - bbf
+	0xBE, 0xAD, 0x43, 0xE5, 0xBF, 0x83, 0x43, 0xE5,
+	0xBF, 0x8D, 0x43, 0xE5, 0xBF, 0x97, 0x43, 0xE5,
+	0xBF, 0xB5, 0x43, 0xE5, 0xBF, 0xB9, 0x43, 0xE6,
+	0x80, 0x92, 0x43, 0xE6, 0x80, 0x9C, 0x43, 0xE6,
+	0x81, 0xB5, 0x43, 0xE6, 0x82, 0x81, 0x43, 0xE6,
+	0x82, 0x94, 0x43, 0xE6, 0x83, 0x87, 0x43, 0xE6,
+	0x83, 0x98, 0x43, 0xE6, 0x83, 0xA1, 0x43, 0xE6,
+	0x84, 0x88, 0x43, 0xE6, 0x85, 0x84, 0x43, 0xE6,
+	// Bytes bc0 - bff
+	0x85, 0x88, 0x43, 0xE6, 0x85, 0x8C, 0x43, 0xE6,
+	0x85, 0x8E, 0x43, 0xE6, 0x85, 0xA0, 0x43, 0xE6,
+	0x85, 0xA8, 0x43, 0xE6, 0x85, 0xBA, 0x43, 0xE6,
+	0x86, 0x8E, 0x43, 0xE6, 0x86, 0x90, 0x43, 0xE6,
+	0x86, 0xA4, 0x43, 0xE6, 0x86, 0xAF, 0x43, 0xE6,
+	0x86, 0xB2, 0x43, 0xE6, 0x87, 0x9E, 0x43, 0xE6,
+	0x87, 0xB2, 0x43, 0xE6, 0x87, 0xB6, 0x43, 0xE6,
+	0x88, 0x80, 0x43, 0xE6, 0x88, 0x88, 0x43, 0xE6,
+	// Bytes c00 - c3f
+	0x88, 0x90, 0x43, 0xE6, 0x88, 0x9B, 0x43, 0xE6,
+	0x88, 0xAE, 0x43, 0xE6, 0x88, 0xB4, 0x43, 0xE6,
+	0x88, 0xB6, 0x43, 0xE6, 0x89, 0x8B, 0x43, 0xE6,
+	0x89, 0x93, 0x43, 0xE6, 0x89, 0x9D, 0x43, 0xE6,
+	0x8A, 0x95, 0x43, 0xE6, 0x8A, 0xB1, 0x43, 0xE6,
+	0x8B, 0x89, 0x43, 0xE6, 0x8B, 0x8F, 0x43, 0xE6,
+	0x8B, 0x93, 0x43, 0xE6, 0x8B, 0x94, 0x43, 0xE6,
+	0x8B, 0xBC, 0x43, 0xE6, 0x8B, 0xBE, 0x43, 0xE6,
+	// Bytes c40 - c7f
+	0x8C, 0x87, 0x43, 0xE6, 0x8C, 0xBD, 0x43, 0xE6,
+	0x8D, 0x90, 0x43, 0xE6, 0x8D, 0x95, 0x43, 0xE6,
+	0x8D, 0xA8, 0x43, 0xE6, 0x8D, 0xBB, 0x43, 0xE6,
+	0x8E, 0x83, 0x43, 0xE6, 0x8E, 0xA0, 0x43, 0xE6,
+	0x8E, 0xA9, 0x43, 0xE6, 0x8F, 0x84, 0x43, 0xE6,
+	0x8F, 0x85, 0x43, 0xE6, 0x8F, 0xA4, 0x43, 0xE6,
+	0x90, 0x9C, 0x43, 0xE6, 0x90, 0xA2, 0x43, 0xE6,
+	0x91, 0x92, 0x43, 0xE6, 0x91, 0xA9, 0x43, 0xE6,
+	// Bytes c80 - cbf
+	0x91, 0xB7, 0x43, 0xE6, 0x91, 0xBE, 0x43, 0xE6,
+	0x92, 0x9A, 0x43, 0xE6, 0x92, 0x9D, 0x43, 0xE6,
+	0x93, 0x84, 0x43, 0xE6, 0x94, 0xAF, 0x43, 0xE6,
+	0x94, 0xB4, 0x43, 0xE6, 0x95, 0x8F, 0x43, 0xE6,
+	0x95, 0x96, 0x43, 0xE6, 0x95, 0xAC, 0x43, 0xE6,
+	0x95, 0xB8, 0x43, 0xE6, 0x96, 0x87, 0x43, 0xE6,
+	0x96, 0x97, 0x43, 0xE6, 0x96, 0x99, 0x43, 0xE6,
+	0x96, 0xA4, 0x43, 0xE6, 0x96, 0xB0, 0x43, 0xE6,
+	// Bytes cc0 - cff
+	0x96, 0xB9, 0x43, 0xE6, 0x97, 0x85, 0x43, 0xE6,
+	0x97, 0xA0, 0x43, 0xE6, 0x97, 0xA2, 0x43, 0xE6,
+	0x97, 0xA3, 0x43, 0xE6, 0x97, 0xA5, 0x43, 0xE6,
+	0x98, 0x93, 0x43, 0xE6, 0x98, 0xA0, 0x43, 0xE6,
+	0x99, 0x89, 0x43, 0xE6, 0x99, 0xB4, 0x43, 0xE6,
+	0x9A, 0x88, 0x43, 0xE6, 0x9A, 0x91, 0x43, 0xE6,
+	0x9A, 0x9C, 0x43, 0xE6, 0x9A, 0xB4, 0x43, 0xE6,
+	0x9B, 0x86, 0x43, 0xE6, 0x9B, 0xB0, 0x43, 0xE6,
+	// Bytes d00 - d3f
+	0x9B, 0xB4, 0x43, 0xE6, 0x9B, 0xB8, 0x43, 0xE6,
+	0x9C, 0x80, 0x43, 0xE6, 0x9C, 0x88, 0x43, 0xE6,
+	0x9C, 0x89, 0x43, 0xE6, 0x9C, 0x97, 0x43, 0xE6,
+	0x9C, 0x9B, 0x43, 0xE6, 0x9C, 0xA1, 0x43, 0xE6,
+	0x9C, 0xA8, 0x43, 0xE6, 0x9D, 0x8E, 0x43, 0xE6,
+	0x9D, 0x93, 0x43, 0xE6, 0x9D, 0x96, 0x43, 0xE6,
+	0x9D, 0x9E, 0x43, 0xE6, 0x9D, 0xBB, 0x43, 0xE6,
+	0x9E, 0x85, 0x43, 0xE6, 0x9E, 0x97, 0x43, 0xE6,
+	// Bytes d40 - d7f
+	0x9F, 0xB3, 0x43, 0xE6, 0x9F, 0xBA, 0x43, 0xE6,
+	0xA0, 0x97, 0x43, 0xE6, 0xA0, 0x9F, 0x43, 0xE6,
+	0xA0, 0xAA, 0x43, 0xE6, 0xA1, 0x92, 0x43, 0xE6,
+	0xA2, 0x81, 0x43, 0xE6, 0xA2, 0x85, 0x43, 0xE6,
+	0xA2, 0x8E, 0x43, 0xE6, 0xA2, 0xA8, 0x43, 0xE6,
+	0xA4, 0x94, 0x43, 0xE6, 0xA5, 0x82, 0x43, 0xE6,
+	0xA6, 0xA3, 0x43, 0xE6, 0xA7, 0xAA, 0x43, 0xE6,
+	0xA8, 0x82, 0x43, 0xE6, 0xA8, 0x93, 0x43, 0xE6,
+	// Bytes d80 - dbf
+	0xAA, 0xA8, 0x43, 0xE6, 0xAB, 0x93, 0x43, 0xE6,
+	0xAB, 0x9B, 0x43, 0xE6, 0xAC, 0x84, 0x43, 0xE6,
+	0xAC, 0xA0, 0x43, 0xE6, 0xAC, 0xA1, 0x43, 0xE6,
+	0xAD, 0x94, 0x43, 0xE6, 0xAD, 0xA2, 0x43, 0xE6,
+	0xAD, 0xA3, 0x43, 0xE6, 0xAD, 0xB2, 0x43, 0xE6,
+	0xAD, 0xB7, 0x43, 0xE6, 0xAD, 0xB9, 0x43, 0xE6,
+	0xAE, 0x9F, 0x43, 0xE6, 0xAE, 0xAE, 0x43, 0xE6,
+	0xAE, 0xB3, 0x43, 0xE6, 0xAE, 0xBA, 0x43, 0xE6,
+	// Bytes dc0 - dff
+	0xAE, 0xBB, 0x43, 0xE6, 0xAF, 0x8B, 0x43, 0xE6,
+	0xAF, 0x8D, 0x43, 0xE6, 0xAF, 0x94, 0x43, 0xE6,
+	0xAF, 0x9B, 0x43, 0xE6, 0xB0, 0x8F, 0x43, 0xE6,
+	0xB0, 0x94, 0x43, 0xE6, 0xB0, 0xB4, 0x43, 0xE6,
+	0xB1, 0x8E, 0x43, 0xE6, 0xB1, 0xA7, 0x43, 0xE6,
+	0xB2, 0x88, 0x43, 0xE6, 0xB2, 0xBF, 0x43, 0xE6,
+	0xB3, 0x8C, 0x43, 0xE6, 0xB3, 0x8D, 0x43, 0xE6,
+	0xB3, 0xA5, 0x43, 0xE6, 0xB3, 0xA8, 0x43, 0xE6,
+	// Bytes e00 - e3f
+	0xB4, 0x96, 0x43, 0xE6, 0xB4, 0x9B, 0x43, 0xE6,
+	0xB4, 0x9E, 0x43, 0xE6, 0xB4, 0xB4, 0x43, 0xE6,
+	0xB4, 0xBE, 0x43, 0xE6, 0xB5, 0x81, 0x43, 0xE6,
+	0xB5, 0xA9, 0x43, 0xE6, 0xB5, 0xAA, 0x43, 0xE6,
+	0xB5, 0xB7, 0x43, 0xE6, 0xB5, 0xB8, 0x43, 0xE6,
+	0xB6, 0x85, 0x43, 0xE6, 0xB7, 0x8B, 0x43, 0xE6,
+	0xB7, 0x9A, 0x43, 0xE6, 0xB7, 0xAA, 0x43, 0xE6,
+	0xB7, 0xB9, 0x43, 0xE6, 0xB8, 0x9A, 0x43, 0xE6,
+	// Bytes e40 - e7f
+	0xB8, 0xAF, 0x43, 0xE6, 0xB9, 0xAE, 0x43, 0xE6,
+	0xBA, 0x80, 0x43, 0xE6, 0xBA, 0x9C, 0x43, 0xE6,
+	0xBA, 0xBA, 0x43, 0xE6, 0xBB, 0x87, 0x43, 0xE6,
+	0xBB, 0x8B, 0x43, 0xE6, 0xBB, 0x91, 0x43, 0xE6,
+	0xBB, 0x9B, 0x43, 0xE6, 0xBC, 0x8F, 0x43, 0xE6,
+	0xBC, 0x94, 0x43, 0xE6, 0xBC, 0xA2, 0x43, 0xE6,
+	0xBC, 0xA3, 0x43, 0xE6, 0xBD, 0xAE, 0x43, 0xE6,
+	0xBF, 0x86, 0x43, 0xE6, 0xBF, 0xAB, 0x43, 0xE6,
+	// Bytes e80 - ebf
+	0xBF, 0xBE, 0x43, 0xE7, 0x80, 0x9B, 0x43, 0xE7,
+	0x80, 0x9E, 0x43, 0xE7, 0x80, 0xB9, 0x43, 0xE7,
+	0x81, 0x8A, 0x43, 0xE7, 0x81, 0xAB, 0x43, 0xE7,
+	0x81, 0xB0, 0x43, 0xE7, 0x81, 0xB7, 0x43, 0xE7,
+	0x81, 0xBD, 0x43, 0xE7, 0x82, 0x99, 0x43, 0xE7,
+	0x82, 0xAD, 0x43, 0xE7, 0x83, 0x88, 0x43, 0xE7,
+	0x83, 0x99, 0x43, 0xE7, 0x84, 0xA1, 0x43, 0xE7,
+	0x85, 0x85, 0x43, 0xE7, 0x85, 0x89, 0x43, 0xE7,
+	// Bytes ec0 - eff
+	0x85, 0xAE, 0x43, 0xE7, 0x86, 0x9C, 0x43, 0xE7,
+	0x87, 0x8E, 0x43, 0xE7, 0x87, 0x90, 0x43, 0xE7,
+	0x88, 0x90, 0x43, 0xE7, 0x88, 0x9B, 0x43, 0xE7,
+	0x88, 0xA8, 0x43, 0xE7, 0x88, 0xAA, 0x43, 0xE7,
+	0x88, 0xAB, 0x43, 0xE7, 0x88, 0xB5, 0x43, 0xE7,
+	0x88, 0xB6, 0x43, 0xE7, 0x88, 0xBB, 0x43, 0xE7,
+	0x88, 0xBF, 0x43, 0xE7, 0x89, 0x87, 0x43, 0xE7,
+	0x89, 0x90, 0x43, 0xE7, 0x89, 0x99, 0x43, 0xE7,
+	// Bytes f00 - f3f
+	0x89, 0x9B, 0x43, 0xE7, 0x89, 0xA2, 0x43, 0xE7,
+	0x89, 0xB9, 0x43, 0xE7, 0x8A, 0x80, 0x43, 0xE7,
+	0x8A, 0x95, 0x43, 0xE7, 0x8A, 0xAC, 0x43, 0xE7,
+	0x8A, 0xAF, 0x43, 0xE7, 0x8B, 0x80, 0x43, 0xE7,
+	0x8B, 0xBC, 0x43, 0xE7, 0x8C, 0xAA, 0x43, 0xE7,
+	0x8D, 0xB5, 0x43, 0xE7, 0x8D, 0xBA, 0x43, 0xE7,
+	0x8E, 0x84, 0x43, 0xE7, 0x8E, 0x87, 0x43, 0xE7,
+	0x8E, 0x89, 0x43, 0xE7, 0x8E, 0x8B, 0x43, 0xE7,
+	// Bytes f40 - f7f
+	0x8E, 0xA5, 0x43, 0xE7, 0x8E, 0xB2, 0x43, 0xE7,
+	0x8F, 0x9E, 0x43, 0xE7, 0x90, 0x86, 0x43, 0xE7,
+	0x90, 0x89, 0x43, 0xE7, 0x90, 0xA2, 0x43, 0xE7,
+	0x91, 0x87, 0x43, 0xE7, 0x91, 0x9C, 0x43, 0xE7,
+	0x91, 0xA9, 0x43, 0xE7, 0x91, 0xB1, 0x43, 0xE7,
+	0x92, 0x85, 0x43, 0xE7, 0x92, 0x89, 0x43, 0xE7,
+	0x92, 0x98, 0x43, 0xE7, 0x93, 0x8A, 0x43, 0xE7,
+	0x93, 0x9C, 0x43, 0xE7, 0x93, 0xA6, 0x43, 0xE7,
+	// Bytes f80 - fbf
+	0x94, 0x86, 0x43, 0xE7, 0x94, 0x98, 0x43, 0xE7,
+	0x94, 0x9F, 0x43, 0xE7, 0x94, 0xA4, 0x43, 0xE7,
+	0x94, 0xA8, 0x43, 0xE7, 0x94, 0xB0, 0x43, 0xE7,
+	0x94, 0xB2, 0x43, 0xE7, 0x94, 0xB3, 0x43, 0xE7,
+	0x94, 0xB7, 0x43, 0xE7, 0x94, 0xBB, 0x43, 0xE7,
+	0x94, 0xBE, 0x43, 0xE7, 0x95, 0x99, 0x43, 0xE7,
+	0x95, 0xA5, 0x43, 0xE7, 0x95, 0xB0, 0x43, 0xE7,
+	0x96, 0x8B, 0x43, 0xE7, 0x96, 0x92, 0x43, 0xE7,
+	// Bytes fc0 - fff
+	0x97, 0xA2, 0x43, 0xE7, 0x98, 0x90, 0x43, 0xE7,
+	0x98, 0x9D, 0x43, 0xE7, 0x98, 0x9F, 0x43, 0xE7,
+	0x99, 0x82, 0x43, 0xE7, 0x99, 0xA9, 0x43, 0xE7,
+	0x99, 0xB6, 0x43, 0xE7, 0x99, 0xBD, 0x43, 0xE7,
+	0x9A, 0xAE, 0x43, 0xE7, 0x9A, 0xBF, 0x43, 0xE7,
+	0x9B, 0x8A, 0x43, 0xE7, 0x9B, 0x9B, 0x43, 0xE7,
+	0x9B, 0xA3, 0x43, 0xE7, 0x9B, 0xA7, 0x43, 0xE7,
+	0x9B, 0xAE, 0x43, 0xE7, 0x9B, 0xB4, 0x43, 0xE7,
+	// Bytes 1000 - 103f
+	0x9C, 0x81, 0x43, 0xE7, 0x9C, 0x9E, 0x43, 0xE7,
+	0x9C, 0x9F, 0x43, 0xE7, 0x9D, 0x80, 0x43, 0xE7,
+	0x9D, 0x8A, 0x43, 0xE7, 0x9E, 0x8B, 0x43, 0xE7,
+	0x9E, 0xA7, 0x43, 0xE7, 0x9F, 0x9B, 0x43, 0xE7,
+	0x9F, 0xA2, 0x43, 0xE7, 0x9F, 0xB3, 0x43, 0xE7,
+	0xA1, 0x8E, 0x43, 0xE7, 0xA1, 0xAB, 0x43, 0xE7,
+	0xA2, 0x8C, 0x43, 0xE7, 0xA2, 0x91, 0x43, 0xE7,
+	0xA3, 0x8A, 0x43, 0xE7, 0xA3, 0x8C, 0x43, 0xE7,
+	// Bytes 1040 - 107f
+	0xA3, 0xBB, 0x43, 0xE7, 0xA4, 0xAA, 0x43, 0xE7,
+	0xA4, 0xBA, 0x43, 0xE7, 0xA4, 0xBC, 0x43, 0xE7,
+	0xA4, 0xBE, 0x43, 0xE7, 0xA5, 0x88, 0x43, 0xE7,
+	0xA5, 0x89, 0x43, 0xE7, 0xA5, 0x90, 0x43, 0xE7,
+	0xA5, 0x96, 0x43, 0xE7, 0xA5, 0x9D, 0x43, 0xE7,
+	0xA5, 0x9E, 0x43, 0xE7, 0xA5, 0xA5, 0x43, 0xE7,
+	0xA5, 0xBF, 0x43, 0xE7, 0xA6, 0x81, 0x43, 0xE7,
+	0xA6, 0x8D, 0x43, 0xE7, 0xA6, 0x8E, 0x43, 0xE7,
+	// Bytes 1080 - 10bf
+	0xA6, 0x8F, 0x43, 0xE7, 0xA6, 0xAE, 0x43, 0xE7,
+	0xA6, 0xB8, 0x43, 0xE7, 0xA6, 0xBE, 0x43, 0xE7,
+	0xA7, 0x8A, 0x43, 0xE7, 0xA7, 0x98, 0x43, 0xE7,
+	0xA7, 0xAB, 0x43, 0xE7, 0xA8, 0x9C, 0x43, 0xE7,
+	0xA9, 0x80, 0x43, 0xE7, 0xA9, 0x8A, 0x43, 0xE7,
+	0xA9, 0x8F, 0x43, 0xE7, 0xA9, 0xB4, 0x43, 0xE7,
+	0xA9, 0xBA, 0x43, 0xE7, 0xAA, 0x81, 0x43, 0xE7,
+	0xAA, 0xB1, 0x43, 0xE7, 0xAB, 0x8B, 0x43, 0xE7,
+	// Bytes 10c0 - 10ff
+	0xAB, 0xAE, 0x43, 0xE7, 0xAB, 0xB9, 0x43, 0xE7,
+	0xAC, 0xA0, 0x43, 0xE7, 0xAE, 0x8F, 0x43, 0xE7,
+	0xAF, 0x80, 0x43, 0xE7, 0xAF, 0x86, 0x43, 0xE7,
+	0xAF, 0x89, 0x43, 0xE7, 0xB0, 0xBE, 0x43, 0xE7,
+	0xB1, 0xA0, 0x43, 0xE7, 0xB1, 0xB3, 0x43, 0xE7,
+	0xB1, 0xBB, 0x43, 0xE7, 0xB2, 0x92, 0x43, 0xE7,
+	0xB2, 0xBE, 0x43, 0xE7, 0xB3, 0x92, 0x43, 0xE7,
+	0xB3, 0x96, 0x43, 0xE7, 0xB3, 0xA3, 0x43, 0xE7,
+	// Bytes 1100 - 113f
+	0xB3, 0xA7, 0x43, 0xE7, 0xB3, 0xA8, 0x43, 0xE7,
+	0xB3, 0xB8, 0x43, 0xE7, 0xB4, 0x80, 0x43, 0xE7,
+	0xB4, 0x90, 0x43, 0xE7, 0xB4, 0xA2, 0x43, 0xE7,
+	0xB4, 0xAF, 0x43, 0xE7, 0xB5, 0x82, 0x43, 0xE7,
+	0xB5, 0x9B, 0x43, 0xE7, 0xB5, 0xA3, 0x43, 0xE7,
+	0xB6, 0xA0, 0x43, 0xE7, 0xB6, 0xBE, 0x43, 0xE7,
+	0xB7, 0x87, 0x43, 0xE7, 0xB7, 0xB4, 0x43, 0xE7,
+	0xB8, 0x82, 0x43, 0xE7, 0xB8, 0x89, 0x43, 0xE7,
+	// Bytes 1140 - 117f
+	0xB8, 0xB7, 0x43, 0xE7, 0xB9, 0x81, 0x43, 0xE7,
+	0xB9, 0x85, 0x43, 0xE7, 0xBC, 0xB6, 0x43, 0xE7,
+	0xBC, 0xBE, 0x43, 0xE7, 0xBD, 0x91, 0x43, 0xE7,
+	0xBD, 0xB2, 0x43, 0xE7, 0xBD, 0xB9, 0x43, 0xE7,
+	0xBD, 0xBA, 0x43, 0xE7, 0xBE, 0x85, 0x43, 0xE7,
+	0xBE, 0x8A, 0x43, 0xE7, 0xBE, 0x95, 0x43, 0xE7,
+	0xBE, 0x9A, 0x43, 0xE7, 0xBE, 0xBD, 0x43, 0xE7,
+	0xBF, 0xBA, 0x43, 0xE8, 0x80, 0x81, 0x43, 0xE8,
+	// Bytes 1180 - 11bf
+	0x80, 0x85, 0x43, 0xE8, 0x80, 0x8C, 0x43, 0xE8,
+	0x80, 0x92, 0x43, 0xE8, 0x80, 0xB3, 0x43, 0xE8,
+	0x81, 0x86, 0x43, 0xE8, 0x81, 0xA0, 0x43, 0xE8,
+	0x81, 0xAF, 0x43, 0xE8, 0x81, 0xB0, 0x43, 0xE8,
+	0x81, 0xBE, 0x43, 0xE8, 0x81, 0xBF, 0x43, 0xE8,
+	0x82, 0x89, 0x43, 0xE8, 0x82, 0x8B, 0x43, 0xE8,
+	0x82, 0xAD, 0x43, 0xE8, 0x82, 0xB2, 0x43, 0xE8,
+	0x84, 0x83, 0x43, 0xE8, 0x84, 0xBE, 0x43, 0xE8,
+	// Bytes 11c0 - 11ff
+	0x87, 0x98, 0x43, 0xE8, 0x87, 0xA3, 0x43, 0xE8,
+	0x87, 0xA8, 0x43, 0xE8, 0x87, 0xAA, 0x43, 0xE8,
+	0x87, 0xAD, 0x43, 0xE8, 0x87, 0xB3, 0x43, 0xE8,
+	0x87, 0xBC, 0x43, 0xE8, 0x88, 0x81, 0x43, 0xE8,
+	0x88, 0x84, 0x43, 0xE8, 0x88, 0x8C, 0x43, 0xE8,
+	0x88, 0x98, 0x43, 0xE8, 0x88, 0x9B, 0x43, 0xE8,
+	0x88, 0x9F, 0x43, 0xE8, 0x89, 0xAE, 0x43, 0xE8,
+	0x89, 0xAF, 0x43, 0xE8, 0x89, 0xB2, 0x43, 0xE8,
+	// Bytes 1200 - 123f
+	0x89, 0xB8, 0x43, 0xE8, 0x89, 0xB9, 0x43, 0xE8,
+	0x8A, 0x8B, 0x43, 0xE8, 0x8A, 0x91, 0x43, 0xE8,
+	0x8A, 0x9D, 0x43, 0xE8, 0x8A, 0xB1, 0x43, 0xE8,
+	0x8A, 0xB3, 0x43, 0xE8, 0x8A, 0xBD, 0x43, 0xE8,
+	0x8B, 0xA5, 0x43, 0xE8, 0x8B, 0xA6, 0x43, 0xE8,
+	0x8C, 0x9D, 0x43, 0xE8, 0x8C, 0xA3, 0x43, 0xE8,
+	0x8C, 0xB6, 0x43, 0xE8, 0x8D, 0x92, 0x43, 0xE8,
+	0x8D, 0x93, 0x43, 0xE8, 0x8D, 0xA3, 0x43, 0xE8,
+	// Bytes 1240 - 127f
+	0x8E, 0xAD, 0x43, 0xE8, 0x8E, 0xBD, 0x43, 0xE8,
+	0x8F, 0x89, 0x43, 0xE8, 0x8F, 0x8A, 0x43, 0xE8,
+	0x8F, 0x8C, 0x43, 0xE8, 0x8F, 0x9C, 0x43, 0xE8,
+	0x8F, 0xA7, 0x43, 0xE8, 0x8F, 0xAF, 0x43, 0xE8,
+	0x8F, 0xB1, 0x43, 0xE8, 0x90, 0xBD, 0x43, 0xE8,
+	0x91, 0x89, 0x43, 0xE8, 0x91, 0x97, 0x43, 0xE8,
+	0x93, 0xAE, 0x43, 0xE8, 0x93, 0xB1, 0x43, 0xE8,
+	0x93, 0xB3, 0x43, 0xE8, 0x93, 0xBC, 0x43, 0xE8,
+	// Bytes 1280 - 12bf
+	0x94, 0x96, 0x43, 0xE8, 0x95, 0xA4, 0x43, 0xE8,
+	0x97, 0x8D, 0x43, 0xE8, 0x97, 0xBA, 0x43, 0xE8,
+	0x98, 0x86, 0x43, 0xE8, 0x98, 0x92, 0x43, 0xE8,
+	0x98, 0xAD, 0x43, 0xE8, 0x98, 0xBF, 0x43, 0xE8,
+	0x99, 0x8D, 0x43, 0xE8, 0x99, 0x90, 0x43, 0xE8,
+	0x99, 0x9C, 0x43, 0xE8, 0x99, 0xA7, 0x43, 0xE8,
+	0x99, 0xA9, 0x43, 0xE8, 0x99, 0xAB, 0x43, 0xE8,
+	0x9A, 0x88, 0x43, 0xE8, 0x9A, 0xA9, 0x43, 0xE8,
+	// Bytes 12c0 - 12ff
+	0x9B, 0xA2, 0x43, 0xE8, 0x9C, 0x8E, 0x43, 0xE8,
+	0x9C, 0xA8, 0x43, 0xE8, 0x9D, 0xAB, 0x43, 0xE8,
+	0x9D, 0xB9, 0x43, 0xE8, 0x9E, 0x86, 0x43, 0xE8,
+	0x9E, 0xBA, 0x43, 0xE8, 0x9F, 0xA1, 0x43, 0xE8,
+	0xA0, 0x81, 0x43, 0xE8, 0xA0, 0x9F, 0x43, 0xE8,
+	0xA1, 0x80, 0x43, 0xE8, 0xA1, 0x8C, 0x43, 0xE8,
+	0xA1, 0xA0, 0x43, 0xE8, 0xA1, 0xA3, 0x43, 0xE8,
+	0xA3, 0x82, 0x43, 0xE8, 0xA3, 0x8F, 0x43, 0xE8,
+	// Bytes 1300 - 133f
+	0xA3, 0x97, 0x43, 0xE8, 0xA3, 0x9E, 0x43, 0xE8,
+	0xA3, 0xA1, 0x43, 0xE8, 0xA3, 0xB8, 0x43, 0xE8,
+	0xA3, 0xBA, 0x43, 0xE8, 0xA4, 0x90, 0x43, 0xE8,
+	0xA5, 0x81, 0x43, 0xE8, 0xA5, 0xA4, 0x43, 0xE8,
+	0xA5, 0xBE, 0x43, 0xE8, 0xA6, 0x86, 0x43, 0xE8,
+	0xA6, 0x8B, 0x43, 0xE8, 0xA6, 0x96, 0x43, 0xE8,
+	0xA7, 0x92, 0x43, 0xE8, 0xA7, 0xA3, 0x43, 0xE8,
+	0xA8, 0x80, 0x43, 0xE8, 0xAA, 0xA0, 0x43, 0xE8,
+	// Bytes 1340 - 137f
+	0xAA, 0xAA, 0x43, 0xE8, 0xAA, 0xBF, 0x43, 0xE8,
+	0xAB, 0x8B, 0x43, 0xE8, 0xAB, 0x92, 0x43, 0xE8,
+	0xAB, 0x96, 0x43, 0xE8, 0xAB, 0xAD, 0x43, 0xE8,
+	0xAB, 0xB8, 0x43, 0xE8, 0xAB, 0xBE, 0x43, 0xE8,
+	0xAC, 0x81, 0x43, 0xE8, 0xAC, 0xB9, 0x43, 0xE8,
+	0xAD, 0x98, 0x43, 0xE8, 0xAE, 0x80, 0x43, 0xE8,
+	0xAE, 0x8A, 0x43, 0xE8, 0xB0, 0xB7, 0x43, 0xE8,
+	0xB1, 0x86, 0x43, 0xE8, 0xB1, 0x88, 0x43, 0xE8,
+	// Bytes 1380 - 13bf
+	0xB1, 0x95, 0x43, 0xE8, 0xB1, 0xB8, 0x43, 0xE8,
+	0xB2, 0x9D, 0x43, 0xE8, 0xB2, 0xA1, 0x43, 0xE8,
+	0xB2, 0xA9, 0x43, 0xE8, 0xB2, 0xAB, 0x43, 0xE8,
+	0xB3, 0x81, 0x43, 0xE8, 0xB3, 0x82, 0x43, 0xE8,
+	0xB3, 0x87, 0x43, 0xE8, 0xB3, 0x88, 0x43, 0xE8,
+	0xB3, 0x93, 0x43, 0xE8, 0xB4, 0x88, 0x43, 0xE8,
+	0xB4, 0x9B, 0x43, 0xE8, 0xB5, 0xA4, 0x43, 0xE8,
+	0xB5, 0xB0, 0x43, 0xE8, 0xB5, 0xB7, 0x43, 0xE8,
+	// Bytes 13c0 - 13ff
+	0xB6, 0xB3, 0x43, 0xE8, 0xB6, 0xBC, 0x43, 0xE8,
+	0xB7, 0x8B, 0x43, 0xE8, 0xB7, 0xAF, 0x43, 0xE8,
+	0xB7, 0xB0, 0x43, 0xE8, 0xBA, 0xAB, 0x43, 0xE8,
+	0xBB, 0x8A, 0x43, 0xE8, 0xBB, 0x94, 0x43, 0xE8,
+	0xBC, 0xA6, 0x43, 0xE8, 0xBC, 0xAA, 0x43, 0xE8,
+	0xBC, 0xB8, 0x43, 0xE8, 0xBC, 0xBB, 0x43, 0xE8,
+	0xBD, 0xA2, 0x43, 0xE8, 0xBE, 0x9B, 0x43, 0xE8,
+	0xBE, 0x9E, 0x43, 0xE8, 0xBE, 0xB0, 0x43, 0xE8,
+	// Bytes 1400 - 143f
+	0xBE, 0xB5, 0x43, 0xE8, 0xBE, 0xB6, 0x43, 0xE9,
+	0x80, 0xA3, 0x43, 0xE9, 0x80, 0xB8, 0x43, 0xE9,
+	0x81, 0x8A, 0x43, 0xE9, 0x81, 0xA9, 0x43, 0xE9,
+	0x81, 0xB2, 0x43, 0xE9, 0x81, 0xBC, 0x43, 0xE9,
+	0x82, 0x8F, 0x43, 0xE9, 0x82, 0x91, 0x43, 0xE9,
+	0x82, 0x94, 0x43, 0xE9, 0x83, 0x8E, 0x43, 0xE9,
+	0x83, 0x9E, 0x43, 0xE9, 0x83, 0xB1, 0x43, 0xE9,
+	0x83, 0xBD, 0x43, 0xE9, 0x84, 0x91, 0x43, 0xE9,
+	// Bytes 1440 - 147f
+	0x84, 0x9B, 0x43, 0xE9, 0x85, 0x89, 0x43, 0xE9,
+	0x85, 0x8D, 0x43, 0xE9, 0x85, 0xAA, 0x43, 0xE9,
+	0x86, 0x99, 0x43, 0xE9, 0x86, 0xB4, 0x43, 0xE9,
+	0x87, 0x86, 0x43, 0xE9, 0x87, 0x8C, 0x43, 0xE9,
+	0x87, 0x8F, 0x43, 0xE9, 0x87, 0x91, 0x43, 0xE9,
+	0x88, 0xB4, 0x43, 0xE9, 0x88, 0xB8, 0x43, 0xE9,
+	0x89, 0xB6, 0x43, 0xE9, 0x89, 0xBC, 0x43, 0xE9,
+	0x8B, 0x97, 0x43, 0xE9, 0x8B, 0x98, 0x43, 0xE9,
+	// Bytes 1480 - 14bf
+	0x8C, 0x84, 0x43, 0xE9, 0x8D, 0x8A, 0x43, 0xE9,
+	0x8F, 0xB9, 0x43, 0xE9, 0x90, 0x95, 0x43, 0xE9,
+	0x95, 0xB7, 0x43, 0xE9, 0x96, 0x80, 0x43, 0xE9,
+	0x96, 0x8B, 0x43, 0xE9, 0x96, 0xAD, 0x43, 0xE9,
+	0x96, 0xB7, 0x43, 0xE9, 0x98, 0x9C, 0x43, 0xE9,
+	0x98, 0xAE, 0x43, 0xE9, 0x99, 0x8B, 0x43, 0xE9,
+	0x99, 0x8D, 0x43, 0xE9, 0x99, 0xB5, 0x43, 0xE9,
+	0x99, 0xB8, 0x43, 0xE9, 0x99, 0xBC, 0x43, 0xE9,
+	// Bytes 14c0 - 14ff
+	0x9A, 0x86, 0x43, 0xE9, 0x9A, 0xA3, 0x43, 0xE9,
+	0x9A, 0xB6, 0x43, 0xE9, 0x9A, 0xB7, 0x43, 0xE9,
+	0x9A, 0xB8, 0x43, 0xE9, 0x9A, 0xB9, 0x43, 0xE9,
+	0x9B, 0x83, 0x43, 0xE9, 0x9B, 0xA2, 0x43, 0xE9,
+	0x9B, 0xA3, 0x43, 0xE9, 0x9B, 0xA8, 0x43, 0xE9,
+	0x9B, 0xB6, 0x43, 0xE9, 0x9B, 0xB7, 0x43, 0xE9,
+	0x9C, 0xA3, 0x43, 0xE9, 0x9C, 0xB2, 0x43, 0xE9,
+	0x9D, 0x88, 0x43, 0xE9, 0x9D, 0x91, 0x43, 0xE9,
+	// Bytes 1500 - 153f
+	0x9D, 0x96, 0x43, 0xE9, 0x9D, 0x9E, 0x43, 0xE9,
+	0x9D, 0xA2, 0x43, 0xE9, 0x9D, 0xA9, 0x43, 0xE9,
+	0x9F, 0x8B, 0x43, 0xE9, 0x9F, 0x9B, 0x43, 0xE9,
+	0x9F, 0xA0, 0x43, 0xE9, 0x9F, 0xAD, 0x43, 0xE9,
+	0x9F, 0xB3, 0x43, 0xE9, 0x9F, 0xBF, 0x43, 0xE9,
+	0xA0, 0x81, 0x43, 0xE9, 0xA0, 0x85, 0x43, 0xE9,
+	0xA0, 0x8B, 0x43, 0xE9, 0xA0, 0x98, 0x43, 0xE9,
+	0xA0, 0xA9, 0x43, 0xE9, 0xA0, 0xBB, 0x43, 0xE9,
+	// Bytes 1540 - 157f
+	0xA1, 0x9E, 0x43, 0xE9, 0xA2, 0xA8, 0x43, 0xE9,
+	0xA3, 0x9B, 0x43, 0xE9, 0xA3, 0x9F, 0x43, 0xE9,
+	0xA3, 0xA2, 0x43, 0xE9, 0xA3, 0xAF, 0x43, 0xE9,
+	0xA3, 0xBC, 0x43, 0xE9, 0xA4, 0xA8, 0x43, 0xE9,
+	0xA4, 0xA9, 0x43, 0xE9, 0xA6, 0x96, 0x43, 0xE9,
+	0xA6, 0x99, 0x43, 0xE9, 0xA6, 0xA7, 0x43, 0xE9,
+	0xA6, 0xAC, 0x43, 0xE9, 0xA7, 0x82, 0x43, 0xE9,
+	0xA7, 0xB1, 0x43, 0xE9, 0xA7, 0xBE, 0x43, 0xE9,
+	// Bytes 1580 - 15bf
+	0xA9, 0xAA, 0x43, 0xE9, 0xAA, 0xA8, 0x43, 0xE9,
+	0xAB, 0x98, 0x43, 0xE9, 0xAB, 0x9F, 0x43, 0xE9,
+	0xAC, 0x92, 0x43, 0xE9, 0xAC, 0xA5, 0x43, 0xE9,
+	0xAC, 0xAF, 0x43, 0xE9, 0xAC, 0xB2, 0x43, 0xE9,
+	0xAC, 0xBC, 0x43, 0xE9, 0xAD, 0x9A, 0x43, 0xE9,
+	0xAD, 0xAF, 0x43, 0xE9, 0xB1, 0x80, 0x43, 0xE9,
+	0xB1, 0x97, 0x43, 0xE9, 0xB3, 0xA5, 0x43, 0xE9,
+	0xB3, 0xBD, 0x43, 0xE9, 0xB5, 0xA7, 0x43, 0xE9,
+	// Bytes 15c0 - 15ff
+	0xB6, 0xB4, 0x43, 0xE9, 0xB7, 0xBA, 0x43, 0xE9,
+	0xB8, 0x9E, 0x43, 0xE9, 0xB9, 0xB5, 0x43, 0xE9,
+	0xB9, 0xBF, 0x43, 0xE9, 0xBA, 0x97, 0x43, 0xE9,
+	0xBA, 0x9F, 0x43, 0xE9, 0xBA, 0xA5, 0x43, 0xE9,
+	0xBA, 0xBB, 0x43, 0xE9, 0xBB, 0x83, 0x43, 0xE9,
+	0xBB, 0x8D, 0x43, 0xE9, 0xBB, 0x8E, 0x43, 0xE9,
+	0xBB, 0x91, 0x43, 0xE9, 0xBB, 0xB9, 0x43, 0xE9,
+	0xBB, 0xBD, 0x43, 0xE9, 0xBB, 0xBE, 0x43, 0xE9,
+	// Bytes 1600 - 163f
+	0xBC, 0x85, 0x43, 0xE9, 0xBC, 0x8E, 0x43, 0xE9,
+	0xBC, 0x8F, 0x43, 0xE9, 0xBC, 0x93, 0x43, 0xE9,
+	0xBC, 0x96, 0x43, 0xE9, 0xBC, 0xA0, 0x43, 0xE9,
+	0xBC, 0xBB, 0x43, 0xE9, 0xBD, 0x83, 0x43, 0xE9,
+	0xBD, 0x8A, 0x43, 0xE9, 0xBD, 0x92, 0x43, 0xE9,
+	0xBE, 0x8D, 0x43, 0xE9, 0xBE, 0x8E, 0x43, 0xE9,
+	0xBE, 0x9C, 0x43, 0xE9, 0xBE, 0x9F, 0x43, 0xE9,
+	0xBE, 0xA0, 0x43, 0xEA, 0x9C, 0xA7, 0x43, 0xEA,
+	// Bytes 1640 - 167f
+	0x9D, 0xAF, 0x43, 0xEA, 0xAC, 0xB7, 0x43, 0xEA,
+	0xAD, 0x92, 0x44, 0xF0, 0xA0, 0x84, 0xA2, 0x44,
+	0xF0, 0xA0, 0x94, 0x9C, 0x44, 0xF0, 0xA0, 0x94,
+	0xA5, 0x44, 0xF0, 0xA0, 0x95, 0x8B, 0x44, 0xF0,
+	0xA0, 0x98, 0xBA, 0x44, 0xF0, 0xA0, 0xA0, 0x84,
+	0x44, 0xF0, 0xA0, 0xA3, 0x9E, 0x44, 0xF0, 0xA0,
+	0xA8, 0xAC, 0x44, 0xF0, 0xA0, 0xAD, 0xA3, 0x44,
+	0xF0, 0xA1, 0x93, 0xA4, 0x44, 0xF0, 0xA1, 0x9A,
+	// Bytes 1680 - 16bf
+	0xA8, 0x44, 0xF0, 0xA1, 0x9B, 0xAA, 0x44, 0xF0,
+	0xA1, 0xA7, 0x88, 0x44, 0xF0, 0xA1, 0xAC, 0x98,
+	0x44, 0xF0, 0xA1, 0xB4, 0x8B, 0x44, 0xF0, 0xA1,
+	0xB7, 0xA4, 0x44, 0xF0, 0xA1, 0xB7, 0xA6, 0x44,
+	0xF0, 0xA2, 0x86, 0x83, 0x44, 0xF0, 0xA2, 0x86,
+	0x9F, 0x44, 0xF0, 0xA2, 0x8C, 0xB1, 0x44, 0xF0,
+	0xA2, 0x9B, 0x94, 0x44, 0xF0, 0xA2, 0xA1, 0x84,
+	0x44, 0xF0, 0xA2, 0xA1, 0x8A, 0x44, 0xF0, 0xA2,
+	// Bytes 16c0 - 16ff
+	0xAC, 0x8C, 0x44, 0xF0, 0xA2, 0xAF, 0xB1, 0x44,
+	0xF0, 0xA3, 0x80, 0x8A, 0x44, 0xF0, 0xA3, 0x8A,
+	0xB8, 0x44, 0xF0, 0xA3, 0x8D, 0x9F, 0x44, 0xF0,
+	0xA3, 0x8E, 0x93, 0x44, 0xF0, 0xA3, 0x8E, 0x9C,
+	0x44, 0xF0, 0xA3, 0x8F, 0x83, 0x44, 0xF0, 0xA3,
+	0x8F, 0x95, 0x44, 0xF0, 0xA3, 0x91, 0xAD, 0x44,
+	0xF0, 0xA3, 0x9A, 0xA3, 0x44, 0xF0, 0xA3, 0xA2,
+	0xA7, 0x44, 0xF0, 0xA3, 0xAA, 0x8D, 0x44, 0xF0,
+	// Bytes 1700 - 173f
+	0xA3, 0xAB, 0xBA, 0x44, 0xF0, 0xA3, 0xB2, 0xBC,
+	0x44, 0xF0, 0xA3, 0xB4, 0x9E, 0x44, 0xF0, 0xA3,
+	0xBB, 0x91, 0x44, 0xF0, 0xA3, 0xBD, 0x9E, 0x44,
+	0xF0, 0xA3, 0xBE, 0x8E, 0x44, 0xF0, 0xA4, 0x89,
+	0xA3, 0x44, 0xF0, 0xA4, 0x8B, 0xAE, 0x44, 0xF0,
+	0xA4, 0x8E, 0xAB, 0x44, 0xF0, 0xA4, 0x98, 0x88,
+	0x44, 0xF0, 0xA4, 0x9C, 0xB5, 0x44, 0xF0, 0xA4,
+	0xA0, 0x94, 0x44, 0xF0, 0xA4, 0xB0, 0xB6, 0x44,
+	// Bytes 1740 - 177f
+	0xF0, 0xA4, 0xB2, 0x92, 0x44, 0xF0, 0xA4, 0xBE,
+	0xA1, 0x44, 0xF0, 0xA4, 0xBE, 0xB8, 0x44, 0xF0,
+	0xA5, 0x81, 0x84, 0x44, 0xF0, 0xA5, 0x83, 0xB2,
+	0x44, 0xF0, 0xA5, 0x83, 0xB3, 0x44, 0xF0, 0xA5,
+	0x84, 0x99, 0x44, 0xF0, 0xA5, 0x84, 0xB3, 0x44,
+	0xF0, 0xA5, 0x89, 0x89, 0x44, 0xF0, 0xA5, 0x90,
+	0x9D, 0x44, 0xF0, 0xA5, 0x98, 0xA6, 0x44, 0xF0,
+	0xA5, 0x9A, 0x9A, 0x44, 0xF0, 0xA5, 0x9B, 0x85,
+	// Bytes 1780 - 17bf
+	0x44, 0xF0, 0xA5, 0xA5, 0xBC, 0x44, 0xF0, 0xA5,
+	0xAA, 0xA7, 0x44, 0xF0, 0xA5, 0xAE, 0xAB, 0x44,
+	0xF0, 0xA5, 0xB2, 0x80, 0x44, 0xF0, 0xA5, 0xB3,
+	0x90, 0x44, 0xF0, 0xA5, 0xBE, 0x86, 0x44, 0xF0,
+	0xA6, 0x87, 0x9A, 0x44, 0xF0, 0xA6, 0x88, 0xA8,
+	0x44, 0xF0, 0xA6, 0x89, 0x87, 0x44, 0xF0, 0xA6,
+	0x8B, 0x99, 0x44, 0xF0, 0xA6, 0x8C, 0xBE, 0x44,
+	0xF0, 0xA6, 0x93, 0x9A, 0x44, 0xF0, 0xA6, 0x94,
+	// Bytes 17c0 - 17ff
+	0xA3, 0x44, 0xF0, 0xA6, 0x96, 0xA8, 0x44, 0xF0,
+	0xA6, 0x9E, 0xA7, 0x44, 0xF0, 0xA6, 0x9E, 0xB5,
+	0x44, 0xF0, 0xA6, 0xAC, 0xBC, 0x44, 0xF0, 0xA6,
+	0xB0, 0xB6, 0x44, 0xF0, 0xA6, 0xB3, 0x95, 0x44,
+	0xF0, 0xA6, 0xB5, 0xAB, 0x44, 0xF0, 0xA6, 0xBC,
+	0xAC, 0x44, 0xF0, 0xA6, 0xBE, 0xB1, 0x44, 0xF0,
+	0xA7, 0x83, 0x92, 0x44, 0xF0, 0xA7, 0x8F, 0x8A,
+	0x44, 0xF0, 0xA7, 0x99, 0xA7, 0x44, 0xF0, 0xA7,
+	// Bytes 1800 - 183f
+	0xA2, 0xAE, 0x44, 0xF0, 0xA7, 0xA5, 0xA6, 0x44,
+	0xF0, 0xA7, 0xB2, 0xA8, 0x44, 0xF0, 0xA7, 0xBB,
+	0x93, 0x44, 0xF0, 0xA7, 0xBC, 0xAF, 0x44, 0xF0,
+	0xA8, 0x97, 0x92, 0x44, 0xF0, 0xA8, 0x97, 0xAD,
+	0x44, 0xF0, 0xA8, 0x9C, 0xAE, 0x44, 0xF0, 0xA8,
+	0xAF, 0xBA, 0x44, 0xF0, 0xA8, 0xB5, 0xB7, 0x44,
+	0xF0, 0xA9, 0x85, 0x85, 0x44, 0xF0, 0xA9, 0x87,
+	0x9F, 0x44, 0xF0, 0xA9, 0x88, 0x9A, 0x44, 0xF0,
+	// Bytes 1840 - 187f
+	0xA9, 0x90, 0x8A, 0x44, 0xF0, 0xA9, 0x92, 0x96,
+	0x44, 0xF0, 0xA9, 0x96, 0xB6, 0x44, 0xF0, 0xA9,
+	0xAC, 0xB0, 0x44, 0xF0, 0xAA, 0x83, 0x8E, 0x44,
+	0xF0, 0xAA, 0x84, 0x85, 0x44, 0xF0, 0xAA, 0x88,
+	0x8E, 0x44, 0xF0, 0xAA, 0x8A, 0x91, 0x44, 0xF0,
+	0xAA, 0x8E, 0x92, 0x44, 0xF0, 0xAA, 0x98, 0x80,
+	0x42, 0x21, 0x21, 0x42, 0x21, 0x3F, 0x42, 0x2E,
+	0x2E, 0x42, 0x30, 0x2C, 0x42, 0x30, 0x2E, 0x42,
+	// Bytes 1880 - 18bf
+	0x31, 0x2C, 0x42, 0x31, 0x2E, 0x42, 0x31, 0x30,
+	0x42, 0x31, 0x31, 0x42, 0x31, 0x32, 0x42, 0x31,
+	0x33, 0x42, 0x31, 0x34, 0x42, 0x31, 0x35, 0x42,
+	0x31, 0x36, 0x42, 0x31, 0x37, 0x42, 0x31, 0x38,
+	0x42, 0x31, 0x39, 0x42, 0x32, 0x2C, 0x42, 0x32,
+	0x2E, 0x42, 0x32, 0x30, 0x42, 0x32, 0x31, 0x42,
+	0x32, 0x32, 0x42, 0x32, 0x33, 0x42, 0x32, 0x34,
+	0x42, 0x32, 0x35, 0x42, 0x32, 0x36, 0x42, 0x32,
+	// Bytes 18c0 - 18ff
+	0x37, 0x42, 0x32, 0x38, 0x42, 0x32, 0x39, 0x42,
+	0x33, 0x2C, 0x42, 0x33, 0x2E, 0x42, 0x33, 0x30,
+	0x42, 0x33, 0x31, 0x42, 0x33, 0x32, 0x42, 0x33,
+	0x33, 0x42, 0x33, 0x34, 0x42, 0x33, 0x35, 0x42,
+	0x33, 0x36, 0x42, 0x33, 0x37, 0x42, 0x33, 0x38,
+	0x42, 0x33, 0x39, 0x42, 0x34, 0x2C, 0x42, 0x34,
+	0x2E, 0x42, 0x34, 0x30, 0x42, 0x34, 0x31, 0x42,
+	0x34, 0x32, 0x42, 0x34, 0x33, 0x42, 0x34, 0x34,
+	// Bytes 1900 - 193f
+	0x42, 0x34, 0x35, 0x42, 0x34, 0x36, 0x42, 0x34,
+	0x37, 0x42, 0x34, 0x38, 0x42, 0x34, 0x39, 0x42,
+	0x35, 0x2C, 0x42, 0x35, 0x2E, 0x42, 0x35, 0x30,
+	0x42, 0x36, 0x2C, 0x42, 0x36, 0x2E, 0x42, 0x37,
+	0x2C, 0x42, 0x37, 0x2E, 0x42, 0x38, 0x2C, 0x42,
+	0x38, 0x2E, 0x42, 0x39, 0x2C, 0x42, 0x39, 0x2E,
+	0x42, 0x3D, 0x3D, 0x42, 0x3F, 0x21, 0x42, 0x3F,
+	0x3F, 0x42, 0x41, 0x55, 0x42, 0x42, 0x71, 0x42,
+	// Bytes 1940 - 197f
+	0x43, 0x44, 0x42, 0x44, 0x4A, 0x42, 0x44, 0x5A,
+	0x42, 0x44, 0x7A, 0x42, 0x47, 0x42, 0x42, 0x47,
+	0x79, 0x42, 0x48, 0x50, 0x42, 0x48, 0x56, 0x42,
+	0x48, 0x67, 0x42, 0x48, 0x7A, 0x42, 0x49, 0x49,
+	0x42, 0x49, 0x4A, 0x42, 0x49, 0x55, 0x42, 0x49,
+	0x56, 0x42, 0x49, 0x58, 0x42, 0x4B, 0x42, 0x42,
+	0x4B, 0x4B, 0x42, 0x4B, 0x4D, 0x42, 0x4C, 0x4A,
+	0x42, 0x4C, 0x6A, 0x42, 0x4D, 0x42, 0x42, 0x4D,
+	// Bytes 1980 - 19bf
+	0x43, 0x42, 0x4D, 0x44, 0x42, 0x4D, 0x52, 0x42,
+	0x4D, 0x56, 0x42, 0x4D, 0x57, 0x42, 0x4E, 0x4A,
+	0x42, 0x4E, 0x6A, 0x42, 0x4E, 0x6F, 0x42, 0x50,
+	0x48, 0x42, 0x50, 0x52, 0x42, 0x50, 0x61, 0x42,
+	0x52, 0x73, 0x42, 0x53, 0x44, 0x42, 0x53, 0x4D,
+	0x42, 0x53, 0x53, 0x42, 0x53, 0x76, 0x42, 0x54,
+	0x4D, 0x42, 0x56, 0x49, 0x42, 0x57, 0x43, 0x42,
+	0x57, 0x5A, 0x42, 0x57, 0x62, 0x42, 0x58, 0x49,
+	// Bytes 19c0 - 19ff
+	0x42, 0x63, 0x63, 0x42, 0x63, 0x64, 0x42, 0x63,
+	0x6D, 0x42, 0x64, 0x42, 0x42, 0x64, 0x61, 0x42,
+	0x64, 0x6C, 0x42, 0x64, 0x6D, 0x42, 0x64, 0x7A,
+	0x42, 0x65, 0x56, 0x42, 0x66, 0x66, 0x42, 0x66,
+	0x69, 0x42, 0x66, 0x6C, 0x42, 0x66, 0x6D, 0x42,
+	0x68, 0x61, 0x42, 0x69, 0x69, 0x42, 0x69, 0x6A,
+	0x42, 0x69, 0x6E, 0x42, 0x69, 0x76, 0x42, 0x69,
+	0x78, 0x42, 0x6B, 0x41, 0x42, 0x6B, 0x56, 0x42,
+	// Bytes 1a00 - 1a3f
+	0x6B, 0x57, 0x42, 0x6B, 0x67, 0x42, 0x6B, 0x6C,
+	0x42, 0x6B, 0x6D, 0x42, 0x6B, 0x74, 0x42, 0x6C,
+	0x6A, 0x42, 0x6C, 0x6D, 0x42, 0x6C, 0x6E, 0x42,
+	0x6C, 0x78, 0x42, 0x6D, 0x32, 0x42, 0x6D, 0x33,
+	0x42, 0x6D, 0x41, 0x42, 0x6D, 0x56, 0x42, 0x6D,
+	0x57, 0x42, 0x6D, 0x62, 0x42, 0x6D, 0x67, 0x42,
+	0x6D, 0x6C, 0x42, 0x6D, 0x6D, 0x42, 0x6D, 0x73,
+	0x42, 0x6E, 0x41, 0x42, 0x6E, 0x46, 0x42, 0x6E,
+	// Bytes 1a40 - 1a7f
+	0x56, 0x42, 0x6E, 0x57, 0x42, 0x6E, 0x6A, 0x42,
+	0x6E, 0x6D, 0x42, 0x6E, 0x73, 0x42, 0x6F, 0x56,
+	0x42, 0x70, 0x41, 0x42, 0x70, 0x46, 0x42, 0x70,
+	0x56, 0x42, 0x70, 0x57, 0x42, 0x70, 0x63, 0x42,
+	0x70, 0x73, 0x42, 0x73, 0x72, 0x42, 0x73, 0x74,
+	0x42, 0x76, 0x69, 0x42, 0x78, 0x69, 0x43, 0x28,
+	0x31, 0x29, 0x43, 0x28, 0x32, 0x29, 0x43, 0x28,
+	0x33, 0x29, 0x43, 0x28, 0x34, 0x29, 0x43, 0x28,
+	// Bytes 1a80 - 1abf
+	0x35, 0x29, 0x43, 0x28, 0x36, 0x29, 0x43, 0x28,
+	0x37, 0x29, 0x43, 0x28, 0x38, 0x29, 0x43, 0x28,
+	0x39, 0x29, 0x43, 0x28, 0x41, 0x29, 0x43, 0x28,
+	0x42, 0x29, 0x43, 0x28, 0x43, 0x29, 0x43, 0x28,
+	0x44, 0x29, 0x43, 0x28, 0x45, 0x29, 0x43, 0x28,
+	0x46, 0x29, 0x43, 0x28, 0x47, 0x29, 0x43, 0x28,
+	0x48, 0x29, 0x43, 0x28, 0x49, 0x29, 0x43, 0x28,
+	0x4A, 0x29, 0x43, 0x28, 0x4B, 0x29, 0x43, 0x28,
+	// Bytes 1ac0 - 1aff
+	0x4C, 0x29, 0x43, 0x28, 0x4D, 0x29, 0x43, 0x28,
+	0x4E, 0x29, 0x43, 0x28, 0x4F, 0x29, 0x43, 0x28,
+	0x50, 0x29, 0x43, 0x28, 0x51, 0x29, 0x43, 0x28,
+	0x52, 0x29, 0x43, 0x28, 0x53, 0x29, 0x43, 0x28,
+	0x54, 0x29, 0x43, 0x28, 0x55, 0x29, 0x43, 0x28,
+	0x56, 0x29, 0x43, 0x28, 0x57, 0x29, 0x43, 0x28,
+	0x58, 0x29, 0x43, 0x28, 0x59, 0x29, 0x43, 0x28,
+	0x5A, 0x29, 0x43, 0x28, 0x61, 0x29, 0x43, 0x28,
+	// Bytes 1b00 - 1b3f
+	0x62, 0x29, 0x43, 0x28, 0x63, 0x29, 0x43, 0x28,
+	0x64, 0x29, 0x43, 0x28, 0x65, 0x29, 0x43, 0x28,
+	0x66, 0x29, 0x43, 0x28, 0x67, 0x29, 0x43, 0x28,
+	0x68, 0x29, 0x43, 0x28, 0x69, 0x29, 0x43, 0x28,
+	0x6A, 0x29, 0x43, 0x28, 0x6B, 0x29, 0x43, 0x28,
+	0x6C, 0x29, 0x43, 0x28, 0x6D, 0x29, 0x43, 0x28,
+	0x6E, 0x29, 0x43, 0x28, 0x6F, 0x29, 0x43, 0x28,
+	0x70, 0x29, 0x43, 0x28, 0x71, 0x29, 0x43, 0x28,
+	// Bytes 1b40 - 1b7f
+	0x72, 0x29, 0x43, 0x28, 0x73, 0x29, 0x43, 0x28,
+	0x74, 0x29, 0x43, 0x28, 0x75, 0x29, 0x43, 0x28,
+	0x76, 0x29, 0x43, 0x28, 0x77, 0x29, 0x43, 0x28,
+	0x78, 0x29, 0x43, 0x28, 0x79, 0x29, 0x43, 0x28,
+	0x7A, 0x29, 0x43, 0x2E, 0x2E, 0x2E, 0x43, 0x31,
+	0x30, 0x2E, 0x43, 0x31, 0x31, 0x2E, 0x43, 0x31,
+	0x32, 0x2E, 0x43, 0x31, 0x33, 0x2E, 0x43, 0x31,
+	0x34, 0x2E, 0x43, 0x31, 0x35, 0x2E, 0x43, 0x31,
+	// Bytes 1b80 - 1bbf
+	0x36, 0x2E, 0x43, 0x31, 0x37, 0x2E, 0x43, 0x31,
+	0x38, 0x2E, 0x43, 0x31, 0x39, 0x2E, 0x43, 0x32,
+	0x30, 0x2E, 0x43, 0x3A, 0x3A, 0x3D, 0x43, 0x3D,
+	0x3D, 0x3D, 0x43, 0x43, 0x6F, 0x2E, 0x43, 0x46,
+	0x41, 0x58, 0x43, 0x47, 0x48, 0x7A, 0x43, 0x47,
+	0x50, 0x61, 0x43, 0x49, 0x49, 0x49, 0x43, 0x4C,
+	0x54, 0x44, 0x43, 0x4C, 0xC2, 0xB7, 0x43, 0x4D,
+	0x48, 0x7A, 0x43, 0x4D, 0x50, 0x61, 0x43, 0x4D,
+	// Bytes 1bc0 - 1bff
+	0xCE, 0xA9, 0x43, 0x50, 0x50, 0x4D, 0x43, 0x50,
+	0x50, 0x56, 0x43, 0x50, 0x54, 0x45, 0x43, 0x54,
+	0x45, 0x4C, 0x43, 0x54, 0x48, 0x7A, 0x43, 0x56,
+	0x49, 0x49, 0x43, 0x58, 0x49, 0x49, 0x43, 0x61,
+	0x2F, 0x63, 0x43, 0x61, 0x2F, 0x73, 0x43, 0x61,
+	0xCA, 0xBE, 0x43, 0x62, 0x61, 0x72, 0x43, 0x63,
+	0x2F, 0x6F, 0x43, 0x63, 0x2F, 0x75, 0x43, 0x63,
+	0x61, 0x6C, 0x43, 0x63, 0x6D, 0x32, 0x43, 0x63,
+	// Bytes 1c00 - 1c3f
+	0x6D, 0x33, 0x43, 0x64, 0x6D, 0x32, 0x43, 0x64,
+	0x6D, 0x33, 0x43, 0x65, 0x72, 0x67, 0x43, 0x66,
+	0x66, 0x69, 0x43, 0x66, 0x66, 0x6C, 0x43, 0x67,
+	0x61, 0x6C, 0x43, 0x68, 0x50, 0x61, 0x43, 0x69,
+	0x69, 0x69, 0x43, 0x6B, 0x48, 0x7A, 0x43, 0x6B,
+	0x50, 0x61, 0x43, 0x6B, 0x6D, 0x32, 0x43, 0x6B,
+	0x6D, 0x33, 0x43, 0x6B, 0xCE, 0xA9, 0x43, 0x6C,
+	0x6F, 0x67, 0x43, 0x6C, 0xC2, 0xB7, 0x43, 0x6D,
+	// Bytes 1c40 - 1c7f
+	0x69, 0x6C, 0x43, 0x6D, 0x6D, 0x32, 0x43, 0x6D,
+	0x6D, 0x33, 0x43, 0x6D, 0x6F, 0x6C, 0x43, 0x72,
+	0x61, 0x64, 0x43, 0x76, 0x69, 0x69, 0x43, 0x78,
+	0x69, 0x69, 0x43, 0xC2, 0xB0, 0x43, 0x43, 0xC2,
+	0xB0, 0x46, 0x43, 0xCA, 0xBC, 0x6E, 0x43, 0xCE,
+	0xBC, 0x41, 0x43, 0xCE, 0xBC, 0x46, 0x43, 0xCE,
+	0xBC, 0x56, 0x43, 0xCE, 0xBC, 0x57, 0x43, 0xCE,
+	0xBC, 0x67, 0x43, 0xCE, 0xBC, 0x6C, 0x43, 0xCE,
+	// Bytes 1c80 - 1cbf
+	0xBC, 0x6D, 0x43, 0xCE, 0xBC, 0x73, 0x44, 0x28,
+	0x31, 0x30, 0x29, 0x44, 0x28, 0x31, 0x31, 0x29,
+	0x44, 0x28, 0x31, 0x32, 0x29, 0x44, 0x28, 0x31,
+	0x33, 0x29, 0x44, 0x28, 0x31, 0x34, 0x29, 0x44,
+	0x28, 0x31, 0x35, 0x29, 0x44, 0x28, 0x31, 0x36,
+	0x29, 0x44, 0x28, 0x31, 0x37, 0x29, 0x44, 0x28,
+	0x31, 0x38, 0x29, 0x44, 0x28, 0x31, 0x39, 0x29,
+	0x44, 0x28, 0x32, 0x30, 0x29, 0x44, 0x30, 0xE7,
+	// Bytes 1cc0 - 1cff
+	0x82, 0xB9, 0x44, 0x31, 0xE2, 0x81, 0x84, 0x44,
+	0x31, 0xE6, 0x97, 0xA5, 0x44, 0x31, 0xE6, 0x9C,
+	0x88, 0x44, 0x31, 0xE7, 0x82, 0xB9, 0x44, 0x32,
+	0xE6, 0x97, 0xA5, 0x44, 0x32, 0xE6, 0x9C, 0x88,
+	0x44, 0x32, 0xE7, 0x82, 0xB9, 0x44, 0x33, 0xE6,
+	0x97, 0xA5, 0x44, 0x33, 0xE6, 0x9C, 0x88, 0x44,
+	0x33, 0xE7, 0x82, 0xB9, 0x44, 0x34, 0xE6, 0x97,
+	0xA5, 0x44, 0x34, 0xE6, 0x9C, 0x88, 0x44, 0x34,
+	// Bytes 1d00 - 1d3f
+	0xE7, 0x82, 0xB9, 0x44, 0x35, 0xE6, 0x97, 0xA5,
+	0x44, 0x35, 0xE6, 0x9C, 0x88, 0x44, 0x35, 0xE7,
+	0x82, 0xB9, 0x44, 0x36, 0xE6, 0x97, 0xA5, 0x44,
+	0x36, 0xE6, 0x9C, 0x88, 0x44, 0x36, 0xE7, 0x82,
+	0xB9, 0x44, 0x37, 0xE6, 0x97, 0xA5, 0x44, 0x37,
+	0xE6, 0x9C, 0x88, 0x44, 0x37, 0xE7, 0x82, 0xB9,
+	0x44, 0x38, 0xE6, 0x97, 0xA5, 0x44, 0x38, 0xE6,
+	0x9C, 0x88, 0x44, 0x38, 0xE7, 0x82, 0xB9, 0x44,
+	// Bytes 1d40 - 1d7f
+	0x39, 0xE6, 0x97, 0xA5, 0x44, 0x39, 0xE6, 0x9C,
+	0x88, 0x44, 0x39, 0xE7, 0x82, 0xB9, 0x44, 0x56,
+	0x49, 0x49, 0x49, 0x44, 0x61, 0x2E, 0x6D, 0x2E,
+	0x44, 0x6B, 0x63, 0x61, 0x6C, 0x44, 0x70, 0x2E,
+	0x6D, 0x2E, 0x44, 0x76, 0x69, 0x69, 0x69, 0x44,
+	0xD5, 0xA5, 0xD6, 0x82, 0x44, 0xD5, 0xB4, 0xD5,
+	0xA5, 0x44, 0xD5, 0xB4, 0xD5, 0xAB, 0x44, 0xD5,
+	0xB4, 0xD5, 0xAD, 0x44, 0xD5, 0xB4, 0xD5, 0xB6,
+	// Bytes 1d80 - 1dbf
+	0x44, 0xD5, 0xBE, 0xD5, 0xB6, 0x44, 0xD7, 0x90,
+	0xD7, 0x9C, 0x44, 0xD8, 0xA7, 0xD9, 0xB4, 0x44,
+	0xD8, 0xA8, 0xD8, 0xAC, 0x44, 0xD8, 0xA8, 0xD8,
+	0xAD, 0x44, 0xD8, 0xA8, 0xD8, 0xAE, 0x44, 0xD8,
+	0xA8, 0xD8, 0xB1, 0x44, 0xD8, 0xA8, 0xD8, 0xB2,
+	0x44, 0xD8, 0xA8, 0xD9, 0x85, 0x44, 0xD8, 0xA8,
+	0xD9, 0x86, 0x44, 0xD8, 0xA8, 0xD9, 0x87, 0x44,
+	0xD8, 0xA8, 0xD9, 0x89, 0x44, 0xD8, 0xA8, 0xD9,
+	// Bytes 1dc0 - 1dff
+	0x8A, 0x44, 0xD8, 0xAA, 0xD8, 0xAC, 0x44, 0xD8,
+	0xAA, 0xD8, 0xAD, 0x44, 0xD8, 0xAA, 0xD8, 0xAE,
+	0x44, 0xD8, 0xAA, 0xD8, 0xB1, 0x44, 0xD8, 0xAA,
+	0xD8, 0xB2, 0x44, 0xD8, 0xAA, 0xD9, 0x85, 0x44,
+	0xD8, 0xAA, 0xD9, 0x86, 0x44, 0xD8, 0xAA, 0xD9,
+	0x87, 0x44, 0xD8, 0xAA, 0xD9, 0x89, 0x44, 0xD8,
+	0xAA, 0xD9, 0x8A, 0x44, 0xD8, 0xAB, 0xD8, 0xAC,
+	0x44, 0xD8, 0xAB, 0xD8, 0xB1, 0x44, 0xD8, 0xAB,
+	// Bytes 1e00 - 1e3f
+	0xD8, 0xB2, 0x44, 0xD8, 0xAB, 0xD9, 0x85, 0x44,
+	0xD8, 0xAB, 0xD9, 0x86, 0x44, 0xD8, 0xAB, 0xD9,
+	0x87, 0x44, 0xD8, 0xAB, 0xD9, 0x89, 0x44, 0xD8,
+	0xAB, 0xD9, 0x8A, 0x44, 0xD8, 0xAC, 0xD8, 0xAD,
+	0x44, 0xD8, 0xAC, 0xD9, 0x85, 0x44, 0xD8, 0xAC,
+	0xD9, 0x89, 0x44, 0xD8, 0xAC, 0xD9, 0x8A, 0x44,
+	0xD8, 0xAD, 0xD8, 0xAC, 0x44, 0xD8, 0xAD, 0xD9,
+	0x85, 0x44, 0xD8, 0xAD, 0xD9, 0x89, 0x44, 0xD8,
+	// Bytes 1e40 - 1e7f
+	0xAD, 0xD9, 0x8A, 0x44, 0xD8, 0xAE, 0xD8, 0xAC,
+	0x44, 0xD8, 0xAE, 0xD8, 0xAD, 0x44, 0xD8, 0xAE,
+	0xD9, 0x85, 0x44, 0xD8, 0xAE, 0xD9, 0x89, 0x44,
+	0xD8, 0xAE, 0xD9, 0x8A, 0x44, 0xD8, 0xB3, 0xD8,
+	0xAC, 0x44, 0xD8, 0xB3, 0xD8, 0xAD, 0x44, 0xD8,
+	0xB3, 0xD8, 0xAE, 0x44, 0xD8, 0xB3, 0xD8, 0xB1,
+	0x44, 0xD8, 0xB3, 0xD9, 0x85, 0x44, 0xD8, 0xB3,
+	0xD9, 0x87, 0x44, 0xD8, 0xB3, 0xD9, 0x89, 0x44,
+	// Bytes 1e80 - 1ebf
+	0xD8, 0xB3, 0xD9, 0x8A, 0x44, 0xD8, 0xB4, 0xD8,
+	0xAC, 0x44, 0xD8, 0xB4, 0xD8, 0xAD, 0x44, 0xD8,
+	0xB4, 0xD8, 0xAE, 0x44, 0xD8, 0xB4, 0xD8, 0xB1,
+	0x44, 0xD8, 0xB4, 0xD9, 0x85, 0x44, 0xD8, 0xB4,
+	0xD9, 0x87, 0x44, 0xD8, 0xB4, 0xD9, 0x89, 0x44,
+	0xD8, 0xB4, 0xD9, 0x8A, 0x44, 0xD8, 0xB5, 0xD8,
+	0xAD, 0x44, 0xD8, 0xB5, 0xD8, 0xAE, 0x44, 0xD8,
+	0xB5, 0xD8, 0xB1, 0x44, 0xD8, 0xB5, 0xD9, 0x85,
+	// Bytes 1ec0 - 1eff
+	0x44, 0xD8, 0xB5, 0xD9, 0x89, 0x44, 0xD8, 0xB5,
+	0xD9, 0x8A, 0x44, 0xD8, 0xB6, 0xD8, 0xAC, 0x44,
+	0xD8, 0xB6, 0xD8, 0xAD, 0x44, 0xD8, 0xB6, 0xD8,
+	0xAE, 0x44, 0xD8, 0xB6, 0xD8, 0xB1, 0x44, 0xD8,
+	0xB6, 0xD9, 0x85, 0x44, 0xD8, 0xB6, 0xD9, 0x89,
+	0x44, 0xD8, 0xB6, 0xD9, 0x8A, 0x44, 0xD8, 0xB7,
+	0xD8, 0xAD, 0x44, 0xD8, 0xB7, 0xD9, 0x85, 0x44,
+	0xD8, 0xB7, 0xD9, 0x89, 0x44, 0xD8, 0xB7, 0xD9,
+	// Bytes 1f00 - 1f3f
+	0x8A, 0x44, 0xD8, 0xB8, 0xD9, 0x85, 0x44, 0xD8,
+	0xB9, 0xD8, 0xAC, 0x44, 0xD8, 0xB9, 0xD9, 0x85,
+	0x44, 0xD8, 0xB9, 0xD9, 0x89, 0x44, 0xD8, 0xB9,
+	0xD9, 0x8A, 0x44, 0xD8, 0xBA, 0xD8, 0xAC, 0x44,
+	0xD8, 0xBA, 0xD9, 0x85, 0x44, 0xD8, 0xBA, 0xD9,
+	0x89, 0x44, 0xD8, 0xBA, 0xD9, 0x8A, 0x44, 0xD9,
+	0x81, 0xD8, 0xAC, 0x44, 0xD9, 0x81, 0xD8, 0xAD,
+	0x44, 0xD9, 0x81, 0xD8, 0xAE, 0x44, 0xD9, 0x81,
+	// Bytes 1f40 - 1f7f
+	0xD9, 0x85, 0x44, 0xD9, 0x81, 0xD9, 0x89, 0x44,
+	0xD9, 0x81, 0xD9, 0x8A, 0x44, 0xD9, 0x82, 0xD8,
+	0xAD, 0x44, 0xD9, 0x82, 0xD9, 0x85, 0x44, 0xD9,
+	0x82, 0xD9, 0x89, 0x44, 0xD9, 0x82, 0xD9, 0x8A,
+	0x44, 0xD9, 0x83, 0xD8, 0xA7, 0x44, 0xD9, 0x83,
+	0xD8, 0xAC, 0x44, 0xD9, 0x83, 0xD8, 0xAD, 0x44,
+	0xD9, 0x83, 0xD8, 0xAE, 0x44, 0xD9, 0x83, 0xD9,
+	0x84, 0x44, 0xD9, 0x83, 0xD9, 0x85, 0x44, 0xD9,
+	// Bytes 1f80 - 1fbf
+	0x83, 0xD9, 0x89, 0x44, 0xD9, 0x83, 0xD9, 0x8A,
+	0x44, 0xD9, 0x84, 0xD8, 0xA7, 0x44, 0xD9, 0x84,
+	0xD8, 0xAC, 0x44, 0xD9, 0x84, 0xD8, 0xAD, 0x44,
+	0xD9, 0x84, 0xD8, 0xAE, 0x44, 0xD9, 0x84, 0xD9,
+	0x85, 0x44, 0xD9, 0x84, 0xD9, 0x87, 0x44, 0xD9,
+	0x84, 0xD9, 0x89, 0x44, 0xD9, 0x84, 0xD9, 0x8A,
+	0x44, 0xD9, 0x85, 0xD8, 0xA7, 0x44, 0xD9, 0x85,
+	0xD8, 0xAC, 0x44, 0xD9, 0x85, 0xD8, 0xAD, 0x44,
+	// Bytes 1fc0 - 1fff
+	0xD9, 0x85, 0xD8, 0xAE, 0x44, 0xD9, 0x85, 0xD9,
+	0x85, 0x44, 0xD9, 0x85, 0xD9, 0x89, 0x44, 0xD9,
+	0x85, 0xD9, 0x8A, 0x44, 0xD9, 0x86, 0xD8, 0xAC,
+	0x44, 0xD9, 0x86, 0xD8, 0xAD, 0x44, 0xD9, 0x86,
+	0xD8, 0xAE, 0x44, 0xD9, 0x86, 0xD8, 0xB1, 0x44,
+	0xD9, 0x86, 0xD8, 0xB2, 0x44, 0xD9, 0x86, 0xD9,
+	0x85, 0x44, 0xD9, 0x86, 0xD9, 0x86, 0x44, 0xD9,
+	0x86, 0xD9, 0x87, 0x44, 0xD9, 0x86, 0xD9, 0x89,
+	// Bytes 2000 - 203f
+	0x44, 0xD9, 0x86, 0xD9, 0x8A, 0x44, 0xD9, 0x87,
+	0xD8, 0xAC, 0x44, 0xD9, 0x87, 0xD9, 0x85, 0x44,
+	0xD9, 0x87, 0xD9, 0x89, 0x44, 0xD9, 0x87, 0xD9,
+	0x8A, 0x44, 0xD9, 0x88, 0xD9, 0xB4, 0x44, 0xD9,
+	0x8A, 0xD8, 0xAC, 0x44, 0xD9, 0x8A, 0xD8, 0xAD,
+	0x44, 0xD9, 0x8A, 0xD8, 0xAE, 0x44, 0xD9, 0x8A,
+	0xD8, 0xB1, 0x44, 0xD9, 0x8A, 0xD8, 0xB2, 0x44,
+	0xD9, 0x8A, 0xD9, 0x85, 0x44, 0xD9, 0x8A, 0xD9,
+	// Bytes 2040 - 207f
+	0x86, 0x44, 0xD9, 0x8A, 0xD9, 0x87, 0x44, 0xD9,
+	0x8A, 0xD9, 0x89, 0x44, 0xD9, 0x8A, 0xD9, 0x8A,
+	0x44, 0xD9, 0x8A, 0xD9, 0xB4, 0x44, 0xDB, 0x87,
+	0xD9, 0xB4, 0x45, 0x28, 0xE1, 0x84, 0x80, 0x29,
+	0x45, 0x28, 0xE1, 0x84, 0x82, 0x29, 0x45, 0x28,
+	0xE1, 0x84, 0x83, 0x29, 0x45, 0x28, 0xE1, 0x84,
+	0x85, 0x29, 0x45, 0x28, 0xE1, 0x84, 0x86, 0x29,
+	0x45, 0x28, 0xE1, 0x84, 0x87, 0x29, 0x45, 0x28,
+	// Bytes 2080 - 20bf
+	0xE1, 0x84, 0x89, 0x29, 0x45, 0x28, 0xE1, 0x84,
+	0x8B, 0x29, 0x45, 0x28, 0xE1, 0x84, 0x8C, 0x29,
+	0x45, 0x28, 0xE1, 0x84, 0x8E, 0x29, 0x45, 0x28,
+	0xE1, 0x84, 0x8F, 0x29, 0x45, 0x28, 0xE1, 0x84,
+	0x90, 0x29, 0x45, 0x28, 0xE1, 0x84, 0x91, 0x29,
+	0x45, 0x28, 0xE1, 0x84, 0x92, 0x29, 0x45, 0x28,
+	0xE4, 0xB8, 0x80, 0x29, 0x45, 0x28, 0xE4, 0xB8,
+	0x83, 0x29, 0x45, 0x28, 0xE4, 0xB8, 0x89, 0x29,
+	// Bytes 20c0 - 20ff
+	0x45, 0x28, 0xE4, 0xB9, 0x9D, 0x29, 0x45, 0x28,
+	0xE4, 0xBA, 0x8C, 0x29, 0x45, 0x28, 0xE4, 0xBA,
+	0x94, 0x29, 0x45, 0x28, 0xE4, 0xBB, 0xA3, 0x29,
+	0x45, 0x28, 0xE4, 0xBC, 0x81, 0x29, 0x45, 0x28,
+	0xE4, 0xBC, 0x91, 0x29, 0x45, 0x28, 0xE5, 0x85,
+	0xAB, 0x29, 0x45, 0x28, 0xE5, 0x85, 0xAD, 0x29,
+	0x45, 0x28, 0xE5, 0x8A, 0xB4, 0x29, 0x45, 0x28,
+	0xE5, 0x8D, 0x81, 0x29, 0x45, 0x28, 0xE5, 0x8D,
+	// Bytes 2100 - 213f
+	0x94, 0x29, 0x45, 0x28, 0xE5, 0x90, 0x8D, 0x29,
+	0x45, 0x28, 0xE5, 0x91, 0xBC, 0x29, 0x45, 0x28,
+	0xE5, 0x9B, 0x9B, 0x29, 0x45, 0x28, 0xE5, 0x9C,
+	0x9F, 0x29, 0x45, 0x28, 0xE5, 0xAD, 0xA6, 0x29,
+	0x45, 0x28, 0xE6, 0x97, 0xA5, 0x29, 0x45, 0x28,
+	0xE6, 0x9C, 0x88, 0x29, 0x45, 0x28, 0xE6, 0x9C,
+	0x89, 0x29, 0x45, 0x28, 0xE6, 0x9C, 0xA8, 0x29,
+	0x45, 0x28, 0xE6, 0xA0, 0xAA, 0x29, 0x45, 0x28,
+	// Bytes 2140 - 217f
+	0xE6, 0xB0, 0xB4, 0x29, 0x45, 0x28, 0xE7, 0x81,
+	0xAB, 0x29, 0x45, 0x28, 0xE7, 0x89, 0xB9, 0x29,
+	0x45, 0x28, 0xE7, 0x9B, 0xA3, 0x29, 0x45, 0x28,
+	0xE7, 0xA4, 0xBE, 0x29, 0x45, 0x28, 0xE7, 0xA5,
+	0x9D, 0x29, 0x45, 0x28, 0xE7, 0xA5, 0xAD, 0x29,
+	0x45, 0x28, 0xE8, 0x87, 0xAA, 0x29, 0x45, 0x28,
+	0xE8, 0x87, 0xB3, 0x29, 0x45, 0x28, 0xE8, 0xB2,
+	0xA1, 0x29, 0x45, 0x28, 0xE8, 0xB3, 0x87, 0x29,
+	// Bytes 2180 - 21bf
+	0x45, 0x28, 0xE9, 0x87, 0x91, 0x29, 0x45, 0x30,
+	0xE2, 0x81, 0x84, 0x33, 0x45, 0x31, 0x30, 0xE6,
+	0x97, 0xA5, 0x45, 0x31, 0x30, 0xE6, 0x9C, 0x88,
+	0x45, 0x31, 0x30, 0xE7, 0x82, 0xB9, 0x45, 0x31,
+	0x31, 0xE6, 0x97, 0xA5, 0x45, 0x31, 0x31, 0xE6,
+	0x9C, 0x88, 0x45, 0x31, 0x31, 0xE7, 0x82, 0xB9,
+	0x45, 0x31, 0x32, 0xE6, 0x97, 0xA5, 0x45, 0x31,
+	0x32, 0xE6, 0x9C, 0x88, 0x45, 0x31, 0x32, 0xE7,
+	// Bytes 21c0 - 21ff
+	0x82, 0xB9, 0x45, 0x31, 0x33, 0xE6, 0x97, 0xA5,
+	0x45, 0x31, 0x33, 0xE7, 0x82, 0xB9, 0x45, 0x31,
+	0x34, 0xE6, 0x97, 0xA5, 0x45, 0x31, 0x34, 0xE7,
+	0x82, 0xB9, 0x45, 0x31, 0x35, 0xE6, 0x97, 0xA5,
+	0x45, 0x31, 0x35, 0xE7, 0x82, 0xB9, 0x45, 0x31,
+	0x36, 0xE6, 0x97, 0xA5, 0x45, 0x31, 0x36, 0xE7,
+	0x82, 0xB9, 0x45, 0x31, 0x37, 0xE6, 0x97, 0xA5,
+	0x45, 0x31, 0x37, 0xE7, 0x82, 0xB9, 0x45, 0x31,
+	// Bytes 2200 - 223f
+	0x38, 0xE6, 0x97, 0xA5, 0x45, 0x31, 0x38, 0xE7,
+	0x82, 0xB9, 0x45, 0x31, 0x39, 0xE6, 0x97, 0xA5,
+	0x45, 0x31, 0x39, 0xE7, 0x82, 0xB9, 0x45, 0x31,
+	0xE2, 0x81, 0x84, 0x32, 0x45, 0x31, 0xE2, 0x81,
+	0x84, 0x33, 0x45, 0x31, 0xE2, 0x81, 0x84, 0x34,
+	0x45, 0x31, 0xE2, 0x81, 0x84, 0x35, 0x45, 0x31,
+	0xE2, 0x81, 0x84, 0x36, 0x45, 0x31, 0xE2, 0x81,
+	0x84, 0x37, 0x45, 0x31, 0xE2, 0x81, 0x84, 0x38,
+	// Bytes 2240 - 227f
+	0x45, 0x31, 0xE2, 0x81, 0x84, 0x39, 0x45, 0x32,
+	0x30, 0xE6, 0x97, 0xA5, 0x45, 0x32, 0x30, 0xE7,
+	0x82, 0xB9, 0x45, 0x32, 0x31, 0xE6, 0x97, 0xA5,
+	0x45, 0x32, 0x31, 0xE7, 0x82, 0xB9, 0x45, 0x32,
+	0x32, 0xE6, 0x97, 0xA5, 0x45, 0x32, 0x32, 0xE7,
+	0x82, 0xB9, 0x45, 0x32, 0x33, 0xE6, 0x97, 0xA5,
+	0x45, 0x32, 0x33, 0xE7, 0x82, 0xB9, 0x45, 0x32,
+	0x34, 0xE6, 0x97, 0xA5, 0x45, 0x32, 0x34, 0xE7,
+	// Bytes 2280 - 22bf
+	0x82, 0xB9, 0x45, 0x32, 0x35, 0xE6, 0x97, 0xA5,
+	0x45, 0x32, 0x36, 0xE6, 0x97, 0xA5, 0x45, 0x32,
+	0x37, 0xE6, 0x97, 0xA5, 0x45, 0x32, 0x38, 0xE6,
+	0x97, 0xA5, 0x45, 0x32, 0x39, 0xE6, 0x97, 0xA5,
+	0x45, 0x32, 0xE2, 0x81, 0x84, 0x33, 0x45, 0x32,
+	0xE2, 0x81, 0x84, 0x35, 0x45, 0x33, 0x30, 0xE6,
+	0x97, 0xA5, 0x45, 0x33, 0x31, 0xE6, 0x97, 0xA5,
+	0x45, 0x33, 0xE2, 0x81, 0x84, 0x34, 0x45, 0x33,
+	// Bytes 22c0 - 22ff
+	0xE2, 0x81, 0x84, 0x35, 0x45, 0x33, 0xE2, 0x81,
+	0x84, 0x38, 0x45, 0x34, 0xE2, 0x81, 0x84, 0x35,
+	0x45, 0x35, 0xE2, 0x81, 0x84, 0x36, 0x45, 0x35,
+	0xE2, 0x81, 0x84, 0x38, 0x45, 0x37, 0xE2, 0x81,
+	0x84, 0x38, 0x45, 0x41, 0xE2, 0x88, 0x95, 0x6D,
+	0x45, 0x56, 0xE2, 0x88, 0x95, 0x6D, 0x45, 0x6D,
+	0xE2, 0x88, 0x95, 0x73, 0x46, 0x31, 0xE2, 0x81,
+	0x84, 0x31, 0x30, 0x46, 0x43, 0xE2, 0x88, 0x95,
+	// Bytes 2300 - 233f
+	0x6B, 0x67, 0x46, 0x6D, 0xE2, 0x88, 0x95, 0x73,
+	0x32, 0x46, 0xD8, 0xA8, 0xD8, 0xAD, 0xD9, 0x8A,
+	0x46, 0xD8, 0xA8, 0xD8, 0xAE, 0xD9, 0x8A, 0x46,
+	0xD8, 0xAA, 0xD8, 0xAC, 0xD9, 0x85, 0x46, 0xD8,
+	0xAA, 0xD8, 0xAC, 0xD9, 0x89, 0x46, 0xD8, 0xAA,
+	0xD8, 0xAC, 0xD9, 0x8A, 0x46, 0xD8, 0xAA, 0xD8,
+	0xAD, 0xD8, 0xAC, 0x46, 0xD8, 0xAA, 0xD8, 0xAD,
+	0xD9, 0x85, 0x46, 0xD8, 0xAA, 0xD8, 0xAE, 0xD9,
+	// Bytes 2340 - 237f
+	0x85, 0x46, 0xD8, 0xAA, 0xD8, 0xAE, 0xD9, 0x89,
+	0x46, 0xD8, 0xAA, 0xD8, 0xAE, 0xD9, 0x8A, 0x46,
+	0xD8, 0xAA, 0xD9, 0x85, 0xD8, 0xAC, 0x46, 0xD8,
+	0xAA, 0xD9, 0x85, 0xD8, 0xAD, 0x46, 0xD8, 0xAA,
+	0xD9, 0x85, 0xD8, 0xAE, 0x46, 0xD8, 0xAA, 0xD9,
+	0x85, 0xD9, 0x89, 0x46, 0xD8, 0xAA, 0xD9, 0x85,
+	0xD9, 0x8A, 0x46, 0xD8, 0xAC, 0xD8, 0xAD, 0xD9,
+	0x89, 0x46, 0xD8, 0xAC, 0xD8, 0xAD, 0xD9, 0x8A,
+	// Bytes 2380 - 23bf
+	0x46, 0xD8, 0xAC, 0xD9, 0x85, 0xD8, 0xAD, 0x46,
+	0xD8, 0xAC, 0xD9, 0x85, 0xD9, 0x89, 0x46, 0xD8,
+	0xAC, 0xD9, 0x85, 0xD9, 0x8A, 0x46, 0xD8, 0xAD,
+	0xD8, 0xAC, 0xD9, 0x8A, 0x46, 0xD8, 0xAD, 0xD9,
+	0x85, 0xD9, 0x89, 0x46, 0xD8, 0xAD, 0xD9, 0x85,
+	0xD9, 0x8A, 0x46, 0xD8, 0xB3, 0xD8, 0xAC, 0xD8,
+	0xAD, 0x46, 0xD8, 0xB3, 0xD8, 0xAC, 0xD9, 0x89,
+	0x46, 0xD8, 0xB3, 0xD8, 0xAD, 0xD8, 0xAC, 0x46,
+	// Bytes 23c0 - 23ff
+	0xD8, 0xB3, 0xD8, 0xAE, 0xD9, 0x89, 0x46, 0xD8,
+	0xB3, 0xD8, 0xAE, 0xD9, 0x8A, 0x46, 0xD8, 0xB3,
+	0xD9, 0x85, 0xD8, 0xAC, 0x46, 0xD8, 0xB3, 0xD9,
+	0x85, 0xD8, 0xAD, 0x46, 0xD8, 0xB3, 0xD9, 0x85,
+	0xD9, 0x85, 0x46, 0xD8, 0xB4, 0xD8, 0xAC, 0xD9,
+	0x8A, 0x46, 0xD8, 0xB4, 0xD8, 0xAD, 0xD9, 0x85,
+	0x46, 0xD8, 0xB4, 0xD8, 0xAD, 0xD9, 0x8A, 0x46,
+	0xD8, 0xB4, 0xD9, 0x85, 0xD8, 0xAE, 0x46, 0xD8,
+	// Bytes 2400 - 243f
+	0xB4, 0xD9, 0x85, 0xD9, 0x85, 0x46, 0xD8, 0xB5,
+	0xD8, 0xAD, 0xD8, 0xAD, 0x46, 0xD8, 0xB5, 0xD8,
+	0xAD, 0xD9, 0x8A, 0x46, 0xD8, 0xB5, 0xD9, 0x84,
+	0xD9, 0x89, 0x46, 0xD8, 0xB5, 0xD9, 0x84, 0xDB,
+	0x92, 0x46, 0xD8, 0xB5, 0xD9, 0x85, 0xD9, 0x85,
+	0x46, 0xD8, 0xB6, 0xD8, 0xAD, 0xD9, 0x89, 0x46,
+	0xD8, 0xB6, 0xD8, 0xAD, 0xD9, 0x8A, 0x46, 0xD8,
+	0xB6, 0xD8, 0xAE, 0xD9, 0x85, 0x46, 0xD8, 0xB7,
+	// Bytes 2440 - 247f
+	0xD9, 0x85, 0xD8, 0xAD, 0x46, 0xD8, 0xB7, 0xD9,
+	0x85, 0xD9, 0x85, 0x46, 0xD8, 0xB7, 0xD9, 0x85,
+	0xD9, 0x8A, 0x46, 0xD8, 0xB9, 0xD8, 0xAC, 0xD9,
+	0x85, 0x46, 0xD8, 0xB9, 0xD9, 0x85, 0xD9, 0x85,
+	0x46, 0xD8, 0xB9, 0xD9, 0x85, 0xD9, 0x89, 0x46,
+	0xD8, 0xB9, 0xD9, 0x85, 0xD9, 0x8A, 0x46, 0xD8,
+	0xBA, 0xD9, 0x85, 0xD9, 0x85, 0x46, 0xD8, 0xBA,
+	0xD9, 0x85, 0xD9, 0x89, 0x46, 0xD8, 0xBA, 0xD9,
+	// Bytes 2480 - 24bf
+	0x85, 0xD9, 0x8A, 0x46, 0xD9, 0x81, 0xD8, 0xAE,
+	0xD9, 0x85, 0x46, 0xD9, 0x81, 0xD9, 0x85, 0xD9,
+	0x8A, 0x46, 0xD9, 0x82, 0xD9, 0x84, 0xDB, 0x92,
+	0x46, 0xD9, 0x82, 0xD9, 0x85, 0xD8, 0xAD, 0x46,
+	0xD9, 0x82, 0xD9, 0x85, 0xD9, 0x85, 0x46, 0xD9,
+	0x82, 0xD9, 0x85, 0xD9, 0x8A, 0x46, 0xD9, 0x83,
+	0xD9, 0x85, 0xD9, 0x85, 0x46, 0xD9, 0x83, 0xD9,
+	0x85, 0xD9, 0x8A, 0x46, 0xD9, 0x84, 0xD8, 0xAC,
+	// Bytes 24c0 - 24ff
+	0xD8, 0xAC, 0x46, 0xD9, 0x84, 0xD8, 0xAC, 0xD9,
+	0x85, 0x46, 0xD9, 0x84, 0xD8, 0xAC, 0xD9, 0x8A,
+	0x46, 0xD9, 0x84, 0xD8, 0xAD, 0xD9, 0x85, 0x46,
+	0xD9, 0x84, 0xD8, 0xAD, 0xD9, 0x89, 0x46, 0xD9,
+	0x84, 0xD8, 0xAD, 0xD9, 0x8A, 0x46, 0xD9, 0x84,
+	0xD8, 0xAE, 0xD9, 0x85, 0x46, 0xD9, 0x84, 0xD9,
+	0x85, 0xD8, 0xAD, 0x46, 0xD9, 0x84, 0xD9, 0x85,
+	0xD9, 0x8A, 0x46, 0xD9, 0x85, 0xD8, 0xAC, 0xD8,
+	// Bytes 2500 - 253f
+	0xAD, 0x46, 0xD9, 0x85, 0xD8, 0xAC, 0xD8, 0xAE,
+	0x46, 0xD9, 0x85, 0xD8, 0xAC, 0xD9, 0x85, 0x46,
+	0xD9, 0x85, 0xD8, 0xAC, 0xD9, 0x8A, 0x46, 0xD9,
+	0x85, 0xD8, 0xAD, 0xD8, 0xAC, 0x46, 0xD9, 0x85,
+	0xD8, 0xAD, 0xD9, 0x85, 0x46, 0xD9, 0x85, 0xD8,
+	0xAD, 0xD9, 0x8A, 0x46, 0xD9, 0x85, 0xD8, 0xAE,
+	0xD8, 0xAC, 0x46, 0xD9, 0x85, 0xD8, 0xAE, 0xD9,
+	0x85, 0x46, 0xD9, 0x85, 0xD8, 0xAE, 0xD9, 0x8A,
+	// Bytes 2540 - 257f
+	0x46, 0xD9, 0x85, 0xD9, 0x85, 0xD9, 0x8A, 0x46,
+	0xD9, 0x86, 0xD8, 0xAC, 0xD8, 0xAD, 0x46, 0xD9,
+	0x86, 0xD8, 0xAC, 0xD9, 0x85, 0x46, 0xD9, 0x86,
+	0xD8, 0xAC, 0xD9, 0x89, 0x46, 0xD9, 0x86, 0xD8,
+	0xAC, 0xD9, 0x8A, 0x46, 0xD9, 0x86, 0xD8, 0xAD,
+	0xD9, 0x85, 0x46, 0xD9, 0x86, 0xD8, 0xAD, 0xD9,
+	0x89, 0x46, 0xD9, 0x86, 0xD8, 0xAD, 0xD9, 0x8A,
+	0x46, 0xD9, 0x86, 0xD9, 0x85, 0xD9, 0x89, 0x46,
+	// Bytes 2580 - 25bf
+	0xD9, 0x86, 0xD9, 0x85, 0xD9, 0x8A, 0x46, 0xD9,
+	0x87, 0xD9, 0x85, 0xD8, 0xAC, 0x46, 0xD9, 0x87,
+	0xD9, 0x85, 0xD9, 0x85, 0x46, 0xD9, 0x8A, 0xD8,
+	0xAC, 0xD9, 0x8A, 0x46, 0xD9, 0x8A, 0xD8, 0xAD,
+	0xD9, 0x8A, 0x46, 0xD9, 0x8A, 0xD9, 0x85, 0xD9,
+	0x85, 0x46, 0xD9, 0x8A, 0xD9, 0x85, 0xD9, 0x8A,
+	0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xD8, 0xA7, 0x46,
+	0xD9, 0x8A, 0xD9, 0x94, 0xD8, 0xAC, 0x46, 0xD9,
+	// Bytes 25c0 - 25ff
+	0x8A, 0xD9, 0x94, 0xD8, 0xAD, 0x46, 0xD9, 0x8A,
+	0xD9, 0x94, 0xD8, 0xAE, 0x46, 0xD9, 0x8A, 0xD9,
+	0x94, 0xD8, 0xB1, 0x46, 0xD9, 0x8A, 0xD9, 0x94,
+	0xD8, 0xB2, 0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xD9,
+	0x85, 0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xD9, 0x86,
+	0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xD9, 0x87, 0x46,
+	0xD9, 0x8A, 0xD9, 0x94, 0xD9, 0x88, 0x46, 0xD9,
+	0x8A, 0xD9, 0x94, 0xD9, 0x89, 0x46, 0xD9, 0x8A,
+	// Bytes 2600 - 263f
+	0xD9, 0x94, 0xD9, 0x8A, 0x46, 0xD9, 0x8A, 0xD9,
+	0x94, 0xDB, 0x86, 0x46, 0xD9, 0x8A, 0xD9, 0x94,
+	0xDB, 0x87, 0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xDB,
+	0x88, 0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xDB, 0x90,
+	0x46, 0xD9, 0x8A, 0xD9, 0x94, 0xDB, 0x95, 0x46,
+	0xE0, 0xB9, 0x8D, 0xE0, 0xB8, 0xB2, 0x46, 0xE0,
+	0xBA, 0xAB, 0xE0, 0xBA, 0x99, 0x46, 0xE0, 0xBA,
+	0xAB, 0xE0, 0xBA, 0xA1, 0x46, 0xE0, 0xBB, 0x8D,
+	// Bytes 2640 - 267f
+	0xE0, 0xBA, 0xB2, 0x46, 0xE0, 0xBD, 0x80, 0xE0,
+	0xBE, 0xB5, 0x46, 0xE0, 0xBD, 0x82, 0xE0, 0xBE,
+	0xB7, 0x46, 0xE0, 0xBD, 0x8C, 0xE0, 0xBE, 0xB7,
+	0x46, 0xE0, 0xBD, 0x91, 0xE0, 0xBE, 0xB7, 0x46,
+	0xE0, 0xBD, 0x96, 0xE0, 0xBE, 0xB7, 0x46, 0xE0,
+	0xBD, 0x9B, 0xE0, 0xBE, 0xB7, 0x46, 0xE0, 0xBE,
+	0x90, 0xE0, 0xBE, 0xB5, 0x46, 0xE0, 0xBE, 0x92,
+	0xE0, 0xBE, 0xB7, 0x46, 0xE0, 0xBE, 0x9C, 0xE0,
+	// Bytes 2680 - 26bf
+	0xBE, 0xB7, 0x46, 0xE0, 0xBE, 0xA1, 0xE0, 0xBE,
+	0xB7, 0x46, 0xE0, 0xBE, 0xA6, 0xE0, 0xBE, 0xB7,
+	0x46, 0xE0, 0xBE, 0xAB, 0xE0, 0xBE, 0xB7, 0x46,
+	0xE2, 0x80, 0xB2, 0xE2, 0x80, 0xB2, 0x46, 0xE2,
+	0x80, 0xB5, 0xE2, 0x80, 0xB5, 0x46, 0xE2, 0x88,
+	0xAB, 0xE2, 0x88, 0xAB, 0x46, 0xE2, 0x88, 0xAE,
+	0xE2, 0x88, 0xAE, 0x46, 0xE3, 0x81, 0xBB, 0xE3,
+	0x81, 0x8B, 0x46, 0xE3, 0x82, 0x88, 0xE3, 0x82,
+	// Bytes 26c0 - 26ff
+	0x8A, 0x46, 0xE3, 0x82, 0xAD, 0xE3, 0x83, 0xAD,
+	0x46, 0xE3, 0x82, 0xB3, 0xE3, 0x82, 0xB3, 0x46,
+	0xE3, 0x82, 0xB3, 0xE3, 0x83, 0x88, 0x46, 0xE3,
+	0x83, 0x88, 0xE3, 0x83, 0xB3, 0x46, 0xE3, 0x83,
+	0x8A, 0xE3, 0x83, 0x8E, 0x46, 0xE3, 0x83, 0x9B,
+	0xE3, 0x83, 0xB3, 0x46, 0xE3, 0x83, 0x9F, 0xE3,
+	0x83, 0xAA, 0x46, 0xE3, 0x83, 0xAA, 0xE3, 0x83,
+	0xA9, 0x46, 0xE3, 0x83, 0xAC, 0xE3, 0x83, 0xA0,
+	// Bytes 2700 - 273f
+	0x46, 0xE4, 0xBB, 0xA4, 0xE5, 0x92, 0x8C, 0x46,
+	0xE5, 0xA4, 0xA7, 0xE6, 0xAD, 0xA3, 0x46, 0xE5,
+	0xB9, 0xB3, 0xE6, 0x88, 0x90, 0x46, 0xE6, 0x98,
+	0x8E, 0xE6, 0xB2, 0xBB, 0x46, 0xE6, 0x98, 0xAD,
+	0xE5, 0x92, 0x8C, 0x47, 0x72, 0x61, 0x64, 0xE2,
+	0x88, 0x95, 0x73, 0x47, 0xE3, 0x80, 0x94, 0x53,
+	0xE3, 0x80, 0x95, 0x48, 0x28, 0xE1, 0x84, 0x80,
+	0xE1, 0x85, 0xA1, 0x29, 0x48, 0x28, 0xE1, 0x84,
+	// Bytes 2740 - 277f
+	0x82, 0xE1, 0x85, 0xA1, 0x29, 0x48, 0x28, 0xE1,
+	0x84, 0x83, 0xE1, 0x85, 0xA1, 0x29, 0x48, 0x28,
+	0xE1, 0x84, 0x85, 0xE1, 0x85, 0xA1, 0x29, 0x48,
+	0x28, 0xE1, 0x84, 0x86, 0xE1, 0x85, 0xA1, 0x29,
+	0x48, 0x28, 0xE1, 0x84, 0x87, 0xE1, 0x85, 0xA1,
+	0x29, 0x48, 0x28, 0xE1, 0x84, 0x89, 0xE1, 0x85,
+	0xA1, 0x29, 0x48, 0x28, 0xE1, 0x84, 0x8B, 0xE1,
+	0x85, 0xA1, 0x29, 0x48, 0x28, 0xE1, 0x84, 0x8C,
+	// Bytes 2780 - 27bf
+	0xE1, 0x85, 0xA1, 0x29, 0x48, 0x28, 0xE1, 0x84,
+	0x8C, 0xE1, 0x85, 0xAE, 0x29, 0x48, 0x28, 0xE1,
+	0x84, 0x8E, 0xE1, 0x85, 0xA1, 0x29, 0x48, 0x28,
+	0xE1, 0x84, 0x8F, 0xE1, 0x85, 0xA1, 0x29, 0x48,
+	0x28, 0xE1, 0x84, 0x90, 0xE1, 0x85, 0xA1, 0x29,
+	0x48, 0x28, 0xE1, 0x84, 0x91, 0xE1, 0x85, 0xA1,
+	0x29, 0x48, 0x28, 0xE1, 0x84, 0x92, 0xE1, 0x85,
+	0xA1, 0x29, 0x48, 0x72, 0x61, 0x64, 0xE2, 0x88,
+	// Bytes 27c0 - 27ff
+	0x95, 0x73, 0x32, 0x48, 0xD8, 0xA7, 0xD9, 0x83,
+	0xD8, 0xA8, 0xD8, 0xB1, 0x48, 0xD8, 0xA7, 0xD9,
+	0x84, 0xD9, 0x84, 0xD9, 0x87, 0x48, 0xD8, 0xB1,
+	0xD8, 0xB3, 0xD9, 0x88, 0xD9, 0x84, 0x48, 0xD8,
+	0xB1, 0xDB, 0x8C, 0xD8, 0xA7, 0xD9, 0x84, 0x48,
+	0xD8, 0xB5, 0xD9, 0x84, 0xD8, 0xB9, 0xD9, 0x85,
+	0x48, 0xD8, 0xB9, 0xD9, 0x84, 0xD9, 0x8A, 0xD9,
+	0x87, 0x48, 0xD9, 0x85, 0xD8, 0xAD, 0xD9, 0x85,
+	// Bytes 2800 - 283f
+	0xD8, 0xAF, 0x48, 0xD9, 0x88, 0xD8, 0xB3, 0xD9,
+	0x84, 0xD9, 0x85, 0x49, 0xE2, 0x80, 0xB2, 0xE2,
+	0x80, 0xB2, 0xE2, 0x80, 0xB2, 0x49, 0xE2, 0x80,
+	0xB5, 0xE2, 0x80, 0xB5, 0xE2, 0x80, 0xB5, 0x49,
+	0xE2, 0x88, 0xAB, 0xE2, 0x88, 0xAB, 0xE2, 0x88,
+	0xAB, 0x49, 0xE2, 0x88, 0xAE, 0xE2, 0x88, 0xAE,
+	0xE2, 0x88, 0xAE, 0x49, 0xE3, 0x80, 0x94, 0xE4,
+	0xB8, 0x89, 0xE3, 0x80, 0x95, 0x49, 0xE3, 0x80,
+	// Bytes 2840 - 287f
+	0x94, 0xE4, 0xBA, 0x8C, 0xE3, 0x80, 0x95, 0x49,
+	0xE3, 0x80, 0x94, 0xE5, 0x8B, 0x9D, 0xE3, 0x80,
+	0x95, 0x49, 0xE3, 0x80, 0x94, 0xE5, 0xAE, 0x89,
+	0xE3, 0x80, 0x95, 0x49, 0xE3, 0x80, 0x94, 0xE6,
+	0x89, 0x93, 0xE3, 0x80, 0x95, 0x49, 0xE3, 0x80,
+	0x94, 0xE6, 0x95, 0x97, 0xE3, 0x80, 0x95, 0x49,
+	0xE3, 0x80, 0x94, 0xE6, 0x9C, 0xAC, 0xE3, 0x80,
+	0x95, 0x49, 0xE3, 0x80, 0x94, 0xE7, 0x82, 0xB9,
+	// Bytes 2880 - 28bf
+	0xE3, 0x80, 0x95, 0x49, 0xE3, 0x80, 0x94, 0xE7,
+	0x9B, 0x97, 0xE3, 0x80, 0x95, 0x49, 0xE3, 0x82,
+	0xA2, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xAB, 0x49,
+	0xE3, 0x82, 0xA4, 0xE3, 0x83, 0xB3, 0xE3, 0x83,
+	0x81, 0x49, 0xE3, 0x82, 0xA6, 0xE3, 0x82, 0xA9,
+	0xE3, 0x83, 0xB3, 0x49, 0xE3, 0x82, 0xAA, 0xE3,
+	0x83, 0xB3, 0xE3, 0x82, 0xB9, 0x49, 0xE3, 0x82,
+	0xAA, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xA0, 0x49,
+	// Bytes 28c0 - 28ff
+	0xE3, 0x82, 0xAB, 0xE3, 0x82, 0xA4, 0xE3, 0x83,
+	0xAA, 0x49, 0xE3, 0x82, 0xB1, 0xE3, 0x83, 0xBC,
+	0xE3, 0x82, 0xB9, 0x49, 0xE3, 0x82, 0xB3, 0xE3,
+	0x83, 0xAB, 0xE3, 0x83, 0x8A, 0x49, 0xE3, 0x82,
+	0xBB, 0xE3, 0x83, 0xB3, 0xE3, 0x83, 0x81, 0x49,
+	0xE3, 0x82, 0xBB, 0xE3, 0x83, 0xB3, 0xE3, 0x83,
+	0x88, 0x49, 0xE3, 0x83, 0x86, 0xE3, 0x82, 0x99,
+	0xE3, 0x82, 0xB7, 0x49, 0xE3, 0x83, 0x88, 0xE3,
+	// Bytes 2900 - 293f
+	0x82, 0x99, 0xE3, 0x83, 0xAB, 0x49, 0xE3, 0x83,
+	0x8E, 0xE3, 0x83, 0x83, 0xE3, 0x83, 0x88, 0x49,
+	0xE3, 0x83, 0x8F, 0xE3, 0x82, 0xA4, 0xE3, 0x83,
+	0x84, 0x49, 0xE3, 0x83, 0x92, 0xE3, 0x82, 0x99,
+	0xE3, 0x83, 0xAB, 0x49, 0xE3, 0x83, 0x92, 0xE3,
+	0x82, 0x9A, 0xE3, 0x82, 0xB3, 0x49, 0xE3, 0x83,
+	0x95, 0xE3, 0x83, 0xA9, 0xE3, 0x83, 0xB3, 0x49,
+	0xE3, 0x83, 0x98, 0xE3, 0x82, 0x9A, 0xE3, 0x82,
+	// Bytes 2940 - 297f
+	0xBD, 0x49, 0xE3, 0x83, 0x98, 0xE3, 0x83, 0xAB,
+	0xE3, 0x83, 0x84, 0x49, 0xE3, 0x83, 0x9B, 0xE3,
+	0x83, 0xBC, 0xE3, 0x83, 0xAB, 0x49, 0xE3, 0x83,
+	0x9B, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xB3, 0x49,
+	0xE3, 0x83, 0x9E, 0xE3, 0x82, 0xA4, 0xE3, 0x83,
+	0xAB, 0x49, 0xE3, 0x83, 0x9E, 0xE3, 0x83, 0x83,
+	0xE3, 0x83, 0x8F, 0x49, 0xE3, 0x83, 0x9E, 0xE3,
+	0x83, 0xAB, 0xE3, 0x82, 0xAF, 0x49, 0xE3, 0x83,
+	// Bytes 2980 - 29bf
+	0xA4, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xAB, 0x49,
+	0xE3, 0x83, 0xA6, 0xE3, 0x82, 0xA2, 0xE3, 0x83,
+	0xB3, 0x49, 0xE3, 0x83, 0xAF, 0xE3, 0x83, 0x83,
+	0xE3, 0x83, 0x88, 0x4C, 0xE2, 0x80, 0xB2, 0xE2,
+	0x80, 0xB2, 0xE2, 0x80, 0xB2, 0xE2, 0x80, 0xB2,
+	0x4C, 0xE2, 0x88, 0xAB, 0xE2, 0x88, 0xAB, 0xE2,
+	0x88, 0xAB, 0xE2, 0x88, 0xAB, 0x4C, 0xE3, 0x82,
+	0xA2, 0xE3, 0x83, 0xAB, 0xE3, 0x83, 0x95, 0xE3,
+	// Bytes 29c0 - 29ff
+	0x82, 0xA1, 0x4C, 0xE3, 0x82, 0xA8, 0xE3, 0x83,
+	0xBC, 0xE3, 0x82, 0xAB, 0xE3, 0x83, 0xBC, 0x4C,
+	0xE3, 0x82, 0xAB, 0xE3, 0x82, 0x99, 0xE3, 0x83,
+	0xAD, 0xE3, 0x83, 0xB3, 0x4C, 0xE3, 0x82, 0xAB,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0xB3, 0xE3, 0x83,
+	0x9E, 0x4C, 0xE3, 0x82, 0xAB, 0xE3, 0x83, 0xA9,
+	0xE3, 0x83, 0x83, 0xE3, 0x83, 0x88, 0x4C, 0xE3,
+	0x82, 0xAB, 0xE3, 0x83, 0xAD, 0xE3, 0x83, 0xAA,
+	// Bytes 2a00 - 2a3f
+	0xE3, 0x83, 0xBC, 0x4C, 0xE3, 0x82, 0xAD, 0xE3,
+	0x82, 0x99, 0xE3, 0x83, 0x8B, 0xE3, 0x83, 0xBC,
+	0x4C, 0xE3, 0x82, 0xAD, 0xE3, 0x83, 0xA5, 0xE3,
+	0x83, 0xAA, 0xE3, 0x83, 0xBC, 0x4C, 0xE3, 0x82,
+	0xAF, 0xE3, 0x82, 0x99, 0xE3, 0x83, 0xA9, 0xE3,
+	0x83, 0xA0, 0x4C, 0xE3, 0x82, 0xAF, 0xE3, 0x83,
+	0xAD, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x8D, 0x4C,
+	0xE3, 0x82, 0xB5, 0xE3, 0x82, 0xA4, 0xE3, 0x82,
+	// Bytes 2a40 - 2a7f
+	0xAF, 0xE3, 0x83, 0xAB, 0x4C, 0xE3, 0x82, 0xBF,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0xBC, 0xE3, 0x82,
+	0xB9, 0x4C, 0xE3, 0x83, 0x8F, 0xE3, 0x82, 0x9A,
+	0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x84, 0x4C, 0xE3,
+	0x83, 0x92, 0xE3, 0x82, 0x9A, 0xE3, 0x82, 0xAF,
+	0xE3, 0x83, 0xAB, 0x4C, 0xE3, 0x83, 0x95, 0xE3,
+	0x82, 0xA3, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x88,
+	0x4C, 0xE3, 0x83, 0x98, 0xE3, 0x82, 0x99, 0xE3,
+	// Bytes 2a80 - 2abf
+	0x83, 0xBC, 0xE3, 0x82, 0xBF, 0x4C, 0xE3, 0x83,
+	0x98, 0xE3, 0x82, 0x9A, 0xE3, 0x83, 0x8B, 0xE3,
+	0x83, 0x92, 0x4C, 0xE3, 0x83, 0x98, 0xE3, 0x82,
+	0x9A, 0xE3, 0x83, 0xB3, 0xE3, 0x82, 0xB9, 0x4C,
+	0xE3, 0x83, 0x9B, 0xE3, 0x82, 0x99, 0xE3, 0x83,
+	0xAB, 0xE3, 0x83, 0x88, 0x4C, 0xE3, 0x83, 0x9E,
+	0xE3, 0x82, 0xA4, 0xE3, 0x82, 0xAF, 0xE3, 0x83,
+	0xAD, 0x4C, 0xE3, 0x83, 0x9F, 0xE3, 0x82, 0xAF,
+	// Bytes 2ac0 - 2aff
+	0xE3, 0x83, 0xAD, 0xE3, 0x83, 0xB3, 0x4C, 0xE3,
+	0x83, 0xA1, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x88,
+	0xE3, 0x83, 0xAB, 0x4C, 0xE3, 0x83, 0xAA, 0xE3,
+	0x83, 0x83, 0xE3, 0x83, 0x88, 0xE3, 0x83, 0xAB,
+	0x4C, 0xE3, 0x83, 0xAB, 0xE3, 0x83, 0x92, 0xE3,
+	0x82, 0x9A, 0xE3, 0x83, 0xBC, 0x4C, 0xE6, 0xA0,
+	0xAA, 0xE5, 0xBC, 0x8F, 0xE4, 0xBC, 0x9A, 0xE7,
+	0xA4, 0xBE, 0x4E, 0x28, 0xE1, 0x84, 0x8B, 0xE1,
+	// Bytes 2b00 - 2b3f
+	0x85, 0xA9, 0xE1, 0x84, 0x92, 0xE1, 0x85, 0xAE,
+	0x29, 0x4F, 0xD8, 0xAC, 0xD9, 0x84, 0x20, 0xD8,
+	0xAC, 0xD9, 0x84, 0xD8, 0xA7, 0xD9, 0x84, 0xD9,
+	0x87, 0x4F, 0xE3, 0x82, 0xA2, 0xE3, 0x83, 0x8F,
+	0xE3, 0x82, 0x9A, 0xE3, 0x83, 0xBC, 0xE3, 0x83,
+	0x88, 0x4F, 0xE3, 0x82, 0xA2, 0xE3, 0x83, 0xB3,
+	0xE3, 0x83, 0x98, 0xE3, 0x82, 0x9A, 0xE3, 0x82,
+	0xA2, 0x4F, 0xE3, 0x82, 0xAD, 0xE3, 0x83, 0xAD,
+	// Bytes 2b40 - 2b7f
+	0xE3, 0x83, 0xAF, 0xE3, 0x83, 0x83, 0xE3, 0x83,
+	0x88, 0x4F, 0xE3, 0x82, 0xB5, 0xE3, 0x83, 0xB3,
+	0xE3, 0x83, 0x81, 0xE3, 0x83, 0xBC, 0xE3, 0x83,
+	0xA0, 0x4F, 0xE3, 0x83, 0x8F, 0xE3, 0x82, 0x99,
+	0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xAC, 0xE3, 0x83,
+	0xAB, 0x4F, 0xE3, 0x83, 0x98, 0xE3, 0x82, 0xAF,
+	0xE3, 0x82, 0xBF, 0xE3, 0x83, 0xBC, 0xE3, 0x83,
+	0xAB, 0x4F, 0xE3, 0x83, 0x9B, 0xE3, 0x82, 0x9A,
+	// Bytes 2b80 - 2bbf
+	0xE3, 0x82, 0xA4, 0xE3, 0x83, 0xB3, 0xE3, 0x83,
+	0x88, 0x4F, 0xE3, 0x83, 0x9E, 0xE3, 0x83, 0xB3,
+	0xE3, 0x82, 0xB7, 0xE3, 0x83, 0xA7, 0xE3, 0x83,
+	0xB3, 0x4F, 0xE3, 0x83, 0xA1, 0xE3, 0x82, 0xAB,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0x88, 0xE3, 0x83,
+	0xB3, 0x4F, 0xE3, 0x83, 0xAB, 0xE3, 0x83, 0xBC,
+	0xE3, 0x83, 0x95, 0xE3, 0x82, 0x99, 0xE3, 0x83,
+	0xAB, 0x51, 0x28, 0xE1, 0x84, 0x8B, 0xE1, 0x85,
+	// Bytes 2bc0 - 2bff
+	0xA9, 0xE1, 0x84, 0x8C, 0xE1, 0x85, 0xA5, 0xE1,
+	0x86, 0xAB, 0x29, 0x52, 0xE3, 0x82, 0xAD, 0xE3,
+	0x82, 0x99, 0xE3, 0x83, 0xAB, 0xE3, 0x82, 0xBF,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0xBC, 0x52, 0xE3,
+	0x82, 0xAD, 0xE3, 0x83, 0xAD, 0xE3, 0x82, 0xAF,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0xA9, 0xE3, 0x83,
+	0xA0, 0x52, 0xE3, 0x82, 0xAD, 0xE3, 0x83, 0xAD,
+	0xE3, 0x83, 0xA1, 0xE3, 0x83, 0xBC, 0xE3, 0x83,
+	// Bytes 2c00 - 2c3f
+	0x88, 0xE3, 0x83, 0xAB, 0x52, 0xE3, 0x82, 0xAF,
+	0xE3, 0x82, 0x99, 0xE3, 0x83, 0xA9, 0xE3, 0x83,
+	0xA0, 0xE3, 0x83, 0x88, 0xE3, 0x83, 0xB3, 0x52,
+	0xE3, 0x82, 0xAF, 0xE3, 0x83, 0xAB, 0xE3, 0x82,
+	0xBB, 0xE3, 0x82, 0x99, 0xE3, 0x82, 0xA4, 0xE3,
+	0x83, 0xAD, 0x52, 0xE3, 0x83, 0x8F, 0xE3, 0x82,
+	0x9A, 0xE3, 0x83, 0xBC, 0xE3, 0x82, 0xBB, 0xE3,
+	0x83, 0xB3, 0xE3, 0x83, 0x88, 0x52, 0xE3, 0x83,
+	// Bytes 2c40 - 2c7f
+	0x92, 0xE3, 0x82, 0x9A, 0xE3, 0x82, 0xA2, 0xE3,
+	0x82, 0xB9, 0xE3, 0x83, 0x88, 0xE3, 0x83, 0xAB,
+	0x52, 0xE3, 0x83, 0x95, 0xE3, 0x82, 0x99, 0xE3,
+	0x83, 0x83, 0xE3, 0x82, 0xB7, 0xE3, 0x82, 0xA7,
+	0xE3, 0x83, 0xAB, 0x52, 0xE3, 0x83, 0x9F, 0xE3,
+	0x83, 0xAA, 0xE3, 0x83, 0x8F, 0xE3, 0x82, 0x99,
+	0xE3, 0x83, 0xBC, 0xE3, 0x83, 0xAB, 0x52, 0xE3,
+	0x83, 0xAC, 0xE3, 0x83, 0xB3, 0xE3, 0x83, 0x88,
+	// Bytes 2c80 - 2cbf
+	0xE3, 0x82, 0xB1, 0xE3, 0x82, 0x99, 0xE3, 0x83,
+	0xB3, 0x61, 0xD8, 0xB5, 0xD9, 0x84, 0xD9, 0x89,
+	0x20, 0xD8, 0xA7, 0xD9, 0x84, 0xD9, 0x84, 0xD9,
+	0x87, 0x20, 0xD8, 0xB9, 0xD9, 0x84, 0xD9, 0x8A,
+	0xD9, 0x87, 0x20, 0xD9, 0x88, 0xD8, 0xB3, 0xD9,
+	0x84, 0xD9, 0x85, 0x06, 0xE0, 0xA7, 0x87, 0xE0,
+	0xA6, 0xBE, 0x01, 0x06, 0xE0, 0xA7, 0x87, 0xE0,
+	0xA7, 0x97, 0x01, 0x06, 0xE0, 0xAD, 0x87, 0xE0,
+	// Bytes 2cc0 - 2cff
+	0xAC, 0xBE, 0x01, 0x06, 0xE0, 0xAD, 0x87, 0xE0,
+	0xAD, 0x96, 0x01, 0x06, 0xE0, 0xAD, 0x87, 0xE0,
+	0xAD, 0x97, 0x01, 0x06, 0xE0, 0xAE, 0x92, 0xE0,
+	0xAF, 0x97, 0x01, 0x06, 0xE0, 0xAF, 0x86, 0xE0,
+	0xAE, 0xBE, 0x01, 0x06, 0xE0, 0xAF, 0x86, 0xE0,
+	0xAF, 0x97, 0x01, 0x06, 0xE0, 0xAF, 0x87, 0xE0,
+	0xAE, 0xBE, 0x01, 0x06, 0xE0, 0xB2, 0xBF, 0xE0,
+	0xB3, 0x95, 0x01, 0x06, 0xE0, 0xB3, 0x86, 0xE0,
+	// Bytes 2d00 - 2d3f
+	0xB3, 0x95, 0x01, 0x06, 0xE0, 0xB3, 0x86, 0xE0,
+	0xB3, 0x96, 0x01, 0x06, 0xE0, 0xB5, 0x86, 0xE0,
+	0xB4, 0xBE, 0x01, 0x06, 0xE0, 0xB5, 0x86, 0xE0,
+	0xB5, 0x97, 0x01, 0x06, 0xE0, 0xB5, 0x87, 0xE0,
+	0xB4, 0xBE, 0x01, 0x06, 0xE0, 0xB7, 0x99, 0xE0,
+	0xB7, 0x9F, 0x01, 0x06, 0xE1, 0x80, 0xA5, 0xE1,
+	0x80, 0xAE, 0x01, 0x06, 0xE1, 0xAC, 0x85, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0x87, 0xE1,
+	// Bytes 2d40 - 2d7f
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0x89, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0x8B, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0x8D, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0x91, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0xBA, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0xBC, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0xBE, 0xE1,
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAC, 0xBF, 0xE1,
+	// Bytes 2d80 - 2dbf
+	0xAC, 0xB5, 0x01, 0x06, 0xE1, 0xAD, 0x82, 0xE1,
+	0xAC, 0xB5, 0x01, 0x08, 0xF0, 0x91, 0x84, 0xB1,
+	0xF0, 0x91, 0x84, 0xA7, 0x01, 0x08, 0xF0, 0x91,
+	0x84, 0xB2, 0xF0, 0x91, 0x84, 0xA7, 0x01, 0x08,
+	0xF0, 0x91, 0x8D, 0x87, 0xF0, 0x91, 0x8C, 0xBE,
+	0x01, 0x08, 0xF0, 0x91, 0x8D, 0x87, 0xF0, 0x91,
+	0x8D, 0x97, 0x01, 0x08, 0xF0, 0x91, 0x92, 0xB9,
+	0xF0, 0x91, 0x92, 0xB0, 0x01, 0x08, 0xF0, 0x91,
+	// Bytes 2dc0 - 2dff
+	0x92, 0xB9, 0xF0, 0x91, 0x92, 0xBA, 0x01, 0x08,
+	0xF0, 0x91, 0x92, 0xB9, 0xF0, 0x91, 0x92, 0xBD,
+	0x01, 0x08, 0xF0, 0x91, 0x96, 0xB8, 0xF0, 0x91,
+	0x96, 0xAF, 0x01, 0x08, 0xF0, 0x91, 0x96, 0xB9,
+	0xF0, 0x91, 0x96, 0xAF, 0x01, 0x08, 0xF0, 0x91,
+	0xA4, 0xB5, 0xF0, 0x91, 0xA4, 0xB0, 0x01, 0x09,
+	0xE0, 0xB3, 0x86, 0xE0, 0xB3, 0x82, 0xE0, 0xB3,
+	0x95, 0x02, 0x09, 0xE0, 0xB7, 0x99, 0xE0, 0xB7,
+	// Bytes 2e00 - 2e3f
+	0x8F, 0xE0, 0xB7, 0x8A, 0x16, 0x44, 0x44, 0x5A,
+	0xCC, 0x8C, 0xCD, 0x44, 0x44, 0x7A, 0xCC, 0x8C,
+	0xCD, 0x44, 0x64, 0x7A, 0xCC, 0x8C, 0xCD, 0x46,
+	0xD9, 0x84, 0xD8, 0xA7, 0xD9, 0x93, 0xCD, 0x46,
+	0xD9, 0x84, 0xD8, 0xA7, 0xD9, 0x94, 0xCD, 0x46,
+	0xD9, 0x84, 0xD8, 0xA7, 0xD9, 0x95, 0xB9, 0x46,
+	0xE1, 0x84, 0x80, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x82, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	// Bytes 2e40 - 2e7f
+	0xE1, 0x84, 0x83, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x85, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x86, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x87, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x89, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x8B, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x8B, 0xE1, 0x85, 0xAE, 0x01, 0x46,
+	0xE1, 0x84, 0x8C, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	// Bytes 2e80 - 2ebf
+	0xE1, 0x84, 0x8E, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x8F, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x90, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x91, 0xE1, 0x85, 0xA1, 0x01, 0x46,
+	0xE1, 0x84, 0x92, 0xE1, 0x85, 0xA1, 0x01, 0x49,
+	0xE3, 0x83, 0xA1, 0xE3, 0x82, 0xAB, 0xE3, 0x82,
+	0x99, 0x11, 0x4C, 0xE1, 0x84, 0x8C, 0xE1, 0x85,
+	0xAE, 0xE1, 0x84, 0x8B, 0xE1, 0x85, 0xB4, 0x01,
+	// Bytes 2ec0 - 2eff
+	0x4C, 0xE3, 0x82, 0xAD, 0xE3, 0x82, 0x99, 0xE3,
+	0x82, 0xAB, 0xE3, 0x82, 0x99, 0x11, 0x4C, 0xE3,
+	0x82, 0xB3, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x9B,
+	0xE3, 0x82, 0x9A, 0x11, 0x4C, 0xE3, 0x83, 0xA4,
+	0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x88, 0xE3, 0x82,
+	0x99, 0x11, 0x4F, 0xE1, 0x84, 0x8E, 0xE1, 0x85,
+	0xA1, 0xE1, 0x86, 0xB7, 0xE1, 0x84, 0x80, 0xE1,
+	0x85, 0xA9, 0x01, 0x4F, 0xE3, 0x82, 0xA4, 0xE3,
+	// Bytes 2f00 - 2f3f
+	0x83, 0x8B, 0xE3, 0x83, 0xB3, 0xE3, 0x82, 0xAF,
+	0xE3, 0x82, 0x99, 0x11, 0x4F, 0xE3, 0x82, 0xB7,
+	0xE3, 0x83, 0xAA, 0xE3, 0x83, 0xB3, 0xE3, 0x82,
+	0xAF, 0xE3, 0x82, 0x99, 0x11, 0x4F, 0xE3, 0x83,
+	0x98, 0xE3, 0x82, 0x9A, 0xE3, 0x83, 0xBC, 0xE3,
+	0x82, 0xB7, 0xE3, 0x82, 0x99, 0x11, 0x4F, 0xE3,
+	0x83, 0x9B, 0xE3, 0x82, 0x9A, 0xE3, 0x83, 0xB3,
+	0xE3, 0x83, 0x88, 0xE3, 0x82, 0x99, 0x11, 0x52,
+	// Bytes 2f40 - 2f7f
+	0xE3, 0x82, 0xA8, 0xE3, 0x82, 0xB9, 0xE3, 0x82,
+	0xAF, 0xE3, 0x83, 0xBC, 0xE3, 0x83, 0x88, 0xE3,
+	0x82, 0x99, 0x11, 0x52, 0xE3, 0x83, 0x95, 0xE3,
+	0x82, 0xA1, 0xE3, 0x83, 0xA9, 0xE3, 0x83, 0x83,
+	0xE3, 0x83, 0x88, 0xE3, 0x82, 0x99, 0x11, 0x86,
+	0xE0, 0xB3, 0x86, 0xE0, 0xB3, 0x82, 0x01, 0x86,
+	0xE0, 0xB7, 0x99, 0xE0, 0xB7, 0x8F, 0x01, 0x03,
+	0x3C, 0xCC, 0xB8, 0x05, 0x03, 0x3D, 0xCC, 0xB8,
+	// Bytes 2f80 - 2fbf
+	0x05, 0x03, 0x3E, 0xCC, 0xB8, 0x05, 0x03, 0x41,
+	0xCC, 0x80, 0xCD, 0x03, 0x41, 0xCC, 0x81, 0xCD,
+	0x03, 0x41, 0xCC, 0x83, 0xCD, 0x03, 0x41, 0xCC,
+	0x84, 0xCD, 0x03, 0x41, 0xCC, 0x89, 0xCD, 0x03,
+	0x41, 0xCC, 0x8C, 0xCD, 0x03, 0x41, 0xCC, 0x8F,
+	0xCD, 0x03, 0x41, 0xCC, 0x91, 0xCD, 0x03, 0x41,
+	0xCC, 0xA5, 0xB9, 0x03, 0x41, 0xCC, 0xA8, 0xA9,
+	0x03, 0x42, 0xCC, 0x87, 0xCD, 0x03, 0x42, 0xCC,
+	// Bytes 2fc0 - 2fff
+	0xA3, 0xB9, 0x03, 0x42, 0xCC, 0xB1, 0xB9, 0x03,
+	0x43, 0xCC, 0x81, 0xCD, 0x03, 0x43, 0xCC, 0x82,
+	0xCD, 0x03, 0x43, 0xCC, 0x87, 0xCD, 0x03, 0x43,
+	0xCC, 0x8C, 0xCD, 0x03, 0x44, 0xCC, 0x87, 0xCD,
+	0x03, 0x44, 0xCC, 0x8C, 0xCD, 0x03, 0x44, 0xCC,
+	0xA3, 0xB9, 0x03, 0x44, 0xCC, 0xA7, 0xA9, 0x03,
+	0x44, 0xCC, 0xAD, 0xB9, 0x03, 0x44, 0xCC, 0xB1,
+	0xB9, 0x03, 0x45, 0xCC, 0x80, 0xCD, 0x03, 0x45,
+	// Bytes 3000 - 303f
+	0xCC, 0x81, 0xCD, 0x03, 0x45, 0xCC, 0x83, 0xCD,
+	0x03, 0x45, 0xCC, 0x86, 0xCD, 0x03, 0x45, 0xCC,
+	0x87, 0xCD, 0x03, 0x45, 0xCC, 0x88, 0xCD, 0x03,
+	0x45, 0xCC, 0x89, 0xCD, 0x03, 0x45, 0xCC, 0x8C,
+	0xCD, 0x03, 0x45, 0xCC, 0x8F, 0xCD, 0x03, 0x45,
+	0xCC, 0x91, 0xCD, 0x03, 0x45, 0xCC, 0xA8, 0xA9,
+	0x03, 0x45, 0xCC, 0xAD, 0xB9, 0x03, 0x45, 0xCC,
+	0xB0, 0xB9, 0x03, 0x46, 0xCC, 0x87, 0xCD, 0x03,
+	// Bytes 3040 - 307f
+	0x47, 0xCC, 0x81, 0xCD, 0x03, 0x47, 0xCC, 0x82,
+	0xCD, 0x03, 0x47, 0xCC, 0x84, 0xCD, 0x03, 0x47,
+	0xCC, 0x86, 0xCD, 0x03, 0x47, 0xCC, 0x87, 0xCD,
+	0x03, 0x47, 0xCC, 0x8C, 0xCD, 0x03, 0x47, 0xCC,
+	0xA7, 0xA9, 0x03, 0x48, 0xCC, 0x82, 0xCD, 0x03,
+	0x48, 0xCC, 0x87, 0xCD, 0x03, 0x48, 0xCC, 0x88,
+	0xCD, 0x03, 0x48, 0xCC, 0x8C, 0xCD, 0x03, 0x48,
+	0xCC, 0xA3, 0xB9, 0x03, 0x48, 0xCC, 0xA7, 0xA9,
+	// Bytes 3080 - 30bf
+	0x03, 0x48, 0xCC, 0xAE, 0xB9, 0x03, 0x49, 0xCC,
+	0x80, 0xCD, 0x03, 0x49, 0xCC, 0x81, 0xCD, 0x03,
+	0x49, 0xCC, 0x82, 0xCD, 0x03, 0x49, 0xCC, 0x83,
+	0xCD, 0x03, 0x49, 0xCC, 0x84, 0xCD, 0x03, 0x49,
+	0xCC, 0x86, 0xCD, 0x03, 0x49, 0xCC, 0x87, 0xCD,
+	0x03, 0x49, 0xCC, 0x89, 0xCD, 0x03, 0x49, 0xCC,
+	0x8C, 0xCD, 0x03, 0x49, 0xCC, 0x8F, 0xCD, 0x03,
+	0x49, 0xCC, 0x91, 0xCD, 0x03, 0x49, 0xCC, 0xA3,
+	// Bytes 30c0 - 30ff
+	0xB9, 0x03, 0x49, 0xCC, 0xA8, 0xA9, 0x03, 0x49,
+	0xCC, 0xB0, 0xB9, 0x03, 0x4A, 0xCC, 0x82, 0xCD,
+	0x03, 0x4B, 0xCC, 0x81, 0xCD, 0x03, 0x4B, 0xCC,
+	0x8C, 0xCD, 0x03, 0x4B, 0xCC, 0xA3, 0xB9, 0x03,
+	0x4B, 0xCC, 0xA7, 0xA9, 0x03, 0x4B, 0xCC, 0xB1,
+	0xB9, 0x03, 0x4C, 0xCC, 0x81, 0xCD, 0x03, 0x4C,
+	0xCC, 0x8C, 0xCD, 0x03, 0x4C, 0xCC, 0xA7, 0xA9,
+	0x03, 0x4C, 0xCC, 0xAD, 0xB9, 0x03, 0x4C, 0xCC,
+	// Bytes 3100 - 313f
+	0xB1, 0xB9, 0x03, 0x4D, 0xCC, 0x81, 0xCD, 0x03,
+	0x4D, 0xCC, 0x87, 0xCD, 0x03, 0x4D, 0xCC, 0xA3,
+	0xB9, 0x03, 0x4E, 0xCC, 0x80, 0xCD, 0x03, 0x4E,
+	0xCC, 0x81, 0xCD, 0x03, 0x4E, 0xCC, 0x83, 0xCD,
+	0x03, 0x4E, 0xCC, 0x87, 0xCD, 0x03, 0x4E, 0xCC,
+	0x8C, 0xCD, 0x03, 0x4E, 0xCC, 0xA3, 0xB9, 0x03,
+	0x4E, 0xCC, 0xA7, 0xA9, 0x03, 0x4E, 0xCC, 0xAD,
+	0xB9, 0x03, 0x4E, 0xCC, 0xB1, 0xB9, 0x03, 0x4F,
+	// Bytes 3140 - 317f
+	0xCC, 0x80, 0xCD, 0x03, 0x4F, 0xCC, 0x81, 0xCD,
+	0x03, 0x4F, 0xCC, 0x86, 0xCD, 0x03, 0x4F, 0xCC,
+	0x89, 0xCD, 0x03, 0x4F, 0xCC, 0x8B, 0xCD, 0x03,
+	0x4F, 0xCC, 0x8C, 0xCD, 0x03, 0x4F, 0xCC, 0x8F,
+	0xCD, 0x03, 0x4F, 0xCC, 0x91, 0xCD, 0x03, 0x50,
+	0xCC, 0x81, 0xCD, 0x03, 0x50, 0xCC, 0x87, 0xCD,
+	0x03, 0x52, 0xCC, 0x81, 0xCD, 0x03, 0x52, 0xCC,
+	0x87, 0xCD, 0x03, 0x52, 0xCC, 0x8C, 0xCD, 0x03,
+	// Bytes 3180 - 31bf
+	0x52, 0xCC, 0x8F, 0xCD, 0x03, 0x52, 0xCC, 0x91,
+	0xCD, 0x03, 0x52, 0xCC, 0xA7, 0xA9, 0x03, 0x52,
+	0xCC, 0xB1, 0xB9, 0x03, 0x53, 0xCC, 0x82, 0xCD,
+	0x03, 0x53, 0xCC, 0x87, 0xCD, 0x03, 0x53, 0xCC,
+	0xA6, 0xB9, 0x03, 0x53, 0xCC, 0xA7, 0xA9, 0x03,
+	0x54, 0xCC, 0x87, 0xCD, 0x03, 0x54, 0xCC, 0x8C,
+	0xCD, 0x03, 0x54, 0xCC, 0xA3, 0xB9, 0x03, 0x54,
+	0xCC, 0xA6, 0xB9, 0x03, 0x54, 0xCC, 0xA7, 0xA9,
+	// Bytes 31c0 - 31ff
+	0x03, 0x54, 0xCC, 0xAD, 0xB9, 0x03, 0x54, 0xCC,
+	0xB1, 0xB9, 0x03, 0x55, 0xCC, 0x80, 0xCD, 0x03,
+	0x55, 0xCC, 0x81, 0xCD, 0x03, 0x55, 0xCC, 0x82,
+	0xCD, 0x03, 0x55, 0xCC, 0x86, 0xCD, 0x03, 0x55,
+	0xCC, 0x89, 0xCD, 0x03, 0x55, 0xCC, 0x8A, 0xCD,
+	0x03, 0x55, 0xCC, 0x8B, 0xCD, 0x03, 0x55, 0xCC,
+	0x8C, 0xCD, 0x03, 0x55, 0xCC, 0x8F, 0xCD, 0x03,
+	0x55, 0xCC, 0x91, 0xCD, 0x03, 0x55, 0xCC, 0xA3,
+	// Bytes 3200 - 323f
+	0xB9, 0x03, 0x55, 0xCC, 0xA4, 0xB9, 0x03, 0x55,
+	0xCC, 0xA8, 0xA9, 0x03, 0x55, 0xCC, 0xAD, 0xB9,
+	0x03, 0x55, 0xCC, 0xB0, 0xB9, 0x03, 0x56, 0xCC,
+	0x83, 0xCD, 0x03, 0x56, 0xCC, 0xA3, 0xB9, 0x03,
+	0x57, 0xCC, 0x80, 0xCD, 0x03, 0x57, 0xCC, 0x81,
+	0xCD, 0x03, 0x57, 0xCC, 0x82, 0xCD, 0x03, 0x57,
+	0xCC, 0x87, 0xCD, 0x03, 0x57, 0xCC, 0x88, 0xCD,
+	0x03, 0x57, 0xCC, 0xA3, 0xB9, 0x03, 0x58, 0xCC,
+	// Bytes 3240 - 327f
+	0x87, 0xCD, 0x03, 0x58, 0xCC, 0x88, 0xCD, 0x03,
+	0x59, 0xCC, 0x80, 0xCD, 0x03, 0x59, 0xCC, 0x81,
+	0xCD, 0x03, 0x59, 0xCC, 0x82, 0xCD, 0x03, 0x59,
+	0xCC, 0x83, 0xCD, 0x03, 0x59, 0xCC, 0x84, 0xCD,
+	0x03, 0x59, 0xCC, 0x87, 0xCD, 0x03, 0x59, 0xCC,
+	0x88, 0xCD, 0x03, 0x59, 0xCC, 0x89, 0xCD, 0x03,
+	0x59, 0xCC, 0xA3, 0xB9, 0x03, 0x5A, 0xCC, 0x81,
+	0xCD, 0x03, 0x5A, 0xCC, 0x82, 0xCD, 0x03, 0x5A,
+	// Bytes 3280 - 32bf
+	0xCC, 0x87, 0xCD, 0x03, 0x5A, 0xCC, 0x8C, 0xCD,
+	0x03, 0x5A, 0xCC, 0xA3, 0xB9, 0x03, 0x5A, 0xCC,
+	0xB1, 0xB9, 0x03, 0x61, 0xCC, 0x80, 0xCD, 0x03,
+	0x61, 0xCC, 0x81, 0xCD, 0x03, 0x61, 0xCC, 0x83,
+	0xCD, 0x03, 0x61, 0xCC, 0x84, 0xCD, 0x03, 0x61,
+	0xCC, 0x89, 0xCD, 0x03, 0x61, 0xCC, 0x8C, 0xCD,
+	0x03, 0x61, 0xCC, 0x8F, 0xCD, 0x03, 0x61, 0xCC,
+	0x91, 0xCD, 0x03, 0x61, 0xCC, 0xA5, 0xB9, 0x03,
+	// Bytes 32c0 - 32ff
+	0x61, 0xCC, 0xA8, 0xA9, 0x03, 0x62, 0xCC, 0x87,
+	0xCD, 0x03, 0x62, 0xCC, 0xA3, 0xB9, 0x03, 0x62,
+	0xCC, 0xB1, 0xB9, 0x03, 0x63, 0xCC, 0x81, 0xCD,
+	0x03, 0x63, 0xCC, 0x82, 0xCD, 0x03, 0x63, 0xCC,
+	0x87, 0xCD, 0x03, 0x63, 0xCC, 0x8C, 0xCD, 0x03,
+	0x64, 0xCC, 0x87, 0xCD, 0x03, 0x64, 0xCC, 0x8C,
+	0xCD, 0x03, 0x64, 0xCC, 0xA3, 0xB9, 0x03, 0x64,
+	0xCC, 0xA7, 0xA9, 0x03, 0x64, 0xCC, 0xAD, 0xB9,
+	// Bytes 3300 - 333f
+	0x03, 0x64, 0xCC, 0xB1, 0xB9, 0x03, 0x65, 0xCC,
+	0x80, 0xCD, 0x03, 0x65, 0xCC, 0x81, 0xCD, 0x03,
+	0x65, 0xCC, 0x83, 0xCD, 0x03, 0x65, 0xCC, 0x86,
+	0xCD, 0x03, 0x65, 0xCC, 0x87, 0xCD, 0x03, 0x65,
+	0xCC, 0x88, 0xCD, 0x03, 0x65, 0xCC, 0x89, 0xCD,
+	0x03, 0x65, 0xCC, 0x8C, 0xCD, 0x03, 0x65, 0xCC,
+	0x8F, 0xCD, 0x03, 0x65, 0xCC, 0x91, 0xCD, 0x03,
+	0x65, 0xCC, 0xA8, 0xA9, 0x03, 0x65, 0xCC, 0xAD,
+	// Bytes 3340 - 337f
+	0xB9, 0x03, 0x65, 0xCC, 0xB0, 0xB9, 0x03, 0x66,
+	0xCC, 0x87, 0xCD, 0x03, 0x67, 0xCC, 0x81, 0xCD,
+	0x03, 0x67, 0xCC, 0x82, 0xCD, 0x03, 0x67, 0xCC,
+	0x84, 0xCD, 0x03, 0x67, 0xCC, 0x86, 0xCD, 0x03,
+	0x67, 0xCC, 0x87, 0xCD, 0x03, 0x67, 0xCC, 0x8C,
+	0xCD, 0x03, 0x67, 0xCC, 0xA7, 0xA9, 0x03, 0x68,
+	0xCC, 0x82, 0xCD, 0x03, 0x68, 0xCC, 0x87, 0xCD,
+	0x03, 0x68, 0xCC, 0x88, 0xCD, 0x03, 0x68, 0xCC,
+	// Bytes 3380 - 33bf
+	0x8C, 0xCD, 0x03, 0x68, 0xCC, 0xA3, 0xB9, 0x03,
+	0x68, 0xCC, 0xA7, 0xA9, 0x03, 0x68, 0xCC, 0xAE,
+	0xB9, 0x03, 0x68, 0xCC, 0xB1, 0xB9, 0x03, 0x69,
+	0xCC, 0x80, 0xCD, 0x03, 0x69, 0xCC, 0x81, 0xCD,
+	0x03, 0x69, 0xCC, 0x82, 0xCD, 0x03, 0x69, 0xCC,
+	0x83, 0xCD, 0x03, 0x69, 0xCC, 0x84, 0xCD, 0x03,
+	0x69, 0xCC, 0x86, 0xCD, 0x03, 0x69, 0xCC, 0x89,
+	0xCD, 0x03, 0x69, 0xCC, 0x8C, 0xCD, 0x03, 0x69,
+	// Bytes 33c0 - 33ff
+	0xCC, 0x8F, 0xCD, 0x03, 0x69, 0xCC, 0x91, 0xCD,
+	0x03, 0x69, 0xCC, 0xA3, 0xB9, 0x03, 0x69, 0xCC,
+	0xA8, 0xA9, 0x03, 0x69, 0xCC, 0xB0, 0xB9, 0x03,
+	0x6A, 0xCC, 0x82, 0xCD, 0x03, 0x6A, 0xCC, 0x8C,
+	0xCD, 0x03, 0x6B, 0xCC, 0x81, 0xCD, 0x03, 0x6B,
+	0xCC, 0x8C, 0xCD, 0x03, 0x6B, 0xCC, 0xA3, 0xB9,
+	0x03, 0x6B, 0xCC, 0xA7, 0xA9, 0x03, 0x6B, 0xCC,
+	0xB1, 0xB9, 0x03, 0x6C, 0xCC, 0x81, 0xCD, 0x03,
+	// Bytes 3400 - 343f
+	0x6C, 0xCC, 0x8C, 0xCD, 0x03, 0x6C, 0xCC, 0xA7,
+	0xA9, 0x03, 0x6C, 0xCC, 0xAD, 0xB9, 0x03, 0x6C,
+	0xCC, 0xB1, 0xB9, 0x03, 0x6D, 0xCC, 0x81, 0xCD,
+	0x03, 0x6D, 0xCC, 0x87, 0xCD, 0x03, 0x6D, 0xCC,
+	0xA3, 0xB9, 0x03, 0x6E, 0xCC, 0x80, 0xCD, 0x03,
+	0x6E, 0xCC, 0x81, 0xCD, 0x03, 0x6E, 0xCC, 0x83,
+	0xCD, 0x03, 0x6E, 0xCC, 0x87, 0xCD, 0x03, 0x6E,
+	0xCC, 0x8C, 0xCD, 0x03, 0x6E, 0xCC, 0xA3, 0xB9,
+	// Bytes 3440 - 347f
+	0x03, 0x6E, 0xCC, 0xA7, 0xA9, 0x03, 0x6E, 0xCC,
+	0xAD, 0xB9, 0x03, 0x6E, 0xCC, 0xB1, 0xB9, 0x03,
+	0x6F, 0xCC, 0x80, 0xCD, 0x03, 0x6F, 0xCC, 0x81,
+	0xCD, 0x03, 0x6F, 0xCC, 0x86, 0xCD, 0x03, 0x6F,
+	0xCC, 0x89, 0xCD, 0x03, 0x6F, 0xCC, 0x8B, 0xCD,
+	0x03, 0x6F, 0xCC, 0x8C, 0xCD, 0x03, 0x6F, 0xCC,
+	0x8F, 0xCD, 0x03, 0x6F, 0xCC, 0x91, 0xCD, 0x03,
+	0x70, 0xCC, 0x81, 0xCD, 0x03, 0x70, 0xCC, 0x87,
+	// Bytes 3480 - 34bf
+	0xCD, 0x03, 0x72, 0xCC, 0x81, 0xCD, 0x03, 0x72,
+	0xCC, 0x87, 0xCD, 0x03, 0x72, 0xCC, 0x8C, 0xCD,
+	0x03, 0x72, 0xCC, 0x8F, 0xCD, 0x03, 0x72, 0xCC,
+	0x91, 0xCD, 0x03, 0x72, 0xCC, 0xA7, 0xA9, 0x03,
+	0x72, 0xCC, 0xB1, 0xB9, 0x03, 0x73, 0xCC, 0x82,
+	0xCD, 0x03, 0x73, 0xCC, 0x87, 0xCD, 0x03, 0x73,
+	0xCC, 0xA6, 0xB9, 0x03, 0x73, 0xCC, 0xA7, 0xA9,
+	0x03, 0x74, 0xCC, 0x87, 0xCD, 0x03, 0x74, 0xCC,
+	// Bytes 34c0 - 34ff
+	0x88, 0xCD, 0x03, 0x74, 0xCC, 0x8C, 0xCD, 0x03,
+	0x74, 0xCC, 0xA3, 0xB9, 0x03, 0x74, 0xCC, 0xA6,
+	0xB9, 0x03, 0x74, 0xCC, 0xA7, 0xA9, 0x03, 0x74,
+	0xCC, 0xAD, 0xB9, 0x03, 0x74, 0xCC, 0xB1, 0xB9,
+	0x03, 0x75, 0xCC, 0x80, 0xCD, 0x03, 0x75, 0xCC,
+	0x81, 0xCD, 0x03, 0x75, 0xCC, 0x82, 0xCD, 0x03,
+	0x75, 0xCC, 0x86, 0xCD, 0x03, 0x75, 0xCC, 0x89,
+	0xCD, 0x03, 0x75, 0xCC, 0x8A, 0xCD, 0x03, 0x75,
+	// Bytes 3500 - 353f
+	0xCC, 0x8B, 0xCD, 0x03, 0x75, 0xCC, 0x8C, 0xCD,
+	0x03, 0x75, 0xCC, 0x8F, 0xCD, 0x03, 0x75, 0xCC,
+	0x91, 0xCD, 0x03, 0x75, 0xCC, 0xA3, 0xB9, 0x03,
+	0x75, 0xCC, 0xA4, 0xB9, 0x03, 0x75, 0xCC, 0xA8,
+	0xA9, 0x03, 0x75, 0xCC, 0xAD, 0xB9, 0x03, 0x75,
+	0xCC, 0xB0, 0xB9, 0x03, 0x76, 0xCC, 0x83, 0xCD,
+	0x03, 0x76, 0xCC, 0xA3, 0xB9, 0x03, 0x77, 0xCC,
+	0x80, 0xCD, 0x03, 0x77, 0xCC, 0x81, 0xCD, 0x03,
+	// Bytes 3540 - 357f
+	0x77, 0xCC, 0x82, 0xCD, 0x03, 0x77, 0xCC, 0x87,
+	0xCD, 0x03, 0x77, 0xCC, 0x88, 0xCD, 0x03, 0x77,
+	0xCC, 0x8A, 0xCD, 0x03, 0x77, 0xCC, 0xA3, 0xB9,
+	0x03, 0x78, 0xCC, 0x87, 0xCD, 0x03, 0x78, 0xCC,
+	0x88, 0xCD, 0x03, 0x79, 0xCC, 0x80, 0xCD, 0x03,
+	0x79, 0xCC, 0x81, 0xCD, 0x03, 0x79, 0xCC, 0x82,
+	0xCD, 0x03, 0x79, 0xCC, 0x83, 0xCD, 0x03, 0x79,
+	0xCC, 0x84, 0xCD, 0x03, 0x79, 0xCC, 0x87, 0xCD,
+	// Bytes 3580 - 35bf
+	0x03, 0x79, 0xCC, 0x88, 0xCD, 0x03, 0x79, 0xCC,
+	0x89, 0xCD, 0x03, 0x79, 0xCC, 0x8A, 0xCD, 0x03,
+	0x79, 0xCC, 0xA3, 0xB9, 0x03, 0x7A, 0xCC, 0x81,
+	0xCD, 0x03, 0x7A, 0xCC, 0x82, 0xCD, 0x03, 0x7A,
+	0xCC, 0x87, 0xCD, 0x03, 0x7A, 0xCC, 0x8C, 0xCD,
+	0x03, 0x7A, 0xCC, 0xA3, 0xB9, 0x03, 0x7A, 0xCC,
+	0xB1, 0xB9, 0x04, 0xC2, 0xA8, 0xCC, 0x80, 0xCE,
+	0x04, 0xC2, 0xA8, 0xCC, 0x81, 0xCE, 0x04, 0xC2,
+	// Bytes 35c0 - 35ff
+	0xA8, 0xCD, 0x82, 0xCE, 0x04, 0xC3, 0x86, 0xCC,
+	0x81, 0xCD, 0x04, 0xC3, 0x86, 0xCC, 0x84, 0xCD,
+	0x04, 0xC3, 0x98, 0xCC, 0x81, 0xCD, 0x04, 0xC3,
+	0xA6, 0xCC, 0x81, 0xCD, 0x04, 0xC3, 0xA6, 0xCC,
+	0x84, 0xCD, 0x04, 0xC3, 0xB8, 0xCC, 0x81, 0xCD,
+	0x04, 0xC5, 0xBF, 0xCC, 0x87, 0xCD, 0x04, 0xC6,
+	0xB7, 0xCC, 0x8C, 0xCD, 0x04, 0xCA, 0x92, 0xCC,
+	0x8C, 0xCD, 0x04, 0xCE, 0x91, 0xCC, 0x80, 0xCD,
+	// Bytes 3600 - 363f
+	0x04, 0xCE, 0x91, 0xCC, 0x81, 0xCD, 0x04, 0xCE,
+	0x91, 0xCC, 0x84, 0xCD, 0x04, 0xCE, 0x91, 0xCC,
+	0x86, 0xCD, 0x04, 0xCE, 0x91, 0xCD, 0x85, 0xDD,
+	0x04, 0xCE, 0x95, 0xCC, 0x80, 0xCD, 0x04, 0xCE,
+	0x95, 0xCC, 0x81, 0xCD, 0x04, 0xCE, 0x97, 0xCC,
+	0x80, 0xCD, 0x04, 0xCE, 0x97, 0xCC, 0x81, 0xCD,
+	0x04, 0xCE, 0x97, 0xCD, 0x85, 0xDD, 0x04, 0xCE,
+	0x99, 0xCC, 0x80, 0xCD, 0x04, 0xCE, 0x99, 0xCC,
+	// Bytes 3640 - 367f
+	0x81, 0xCD, 0x04, 0xCE, 0x99, 0xCC, 0x84, 0xCD,
+	0x04, 0xCE, 0x99, 0xCC, 0x86, 0xCD, 0x04, 0xCE,
+	0x99, 0xCC, 0x88, 0xCD, 0x04, 0xCE, 0x9F, 0xCC,
+	0x80, 0xCD, 0x04, 0xCE, 0x9F, 0xCC, 0x81, 0xCD,
+	0x04, 0xCE, 0xA1, 0xCC, 0x94, 0xCD, 0x04, 0xCE,
+	0xA5, 0xCC, 0x80, 0xCD, 0x04, 0xCE, 0xA5, 0xCC,
+	0x81, 0xCD, 0x04, 0xCE, 0xA5, 0xCC, 0x84, 0xCD,
+	0x04, 0xCE, 0xA5, 0xCC, 0x86, 0xCD, 0x04, 0xCE,
+	// Bytes 3680 - 36bf
+	0xA5, 0xCC, 0x88, 0xCD, 0x04, 0xCE, 0xA9, 0xCC,
+	0x80, 0xCD, 0x04, 0xCE, 0xA9, 0xCC, 0x81, 0xCD,
+	0x04, 0xCE, 0xA9, 0xCD, 0x85, 0xDD, 0x04, 0xCE,
+	0xB1, 0xCC, 0x84, 0xCD, 0x04, 0xCE, 0xB1, 0xCC,
+	0x86, 0xCD, 0x04, 0xCE, 0xB1, 0xCD, 0x85, 0xDD,
+	0x04, 0xCE, 0xB5, 0xCC, 0x80, 0xCD, 0x04, 0xCE,
+	0xB5, 0xCC, 0x81, 0xCD, 0x04, 0xCE, 0xB7, 0xCD,
+	0x85, 0xDD, 0x04, 0xCE, 0xB9, 0xCC, 0x80, 0xCD,
+	// Bytes 36c0 - 36ff
+	0x04, 0xCE, 0xB9, 0xCC, 0x81, 0xCD, 0x04, 0xCE,
+	0xB9, 0xCC, 0x84, 0xCD, 0x04, 0xCE, 0xB9, 0xCC,
+	0x86, 0xCD, 0x04, 0xCE, 0xB9, 0xCD, 0x82, 0xCD,
+	0x04, 0xCE, 0xBF, 0xCC, 0x80, 0xCD, 0x04, 0xCE,
+	0xBF, 0xCC, 0x81, 0xCD, 0x04, 0xCF, 0x81, 0xCC,
+	0x93, 0xCD, 0x04, 0xCF, 0x81, 0xCC, 0x94, 0xCD,
+	0x04, 0xCF, 0x85, 0xCC, 0x80, 0xCD, 0x04, 0xCF,
+	0x85, 0xCC, 0x81, 0xCD, 0x04, 0xCF, 0x85, 0xCC,
+	// Bytes 3700 - 373f
+	0x84, 0xCD, 0x04, 0xCF, 0x85, 0xCC, 0x86, 0xCD,
+	0x04, 0xCF, 0x85, 0xCD, 0x82, 0xCD, 0x04, 0xCF,
+	0x89, 0xCD, 0x85, 0xDD, 0x04, 0xCF, 0x92, 0xCC,
+	0x81, 0xCD, 0x04, 0xCF, 0x92, 0xCC, 0x88, 0xCD,
+	0x04, 0xD0, 0x86, 0xCC, 0x88, 0xCD, 0x04, 0xD0,
+	0x90, 0xCC, 0x86, 0xCD, 0x04, 0xD0, 0x90, 0xCC,
+	0x88, 0xCD, 0x04, 0xD0, 0x93, 0xCC, 0x81, 0xCD,
+	0x04, 0xD0, 0x95, 0xCC, 0x80, 0xCD, 0x04, 0xD0,
+	// Bytes 3740 - 377f
+	0x95, 0xCC, 0x86, 0xCD, 0x04, 0xD0, 0x95, 0xCC,
+	0x88, 0xCD, 0x04, 0xD0, 0x96, 0xCC, 0x86, 0xCD,
+	0x04, 0xD0, 0x96, 0xCC, 0x88, 0xCD, 0x04, 0xD0,
+	0x97, 0xCC, 0x88, 0xCD, 0x04, 0xD0, 0x98, 0xCC,
+	0x80, 0xCD, 0x04, 0xD0, 0x98, 0xCC, 0x84, 0xCD,
+	0x04, 0xD0, 0x98, 0xCC, 0x86, 0xCD, 0x04, 0xD0,
+	0x98, 0xCC, 0x88, 0xCD, 0x04, 0xD0, 0x9A, 0xCC,
+	0x81, 0xCD, 0x04, 0xD0, 0x9E, 0xCC, 0x88, 0xCD,
+	// Bytes 3780 - 37bf
+	0x04, 0xD0, 0xA3, 0xCC, 0x84, 0xCD, 0x04, 0xD0,
+	0xA3, 0xCC, 0x86, 0xCD, 0x04, 0xD0, 0xA3, 0xCC,
+	0x88, 0xCD, 0x04, 0xD0, 0xA3, 0xCC, 0x8B, 0xCD,
+	0x04, 0xD0, 0xA7, 0xCC, 0x88, 0xCD, 0x04, 0xD0,
+	0xAB, 0xCC, 0x88, 0xCD, 0x04, 0xD0, 0xAD, 0xCC,
+	0x88, 0xCD, 0x04, 0xD0, 0xB0, 0xCC, 0x86, 0xCD,
+	0x04, 0xD0, 0xB0, 0xCC, 0x88, 0xCD, 0x04, 0xD0,
+	0xB3, 0xCC, 0x81, 0xCD, 0x04, 0xD0, 0xB5, 0xCC,
+	// Bytes 37c0 - 37ff
+	0x80, 0xCD, 0x04, 0xD0, 0xB5, 0xCC, 0x86, 0xCD,
+	0x04, 0xD0, 0xB5, 0xCC, 0x88, 0xCD, 0x04, 0xD0,
+	0xB6, 0xCC, 0x86, 0xCD, 0x04, 0xD0, 0xB6, 0xCC,
+	0x88, 0xCD, 0x04, 0xD0, 0xB7, 0xCC, 0x88, 0xCD,
+	0x04, 0xD0, 0xB8, 0xCC, 0x80, 0xCD, 0x04, 0xD0,
+	0xB8, 0xCC, 0x84, 0xCD, 0x04, 0xD0, 0xB8, 0xCC,
+	0x86, 0xCD, 0x04, 0xD0, 0xB8, 0xCC, 0x88, 0xCD,
+	0x04, 0xD0, 0xBA, 0xCC, 0x81, 0xCD, 0x04, 0xD0,
+	// Bytes 3800 - 383f
+	0xBE, 0xCC, 0x88, 0xCD, 0x04, 0xD1, 0x83, 0xCC,
+	0x84, 0xCD, 0x04, 0xD1, 0x83, 0xCC, 0x86, 0xCD,
+	0x04, 0xD1, 0x83, 0xCC, 0x88, 0xCD, 0x04, 0xD1,
+	0x83, 0xCC, 0x8B, 0xCD, 0x04, 0xD1, 0x87, 0xCC,
+	0x88, 0xCD, 0x04, 0xD1, 0x8B, 0xCC, 0x88, 0xCD,
+	0x04, 0xD1, 0x8D, 0xCC, 0x88, 0xCD, 0x04, 0xD1,
+	0x96, 0xCC, 0x88, 0xCD, 0x04, 0xD1, 0xB4, 0xCC,
+	0x8F, 0xCD, 0x04, 0xD1, 0xB5, 0xCC, 0x8F, 0xCD,
+	// Bytes 3840 - 387f
+	0x04, 0xD3, 0x98, 0xCC, 0x88, 0xCD, 0x04, 0xD3,
+	0x99, 0xCC, 0x88, 0xCD, 0x04, 0xD3, 0xA8, 0xCC,
+	0x88, 0xCD, 0x04, 0xD3, 0xA9, 0xCC, 0x88, 0xCD,
+	0x04, 0xD8, 0xA7, 0xD9, 0x93, 0xCD, 0x04, 0xD8,
+	0xA7, 0xD9, 0x94, 0xCD, 0x04, 0xD8, 0xA7, 0xD9,
+	0x95, 0xB9, 0x04, 0xD9, 0x88, 0xD9, 0x94, 0xCD,
+	0x04, 0xD9, 0x8A, 0xD9, 0x94, 0xCD, 0x04, 0xDB,
+	0x81, 0xD9, 0x94, 0xCD, 0x04, 0xDB, 0x92, 0xD9,
+	// Bytes 3880 - 38bf
+	0x94, 0xCD, 0x04, 0xDB, 0x95, 0xD9, 0x94, 0xCD,
+	0x05, 0x41, 0xCC, 0x82, 0xCC, 0x80, 0xCE, 0x05,
+	0x41, 0xCC, 0x82, 0xCC, 0x81, 0xCE, 0x05, 0x41,
+	0xCC, 0x82, 0xCC, 0x83, 0xCE, 0x05, 0x41, 0xCC,
+	0x82, 0xCC, 0x89, 0xCE, 0x05, 0x41, 0xCC, 0x86,
+	0xCC, 0x80, 0xCE, 0x05, 0x41, 0xCC, 0x86, 0xCC,
+	0x81, 0xCE, 0x05, 0x41, 0xCC, 0x86, 0xCC, 0x83,
+	0xCE, 0x05, 0x41, 0xCC, 0x86, 0xCC, 0x89, 0xCE,
+	// Bytes 38c0 - 38ff
+	0x05, 0x41, 0xCC, 0x87, 0xCC, 0x84, 0xCE, 0x05,
+	0x41, 0xCC, 0x88, 0xCC, 0x84, 0xCE, 0x05, 0x41,
+	0xCC, 0x8A, 0xCC, 0x81, 0xCE, 0x05, 0x41, 0xCC,
+	0xA3, 0xCC, 0x82, 0xCE, 0x05, 0x41, 0xCC, 0xA3,
+	0xCC, 0x86, 0xCE, 0x05, 0x43, 0xCC, 0xA7, 0xCC,
+	0x81, 0xCE, 0x05, 0x45, 0xCC, 0x82, 0xCC, 0x80,
+	0xCE, 0x05, 0x45, 0xCC, 0x82, 0xCC, 0x81, 0xCE,
+	0x05, 0x45, 0xCC, 0x82, 0xCC, 0x83, 0xCE, 0x05,
+	// Bytes 3900 - 393f
+	0x45, 0xCC, 0x82, 0xCC, 0x89, 0xCE, 0x05, 0x45,
+	0xCC, 0x84, 0xCC, 0x80, 0xCE, 0x05, 0x45, 0xCC,
+	0x84, 0xCC, 0x81, 0xCE, 0x05, 0x45, 0xCC, 0xA3,
+	0xCC, 0x82, 0xCE, 0x05, 0x45, 0xCC, 0xA7, 0xCC,
+	0x86, 0xCE, 0x05, 0x49, 0xCC, 0x88, 0xCC, 0x81,
+	0xCE, 0x05, 0x4C, 0xCC, 0xA3, 0xCC, 0x84, 0xCE,
+	0x05, 0x4F, 0xCC, 0x82, 0xCC, 0x80, 0xCE, 0x05,
+	0x4F, 0xCC, 0x82, 0xCC, 0x81, 0xCE, 0x05, 0x4F,
+	// Bytes 3940 - 397f
+	0xCC, 0x82, 0xCC, 0x83, 0xCE, 0x05, 0x4F, 0xCC,
+	0x82, 0xCC, 0x89, 0xCE, 0x05, 0x4F, 0xCC, 0x83,
+	0xCC, 0x81, 0xCE, 0x05, 0x4F, 0xCC, 0x83, 0xCC,
+	0x84, 0xCE, 0x05, 0x4F, 0xCC, 0x83, 0xCC, 0x88,
+	0xCE, 0x05, 0x4F, 0xCC, 0x84, 0xCC, 0x80, 0xCE,
+	0x05, 0x4F, 0xCC, 0x84, 0xCC, 0x81, 0xCE, 0x05,
+	0x4F, 0xCC, 0x87, 0xCC, 0x84, 0xCE, 0x05, 0x4F,
+	0xCC, 0x88, 0xCC, 0x84, 0xCE, 0x05, 0x4F, 0xCC,
+	// Bytes 3980 - 39bf
+	0x9B, 0xCC, 0x80, 0xCE, 0x05, 0x4F, 0xCC, 0x9B,
+	0xCC, 0x81, 0xCE, 0x05, 0x4F, 0xCC, 0x9B, 0xCC,
+	0x83, 0xCE, 0x05, 0x4F, 0xCC, 0x9B, 0xCC, 0x89,
+	0xCE, 0x05, 0x4F, 0xCC, 0x9B, 0xCC, 0xA3, 0xBA,
+	0x05, 0x4F, 0xCC, 0xA3, 0xCC, 0x82, 0xCE, 0x05,
+	0x4F, 0xCC, 0xA8, 0xCC, 0x84, 0xCE, 0x05, 0x52,
+	0xCC, 0xA3, 0xCC, 0x84, 0xCE, 0x05, 0x53, 0xCC,
+	0x81, 0xCC, 0x87, 0xCE, 0x05, 0x53, 0xCC, 0x8C,
+	// Bytes 39c0 - 39ff
+	0xCC, 0x87, 0xCE, 0x05, 0x53, 0xCC, 0xA3, 0xCC,
+	0x87, 0xCE, 0x05, 0x55, 0xCC, 0x83, 0xCC, 0x81,
+	0xCE, 0x05, 0x55, 0xCC, 0x84, 0xCC, 0x88, 0xCE,
+	0x05, 0x55, 0xCC, 0x88, 0xCC, 0x80, 0xCE, 0x05,
+	0x55, 0xCC, 0x88, 0xCC, 0x81, 0xCE, 0x05, 0x55,
+	0xCC, 0x88, 0xCC, 0x84, 0xCE, 0x05, 0x55, 0xCC,
+	0x88, 0xCC, 0x8C, 0xCE, 0x05, 0x55, 0xCC, 0x9B,
+	0xCC, 0x80, 0xCE, 0x05, 0x55, 0xCC, 0x9B, 0xCC,
+	// Bytes 3a00 - 3a3f
+	0x81, 0xCE, 0x05, 0x55, 0xCC, 0x9B, 0xCC, 0x83,
+	0xCE, 0x05, 0x55, 0xCC, 0x9B, 0xCC, 0x89, 0xCE,
+	0x05, 0x55, 0xCC, 0x9B, 0xCC, 0xA3, 0xBA, 0x05,
+	0x61, 0xCC, 0x82, 0xCC, 0x80, 0xCE, 0x05, 0x61,
+	0xCC, 0x82, 0xCC, 0x81, 0xCE, 0x05, 0x61, 0xCC,
+	0x82, 0xCC, 0x83, 0xCE, 0x05, 0x61, 0xCC, 0x82,
+	0xCC, 0x89, 0xCE, 0x05, 0x61, 0xCC, 0x86, 0xCC,
+	0x80, 0xCE, 0x05, 0x61, 0xCC, 0x86, 0xCC, 0x81,
+	// Bytes 3a40 - 3a7f
+	0xCE, 0x05, 0x61, 0xCC, 0x86, 0xCC, 0x83, 0xCE,
+	0x05, 0x61, 0xCC, 0x86, 0xCC, 0x89, 0xCE, 0x05,
+	0x61, 0xCC, 0x87, 0xCC, 0x84, 0xCE, 0x05, 0x61,
+	0xCC, 0x88, 0xCC, 0x84, 0xCE, 0x05, 0x61, 0xCC,
+	0x8A, 0xCC, 0x81, 0xCE, 0x05, 0x61, 0xCC, 0xA3,
+	0xCC, 0x82, 0xCE, 0x05, 0x61, 0xCC, 0xA3, 0xCC,
+	0x86, 0xCE, 0x05, 0x63, 0xCC, 0xA7, 0xCC, 0x81,
+	0xCE, 0x05, 0x65, 0xCC, 0x82, 0xCC, 0x80, 0xCE,
+	// Bytes 3a80 - 3abf
+	0x05, 0x65, 0xCC, 0x82, 0xCC, 0x81, 0xCE, 0x05,
+	0x65, 0xCC, 0x82, 0xCC, 0x83, 0xCE, 0x05, 0x65,
+	0xCC, 0x82, 0xCC, 0x89, 0xCE, 0x05, 0x65, 0xCC,
+	0x84, 0xCC, 0x80, 0xCE, 0x05, 0x65, 0xCC, 0x84,
+	0xCC, 0x81, 0xCE, 0x05, 0x65, 0xCC, 0xA3, 0xCC,
+	0x82, 0xCE, 0x05, 0x65, 0xCC, 0xA7, 0xCC, 0x86,
+	0xCE, 0x05, 0x69, 0xCC, 0x88, 0xCC, 0x81, 0xCE,
+	0x05, 0x6C, 0xCC, 0xA3, 0xCC, 0x84, 0xCE, 0x05,
+	// Bytes 3ac0 - 3aff
+	0x6F, 0xCC, 0x82, 0xCC, 0x80, 0xCE, 0x05, 0x6F,
+	0xCC, 0x82, 0xCC, 0x81, 0xCE, 0x05, 0x6F, 0xCC,
+	0x82, 0xCC, 0x83, 0xCE, 0x05, 0x6F, 0xCC, 0x82,
+	0xCC, 0x89, 0xCE, 0x05, 0x6F, 0xCC, 0x83, 0xCC,
+	0x81, 0xCE, 0x05, 0x6F, 0xCC, 0x83, 0xCC, 0x84,
+	0xCE, 0x05, 0x6F, 0xCC, 0x83, 0xCC, 0x88, 0xCE,
+	0x05, 0x6F, 0xCC, 0x84, 0xCC, 0x80, 0xCE, 0x05,
+	0x6F, 0xCC, 0x84, 0xCC, 0x81, 0xCE, 0x05, 0x6F,
+	// Bytes 3b00 - 3b3f
+	0xCC, 0x87, 0xCC, 0x84, 0xCE, 0x05, 0x6F, 0xCC,
+	0x88, 0xCC, 0x84, 0xCE, 0x05, 0x6F, 0xCC, 0x9B,
+	0xCC, 0x80, 0xCE, 0x05, 0x6F, 0xCC, 0x9B, 0xCC,
+	0x81, 0xCE, 0x05, 0x6F, 0xCC, 0x9B, 0xCC, 0x83,
+	0xCE, 0x05, 0x6F, 0xCC, 0x9B, 0xCC, 0x89, 0xCE,
+	0x05, 0x6F, 0xCC, 0x9B, 0xCC, 0xA3, 0xBA, 0x05,
+	0x6F, 0xCC, 0xA3, 0xCC, 0x82, 0xCE, 0x05, 0x6F,
+	0xCC, 0xA8, 0xCC, 0x84, 0xCE, 0x05, 0x72, 0xCC,
+	// Bytes 3b40 - 3b7f
+	0xA3, 0xCC, 0x84, 0xCE, 0x05, 0x73, 0xCC, 0x81,
+	0xCC, 0x87, 0xCE, 0x05, 0x73, 0xCC, 0x8C, 0xCC,
+	0x87, 0xCE, 0x05, 0x73, 0xCC, 0xA3, 0xCC, 0x87,
+	0xCE, 0x05, 0x75, 0xCC, 0x83, 0xCC, 0x81, 0xCE,
+	0x05, 0x75, 0xCC, 0x84, 0xCC, 0x88, 0xCE, 0x05,
+	0x75, 0xCC, 0x88, 0xCC, 0x80, 0xCE, 0x05, 0x75,
+	0xCC, 0x88, 0xCC, 0x81, 0xCE, 0x05, 0x75, 0xCC,
+	0x88, 0xCC, 0x84, 0xCE, 0x05, 0x75, 0xCC, 0x88,
+	// Bytes 3b80 - 3bbf
+	0xCC, 0x8C, 0xCE, 0x05, 0x75, 0xCC, 0x9B, 0xCC,
+	0x80, 0xCE, 0x05, 0x75, 0xCC, 0x9B, 0xCC, 0x81,
+	0xCE, 0x05, 0x75, 0xCC, 0x9B, 0xCC, 0x83, 0xCE,
+	0x05, 0x75, 0xCC, 0x9B, 0xCC, 0x89, 0xCE, 0x05,
+	0x75, 0xCC, 0x9B, 0xCC, 0xA3, 0xBA, 0x05, 0xE1,
+	0xBE, 0xBF, 0xCC, 0x80, 0xCE, 0x05, 0xE1, 0xBE,
+	0xBF, 0xCC, 0x81, 0xCE, 0x05, 0xE1, 0xBE, 0xBF,
+	0xCD, 0x82, 0xCE, 0x05, 0xE1, 0xBF, 0xBE, 0xCC,
+	// Bytes 3bc0 - 3bff
+	0x80, 0xCE, 0x05, 0xE1, 0xBF, 0xBE, 0xCC, 0x81,
+	0xCE, 0x05, 0xE1, 0xBF, 0xBE, 0xCD, 0x82, 0xCE,
+	0x05, 0xE2, 0x86, 0x90, 0xCC, 0xB8, 0x05, 0x05,
+	0xE2, 0x86, 0x92, 0xCC, 0xB8, 0x05, 0x05, 0xE2,
+	0x86, 0x94, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x87,
+	0x90, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x87, 0x92,
+	0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x87, 0x94, 0xCC,
+	0xB8, 0x05, 0x05, 0xE2, 0x88, 0x83, 0xCC, 0xB8,
+	// Bytes 3c00 - 3c3f
+	0x05, 0x05, 0xE2, 0x88, 0x88, 0xCC, 0xB8, 0x05,
+	0x05, 0xE2, 0x88, 0x8B, 0xCC, 0xB8, 0x05, 0x05,
+	0xE2, 0x88, 0xA3, 0xCC, 0xB8, 0x05, 0x05, 0xE2,
+	0x88, 0xA5, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x88,
+	0xBC, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x89, 0x83,
+	0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x89, 0x85, 0xCC,
+	0xB8, 0x05, 0x05, 0xE2, 0x89, 0x88, 0xCC, 0xB8,
+	0x05, 0x05, 0xE2, 0x89, 0x8D, 0xCC, 0xB8, 0x05,
+	// Bytes 3c40 - 3c7f
+	0x05, 0xE2, 0x89, 0xA1, 0xCC, 0xB8, 0x05, 0x05,
+	0xE2, 0x89, 0xA4, 0xCC, 0xB8, 0x05, 0x05, 0xE2,
+	0x89, 0xA5, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x89,
+	0xB2, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x89, 0xB3,
+	0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x89, 0xB6, 0xCC,
+	0xB8, 0x05, 0x05, 0xE2, 0x89, 0xB7, 0xCC, 0xB8,
+	0x05, 0x05, 0xE2, 0x89, 0xBA, 0xCC, 0xB8, 0x05,
+	0x05, 0xE2, 0x89, 0xBB, 0xCC, 0xB8, 0x05, 0x05,
+	// Bytes 3c80 - 3cbf
+	0xE2, 0x89, 0xBC, 0xCC, 0xB8, 0x05, 0x05, 0xE2,
+	0x89, 0xBD, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A,
+	0x82, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A, 0x83,
+	0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A, 0x86, 0xCC,
+	0xB8, 0x05, 0x05, 0xE2, 0x8A, 0x87, 0xCC, 0xB8,
+	0x05, 0x05, 0xE2, 0x8A, 0x91, 0xCC, 0xB8, 0x05,
+	0x05, 0xE2, 0x8A, 0x92, 0xCC, 0xB8, 0x05, 0x05,
+	0xE2, 0x8A, 0xA2, 0xCC, 0xB8, 0x05, 0x05, 0xE2,
+	// Bytes 3cc0 - 3cff
+	0x8A, 0xA8, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A,
+	0xA9, 0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A, 0xAB,
+	0xCC, 0xB8, 0x05, 0x05, 0xE2, 0x8A, 0xB2, 0xCC,
+	0xB8, 0x05, 0x05, 0xE2, 0x8A, 0xB3, 0xCC, 0xB8,
+	0x05, 0x05, 0xE2, 0x8A, 0xB4, 0xCC, 0xB8, 0x05,
+	0x05, 0xE2, 0x8A, 0xB5, 0xCC, 0xB8, 0x05, 0x06,
+	0xCE, 0x91, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0x91, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	// Bytes 3d00 - 3d3f
+	0xCE, 0x95, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x95, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0x95, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x95, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0x97, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0x97, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0x99, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x99, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	// Bytes 3d40 - 3d7f
+	0xCE, 0x99, 0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0x99, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x99, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0x99, 0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0x9F, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x9F, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0x9F, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0x9F, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	// Bytes 3d80 - 3dbf
+	0xCE, 0xA5, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xA5, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xA5, 0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0xA9, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xA9, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB1, 0xCC, 0x80, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB1, 0xCC, 0x81, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB1, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	// Bytes 3dc0 - 3dff
+	0xCE, 0xB1, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB1, 0xCD, 0x82, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB5, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xB5, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xB5, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xB5, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xB7, 0xCC, 0x80, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB7, 0xCC, 0x81, 0xCD, 0x85, 0xDE, 0x06,
+	// Bytes 3e00 - 3e3f
+	0xCE, 0xB7, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB7, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB7, 0xCD, 0x82, 0xCD, 0x85, 0xDE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x88, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x88, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x88, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	// Bytes 3e40 - 3e7f
+	0xCE, 0xB9, 0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xB9, 0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x06,
+	0xCE, 0xBF, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xBF, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	0xCE, 0xBF, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCE, 0xBF, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	// Bytes 3e80 - 3ebf
+	0xCF, 0x85, 0xCC, 0x88, 0xCC, 0x80, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x88, 0xCC, 0x81, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x88, 0xCD, 0x82, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x06,
+	0xCF, 0x85, 0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x06,
+	// Bytes 3ec0 - 3eff
+	0xCF, 0x85, 0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x06,
+	0xCF, 0x89, 0xCC, 0x80, 0xCD, 0x85, 0xDE, 0x06,
+	0xCF, 0x89, 0xCC, 0x81, 0xCD, 0x85, 0xDE, 0x06,
+	0xCF, 0x89, 0xCC, 0x93, 0xCD, 0x85, 0xDE, 0x06,
+	0xCF, 0x89, 0xCC, 0x94, 0xCD, 0x85, 0xDE, 0x06,
+	0xCF, 0x89, 0xCD, 0x82, 0xCD, 0x85, 0xDE, 0x06,
+	0xE0, 0xA4, 0xA8, 0xE0, 0xA4, 0xBC, 0x0D, 0x06,
+	0xE0, 0xA4, 0xB0, 0xE0, 0xA4, 0xBC, 0x0D, 0x06,
+	// Bytes 3f00 - 3f3f
+	0xE0, 0xA4, 0xB3, 0xE0, 0xA4, 0xBC, 0x0D, 0x06,
+	0xE0, 0xB1, 0x86, 0xE0, 0xB1, 0x96, 0x89, 0x06,
+	0xE0, 0xB7, 0x99, 0xE0, 0xB7, 0x8A, 0x15, 0x06,
+	0xE3, 0x81, 0x86, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x8B, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x8D, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x8F, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x91, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	// Bytes 3f40 - 3f7f
+	0xE3, 0x81, 0x93, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x95, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x97, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x99, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x9B, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x9D, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0x9F, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xA1, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	// Bytes 3f80 - 3fbf
+	0xE3, 0x81, 0xA4, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xA6, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xA8, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xAF, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xAF, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x81, 0xB2, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xB2, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x81, 0xB5, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	// Bytes 3fc0 - 3fff
+	0xE3, 0x81, 0xB5, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x81, 0xB8, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xB8, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x81, 0xBB, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x81, 0xBB, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x82, 0x9D, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xA6, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xAB, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	// Bytes 4000 - 403f
+	0xE3, 0x82, 0xAD, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xAF, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xB1, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xB3, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xB5, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xB7, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xB9, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xBB, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	// Bytes 4040 - 407f
+	0xE3, 0x82, 0xBD, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x82, 0xBF, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x81, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x84, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x86, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x88, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x8F, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x8F, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	// Bytes 4080 - 40bf
+	0xE3, 0x83, 0x92, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x92, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x83, 0x95, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x95, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x83, 0x98, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x98, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	0xE3, 0x83, 0x9B, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0x9B, 0xE3, 0x82, 0x9A, 0x11, 0x06,
+	// Bytes 40c0 - 40ff
+	0xE3, 0x83, 0xAF, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0xB0, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0xB1, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0xB2, 0xE3, 0x82, 0x99, 0x11, 0x06,
+	0xE3, 0x83, 0xBD, 0xE3, 0x82, 0x99, 0x11, 0x08,
+	0xCE, 0x91, 0xCC, 0x93, 0xCC, 0x80, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0x91, 0xCC, 0x93, 0xCC, 0x81,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x91, 0xCC, 0x93,
+	// Bytes 4100 - 413f
+	0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x91,
+	0xCC, 0x94, 0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0x91, 0xCC, 0x94, 0xCC, 0x81, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0x91, 0xCC, 0x94, 0xCD, 0x82,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x97, 0xCC, 0x93,
+	0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x97,
+	0xCC, 0x93, 0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0x97, 0xCC, 0x93, 0xCD, 0x82, 0xCD, 0x85,
+	// Bytes 4140 - 417f
+	0xDF, 0x08, 0xCE, 0x97, 0xCC, 0x94, 0xCC, 0x80,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x97, 0xCC, 0x94,
+	0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0x97,
+	0xCC, 0x94, 0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0xA9, 0xCC, 0x93, 0xCC, 0x80, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0xA9, 0xCC, 0x93, 0xCC, 0x81,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xA9, 0xCC, 0x93,
+	0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xA9,
+	// Bytes 4180 - 41bf
+	0xCC, 0x94, 0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0xA9, 0xCC, 0x94, 0xCC, 0x81, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0xA9, 0xCC, 0x94, 0xCD, 0x82,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB1, 0xCC, 0x93,
+	0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB1,
+	0xCC, 0x93, 0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0xB1, 0xCC, 0x93, 0xCD, 0x82, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0xB1, 0xCC, 0x94, 0xCC, 0x80,
+	// Bytes 41c0 - 41ff
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB1, 0xCC, 0x94,
+	0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB1,
+	0xCC, 0x94, 0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08,
+	0xCE, 0xB7, 0xCC, 0x93, 0xCC, 0x80, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0xB7, 0xCC, 0x93, 0xCC, 0x81,
+	0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB7, 0xCC, 0x93,
+	0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08, 0xCE, 0xB7,
+	0xCC, 0x94, 0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08,
+	// Bytes 4200 - 423f
+	0xCE, 0xB7, 0xCC, 0x94, 0xCC, 0x81, 0xCD, 0x85,
+	0xDF, 0x08, 0xCE, 0xB7, 0xCC, 0x94, 0xCD, 0x82,
+	0xCD, 0x85, 0xDF, 0x08, 0xCF, 0x89, 0xCC, 0x93,
+	0xCC, 0x80, 0xCD, 0x85, 0xDF, 0x08, 0xCF, 0x89,
+	0xCC, 0x93, 0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08,
+	0xCF, 0x89, 0xCC, 0x93, 0xCD, 0x82, 0xCD, 0x85,
+	0xDF, 0x08, 0xCF, 0x89, 0xCC, 0x94, 0xCC, 0x80,
+	0xCD, 0x85, 0xDF, 0x08, 0xCF, 0x89, 0xCC, 0x94,
+	// Bytes 4240 - 427f
+	0xCC, 0x81, 0xCD, 0x85, 0xDF, 0x08, 0xCF, 0x89,
+	0xCC, 0x94, 0xCD, 0x82, 0xCD, 0x85, 0xDF, 0x08,
+	0xF0, 0x91, 0x82, 0x99, 0xF0, 0x91, 0x82, 0xBA,
+	0x0D, 0x08, 0xF0, 0x91, 0x82, 0x9B, 0xF0, 0x91,
+	0x82, 0xBA, 0x0D, 0x08, 0xF0, 0x91, 0x82, 0xA5,
+	0xF0, 0x91, 0x82, 0xBA, 0x0D, 0x42, 0xC2, 0xB4,
+	0x01, 0x43, 0x20, 0xCC, 0x81, 0xCD, 0x43, 0x20,
+	0xCC, 0x83, 0xCD, 0x43, 0x20, 0xCC, 0x84, 0xCD,
+	// Bytes 4280 - 42bf
+	0x43, 0x20, 0xCC, 0x85, 0xCD, 0x43, 0x20, 0xCC,
+	0x86, 0xCD, 0x43, 0x20, 0xCC, 0x87, 0xCD, 0x43,
+	0x20, 0xCC, 0x88, 0xCD, 0x43, 0x20, 0xCC, 0x8A,
+	0xCD, 0x43, 0x20, 0xCC, 0x8B, 0xCD, 0x43, 0x20,
+	0xCC, 0x93, 0xCD, 0x43, 0x20, 0xCC, 0x94, 0xCD,
+	0x43, 0x20, 0xCC, 0xA7, 0xA9, 0x43, 0x20, 0xCC,
+	0xA8, 0xA9, 0x43, 0x20, 0xCC, 0xB3, 0xB9, 0x43,
+	0x20, 0xCD, 0x82, 0xCD, 0x43, 0x20, 0xCD, 0x85,
+	// Bytes 42c0 - 42ff
+	0xDD, 0x43, 0x20, 0xD9, 0x8B, 0x5D, 0x43, 0x20,
+	0xD9, 0x8C, 0x61, 0x43, 0x20, 0xD9, 0x8D, 0x65,
+	0x43, 0x20, 0xD9, 0x8E, 0x69, 0x43, 0x20, 0xD9,
+	0x8F, 0x6D, 0x43, 0x20, 0xD9, 0x90, 0x71, 0x43,
+	0x20, 0xD9, 0x91, 0x75, 0x43, 0x20, 0xD9, 0x92,
+	0x79, 0x43, 0x41, 0xCC, 0x8A, 0xCD, 0x43, 0x73,
+	0xCC, 0x87, 0xCD, 0x44, 0x20, 0xE3, 0x82, 0x99,
+	0x11, 0x44, 0x20, 0xE3, 0x82, 0x9A, 0x11, 0x44,
+	// Bytes 4300 - 433f
+	0xC2, 0xA8, 0xCC, 0x81, 0xCE, 0x44, 0xCE, 0x91,
+	0xCC, 0x81, 0xCD, 0x44, 0xCE, 0x95, 0xCC, 0x81,
+	0xCD, 0x44, 0xCE, 0x97, 0xCC, 0x81, 0xCD, 0x44,
+	0xCE, 0x99, 0xCC, 0x81, 0xCD, 0x44, 0xCE, 0x9F,
+	0xCC, 0x81, 0xCD, 0x44, 0xCE, 0xA5, 0xCC, 0x81,
+	0xCD, 0x44, 0xCE, 0xA5, 0xCC, 0x88, 0xCD, 0x44,
+	0xCE, 0xA9, 0xCC, 0x81, 0xCD, 0x44, 0xCE, 0xB1,
+	0xCC, 0x81, 0xCD, 0x44, 0xCE, 0xB5, 0xCC, 0x81,
+	// Bytes 4340 - 437f
+	0xCD, 0x44, 0xCE, 0xB7, 0xCC, 0x81, 0xCD, 0x44,
+	0xCE, 0xB9, 0xCC, 0x81, 0xCD, 0x44, 0xCE, 0xBF,
+	0xCC, 0x81, 0xCD, 0x44, 0xCF, 0x85, 0xCC, 0x81,
+	0xCD, 0x44, 0xCF, 0x89, 0xCC, 0x81, 0xCD, 0x44,
+	0xD7, 0x90, 0xD6, 0xB7, 0x35, 0x44, 0xD7, 0x90,
+	0xD6, 0xB8, 0x39, 0x44, 0xD7, 0x90, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0x91, 0xD6, 0xBC, 0x45, 0x44,
+	0xD7, 0x91, 0xD6, 0xBF, 0x4D, 0x44, 0xD7, 0x92,
+	// Bytes 4380 - 43bf
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0x93, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0x94, 0xD6, 0xBC, 0x45, 0x44,
+	0xD7, 0x95, 0xD6, 0xB9, 0x3D, 0x44, 0xD7, 0x95,
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0x96, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0x98, 0xD6, 0xBC, 0x45, 0x44,
+	0xD7, 0x99, 0xD6, 0xB4, 0x29, 0x44, 0xD7, 0x99,
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0x9A, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0x9B, 0xD6, 0xBC, 0x45, 0x44,
+	// Bytes 43c0 - 43ff
+	0xD7, 0x9B, 0xD6, 0xBF, 0x4D, 0x44, 0xD7, 0x9C,
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0x9E, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0xA0, 0xD6, 0xBC, 0x45, 0x44,
+	0xD7, 0xA1, 0xD6, 0xBC, 0x45, 0x44, 0xD7, 0xA3,
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0xA4, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0xA4, 0xD6, 0xBF, 0x4D, 0x44,
+	0xD7, 0xA6, 0xD6, 0xBC, 0x45, 0x44, 0xD7, 0xA7,
+	0xD6, 0xBC, 0x45, 0x44, 0xD7, 0xA8, 0xD6, 0xBC,
+	// Bytes 4400 - 443f
+	0x45, 0x44, 0xD7, 0xA9, 0xD6, 0xBC, 0x45, 0x44,
+	0xD7, 0xA9, 0xD7, 0x81, 0x51, 0x44, 0xD7, 0xA9,
+	0xD7, 0x82, 0x55, 0x44, 0xD7, 0xAA, 0xD6, 0xBC,
+	0x45, 0x44, 0xD7, 0xB2, 0xD6, 0xB7, 0x35, 0x44,
+	0xD8, 0xA7, 0xD9, 0x8B, 0x5D, 0x44, 0xD8, 0xA7,
+	0xD9, 0x93, 0xCD, 0x44, 0xD8, 0xA7, 0xD9, 0x94,
+	0xCD, 0x44, 0xD8, 0xA7, 0xD9, 0x95, 0xB9, 0x44,
+	0xD8, 0xB0, 0xD9, 0xB0, 0x7D, 0x44, 0xD8, 0xB1,
+	// Bytes 4440 - 447f
+	0xD9, 0xB0, 0x7D, 0x44, 0xD9, 0x80, 0xD9, 0x8B,
+	0x5D, 0x44, 0xD9, 0x80, 0xD9, 0x8E, 0x69, 0x44,
+	0xD9, 0x80, 0xD9, 0x8F, 0x6D, 0x44, 0xD9, 0x80,
+	0xD9, 0x90, 0x71, 0x44, 0xD9, 0x80, 0xD9, 0x91,
+	0x75, 0x44, 0xD9, 0x80, 0xD9, 0x92, 0x79, 0x44,
+	0xD9, 0x87, 0xD9, 0xB0, 0x7D, 0x44, 0xD9, 0x88,
+	0xD9, 0x94, 0xCD, 0x44, 0xD9, 0x89, 0xD9, 0xB0,
+	0x7D, 0x44, 0xD9, 0x8A, 0xD9, 0x94, 0xCD, 0x44,
+	// Bytes 4480 - 44bf
+	0xDB, 0x92, 0xD9, 0x94, 0xCD, 0x44, 0xDB, 0x95,
+	0xD9, 0x94, 0xCD, 0x45, 0x20, 0xCC, 0x88, 0xCC,
+	0x80, 0xCE, 0x45, 0x20, 0xCC, 0x88, 0xCC, 0x81,
+	0xCE, 0x45, 0x20, 0xCC, 0x88, 0xCD, 0x82, 0xCE,
+	0x45, 0x20, 0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x45,
+	0x20, 0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x45, 0x20,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x45, 0x20, 0xCC,
+	0x94, 0xCC, 0x80, 0xCE, 0x45, 0x20, 0xCC, 0x94,
+	// Bytes 44c0 - 44ff
+	0xCC, 0x81, 0xCE, 0x45, 0x20, 0xCC, 0x94, 0xCD,
+	0x82, 0xCE, 0x45, 0x20, 0xD9, 0x8C, 0xD9, 0x91,
+	0x76, 0x45, 0x20, 0xD9, 0x8D, 0xD9, 0x91, 0x76,
+	0x45, 0x20, 0xD9, 0x8E, 0xD9, 0x91, 0x76, 0x45,
+	0x20, 0xD9, 0x8F, 0xD9, 0x91, 0x76, 0x45, 0x20,
+	0xD9, 0x90, 0xD9, 0x91, 0x76, 0x45, 0x20, 0xD9,
+	0x91, 0xD9, 0xB0, 0x7E, 0x45, 0xE2, 0xAB, 0x9D,
+	0xCC, 0xB8, 0x05, 0x46, 0xCE, 0xB9, 0xCC, 0x88,
+	// Bytes 4500 - 453f
+	0xCC, 0x81, 0xCE, 0x46, 0xCF, 0x85, 0xCC, 0x88,
+	0xCC, 0x81, 0xCE, 0x46, 0xD7, 0xA9, 0xD6, 0xBC,
+	0xD7, 0x81, 0x52, 0x46, 0xD7, 0xA9, 0xD6, 0xBC,
+	0xD7, 0x82, 0x56, 0x46, 0xD9, 0x80, 0xD9, 0x8E,
+	0xD9, 0x91, 0x76, 0x46, 0xD9, 0x80, 0xD9, 0x8F,
+	0xD9, 0x91, 0x76, 0x46, 0xD9, 0x80, 0xD9, 0x90,
+	0xD9, 0x91, 0x76, 0x46, 0xE0, 0xA4, 0x95, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0x96, 0xE0,
+	// Bytes 4540 - 457f
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0x97, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0x9C, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0xA1, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0xA2, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0xAB, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA4, 0xAF, 0xE0,
+	0xA4, 0xBC, 0x0D, 0x46, 0xE0, 0xA6, 0xA1, 0xE0,
+	0xA6, 0xBC, 0x0D, 0x46, 0xE0, 0xA6, 0xA2, 0xE0,
+	// Bytes 4580 - 45bf
+	0xA6, 0xBC, 0x0D, 0x46, 0xE0, 0xA6, 0xAF, 0xE0,
+	0xA6, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0x96, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0x97, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0x9C, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0xAB, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0xB2, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xA8, 0xB8, 0xE0,
+	0xA8, 0xBC, 0x0D, 0x46, 0xE0, 0xAC, 0xA1, 0xE0,
+	// Bytes 45c0 - 45ff
+	0xAC, 0xBC, 0x0D, 0x46, 0xE0, 0xAC, 0xA2, 0xE0,
+	0xAC, 0xBC, 0x0D, 0x46, 0xE0, 0xBE, 0xB2, 0xE0,
+	0xBE, 0x80, 0xA1, 0x46, 0xE0, 0xBE, 0xB3, 0xE0,
+	0xBE, 0x80, 0xA1, 0x46, 0xE3, 0x83, 0x86, 0xE3,
+	0x82, 0x99, 0x11, 0x48, 0xF0, 0x9D, 0x85, 0x97,
+	0xF0, 0x9D, 0x85, 0xA5, 0xB1, 0x48, 0xF0, 0x9D,
+	0x85, 0x98, 0xF0, 0x9D, 0x85, 0xA5, 0xB1, 0x48,
+	0xF0, 0x9D, 0x86, 0xB9, 0xF0, 0x9D, 0x85, 0xA5,
+	// Bytes 4600 - 463f
+	0xB1, 0x48, 0xF0, 0x9D, 0x86, 0xBA, 0xF0, 0x9D,
+	0x85, 0xA5, 0xB1, 0x49, 0xE0, 0xBE, 0xB2, 0xE0,
+	0xBD, 0xB1, 0xE0, 0xBE, 0x80, 0xA2, 0x49, 0xE0,
+	0xBE, 0xB3, 0xE0, 0xBD, 0xB1, 0xE0, 0xBE, 0x80,
+	0xA2, 0x4C, 0xF0, 0x9D, 0x85, 0x98, 0xF0, 0x9D,
+	0x85, 0xA5, 0xF0, 0x9D, 0x85, 0xAE, 0xB2, 0x4C,
+	0xF0, 0x9D, 0x85, 0x98, 0xF0, 0x9D, 0x85, 0xA5,
+	0xF0, 0x9D, 0x85, 0xAF, 0xB2, 0x4C, 0xF0, 0x9D,
+	// Bytes 4640 - 467f
+	0x85, 0x98, 0xF0, 0x9D, 0x85, 0xA5, 0xF0, 0x9D,
+	0x85, 0xB0, 0xB2, 0x4C, 0xF0, 0x9D, 0x85, 0x98,
+	0xF0, 0x9D, 0x85, 0xA5, 0xF0, 0x9D, 0x85, 0xB1,
+	0xB2, 0x4C, 0xF0, 0x9D, 0x85, 0x98, 0xF0, 0x9D,
+	0x85, 0xA5, 0xF0, 0x9D, 0x85, 0xB2, 0xB2, 0x4C,
+	0xF0, 0x9D, 0x86, 0xB9, 0xF0, 0x9D, 0x85, 0xA5,
+	0xF0, 0x9D, 0x85, 0xAE, 0xB2, 0x4C, 0xF0, 0x9D,
+	0x86, 0xB9, 0xF0, 0x9D, 0x85, 0xA5, 0xF0, 0x9D,
+	// Bytes 4680 - 46bf
+	0x85, 0xAF, 0xB2, 0x4C, 0xF0, 0x9D, 0x86, 0xBA,
+	0xF0, 0x9D, 0x85, 0xA5, 0xF0, 0x9D, 0x85, 0xAE,
+	0xB2, 0x4C, 0xF0, 0x9D, 0x86, 0xBA, 0xF0, 0x9D,
+	0x85, 0xA5, 0xF0, 0x9D, 0x85, 0xAF, 0xB2, 0x83,
+	0x41, 0xCC, 0x82, 0xCD, 0x83, 0x41, 0xCC, 0x86,
+	0xCD, 0x83, 0x41, 0xCC, 0x87, 0xCD, 0x83, 0x41,
+	0xCC, 0x88, 0xCD, 0x83, 0x41, 0xCC, 0x8A, 0xCD,
+	0x83, 0x41, 0xCC, 0xA3, 0xB9, 0x83, 0x43, 0xCC,
+	// Bytes 46c0 - 46ff
+	0xA7, 0xA9, 0x83, 0x45, 0xCC, 0x82, 0xCD, 0x83,
+	0x45, 0xCC, 0x84, 0xCD, 0x83, 0x45, 0xCC, 0xA3,
+	0xB9, 0x83, 0x45, 0xCC, 0xA7, 0xA9, 0x83, 0x49,
+	0xCC, 0x88, 0xCD, 0x83, 0x4C, 0xCC, 0xA3, 0xB9,
+	0x83, 0x4F, 0xCC, 0x82, 0xCD, 0x83, 0x4F, 0xCC,
+	0x83, 0xCD, 0x83, 0x4F, 0xCC, 0x84, 0xCD, 0x83,
+	0x4F, 0xCC, 0x87, 0xCD, 0x83, 0x4F, 0xCC, 0x88,
+	0xCD, 0x83, 0x4F, 0xCC, 0x9B, 0xB1, 0x83, 0x4F,
+	// Bytes 4700 - 473f
+	0xCC, 0xA3, 0xB9, 0x83, 0x4F, 0xCC, 0xA8, 0xA9,
+	0x83, 0x52, 0xCC, 0xA3, 0xB9, 0x83, 0x53, 0xCC,
+	0x81, 0xCD, 0x83, 0x53, 0xCC, 0x8C, 0xCD, 0x83,
+	0x53, 0xCC, 0xA3, 0xB9, 0x83, 0x55, 0xCC, 0x83,
+	0xCD, 0x83, 0x55, 0xCC, 0x84, 0xCD, 0x83, 0x55,
+	0xCC, 0x88, 0xCD, 0x83, 0x55, 0xCC, 0x9B, 0xB1,
+	0x83, 0x61, 0xCC, 0x82, 0xCD, 0x83, 0x61, 0xCC,
+	0x86, 0xCD, 0x83, 0x61, 0xCC, 0x87, 0xCD, 0x83,
+	// Bytes 4740 - 477f
+	0x61, 0xCC, 0x88, 0xCD, 0x83, 0x61, 0xCC, 0x8A,
+	0xCD, 0x83, 0x61, 0xCC, 0xA3, 0xB9, 0x83, 0x63,
+	0xCC, 0xA7, 0xA9, 0x83, 0x65, 0xCC, 0x82, 0xCD,
+	0x83, 0x65, 0xCC, 0x84, 0xCD, 0x83, 0x65, 0xCC,
+	0xA3, 0xB9, 0x83, 0x65, 0xCC, 0xA7, 0xA9, 0x83,
+	0x69, 0xCC, 0x88, 0xCD, 0x83, 0x6C, 0xCC, 0xA3,
+	0xB9, 0x83, 0x6F, 0xCC, 0x82, 0xCD, 0x83, 0x6F,
+	0xCC, 0x83, 0xCD, 0x83, 0x6F, 0xCC, 0x84, 0xCD,
+	// Bytes 4780 - 47bf
+	0x83, 0x6F, 0xCC, 0x87, 0xCD, 0x83, 0x6F, 0xCC,
+	0x88, 0xCD, 0x83, 0x6F, 0xCC, 0x9B, 0xB1, 0x83,
+	0x6F, 0xCC, 0xA3, 0xB9, 0x83, 0x6F, 0xCC, 0xA8,
+	0xA9, 0x83, 0x72, 0xCC, 0xA3, 0xB9, 0x83, 0x73,
+	0xCC, 0x81, 0xCD, 0x83, 0x73, 0xCC, 0x8C, 0xCD,
+	0x83, 0x73, 0xCC, 0xA3, 0xB9, 0x83, 0x75, 0xCC,
+	0x83, 0xCD, 0x83, 0x75, 0xCC, 0x84, 0xCD, 0x83,
+	0x75, 0xCC, 0x88, 0xCD, 0x83, 0x75, 0xCC, 0x9B,
+	// Bytes 47c0 - 47ff
+	0xB1, 0x84, 0xCE, 0x91, 0xCC, 0x93, 0xCD, 0x84,
+	0xCE, 0x91, 0xCC, 0x94, 0xCD, 0x84, 0xCE, 0x95,
+	0xCC, 0x93, 0xCD, 0x84, 0xCE, 0x95, 0xCC, 0x94,
+	0xCD, 0x84, 0xCE, 0x97, 0xCC, 0x93, 0xCD, 0x84,
+	0xCE, 0x97, 0xCC, 0x94, 0xCD, 0x84, 0xCE, 0x99,
+	0xCC, 0x93, 0xCD, 0x84, 0xCE, 0x99, 0xCC, 0x94,
+	0xCD, 0x84, 0xCE, 0x9F, 0xCC, 0x93, 0xCD, 0x84,
+	0xCE, 0x9F, 0xCC, 0x94, 0xCD, 0x84, 0xCE, 0xA5,
+	// Bytes 4800 - 483f
+	0xCC, 0x94, 0xCD, 0x84, 0xCE, 0xA9, 0xCC, 0x93,
+	0xCD, 0x84, 0xCE, 0xA9, 0xCC, 0x94, 0xCD, 0x84,
+	0xCE, 0xB1, 0xCC, 0x80, 0xCD, 0x84, 0xCE, 0xB1,
+	0xCC, 0x81, 0xCD, 0x84, 0xCE, 0xB1, 0xCC, 0x93,
+	0xCD, 0x84, 0xCE, 0xB1, 0xCC, 0x94, 0xCD, 0x84,
+	0xCE, 0xB1, 0xCD, 0x82, 0xCD, 0x84, 0xCE, 0xB5,
+	0xCC, 0x93, 0xCD, 0x84, 0xCE, 0xB5, 0xCC, 0x94,
+	0xCD, 0x84, 0xCE, 0xB7, 0xCC, 0x80, 0xCD, 0x84,
+	// Bytes 4840 - 487f
+	0xCE, 0xB7, 0xCC, 0x81, 0xCD, 0x84, 0xCE, 0xB7,
+	0xCC, 0x93, 0xCD, 0x84, 0xCE, 0xB7, 0xCC, 0x94,
+	0xCD, 0x84, 0xCE, 0xB7, 0xCD, 0x82, 0xCD, 0x84,
+	0xCE, 0xB9, 0xCC, 0x88, 0xCD, 0x84, 0xCE, 0xB9,
+	0xCC, 0x93, 0xCD, 0x84, 0xCE, 0xB9, 0xCC, 0x94,
+	0xCD, 0x84, 0xCE, 0xBF, 0xCC, 0x93, 0xCD, 0x84,
+	0xCE, 0xBF, 0xCC, 0x94, 0xCD, 0x84, 0xCF, 0x85,
+	0xCC, 0x88, 0xCD, 0x84, 0xCF, 0x85, 0xCC, 0x93,
+	// Bytes 4880 - 48bf
+	0xCD, 0x84, 0xCF, 0x85, 0xCC, 0x94, 0xCD, 0x84,
+	0xCF, 0x89, 0xCC, 0x80, 0xCD, 0x84, 0xCF, 0x89,
+	0xCC, 0x81, 0xCD, 0x84, 0xCF, 0x89, 0xCC, 0x93,
+	0xCD, 0x84, 0xCF, 0x89, 0xCC, 0x94, 0xCD, 0x84,
+	0xCF, 0x89, 0xCD, 0x82, 0xCD, 0x86, 0xCE, 0x91,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0x91,
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0x91,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0x91,
+	// Bytes 48c0 - 48ff
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0x91,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0x91,
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0x97,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0x97,
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0x97,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0x97,
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0x97,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0x97,
+	// Bytes 4900 - 493f
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xA9,
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xB1,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xB1,
+	// Bytes 4940 - 497f
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xB1,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xB1,
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xB1,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xB1,
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xB7,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xB7,
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xB7,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCE, 0xB7,
+	// Bytes 4980 - 49bf
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCE, 0xB7,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCE, 0xB7,
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x86, 0xCF, 0x89,
+	0xCC, 0x93, 0xCC, 0x80, 0xCE, 0x86, 0xCF, 0x89,
+	0xCC, 0x93, 0xCC, 0x81, 0xCE, 0x86, 0xCF, 0x89,
+	0xCC, 0x93, 0xCD, 0x82, 0xCE, 0x86, 0xCF, 0x89,
+	0xCC, 0x94, 0xCC, 0x80, 0xCE, 0x86, 0xCF, 0x89,
+	0xCC, 0x94, 0xCC, 0x81, 0xCE, 0x86, 0xCF, 0x89,
+	// Bytes 49c0 - 49ff
+	0xCC, 0x94, 0xCD, 0x82, 0xCE, 0x42, 0xCC, 0x80,
+	0xCD, 0x33, 0x42, 0xCC, 0x81, 0xCD, 0x33, 0x42,
+	0xCC, 0x93, 0xCD, 0x33, 0x43, 0xE1, 0x85, 0xA1,
+	0x01, 0x00, 0x43, 0xE1, 0x85, 0xA2, 0x01, 0x00,
+	0x43, 0xE1, 0x85, 0xA3, 0x01, 0x00, 0x43, 0xE1,
+	0x85, 0xA4, 0x01, 0x00, 0x43, 0xE1, 0x85, 0xA5,
+	0x01, 0x00, 0x43, 0xE1, 0x85, 0xA6, 0x01, 0x00,
+	0x43, 0xE1, 0x85, 0xA7, 0x01, 0x00, 0x43, 0xE1,
+	// Bytes 4a00 - 4a3f
+	0x85, 0xA8, 0x01, 0x00, 0x43, 0xE1, 0x85, 0xA9,
+	0x01, 0x00, 0x43, 0xE1, 0x85, 0xAA, 0x01, 0x00,
+	0x43, 0xE1, 0x85, 0xAB, 0x01, 0x00, 0x43, 0xE1,
+	0x85, 0xAC, 0x01, 0x00, 0x43, 0xE1, 0x85, 0xAD,
+	0x01, 0x00, 0x43, 0xE1, 0x85, 0xAE, 0x01, 0x00,
+	0x43, 0xE1, 0x85, 0xAF, 0x01, 0x00, 0x43, 0xE1,
+	0x85, 0xB0, 0x01, 0x00, 0x43, 0xE1, 0x85, 0xB1,
+	0x01, 0x00, 0x43, 0xE1, 0x85, 0xB2, 0x01, 0x00,
+	// Bytes 4a40 - 4a7f
+	0x43, 0xE1, 0x85, 0xB3, 0x01, 0x00, 0x43, 0xE1,
+	0x85, 0xB4, 0x01, 0x00, 0x43, 0xE1, 0x85, 0xB5,
+	0x01, 0x00, 0x43, 0xE1, 0x86, 0xAA, 0x01, 0x00,
+	0x43, 0xE1, 0x86, 0xAC, 0x01, 0x00, 0x43, 0xE1,
+	0x86, 0xAD, 0x01, 0x00, 0x43, 0xE1, 0x86, 0xB0,
+	0x01, 0x00, 0x43, 0xE1, 0x86, 0xB1, 0x01, 0x00,
+	0x43, 0xE1, 0x86, 0xB2, 0x01, 0x00, 0x43, 0xE1,
+	0x86, 0xB3, 0x01, 0x00, 0x43, 0xE1, 0x86, 0xB4,
+	// Bytes 4a80 - 4abf
+	0x01, 0x00, 0x43, 0xE1, 0x86, 0xB5, 0x01, 0x00,
+	0x44, 0xCC, 0x88, 0xCC, 0x81, 0xCE, 0x33, 0x43,
+	0xE3, 0x82, 0x99, 0x11, 0x04, 0x43, 0xE3, 0x82,
+	0x9A, 0x11, 0x04, 0x46, 0xE0, 0xBD, 0xB1, 0xE0,
+	0xBD, 0xB2, 0xA2, 0x27, 0x46, 0xE0, 0xBD, 0xB1,
+	0xE0, 0xBD, 0xB4, 0xA6, 0x27, 0x46, 0xE0, 0xBD,
+	0xB1, 0xE0, 0xBE, 0x80, 0xA2, 0x27, 0x00, 0x01,
+}
+
+// lookup returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *nfcTrie) lookup(s []byte) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return nfcValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = nfcIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *nfcTrie) lookupUnsafe(s []byte) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return nfcValues[c0]
+	}
+	i := nfcIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = nfcIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = nfcIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// lookupString returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *nfcTrie) lookupString(s string) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return nfcValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := nfcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = nfcIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *nfcTrie) lookupStringUnsafe(s string) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return nfcValues[c0]
+	}
+	i := nfcIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = nfcIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = nfcIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// nfcTrie. Total size: 10680 bytes (10.43 KiB). Checksum: a555db76d4becdd2.
+type nfcTrie struct{}
+
+func newNfcTrie(i int) *nfcTrie {
+	return &nfcTrie{}
+}
+
+// lookupValue determines the type of block n and looks up the value for b.
+func (t *nfcTrie) lookupValue(n uint32, b byte) uint16 {
+	switch {
+	case n < 46:
+		return uint16(nfcValues[n<<6+uint32(b)])
+	default:
+		n -= 46
+		return uint16(nfcSparse.lookup(n, b))
+	}
+}
+
+// nfcValues: 48 blocks, 3072 entries, 6144 bytes
+// The third block is the zero block.
+var nfcValues = [3072]uint16{
+	// Block 0x0, offset 0x0
+	0x3c: 0xa000, 0x3d: 0xa000, 0x3e: 0xa000,
+	// Block 0x1, offset 0x40
+	0x41: 0xa000, 0x42: 0xa000, 0x43: 0xa000, 0x44: 0xa000, 0x45: 0xa000,
+	0x46: 0xa000, 0x47: 0xa000, 0x48: 0xa000, 0x49: 0xa000, 0x4a: 0xa000, 0x4b: 0xa000,
+	0x4c: 0xa000, 0x4d: 0xa000, 0x4e: 0xa000, 0x4f: 0xa000, 0x50: 0xa000,
+	0x52: 0xa000, 0x53: 0xa000, 0x54: 0xa000, 0x55: 0xa000, 0x56: 0xa000, 0x57: 0xa000,
+	0x58: 0xa000, 0x59: 0xa000, 0x5a: 0xa000,
+	0x61: 0xa000, 0x62: 0xa000, 0x63: 0xa000,
+	0x64: 0xa000, 0x65: 0xa000, 0x66: 0xa000, 0x67: 0xa000, 0x68: 0xa000, 0x69: 0xa000,
+	0x6a: 0xa000, 0x6b: 0xa000, 0x6c: 0xa000, 0x6d: 0xa000, 0x6e: 0xa000, 0x6f: 0xa000,
+	0x70: 0xa000, 0x72: 0xa000, 0x73: 0xa000, 0x74: 0xa000, 0x75: 0xa000,
+	0x76: 0xa000, 0x77: 0xa000, 0x78: 0xa000, 0x79: 0xa000, 0x7a: 0xa000,
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc0: 0x2f86, 0xc1: 0x2f8b, 0xc2: 0x469f, 0xc3: 0x2f90, 0xc4: 0x46ae, 0xc5: 0x46b3,
+	0xc6: 0xa000, 0xc7: 0x46bd, 0xc8: 0x2ff9, 0xc9: 0x2ffe, 0xca: 0x46c2, 0xcb: 0x3012,
+	0xcc: 0x3085, 0xcd: 0x308a, 0xce: 0x308f, 0xcf: 0x46d6, 0xd1: 0x311b,
+	0xd2: 0x313e, 0xd3: 0x3143, 0xd4: 0x46e0, 0xd5: 0x46e5, 0xd6: 0x46f4,
+	0xd8: 0xa000, 0xd9: 0x31ca, 0xda: 0x31cf, 0xdb: 0x31d4, 0xdc: 0x4726, 0xdd: 0x324c,
+	0xe0: 0x3292, 0xe1: 0x3297, 0xe2: 0x4730, 0xe3: 0x329c,
+	0xe4: 0x473f, 0xe5: 0x4744, 0xe6: 0xa000, 0xe7: 0x474e, 0xe8: 0x3305, 0xe9: 0x330a,
+	0xea: 0x4753, 0xeb: 0x331e, 0xec: 0x3396, 0xed: 0x339b, 0xee: 0x33a0, 0xef: 0x4767,
+	0xf1: 0x342c, 0xf2: 0x344f, 0xf3: 0x3454, 0xf4: 0x4771, 0xf5: 0x4776,
+	0xf6: 0x4785, 0xf8: 0xa000, 0xf9: 0x34e0, 0xfa: 0x34e5, 0xfb: 0x34ea,
+	0xfc: 0x47b7, 0xfd: 0x3567, 0xff: 0x3580,
+	// Block 0x4, offset 0x100
+	0x100: 0x2f95, 0x101: 0x32a1, 0x102: 0x46a4, 0x103: 0x4735, 0x104: 0x2fb3, 0x105: 0x32bf,
+	0x106: 0x2fc7, 0x107: 0x32d3, 0x108: 0x2fcc, 0x109: 0x32d8, 0x10a: 0x2fd1, 0x10b: 0x32dd,
+	0x10c: 0x2fd6, 0x10d: 0x32e2, 0x10e: 0x2fe0, 0x10f: 0x32ec,
+	0x112: 0x46c7, 0x113: 0x4758, 0x114: 0x3008, 0x115: 0x3314, 0x116: 0x300d, 0x117: 0x3319,
+	0x118: 0x302b, 0x119: 0x3337, 0x11a: 0x301c, 0x11b: 0x3328, 0x11c: 0x3044, 0x11d: 0x3350,
+	0x11e: 0x304e, 0x11f: 0x335a, 0x120: 0x3053, 0x121: 0x335f, 0x122: 0x305d, 0x123: 0x3369,
+	0x124: 0x3062, 0x125: 0x336e, 0x128: 0x3094, 0x129: 0x33a5,
+	0x12a: 0x3099, 0x12b: 0x33aa, 0x12c: 0x309e, 0x12d: 0x33af, 0x12e: 0x30c1, 0x12f: 0x33cd,
+	0x130: 0x30a3, 0x134: 0x30cb, 0x135: 0x33d7,
+	0x136: 0x30df, 0x137: 0x33f0, 0x139: 0x30e9, 0x13a: 0x33fa, 0x13b: 0x30f3,
+	0x13c: 0x3404, 0x13d: 0x30ee, 0x13e: 0x33ff,
+	// Block 0x5, offset 0x140
+	0x143: 0x3116, 0x144: 0x3427, 0x145: 0x312f,
+	0x146: 0x3440, 0x147: 0x3125, 0x148: 0x3436,
+	0x14c: 0x46ea, 0x14d: 0x477b, 0x14e: 0x3148, 0x14f: 0x3459, 0x150: 0x3152, 0x151: 0x3463,
+	0x154: 0x3170, 0x155: 0x3481, 0x156: 0x3189, 0x157: 0x349a,
+	0x158: 0x317a, 0x159: 0x348b, 0x15a: 0x470d, 0x15b: 0x479e, 0x15c: 0x3193, 0x15d: 0x34a4,
+	0x15e: 0x31a2, 0x15f: 0x34b3, 0x160: 0x4712, 0x161: 0x47a3, 0x162: 0x31bb, 0x163: 0x34d1,
+	0x164: 0x31ac, 0x165: 0x34c2, 0x168: 0x471c, 0x169: 0x47ad,
+	0x16a: 0x4721, 0x16b: 0x47b2, 0x16c: 0x31d9, 0x16d: 0x34ef, 0x16e: 0x31e3, 0x16f: 0x34f9,
+	0x170: 0x31e8, 0x171: 0x34fe, 0x172: 0x3206, 0x173: 0x351c, 0x174: 0x3229, 0x175: 0x353f,
+	0x176: 0x3251, 0x177: 0x356c, 0x178: 0x3265, 0x179: 0x3274, 0x17a: 0x3594, 0x17b: 0x327e,
+	0x17c: 0x359e, 0x17d: 0x3283, 0x17e: 0x35a3, 0x17f: 0xa000,
+	// Block 0x6, offset 0x180
+	0x184: 0x8100, 0x185: 0x8100,
+	0x186: 0x8100,
+	0x18d: 0x2f9f, 0x18e: 0x32ab, 0x18f: 0x30ad, 0x190: 0x33b9, 0x191: 0x3157,
+	0x192: 0x3468, 0x193: 0x31ed, 0x194: 0x3503, 0x195: 0x39e6, 0x196: 0x3b75, 0x197: 0x39df,
+	0x198: 0x3b6e, 0x199: 0x39ed, 0x19a: 0x3b7c, 0x19b: 0x39d8, 0x19c: 0x3b67,
+	0x19e: 0x38c7, 0x19f: 0x3a56, 0x1a0: 0x38c0, 0x1a1: 0x3a4f, 0x1a2: 0x35ca, 0x1a3: 0x35dc,
+	0x1a6: 0x3058, 0x1a7: 0x3364, 0x1a8: 0x30d5, 0x1a9: 0x33e6,
+	0x1aa: 0x4703, 0x1ab: 0x4794, 0x1ac: 0x39a7, 0x1ad: 0x3b36, 0x1ae: 0x35ee, 0x1af: 0x35f4,
+	0x1b0: 0x33dc, 0x1b4: 0x303f, 0x1b5: 0x334b,
+	0x1b8: 0x3111, 0x1b9: 0x3422, 0x1ba: 0x38ce, 0x1bb: 0x3a5d,
+	0x1bc: 0x35c4, 0x1bd: 0x35d6, 0x1be: 0x35d0, 0x1bf: 0x35e2,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x2fa4, 0x1c1: 0x32b0, 0x1c2: 0x2fa9, 0x1c3: 0x32b5, 0x1c4: 0x3021, 0x1c5: 0x332d,
+	0x1c6: 0x3026, 0x1c7: 0x3332, 0x1c8: 0x30b2, 0x1c9: 0x33be, 0x1ca: 0x30b7, 0x1cb: 0x33c3,
+	0x1cc: 0x315c, 0x1cd: 0x346d, 0x1ce: 0x3161, 0x1cf: 0x3472, 0x1d0: 0x317f, 0x1d1: 0x3490,
+	0x1d2: 0x3184, 0x1d3: 0x3495, 0x1d4: 0x31f2, 0x1d5: 0x3508, 0x1d6: 0x31f7, 0x1d7: 0x350d,
+	0x1d8: 0x319d, 0x1d9: 0x34ae, 0x1da: 0x31b6, 0x1db: 0x34cc,
+	0x1de: 0x3071, 0x1df: 0x337d,
+	0x1e6: 0x46a9, 0x1e7: 0x473a, 0x1e8: 0x46d1, 0x1e9: 0x4762,
+	0x1ea: 0x3976, 0x1eb: 0x3b05, 0x1ec: 0x3953, 0x1ed: 0x3ae2, 0x1ee: 0x46ef, 0x1ef: 0x4780,
+	0x1f0: 0x396f, 0x1f1: 0x3afe, 0x1f2: 0x325b, 0x1f3: 0x3576,
+	// Block 0x8, offset 0x200
+	0x200: 0x9933, 0x201: 0x9933, 0x202: 0x9933, 0x203: 0x9933, 0x204: 0x9933, 0x205: 0x8133,
+	0x206: 0x9933, 0x207: 0x9933, 0x208: 0x9933, 0x209: 0x9933, 0x20a: 0x9933, 0x20b: 0x9933,
+	0x20c: 0x9933, 0x20d: 0x8133, 0x20e: 0x8133, 0x20f: 0x9933, 0x210: 0x8133, 0x211: 0x9933,
+	0x212: 0x8133, 0x213: 0x9933, 0x214: 0x9933, 0x215: 0x8134, 0x216: 0x812e, 0x217: 0x812e,
+	0x218: 0x812e, 0x219: 0x812e, 0x21a: 0x8134, 0x21b: 0x992c, 0x21c: 0x812e, 0x21d: 0x812e,
+	0x21e: 0x812e, 0x21f: 0x812e, 0x220: 0x812e, 0x221: 0x812a, 0x222: 0x812a, 0x223: 0x992e,
+	0x224: 0x992e, 0x225: 0x992e, 0x226: 0x992e, 0x227: 0x992a, 0x228: 0x992a, 0x229: 0x812e,
+	0x22a: 0x812e, 0x22b: 0x812e, 0x22c: 0x812e, 0x22d: 0x992e, 0x22e: 0x992e, 0x22f: 0x812e,
+	0x230: 0x992e, 0x231: 0x992e, 0x232: 0x812e, 0x233: 0x812e, 0x234: 0x8101, 0x235: 0x8101,
+	0x236: 0x8101, 0x237: 0x8101, 0x238: 0x9901, 0x239: 0x812e, 0x23a: 0x812e, 0x23b: 0x812e,
+	0x23c: 0x812e, 0x23d: 0x8133, 0x23e: 0x8133, 0x23f: 0x8133,
+	// Block 0x9, offset 0x240
+	0x240: 0x49c5, 0x241: 0x49ca, 0x242: 0x9933, 0x243: 0x49cf, 0x244: 0x4a88, 0x245: 0x9937,
+	0x246: 0x8133, 0x247: 0x812e, 0x248: 0x812e, 0x249: 0x812e, 0x24a: 0x8133, 0x24b: 0x8133,
+	0x24c: 0x8133, 0x24d: 0x812e, 0x24e: 0x812e, 0x250: 0x8133, 0x251: 0x8133,
+	0x252: 0x8133, 0x253: 0x812e, 0x254: 0x812e, 0x255: 0x812e, 0x256: 0x812e, 0x257: 0x8133,
+	0x258: 0x8134, 0x259: 0x812e, 0x25a: 0x812e, 0x25b: 0x8133, 0x25c: 0x8135, 0x25d: 0x8136,
+	0x25e: 0x8136, 0x25f: 0x8135, 0x260: 0x8136, 0x261: 0x8136, 0x262: 0x8135, 0x263: 0x8133,
+	0x264: 0x8133, 0x265: 0x8133, 0x266: 0x8133, 0x267: 0x8133, 0x268: 0x8133, 0x269: 0x8133,
+	0x26a: 0x8133, 0x26b: 0x8133, 0x26c: 0x8133, 0x26d: 0x8133, 0x26e: 0x8133, 0x26f: 0x8133,
+	0x274: 0x0173,
+	0x27a: 0x8100,
+	0x27e: 0x0037,
+	// Block 0xa, offset 0x280
+	0x284: 0x8100, 0x285: 0x35b8,
+	0x286: 0x3600, 0x287: 0x00ce, 0x288: 0x361e, 0x289: 0x362a, 0x28a: 0x363c,
+	0x28c: 0x365a, 0x28e: 0x366c, 0x28f: 0x368a, 0x290: 0x3e1f, 0x291: 0xa000,
+	0x295: 0xa000, 0x297: 0xa000,
+	0x299: 0xa000,
+	0x29f: 0xa000, 0x2a1: 0xa000,
+	0x2a5: 0xa000, 0x2a9: 0xa000,
+	0x2aa: 0x364e, 0x2ab: 0x367e, 0x2ac: 0x4815, 0x2ad: 0x36ae, 0x2ae: 0x483f, 0x2af: 0x36c0,
+	0x2b0: 0x3e87, 0x2b1: 0xa000, 0x2b5: 0xa000,
+	0x2b7: 0xa000, 0x2b9: 0xa000,
+	0x2bf: 0xa000,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0x3738, 0x2c1: 0x3744, 0x2c3: 0x3732,
+	0x2c6: 0xa000, 0x2c7: 0x3720,
+	0x2cc: 0x3774, 0x2cd: 0x375c, 0x2ce: 0x3786, 0x2d0: 0xa000,
+	0x2d3: 0xa000, 0x2d5: 0xa000, 0x2d6: 0xa000, 0x2d7: 0xa000,
+	0x2d8: 0xa000, 0x2d9: 0x3768, 0x2da: 0xa000,
+	0x2de: 0xa000, 0x2e3: 0xa000,
+	0x2e7: 0xa000,
+	0x2eb: 0xa000, 0x2ed: 0xa000,
+	0x2f0: 0xa000, 0x2f3: 0xa000, 0x2f5: 0xa000,
+	0x2f6: 0xa000, 0x2f7: 0xa000, 0x2f8: 0xa000, 0x2f9: 0x37ec, 0x2fa: 0xa000,
+	0x2fe: 0xa000,
+	// Block 0xc, offset 0x300
+	0x301: 0x374a, 0x302: 0x37ce,
+	0x310: 0x3726, 0x311: 0x37aa,
+	0x312: 0x372c, 0x313: 0x37b0, 0x316: 0x373e, 0x317: 0x37c2,
+	0x318: 0xa000, 0x319: 0xa000, 0x31a: 0x3840, 0x31b: 0x3846, 0x31c: 0x3750, 0x31d: 0x37d4,
+	0x31e: 0x3756, 0x31f: 0x37da, 0x322: 0x3762, 0x323: 0x37e6,
+	0x324: 0x376e, 0x325: 0x37f2, 0x326: 0x377a, 0x327: 0x37fe, 0x328: 0xa000, 0x329: 0xa000,
+	0x32a: 0x384c, 0x32b: 0x3852, 0x32c: 0x37a4, 0x32d: 0x3828, 0x32e: 0x3780, 0x32f: 0x3804,
+	0x330: 0x378c, 0x331: 0x3810, 0x332: 0x3792, 0x333: 0x3816, 0x334: 0x3798, 0x335: 0x381c,
+	0x338: 0x379e, 0x339: 0x3822,
+	// Block 0xd, offset 0x340
+	0x351: 0x812e,
+	0x352: 0x8133, 0x353: 0x8133, 0x354: 0x8133, 0x355: 0x8133, 0x356: 0x812e, 0x357: 0x8133,
+	0x358: 0x8133, 0x359: 0x8133, 0x35a: 0x812f, 0x35b: 0x812e, 0x35c: 0x8133, 0x35d: 0x8133,
+	0x35e: 0x8133, 0x35f: 0x8133, 0x360: 0x8133, 0x361: 0x8133, 0x362: 0x812e, 0x363: 0x812e,
+	0x364: 0x812e, 0x365: 0x812e, 0x366: 0x812e, 0x367: 0x812e, 0x368: 0x8133, 0x369: 0x8133,
+	0x36a: 0x812e, 0x36b: 0x8133, 0x36c: 0x8133, 0x36d: 0x812f, 0x36e: 0x8132, 0x36f: 0x8133,
+	0x370: 0x8106, 0x371: 0x8107, 0x372: 0x8108, 0x373: 0x8109, 0x374: 0x810a, 0x375: 0x810b,
+	0x376: 0x810c, 0x377: 0x810d, 0x378: 0x810e, 0x379: 0x810f, 0x37a: 0x810f, 0x37b: 0x8110,
+	0x37c: 0x8111, 0x37d: 0x8112, 0x37f: 0x8113,
+	// Block 0xe, offset 0x380
+	0x388: 0xa000, 0x38a: 0xa000, 0x38b: 0x8117,
+	0x38c: 0x8118, 0x38d: 0x8119, 0x38e: 0x811a, 0x38f: 0x811b, 0x390: 0x811c, 0x391: 0x811d,
+	0x392: 0x811e, 0x393: 0x9933, 0x394: 0x9933, 0x395: 0x992e, 0x396: 0x812e, 0x397: 0x8133,
+	0x398: 0x8133, 0x399: 0x8133, 0x39a: 0x8133, 0x39b: 0x8133, 0x39c: 0x812e, 0x39d: 0x8133,
+	0x39e: 0x8133, 0x39f: 0x812e,
+	0x3b0: 0x811f,
+	// Block 0xf, offset 0x3c0
+	0x3d3: 0x812e, 0x3d4: 0x8133, 0x3d5: 0x8133, 0x3d6: 0x8133, 0x3d7: 0x8133,
+	0x3d8: 0x8133, 0x3d9: 0x8133, 0x3da: 0x8133, 0x3db: 0x8133, 0x3dc: 0x8133, 0x3dd: 0x8133,
+	0x3de: 0x8133, 0x3df: 0x8133, 0x3e0: 0x8133, 0x3e1: 0x8133, 0x3e3: 0x812e,
+	0x3e4: 0x8133, 0x3e5: 0x8133, 0x3e6: 0x812e, 0x3e7: 0x8133, 0x3e8: 0x8133, 0x3e9: 0x812e,
+	0x3ea: 0x8133, 0x3eb: 0x8133, 0x3ec: 0x8133, 0x3ed: 0x812e, 0x3ee: 0x812e, 0x3ef: 0x812e,
+	0x3f0: 0x8117, 0x3f1: 0x8118, 0x3f2: 0x8119, 0x3f3: 0x8133, 0x3f4: 0x8133, 0x3f5: 0x8133,
+	0x3f6: 0x812e, 0x3f7: 0x8133, 0x3f8: 0x8133, 0x3f9: 0x812e, 0x3fa: 0x812e, 0x3fb: 0x8133,
+	0x3fc: 0x8133, 0x3fd: 0x8133, 0x3fe: 0x8133, 0x3ff: 0x8133,
+	// Block 0x10, offset 0x400
+	0x405: 0xa000,
+	0x406: 0x2d33, 0x407: 0xa000, 0x408: 0x2d3b, 0x409: 0xa000, 0x40a: 0x2d43, 0x40b: 0xa000,
+	0x40c: 0x2d4b, 0x40d: 0xa000, 0x40e: 0x2d53, 0x411: 0xa000,
+	0x412: 0x2d5b,
+	0x434: 0x8103, 0x435: 0x9900,
+	0x43a: 0xa000, 0x43b: 0x2d63,
+	0x43c: 0xa000, 0x43d: 0x2d6b, 0x43e: 0xa000, 0x43f: 0xa000,
+	// Block 0x11, offset 0x440
+	0x440: 0x8133, 0x441: 0x8133, 0x442: 0x812e, 0x443: 0x8133, 0x444: 0x8133, 0x445: 0x8133,
+	0x446: 0x8133, 0x447: 0x8133, 0x448: 0x8133, 0x449: 0x8133, 0x44a: 0x812e, 0x44b: 0x8133,
+	0x44c: 0x8133, 0x44d: 0x8136, 0x44e: 0x812b, 0x44f: 0x812e, 0x450: 0x812a, 0x451: 0x8133,
+	0x452: 0x8133, 0x453: 0x8133, 0x454: 0x8133, 0x455: 0x8133, 0x456: 0x8133, 0x457: 0x8133,
+	0x458: 0x8133, 0x459: 0x8133, 0x45a: 0x8133, 0x45b: 0x8133, 0x45c: 0x8133, 0x45d: 0x8133,
+	0x45e: 0x8133, 0x45f: 0x8133, 0x460: 0x8133, 0x461: 0x8133, 0x462: 0x8133, 0x463: 0x8133,
+	0x464: 0x8133, 0x465: 0x8133, 0x466: 0x8133, 0x467: 0x8133, 0x468: 0x8133, 0x469: 0x8133,
+	0x46a: 0x8133, 0x46b: 0x8133, 0x46c: 0x8133, 0x46d: 0x8133, 0x46e: 0x8133, 0x46f: 0x8133,
+	0x470: 0x8133, 0x471: 0x8133, 0x472: 0x8133, 0x473: 0x8133, 0x474: 0x8133, 0x475: 0x8133,
+	0x476: 0x8134, 0x477: 0x8132, 0x478: 0x8132, 0x479: 0x812e, 0x47b: 0x8133,
+	0x47c: 0x8135, 0x47d: 0x812e, 0x47e: 0x8133, 0x47f: 0x812e,
+	// Block 0x12, offset 0x480
+	0x480: 0x2fae, 0x481: 0x32ba, 0x482: 0x2fb8, 0x483: 0x32c4, 0x484: 0x2fbd, 0x485: 0x32c9,
+	0x486: 0x2fc2, 0x487: 0x32ce, 0x488: 0x38e3, 0x489: 0x3a72, 0x48a: 0x2fdb, 0x48b: 0x32e7,
+	0x48c: 0x2fe5, 0x48d: 0x32f1, 0x48e: 0x2ff4, 0x48f: 0x3300, 0x490: 0x2fea, 0x491: 0x32f6,
+	0x492: 0x2fef, 0x493: 0x32fb, 0x494: 0x3906, 0x495: 0x3a95, 0x496: 0x390d, 0x497: 0x3a9c,
+	0x498: 0x3030, 0x499: 0x333c, 0x49a: 0x3035, 0x49b: 0x3341, 0x49c: 0x391b, 0x49d: 0x3aaa,
+	0x49e: 0x303a, 0x49f: 0x3346, 0x4a0: 0x3049, 0x4a1: 0x3355, 0x4a2: 0x3067, 0x4a3: 0x3373,
+	0x4a4: 0x3076, 0x4a5: 0x3382, 0x4a6: 0x306c, 0x4a7: 0x3378, 0x4a8: 0x307b, 0x4a9: 0x3387,
+	0x4aa: 0x3080, 0x4ab: 0x338c, 0x4ac: 0x30c6, 0x4ad: 0x33d2, 0x4ae: 0x3922, 0x4af: 0x3ab1,
+	0x4b0: 0x30d0, 0x4b1: 0x33e1, 0x4b2: 0x30da, 0x4b3: 0x33eb, 0x4b4: 0x30e4, 0x4b5: 0x33f5,
+	0x4b6: 0x46db, 0x4b7: 0x476c, 0x4b8: 0x3929, 0x4b9: 0x3ab8, 0x4ba: 0x30fd, 0x4bb: 0x340e,
+	0x4bc: 0x30f8, 0x4bd: 0x3409, 0x4be: 0x3102, 0x4bf: 0x3413,
+	// Block 0x13, offset 0x4c0
+	0x4c0: 0x3107, 0x4c1: 0x3418, 0x4c2: 0x310c, 0x4c3: 0x341d, 0x4c4: 0x3120, 0x4c5: 0x3431,
+	0x4c6: 0x312a, 0x4c7: 0x343b, 0x4c8: 0x3139, 0x4c9: 0x344a, 0x4ca: 0x3134, 0x4cb: 0x3445,
+	0x4cc: 0x394c, 0x4cd: 0x3adb, 0x4ce: 0x395a, 0x4cf: 0x3ae9, 0x4d0: 0x3961, 0x4d1: 0x3af0,
+	0x4d2: 0x3968, 0x4d3: 0x3af7, 0x4d4: 0x3166, 0x4d5: 0x3477, 0x4d6: 0x316b, 0x4d7: 0x347c,
+	0x4d8: 0x3175, 0x4d9: 0x3486, 0x4da: 0x4708, 0x4db: 0x4799, 0x4dc: 0x39ae, 0x4dd: 0x3b3d,
+	0x4de: 0x318e, 0x4df: 0x349f, 0x4e0: 0x3198, 0x4e1: 0x34a9, 0x4e2: 0x4717, 0x4e3: 0x47a8,
+	0x4e4: 0x39b5, 0x4e5: 0x3b44, 0x4e6: 0x39bc, 0x4e7: 0x3b4b, 0x4e8: 0x39c3, 0x4e9: 0x3b52,
+	0x4ea: 0x31a7, 0x4eb: 0x34b8, 0x4ec: 0x31b1, 0x4ed: 0x34c7, 0x4ee: 0x31c5, 0x4ef: 0x34db,
+	0x4f0: 0x31c0, 0x4f1: 0x34d6, 0x4f2: 0x3201, 0x4f3: 0x3517, 0x4f4: 0x3210, 0x4f5: 0x3526,
+	0x4f6: 0x320b, 0x4f7: 0x3521, 0x4f8: 0x39ca, 0x4f9: 0x3b59, 0x4fa: 0x39d1, 0x4fb: 0x3b60,
+	0x4fc: 0x3215, 0x4fd: 0x352b, 0x4fe: 0x321a, 0x4ff: 0x3530,
+	// Block 0x14, offset 0x500
+	0x500: 0x321f, 0x501: 0x3535, 0x502: 0x3224, 0x503: 0x353a, 0x504: 0x3233, 0x505: 0x3549,
+	0x506: 0x322e, 0x507: 0x3544, 0x508: 0x3238, 0x509: 0x3553, 0x50a: 0x323d, 0x50b: 0x3558,
+	0x50c: 0x3242, 0x50d: 0x355d, 0x50e: 0x3260, 0x50f: 0x357b, 0x510: 0x3279, 0x511: 0x3599,
+	0x512: 0x3288, 0x513: 0x35a8, 0x514: 0x328d, 0x515: 0x35ad, 0x516: 0x3391, 0x517: 0x34bd,
+	0x518: 0x354e, 0x519: 0x358a, 0x51b: 0x35e8,
+	0x520: 0x46b8, 0x521: 0x4749, 0x522: 0x2f9a, 0x523: 0x32a6,
+	0x524: 0x388f, 0x525: 0x3a1e, 0x526: 0x3888, 0x527: 0x3a17, 0x528: 0x389d, 0x529: 0x3a2c,
+	0x52a: 0x3896, 0x52b: 0x3a25, 0x52c: 0x38d5, 0x52d: 0x3a64, 0x52e: 0x38ab, 0x52f: 0x3a3a,
+	0x530: 0x38a4, 0x531: 0x3a33, 0x532: 0x38b9, 0x533: 0x3a48, 0x534: 0x38b2, 0x535: 0x3a41,
+	0x536: 0x38dc, 0x537: 0x3a6b, 0x538: 0x46cc, 0x539: 0x475d, 0x53a: 0x3017, 0x53b: 0x3323,
+	0x53c: 0x3003, 0x53d: 0x330f, 0x53e: 0x38f1, 0x53f: 0x3a80,
+	// Block 0x15, offset 0x540
+	0x540: 0x38ea, 0x541: 0x3a79, 0x542: 0x38ff, 0x543: 0x3a8e, 0x544: 0x38f8, 0x545: 0x3a87,
+	0x546: 0x3914, 0x547: 0x3aa3, 0x548: 0x30a8, 0x549: 0x33b4, 0x54a: 0x30bc, 0x54b: 0x33c8,
+	0x54c: 0x46fe, 0x54d: 0x478f, 0x54e: 0x314d, 0x54f: 0x345e, 0x550: 0x3937, 0x551: 0x3ac6,
+	0x552: 0x3930, 0x553: 0x3abf, 0x554: 0x3945, 0x555: 0x3ad4, 0x556: 0x393e, 0x557: 0x3acd,
+	0x558: 0x39a0, 0x559: 0x3b2f, 0x55a: 0x3984, 0x55b: 0x3b13, 0x55c: 0x397d, 0x55d: 0x3b0c,
+	0x55e: 0x3992, 0x55f: 0x3b21, 0x560: 0x398b, 0x561: 0x3b1a, 0x562: 0x3999, 0x563: 0x3b28,
+	0x564: 0x31fc, 0x565: 0x3512, 0x566: 0x31de, 0x567: 0x34f4, 0x568: 0x39fb, 0x569: 0x3b8a,
+	0x56a: 0x39f4, 0x56b: 0x3b83, 0x56c: 0x3a09, 0x56d: 0x3b98, 0x56e: 0x3a02, 0x56f: 0x3b91,
+	0x570: 0x3a10, 0x571: 0x3b9f, 0x572: 0x3247, 0x573: 0x3562, 0x574: 0x326f, 0x575: 0x358f,
+	0x576: 0x326a, 0x577: 0x3585, 0x578: 0x3256, 0x579: 0x3571,
+	// Block 0x16, offset 0x580
+	0x580: 0x481b, 0x581: 0x4821, 0x582: 0x4935, 0x583: 0x494d, 0x584: 0x493d, 0x585: 0x4955,
+	0x586: 0x4945, 0x587: 0x495d, 0x588: 0x47c1, 0x589: 0x47c7, 0x58a: 0x48a5, 0x58b: 0x48bd,
+	0x58c: 0x48ad, 0x58d: 0x48c5, 0x58e: 0x48b5, 0x58f: 0x48cd, 0x590: 0x482d, 0x591: 0x4833,
+	0x592: 0x3dcf, 0x593: 0x3ddf, 0x594: 0x3dd7, 0x595: 0x3de7,
+	0x598: 0x47cd, 0x599: 0x47d3, 0x59a: 0x3cff, 0x59b: 0x3d0f, 0x59c: 0x3d07, 0x59d: 0x3d17,
+	0x5a0: 0x4845, 0x5a1: 0x484b, 0x5a2: 0x4965, 0x5a3: 0x497d,
+	0x5a4: 0x496d, 0x5a5: 0x4985, 0x5a6: 0x4975, 0x5a7: 0x498d, 0x5a8: 0x47d9, 0x5a9: 0x47df,
+	0x5aa: 0x48d5, 0x5ab: 0x48ed, 0x5ac: 0x48dd, 0x5ad: 0x48f5, 0x5ae: 0x48e5, 0x5af: 0x48fd,
+	0x5b0: 0x485d, 0x5b1: 0x4863, 0x5b2: 0x3e2f, 0x5b3: 0x3e47, 0x5b4: 0x3e37, 0x5b5: 0x3e4f,
+	0x5b6: 0x3e3f, 0x5b7: 0x3e57, 0x5b8: 0x47e5, 0x5b9: 0x47eb, 0x5ba: 0x3d2f, 0x5bb: 0x3d47,
+	0x5bc: 0x3d37, 0x5bd: 0x3d4f, 0x5be: 0x3d3f, 0x5bf: 0x3d57,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x4869, 0x5c1: 0x486f, 0x5c2: 0x3e5f, 0x5c3: 0x3e6f, 0x5c4: 0x3e67, 0x5c5: 0x3e77,
+	0x5c8: 0x47f1, 0x5c9: 0x47f7, 0x5ca: 0x3d5f, 0x5cb: 0x3d6f,
+	0x5cc: 0x3d67, 0x5cd: 0x3d77, 0x5d0: 0x487b, 0x5d1: 0x4881,
+	0x5d2: 0x3e97, 0x5d3: 0x3eaf, 0x5d4: 0x3e9f, 0x5d5: 0x3eb7, 0x5d6: 0x3ea7, 0x5d7: 0x3ebf,
+	0x5d9: 0x47fd, 0x5db: 0x3d7f, 0x5dd: 0x3d87,
+	0x5df: 0x3d8f, 0x5e0: 0x4893, 0x5e1: 0x4899, 0x5e2: 0x4995, 0x5e3: 0x49ad,
+	0x5e4: 0x499d, 0x5e5: 0x49b5, 0x5e6: 0x49a5, 0x5e7: 0x49bd, 0x5e8: 0x4803, 0x5e9: 0x4809,
+	0x5ea: 0x4905, 0x5eb: 0x491d, 0x5ec: 0x490d, 0x5ed: 0x4925, 0x5ee: 0x4915, 0x5ef: 0x492d,
+	0x5f0: 0x480f, 0x5f1: 0x4335, 0x5f2: 0x36a8, 0x5f3: 0x433b, 0x5f4: 0x4839, 0x5f5: 0x4341,
+	0x5f6: 0x36ba, 0x5f7: 0x4347, 0x5f8: 0x36d8, 0x5f9: 0x434d, 0x5fa: 0x36f0, 0x5fb: 0x4353,
+	0x5fc: 0x4887, 0x5fd: 0x4359,
+	// Block 0x18, offset 0x600
+	0x600: 0x3db7, 0x601: 0x3dbf, 0x602: 0x419b, 0x603: 0x41b9, 0x604: 0x41a5, 0x605: 0x41c3,
+	0x606: 0x41af, 0x607: 0x41cd, 0x608: 0x3cef, 0x609: 0x3cf7, 0x60a: 0x40e7, 0x60b: 0x4105,
+	0x60c: 0x40f1, 0x60d: 0x410f, 0x60e: 0x40fb, 0x60f: 0x4119, 0x610: 0x3dff, 0x611: 0x3e07,
+	0x612: 0x41d7, 0x613: 0x41f5, 0x614: 0x41e1, 0x615: 0x41ff, 0x616: 0x41eb, 0x617: 0x4209,
+	0x618: 0x3d1f, 0x619: 0x3d27, 0x61a: 0x4123, 0x61b: 0x4141, 0x61c: 0x412d, 0x61d: 0x414b,
+	0x61e: 0x4137, 0x61f: 0x4155, 0x620: 0x3ed7, 0x621: 0x3edf, 0x622: 0x4213, 0x623: 0x4231,
+	0x624: 0x421d, 0x625: 0x423b, 0x626: 0x4227, 0x627: 0x4245, 0x628: 0x3d97, 0x629: 0x3d9f,
+	0x62a: 0x415f, 0x62b: 0x417d, 0x62c: 0x4169, 0x62d: 0x4187, 0x62e: 0x4173, 0x62f: 0x4191,
+	0x630: 0x369c, 0x631: 0x3696, 0x632: 0x3da7, 0x633: 0x36a2, 0x634: 0x3daf,
+	0x636: 0x4827, 0x637: 0x3dc7, 0x638: 0x360c, 0x639: 0x3606, 0x63a: 0x35fa, 0x63b: 0x4305,
+	0x63c: 0x3612, 0x63d: 0x8100, 0x63e: 0x01d6, 0x63f: 0xa100,
+	// Block 0x19, offset 0x640
+	0x640: 0x8100, 0x641: 0x35be, 0x642: 0x3def, 0x643: 0x36b4, 0x644: 0x3df7,
+	0x646: 0x4851, 0x647: 0x3e0f, 0x648: 0x3618, 0x649: 0x430b, 0x64a: 0x3624, 0x64b: 0x4311,
+	0x64c: 0x3630, 0x64d: 0x3ba6, 0x64e: 0x3bad, 0x64f: 0x3bb4, 0x650: 0x36cc, 0x651: 0x36c6,
+	0x652: 0x3e17, 0x653: 0x44fb, 0x656: 0x36d2, 0x657: 0x3e27,
+	0x658: 0x3648, 0x659: 0x3642, 0x65a: 0x3636, 0x65b: 0x4317, 0x65d: 0x3bbb,
+	0x65e: 0x3bc2, 0x65f: 0x3bc9, 0x660: 0x3702, 0x661: 0x36fc, 0x662: 0x3e7f, 0x663: 0x4503,
+	0x664: 0x36e4, 0x665: 0x36ea, 0x666: 0x3708, 0x667: 0x3e8f, 0x668: 0x3678, 0x669: 0x3672,
+	0x66a: 0x3666, 0x66b: 0x4323, 0x66c: 0x3660, 0x66d: 0x35b2, 0x66e: 0x42ff, 0x66f: 0x0081,
+	0x672: 0x3ec7, 0x673: 0x370e, 0x674: 0x3ecf,
+	0x676: 0x489f, 0x677: 0x3ee7, 0x678: 0x3654, 0x679: 0x431d, 0x67a: 0x3684, 0x67b: 0x432f,
+	0x67c: 0x3690, 0x67d: 0x426d, 0x67e: 0xa100,
+	// Block 0x1a, offset 0x680
+	0x681: 0x3c1d, 0x683: 0xa000, 0x684: 0x3c24, 0x685: 0xa000,
+	0x687: 0x3c2b, 0x688: 0xa000, 0x689: 0x3c32,
+	0x68d: 0xa000,
+	0x6a0: 0x2f7c, 0x6a1: 0xa000, 0x6a2: 0x3c40,
+	0x6a4: 0xa000, 0x6a5: 0xa000,
+	0x6ad: 0x3c39, 0x6ae: 0x2f77, 0x6af: 0x2f81,
+	0x6b0: 0x3c47, 0x6b1: 0x3c4e, 0x6b2: 0xa000, 0x6b3: 0xa000, 0x6b4: 0x3c55, 0x6b5: 0x3c5c,
+	0x6b6: 0xa000, 0x6b7: 0xa000, 0x6b8: 0x3c63, 0x6b9: 0x3c6a, 0x6ba: 0xa000, 0x6bb: 0xa000,
+	0x6bc: 0xa000, 0x6bd: 0xa000,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0x3c71, 0x6c1: 0x3c78, 0x6c2: 0xa000, 0x6c3: 0xa000, 0x6c4: 0x3c8d, 0x6c5: 0x3c94,
+	0x6c6: 0xa000, 0x6c7: 0xa000, 0x6c8: 0x3c9b, 0x6c9: 0x3ca2,
+	0x6d1: 0xa000,
+	0x6d2: 0xa000,
+	0x6e2: 0xa000,
+	0x6e8: 0xa000, 0x6e9: 0xa000,
+	0x6eb: 0xa000, 0x6ec: 0x3cb7, 0x6ed: 0x3cbe, 0x6ee: 0x3cc5, 0x6ef: 0x3ccc,
+	0x6f2: 0xa000, 0x6f3: 0xa000, 0x6f4: 0xa000, 0x6f5: 0xa000,
+	// Block 0x1c, offset 0x700
+	0x706: 0xa000, 0x70b: 0xa000,
+	0x70c: 0x3f1f, 0x70d: 0xa000, 0x70e: 0x3f27, 0x70f: 0xa000, 0x710: 0x3f2f, 0x711: 0xa000,
+	0x712: 0x3f37, 0x713: 0xa000, 0x714: 0x3f3f, 0x715: 0xa000, 0x716: 0x3f47, 0x717: 0xa000,
+	0x718: 0x3f4f, 0x719: 0xa000, 0x71a: 0x3f57, 0x71b: 0xa000, 0x71c: 0x3f5f, 0x71d: 0xa000,
+	0x71e: 0x3f67, 0x71f: 0xa000, 0x720: 0x3f6f, 0x721: 0xa000, 0x722: 0x3f77,
+	0x724: 0xa000, 0x725: 0x3f7f, 0x726: 0xa000, 0x727: 0x3f87, 0x728: 0xa000, 0x729: 0x3f8f,
+	0x72f: 0xa000,
+	0x730: 0x3f97, 0x731: 0x3f9f, 0x732: 0xa000, 0x733: 0x3fa7, 0x734: 0x3faf, 0x735: 0xa000,
+	0x736: 0x3fb7, 0x737: 0x3fbf, 0x738: 0xa000, 0x739: 0x3fc7, 0x73a: 0x3fcf, 0x73b: 0xa000,
+	0x73c: 0x3fd7, 0x73d: 0x3fdf,
+	// Block 0x1d, offset 0x740
+	0x754: 0x3f17,
+	0x759: 0x9904, 0x75a: 0x9904, 0x75b: 0x8100, 0x75c: 0x8100, 0x75d: 0xa000,
+	0x75e: 0x3fe7,
+	0x766: 0xa000,
+	0x76b: 0xa000, 0x76c: 0x3ff7, 0x76d: 0xa000, 0x76e: 0x3fff, 0x76f: 0xa000,
+	0x770: 0x4007, 0x771: 0xa000, 0x772: 0x400f, 0x773: 0xa000, 0x774: 0x4017, 0x775: 0xa000,
+	0x776: 0x401f, 0x777: 0xa000, 0x778: 0x4027, 0x779: 0xa000, 0x77a: 0x402f, 0x77b: 0xa000,
+	0x77c: 0x4037, 0x77d: 0xa000, 0x77e: 0x403f, 0x77f: 0xa000,
+	// Block 0x1e, offset 0x780
+	0x780: 0x4047, 0x781: 0xa000, 0x782: 0x404f, 0x784: 0xa000, 0x785: 0x4057,
+	0x786: 0xa000, 0x787: 0x405f, 0x788: 0xa000, 0x789: 0x4067,
+	0x78f: 0xa000, 0x790: 0x406f, 0x791: 0x4077,
+	0x792: 0xa000, 0x793: 0x407f, 0x794: 0x4087, 0x795: 0xa000, 0x796: 0x408f, 0x797: 0x4097,
+	0x798: 0xa000, 0x799: 0x409f, 0x79a: 0x40a7, 0x79b: 0xa000, 0x79c: 0x40af, 0x79d: 0x40b7,
+	0x7af: 0xa000,
+	0x7b0: 0xa000, 0x7b1: 0xa000, 0x7b2: 0xa000, 0x7b4: 0x3fef,
+	0x7b7: 0x40bf, 0x7b8: 0x40c7, 0x7b9: 0x40cf, 0x7ba: 0x40d7,
+	0x7bd: 0xa000, 0x7be: 0x40df,
+	// Block 0x1f, offset 0x7c0
+	0x7c0: 0x137a, 0x7c1: 0x0cfe, 0x7c2: 0x13d6, 0x7c3: 0x13a2, 0x7c4: 0x0e5a, 0x7c5: 0x06ee,
+	0x7c6: 0x08e2, 0x7c7: 0x162e, 0x7c8: 0x162e, 0x7c9: 0x0a0e, 0x7ca: 0x1462, 0x7cb: 0x0946,
+	0x7cc: 0x0a0a, 0x7cd: 0x0bf2, 0x7ce: 0x0fd2, 0x7cf: 0x1162, 0x7d0: 0x129a, 0x7d1: 0x12d6,
+	0x7d2: 0x130a, 0x7d3: 0x141e, 0x7d4: 0x0d76, 0x7d5: 0x0e02, 0x7d6: 0x0eae, 0x7d7: 0x0f46,
+	0x7d8: 0x1262, 0x7d9: 0x144a, 0x7da: 0x1576, 0x7db: 0x0712, 0x7dc: 0x08b6, 0x7dd: 0x0d8a,
+	0x7de: 0x0ed2, 0x7df: 0x1296, 0x7e0: 0x15c6, 0x7e1: 0x0ab6, 0x7e2: 0x0e7a, 0x7e3: 0x1286,
+	0x7e4: 0x131a, 0x7e5: 0x0c26, 0x7e6: 0x11be, 0x7e7: 0x12e2, 0x7e8: 0x0b22, 0x7e9: 0x0d12,
+	0x7ea: 0x0e1a, 0x7eb: 0x0f1e, 0x7ec: 0x142a, 0x7ed: 0x0752, 0x7ee: 0x07ea, 0x7ef: 0x0856,
+	0x7f0: 0x0c8e, 0x7f1: 0x0d82, 0x7f2: 0x0ece, 0x7f3: 0x0ff2, 0x7f4: 0x117a, 0x7f5: 0x128e,
+	0x7f6: 0x12a6, 0x7f7: 0x13ca, 0x7f8: 0x14f2, 0x7f9: 0x15a6, 0x7fa: 0x15c2, 0x7fb: 0x102e,
+	0x7fc: 0x106e, 0x7fd: 0x1126, 0x7fe: 0x1246, 0x7ff: 0x147e,
+	// Block 0x20, offset 0x800
+	0x800: 0x15ce, 0x801: 0x134e, 0x802: 0x09ca, 0x803: 0x0b3e, 0x804: 0x10de, 0x805: 0x119e,
+	0x806: 0x0f02, 0x807: 0x1036, 0x808: 0x139a, 0x809: 0x14ea, 0x80a: 0x09c6, 0x80b: 0x0a92,
+	0x80c: 0x0d7a, 0x80d: 0x0e2e, 0x80e: 0x0e62, 0x80f: 0x1116, 0x810: 0x113e, 0x811: 0x14aa,
+	0x812: 0x0852, 0x813: 0x11aa, 0x814: 0x07f6, 0x815: 0x07f2, 0x816: 0x109a, 0x817: 0x112a,
+	0x818: 0x125e, 0x819: 0x14b2, 0x81a: 0x136a, 0x81b: 0x0c2a, 0x81c: 0x0d76, 0x81d: 0x135a,
+	0x81e: 0x06fa, 0x81f: 0x0a66, 0x820: 0x0b96, 0x821: 0x0f32, 0x822: 0x0fb2, 0x823: 0x0876,
+	0x824: 0x103e, 0x825: 0x0762, 0x826: 0x0b7a, 0x827: 0x06da, 0x828: 0x0dee, 0x829: 0x0ca6,
+	0x82a: 0x1112, 0x82b: 0x08ca, 0x82c: 0x09b6, 0x82d: 0x0ffe, 0x82e: 0x1266, 0x82f: 0x133e,
+	0x830: 0x0dba, 0x831: 0x13fa, 0x832: 0x0de6, 0x833: 0x0c3a, 0x834: 0x121e, 0x835: 0x0c5a,
+	0x836: 0x0fae, 0x837: 0x072e, 0x838: 0x07aa, 0x839: 0x07ee, 0x83a: 0x0d56, 0x83b: 0x10fe,
+	0x83c: 0x11f6, 0x83d: 0x134a, 0x83e: 0x145e, 0x83f: 0x085e,
+	// Block 0x21, offset 0x840
+	0x840: 0x0912, 0x841: 0x0a1a, 0x842: 0x0b32, 0x843: 0x0cc2, 0x844: 0x0e7e, 0x845: 0x1042,
+	0x846: 0x149a, 0x847: 0x157e, 0x848: 0x15d2, 0x849: 0x15ea, 0x84a: 0x083a, 0x84b: 0x0cf6,
+	0x84c: 0x0da6, 0x84d: 0x13ee, 0x84e: 0x0afe, 0x84f: 0x0bda, 0x850: 0x0bf6, 0x851: 0x0c86,
+	0x852: 0x0e6e, 0x853: 0x0eba, 0x854: 0x0f6a, 0x855: 0x108e, 0x856: 0x1132, 0x857: 0x1196,
+	0x858: 0x13de, 0x859: 0x126e, 0x85a: 0x1406, 0x85b: 0x1482, 0x85c: 0x0812, 0x85d: 0x083e,
+	0x85e: 0x0926, 0x85f: 0x0eaa, 0x860: 0x12f6, 0x861: 0x133e, 0x862: 0x0b1e, 0x863: 0x0b8e,
+	0x864: 0x0c52, 0x865: 0x0db2, 0x866: 0x10da, 0x867: 0x0f26, 0x868: 0x073e, 0x869: 0x0982,
+	0x86a: 0x0a66, 0x86b: 0x0aca, 0x86c: 0x0b9a, 0x86d: 0x0f42, 0x86e: 0x0f5e, 0x86f: 0x116e,
+	0x870: 0x118e, 0x871: 0x1466, 0x872: 0x14e6, 0x873: 0x14f6, 0x874: 0x1532, 0x875: 0x0756,
+	0x876: 0x1082, 0x877: 0x1452, 0x878: 0x14ce, 0x879: 0x0bb2, 0x87a: 0x071a, 0x87b: 0x077a,
+	0x87c: 0x0a6a, 0x87d: 0x0a8a, 0x87e: 0x0cb2, 0x87f: 0x0d76,
+	// Block 0x22, offset 0x880
+	0x880: 0x0ec6, 0x881: 0x0fce, 0x882: 0x127a, 0x883: 0x141a, 0x884: 0x1626, 0x885: 0x0ce6,
+	0x886: 0x14a6, 0x887: 0x0836, 0x888: 0x0d32, 0x889: 0x0d3e, 0x88a: 0x0e12, 0x88b: 0x0e4a,
+	0x88c: 0x0f4e, 0x88d: 0x0faa, 0x88e: 0x102a, 0x88f: 0x110e, 0x890: 0x153e, 0x891: 0x07b2,
+	0x892: 0x0c06, 0x893: 0x14b6, 0x894: 0x076a, 0x895: 0x0aae, 0x896: 0x0e32, 0x897: 0x13e2,
+	0x898: 0x0b6a, 0x899: 0x0bba, 0x89a: 0x0d46, 0x89b: 0x0f32, 0x89c: 0x14be, 0x89d: 0x081a,
+	0x89e: 0x0902, 0x89f: 0x0a9a, 0x8a0: 0x0cd6, 0x8a1: 0x0d22, 0x8a2: 0x0d62, 0x8a3: 0x0df6,
+	0x8a4: 0x0f4a, 0x8a5: 0x0fbe, 0x8a6: 0x115a, 0x8a7: 0x12fa, 0x8a8: 0x1306, 0x8a9: 0x145a,
+	0x8aa: 0x14da, 0x8ab: 0x0886, 0x8ac: 0x0e4e, 0x8ad: 0x0906, 0x8ae: 0x0eca, 0x8af: 0x0f6e,
+	0x8b0: 0x128a, 0x8b1: 0x14c2, 0x8b2: 0x15ae, 0x8b3: 0x15d6, 0x8b4: 0x0d3a, 0x8b5: 0x0e2a,
+	0x8b6: 0x11c6, 0x8b7: 0x10ba, 0x8b8: 0x10c6, 0x8b9: 0x10ea, 0x8ba: 0x0f1a, 0x8bb: 0x0ea2,
+	0x8bc: 0x1366, 0x8bd: 0x0736, 0x8be: 0x122e, 0x8bf: 0x081e,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x080e, 0x8c1: 0x0b0e, 0x8c2: 0x0c2e, 0x8c3: 0x10f6, 0x8c4: 0x0a56, 0x8c5: 0x0e06,
+	0x8c6: 0x0cf2, 0x8c7: 0x13ea, 0x8c8: 0x12ea, 0x8c9: 0x14ae, 0x8ca: 0x1326, 0x8cb: 0x0b2a,
+	0x8cc: 0x078a, 0x8cd: 0x095e, 0x8d0: 0x09b2,
+	0x8d2: 0x0ce2, 0x8d5: 0x07fa, 0x8d6: 0x0f22, 0x8d7: 0x0fe6,
+	0x8d8: 0x104a, 0x8d9: 0x1066, 0x8da: 0x106a, 0x8db: 0x107e, 0x8dc: 0x14fe, 0x8dd: 0x10ee,
+	0x8de: 0x1172, 0x8e0: 0x1292, 0x8e2: 0x1356,
+	0x8e5: 0x140a, 0x8e6: 0x1436,
+	0x8ea: 0x1552, 0x8eb: 0x1556, 0x8ec: 0x155a, 0x8ed: 0x15be, 0x8ee: 0x142e, 0x8ef: 0x14ca,
+	0x8f0: 0x075a, 0x8f1: 0x077e, 0x8f2: 0x0792, 0x8f3: 0x084e, 0x8f4: 0x085a, 0x8f5: 0x089a,
+	0x8f6: 0x094e, 0x8f7: 0x096a, 0x8f8: 0x0972, 0x8f9: 0x09ae, 0x8fa: 0x09ba, 0x8fb: 0x0a96,
+	0x8fc: 0x0a9e, 0x8fd: 0x0ba6, 0x8fe: 0x0bce, 0x8ff: 0x0bd6,
+	// Block 0x24, offset 0x900
+	0x900: 0x0bee, 0x901: 0x0c9a, 0x902: 0x0cca, 0x903: 0x0cea, 0x904: 0x0d5a, 0x905: 0x0e1e,
+	0x906: 0x0e3a, 0x907: 0x0e6a, 0x908: 0x0ebe, 0x909: 0x0ede, 0x90a: 0x0f52, 0x90b: 0x1032,
+	0x90c: 0x104e, 0x90d: 0x1056, 0x90e: 0x1052, 0x90f: 0x105a, 0x910: 0x105e, 0x911: 0x1062,
+	0x912: 0x1076, 0x913: 0x107a, 0x914: 0x109e, 0x915: 0x10b2, 0x916: 0x10ce, 0x917: 0x1132,
+	0x918: 0x113a, 0x919: 0x1142, 0x91a: 0x1156, 0x91b: 0x117e, 0x91c: 0x11ce, 0x91d: 0x1202,
+	0x91e: 0x1202, 0x91f: 0x126a, 0x920: 0x1312, 0x921: 0x132a, 0x922: 0x135e, 0x923: 0x1362,
+	0x924: 0x13a6, 0x925: 0x13aa, 0x926: 0x1402, 0x927: 0x140a, 0x928: 0x14de, 0x929: 0x1522,
+	0x92a: 0x153a, 0x92b: 0x0b9e, 0x92c: 0x1721, 0x92d: 0x11e6,
+	0x930: 0x06e2, 0x931: 0x07e6, 0x932: 0x07a6, 0x933: 0x074e, 0x934: 0x078e, 0x935: 0x07ba,
+	0x936: 0x084a, 0x937: 0x0866, 0x938: 0x094e, 0x939: 0x093a, 0x93a: 0x094a, 0x93b: 0x0966,
+	0x93c: 0x09b2, 0x93d: 0x09c2, 0x93e: 0x0a06, 0x93f: 0x0a12,
+	// Block 0x25, offset 0x940
+	0x940: 0x0a2e, 0x941: 0x0a3e, 0x942: 0x0b26, 0x943: 0x0b2e, 0x944: 0x0b5e, 0x945: 0x0b7e,
+	0x946: 0x0bae, 0x947: 0x0bc6, 0x948: 0x0bb6, 0x949: 0x0bd6, 0x94a: 0x0bca, 0x94b: 0x0bee,
+	0x94c: 0x0c0a, 0x94d: 0x0c62, 0x94e: 0x0c6e, 0x94f: 0x0c76, 0x950: 0x0c9e, 0x951: 0x0ce2,
+	0x952: 0x0d12, 0x953: 0x0d16, 0x954: 0x0d2a, 0x955: 0x0daa, 0x956: 0x0dba, 0x957: 0x0e12,
+	0x958: 0x0e5e, 0x959: 0x0e56, 0x95a: 0x0e6a, 0x95b: 0x0e86, 0x95c: 0x0ebe, 0x95d: 0x1016,
+	0x95e: 0x0ee2, 0x95f: 0x0f16, 0x960: 0x0f22, 0x961: 0x0f62, 0x962: 0x0f7e, 0x963: 0x0fa2,
+	0x964: 0x0fc6, 0x965: 0x0fca, 0x966: 0x0fe6, 0x967: 0x0fea, 0x968: 0x0ffa, 0x969: 0x100e,
+	0x96a: 0x100a, 0x96b: 0x103a, 0x96c: 0x10b6, 0x96d: 0x10ce, 0x96e: 0x10e6, 0x96f: 0x111e,
+	0x970: 0x1132, 0x971: 0x114e, 0x972: 0x117e, 0x973: 0x1232, 0x974: 0x125a, 0x975: 0x12ce,
+	0x976: 0x1316, 0x977: 0x1322, 0x978: 0x132a, 0x979: 0x1342, 0x97a: 0x1356, 0x97b: 0x1346,
+	0x97c: 0x135e, 0x97d: 0x135a, 0x97e: 0x1352, 0x97f: 0x1362,
+	// Block 0x26, offset 0x980
+	0x980: 0x136e, 0x981: 0x13aa, 0x982: 0x13e6, 0x983: 0x1416, 0x984: 0x144e, 0x985: 0x146e,
+	0x986: 0x14ba, 0x987: 0x14de, 0x988: 0x14fe, 0x989: 0x1512, 0x98a: 0x1522, 0x98b: 0x152e,
+	0x98c: 0x153a, 0x98d: 0x158e, 0x98e: 0x162e, 0x98f: 0x16b8, 0x990: 0x16b3, 0x991: 0x16e5,
+	0x992: 0x060a, 0x993: 0x0632, 0x994: 0x0636, 0x995: 0x1767, 0x996: 0x1794, 0x997: 0x180c,
+	0x998: 0x161a, 0x999: 0x162a,
+	// Block 0x27, offset 0x9c0
+	0x9c0: 0x06fe, 0x9c1: 0x06f6, 0x9c2: 0x0706, 0x9c3: 0x164a, 0x9c4: 0x074a, 0x9c5: 0x075a,
+	0x9c6: 0x075e, 0x9c7: 0x0766, 0x9c8: 0x076e, 0x9c9: 0x0772, 0x9ca: 0x077e, 0x9cb: 0x0776,
+	0x9cc: 0x05b6, 0x9cd: 0x165e, 0x9ce: 0x0792, 0x9cf: 0x0796, 0x9d0: 0x079a, 0x9d1: 0x07b6,
+	0x9d2: 0x164f, 0x9d3: 0x05ba, 0x9d4: 0x07a2, 0x9d5: 0x07c2, 0x9d6: 0x1659, 0x9d7: 0x07d2,
+	0x9d8: 0x07da, 0x9d9: 0x073a, 0x9da: 0x07e2, 0x9db: 0x07e6, 0x9dc: 0x1834, 0x9dd: 0x0802,
+	0x9de: 0x080a, 0x9df: 0x05c2, 0x9e0: 0x0822, 0x9e1: 0x0826, 0x9e2: 0x082e, 0x9e3: 0x0832,
+	0x9e4: 0x05c6, 0x9e5: 0x084a, 0x9e6: 0x084e, 0x9e7: 0x085a, 0x9e8: 0x0866, 0x9e9: 0x086a,
+	0x9ea: 0x086e, 0x9eb: 0x0876, 0x9ec: 0x0896, 0x9ed: 0x089a, 0x9ee: 0x08a2, 0x9ef: 0x08b2,
+	0x9f0: 0x08ba, 0x9f1: 0x08be, 0x9f2: 0x08be, 0x9f3: 0x08be, 0x9f4: 0x166d, 0x9f5: 0x0e96,
+	0x9f6: 0x08d2, 0x9f7: 0x08da, 0x9f8: 0x1672, 0x9f9: 0x08e6, 0x9fa: 0x08ee, 0x9fb: 0x08f6,
+	0x9fc: 0x091e, 0x9fd: 0x090a, 0x9fe: 0x0916, 0x9ff: 0x091a,
+	// Block 0x28, offset 0xa00
+	0xa00: 0x0922, 0xa01: 0x092a, 0xa02: 0x092e, 0xa03: 0x0936, 0xa04: 0x093e, 0xa05: 0x0942,
+	0xa06: 0x0942, 0xa07: 0x094a, 0xa08: 0x0952, 0xa09: 0x0956, 0xa0a: 0x0962, 0xa0b: 0x0986,
+	0xa0c: 0x096a, 0xa0d: 0x098a, 0xa0e: 0x096e, 0xa0f: 0x0976, 0xa10: 0x080e, 0xa11: 0x09d2,
+	0xa12: 0x099a, 0xa13: 0x099e, 0xa14: 0x09a2, 0xa15: 0x0996, 0xa16: 0x09aa, 0xa17: 0x09a6,
+	0xa18: 0x09be, 0xa19: 0x1677, 0xa1a: 0x09da, 0xa1b: 0x09de, 0xa1c: 0x09e6, 0xa1d: 0x09f2,
+	0xa1e: 0x09fa, 0xa1f: 0x0a16, 0xa20: 0x167c, 0xa21: 0x1681, 0xa22: 0x0a22, 0xa23: 0x0a26,
+	0xa24: 0x0a2a, 0xa25: 0x0a1e, 0xa26: 0x0a32, 0xa27: 0x05ca, 0xa28: 0x05ce, 0xa29: 0x0a3a,
+	0xa2a: 0x0a42, 0xa2b: 0x0a42, 0xa2c: 0x1686, 0xa2d: 0x0a5e, 0xa2e: 0x0a62, 0xa2f: 0x0a66,
+	0xa30: 0x0a6e, 0xa31: 0x168b, 0xa32: 0x0a76, 0xa33: 0x0a7a, 0xa34: 0x0b52, 0xa35: 0x0a82,
+	0xa36: 0x05d2, 0xa37: 0x0a8e, 0xa38: 0x0a9e, 0xa39: 0x0aaa, 0xa3a: 0x0aa6, 0xa3b: 0x1695,
+	0xa3c: 0x0ab2, 0xa3d: 0x169a, 0xa3e: 0x0abe, 0xa3f: 0x0aba,
+	// Block 0x29, offset 0xa40
+	0xa40: 0x0ac2, 0xa41: 0x0ad2, 0xa42: 0x0ad6, 0xa43: 0x05d6, 0xa44: 0x0ae6, 0xa45: 0x0aee,
+	0xa46: 0x0af2, 0xa47: 0x0af6, 0xa48: 0x05da, 0xa49: 0x169f, 0xa4a: 0x05de, 0xa4b: 0x0b12,
+	0xa4c: 0x0b16, 0xa4d: 0x0b1a, 0xa4e: 0x0b22, 0xa4f: 0x1866, 0xa50: 0x0b3a, 0xa51: 0x16a9,
+	0xa52: 0x16a9, 0xa53: 0x11da, 0xa54: 0x0b4a, 0xa55: 0x0b4a, 0xa56: 0x05e2, 0xa57: 0x16cc,
+	0xa58: 0x179e, 0xa59: 0x0b5a, 0xa5a: 0x0b62, 0xa5b: 0x05e6, 0xa5c: 0x0b76, 0xa5d: 0x0b86,
+	0xa5e: 0x0b8a, 0xa5f: 0x0b92, 0xa60: 0x0ba2, 0xa61: 0x05ee, 0xa62: 0x05ea, 0xa63: 0x0ba6,
+	0xa64: 0x16ae, 0xa65: 0x0baa, 0xa66: 0x0bbe, 0xa67: 0x0bc2, 0xa68: 0x0bc6, 0xa69: 0x0bc2,
+	0xa6a: 0x0bd2, 0xa6b: 0x0bd6, 0xa6c: 0x0be6, 0xa6d: 0x0bde, 0xa6e: 0x0be2, 0xa6f: 0x0bea,
+	0xa70: 0x0bee, 0xa71: 0x0bf2, 0xa72: 0x0bfe, 0xa73: 0x0c02, 0xa74: 0x0c1a, 0xa75: 0x0c22,
+	0xa76: 0x0c32, 0xa77: 0x0c46, 0xa78: 0x16bd, 0xa79: 0x0c42, 0xa7a: 0x0c36, 0xa7b: 0x0c4e,
+	0xa7c: 0x0c56, 0xa7d: 0x0c6a, 0xa7e: 0x16c2, 0xa7f: 0x0c72,
+	// Block 0x2a, offset 0xa80
+	0xa80: 0x0c66, 0xa81: 0x0c5e, 0xa82: 0x05f2, 0xa83: 0x0c7a, 0xa84: 0x0c82, 0xa85: 0x0c8a,
+	0xa86: 0x0c7e, 0xa87: 0x05f6, 0xa88: 0x0c9a, 0xa89: 0x0ca2, 0xa8a: 0x16c7, 0xa8b: 0x0cce,
+	0xa8c: 0x0d02, 0xa8d: 0x0cde, 0xa8e: 0x0602, 0xa8f: 0x0cea, 0xa90: 0x05fe, 0xa91: 0x05fa,
+	0xa92: 0x07c6, 0xa93: 0x07ca, 0xa94: 0x0d06, 0xa95: 0x0cee, 0xa96: 0x11ae, 0xa97: 0x0666,
+	0xa98: 0x0d12, 0xa99: 0x0d16, 0xa9a: 0x0d1a, 0xa9b: 0x0d2e, 0xa9c: 0x0d26, 0xa9d: 0x16e0,
+	0xa9e: 0x0606, 0xa9f: 0x0d42, 0xaa0: 0x0d36, 0xaa1: 0x0d52, 0xaa2: 0x0d5a, 0xaa3: 0x16ea,
+	0xaa4: 0x0d5e, 0xaa5: 0x0d4a, 0xaa6: 0x0d66, 0xaa7: 0x060a, 0xaa8: 0x0d6a, 0xaa9: 0x0d6e,
+	0xaaa: 0x0d72, 0xaab: 0x0d7e, 0xaac: 0x16ef, 0xaad: 0x0d86, 0xaae: 0x060e, 0xaaf: 0x0d92,
+	0xab0: 0x16f4, 0xab1: 0x0d96, 0xab2: 0x0612, 0xab3: 0x0da2, 0xab4: 0x0dae, 0xab5: 0x0dba,
+	0xab6: 0x0dbe, 0xab7: 0x16f9, 0xab8: 0x1690, 0xab9: 0x16fe, 0xaba: 0x0dde, 0xabb: 0x1703,
+	0xabc: 0x0dea, 0xabd: 0x0df2, 0xabe: 0x0de2, 0xabf: 0x0dfe,
+	// Block 0x2b, offset 0xac0
+	0xac0: 0x0e0e, 0xac1: 0x0e1e, 0xac2: 0x0e12, 0xac3: 0x0e16, 0xac4: 0x0e22, 0xac5: 0x0e26,
+	0xac6: 0x1708, 0xac7: 0x0e0a, 0xac8: 0x0e3e, 0xac9: 0x0e42, 0xaca: 0x0616, 0xacb: 0x0e56,
+	0xacc: 0x0e52, 0xacd: 0x170d, 0xace: 0x0e36, 0xacf: 0x0e72, 0xad0: 0x1712, 0xad1: 0x1717,
+	0xad2: 0x0e76, 0xad3: 0x0e8a, 0xad4: 0x0e86, 0xad5: 0x0e82, 0xad6: 0x061a, 0xad7: 0x0e8e,
+	0xad8: 0x0e9e, 0xad9: 0x0e9a, 0xada: 0x0ea6, 0xadb: 0x1654, 0xadc: 0x0eb6, 0xadd: 0x171c,
+	0xade: 0x0ec2, 0xadf: 0x1726, 0xae0: 0x0ed6, 0xae1: 0x0ee2, 0xae2: 0x0ef6, 0xae3: 0x172b,
+	0xae4: 0x0f0a, 0xae5: 0x0f0e, 0xae6: 0x1730, 0xae7: 0x1735, 0xae8: 0x0f2a, 0xae9: 0x0f3a,
+	0xaea: 0x061e, 0xaeb: 0x0f3e, 0xaec: 0x0622, 0xaed: 0x0622, 0xaee: 0x0f56, 0xaef: 0x0f5a,
+	0xaf0: 0x0f62, 0xaf1: 0x0f66, 0xaf2: 0x0f72, 0xaf3: 0x0626, 0xaf4: 0x0f8a, 0xaf5: 0x173a,
+	0xaf6: 0x0fa6, 0xaf7: 0x173f, 0xaf8: 0x0fb2, 0xaf9: 0x16a4, 0xafa: 0x0fc2, 0xafb: 0x1744,
+	0xafc: 0x1749, 0xafd: 0x174e, 0xafe: 0x062a, 0xaff: 0x062e,
+	// Block 0x2c, offset 0xb00
+	0xb00: 0x0ffa, 0xb01: 0x1758, 0xb02: 0x1753, 0xb03: 0x175d, 0xb04: 0x1762, 0xb05: 0x1002,
+	0xb06: 0x1006, 0xb07: 0x1006, 0xb08: 0x100e, 0xb09: 0x0636, 0xb0a: 0x1012, 0xb0b: 0x063a,
+	0xb0c: 0x063e, 0xb0d: 0x176c, 0xb0e: 0x1026, 0xb0f: 0x102e, 0xb10: 0x103a, 0xb11: 0x0642,
+	0xb12: 0x1771, 0xb13: 0x105e, 0xb14: 0x1776, 0xb15: 0x177b, 0xb16: 0x107e, 0xb17: 0x1096,
+	0xb18: 0x0646, 0xb19: 0x109e, 0xb1a: 0x10a2, 0xb1b: 0x10a6, 0xb1c: 0x1780, 0xb1d: 0x1785,
+	0xb1e: 0x1785, 0xb1f: 0x10be, 0xb20: 0x064a, 0xb21: 0x178a, 0xb22: 0x10d2, 0xb23: 0x10d6,
+	0xb24: 0x064e, 0xb25: 0x178f, 0xb26: 0x10f2, 0xb27: 0x0652, 0xb28: 0x1102, 0xb29: 0x10fa,
+	0xb2a: 0x110a, 0xb2b: 0x1799, 0xb2c: 0x1122, 0xb2d: 0x0656, 0xb2e: 0x112e, 0xb2f: 0x1136,
+	0xb30: 0x1146, 0xb31: 0x065a, 0xb32: 0x17a3, 0xb33: 0x17a8, 0xb34: 0x065e, 0xb35: 0x17ad,
+	0xb36: 0x115e, 0xb37: 0x17b2, 0xb38: 0x116a, 0xb39: 0x1176, 0xb3a: 0x117e, 0xb3b: 0x17b7,
+	0xb3c: 0x17bc, 0xb3d: 0x1192, 0xb3e: 0x17c1, 0xb3f: 0x119a,
+	// Block 0x2d, offset 0xb40
+	0xb40: 0x16d1, 0xb41: 0x0662, 0xb42: 0x11b2, 0xb43: 0x11b6, 0xb44: 0x066a, 0xb45: 0x11ba,
+	0xb46: 0x0a36, 0xb47: 0x17c6, 0xb48: 0x17cb, 0xb49: 0x16d6, 0xb4a: 0x16db, 0xb4b: 0x11da,
+	0xb4c: 0x11de, 0xb4d: 0x13f6, 0xb4e: 0x066e, 0xb4f: 0x120a, 0xb50: 0x1206, 0xb51: 0x120e,
+	0xb52: 0x0842, 0xb53: 0x1212, 0xb54: 0x1216, 0xb55: 0x121a, 0xb56: 0x1222, 0xb57: 0x17d0,
+	0xb58: 0x121e, 0xb59: 0x1226, 0xb5a: 0x123a, 0xb5b: 0x123e, 0xb5c: 0x122a, 0xb5d: 0x1242,
+	0xb5e: 0x1256, 0xb5f: 0x126a, 0xb60: 0x1236, 0xb61: 0x124a, 0xb62: 0x124e, 0xb63: 0x1252,
+	0xb64: 0x17d5, 0xb65: 0x17df, 0xb66: 0x17da, 0xb67: 0x0672, 0xb68: 0x1272, 0xb69: 0x1276,
+	0xb6a: 0x127e, 0xb6b: 0x17f3, 0xb6c: 0x1282, 0xb6d: 0x17e4, 0xb6e: 0x0676, 0xb6f: 0x067a,
+	0xb70: 0x17e9, 0xb71: 0x17ee, 0xb72: 0x067e, 0xb73: 0x12a2, 0xb74: 0x12a6, 0xb75: 0x12aa,
+	0xb76: 0x12ae, 0xb77: 0x12ba, 0xb78: 0x12b6, 0xb79: 0x12c2, 0xb7a: 0x12be, 0xb7b: 0x12ce,
+	0xb7c: 0x12c6, 0xb7d: 0x12ca, 0xb7e: 0x12d2, 0xb7f: 0x0682,
+	// Block 0x2e, offset 0xb80
+	0xb80: 0x12da, 0xb81: 0x12de, 0xb82: 0x0686, 0xb83: 0x12ee, 0xb84: 0x12f2, 0xb85: 0x17f8,
+	0xb86: 0x12fe, 0xb87: 0x1302, 0xb88: 0x068a, 0xb89: 0x130e, 0xb8a: 0x05be, 0xb8b: 0x17fd,
+	0xb8c: 0x1802, 0xb8d: 0x068e, 0xb8e: 0x0692, 0xb8f: 0x133a, 0xb90: 0x1352, 0xb91: 0x136e,
+	0xb92: 0x137e, 0xb93: 0x1807, 0xb94: 0x1392, 0xb95: 0x1396, 0xb96: 0x13ae, 0xb97: 0x13ba,
+	0xb98: 0x1811, 0xb99: 0x1663, 0xb9a: 0x13c6, 0xb9b: 0x13c2, 0xb9c: 0x13ce, 0xb9d: 0x1668,
+	0xb9e: 0x13da, 0xb9f: 0x13e6, 0xba0: 0x1816, 0xba1: 0x181b, 0xba2: 0x1426, 0xba3: 0x1432,
+	0xba4: 0x143a, 0xba5: 0x1820, 0xba6: 0x143e, 0xba7: 0x146a, 0xba8: 0x1476, 0xba9: 0x147a,
+	0xbaa: 0x1472, 0xbab: 0x1486, 0xbac: 0x148a, 0xbad: 0x1825, 0xbae: 0x1496, 0xbaf: 0x0696,
+	0xbb0: 0x149e, 0xbb1: 0x182a, 0xbb2: 0x069a, 0xbb3: 0x14d6, 0xbb4: 0x0ac6, 0xbb5: 0x14ee,
+	0xbb6: 0x182f, 0xbb7: 0x1839, 0xbb8: 0x069e, 0xbb9: 0x06a2, 0xbba: 0x1516, 0xbbb: 0x183e,
+	0xbbc: 0x06a6, 0xbbd: 0x1843, 0xbbe: 0x152e, 0xbbf: 0x152e,
+	// Block 0x2f, offset 0xbc0
+	0xbc0: 0x1536, 0xbc1: 0x1848, 0xbc2: 0x154e, 0xbc3: 0x06aa, 0xbc4: 0x155e, 0xbc5: 0x156a,
+	0xbc6: 0x1572, 0xbc7: 0x157a, 0xbc8: 0x06ae, 0xbc9: 0x184d, 0xbca: 0x158e, 0xbcb: 0x15aa,
+	0xbcc: 0x15b6, 0xbcd: 0x06b2, 0xbce: 0x06b6, 0xbcf: 0x15ba, 0xbd0: 0x1852, 0xbd1: 0x06ba,
+	0xbd2: 0x1857, 0xbd3: 0x185c, 0xbd4: 0x1861, 0xbd5: 0x15de, 0xbd6: 0x06be, 0xbd7: 0x15f2,
+	0xbd8: 0x15fa, 0xbd9: 0x15fe, 0xbda: 0x1606, 0xbdb: 0x160e, 0xbdc: 0x1616, 0xbdd: 0x186b,
+}
+
+// nfcIndex: 22 blocks, 1408 entries, 1408 bytes
+// Block 0 is the zero block.
+var nfcIndex = [1408]uint8{
+	// Block 0x0, offset 0x0
+	// Block 0x1, offset 0x40
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc2: 0x2e, 0xc3: 0x01, 0xc4: 0x02, 0xc5: 0x03, 0xc6: 0x2f, 0xc7: 0x04,
+	0xc8: 0x05, 0xca: 0x30, 0xcb: 0x31, 0xcc: 0x06, 0xcd: 0x07, 0xce: 0x08, 0xcf: 0x32,
+	0xd0: 0x09, 0xd1: 0x33, 0xd2: 0x34, 0xd3: 0x0a, 0xd6: 0x0b, 0xd7: 0x35,
+	0xd8: 0x36, 0xd9: 0x0c, 0xdb: 0x37, 0xdc: 0x38, 0xdd: 0x39, 0xdf: 0x3a,
+	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05,
+	0xea: 0x06, 0xeb: 0x07, 0xec: 0x08, 0xed: 0x09, 0xef: 0x0a,
+	0xf0: 0x13,
+	// Block 0x4, offset 0x100
+	0x120: 0x3b, 0x121: 0x3c, 0x123: 0x0d, 0x124: 0x3d, 0x125: 0x3e, 0x126: 0x3f, 0x127: 0x40,
+	0x128: 0x41, 0x129: 0x42, 0x12a: 0x43, 0x12b: 0x44, 0x12c: 0x3f, 0x12d: 0x45, 0x12e: 0x46, 0x12f: 0x47,
+	0x131: 0x48, 0x132: 0x49, 0x133: 0x4a, 0x134: 0x4b, 0x135: 0x4c, 0x137: 0x4d,
+	0x138: 0x4e, 0x139: 0x4f, 0x13a: 0x50, 0x13b: 0x51, 0x13c: 0x52, 0x13d: 0x53, 0x13e: 0x54, 0x13f: 0x55,
+	// Block 0x5, offset 0x140
+	0x140: 0x56, 0x142: 0x57, 0x144: 0x58, 0x145: 0x59, 0x146: 0x5a, 0x147: 0x5b,
+	0x14d: 0x5c,
+	0x15c: 0x5d, 0x15f: 0x5e,
+	0x162: 0x5f, 0x164: 0x60,
+	0x168: 0x61, 0x169: 0x62, 0x16a: 0x63, 0x16b: 0x64, 0x16c: 0x0e, 0x16d: 0x65, 0x16e: 0x66, 0x16f: 0x67,
+	0x170: 0x68, 0x173: 0x69, 0x177: 0x0f,
+	0x178: 0x10, 0x179: 0x11, 0x17a: 0x12, 0x17b: 0x13, 0x17c: 0x14, 0x17d: 0x15, 0x17e: 0x16, 0x17f: 0x17,
+	// Block 0x6, offset 0x180
+	0x180: 0x6a, 0x183: 0x6b, 0x184: 0x6c, 0x186: 0x6d, 0x187: 0x6e,
+	0x188: 0x6f, 0x189: 0x18, 0x18a: 0x19, 0x18b: 0x70, 0x18c: 0x71,
+	0x1ab: 0x72,
+	0x1b3: 0x73, 0x1b5: 0x74, 0x1b7: 0x75,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x76, 0x1c1: 0x1a, 0x1c2: 0x1b, 0x1c3: 0x1c, 0x1c4: 0x77, 0x1c5: 0x78,
+	0x1c9: 0x79, 0x1cc: 0x7a, 0x1cd: 0x7b,
+	// Block 0x8, offset 0x200
+	0x219: 0x7c, 0x21a: 0x7d, 0x21b: 0x7e,
+	0x220: 0x7f, 0x223: 0x80, 0x224: 0x81, 0x225: 0x82, 0x226: 0x83, 0x227: 0x84,
+	0x22a: 0x85, 0x22b: 0x86, 0x22f: 0x87,
+	0x230: 0x88, 0x231: 0x89, 0x232: 0x8a, 0x233: 0x8b, 0x234: 0x8c, 0x235: 0x8d, 0x236: 0x8e, 0x237: 0x88,
+	0x238: 0x89, 0x239: 0x8a, 0x23a: 0x8b, 0x23b: 0x8c, 0x23c: 0x8d, 0x23d: 0x8e, 0x23e: 0x88, 0x23f: 0x89,
+	// Block 0x9, offset 0x240
+	0x240: 0x8a, 0x241: 0x8b, 0x242: 0x8c, 0x243: 0x8d, 0x244: 0x8e, 0x245: 0x88, 0x246: 0x89, 0x247: 0x8a,
+	0x248: 0x8b, 0x249: 0x8c, 0x24a: 0x8d, 0x24b: 0x8e, 0x24c: 0x88, 0x24d: 0x89, 0x24e: 0x8a, 0x24f: 0x8b,
+	0x250: 0x8c, 0x251: 0x8d, 0x252: 0x8e, 0x253: 0x88, 0x254: 0x89, 0x255: 0x8a, 0x256: 0x8b, 0x257: 0x8c,
+	0x258: 0x8d, 0x259: 0x8e, 0x25a: 0x88, 0x25b: 0x89, 0x25c: 0x8a, 0x25d: 0x8b, 0x25e: 0x8c, 0x25f: 0x8d,
+	0x260: 0x8e, 0x261: 0x88, 0x262: 0x89, 0x263: 0x8a, 0x264: 0x8b, 0x265: 0x8c, 0x266: 0x8d, 0x267: 0x8e,
+	0x268: 0x88, 0x269: 0x89, 0x26a: 0x8a, 0x26b: 0x8b, 0x26c: 0x8c, 0x26d: 0x8d, 0x26e: 0x8e, 0x26f: 0x88,
+	0x270: 0x89, 0x271: 0x8a, 0x272: 0x8b, 0x273: 0x8c, 0x274: 0x8d, 0x275: 0x8e, 0x276: 0x88, 0x277: 0x89,
+	0x278: 0x8a, 0x279: 0x8b, 0x27a: 0x8c, 0x27b: 0x8d, 0x27c: 0x8e, 0x27d: 0x88, 0x27e: 0x89, 0x27f: 0x8a,
+	// Block 0xa, offset 0x280
+	0x280: 0x8b, 0x281: 0x8c, 0x282: 0x8d, 0x283: 0x8e, 0x284: 0x88, 0x285: 0x89, 0x286: 0x8a, 0x287: 0x8b,
+	0x288: 0x8c, 0x289: 0x8d, 0x28a: 0x8e, 0x28b: 0x88, 0x28c: 0x89, 0x28d: 0x8a, 0x28e: 0x8b, 0x28f: 0x8c,
+	0x290: 0x8d, 0x291: 0x8e, 0x292: 0x88, 0x293: 0x89, 0x294: 0x8a, 0x295: 0x8b, 0x296: 0x8c, 0x297: 0x8d,
+	0x298: 0x8e, 0x299: 0x88, 0x29a: 0x89, 0x29b: 0x8a, 0x29c: 0x8b, 0x29d: 0x8c, 0x29e: 0x8d, 0x29f: 0x8e,
+	0x2a0: 0x88, 0x2a1: 0x89, 0x2a2: 0x8a, 0x2a3: 0x8b, 0x2a4: 0x8c, 0x2a5: 0x8d, 0x2a6: 0x8e, 0x2a7: 0x88,
+	0x2a8: 0x89, 0x2a9: 0x8a, 0x2aa: 0x8b, 0x2ab: 0x8c, 0x2ac: 0x8d, 0x2ad: 0x8e, 0x2ae: 0x88, 0x2af: 0x89,
+	0x2b0: 0x8a, 0x2b1: 0x8b, 0x2b2: 0x8c, 0x2b3: 0x8d, 0x2b4: 0x8e, 0x2b5: 0x88, 0x2b6: 0x89, 0x2b7: 0x8a,
+	0x2b8: 0x8b, 0x2b9: 0x8c, 0x2ba: 0x8d, 0x2bb: 0x8e, 0x2bc: 0x88, 0x2bd: 0x89, 0x2be: 0x8a, 0x2bf: 0x8b,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0x8c, 0x2c1: 0x8d, 0x2c2: 0x8e, 0x2c3: 0x88, 0x2c4: 0x89, 0x2c5: 0x8a, 0x2c6: 0x8b, 0x2c7: 0x8c,
+	0x2c8: 0x8d, 0x2c9: 0x8e, 0x2ca: 0x88, 0x2cb: 0x89, 0x2cc: 0x8a, 0x2cd: 0x8b, 0x2ce: 0x8c, 0x2cf: 0x8d,
+	0x2d0: 0x8e, 0x2d1: 0x88, 0x2d2: 0x89, 0x2d3: 0x8a, 0x2d4: 0x8b, 0x2d5: 0x8c, 0x2d6: 0x8d, 0x2d7: 0x8e,
+	0x2d8: 0x88, 0x2d9: 0x89, 0x2da: 0x8a, 0x2db: 0x8b, 0x2dc: 0x8c, 0x2dd: 0x8d, 0x2de: 0x8f,
+	// Block 0xc, offset 0x300
+	0x324: 0x1d, 0x325: 0x1e, 0x326: 0x1f, 0x327: 0x20,
+	0x328: 0x21, 0x329: 0x22, 0x32a: 0x23, 0x32b: 0x24, 0x32c: 0x90, 0x32d: 0x91, 0x32e: 0x92,
+	0x331: 0x93, 0x332: 0x94, 0x333: 0x95, 0x334: 0x96,
+	0x338: 0x97, 0x339: 0x98, 0x33a: 0x99, 0x33b: 0x9a, 0x33e: 0x9b, 0x33f: 0x9c,
+	// Block 0xd, offset 0x340
+	0x347: 0x9d,
+	0x34b: 0x9e, 0x34d: 0x9f,
+	0x368: 0xa0, 0x36b: 0xa1,
+	0x374: 0xa2,
+	0x37a: 0xa3, 0x37d: 0xa4,
+	// Block 0xe, offset 0x380
+	0x381: 0xa5, 0x382: 0xa6, 0x384: 0xa7, 0x385: 0x83, 0x387: 0xa8,
+	0x388: 0xa9, 0x38b: 0xaa, 0x38c: 0xab, 0x38d: 0xac,
+	0x391: 0xad, 0x392: 0xae, 0x393: 0xaf, 0x396: 0xb0, 0x397: 0xb1,
+	0x398: 0x74, 0x39a: 0xb2, 0x39c: 0xb3,
+	0x3a0: 0xb4, 0x3a4: 0xb5, 0x3a5: 0xb6, 0x3a7: 0xb7,
+	0x3a8: 0xb8, 0x3a9: 0xb9, 0x3aa: 0xba,
+	0x3b0: 0x74, 0x3b5: 0xbb, 0x3b6: 0xbc,
+	// Block 0xf, offset 0x3c0
+	0x3eb: 0xbd, 0x3ec: 0xbe,
+	0x3ff: 0xbf,
+	// Block 0x10, offset 0x400
+	0x432: 0xc0,
+	// Block 0x11, offset 0x440
+	0x445: 0xc1, 0x446: 0xc2, 0x447: 0xc3,
+	0x449: 0xc4,
+	// Block 0x12, offset 0x480
+	0x480: 0xc5, 0x484: 0xbe,
+	0x48b: 0xc6,
+	0x4a3: 0xc7, 0x4a5: 0xc8,
+	// Block 0x13, offset 0x4c0
+	0x4c8: 0xc9,
+	// Block 0x14, offset 0x500
+	0x520: 0x25, 0x521: 0x26, 0x522: 0x27, 0x523: 0x28, 0x524: 0x29, 0x525: 0x2a, 0x526: 0x2b, 0x527: 0x2c,
+	0x528: 0x2d,
+	// Block 0x15, offset 0x540
+	0x550: 0x0b, 0x551: 0x0c, 0x556: 0x0d,
+	0x55b: 0x0e, 0x55d: 0x0f, 0x55e: 0x10, 0x55f: 0x11,
+	0x56f: 0x12,
+}
+
+// nfcSparseOffset: 156 entries, 312 bytes
+var nfcSparseOffset = []uint16{0x0, 0x5, 0x9, 0xb, 0xd, 0x18, 0x28, 0x2a, 0x2f, 0x3a, 0x49, 0x56, 0x5e, 0x63, 0x68, 0x6a, 0x72, 0x79, 0x7c, 0x84, 0x88, 0x8c, 0x8e, 0x90, 0x99, 0x9d, 0xa4, 0xa9, 0xac, 0xb6, 0xb9, 0xc0, 0xc8, 0xcb, 0xcd, 0xd0, 0xd2, 0xd7, 0xe8, 0xf4, 0xf6, 0xfc, 0xfe, 0x100, 0x102, 0x104, 0x106, 0x108, 0x10b, 0x10e, 0x110, 0x113, 0x116, 0x11a, 0x120, 0x122, 0x12b, 0x12d, 0x130, 0x132, 0x13d, 0x141, 0x14f, 0x152, 0x158, 0x15e, 0x169, 0x16d, 0x16f, 0x171, 0x173, 0x175, 0x177, 0x17d, 0x181, 0x183, 0x185, 0x18d, 0x191, 0x194, 0x196, 0x198, 0x19b, 0x19e, 0x1a0, 0x1a2, 0x1a4, 0x1a6, 0x1ac, 0x1af, 0x1b1, 0x1b8, 0x1be, 0x1c4, 0x1cc, 0x1d2, 0x1d8, 0x1de, 0x1e2, 0x1f0, 0x1f9, 0x1fc, 0x1ff, 0x201, 0x204, 0x206, 0x20a, 0x20f, 0x211, 0x213, 0x218, 0x21e, 0x220, 0x222, 0x224, 0x22a, 0x22d, 0x22f, 0x231, 0x237, 0x23a, 0x242, 0x249, 0x24c, 0x24f, 0x251, 0x254, 0x25c, 0x260, 0x267, 0x26a, 0x270, 0x272, 0x275, 0x277, 0x27a, 0x27f, 0x281, 0x283, 0x285, 0x287, 0x289, 0x28c, 0x28e, 0x290, 0x292, 0x294, 0x296, 0x2a3, 0x2ad, 0x2af, 0x2b1, 0x2b7, 0x2b9, 0x2bb, 0x2be}
+
+// nfcSparseValues: 704 entries, 2816 bytes
+var nfcSparseValues = [704]valueRange{
+	// Block 0x0, offset 0x0
+	{value: 0x0000, lo: 0x04},
+	{value: 0xa100, lo: 0xa8, hi: 0xa8},
+	{value: 0x8100, lo: 0xaf, hi: 0xaf},
+	{value: 0x8100, lo: 0xb4, hi: 0xb4},
+	{value: 0x8100, lo: 0xb8, hi: 0xb8},
+	// Block 0x1, offset 0x5
+	{value: 0x0091, lo: 0x03},
+	{value: 0x46f9, lo: 0xa0, hi: 0xa1},
+	{value: 0x472b, lo: 0xaf, hi: 0xb0},
+	{value: 0xa000, lo: 0xb7, hi: 0xb7},
+	// Block 0x2, offset 0x9
+	{value: 0x0000, lo: 0x01},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	// Block 0x3, offset 0xb
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0x98, hi: 0x9d},
+	// Block 0x4, offset 0xd
+	{value: 0x0006, lo: 0x0a},
+	{value: 0xa000, lo: 0x81, hi: 0x81},
+	{value: 0xa000, lo: 0x85, hi: 0x85},
+	{value: 0xa000, lo: 0x89, hi: 0x89},
+	{value: 0x4857, lo: 0x8a, hi: 0x8a},
+	{value: 0x4875, lo: 0x8b, hi: 0x8b},
+	{value: 0x36de, lo: 0x8c, hi: 0x8c},
+	{value: 0x36f6, lo: 0x8d, hi: 0x8d},
+	{value: 0x488d, lo: 0x8e, hi: 0x8e},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x3714, lo: 0x93, hi: 0x94},
+	// Block 0x5, offset 0x18
+	{value: 0x0000, lo: 0x0f},
+	{value: 0xa000, lo: 0x83, hi: 0x83},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0xa000, lo: 0x8b, hi: 0x8b},
+	{value: 0xa000, lo: 0x8d, hi: 0x8d},
+	{value: 0x37bc, lo: 0x90, hi: 0x90},
+	{value: 0x37c8, lo: 0x91, hi: 0x91},
+	{value: 0x37b6, lo: 0x93, hi: 0x93},
+	{value: 0xa000, lo: 0x96, hi: 0x96},
+	{value: 0x382e, lo: 0x97, hi: 0x97},
+	{value: 0x37f8, lo: 0x9c, hi: 0x9c},
+	{value: 0x37e0, lo: 0x9d, hi: 0x9d},
+	{value: 0x380a, lo: 0x9e, hi: 0x9e},
+	{value: 0xa000, lo: 0xb4, hi: 0xb5},
+	{value: 0x3834, lo: 0xb6, hi: 0xb6},
+	{value: 0x383a, lo: 0xb7, hi: 0xb7},
+	// Block 0x6, offset 0x28
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x83, hi: 0x87},
+	// Block 0x7, offset 0x2a
+	{value: 0x0001, lo: 0x04},
+	{value: 0x8114, lo: 0x81, hi: 0x82},
+	{value: 0x8133, lo: 0x84, hi: 0x84},
+	{value: 0x812e, lo: 0x85, hi: 0x85},
+	{value: 0x810e, lo: 0x87, hi: 0x87},
+	// Block 0x8, offset 0x2f
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x8133, lo: 0x90, hi: 0x97},
+	{value: 0x811a, lo: 0x98, hi: 0x98},
+	{value: 0x811b, lo: 0x99, hi: 0x99},
+	{value: 0x811c, lo: 0x9a, hi: 0x9a},
+	{value: 0x3858, lo: 0xa2, hi: 0xa2},
+	{value: 0x385e, lo: 0xa3, hi: 0xa3},
+	{value: 0x386a, lo: 0xa4, hi: 0xa4},
+	{value: 0x3864, lo: 0xa5, hi: 0xa5},
+	{value: 0x3870, lo: 0xa6, hi: 0xa6},
+	{value: 0xa000, lo: 0xa7, hi: 0xa7},
+	// Block 0x9, offset 0x3a
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3882, lo: 0x80, hi: 0x80},
+	{value: 0xa000, lo: 0x81, hi: 0x81},
+	{value: 0x3876, lo: 0x82, hi: 0x82},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x387c, lo: 0x93, hi: 0x93},
+	{value: 0xa000, lo: 0x95, hi: 0x95},
+	{value: 0x8133, lo: 0x96, hi: 0x9c},
+	{value: 0x8133, lo: 0x9f, hi: 0xa2},
+	{value: 0x812e, lo: 0xa3, hi: 0xa3},
+	{value: 0x8133, lo: 0xa4, hi: 0xa4},
+	{value: 0x8133, lo: 0xa7, hi: 0xa8},
+	{value: 0x812e, lo: 0xaa, hi: 0xaa},
+	{value: 0x8133, lo: 0xab, hi: 0xac},
+	{value: 0x812e, lo: 0xad, hi: 0xad},
+	// Block 0xa, offset 0x49
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x8120, lo: 0x91, hi: 0x91},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	{value: 0x812e, lo: 0xb1, hi: 0xb1},
+	{value: 0x8133, lo: 0xb2, hi: 0xb3},
+	{value: 0x812e, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb5, hi: 0xb6},
+	{value: 0x812e, lo: 0xb7, hi: 0xb9},
+	{value: 0x8133, lo: 0xba, hi: 0xba},
+	{value: 0x812e, lo: 0xbb, hi: 0xbc},
+	{value: 0x8133, lo: 0xbd, hi: 0xbd},
+	{value: 0x812e, lo: 0xbe, hi: 0xbe},
+	{value: 0x8133, lo: 0xbf, hi: 0xbf},
+	// Block 0xb, offset 0x56
+	{value: 0x0005, lo: 0x07},
+	{value: 0x8133, lo: 0x80, hi: 0x80},
+	{value: 0x8133, lo: 0x81, hi: 0x81},
+	{value: 0x812e, lo: 0x82, hi: 0x83},
+	{value: 0x812e, lo: 0x84, hi: 0x85},
+	{value: 0x812e, lo: 0x86, hi: 0x87},
+	{value: 0x812e, lo: 0x88, hi: 0x89},
+	{value: 0x8133, lo: 0x8a, hi: 0x8a},
+	// Block 0xc, offset 0x5e
+	{value: 0x0000, lo: 0x04},
+	{value: 0x8133, lo: 0xab, hi: 0xb1},
+	{value: 0x812e, lo: 0xb2, hi: 0xb2},
+	{value: 0x8133, lo: 0xb3, hi: 0xb3},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	// Block 0xd, offset 0x63
+	{value: 0x0000, lo: 0x04},
+	{value: 0x8133, lo: 0x96, hi: 0x99},
+	{value: 0x8133, lo: 0x9b, hi: 0xa3},
+	{value: 0x8133, lo: 0xa5, hi: 0xa7},
+	{value: 0x8133, lo: 0xa9, hi: 0xad},
+	// Block 0xe, offset 0x68
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x99, hi: 0x9b},
+	// Block 0xf, offset 0x6a
+	{value: 0x0000, lo: 0x07},
+	{value: 0xa000, lo: 0xa8, hi: 0xa8},
+	{value: 0x3eef, lo: 0xa9, hi: 0xa9},
+	{value: 0xa000, lo: 0xb0, hi: 0xb0},
+	{value: 0x3ef7, lo: 0xb1, hi: 0xb1},
+	{value: 0xa000, lo: 0xb3, hi: 0xb3},
+	{value: 0x3eff, lo: 0xb4, hi: 0xb4},
+	{value: 0x9903, lo: 0xbc, hi: 0xbc},
+	// Block 0x10, offset 0x72
+	{value: 0x0008, lo: 0x06},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x8133, lo: 0x91, hi: 0x91},
+	{value: 0x812e, lo: 0x92, hi: 0x92},
+	{value: 0x8133, lo: 0x93, hi: 0x93},
+	{value: 0x8133, lo: 0x94, hi: 0x94},
+	{value: 0x4533, lo: 0x98, hi: 0x9f},
+	// Block 0x11, offset 0x79
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x12, offset 0x7c
+	{value: 0x0008, lo: 0x07},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2cab, lo: 0x8b, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	{value: 0x4573, lo: 0x9c, hi: 0x9d},
+	{value: 0x4583, lo: 0x9f, hi: 0x9f},
+	{value: 0x8133, lo: 0xbe, hi: 0xbe},
+	// Block 0x13, offset 0x84
+	{value: 0x0000, lo: 0x03},
+	{value: 0x45ab, lo: 0xb3, hi: 0xb3},
+	{value: 0x45b3, lo: 0xb6, hi: 0xb6},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	// Block 0x14, offset 0x88
+	{value: 0x0008, lo: 0x03},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x458b, lo: 0x99, hi: 0x9b},
+	{value: 0x45a3, lo: 0x9e, hi: 0x9e},
+	// Block 0x15, offset 0x8c
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	// Block 0x16, offset 0x8e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	// Block 0x17, offset 0x90
+	{value: 0x0000, lo: 0x08},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2cc3, lo: 0x88, hi: 0x88},
+	{value: 0x2cbb, lo: 0x8b, hi: 0x8b},
+	{value: 0x2ccb, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x96, hi: 0x97},
+	{value: 0x45bb, lo: 0x9c, hi: 0x9c},
+	{value: 0x45c3, lo: 0x9d, hi: 0x9d},
+	// Block 0x18, offset 0x99
+	{value: 0x0000, lo: 0x03},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x2cd3, lo: 0x94, hi: 0x94},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x19, offset 0x9d
+	{value: 0x0000, lo: 0x06},
+	{value: 0xa000, lo: 0x86, hi: 0x87},
+	{value: 0x2cdb, lo: 0x8a, hi: 0x8a},
+	{value: 0x2ceb, lo: 0x8b, hi: 0x8b},
+	{value: 0x2ce3, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	// Block 0x1a, offset 0xa4
+	{value: 0x1801, lo: 0x04},
+	{value: 0xa000, lo: 0x86, hi: 0x86},
+	{value: 0x3f07, lo: 0x88, hi: 0x88},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x8121, lo: 0x95, hi: 0x96},
+	// Block 0x1b, offset 0xa9
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	{value: 0xa000, lo: 0xbf, hi: 0xbf},
+	// Block 0x1c, offset 0xac
+	{value: 0x0000, lo: 0x09},
+	{value: 0x2cf3, lo: 0x80, hi: 0x80},
+	{value: 0x9900, lo: 0x82, hi: 0x82},
+	{value: 0xa000, lo: 0x86, hi: 0x86},
+	{value: 0x2cfb, lo: 0x87, hi: 0x87},
+	{value: 0x2d03, lo: 0x88, hi: 0x88},
+	{value: 0x2f67, lo: 0x8a, hi: 0x8a},
+	{value: 0x2def, lo: 0x8b, hi: 0x8b},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x95, hi: 0x96},
+	// Block 0x1d, offset 0xb6
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xbb, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x1e, offset 0xb9
+	{value: 0x0000, lo: 0x06},
+	{value: 0xa000, lo: 0x86, hi: 0x87},
+	{value: 0x2d0b, lo: 0x8a, hi: 0x8a},
+	{value: 0x2d1b, lo: 0x8b, hi: 0x8b},
+	{value: 0x2d13, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	// Block 0x1f, offset 0xc0
+	{value: 0x6bdd, lo: 0x07},
+	{value: 0x9905, lo: 0x8a, hi: 0x8a},
+	{value: 0x9900, lo: 0x8f, hi: 0x8f},
+	{value: 0xa000, lo: 0x99, hi: 0x99},
+	{value: 0x3f0f, lo: 0x9a, hi: 0x9a},
+	{value: 0x2f6f, lo: 0x9c, hi: 0x9c},
+	{value: 0x2dfa, lo: 0x9d, hi: 0x9d},
+	{value: 0x2d23, lo: 0x9e, hi: 0x9f},
+	// Block 0x20, offset 0xc8
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8123, lo: 0xb8, hi: 0xb9},
+	{value: 0x8105, lo: 0xba, hi: 0xba},
+	// Block 0x21, offset 0xcb
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8124, lo: 0x88, hi: 0x8b},
+	// Block 0x22, offset 0xcd
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8125, lo: 0xb8, hi: 0xb9},
+	{value: 0x8105, lo: 0xba, hi: 0xba},
+	// Block 0x23, offset 0xd0
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8126, lo: 0x88, hi: 0x8b},
+	// Block 0x24, offset 0xd2
+	{value: 0x0000, lo: 0x04},
+	{value: 0x812e, lo: 0x98, hi: 0x99},
+	{value: 0x812e, lo: 0xb5, hi: 0xb5},
+	{value: 0x812e, lo: 0xb7, hi: 0xb7},
+	{value: 0x812c, lo: 0xb9, hi: 0xb9},
+	// Block 0x25, offset 0xd7
+	{value: 0x0000, lo: 0x10},
+	{value: 0x264a, lo: 0x83, hi: 0x83},
+	{value: 0x2651, lo: 0x8d, hi: 0x8d},
+	{value: 0x2658, lo: 0x92, hi: 0x92},
+	{value: 0x265f, lo: 0x97, hi: 0x97},
+	{value: 0x2666, lo: 0x9c, hi: 0x9c},
+	{value: 0x2643, lo: 0xa9, hi: 0xa9},
+	{value: 0x8127, lo: 0xb1, hi: 0xb1},
+	{value: 0x8128, lo: 0xb2, hi: 0xb2},
+	{value: 0x4a9b, lo: 0xb3, hi: 0xb3},
+	{value: 0x8129, lo: 0xb4, hi: 0xb4},
+	{value: 0x4aa4, lo: 0xb5, hi: 0xb5},
+	{value: 0x45cb, lo: 0xb6, hi: 0xb6},
+	{value: 0x8200, lo: 0xb7, hi: 0xb7},
+	{value: 0x45d3, lo: 0xb8, hi: 0xb8},
+	{value: 0x8200, lo: 0xb9, hi: 0xb9},
+	{value: 0x8128, lo: 0xba, hi: 0xbd},
+	// Block 0x26, offset 0xe8
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x8128, lo: 0x80, hi: 0x80},
+	{value: 0x4aad, lo: 0x81, hi: 0x81},
+	{value: 0x8133, lo: 0x82, hi: 0x83},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0x86, hi: 0x87},
+	{value: 0x2674, lo: 0x93, hi: 0x93},
+	{value: 0x267b, lo: 0x9d, hi: 0x9d},
+	{value: 0x2682, lo: 0xa2, hi: 0xa2},
+	{value: 0x2689, lo: 0xa7, hi: 0xa7},
+	{value: 0x2690, lo: 0xac, hi: 0xac},
+	{value: 0x266d, lo: 0xb9, hi: 0xb9},
+	// Block 0x27, offset 0xf4
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x86, hi: 0x86},
+	// Block 0x28, offset 0xf6
+	{value: 0x0000, lo: 0x05},
+	{value: 0xa000, lo: 0xa5, hi: 0xa5},
+	{value: 0x2d2b, lo: 0xa6, hi: 0xa6},
+	{value: 0x9900, lo: 0xae, hi: 0xae},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	{value: 0x8105, lo: 0xb9, hi: 0xba},
+	// Block 0x29, offset 0xfc
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x8d, hi: 0x8d},
+	// Block 0x2a, offset 0xfe
+	{value: 0x0000, lo: 0x01},
+	{value: 0xa000, lo: 0x80, hi: 0x92},
+	// Block 0x2b, offset 0x100
+	{value: 0x0000, lo: 0x01},
+	{value: 0xb900, lo: 0xa1, hi: 0xb5},
+	// Block 0x2c, offset 0x102
+	{value: 0x0000, lo: 0x01},
+	{value: 0x9900, lo: 0xa8, hi: 0xbf},
+	// Block 0x2d, offset 0x104
+	{value: 0x0000, lo: 0x01},
+	{value: 0x9900, lo: 0x80, hi: 0x82},
+	// Block 0x2e, offset 0x106
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x9d, hi: 0x9f},
+	// Block 0x2f, offset 0x108
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x94, hi: 0x94},
+	{value: 0x8105, lo: 0xb4, hi: 0xb4},
+	// Block 0x30, offset 0x10b
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x92, hi: 0x92},
+	{value: 0x8133, lo: 0x9d, hi: 0x9d},
+	// Block 0x31, offset 0x10e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8132, lo: 0xa9, hi: 0xa9},
+	// Block 0x32, offset 0x110
+	{value: 0x0004, lo: 0x02},
+	{value: 0x812f, lo: 0xb9, hi: 0xba},
+	{value: 0x812e, lo: 0xbb, hi: 0xbb},
+	// Block 0x33, offset 0x113
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x97, hi: 0x97},
+	{value: 0x812e, lo: 0x98, hi: 0x98},
+	// Block 0x34, offset 0x116
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8105, lo: 0xa0, hi: 0xa0},
+	{value: 0x8133, lo: 0xb5, hi: 0xbc},
+	{value: 0x812e, lo: 0xbf, hi: 0xbf},
+	// Block 0x35, offset 0x11a
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0xb0, hi: 0xb4},
+	{value: 0x812e, lo: 0xb5, hi: 0xba},
+	{value: 0x8133, lo: 0xbb, hi: 0xbc},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	{value: 0x812e, lo: 0xbf, hi: 0xbf},
+	// Block 0x36, offset 0x120
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x80, hi: 0x80},
+	// Block 0x37, offset 0x122
+	{value: 0x0000, lo: 0x08},
+	{value: 0x2d73, lo: 0x80, hi: 0x80},
+	{value: 0x2d7b, lo: 0x81, hi: 0x81},
+	{value: 0xa000, lo: 0x82, hi: 0x82},
+	{value: 0x2d83, lo: 0x83, hi: 0x83},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0xab, hi: 0xab},
+	{value: 0x812e, lo: 0xac, hi: 0xac},
+	{value: 0x8133, lo: 0xad, hi: 0xb3},
+	// Block 0x38, offset 0x12b
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xaa, hi: 0xab},
+	// Block 0x39, offset 0x12d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xa6, hi: 0xa6},
+	{value: 0x8105, lo: 0xb2, hi: 0xb3},
+	// Block 0x3a, offset 0x130
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	// Block 0x3b, offset 0x132
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x8133, lo: 0x90, hi: 0x92},
+	{value: 0x8101, lo: 0x94, hi: 0x94},
+	{value: 0x812e, lo: 0x95, hi: 0x99},
+	{value: 0x8133, lo: 0x9a, hi: 0x9b},
+	{value: 0x812e, lo: 0x9c, hi: 0x9f},
+	{value: 0x8133, lo: 0xa0, hi: 0xa0},
+	{value: 0x8101, lo: 0xa2, hi: 0xa8},
+	{value: 0x812e, lo: 0xad, hi: 0xad},
+	{value: 0x8133, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb8, hi: 0xb9},
+	// Block 0x3c, offset 0x13d
+	{value: 0x0004, lo: 0x03},
+	{value: 0x0436, lo: 0x80, hi: 0x81},
+	{value: 0x8100, lo: 0x97, hi: 0x97},
+	{value: 0x8100, lo: 0xbe, hi: 0xbe},
+	// Block 0x3d, offset 0x141
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x8133, lo: 0x90, hi: 0x91},
+	{value: 0x8101, lo: 0x92, hi: 0x93},
+	{value: 0x8133, lo: 0x94, hi: 0x97},
+	{value: 0x8101, lo: 0x98, hi: 0x9a},
+	{value: 0x8133, lo: 0x9b, hi: 0x9c},
+	{value: 0x8133, lo: 0xa1, hi: 0xa1},
+	{value: 0x8101, lo: 0xa5, hi: 0xa6},
+	{value: 0x8133, lo: 0xa7, hi: 0xa7},
+	{value: 0x812e, lo: 0xa8, hi: 0xa8},
+	{value: 0x8133, lo: 0xa9, hi: 0xa9},
+	{value: 0x8101, lo: 0xaa, hi: 0xab},
+	{value: 0x812e, lo: 0xac, hi: 0xaf},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	// Block 0x3e, offset 0x14f
+	{value: 0x4292, lo: 0x02},
+	{value: 0x01bb, lo: 0xa6, hi: 0xa6},
+	{value: 0x0057, lo: 0xaa, hi: 0xab},
+	// Block 0x3f, offset 0x152
+	{value: 0x0007, lo: 0x05},
+	{value: 0xa000, lo: 0x90, hi: 0x90},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0xa000, lo: 0x94, hi: 0x94},
+	{value: 0x3bd0, lo: 0x9a, hi: 0x9b},
+	{value: 0x3bde, lo: 0xae, hi: 0xae},
+	// Block 0x40, offset 0x158
+	{value: 0x000e, lo: 0x05},
+	{value: 0x3be5, lo: 0x8d, hi: 0x8e},
+	{value: 0x3bec, lo: 0x8f, hi: 0x8f},
+	{value: 0xa000, lo: 0x90, hi: 0x90},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0xa000, lo: 0x94, hi: 0x94},
+	// Block 0x41, offset 0x15e
+	{value: 0x63f1, lo: 0x0a},
+	{value: 0xa000, lo: 0x83, hi: 0x83},
+	{value: 0x3bfa, lo: 0x84, hi: 0x84},
+	{value: 0xa000, lo: 0x88, hi: 0x88},
+	{value: 0x3c01, lo: 0x89, hi: 0x89},
+	{value: 0xa000, lo: 0x8b, hi: 0x8b},
+	{value: 0x3c08, lo: 0x8c, hi: 0x8c},
+	{value: 0xa000, lo: 0xa3, hi: 0xa3},
+	{value: 0x3c0f, lo: 0xa4, hi: 0xa5},
+	{value: 0x3c16, lo: 0xa6, hi: 0xa6},
+	{value: 0xa000, lo: 0xbc, hi: 0xbc},
+	// Block 0x42, offset 0x169
+	{value: 0x0007, lo: 0x03},
+	{value: 0x3c7f, lo: 0xa0, hi: 0xa1},
+	{value: 0x3ca9, lo: 0xa2, hi: 0xa3},
+	{value: 0x3cd3, lo: 0xaa, hi: 0xad},
+	// Block 0x43, offset 0x16d
+	{value: 0x0004, lo: 0x01},
+	{value: 0x048e, lo: 0xa9, hi: 0xaa},
+	// Block 0x44, offset 0x16f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x44f4, lo: 0x9c, hi: 0x9c},
+	// Block 0x45, offset 0x171
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xaf, hi: 0xb1},
+	// Block 0x46, offset 0x173
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x47, offset 0x175
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xa0, hi: 0xbf},
+	// Block 0x48, offset 0x177
+	{value: 0x0000, lo: 0x05},
+	{value: 0x812d, lo: 0xaa, hi: 0xaa},
+	{value: 0x8132, lo: 0xab, hi: 0xab},
+	{value: 0x8134, lo: 0xac, hi: 0xac},
+	{value: 0x812f, lo: 0xad, hi: 0xad},
+	{value: 0x8130, lo: 0xae, hi: 0xaf},
+	// Block 0x49, offset 0x17d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x4ab6, lo: 0xb3, hi: 0xb3},
+	{value: 0x4ab6, lo: 0xb5, hi: 0xb6},
+	{value: 0x4ab6, lo: 0xba, hi: 0xbf},
+	// Block 0x4a, offset 0x181
+	{value: 0x0000, lo: 0x01},
+	{value: 0x4ab6, lo: 0x8f, hi: 0xa3},
+	// Block 0x4b, offset 0x183
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0xae, hi: 0xbe},
+	// Block 0x4c, offset 0x185
+	{value: 0x0000, lo: 0x07},
+	{value: 0x8100, lo: 0x84, hi: 0x84},
+	{value: 0x8100, lo: 0x87, hi: 0x87},
+	{value: 0x8100, lo: 0x90, hi: 0x90},
+	{value: 0x8100, lo: 0x9e, hi: 0x9e},
+	{value: 0x8100, lo: 0xa1, hi: 0xa1},
+	{value: 0x8100, lo: 0xb2, hi: 0xb2},
+	{value: 0x8100, lo: 0xbb, hi: 0xbb},
+	// Block 0x4d, offset 0x18d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8100, lo: 0x80, hi: 0x80},
+	{value: 0x8100, lo: 0x8b, hi: 0x8b},
+	{value: 0x8100, lo: 0x8e, hi: 0x8e},
+	// Block 0x4e, offset 0x191
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0xaf, hi: 0xaf},
+	{value: 0x8133, lo: 0xb4, hi: 0xbd},
+	// Block 0x4f, offset 0x194
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x9e, hi: 0x9f},
+	// Block 0x50, offset 0x196
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb0, hi: 0xb1},
+	// Block 0x51, offset 0x198
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x86, hi: 0x86},
+	{value: 0x8105, lo: 0xac, hi: 0xac},
+	// Block 0x52, offset 0x19b
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0xa0, hi: 0xb1},
+	// Block 0x53, offset 0x19e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xab, hi: 0xad},
+	// Block 0x54, offset 0x1a0
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x93, hi: 0x93},
+	// Block 0x55, offset 0x1a2
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xb3, hi: 0xb3},
+	// Block 0x56, offset 0x1a4
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x80, hi: 0x80},
+	// Block 0x57, offset 0x1a6
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	{value: 0x8133, lo: 0xb2, hi: 0xb3},
+	{value: 0x812e, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb7, hi: 0xb8},
+	{value: 0x8133, lo: 0xbe, hi: 0xbf},
+	// Block 0x58, offset 0x1ac
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x81, hi: 0x81},
+	{value: 0x8105, lo: 0xb6, hi: 0xb6},
+	// Block 0x59, offset 0x1af
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xad, hi: 0xad},
+	// Block 0x5a, offset 0x1b1
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe500, lo: 0x80, hi: 0x80},
+	{value: 0xc600, lo: 0x81, hi: 0x9b},
+	{value: 0xe500, lo: 0x9c, hi: 0x9c},
+	{value: 0xc600, lo: 0x9d, hi: 0xb7},
+	{value: 0xe500, lo: 0xb8, hi: 0xb8},
+	{value: 0xc600, lo: 0xb9, hi: 0xbf},
+	// Block 0x5b, offset 0x1b8
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x93},
+	{value: 0xe500, lo: 0x94, hi: 0x94},
+	{value: 0xc600, lo: 0x95, hi: 0xaf},
+	{value: 0xe500, lo: 0xb0, hi: 0xb0},
+	{value: 0xc600, lo: 0xb1, hi: 0xbf},
+	// Block 0x5c, offset 0x1be
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x8b},
+	{value: 0xe500, lo: 0x8c, hi: 0x8c},
+	{value: 0xc600, lo: 0x8d, hi: 0xa7},
+	{value: 0xe500, lo: 0xa8, hi: 0xa8},
+	{value: 0xc600, lo: 0xa9, hi: 0xbf},
+	// Block 0x5d, offset 0x1c4
+	{value: 0x0000, lo: 0x07},
+	{value: 0xc600, lo: 0x80, hi: 0x83},
+	{value: 0xe500, lo: 0x84, hi: 0x84},
+	{value: 0xc600, lo: 0x85, hi: 0x9f},
+	{value: 0xe500, lo: 0xa0, hi: 0xa0},
+	{value: 0xc600, lo: 0xa1, hi: 0xbb},
+	{value: 0xe500, lo: 0xbc, hi: 0xbc},
+	{value: 0xc600, lo: 0xbd, hi: 0xbf},
+	// Block 0x5e, offset 0x1cc
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x97},
+	{value: 0xe500, lo: 0x98, hi: 0x98},
+	{value: 0xc600, lo: 0x99, hi: 0xb3},
+	{value: 0xe500, lo: 0xb4, hi: 0xb4},
+	{value: 0xc600, lo: 0xb5, hi: 0xbf},
+	// Block 0x5f, offset 0x1d2
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x8f},
+	{value: 0xe500, lo: 0x90, hi: 0x90},
+	{value: 0xc600, lo: 0x91, hi: 0xab},
+	{value: 0xe500, lo: 0xac, hi: 0xac},
+	{value: 0xc600, lo: 0xad, hi: 0xbf},
+	// Block 0x60, offset 0x1d8
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x87},
+	{value: 0xe500, lo: 0x88, hi: 0x88},
+	{value: 0xc600, lo: 0x89, hi: 0xa3},
+	{value: 0xe500, lo: 0xa4, hi: 0xa4},
+	{value: 0xc600, lo: 0xa5, hi: 0xbf},
+	// Block 0x61, offset 0x1de
+	{value: 0x0000, lo: 0x03},
+	{value: 0xc600, lo: 0x80, hi: 0x87},
+	{value: 0xe500, lo: 0x88, hi: 0x88},
+	{value: 0xc600, lo: 0x89, hi: 0xa3},
+	// Block 0x62, offset 0x1e2
+	{value: 0x0006, lo: 0x0d},
+	{value: 0x43a7, lo: 0x9d, hi: 0x9d},
+	{value: 0x8116, lo: 0x9e, hi: 0x9e},
+	{value: 0x4419, lo: 0x9f, hi: 0x9f},
+	{value: 0x4407, lo: 0xaa, hi: 0xab},
+	{value: 0x450b, lo: 0xac, hi: 0xac},
+	{value: 0x4513, lo: 0xad, hi: 0xad},
+	{value: 0x435f, lo: 0xae, hi: 0xb1},
+	{value: 0x437d, lo: 0xb2, hi: 0xb4},
+	{value: 0x4395, lo: 0xb5, hi: 0xb6},
+	{value: 0x43a1, lo: 0xb8, hi: 0xb8},
+	{value: 0x43ad, lo: 0xb9, hi: 0xbb},
+	{value: 0x43c5, lo: 0xbc, hi: 0xbc},
+	{value: 0x43cb, lo: 0xbe, hi: 0xbe},
+	// Block 0x63, offset 0x1f0
+	{value: 0x0006, lo: 0x08},
+	{value: 0x43d1, lo: 0x80, hi: 0x81},
+	{value: 0x43dd, lo: 0x83, hi: 0x84},
+	{value: 0x43ef, lo: 0x86, hi: 0x89},
+	{value: 0x4413, lo: 0x8a, hi: 0x8a},
+	{value: 0x438f, lo: 0x8b, hi: 0x8b},
+	{value: 0x4377, lo: 0x8c, hi: 0x8c},
+	{value: 0x43bf, lo: 0x8d, hi: 0x8d},
+	{value: 0x43e9, lo: 0x8e, hi: 0x8e},
+	// Block 0x64, offset 0x1f9
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8100, lo: 0xa4, hi: 0xa5},
+	{value: 0x8100, lo: 0xb0, hi: 0xb1},
+	// Block 0x65, offset 0x1fc
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8100, lo: 0x9b, hi: 0x9d},
+	{value: 0x8200, lo: 0x9e, hi: 0xa3},
+	// Block 0x66, offset 0x1ff
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0x90, hi: 0x90},
+	// Block 0x67, offset 0x201
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8100, lo: 0x99, hi: 0x99},
+	{value: 0x8200, lo: 0xb2, hi: 0xb4},
+	// Block 0x68, offset 0x204
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0xbc, hi: 0xbd},
+	// Block 0x69, offset 0x206
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8133, lo: 0xa0, hi: 0xa6},
+	{value: 0x812e, lo: 0xa7, hi: 0xad},
+	{value: 0x8133, lo: 0xae, hi: 0xaf},
+	// Block 0x6a, offset 0x20a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x8100, lo: 0x89, hi: 0x8c},
+	{value: 0x8100, lo: 0xb0, hi: 0xb2},
+	{value: 0x8100, lo: 0xb4, hi: 0xb4},
+	{value: 0x8100, lo: 0xb6, hi: 0xbf},
+	// Block 0x6b, offset 0x20f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0x81, hi: 0x8c},
+	// Block 0x6c, offset 0x211
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0xb5, hi: 0xba},
+	// Block 0x6d, offset 0x213
+	{value: 0x0000, lo: 0x04},
+	{value: 0x4ab6, lo: 0x9e, hi: 0x9f},
+	{value: 0x4ab6, lo: 0xa3, hi: 0xa3},
+	{value: 0x4ab6, lo: 0xa5, hi: 0xa6},
+	{value: 0x4ab6, lo: 0xaa, hi: 0xaf},
+	// Block 0x6e, offset 0x218
+	{value: 0x0000, lo: 0x05},
+	{value: 0x4ab6, lo: 0x82, hi: 0x87},
+	{value: 0x4ab6, lo: 0x8a, hi: 0x8f},
+	{value: 0x4ab6, lo: 0x92, hi: 0x97},
+	{value: 0x4ab6, lo: 0x9a, hi: 0x9c},
+	{value: 0x8100, lo: 0xa3, hi: 0xa3},
+	// Block 0x6f, offset 0x21e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	// Block 0x70, offset 0x220
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xa0, hi: 0xa0},
+	// Block 0x71, offset 0x222
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb6, hi: 0xba},
+	// Block 0x72, offset 0x224
+	{value: 0x002d, lo: 0x05},
+	{value: 0x812e, lo: 0x8d, hi: 0x8d},
+	{value: 0x8133, lo: 0x8f, hi: 0x8f},
+	{value: 0x8133, lo: 0xb8, hi: 0xb8},
+	{value: 0x8101, lo: 0xb9, hi: 0xba},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x73, offset 0x22a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0xa5, hi: 0xa5},
+	{value: 0x812e, lo: 0xa6, hi: 0xa6},
+	// Block 0x74, offset 0x22d
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xa4, hi: 0xa7},
+	// Block 0x75, offset 0x22f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xab, hi: 0xac},
+	// Block 0x76, offset 0x231
+	{value: 0x0000, lo: 0x05},
+	{value: 0x812e, lo: 0x86, hi: 0x87},
+	{value: 0x8133, lo: 0x88, hi: 0x8a},
+	{value: 0x812e, lo: 0x8b, hi: 0x8b},
+	{value: 0x8133, lo: 0x8c, hi: 0x8c},
+	{value: 0x812e, lo: 0x8d, hi: 0x90},
+	// Block 0x77, offset 0x237
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x86, hi: 0x86},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x78, offset 0x23a
+	{value: 0x17fe, lo: 0x07},
+	{value: 0xa000, lo: 0x99, hi: 0x99},
+	{value: 0x424f, lo: 0x9a, hi: 0x9a},
+	{value: 0xa000, lo: 0x9b, hi: 0x9b},
+	{value: 0x4259, lo: 0x9c, hi: 0x9c},
+	{value: 0xa000, lo: 0xa5, hi: 0xa5},
+	{value: 0x4263, lo: 0xab, hi: 0xab},
+	{value: 0x8105, lo: 0xb9, hi: 0xba},
+	// Block 0x79, offset 0x242
+	{value: 0x0000, lo: 0x06},
+	{value: 0x8133, lo: 0x80, hi: 0x82},
+	{value: 0x9900, lo: 0xa7, hi: 0xa7},
+	{value: 0x2d8b, lo: 0xae, hi: 0xae},
+	{value: 0x2d95, lo: 0xaf, hi: 0xaf},
+	{value: 0xa000, lo: 0xb1, hi: 0xb2},
+	{value: 0x8105, lo: 0xb3, hi: 0xb4},
+	// Block 0x7a, offset 0x249
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x80, hi: 0x80},
+	{value: 0x8103, lo: 0x8a, hi: 0x8a},
+	// Block 0x7b, offset 0x24c
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb5, hi: 0xb5},
+	{value: 0x8103, lo: 0xb6, hi: 0xb6},
+	// Block 0x7c, offset 0x24f
+	{value: 0x0002, lo: 0x01},
+	{value: 0x8103, lo: 0xa9, hi: 0xaa},
+	// Block 0x7d, offset 0x251
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbb, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x7e, offset 0x254
+	{value: 0x0000, lo: 0x07},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2d9f, lo: 0x8b, hi: 0x8b},
+	{value: 0x2da9, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	{value: 0x8133, lo: 0xa6, hi: 0xac},
+	{value: 0x8133, lo: 0xb0, hi: 0xb4},
+	// Block 0x7f, offset 0x25c
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8105, lo: 0x82, hi: 0x82},
+	{value: 0x8103, lo: 0x86, hi: 0x86},
+	{value: 0x8133, lo: 0x9e, hi: 0x9e},
+	// Block 0x80, offset 0x260
+	{value: 0x6b4d, lo: 0x06},
+	{value: 0x9900, lo: 0xb0, hi: 0xb0},
+	{value: 0xa000, lo: 0xb9, hi: 0xb9},
+	{value: 0x9900, lo: 0xba, hi: 0xba},
+	{value: 0x2dbd, lo: 0xbb, hi: 0xbb},
+	{value: 0x2db3, lo: 0xbc, hi: 0xbd},
+	{value: 0x2dc7, lo: 0xbe, hi: 0xbe},
+	// Block 0x81, offset 0x267
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x82, hi: 0x82},
+	{value: 0x8103, lo: 0x83, hi: 0x83},
+	// Block 0x82, offset 0x26a
+	{value: 0x0000, lo: 0x05},
+	{value: 0x9900, lo: 0xaf, hi: 0xaf},
+	{value: 0xa000, lo: 0xb8, hi: 0xb9},
+	{value: 0x2dd1, lo: 0xba, hi: 0xba},
+	{value: 0x2ddb, lo: 0xbb, hi: 0xbb},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x83, offset 0x270
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0x80, hi: 0x80},
+	// Block 0x84, offset 0x272
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb6, hi: 0xb6},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	// Block 0x85, offset 0x275
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xab, hi: 0xab},
+	// Block 0x86, offset 0x277
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb9, hi: 0xb9},
+	{value: 0x8103, lo: 0xba, hi: 0xba},
+	// Block 0x87, offset 0x27a
+	{value: 0x0000, lo: 0x04},
+	{value: 0x9900, lo: 0xb0, hi: 0xb0},
+	{value: 0xa000, lo: 0xb5, hi: 0xb5},
+	{value: 0x2de5, lo: 0xb8, hi: 0xb8},
+	{value: 0x8105, lo: 0xbd, hi: 0xbe},
+	// Block 0x88, offset 0x27f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0x83, hi: 0x83},
+	// Block 0x89, offset 0x281
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xa0, hi: 0xa0},
+	// Block 0x8a, offset 0x283
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xb4, hi: 0xb4},
+	// Block 0x8b, offset 0x285
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x87, hi: 0x87},
+	// Block 0x8c, offset 0x287
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x99, hi: 0x99},
+	// Block 0x8d, offset 0x289
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0x82, hi: 0x82},
+	{value: 0x8105, lo: 0x84, hi: 0x85},
+	// Block 0x8e, offset 0x28c
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x97, hi: 0x97},
+	// Block 0x8f, offset 0x28e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8101, lo: 0xb0, hi: 0xb4},
+	// Block 0x90, offset 0x290
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb0, hi: 0xb6},
+	// Block 0x91, offset 0x292
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8102, lo: 0xb0, hi: 0xb1},
+	// Block 0x92, offset 0x294
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8101, lo: 0x9e, hi: 0x9e},
+	// Block 0x93, offset 0x296
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x45e3, lo: 0x9e, hi: 0x9e},
+	{value: 0x45ed, lo: 0x9f, hi: 0x9f},
+	{value: 0x4621, lo: 0xa0, hi: 0xa0},
+	{value: 0x462f, lo: 0xa1, hi: 0xa1},
+	{value: 0x463d, lo: 0xa2, hi: 0xa2},
+	{value: 0x464b, lo: 0xa3, hi: 0xa3},
+	{value: 0x4659, lo: 0xa4, hi: 0xa4},
+	{value: 0x812c, lo: 0xa5, hi: 0xa6},
+	{value: 0x8101, lo: 0xa7, hi: 0xa9},
+	{value: 0x8131, lo: 0xad, hi: 0xad},
+	{value: 0x812c, lo: 0xae, hi: 0xb2},
+	{value: 0x812e, lo: 0xbb, hi: 0xbf},
+	// Block 0x94, offset 0x2a3
+	{value: 0x0000, lo: 0x09},
+	{value: 0x812e, lo: 0x80, hi: 0x82},
+	{value: 0x8133, lo: 0x85, hi: 0x89},
+	{value: 0x812e, lo: 0x8a, hi: 0x8b},
+	{value: 0x8133, lo: 0xaa, hi: 0xad},
+	{value: 0x45f7, lo: 0xbb, hi: 0xbb},
+	{value: 0x4601, lo: 0xbc, hi: 0xbc},
+	{value: 0x4667, lo: 0xbd, hi: 0xbd},
+	{value: 0x4683, lo: 0xbe, hi: 0xbe},
+	{value: 0x4675, lo: 0xbf, hi: 0xbf},
+	// Block 0x95, offset 0x2ad
+	{value: 0x0000, lo: 0x01},
+	{value: 0x4691, lo: 0x80, hi: 0x80},
+	// Block 0x96, offset 0x2af
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x82, hi: 0x84},
+	// Block 0x97, offset 0x2b1
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0x80, hi: 0x86},
+	{value: 0x8133, lo: 0x88, hi: 0x98},
+	{value: 0x8133, lo: 0x9b, hi: 0xa1},
+	{value: 0x8133, lo: 0xa3, hi: 0xa4},
+	{value: 0x8133, lo: 0xa6, hi: 0xaa},
+	// Block 0x98, offset 0x2b7
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xac, hi: 0xaf},
+	// Block 0x99, offset 0x2b9
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x90, hi: 0x96},
+	// Block 0x9a, offset 0x2bb
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x84, hi: 0x89},
+	{value: 0x8103, lo: 0x8a, hi: 0x8a},
+	// Block 0x9b, offset 0x2be
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8100, lo: 0x93, hi: 0x93},
+}
+
+// lookup returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *nfkcTrie) lookup(s []byte) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return nfkcValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfkcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfkcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = nfkcIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *nfkcTrie) lookupUnsafe(s []byte) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return nfkcValues[c0]
+	}
+	i := nfkcIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = nfkcIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = nfkcIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// lookupString returns the trie value for the first UTF-8 encoding in s and
+// the width in bytes of this encoding. The size will be 0 if s does not
+// hold enough bytes to complete the encoding. len(s) must be greater than 0.
+func (t *nfkcTrie) lookupString(s string) (v uint16, sz int) {
+	c0 := s[0]
+	switch {
+	case c0 < 0x80: // is ASCII
+		return nfkcValues[c0], 1
+	case c0 < 0xC2:
+		return 0, 1 // Illegal UTF-8: not a starter, not ASCII.
+	case c0 < 0xE0: // 2-byte UTF-8
+		if len(s) < 2 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c1), 2
+	case c0 < 0xF0: // 3-byte UTF-8
+		if len(s) < 3 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfkcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c2), 3
+	case c0 < 0xF8: // 4-byte UTF-8
+		if len(s) < 4 {
+			return 0, 0
+		}
+		i := nfkcIndex[c0]
+		c1 := s[1]
+		if c1 < 0x80 || 0xC0 <= c1 {
+			return 0, 1 // Illegal UTF-8: not a continuation byte.
+		}
+		o := uint32(i)<<6 + uint32(c1)
+		i = nfkcIndex[o]
+		c2 := s[2]
+		if c2 < 0x80 || 0xC0 <= c2 {
+			return 0, 2 // Illegal UTF-8: not a continuation byte.
+		}
+		o = uint32(i)<<6 + uint32(c2)
+		i = nfkcIndex[o]
+		c3 := s[3]
+		if c3 < 0x80 || 0xC0 <= c3 {
+			return 0, 3 // Illegal UTF-8: not a continuation byte.
+		}
+		return t.lookupValue(uint32(i), c3), 4
+	}
+	// Illegal rune
+	return 0, 1
+}
+
+// lookupStringUnsafe returns the trie value for the first UTF-8 encoding in s.
+// s must start with a full and valid UTF-8 encoded rune.
+func (t *nfkcTrie) lookupStringUnsafe(s string) uint16 {
+	c0 := s[0]
+	if c0 < 0x80 { // is ASCII
+		return nfkcValues[c0]
+	}
+	i := nfkcIndex[c0]
+	if c0 < 0xE0 { // 2-byte UTF-8
+		return t.lookupValue(uint32(i), s[1])
+	}
+	i = nfkcIndex[uint32(i)<<6+uint32(s[1])]
+	if c0 < 0xF0 { // 3-byte UTF-8
+		return t.lookupValue(uint32(i), s[2])
+	}
+	i = nfkcIndex[uint32(i)<<6+uint32(s[2])]
+	if c0 < 0xF8 { // 4-byte UTF-8
+		return t.lookupValue(uint32(i), s[3])
+	}
+	return 0
+}
+
+// nfkcTrie. Total size: 18768 bytes (18.33 KiB). Checksum: c51186dd2412943d.
+type nfkcTrie struct{}
+
+func newNfkcTrie(i int) *nfkcTrie {
+	return &nfkcTrie{}
+}
+
+// lookupValue determines the type of block n and looks up the value for b.
+func (t *nfkcTrie) lookupValue(n uint32, b byte) uint16 {
+	switch {
+	case n < 92:
+		return uint16(nfkcValues[n<<6+uint32(b)])
+	default:
+		n -= 92
+		return uint16(nfkcSparse.lookup(n, b))
+	}
+}
+
+// nfkcValues: 94 blocks, 6016 entries, 12032 bytes
+// The third block is the zero block.
+var nfkcValues = [6016]uint16{
+	// Block 0x0, offset 0x0
+	0x3c: 0xa000, 0x3d: 0xa000, 0x3e: 0xa000,
+	// Block 0x1, offset 0x40
+	0x41: 0xa000, 0x42: 0xa000, 0x43: 0xa000, 0x44: 0xa000, 0x45: 0xa000,
+	0x46: 0xa000, 0x47: 0xa000, 0x48: 0xa000, 0x49: 0xa000, 0x4a: 0xa000, 0x4b: 0xa000,
+	0x4c: 0xa000, 0x4d: 0xa000, 0x4e: 0xa000, 0x4f: 0xa000, 0x50: 0xa000,
+	0x52: 0xa000, 0x53: 0xa000, 0x54: 0xa000, 0x55: 0xa000, 0x56: 0xa000, 0x57: 0xa000,
+	0x58: 0xa000, 0x59: 0xa000, 0x5a: 0xa000,
+	0x61: 0xa000, 0x62: 0xa000, 0x63: 0xa000,
+	0x64: 0xa000, 0x65: 0xa000, 0x66: 0xa000, 0x67: 0xa000, 0x68: 0xa000, 0x69: 0xa000,
+	0x6a: 0xa000, 0x6b: 0xa000, 0x6c: 0xa000, 0x6d: 0xa000, 0x6e: 0xa000, 0x6f: 0xa000,
+	0x70: 0xa000, 0x72: 0xa000, 0x73: 0xa000, 0x74: 0xa000, 0x75: 0xa000,
+	0x76: 0xa000, 0x77: 0xa000, 0x78: 0xa000, 0x79: 0xa000, 0x7a: 0xa000,
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc0: 0x2f86, 0xc1: 0x2f8b, 0xc2: 0x469f, 0xc3: 0x2f90, 0xc4: 0x46ae, 0xc5: 0x46b3,
+	0xc6: 0xa000, 0xc7: 0x46bd, 0xc8: 0x2ff9, 0xc9: 0x2ffe, 0xca: 0x46c2, 0xcb: 0x3012,
+	0xcc: 0x3085, 0xcd: 0x308a, 0xce: 0x308f, 0xcf: 0x46d6, 0xd1: 0x311b,
+	0xd2: 0x313e, 0xd3: 0x3143, 0xd4: 0x46e0, 0xd5: 0x46e5, 0xd6: 0x46f4,
+	0xd8: 0xa000, 0xd9: 0x31ca, 0xda: 0x31cf, 0xdb: 0x31d4, 0xdc: 0x4726, 0xdd: 0x324c,
+	0xe0: 0x3292, 0xe1: 0x3297, 0xe2: 0x4730, 0xe3: 0x329c,
+	0xe4: 0x473f, 0xe5: 0x4744, 0xe6: 0xa000, 0xe7: 0x474e, 0xe8: 0x3305, 0xe9: 0x330a,
+	0xea: 0x4753, 0xeb: 0x331e, 0xec: 0x3396, 0xed: 0x339b, 0xee: 0x33a0, 0xef: 0x4767,
+	0xf1: 0x342c, 0xf2: 0x344f, 0xf3: 0x3454, 0xf4: 0x4771, 0xf5: 0x4776,
+	0xf6: 0x4785, 0xf8: 0xa000, 0xf9: 0x34e0, 0xfa: 0x34e5, 0xfb: 0x34ea,
+	0xfc: 0x47b7, 0xfd: 0x3567, 0xff: 0x3580,
+	// Block 0x4, offset 0x100
+	0x100: 0x2f95, 0x101: 0x32a1, 0x102: 0x46a4, 0x103: 0x4735, 0x104: 0x2fb3, 0x105: 0x32bf,
+	0x106: 0x2fc7, 0x107: 0x32d3, 0x108: 0x2fcc, 0x109: 0x32d8, 0x10a: 0x2fd1, 0x10b: 0x32dd,
+	0x10c: 0x2fd6, 0x10d: 0x32e2, 0x10e: 0x2fe0, 0x10f: 0x32ec,
+	0x112: 0x46c7, 0x113: 0x4758, 0x114: 0x3008, 0x115: 0x3314, 0x116: 0x300d, 0x117: 0x3319,
+	0x118: 0x302b, 0x119: 0x3337, 0x11a: 0x301c, 0x11b: 0x3328, 0x11c: 0x3044, 0x11d: 0x3350,
+	0x11e: 0x304e, 0x11f: 0x335a, 0x120: 0x3053, 0x121: 0x335f, 0x122: 0x305d, 0x123: 0x3369,
+	0x124: 0x3062, 0x125: 0x336e, 0x128: 0x3094, 0x129: 0x33a5,
+	0x12a: 0x3099, 0x12b: 0x33aa, 0x12c: 0x309e, 0x12d: 0x33af, 0x12e: 0x30c1, 0x12f: 0x33cd,
+	0x130: 0x30a3, 0x132: 0x1960, 0x133: 0x19ed, 0x134: 0x30cb, 0x135: 0x33d7,
+	0x136: 0x30df, 0x137: 0x33f0, 0x139: 0x30e9, 0x13a: 0x33fa, 0x13b: 0x30f3,
+	0x13c: 0x3404, 0x13d: 0x30ee, 0x13e: 0x33ff, 0x13f: 0x1bb2,
+	// Block 0x5, offset 0x140
+	0x140: 0x1c3a, 0x143: 0x3116, 0x144: 0x3427, 0x145: 0x312f,
+	0x146: 0x3440, 0x147: 0x3125, 0x148: 0x3436, 0x149: 0x1c62,
+	0x14c: 0x46ea, 0x14d: 0x477b, 0x14e: 0x3148, 0x14f: 0x3459, 0x150: 0x3152, 0x151: 0x3463,
+	0x154: 0x3170, 0x155: 0x3481, 0x156: 0x3189, 0x157: 0x349a,
+	0x158: 0x317a, 0x159: 0x348b, 0x15a: 0x470d, 0x15b: 0x479e, 0x15c: 0x3193, 0x15d: 0x34a4,
+	0x15e: 0x31a2, 0x15f: 0x34b3, 0x160: 0x4712, 0x161: 0x47a3, 0x162: 0x31bb, 0x163: 0x34d1,
+	0x164: 0x31ac, 0x165: 0x34c2, 0x168: 0x471c, 0x169: 0x47ad,
+	0x16a: 0x4721, 0x16b: 0x47b2, 0x16c: 0x31d9, 0x16d: 0x34ef, 0x16e: 0x31e3, 0x16f: 0x34f9,
+	0x170: 0x31e8, 0x171: 0x34fe, 0x172: 0x3206, 0x173: 0x351c, 0x174: 0x3229, 0x175: 0x353f,
+	0x176: 0x3251, 0x177: 0x356c, 0x178: 0x3265, 0x179: 0x3274, 0x17a: 0x3594, 0x17b: 0x327e,
+	0x17c: 0x359e, 0x17d: 0x3283, 0x17e: 0x35a3, 0x17f: 0x00a7,
+	// Block 0x6, offset 0x180
+	0x184: 0x2e05, 0x185: 0x2e0b,
+	0x186: 0x2e11, 0x187: 0x1975, 0x188: 0x1978, 0x189: 0x1a0e, 0x18a: 0x198d, 0x18b: 0x1990,
+	0x18c: 0x1a44, 0x18d: 0x2f9f, 0x18e: 0x32ab, 0x18f: 0x30ad, 0x190: 0x33b9, 0x191: 0x3157,
+	0x192: 0x3468, 0x193: 0x31ed, 0x194: 0x3503, 0x195: 0x39e6, 0x196: 0x3b75, 0x197: 0x39df,
+	0x198: 0x3b6e, 0x199: 0x39ed, 0x19a: 0x3b7c, 0x19b: 0x39d8, 0x19c: 0x3b67,
+	0x19e: 0x38c7, 0x19f: 0x3a56, 0x1a0: 0x38c0, 0x1a1: 0x3a4f, 0x1a2: 0x35ca, 0x1a3: 0x35dc,
+	0x1a6: 0x3058, 0x1a7: 0x3364, 0x1a8: 0x30d5, 0x1a9: 0x33e6,
+	0x1aa: 0x4703, 0x1ab: 0x4794, 0x1ac: 0x39a7, 0x1ad: 0x3b36, 0x1ae: 0x35ee, 0x1af: 0x35f4,
+	0x1b0: 0x33dc, 0x1b1: 0x1945, 0x1b2: 0x1948, 0x1b3: 0x19d5, 0x1b4: 0x303f, 0x1b5: 0x334b,
+	0x1b8: 0x3111, 0x1b9: 0x3422, 0x1ba: 0x38ce, 0x1bb: 0x3a5d,
+	0x1bc: 0x35c4, 0x1bd: 0x35d6, 0x1be: 0x35d0, 0x1bf: 0x35e2,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0x2fa4, 0x1c1: 0x32b0, 0x1c2: 0x2fa9, 0x1c3: 0x32b5, 0x1c4: 0x3021, 0x1c5: 0x332d,
+	0x1c6: 0x3026, 0x1c7: 0x3332, 0x1c8: 0x30b2, 0x1c9: 0x33be, 0x1ca: 0x30b7, 0x1cb: 0x33c3,
+	0x1cc: 0x315c, 0x1cd: 0x346d, 0x1ce: 0x3161, 0x1cf: 0x3472, 0x1d0: 0x317f, 0x1d1: 0x3490,
+	0x1d2: 0x3184, 0x1d3: 0x3495, 0x1d4: 0x31f2, 0x1d5: 0x3508, 0x1d6: 0x31f7, 0x1d7: 0x350d,
+	0x1d8: 0x319d, 0x1d9: 0x34ae, 0x1da: 0x31b6, 0x1db: 0x34cc,
+	0x1de: 0x3071, 0x1df: 0x337d,
+	0x1e6: 0x46a9, 0x1e7: 0x473a, 0x1e8: 0x46d1, 0x1e9: 0x4762,
+	0x1ea: 0x3976, 0x1eb: 0x3b05, 0x1ec: 0x3953, 0x1ed: 0x3ae2, 0x1ee: 0x46ef, 0x1ef: 0x4780,
+	0x1f0: 0x396f, 0x1f1: 0x3afe, 0x1f2: 0x325b, 0x1f3: 0x3576,
+	// Block 0x8, offset 0x200
+	0x200: 0x9933, 0x201: 0x9933, 0x202: 0x9933, 0x203: 0x9933, 0x204: 0x9933, 0x205: 0x8133,
+	0x206: 0x9933, 0x207: 0x9933, 0x208: 0x9933, 0x209: 0x9933, 0x20a: 0x9933, 0x20b: 0x9933,
+	0x20c: 0x9933, 0x20d: 0x8133, 0x20e: 0x8133, 0x20f: 0x9933, 0x210: 0x8133, 0x211: 0x9933,
+	0x212: 0x8133, 0x213: 0x9933, 0x214: 0x9933, 0x215: 0x8134, 0x216: 0x812e, 0x217: 0x812e,
+	0x218: 0x812e, 0x219: 0x812e, 0x21a: 0x8134, 0x21b: 0x992c, 0x21c: 0x812e, 0x21d: 0x812e,
+	0x21e: 0x812e, 0x21f: 0x812e, 0x220: 0x812e, 0x221: 0x812a, 0x222: 0x812a, 0x223: 0x992e,
+	0x224: 0x992e, 0x225: 0x992e, 0x226: 0x992e, 0x227: 0x992a, 0x228: 0x992a, 0x229: 0x812e,
+	0x22a: 0x812e, 0x22b: 0x812e, 0x22c: 0x812e, 0x22d: 0x992e, 0x22e: 0x992e, 0x22f: 0x812e,
+	0x230: 0x992e, 0x231: 0x992e, 0x232: 0x812e, 0x233: 0x812e, 0x234: 0x8101, 0x235: 0x8101,
+	0x236: 0x8101, 0x237: 0x8101, 0x238: 0x9901, 0x239: 0x812e, 0x23a: 0x812e, 0x23b: 0x812e,
+	0x23c: 0x812e, 0x23d: 0x8133, 0x23e: 0x8133, 0x23f: 0x8133,
+	// Block 0x9, offset 0x240
+	0x240: 0x49c5, 0x241: 0x49ca, 0x242: 0x9933, 0x243: 0x49cf, 0x244: 0x4a88, 0x245: 0x9937,
+	0x246: 0x8133, 0x247: 0x812e, 0x248: 0x812e, 0x249: 0x812e, 0x24a: 0x8133, 0x24b: 0x8133,
+	0x24c: 0x8133, 0x24d: 0x812e, 0x24e: 0x812e, 0x250: 0x8133, 0x251: 0x8133,
+	0x252: 0x8133, 0x253: 0x812e, 0x254: 0x812e, 0x255: 0x812e, 0x256: 0x812e, 0x257: 0x8133,
+	0x258: 0x8134, 0x259: 0x812e, 0x25a: 0x812e, 0x25b: 0x8133, 0x25c: 0x8135, 0x25d: 0x8136,
+	0x25e: 0x8136, 0x25f: 0x8135, 0x260: 0x8136, 0x261: 0x8136, 0x262: 0x8135, 0x263: 0x8133,
+	0x264: 0x8133, 0x265: 0x8133, 0x266: 0x8133, 0x267: 0x8133, 0x268: 0x8133, 0x269: 0x8133,
+	0x26a: 0x8133, 0x26b: 0x8133, 0x26c: 0x8133, 0x26d: 0x8133, 0x26e: 0x8133, 0x26f: 0x8133,
+	0x274: 0x0173,
+	0x27a: 0x42bc,
+	0x27e: 0x0037,
+	// Block 0xa, offset 0x280
+	0x284: 0x4271, 0x285: 0x4492,
+	0x286: 0x3600, 0x287: 0x00ce, 0x288: 0x361e, 0x289: 0x362a, 0x28a: 0x363c,
+	0x28c: 0x365a, 0x28e: 0x366c, 0x28f: 0x368a, 0x290: 0x3e1f, 0x291: 0xa000,
+	0x295: 0xa000, 0x297: 0xa000,
+	0x299: 0xa000,
+	0x29f: 0xa000, 0x2a1: 0xa000,
+	0x2a5: 0xa000, 0x2a9: 0xa000,
+	0x2aa: 0x364e, 0x2ab: 0x367e, 0x2ac: 0x4815, 0x2ad: 0x36ae, 0x2ae: 0x483f, 0x2af: 0x36c0,
+	0x2b0: 0x3e87, 0x2b1: 0xa000, 0x2b5: 0xa000,
+	0x2b7: 0xa000, 0x2b9: 0xa000,
+	0x2bf: 0xa000,
+	// Block 0xb, offset 0x2c0
+	0x2c1: 0xa000, 0x2c5: 0xa000,
+	0x2c9: 0xa000, 0x2ca: 0x4857, 0x2cb: 0x4875,
+	0x2cc: 0x36de, 0x2cd: 0x36f6, 0x2ce: 0x488d, 0x2d0: 0x01c1, 0x2d1: 0x01d3,
+	0x2d2: 0x01af, 0x2d3: 0x4323, 0x2d4: 0x4329, 0x2d5: 0x01fd, 0x2d6: 0x01eb,
+	0x2f0: 0x01d9, 0x2f1: 0x01ee, 0x2f2: 0x01f1, 0x2f4: 0x018b, 0x2f5: 0x01ca,
+	0x2f9: 0x01a9,
+	// Block 0xc, offset 0x300
+	0x300: 0x3738, 0x301: 0x3744, 0x303: 0x3732,
+	0x306: 0xa000, 0x307: 0x3720,
+	0x30c: 0x3774, 0x30d: 0x375c, 0x30e: 0x3786, 0x310: 0xa000,
+	0x313: 0xa000, 0x315: 0xa000, 0x316: 0xa000, 0x317: 0xa000,
+	0x318: 0xa000, 0x319: 0x3768, 0x31a: 0xa000,
+	0x31e: 0xa000, 0x323: 0xa000,
+	0x327: 0xa000,
+	0x32b: 0xa000, 0x32d: 0xa000,
+	0x330: 0xa000, 0x333: 0xa000, 0x335: 0xa000,
+	0x336: 0xa000, 0x337: 0xa000, 0x338: 0xa000, 0x339: 0x37ec, 0x33a: 0xa000,
+	0x33e: 0xa000,
+	// Block 0xd, offset 0x340
+	0x341: 0x374a, 0x342: 0x37ce,
+	0x350: 0x3726, 0x351: 0x37aa,
+	0x352: 0x372c, 0x353: 0x37b0, 0x356: 0x373e, 0x357: 0x37c2,
+	0x358: 0xa000, 0x359: 0xa000, 0x35a: 0x3840, 0x35b: 0x3846, 0x35c: 0x3750, 0x35d: 0x37d4,
+	0x35e: 0x3756, 0x35f: 0x37da, 0x362: 0x3762, 0x363: 0x37e6,
+	0x364: 0x376e, 0x365: 0x37f2, 0x366: 0x377a, 0x367: 0x37fe, 0x368: 0xa000, 0x369: 0xa000,
+	0x36a: 0x384c, 0x36b: 0x3852, 0x36c: 0x37a4, 0x36d: 0x3828, 0x36e: 0x3780, 0x36f: 0x3804,
+	0x370: 0x378c, 0x371: 0x3810, 0x372: 0x3792, 0x373: 0x3816, 0x374: 0x3798, 0x375: 0x381c,
+	0x378: 0x379e, 0x379: 0x3822,
+	// Block 0xe, offset 0x380
+	0x387: 0x1d67,
+	0x391: 0x812e,
+	0x392: 0x8133, 0x393: 0x8133, 0x394: 0x8133, 0x395: 0x8133, 0x396: 0x812e, 0x397: 0x8133,
+	0x398: 0x8133, 0x399: 0x8133, 0x39a: 0x812f, 0x39b: 0x812e, 0x39c: 0x8133, 0x39d: 0x8133,
+	0x39e: 0x8133, 0x39f: 0x8133, 0x3a0: 0x8133, 0x3a1: 0x8133, 0x3a2: 0x812e, 0x3a3: 0x812e,
+	0x3a4: 0x812e, 0x3a5: 0x812e, 0x3a6: 0x812e, 0x3a7: 0x812e, 0x3a8: 0x8133, 0x3a9: 0x8133,
+	0x3aa: 0x812e, 0x3ab: 0x8133, 0x3ac: 0x8133, 0x3ad: 0x812f, 0x3ae: 0x8132, 0x3af: 0x8133,
+	0x3b0: 0x8106, 0x3b1: 0x8107, 0x3b2: 0x8108, 0x3b3: 0x8109, 0x3b4: 0x810a, 0x3b5: 0x810b,
+	0x3b6: 0x810c, 0x3b7: 0x810d, 0x3b8: 0x810e, 0x3b9: 0x810f, 0x3ba: 0x810f, 0x3bb: 0x8110,
+	0x3bc: 0x8111, 0x3bd: 0x8112, 0x3bf: 0x8113,
+	// Block 0xf, offset 0x3c0
+	0x3c8: 0xa000, 0x3ca: 0xa000, 0x3cb: 0x8117,
+	0x3cc: 0x8118, 0x3cd: 0x8119, 0x3ce: 0x811a, 0x3cf: 0x811b, 0x3d0: 0x811c, 0x3d1: 0x811d,
+	0x3d2: 0x811e, 0x3d3: 0x9933, 0x3d4: 0x9933, 0x3d5: 0x992e, 0x3d6: 0x812e, 0x3d7: 0x8133,
+	0x3d8: 0x8133, 0x3d9: 0x8133, 0x3da: 0x8133, 0x3db: 0x8133, 0x3dc: 0x812e, 0x3dd: 0x8133,
+	0x3de: 0x8133, 0x3df: 0x812e,
+	0x3f0: 0x811f, 0x3f5: 0x1d8a,
+	0x3f6: 0x2019, 0x3f7: 0x2055, 0x3f8: 0x2050,
+	// Block 0x10, offset 0x400
+	0x413: 0x812e, 0x414: 0x8133, 0x415: 0x8133, 0x416: 0x8133, 0x417: 0x8133,
+	0x418: 0x8133, 0x419: 0x8133, 0x41a: 0x8133, 0x41b: 0x8133, 0x41c: 0x8133, 0x41d: 0x8133,
+	0x41e: 0x8133, 0x41f: 0x8133, 0x420: 0x8133, 0x421: 0x8133, 0x423: 0x812e,
+	0x424: 0x8133, 0x425: 0x8133, 0x426: 0x812e, 0x427: 0x8133, 0x428: 0x8133, 0x429: 0x812e,
+	0x42a: 0x8133, 0x42b: 0x8133, 0x42c: 0x8133, 0x42d: 0x812e, 0x42e: 0x812e, 0x42f: 0x812e,
+	0x430: 0x8117, 0x431: 0x8118, 0x432: 0x8119, 0x433: 0x8133, 0x434: 0x8133, 0x435: 0x8133,
+	0x436: 0x812e, 0x437: 0x8133, 0x438: 0x8133, 0x439: 0x812e, 0x43a: 0x812e, 0x43b: 0x8133,
+	0x43c: 0x8133, 0x43d: 0x8133, 0x43e: 0x8133, 0x43f: 0x8133,
+	// Block 0x11, offset 0x440
+	0x445: 0xa000,
+	0x446: 0x2d33, 0x447: 0xa000, 0x448: 0x2d3b, 0x449: 0xa000, 0x44a: 0x2d43, 0x44b: 0xa000,
+	0x44c: 0x2d4b, 0x44d: 0xa000, 0x44e: 0x2d53, 0x451: 0xa000,
+	0x452: 0x2d5b,
+	0x474: 0x8103, 0x475: 0x9900,
+	0x47a: 0xa000, 0x47b: 0x2d63,
+	0x47c: 0xa000, 0x47d: 0x2d6b, 0x47e: 0xa000, 0x47f: 0xa000,
+	// Block 0x12, offset 0x480
+	0x480: 0x0069, 0x481: 0x006b, 0x482: 0x006f, 0x483: 0x0083, 0x484: 0x00f5, 0x485: 0x00f8,
+	0x486: 0x0416, 0x487: 0x0085, 0x488: 0x0089, 0x489: 0x008b, 0x48a: 0x0104, 0x48b: 0x0107,
+	0x48c: 0x010a, 0x48d: 0x008f, 0x48f: 0x0097, 0x490: 0x009b, 0x491: 0x00e0,
+	0x492: 0x009f, 0x493: 0x00fe, 0x494: 0x041a, 0x495: 0x041e, 0x496: 0x00a1, 0x497: 0x00a9,
+	0x498: 0x00ab, 0x499: 0x0426, 0x49a: 0x012b, 0x49b: 0x00ad, 0x49c: 0x042a, 0x49d: 0x01c1,
+	0x49e: 0x01c4, 0x49f: 0x01c7, 0x4a0: 0x01fd, 0x4a1: 0x0200, 0x4a2: 0x0093, 0x4a3: 0x00a5,
+	0x4a4: 0x00ab, 0x4a5: 0x00ad, 0x4a6: 0x01c1, 0x4a7: 0x01c4, 0x4a8: 0x01ee, 0x4a9: 0x01fd,
+	0x4aa: 0x0200,
+	0x4b8: 0x020f,
+	// Block 0x13, offset 0x4c0
+	0x4db: 0x00fb, 0x4dc: 0x0087, 0x4dd: 0x0101,
+	0x4de: 0x00d4, 0x4df: 0x010a, 0x4e0: 0x008d, 0x4e1: 0x010d, 0x4e2: 0x0110, 0x4e3: 0x0116,
+	0x4e4: 0x011c, 0x4e5: 0x011f, 0x4e6: 0x0122, 0x4e7: 0x042e, 0x4e8: 0x016d, 0x4e9: 0x0128,
+	0x4ea: 0x0432, 0x4eb: 0x0170, 0x4ec: 0x0131, 0x4ed: 0x012e, 0x4ee: 0x0134, 0x4ef: 0x0137,
+	0x4f0: 0x013a, 0x4f1: 0x013d, 0x4f2: 0x0140, 0x4f3: 0x014c, 0x4f4: 0x014f, 0x4f5: 0x00ec,
+	0x4f6: 0x0152, 0x4f7: 0x0155, 0x4f8: 0x0422, 0x4f9: 0x0158, 0x4fa: 0x015b, 0x4fb: 0x00b5,
+	0x4fc: 0x0161, 0x4fd: 0x0164, 0x4fe: 0x0167, 0x4ff: 0x01d3,
+	// Block 0x14, offset 0x500
+	0x500: 0x8133, 0x501: 0x8133, 0x502: 0x812e, 0x503: 0x8133, 0x504: 0x8133, 0x505: 0x8133,
+	0x506: 0x8133, 0x507: 0x8133, 0x508: 0x8133, 0x509: 0x8133, 0x50a: 0x812e, 0x50b: 0x8133,
+	0x50c: 0x8133, 0x50d: 0x8136, 0x50e: 0x812b, 0x50f: 0x812e, 0x510: 0x812a, 0x511: 0x8133,
+	0x512: 0x8133, 0x513: 0x8133, 0x514: 0x8133, 0x515: 0x8133, 0x516: 0x8133, 0x517: 0x8133,
+	0x518: 0x8133, 0x519: 0x8133, 0x51a: 0x8133, 0x51b: 0x8133, 0x51c: 0x8133, 0x51d: 0x8133,
+	0x51e: 0x8133, 0x51f: 0x8133, 0x520: 0x8133, 0x521: 0x8133, 0x522: 0x8133, 0x523: 0x8133,
+	0x524: 0x8133, 0x525: 0x8133, 0x526: 0x8133, 0x527: 0x8133, 0x528: 0x8133, 0x529: 0x8133,
+	0x52a: 0x8133, 0x52b: 0x8133, 0x52c: 0x8133, 0x52d: 0x8133, 0x52e: 0x8133, 0x52f: 0x8133,
+	0x530: 0x8133, 0x531: 0x8133, 0x532: 0x8133, 0x533: 0x8133, 0x534: 0x8133, 0x535: 0x8133,
+	0x536: 0x8134, 0x537: 0x8132, 0x538: 0x8132, 0x539: 0x812e, 0x53b: 0x8133,
+	0x53c: 0x8135, 0x53d: 0x812e, 0x53e: 0x8133, 0x53f: 0x812e,
+	// Block 0x15, offset 0x540
+	0x540: 0x2fae, 0x541: 0x32ba, 0x542: 0x2fb8, 0x543: 0x32c4, 0x544: 0x2fbd, 0x545: 0x32c9,
+	0x546: 0x2fc2, 0x547: 0x32ce, 0x548: 0x38e3, 0x549: 0x3a72, 0x54a: 0x2fdb, 0x54b: 0x32e7,
+	0x54c: 0x2fe5, 0x54d: 0x32f1, 0x54e: 0x2ff4, 0x54f: 0x3300, 0x550: 0x2fea, 0x551: 0x32f6,
+	0x552: 0x2fef, 0x553: 0x32fb, 0x554: 0x3906, 0x555: 0x3a95, 0x556: 0x390d, 0x557: 0x3a9c,
+	0x558: 0x3030, 0x559: 0x333c, 0x55a: 0x3035, 0x55b: 0x3341, 0x55c: 0x391b, 0x55d: 0x3aaa,
+	0x55e: 0x303a, 0x55f: 0x3346, 0x560: 0x3049, 0x561: 0x3355, 0x562: 0x3067, 0x563: 0x3373,
+	0x564: 0x3076, 0x565: 0x3382, 0x566: 0x306c, 0x567: 0x3378, 0x568: 0x307b, 0x569: 0x3387,
+	0x56a: 0x3080, 0x56b: 0x338c, 0x56c: 0x30c6, 0x56d: 0x33d2, 0x56e: 0x3922, 0x56f: 0x3ab1,
+	0x570: 0x30d0, 0x571: 0x33e1, 0x572: 0x30da, 0x573: 0x33eb, 0x574: 0x30e4, 0x575: 0x33f5,
+	0x576: 0x46db, 0x577: 0x476c, 0x578: 0x3929, 0x579: 0x3ab8, 0x57a: 0x30fd, 0x57b: 0x340e,
+	0x57c: 0x30f8, 0x57d: 0x3409, 0x57e: 0x3102, 0x57f: 0x3413,
+	// Block 0x16, offset 0x580
+	0x580: 0x3107, 0x581: 0x3418, 0x582: 0x310c, 0x583: 0x341d, 0x584: 0x3120, 0x585: 0x3431,
+	0x586: 0x312a, 0x587: 0x343b, 0x588: 0x3139, 0x589: 0x344a, 0x58a: 0x3134, 0x58b: 0x3445,
+	0x58c: 0x394c, 0x58d: 0x3adb, 0x58e: 0x395a, 0x58f: 0x3ae9, 0x590: 0x3961, 0x591: 0x3af0,
+	0x592: 0x3968, 0x593: 0x3af7, 0x594: 0x3166, 0x595: 0x3477, 0x596: 0x316b, 0x597: 0x347c,
+	0x598: 0x3175, 0x599: 0x3486, 0x59a: 0x4708, 0x59b: 0x4799, 0x59c: 0x39ae, 0x59d: 0x3b3d,
+	0x59e: 0x318e, 0x59f: 0x349f, 0x5a0: 0x3198, 0x5a1: 0x34a9, 0x5a2: 0x4717, 0x5a3: 0x47a8,
+	0x5a4: 0x39b5, 0x5a5: 0x3b44, 0x5a6: 0x39bc, 0x5a7: 0x3b4b, 0x5a8: 0x39c3, 0x5a9: 0x3b52,
+	0x5aa: 0x31a7, 0x5ab: 0x34b8, 0x5ac: 0x31b1, 0x5ad: 0x34c7, 0x5ae: 0x31c5, 0x5af: 0x34db,
+	0x5b0: 0x31c0, 0x5b1: 0x34d6, 0x5b2: 0x3201, 0x5b3: 0x3517, 0x5b4: 0x3210, 0x5b5: 0x3526,
+	0x5b6: 0x320b, 0x5b7: 0x3521, 0x5b8: 0x39ca, 0x5b9: 0x3b59, 0x5ba: 0x39d1, 0x5bb: 0x3b60,
+	0x5bc: 0x3215, 0x5bd: 0x352b, 0x5be: 0x321a, 0x5bf: 0x3530,
+	// Block 0x17, offset 0x5c0
+	0x5c0: 0x321f, 0x5c1: 0x3535, 0x5c2: 0x3224, 0x5c3: 0x353a, 0x5c4: 0x3233, 0x5c5: 0x3549,
+	0x5c6: 0x322e, 0x5c7: 0x3544, 0x5c8: 0x3238, 0x5c9: 0x3553, 0x5ca: 0x323d, 0x5cb: 0x3558,
+	0x5cc: 0x3242, 0x5cd: 0x355d, 0x5ce: 0x3260, 0x5cf: 0x357b, 0x5d0: 0x3279, 0x5d1: 0x3599,
+	0x5d2: 0x3288, 0x5d3: 0x35a8, 0x5d4: 0x328d, 0x5d5: 0x35ad, 0x5d6: 0x3391, 0x5d7: 0x34bd,
+	0x5d8: 0x354e, 0x5d9: 0x358a, 0x5da: 0x1be6, 0x5db: 0x42ee,
+	0x5e0: 0x46b8, 0x5e1: 0x4749, 0x5e2: 0x2f9a, 0x5e3: 0x32a6,
+	0x5e4: 0x388f, 0x5e5: 0x3a1e, 0x5e6: 0x3888, 0x5e7: 0x3a17, 0x5e8: 0x389d, 0x5e9: 0x3a2c,
+	0x5ea: 0x3896, 0x5eb: 0x3a25, 0x5ec: 0x38d5, 0x5ed: 0x3a64, 0x5ee: 0x38ab, 0x5ef: 0x3a3a,
+	0x5f0: 0x38a4, 0x5f1: 0x3a33, 0x5f2: 0x38b9, 0x5f3: 0x3a48, 0x5f4: 0x38b2, 0x5f5: 0x3a41,
+	0x5f6: 0x38dc, 0x5f7: 0x3a6b, 0x5f8: 0x46cc, 0x5f9: 0x475d, 0x5fa: 0x3017, 0x5fb: 0x3323,
+	0x5fc: 0x3003, 0x5fd: 0x330f, 0x5fe: 0x38f1, 0x5ff: 0x3a80,
+	// Block 0x18, offset 0x600
+	0x600: 0x38ea, 0x601: 0x3a79, 0x602: 0x38ff, 0x603: 0x3a8e, 0x604: 0x38f8, 0x605: 0x3a87,
+	0x606: 0x3914, 0x607: 0x3aa3, 0x608: 0x30a8, 0x609: 0x33b4, 0x60a: 0x30bc, 0x60b: 0x33c8,
+	0x60c: 0x46fe, 0x60d: 0x478f, 0x60e: 0x314d, 0x60f: 0x345e, 0x610: 0x3937, 0x611: 0x3ac6,
+	0x612: 0x3930, 0x613: 0x3abf, 0x614: 0x3945, 0x615: 0x3ad4, 0x616: 0x393e, 0x617: 0x3acd,
+	0x618: 0x39a0, 0x619: 0x3b2f, 0x61a: 0x3984, 0x61b: 0x3b13, 0x61c: 0x397d, 0x61d: 0x3b0c,
+	0x61e: 0x3992, 0x61f: 0x3b21, 0x620: 0x398b, 0x621: 0x3b1a, 0x622: 0x3999, 0x623: 0x3b28,
+	0x624: 0x31fc, 0x625: 0x3512, 0x626: 0x31de, 0x627: 0x34f4, 0x628: 0x39fb, 0x629: 0x3b8a,
+	0x62a: 0x39f4, 0x62b: 0x3b83, 0x62c: 0x3a09, 0x62d: 0x3b98, 0x62e: 0x3a02, 0x62f: 0x3b91,
+	0x630: 0x3a10, 0x631: 0x3b9f, 0x632: 0x3247, 0x633: 0x3562, 0x634: 0x326f, 0x635: 0x358f,
+	0x636: 0x326a, 0x637: 0x3585, 0x638: 0x3256, 0x639: 0x3571,
+	// Block 0x19, offset 0x640
+	0x640: 0x481b, 0x641: 0x4821, 0x642: 0x4935, 0x643: 0x494d, 0x644: 0x493d, 0x645: 0x4955,
+	0x646: 0x4945, 0x647: 0x495d, 0x648: 0x47c1, 0x649: 0x47c7, 0x64a: 0x48a5, 0x64b: 0x48bd,
+	0x64c: 0x48ad, 0x64d: 0x48c5, 0x64e: 0x48b5, 0x64f: 0x48cd, 0x650: 0x482d, 0x651: 0x4833,
+	0x652: 0x3dcf, 0x653: 0x3ddf, 0x654: 0x3dd7, 0x655: 0x3de7,
+	0x658: 0x47cd, 0x659: 0x47d3, 0x65a: 0x3cff, 0x65b: 0x3d0f, 0x65c: 0x3d07, 0x65d: 0x3d17,
+	0x660: 0x4845, 0x661: 0x484b, 0x662: 0x4965, 0x663: 0x497d,
+	0x664: 0x496d, 0x665: 0x4985, 0x666: 0x4975, 0x667: 0x498d, 0x668: 0x47d9, 0x669: 0x47df,
+	0x66a: 0x48d5, 0x66b: 0x48ed, 0x66c: 0x48dd, 0x66d: 0x48f5, 0x66e: 0x48e5, 0x66f: 0x48fd,
+	0x670: 0x485d, 0x671: 0x4863, 0x672: 0x3e2f, 0x673: 0x3e47, 0x674: 0x3e37, 0x675: 0x3e4f,
+	0x676: 0x3e3f, 0x677: 0x3e57, 0x678: 0x47e5, 0x679: 0x47eb, 0x67a: 0x3d2f, 0x67b: 0x3d47,
+	0x67c: 0x3d37, 0x67d: 0x3d4f, 0x67e: 0x3d3f, 0x67f: 0x3d57,
+	// Block 0x1a, offset 0x680
+	0x680: 0x4869, 0x681: 0x486f, 0x682: 0x3e5f, 0x683: 0x3e6f, 0x684: 0x3e67, 0x685: 0x3e77,
+	0x688: 0x47f1, 0x689: 0x47f7, 0x68a: 0x3d5f, 0x68b: 0x3d6f,
+	0x68c: 0x3d67, 0x68d: 0x3d77, 0x690: 0x487b, 0x691: 0x4881,
+	0x692: 0x3e97, 0x693: 0x3eaf, 0x694: 0x3e9f, 0x695: 0x3eb7, 0x696: 0x3ea7, 0x697: 0x3ebf,
+	0x699: 0x47fd, 0x69b: 0x3d7f, 0x69d: 0x3d87,
+	0x69f: 0x3d8f, 0x6a0: 0x4893, 0x6a1: 0x4899, 0x6a2: 0x4995, 0x6a3: 0x49ad,
+	0x6a4: 0x499d, 0x6a5: 0x49b5, 0x6a6: 0x49a5, 0x6a7: 0x49bd, 0x6a8: 0x4803, 0x6a9: 0x4809,
+	0x6aa: 0x4905, 0x6ab: 0x491d, 0x6ac: 0x490d, 0x6ad: 0x4925, 0x6ae: 0x4915, 0x6af: 0x492d,
+	0x6b0: 0x480f, 0x6b1: 0x4335, 0x6b2: 0x36a8, 0x6b3: 0x433b, 0x6b4: 0x4839, 0x6b5: 0x4341,
+	0x6b6: 0x36ba, 0x6b7: 0x4347, 0x6b8: 0x36d8, 0x6b9: 0x434d, 0x6ba: 0x36f0, 0x6bb: 0x4353,
+	0x6bc: 0x4887, 0x6bd: 0x4359,
+	// Block 0x1b, offset 0x6c0
+	0x6c0: 0x3db7, 0x6c1: 0x3dbf, 0x6c2: 0x419b, 0x6c3: 0x41b9, 0x6c4: 0x41a5, 0x6c5: 0x41c3,
+	0x6c6: 0x41af, 0x6c7: 0x41cd, 0x6c8: 0x3cef, 0x6c9: 0x3cf7, 0x6ca: 0x40e7, 0x6cb: 0x4105,
+	0x6cc: 0x40f1, 0x6cd: 0x410f, 0x6ce: 0x40fb, 0x6cf: 0x4119, 0x6d0: 0x3dff, 0x6d1: 0x3e07,
+	0x6d2: 0x41d7, 0x6d3: 0x41f5, 0x6d4: 0x41e1, 0x6d5: 0x41ff, 0x6d6: 0x41eb, 0x6d7: 0x4209,
+	0x6d8: 0x3d1f, 0x6d9: 0x3d27, 0x6da: 0x4123, 0x6db: 0x4141, 0x6dc: 0x412d, 0x6dd: 0x414b,
+	0x6de: 0x4137, 0x6df: 0x4155, 0x6e0: 0x3ed7, 0x6e1: 0x3edf, 0x6e2: 0x4213, 0x6e3: 0x4231,
+	0x6e4: 0x421d, 0x6e5: 0x423b, 0x6e6: 0x4227, 0x6e7: 0x4245, 0x6e8: 0x3d97, 0x6e9: 0x3d9f,
+	0x6ea: 0x415f, 0x6eb: 0x417d, 0x6ec: 0x4169, 0x6ed: 0x4187, 0x6ee: 0x4173, 0x6ef: 0x4191,
+	0x6f0: 0x369c, 0x6f1: 0x3696, 0x6f2: 0x3da7, 0x6f3: 0x36a2, 0x6f4: 0x3daf,
+	0x6f6: 0x4827, 0x6f7: 0x3dc7, 0x6f8: 0x360c, 0x6f9: 0x3606, 0x6fa: 0x35fa, 0x6fb: 0x4305,
+	0x6fc: 0x3612, 0x6fd: 0x429e, 0x6fe: 0x01d6, 0x6ff: 0x429e,
+	// Block 0x1c, offset 0x700
+	0x700: 0x42b7, 0x701: 0x4499, 0x702: 0x3def, 0x703: 0x36b4, 0x704: 0x3df7,
+	0x706: 0x4851, 0x707: 0x3e0f, 0x708: 0x3618, 0x709: 0x430b, 0x70a: 0x3624, 0x70b: 0x4311,
+	0x70c: 0x3630, 0x70d: 0x44a0, 0x70e: 0x44a7, 0x70f: 0x44ae, 0x710: 0x36cc, 0x711: 0x36c6,
+	0x712: 0x3e17, 0x713: 0x44fb, 0x716: 0x36d2, 0x717: 0x3e27,
+	0x718: 0x3648, 0x719: 0x3642, 0x71a: 0x3636, 0x71b: 0x4317, 0x71d: 0x44b5,
+	0x71e: 0x44bc, 0x71f: 0x44c3, 0x720: 0x3702, 0x721: 0x36fc, 0x722: 0x3e7f, 0x723: 0x4503,
+	0x724: 0x36e4, 0x725: 0x36ea, 0x726: 0x3708, 0x727: 0x3e8f, 0x728: 0x3678, 0x729: 0x3672,
+	0x72a: 0x3666, 0x72b: 0x4323, 0x72c: 0x3660, 0x72d: 0x448b, 0x72e: 0x4492, 0x72f: 0x0081,
+	0x732: 0x3ec7, 0x733: 0x370e, 0x734: 0x3ecf,
+	0x736: 0x489f, 0x737: 0x3ee7, 0x738: 0x3654, 0x739: 0x431d, 0x73a: 0x3684, 0x73b: 0x432f,
+	0x73c: 0x3690, 0x73d: 0x4271, 0x73e: 0x42a3,
+	// Block 0x1d, offset 0x740
+	0x740: 0x1bde, 0x741: 0x1be2, 0x742: 0x0047, 0x743: 0x1c5a, 0x745: 0x1bee,
+	0x746: 0x1bf2, 0x747: 0x00e9, 0x749: 0x1c5e, 0x74a: 0x008f, 0x74b: 0x0051,
+	0x74c: 0x0051, 0x74d: 0x0051, 0x74e: 0x0091, 0x74f: 0x00da, 0x750: 0x0053, 0x751: 0x0053,
+	0x752: 0x0059, 0x753: 0x0099, 0x755: 0x005d, 0x756: 0x1993,
+	0x759: 0x0061, 0x75a: 0x0063, 0x75b: 0x0065, 0x75c: 0x0065, 0x75d: 0x0065,
+	0x760: 0x19a5, 0x761: 0x1bce, 0x762: 0x19ae,
+	0x764: 0x0075, 0x766: 0x01bb, 0x768: 0x0075,
+	0x76a: 0x0057, 0x76b: 0x42e9, 0x76c: 0x0045, 0x76d: 0x0047, 0x76f: 0x008b,
+	0x770: 0x004b, 0x771: 0x004d, 0x773: 0x005b, 0x774: 0x009f, 0x775: 0x0218,
+	0x776: 0x021b, 0x777: 0x021e, 0x778: 0x0221, 0x779: 0x0093, 0x77b: 0x1b9e,
+	0x77c: 0x01eb, 0x77d: 0x01c4, 0x77e: 0x017c, 0x77f: 0x01a3,
+	// Block 0x1e, offset 0x780
+	0x780: 0x0466, 0x785: 0x0049,
+	0x786: 0x0089, 0x787: 0x008b, 0x788: 0x0093, 0x789: 0x0095,
+	0x790: 0x2234, 0x791: 0x2240,
+	0x792: 0x22f4, 0x793: 0x221c, 0x794: 0x22a0, 0x795: 0x2228, 0x796: 0x22a6, 0x797: 0x22be,
+	0x798: 0x22ca, 0x799: 0x222e, 0x79a: 0x22d0, 0x79b: 0x223a, 0x79c: 0x22c4, 0x79d: 0x22d6,
+	0x79e: 0x22dc, 0x79f: 0x1cc2, 0x7a0: 0x0053, 0x7a1: 0x195d, 0x7a2: 0x1baa, 0x7a3: 0x1966,
+	0x7a4: 0x006d, 0x7a5: 0x19b1, 0x7a6: 0x1bd6, 0x7a7: 0x1d4e, 0x7a8: 0x1969, 0x7a9: 0x0071,
+	0x7aa: 0x19bd, 0x7ab: 0x1bda, 0x7ac: 0x0059, 0x7ad: 0x0047, 0x7ae: 0x0049, 0x7af: 0x005b,
+	0x7b0: 0x0093, 0x7b1: 0x19ea, 0x7b2: 0x1c1e, 0x7b3: 0x19f3, 0x7b4: 0x00ad, 0x7b5: 0x1a68,
+	0x7b6: 0x1c52, 0x7b7: 0x1d62, 0x7b8: 0x19f6, 0x7b9: 0x00b1, 0x7ba: 0x1a6b, 0x7bb: 0x1c56,
+	0x7bc: 0x0099, 0x7bd: 0x0087, 0x7be: 0x0089, 0x7bf: 0x009b,
+	// Block 0x1f, offset 0x7c0
+	0x7c1: 0x3c1d, 0x7c3: 0xa000, 0x7c4: 0x3c24, 0x7c5: 0xa000,
+	0x7c7: 0x3c2b, 0x7c8: 0xa000, 0x7c9: 0x3c32,
+	0x7cd: 0xa000,
+	0x7e0: 0x2f7c, 0x7e1: 0xa000, 0x7e2: 0x3c40,
+	0x7e4: 0xa000, 0x7e5: 0xa000,
+	0x7ed: 0x3c39, 0x7ee: 0x2f77, 0x7ef: 0x2f81,
+	0x7f0: 0x3c47, 0x7f1: 0x3c4e, 0x7f2: 0xa000, 0x7f3: 0xa000, 0x7f4: 0x3c55, 0x7f5: 0x3c5c,
+	0x7f6: 0xa000, 0x7f7: 0xa000, 0x7f8: 0x3c63, 0x7f9: 0x3c6a, 0x7fa: 0xa000, 0x7fb: 0xa000,
+	0x7fc: 0xa000, 0x7fd: 0xa000,
+	// Block 0x20, offset 0x800
+	0x800: 0x3c71, 0x801: 0x3c78, 0x802: 0xa000, 0x803: 0xa000, 0x804: 0x3c8d, 0x805: 0x3c94,
+	0x806: 0xa000, 0x807: 0xa000, 0x808: 0x3c9b, 0x809: 0x3ca2,
+	0x811: 0xa000,
+	0x812: 0xa000,
+	0x822: 0xa000,
+	0x828: 0xa000, 0x829: 0xa000,
+	0x82b: 0xa000, 0x82c: 0x3cb7, 0x82d: 0x3cbe, 0x82e: 0x3cc5, 0x82f: 0x3ccc,
+	0x832: 0xa000, 0x833: 0xa000, 0x834: 0xa000, 0x835: 0xa000,
+	// Block 0x21, offset 0x840
+	0x860: 0x0023, 0x861: 0x0025, 0x862: 0x0027, 0x863: 0x0029,
+	0x864: 0x002b, 0x865: 0x002d, 0x866: 0x002f, 0x867: 0x0031, 0x868: 0x0033, 0x869: 0x1885,
+	0x86a: 0x1888, 0x86b: 0x188b, 0x86c: 0x188e, 0x86d: 0x1891, 0x86e: 0x1894, 0x86f: 0x1897,
+	0x870: 0x189a, 0x871: 0x189d, 0x872: 0x18a0, 0x873: 0x18a9, 0x874: 0x1a6e, 0x875: 0x1a72,
+	0x876: 0x1a76, 0x877: 0x1a7a, 0x878: 0x1a7e, 0x879: 0x1a82, 0x87a: 0x1a86, 0x87b: 0x1a8a,
+	0x87c: 0x1a8e, 0x87d: 0x1c86, 0x87e: 0x1c8b, 0x87f: 0x1c90,
+	// Block 0x22, offset 0x880
+	0x880: 0x1c95, 0x881: 0x1c9a, 0x882: 0x1c9f, 0x883: 0x1ca4, 0x884: 0x1ca9, 0x885: 0x1cae,
+	0x886: 0x1cb3, 0x887: 0x1cb8, 0x888: 0x1882, 0x889: 0x18a6, 0x88a: 0x18ca, 0x88b: 0x18ee,
+	0x88c: 0x1912, 0x88d: 0x191b, 0x88e: 0x1921, 0x88f: 0x1927, 0x890: 0x192d, 0x891: 0x1b66,
+	0x892: 0x1b6a, 0x893: 0x1b6e, 0x894: 0x1b72, 0x895: 0x1b76, 0x896: 0x1b7a, 0x897: 0x1b7e,
+	0x898: 0x1b82, 0x899: 0x1b86, 0x89a: 0x1b8a, 0x89b: 0x1b8e, 0x89c: 0x1afa, 0x89d: 0x1afe,
+	0x89e: 0x1b02, 0x89f: 0x1b06, 0x8a0: 0x1b0a, 0x8a1: 0x1b0e, 0x8a2: 0x1b12, 0x8a3: 0x1b16,
+	0x8a4: 0x1b1a, 0x8a5: 0x1b1e, 0x8a6: 0x1b22, 0x8a7: 0x1b26, 0x8a8: 0x1b2a, 0x8a9: 0x1b2e,
+	0x8aa: 0x1b32, 0x8ab: 0x1b36, 0x8ac: 0x1b3a, 0x8ad: 0x1b3e, 0x8ae: 0x1b42, 0x8af: 0x1b46,
+	0x8b0: 0x1b4a, 0x8b1: 0x1b4e, 0x8b2: 0x1b52, 0x8b3: 0x1b56, 0x8b4: 0x1b5a, 0x8b5: 0x1b5e,
+	0x8b6: 0x0043, 0x8b7: 0x0045, 0x8b8: 0x0047, 0x8b9: 0x0049, 0x8ba: 0x004b, 0x8bb: 0x004d,
+	0x8bc: 0x004f, 0x8bd: 0x0051, 0x8be: 0x0053, 0x8bf: 0x0055,
+	// Block 0x23, offset 0x8c0
+	0x8c0: 0x06c2, 0x8c1: 0x06e6, 0x8c2: 0x06f2, 0x8c3: 0x0702, 0x8c4: 0x070a, 0x8c5: 0x0716,
+	0x8c6: 0x071e, 0x8c7: 0x0726, 0x8c8: 0x0732, 0x8c9: 0x0786, 0x8ca: 0x079e, 0x8cb: 0x07ae,
+	0x8cc: 0x07be, 0x8cd: 0x07ce, 0x8ce: 0x07de, 0x8cf: 0x07fe, 0x8d0: 0x0802, 0x8d1: 0x0806,
+	0x8d2: 0x083a, 0x8d3: 0x0862, 0x8d4: 0x0872, 0x8d5: 0x087a, 0x8d6: 0x087e, 0x8d7: 0x088a,
+	0x8d8: 0x08a6, 0x8d9: 0x08aa, 0x8da: 0x08c2, 0x8db: 0x08c6, 0x8dc: 0x08ce, 0x8dd: 0x08de,
+	0x8de: 0x097a, 0x8df: 0x098e, 0x8e0: 0x09ce, 0x8e1: 0x09e2, 0x8e2: 0x09ea, 0x8e3: 0x09ee,
+	0x8e4: 0x09fe, 0x8e5: 0x0a1a, 0x8e6: 0x0a46, 0x8e7: 0x0a52, 0x8e8: 0x0a72, 0x8e9: 0x0a7e,
+	0x8ea: 0x0a82, 0x8eb: 0x0a86, 0x8ec: 0x0a9e, 0x8ed: 0x0aa2, 0x8ee: 0x0ace, 0x8ef: 0x0ada,
+	0x8f0: 0x0ae2, 0x8f1: 0x0aea, 0x8f2: 0x0afa, 0x8f3: 0x0b02, 0x8f4: 0x0b0a, 0x8f5: 0x0b36,
+	0x8f6: 0x0b3a, 0x8f7: 0x0b42, 0x8f8: 0x0b46, 0x8f9: 0x0b4e, 0x8fa: 0x0b56, 0x8fb: 0x0b66,
+	0x8fc: 0x0b82, 0x8fd: 0x0bfa, 0x8fe: 0x0c0e, 0x8ff: 0x0c12,
+	// Block 0x24, offset 0x900
+	0x900: 0x0c92, 0x901: 0x0c96, 0x902: 0x0caa, 0x903: 0x0cae, 0x904: 0x0cb6, 0x905: 0x0cbe,
+	0x906: 0x0cc6, 0x907: 0x0cd2, 0x908: 0x0cfa, 0x909: 0x0d0a, 0x90a: 0x0d1e, 0x90b: 0x0d8e,
+	0x90c: 0x0d9a, 0x90d: 0x0daa, 0x90e: 0x0db6, 0x90f: 0x0dc2, 0x910: 0x0dca, 0x911: 0x0dce,
+	0x912: 0x0dd2, 0x913: 0x0dd6, 0x914: 0x0dda, 0x915: 0x0e92, 0x916: 0x0eda, 0x917: 0x0ee6,
+	0x918: 0x0eea, 0x919: 0x0eee, 0x91a: 0x0ef2, 0x91b: 0x0efa, 0x91c: 0x0efe, 0x91d: 0x0f12,
+	0x91e: 0x0f2e, 0x91f: 0x0f36, 0x920: 0x0f76, 0x921: 0x0f7a, 0x922: 0x0f82, 0x923: 0x0f86,
+	0x924: 0x0f8e, 0x925: 0x0f92, 0x926: 0x0fb6, 0x927: 0x0fba, 0x928: 0x0fd6, 0x929: 0x0fda,
+	0x92a: 0x0fde, 0x92b: 0x0fe2, 0x92c: 0x0ff6, 0x92d: 0x101a, 0x92e: 0x101e, 0x92f: 0x1022,
+	0x930: 0x1046, 0x931: 0x1086, 0x932: 0x108a, 0x933: 0x10aa, 0x934: 0x10ba, 0x935: 0x10c2,
+	0x936: 0x10e2, 0x937: 0x1106, 0x938: 0x114a, 0x939: 0x1152, 0x93a: 0x1166, 0x93b: 0x1172,
+	0x93c: 0x117a, 0x93d: 0x1182, 0x93e: 0x1186, 0x93f: 0x118a,
+	// Block 0x25, offset 0x940
+	0x940: 0x11a2, 0x941: 0x11a6, 0x942: 0x11c2, 0x943: 0x11ca, 0x944: 0x11d2, 0x945: 0x11d6,
+	0x946: 0x11e2, 0x947: 0x11ea, 0x948: 0x11ee, 0x949: 0x11f2, 0x94a: 0x11fa, 0x94b: 0x11fe,
+	0x94c: 0x129e, 0x94d: 0x12b2, 0x94e: 0x12e6, 0x94f: 0x12ea, 0x950: 0x12f2, 0x951: 0x131e,
+	0x952: 0x1326, 0x953: 0x132e, 0x954: 0x1336, 0x955: 0x1372, 0x956: 0x1376, 0x957: 0x137e,
+	0x958: 0x1382, 0x959: 0x1386, 0x95a: 0x13b2, 0x95b: 0x13b6, 0x95c: 0x13be, 0x95d: 0x13d2,
+	0x95e: 0x13d6, 0x95f: 0x13f2, 0x960: 0x13fa, 0x961: 0x13fe, 0x962: 0x1422, 0x963: 0x1442,
+	0x964: 0x1456, 0x965: 0x145a, 0x966: 0x1462, 0x967: 0x148e, 0x968: 0x1492, 0x969: 0x14a2,
+	0x96a: 0x14c6, 0x96b: 0x14d2, 0x96c: 0x14e2, 0x96d: 0x14fa, 0x96e: 0x1502, 0x96f: 0x1506,
+	0x970: 0x150a, 0x971: 0x150e, 0x972: 0x151a, 0x973: 0x151e, 0x974: 0x1526, 0x975: 0x1542,
+	0x976: 0x1546, 0x977: 0x154a, 0x978: 0x1562, 0x979: 0x1566, 0x97a: 0x156e, 0x97b: 0x1582,
+	0x97c: 0x1586, 0x97d: 0x158a, 0x97e: 0x1592, 0x97f: 0x1596,
+	// Block 0x26, offset 0x980
+	0x986: 0xa000, 0x98b: 0xa000,
+	0x98c: 0x3f1f, 0x98d: 0xa000, 0x98e: 0x3f27, 0x98f: 0xa000, 0x990: 0x3f2f, 0x991: 0xa000,
+	0x992: 0x3f37, 0x993: 0xa000, 0x994: 0x3f3f, 0x995: 0xa000, 0x996: 0x3f47, 0x997: 0xa000,
+	0x998: 0x3f4f, 0x999: 0xa000, 0x99a: 0x3f57, 0x99b: 0xa000, 0x99c: 0x3f5f, 0x99d: 0xa000,
+	0x99e: 0x3f67, 0x99f: 0xa000, 0x9a0: 0x3f6f, 0x9a1: 0xa000, 0x9a2: 0x3f77,
+	0x9a4: 0xa000, 0x9a5: 0x3f7f, 0x9a6: 0xa000, 0x9a7: 0x3f87, 0x9a8: 0xa000, 0x9a9: 0x3f8f,
+	0x9af: 0xa000,
+	0x9b0: 0x3f97, 0x9b1: 0x3f9f, 0x9b2: 0xa000, 0x9b3: 0x3fa7, 0x9b4: 0x3faf, 0x9b5: 0xa000,
+	0x9b6: 0x3fb7, 0x9b7: 0x3fbf, 0x9b8: 0xa000, 0x9b9: 0x3fc7, 0x9ba: 0x3fcf, 0x9bb: 0xa000,
+	0x9bc: 0x3fd7, 0x9bd: 0x3fdf,
+	// Block 0x27, offset 0x9c0
+	0x9d4: 0x3f17,
+	0x9d9: 0x9904, 0x9da: 0x9904, 0x9db: 0x42f3, 0x9dc: 0x42f9, 0x9dd: 0xa000,
+	0x9de: 0x3fe7, 0x9df: 0x26ba,
+	0x9e6: 0xa000,
+	0x9eb: 0xa000, 0x9ec: 0x3ff7, 0x9ed: 0xa000, 0x9ee: 0x3fff, 0x9ef: 0xa000,
+	0x9f0: 0x4007, 0x9f1: 0xa000, 0x9f2: 0x400f, 0x9f3: 0xa000, 0x9f4: 0x4017, 0x9f5: 0xa000,
+	0x9f6: 0x401f, 0x9f7: 0xa000, 0x9f8: 0x4027, 0x9f9: 0xa000, 0x9fa: 0x402f, 0x9fb: 0xa000,
+	0x9fc: 0x4037, 0x9fd: 0xa000, 0x9fe: 0x403f, 0x9ff: 0xa000,
+	// Block 0x28, offset 0xa00
+	0xa00: 0x4047, 0xa01: 0xa000, 0xa02: 0x404f, 0xa04: 0xa000, 0xa05: 0x4057,
+	0xa06: 0xa000, 0xa07: 0x405f, 0xa08: 0xa000, 0xa09: 0x4067,
+	0xa0f: 0xa000, 0xa10: 0x406f, 0xa11: 0x4077,
+	0xa12: 0xa000, 0xa13: 0x407f, 0xa14: 0x4087, 0xa15: 0xa000, 0xa16: 0x408f, 0xa17: 0x4097,
+	0xa18: 0xa000, 0xa19: 0x409f, 0xa1a: 0x40a7, 0xa1b: 0xa000, 0xa1c: 0x40af, 0xa1d: 0x40b7,
+	0xa2f: 0xa000,
+	0xa30: 0xa000, 0xa31: 0xa000, 0xa32: 0xa000, 0xa34: 0x3fef,
+	0xa37: 0x40bf, 0xa38: 0x40c7, 0xa39: 0x40cf, 0xa3a: 0x40d7,
+	0xa3d: 0xa000, 0xa3e: 0x40df, 0xa3f: 0x26cf,
+	// Block 0x29, offset 0xa40
+	0xa40: 0x036a, 0xa41: 0x032e, 0xa42: 0x0332, 0xa43: 0x0336, 0xa44: 0x037e, 0xa45: 0x033a,
+	0xa46: 0x033e, 0xa47: 0x0342, 0xa48: 0x0346, 0xa49: 0x034a, 0xa4a: 0x034e, 0xa4b: 0x0352,
+	0xa4c: 0x0356, 0xa4d: 0x035a, 0xa4e: 0x035e, 0xa4f: 0x49d4, 0xa50: 0x49da, 0xa51: 0x49e0,
+	0xa52: 0x49e6, 0xa53: 0x49ec, 0xa54: 0x49f2, 0xa55: 0x49f8, 0xa56: 0x49fe, 0xa57: 0x4a04,
+	0xa58: 0x4a0a, 0xa59: 0x4a10, 0xa5a: 0x4a16, 0xa5b: 0x4a1c, 0xa5c: 0x4a22, 0xa5d: 0x4a28,
+	0xa5e: 0x4a2e, 0xa5f: 0x4a34, 0xa60: 0x4a3a, 0xa61: 0x4a40, 0xa62: 0x4a46, 0xa63: 0x4a4c,
+	0xa64: 0x03c6, 0xa65: 0x0362, 0xa66: 0x0366, 0xa67: 0x03ea, 0xa68: 0x03ee, 0xa69: 0x03f2,
+	0xa6a: 0x03f6, 0xa6b: 0x03fa, 0xa6c: 0x03fe, 0xa6d: 0x0402, 0xa6e: 0x036e, 0xa6f: 0x0406,
+	0xa70: 0x040a, 0xa71: 0x0372, 0xa72: 0x0376, 0xa73: 0x037a, 0xa74: 0x0382, 0xa75: 0x0386,
+	0xa76: 0x038a, 0xa77: 0x038e, 0xa78: 0x0392, 0xa79: 0x0396, 0xa7a: 0x039a, 0xa7b: 0x039e,
+	0xa7c: 0x03a2, 0xa7d: 0x03a6, 0xa7e: 0x03aa, 0xa7f: 0x03ae,
+	// Block 0x2a, offset 0xa80
+	0xa80: 0x03b2, 0xa81: 0x03b6, 0xa82: 0x040e, 0xa83: 0x0412, 0xa84: 0x03ba, 0xa85: 0x03be,
+	0xa86: 0x03c2, 0xa87: 0x03ca, 0xa88: 0x03ce, 0xa89: 0x03d2, 0xa8a: 0x03d6, 0xa8b: 0x03da,
+	0xa8c: 0x03de, 0xa8d: 0x03e2, 0xa8e: 0x03e6,
+	0xa92: 0x06c2, 0xa93: 0x071e, 0xa94: 0x06ce, 0xa95: 0x097e, 0xa96: 0x06d2, 0xa97: 0x06ea,
+	0xa98: 0x06d6, 0xa99: 0x0f96, 0xa9a: 0x070a, 0xa9b: 0x06de, 0xa9c: 0x06c6, 0xa9d: 0x0a02,
+	0xa9e: 0x0992, 0xa9f: 0x0732,
+	// Block 0x2b, offset 0xac0
+	0xac0: 0x205a, 0xac1: 0x2060, 0xac2: 0x2066, 0xac3: 0x206c, 0xac4: 0x2072, 0xac5: 0x2078,
+	0xac6: 0x207e, 0xac7: 0x2084, 0xac8: 0x208a, 0xac9: 0x2090, 0xaca: 0x2096, 0xacb: 0x209c,
+	0xacc: 0x20a2, 0xacd: 0x20a8, 0xace: 0x2733, 0xacf: 0x273c, 0xad0: 0x2745, 0xad1: 0x274e,
+	0xad2: 0x2757, 0xad3: 0x2760, 0xad4: 0x2769, 0xad5: 0x2772, 0xad6: 0x277b, 0xad7: 0x278d,
+	0xad8: 0x2796, 0xad9: 0x279f, 0xada: 0x27a8, 0xadb: 0x27b1, 0xadc: 0x2784, 0xadd: 0x2bb9,
+	0xade: 0x2afa, 0xae0: 0x20ae, 0xae1: 0x20c6, 0xae2: 0x20ba, 0xae3: 0x210e,
+	0xae4: 0x20cc, 0xae5: 0x20ea, 0xae6: 0x20b4, 0xae7: 0x20e4, 0xae8: 0x20c0, 0xae9: 0x20f6,
+	0xaea: 0x2126, 0xaeb: 0x2144, 0xaec: 0x213e, 0xaed: 0x2132, 0xaee: 0x2180, 0xaef: 0x2114,
+	0xaf0: 0x2120, 0xaf1: 0x2138, 0xaf2: 0x212c, 0xaf3: 0x2156, 0xaf4: 0x2102, 0xaf5: 0x214a,
+	0xaf6: 0x2174, 0xaf7: 0x215c, 0xaf8: 0x20f0, 0xaf9: 0x20d2, 0xafa: 0x2108, 0xafb: 0x211a,
+	0xafc: 0x2150, 0xafd: 0x20d8, 0xafe: 0x217a, 0xaff: 0x20fc,
+	// Block 0x2c, offset 0xb00
+	0xb00: 0x2162, 0xb01: 0x20de, 0xb02: 0x2168, 0xb03: 0x216e, 0xb04: 0x0932, 0xb05: 0x0b06,
+	0xb06: 0x0caa, 0xb07: 0x10ca,
+	0xb10: 0x1bca, 0xb11: 0x18ac,
+	0xb12: 0x18af, 0xb13: 0x18b2, 0xb14: 0x18b5, 0xb15: 0x18b8, 0xb16: 0x18bb, 0xb17: 0x18be,
+	0xb18: 0x18c1, 0xb19: 0x18c4, 0xb1a: 0x18cd, 0xb1b: 0x18d0, 0xb1c: 0x18d3, 0xb1d: 0x18d6,
+	0xb1e: 0x18d9, 0xb1f: 0x18dc, 0xb20: 0x0316, 0xb21: 0x031e, 0xb22: 0x0322, 0xb23: 0x032a,
+	0xb24: 0x032e, 0xb25: 0x0332, 0xb26: 0x033a, 0xb27: 0x0342, 0xb28: 0x0346, 0xb29: 0x034e,
+	0xb2a: 0x0352, 0xb2b: 0x0356, 0xb2c: 0x035a, 0xb2d: 0x035e, 0xb2e: 0x2e2f, 0xb2f: 0x2e37,
+	0xb30: 0x2e3f, 0xb31: 0x2e47, 0xb32: 0x2e4f, 0xb33: 0x2e57, 0xb34: 0x2e5f, 0xb35: 0x2e67,
+	0xb36: 0x2e77, 0xb37: 0x2e7f, 0xb38: 0x2e87, 0xb39: 0x2e8f, 0xb3a: 0x2e97, 0xb3b: 0x2e9f,
+	0xb3c: 0x2eea, 0xb3d: 0x2eb2, 0xb3e: 0x2e6f,
+	// Block 0x2d, offset 0xb40
+	0xb40: 0x06c2, 0xb41: 0x071e, 0xb42: 0x06ce, 0xb43: 0x097e, 0xb44: 0x0722, 0xb45: 0x07b2,
+	0xb46: 0x06ca, 0xb47: 0x07ae, 0xb48: 0x070e, 0xb49: 0x088a, 0xb4a: 0x0d0a, 0xb4b: 0x0e92,
+	0xb4c: 0x0dda, 0xb4d: 0x0d1e, 0xb4e: 0x1462, 0xb4f: 0x098e, 0xb50: 0x0cd2, 0xb51: 0x0d4e,
+	0xb52: 0x0d0e, 0xb53: 0x104e, 0xb54: 0x08fe, 0xb55: 0x0f06, 0xb56: 0x138a, 0xb57: 0x1062,
+	0xb58: 0x0846, 0xb59: 0x1092, 0xb5a: 0x0f9e, 0xb5b: 0x0a1a, 0xb5c: 0x1412, 0xb5d: 0x0782,
+	0xb5e: 0x08ae, 0xb5f: 0x0dfa, 0xb60: 0x152a, 0xb61: 0x0746, 0xb62: 0x07d6, 0xb63: 0x0d9e,
+	0xb64: 0x06d2, 0xb65: 0x06ea, 0xb66: 0x06d6, 0xb67: 0x0ade, 0xb68: 0x08f2, 0xb69: 0x0882,
+	0xb6a: 0x0a5a, 0xb6b: 0x0a4e, 0xb6c: 0x0fee, 0xb6d: 0x0742, 0xb6e: 0x139e, 0xb6f: 0x089e,
+	0xb70: 0x09f6, 0xb71: 0x18df, 0xb72: 0x18e2, 0xb73: 0x18e5, 0xb74: 0x18e8, 0xb75: 0x18f1,
+	0xb76: 0x18f4, 0xb77: 0x18f7, 0xb78: 0x18fa, 0xb79: 0x18fd, 0xb7a: 0x1900, 0xb7b: 0x1903,
+	0xb7c: 0x1906, 0xb7d: 0x1909, 0xb7e: 0x190c, 0xb7f: 0x1915,
+	// Block 0x2e, offset 0xb80
+	0xb80: 0x1ccc, 0xb81: 0x1cdb, 0xb82: 0x1cea, 0xb83: 0x1cf9, 0xb84: 0x1d08, 0xb85: 0x1d17,
+	0xb86: 0x1d26, 0xb87: 0x1d35, 0xb88: 0x1d44, 0xb89: 0x2192, 0xb8a: 0x21a4, 0xb8b: 0x21b6,
+	0xb8c: 0x1957, 0xb8d: 0x1c0a, 0xb8e: 0x19d8, 0xb8f: 0x1bae, 0xb90: 0x04ce, 0xb91: 0x04d6,
+	0xb92: 0x04de, 0xb93: 0x04e6, 0xb94: 0x04ee, 0xb95: 0x04f2, 0xb96: 0x04f6, 0xb97: 0x04fa,
+	0xb98: 0x04fe, 0xb99: 0x0502, 0xb9a: 0x0506, 0xb9b: 0x050a, 0xb9c: 0x050e, 0xb9d: 0x0512,
+	0xb9e: 0x0516, 0xb9f: 0x051a, 0xba0: 0x051e, 0xba1: 0x0526, 0xba2: 0x052a, 0xba3: 0x052e,
+	0xba4: 0x0532, 0xba5: 0x0536, 0xba6: 0x053a, 0xba7: 0x053e, 0xba8: 0x0542, 0xba9: 0x0546,
+	0xbaa: 0x054a, 0xbab: 0x054e, 0xbac: 0x0552, 0xbad: 0x0556, 0xbae: 0x055a, 0xbaf: 0x055e,
+	0xbb0: 0x0562, 0xbb1: 0x0566, 0xbb2: 0x056a, 0xbb3: 0x0572, 0xbb4: 0x057a, 0xbb5: 0x0582,
+	0xbb6: 0x0586, 0xbb7: 0x058a, 0xbb8: 0x058e, 0xbb9: 0x0592, 0xbba: 0x0596, 0xbbb: 0x059a,
+	0xbbc: 0x059e, 0xbbd: 0x05a2, 0xbbe: 0x05a6, 0xbbf: 0x2700,
+	// Block 0x2f, offset 0xbc0
+	0xbc0: 0x2b19, 0xbc1: 0x29b5, 0xbc2: 0x2b29, 0xbc3: 0x288d, 0xbc4: 0x2efb, 0xbc5: 0x2897,
+	0xbc6: 0x28a1, 0xbc7: 0x2f3f, 0xbc8: 0x29c2, 0xbc9: 0x28ab, 0xbca: 0x28b5, 0xbcb: 0x28bf,
+	0xbcc: 0x29e9, 0xbcd: 0x29f6, 0xbce: 0x29cf, 0xbcf: 0x29dc, 0xbd0: 0x2ec0, 0xbd1: 0x2a03,
+	0xbd2: 0x2a10, 0xbd3: 0x2bcb, 0xbd4: 0x26c1, 0xbd5: 0x2bde, 0xbd6: 0x2bf1, 0xbd7: 0x2b39,
+	0xbd8: 0x2a1d, 0xbd9: 0x2c04, 0xbda: 0x2c17, 0xbdb: 0x2a2a, 0xbdc: 0x28c9, 0xbdd: 0x28d3,
+	0xbde: 0x2ece, 0xbdf: 0x2a37, 0xbe0: 0x2b49, 0xbe1: 0x2f0c, 0xbe2: 0x28dd, 0xbe3: 0x28e7,
+	0xbe4: 0x2a44, 0xbe5: 0x28f1, 0xbe6: 0x28fb, 0xbe7: 0x26d6, 0xbe8: 0x26dd, 0xbe9: 0x2905,
+	0xbea: 0x290f, 0xbeb: 0x2c2a, 0xbec: 0x2a51, 0xbed: 0x2b59, 0xbee: 0x2c3d, 0xbef: 0x2a5e,
+	0xbf0: 0x2923, 0xbf1: 0x2919, 0xbf2: 0x2f53, 0xbf3: 0x2a6b, 0xbf4: 0x2c50, 0xbf5: 0x292d,
+	0xbf6: 0x2b69, 0xbf7: 0x2937, 0xbf8: 0x2a85, 0xbf9: 0x2941, 0xbfa: 0x2a92, 0xbfb: 0x2f1d,
+	0xbfc: 0x2a78, 0xbfd: 0x2b79, 0xbfe: 0x2a9f, 0xbff: 0x26e4,
+	// Block 0x30, offset 0xc00
+	0xc00: 0x2f2e, 0xc01: 0x294b, 0xc02: 0x2955, 0xc03: 0x2aac, 0xc04: 0x295f, 0xc05: 0x2969,
+	0xc06: 0x2973, 0xc07: 0x2b89, 0xc08: 0x2ab9, 0xc09: 0x26eb, 0xc0a: 0x2c63, 0xc0b: 0x2ea7,
+	0xc0c: 0x2b99, 0xc0d: 0x2ac6, 0xc0e: 0x2edc, 0xc0f: 0x297d, 0xc10: 0x2987, 0xc11: 0x2ad3,
+	0xc12: 0x26f2, 0xc13: 0x2ae0, 0xc14: 0x2ba9, 0xc15: 0x26f9, 0xc16: 0x2c76, 0xc17: 0x2991,
+	0xc18: 0x1cbd, 0xc19: 0x1cd1, 0xc1a: 0x1ce0, 0xc1b: 0x1cef, 0xc1c: 0x1cfe, 0xc1d: 0x1d0d,
+	0xc1e: 0x1d1c, 0xc1f: 0x1d2b, 0xc20: 0x1d3a, 0xc21: 0x1d49, 0xc22: 0x2198, 0xc23: 0x21aa,
+	0xc24: 0x21bc, 0xc25: 0x21c8, 0xc26: 0x21d4, 0xc27: 0x21e0, 0xc28: 0x21ec, 0xc29: 0x21f8,
+	0xc2a: 0x2204, 0xc2b: 0x2210, 0xc2c: 0x224c, 0xc2d: 0x2258, 0xc2e: 0x2264, 0xc2f: 0x2270,
+	0xc30: 0x227c, 0xc31: 0x1c1a, 0xc32: 0x19cc, 0xc33: 0x1939, 0xc34: 0x1bea, 0xc35: 0x1a4d,
+	0xc36: 0x1a5c, 0xc37: 0x19d2, 0xc38: 0x1c02, 0xc39: 0x1c06, 0xc3a: 0x1963, 0xc3b: 0x270e,
+	0xc3c: 0x271c, 0xc3d: 0x2707, 0xc3e: 0x2715, 0xc3f: 0x2aed,
+	// Block 0x31, offset 0xc40
+	0xc40: 0x1a50, 0xc41: 0x1a38, 0xc42: 0x1c66, 0xc43: 0x1a20, 0xc44: 0x19f9, 0xc45: 0x196c,
+	0xc46: 0x197b, 0xc47: 0x194b, 0xc48: 0x1bf6, 0xc49: 0x1d58, 0xc4a: 0x1a53, 0xc4b: 0x1a3b,
+	0xc4c: 0x1c6a, 0xc4d: 0x1c76, 0xc4e: 0x1a2c, 0xc4f: 0x1a02, 0xc50: 0x195a, 0xc51: 0x1c22,
+	0xc52: 0x1bb6, 0xc53: 0x1ba2, 0xc54: 0x1bd2, 0xc55: 0x1c7a, 0xc56: 0x1a2f, 0xc57: 0x19cf,
+	0xc58: 0x1a05, 0xc59: 0x19e4, 0xc5a: 0x1a47, 0xc5b: 0x1c7e, 0xc5c: 0x1a32, 0xc5d: 0x19c6,
+	0xc5e: 0x1a08, 0xc5f: 0x1c42, 0xc60: 0x1bfa, 0xc61: 0x1a1a, 0xc62: 0x1c2a, 0xc63: 0x1c46,
+	0xc64: 0x1bfe, 0xc65: 0x1a1d, 0xc66: 0x1c2e, 0xc67: 0x22ee, 0xc68: 0x2302, 0xc69: 0x199c,
+	0xc6a: 0x1c26, 0xc6b: 0x1bba, 0xc6c: 0x1ba6, 0xc6d: 0x1c4e, 0xc6e: 0x2723, 0xc6f: 0x27ba,
+	0xc70: 0x1a5f, 0xc71: 0x1a4a, 0xc72: 0x1c82, 0xc73: 0x1a35, 0xc74: 0x1a56, 0xc75: 0x1a3e,
+	0xc76: 0x1c6e, 0xc77: 0x1a23, 0xc78: 0x19fc, 0xc79: 0x1987, 0xc7a: 0x1a59, 0xc7b: 0x1a41,
+	0xc7c: 0x1c72, 0xc7d: 0x1a26, 0xc7e: 0x19ff, 0xc7f: 0x198a,
+	// Block 0x32, offset 0xc80
+	0xc80: 0x1c32, 0xc81: 0x1bbe, 0xc82: 0x1d53, 0xc83: 0x193c, 0xc84: 0x19c0, 0xc85: 0x19c3,
+	0xc86: 0x22fb, 0xc87: 0x1b9a, 0xc88: 0x19c9, 0xc89: 0x194e, 0xc8a: 0x19e7, 0xc8b: 0x1951,
+	0xc8c: 0x19f0, 0xc8d: 0x196f, 0xc8e: 0x1972, 0xc8f: 0x1a0b, 0xc90: 0x1a11, 0xc91: 0x1a14,
+	0xc92: 0x1c36, 0xc93: 0x1a17, 0xc94: 0x1a29, 0xc95: 0x1c3e, 0xc96: 0x1c4a, 0xc97: 0x1996,
+	0xc98: 0x1d5d, 0xc99: 0x1bc2, 0xc9a: 0x1999, 0xc9b: 0x1a62, 0xc9c: 0x19ab, 0xc9d: 0x19ba,
+	0xc9e: 0x22e8, 0xc9f: 0x22e2, 0xca0: 0x1cc7, 0xca1: 0x1cd6, 0xca2: 0x1ce5, 0xca3: 0x1cf4,
+	0xca4: 0x1d03, 0xca5: 0x1d12, 0xca6: 0x1d21, 0xca7: 0x1d30, 0xca8: 0x1d3f, 0xca9: 0x218c,
+	0xcaa: 0x219e, 0xcab: 0x21b0, 0xcac: 0x21c2, 0xcad: 0x21ce, 0xcae: 0x21da, 0xcaf: 0x21e6,
+	0xcb0: 0x21f2, 0xcb1: 0x21fe, 0xcb2: 0x220a, 0xcb3: 0x2246, 0xcb4: 0x2252, 0xcb5: 0x225e,
+	0xcb6: 0x226a, 0xcb7: 0x2276, 0xcb8: 0x2282, 0xcb9: 0x2288, 0xcba: 0x228e, 0xcbb: 0x2294,
+	0xcbc: 0x229a, 0xcbd: 0x22ac, 0xcbe: 0x22b2, 0xcbf: 0x1c16,
+	// Block 0x33, offset 0xcc0
+	0xcc0: 0x137a, 0xcc1: 0x0cfe, 0xcc2: 0x13d6, 0xcc3: 0x13a2, 0xcc4: 0x0e5a, 0xcc5: 0x06ee,
+	0xcc6: 0x08e2, 0xcc7: 0x162e, 0xcc8: 0x162e, 0xcc9: 0x0a0e, 0xcca: 0x1462, 0xccb: 0x0946,
+	0xccc: 0x0a0a, 0xccd: 0x0bf2, 0xcce: 0x0fd2, 0xccf: 0x1162, 0xcd0: 0x129a, 0xcd1: 0x12d6,
+	0xcd2: 0x130a, 0xcd3: 0x141e, 0xcd4: 0x0d76, 0xcd5: 0x0e02, 0xcd6: 0x0eae, 0xcd7: 0x0f46,
+	0xcd8: 0x1262, 0xcd9: 0x144a, 0xcda: 0x1576, 0xcdb: 0x0712, 0xcdc: 0x08b6, 0xcdd: 0x0d8a,
+	0xcde: 0x0ed2, 0xcdf: 0x1296, 0xce0: 0x15c6, 0xce1: 0x0ab6, 0xce2: 0x0e7a, 0xce3: 0x1286,
+	0xce4: 0x131a, 0xce5: 0x0c26, 0xce6: 0x11be, 0xce7: 0x12e2, 0xce8: 0x0b22, 0xce9: 0x0d12,
+	0xcea: 0x0e1a, 0xceb: 0x0f1e, 0xcec: 0x142a, 0xced: 0x0752, 0xcee: 0x07ea, 0xcef: 0x0856,
+	0xcf0: 0x0c8e, 0xcf1: 0x0d82, 0xcf2: 0x0ece, 0xcf3: 0x0ff2, 0xcf4: 0x117a, 0xcf5: 0x128e,
+	0xcf6: 0x12a6, 0xcf7: 0x13ca, 0xcf8: 0x14f2, 0xcf9: 0x15a6, 0xcfa: 0x15c2, 0xcfb: 0x102e,
+	0xcfc: 0x106e, 0xcfd: 0x1126, 0xcfe: 0x1246, 0xcff: 0x147e,
+	// Block 0x34, offset 0xd00
+	0xd00: 0x15ce, 0xd01: 0x134e, 0xd02: 0x09ca, 0xd03: 0x0b3e, 0xd04: 0x10de, 0xd05: 0x119e,
+	0xd06: 0x0f02, 0xd07: 0x1036, 0xd08: 0x139a, 0xd09: 0x14ea, 0xd0a: 0x09c6, 0xd0b: 0x0a92,
+	0xd0c: 0x0d7a, 0xd0d: 0x0e2e, 0xd0e: 0x0e62, 0xd0f: 0x1116, 0xd10: 0x113e, 0xd11: 0x14aa,
+	0xd12: 0x0852, 0xd13: 0x11aa, 0xd14: 0x07f6, 0xd15: 0x07f2, 0xd16: 0x109a, 0xd17: 0x112a,
+	0xd18: 0x125e, 0xd19: 0x14b2, 0xd1a: 0x136a, 0xd1b: 0x0c2a, 0xd1c: 0x0d76, 0xd1d: 0x135a,
+	0xd1e: 0x06fa, 0xd1f: 0x0a66, 0xd20: 0x0b96, 0xd21: 0x0f32, 0xd22: 0x0fb2, 0xd23: 0x0876,
+	0xd24: 0x103e, 0xd25: 0x0762, 0xd26: 0x0b7a, 0xd27: 0x06da, 0xd28: 0x0dee, 0xd29: 0x0ca6,
+	0xd2a: 0x1112, 0xd2b: 0x08ca, 0xd2c: 0x09b6, 0xd2d: 0x0ffe, 0xd2e: 0x1266, 0xd2f: 0x133e,
+	0xd30: 0x0dba, 0xd31: 0x13fa, 0xd32: 0x0de6, 0xd33: 0x0c3a, 0xd34: 0x121e, 0xd35: 0x0c5a,
+	0xd36: 0x0fae, 0xd37: 0x072e, 0xd38: 0x07aa, 0xd39: 0x07ee, 0xd3a: 0x0d56, 0xd3b: 0x10fe,
+	0xd3c: 0x11f6, 0xd3d: 0x134a, 0xd3e: 0x145e, 0xd3f: 0x085e,
+	// Block 0x35, offset 0xd40
+	0xd40: 0x0912, 0xd41: 0x0a1a, 0xd42: 0x0b32, 0xd43: 0x0cc2, 0xd44: 0x0e7e, 0xd45: 0x1042,
+	0xd46: 0x149a, 0xd47: 0x157e, 0xd48: 0x15d2, 0xd49: 0x15ea, 0xd4a: 0x083a, 0xd4b: 0x0cf6,
+	0xd4c: 0x0da6, 0xd4d: 0x13ee, 0xd4e: 0x0afe, 0xd4f: 0x0bda, 0xd50: 0x0bf6, 0xd51: 0x0c86,
+	0xd52: 0x0e6e, 0xd53: 0x0eba, 0xd54: 0x0f6a, 0xd55: 0x108e, 0xd56: 0x1132, 0xd57: 0x1196,
+	0xd58: 0x13de, 0xd59: 0x126e, 0xd5a: 0x1406, 0xd5b: 0x1482, 0xd5c: 0x0812, 0xd5d: 0x083e,
+	0xd5e: 0x0926, 0xd5f: 0x0eaa, 0xd60: 0x12f6, 0xd61: 0x133e, 0xd62: 0x0b1e, 0xd63: 0x0b8e,
+	0xd64: 0x0c52, 0xd65: 0x0db2, 0xd66: 0x10da, 0xd67: 0x0f26, 0xd68: 0x073e, 0xd69: 0x0982,
+	0xd6a: 0x0a66, 0xd6b: 0x0aca, 0xd6c: 0x0b9a, 0xd6d: 0x0f42, 0xd6e: 0x0f5e, 0xd6f: 0x116e,
+	0xd70: 0x118e, 0xd71: 0x1466, 0xd72: 0x14e6, 0xd73: 0x14f6, 0xd74: 0x1532, 0xd75: 0x0756,
+	0xd76: 0x1082, 0xd77: 0x1452, 0xd78: 0x14ce, 0xd79: 0x0bb2, 0xd7a: 0x071a, 0xd7b: 0x077a,
+	0xd7c: 0x0a6a, 0xd7d: 0x0a8a, 0xd7e: 0x0cb2, 0xd7f: 0x0d76,
+	// Block 0x36, offset 0xd80
+	0xd80: 0x0ec6, 0xd81: 0x0fce, 0xd82: 0x127a, 0xd83: 0x141a, 0xd84: 0x1626, 0xd85: 0x0ce6,
+	0xd86: 0x14a6, 0xd87: 0x0836, 0xd88: 0x0d32, 0xd89: 0x0d3e, 0xd8a: 0x0e12, 0xd8b: 0x0e4a,
+	0xd8c: 0x0f4e, 0xd8d: 0x0faa, 0xd8e: 0x102a, 0xd8f: 0x110e, 0xd90: 0x153e, 0xd91: 0x07b2,
+	0xd92: 0x0c06, 0xd93: 0x14b6, 0xd94: 0x076a, 0xd95: 0x0aae, 0xd96: 0x0e32, 0xd97: 0x13e2,
+	0xd98: 0x0b6a, 0xd99: 0x0bba, 0xd9a: 0x0d46, 0xd9b: 0x0f32, 0xd9c: 0x14be, 0xd9d: 0x081a,
+	0xd9e: 0x0902, 0xd9f: 0x0a9a, 0xda0: 0x0cd6, 0xda1: 0x0d22, 0xda2: 0x0d62, 0xda3: 0x0df6,
+	0xda4: 0x0f4a, 0xda5: 0x0fbe, 0xda6: 0x115a, 0xda7: 0x12fa, 0xda8: 0x1306, 0xda9: 0x145a,
+	0xdaa: 0x14da, 0xdab: 0x0886, 0xdac: 0x0e4e, 0xdad: 0x0906, 0xdae: 0x0eca, 0xdaf: 0x0f6e,
+	0xdb0: 0x128a, 0xdb1: 0x14c2, 0xdb2: 0x15ae, 0xdb3: 0x15d6, 0xdb4: 0x0d3a, 0xdb5: 0x0e2a,
+	0xdb6: 0x11c6, 0xdb7: 0x10ba, 0xdb8: 0x10c6, 0xdb9: 0x10ea, 0xdba: 0x0f1a, 0xdbb: 0x0ea2,
+	0xdbc: 0x1366, 0xdbd: 0x0736, 0xdbe: 0x122e, 0xdbf: 0x081e,
+	// Block 0x37, offset 0xdc0
+	0xdc0: 0x080e, 0xdc1: 0x0b0e, 0xdc2: 0x0c2e, 0xdc3: 0x10f6, 0xdc4: 0x0a56, 0xdc5: 0x0e06,
+	0xdc6: 0x0cf2, 0xdc7: 0x13ea, 0xdc8: 0x12ea, 0xdc9: 0x14ae, 0xdca: 0x1326, 0xdcb: 0x0b2a,
+	0xdcc: 0x078a, 0xdcd: 0x095e, 0xdd0: 0x09b2,
+	0xdd2: 0x0ce2, 0xdd5: 0x07fa, 0xdd6: 0x0f22, 0xdd7: 0x0fe6,
+	0xdd8: 0x104a, 0xdd9: 0x1066, 0xdda: 0x106a, 0xddb: 0x107e, 0xddc: 0x14fe, 0xddd: 0x10ee,
+	0xdde: 0x1172, 0xde0: 0x1292, 0xde2: 0x1356,
+	0xde5: 0x140a, 0xde6: 0x1436,
+	0xdea: 0x1552, 0xdeb: 0x1556, 0xdec: 0x155a, 0xded: 0x15be, 0xdee: 0x142e, 0xdef: 0x14ca,
+	0xdf0: 0x075a, 0xdf1: 0x077e, 0xdf2: 0x0792, 0xdf3: 0x084e, 0xdf4: 0x085a, 0xdf5: 0x089a,
+	0xdf6: 0x094e, 0xdf7: 0x096a, 0xdf8: 0x0972, 0xdf9: 0x09ae, 0xdfa: 0x09ba, 0xdfb: 0x0a96,
+	0xdfc: 0x0a9e, 0xdfd: 0x0ba6, 0xdfe: 0x0bce, 0xdff: 0x0bd6,
+	// Block 0x38, offset 0xe00
+	0xe00: 0x0bee, 0xe01: 0x0c9a, 0xe02: 0x0cca, 0xe03: 0x0cea, 0xe04: 0x0d5a, 0xe05: 0x0e1e,
+	0xe06: 0x0e3a, 0xe07: 0x0e6a, 0xe08: 0x0ebe, 0xe09: 0x0ede, 0xe0a: 0x0f52, 0xe0b: 0x1032,
+	0xe0c: 0x104e, 0xe0d: 0x1056, 0xe0e: 0x1052, 0xe0f: 0x105a, 0xe10: 0x105e, 0xe11: 0x1062,
+	0xe12: 0x1076, 0xe13: 0x107a, 0xe14: 0x109e, 0xe15: 0x10b2, 0xe16: 0x10ce, 0xe17: 0x1132,
+	0xe18: 0x113a, 0xe19: 0x1142, 0xe1a: 0x1156, 0xe1b: 0x117e, 0xe1c: 0x11ce, 0xe1d: 0x1202,
+	0xe1e: 0x1202, 0xe1f: 0x126a, 0xe20: 0x1312, 0xe21: 0x132a, 0xe22: 0x135e, 0xe23: 0x1362,
+	0xe24: 0x13a6, 0xe25: 0x13aa, 0xe26: 0x1402, 0xe27: 0x140a, 0xe28: 0x14de, 0xe29: 0x1522,
+	0xe2a: 0x153a, 0xe2b: 0x0b9e, 0xe2c: 0x1721, 0xe2d: 0x11e6,
+	0xe30: 0x06e2, 0xe31: 0x07e6, 0xe32: 0x07a6, 0xe33: 0x074e, 0xe34: 0x078e, 0xe35: 0x07ba,
+	0xe36: 0x084a, 0xe37: 0x0866, 0xe38: 0x094e, 0xe39: 0x093a, 0xe3a: 0x094a, 0xe3b: 0x0966,
+	0xe3c: 0x09b2, 0xe3d: 0x09c2, 0xe3e: 0x0a06, 0xe3f: 0x0a12,
+	// Block 0x39, offset 0xe40
+	0xe40: 0x0a2e, 0xe41: 0x0a3e, 0xe42: 0x0b26, 0xe43: 0x0b2e, 0xe44: 0x0b5e, 0xe45: 0x0b7e,
+	0xe46: 0x0bae, 0xe47: 0x0bc6, 0xe48: 0x0bb6, 0xe49: 0x0bd6, 0xe4a: 0x0bca, 0xe4b: 0x0bee,
+	0xe4c: 0x0c0a, 0xe4d: 0x0c62, 0xe4e: 0x0c6e, 0xe4f: 0x0c76, 0xe50: 0x0c9e, 0xe51: 0x0ce2,
+	0xe52: 0x0d12, 0xe53: 0x0d16, 0xe54: 0x0d2a, 0xe55: 0x0daa, 0xe56: 0x0dba, 0xe57: 0x0e12,
+	0xe58: 0x0e5e, 0xe59: 0x0e56, 0xe5a: 0x0e6a, 0xe5b: 0x0e86, 0xe5c: 0x0ebe, 0xe5d: 0x1016,
+	0xe5e: 0x0ee2, 0xe5f: 0x0f16, 0xe60: 0x0f22, 0xe61: 0x0f62, 0xe62: 0x0f7e, 0xe63: 0x0fa2,
+	0xe64: 0x0fc6, 0xe65: 0x0fca, 0xe66: 0x0fe6, 0xe67: 0x0fea, 0xe68: 0x0ffa, 0xe69: 0x100e,
+	0xe6a: 0x100a, 0xe6b: 0x103a, 0xe6c: 0x10b6, 0xe6d: 0x10ce, 0xe6e: 0x10e6, 0xe6f: 0x111e,
+	0xe70: 0x1132, 0xe71: 0x114e, 0xe72: 0x117e, 0xe73: 0x1232, 0xe74: 0x125a, 0xe75: 0x12ce,
+	0xe76: 0x1316, 0xe77: 0x1322, 0xe78: 0x132a, 0xe79: 0x1342, 0xe7a: 0x1356, 0xe7b: 0x1346,
+	0xe7c: 0x135e, 0xe7d: 0x135a, 0xe7e: 0x1352, 0xe7f: 0x1362,
+	// Block 0x3a, offset 0xe80
+	0xe80: 0x136e, 0xe81: 0x13aa, 0xe82: 0x13e6, 0xe83: 0x1416, 0xe84: 0x144e, 0xe85: 0x146e,
+	0xe86: 0x14ba, 0xe87: 0x14de, 0xe88: 0x14fe, 0xe89: 0x1512, 0xe8a: 0x1522, 0xe8b: 0x152e,
+	0xe8c: 0x153a, 0xe8d: 0x158e, 0xe8e: 0x162e, 0xe8f: 0x16b8, 0xe90: 0x16b3, 0xe91: 0x16e5,
+	0xe92: 0x060a, 0xe93: 0x0632, 0xe94: 0x0636, 0xe95: 0x1767, 0xe96: 0x1794, 0xe97: 0x180c,
+	0xe98: 0x161a, 0xe99: 0x162a,
+	// Block 0x3b, offset 0xec0
+	0xec0: 0x19db, 0xec1: 0x19de, 0xec2: 0x19e1, 0xec3: 0x1c0e, 0xec4: 0x1c12, 0xec5: 0x1a65,
+	0xec6: 0x1a65,
+	0xed3: 0x1d7b, 0xed4: 0x1d6c, 0xed5: 0x1d71, 0xed6: 0x1d80, 0xed7: 0x1d76,
+	0xedd: 0x43a7,
+	0xede: 0x8116, 0xedf: 0x4419, 0xee0: 0x0230, 0xee1: 0x0218, 0xee2: 0x0221, 0xee3: 0x0224,
+	0xee4: 0x0227, 0xee5: 0x022a, 0xee6: 0x022d, 0xee7: 0x0233, 0xee8: 0x0236, 0xee9: 0x0017,
+	0xeea: 0x4407, 0xeeb: 0x440d, 0xeec: 0x450b, 0xeed: 0x4513, 0xeee: 0x435f, 0xeef: 0x4365,
+	0xef0: 0x436b, 0xef1: 0x4371, 0xef2: 0x437d, 0xef3: 0x4383, 0xef4: 0x4389, 0xef5: 0x4395,
+	0xef6: 0x439b, 0xef8: 0x43a1, 0xef9: 0x43ad, 0xefa: 0x43b3, 0xefb: 0x43b9,
+	0xefc: 0x43c5, 0xefe: 0x43cb,
+	// Block 0x3c, offset 0xf00
+	0xf00: 0x43d1, 0xf01: 0x43d7, 0xf03: 0x43dd, 0xf04: 0x43e3,
+	0xf06: 0x43ef, 0xf07: 0x43f5, 0xf08: 0x43fb, 0xf09: 0x4401, 0xf0a: 0x4413, 0xf0b: 0x438f,
+	0xf0c: 0x4377, 0xf0d: 0x43bf, 0xf0e: 0x43e9, 0xf0f: 0x1d85, 0xf10: 0x029c, 0xf11: 0x029c,
+	0xf12: 0x02a5, 0xf13: 0x02a5, 0xf14: 0x02a5, 0xf15: 0x02a5, 0xf16: 0x02a8, 0xf17: 0x02a8,
+	0xf18: 0x02a8, 0xf19: 0x02a8, 0xf1a: 0x02ae, 0xf1b: 0x02ae, 0xf1c: 0x02ae, 0xf1d: 0x02ae,
+	0xf1e: 0x02a2, 0xf1f: 0x02a2, 0xf20: 0x02a2, 0xf21: 0x02a2, 0xf22: 0x02ab, 0xf23: 0x02ab,
+	0xf24: 0x02ab, 0xf25: 0x02ab, 0xf26: 0x029f, 0xf27: 0x029f, 0xf28: 0x029f, 0xf29: 0x029f,
+	0xf2a: 0x02d2, 0xf2b: 0x02d2, 0xf2c: 0x02d2, 0xf2d: 0x02d2, 0xf2e: 0x02d5, 0xf2f: 0x02d5,
+	0xf30: 0x02d5, 0xf31: 0x02d5, 0xf32: 0x02b4, 0xf33: 0x02b4, 0xf34: 0x02b4, 0xf35: 0x02b4,
+	0xf36: 0x02b1, 0xf37: 0x02b1, 0xf38: 0x02b1, 0xf39: 0x02b1, 0xf3a: 0x02b7, 0xf3b: 0x02b7,
+	0xf3c: 0x02b7, 0xf3d: 0x02b7, 0xf3e: 0x02ba, 0xf3f: 0x02ba,
+	// Block 0x3d, offset 0xf40
+	0xf40: 0x02ba, 0xf41: 0x02ba, 0xf42: 0x02c3, 0xf43: 0x02c3, 0xf44: 0x02c0, 0xf45: 0x02c0,
+	0xf46: 0x02c6, 0xf47: 0x02c6, 0xf48: 0x02bd, 0xf49: 0x02bd, 0xf4a: 0x02cc, 0xf4b: 0x02cc,
+	0xf4c: 0x02c9, 0xf4d: 0x02c9, 0xf4e: 0x02d8, 0xf4f: 0x02d8, 0xf50: 0x02d8, 0xf51: 0x02d8,
+	0xf52: 0x02de, 0xf53: 0x02de, 0xf54: 0x02de, 0xf55: 0x02de, 0xf56: 0x02e4, 0xf57: 0x02e4,
+	0xf58: 0x02e4, 0xf59: 0x02e4, 0xf5a: 0x02e1, 0xf5b: 0x02e1, 0xf5c: 0x02e1, 0xf5d: 0x02e1,
+	0xf5e: 0x02e7, 0xf5f: 0x02e7, 0xf60: 0x02ea, 0xf61: 0x02ea, 0xf62: 0x02ea, 0xf63: 0x02ea,
+	0xf64: 0x4485, 0xf65: 0x4485, 0xf66: 0x02f0, 0xf67: 0x02f0, 0xf68: 0x02f0, 0xf69: 0x02f0,
+	0xf6a: 0x02ed, 0xf6b: 0x02ed, 0xf6c: 0x02ed, 0xf6d: 0x02ed, 0xf6e: 0x030b, 0xf6f: 0x030b,
+	0xf70: 0x447f, 0xf71: 0x447f,
+	// Block 0x3e, offset 0xf80
+	0xf93: 0x02db, 0xf94: 0x02db, 0xf95: 0x02db, 0xf96: 0x02db, 0xf97: 0x02f9,
+	0xf98: 0x02f9, 0xf99: 0x02f6, 0xf9a: 0x02f6, 0xf9b: 0x02fc, 0xf9c: 0x02fc, 0xf9d: 0x2055,
+	0xf9e: 0x0302, 0xf9f: 0x0302, 0xfa0: 0x02f3, 0xfa1: 0x02f3, 0xfa2: 0x02ff, 0xfa3: 0x02ff,
+	0xfa4: 0x0308, 0xfa5: 0x0308, 0xfa6: 0x0308, 0xfa7: 0x0308, 0xfa8: 0x0290, 0xfa9: 0x0290,
+	0xfaa: 0x25b0, 0xfab: 0x25b0, 0xfac: 0x2620, 0xfad: 0x2620, 0xfae: 0x25ef, 0xfaf: 0x25ef,
+	0xfb0: 0x260b, 0xfb1: 0x260b, 0xfb2: 0x2604, 0xfb3: 0x2604, 0xfb4: 0x2612, 0xfb5: 0x2612,
+	0xfb6: 0x2619, 0xfb7: 0x2619, 0xfb8: 0x2619, 0xfb9: 0x25f6, 0xfba: 0x25f6, 0xfbb: 0x25f6,
+	0xfbc: 0x0305, 0xfbd: 0x0305, 0xfbe: 0x0305, 0xfbf: 0x0305,
+	// Block 0x3f, offset 0xfc0
+	0xfc0: 0x25b7, 0xfc1: 0x25be, 0xfc2: 0x25da, 0xfc3: 0x25f6, 0xfc4: 0x25fd, 0xfc5: 0x1d8f,
+	0xfc6: 0x1d94, 0xfc7: 0x1d99, 0xfc8: 0x1da8, 0xfc9: 0x1db7, 0xfca: 0x1dbc, 0xfcb: 0x1dc1,
+	0xfcc: 0x1dc6, 0xfcd: 0x1dcb, 0xfce: 0x1dda, 0xfcf: 0x1de9, 0xfd0: 0x1dee, 0xfd1: 0x1df3,
+	0xfd2: 0x1e02, 0xfd3: 0x1e11, 0xfd4: 0x1e16, 0xfd5: 0x1e1b, 0xfd6: 0x1e20, 0xfd7: 0x1e2f,
+	0xfd8: 0x1e34, 0xfd9: 0x1e43, 0xfda: 0x1e48, 0xfdb: 0x1e4d, 0xfdc: 0x1e5c, 0xfdd: 0x1e61,
+	0xfde: 0x1e66, 0xfdf: 0x1e70, 0xfe0: 0x1eac, 0xfe1: 0x1ebb, 0xfe2: 0x1eca, 0xfe3: 0x1ecf,
+	0xfe4: 0x1ed4, 0xfe5: 0x1ede, 0xfe6: 0x1eed, 0xfe7: 0x1ef2, 0xfe8: 0x1f01, 0xfe9: 0x1f06,
+	0xfea: 0x1f0b, 0xfeb: 0x1f1a, 0xfec: 0x1f1f, 0xfed: 0x1f2e, 0xfee: 0x1f33, 0xfef: 0x1f38,
+	0xff0: 0x1f3d, 0xff1: 0x1f42, 0xff2: 0x1f47, 0xff3: 0x1f4c, 0xff4: 0x1f51, 0xff5: 0x1f56,
+	0xff6: 0x1f5b, 0xff7: 0x1f60, 0xff8: 0x1f65, 0xff9: 0x1f6a, 0xffa: 0x1f6f, 0xffb: 0x1f74,
+	0xffc: 0x1f79, 0xffd: 0x1f7e, 0xffe: 0x1f83, 0xfff: 0x1f8d,
+	// Block 0x40, offset 0x1000
+	0x1000: 0x1f92, 0x1001: 0x1f97, 0x1002: 0x1f9c, 0x1003: 0x1fa6, 0x1004: 0x1fab, 0x1005: 0x1fb5,
+	0x1006: 0x1fba, 0x1007: 0x1fbf, 0x1008: 0x1fc4, 0x1009: 0x1fc9, 0x100a: 0x1fce, 0x100b: 0x1fd3,
+	0x100c: 0x1fd8, 0x100d: 0x1fdd, 0x100e: 0x1fec, 0x100f: 0x1ffb, 0x1010: 0x2000, 0x1011: 0x2005,
+	0x1012: 0x200a, 0x1013: 0x200f, 0x1014: 0x2014, 0x1015: 0x201e, 0x1016: 0x2023, 0x1017: 0x2028,
+	0x1018: 0x2037, 0x1019: 0x2046, 0x101a: 0x204b, 0x101b: 0x4437, 0x101c: 0x443d, 0x101d: 0x4473,
+	0x101e: 0x44ca, 0x101f: 0x44d1, 0x1020: 0x44d8, 0x1021: 0x44df, 0x1022: 0x44e6, 0x1023: 0x44ed,
+	0x1024: 0x25cc, 0x1025: 0x25d3, 0x1026: 0x25da, 0x1027: 0x25e1, 0x1028: 0x25f6, 0x1029: 0x25fd,
+	0x102a: 0x1d9e, 0x102b: 0x1da3, 0x102c: 0x1da8, 0x102d: 0x1dad, 0x102e: 0x1db7, 0x102f: 0x1dbc,
+	0x1030: 0x1dd0, 0x1031: 0x1dd5, 0x1032: 0x1dda, 0x1033: 0x1ddf, 0x1034: 0x1de9, 0x1035: 0x1dee,
+	0x1036: 0x1df8, 0x1037: 0x1dfd, 0x1038: 0x1e02, 0x1039: 0x1e07, 0x103a: 0x1e11, 0x103b: 0x1e16,
+	0x103c: 0x1f42, 0x103d: 0x1f47, 0x103e: 0x1f56, 0x103f: 0x1f5b,
+	// Block 0x41, offset 0x1040
+	0x1040: 0x1f60, 0x1041: 0x1f74, 0x1042: 0x1f79, 0x1043: 0x1f7e, 0x1044: 0x1f83, 0x1045: 0x1f9c,
+	0x1046: 0x1fa6, 0x1047: 0x1fab, 0x1048: 0x1fb0, 0x1049: 0x1fc4, 0x104a: 0x1fe2, 0x104b: 0x1fe7,
+	0x104c: 0x1fec, 0x104d: 0x1ff1, 0x104e: 0x1ffb, 0x104f: 0x2000, 0x1050: 0x4473, 0x1051: 0x202d,
+	0x1052: 0x2032, 0x1053: 0x2037, 0x1054: 0x203c, 0x1055: 0x2046, 0x1056: 0x204b, 0x1057: 0x25b7,
+	0x1058: 0x25be, 0x1059: 0x25c5, 0x105a: 0x25da, 0x105b: 0x25e8, 0x105c: 0x1d8f, 0x105d: 0x1d94,
+	0x105e: 0x1d99, 0x105f: 0x1da8, 0x1060: 0x1db2, 0x1061: 0x1dc1, 0x1062: 0x1dc6, 0x1063: 0x1dcb,
+	0x1064: 0x1dda, 0x1065: 0x1de4, 0x1066: 0x1e02, 0x1067: 0x1e1b, 0x1068: 0x1e20, 0x1069: 0x1e2f,
+	0x106a: 0x1e34, 0x106b: 0x1e43, 0x106c: 0x1e4d, 0x106d: 0x1e5c, 0x106e: 0x1e61, 0x106f: 0x1e66,
+	0x1070: 0x1e70, 0x1071: 0x1eac, 0x1072: 0x1eb1, 0x1073: 0x1ebb, 0x1074: 0x1eca, 0x1075: 0x1ecf,
+	0x1076: 0x1ed4, 0x1077: 0x1ede, 0x1078: 0x1eed, 0x1079: 0x1f01, 0x107a: 0x1f06, 0x107b: 0x1f0b,
+	0x107c: 0x1f1a, 0x107d: 0x1f1f, 0x107e: 0x1f2e, 0x107f: 0x1f33,
+	// Block 0x42, offset 0x1080
+	0x1080: 0x1f38, 0x1081: 0x1f3d, 0x1082: 0x1f4c, 0x1083: 0x1f51, 0x1084: 0x1f65, 0x1085: 0x1f6a,
+	0x1086: 0x1f6f, 0x1087: 0x1f74, 0x1088: 0x1f79, 0x1089: 0x1f8d, 0x108a: 0x1f92, 0x108b: 0x1f97,
+	0x108c: 0x1f9c, 0x108d: 0x1fa1, 0x108e: 0x1fb5, 0x108f: 0x1fba, 0x1090: 0x1fbf, 0x1091: 0x1fc4,
+	0x1092: 0x1fd3, 0x1093: 0x1fd8, 0x1094: 0x1fdd, 0x1095: 0x1fec, 0x1096: 0x1ff6, 0x1097: 0x2005,
+	0x1098: 0x200a, 0x1099: 0x4467, 0x109a: 0x201e, 0x109b: 0x2023, 0x109c: 0x2028, 0x109d: 0x2037,
+	0x109e: 0x2041, 0x109f: 0x25da, 0x10a0: 0x25e8, 0x10a1: 0x1da8, 0x10a2: 0x1db2, 0x10a3: 0x1dda,
+	0x10a4: 0x1de4, 0x10a5: 0x1e02, 0x10a6: 0x1e0c, 0x10a7: 0x1e70, 0x10a8: 0x1e75, 0x10a9: 0x1e98,
+	0x10aa: 0x1e9d, 0x10ab: 0x1f74, 0x10ac: 0x1f79, 0x10ad: 0x1f9c, 0x10ae: 0x1fec, 0x10af: 0x1ff6,
+	0x10b0: 0x2037, 0x10b1: 0x2041, 0x10b2: 0x451b, 0x10b3: 0x4523, 0x10b4: 0x452b, 0x10b5: 0x1ef7,
+	0x10b6: 0x1efc, 0x10b7: 0x1f10, 0x10b8: 0x1f15, 0x10b9: 0x1f24, 0x10ba: 0x1f29, 0x10bb: 0x1e7a,
+	0x10bc: 0x1e7f, 0x10bd: 0x1ea2, 0x10be: 0x1ea7, 0x10bf: 0x1e39,
+	// Block 0x43, offset 0x10c0
+	0x10c0: 0x1e3e, 0x10c1: 0x1e25, 0x10c2: 0x1e2a, 0x10c3: 0x1e52, 0x10c4: 0x1e57, 0x10c5: 0x1ec0,
+	0x10c6: 0x1ec5, 0x10c7: 0x1ee3, 0x10c8: 0x1ee8, 0x10c9: 0x1e84, 0x10ca: 0x1e89, 0x10cb: 0x1e8e,
+	0x10cc: 0x1e98, 0x10cd: 0x1e93, 0x10ce: 0x1e6b, 0x10cf: 0x1eb6, 0x10d0: 0x1ed9, 0x10d1: 0x1ef7,
+	0x10d2: 0x1efc, 0x10d3: 0x1f10, 0x10d4: 0x1f15, 0x10d5: 0x1f24, 0x10d6: 0x1f29, 0x10d7: 0x1e7a,
+	0x10d8: 0x1e7f, 0x10d9: 0x1ea2, 0x10da: 0x1ea7, 0x10db: 0x1e39, 0x10dc: 0x1e3e, 0x10dd: 0x1e25,
+	0x10de: 0x1e2a, 0x10df: 0x1e52, 0x10e0: 0x1e57, 0x10e1: 0x1ec0, 0x10e2: 0x1ec5, 0x10e3: 0x1ee3,
+	0x10e4: 0x1ee8, 0x10e5: 0x1e84, 0x10e6: 0x1e89, 0x10e7: 0x1e8e, 0x10e8: 0x1e98, 0x10e9: 0x1e93,
+	0x10ea: 0x1e6b, 0x10eb: 0x1eb6, 0x10ec: 0x1ed9, 0x10ed: 0x1e84, 0x10ee: 0x1e89, 0x10ef: 0x1e8e,
+	0x10f0: 0x1e98, 0x10f1: 0x1e75, 0x10f2: 0x1e9d, 0x10f3: 0x1ef2, 0x10f4: 0x1e5c, 0x10f5: 0x1e61,
+	0x10f6: 0x1e66, 0x10f7: 0x1e84, 0x10f8: 0x1e89, 0x10f9: 0x1e8e, 0x10fa: 0x1ef2, 0x10fb: 0x1f01,
+	0x10fc: 0x441f, 0x10fd: 0x441f,
+	// Block 0x44, offset 0x1100
+	0x1110: 0x2317, 0x1111: 0x232c,
+	0x1112: 0x232c, 0x1113: 0x2333, 0x1114: 0x233a, 0x1115: 0x234f, 0x1116: 0x2356, 0x1117: 0x235d,
+	0x1118: 0x2380, 0x1119: 0x2380, 0x111a: 0x23a3, 0x111b: 0x239c, 0x111c: 0x23b8, 0x111d: 0x23aa,
+	0x111e: 0x23b1, 0x111f: 0x23d4, 0x1120: 0x23d4, 0x1121: 0x23cd, 0x1122: 0x23db, 0x1123: 0x23db,
+	0x1124: 0x2405, 0x1125: 0x2405, 0x1126: 0x2421, 0x1127: 0x23e9, 0x1128: 0x23e9, 0x1129: 0x23e2,
+	0x112a: 0x23f7, 0x112b: 0x23f7, 0x112c: 0x23fe, 0x112d: 0x23fe, 0x112e: 0x2428, 0x112f: 0x2436,
+	0x1130: 0x2436, 0x1131: 0x243d, 0x1132: 0x243d, 0x1133: 0x2444, 0x1134: 0x244b, 0x1135: 0x2452,
+	0x1136: 0x2459, 0x1137: 0x2459, 0x1138: 0x2460, 0x1139: 0x246e, 0x113a: 0x247c, 0x113b: 0x2475,
+	0x113c: 0x2483, 0x113d: 0x2483, 0x113e: 0x2498, 0x113f: 0x249f,
+	// Block 0x45, offset 0x1140
+	0x1140: 0x24d0, 0x1141: 0x24de, 0x1142: 0x24d7, 0x1143: 0x24bb, 0x1144: 0x24bb, 0x1145: 0x24e5,
+	0x1146: 0x24e5, 0x1147: 0x24ec, 0x1148: 0x24ec, 0x1149: 0x2516, 0x114a: 0x251d, 0x114b: 0x2524,
+	0x114c: 0x24fa, 0x114d: 0x2508, 0x114e: 0x252b, 0x114f: 0x2532,
+	0x1152: 0x2501, 0x1153: 0x2586, 0x1154: 0x258d, 0x1155: 0x2563, 0x1156: 0x256a, 0x1157: 0x254e,
+	0x1158: 0x254e, 0x1159: 0x2555, 0x115a: 0x257f, 0x115b: 0x2578, 0x115c: 0x25a2, 0x115d: 0x25a2,
+	0x115e: 0x2310, 0x115f: 0x2325, 0x1160: 0x231e, 0x1161: 0x2348, 0x1162: 0x2341, 0x1163: 0x236b,
+	0x1164: 0x2364, 0x1165: 0x238e, 0x1166: 0x2372, 0x1167: 0x2387, 0x1168: 0x23bf, 0x1169: 0x240c,
+	0x116a: 0x23f0, 0x116b: 0x242f, 0x116c: 0x24c9, 0x116d: 0x24f3, 0x116e: 0x259b, 0x116f: 0x2594,
+	0x1170: 0x25a9, 0x1171: 0x2540, 0x1172: 0x24a6, 0x1173: 0x2571, 0x1174: 0x2498, 0x1175: 0x24d0,
+	0x1176: 0x2467, 0x1177: 0x24b4, 0x1178: 0x2547, 0x1179: 0x2539, 0x117a: 0x24c2, 0x117b: 0x24ad,
+	0x117c: 0x24c2, 0x117d: 0x2547, 0x117e: 0x2379, 0x117f: 0x2395,
+	// Block 0x46, offset 0x1180
+	0x1180: 0x250f, 0x1181: 0x248a, 0x1182: 0x2309, 0x1183: 0x24ad, 0x1184: 0x2452, 0x1185: 0x2421,
+	0x1186: 0x23c6, 0x1187: 0x255c,
+	0x11b0: 0x241a, 0x11b1: 0x2491, 0x11b2: 0x27cc, 0x11b3: 0x27c3, 0x11b4: 0x27f9, 0x11b5: 0x27e7,
+	0x11b6: 0x27d5, 0x11b7: 0x27f0, 0x11b8: 0x2802, 0x11b9: 0x2413, 0x11ba: 0x2c89, 0x11bb: 0x2b09,
+	0x11bc: 0x27de,
+	// Block 0x47, offset 0x11c0
+	0x11d0: 0x0019, 0x11d1: 0x0486,
+	0x11d2: 0x048a, 0x11d3: 0x0035, 0x11d4: 0x0037, 0x11d5: 0x0003, 0x11d6: 0x003f, 0x11d7: 0x04c2,
+	0x11d8: 0x04c6, 0x11d9: 0x1b62,
+	0x11e0: 0x8133, 0x11e1: 0x8133, 0x11e2: 0x8133, 0x11e3: 0x8133,
+	0x11e4: 0x8133, 0x11e5: 0x8133, 0x11e6: 0x8133, 0x11e7: 0x812e, 0x11e8: 0x812e, 0x11e9: 0x812e,
+	0x11ea: 0x812e, 0x11eb: 0x812e, 0x11ec: 0x812e, 0x11ed: 0x812e, 0x11ee: 0x8133, 0x11ef: 0x8133,
+	0x11f0: 0x1876, 0x11f1: 0x0446, 0x11f2: 0x0442, 0x11f3: 0x007f, 0x11f4: 0x007f, 0x11f5: 0x0011,
+	0x11f6: 0x0013, 0x11f7: 0x00b7, 0x11f8: 0x00bb, 0x11f9: 0x04ba, 0x11fa: 0x04be, 0x11fb: 0x04ae,
+	0x11fc: 0x04b2, 0x11fd: 0x0496, 0x11fe: 0x049a, 0x11ff: 0x048e,
+	// Block 0x48, offset 0x1200
+	0x1200: 0x0492, 0x1201: 0x049e, 0x1202: 0x04a2, 0x1203: 0x04a6, 0x1204: 0x04aa,
+	0x1207: 0x0077, 0x1208: 0x007b, 0x1209: 0x4280, 0x120a: 0x4280, 0x120b: 0x4280,
+	0x120c: 0x4280, 0x120d: 0x007f, 0x120e: 0x007f, 0x120f: 0x007f, 0x1210: 0x0019, 0x1211: 0x0486,
+	0x1212: 0x001d, 0x1214: 0x0037, 0x1215: 0x0035, 0x1216: 0x003f, 0x1217: 0x0003,
+	0x1218: 0x0446, 0x1219: 0x0011, 0x121a: 0x0013, 0x121b: 0x00b7, 0x121c: 0x00bb, 0x121d: 0x04ba,
+	0x121e: 0x04be, 0x121f: 0x0007, 0x1220: 0x000d, 0x1221: 0x0015, 0x1222: 0x0017, 0x1223: 0x001b,
+	0x1224: 0x0039, 0x1225: 0x003d, 0x1226: 0x003b, 0x1228: 0x0079, 0x1229: 0x0009,
+	0x122a: 0x000b, 0x122b: 0x0041,
+	0x1230: 0x42c1, 0x1231: 0x4443, 0x1232: 0x42c6, 0x1234: 0x42cb,
+	0x1236: 0x42d0, 0x1237: 0x4449, 0x1238: 0x42d5, 0x1239: 0x444f, 0x123a: 0x42da, 0x123b: 0x4455,
+	0x123c: 0x42df, 0x123d: 0x445b, 0x123e: 0x42e4, 0x123f: 0x4461,
+	// Block 0x49, offset 0x1240
+	0x1240: 0x0239, 0x1241: 0x4425, 0x1242: 0x4425, 0x1243: 0x442b, 0x1244: 0x442b, 0x1245: 0x446d,
+	0x1246: 0x446d, 0x1247: 0x4431, 0x1248: 0x4431, 0x1249: 0x4479, 0x124a: 0x4479, 0x124b: 0x4479,
+	0x124c: 0x4479, 0x124d: 0x023c, 0x124e: 0x023c, 0x124f: 0x023f, 0x1250: 0x023f, 0x1251: 0x023f,
+	0x1252: 0x023f, 0x1253: 0x0242, 0x1254: 0x0242, 0x1255: 0x0245, 0x1256: 0x0245, 0x1257: 0x0245,
+	0x1258: 0x0245, 0x1259: 0x0248, 0x125a: 0x0248, 0x125b: 0x0248, 0x125c: 0x0248, 0x125d: 0x024b,
+	0x125e: 0x024b, 0x125f: 0x024b, 0x1260: 0x024b, 0x1261: 0x024e, 0x1262: 0x024e, 0x1263: 0x024e,
+	0x1264: 0x024e, 0x1265: 0x0251, 0x1266: 0x0251, 0x1267: 0x0251, 0x1268: 0x0251, 0x1269: 0x0254,
+	0x126a: 0x0254, 0x126b: 0x0257, 0x126c: 0x0257, 0x126d: 0x025a, 0x126e: 0x025a, 0x126f: 0x025d,
+	0x1270: 0x025d, 0x1271: 0x0260, 0x1272: 0x0260, 0x1273: 0x0260, 0x1274: 0x0260, 0x1275: 0x0263,
+	0x1276: 0x0263, 0x1277: 0x0263, 0x1278: 0x0263, 0x1279: 0x0266, 0x127a: 0x0266, 0x127b: 0x0266,
+	0x127c: 0x0266, 0x127d: 0x0269, 0x127e: 0x0269, 0x127f: 0x0269,
+	// Block 0x4a, offset 0x1280
+	0x1280: 0x0269, 0x1281: 0x026c, 0x1282: 0x026c, 0x1283: 0x026c, 0x1284: 0x026c, 0x1285: 0x026f,
+	0x1286: 0x026f, 0x1287: 0x026f, 0x1288: 0x026f, 0x1289: 0x0272, 0x128a: 0x0272, 0x128b: 0x0272,
+	0x128c: 0x0272, 0x128d: 0x0275, 0x128e: 0x0275, 0x128f: 0x0275, 0x1290: 0x0275, 0x1291: 0x0278,
+	0x1292: 0x0278, 0x1293: 0x0278, 0x1294: 0x0278, 0x1295: 0x027b, 0x1296: 0x027b, 0x1297: 0x027b,
+	0x1298: 0x027b, 0x1299: 0x027e, 0x129a: 0x027e, 0x129b: 0x027e, 0x129c: 0x027e, 0x129d: 0x0281,
+	0x129e: 0x0281, 0x129f: 0x0281, 0x12a0: 0x0281, 0x12a1: 0x0284, 0x12a2: 0x0284, 0x12a3: 0x0284,
+	0x12a4: 0x0284, 0x12a5: 0x0287, 0x12a6: 0x0287, 0x12a7: 0x0287, 0x12a8: 0x0287, 0x12a9: 0x028a,
+	0x12aa: 0x028a, 0x12ab: 0x028a, 0x12ac: 0x028a, 0x12ad: 0x028d, 0x12ae: 0x028d, 0x12af: 0x0290,
+	0x12b0: 0x0290, 0x12b1: 0x0293, 0x12b2: 0x0293, 0x12b3: 0x0293, 0x12b4: 0x0293, 0x12b5: 0x2e17,
+	0x12b6: 0x2e17, 0x12b7: 0x2e1f, 0x12b8: 0x2e1f, 0x12b9: 0x2e27, 0x12ba: 0x2e27, 0x12bb: 0x1f88,
+	0x12bc: 0x1f88,
+	// Block 0x4b, offset 0x12c0
+	0x12c0: 0x0081, 0x12c1: 0x0083, 0x12c2: 0x0085, 0x12c3: 0x0087, 0x12c4: 0x0089, 0x12c5: 0x008b,
+	0x12c6: 0x008d, 0x12c7: 0x008f, 0x12c8: 0x0091, 0x12c9: 0x0093, 0x12ca: 0x0095, 0x12cb: 0x0097,
+	0x12cc: 0x0099, 0x12cd: 0x009b, 0x12ce: 0x009d, 0x12cf: 0x009f, 0x12d0: 0x00a1, 0x12d1: 0x00a3,
+	0x12d2: 0x00a5, 0x12d3: 0x00a7, 0x12d4: 0x00a9, 0x12d5: 0x00ab, 0x12d6: 0x00ad, 0x12d7: 0x00af,
+	0x12d8: 0x00b1, 0x12d9: 0x00b3, 0x12da: 0x00b5, 0x12db: 0x00b7, 0x12dc: 0x00b9, 0x12dd: 0x00bb,
+	0x12de: 0x00bd, 0x12df: 0x047a, 0x12e0: 0x047e, 0x12e1: 0x048a, 0x12e2: 0x049e, 0x12e3: 0x04a2,
+	0x12e4: 0x0486, 0x12e5: 0x05ae, 0x12e6: 0x05a6, 0x12e7: 0x04ca, 0x12e8: 0x04d2, 0x12e9: 0x04da,
+	0x12ea: 0x04e2, 0x12eb: 0x04ea, 0x12ec: 0x056e, 0x12ed: 0x0576, 0x12ee: 0x057e, 0x12ef: 0x0522,
+	0x12f0: 0x05b2, 0x12f1: 0x04ce, 0x12f2: 0x04d6, 0x12f3: 0x04de, 0x12f4: 0x04e6, 0x12f5: 0x04ee,
+	0x12f6: 0x04f2, 0x12f7: 0x04f6, 0x12f8: 0x04fa, 0x12f9: 0x04fe, 0x12fa: 0x0502, 0x12fb: 0x0506,
+	0x12fc: 0x050a, 0x12fd: 0x050e, 0x12fe: 0x0512, 0x12ff: 0x0516,
+	// Block 0x4c, offset 0x1300
+	0x1300: 0x051a, 0x1301: 0x051e, 0x1302: 0x0526, 0x1303: 0x052a, 0x1304: 0x052e, 0x1305: 0x0532,
+	0x1306: 0x0536, 0x1307: 0x053a, 0x1308: 0x053e, 0x1309: 0x0542, 0x130a: 0x0546, 0x130b: 0x054a,
+	0x130c: 0x054e, 0x130d: 0x0552, 0x130e: 0x0556, 0x130f: 0x055a, 0x1310: 0x055e, 0x1311: 0x0562,
+	0x1312: 0x0566, 0x1313: 0x056a, 0x1314: 0x0572, 0x1315: 0x057a, 0x1316: 0x0582, 0x1317: 0x0586,
+	0x1318: 0x058a, 0x1319: 0x058e, 0x131a: 0x0592, 0x131b: 0x0596, 0x131c: 0x059a, 0x131d: 0x05aa,
+	0x131e: 0x4a8f, 0x131f: 0x4a95, 0x1320: 0x03c6, 0x1321: 0x0316, 0x1322: 0x031a, 0x1323: 0x4a52,
+	0x1324: 0x031e, 0x1325: 0x4a58, 0x1326: 0x4a5e, 0x1327: 0x0322, 0x1328: 0x0326, 0x1329: 0x032a,
+	0x132a: 0x4a64, 0x132b: 0x4a6a, 0x132c: 0x4a70, 0x132d: 0x4a76, 0x132e: 0x4a7c, 0x132f: 0x4a82,
+	0x1330: 0x036a, 0x1331: 0x032e, 0x1332: 0x0332, 0x1333: 0x0336, 0x1334: 0x037e, 0x1335: 0x033a,
+	0x1336: 0x033e, 0x1337: 0x0342, 0x1338: 0x0346, 0x1339: 0x034a, 0x133a: 0x034e, 0x133b: 0x0352,
+	0x133c: 0x0356, 0x133d: 0x035a, 0x133e: 0x035e,
+	// Block 0x4d, offset 0x1340
+	0x1342: 0x49d4, 0x1343: 0x49da, 0x1344: 0x49e0, 0x1345: 0x49e6,
+	0x1346: 0x49ec, 0x1347: 0x49f2, 0x134a: 0x49f8, 0x134b: 0x49fe,
+	0x134c: 0x4a04, 0x134d: 0x4a0a, 0x134e: 0x4a10, 0x134f: 0x4a16,
+	0x1352: 0x4a1c, 0x1353: 0x4a22, 0x1354: 0x4a28, 0x1355: 0x4a2e, 0x1356: 0x4a34, 0x1357: 0x4a3a,
+	0x135a: 0x4a40, 0x135b: 0x4a46, 0x135c: 0x4a4c,
+	0x1360: 0x00bf, 0x1361: 0x00c2, 0x1362: 0x00cb, 0x1363: 0x427b,
+	0x1364: 0x00c8, 0x1365: 0x00c5, 0x1366: 0x044a, 0x1368: 0x046e, 0x1369: 0x044e,
+	0x136a: 0x0452, 0x136b: 0x0456, 0x136c: 0x045a, 0x136d: 0x0472, 0x136e: 0x0476,
+	// Block 0x4e, offset 0x1380
+	0x1380: 0x0063, 0x1381: 0x0065, 0x1382: 0x0067, 0x1383: 0x0069, 0x1384: 0x006b, 0x1385: 0x006d,
+	0x1386: 0x006f, 0x1387: 0x0071, 0x1388: 0x0073, 0x1389: 0x0075, 0x138a: 0x0083, 0x138b: 0x0085,
+	0x138c: 0x0087, 0x138d: 0x0089, 0x138e: 0x008b, 0x138f: 0x008d, 0x1390: 0x008f, 0x1391: 0x0091,
+	0x1392: 0x0093, 0x1393: 0x0095, 0x1394: 0x0097, 0x1395: 0x0099, 0x1396: 0x009b, 0x1397: 0x009d,
+	0x1398: 0x009f, 0x1399: 0x00a1, 0x139a: 0x00a3, 0x139b: 0x00a5, 0x139c: 0x00a7, 0x139d: 0x00a9,
+	0x139e: 0x00ab, 0x139f: 0x00ad, 0x13a0: 0x00af, 0x13a1: 0x00b1, 0x13a2: 0x00b3, 0x13a3: 0x00b5,
+	0x13a4: 0x00dd, 0x13a5: 0x00f2, 0x13a8: 0x0176, 0x13a9: 0x0179,
+	0x13aa: 0x017c, 0x13ab: 0x017f, 0x13ac: 0x0182, 0x13ad: 0x0185, 0x13ae: 0x0188, 0x13af: 0x018b,
+	0x13b0: 0x018e, 0x13b1: 0x0191, 0x13b2: 0x0194, 0x13b3: 0x0197, 0x13b4: 0x019a, 0x13b5: 0x019d,
+	0x13b6: 0x01a0, 0x13b7: 0x01a3, 0x13b8: 0x01a6, 0x13b9: 0x018b, 0x13ba: 0x01a9, 0x13bb: 0x01ac,
+	0x13bc: 0x01af, 0x13bd: 0x01b2, 0x13be: 0x01b5, 0x13bf: 0x01b8,
+	// Block 0x4f, offset 0x13c0
+	0x13c0: 0x0200, 0x13c1: 0x0203, 0x13c2: 0x0206, 0x13c3: 0x045e, 0x13c4: 0x01ca, 0x13c5: 0x01d3,
+	0x13c6: 0x01d9, 0x13c7: 0x01fd, 0x13c8: 0x01ee, 0x13c9: 0x01eb, 0x13ca: 0x0209, 0x13cb: 0x020c,
+	0x13ce: 0x0021, 0x13cf: 0x0023, 0x13d0: 0x0025, 0x13d1: 0x0027,
+	0x13d2: 0x0029, 0x13d3: 0x002b, 0x13d4: 0x002d, 0x13d5: 0x002f, 0x13d6: 0x0031, 0x13d7: 0x0033,
+	0x13d8: 0x0021, 0x13d9: 0x0023, 0x13da: 0x0025, 0x13db: 0x0027, 0x13dc: 0x0029, 0x13dd: 0x002b,
+	0x13de: 0x002d, 0x13df: 0x002f, 0x13e0: 0x0031, 0x13e1: 0x0033, 0x13e2: 0x0021, 0x13e3: 0x0023,
+	0x13e4: 0x0025, 0x13e5: 0x0027, 0x13e6: 0x0029, 0x13e7: 0x002b, 0x13e8: 0x002d, 0x13e9: 0x002f,
+	0x13ea: 0x0031, 0x13eb: 0x0033, 0x13ec: 0x0021, 0x13ed: 0x0023, 0x13ee: 0x0025, 0x13ef: 0x0027,
+	0x13f0: 0x0029, 0x13f1: 0x002b, 0x13f2: 0x002d, 0x13f3: 0x002f, 0x13f4: 0x0031, 0x13f5: 0x0033,
+	0x13f6: 0x0021, 0x13f7: 0x0023, 0x13f8: 0x0025, 0x13f9: 0x0027, 0x13fa: 0x0029, 0x13fb: 0x002b,
+	0x13fc: 0x002d, 0x13fd: 0x002f, 0x13fe: 0x0031, 0x13ff: 0x0033,
+	// Block 0x50, offset 0x1400
+	0x1400: 0x023c, 0x1401: 0x023f, 0x1402: 0x024b, 0x1403: 0x0254, 0x1405: 0x028d,
+	0x1406: 0x025d, 0x1407: 0x024e, 0x1408: 0x026c, 0x1409: 0x0293, 0x140a: 0x027e, 0x140b: 0x0281,
+	0x140c: 0x0284, 0x140d: 0x0287, 0x140e: 0x0260, 0x140f: 0x0272, 0x1410: 0x0278, 0x1411: 0x0266,
+	0x1412: 0x027b, 0x1413: 0x025a, 0x1414: 0x0263, 0x1415: 0x0245, 0x1416: 0x0248, 0x1417: 0x0251,
+	0x1418: 0x0257, 0x1419: 0x0269, 0x141a: 0x026f, 0x141b: 0x0275, 0x141c: 0x0296, 0x141d: 0x02e7,
+	0x141e: 0x02cf, 0x141f: 0x0299, 0x1421: 0x023f, 0x1422: 0x024b,
+	0x1424: 0x028a, 0x1427: 0x024e, 0x1429: 0x0293,
+	0x142a: 0x027e, 0x142b: 0x0281, 0x142c: 0x0284, 0x142d: 0x0287, 0x142e: 0x0260, 0x142f: 0x0272,
+	0x1430: 0x0278, 0x1431: 0x0266, 0x1432: 0x027b, 0x1434: 0x0263, 0x1435: 0x0245,
+	0x1436: 0x0248, 0x1437: 0x0251, 0x1439: 0x0269, 0x143b: 0x0275,
+	// Block 0x51, offset 0x1440
+	0x1442: 0x024b,
+	0x1447: 0x024e, 0x1449: 0x0293, 0x144b: 0x0281,
+	0x144d: 0x0287, 0x144e: 0x0260, 0x144f: 0x0272, 0x1451: 0x0266,
+	0x1452: 0x027b, 0x1454: 0x0263, 0x1457: 0x0251,
+	0x1459: 0x0269, 0x145b: 0x0275, 0x145d: 0x02e7,
+	0x145f: 0x0299, 0x1461: 0x023f, 0x1462: 0x024b,
+	0x1464: 0x028a, 0x1467: 0x024e, 0x1468: 0x026c, 0x1469: 0x0293,
+	0x146a: 0x027e, 0x146c: 0x0284, 0x146d: 0x0287, 0x146e: 0x0260, 0x146f: 0x0272,
+	0x1470: 0x0278, 0x1471: 0x0266, 0x1472: 0x027b, 0x1474: 0x0263, 0x1475: 0x0245,
+	0x1476: 0x0248, 0x1477: 0x0251, 0x1479: 0x0269, 0x147a: 0x026f, 0x147b: 0x0275,
+	0x147c: 0x0296, 0x147e: 0x02cf,
+	// Block 0x52, offset 0x1480
+	0x1480: 0x023c, 0x1481: 0x023f, 0x1482: 0x024b, 0x1483: 0x0254, 0x1484: 0x028a, 0x1485: 0x028d,
+	0x1486: 0x025d, 0x1487: 0x024e, 0x1488: 0x026c, 0x1489: 0x0293, 0x148b: 0x0281,
+	0x148c: 0x0284, 0x148d: 0x0287, 0x148e: 0x0260, 0x148f: 0x0272, 0x1490: 0x0278, 0x1491: 0x0266,
+	0x1492: 0x027b, 0x1493: 0x025a, 0x1494: 0x0263, 0x1495: 0x0245, 0x1496: 0x0248, 0x1497: 0x0251,
+	0x1498: 0x0257, 0x1499: 0x0269, 0x149a: 0x026f, 0x149b: 0x0275,
+	0x14a1: 0x023f, 0x14a2: 0x024b, 0x14a3: 0x0254,
+	0x14a5: 0x028d, 0x14a6: 0x025d, 0x14a7: 0x024e, 0x14a8: 0x026c, 0x14a9: 0x0293,
+	0x14ab: 0x0281, 0x14ac: 0x0284, 0x14ad: 0x0287, 0x14ae: 0x0260, 0x14af: 0x0272,
+	0x14b0: 0x0278, 0x14b1: 0x0266, 0x14b2: 0x027b, 0x14b3: 0x025a, 0x14b4: 0x0263, 0x14b5: 0x0245,
+	0x14b6: 0x0248, 0x14b7: 0x0251, 0x14b8: 0x0257, 0x14b9: 0x0269, 0x14ba: 0x026f, 0x14bb: 0x0275,
+	// Block 0x53, offset 0x14c0
+	0x14c0: 0x187c, 0x14c1: 0x1879, 0x14c2: 0x187f, 0x14c3: 0x18a3, 0x14c4: 0x18c7, 0x14c5: 0x18eb,
+	0x14c6: 0x190f, 0x14c7: 0x1918, 0x14c8: 0x191e, 0x14c9: 0x1924, 0x14ca: 0x192a,
+	0x14d0: 0x1a92, 0x14d1: 0x1a96,
+	0x14d2: 0x1a9a, 0x14d3: 0x1a9e, 0x14d4: 0x1aa2, 0x14d5: 0x1aa6, 0x14d6: 0x1aaa, 0x14d7: 0x1aae,
+	0x14d8: 0x1ab2, 0x14d9: 0x1ab6, 0x14da: 0x1aba, 0x14db: 0x1abe, 0x14dc: 0x1ac2, 0x14dd: 0x1ac6,
+	0x14de: 0x1aca, 0x14df: 0x1ace, 0x14e0: 0x1ad2, 0x14e1: 0x1ad6, 0x14e2: 0x1ada, 0x14e3: 0x1ade,
+	0x14e4: 0x1ae2, 0x14e5: 0x1ae6, 0x14e6: 0x1aea, 0x14e7: 0x1aee, 0x14e8: 0x1af2, 0x14e9: 0x1af6,
+	0x14ea: 0x272b, 0x14eb: 0x0047, 0x14ec: 0x0065, 0x14ed: 0x193f, 0x14ee: 0x19b7,
+	0x14f0: 0x0043, 0x14f1: 0x0045, 0x14f2: 0x0047, 0x14f3: 0x0049, 0x14f4: 0x004b, 0x14f5: 0x004d,
+	0x14f6: 0x004f, 0x14f7: 0x0051, 0x14f8: 0x0053, 0x14f9: 0x0055, 0x14fa: 0x0057, 0x14fb: 0x0059,
+	0x14fc: 0x005b, 0x14fd: 0x005d, 0x14fe: 0x005f, 0x14ff: 0x0061,
+	// Block 0x54, offset 0x1500
+	0x1500: 0x26b3, 0x1501: 0x26c8, 0x1502: 0x0506,
+	0x1510: 0x0c12, 0x1511: 0x0a4a,
+	0x1512: 0x08d6, 0x1513: 0x45db, 0x1514: 0x071e, 0x1515: 0x09f2, 0x1516: 0x1332, 0x1517: 0x0a02,
+	0x1518: 0x072a, 0x1519: 0x0cda, 0x151a: 0x0eb2, 0x151b: 0x0cb2, 0x151c: 0x082a, 0x151d: 0x0b6e,
+	0x151e: 0x07c2, 0x151f: 0x0cba, 0x1520: 0x0816, 0x1521: 0x111a, 0x1522: 0x0f86, 0x1523: 0x138e,
+	0x1524: 0x09d6, 0x1525: 0x090e, 0x1526: 0x0e66, 0x1527: 0x0c1e, 0x1528: 0x0c4a, 0x1529: 0x06c2,
+	0x152a: 0x06ce, 0x152b: 0x140e, 0x152c: 0x0ade, 0x152d: 0x06ea, 0x152e: 0x08f2, 0x152f: 0x0c3e,
+	0x1530: 0x13b6, 0x1531: 0x0c16, 0x1532: 0x1072, 0x1533: 0x10ae, 0x1534: 0x08fa, 0x1535: 0x0e46,
+	0x1536: 0x0d0e, 0x1537: 0x0d0a, 0x1538: 0x0f9a, 0x1539: 0x082e, 0x153a: 0x095a, 0x153b: 0x1446,
+	// Block 0x55, offset 0x1540
+	0x1540: 0x06fe, 0x1541: 0x06f6, 0x1542: 0x0706, 0x1543: 0x164a, 0x1544: 0x074a, 0x1545: 0x075a,
+	0x1546: 0x075e, 0x1547: 0x0766, 0x1548: 0x076e, 0x1549: 0x0772, 0x154a: 0x077e, 0x154b: 0x0776,
+	0x154c: 0x05b6, 0x154d: 0x165e, 0x154e: 0x0792, 0x154f: 0x0796, 0x1550: 0x079a, 0x1551: 0x07b6,
+	0x1552: 0x164f, 0x1553: 0x05ba, 0x1554: 0x07a2, 0x1555: 0x07c2, 0x1556: 0x1659, 0x1557: 0x07d2,
+	0x1558: 0x07da, 0x1559: 0x073a, 0x155a: 0x07e2, 0x155b: 0x07e6, 0x155c: 0x1834, 0x155d: 0x0802,
+	0x155e: 0x080a, 0x155f: 0x05c2, 0x1560: 0x0822, 0x1561: 0x0826, 0x1562: 0x082e, 0x1563: 0x0832,
+	0x1564: 0x05c6, 0x1565: 0x084a, 0x1566: 0x084e, 0x1567: 0x085a, 0x1568: 0x0866, 0x1569: 0x086a,
+	0x156a: 0x086e, 0x156b: 0x0876, 0x156c: 0x0896, 0x156d: 0x089a, 0x156e: 0x08a2, 0x156f: 0x08b2,
+	0x1570: 0x08ba, 0x1571: 0x08be, 0x1572: 0x08be, 0x1573: 0x08be, 0x1574: 0x166d, 0x1575: 0x0e96,
+	0x1576: 0x08d2, 0x1577: 0x08da, 0x1578: 0x1672, 0x1579: 0x08e6, 0x157a: 0x08ee, 0x157b: 0x08f6,
+	0x157c: 0x091e, 0x157d: 0x090a, 0x157e: 0x0916, 0x157f: 0x091a,
+	// Block 0x56, offset 0x1580
+	0x1580: 0x0922, 0x1581: 0x092a, 0x1582: 0x092e, 0x1583: 0x0936, 0x1584: 0x093e, 0x1585: 0x0942,
+	0x1586: 0x0942, 0x1587: 0x094a, 0x1588: 0x0952, 0x1589: 0x0956, 0x158a: 0x0962, 0x158b: 0x0986,
+	0x158c: 0x096a, 0x158d: 0x098a, 0x158e: 0x096e, 0x158f: 0x0976, 0x1590: 0x080e, 0x1591: 0x09d2,
+	0x1592: 0x099a, 0x1593: 0x099e, 0x1594: 0x09a2, 0x1595: 0x0996, 0x1596: 0x09aa, 0x1597: 0x09a6,
+	0x1598: 0x09be, 0x1599: 0x1677, 0x159a: 0x09da, 0x159b: 0x09de, 0x159c: 0x09e6, 0x159d: 0x09f2,
+	0x159e: 0x09fa, 0x159f: 0x0a16, 0x15a0: 0x167c, 0x15a1: 0x1681, 0x15a2: 0x0a22, 0x15a3: 0x0a26,
+	0x15a4: 0x0a2a, 0x15a5: 0x0a1e, 0x15a6: 0x0a32, 0x15a7: 0x05ca, 0x15a8: 0x05ce, 0x15a9: 0x0a3a,
+	0x15aa: 0x0a42, 0x15ab: 0x0a42, 0x15ac: 0x1686, 0x15ad: 0x0a5e, 0x15ae: 0x0a62, 0x15af: 0x0a66,
+	0x15b0: 0x0a6e, 0x15b1: 0x168b, 0x15b2: 0x0a76, 0x15b3: 0x0a7a, 0x15b4: 0x0b52, 0x15b5: 0x0a82,
+	0x15b6: 0x05d2, 0x15b7: 0x0a8e, 0x15b8: 0x0a9e, 0x15b9: 0x0aaa, 0x15ba: 0x0aa6, 0x15bb: 0x1695,
+	0x15bc: 0x0ab2, 0x15bd: 0x169a, 0x15be: 0x0abe, 0x15bf: 0x0aba,
+	// Block 0x57, offset 0x15c0
+	0x15c0: 0x0ac2, 0x15c1: 0x0ad2, 0x15c2: 0x0ad6, 0x15c3: 0x05d6, 0x15c4: 0x0ae6, 0x15c5: 0x0aee,
+	0x15c6: 0x0af2, 0x15c7: 0x0af6, 0x15c8: 0x05da, 0x15c9: 0x169f, 0x15ca: 0x05de, 0x15cb: 0x0b12,
+	0x15cc: 0x0b16, 0x15cd: 0x0b1a, 0x15ce: 0x0b22, 0x15cf: 0x1866, 0x15d0: 0x0b3a, 0x15d1: 0x16a9,
+	0x15d2: 0x16a9, 0x15d3: 0x11da, 0x15d4: 0x0b4a, 0x15d5: 0x0b4a, 0x15d6: 0x05e2, 0x15d7: 0x16cc,
+	0x15d8: 0x179e, 0x15d9: 0x0b5a, 0x15da: 0x0b62, 0x15db: 0x05e6, 0x15dc: 0x0b76, 0x15dd: 0x0b86,
+	0x15de: 0x0b8a, 0x15df: 0x0b92, 0x15e0: 0x0ba2, 0x15e1: 0x05ee, 0x15e2: 0x05ea, 0x15e3: 0x0ba6,
+	0x15e4: 0x16ae, 0x15e5: 0x0baa, 0x15e6: 0x0bbe, 0x15e7: 0x0bc2, 0x15e8: 0x0bc6, 0x15e9: 0x0bc2,
+	0x15ea: 0x0bd2, 0x15eb: 0x0bd6, 0x15ec: 0x0be6, 0x15ed: 0x0bde, 0x15ee: 0x0be2, 0x15ef: 0x0bea,
+	0x15f0: 0x0bee, 0x15f1: 0x0bf2, 0x15f2: 0x0bfe, 0x15f3: 0x0c02, 0x15f4: 0x0c1a, 0x15f5: 0x0c22,
+	0x15f6: 0x0c32, 0x15f7: 0x0c46, 0x15f8: 0x16bd, 0x15f9: 0x0c42, 0x15fa: 0x0c36, 0x15fb: 0x0c4e,
+	0x15fc: 0x0c56, 0x15fd: 0x0c6a, 0x15fe: 0x16c2, 0x15ff: 0x0c72,
+	// Block 0x58, offset 0x1600
+	0x1600: 0x0c66, 0x1601: 0x0c5e, 0x1602: 0x05f2, 0x1603: 0x0c7a, 0x1604: 0x0c82, 0x1605: 0x0c8a,
+	0x1606: 0x0c7e, 0x1607: 0x05f6, 0x1608: 0x0c9a, 0x1609: 0x0ca2, 0x160a: 0x16c7, 0x160b: 0x0cce,
+	0x160c: 0x0d02, 0x160d: 0x0cde, 0x160e: 0x0602, 0x160f: 0x0cea, 0x1610: 0x05fe, 0x1611: 0x05fa,
+	0x1612: 0x07c6, 0x1613: 0x07ca, 0x1614: 0x0d06, 0x1615: 0x0cee, 0x1616: 0x11ae, 0x1617: 0x0666,
+	0x1618: 0x0d12, 0x1619: 0x0d16, 0x161a: 0x0d1a, 0x161b: 0x0d2e, 0x161c: 0x0d26, 0x161d: 0x16e0,
+	0x161e: 0x0606, 0x161f: 0x0d42, 0x1620: 0x0d36, 0x1621: 0x0d52, 0x1622: 0x0d5a, 0x1623: 0x16ea,
+	0x1624: 0x0d5e, 0x1625: 0x0d4a, 0x1626: 0x0d66, 0x1627: 0x060a, 0x1628: 0x0d6a, 0x1629: 0x0d6e,
+	0x162a: 0x0d72, 0x162b: 0x0d7e, 0x162c: 0x16ef, 0x162d: 0x0d86, 0x162e: 0x060e, 0x162f: 0x0d92,
+	0x1630: 0x16f4, 0x1631: 0x0d96, 0x1632: 0x0612, 0x1633: 0x0da2, 0x1634: 0x0dae, 0x1635: 0x0dba,
+	0x1636: 0x0dbe, 0x1637: 0x16f9, 0x1638: 0x1690, 0x1639: 0x16fe, 0x163a: 0x0dde, 0x163b: 0x1703,
+	0x163c: 0x0dea, 0x163d: 0x0df2, 0x163e: 0x0de2, 0x163f: 0x0dfe,
+	// Block 0x59, offset 0x1640
+	0x1640: 0x0e0e, 0x1641: 0x0e1e, 0x1642: 0x0e12, 0x1643: 0x0e16, 0x1644: 0x0e22, 0x1645: 0x0e26,
+	0x1646: 0x1708, 0x1647: 0x0e0a, 0x1648: 0x0e3e, 0x1649: 0x0e42, 0x164a: 0x0616, 0x164b: 0x0e56,
+	0x164c: 0x0e52, 0x164d: 0x170d, 0x164e: 0x0e36, 0x164f: 0x0e72, 0x1650: 0x1712, 0x1651: 0x1717,
+	0x1652: 0x0e76, 0x1653: 0x0e8a, 0x1654: 0x0e86, 0x1655: 0x0e82, 0x1656: 0x061a, 0x1657: 0x0e8e,
+	0x1658: 0x0e9e, 0x1659: 0x0e9a, 0x165a: 0x0ea6, 0x165b: 0x1654, 0x165c: 0x0eb6, 0x165d: 0x171c,
+	0x165e: 0x0ec2, 0x165f: 0x1726, 0x1660: 0x0ed6, 0x1661: 0x0ee2, 0x1662: 0x0ef6, 0x1663: 0x172b,
+	0x1664: 0x0f0a, 0x1665: 0x0f0e, 0x1666: 0x1730, 0x1667: 0x1735, 0x1668: 0x0f2a, 0x1669: 0x0f3a,
+	0x166a: 0x061e, 0x166b: 0x0f3e, 0x166c: 0x0622, 0x166d: 0x0622, 0x166e: 0x0f56, 0x166f: 0x0f5a,
+	0x1670: 0x0f62, 0x1671: 0x0f66, 0x1672: 0x0f72, 0x1673: 0x0626, 0x1674: 0x0f8a, 0x1675: 0x173a,
+	0x1676: 0x0fa6, 0x1677: 0x173f, 0x1678: 0x0fb2, 0x1679: 0x16a4, 0x167a: 0x0fc2, 0x167b: 0x1744,
+	0x167c: 0x1749, 0x167d: 0x174e, 0x167e: 0x062a, 0x167f: 0x062e,
+	// Block 0x5a, offset 0x1680
+	0x1680: 0x0ffa, 0x1681: 0x1758, 0x1682: 0x1753, 0x1683: 0x175d, 0x1684: 0x1762, 0x1685: 0x1002,
+	0x1686: 0x1006, 0x1687: 0x1006, 0x1688: 0x100e, 0x1689: 0x0636, 0x168a: 0x1012, 0x168b: 0x063a,
+	0x168c: 0x063e, 0x168d: 0x176c, 0x168e: 0x1026, 0x168f: 0x102e, 0x1690: 0x103a, 0x1691: 0x0642,
+	0x1692: 0x1771, 0x1693: 0x105e, 0x1694: 0x1776, 0x1695: 0x177b, 0x1696: 0x107e, 0x1697: 0x1096,
+	0x1698: 0x0646, 0x1699: 0x109e, 0x169a: 0x10a2, 0x169b: 0x10a6, 0x169c: 0x1780, 0x169d: 0x1785,
+	0x169e: 0x1785, 0x169f: 0x10be, 0x16a0: 0x064a, 0x16a1: 0x178a, 0x16a2: 0x10d2, 0x16a3: 0x10d6,
+	0x16a4: 0x064e, 0x16a5: 0x178f, 0x16a6: 0x10f2, 0x16a7: 0x0652, 0x16a8: 0x1102, 0x16a9: 0x10fa,
+	0x16aa: 0x110a, 0x16ab: 0x1799, 0x16ac: 0x1122, 0x16ad: 0x0656, 0x16ae: 0x112e, 0x16af: 0x1136,
+	0x16b0: 0x1146, 0x16b1: 0x065a, 0x16b2: 0x17a3, 0x16b3: 0x17a8, 0x16b4: 0x065e, 0x16b5: 0x17ad,
+	0x16b6: 0x115e, 0x16b7: 0x17b2, 0x16b8: 0x116a, 0x16b9: 0x1176, 0x16ba: 0x117e, 0x16bb: 0x17b7,
+	0x16bc: 0x17bc, 0x16bd: 0x1192, 0x16be: 0x17c1, 0x16bf: 0x119a,
+	// Block 0x5b, offset 0x16c0
+	0x16c0: 0x16d1, 0x16c1: 0x0662, 0x16c2: 0x11b2, 0x16c3: 0x11b6, 0x16c4: 0x066a, 0x16c5: 0x11ba,
+	0x16c6: 0x0a36, 0x16c7: 0x17c6, 0x16c8: 0x17cb, 0x16c9: 0x16d6, 0x16ca: 0x16db, 0x16cb: 0x11da,
+	0x16cc: 0x11de, 0x16cd: 0x13f6, 0x16ce: 0x066e, 0x16cf: 0x120a, 0x16d0: 0x1206, 0x16d1: 0x120e,
+	0x16d2: 0x0842, 0x16d3: 0x1212, 0x16d4: 0x1216, 0x16d5: 0x121a, 0x16d6: 0x1222, 0x16d7: 0x17d0,
+	0x16d8: 0x121e, 0x16d9: 0x1226, 0x16da: 0x123a, 0x16db: 0x123e, 0x16dc: 0x122a, 0x16dd: 0x1242,
+	0x16de: 0x1256, 0x16df: 0x126a, 0x16e0: 0x1236, 0x16e1: 0x124a, 0x16e2: 0x124e, 0x16e3: 0x1252,
+	0x16e4: 0x17d5, 0x16e5: 0x17df, 0x16e6: 0x17da, 0x16e7: 0x0672, 0x16e8: 0x1272, 0x16e9: 0x1276,
+	0x16ea: 0x127e, 0x16eb: 0x17f3, 0x16ec: 0x1282, 0x16ed: 0x17e4, 0x16ee: 0x0676, 0x16ef: 0x067a,
+	0x16f0: 0x17e9, 0x16f1: 0x17ee, 0x16f2: 0x067e, 0x16f3: 0x12a2, 0x16f4: 0x12a6, 0x16f5: 0x12aa,
+	0x16f6: 0x12ae, 0x16f7: 0x12ba, 0x16f8: 0x12b6, 0x16f9: 0x12c2, 0x16fa: 0x12be, 0x16fb: 0x12ce,
+	0x16fc: 0x12c6, 0x16fd: 0x12ca, 0x16fe: 0x12d2, 0x16ff: 0x0682,
+	// Block 0x5c, offset 0x1700
+	0x1700: 0x12da, 0x1701: 0x12de, 0x1702: 0x0686, 0x1703: 0x12ee, 0x1704: 0x12f2, 0x1705: 0x17f8,
+	0x1706: 0x12fe, 0x1707: 0x1302, 0x1708: 0x068a, 0x1709: 0x130e, 0x170a: 0x05be, 0x170b: 0x17fd,
+	0x170c: 0x1802, 0x170d: 0x068e, 0x170e: 0x0692, 0x170f: 0x133a, 0x1710: 0x1352, 0x1711: 0x136e,
+	0x1712: 0x137e, 0x1713: 0x1807, 0x1714: 0x1392, 0x1715: 0x1396, 0x1716: 0x13ae, 0x1717: 0x13ba,
+	0x1718: 0x1811, 0x1719: 0x1663, 0x171a: 0x13c6, 0x171b: 0x13c2, 0x171c: 0x13ce, 0x171d: 0x1668,
+	0x171e: 0x13da, 0x171f: 0x13e6, 0x1720: 0x1816, 0x1721: 0x181b, 0x1722: 0x1426, 0x1723: 0x1432,
+	0x1724: 0x143a, 0x1725: 0x1820, 0x1726: 0x143e, 0x1727: 0x146a, 0x1728: 0x1476, 0x1729: 0x147a,
+	0x172a: 0x1472, 0x172b: 0x1486, 0x172c: 0x148a, 0x172d: 0x1825, 0x172e: 0x1496, 0x172f: 0x0696,
+	0x1730: 0x149e, 0x1731: 0x182a, 0x1732: 0x069a, 0x1733: 0x14d6, 0x1734: 0x0ac6, 0x1735: 0x14ee,
+	0x1736: 0x182f, 0x1737: 0x1839, 0x1738: 0x069e, 0x1739: 0x06a2, 0x173a: 0x1516, 0x173b: 0x183e,
+	0x173c: 0x06a6, 0x173d: 0x1843, 0x173e: 0x152e, 0x173f: 0x152e,
+	// Block 0x5d, offset 0x1740
+	0x1740: 0x1536, 0x1741: 0x1848, 0x1742: 0x154e, 0x1743: 0x06aa, 0x1744: 0x155e, 0x1745: 0x156a,
+	0x1746: 0x1572, 0x1747: 0x157a, 0x1748: 0x06ae, 0x1749: 0x184d, 0x174a: 0x158e, 0x174b: 0x15aa,
+	0x174c: 0x15b6, 0x174d: 0x06b2, 0x174e: 0x06b6, 0x174f: 0x15ba, 0x1750: 0x1852, 0x1751: 0x06ba,
+	0x1752: 0x1857, 0x1753: 0x185c, 0x1754: 0x1861, 0x1755: 0x15de, 0x1756: 0x06be, 0x1757: 0x15f2,
+	0x1758: 0x15fa, 0x1759: 0x15fe, 0x175a: 0x1606, 0x175b: 0x160e, 0x175c: 0x1616, 0x175d: 0x186b,
+}
+
+// nfkcIndex: 22 blocks, 1408 entries, 2816 bytes
+// Block 0 is the zero block.
+var nfkcIndex = [1408]uint16{
+	// Block 0x0, offset 0x0
+	// Block 0x1, offset 0x40
+	// Block 0x2, offset 0x80
+	// Block 0x3, offset 0xc0
+	0xc2: 0x5c, 0xc3: 0x01, 0xc4: 0x02, 0xc5: 0x03, 0xc6: 0x5d, 0xc7: 0x04,
+	0xc8: 0x05, 0xca: 0x5e, 0xcb: 0x5f, 0xcc: 0x06, 0xcd: 0x07, 0xce: 0x08, 0xcf: 0x09,
+	0xd0: 0x0a, 0xd1: 0x60, 0xd2: 0x61, 0xd3: 0x0b, 0xd6: 0x0c, 0xd7: 0x62,
+	0xd8: 0x63, 0xd9: 0x0d, 0xdb: 0x64, 0xdc: 0x65, 0xdd: 0x66, 0xdf: 0x67,
+	0xe0: 0x02, 0xe1: 0x03, 0xe2: 0x04, 0xe3: 0x05,
+	0xea: 0x06, 0xeb: 0x07, 0xec: 0x08, 0xed: 0x09, 0xef: 0x0a,
+	0xf0: 0x13,
+	// Block 0x4, offset 0x100
+	0x120: 0x68, 0x121: 0x69, 0x123: 0x0e, 0x124: 0x6a, 0x125: 0x6b, 0x126: 0x6c, 0x127: 0x6d,
+	0x128: 0x6e, 0x129: 0x6f, 0x12a: 0x70, 0x12b: 0x71, 0x12c: 0x6c, 0x12d: 0x72, 0x12e: 0x73, 0x12f: 0x74,
+	0x131: 0x75, 0x132: 0x76, 0x133: 0x77, 0x134: 0x78, 0x135: 0x79, 0x137: 0x7a,
+	0x138: 0x7b, 0x139: 0x7c, 0x13a: 0x7d, 0x13b: 0x7e, 0x13c: 0x7f, 0x13d: 0x80, 0x13e: 0x81, 0x13f: 0x82,
+	// Block 0x5, offset 0x140
+	0x140: 0x83, 0x142: 0x84, 0x143: 0x85, 0x144: 0x86, 0x145: 0x87, 0x146: 0x88, 0x147: 0x89,
+	0x14d: 0x8a,
+	0x15c: 0x8b, 0x15f: 0x8c,
+	0x162: 0x8d, 0x164: 0x8e,
+	0x168: 0x8f, 0x169: 0x90, 0x16a: 0x91, 0x16b: 0x92, 0x16c: 0x0f, 0x16d: 0x93, 0x16e: 0x94, 0x16f: 0x95,
+	0x170: 0x96, 0x173: 0x97, 0x174: 0x98, 0x175: 0x10, 0x176: 0x11, 0x177: 0x12,
+	0x178: 0x13, 0x179: 0x14, 0x17a: 0x15, 0x17b: 0x16, 0x17c: 0x17, 0x17d: 0x18, 0x17e: 0x19, 0x17f: 0x1a,
+	// Block 0x6, offset 0x180
+	0x180: 0x99, 0x181: 0x9a, 0x182: 0x9b, 0x183: 0x9c, 0x184: 0x1b, 0x185: 0x1c, 0x186: 0x9d, 0x187: 0x9e,
+	0x188: 0x9f, 0x189: 0x1d, 0x18a: 0x1e, 0x18b: 0xa0, 0x18c: 0xa1,
+	0x191: 0x1f, 0x192: 0x20, 0x193: 0xa2,
+	0x1a8: 0xa3, 0x1a9: 0xa4, 0x1ab: 0xa5,
+	0x1b1: 0xa6, 0x1b3: 0xa7, 0x1b5: 0xa8, 0x1b7: 0xa9,
+	0x1ba: 0xaa, 0x1bb: 0xab, 0x1bc: 0x21, 0x1bd: 0x22, 0x1be: 0x23, 0x1bf: 0xac,
+	// Block 0x7, offset 0x1c0
+	0x1c0: 0xad, 0x1c1: 0x24, 0x1c2: 0x25, 0x1c3: 0x26, 0x1c4: 0xae, 0x1c5: 0x27, 0x1c6: 0x28,
+	0x1c8: 0x29, 0x1c9: 0x2a, 0x1ca: 0x2b, 0x1cb: 0x2c, 0x1cc: 0x2d, 0x1cd: 0x2e, 0x1ce: 0x2f, 0x1cf: 0x30,
+	// Block 0x8, offset 0x200
+	0x219: 0xaf, 0x21a: 0xb0, 0x21b: 0xb1, 0x21d: 0xb2, 0x21f: 0xb3,
+	0x220: 0xb4, 0x223: 0xb5, 0x224: 0xb6, 0x225: 0xb7, 0x226: 0xb8, 0x227: 0xb9,
+	0x22a: 0xba, 0x22b: 0xbb, 0x22d: 0xbc, 0x22f: 0xbd,
+	0x230: 0xbe, 0x231: 0xbf, 0x232: 0xc0, 0x233: 0xc1, 0x234: 0xc2, 0x235: 0xc3, 0x236: 0xc4, 0x237: 0xbe,
+	0x238: 0xbf, 0x239: 0xc0, 0x23a: 0xc1, 0x23b: 0xc2, 0x23c: 0xc3, 0x23d: 0xc4, 0x23e: 0xbe, 0x23f: 0xbf,
+	// Block 0x9, offset 0x240
+	0x240: 0xc0, 0x241: 0xc1, 0x242: 0xc2, 0x243: 0xc3, 0x244: 0xc4, 0x245: 0xbe, 0x246: 0xbf, 0x247: 0xc0,
+	0x248: 0xc1, 0x249: 0xc2, 0x24a: 0xc3, 0x24b: 0xc4, 0x24c: 0xbe, 0x24d: 0xbf, 0x24e: 0xc0, 0x24f: 0xc1,
+	0x250: 0xc2, 0x251: 0xc3, 0x252: 0xc4, 0x253: 0xbe, 0x254: 0xbf, 0x255: 0xc0, 0x256: 0xc1, 0x257: 0xc2,
+	0x258: 0xc3, 0x259: 0xc4, 0x25a: 0xbe, 0x25b: 0xbf, 0x25c: 0xc0, 0x25d: 0xc1, 0x25e: 0xc2, 0x25f: 0xc3,
+	0x260: 0xc4, 0x261: 0xbe, 0x262: 0xbf, 0x263: 0xc0, 0x264: 0xc1, 0x265: 0xc2, 0x266: 0xc3, 0x267: 0xc4,
+	0x268: 0xbe, 0x269: 0xbf, 0x26a: 0xc0, 0x26b: 0xc1, 0x26c: 0xc2, 0x26d: 0xc3, 0x26e: 0xc4, 0x26f: 0xbe,
+	0x270: 0xbf, 0x271: 0xc0, 0x272: 0xc1, 0x273: 0xc2, 0x274: 0xc3, 0x275: 0xc4, 0x276: 0xbe, 0x277: 0xbf,
+	0x278: 0xc0, 0x279: 0xc1, 0x27a: 0xc2, 0x27b: 0xc3, 0x27c: 0xc4, 0x27d: 0xbe, 0x27e: 0xbf, 0x27f: 0xc0,
+	// Block 0xa, offset 0x280
+	0x280: 0xc1, 0x281: 0xc2, 0x282: 0xc3, 0x283: 0xc4, 0x284: 0xbe, 0x285: 0xbf, 0x286: 0xc0, 0x287: 0xc1,
+	0x288: 0xc2, 0x289: 0xc3, 0x28a: 0xc4, 0x28b: 0xbe, 0x28c: 0xbf, 0x28d: 0xc0, 0x28e: 0xc1, 0x28f: 0xc2,
+	0x290: 0xc3, 0x291: 0xc4, 0x292: 0xbe, 0x293: 0xbf, 0x294: 0xc0, 0x295: 0xc1, 0x296: 0xc2, 0x297: 0xc3,
+	0x298: 0xc4, 0x299: 0xbe, 0x29a: 0xbf, 0x29b: 0xc0, 0x29c: 0xc1, 0x29d: 0xc2, 0x29e: 0xc3, 0x29f: 0xc4,
+	0x2a0: 0xbe, 0x2a1: 0xbf, 0x2a2: 0xc0, 0x2a3: 0xc1, 0x2a4: 0xc2, 0x2a5: 0xc3, 0x2a6: 0xc4, 0x2a7: 0xbe,
+	0x2a8: 0xbf, 0x2a9: 0xc0, 0x2aa: 0xc1, 0x2ab: 0xc2, 0x2ac: 0xc3, 0x2ad: 0xc4, 0x2ae: 0xbe, 0x2af: 0xbf,
+	0x2b0: 0xc0, 0x2b1: 0xc1, 0x2b2: 0xc2, 0x2b3: 0xc3, 0x2b4: 0xc4, 0x2b5: 0xbe, 0x2b6: 0xbf, 0x2b7: 0xc0,
+	0x2b8: 0xc1, 0x2b9: 0xc2, 0x2ba: 0xc3, 0x2bb: 0xc4, 0x2bc: 0xbe, 0x2bd: 0xbf, 0x2be: 0xc0, 0x2bf: 0xc1,
+	// Block 0xb, offset 0x2c0
+	0x2c0: 0xc2, 0x2c1: 0xc3, 0x2c2: 0xc4, 0x2c3: 0xbe, 0x2c4: 0xbf, 0x2c5: 0xc0, 0x2c6: 0xc1, 0x2c7: 0xc2,
+	0x2c8: 0xc3, 0x2c9: 0xc4, 0x2ca: 0xbe, 0x2cb: 0xbf, 0x2cc: 0xc0, 0x2cd: 0xc1, 0x2ce: 0xc2, 0x2cf: 0xc3,
+	0x2d0: 0xc4, 0x2d1: 0xbe, 0x2d2: 0xbf, 0x2d3: 0xc0, 0x2d4: 0xc1, 0x2d5: 0xc2, 0x2d6: 0xc3, 0x2d7: 0xc4,
+	0x2d8: 0xbe, 0x2d9: 0xbf, 0x2da: 0xc0, 0x2db: 0xc1, 0x2dc: 0xc2, 0x2dd: 0xc3, 0x2de: 0xc5,
+	// Block 0xc, offset 0x300
+	0x324: 0x31, 0x325: 0x32, 0x326: 0x33, 0x327: 0x34,
+	0x328: 0x35, 0x329: 0x36, 0x32a: 0x37, 0x32b: 0x38, 0x32c: 0x39, 0x32d: 0x3a, 0x32e: 0x3b, 0x32f: 0x3c,
+	0x330: 0x3d, 0x331: 0x3e, 0x332: 0x3f, 0x333: 0x40, 0x334: 0x41, 0x335: 0x42, 0x336: 0x43, 0x337: 0x44,
+	0x338: 0x45, 0x339: 0x46, 0x33a: 0x47, 0x33b: 0x48, 0x33c: 0xc6, 0x33d: 0x49, 0x33e: 0x4a, 0x33f: 0x4b,
+	// Block 0xd, offset 0x340
+	0x347: 0xc7,
+	0x34b: 0xc8, 0x34d: 0xc9,
+	0x368: 0xca, 0x36b: 0xcb,
+	0x374: 0xcc,
+	0x37a: 0xcd, 0x37d: 0xce,
+	// Block 0xe, offset 0x380
+	0x381: 0xcf, 0x382: 0xd0, 0x384: 0xd1, 0x385: 0xb8, 0x387: 0xd2,
+	0x388: 0xd3, 0x38b: 0xd4, 0x38c: 0xd5, 0x38d: 0xd6,
+	0x391: 0xd7, 0x392: 0xd8, 0x393: 0xd9, 0x396: 0xda, 0x397: 0xdb,
+	0x398: 0xdc, 0x39a: 0xdd, 0x39c: 0xde,
+	0x3a0: 0xdf, 0x3a4: 0xe0, 0x3a5: 0xe1, 0x3a7: 0xe2,
+	0x3a8: 0xe3, 0x3a9: 0xe4, 0x3aa: 0xe5,
+	0x3b0: 0xdc, 0x3b5: 0xe6, 0x3b6: 0xe7,
+	// Block 0xf, offset 0x3c0
+	0x3eb: 0xe8, 0x3ec: 0xe9,
+	0x3ff: 0xea,
+	// Block 0x10, offset 0x400
+	0x432: 0xeb,
+	// Block 0x11, offset 0x440
+	0x445: 0xec, 0x446: 0xed, 0x447: 0xee,
+	0x449: 0xef,
+	0x450: 0xf0, 0x451: 0xf1, 0x452: 0xf2, 0x453: 0xf3, 0x454: 0xf4, 0x455: 0xf5, 0x456: 0xf6, 0x457: 0xf7,
+	0x458: 0xf8, 0x459: 0xf9, 0x45a: 0x4c, 0x45b: 0xfa, 0x45c: 0xfb, 0x45d: 0xfc, 0x45e: 0xfd, 0x45f: 0x4d,
+	// Block 0x12, offset 0x480
+	0x480: 0xfe, 0x484: 0xe9,
+	0x48b: 0xff,
+	0x4a3: 0x100, 0x4a5: 0x101,
+	0x4b8: 0x4e, 0x4b9: 0x4f, 0x4ba: 0x50,
+	// Block 0x13, offset 0x4c0
+	0x4c4: 0x51, 0x4c5: 0x102, 0x4c6: 0x103,
+	0x4c8: 0x52, 0x4c9: 0x104,
+	0x4ef: 0x105,
+	// Block 0x14, offset 0x500
+	0x520: 0x53, 0x521: 0x54, 0x522: 0x55, 0x523: 0x56, 0x524: 0x57, 0x525: 0x58, 0x526: 0x59, 0x527: 0x5a,
+	0x528: 0x5b,
+	// Block 0x15, offset 0x540
+	0x550: 0x0b, 0x551: 0x0c, 0x556: 0x0d,
+	0x55b: 0x0e, 0x55d: 0x0f, 0x55e: 0x10, 0x55f: 0x11,
+	0x56f: 0x12,
+}
+
+// nfkcSparseOffset: 170 entries, 340 bytes
+var nfkcSparseOffset = []uint16{0x0, 0xe, 0x12, 0x1b, 0x25, 0x35, 0x37, 0x3c, 0x47, 0x56, 0x63, 0x6b, 0x70, 0x75, 0x77, 0x7f, 0x86, 0x89, 0x91, 0x95, 0x99, 0x9b, 0x9d, 0xa6, 0xaa, 0xb1, 0xb6, 0xb9, 0xc3, 0xc6, 0xcd, 0xd5, 0xd9, 0xdb, 0xdf, 0xe3, 0xe9, 0xfa, 0x106, 0x108, 0x10e, 0x110, 0x112, 0x114, 0x116, 0x118, 0x11a, 0x11c, 0x11f, 0x122, 0x124, 0x127, 0x12a, 0x12e, 0x134, 0x136, 0x13f, 0x141, 0x144, 0x146, 0x151, 0x15c, 0x16a, 0x178, 0x188, 0x196, 0x19d, 0x1a3, 0x1b2, 0x1b6, 0x1b8, 0x1bc, 0x1be, 0x1c1, 0x1c3, 0x1c6, 0x1c8, 0x1cb, 0x1cd, 0x1cf, 0x1d1, 0x1dd, 0x1e7, 0x1f1, 0x1f4, 0x1f8, 0x1fa, 0x1fc, 0x1fe, 0x201, 0x204, 0x206, 0x208, 0x20a, 0x20c, 0x212, 0x215, 0x21a, 0x21c, 0x223, 0x229, 0x22f, 0x237, 0x23d, 0x243, 0x249, 0x24d, 0x24f, 0x251, 0x253, 0x255, 0x25b, 0x25e, 0x260, 0x262, 0x268, 0x26b, 0x273, 0x27a, 0x27d, 0x280, 0x282, 0x285, 0x28d, 0x291, 0x298, 0x29b, 0x2a1, 0x2a3, 0x2a5, 0x2a8, 0x2aa, 0x2ad, 0x2b2, 0x2b4, 0x2b6, 0x2b8, 0x2ba, 0x2bc, 0x2bf, 0x2c1, 0x2c3, 0x2c5, 0x2c7, 0x2c9, 0x2d6, 0x2e0, 0x2e2, 0x2e4, 0x2e8, 0x2ed, 0x2f9, 0x2fe, 0x307, 0x30d, 0x312, 0x316, 0x31b, 0x31f, 0x32f, 0x33d, 0x34b, 0x359, 0x35f, 0x361, 0x363, 0x366, 0x371, 0x373, 0x37d}
+
+// nfkcSparseValues: 895 entries, 3580 bytes
+var nfkcSparseValues = [895]valueRange{
+	// Block 0x0, offset 0x0
+	{value: 0x0002, lo: 0x0d},
+	{value: 0x0001, lo: 0xa0, hi: 0xa0},
+	{value: 0x428f, lo: 0xa8, hi: 0xa8},
+	{value: 0x0083, lo: 0xaa, hi: 0xaa},
+	{value: 0x427b, lo: 0xaf, hi: 0xaf},
+	{value: 0x0025, lo: 0xb2, hi: 0xb3},
+	{value: 0x4271, lo: 0xb4, hi: 0xb4},
+	{value: 0x01df, lo: 0xb5, hi: 0xb5},
+	{value: 0x42a8, lo: 0xb8, hi: 0xb8},
+	{value: 0x0023, lo: 0xb9, hi: 0xb9},
+	{value: 0x009f, lo: 0xba, hi: 0xba},
+	{value: 0x2222, lo: 0xbc, hi: 0xbc},
+	{value: 0x2216, lo: 0xbd, hi: 0xbd},
+	{value: 0x22b8, lo: 0xbe, hi: 0xbe},
+	// Block 0x1, offset 0xe
+	{value: 0x0091, lo: 0x03},
+	{value: 0x46f9, lo: 0xa0, hi: 0xa1},
+	{value: 0x472b, lo: 0xaf, hi: 0xb0},
+	{value: 0xa000, lo: 0xb7, hi: 0xb7},
+	// Block 0x2, offset 0x12
+	{value: 0x0003, lo: 0x08},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x0091, lo: 0xb0, hi: 0xb0},
+	{value: 0x0119, lo: 0xb1, hi: 0xb1},
+	{value: 0x0095, lo: 0xb2, hi: 0xb2},
+	{value: 0x00a5, lo: 0xb3, hi: 0xb3},
+	{value: 0x0143, lo: 0xb4, hi: 0xb6},
+	{value: 0x00af, lo: 0xb7, hi: 0xb7},
+	{value: 0x00b3, lo: 0xb8, hi: 0xb8},
+	// Block 0x3, offset 0x1b
+	{value: 0x000a, lo: 0x09},
+	{value: 0x4285, lo: 0x98, hi: 0x98},
+	{value: 0x428a, lo: 0x99, hi: 0x9a},
+	{value: 0x42ad, lo: 0x9b, hi: 0x9b},
+	{value: 0x4276, lo: 0x9c, hi: 0x9c},
+	{value: 0x4299, lo: 0x9d, hi: 0x9d},
+	{value: 0x0113, lo: 0xa0, hi: 0xa0},
+	{value: 0x0099, lo: 0xa1, hi: 0xa1},
+	{value: 0x00a7, lo: 0xa2, hi: 0xa3},
+	{value: 0x016a, lo: 0xa4, hi: 0xa4},
+	// Block 0x4, offset 0x25
+	{value: 0x0000, lo: 0x0f},
+	{value: 0xa000, lo: 0x83, hi: 0x83},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0xa000, lo: 0x8b, hi: 0x8b},
+	{value: 0xa000, lo: 0x8d, hi: 0x8d},
+	{value: 0x37bc, lo: 0x90, hi: 0x90},
+	{value: 0x37c8, lo: 0x91, hi: 0x91},
+	{value: 0x37b6, lo: 0x93, hi: 0x93},
+	{value: 0xa000, lo: 0x96, hi: 0x96},
+	{value: 0x382e, lo: 0x97, hi: 0x97},
+	{value: 0x37f8, lo: 0x9c, hi: 0x9c},
+	{value: 0x37e0, lo: 0x9d, hi: 0x9d},
+	{value: 0x380a, lo: 0x9e, hi: 0x9e},
+	{value: 0xa000, lo: 0xb4, hi: 0xb5},
+	{value: 0x3834, lo: 0xb6, hi: 0xb6},
+	{value: 0x383a, lo: 0xb7, hi: 0xb7},
+	// Block 0x5, offset 0x35
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x83, hi: 0x87},
+	// Block 0x6, offset 0x37
+	{value: 0x0001, lo: 0x04},
+	{value: 0x8114, lo: 0x81, hi: 0x82},
+	{value: 0x8133, lo: 0x84, hi: 0x84},
+	{value: 0x812e, lo: 0x85, hi: 0x85},
+	{value: 0x810e, lo: 0x87, hi: 0x87},
+	// Block 0x7, offset 0x3c
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x8133, lo: 0x90, hi: 0x97},
+	{value: 0x811a, lo: 0x98, hi: 0x98},
+	{value: 0x811b, lo: 0x99, hi: 0x99},
+	{value: 0x811c, lo: 0x9a, hi: 0x9a},
+	{value: 0x3858, lo: 0xa2, hi: 0xa2},
+	{value: 0x385e, lo: 0xa3, hi: 0xa3},
+	{value: 0x386a, lo: 0xa4, hi: 0xa4},
+	{value: 0x3864, lo: 0xa5, hi: 0xa5},
+	{value: 0x3870, lo: 0xa6, hi: 0xa6},
+	{value: 0xa000, lo: 0xa7, hi: 0xa7},
+	// Block 0x8, offset 0x47
+	{value: 0x0000, lo: 0x0e},
+	{value: 0x3882, lo: 0x80, hi: 0x80},
+	{value: 0xa000, lo: 0x81, hi: 0x81},
+	{value: 0x3876, lo: 0x82, hi: 0x82},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x387c, lo: 0x93, hi: 0x93},
+	{value: 0xa000, lo: 0x95, hi: 0x95},
+	{value: 0x8133, lo: 0x96, hi: 0x9c},
+	{value: 0x8133, lo: 0x9f, hi: 0xa2},
+	{value: 0x812e, lo: 0xa3, hi: 0xa3},
+	{value: 0x8133, lo: 0xa4, hi: 0xa4},
+	{value: 0x8133, lo: 0xa7, hi: 0xa8},
+	{value: 0x812e, lo: 0xaa, hi: 0xaa},
+	{value: 0x8133, lo: 0xab, hi: 0xac},
+	{value: 0x812e, lo: 0xad, hi: 0xad},
+	// Block 0x9, offset 0x56
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x8120, lo: 0x91, hi: 0x91},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	{value: 0x812e, lo: 0xb1, hi: 0xb1},
+	{value: 0x8133, lo: 0xb2, hi: 0xb3},
+	{value: 0x812e, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb5, hi: 0xb6},
+	{value: 0x812e, lo: 0xb7, hi: 0xb9},
+	{value: 0x8133, lo: 0xba, hi: 0xba},
+	{value: 0x812e, lo: 0xbb, hi: 0xbc},
+	{value: 0x8133, lo: 0xbd, hi: 0xbd},
+	{value: 0x812e, lo: 0xbe, hi: 0xbe},
+	{value: 0x8133, lo: 0xbf, hi: 0xbf},
+	// Block 0xa, offset 0x63
+	{value: 0x0005, lo: 0x07},
+	{value: 0x8133, lo: 0x80, hi: 0x80},
+	{value: 0x8133, lo: 0x81, hi: 0x81},
+	{value: 0x812e, lo: 0x82, hi: 0x83},
+	{value: 0x812e, lo: 0x84, hi: 0x85},
+	{value: 0x812e, lo: 0x86, hi: 0x87},
+	{value: 0x812e, lo: 0x88, hi: 0x89},
+	{value: 0x8133, lo: 0x8a, hi: 0x8a},
+	// Block 0xb, offset 0x6b
+	{value: 0x0000, lo: 0x04},
+	{value: 0x8133, lo: 0xab, hi: 0xb1},
+	{value: 0x812e, lo: 0xb2, hi: 0xb2},
+	{value: 0x8133, lo: 0xb3, hi: 0xb3},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	// Block 0xc, offset 0x70
+	{value: 0x0000, lo: 0x04},
+	{value: 0x8133, lo: 0x96, hi: 0x99},
+	{value: 0x8133, lo: 0x9b, hi: 0xa3},
+	{value: 0x8133, lo: 0xa5, hi: 0xa7},
+	{value: 0x8133, lo: 0xa9, hi: 0xad},
+	// Block 0xd, offset 0x75
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x99, hi: 0x9b},
+	// Block 0xe, offset 0x77
+	{value: 0x0000, lo: 0x07},
+	{value: 0xa000, lo: 0xa8, hi: 0xa8},
+	{value: 0x3eef, lo: 0xa9, hi: 0xa9},
+	{value: 0xa000, lo: 0xb0, hi: 0xb0},
+	{value: 0x3ef7, lo: 0xb1, hi: 0xb1},
+	{value: 0xa000, lo: 0xb3, hi: 0xb3},
+	{value: 0x3eff, lo: 0xb4, hi: 0xb4},
+	{value: 0x9903, lo: 0xbc, hi: 0xbc},
+	// Block 0xf, offset 0x7f
+	{value: 0x0008, lo: 0x06},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x8133, lo: 0x91, hi: 0x91},
+	{value: 0x812e, lo: 0x92, hi: 0x92},
+	{value: 0x8133, lo: 0x93, hi: 0x93},
+	{value: 0x8133, lo: 0x94, hi: 0x94},
+	{value: 0x4533, lo: 0x98, hi: 0x9f},
+	// Block 0x10, offset 0x86
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x11, offset 0x89
+	{value: 0x0008, lo: 0x07},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2cab, lo: 0x8b, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	{value: 0x4573, lo: 0x9c, hi: 0x9d},
+	{value: 0x4583, lo: 0x9f, hi: 0x9f},
+	{value: 0x8133, lo: 0xbe, hi: 0xbe},
+	// Block 0x12, offset 0x91
+	{value: 0x0000, lo: 0x03},
+	{value: 0x45ab, lo: 0xb3, hi: 0xb3},
+	{value: 0x45b3, lo: 0xb6, hi: 0xb6},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	// Block 0x13, offset 0x95
+	{value: 0x0008, lo: 0x03},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x458b, lo: 0x99, hi: 0x9b},
+	{value: 0x45a3, lo: 0x9e, hi: 0x9e},
+	// Block 0x14, offset 0x99
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	// Block 0x15, offset 0x9b
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	// Block 0x16, offset 0x9d
+	{value: 0x0000, lo: 0x08},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2cc3, lo: 0x88, hi: 0x88},
+	{value: 0x2cbb, lo: 0x8b, hi: 0x8b},
+	{value: 0x2ccb, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x96, hi: 0x97},
+	{value: 0x45bb, lo: 0x9c, hi: 0x9c},
+	{value: 0x45c3, lo: 0x9d, hi: 0x9d},
+	// Block 0x17, offset 0xa6
+	{value: 0x0000, lo: 0x03},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0x2cd3, lo: 0x94, hi: 0x94},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x18, offset 0xaa
+	{value: 0x0000, lo: 0x06},
+	{value: 0xa000, lo: 0x86, hi: 0x87},
+	{value: 0x2cdb, lo: 0x8a, hi: 0x8a},
+	{value: 0x2ceb, lo: 0x8b, hi: 0x8b},
+	{value: 0x2ce3, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	// Block 0x19, offset 0xb1
+	{value: 0x1801, lo: 0x04},
+	{value: 0xa000, lo: 0x86, hi: 0x86},
+	{value: 0x3f07, lo: 0x88, hi: 0x88},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x8121, lo: 0x95, hi: 0x96},
+	// Block 0x1a, offset 0xb6
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbc, hi: 0xbc},
+	{value: 0xa000, lo: 0xbf, hi: 0xbf},
+	// Block 0x1b, offset 0xb9
+	{value: 0x0000, lo: 0x09},
+	{value: 0x2cf3, lo: 0x80, hi: 0x80},
+	{value: 0x9900, lo: 0x82, hi: 0x82},
+	{value: 0xa000, lo: 0x86, hi: 0x86},
+	{value: 0x2cfb, lo: 0x87, hi: 0x87},
+	{value: 0x2d03, lo: 0x88, hi: 0x88},
+	{value: 0x2f67, lo: 0x8a, hi: 0x8a},
+	{value: 0x2def, lo: 0x8b, hi: 0x8b},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x95, hi: 0x96},
+	// Block 0x1c, offset 0xc3
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xbb, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x1d, offset 0xc6
+	{value: 0x0000, lo: 0x06},
+	{value: 0xa000, lo: 0x86, hi: 0x87},
+	{value: 0x2d0b, lo: 0x8a, hi: 0x8a},
+	{value: 0x2d1b, lo: 0x8b, hi: 0x8b},
+	{value: 0x2d13, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	// Block 0x1e, offset 0xcd
+	{value: 0x6bdd, lo: 0x07},
+	{value: 0x9905, lo: 0x8a, hi: 0x8a},
+	{value: 0x9900, lo: 0x8f, hi: 0x8f},
+	{value: 0xa000, lo: 0x99, hi: 0x99},
+	{value: 0x3f0f, lo: 0x9a, hi: 0x9a},
+	{value: 0x2f6f, lo: 0x9c, hi: 0x9c},
+	{value: 0x2dfa, lo: 0x9d, hi: 0x9d},
+	{value: 0x2d23, lo: 0x9e, hi: 0x9f},
+	// Block 0x1f, offset 0xd5
+	{value: 0x0000, lo: 0x03},
+	{value: 0x2627, lo: 0xb3, hi: 0xb3},
+	{value: 0x8123, lo: 0xb8, hi: 0xb9},
+	{value: 0x8105, lo: 0xba, hi: 0xba},
+	// Block 0x20, offset 0xd9
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8124, lo: 0x88, hi: 0x8b},
+	// Block 0x21, offset 0xdb
+	{value: 0x0000, lo: 0x03},
+	{value: 0x263c, lo: 0xb3, hi: 0xb3},
+	{value: 0x8125, lo: 0xb8, hi: 0xb9},
+	{value: 0x8105, lo: 0xba, hi: 0xba},
+	// Block 0x22, offset 0xdf
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8126, lo: 0x88, hi: 0x8b},
+	{value: 0x262e, lo: 0x9c, hi: 0x9c},
+	{value: 0x2635, lo: 0x9d, hi: 0x9d},
+	// Block 0x23, offset 0xe3
+	{value: 0x0000, lo: 0x05},
+	{value: 0x030e, lo: 0x8c, hi: 0x8c},
+	{value: 0x812e, lo: 0x98, hi: 0x99},
+	{value: 0x812e, lo: 0xb5, hi: 0xb5},
+	{value: 0x812e, lo: 0xb7, hi: 0xb7},
+	{value: 0x812c, lo: 0xb9, hi: 0xb9},
+	// Block 0x24, offset 0xe9
+	{value: 0x0000, lo: 0x10},
+	{value: 0x264a, lo: 0x83, hi: 0x83},
+	{value: 0x2651, lo: 0x8d, hi: 0x8d},
+	{value: 0x2658, lo: 0x92, hi: 0x92},
+	{value: 0x265f, lo: 0x97, hi: 0x97},
+	{value: 0x2666, lo: 0x9c, hi: 0x9c},
+	{value: 0x2643, lo: 0xa9, hi: 0xa9},
+	{value: 0x8127, lo: 0xb1, hi: 0xb1},
+	{value: 0x8128, lo: 0xb2, hi: 0xb2},
+	{value: 0x4a9b, lo: 0xb3, hi: 0xb3},
+	{value: 0x8129, lo: 0xb4, hi: 0xb4},
+	{value: 0x4aa4, lo: 0xb5, hi: 0xb5},
+	{value: 0x45cb, lo: 0xb6, hi: 0xb6},
+	{value: 0x460b, lo: 0xb7, hi: 0xb7},
+	{value: 0x45d3, lo: 0xb8, hi: 0xb8},
+	{value: 0x4616, lo: 0xb9, hi: 0xb9},
+	{value: 0x8128, lo: 0xba, hi: 0xbd},
+	// Block 0x25, offset 0xfa
+	{value: 0x0000, lo: 0x0b},
+	{value: 0x8128, lo: 0x80, hi: 0x80},
+	{value: 0x4aad, lo: 0x81, hi: 0x81},
+	{value: 0x8133, lo: 0x82, hi: 0x83},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0x86, hi: 0x87},
+	{value: 0x2674, lo: 0x93, hi: 0x93},
+	{value: 0x267b, lo: 0x9d, hi: 0x9d},
+	{value: 0x2682, lo: 0xa2, hi: 0xa2},
+	{value: 0x2689, lo: 0xa7, hi: 0xa7},
+	{value: 0x2690, lo: 0xac, hi: 0xac},
+	{value: 0x266d, lo: 0xb9, hi: 0xb9},
+	// Block 0x26, offset 0x106
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x86, hi: 0x86},
+	// Block 0x27, offset 0x108
+	{value: 0x0000, lo: 0x05},
+	{value: 0xa000, lo: 0xa5, hi: 0xa5},
+	{value: 0x2d2b, lo: 0xa6, hi: 0xa6},
+	{value: 0x9900, lo: 0xae, hi: 0xae},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	{value: 0x8105, lo: 0xb9, hi: 0xba},
+	// Block 0x28, offset 0x10e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x8d, hi: 0x8d},
+	// Block 0x29, offset 0x110
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0312, lo: 0xbc, hi: 0xbc},
+	// Block 0x2a, offset 0x112
+	{value: 0x0000, lo: 0x01},
+	{value: 0xa000, lo: 0x80, hi: 0x92},
+	// Block 0x2b, offset 0x114
+	{value: 0x0000, lo: 0x01},
+	{value: 0xb900, lo: 0xa1, hi: 0xb5},
+	// Block 0x2c, offset 0x116
+	{value: 0x0000, lo: 0x01},
+	{value: 0x9900, lo: 0xa8, hi: 0xbf},
+	// Block 0x2d, offset 0x118
+	{value: 0x0000, lo: 0x01},
+	{value: 0x9900, lo: 0x80, hi: 0x82},
+	// Block 0x2e, offset 0x11a
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x9d, hi: 0x9f},
+	// Block 0x2f, offset 0x11c
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x94, hi: 0x94},
+	{value: 0x8105, lo: 0xb4, hi: 0xb4},
+	// Block 0x30, offset 0x11f
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x92, hi: 0x92},
+	{value: 0x8133, lo: 0x9d, hi: 0x9d},
+	// Block 0x31, offset 0x122
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8132, lo: 0xa9, hi: 0xa9},
+	// Block 0x32, offset 0x124
+	{value: 0x0004, lo: 0x02},
+	{value: 0x812f, lo: 0xb9, hi: 0xba},
+	{value: 0x812e, lo: 0xbb, hi: 0xbb},
+	// Block 0x33, offset 0x127
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x97, hi: 0x97},
+	{value: 0x812e, lo: 0x98, hi: 0x98},
+	// Block 0x34, offset 0x12a
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8105, lo: 0xa0, hi: 0xa0},
+	{value: 0x8133, lo: 0xb5, hi: 0xbc},
+	{value: 0x812e, lo: 0xbf, hi: 0xbf},
+	// Block 0x35, offset 0x12e
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0xb0, hi: 0xb4},
+	{value: 0x812e, lo: 0xb5, hi: 0xba},
+	{value: 0x8133, lo: 0xbb, hi: 0xbc},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	{value: 0x812e, lo: 0xbf, hi: 0xbf},
+	// Block 0x36, offset 0x134
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x80, hi: 0x80},
+	// Block 0x37, offset 0x136
+	{value: 0x0000, lo: 0x08},
+	{value: 0x2d73, lo: 0x80, hi: 0x80},
+	{value: 0x2d7b, lo: 0x81, hi: 0x81},
+	{value: 0xa000, lo: 0x82, hi: 0x82},
+	{value: 0x2d83, lo: 0x83, hi: 0x83},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0xab, hi: 0xab},
+	{value: 0x812e, lo: 0xac, hi: 0xac},
+	{value: 0x8133, lo: 0xad, hi: 0xb3},
+	// Block 0x38, offset 0x13f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xaa, hi: 0xab},
+	// Block 0x39, offset 0x141
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xa6, hi: 0xa6},
+	{value: 0x8105, lo: 0xb2, hi: 0xb3},
+	// Block 0x3a, offset 0x144
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	// Block 0x3b, offset 0x146
+	{value: 0x0000, lo: 0x0a},
+	{value: 0x8133, lo: 0x90, hi: 0x92},
+	{value: 0x8101, lo: 0x94, hi: 0x94},
+	{value: 0x812e, lo: 0x95, hi: 0x99},
+	{value: 0x8133, lo: 0x9a, hi: 0x9b},
+	{value: 0x812e, lo: 0x9c, hi: 0x9f},
+	{value: 0x8133, lo: 0xa0, hi: 0xa0},
+	{value: 0x8101, lo: 0xa2, hi: 0xa8},
+	{value: 0x812e, lo: 0xad, hi: 0xad},
+	{value: 0x8133, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb8, hi: 0xb9},
+	// Block 0x3c, offset 0x151
+	{value: 0x0002, lo: 0x0a},
+	{value: 0x0043, lo: 0xac, hi: 0xac},
+	{value: 0x00d1, lo: 0xad, hi: 0xad},
+	{value: 0x0045, lo: 0xae, hi: 0xae},
+	{value: 0x0049, lo: 0xb0, hi: 0xb1},
+	{value: 0x00e6, lo: 0xb2, hi: 0xb2},
+	{value: 0x004f, lo: 0xb3, hi: 0xba},
+	{value: 0x005f, lo: 0xbc, hi: 0xbc},
+	{value: 0x00ef, lo: 0xbd, hi: 0xbd},
+	{value: 0x0061, lo: 0xbe, hi: 0xbe},
+	{value: 0x0065, lo: 0xbf, hi: 0xbf},
+	// Block 0x3d, offset 0x15c
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x0001, lo: 0x80, hi: 0x8a},
+	{value: 0x043e, lo: 0x91, hi: 0x91},
+	{value: 0x42b2, lo: 0x97, hi: 0x97},
+	{value: 0x001d, lo: 0xa4, hi: 0xa4},
+	{value: 0x1876, lo: 0xa5, hi: 0xa5},
+	{value: 0x1b62, lo: 0xa6, hi: 0xa6},
+	{value: 0x0001, lo: 0xaf, hi: 0xaf},
+	{value: 0x2697, lo: 0xb3, hi: 0xb3},
+	{value: 0x280b, lo: 0xb4, hi: 0xb4},
+	{value: 0x269e, lo: 0xb6, hi: 0xb6},
+	{value: 0x2815, lo: 0xb7, hi: 0xb7},
+	{value: 0x1870, lo: 0xbc, hi: 0xbc},
+	{value: 0x4280, lo: 0xbe, hi: 0xbe},
+	// Block 0x3e, offset 0x16a
+	{value: 0x0002, lo: 0x0d},
+	{value: 0x1936, lo: 0x87, hi: 0x87},
+	{value: 0x1933, lo: 0x88, hi: 0x88},
+	{value: 0x1873, lo: 0x89, hi: 0x89},
+	{value: 0x299b, lo: 0x97, hi: 0x97},
+	{value: 0x0001, lo: 0x9f, hi: 0x9f},
+	{value: 0x0021, lo: 0xb0, hi: 0xb0},
+	{value: 0x0093, lo: 0xb1, hi: 0xb1},
+	{value: 0x0029, lo: 0xb4, hi: 0xb9},
+	{value: 0x0017, lo: 0xba, hi: 0xba},
+	{value: 0x046a, lo: 0xbb, hi: 0xbb},
+	{value: 0x003b, lo: 0xbc, hi: 0xbc},
+	{value: 0x0011, lo: 0xbd, hi: 0xbe},
+	{value: 0x009d, lo: 0xbf, hi: 0xbf},
+	// Block 0x3f, offset 0x178
+	{value: 0x0002, lo: 0x0f},
+	{value: 0x0021, lo: 0x80, hi: 0x89},
+	{value: 0x0017, lo: 0x8a, hi: 0x8a},
+	{value: 0x046a, lo: 0x8b, hi: 0x8b},
+	{value: 0x003b, lo: 0x8c, hi: 0x8c},
+	{value: 0x0011, lo: 0x8d, hi: 0x8e},
+	{value: 0x0083, lo: 0x90, hi: 0x90},
+	{value: 0x008b, lo: 0x91, hi: 0x91},
+	{value: 0x009f, lo: 0x92, hi: 0x92},
+	{value: 0x00b1, lo: 0x93, hi: 0x93},
+	{value: 0x0104, lo: 0x94, hi: 0x94},
+	{value: 0x0091, lo: 0x95, hi: 0x95},
+	{value: 0x0097, lo: 0x96, hi: 0x99},
+	{value: 0x00a1, lo: 0x9a, hi: 0x9a},
+	{value: 0x00a7, lo: 0x9b, hi: 0x9c},
+	{value: 0x199f, lo: 0xa8, hi: 0xa8},
+	// Block 0x40, offset 0x188
+	{value: 0x0000, lo: 0x0d},
+	{value: 0x8133, lo: 0x90, hi: 0x91},
+	{value: 0x8101, lo: 0x92, hi: 0x93},
+	{value: 0x8133, lo: 0x94, hi: 0x97},
+	{value: 0x8101, lo: 0x98, hi: 0x9a},
+	{value: 0x8133, lo: 0x9b, hi: 0x9c},
+	{value: 0x8133, lo: 0xa1, hi: 0xa1},
+	{value: 0x8101, lo: 0xa5, hi: 0xa6},
+	{value: 0x8133, lo: 0xa7, hi: 0xa7},
+	{value: 0x812e, lo: 0xa8, hi: 0xa8},
+	{value: 0x8133, lo: 0xa9, hi: 0xa9},
+	{value: 0x8101, lo: 0xaa, hi: 0xab},
+	{value: 0x812e, lo: 0xac, hi: 0xaf},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	// Block 0x41, offset 0x196
+	{value: 0x0007, lo: 0x06},
+	{value: 0x2186, lo: 0x89, hi: 0x89},
+	{value: 0xa000, lo: 0x90, hi: 0x90},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0xa000, lo: 0x94, hi: 0x94},
+	{value: 0x3bd0, lo: 0x9a, hi: 0x9b},
+	{value: 0x3bde, lo: 0xae, hi: 0xae},
+	// Block 0x42, offset 0x19d
+	{value: 0x000e, lo: 0x05},
+	{value: 0x3be5, lo: 0x8d, hi: 0x8e},
+	{value: 0x3bec, lo: 0x8f, hi: 0x8f},
+	{value: 0xa000, lo: 0x90, hi: 0x90},
+	{value: 0xa000, lo: 0x92, hi: 0x92},
+	{value: 0xa000, lo: 0x94, hi: 0x94},
+	// Block 0x43, offset 0x1a3
+	{value: 0x017a, lo: 0x0e},
+	{value: 0xa000, lo: 0x83, hi: 0x83},
+	{value: 0x3bfa, lo: 0x84, hi: 0x84},
+	{value: 0xa000, lo: 0x88, hi: 0x88},
+	{value: 0x3c01, lo: 0x89, hi: 0x89},
+	{value: 0xa000, lo: 0x8b, hi: 0x8b},
+	{value: 0x3c08, lo: 0x8c, hi: 0x8c},
+	{value: 0xa000, lo: 0xa3, hi: 0xa3},
+	{value: 0x3c0f, lo: 0xa4, hi: 0xa4},
+	{value: 0xa000, lo: 0xa5, hi: 0xa5},
+	{value: 0x3c16, lo: 0xa6, hi: 0xa6},
+	{value: 0x26a5, lo: 0xac, hi: 0xad},
+	{value: 0x26ac, lo: 0xaf, hi: 0xaf},
+	{value: 0x2829, lo: 0xb0, hi: 0xb0},
+	{value: 0xa000, lo: 0xbc, hi: 0xbc},
+	// Block 0x44, offset 0x1b2
+	{value: 0x0007, lo: 0x03},
+	{value: 0x3c7f, lo: 0xa0, hi: 0xa1},
+	{value: 0x3ca9, lo: 0xa2, hi: 0xa3},
+	{value: 0x3cd3, lo: 0xaa, hi: 0xad},
+	// Block 0x45, offset 0x1b6
+	{value: 0x0004, lo: 0x01},
+	{value: 0x048e, lo: 0xa9, hi: 0xaa},
+	// Block 0x46, offset 0x1b8
+	{value: 0x0002, lo: 0x03},
+	{value: 0x0057, lo: 0x80, hi: 0x8f},
+	{value: 0x0083, lo: 0x90, hi: 0xa9},
+	{value: 0x0021, lo: 0xaa, hi: 0xaa},
+	// Block 0x47, offset 0x1bc
+	{value: 0x0000, lo: 0x01},
+	{value: 0x29a8, lo: 0x8c, hi: 0x8c},
+	// Block 0x48, offset 0x1be
+	{value: 0x0266, lo: 0x02},
+	{value: 0x1b92, lo: 0xb4, hi: 0xb4},
+	{value: 0x1930, lo: 0xb5, hi: 0xb6},
+	// Block 0x49, offset 0x1c1
+	{value: 0x0000, lo: 0x01},
+	{value: 0x44f4, lo: 0x9c, hi: 0x9c},
+	// Block 0x4a, offset 0x1c3
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0095, lo: 0xbc, hi: 0xbc},
+	{value: 0x006d, lo: 0xbd, hi: 0xbd},
+	// Block 0x4b, offset 0x1c6
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xaf, hi: 0xb1},
+	// Block 0x4c, offset 0x1c8
+	{value: 0x0000, lo: 0x02},
+	{value: 0x0482, lo: 0xaf, hi: 0xaf},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x4d, offset 0x1cb
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xa0, hi: 0xbf},
+	// Block 0x4e, offset 0x1cd
+	{value: 0x0000, lo: 0x01},
+	{value: 0x0dc6, lo: 0x9f, hi: 0x9f},
+	// Block 0x4f, offset 0x1cf
+	{value: 0x0000, lo: 0x01},
+	{value: 0x1632, lo: 0xb3, hi: 0xb3},
+	// Block 0x50, offset 0x1d1
+	{value: 0x0004, lo: 0x0b},
+	{value: 0x159a, lo: 0x80, hi: 0x82},
+	{value: 0x15b2, lo: 0x83, hi: 0x83},
+	{value: 0x15ca, lo: 0x84, hi: 0x85},
+	{value: 0x15da, lo: 0x86, hi: 0x89},
+	{value: 0x15ee, lo: 0x8a, hi: 0x8c},
+	{value: 0x1602, lo: 0x8d, hi: 0x8d},
+	{value: 0x160a, lo: 0x8e, hi: 0x8e},
+	{value: 0x1612, lo: 0x8f, hi: 0x90},
+	{value: 0x161e, lo: 0x91, hi: 0x93},
+	{value: 0x162e, lo: 0x94, hi: 0x94},
+	{value: 0x1636, lo: 0x95, hi: 0x95},
+	// Block 0x51, offset 0x1dd
+	{value: 0x0004, lo: 0x09},
+	{value: 0x0001, lo: 0x80, hi: 0x80},
+	{value: 0x812d, lo: 0xaa, hi: 0xaa},
+	{value: 0x8132, lo: 0xab, hi: 0xab},
+	{value: 0x8134, lo: 0xac, hi: 0xac},
+	{value: 0x812f, lo: 0xad, hi: 0xad},
+	{value: 0x8130, lo: 0xae, hi: 0xae},
+	{value: 0x8130, lo: 0xaf, hi: 0xaf},
+	{value: 0x04b6, lo: 0xb6, hi: 0xb6},
+	{value: 0x088a, lo: 0xb8, hi: 0xba},
+	// Block 0x52, offset 0x1e7
+	{value: 0x0006, lo: 0x09},
+	{value: 0x0316, lo: 0xb1, hi: 0xb1},
+	{value: 0x031a, lo: 0xb2, hi: 0xb2},
+	{value: 0x4a52, lo: 0xb3, hi: 0xb3},
+	{value: 0x031e, lo: 0xb4, hi: 0xb4},
+	{value: 0x4a58, lo: 0xb5, hi: 0xb6},
+	{value: 0x0322, lo: 0xb7, hi: 0xb7},
+	{value: 0x0326, lo: 0xb8, hi: 0xb8},
+	{value: 0x032a, lo: 0xb9, hi: 0xb9},
+	{value: 0x4a64, lo: 0xba, hi: 0xbf},
+	// Block 0x53, offset 0x1f1
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0xaf, hi: 0xaf},
+	{value: 0x8133, lo: 0xb4, hi: 0xbd},
+	// Block 0x54, offset 0x1f4
+	{value: 0x0000, lo: 0x03},
+	{value: 0x0212, lo: 0x9c, hi: 0x9c},
+	{value: 0x0215, lo: 0x9d, hi: 0x9d},
+	{value: 0x8133, lo: 0x9e, hi: 0x9f},
+	// Block 0x55, offset 0x1f8
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb0, hi: 0xb1},
+	// Block 0x56, offset 0x1fa
+	{value: 0x0000, lo: 0x01},
+	{value: 0x163e, lo: 0xb0, hi: 0xb0},
+	// Block 0x57, offset 0x1fc
+	{value: 0x000c, lo: 0x01},
+	{value: 0x00d7, lo: 0xb8, hi: 0xb9},
+	// Block 0x58, offset 0x1fe
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x86, hi: 0x86},
+	{value: 0x8105, lo: 0xac, hi: 0xac},
+	// Block 0x59, offset 0x201
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x84, hi: 0x84},
+	{value: 0x8133, lo: 0xa0, hi: 0xb1},
+	// Block 0x5a, offset 0x204
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xab, hi: 0xad},
+	// Block 0x5b, offset 0x206
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x93, hi: 0x93},
+	// Block 0x5c, offset 0x208
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0xb3, hi: 0xb3},
+	// Block 0x5d, offset 0x20a
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x80, hi: 0x80},
+	// Block 0x5e, offset 0x20c
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0xb0, hi: 0xb0},
+	{value: 0x8133, lo: 0xb2, hi: 0xb3},
+	{value: 0x812e, lo: 0xb4, hi: 0xb4},
+	{value: 0x8133, lo: 0xb7, hi: 0xb8},
+	{value: 0x8133, lo: 0xbe, hi: 0xbf},
+	// Block 0x5f, offset 0x212
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x81, hi: 0x81},
+	{value: 0x8105, lo: 0xb6, hi: 0xb6},
+	// Block 0x60, offset 0x215
+	{value: 0x0008, lo: 0x04},
+	{value: 0x163a, lo: 0x9c, hi: 0x9d},
+	{value: 0x0125, lo: 0x9e, hi: 0x9e},
+	{value: 0x1646, lo: 0x9f, hi: 0x9f},
+	{value: 0x015e, lo: 0xa9, hi: 0xa9},
+	// Block 0x61, offset 0x21a
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xad, hi: 0xad},
+	// Block 0x62, offset 0x21c
+	{value: 0x0000, lo: 0x06},
+	{value: 0xe500, lo: 0x80, hi: 0x80},
+	{value: 0xc600, lo: 0x81, hi: 0x9b},
+	{value: 0xe500, lo: 0x9c, hi: 0x9c},
+	{value: 0xc600, lo: 0x9d, hi: 0xb7},
+	{value: 0xe500, lo: 0xb8, hi: 0xb8},
+	{value: 0xc600, lo: 0xb9, hi: 0xbf},
+	// Block 0x63, offset 0x223
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x93},
+	{value: 0xe500, lo: 0x94, hi: 0x94},
+	{value: 0xc600, lo: 0x95, hi: 0xaf},
+	{value: 0xe500, lo: 0xb0, hi: 0xb0},
+	{value: 0xc600, lo: 0xb1, hi: 0xbf},
+	// Block 0x64, offset 0x229
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x8b},
+	{value: 0xe500, lo: 0x8c, hi: 0x8c},
+	{value: 0xc600, lo: 0x8d, hi: 0xa7},
+	{value: 0xe500, lo: 0xa8, hi: 0xa8},
+	{value: 0xc600, lo: 0xa9, hi: 0xbf},
+	// Block 0x65, offset 0x22f
+	{value: 0x0000, lo: 0x07},
+	{value: 0xc600, lo: 0x80, hi: 0x83},
+	{value: 0xe500, lo: 0x84, hi: 0x84},
+	{value: 0xc600, lo: 0x85, hi: 0x9f},
+	{value: 0xe500, lo: 0xa0, hi: 0xa0},
+	{value: 0xc600, lo: 0xa1, hi: 0xbb},
+	{value: 0xe500, lo: 0xbc, hi: 0xbc},
+	{value: 0xc600, lo: 0xbd, hi: 0xbf},
+	// Block 0x66, offset 0x237
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x97},
+	{value: 0xe500, lo: 0x98, hi: 0x98},
+	{value: 0xc600, lo: 0x99, hi: 0xb3},
+	{value: 0xe500, lo: 0xb4, hi: 0xb4},
+	{value: 0xc600, lo: 0xb5, hi: 0xbf},
+	// Block 0x67, offset 0x23d
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x8f},
+	{value: 0xe500, lo: 0x90, hi: 0x90},
+	{value: 0xc600, lo: 0x91, hi: 0xab},
+	{value: 0xe500, lo: 0xac, hi: 0xac},
+	{value: 0xc600, lo: 0xad, hi: 0xbf},
+	// Block 0x68, offset 0x243
+	{value: 0x0000, lo: 0x05},
+	{value: 0xc600, lo: 0x80, hi: 0x87},
+	{value: 0xe500, lo: 0x88, hi: 0x88},
+	{value: 0xc600, lo: 0x89, hi: 0xa3},
+	{value: 0xe500, lo: 0xa4, hi: 0xa4},
+	{value: 0xc600, lo: 0xa5, hi: 0xbf},
+	// Block 0x69, offset 0x249
+	{value: 0x0000, lo: 0x03},
+	{value: 0xc600, lo: 0x80, hi: 0x87},
+	{value: 0xe500, lo: 0x88, hi: 0x88},
+	{value: 0xc600, lo: 0x89, hi: 0xa3},
+	// Block 0x6a, offset 0x24d
+	{value: 0x0002, lo: 0x01},
+	{value: 0x0003, lo: 0x81, hi: 0xbf},
+	// Block 0x6b, offset 0x24f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xbd, hi: 0xbd},
+	// Block 0x6c, offset 0x251
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0xa0, hi: 0xa0},
+	// Block 0x6d, offset 0x253
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb6, hi: 0xba},
+	// Block 0x6e, offset 0x255
+	{value: 0x002d, lo: 0x05},
+	{value: 0x812e, lo: 0x8d, hi: 0x8d},
+	{value: 0x8133, lo: 0x8f, hi: 0x8f},
+	{value: 0x8133, lo: 0xb8, hi: 0xb8},
+	{value: 0x8101, lo: 0xb9, hi: 0xba},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x6f, offset 0x25b
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0xa5, hi: 0xa5},
+	{value: 0x812e, lo: 0xa6, hi: 0xa6},
+	// Block 0x70, offset 0x25e
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xa4, hi: 0xa7},
+	// Block 0x71, offset 0x260
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xab, hi: 0xac},
+	// Block 0x72, offset 0x262
+	{value: 0x0000, lo: 0x05},
+	{value: 0x812e, lo: 0x86, hi: 0x87},
+	{value: 0x8133, lo: 0x88, hi: 0x8a},
+	{value: 0x812e, lo: 0x8b, hi: 0x8b},
+	{value: 0x8133, lo: 0x8c, hi: 0x8c},
+	{value: 0x812e, lo: 0x8d, hi: 0x90},
+	// Block 0x73, offset 0x268
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x86, hi: 0x86},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x74, offset 0x26b
+	{value: 0x17fe, lo: 0x07},
+	{value: 0xa000, lo: 0x99, hi: 0x99},
+	{value: 0x424f, lo: 0x9a, hi: 0x9a},
+	{value: 0xa000, lo: 0x9b, hi: 0x9b},
+	{value: 0x4259, lo: 0x9c, hi: 0x9c},
+	{value: 0xa000, lo: 0xa5, hi: 0xa5},
+	{value: 0x4263, lo: 0xab, hi: 0xab},
+	{value: 0x8105, lo: 0xb9, hi: 0xba},
+	// Block 0x75, offset 0x273
+	{value: 0x0000, lo: 0x06},
+	{value: 0x8133, lo: 0x80, hi: 0x82},
+	{value: 0x9900, lo: 0xa7, hi: 0xa7},
+	{value: 0x2d8b, lo: 0xae, hi: 0xae},
+	{value: 0x2d95, lo: 0xaf, hi: 0xaf},
+	{value: 0xa000, lo: 0xb1, hi: 0xb2},
+	{value: 0x8105, lo: 0xb3, hi: 0xb4},
+	// Block 0x76, offset 0x27a
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x80, hi: 0x80},
+	{value: 0x8103, lo: 0x8a, hi: 0x8a},
+	// Block 0x77, offset 0x27d
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb5, hi: 0xb5},
+	{value: 0x8103, lo: 0xb6, hi: 0xb6},
+	// Block 0x78, offset 0x280
+	{value: 0x0002, lo: 0x01},
+	{value: 0x8103, lo: 0xa9, hi: 0xaa},
+	// Block 0x79, offset 0x282
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0xbb, hi: 0xbc},
+	{value: 0x9900, lo: 0xbe, hi: 0xbe},
+	// Block 0x7a, offset 0x285
+	{value: 0x0000, lo: 0x07},
+	{value: 0xa000, lo: 0x87, hi: 0x87},
+	{value: 0x2d9f, lo: 0x8b, hi: 0x8b},
+	{value: 0x2da9, lo: 0x8c, hi: 0x8c},
+	{value: 0x8105, lo: 0x8d, hi: 0x8d},
+	{value: 0x9900, lo: 0x97, hi: 0x97},
+	{value: 0x8133, lo: 0xa6, hi: 0xac},
+	{value: 0x8133, lo: 0xb0, hi: 0xb4},
+	// Block 0x7b, offset 0x28d
+	{value: 0x0000, lo: 0x03},
+	{value: 0x8105, lo: 0x82, hi: 0x82},
+	{value: 0x8103, lo: 0x86, hi: 0x86},
+	{value: 0x8133, lo: 0x9e, hi: 0x9e},
+	// Block 0x7c, offset 0x291
+	{value: 0x6b4d, lo: 0x06},
+	{value: 0x9900, lo: 0xb0, hi: 0xb0},
+	{value: 0xa000, lo: 0xb9, hi: 0xb9},
+	{value: 0x9900, lo: 0xba, hi: 0xba},
+	{value: 0x2dbd, lo: 0xbb, hi: 0xbb},
+	{value: 0x2db3, lo: 0xbc, hi: 0xbd},
+	{value: 0x2dc7, lo: 0xbe, hi: 0xbe},
+	// Block 0x7d, offset 0x298
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0x82, hi: 0x82},
+	{value: 0x8103, lo: 0x83, hi: 0x83},
+	// Block 0x7e, offset 0x29b
+	{value: 0x0000, lo: 0x05},
+	{value: 0x9900, lo: 0xaf, hi: 0xaf},
+	{value: 0xa000, lo: 0xb8, hi: 0xb9},
+	{value: 0x2dd1, lo: 0xba, hi: 0xba},
+	{value: 0x2ddb, lo: 0xbb, hi: 0xbb},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x7f, offset 0x2a1
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0x80, hi: 0x80},
+	// Block 0x80, offset 0x2a3
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xbf, hi: 0xbf},
+	// Block 0x81, offset 0x2a5
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb6, hi: 0xb6},
+	{value: 0x8103, lo: 0xb7, hi: 0xb7},
+	// Block 0x82, offset 0x2a8
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xab, hi: 0xab},
+	// Block 0x83, offset 0x2aa
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8105, lo: 0xb9, hi: 0xb9},
+	{value: 0x8103, lo: 0xba, hi: 0xba},
+	// Block 0x84, offset 0x2ad
+	{value: 0x0000, lo: 0x04},
+	{value: 0x9900, lo: 0xb0, hi: 0xb0},
+	{value: 0xa000, lo: 0xb5, hi: 0xb5},
+	{value: 0x2de5, lo: 0xb8, hi: 0xb8},
+	{value: 0x8105, lo: 0xbd, hi: 0xbe},
+	// Block 0x85, offset 0x2b2
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8103, lo: 0x83, hi: 0x83},
+	// Block 0x86, offset 0x2b4
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xa0, hi: 0xa0},
+	// Block 0x87, offset 0x2b6
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0xb4, hi: 0xb4},
+	// Block 0x88, offset 0x2b8
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x87, hi: 0x87},
+	// Block 0x89, offset 0x2ba
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x99, hi: 0x99},
+	// Block 0x8a, offset 0x2bc
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8103, lo: 0x82, hi: 0x82},
+	{value: 0x8105, lo: 0x84, hi: 0x85},
+	// Block 0x8b, offset 0x2bf
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8105, lo: 0x97, hi: 0x97},
+	// Block 0x8c, offset 0x2c1
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8101, lo: 0xb0, hi: 0xb4},
+	// Block 0x8d, offset 0x2c3
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xb0, hi: 0xb6},
+	// Block 0x8e, offset 0x2c5
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8102, lo: 0xb0, hi: 0xb1},
+	// Block 0x8f, offset 0x2c7
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8101, lo: 0x9e, hi: 0x9e},
+	// Block 0x90, offset 0x2c9
+	{value: 0x0000, lo: 0x0c},
+	{value: 0x45e3, lo: 0x9e, hi: 0x9e},
+	{value: 0x45ed, lo: 0x9f, hi: 0x9f},
+	{value: 0x4621, lo: 0xa0, hi: 0xa0},
+	{value: 0x462f, lo: 0xa1, hi: 0xa1},
+	{value: 0x463d, lo: 0xa2, hi: 0xa2},
+	{value: 0x464b, lo: 0xa3, hi: 0xa3},
+	{value: 0x4659, lo: 0xa4, hi: 0xa4},
+	{value: 0x812c, lo: 0xa5, hi: 0xa6},
+	{value: 0x8101, lo: 0xa7, hi: 0xa9},
+	{value: 0x8131, lo: 0xad, hi: 0xad},
+	{value: 0x812c, lo: 0xae, hi: 0xb2},
+	{value: 0x812e, lo: 0xbb, hi: 0xbf},
+	// Block 0x91, offset 0x2d6
+	{value: 0x0000, lo: 0x09},
+	{value: 0x812e, lo: 0x80, hi: 0x82},
+	{value: 0x8133, lo: 0x85, hi: 0x89},
+	{value: 0x812e, lo: 0x8a, hi: 0x8b},
+	{value: 0x8133, lo: 0xaa, hi: 0xad},
+	{value: 0x45f7, lo: 0xbb, hi: 0xbb},
+	{value: 0x4601, lo: 0xbc, hi: 0xbc},
+	{value: 0x4667, lo: 0xbd, hi: 0xbd},
+	{value: 0x4683, lo: 0xbe, hi: 0xbe},
+	{value: 0x4675, lo: 0xbf, hi: 0xbf},
+	// Block 0x92, offset 0x2e0
+	{value: 0x0000, lo: 0x01},
+	{value: 0x4691, lo: 0x80, hi: 0x80},
+	// Block 0x93, offset 0x2e2
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0x82, hi: 0x84},
+	// Block 0x94, offset 0x2e4
+	{value: 0x0002, lo: 0x03},
+	{value: 0x0043, lo: 0x80, hi: 0x99},
+	{value: 0x0083, lo: 0x9a, hi: 0xb3},
+	{value: 0x0043, lo: 0xb4, hi: 0xbf},
+	// Block 0x95, offset 0x2e8
+	{value: 0x0002, lo: 0x04},
+	{value: 0x005b, lo: 0x80, hi: 0x8d},
+	{value: 0x0083, lo: 0x8e, hi: 0x94},
+	{value: 0x0093, lo: 0x96, hi: 0xa7},
+	{value: 0x0043, lo: 0xa8, hi: 0xbf},
+	// Block 0x96, offset 0x2ed
+	{value: 0x0002, lo: 0x0b},
+	{value: 0x0073, lo: 0x80, hi: 0x81},
+	{value: 0x0083, lo: 0x82, hi: 0x9b},
+	{value: 0x0043, lo: 0x9c, hi: 0x9c},
+	{value: 0x0047, lo: 0x9e, hi: 0x9f},
+	{value: 0x004f, lo: 0xa2, hi: 0xa2},
+	{value: 0x0055, lo: 0xa5, hi: 0xa6},
+	{value: 0x005d, lo: 0xa9, hi: 0xac},
+	{value: 0x0067, lo: 0xae, hi: 0xb5},
+	{value: 0x0083, lo: 0xb6, hi: 0xb9},
+	{value: 0x008d, lo: 0xbb, hi: 0xbb},
+	{value: 0x0091, lo: 0xbd, hi: 0xbf},
+	// Block 0x97, offset 0x2f9
+	{value: 0x0002, lo: 0x04},
+	{value: 0x0097, lo: 0x80, hi: 0x83},
+	{value: 0x00a1, lo: 0x85, hi: 0x8f},
+	{value: 0x0043, lo: 0x90, hi: 0xa9},
+	{value: 0x0083, lo: 0xaa, hi: 0xbf},
+	// Block 0x98, offset 0x2fe
+	{value: 0x0002, lo: 0x08},
+	{value: 0x00af, lo: 0x80, hi: 0x83},
+	{value: 0x0043, lo: 0x84, hi: 0x85},
+	{value: 0x0049, lo: 0x87, hi: 0x8a},
+	{value: 0x0055, lo: 0x8d, hi: 0x94},
+	{value: 0x0067, lo: 0x96, hi: 0x9c},
+	{value: 0x0083, lo: 0x9e, hi: 0xb7},
+	{value: 0x0043, lo: 0xb8, hi: 0xb9},
+	{value: 0x0049, lo: 0xbb, hi: 0xbe},
+	// Block 0x99, offset 0x307
+	{value: 0x0002, lo: 0x05},
+	{value: 0x0053, lo: 0x80, hi: 0x84},
+	{value: 0x005f, lo: 0x86, hi: 0x86},
+	{value: 0x0067, lo: 0x8a, hi: 0x90},
+	{value: 0x0083, lo: 0x92, hi: 0xab},
+	{value: 0x0043, lo: 0xac, hi: 0xbf},
+	// Block 0x9a, offset 0x30d
+	{value: 0x0002, lo: 0x04},
+	{value: 0x006b, lo: 0x80, hi: 0x85},
+	{value: 0x0083, lo: 0x86, hi: 0x9f},
+	{value: 0x0043, lo: 0xa0, hi: 0xb9},
+	{value: 0x0083, lo: 0xba, hi: 0xbf},
+	// Block 0x9b, offset 0x312
+	{value: 0x0002, lo: 0x03},
+	{value: 0x008f, lo: 0x80, hi: 0x93},
+	{value: 0x0043, lo: 0x94, hi: 0xad},
+	{value: 0x0083, lo: 0xae, hi: 0xbf},
+	// Block 0x9c, offset 0x316
+	{value: 0x0002, lo: 0x04},
+	{value: 0x00a7, lo: 0x80, hi: 0x87},
+	{value: 0x0043, lo: 0x88, hi: 0xa1},
+	{value: 0x0083, lo: 0xa2, hi: 0xbb},
+	{value: 0x0043, lo: 0xbc, hi: 0xbf},
+	// Block 0x9d, offset 0x31b
+	{value: 0x0002, lo: 0x03},
+	{value: 0x004b, lo: 0x80, hi: 0x95},
+	{value: 0x0083, lo: 0x96, hi: 0xaf},
+	{value: 0x0043, lo: 0xb0, hi: 0xbf},
+	// Block 0x9e, offset 0x31f
+	{value: 0x0003, lo: 0x0f},
+	{value: 0x01bb, lo: 0x80, hi: 0x80},
+	{value: 0x0462, lo: 0x81, hi: 0x81},
+	{value: 0x01be, lo: 0x82, hi: 0x9a},
+	{value: 0x045e, lo: 0x9b, hi: 0x9b},
+	{value: 0x01ca, lo: 0x9c, hi: 0x9c},
+	{value: 0x01d3, lo: 0x9d, hi: 0x9d},
+	{value: 0x01d9, lo: 0x9e, hi: 0x9e},
+	{value: 0x01fd, lo: 0x9f, hi: 0x9f},
+	{value: 0x01ee, lo: 0xa0, hi: 0xa0},
+	{value: 0x01eb, lo: 0xa1, hi: 0xa1},
+	{value: 0x0176, lo: 0xa2, hi: 0xb2},
+	{value: 0x018b, lo: 0xb3, hi: 0xb3},
+	{value: 0x01a9, lo: 0xb4, hi: 0xba},
+	{value: 0x0462, lo: 0xbb, hi: 0xbb},
+	{value: 0x01be, lo: 0xbc, hi: 0xbf},
+	// Block 0x9f, offset 0x32f
+	{value: 0x0003, lo: 0x0d},
+	{value: 0x01ca, lo: 0x80, hi: 0x94},
+	{value: 0x045e, lo: 0x95, hi: 0x95},
+	{value: 0x01ca, lo: 0x96, hi: 0x96},
+	{value: 0x01d3, lo: 0x97, hi: 0x97},
+	{value: 0x01d9, lo: 0x98, hi: 0x98},
+	{value: 0x01fd, lo: 0x99, hi: 0x99},
+	{value: 0x01ee, lo: 0x9a, hi: 0x9a},
+	{value: 0x01eb, lo: 0x9b, hi: 0x9b},
+	{value: 0x0176, lo: 0x9c, hi: 0xac},
+	{value: 0x018b, lo: 0xad, hi: 0xad},
+	{value: 0x01a9, lo: 0xae, hi: 0xb4},
+	{value: 0x0462, lo: 0xb5, hi: 0xb5},
+	{value: 0x01be, lo: 0xb6, hi: 0xbf},
+	// Block 0xa0, offset 0x33d
+	{value: 0x0003, lo: 0x0d},
+	{value: 0x01dc, lo: 0x80, hi: 0x8e},
+	{value: 0x045e, lo: 0x8f, hi: 0x8f},
+	{value: 0x01ca, lo: 0x90, hi: 0x90},
+	{value: 0x01d3, lo: 0x91, hi: 0x91},
+	{value: 0x01d9, lo: 0x92, hi: 0x92},
+	{value: 0x01fd, lo: 0x93, hi: 0x93},
+	{value: 0x01ee, lo: 0x94, hi: 0x94},
+	{value: 0x01eb, lo: 0x95, hi: 0x95},
+	{value: 0x0176, lo: 0x96, hi: 0xa6},
+	{value: 0x018b, lo: 0xa7, hi: 0xa7},
+	{value: 0x01a9, lo: 0xa8, hi: 0xae},
+	{value: 0x0462, lo: 0xaf, hi: 0xaf},
+	{value: 0x01be, lo: 0xb0, hi: 0xbf},
+	// Block 0xa1, offset 0x34b
+	{value: 0x0003, lo: 0x0d},
+	{value: 0x01ee, lo: 0x80, hi: 0x88},
+	{value: 0x045e, lo: 0x89, hi: 0x89},
+	{value: 0x01ca, lo: 0x8a, hi: 0x8a},
+	{value: 0x01d3, lo: 0x8b, hi: 0x8b},
+	{value: 0x01d9, lo: 0x8c, hi: 0x8c},
+	{value: 0x01fd, lo: 0x8d, hi: 0x8d},
+	{value: 0x01ee, lo: 0x8e, hi: 0x8e},
+	{value: 0x01eb, lo: 0x8f, hi: 0x8f},
+	{value: 0x0176, lo: 0x90, hi: 0xa0},
+	{value: 0x018b, lo: 0xa1, hi: 0xa1},
+	{value: 0x01a9, lo: 0xa2, hi: 0xa8},
+	{value: 0x0462, lo: 0xa9, hi: 0xa9},
+	{value: 0x01be, lo: 0xaa, hi: 0xbf},
+	// Block 0xa2, offset 0x359
+	{value: 0x0000, lo: 0x05},
+	{value: 0x8133, lo: 0x80, hi: 0x86},
+	{value: 0x8133, lo: 0x88, hi: 0x98},
+	{value: 0x8133, lo: 0x9b, hi: 0xa1},
+	{value: 0x8133, lo: 0xa3, hi: 0xa4},
+	{value: 0x8133, lo: 0xa6, hi: 0xaa},
+	// Block 0xa3, offset 0x35f
+	{value: 0x0000, lo: 0x01},
+	{value: 0x8133, lo: 0xac, hi: 0xaf},
+	// Block 0xa4, offset 0x361
+	{value: 0x0000, lo: 0x01},
+	{value: 0x812e, lo: 0x90, hi: 0x96},
+	// Block 0xa5, offset 0x363
+	{value: 0x0000, lo: 0x02},
+	{value: 0x8133, lo: 0x84, hi: 0x89},
+	{value: 0x8103, lo: 0x8a, hi: 0x8a},
+	// Block 0xa6, offset 0x366
+	{value: 0x0002, lo: 0x0a},
+	{value: 0x0063, lo: 0x80, hi: 0x89},
+	{value: 0x1954, lo: 0x8a, hi: 0x8a},
+	{value: 0x1987, lo: 0x8b, hi: 0x8b},
+	{value: 0x19a2, lo: 0x8c, hi: 0x8c},
+	{value: 0x19a8, lo: 0x8d, hi: 0x8d},
+	{value: 0x1bc6, lo: 0x8e, hi: 0x8e},
+	{value: 0x19b4, lo: 0x8f, hi: 0x8f},
+	{value: 0x197e, lo: 0xaa, hi: 0xaa},
+	{value: 0x1981, lo: 0xab, hi: 0xab},
+	{value: 0x1984, lo: 0xac, hi: 0xac},
+	// Block 0xa7, offset 0x371
+	{value: 0x0000, lo: 0x01},
+	{value: 0x1942, lo: 0x90, hi: 0x90},
+	// Block 0xa8, offset 0x373
+	{value: 0x0028, lo: 0x09},
+	{value: 0x286f, lo: 0x80, hi: 0x80},
+	{value: 0x2833, lo: 0x81, hi: 0x81},
+	{value: 0x283d, lo: 0x82, hi: 0x82},
+	{value: 0x2851, lo: 0x83, hi: 0x84},
+	{value: 0x285b, lo: 0x85, hi: 0x86},
+	{value: 0x2847, lo: 0x87, hi: 0x87},
+	{value: 0x2865, lo: 0x88, hi: 0x88},
+	{value: 0x0b72, lo: 0x90, hi: 0x90},
+	{value: 0x08ea, lo: 0x91, hi: 0x91},
+	// Block 0xa9, offset 0x37d
+	{value: 0x0002, lo: 0x01},
+	{value: 0x0021, lo: 0xb0, hi: 0xb9},
+}
+
+// recompMap: 7528 bytes (entries only)
+var recompMap map[uint32]rune
+var recompMapOnce sync.Once
+
+const recompMapPacked = "" +
+	"\x00A\x03\x00\x00\x00\x00\xc0" + // 0x00410300: 0x000000C0
+	"\x00A\x03\x01\x00\x00\x00\xc1" + // 0x00410301: 0x000000C1
+	"\x00A\x03\x02\x00\x00\x00\xc2" + // 0x00410302: 0x000000C2
+	"\x00A\x03\x03\x00\x00\x00\xc3" + // 0x00410303: 0x000000C3
+	"\x00A\x03\b\x00\x00\x00\xc4" + // 0x00410308: 0x000000C4
+	"\x00A\x03\n\x00\x00\x00\xc5" + // 0x0041030A: 0x000000C5
+	"\x00C\x03'\x00\x00\x00\xc7" + // 0x00430327: 0x000000C7
+	"\x00E\x03\x00\x00\x00\x00\xc8" + // 0x00450300: 0x000000C8
+	"\x00E\x03\x01\x00\x00\x00\xc9" + // 0x00450301: 0x000000C9
+	"\x00E\x03\x02\x00\x00\x00\xca" + // 0x00450302: 0x000000CA
+	"\x00E\x03\b\x00\x00\x00\xcb" + // 0x00450308: 0x000000CB
+	"\x00I\x03\x00\x00\x00\x00\xcc" + // 0x00490300: 0x000000CC
+	"\x00I\x03\x01\x00\x00\x00\xcd" + // 0x00490301: 0x000000CD
+	"\x00I\x03\x02\x00\x00\x00\xce" + // 0x00490302: 0x000000CE
+	"\x00I\x03\b\x00\x00\x00\xcf" + // 0x00490308: 0x000000CF
+	"\x00N\x03\x03\x00\x00\x00\xd1" + // 0x004E0303: 0x000000D1
+	"\x00O\x03\x00\x00\x00\x00\xd2" + // 0x004F0300: 0x000000D2
+	"\x00O\x03\x01\x00\x00\x00\xd3" + // 0x004F0301: 0x000000D3
+	"\x00O\x03\x02\x00\x00\x00\xd4" + // 0x004F0302: 0x000000D4
+	"\x00O\x03\x03\x00\x00\x00\xd5" + // 0x004F0303: 0x000000D5
+	"\x00O\x03\b\x00\x00\x00\xd6" + // 0x004F0308: 0x000000D6
+	"\x00U\x03\x00\x00\x00\x00\xd9" + // 0x00550300: 0x000000D9
+	"\x00U\x03\x01\x00\x00\x00\xda" + // 0x00550301: 0x000000DA
+	"\x00U\x03\x02\x00\x00\x00\xdb" + // 0x00550302: 0x000000DB
+	"\x00U\x03\b\x00\x00\x00\xdc" + // 0x00550308: 0x000000DC
+	"\x00Y\x03\x01\x00\x00\x00\xdd" + // 0x00590301: 0x000000DD
+	"\x00a\x03\x00\x00\x00\x00\xe0" + // 0x00610300: 0x000000E0
+	"\x00a\x03\x01\x00\x00\x00\xe1" + // 0x00610301: 0x000000E1
+	"\x00a\x03\x02\x00\x00\x00\xe2" + // 0x00610302: 0x000000E2
+	"\x00a\x03\x03\x00\x00\x00\xe3" + // 0x00610303: 0x000000E3
+	"\x00a\x03\b\x00\x00\x00\xe4" + // 0x00610308: 0x000000E4
+	"\x00a\x03\n\x00\x00\x00\xe5" + // 0x0061030A: 0x000000E5
+	"\x00c\x03'\x00\x00\x00\xe7" + // 0x00630327: 0x000000E7
+	"\x00e\x03\x00\x00\x00\x00\xe8" + // 0x00650300: 0x000000E8
+	"\x00e\x03\x01\x00\x00\x00\xe9" + // 0x00650301: 0x000000E9
+	"\x00e\x03\x02\x00\x00\x00\xea" + // 0x00650302: 0x000000EA
+	"\x00e\x03\b\x00\x00\x00\xeb" + // 0x00650308: 0x000000EB
+	"\x00i\x03\x00\x00\x00\x00\xec" + // 0x00690300: 0x000000EC
+	"\x00i\x03\x01\x00\x00\x00\xed" + // 0x00690301: 0x000000ED
+	"\x00i\x03\x02\x00\x00\x00\xee" + // 0x00690302: 0x000000EE
+	"\x00i\x03\b\x00\x00\x00\xef" + // 0x00690308: 0x000000EF
+	"\x00n\x03\x03\x00\x00\x00\xf1" + // 0x006E0303: 0x000000F1
+	"\x00o\x03\x00\x00\x00\x00\xf2" + // 0x006F0300: 0x000000F2
+	"\x00o\x03\x01\x00\x00\x00\xf3" + // 0x006F0301: 0x000000F3
+	"\x00o\x03\x02\x00\x00\x00\xf4" + // 0x006F0302: 0x000000F4
+	"\x00o\x03\x03\x00\x00\x00\xf5" + // 0x006F0303: 0x000000F5
+	"\x00o\x03\b\x00\x00\x00\xf6" + // 0x006F0308: 0x000000F6
+	"\x00u\x03\x00\x00\x00\x00\xf9" + // 0x00750300: 0x000000F9
+	"\x00u\x03\x01\x00\x00\x00\xfa" + // 0x00750301: 0x000000FA
+	"\x00u\x03\x02\x00\x00\x00\xfb" + // 0x00750302: 0x000000FB
+	"\x00u\x03\b\x00\x00\x00\xfc" + // 0x00750308: 0x000000FC
+	"\x00y\x03\x01\x00\x00\x00\xfd" + // 0x00790301: 0x000000FD
+	"\x00y\x03\b\x00\x00\x00\xff" + // 0x00790308: 0x000000FF
+	"\x00A\x03\x04\x00\x00\x01\x00" + // 0x00410304: 0x00000100
+	"\x00a\x03\x04\x00\x00\x01\x01" + // 0x00610304: 0x00000101
+	"\x00A\x03\x06\x00\x00\x01\x02" + // 0x00410306: 0x00000102
+	"\x00a\x03\x06\x00\x00\x01\x03" + // 0x00610306: 0x00000103
+	"\x00A\x03(\x00\x00\x01\x04" + // 0x00410328: 0x00000104
+	"\x00a\x03(\x00\x00\x01\x05" + // 0x00610328: 0x00000105
+	"\x00C\x03\x01\x00\x00\x01\x06" + // 0x00430301: 0x00000106
+	"\x00c\x03\x01\x00\x00\x01\a" + // 0x00630301: 0x00000107
+	"\x00C\x03\x02\x00\x00\x01\b" + // 0x00430302: 0x00000108
+	"\x00c\x03\x02\x00\x00\x01\t" + // 0x00630302: 0x00000109
+	"\x00C\x03\a\x00\x00\x01\n" + // 0x00430307: 0x0000010A
+	"\x00c\x03\a\x00\x00\x01\v" + // 0x00630307: 0x0000010B
+	"\x00C\x03\f\x00\x00\x01\f" + // 0x0043030C: 0x0000010C
+	"\x00c\x03\f\x00\x00\x01\r" + // 0x0063030C: 0x0000010D
+	"\x00D\x03\f\x00\x00\x01\x0e" + // 0x0044030C: 0x0000010E
+	"\x00d\x03\f\x00\x00\x01\x0f" + // 0x0064030C: 0x0000010F
+	"\x00E\x03\x04\x00\x00\x01\x12" + // 0x00450304: 0x00000112
+	"\x00e\x03\x04\x00\x00\x01\x13" + // 0x00650304: 0x00000113
+	"\x00E\x03\x06\x00\x00\x01\x14" + // 0x00450306: 0x00000114
+	"\x00e\x03\x06\x00\x00\x01\x15" + // 0x00650306: 0x00000115
+	"\x00E\x03\a\x00\x00\x01\x16" + // 0x00450307: 0x00000116
+	"\x00e\x03\a\x00\x00\x01\x17" + // 0x00650307: 0x00000117
+	"\x00E\x03(\x00\x00\x01\x18" + // 0x00450328: 0x00000118
+	"\x00e\x03(\x00\x00\x01\x19" + // 0x00650328: 0x00000119
+	"\x00E\x03\f\x00\x00\x01\x1a" + // 0x0045030C: 0x0000011A
+	"\x00e\x03\f\x00\x00\x01\x1b" + // 0x0065030C: 0x0000011B
+	"\x00G\x03\x02\x00\x00\x01\x1c" + // 0x00470302: 0x0000011C
+	"\x00g\x03\x02\x00\x00\x01\x1d" + // 0x00670302: 0x0000011D
+	"\x00G\x03\x06\x00\x00\x01\x1e" + // 0x00470306: 0x0000011E
+	"\x00g\x03\x06\x00\x00\x01\x1f" + // 0x00670306: 0x0000011F
+	"\x00G\x03\a\x00\x00\x01 " + // 0x00470307: 0x00000120
+	"\x00g\x03\a\x00\x00\x01!" + // 0x00670307: 0x00000121
+	"\x00G\x03'\x00\x00\x01\"" + // 0x00470327: 0x00000122
+	"\x00g\x03'\x00\x00\x01#" + // 0x00670327: 0x00000123
+	"\x00H\x03\x02\x00\x00\x01$" + // 0x00480302: 0x00000124
+	"\x00h\x03\x02\x00\x00\x01%" + // 0x00680302: 0x00000125
+	"\x00I\x03\x03\x00\x00\x01(" + // 0x00490303: 0x00000128
+	"\x00i\x03\x03\x00\x00\x01)" + // 0x00690303: 0x00000129
+	"\x00I\x03\x04\x00\x00\x01*" + // 0x00490304: 0x0000012A
+	"\x00i\x03\x04\x00\x00\x01+" + // 0x00690304: 0x0000012B
+	"\x00I\x03\x06\x00\x00\x01," + // 0x00490306: 0x0000012C
+	"\x00i\x03\x06\x00\x00\x01-" + // 0x00690306: 0x0000012D
+	"\x00I\x03(\x00\x00\x01." + // 0x00490328: 0x0000012E
+	"\x00i\x03(\x00\x00\x01/" + // 0x00690328: 0x0000012F
+	"\x00I\x03\a\x00\x00\x010" + // 0x00490307: 0x00000130
+	"\x00J\x03\x02\x00\x00\x014" + // 0x004A0302: 0x00000134
+	"\x00j\x03\x02\x00\x00\x015" + // 0x006A0302: 0x00000135
+	"\x00K\x03'\x00\x00\x016" + // 0x004B0327: 0x00000136
+	"\x00k\x03'\x00\x00\x017" + // 0x006B0327: 0x00000137
+	"\x00L\x03\x01\x00\x00\x019" + // 0x004C0301: 0x00000139
+	"\x00l\x03\x01\x00\x00\x01:" + // 0x006C0301: 0x0000013A
+	"\x00L\x03'\x00\x00\x01;" + // 0x004C0327: 0x0000013B
+	"\x00l\x03'\x00\x00\x01<" + // 0x006C0327: 0x0000013C
+	"\x00L\x03\f\x00\x00\x01=" + // 0x004C030C: 0x0000013D
+	"\x00l\x03\f\x00\x00\x01>" + // 0x006C030C: 0x0000013E
+	"\x00N\x03\x01\x00\x00\x01C" + // 0x004E0301: 0x00000143
+	"\x00n\x03\x01\x00\x00\x01D" + // 0x006E0301: 0x00000144
+	"\x00N\x03'\x00\x00\x01E" + // 0x004E0327: 0x00000145
+	"\x00n\x03'\x00\x00\x01F" + // 0x006E0327: 0x00000146
+	"\x00N\x03\f\x00\x00\x01G" + // 0x004E030C: 0x00000147
+	"\x00n\x03\f\x00\x00\x01H" + // 0x006E030C: 0x00000148
+	"\x00O\x03\x04\x00\x00\x01L" + // 0x004F0304: 0x0000014C
+	"\x00o\x03\x04\x00\x00\x01M" + // 0x006F0304: 0x0000014D
+	"\x00O\x03\x06\x00\x00\x01N" + // 0x004F0306: 0x0000014E
+	"\x00o\x03\x06\x00\x00\x01O" + // 0x006F0306: 0x0000014F
+	"\x00O\x03\v\x00\x00\x01P" + // 0x004F030B: 0x00000150
+	"\x00o\x03\v\x00\x00\x01Q" + // 0x006F030B: 0x00000151
+	"\x00R\x03\x01\x00\x00\x01T" + // 0x00520301: 0x00000154
+	"\x00r\x03\x01\x00\x00\x01U" + // 0x00720301: 0x00000155
+	"\x00R\x03'\x00\x00\x01V" + // 0x00520327: 0x00000156
+	"\x00r\x03'\x00\x00\x01W" + // 0x00720327: 0x00000157
+	"\x00R\x03\f\x00\x00\x01X" + // 0x0052030C: 0x00000158
+	"\x00r\x03\f\x00\x00\x01Y" + // 0x0072030C: 0x00000159
+	"\x00S\x03\x01\x00\x00\x01Z" + // 0x00530301: 0x0000015A
+	"\x00s\x03\x01\x00\x00\x01[" + // 0x00730301: 0x0000015B
+	"\x00S\x03\x02\x00\x00\x01\\" + // 0x00530302: 0x0000015C
+	"\x00s\x03\x02\x00\x00\x01]" + // 0x00730302: 0x0000015D
+	"\x00S\x03'\x00\x00\x01^" + // 0x00530327: 0x0000015E
+	"\x00s\x03'\x00\x00\x01_" + // 0x00730327: 0x0000015F
+	"\x00S\x03\f\x00\x00\x01`" + // 0x0053030C: 0x00000160
+	"\x00s\x03\f\x00\x00\x01a" + // 0x0073030C: 0x00000161
+	"\x00T\x03'\x00\x00\x01b" + // 0x00540327: 0x00000162
+	"\x00t\x03'\x00\x00\x01c" + // 0x00740327: 0x00000163
+	"\x00T\x03\f\x00\x00\x01d" + // 0x0054030C: 0x00000164
+	"\x00t\x03\f\x00\x00\x01e" + // 0x0074030C: 0x00000165
+	"\x00U\x03\x03\x00\x00\x01h" + // 0x00550303: 0x00000168
+	"\x00u\x03\x03\x00\x00\x01i" + // 0x00750303: 0x00000169
+	"\x00U\x03\x04\x00\x00\x01j" + // 0x00550304: 0x0000016A
+	"\x00u\x03\x04\x00\x00\x01k" + // 0x00750304: 0x0000016B
+	"\x00U\x03\x06\x00\x00\x01l" + // 0x00550306: 0x0000016C
+	"\x00u\x03\x06\x00\x00\x01m" + // 0x00750306: 0x0000016D
+	"\x00U\x03\n\x00\x00\x01n" + // 0x0055030A: 0x0000016E
+	"\x00u\x03\n\x00\x00\x01o" + // 0x0075030A: 0x0000016F
+	"\x00U\x03\v\x00\x00\x01p" + // 0x0055030B: 0x00000170
+	"\x00u\x03\v\x00\x00\x01q" + // 0x0075030B: 0x00000171
+	"\x00U\x03(\x00\x00\x01r" + // 0x00550328: 0x00000172
+	"\x00u\x03(\x00\x00\x01s" + // 0x00750328: 0x00000173
+	"\x00W\x03\x02\x00\x00\x01t" + // 0x00570302: 0x00000174
+	"\x00w\x03\x02\x00\x00\x01u" + // 0x00770302: 0x00000175
+	"\x00Y\x03\x02\x00\x00\x01v" + // 0x00590302: 0x00000176
+	"\x00y\x03\x02\x00\x00\x01w" + // 0x00790302: 0x00000177
+	"\x00Y\x03\b\x00\x00\x01x" + // 0x00590308: 0x00000178
+	"\x00Z\x03\x01\x00\x00\x01y" + // 0x005A0301: 0x00000179
+	"\x00z\x03\x01\x00\x00\x01z" + // 0x007A0301: 0x0000017A
+	"\x00Z\x03\a\x00\x00\x01{" + // 0x005A0307: 0x0000017B
+	"\x00z\x03\a\x00\x00\x01|" + // 0x007A0307: 0x0000017C
+	"\x00Z\x03\f\x00\x00\x01}" + // 0x005A030C: 0x0000017D
+	"\x00z\x03\f\x00\x00\x01~" + // 0x007A030C: 0x0000017E
+	"\x00O\x03\x1b\x00\x00\x01\xa0" + // 0x004F031B: 0x000001A0
+	"\x00o\x03\x1b\x00\x00\x01\xa1" + // 0x006F031B: 0x000001A1
+	"\x00U\x03\x1b\x00\x00\x01\xaf" + // 0x0055031B: 0x000001AF
+	"\x00u\x03\x1b\x00\x00\x01\xb0" + // 0x0075031B: 0x000001B0
+	"\x00A\x03\f\x00\x00\x01\xcd" + // 0x0041030C: 0x000001CD
+	"\x00a\x03\f\x00\x00\x01\xce" + // 0x0061030C: 0x000001CE
+	"\x00I\x03\f\x00\x00\x01\xcf" + // 0x0049030C: 0x000001CF
+	"\x00i\x03\f\x00\x00\x01\xd0" + // 0x0069030C: 0x000001D0
+	"\x00O\x03\f\x00\x00\x01\xd1" + // 0x004F030C: 0x000001D1
+	"\x00o\x03\f\x00\x00\x01\xd2" + // 0x006F030C: 0x000001D2
+	"\x00U\x03\f\x00\x00\x01\xd3" + // 0x0055030C: 0x000001D3
+	"\x00u\x03\f\x00\x00\x01\xd4" + // 0x0075030C: 0x000001D4
+	"\x00\xdc\x03\x04\x00\x00\x01\xd5" + // 0x00DC0304: 0x000001D5
+	"\x00\xfc\x03\x04\x00\x00\x01\xd6" + // 0x00FC0304: 0x000001D6
+	"\x00\xdc\x03\x01\x00\x00\x01\xd7" + // 0x00DC0301: 0x000001D7
+	"\x00\xfc\x03\x01\x00\x00\x01\xd8" + // 0x00FC0301: 0x000001D8
+	"\x00\xdc\x03\f\x00\x00\x01\xd9" + // 0x00DC030C: 0x000001D9
+	"\x00\xfc\x03\f\x00\x00\x01\xda" + // 0x00FC030C: 0x000001DA
+	"\x00\xdc\x03\x00\x00\x00\x01\xdb" + // 0x00DC0300: 0x000001DB
+	"\x00\xfc\x03\x00\x00\x00\x01\xdc" + // 0x00FC0300: 0x000001DC
+	"\x00\xc4\x03\x04\x00\x00\x01\xde" + // 0x00C40304: 0x000001DE
+	"\x00\xe4\x03\x04\x00\x00\x01\xdf" + // 0x00E40304: 0x000001DF
+	"\x02&\x03\x04\x00\x00\x01\xe0" + // 0x02260304: 0x000001E0
+	"\x02'\x03\x04\x00\x00\x01\xe1" + // 0x02270304: 0x000001E1
+	"\x00\xc6\x03\x04\x00\x00\x01\xe2" + // 0x00C60304: 0x000001E2
+	"\x00\xe6\x03\x04\x00\x00\x01\xe3" + // 0x00E60304: 0x000001E3
+	"\x00G\x03\f\x00\x00\x01\xe6" + // 0x0047030C: 0x000001E6
+	"\x00g\x03\f\x00\x00\x01\xe7" + // 0x0067030C: 0x000001E7
+	"\x00K\x03\f\x00\x00\x01\xe8" + // 0x004B030C: 0x000001E8
+	"\x00k\x03\f\x00\x00\x01\xe9" + // 0x006B030C: 0x000001E9
+	"\x00O\x03(\x00\x00\x01\xea" + // 0x004F0328: 0x000001EA
+	"\x00o\x03(\x00\x00\x01\xeb" + // 0x006F0328: 0x000001EB
+	"\x01\xea\x03\x04\x00\x00\x01\xec" + // 0x01EA0304: 0x000001EC
+	"\x01\xeb\x03\x04\x00\x00\x01\xed" + // 0x01EB0304: 0x000001ED
+	"\x01\xb7\x03\f\x00\x00\x01\xee" + // 0x01B7030C: 0x000001EE
+	"\x02\x92\x03\f\x00\x00\x01\xef" + // 0x0292030C: 0x000001EF
+	"\x00j\x03\f\x00\x00\x01\xf0" + // 0x006A030C: 0x000001F0
+	"\x00G\x03\x01\x00\x00\x01\xf4" + // 0x00470301: 0x000001F4
+	"\x00g\x03\x01\x00\x00\x01\xf5" + // 0x00670301: 0x000001F5
+	"\x00N\x03\x00\x00\x00\x01\xf8" + // 0x004E0300: 0x000001F8
+	"\x00n\x03\x00\x00\x00\x01\xf9" + // 0x006E0300: 0x000001F9
+	"\x00\xc5\x03\x01\x00\x00\x01\xfa" + // 0x00C50301: 0x000001FA
+	"\x00\xe5\x03\x01\x00\x00\x01\xfb" + // 0x00E50301: 0x000001FB
+	"\x00\xc6\x03\x01\x00\x00\x01\xfc" + // 0x00C60301: 0x000001FC
+	"\x00\xe6\x03\x01\x00\x00\x01\xfd" + // 0x00E60301: 0x000001FD
+	"\x00\xd8\x03\x01\x00\x00\x01\xfe" + // 0x00D80301: 0x000001FE
+	"\x00\xf8\x03\x01\x00\x00\x01\xff" + // 0x00F80301: 0x000001FF
+	"\x00A\x03\x0f\x00\x00\x02\x00" + // 0x0041030F: 0x00000200
+	"\x00a\x03\x0f\x00\x00\x02\x01" + // 0x0061030F: 0x00000201
+	"\x00A\x03\x11\x00\x00\x02\x02" + // 0x00410311: 0x00000202
+	"\x00a\x03\x11\x00\x00\x02\x03" + // 0x00610311: 0x00000203
+	"\x00E\x03\x0f\x00\x00\x02\x04" + // 0x0045030F: 0x00000204
+	"\x00e\x03\x0f\x00\x00\x02\x05" + // 0x0065030F: 0x00000205
+	"\x00E\x03\x11\x00\x00\x02\x06" + // 0x00450311: 0x00000206
+	"\x00e\x03\x11\x00\x00\x02\a" + // 0x00650311: 0x00000207
+	"\x00I\x03\x0f\x00\x00\x02\b" + // 0x0049030F: 0x00000208
+	"\x00i\x03\x0f\x00\x00\x02\t" + // 0x0069030F: 0x00000209
+	"\x00I\x03\x11\x00\x00\x02\n" + // 0x00490311: 0x0000020A
+	"\x00i\x03\x11\x00\x00\x02\v" + // 0x00690311: 0x0000020B
+	"\x00O\x03\x0f\x00\x00\x02\f" + // 0x004F030F: 0x0000020C
+	"\x00o\x03\x0f\x00\x00\x02\r" + // 0x006F030F: 0x0000020D
+	"\x00O\x03\x11\x00\x00\x02\x0e" + // 0x004F0311: 0x0000020E
+	"\x00o\x03\x11\x00\x00\x02\x0f" + // 0x006F0311: 0x0000020F
+	"\x00R\x03\x0f\x00\x00\x02\x10" + // 0x0052030F: 0x00000210
+	"\x00r\x03\x0f\x00\x00\x02\x11" + // 0x0072030F: 0x00000211
+	"\x00R\x03\x11\x00\x00\x02\x12" + // 0x00520311: 0x00000212
+	"\x00r\x03\x11\x00\x00\x02\x13" + // 0x00720311: 0x00000213
+	"\x00U\x03\x0f\x00\x00\x02\x14" + // 0x0055030F: 0x00000214
+	"\x00u\x03\x0f\x00\x00\x02\x15" + // 0x0075030F: 0x00000215
+	"\x00U\x03\x11\x00\x00\x02\x16" + // 0x00550311: 0x00000216
+	"\x00u\x03\x11\x00\x00\x02\x17" + // 0x00750311: 0x00000217
+	"\x00S\x03&\x00\x00\x02\x18" + // 0x00530326: 0x00000218
+	"\x00s\x03&\x00\x00\x02\x19" + // 0x00730326: 0x00000219
+	"\x00T\x03&\x00\x00\x02\x1a" + // 0x00540326: 0x0000021A
+	"\x00t\x03&\x00\x00\x02\x1b" + // 0x00740326: 0x0000021B
+	"\x00H\x03\f\x00\x00\x02\x1e" + // 0x0048030C: 0x0000021E
+	"\x00h\x03\f\x00\x00\x02\x1f" + // 0x0068030C: 0x0000021F
+	"\x00A\x03\a\x00\x00\x02&" + // 0x00410307: 0x00000226
+	"\x00a\x03\a\x00\x00\x02'" + // 0x00610307: 0x00000227
+	"\x00E\x03'\x00\x00\x02(" + // 0x00450327: 0x00000228
+	"\x00e\x03'\x00\x00\x02)" + // 0x00650327: 0x00000229
+	"\x00\xd6\x03\x04\x00\x00\x02*" + // 0x00D60304: 0x0000022A
+	"\x00\xf6\x03\x04\x00\x00\x02+" + // 0x00F60304: 0x0000022B
+	"\x00\xd5\x03\x04\x00\x00\x02," + // 0x00D50304: 0x0000022C
+	"\x00\xf5\x03\x04\x00\x00\x02-" + // 0x00F50304: 0x0000022D
+	"\x00O\x03\a\x00\x00\x02." + // 0x004F0307: 0x0000022E
+	"\x00o\x03\a\x00\x00\x02/" + // 0x006F0307: 0x0000022F
+	"\x02.\x03\x04\x00\x00\x020" + // 0x022E0304: 0x00000230
+	"\x02/\x03\x04\x00\x00\x021" + // 0x022F0304: 0x00000231
+	"\x00Y\x03\x04\x00\x00\x022" + // 0x00590304: 0x00000232
+	"\x00y\x03\x04\x00\x00\x023" + // 0x00790304: 0x00000233
+	"\x00\xa8\x03\x01\x00\x00\x03\x85" + // 0x00A80301: 0x00000385
+	"\x03\x91\x03\x01\x00\x00\x03\x86" + // 0x03910301: 0x00000386
+	"\x03\x95\x03\x01\x00\x00\x03\x88" + // 0x03950301: 0x00000388
+	"\x03\x97\x03\x01\x00\x00\x03\x89" + // 0x03970301: 0x00000389
+	"\x03\x99\x03\x01\x00\x00\x03\x8a" + // 0x03990301: 0x0000038A
+	"\x03\x9f\x03\x01\x00\x00\x03\x8c" + // 0x039F0301: 0x0000038C
+	"\x03\xa5\x03\x01\x00\x00\x03\x8e" + // 0x03A50301: 0x0000038E
+	"\x03\xa9\x03\x01\x00\x00\x03\x8f" + // 0x03A90301: 0x0000038F
+	"\x03\xca\x03\x01\x00\x00\x03\x90" + // 0x03CA0301: 0x00000390
+	"\x03\x99\x03\b\x00\x00\x03\xaa" + // 0x03990308: 0x000003AA
+	"\x03\xa5\x03\b\x00\x00\x03\xab" + // 0x03A50308: 0x000003AB
+	"\x03\xb1\x03\x01\x00\x00\x03\xac" + // 0x03B10301: 0x000003AC
+	"\x03\xb5\x03\x01\x00\x00\x03\xad" + // 0x03B50301: 0x000003AD
+	"\x03\xb7\x03\x01\x00\x00\x03\xae" + // 0x03B70301: 0x000003AE
+	"\x03\xb9\x03\x01\x00\x00\x03\xaf" + // 0x03B90301: 0x000003AF
+	"\x03\xcb\x03\x01\x00\x00\x03\xb0" + // 0x03CB0301: 0x000003B0
+	"\x03\xb9\x03\b\x00\x00\x03\xca" + // 0x03B90308: 0x000003CA
+	"\x03\xc5\x03\b\x00\x00\x03\xcb" + // 0x03C50308: 0x000003CB
+	"\x03\xbf\x03\x01\x00\x00\x03\xcc" + // 0x03BF0301: 0x000003CC
+	"\x03\xc5\x03\x01\x00\x00\x03\xcd" + // 0x03C50301: 0x000003CD
+	"\x03\xc9\x03\x01\x00\x00\x03\xce" + // 0x03C90301: 0x000003CE
+	"\x03\xd2\x03\x01\x00\x00\x03\xd3" + // 0x03D20301: 0x000003D3
+	"\x03\xd2\x03\b\x00\x00\x03\xd4" + // 0x03D20308: 0x000003D4
+	"\x04\x15\x03\x00\x00\x00\x04\x00" + // 0x04150300: 0x00000400
+	"\x04\x15\x03\b\x00\x00\x04\x01" + // 0x04150308: 0x00000401
+	"\x04\x13\x03\x01\x00\x00\x04\x03" + // 0x04130301: 0x00000403
+	"\x04\x06\x03\b\x00\x00\x04\a" + // 0x04060308: 0x00000407
+	"\x04\x1a\x03\x01\x00\x00\x04\f" + // 0x041A0301: 0x0000040C
+	"\x04\x18\x03\x00\x00\x00\x04\r" + // 0x04180300: 0x0000040D
+	"\x04#\x03\x06\x00\x00\x04\x0e" + // 0x04230306: 0x0000040E
+	"\x04\x18\x03\x06\x00\x00\x04\x19" + // 0x04180306: 0x00000419
+	"\x048\x03\x06\x00\x00\x049" + // 0x04380306: 0x00000439
+	"\x045\x03\x00\x00\x00\x04P" + // 0x04350300: 0x00000450
+	"\x045\x03\b\x00\x00\x04Q" + // 0x04350308: 0x00000451
+	"\x043\x03\x01\x00\x00\x04S" + // 0x04330301: 0x00000453
+	"\x04V\x03\b\x00\x00\x04W" + // 0x04560308: 0x00000457
+	"\x04:\x03\x01\x00\x00\x04\\" + // 0x043A0301: 0x0000045C
+	"\x048\x03\x00\x00\x00\x04]" + // 0x04380300: 0x0000045D
+	"\x04C\x03\x06\x00\x00\x04^" + // 0x04430306: 0x0000045E
+	"\x04t\x03\x0f\x00\x00\x04v" + // 0x0474030F: 0x00000476
+	"\x04u\x03\x0f\x00\x00\x04w" + // 0x0475030F: 0x00000477
+	"\x04\x16\x03\x06\x00\x00\x04\xc1" + // 0x04160306: 0x000004C1
+	"\x046\x03\x06\x00\x00\x04\xc2" + // 0x04360306: 0x000004C2
+	"\x04\x10\x03\x06\x00\x00\x04\xd0" + // 0x04100306: 0x000004D0
+	"\x040\x03\x06\x00\x00\x04\xd1" + // 0x04300306: 0x000004D1
+	"\x04\x10\x03\b\x00\x00\x04\xd2" + // 0x04100308: 0x000004D2
+	"\x040\x03\b\x00\x00\x04\xd3" + // 0x04300308: 0x000004D3
+	"\x04\x15\x03\x06\x00\x00\x04\xd6" + // 0x04150306: 0x000004D6
+	"\x045\x03\x06\x00\x00\x04\xd7" + // 0x04350306: 0x000004D7
+	"\x04\xd8\x03\b\x00\x00\x04\xda" + // 0x04D80308: 0x000004DA
+	"\x04\xd9\x03\b\x00\x00\x04\xdb" + // 0x04D90308: 0x000004DB
+	"\x04\x16\x03\b\x00\x00\x04\xdc" + // 0x04160308: 0x000004DC
+	"\x046\x03\b\x00\x00\x04\xdd" + // 0x04360308: 0x000004DD
+	"\x04\x17\x03\b\x00\x00\x04\xde" + // 0x04170308: 0x000004DE
+	"\x047\x03\b\x00\x00\x04\xdf" + // 0x04370308: 0x000004DF
+	"\x04\x18\x03\x04\x00\x00\x04\xe2" + // 0x04180304: 0x000004E2
+	"\x048\x03\x04\x00\x00\x04\xe3" + // 0x04380304: 0x000004E3
+	"\x04\x18\x03\b\x00\x00\x04\xe4" + // 0x04180308: 0x000004E4
+	"\x048\x03\b\x00\x00\x04\xe5" + // 0x04380308: 0x000004E5
+	"\x04\x1e\x03\b\x00\x00\x04\xe6" + // 0x041E0308: 0x000004E6
+	"\x04>\x03\b\x00\x00\x04\xe7" + // 0x043E0308: 0x000004E7
+	"\x04\xe8\x03\b\x00\x00\x04\xea" + // 0x04E80308: 0x000004EA
+	"\x04\xe9\x03\b\x00\x00\x04\xeb" + // 0x04E90308: 0x000004EB
+	"\x04-\x03\b\x00\x00\x04\xec" + // 0x042D0308: 0x000004EC
+	"\x04M\x03\b\x00\x00\x04\xed" + // 0x044D0308: 0x000004ED
+	"\x04#\x03\x04\x00\x00\x04\xee" + // 0x04230304: 0x000004EE
+	"\x04C\x03\x04\x00\x00\x04\xef" + // 0x04430304: 0x000004EF
+	"\x04#\x03\b\x00\x00\x04\xf0" + // 0x04230308: 0x000004F0
+	"\x04C\x03\b\x00\x00\x04\xf1" + // 0x04430308: 0x000004F1
+	"\x04#\x03\v\x00\x00\x04\xf2" + // 0x0423030B: 0x000004F2
+	"\x04C\x03\v\x00\x00\x04\xf3" + // 0x0443030B: 0x000004F3
+	"\x04'\x03\b\x00\x00\x04\xf4" + // 0x04270308: 0x000004F4
+	"\x04G\x03\b\x00\x00\x04\xf5" + // 0x04470308: 0x000004F5
+	"\x04+\x03\b\x00\x00\x04\xf8" + // 0x042B0308: 0x000004F8
+	"\x04K\x03\b\x00\x00\x04\xf9" + // 0x044B0308: 0x000004F9
+	"\x06'\x06S\x00\x00\x06\"" + // 0x06270653: 0x00000622
+	"\x06'\x06T\x00\x00\x06#" + // 0x06270654: 0x00000623
+	"\x06H\x06T\x00\x00\x06$" + // 0x06480654: 0x00000624
+	"\x06'\x06U\x00\x00\x06%" + // 0x06270655: 0x00000625
+	"\x06J\x06T\x00\x00\x06&" + // 0x064A0654: 0x00000626
+	"\x06\xd5\x06T\x00\x00\x06\xc0" + // 0x06D50654: 0x000006C0
+	"\x06\xc1\x06T\x00\x00\x06\xc2" + // 0x06C10654: 0x000006C2
+	"\x06\xd2\x06T\x00\x00\x06\xd3" + // 0x06D20654: 0x000006D3
+	"\t(\t<\x00\x00\t)" + // 0x0928093C: 0x00000929
+	"\t0\t<\x00\x00\t1" + // 0x0930093C: 0x00000931
+	"\t3\t<\x00\x00\t4" + // 0x0933093C: 0x00000934
+	"\t\xc7\t\xbe\x00\x00\t\xcb" + // 0x09C709BE: 0x000009CB
+	"\t\xc7\t\xd7\x00\x00\t\xcc" + // 0x09C709D7: 0x000009CC
+	"\vG\vV\x00\x00\vH" + // 0x0B470B56: 0x00000B48
+	"\vG\v>\x00\x00\vK" + // 0x0B470B3E: 0x00000B4B
+	"\vG\vW\x00\x00\vL" + // 0x0B470B57: 0x00000B4C
+	"\v\x92\v\xd7\x00\x00\v\x94" + // 0x0B920BD7: 0x00000B94
+	"\v\xc6\v\xbe\x00\x00\v\xca" + // 0x0BC60BBE: 0x00000BCA
+	"\v\xc7\v\xbe\x00\x00\v\xcb" + // 0x0BC70BBE: 0x00000BCB
+	"\v\xc6\v\xd7\x00\x00\v\xcc" + // 0x0BC60BD7: 0x00000BCC
+	"\fF\fV\x00\x00\fH" + // 0x0C460C56: 0x00000C48
+	"\f\xbf\f\xd5\x00\x00\f\xc0" + // 0x0CBF0CD5: 0x00000CC0
+	"\f\xc6\f\xd5\x00\x00\f\xc7" + // 0x0CC60CD5: 0x00000CC7
+	"\f\xc6\f\xd6\x00\x00\f\xc8" + // 0x0CC60CD6: 0x00000CC8
+	"\f\xc6\f\xc2\x00\x00\f\xca" + // 0x0CC60CC2: 0x00000CCA
+	"\f\xca\f\xd5\x00\x00\f\xcb" + // 0x0CCA0CD5: 0x00000CCB
+	"\rF\r>\x00\x00\rJ" + // 0x0D460D3E: 0x00000D4A
+	"\rG\r>\x00\x00\rK" + // 0x0D470D3E: 0x00000D4B
+	"\rF\rW\x00\x00\rL" + // 0x0D460D57: 0x00000D4C
+	"\r\xd9\r\xca\x00\x00\r\xda" + // 0x0DD90DCA: 0x00000DDA
+	"\r\xd9\r\xcf\x00\x00\r\xdc" + // 0x0DD90DCF: 0x00000DDC
+	"\r\xdc\r\xca\x00\x00\r\xdd" + // 0x0DDC0DCA: 0x00000DDD
+	"\r\xd9\r\xdf\x00\x00\r\xde" + // 0x0DD90DDF: 0x00000DDE
+	"\x10%\x10.\x00\x00\x10&" + // 0x1025102E: 0x00001026
+	"\x1b\x05\x1b5\x00\x00\x1b\x06" + // 0x1B051B35: 0x00001B06
+	"\x1b\a\x1b5\x00\x00\x1b\b" + // 0x1B071B35: 0x00001B08
+	"\x1b\t\x1b5\x00\x00\x1b\n" + // 0x1B091B35: 0x00001B0A
+	"\x1b\v\x1b5\x00\x00\x1b\f" + // 0x1B0B1B35: 0x00001B0C
+	"\x1b\r\x1b5\x00\x00\x1b\x0e" + // 0x1B0D1B35: 0x00001B0E
+	"\x1b\x11\x1b5\x00\x00\x1b\x12" + // 0x1B111B35: 0x00001B12
+	"\x1b:\x1b5\x00\x00\x1b;" + // 0x1B3A1B35: 0x00001B3B
+	"\x1b<\x1b5\x00\x00\x1b=" + // 0x1B3C1B35: 0x00001B3D
+	"\x1b>\x1b5\x00\x00\x1b@" + // 0x1B3E1B35: 0x00001B40
+	"\x1b?\x1b5\x00\x00\x1bA" + // 0x1B3F1B35: 0x00001B41
+	"\x1bB\x1b5\x00\x00\x1bC" + // 0x1B421B35: 0x00001B43
+	"\x00A\x03%\x00\x00\x1e\x00" + // 0x00410325: 0x00001E00
+	"\x00a\x03%\x00\x00\x1e\x01" + // 0x00610325: 0x00001E01
+	"\x00B\x03\a\x00\x00\x1e\x02" + // 0x00420307: 0x00001E02
+	"\x00b\x03\a\x00\x00\x1e\x03" + // 0x00620307: 0x00001E03
+	"\x00B\x03#\x00\x00\x1e\x04" + // 0x00420323: 0x00001E04
+	"\x00b\x03#\x00\x00\x1e\x05" + // 0x00620323: 0x00001E05
+	"\x00B\x031\x00\x00\x1e\x06" + // 0x00420331: 0x00001E06
+	"\x00b\x031\x00\x00\x1e\a" + // 0x00620331: 0x00001E07
+	"\x00\xc7\x03\x01\x00\x00\x1e\b" + // 0x00C70301: 0x00001E08
+	"\x00\xe7\x03\x01\x00\x00\x1e\t" + // 0x00E70301: 0x00001E09
+	"\x00D\x03\a\x00\x00\x1e\n" + // 0x00440307: 0x00001E0A
+	"\x00d\x03\a\x00\x00\x1e\v" + // 0x00640307: 0x00001E0B
+	"\x00D\x03#\x00\x00\x1e\f" + // 0x00440323: 0x00001E0C
+	"\x00d\x03#\x00\x00\x1e\r" + // 0x00640323: 0x00001E0D
+	"\x00D\x031\x00\x00\x1e\x0e" + // 0x00440331: 0x00001E0E
+	"\x00d\x031\x00\x00\x1e\x0f" + // 0x00640331: 0x00001E0F
+	"\x00D\x03'\x00\x00\x1e\x10" + // 0x00440327: 0x00001E10
+	"\x00d\x03'\x00\x00\x1e\x11" + // 0x00640327: 0x00001E11
+	"\x00D\x03-\x00\x00\x1e\x12" + // 0x0044032D: 0x00001E12
+	"\x00d\x03-\x00\x00\x1e\x13" + // 0x0064032D: 0x00001E13
+	"\x01\x12\x03\x00\x00\x00\x1e\x14" + // 0x01120300: 0x00001E14
+	"\x01\x13\x03\x00\x00\x00\x1e\x15" + // 0x01130300: 0x00001E15
+	"\x01\x12\x03\x01\x00\x00\x1e\x16" + // 0x01120301: 0x00001E16
+	"\x01\x13\x03\x01\x00\x00\x1e\x17" + // 0x01130301: 0x00001E17
+	"\x00E\x03-\x00\x00\x1e\x18" + // 0x0045032D: 0x00001E18
+	"\x00e\x03-\x00\x00\x1e\x19" + // 0x0065032D: 0x00001E19
+	"\x00E\x030\x00\x00\x1e\x1a" + // 0x00450330: 0x00001E1A
+	"\x00e\x030\x00\x00\x1e\x1b" + // 0x00650330: 0x00001E1B
+	"\x02(\x03\x06\x00\x00\x1e\x1c" + // 0x02280306: 0x00001E1C
+	"\x02)\x03\x06\x00\x00\x1e\x1d" + // 0x02290306: 0x00001E1D
+	"\x00F\x03\a\x00\x00\x1e\x1e" + // 0x00460307: 0x00001E1E
+	"\x00f\x03\a\x00\x00\x1e\x1f" + // 0x00660307: 0x00001E1F
+	"\x00G\x03\x04\x00\x00\x1e " + // 0x00470304: 0x00001E20
+	"\x00g\x03\x04\x00\x00\x1e!" + // 0x00670304: 0x00001E21
+	"\x00H\x03\a\x00\x00\x1e\"" + // 0x00480307: 0x00001E22
+	"\x00h\x03\a\x00\x00\x1e#" + // 0x00680307: 0x00001E23
+	"\x00H\x03#\x00\x00\x1e$" + // 0x00480323: 0x00001E24
+	"\x00h\x03#\x00\x00\x1e%" + // 0x00680323: 0x00001E25
+	"\x00H\x03\b\x00\x00\x1e&" + // 0x00480308: 0x00001E26
+	"\x00h\x03\b\x00\x00\x1e'" + // 0x00680308: 0x00001E27
+	"\x00H\x03'\x00\x00\x1e(" + // 0x00480327: 0x00001E28
+	"\x00h\x03'\x00\x00\x1e)" + // 0x00680327: 0x00001E29
+	"\x00H\x03.\x00\x00\x1e*" + // 0x0048032E: 0x00001E2A
+	"\x00h\x03.\x00\x00\x1e+" + // 0x0068032E: 0x00001E2B
+	"\x00I\x030\x00\x00\x1e," + // 0x00490330: 0x00001E2C
+	"\x00i\x030\x00\x00\x1e-" + // 0x00690330: 0x00001E2D
+	"\x00\xcf\x03\x01\x00\x00\x1e." + // 0x00CF0301: 0x00001E2E
+	"\x00\xef\x03\x01\x00\x00\x1e/" + // 0x00EF0301: 0x00001E2F
+	"\x00K\x03\x01\x00\x00\x1e0" + // 0x004B0301: 0x00001E30
+	"\x00k\x03\x01\x00\x00\x1e1" + // 0x006B0301: 0x00001E31
+	"\x00K\x03#\x00\x00\x1e2" + // 0x004B0323: 0x00001E32
+	"\x00k\x03#\x00\x00\x1e3" + // 0x006B0323: 0x00001E33
+	"\x00K\x031\x00\x00\x1e4" + // 0x004B0331: 0x00001E34
+	"\x00k\x031\x00\x00\x1e5" + // 0x006B0331: 0x00001E35
+	"\x00L\x03#\x00\x00\x1e6" + // 0x004C0323: 0x00001E36
+	"\x00l\x03#\x00\x00\x1e7" + // 0x006C0323: 0x00001E37
+	"\x1e6\x03\x04\x00\x00\x1e8" + // 0x1E360304: 0x00001E38
+	"\x1e7\x03\x04\x00\x00\x1e9" + // 0x1E370304: 0x00001E39
+	"\x00L\x031\x00\x00\x1e:" + // 0x004C0331: 0x00001E3A
+	"\x00l\x031\x00\x00\x1e;" + // 0x006C0331: 0x00001E3B
+	"\x00L\x03-\x00\x00\x1e<" + // 0x004C032D: 0x00001E3C
+	"\x00l\x03-\x00\x00\x1e=" + // 0x006C032D: 0x00001E3D
+	"\x00M\x03\x01\x00\x00\x1e>" + // 0x004D0301: 0x00001E3E
+	"\x00m\x03\x01\x00\x00\x1e?" + // 0x006D0301: 0x00001E3F
+	"\x00M\x03\a\x00\x00\x1e@" + // 0x004D0307: 0x00001E40
+	"\x00m\x03\a\x00\x00\x1eA" + // 0x006D0307: 0x00001E41
+	"\x00M\x03#\x00\x00\x1eB" + // 0x004D0323: 0x00001E42
+	"\x00m\x03#\x00\x00\x1eC" + // 0x006D0323: 0x00001E43
+	"\x00N\x03\a\x00\x00\x1eD" + // 0x004E0307: 0x00001E44
+	"\x00n\x03\a\x00\x00\x1eE" + // 0x006E0307: 0x00001E45
+	"\x00N\x03#\x00\x00\x1eF" + // 0x004E0323: 0x00001E46
+	"\x00n\x03#\x00\x00\x1eG" + // 0x006E0323: 0x00001E47
+	"\x00N\x031\x00\x00\x1eH" + // 0x004E0331: 0x00001E48
+	"\x00n\x031\x00\x00\x1eI" + // 0x006E0331: 0x00001E49
+	"\x00N\x03-\x00\x00\x1eJ" + // 0x004E032D: 0x00001E4A
+	"\x00n\x03-\x00\x00\x1eK" + // 0x006E032D: 0x00001E4B
+	"\x00\xd5\x03\x01\x00\x00\x1eL" + // 0x00D50301: 0x00001E4C
+	"\x00\xf5\x03\x01\x00\x00\x1eM" + // 0x00F50301: 0x00001E4D
+	"\x00\xd5\x03\b\x00\x00\x1eN" + // 0x00D50308: 0x00001E4E
+	"\x00\xf5\x03\b\x00\x00\x1eO" + // 0x00F50308: 0x00001E4F
+	"\x01L\x03\x00\x00\x00\x1eP" + // 0x014C0300: 0x00001E50
+	"\x01M\x03\x00\x00\x00\x1eQ" + // 0x014D0300: 0x00001E51
+	"\x01L\x03\x01\x00\x00\x1eR" + // 0x014C0301: 0x00001E52
+	"\x01M\x03\x01\x00\x00\x1eS" + // 0x014D0301: 0x00001E53
+	"\x00P\x03\x01\x00\x00\x1eT" + // 0x00500301: 0x00001E54
+	"\x00p\x03\x01\x00\x00\x1eU" + // 0x00700301: 0x00001E55
+	"\x00P\x03\a\x00\x00\x1eV" + // 0x00500307: 0x00001E56
+	"\x00p\x03\a\x00\x00\x1eW" + // 0x00700307: 0x00001E57
+	"\x00R\x03\a\x00\x00\x1eX" + // 0x00520307: 0x00001E58
+	"\x00r\x03\a\x00\x00\x1eY" + // 0x00720307: 0x00001E59
+	"\x00R\x03#\x00\x00\x1eZ" + // 0x00520323: 0x00001E5A
+	"\x00r\x03#\x00\x00\x1e[" + // 0x00720323: 0x00001E5B
+	"\x1eZ\x03\x04\x00\x00\x1e\\" + // 0x1E5A0304: 0x00001E5C
+	"\x1e[\x03\x04\x00\x00\x1e]" + // 0x1E5B0304: 0x00001E5D
+	"\x00R\x031\x00\x00\x1e^" + // 0x00520331: 0x00001E5E
+	"\x00r\x031\x00\x00\x1e_" + // 0x00720331: 0x00001E5F
+	"\x00S\x03\a\x00\x00\x1e`" + // 0x00530307: 0x00001E60
+	"\x00s\x03\a\x00\x00\x1ea" + // 0x00730307: 0x00001E61
+	"\x00S\x03#\x00\x00\x1eb" + // 0x00530323: 0x00001E62
+	"\x00s\x03#\x00\x00\x1ec" + // 0x00730323: 0x00001E63
+	"\x01Z\x03\a\x00\x00\x1ed" + // 0x015A0307: 0x00001E64
+	"\x01[\x03\a\x00\x00\x1ee" + // 0x015B0307: 0x00001E65
+	"\x01`\x03\a\x00\x00\x1ef" + // 0x01600307: 0x00001E66
+	"\x01a\x03\a\x00\x00\x1eg" + // 0x01610307: 0x00001E67
+	"\x1eb\x03\a\x00\x00\x1eh" + // 0x1E620307: 0x00001E68
+	"\x1ec\x03\a\x00\x00\x1ei" + // 0x1E630307: 0x00001E69
+	"\x00T\x03\a\x00\x00\x1ej" + // 0x00540307: 0x00001E6A
+	"\x00t\x03\a\x00\x00\x1ek" + // 0x00740307: 0x00001E6B
+	"\x00T\x03#\x00\x00\x1el" + // 0x00540323: 0x00001E6C
+	"\x00t\x03#\x00\x00\x1em" + // 0x00740323: 0x00001E6D
+	"\x00T\x031\x00\x00\x1en" + // 0x00540331: 0x00001E6E
+	"\x00t\x031\x00\x00\x1eo" + // 0x00740331: 0x00001E6F
+	"\x00T\x03-\x00\x00\x1ep" + // 0x0054032D: 0x00001E70
+	"\x00t\x03-\x00\x00\x1eq" + // 0x0074032D: 0x00001E71
+	"\x00U\x03$\x00\x00\x1er" + // 0x00550324: 0x00001E72
+	"\x00u\x03$\x00\x00\x1es" + // 0x00750324: 0x00001E73
+	"\x00U\x030\x00\x00\x1et" + // 0x00550330: 0x00001E74
+	"\x00u\x030\x00\x00\x1eu" + // 0x00750330: 0x00001E75
+	"\x00U\x03-\x00\x00\x1ev" + // 0x0055032D: 0x00001E76
+	"\x00u\x03-\x00\x00\x1ew" + // 0x0075032D: 0x00001E77
+	"\x01h\x03\x01\x00\x00\x1ex" + // 0x01680301: 0x00001E78
+	"\x01i\x03\x01\x00\x00\x1ey" + // 0x01690301: 0x00001E79
+	"\x01j\x03\b\x00\x00\x1ez" + // 0x016A0308: 0x00001E7A
+	"\x01k\x03\b\x00\x00\x1e{" + // 0x016B0308: 0x00001E7B
+	"\x00V\x03\x03\x00\x00\x1e|" + // 0x00560303: 0x00001E7C
+	"\x00v\x03\x03\x00\x00\x1e}" + // 0x00760303: 0x00001E7D
+	"\x00V\x03#\x00\x00\x1e~" + // 0x00560323: 0x00001E7E
+	"\x00v\x03#\x00\x00\x1e\u007f" + // 0x00760323: 0x00001E7F
+	"\x00W\x03\x00\x00\x00\x1e\x80" + // 0x00570300: 0x00001E80
+	"\x00w\x03\x00\x00\x00\x1e\x81" + // 0x00770300: 0x00001E81
+	"\x00W\x03\x01\x00\x00\x1e\x82" + // 0x00570301: 0x00001E82
+	"\x00w\x03\x01\x00\x00\x1e\x83" + // 0x00770301: 0x00001E83
+	"\x00W\x03\b\x00\x00\x1e\x84" + // 0x00570308: 0x00001E84
+	"\x00w\x03\b\x00\x00\x1e\x85" + // 0x00770308: 0x00001E85
+	"\x00W\x03\a\x00\x00\x1e\x86" + // 0x00570307: 0x00001E86
+	"\x00w\x03\a\x00\x00\x1e\x87" + // 0x00770307: 0x00001E87
+	"\x00W\x03#\x00\x00\x1e\x88" + // 0x00570323: 0x00001E88
+	"\x00w\x03#\x00\x00\x1e\x89" + // 0x00770323: 0x00001E89
+	"\x00X\x03\a\x00\x00\x1e\x8a" + // 0x00580307: 0x00001E8A
+	"\x00x\x03\a\x00\x00\x1e\x8b" + // 0x00780307: 0x00001E8B
+	"\x00X\x03\b\x00\x00\x1e\x8c" + // 0x00580308: 0x00001E8C
+	"\x00x\x03\b\x00\x00\x1e\x8d" + // 0x00780308: 0x00001E8D
+	"\x00Y\x03\a\x00\x00\x1e\x8e" + // 0x00590307: 0x00001E8E
+	"\x00y\x03\a\x00\x00\x1e\x8f" + // 0x00790307: 0x00001E8F
+	"\x00Z\x03\x02\x00\x00\x1e\x90" + // 0x005A0302: 0x00001E90
+	"\x00z\x03\x02\x00\x00\x1e\x91" + // 0x007A0302: 0x00001E91
+	"\x00Z\x03#\x00\x00\x1e\x92" + // 0x005A0323: 0x00001E92
+	"\x00z\x03#\x00\x00\x1e\x93" + // 0x007A0323: 0x00001E93
+	"\x00Z\x031\x00\x00\x1e\x94" + // 0x005A0331: 0x00001E94
+	"\x00z\x031\x00\x00\x1e\x95" + // 0x007A0331: 0x00001E95
+	"\x00h\x031\x00\x00\x1e\x96" + // 0x00680331: 0x00001E96
+	"\x00t\x03\b\x00\x00\x1e\x97" + // 0x00740308: 0x00001E97
+	"\x00w\x03\n\x00\x00\x1e\x98" + // 0x0077030A: 0x00001E98
+	"\x00y\x03\n\x00\x00\x1e\x99" + // 0x0079030A: 0x00001E99
+	"\x01\u007f\x03\a\x00\x00\x1e\x9b" + // 0x017F0307: 0x00001E9B
+	"\x00A\x03#\x00\x00\x1e\xa0" + // 0x00410323: 0x00001EA0
+	"\x00a\x03#\x00\x00\x1e\xa1" + // 0x00610323: 0x00001EA1
+	"\x00A\x03\t\x00\x00\x1e\xa2" + // 0x00410309: 0x00001EA2
+	"\x00a\x03\t\x00\x00\x1e\xa3" + // 0x00610309: 0x00001EA3
+	"\x00\xc2\x03\x01\x00\x00\x1e\xa4" + // 0x00C20301: 0x00001EA4
+	"\x00\xe2\x03\x01\x00\x00\x1e\xa5" + // 0x00E20301: 0x00001EA5
+	"\x00\xc2\x03\x00\x00\x00\x1e\xa6" + // 0x00C20300: 0x00001EA6
+	"\x00\xe2\x03\x00\x00\x00\x1e\xa7" + // 0x00E20300: 0x00001EA7
+	"\x00\xc2\x03\t\x00\x00\x1e\xa8" + // 0x00C20309: 0x00001EA8
+	"\x00\xe2\x03\t\x00\x00\x1e\xa9" + // 0x00E20309: 0x00001EA9
+	"\x00\xc2\x03\x03\x00\x00\x1e\xaa" + // 0x00C20303: 0x00001EAA
+	"\x00\xe2\x03\x03\x00\x00\x1e\xab" + // 0x00E20303: 0x00001EAB
+	"\x1e\xa0\x03\x02\x00\x00\x1e\xac" + // 0x1EA00302: 0x00001EAC
+	"\x1e\xa1\x03\x02\x00\x00\x1e\xad" + // 0x1EA10302: 0x00001EAD
+	"\x01\x02\x03\x01\x00\x00\x1e\xae" + // 0x01020301: 0x00001EAE
+	"\x01\x03\x03\x01\x00\x00\x1e\xaf" + // 0x01030301: 0x00001EAF
+	"\x01\x02\x03\x00\x00\x00\x1e\xb0" + // 0x01020300: 0x00001EB0
+	"\x01\x03\x03\x00\x00\x00\x1e\xb1" + // 0x01030300: 0x00001EB1
+	"\x01\x02\x03\t\x00\x00\x1e\xb2" + // 0x01020309: 0x00001EB2
+	"\x01\x03\x03\t\x00\x00\x1e\xb3" + // 0x01030309: 0x00001EB3
+	"\x01\x02\x03\x03\x00\x00\x1e\xb4" + // 0x01020303: 0x00001EB4
+	"\x01\x03\x03\x03\x00\x00\x1e\xb5" + // 0x01030303: 0x00001EB5
+	"\x1e\xa0\x03\x06\x00\x00\x1e\xb6" + // 0x1EA00306: 0x00001EB6
+	"\x1e\xa1\x03\x06\x00\x00\x1e\xb7" + // 0x1EA10306: 0x00001EB7
+	"\x00E\x03#\x00\x00\x1e\xb8" + // 0x00450323: 0x00001EB8
+	"\x00e\x03#\x00\x00\x1e\xb9" + // 0x00650323: 0x00001EB9
+	"\x00E\x03\t\x00\x00\x1e\xba" + // 0x00450309: 0x00001EBA
+	"\x00e\x03\t\x00\x00\x1e\xbb" + // 0x00650309: 0x00001EBB
+	"\x00E\x03\x03\x00\x00\x1e\xbc" + // 0x00450303: 0x00001EBC
+	"\x00e\x03\x03\x00\x00\x1e\xbd" + // 0x00650303: 0x00001EBD
+	"\x00\xca\x03\x01\x00\x00\x1e\xbe" + // 0x00CA0301: 0x00001EBE
+	"\x00\xea\x03\x01\x00\x00\x1e\xbf" + // 0x00EA0301: 0x00001EBF
+	"\x00\xca\x03\x00\x00\x00\x1e\xc0" + // 0x00CA0300: 0x00001EC0
+	"\x00\xea\x03\x00\x00\x00\x1e\xc1" + // 0x00EA0300: 0x00001EC1
+	"\x00\xca\x03\t\x00\x00\x1e\xc2" + // 0x00CA0309: 0x00001EC2
+	"\x00\xea\x03\t\x00\x00\x1e\xc3" + // 0x00EA0309: 0x00001EC3
+	"\x00\xca\x03\x03\x00\x00\x1e\xc4" + // 0x00CA0303: 0x00001EC4
+	"\x00\xea\x03\x03\x00\x00\x1e\xc5" + // 0x00EA0303: 0x00001EC5
+	"\x1e\xb8\x03\x02\x00\x00\x1e\xc6" + // 0x1EB80302: 0x00001EC6
+	"\x1e\xb9\x03\x02\x00\x00\x1e\xc7" + // 0x1EB90302: 0x00001EC7
+	"\x00I\x03\t\x00\x00\x1e\xc8" + // 0x00490309: 0x00001EC8
+	"\x00i\x03\t\x00\x00\x1e\xc9" + // 0x00690309: 0x00001EC9
+	"\x00I\x03#\x00\x00\x1e\xca" + // 0x00490323: 0x00001ECA
+	"\x00i\x03#\x00\x00\x1e\xcb" + // 0x00690323: 0x00001ECB
+	"\x00O\x03#\x00\x00\x1e\xcc" + // 0x004F0323: 0x00001ECC
+	"\x00o\x03#\x00\x00\x1e\xcd" + // 0x006F0323: 0x00001ECD
+	"\x00O\x03\t\x00\x00\x1e\xce" + // 0x004F0309: 0x00001ECE
+	"\x00o\x03\t\x00\x00\x1e\xcf" + // 0x006F0309: 0x00001ECF
+	"\x00\xd4\x03\x01\x00\x00\x1e\xd0" + // 0x00D40301: 0x00001ED0
+	"\x00\xf4\x03\x01\x00\x00\x1e\xd1" + // 0x00F40301: 0x00001ED1
+	"\x00\xd4\x03\x00\x00\x00\x1e\xd2" + // 0x00D40300: 0x00001ED2
+	"\x00\xf4\x03\x00\x00\x00\x1e\xd3" + // 0x00F40300: 0x00001ED3
+	"\x00\xd4\x03\t\x00\x00\x1e\xd4" + // 0x00D40309: 0x00001ED4
+	"\x00\xf4\x03\t\x00\x00\x1e\xd5" + // 0x00F40309: 0x00001ED5
+	"\x00\xd4\x03\x03\x00\x00\x1e\xd6" + // 0x00D40303: 0x00001ED6
+	"\x00\xf4\x03\x03\x00\x00\x1e\xd7" + // 0x00F40303: 0x00001ED7
+	"\x1e\xcc\x03\x02\x00\x00\x1e\xd8" + // 0x1ECC0302: 0x00001ED8
+	"\x1e\xcd\x03\x02\x00\x00\x1e\xd9" + // 0x1ECD0302: 0x00001ED9
+	"\x01\xa0\x03\x01\x00\x00\x1e\xda" + // 0x01A00301: 0x00001EDA
+	"\x01\xa1\x03\x01\x00\x00\x1e\xdb" + // 0x01A10301: 0x00001EDB
+	"\x01\xa0\x03\x00\x00\x00\x1e\xdc" + // 0x01A00300: 0x00001EDC
+	"\x01\xa1\x03\x00\x00\x00\x1e\xdd" + // 0x01A10300: 0x00001EDD
+	"\x01\xa0\x03\t\x00\x00\x1e\xde" + // 0x01A00309: 0x00001EDE
+	"\x01\xa1\x03\t\x00\x00\x1e\xdf" + // 0x01A10309: 0x00001EDF
+	"\x01\xa0\x03\x03\x00\x00\x1e\xe0" + // 0x01A00303: 0x00001EE0
+	"\x01\xa1\x03\x03\x00\x00\x1e\xe1" + // 0x01A10303: 0x00001EE1
+	"\x01\xa0\x03#\x00\x00\x1e\xe2" + // 0x01A00323: 0x00001EE2
+	"\x01\xa1\x03#\x00\x00\x1e\xe3" + // 0x01A10323: 0x00001EE3
+	"\x00U\x03#\x00\x00\x1e\xe4" + // 0x00550323: 0x00001EE4
+	"\x00u\x03#\x00\x00\x1e\xe5" + // 0x00750323: 0x00001EE5
+	"\x00U\x03\t\x00\x00\x1e\xe6" + // 0x00550309: 0x00001EE6
+	"\x00u\x03\t\x00\x00\x1e\xe7" + // 0x00750309: 0x00001EE7
+	"\x01\xaf\x03\x01\x00\x00\x1e\xe8" + // 0x01AF0301: 0x00001EE8
+	"\x01\xb0\x03\x01\x00\x00\x1e\xe9" + // 0x01B00301: 0x00001EE9
+	"\x01\xaf\x03\x00\x00\x00\x1e\xea" + // 0x01AF0300: 0x00001EEA
+	"\x01\xb0\x03\x00\x00\x00\x1e\xeb" + // 0x01B00300: 0x00001EEB
+	"\x01\xaf\x03\t\x00\x00\x1e\xec" + // 0x01AF0309: 0x00001EEC
+	"\x01\xb0\x03\t\x00\x00\x1e\xed" + // 0x01B00309: 0x00001EED
+	"\x01\xaf\x03\x03\x00\x00\x1e\xee" + // 0x01AF0303: 0x00001EEE
+	"\x01\xb0\x03\x03\x00\x00\x1e\xef" + // 0x01B00303: 0x00001EEF
+	"\x01\xaf\x03#\x00\x00\x1e\xf0" + // 0x01AF0323: 0x00001EF0
+	"\x01\xb0\x03#\x00\x00\x1e\xf1" + // 0x01B00323: 0x00001EF1
+	"\x00Y\x03\x00\x00\x00\x1e\xf2" + // 0x00590300: 0x00001EF2
+	"\x00y\x03\x00\x00\x00\x1e\xf3" + // 0x00790300: 0x00001EF3
+	"\x00Y\x03#\x00\x00\x1e\xf4" + // 0x00590323: 0x00001EF4
+	"\x00y\x03#\x00\x00\x1e\xf5" + // 0x00790323: 0x00001EF5
+	"\x00Y\x03\t\x00\x00\x1e\xf6" + // 0x00590309: 0x00001EF6
+	"\x00y\x03\t\x00\x00\x1e\xf7" + // 0x00790309: 0x00001EF7
+	"\x00Y\x03\x03\x00\x00\x1e\xf8" + // 0x00590303: 0x00001EF8
+	"\x00y\x03\x03\x00\x00\x1e\xf9" + // 0x00790303: 0x00001EF9
+	"\x03\xb1\x03\x13\x00\x00\x1f\x00" + // 0x03B10313: 0x00001F00
+	"\x03\xb1\x03\x14\x00\x00\x1f\x01" + // 0x03B10314: 0x00001F01
+	"\x1f\x00\x03\x00\x00\x00\x1f\x02" + // 0x1F000300: 0x00001F02
+	"\x1f\x01\x03\x00\x00\x00\x1f\x03" + // 0x1F010300: 0x00001F03
+	"\x1f\x00\x03\x01\x00\x00\x1f\x04" + // 0x1F000301: 0x00001F04
+	"\x1f\x01\x03\x01\x00\x00\x1f\x05" + // 0x1F010301: 0x00001F05
+	"\x1f\x00\x03B\x00\x00\x1f\x06" + // 0x1F000342: 0x00001F06
+	"\x1f\x01\x03B\x00\x00\x1f\a" + // 0x1F010342: 0x00001F07
+	"\x03\x91\x03\x13\x00\x00\x1f\b" + // 0x03910313: 0x00001F08
+	"\x03\x91\x03\x14\x00\x00\x1f\t" + // 0x03910314: 0x00001F09
+	"\x1f\b\x03\x00\x00\x00\x1f\n" + // 0x1F080300: 0x00001F0A
+	"\x1f\t\x03\x00\x00\x00\x1f\v" + // 0x1F090300: 0x00001F0B
+	"\x1f\b\x03\x01\x00\x00\x1f\f" + // 0x1F080301: 0x00001F0C
+	"\x1f\t\x03\x01\x00\x00\x1f\r" + // 0x1F090301: 0x00001F0D
+	"\x1f\b\x03B\x00\x00\x1f\x0e" + // 0x1F080342: 0x00001F0E
+	"\x1f\t\x03B\x00\x00\x1f\x0f" + // 0x1F090342: 0x00001F0F
+	"\x03\xb5\x03\x13\x00\x00\x1f\x10" + // 0x03B50313: 0x00001F10
+	"\x03\xb5\x03\x14\x00\x00\x1f\x11" + // 0x03B50314: 0x00001F11
+	"\x1f\x10\x03\x00\x00\x00\x1f\x12" + // 0x1F100300: 0x00001F12
+	"\x1f\x11\x03\x00\x00\x00\x1f\x13" + // 0x1F110300: 0x00001F13
+	"\x1f\x10\x03\x01\x00\x00\x1f\x14" + // 0x1F100301: 0x00001F14
+	"\x1f\x11\x03\x01\x00\x00\x1f\x15" + // 0x1F110301: 0x00001F15
+	"\x03\x95\x03\x13\x00\x00\x1f\x18" + // 0x03950313: 0x00001F18
+	"\x03\x95\x03\x14\x00\x00\x1f\x19" + // 0x03950314: 0x00001F19
+	"\x1f\x18\x03\x00\x00\x00\x1f\x1a" + // 0x1F180300: 0x00001F1A
+	"\x1f\x19\x03\x00\x00\x00\x1f\x1b" + // 0x1F190300: 0x00001F1B
+	"\x1f\x18\x03\x01\x00\x00\x1f\x1c" + // 0x1F180301: 0x00001F1C
+	"\x1f\x19\x03\x01\x00\x00\x1f\x1d" + // 0x1F190301: 0x00001F1D
+	"\x03\xb7\x03\x13\x00\x00\x1f " + // 0x03B70313: 0x00001F20
+	"\x03\xb7\x03\x14\x00\x00\x1f!" + // 0x03B70314: 0x00001F21
+	"\x1f \x03\x00\x00\x00\x1f\"" + // 0x1F200300: 0x00001F22
+	"\x1f!\x03\x00\x00\x00\x1f#" + // 0x1F210300: 0x00001F23
+	"\x1f \x03\x01\x00\x00\x1f$" + // 0x1F200301: 0x00001F24
+	"\x1f!\x03\x01\x00\x00\x1f%" + // 0x1F210301: 0x00001F25
+	"\x1f \x03B\x00\x00\x1f&" + // 0x1F200342: 0x00001F26
+	"\x1f!\x03B\x00\x00\x1f'" + // 0x1F210342: 0x00001F27
+	"\x03\x97\x03\x13\x00\x00\x1f(" + // 0x03970313: 0x00001F28
+	"\x03\x97\x03\x14\x00\x00\x1f)" + // 0x03970314: 0x00001F29
+	"\x1f(\x03\x00\x00\x00\x1f*" + // 0x1F280300: 0x00001F2A
+	"\x1f)\x03\x00\x00\x00\x1f+" + // 0x1F290300: 0x00001F2B
+	"\x1f(\x03\x01\x00\x00\x1f," + // 0x1F280301: 0x00001F2C
+	"\x1f)\x03\x01\x00\x00\x1f-" + // 0x1F290301: 0x00001F2D
+	"\x1f(\x03B\x00\x00\x1f." + // 0x1F280342: 0x00001F2E
+	"\x1f)\x03B\x00\x00\x1f/" + // 0x1F290342: 0x00001F2F
+	"\x03\xb9\x03\x13\x00\x00\x1f0" + // 0x03B90313: 0x00001F30
+	"\x03\xb9\x03\x14\x00\x00\x1f1" + // 0x03B90314: 0x00001F31
+	"\x1f0\x03\x00\x00\x00\x1f2" + // 0x1F300300: 0x00001F32
+	"\x1f1\x03\x00\x00\x00\x1f3" + // 0x1F310300: 0x00001F33
+	"\x1f0\x03\x01\x00\x00\x1f4" + // 0x1F300301: 0x00001F34
+	"\x1f1\x03\x01\x00\x00\x1f5" + // 0x1F310301: 0x00001F35
+	"\x1f0\x03B\x00\x00\x1f6" + // 0x1F300342: 0x00001F36
+	"\x1f1\x03B\x00\x00\x1f7" + // 0x1F310342: 0x00001F37
+	"\x03\x99\x03\x13\x00\x00\x1f8" + // 0x03990313: 0x00001F38
+	"\x03\x99\x03\x14\x00\x00\x1f9" + // 0x03990314: 0x00001F39
+	"\x1f8\x03\x00\x00\x00\x1f:" + // 0x1F380300: 0x00001F3A
+	"\x1f9\x03\x00\x00\x00\x1f;" + // 0x1F390300: 0x00001F3B
+	"\x1f8\x03\x01\x00\x00\x1f<" + // 0x1F380301: 0x00001F3C
+	"\x1f9\x03\x01\x00\x00\x1f=" + // 0x1F390301: 0x00001F3D
+	"\x1f8\x03B\x00\x00\x1f>" + // 0x1F380342: 0x00001F3E
+	"\x1f9\x03B\x00\x00\x1f?" + // 0x1F390342: 0x00001F3F
+	"\x03\xbf\x03\x13\x00\x00\x1f@" + // 0x03BF0313: 0x00001F40
+	"\x03\xbf\x03\x14\x00\x00\x1fA" + // 0x03BF0314: 0x00001F41
+	"\x1f@\x03\x00\x00\x00\x1fB" + // 0x1F400300: 0x00001F42
+	"\x1fA\x03\x00\x00\x00\x1fC" + // 0x1F410300: 0x00001F43
+	"\x1f@\x03\x01\x00\x00\x1fD" + // 0x1F400301: 0x00001F44
+	"\x1fA\x03\x01\x00\x00\x1fE" + // 0x1F410301: 0x00001F45
+	"\x03\x9f\x03\x13\x00\x00\x1fH" + // 0x039F0313: 0x00001F48
+	"\x03\x9f\x03\x14\x00\x00\x1fI" + // 0x039F0314: 0x00001F49
+	"\x1fH\x03\x00\x00\x00\x1fJ" + // 0x1F480300: 0x00001F4A
+	"\x1fI\x03\x00\x00\x00\x1fK" + // 0x1F490300: 0x00001F4B
+	"\x1fH\x03\x01\x00\x00\x1fL" + // 0x1F480301: 0x00001F4C
+	"\x1fI\x03\x01\x00\x00\x1fM" + // 0x1F490301: 0x00001F4D
+	"\x03\xc5\x03\x13\x00\x00\x1fP" + // 0x03C50313: 0x00001F50
+	"\x03\xc5\x03\x14\x00\x00\x1fQ" + // 0x03C50314: 0x00001F51
+	"\x1fP\x03\x00\x00\x00\x1fR" + // 0x1F500300: 0x00001F52
+	"\x1fQ\x03\x00\x00\x00\x1fS" + // 0x1F510300: 0x00001F53
+	"\x1fP\x03\x01\x00\x00\x1fT" + // 0x1F500301: 0x00001F54
+	"\x1fQ\x03\x01\x00\x00\x1fU" + // 0x1F510301: 0x00001F55
+	"\x1fP\x03B\x00\x00\x1fV" + // 0x1F500342: 0x00001F56
+	"\x1fQ\x03B\x00\x00\x1fW" + // 0x1F510342: 0x00001F57
+	"\x03\xa5\x03\x14\x00\x00\x1fY" + // 0x03A50314: 0x00001F59
+	"\x1fY\x03\x00\x00\x00\x1f[" + // 0x1F590300: 0x00001F5B
+	"\x1fY\x03\x01\x00\x00\x1f]" + // 0x1F590301: 0x00001F5D
+	"\x1fY\x03B\x00\x00\x1f_" + // 0x1F590342: 0x00001F5F
+	"\x03\xc9\x03\x13\x00\x00\x1f`" + // 0x03C90313: 0x00001F60
+	"\x03\xc9\x03\x14\x00\x00\x1fa" + // 0x03C90314: 0x00001F61
+	"\x1f`\x03\x00\x00\x00\x1fb" + // 0x1F600300: 0x00001F62
+	"\x1fa\x03\x00\x00\x00\x1fc" + // 0x1F610300: 0x00001F63
+	"\x1f`\x03\x01\x00\x00\x1fd" + // 0x1F600301: 0x00001F64
+	"\x1fa\x03\x01\x00\x00\x1fe" + // 0x1F610301: 0x00001F65
+	"\x1f`\x03B\x00\x00\x1ff" + // 0x1F600342: 0x00001F66
+	"\x1fa\x03B\x00\x00\x1fg" + // 0x1F610342: 0x00001F67
+	"\x03\xa9\x03\x13\x00\x00\x1fh" + // 0x03A90313: 0x00001F68
+	"\x03\xa9\x03\x14\x00\x00\x1fi" + // 0x03A90314: 0x00001F69
+	"\x1fh\x03\x00\x00\x00\x1fj" + // 0x1F680300: 0x00001F6A
+	"\x1fi\x03\x00\x00\x00\x1fk" + // 0x1F690300: 0x00001F6B
+	"\x1fh\x03\x01\x00\x00\x1fl" + // 0x1F680301: 0x00001F6C
+	"\x1fi\x03\x01\x00\x00\x1fm" + // 0x1F690301: 0x00001F6D
+	"\x1fh\x03B\x00\x00\x1fn" + // 0x1F680342: 0x00001F6E
+	"\x1fi\x03B\x00\x00\x1fo" + // 0x1F690342: 0x00001F6F
+	"\x03\xb1\x03\x00\x00\x00\x1fp" + // 0x03B10300: 0x00001F70
+	"\x03\xb5\x03\x00\x00\x00\x1fr" + // 0x03B50300: 0x00001F72
+	"\x03\xb7\x03\x00\x00\x00\x1ft" + // 0x03B70300: 0x00001F74
+	"\x03\xb9\x03\x00\x00\x00\x1fv" + // 0x03B90300: 0x00001F76
+	"\x03\xbf\x03\x00\x00\x00\x1fx" + // 0x03BF0300: 0x00001F78
+	"\x03\xc5\x03\x00\x00\x00\x1fz" + // 0x03C50300: 0x00001F7A
+	"\x03\xc9\x03\x00\x00\x00\x1f|" + // 0x03C90300: 0x00001F7C
+	"\x1f\x00\x03E\x00\x00\x1f\x80" + // 0x1F000345: 0x00001F80
+	"\x1f\x01\x03E\x00\x00\x1f\x81" + // 0x1F010345: 0x00001F81
+	"\x1f\x02\x03E\x00\x00\x1f\x82" + // 0x1F020345: 0x00001F82
+	"\x1f\x03\x03E\x00\x00\x1f\x83" + // 0x1F030345: 0x00001F83
+	"\x1f\x04\x03E\x00\x00\x1f\x84" + // 0x1F040345: 0x00001F84
+	"\x1f\x05\x03E\x00\x00\x1f\x85" + // 0x1F050345: 0x00001F85
+	"\x1f\x06\x03E\x00\x00\x1f\x86" + // 0x1F060345: 0x00001F86
+	"\x1f\a\x03E\x00\x00\x1f\x87" + // 0x1F070345: 0x00001F87
+	"\x1f\b\x03E\x00\x00\x1f\x88" + // 0x1F080345: 0x00001F88
+	"\x1f\t\x03E\x00\x00\x1f\x89" + // 0x1F090345: 0x00001F89
+	"\x1f\n\x03E\x00\x00\x1f\x8a" + // 0x1F0A0345: 0x00001F8A
+	"\x1f\v\x03E\x00\x00\x1f\x8b" + // 0x1F0B0345: 0x00001F8B
+	"\x1f\f\x03E\x00\x00\x1f\x8c" + // 0x1F0C0345: 0x00001F8C
+	"\x1f\r\x03E\x00\x00\x1f\x8d" + // 0x1F0D0345: 0x00001F8D
+	"\x1f\x0e\x03E\x00\x00\x1f\x8e" + // 0x1F0E0345: 0x00001F8E
+	"\x1f\x0f\x03E\x00\x00\x1f\x8f" + // 0x1F0F0345: 0x00001F8F
+	"\x1f \x03E\x00\x00\x1f\x90" + // 0x1F200345: 0x00001F90
+	"\x1f!\x03E\x00\x00\x1f\x91" + // 0x1F210345: 0x00001F91
+	"\x1f\"\x03E\x00\x00\x1f\x92" + // 0x1F220345: 0x00001F92
+	"\x1f#\x03E\x00\x00\x1f\x93" + // 0x1F230345: 0x00001F93
+	"\x1f$\x03E\x00\x00\x1f\x94" + // 0x1F240345: 0x00001F94
+	"\x1f%\x03E\x00\x00\x1f\x95" + // 0x1F250345: 0x00001F95
+	"\x1f&\x03E\x00\x00\x1f\x96" + // 0x1F260345: 0x00001F96
+	"\x1f'\x03E\x00\x00\x1f\x97" + // 0x1F270345: 0x00001F97
+	"\x1f(\x03E\x00\x00\x1f\x98" + // 0x1F280345: 0x00001F98
+	"\x1f)\x03E\x00\x00\x1f\x99" + // 0x1F290345: 0x00001F99
+	"\x1f*\x03E\x00\x00\x1f\x9a" + // 0x1F2A0345: 0x00001F9A
+	"\x1f+\x03E\x00\x00\x1f\x9b" + // 0x1F2B0345: 0x00001F9B
+	"\x1f,\x03E\x00\x00\x1f\x9c" + // 0x1F2C0345: 0x00001F9C
+	"\x1f-\x03E\x00\x00\x1f\x9d" + // 0x1F2D0345: 0x00001F9D
+	"\x1f.\x03E\x00\x00\x1f\x9e" + // 0x1F2E0345: 0x00001F9E
+	"\x1f/\x03E\x00\x00\x1f\x9f" + // 0x1F2F0345: 0x00001F9F
+	"\x1f`\x03E\x00\x00\x1f\xa0" + // 0x1F600345: 0x00001FA0
+	"\x1fa\x03E\x00\x00\x1f\xa1" + // 0x1F610345: 0x00001FA1
+	"\x1fb\x03E\x00\x00\x1f\xa2" + // 0x1F620345: 0x00001FA2
+	"\x1fc\x03E\x00\x00\x1f\xa3" + // 0x1F630345: 0x00001FA3
+	"\x1fd\x03E\x00\x00\x1f\xa4" + // 0x1F640345: 0x00001FA4
+	"\x1fe\x03E\x00\x00\x1f\xa5" + // 0x1F650345: 0x00001FA5
+	"\x1ff\x03E\x00\x00\x1f\xa6" + // 0x1F660345: 0x00001FA6
+	"\x1fg\x03E\x00\x00\x1f\xa7" + // 0x1F670345: 0x00001FA7
+	"\x1fh\x03E\x00\x00\x1f\xa8" + // 0x1F680345: 0x00001FA8
+	"\x1fi\x03E\x00\x00\x1f\xa9" + // 0x1F690345: 0x00001FA9
+	"\x1fj\x03E\x00\x00\x1f\xaa" + // 0x1F6A0345: 0x00001FAA
+	"\x1fk\x03E\x00\x00\x1f\xab" + // 0x1F6B0345: 0x00001FAB
+	"\x1fl\x03E\x00\x00\x1f\xac" + // 0x1F6C0345: 0x00001FAC
+	"\x1fm\x03E\x00\x00\x1f\xad" + // 0x1F6D0345: 0x00001FAD
+	"\x1fn\x03E\x00\x00\x1f\xae" + // 0x1F6E0345: 0x00001FAE
+	"\x1fo\x03E\x00\x00\x1f\xaf" + // 0x1F6F0345: 0x00001FAF
+	"\x03\xb1\x03\x06\x00\x00\x1f\xb0" + // 0x03B10306: 0x00001FB0
+	"\x03\xb1\x03\x04\x00\x00\x1f\xb1" + // 0x03B10304: 0x00001FB1
+	"\x1fp\x03E\x00\x00\x1f\xb2" + // 0x1F700345: 0x00001FB2
+	"\x03\xb1\x03E\x00\x00\x1f\xb3" + // 0x03B10345: 0x00001FB3
+	"\x03\xac\x03E\x00\x00\x1f\xb4" + // 0x03AC0345: 0x00001FB4
+	"\x03\xb1\x03B\x00\x00\x1f\xb6" + // 0x03B10342: 0x00001FB6
+	"\x1f\xb6\x03E\x00\x00\x1f\xb7" + // 0x1FB60345: 0x00001FB7
+	"\x03\x91\x03\x06\x00\x00\x1f\xb8" + // 0x03910306: 0x00001FB8
+	"\x03\x91\x03\x04\x00\x00\x1f\xb9" + // 0x03910304: 0x00001FB9
+	"\x03\x91\x03\x00\x00\x00\x1f\xba" + // 0x03910300: 0x00001FBA
+	"\x03\x91\x03E\x00\x00\x1f\xbc" + // 0x03910345: 0x00001FBC
+	"\x00\xa8\x03B\x00\x00\x1f\xc1" + // 0x00A80342: 0x00001FC1
+	"\x1ft\x03E\x00\x00\x1f\xc2" + // 0x1F740345: 0x00001FC2
+	"\x03\xb7\x03E\x00\x00\x1f\xc3" + // 0x03B70345: 0x00001FC3
+	"\x03\xae\x03E\x00\x00\x1f\xc4" + // 0x03AE0345: 0x00001FC4
+	"\x03\xb7\x03B\x00\x00\x1f\xc6" + // 0x03B70342: 0x00001FC6
+	"\x1f\xc6\x03E\x00\x00\x1f\xc7" + // 0x1FC60345: 0x00001FC7
+	"\x03\x95\x03\x00\x00\x00\x1f\xc8" + // 0x03950300: 0x00001FC8
+	"\x03\x97\x03\x00\x00\x00\x1f\xca" + // 0x03970300: 0x00001FCA
+	"\x03\x97\x03E\x00\x00\x1f\xcc" + // 0x03970345: 0x00001FCC
+	"\x1f\xbf\x03\x00\x00\x00\x1f\xcd" + // 0x1FBF0300: 0x00001FCD
+	"\x1f\xbf\x03\x01\x00\x00\x1f\xce" + // 0x1FBF0301: 0x00001FCE
+	"\x1f\xbf\x03B\x00\x00\x1f\xcf" + // 0x1FBF0342: 0x00001FCF
+	"\x03\xb9\x03\x06\x00\x00\x1f\xd0" + // 0x03B90306: 0x00001FD0
+	"\x03\xb9\x03\x04\x00\x00\x1f\xd1" + // 0x03B90304: 0x00001FD1
+	"\x03\xca\x03\x00\x00\x00\x1f\xd2" + // 0x03CA0300: 0x00001FD2
+	"\x03\xb9\x03B\x00\x00\x1f\xd6" + // 0x03B90342: 0x00001FD6
+	"\x03\xca\x03B\x00\x00\x1f\xd7" + // 0x03CA0342: 0x00001FD7
+	"\x03\x99\x03\x06\x00\x00\x1f\xd8" + // 0x03990306: 0x00001FD8
+	"\x03\x99\x03\x04\x00\x00\x1f\xd9" + // 0x03990304: 0x00001FD9
+	"\x03\x99\x03\x00\x00\x00\x1f\xda" + // 0x03990300: 0x00001FDA
+	"\x1f\xfe\x03\x00\x00\x00\x1f\xdd" + // 0x1FFE0300: 0x00001FDD
+	"\x1f\xfe\x03\x01\x00\x00\x1f\xde" + // 0x1FFE0301: 0x00001FDE
+	"\x1f\xfe\x03B\x00\x00\x1f\xdf" + // 0x1FFE0342: 0x00001FDF
+	"\x03\xc5\x03\x06\x00\x00\x1f\xe0" + // 0x03C50306: 0x00001FE0
+	"\x03\xc5\x03\x04\x00\x00\x1f\xe1" + // 0x03C50304: 0x00001FE1
+	"\x03\xcb\x03\x00\x00\x00\x1f\xe2" + // 0x03CB0300: 0x00001FE2
+	"\x03\xc1\x03\x13\x00\x00\x1f\xe4" + // 0x03C10313: 0x00001FE4
+	"\x03\xc1\x03\x14\x00\x00\x1f\xe5" + // 0x03C10314: 0x00001FE5
+	"\x03\xc5\x03B\x00\x00\x1f\xe6" + // 0x03C50342: 0x00001FE6
+	"\x03\xcb\x03B\x00\x00\x1f\xe7" + // 0x03CB0342: 0x00001FE7
+	"\x03\xa5\x03\x06\x00\x00\x1f\xe8" + // 0x03A50306: 0x00001FE8
+	"\x03\xa5\x03\x04\x00\x00\x1f\xe9" + // 0x03A50304: 0x00001FE9
+	"\x03\xa5\x03\x00\x00\x00\x1f\xea" + // 0x03A50300: 0x00001FEA
+	"\x03\xa1\x03\x14\x00\x00\x1f\xec" + // 0x03A10314: 0x00001FEC
+	"\x00\xa8\x03\x00\x00\x00\x1f\xed" + // 0x00A80300: 0x00001FED
+	"\x1f|\x03E\x00\x00\x1f\xf2" + // 0x1F7C0345: 0x00001FF2
+	"\x03\xc9\x03E\x00\x00\x1f\xf3" + // 0x03C90345: 0x00001FF3
+	"\x03\xce\x03E\x00\x00\x1f\xf4" + // 0x03CE0345: 0x00001FF4
+	"\x03\xc9\x03B\x00\x00\x1f\xf6" + // 0x03C90342: 0x00001FF6
+	"\x1f\xf6\x03E\x00\x00\x1f\xf7" + // 0x1FF60345: 0x00001FF7
+	"\x03\x9f\x03\x00\x00\x00\x1f\xf8" + // 0x039F0300: 0x00001FF8
+	"\x03\xa9\x03\x00\x00\x00\x1f\xfa" + // 0x03A90300: 0x00001FFA
+	"\x03\xa9\x03E\x00\x00\x1f\xfc" + // 0x03A90345: 0x00001FFC
+	"!\x90\x038\x00\x00!\x9a" + // 0x21900338: 0x0000219A
+	"!\x92\x038\x00\x00!\x9b" + // 0x21920338: 0x0000219B
+	"!\x94\x038\x00\x00!\xae" + // 0x21940338: 0x000021AE
+	"!\xd0\x038\x00\x00!\xcd" + // 0x21D00338: 0x000021CD
+	"!\xd4\x038\x00\x00!\xce" + // 0x21D40338: 0x000021CE
+	"!\xd2\x038\x00\x00!\xcf" + // 0x21D20338: 0x000021CF
+	"\"\x03\x038\x00\x00\"\x04" + // 0x22030338: 0x00002204
+	"\"\b\x038\x00\x00\"\t" + // 0x22080338: 0x00002209
+	"\"\v\x038\x00\x00\"\f" + // 0x220B0338: 0x0000220C
+	"\"#\x038\x00\x00\"$" + // 0x22230338: 0x00002224
+	"\"%\x038\x00\x00\"&" + // 0x22250338: 0x00002226
+	"\"<\x038\x00\x00\"A" + // 0x223C0338: 0x00002241
+	"\"C\x038\x00\x00\"D" + // 0x22430338: 0x00002244
+	"\"E\x038\x00\x00\"G" + // 0x22450338: 0x00002247
+	"\"H\x038\x00\x00\"I" + // 0x22480338: 0x00002249
+	"\x00=\x038\x00\x00\"`" + // 0x003D0338: 0x00002260
+	"\"a\x038\x00\x00\"b" + // 0x22610338: 0x00002262
+	"\"M\x038\x00\x00\"m" + // 0x224D0338: 0x0000226D
+	"\x00<\x038\x00\x00\"n" + // 0x003C0338: 0x0000226E
+	"\x00>\x038\x00\x00\"o" + // 0x003E0338: 0x0000226F
+	"\"d\x038\x00\x00\"p" + // 0x22640338: 0x00002270
+	"\"e\x038\x00\x00\"q" + // 0x22650338: 0x00002271
+	"\"r\x038\x00\x00\"t" + // 0x22720338: 0x00002274
+	"\"s\x038\x00\x00\"u" + // 0x22730338: 0x00002275
+	"\"v\x038\x00\x00\"x" + // 0x22760338: 0x00002278
+	"\"w\x038\x00\x00\"y" + // 0x22770338: 0x00002279
+	"\"z\x038\x00\x00\"\x80" + // 0x227A0338: 0x00002280
+	"\"{\x038\x00\x00\"\x81" + // 0x227B0338: 0x00002281
+	"\"\x82\x038\x00\x00\"\x84" + // 0x22820338: 0x00002284
+	"\"\x83\x038\x00\x00\"\x85" + // 0x22830338: 0x00002285
+	"\"\x86\x038\x00\x00\"\x88" + // 0x22860338: 0x00002288
+	"\"\x87\x038\x00\x00\"\x89" + // 0x22870338: 0x00002289
+	"\"\xa2\x038\x00\x00\"\xac" + // 0x22A20338: 0x000022AC
+	"\"\xa8\x038\x00\x00\"\xad" + // 0x22A80338: 0x000022AD
+	"\"\xa9\x038\x00\x00\"\xae" + // 0x22A90338: 0x000022AE
+	"\"\xab\x038\x00\x00\"\xaf" + // 0x22AB0338: 0x000022AF
+	"\"|\x038\x00\x00\"\xe0" + // 0x227C0338: 0x000022E0
+	"\"}\x038\x00\x00\"\xe1" + // 0x227D0338: 0x000022E1
+	"\"\x91\x038\x00\x00\"\xe2" + // 0x22910338: 0x000022E2
+	"\"\x92\x038\x00\x00\"\xe3" + // 0x22920338: 0x000022E3
+	"\"\xb2\x038\x00\x00\"\xea" + // 0x22B20338: 0x000022EA
+	"\"\xb3\x038\x00\x00\"\xeb" + // 0x22B30338: 0x000022EB
+	"\"\xb4\x038\x00\x00\"\xec" + // 0x22B40338: 0x000022EC
+	"\"\xb5\x038\x00\x00\"\xed" + // 0x22B50338: 0x000022ED
+	"0K0\x99\x00\x000L" + // 0x304B3099: 0x0000304C
+	"0M0\x99\x00\x000N" + // 0x304D3099: 0x0000304E
+	"0O0\x99\x00\x000P" + // 0x304F3099: 0x00003050
+	"0Q0\x99\x00\x000R" + // 0x30513099: 0x00003052
+	"0S0\x99\x00\x000T" + // 0x30533099: 0x00003054
+	"0U0\x99\x00\x000V" + // 0x30553099: 0x00003056
+	"0W0\x99\x00\x000X" + // 0x30573099: 0x00003058
+	"0Y0\x99\x00\x000Z" + // 0x30593099: 0x0000305A
+	"0[0\x99\x00\x000\\" + // 0x305B3099: 0x0000305C
+	"0]0\x99\x00\x000^" + // 0x305D3099: 0x0000305E
+	"0_0\x99\x00\x000`" + // 0x305F3099: 0x00003060
+	"0a0\x99\x00\x000b" + // 0x30613099: 0x00003062
+	"0d0\x99\x00\x000e" + // 0x30643099: 0x00003065
+	"0f0\x99\x00\x000g" + // 0x30663099: 0x00003067
+	"0h0\x99\x00\x000i" + // 0x30683099: 0x00003069
+	"0o0\x99\x00\x000p" + // 0x306F3099: 0x00003070
+	"0o0\x9a\x00\x000q" + // 0x306F309A: 0x00003071
+	"0r0\x99\x00\x000s" + // 0x30723099: 0x00003073
+	"0r0\x9a\x00\x000t" + // 0x3072309A: 0x00003074
+	"0u0\x99\x00\x000v" + // 0x30753099: 0x00003076
+	"0u0\x9a\x00\x000w" + // 0x3075309A: 0x00003077
+	"0x0\x99\x00\x000y" + // 0x30783099: 0x00003079
+	"0x0\x9a\x00\x000z" + // 0x3078309A: 0x0000307A
+	"0{0\x99\x00\x000|" + // 0x307B3099: 0x0000307C
+	"0{0\x9a\x00\x000}" + // 0x307B309A: 0x0000307D
+	"0F0\x99\x00\x000\x94" + // 0x30463099: 0x00003094
+	"0\x9d0\x99\x00\x000\x9e" + // 0x309D3099: 0x0000309E
+	"0\xab0\x99\x00\x000\xac" + // 0x30AB3099: 0x000030AC
+	"0\xad0\x99\x00\x000\xae" + // 0x30AD3099: 0x000030AE
+	"0\xaf0\x99\x00\x000\xb0" + // 0x30AF3099: 0x000030B0
+	"0\xb10\x99\x00\x000\xb2" + // 0x30B13099: 0x000030B2
+	"0\xb30\x99\x00\x000\xb4" + // 0x30B33099: 0x000030B4
+	"0\xb50\x99\x00\x000\xb6" + // 0x30B53099: 0x000030B6
+	"0\xb70\x99\x00\x000\xb8" + // 0x30B73099: 0x000030B8
+	"0\xb90\x99\x00\x000\xba" + // 0x30B93099: 0x000030BA
+	"0\xbb0\x99\x00\x000\xbc" + // 0x30BB3099: 0x000030BC
+	"0\xbd0\x99\x00\x000\xbe" + // 0x30BD3099: 0x000030BE
+	"0\xbf0\x99\x00\x000\xc0" + // 0x30BF3099: 0x000030C0
+	"0\xc10\x99\x00\x000\xc2" + // 0x30C13099: 0x000030C2
+	"0\xc40\x99\x00\x000\xc5" + // 0x30C43099: 0x000030C5
+	"0\xc60\x99\x00\x000\xc7" + // 0x30C63099: 0x000030C7
+	"0\xc80\x99\x00\x000\xc9" + // 0x30C83099: 0x000030C9
+	"0\xcf0\x99\x00\x000\xd0" + // 0x30CF3099: 0x000030D0
+	"0\xcf0\x9a\x00\x000\xd1" + // 0x30CF309A: 0x000030D1
+	"0\xd20\x99\x00\x000\xd3" + // 0x30D23099: 0x000030D3
+	"0\xd20\x9a\x00\x000\xd4" + // 0x30D2309A: 0x000030D4
+	"0\xd50\x99\x00\x000\xd6" + // 0x30D53099: 0x000030D6
+	"0\xd50\x9a\x00\x000\xd7" + // 0x30D5309A: 0x000030D7
+	"0\xd80\x99\x00\x000\xd9" + // 0x30D83099: 0x000030D9
+	"0\xd80\x9a\x00\x000\xda" + // 0x30D8309A: 0x000030DA
+	"0\xdb0\x99\x00\x000\xdc" + // 0x30DB3099: 0x000030DC
+	"0\xdb0\x9a\x00\x000\xdd" + // 0x30DB309A: 0x000030DD
+	"0\xa60\x99\x00\x000\xf4" + // 0x30A63099: 0x000030F4
+	"0\xef0\x99\x00\x000\xf7" + // 0x30EF3099: 0x000030F7
+	"0\xf00\x99\x00\x000\xf8" + // 0x30F03099: 0x000030F8
+	"0\xf10\x99\x00\x000\xf9" + // 0x30F13099: 0x000030F9
+	"0\xf20\x99\x00\x000\xfa" + // 0x30F23099: 0x000030FA
+	"0\xfd0\x99\x00\x000\xfe" + // 0x30FD3099: 0x000030FE
+	"\x10\x99\x10\xba\x00\x01\x10\x9a" + // 0x109910BA: 0x0001109A
+	"\x10\x9b\x10\xba\x00\x01\x10\x9c" + // 0x109B10BA: 0x0001109C
+	"\x10\xa5\x10\xba\x00\x01\x10\xab" + // 0x10A510BA: 0x000110AB
+	"\x111\x11'\x00\x01\x11." + // 0x11311127: 0x0001112E
+	"\x112\x11'\x00\x01\x11/" + // 0x11321127: 0x0001112F
+	"\x13G\x13>\x00\x01\x13K" + // 0x1347133E: 0x0001134B
+	"\x13G\x13W\x00\x01\x13L" + // 0x13471357: 0x0001134C
+	"\x14\xb9\x14\xba\x00\x01\x14\xbb" + // 0x14B914BA: 0x000114BB
+	"\x14\xb9\x14\xb0\x00\x01\x14\xbc" + // 0x14B914B0: 0x000114BC
+	"\x14\xb9\x14\xbd\x00\x01\x14\xbe" + // 0x14B914BD: 0x000114BE
+	"\x15\xb8\x15\xaf\x00\x01\x15\xba" + // 0x15B815AF: 0x000115BA
+	"\x15\xb9\x15\xaf\x00\x01\x15\xbb" + // 0x15B915AF: 0x000115BB
+	"\x195\x190\x00\x01\x198" + // 0x19351930: 0x00011938
+	""
+	// Total size of tables: 55KB (56160 bytes)
diff --git a/vendor/golang.org/x/text/unicode/norm/tables9.0.0.go b/vendor/golang.org/x/text/unicode/norm/tables9.0.0.go
index 9429069..0175eae 100644
--- a/vendor/golang.org/x/text/unicode/norm/tables9.0.0.go
+++ b/vendor/golang.org/x/text/unicode/norm/tables9.0.0.go
@@ -1,5 +1,6 @@
 // Code generated by running "go generate" in golang.org/x/text. DO NOT EDIT.
 
+//go:build !go1.10
 // +build !go1.10
 
 package norm
diff --git a/vendor/google.golang.org/genproto/googleapis/api/annotations/annotations.pb.go b/vendor/google.golang.org/genproto/googleapis/api/annotations/annotations.pb.go
index bf2f703..191bea4 100644
--- a/vendor/google.golang.org/genproto/googleapis/api/annotations/annotations.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/api/annotations/annotations.pb.go
@@ -1,55 +1,119 @@
+// Copyright 2015 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.26.0
+// 	protoc        v3.12.2
 // source: google/api/annotations.proto
 
 package annotations
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
 
-	proto "github.com/golang/protobuf/proto"
-	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
-// 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
-
-var E_Http = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.MethodOptions)(nil),
-	ExtensionType: (*HttpRule)(nil),
-	Field:         72295728,
-	Name:          "google.api.http",
-	Tag:           "bytes,72295728,opt,name=http",
-	Filename:      "google/api/annotations.proto",
+var file_google_api_annotations_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptorpb.MethodOptions)(nil),
+		ExtensionType: (*HttpRule)(nil),
+		Field:         72295728,
+		Name:          "google.api.http",
+		Tag:           "bytes,72295728,opt,name=http",
+		Filename:      "google/api/annotations.proto",
+	},
 }
 
-func init() {
-	proto.RegisterExtension(E_Http)
+// Extension fields to descriptorpb.MethodOptions.
+var (
+	// See `HttpRule`.
+	//
+	// optional google.api.HttpRule http = 72295728;
+	E_Http = &file_google_api_annotations_proto_extTypes[0]
+)
+
+var File_google_api_annotations_proto protoreflect.FileDescriptor
+
+var file_google_api_annotations_proto_rawDesc = []byte{
+	0x0a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e,
+	0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x1a, 0x15, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x74, 0x74, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x3a, 0x4b, 0x0a, 0x04, 0x68, 0x74, 0x74, 0x70, 0x12, 0x1e, 0x2e, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65,
+	0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xb0, 0xca, 0xbc, 0x22,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70,
+	0x69, 0x2e, 0x48, 0x74, 0x74, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x04, 0x68, 0x74, 0x74, 0x70,
+	0x42, 0x6e, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61,
+	0x70, 0x69, 0x42, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50,
+	0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x41, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67,
+	0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x65, 0x6e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x61, 0x70,
+	0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3b, 0x61, 0x6e,
+	0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0xa2, 0x02, 0x04, 0x47, 0x41, 0x50, 0x49,
+	0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("google/api/annotations.proto", fileDescriptor_c591c5aa9fb79aab) }
+var file_google_api_annotations_proto_goTypes = []interface{}{
+	(*descriptorpb.MethodOptions)(nil), // 0: google.protobuf.MethodOptions
+	(*HttpRule)(nil),                   // 1: google.api.HttpRule
+}
+var file_google_api_annotations_proto_depIdxs = []int32{
+	0, // 0: google.api.http:extendee -> google.protobuf.MethodOptions
+	1, // 1: google.api.http:type_name -> google.api.HttpRule
+	2, // [2:2] is the sub-list for method output_type
+	2, // [2:2] is the sub-list for method input_type
+	1, // [1:2] is the sub-list for extension type_name
+	0, // [0:1] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
 
-var fileDescriptor_c591c5aa9fb79aab = []byte{
-	// 208 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x49, 0xcf, 0xcf, 0x4f,
-	0xcf, 0x49, 0xd5, 0x4f, 0x2c, 0xc8, 0xd4, 0x4f, 0xcc, 0xcb, 0xcb, 0x2f, 0x49, 0x2c, 0xc9, 0xcc,
-	0xcf, 0x2b, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x82, 0xc8, 0xea, 0x25, 0x16, 0x64,
-	0x4a, 0x89, 0x22, 0xa9, 0xcc, 0x28, 0x29, 0x29, 0x80, 0x28, 0x91, 0x52, 0x80, 0x0a, 0x83, 0x79,
-	0x49, 0xa5, 0x69, 0xfa, 0x29, 0xa9, 0xc5, 0xc9, 0x45, 0x99, 0x05, 0x25, 0xf9, 0x45, 0x10, 0x15,
-	0x56, 0xde, 0x5c, 0x2c, 0x20, 0xf5, 0x42, 0x72, 0x7a, 0x50, 0xd3, 0x60, 0x4a, 0xf5, 0x7c, 0x53,
-	0x4b, 0x32, 0xf2, 0x53, 0xfc, 0x0b, 0xc0, 0x56, 0x4a, 0x6c, 0x38, 0xb5, 0x47, 0x49, 0x81, 0x51,
-	0x83, 0xdb, 0x48, 0x44, 0x0f, 0x61, 0xad, 0x9e, 0x47, 0x49, 0x49, 0x41, 0x50, 0x69, 0x4e, 0x6a,
-	0x10, 0xd8, 0x10, 0xa7, 0x3c, 0x2e, 0xbe, 0xe4, 0xfc, 0x5c, 0x24, 0x05, 0x4e, 0x02, 0x8e, 0x08,
-	0x67, 0x07, 0x80, 0x4c, 0x0e, 0x60, 0x8c, 0x72, 0x84, 0xca, 0xa7, 0xe7, 0xe7, 0x24, 0xe6, 0xa5,
-	0xeb, 0xe5, 0x17, 0xa5, 0xeb, 0xa7, 0xa7, 0xe6, 0x81, 0xed, 0xd5, 0x87, 0x48, 0x25, 0x16, 0x64,
-	0x16, 0xa3, 0x7b, 0xda, 0x1a, 0x89, 0xbd, 0x88, 0x89, 0xc5, 0xdd, 0x31, 0xc0, 0x33, 0x89, 0x0d,
-	0xac, 0xc9, 0x18, 0x10, 0x00, 0x00, 0xff, 0xff, 0xe3, 0x29, 0x19, 0x62, 0x28, 0x01, 0x00, 0x00,
+func init() { file_google_api_annotations_proto_init() }
+func file_google_api_annotations_proto_init() {
+	if File_google_api_annotations_proto != nil {
+		return
+	}
+	file_google_api_http_proto_init()
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_api_annotations_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 1,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_api_annotations_proto_goTypes,
+		DependencyIndexes: file_google_api_annotations_proto_depIdxs,
+		ExtensionInfos:    file_google_api_annotations_proto_extTypes,
+	}.Build()
+	File_google_api_annotations_proto = out.File
+	file_google_api_annotations_proto_rawDesc = nil
+	file_google_api_annotations_proto_goTypes = nil
+	file_google_api_annotations_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/genproto/googleapis/api/annotations/client.pb.go b/vendor/google.golang.org/genproto/googleapis/api/annotations/client.pb.go
index 867fc0c..66fdb65 100644
--- a/vendor/google.golang.org/genproto/googleapis/api/annotations/client.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/api/annotations/client.pb.go
@@ -1,79 +1,214 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.26.0
+// 	protoc        v3.12.2
 // source: google/api/client.proto
 
 package annotations
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
 
-	proto "github.com/golang/protobuf/proto"
-	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
-// 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
-
-var E_MethodSignature = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.MethodOptions)(nil),
-	ExtensionType: ([]string)(nil),
-	Field:         1051,
-	Name:          "google.api.method_signature",
-	Tag:           "bytes,1051,rep,name=method_signature",
-	Filename:      "google/api/client.proto",
+var file_google_api_client_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptorpb.MethodOptions)(nil),
+		ExtensionType: ([]string)(nil),
+		Field:         1051,
+		Name:          "google.api.method_signature",
+		Tag:           "bytes,1051,rep,name=method_signature",
+		Filename:      "google/api/client.proto",
+	},
+	{
+		ExtendedType:  (*descriptorpb.ServiceOptions)(nil),
+		ExtensionType: (*string)(nil),
+		Field:         1049,
+		Name:          "google.api.default_host",
+		Tag:           "bytes,1049,opt,name=default_host",
+		Filename:      "google/api/client.proto",
+	},
+	{
+		ExtendedType:  (*descriptorpb.ServiceOptions)(nil),
+		ExtensionType: (*string)(nil),
+		Field:         1050,
+		Name:          "google.api.oauth_scopes",
+		Tag:           "bytes,1050,opt,name=oauth_scopes",
+		Filename:      "google/api/client.proto",
+	},
 }
 
-var E_DefaultHost = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.ServiceOptions)(nil),
-	ExtensionType: (*string)(nil),
-	Field:         1049,
-	Name:          "google.api.default_host",
-	Tag:           "bytes,1049,opt,name=default_host",
-	Filename:      "google/api/client.proto",
+// Extension fields to descriptorpb.MethodOptions.
+var (
+	// A definition of a client library method signature.
+	//
+	// In client libraries, each proto RPC corresponds to one or more methods
+	// which the end user is able to call, and calls the underlying RPC.
+	// Normally, this method receives a single argument (a struct or instance
+	// corresponding to the RPC request object). Defining this field will
+	// add one or more overloads providing flattened or simpler method signatures
+	// in some languages.
+	//
+	// The fields on the method signature are provided as a comma-separated
+	// string.
+	//
+	// For example, the proto RPC and annotation:
+	//
+	//   rpc CreateSubscription(CreateSubscriptionRequest)
+	//       returns (Subscription) {
+	//     option (google.api.method_signature) = "name,topic";
+	//   }
+	//
+	// Would add the following Java overload (in addition to the method accepting
+	// the request object):
+	//
+	//   public final Subscription createSubscription(String name, String topic)
+	//
+	// The following backwards-compatibility guidelines apply:
+	//
+	//   * Adding this annotation to an unannotated method is backwards
+	//     compatible.
+	//   * Adding this annotation to a method which already has existing
+	//     method signature annotations is backwards compatible if and only if
+	//     the new method signature annotation is last in the sequence.
+	//   * Modifying or removing an existing method signature annotation is
+	//     a breaking change.
+	//   * Re-ordering existing method signature annotations is a breaking
+	//     change.
+	//
+	// repeated string method_signature = 1051;
+	E_MethodSignature = &file_google_api_client_proto_extTypes[0]
+)
+
+// Extension fields to descriptorpb.ServiceOptions.
+var (
+	// The hostname for this service.
+	// This should be specified with no prefix or protocol.
+	//
+	// Example:
+	//
+	//   service Foo {
+	//     option (google.api.default_host) = "foo.googleapi.com";
+	//     ...
+	//   }
+	//
+	// optional string default_host = 1049;
+	E_DefaultHost = &file_google_api_client_proto_extTypes[1]
+	// OAuth scopes needed for the client.
+	//
+	// Example:
+	//
+	//   service Foo {
+	//     option (google.api.oauth_scopes) = \
+	//       "https://www.googleapis.com/auth/cloud-platform";
+	//     ...
+	//   }
+	//
+	// If there is more than one scope, use a comma-separated string:
+	//
+	// Example:
+	//
+	//   service Foo {
+	//     option (google.api.oauth_scopes) = \
+	//       "https://www.googleapis.com/auth/cloud-platform,"
+	//       "https://www.googleapis.com/auth/monitoring";
+	//     ...
+	//   }
+	//
+	// optional string oauth_scopes = 1050;
+	E_OauthScopes = &file_google_api_client_proto_extTypes[2]
+)
+
+var File_google_api_client_proto protoreflect.FileDescriptor
+
+var file_google_api_client_proto_rawDesc = []byte{
+	0x0a, 0x17, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x63, 0x6c, 0x69,
+	0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x61, 0x70, 0x69, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x3a, 0x4a, 0x0a, 0x10, 0x6d, 0x65, 0x74, 0x68, 0x6f,
+	0x64, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x1e, 0x2e, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65,
+	0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x9b, 0x08, 0x20, 0x03,
+	0x28, 0x09, 0x52, 0x0f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74,
+	0x75, 0x72, 0x65, 0x3a, 0x43, 0x0a, 0x0c, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x68,
+	0x6f, 0x73, 0x74, 0x12, 0x1f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x18, 0x99, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x66,
+	0x61, 0x75, 0x6c, 0x74, 0x48, 0x6f, 0x73, 0x74, 0x3a, 0x43, 0x0a, 0x0c, 0x6f, 0x61, 0x75, 0x74,
+	0x68, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69,
+	0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x9a, 0x08, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0b, 0x6f, 0x61, 0x75, 0x74, 0x68, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x73, 0x42, 0x69, 0x0a,
+	0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x42,
+	0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x41,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72,
+	0x67, 0x2f, 0x67, 0x65, 0x6e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0xa2, 0x02, 0x04, 0x47, 0x41, 0x50, 0x49, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-var E_OauthScopes = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.ServiceOptions)(nil),
-	ExtensionType: (*string)(nil),
-	Field:         1050,
-	Name:          "google.api.oauth_scopes",
-	Tag:           "bytes,1050,opt,name=oauth_scopes",
-	Filename:      "google/api/client.proto",
+var file_google_api_client_proto_goTypes = []interface{}{
+	(*descriptorpb.MethodOptions)(nil),  // 0: google.protobuf.MethodOptions
+	(*descriptorpb.ServiceOptions)(nil), // 1: google.protobuf.ServiceOptions
+}
+var file_google_api_client_proto_depIdxs = []int32{
+	0, // 0: google.api.method_signature:extendee -> google.protobuf.MethodOptions
+	1, // 1: google.api.default_host:extendee -> google.protobuf.ServiceOptions
+	1, // 2: google.api.oauth_scopes:extendee -> google.protobuf.ServiceOptions
+	3, // [3:3] is the sub-list for method output_type
+	3, // [3:3] is the sub-list for method input_type
+	3, // [3:3] is the sub-list for extension type_name
+	0, // [0:3] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
 }
 
-func init() {
-	proto.RegisterExtension(E_MethodSignature)
-	proto.RegisterExtension(E_DefaultHost)
-	proto.RegisterExtension(E_OauthScopes)
-}
-
-func init() { proto.RegisterFile("google/api/client.proto", fileDescriptor_78f2c6f7c3a942c1) }
-
-var fileDescriptor_78f2c6f7c3a942c1 = []byte{
-	// 262 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0x3f, 0x4f, 0xc3, 0x30,
-	0x10, 0xc5, 0x55, 0x40, 0xa8, 0x75, 0x11, 0xa0, 0x2c, 0x20, 0x06, 0xc8, 0xd8, 0xc9, 0x1e, 0xd8,
-	0xca, 0xd4, 0x76, 0xe0, 0x8f, 0x84, 0x88, 0x9a, 0x8d, 0x25, 0x72, 0x9d, 0xab, 0x63, 0x29, 0xf5,
-	0x59, 0xf6, 0x85, 0xef, 0x02, 0x6c, 0x7c, 0x52, 0x54, 0xc7, 0x11, 0x48, 0x0c, 0x6c, 0x27, 0xbd,
-	0xf7, 0xfb, 0x9d, 0xf4, 0xd8, 0x85, 0x46, 0xd4, 0x2d, 0x08, 0xe9, 0x8c, 0x50, 0xad, 0x01, 0x4b,
-	0xdc, 0x79, 0x24, 0xcc, 0x58, 0x1f, 0x70, 0xe9, 0xcc, 0x55, 0x9e, 0x4a, 0x31, 0xd9, 0x74, 0x5b,
-	0x51, 0x43, 0x50, 0xde, 0x38, 0x42, 0xdf, 0xb7, 0xe7, 0x4f, 0xec, 0x7c, 0x07, 0xd4, 0x60, 0x5d,
-	0x05, 0xa3, 0xad, 0xa4, 0xce, 0x43, 0x76, 0xcd, 0x93, 0x62, 0xc0, 0xf8, 0x73, 0xac, 0xbc, 0x38,
-	0x32, 0x68, 0xc3, 0xe5, 0xe7, 0x38, 0x3f, 0x9c, 0x4d, 0xd6, 0x67, 0x3d, 0x58, 0x0e, 0xdc, 0x7c,
-	0xc5, 0x4e, 0x6a, 0xd8, 0xca, 0xae, 0xa5, 0xaa, 0xc1, 0x40, 0xd9, 0xcd, 0x1f, 0x4f, 0x09, 0xfe,
-	0xcd, 0x28, 0x18, 0x44, 0xef, 0xe3, 0x7c, 0x34, 0x9b, 0xac, 0xa7, 0x89, 0x7a, 0xc0, 0x40, 0x7b,
-	0x09, 0xca, 0x8e, 0x9a, 0x2a, 0x28, 0x74, 0x10, 0xfe, 0x97, 0x7c, 0x24, 0x49, 0xa4, 0xca, 0x08,
-	0x2d, 0x0d, 0x3b, 0x55, 0xb8, 0xe3, 0x3f, 0x4b, 0x2c, 0xa7, 0xab, 0xb8, 0x51, 0xb1, 0x97, 0x14,
-	0xa3, 0xd7, 0x45, 0x8a, 0x34, 0xb6, 0xd2, 0x6a, 0x8e, 0x5e, 0x0b, 0x0d, 0x36, 0xbe, 0x10, 0x7d,
-	0x24, 0x9d, 0x09, 0x71, 0x5c, 0x69, 0x2d, 0x92, 0x8c, 0xbf, 0xee, 0x7e, 0xdd, 0x5f, 0x07, 0x47,
-	0xf7, 0x8b, 0xe2, 0x71, 0x73, 0x1c, 0xa1, 0xdb, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcc, 0xc2,
-	0xcf, 0x71, 0x90, 0x01, 0x00, 0x00,
+func init() { file_google_api_client_proto_init() }
+func file_google_api_client_proto_init() {
+	if File_google_api_client_proto != nil {
+		return
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_api_client_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 3,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_api_client_proto_goTypes,
+		DependencyIndexes: file_google_api_client_proto_depIdxs,
+		ExtensionInfos:    file_google_api_client_proto_extTypes,
+	}.Build()
+	File_google_api_client_proto = out.File
+	file_google_api_client_proto_rawDesc = nil
+	file_google_api_client_proto_goTypes = nil
+	file_google_api_client_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/genproto/googleapis/api/annotations/field_behavior.pb.go b/vendor/google.golang.org/genproto/googleapis/api/annotations/field_behavior.pb.go
index 31f87dd..164e0df 100644
--- a/vendor/google.golang.org/genproto/googleapis/api/annotations/field_behavior.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/api/annotations/field_behavior.pb.go
@@ -1,26 +1,40 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.27.1
+// 	protoc        v3.12.2
 // source: google/api/field_behavior.proto
 
 package annotations
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
+	sync "sync"
 
-	proto "github.com/golang/protobuf/proto"
-	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
 )
 
-// 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
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // An indicator of the behavior of a given field (for example, that a field
 // is required in requests, or given as output but ignored as input).
@@ -54,69 +68,183 @@
 	// This indicates that the field may be set once in a request to create a
 	// resource, but may not be changed thereafter.
 	FieldBehavior_IMMUTABLE FieldBehavior = 5
+	// Denotes that a (repeated) field is an unordered list.
+	// This indicates that the service may provide the elements of the list
+	// in any arbitrary  order, rather than the order the user originally
+	// provided. Additionally, the list's order may or may not be stable.
+	FieldBehavior_UNORDERED_LIST FieldBehavior = 6
+	// Denotes that this field returns a non-empty default value if not set.
+	// This indicates that if the user provides the empty value in a request,
+	// a non-empty value will be returned. The user will not be aware of what
+	// non-empty value to expect.
+	FieldBehavior_NON_EMPTY_DEFAULT FieldBehavior = 7
 )
 
-var FieldBehavior_name = map[int32]string{
-	0: "FIELD_BEHAVIOR_UNSPECIFIED",
-	1: "OPTIONAL",
-	2: "REQUIRED",
-	3: "OUTPUT_ONLY",
-	4: "INPUT_ONLY",
-	5: "IMMUTABLE",
-}
+// Enum value maps for FieldBehavior.
+var (
+	FieldBehavior_name = map[int32]string{
+		0: "FIELD_BEHAVIOR_UNSPECIFIED",
+		1: "OPTIONAL",
+		2: "REQUIRED",
+		3: "OUTPUT_ONLY",
+		4: "INPUT_ONLY",
+		5: "IMMUTABLE",
+		6: "UNORDERED_LIST",
+		7: "NON_EMPTY_DEFAULT",
+	}
+	FieldBehavior_value = map[string]int32{
+		"FIELD_BEHAVIOR_UNSPECIFIED": 0,
+		"OPTIONAL":                   1,
+		"REQUIRED":                   2,
+		"OUTPUT_ONLY":                3,
+		"INPUT_ONLY":                 4,
+		"IMMUTABLE":                  5,
+		"UNORDERED_LIST":             6,
+		"NON_EMPTY_DEFAULT":          7,
+	}
+)
 
-var FieldBehavior_value = map[string]int32{
-	"FIELD_BEHAVIOR_UNSPECIFIED": 0,
-	"OPTIONAL":                   1,
-	"REQUIRED":                   2,
-	"OUTPUT_ONLY":                3,
-	"INPUT_ONLY":                 4,
-	"IMMUTABLE":                  5,
+func (x FieldBehavior) Enum() *FieldBehavior {
+	p := new(FieldBehavior)
+	*p = x
+	return p
 }
 
 func (x FieldBehavior) String() string {
-	return proto.EnumName(FieldBehavior_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (FieldBehavior) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_api_field_behavior_proto_enumTypes[0].Descriptor()
+}
+
+func (FieldBehavior) Type() protoreflect.EnumType {
+	return &file_google_api_field_behavior_proto_enumTypes[0]
+}
+
+func (x FieldBehavior) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use FieldBehavior.Descriptor instead.
 func (FieldBehavior) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_4648f18fd5079967, []int{0}
+	return file_google_api_field_behavior_proto_rawDescGZIP(), []int{0}
 }
 
-var E_FieldBehavior = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.FieldOptions)(nil),
-	ExtensionType: ([]FieldBehavior)(nil),
-	Field:         1052,
-	Name:          "google.api.field_behavior",
-	Tag:           "varint,1052,rep,name=field_behavior,enum=google.api.FieldBehavior",
-	Filename:      "google/api/field_behavior.proto",
+var file_google_api_field_behavior_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptorpb.FieldOptions)(nil),
+		ExtensionType: ([]FieldBehavior)(nil),
+		Field:         1052,
+		Name:          "google.api.field_behavior",
+		Tag:           "varint,1052,rep,name=field_behavior,enum=google.api.FieldBehavior",
+		Filename:      "google/api/field_behavior.proto",
+	},
 }
 
-func init() {
-	proto.RegisterEnum("google.api.FieldBehavior", FieldBehavior_name, FieldBehavior_value)
-	proto.RegisterExtension(E_FieldBehavior)
+// Extension fields to descriptorpb.FieldOptions.
+var (
+	// A designation of a specific field behavior (required, output only, etc.)
+	// in protobuf messages.
+	//
+	// Examples:
+	//
+	//   string name = 1 [(google.api.field_behavior) = REQUIRED];
+	//   State state = 1 [(google.api.field_behavior) = OUTPUT_ONLY];
+	//   google.protobuf.Duration ttl = 1
+	//     [(google.api.field_behavior) = INPUT_ONLY];
+	//   google.protobuf.Timestamp expire_time = 1
+	//     [(google.api.field_behavior) = OUTPUT_ONLY,
+	//      (google.api.field_behavior) = IMMUTABLE];
+	//
+	// repeated google.api.FieldBehavior field_behavior = 1052;
+	E_FieldBehavior = &file_google_api_field_behavior_proto_extTypes[0]
+)
+
+var File_google_api_field_behavior_proto protoreflect.FileDescriptor
+
+var file_google_api_field_behavior_proto_rawDesc = []byte{
+	0x0a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x66, 0x69, 0x65,
+	0x6c, 0x64, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x12, 0x0a, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x1a, 0x20, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2a,
+	0xa6, 0x01, 0x0a, 0x0d, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f,
+	0x72, 0x12, 0x1e, 0x0a, 0x1a, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56,
+	0x49, 0x4f, 0x52, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10,
+	0x00, 0x12, 0x0c, 0x0a, 0x08, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x10, 0x01, 0x12,
+	0x0c, 0x0a, 0x08, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0f, 0x0a,
+	0x0b, 0x4f, 0x55, 0x54, 0x50, 0x55, 0x54, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x03, 0x12, 0x0e,
+	0x0a, 0x0a, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x5f, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x04, 0x12, 0x0d,
+	0x0a, 0x09, 0x49, 0x4d, 0x4d, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x05, 0x12, 0x12, 0x0a,
+	0x0e, 0x55, 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x45, 0x44, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x10,
+	0x06, 0x12, 0x15, 0x0a, 0x11, 0x4e, 0x4f, 0x4e, 0x5f, 0x45, 0x4d, 0x50, 0x54, 0x59, 0x5f, 0x44,
+	0x45, 0x46, 0x41, 0x55, 0x4c, 0x54, 0x10, 0x07, 0x3a, 0x60, 0x0a, 0x0e, 0x66, 0x69, 0x65, 0x6c,
+	0x64, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x1d, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65,
+	0x6c, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x9c, 0x08, 0x20, 0x03, 0x28, 0x0e,
+	0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x46, 0x69,
+	0x65, 0x6c, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0d, 0x66, 0x69, 0x65,
+	0x6c, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x42, 0x70, 0x0a, 0x0e, 0x63, 0x6f,
+	0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x42, 0x12, 0x46, 0x69,
+	0x65, 0x6c, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f,
+	0x50, 0x01, 0x5a, 0x41, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e,
+	0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x65, 0x6e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e,
+	0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0xa2, 0x02, 0x04, 0x47, 0x41, 0x50, 0x49, 0x62, 0x06, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("google/api/field_behavior.proto", fileDescriptor_4648f18fd5079967) }
+var (
+	file_google_api_field_behavior_proto_rawDescOnce sync.Once
+	file_google_api_field_behavior_proto_rawDescData = file_google_api_field_behavior_proto_rawDesc
+)
 
-var fileDescriptor_4648f18fd5079967 = []byte{
-	// 303 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x54, 0x90, 0x4f, 0x4f, 0xb3, 0x30,
-	0x1c, 0xc7, 0x9f, 0xfd, 0x79, 0xcc, 0xac, 0x0e, 0x49, 0x4f, 0xba, 0x44, 0xdd, 0xd1, 0x78, 0x28,
-	0x89, 0xde, 0xf4, 0x04, 0xae, 0xd3, 0x26, 0x8c, 0x56, 0x04, 0x13, 0xbd, 0x60, 0xb7, 0xb1, 0xda,
-	0x64, 0xd2, 0x06, 0xd0, 0x8b, 0x6f, 0xc5, 0x93, 0xaf, 0xd4, 0xd0, 0x31, 0x85, 0x5b, 0xbf, 0xf9,
-	0x7d, 0xfa, 0xeb, 0xe7, 0x5b, 0x70, 0x2a, 0x94, 0x12, 0xeb, 0xd4, 0xe1, 0x5a, 0x3a, 0x2b, 0x99,
-	0xae, 0x97, 0xc9, 0x3c, 0x7d, 0xe5, 0x1f, 0x52, 0xe5, 0x48, 0xe7, 0xaa, 0x54, 0x10, 0x6c, 0x00,
-	0xc4, 0xb5, 0x1c, 0x8d, 0x6b, 0xd8, 0x4c, 0xe6, 0xef, 0x2b, 0x67, 0x99, 0x16, 0x8b, 0x5c, 0xea,
-	0x72, 0x4b, 0x9f, 0x7f, 0x82, 0xe1, 0xb4, 0xda, 0xe2, 0xd5, 0x4b, 0xe0, 0x09, 0x18, 0x4d, 0x09,
-	0xf6, 0x27, 0x89, 0x87, 0xef, 0xdc, 0x47, 0x42, 0xc3, 0x24, 0x0e, 0x1e, 0x18, 0xbe, 0x21, 0x53,
-	0x82, 0x27, 0xf6, 0x3f, 0xb8, 0x0f, 0x06, 0x94, 0x45, 0x84, 0x06, 0xae, 0x6f, 0x77, 0xaa, 0x14,
-	0xe2, 0xfb, 0x98, 0x84, 0x78, 0x62, 0x77, 0xe1, 0x01, 0xd8, 0xa3, 0x71, 0xc4, 0xe2, 0x28, 0xa1,
-	0x81, 0xff, 0x64, 0xf7, 0xa0, 0x05, 0x00, 0x09, 0x7e, 0x73, 0x1f, 0x0e, 0xc1, 0x2e, 0x99, 0xcd,
-	0xe2, 0xc8, 0xf5, 0x7c, 0x6c, 0xff, 0xbf, 0x7a, 0x01, 0x56, 0xbb, 0x02, 0x3c, 0x46, 0xb5, 0xfd,
-	0xd6, 0x18, 0x19, 0x3b, 0xaa, 0x4b, 0xa9, 0xb2, 0xe2, 0xf0, 0x6b, 0x30, 0xee, 0x9d, 0x59, 0x17,
-	0x47, 0xe8, 0xaf, 0x23, 0x6a, 0xe9, 0x87, 0xc3, 0x55, 0x33, 0x7a, 0x1a, 0x58, 0x0b, 0xf5, 0xd6,
-	0xc0, 0x3d, 0xd8, 0xe2, 0x59, 0xf5, 0x0c, 0xeb, 0x3c, 0xbb, 0x35, 0x21, 0xd4, 0x9a, 0x67, 0x02,
-	0xa9, 0x5c, 0x38, 0x22, 0xcd, 0x8c, 0x84, 0xb3, 0x19, 0x71, 0x2d, 0x0b, 0xf3, 0xe9, 0x3c, 0xcb,
-	0x54, 0xc9, 0x8d, 0xcf, 0x75, 0xe3, 0xfc, 0xdd, 0xed, 0xdf, 0xba, 0x8c, 0xcc, 0x77, 0xcc, 0xa5,
-	0xcb, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xfc, 0x94, 0x57, 0x94, 0xa8, 0x01, 0x00, 0x00,
+func file_google_api_field_behavior_proto_rawDescGZIP() []byte {
+	file_google_api_field_behavior_proto_rawDescOnce.Do(func() {
+		file_google_api_field_behavior_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_api_field_behavior_proto_rawDescData)
+	})
+	return file_google_api_field_behavior_proto_rawDescData
+}
+
+var file_google_api_field_behavior_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
+var file_google_api_field_behavior_proto_goTypes = []interface{}{
+	(FieldBehavior)(0),                // 0: google.api.FieldBehavior
+	(*descriptorpb.FieldOptions)(nil), // 1: google.protobuf.FieldOptions
+}
+var file_google_api_field_behavior_proto_depIdxs = []int32{
+	1, // 0: google.api.field_behavior:extendee -> google.protobuf.FieldOptions
+	0, // 1: google.api.field_behavior:type_name -> google.api.FieldBehavior
+	2, // [2:2] is the sub-list for method output_type
+	2, // [2:2] is the sub-list for method input_type
+	1, // [1:2] is the sub-list for extension type_name
+	0, // [0:1] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_google_api_field_behavior_proto_init() }
+func file_google_api_field_behavior_proto_init() {
+	if File_google_api_field_behavior_proto != nil {
+		return
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_api_field_behavior_proto_rawDesc,
+			NumEnums:      1,
+			NumMessages:   0,
+			NumExtensions: 1,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_api_field_behavior_proto_goTypes,
+		DependencyIndexes: file_google_api_field_behavior_proto_depIdxs,
+		EnumInfos:         file_google_api_field_behavior_proto_enumTypes,
+		ExtensionInfos:    file_google_api_field_behavior_proto_extTypes,
+	}.Build()
+	File_google_api_field_behavior_proto = out.File
+	file_google_api_field_behavior_proto_rawDesc = nil
+	file_google_api_field_behavior_proto_goTypes = nil
+	file_google_api_field_behavior_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/genproto/googleapis/api/annotations/http.pb.go b/vendor/google.golang.org/genproto/googleapis/api/annotations/http.pb.go
index a638703..4f34ab7 100644
--- a/vendor/google.golang.org/genproto/googleapis/api/annotations/http.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/api/annotations/http.pb.go
@@ -1,30 +1,48 @@
+// Copyright 2015 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.26.0
+// 	protoc        v3.12.2
 // source: google/api/http.proto
 
 package annotations
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
+	sync "sync"
 
-	proto "github.com/golang/protobuf/proto"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
 )
 
-// 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
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // Defines the HTTP configuration for an API service. It contains a list of
 // [HttpRule][google.api.HttpRule], each specifying the mapping of an RPC method
 // to one or more HTTP REST API methods.
 type Http struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A list of HTTP configuration rules that apply to individual API methods.
 	//
 	// **NOTE:** All service configuration rules follow "last one wins" order.
@@ -35,47 +53,51 @@
 	//
 	// The default behavior is to not decode RFC 6570 reserved characters in multi
 	// segment matches.
-	FullyDecodeReservedExpansion bool     `protobuf:"varint,2,opt,name=fully_decode_reserved_expansion,json=fullyDecodeReservedExpansion,proto3" json:"fully_decode_reserved_expansion,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{} `json:"-"`
-	XXX_unrecognized             []byte   `json:"-"`
-	XXX_sizecache                int32    `json:"-"`
+	FullyDecodeReservedExpansion bool `protobuf:"varint,2,opt,name=fully_decode_reserved_expansion,json=fullyDecodeReservedExpansion,proto3" json:"fully_decode_reserved_expansion,omitempty"`
 }
 
-func (m *Http) Reset()         { *m = Http{} }
-func (m *Http) String() string { return proto.CompactTextString(m) }
-func (*Http) ProtoMessage()    {}
+func (x *Http) Reset() {
+	*x = Http{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_api_http_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Http) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Http) ProtoMessage() {}
+
+func (x *Http) ProtoReflect() protoreflect.Message {
+	mi := &file_google_api_http_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Http.ProtoReflect.Descriptor instead.
 func (*Http) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ff9994be407cdcc9, []int{0}
+	return file_google_api_http_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *Http) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Http.Unmarshal(m, b)
-}
-func (m *Http) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Http.Marshal(b, m, deterministic)
-}
-func (m *Http) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Http.Merge(m, src)
-}
-func (m *Http) XXX_Size() int {
-	return xxx_messageInfo_Http.Size(m)
-}
-func (m *Http) XXX_DiscardUnknown() {
-	xxx_messageInfo_Http.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Http proto.InternalMessageInfo
-
-func (m *Http) GetRules() []*HttpRule {
-	if m != nil {
-		return m.Rules
+func (x *Http) GetRules() []*HttpRule {
+	if x != nil {
+		return x.Rules
 	}
 	return nil
 }
 
-func (m *Http) GetFullyDecodeReservedExpansion() bool {
-	if m != nil {
-		return m.FullyDecodeReservedExpansion
+func (x *Http) GetFullyDecodeReservedExpansion() bool {
+	if x != nil {
+		return x.FullyDecodeReservedExpansion
 	}
 	return false
 }
@@ -350,6 +372,10 @@
 // the request or response body to a repeated field. However, some gRPC
 // Transcoding implementations may not support this feature.
 type HttpRule struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Selects a method to which this rule applies.
 	//
 	// Refer to [selector][google.api.DocumentationRule.selector] for syntax details.
@@ -358,7 +384,7 @@
 	// used with any of the {get|put|post|delete|patch} methods. A custom method
 	// can be defined using the 'custom' field.
 	//
-	// Types that are valid to be assigned to Pattern:
+	// Types that are assignable to Pattern:
 	//	*HttpRule_Get
 	//	*HttpRule_Put
 	//	*HttpRule_Post
@@ -383,69 +409,153 @@
 	// Additional HTTP bindings for the selector. Nested bindings must
 	// not contain an `additional_bindings` field themselves (that is,
 	// the nesting may only be one level deep).
-	AdditionalBindings   []*HttpRule `protobuf:"bytes,11,rep,name=additional_bindings,json=additionalBindings,proto3" json:"additional_bindings,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
+	AdditionalBindings []*HttpRule `protobuf:"bytes,11,rep,name=additional_bindings,json=additionalBindings,proto3" json:"additional_bindings,omitempty"`
 }
 
-func (m *HttpRule) Reset()         { *m = HttpRule{} }
-func (m *HttpRule) String() string { return proto.CompactTextString(m) }
-func (*HttpRule) ProtoMessage()    {}
+func (x *HttpRule) Reset() {
+	*x = HttpRule{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_api_http_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *HttpRule) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*HttpRule) ProtoMessage() {}
+
+func (x *HttpRule) ProtoReflect() protoreflect.Message {
+	mi := &file_google_api_http_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use HttpRule.ProtoReflect.Descriptor instead.
 func (*HttpRule) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ff9994be407cdcc9, []int{1}
+	return file_google_api_http_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *HttpRule) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HttpRule.Unmarshal(m, b)
-}
-func (m *HttpRule) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HttpRule.Marshal(b, m, deterministic)
-}
-func (m *HttpRule) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HttpRule.Merge(m, src)
-}
-func (m *HttpRule) XXX_Size() int {
-	return xxx_messageInfo_HttpRule.Size(m)
-}
-func (m *HttpRule) XXX_DiscardUnknown() {
-	xxx_messageInfo_HttpRule.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HttpRule proto.InternalMessageInfo
-
-func (m *HttpRule) GetSelector() string {
-	if m != nil {
-		return m.Selector
+func (x *HttpRule) GetSelector() string {
+	if x != nil {
+		return x.Selector
 	}
 	return ""
 }
 
+func (m *HttpRule) GetPattern() isHttpRule_Pattern {
+	if m != nil {
+		return m.Pattern
+	}
+	return nil
+}
+
+func (x *HttpRule) GetGet() string {
+	if x, ok := x.GetPattern().(*HttpRule_Get); ok {
+		return x.Get
+	}
+	return ""
+}
+
+func (x *HttpRule) GetPut() string {
+	if x, ok := x.GetPattern().(*HttpRule_Put); ok {
+		return x.Put
+	}
+	return ""
+}
+
+func (x *HttpRule) GetPost() string {
+	if x, ok := x.GetPattern().(*HttpRule_Post); ok {
+		return x.Post
+	}
+	return ""
+}
+
+func (x *HttpRule) GetDelete() string {
+	if x, ok := x.GetPattern().(*HttpRule_Delete); ok {
+		return x.Delete
+	}
+	return ""
+}
+
+func (x *HttpRule) GetPatch() string {
+	if x, ok := x.GetPattern().(*HttpRule_Patch); ok {
+		return x.Patch
+	}
+	return ""
+}
+
+func (x *HttpRule) GetCustom() *CustomHttpPattern {
+	if x, ok := x.GetPattern().(*HttpRule_Custom); ok {
+		return x.Custom
+	}
+	return nil
+}
+
+func (x *HttpRule) GetBody() string {
+	if x != nil {
+		return x.Body
+	}
+	return ""
+}
+
+func (x *HttpRule) GetResponseBody() string {
+	if x != nil {
+		return x.ResponseBody
+	}
+	return ""
+}
+
+func (x *HttpRule) GetAdditionalBindings() []*HttpRule {
+	if x != nil {
+		return x.AdditionalBindings
+	}
+	return nil
+}
+
 type isHttpRule_Pattern interface {
 	isHttpRule_Pattern()
 }
 
 type HttpRule_Get struct {
+	// Maps to HTTP GET. Used for listing and getting information about
+	// resources.
 	Get string `protobuf:"bytes,2,opt,name=get,proto3,oneof"`
 }
 
 type HttpRule_Put struct {
+	// Maps to HTTP PUT. Used for replacing a resource.
 	Put string `protobuf:"bytes,3,opt,name=put,proto3,oneof"`
 }
 
 type HttpRule_Post struct {
+	// Maps to HTTP POST. Used for creating a resource or performing an action.
 	Post string `protobuf:"bytes,4,opt,name=post,proto3,oneof"`
 }
 
 type HttpRule_Delete struct {
+	// Maps to HTTP DELETE. Used for deleting a resource.
 	Delete string `protobuf:"bytes,5,opt,name=delete,proto3,oneof"`
 }
 
 type HttpRule_Patch struct {
+	// Maps to HTTP PATCH. Used for updating a resource.
 	Patch string `protobuf:"bytes,6,opt,name=patch,proto3,oneof"`
 }
 
 type HttpRule_Custom struct {
+	// The custom pattern is used for specifying an HTTP method that is not
+	// included in the `pattern` field, such as HEAD, or "*" to leave the
+	// HTTP method unspecified for this rule. The wild-card rule is useful
+	// for services that provide content to Web (HTML) clients.
 	Custom *CustomHttpPattern `protobuf:"bytes,8,opt,name=custom,proto3,oneof"`
 }
 
@@ -461,79 +571,185 @@
 
 func (*HttpRule_Custom) isHttpRule_Pattern() {}
 
-func (m *HttpRule) GetPattern() isHttpRule_Pattern {
-	if m != nil {
-		return m.Pattern
-	}
-	return nil
+// A custom pattern is used for defining custom HTTP verb.
+type CustomHttpPattern struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The name of this custom HTTP verb.
+	Kind string `protobuf:"bytes,1,opt,name=kind,proto3" json:"kind,omitempty"`
+	// The path matched by this custom verb.
+	Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"`
 }
 
-func (m *HttpRule) GetGet() string {
-	if x, ok := m.GetPattern().(*HttpRule_Get); ok {
-		return x.Get
+func (x *CustomHttpPattern) Reset() {
+	*x = CustomHttpPattern{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_api_http_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CustomHttpPattern) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CustomHttpPattern) ProtoMessage() {}
+
+func (x *CustomHttpPattern) ProtoReflect() protoreflect.Message {
+	mi := &file_google_api_http_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CustomHttpPattern.ProtoReflect.Descriptor instead.
+func (*CustomHttpPattern) Descriptor() ([]byte, []int) {
+	return file_google_api_http_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *CustomHttpPattern) GetKind() string {
+	if x != nil {
+		return x.Kind
 	}
 	return ""
 }
 
-func (m *HttpRule) GetPut() string {
-	if x, ok := m.GetPattern().(*HttpRule_Put); ok {
-		return x.Put
+func (x *CustomHttpPattern) GetPath() string {
+	if x != nil {
+		return x.Path
 	}
 	return ""
 }
 
-func (m *HttpRule) GetPost() string {
-	if x, ok := m.GetPattern().(*HttpRule_Post); ok {
-		return x.Post
-	}
-	return ""
+var File_google_api_http_proto protoreflect.FileDescriptor
+
+var file_google_api_http_proto_rawDesc = []byte{
+	0x0a, 0x15, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x68, 0x74, 0x74,
+	0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x61, 0x70, 0x69, 0x22, 0x79, 0x0a, 0x04, 0x48, 0x74, 0x74, 0x70, 0x12, 0x2a, 0x0a, 0x05, 0x72,
+	0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x48, 0x74, 0x74, 0x70, 0x52, 0x75, 0x6c, 0x65,
+	0x52, 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x45, 0x0a, 0x1f, 0x66, 0x75, 0x6c, 0x6c, 0x79,
+	0x5f, 0x64, 0x65, 0x63, 0x6f, 0x64, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64,
+	0x5f, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08,
+	0x52, 0x1c, 0x66, 0x75, 0x6c, 0x6c, 0x79, 0x44, 0x65, 0x63, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73,
+	0x65, 0x72, 0x76, 0x65, 0x64, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xda,
+	0x02, 0x0a, 0x08, 0x48, 0x74, 0x74, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73,
+	0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73,
+	0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x03, 0x67, 0x65, 0x74, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x67, 0x65, 0x74, 0x12, 0x12, 0x0a, 0x03, 0x70,
+	0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x70, 0x75, 0x74, 0x12,
+	0x14, 0x0a, 0x04, 0x70, 0x6f, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,
+	0x04, 0x70, 0x6f, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x06, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18,
+	0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12,
+	0x16, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x63, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00,
+	0x52, 0x05, 0x70, 0x61, 0x74, 0x63, 0x68, 0x12, 0x37, 0x0a, 0x06, 0x63, 0x75, 0x73, 0x74, 0x6f,
+	0x6d, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x61, 0x70, 0x69, 0x2e, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x74, 0x74, 0x70, 0x50,
+	0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x48, 0x00, 0x52, 0x06, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d,
+	0x12, 0x12, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
+	0x62, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x5f, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x45, 0x0a, 0x13, 0x61, 0x64, 0x64,
+	0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x62, 0x69, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x73,
+	0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x61, 0x70, 0x69, 0x2e, 0x48, 0x74, 0x74, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x12, 0x61, 0x64,
+	0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x42, 0x69, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x73,
+	0x42, 0x09, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x22, 0x3b, 0x0a, 0x11, 0x43,
+	0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x74, 0x74, 0x70, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e,
+	0x12, 0x12, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
+	0x6b, 0x69, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x42, 0x6a, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x42, 0x09, 0x48, 0x74, 0x74, 0x70,
+	0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x41, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x65, 0x6e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x61,
+	0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3b, 0x61,
+	0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0xf8, 0x01, 0x01, 0xa2, 0x02, 0x04,
+	0x47, 0x41, 0x50, 0x49, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func (m *HttpRule) GetDelete() string {
-	if x, ok := m.GetPattern().(*HttpRule_Delete); ok {
-		return x.Delete
-	}
-	return ""
+var (
+	file_google_api_http_proto_rawDescOnce sync.Once
+	file_google_api_http_proto_rawDescData = file_google_api_http_proto_rawDesc
+)
+
+func file_google_api_http_proto_rawDescGZIP() []byte {
+	file_google_api_http_proto_rawDescOnce.Do(func() {
+		file_google_api_http_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_api_http_proto_rawDescData)
+	})
+	return file_google_api_http_proto_rawDescData
 }
 
-func (m *HttpRule) GetPatch() string {
-	if x, ok := m.GetPattern().(*HttpRule_Patch); ok {
-		return x.Patch
-	}
-	return ""
+var file_google_api_http_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
+var file_google_api_http_proto_goTypes = []interface{}{
+	(*Http)(nil),              // 0: google.api.Http
+	(*HttpRule)(nil),          // 1: google.api.HttpRule
+	(*CustomHttpPattern)(nil), // 2: google.api.CustomHttpPattern
+}
+var file_google_api_http_proto_depIdxs = []int32{
+	1, // 0: google.api.Http.rules:type_name -> google.api.HttpRule
+	2, // 1: google.api.HttpRule.custom:type_name -> google.api.CustomHttpPattern
+	1, // 2: google.api.HttpRule.additional_bindings:type_name -> google.api.HttpRule
+	3, // [3:3] is the sub-list for method output_type
+	3, // [3:3] is the sub-list for method input_type
+	3, // [3:3] is the sub-list for extension type_name
+	3, // [3:3] is the sub-list for extension extendee
+	0, // [0:3] is the sub-list for field type_name
 }
 
-func (m *HttpRule) GetCustom() *CustomHttpPattern {
-	if x, ok := m.GetPattern().(*HttpRule_Custom); ok {
-		return x.Custom
+func init() { file_google_api_http_proto_init() }
+func file_google_api_http_proto_init() {
+	if File_google_api_http_proto != nil {
+		return
 	}
-	return nil
-}
-
-func (m *HttpRule) GetBody() string {
-	if m != nil {
-		return m.Body
+	if !protoimpl.UnsafeEnabled {
+		file_google_api_http_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Http); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_api_http_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*HttpRule); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_api_http_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CustomHttpPattern); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
 	}
-	return ""
-}
-
-func (m *HttpRule) GetResponseBody() string {
-	if m != nil {
-		return m.ResponseBody
-	}
-	return ""
-}
-
-func (m *HttpRule) GetAdditionalBindings() []*HttpRule {
-	if m != nil {
-		return m.AdditionalBindings
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*HttpRule) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
+	file_google_api_http_proto_msgTypes[1].OneofWrappers = []interface{}{
 		(*HttpRule_Get)(nil),
 		(*HttpRule_Put)(nil),
 		(*HttpRule_Post)(nil),
@@ -541,93 +757,22 @@
 		(*HttpRule_Patch)(nil),
 		(*HttpRule_Custom)(nil),
 	}
-}
-
-// A custom pattern is used for defining custom HTTP verb.
-type CustomHttpPattern struct {
-	// The name of this custom HTTP verb.
-	Kind string `protobuf:"bytes,1,opt,name=kind,proto3" json:"kind,omitempty"`
-	// The path matched by this custom verb.
-	Path                 string   `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *CustomHttpPattern) Reset()         { *m = CustomHttpPattern{} }
-func (m *CustomHttpPattern) String() string { return proto.CompactTextString(m) }
-func (*CustomHttpPattern) ProtoMessage()    {}
-func (*CustomHttpPattern) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ff9994be407cdcc9, []int{2}
-}
-
-func (m *CustomHttpPattern) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CustomHttpPattern.Unmarshal(m, b)
-}
-func (m *CustomHttpPattern) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CustomHttpPattern.Marshal(b, m, deterministic)
-}
-func (m *CustomHttpPattern) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CustomHttpPattern.Merge(m, src)
-}
-func (m *CustomHttpPattern) XXX_Size() int {
-	return xxx_messageInfo_CustomHttpPattern.Size(m)
-}
-func (m *CustomHttpPattern) XXX_DiscardUnknown() {
-	xxx_messageInfo_CustomHttpPattern.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CustomHttpPattern proto.InternalMessageInfo
-
-func (m *CustomHttpPattern) GetKind() string {
-	if m != nil {
-		return m.Kind
-	}
-	return ""
-}
-
-func (m *CustomHttpPattern) GetPath() string {
-	if m != nil {
-		return m.Path
-	}
-	return ""
-}
-
-func init() {
-	proto.RegisterType((*Http)(nil), "google.api.Http")
-	proto.RegisterType((*HttpRule)(nil), "google.api.HttpRule")
-	proto.RegisterType((*CustomHttpPattern)(nil), "google.api.CustomHttpPattern")
-}
-
-func init() { proto.RegisterFile("google/api/http.proto", fileDescriptor_ff9994be407cdcc9) }
-
-var fileDescriptor_ff9994be407cdcc9 = []byte{
-	// 419 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x92, 0xc1, 0x8e, 0xd3, 0x30,
-	0x10, 0x86, 0x49, 0x9b, 0x76, 0xdb, 0xe9, 0x82, 0x84, 0x59, 0x90, 0x85, 0x40, 0x54, 0xe5, 0x52,
-	0x71, 0x48, 0xa5, 0xe5, 0xc0, 0x61, 0x4f, 0x1b, 0xa8, 0x58, 0x6e, 0x55, 0x8e, 0x5c, 0x22, 0x37,
-	0x1e, 0x52, 0x83, 0xd7, 0xb6, 0xe2, 0x09, 0xa2, 0xaf, 0xc3, 0x63, 0xf1, 0x24, 0x1c, 0x91, 0x9d,
-	0x84, 0x56, 0x42, 0xe2, 0x36, 0xf3, 0xff, 0x9f, 0xa7, 0x7f, 0x27, 0x03, 0x4f, 0x6b, 0x6b, 0x6b,
-	0x8d, 0x1b, 0xe1, 0xd4, 0xe6, 0x40, 0xe4, 0x32, 0xd7, 0x58, 0xb2, 0x0c, 0x3a, 0x39, 0x13, 0x4e,
-	0xad, 0x8e, 0x90, 0xde, 0x11, 0x39, 0xf6, 0x06, 0x26, 0x4d, 0xab, 0xd1, 0xf3, 0x64, 0x39, 0x5e,
-	0x2f, 0xae, 0xaf, 0xb2, 0x13, 0x93, 0x05, 0xa0, 0x68, 0x35, 0x16, 0x1d, 0xc2, 0xb6, 0xf0, 0xea,
-	0x4b, 0xab, 0xf5, 0xb1, 0x94, 0x58, 0x59, 0x89, 0x65, 0x83, 0x1e, 0x9b, 0xef, 0x28, 0x4b, 0xfc,
-	0xe1, 0x84, 0xf1, 0xca, 0x1a, 0x3e, 0x5a, 0x26, 0xeb, 0x59, 0xf1, 0x22, 0x62, 0x1f, 0x22, 0x55,
-	0xf4, 0xd0, 0x76, 0x60, 0x56, 0xbf, 0x46, 0x30, 0x1b, 0x46, 0xb3, 0xe7, 0x30, 0xf3, 0xa8, 0xb1,
-	0x22, 0xdb, 0xf0, 0x64, 0x99, 0xac, 0xe7, 0xc5, 0xdf, 0x9e, 0x31, 0x18, 0xd7, 0x48, 0x71, 0xe6,
-	0xfc, 0xee, 0x41, 0x11, 0x9a, 0xa0, 0xb9, 0x96, 0xf8, 0x78, 0xd0, 0x5c, 0x4b, 0xec, 0x0a, 0x52,
-	0x67, 0x3d, 0xf1, 0xb4, 0x17, 0x63, 0xc7, 0x38, 0x4c, 0x25, 0x6a, 0x24, 0xe4, 0x93, 0x5e, 0xef,
-	0x7b, 0xf6, 0x0c, 0x26, 0x4e, 0x50, 0x75, 0xe0, 0xd3, 0xde, 0xe8, 0x5a, 0xf6, 0x0e, 0xa6, 0x55,
-	0xeb, 0xc9, 0xde, 0xf3, 0xd9, 0x32, 0x59, 0x2f, 0xae, 0x5f, 0x9e, 0x2f, 0xe3, 0x7d, 0x74, 0x42,
-	0xee, 0x9d, 0x20, 0xc2, 0xc6, 0x84, 0x81, 0x1d, 0xce, 0x18, 0xa4, 0x7b, 0x2b, 0x8f, 0xfc, 0x22,
-	0xfe, 0x81, 0x58, 0xb3, 0xd7, 0xf0, 0xb0, 0x41, 0xef, 0xac, 0xf1, 0x58, 0x46, 0xf3, 0x32, 0x9a,
-	0x97, 0x83, 0x98, 0x07, 0x68, 0x0b, 0x4f, 0x84, 0x94, 0x8a, 0x94, 0x35, 0x42, 0x97, 0x7b, 0x65,
-	0xa4, 0x32, 0xb5, 0xe7, 0x8b, 0xff, 0x7c, 0x0b, 0x76, 0x7a, 0x90, 0xf7, 0x7c, 0x3e, 0x87, 0x0b,
-	0xd7, 0x85, 0x5a, 0xdd, 0xc0, 0xe3, 0x7f, 0x92, 0x86, 0x7c, 0xdf, 0x94, 0x91, 0xfd, 0x82, 0x63,
-	0x1d, 0x34, 0x27, 0xe8, 0xd0, 0x6d, 0xb7, 0x88, 0x75, 0xfe, 0x15, 0x1e, 0x55, 0xf6, 0xfe, 0xec,
-	0x67, 0xf3, 0x79, 0x1c, 0x13, 0xae, 0x67, 0x97, 0x7c, 0xbe, 0xed, 0x8d, 0xda, 0x6a, 0x61, 0xea,
-	0xcc, 0x36, 0xf5, 0xa6, 0x46, 0x13, 0x6f, 0x6b, 0xd3, 0x59, 0xc2, 0x29, 0x1f, 0xaf, 0x4e, 0x18,
-	0x63, 0x49, 0x84, 0x98, 0xfe, 0xe6, 0xac, 0xfe, 0x9d, 0x24, 0x3f, 0x47, 0xe9, 0xc7, 0xdb, 0xdd,
-	0xa7, 0xfd, 0x34, 0xbe, 0x7b, 0xfb, 0x27, 0x00, 0x00, 0xff, 0xff, 0xae, 0xde, 0xa1, 0xd0, 0xac,
-	0x02, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_api_http_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   3,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_api_http_proto_goTypes,
+		DependencyIndexes: file_google_api_http_proto_depIdxs,
+		MessageInfos:      file_google_api_http_proto_msgTypes,
+	}.Build()
+	File_google_api_http_proto = out.File
+	file_google_api_http_proto_rawDesc = nil
+	file_google_api_http_proto_goTypes = nil
+	file_google_api_http_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/genproto/googleapis/api/annotations/resource.pb.go b/vendor/google.golang.org/genproto/googleapis/api/annotations/resource.pb.go
index af057b9..3571ad6 100644
--- a/vendor/google.golang.org/genproto/googleapis/api/annotations/resource.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/api/annotations/resource.pb.go
@@ -1,26 +1,40 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.26.0
+// 	protoc        v3.12.2
 // source: google/api/resource.proto
 
 package annotations
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
+	sync "sync"
 
-	proto "github.com/golang/protobuf/proto"
-	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	descriptorpb "google.golang.org/protobuf/types/descriptorpb"
 )
 
-// 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
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // A description of the historical or future-looking state of the
 // resource pattern.
@@ -38,24 +52,101 @@
 	ResourceDescriptor_FUTURE_MULTI_PATTERN ResourceDescriptor_History = 2
 )
 
-var ResourceDescriptor_History_name = map[int32]string{
-	0: "HISTORY_UNSPECIFIED",
-	1: "ORIGINALLY_SINGLE_PATTERN",
-	2: "FUTURE_MULTI_PATTERN",
-}
+// Enum value maps for ResourceDescriptor_History.
+var (
+	ResourceDescriptor_History_name = map[int32]string{
+		0: "HISTORY_UNSPECIFIED",
+		1: "ORIGINALLY_SINGLE_PATTERN",
+		2: "FUTURE_MULTI_PATTERN",
+	}
+	ResourceDescriptor_History_value = map[string]int32{
+		"HISTORY_UNSPECIFIED":       0,
+		"ORIGINALLY_SINGLE_PATTERN": 1,
+		"FUTURE_MULTI_PATTERN":      2,
+	}
+)
 
-var ResourceDescriptor_History_value = map[string]int32{
-	"HISTORY_UNSPECIFIED":       0,
-	"ORIGINALLY_SINGLE_PATTERN": 1,
-	"FUTURE_MULTI_PATTERN":      2,
+func (x ResourceDescriptor_History) Enum() *ResourceDescriptor_History {
+	p := new(ResourceDescriptor_History)
+	*p = x
+	return p
 }
 
 func (x ResourceDescriptor_History) String() string {
-	return proto.EnumName(ResourceDescriptor_History_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (ResourceDescriptor_History) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_api_resource_proto_enumTypes[0].Descriptor()
+}
+
+func (ResourceDescriptor_History) Type() protoreflect.EnumType {
+	return &file_google_api_resource_proto_enumTypes[0]
+}
+
+func (x ResourceDescriptor_History) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use ResourceDescriptor_History.Descriptor instead.
 func (ResourceDescriptor_History) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_465e9122405d1bb5, []int{0, 0}
+	return file_google_api_resource_proto_rawDescGZIP(), []int{0, 0}
+}
+
+// A flag representing a specific style that a resource claims to conform to.
+type ResourceDescriptor_Style int32
+
+const (
+	// The unspecified value. Do not use.
+	ResourceDescriptor_STYLE_UNSPECIFIED ResourceDescriptor_Style = 0
+	// This resource is intended to be "declarative-friendly".
+	//
+	// Declarative-friendly resources must be more strictly consistent, and
+	// setting this to true communicates to tools that this resource should
+	// adhere to declarative-friendly expectations.
+	//
+	// Note: This is used by the API linter (linter.aip.dev) to enable
+	// additional checks.
+	ResourceDescriptor_DECLARATIVE_FRIENDLY ResourceDescriptor_Style = 1
+)
+
+// Enum value maps for ResourceDescriptor_Style.
+var (
+	ResourceDescriptor_Style_name = map[int32]string{
+		0: "STYLE_UNSPECIFIED",
+		1: "DECLARATIVE_FRIENDLY",
+	}
+	ResourceDescriptor_Style_value = map[string]int32{
+		"STYLE_UNSPECIFIED":    0,
+		"DECLARATIVE_FRIENDLY": 1,
+	}
+)
+
+func (x ResourceDescriptor_Style) Enum() *ResourceDescriptor_Style {
+	p := new(ResourceDescriptor_Style)
+	*p = x
+	return p
+}
+
+func (x ResourceDescriptor_Style) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (ResourceDescriptor_Style) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_api_resource_proto_enumTypes[1].Descriptor()
+}
+
+func (ResourceDescriptor_Style) Type() protoreflect.EnumType {
+	return &file_google_api_resource_proto_enumTypes[1]
+}
+
+func (x ResourceDescriptor_Style) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use ResourceDescriptor_Style.Descriptor instead.
+func (ResourceDescriptor_Style) EnumDescriptor() ([]byte, []int) {
+	return file_google_api_resource_proto_rawDescGZIP(), []int{0, 1}
 }
 
 // A simple descriptor of a resource type.
@@ -66,31 +157,111 @@
 //
 // Example:
 //
-//   message Topic {
-//     // Indicates this message defines a resource schema.
-//     // Declares the resource type in the format of {service}/{kind}.
-//     // For Kubernetes resources, the format is {api group}/{kind}.
-//     option (google.api.resource) = {
-//       type: "pubsub.googleapis.com/Topic"
-//       pattern: "projects/{project}/topics/{topic}"
-//     };
-//   }
+//     message Topic {
+//       // Indicates this message defines a resource schema.
+//       // Declares the resource type in the format of {service}/{kind}.
+//       // For Kubernetes resources, the format is {api group}/{kind}.
+//       option (google.api.resource) = {
+//         type: "pubsub.googleapis.com/Topic"
+//         name_descriptor: {
+//           pattern: "projects/{project}/topics/{topic}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//           parent_name_extractor: "projects/{project}"
+//         }
+//       };
+//     }
+//
+// The ResourceDescriptor Yaml config will look like:
+//
+//     resources:
+//     - type: "pubsub.googleapis.com/Topic"
+//       name_descriptor:
+//         - pattern: "projects/{project}/topics/{topic}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//           parent_name_extractor: "projects/{project}"
 //
 // Sometimes, resources have multiple patterns, typically because they can
 // live under multiple parents.
 //
 // Example:
 //
-//   message LogEntry {
-//     option (google.api.resource) = {
-//       type: "logging.googleapis.com/LogEntry"
-//       pattern: "projects/{project}/logs/{log}"
-//       pattern: "organizations/{organization}/logs/{log}"
-//       pattern: "folders/{folder}/logs/{log}"
-//       pattern: "billingAccounts/{billing_account}/logs/{log}"
-//     };
-//   }
+//     message LogEntry {
+//       option (google.api.resource) = {
+//         type: "logging.googleapis.com/LogEntry"
+//         name_descriptor: {
+//           pattern: "projects/{project}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//           parent_name_extractor: "projects/{project}"
+//         }
+//         name_descriptor: {
+//           pattern: "folders/{folder}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Folder"
+//           parent_name_extractor: "folders/{folder}"
+//         }
+//         name_descriptor: {
+//           pattern: "organizations/{organization}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Organization"
+//           parent_name_extractor: "organizations/{organization}"
+//         }
+//         name_descriptor: {
+//           pattern: "billingAccounts/{billing_account}/logs/{log}"
+//           parent_type: "billing.googleapis.com/BillingAccount"
+//           parent_name_extractor: "billingAccounts/{billing_account}"
+//         }
+//       };
+//     }
+//
+// The ResourceDescriptor Yaml config will look like:
+//
+//     resources:
+//     - type: 'logging.googleapis.com/LogEntry'
+//       name_descriptor:
+//         - pattern: "projects/{project}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//           parent_name_extractor: "projects/{project}"
+//         - pattern: "folders/{folder}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Folder"
+//           parent_name_extractor: "folders/{folder}"
+//         - pattern: "organizations/{organization}/logs/{log}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Organization"
+//           parent_name_extractor: "organizations/{organization}"
+//         - pattern: "billingAccounts/{billing_account}/logs/{log}"
+//           parent_type: "billing.googleapis.com/BillingAccount"
+//           parent_name_extractor: "billingAccounts/{billing_account}"
+//
+// For flexible resources, the resource name doesn't contain parent names, but
+// the resource itself has parents for policy evaluation.
+//
+// Example:
+//
+//     message Shelf {
+//       option (google.api.resource) = {
+//         type: "library.googleapis.com/Shelf"
+//         name_descriptor: {
+//           pattern: "shelves/{shelf}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//         }
+//         name_descriptor: {
+//           pattern: "shelves/{shelf}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Folder"
+//         }
+//       };
+//     }
+//
+// The ResourceDescriptor Yaml config will look like:
+//
+//     resources:
+//     - type: 'library.googleapis.com/Shelf'
+//       name_descriptor:
+//         - pattern: "shelves/{shelf}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Project"
+//         - pattern: "shelves/{shelf}"
+//           parent_type: "cloudresourcemanager.googleapis.com/Folder"
 type ResourceDescriptor struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// The resource type. It must be in the format of
 	// {service_name}/{resource_type_kind}. The `resource_type_kind` must be
 	// singular and must not include version numbers.
@@ -102,11 +273,20 @@
 	// should use PascalCase (UpperCamelCase). The maximum number of
 	// characters allowed for the `resource_type_kind` is 100.
 	Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
-	// Optional. The valid resource name pattern(s) for this resource type.
+	// Optional. The relative resource name pattern associated with this resource
+	// type. The DNS prefix of the full resource name shouldn't be specified here.
+	//
+	// The path pattern must follow the syntax, which aligns with HTTP binding
+	// syntax:
+	//
+	//     Template = Segment { "/" Segment } ;
+	//     Segment = LITERAL | Variable ;
+	//     Variable = "{" LITERAL "}" ;
 	//
 	// Examples:
-	//   - "projects/{project}/topics/{topic}"
-	//   - "projects/{project}/knowledgeBases/{knowledge_base}"
+	//
+	//     - "projects/{project}/topics/{topic}"
+	//     - "projects/{project}/knowledgeBases/{knowledge_base}"
 	//
 	// The components in braces correspond to the IDs for each resource in the
 	// hierarchy. It is expected that, if multiple patterns are provided,
@@ -119,199 +299,418 @@
 	// Optional. The historical or future-looking state of the resource pattern.
 	//
 	// Example:
-	//   // The InspectTemplate message originally only supported resource
-	//   // names with organization, and project was added later.
-	//   message InspectTemplate {
-	//     option (google.api.resource) = {
-	//       type: "dlp.googleapis.com/InspectTemplate"
-	//       pattern:
-	//       "organizations/{organization}/inspectTemplates/{inspect_template}"
-	//       pattern: "projects/{project}/inspectTemplates/{inspect_template}"
-	//       history: ORIGINALLY_SINGLE_PATTERN
-	//     };
-	//   }
-	History              ResourceDescriptor_History `protobuf:"varint,4,opt,name=history,proto3,enum=google.api.ResourceDescriptor_History" json:"history,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
+	//
+	//     // The InspectTemplate message originally only supported resource
+	//     // names with organization, and project was added later.
+	//     message InspectTemplate {
+	//       option (google.api.resource) = {
+	//         type: "dlp.googleapis.com/InspectTemplate"
+	//         pattern:
+	//         "organizations/{organization}/inspectTemplates/{inspect_template}"
+	//         pattern: "projects/{project}/inspectTemplates/{inspect_template}"
+	//         history: ORIGINALLY_SINGLE_PATTERN
+	//       };
+	//     }
+	History ResourceDescriptor_History `protobuf:"varint,4,opt,name=history,proto3,enum=google.api.ResourceDescriptor_History" json:"history,omitempty"`
+	// The plural name used in the resource name and permission names, such as
+	// 'projects' for the resource name of 'projects/{project}' and the permission
+	// name of 'cloudresourcemanager.googleapis.com/projects.get'. It is the same
+	// concept of the `plural` field in k8s CRD spec
+	// https://kubernetes.io/docs/tasks/access-kubernetes-api/custom-resources/custom-resource-definitions/
+	//
+	// Note: The plural form is required even for singleton resources. See
+	// https://aip.dev/156
+	Plural string `protobuf:"bytes,5,opt,name=plural,proto3" json:"plural,omitempty"`
+	// The same concept of the `singular` field in k8s CRD spec
+	// https://kubernetes.io/docs/tasks/access-kubernetes-api/custom-resources/custom-resource-definitions/
+	// Such as "project" for the `resourcemanager.googleapis.com/Project` type.
+	Singular string `protobuf:"bytes,6,opt,name=singular,proto3" json:"singular,omitempty"`
+	// Style flag(s) for this resource.
+	// These indicate that a resource is expected to conform to a given
+	// style. See the specific style flags for additional information.
+	Style []ResourceDescriptor_Style `protobuf:"varint,10,rep,packed,name=style,proto3,enum=google.api.ResourceDescriptor_Style" json:"style,omitempty"`
 }
 
-func (m *ResourceDescriptor) Reset()         { *m = ResourceDescriptor{} }
-func (m *ResourceDescriptor) String() string { return proto.CompactTextString(m) }
-func (*ResourceDescriptor) ProtoMessage()    {}
+func (x *ResourceDescriptor) Reset() {
+	*x = ResourceDescriptor{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_api_resource_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ResourceDescriptor) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ResourceDescriptor) ProtoMessage() {}
+
+func (x *ResourceDescriptor) ProtoReflect() protoreflect.Message {
+	mi := &file_google_api_resource_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ResourceDescriptor.ProtoReflect.Descriptor instead.
 func (*ResourceDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_465e9122405d1bb5, []int{0}
+	return file_google_api_resource_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *ResourceDescriptor) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResourceDescriptor.Unmarshal(m, b)
-}
-func (m *ResourceDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResourceDescriptor.Marshal(b, m, deterministic)
-}
-func (m *ResourceDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResourceDescriptor.Merge(m, src)
-}
-func (m *ResourceDescriptor) XXX_Size() int {
-	return xxx_messageInfo_ResourceDescriptor.Size(m)
-}
-func (m *ResourceDescriptor) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResourceDescriptor.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResourceDescriptor proto.InternalMessageInfo
-
-func (m *ResourceDescriptor) GetType() string {
-	if m != nil {
-		return m.Type
+func (x *ResourceDescriptor) GetType() string {
+	if x != nil {
+		return x.Type
 	}
 	return ""
 }
 
-func (m *ResourceDescriptor) GetPattern() []string {
-	if m != nil {
-		return m.Pattern
+func (x *ResourceDescriptor) GetPattern() []string {
+	if x != nil {
+		return x.Pattern
 	}
 	return nil
 }
 
-func (m *ResourceDescriptor) GetNameField() string {
-	if m != nil {
-		return m.NameField
+func (x *ResourceDescriptor) GetNameField() string {
+	if x != nil {
+		return x.NameField
 	}
 	return ""
 }
 
-func (m *ResourceDescriptor) GetHistory() ResourceDescriptor_History {
-	if m != nil {
-		return m.History
+func (x *ResourceDescriptor) GetHistory() ResourceDescriptor_History {
+	if x != nil {
+		return x.History
 	}
 	return ResourceDescriptor_HISTORY_UNSPECIFIED
 }
 
-// Defines a proto annotation that describes a field that refers to a resource.
+func (x *ResourceDescriptor) GetPlural() string {
+	if x != nil {
+		return x.Plural
+	}
+	return ""
+}
+
+func (x *ResourceDescriptor) GetSingular() string {
+	if x != nil {
+		return x.Singular
+	}
+	return ""
+}
+
+func (x *ResourceDescriptor) GetStyle() []ResourceDescriptor_Style {
+	if x != nil {
+		return x.Style
+	}
+	return nil
+}
+
+// Defines a proto annotation that describes a string field that refers to
+// an API resource.
 type ResourceReference struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// The resource type that the annotated field references.
 	//
 	// Example:
 	//
-	//   message Subscription {
-	//     string topic = 2 [(google.api.resource_reference) = {
-	//       type = "pubsub.googleapis.com/Topic"
-	//     }];
-	//   }
-	Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
-	// The resource type of a child collection that the annotated field
-	// references. This is useful for `parent` fields where a resource has more
-	// than one possible type of parent.
+	//     message Subscription {
+	//       string topic = 2 [(google.api.resource_reference) = {
+	//         type: "pubsub.googleapis.com/Topic"
+	//       }];
+	//     }
+	//
+	// Occasionally, a field may reference an arbitrary resource. In this case,
+	// APIs use the special value * in their resource reference.
 	//
 	// Example:
 	//
-	//   message ListLogEntriesRequest {
-	//     string parent = 1 [(google.api.resource_reference) = {
-	//       child_type: "logging.googleapis.com/LogEntry"
-	//     };
-	//   }
-	ChildType            string   `protobuf:"bytes,2,opt,name=child_type,json=childType,proto3" json:"child_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	//     message GetIamPolicyRequest {
+	//       string resource = 2 [(google.api.resource_reference) = {
+	//         type: "*"
+	//       }];
+	//     }
+	Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"`
+	// The resource type of a child collection that the annotated field
+	// references. This is useful for annotating the `parent` field that
+	// doesn't have a fixed resource type.
+	//
+	// Example:
+	//
+	//     message ListLogEntriesRequest {
+	//       string parent = 1 [(google.api.resource_reference) = {
+	//         child_type: "logging.googleapis.com/LogEntry"
+	//       };
+	//     }
+	ChildType string `protobuf:"bytes,2,opt,name=child_type,json=childType,proto3" json:"child_type,omitempty"`
 }
 
-func (m *ResourceReference) Reset()         { *m = ResourceReference{} }
-func (m *ResourceReference) String() string { return proto.CompactTextString(m) }
-func (*ResourceReference) ProtoMessage()    {}
+func (x *ResourceReference) Reset() {
+	*x = ResourceReference{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_api_resource_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ResourceReference) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ResourceReference) ProtoMessage() {}
+
+func (x *ResourceReference) ProtoReflect() protoreflect.Message {
+	mi := &file_google_api_resource_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ResourceReference.ProtoReflect.Descriptor instead.
 func (*ResourceReference) Descriptor() ([]byte, []int) {
-	return fileDescriptor_465e9122405d1bb5, []int{1}
+	return file_google_api_resource_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *ResourceReference) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResourceReference.Unmarshal(m, b)
-}
-func (m *ResourceReference) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResourceReference.Marshal(b, m, deterministic)
-}
-func (m *ResourceReference) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResourceReference.Merge(m, src)
-}
-func (m *ResourceReference) XXX_Size() int {
-	return xxx_messageInfo_ResourceReference.Size(m)
-}
-func (m *ResourceReference) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResourceReference.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResourceReference proto.InternalMessageInfo
-
-func (m *ResourceReference) GetType() string {
-	if m != nil {
-		return m.Type
+func (x *ResourceReference) GetType() string {
+	if x != nil {
+		return x.Type
 	}
 	return ""
 }
 
-func (m *ResourceReference) GetChildType() string {
-	if m != nil {
-		return m.ChildType
+func (x *ResourceReference) GetChildType() string {
+	if x != nil {
+		return x.ChildType
 	}
 	return ""
 }
 
-var E_ResourceReference = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.FieldOptions)(nil),
-	ExtensionType: (*ResourceReference)(nil),
-	Field:         1055,
-	Name:          "google.api.resource_reference",
-	Tag:           "bytes,1055,opt,name=resource_reference",
-	Filename:      "google/api/resource.proto",
+var file_google_api_resource_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptorpb.FieldOptions)(nil),
+		ExtensionType: (*ResourceReference)(nil),
+		Field:         1055,
+		Name:          "google.api.resource_reference",
+		Tag:           "bytes,1055,opt,name=resource_reference",
+		Filename:      "google/api/resource.proto",
+	},
+	{
+		ExtendedType:  (*descriptorpb.FileOptions)(nil),
+		ExtensionType: ([]*ResourceDescriptor)(nil),
+		Field:         1053,
+		Name:          "google.api.resource_definition",
+		Tag:           "bytes,1053,rep,name=resource_definition",
+		Filename:      "google/api/resource.proto",
+	},
+	{
+		ExtendedType:  (*descriptorpb.MessageOptions)(nil),
+		ExtensionType: (*ResourceDescriptor)(nil),
+		Field:         1053,
+		Name:          "google.api.resource",
+		Tag:           "bytes,1053,opt,name=resource",
+		Filename:      "google/api/resource.proto",
+	},
 }
 
-var E_Resource = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.MessageOptions)(nil),
-	ExtensionType: (*ResourceDescriptor)(nil),
-	Field:         1053,
-	Name:          "google.api.resource",
-	Tag:           "bytes,1053,opt,name=resource",
-	Filename:      "google/api/resource.proto",
+// Extension fields to descriptorpb.FieldOptions.
+var (
+	// An annotation that describes a resource reference, see
+	// [ResourceReference][].
+	//
+	// optional google.api.ResourceReference resource_reference = 1055;
+	E_ResourceReference = &file_google_api_resource_proto_extTypes[0]
+)
+
+// Extension fields to descriptorpb.FileOptions.
+var (
+	// An annotation that describes a resource definition without a corresponding
+	// message; see [ResourceDescriptor][].
+	//
+	// repeated google.api.ResourceDescriptor resource_definition = 1053;
+	E_ResourceDefinition = &file_google_api_resource_proto_extTypes[1]
+)
+
+// Extension fields to descriptorpb.MessageOptions.
+var (
+	// An annotation that describes a resource definition, see
+	// [ResourceDescriptor][].
+	//
+	// optional google.api.ResourceDescriptor resource = 1053;
+	E_Resource = &file_google_api_resource_proto_extTypes[2]
+)
+
+var File_google_api_resource_proto protoreflect.FileDescriptor
+
+var file_google_api_resource_proto_rawDesc = []byte{
+	0x0a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x72, 0x65, 0x73,
+	0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xaa, 0x03, 0x0a, 0x12, 0x52, 0x65,
+	0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
+	0x74, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18,
+	0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1d,
+	0x0a, 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x40, 0x0a,
+	0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26,
+	0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x52, 0x65, 0x73, 0x6f,
+	0x75, 0x72, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x48,
+	0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12,
+	0x16, 0x0a, 0x06, 0x70, 0x6c, 0x75, 0x72, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x06, 0x70, 0x6c, 0x75, 0x72, 0x61, 0x6c, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x69, 0x6e, 0x67, 0x75,
+	0x6c, 0x61, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x69, 0x6e, 0x67, 0x75,
+	0x6c, 0x61, 0x72, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x79, 0x6c, 0x65, 0x18, 0x0a, 0x20, 0x03,
+	0x28, 0x0e, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e,
+	0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x6f, 0x72, 0x2e, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x52, 0x05, 0x73, 0x74, 0x79, 0x6c, 0x65, 0x22,
+	0x5b, 0x0a, 0x07, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x17, 0x0a, 0x13, 0x48, 0x49,
+	0x53, 0x54, 0x4f, 0x52, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45,
+	0x44, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x4f, 0x52, 0x49, 0x47, 0x49, 0x4e, 0x41, 0x4c, 0x4c,
+	0x59, 0x5f, 0x53, 0x49, 0x4e, 0x47, 0x4c, 0x45, 0x5f, 0x50, 0x41, 0x54, 0x54, 0x45, 0x52, 0x4e,
+	0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x46, 0x55, 0x54, 0x55, 0x52, 0x45, 0x5f, 0x4d, 0x55, 0x4c,
+	0x54, 0x49, 0x5f, 0x50, 0x41, 0x54, 0x54, 0x45, 0x52, 0x4e, 0x10, 0x02, 0x22, 0x38, 0x0a, 0x05,
+	0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x59, 0x4c, 0x45, 0x5f, 0x55,
+	0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14,
+	0x44, 0x45, 0x43, 0x4c, 0x41, 0x52, 0x41, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x46, 0x52, 0x49, 0x45,
+	0x4e, 0x44, 0x4c, 0x59, 0x10, 0x01, 0x22, 0x46, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
+	0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74,
+	0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12,
+	0x1d, 0x0a, 0x0a, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x3a, 0x6c,
+	0x0a, 0x12, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72,
+	0x65, 0x6e, 0x63, 0x65, 0x12, 0x1d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x73, 0x18, 0x9f, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+	0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75,
+	0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x3a, 0x6e, 0x0a, 0x13,
+	0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74,
+	0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x18, 0x9d, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x65,
+	0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x12, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
+	0x63, 0x65, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x5c, 0x0a, 0x08,
+	0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61,
+	0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x9d, 0x08, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x52, 0x65,
+	0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x6e, 0x0a, 0x0e, 0x63, 0x6f,
+	0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x61, 0x70, 0x69, 0x42, 0x0d, 0x52, 0x65,
+	0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x41, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67,
+	0x2f, 0x67, 0x65, 0x6e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x61, 0x70, 0x69, 0x73, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x3b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0xf8, 0x01, 0x01, 0xa2, 0x02, 0x04, 0x47, 0x41, 0x50, 0x49, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x33,
 }
 
-func init() {
-	proto.RegisterEnum("google.api.ResourceDescriptor_History", ResourceDescriptor_History_name, ResourceDescriptor_History_value)
-	proto.RegisterType((*ResourceDescriptor)(nil), "google.api.ResourceDescriptor")
-	proto.RegisterType((*ResourceReference)(nil), "google.api.ResourceReference")
-	proto.RegisterExtension(E_ResourceReference)
-	proto.RegisterExtension(E_Resource)
+var (
+	file_google_api_resource_proto_rawDescOnce sync.Once
+	file_google_api_resource_proto_rawDescData = file_google_api_resource_proto_rawDesc
+)
+
+func file_google_api_resource_proto_rawDescGZIP() []byte {
+	file_google_api_resource_proto_rawDescOnce.Do(func() {
+		file_google_api_resource_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_api_resource_proto_rawDescData)
+	})
+	return file_google_api_resource_proto_rawDescData
 }
 
-func init() { proto.RegisterFile("google/api/resource.proto", fileDescriptor_465e9122405d1bb5) }
+var file_google_api_resource_proto_enumTypes = make([]protoimpl.EnumInfo, 2)
+var file_google_api_resource_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
+var file_google_api_resource_proto_goTypes = []interface{}{
+	(ResourceDescriptor_History)(0),     // 0: google.api.ResourceDescriptor.History
+	(ResourceDescriptor_Style)(0),       // 1: google.api.ResourceDescriptor.Style
+	(*ResourceDescriptor)(nil),          // 2: google.api.ResourceDescriptor
+	(*ResourceReference)(nil),           // 3: google.api.ResourceReference
+	(*descriptorpb.FieldOptions)(nil),   // 4: google.protobuf.FieldOptions
+	(*descriptorpb.FileOptions)(nil),    // 5: google.protobuf.FileOptions
+	(*descriptorpb.MessageOptions)(nil), // 6: google.protobuf.MessageOptions
+}
+var file_google_api_resource_proto_depIdxs = []int32{
+	0, // 0: google.api.ResourceDescriptor.history:type_name -> google.api.ResourceDescriptor.History
+	1, // 1: google.api.ResourceDescriptor.style:type_name -> google.api.ResourceDescriptor.Style
+	4, // 2: google.api.resource_reference:extendee -> google.protobuf.FieldOptions
+	5, // 3: google.api.resource_definition:extendee -> google.protobuf.FileOptions
+	6, // 4: google.api.resource:extendee -> google.protobuf.MessageOptions
+	3, // 5: google.api.resource_reference:type_name -> google.api.ResourceReference
+	2, // 6: google.api.resource_definition:type_name -> google.api.ResourceDescriptor
+	2, // 7: google.api.resource:type_name -> google.api.ResourceDescriptor
+	8, // [8:8] is the sub-list for method output_type
+	8, // [8:8] is the sub-list for method input_type
+	5, // [5:8] is the sub-list for extension type_name
+	2, // [2:5] is the sub-list for extension extendee
+	0, // [0:2] is the sub-list for field type_name
+}
 
-var fileDescriptor_465e9122405d1bb5 = []byte{
-	// 430 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x52, 0x41, 0x6f, 0xd3, 0x30,
-	0x18, 0x25, 0x59, 0x45, 0xd7, 0x0f, 0x31, 0x6d, 0x06, 0x89, 0x0c, 0x29, 0x10, 0xf5, 0x80, 0x7a,
-	0x4a, 0xa4, 0x71, 0x1b, 0x17, 0x3a, 0x96, 0x76, 0x91, 0xba, 0x36, 0x72, 0xd3, 0xc3, 0x00, 0x29,
-	0xf2, 0xd2, 0xaf, 0x59, 0xa4, 0xcc, 0xb6, 0x9c, 0xec, 0xd0, 0x1b, 0x7f, 0x04, 0x21, 0xf1, 0x2b,
-	0x39, 0xa2, 0x3a, 0x71, 0x98, 0xd8, 0xb4, 0x9b, 0xf3, 0xde, 0xfb, 0xbe, 0xf7, 0xfc, 0x1c, 0x38,
-	0xce, 0x85, 0xc8, 0x4b, 0x0c, 0x98, 0x2c, 0x02, 0x85, 0x95, 0xb8, 0x53, 0x19, 0xfa, 0x52, 0x89,
-	0x5a, 0x10, 0x68, 0x28, 0x9f, 0xc9, 0xe2, 0xad, 0xd7, 0xca, 0x34, 0x73, 0x7d, 0xb7, 0x09, 0xd6,
-	0x58, 0x65, 0xaa, 0x90, 0xb5, 0x50, 0x8d, 0x7a, 0xf8, 0xc3, 0x06, 0x42, 0xdb, 0x05, 0xe7, 0x1d,
-	0x49, 0x08, 0xf4, 0xea, 0xad, 0x44, 0xc7, 0xf2, 0xac, 0xd1, 0x80, 0xea, 0x33, 0x71, 0xa0, 0x2f,
-	0x59, 0x5d, 0xa3, 0xe2, 0x8e, 0xed, 0xed, 0x8d, 0x06, 0xd4, 0x7c, 0x12, 0x17, 0x80, 0xb3, 0x5b,
-	0x4c, 0x37, 0x05, 0x96, 0x6b, 0x67, 0x4f, 0xcf, 0x0c, 0x76, 0xc8, 0x64, 0x07, 0x90, 0xcf, 0xd0,
-	0xbf, 0x29, 0xaa, 0x5a, 0xa8, 0xad, 0xd3, 0xf3, 0xac, 0xd1, 0xc1, 0xc9, 0x07, 0xff, 0x5f, 0x46,
-	0xff, 0xa1, 0xbb, 0x7f, 0xd1, 0xa8, 0xa9, 0x19, 0x1b, 0x7e, 0x83, 0x7e, 0x8b, 0x91, 0x37, 0xf0,
-	0xea, 0x22, 0x5a, 0x26, 0x0b, 0x7a, 0x95, 0xae, 0xe6, 0xcb, 0x38, 0xfc, 0x12, 0x4d, 0xa2, 0xf0,
-	0xfc, 0xf0, 0x19, 0x71, 0xe1, 0x78, 0x41, 0xa3, 0x69, 0x34, 0x1f, 0xcf, 0x66, 0x57, 0xe9, 0x32,
-	0x9a, 0x4f, 0x67, 0x61, 0x1a, 0x8f, 0x93, 0x24, 0xa4, 0xf3, 0x43, 0x8b, 0x38, 0xf0, 0x7a, 0xb2,
-	0x4a, 0x56, 0x34, 0x4c, 0x2f, 0x57, 0xb3, 0x24, 0xea, 0x18, 0x7b, 0x38, 0x81, 0x23, 0x93, 0x81,
-	0xe2, 0x06, 0x15, 0xf2, 0x0c, 0x1f, 0x2d, 0xc0, 0x05, 0xc8, 0x6e, 0x8a, 0x72, 0x9d, 0x6a, 0xc6,
-	0x6e, 0xae, 0xa9, 0x91, 0x64, 0x2b, 0xf1, 0xb4, 0x04, 0x62, 0x9e, 0x22, 0x55, 0xdd, 0x22, 0xd7,
-	0xdc, 0xd5, 0xbc, 0x81, 0xaf, 0x4b, 0x59, 0xc8, 0xba, 0x10, 0xbc, 0x72, 0x7e, 0xed, 0x7b, 0xd6,
-	0xe8, 0xc5, 0x89, 0xfb, 0x58, 0x23, 0x5d, 0x1a, 0x7a, 0xa4, 0xfe, 0x87, 0x4e, 0xbf, 0xc3, 0xbe,
-	0x01, 0xc9, 0xfb, 0x07, 0x1e, 0x97, 0x58, 0x55, 0x2c, 0x47, 0xe3, 0xf2, 0xb3, 0x71, 0x79, 0xf7,
-	0x74, 0xef, 0xb4, 0xdb, 0x78, 0xc6, 0xe1, 0x20, 0x13, 0xb7, 0xf7, 0xe4, 0x67, 0x2f, 0x8d, 0x3e,
-	0xde, 0x79, 0xc4, 0xd6, 0xd7, 0x71, 0x4b, 0xe6, 0xa2, 0x64, 0x3c, 0xf7, 0x85, 0xca, 0x83, 0x1c,
-	0xb9, 0x4e, 0x10, 0x34, 0x14, 0x93, 0x45, 0xa5, 0xff, 0x50, 0xc6, 0xb9, 0xa8, 0x99, 0x8e, 0xf2,
-	0xe9, 0xde, 0xf9, 0x8f, 0x65, 0xfd, 0xb6, 0x7b, 0xd3, 0x71, 0x1c, 0x5d, 0x3f, 0xd7, 0x73, 0x1f,
-	0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0xb5, 0x1e, 0x07, 0x80, 0xd8, 0x02, 0x00, 0x00,
+func init() { file_google_api_resource_proto_init() }
+func file_google_api_resource_proto_init() {
+	if File_google_api_resource_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_api_resource_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ResourceDescriptor); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_api_resource_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ResourceReference); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_api_resource_proto_rawDesc,
+			NumEnums:      2,
+			NumMessages:   2,
+			NumExtensions: 3,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_api_resource_proto_goTypes,
+		DependencyIndexes: file_google_api_resource_proto_depIdxs,
+		EnumInfos:         file_google_api_resource_proto_enumTypes,
+		MessageInfos:      file_google_api_resource_proto_msgTypes,
+		ExtensionInfos:    file_google_api_resource_proto_extTypes,
+	}.Build()
+	File_google_api_resource_proto = out.File
+	file_google_api_resource_proto_rawDesc = nil
+	file_google_api_resource_proto_goTypes = nil
+	file_google_api_resource_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/genproto/googleapis/rpc/status/status.pb.go b/vendor/google.golang.org/genproto/googleapis/rpc/status/status.pb.go
index 0b9907f..f34a38e 100644
--- a/vendor/google.golang.org/genproto/googleapis/rpc/status/status.pb.go
+++ b/vendor/google.golang.org/genproto/googleapis/rpc/status/status.pb.go
@@ -1,163 +1,201 @@
+// Copyright 2020 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.26.0
+// 	protoc        v3.12.2
 // source: google/rpc/status.proto
 
 package status
 
 import (
-	fmt "fmt"
-	math "math"
+	reflect "reflect"
+	sync "sync"
 
-	proto "github.com/golang/protobuf/proto"
-	any "github.com/golang/protobuf/ptypes/any"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	anypb "google.golang.org/protobuf/types/known/anypb"
 )
 
-// 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
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // The `Status` type defines a logical error model that is suitable for
 // different programming environments, including REST APIs and RPC APIs. It is
-// used by [gRPC](https://github.com/grpc). The error model is designed to be:
+// used by [gRPC](https://github.com/grpc). Each `Status` message contains
+// three pieces of data: error code, error message, and error details.
 //
-// - Simple to use and understand for most users
-// - Flexible enough to meet unexpected needs
-//
-// # Overview
-//
-// The `Status` message contains three pieces of data: error code, error
-// message, and error details. The error code should be an enum value of
-// [google.rpc.Code][google.rpc.Code], but it may accept additional error codes
-// if needed.  The error message should be a developer-facing English message
-// that helps developers *understand* and *resolve* the error. If a localized
-// user-facing error message is needed, put the localized message in the error
-// details or localize it in the client. The optional error details may contain
-// arbitrary information about the error. There is a predefined set of error
-// detail types in the package `google.rpc` that can be used for common error
-// conditions.
-//
-// # Language mapping
-//
-// The `Status` message is the logical representation of the error model, but it
-// is not necessarily the actual wire format. When the `Status` message is
-// exposed in different client libraries and different wire protocols, it can be
-// mapped differently. For example, it will likely be mapped to some exceptions
-// in Java, but more likely mapped to some error codes in C.
-//
-// # Other uses
-//
-// The error model and the `Status` message can be used in a variety of
-// environments, either with or without APIs, to provide a
-// consistent developer experience across different environments.
-//
-// Example uses of this error model include:
-//
-// - Partial errors. If a service needs to return partial errors to the client,
-//     it may embed the `Status` in the normal response to indicate the partial
-//     errors.
-//
-// - Workflow errors. A typical workflow has multiple steps. Each step may
-//     have a `Status` message for error reporting.
-//
-// - Batch operations. If a client uses batch request and batch response, the
-//     `Status` message should be used directly inside batch response, one for
-//     each error sub-response.
-//
-// - Asynchronous operations. If an API call embeds asynchronous operation
-//     results in its response, the status of those operations should be
-//     represented directly using the `Status` message.
-//
-// - Logging. If some API errors are stored in logs, the message `Status` could
-//     be used directly after any stripping needed for security/privacy reasons.
+// You can find out more about this error model and how to work with it in the
+// [API Design Guide](https://cloud.google.com/apis/design/errors).
 type Status struct {
-	// The status code, which should be an enum value of
-	// [google.rpc.Code][google.rpc.Code].
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The status code, which should be an enum value of [google.rpc.Code][google.rpc.Code].
 	Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"`
 	// A developer-facing error message, which should be in English. Any
 	// user-facing error message should be localized and sent in the
-	// [google.rpc.Status.details][google.rpc.Status.details] field, or localized
-	// by the client.
+	// [google.rpc.Status.details][google.rpc.Status.details] field, or localized by the client.
 	Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
 	// A list of messages that carry the error details.  There is a common set of
 	// message types for APIs to use.
-	Details              []*any.Any `protobuf:"bytes,3,rep,name=details,proto3" json:"details,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	Details []*anypb.Any `protobuf:"bytes,3,rep,name=details,proto3" json:"details,omitempty"`
 }
 
-func (m *Status) Reset()         { *m = Status{} }
-func (m *Status) String() string { return proto.CompactTextString(m) }
-func (*Status) ProtoMessage()    {}
+func (x *Status) Reset() {
+	*x = Status{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_rpc_status_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Status) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Status) ProtoMessage() {}
+
+func (x *Status) ProtoReflect() protoreflect.Message {
+	mi := &file_google_rpc_status_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Status.ProtoReflect.Descriptor instead.
 func (*Status) Descriptor() ([]byte, []int) {
-	return fileDescriptor_24d244abaf643bfe, []int{0}
+	return file_google_rpc_status_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *Status) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Status.Unmarshal(m, b)
-}
-func (m *Status) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Status.Marshal(b, m, deterministic)
-}
-func (m *Status) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Status.Merge(m, src)
-}
-func (m *Status) XXX_Size() int {
-	return xxx_messageInfo_Status.Size(m)
-}
-func (m *Status) XXX_DiscardUnknown() {
-	xxx_messageInfo_Status.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Status proto.InternalMessageInfo
-
-func (m *Status) GetCode() int32 {
-	if m != nil {
-		return m.Code
+func (x *Status) GetCode() int32 {
+	if x != nil {
+		return x.Code
 	}
 	return 0
 }
 
-func (m *Status) GetMessage() string {
-	if m != nil {
-		return m.Message
+func (x *Status) GetMessage() string {
+	if x != nil {
+		return x.Message
 	}
 	return ""
 }
 
-func (m *Status) GetDetails() []*any.Any {
-	if m != nil {
-		return m.Details
+func (x *Status) GetDetails() []*anypb.Any {
+	if x != nil {
+		return x.Details
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterType((*Status)(nil), "google.rpc.Status")
+var File_google_rpc_status_proto protoreflect.FileDescriptor
+
+var file_google_rpc_status_proto_rawDesc = []byte{
+	0x0a, 0x17, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x74, 0x61,
+	0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x72, 0x70, 0x63, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x22, 0x66, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f,
+	0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18,
+	0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61,
+	0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52,
+	0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x61, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x42, 0x0b, 0x53, 0x74, 0x61, 0x74,
+	0x75, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x37, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x65, 0x6e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x61, 0x70, 0x69, 0x73,
+	0x2f, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x3b, 0x73, 0x74, 0x61, 0x74,
+	0x75, 0x73, 0xf8, 0x01, 0x01, 0xa2, 0x02, 0x03, 0x52, 0x50, 0x43, 0x62, 0x06, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("google/rpc/status.proto", fileDescriptor_24d244abaf643bfe) }
+var (
+	file_google_rpc_status_proto_rawDescOnce sync.Once
+	file_google_rpc_status_proto_rawDescData = file_google_rpc_status_proto_rawDesc
+)
 
-var fileDescriptor_24d244abaf643bfe = []byte{
-	// 209 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x4f, 0xcf, 0xcf, 0x4f,
-	0xcf, 0x49, 0xd5, 0x2f, 0x2a, 0x48, 0xd6, 0x2f, 0x2e, 0x49, 0x2c, 0x29, 0x2d, 0xd6, 0x2b, 0x28,
-	0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x82, 0x48, 0xe8, 0x15, 0x15, 0x24, 0x4b, 0x49, 0x42, 0x15, 0x81,
-	0x65, 0x92, 0x4a, 0xd3, 0xf4, 0x13, 0xf3, 0x2a, 0x21, 0xca, 0x94, 0xd2, 0xb8, 0xd8, 0x82, 0xc1,
-	0xda, 0x84, 0x84, 0xb8, 0x58, 0x92, 0xf3, 0x53, 0x52, 0x25, 0x18, 0x15, 0x18, 0x35, 0x58, 0x83,
-	0xc0, 0x6c, 0x21, 0x09, 0x2e, 0xf6, 0xdc, 0xd4, 0xe2, 0xe2, 0xc4, 0xf4, 0x54, 0x09, 0x26, 0x05,
-	0x46, 0x0d, 0xce, 0x20, 0x18, 0x57, 0x48, 0x8f, 0x8b, 0x3d, 0x25, 0xb5, 0x24, 0x31, 0x33, 0xa7,
-	0x58, 0x82, 0x59, 0x81, 0x59, 0x83, 0xdb, 0x48, 0x44, 0x0f, 0x6a, 0x21, 0xcc, 0x12, 0x3d, 0xc7,
-	0xbc, 0xca, 0x20, 0x98, 0x22, 0xa7, 0x38, 0x2e, 0xbe, 0xe4, 0xfc, 0x5c, 0x3d, 0x84, 0xa3, 0x9c,
-	0xb8, 0x21, 0xf6, 0x06, 0x80, 0x94, 0x07, 0x30, 0x46, 0x99, 0x43, 0xa5, 0xd2, 0xf3, 0x73, 0x12,
-	0xf3, 0xd2, 0xf5, 0xf2, 0x8b, 0xd2, 0xf5, 0xd3, 0x53, 0xf3, 0xc0, 0x86, 0xe9, 0x43, 0xa4, 0x12,
-	0x0b, 0x32, 0x8b, 0x91, 0xfc, 0x69, 0x0d, 0xa1, 0x16, 0x31, 0x31, 0x07, 0x05, 0x38, 0x27, 0xb1,
-	0x81, 0x55, 0x1a, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0xa4, 0x53, 0xf0, 0x7c, 0x10, 0x01, 0x00,
-	0x00,
+func file_google_rpc_status_proto_rawDescGZIP() []byte {
+	file_google_rpc_status_proto_rawDescOnce.Do(func() {
+		file_google_rpc_status_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_rpc_status_proto_rawDescData)
+	})
+	return file_google_rpc_status_proto_rawDescData
+}
+
+var file_google_rpc_status_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_google_rpc_status_proto_goTypes = []interface{}{
+	(*Status)(nil),    // 0: google.rpc.Status
+	(*anypb.Any)(nil), // 1: google.protobuf.Any
+}
+var file_google_rpc_status_proto_depIdxs = []int32{
+	1, // 0: google.rpc.Status.details:type_name -> google.protobuf.Any
+	1, // [1:1] is the sub-list for method output_type
+	1, // [1:1] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_google_rpc_status_proto_init() }
+func file_google_rpc_status_proto_init() {
+	if File_google_rpc_status_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_rpc_status_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Status); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_rpc_status_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_rpc_status_proto_goTypes,
+		DependencyIndexes: file_google_rpc_status_proto_depIdxs,
+		MessageInfos:      file_google_rpc_status_proto_msgTypes,
+	}.Build()
+	File_google_rpc_status_proto = out.File
+	file_google_rpc_status_proto_rawDesc = nil
+	file_google_rpc_status_proto_goTypes = nil
+	file_google_rpc_status_proto_depIdxs = nil
 }
diff --git a/vendor/google.golang.org/grpc/reflection/README.md b/vendor/google.golang.org/grpc/reflection/README.md
new file mode 100644
index 0000000..04b6371
--- /dev/null
+++ b/vendor/google.golang.org/grpc/reflection/README.md
@@ -0,0 +1,18 @@
+# Reflection
+
+Package reflection implements server reflection service.
+
+The service implemented is defined in: https://github.com/grpc/grpc/blob/master/src/proto/grpc/reflection/v1alpha/reflection.proto.
+
+To register server reflection on a gRPC server:
+```go
+import "google.golang.org/grpc/reflection"
+
+s := grpc.NewServer()
+pb.RegisterYourOwnServer(s, &server{})
+
+// Register reflection service on gRPC server.
+reflection.Register(s)
+
+s.Serve(lis)
+```
diff --git a/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go
new file mode 100644
index 0000000..0a12ad2
--- /dev/null
+++ b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.pb.go
@@ -0,0 +1,750 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: grpc_reflection_v1alpha/reflection.proto
+
+package grpc_reflection_v1alpha
+
+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
+
+// The message sent by the client when calling ServerReflectionInfo method.
+type ServerReflectionRequest struct {
+	Host string `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"`
+	// To use reflection service, the client should set one of the following
+	// fields in message_request. The server distinguishes requests by their
+	// defined field and then handles them using corresponding methods.
+	//
+	// Types that are valid to be assigned to MessageRequest:
+	//	*ServerReflectionRequest_FileByFilename
+	//	*ServerReflectionRequest_FileContainingSymbol
+	//	*ServerReflectionRequest_FileContainingExtension
+	//	*ServerReflectionRequest_AllExtensionNumbersOfType
+	//	*ServerReflectionRequest_ListServices
+	MessageRequest       isServerReflectionRequest_MessageRequest `protobuf_oneof:"message_request"`
+	XXX_NoUnkeyedLiteral struct{}                                 `json:"-"`
+	XXX_unrecognized     []byte                                   `json:"-"`
+	XXX_sizecache        int32                                    `json:"-"`
+}
+
+func (m *ServerReflectionRequest) Reset()         { *m = ServerReflectionRequest{} }
+func (m *ServerReflectionRequest) String() string { return proto.CompactTextString(m) }
+func (*ServerReflectionRequest) ProtoMessage()    {}
+func (*ServerReflectionRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{0}
+}
+
+func (m *ServerReflectionRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ServerReflectionRequest.Unmarshal(m, b)
+}
+func (m *ServerReflectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ServerReflectionRequest.Marshal(b, m, deterministic)
+}
+func (m *ServerReflectionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ServerReflectionRequest.Merge(m, src)
+}
+func (m *ServerReflectionRequest) XXX_Size() int {
+	return xxx_messageInfo_ServerReflectionRequest.Size(m)
+}
+func (m *ServerReflectionRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_ServerReflectionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ServerReflectionRequest proto.InternalMessageInfo
+
+func (m *ServerReflectionRequest) GetHost() string {
+	if m != nil {
+		return m.Host
+	}
+	return ""
+}
+
+type isServerReflectionRequest_MessageRequest interface {
+	isServerReflectionRequest_MessageRequest()
+}
+
+type ServerReflectionRequest_FileByFilename struct {
+	FileByFilename string `protobuf:"bytes,3,opt,name=file_by_filename,json=fileByFilename,proto3,oneof"`
+}
+
+type ServerReflectionRequest_FileContainingSymbol struct {
+	FileContainingSymbol string `protobuf:"bytes,4,opt,name=file_containing_symbol,json=fileContainingSymbol,proto3,oneof"`
+}
+
+type ServerReflectionRequest_FileContainingExtension struct {
+	FileContainingExtension *ExtensionRequest `protobuf:"bytes,5,opt,name=file_containing_extension,json=fileContainingExtension,proto3,oneof"`
+}
+
+type ServerReflectionRequest_AllExtensionNumbersOfType struct {
+	AllExtensionNumbersOfType string `protobuf:"bytes,6,opt,name=all_extension_numbers_of_type,json=allExtensionNumbersOfType,proto3,oneof"`
+}
+
+type ServerReflectionRequest_ListServices struct {
+	ListServices string `protobuf:"bytes,7,opt,name=list_services,json=listServices,proto3,oneof"`
+}
+
+func (*ServerReflectionRequest_FileByFilename) isServerReflectionRequest_MessageRequest() {}
+
+func (*ServerReflectionRequest_FileContainingSymbol) isServerReflectionRequest_MessageRequest() {}
+
+func (*ServerReflectionRequest_FileContainingExtension) isServerReflectionRequest_MessageRequest() {}
+
+func (*ServerReflectionRequest_AllExtensionNumbersOfType) isServerReflectionRequest_MessageRequest() {}
+
+func (*ServerReflectionRequest_ListServices) isServerReflectionRequest_MessageRequest() {}
+
+func (m *ServerReflectionRequest) GetMessageRequest() isServerReflectionRequest_MessageRequest {
+	if m != nil {
+		return m.MessageRequest
+	}
+	return nil
+}
+
+func (m *ServerReflectionRequest) GetFileByFilename() string {
+	if x, ok := m.GetMessageRequest().(*ServerReflectionRequest_FileByFilename); ok {
+		return x.FileByFilename
+	}
+	return ""
+}
+
+func (m *ServerReflectionRequest) GetFileContainingSymbol() string {
+	if x, ok := m.GetMessageRequest().(*ServerReflectionRequest_FileContainingSymbol); ok {
+		return x.FileContainingSymbol
+	}
+	return ""
+}
+
+func (m *ServerReflectionRequest) GetFileContainingExtension() *ExtensionRequest {
+	if x, ok := m.GetMessageRequest().(*ServerReflectionRequest_FileContainingExtension); ok {
+		return x.FileContainingExtension
+	}
+	return nil
+}
+
+func (m *ServerReflectionRequest) GetAllExtensionNumbersOfType() string {
+	if x, ok := m.GetMessageRequest().(*ServerReflectionRequest_AllExtensionNumbersOfType); ok {
+		return x.AllExtensionNumbersOfType
+	}
+	return ""
+}
+
+func (m *ServerReflectionRequest) GetListServices() string {
+	if x, ok := m.GetMessageRequest().(*ServerReflectionRequest_ListServices); ok {
+		return x.ListServices
+	}
+	return ""
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*ServerReflectionRequest) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*ServerReflectionRequest_FileByFilename)(nil),
+		(*ServerReflectionRequest_FileContainingSymbol)(nil),
+		(*ServerReflectionRequest_FileContainingExtension)(nil),
+		(*ServerReflectionRequest_AllExtensionNumbersOfType)(nil),
+		(*ServerReflectionRequest_ListServices)(nil),
+	}
+}
+
+// The type name and extension number sent by the client when requesting
+// file_containing_extension.
+type ExtensionRequest struct {
+	// Fully-qualified type name. The format should be <package>.<type>
+	ContainingType       string   `protobuf:"bytes,1,opt,name=containing_type,json=containingType,proto3" json:"containing_type,omitempty"`
+	ExtensionNumber      int32    `protobuf:"varint,2,opt,name=extension_number,json=extensionNumber,proto3" json:"extension_number,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ExtensionRequest) Reset()         { *m = ExtensionRequest{} }
+func (m *ExtensionRequest) String() string { return proto.CompactTextString(m) }
+func (*ExtensionRequest) ProtoMessage()    {}
+func (*ExtensionRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{1}
+}
+
+func (m *ExtensionRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ExtensionRequest.Unmarshal(m, b)
+}
+func (m *ExtensionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ExtensionRequest.Marshal(b, m, deterministic)
+}
+func (m *ExtensionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExtensionRequest.Merge(m, src)
+}
+func (m *ExtensionRequest) XXX_Size() int {
+	return xxx_messageInfo_ExtensionRequest.Size(m)
+}
+func (m *ExtensionRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_ExtensionRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ExtensionRequest proto.InternalMessageInfo
+
+func (m *ExtensionRequest) GetContainingType() string {
+	if m != nil {
+		return m.ContainingType
+	}
+	return ""
+}
+
+func (m *ExtensionRequest) GetExtensionNumber() int32 {
+	if m != nil {
+		return m.ExtensionNumber
+	}
+	return 0
+}
+
+// The message sent by the server to answer ServerReflectionInfo method.
+type ServerReflectionResponse struct {
+	ValidHost       string                   `protobuf:"bytes,1,opt,name=valid_host,json=validHost,proto3" json:"valid_host,omitempty"`
+	OriginalRequest *ServerReflectionRequest `protobuf:"bytes,2,opt,name=original_request,json=originalRequest,proto3" json:"original_request,omitempty"`
+	// The server sets one of the following fields according to the
+	// message_request in the request.
+	//
+	// Types that are valid to be assigned to MessageResponse:
+	//	*ServerReflectionResponse_FileDescriptorResponse
+	//	*ServerReflectionResponse_AllExtensionNumbersResponse
+	//	*ServerReflectionResponse_ListServicesResponse
+	//	*ServerReflectionResponse_ErrorResponse
+	MessageResponse      isServerReflectionResponse_MessageResponse `protobuf_oneof:"message_response"`
+	XXX_NoUnkeyedLiteral struct{}                                   `json:"-"`
+	XXX_unrecognized     []byte                                     `json:"-"`
+	XXX_sizecache        int32                                      `json:"-"`
+}
+
+func (m *ServerReflectionResponse) Reset()         { *m = ServerReflectionResponse{} }
+func (m *ServerReflectionResponse) String() string { return proto.CompactTextString(m) }
+func (*ServerReflectionResponse) ProtoMessage()    {}
+func (*ServerReflectionResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{2}
+}
+
+func (m *ServerReflectionResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ServerReflectionResponse.Unmarshal(m, b)
+}
+func (m *ServerReflectionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ServerReflectionResponse.Marshal(b, m, deterministic)
+}
+func (m *ServerReflectionResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ServerReflectionResponse.Merge(m, src)
+}
+func (m *ServerReflectionResponse) XXX_Size() int {
+	return xxx_messageInfo_ServerReflectionResponse.Size(m)
+}
+func (m *ServerReflectionResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_ServerReflectionResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ServerReflectionResponse proto.InternalMessageInfo
+
+func (m *ServerReflectionResponse) GetValidHost() string {
+	if m != nil {
+		return m.ValidHost
+	}
+	return ""
+}
+
+func (m *ServerReflectionResponse) GetOriginalRequest() *ServerReflectionRequest {
+	if m != nil {
+		return m.OriginalRequest
+	}
+	return nil
+}
+
+type isServerReflectionResponse_MessageResponse interface {
+	isServerReflectionResponse_MessageResponse()
+}
+
+type ServerReflectionResponse_FileDescriptorResponse struct {
+	FileDescriptorResponse *FileDescriptorResponse `protobuf:"bytes,4,opt,name=file_descriptor_response,json=fileDescriptorResponse,proto3,oneof"`
+}
+
+type ServerReflectionResponse_AllExtensionNumbersResponse struct {
+	AllExtensionNumbersResponse *ExtensionNumberResponse `protobuf:"bytes,5,opt,name=all_extension_numbers_response,json=allExtensionNumbersResponse,proto3,oneof"`
+}
+
+type ServerReflectionResponse_ListServicesResponse struct {
+	ListServicesResponse *ListServiceResponse `protobuf:"bytes,6,opt,name=list_services_response,json=listServicesResponse,proto3,oneof"`
+}
+
+type ServerReflectionResponse_ErrorResponse struct {
+	ErrorResponse *ErrorResponse `protobuf:"bytes,7,opt,name=error_response,json=errorResponse,proto3,oneof"`
+}
+
+func (*ServerReflectionResponse_FileDescriptorResponse) isServerReflectionResponse_MessageResponse() {}
+
+func (*ServerReflectionResponse_AllExtensionNumbersResponse) isServerReflectionResponse_MessageResponse() {
+}
+
+func (*ServerReflectionResponse_ListServicesResponse) isServerReflectionResponse_MessageResponse() {}
+
+func (*ServerReflectionResponse_ErrorResponse) isServerReflectionResponse_MessageResponse() {}
+
+func (m *ServerReflectionResponse) GetMessageResponse() isServerReflectionResponse_MessageResponse {
+	if m != nil {
+		return m.MessageResponse
+	}
+	return nil
+}
+
+func (m *ServerReflectionResponse) GetFileDescriptorResponse() *FileDescriptorResponse {
+	if x, ok := m.GetMessageResponse().(*ServerReflectionResponse_FileDescriptorResponse); ok {
+		return x.FileDescriptorResponse
+	}
+	return nil
+}
+
+func (m *ServerReflectionResponse) GetAllExtensionNumbersResponse() *ExtensionNumberResponse {
+	if x, ok := m.GetMessageResponse().(*ServerReflectionResponse_AllExtensionNumbersResponse); ok {
+		return x.AllExtensionNumbersResponse
+	}
+	return nil
+}
+
+func (m *ServerReflectionResponse) GetListServicesResponse() *ListServiceResponse {
+	if x, ok := m.GetMessageResponse().(*ServerReflectionResponse_ListServicesResponse); ok {
+		return x.ListServicesResponse
+	}
+	return nil
+}
+
+func (m *ServerReflectionResponse) GetErrorResponse() *ErrorResponse {
+	if x, ok := m.GetMessageResponse().(*ServerReflectionResponse_ErrorResponse); ok {
+		return x.ErrorResponse
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*ServerReflectionResponse) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*ServerReflectionResponse_FileDescriptorResponse)(nil),
+		(*ServerReflectionResponse_AllExtensionNumbersResponse)(nil),
+		(*ServerReflectionResponse_ListServicesResponse)(nil),
+		(*ServerReflectionResponse_ErrorResponse)(nil),
+	}
+}
+
+// Serialized FileDescriptorProto messages sent by the server answering
+// a file_by_filename, file_containing_symbol, or file_containing_extension
+// request.
+type FileDescriptorResponse struct {
+	// Serialized FileDescriptorProto messages. We avoid taking a dependency on
+	// descriptor.proto, which uses proto2 only features, by making them opaque
+	// bytes instead.
+	FileDescriptorProto  [][]byte `protobuf:"bytes,1,rep,name=file_descriptor_proto,json=fileDescriptorProto,proto3" json:"file_descriptor_proto,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *FileDescriptorResponse) Reset()         { *m = FileDescriptorResponse{} }
+func (m *FileDescriptorResponse) String() string { return proto.CompactTextString(m) }
+func (*FileDescriptorResponse) ProtoMessage()    {}
+func (*FileDescriptorResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{3}
+}
+
+func (m *FileDescriptorResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FileDescriptorResponse.Unmarshal(m, b)
+}
+func (m *FileDescriptorResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FileDescriptorResponse.Marshal(b, m, deterministic)
+}
+func (m *FileDescriptorResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FileDescriptorResponse.Merge(m, src)
+}
+func (m *FileDescriptorResponse) XXX_Size() int {
+	return xxx_messageInfo_FileDescriptorResponse.Size(m)
+}
+func (m *FileDescriptorResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_FileDescriptorResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FileDescriptorResponse proto.InternalMessageInfo
+
+func (m *FileDescriptorResponse) GetFileDescriptorProto() [][]byte {
+	if m != nil {
+		return m.FileDescriptorProto
+	}
+	return nil
+}
+
+// A list of extension numbers sent by the server answering
+// all_extension_numbers_of_type request.
+type ExtensionNumberResponse struct {
+	// Full name of the base type, including the package name. The format
+	// is <package>.<type>
+	BaseTypeName         string   `protobuf:"bytes,1,opt,name=base_type_name,json=baseTypeName,proto3" json:"base_type_name,omitempty"`
+	ExtensionNumber      []int32  `protobuf:"varint,2,rep,packed,name=extension_number,json=extensionNumber,proto3" json:"extension_number,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ExtensionNumberResponse) Reset()         { *m = ExtensionNumberResponse{} }
+func (m *ExtensionNumberResponse) String() string { return proto.CompactTextString(m) }
+func (*ExtensionNumberResponse) ProtoMessage()    {}
+func (*ExtensionNumberResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{4}
+}
+
+func (m *ExtensionNumberResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ExtensionNumberResponse.Unmarshal(m, b)
+}
+func (m *ExtensionNumberResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ExtensionNumberResponse.Marshal(b, m, deterministic)
+}
+func (m *ExtensionNumberResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExtensionNumberResponse.Merge(m, src)
+}
+func (m *ExtensionNumberResponse) XXX_Size() int {
+	return xxx_messageInfo_ExtensionNumberResponse.Size(m)
+}
+func (m *ExtensionNumberResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_ExtensionNumberResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ExtensionNumberResponse proto.InternalMessageInfo
+
+func (m *ExtensionNumberResponse) GetBaseTypeName() string {
+	if m != nil {
+		return m.BaseTypeName
+	}
+	return ""
+}
+
+func (m *ExtensionNumberResponse) GetExtensionNumber() []int32 {
+	if m != nil {
+		return m.ExtensionNumber
+	}
+	return nil
+}
+
+// A list of ServiceResponse sent by the server answering list_services request.
+type ListServiceResponse struct {
+	// The information of each service may be expanded in the future, so we use
+	// ServiceResponse message to encapsulate it.
+	Service              []*ServiceResponse `protobuf:"bytes,1,rep,name=service,proto3" json:"service,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
+	XXX_unrecognized     []byte             `json:"-"`
+	XXX_sizecache        int32              `json:"-"`
+}
+
+func (m *ListServiceResponse) Reset()         { *m = ListServiceResponse{} }
+func (m *ListServiceResponse) String() string { return proto.CompactTextString(m) }
+func (*ListServiceResponse) ProtoMessage()    {}
+func (*ListServiceResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{5}
+}
+
+func (m *ListServiceResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ListServiceResponse.Unmarshal(m, b)
+}
+func (m *ListServiceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ListServiceResponse.Marshal(b, m, deterministic)
+}
+func (m *ListServiceResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ListServiceResponse.Merge(m, src)
+}
+func (m *ListServiceResponse) XXX_Size() int {
+	return xxx_messageInfo_ListServiceResponse.Size(m)
+}
+func (m *ListServiceResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_ListServiceResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ListServiceResponse proto.InternalMessageInfo
+
+func (m *ListServiceResponse) GetService() []*ServiceResponse {
+	if m != nil {
+		return m.Service
+	}
+	return nil
+}
+
+// The information of a single service used by ListServiceResponse to answer
+// list_services request.
+type ServiceResponse struct {
+	// Full name of a registered service, including its package name. The format
+	// is <package>.<service>
+	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ServiceResponse) Reset()         { *m = ServiceResponse{} }
+func (m *ServiceResponse) String() string { return proto.CompactTextString(m) }
+func (*ServiceResponse) ProtoMessage()    {}
+func (*ServiceResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{6}
+}
+
+func (m *ServiceResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ServiceResponse.Unmarshal(m, b)
+}
+func (m *ServiceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ServiceResponse.Marshal(b, m, deterministic)
+}
+func (m *ServiceResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ServiceResponse.Merge(m, src)
+}
+func (m *ServiceResponse) XXX_Size() int {
+	return xxx_messageInfo_ServiceResponse.Size(m)
+}
+func (m *ServiceResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_ServiceResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ServiceResponse proto.InternalMessageInfo
+
+func (m *ServiceResponse) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+// The error code and error message sent by the server when an error occurs.
+type ErrorResponse struct {
+	// This field uses the error codes defined in grpc::StatusCode.
+	ErrorCode            int32    `protobuf:"varint,1,opt,name=error_code,json=errorCode,proto3" json:"error_code,omitempty"`
+	ErrorMessage         string   `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ErrorResponse) Reset()         { *m = ErrorResponse{} }
+func (m *ErrorResponse) String() string { return proto.CompactTextString(m) }
+func (*ErrorResponse) ProtoMessage()    {}
+func (*ErrorResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_42a8ac412db3cb03, []int{7}
+}
+
+func (m *ErrorResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ErrorResponse.Unmarshal(m, b)
+}
+func (m *ErrorResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ErrorResponse.Marshal(b, m, deterministic)
+}
+func (m *ErrorResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ErrorResponse.Merge(m, src)
+}
+func (m *ErrorResponse) XXX_Size() int {
+	return xxx_messageInfo_ErrorResponse.Size(m)
+}
+func (m *ErrorResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_ErrorResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ErrorResponse proto.InternalMessageInfo
+
+func (m *ErrorResponse) GetErrorCode() int32 {
+	if m != nil {
+		return m.ErrorCode
+	}
+	return 0
+}
+
+func (m *ErrorResponse) GetErrorMessage() string {
+	if m != nil {
+		return m.ErrorMessage
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterType((*ServerReflectionRequest)(nil), "grpc.reflection.v1alpha.ServerReflectionRequest")
+	proto.RegisterType((*ExtensionRequest)(nil), "grpc.reflection.v1alpha.ExtensionRequest")
+	proto.RegisterType((*ServerReflectionResponse)(nil), "grpc.reflection.v1alpha.ServerReflectionResponse")
+	proto.RegisterType((*FileDescriptorResponse)(nil), "grpc.reflection.v1alpha.FileDescriptorResponse")
+	proto.RegisterType((*ExtensionNumberResponse)(nil), "grpc.reflection.v1alpha.ExtensionNumberResponse")
+	proto.RegisterType((*ListServiceResponse)(nil), "grpc.reflection.v1alpha.ListServiceResponse")
+	proto.RegisterType((*ServiceResponse)(nil), "grpc.reflection.v1alpha.ServiceResponse")
+	proto.RegisterType((*ErrorResponse)(nil), "grpc.reflection.v1alpha.ErrorResponse")
+}
+
+func init() {
+	proto.RegisterFile("grpc_reflection_v1alpha/reflection.proto", fileDescriptor_42a8ac412db3cb03)
+}
+
+var fileDescriptor_42a8ac412db3cb03 = []byte{
+	// 656 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0x51, 0x73, 0xd2, 0x40,
+	0x10, 0x6e, 0x5a, 0x68, 0x87, 0x85, 0x02, 0x5e, 0x2b, 0xa4, 0x3a, 0x75, 0x98, 0x68, 0x35, 0x75,
+	0x1c, 0xda, 0xe2, 0x8c, 0x3f, 0x80, 0xaa, 0x83, 0x33, 0xb5, 0x75, 0x0e, 0x5f, 0x1c, 0x1f, 0x6e,
+	0x02, 0x2c, 0x34, 0x1a, 0x72, 0xf1, 0x2e, 0x45, 0x79, 0xf2, 0x47, 0xf8, 0xa3, 0xfc, 0x4b, 0x3e,
+	0x3a, 0x77, 0x09, 0x21, 0xa4, 0x44, 0xa7, 0x4f, 0x30, 0xdf, 0xee, 0xde, 0xb7, 0xbb, 0xdf, 0xb7,
+	0x01, 0x7b, 0x22, 0x82, 0x21, 0x13, 0x38, 0xf6, 0x70, 0x18, 0xba, 0xdc, 0x67, 0xb3, 0x33, 0xc7,
+	0x0b, 0xae, 0x9d, 0x93, 0x25, 0xd4, 0x0e, 0x04, 0x0f, 0x39, 0x69, 0xaa, 0xcc, 0x76, 0x0a, 0x8e,
+	0x33, 0xad, 0x3f, 0x9b, 0xd0, 0xec, 0xa3, 0x98, 0xa1, 0xa0, 0x49, 0x90, 0xe2, 0xb7, 0x1b, 0x94,
+	0x21, 0x21, 0x50, 0xb8, 0xe6, 0x32, 0x34, 0x8d, 0x96, 0x61, 0x97, 0xa8, 0xfe, 0x4f, 0x9e, 0x43,
+	0x7d, 0xec, 0x7a, 0xc8, 0x06, 0x73, 0xa6, 0x7e, 0x7d, 0x67, 0x8a, 0xe6, 0x96, 0x8a, 0xf7, 0x36,
+	0x68, 0x55, 0x21, 0xdd, 0xf9, 0xdb, 0x18, 0x27, 0xaf, 0xa0, 0xa1, 0x73, 0x87, 0xdc, 0x0f, 0x1d,
+	0xd7, 0x77, 0xfd, 0x09, 0x93, 0xf3, 0xe9, 0x80, 0x7b, 0x66, 0x21, 0xae, 0xd8, 0x57, 0xf1, 0xf3,
+	0x24, 0xdc, 0xd7, 0x51, 0x32, 0x81, 0x83, 0x6c, 0x1d, 0xfe, 0x08, 0xd1, 0x97, 0x2e, 0xf7, 0xcd,
+	0x62, 0xcb, 0xb0, 0xcb, 0x9d, 0xe3, 0x76, 0xce, 0x40, 0xed, 0x37, 0x8b, 0xcc, 0x78, 0x8a, 0xde,
+	0x06, 0x6d, 0xae, 0xb2, 0x24, 0x19, 0xa4, 0x0b, 0x87, 0x8e, 0xe7, 0x2d, 0x1f, 0x67, 0xfe, 0xcd,
+	0x74, 0x80, 0x42, 0x32, 0x3e, 0x66, 0xe1, 0x3c, 0x40, 0x73, 0x3b, 0xee, 0xf3, 0xc0, 0xf1, 0xbc,
+	0xa4, 0xec, 0x32, 0x4a, 0xba, 0x1a, 0x7f, 0x9c, 0x07, 0x48, 0x8e, 0x60, 0xd7, 0x73, 0x65, 0xc8,
+	0x24, 0x8a, 0x99, 0x3b, 0x44, 0x69, 0xee, 0xc4, 0x35, 0x15, 0x05, 0xf7, 0x63, 0xb4, 0x7b, 0x0f,
+	0x6a, 0x53, 0x94, 0xd2, 0x99, 0x20, 0x13, 0x51, 0x63, 0xd6, 0x18, 0xea, 0xd9, 0x66, 0xc9, 0x33,
+	0xa8, 0xa5, 0xa6, 0xd6, 0x3d, 0x44, 0xdb, 0xaf, 0x2e, 0x61, 0x4d, 0x7b, 0x0c, 0xf5, 0x6c, 0xdb,
+	0xe6, 0x66, 0xcb, 0xb0, 0x8b, 0xb4, 0x86, 0xab, 0x8d, 0x5a, 0xbf, 0x0b, 0x60, 0xde, 0x96, 0x58,
+	0x06, 0xdc, 0x97, 0x48, 0x0e, 0x01, 0x66, 0x8e, 0xe7, 0x8e, 0x58, 0x4a, 0xe9, 0x92, 0x46, 0x7a,
+	0x4a, 0xee, 0xcf, 0x50, 0xe7, 0xc2, 0x9d, 0xb8, 0xbe, 0xe3, 0x2d, 0xfa, 0xd6, 0x34, 0xe5, 0xce,
+	0x69, 0xae, 0x02, 0x39, 0x76, 0xa2, 0xb5, 0xc5, 0x4b, 0x8b, 0x61, 0xbf, 0x82, 0xa9, 0x75, 0x1e,
+	0xa1, 0x1c, 0x0a, 0x37, 0x08, 0xb9, 0x60, 0x22, 0xee, 0x4b, 0x3b, 0xa4, 0xdc, 0x39, 0xc9, 0x25,
+	0x51, 0x26, 0x7b, 0x9d, 0xd4, 0x2d, 0xc6, 0xe9, 0x6d, 0x50, 0x6d, 0xb9, 0xdb, 0x11, 0xf2, 0x1d,
+	0x1e, 0xad, 0xd7, 0x3a, 0xa1, 0x2c, 0xfe, 0x67, 0xae, 0x8c, 0x01, 0x52, 0x9c, 0x0f, 0xd7, 0xd8,
+	0x23, 0x21, 0x1e, 0x41, 0x63, 0xc5, 0x20, 0x4b, 0xc2, 0x6d, 0x4d, 0xf8, 0x22, 0x97, 0xf0, 0x62,
+	0x69, 0xa0, 0x14, 0xd9, 0x7e, 0xda, 0x57, 0x09, 0xcb, 0x15, 0x54, 0x51, 0x88, 0xf4, 0x06, 0x77,
+	0xf4, 0xeb, 0x4f, 0xf3, 0xc7, 0x51, 0xe9, 0xa9, 0x77, 0x77, 0x31, 0x0d, 0x74, 0x09, 0xd4, 0x97,
+	0x86, 0x8d, 0x30, 0xeb, 0x02, 0x1a, 0xeb, 0xf7, 0x4e, 0x3a, 0x70, 0x3f, 0x2b, 0xa5, 0xfe, 0xf0,
+	0x98, 0x46, 0x6b, 0xcb, 0xae, 0xd0, 0xbd, 0x55, 0x51, 0x3e, 0xa8, 0x90, 0xf5, 0x05, 0x9a, 0x39,
+	0x2b, 0x25, 0x4f, 0xa0, 0x3a, 0x70, 0x24, 0xea, 0x03, 0x60, 0xfa, 0x1b, 0x13, 0x39, 0xb3, 0xa2,
+	0x50, 0xe5, 0xff, 0x4b, 0xf5, 0x7d, 0x59, 0x7f, 0x03, 0x5b, 0xeb, 0x6e, 0xe0, 0x13, 0xec, 0xad,
+	0xd9, 0x26, 0xe9, 0xc2, 0x4e, 0x2c, 0x8b, 0x6e, 0xb4, 0xdc, 0xb1, 0xff, 0xe9, 0xea, 0x54, 0x29,
+	0x5d, 0x14, 0x5a, 0x47, 0x50, 0xcb, 0x3e, 0x4b, 0xa0, 0x90, 0x6a, 0x5a, 0xff, 0xb7, 0xfa, 0xb0,
+	0xbb, 0xb2, 0x71, 0x75, 0x79, 0x91, 0x62, 0x43, 0x3e, 0x8a, 0x52, 0x8b, 0xb4, 0xa4, 0x91, 0x73,
+	0x3e, 0x42, 0xf2, 0x18, 0x22, 0x41, 0x58, 0xac, 0x82, 0x3e, 0xbb, 0x12, 0xad, 0x68, 0xf0, 0x7d,
+	0x84, 0x75, 0x7e, 0x19, 0x50, 0xcf, 0x9e, 0x1b, 0xf9, 0x09, 0xfb, 0x59, 0xec, 0x9d, 0x3f, 0xe6,
+	0xe4, 0xce, 0x17, 0xfb, 0xe0, 0xec, 0x0e, 0x15, 0xd1, 0x54, 0xb6, 0x71, 0x6a, 0x0c, 0xb6, 0xb5,
+	0xf4, 0x2f, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x85, 0x02, 0x09, 0x9d, 0x9f, 0x06, 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
+
+// ServerReflectionClient is the client API for ServerReflection service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type ServerReflectionClient interface {
+	// The reflection service is structured as a bidirectional stream, ensuring
+	// all related requests go to a single server.
+	ServerReflectionInfo(ctx context.Context, opts ...grpc.CallOption) (ServerReflection_ServerReflectionInfoClient, error)
+}
+
+type serverReflectionClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewServerReflectionClient(cc *grpc.ClientConn) ServerReflectionClient {
+	return &serverReflectionClient{cc}
+}
+
+func (c *serverReflectionClient) ServerReflectionInfo(ctx context.Context, opts ...grpc.CallOption) (ServerReflection_ServerReflectionInfoClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_ServerReflection_serviceDesc.Streams[0], "/grpc.reflection.v1alpha.ServerReflection/ServerReflectionInfo", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &serverReflectionServerReflectionInfoClient{stream}
+	return x, nil
+}
+
+type ServerReflection_ServerReflectionInfoClient interface {
+	Send(*ServerReflectionRequest) error
+	Recv() (*ServerReflectionResponse, error)
+	grpc.ClientStream
+}
+
+type serverReflectionServerReflectionInfoClient struct {
+	grpc.ClientStream
+}
+
+func (x *serverReflectionServerReflectionInfoClient) Send(m *ServerReflectionRequest) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *serverReflectionServerReflectionInfoClient) Recv() (*ServerReflectionResponse, error) {
+	m := new(ServerReflectionResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// ServerReflectionServer is the server API for ServerReflection service.
+type ServerReflectionServer interface {
+	// The reflection service is structured as a bidirectional stream, ensuring
+	// all related requests go to a single server.
+	ServerReflectionInfo(ServerReflection_ServerReflectionInfoServer) error
+}
+
+// UnimplementedServerReflectionServer can be embedded to have forward compatible implementations.
+type UnimplementedServerReflectionServer struct {
+}
+
+func (*UnimplementedServerReflectionServer) ServerReflectionInfo(srv ServerReflection_ServerReflectionInfoServer) error {
+	return status.Errorf(codes.Unimplemented, "method ServerReflectionInfo not implemented")
+}
+
+func RegisterServerReflectionServer(s *grpc.Server, srv ServerReflectionServer) {
+	s.RegisterService(&_ServerReflection_serviceDesc, srv)
+}
+
+func _ServerReflection_ServerReflectionInfo_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(ServerReflectionServer).ServerReflectionInfo(&serverReflectionServerReflectionInfoServer{stream})
+}
+
+type ServerReflection_ServerReflectionInfoServer interface {
+	Send(*ServerReflectionResponse) error
+	Recv() (*ServerReflectionRequest, error)
+	grpc.ServerStream
+}
+
+type serverReflectionServerReflectionInfoServer struct {
+	grpc.ServerStream
+}
+
+func (x *serverReflectionServerReflectionInfoServer) Send(m *ServerReflectionResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *serverReflectionServerReflectionInfoServer) Recv() (*ServerReflectionRequest, error) {
+	m := new(ServerReflectionRequest)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+var _ServerReflection_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "grpc.reflection.v1alpha.ServerReflection",
+	HandlerType: (*ServerReflectionServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "ServerReflectionInfo",
+			Handler:       _ServerReflection_ServerReflectionInfo_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+	},
+	Metadata: "grpc_reflection_v1alpha/reflection.proto",
+}
diff --git a/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.proto b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.proto
new file mode 100644
index 0000000..99b00df
--- /dev/null
+++ b/vendor/google.golang.org/grpc/reflection/grpc_reflection_v1alpha/reflection.proto
@@ -0,0 +1,136 @@
+// Copyright 2016 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.
+
+// Service exported by server reflection
+
+syntax = "proto3";
+
+package grpc.reflection.v1alpha;
+
+service ServerReflection {
+  // The reflection service is structured as a bidirectional stream, ensuring
+  // all related requests go to a single server.
+  rpc ServerReflectionInfo(stream ServerReflectionRequest)
+      returns (stream ServerReflectionResponse);
+}
+
+// The message sent by the client when calling ServerReflectionInfo method.
+message ServerReflectionRequest {
+  string host = 1;
+  // To use reflection service, the client should set one of the following
+  // fields in message_request. The server distinguishes requests by their
+  // defined field and then handles them using corresponding methods.
+  oneof message_request {
+    // Find a proto file by the file name.
+    string file_by_filename = 3;
+
+    // Find the proto file that declares the given fully-qualified symbol name.
+    // This field should be a fully-qualified symbol name
+    // (e.g. <package>.<service>[.<method>] or <package>.<type>).
+    string file_containing_symbol = 4;
+
+    // Find the proto file which defines an extension extending the given
+    // message type with the given field number.
+    ExtensionRequest file_containing_extension = 5;
+
+    // Finds the tag numbers used by all known extensions of extendee_type, and
+    // appends them to ExtensionNumberResponse in an undefined order.
+    // Its corresponding method is best-effort: it's not guaranteed that the
+    // reflection service will implement this method, and it's not guaranteed
+    // that this method will provide all extensions. Returns
+    // StatusCode::UNIMPLEMENTED if it's not implemented.
+    // This field should be a fully-qualified type name. The format is
+    // <package>.<type>
+    string all_extension_numbers_of_type = 6;
+
+    // List the full names of registered services. The content will not be
+    // checked.
+    string list_services = 7;
+  }
+}
+
+// The type name and extension number sent by the client when requesting
+// file_containing_extension.
+message ExtensionRequest {
+  // Fully-qualified type name. The format should be <package>.<type>
+  string containing_type = 1;
+  int32 extension_number = 2;
+}
+
+// The message sent by the server to answer ServerReflectionInfo method.
+message ServerReflectionResponse {
+  string valid_host = 1;
+  ServerReflectionRequest original_request = 2;
+  // The server sets one of the following fields according to the
+  // message_request in the request.
+  oneof message_response {
+    // This message is used to answer file_by_filename, file_containing_symbol,
+    // file_containing_extension requests with transitive dependencies.
+    // As the repeated label is not allowed in oneof fields, we use a
+    // FileDescriptorResponse message to encapsulate the repeated fields.
+    // The reflection service is allowed to avoid sending FileDescriptorProtos
+    // that were previously sent in response to earlier requests in the stream.
+    FileDescriptorResponse file_descriptor_response = 4;
+
+    // This message is used to answer all_extension_numbers_of_type requests.
+    ExtensionNumberResponse all_extension_numbers_response = 5;
+
+    // This message is used to answer list_services requests.
+    ListServiceResponse list_services_response = 6;
+
+    // This message is used when an error occurs.
+    ErrorResponse error_response = 7;
+  }
+}
+
+// Serialized FileDescriptorProto messages sent by the server answering
+// a file_by_filename, file_containing_symbol, or file_containing_extension
+// request.
+message FileDescriptorResponse {
+  // Serialized FileDescriptorProto messages. We avoid taking a dependency on
+  // descriptor.proto, which uses proto2 only features, by making them opaque
+  // bytes instead.
+  repeated bytes file_descriptor_proto = 1;
+}
+
+// A list of extension numbers sent by the server answering
+// all_extension_numbers_of_type request.
+message ExtensionNumberResponse {
+  // Full name of the base type, including the package name. The format
+  // is <package>.<type>
+  string base_type_name = 1;
+  repeated int32 extension_number = 2;
+}
+
+// A list of ServiceResponse sent by the server answering list_services request.
+message ListServiceResponse {
+  // The information of each service may be expanded in the future, so we use
+  // ServiceResponse message to encapsulate it.
+  repeated ServiceResponse service = 1;
+}
+
+// The information of a single service used by ListServiceResponse to answer
+// list_services request.
+message ServiceResponse {
+  // Full name of a registered service, including its package name. The format
+  // is <package>.<service>
+  string name = 1;
+}
+
+// The error code and error message sent by the server when an error occurs.
+message ErrorResponse {
+  // This field uses the error codes defined in grpc::StatusCode.
+  int32 error_code = 1;
+  string error_message = 2;
+}
diff --git a/vendor/google.golang.org/grpc/reflection/serverreflection.go b/vendor/google.golang.org/grpc/reflection/serverreflection.go
new file mode 100644
index 0000000..dd22a2d
--- /dev/null
+++ b/vendor/google.golang.org/grpc/reflection/serverreflection.go
@@ -0,0 +1,454 @@
+/*
+ *
+ * Copyright 2016 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 protoc --go_out=plugins=grpc:. grpc_reflection_v1alpha/reflection.proto
+
+/*
+Package reflection implements server reflection service.
+
+The service implemented is defined in:
+https://github.com/grpc/grpc/blob/master/src/proto/grpc/reflection/v1alpha/reflection.proto.
+
+To register server reflection on a gRPC server:
+	import "google.golang.org/grpc/reflection"
+
+	s := grpc.NewServer()
+	pb.RegisterYourOwnServer(s, &server{})
+
+	// Register reflection service on gRPC server.
+	reflection.Register(s)
+
+	s.Serve(lis)
+
+*/
+package reflection // import "google.golang.org/grpc/reflection"
+
+import (
+	"bytes"
+	"compress/gzip"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"reflect"
+	"sort"
+	"sync"
+
+	"github.com/golang/protobuf/proto"
+	dpb "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	rpb "google.golang.org/grpc/reflection/grpc_reflection_v1alpha"
+	"google.golang.org/grpc/status"
+)
+
+type serverReflectionServer struct {
+	s *grpc.Server
+
+	initSymbols  sync.Once
+	serviceNames []string
+	symbols      map[string]*dpb.FileDescriptorProto // map of fully-qualified names to files
+}
+
+// Register registers the server reflection service on the given gRPC server.
+func Register(s *grpc.Server) {
+	rpb.RegisterServerReflectionServer(s, &serverReflectionServer{
+		s: s,
+	})
+}
+
+// protoMessage is used for type assertion on proto messages.
+// Generated proto message implements function Descriptor(), but Descriptor()
+// is not part of interface proto.Message. This interface is needed to
+// call Descriptor().
+type protoMessage interface {
+	Descriptor() ([]byte, []int)
+}
+
+func (s *serverReflectionServer) getSymbols() (svcNames []string, symbolIndex map[string]*dpb.FileDescriptorProto) {
+	s.initSymbols.Do(func() {
+		serviceInfo := s.s.GetServiceInfo()
+
+		s.symbols = map[string]*dpb.FileDescriptorProto{}
+		s.serviceNames = make([]string, 0, len(serviceInfo))
+		processed := map[string]struct{}{}
+		for svc, info := range serviceInfo {
+			s.serviceNames = append(s.serviceNames, svc)
+			fdenc, ok := parseMetadata(info.Metadata)
+			if !ok {
+				continue
+			}
+			fd, err := decodeFileDesc(fdenc)
+			if err != nil {
+				continue
+			}
+			s.processFile(fd, processed)
+		}
+		sort.Strings(s.serviceNames)
+	})
+
+	return s.serviceNames, s.symbols
+}
+
+func (s *serverReflectionServer) processFile(fd *dpb.FileDescriptorProto, processed map[string]struct{}) {
+	filename := fd.GetName()
+	if _, ok := processed[filename]; ok {
+		return
+	}
+	processed[filename] = struct{}{}
+
+	prefix := fd.GetPackage()
+
+	for _, msg := range fd.MessageType {
+		s.processMessage(fd, prefix, msg)
+	}
+	for _, en := range fd.EnumType {
+		s.processEnum(fd, prefix, en)
+	}
+	for _, ext := range fd.Extension {
+		s.processField(fd, prefix, ext)
+	}
+	for _, svc := range fd.Service {
+		svcName := fqn(prefix, svc.GetName())
+		s.symbols[svcName] = fd
+		for _, meth := range svc.Method {
+			name := fqn(svcName, meth.GetName())
+			s.symbols[name] = fd
+		}
+	}
+
+	for _, dep := range fd.Dependency {
+		fdenc := proto.FileDescriptor(dep)
+		fdDep, err := decodeFileDesc(fdenc)
+		if err != nil {
+			continue
+		}
+		s.processFile(fdDep, processed)
+	}
+}
+
+func (s *serverReflectionServer) processMessage(fd *dpb.FileDescriptorProto, prefix string, msg *dpb.DescriptorProto) {
+	msgName := fqn(prefix, msg.GetName())
+	s.symbols[msgName] = fd
+
+	for _, nested := range msg.NestedType {
+		s.processMessage(fd, msgName, nested)
+	}
+	for _, en := range msg.EnumType {
+		s.processEnum(fd, msgName, en)
+	}
+	for _, ext := range msg.Extension {
+		s.processField(fd, msgName, ext)
+	}
+	for _, fld := range msg.Field {
+		s.processField(fd, msgName, fld)
+	}
+	for _, oneof := range msg.OneofDecl {
+		oneofName := fqn(msgName, oneof.GetName())
+		s.symbols[oneofName] = fd
+	}
+}
+
+func (s *serverReflectionServer) processEnum(fd *dpb.FileDescriptorProto, prefix string, en *dpb.EnumDescriptorProto) {
+	enName := fqn(prefix, en.GetName())
+	s.symbols[enName] = fd
+
+	for _, val := range en.Value {
+		valName := fqn(enName, val.GetName())
+		s.symbols[valName] = fd
+	}
+}
+
+func (s *serverReflectionServer) processField(fd *dpb.FileDescriptorProto, prefix string, fld *dpb.FieldDescriptorProto) {
+	fldName := fqn(prefix, fld.GetName())
+	s.symbols[fldName] = fd
+}
+
+func fqn(prefix, name string) string {
+	if prefix == "" {
+		return name
+	}
+	return prefix + "." + name
+}
+
+// fileDescForType gets the file descriptor for the given type.
+// The given type should be a proto message.
+func (s *serverReflectionServer) fileDescForType(st reflect.Type) (*dpb.FileDescriptorProto, error) {
+	m, ok := reflect.Zero(reflect.PtrTo(st)).Interface().(protoMessage)
+	if !ok {
+		return nil, fmt.Errorf("failed to create message from type: %v", st)
+	}
+	enc, _ := m.Descriptor()
+
+	return decodeFileDesc(enc)
+}
+
+// decodeFileDesc does decompression and unmarshalling on the given
+// file descriptor byte slice.
+func decodeFileDesc(enc []byte) (*dpb.FileDescriptorProto, error) {
+	raw, err := decompress(enc)
+	if err != nil {
+		return nil, fmt.Errorf("failed to decompress enc: %v", err)
+	}
+
+	fd := new(dpb.FileDescriptorProto)
+	if err := proto.Unmarshal(raw, fd); err != nil {
+		return nil, fmt.Errorf("bad descriptor: %v", err)
+	}
+	return fd, nil
+}
+
+// decompress does gzip decompression.
+func decompress(b []byte) ([]byte, error) {
+	r, err := gzip.NewReader(bytes.NewReader(b))
+	if err != nil {
+		return nil, fmt.Errorf("bad gzipped descriptor: %v", err)
+	}
+	out, err := ioutil.ReadAll(r)
+	if err != nil {
+		return nil, fmt.Errorf("bad gzipped descriptor: %v", err)
+	}
+	return out, nil
+}
+
+func typeForName(name string) (reflect.Type, error) {
+	pt := proto.MessageType(name)
+	if pt == nil {
+		return nil, fmt.Errorf("unknown type: %q", name)
+	}
+	st := pt.Elem()
+
+	return st, nil
+}
+
+func fileDescContainingExtension(st reflect.Type, ext int32) (*dpb.FileDescriptorProto, error) {
+	m, ok := reflect.Zero(reflect.PtrTo(st)).Interface().(proto.Message)
+	if !ok {
+		return nil, fmt.Errorf("failed to create message from type: %v", st)
+	}
+
+	var extDesc *proto.ExtensionDesc
+	for id, desc := range proto.RegisteredExtensions(m) {
+		if id == ext {
+			extDesc = desc
+			break
+		}
+	}
+
+	if extDesc == nil {
+		return nil, fmt.Errorf("failed to find registered extension for extension number %v", ext)
+	}
+
+	return decodeFileDesc(proto.FileDescriptor(extDesc.Filename))
+}
+
+func (s *serverReflectionServer) allExtensionNumbersForType(st reflect.Type) ([]int32, error) {
+	m, ok := reflect.Zero(reflect.PtrTo(st)).Interface().(proto.Message)
+	if !ok {
+		return nil, fmt.Errorf("failed to create message from type: %v", st)
+	}
+
+	exts := proto.RegisteredExtensions(m)
+	out := make([]int32, 0, len(exts))
+	for id := range exts {
+		out = append(out, id)
+	}
+	return out, nil
+}
+
+// fileDescEncodingByFilename finds the file descriptor for given filename,
+// does marshalling on it and returns the marshalled result.
+func (s *serverReflectionServer) fileDescEncodingByFilename(name string) ([]byte, error) {
+	enc := proto.FileDescriptor(name)
+	if enc == nil {
+		return nil, fmt.Errorf("unknown file: %v", name)
+	}
+	fd, err := decodeFileDesc(enc)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Marshal(fd)
+}
+
+// parseMetadata finds the file descriptor bytes specified meta.
+// For SupportPackageIsVersion4, m is the name of the proto file, we
+// call proto.FileDescriptor to get the byte slice.
+// For SupportPackageIsVersion3, m is a byte slice itself.
+func parseMetadata(meta interface{}) ([]byte, bool) {
+	// Check if meta is the file name.
+	if fileNameForMeta, ok := meta.(string); ok {
+		return proto.FileDescriptor(fileNameForMeta), true
+	}
+
+	// Check if meta is the byte slice.
+	if enc, ok := meta.([]byte); ok {
+		return enc, true
+	}
+
+	return nil, false
+}
+
+// fileDescEncodingContainingSymbol finds the file descriptor containing the given symbol,
+// does marshalling on it and returns the marshalled result.
+// The given symbol can be a type, a service or a method.
+func (s *serverReflectionServer) fileDescEncodingContainingSymbol(name string) ([]byte, error) {
+	_, symbols := s.getSymbols()
+	fd := symbols[name]
+	if fd == nil {
+		// Check if it's a type name that was not present in the
+		// transitive dependencies of the registered services.
+		if st, err := typeForName(name); err == nil {
+			fd, err = s.fileDescForType(st)
+			if err != nil {
+				return nil, err
+			}
+		}
+	}
+
+	if fd == nil {
+		return nil, fmt.Errorf("unknown symbol: %v", name)
+	}
+
+	return proto.Marshal(fd)
+}
+
+// fileDescEncodingContainingExtension finds the file descriptor containing given extension,
+// does marshalling on it and returns the marshalled result.
+func (s *serverReflectionServer) fileDescEncodingContainingExtension(typeName string, extNum int32) ([]byte, error) {
+	st, err := typeForName(typeName)
+	if err != nil {
+		return nil, err
+	}
+	fd, err := fileDescContainingExtension(st, extNum)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Marshal(fd)
+}
+
+// allExtensionNumbersForTypeName returns all extension numbers for the given type.
+func (s *serverReflectionServer) allExtensionNumbersForTypeName(name string) ([]int32, error) {
+	st, err := typeForName(name)
+	if err != nil {
+		return nil, err
+	}
+	extNums, err := s.allExtensionNumbersForType(st)
+	if err != nil {
+		return nil, err
+	}
+	return extNums, nil
+}
+
+// ServerReflectionInfo is the reflection service handler.
+func (s *serverReflectionServer) ServerReflectionInfo(stream rpb.ServerReflection_ServerReflectionInfoServer) error {
+	for {
+		in, err := stream.Recv()
+		if err == io.EOF {
+			return nil
+		}
+		if err != nil {
+			return err
+		}
+
+		out := &rpb.ServerReflectionResponse{
+			ValidHost:       in.Host,
+			OriginalRequest: in,
+		}
+		switch req := in.MessageRequest.(type) {
+		case *rpb.ServerReflectionRequest_FileByFilename:
+			b, err := s.fileDescEncodingByFilename(req.FileByFilename)
+			if err != nil {
+				out.MessageResponse = &rpb.ServerReflectionResponse_ErrorResponse{
+					ErrorResponse: &rpb.ErrorResponse{
+						ErrorCode:    int32(codes.NotFound),
+						ErrorMessage: err.Error(),
+					},
+				}
+			} else {
+				out.MessageResponse = &rpb.ServerReflectionResponse_FileDescriptorResponse{
+					FileDescriptorResponse: &rpb.FileDescriptorResponse{FileDescriptorProto: [][]byte{b}},
+				}
+			}
+		case *rpb.ServerReflectionRequest_FileContainingSymbol:
+			b, err := s.fileDescEncodingContainingSymbol(req.FileContainingSymbol)
+			if err != nil {
+				out.MessageResponse = &rpb.ServerReflectionResponse_ErrorResponse{
+					ErrorResponse: &rpb.ErrorResponse{
+						ErrorCode:    int32(codes.NotFound),
+						ErrorMessage: err.Error(),
+					},
+				}
+			} else {
+				out.MessageResponse = &rpb.ServerReflectionResponse_FileDescriptorResponse{
+					FileDescriptorResponse: &rpb.FileDescriptorResponse{FileDescriptorProto: [][]byte{b}},
+				}
+			}
+		case *rpb.ServerReflectionRequest_FileContainingExtension:
+			typeName := req.FileContainingExtension.ContainingType
+			extNum := req.FileContainingExtension.ExtensionNumber
+			b, err := s.fileDescEncodingContainingExtension(typeName, extNum)
+			if err != nil {
+				out.MessageResponse = &rpb.ServerReflectionResponse_ErrorResponse{
+					ErrorResponse: &rpb.ErrorResponse{
+						ErrorCode:    int32(codes.NotFound),
+						ErrorMessage: err.Error(),
+					},
+				}
+			} else {
+				out.MessageResponse = &rpb.ServerReflectionResponse_FileDescriptorResponse{
+					FileDescriptorResponse: &rpb.FileDescriptorResponse{FileDescriptorProto: [][]byte{b}},
+				}
+			}
+		case *rpb.ServerReflectionRequest_AllExtensionNumbersOfType:
+			extNums, err := s.allExtensionNumbersForTypeName(req.AllExtensionNumbersOfType)
+			if err != nil {
+				out.MessageResponse = &rpb.ServerReflectionResponse_ErrorResponse{
+					ErrorResponse: &rpb.ErrorResponse{
+						ErrorCode:    int32(codes.NotFound),
+						ErrorMessage: err.Error(),
+					},
+				}
+			} else {
+				out.MessageResponse = &rpb.ServerReflectionResponse_AllExtensionNumbersResponse{
+					AllExtensionNumbersResponse: &rpb.ExtensionNumberResponse{
+						BaseTypeName:    req.AllExtensionNumbersOfType,
+						ExtensionNumber: extNums,
+					},
+				}
+			}
+		case *rpb.ServerReflectionRequest_ListServices:
+			svcNames, _ := s.getSymbols()
+			serviceResponses := make([]*rpb.ServiceResponse, len(svcNames))
+			for i, n := range svcNames {
+				serviceResponses[i] = &rpb.ServiceResponse{
+					Name: n,
+				}
+			}
+			out.MessageResponse = &rpb.ServerReflectionResponse_ListServicesResponse{
+				ListServicesResponse: &rpb.ListServiceResponse{
+					Service: serviceResponses,
+				},
+			}
+		default:
+			return status.Errorf(codes.InvalidArgument, "invalid MessageRequest: %v", in.MessageRequest)
+		}
+
+		if err := stream.Send(out); err != nil {
+			return err
+		}
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/AUTHORS b/vendor/google.golang.org/protobuf/AUTHORS
new file mode 100644
index 0000000..2b00ddb
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/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 https://tip.golang.org/AUTHORS.
diff --git a/vendor/google.golang.org/protobuf/CONTRIBUTORS b/vendor/google.golang.org/protobuf/CONTRIBUTORS
new file mode 100644
index 0000000..1fbd3e9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/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 https://tip.golang.org/CONTRIBUTORS.
diff --git a/vendor/google.golang.org/protobuf/LICENSE b/vendor/google.golang.org/protobuf/LICENSE
new file mode 100644
index 0000000..49ea0f9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/LICENSE
@@ -0,0 +1,27 @@
+Copyright (c) 2018 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/google.golang.org/protobuf/PATENTS b/vendor/google.golang.org/protobuf/PATENTS
new file mode 100644
index 0000000..7330990
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/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/google.golang.org/protobuf/encoding/prototext/decode.go b/vendor/google.golang.org/protobuf/encoding/prototext/decode.go
new file mode 100644
index 0000000..179d6e8
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/encoding/prototext/decode.go
@@ -0,0 +1,770 @@
+// 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 prototext
+
+import (
+	"fmt"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/encoding/text"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/internal/set"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// Unmarshal reads the given []byte into the given proto.Message.
+// The provided message must be mutable (e.g., a non-nil pointer to a message).
+func Unmarshal(b []byte, m proto.Message) error {
+	return UnmarshalOptions{}.Unmarshal(b, m)
+}
+
+// UnmarshalOptions is a configurable textproto format unmarshaler.
+type UnmarshalOptions struct {
+	pragma.NoUnkeyedLiterals
+
+	// AllowPartial accepts input for messages that will result in missing
+	// required fields. If AllowPartial is false (the default), Unmarshal will
+	// return error if there are any missing required fields.
+	AllowPartial bool
+
+	// DiscardUnknown specifies whether to ignore unknown fields when parsing.
+	// An unknown field is any field whose field name or field number does not
+	// resolve to any known or extension field in the message.
+	// By default, unmarshal rejects unknown fields as an error.
+	DiscardUnknown bool
+
+	// Resolver is used for looking up types when unmarshaling
+	// google.protobuf.Any messages or extension fields.
+	// If nil, this defaults to using protoregistry.GlobalTypes.
+	Resolver interface {
+		protoregistry.MessageTypeResolver
+		protoregistry.ExtensionTypeResolver
+	}
+}
+
+// Unmarshal reads the given []byte and populates the given proto.Message
+// using options in the UnmarshalOptions object.
+// The provided message must be mutable (e.g., a non-nil pointer to a message).
+func (o UnmarshalOptions) Unmarshal(b []byte, m proto.Message) error {
+	return o.unmarshal(b, m)
+}
+
+// unmarshal is a centralized function that all unmarshal operations go through.
+// For profiling purposes, avoid changing the name of this function or
+// introducing other code paths for unmarshal that do not go through this.
+func (o UnmarshalOptions) unmarshal(b []byte, m proto.Message) error {
+	proto.Reset(m)
+
+	if o.Resolver == nil {
+		o.Resolver = protoregistry.GlobalTypes
+	}
+
+	dec := decoder{text.NewDecoder(b), o}
+	if err := dec.unmarshalMessage(m.ProtoReflect(), false); err != nil {
+		return err
+	}
+	if o.AllowPartial {
+		return nil
+	}
+	return proto.CheckInitialized(m)
+}
+
+type decoder struct {
+	*text.Decoder
+	opts UnmarshalOptions
+}
+
+// newError returns an error object with position info.
+func (d decoder) newError(pos int, f string, x ...interface{}) error {
+	line, column := d.Position(pos)
+	head := fmt.Sprintf("(line %d:%d): ", line, column)
+	return errors.New(head+f, x...)
+}
+
+// unexpectedTokenError returns a syntax error for the given unexpected token.
+func (d decoder) unexpectedTokenError(tok text.Token) error {
+	return d.syntaxError(tok.Pos(), "unexpected token: %s", tok.RawString())
+}
+
+// syntaxError returns a syntax error for given position.
+func (d decoder) syntaxError(pos int, f string, x ...interface{}) error {
+	line, column := d.Position(pos)
+	head := fmt.Sprintf("syntax error (line %d:%d): ", line, column)
+	return errors.New(head+f, x...)
+}
+
+// unmarshalMessage unmarshals into the given protoreflect.Message.
+func (d decoder) unmarshalMessage(m pref.Message, checkDelims bool) error {
+	messageDesc := m.Descriptor()
+	if !flags.ProtoLegacy && messageset.IsMessageSet(messageDesc) {
+		return errors.New("no support for proto1 MessageSets")
+	}
+
+	if messageDesc.FullName() == genid.Any_message_fullname {
+		return d.unmarshalAny(m, checkDelims)
+	}
+
+	if checkDelims {
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+
+		if tok.Kind() != text.MessageOpen {
+			return d.unexpectedTokenError(tok)
+		}
+	}
+
+	var seenNums set.Ints
+	var seenOneofs set.Ints
+	fieldDescs := messageDesc.Fields()
+
+	for {
+		// Read field name.
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+		switch typ := tok.Kind(); typ {
+		case text.Name:
+			// Continue below.
+		case text.EOF:
+			if checkDelims {
+				return text.ErrUnexpectedEOF
+			}
+			return nil
+		default:
+			if checkDelims && typ == text.MessageClose {
+				return nil
+			}
+			return d.unexpectedTokenError(tok)
+		}
+
+		// Resolve the field descriptor.
+		var name pref.Name
+		var fd pref.FieldDescriptor
+		var xt pref.ExtensionType
+		var xtErr error
+		var isFieldNumberName bool
+
+		switch tok.NameKind() {
+		case text.IdentName:
+			name = pref.Name(tok.IdentName())
+			fd = fieldDescs.ByTextName(string(name))
+
+		case text.TypeName:
+			// Handle extensions only. This code path is not for Any.
+			xt, xtErr = d.opts.Resolver.FindExtensionByName(pref.FullName(tok.TypeName()))
+
+		case text.FieldNumber:
+			isFieldNumberName = true
+			num := pref.FieldNumber(tok.FieldNumber())
+			if !num.IsValid() {
+				return d.newError(tok.Pos(), "invalid field number: %d", num)
+			}
+			fd = fieldDescs.ByNumber(num)
+			if fd == nil {
+				xt, xtErr = d.opts.Resolver.FindExtensionByNumber(messageDesc.FullName(), num)
+			}
+		}
+
+		if xt != nil {
+			fd = xt.TypeDescriptor()
+			if !messageDesc.ExtensionRanges().Has(fd.Number()) || fd.ContainingMessage().FullName() != messageDesc.FullName() {
+				return d.newError(tok.Pos(), "message %v cannot be extended by %v", messageDesc.FullName(), fd.FullName())
+			}
+		} else if xtErr != nil && xtErr != protoregistry.NotFound {
+			return d.newError(tok.Pos(), "unable to resolve [%s]: %v", tok.RawString(), xtErr)
+		}
+		if flags.ProtoLegacy {
+			if fd != nil && fd.IsWeak() && fd.Message().IsPlaceholder() {
+				fd = nil // reset since the weak reference is not linked in
+			}
+		}
+
+		// Handle unknown fields.
+		if fd == nil {
+			if d.opts.DiscardUnknown || messageDesc.ReservedNames().Has(name) {
+				d.skipValue()
+				continue
+			}
+			return d.newError(tok.Pos(), "unknown field: %v", tok.RawString())
+		}
+
+		// Handle fields identified by field number.
+		if isFieldNumberName {
+			// TODO: Add an option to permit parsing field numbers.
+			//
+			// This requires careful thought as the MarshalOptions.EmitUnknown
+			// option allows formatting unknown fields as the field number and the
+			// best-effort textual representation of the field value.  In that case,
+			// it may not be possible to unmarshal the value from a parser that does
+			// have information about the unknown field.
+			return d.newError(tok.Pos(), "cannot specify field by number: %v", tok.RawString())
+		}
+
+		switch {
+		case fd.IsList():
+			kind := fd.Kind()
+			if kind != pref.MessageKind && kind != pref.GroupKind && !tok.HasSeparator() {
+				return d.syntaxError(tok.Pos(), "missing field separator :")
+			}
+
+			list := m.Mutable(fd).List()
+			if err := d.unmarshalList(fd, list); err != nil {
+				return err
+			}
+
+		case fd.IsMap():
+			mmap := m.Mutable(fd).Map()
+			if err := d.unmarshalMap(fd, mmap); err != nil {
+				return err
+			}
+
+		default:
+			kind := fd.Kind()
+			if kind != pref.MessageKind && kind != pref.GroupKind && !tok.HasSeparator() {
+				return d.syntaxError(tok.Pos(), "missing field separator :")
+			}
+
+			// If field is a oneof, check if it has already been set.
+			if od := fd.ContainingOneof(); od != nil {
+				idx := uint64(od.Index())
+				if seenOneofs.Has(idx) {
+					return d.newError(tok.Pos(), "error parsing %q, oneof %v is already set", tok.RawString(), od.FullName())
+				}
+				seenOneofs.Set(idx)
+			}
+
+			num := uint64(fd.Number())
+			if seenNums.Has(num) {
+				return d.newError(tok.Pos(), "non-repeated field %q is repeated", tok.RawString())
+			}
+
+			if err := d.unmarshalSingular(fd, m); err != nil {
+				return err
+			}
+			seenNums.Set(num)
+		}
+	}
+
+	return nil
+}
+
+// unmarshalSingular unmarshals a non-repeated field value specified by the
+// given FieldDescriptor.
+func (d decoder) unmarshalSingular(fd pref.FieldDescriptor, m pref.Message) error {
+	var val pref.Value
+	var err error
+	switch fd.Kind() {
+	case pref.MessageKind, pref.GroupKind:
+		val = m.NewField(fd)
+		err = d.unmarshalMessage(val.Message(), true)
+	default:
+		val, err = d.unmarshalScalar(fd)
+	}
+	if err == nil {
+		m.Set(fd, val)
+	}
+	return err
+}
+
+// unmarshalScalar unmarshals a scalar/enum protoreflect.Value specified by the
+// given FieldDescriptor.
+func (d decoder) unmarshalScalar(fd pref.FieldDescriptor) (pref.Value, error) {
+	tok, err := d.Read()
+	if err != nil {
+		return pref.Value{}, err
+	}
+
+	if tok.Kind() != text.Scalar {
+		return pref.Value{}, d.unexpectedTokenError(tok)
+	}
+
+	kind := fd.Kind()
+	switch kind {
+	case pref.BoolKind:
+		if b, ok := tok.Bool(); ok {
+			return pref.ValueOfBool(b), nil
+		}
+
+	case pref.Int32Kind, pref.Sint32Kind, pref.Sfixed32Kind:
+		if n, ok := tok.Int32(); ok {
+			return pref.ValueOfInt32(n), nil
+		}
+
+	case pref.Int64Kind, pref.Sint64Kind, pref.Sfixed64Kind:
+		if n, ok := tok.Int64(); ok {
+			return pref.ValueOfInt64(n), nil
+		}
+
+	case pref.Uint32Kind, pref.Fixed32Kind:
+		if n, ok := tok.Uint32(); ok {
+			return pref.ValueOfUint32(n), nil
+		}
+
+	case pref.Uint64Kind, pref.Fixed64Kind:
+		if n, ok := tok.Uint64(); ok {
+			return pref.ValueOfUint64(n), nil
+		}
+
+	case pref.FloatKind:
+		if n, ok := tok.Float32(); ok {
+			return pref.ValueOfFloat32(n), nil
+		}
+
+	case pref.DoubleKind:
+		if n, ok := tok.Float64(); ok {
+			return pref.ValueOfFloat64(n), nil
+		}
+
+	case pref.StringKind:
+		if s, ok := tok.String(); ok {
+			if strs.EnforceUTF8(fd) && !utf8.ValidString(s) {
+				return pref.Value{}, d.newError(tok.Pos(), "contains invalid UTF-8")
+			}
+			return pref.ValueOfString(s), nil
+		}
+
+	case pref.BytesKind:
+		if b, ok := tok.String(); ok {
+			return pref.ValueOfBytes([]byte(b)), nil
+		}
+
+	case pref.EnumKind:
+		if lit, ok := tok.Enum(); ok {
+			// Lookup EnumNumber based on name.
+			if enumVal := fd.Enum().Values().ByName(pref.Name(lit)); enumVal != nil {
+				return pref.ValueOfEnum(enumVal.Number()), nil
+			}
+		}
+		if num, ok := tok.Int32(); ok {
+			return pref.ValueOfEnum(pref.EnumNumber(num)), nil
+		}
+
+	default:
+		panic(fmt.Sprintf("invalid scalar kind %v", kind))
+	}
+
+	return pref.Value{}, d.newError(tok.Pos(), "invalid value for %v type: %v", kind, tok.RawString())
+}
+
+// unmarshalList unmarshals into given protoreflect.List. A list value can
+// either be in [] syntax or simply just a single scalar/message value.
+func (d decoder) unmarshalList(fd pref.FieldDescriptor, list pref.List) error {
+	tok, err := d.Peek()
+	if err != nil {
+		return err
+	}
+
+	switch fd.Kind() {
+	case pref.MessageKind, pref.GroupKind:
+		switch tok.Kind() {
+		case text.ListOpen:
+			d.Read()
+			for {
+				tok, err := d.Peek()
+				if err != nil {
+					return err
+				}
+
+				switch tok.Kind() {
+				case text.ListClose:
+					d.Read()
+					return nil
+				case text.MessageOpen:
+					pval := list.NewElement()
+					if err := d.unmarshalMessage(pval.Message(), true); err != nil {
+						return err
+					}
+					list.Append(pval)
+				default:
+					return d.unexpectedTokenError(tok)
+				}
+			}
+
+		case text.MessageOpen:
+			pval := list.NewElement()
+			if err := d.unmarshalMessage(pval.Message(), true); err != nil {
+				return err
+			}
+			list.Append(pval)
+			return nil
+		}
+
+	default:
+		switch tok.Kind() {
+		case text.ListOpen:
+			d.Read()
+			for {
+				tok, err := d.Peek()
+				if err != nil {
+					return err
+				}
+
+				switch tok.Kind() {
+				case text.ListClose:
+					d.Read()
+					return nil
+				case text.Scalar:
+					pval, err := d.unmarshalScalar(fd)
+					if err != nil {
+						return err
+					}
+					list.Append(pval)
+				default:
+					return d.unexpectedTokenError(tok)
+				}
+			}
+
+		case text.Scalar:
+			pval, err := d.unmarshalScalar(fd)
+			if err != nil {
+				return err
+			}
+			list.Append(pval)
+			return nil
+		}
+	}
+
+	return d.unexpectedTokenError(tok)
+}
+
+// unmarshalMap unmarshals into given protoreflect.Map. A map value is a
+// textproto message containing {key: <kvalue>, value: <mvalue>}.
+func (d decoder) unmarshalMap(fd pref.FieldDescriptor, mmap pref.Map) error {
+	// Determine ahead whether map entry is a scalar type or a message type in
+	// order to call the appropriate unmarshalMapValue func inside
+	// unmarshalMapEntry.
+	var unmarshalMapValue func() (pref.Value, error)
+	switch fd.MapValue().Kind() {
+	case pref.MessageKind, pref.GroupKind:
+		unmarshalMapValue = func() (pref.Value, error) {
+			pval := mmap.NewValue()
+			if err := d.unmarshalMessage(pval.Message(), true); err != nil {
+				return pref.Value{}, err
+			}
+			return pval, nil
+		}
+	default:
+		unmarshalMapValue = func() (pref.Value, error) {
+			return d.unmarshalScalar(fd.MapValue())
+		}
+	}
+
+	tok, err := d.Read()
+	if err != nil {
+		return err
+	}
+	switch tok.Kind() {
+	case text.MessageOpen:
+		return d.unmarshalMapEntry(fd, mmap, unmarshalMapValue)
+
+	case text.ListOpen:
+		for {
+			tok, err := d.Read()
+			if err != nil {
+				return err
+			}
+			switch tok.Kind() {
+			case text.ListClose:
+				return nil
+			case text.MessageOpen:
+				if err := d.unmarshalMapEntry(fd, mmap, unmarshalMapValue); err != nil {
+					return err
+				}
+			default:
+				return d.unexpectedTokenError(tok)
+			}
+		}
+
+	default:
+		return d.unexpectedTokenError(tok)
+	}
+}
+
+// unmarshalMap unmarshals into given protoreflect.Map. A map value is a
+// textproto message containing {key: <kvalue>, value: <mvalue>}.
+func (d decoder) unmarshalMapEntry(fd pref.FieldDescriptor, mmap pref.Map, unmarshalMapValue func() (pref.Value, error)) error {
+	var key pref.MapKey
+	var pval pref.Value
+Loop:
+	for {
+		// Read field name.
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+		switch tok.Kind() {
+		case text.Name:
+			if tok.NameKind() != text.IdentName {
+				if !d.opts.DiscardUnknown {
+					return d.newError(tok.Pos(), "unknown map entry field %q", tok.RawString())
+				}
+				d.skipValue()
+				continue Loop
+			}
+			// Continue below.
+		case text.MessageClose:
+			break Loop
+		default:
+			return d.unexpectedTokenError(tok)
+		}
+
+		switch name := pref.Name(tok.IdentName()); name {
+		case genid.MapEntry_Key_field_name:
+			if !tok.HasSeparator() {
+				return d.syntaxError(tok.Pos(), "missing field separator :")
+			}
+			if key.IsValid() {
+				return d.newError(tok.Pos(), "map entry %q cannot be repeated", name)
+			}
+			val, err := d.unmarshalScalar(fd.MapKey())
+			if err != nil {
+				return err
+			}
+			key = val.MapKey()
+
+		case genid.MapEntry_Value_field_name:
+			if kind := fd.MapValue().Kind(); (kind != pref.MessageKind) && (kind != pref.GroupKind) {
+				if !tok.HasSeparator() {
+					return d.syntaxError(tok.Pos(), "missing field separator :")
+				}
+			}
+			if pval.IsValid() {
+				return d.newError(tok.Pos(), "map entry %q cannot be repeated", name)
+			}
+			pval, err = unmarshalMapValue()
+			if err != nil {
+				return err
+			}
+
+		default:
+			if !d.opts.DiscardUnknown {
+				return d.newError(tok.Pos(), "unknown map entry field %q", name)
+			}
+			d.skipValue()
+		}
+	}
+
+	if !key.IsValid() {
+		key = fd.MapKey().Default().MapKey()
+	}
+	if !pval.IsValid() {
+		switch fd.MapValue().Kind() {
+		case pref.MessageKind, pref.GroupKind:
+			// If value field is not set for message/group types, construct an
+			// empty one as default.
+			pval = mmap.NewValue()
+		default:
+			pval = fd.MapValue().Default()
+		}
+	}
+	mmap.Set(key, pval)
+	return nil
+}
+
+// unmarshalAny unmarshals an Any textproto. It can either be in expanded form
+// or non-expanded form.
+func (d decoder) unmarshalAny(m pref.Message, checkDelims bool) error {
+	var typeURL string
+	var bValue []byte
+	var seenTypeUrl bool
+	var seenValue bool
+	var isExpanded bool
+
+	if checkDelims {
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+
+		if tok.Kind() != text.MessageOpen {
+			return d.unexpectedTokenError(tok)
+		}
+	}
+
+Loop:
+	for {
+		// Read field name. Can only have 3 possible field names, i.e. type_url,
+		// value and type URL name inside [].
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+		if typ := tok.Kind(); typ != text.Name {
+			if checkDelims {
+				if typ == text.MessageClose {
+					break Loop
+				}
+			} else if typ == text.EOF {
+				break Loop
+			}
+			return d.unexpectedTokenError(tok)
+		}
+
+		switch tok.NameKind() {
+		case text.IdentName:
+			// Both type_url and value fields require field separator :.
+			if !tok.HasSeparator() {
+				return d.syntaxError(tok.Pos(), "missing field separator :")
+			}
+
+			switch name := pref.Name(tok.IdentName()); name {
+			case genid.Any_TypeUrl_field_name:
+				if seenTypeUrl {
+					return d.newError(tok.Pos(), "duplicate %v field", genid.Any_TypeUrl_field_fullname)
+				}
+				if isExpanded {
+					return d.newError(tok.Pos(), "conflict with [%s] field", typeURL)
+				}
+				tok, err := d.Read()
+				if err != nil {
+					return err
+				}
+				var ok bool
+				typeURL, ok = tok.String()
+				if !ok {
+					return d.newError(tok.Pos(), "invalid %v field value: %v", genid.Any_TypeUrl_field_fullname, tok.RawString())
+				}
+				seenTypeUrl = true
+
+			case genid.Any_Value_field_name:
+				if seenValue {
+					return d.newError(tok.Pos(), "duplicate %v field", genid.Any_Value_field_fullname)
+				}
+				if isExpanded {
+					return d.newError(tok.Pos(), "conflict with [%s] field", typeURL)
+				}
+				tok, err := d.Read()
+				if err != nil {
+					return err
+				}
+				s, ok := tok.String()
+				if !ok {
+					return d.newError(tok.Pos(), "invalid %v field value: %v", genid.Any_Value_field_fullname, tok.RawString())
+				}
+				bValue = []byte(s)
+				seenValue = true
+
+			default:
+				if !d.opts.DiscardUnknown {
+					return d.newError(tok.Pos(), "invalid field name %q in %v message", tok.RawString(), genid.Any_message_fullname)
+				}
+			}
+
+		case text.TypeName:
+			if isExpanded {
+				return d.newError(tok.Pos(), "cannot have more than one type")
+			}
+			if seenTypeUrl {
+				return d.newError(tok.Pos(), "conflict with type_url field")
+			}
+			typeURL = tok.TypeName()
+			var err error
+			bValue, err = d.unmarshalExpandedAny(typeURL, tok.Pos())
+			if err != nil {
+				return err
+			}
+			isExpanded = true
+
+		default:
+			if !d.opts.DiscardUnknown {
+				return d.newError(tok.Pos(), "invalid field name %q in %v message", tok.RawString(), genid.Any_message_fullname)
+			}
+		}
+	}
+
+	fds := m.Descriptor().Fields()
+	if len(typeURL) > 0 {
+		m.Set(fds.ByNumber(genid.Any_TypeUrl_field_number), pref.ValueOfString(typeURL))
+	}
+	if len(bValue) > 0 {
+		m.Set(fds.ByNumber(genid.Any_Value_field_number), pref.ValueOfBytes(bValue))
+	}
+	return nil
+}
+
+func (d decoder) unmarshalExpandedAny(typeURL string, pos int) ([]byte, error) {
+	mt, err := d.opts.Resolver.FindMessageByURL(typeURL)
+	if err != nil {
+		return nil, d.newError(pos, "unable to resolve message [%v]: %v", typeURL, err)
+	}
+	// Create new message for the embedded message type and unmarshal the value
+	// field into it.
+	m := mt.New()
+	if err := d.unmarshalMessage(m, true); err != nil {
+		return nil, err
+	}
+	// Serialize the embedded message and return the resulting bytes.
+	b, err := proto.MarshalOptions{
+		AllowPartial:  true, // Never check required fields inside an Any.
+		Deterministic: true,
+	}.Marshal(m.Interface())
+	if err != nil {
+		return nil, d.newError(pos, "error in marshaling message into Any.value: %v", err)
+	}
+	return b, nil
+}
+
+// skipValue makes the decoder parse a field value in order to advance the read
+// to the next field. It relies on Read returning an error if the types are not
+// in valid sequence.
+func (d decoder) skipValue() error {
+	tok, err := d.Read()
+	if err != nil {
+		return err
+	}
+	// Only need to continue reading for messages and lists.
+	switch tok.Kind() {
+	case text.MessageOpen:
+		return d.skipMessageValue()
+
+	case text.ListOpen:
+		for {
+			tok, err := d.Read()
+			if err != nil {
+				return err
+			}
+			switch tok.Kind() {
+			case text.ListClose:
+				return nil
+			case text.MessageOpen:
+				return d.skipMessageValue()
+			default:
+				// Skip items. This will not validate whether skipped values are
+				// of the same type or not, same behavior as C++
+				// TextFormat::Parser::AllowUnknownField(true) version 3.8.0.
+			}
+		}
+	}
+	return nil
+}
+
+// skipMessageValue makes the decoder parse and skip over all fields in a
+// message. It assumes that the previous read type is MessageOpen.
+func (d decoder) skipMessageValue() error {
+	for {
+		tok, err := d.Read()
+		if err != nil {
+			return err
+		}
+		switch tok.Kind() {
+		case text.MessageClose:
+			return nil
+		case text.Name:
+			if err := d.skipValue(); err != nil {
+				return err
+			}
+		}
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/encoding/prototext/doc.go b/vendor/google.golang.org/protobuf/encoding/prototext/doc.go
new file mode 100644
index 0000000..162b4f9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/encoding/prototext/doc.go
@@ -0,0 +1,7 @@
+// 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.
+
+// Package prototext marshals and unmarshals protocol buffer messages as the
+// textproto format.
+package prototext
diff --git a/vendor/google.golang.org/protobuf/encoding/prototext/encode.go b/vendor/google.golang.org/protobuf/encoding/prototext/encode.go
new file mode 100644
index 0000000..8d5304d
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/encoding/prototext/encode.go
@@ -0,0 +1,371 @@
+// 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 prototext
+
+import (
+	"fmt"
+	"strconv"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/encoding/text"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/order"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+const defaultIndent = "  "
+
+// Format formats the message as a multiline string.
+// This function is only intended for human consumption and ignores errors.
+// Do not depend on the output being stable. It may change over time across
+// different versions of the program.
+func Format(m proto.Message) string {
+	return MarshalOptions{Multiline: true}.Format(m)
+}
+
+// Marshal writes the given proto.Message in textproto format using default
+// options. Do not depend on the output being stable. It may change over time
+// across different versions of the program.
+func Marshal(m proto.Message) ([]byte, error) {
+	return MarshalOptions{}.Marshal(m)
+}
+
+// MarshalOptions is a configurable text format marshaler.
+type MarshalOptions struct {
+	pragma.NoUnkeyedLiterals
+
+	// Multiline specifies whether the marshaler should format the output in
+	// indented-form with every textual element on a new line.
+	// If Indent is an empty string, then an arbitrary indent is chosen.
+	Multiline bool
+
+	// Indent specifies the set of indentation characters to use in a multiline
+	// formatted output such that every entry is preceded by Indent and
+	// terminated by a newline. If non-empty, then Multiline is treated as true.
+	// Indent can only be composed of space or tab characters.
+	Indent string
+
+	// EmitASCII specifies whether to format strings and bytes as ASCII only
+	// as opposed to using UTF-8 encoding when possible.
+	EmitASCII bool
+
+	// allowInvalidUTF8 specifies whether to permit the encoding of strings
+	// with invalid UTF-8. This is unexported as it is intended to only
+	// be specified by the Format method.
+	allowInvalidUTF8 bool
+
+	// AllowPartial allows messages that have missing required fields to marshal
+	// without returning an error. If AllowPartial is false (the default),
+	// Marshal will return error if there are any missing required fields.
+	AllowPartial bool
+
+	// EmitUnknown specifies whether to emit unknown fields in the output.
+	// If specified, the unmarshaler may be unable to parse the output.
+	// The default is to exclude unknown fields.
+	EmitUnknown bool
+
+	// Resolver is used for looking up types when expanding google.protobuf.Any
+	// messages. If nil, this defaults to using protoregistry.GlobalTypes.
+	Resolver interface {
+		protoregistry.ExtensionTypeResolver
+		protoregistry.MessageTypeResolver
+	}
+}
+
+// Format formats the message as a string.
+// This method is only intended for human consumption and ignores errors.
+// Do not depend on the output being stable. It may change over time across
+// different versions of the program.
+func (o MarshalOptions) Format(m proto.Message) string {
+	if m == nil || !m.ProtoReflect().IsValid() {
+		return "<nil>" // invalid syntax, but okay since this is for debugging
+	}
+	o.allowInvalidUTF8 = true
+	o.AllowPartial = true
+	o.EmitUnknown = true
+	b, _ := o.Marshal(m)
+	return string(b)
+}
+
+// Marshal writes the given proto.Message in textproto format using options in
+// MarshalOptions object. Do not depend on the output being stable. It may
+// change over time across different versions of the program.
+func (o MarshalOptions) Marshal(m proto.Message) ([]byte, error) {
+	return o.marshal(m)
+}
+
+// marshal is a centralized function that all marshal operations go through.
+// For profiling purposes, avoid changing the name of this function or
+// introducing other code paths for marshal that do not go through this.
+func (o MarshalOptions) marshal(m proto.Message) ([]byte, error) {
+	var delims = [2]byte{'{', '}'}
+
+	if o.Multiline && o.Indent == "" {
+		o.Indent = defaultIndent
+	}
+	if o.Resolver == nil {
+		o.Resolver = protoregistry.GlobalTypes
+	}
+
+	internalEnc, err := text.NewEncoder(o.Indent, delims, o.EmitASCII)
+	if err != nil {
+		return nil, err
+	}
+
+	// Treat nil message interface as an empty message,
+	// in which case there is nothing to output.
+	if m == nil {
+		return []byte{}, nil
+	}
+
+	enc := encoder{internalEnc, o}
+	err = enc.marshalMessage(m.ProtoReflect(), false)
+	if err != nil {
+		return nil, err
+	}
+	out := enc.Bytes()
+	if len(o.Indent) > 0 && len(out) > 0 {
+		out = append(out, '\n')
+	}
+	if o.AllowPartial {
+		return out, nil
+	}
+	return out, proto.CheckInitialized(m)
+}
+
+type encoder struct {
+	*text.Encoder
+	opts MarshalOptions
+}
+
+// marshalMessage marshals the given protoreflect.Message.
+func (e encoder) marshalMessage(m pref.Message, inclDelims bool) error {
+	messageDesc := m.Descriptor()
+	if !flags.ProtoLegacy && messageset.IsMessageSet(messageDesc) {
+		return errors.New("no support for proto1 MessageSets")
+	}
+
+	if inclDelims {
+		e.StartMessage()
+		defer e.EndMessage()
+	}
+
+	// Handle Any expansion.
+	if messageDesc.FullName() == genid.Any_message_fullname {
+		if e.marshalAny(m) {
+			return nil
+		}
+		// If unable to expand, continue on to marshal Any as a regular message.
+	}
+
+	// Marshal fields.
+	var err error
+	order.RangeFields(m, order.IndexNameFieldOrder, func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		if err = e.marshalField(fd.TextName(), v, fd); err != nil {
+			return false
+		}
+		return true
+	})
+	if err != nil {
+		return err
+	}
+
+	// Marshal unknown fields.
+	if e.opts.EmitUnknown {
+		e.marshalUnknown(m.GetUnknown())
+	}
+
+	return nil
+}
+
+// marshalField marshals the given field with protoreflect.Value.
+func (e encoder) marshalField(name string, val pref.Value, fd pref.FieldDescriptor) error {
+	switch {
+	case fd.IsList():
+		return e.marshalList(name, val.List(), fd)
+	case fd.IsMap():
+		return e.marshalMap(name, val.Map(), fd)
+	default:
+		e.WriteName(name)
+		return e.marshalSingular(val, fd)
+	}
+}
+
+// marshalSingular marshals the given non-repeated field value. This includes
+// all scalar types, enums, messages, and groups.
+func (e encoder) marshalSingular(val pref.Value, fd pref.FieldDescriptor) error {
+	kind := fd.Kind()
+	switch kind {
+	case pref.BoolKind:
+		e.WriteBool(val.Bool())
+
+	case pref.StringKind:
+		s := val.String()
+		if !e.opts.allowInvalidUTF8 && strs.EnforceUTF8(fd) && !utf8.ValidString(s) {
+			return errors.InvalidUTF8(string(fd.FullName()))
+		}
+		e.WriteString(s)
+
+	case pref.Int32Kind, pref.Int64Kind,
+		pref.Sint32Kind, pref.Sint64Kind,
+		pref.Sfixed32Kind, pref.Sfixed64Kind:
+		e.WriteInt(val.Int())
+
+	case pref.Uint32Kind, pref.Uint64Kind,
+		pref.Fixed32Kind, pref.Fixed64Kind:
+		e.WriteUint(val.Uint())
+
+	case pref.FloatKind:
+		// Encoder.WriteFloat handles the special numbers NaN and infinites.
+		e.WriteFloat(val.Float(), 32)
+
+	case pref.DoubleKind:
+		// Encoder.WriteFloat handles the special numbers NaN and infinites.
+		e.WriteFloat(val.Float(), 64)
+
+	case pref.BytesKind:
+		e.WriteString(string(val.Bytes()))
+
+	case pref.EnumKind:
+		num := val.Enum()
+		if desc := fd.Enum().Values().ByNumber(num); desc != nil {
+			e.WriteLiteral(string(desc.Name()))
+		} else {
+			// Use numeric value if there is no enum description.
+			e.WriteInt(int64(num))
+		}
+
+	case pref.MessageKind, pref.GroupKind:
+		return e.marshalMessage(val.Message(), true)
+
+	default:
+		panic(fmt.Sprintf("%v has unknown kind: %v", fd.FullName(), kind))
+	}
+	return nil
+}
+
+// marshalList marshals the given protoreflect.List as multiple name-value fields.
+func (e encoder) marshalList(name string, list pref.List, fd pref.FieldDescriptor) error {
+	size := list.Len()
+	for i := 0; i < size; i++ {
+		e.WriteName(name)
+		if err := e.marshalSingular(list.Get(i), fd); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// marshalMap marshals the given protoreflect.Map as multiple name-value fields.
+func (e encoder) marshalMap(name string, mmap pref.Map, fd pref.FieldDescriptor) error {
+	var err error
+	order.RangeEntries(mmap, order.GenericKeyOrder, func(key pref.MapKey, val pref.Value) bool {
+		e.WriteName(name)
+		e.StartMessage()
+		defer e.EndMessage()
+
+		e.WriteName(string(genid.MapEntry_Key_field_name))
+		err = e.marshalSingular(key.Value(), fd.MapKey())
+		if err != nil {
+			return false
+		}
+
+		e.WriteName(string(genid.MapEntry_Value_field_name))
+		err = e.marshalSingular(val, fd.MapValue())
+		if err != nil {
+			return false
+		}
+		return true
+	})
+	return err
+}
+
+// marshalUnknown parses the given []byte and marshals fields out.
+// This function assumes proper encoding in the given []byte.
+func (e encoder) marshalUnknown(b []byte) {
+	const dec = 10
+	const hex = 16
+	for len(b) > 0 {
+		num, wtype, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		e.WriteName(strconv.FormatInt(int64(num), dec))
+
+		switch wtype {
+		case protowire.VarintType:
+			var v uint64
+			v, n = protowire.ConsumeVarint(b)
+			e.WriteUint(v)
+		case protowire.Fixed32Type:
+			var v uint32
+			v, n = protowire.ConsumeFixed32(b)
+			e.WriteLiteral("0x" + strconv.FormatUint(uint64(v), hex))
+		case protowire.Fixed64Type:
+			var v uint64
+			v, n = protowire.ConsumeFixed64(b)
+			e.WriteLiteral("0x" + strconv.FormatUint(v, hex))
+		case protowire.BytesType:
+			var v []byte
+			v, n = protowire.ConsumeBytes(b)
+			e.WriteString(string(v))
+		case protowire.StartGroupType:
+			e.StartMessage()
+			var v []byte
+			v, n = protowire.ConsumeGroup(num, b)
+			e.marshalUnknown(v)
+			e.EndMessage()
+		default:
+			panic(fmt.Sprintf("prototext: error parsing unknown field wire type: %v", wtype))
+		}
+
+		b = b[n:]
+	}
+}
+
+// marshalAny marshals the given google.protobuf.Any message in expanded form.
+// It returns true if it was able to marshal, else false.
+func (e encoder) marshalAny(any pref.Message) bool {
+	// Construct the embedded message.
+	fds := any.Descriptor().Fields()
+	fdType := fds.ByNumber(genid.Any_TypeUrl_field_number)
+	typeURL := any.Get(fdType).String()
+	mt, err := e.opts.Resolver.FindMessageByURL(typeURL)
+	if err != nil {
+		return false
+	}
+	m := mt.New().Interface()
+
+	// Unmarshal bytes into embedded message.
+	fdValue := fds.ByNumber(genid.Any_Value_field_number)
+	value := any.Get(fdValue)
+	err = proto.UnmarshalOptions{
+		AllowPartial: true,
+		Resolver:     e.opts.Resolver,
+	}.Unmarshal(value.Bytes(), m)
+	if err != nil {
+		return false
+	}
+
+	// Get current encoder position. If marshaling fails, reset encoder output
+	// back to this position.
+	pos := e.Snapshot()
+
+	// Field name is the proto field name enclosed in [].
+	e.WriteName("[" + typeURL + "]")
+	err = e.marshalMessage(m.ProtoReflect(), true)
+	if err != nil {
+		e.Reset(pos)
+		return false
+	}
+	return true
+}
diff --git a/vendor/google.golang.org/protobuf/encoding/protowire/wire.go b/vendor/google.golang.org/protobuf/encoding/protowire/wire.go
new file mode 100644
index 0000000..a427f8b
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/encoding/protowire/wire.go
@@ -0,0 +1,538 @@
+// 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 protowire parses and formats the raw wire encoding.
+// See https://developers.google.com/protocol-buffers/docs/encoding.
+//
+// For marshaling and unmarshaling entire protobuf messages,
+// use the "google.golang.org/protobuf/proto" package instead.
+package protowire
+
+import (
+	"io"
+	"math"
+	"math/bits"
+
+	"google.golang.org/protobuf/internal/errors"
+)
+
+// Number represents the field number.
+type Number int32
+
+const (
+	MinValidNumber      Number = 1
+	FirstReservedNumber Number = 19000
+	LastReservedNumber  Number = 19999
+	MaxValidNumber      Number = 1<<29 - 1
+)
+
+// IsValid reports whether the field number is semantically valid.
+//
+// Note that while numbers within the reserved range are semantically invalid,
+// they are syntactically valid in the wire format.
+// Implementations may treat records with reserved field numbers as unknown.
+func (n Number) IsValid() bool {
+	return MinValidNumber <= n && n < FirstReservedNumber || LastReservedNumber < n && n <= MaxValidNumber
+}
+
+// Type represents the wire type.
+type Type int8
+
+const (
+	VarintType     Type = 0
+	Fixed32Type    Type = 5
+	Fixed64Type    Type = 1
+	BytesType      Type = 2
+	StartGroupType Type = 3
+	EndGroupType   Type = 4
+)
+
+const (
+	_ = -iota
+	errCodeTruncated
+	errCodeFieldNumber
+	errCodeOverflow
+	errCodeReserved
+	errCodeEndGroup
+)
+
+var (
+	errFieldNumber = errors.New("invalid field number")
+	errOverflow    = errors.New("variable length integer overflow")
+	errReserved    = errors.New("cannot parse reserved wire type")
+	errEndGroup    = errors.New("mismatching end group marker")
+	errParse       = errors.New("parse error")
+)
+
+// ParseError converts an error code into an error value.
+// This returns nil if n is a non-negative number.
+func ParseError(n int) error {
+	if n >= 0 {
+		return nil
+	}
+	switch n {
+	case errCodeTruncated:
+		return io.ErrUnexpectedEOF
+	case errCodeFieldNumber:
+		return errFieldNumber
+	case errCodeOverflow:
+		return errOverflow
+	case errCodeReserved:
+		return errReserved
+	case errCodeEndGroup:
+		return errEndGroup
+	default:
+		return errParse
+	}
+}
+
+// ConsumeField parses an entire field record (both tag and value) and returns
+// the field number, the wire type, and the total length.
+// This returns a negative length upon an error (see ParseError).
+//
+// The total length includes the tag header and the end group marker (if the
+// field is a group).
+func ConsumeField(b []byte) (Number, Type, int) {
+	num, typ, n := ConsumeTag(b)
+	if n < 0 {
+		return 0, 0, n // forward error code
+	}
+	m := ConsumeFieldValue(num, typ, b[n:])
+	if m < 0 {
+		return 0, 0, m // forward error code
+	}
+	return num, typ, n + m
+}
+
+// ConsumeFieldValue parses a field value and returns its length.
+// This assumes that the field Number and wire Type have already been parsed.
+// This returns a negative length upon an error (see ParseError).
+//
+// When parsing a group, the length includes the end group marker and
+// the end group is verified to match the starting field number.
+func ConsumeFieldValue(num Number, typ Type, b []byte) (n int) {
+	switch typ {
+	case VarintType:
+		_, n = ConsumeVarint(b)
+		return n
+	case Fixed32Type:
+		_, n = ConsumeFixed32(b)
+		return n
+	case Fixed64Type:
+		_, n = ConsumeFixed64(b)
+		return n
+	case BytesType:
+		_, n = ConsumeBytes(b)
+		return n
+	case StartGroupType:
+		n0 := len(b)
+		for {
+			num2, typ2, n := ConsumeTag(b)
+			if n < 0 {
+				return n // forward error code
+			}
+			b = b[n:]
+			if typ2 == EndGroupType {
+				if num != num2 {
+					return errCodeEndGroup
+				}
+				return n0 - len(b)
+			}
+
+			n = ConsumeFieldValue(num2, typ2, b)
+			if n < 0 {
+				return n // forward error code
+			}
+			b = b[n:]
+		}
+	case EndGroupType:
+		return errCodeEndGroup
+	default:
+		return errCodeReserved
+	}
+}
+
+// AppendTag encodes num and typ as a varint-encoded tag and appends it to b.
+func AppendTag(b []byte, num Number, typ Type) []byte {
+	return AppendVarint(b, EncodeTag(num, typ))
+}
+
+// ConsumeTag parses b as a varint-encoded tag, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeTag(b []byte) (Number, Type, int) {
+	v, n := ConsumeVarint(b)
+	if n < 0 {
+		return 0, 0, n // forward error code
+	}
+	num, typ := DecodeTag(v)
+	if num < MinValidNumber {
+		return 0, 0, errCodeFieldNumber
+	}
+	return num, typ, n
+}
+
+func SizeTag(num Number) int {
+	return SizeVarint(EncodeTag(num, 0)) // wire type has no effect on size
+}
+
+// AppendVarint appends v to b as a varint-encoded uint64.
+func AppendVarint(b []byte, v uint64) []byte {
+	switch {
+	case v < 1<<7:
+		b = append(b, byte(v))
+	case v < 1<<14:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte(v>>7))
+	case v < 1<<21:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte(v>>14))
+	case v < 1<<28:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte(v>>21))
+	case v < 1<<35:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte(v>>28))
+	case v < 1<<42:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte((v>>28)&0x7f|0x80),
+			byte(v>>35))
+	case v < 1<<49:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte((v>>28)&0x7f|0x80),
+			byte((v>>35)&0x7f|0x80),
+			byte(v>>42))
+	case v < 1<<56:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte((v>>28)&0x7f|0x80),
+			byte((v>>35)&0x7f|0x80),
+			byte((v>>42)&0x7f|0x80),
+			byte(v>>49))
+	case v < 1<<63:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte((v>>28)&0x7f|0x80),
+			byte((v>>35)&0x7f|0x80),
+			byte((v>>42)&0x7f|0x80),
+			byte((v>>49)&0x7f|0x80),
+			byte(v>>56))
+	default:
+		b = append(b,
+			byte((v>>0)&0x7f|0x80),
+			byte((v>>7)&0x7f|0x80),
+			byte((v>>14)&0x7f|0x80),
+			byte((v>>21)&0x7f|0x80),
+			byte((v>>28)&0x7f|0x80),
+			byte((v>>35)&0x7f|0x80),
+			byte((v>>42)&0x7f|0x80),
+			byte((v>>49)&0x7f|0x80),
+			byte((v>>56)&0x7f|0x80),
+			1)
+	}
+	return b
+}
+
+// ConsumeVarint parses b as a varint-encoded uint64, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeVarint(b []byte) (v uint64, n int) {
+	var y uint64
+	if len(b) <= 0 {
+		return 0, errCodeTruncated
+	}
+	v = uint64(b[0])
+	if v < 0x80 {
+		return v, 1
+	}
+	v -= 0x80
+
+	if len(b) <= 1 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[1])
+	v += y << 7
+	if y < 0x80 {
+		return v, 2
+	}
+	v -= 0x80 << 7
+
+	if len(b) <= 2 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[2])
+	v += y << 14
+	if y < 0x80 {
+		return v, 3
+	}
+	v -= 0x80 << 14
+
+	if len(b) <= 3 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[3])
+	v += y << 21
+	if y < 0x80 {
+		return v, 4
+	}
+	v -= 0x80 << 21
+
+	if len(b) <= 4 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[4])
+	v += y << 28
+	if y < 0x80 {
+		return v, 5
+	}
+	v -= 0x80 << 28
+
+	if len(b) <= 5 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[5])
+	v += y << 35
+	if y < 0x80 {
+		return v, 6
+	}
+	v -= 0x80 << 35
+
+	if len(b) <= 6 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[6])
+	v += y << 42
+	if y < 0x80 {
+		return v, 7
+	}
+	v -= 0x80 << 42
+
+	if len(b) <= 7 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[7])
+	v += y << 49
+	if y < 0x80 {
+		return v, 8
+	}
+	v -= 0x80 << 49
+
+	if len(b) <= 8 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[8])
+	v += y << 56
+	if y < 0x80 {
+		return v, 9
+	}
+	v -= 0x80 << 56
+
+	if len(b) <= 9 {
+		return 0, errCodeTruncated
+	}
+	y = uint64(b[9])
+	v += y << 63
+	if y < 2 {
+		return v, 10
+	}
+	return 0, errCodeOverflow
+}
+
+// SizeVarint returns the encoded size of a varint.
+// The size is guaranteed to be within 1 and 10, inclusive.
+func SizeVarint(v uint64) int {
+	// This computes 1 + (bits.Len64(v)-1)/7.
+	// 9/64 is a good enough approximation of 1/7
+	return int(9*uint32(bits.Len64(v))+64) / 64
+}
+
+// AppendFixed32 appends v to b as a little-endian uint32.
+func AppendFixed32(b []byte, v uint32) []byte {
+	return append(b,
+		byte(v>>0),
+		byte(v>>8),
+		byte(v>>16),
+		byte(v>>24))
+}
+
+// ConsumeFixed32 parses b as a little-endian uint32, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeFixed32(b []byte) (v uint32, n int) {
+	if len(b) < 4 {
+		return 0, errCodeTruncated
+	}
+	v = uint32(b[0])<<0 | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24
+	return v, 4
+}
+
+// SizeFixed32 returns the encoded size of a fixed32; which is always 4.
+func SizeFixed32() int {
+	return 4
+}
+
+// AppendFixed64 appends v to b as a little-endian uint64.
+func AppendFixed64(b []byte, v uint64) []byte {
+	return append(b,
+		byte(v>>0),
+		byte(v>>8),
+		byte(v>>16),
+		byte(v>>24),
+		byte(v>>32),
+		byte(v>>40),
+		byte(v>>48),
+		byte(v>>56))
+}
+
+// ConsumeFixed64 parses b as a little-endian uint64, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeFixed64(b []byte) (v uint64, n int) {
+	if len(b) < 8 {
+		return 0, errCodeTruncated
+	}
+	v = uint64(b[0])<<0 | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56
+	return v, 8
+}
+
+// SizeFixed64 returns the encoded size of a fixed64; which is always 8.
+func SizeFixed64() int {
+	return 8
+}
+
+// AppendBytes appends v to b as a length-prefixed bytes value.
+func AppendBytes(b []byte, v []byte) []byte {
+	return append(AppendVarint(b, uint64(len(v))), v...)
+}
+
+// ConsumeBytes parses b as a length-prefixed bytes value, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeBytes(b []byte) (v []byte, n int) {
+	m, n := ConsumeVarint(b)
+	if n < 0 {
+		return nil, n // forward error code
+	}
+	if m > uint64(len(b[n:])) {
+		return nil, errCodeTruncated
+	}
+	return b[n:][:m], n + int(m)
+}
+
+// SizeBytes returns the encoded size of a length-prefixed bytes value,
+// given only the length.
+func SizeBytes(n int) int {
+	return SizeVarint(uint64(n)) + n
+}
+
+// AppendString appends v to b as a length-prefixed bytes value.
+func AppendString(b []byte, v string) []byte {
+	return append(AppendVarint(b, uint64(len(v))), v...)
+}
+
+// ConsumeString parses b as a length-prefixed bytes value, reporting its length.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeString(b []byte) (v string, n int) {
+	bb, n := ConsumeBytes(b)
+	return string(bb), n
+}
+
+// AppendGroup appends v to b as group value, with a trailing end group marker.
+// The value v must not contain the end marker.
+func AppendGroup(b []byte, num Number, v []byte) []byte {
+	return AppendVarint(append(b, v...), EncodeTag(num, EndGroupType))
+}
+
+// ConsumeGroup parses b as a group value until the trailing end group marker,
+// and verifies that the end marker matches the provided num. The value v
+// does not contain the end marker, while the length does contain the end marker.
+// This returns a negative length upon an error (see ParseError).
+func ConsumeGroup(num Number, b []byte) (v []byte, n int) {
+	n = ConsumeFieldValue(num, StartGroupType, b)
+	if n < 0 {
+		return nil, n // forward error code
+	}
+	b = b[:n]
+
+	// Truncate off end group marker, but need to handle denormalized varints.
+	// Assuming end marker is never 0 (which is always the case since
+	// EndGroupType is non-zero), we can truncate all trailing bytes where the
+	// lower 7 bits are all zero (implying that the varint is denormalized).
+	for len(b) > 0 && b[len(b)-1]&0x7f == 0 {
+		b = b[:len(b)-1]
+	}
+	b = b[:len(b)-SizeTag(num)]
+	return b, n
+}
+
+// SizeGroup returns the encoded size of a group, given only the length.
+func SizeGroup(num Number, n int) int {
+	return n + SizeTag(num)
+}
+
+// DecodeTag decodes the field Number and wire Type from its unified form.
+// The Number is -1 if the decoded field number overflows int32.
+// Other than overflow, this does not check for field number validity.
+func DecodeTag(x uint64) (Number, Type) {
+	// NOTE: MessageSet allows for larger field numbers than normal.
+	if x>>3 > uint64(math.MaxInt32) {
+		return -1, 0
+	}
+	return Number(x >> 3), Type(x & 7)
+}
+
+// EncodeTag encodes the field Number and wire Type into its unified form.
+func EncodeTag(num Number, typ Type) uint64 {
+	return uint64(num)<<3 | uint64(typ&7)
+}
+
+// DecodeZigZag decodes a zig-zag-encoded uint64 as an int64.
+//	Input:  {…,  5,  3,  1,  0,  2,  4,  6, …}
+//	Output: {…, -3, -2, -1,  0, +1, +2, +3, …}
+func DecodeZigZag(x uint64) int64 {
+	return int64(x>>1) ^ int64(x)<<63>>63
+}
+
+// EncodeZigZag encodes an int64 as a zig-zag-encoded uint64.
+//	Input:  {…, -3, -2, -1,  0, +1, +2, +3, …}
+//	Output: {…,  5,  3,  1,  0,  2,  4,  6, …}
+func EncodeZigZag(x int64) uint64 {
+	return uint64(x<<1) ^ uint64(x>>63)
+}
+
+// DecodeBool decodes a uint64 as a bool.
+//	Input:  {    0,    1,    2, …}
+//	Output: {false, true, true, …}
+func DecodeBool(x uint64) bool {
+	return x != 0
+}
+
+// EncodeBool encodes a bool as a uint64.
+//	Input:  {false, true}
+//	Output: {    0,    1}
+func EncodeBool(x bool) uint64 {
+	if x {
+		return 1
+	}
+	return 0
+}
diff --git a/vendor/google.golang.org/protobuf/internal/descfmt/stringer.go b/vendor/google.golang.org/protobuf/internal/descfmt/stringer.go
new file mode 100644
index 0000000..360c633
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/descfmt/stringer.go
@@ -0,0 +1,318 @@
+// 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 descfmt provides functionality to format descriptors.
+package descfmt
+
+import (
+	"fmt"
+	"io"
+	"reflect"
+	"strconv"
+	"strings"
+
+	"google.golang.org/protobuf/internal/detrand"
+	"google.golang.org/protobuf/internal/pragma"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type list interface {
+	Len() int
+	pragma.DoNotImplement
+}
+
+func FormatList(s fmt.State, r rune, vs list) {
+	io.WriteString(s, formatListOpt(vs, true, r == 'v' && (s.Flag('+') || s.Flag('#'))))
+}
+func formatListOpt(vs list, isRoot, allowMulti bool) string {
+	start, end := "[", "]"
+	if isRoot {
+		var name string
+		switch vs.(type) {
+		case pref.Names:
+			name = "Names"
+		case pref.FieldNumbers:
+			name = "FieldNumbers"
+		case pref.FieldRanges:
+			name = "FieldRanges"
+		case pref.EnumRanges:
+			name = "EnumRanges"
+		case pref.FileImports:
+			name = "FileImports"
+		case pref.Descriptor:
+			name = reflect.ValueOf(vs).MethodByName("Get").Type().Out(0).Name() + "s"
+		default:
+			name = reflect.ValueOf(vs).Elem().Type().Name()
+		}
+		start, end = name+"{", "}"
+	}
+
+	var ss []string
+	switch vs := vs.(type) {
+	case pref.Names:
+		for i := 0; i < vs.Len(); i++ {
+			ss = append(ss, fmt.Sprint(vs.Get(i)))
+		}
+		return start + joinStrings(ss, false) + end
+	case pref.FieldNumbers:
+		for i := 0; i < vs.Len(); i++ {
+			ss = append(ss, fmt.Sprint(vs.Get(i)))
+		}
+		return start + joinStrings(ss, false) + end
+	case pref.FieldRanges:
+		for i := 0; i < vs.Len(); i++ {
+			r := vs.Get(i)
+			if r[0]+1 == r[1] {
+				ss = append(ss, fmt.Sprintf("%d", r[0]))
+			} else {
+				ss = append(ss, fmt.Sprintf("%d:%d", r[0], r[1])) // enum ranges are end exclusive
+			}
+		}
+		return start + joinStrings(ss, false) + end
+	case pref.EnumRanges:
+		for i := 0; i < vs.Len(); i++ {
+			r := vs.Get(i)
+			if r[0] == r[1] {
+				ss = append(ss, fmt.Sprintf("%d", r[0]))
+			} else {
+				ss = append(ss, fmt.Sprintf("%d:%d", r[0], int64(r[1])+1)) // enum ranges are end inclusive
+			}
+		}
+		return start + joinStrings(ss, false) + end
+	case pref.FileImports:
+		for i := 0; i < vs.Len(); i++ {
+			var rs records
+			rs.Append(reflect.ValueOf(vs.Get(i)), "Path", "Package", "IsPublic", "IsWeak")
+			ss = append(ss, "{"+rs.Join()+"}")
+		}
+		return start + joinStrings(ss, allowMulti) + end
+	default:
+		_, isEnumValue := vs.(pref.EnumValueDescriptors)
+		for i := 0; i < vs.Len(); i++ {
+			m := reflect.ValueOf(vs).MethodByName("Get")
+			v := m.Call([]reflect.Value{reflect.ValueOf(i)})[0].Interface()
+			ss = append(ss, formatDescOpt(v.(pref.Descriptor), false, allowMulti && !isEnumValue))
+		}
+		return start + joinStrings(ss, allowMulti && isEnumValue) + end
+	}
+}
+
+// descriptorAccessors is a list of accessors to print for each descriptor.
+//
+// Do not print all accessors since some contain redundant information,
+// while others are pointers that we do not want to follow since the descriptor
+// is actually a cyclic graph.
+//
+// Using a list allows us to print the accessors in a sensible order.
+var descriptorAccessors = map[reflect.Type][]string{
+	reflect.TypeOf((*pref.FileDescriptor)(nil)).Elem():      {"Path", "Package", "Imports", "Messages", "Enums", "Extensions", "Services"},
+	reflect.TypeOf((*pref.MessageDescriptor)(nil)).Elem():   {"IsMapEntry", "Fields", "Oneofs", "ReservedNames", "ReservedRanges", "RequiredNumbers", "ExtensionRanges", "Messages", "Enums", "Extensions"},
+	reflect.TypeOf((*pref.FieldDescriptor)(nil)).Elem():     {"Number", "Cardinality", "Kind", "HasJSONName", "JSONName", "HasPresence", "IsExtension", "IsPacked", "IsWeak", "IsList", "IsMap", "MapKey", "MapValue", "HasDefault", "Default", "ContainingOneof", "ContainingMessage", "Message", "Enum"},
+	reflect.TypeOf((*pref.OneofDescriptor)(nil)).Elem():     {"Fields"}, // not directly used; must keep in sync with formatDescOpt
+	reflect.TypeOf((*pref.EnumDescriptor)(nil)).Elem():      {"Values", "ReservedNames", "ReservedRanges"},
+	reflect.TypeOf((*pref.EnumValueDescriptor)(nil)).Elem(): {"Number"},
+	reflect.TypeOf((*pref.ServiceDescriptor)(nil)).Elem():   {"Methods"},
+	reflect.TypeOf((*pref.MethodDescriptor)(nil)).Elem():    {"Input", "Output", "IsStreamingClient", "IsStreamingServer"},
+}
+
+func FormatDesc(s fmt.State, r rune, t pref.Descriptor) {
+	io.WriteString(s, formatDescOpt(t, true, r == 'v' && (s.Flag('+') || s.Flag('#'))))
+}
+func formatDescOpt(t pref.Descriptor, isRoot, allowMulti bool) string {
+	rv := reflect.ValueOf(t)
+	rt := rv.MethodByName("ProtoType").Type().In(0)
+
+	start, end := "{", "}"
+	if isRoot {
+		start = rt.Name() + "{"
+	}
+
+	_, isFile := t.(pref.FileDescriptor)
+	rs := records{allowMulti: allowMulti}
+	if t.IsPlaceholder() {
+		if isFile {
+			rs.Append(rv, "Path", "Package", "IsPlaceholder")
+		} else {
+			rs.Append(rv, "FullName", "IsPlaceholder")
+		}
+	} else {
+		switch {
+		case isFile:
+			rs.Append(rv, "Syntax")
+		case isRoot:
+			rs.Append(rv, "Syntax", "FullName")
+		default:
+			rs.Append(rv, "Name")
+		}
+		switch t := t.(type) {
+		case pref.FieldDescriptor:
+			for _, s := range descriptorAccessors[rt] {
+				switch s {
+				case "MapKey":
+					if k := t.MapKey(); k != nil {
+						rs.recs = append(rs.recs, [2]string{"MapKey", k.Kind().String()})
+					}
+				case "MapValue":
+					if v := t.MapValue(); v != nil {
+						switch v.Kind() {
+						case pref.EnumKind:
+							rs.recs = append(rs.recs, [2]string{"MapValue", string(v.Enum().FullName())})
+						case pref.MessageKind, pref.GroupKind:
+							rs.recs = append(rs.recs, [2]string{"MapValue", string(v.Message().FullName())})
+						default:
+							rs.recs = append(rs.recs, [2]string{"MapValue", v.Kind().String()})
+						}
+					}
+				case "ContainingOneof":
+					if od := t.ContainingOneof(); od != nil {
+						rs.recs = append(rs.recs, [2]string{"Oneof", string(od.Name())})
+					}
+				case "ContainingMessage":
+					if t.IsExtension() {
+						rs.recs = append(rs.recs, [2]string{"Extendee", string(t.ContainingMessage().FullName())})
+					}
+				case "Message":
+					if !t.IsMap() {
+						rs.Append(rv, s)
+					}
+				default:
+					rs.Append(rv, s)
+				}
+			}
+		case pref.OneofDescriptor:
+			var ss []string
+			fs := t.Fields()
+			for i := 0; i < fs.Len(); i++ {
+				ss = append(ss, string(fs.Get(i).Name()))
+			}
+			if len(ss) > 0 {
+				rs.recs = append(rs.recs, [2]string{"Fields", "[" + joinStrings(ss, false) + "]"})
+			}
+		default:
+			rs.Append(rv, descriptorAccessors[rt]...)
+		}
+		if rv.MethodByName("GoType").IsValid() {
+			rs.Append(rv, "GoType")
+		}
+	}
+	return start + rs.Join() + end
+}
+
+type records struct {
+	recs       [][2]string
+	allowMulti bool
+}
+
+func (rs *records) Append(v reflect.Value, accessors ...string) {
+	for _, a := range accessors {
+		var rv reflect.Value
+		if m := v.MethodByName(a); m.IsValid() {
+			rv = m.Call(nil)[0]
+		}
+		if v.Kind() == reflect.Struct && !rv.IsValid() {
+			rv = v.FieldByName(a)
+		}
+		if !rv.IsValid() {
+			panic(fmt.Sprintf("unknown accessor: %v.%s", v.Type(), a))
+		}
+		if _, ok := rv.Interface().(pref.Value); ok {
+			rv = rv.MethodByName("Interface").Call(nil)[0]
+			if !rv.IsNil() {
+				rv = rv.Elem()
+			}
+		}
+
+		// Ignore zero values.
+		var isZero bool
+		switch rv.Kind() {
+		case reflect.Interface, reflect.Slice:
+			isZero = rv.IsNil()
+		case reflect.Bool:
+			isZero = rv.Bool() == false
+		case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
+			isZero = rv.Int() == 0
+		case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64:
+			isZero = rv.Uint() == 0
+		case reflect.String:
+			isZero = rv.String() == ""
+		}
+		if n, ok := rv.Interface().(list); ok {
+			isZero = n.Len() == 0
+		}
+		if isZero {
+			continue
+		}
+
+		// Format the value.
+		var s string
+		v := rv.Interface()
+		switch v := v.(type) {
+		case list:
+			s = formatListOpt(v, false, rs.allowMulti)
+		case pref.FieldDescriptor, pref.OneofDescriptor, pref.EnumValueDescriptor, pref.MethodDescriptor:
+			s = string(v.(pref.Descriptor).Name())
+		case pref.Descriptor:
+			s = string(v.FullName())
+		case string:
+			s = strconv.Quote(v)
+		case []byte:
+			s = fmt.Sprintf("%q", v)
+		default:
+			s = fmt.Sprint(v)
+		}
+		rs.recs = append(rs.recs, [2]string{a, s})
+	}
+}
+
+func (rs *records) Join() string {
+	var ss []string
+
+	// In single line mode, simply join all records with commas.
+	if !rs.allowMulti {
+		for _, r := range rs.recs {
+			ss = append(ss, r[0]+formatColon(0)+r[1])
+		}
+		return joinStrings(ss, false)
+	}
+
+	// In allowMulti line mode, align single line records for more readable output.
+	var maxLen int
+	flush := func(i int) {
+		for _, r := range rs.recs[len(ss):i] {
+			ss = append(ss, r[0]+formatColon(maxLen-len(r[0]))+r[1])
+		}
+		maxLen = 0
+	}
+	for i, r := range rs.recs {
+		if isMulti := strings.Contains(r[1], "\n"); isMulti {
+			flush(i)
+			ss = append(ss, r[0]+formatColon(0)+strings.Join(strings.Split(r[1], "\n"), "\n\t"))
+		} else if maxLen < len(r[0]) {
+			maxLen = len(r[0])
+		}
+	}
+	flush(len(rs.recs))
+	return joinStrings(ss, true)
+}
+
+func formatColon(padding int) string {
+	// Deliberately introduce instability into the debug output to
+	// discourage users from performing string comparisons.
+	// This provides us flexibility to change the output in the future.
+	if detrand.Bool() {
+		return ":" + strings.Repeat(" ", 1+padding) // use non-breaking spaces (U+00a0)
+	} else {
+		return ":" + strings.Repeat(" ", 1+padding) // use regular spaces (U+0020)
+	}
+}
+
+func joinStrings(ss []string, isMulti bool) string {
+	if len(ss) == 0 {
+		return ""
+	}
+	if isMulti {
+		return "\n\t" + strings.Join(ss, "\n\t") + "\n"
+	}
+	return strings.Join(ss, ", ")
+}
diff --git a/vendor/google.golang.org/protobuf/internal/descopts/options.go b/vendor/google.golang.org/protobuf/internal/descopts/options.go
new file mode 100644
index 0000000..8401be8
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/descopts/options.go
@@ -0,0 +1,29 @@
+// 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.
+
+// Package descopts contains the nil pointers to concrete descriptor options.
+//
+// This package exists as a form of reverse dependency injection so that certain
+// packages (e.g., internal/filedesc and internal/filetype can avoid a direct
+// dependency on the descriptor proto package).
+package descopts
+
+import pref "google.golang.org/protobuf/reflect/protoreflect"
+
+// These variables are set by the init function in descriptor.pb.go via logic
+// in internal/filetype. In other words, so long as the descriptor proto package
+// is linked in, these variables will be populated.
+//
+// Each variable is populated with a nil pointer to the options struct.
+var (
+	File           pref.ProtoMessage
+	Enum           pref.ProtoMessage
+	EnumValue      pref.ProtoMessage
+	Message        pref.ProtoMessage
+	Field          pref.ProtoMessage
+	Oneof          pref.ProtoMessage
+	ExtensionRange pref.ProtoMessage
+	Service        pref.ProtoMessage
+	Method         pref.ProtoMessage
+)
diff --git a/vendor/google.golang.org/protobuf/internal/detrand/rand.go b/vendor/google.golang.org/protobuf/internal/detrand/rand.go
new file mode 100644
index 0000000..49c8676
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/detrand/rand.go
@@ -0,0 +1,69 @@
+// 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 detrand provides deterministically random functionality.
+//
+// The pseudo-randomness of these functions is seeded by the program binary
+// itself and guarantees that the output does not change within a program,
+// while ensuring that the output is unstable across different builds.
+package detrand
+
+import (
+	"encoding/binary"
+	"hash/fnv"
+	"os"
+)
+
+// Disable disables detrand such that all functions returns the zero value.
+// This function is not concurrent-safe and must be called during program init.
+func Disable() {
+	randSeed = 0
+}
+
+// Bool returns a deterministically random boolean.
+func Bool() bool {
+	return randSeed%2 == 1
+}
+
+// Intn returns a deterministically random integer between 0 and n-1, inclusive.
+func Intn(n int) int {
+	if n <= 0 {
+		panic("must be positive")
+	}
+	return int(randSeed % uint64(n))
+}
+
+// randSeed is a best-effort at an approximate hash of the Go binary.
+var randSeed = binaryHash()
+
+func binaryHash() uint64 {
+	// Open the Go binary.
+	s, err := os.Executable()
+	if err != nil {
+		return 0
+	}
+	f, err := os.Open(s)
+	if err != nil {
+		return 0
+	}
+	defer f.Close()
+
+	// Hash the size and several samples of the Go binary.
+	const numSamples = 8
+	var buf [64]byte
+	h := fnv.New64()
+	fi, err := f.Stat()
+	if err != nil {
+		return 0
+	}
+	binary.LittleEndian.PutUint64(buf[:8], uint64(fi.Size()))
+	h.Write(buf[:8])
+	for i := int64(0); i < numSamples; i++ {
+		if _, err := f.ReadAt(buf[:], i*fi.Size()/numSamples); err != nil {
+			return 0
+		}
+		h.Write(buf[:])
+	}
+	return h.Sum64()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/defval/default.go b/vendor/google.golang.org/protobuf/internal/encoding/defval/default.go
new file mode 100644
index 0000000..fdd9b13
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/defval/default.go
@@ -0,0 +1,213 @@
+// 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 defval marshals and unmarshals textual forms of default values.
+//
+// This package handles both the form historically used in Go struct field tags
+// and also the form used by google.protobuf.FieldDescriptorProto.default_value
+// since they differ in superficial ways.
+package defval
+
+import (
+	"fmt"
+	"math"
+	"strconv"
+
+	ptext "google.golang.org/protobuf/internal/encoding/text"
+	errors "google.golang.org/protobuf/internal/errors"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// Format is the serialization format used to represent the default value.
+type Format int
+
+const (
+	_ Format = iota
+
+	// Descriptor uses the serialization format that protoc uses with the
+	// google.protobuf.FieldDescriptorProto.default_value field.
+	Descriptor
+
+	// GoTag uses the historical serialization format in Go struct field tags.
+	GoTag
+)
+
+// Unmarshal deserializes the default string s according to the given kind k.
+// When k is an enum, a list of enum value descriptors must be provided.
+func Unmarshal(s string, k pref.Kind, evs pref.EnumValueDescriptors, f Format) (pref.Value, pref.EnumValueDescriptor, error) {
+	switch k {
+	case pref.BoolKind:
+		if f == GoTag {
+			switch s {
+			case "1":
+				return pref.ValueOfBool(true), nil, nil
+			case "0":
+				return pref.ValueOfBool(false), nil, nil
+			}
+		} else {
+			switch s {
+			case "true":
+				return pref.ValueOfBool(true), nil, nil
+			case "false":
+				return pref.ValueOfBool(false), nil, nil
+			}
+		}
+	case pref.EnumKind:
+		if f == GoTag {
+			// Go tags use the numeric form of the enum value.
+			if n, err := strconv.ParseInt(s, 10, 32); err == nil {
+				if ev := evs.ByNumber(pref.EnumNumber(n)); ev != nil {
+					return pref.ValueOfEnum(ev.Number()), ev, nil
+				}
+			}
+		} else {
+			// Descriptor default_value use the enum identifier.
+			ev := evs.ByName(pref.Name(s))
+			if ev != nil {
+				return pref.ValueOfEnum(ev.Number()), ev, nil
+			}
+		}
+	case pref.Int32Kind, pref.Sint32Kind, pref.Sfixed32Kind:
+		if v, err := strconv.ParseInt(s, 10, 32); err == nil {
+			return pref.ValueOfInt32(int32(v)), nil, nil
+		}
+	case pref.Int64Kind, pref.Sint64Kind, pref.Sfixed64Kind:
+		if v, err := strconv.ParseInt(s, 10, 64); err == nil {
+			return pref.ValueOfInt64(int64(v)), nil, nil
+		}
+	case pref.Uint32Kind, pref.Fixed32Kind:
+		if v, err := strconv.ParseUint(s, 10, 32); err == nil {
+			return pref.ValueOfUint32(uint32(v)), nil, nil
+		}
+	case pref.Uint64Kind, pref.Fixed64Kind:
+		if v, err := strconv.ParseUint(s, 10, 64); err == nil {
+			return pref.ValueOfUint64(uint64(v)), nil, nil
+		}
+	case pref.FloatKind, pref.DoubleKind:
+		var v float64
+		var err error
+		switch s {
+		case "-inf":
+			v = math.Inf(-1)
+		case "inf":
+			v = math.Inf(+1)
+		case "nan":
+			v = math.NaN()
+		default:
+			v, err = strconv.ParseFloat(s, 64)
+		}
+		if err == nil {
+			if k == pref.FloatKind {
+				return pref.ValueOfFloat32(float32(v)), nil, nil
+			} else {
+				return pref.ValueOfFloat64(float64(v)), nil, nil
+			}
+		}
+	case pref.StringKind:
+		// String values are already unescaped and can be used as is.
+		return pref.ValueOfString(s), nil, nil
+	case pref.BytesKind:
+		if b, ok := unmarshalBytes(s); ok {
+			return pref.ValueOfBytes(b), nil, nil
+		}
+	}
+	return pref.Value{}, nil, errors.New("could not parse value for %v: %q", k, s)
+}
+
+// Marshal serializes v as the default string according to the given kind k.
+// When specifying the Descriptor format for an enum kind, the associated
+// enum value descriptor must be provided.
+func Marshal(v pref.Value, ev pref.EnumValueDescriptor, k pref.Kind, f Format) (string, error) {
+	switch k {
+	case pref.BoolKind:
+		if f == GoTag {
+			if v.Bool() {
+				return "1", nil
+			} else {
+				return "0", nil
+			}
+		} else {
+			if v.Bool() {
+				return "true", nil
+			} else {
+				return "false", nil
+			}
+		}
+	case pref.EnumKind:
+		if f == GoTag {
+			return strconv.FormatInt(int64(v.Enum()), 10), nil
+		} else {
+			return string(ev.Name()), nil
+		}
+	case pref.Int32Kind, pref.Sint32Kind, pref.Sfixed32Kind, pref.Int64Kind, pref.Sint64Kind, pref.Sfixed64Kind:
+		return strconv.FormatInt(v.Int(), 10), nil
+	case pref.Uint32Kind, pref.Fixed32Kind, pref.Uint64Kind, pref.Fixed64Kind:
+		return strconv.FormatUint(v.Uint(), 10), nil
+	case pref.FloatKind, pref.DoubleKind:
+		f := v.Float()
+		switch {
+		case math.IsInf(f, -1):
+			return "-inf", nil
+		case math.IsInf(f, +1):
+			return "inf", nil
+		case math.IsNaN(f):
+			return "nan", nil
+		default:
+			if k == pref.FloatKind {
+				return strconv.FormatFloat(f, 'g', -1, 32), nil
+			} else {
+				return strconv.FormatFloat(f, 'g', -1, 64), nil
+			}
+		}
+	case pref.StringKind:
+		// String values are serialized as is without any escaping.
+		return v.String(), nil
+	case pref.BytesKind:
+		if s, ok := marshalBytes(v.Bytes()); ok {
+			return s, nil
+		}
+	}
+	return "", errors.New("could not format value for %v: %v", k, v)
+}
+
+// unmarshalBytes deserializes bytes by applying C unescaping.
+func unmarshalBytes(s string) ([]byte, bool) {
+	// Bytes values use the same escaping as the text format,
+	// however they lack the surrounding double quotes.
+	v, err := ptext.UnmarshalString(`"` + s + `"`)
+	if err != nil {
+		return nil, false
+	}
+	return []byte(v), true
+}
+
+// marshalBytes serializes bytes by using C escaping.
+// To match the exact output of protoc, this is identical to the
+// CEscape function in strutil.cc of the protoc source code.
+func marshalBytes(b []byte) (string, bool) {
+	var s []byte
+	for _, c := range b {
+		switch c {
+		case '\n':
+			s = append(s, `\n`...)
+		case '\r':
+			s = append(s, `\r`...)
+		case '\t':
+			s = append(s, `\t`...)
+		case '"':
+			s = append(s, `\"`...)
+		case '\'':
+			s = append(s, `\'`...)
+		case '\\':
+			s = append(s, `\\`...)
+		default:
+			if printableASCII := c >= 0x20 && c <= 0x7e; printableASCII {
+				s = append(s, c)
+			} else {
+				s = append(s, fmt.Sprintf(`\%03o`, c)...)
+			}
+		}
+	}
+	return string(s), true
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/messageset/messageset.go b/vendor/google.golang.org/protobuf/internal/encoding/messageset/messageset.go
new file mode 100644
index 0000000..c1866f3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/messageset/messageset.go
@@ -0,0 +1,241 @@
+// 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.
+
+// Package messageset encodes and decodes the obsolete MessageSet wire format.
+package messageset
+
+import (
+	"math"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// The MessageSet wire format is equivalent to a message defined as follows,
+// where each Item defines an extension field with a field number of 'type_id'
+// and content of 'message'. MessageSet extensions must be non-repeated message
+// fields.
+//
+//	message MessageSet {
+//		repeated group Item = 1 {
+//			required int32 type_id = 2;
+//			required string message = 3;
+//		}
+//	}
+const (
+	FieldItem    = protowire.Number(1)
+	FieldTypeID  = protowire.Number(2)
+	FieldMessage = protowire.Number(3)
+)
+
+// ExtensionName is the field name for extensions of MessageSet.
+//
+// A valid MessageSet extension must be of the form:
+//	message MyMessage {
+//		extend proto2.bridge.MessageSet {
+//			optional MyMessage message_set_extension = 1234;
+//		}
+//		...
+//	}
+const ExtensionName = "message_set_extension"
+
+// IsMessageSet returns whether the message uses the MessageSet wire format.
+func IsMessageSet(md pref.MessageDescriptor) bool {
+	xmd, ok := md.(interface{ IsMessageSet() bool })
+	return ok && xmd.IsMessageSet()
+}
+
+// IsMessageSetExtension reports this field properly extends a MessageSet.
+func IsMessageSetExtension(fd pref.FieldDescriptor) bool {
+	switch {
+	case fd.Name() != ExtensionName:
+		return false
+	case !IsMessageSet(fd.ContainingMessage()):
+		return false
+	case fd.FullName().Parent() != fd.Message().FullName():
+		return false
+	}
+	return true
+}
+
+// SizeField returns the size of a MessageSet item field containing an extension
+// with the given field number, not counting the contents of the message subfield.
+func SizeField(num protowire.Number) int {
+	return 2*protowire.SizeTag(FieldItem) + protowire.SizeTag(FieldTypeID) + protowire.SizeVarint(uint64(num))
+}
+
+// Unmarshal parses a MessageSet.
+//
+// It calls fn with the type ID and value of each item in the MessageSet.
+// Unknown fields are discarded.
+//
+// If wantLen is true, the item values include the varint length prefix.
+// This is ugly, but simplifies the fast-path decoder in internal/impl.
+func Unmarshal(b []byte, wantLen bool, fn func(typeID protowire.Number, value []byte) error) error {
+	for len(b) > 0 {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return protowire.ParseError(n)
+		}
+		b = b[n:]
+		if num != FieldItem || wtyp != protowire.StartGroupType {
+			n := protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return protowire.ParseError(n)
+			}
+			b = b[n:]
+			continue
+		}
+		typeID, value, n, err := ConsumeFieldValue(b, wantLen)
+		if err != nil {
+			return err
+		}
+		b = b[n:]
+		if typeID == 0 {
+			continue
+		}
+		if err := fn(typeID, value); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// ConsumeFieldValue parses b as a MessageSet item field value until and including
+// the trailing end group marker. It assumes the start group tag has already been parsed.
+// It returns the contents of the type_id and message subfields and the total
+// item length.
+//
+// If wantLen is true, the returned message value includes the length prefix.
+func ConsumeFieldValue(b []byte, wantLen bool) (typeid protowire.Number, message []byte, n int, err error) {
+	ilen := len(b)
+	for {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return 0, nil, 0, protowire.ParseError(n)
+		}
+		b = b[n:]
+		switch {
+		case num == FieldItem && wtyp == protowire.EndGroupType:
+			if wantLen && len(message) == 0 {
+				// The message field was missing, which should never happen.
+				// Be prepared for this case anyway.
+				message = protowire.AppendVarint(message, 0)
+			}
+			return typeid, message, ilen - len(b), nil
+		case num == FieldTypeID && wtyp == protowire.VarintType:
+			v, n := protowire.ConsumeVarint(b)
+			if n < 0 {
+				return 0, nil, 0, protowire.ParseError(n)
+			}
+			b = b[n:]
+			if v < 1 || v > math.MaxInt32 {
+				return 0, nil, 0, errors.New("invalid type_id in message set")
+			}
+			typeid = protowire.Number(v)
+		case num == FieldMessage && wtyp == protowire.BytesType:
+			m, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, nil, 0, protowire.ParseError(n)
+			}
+			if message == nil {
+				if wantLen {
+					message = b[:n:n]
+				} else {
+					message = m[:len(m):len(m)]
+				}
+			} else {
+				// This case should never happen in practice, but handle it for
+				// correctness: The MessageSet item contains multiple message
+				// fields, which need to be merged.
+				//
+				// In the case where we're returning the length, this becomes
+				// quite inefficient since we need to strip the length off
+				// the existing data and reconstruct it with the combined length.
+				if wantLen {
+					_, nn := protowire.ConsumeVarint(message)
+					m0 := message[nn:]
+					message = nil
+					message = protowire.AppendVarint(message, uint64(len(m0)+len(m)))
+					message = append(message, m0...)
+					message = append(message, m...)
+				} else {
+					message = append(message, m...)
+				}
+			}
+			b = b[n:]
+		default:
+			// We have no place to put it, so we just ignore unknown fields.
+			n := protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return 0, nil, 0, protowire.ParseError(n)
+			}
+			b = b[n:]
+		}
+	}
+}
+
+// AppendFieldStart appends the start of a MessageSet item field containing
+// an extension with the given number. The caller must add the message
+// subfield (including the tag).
+func AppendFieldStart(b []byte, num protowire.Number) []byte {
+	b = protowire.AppendTag(b, FieldItem, protowire.StartGroupType)
+	b = protowire.AppendTag(b, FieldTypeID, protowire.VarintType)
+	b = protowire.AppendVarint(b, uint64(num))
+	return b
+}
+
+// AppendFieldEnd appends the trailing end group marker for a MessageSet item field.
+func AppendFieldEnd(b []byte) []byte {
+	return protowire.AppendTag(b, FieldItem, protowire.EndGroupType)
+}
+
+// SizeUnknown returns the size of an unknown fields section in MessageSet format.
+//
+// See AppendUnknown.
+func SizeUnknown(unknown []byte) (size int) {
+	for len(unknown) > 0 {
+		num, typ, n := protowire.ConsumeTag(unknown)
+		if n < 0 || typ != protowire.BytesType {
+			return 0
+		}
+		unknown = unknown[n:]
+		_, n = protowire.ConsumeBytes(unknown)
+		if n < 0 {
+			return 0
+		}
+		unknown = unknown[n:]
+		size += SizeField(num) + protowire.SizeTag(FieldMessage) + n
+	}
+	return size
+}
+
+// AppendUnknown appends unknown fields to b in MessageSet format.
+//
+// For historic reasons, unresolved items in a MessageSet are stored in a
+// message's unknown fields section in non-MessageSet format. That is, an
+// unknown item with typeID T and value V appears in the unknown fields as
+// a field with number T and value V.
+//
+// This function converts the unknown fields back into MessageSet form.
+func AppendUnknown(b, unknown []byte) ([]byte, error) {
+	for len(unknown) > 0 {
+		num, typ, n := protowire.ConsumeTag(unknown)
+		if n < 0 || typ != protowire.BytesType {
+			return nil, errors.New("invalid data in message set unknown fields")
+		}
+		unknown = unknown[n:]
+		_, n = protowire.ConsumeBytes(unknown)
+		if n < 0 {
+			return nil, errors.New("invalid data in message set unknown fields")
+		}
+		b = AppendFieldStart(b, num)
+		b = protowire.AppendTag(b, FieldMessage, protowire.BytesType)
+		b = append(b, unknown[:n]...)
+		b = AppendFieldEnd(b)
+		unknown = unknown[n:]
+	}
+	return b, nil
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/tag/tag.go b/vendor/google.golang.org/protobuf/internal/encoding/tag/tag.go
new file mode 100644
index 0000000..38f1931
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/tag/tag.go
@@ -0,0 +1,207 @@
+// 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 tag marshals and unmarshals the legacy struct tags as generated
+// by historical versions of protoc-gen-go.
+package tag
+
+import (
+	"reflect"
+	"strconv"
+	"strings"
+
+	defval "google.golang.org/protobuf/internal/encoding/defval"
+	fdesc "google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/strs"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+var byteType = reflect.TypeOf(byte(0))
+
+// Unmarshal decodes the tag into a prototype.Field.
+//
+// The goType is needed to determine the original protoreflect.Kind since the
+// tag does not record sufficient information to determine that.
+// The type is the underlying field type (e.g., a repeated field may be
+// represented by []T, but the Go type passed in is just T).
+// A list of enum value descriptors must be provided for enum fields.
+// This does not populate the Enum or Message (except for weak message).
+//
+// This function is a best effort attempt; parsing errors are ignored.
+func Unmarshal(tag string, goType reflect.Type, evs pref.EnumValueDescriptors) pref.FieldDescriptor {
+	f := new(fdesc.Field)
+	f.L0.ParentFile = fdesc.SurrogateProto2
+	for len(tag) > 0 {
+		i := strings.IndexByte(tag, ',')
+		if i < 0 {
+			i = len(tag)
+		}
+		switch s := tag[:i]; {
+		case strings.HasPrefix(s, "name="):
+			f.L0.FullName = pref.FullName(s[len("name="):])
+		case strings.Trim(s, "0123456789") == "":
+			n, _ := strconv.ParseUint(s, 10, 32)
+			f.L1.Number = pref.FieldNumber(n)
+		case s == "opt":
+			f.L1.Cardinality = pref.Optional
+		case s == "req":
+			f.L1.Cardinality = pref.Required
+		case s == "rep":
+			f.L1.Cardinality = pref.Repeated
+		case s == "varint":
+			switch goType.Kind() {
+			case reflect.Bool:
+				f.L1.Kind = pref.BoolKind
+			case reflect.Int32:
+				f.L1.Kind = pref.Int32Kind
+			case reflect.Int64:
+				f.L1.Kind = pref.Int64Kind
+			case reflect.Uint32:
+				f.L1.Kind = pref.Uint32Kind
+			case reflect.Uint64:
+				f.L1.Kind = pref.Uint64Kind
+			}
+		case s == "zigzag32":
+			if goType.Kind() == reflect.Int32 {
+				f.L1.Kind = pref.Sint32Kind
+			}
+		case s == "zigzag64":
+			if goType.Kind() == reflect.Int64 {
+				f.L1.Kind = pref.Sint64Kind
+			}
+		case s == "fixed32":
+			switch goType.Kind() {
+			case reflect.Int32:
+				f.L1.Kind = pref.Sfixed32Kind
+			case reflect.Uint32:
+				f.L1.Kind = pref.Fixed32Kind
+			case reflect.Float32:
+				f.L1.Kind = pref.FloatKind
+			}
+		case s == "fixed64":
+			switch goType.Kind() {
+			case reflect.Int64:
+				f.L1.Kind = pref.Sfixed64Kind
+			case reflect.Uint64:
+				f.L1.Kind = pref.Fixed64Kind
+			case reflect.Float64:
+				f.L1.Kind = pref.DoubleKind
+			}
+		case s == "bytes":
+			switch {
+			case goType.Kind() == reflect.String:
+				f.L1.Kind = pref.StringKind
+			case goType.Kind() == reflect.Slice && goType.Elem() == byteType:
+				f.L1.Kind = pref.BytesKind
+			default:
+				f.L1.Kind = pref.MessageKind
+			}
+		case s == "group":
+			f.L1.Kind = pref.GroupKind
+		case strings.HasPrefix(s, "enum="):
+			f.L1.Kind = pref.EnumKind
+		case strings.HasPrefix(s, "json="):
+			jsonName := s[len("json="):]
+			if jsonName != strs.JSONCamelCase(string(f.L0.FullName.Name())) {
+				f.L1.StringName.InitJSON(jsonName)
+			}
+		case s == "packed":
+			f.L1.HasPacked = true
+			f.L1.IsPacked = true
+		case strings.HasPrefix(s, "weak="):
+			f.L1.IsWeak = true
+			f.L1.Message = fdesc.PlaceholderMessage(pref.FullName(s[len("weak="):]))
+		case strings.HasPrefix(s, "def="):
+			// The default tag is special in that everything afterwards is the
+			// default regardless of the presence of commas.
+			s, i = tag[len("def="):], len(tag)
+			v, ev, _ := defval.Unmarshal(s, f.L1.Kind, evs, defval.GoTag)
+			f.L1.Default = fdesc.DefaultValue(v, ev)
+		case s == "proto3":
+			f.L0.ParentFile = fdesc.SurrogateProto3
+		}
+		tag = strings.TrimPrefix(tag[i:], ",")
+	}
+
+	// The generator uses the group message name instead of the field name.
+	// We obtain the real field name by lowercasing the group name.
+	if f.L1.Kind == pref.GroupKind {
+		f.L0.FullName = pref.FullName(strings.ToLower(string(f.L0.FullName)))
+	}
+	return f
+}
+
+// Marshal encodes the protoreflect.FieldDescriptor as a tag.
+//
+// The enumName must be provided if the kind is an enum.
+// Historically, the formulation of the enum "name" was the proto package
+// dot-concatenated with the generated Go identifier for the enum type.
+// Depending on the context on how Marshal is called, there are different ways
+// through which that information is determined. As such it is the caller's
+// responsibility to provide a function to obtain that information.
+func Marshal(fd pref.FieldDescriptor, enumName string) string {
+	var tag []string
+	switch fd.Kind() {
+	case pref.BoolKind, pref.EnumKind, pref.Int32Kind, pref.Uint32Kind, pref.Int64Kind, pref.Uint64Kind:
+		tag = append(tag, "varint")
+	case pref.Sint32Kind:
+		tag = append(tag, "zigzag32")
+	case pref.Sint64Kind:
+		tag = append(tag, "zigzag64")
+	case pref.Sfixed32Kind, pref.Fixed32Kind, pref.FloatKind:
+		tag = append(tag, "fixed32")
+	case pref.Sfixed64Kind, pref.Fixed64Kind, pref.DoubleKind:
+		tag = append(tag, "fixed64")
+	case pref.StringKind, pref.BytesKind, pref.MessageKind:
+		tag = append(tag, "bytes")
+	case pref.GroupKind:
+		tag = append(tag, "group")
+	}
+	tag = append(tag, strconv.Itoa(int(fd.Number())))
+	switch fd.Cardinality() {
+	case pref.Optional:
+		tag = append(tag, "opt")
+	case pref.Required:
+		tag = append(tag, "req")
+	case pref.Repeated:
+		tag = append(tag, "rep")
+	}
+	if fd.IsPacked() {
+		tag = append(tag, "packed")
+	}
+	name := string(fd.Name())
+	if fd.Kind() == pref.GroupKind {
+		// The name of the FieldDescriptor for a group field is
+		// lowercased. To find the original capitalization, we
+		// look in the field's MessageType.
+		name = string(fd.Message().Name())
+	}
+	tag = append(tag, "name="+name)
+	if jsonName := fd.JSONName(); jsonName != "" && jsonName != name && !fd.IsExtension() {
+		// NOTE: The jsonName != name condition is suspect, but it preserve
+		// the exact same semantics from the previous generator.
+		tag = append(tag, "json="+jsonName)
+	}
+	if fd.IsWeak() {
+		tag = append(tag, "weak="+string(fd.Message().FullName()))
+	}
+	// The previous implementation does not tag extension fields as proto3,
+	// even when the field is defined in a proto3 file. Match that behavior
+	// for consistency.
+	if fd.Syntax() == pref.Proto3 && !fd.IsExtension() {
+		tag = append(tag, "proto3")
+	}
+	if fd.Kind() == pref.EnumKind && enumName != "" {
+		tag = append(tag, "enum="+enumName)
+	}
+	if fd.ContainingOneof() != nil {
+		tag = append(tag, "oneof")
+	}
+	// This must appear last in the tag, since commas in strings aren't escaped.
+	if fd.HasDefault() {
+		def, _ := defval.Marshal(fd.Default(), fd.DefaultEnumValue(), fd.Kind(), defval.GoTag)
+		tag = append(tag, "def="+def)
+	}
+	return strings.Join(tag, ",")
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/decode.go b/vendor/google.golang.org/protobuf/internal/encoding/text/decode.go
new file mode 100644
index 0000000..eb10ea1
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/decode.go
@@ -0,0 +1,665 @@
+// 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 text
+
+import (
+	"bytes"
+	"fmt"
+	"io"
+	"regexp"
+	"strconv"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/internal/errors"
+)
+
+// Decoder is a token-based textproto decoder.
+type Decoder struct {
+	// lastCall is last method called, either readCall or peekCall.
+	// Initial value is readCall.
+	lastCall call
+
+	// lastToken contains the last read token.
+	lastToken Token
+
+	// lastErr contains the last read error.
+	lastErr error
+
+	// openStack is a stack containing the byte characters for MessageOpen and
+	// ListOpen kinds. The top of stack represents the message or the list that
+	// the current token is nested in. An empty stack means the current token is
+	// at the top level message. The characters '{' and '<' both represent the
+	// MessageOpen kind.
+	openStack []byte
+
+	// orig is used in reporting line and column.
+	orig []byte
+	// in contains the unconsumed input.
+	in []byte
+}
+
+// NewDecoder returns a Decoder to read the given []byte.
+func NewDecoder(b []byte) *Decoder {
+	return &Decoder{orig: b, in: b}
+}
+
+// ErrUnexpectedEOF means that EOF was encountered in the middle of the input.
+var ErrUnexpectedEOF = errors.New("%v", io.ErrUnexpectedEOF)
+
+// call specifies which Decoder method was invoked.
+type call uint8
+
+const (
+	readCall call = iota
+	peekCall
+)
+
+// Peek looks ahead and returns the next token and error without advancing a read.
+func (d *Decoder) Peek() (Token, error) {
+	defer func() { d.lastCall = peekCall }()
+	if d.lastCall == readCall {
+		d.lastToken, d.lastErr = d.Read()
+	}
+	return d.lastToken, d.lastErr
+}
+
+// Read returns the next token.
+// It will return an error if there is no valid token.
+func (d *Decoder) Read() (Token, error) {
+	defer func() { d.lastCall = readCall }()
+	if d.lastCall == peekCall {
+		return d.lastToken, d.lastErr
+	}
+
+	tok, err := d.parseNext(d.lastToken.kind)
+	if err != nil {
+		return Token{}, err
+	}
+
+	switch tok.kind {
+	case comma, semicolon:
+		tok, err = d.parseNext(tok.kind)
+		if err != nil {
+			return Token{}, err
+		}
+	}
+	d.lastToken = tok
+	return tok, nil
+}
+
+const (
+	mismatchedFmt = "mismatched close character %q"
+	unexpectedFmt = "unexpected character %q"
+)
+
+// parseNext parses the next Token based on given last kind.
+func (d *Decoder) parseNext(lastKind Kind) (Token, error) {
+	// Trim leading spaces.
+	d.consume(0)
+	isEOF := false
+	if len(d.in) == 0 {
+		isEOF = true
+	}
+
+	switch lastKind {
+	case EOF:
+		return d.consumeToken(EOF, 0, 0), nil
+
+	case bof:
+		// Start of top level message. Next token can be EOF or Name.
+		if isEOF {
+			return d.consumeToken(EOF, 0, 0), nil
+		}
+		return d.parseFieldName()
+
+	case Name:
+		// Next token can be MessageOpen, ListOpen or Scalar.
+		if isEOF {
+			return Token{}, ErrUnexpectedEOF
+		}
+		switch ch := d.in[0]; ch {
+		case '{', '<':
+			d.pushOpenStack(ch)
+			return d.consumeToken(MessageOpen, 1, 0), nil
+		case '[':
+			d.pushOpenStack(ch)
+			return d.consumeToken(ListOpen, 1, 0), nil
+		default:
+			return d.parseScalar()
+		}
+
+	case Scalar:
+		openKind, closeCh := d.currentOpenKind()
+		switch openKind {
+		case bof:
+			// Top level message.
+			// 	Next token can be EOF, comma, semicolon or Name.
+			if isEOF {
+				return d.consumeToken(EOF, 0, 0), nil
+			}
+			switch d.in[0] {
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		case MessageOpen:
+			// Next token can be MessageClose, comma, semicolon or Name.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case closeCh:
+				d.popOpenStack()
+				return d.consumeToken(MessageClose, 1, 0), nil
+			case otherCloseChar[closeCh]:
+				return Token{}, d.newSyntaxError(mismatchedFmt, ch)
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		case ListOpen:
+			// Next token can be ListClose or comma.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case ']':
+				d.popOpenStack()
+				return d.consumeToken(ListClose, 1, 0), nil
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			default:
+				return Token{}, d.newSyntaxError(unexpectedFmt, ch)
+			}
+		}
+
+	case MessageOpen:
+		// Next token can be MessageClose or Name.
+		if isEOF {
+			return Token{}, ErrUnexpectedEOF
+		}
+		_, closeCh := d.currentOpenKind()
+		switch ch := d.in[0]; ch {
+		case closeCh:
+			d.popOpenStack()
+			return d.consumeToken(MessageClose, 1, 0), nil
+		case otherCloseChar[closeCh]:
+			return Token{}, d.newSyntaxError(mismatchedFmt, ch)
+		default:
+			return d.parseFieldName()
+		}
+
+	case MessageClose:
+		openKind, closeCh := d.currentOpenKind()
+		switch openKind {
+		case bof:
+			// Top level message.
+			// Next token can be EOF, comma, semicolon or Name.
+			if isEOF {
+				return d.consumeToken(EOF, 0, 0), nil
+			}
+			switch ch := d.in[0]; ch {
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		case MessageOpen:
+			// Next token can be MessageClose, comma, semicolon or Name.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case closeCh:
+				d.popOpenStack()
+				return d.consumeToken(MessageClose, 1, 0), nil
+			case otherCloseChar[closeCh]:
+				return Token{}, d.newSyntaxError(mismatchedFmt, ch)
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		case ListOpen:
+			// Next token can be ListClose or comma
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case closeCh:
+				d.popOpenStack()
+				return d.consumeToken(ListClose, 1, 0), nil
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			default:
+				return Token{}, d.newSyntaxError(unexpectedFmt, ch)
+			}
+		}
+
+	case ListOpen:
+		// Next token can be ListClose, MessageStart or Scalar.
+		if isEOF {
+			return Token{}, ErrUnexpectedEOF
+		}
+		switch ch := d.in[0]; ch {
+		case ']':
+			d.popOpenStack()
+			return d.consumeToken(ListClose, 1, 0), nil
+		case '{', '<':
+			d.pushOpenStack(ch)
+			return d.consumeToken(MessageOpen, 1, 0), nil
+		default:
+			return d.parseScalar()
+		}
+
+	case ListClose:
+		openKind, closeCh := d.currentOpenKind()
+		switch openKind {
+		case bof:
+			// Top level message.
+			// Next token can be EOF, comma, semicolon or Name.
+			if isEOF {
+				return d.consumeToken(EOF, 0, 0), nil
+			}
+			switch ch := d.in[0]; ch {
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		case MessageOpen:
+			// Next token can be MessageClose, comma, semicolon or Name.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case closeCh:
+				d.popOpenStack()
+				return d.consumeToken(MessageClose, 1, 0), nil
+			case otherCloseChar[closeCh]:
+				return Token{}, d.newSyntaxError(mismatchedFmt, ch)
+			case ',':
+				return d.consumeToken(comma, 1, 0), nil
+			case ';':
+				return d.consumeToken(semicolon, 1, 0), nil
+			default:
+				return d.parseFieldName()
+			}
+
+		default:
+			// It is not possible to have this case. Let it panic below.
+		}
+
+	case comma, semicolon:
+		openKind, closeCh := d.currentOpenKind()
+		switch openKind {
+		case bof:
+			// Top level message. Next token can be EOF or Name.
+			if isEOF {
+				return d.consumeToken(EOF, 0, 0), nil
+			}
+			return d.parseFieldName()
+
+		case MessageOpen:
+			// Next token can be MessageClose or Name.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case closeCh:
+				d.popOpenStack()
+				return d.consumeToken(MessageClose, 1, 0), nil
+			case otherCloseChar[closeCh]:
+				return Token{}, d.newSyntaxError(mismatchedFmt, ch)
+			default:
+				return d.parseFieldName()
+			}
+
+		case ListOpen:
+			if lastKind == semicolon {
+				// It is not be possible to have this case as logic here
+				// should not have produced a semicolon Token when inside a
+				// list. Let it panic below.
+				break
+			}
+			// Next token can be MessageOpen or Scalar.
+			if isEOF {
+				return Token{}, ErrUnexpectedEOF
+			}
+			switch ch := d.in[0]; ch {
+			case '{', '<':
+				d.pushOpenStack(ch)
+				return d.consumeToken(MessageOpen, 1, 0), nil
+			default:
+				return d.parseScalar()
+			}
+		}
+	}
+
+	line, column := d.Position(len(d.orig) - len(d.in))
+	panic(fmt.Sprintf("Decoder.parseNext: bug at handling line %d:%d with lastKind=%v", line, column, lastKind))
+}
+
+var otherCloseChar = map[byte]byte{
+	'}': '>',
+	'>': '}',
+}
+
+// currentOpenKind indicates whether current position is inside a message, list
+// or top-level message by returning MessageOpen, ListOpen or bof respectively.
+// If the returned kind is either a MessageOpen or ListOpen, it also returns the
+// corresponding closing character.
+func (d *Decoder) currentOpenKind() (Kind, byte) {
+	if len(d.openStack) == 0 {
+		return bof, 0
+	}
+	openCh := d.openStack[len(d.openStack)-1]
+	switch openCh {
+	case '{':
+		return MessageOpen, '}'
+	case '<':
+		return MessageOpen, '>'
+	case '[':
+		return ListOpen, ']'
+	}
+	panic(fmt.Sprintf("Decoder: openStack contains invalid byte %s", string(openCh)))
+}
+
+func (d *Decoder) pushOpenStack(ch byte) {
+	d.openStack = append(d.openStack, ch)
+}
+
+func (d *Decoder) popOpenStack() {
+	d.openStack = d.openStack[:len(d.openStack)-1]
+}
+
+// parseFieldName parses field name and separator.
+func (d *Decoder) parseFieldName() (tok Token, err error) {
+	defer func() {
+		if err == nil && d.tryConsumeChar(':') {
+			tok.attrs |= hasSeparator
+		}
+	}()
+
+	// Extension or Any type URL.
+	if d.in[0] == '[' {
+		return d.parseTypeName()
+	}
+
+	// Identifier.
+	if size := parseIdent(d.in, false); size > 0 {
+		return d.consumeToken(Name, size, uint8(IdentName)), nil
+	}
+
+	// Field number. Identify if input is a valid number that is not negative
+	// and is decimal integer within 32-bit range.
+	if num := parseNumber(d.in); num.size > 0 {
+		if !num.neg && num.kind == numDec {
+			if _, err := strconv.ParseInt(string(d.in[:num.size]), 10, 32); err == nil {
+				return d.consumeToken(Name, num.size, uint8(FieldNumber)), nil
+			}
+		}
+		return Token{}, d.newSyntaxError("invalid field number: %s", d.in[:num.size])
+	}
+
+	return Token{}, d.newSyntaxError("invalid field name: %s", errRegexp.Find(d.in))
+}
+
+// parseTypeName parses Any type URL or extension field name. The name is
+// enclosed in [ and ] characters. The C++ parser does not handle many legal URL
+// strings. This implementation is more liberal and allows for the pattern
+// ^[-_a-zA-Z0-9]+([./][-_a-zA-Z0-9]+)*`). Whitespaces and comments are allowed
+// in between [ ], '.', '/' and the sub names.
+func (d *Decoder) parseTypeName() (Token, error) {
+	startPos := len(d.orig) - len(d.in)
+	// Use alias s to advance first in order to use d.in for error handling.
+	// Caller already checks for [ as first character.
+	s := consume(d.in[1:], 0)
+	if len(s) == 0 {
+		return Token{}, ErrUnexpectedEOF
+	}
+
+	var name []byte
+	for len(s) > 0 && isTypeNameChar(s[0]) {
+		name = append(name, s[0])
+		s = s[1:]
+	}
+	s = consume(s, 0)
+
+	var closed bool
+	for len(s) > 0 && !closed {
+		switch {
+		case s[0] == ']':
+			s = s[1:]
+			closed = true
+
+		case s[0] == '/', s[0] == '.':
+			if len(name) > 0 && (name[len(name)-1] == '/' || name[len(name)-1] == '.') {
+				return Token{}, d.newSyntaxError("invalid type URL/extension field name: %s",
+					d.orig[startPos:len(d.orig)-len(s)+1])
+			}
+			name = append(name, s[0])
+			s = s[1:]
+			s = consume(s, 0)
+			for len(s) > 0 && isTypeNameChar(s[0]) {
+				name = append(name, s[0])
+				s = s[1:]
+			}
+			s = consume(s, 0)
+
+		default:
+			return Token{}, d.newSyntaxError(
+				"invalid type URL/extension field name: %s", d.orig[startPos:len(d.orig)-len(s)+1])
+		}
+	}
+
+	if !closed {
+		return Token{}, ErrUnexpectedEOF
+	}
+
+	// First character cannot be '.'. Last character cannot be '.' or '/'.
+	size := len(name)
+	if size == 0 || name[0] == '.' || name[size-1] == '.' || name[size-1] == '/' {
+		return Token{}, d.newSyntaxError("invalid type URL/extension field name: %s",
+			d.orig[startPos:len(d.orig)-len(s)])
+	}
+
+	d.in = s
+	endPos := len(d.orig) - len(d.in)
+	d.consume(0)
+
+	return Token{
+		kind:  Name,
+		attrs: uint8(TypeName),
+		pos:   startPos,
+		raw:   d.orig[startPos:endPos],
+		str:   string(name),
+	}, nil
+}
+
+func isTypeNameChar(b byte) bool {
+	return (b == '-' || b == '_' ||
+		('0' <= b && b <= '9') ||
+		('a' <= b && b <= 'z') ||
+		('A' <= b && b <= 'Z'))
+}
+
+func isWhiteSpace(b byte) bool {
+	switch b {
+	case ' ', '\n', '\r', '\t':
+		return true
+	default:
+		return false
+	}
+}
+
+// parseIdent parses an unquoted proto identifier and returns size.
+// If allowNeg is true, it allows '-' to be the first character in the
+// identifier. This is used when parsing literal values like -infinity, etc.
+// Regular expression matches an identifier: `^[_a-zA-Z][_a-zA-Z0-9]*`
+func parseIdent(input []byte, allowNeg bool) int {
+	var size int
+
+	s := input
+	if len(s) == 0 {
+		return 0
+	}
+
+	if allowNeg && s[0] == '-' {
+		s = s[1:]
+		size++
+		if len(s) == 0 {
+			return 0
+		}
+	}
+
+	switch {
+	case s[0] == '_',
+		'a' <= s[0] && s[0] <= 'z',
+		'A' <= s[0] && s[0] <= 'Z':
+		s = s[1:]
+		size++
+	default:
+		return 0
+	}
+
+	for len(s) > 0 && (s[0] == '_' ||
+		'a' <= s[0] && s[0] <= 'z' ||
+		'A' <= s[0] && s[0] <= 'Z' ||
+		'0' <= s[0] && s[0] <= '9') {
+		s = s[1:]
+		size++
+	}
+
+	if len(s) > 0 && !isDelim(s[0]) {
+		return 0
+	}
+
+	return size
+}
+
+// parseScalar parses for a string, literal or number value.
+func (d *Decoder) parseScalar() (Token, error) {
+	if d.in[0] == '"' || d.in[0] == '\'' {
+		return d.parseStringValue()
+	}
+
+	if tok, ok := d.parseLiteralValue(); ok {
+		return tok, nil
+	}
+
+	if tok, ok := d.parseNumberValue(); ok {
+		return tok, nil
+	}
+
+	return Token{}, d.newSyntaxError("invalid scalar value: %s", errRegexp.Find(d.in))
+}
+
+// parseLiteralValue parses a literal value. A literal value is used for
+// bools, special floats and enums. This function simply identifies that the
+// field value is a literal.
+func (d *Decoder) parseLiteralValue() (Token, bool) {
+	size := parseIdent(d.in, true)
+	if size == 0 {
+		return Token{}, false
+	}
+	return d.consumeToken(Scalar, size, literalValue), true
+}
+
+// consumeToken constructs a Token for given Kind from d.in and consumes given
+// size-length from it.
+func (d *Decoder) consumeToken(kind Kind, size int, attrs uint8) Token {
+	// Important to compute raw and pos before consuming.
+	tok := Token{
+		kind:  kind,
+		attrs: attrs,
+		pos:   len(d.orig) - len(d.in),
+		raw:   d.in[:size],
+	}
+	d.consume(size)
+	return tok
+}
+
+// newSyntaxError returns a syntax error with line and column information for
+// current position.
+func (d *Decoder) newSyntaxError(f string, x ...interface{}) error {
+	e := errors.New(f, x...)
+	line, column := d.Position(len(d.orig) - len(d.in))
+	return errors.New("syntax error (line %d:%d): %v", line, column, e)
+}
+
+// Position returns line and column number of given index of the original input.
+// It will panic if index is out of range.
+func (d *Decoder) Position(idx int) (line int, column int) {
+	b := d.orig[:idx]
+	line = bytes.Count(b, []byte("\n")) + 1
+	if i := bytes.LastIndexByte(b, '\n'); i >= 0 {
+		b = b[i+1:]
+	}
+	column = utf8.RuneCount(b) + 1 // ignore multi-rune characters
+	return line, column
+}
+
+func (d *Decoder) tryConsumeChar(c byte) bool {
+	if len(d.in) > 0 && d.in[0] == c {
+		d.consume(1)
+		return true
+	}
+	return false
+}
+
+// consume consumes n bytes of input and any subsequent whitespace or comments.
+func (d *Decoder) consume(n int) {
+	d.in = consume(d.in, n)
+	return
+}
+
+// consume consumes n bytes of input and any subsequent whitespace or comments.
+func consume(b []byte, n int) []byte {
+	b = b[n:]
+	for len(b) > 0 {
+		switch b[0] {
+		case ' ', '\n', '\r', '\t':
+			b = b[1:]
+		case '#':
+			if i := bytes.IndexByte(b, '\n'); i >= 0 {
+				b = b[i+len("\n"):]
+			} else {
+				b = nil
+			}
+		default:
+			return b
+		}
+	}
+	return b
+}
+
+// Any sequence that looks like a non-delimiter (for error reporting).
+var errRegexp = regexp.MustCompile(`^([-+._a-zA-Z0-9\/]+|.)`)
+
+// isDelim returns true if given byte is a delimiter character.
+func isDelim(c byte) bool {
+	return !(c == '-' || c == '+' || c == '.' || c == '_' ||
+		('a' <= c && c <= 'z') ||
+		('A' <= c && c <= 'Z') ||
+		('0' <= c && c <= '9'))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/decode_number.go b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_number.go
new file mode 100644
index 0000000..f2d90b7
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_number.go
@@ -0,0 +1,190 @@
+// 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 text
+
+// parseNumberValue parses a number from the input and returns a Token object.
+func (d *Decoder) parseNumberValue() (Token, bool) {
+	in := d.in
+	num := parseNumber(in)
+	if num.size == 0 {
+		return Token{}, false
+	}
+	numAttrs := num.kind
+	if num.neg {
+		numAttrs |= isNegative
+	}
+	strSize := num.size
+	last := num.size - 1
+	if num.kind == numFloat && (d.in[last] == 'f' || d.in[last] == 'F') {
+		strSize = last
+	}
+	tok := Token{
+		kind:     Scalar,
+		attrs:    numberValue,
+		pos:      len(d.orig) - len(d.in),
+		raw:      d.in[:num.size],
+		str:      string(d.in[:strSize]),
+		numAttrs: numAttrs,
+	}
+	d.consume(num.size)
+	return tok, true
+}
+
+const (
+	numDec uint8 = (1 << iota) / 2
+	numHex
+	numOct
+	numFloat
+)
+
+// number is the result of parsing out a valid number from parseNumber. It
+// contains data for doing float or integer conversion via the strconv package
+// in conjunction with the input bytes.
+type number struct {
+	kind uint8
+	neg  bool
+	size int
+}
+
+// parseNumber constructs a number object from given input. It allows for the
+// following patterns:
+//   integer: ^-?([1-9][0-9]*|0[xX][0-9a-fA-F]+|0[0-7]*)
+//   float: ^-?((0|[1-9][0-9]*)?([.][0-9]*)?([eE][+-]?[0-9]+)?[fF]?)
+// It also returns the number of parsed bytes for the given number, 0 if it is
+// not a number.
+func parseNumber(input []byte) number {
+	kind := numDec
+	var size int
+	var neg bool
+
+	s := input
+	if len(s) == 0 {
+		return number{}
+	}
+
+	// Optional -
+	if s[0] == '-' {
+		neg = true
+		s = s[1:]
+		size++
+		if len(s) == 0 {
+			return number{}
+		}
+	}
+
+	// C++ allows for whitespace and comments in between the negative sign and
+	// the rest of the number. This logic currently does not but is consistent
+	// with v1.
+
+	switch {
+	case s[0] == '0':
+		if len(s) > 1 {
+			switch {
+			case s[1] == 'x' || s[1] == 'X':
+				// Parse as hex number.
+				kind = numHex
+				n := 2
+				s = s[2:]
+				for len(s) > 0 && (('0' <= s[0] && s[0] <= '9') ||
+					('a' <= s[0] && s[0] <= 'f') ||
+					('A' <= s[0] && s[0] <= 'F')) {
+					s = s[1:]
+					n++
+				}
+				if n == 2 {
+					return number{}
+				}
+				size += n
+
+			case '0' <= s[1] && s[1] <= '7':
+				// Parse as octal number.
+				kind = numOct
+				n := 2
+				s = s[2:]
+				for len(s) > 0 && '0' <= s[0] && s[0] <= '7' {
+					s = s[1:]
+					n++
+				}
+				size += n
+			}
+
+			if kind&(numHex|numOct) > 0 {
+				if len(s) > 0 && !isDelim(s[0]) {
+					return number{}
+				}
+				return number{kind: kind, neg: neg, size: size}
+			}
+		}
+		s = s[1:]
+		size++
+
+	case '1' <= s[0] && s[0] <= '9':
+		n := 1
+		s = s[1:]
+		for len(s) > 0 && '0' <= s[0] && s[0] <= '9' {
+			s = s[1:]
+			n++
+		}
+		size += n
+
+	case s[0] == '.':
+		// Set kind to numFloat to signify the intent to parse as float. And
+		// that it needs to have other digits after '.'.
+		kind = numFloat
+
+	default:
+		return number{}
+	}
+
+	// . followed by 0 or more digits.
+	if len(s) > 0 && s[0] == '.' {
+		n := 1
+		s = s[1:]
+		// If decimal point was before any digits, it should be followed by
+		// other digits.
+		if len(s) == 0 && kind == numFloat {
+			return number{}
+		}
+		for len(s) > 0 && '0' <= s[0] && s[0] <= '9' {
+			s = s[1:]
+			n++
+		}
+		size += n
+		kind = numFloat
+	}
+
+	// e or E followed by an optional - or + and 1 or more digits.
+	if len(s) >= 2 && (s[0] == 'e' || s[0] == 'E') {
+		kind = numFloat
+		s = s[1:]
+		n := 1
+		if s[0] == '+' || s[0] == '-' {
+			s = s[1:]
+			n++
+			if len(s) == 0 {
+				return number{}
+			}
+		}
+		for len(s) > 0 && '0' <= s[0] && s[0] <= '9' {
+			s = s[1:]
+			n++
+		}
+		size += n
+	}
+
+	// Optional suffix f or F for floats.
+	if len(s) > 0 && (s[0] == 'f' || s[0] == 'F') {
+		kind = numFloat
+		s = s[1:]
+		size++
+	}
+
+	// Check that next byte is a delimiter or it is at the end.
+	if len(s) > 0 && !isDelim(s[0]) {
+		return number{}
+	}
+
+	return number{kind: kind, neg: neg, size: size}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/decode_string.go b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_string.go
new file mode 100644
index 0000000..d4d3490
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_string.go
@@ -0,0 +1,161 @@
+// 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 text
+
+import (
+	"bytes"
+	"strconv"
+	"strings"
+	"unicode"
+	"unicode/utf16"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/internal/strs"
+)
+
+// parseStringValue parses string field token.
+// This differs from parseString since the text format allows
+// multiple back-to-back string literals where they are semantically treated
+// as a single large string with all values concatenated.
+//
+// E.g., `"foo" "bar" "baz"` => "foobarbaz"
+func (d *Decoder) parseStringValue() (Token, error) {
+	// Note that the ending quote is sufficient to unambiguously mark the end
+	// of a string. Thus, the text grammar does not require intervening
+	// whitespace or control characters in-between strings.
+	// Thus, the following is valid:
+	//	`"foo"'bar'"baz"` => "foobarbaz"
+	in0 := d.in
+	var ss []string
+	for len(d.in) > 0 && (d.in[0] == '"' || d.in[0] == '\'') {
+		s, err := d.parseString()
+		if err != nil {
+			return Token{}, err
+		}
+		ss = append(ss, s)
+	}
+	// d.in already points to the end of the value at this point.
+	return Token{
+		kind:  Scalar,
+		attrs: stringValue,
+		pos:   len(d.orig) - len(in0),
+		raw:   in0[:len(in0)-len(d.in)],
+		str:   strings.Join(ss, ""),
+	}, nil
+}
+
+// parseString parses a string value enclosed in " or '.
+func (d *Decoder) parseString() (string, error) {
+	in := d.in
+	if len(in) == 0 {
+		return "", ErrUnexpectedEOF
+	}
+	quote := in[0]
+	in = in[1:]
+	i := indexNeedEscapeInBytes(in)
+	in, out := in[i:], in[:i:i] // set cap to prevent mutations
+	for len(in) > 0 {
+		switch r, n := utf8.DecodeRune(in); {
+		case r == utf8.RuneError && n == 1:
+			return "", d.newSyntaxError("invalid UTF-8 detected")
+		case r == 0 || r == '\n':
+			return "", d.newSyntaxError("invalid character %q in string", r)
+		case r == rune(quote):
+			in = in[1:]
+			d.consume(len(d.in) - len(in))
+			return string(out), nil
+		case r == '\\':
+			if len(in) < 2 {
+				return "", ErrUnexpectedEOF
+			}
+			switch r := in[1]; r {
+			case '"', '\'', '\\', '?':
+				in, out = in[2:], append(out, r)
+			case 'a':
+				in, out = in[2:], append(out, '\a')
+			case 'b':
+				in, out = in[2:], append(out, '\b')
+			case 'n':
+				in, out = in[2:], append(out, '\n')
+			case 'r':
+				in, out = in[2:], append(out, '\r')
+			case 't':
+				in, out = in[2:], append(out, '\t')
+			case 'v':
+				in, out = in[2:], append(out, '\v')
+			case 'f':
+				in, out = in[2:], append(out, '\f')
+			case '0', '1', '2', '3', '4', '5', '6', '7':
+				// One, two, or three octal characters.
+				n := len(in[1:]) - len(bytes.TrimLeft(in[1:], "01234567"))
+				if n > 3 {
+					n = 3
+				}
+				v, err := strconv.ParseUint(string(in[1:1+n]), 8, 8)
+				if err != nil {
+					return "", d.newSyntaxError("invalid octal escape code %q in string", in[:1+n])
+				}
+				in, out = in[1+n:], append(out, byte(v))
+			case 'x':
+				// One or two hexadecimal characters.
+				n := len(in[2:]) - len(bytes.TrimLeft(in[2:], "0123456789abcdefABCDEF"))
+				if n > 2 {
+					n = 2
+				}
+				v, err := strconv.ParseUint(string(in[2:2+n]), 16, 8)
+				if err != nil {
+					return "", d.newSyntaxError("invalid hex escape code %q in string", in[:2+n])
+				}
+				in, out = in[2+n:], append(out, byte(v))
+			case 'u', 'U':
+				// Four or eight hexadecimal characters
+				n := 6
+				if r == 'U' {
+					n = 10
+				}
+				if len(in) < n {
+					return "", ErrUnexpectedEOF
+				}
+				v, err := strconv.ParseUint(string(in[2:n]), 16, 32)
+				if utf8.MaxRune < v || err != nil {
+					return "", d.newSyntaxError("invalid Unicode escape code %q in string", in[:n])
+				}
+				in = in[n:]
+
+				r := rune(v)
+				if utf16.IsSurrogate(r) {
+					if len(in) < 6 {
+						return "", ErrUnexpectedEOF
+					}
+					v, err := strconv.ParseUint(string(in[2:6]), 16, 16)
+					r = utf16.DecodeRune(r, rune(v))
+					if in[0] != '\\' || in[1] != 'u' || r == unicode.ReplacementChar || err != nil {
+						return "", d.newSyntaxError("invalid Unicode escape code %q in string", in[:6])
+					}
+					in = in[6:]
+				}
+				out = append(out, string(r)...)
+			default:
+				return "", d.newSyntaxError("invalid escape code %q in string", in[:2])
+			}
+		default:
+			i := indexNeedEscapeInBytes(in[n:])
+			in, out = in[n+i:], append(out, in[:n+i]...)
+		}
+	}
+	return "", ErrUnexpectedEOF
+}
+
+// indexNeedEscapeInString returns the index of the character that needs
+// escaping. If no characters need escaping, this returns the input length.
+func indexNeedEscapeInBytes(b []byte) int { return indexNeedEscapeInString(strs.UnsafeString(b)) }
+
+// UnmarshalString returns an unescaped string given a textproto string value.
+// String value needs to contain single or double quotes. This is only used by
+// internal/encoding/defval package for unmarshaling bytes.
+func UnmarshalString(s string) (string, error) {
+	d := NewDecoder([]byte(s))
+	return d.parseString()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/decode_token.go b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_token.go
new file mode 100644
index 0000000..83d2b0d
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/decode_token.go
@@ -0,0 +1,373 @@
+// 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 text
+
+import (
+	"bytes"
+	"fmt"
+	"math"
+	"strconv"
+	"strings"
+
+	"google.golang.org/protobuf/internal/flags"
+)
+
+// Kind represents a token kind expressible in the textproto format.
+type Kind uint8
+
+// Kind values.
+const (
+	Invalid Kind = iota
+	EOF
+	Name   // Name indicates the field name.
+	Scalar // Scalar are scalar values, e.g. "string", 47, ENUM_LITERAL, true.
+	MessageOpen
+	MessageClose
+	ListOpen
+	ListClose
+
+	// comma and semi-colon are only for parsing in between values and should not be exposed.
+	comma
+	semicolon
+
+	// bof indicates beginning of file, which is the default token
+	// kind at the beginning of parsing.
+	bof = Invalid
+)
+
+func (t Kind) String() string {
+	switch t {
+	case Invalid:
+		return "<invalid>"
+	case EOF:
+		return "eof"
+	case Scalar:
+		return "scalar"
+	case Name:
+		return "name"
+	case MessageOpen:
+		return "{"
+	case MessageClose:
+		return "}"
+	case ListOpen:
+		return "["
+	case ListClose:
+		return "]"
+	case comma:
+		return ","
+	case semicolon:
+		return ";"
+	default:
+		return fmt.Sprintf("<invalid:%v>", uint8(t))
+	}
+}
+
+// NameKind represents different types of field names.
+type NameKind uint8
+
+// NameKind values.
+const (
+	IdentName NameKind = iota + 1
+	TypeName
+	FieldNumber
+)
+
+func (t NameKind) String() string {
+	switch t {
+	case IdentName:
+		return "IdentName"
+	case TypeName:
+		return "TypeName"
+	case FieldNumber:
+		return "FieldNumber"
+	default:
+		return fmt.Sprintf("<invalid:%v>", uint8(t))
+	}
+}
+
+// Bit mask in Token.attrs to indicate if a Name token is followed by the
+// separator char ':'. The field name separator char is optional for message
+// field or repeated message field, but required for all other types. Decoder
+// simply indicates whether a Name token is followed by separator or not.  It is
+// up to the prototext package to validate.
+const hasSeparator = 1 << 7
+
+// Scalar value types.
+const (
+	numberValue = iota + 1
+	stringValue
+	literalValue
+)
+
+// Bit mask in Token.numAttrs to indicate that the number is a negative.
+const isNegative = 1 << 7
+
+// Token provides a parsed token kind and value. Values are provided by the
+// different accessor methods.
+type Token struct {
+	// Kind of the Token object.
+	kind Kind
+	// attrs contains metadata for the following Kinds:
+	// Name: hasSeparator bit and one of NameKind.
+	// Scalar: one of numberValue, stringValue, literalValue.
+	attrs uint8
+	// numAttrs contains metadata for numberValue:
+	// - highest bit is whether negative or positive.
+	// - lower bits indicate one of numDec, numHex, numOct, numFloat.
+	numAttrs uint8
+	// pos provides the position of the token in the original input.
+	pos int
+	// raw bytes of the serialized token.
+	// This is a subslice into the original input.
+	raw []byte
+	// str contains parsed string for the following:
+	// - stringValue of Scalar kind
+	// - numberValue of Scalar kind
+	// - TypeName of Name kind
+	str string
+}
+
+// Kind returns the token kind.
+func (t Token) Kind() Kind {
+	return t.kind
+}
+
+// RawString returns the read value in string.
+func (t Token) RawString() string {
+	return string(t.raw)
+}
+
+// Pos returns the token position from the input.
+func (t Token) Pos() int {
+	return t.pos
+}
+
+// NameKind returns IdentName, TypeName or FieldNumber.
+// It panics if type is not Name.
+func (t Token) NameKind() NameKind {
+	if t.kind == Name {
+		return NameKind(t.attrs &^ hasSeparator)
+	}
+	panic(fmt.Sprintf("Token is not a Name type: %s", t.kind))
+}
+
+// HasSeparator returns true if the field name is followed by the separator char
+// ':', else false. It panics if type is not Name.
+func (t Token) HasSeparator() bool {
+	if t.kind == Name {
+		return t.attrs&hasSeparator != 0
+	}
+	panic(fmt.Sprintf("Token is not a Name type: %s", t.kind))
+}
+
+// IdentName returns the value for IdentName type.
+func (t Token) IdentName() string {
+	if t.kind == Name && t.attrs&uint8(IdentName) != 0 {
+		return string(t.raw)
+	}
+	panic(fmt.Sprintf("Token is not an IdentName: %s:%s", t.kind, NameKind(t.attrs&^hasSeparator)))
+}
+
+// TypeName returns the value for TypeName type.
+func (t Token) TypeName() string {
+	if t.kind == Name && t.attrs&uint8(TypeName) != 0 {
+		return t.str
+	}
+	panic(fmt.Sprintf("Token is not a TypeName: %s:%s", t.kind, NameKind(t.attrs&^hasSeparator)))
+}
+
+// FieldNumber returns the value for FieldNumber type. It returns a
+// non-negative int32 value. Caller will still need to validate for the correct
+// field number range.
+func (t Token) FieldNumber() int32 {
+	if t.kind != Name || t.attrs&uint8(FieldNumber) == 0 {
+		panic(fmt.Sprintf("Token is not a FieldNumber: %s:%s", t.kind, NameKind(t.attrs&^hasSeparator)))
+	}
+	// Following should not return an error as it had already been called right
+	// before this Token was constructed.
+	num, _ := strconv.ParseInt(string(t.raw), 10, 32)
+	return int32(num)
+}
+
+// String returns the string value for a Scalar type.
+func (t Token) String() (string, bool) {
+	if t.kind != Scalar || t.attrs != stringValue {
+		return "", false
+	}
+	return t.str, true
+}
+
+// Enum returns the literal value for a Scalar type for use as enum literals.
+func (t Token) Enum() (string, bool) {
+	if t.kind != Scalar || t.attrs != literalValue || (len(t.raw) > 0 && t.raw[0] == '-') {
+		return "", false
+	}
+	return string(t.raw), true
+}
+
+// Bool returns the bool value for a Scalar type.
+func (t Token) Bool() (bool, bool) {
+	if t.kind != Scalar {
+		return false, false
+	}
+	switch t.attrs {
+	case literalValue:
+		if b, ok := boolLits[string(t.raw)]; ok {
+			return b, true
+		}
+	case numberValue:
+		// Unsigned integer representation of 0 or 1 is permitted: 00, 0x0, 01,
+		// 0x1, etc.
+		n, err := strconv.ParseUint(t.str, 0, 64)
+		if err == nil {
+			switch n {
+			case 0:
+				return false, true
+			case 1:
+				return true, true
+			}
+		}
+	}
+	return false, false
+}
+
+// These exact boolean literals are the ones supported in C++.
+var boolLits = map[string]bool{
+	"t":     true,
+	"true":  true,
+	"True":  true,
+	"f":     false,
+	"false": false,
+	"False": false,
+}
+
+// Uint64 returns the uint64 value for a Scalar type.
+func (t Token) Uint64() (uint64, bool) {
+	if t.kind != Scalar || t.attrs != numberValue ||
+		t.numAttrs&isNegative > 0 || t.numAttrs&numFloat > 0 {
+		return 0, false
+	}
+	n, err := strconv.ParseUint(t.str, 0, 64)
+	if err != nil {
+		return 0, false
+	}
+	return n, true
+}
+
+// Uint32 returns the uint32 value for a Scalar type.
+func (t Token) Uint32() (uint32, bool) {
+	if t.kind != Scalar || t.attrs != numberValue ||
+		t.numAttrs&isNegative > 0 || t.numAttrs&numFloat > 0 {
+		return 0, false
+	}
+	n, err := strconv.ParseUint(t.str, 0, 32)
+	if err != nil {
+		return 0, false
+	}
+	return uint32(n), true
+}
+
+// Int64 returns the int64 value for a Scalar type.
+func (t Token) Int64() (int64, bool) {
+	if t.kind != Scalar || t.attrs != numberValue || t.numAttrs&numFloat > 0 {
+		return 0, false
+	}
+	if n, err := strconv.ParseInt(t.str, 0, 64); err == nil {
+		return n, true
+	}
+	// C++ accepts large positive hex numbers as negative values.
+	// This feature is here for proto1 backwards compatibility purposes.
+	if flags.ProtoLegacy && (t.numAttrs == numHex) {
+		if n, err := strconv.ParseUint(t.str, 0, 64); err == nil {
+			return int64(n), true
+		}
+	}
+	return 0, false
+}
+
+// Int32 returns the int32 value for a Scalar type.
+func (t Token) Int32() (int32, bool) {
+	if t.kind != Scalar || t.attrs != numberValue || t.numAttrs&numFloat > 0 {
+		return 0, false
+	}
+	if n, err := strconv.ParseInt(t.str, 0, 32); err == nil {
+		return int32(n), true
+	}
+	// C++ accepts large positive hex numbers as negative values.
+	// This feature is here for proto1 backwards compatibility purposes.
+	if flags.ProtoLegacy && (t.numAttrs == numHex) {
+		if n, err := strconv.ParseUint(t.str, 0, 32); err == nil {
+			return int32(n), true
+		}
+	}
+	return 0, false
+}
+
+// Float64 returns the float64 value for a Scalar type.
+func (t Token) Float64() (float64, bool) {
+	if t.kind != Scalar {
+		return 0, false
+	}
+	switch t.attrs {
+	case literalValue:
+		if f, ok := floatLits[strings.ToLower(string(t.raw))]; ok {
+			return f, true
+		}
+	case numberValue:
+		n, err := strconv.ParseFloat(t.str, 64)
+		if err == nil {
+			return n, true
+		}
+		nerr := err.(*strconv.NumError)
+		if nerr.Err == strconv.ErrRange {
+			return n, true
+		}
+	}
+	return 0, false
+}
+
+// Float32 returns the float32 value for a Scalar type.
+func (t Token) Float32() (float32, bool) {
+	if t.kind != Scalar {
+		return 0, false
+	}
+	switch t.attrs {
+	case literalValue:
+		if f, ok := floatLits[strings.ToLower(string(t.raw))]; ok {
+			return float32(f), true
+		}
+	case numberValue:
+		n, err := strconv.ParseFloat(t.str, 64)
+		if err == nil {
+			// Overflows are treated as (-)infinity.
+			return float32(n), true
+		}
+		nerr := err.(*strconv.NumError)
+		if nerr.Err == strconv.ErrRange {
+			return float32(n), true
+		}
+	}
+	return 0, false
+}
+
+// These are the supported float literals which C++ permits case-insensitive
+// variants of these.
+var floatLits = map[string]float64{
+	"nan":       math.NaN(),
+	"inf":       math.Inf(1),
+	"infinity":  math.Inf(1),
+	"-inf":      math.Inf(-1),
+	"-infinity": math.Inf(-1),
+}
+
+// TokenEquals returns true if given Tokens are equal, else false.
+func TokenEquals(x, y Token) bool {
+	return x.kind == y.kind &&
+		x.attrs == y.attrs &&
+		x.numAttrs == y.numAttrs &&
+		x.pos == y.pos &&
+		bytes.Equal(x.raw, y.raw) &&
+		x.str == y.str
+}
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/doc.go b/vendor/google.golang.org/protobuf/internal/encoding/text/doc.go
new file mode 100644
index 0000000..0ce8d6f
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/doc.go
@@ -0,0 +1,29 @@
+// 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 text implements the text format for protocol buffers.
+// This package has no semantic understanding for protocol buffers and is only
+// a parser and composer for the format.
+//
+// There is no formal specification for the protobuf text format, as such the
+// C++ implementation (see google::protobuf::TextFormat) is the reference
+// implementation of the text format.
+//
+// This package is neither a superset nor a subset of the C++ implementation.
+// This implementation permits a more liberal grammar in some cases to be
+// backwards compatible with the historical Go implementation.
+// Future parsings unique to Go should not be added.
+// Some grammars allowed by the C++ implementation are deliberately
+// not implemented here because they are considered a bug by the protobuf team
+// and should not be replicated.
+//
+// The Go implementation should implement a sufficient amount of the C++
+// grammar such that the default text serialization by C++ can be parsed by Go.
+// However, just because the C++ parser accepts some input does not mean that
+// the Go implementation should as well.
+//
+// The text format is almost a superset of JSON except:
+//	* message keys are not quoted strings, but identifiers
+//	* the top-level value must be a message without the delimiters
+package text
diff --git a/vendor/google.golang.org/protobuf/internal/encoding/text/encode.go b/vendor/google.golang.org/protobuf/internal/encoding/text/encode.go
new file mode 100644
index 0000000..da289cc
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/encoding/text/encode.go
@@ -0,0 +1,270 @@
+// 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 text
+
+import (
+	"math"
+	"math/bits"
+	"strconv"
+	"strings"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/internal/detrand"
+	"google.golang.org/protobuf/internal/errors"
+)
+
+// encType represents an encoding type.
+type encType uint8
+
+const (
+	_ encType = (1 << iota) / 2
+	name
+	scalar
+	messageOpen
+	messageClose
+)
+
+// Encoder provides methods to write out textproto constructs and values. The user is
+// responsible for producing valid sequences of constructs and values.
+type Encoder struct {
+	encoderState
+
+	indent      string
+	delims      [2]byte
+	outputASCII bool
+}
+
+type encoderState struct {
+	lastType encType
+	indents  []byte
+	out      []byte
+}
+
+// NewEncoder returns an Encoder.
+//
+// If indent is a non-empty string, it causes every entry in a List or Message
+// to be preceded by the indent and trailed by a newline.
+//
+// If delims is not the zero value, it controls the delimiter characters used
+// for messages (e.g., "{}" vs "<>").
+//
+// If outputASCII is true, strings will be serialized in such a way that
+// multi-byte UTF-8 sequences are escaped. This property ensures that the
+// overall output is ASCII (as opposed to UTF-8).
+func NewEncoder(indent string, delims [2]byte, outputASCII bool) (*Encoder, error) {
+	e := &Encoder{}
+	if len(indent) > 0 {
+		if strings.Trim(indent, " \t") != "" {
+			return nil, errors.New("indent may only be composed of space and tab characters")
+		}
+		e.indent = indent
+	}
+	switch delims {
+	case [2]byte{0, 0}:
+		e.delims = [2]byte{'{', '}'}
+	case [2]byte{'{', '}'}, [2]byte{'<', '>'}:
+		e.delims = delims
+	default:
+		return nil, errors.New("delimiters may only be \"{}\" or \"<>\"")
+	}
+	e.outputASCII = outputASCII
+
+	return e, nil
+}
+
+// Bytes returns the content of the written bytes.
+func (e *Encoder) Bytes() []byte {
+	return e.out
+}
+
+// StartMessage writes out the '{' or '<' symbol.
+func (e *Encoder) StartMessage() {
+	e.prepareNext(messageOpen)
+	e.out = append(e.out, e.delims[0])
+}
+
+// EndMessage writes out the '}' or '>' symbol.
+func (e *Encoder) EndMessage() {
+	e.prepareNext(messageClose)
+	e.out = append(e.out, e.delims[1])
+}
+
+// WriteName writes out the field name and the separator ':'.
+func (e *Encoder) WriteName(s string) {
+	e.prepareNext(name)
+	e.out = append(e.out, s...)
+	e.out = append(e.out, ':')
+}
+
+// WriteBool writes out the given boolean value.
+func (e *Encoder) WriteBool(b bool) {
+	if b {
+		e.WriteLiteral("true")
+	} else {
+		e.WriteLiteral("false")
+	}
+}
+
+// WriteString writes out the given string value.
+func (e *Encoder) WriteString(s string) {
+	e.prepareNext(scalar)
+	e.out = appendString(e.out, s, e.outputASCII)
+}
+
+func appendString(out []byte, in string, outputASCII bool) []byte {
+	out = append(out, '"')
+	i := indexNeedEscapeInString(in)
+	in, out = in[i:], append(out, in[:i]...)
+	for len(in) > 0 {
+		switch r, n := utf8.DecodeRuneInString(in); {
+		case r == utf8.RuneError && n == 1:
+			// We do not report invalid UTF-8 because strings in the text format
+			// are used to represent both the proto string and bytes type.
+			r = rune(in[0])
+			fallthrough
+		case r < ' ' || r == '"' || r == '\\' || r == 0x7f:
+			out = append(out, '\\')
+			switch r {
+			case '"', '\\':
+				out = append(out, byte(r))
+			case '\n':
+				out = append(out, 'n')
+			case '\r':
+				out = append(out, 'r')
+			case '\t':
+				out = append(out, 't')
+			default:
+				out = append(out, 'x')
+				out = append(out, "00"[1+(bits.Len32(uint32(r))-1)/4:]...)
+				out = strconv.AppendUint(out, uint64(r), 16)
+			}
+			in = in[n:]
+		case r >= utf8.RuneSelf && (outputASCII || r <= 0x009f):
+			out = append(out, '\\')
+			if r <= math.MaxUint16 {
+				out = append(out, 'u')
+				out = append(out, "0000"[1+(bits.Len32(uint32(r))-1)/4:]...)
+				out = strconv.AppendUint(out, uint64(r), 16)
+			} else {
+				out = append(out, 'U')
+				out = append(out, "00000000"[1+(bits.Len32(uint32(r))-1)/4:]...)
+				out = strconv.AppendUint(out, uint64(r), 16)
+			}
+			in = in[n:]
+		default:
+			i := indexNeedEscapeInString(in[n:])
+			in, out = in[n+i:], append(out, in[:n+i]...)
+		}
+	}
+	out = append(out, '"')
+	return out
+}
+
+// indexNeedEscapeInString returns the index of the character that needs
+// escaping. If no characters need escaping, this returns the input length.
+func indexNeedEscapeInString(s string) int {
+	for i := 0; i < len(s); i++ {
+		if c := s[i]; c < ' ' || c == '"' || c == '\'' || c == '\\' || c >= 0x7f {
+			return i
+		}
+	}
+	return len(s)
+}
+
+// WriteFloat writes out the given float value for given bitSize.
+func (e *Encoder) WriteFloat(n float64, bitSize int) {
+	e.prepareNext(scalar)
+	e.out = appendFloat(e.out, n, bitSize)
+}
+
+func appendFloat(out []byte, n float64, bitSize int) []byte {
+	switch {
+	case math.IsNaN(n):
+		return append(out, "nan"...)
+	case math.IsInf(n, +1):
+		return append(out, "inf"...)
+	case math.IsInf(n, -1):
+		return append(out, "-inf"...)
+	default:
+		return strconv.AppendFloat(out, n, 'g', -1, bitSize)
+	}
+}
+
+// WriteInt writes out the given signed integer value.
+func (e *Encoder) WriteInt(n int64) {
+	e.prepareNext(scalar)
+	e.out = append(e.out, strconv.FormatInt(n, 10)...)
+}
+
+// WriteUint writes out the given unsigned integer value.
+func (e *Encoder) WriteUint(n uint64) {
+	e.prepareNext(scalar)
+	e.out = append(e.out, strconv.FormatUint(n, 10)...)
+}
+
+// WriteLiteral writes out the given string as a literal value without quotes.
+// This is used for writing enum literal strings.
+func (e *Encoder) WriteLiteral(s string) {
+	e.prepareNext(scalar)
+	e.out = append(e.out, s...)
+}
+
+// prepareNext adds possible space and indentation for the next value based
+// on last encType and indent option. It also updates e.lastType to next.
+func (e *Encoder) prepareNext(next encType) {
+	defer func() {
+		e.lastType = next
+	}()
+
+	// Single line.
+	if len(e.indent) == 0 {
+		// Add space after each field before the next one.
+		if e.lastType&(scalar|messageClose) != 0 && next == name {
+			e.out = append(e.out, ' ')
+			// Add a random extra space to make output unstable.
+			if detrand.Bool() {
+				e.out = append(e.out, ' ')
+			}
+		}
+		return
+	}
+
+	// Multi-line.
+	switch {
+	case e.lastType == name:
+		e.out = append(e.out, ' ')
+		// Add a random extra space after name: to make output unstable.
+		if detrand.Bool() {
+			e.out = append(e.out, ' ')
+		}
+
+	case e.lastType == messageOpen && next != messageClose:
+		e.indents = append(e.indents, e.indent...)
+		e.out = append(e.out, '\n')
+		e.out = append(e.out, e.indents...)
+
+	case e.lastType&(scalar|messageClose) != 0:
+		if next == messageClose {
+			e.indents = e.indents[:len(e.indents)-len(e.indent)]
+		}
+		e.out = append(e.out, '\n')
+		e.out = append(e.out, e.indents...)
+	}
+}
+
+// Snapshot returns the current snapshot for use in Reset.
+func (e *Encoder) Snapshot() encoderState {
+	return e.encoderState
+}
+
+// Reset resets the Encoder to the given encoderState from a Snapshot.
+func (e *Encoder) Reset(es encoderState) {
+	e.encoderState = es
+}
+
+// AppendString appends the escaped form of the input string to b.
+func AppendString(b []byte, s string) []byte {
+	return appendString(b, s, false)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/errors/errors.go b/vendor/google.golang.org/protobuf/internal/errors/errors.go
new file mode 100644
index 0000000..20c17b3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/errors/errors.go
@@ -0,0 +1,89 @@
+// 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 errors implements functions to manipulate errors.
+package errors
+
+import (
+	"errors"
+	"fmt"
+
+	"google.golang.org/protobuf/internal/detrand"
+)
+
+// Error is a sentinel matching all errors produced by this package.
+var Error = errors.New("protobuf error")
+
+// New formats a string according to the format specifier and arguments and
+// returns an error that has a "proto" prefix.
+func New(f string, x ...interface{}) error {
+	return &prefixError{s: format(f, x...)}
+}
+
+type prefixError struct{ s string }
+
+var prefix = func() string {
+	// Deliberately introduce instability into the error message string to
+	// discourage users from performing error string comparisons.
+	if detrand.Bool() {
+		return "proto: " // use non-breaking spaces (U+00a0)
+	} else {
+		return "proto: " // use regular spaces (U+0020)
+	}
+}()
+
+func (e *prefixError) Error() string {
+	return prefix + e.s
+}
+
+func (e *prefixError) Unwrap() error {
+	return Error
+}
+
+// Wrap returns an error that has a "proto" prefix, the formatted string described
+// by the format specifier and arguments, and a suffix of err. The error wraps err.
+func Wrap(err error, f string, x ...interface{}) error {
+	return &wrapError{
+		s:   format(f, x...),
+		err: err,
+	}
+}
+
+type wrapError struct {
+	s   string
+	err error
+}
+
+func (e *wrapError) Error() string {
+	return format("%v%v: %v", prefix, e.s, e.err)
+}
+
+func (e *wrapError) Unwrap() error {
+	return e.err
+}
+
+func (e *wrapError) Is(target error) bool {
+	return target == Error
+}
+
+func format(f string, x ...interface{}) string {
+	// avoid "proto: " prefix when chaining
+	for i := 0; i < len(x); i++ {
+		switch e := x[i].(type) {
+		case *prefixError:
+			x[i] = e.s
+		case *wrapError:
+			x[i] = format("%v: %v", e.s, e.err)
+		}
+	}
+	return fmt.Sprintf(f, x...)
+}
+
+func InvalidUTF8(name string) error {
+	return New("field %v contains invalid UTF-8", name)
+}
+
+func RequiredNotSet(name string) error {
+	return New("required field %v not set", name)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/errors/is_go112.go b/vendor/google.golang.org/protobuf/internal/errors/is_go112.go
new file mode 100644
index 0000000..f90e909
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/errors/is_go112.go
@@ -0,0 +1,39 @@
+// Copyright 2020 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.13
+
+package errors
+
+import "reflect"
+
+// Is is a copy of Go 1.13's errors.Is for use with older Go versions.
+func Is(err, target error) bool {
+	if target == nil {
+		return err == target
+	}
+
+	isComparable := reflect.TypeOf(target).Comparable()
+	for {
+		if isComparable && err == target {
+			return true
+		}
+		if x, ok := err.(interface{ Is(error) bool }); ok && x.Is(target) {
+			return true
+		}
+		if err = unwrap(err); err == nil {
+			return false
+		}
+	}
+}
+
+func unwrap(err error) error {
+	u, ok := err.(interface {
+		Unwrap() error
+	})
+	if !ok {
+		return nil
+	}
+	return u.Unwrap()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/errors/is_go113.go b/vendor/google.golang.org/protobuf/internal/errors/is_go113.go
new file mode 100644
index 0000000..dc05f41
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/errors/is_go113.go
@@ -0,0 +1,12 @@
+// Copyright 2020 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.13
+
+package errors
+
+import "errors"
+
+// Is is errors.Is.
+func Is(err, target error) bool { return errors.Is(err, target) }
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/build.go b/vendor/google.golang.org/protobuf/internal/filedesc/build.go
new file mode 100644
index 0000000..b293b69
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/build.go
@@ -0,0 +1,158 @@
+// 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.
+
+// Package filedesc provides functionality for constructing descriptors.
+//
+// The types in this package implement interfaces in the protoreflect package
+// related to protobuf descripriptors.
+package filedesc
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// Builder construct a protoreflect.FileDescriptor from the raw descriptor.
+type Builder struct {
+	// GoPackagePath is the Go package path that is invoking this builder.
+	GoPackagePath string
+
+	// RawDescriptor is the wire-encoded bytes of FileDescriptorProto
+	// and must be populated.
+	RawDescriptor []byte
+
+	// NumEnums is the total number of enums declared in the file.
+	NumEnums int32
+	// NumMessages is the total number of messages declared in the file.
+	// It includes the implicit message declarations for map entries.
+	NumMessages int32
+	// NumExtensions is the total number of extensions declared in the file.
+	NumExtensions int32
+	// NumServices is the total number of services declared in the file.
+	NumServices int32
+
+	// TypeResolver resolves extension field types for descriptor options.
+	// If nil, it uses protoregistry.GlobalTypes.
+	TypeResolver interface {
+		preg.ExtensionTypeResolver
+	}
+
+	// FileRegistry is use to lookup file, enum, and message dependencies.
+	// Once constructed, the file descriptor is registered here.
+	// If nil, it uses protoregistry.GlobalFiles.
+	FileRegistry interface {
+		FindFileByPath(string) (protoreflect.FileDescriptor, error)
+		FindDescriptorByName(pref.FullName) (pref.Descriptor, error)
+		RegisterFile(pref.FileDescriptor) error
+	}
+}
+
+// resolverByIndex is an interface Builder.FileRegistry may implement.
+// If so, it permits looking up an enum or message dependency based on the
+// sub-list and element index into filetype.Builder.DependencyIndexes.
+type resolverByIndex interface {
+	FindEnumByIndex(int32, int32, []Enum, []Message) pref.EnumDescriptor
+	FindMessageByIndex(int32, int32, []Enum, []Message) pref.MessageDescriptor
+}
+
+// Indexes of each sub-list in filetype.Builder.DependencyIndexes.
+const (
+	listFieldDeps int32 = iota
+	listExtTargets
+	listExtDeps
+	listMethInDeps
+	listMethOutDeps
+)
+
+// Out is the output of the Builder.
+type Out struct {
+	File pref.FileDescriptor
+
+	// Enums is all enum descriptors in "flattened ordering".
+	Enums []Enum
+	// Messages is all message descriptors in "flattened ordering".
+	// It includes the implicit message declarations for map entries.
+	Messages []Message
+	// Extensions is all extension descriptors in "flattened ordering".
+	Extensions []Extension
+	// Service is all service descriptors in "flattened ordering".
+	Services []Service
+}
+
+// Build constructs a FileDescriptor given the parameters set in Builder.
+// It assumes that the inputs are well-formed and panics if any inconsistencies
+// are encountered.
+//
+// If NumEnums+NumMessages+NumExtensions+NumServices is zero,
+// then Build automatically derives them from the raw descriptor.
+func (db Builder) Build() (out Out) {
+	// Populate the counts if uninitialized.
+	if db.NumEnums+db.NumMessages+db.NumExtensions+db.NumServices == 0 {
+		db.unmarshalCounts(db.RawDescriptor, true)
+	}
+
+	// Initialize resolvers and registries if unpopulated.
+	if db.TypeResolver == nil {
+		db.TypeResolver = preg.GlobalTypes
+	}
+	if db.FileRegistry == nil {
+		db.FileRegistry = preg.GlobalFiles
+	}
+
+	fd := newRawFile(db)
+	out.File = fd
+	out.Enums = fd.allEnums
+	out.Messages = fd.allMessages
+	out.Extensions = fd.allExtensions
+	out.Services = fd.allServices
+
+	if err := db.FileRegistry.RegisterFile(fd); err != nil {
+		panic(err)
+	}
+	return out
+}
+
+// unmarshalCounts counts the number of enum, message, extension, and service
+// declarations in the raw message, which is either a FileDescriptorProto
+// or a MessageDescriptorProto depending on whether isFile is set.
+func (db *Builder) unmarshalCounts(b []byte, isFile bool) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			if isFile {
+				switch num {
+				case genid.FileDescriptorProto_EnumType_field_number:
+					db.NumEnums++
+				case genid.FileDescriptorProto_MessageType_field_number:
+					db.unmarshalCounts(v, false)
+					db.NumMessages++
+				case genid.FileDescriptorProto_Extension_field_number:
+					db.NumExtensions++
+				case genid.FileDescriptorProto_Service_field_number:
+					db.NumServices++
+				}
+			} else {
+				switch num {
+				case genid.DescriptorProto_EnumType_field_number:
+					db.NumEnums++
+				case genid.DescriptorProto_NestedType_field_number:
+					db.unmarshalCounts(v, false)
+					db.NumMessages++
+				case genid.DescriptorProto_Extension_field_number:
+					db.NumExtensions++
+				}
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/desc.go b/vendor/google.golang.org/protobuf/internal/filedesc/desc.go
new file mode 100644
index 0000000..98ab142
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/desc.go
@@ -0,0 +1,631 @@
+// 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.
+
+package filedesc
+
+import (
+	"bytes"
+	"fmt"
+	"sync"
+	"sync/atomic"
+
+	"google.golang.org/protobuf/internal/descfmt"
+	"google.golang.org/protobuf/internal/descopts"
+	"google.golang.org/protobuf/internal/encoding/defval"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/internal/strs"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// The types in this file may have a suffix:
+//	• L0: Contains fields common to all descriptors (except File) and
+//	must be initialized up front.
+//	• L1: Contains fields specific to a descriptor and
+//	must be initialized up front.
+//	• L2: Contains fields that are lazily initialized when constructing
+//	from the raw file descriptor. When constructing as a literal, the L2
+//	fields must be initialized up front.
+//
+// The types are exported so that packages like reflect/protodesc can
+// directly construct descriptors.
+
+type (
+	File struct {
+		fileRaw
+		L1 FileL1
+
+		once uint32     // atomically set if L2 is valid
+		mu   sync.Mutex // protects L2
+		L2   *FileL2
+	}
+	FileL1 struct {
+		Syntax  pref.Syntax
+		Path    string
+		Package pref.FullName
+
+		Enums      Enums
+		Messages   Messages
+		Extensions Extensions
+		Services   Services
+	}
+	FileL2 struct {
+		Options   func() pref.ProtoMessage
+		Imports   FileImports
+		Locations SourceLocations
+	}
+)
+
+func (fd *File) ParentFile() pref.FileDescriptor { return fd }
+func (fd *File) Parent() pref.Descriptor         { return nil }
+func (fd *File) Index() int                      { return 0 }
+func (fd *File) Syntax() pref.Syntax             { return fd.L1.Syntax }
+func (fd *File) Name() pref.Name                 { return fd.L1.Package.Name() }
+func (fd *File) FullName() pref.FullName         { return fd.L1.Package }
+func (fd *File) IsPlaceholder() bool             { return false }
+func (fd *File) Options() pref.ProtoMessage {
+	if f := fd.lazyInit().Options; f != nil {
+		return f()
+	}
+	return descopts.File
+}
+func (fd *File) Path() string                          { return fd.L1.Path }
+func (fd *File) Package() pref.FullName                { return fd.L1.Package }
+func (fd *File) Imports() pref.FileImports             { return &fd.lazyInit().Imports }
+func (fd *File) Enums() pref.EnumDescriptors           { return &fd.L1.Enums }
+func (fd *File) Messages() pref.MessageDescriptors     { return &fd.L1.Messages }
+func (fd *File) Extensions() pref.ExtensionDescriptors { return &fd.L1.Extensions }
+func (fd *File) Services() pref.ServiceDescriptors     { return &fd.L1.Services }
+func (fd *File) SourceLocations() pref.SourceLocations { return &fd.lazyInit().Locations }
+func (fd *File) Format(s fmt.State, r rune)            { descfmt.FormatDesc(s, r, fd) }
+func (fd *File) ProtoType(pref.FileDescriptor)         {}
+func (fd *File) ProtoInternal(pragma.DoNotImplement)   {}
+
+func (fd *File) lazyInit() *FileL2 {
+	if atomic.LoadUint32(&fd.once) == 0 {
+		fd.lazyInitOnce()
+	}
+	return fd.L2
+}
+
+func (fd *File) lazyInitOnce() {
+	fd.mu.Lock()
+	if fd.L2 == nil {
+		fd.lazyRawInit() // recursively initializes all L2 structures
+	}
+	atomic.StoreUint32(&fd.once, 1)
+	fd.mu.Unlock()
+}
+
+// GoPackagePath is a pseudo-internal API for determining the Go package path
+// that this file descriptor is declared in.
+//
+// WARNING: This method is exempt from the compatibility promise and may be
+// removed in the future without warning.
+func (fd *File) GoPackagePath() string {
+	return fd.builder.GoPackagePath
+}
+
+type (
+	Enum struct {
+		Base
+		L1 EnumL1
+		L2 *EnumL2 // protected by fileDesc.once
+	}
+	EnumL1 struct {
+		eagerValues bool // controls whether EnumL2.Values is already populated
+	}
+	EnumL2 struct {
+		Options        func() pref.ProtoMessage
+		Values         EnumValues
+		ReservedNames  Names
+		ReservedRanges EnumRanges
+	}
+
+	EnumValue struct {
+		Base
+		L1 EnumValueL1
+	}
+	EnumValueL1 struct {
+		Options func() pref.ProtoMessage
+		Number  pref.EnumNumber
+	}
+)
+
+func (ed *Enum) Options() pref.ProtoMessage {
+	if f := ed.lazyInit().Options; f != nil {
+		return f()
+	}
+	return descopts.Enum
+}
+func (ed *Enum) Values() pref.EnumValueDescriptors {
+	if ed.L1.eagerValues {
+		return &ed.L2.Values
+	}
+	return &ed.lazyInit().Values
+}
+func (ed *Enum) ReservedNames() pref.Names       { return &ed.lazyInit().ReservedNames }
+func (ed *Enum) ReservedRanges() pref.EnumRanges { return &ed.lazyInit().ReservedRanges }
+func (ed *Enum) Format(s fmt.State, r rune)      { descfmt.FormatDesc(s, r, ed) }
+func (ed *Enum) ProtoType(pref.EnumDescriptor)   {}
+func (ed *Enum) lazyInit() *EnumL2 {
+	ed.L0.ParentFile.lazyInit() // implicitly initializes L2
+	return ed.L2
+}
+
+func (ed *EnumValue) Options() pref.ProtoMessage {
+	if f := ed.L1.Options; f != nil {
+		return f()
+	}
+	return descopts.EnumValue
+}
+func (ed *EnumValue) Number() pref.EnumNumber            { return ed.L1.Number }
+func (ed *EnumValue) Format(s fmt.State, r rune)         { descfmt.FormatDesc(s, r, ed) }
+func (ed *EnumValue) ProtoType(pref.EnumValueDescriptor) {}
+
+type (
+	Message struct {
+		Base
+		L1 MessageL1
+		L2 *MessageL2 // protected by fileDesc.once
+	}
+	MessageL1 struct {
+		Enums        Enums
+		Messages     Messages
+		Extensions   Extensions
+		IsMapEntry   bool // promoted from google.protobuf.MessageOptions
+		IsMessageSet bool // promoted from google.protobuf.MessageOptions
+	}
+	MessageL2 struct {
+		Options               func() pref.ProtoMessage
+		Fields                Fields
+		Oneofs                Oneofs
+		ReservedNames         Names
+		ReservedRanges        FieldRanges
+		RequiredNumbers       FieldNumbers // must be consistent with Fields.Cardinality
+		ExtensionRanges       FieldRanges
+		ExtensionRangeOptions []func() pref.ProtoMessage // must be same length as ExtensionRanges
+	}
+
+	Field struct {
+		Base
+		L1 FieldL1
+	}
+	FieldL1 struct {
+		Options          func() pref.ProtoMessage
+		Number           pref.FieldNumber
+		Cardinality      pref.Cardinality // must be consistent with Message.RequiredNumbers
+		Kind             pref.Kind
+		StringName       stringName
+		IsProto3Optional bool // promoted from google.protobuf.FieldDescriptorProto
+		IsWeak           bool // promoted from google.protobuf.FieldOptions
+		HasPacked        bool // promoted from google.protobuf.FieldOptions
+		IsPacked         bool // promoted from google.protobuf.FieldOptions
+		HasEnforceUTF8   bool // promoted from google.protobuf.FieldOptions
+		EnforceUTF8      bool // promoted from google.protobuf.FieldOptions
+		Default          defaultValue
+		ContainingOneof  pref.OneofDescriptor // must be consistent with Message.Oneofs.Fields
+		Enum             pref.EnumDescriptor
+		Message          pref.MessageDescriptor
+	}
+
+	Oneof struct {
+		Base
+		L1 OneofL1
+	}
+	OneofL1 struct {
+		Options func() pref.ProtoMessage
+		Fields  OneofFields // must be consistent with Message.Fields.ContainingOneof
+	}
+)
+
+func (md *Message) Options() pref.ProtoMessage {
+	if f := md.lazyInit().Options; f != nil {
+		return f()
+	}
+	return descopts.Message
+}
+func (md *Message) IsMapEntry() bool                   { return md.L1.IsMapEntry }
+func (md *Message) Fields() pref.FieldDescriptors      { return &md.lazyInit().Fields }
+func (md *Message) Oneofs() pref.OneofDescriptors      { return &md.lazyInit().Oneofs }
+func (md *Message) ReservedNames() pref.Names          { return &md.lazyInit().ReservedNames }
+func (md *Message) ReservedRanges() pref.FieldRanges   { return &md.lazyInit().ReservedRanges }
+func (md *Message) RequiredNumbers() pref.FieldNumbers { return &md.lazyInit().RequiredNumbers }
+func (md *Message) ExtensionRanges() pref.FieldRanges  { return &md.lazyInit().ExtensionRanges }
+func (md *Message) ExtensionRangeOptions(i int) pref.ProtoMessage {
+	if f := md.lazyInit().ExtensionRangeOptions[i]; f != nil {
+		return f()
+	}
+	return descopts.ExtensionRange
+}
+func (md *Message) Enums() pref.EnumDescriptors           { return &md.L1.Enums }
+func (md *Message) Messages() pref.MessageDescriptors     { return &md.L1.Messages }
+func (md *Message) Extensions() pref.ExtensionDescriptors { return &md.L1.Extensions }
+func (md *Message) ProtoType(pref.MessageDescriptor)      {}
+func (md *Message) Format(s fmt.State, r rune)            { descfmt.FormatDesc(s, r, md) }
+func (md *Message) lazyInit() *MessageL2 {
+	md.L0.ParentFile.lazyInit() // implicitly initializes L2
+	return md.L2
+}
+
+// IsMessageSet is a pseudo-internal API for checking whether a message
+// should serialize in the proto1 message format.
+//
+// WARNING: This method is exempt from the compatibility promise and may be
+// removed in the future without warning.
+func (md *Message) IsMessageSet() bool {
+	return md.L1.IsMessageSet
+}
+
+func (fd *Field) Options() pref.ProtoMessage {
+	if f := fd.L1.Options; f != nil {
+		return f()
+	}
+	return descopts.Field
+}
+func (fd *Field) Number() pref.FieldNumber      { return fd.L1.Number }
+func (fd *Field) Cardinality() pref.Cardinality { return fd.L1.Cardinality }
+func (fd *Field) Kind() pref.Kind               { return fd.L1.Kind }
+func (fd *Field) HasJSONName() bool             { return fd.L1.StringName.hasJSON }
+func (fd *Field) JSONName() string              { return fd.L1.StringName.getJSON(fd) }
+func (fd *Field) TextName() string              { return fd.L1.StringName.getText(fd) }
+func (fd *Field) HasPresence() bool {
+	return fd.L1.Cardinality != pref.Repeated && (fd.L0.ParentFile.L1.Syntax == pref.Proto2 || fd.L1.Message != nil || fd.L1.ContainingOneof != nil)
+}
+func (fd *Field) HasOptionalKeyword() bool {
+	return (fd.L0.ParentFile.L1.Syntax == pref.Proto2 && fd.L1.Cardinality == pref.Optional && fd.L1.ContainingOneof == nil) || fd.L1.IsProto3Optional
+}
+func (fd *Field) IsPacked() bool {
+	if !fd.L1.HasPacked && fd.L0.ParentFile.L1.Syntax != pref.Proto2 && fd.L1.Cardinality == pref.Repeated {
+		switch fd.L1.Kind {
+		case pref.StringKind, pref.BytesKind, pref.MessageKind, pref.GroupKind:
+		default:
+			return true
+		}
+	}
+	return fd.L1.IsPacked
+}
+func (fd *Field) IsExtension() bool { return false }
+func (fd *Field) IsWeak() bool      { return fd.L1.IsWeak }
+func (fd *Field) IsList() bool      { return fd.Cardinality() == pref.Repeated && !fd.IsMap() }
+func (fd *Field) IsMap() bool       { return fd.Message() != nil && fd.Message().IsMapEntry() }
+func (fd *Field) MapKey() pref.FieldDescriptor {
+	if !fd.IsMap() {
+		return nil
+	}
+	return fd.Message().Fields().ByNumber(genid.MapEntry_Key_field_number)
+}
+func (fd *Field) MapValue() pref.FieldDescriptor {
+	if !fd.IsMap() {
+		return nil
+	}
+	return fd.Message().Fields().ByNumber(genid.MapEntry_Value_field_number)
+}
+func (fd *Field) HasDefault() bool                           { return fd.L1.Default.has }
+func (fd *Field) Default() pref.Value                        { return fd.L1.Default.get(fd) }
+func (fd *Field) DefaultEnumValue() pref.EnumValueDescriptor { return fd.L1.Default.enum }
+func (fd *Field) ContainingOneof() pref.OneofDescriptor      { return fd.L1.ContainingOneof }
+func (fd *Field) ContainingMessage() pref.MessageDescriptor {
+	return fd.L0.Parent.(pref.MessageDescriptor)
+}
+func (fd *Field) Enum() pref.EnumDescriptor {
+	return fd.L1.Enum
+}
+func (fd *Field) Message() pref.MessageDescriptor {
+	if fd.L1.IsWeak {
+		if d, _ := protoregistry.GlobalFiles.FindDescriptorByName(fd.L1.Message.FullName()); d != nil {
+			return d.(pref.MessageDescriptor)
+		}
+	}
+	return fd.L1.Message
+}
+func (fd *Field) Format(s fmt.State, r rune)     { descfmt.FormatDesc(s, r, fd) }
+func (fd *Field) ProtoType(pref.FieldDescriptor) {}
+
+// EnforceUTF8 is a pseudo-internal API to determine whether to enforce UTF-8
+// validation for the string field. This exists for Google-internal use only
+// since proto3 did not enforce UTF-8 validity prior to the open-source release.
+// If this method does not exist, the default is to enforce valid UTF-8.
+//
+// WARNING: This method is exempt from the compatibility promise and may be
+// removed in the future without warning.
+func (fd *Field) EnforceUTF8() bool {
+	if fd.L1.HasEnforceUTF8 {
+		return fd.L1.EnforceUTF8
+	}
+	return fd.L0.ParentFile.L1.Syntax == pref.Proto3
+}
+
+func (od *Oneof) IsSynthetic() bool {
+	return od.L0.ParentFile.L1.Syntax == pref.Proto3 && len(od.L1.Fields.List) == 1 && od.L1.Fields.List[0].HasOptionalKeyword()
+}
+func (od *Oneof) Options() pref.ProtoMessage {
+	if f := od.L1.Options; f != nil {
+		return f()
+	}
+	return descopts.Oneof
+}
+func (od *Oneof) Fields() pref.FieldDescriptors  { return &od.L1.Fields }
+func (od *Oneof) Format(s fmt.State, r rune)     { descfmt.FormatDesc(s, r, od) }
+func (od *Oneof) ProtoType(pref.OneofDescriptor) {}
+
+type (
+	Extension struct {
+		Base
+		L1 ExtensionL1
+		L2 *ExtensionL2 // protected by fileDesc.once
+	}
+	ExtensionL1 struct {
+		Number      pref.FieldNumber
+		Extendee    pref.MessageDescriptor
+		Cardinality pref.Cardinality
+		Kind        pref.Kind
+	}
+	ExtensionL2 struct {
+		Options          func() pref.ProtoMessage
+		StringName       stringName
+		IsProto3Optional bool // promoted from google.protobuf.FieldDescriptorProto
+		IsPacked         bool // promoted from google.protobuf.FieldOptions
+		Default          defaultValue
+		Enum             pref.EnumDescriptor
+		Message          pref.MessageDescriptor
+	}
+)
+
+func (xd *Extension) Options() pref.ProtoMessage {
+	if f := xd.lazyInit().Options; f != nil {
+		return f()
+	}
+	return descopts.Field
+}
+func (xd *Extension) Number() pref.FieldNumber      { return xd.L1.Number }
+func (xd *Extension) Cardinality() pref.Cardinality { return xd.L1.Cardinality }
+func (xd *Extension) Kind() pref.Kind               { return xd.L1.Kind }
+func (xd *Extension) HasJSONName() bool             { return xd.lazyInit().StringName.hasJSON }
+func (xd *Extension) JSONName() string              { return xd.lazyInit().StringName.getJSON(xd) }
+func (xd *Extension) TextName() string              { return xd.lazyInit().StringName.getText(xd) }
+func (xd *Extension) HasPresence() bool             { return xd.L1.Cardinality != pref.Repeated }
+func (xd *Extension) HasOptionalKeyword() bool {
+	return (xd.L0.ParentFile.L1.Syntax == pref.Proto2 && xd.L1.Cardinality == pref.Optional) || xd.lazyInit().IsProto3Optional
+}
+func (xd *Extension) IsPacked() bool                             { return xd.lazyInit().IsPacked }
+func (xd *Extension) IsExtension() bool                          { return true }
+func (xd *Extension) IsWeak() bool                               { return false }
+func (xd *Extension) IsList() bool                               { return xd.Cardinality() == pref.Repeated }
+func (xd *Extension) IsMap() bool                                { return false }
+func (xd *Extension) MapKey() pref.FieldDescriptor               { return nil }
+func (xd *Extension) MapValue() pref.FieldDescriptor             { return nil }
+func (xd *Extension) HasDefault() bool                           { return xd.lazyInit().Default.has }
+func (xd *Extension) Default() pref.Value                        { return xd.lazyInit().Default.get(xd) }
+func (xd *Extension) DefaultEnumValue() pref.EnumValueDescriptor { return xd.lazyInit().Default.enum }
+func (xd *Extension) ContainingOneof() pref.OneofDescriptor      { return nil }
+func (xd *Extension) ContainingMessage() pref.MessageDescriptor  { return xd.L1.Extendee }
+func (xd *Extension) Enum() pref.EnumDescriptor                  { return xd.lazyInit().Enum }
+func (xd *Extension) Message() pref.MessageDescriptor            { return xd.lazyInit().Message }
+func (xd *Extension) Format(s fmt.State, r rune)                 { descfmt.FormatDesc(s, r, xd) }
+func (xd *Extension) ProtoType(pref.FieldDescriptor)             {}
+func (xd *Extension) ProtoInternal(pragma.DoNotImplement)        {}
+func (xd *Extension) lazyInit() *ExtensionL2 {
+	xd.L0.ParentFile.lazyInit() // implicitly initializes L2
+	return xd.L2
+}
+
+type (
+	Service struct {
+		Base
+		L1 ServiceL1
+		L2 *ServiceL2 // protected by fileDesc.once
+	}
+	ServiceL1 struct{}
+	ServiceL2 struct {
+		Options func() pref.ProtoMessage
+		Methods Methods
+	}
+
+	Method struct {
+		Base
+		L1 MethodL1
+	}
+	MethodL1 struct {
+		Options           func() pref.ProtoMessage
+		Input             pref.MessageDescriptor
+		Output            pref.MessageDescriptor
+		IsStreamingClient bool
+		IsStreamingServer bool
+	}
+)
+
+func (sd *Service) Options() pref.ProtoMessage {
+	if f := sd.lazyInit().Options; f != nil {
+		return f()
+	}
+	return descopts.Service
+}
+func (sd *Service) Methods() pref.MethodDescriptors     { return &sd.lazyInit().Methods }
+func (sd *Service) Format(s fmt.State, r rune)          { descfmt.FormatDesc(s, r, sd) }
+func (sd *Service) ProtoType(pref.ServiceDescriptor)    {}
+func (sd *Service) ProtoInternal(pragma.DoNotImplement) {}
+func (sd *Service) lazyInit() *ServiceL2 {
+	sd.L0.ParentFile.lazyInit() // implicitly initializes L2
+	return sd.L2
+}
+
+func (md *Method) Options() pref.ProtoMessage {
+	if f := md.L1.Options; f != nil {
+		return f()
+	}
+	return descopts.Method
+}
+func (md *Method) Input() pref.MessageDescriptor       { return md.L1.Input }
+func (md *Method) Output() pref.MessageDescriptor      { return md.L1.Output }
+func (md *Method) IsStreamingClient() bool             { return md.L1.IsStreamingClient }
+func (md *Method) IsStreamingServer() bool             { return md.L1.IsStreamingServer }
+func (md *Method) Format(s fmt.State, r rune)          { descfmt.FormatDesc(s, r, md) }
+func (md *Method) ProtoType(pref.MethodDescriptor)     {}
+func (md *Method) ProtoInternal(pragma.DoNotImplement) {}
+
+// Surrogate files are can be used to create standalone descriptors
+// where the syntax is only information derived from the parent file.
+var (
+	SurrogateProto2 = &File{L1: FileL1{Syntax: pref.Proto2}, L2: &FileL2{}}
+	SurrogateProto3 = &File{L1: FileL1{Syntax: pref.Proto3}, L2: &FileL2{}}
+)
+
+type (
+	Base struct {
+		L0 BaseL0
+	}
+	BaseL0 struct {
+		FullName   pref.FullName // must be populated
+		ParentFile *File         // must be populated
+		Parent     pref.Descriptor
+		Index      int
+	}
+)
+
+func (d *Base) Name() pref.Name         { return d.L0.FullName.Name() }
+func (d *Base) FullName() pref.FullName { return d.L0.FullName }
+func (d *Base) ParentFile() pref.FileDescriptor {
+	if d.L0.ParentFile == SurrogateProto2 || d.L0.ParentFile == SurrogateProto3 {
+		return nil // surrogate files are not real parents
+	}
+	return d.L0.ParentFile
+}
+func (d *Base) Parent() pref.Descriptor             { return d.L0.Parent }
+func (d *Base) Index() int                          { return d.L0.Index }
+func (d *Base) Syntax() pref.Syntax                 { return d.L0.ParentFile.Syntax() }
+func (d *Base) IsPlaceholder() bool                 { return false }
+func (d *Base) ProtoInternal(pragma.DoNotImplement) {}
+
+type stringName struct {
+	hasJSON  bool
+	once     sync.Once
+	nameJSON string
+	nameText string
+}
+
+// InitJSON initializes the name. It is exported for use by other internal packages.
+func (s *stringName) InitJSON(name string) {
+	s.hasJSON = true
+	s.nameJSON = name
+}
+
+func (s *stringName) lazyInit(fd pref.FieldDescriptor) *stringName {
+	s.once.Do(func() {
+		if fd.IsExtension() {
+			// For extensions, JSON and text are formatted the same way.
+			var name string
+			if messageset.IsMessageSetExtension(fd) {
+				name = string("[" + fd.FullName().Parent() + "]")
+			} else {
+				name = string("[" + fd.FullName() + "]")
+			}
+			s.nameJSON = name
+			s.nameText = name
+		} else {
+			// Format the JSON name.
+			if !s.hasJSON {
+				s.nameJSON = strs.JSONCamelCase(string(fd.Name()))
+			}
+
+			// Format the text name.
+			s.nameText = string(fd.Name())
+			if fd.Kind() == pref.GroupKind {
+				s.nameText = string(fd.Message().Name())
+			}
+		}
+	})
+	return s
+}
+
+func (s *stringName) getJSON(fd pref.FieldDescriptor) string { return s.lazyInit(fd).nameJSON }
+func (s *stringName) getText(fd pref.FieldDescriptor) string { return s.lazyInit(fd).nameText }
+
+func DefaultValue(v pref.Value, ev pref.EnumValueDescriptor) defaultValue {
+	dv := defaultValue{has: v.IsValid(), val: v, enum: ev}
+	if b, ok := v.Interface().([]byte); ok {
+		// Store a copy of the default bytes, so that we can detect
+		// accidental mutations of the original value.
+		dv.bytes = append([]byte(nil), b...)
+	}
+	return dv
+}
+
+func unmarshalDefault(b []byte, k pref.Kind, pf *File, ed pref.EnumDescriptor) defaultValue {
+	var evs pref.EnumValueDescriptors
+	if k == pref.EnumKind {
+		// If the enum is declared within the same file, be careful not to
+		// blindly call the Values method, lest we bind ourselves in a deadlock.
+		if e, ok := ed.(*Enum); ok && e.L0.ParentFile == pf {
+			evs = &e.L2.Values
+		} else {
+			evs = ed.Values()
+		}
+
+		// If we are unable to resolve the enum dependency, use a placeholder
+		// enum value since we will not be able to parse the default value.
+		if ed.IsPlaceholder() && pref.Name(b).IsValid() {
+			v := pref.ValueOfEnum(0)
+			ev := PlaceholderEnumValue(ed.FullName().Parent().Append(pref.Name(b)))
+			return DefaultValue(v, ev)
+		}
+	}
+
+	v, ev, err := defval.Unmarshal(string(b), k, evs, defval.Descriptor)
+	if err != nil {
+		panic(err)
+	}
+	return DefaultValue(v, ev)
+}
+
+type defaultValue struct {
+	has   bool
+	val   pref.Value
+	enum  pref.EnumValueDescriptor
+	bytes []byte
+}
+
+func (dv *defaultValue) get(fd pref.FieldDescriptor) pref.Value {
+	// Return the zero value as the default if unpopulated.
+	if !dv.has {
+		if fd.Cardinality() == pref.Repeated {
+			return pref.Value{}
+		}
+		switch fd.Kind() {
+		case pref.BoolKind:
+			return pref.ValueOfBool(false)
+		case pref.Int32Kind, pref.Sint32Kind, pref.Sfixed32Kind:
+			return pref.ValueOfInt32(0)
+		case pref.Int64Kind, pref.Sint64Kind, pref.Sfixed64Kind:
+			return pref.ValueOfInt64(0)
+		case pref.Uint32Kind, pref.Fixed32Kind:
+			return pref.ValueOfUint32(0)
+		case pref.Uint64Kind, pref.Fixed64Kind:
+			return pref.ValueOfUint64(0)
+		case pref.FloatKind:
+			return pref.ValueOfFloat32(0)
+		case pref.DoubleKind:
+			return pref.ValueOfFloat64(0)
+		case pref.StringKind:
+			return pref.ValueOfString("")
+		case pref.BytesKind:
+			return pref.ValueOfBytes(nil)
+		case pref.EnumKind:
+			if evs := fd.Enum().Values(); evs.Len() > 0 {
+				return pref.ValueOfEnum(evs.Get(0).Number())
+			}
+			return pref.ValueOfEnum(0)
+		}
+	}
+
+	if len(dv.bytes) > 0 && !bytes.Equal(dv.bytes, dv.val.Bytes()) {
+		// TODO: Avoid panic if we're running with the race detector
+		// and instead spawn a goroutine that periodically resets
+		// this value back to the original to induce a race.
+		panic(fmt.Sprintf("detected mutation on the default bytes for %v", fd.FullName()))
+	}
+	return dv.val
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/desc_init.go b/vendor/google.golang.org/protobuf/internal/filedesc/desc_init.go
new file mode 100644
index 0000000..66e1fee
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/desc_init.go
@@ -0,0 +1,471 @@
+// 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.
+
+package filedesc
+
+import (
+	"sync"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/strs"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// fileRaw is a data struct used when initializing a file descriptor from
+// a raw FileDescriptorProto.
+type fileRaw struct {
+	builder       Builder
+	allEnums      []Enum
+	allMessages   []Message
+	allExtensions []Extension
+	allServices   []Service
+}
+
+func newRawFile(db Builder) *File {
+	fd := &File{fileRaw: fileRaw{builder: db}}
+	fd.initDecls(db.NumEnums, db.NumMessages, db.NumExtensions, db.NumServices)
+	fd.unmarshalSeed(db.RawDescriptor)
+
+	// Extended message targets are eagerly resolved since registration
+	// needs this information at program init time.
+	for i := range fd.allExtensions {
+		xd := &fd.allExtensions[i]
+		xd.L1.Extendee = fd.resolveMessageDependency(xd.L1.Extendee, listExtTargets, int32(i))
+	}
+
+	fd.checkDecls()
+	return fd
+}
+
+// initDecls pre-allocates slices for the exact number of enums, messages
+// (including map entries), extensions, and services declared in the proto file.
+// This is done to avoid regrowing the slice, which would change the address
+// for any previously seen declaration.
+//
+// The alloc methods "allocates" slices by pulling from the capacity.
+func (fd *File) initDecls(numEnums, numMessages, numExtensions, numServices int32) {
+	fd.allEnums = make([]Enum, 0, numEnums)
+	fd.allMessages = make([]Message, 0, numMessages)
+	fd.allExtensions = make([]Extension, 0, numExtensions)
+	fd.allServices = make([]Service, 0, numServices)
+}
+
+func (fd *File) allocEnums(n int) []Enum {
+	total := len(fd.allEnums)
+	es := fd.allEnums[total : total+n]
+	fd.allEnums = fd.allEnums[:total+n]
+	return es
+}
+func (fd *File) allocMessages(n int) []Message {
+	total := len(fd.allMessages)
+	ms := fd.allMessages[total : total+n]
+	fd.allMessages = fd.allMessages[:total+n]
+	return ms
+}
+func (fd *File) allocExtensions(n int) []Extension {
+	total := len(fd.allExtensions)
+	xs := fd.allExtensions[total : total+n]
+	fd.allExtensions = fd.allExtensions[:total+n]
+	return xs
+}
+func (fd *File) allocServices(n int) []Service {
+	total := len(fd.allServices)
+	xs := fd.allServices[total : total+n]
+	fd.allServices = fd.allServices[:total+n]
+	return xs
+}
+
+// checkDecls performs a sanity check that the expected number of expected
+// declarations matches the number that were found in the descriptor proto.
+func (fd *File) checkDecls() {
+	switch {
+	case len(fd.allEnums) != cap(fd.allEnums):
+	case len(fd.allMessages) != cap(fd.allMessages):
+	case len(fd.allExtensions) != cap(fd.allExtensions):
+	case len(fd.allServices) != cap(fd.allServices):
+	default:
+		return
+	}
+	panic("mismatching cardinality")
+}
+
+func (fd *File) unmarshalSeed(b []byte) {
+	sb := getBuilder()
+	defer putBuilder(sb)
+
+	var prevField pref.FieldNumber
+	var numEnums, numMessages, numExtensions, numServices int
+	var posEnums, posMessages, posExtensions, posServices int
+	b0 := b
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.FileDescriptorProto_Syntax_field_number:
+				switch string(v) {
+				case "proto2":
+					fd.L1.Syntax = pref.Proto2
+				case "proto3":
+					fd.L1.Syntax = pref.Proto3
+				default:
+					panic("invalid syntax")
+				}
+			case genid.FileDescriptorProto_Name_field_number:
+				fd.L1.Path = sb.MakeString(v)
+			case genid.FileDescriptorProto_Package_field_number:
+				fd.L1.Package = pref.FullName(sb.MakeString(v))
+			case genid.FileDescriptorProto_EnumType_field_number:
+				if prevField != genid.FileDescriptorProto_EnumType_field_number {
+					if numEnums > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posEnums = len(b0) - len(b) - n - m
+				}
+				numEnums++
+			case genid.FileDescriptorProto_MessageType_field_number:
+				if prevField != genid.FileDescriptorProto_MessageType_field_number {
+					if numMessages > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posMessages = len(b0) - len(b) - n - m
+				}
+				numMessages++
+			case genid.FileDescriptorProto_Extension_field_number:
+				if prevField != genid.FileDescriptorProto_Extension_field_number {
+					if numExtensions > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posExtensions = len(b0) - len(b) - n - m
+				}
+				numExtensions++
+			case genid.FileDescriptorProto_Service_field_number:
+				if prevField != genid.FileDescriptorProto_Service_field_number {
+					if numServices > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posServices = len(b0) - len(b) - n - m
+				}
+				numServices++
+			}
+			prevField = num
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+			prevField = -1 // ignore known field numbers of unknown wire type
+		}
+	}
+
+	// If syntax is missing, it is assumed to be proto2.
+	if fd.L1.Syntax == 0 {
+		fd.L1.Syntax = pref.Proto2
+	}
+
+	// Must allocate all declarations before parsing each descriptor type
+	// to ensure we handled all descriptors in "flattened ordering".
+	if numEnums > 0 {
+		fd.L1.Enums.List = fd.allocEnums(numEnums)
+	}
+	if numMessages > 0 {
+		fd.L1.Messages.List = fd.allocMessages(numMessages)
+	}
+	if numExtensions > 0 {
+		fd.L1.Extensions.List = fd.allocExtensions(numExtensions)
+	}
+	if numServices > 0 {
+		fd.L1.Services.List = fd.allocServices(numServices)
+	}
+
+	if numEnums > 0 {
+		b := b0[posEnums:]
+		for i := range fd.L1.Enums.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			fd.L1.Enums.List[i].unmarshalSeed(v, sb, fd, fd, i)
+			b = b[n+m:]
+		}
+	}
+	if numMessages > 0 {
+		b := b0[posMessages:]
+		for i := range fd.L1.Messages.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			fd.L1.Messages.List[i].unmarshalSeed(v, sb, fd, fd, i)
+			b = b[n+m:]
+		}
+	}
+	if numExtensions > 0 {
+		b := b0[posExtensions:]
+		for i := range fd.L1.Extensions.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			fd.L1.Extensions.List[i].unmarshalSeed(v, sb, fd, fd, i)
+			b = b[n+m:]
+		}
+	}
+	if numServices > 0 {
+		b := b0[posServices:]
+		for i := range fd.L1.Services.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			fd.L1.Services.List[i].unmarshalSeed(v, sb, fd, fd, i)
+			b = b[n+m:]
+		}
+	}
+}
+
+func (ed *Enum) unmarshalSeed(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	ed.L0.ParentFile = pf
+	ed.L0.Parent = pd
+	ed.L0.Index = i
+
+	var numValues int
+	for b := b; len(b) > 0; {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumDescriptorProto_Name_field_number:
+				ed.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.EnumDescriptorProto_Value_field_number:
+				numValues++
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+
+	// Only construct enum value descriptors for top-level enums since
+	// they are needed for registration.
+	if pd != pf {
+		return
+	}
+	ed.L1.eagerValues = true
+	ed.L2 = new(EnumL2)
+	ed.L2.Values.List = make([]EnumValue, numValues)
+	for i := 0; len(b) > 0; {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumDescriptorProto_Value_field_number:
+				ed.L2.Values.List[i].unmarshalFull(v, sb, pf, ed, i)
+				i++
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func (md *Message) unmarshalSeed(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	md.L0.ParentFile = pf
+	md.L0.Parent = pd
+	md.L0.Index = i
+
+	var prevField pref.FieldNumber
+	var numEnums, numMessages, numExtensions int
+	var posEnums, posMessages, posExtensions int
+	b0 := b
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.DescriptorProto_Name_field_number:
+				md.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.DescriptorProto_EnumType_field_number:
+				if prevField != genid.DescriptorProto_EnumType_field_number {
+					if numEnums > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posEnums = len(b0) - len(b) - n - m
+				}
+				numEnums++
+			case genid.DescriptorProto_NestedType_field_number:
+				if prevField != genid.DescriptorProto_NestedType_field_number {
+					if numMessages > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posMessages = len(b0) - len(b) - n - m
+				}
+				numMessages++
+			case genid.DescriptorProto_Extension_field_number:
+				if prevField != genid.DescriptorProto_Extension_field_number {
+					if numExtensions > 0 {
+						panic("non-contiguous repeated field")
+					}
+					posExtensions = len(b0) - len(b) - n - m
+				}
+				numExtensions++
+			case genid.DescriptorProto_Options_field_number:
+				md.unmarshalSeedOptions(v)
+			}
+			prevField = num
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+			prevField = -1 // ignore known field numbers of unknown wire type
+		}
+	}
+
+	// Must allocate all declarations before parsing each descriptor type
+	// to ensure we handled all descriptors in "flattened ordering".
+	if numEnums > 0 {
+		md.L1.Enums.List = pf.allocEnums(numEnums)
+	}
+	if numMessages > 0 {
+		md.L1.Messages.List = pf.allocMessages(numMessages)
+	}
+	if numExtensions > 0 {
+		md.L1.Extensions.List = pf.allocExtensions(numExtensions)
+	}
+
+	if numEnums > 0 {
+		b := b0[posEnums:]
+		for i := range md.L1.Enums.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			md.L1.Enums.List[i].unmarshalSeed(v, sb, pf, md, i)
+			b = b[n+m:]
+		}
+	}
+	if numMessages > 0 {
+		b := b0[posMessages:]
+		for i := range md.L1.Messages.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			md.L1.Messages.List[i].unmarshalSeed(v, sb, pf, md, i)
+			b = b[n+m:]
+		}
+	}
+	if numExtensions > 0 {
+		b := b0[posExtensions:]
+		for i := range md.L1.Extensions.List {
+			_, n := protowire.ConsumeVarint(b)
+			v, m := protowire.ConsumeBytes(b[n:])
+			md.L1.Extensions.List[i].unmarshalSeed(v, sb, pf, md, i)
+			b = b[n+m:]
+		}
+	}
+}
+
+func (md *Message) unmarshalSeedOptions(b []byte) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.MessageOptions_MapEntry_field_number:
+				md.L1.IsMapEntry = protowire.DecodeBool(v)
+			case genid.MessageOptions_MessageSetWireFormat_field_number:
+				md.L1.IsMessageSet = protowire.DecodeBool(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func (xd *Extension) unmarshalSeed(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	xd.L0.ParentFile = pf
+	xd.L0.Parent = pd
+	xd.L0.Index = i
+
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_Number_field_number:
+				xd.L1.Number = pref.FieldNumber(v)
+			case genid.FieldDescriptorProto_Label_field_number:
+				xd.L1.Cardinality = pref.Cardinality(v)
+			case genid.FieldDescriptorProto_Type_field_number:
+				xd.L1.Kind = pref.Kind(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_Name_field_number:
+				xd.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.FieldDescriptorProto_Extendee_field_number:
+				xd.L1.Extendee = PlaceholderMessage(makeFullName(sb, v))
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func (sd *Service) unmarshalSeed(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	sd.L0.ParentFile = pf
+	sd.L0.Parent = pd
+	sd.L0.Index = i
+
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.ServiceDescriptorProto_Name_field_number:
+				sd.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+var nameBuilderPool = sync.Pool{
+	New: func() interface{} { return new(strs.Builder) },
+}
+
+func getBuilder() *strs.Builder {
+	return nameBuilderPool.Get().(*strs.Builder)
+}
+func putBuilder(b *strs.Builder) {
+	nameBuilderPool.Put(b)
+}
+
+// makeFullName converts b to a protoreflect.FullName,
+// where b must start with a leading dot.
+func makeFullName(sb *strs.Builder, b []byte) pref.FullName {
+	if len(b) == 0 || b[0] != '.' {
+		panic("name reference must be fully qualified")
+	}
+	return pref.FullName(sb.MakeString(b[1:]))
+}
+
+func appendFullName(sb *strs.Builder, prefix pref.FullName, suffix []byte) pref.FullName {
+	return sb.AppendFullName(prefix, pref.Name(strs.UnsafeString(suffix)))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/desc_lazy.go b/vendor/google.golang.org/protobuf/internal/filedesc/desc_lazy.go
new file mode 100644
index 0000000..198451e
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/desc_lazy.go
@@ -0,0 +1,704 @@
+// 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.
+
+package filedesc
+
+import (
+	"reflect"
+	"sync"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/descopts"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+func (fd *File) lazyRawInit() {
+	fd.unmarshalFull(fd.builder.RawDescriptor)
+	fd.resolveMessages()
+	fd.resolveExtensions()
+	fd.resolveServices()
+}
+
+func (file *File) resolveMessages() {
+	var depIdx int32
+	for i := range file.allMessages {
+		md := &file.allMessages[i]
+
+		// Resolve message field dependencies.
+		for j := range md.L2.Fields.List {
+			fd := &md.L2.Fields.List[j]
+
+			// Weak fields are resolved upon actual use.
+			if fd.L1.IsWeak {
+				continue
+			}
+
+			// Resolve message field dependency.
+			switch fd.L1.Kind {
+			case pref.EnumKind:
+				fd.L1.Enum = file.resolveEnumDependency(fd.L1.Enum, listFieldDeps, depIdx)
+				depIdx++
+			case pref.MessageKind, pref.GroupKind:
+				fd.L1.Message = file.resolveMessageDependency(fd.L1.Message, listFieldDeps, depIdx)
+				depIdx++
+			}
+
+			// Default is resolved here since it depends on Enum being resolved.
+			if v := fd.L1.Default.val; v.IsValid() {
+				fd.L1.Default = unmarshalDefault(v.Bytes(), fd.L1.Kind, file, fd.L1.Enum)
+			}
+		}
+	}
+}
+
+func (file *File) resolveExtensions() {
+	var depIdx int32
+	for i := range file.allExtensions {
+		xd := &file.allExtensions[i]
+
+		// Resolve extension field dependency.
+		switch xd.L1.Kind {
+		case pref.EnumKind:
+			xd.L2.Enum = file.resolveEnumDependency(xd.L2.Enum, listExtDeps, depIdx)
+			depIdx++
+		case pref.MessageKind, pref.GroupKind:
+			xd.L2.Message = file.resolveMessageDependency(xd.L2.Message, listExtDeps, depIdx)
+			depIdx++
+		}
+
+		// Default is resolved here since it depends on Enum being resolved.
+		if v := xd.L2.Default.val; v.IsValid() {
+			xd.L2.Default = unmarshalDefault(v.Bytes(), xd.L1.Kind, file, xd.L2.Enum)
+		}
+	}
+}
+
+func (file *File) resolveServices() {
+	var depIdx int32
+	for i := range file.allServices {
+		sd := &file.allServices[i]
+
+		// Resolve method dependencies.
+		for j := range sd.L2.Methods.List {
+			md := &sd.L2.Methods.List[j]
+			md.L1.Input = file.resolveMessageDependency(md.L1.Input, listMethInDeps, depIdx)
+			md.L1.Output = file.resolveMessageDependency(md.L1.Output, listMethOutDeps, depIdx)
+			depIdx++
+		}
+	}
+}
+
+func (file *File) resolveEnumDependency(ed pref.EnumDescriptor, i, j int32) pref.EnumDescriptor {
+	r := file.builder.FileRegistry
+	if r, ok := r.(resolverByIndex); ok {
+		if ed2 := r.FindEnumByIndex(i, j, file.allEnums, file.allMessages); ed2 != nil {
+			return ed2
+		}
+	}
+	for i := range file.allEnums {
+		if ed2 := &file.allEnums[i]; ed2.L0.FullName == ed.FullName() {
+			return ed2
+		}
+	}
+	if d, _ := r.FindDescriptorByName(ed.FullName()); d != nil {
+		return d.(pref.EnumDescriptor)
+	}
+	return ed
+}
+
+func (file *File) resolveMessageDependency(md pref.MessageDescriptor, i, j int32) pref.MessageDescriptor {
+	r := file.builder.FileRegistry
+	if r, ok := r.(resolverByIndex); ok {
+		if md2 := r.FindMessageByIndex(i, j, file.allEnums, file.allMessages); md2 != nil {
+			return md2
+		}
+	}
+	for i := range file.allMessages {
+		if md2 := &file.allMessages[i]; md2.L0.FullName == md.FullName() {
+			return md2
+		}
+	}
+	if d, _ := r.FindDescriptorByName(md.FullName()); d != nil {
+		return d.(pref.MessageDescriptor)
+	}
+	return md
+}
+
+func (fd *File) unmarshalFull(b []byte) {
+	sb := getBuilder()
+	defer putBuilder(sb)
+
+	var enumIdx, messageIdx, extensionIdx, serviceIdx int
+	var rawOptions []byte
+	fd.L2 = new(FileL2)
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FileDescriptorProto_PublicDependency_field_number:
+				fd.L2.Imports[v].IsPublic = true
+			case genid.FileDescriptorProto_WeakDependency_field_number:
+				fd.L2.Imports[v].IsWeak = true
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.FileDescriptorProto_Dependency_field_number:
+				path := sb.MakeString(v)
+				imp, _ := fd.builder.FileRegistry.FindFileByPath(path)
+				if imp == nil {
+					imp = PlaceholderFile(path)
+				}
+				fd.L2.Imports = append(fd.L2.Imports, pref.FileImport{FileDescriptor: imp})
+			case genid.FileDescriptorProto_EnumType_field_number:
+				fd.L1.Enums.List[enumIdx].unmarshalFull(v, sb)
+				enumIdx++
+			case genid.FileDescriptorProto_MessageType_field_number:
+				fd.L1.Messages.List[messageIdx].unmarshalFull(v, sb)
+				messageIdx++
+			case genid.FileDescriptorProto_Extension_field_number:
+				fd.L1.Extensions.List[extensionIdx].unmarshalFull(v, sb)
+				extensionIdx++
+			case genid.FileDescriptorProto_Service_field_number:
+				fd.L1.Services.List[serviceIdx].unmarshalFull(v, sb)
+				serviceIdx++
+			case genid.FileDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	fd.L2.Options = fd.builder.optionsUnmarshaler(&descopts.File, rawOptions)
+}
+
+func (ed *Enum) unmarshalFull(b []byte, sb *strs.Builder) {
+	var rawValues [][]byte
+	var rawOptions []byte
+	if !ed.L1.eagerValues {
+		ed.L2 = new(EnumL2)
+	}
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumDescriptorProto_Value_field_number:
+				rawValues = append(rawValues, v)
+			case genid.EnumDescriptorProto_ReservedName_field_number:
+				ed.L2.ReservedNames.List = append(ed.L2.ReservedNames.List, pref.Name(sb.MakeString(v)))
+			case genid.EnumDescriptorProto_ReservedRange_field_number:
+				ed.L2.ReservedRanges.List = append(ed.L2.ReservedRanges.List, unmarshalEnumReservedRange(v))
+			case genid.EnumDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	if !ed.L1.eagerValues && len(rawValues) > 0 {
+		ed.L2.Values.List = make([]EnumValue, len(rawValues))
+		for i, b := range rawValues {
+			ed.L2.Values.List[i].unmarshalFull(b, sb, ed.L0.ParentFile, ed, i)
+		}
+	}
+	ed.L2.Options = ed.L0.ParentFile.builder.optionsUnmarshaler(&descopts.Enum, rawOptions)
+}
+
+func unmarshalEnumReservedRange(b []byte) (r [2]pref.EnumNumber) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumDescriptorProto_EnumReservedRange_Start_field_number:
+				r[0] = pref.EnumNumber(v)
+			case genid.EnumDescriptorProto_EnumReservedRange_End_field_number:
+				r[1] = pref.EnumNumber(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	return r
+}
+
+func (vd *EnumValue) unmarshalFull(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	vd.L0.ParentFile = pf
+	vd.L0.Parent = pd
+	vd.L0.Index = i
+
+	var rawOptions []byte
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumValueDescriptorProto_Number_field_number:
+				vd.L1.Number = pref.EnumNumber(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.EnumValueDescriptorProto_Name_field_number:
+				// NOTE: Enum values are in the same scope as the enum parent.
+				vd.L0.FullName = appendFullName(sb, pd.Parent().FullName(), v)
+			case genid.EnumValueDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	vd.L1.Options = pf.builder.optionsUnmarshaler(&descopts.EnumValue, rawOptions)
+}
+
+func (md *Message) unmarshalFull(b []byte, sb *strs.Builder) {
+	var rawFields, rawOneofs [][]byte
+	var enumIdx, messageIdx, extensionIdx int
+	var rawOptions []byte
+	md.L2 = new(MessageL2)
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.DescriptorProto_Field_field_number:
+				rawFields = append(rawFields, v)
+			case genid.DescriptorProto_OneofDecl_field_number:
+				rawOneofs = append(rawOneofs, v)
+			case genid.DescriptorProto_ReservedName_field_number:
+				md.L2.ReservedNames.List = append(md.L2.ReservedNames.List, pref.Name(sb.MakeString(v)))
+			case genid.DescriptorProto_ReservedRange_field_number:
+				md.L2.ReservedRanges.List = append(md.L2.ReservedRanges.List, unmarshalMessageReservedRange(v))
+			case genid.DescriptorProto_ExtensionRange_field_number:
+				r, rawOptions := unmarshalMessageExtensionRange(v)
+				opts := md.L0.ParentFile.builder.optionsUnmarshaler(&descopts.ExtensionRange, rawOptions)
+				md.L2.ExtensionRanges.List = append(md.L2.ExtensionRanges.List, r)
+				md.L2.ExtensionRangeOptions = append(md.L2.ExtensionRangeOptions, opts)
+			case genid.DescriptorProto_EnumType_field_number:
+				md.L1.Enums.List[enumIdx].unmarshalFull(v, sb)
+				enumIdx++
+			case genid.DescriptorProto_NestedType_field_number:
+				md.L1.Messages.List[messageIdx].unmarshalFull(v, sb)
+				messageIdx++
+			case genid.DescriptorProto_Extension_field_number:
+				md.L1.Extensions.List[extensionIdx].unmarshalFull(v, sb)
+				extensionIdx++
+			case genid.DescriptorProto_Options_field_number:
+				md.unmarshalOptions(v)
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	if len(rawFields) > 0 || len(rawOneofs) > 0 {
+		md.L2.Fields.List = make([]Field, len(rawFields))
+		md.L2.Oneofs.List = make([]Oneof, len(rawOneofs))
+		for i, b := range rawFields {
+			fd := &md.L2.Fields.List[i]
+			fd.unmarshalFull(b, sb, md.L0.ParentFile, md, i)
+			if fd.L1.Cardinality == pref.Required {
+				md.L2.RequiredNumbers.List = append(md.L2.RequiredNumbers.List, fd.L1.Number)
+			}
+		}
+		for i, b := range rawOneofs {
+			od := &md.L2.Oneofs.List[i]
+			od.unmarshalFull(b, sb, md.L0.ParentFile, md, i)
+		}
+	}
+	md.L2.Options = md.L0.ParentFile.builder.optionsUnmarshaler(&descopts.Message, rawOptions)
+}
+
+func (md *Message) unmarshalOptions(b []byte) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.MessageOptions_MapEntry_field_number:
+				md.L1.IsMapEntry = protowire.DecodeBool(v)
+			case genid.MessageOptions_MessageSetWireFormat_field_number:
+				md.L1.IsMessageSet = protowire.DecodeBool(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func unmarshalMessageReservedRange(b []byte) (r [2]pref.FieldNumber) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.DescriptorProto_ReservedRange_Start_field_number:
+				r[0] = pref.FieldNumber(v)
+			case genid.DescriptorProto_ReservedRange_End_field_number:
+				r[1] = pref.FieldNumber(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	return r
+}
+
+func unmarshalMessageExtensionRange(b []byte) (r [2]pref.FieldNumber, rawOptions []byte) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.DescriptorProto_ExtensionRange_Start_field_number:
+				r[0] = pref.FieldNumber(v)
+			case genid.DescriptorProto_ExtensionRange_End_field_number:
+				r[1] = pref.FieldNumber(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.DescriptorProto_ExtensionRange_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	return r, rawOptions
+}
+
+func (fd *Field) unmarshalFull(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	fd.L0.ParentFile = pf
+	fd.L0.Parent = pd
+	fd.L0.Index = i
+
+	var rawTypeName []byte
+	var rawOptions []byte
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_Number_field_number:
+				fd.L1.Number = pref.FieldNumber(v)
+			case genid.FieldDescriptorProto_Label_field_number:
+				fd.L1.Cardinality = pref.Cardinality(v)
+			case genid.FieldDescriptorProto_Type_field_number:
+				fd.L1.Kind = pref.Kind(v)
+			case genid.FieldDescriptorProto_OneofIndex_field_number:
+				// In Message.unmarshalFull, we allocate slices for both
+				// the field and oneof descriptors before unmarshaling either
+				// of them. This ensures pointers to slice elements are stable.
+				od := &pd.(*Message).L2.Oneofs.List[v]
+				od.L1.Fields.List = append(od.L1.Fields.List, fd)
+				if fd.L1.ContainingOneof != nil {
+					panic("oneof type already set")
+				}
+				fd.L1.ContainingOneof = od
+			case genid.FieldDescriptorProto_Proto3Optional_field_number:
+				fd.L1.IsProto3Optional = protowire.DecodeBool(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_Name_field_number:
+				fd.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.FieldDescriptorProto_JsonName_field_number:
+				fd.L1.StringName.InitJSON(sb.MakeString(v))
+			case genid.FieldDescriptorProto_DefaultValue_field_number:
+				fd.L1.Default.val = pref.ValueOfBytes(v) // temporarily store as bytes; later resolved in resolveMessages
+			case genid.FieldDescriptorProto_TypeName_field_number:
+				rawTypeName = v
+			case genid.FieldDescriptorProto_Options_field_number:
+				fd.unmarshalOptions(v)
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	if rawTypeName != nil {
+		name := makeFullName(sb, rawTypeName)
+		switch fd.L1.Kind {
+		case pref.EnumKind:
+			fd.L1.Enum = PlaceholderEnum(name)
+		case pref.MessageKind, pref.GroupKind:
+			fd.L1.Message = PlaceholderMessage(name)
+		}
+	}
+	fd.L1.Options = pf.builder.optionsUnmarshaler(&descopts.Field, rawOptions)
+}
+
+func (fd *Field) unmarshalOptions(b []byte) {
+	const FieldOptions_EnforceUTF8 = 13
+
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldOptions_Packed_field_number:
+				fd.L1.HasPacked = true
+				fd.L1.IsPacked = protowire.DecodeBool(v)
+			case genid.FieldOptions_Weak_field_number:
+				fd.L1.IsWeak = protowire.DecodeBool(v)
+			case FieldOptions_EnforceUTF8:
+				fd.L1.HasEnforceUTF8 = true
+				fd.L1.EnforceUTF8 = protowire.DecodeBool(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func (od *Oneof) unmarshalFull(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	od.L0.ParentFile = pf
+	od.L0.Parent = pd
+	od.L0.Index = i
+
+	var rawOptions []byte
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.OneofDescriptorProto_Name_field_number:
+				od.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.OneofDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	od.L1.Options = pf.builder.optionsUnmarshaler(&descopts.Oneof, rawOptions)
+}
+
+func (xd *Extension) unmarshalFull(b []byte, sb *strs.Builder) {
+	var rawTypeName []byte
+	var rawOptions []byte
+	xd.L2 = new(ExtensionL2)
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_Proto3Optional_field_number:
+				xd.L2.IsProto3Optional = protowire.DecodeBool(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldDescriptorProto_JsonName_field_number:
+				xd.L2.StringName.InitJSON(sb.MakeString(v))
+			case genid.FieldDescriptorProto_DefaultValue_field_number:
+				xd.L2.Default.val = pref.ValueOfBytes(v) // temporarily store as bytes; later resolved in resolveExtensions
+			case genid.FieldDescriptorProto_TypeName_field_number:
+				rawTypeName = v
+			case genid.FieldDescriptorProto_Options_field_number:
+				xd.unmarshalOptions(v)
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	if rawTypeName != nil {
+		name := makeFullName(sb, rawTypeName)
+		switch xd.L1.Kind {
+		case pref.EnumKind:
+			xd.L2.Enum = PlaceholderEnum(name)
+		case pref.MessageKind, pref.GroupKind:
+			xd.L2.Message = PlaceholderMessage(name)
+		}
+	}
+	xd.L2.Options = xd.L0.ParentFile.builder.optionsUnmarshaler(&descopts.Field, rawOptions)
+}
+
+func (xd *Extension) unmarshalOptions(b []byte) {
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.FieldOptions_Packed_field_number:
+				xd.L2.IsPacked = protowire.DecodeBool(v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+}
+
+func (sd *Service) unmarshalFull(b []byte, sb *strs.Builder) {
+	var rawMethods [][]byte
+	var rawOptions []byte
+	sd.L2 = new(ServiceL2)
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.ServiceDescriptorProto_Method_field_number:
+				rawMethods = append(rawMethods, v)
+			case genid.ServiceDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	if len(rawMethods) > 0 {
+		sd.L2.Methods.List = make([]Method, len(rawMethods))
+		for i, b := range rawMethods {
+			sd.L2.Methods.List[i].unmarshalFull(b, sb, sd.L0.ParentFile, sd, i)
+		}
+	}
+	sd.L2.Options = sd.L0.ParentFile.builder.optionsUnmarshaler(&descopts.Service, rawOptions)
+}
+
+func (md *Method) unmarshalFull(b []byte, sb *strs.Builder, pf *File, pd pref.Descriptor, i int) {
+	md.L0.ParentFile = pf
+	md.L0.Parent = pd
+	md.L0.Index = i
+
+	var rawOptions []byte
+	for len(b) > 0 {
+		num, typ, n := protowire.ConsumeTag(b)
+		b = b[n:]
+		switch typ {
+		case protowire.VarintType:
+			v, m := protowire.ConsumeVarint(b)
+			b = b[m:]
+			switch num {
+			case genid.MethodDescriptorProto_ClientStreaming_field_number:
+				md.L1.IsStreamingClient = protowire.DecodeBool(v)
+			case genid.MethodDescriptorProto_ServerStreaming_field_number:
+				md.L1.IsStreamingServer = protowire.DecodeBool(v)
+			}
+		case protowire.BytesType:
+			v, m := protowire.ConsumeBytes(b)
+			b = b[m:]
+			switch num {
+			case genid.MethodDescriptorProto_Name_field_number:
+				md.L0.FullName = appendFullName(sb, pd.FullName(), v)
+			case genid.MethodDescriptorProto_InputType_field_number:
+				md.L1.Input = PlaceholderMessage(makeFullName(sb, v))
+			case genid.MethodDescriptorProto_OutputType_field_number:
+				md.L1.Output = PlaceholderMessage(makeFullName(sb, v))
+			case genid.MethodDescriptorProto_Options_field_number:
+				rawOptions = appendOptions(rawOptions, v)
+			}
+		default:
+			m := protowire.ConsumeFieldValue(num, typ, b)
+			b = b[m:]
+		}
+	}
+	md.L1.Options = pf.builder.optionsUnmarshaler(&descopts.Method, rawOptions)
+}
+
+// appendOptions appends src to dst, where the returned slice is never nil.
+// This is necessary to distinguish between empty and unpopulated options.
+func appendOptions(dst, src []byte) []byte {
+	if dst == nil {
+		dst = []byte{}
+	}
+	return append(dst, src...)
+}
+
+// optionsUnmarshaler constructs a lazy unmarshal function for an options message.
+//
+// The type of message to unmarshal to is passed as a pointer since the
+// vars in descopts may not yet be populated at the time this function is called.
+func (db *Builder) optionsUnmarshaler(p *pref.ProtoMessage, b []byte) func() pref.ProtoMessage {
+	if b == nil {
+		return nil
+	}
+	var opts pref.ProtoMessage
+	var once sync.Once
+	return func() pref.ProtoMessage {
+		once.Do(func() {
+			if *p == nil {
+				panic("Descriptor.Options called without importing the descriptor package")
+			}
+			opts = reflect.New(reflect.TypeOf(*p).Elem()).Interface().(pref.ProtoMessage)
+			if err := (proto.UnmarshalOptions{
+				AllowPartial: true,
+				Resolver:     db.TypeResolver,
+			}).Unmarshal(b, opts); err != nil {
+				panic(err)
+			}
+		})
+		return opts
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/desc_list.go b/vendor/google.golang.org/protobuf/internal/filedesc/desc_list.go
new file mode 100644
index 0000000..aa294ff
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/desc_list.go
@@ -0,0 +1,450 @@
+// 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.
+
+package filedesc
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"sync"
+
+	"google.golang.org/protobuf/internal/genid"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/descfmt"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type FileImports []pref.FileImport
+
+func (p *FileImports) Len() int                            { return len(*p) }
+func (p *FileImports) Get(i int) pref.FileImport           { return (*p)[i] }
+func (p *FileImports) Format(s fmt.State, r rune)          { descfmt.FormatList(s, r, p) }
+func (p *FileImports) ProtoInternal(pragma.DoNotImplement) {}
+
+type Names struct {
+	List []pref.Name
+	once sync.Once
+	has  map[pref.Name]int // protected by once
+}
+
+func (p *Names) Len() int                            { return len(p.List) }
+func (p *Names) Get(i int) pref.Name                 { return p.List[i] }
+func (p *Names) Has(s pref.Name) bool                { return p.lazyInit().has[s] > 0 }
+func (p *Names) Format(s fmt.State, r rune)          { descfmt.FormatList(s, r, p) }
+func (p *Names) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Names) lazyInit() *Names {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.has = make(map[pref.Name]int, len(p.List))
+			for _, s := range p.List {
+				p.has[s] = p.has[s] + 1
+			}
+		}
+	})
+	return p
+}
+
+// CheckValid reports any errors with the set of names with an error message
+// that completes the sentence: "ranges is invalid because it has ..."
+func (p *Names) CheckValid() error {
+	for s, n := range p.lazyInit().has {
+		switch {
+		case n > 1:
+			return errors.New("duplicate name: %q", s)
+		case false && !s.IsValid():
+			// NOTE: The C++ implementation does not validate the identifier.
+			// See https://github.com/protocolbuffers/protobuf/issues/6335.
+			return errors.New("invalid name: %q", s)
+		}
+	}
+	return nil
+}
+
+type EnumRanges struct {
+	List   [][2]pref.EnumNumber // start inclusive; end inclusive
+	once   sync.Once
+	sorted [][2]pref.EnumNumber // protected by once
+}
+
+func (p *EnumRanges) Len() int                     { return len(p.List) }
+func (p *EnumRanges) Get(i int) [2]pref.EnumNumber { return p.List[i] }
+func (p *EnumRanges) Has(n pref.EnumNumber) bool {
+	for ls := p.lazyInit().sorted; len(ls) > 0; {
+		i := len(ls) / 2
+		switch r := enumRange(ls[i]); {
+		case n < r.Start():
+			ls = ls[:i] // search lower
+		case n > r.End():
+			ls = ls[i+1:] // search upper
+		default:
+			return true
+		}
+	}
+	return false
+}
+func (p *EnumRanges) Format(s fmt.State, r rune)          { descfmt.FormatList(s, r, p) }
+func (p *EnumRanges) ProtoInternal(pragma.DoNotImplement) {}
+func (p *EnumRanges) lazyInit() *EnumRanges {
+	p.once.Do(func() {
+		p.sorted = append(p.sorted, p.List...)
+		sort.Slice(p.sorted, func(i, j int) bool {
+			return p.sorted[i][0] < p.sorted[j][0]
+		})
+	})
+	return p
+}
+
+// CheckValid reports any errors with the set of names with an error message
+// that completes the sentence: "ranges is invalid because it has ..."
+func (p *EnumRanges) CheckValid() error {
+	var rp enumRange
+	for i, r := range p.lazyInit().sorted {
+		r := enumRange(r)
+		switch {
+		case !(r.Start() <= r.End()):
+			return errors.New("invalid range: %v", r)
+		case !(rp.End() < r.Start()) && i > 0:
+			return errors.New("overlapping ranges: %v with %v", rp, r)
+		}
+		rp = r
+	}
+	return nil
+}
+
+type enumRange [2]protoreflect.EnumNumber
+
+func (r enumRange) Start() protoreflect.EnumNumber { return r[0] } // inclusive
+func (r enumRange) End() protoreflect.EnumNumber   { return r[1] } // inclusive
+func (r enumRange) String() string {
+	if r.Start() == r.End() {
+		return fmt.Sprintf("%d", r.Start())
+	}
+	return fmt.Sprintf("%d to %d", r.Start(), r.End())
+}
+
+type FieldRanges struct {
+	List   [][2]pref.FieldNumber // start inclusive; end exclusive
+	once   sync.Once
+	sorted [][2]pref.FieldNumber // protected by once
+}
+
+func (p *FieldRanges) Len() int                      { return len(p.List) }
+func (p *FieldRanges) Get(i int) [2]pref.FieldNumber { return p.List[i] }
+func (p *FieldRanges) Has(n pref.FieldNumber) bool {
+	for ls := p.lazyInit().sorted; len(ls) > 0; {
+		i := len(ls) / 2
+		switch r := fieldRange(ls[i]); {
+		case n < r.Start():
+			ls = ls[:i] // search lower
+		case n > r.End():
+			ls = ls[i+1:] // search upper
+		default:
+			return true
+		}
+	}
+	return false
+}
+func (p *FieldRanges) Format(s fmt.State, r rune)          { descfmt.FormatList(s, r, p) }
+func (p *FieldRanges) ProtoInternal(pragma.DoNotImplement) {}
+func (p *FieldRanges) lazyInit() *FieldRanges {
+	p.once.Do(func() {
+		p.sorted = append(p.sorted, p.List...)
+		sort.Slice(p.sorted, func(i, j int) bool {
+			return p.sorted[i][0] < p.sorted[j][0]
+		})
+	})
+	return p
+}
+
+// CheckValid reports any errors with the set of ranges with an error message
+// that completes the sentence: "ranges is invalid because it has ..."
+func (p *FieldRanges) CheckValid(isMessageSet bool) error {
+	var rp fieldRange
+	for i, r := range p.lazyInit().sorted {
+		r := fieldRange(r)
+		switch {
+		case !isValidFieldNumber(r.Start(), isMessageSet):
+			return errors.New("invalid field number: %d", r.Start())
+		case !isValidFieldNumber(r.End(), isMessageSet):
+			return errors.New("invalid field number: %d", r.End())
+		case !(r.Start() <= r.End()):
+			return errors.New("invalid range: %v", r)
+		case !(rp.End() < r.Start()) && i > 0:
+			return errors.New("overlapping ranges: %v with %v", rp, r)
+		}
+		rp = r
+	}
+	return nil
+}
+
+// isValidFieldNumber reports whether the field number is valid.
+// Unlike the FieldNumber.IsValid method, it allows ranges that cover the
+// reserved number range.
+func isValidFieldNumber(n protoreflect.FieldNumber, isMessageSet bool) bool {
+	return protowire.MinValidNumber <= n && (n <= protowire.MaxValidNumber || isMessageSet)
+}
+
+// CheckOverlap reports an error if p and q overlap.
+func (p *FieldRanges) CheckOverlap(q *FieldRanges) error {
+	rps := p.lazyInit().sorted
+	rqs := q.lazyInit().sorted
+	for pi, qi := 0, 0; pi < len(rps) && qi < len(rqs); {
+		rp := fieldRange(rps[pi])
+		rq := fieldRange(rqs[qi])
+		if !(rp.End() < rq.Start() || rq.End() < rp.Start()) {
+			return errors.New("overlapping ranges: %v with %v", rp, rq)
+		}
+		if rp.Start() < rq.Start() {
+			pi++
+		} else {
+			qi++
+		}
+	}
+	return nil
+}
+
+type fieldRange [2]protoreflect.FieldNumber
+
+func (r fieldRange) Start() protoreflect.FieldNumber { return r[0] }     // inclusive
+func (r fieldRange) End() protoreflect.FieldNumber   { return r[1] - 1 } // inclusive
+func (r fieldRange) String() string {
+	if r.Start() == r.End() {
+		return fmt.Sprintf("%d", r.Start())
+	}
+	return fmt.Sprintf("%d to %d", r.Start(), r.End())
+}
+
+type FieldNumbers struct {
+	List []pref.FieldNumber
+	once sync.Once
+	has  map[pref.FieldNumber]struct{} // protected by once
+}
+
+func (p *FieldNumbers) Len() int                   { return len(p.List) }
+func (p *FieldNumbers) Get(i int) pref.FieldNumber { return p.List[i] }
+func (p *FieldNumbers) Has(n pref.FieldNumber) bool {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.has = make(map[pref.FieldNumber]struct{}, len(p.List))
+			for _, n := range p.List {
+				p.has[n] = struct{}{}
+			}
+		}
+	})
+	_, ok := p.has[n]
+	return ok
+}
+func (p *FieldNumbers) Format(s fmt.State, r rune)          { descfmt.FormatList(s, r, p) }
+func (p *FieldNumbers) ProtoInternal(pragma.DoNotImplement) {}
+
+type OneofFields struct {
+	List   []pref.FieldDescriptor
+	once   sync.Once
+	byName map[pref.Name]pref.FieldDescriptor        // protected by once
+	byJSON map[string]pref.FieldDescriptor           // protected by once
+	byText map[string]pref.FieldDescriptor           // protected by once
+	byNum  map[pref.FieldNumber]pref.FieldDescriptor // protected by once
+}
+
+func (p *OneofFields) Len() int                                         { return len(p.List) }
+func (p *OneofFields) Get(i int) pref.FieldDescriptor                   { return p.List[i] }
+func (p *OneofFields) ByName(s pref.Name) pref.FieldDescriptor          { return p.lazyInit().byName[s] }
+func (p *OneofFields) ByJSONName(s string) pref.FieldDescriptor         { return p.lazyInit().byJSON[s] }
+func (p *OneofFields) ByTextName(s string) pref.FieldDescriptor         { return p.lazyInit().byText[s] }
+func (p *OneofFields) ByNumber(n pref.FieldNumber) pref.FieldDescriptor { return p.lazyInit().byNum[n] }
+func (p *OneofFields) Format(s fmt.State, r rune)                       { descfmt.FormatList(s, r, p) }
+func (p *OneofFields) ProtoInternal(pragma.DoNotImplement)              {}
+
+func (p *OneofFields) lazyInit() *OneofFields {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[pref.Name]pref.FieldDescriptor, len(p.List))
+			p.byJSON = make(map[string]pref.FieldDescriptor, len(p.List))
+			p.byText = make(map[string]pref.FieldDescriptor, len(p.List))
+			p.byNum = make(map[pref.FieldNumber]pref.FieldDescriptor, len(p.List))
+			for _, f := range p.List {
+				// Field names and numbers are guaranteed to be unique.
+				p.byName[f.Name()] = f
+				p.byJSON[f.JSONName()] = f
+				p.byText[f.TextName()] = f
+				p.byNum[f.Number()] = f
+			}
+		}
+	})
+	return p
+}
+
+type SourceLocations struct {
+	// List is a list of SourceLocations.
+	// The SourceLocation.Next field does not need to be populated
+	// as it will be lazily populated upon first need.
+	List []pref.SourceLocation
+
+	// File is the parent file descriptor that these locations are relative to.
+	// If non-nil, ByDescriptor verifies that the provided descriptor
+	// is a child of this file descriptor.
+	File pref.FileDescriptor
+
+	once   sync.Once
+	byPath map[pathKey]int
+}
+
+func (p *SourceLocations) Len() int                      { return len(p.List) }
+func (p *SourceLocations) Get(i int) pref.SourceLocation { return p.lazyInit().List[i] }
+func (p *SourceLocations) byKey(k pathKey) pref.SourceLocation {
+	if i, ok := p.lazyInit().byPath[k]; ok {
+		return p.List[i]
+	}
+	return pref.SourceLocation{}
+}
+func (p *SourceLocations) ByPath(path pref.SourcePath) pref.SourceLocation {
+	return p.byKey(newPathKey(path))
+}
+func (p *SourceLocations) ByDescriptor(desc pref.Descriptor) pref.SourceLocation {
+	if p.File != nil && desc != nil && p.File != desc.ParentFile() {
+		return pref.SourceLocation{} // mismatching parent files
+	}
+	var pathArr [16]int32
+	path := pathArr[:0]
+	for {
+		switch desc.(type) {
+		case pref.FileDescriptor:
+			// Reverse the path since it was constructed in reverse.
+			for i, j := 0, len(path)-1; i < j; i, j = i+1, j-1 {
+				path[i], path[j] = path[j], path[i]
+			}
+			return p.byKey(newPathKey(path))
+		case pref.MessageDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.FileDescriptor:
+				path = append(path, int32(genid.FileDescriptorProto_MessageType_field_number))
+			case pref.MessageDescriptor:
+				path = append(path, int32(genid.DescriptorProto_NestedType_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		case pref.FieldDescriptor:
+			isExtension := desc.(pref.FieldDescriptor).IsExtension()
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			if isExtension {
+				switch desc.(type) {
+				case pref.FileDescriptor:
+					path = append(path, int32(genid.FileDescriptorProto_Extension_field_number))
+				case pref.MessageDescriptor:
+					path = append(path, int32(genid.DescriptorProto_Extension_field_number))
+				default:
+					return pref.SourceLocation{}
+				}
+			} else {
+				switch desc.(type) {
+				case pref.MessageDescriptor:
+					path = append(path, int32(genid.DescriptorProto_Field_field_number))
+				default:
+					return pref.SourceLocation{}
+				}
+			}
+		case pref.OneofDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.MessageDescriptor:
+				path = append(path, int32(genid.DescriptorProto_OneofDecl_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		case pref.EnumDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.FileDescriptor:
+				path = append(path, int32(genid.FileDescriptorProto_EnumType_field_number))
+			case pref.MessageDescriptor:
+				path = append(path, int32(genid.DescriptorProto_EnumType_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		case pref.EnumValueDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.EnumDescriptor:
+				path = append(path, int32(genid.EnumDescriptorProto_Value_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		case pref.ServiceDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.FileDescriptor:
+				path = append(path, int32(genid.FileDescriptorProto_Service_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		case pref.MethodDescriptor:
+			path = append(path, int32(desc.Index()))
+			desc = desc.Parent()
+			switch desc.(type) {
+			case pref.ServiceDescriptor:
+				path = append(path, int32(genid.ServiceDescriptorProto_Method_field_number))
+			default:
+				return pref.SourceLocation{}
+			}
+		default:
+			return pref.SourceLocation{}
+		}
+	}
+}
+func (p *SourceLocations) lazyInit() *SourceLocations {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			// Collect all the indexes for a given path.
+			pathIdxs := make(map[pathKey][]int, len(p.List))
+			for i, l := range p.List {
+				k := newPathKey(l.Path)
+				pathIdxs[k] = append(pathIdxs[k], i)
+			}
+
+			// Update the next index for all locations.
+			p.byPath = make(map[pathKey]int, len(p.List))
+			for k, idxs := range pathIdxs {
+				for i := 0; i < len(idxs)-1; i++ {
+					p.List[idxs[i]].Next = idxs[i+1]
+				}
+				p.List[idxs[len(idxs)-1]].Next = 0
+				p.byPath[k] = idxs[0] // record the first location for this path
+			}
+		}
+	})
+	return p
+}
+func (p *SourceLocations) ProtoInternal(pragma.DoNotImplement) {}
+
+// pathKey is a comparable representation of protoreflect.SourcePath.
+type pathKey struct {
+	arr [16]uint8 // first n-1 path segments; last element is the length
+	str string    // used if the path does not fit in arr
+}
+
+func newPathKey(p pref.SourcePath) (k pathKey) {
+	if len(p) < len(k.arr) {
+		for i, ps := range p {
+			if ps < 0 || math.MaxUint8 <= ps {
+				return pathKey{str: p.String()}
+			}
+			k.arr[i] = uint8(ps)
+		}
+		k.arr[len(k.arr)-1] = uint8(len(p))
+		return k
+	}
+	return pathKey{str: p.String()}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/desc_list_gen.go b/vendor/google.golang.org/protobuf/internal/filedesc/desc_list_gen.go
new file mode 100644
index 0000000..30db19f
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/desc_list_gen.go
@@ -0,0 +1,356 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package filedesc
+
+import (
+	"fmt"
+	"sync"
+
+	"google.golang.org/protobuf/internal/descfmt"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type Enums struct {
+	List   []Enum
+	once   sync.Once
+	byName map[protoreflect.Name]*Enum // protected by once
+}
+
+func (p *Enums) Len() int {
+	return len(p.List)
+}
+func (p *Enums) Get(i int) protoreflect.EnumDescriptor {
+	return &p.List[i]
+}
+func (p *Enums) ByName(s protoreflect.Name) protoreflect.EnumDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Enums) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Enums) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Enums) lazyInit() *Enums {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Enum, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type EnumValues struct {
+	List   []EnumValue
+	once   sync.Once
+	byName map[protoreflect.Name]*EnumValue       // protected by once
+	byNum  map[protoreflect.EnumNumber]*EnumValue // protected by once
+}
+
+func (p *EnumValues) Len() int {
+	return len(p.List)
+}
+func (p *EnumValues) Get(i int) protoreflect.EnumValueDescriptor {
+	return &p.List[i]
+}
+func (p *EnumValues) ByName(s protoreflect.Name) protoreflect.EnumValueDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *EnumValues) ByNumber(n protoreflect.EnumNumber) protoreflect.EnumValueDescriptor {
+	if d := p.lazyInit().byNum[n]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *EnumValues) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *EnumValues) ProtoInternal(pragma.DoNotImplement) {}
+func (p *EnumValues) lazyInit() *EnumValues {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*EnumValue, len(p.List))
+			p.byNum = make(map[protoreflect.EnumNumber]*EnumValue, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+				if _, ok := p.byNum[d.Number()]; !ok {
+					p.byNum[d.Number()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Messages struct {
+	List   []Message
+	once   sync.Once
+	byName map[protoreflect.Name]*Message // protected by once
+}
+
+func (p *Messages) Len() int {
+	return len(p.List)
+}
+func (p *Messages) Get(i int) protoreflect.MessageDescriptor {
+	return &p.List[i]
+}
+func (p *Messages) ByName(s protoreflect.Name) protoreflect.MessageDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Messages) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Messages) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Messages) lazyInit() *Messages {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Message, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Fields struct {
+	List   []Field
+	once   sync.Once
+	byName map[protoreflect.Name]*Field        // protected by once
+	byJSON map[string]*Field                   // protected by once
+	byText map[string]*Field                   // protected by once
+	byNum  map[protoreflect.FieldNumber]*Field // protected by once
+}
+
+func (p *Fields) Len() int {
+	return len(p.List)
+}
+func (p *Fields) Get(i int) protoreflect.FieldDescriptor {
+	return &p.List[i]
+}
+func (p *Fields) ByName(s protoreflect.Name) protoreflect.FieldDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Fields) ByJSONName(s string) protoreflect.FieldDescriptor {
+	if d := p.lazyInit().byJSON[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Fields) ByTextName(s string) protoreflect.FieldDescriptor {
+	if d := p.lazyInit().byText[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Fields) ByNumber(n protoreflect.FieldNumber) protoreflect.FieldDescriptor {
+	if d := p.lazyInit().byNum[n]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Fields) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Fields) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Fields) lazyInit() *Fields {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Field, len(p.List))
+			p.byJSON = make(map[string]*Field, len(p.List))
+			p.byText = make(map[string]*Field, len(p.List))
+			p.byNum = make(map[protoreflect.FieldNumber]*Field, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+				if _, ok := p.byJSON[d.JSONName()]; !ok {
+					p.byJSON[d.JSONName()] = d
+				}
+				if _, ok := p.byText[d.TextName()]; !ok {
+					p.byText[d.TextName()] = d
+				}
+				if _, ok := p.byNum[d.Number()]; !ok {
+					p.byNum[d.Number()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Oneofs struct {
+	List   []Oneof
+	once   sync.Once
+	byName map[protoreflect.Name]*Oneof // protected by once
+}
+
+func (p *Oneofs) Len() int {
+	return len(p.List)
+}
+func (p *Oneofs) Get(i int) protoreflect.OneofDescriptor {
+	return &p.List[i]
+}
+func (p *Oneofs) ByName(s protoreflect.Name) protoreflect.OneofDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Oneofs) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Oneofs) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Oneofs) lazyInit() *Oneofs {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Oneof, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Extensions struct {
+	List   []Extension
+	once   sync.Once
+	byName map[protoreflect.Name]*Extension // protected by once
+}
+
+func (p *Extensions) Len() int {
+	return len(p.List)
+}
+func (p *Extensions) Get(i int) protoreflect.ExtensionDescriptor {
+	return &p.List[i]
+}
+func (p *Extensions) ByName(s protoreflect.Name) protoreflect.ExtensionDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Extensions) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Extensions) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Extensions) lazyInit() *Extensions {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Extension, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Services struct {
+	List   []Service
+	once   sync.Once
+	byName map[protoreflect.Name]*Service // protected by once
+}
+
+func (p *Services) Len() int {
+	return len(p.List)
+}
+func (p *Services) Get(i int) protoreflect.ServiceDescriptor {
+	return &p.List[i]
+}
+func (p *Services) ByName(s protoreflect.Name) protoreflect.ServiceDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Services) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Services) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Services) lazyInit() *Services {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Service, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
+
+type Methods struct {
+	List   []Method
+	once   sync.Once
+	byName map[protoreflect.Name]*Method // protected by once
+}
+
+func (p *Methods) Len() int {
+	return len(p.List)
+}
+func (p *Methods) Get(i int) protoreflect.MethodDescriptor {
+	return &p.List[i]
+}
+func (p *Methods) ByName(s protoreflect.Name) protoreflect.MethodDescriptor {
+	if d := p.lazyInit().byName[s]; d != nil {
+		return d
+	}
+	return nil
+}
+func (p *Methods) Format(s fmt.State, r rune) {
+	descfmt.FormatList(s, r, p)
+}
+func (p *Methods) ProtoInternal(pragma.DoNotImplement) {}
+func (p *Methods) lazyInit() *Methods {
+	p.once.Do(func() {
+		if len(p.List) > 0 {
+			p.byName = make(map[protoreflect.Name]*Method, len(p.List))
+			for i := range p.List {
+				d := &p.List[i]
+				if _, ok := p.byName[d.Name()]; !ok {
+					p.byName[d.Name()] = d
+				}
+			}
+		}
+	})
+	return p
+}
diff --git a/vendor/google.golang.org/protobuf/internal/filedesc/placeholder.go b/vendor/google.golang.org/protobuf/internal/filedesc/placeholder.go
new file mode 100644
index 0000000..dbf2c60
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filedesc/placeholder.go
@@ -0,0 +1,107 @@
+// 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.
+
+package filedesc
+
+import (
+	"google.golang.org/protobuf/internal/descopts"
+	"google.golang.org/protobuf/internal/pragma"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+var (
+	emptyNames           = new(Names)
+	emptyEnumRanges      = new(EnumRanges)
+	emptyFieldRanges     = new(FieldRanges)
+	emptyFieldNumbers    = new(FieldNumbers)
+	emptySourceLocations = new(SourceLocations)
+
+	emptyFiles      = new(FileImports)
+	emptyMessages   = new(Messages)
+	emptyFields     = new(Fields)
+	emptyOneofs     = new(Oneofs)
+	emptyEnums      = new(Enums)
+	emptyEnumValues = new(EnumValues)
+	emptyExtensions = new(Extensions)
+	emptyServices   = new(Services)
+)
+
+// PlaceholderFile is a placeholder, representing only the file path.
+type PlaceholderFile string
+
+func (f PlaceholderFile) ParentFile() pref.FileDescriptor       { return f }
+func (f PlaceholderFile) Parent() pref.Descriptor               { return nil }
+func (f PlaceholderFile) Index() int                            { return 0 }
+func (f PlaceholderFile) Syntax() pref.Syntax                   { return 0 }
+func (f PlaceholderFile) Name() pref.Name                       { return "" }
+func (f PlaceholderFile) FullName() pref.FullName               { return "" }
+func (f PlaceholderFile) IsPlaceholder() bool                   { return true }
+func (f PlaceholderFile) Options() pref.ProtoMessage            { return descopts.File }
+func (f PlaceholderFile) Path() string                          { return string(f) }
+func (f PlaceholderFile) Package() pref.FullName                { return "" }
+func (f PlaceholderFile) Imports() pref.FileImports             { return emptyFiles }
+func (f PlaceholderFile) Messages() pref.MessageDescriptors     { return emptyMessages }
+func (f PlaceholderFile) Enums() pref.EnumDescriptors           { return emptyEnums }
+func (f PlaceholderFile) Extensions() pref.ExtensionDescriptors { return emptyExtensions }
+func (f PlaceholderFile) Services() pref.ServiceDescriptors     { return emptyServices }
+func (f PlaceholderFile) SourceLocations() pref.SourceLocations { return emptySourceLocations }
+func (f PlaceholderFile) ProtoType(pref.FileDescriptor)         { return }
+func (f PlaceholderFile) ProtoInternal(pragma.DoNotImplement)   { return }
+
+// PlaceholderEnum is a placeholder, representing only the full name.
+type PlaceholderEnum pref.FullName
+
+func (e PlaceholderEnum) ParentFile() pref.FileDescriptor     { return nil }
+func (e PlaceholderEnum) Parent() pref.Descriptor             { return nil }
+func (e PlaceholderEnum) Index() int                          { return 0 }
+func (e PlaceholderEnum) Syntax() pref.Syntax                 { return 0 }
+func (e PlaceholderEnum) Name() pref.Name                     { return pref.FullName(e).Name() }
+func (e PlaceholderEnum) FullName() pref.FullName             { return pref.FullName(e) }
+func (e PlaceholderEnum) IsPlaceholder() bool                 { return true }
+func (e PlaceholderEnum) Options() pref.ProtoMessage          { return descopts.Enum }
+func (e PlaceholderEnum) Values() pref.EnumValueDescriptors   { return emptyEnumValues }
+func (e PlaceholderEnum) ReservedNames() pref.Names           { return emptyNames }
+func (e PlaceholderEnum) ReservedRanges() pref.EnumRanges     { return emptyEnumRanges }
+func (e PlaceholderEnum) ProtoType(pref.EnumDescriptor)       { return }
+func (e PlaceholderEnum) ProtoInternal(pragma.DoNotImplement) { return }
+
+// PlaceholderEnumValue is a placeholder, representing only the full name.
+type PlaceholderEnumValue pref.FullName
+
+func (e PlaceholderEnumValue) ParentFile() pref.FileDescriptor     { return nil }
+func (e PlaceholderEnumValue) Parent() pref.Descriptor             { return nil }
+func (e PlaceholderEnumValue) Index() int                          { return 0 }
+func (e PlaceholderEnumValue) Syntax() pref.Syntax                 { return 0 }
+func (e PlaceholderEnumValue) Name() pref.Name                     { return pref.FullName(e).Name() }
+func (e PlaceholderEnumValue) FullName() pref.FullName             { return pref.FullName(e) }
+func (e PlaceholderEnumValue) IsPlaceholder() bool                 { return true }
+func (e PlaceholderEnumValue) Options() pref.ProtoMessage          { return descopts.EnumValue }
+func (e PlaceholderEnumValue) Number() pref.EnumNumber             { return 0 }
+func (e PlaceholderEnumValue) ProtoType(pref.EnumValueDescriptor)  { return }
+func (e PlaceholderEnumValue) ProtoInternal(pragma.DoNotImplement) { return }
+
+// PlaceholderMessage is a placeholder, representing only the full name.
+type PlaceholderMessage pref.FullName
+
+func (m PlaceholderMessage) ParentFile() pref.FileDescriptor             { return nil }
+func (m PlaceholderMessage) Parent() pref.Descriptor                     { return nil }
+func (m PlaceholderMessage) Index() int                                  { return 0 }
+func (m PlaceholderMessage) Syntax() pref.Syntax                         { return 0 }
+func (m PlaceholderMessage) Name() pref.Name                             { return pref.FullName(m).Name() }
+func (m PlaceholderMessage) FullName() pref.FullName                     { return pref.FullName(m) }
+func (m PlaceholderMessage) IsPlaceholder() bool                         { return true }
+func (m PlaceholderMessage) Options() pref.ProtoMessage                  { return descopts.Message }
+func (m PlaceholderMessage) IsMapEntry() bool                            { return false }
+func (m PlaceholderMessage) Fields() pref.FieldDescriptors               { return emptyFields }
+func (m PlaceholderMessage) Oneofs() pref.OneofDescriptors               { return emptyOneofs }
+func (m PlaceholderMessage) ReservedNames() pref.Names                   { return emptyNames }
+func (m PlaceholderMessage) ReservedRanges() pref.FieldRanges            { return emptyFieldRanges }
+func (m PlaceholderMessage) RequiredNumbers() pref.FieldNumbers          { return emptyFieldNumbers }
+func (m PlaceholderMessage) ExtensionRanges() pref.FieldRanges           { return emptyFieldRanges }
+func (m PlaceholderMessage) ExtensionRangeOptions(int) pref.ProtoMessage { panic("index out of range") }
+func (m PlaceholderMessage) Messages() pref.MessageDescriptors           { return emptyMessages }
+func (m PlaceholderMessage) Enums() pref.EnumDescriptors                 { return emptyEnums }
+func (m PlaceholderMessage) Extensions() pref.ExtensionDescriptors       { return emptyExtensions }
+func (m PlaceholderMessage) ProtoType(pref.MessageDescriptor)            { return }
+func (m PlaceholderMessage) ProtoInternal(pragma.DoNotImplement)         { return }
diff --git a/vendor/google.golang.org/protobuf/internal/filetype/build.go b/vendor/google.golang.org/protobuf/internal/filetype/build.go
new file mode 100644
index 0000000..0a0dd35
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/filetype/build.go
@@ -0,0 +1,297 @@
+// 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.
+
+// Package filetype provides functionality for wrapping descriptors
+// with Go type information.
+package filetype
+
+import (
+	"reflect"
+
+	"google.golang.org/protobuf/internal/descopts"
+	fdesc "google.golang.org/protobuf/internal/filedesc"
+	pimpl "google.golang.org/protobuf/internal/impl"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// Builder constructs type descriptors from a raw file descriptor
+// and associated Go types for each enum and message declaration.
+//
+//
+// Flattened Ordering
+//
+// The protobuf type system represents declarations as a tree. Certain nodes in
+// the tree require us to either associate it with a concrete Go type or to
+// resolve a dependency, which is information that must be provided separately
+// since it cannot be derived from the file descriptor alone.
+//
+// However, representing a tree as Go literals is difficult to simply do in a
+// space and time efficient way. Thus, we store them as a flattened list of
+// objects where the serialization order from the tree-based form is important.
+//
+// The "flattened ordering" is defined as a tree traversal of all enum, message,
+// extension, and service declarations using the following algorithm:
+//
+//	def VisitFileDecls(fd):
+//		for e in fd.Enums:      yield e
+//		for m in fd.Messages:   yield m
+//		for x in fd.Extensions: yield x
+//		for s in fd.Services:   yield s
+//		for m in fd.Messages:   yield from VisitMessageDecls(m)
+//
+//	def VisitMessageDecls(md):
+//		for e in md.Enums:      yield e
+//		for m in md.Messages:   yield m
+//		for x in md.Extensions: yield x
+//		for m in md.Messages:   yield from VisitMessageDecls(m)
+//
+// The traversal starts at the root file descriptor and yields each direct
+// declaration within each node before traversing into sub-declarations
+// that children themselves may have.
+type Builder struct {
+	// File is the underlying file descriptor builder.
+	File fdesc.Builder
+
+	// GoTypes is a unique set of the Go types for all declarations and
+	// dependencies. Each type is represented as a zero value of the Go type.
+	//
+	// Declarations are Go types generated for enums and messages directly
+	// declared (not publicly imported) in the proto source file.
+	// Messages for map entries are accounted for, but represented by nil.
+	// Enum declarations in "flattened ordering" come first, followed by
+	// message declarations in "flattened ordering".
+	//
+	// Dependencies are Go types for enums or messages referenced by
+	// message fields (excluding weak fields), for parent extended messages of
+	// extension fields, for enums or messages referenced by extension fields,
+	// and for input and output messages referenced by service methods.
+	// Dependencies must come after declarations, but the ordering of
+	// dependencies themselves is unspecified.
+	GoTypes []interface{}
+
+	// DependencyIndexes is an ordered list of indexes into GoTypes for the
+	// dependencies of messages, extensions, or services.
+	//
+	// There are 5 sub-lists in "flattened ordering" concatenated back-to-back:
+	//	0. Message field dependencies: list of the enum or message type
+	//	referred to by every message field.
+	//	1. Extension field targets: list of the extended parent message of
+	//	every extension.
+	//	2. Extension field dependencies: list of the enum or message type
+	//	referred to by every extension field.
+	//	3. Service method inputs: list of the input message type
+	//	referred to by every service method.
+	//	4. Service method outputs: list of the output message type
+	//	referred to by every service method.
+	//
+	// The offset into DependencyIndexes for the start of each sub-list
+	// is appended to the end in reverse order.
+	DependencyIndexes []int32
+
+	// EnumInfos is a list of enum infos in "flattened ordering".
+	EnumInfos []pimpl.EnumInfo
+
+	// MessageInfos is a list of message infos in "flattened ordering".
+	// If provided, the GoType and PBType for each element is populated.
+	//
+	// Requirement: len(MessageInfos) == len(Build.Messages)
+	MessageInfos []pimpl.MessageInfo
+
+	// ExtensionInfos is a list of extension infos in "flattened ordering".
+	// Each element is initialized and registered with the protoregistry package.
+	//
+	// Requirement: len(LegacyExtensions) == len(Build.Extensions)
+	ExtensionInfos []pimpl.ExtensionInfo
+
+	// TypeRegistry is the registry to register each type descriptor.
+	// If nil, it uses protoregistry.GlobalTypes.
+	TypeRegistry interface {
+		RegisterMessage(pref.MessageType) error
+		RegisterEnum(pref.EnumType) error
+		RegisterExtension(pref.ExtensionType) error
+	}
+}
+
+// Out is the output of the builder.
+type Out struct {
+	File pref.FileDescriptor
+}
+
+func (tb Builder) Build() (out Out) {
+	// Replace the resolver with one that resolves dependencies by index,
+	// which is faster and more reliable than relying on the global registry.
+	if tb.File.FileRegistry == nil {
+		tb.File.FileRegistry = preg.GlobalFiles
+	}
+	tb.File.FileRegistry = &resolverByIndex{
+		goTypes:      tb.GoTypes,
+		depIdxs:      tb.DependencyIndexes,
+		fileRegistry: tb.File.FileRegistry,
+	}
+
+	// Initialize registry if unpopulated.
+	if tb.TypeRegistry == nil {
+		tb.TypeRegistry = preg.GlobalTypes
+	}
+
+	fbOut := tb.File.Build()
+	out.File = fbOut.File
+
+	// Process enums.
+	enumGoTypes := tb.GoTypes[:len(fbOut.Enums)]
+	if len(tb.EnumInfos) != len(fbOut.Enums) {
+		panic("mismatching enum lengths")
+	}
+	if len(fbOut.Enums) > 0 {
+		for i := range fbOut.Enums {
+			tb.EnumInfos[i] = pimpl.EnumInfo{
+				GoReflectType: reflect.TypeOf(enumGoTypes[i]),
+				Desc:          &fbOut.Enums[i],
+			}
+			// Register enum types.
+			if err := tb.TypeRegistry.RegisterEnum(&tb.EnumInfos[i]); err != nil {
+				panic(err)
+			}
+		}
+	}
+
+	// Process messages.
+	messageGoTypes := tb.GoTypes[len(fbOut.Enums):][:len(fbOut.Messages)]
+	if len(tb.MessageInfos) != len(fbOut.Messages) {
+		panic("mismatching message lengths")
+	}
+	if len(fbOut.Messages) > 0 {
+		for i := range fbOut.Messages {
+			if messageGoTypes[i] == nil {
+				continue // skip map entry
+			}
+
+			tb.MessageInfos[i].GoReflectType = reflect.TypeOf(messageGoTypes[i])
+			tb.MessageInfos[i].Desc = &fbOut.Messages[i]
+
+			// Register message types.
+			if err := tb.TypeRegistry.RegisterMessage(&tb.MessageInfos[i]); err != nil {
+				panic(err)
+			}
+		}
+
+		// As a special-case for descriptor.proto,
+		// locally register concrete message type for the options.
+		if out.File.Path() == "google/protobuf/descriptor.proto" && out.File.Package() == "google.protobuf" {
+			for i := range fbOut.Messages {
+				switch fbOut.Messages[i].Name() {
+				case "FileOptions":
+					descopts.File = messageGoTypes[i].(pref.ProtoMessage)
+				case "EnumOptions":
+					descopts.Enum = messageGoTypes[i].(pref.ProtoMessage)
+				case "EnumValueOptions":
+					descopts.EnumValue = messageGoTypes[i].(pref.ProtoMessage)
+				case "MessageOptions":
+					descopts.Message = messageGoTypes[i].(pref.ProtoMessage)
+				case "FieldOptions":
+					descopts.Field = messageGoTypes[i].(pref.ProtoMessage)
+				case "OneofOptions":
+					descopts.Oneof = messageGoTypes[i].(pref.ProtoMessage)
+				case "ExtensionRangeOptions":
+					descopts.ExtensionRange = messageGoTypes[i].(pref.ProtoMessage)
+				case "ServiceOptions":
+					descopts.Service = messageGoTypes[i].(pref.ProtoMessage)
+				case "MethodOptions":
+					descopts.Method = messageGoTypes[i].(pref.ProtoMessage)
+				}
+			}
+		}
+	}
+
+	// Process extensions.
+	if len(tb.ExtensionInfos) != len(fbOut.Extensions) {
+		panic("mismatching extension lengths")
+	}
+	var depIdx int32
+	for i := range fbOut.Extensions {
+		// For enum and message kinds, determine the referent Go type so
+		// that we can construct their constructors.
+		const listExtDeps = 2
+		var goType reflect.Type
+		switch fbOut.Extensions[i].L1.Kind {
+		case pref.EnumKind:
+			j := depIdxs.Get(tb.DependencyIndexes, listExtDeps, depIdx)
+			goType = reflect.TypeOf(tb.GoTypes[j])
+			depIdx++
+		case pref.MessageKind, pref.GroupKind:
+			j := depIdxs.Get(tb.DependencyIndexes, listExtDeps, depIdx)
+			goType = reflect.TypeOf(tb.GoTypes[j])
+			depIdx++
+		default:
+			goType = goTypeForPBKind[fbOut.Extensions[i].L1.Kind]
+		}
+		if fbOut.Extensions[i].IsList() {
+			goType = reflect.SliceOf(goType)
+		}
+
+		pimpl.InitExtensionInfo(&tb.ExtensionInfos[i], &fbOut.Extensions[i], goType)
+
+		// Register extension types.
+		if err := tb.TypeRegistry.RegisterExtension(&tb.ExtensionInfos[i]); err != nil {
+			panic(err)
+		}
+	}
+
+	return out
+}
+
+var goTypeForPBKind = map[pref.Kind]reflect.Type{
+	pref.BoolKind:     reflect.TypeOf(bool(false)),
+	pref.Int32Kind:    reflect.TypeOf(int32(0)),
+	pref.Sint32Kind:   reflect.TypeOf(int32(0)),
+	pref.Sfixed32Kind: reflect.TypeOf(int32(0)),
+	pref.Int64Kind:    reflect.TypeOf(int64(0)),
+	pref.Sint64Kind:   reflect.TypeOf(int64(0)),
+	pref.Sfixed64Kind: reflect.TypeOf(int64(0)),
+	pref.Uint32Kind:   reflect.TypeOf(uint32(0)),
+	pref.Fixed32Kind:  reflect.TypeOf(uint32(0)),
+	pref.Uint64Kind:   reflect.TypeOf(uint64(0)),
+	pref.Fixed64Kind:  reflect.TypeOf(uint64(0)),
+	pref.FloatKind:    reflect.TypeOf(float32(0)),
+	pref.DoubleKind:   reflect.TypeOf(float64(0)),
+	pref.StringKind:   reflect.TypeOf(string("")),
+	pref.BytesKind:    reflect.TypeOf([]byte(nil)),
+}
+
+type depIdxs []int32
+
+// Get retrieves the jth element of the ith sub-list.
+func (x depIdxs) Get(i, j int32) int32 {
+	return x[x[int32(len(x))-i-1]+j]
+}
+
+type (
+	resolverByIndex struct {
+		goTypes []interface{}
+		depIdxs depIdxs
+		fileRegistry
+	}
+	fileRegistry interface {
+		FindFileByPath(string) (pref.FileDescriptor, error)
+		FindDescriptorByName(pref.FullName) (pref.Descriptor, error)
+		RegisterFile(pref.FileDescriptor) error
+	}
+)
+
+func (r *resolverByIndex) FindEnumByIndex(i, j int32, es []fdesc.Enum, ms []fdesc.Message) pref.EnumDescriptor {
+	if depIdx := int(r.depIdxs.Get(i, j)); int(depIdx) < len(es)+len(ms) {
+		return &es[depIdx]
+	} else {
+		return pimpl.Export{}.EnumDescriptorOf(r.goTypes[depIdx])
+	}
+}
+
+func (r *resolverByIndex) FindMessageByIndex(i, j int32, es []fdesc.Enum, ms []fdesc.Message) pref.MessageDescriptor {
+	if depIdx := int(r.depIdxs.Get(i, j)); depIdx < len(es)+len(ms) {
+		return &ms[depIdx-len(es)]
+	} else {
+		return pimpl.Export{}.MessageDescriptorOf(r.goTypes[depIdx])
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/flags/flags.go b/vendor/google.golang.org/protobuf/internal/flags/flags.go
new file mode 100644
index 0000000..58372dd
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/flags/flags.go
@@ -0,0 +1,24 @@
+// 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 flags provides a set of flags controlled by build tags.
+package flags
+
+// ProtoLegacy specifies whether to enable support for legacy functionality
+// such as MessageSets, weak fields, and various other obscure behavior
+// that is necessary to maintain backwards compatibility with proto1 or
+// the pre-release variants of proto2 and proto3.
+//
+// This is disabled by default unless built with the "protolegacy" tag.
+//
+// WARNING: The compatibility agreement covers nothing provided by this flag.
+// As such, functionality may suddenly be removed or changed at our discretion.
+const ProtoLegacy = protoLegacy
+
+// LazyUnmarshalExtensions specifies whether to lazily unmarshal extensions.
+//
+// Lazy extension unmarshaling validates the contents of message-valued
+// extension fields at unmarshal time, but defers creating the message
+// structure until the extension is first accessed.
+const LazyUnmarshalExtensions = ProtoLegacy
diff --git a/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_disable.go b/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_disable.go
new file mode 100644
index 0000000..a72995f
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_disable.go
@@ -0,0 +1,9 @@
+// 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 !protolegacy
+
+package flags
+
+const protoLegacy = false
diff --git a/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_enable.go b/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_enable.go
new file mode 100644
index 0000000..772e2f0
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/flags/proto_legacy_enable.go
@@ -0,0 +1,9 @@
+// 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 protolegacy
+
+package flags
+
+const protoLegacy = true
diff --git a/vendor/google.golang.org/protobuf/internal/genid/any_gen.go b/vendor/google.golang.org/protobuf/internal/genid/any_gen.go
new file mode 100644
index 0000000..e6f7d47
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/any_gen.go
@@ -0,0 +1,34 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_any_proto = "google/protobuf/any.proto"
+
+// Names for google.protobuf.Any.
+const (
+	Any_message_name     protoreflect.Name     = "Any"
+	Any_message_fullname protoreflect.FullName = "google.protobuf.Any"
+)
+
+// Field names for google.protobuf.Any.
+const (
+	Any_TypeUrl_field_name protoreflect.Name = "type_url"
+	Any_Value_field_name   protoreflect.Name = "value"
+
+	Any_TypeUrl_field_fullname protoreflect.FullName = "google.protobuf.Any.type_url"
+	Any_Value_field_fullname   protoreflect.FullName = "google.protobuf.Any.value"
+)
+
+// Field numbers for google.protobuf.Any.
+const (
+	Any_TypeUrl_field_number protoreflect.FieldNumber = 1
+	Any_Value_field_number   protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/api_gen.go b/vendor/google.golang.org/protobuf/internal/genid/api_gen.go
new file mode 100644
index 0000000..df8f918
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/api_gen.go
@@ -0,0 +1,106 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_api_proto = "google/protobuf/api.proto"
+
+// Names for google.protobuf.Api.
+const (
+	Api_message_name     protoreflect.Name     = "Api"
+	Api_message_fullname protoreflect.FullName = "google.protobuf.Api"
+)
+
+// Field names for google.protobuf.Api.
+const (
+	Api_Name_field_name          protoreflect.Name = "name"
+	Api_Methods_field_name       protoreflect.Name = "methods"
+	Api_Options_field_name       protoreflect.Name = "options"
+	Api_Version_field_name       protoreflect.Name = "version"
+	Api_SourceContext_field_name protoreflect.Name = "source_context"
+	Api_Mixins_field_name        protoreflect.Name = "mixins"
+	Api_Syntax_field_name        protoreflect.Name = "syntax"
+
+	Api_Name_field_fullname          protoreflect.FullName = "google.protobuf.Api.name"
+	Api_Methods_field_fullname       protoreflect.FullName = "google.protobuf.Api.methods"
+	Api_Options_field_fullname       protoreflect.FullName = "google.protobuf.Api.options"
+	Api_Version_field_fullname       protoreflect.FullName = "google.protobuf.Api.version"
+	Api_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Api.source_context"
+	Api_Mixins_field_fullname        protoreflect.FullName = "google.protobuf.Api.mixins"
+	Api_Syntax_field_fullname        protoreflect.FullName = "google.protobuf.Api.syntax"
+)
+
+// Field numbers for google.protobuf.Api.
+const (
+	Api_Name_field_number          protoreflect.FieldNumber = 1
+	Api_Methods_field_number       protoreflect.FieldNumber = 2
+	Api_Options_field_number       protoreflect.FieldNumber = 3
+	Api_Version_field_number       protoreflect.FieldNumber = 4
+	Api_SourceContext_field_number protoreflect.FieldNumber = 5
+	Api_Mixins_field_number        protoreflect.FieldNumber = 6
+	Api_Syntax_field_number        protoreflect.FieldNumber = 7
+)
+
+// Names for google.protobuf.Method.
+const (
+	Method_message_name     protoreflect.Name     = "Method"
+	Method_message_fullname protoreflect.FullName = "google.protobuf.Method"
+)
+
+// Field names for google.protobuf.Method.
+const (
+	Method_Name_field_name              protoreflect.Name = "name"
+	Method_RequestTypeUrl_field_name    protoreflect.Name = "request_type_url"
+	Method_RequestStreaming_field_name  protoreflect.Name = "request_streaming"
+	Method_ResponseTypeUrl_field_name   protoreflect.Name = "response_type_url"
+	Method_ResponseStreaming_field_name protoreflect.Name = "response_streaming"
+	Method_Options_field_name           protoreflect.Name = "options"
+	Method_Syntax_field_name            protoreflect.Name = "syntax"
+
+	Method_Name_field_fullname              protoreflect.FullName = "google.protobuf.Method.name"
+	Method_RequestTypeUrl_field_fullname    protoreflect.FullName = "google.protobuf.Method.request_type_url"
+	Method_RequestStreaming_field_fullname  protoreflect.FullName = "google.protobuf.Method.request_streaming"
+	Method_ResponseTypeUrl_field_fullname   protoreflect.FullName = "google.protobuf.Method.response_type_url"
+	Method_ResponseStreaming_field_fullname protoreflect.FullName = "google.protobuf.Method.response_streaming"
+	Method_Options_field_fullname           protoreflect.FullName = "google.protobuf.Method.options"
+	Method_Syntax_field_fullname            protoreflect.FullName = "google.protobuf.Method.syntax"
+)
+
+// Field numbers for google.protobuf.Method.
+const (
+	Method_Name_field_number              protoreflect.FieldNumber = 1
+	Method_RequestTypeUrl_field_number    protoreflect.FieldNumber = 2
+	Method_RequestStreaming_field_number  protoreflect.FieldNumber = 3
+	Method_ResponseTypeUrl_field_number   protoreflect.FieldNumber = 4
+	Method_ResponseStreaming_field_number protoreflect.FieldNumber = 5
+	Method_Options_field_number           protoreflect.FieldNumber = 6
+	Method_Syntax_field_number            protoreflect.FieldNumber = 7
+)
+
+// Names for google.protobuf.Mixin.
+const (
+	Mixin_message_name     protoreflect.Name     = "Mixin"
+	Mixin_message_fullname protoreflect.FullName = "google.protobuf.Mixin"
+)
+
+// Field names for google.protobuf.Mixin.
+const (
+	Mixin_Name_field_name protoreflect.Name = "name"
+	Mixin_Root_field_name protoreflect.Name = "root"
+
+	Mixin_Name_field_fullname protoreflect.FullName = "google.protobuf.Mixin.name"
+	Mixin_Root_field_fullname protoreflect.FullName = "google.protobuf.Mixin.root"
+)
+
+// Field numbers for google.protobuf.Mixin.
+const (
+	Mixin_Name_field_number protoreflect.FieldNumber = 1
+	Mixin_Root_field_number protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/descriptor_gen.go b/vendor/google.golang.org/protobuf/internal/genid/descriptor_gen.go
new file mode 100644
index 0000000..e3cdf1c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/descriptor_gen.go
@@ -0,0 +1,829 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_descriptor_proto = "google/protobuf/descriptor.proto"
+
+// Names for google.protobuf.FileDescriptorSet.
+const (
+	FileDescriptorSet_message_name     protoreflect.Name     = "FileDescriptorSet"
+	FileDescriptorSet_message_fullname protoreflect.FullName = "google.protobuf.FileDescriptorSet"
+)
+
+// Field names for google.protobuf.FileDescriptorSet.
+const (
+	FileDescriptorSet_File_field_name protoreflect.Name = "file"
+
+	FileDescriptorSet_File_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorSet.file"
+)
+
+// Field numbers for google.protobuf.FileDescriptorSet.
+const (
+	FileDescriptorSet_File_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.FileDescriptorProto.
+const (
+	FileDescriptorProto_message_name     protoreflect.Name     = "FileDescriptorProto"
+	FileDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto"
+)
+
+// Field names for google.protobuf.FileDescriptorProto.
+const (
+	FileDescriptorProto_Name_field_name             protoreflect.Name = "name"
+	FileDescriptorProto_Package_field_name          protoreflect.Name = "package"
+	FileDescriptorProto_Dependency_field_name       protoreflect.Name = "dependency"
+	FileDescriptorProto_PublicDependency_field_name protoreflect.Name = "public_dependency"
+	FileDescriptorProto_WeakDependency_field_name   protoreflect.Name = "weak_dependency"
+	FileDescriptorProto_MessageType_field_name      protoreflect.Name = "message_type"
+	FileDescriptorProto_EnumType_field_name         protoreflect.Name = "enum_type"
+	FileDescriptorProto_Service_field_name          protoreflect.Name = "service"
+	FileDescriptorProto_Extension_field_name        protoreflect.Name = "extension"
+	FileDescriptorProto_Options_field_name          protoreflect.Name = "options"
+	FileDescriptorProto_SourceCodeInfo_field_name   protoreflect.Name = "source_code_info"
+	FileDescriptorProto_Syntax_field_name           protoreflect.Name = "syntax"
+
+	FileDescriptorProto_Name_field_fullname             protoreflect.FullName = "google.protobuf.FileDescriptorProto.name"
+	FileDescriptorProto_Package_field_fullname          protoreflect.FullName = "google.protobuf.FileDescriptorProto.package"
+	FileDescriptorProto_Dependency_field_fullname       protoreflect.FullName = "google.protobuf.FileDescriptorProto.dependency"
+	FileDescriptorProto_PublicDependency_field_fullname protoreflect.FullName = "google.protobuf.FileDescriptorProto.public_dependency"
+	FileDescriptorProto_WeakDependency_field_fullname   protoreflect.FullName = "google.protobuf.FileDescriptorProto.weak_dependency"
+	FileDescriptorProto_MessageType_field_fullname      protoreflect.FullName = "google.protobuf.FileDescriptorProto.message_type"
+	FileDescriptorProto_EnumType_field_fullname         protoreflect.FullName = "google.protobuf.FileDescriptorProto.enum_type"
+	FileDescriptorProto_Service_field_fullname          protoreflect.FullName = "google.protobuf.FileDescriptorProto.service"
+	FileDescriptorProto_Extension_field_fullname        protoreflect.FullName = "google.protobuf.FileDescriptorProto.extension"
+	FileDescriptorProto_Options_field_fullname          protoreflect.FullName = "google.protobuf.FileDescriptorProto.options"
+	FileDescriptorProto_SourceCodeInfo_field_fullname   protoreflect.FullName = "google.protobuf.FileDescriptorProto.source_code_info"
+	FileDescriptorProto_Syntax_field_fullname           protoreflect.FullName = "google.protobuf.FileDescriptorProto.syntax"
+)
+
+// Field numbers for google.protobuf.FileDescriptorProto.
+const (
+	FileDescriptorProto_Name_field_number             protoreflect.FieldNumber = 1
+	FileDescriptorProto_Package_field_number          protoreflect.FieldNumber = 2
+	FileDescriptorProto_Dependency_field_number       protoreflect.FieldNumber = 3
+	FileDescriptorProto_PublicDependency_field_number protoreflect.FieldNumber = 10
+	FileDescriptorProto_WeakDependency_field_number   protoreflect.FieldNumber = 11
+	FileDescriptorProto_MessageType_field_number      protoreflect.FieldNumber = 4
+	FileDescriptorProto_EnumType_field_number         protoreflect.FieldNumber = 5
+	FileDescriptorProto_Service_field_number          protoreflect.FieldNumber = 6
+	FileDescriptorProto_Extension_field_number        protoreflect.FieldNumber = 7
+	FileDescriptorProto_Options_field_number          protoreflect.FieldNumber = 8
+	FileDescriptorProto_SourceCodeInfo_field_number   protoreflect.FieldNumber = 9
+	FileDescriptorProto_Syntax_field_number           protoreflect.FieldNumber = 12
+)
+
+// Names for google.protobuf.DescriptorProto.
+const (
+	DescriptorProto_message_name     protoreflect.Name     = "DescriptorProto"
+	DescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto"
+)
+
+// Field names for google.protobuf.DescriptorProto.
+const (
+	DescriptorProto_Name_field_name           protoreflect.Name = "name"
+	DescriptorProto_Field_field_name          protoreflect.Name = "field"
+	DescriptorProto_Extension_field_name      protoreflect.Name = "extension"
+	DescriptorProto_NestedType_field_name     protoreflect.Name = "nested_type"
+	DescriptorProto_EnumType_field_name       protoreflect.Name = "enum_type"
+	DescriptorProto_ExtensionRange_field_name protoreflect.Name = "extension_range"
+	DescriptorProto_OneofDecl_field_name      protoreflect.Name = "oneof_decl"
+	DescriptorProto_Options_field_name        protoreflect.Name = "options"
+	DescriptorProto_ReservedRange_field_name  protoreflect.Name = "reserved_range"
+	DescriptorProto_ReservedName_field_name   protoreflect.Name = "reserved_name"
+
+	DescriptorProto_Name_field_fullname           protoreflect.FullName = "google.protobuf.DescriptorProto.name"
+	DescriptorProto_Field_field_fullname          protoreflect.FullName = "google.protobuf.DescriptorProto.field"
+	DescriptorProto_Extension_field_fullname      protoreflect.FullName = "google.protobuf.DescriptorProto.extension"
+	DescriptorProto_NestedType_field_fullname     protoreflect.FullName = "google.protobuf.DescriptorProto.nested_type"
+	DescriptorProto_EnumType_field_fullname       protoreflect.FullName = "google.protobuf.DescriptorProto.enum_type"
+	DescriptorProto_ExtensionRange_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.extension_range"
+	DescriptorProto_OneofDecl_field_fullname      protoreflect.FullName = "google.protobuf.DescriptorProto.oneof_decl"
+	DescriptorProto_Options_field_fullname        protoreflect.FullName = "google.protobuf.DescriptorProto.options"
+	DescriptorProto_ReservedRange_field_fullname  protoreflect.FullName = "google.protobuf.DescriptorProto.reserved_range"
+	DescriptorProto_ReservedName_field_fullname   protoreflect.FullName = "google.protobuf.DescriptorProto.reserved_name"
+)
+
+// Field numbers for google.protobuf.DescriptorProto.
+const (
+	DescriptorProto_Name_field_number           protoreflect.FieldNumber = 1
+	DescriptorProto_Field_field_number          protoreflect.FieldNumber = 2
+	DescriptorProto_Extension_field_number      protoreflect.FieldNumber = 6
+	DescriptorProto_NestedType_field_number     protoreflect.FieldNumber = 3
+	DescriptorProto_EnumType_field_number       protoreflect.FieldNumber = 4
+	DescriptorProto_ExtensionRange_field_number protoreflect.FieldNumber = 5
+	DescriptorProto_OneofDecl_field_number      protoreflect.FieldNumber = 8
+	DescriptorProto_Options_field_number        protoreflect.FieldNumber = 7
+	DescriptorProto_ReservedRange_field_number  protoreflect.FieldNumber = 9
+	DescriptorProto_ReservedName_field_number   protoreflect.FieldNumber = 10
+)
+
+// Names for google.protobuf.DescriptorProto.ExtensionRange.
+const (
+	DescriptorProto_ExtensionRange_message_name     protoreflect.Name     = "ExtensionRange"
+	DescriptorProto_ExtensionRange_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange"
+)
+
+// Field names for google.protobuf.DescriptorProto.ExtensionRange.
+const (
+	DescriptorProto_ExtensionRange_Start_field_name   protoreflect.Name = "start"
+	DescriptorProto_ExtensionRange_End_field_name     protoreflect.Name = "end"
+	DescriptorProto_ExtensionRange_Options_field_name protoreflect.Name = "options"
+
+	DescriptorProto_ExtensionRange_Start_field_fullname   protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.start"
+	DescriptorProto_ExtensionRange_End_field_fullname     protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.end"
+	DescriptorProto_ExtensionRange_Options_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ExtensionRange.options"
+)
+
+// Field numbers for google.protobuf.DescriptorProto.ExtensionRange.
+const (
+	DescriptorProto_ExtensionRange_Start_field_number   protoreflect.FieldNumber = 1
+	DescriptorProto_ExtensionRange_End_field_number     protoreflect.FieldNumber = 2
+	DescriptorProto_ExtensionRange_Options_field_number protoreflect.FieldNumber = 3
+)
+
+// Names for google.protobuf.DescriptorProto.ReservedRange.
+const (
+	DescriptorProto_ReservedRange_message_name     protoreflect.Name     = "ReservedRange"
+	DescriptorProto_ReservedRange_message_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange"
+)
+
+// Field names for google.protobuf.DescriptorProto.ReservedRange.
+const (
+	DescriptorProto_ReservedRange_Start_field_name protoreflect.Name = "start"
+	DescriptorProto_ReservedRange_End_field_name   protoreflect.Name = "end"
+
+	DescriptorProto_ReservedRange_Start_field_fullname protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange.start"
+	DescriptorProto_ReservedRange_End_field_fullname   protoreflect.FullName = "google.protobuf.DescriptorProto.ReservedRange.end"
+)
+
+// Field numbers for google.protobuf.DescriptorProto.ReservedRange.
+const (
+	DescriptorProto_ReservedRange_Start_field_number protoreflect.FieldNumber = 1
+	DescriptorProto_ReservedRange_End_field_number   protoreflect.FieldNumber = 2
+)
+
+// Names for google.protobuf.ExtensionRangeOptions.
+const (
+	ExtensionRangeOptions_message_name     protoreflect.Name     = "ExtensionRangeOptions"
+	ExtensionRangeOptions_message_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions"
+)
+
+// Field names for google.protobuf.ExtensionRangeOptions.
+const (
+	ExtensionRangeOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	ExtensionRangeOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.ExtensionRangeOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.ExtensionRangeOptions.
+const (
+	ExtensionRangeOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.FieldDescriptorProto.
+const (
+	FieldDescriptorProto_message_name     protoreflect.Name     = "FieldDescriptorProto"
+	FieldDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto"
+)
+
+// Field names for google.protobuf.FieldDescriptorProto.
+const (
+	FieldDescriptorProto_Name_field_name           protoreflect.Name = "name"
+	FieldDescriptorProto_Number_field_name         protoreflect.Name = "number"
+	FieldDescriptorProto_Label_field_name          protoreflect.Name = "label"
+	FieldDescriptorProto_Type_field_name           protoreflect.Name = "type"
+	FieldDescriptorProto_TypeName_field_name       protoreflect.Name = "type_name"
+	FieldDescriptorProto_Extendee_field_name       protoreflect.Name = "extendee"
+	FieldDescriptorProto_DefaultValue_field_name   protoreflect.Name = "default_value"
+	FieldDescriptorProto_OneofIndex_field_name     protoreflect.Name = "oneof_index"
+	FieldDescriptorProto_JsonName_field_name       protoreflect.Name = "json_name"
+	FieldDescriptorProto_Options_field_name        protoreflect.Name = "options"
+	FieldDescriptorProto_Proto3Optional_field_name protoreflect.Name = "proto3_optional"
+
+	FieldDescriptorProto_Name_field_fullname           protoreflect.FullName = "google.protobuf.FieldDescriptorProto.name"
+	FieldDescriptorProto_Number_field_fullname         protoreflect.FullName = "google.protobuf.FieldDescriptorProto.number"
+	FieldDescriptorProto_Label_field_fullname          protoreflect.FullName = "google.protobuf.FieldDescriptorProto.label"
+	FieldDescriptorProto_Type_field_fullname           protoreflect.FullName = "google.protobuf.FieldDescriptorProto.type"
+	FieldDescriptorProto_TypeName_field_fullname       protoreflect.FullName = "google.protobuf.FieldDescriptorProto.type_name"
+	FieldDescriptorProto_Extendee_field_fullname       protoreflect.FullName = "google.protobuf.FieldDescriptorProto.extendee"
+	FieldDescriptorProto_DefaultValue_field_fullname   protoreflect.FullName = "google.protobuf.FieldDescriptorProto.default_value"
+	FieldDescriptorProto_OneofIndex_field_fullname     protoreflect.FullName = "google.protobuf.FieldDescriptorProto.oneof_index"
+	FieldDescriptorProto_JsonName_field_fullname       protoreflect.FullName = "google.protobuf.FieldDescriptorProto.json_name"
+	FieldDescriptorProto_Options_field_fullname        protoreflect.FullName = "google.protobuf.FieldDescriptorProto.options"
+	FieldDescriptorProto_Proto3Optional_field_fullname protoreflect.FullName = "google.protobuf.FieldDescriptorProto.proto3_optional"
+)
+
+// Field numbers for google.protobuf.FieldDescriptorProto.
+const (
+	FieldDescriptorProto_Name_field_number           protoreflect.FieldNumber = 1
+	FieldDescriptorProto_Number_field_number         protoreflect.FieldNumber = 3
+	FieldDescriptorProto_Label_field_number          protoreflect.FieldNumber = 4
+	FieldDescriptorProto_Type_field_number           protoreflect.FieldNumber = 5
+	FieldDescriptorProto_TypeName_field_number       protoreflect.FieldNumber = 6
+	FieldDescriptorProto_Extendee_field_number       protoreflect.FieldNumber = 2
+	FieldDescriptorProto_DefaultValue_field_number   protoreflect.FieldNumber = 7
+	FieldDescriptorProto_OneofIndex_field_number     protoreflect.FieldNumber = 9
+	FieldDescriptorProto_JsonName_field_number       protoreflect.FieldNumber = 10
+	FieldDescriptorProto_Options_field_number        protoreflect.FieldNumber = 8
+	FieldDescriptorProto_Proto3Optional_field_number protoreflect.FieldNumber = 17
+)
+
+// Full and short names for google.protobuf.FieldDescriptorProto.Type.
+const (
+	FieldDescriptorProto_Type_enum_fullname = "google.protobuf.FieldDescriptorProto.Type"
+	FieldDescriptorProto_Type_enum_name     = "Type"
+)
+
+// Full and short names for google.protobuf.FieldDescriptorProto.Label.
+const (
+	FieldDescriptorProto_Label_enum_fullname = "google.protobuf.FieldDescriptorProto.Label"
+	FieldDescriptorProto_Label_enum_name     = "Label"
+)
+
+// Names for google.protobuf.OneofDescriptorProto.
+const (
+	OneofDescriptorProto_message_name     protoreflect.Name     = "OneofDescriptorProto"
+	OneofDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.OneofDescriptorProto"
+)
+
+// Field names for google.protobuf.OneofDescriptorProto.
+const (
+	OneofDescriptorProto_Name_field_name    protoreflect.Name = "name"
+	OneofDescriptorProto_Options_field_name protoreflect.Name = "options"
+
+	OneofDescriptorProto_Name_field_fullname    protoreflect.FullName = "google.protobuf.OneofDescriptorProto.name"
+	OneofDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.OneofDescriptorProto.options"
+)
+
+// Field numbers for google.protobuf.OneofDescriptorProto.
+const (
+	OneofDescriptorProto_Name_field_number    protoreflect.FieldNumber = 1
+	OneofDescriptorProto_Options_field_number protoreflect.FieldNumber = 2
+)
+
+// Names for google.protobuf.EnumDescriptorProto.
+const (
+	EnumDescriptorProto_message_name     protoreflect.Name     = "EnumDescriptorProto"
+	EnumDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto"
+)
+
+// Field names for google.protobuf.EnumDescriptorProto.
+const (
+	EnumDescriptorProto_Name_field_name          protoreflect.Name = "name"
+	EnumDescriptorProto_Value_field_name         protoreflect.Name = "value"
+	EnumDescriptorProto_Options_field_name       protoreflect.Name = "options"
+	EnumDescriptorProto_ReservedRange_field_name protoreflect.Name = "reserved_range"
+	EnumDescriptorProto_ReservedName_field_name  protoreflect.Name = "reserved_name"
+
+	EnumDescriptorProto_Name_field_fullname          protoreflect.FullName = "google.protobuf.EnumDescriptorProto.name"
+	EnumDescriptorProto_Value_field_fullname         protoreflect.FullName = "google.protobuf.EnumDescriptorProto.value"
+	EnumDescriptorProto_Options_field_fullname       protoreflect.FullName = "google.protobuf.EnumDescriptorProto.options"
+	EnumDescriptorProto_ReservedRange_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.reserved_range"
+	EnumDescriptorProto_ReservedName_field_fullname  protoreflect.FullName = "google.protobuf.EnumDescriptorProto.reserved_name"
+)
+
+// Field numbers for google.protobuf.EnumDescriptorProto.
+const (
+	EnumDescriptorProto_Name_field_number          protoreflect.FieldNumber = 1
+	EnumDescriptorProto_Value_field_number         protoreflect.FieldNumber = 2
+	EnumDescriptorProto_Options_field_number       protoreflect.FieldNumber = 3
+	EnumDescriptorProto_ReservedRange_field_number protoreflect.FieldNumber = 4
+	EnumDescriptorProto_ReservedName_field_number  protoreflect.FieldNumber = 5
+)
+
+// Names for google.protobuf.EnumDescriptorProto.EnumReservedRange.
+const (
+	EnumDescriptorProto_EnumReservedRange_message_name     protoreflect.Name     = "EnumReservedRange"
+	EnumDescriptorProto_EnumReservedRange_message_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange"
+)
+
+// Field names for google.protobuf.EnumDescriptorProto.EnumReservedRange.
+const (
+	EnumDescriptorProto_EnumReservedRange_Start_field_name protoreflect.Name = "start"
+	EnumDescriptorProto_EnumReservedRange_End_field_name   protoreflect.Name = "end"
+
+	EnumDescriptorProto_EnumReservedRange_Start_field_fullname protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange.start"
+	EnumDescriptorProto_EnumReservedRange_End_field_fullname   protoreflect.FullName = "google.protobuf.EnumDescriptorProto.EnumReservedRange.end"
+)
+
+// Field numbers for google.protobuf.EnumDescriptorProto.EnumReservedRange.
+const (
+	EnumDescriptorProto_EnumReservedRange_Start_field_number protoreflect.FieldNumber = 1
+	EnumDescriptorProto_EnumReservedRange_End_field_number   protoreflect.FieldNumber = 2
+)
+
+// Names for google.protobuf.EnumValueDescriptorProto.
+const (
+	EnumValueDescriptorProto_message_name     protoreflect.Name     = "EnumValueDescriptorProto"
+	EnumValueDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto"
+)
+
+// Field names for google.protobuf.EnumValueDescriptorProto.
+const (
+	EnumValueDescriptorProto_Name_field_name    protoreflect.Name = "name"
+	EnumValueDescriptorProto_Number_field_name  protoreflect.Name = "number"
+	EnumValueDescriptorProto_Options_field_name protoreflect.Name = "options"
+
+	EnumValueDescriptorProto_Name_field_fullname    protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.name"
+	EnumValueDescriptorProto_Number_field_fullname  protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.number"
+	EnumValueDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.EnumValueDescriptorProto.options"
+)
+
+// Field numbers for google.protobuf.EnumValueDescriptorProto.
+const (
+	EnumValueDescriptorProto_Name_field_number    protoreflect.FieldNumber = 1
+	EnumValueDescriptorProto_Number_field_number  protoreflect.FieldNumber = 2
+	EnumValueDescriptorProto_Options_field_number protoreflect.FieldNumber = 3
+)
+
+// Names for google.protobuf.ServiceDescriptorProto.
+const (
+	ServiceDescriptorProto_message_name     protoreflect.Name     = "ServiceDescriptorProto"
+	ServiceDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto"
+)
+
+// Field names for google.protobuf.ServiceDescriptorProto.
+const (
+	ServiceDescriptorProto_Name_field_name    protoreflect.Name = "name"
+	ServiceDescriptorProto_Method_field_name  protoreflect.Name = "method"
+	ServiceDescriptorProto_Options_field_name protoreflect.Name = "options"
+
+	ServiceDescriptorProto_Name_field_fullname    protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.name"
+	ServiceDescriptorProto_Method_field_fullname  protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.method"
+	ServiceDescriptorProto_Options_field_fullname protoreflect.FullName = "google.protobuf.ServiceDescriptorProto.options"
+)
+
+// Field numbers for google.protobuf.ServiceDescriptorProto.
+const (
+	ServiceDescriptorProto_Name_field_number    protoreflect.FieldNumber = 1
+	ServiceDescriptorProto_Method_field_number  protoreflect.FieldNumber = 2
+	ServiceDescriptorProto_Options_field_number protoreflect.FieldNumber = 3
+)
+
+// Names for google.protobuf.MethodDescriptorProto.
+const (
+	MethodDescriptorProto_message_name     protoreflect.Name     = "MethodDescriptorProto"
+	MethodDescriptorProto_message_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto"
+)
+
+// Field names for google.protobuf.MethodDescriptorProto.
+const (
+	MethodDescriptorProto_Name_field_name            protoreflect.Name = "name"
+	MethodDescriptorProto_InputType_field_name       protoreflect.Name = "input_type"
+	MethodDescriptorProto_OutputType_field_name      protoreflect.Name = "output_type"
+	MethodDescriptorProto_Options_field_name         protoreflect.Name = "options"
+	MethodDescriptorProto_ClientStreaming_field_name protoreflect.Name = "client_streaming"
+	MethodDescriptorProto_ServerStreaming_field_name protoreflect.Name = "server_streaming"
+
+	MethodDescriptorProto_Name_field_fullname            protoreflect.FullName = "google.protobuf.MethodDescriptorProto.name"
+	MethodDescriptorProto_InputType_field_fullname       protoreflect.FullName = "google.protobuf.MethodDescriptorProto.input_type"
+	MethodDescriptorProto_OutputType_field_fullname      protoreflect.FullName = "google.protobuf.MethodDescriptorProto.output_type"
+	MethodDescriptorProto_Options_field_fullname         protoreflect.FullName = "google.protobuf.MethodDescriptorProto.options"
+	MethodDescriptorProto_ClientStreaming_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.client_streaming"
+	MethodDescriptorProto_ServerStreaming_field_fullname protoreflect.FullName = "google.protobuf.MethodDescriptorProto.server_streaming"
+)
+
+// Field numbers for google.protobuf.MethodDescriptorProto.
+const (
+	MethodDescriptorProto_Name_field_number            protoreflect.FieldNumber = 1
+	MethodDescriptorProto_InputType_field_number       protoreflect.FieldNumber = 2
+	MethodDescriptorProto_OutputType_field_number      protoreflect.FieldNumber = 3
+	MethodDescriptorProto_Options_field_number         protoreflect.FieldNumber = 4
+	MethodDescriptorProto_ClientStreaming_field_number protoreflect.FieldNumber = 5
+	MethodDescriptorProto_ServerStreaming_field_number protoreflect.FieldNumber = 6
+)
+
+// Names for google.protobuf.FileOptions.
+const (
+	FileOptions_message_name     protoreflect.Name     = "FileOptions"
+	FileOptions_message_fullname protoreflect.FullName = "google.protobuf.FileOptions"
+)
+
+// Field names for google.protobuf.FileOptions.
+const (
+	FileOptions_JavaPackage_field_name               protoreflect.Name = "java_package"
+	FileOptions_JavaOuterClassname_field_name        protoreflect.Name = "java_outer_classname"
+	FileOptions_JavaMultipleFiles_field_name         protoreflect.Name = "java_multiple_files"
+	FileOptions_JavaGenerateEqualsAndHash_field_name protoreflect.Name = "java_generate_equals_and_hash"
+	FileOptions_JavaStringCheckUtf8_field_name       protoreflect.Name = "java_string_check_utf8"
+	FileOptions_OptimizeFor_field_name               protoreflect.Name = "optimize_for"
+	FileOptions_GoPackage_field_name                 protoreflect.Name = "go_package"
+	FileOptions_CcGenericServices_field_name         protoreflect.Name = "cc_generic_services"
+	FileOptions_JavaGenericServices_field_name       protoreflect.Name = "java_generic_services"
+	FileOptions_PyGenericServices_field_name         protoreflect.Name = "py_generic_services"
+	FileOptions_PhpGenericServices_field_name        protoreflect.Name = "php_generic_services"
+	FileOptions_Deprecated_field_name                protoreflect.Name = "deprecated"
+	FileOptions_CcEnableArenas_field_name            protoreflect.Name = "cc_enable_arenas"
+	FileOptions_ObjcClassPrefix_field_name           protoreflect.Name = "objc_class_prefix"
+	FileOptions_CsharpNamespace_field_name           protoreflect.Name = "csharp_namespace"
+	FileOptions_SwiftPrefix_field_name               protoreflect.Name = "swift_prefix"
+	FileOptions_PhpClassPrefix_field_name            protoreflect.Name = "php_class_prefix"
+	FileOptions_PhpNamespace_field_name              protoreflect.Name = "php_namespace"
+	FileOptions_PhpMetadataNamespace_field_name      protoreflect.Name = "php_metadata_namespace"
+	FileOptions_RubyPackage_field_name               protoreflect.Name = "ruby_package"
+	FileOptions_UninterpretedOption_field_name       protoreflect.Name = "uninterpreted_option"
+
+	FileOptions_JavaPackage_field_fullname               protoreflect.FullName = "google.protobuf.FileOptions.java_package"
+	FileOptions_JavaOuterClassname_field_fullname        protoreflect.FullName = "google.protobuf.FileOptions.java_outer_classname"
+	FileOptions_JavaMultipleFiles_field_fullname         protoreflect.FullName = "google.protobuf.FileOptions.java_multiple_files"
+	FileOptions_JavaGenerateEqualsAndHash_field_fullname protoreflect.FullName = "google.protobuf.FileOptions.java_generate_equals_and_hash"
+	FileOptions_JavaStringCheckUtf8_field_fullname       protoreflect.FullName = "google.protobuf.FileOptions.java_string_check_utf8"
+	FileOptions_OptimizeFor_field_fullname               protoreflect.FullName = "google.protobuf.FileOptions.optimize_for"
+	FileOptions_GoPackage_field_fullname                 protoreflect.FullName = "google.protobuf.FileOptions.go_package"
+	FileOptions_CcGenericServices_field_fullname         protoreflect.FullName = "google.protobuf.FileOptions.cc_generic_services"
+	FileOptions_JavaGenericServices_field_fullname       protoreflect.FullName = "google.protobuf.FileOptions.java_generic_services"
+	FileOptions_PyGenericServices_field_fullname         protoreflect.FullName = "google.protobuf.FileOptions.py_generic_services"
+	FileOptions_PhpGenericServices_field_fullname        protoreflect.FullName = "google.protobuf.FileOptions.php_generic_services"
+	FileOptions_Deprecated_field_fullname                protoreflect.FullName = "google.protobuf.FileOptions.deprecated"
+	FileOptions_CcEnableArenas_field_fullname            protoreflect.FullName = "google.protobuf.FileOptions.cc_enable_arenas"
+	FileOptions_ObjcClassPrefix_field_fullname           protoreflect.FullName = "google.protobuf.FileOptions.objc_class_prefix"
+	FileOptions_CsharpNamespace_field_fullname           protoreflect.FullName = "google.protobuf.FileOptions.csharp_namespace"
+	FileOptions_SwiftPrefix_field_fullname               protoreflect.FullName = "google.protobuf.FileOptions.swift_prefix"
+	FileOptions_PhpClassPrefix_field_fullname            protoreflect.FullName = "google.protobuf.FileOptions.php_class_prefix"
+	FileOptions_PhpNamespace_field_fullname              protoreflect.FullName = "google.protobuf.FileOptions.php_namespace"
+	FileOptions_PhpMetadataNamespace_field_fullname      protoreflect.FullName = "google.protobuf.FileOptions.php_metadata_namespace"
+	FileOptions_RubyPackage_field_fullname               protoreflect.FullName = "google.protobuf.FileOptions.ruby_package"
+	FileOptions_UninterpretedOption_field_fullname       protoreflect.FullName = "google.protobuf.FileOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.FileOptions.
+const (
+	FileOptions_JavaPackage_field_number               protoreflect.FieldNumber = 1
+	FileOptions_JavaOuterClassname_field_number        protoreflect.FieldNumber = 8
+	FileOptions_JavaMultipleFiles_field_number         protoreflect.FieldNumber = 10
+	FileOptions_JavaGenerateEqualsAndHash_field_number protoreflect.FieldNumber = 20
+	FileOptions_JavaStringCheckUtf8_field_number       protoreflect.FieldNumber = 27
+	FileOptions_OptimizeFor_field_number               protoreflect.FieldNumber = 9
+	FileOptions_GoPackage_field_number                 protoreflect.FieldNumber = 11
+	FileOptions_CcGenericServices_field_number         protoreflect.FieldNumber = 16
+	FileOptions_JavaGenericServices_field_number       protoreflect.FieldNumber = 17
+	FileOptions_PyGenericServices_field_number         protoreflect.FieldNumber = 18
+	FileOptions_PhpGenericServices_field_number        protoreflect.FieldNumber = 42
+	FileOptions_Deprecated_field_number                protoreflect.FieldNumber = 23
+	FileOptions_CcEnableArenas_field_number            protoreflect.FieldNumber = 31
+	FileOptions_ObjcClassPrefix_field_number           protoreflect.FieldNumber = 36
+	FileOptions_CsharpNamespace_field_number           protoreflect.FieldNumber = 37
+	FileOptions_SwiftPrefix_field_number               protoreflect.FieldNumber = 39
+	FileOptions_PhpClassPrefix_field_number            protoreflect.FieldNumber = 40
+	FileOptions_PhpNamespace_field_number              protoreflect.FieldNumber = 41
+	FileOptions_PhpMetadataNamespace_field_number      protoreflect.FieldNumber = 44
+	FileOptions_RubyPackage_field_number               protoreflect.FieldNumber = 45
+	FileOptions_UninterpretedOption_field_number       protoreflect.FieldNumber = 999
+)
+
+// Full and short names for google.protobuf.FileOptions.OptimizeMode.
+const (
+	FileOptions_OptimizeMode_enum_fullname = "google.protobuf.FileOptions.OptimizeMode"
+	FileOptions_OptimizeMode_enum_name     = "OptimizeMode"
+)
+
+// Names for google.protobuf.MessageOptions.
+const (
+	MessageOptions_message_name     protoreflect.Name     = "MessageOptions"
+	MessageOptions_message_fullname protoreflect.FullName = "google.protobuf.MessageOptions"
+)
+
+// Field names for google.protobuf.MessageOptions.
+const (
+	MessageOptions_MessageSetWireFormat_field_name         protoreflect.Name = "message_set_wire_format"
+	MessageOptions_NoStandardDescriptorAccessor_field_name protoreflect.Name = "no_standard_descriptor_accessor"
+	MessageOptions_Deprecated_field_name                   protoreflect.Name = "deprecated"
+	MessageOptions_MapEntry_field_name                     protoreflect.Name = "map_entry"
+	MessageOptions_UninterpretedOption_field_name          protoreflect.Name = "uninterpreted_option"
+
+	MessageOptions_MessageSetWireFormat_field_fullname         protoreflect.FullName = "google.protobuf.MessageOptions.message_set_wire_format"
+	MessageOptions_NoStandardDescriptorAccessor_field_fullname protoreflect.FullName = "google.protobuf.MessageOptions.no_standard_descriptor_accessor"
+	MessageOptions_Deprecated_field_fullname                   protoreflect.FullName = "google.protobuf.MessageOptions.deprecated"
+	MessageOptions_MapEntry_field_fullname                     protoreflect.FullName = "google.protobuf.MessageOptions.map_entry"
+	MessageOptions_UninterpretedOption_field_fullname          protoreflect.FullName = "google.protobuf.MessageOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.MessageOptions.
+const (
+	MessageOptions_MessageSetWireFormat_field_number         protoreflect.FieldNumber = 1
+	MessageOptions_NoStandardDescriptorAccessor_field_number protoreflect.FieldNumber = 2
+	MessageOptions_Deprecated_field_number                   protoreflect.FieldNumber = 3
+	MessageOptions_MapEntry_field_number                     protoreflect.FieldNumber = 7
+	MessageOptions_UninterpretedOption_field_number          protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.FieldOptions.
+const (
+	FieldOptions_message_name     protoreflect.Name     = "FieldOptions"
+	FieldOptions_message_fullname protoreflect.FullName = "google.protobuf.FieldOptions"
+)
+
+// Field names for google.protobuf.FieldOptions.
+const (
+	FieldOptions_Ctype_field_name               protoreflect.Name = "ctype"
+	FieldOptions_Packed_field_name              protoreflect.Name = "packed"
+	FieldOptions_Jstype_field_name              protoreflect.Name = "jstype"
+	FieldOptions_Lazy_field_name                protoreflect.Name = "lazy"
+	FieldOptions_Deprecated_field_name          protoreflect.Name = "deprecated"
+	FieldOptions_Weak_field_name                protoreflect.Name = "weak"
+	FieldOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	FieldOptions_Ctype_field_fullname               protoreflect.FullName = "google.protobuf.FieldOptions.ctype"
+	FieldOptions_Packed_field_fullname              protoreflect.FullName = "google.protobuf.FieldOptions.packed"
+	FieldOptions_Jstype_field_fullname              protoreflect.FullName = "google.protobuf.FieldOptions.jstype"
+	FieldOptions_Lazy_field_fullname                protoreflect.FullName = "google.protobuf.FieldOptions.lazy"
+	FieldOptions_Deprecated_field_fullname          protoreflect.FullName = "google.protobuf.FieldOptions.deprecated"
+	FieldOptions_Weak_field_fullname                protoreflect.FullName = "google.protobuf.FieldOptions.weak"
+	FieldOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.FieldOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.FieldOptions.
+const (
+	FieldOptions_Ctype_field_number               protoreflect.FieldNumber = 1
+	FieldOptions_Packed_field_number              protoreflect.FieldNumber = 2
+	FieldOptions_Jstype_field_number              protoreflect.FieldNumber = 6
+	FieldOptions_Lazy_field_number                protoreflect.FieldNumber = 5
+	FieldOptions_Deprecated_field_number          protoreflect.FieldNumber = 3
+	FieldOptions_Weak_field_number                protoreflect.FieldNumber = 10
+	FieldOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Full and short names for google.protobuf.FieldOptions.CType.
+const (
+	FieldOptions_CType_enum_fullname = "google.protobuf.FieldOptions.CType"
+	FieldOptions_CType_enum_name     = "CType"
+)
+
+// Full and short names for google.protobuf.FieldOptions.JSType.
+const (
+	FieldOptions_JSType_enum_fullname = "google.protobuf.FieldOptions.JSType"
+	FieldOptions_JSType_enum_name     = "JSType"
+)
+
+// Names for google.protobuf.OneofOptions.
+const (
+	OneofOptions_message_name     protoreflect.Name     = "OneofOptions"
+	OneofOptions_message_fullname protoreflect.FullName = "google.protobuf.OneofOptions"
+)
+
+// Field names for google.protobuf.OneofOptions.
+const (
+	OneofOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	OneofOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.OneofOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.OneofOptions.
+const (
+	OneofOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.EnumOptions.
+const (
+	EnumOptions_message_name     protoreflect.Name     = "EnumOptions"
+	EnumOptions_message_fullname protoreflect.FullName = "google.protobuf.EnumOptions"
+)
+
+// Field names for google.protobuf.EnumOptions.
+const (
+	EnumOptions_AllowAlias_field_name          protoreflect.Name = "allow_alias"
+	EnumOptions_Deprecated_field_name          protoreflect.Name = "deprecated"
+	EnumOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	EnumOptions_AllowAlias_field_fullname          protoreflect.FullName = "google.protobuf.EnumOptions.allow_alias"
+	EnumOptions_Deprecated_field_fullname          protoreflect.FullName = "google.protobuf.EnumOptions.deprecated"
+	EnumOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.EnumOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.EnumOptions.
+const (
+	EnumOptions_AllowAlias_field_number          protoreflect.FieldNumber = 2
+	EnumOptions_Deprecated_field_number          protoreflect.FieldNumber = 3
+	EnumOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.EnumValueOptions.
+const (
+	EnumValueOptions_message_name     protoreflect.Name     = "EnumValueOptions"
+	EnumValueOptions_message_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions"
+)
+
+// Field names for google.protobuf.EnumValueOptions.
+const (
+	EnumValueOptions_Deprecated_field_name          protoreflect.Name = "deprecated"
+	EnumValueOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	EnumValueOptions_Deprecated_field_fullname          protoreflect.FullName = "google.protobuf.EnumValueOptions.deprecated"
+	EnumValueOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.EnumValueOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.EnumValueOptions.
+const (
+	EnumValueOptions_Deprecated_field_number          protoreflect.FieldNumber = 1
+	EnumValueOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.ServiceOptions.
+const (
+	ServiceOptions_message_name     protoreflect.Name     = "ServiceOptions"
+	ServiceOptions_message_fullname protoreflect.FullName = "google.protobuf.ServiceOptions"
+)
+
+// Field names for google.protobuf.ServiceOptions.
+const (
+	ServiceOptions_Deprecated_field_name          protoreflect.Name = "deprecated"
+	ServiceOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	ServiceOptions_Deprecated_field_fullname          protoreflect.FullName = "google.protobuf.ServiceOptions.deprecated"
+	ServiceOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.ServiceOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.ServiceOptions.
+const (
+	ServiceOptions_Deprecated_field_number          protoreflect.FieldNumber = 33
+	ServiceOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Names for google.protobuf.MethodOptions.
+const (
+	MethodOptions_message_name     protoreflect.Name     = "MethodOptions"
+	MethodOptions_message_fullname protoreflect.FullName = "google.protobuf.MethodOptions"
+)
+
+// Field names for google.protobuf.MethodOptions.
+const (
+	MethodOptions_Deprecated_field_name          protoreflect.Name = "deprecated"
+	MethodOptions_IdempotencyLevel_field_name    protoreflect.Name = "idempotency_level"
+	MethodOptions_UninterpretedOption_field_name protoreflect.Name = "uninterpreted_option"
+
+	MethodOptions_Deprecated_field_fullname          protoreflect.FullName = "google.protobuf.MethodOptions.deprecated"
+	MethodOptions_IdempotencyLevel_field_fullname    protoreflect.FullName = "google.protobuf.MethodOptions.idempotency_level"
+	MethodOptions_UninterpretedOption_field_fullname protoreflect.FullName = "google.protobuf.MethodOptions.uninterpreted_option"
+)
+
+// Field numbers for google.protobuf.MethodOptions.
+const (
+	MethodOptions_Deprecated_field_number          protoreflect.FieldNumber = 33
+	MethodOptions_IdempotencyLevel_field_number    protoreflect.FieldNumber = 34
+	MethodOptions_UninterpretedOption_field_number protoreflect.FieldNumber = 999
+)
+
+// Full and short names for google.protobuf.MethodOptions.IdempotencyLevel.
+const (
+	MethodOptions_IdempotencyLevel_enum_fullname = "google.protobuf.MethodOptions.IdempotencyLevel"
+	MethodOptions_IdempotencyLevel_enum_name     = "IdempotencyLevel"
+)
+
+// Names for google.protobuf.UninterpretedOption.
+const (
+	UninterpretedOption_message_name     protoreflect.Name     = "UninterpretedOption"
+	UninterpretedOption_message_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption"
+)
+
+// Field names for google.protobuf.UninterpretedOption.
+const (
+	UninterpretedOption_Name_field_name             protoreflect.Name = "name"
+	UninterpretedOption_IdentifierValue_field_name  protoreflect.Name = "identifier_value"
+	UninterpretedOption_PositiveIntValue_field_name protoreflect.Name = "positive_int_value"
+	UninterpretedOption_NegativeIntValue_field_name protoreflect.Name = "negative_int_value"
+	UninterpretedOption_DoubleValue_field_name      protoreflect.Name = "double_value"
+	UninterpretedOption_StringValue_field_name      protoreflect.Name = "string_value"
+	UninterpretedOption_AggregateValue_field_name   protoreflect.Name = "aggregate_value"
+
+	UninterpretedOption_Name_field_fullname             protoreflect.FullName = "google.protobuf.UninterpretedOption.name"
+	UninterpretedOption_IdentifierValue_field_fullname  protoreflect.FullName = "google.protobuf.UninterpretedOption.identifier_value"
+	UninterpretedOption_PositiveIntValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.positive_int_value"
+	UninterpretedOption_NegativeIntValue_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.negative_int_value"
+	UninterpretedOption_DoubleValue_field_fullname      protoreflect.FullName = "google.protobuf.UninterpretedOption.double_value"
+	UninterpretedOption_StringValue_field_fullname      protoreflect.FullName = "google.protobuf.UninterpretedOption.string_value"
+	UninterpretedOption_AggregateValue_field_fullname   protoreflect.FullName = "google.protobuf.UninterpretedOption.aggregate_value"
+)
+
+// Field numbers for google.protobuf.UninterpretedOption.
+const (
+	UninterpretedOption_Name_field_number             protoreflect.FieldNumber = 2
+	UninterpretedOption_IdentifierValue_field_number  protoreflect.FieldNumber = 3
+	UninterpretedOption_PositiveIntValue_field_number protoreflect.FieldNumber = 4
+	UninterpretedOption_NegativeIntValue_field_number protoreflect.FieldNumber = 5
+	UninterpretedOption_DoubleValue_field_number      protoreflect.FieldNumber = 6
+	UninterpretedOption_StringValue_field_number      protoreflect.FieldNumber = 7
+	UninterpretedOption_AggregateValue_field_number   protoreflect.FieldNumber = 8
+)
+
+// Names for google.protobuf.UninterpretedOption.NamePart.
+const (
+	UninterpretedOption_NamePart_message_name     protoreflect.Name     = "NamePart"
+	UninterpretedOption_NamePart_message_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart"
+)
+
+// Field names for google.protobuf.UninterpretedOption.NamePart.
+const (
+	UninterpretedOption_NamePart_NamePart_field_name    protoreflect.Name = "name_part"
+	UninterpretedOption_NamePart_IsExtension_field_name protoreflect.Name = "is_extension"
+
+	UninterpretedOption_NamePart_NamePart_field_fullname    protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart.name_part"
+	UninterpretedOption_NamePart_IsExtension_field_fullname protoreflect.FullName = "google.protobuf.UninterpretedOption.NamePart.is_extension"
+)
+
+// Field numbers for google.protobuf.UninterpretedOption.NamePart.
+const (
+	UninterpretedOption_NamePart_NamePart_field_number    protoreflect.FieldNumber = 1
+	UninterpretedOption_NamePart_IsExtension_field_number protoreflect.FieldNumber = 2
+)
+
+// Names for google.protobuf.SourceCodeInfo.
+const (
+	SourceCodeInfo_message_name     protoreflect.Name     = "SourceCodeInfo"
+	SourceCodeInfo_message_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo"
+)
+
+// Field names for google.protobuf.SourceCodeInfo.
+const (
+	SourceCodeInfo_Location_field_name protoreflect.Name = "location"
+
+	SourceCodeInfo_Location_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.location"
+)
+
+// Field numbers for google.protobuf.SourceCodeInfo.
+const (
+	SourceCodeInfo_Location_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.SourceCodeInfo.Location.
+const (
+	SourceCodeInfo_Location_message_name     protoreflect.Name     = "Location"
+	SourceCodeInfo_Location_message_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location"
+)
+
+// Field names for google.protobuf.SourceCodeInfo.Location.
+const (
+	SourceCodeInfo_Location_Path_field_name                    protoreflect.Name = "path"
+	SourceCodeInfo_Location_Span_field_name                    protoreflect.Name = "span"
+	SourceCodeInfo_Location_LeadingComments_field_name         protoreflect.Name = "leading_comments"
+	SourceCodeInfo_Location_TrailingComments_field_name        protoreflect.Name = "trailing_comments"
+	SourceCodeInfo_Location_LeadingDetachedComments_field_name protoreflect.Name = "leading_detached_comments"
+
+	SourceCodeInfo_Location_Path_field_fullname                    protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.path"
+	SourceCodeInfo_Location_Span_field_fullname                    protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.span"
+	SourceCodeInfo_Location_LeadingComments_field_fullname         protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.leading_comments"
+	SourceCodeInfo_Location_TrailingComments_field_fullname        protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.trailing_comments"
+	SourceCodeInfo_Location_LeadingDetachedComments_field_fullname protoreflect.FullName = "google.protobuf.SourceCodeInfo.Location.leading_detached_comments"
+)
+
+// Field numbers for google.protobuf.SourceCodeInfo.Location.
+const (
+	SourceCodeInfo_Location_Path_field_number                    protoreflect.FieldNumber = 1
+	SourceCodeInfo_Location_Span_field_number                    protoreflect.FieldNumber = 2
+	SourceCodeInfo_Location_LeadingComments_field_number         protoreflect.FieldNumber = 3
+	SourceCodeInfo_Location_TrailingComments_field_number        protoreflect.FieldNumber = 4
+	SourceCodeInfo_Location_LeadingDetachedComments_field_number protoreflect.FieldNumber = 6
+)
+
+// Names for google.protobuf.GeneratedCodeInfo.
+const (
+	GeneratedCodeInfo_message_name     protoreflect.Name     = "GeneratedCodeInfo"
+	GeneratedCodeInfo_message_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo"
+)
+
+// Field names for google.protobuf.GeneratedCodeInfo.
+const (
+	GeneratedCodeInfo_Annotation_field_name protoreflect.Name = "annotation"
+
+	GeneratedCodeInfo_Annotation_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.annotation"
+)
+
+// Field numbers for google.protobuf.GeneratedCodeInfo.
+const (
+	GeneratedCodeInfo_Annotation_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.GeneratedCodeInfo.Annotation.
+const (
+	GeneratedCodeInfo_Annotation_message_name     protoreflect.Name     = "Annotation"
+	GeneratedCodeInfo_Annotation_message_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation"
+)
+
+// Field names for google.protobuf.GeneratedCodeInfo.Annotation.
+const (
+	GeneratedCodeInfo_Annotation_Path_field_name       protoreflect.Name = "path"
+	GeneratedCodeInfo_Annotation_SourceFile_field_name protoreflect.Name = "source_file"
+	GeneratedCodeInfo_Annotation_Begin_field_name      protoreflect.Name = "begin"
+	GeneratedCodeInfo_Annotation_End_field_name        protoreflect.Name = "end"
+
+	GeneratedCodeInfo_Annotation_Path_field_fullname       protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.path"
+	GeneratedCodeInfo_Annotation_SourceFile_field_fullname protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.source_file"
+	GeneratedCodeInfo_Annotation_Begin_field_fullname      protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.begin"
+	GeneratedCodeInfo_Annotation_End_field_fullname        protoreflect.FullName = "google.protobuf.GeneratedCodeInfo.Annotation.end"
+)
+
+// Field numbers for google.protobuf.GeneratedCodeInfo.Annotation.
+const (
+	GeneratedCodeInfo_Annotation_Path_field_number       protoreflect.FieldNumber = 1
+	GeneratedCodeInfo_Annotation_SourceFile_field_number protoreflect.FieldNumber = 2
+	GeneratedCodeInfo_Annotation_Begin_field_number      protoreflect.FieldNumber = 3
+	GeneratedCodeInfo_Annotation_End_field_number        protoreflect.FieldNumber = 4
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/doc.go b/vendor/google.golang.org/protobuf/internal/genid/doc.go
new file mode 100644
index 0000000..45ccd01
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/doc.go
@@ -0,0 +1,11 @@
+// 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.
+
+// Package genid contains constants for declarations in descriptor.proto
+// and the well-known types.
+package genid
+
+import protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+const GoogleProtobuf_package protoreflect.FullName = "google.protobuf"
diff --git a/vendor/google.golang.org/protobuf/internal/genid/duration_gen.go b/vendor/google.golang.org/protobuf/internal/genid/duration_gen.go
new file mode 100644
index 0000000..b070ef4
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/duration_gen.go
@@ -0,0 +1,34 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_duration_proto = "google/protobuf/duration.proto"
+
+// Names for google.protobuf.Duration.
+const (
+	Duration_message_name     protoreflect.Name     = "Duration"
+	Duration_message_fullname protoreflect.FullName = "google.protobuf.Duration"
+)
+
+// Field names for google.protobuf.Duration.
+const (
+	Duration_Seconds_field_name protoreflect.Name = "seconds"
+	Duration_Nanos_field_name   protoreflect.Name = "nanos"
+
+	Duration_Seconds_field_fullname protoreflect.FullName = "google.protobuf.Duration.seconds"
+	Duration_Nanos_field_fullname   protoreflect.FullName = "google.protobuf.Duration.nanos"
+)
+
+// Field numbers for google.protobuf.Duration.
+const (
+	Duration_Seconds_field_number protoreflect.FieldNumber = 1
+	Duration_Nanos_field_number   protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/empty_gen.go b/vendor/google.golang.org/protobuf/internal/genid/empty_gen.go
new file mode 100644
index 0000000..762abb3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/empty_gen.go
@@ -0,0 +1,19 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_empty_proto = "google/protobuf/empty.proto"
+
+// Names for google.protobuf.Empty.
+const (
+	Empty_message_name     protoreflect.Name     = "Empty"
+	Empty_message_fullname protoreflect.FullName = "google.protobuf.Empty"
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/field_mask_gen.go b/vendor/google.golang.org/protobuf/internal/genid/field_mask_gen.go
new file mode 100644
index 0000000..70bed45
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/field_mask_gen.go
@@ -0,0 +1,31 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_field_mask_proto = "google/protobuf/field_mask.proto"
+
+// Names for google.protobuf.FieldMask.
+const (
+	FieldMask_message_name     protoreflect.Name     = "FieldMask"
+	FieldMask_message_fullname protoreflect.FullName = "google.protobuf.FieldMask"
+)
+
+// Field names for google.protobuf.FieldMask.
+const (
+	FieldMask_Paths_field_name protoreflect.Name = "paths"
+
+	FieldMask_Paths_field_fullname protoreflect.FullName = "google.protobuf.FieldMask.paths"
+)
+
+// Field numbers for google.protobuf.FieldMask.
+const (
+	FieldMask_Paths_field_number protoreflect.FieldNumber = 1
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/goname.go b/vendor/google.golang.org/protobuf/internal/genid/goname.go
new file mode 100644
index 0000000..693d2e9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/goname.go
@@ -0,0 +1,25 @@
+// 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.
+
+package genid
+
+// Go names of implementation-specific struct fields in generated messages.
+const (
+	State_goname = "state"
+
+	SizeCache_goname  = "sizeCache"
+	SizeCacheA_goname = "XXX_sizecache"
+
+	WeakFields_goname  = "weakFields"
+	WeakFieldsA_goname = "XXX_weak"
+
+	UnknownFields_goname  = "unknownFields"
+	UnknownFieldsA_goname = "XXX_unrecognized"
+
+	ExtensionFields_goname  = "extensionFields"
+	ExtensionFieldsA_goname = "XXX_InternalExtensions"
+	ExtensionFieldsB_goname = "XXX_extensions"
+
+	WeakFieldPrefix_goname = "XXX_weak_"
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/map_entry.go b/vendor/google.golang.org/protobuf/internal/genid/map_entry.go
new file mode 100644
index 0000000..8f9ea02
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/map_entry.go
@@ -0,0 +1,16 @@
+// 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.
+
+package genid
+
+import protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+// Generic field names and numbers for synthetic map entry messages.
+const (
+	MapEntry_Key_field_name   protoreflect.Name = "key"
+	MapEntry_Value_field_name protoreflect.Name = "value"
+
+	MapEntry_Key_field_number   protoreflect.FieldNumber = 1
+	MapEntry_Value_field_number protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/source_context_gen.go b/vendor/google.golang.org/protobuf/internal/genid/source_context_gen.go
new file mode 100644
index 0000000..3e99ae1
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/source_context_gen.go
@@ -0,0 +1,31 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_source_context_proto = "google/protobuf/source_context.proto"
+
+// Names for google.protobuf.SourceContext.
+const (
+	SourceContext_message_name     protoreflect.Name     = "SourceContext"
+	SourceContext_message_fullname protoreflect.FullName = "google.protobuf.SourceContext"
+)
+
+// Field names for google.protobuf.SourceContext.
+const (
+	SourceContext_FileName_field_name protoreflect.Name = "file_name"
+
+	SourceContext_FileName_field_fullname protoreflect.FullName = "google.protobuf.SourceContext.file_name"
+)
+
+// Field numbers for google.protobuf.SourceContext.
+const (
+	SourceContext_FileName_field_number protoreflect.FieldNumber = 1
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/struct_gen.go b/vendor/google.golang.org/protobuf/internal/genid/struct_gen.go
new file mode 100644
index 0000000..1a38944
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/struct_gen.go
@@ -0,0 +1,116 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_struct_proto = "google/protobuf/struct.proto"
+
+// Full and short names for google.protobuf.NullValue.
+const (
+	NullValue_enum_fullname = "google.protobuf.NullValue"
+	NullValue_enum_name     = "NullValue"
+)
+
+// Names for google.protobuf.Struct.
+const (
+	Struct_message_name     protoreflect.Name     = "Struct"
+	Struct_message_fullname protoreflect.FullName = "google.protobuf.Struct"
+)
+
+// Field names for google.protobuf.Struct.
+const (
+	Struct_Fields_field_name protoreflect.Name = "fields"
+
+	Struct_Fields_field_fullname protoreflect.FullName = "google.protobuf.Struct.fields"
+)
+
+// Field numbers for google.protobuf.Struct.
+const (
+	Struct_Fields_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.Struct.FieldsEntry.
+const (
+	Struct_FieldsEntry_message_name     protoreflect.Name     = "FieldsEntry"
+	Struct_FieldsEntry_message_fullname protoreflect.FullName = "google.protobuf.Struct.FieldsEntry"
+)
+
+// Field names for google.protobuf.Struct.FieldsEntry.
+const (
+	Struct_FieldsEntry_Key_field_name   protoreflect.Name = "key"
+	Struct_FieldsEntry_Value_field_name protoreflect.Name = "value"
+
+	Struct_FieldsEntry_Key_field_fullname   protoreflect.FullName = "google.protobuf.Struct.FieldsEntry.key"
+	Struct_FieldsEntry_Value_field_fullname protoreflect.FullName = "google.protobuf.Struct.FieldsEntry.value"
+)
+
+// Field numbers for google.protobuf.Struct.FieldsEntry.
+const (
+	Struct_FieldsEntry_Key_field_number   protoreflect.FieldNumber = 1
+	Struct_FieldsEntry_Value_field_number protoreflect.FieldNumber = 2
+)
+
+// Names for google.protobuf.Value.
+const (
+	Value_message_name     protoreflect.Name     = "Value"
+	Value_message_fullname protoreflect.FullName = "google.protobuf.Value"
+)
+
+// Field names for google.protobuf.Value.
+const (
+	Value_NullValue_field_name   protoreflect.Name = "null_value"
+	Value_NumberValue_field_name protoreflect.Name = "number_value"
+	Value_StringValue_field_name protoreflect.Name = "string_value"
+	Value_BoolValue_field_name   protoreflect.Name = "bool_value"
+	Value_StructValue_field_name protoreflect.Name = "struct_value"
+	Value_ListValue_field_name   protoreflect.Name = "list_value"
+
+	Value_NullValue_field_fullname   protoreflect.FullName = "google.protobuf.Value.null_value"
+	Value_NumberValue_field_fullname protoreflect.FullName = "google.protobuf.Value.number_value"
+	Value_StringValue_field_fullname protoreflect.FullName = "google.protobuf.Value.string_value"
+	Value_BoolValue_field_fullname   protoreflect.FullName = "google.protobuf.Value.bool_value"
+	Value_StructValue_field_fullname protoreflect.FullName = "google.protobuf.Value.struct_value"
+	Value_ListValue_field_fullname   protoreflect.FullName = "google.protobuf.Value.list_value"
+)
+
+// Field numbers for google.protobuf.Value.
+const (
+	Value_NullValue_field_number   protoreflect.FieldNumber = 1
+	Value_NumberValue_field_number protoreflect.FieldNumber = 2
+	Value_StringValue_field_number protoreflect.FieldNumber = 3
+	Value_BoolValue_field_number   protoreflect.FieldNumber = 4
+	Value_StructValue_field_number protoreflect.FieldNumber = 5
+	Value_ListValue_field_number   protoreflect.FieldNumber = 6
+)
+
+// Oneof names for google.protobuf.Value.
+const (
+	Value_Kind_oneof_name protoreflect.Name = "kind"
+
+	Value_Kind_oneof_fullname protoreflect.FullName = "google.protobuf.Value.kind"
+)
+
+// Names for google.protobuf.ListValue.
+const (
+	ListValue_message_name     protoreflect.Name     = "ListValue"
+	ListValue_message_fullname protoreflect.FullName = "google.protobuf.ListValue"
+)
+
+// Field names for google.protobuf.ListValue.
+const (
+	ListValue_Values_field_name protoreflect.Name = "values"
+
+	ListValue_Values_field_fullname protoreflect.FullName = "google.protobuf.ListValue.values"
+)
+
+// Field numbers for google.protobuf.ListValue.
+const (
+	ListValue_Values_field_number protoreflect.FieldNumber = 1
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/timestamp_gen.go b/vendor/google.golang.org/protobuf/internal/genid/timestamp_gen.go
new file mode 100644
index 0000000..f5cd563
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/timestamp_gen.go
@@ -0,0 +1,34 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_timestamp_proto = "google/protobuf/timestamp.proto"
+
+// Names for google.protobuf.Timestamp.
+const (
+	Timestamp_message_name     protoreflect.Name     = "Timestamp"
+	Timestamp_message_fullname protoreflect.FullName = "google.protobuf.Timestamp"
+)
+
+// Field names for google.protobuf.Timestamp.
+const (
+	Timestamp_Seconds_field_name protoreflect.Name = "seconds"
+	Timestamp_Nanos_field_name   protoreflect.Name = "nanos"
+
+	Timestamp_Seconds_field_fullname protoreflect.FullName = "google.protobuf.Timestamp.seconds"
+	Timestamp_Nanos_field_fullname   protoreflect.FullName = "google.protobuf.Timestamp.nanos"
+)
+
+// Field numbers for google.protobuf.Timestamp.
+const (
+	Timestamp_Seconds_field_number protoreflect.FieldNumber = 1
+	Timestamp_Nanos_field_number   protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/type_gen.go b/vendor/google.golang.org/protobuf/internal/genid/type_gen.go
new file mode 100644
index 0000000..3bc7101
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/type_gen.go
@@ -0,0 +1,184 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_type_proto = "google/protobuf/type.proto"
+
+// Full and short names for google.protobuf.Syntax.
+const (
+	Syntax_enum_fullname = "google.protobuf.Syntax"
+	Syntax_enum_name     = "Syntax"
+)
+
+// Names for google.protobuf.Type.
+const (
+	Type_message_name     protoreflect.Name     = "Type"
+	Type_message_fullname protoreflect.FullName = "google.protobuf.Type"
+)
+
+// Field names for google.protobuf.Type.
+const (
+	Type_Name_field_name          protoreflect.Name = "name"
+	Type_Fields_field_name        protoreflect.Name = "fields"
+	Type_Oneofs_field_name        protoreflect.Name = "oneofs"
+	Type_Options_field_name       protoreflect.Name = "options"
+	Type_SourceContext_field_name protoreflect.Name = "source_context"
+	Type_Syntax_field_name        protoreflect.Name = "syntax"
+
+	Type_Name_field_fullname          protoreflect.FullName = "google.protobuf.Type.name"
+	Type_Fields_field_fullname        protoreflect.FullName = "google.protobuf.Type.fields"
+	Type_Oneofs_field_fullname        protoreflect.FullName = "google.protobuf.Type.oneofs"
+	Type_Options_field_fullname       protoreflect.FullName = "google.protobuf.Type.options"
+	Type_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Type.source_context"
+	Type_Syntax_field_fullname        protoreflect.FullName = "google.protobuf.Type.syntax"
+)
+
+// Field numbers for google.protobuf.Type.
+const (
+	Type_Name_field_number          protoreflect.FieldNumber = 1
+	Type_Fields_field_number        protoreflect.FieldNumber = 2
+	Type_Oneofs_field_number        protoreflect.FieldNumber = 3
+	Type_Options_field_number       protoreflect.FieldNumber = 4
+	Type_SourceContext_field_number protoreflect.FieldNumber = 5
+	Type_Syntax_field_number        protoreflect.FieldNumber = 6
+)
+
+// Names for google.protobuf.Field.
+const (
+	Field_message_name     protoreflect.Name     = "Field"
+	Field_message_fullname protoreflect.FullName = "google.protobuf.Field"
+)
+
+// Field names for google.protobuf.Field.
+const (
+	Field_Kind_field_name         protoreflect.Name = "kind"
+	Field_Cardinality_field_name  protoreflect.Name = "cardinality"
+	Field_Number_field_name       protoreflect.Name = "number"
+	Field_Name_field_name         protoreflect.Name = "name"
+	Field_TypeUrl_field_name      protoreflect.Name = "type_url"
+	Field_OneofIndex_field_name   protoreflect.Name = "oneof_index"
+	Field_Packed_field_name       protoreflect.Name = "packed"
+	Field_Options_field_name      protoreflect.Name = "options"
+	Field_JsonName_field_name     protoreflect.Name = "json_name"
+	Field_DefaultValue_field_name protoreflect.Name = "default_value"
+
+	Field_Kind_field_fullname         protoreflect.FullName = "google.protobuf.Field.kind"
+	Field_Cardinality_field_fullname  protoreflect.FullName = "google.protobuf.Field.cardinality"
+	Field_Number_field_fullname       protoreflect.FullName = "google.protobuf.Field.number"
+	Field_Name_field_fullname         protoreflect.FullName = "google.protobuf.Field.name"
+	Field_TypeUrl_field_fullname      protoreflect.FullName = "google.protobuf.Field.type_url"
+	Field_OneofIndex_field_fullname   protoreflect.FullName = "google.protobuf.Field.oneof_index"
+	Field_Packed_field_fullname       protoreflect.FullName = "google.protobuf.Field.packed"
+	Field_Options_field_fullname      protoreflect.FullName = "google.protobuf.Field.options"
+	Field_JsonName_field_fullname     protoreflect.FullName = "google.protobuf.Field.json_name"
+	Field_DefaultValue_field_fullname protoreflect.FullName = "google.protobuf.Field.default_value"
+)
+
+// Field numbers for google.protobuf.Field.
+const (
+	Field_Kind_field_number         protoreflect.FieldNumber = 1
+	Field_Cardinality_field_number  protoreflect.FieldNumber = 2
+	Field_Number_field_number       protoreflect.FieldNumber = 3
+	Field_Name_field_number         protoreflect.FieldNumber = 4
+	Field_TypeUrl_field_number      protoreflect.FieldNumber = 6
+	Field_OneofIndex_field_number   protoreflect.FieldNumber = 7
+	Field_Packed_field_number       protoreflect.FieldNumber = 8
+	Field_Options_field_number      protoreflect.FieldNumber = 9
+	Field_JsonName_field_number     protoreflect.FieldNumber = 10
+	Field_DefaultValue_field_number protoreflect.FieldNumber = 11
+)
+
+// Full and short names for google.protobuf.Field.Kind.
+const (
+	Field_Kind_enum_fullname = "google.protobuf.Field.Kind"
+	Field_Kind_enum_name     = "Kind"
+)
+
+// Full and short names for google.protobuf.Field.Cardinality.
+const (
+	Field_Cardinality_enum_fullname = "google.protobuf.Field.Cardinality"
+	Field_Cardinality_enum_name     = "Cardinality"
+)
+
+// Names for google.protobuf.Enum.
+const (
+	Enum_message_name     protoreflect.Name     = "Enum"
+	Enum_message_fullname protoreflect.FullName = "google.protobuf.Enum"
+)
+
+// Field names for google.protobuf.Enum.
+const (
+	Enum_Name_field_name          protoreflect.Name = "name"
+	Enum_Enumvalue_field_name     protoreflect.Name = "enumvalue"
+	Enum_Options_field_name       protoreflect.Name = "options"
+	Enum_SourceContext_field_name protoreflect.Name = "source_context"
+	Enum_Syntax_field_name        protoreflect.Name = "syntax"
+
+	Enum_Name_field_fullname          protoreflect.FullName = "google.protobuf.Enum.name"
+	Enum_Enumvalue_field_fullname     protoreflect.FullName = "google.protobuf.Enum.enumvalue"
+	Enum_Options_field_fullname       protoreflect.FullName = "google.protobuf.Enum.options"
+	Enum_SourceContext_field_fullname protoreflect.FullName = "google.protobuf.Enum.source_context"
+	Enum_Syntax_field_fullname        protoreflect.FullName = "google.protobuf.Enum.syntax"
+)
+
+// Field numbers for google.protobuf.Enum.
+const (
+	Enum_Name_field_number          protoreflect.FieldNumber = 1
+	Enum_Enumvalue_field_number     protoreflect.FieldNumber = 2
+	Enum_Options_field_number       protoreflect.FieldNumber = 3
+	Enum_SourceContext_field_number protoreflect.FieldNumber = 4
+	Enum_Syntax_field_number        protoreflect.FieldNumber = 5
+)
+
+// Names for google.protobuf.EnumValue.
+const (
+	EnumValue_message_name     protoreflect.Name     = "EnumValue"
+	EnumValue_message_fullname protoreflect.FullName = "google.protobuf.EnumValue"
+)
+
+// Field names for google.protobuf.EnumValue.
+const (
+	EnumValue_Name_field_name    protoreflect.Name = "name"
+	EnumValue_Number_field_name  protoreflect.Name = "number"
+	EnumValue_Options_field_name protoreflect.Name = "options"
+
+	EnumValue_Name_field_fullname    protoreflect.FullName = "google.protobuf.EnumValue.name"
+	EnumValue_Number_field_fullname  protoreflect.FullName = "google.protobuf.EnumValue.number"
+	EnumValue_Options_field_fullname protoreflect.FullName = "google.protobuf.EnumValue.options"
+)
+
+// Field numbers for google.protobuf.EnumValue.
+const (
+	EnumValue_Name_field_number    protoreflect.FieldNumber = 1
+	EnumValue_Number_field_number  protoreflect.FieldNumber = 2
+	EnumValue_Options_field_number protoreflect.FieldNumber = 3
+)
+
+// Names for google.protobuf.Option.
+const (
+	Option_message_name     protoreflect.Name     = "Option"
+	Option_message_fullname protoreflect.FullName = "google.protobuf.Option"
+)
+
+// Field names for google.protobuf.Option.
+const (
+	Option_Name_field_name  protoreflect.Name = "name"
+	Option_Value_field_name protoreflect.Name = "value"
+
+	Option_Name_field_fullname  protoreflect.FullName = "google.protobuf.Option.name"
+	Option_Value_field_fullname protoreflect.FullName = "google.protobuf.Option.value"
+)
+
+// Field numbers for google.protobuf.Option.
+const (
+	Option_Name_field_number  protoreflect.FieldNumber = 1
+	Option_Value_field_number protoreflect.FieldNumber = 2
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/wrappers.go b/vendor/google.golang.org/protobuf/internal/genid/wrappers.go
new file mode 100644
index 0000000..429384b
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/wrappers.go
@@ -0,0 +1,13 @@
+// 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.
+
+package genid
+
+import protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+// Generic field name and number for messages in wrappers.proto.
+const (
+	WrapperValue_Value_field_name   protoreflect.Name        = "value"
+	WrapperValue_Value_field_number protoreflect.FieldNumber = 1
+)
diff --git a/vendor/google.golang.org/protobuf/internal/genid/wrappers_gen.go b/vendor/google.golang.org/protobuf/internal/genid/wrappers_gen.go
new file mode 100644
index 0000000..72527d2
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/genid/wrappers_gen.go
@@ -0,0 +1,175 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package genid
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+const File_google_protobuf_wrappers_proto = "google/protobuf/wrappers.proto"
+
+// Names for google.protobuf.DoubleValue.
+const (
+	DoubleValue_message_name     protoreflect.Name     = "DoubleValue"
+	DoubleValue_message_fullname protoreflect.FullName = "google.protobuf.DoubleValue"
+)
+
+// Field names for google.protobuf.DoubleValue.
+const (
+	DoubleValue_Value_field_name protoreflect.Name = "value"
+
+	DoubleValue_Value_field_fullname protoreflect.FullName = "google.protobuf.DoubleValue.value"
+)
+
+// Field numbers for google.protobuf.DoubleValue.
+const (
+	DoubleValue_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.FloatValue.
+const (
+	FloatValue_message_name     protoreflect.Name     = "FloatValue"
+	FloatValue_message_fullname protoreflect.FullName = "google.protobuf.FloatValue"
+)
+
+// Field names for google.protobuf.FloatValue.
+const (
+	FloatValue_Value_field_name protoreflect.Name = "value"
+
+	FloatValue_Value_field_fullname protoreflect.FullName = "google.protobuf.FloatValue.value"
+)
+
+// Field numbers for google.protobuf.FloatValue.
+const (
+	FloatValue_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.Int64Value.
+const (
+	Int64Value_message_name     protoreflect.Name     = "Int64Value"
+	Int64Value_message_fullname protoreflect.FullName = "google.protobuf.Int64Value"
+)
+
+// Field names for google.protobuf.Int64Value.
+const (
+	Int64Value_Value_field_name protoreflect.Name = "value"
+
+	Int64Value_Value_field_fullname protoreflect.FullName = "google.protobuf.Int64Value.value"
+)
+
+// Field numbers for google.protobuf.Int64Value.
+const (
+	Int64Value_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.UInt64Value.
+const (
+	UInt64Value_message_name     protoreflect.Name     = "UInt64Value"
+	UInt64Value_message_fullname protoreflect.FullName = "google.protobuf.UInt64Value"
+)
+
+// Field names for google.protobuf.UInt64Value.
+const (
+	UInt64Value_Value_field_name protoreflect.Name = "value"
+
+	UInt64Value_Value_field_fullname protoreflect.FullName = "google.protobuf.UInt64Value.value"
+)
+
+// Field numbers for google.protobuf.UInt64Value.
+const (
+	UInt64Value_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.Int32Value.
+const (
+	Int32Value_message_name     protoreflect.Name     = "Int32Value"
+	Int32Value_message_fullname protoreflect.FullName = "google.protobuf.Int32Value"
+)
+
+// Field names for google.protobuf.Int32Value.
+const (
+	Int32Value_Value_field_name protoreflect.Name = "value"
+
+	Int32Value_Value_field_fullname protoreflect.FullName = "google.protobuf.Int32Value.value"
+)
+
+// Field numbers for google.protobuf.Int32Value.
+const (
+	Int32Value_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.UInt32Value.
+const (
+	UInt32Value_message_name     protoreflect.Name     = "UInt32Value"
+	UInt32Value_message_fullname protoreflect.FullName = "google.protobuf.UInt32Value"
+)
+
+// Field names for google.protobuf.UInt32Value.
+const (
+	UInt32Value_Value_field_name protoreflect.Name = "value"
+
+	UInt32Value_Value_field_fullname protoreflect.FullName = "google.protobuf.UInt32Value.value"
+)
+
+// Field numbers for google.protobuf.UInt32Value.
+const (
+	UInt32Value_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.BoolValue.
+const (
+	BoolValue_message_name     protoreflect.Name     = "BoolValue"
+	BoolValue_message_fullname protoreflect.FullName = "google.protobuf.BoolValue"
+)
+
+// Field names for google.protobuf.BoolValue.
+const (
+	BoolValue_Value_field_name protoreflect.Name = "value"
+
+	BoolValue_Value_field_fullname protoreflect.FullName = "google.protobuf.BoolValue.value"
+)
+
+// Field numbers for google.protobuf.BoolValue.
+const (
+	BoolValue_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.StringValue.
+const (
+	StringValue_message_name     protoreflect.Name     = "StringValue"
+	StringValue_message_fullname protoreflect.FullName = "google.protobuf.StringValue"
+)
+
+// Field names for google.protobuf.StringValue.
+const (
+	StringValue_Value_field_name protoreflect.Name = "value"
+
+	StringValue_Value_field_fullname protoreflect.FullName = "google.protobuf.StringValue.value"
+)
+
+// Field numbers for google.protobuf.StringValue.
+const (
+	StringValue_Value_field_number protoreflect.FieldNumber = 1
+)
+
+// Names for google.protobuf.BytesValue.
+const (
+	BytesValue_message_name     protoreflect.Name     = "BytesValue"
+	BytesValue_message_fullname protoreflect.FullName = "google.protobuf.BytesValue"
+)
+
+// Field names for google.protobuf.BytesValue.
+const (
+	BytesValue_Value_field_name protoreflect.Name = "value"
+
+	BytesValue_Value_field_fullname protoreflect.FullName = "google.protobuf.BytesValue.value"
+)
+
+// Field numbers for google.protobuf.BytesValue.
+const (
+	BytesValue_Value_field_number protoreflect.FieldNumber = 1
+)
diff --git a/vendor/google.golang.org/protobuf/internal/impl/api_export.go b/vendor/google.golang.org/protobuf/internal/impl/api_export.go
new file mode 100644
index 0000000..abee5f3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/api_export.go
@@ -0,0 +1,177 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+	"strconv"
+
+	"google.golang.org/protobuf/encoding/prototext"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/proto"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// Export is a zero-length named type that exists only to export a set of
+// functions that we do not want to appear in godoc.
+type Export struct{}
+
+// NewError formats a string according to the format specifier and arguments and
+// returns an error that has a "proto" prefix.
+func (Export) NewError(f string, x ...interface{}) error {
+	return errors.New(f, x...)
+}
+
+// enum is any enum type generated by protoc-gen-go
+// and must be a named int32 type.
+type enum = interface{}
+
+// EnumOf returns the protoreflect.Enum interface over e.
+// It returns nil if e is nil.
+func (Export) EnumOf(e enum) pref.Enum {
+	switch e := e.(type) {
+	case nil:
+		return nil
+	case pref.Enum:
+		return e
+	default:
+		return legacyWrapEnum(reflect.ValueOf(e))
+	}
+}
+
+// EnumDescriptorOf returns the protoreflect.EnumDescriptor for e.
+// It returns nil if e is nil.
+func (Export) EnumDescriptorOf(e enum) pref.EnumDescriptor {
+	switch e := e.(type) {
+	case nil:
+		return nil
+	case pref.Enum:
+		return e.Descriptor()
+	default:
+		return LegacyLoadEnumDesc(reflect.TypeOf(e))
+	}
+}
+
+// EnumTypeOf returns the protoreflect.EnumType for e.
+// It returns nil if e is nil.
+func (Export) EnumTypeOf(e enum) pref.EnumType {
+	switch e := e.(type) {
+	case nil:
+		return nil
+	case pref.Enum:
+		return e.Type()
+	default:
+		return legacyLoadEnumType(reflect.TypeOf(e))
+	}
+}
+
+// EnumStringOf returns the enum value as a string, either as the name if
+// the number is resolvable, or the number formatted as a string.
+func (Export) EnumStringOf(ed pref.EnumDescriptor, n pref.EnumNumber) string {
+	ev := ed.Values().ByNumber(n)
+	if ev != nil {
+		return string(ev.Name())
+	}
+	return strconv.Itoa(int(n))
+}
+
+// message is any message type generated by protoc-gen-go
+// and must be a pointer to a named struct type.
+type message = interface{}
+
+// legacyMessageWrapper wraps a v2 message as a v1 message.
+type legacyMessageWrapper struct{ m pref.ProtoMessage }
+
+func (m legacyMessageWrapper) Reset()         { proto.Reset(m.m) }
+func (m legacyMessageWrapper) String() string { return Export{}.MessageStringOf(m.m) }
+func (m legacyMessageWrapper) ProtoMessage()  {}
+
+// ProtoMessageV1Of converts either a v1 or v2 message to a v1 message.
+// It returns nil if m is nil.
+func (Export) ProtoMessageV1Of(m message) piface.MessageV1 {
+	switch mv := m.(type) {
+	case nil:
+		return nil
+	case piface.MessageV1:
+		return mv
+	case unwrapper:
+		return Export{}.ProtoMessageV1Of(mv.protoUnwrap())
+	case pref.ProtoMessage:
+		return legacyMessageWrapper{mv}
+	default:
+		panic(fmt.Sprintf("message %T is neither a v1 or v2 Message", m))
+	}
+}
+
+func (Export) protoMessageV2Of(m message) pref.ProtoMessage {
+	switch mv := m.(type) {
+	case nil:
+		return nil
+	case pref.ProtoMessage:
+		return mv
+	case legacyMessageWrapper:
+		return mv.m
+	case piface.MessageV1:
+		return nil
+	default:
+		panic(fmt.Sprintf("message %T is neither a v1 or v2 Message", m))
+	}
+}
+
+// ProtoMessageV2Of converts either a v1 or v2 message to a v2 message.
+// It returns nil if m is nil.
+func (Export) ProtoMessageV2Of(m message) pref.ProtoMessage {
+	if m == nil {
+		return nil
+	}
+	if mv := (Export{}).protoMessageV2Of(m); mv != nil {
+		return mv
+	}
+	return legacyWrapMessage(reflect.ValueOf(m)).Interface()
+}
+
+// MessageOf returns the protoreflect.Message interface over m.
+// It returns nil if m is nil.
+func (Export) MessageOf(m message) pref.Message {
+	if m == nil {
+		return nil
+	}
+	if mv := (Export{}).protoMessageV2Of(m); mv != nil {
+		return mv.ProtoReflect()
+	}
+	return legacyWrapMessage(reflect.ValueOf(m))
+}
+
+// MessageDescriptorOf returns the protoreflect.MessageDescriptor for m.
+// It returns nil if m is nil.
+func (Export) MessageDescriptorOf(m message) pref.MessageDescriptor {
+	if m == nil {
+		return nil
+	}
+	if mv := (Export{}).protoMessageV2Of(m); mv != nil {
+		return mv.ProtoReflect().Descriptor()
+	}
+	return LegacyLoadMessageDesc(reflect.TypeOf(m))
+}
+
+// MessageTypeOf returns the protoreflect.MessageType for m.
+// It returns nil if m is nil.
+func (Export) MessageTypeOf(m message) pref.MessageType {
+	if m == nil {
+		return nil
+	}
+	if mv := (Export{}).protoMessageV2Of(m); mv != nil {
+		return mv.ProtoReflect().Type()
+	}
+	return legacyLoadMessageType(reflect.TypeOf(m), "")
+}
+
+// MessageStringOf returns the message value as a string,
+// which is the message serialized in the protobuf text format.
+func (Export) MessageStringOf(m pref.ProtoMessage) string {
+	return prototext.MarshalOptions{Multiline: false}.Format(m)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/checkinit.go b/vendor/google.golang.org/protobuf/internal/impl/checkinit.go
new file mode 100644
index 0000000..b82341e
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/checkinit.go
@@ -0,0 +1,141 @@
+// 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.
+
+package impl
+
+import (
+	"sync"
+
+	"google.golang.org/protobuf/internal/errors"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+func (mi *MessageInfo) checkInitialized(in piface.CheckInitializedInput) (piface.CheckInitializedOutput, error) {
+	var p pointer
+	if ms, ok := in.Message.(*messageState); ok {
+		p = ms.pointer()
+	} else {
+		p = in.Message.(*messageReflectWrapper).pointer()
+	}
+	return piface.CheckInitializedOutput{}, mi.checkInitializedPointer(p)
+}
+
+func (mi *MessageInfo) checkInitializedPointer(p pointer) error {
+	mi.init()
+	if !mi.needsInitCheck {
+		return nil
+	}
+	if p.IsNil() {
+		for _, f := range mi.orderedCoderFields {
+			if f.isRequired {
+				return errors.RequiredNotSet(string(mi.Desc.Fields().ByNumber(f.num).FullName()))
+			}
+		}
+		return nil
+	}
+	if mi.extensionOffset.IsValid() {
+		e := p.Apply(mi.extensionOffset).Extensions()
+		if err := mi.isInitExtensions(e); err != nil {
+			return err
+		}
+	}
+	for _, f := range mi.orderedCoderFields {
+		if !f.isRequired && f.funcs.isInit == nil {
+			continue
+		}
+		fptr := p.Apply(f.offset)
+		if f.isPointer && fptr.Elem().IsNil() {
+			if f.isRequired {
+				return errors.RequiredNotSet(string(mi.Desc.Fields().ByNumber(f.num).FullName()))
+			}
+			continue
+		}
+		if f.funcs.isInit == nil {
+			continue
+		}
+		if err := f.funcs.isInit(fptr, f); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (mi *MessageInfo) isInitExtensions(ext *map[int32]ExtensionField) error {
+	if ext == nil {
+		return nil
+	}
+	for _, x := range *ext {
+		ei := getExtensionFieldInfo(x.Type())
+		if ei.funcs.isInit == nil {
+			continue
+		}
+		v := x.Value()
+		if !v.IsValid() {
+			continue
+		}
+		if err := ei.funcs.isInit(v); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+var (
+	needsInitCheckMu  sync.Mutex
+	needsInitCheckMap sync.Map
+)
+
+// needsInitCheck reports whether a message needs to be checked for partial initialization.
+//
+// It returns true if the message transitively includes any required or extension fields.
+func needsInitCheck(md pref.MessageDescriptor) bool {
+	if v, ok := needsInitCheckMap.Load(md); ok {
+		if has, ok := v.(bool); ok {
+			return has
+		}
+	}
+	needsInitCheckMu.Lock()
+	defer needsInitCheckMu.Unlock()
+	return needsInitCheckLocked(md)
+}
+
+func needsInitCheckLocked(md pref.MessageDescriptor) (has bool) {
+	if v, ok := needsInitCheckMap.Load(md); ok {
+		// If has is true, we've previously determined that this message
+		// needs init checks.
+		//
+		// If has is false, we've previously determined that it can never
+		// be uninitialized.
+		//
+		// If has is not a bool, we've just encountered a cycle in the
+		// message graph. In this case, it is safe to return false: If
+		// the message does have required fields, we'll detect them later
+		// in the graph traversal.
+		has, ok := v.(bool)
+		return ok && has
+	}
+	needsInitCheckMap.Store(md, struct{}{}) // avoid cycles while descending into this message
+	defer func() {
+		needsInitCheckMap.Store(md, has)
+	}()
+	if md.RequiredNumbers().Len() > 0 {
+		return true
+	}
+	if md.ExtensionRanges().Len() > 0 {
+		return true
+	}
+	for i := 0; i < md.Fields().Len(); i++ {
+		fd := md.Fields().Get(i)
+		// Map keys are never messages, so just consider the map value.
+		if fd.IsMap() {
+			fd = fd.MapValue()
+		}
+		fmd := fd.Message()
+		if fmd != nil && needsInitCheckLocked(fmd) {
+			return true
+		}
+	}
+	return false
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_extension.go b/vendor/google.golang.org/protobuf/internal/impl/codec_extension.go
new file mode 100644
index 0000000..08d3517
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_extension.go
@@ -0,0 +1,223 @@
+// 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.
+
+package impl
+
+import (
+	"sync"
+	"sync/atomic"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type extensionFieldInfo struct {
+	wiretag             uint64
+	tagsize             int
+	unmarshalNeedsValue bool
+	funcs               valueCoderFuncs
+	validation          validationInfo
+}
+
+var legacyExtensionFieldInfoCache sync.Map // map[protoreflect.ExtensionType]*extensionFieldInfo
+
+func getExtensionFieldInfo(xt pref.ExtensionType) *extensionFieldInfo {
+	if xi, ok := xt.(*ExtensionInfo); ok {
+		xi.lazyInit()
+		return xi.info
+	}
+	return legacyLoadExtensionFieldInfo(xt)
+}
+
+// legacyLoadExtensionFieldInfo dynamically loads a *ExtensionInfo for xt.
+func legacyLoadExtensionFieldInfo(xt pref.ExtensionType) *extensionFieldInfo {
+	if xi, ok := legacyExtensionFieldInfoCache.Load(xt); ok {
+		return xi.(*extensionFieldInfo)
+	}
+	e := makeExtensionFieldInfo(xt.TypeDescriptor())
+	if e, ok := legacyMessageTypeCache.LoadOrStore(xt, e); ok {
+		return e.(*extensionFieldInfo)
+	}
+	return e
+}
+
+func makeExtensionFieldInfo(xd pref.ExtensionDescriptor) *extensionFieldInfo {
+	var wiretag uint64
+	if !xd.IsPacked() {
+		wiretag = protowire.EncodeTag(xd.Number(), wireTypes[xd.Kind()])
+	} else {
+		wiretag = protowire.EncodeTag(xd.Number(), protowire.BytesType)
+	}
+	e := &extensionFieldInfo{
+		wiretag: wiretag,
+		tagsize: protowire.SizeVarint(wiretag),
+		funcs:   encoderFuncsForValue(xd),
+	}
+	// Does the unmarshal function need a value passed to it?
+	// This is true for composite types, where we pass in a message, list, or map to fill in,
+	// and for enums, where we pass in a prototype value to specify the concrete enum type.
+	switch xd.Kind() {
+	case pref.MessageKind, pref.GroupKind, pref.EnumKind:
+		e.unmarshalNeedsValue = true
+	default:
+		if xd.Cardinality() == pref.Repeated {
+			e.unmarshalNeedsValue = true
+		}
+	}
+	return e
+}
+
+type lazyExtensionValue struct {
+	atomicOnce uint32 // atomically set if value is valid
+	mu         sync.Mutex
+	xi         *extensionFieldInfo
+	value      pref.Value
+	b          []byte
+	fn         func() pref.Value
+}
+
+type ExtensionField struct {
+	typ pref.ExtensionType
+
+	// value is either the value of GetValue,
+	// or a *lazyExtensionValue that then returns the value of GetValue.
+	value pref.Value
+	lazy  *lazyExtensionValue
+}
+
+func (f *ExtensionField) appendLazyBytes(xt pref.ExtensionType, xi *extensionFieldInfo, num protowire.Number, wtyp protowire.Type, b []byte) {
+	if f.lazy == nil {
+		f.lazy = &lazyExtensionValue{xi: xi}
+	}
+	f.typ = xt
+	f.lazy.xi = xi
+	f.lazy.b = protowire.AppendTag(f.lazy.b, num, wtyp)
+	f.lazy.b = append(f.lazy.b, b...)
+}
+
+func (f *ExtensionField) canLazy(xt pref.ExtensionType) bool {
+	if f.typ == nil {
+		return true
+	}
+	if f.typ == xt && f.lazy != nil && atomic.LoadUint32(&f.lazy.atomicOnce) == 0 {
+		return true
+	}
+	return false
+}
+
+func (f *ExtensionField) lazyInit() {
+	f.lazy.mu.Lock()
+	defer f.lazy.mu.Unlock()
+	if atomic.LoadUint32(&f.lazy.atomicOnce) == 1 {
+		return
+	}
+	if f.lazy.xi != nil {
+		b := f.lazy.b
+		val := f.typ.New()
+		for len(b) > 0 {
+			var tag uint64
+			if b[0] < 0x80 {
+				tag = uint64(b[0])
+				b = b[1:]
+			} else if len(b) >= 2 && b[1] < 128 {
+				tag = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				b = b[2:]
+			} else {
+				var n int
+				tag, n = protowire.ConsumeVarint(b)
+				if n < 0 {
+					panic(errors.New("bad tag in lazy extension decoding"))
+				}
+				b = b[n:]
+			}
+			num := protowire.Number(tag >> 3)
+			wtyp := protowire.Type(tag & 7)
+			var out unmarshalOutput
+			var err error
+			val, out, err = f.lazy.xi.funcs.unmarshal(b, val, num, wtyp, lazyUnmarshalOptions)
+			if err != nil {
+				panic(errors.New("decode failure in lazy extension decoding: %v", err))
+			}
+			b = b[out.n:]
+		}
+		f.lazy.value = val
+	} else {
+		f.lazy.value = f.lazy.fn()
+	}
+	f.lazy.xi = nil
+	f.lazy.fn = nil
+	f.lazy.b = nil
+	atomic.StoreUint32(&f.lazy.atomicOnce, 1)
+}
+
+// Set sets the type and value of the extension field.
+// This must not be called concurrently.
+func (f *ExtensionField) Set(t pref.ExtensionType, v pref.Value) {
+	f.typ = t
+	f.value = v
+	f.lazy = nil
+}
+
+// SetLazy sets the type and a value that is to be lazily evaluated upon first use.
+// This must not be called concurrently.
+func (f *ExtensionField) SetLazy(t pref.ExtensionType, fn func() pref.Value) {
+	f.typ = t
+	f.lazy = &lazyExtensionValue{fn: fn}
+}
+
+// Value returns the value of the extension field.
+// This may be called concurrently.
+func (f *ExtensionField) Value() pref.Value {
+	if f.lazy != nil {
+		if atomic.LoadUint32(&f.lazy.atomicOnce) == 0 {
+			f.lazyInit()
+		}
+		return f.lazy.value
+	}
+	return f.value
+}
+
+// Type returns the type of the extension field.
+// This may be called concurrently.
+func (f ExtensionField) Type() pref.ExtensionType {
+	return f.typ
+}
+
+// IsSet returns whether the extension field is set.
+// This may be called concurrently.
+func (f ExtensionField) IsSet() bool {
+	return f.typ != nil
+}
+
+// IsLazy reports whether a field is lazily encoded.
+// It is exported for testing.
+func IsLazy(m pref.Message, fd pref.FieldDescriptor) bool {
+	var mi *MessageInfo
+	var p pointer
+	switch m := m.(type) {
+	case *messageState:
+		mi = m.messageInfo()
+		p = m.pointer()
+	case *messageReflectWrapper:
+		mi = m.messageInfo()
+		p = m.pointer()
+	default:
+		return false
+	}
+	xd, ok := fd.(pref.ExtensionTypeDescriptor)
+	if !ok {
+		return false
+	}
+	xt := xd.Type()
+	ext := mi.extensionMap(p)
+	if ext == nil {
+		return false
+	}
+	f, ok := (*ext)[int32(fd.Number())]
+	if !ok {
+		return false
+	}
+	return f.typ == xt && f.lazy != nil && atomic.LoadUint32(&f.lazy.atomicOnce) == 0
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_field.go b/vendor/google.golang.org/protobuf/internal/impl/codec_field.go
new file mode 100644
index 0000000..cb4b482
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_field.go
@@ -0,0 +1,830 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+	"reflect"
+	"sync"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/proto"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+type errInvalidUTF8 struct{}
+
+func (errInvalidUTF8) Error() string     { return "string field contains invalid UTF-8" }
+func (errInvalidUTF8) InvalidUTF8() bool { return true }
+func (errInvalidUTF8) Unwrap() error     { return errors.Error }
+
+// initOneofFieldCoders initializes the fast-path functions for the fields in a oneof.
+//
+// For size, marshal, and isInit operations, functions are set only on the first field
+// in the oneof. The functions are called when the oneof is non-nil, and will dispatch
+// to the appropriate field-specific function as necessary.
+//
+// The unmarshal function is set on each field individually as usual.
+func (mi *MessageInfo) initOneofFieldCoders(od pref.OneofDescriptor, si structInfo) {
+	fs := si.oneofsByName[od.Name()]
+	ft := fs.Type
+	oneofFields := make(map[reflect.Type]*coderFieldInfo)
+	needIsInit := false
+	fields := od.Fields()
+	for i, lim := 0, fields.Len(); i < lim; i++ {
+		fd := od.Fields().Get(i)
+		num := fd.Number()
+		// Make a copy of the original coderFieldInfo for use in unmarshaling.
+		//
+		// oneofFields[oneofType].funcs.marshal is the field-specific marshal function.
+		//
+		// mi.coderFields[num].marshal is set on only the first field in the oneof,
+		// and dispatches to the field-specific marshaler in oneofFields.
+		cf := *mi.coderFields[num]
+		ot := si.oneofWrappersByNumber[num]
+		cf.ft = ot.Field(0).Type
+		cf.mi, cf.funcs = fieldCoder(fd, cf.ft)
+		oneofFields[ot] = &cf
+		if cf.funcs.isInit != nil {
+			needIsInit = true
+		}
+		mi.coderFields[num].funcs.unmarshal = func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+			var vw reflect.Value         // pointer to wrapper type
+			vi := p.AsValueOf(ft).Elem() // oneof field value of interface kind
+			if !vi.IsNil() && !vi.Elem().IsNil() && vi.Elem().Elem().Type() == ot {
+				vw = vi.Elem()
+			} else {
+				vw = reflect.New(ot)
+			}
+			out, err := cf.funcs.unmarshal(b, pointerOfValue(vw).Apply(zeroOffset), wtyp, &cf, opts)
+			if err != nil {
+				return out, err
+			}
+			vi.Set(vw)
+			return out, nil
+		}
+	}
+	getInfo := func(p pointer) (pointer, *coderFieldInfo) {
+		v := p.AsValueOf(ft).Elem()
+		if v.IsNil() {
+			return pointer{}, nil
+		}
+		v = v.Elem() // interface -> *struct
+		if v.IsNil() {
+			return pointer{}, nil
+		}
+		return pointerOfValue(v).Apply(zeroOffset), oneofFields[v.Elem().Type()]
+	}
+	first := mi.coderFields[od.Fields().Get(0).Number()]
+	first.funcs.size = func(p pointer, _ *coderFieldInfo, opts marshalOptions) int {
+		p, info := getInfo(p)
+		if info == nil || info.funcs.size == nil {
+			return 0
+		}
+		return info.funcs.size(p, info, opts)
+	}
+	first.funcs.marshal = func(b []byte, p pointer, _ *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+		p, info := getInfo(p)
+		if info == nil || info.funcs.marshal == nil {
+			return b, nil
+		}
+		return info.funcs.marshal(b, p, info, opts)
+	}
+	first.funcs.merge = func(dst, src pointer, _ *coderFieldInfo, opts mergeOptions) {
+		srcp, srcinfo := getInfo(src)
+		if srcinfo == nil || srcinfo.funcs.merge == nil {
+			return
+		}
+		dstp, dstinfo := getInfo(dst)
+		if dstinfo != srcinfo {
+			dst.AsValueOf(ft).Elem().Set(reflect.New(src.AsValueOf(ft).Elem().Elem().Elem().Type()))
+			dstp = pointerOfValue(dst.AsValueOf(ft).Elem().Elem()).Apply(zeroOffset)
+		}
+		srcinfo.funcs.merge(dstp, srcp, srcinfo, opts)
+	}
+	if needIsInit {
+		first.funcs.isInit = func(p pointer, _ *coderFieldInfo) error {
+			p, info := getInfo(p)
+			if info == nil || info.funcs.isInit == nil {
+				return nil
+			}
+			return info.funcs.isInit(p, info)
+		}
+	}
+}
+
+func makeWeakMessageFieldCoder(fd pref.FieldDescriptor) pointerCoderFuncs {
+	var once sync.Once
+	var messageType pref.MessageType
+	lazyInit := func() {
+		once.Do(func() {
+			messageName := fd.Message().FullName()
+			messageType, _ = preg.GlobalTypes.FindMessageByName(messageName)
+		})
+	}
+
+	return pointerCoderFuncs{
+		size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+			m, ok := p.WeakFields().get(f.num)
+			if !ok {
+				return 0
+			}
+			lazyInit()
+			if messageType == nil {
+				panic(fmt.Sprintf("weak message %v is not linked in", fd.Message().FullName()))
+			}
+			return sizeMessage(m, f.tagsize, opts)
+		},
+		marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+			m, ok := p.WeakFields().get(f.num)
+			if !ok {
+				return b, nil
+			}
+			lazyInit()
+			if messageType == nil {
+				panic(fmt.Sprintf("weak message %v is not linked in", fd.Message().FullName()))
+			}
+			return appendMessage(b, m, f.wiretag, opts)
+		},
+		unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+			fs := p.WeakFields()
+			m, ok := fs.get(f.num)
+			if !ok {
+				lazyInit()
+				if messageType == nil {
+					return unmarshalOutput{}, errUnknown
+				}
+				m = messageType.New().Interface()
+				fs.set(f.num, m)
+			}
+			return consumeMessage(b, m, wtyp, opts)
+		},
+		isInit: func(p pointer, f *coderFieldInfo) error {
+			m, ok := p.WeakFields().get(f.num)
+			if !ok {
+				return nil
+			}
+			return proto.CheckInitialized(m)
+		},
+		merge: func(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+			sm, ok := src.WeakFields().get(f.num)
+			if !ok {
+				return
+			}
+			dm, ok := dst.WeakFields().get(f.num)
+			if !ok {
+				lazyInit()
+				if messageType == nil {
+					panic(fmt.Sprintf("weak message %v is not linked in", fd.Message().FullName()))
+				}
+				dm = messageType.New().Interface()
+				dst.WeakFields().set(f.num, dm)
+			}
+			opts.Merge(dm, sm)
+		},
+	}
+}
+
+func makeMessageFieldCoder(fd pref.FieldDescriptor, ft reflect.Type) pointerCoderFuncs {
+	if mi := getMessageInfo(ft); mi != nil {
+		funcs := pointerCoderFuncs{
+			size:      sizeMessageInfo,
+			marshal:   appendMessageInfo,
+			unmarshal: consumeMessageInfo,
+			merge:     mergeMessage,
+		}
+		if needsInitCheck(mi.Desc) {
+			funcs.isInit = isInitMessageInfo
+		}
+		return funcs
+	} else {
+		return pointerCoderFuncs{
+			size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return sizeMessage(m, f.tagsize, opts)
+			},
+			marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return appendMessage(b, m, f.wiretag, opts)
+			},
+			unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+				mp := p.AsValueOf(ft).Elem()
+				if mp.IsNil() {
+					mp.Set(reflect.New(ft.Elem()))
+				}
+				return consumeMessage(b, asMessage(mp), wtyp, opts)
+			},
+			isInit: func(p pointer, f *coderFieldInfo) error {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return proto.CheckInitialized(m)
+			},
+			merge: mergeMessage,
+		}
+	}
+}
+
+func sizeMessageInfo(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+	return protowire.SizeBytes(f.mi.sizePointer(p.Elem(), opts)) + f.tagsize
+}
+
+func appendMessageInfo(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(f.mi.sizePointer(p.Elem(), opts)))
+	return f.mi.marshalAppendPointer(b, p.Elem(), opts)
+}
+
+func consumeMessageInfo(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if p.Elem().IsNil() {
+		p.SetPointer(pointerOfValue(reflect.New(f.mi.GoReflectType.Elem())))
+	}
+	o, err := f.mi.unmarshalPointer(v, p.Elem(), 0, opts)
+	if err != nil {
+		return out, err
+	}
+	out.n = n
+	out.initialized = o.initialized
+	return out, nil
+}
+
+func isInitMessageInfo(p pointer, f *coderFieldInfo) error {
+	return f.mi.checkInitializedPointer(p.Elem())
+}
+
+func sizeMessage(m proto.Message, tagsize int, _ marshalOptions) int {
+	return protowire.SizeBytes(proto.Size(m)) + tagsize
+}
+
+func appendMessage(b []byte, m proto.Message, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, uint64(proto.Size(m)))
+	return opts.Options().MarshalAppend(b, m)
+}
+
+func consumeMessage(b []byte, m proto.Message, wtyp protowire.Type, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     v,
+		Message: m.ProtoReflect(),
+	})
+	if err != nil {
+		return out, err
+	}
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return out, nil
+}
+
+func sizeMessageValue(v pref.Value, tagsize int, opts marshalOptions) int {
+	m := v.Message().Interface()
+	return sizeMessage(m, tagsize, opts)
+}
+
+func appendMessageValue(b []byte, v pref.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	m := v.Message().Interface()
+	return appendMessage(b, m, wiretag, opts)
+}
+
+func consumeMessageValue(b []byte, v pref.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (pref.Value, unmarshalOutput, error) {
+	m := v.Message().Interface()
+	out, err := consumeMessage(b, m, wtyp, opts)
+	return v, out, err
+}
+
+func isInitMessageValue(v pref.Value) error {
+	m := v.Message().Interface()
+	return proto.CheckInitialized(m)
+}
+
+var coderMessageValue = valueCoderFuncs{
+	size:      sizeMessageValue,
+	marshal:   appendMessageValue,
+	unmarshal: consumeMessageValue,
+	isInit:    isInitMessageValue,
+	merge:     mergeMessageValue,
+}
+
+func sizeGroupValue(v pref.Value, tagsize int, opts marshalOptions) int {
+	m := v.Message().Interface()
+	return sizeGroup(m, tagsize, opts)
+}
+
+func appendGroupValue(b []byte, v pref.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	m := v.Message().Interface()
+	return appendGroup(b, m, wiretag, opts)
+}
+
+func consumeGroupValue(b []byte, v pref.Value, num protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (pref.Value, unmarshalOutput, error) {
+	m := v.Message().Interface()
+	out, err := consumeGroup(b, m, num, wtyp, opts)
+	return v, out, err
+}
+
+var coderGroupValue = valueCoderFuncs{
+	size:      sizeGroupValue,
+	marshal:   appendGroupValue,
+	unmarshal: consumeGroupValue,
+	isInit:    isInitMessageValue,
+	merge:     mergeMessageValue,
+}
+
+func makeGroupFieldCoder(fd pref.FieldDescriptor, ft reflect.Type) pointerCoderFuncs {
+	num := fd.Number()
+	if mi := getMessageInfo(ft); mi != nil {
+		funcs := pointerCoderFuncs{
+			size:      sizeGroupType,
+			marshal:   appendGroupType,
+			unmarshal: consumeGroupType,
+			merge:     mergeMessage,
+		}
+		if needsInitCheck(mi.Desc) {
+			funcs.isInit = isInitMessageInfo
+		}
+		return funcs
+	} else {
+		return pointerCoderFuncs{
+			size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return sizeGroup(m, f.tagsize, opts)
+			},
+			marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return appendGroup(b, m, f.wiretag, opts)
+			},
+			unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+				mp := p.AsValueOf(ft).Elem()
+				if mp.IsNil() {
+					mp.Set(reflect.New(ft.Elem()))
+				}
+				return consumeGroup(b, asMessage(mp), num, wtyp, opts)
+			},
+			isInit: func(p pointer, f *coderFieldInfo) error {
+				m := asMessage(p.AsValueOf(ft).Elem())
+				return proto.CheckInitialized(m)
+			},
+			merge: mergeMessage,
+		}
+	}
+}
+
+func sizeGroupType(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+	return 2*f.tagsize + f.mi.sizePointer(p.Elem(), opts)
+}
+
+func appendGroupType(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, f.wiretag) // start group
+	b, err := f.mi.marshalAppendPointer(b, p.Elem(), opts)
+	b = protowire.AppendVarint(b, f.wiretag+1) // end group
+	return b, err
+}
+
+func consumeGroupType(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.StartGroupType {
+		return out, errUnknown
+	}
+	if p.Elem().IsNil() {
+		p.SetPointer(pointerOfValue(reflect.New(f.mi.GoReflectType.Elem())))
+	}
+	return f.mi.unmarshalPointer(b, p.Elem(), f.num, opts)
+}
+
+func sizeGroup(m proto.Message, tagsize int, _ marshalOptions) int {
+	return 2*tagsize + proto.Size(m)
+}
+
+func appendGroup(b []byte, m proto.Message, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag) // start group
+	b, err := opts.Options().MarshalAppend(b, m)
+	b = protowire.AppendVarint(b, wiretag+1) // end group
+	return b, err
+}
+
+func consumeGroup(b []byte, m proto.Message, num protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.StartGroupType {
+		return out, errUnknown
+	}
+	b, n := protowire.ConsumeGroup(num, b)
+	if n < 0 {
+		return out, errDecode
+	}
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     b,
+		Message: m.ProtoReflect(),
+	})
+	if err != nil {
+		return out, err
+	}
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return out, nil
+}
+
+func makeMessageSliceFieldCoder(fd pref.FieldDescriptor, ft reflect.Type) pointerCoderFuncs {
+	if mi := getMessageInfo(ft); mi != nil {
+		funcs := pointerCoderFuncs{
+			size:      sizeMessageSliceInfo,
+			marshal:   appendMessageSliceInfo,
+			unmarshal: consumeMessageSliceInfo,
+			merge:     mergeMessageSlice,
+		}
+		if needsInitCheck(mi.Desc) {
+			funcs.isInit = isInitMessageSliceInfo
+		}
+		return funcs
+	}
+	return pointerCoderFuncs{
+		size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+			return sizeMessageSlice(p, ft, f.tagsize, opts)
+		},
+		marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+			return appendMessageSlice(b, p, f.wiretag, ft, opts)
+		},
+		unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+			return consumeMessageSlice(b, p, ft, wtyp, opts)
+		},
+		isInit: func(p pointer, f *coderFieldInfo) error {
+			return isInitMessageSlice(p, ft)
+		},
+		merge: mergeMessageSlice,
+	}
+}
+
+func sizeMessageSliceInfo(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+	s := p.PointerSlice()
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeBytes(f.mi.sizePointer(v, opts)) + f.tagsize
+	}
+	return n
+}
+
+func appendMessageSliceInfo(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := p.PointerSlice()
+	var err error
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		siz := f.mi.sizePointer(v, opts)
+		b = protowire.AppendVarint(b, uint64(siz))
+		b, err = f.mi.marshalAppendPointer(b, v, opts)
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func consumeMessageSliceInfo(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	m := reflect.New(f.mi.GoReflectType.Elem()).Interface()
+	mp := pointerOfIface(m)
+	o, err := f.mi.unmarshalPointer(v, mp, 0, opts)
+	if err != nil {
+		return out, err
+	}
+	p.AppendPointerSlice(mp)
+	out.n = n
+	out.initialized = o.initialized
+	return out, nil
+}
+
+func isInitMessageSliceInfo(p pointer, f *coderFieldInfo) error {
+	s := p.PointerSlice()
+	for _, v := range s {
+		if err := f.mi.checkInitializedPointer(v); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func sizeMessageSlice(p pointer, goType reflect.Type, tagsize int, _ marshalOptions) int {
+	s := p.PointerSlice()
+	n := 0
+	for _, v := range s {
+		m := asMessage(v.AsValueOf(goType.Elem()))
+		n += protowire.SizeBytes(proto.Size(m)) + tagsize
+	}
+	return n
+}
+
+func appendMessageSlice(b []byte, p pointer, wiretag uint64, goType reflect.Type, opts marshalOptions) ([]byte, error) {
+	s := p.PointerSlice()
+	var err error
+	for _, v := range s {
+		m := asMessage(v.AsValueOf(goType.Elem()))
+		b = protowire.AppendVarint(b, wiretag)
+		siz := proto.Size(m)
+		b = protowire.AppendVarint(b, uint64(siz))
+		b, err = opts.Options().MarshalAppend(b, m)
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func consumeMessageSlice(b []byte, p pointer, goType reflect.Type, wtyp protowire.Type, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	mp := reflect.New(goType.Elem())
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     v,
+		Message: asMessage(mp).ProtoReflect(),
+	})
+	if err != nil {
+		return out, err
+	}
+	p.AppendPointerSlice(pointerOfValue(mp))
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return out, nil
+}
+
+func isInitMessageSlice(p pointer, goType reflect.Type) error {
+	s := p.PointerSlice()
+	for _, v := range s {
+		m := asMessage(v.AsValueOf(goType.Elem()))
+		if err := proto.CheckInitialized(m); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// Slices of messages
+
+func sizeMessageSliceValue(listv pref.Value, tagsize int, opts marshalOptions) int {
+	list := listv.List()
+	n := 0
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		m := list.Get(i).Message().Interface()
+		n += protowire.SizeBytes(proto.Size(m)) + tagsize
+	}
+	return n
+}
+
+func appendMessageSliceValue(b []byte, listv pref.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	mopts := opts.Options()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		m := list.Get(i).Message().Interface()
+		b = protowire.AppendVarint(b, wiretag)
+		siz := proto.Size(m)
+		b = protowire.AppendVarint(b, uint64(siz))
+		var err error
+		b, err = mopts.MarshalAppend(b, m)
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func consumeMessageSliceValue(b []byte, listv pref.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ pref.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp != protowire.BytesType {
+		return pref.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return pref.Value{}, out, errDecode
+	}
+	m := list.NewElement()
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     v,
+		Message: m.Message(),
+	})
+	if err != nil {
+		return pref.Value{}, out, err
+	}
+	list.Append(m)
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return listv, out, nil
+}
+
+func isInitMessageSliceValue(listv pref.Value) error {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		m := list.Get(i).Message().Interface()
+		if err := proto.CheckInitialized(m); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+var coderMessageSliceValue = valueCoderFuncs{
+	size:      sizeMessageSliceValue,
+	marshal:   appendMessageSliceValue,
+	unmarshal: consumeMessageSliceValue,
+	isInit:    isInitMessageSliceValue,
+	merge:     mergeMessageListValue,
+}
+
+func sizeGroupSliceValue(listv pref.Value, tagsize int, opts marshalOptions) int {
+	list := listv.List()
+	n := 0
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		m := list.Get(i).Message().Interface()
+		n += 2*tagsize + proto.Size(m)
+	}
+	return n
+}
+
+func appendGroupSliceValue(b []byte, listv pref.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	mopts := opts.Options()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		m := list.Get(i).Message().Interface()
+		b = protowire.AppendVarint(b, wiretag) // start group
+		var err error
+		b, err = mopts.MarshalAppend(b, m)
+		if err != nil {
+			return b, err
+		}
+		b = protowire.AppendVarint(b, wiretag+1) // end group
+	}
+	return b, nil
+}
+
+func consumeGroupSliceValue(b []byte, listv pref.Value, num protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ pref.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp != protowire.StartGroupType {
+		return pref.Value{}, out, errUnknown
+	}
+	b, n := protowire.ConsumeGroup(num, b)
+	if n < 0 {
+		return pref.Value{}, out, errDecode
+	}
+	m := list.NewElement()
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     b,
+		Message: m.Message(),
+	})
+	if err != nil {
+		return pref.Value{}, out, err
+	}
+	list.Append(m)
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return listv, out, nil
+}
+
+var coderGroupSliceValue = valueCoderFuncs{
+	size:      sizeGroupSliceValue,
+	marshal:   appendGroupSliceValue,
+	unmarshal: consumeGroupSliceValue,
+	isInit:    isInitMessageSliceValue,
+	merge:     mergeMessageListValue,
+}
+
+func makeGroupSliceFieldCoder(fd pref.FieldDescriptor, ft reflect.Type) pointerCoderFuncs {
+	num := fd.Number()
+	if mi := getMessageInfo(ft); mi != nil {
+		funcs := pointerCoderFuncs{
+			size:      sizeGroupSliceInfo,
+			marshal:   appendGroupSliceInfo,
+			unmarshal: consumeGroupSliceInfo,
+			merge:     mergeMessageSlice,
+		}
+		if needsInitCheck(mi.Desc) {
+			funcs.isInit = isInitMessageSliceInfo
+		}
+		return funcs
+	}
+	return pointerCoderFuncs{
+		size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+			return sizeGroupSlice(p, ft, f.tagsize, opts)
+		},
+		marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+			return appendGroupSlice(b, p, f.wiretag, ft, opts)
+		},
+		unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+			return consumeGroupSlice(b, p, num, wtyp, ft, opts)
+		},
+		isInit: func(p pointer, f *coderFieldInfo) error {
+			return isInitMessageSlice(p, ft)
+		},
+		merge: mergeMessageSlice,
+	}
+}
+
+func sizeGroupSlice(p pointer, messageType reflect.Type, tagsize int, _ marshalOptions) int {
+	s := p.PointerSlice()
+	n := 0
+	for _, v := range s {
+		m := asMessage(v.AsValueOf(messageType.Elem()))
+		n += 2*tagsize + proto.Size(m)
+	}
+	return n
+}
+
+func appendGroupSlice(b []byte, p pointer, wiretag uint64, messageType reflect.Type, opts marshalOptions) ([]byte, error) {
+	s := p.PointerSlice()
+	var err error
+	for _, v := range s {
+		m := asMessage(v.AsValueOf(messageType.Elem()))
+		b = protowire.AppendVarint(b, wiretag) // start group
+		b, err = opts.Options().MarshalAppend(b, m)
+		if err != nil {
+			return b, err
+		}
+		b = protowire.AppendVarint(b, wiretag+1) // end group
+	}
+	return b, nil
+}
+
+func consumeGroupSlice(b []byte, p pointer, num protowire.Number, wtyp protowire.Type, goType reflect.Type, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.StartGroupType {
+		return out, errUnknown
+	}
+	b, n := protowire.ConsumeGroup(num, b)
+	if n < 0 {
+		return out, errDecode
+	}
+	mp := reflect.New(goType.Elem())
+	o, err := opts.Options().UnmarshalState(piface.UnmarshalInput{
+		Buf:     b,
+		Message: asMessage(mp).ProtoReflect(),
+	})
+	if err != nil {
+		return out, err
+	}
+	p.AppendPointerSlice(pointerOfValue(mp))
+	out.n = n
+	out.initialized = o.Flags&piface.UnmarshalInitialized != 0
+	return out, nil
+}
+
+func sizeGroupSliceInfo(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+	s := p.PointerSlice()
+	n := 0
+	for _, v := range s {
+		n += 2*f.tagsize + f.mi.sizePointer(v, opts)
+	}
+	return n
+}
+
+func appendGroupSliceInfo(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := p.PointerSlice()
+	var err error
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag) // start group
+		b, err = f.mi.marshalAppendPointer(b, v, opts)
+		if err != nil {
+			return b, err
+		}
+		b = protowire.AppendVarint(b, f.wiretag+1) // end group
+	}
+	return b, nil
+}
+
+func consumeGroupSliceInfo(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+	if wtyp != protowire.StartGroupType {
+		return unmarshalOutput{}, errUnknown
+	}
+	m := reflect.New(f.mi.GoReflectType.Elem()).Interface()
+	mp := pointerOfIface(m)
+	out, err := f.mi.unmarshalPointer(b, mp, f.num, opts)
+	if err != nil {
+		return out, err
+	}
+	p.AppendPointerSlice(mp)
+	return out, nil
+}
+
+func asMessage(v reflect.Value) pref.ProtoMessage {
+	if m, ok := v.Interface().(pref.ProtoMessage); ok {
+		return m
+	}
+	return legacyWrapMessage(v).Interface()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_gen.go b/vendor/google.golang.org/protobuf/internal/impl/codec_gen.go
new file mode 100644
index 0000000..1a509b6
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_gen.go
@@ -0,0 +1,5637 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package impl
+
+import (
+	"math"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// sizeBool returns the size of wire encoding a bool pointer as a Bool.
+func sizeBool(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Bool()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeBool(v))
+}
+
+// appendBool wire encodes a bool pointer as a Bool.
+func appendBool(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bool()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeBool(v))
+	return b, nil
+}
+
+// consumeBool wire decodes a bool pointer as a Bool.
+func consumeBool(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Bool() = protowire.DecodeBool(v)
+	out.n = n
+	return out, nil
+}
+
+var coderBool = pointerCoderFuncs{
+	size:      sizeBool,
+	marshal:   appendBool,
+	unmarshal: consumeBool,
+	merge:     mergeBool,
+}
+
+// sizeBoolNoZero returns the size of wire encoding a bool pointer as a Bool.
+// The zero value is not encoded.
+func sizeBoolNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Bool()
+	if v == false {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeBool(v))
+}
+
+// appendBoolNoZero wire encodes a bool pointer as a Bool.
+// The zero value is not encoded.
+func appendBoolNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bool()
+	if v == false {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeBool(v))
+	return b, nil
+}
+
+var coderBoolNoZero = pointerCoderFuncs{
+	size:      sizeBoolNoZero,
+	marshal:   appendBoolNoZero,
+	unmarshal: consumeBool,
+	merge:     mergeBoolNoZero,
+}
+
+// sizeBoolPtr returns the size of wire encoding a *bool pointer as a Bool.
+// It panics if the pointer is nil.
+func sizeBoolPtr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.BoolPtr()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeBool(v))
+}
+
+// appendBoolPtr wire encodes a *bool pointer as a Bool.
+// It panics if the pointer is nil.
+func appendBoolPtr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.BoolPtr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeBool(v))
+	return b, nil
+}
+
+// consumeBoolPtr wire decodes a *bool pointer as a Bool.
+func consumeBoolPtr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.BoolPtr()
+	if *vp == nil {
+		*vp = new(bool)
+	}
+	**vp = protowire.DecodeBool(v)
+	out.n = n
+	return out, nil
+}
+
+var coderBoolPtr = pointerCoderFuncs{
+	size:      sizeBoolPtr,
+	marshal:   appendBoolPtr,
+	unmarshal: consumeBoolPtr,
+	merge:     mergeBoolPtr,
+}
+
+// sizeBoolSlice returns the size of wire encoding a []bool pointer as a repeated Bool.
+func sizeBoolSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.BoolSlice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(protowire.EncodeBool(v))
+	}
+	return size
+}
+
+// appendBoolSlice encodes a []bool pointer as a repeated Bool.
+func appendBoolSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.BoolSlice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeBool(v))
+	}
+	return b, nil
+}
+
+// consumeBoolSlice wire decodes a []bool pointer as a repeated Bool.
+func consumeBoolSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.BoolSlice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, protowire.DecodeBool(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, protowire.DecodeBool(v))
+	out.n = n
+	return out, nil
+}
+
+var coderBoolSlice = pointerCoderFuncs{
+	size:      sizeBoolSlice,
+	marshal:   appendBoolSlice,
+	unmarshal: consumeBoolSlice,
+	merge:     mergeBoolSlice,
+}
+
+// sizeBoolPackedSlice returns the size of wire encoding a []bool pointer as a packed repeated Bool.
+func sizeBoolPackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.BoolSlice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeBool(v))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendBoolPackedSlice encodes a []bool pointer as a packed repeated Bool.
+func appendBoolPackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.BoolSlice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeBool(v))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, protowire.EncodeBool(v))
+	}
+	return b, nil
+}
+
+var coderBoolPackedSlice = pointerCoderFuncs{
+	size:      sizeBoolPackedSlice,
+	marshal:   appendBoolPackedSlice,
+	unmarshal: consumeBoolSlice,
+	merge:     mergeBoolSlice,
+}
+
+// sizeBoolValue returns the size of wire encoding a bool value as a Bool.
+func sizeBoolValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(protowire.EncodeBool(v.Bool()))
+}
+
+// appendBoolValue encodes a bool value as a Bool.
+func appendBoolValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeBool(v.Bool()))
+	return b, nil
+}
+
+// consumeBoolValue decodes a bool value as a Bool.
+func consumeBoolValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfBool(protowire.DecodeBool(v)), out, nil
+}
+
+var coderBoolValue = valueCoderFuncs{
+	size:      sizeBoolValue,
+	marshal:   appendBoolValue,
+	unmarshal: consumeBoolValue,
+	merge:     mergeScalarValue,
+}
+
+// sizeBoolSliceValue returns the size of wire encoding a []bool value as a repeated Bool.
+func sizeBoolSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(protowire.EncodeBool(v.Bool()))
+	}
+	return size
+}
+
+// appendBoolSliceValue encodes a []bool value as a repeated Bool.
+func appendBoolSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeBool(v.Bool()))
+	}
+	return b, nil
+}
+
+// consumeBoolSliceValue wire decodes a []bool value as a repeated Bool.
+func consumeBoolSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfBool(protowire.DecodeBool(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfBool(protowire.DecodeBool(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderBoolSliceValue = valueCoderFuncs{
+	size:      sizeBoolSliceValue,
+	marshal:   appendBoolSliceValue,
+	unmarshal: consumeBoolSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeBoolPackedSliceValue returns the size of wire encoding a []bool value as a packed repeated Bool.
+func sizeBoolPackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeBool(v.Bool()))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendBoolPackedSliceValue encodes a []bool value as a packed repeated Bool.
+func appendBoolPackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeBool(v.Bool()))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, protowire.EncodeBool(v.Bool()))
+	}
+	return b, nil
+}
+
+var coderBoolPackedSliceValue = valueCoderFuncs{
+	size:      sizeBoolPackedSliceValue,
+	marshal:   appendBoolPackedSliceValue,
+	unmarshal: consumeBoolSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeEnumValue returns the size of wire encoding a  value as a Enum.
+func sizeEnumValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(uint64(v.Enum()))
+}
+
+// appendEnumValue encodes a  value as a Enum.
+func appendEnumValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, uint64(v.Enum()))
+	return b, nil
+}
+
+// consumeEnumValue decodes a  value as a Enum.
+func consumeEnumValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)), out, nil
+}
+
+var coderEnumValue = valueCoderFuncs{
+	size:      sizeEnumValue,
+	marshal:   appendEnumValue,
+	unmarshal: consumeEnumValue,
+	merge:     mergeScalarValue,
+}
+
+// sizeEnumSliceValue returns the size of wire encoding a [] value as a repeated Enum.
+func sizeEnumSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(uint64(v.Enum()))
+	}
+	return size
+}
+
+// appendEnumSliceValue encodes a [] value as a repeated Enum.
+func appendEnumSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, uint64(v.Enum()))
+	}
+	return b, nil
+}
+
+// consumeEnumSliceValue wire decodes a [] value as a repeated Enum.
+func consumeEnumSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderEnumSliceValue = valueCoderFuncs{
+	size:      sizeEnumSliceValue,
+	marshal:   appendEnumSliceValue,
+	unmarshal: consumeEnumSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeEnumPackedSliceValue returns the size of wire encoding a [] value as a packed repeated Enum.
+func sizeEnumPackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(v.Enum()))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendEnumPackedSliceValue encodes a [] value as a packed repeated Enum.
+func appendEnumPackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(v.Enum()))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, uint64(v.Enum()))
+	}
+	return b, nil
+}
+
+var coderEnumPackedSliceValue = valueCoderFuncs{
+	size:      sizeEnumPackedSliceValue,
+	marshal:   appendEnumPackedSliceValue,
+	unmarshal: consumeEnumSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeInt32 returns the size of wire encoding a int32 pointer as a Int32.
+func sizeInt32(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int32()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt32 wire encodes a int32 pointer as a Int32.
+func appendInt32(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeInt32 wire decodes a int32 pointer as a Int32.
+func consumeInt32(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int32() = int32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderInt32 = pointerCoderFuncs{
+	size:      sizeInt32,
+	marshal:   appendInt32,
+	unmarshal: consumeInt32,
+	merge:     mergeInt32,
+}
+
+// sizeInt32NoZero returns the size of wire encoding a int32 pointer as a Int32.
+// The zero value is not encoded.
+func sizeInt32NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int32()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt32NoZero wire encodes a int32 pointer as a Int32.
+// The zero value is not encoded.
+func appendInt32NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+var coderInt32NoZero = pointerCoderFuncs{
+	size:      sizeInt32NoZero,
+	marshal:   appendInt32NoZero,
+	unmarshal: consumeInt32,
+	merge:     mergeInt32NoZero,
+}
+
+// sizeInt32Ptr returns the size of wire encoding a *int32 pointer as a Int32.
+// It panics if the pointer is nil.
+func sizeInt32Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Int32Ptr()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt32Ptr wire encodes a *int32 pointer as a Int32.
+// It panics if the pointer is nil.
+func appendInt32Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeInt32Ptr wire decodes a *int32 pointer as a Int32.
+func consumeInt32Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int32Ptr()
+	if *vp == nil {
+		*vp = new(int32)
+	}
+	**vp = int32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderInt32Ptr = pointerCoderFuncs{
+	size:      sizeInt32Ptr,
+	marshal:   appendInt32Ptr,
+	unmarshal: consumeInt32Ptr,
+	merge:     mergeInt32Ptr,
+}
+
+// sizeInt32Slice returns the size of wire encoding a []int32 pointer as a repeated Int32.
+func sizeInt32Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(uint64(v))
+	}
+	return size
+}
+
+// appendInt32Slice encodes a []int32 pointer as a repeated Int32.
+func appendInt32Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+// consumeInt32Slice wire decodes a []int32 pointer as a repeated Int32.
+func consumeInt32Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, int32(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, int32(v))
+	out.n = n
+	return out, nil
+}
+
+var coderInt32Slice = pointerCoderFuncs{
+	size:      sizeInt32Slice,
+	marshal:   appendInt32Slice,
+	unmarshal: consumeInt32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeInt32PackedSlice returns the size of wire encoding a []int32 pointer as a packed repeated Int32.
+func sizeInt32PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendInt32PackedSlice encodes a []int32 pointer as a packed repeated Int32.
+func appendInt32PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+var coderInt32PackedSlice = pointerCoderFuncs{
+	size:      sizeInt32PackedSlice,
+	marshal:   appendInt32PackedSlice,
+	unmarshal: consumeInt32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeInt32Value returns the size of wire encoding a int32 value as a Int32.
+func sizeInt32Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(uint64(int32(v.Int())))
+}
+
+// appendInt32Value encodes a int32 value as a Int32.
+func appendInt32Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, uint64(int32(v.Int())))
+	return b, nil
+}
+
+// consumeInt32Value decodes a int32 value as a Int32.
+func consumeInt32Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt32(int32(v)), out, nil
+}
+
+var coderInt32Value = valueCoderFuncs{
+	size:      sizeInt32Value,
+	marshal:   appendInt32Value,
+	unmarshal: consumeInt32Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeInt32SliceValue returns the size of wire encoding a []int32 value as a repeated Int32.
+func sizeInt32SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(uint64(int32(v.Int())))
+	}
+	return size
+}
+
+// appendInt32SliceValue encodes a []int32 value as a repeated Int32.
+func appendInt32SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, uint64(int32(v.Int())))
+	}
+	return b, nil
+}
+
+// consumeInt32SliceValue wire decodes a []int32 value as a repeated Int32.
+func consumeInt32SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt32(int32(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt32(int32(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderInt32SliceValue = valueCoderFuncs{
+	size:      sizeInt32SliceValue,
+	marshal:   appendInt32SliceValue,
+	unmarshal: consumeInt32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeInt32PackedSliceValue returns the size of wire encoding a []int32 value as a packed repeated Int32.
+func sizeInt32PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(int32(v.Int())))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendInt32PackedSliceValue encodes a []int32 value as a packed repeated Int32.
+func appendInt32PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(int32(v.Int())))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, uint64(int32(v.Int())))
+	}
+	return b, nil
+}
+
+var coderInt32PackedSliceValue = valueCoderFuncs{
+	size:      sizeInt32PackedSliceValue,
+	marshal:   appendInt32PackedSliceValue,
+	unmarshal: consumeInt32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSint32 returns the size of wire encoding a int32 pointer as a Sint32.
+func sizeSint32(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int32()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+}
+
+// appendSint32 wire encodes a int32 pointer as a Sint32.
+func appendSint32(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(v)))
+	return b, nil
+}
+
+// consumeSint32 wire decodes a int32 pointer as a Sint32.
+func consumeSint32(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int32() = int32(protowire.DecodeZigZag(v & math.MaxUint32))
+	out.n = n
+	return out, nil
+}
+
+var coderSint32 = pointerCoderFuncs{
+	size:      sizeSint32,
+	marshal:   appendSint32,
+	unmarshal: consumeSint32,
+	merge:     mergeInt32,
+}
+
+// sizeSint32NoZero returns the size of wire encoding a int32 pointer as a Sint32.
+// The zero value is not encoded.
+func sizeSint32NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int32()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+}
+
+// appendSint32NoZero wire encodes a int32 pointer as a Sint32.
+// The zero value is not encoded.
+func appendSint32NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(v)))
+	return b, nil
+}
+
+var coderSint32NoZero = pointerCoderFuncs{
+	size:      sizeSint32NoZero,
+	marshal:   appendSint32NoZero,
+	unmarshal: consumeSint32,
+	merge:     mergeInt32NoZero,
+}
+
+// sizeSint32Ptr returns the size of wire encoding a *int32 pointer as a Sint32.
+// It panics if the pointer is nil.
+func sizeSint32Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Int32Ptr()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+}
+
+// appendSint32Ptr wire encodes a *int32 pointer as a Sint32.
+// It panics if the pointer is nil.
+func appendSint32Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(v)))
+	return b, nil
+}
+
+// consumeSint32Ptr wire decodes a *int32 pointer as a Sint32.
+func consumeSint32Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int32Ptr()
+	if *vp == nil {
+		*vp = new(int32)
+	}
+	**vp = int32(protowire.DecodeZigZag(v & math.MaxUint32))
+	out.n = n
+	return out, nil
+}
+
+var coderSint32Ptr = pointerCoderFuncs{
+	size:      sizeSint32Ptr,
+	marshal:   appendSint32Ptr,
+	unmarshal: consumeSint32Ptr,
+	merge:     mergeInt32Ptr,
+}
+
+// sizeSint32Slice returns the size of wire encoding a []int32 pointer as a repeated Sint32.
+func sizeSint32Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+	}
+	return size
+}
+
+// appendSint32Slice encodes a []int32 pointer as a repeated Sint32.
+func appendSint32Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(v)))
+	}
+	return b, nil
+}
+
+// consumeSint32Slice wire decodes a []int32 pointer as a repeated Sint32.
+func consumeSint32Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, int32(protowire.DecodeZigZag(v&math.MaxUint32)))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, int32(protowire.DecodeZigZag(v&math.MaxUint32)))
+	out.n = n
+	return out, nil
+}
+
+var coderSint32Slice = pointerCoderFuncs{
+	size:      sizeSint32Slice,
+	marshal:   appendSint32Slice,
+	unmarshal: consumeSint32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeSint32PackedSlice returns the size of wire encoding a []int32 pointer as a packed repeated Sint32.
+func sizeSint32PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendSint32PackedSlice encodes a []int32 pointer as a packed repeated Sint32.
+func appendSint32PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeZigZag(int64(v)))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(v)))
+	}
+	return b, nil
+}
+
+var coderSint32PackedSlice = pointerCoderFuncs{
+	size:      sizeSint32PackedSlice,
+	marshal:   appendSint32PackedSlice,
+	unmarshal: consumeSint32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeSint32Value returns the size of wire encoding a int32 value as a Sint32.
+func sizeSint32Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(int32(v.Int()))))
+}
+
+// appendSint32Value encodes a int32 value as a Sint32.
+func appendSint32Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(int32(v.Int()))))
+	return b, nil
+}
+
+// consumeSint32Value decodes a int32 value as a Sint32.
+func consumeSint32Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))), out, nil
+}
+
+var coderSint32Value = valueCoderFuncs{
+	size:      sizeSint32Value,
+	marshal:   appendSint32Value,
+	unmarshal: consumeSint32Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeSint32SliceValue returns the size of wire encoding a []int32 value as a repeated Sint32.
+func sizeSint32SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(protowire.EncodeZigZag(int64(int32(v.Int()))))
+	}
+	return size
+}
+
+// appendSint32SliceValue encodes a []int32 value as a repeated Sint32.
+func appendSint32SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(int32(v.Int()))))
+	}
+	return b, nil
+}
+
+// consumeSint32SliceValue wire decodes a []int32 value as a repeated Sint32.
+func consumeSint32SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderSint32SliceValue = valueCoderFuncs{
+	size:      sizeSint32SliceValue,
+	marshal:   appendSint32SliceValue,
+	unmarshal: consumeSint32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSint32PackedSliceValue returns the size of wire encoding a []int32 value as a packed repeated Sint32.
+func sizeSint32PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeZigZag(int64(int32(v.Int()))))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendSint32PackedSliceValue encodes a []int32 value as a packed repeated Sint32.
+func appendSint32PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeZigZag(int64(int32(v.Int()))))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(int32(v.Int()))))
+	}
+	return b, nil
+}
+
+var coderSint32PackedSliceValue = valueCoderFuncs{
+	size:      sizeSint32PackedSliceValue,
+	marshal:   appendSint32PackedSliceValue,
+	unmarshal: consumeSint32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeUint32 returns the size of wire encoding a uint32 pointer as a Uint32.
+func sizeUint32(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint32()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendUint32 wire encodes a uint32 pointer as a Uint32.
+func appendUint32(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeUint32 wire decodes a uint32 pointer as a Uint32.
+func consumeUint32(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Uint32() = uint32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderUint32 = pointerCoderFuncs{
+	size:      sizeUint32,
+	marshal:   appendUint32,
+	unmarshal: consumeUint32,
+	merge:     mergeUint32,
+}
+
+// sizeUint32NoZero returns the size of wire encoding a uint32 pointer as a Uint32.
+// The zero value is not encoded.
+func sizeUint32NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint32()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendUint32NoZero wire encodes a uint32 pointer as a Uint32.
+// The zero value is not encoded.
+func appendUint32NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint32()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+var coderUint32NoZero = pointerCoderFuncs{
+	size:      sizeUint32NoZero,
+	marshal:   appendUint32NoZero,
+	unmarshal: consumeUint32,
+	merge:     mergeUint32NoZero,
+}
+
+// sizeUint32Ptr returns the size of wire encoding a *uint32 pointer as a Uint32.
+// It panics if the pointer is nil.
+func sizeUint32Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Uint32Ptr()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendUint32Ptr wire encodes a *uint32 pointer as a Uint32.
+// It panics if the pointer is nil.
+func appendUint32Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Uint32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeUint32Ptr wire decodes a *uint32 pointer as a Uint32.
+func consumeUint32Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Uint32Ptr()
+	if *vp == nil {
+		*vp = new(uint32)
+	}
+	**vp = uint32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderUint32Ptr = pointerCoderFuncs{
+	size:      sizeUint32Ptr,
+	marshal:   appendUint32Ptr,
+	unmarshal: consumeUint32Ptr,
+	merge:     mergeUint32Ptr,
+}
+
+// sizeUint32Slice returns the size of wire encoding a []uint32 pointer as a repeated Uint32.
+func sizeUint32Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint32Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(uint64(v))
+	}
+	return size
+}
+
+// appendUint32Slice encodes a []uint32 pointer as a repeated Uint32.
+func appendUint32Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+// consumeUint32Slice wire decodes a []uint32 pointer as a repeated Uint32.
+func consumeUint32Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Uint32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, uint32(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, uint32(v))
+	out.n = n
+	return out, nil
+}
+
+var coderUint32Slice = pointerCoderFuncs{
+	size:      sizeUint32Slice,
+	marshal:   appendUint32Slice,
+	unmarshal: consumeUint32Slice,
+	merge:     mergeUint32Slice,
+}
+
+// sizeUint32PackedSlice returns the size of wire encoding a []uint32 pointer as a packed repeated Uint32.
+func sizeUint32PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendUint32PackedSlice encodes a []uint32 pointer as a packed repeated Uint32.
+func appendUint32PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+var coderUint32PackedSlice = pointerCoderFuncs{
+	size:      sizeUint32PackedSlice,
+	marshal:   appendUint32PackedSlice,
+	unmarshal: consumeUint32Slice,
+	merge:     mergeUint32Slice,
+}
+
+// sizeUint32Value returns the size of wire encoding a uint32 value as a Uint32.
+func sizeUint32Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(uint64(uint32(v.Uint())))
+}
+
+// appendUint32Value encodes a uint32 value as a Uint32.
+func appendUint32Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, uint64(uint32(v.Uint())))
+	return b, nil
+}
+
+// consumeUint32Value decodes a uint32 value as a Uint32.
+func consumeUint32Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfUint32(uint32(v)), out, nil
+}
+
+var coderUint32Value = valueCoderFuncs{
+	size:      sizeUint32Value,
+	marshal:   appendUint32Value,
+	unmarshal: consumeUint32Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeUint32SliceValue returns the size of wire encoding a []uint32 value as a repeated Uint32.
+func sizeUint32SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(uint64(uint32(v.Uint())))
+	}
+	return size
+}
+
+// appendUint32SliceValue encodes a []uint32 value as a repeated Uint32.
+func appendUint32SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, uint64(uint32(v.Uint())))
+	}
+	return b, nil
+}
+
+// consumeUint32SliceValue wire decodes a []uint32 value as a repeated Uint32.
+func consumeUint32SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfUint32(uint32(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfUint32(uint32(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderUint32SliceValue = valueCoderFuncs{
+	size:      sizeUint32SliceValue,
+	marshal:   appendUint32SliceValue,
+	unmarshal: consumeUint32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeUint32PackedSliceValue returns the size of wire encoding a []uint32 value as a packed repeated Uint32.
+func sizeUint32PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(uint32(v.Uint())))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendUint32PackedSliceValue encodes a []uint32 value as a packed repeated Uint32.
+func appendUint32PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(uint32(v.Uint())))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, uint64(uint32(v.Uint())))
+	}
+	return b, nil
+}
+
+var coderUint32PackedSliceValue = valueCoderFuncs{
+	size:      sizeUint32PackedSliceValue,
+	marshal:   appendUint32PackedSliceValue,
+	unmarshal: consumeUint32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeInt64 returns the size of wire encoding a int64 pointer as a Int64.
+func sizeInt64(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int64()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt64 wire encodes a int64 pointer as a Int64.
+func appendInt64(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeInt64 wire decodes a int64 pointer as a Int64.
+func consumeInt64(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int64() = int64(v)
+	out.n = n
+	return out, nil
+}
+
+var coderInt64 = pointerCoderFuncs{
+	size:      sizeInt64,
+	marshal:   appendInt64,
+	unmarshal: consumeInt64,
+	merge:     mergeInt64,
+}
+
+// sizeInt64NoZero returns the size of wire encoding a int64 pointer as a Int64.
+// The zero value is not encoded.
+func sizeInt64NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int64()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt64NoZero wire encodes a int64 pointer as a Int64.
+// The zero value is not encoded.
+func appendInt64NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+var coderInt64NoZero = pointerCoderFuncs{
+	size:      sizeInt64NoZero,
+	marshal:   appendInt64NoZero,
+	unmarshal: consumeInt64,
+	merge:     mergeInt64NoZero,
+}
+
+// sizeInt64Ptr returns the size of wire encoding a *int64 pointer as a Int64.
+// It panics if the pointer is nil.
+func sizeInt64Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Int64Ptr()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+// appendInt64Ptr wire encodes a *int64 pointer as a Int64.
+// It panics if the pointer is nil.
+func appendInt64Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+// consumeInt64Ptr wire decodes a *int64 pointer as a Int64.
+func consumeInt64Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int64Ptr()
+	if *vp == nil {
+		*vp = new(int64)
+	}
+	**vp = int64(v)
+	out.n = n
+	return out, nil
+}
+
+var coderInt64Ptr = pointerCoderFuncs{
+	size:      sizeInt64Ptr,
+	marshal:   appendInt64Ptr,
+	unmarshal: consumeInt64Ptr,
+	merge:     mergeInt64Ptr,
+}
+
+// sizeInt64Slice returns the size of wire encoding a []int64 pointer as a repeated Int64.
+func sizeInt64Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(uint64(v))
+	}
+	return size
+}
+
+// appendInt64Slice encodes a []int64 pointer as a repeated Int64.
+func appendInt64Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+// consumeInt64Slice wire decodes a []int64 pointer as a repeated Int64.
+func consumeInt64Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, int64(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, int64(v))
+	out.n = n
+	return out, nil
+}
+
+var coderInt64Slice = pointerCoderFuncs{
+	size:      sizeInt64Slice,
+	marshal:   appendInt64Slice,
+	unmarshal: consumeInt64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeInt64PackedSlice returns the size of wire encoding a []int64 pointer as a packed repeated Int64.
+func sizeInt64PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendInt64PackedSlice encodes a []int64 pointer as a packed repeated Int64.
+func appendInt64PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(uint64(v))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, uint64(v))
+	}
+	return b, nil
+}
+
+var coderInt64PackedSlice = pointerCoderFuncs{
+	size:      sizeInt64PackedSlice,
+	marshal:   appendInt64PackedSlice,
+	unmarshal: consumeInt64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeInt64Value returns the size of wire encoding a int64 value as a Int64.
+func sizeInt64Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(uint64(v.Int()))
+}
+
+// appendInt64Value encodes a int64 value as a Int64.
+func appendInt64Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, uint64(v.Int()))
+	return b, nil
+}
+
+// consumeInt64Value decodes a int64 value as a Int64.
+func consumeInt64Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt64(int64(v)), out, nil
+}
+
+var coderInt64Value = valueCoderFuncs{
+	size:      sizeInt64Value,
+	marshal:   appendInt64Value,
+	unmarshal: consumeInt64Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeInt64SliceValue returns the size of wire encoding a []int64 value as a repeated Int64.
+func sizeInt64SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(uint64(v.Int()))
+	}
+	return size
+}
+
+// appendInt64SliceValue encodes a []int64 value as a repeated Int64.
+func appendInt64SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, uint64(v.Int()))
+	}
+	return b, nil
+}
+
+// consumeInt64SliceValue wire decodes a []int64 value as a repeated Int64.
+func consumeInt64SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt64(int64(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt64(int64(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderInt64SliceValue = valueCoderFuncs{
+	size:      sizeInt64SliceValue,
+	marshal:   appendInt64SliceValue,
+	unmarshal: consumeInt64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeInt64PackedSliceValue returns the size of wire encoding a []int64 value as a packed repeated Int64.
+func sizeInt64PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(v.Int()))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendInt64PackedSliceValue encodes a []int64 value as a packed repeated Int64.
+func appendInt64PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(uint64(v.Int()))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, uint64(v.Int()))
+	}
+	return b, nil
+}
+
+var coderInt64PackedSliceValue = valueCoderFuncs{
+	size:      sizeInt64PackedSliceValue,
+	marshal:   appendInt64PackedSliceValue,
+	unmarshal: consumeInt64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSint64 returns the size of wire encoding a int64 pointer as a Sint64.
+func sizeSint64(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int64()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v))
+}
+
+// appendSint64 wire encodes a int64 pointer as a Sint64.
+func appendSint64(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(v))
+	return b, nil
+}
+
+// consumeSint64 wire decodes a int64 pointer as a Sint64.
+func consumeSint64(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int64() = protowire.DecodeZigZag(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSint64 = pointerCoderFuncs{
+	size:      sizeSint64,
+	marshal:   appendSint64,
+	unmarshal: consumeSint64,
+	merge:     mergeInt64,
+}
+
+// sizeSint64NoZero returns the size of wire encoding a int64 pointer as a Sint64.
+// The zero value is not encoded.
+func sizeSint64NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int64()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v))
+}
+
+// appendSint64NoZero wire encodes a int64 pointer as a Sint64.
+// The zero value is not encoded.
+func appendSint64NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(v))
+	return b, nil
+}
+
+var coderSint64NoZero = pointerCoderFuncs{
+	size:      sizeSint64NoZero,
+	marshal:   appendSint64NoZero,
+	unmarshal: consumeSint64,
+	merge:     mergeInt64NoZero,
+}
+
+// sizeSint64Ptr returns the size of wire encoding a *int64 pointer as a Sint64.
+// It panics if the pointer is nil.
+func sizeSint64Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Int64Ptr()
+	return f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v))
+}
+
+// appendSint64Ptr wire encodes a *int64 pointer as a Sint64.
+// It panics if the pointer is nil.
+func appendSint64Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(v))
+	return b, nil
+}
+
+// consumeSint64Ptr wire decodes a *int64 pointer as a Sint64.
+func consumeSint64Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int64Ptr()
+	if *vp == nil {
+		*vp = new(int64)
+	}
+	**vp = protowire.DecodeZigZag(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSint64Ptr = pointerCoderFuncs{
+	size:      sizeSint64Ptr,
+	marshal:   appendSint64Ptr,
+	unmarshal: consumeSint64Ptr,
+	merge:     mergeInt64Ptr,
+}
+
+// sizeSint64Slice returns the size of wire encoding a []int64 pointer as a repeated Sint64.
+func sizeSint64Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v))
+	}
+	return size
+}
+
+// appendSint64Slice encodes a []int64 pointer as a repeated Sint64.
+func appendSint64Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(v))
+	}
+	return b, nil
+}
+
+// consumeSint64Slice wire decodes a []int64 pointer as a repeated Sint64.
+func consumeSint64Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, protowire.DecodeZigZag(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, protowire.DecodeZigZag(v))
+	out.n = n
+	return out, nil
+}
+
+var coderSint64Slice = pointerCoderFuncs{
+	size:      sizeSint64Slice,
+	marshal:   appendSint64Slice,
+	unmarshal: consumeSint64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeSint64PackedSlice returns the size of wire encoding a []int64 pointer as a packed repeated Sint64.
+func sizeSint64PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeZigZag(v))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendSint64PackedSlice encodes a []int64 pointer as a packed repeated Sint64.
+func appendSint64PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(protowire.EncodeZigZag(v))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(v))
+	}
+	return b, nil
+}
+
+var coderSint64PackedSlice = pointerCoderFuncs{
+	size:      sizeSint64PackedSlice,
+	marshal:   appendSint64PackedSlice,
+	unmarshal: consumeSint64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeSint64Value returns the size of wire encoding a int64 value as a Sint64.
+func sizeSint64Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v.Int()))
+}
+
+// appendSint64Value encodes a int64 value as a Sint64.
+func appendSint64Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, protowire.EncodeZigZag(v.Int()))
+	return b, nil
+}
+
+// consumeSint64Value decodes a int64 value as a Sint64.
+func consumeSint64Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)), out, nil
+}
+
+var coderSint64Value = valueCoderFuncs{
+	size:      sizeSint64Value,
+	marshal:   appendSint64Value,
+	unmarshal: consumeSint64Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeSint64SliceValue returns the size of wire encoding a []int64 value as a repeated Sint64.
+func sizeSint64SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(protowire.EncodeZigZag(v.Int()))
+	}
+	return size
+}
+
+// appendSint64SliceValue encodes a []int64 value as a repeated Sint64.
+func appendSint64SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(v.Int()))
+	}
+	return b, nil
+}
+
+// consumeSint64SliceValue wire decodes a []int64 value as a repeated Sint64.
+func consumeSint64SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderSint64SliceValue = valueCoderFuncs{
+	size:      sizeSint64SliceValue,
+	marshal:   appendSint64SliceValue,
+	unmarshal: consumeSint64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSint64PackedSliceValue returns the size of wire encoding a []int64 value as a packed repeated Sint64.
+func sizeSint64PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeZigZag(v.Int()))
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendSint64PackedSliceValue encodes a []int64 value as a packed repeated Sint64.
+func appendSint64PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(protowire.EncodeZigZag(v.Int()))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(v.Int()))
+	}
+	return b, nil
+}
+
+var coderSint64PackedSliceValue = valueCoderFuncs{
+	size:      sizeSint64PackedSliceValue,
+	marshal:   appendSint64PackedSliceValue,
+	unmarshal: consumeSint64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeUint64 returns the size of wire encoding a uint64 pointer as a Uint64.
+func sizeUint64(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint64()
+	return f.tagsize + protowire.SizeVarint(v)
+}
+
+// appendUint64 wire encodes a uint64 pointer as a Uint64.
+func appendUint64(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, v)
+	return b, nil
+}
+
+// consumeUint64 wire decodes a uint64 pointer as a Uint64.
+func consumeUint64(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Uint64() = v
+	out.n = n
+	return out, nil
+}
+
+var coderUint64 = pointerCoderFuncs{
+	size:      sizeUint64,
+	marshal:   appendUint64,
+	unmarshal: consumeUint64,
+	merge:     mergeUint64,
+}
+
+// sizeUint64NoZero returns the size of wire encoding a uint64 pointer as a Uint64.
+// The zero value is not encoded.
+func sizeUint64NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint64()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeVarint(v)
+}
+
+// appendUint64NoZero wire encodes a uint64 pointer as a Uint64.
+// The zero value is not encoded.
+func appendUint64NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint64()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, v)
+	return b, nil
+}
+
+var coderUint64NoZero = pointerCoderFuncs{
+	size:      sizeUint64NoZero,
+	marshal:   appendUint64NoZero,
+	unmarshal: consumeUint64,
+	merge:     mergeUint64NoZero,
+}
+
+// sizeUint64Ptr returns the size of wire encoding a *uint64 pointer as a Uint64.
+// It panics if the pointer is nil.
+func sizeUint64Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.Uint64Ptr()
+	return f.tagsize + protowire.SizeVarint(v)
+}
+
+// appendUint64Ptr wire encodes a *uint64 pointer as a Uint64.
+// It panics if the pointer is nil.
+func appendUint64Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Uint64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, v)
+	return b, nil
+}
+
+// consumeUint64Ptr wire decodes a *uint64 pointer as a Uint64.
+func consumeUint64Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Uint64Ptr()
+	if *vp == nil {
+		*vp = new(uint64)
+	}
+	**vp = v
+	out.n = n
+	return out, nil
+}
+
+var coderUint64Ptr = pointerCoderFuncs{
+	size:      sizeUint64Ptr,
+	marshal:   appendUint64Ptr,
+	unmarshal: consumeUint64Ptr,
+	merge:     mergeUint64Ptr,
+}
+
+// sizeUint64Slice returns the size of wire encoding a []uint64 pointer as a repeated Uint64.
+func sizeUint64Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint64Slice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeVarint(v)
+	}
+	return size
+}
+
+// appendUint64Slice encodes a []uint64 pointer as a repeated Uint64.
+func appendUint64Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, v)
+	}
+	return b, nil
+}
+
+// consumeUint64Slice wire decodes a []uint64 pointer as a repeated Uint64.
+func consumeUint64Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Uint64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, v)
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, v)
+	out.n = n
+	return out, nil
+}
+
+var coderUint64Slice = pointerCoderFuncs{
+	size:      sizeUint64Slice,
+	marshal:   appendUint64Slice,
+	unmarshal: consumeUint64Slice,
+	merge:     mergeUint64Slice,
+}
+
+// sizeUint64PackedSlice returns the size of wire encoding a []uint64 pointer as a packed repeated Uint64.
+func sizeUint64PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(v)
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendUint64PackedSlice encodes a []uint64 pointer as a packed repeated Uint64.
+func appendUint64PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for _, v := range s {
+		n += protowire.SizeVarint(v)
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendVarint(b, v)
+	}
+	return b, nil
+}
+
+var coderUint64PackedSlice = pointerCoderFuncs{
+	size:      sizeUint64PackedSlice,
+	marshal:   appendUint64PackedSlice,
+	unmarshal: consumeUint64Slice,
+	merge:     mergeUint64Slice,
+}
+
+// sizeUint64Value returns the size of wire encoding a uint64 value as a Uint64.
+func sizeUint64Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeVarint(v.Uint())
+}
+
+// appendUint64Value encodes a uint64 value as a Uint64.
+func appendUint64Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendVarint(b, v.Uint())
+	return b, nil
+}
+
+// consumeUint64Value decodes a uint64 value as a Uint64.
+func consumeUint64Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfUint64(v), out, nil
+}
+
+var coderUint64Value = valueCoderFuncs{
+	size:      sizeUint64Value,
+	marshal:   appendUint64Value,
+	unmarshal: consumeUint64Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeUint64SliceValue returns the size of wire encoding a []uint64 value as a repeated Uint64.
+func sizeUint64SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeVarint(v.Uint())
+	}
+	return size
+}
+
+// appendUint64SliceValue encodes a []uint64 value as a repeated Uint64.
+func appendUint64SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendVarint(b, v.Uint())
+	}
+	return b, nil
+}
+
+// consumeUint64SliceValue wire decodes a []uint64 value as a repeated Uint64.
+func consumeUint64SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			var v uint64
+			var n int
+			if len(b) >= 1 && b[0] < 0x80 {
+				v = uint64(b[0])
+				n = 1
+			} else if len(b) >= 2 && b[1] < 128 {
+				v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				n = 2
+			} else {
+				v, n = protowire.ConsumeVarint(b)
+			}
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfUint64(v))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	var v uint64
+	var n int
+	if len(b) >= 1 && b[0] < 0x80 {
+		v = uint64(b[0])
+		n = 1
+	} else if len(b) >= 2 && b[1] < 128 {
+		v = uint64(b[0]&0x7f) + uint64(b[1])<<7
+		n = 2
+	} else {
+		v, n = protowire.ConsumeVarint(b)
+	}
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfUint64(v))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderUint64SliceValue = valueCoderFuncs{
+	size:      sizeUint64SliceValue,
+	marshal:   appendUint64SliceValue,
+	unmarshal: consumeUint64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeUint64PackedSliceValue returns the size of wire encoding a []uint64 value as a packed repeated Uint64.
+func sizeUint64PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i, llen := 0, llen; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(v.Uint())
+	}
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendUint64PackedSliceValue encodes a []uint64 value as a packed repeated Uint64.
+func appendUint64PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		n += protowire.SizeVarint(v.Uint())
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, v.Uint())
+	}
+	return b, nil
+}
+
+var coderUint64PackedSliceValue = valueCoderFuncs{
+	size:      sizeUint64PackedSliceValue,
+	marshal:   appendUint64PackedSliceValue,
+	unmarshal: consumeUint64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSfixed32 returns the size of wire encoding a int32 pointer as a Sfixed32.
+func sizeSfixed32(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendSfixed32 wire encodes a int32 pointer as a Sfixed32.
+func appendSfixed32(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, uint32(v))
+	return b, nil
+}
+
+// consumeSfixed32 wire decodes a int32 pointer as a Sfixed32.
+func consumeSfixed32(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int32() = int32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed32 = pointerCoderFuncs{
+	size:      sizeSfixed32,
+	marshal:   appendSfixed32,
+	unmarshal: consumeSfixed32,
+	merge:     mergeInt32,
+}
+
+// sizeSfixed32NoZero returns the size of wire encoding a int32 pointer as a Sfixed32.
+// The zero value is not encoded.
+func sizeSfixed32NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int32()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendSfixed32NoZero wire encodes a int32 pointer as a Sfixed32.
+// The zero value is not encoded.
+func appendSfixed32NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int32()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, uint32(v))
+	return b, nil
+}
+
+var coderSfixed32NoZero = pointerCoderFuncs{
+	size:      sizeSfixed32NoZero,
+	marshal:   appendSfixed32NoZero,
+	unmarshal: consumeSfixed32,
+	merge:     mergeInt32NoZero,
+}
+
+// sizeSfixed32Ptr returns the size of wire encoding a *int32 pointer as a Sfixed32.
+// It panics if the pointer is nil.
+func sizeSfixed32Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendSfixed32Ptr wire encodes a *int32 pointer as a Sfixed32.
+// It panics if the pointer is nil.
+func appendSfixed32Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, uint32(v))
+	return b, nil
+}
+
+// consumeSfixed32Ptr wire decodes a *int32 pointer as a Sfixed32.
+func consumeSfixed32Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int32Ptr()
+	if *vp == nil {
+		*vp = new(int32)
+	}
+	**vp = int32(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed32Ptr = pointerCoderFuncs{
+	size:      sizeSfixed32Ptr,
+	marshal:   appendSfixed32Ptr,
+	unmarshal: consumeSfixed32Ptr,
+	merge:     mergeInt32Ptr,
+}
+
+// sizeSfixed32Slice returns the size of wire encoding a []int32 pointer as a repeated Sfixed32.
+func sizeSfixed32Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendSfixed32Slice encodes a []int32 pointer as a repeated Sfixed32.
+func appendSfixed32Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed32(b, uint32(v))
+	}
+	return b, nil
+}
+
+// consumeSfixed32Slice wire decodes a []int32 pointer as a repeated Sfixed32.
+func consumeSfixed32Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, int32(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, int32(v))
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed32Slice = pointerCoderFuncs{
+	size:      sizeSfixed32Slice,
+	marshal:   appendSfixed32Slice,
+	unmarshal: consumeSfixed32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeSfixed32PackedSlice returns the size of wire encoding a []int32 pointer as a packed repeated Sfixed32.
+func sizeSfixed32PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed32()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendSfixed32PackedSlice encodes a []int32 pointer as a packed repeated Sfixed32.
+func appendSfixed32PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed32(b, uint32(v))
+	}
+	return b, nil
+}
+
+var coderSfixed32PackedSlice = pointerCoderFuncs{
+	size:      sizeSfixed32PackedSlice,
+	marshal:   appendSfixed32PackedSlice,
+	unmarshal: consumeSfixed32Slice,
+	merge:     mergeInt32Slice,
+}
+
+// sizeSfixed32Value returns the size of wire encoding a int32 value as a Sfixed32.
+func sizeSfixed32Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed32()
+}
+
+// appendSfixed32Value encodes a int32 value as a Sfixed32.
+func appendSfixed32Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed32(b, uint32(v.Int()))
+	return b, nil
+}
+
+// consumeSfixed32Value decodes a int32 value as a Sfixed32.
+func consumeSfixed32Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt32(int32(v)), out, nil
+}
+
+var coderSfixed32Value = valueCoderFuncs{
+	size:      sizeSfixed32Value,
+	marshal:   appendSfixed32Value,
+	unmarshal: consumeSfixed32Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeSfixed32SliceValue returns the size of wire encoding a []int32 value as a repeated Sfixed32.
+func sizeSfixed32SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendSfixed32SliceValue encodes a []int32 value as a repeated Sfixed32.
+func appendSfixed32SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed32(b, uint32(v.Int()))
+	}
+	return b, nil
+}
+
+// consumeSfixed32SliceValue wire decodes a []int32 value as a repeated Sfixed32.
+func consumeSfixed32SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt32(int32(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt32(int32(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderSfixed32SliceValue = valueCoderFuncs{
+	size:      sizeSfixed32SliceValue,
+	marshal:   appendSfixed32SliceValue,
+	unmarshal: consumeSfixed32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSfixed32PackedSliceValue returns the size of wire encoding a []int32 value as a packed repeated Sfixed32.
+func sizeSfixed32PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed32()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendSfixed32PackedSliceValue encodes a []int32 value as a packed repeated Sfixed32.
+func appendSfixed32PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed32(b, uint32(v.Int()))
+	}
+	return b, nil
+}
+
+var coderSfixed32PackedSliceValue = valueCoderFuncs{
+	size:      sizeSfixed32PackedSliceValue,
+	marshal:   appendSfixed32PackedSliceValue,
+	unmarshal: consumeSfixed32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFixed32 returns the size of wire encoding a uint32 pointer as a Fixed32.
+func sizeFixed32(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFixed32 wire encodes a uint32 pointer as a Fixed32.
+func appendFixed32(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, v)
+	return b, nil
+}
+
+// consumeFixed32 wire decodes a uint32 pointer as a Fixed32.
+func consumeFixed32(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Uint32() = v
+	out.n = n
+	return out, nil
+}
+
+var coderFixed32 = pointerCoderFuncs{
+	size:      sizeFixed32,
+	marshal:   appendFixed32,
+	unmarshal: consumeFixed32,
+	merge:     mergeUint32,
+}
+
+// sizeFixed32NoZero returns the size of wire encoding a uint32 pointer as a Fixed32.
+// The zero value is not encoded.
+func sizeFixed32NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint32()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFixed32NoZero wire encodes a uint32 pointer as a Fixed32.
+// The zero value is not encoded.
+func appendFixed32NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint32()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, v)
+	return b, nil
+}
+
+var coderFixed32NoZero = pointerCoderFuncs{
+	size:      sizeFixed32NoZero,
+	marshal:   appendFixed32NoZero,
+	unmarshal: consumeFixed32,
+	merge:     mergeUint32NoZero,
+}
+
+// sizeFixed32Ptr returns the size of wire encoding a *uint32 pointer as a Fixed32.
+// It panics if the pointer is nil.
+func sizeFixed32Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFixed32Ptr wire encodes a *uint32 pointer as a Fixed32.
+// It panics if the pointer is nil.
+func appendFixed32Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Uint32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, v)
+	return b, nil
+}
+
+// consumeFixed32Ptr wire decodes a *uint32 pointer as a Fixed32.
+func consumeFixed32Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Uint32Ptr()
+	if *vp == nil {
+		*vp = new(uint32)
+	}
+	**vp = v
+	out.n = n
+	return out, nil
+}
+
+var coderFixed32Ptr = pointerCoderFuncs{
+	size:      sizeFixed32Ptr,
+	marshal:   appendFixed32Ptr,
+	unmarshal: consumeFixed32Ptr,
+	merge:     mergeUint32Ptr,
+}
+
+// sizeFixed32Slice returns the size of wire encoding a []uint32 pointer as a repeated Fixed32.
+func sizeFixed32Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint32Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendFixed32Slice encodes a []uint32 pointer as a repeated Fixed32.
+func appendFixed32Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed32(b, v)
+	}
+	return b, nil
+}
+
+// consumeFixed32Slice wire decodes a []uint32 pointer as a repeated Fixed32.
+func consumeFixed32Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Uint32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, v)
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, v)
+	out.n = n
+	return out, nil
+}
+
+var coderFixed32Slice = pointerCoderFuncs{
+	size:      sizeFixed32Slice,
+	marshal:   appendFixed32Slice,
+	unmarshal: consumeFixed32Slice,
+	merge:     mergeUint32Slice,
+}
+
+// sizeFixed32PackedSlice returns the size of wire encoding a []uint32 pointer as a packed repeated Fixed32.
+func sizeFixed32PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed32()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendFixed32PackedSlice encodes a []uint32 pointer as a packed repeated Fixed32.
+func appendFixed32PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed32(b, v)
+	}
+	return b, nil
+}
+
+var coderFixed32PackedSlice = pointerCoderFuncs{
+	size:      sizeFixed32PackedSlice,
+	marshal:   appendFixed32PackedSlice,
+	unmarshal: consumeFixed32Slice,
+	merge:     mergeUint32Slice,
+}
+
+// sizeFixed32Value returns the size of wire encoding a uint32 value as a Fixed32.
+func sizeFixed32Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed32()
+}
+
+// appendFixed32Value encodes a uint32 value as a Fixed32.
+func appendFixed32Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed32(b, uint32(v.Uint()))
+	return b, nil
+}
+
+// consumeFixed32Value decodes a uint32 value as a Fixed32.
+func consumeFixed32Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfUint32(uint32(v)), out, nil
+}
+
+var coderFixed32Value = valueCoderFuncs{
+	size:      sizeFixed32Value,
+	marshal:   appendFixed32Value,
+	unmarshal: consumeFixed32Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeFixed32SliceValue returns the size of wire encoding a []uint32 value as a repeated Fixed32.
+func sizeFixed32SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendFixed32SliceValue encodes a []uint32 value as a repeated Fixed32.
+func appendFixed32SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed32(b, uint32(v.Uint()))
+	}
+	return b, nil
+}
+
+// consumeFixed32SliceValue wire decodes a []uint32 value as a repeated Fixed32.
+func consumeFixed32SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfUint32(uint32(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfUint32(uint32(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderFixed32SliceValue = valueCoderFuncs{
+	size:      sizeFixed32SliceValue,
+	marshal:   appendFixed32SliceValue,
+	unmarshal: consumeFixed32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFixed32PackedSliceValue returns the size of wire encoding a []uint32 value as a packed repeated Fixed32.
+func sizeFixed32PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed32()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendFixed32PackedSliceValue encodes a []uint32 value as a packed repeated Fixed32.
+func appendFixed32PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed32(b, uint32(v.Uint()))
+	}
+	return b, nil
+}
+
+var coderFixed32PackedSliceValue = valueCoderFuncs{
+	size:      sizeFixed32PackedSliceValue,
+	marshal:   appendFixed32PackedSliceValue,
+	unmarshal: consumeFixed32SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFloat returns the size of wire encoding a float32 pointer as a Float.
+func sizeFloat(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFloat wire encodes a float32 pointer as a Float.
+func appendFloat(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Float32()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, math.Float32bits(v))
+	return b, nil
+}
+
+// consumeFloat wire decodes a float32 pointer as a Float.
+func consumeFloat(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Float32() = math.Float32frombits(v)
+	out.n = n
+	return out, nil
+}
+
+var coderFloat = pointerCoderFuncs{
+	size:      sizeFloat,
+	marshal:   appendFloat,
+	unmarshal: consumeFloat,
+	merge:     mergeFloat32,
+}
+
+// sizeFloatNoZero returns the size of wire encoding a float32 pointer as a Float.
+// The zero value is not encoded.
+func sizeFloatNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Float32()
+	if v == 0 && !math.Signbit(float64(v)) {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFloatNoZero wire encodes a float32 pointer as a Float.
+// The zero value is not encoded.
+func appendFloatNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Float32()
+	if v == 0 && !math.Signbit(float64(v)) {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, math.Float32bits(v))
+	return b, nil
+}
+
+var coderFloatNoZero = pointerCoderFuncs{
+	size:      sizeFloatNoZero,
+	marshal:   appendFloatNoZero,
+	unmarshal: consumeFloat,
+	merge:     mergeFloat32NoZero,
+}
+
+// sizeFloatPtr returns the size of wire encoding a *float32 pointer as a Float.
+// It panics if the pointer is nil.
+func sizeFloatPtr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed32()
+}
+
+// appendFloatPtr wire encodes a *float32 pointer as a Float.
+// It panics if the pointer is nil.
+func appendFloatPtr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Float32Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed32(b, math.Float32bits(v))
+	return b, nil
+}
+
+// consumeFloatPtr wire decodes a *float32 pointer as a Float.
+func consumeFloatPtr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Float32Ptr()
+	if *vp == nil {
+		*vp = new(float32)
+	}
+	**vp = math.Float32frombits(v)
+	out.n = n
+	return out, nil
+}
+
+var coderFloatPtr = pointerCoderFuncs{
+	size:      sizeFloatPtr,
+	marshal:   appendFloatPtr,
+	unmarshal: consumeFloatPtr,
+	merge:     mergeFloat32Ptr,
+}
+
+// sizeFloatSlice returns the size of wire encoding a []float32 pointer as a repeated Float.
+func sizeFloatSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Float32Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendFloatSlice encodes a []float32 pointer as a repeated Float.
+func appendFloatSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Float32Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed32(b, math.Float32bits(v))
+	}
+	return b, nil
+}
+
+// consumeFloatSlice wire decodes a []float32 pointer as a repeated Float.
+func consumeFloatSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Float32Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, math.Float32frombits(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, math.Float32frombits(v))
+	out.n = n
+	return out, nil
+}
+
+var coderFloatSlice = pointerCoderFuncs{
+	size:      sizeFloatSlice,
+	marshal:   appendFloatSlice,
+	unmarshal: consumeFloatSlice,
+	merge:     mergeFloat32Slice,
+}
+
+// sizeFloatPackedSlice returns the size of wire encoding a []float32 pointer as a packed repeated Float.
+func sizeFloatPackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Float32Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed32()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendFloatPackedSlice encodes a []float32 pointer as a packed repeated Float.
+func appendFloatPackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Float32Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed32(b, math.Float32bits(v))
+	}
+	return b, nil
+}
+
+var coderFloatPackedSlice = pointerCoderFuncs{
+	size:      sizeFloatPackedSlice,
+	marshal:   appendFloatPackedSlice,
+	unmarshal: consumeFloatSlice,
+	merge:     mergeFloat32Slice,
+}
+
+// sizeFloatValue returns the size of wire encoding a float32 value as a Float.
+func sizeFloatValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed32()
+}
+
+// appendFloatValue encodes a float32 value as a Float.
+func appendFloatValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed32(b, math.Float32bits(float32(v.Float())))
+	return b, nil
+}
+
+// consumeFloatValue decodes a float32 value as a Float.
+func consumeFloatValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))), out, nil
+}
+
+var coderFloatValue = valueCoderFuncs{
+	size:      sizeFloatValue,
+	marshal:   appendFloatValue,
+	unmarshal: consumeFloatValue,
+	merge:     mergeScalarValue,
+}
+
+// sizeFloatSliceValue returns the size of wire encoding a []float32 value as a repeated Float.
+func sizeFloatSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed32())
+	return size
+}
+
+// appendFloatSliceValue encodes a []float32 value as a repeated Float.
+func appendFloatSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed32(b, math.Float32bits(float32(v.Float())))
+	}
+	return b, nil
+}
+
+// consumeFloatSliceValue wire decodes a []float32 value as a repeated Float.
+func consumeFloatSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed32(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed32Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed32(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderFloatSliceValue = valueCoderFuncs{
+	size:      sizeFloatSliceValue,
+	marshal:   appendFloatSliceValue,
+	unmarshal: consumeFloatSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFloatPackedSliceValue returns the size of wire encoding a []float32 value as a packed repeated Float.
+func sizeFloatPackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed32()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendFloatPackedSliceValue encodes a []float32 value as a packed repeated Float.
+func appendFloatPackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed32()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed32(b, math.Float32bits(float32(v.Float())))
+	}
+	return b, nil
+}
+
+var coderFloatPackedSliceValue = valueCoderFuncs{
+	size:      sizeFloatPackedSliceValue,
+	marshal:   appendFloatPackedSliceValue,
+	unmarshal: consumeFloatSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSfixed64 returns the size of wire encoding a int64 pointer as a Sfixed64.
+func sizeSfixed64(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendSfixed64 wire encodes a int64 pointer as a Sfixed64.
+func appendSfixed64(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, uint64(v))
+	return b, nil
+}
+
+// consumeSfixed64 wire decodes a int64 pointer as a Sfixed64.
+func consumeSfixed64(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Int64() = int64(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed64 = pointerCoderFuncs{
+	size:      sizeSfixed64,
+	marshal:   appendSfixed64,
+	unmarshal: consumeSfixed64,
+	merge:     mergeInt64,
+}
+
+// sizeSfixed64NoZero returns the size of wire encoding a int64 pointer as a Sfixed64.
+// The zero value is not encoded.
+func sizeSfixed64NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Int64()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendSfixed64NoZero wire encodes a int64 pointer as a Sfixed64.
+// The zero value is not encoded.
+func appendSfixed64NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Int64()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, uint64(v))
+	return b, nil
+}
+
+var coderSfixed64NoZero = pointerCoderFuncs{
+	size:      sizeSfixed64NoZero,
+	marshal:   appendSfixed64NoZero,
+	unmarshal: consumeSfixed64,
+	merge:     mergeInt64NoZero,
+}
+
+// sizeSfixed64Ptr returns the size of wire encoding a *int64 pointer as a Sfixed64.
+// It panics if the pointer is nil.
+func sizeSfixed64Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendSfixed64Ptr wire encodes a *int64 pointer as a Sfixed64.
+// It panics if the pointer is nil.
+func appendSfixed64Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Int64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, uint64(v))
+	return b, nil
+}
+
+// consumeSfixed64Ptr wire decodes a *int64 pointer as a Sfixed64.
+func consumeSfixed64Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Int64Ptr()
+	if *vp == nil {
+		*vp = new(int64)
+	}
+	**vp = int64(v)
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed64Ptr = pointerCoderFuncs{
+	size:      sizeSfixed64Ptr,
+	marshal:   appendSfixed64Ptr,
+	unmarshal: consumeSfixed64Ptr,
+	merge:     mergeInt64Ptr,
+}
+
+// sizeSfixed64Slice returns the size of wire encoding a []int64 pointer as a repeated Sfixed64.
+func sizeSfixed64Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendSfixed64Slice encodes a []int64 pointer as a repeated Sfixed64.
+func appendSfixed64Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed64(b, uint64(v))
+	}
+	return b, nil
+}
+
+// consumeSfixed64Slice wire decodes a []int64 pointer as a repeated Sfixed64.
+func consumeSfixed64Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Int64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, int64(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, int64(v))
+	out.n = n
+	return out, nil
+}
+
+var coderSfixed64Slice = pointerCoderFuncs{
+	size:      sizeSfixed64Slice,
+	marshal:   appendSfixed64Slice,
+	unmarshal: consumeSfixed64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeSfixed64PackedSlice returns the size of wire encoding a []int64 pointer as a packed repeated Sfixed64.
+func sizeSfixed64PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed64()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendSfixed64PackedSlice encodes a []int64 pointer as a packed repeated Sfixed64.
+func appendSfixed64PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Int64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed64(b, uint64(v))
+	}
+	return b, nil
+}
+
+var coderSfixed64PackedSlice = pointerCoderFuncs{
+	size:      sizeSfixed64PackedSlice,
+	marshal:   appendSfixed64PackedSlice,
+	unmarshal: consumeSfixed64Slice,
+	merge:     mergeInt64Slice,
+}
+
+// sizeSfixed64Value returns the size of wire encoding a int64 value as a Sfixed64.
+func sizeSfixed64Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed64()
+}
+
+// appendSfixed64Value encodes a int64 value as a Sfixed64.
+func appendSfixed64Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed64(b, uint64(v.Int()))
+	return b, nil
+}
+
+// consumeSfixed64Value decodes a int64 value as a Sfixed64.
+func consumeSfixed64Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfInt64(int64(v)), out, nil
+}
+
+var coderSfixed64Value = valueCoderFuncs{
+	size:      sizeSfixed64Value,
+	marshal:   appendSfixed64Value,
+	unmarshal: consumeSfixed64Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeSfixed64SliceValue returns the size of wire encoding a []int64 value as a repeated Sfixed64.
+func sizeSfixed64SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendSfixed64SliceValue encodes a []int64 value as a repeated Sfixed64.
+func appendSfixed64SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed64(b, uint64(v.Int()))
+	}
+	return b, nil
+}
+
+// consumeSfixed64SliceValue wire decodes a []int64 value as a repeated Sfixed64.
+func consumeSfixed64SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfInt64(int64(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfInt64(int64(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderSfixed64SliceValue = valueCoderFuncs{
+	size:      sizeSfixed64SliceValue,
+	marshal:   appendSfixed64SliceValue,
+	unmarshal: consumeSfixed64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeSfixed64PackedSliceValue returns the size of wire encoding a []int64 value as a packed repeated Sfixed64.
+func sizeSfixed64PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed64()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendSfixed64PackedSliceValue encodes a []int64 value as a packed repeated Sfixed64.
+func appendSfixed64PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed64(b, uint64(v.Int()))
+	}
+	return b, nil
+}
+
+var coderSfixed64PackedSliceValue = valueCoderFuncs{
+	size:      sizeSfixed64PackedSliceValue,
+	marshal:   appendSfixed64PackedSliceValue,
+	unmarshal: consumeSfixed64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFixed64 returns the size of wire encoding a uint64 pointer as a Fixed64.
+func sizeFixed64(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendFixed64 wire encodes a uint64 pointer as a Fixed64.
+func appendFixed64(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, v)
+	return b, nil
+}
+
+// consumeFixed64 wire decodes a uint64 pointer as a Fixed64.
+func consumeFixed64(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Uint64() = v
+	out.n = n
+	return out, nil
+}
+
+var coderFixed64 = pointerCoderFuncs{
+	size:      sizeFixed64,
+	marshal:   appendFixed64,
+	unmarshal: consumeFixed64,
+	merge:     mergeUint64,
+}
+
+// sizeFixed64NoZero returns the size of wire encoding a uint64 pointer as a Fixed64.
+// The zero value is not encoded.
+func sizeFixed64NoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Uint64()
+	if v == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendFixed64NoZero wire encodes a uint64 pointer as a Fixed64.
+// The zero value is not encoded.
+func appendFixed64NoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Uint64()
+	if v == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, v)
+	return b, nil
+}
+
+var coderFixed64NoZero = pointerCoderFuncs{
+	size:      sizeFixed64NoZero,
+	marshal:   appendFixed64NoZero,
+	unmarshal: consumeFixed64,
+	merge:     mergeUint64NoZero,
+}
+
+// sizeFixed64Ptr returns the size of wire encoding a *uint64 pointer as a Fixed64.
+// It panics if the pointer is nil.
+func sizeFixed64Ptr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendFixed64Ptr wire encodes a *uint64 pointer as a Fixed64.
+// It panics if the pointer is nil.
+func appendFixed64Ptr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Uint64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, v)
+	return b, nil
+}
+
+// consumeFixed64Ptr wire decodes a *uint64 pointer as a Fixed64.
+func consumeFixed64Ptr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Uint64Ptr()
+	if *vp == nil {
+		*vp = new(uint64)
+	}
+	**vp = v
+	out.n = n
+	return out, nil
+}
+
+var coderFixed64Ptr = pointerCoderFuncs{
+	size:      sizeFixed64Ptr,
+	marshal:   appendFixed64Ptr,
+	unmarshal: consumeFixed64Ptr,
+	merge:     mergeUint64Ptr,
+}
+
+// sizeFixed64Slice returns the size of wire encoding a []uint64 pointer as a repeated Fixed64.
+func sizeFixed64Slice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint64Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendFixed64Slice encodes a []uint64 pointer as a repeated Fixed64.
+func appendFixed64Slice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed64(b, v)
+	}
+	return b, nil
+}
+
+// consumeFixed64Slice wire decodes a []uint64 pointer as a repeated Fixed64.
+func consumeFixed64Slice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Uint64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, v)
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, v)
+	out.n = n
+	return out, nil
+}
+
+var coderFixed64Slice = pointerCoderFuncs{
+	size:      sizeFixed64Slice,
+	marshal:   appendFixed64Slice,
+	unmarshal: consumeFixed64Slice,
+	merge:     mergeUint64Slice,
+}
+
+// sizeFixed64PackedSlice returns the size of wire encoding a []uint64 pointer as a packed repeated Fixed64.
+func sizeFixed64PackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Uint64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed64()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendFixed64PackedSlice encodes a []uint64 pointer as a packed repeated Fixed64.
+func appendFixed64PackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Uint64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed64(b, v)
+	}
+	return b, nil
+}
+
+var coderFixed64PackedSlice = pointerCoderFuncs{
+	size:      sizeFixed64PackedSlice,
+	marshal:   appendFixed64PackedSlice,
+	unmarshal: consumeFixed64Slice,
+	merge:     mergeUint64Slice,
+}
+
+// sizeFixed64Value returns the size of wire encoding a uint64 value as a Fixed64.
+func sizeFixed64Value(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed64()
+}
+
+// appendFixed64Value encodes a uint64 value as a Fixed64.
+func appendFixed64Value(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed64(b, v.Uint())
+	return b, nil
+}
+
+// consumeFixed64Value decodes a uint64 value as a Fixed64.
+func consumeFixed64Value(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfUint64(v), out, nil
+}
+
+var coderFixed64Value = valueCoderFuncs{
+	size:      sizeFixed64Value,
+	marshal:   appendFixed64Value,
+	unmarshal: consumeFixed64Value,
+	merge:     mergeScalarValue,
+}
+
+// sizeFixed64SliceValue returns the size of wire encoding a []uint64 value as a repeated Fixed64.
+func sizeFixed64SliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendFixed64SliceValue encodes a []uint64 value as a repeated Fixed64.
+func appendFixed64SliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed64(b, v.Uint())
+	}
+	return b, nil
+}
+
+// consumeFixed64SliceValue wire decodes a []uint64 value as a repeated Fixed64.
+func consumeFixed64SliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfUint64(v))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfUint64(v))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderFixed64SliceValue = valueCoderFuncs{
+	size:      sizeFixed64SliceValue,
+	marshal:   appendFixed64SliceValue,
+	unmarshal: consumeFixed64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeFixed64PackedSliceValue returns the size of wire encoding a []uint64 value as a packed repeated Fixed64.
+func sizeFixed64PackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed64()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendFixed64PackedSliceValue encodes a []uint64 value as a packed repeated Fixed64.
+func appendFixed64PackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed64(b, v.Uint())
+	}
+	return b, nil
+}
+
+var coderFixed64PackedSliceValue = valueCoderFuncs{
+	size:      sizeFixed64PackedSliceValue,
+	marshal:   appendFixed64PackedSliceValue,
+	unmarshal: consumeFixed64SliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeDouble returns the size of wire encoding a float64 pointer as a Double.
+func sizeDouble(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendDouble wire encodes a float64 pointer as a Double.
+func appendDouble(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Float64()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, math.Float64bits(v))
+	return b, nil
+}
+
+// consumeDouble wire decodes a float64 pointer as a Double.
+func consumeDouble(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Float64() = math.Float64frombits(v)
+	out.n = n
+	return out, nil
+}
+
+var coderDouble = pointerCoderFuncs{
+	size:      sizeDouble,
+	marshal:   appendDouble,
+	unmarshal: consumeDouble,
+	merge:     mergeFloat64,
+}
+
+// sizeDoubleNoZero returns the size of wire encoding a float64 pointer as a Double.
+// The zero value is not encoded.
+func sizeDoubleNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Float64()
+	if v == 0 && !math.Signbit(float64(v)) {
+		return 0
+	}
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendDoubleNoZero wire encodes a float64 pointer as a Double.
+// The zero value is not encoded.
+func appendDoubleNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Float64()
+	if v == 0 && !math.Signbit(float64(v)) {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, math.Float64bits(v))
+	return b, nil
+}
+
+var coderDoubleNoZero = pointerCoderFuncs{
+	size:      sizeDoubleNoZero,
+	marshal:   appendDoubleNoZero,
+	unmarshal: consumeDouble,
+	merge:     mergeFloat64NoZero,
+}
+
+// sizeDoublePtr returns the size of wire encoding a *float64 pointer as a Double.
+// It panics if the pointer is nil.
+func sizeDoublePtr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return f.tagsize + protowire.SizeFixed64()
+}
+
+// appendDoublePtr wire encodes a *float64 pointer as a Double.
+// It panics if the pointer is nil.
+func appendDoublePtr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.Float64Ptr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendFixed64(b, math.Float64bits(v))
+	return b, nil
+}
+
+// consumeDoublePtr wire decodes a *float64 pointer as a Double.
+func consumeDoublePtr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.Float64Ptr()
+	if *vp == nil {
+		*vp = new(float64)
+	}
+	**vp = math.Float64frombits(v)
+	out.n = n
+	return out, nil
+}
+
+var coderDoublePtr = pointerCoderFuncs{
+	size:      sizeDoublePtr,
+	marshal:   appendDoublePtr,
+	unmarshal: consumeDoublePtr,
+	merge:     mergeFloat64Ptr,
+}
+
+// sizeDoubleSlice returns the size of wire encoding a []float64 pointer as a repeated Double.
+func sizeDoubleSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Float64Slice()
+	size = len(s) * (f.tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendDoubleSlice encodes a []float64 pointer as a repeated Double.
+func appendDoubleSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Float64Slice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendFixed64(b, math.Float64bits(v))
+	}
+	return b, nil
+}
+
+// consumeDoubleSlice wire decodes a []float64 pointer as a repeated Double.
+func consumeDoubleSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.Float64Slice()
+	if wtyp == protowire.BytesType {
+		s := *sp
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			s = append(s, math.Float64frombits(v))
+			b = b[n:]
+		}
+		*sp = s
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, math.Float64frombits(v))
+	out.n = n
+	return out, nil
+}
+
+var coderDoubleSlice = pointerCoderFuncs{
+	size:      sizeDoubleSlice,
+	marshal:   appendDoubleSlice,
+	unmarshal: consumeDoubleSlice,
+	merge:     mergeFloat64Slice,
+}
+
+// sizeDoublePackedSlice returns the size of wire encoding a []float64 pointer as a packed repeated Double.
+func sizeDoublePackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.Float64Slice()
+	if len(s) == 0 {
+		return 0
+	}
+	n := len(s) * protowire.SizeFixed64()
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+// appendDoublePackedSlice encodes a []float64 pointer as a packed repeated Double.
+func appendDoublePackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.Float64Slice()
+	if len(s) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := len(s) * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for _, v := range s {
+		b = protowire.AppendFixed64(b, math.Float64bits(v))
+	}
+	return b, nil
+}
+
+var coderDoublePackedSlice = pointerCoderFuncs{
+	size:      sizeDoublePackedSlice,
+	marshal:   appendDoublePackedSlice,
+	unmarshal: consumeDoubleSlice,
+	merge:     mergeFloat64Slice,
+}
+
+// sizeDoubleValue returns the size of wire encoding a float64 value as a Double.
+func sizeDoubleValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeFixed64()
+}
+
+// appendDoubleValue encodes a float64 value as a Double.
+func appendDoubleValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendFixed64(b, math.Float64bits(v.Float()))
+	return b, nil
+}
+
+// consumeDoubleValue decodes a float64 value as a Double.
+func consumeDoubleValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfFloat64(math.Float64frombits(v)), out, nil
+}
+
+var coderDoubleValue = valueCoderFuncs{
+	size:      sizeDoubleValue,
+	marshal:   appendDoubleValue,
+	unmarshal: consumeDoubleValue,
+	merge:     mergeScalarValue,
+}
+
+// sizeDoubleSliceValue returns the size of wire encoding a []float64 value as a repeated Double.
+func sizeDoubleSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	size = list.Len() * (tagsize + protowire.SizeFixed64())
+	return size
+}
+
+// appendDoubleSliceValue encodes a []float64 value as a repeated Double.
+func appendDoubleSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendFixed64(b, math.Float64bits(v.Float()))
+	}
+	return b, nil
+}
+
+// consumeDoubleSliceValue wire decodes a []float64 value as a repeated Double.
+func consumeDoubleSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return protoreflect.Value{}, out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeFixed64(b)
+			if n < 0 {
+				return protoreflect.Value{}, out, errDecode
+			}
+			list.Append(protoreflect.ValueOfFloat64(math.Float64frombits(v)))
+			b = b[n:]
+		}
+		out.n = n
+		return listv, out, nil
+	}
+	if wtyp != protowire.Fixed64Type {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeFixed64(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfFloat64(math.Float64frombits(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderDoubleSliceValue = valueCoderFuncs{
+	size:      sizeDoubleSliceValue,
+	marshal:   appendDoubleSliceValue,
+	unmarshal: consumeDoubleSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeDoublePackedSliceValue returns the size of wire encoding a []float64 value as a packed repeated Double.
+func sizeDoublePackedSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := llen * protowire.SizeFixed64()
+	return tagsize + protowire.SizeBytes(n)
+}
+
+// appendDoublePackedSliceValue encodes a []float64 value as a packed repeated Double.
+func appendDoublePackedSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	llen := list.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, wiretag)
+	n := llen * protowire.SizeFixed64()
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendFixed64(b, math.Float64bits(v.Float()))
+	}
+	return b, nil
+}
+
+var coderDoublePackedSliceValue = valueCoderFuncs{
+	size:      sizeDoublePackedSliceValue,
+	marshal:   appendDoublePackedSliceValue,
+	unmarshal: consumeDoubleSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeString returns the size of wire encoding a string pointer as a String.
+func sizeString(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.String()
+	return f.tagsize + protowire.SizeBytes(len(v))
+}
+
+// appendString wire encodes a string pointer as a String.
+func appendString(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.String()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	return b, nil
+}
+
+// consumeString wire decodes a string pointer as a String.
+func consumeString(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.String() = string(v)
+	out.n = n
+	return out, nil
+}
+
+var coderString = pointerCoderFuncs{
+	size:      sizeString,
+	marshal:   appendString,
+	unmarshal: consumeString,
+	merge:     mergeString,
+}
+
+// appendStringValidateUTF8 wire encodes a string pointer as a String.
+func appendStringValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.String()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	if !utf8.ValidString(v) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+// consumeStringValidateUTF8 wire decodes a string pointer as a String.
+func consumeStringValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	*p.String() = string(v)
+	out.n = n
+	return out, nil
+}
+
+var coderStringValidateUTF8 = pointerCoderFuncs{
+	size:      sizeString,
+	marshal:   appendStringValidateUTF8,
+	unmarshal: consumeStringValidateUTF8,
+	merge:     mergeString,
+}
+
+// sizeStringNoZero returns the size of wire encoding a string pointer as a String.
+// The zero value is not encoded.
+func sizeStringNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.String()
+	if len(v) == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeBytes(len(v))
+}
+
+// appendStringNoZero wire encodes a string pointer as a String.
+// The zero value is not encoded.
+func appendStringNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.String()
+	if len(v) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	return b, nil
+}
+
+var coderStringNoZero = pointerCoderFuncs{
+	size:      sizeStringNoZero,
+	marshal:   appendStringNoZero,
+	unmarshal: consumeString,
+	merge:     mergeStringNoZero,
+}
+
+// appendStringNoZeroValidateUTF8 wire encodes a string pointer as a String.
+// The zero value is not encoded.
+func appendStringNoZeroValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.String()
+	if len(v) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	if !utf8.ValidString(v) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+var coderStringNoZeroValidateUTF8 = pointerCoderFuncs{
+	size:      sizeStringNoZero,
+	marshal:   appendStringNoZeroValidateUTF8,
+	unmarshal: consumeStringValidateUTF8,
+	merge:     mergeStringNoZero,
+}
+
+// sizeStringPtr returns the size of wire encoding a *string pointer as a String.
+// It panics if the pointer is nil.
+func sizeStringPtr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := **p.StringPtr()
+	return f.tagsize + protowire.SizeBytes(len(v))
+}
+
+// appendStringPtr wire encodes a *string pointer as a String.
+// It panics if the pointer is nil.
+func appendStringPtr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.StringPtr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	return b, nil
+}
+
+// consumeStringPtr wire decodes a *string pointer as a String.
+func consumeStringPtr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	vp := p.StringPtr()
+	if *vp == nil {
+		*vp = new(string)
+	}
+	**vp = string(v)
+	out.n = n
+	return out, nil
+}
+
+var coderStringPtr = pointerCoderFuncs{
+	size:      sizeStringPtr,
+	marshal:   appendStringPtr,
+	unmarshal: consumeStringPtr,
+	merge:     mergeStringPtr,
+}
+
+// appendStringPtrValidateUTF8 wire encodes a *string pointer as a String.
+// It panics if the pointer is nil.
+func appendStringPtrValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := **p.StringPtr()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendString(b, v)
+	if !utf8.ValidString(v) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+// consumeStringPtrValidateUTF8 wire decodes a *string pointer as a String.
+func consumeStringPtrValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	vp := p.StringPtr()
+	if *vp == nil {
+		*vp = new(string)
+	}
+	**vp = string(v)
+	out.n = n
+	return out, nil
+}
+
+var coderStringPtrValidateUTF8 = pointerCoderFuncs{
+	size:      sizeStringPtr,
+	marshal:   appendStringPtrValidateUTF8,
+	unmarshal: consumeStringPtrValidateUTF8,
+	merge:     mergeStringPtr,
+}
+
+// sizeStringSlice returns the size of wire encoding a []string pointer as a repeated String.
+func sizeStringSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.StringSlice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeBytes(len(v))
+	}
+	return size
+}
+
+// appendStringSlice encodes a []string pointer as a repeated String.
+func appendStringSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.StringSlice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendString(b, v)
+	}
+	return b, nil
+}
+
+// consumeStringSlice wire decodes a []string pointer as a repeated String.
+func consumeStringSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.StringSlice()
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, string(v))
+	out.n = n
+	return out, nil
+}
+
+var coderStringSlice = pointerCoderFuncs{
+	size:      sizeStringSlice,
+	marshal:   appendStringSlice,
+	unmarshal: consumeStringSlice,
+	merge:     mergeStringSlice,
+}
+
+// appendStringSliceValidateUTF8 encodes a []string pointer as a repeated String.
+func appendStringSliceValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.StringSlice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendString(b, v)
+		if !utf8.ValidString(v) {
+			return b, errInvalidUTF8{}
+		}
+	}
+	return b, nil
+}
+
+// consumeStringSliceValidateUTF8 wire decodes a []string pointer as a repeated String.
+func consumeStringSliceValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	sp := p.StringSlice()
+	*sp = append(*sp, string(v))
+	out.n = n
+	return out, nil
+}
+
+var coderStringSliceValidateUTF8 = pointerCoderFuncs{
+	size:      sizeStringSlice,
+	marshal:   appendStringSliceValidateUTF8,
+	unmarshal: consumeStringSliceValidateUTF8,
+	merge:     mergeStringSlice,
+}
+
+// sizeStringValue returns the size of wire encoding a string value as a String.
+func sizeStringValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeBytes(len(v.String()))
+}
+
+// appendStringValue encodes a string value as a String.
+func appendStringValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendString(b, v.String())
+	return b, nil
+}
+
+// consumeStringValue decodes a string value as a String.
+func consumeStringValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfString(string(v)), out, nil
+}
+
+var coderStringValue = valueCoderFuncs{
+	size:      sizeStringValue,
+	marshal:   appendStringValue,
+	unmarshal: consumeStringValue,
+	merge:     mergeScalarValue,
+}
+
+// appendStringValueValidateUTF8 encodes a string value as a String.
+func appendStringValueValidateUTF8(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendString(b, v.String())
+	if !utf8.ValidString(v.String()) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+// consumeStringValueValidateUTF8 decodes a string value as a String.
+func consumeStringValueValidateUTF8(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return protoreflect.Value{}, out, errInvalidUTF8{}
+	}
+	out.n = n
+	return protoreflect.ValueOfString(string(v)), out, nil
+}
+
+var coderStringValueValidateUTF8 = valueCoderFuncs{
+	size:      sizeStringValue,
+	marshal:   appendStringValueValidateUTF8,
+	unmarshal: consumeStringValueValidateUTF8,
+	merge:     mergeScalarValue,
+}
+
+// sizeStringSliceValue returns the size of wire encoding a []string value as a repeated String.
+func sizeStringSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeBytes(len(v.String()))
+	}
+	return size
+}
+
+// appendStringSliceValue encodes a []string value as a repeated String.
+func appendStringSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendString(b, v.String())
+	}
+	return b, nil
+}
+
+// consumeStringSliceValue wire decodes a []string value as a repeated String.
+func consumeStringSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp != protowire.BytesType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfString(string(v)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderStringSliceValue = valueCoderFuncs{
+	size:      sizeStringSliceValue,
+	marshal:   appendStringSliceValue,
+	unmarshal: consumeStringSliceValue,
+	merge:     mergeListValue,
+}
+
+// sizeBytes returns the size of wire encoding a []byte pointer as a Bytes.
+func sizeBytes(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Bytes()
+	return f.tagsize + protowire.SizeBytes(len(v))
+}
+
+// appendBytes wire encodes a []byte pointer as a Bytes.
+func appendBytes(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bytes()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendBytes(b, v)
+	return b, nil
+}
+
+// consumeBytes wire decodes a []byte pointer as a Bytes.
+func consumeBytes(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Bytes() = append(emptyBuf[:], v...)
+	out.n = n
+	return out, nil
+}
+
+var coderBytes = pointerCoderFuncs{
+	size:      sizeBytes,
+	marshal:   appendBytes,
+	unmarshal: consumeBytes,
+	merge:     mergeBytes,
+}
+
+// appendBytesValidateUTF8 wire encodes a []byte pointer as a Bytes.
+func appendBytesValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bytes()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendBytes(b, v)
+	if !utf8.Valid(v) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+// consumeBytesValidateUTF8 wire decodes a []byte pointer as a Bytes.
+func consumeBytesValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	*p.Bytes() = append(emptyBuf[:], v...)
+	out.n = n
+	return out, nil
+}
+
+var coderBytesValidateUTF8 = pointerCoderFuncs{
+	size:      sizeBytes,
+	marshal:   appendBytesValidateUTF8,
+	unmarshal: consumeBytesValidateUTF8,
+	merge:     mergeBytes,
+}
+
+// sizeBytesNoZero returns the size of wire encoding a []byte pointer as a Bytes.
+// The zero value is not encoded.
+func sizeBytesNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	v := *p.Bytes()
+	if len(v) == 0 {
+		return 0
+	}
+	return f.tagsize + protowire.SizeBytes(len(v))
+}
+
+// appendBytesNoZero wire encodes a []byte pointer as a Bytes.
+// The zero value is not encoded.
+func appendBytesNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bytes()
+	if len(v) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendBytes(b, v)
+	return b, nil
+}
+
+// consumeBytesNoZero wire decodes a []byte pointer as a Bytes.
+// The zero value is not decoded.
+func consumeBytesNoZero(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*p.Bytes() = append(([]byte)(nil), v...)
+	out.n = n
+	return out, nil
+}
+
+var coderBytesNoZero = pointerCoderFuncs{
+	size:      sizeBytesNoZero,
+	marshal:   appendBytesNoZero,
+	unmarshal: consumeBytesNoZero,
+	merge:     mergeBytesNoZero,
+}
+
+// appendBytesNoZeroValidateUTF8 wire encodes a []byte pointer as a Bytes.
+// The zero value is not encoded.
+func appendBytesNoZeroValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := *p.Bytes()
+	if len(v) == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendBytes(b, v)
+	if !utf8.Valid(v) {
+		return b, errInvalidUTF8{}
+	}
+	return b, nil
+}
+
+// consumeBytesNoZeroValidateUTF8 wire decodes a []byte pointer as a Bytes.
+func consumeBytesNoZeroValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	*p.Bytes() = append(([]byte)(nil), v...)
+	out.n = n
+	return out, nil
+}
+
+var coderBytesNoZeroValidateUTF8 = pointerCoderFuncs{
+	size:      sizeBytesNoZero,
+	marshal:   appendBytesNoZeroValidateUTF8,
+	unmarshal: consumeBytesNoZeroValidateUTF8,
+	merge:     mergeBytesNoZero,
+}
+
+// sizeBytesSlice returns the size of wire encoding a [][]byte pointer as a repeated Bytes.
+func sizeBytesSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := *p.BytesSlice()
+	for _, v := range s {
+		size += f.tagsize + protowire.SizeBytes(len(v))
+	}
+	return size
+}
+
+// appendBytesSlice encodes a [][]byte pointer as a repeated Bytes.
+func appendBytesSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.BytesSlice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendBytes(b, v)
+	}
+	return b, nil
+}
+
+// consumeBytesSlice wire decodes a [][]byte pointer as a repeated Bytes.
+func consumeBytesSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	sp := p.BytesSlice()
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	*sp = append(*sp, append(emptyBuf[:], v...))
+	out.n = n
+	return out, nil
+}
+
+var coderBytesSlice = pointerCoderFuncs{
+	size:      sizeBytesSlice,
+	marshal:   appendBytesSlice,
+	unmarshal: consumeBytesSlice,
+	merge:     mergeBytesSlice,
+}
+
+// appendBytesSliceValidateUTF8 encodes a [][]byte pointer as a repeated Bytes.
+func appendBytesSliceValidateUTF8(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := *p.BytesSlice()
+	for _, v := range s {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendBytes(b, v)
+		if !utf8.Valid(v) {
+			return b, errInvalidUTF8{}
+		}
+	}
+	return b, nil
+}
+
+// consumeBytesSliceValidateUTF8 wire decodes a [][]byte pointer as a repeated Bytes.
+func consumeBytesSliceValidateUTF8(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	if !utf8.Valid(v) {
+		return out, errInvalidUTF8{}
+	}
+	sp := p.BytesSlice()
+	*sp = append(*sp, append(emptyBuf[:], v...))
+	out.n = n
+	return out, nil
+}
+
+var coderBytesSliceValidateUTF8 = pointerCoderFuncs{
+	size:      sizeBytesSlice,
+	marshal:   appendBytesSliceValidateUTF8,
+	unmarshal: consumeBytesSliceValidateUTF8,
+	merge:     mergeBytesSlice,
+}
+
+// sizeBytesValue returns the size of wire encoding a []byte value as a Bytes.
+func sizeBytesValue(v protoreflect.Value, tagsize int, opts marshalOptions) int {
+	return tagsize + protowire.SizeBytes(len(v.Bytes()))
+}
+
+// appendBytesValue encodes a []byte value as a Bytes.
+func appendBytesValue(b []byte, v protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	b = protowire.AppendVarint(b, wiretag)
+	b = protowire.AppendBytes(b, v.Bytes())
+	return b, nil
+}
+
+// consumeBytesValue decodes a []byte value as a Bytes.
+func consumeBytesValue(b []byte, _ protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	out.n = n
+	return protoreflect.ValueOfBytes(append(emptyBuf[:], v...)), out, nil
+}
+
+var coderBytesValue = valueCoderFuncs{
+	size:      sizeBytesValue,
+	marshal:   appendBytesValue,
+	unmarshal: consumeBytesValue,
+	merge:     mergeBytesValue,
+}
+
+// sizeBytesSliceValue returns the size of wire encoding a [][]byte value as a repeated Bytes.
+func sizeBytesSliceValue(listv protoreflect.Value, tagsize int, opts marshalOptions) (size int) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		size += tagsize + protowire.SizeBytes(len(v.Bytes()))
+	}
+	return size
+}
+
+// appendBytesSliceValue encodes a [][]byte value as a repeated Bytes.
+func appendBytesSliceValue(b []byte, listv protoreflect.Value, wiretag uint64, opts marshalOptions) ([]byte, error) {
+	list := listv.List()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		v := list.Get(i)
+		b = protowire.AppendVarint(b, wiretag)
+		b = protowire.AppendBytes(b, v.Bytes())
+	}
+	return b, nil
+}
+
+// consumeBytesSliceValue wire decodes a [][]byte value as a repeated Bytes.
+func consumeBytesSliceValue(b []byte, listv protoreflect.Value, _ protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (_ protoreflect.Value, out unmarshalOutput, err error) {
+	list := listv.List()
+	if wtyp != protowire.BytesType {
+		return protoreflect.Value{}, out, errUnknown
+	}
+	v, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return protoreflect.Value{}, out, errDecode
+	}
+	list.Append(protoreflect.ValueOfBytes(append(emptyBuf[:], v...)))
+	out.n = n
+	return listv, out, nil
+}
+
+var coderBytesSliceValue = valueCoderFuncs{
+	size:      sizeBytesSliceValue,
+	marshal:   appendBytesSliceValue,
+	unmarshal: consumeBytesSliceValue,
+	merge:     mergeBytesListValue,
+}
+
+// We append to an empty array rather than a nil []byte to get non-nil zero-length byte slices.
+var emptyBuf [0]byte
+
+var wireTypes = map[protoreflect.Kind]protowire.Type{
+	protoreflect.BoolKind:     protowire.VarintType,
+	protoreflect.EnumKind:     protowire.VarintType,
+	protoreflect.Int32Kind:    protowire.VarintType,
+	protoreflect.Sint32Kind:   protowire.VarintType,
+	protoreflect.Uint32Kind:   protowire.VarintType,
+	protoreflect.Int64Kind:    protowire.VarintType,
+	protoreflect.Sint64Kind:   protowire.VarintType,
+	protoreflect.Uint64Kind:   protowire.VarintType,
+	protoreflect.Sfixed32Kind: protowire.Fixed32Type,
+	protoreflect.Fixed32Kind:  protowire.Fixed32Type,
+	protoreflect.FloatKind:    protowire.Fixed32Type,
+	protoreflect.Sfixed64Kind: protowire.Fixed64Type,
+	protoreflect.Fixed64Kind:  protowire.Fixed64Type,
+	protoreflect.DoubleKind:   protowire.Fixed64Type,
+	protoreflect.StringKind:   protowire.BytesType,
+	protoreflect.BytesKind:    protowire.BytesType,
+	protoreflect.MessageKind:  protowire.BytesType,
+	protoreflect.GroupKind:    protowire.StartGroupType,
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_map.go b/vendor/google.golang.org/protobuf/internal/impl/codec_map.go
new file mode 100644
index 0000000..c1245fe
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_map.go
@@ -0,0 +1,388 @@
+// 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.
+
+package impl
+
+import (
+	"reflect"
+	"sort"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/genid"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type mapInfo struct {
+	goType     reflect.Type
+	keyWiretag uint64
+	valWiretag uint64
+	keyFuncs   valueCoderFuncs
+	valFuncs   valueCoderFuncs
+	keyZero    pref.Value
+	keyKind    pref.Kind
+	conv       *mapConverter
+}
+
+func encoderFuncsForMap(fd pref.FieldDescriptor, ft reflect.Type) (valueMessage *MessageInfo, funcs pointerCoderFuncs) {
+	// TODO: Consider generating specialized map coders.
+	keyField := fd.MapKey()
+	valField := fd.MapValue()
+	keyWiretag := protowire.EncodeTag(1, wireTypes[keyField.Kind()])
+	valWiretag := protowire.EncodeTag(2, wireTypes[valField.Kind()])
+	keyFuncs := encoderFuncsForValue(keyField)
+	valFuncs := encoderFuncsForValue(valField)
+	conv := newMapConverter(ft, fd)
+
+	mapi := &mapInfo{
+		goType:     ft,
+		keyWiretag: keyWiretag,
+		valWiretag: valWiretag,
+		keyFuncs:   keyFuncs,
+		valFuncs:   valFuncs,
+		keyZero:    keyField.Default(),
+		keyKind:    keyField.Kind(),
+		conv:       conv,
+	}
+	if valField.Kind() == pref.MessageKind {
+		valueMessage = getMessageInfo(ft.Elem())
+	}
+
+	funcs = pointerCoderFuncs{
+		size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+			return sizeMap(p.AsValueOf(ft).Elem(), mapi, f, opts)
+		},
+		marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+			return appendMap(b, p.AsValueOf(ft).Elem(), mapi, f, opts)
+		},
+		unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+			mp := p.AsValueOf(ft)
+			if mp.Elem().IsNil() {
+				mp.Elem().Set(reflect.MakeMap(mapi.goType))
+			}
+			if f.mi == nil {
+				return consumeMap(b, mp.Elem(), wtyp, mapi, f, opts)
+			} else {
+				return consumeMapOfMessage(b, mp.Elem(), wtyp, mapi, f, opts)
+			}
+		},
+	}
+	switch valField.Kind() {
+	case pref.MessageKind:
+		funcs.merge = mergeMapOfMessage
+	case pref.BytesKind:
+		funcs.merge = mergeMapOfBytes
+	default:
+		funcs.merge = mergeMap
+	}
+	if valFuncs.isInit != nil {
+		funcs.isInit = func(p pointer, f *coderFieldInfo) error {
+			return isInitMap(p.AsValueOf(ft).Elem(), mapi, f)
+		}
+	}
+	return valueMessage, funcs
+}
+
+const (
+	mapKeyTagSize = 1 // field 1, tag size 1.
+	mapValTagSize = 1 // field 2, tag size 2.
+)
+
+func sizeMap(mapv reflect.Value, mapi *mapInfo, f *coderFieldInfo, opts marshalOptions) int {
+	if mapv.Len() == 0 {
+		return 0
+	}
+	n := 0
+	iter := mapRange(mapv)
+	for iter.Next() {
+		key := mapi.conv.keyConv.PBValueOf(iter.Key()).MapKey()
+		keySize := mapi.keyFuncs.size(key.Value(), mapKeyTagSize, opts)
+		var valSize int
+		value := mapi.conv.valConv.PBValueOf(iter.Value())
+		if f.mi == nil {
+			valSize = mapi.valFuncs.size(value, mapValTagSize, opts)
+		} else {
+			p := pointerOfValue(iter.Value())
+			valSize += mapValTagSize
+			valSize += protowire.SizeBytes(f.mi.sizePointer(p, opts))
+		}
+		n += f.tagsize + protowire.SizeBytes(keySize+valSize)
+	}
+	return n
+}
+
+func consumeMap(b []byte, mapv reflect.Value, wtyp protowire.Type, mapi *mapInfo, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	b, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	var (
+		key = mapi.keyZero
+		val = mapi.conv.valConv.New()
+	)
+	for len(b) > 0 {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		if num > protowire.MaxValidNumber {
+			return out, errDecode
+		}
+		b = b[n:]
+		err := errUnknown
+		switch num {
+		case genid.MapEntry_Key_field_number:
+			var v pref.Value
+			var o unmarshalOutput
+			v, o, err = mapi.keyFuncs.unmarshal(b, key, num, wtyp, opts)
+			if err != nil {
+				break
+			}
+			key = v
+			n = o.n
+		case genid.MapEntry_Value_field_number:
+			var v pref.Value
+			var o unmarshalOutput
+			v, o, err = mapi.valFuncs.unmarshal(b, val, num, wtyp, opts)
+			if err != nil {
+				break
+			}
+			val = v
+			n = o.n
+		}
+		if err == errUnknown {
+			n = protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return out, errDecode
+			}
+		} else if err != nil {
+			return out, err
+		}
+		b = b[n:]
+	}
+	mapv.SetMapIndex(mapi.conv.keyConv.GoValueOf(key), mapi.conv.valConv.GoValueOf(val))
+	out.n = n
+	return out, nil
+}
+
+func consumeMapOfMessage(b []byte, mapv reflect.Value, wtyp protowire.Type, mapi *mapInfo, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.BytesType {
+		return out, errUnknown
+	}
+	b, n := protowire.ConsumeBytes(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	var (
+		key = mapi.keyZero
+		val = reflect.New(f.mi.GoReflectType.Elem())
+	)
+	for len(b) > 0 {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		if num > protowire.MaxValidNumber {
+			return out, errDecode
+		}
+		b = b[n:]
+		err := errUnknown
+		switch num {
+		case 1:
+			var v pref.Value
+			var o unmarshalOutput
+			v, o, err = mapi.keyFuncs.unmarshal(b, key, num, wtyp, opts)
+			if err != nil {
+				break
+			}
+			key = v
+			n = o.n
+		case 2:
+			if wtyp != protowire.BytesType {
+				break
+			}
+			var v []byte
+			v, n = protowire.ConsumeBytes(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			var o unmarshalOutput
+			o, err = f.mi.unmarshalPointer(v, pointerOfValue(val), 0, opts)
+			if o.initialized {
+				// Consider this map item initialized so long as we see
+				// an initialized value.
+				out.initialized = true
+			}
+		}
+		if err == errUnknown {
+			n = protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return out, errDecode
+			}
+		} else if err != nil {
+			return out, err
+		}
+		b = b[n:]
+	}
+	mapv.SetMapIndex(mapi.conv.keyConv.GoValueOf(key), val)
+	out.n = n
+	return out, nil
+}
+
+func appendMapItem(b []byte, keyrv, valrv reflect.Value, mapi *mapInfo, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	if f.mi == nil {
+		key := mapi.conv.keyConv.PBValueOf(keyrv).MapKey()
+		val := mapi.conv.valConv.PBValueOf(valrv)
+		size := 0
+		size += mapi.keyFuncs.size(key.Value(), mapKeyTagSize, opts)
+		size += mapi.valFuncs.size(val, mapValTagSize, opts)
+		b = protowire.AppendVarint(b, uint64(size))
+		b, err := mapi.keyFuncs.marshal(b, key.Value(), mapi.keyWiretag, opts)
+		if err != nil {
+			return nil, err
+		}
+		return mapi.valFuncs.marshal(b, val, mapi.valWiretag, opts)
+	} else {
+		key := mapi.conv.keyConv.PBValueOf(keyrv).MapKey()
+		val := pointerOfValue(valrv)
+		valSize := f.mi.sizePointer(val, opts)
+		size := 0
+		size += mapi.keyFuncs.size(key.Value(), mapKeyTagSize, opts)
+		size += mapValTagSize + protowire.SizeBytes(valSize)
+		b = protowire.AppendVarint(b, uint64(size))
+		b, err := mapi.keyFuncs.marshal(b, key.Value(), mapi.keyWiretag, opts)
+		if err != nil {
+			return nil, err
+		}
+		b = protowire.AppendVarint(b, mapi.valWiretag)
+		b = protowire.AppendVarint(b, uint64(valSize))
+		return f.mi.marshalAppendPointer(b, val, opts)
+	}
+}
+
+func appendMap(b []byte, mapv reflect.Value, mapi *mapInfo, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	if mapv.Len() == 0 {
+		return b, nil
+	}
+	if opts.Deterministic() {
+		return appendMapDeterministic(b, mapv, mapi, f, opts)
+	}
+	iter := mapRange(mapv)
+	for iter.Next() {
+		var err error
+		b = protowire.AppendVarint(b, f.wiretag)
+		b, err = appendMapItem(b, iter.Key(), iter.Value(), mapi, f, opts)
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func appendMapDeterministic(b []byte, mapv reflect.Value, mapi *mapInfo, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	keys := mapv.MapKeys()
+	sort.Slice(keys, func(i, j int) bool {
+		switch keys[i].Kind() {
+		case reflect.Bool:
+			return !keys[i].Bool() && keys[j].Bool()
+		case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
+			return keys[i].Int() < keys[j].Int()
+		case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr:
+			return keys[i].Uint() < keys[j].Uint()
+		case reflect.Float32, reflect.Float64:
+			return keys[i].Float() < keys[j].Float()
+		case reflect.String:
+			return keys[i].String() < keys[j].String()
+		default:
+			panic("invalid kind: " + keys[i].Kind().String())
+		}
+	})
+	for _, key := range keys {
+		var err error
+		b = protowire.AppendVarint(b, f.wiretag)
+		b, err = appendMapItem(b, key, mapv.MapIndex(key), mapi, f, opts)
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func isInitMap(mapv reflect.Value, mapi *mapInfo, f *coderFieldInfo) error {
+	if mi := f.mi; mi != nil {
+		mi.init()
+		if !mi.needsInitCheck {
+			return nil
+		}
+		iter := mapRange(mapv)
+		for iter.Next() {
+			val := pointerOfValue(iter.Value())
+			if err := mi.checkInitializedPointer(val); err != nil {
+				return err
+			}
+		}
+	} else {
+		iter := mapRange(mapv)
+		for iter.Next() {
+			val := mapi.conv.valConv.PBValueOf(iter.Value())
+			if err := mapi.valFuncs.isInit(val); err != nil {
+				return err
+			}
+		}
+	}
+	return nil
+}
+
+func mergeMap(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+	dstm := dst.AsValueOf(f.ft).Elem()
+	srcm := src.AsValueOf(f.ft).Elem()
+	if srcm.Len() == 0 {
+		return
+	}
+	if dstm.IsNil() {
+		dstm.Set(reflect.MakeMap(f.ft))
+	}
+	iter := mapRange(srcm)
+	for iter.Next() {
+		dstm.SetMapIndex(iter.Key(), iter.Value())
+	}
+}
+
+func mergeMapOfBytes(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+	dstm := dst.AsValueOf(f.ft).Elem()
+	srcm := src.AsValueOf(f.ft).Elem()
+	if srcm.Len() == 0 {
+		return
+	}
+	if dstm.IsNil() {
+		dstm.Set(reflect.MakeMap(f.ft))
+	}
+	iter := mapRange(srcm)
+	for iter.Next() {
+		dstm.SetMapIndex(iter.Key(), reflect.ValueOf(append(emptyBuf[:], iter.Value().Bytes()...)))
+	}
+}
+
+func mergeMapOfMessage(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+	dstm := dst.AsValueOf(f.ft).Elem()
+	srcm := src.AsValueOf(f.ft).Elem()
+	if srcm.Len() == 0 {
+		return
+	}
+	if dstm.IsNil() {
+		dstm.Set(reflect.MakeMap(f.ft))
+	}
+	iter := mapRange(srcm)
+	for iter.Next() {
+		val := reflect.New(f.ft.Elem().Elem())
+		if f.mi != nil {
+			f.mi.mergePointer(pointerOfValue(val), pointerOfValue(iter.Value()), opts)
+		} else {
+			opts.Merge(asMessage(val), asMessage(iter.Value()))
+		}
+		dstm.SetMapIndex(iter.Key(), val)
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_map_go111.go b/vendor/google.golang.org/protobuf/internal/impl/codec_map_go111.go
new file mode 100644
index 0000000..2706bb6
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_map_go111.go
@@ -0,0 +1,37 @@
+// 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.
+
+// +build !go1.12
+
+package impl
+
+import "reflect"
+
+type mapIter struct {
+	v    reflect.Value
+	keys []reflect.Value
+}
+
+// mapRange provides a less-efficient equivalent to
+// the Go 1.12 reflect.Value.MapRange method.
+func mapRange(v reflect.Value) *mapIter {
+	return &mapIter{v: v}
+}
+
+func (i *mapIter) Next() bool {
+	if i.keys == nil {
+		i.keys = i.v.MapKeys()
+	} else {
+		i.keys = i.keys[1:]
+	}
+	return len(i.keys) > 0
+}
+
+func (i *mapIter) Key() reflect.Value {
+	return i.keys[0]
+}
+
+func (i *mapIter) Value() reflect.Value {
+	return i.v.MapIndex(i.keys[0])
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_map_go112.go b/vendor/google.golang.org/protobuf/internal/impl/codec_map_go112.go
new file mode 100644
index 0000000..1533ef6
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_map_go112.go
@@ -0,0 +1,11 @@
+// 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.
+
+// +build go1.12
+
+package impl
+
+import "reflect"
+
+func mapRange(v reflect.Value) *reflect.MapIter { return v.MapRange() }
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_message.go b/vendor/google.golang.org/protobuf/internal/impl/codec_message.go
new file mode 100644
index 0000000..cd40527
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_message.go
@@ -0,0 +1,217 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+	"reflect"
+	"sort"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/order"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// coderMessageInfo contains per-message information used by the fast-path functions.
+// This is a different type from MessageInfo to keep MessageInfo as general-purpose as
+// possible.
+type coderMessageInfo struct {
+	methods piface.Methods
+
+	orderedCoderFields []*coderFieldInfo
+	denseCoderFields   []*coderFieldInfo
+	coderFields        map[protowire.Number]*coderFieldInfo
+	sizecacheOffset    offset
+	unknownOffset      offset
+	unknownPtrKind     bool
+	extensionOffset    offset
+	needsInitCheck     bool
+	isMessageSet       bool
+	numRequiredFields  uint8
+}
+
+type coderFieldInfo struct {
+	funcs      pointerCoderFuncs // fast-path per-field functions
+	mi         *MessageInfo      // field's message
+	ft         reflect.Type
+	validation validationInfo   // information used by message validation
+	num        pref.FieldNumber // field number
+	offset     offset           // struct field offset
+	wiretag    uint64           // field tag (number + wire type)
+	tagsize    int              // size of the varint-encoded tag
+	isPointer  bool             // true if IsNil may be called on the struct field
+	isRequired bool             // true if field is required
+}
+
+func (mi *MessageInfo) makeCoderMethods(t reflect.Type, si structInfo) {
+	mi.sizecacheOffset = invalidOffset
+	mi.unknownOffset = invalidOffset
+	mi.extensionOffset = invalidOffset
+
+	if si.sizecacheOffset.IsValid() && si.sizecacheType == sizecacheType {
+		mi.sizecacheOffset = si.sizecacheOffset
+	}
+	if si.unknownOffset.IsValid() && (si.unknownType == unknownFieldsAType || si.unknownType == unknownFieldsBType) {
+		mi.unknownOffset = si.unknownOffset
+		mi.unknownPtrKind = si.unknownType.Kind() == reflect.Ptr
+	}
+	if si.extensionOffset.IsValid() && si.extensionType == extensionFieldsType {
+		mi.extensionOffset = si.extensionOffset
+	}
+
+	mi.coderFields = make(map[protowire.Number]*coderFieldInfo)
+	fields := mi.Desc.Fields()
+	preallocFields := make([]coderFieldInfo, fields.Len())
+	for i := 0; i < fields.Len(); i++ {
+		fd := fields.Get(i)
+
+		fs := si.fieldsByNumber[fd.Number()]
+		isOneof := fd.ContainingOneof() != nil && !fd.ContainingOneof().IsSynthetic()
+		if isOneof {
+			fs = si.oneofsByName[fd.ContainingOneof().Name()]
+		}
+		ft := fs.Type
+		var wiretag uint64
+		if !fd.IsPacked() {
+			wiretag = protowire.EncodeTag(fd.Number(), wireTypes[fd.Kind()])
+		} else {
+			wiretag = protowire.EncodeTag(fd.Number(), protowire.BytesType)
+		}
+		var fieldOffset offset
+		var funcs pointerCoderFuncs
+		var childMessage *MessageInfo
+		switch {
+		case ft == nil:
+			// This never occurs for generated message types.
+			// It implies that a hand-crafted type has missing Go fields
+			// for specific protobuf message fields.
+			funcs = pointerCoderFuncs{
+				size: func(p pointer, f *coderFieldInfo, opts marshalOptions) int {
+					return 0
+				},
+				marshal: func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+					return nil, nil
+				},
+				unmarshal: func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error) {
+					panic("missing Go struct field for " + string(fd.FullName()))
+				},
+				isInit: func(p pointer, f *coderFieldInfo) error {
+					panic("missing Go struct field for " + string(fd.FullName()))
+				},
+				merge: func(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+					panic("missing Go struct field for " + string(fd.FullName()))
+				},
+			}
+		case isOneof:
+			fieldOffset = offsetOf(fs, mi.Exporter)
+		case fd.IsWeak():
+			fieldOffset = si.weakOffset
+			funcs = makeWeakMessageFieldCoder(fd)
+		default:
+			fieldOffset = offsetOf(fs, mi.Exporter)
+			childMessage, funcs = fieldCoder(fd, ft)
+		}
+		cf := &preallocFields[i]
+		*cf = coderFieldInfo{
+			num:        fd.Number(),
+			offset:     fieldOffset,
+			wiretag:    wiretag,
+			ft:         ft,
+			tagsize:    protowire.SizeVarint(wiretag),
+			funcs:      funcs,
+			mi:         childMessage,
+			validation: newFieldValidationInfo(mi, si, fd, ft),
+			isPointer:  fd.Cardinality() == pref.Repeated || fd.HasPresence(),
+			isRequired: fd.Cardinality() == pref.Required,
+		}
+		mi.orderedCoderFields = append(mi.orderedCoderFields, cf)
+		mi.coderFields[cf.num] = cf
+	}
+	for i, oneofs := 0, mi.Desc.Oneofs(); i < oneofs.Len(); i++ {
+		if od := oneofs.Get(i); !od.IsSynthetic() {
+			mi.initOneofFieldCoders(od, si)
+		}
+	}
+	if messageset.IsMessageSet(mi.Desc) {
+		if !mi.extensionOffset.IsValid() {
+			panic(fmt.Sprintf("%v: MessageSet with no extensions field", mi.Desc.FullName()))
+		}
+		if !mi.unknownOffset.IsValid() {
+			panic(fmt.Sprintf("%v: MessageSet with no unknown field", mi.Desc.FullName()))
+		}
+		mi.isMessageSet = true
+	}
+	sort.Slice(mi.orderedCoderFields, func(i, j int) bool {
+		return mi.orderedCoderFields[i].num < mi.orderedCoderFields[j].num
+	})
+
+	var maxDense pref.FieldNumber
+	for _, cf := range mi.orderedCoderFields {
+		if cf.num >= 16 && cf.num >= 2*maxDense {
+			break
+		}
+		maxDense = cf.num
+	}
+	mi.denseCoderFields = make([]*coderFieldInfo, maxDense+1)
+	for _, cf := range mi.orderedCoderFields {
+		if int(cf.num) >= len(mi.denseCoderFields) {
+			break
+		}
+		mi.denseCoderFields[cf.num] = cf
+	}
+
+	// To preserve compatibility with historic wire output, marshal oneofs last.
+	if mi.Desc.Oneofs().Len() > 0 {
+		sort.Slice(mi.orderedCoderFields, func(i, j int) bool {
+			fi := fields.ByNumber(mi.orderedCoderFields[i].num)
+			fj := fields.ByNumber(mi.orderedCoderFields[j].num)
+			return order.LegacyFieldOrder(fi, fj)
+		})
+	}
+
+	mi.needsInitCheck = needsInitCheck(mi.Desc)
+	if mi.methods.Marshal == nil && mi.methods.Size == nil {
+		mi.methods.Flags |= piface.SupportMarshalDeterministic
+		mi.methods.Marshal = mi.marshal
+		mi.methods.Size = mi.size
+	}
+	if mi.methods.Unmarshal == nil {
+		mi.methods.Flags |= piface.SupportUnmarshalDiscardUnknown
+		mi.methods.Unmarshal = mi.unmarshal
+	}
+	if mi.methods.CheckInitialized == nil {
+		mi.methods.CheckInitialized = mi.checkInitialized
+	}
+	if mi.methods.Merge == nil {
+		mi.methods.Merge = mi.merge
+	}
+}
+
+// getUnknownBytes returns a *[]byte for the unknown fields.
+// It is the caller's responsibility to check whether the pointer is nil.
+// This function is specially designed to be inlineable.
+func (mi *MessageInfo) getUnknownBytes(p pointer) *[]byte {
+	if mi.unknownPtrKind {
+		return *p.Apply(mi.unknownOffset).BytesPtr()
+	} else {
+		return p.Apply(mi.unknownOffset).Bytes()
+	}
+}
+
+// mutableUnknownBytes returns a *[]byte for the unknown fields.
+// The returned pointer is guaranteed to not be nil.
+func (mi *MessageInfo) mutableUnknownBytes(p pointer) *[]byte {
+	if mi.unknownPtrKind {
+		bp := p.Apply(mi.unknownOffset).BytesPtr()
+		if *bp == nil {
+			*bp = new([]byte)
+		}
+		return *bp
+	} else {
+		return p.Apply(mi.unknownOffset).Bytes()
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_messageset.go b/vendor/google.golang.org/protobuf/internal/impl/codec_messageset.go
new file mode 100644
index 0000000..b7a23fa
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_messageset.go
@@ -0,0 +1,123 @@
+// 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.
+
+package impl
+
+import (
+	"sort"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+)
+
+func sizeMessageSet(mi *MessageInfo, p pointer, opts marshalOptions) (size int) {
+	if !flags.ProtoLegacy {
+		return 0
+	}
+
+	ext := *p.Apply(mi.extensionOffset).Extensions()
+	for _, x := range ext {
+		xi := getExtensionFieldInfo(x.Type())
+		if xi.funcs.size == nil {
+			continue
+		}
+		num, _ := protowire.DecodeTag(xi.wiretag)
+		size += messageset.SizeField(num)
+		size += xi.funcs.size(x.Value(), protowire.SizeTag(messageset.FieldMessage), opts)
+	}
+
+	if u := mi.getUnknownBytes(p); u != nil {
+		size += messageset.SizeUnknown(*u)
+	}
+
+	return size
+}
+
+func marshalMessageSet(mi *MessageInfo, b []byte, p pointer, opts marshalOptions) ([]byte, error) {
+	if !flags.ProtoLegacy {
+		return b, errors.New("no support for message_set_wire_format")
+	}
+
+	ext := *p.Apply(mi.extensionOffset).Extensions()
+	switch len(ext) {
+	case 0:
+	case 1:
+		// Fast-path for one extension: Don't bother sorting the keys.
+		for _, x := range ext {
+			var err error
+			b, err = marshalMessageSetField(mi, b, x, opts)
+			if err != nil {
+				return b, err
+			}
+		}
+	default:
+		// Sort the keys to provide a deterministic encoding.
+		// Not sure this is required, but the old code does it.
+		keys := make([]int, 0, len(ext))
+		for k := range ext {
+			keys = append(keys, int(k))
+		}
+		sort.Ints(keys)
+		for _, k := range keys {
+			var err error
+			b, err = marshalMessageSetField(mi, b, ext[int32(k)], opts)
+			if err != nil {
+				return b, err
+			}
+		}
+	}
+
+	if u := mi.getUnknownBytes(p); u != nil {
+		var err error
+		b, err = messageset.AppendUnknown(b, *u)
+		if err != nil {
+			return b, err
+		}
+	}
+
+	return b, nil
+}
+
+func marshalMessageSetField(mi *MessageInfo, b []byte, x ExtensionField, opts marshalOptions) ([]byte, error) {
+	xi := getExtensionFieldInfo(x.Type())
+	num, _ := protowire.DecodeTag(xi.wiretag)
+	b = messageset.AppendFieldStart(b, num)
+	b, err := xi.funcs.marshal(b, x.Value(), protowire.EncodeTag(messageset.FieldMessage, protowire.BytesType), opts)
+	if err != nil {
+		return b, err
+	}
+	b = messageset.AppendFieldEnd(b)
+	return b, nil
+}
+
+func unmarshalMessageSet(mi *MessageInfo, b []byte, p pointer, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if !flags.ProtoLegacy {
+		return out, errors.New("no support for message_set_wire_format")
+	}
+
+	ep := p.Apply(mi.extensionOffset).Extensions()
+	if *ep == nil {
+		*ep = make(map[int32]ExtensionField)
+	}
+	ext := *ep
+	initialized := true
+	err = messageset.Unmarshal(b, true, func(num protowire.Number, v []byte) error {
+		o, err := mi.unmarshalExtension(v, num, protowire.BytesType, ext, opts)
+		if err == errUnknown {
+			u := mi.mutableUnknownBytes(p)
+			*u = protowire.AppendTag(*u, num, protowire.BytesType)
+			*u = append(*u, v...)
+			return nil
+		}
+		if !o.initialized {
+			initialized = false
+		}
+		return err
+	})
+	out.n = len(b)
+	out.initialized = initialized
+	return out, err
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_reflect.go b/vendor/google.golang.org/protobuf/internal/impl/codec_reflect.go
new file mode 100644
index 0000000..90705e3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_reflect.go
@@ -0,0 +1,209 @@
+// 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.
+
+// +build purego appengine
+
+package impl
+
+import (
+	"reflect"
+
+	"google.golang.org/protobuf/encoding/protowire"
+)
+
+func sizeEnum(p pointer, f *coderFieldInfo, _ marshalOptions) (size int) {
+	v := p.v.Elem().Int()
+	return f.tagsize + protowire.SizeVarint(uint64(v))
+}
+
+func appendEnum(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	v := p.v.Elem().Int()
+	b = protowire.AppendVarint(b, f.wiretag)
+	b = protowire.AppendVarint(b, uint64(v))
+	return b, nil
+}
+
+func consumeEnum(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, _ unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeVarint(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	p.v.Elem().SetInt(int64(v))
+	out.n = n
+	return out, nil
+}
+
+func mergeEnum(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	dst.v.Elem().Set(src.v.Elem())
+}
+
+var coderEnum = pointerCoderFuncs{
+	size:      sizeEnum,
+	marshal:   appendEnum,
+	unmarshal: consumeEnum,
+	merge:     mergeEnum,
+}
+
+func sizeEnumNoZero(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	if p.v.Elem().Int() == 0 {
+		return 0
+	}
+	return sizeEnum(p, f, opts)
+}
+
+func appendEnumNoZero(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	if p.v.Elem().Int() == 0 {
+		return b, nil
+	}
+	return appendEnum(b, p, f, opts)
+}
+
+func mergeEnumNoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	if src.v.Elem().Int() != 0 {
+		dst.v.Elem().Set(src.v.Elem())
+	}
+}
+
+var coderEnumNoZero = pointerCoderFuncs{
+	size:      sizeEnumNoZero,
+	marshal:   appendEnumNoZero,
+	unmarshal: consumeEnum,
+	merge:     mergeEnumNoZero,
+}
+
+func sizeEnumPtr(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	return sizeEnum(pointer{p.v.Elem()}, f, opts)
+}
+
+func appendEnumPtr(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	return appendEnum(b, pointer{p.v.Elem()}, f, opts)
+}
+
+func consumeEnumPtr(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	if p.v.Elem().IsNil() {
+		p.v.Elem().Set(reflect.New(p.v.Elem().Type().Elem()))
+	}
+	return consumeEnum(b, pointer{p.v.Elem()}, wtyp, f, opts)
+}
+
+func mergeEnumPtr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	if !src.v.Elem().IsNil() {
+		v := reflect.New(dst.v.Type().Elem().Elem())
+		v.Elem().Set(src.v.Elem().Elem())
+		dst.v.Elem().Set(v)
+	}
+}
+
+var coderEnumPtr = pointerCoderFuncs{
+	size:      sizeEnumPtr,
+	marshal:   appendEnumPtr,
+	unmarshal: consumeEnumPtr,
+	merge:     mergeEnumPtr,
+}
+
+func sizeEnumSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := p.v.Elem()
+	for i, llen := 0, s.Len(); i < llen; i++ {
+		size += protowire.SizeVarint(uint64(s.Index(i).Int())) + f.tagsize
+	}
+	return size
+}
+
+func appendEnumSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := p.v.Elem()
+	for i, llen := 0, s.Len(); i < llen; i++ {
+		b = protowire.AppendVarint(b, f.wiretag)
+		b = protowire.AppendVarint(b, uint64(s.Index(i).Int()))
+	}
+	return b, nil
+}
+
+func consumeEnumSlice(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	s := p.v.Elem()
+	if wtyp == protowire.BytesType {
+		b, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, errDecode
+		}
+		for len(b) > 0 {
+			v, n := protowire.ConsumeVarint(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			rv := reflect.New(s.Type().Elem()).Elem()
+			rv.SetInt(int64(v))
+			s.Set(reflect.Append(s, rv))
+			b = b[n:]
+		}
+		out.n = n
+		return out, nil
+	}
+	if wtyp != protowire.VarintType {
+		return out, errUnknown
+	}
+	v, n := protowire.ConsumeVarint(b)
+	if n < 0 {
+		return out, errDecode
+	}
+	rv := reflect.New(s.Type().Elem()).Elem()
+	rv.SetInt(int64(v))
+	s.Set(reflect.Append(s, rv))
+	out.n = n
+	return out, nil
+}
+
+func mergeEnumSlice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	dst.v.Elem().Set(reflect.AppendSlice(dst.v.Elem(), src.v.Elem()))
+}
+
+var coderEnumSlice = pointerCoderFuncs{
+	size:      sizeEnumSlice,
+	marshal:   appendEnumSlice,
+	unmarshal: consumeEnumSlice,
+	merge:     mergeEnumSlice,
+}
+
+func sizeEnumPackedSlice(p pointer, f *coderFieldInfo, opts marshalOptions) (size int) {
+	s := p.v.Elem()
+	llen := s.Len()
+	if llen == 0 {
+		return 0
+	}
+	n := 0
+	for i := 0; i < llen; i++ {
+		n += protowire.SizeVarint(uint64(s.Index(i).Int()))
+	}
+	return f.tagsize + protowire.SizeBytes(n)
+}
+
+func appendEnumPackedSlice(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error) {
+	s := p.v.Elem()
+	llen := s.Len()
+	if llen == 0 {
+		return b, nil
+	}
+	b = protowire.AppendVarint(b, f.wiretag)
+	n := 0
+	for i := 0; i < llen; i++ {
+		n += protowire.SizeVarint(uint64(s.Index(i).Int()))
+	}
+	b = protowire.AppendVarint(b, uint64(n))
+	for i := 0; i < llen; i++ {
+		b = protowire.AppendVarint(b, uint64(s.Index(i).Int()))
+	}
+	return b, nil
+}
+
+var coderEnumPackedSlice = pointerCoderFuncs{
+	size:      sizeEnumPackedSlice,
+	marshal:   appendEnumPackedSlice,
+	unmarshal: consumeEnumSlice,
+	merge:     mergeEnumSlice,
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_tables.go b/vendor/google.golang.org/protobuf/internal/impl/codec_tables.go
new file mode 100644
index 0000000..e899712
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_tables.go
@@ -0,0 +1,557 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+	"reflect"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/strs"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// pointerCoderFuncs is a set of pointer encoding functions.
+type pointerCoderFuncs struct {
+	mi        *MessageInfo
+	size      func(p pointer, f *coderFieldInfo, opts marshalOptions) int
+	marshal   func(b []byte, p pointer, f *coderFieldInfo, opts marshalOptions) ([]byte, error)
+	unmarshal func(b []byte, p pointer, wtyp protowire.Type, f *coderFieldInfo, opts unmarshalOptions) (unmarshalOutput, error)
+	isInit    func(p pointer, f *coderFieldInfo) error
+	merge     func(dst, src pointer, f *coderFieldInfo, opts mergeOptions)
+}
+
+// valueCoderFuncs is a set of protoreflect.Value encoding functions.
+type valueCoderFuncs struct {
+	size      func(v pref.Value, tagsize int, opts marshalOptions) int
+	marshal   func(b []byte, v pref.Value, wiretag uint64, opts marshalOptions) ([]byte, error)
+	unmarshal func(b []byte, v pref.Value, num protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (pref.Value, unmarshalOutput, error)
+	isInit    func(v pref.Value) error
+	merge     func(dst, src pref.Value, opts mergeOptions) pref.Value
+}
+
+// fieldCoder returns pointer functions for a field, used for operating on
+// struct fields.
+func fieldCoder(fd pref.FieldDescriptor, ft reflect.Type) (*MessageInfo, pointerCoderFuncs) {
+	switch {
+	case fd.IsMap():
+		return encoderFuncsForMap(fd, ft)
+	case fd.Cardinality() == pref.Repeated && !fd.IsPacked():
+		// Repeated fields (not packed).
+		if ft.Kind() != reflect.Slice {
+			break
+		}
+		ft := ft.Elem()
+		switch fd.Kind() {
+		case pref.BoolKind:
+			if ft.Kind() == reflect.Bool {
+				return nil, coderBoolSlice
+			}
+		case pref.EnumKind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderEnumSlice
+			}
+		case pref.Int32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderInt32Slice
+			}
+		case pref.Sint32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSint32Slice
+			}
+		case pref.Uint32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderUint32Slice
+			}
+		case pref.Int64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderInt64Slice
+			}
+		case pref.Sint64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSint64Slice
+			}
+		case pref.Uint64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderUint64Slice
+			}
+		case pref.Sfixed32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSfixed32Slice
+			}
+		case pref.Fixed32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderFixed32Slice
+			}
+		case pref.FloatKind:
+			if ft.Kind() == reflect.Float32 {
+				return nil, coderFloatSlice
+			}
+		case pref.Sfixed64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSfixed64Slice
+			}
+		case pref.Fixed64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderFixed64Slice
+			}
+		case pref.DoubleKind:
+			if ft.Kind() == reflect.Float64 {
+				return nil, coderDoubleSlice
+			}
+		case pref.StringKind:
+			if ft.Kind() == reflect.String && strs.EnforceUTF8(fd) {
+				return nil, coderStringSliceValidateUTF8
+			}
+			if ft.Kind() == reflect.String {
+				return nil, coderStringSlice
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 && strs.EnforceUTF8(fd) {
+				return nil, coderBytesSliceValidateUTF8
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytesSlice
+			}
+		case pref.BytesKind:
+			if ft.Kind() == reflect.String {
+				return nil, coderStringSlice
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytesSlice
+			}
+		case pref.MessageKind:
+			return getMessageInfo(ft), makeMessageSliceFieldCoder(fd, ft)
+		case pref.GroupKind:
+			return getMessageInfo(ft), makeGroupSliceFieldCoder(fd, ft)
+		}
+	case fd.Cardinality() == pref.Repeated && fd.IsPacked():
+		// Packed repeated fields.
+		//
+		// Only repeated fields of primitive numeric types
+		// (Varint, Fixed32, or Fixed64 wire type) can be packed.
+		if ft.Kind() != reflect.Slice {
+			break
+		}
+		ft := ft.Elem()
+		switch fd.Kind() {
+		case pref.BoolKind:
+			if ft.Kind() == reflect.Bool {
+				return nil, coderBoolPackedSlice
+			}
+		case pref.EnumKind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderEnumPackedSlice
+			}
+		case pref.Int32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderInt32PackedSlice
+			}
+		case pref.Sint32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSint32PackedSlice
+			}
+		case pref.Uint32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderUint32PackedSlice
+			}
+		case pref.Int64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderInt64PackedSlice
+			}
+		case pref.Sint64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSint64PackedSlice
+			}
+		case pref.Uint64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderUint64PackedSlice
+			}
+		case pref.Sfixed32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSfixed32PackedSlice
+			}
+		case pref.Fixed32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderFixed32PackedSlice
+			}
+		case pref.FloatKind:
+			if ft.Kind() == reflect.Float32 {
+				return nil, coderFloatPackedSlice
+			}
+		case pref.Sfixed64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSfixed64PackedSlice
+			}
+		case pref.Fixed64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderFixed64PackedSlice
+			}
+		case pref.DoubleKind:
+			if ft.Kind() == reflect.Float64 {
+				return nil, coderDoublePackedSlice
+			}
+		}
+	case fd.Kind() == pref.MessageKind:
+		return getMessageInfo(ft), makeMessageFieldCoder(fd, ft)
+	case fd.Kind() == pref.GroupKind:
+		return getMessageInfo(ft), makeGroupFieldCoder(fd, ft)
+	case fd.Syntax() == pref.Proto3 && fd.ContainingOneof() == nil:
+		// Populated oneof fields always encode even if set to the zero value,
+		// which normally are not encoded in proto3.
+		switch fd.Kind() {
+		case pref.BoolKind:
+			if ft.Kind() == reflect.Bool {
+				return nil, coderBoolNoZero
+			}
+		case pref.EnumKind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderEnumNoZero
+			}
+		case pref.Int32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderInt32NoZero
+			}
+		case pref.Sint32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSint32NoZero
+			}
+		case pref.Uint32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderUint32NoZero
+			}
+		case pref.Int64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderInt64NoZero
+			}
+		case pref.Sint64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSint64NoZero
+			}
+		case pref.Uint64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderUint64NoZero
+			}
+		case pref.Sfixed32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSfixed32NoZero
+			}
+		case pref.Fixed32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderFixed32NoZero
+			}
+		case pref.FloatKind:
+			if ft.Kind() == reflect.Float32 {
+				return nil, coderFloatNoZero
+			}
+		case pref.Sfixed64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSfixed64NoZero
+			}
+		case pref.Fixed64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderFixed64NoZero
+			}
+		case pref.DoubleKind:
+			if ft.Kind() == reflect.Float64 {
+				return nil, coderDoubleNoZero
+			}
+		case pref.StringKind:
+			if ft.Kind() == reflect.String && strs.EnforceUTF8(fd) {
+				return nil, coderStringNoZeroValidateUTF8
+			}
+			if ft.Kind() == reflect.String {
+				return nil, coderStringNoZero
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 && strs.EnforceUTF8(fd) {
+				return nil, coderBytesNoZeroValidateUTF8
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytesNoZero
+			}
+		case pref.BytesKind:
+			if ft.Kind() == reflect.String {
+				return nil, coderStringNoZero
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytesNoZero
+			}
+		}
+	case ft.Kind() == reflect.Ptr:
+		ft := ft.Elem()
+		switch fd.Kind() {
+		case pref.BoolKind:
+			if ft.Kind() == reflect.Bool {
+				return nil, coderBoolPtr
+			}
+		case pref.EnumKind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderEnumPtr
+			}
+		case pref.Int32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderInt32Ptr
+			}
+		case pref.Sint32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSint32Ptr
+			}
+		case pref.Uint32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderUint32Ptr
+			}
+		case pref.Int64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderInt64Ptr
+			}
+		case pref.Sint64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSint64Ptr
+			}
+		case pref.Uint64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderUint64Ptr
+			}
+		case pref.Sfixed32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSfixed32Ptr
+			}
+		case pref.Fixed32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderFixed32Ptr
+			}
+		case pref.FloatKind:
+			if ft.Kind() == reflect.Float32 {
+				return nil, coderFloatPtr
+			}
+		case pref.Sfixed64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSfixed64Ptr
+			}
+		case pref.Fixed64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderFixed64Ptr
+			}
+		case pref.DoubleKind:
+			if ft.Kind() == reflect.Float64 {
+				return nil, coderDoublePtr
+			}
+		case pref.StringKind:
+			if ft.Kind() == reflect.String && strs.EnforceUTF8(fd) {
+				return nil, coderStringPtrValidateUTF8
+			}
+			if ft.Kind() == reflect.String {
+				return nil, coderStringPtr
+			}
+		case pref.BytesKind:
+			if ft.Kind() == reflect.String {
+				return nil, coderStringPtr
+			}
+		}
+	default:
+		switch fd.Kind() {
+		case pref.BoolKind:
+			if ft.Kind() == reflect.Bool {
+				return nil, coderBool
+			}
+		case pref.EnumKind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderEnum
+			}
+		case pref.Int32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderInt32
+			}
+		case pref.Sint32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSint32
+			}
+		case pref.Uint32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderUint32
+			}
+		case pref.Int64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderInt64
+			}
+		case pref.Sint64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSint64
+			}
+		case pref.Uint64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderUint64
+			}
+		case pref.Sfixed32Kind:
+			if ft.Kind() == reflect.Int32 {
+				return nil, coderSfixed32
+			}
+		case pref.Fixed32Kind:
+			if ft.Kind() == reflect.Uint32 {
+				return nil, coderFixed32
+			}
+		case pref.FloatKind:
+			if ft.Kind() == reflect.Float32 {
+				return nil, coderFloat
+			}
+		case pref.Sfixed64Kind:
+			if ft.Kind() == reflect.Int64 {
+				return nil, coderSfixed64
+			}
+		case pref.Fixed64Kind:
+			if ft.Kind() == reflect.Uint64 {
+				return nil, coderFixed64
+			}
+		case pref.DoubleKind:
+			if ft.Kind() == reflect.Float64 {
+				return nil, coderDouble
+			}
+		case pref.StringKind:
+			if ft.Kind() == reflect.String && strs.EnforceUTF8(fd) {
+				return nil, coderStringValidateUTF8
+			}
+			if ft.Kind() == reflect.String {
+				return nil, coderString
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 && strs.EnforceUTF8(fd) {
+				return nil, coderBytesValidateUTF8
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytes
+			}
+		case pref.BytesKind:
+			if ft.Kind() == reflect.String {
+				return nil, coderString
+			}
+			if ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8 {
+				return nil, coderBytes
+			}
+		}
+	}
+	panic(fmt.Sprintf("invalid type: no encoder for %v %v %v/%v", fd.FullName(), fd.Cardinality(), fd.Kind(), ft))
+}
+
+// encoderFuncsForValue returns value functions for a field, used for
+// extension values and map encoding.
+func encoderFuncsForValue(fd pref.FieldDescriptor) valueCoderFuncs {
+	switch {
+	case fd.Cardinality() == pref.Repeated && !fd.IsPacked():
+		switch fd.Kind() {
+		case pref.BoolKind:
+			return coderBoolSliceValue
+		case pref.EnumKind:
+			return coderEnumSliceValue
+		case pref.Int32Kind:
+			return coderInt32SliceValue
+		case pref.Sint32Kind:
+			return coderSint32SliceValue
+		case pref.Uint32Kind:
+			return coderUint32SliceValue
+		case pref.Int64Kind:
+			return coderInt64SliceValue
+		case pref.Sint64Kind:
+			return coderSint64SliceValue
+		case pref.Uint64Kind:
+			return coderUint64SliceValue
+		case pref.Sfixed32Kind:
+			return coderSfixed32SliceValue
+		case pref.Fixed32Kind:
+			return coderFixed32SliceValue
+		case pref.FloatKind:
+			return coderFloatSliceValue
+		case pref.Sfixed64Kind:
+			return coderSfixed64SliceValue
+		case pref.Fixed64Kind:
+			return coderFixed64SliceValue
+		case pref.DoubleKind:
+			return coderDoubleSliceValue
+		case pref.StringKind:
+			// We don't have a UTF-8 validating coder for repeated string fields.
+			// Value coders are used for extensions and maps.
+			// Extensions are never proto3, and maps never contain lists.
+			return coderStringSliceValue
+		case pref.BytesKind:
+			return coderBytesSliceValue
+		case pref.MessageKind:
+			return coderMessageSliceValue
+		case pref.GroupKind:
+			return coderGroupSliceValue
+		}
+	case fd.Cardinality() == pref.Repeated && fd.IsPacked():
+		switch fd.Kind() {
+		case pref.BoolKind:
+			return coderBoolPackedSliceValue
+		case pref.EnumKind:
+			return coderEnumPackedSliceValue
+		case pref.Int32Kind:
+			return coderInt32PackedSliceValue
+		case pref.Sint32Kind:
+			return coderSint32PackedSliceValue
+		case pref.Uint32Kind:
+			return coderUint32PackedSliceValue
+		case pref.Int64Kind:
+			return coderInt64PackedSliceValue
+		case pref.Sint64Kind:
+			return coderSint64PackedSliceValue
+		case pref.Uint64Kind:
+			return coderUint64PackedSliceValue
+		case pref.Sfixed32Kind:
+			return coderSfixed32PackedSliceValue
+		case pref.Fixed32Kind:
+			return coderFixed32PackedSliceValue
+		case pref.FloatKind:
+			return coderFloatPackedSliceValue
+		case pref.Sfixed64Kind:
+			return coderSfixed64PackedSliceValue
+		case pref.Fixed64Kind:
+			return coderFixed64PackedSliceValue
+		case pref.DoubleKind:
+			return coderDoublePackedSliceValue
+		}
+	default:
+		switch fd.Kind() {
+		default:
+		case pref.BoolKind:
+			return coderBoolValue
+		case pref.EnumKind:
+			return coderEnumValue
+		case pref.Int32Kind:
+			return coderInt32Value
+		case pref.Sint32Kind:
+			return coderSint32Value
+		case pref.Uint32Kind:
+			return coderUint32Value
+		case pref.Int64Kind:
+			return coderInt64Value
+		case pref.Sint64Kind:
+			return coderSint64Value
+		case pref.Uint64Kind:
+			return coderUint64Value
+		case pref.Sfixed32Kind:
+			return coderSfixed32Value
+		case pref.Fixed32Kind:
+			return coderFixed32Value
+		case pref.FloatKind:
+			return coderFloatValue
+		case pref.Sfixed64Kind:
+			return coderSfixed64Value
+		case pref.Fixed64Kind:
+			return coderFixed64Value
+		case pref.DoubleKind:
+			return coderDoubleValue
+		case pref.StringKind:
+			if strs.EnforceUTF8(fd) {
+				return coderStringValueValidateUTF8
+			}
+			return coderStringValue
+		case pref.BytesKind:
+			return coderBytesValue
+		case pref.MessageKind:
+			return coderMessageValue
+		case pref.GroupKind:
+			return coderGroupValue
+		}
+	}
+	panic(fmt.Sprintf("invalid field: no encoder for %v %v %v", fd.FullName(), fd.Cardinality(), fd.Kind()))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/codec_unsafe.go b/vendor/google.golang.org/protobuf/internal/impl/codec_unsafe.go
new file mode 100644
index 0000000..e118af1
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/codec_unsafe.go
@@ -0,0 +1,17 @@
+// 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.
+
+// +build !purego,!appengine
+
+package impl
+
+// When using unsafe pointers, we can just treat enum values as int32s.
+
+var (
+	coderEnumNoZero      = coderInt32NoZero
+	coderEnum            = coderInt32
+	coderEnumPtr         = coderInt32Ptr
+	coderEnumSlice       = coderInt32Slice
+	coderEnumPackedSlice = coderInt32PackedSlice
+)
diff --git a/vendor/google.golang.org/protobuf/internal/impl/convert.go b/vendor/google.golang.org/protobuf/internal/impl/convert.go
new file mode 100644
index 0000000..acd61bb
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/convert.go
@@ -0,0 +1,496 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// unwrapper unwraps the value to the underlying value.
+// This is implemented by List and Map.
+type unwrapper interface {
+	protoUnwrap() interface{}
+}
+
+// A Converter coverts to/from Go reflect.Value types and protobuf protoreflect.Value types.
+type Converter interface {
+	// PBValueOf converts a reflect.Value to a protoreflect.Value.
+	PBValueOf(reflect.Value) pref.Value
+
+	// GoValueOf converts a protoreflect.Value to a reflect.Value.
+	GoValueOf(pref.Value) reflect.Value
+
+	// IsValidPB returns whether a protoreflect.Value is compatible with this type.
+	IsValidPB(pref.Value) bool
+
+	// IsValidGo returns whether a reflect.Value is compatible with this type.
+	IsValidGo(reflect.Value) bool
+
+	// New returns a new field value.
+	// For scalars, it returns the default value of the field.
+	// For composite types, it returns a new mutable value.
+	New() pref.Value
+
+	// Zero returns a new field value.
+	// For scalars, it returns the default value of the field.
+	// For composite types, it returns an immutable, empty value.
+	Zero() pref.Value
+}
+
+// NewConverter matches a Go type with a protobuf field and returns a Converter
+// that converts between the two. Enums must be a named int32 kind that
+// implements protoreflect.Enum, and messages must be pointer to a named
+// struct type that implements protoreflect.ProtoMessage.
+//
+// This matcher deliberately supports a wider range of Go types than what
+// protoc-gen-go historically generated to be able to automatically wrap some
+// v1 messages generated by other forks of protoc-gen-go.
+func NewConverter(t reflect.Type, fd pref.FieldDescriptor) Converter {
+	switch {
+	case fd.IsList():
+		return newListConverter(t, fd)
+	case fd.IsMap():
+		return newMapConverter(t, fd)
+	default:
+		return newSingularConverter(t, fd)
+	}
+	panic(fmt.Sprintf("invalid Go type %v for field %v", t, fd.FullName()))
+}
+
+var (
+	boolType    = reflect.TypeOf(bool(false))
+	int32Type   = reflect.TypeOf(int32(0))
+	int64Type   = reflect.TypeOf(int64(0))
+	uint32Type  = reflect.TypeOf(uint32(0))
+	uint64Type  = reflect.TypeOf(uint64(0))
+	float32Type = reflect.TypeOf(float32(0))
+	float64Type = reflect.TypeOf(float64(0))
+	stringType  = reflect.TypeOf(string(""))
+	bytesType   = reflect.TypeOf([]byte(nil))
+	byteType    = reflect.TypeOf(byte(0))
+)
+
+var (
+	boolZero    = pref.ValueOfBool(false)
+	int32Zero   = pref.ValueOfInt32(0)
+	int64Zero   = pref.ValueOfInt64(0)
+	uint32Zero  = pref.ValueOfUint32(0)
+	uint64Zero  = pref.ValueOfUint64(0)
+	float32Zero = pref.ValueOfFloat32(0)
+	float64Zero = pref.ValueOfFloat64(0)
+	stringZero  = pref.ValueOfString("")
+	bytesZero   = pref.ValueOfBytes(nil)
+)
+
+func newSingularConverter(t reflect.Type, fd pref.FieldDescriptor) Converter {
+	defVal := func(fd pref.FieldDescriptor, zero pref.Value) pref.Value {
+		if fd.Cardinality() == pref.Repeated {
+			// Default isn't defined for repeated fields.
+			return zero
+		}
+		return fd.Default()
+	}
+	switch fd.Kind() {
+	case pref.BoolKind:
+		if t.Kind() == reflect.Bool {
+			return &boolConverter{t, defVal(fd, boolZero)}
+		}
+	case pref.Int32Kind, pref.Sint32Kind, pref.Sfixed32Kind:
+		if t.Kind() == reflect.Int32 {
+			return &int32Converter{t, defVal(fd, int32Zero)}
+		}
+	case pref.Int64Kind, pref.Sint64Kind, pref.Sfixed64Kind:
+		if t.Kind() == reflect.Int64 {
+			return &int64Converter{t, defVal(fd, int64Zero)}
+		}
+	case pref.Uint32Kind, pref.Fixed32Kind:
+		if t.Kind() == reflect.Uint32 {
+			return &uint32Converter{t, defVal(fd, uint32Zero)}
+		}
+	case pref.Uint64Kind, pref.Fixed64Kind:
+		if t.Kind() == reflect.Uint64 {
+			return &uint64Converter{t, defVal(fd, uint64Zero)}
+		}
+	case pref.FloatKind:
+		if t.Kind() == reflect.Float32 {
+			return &float32Converter{t, defVal(fd, float32Zero)}
+		}
+	case pref.DoubleKind:
+		if t.Kind() == reflect.Float64 {
+			return &float64Converter{t, defVal(fd, float64Zero)}
+		}
+	case pref.StringKind:
+		if t.Kind() == reflect.String || (t.Kind() == reflect.Slice && t.Elem() == byteType) {
+			return &stringConverter{t, defVal(fd, stringZero)}
+		}
+	case pref.BytesKind:
+		if t.Kind() == reflect.String || (t.Kind() == reflect.Slice && t.Elem() == byteType) {
+			return &bytesConverter{t, defVal(fd, bytesZero)}
+		}
+	case pref.EnumKind:
+		// Handle enums, which must be a named int32 type.
+		if t.Kind() == reflect.Int32 {
+			return newEnumConverter(t, fd)
+		}
+	case pref.MessageKind, pref.GroupKind:
+		return newMessageConverter(t)
+	}
+	panic(fmt.Sprintf("invalid Go type %v for field %v", t, fd.FullName()))
+}
+
+type boolConverter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *boolConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfBool(v.Bool())
+}
+func (c *boolConverter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(v.Bool()).Convert(c.goType)
+}
+func (c *boolConverter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(bool)
+	return ok
+}
+func (c *boolConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *boolConverter) New() pref.Value  { return c.def }
+func (c *boolConverter) Zero() pref.Value { return c.def }
+
+type int32Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *int32Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfInt32(int32(v.Int()))
+}
+func (c *int32Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(int32(v.Int())).Convert(c.goType)
+}
+func (c *int32Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(int32)
+	return ok
+}
+func (c *int32Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *int32Converter) New() pref.Value  { return c.def }
+func (c *int32Converter) Zero() pref.Value { return c.def }
+
+type int64Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *int64Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfInt64(int64(v.Int()))
+}
+func (c *int64Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(int64(v.Int())).Convert(c.goType)
+}
+func (c *int64Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(int64)
+	return ok
+}
+func (c *int64Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *int64Converter) New() pref.Value  { return c.def }
+func (c *int64Converter) Zero() pref.Value { return c.def }
+
+type uint32Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *uint32Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfUint32(uint32(v.Uint()))
+}
+func (c *uint32Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(uint32(v.Uint())).Convert(c.goType)
+}
+func (c *uint32Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(uint32)
+	return ok
+}
+func (c *uint32Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *uint32Converter) New() pref.Value  { return c.def }
+func (c *uint32Converter) Zero() pref.Value { return c.def }
+
+type uint64Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *uint64Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfUint64(uint64(v.Uint()))
+}
+func (c *uint64Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(uint64(v.Uint())).Convert(c.goType)
+}
+func (c *uint64Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(uint64)
+	return ok
+}
+func (c *uint64Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *uint64Converter) New() pref.Value  { return c.def }
+func (c *uint64Converter) Zero() pref.Value { return c.def }
+
+type float32Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *float32Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfFloat32(float32(v.Float()))
+}
+func (c *float32Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(float32(v.Float())).Convert(c.goType)
+}
+func (c *float32Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(float32)
+	return ok
+}
+func (c *float32Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *float32Converter) New() pref.Value  { return c.def }
+func (c *float32Converter) Zero() pref.Value { return c.def }
+
+type float64Converter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *float64Converter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfFloat64(float64(v.Float()))
+}
+func (c *float64Converter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(float64(v.Float())).Convert(c.goType)
+}
+func (c *float64Converter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(float64)
+	return ok
+}
+func (c *float64Converter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *float64Converter) New() pref.Value  { return c.def }
+func (c *float64Converter) Zero() pref.Value { return c.def }
+
+type stringConverter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *stringConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfString(v.Convert(stringType).String())
+}
+func (c *stringConverter) GoValueOf(v pref.Value) reflect.Value {
+	// pref.Value.String never panics, so we go through an interface
+	// conversion here to check the type.
+	s := v.Interface().(string)
+	if c.goType.Kind() == reflect.Slice && s == "" {
+		return reflect.Zero(c.goType) // ensure empty string is []byte(nil)
+	}
+	return reflect.ValueOf(s).Convert(c.goType)
+}
+func (c *stringConverter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(string)
+	return ok
+}
+func (c *stringConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *stringConverter) New() pref.Value  { return c.def }
+func (c *stringConverter) Zero() pref.Value { return c.def }
+
+type bytesConverter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func (c *bytesConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	if c.goType.Kind() == reflect.String && v.Len() == 0 {
+		return pref.ValueOfBytes(nil) // ensure empty string is []byte(nil)
+	}
+	return pref.ValueOfBytes(v.Convert(bytesType).Bytes())
+}
+func (c *bytesConverter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(v.Bytes()).Convert(c.goType)
+}
+func (c *bytesConverter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().([]byte)
+	return ok
+}
+func (c *bytesConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+func (c *bytesConverter) New() pref.Value  { return c.def }
+func (c *bytesConverter) Zero() pref.Value { return c.def }
+
+type enumConverter struct {
+	goType reflect.Type
+	def    pref.Value
+}
+
+func newEnumConverter(goType reflect.Type, fd pref.FieldDescriptor) Converter {
+	var def pref.Value
+	if fd.Cardinality() == pref.Repeated {
+		def = pref.ValueOfEnum(fd.Enum().Values().Get(0).Number())
+	} else {
+		def = fd.Default()
+	}
+	return &enumConverter{goType, def}
+}
+
+func (c *enumConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfEnum(pref.EnumNumber(v.Int()))
+}
+
+func (c *enumConverter) GoValueOf(v pref.Value) reflect.Value {
+	return reflect.ValueOf(v.Enum()).Convert(c.goType)
+}
+
+func (c *enumConverter) IsValidPB(v pref.Value) bool {
+	_, ok := v.Interface().(pref.EnumNumber)
+	return ok
+}
+
+func (c *enumConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+
+func (c *enumConverter) New() pref.Value {
+	return c.def
+}
+
+func (c *enumConverter) Zero() pref.Value {
+	return c.def
+}
+
+type messageConverter struct {
+	goType reflect.Type
+}
+
+func newMessageConverter(goType reflect.Type) Converter {
+	return &messageConverter{goType}
+}
+
+func (c *messageConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	if c.isNonPointer() {
+		if v.CanAddr() {
+			v = v.Addr() // T => *T
+		} else {
+			v = reflect.Zero(reflect.PtrTo(v.Type()))
+		}
+	}
+	if m, ok := v.Interface().(pref.ProtoMessage); ok {
+		return pref.ValueOfMessage(m.ProtoReflect())
+	}
+	return pref.ValueOfMessage(legacyWrapMessage(v))
+}
+
+func (c *messageConverter) GoValueOf(v pref.Value) reflect.Value {
+	m := v.Message()
+	var rv reflect.Value
+	if u, ok := m.(unwrapper); ok {
+		rv = reflect.ValueOf(u.protoUnwrap())
+	} else {
+		rv = reflect.ValueOf(m.Interface())
+	}
+	if c.isNonPointer() {
+		if rv.Type() != reflect.PtrTo(c.goType) {
+			panic(fmt.Sprintf("invalid type: got %v, want %v", rv.Type(), reflect.PtrTo(c.goType)))
+		}
+		if !rv.IsNil() {
+			rv = rv.Elem() // *T => T
+		} else {
+			rv = reflect.Zero(rv.Type().Elem())
+		}
+	}
+	if rv.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", rv.Type(), c.goType))
+	}
+	return rv
+}
+
+func (c *messageConverter) IsValidPB(v pref.Value) bool {
+	m := v.Message()
+	var rv reflect.Value
+	if u, ok := m.(unwrapper); ok {
+		rv = reflect.ValueOf(u.protoUnwrap())
+	} else {
+		rv = reflect.ValueOf(m.Interface())
+	}
+	if c.isNonPointer() {
+		return rv.Type() == reflect.PtrTo(c.goType)
+	}
+	return rv.Type() == c.goType
+}
+
+func (c *messageConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+
+func (c *messageConverter) New() pref.Value {
+	if c.isNonPointer() {
+		return c.PBValueOf(reflect.New(c.goType).Elem())
+	}
+	return c.PBValueOf(reflect.New(c.goType.Elem()))
+}
+
+func (c *messageConverter) Zero() pref.Value {
+	return c.PBValueOf(reflect.Zero(c.goType))
+}
+
+// isNonPointer reports whether the type is a non-pointer type.
+// This never occurs for generated message types.
+func (c *messageConverter) isNonPointer() bool {
+	return c.goType.Kind() != reflect.Ptr
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/convert_list.go b/vendor/google.golang.org/protobuf/internal/impl/convert_list.go
new file mode 100644
index 0000000..6fccab5
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/convert_list.go
@@ -0,0 +1,141 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+func newListConverter(t reflect.Type, fd pref.FieldDescriptor) Converter {
+	switch {
+	case t.Kind() == reflect.Ptr && t.Elem().Kind() == reflect.Slice:
+		return &listPtrConverter{t, newSingularConverter(t.Elem().Elem(), fd)}
+	case t.Kind() == reflect.Slice:
+		return &listConverter{t, newSingularConverter(t.Elem(), fd)}
+	}
+	panic(fmt.Sprintf("invalid Go type %v for field %v", t, fd.FullName()))
+}
+
+type listConverter struct {
+	goType reflect.Type // []T
+	c      Converter
+}
+
+func (c *listConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	pv := reflect.New(c.goType)
+	pv.Elem().Set(v)
+	return pref.ValueOfList(&listReflect{pv, c.c})
+}
+
+func (c *listConverter) GoValueOf(v pref.Value) reflect.Value {
+	rv := v.List().(*listReflect).v
+	if rv.IsNil() {
+		return reflect.Zero(c.goType)
+	}
+	return rv.Elem()
+}
+
+func (c *listConverter) IsValidPB(v pref.Value) bool {
+	list, ok := v.Interface().(*listReflect)
+	if !ok {
+		return false
+	}
+	return list.v.Type().Elem() == c.goType
+}
+
+func (c *listConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+
+func (c *listConverter) New() pref.Value {
+	return pref.ValueOfList(&listReflect{reflect.New(c.goType), c.c})
+}
+
+func (c *listConverter) Zero() pref.Value {
+	return pref.ValueOfList(&listReflect{reflect.Zero(reflect.PtrTo(c.goType)), c.c})
+}
+
+type listPtrConverter struct {
+	goType reflect.Type // *[]T
+	c      Converter
+}
+
+func (c *listPtrConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfList(&listReflect{v, c.c})
+}
+
+func (c *listPtrConverter) GoValueOf(v pref.Value) reflect.Value {
+	return v.List().(*listReflect).v
+}
+
+func (c *listPtrConverter) IsValidPB(v pref.Value) bool {
+	list, ok := v.Interface().(*listReflect)
+	if !ok {
+		return false
+	}
+	return list.v.Type() == c.goType
+}
+
+func (c *listPtrConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+
+func (c *listPtrConverter) New() pref.Value {
+	return c.PBValueOf(reflect.New(c.goType.Elem()))
+}
+
+func (c *listPtrConverter) Zero() pref.Value {
+	return c.PBValueOf(reflect.Zero(c.goType))
+}
+
+type listReflect struct {
+	v    reflect.Value // *[]T
+	conv Converter
+}
+
+func (ls *listReflect) Len() int {
+	if ls.v.IsNil() {
+		return 0
+	}
+	return ls.v.Elem().Len()
+}
+func (ls *listReflect) Get(i int) pref.Value {
+	return ls.conv.PBValueOf(ls.v.Elem().Index(i))
+}
+func (ls *listReflect) Set(i int, v pref.Value) {
+	ls.v.Elem().Index(i).Set(ls.conv.GoValueOf(v))
+}
+func (ls *listReflect) Append(v pref.Value) {
+	ls.v.Elem().Set(reflect.Append(ls.v.Elem(), ls.conv.GoValueOf(v)))
+}
+func (ls *listReflect) AppendMutable() pref.Value {
+	if _, ok := ls.conv.(*messageConverter); !ok {
+		panic("invalid AppendMutable on list with non-message type")
+	}
+	v := ls.NewElement()
+	ls.Append(v)
+	return v
+}
+func (ls *listReflect) Truncate(i int) {
+	ls.v.Elem().Set(ls.v.Elem().Slice(0, i))
+}
+func (ls *listReflect) NewElement() pref.Value {
+	return ls.conv.New()
+}
+func (ls *listReflect) IsValid() bool {
+	return !ls.v.IsNil()
+}
+func (ls *listReflect) protoUnwrap() interface{} {
+	return ls.v.Interface()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/convert_map.go b/vendor/google.golang.org/protobuf/internal/impl/convert_map.go
new file mode 100644
index 0000000..de06b25
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/convert_map.go
@@ -0,0 +1,121 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type mapConverter struct {
+	goType           reflect.Type // map[K]V
+	keyConv, valConv Converter
+}
+
+func newMapConverter(t reflect.Type, fd pref.FieldDescriptor) *mapConverter {
+	if t.Kind() != reflect.Map {
+		panic(fmt.Sprintf("invalid Go type %v for field %v", t, fd.FullName()))
+	}
+	return &mapConverter{
+		goType:  t,
+		keyConv: newSingularConverter(t.Key(), fd.MapKey()),
+		valConv: newSingularConverter(t.Elem(), fd.MapValue()),
+	}
+}
+
+func (c *mapConverter) PBValueOf(v reflect.Value) pref.Value {
+	if v.Type() != c.goType {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", v.Type(), c.goType))
+	}
+	return pref.ValueOfMap(&mapReflect{v, c.keyConv, c.valConv})
+}
+
+func (c *mapConverter) GoValueOf(v pref.Value) reflect.Value {
+	return v.Map().(*mapReflect).v
+}
+
+func (c *mapConverter) IsValidPB(v pref.Value) bool {
+	mapv, ok := v.Interface().(*mapReflect)
+	if !ok {
+		return false
+	}
+	return mapv.v.Type() == c.goType
+}
+
+func (c *mapConverter) IsValidGo(v reflect.Value) bool {
+	return v.IsValid() && v.Type() == c.goType
+}
+
+func (c *mapConverter) New() pref.Value {
+	return c.PBValueOf(reflect.MakeMap(c.goType))
+}
+
+func (c *mapConverter) Zero() pref.Value {
+	return c.PBValueOf(reflect.Zero(c.goType))
+}
+
+type mapReflect struct {
+	v       reflect.Value // map[K]V
+	keyConv Converter
+	valConv Converter
+}
+
+func (ms *mapReflect) Len() int {
+	return ms.v.Len()
+}
+func (ms *mapReflect) Has(k pref.MapKey) bool {
+	rk := ms.keyConv.GoValueOf(k.Value())
+	rv := ms.v.MapIndex(rk)
+	return rv.IsValid()
+}
+func (ms *mapReflect) Get(k pref.MapKey) pref.Value {
+	rk := ms.keyConv.GoValueOf(k.Value())
+	rv := ms.v.MapIndex(rk)
+	if !rv.IsValid() {
+		return pref.Value{}
+	}
+	return ms.valConv.PBValueOf(rv)
+}
+func (ms *mapReflect) Set(k pref.MapKey, v pref.Value) {
+	rk := ms.keyConv.GoValueOf(k.Value())
+	rv := ms.valConv.GoValueOf(v)
+	ms.v.SetMapIndex(rk, rv)
+}
+func (ms *mapReflect) Clear(k pref.MapKey) {
+	rk := ms.keyConv.GoValueOf(k.Value())
+	ms.v.SetMapIndex(rk, reflect.Value{})
+}
+func (ms *mapReflect) Mutable(k pref.MapKey) pref.Value {
+	if _, ok := ms.valConv.(*messageConverter); !ok {
+		panic("invalid Mutable on map with non-message value type")
+	}
+	v := ms.Get(k)
+	if !v.IsValid() {
+		v = ms.NewValue()
+		ms.Set(k, v)
+	}
+	return v
+}
+func (ms *mapReflect) Range(f func(pref.MapKey, pref.Value) bool) {
+	iter := mapRange(ms.v)
+	for iter.Next() {
+		k := ms.keyConv.PBValueOf(iter.Key()).MapKey()
+		v := ms.valConv.PBValueOf(iter.Value())
+		if !f(k, v) {
+			return
+		}
+	}
+}
+func (ms *mapReflect) NewValue() pref.Value {
+	return ms.valConv.New()
+}
+func (ms *mapReflect) IsValid() bool {
+	return !ms.v.IsNil()
+}
+func (ms *mapReflect) protoUnwrap() interface{} {
+	return ms.v.Interface()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/decode.go b/vendor/google.golang.org/protobuf/internal/impl/decode.go
new file mode 100644
index 0000000..949dc49
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/decode.go
@@ -0,0 +1,276 @@
+// 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.
+
+package impl
+
+import (
+	"math/bits"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+	"google.golang.org/protobuf/runtime/protoiface"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+var errDecode = errors.New("cannot parse invalid wire-format data")
+
+type unmarshalOptions struct {
+	flags    protoiface.UnmarshalInputFlags
+	resolver interface {
+		FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error)
+		FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error)
+	}
+}
+
+func (o unmarshalOptions) Options() proto.UnmarshalOptions {
+	return proto.UnmarshalOptions{
+		Merge:          true,
+		AllowPartial:   true,
+		DiscardUnknown: o.DiscardUnknown(),
+		Resolver:       o.resolver,
+	}
+}
+
+func (o unmarshalOptions) DiscardUnknown() bool { return o.flags&piface.UnmarshalDiscardUnknown != 0 }
+
+func (o unmarshalOptions) IsDefault() bool {
+	return o.flags == 0 && o.resolver == preg.GlobalTypes
+}
+
+var lazyUnmarshalOptions = unmarshalOptions{
+	resolver: preg.GlobalTypes,
+}
+
+type unmarshalOutput struct {
+	n           int // number of bytes consumed
+	initialized bool
+}
+
+// unmarshal is protoreflect.Methods.Unmarshal.
+func (mi *MessageInfo) unmarshal(in piface.UnmarshalInput) (piface.UnmarshalOutput, error) {
+	var p pointer
+	if ms, ok := in.Message.(*messageState); ok {
+		p = ms.pointer()
+	} else {
+		p = in.Message.(*messageReflectWrapper).pointer()
+	}
+	out, err := mi.unmarshalPointer(in.Buf, p, 0, unmarshalOptions{
+		flags:    in.Flags,
+		resolver: in.Resolver,
+	})
+	var flags piface.UnmarshalOutputFlags
+	if out.initialized {
+		flags |= piface.UnmarshalInitialized
+	}
+	return piface.UnmarshalOutput{
+		Flags: flags,
+	}, err
+}
+
+// errUnknown is returned during unmarshaling to indicate a parse error that
+// should result in a field being placed in the unknown fields section (for example,
+// when the wire type doesn't match) as opposed to the entire unmarshal operation
+// failing (for example, when a field extends past the available input).
+//
+// This is a sentinel error which should never be visible to the user.
+var errUnknown = errors.New("unknown")
+
+func (mi *MessageInfo) unmarshalPointer(b []byte, p pointer, groupTag protowire.Number, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	mi.init()
+	if flags.ProtoLegacy && mi.isMessageSet {
+		return unmarshalMessageSet(mi, b, p, opts)
+	}
+	initialized := true
+	var requiredMask uint64
+	var exts *map[int32]ExtensionField
+	start := len(b)
+	for len(b) > 0 {
+		// Parse the tag (field number and wire type).
+		var tag uint64
+		if b[0] < 0x80 {
+			tag = uint64(b[0])
+			b = b[1:]
+		} else if len(b) >= 2 && b[1] < 128 {
+			tag = uint64(b[0]&0x7f) + uint64(b[1])<<7
+			b = b[2:]
+		} else {
+			var n int
+			tag, n = protowire.ConsumeVarint(b)
+			if n < 0 {
+				return out, errDecode
+			}
+			b = b[n:]
+		}
+		var num protowire.Number
+		if n := tag >> 3; n < uint64(protowire.MinValidNumber) || n > uint64(protowire.MaxValidNumber) {
+			return out, errDecode
+		} else {
+			num = protowire.Number(n)
+		}
+		wtyp := protowire.Type(tag & 7)
+
+		if wtyp == protowire.EndGroupType {
+			if num != groupTag {
+				return out, errDecode
+			}
+			groupTag = 0
+			break
+		}
+
+		var f *coderFieldInfo
+		if int(num) < len(mi.denseCoderFields) {
+			f = mi.denseCoderFields[num]
+		} else {
+			f = mi.coderFields[num]
+		}
+		var n int
+		err := errUnknown
+		switch {
+		case f != nil:
+			if f.funcs.unmarshal == nil {
+				break
+			}
+			var o unmarshalOutput
+			o, err = f.funcs.unmarshal(b, p.Apply(f.offset), wtyp, f, opts)
+			n = o.n
+			if err != nil {
+				break
+			}
+			requiredMask |= f.validation.requiredBit
+			if f.funcs.isInit != nil && !o.initialized {
+				initialized = false
+			}
+		default:
+			// Possible extension.
+			if exts == nil && mi.extensionOffset.IsValid() {
+				exts = p.Apply(mi.extensionOffset).Extensions()
+				if *exts == nil {
+					*exts = make(map[int32]ExtensionField)
+				}
+			}
+			if exts == nil {
+				break
+			}
+			var o unmarshalOutput
+			o, err = mi.unmarshalExtension(b, num, wtyp, *exts, opts)
+			if err != nil {
+				break
+			}
+			n = o.n
+			if !o.initialized {
+				initialized = false
+			}
+		}
+		if err != nil {
+			if err != errUnknown {
+				return out, err
+			}
+			n = protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return out, errDecode
+			}
+			if !opts.DiscardUnknown() && mi.unknownOffset.IsValid() {
+				u := mi.mutableUnknownBytes(p)
+				*u = protowire.AppendTag(*u, num, wtyp)
+				*u = append(*u, b[:n]...)
+			}
+		}
+		b = b[n:]
+	}
+	if groupTag != 0 {
+		return out, errDecode
+	}
+	if mi.numRequiredFields > 0 && bits.OnesCount64(requiredMask) != int(mi.numRequiredFields) {
+		initialized = false
+	}
+	if initialized {
+		out.initialized = true
+	}
+	out.n = start - len(b)
+	return out, nil
+}
+
+func (mi *MessageInfo) unmarshalExtension(b []byte, num protowire.Number, wtyp protowire.Type, exts map[int32]ExtensionField, opts unmarshalOptions) (out unmarshalOutput, err error) {
+	x := exts[int32(num)]
+	xt := x.Type()
+	if xt == nil {
+		var err error
+		xt, err = opts.resolver.FindExtensionByNumber(mi.Desc.FullName(), num)
+		if err != nil {
+			if err == preg.NotFound {
+				return out, errUnknown
+			}
+			return out, errors.New("%v: unable to resolve extension %v: %v", mi.Desc.FullName(), num, err)
+		}
+	}
+	xi := getExtensionFieldInfo(xt)
+	if xi.funcs.unmarshal == nil {
+		return out, errUnknown
+	}
+	if flags.LazyUnmarshalExtensions {
+		if opts.IsDefault() && x.canLazy(xt) {
+			out, valid := skipExtension(b, xi, num, wtyp, opts)
+			switch valid {
+			case ValidationValid:
+				if out.initialized {
+					x.appendLazyBytes(xt, xi, num, wtyp, b[:out.n])
+					exts[int32(num)] = x
+					return out, nil
+				}
+			case ValidationInvalid:
+				return out, errDecode
+			case ValidationUnknown:
+			}
+		}
+	}
+	ival := x.Value()
+	if !ival.IsValid() && xi.unmarshalNeedsValue {
+		// Create a new message, list, or map value to fill in.
+		// For enums, create a prototype value to let the unmarshal func know the
+		// concrete type.
+		ival = xt.New()
+	}
+	v, out, err := xi.funcs.unmarshal(b, ival, num, wtyp, opts)
+	if err != nil {
+		return out, err
+	}
+	if xi.funcs.isInit == nil {
+		out.initialized = true
+	}
+	x.Set(xt, v)
+	exts[int32(num)] = x
+	return out, nil
+}
+
+func skipExtension(b []byte, xi *extensionFieldInfo, num protowire.Number, wtyp protowire.Type, opts unmarshalOptions) (out unmarshalOutput, _ ValidationStatus) {
+	if xi.validation.mi == nil {
+		return out, ValidationUnknown
+	}
+	xi.validation.mi.init()
+	switch xi.validation.typ {
+	case validationTypeMessage:
+		if wtyp != protowire.BytesType {
+			return out, ValidationUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return out, ValidationUnknown
+		}
+		out, st := xi.validation.mi.validate(v, 0, opts)
+		out.n = n
+		return out, st
+	case validationTypeGroup:
+		if wtyp != protowire.StartGroupType {
+			return out, ValidationUnknown
+		}
+		out, st := xi.validation.mi.validate(b, num, opts)
+		return out, st
+	default:
+		return out, ValidationUnknown
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/encode.go b/vendor/google.golang.org/protobuf/internal/impl/encode.go
new file mode 100644
index 0000000..845c67d
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/encode.go
@@ -0,0 +1,201 @@
+// 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.
+
+package impl
+
+import (
+	"math"
+	"sort"
+	"sync/atomic"
+
+	"google.golang.org/protobuf/internal/flags"
+	proto "google.golang.org/protobuf/proto"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+type marshalOptions struct {
+	flags piface.MarshalInputFlags
+}
+
+func (o marshalOptions) Options() proto.MarshalOptions {
+	return proto.MarshalOptions{
+		AllowPartial:  true,
+		Deterministic: o.Deterministic(),
+		UseCachedSize: o.UseCachedSize(),
+	}
+}
+
+func (o marshalOptions) Deterministic() bool { return o.flags&piface.MarshalDeterministic != 0 }
+func (o marshalOptions) UseCachedSize() bool { return o.flags&piface.MarshalUseCachedSize != 0 }
+
+// size is protoreflect.Methods.Size.
+func (mi *MessageInfo) size(in piface.SizeInput) piface.SizeOutput {
+	var p pointer
+	if ms, ok := in.Message.(*messageState); ok {
+		p = ms.pointer()
+	} else {
+		p = in.Message.(*messageReflectWrapper).pointer()
+	}
+	size := mi.sizePointer(p, marshalOptions{
+		flags: in.Flags,
+	})
+	return piface.SizeOutput{Size: size}
+}
+
+func (mi *MessageInfo) sizePointer(p pointer, opts marshalOptions) (size int) {
+	mi.init()
+	if p.IsNil() {
+		return 0
+	}
+	if opts.UseCachedSize() && mi.sizecacheOffset.IsValid() {
+		if size := atomic.LoadInt32(p.Apply(mi.sizecacheOffset).Int32()); size >= 0 {
+			return int(size)
+		}
+	}
+	return mi.sizePointerSlow(p, opts)
+}
+
+func (mi *MessageInfo) sizePointerSlow(p pointer, opts marshalOptions) (size int) {
+	if flags.ProtoLegacy && mi.isMessageSet {
+		size = sizeMessageSet(mi, p, opts)
+		if mi.sizecacheOffset.IsValid() {
+			atomic.StoreInt32(p.Apply(mi.sizecacheOffset).Int32(), int32(size))
+		}
+		return size
+	}
+	if mi.extensionOffset.IsValid() {
+		e := p.Apply(mi.extensionOffset).Extensions()
+		size += mi.sizeExtensions(e, opts)
+	}
+	for _, f := range mi.orderedCoderFields {
+		if f.funcs.size == nil {
+			continue
+		}
+		fptr := p.Apply(f.offset)
+		if f.isPointer && fptr.Elem().IsNil() {
+			continue
+		}
+		size += f.funcs.size(fptr, f, opts)
+	}
+	if mi.unknownOffset.IsValid() {
+		if u := mi.getUnknownBytes(p); u != nil {
+			size += len(*u)
+		}
+	}
+	if mi.sizecacheOffset.IsValid() {
+		if size > math.MaxInt32 {
+			// The size is too large for the int32 sizecache field.
+			// We will need to recompute the size when encoding;
+			// unfortunately expensive, but better than invalid output.
+			atomic.StoreInt32(p.Apply(mi.sizecacheOffset).Int32(), -1)
+		} else {
+			atomic.StoreInt32(p.Apply(mi.sizecacheOffset).Int32(), int32(size))
+		}
+	}
+	return size
+}
+
+// marshal is protoreflect.Methods.Marshal.
+func (mi *MessageInfo) marshal(in piface.MarshalInput) (out piface.MarshalOutput, err error) {
+	var p pointer
+	if ms, ok := in.Message.(*messageState); ok {
+		p = ms.pointer()
+	} else {
+		p = in.Message.(*messageReflectWrapper).pointer()
+	}
+	b, err := mi.marshalAppendPointer(in.Buf, p, marshalOptions{
+		flags: in.Flags,
+	})
+	return piface.MarshalOutput{Buf: b}, err
+}
+
+func (mi *MessageInfo) marshalAppendPointer(b []byte, p pointer, opts marshalOptions) ([]byte, error) {
+	mi.init()
+	if p.IsNil() {
+		return b, nil
+	}
+	if flags.ProtoLegacy && mi.isMessageSet {
+		return marshalMessageSet(mi, b, p, opts)
+	}
+	var err error
+	// The old marshaler encodes extensions at beginning.
+	if mi.extensionOffset.IsValid() {
+		e := p.Apply(mi.extensionOffset).Extensions()
+		// TODO: Special handling for MessageSet?
+		b, err = mi.appendExtensions(b, e, opts)
+		if err != nil {
+			return b, err
+		}
+	}
+	for _, f := range mi.orderedCoderFields {
+		if f.funcs.marshal == nil {
+			continue
+		}
+		fptr := p.Apply(f.offset)
+		if f.isPointer && fptr.Elem().IsNil() {
+			continue
+		}
+		b, err = f.funcs.marshal(b, fptr, f, opts)
+		if err != nil {
+			return b, err
+		}
+	}
+	if mi.unknownOffset.IsValid() && !mi.isMessageSet {
+		if u := mi.getUnknownBytes(p); u != nil {
+			b = append(b, (*u)...)
+		}
+	}
+	return b, nil
+}
+
+func (mi *MessageInfo) sizeExtensions(ext *map[int32]ExtensionField, opts marshalOptions) (n int) {
+	if ext == nil {
+		return 0
+	}
+	for _, x := range *ext {
+		xi := getExtensionFieldInfo(x.Type())
+		if xi.funcs.size == nil {
+			continue
+		}
+		n += xi.funcs.size(x.Value(), xi.tagsize, opts)
+	}
+	return n
+}
+
+func (mi *MessageInfo) appendExtensions(b []byte, ext *map[int32]ExtensionField, opts marshalOptions) ([]byte, error) {
+	if ext == nil {
+		return b, nil
+	}
+
+	switch len(*ext) {
+	case 0:
+		return b, nil
+	case 1:
+		// Fast-path for one extension: Don't bother sorting the keys.
+		var err error
+		for _, x := range *ext {
+			xi := getExtensionFieldInfo(x.Type())
+			b, err = xi.funcs.marshal(b, x.Value(), xi.wiretag, opts)
+		}
+		return b, err
+	default:
+		// Sort the keys to provide a deterministic encoding.
+		// Not sure this is required, but the old code does it.
+		keys := make([]int, 0, len(*ext))
+		for k := range *ext {
+			keys = append(keys, int(k))
+		}
+		sort.Ints(keys)
+		var err error
+		for _, k := range keys {
+			x := (*ext)[int32(k)]
+			xi := getExtensionFieldInfo(x.Type())
+			b, err = xi.funcs.marshal(b, x.Value(), xi.wiretag, opts)
+			if err != nil {
+				return b, err
+			}
+		}
+		return b, nil
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/enum.go b/vendor/google.golang.org/protobuf/internal/impl/enum.go
new file mode 100644
index 0000000..8c1eab4
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/enum.go
@@ -0,0 +1,21 @@
+// 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.
+
+package impl
+
+import (
+	"reflect"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type EnumInfo struct {
+	GoReflectType reflect.Type // int32 kind
+	Desc          pref.EnumDescriptor
+}
+
+func (t *EnumInfo) New(n pref.EnumNumber) pref.Enum {
+	return reflect.ValueOf(n).Convert(t.GoReflectType).Interface().(pref.Enum)
+}
+func (t *EnumInfo) Descriptor() pref.EnumDescriptor { return t.Desc }
diff --git a/vendor/google.golang.org/protobuf/internal/impl/extension.go b/vendor/google.golang.org/protobuf/internal/impl/extension.go
new file mode 100644
index 0000000..e904fd9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/extension.go
@@ -0,0 +1,156 @@
+// 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.
+
+package impl
+
+import (
+	"reflect"
+	"sync"
+	"sync/atomic"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// ExtensionInfo implements ExtensionType.
+//
+// This type contains a number of exported fields for legacy compatibility.
+// The only non-deprecated use of this type is through the methods of the
+// ExtensionType interface.
+type ExtensionInfo struct {
+	// An ExtensionInfo may exist in several stages of initialization.
+	//
+	// extensionInfoUninitialized: Some or all of the legacy exported
+	// fields may be set, but none of the unexported fields have been
+	// initialized. This is the starting state for an ExtensionInfo
+	// in legacy generated code.
+	//
+	// extensionInfoDescInit: The desc field is set, but other unexported fields
+	// may not be initialized. Legacy exported fields may or may not be set.
+	// This is the starting state for an ExtensionInfo in newly generated code.
+	//
+	// extensionInfoFullInit: The ExtensionInfo is fully initialized.
+	// This state is only entered after lazy initialization is complete.
+	init uint32
+	mu   sync.Mutex
+
+	goType reflect.Type
+	desc   extensionTypeDescriptor
+	conv   Converter
+	info   *extensionFieldInfo // for fast-path method implementations
+
+	// ExtendedType is a typed nil-pointer to the parent message type that
+	// is being extended. It is possible for this to be unpopulated in v2
+	// since the message may no longer implement the MessageV1 interface.
+	//
+	// Deprecated: Use the ExtendedType method instead.
+	ExtendedType piface.MessageV1
+
+	// ExtensionType is the zero value of the extension type.
+	//
+	// For historical reasons, reflect.TypeOf(ExtensionType) and the
+	// type returned by InterfaceOf may not be identical.
+	//
+	// Deprecated: Use InterfaceOf(xt.Zero()) instead.
+	ExtensionType interface{}
+
+	// Field is the field number of the extension.
+	//
+	// Deprecated: Use the Descriptor().Number method instead.
+	Field int32
+
+	// Name is the fully qualified name of extension.
+	//
+	// Deprecated: Use the Descriptor().FullName method instead.
+	Name string
+
+	// Tag is the protobuf struct tag used in the v1 API.
+	//
+	// Deprecated: Do not use.
+	Tag string
+
+	// Filename is the proto filename in which the extension is defined.
+	//
+	// Deprecated: Use Descriptor().ParentFile().Path() instead.
+	Filename string
+}
+
+// Stages of initialization: See the ExtensionInfo.init field.
+const (
+	extensionInfoUninitialized = 0
+	extensionInfoDescInit      = 1
+	extensionInfoFullInit      = 2
+)
+
+func InitExtensionInfo(xi *ExtensionInfo, xd pref.ExtensionDescriptor, goType reflect.Type) {
+	xi.goType = goType
+	xi.desc = extensionTypeDescriptor{xd, xi}
+	xi.init = extensionInfoDescInit
+}
+
+func (xi *ExtensionInfo) New() pref.Value {
+	return xi.lazyInit().New()
+}
+func (xi *ExtensionInfo) Zero() pref.Value {
+	return xi.lazyInit().Zero()
+}
+func (xi *ExtensionInfo) ValueOf(v interface{}) pref.Value {
+	return xi.lazyInit().PBValueOf(reflect.ValueOf(v))
+}
+func (xi *ExtensionInfo) InterfaceOf(v pref.Value) interface{} {
+	return xi.lazyInit().GoValueOf(v).Interface()
+}
+func (xi *ExtensionInfo) IsValidValue(v pref.Value) bool {
+	return xi.lazyInit().IsValidPB(v)
+}
+func (xi *ExtensionInfo) IsValidInterface(v interface{}) bool {
+	return xi.lazyInit().IsValidGo(reflect.ValueOf(v))
+}
+func (xi *ExtensionInfo) TypeDescriptor() pref.ExtensionTypeDescriptor {
+	if atomic.LoadUint32(&xi.init) < extensionInfoDescInit {
+		xi.lazyInitSlow()
+	}
+	return &xi.desc
+}
+
+func (xi *ExtensionInfo) lazyInit() Converter {
+	if atomic.LoadUint32(&xi.init) < extensionInfoFullInit {
+		xi.lazyInitSlow()
+	}
+	return xi.conv
+}
+
+func (xi *ExtensionInfo) lazyInitSlow() {
+	xi.mu.Lock()
+	defer xi.mu.Unlock()
+
+	if xi.init == extensionInfoFullInit {
+		return
+	}
+	defer atomic.StoreUint32(&xi.init, extensionInfoFullInit)
+
+	if xi.desc.ExtensionDescriptor == nil {
+		xi.initFromLegacy()
+	}
+	if !xi.desc.ExtensionDescriptor.IsPlaceholder() {
+		if xi.ExtensionType == nil {
+			xi.initToLegacy()
+		}
+		xi.conv = NewConverter(xi.goType, xi.desc.ExtensionDescriptor)
+		xi.info = makeExtensionFieldInfo(xi.desc.ExtensionDescriptor)
+		xi.info.validation = newValidationInfo(xi.desc.ExtensionDescriptor, xi.goType)
+	}
+}
+
+type extensionTypeDescriptor struct {
+	pref.ExtensionDescriptor
+	xi *ExtensionInfo
+}
+
+func (xtd *extensionTypeDescriptor) Type() pref.ExtensionType {
+	return xtd.xi
+}
+func (xtd *extensionTypeDescriptor) Descriptor() pref.ExtensionDescriptor {
+	return xtd.ExtensionDescriptor
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/legacy_enum.go b/vendor/google.golang.org/protobuf/internal/impl/legacy_enum.go
new file mode 100644
index 0000000..f7d7ffb
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/legacy_enum.go
@@ -0,0 +1,219 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+	"strings"
+	"sync"
+
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// legacyEnumName returns the name of enums used in legacy code.
+// It is neither the protobuf full name nor the qualified Go name,
+// but rather an odd hybrid of both.
+func legacyEnumName(ed pref.EnumDescriptor) string {
+	var protoPkg string
+	enumName := string(ed.FullName())
+	if fd := ed.ParentFile(); fd != nil {
+		protoPkg = string(fd.Package())
+		enumName = strings.TrimPrefix(enumName, protoPkg+".")
+	}
+	if protoPkg == "" {
+		return strs.GoCamelCase(enumName)
+	}
+	return protoPkg + "." + strs.GoCamelCase(enumName)
+}
+
+// legacyWrapEnum wraps v as a protoreflect.Enum,
+// where v must be a int32 kind and not implement the v2 API already.
+func legacyWrapEnum(v reflect.Value) pref.Enum {
+	et := legacyLoadEnumType(v.Type())
+	return et.New(pref.EnumNumber(v.Int()))
+}
+
+var legacyEnumTypeCache sync.Map // map[reflect.Type]protoreflect.EnumType
+
+// legacyLoadEnumType dynamically loads a protoreflect.EnumType for t,
+// where t must be an int32 kind and not implement the v2 API already.
+func legacyLoadEnumType(t reflect.Type) pref.EnumType {
+	// Fast-path: check if a EnumType is cached for this concrete type.
+	if et, ok := legacyEnumTypeCache.Load(t); ok {
+		return et.(pref.EnumType)
+	}
+
+	// Slow-path: derive enum descriptor and initialize EnumType.
+	var et pref.EnumType
+	ed := LegacyLoadEnumDesc(t)
+	et = &legacyEnumType{
+		desc:   ed,
+		goType: t,
+	}
+	if et, ok := legacyEnumTypeCache.LoadOrStore(t, et); ok {
+		return et.(pref.EnumType)
+	}
+	return et
+}
+
+type legacyEnumType struct {
+	desc   pref.EnumDescriptor
+	goType reflect.Type
+	m      sync.Map // map[protoreflect.EnumNumber]proto.Enum
+}
+
+func (t *legacyEnumType) New(n pref.EnumNumber) pref.Enum {
+	if e, ok := t.m.Load(n); ok {
+		return e.(pref.Enum)
+	}
+	e := &legacyEnumWrapper{num: n, pbTyp: t, goTyp: t.goType}
+	t.m.Store(n, e)
+	return e
+}
+func (t *legacyEnumType) Descriptor() pref.EnumDescriptor {
+	return t.desc
+}
+
+type legacyEnumWrapper struct {
+	num   pref.EnumNumber
+	pbTyp pref.EnumType
+	goTyp reflect.Type
+}
+
+func (e *legacyEnumWrapper) Descriptor() pref.EnumDescriptor {
+	return e.pbTyp.Descriptor()
+}
+func (e *legacyEnumWrapper) Type() pref.EnumType {
+	return e.pbTyp
+}
+func (e *legacyEnumWrapper) Number() pref.EnumNumber {
+	return e.num
+}
+func (e *legacyEnumWrapper) ProtoReflect() pref.Enum {
+	return e
+}
+func (e *legacyEnumWrapper) protoUnwrap() interface{} {
+	v := reflect.New(e.goTyp).Elem()
+	v.SetInt(int64(e.num))
+	return v.Interface()
+}
+
+var (
+	_ pref.Enum = (*legacyEnumWrapper)(nil)
+	_ unwrapper = (*legacyEnumWrapper)(nil)
+)
+
+var legacyEnumDescCache sync.Map // map[reflect.Type]protoreflect.EnumDescriptor
+
+// LegacyLoadEnumDesc returns an EnumDescriptor derived from the Go type,
+// which must be an int32 kind and not implement the v2 API already.
+//
+// This is exported for testing purposes.
+func LegacyLoadEnumDesc(t reflect.Type) pref.EnumDescriptor {
+	// Fast-path: check if an EnumDescriptor is cached for this concrete type.
+	if ed, ok := legacyEnumDescCache.Load(t); ok {
+		return ed.(pref.EnumDescriptor)
+	}
+
+	// Slow-path: initialize EnumDescriptor from the raw descriptor.
+	ev := reflect.Zero(t).Interface()
+	if _, ok := ev.(pref.Enum); ok {
+		panic(fmt.Sprintf("%v already implements proto.Enum", t))
+	}
+	edV1, ok := ev.(enumV1)
+	if !ok {
+		return aberrantLoadEnumDesc(t)
+	}
+	b, idxs := edV1.EnumDescriptor()
+
+	var ed pref.EnumDescriptor
+	if len(idxs) == 1 {
+		ed = legacyLoadFileDesc(b).Enums().Get(idxs[0])
+	} else {
+		md := legacyLoadFileDesc(b).Messages().Get(idxs[0])
+		for _, i := range idxs[1 : len(idxs)-1] {
+			md = md.Messages().Get(i)
+		}
+		ed = md.Enums().Get(idxs[len(idxs)-1])
+	}
+	if ed, ok := legacyEnumDescCache.LoadOrStore(t, ed); ok {
+		return ed.(protoreflect.EnumDescriptor)
+	}
+	return ed
+}
+
+var aberrantEnumDescCache sync.Map // map[reflect.Type]protoreflect.EnumDescriptor
+
+// aberrantLoadEnumDesc returns an EnumDescriptor derived from the Go type,
+// which must not implement protoreflect.Enum or enumV1.
+//
+// If the type does not implement enumV1, then there is no reliable
+// way to derive the original protobuf type information.
+// We are unable to use the global enum registry since it is
+// unfortunately keyed by the protobuf full name, which we also do not know.
+// Thus, this produces some bogus enum descriptor based on the Go type name.
+func aberrantLoadEnumDesc(t reflect.Type) pref.EnumDescriptor {
+	// Fast-path: check if an EnumDescriptor is cached for this concrete type.
+	if ed, ok := aberrantEnumDescCache.Load(t); ok {
+		return ed.(pref.EnumDescriptor)
+	}
+
+	// Slow-path: construct a bogus, but unique EnumDescriptor.
+	ed := &filedesc.Enum{L2: new(filedesc.EnumL2)}
+	ed.L0.FullName = AberrantDeriveFullName(t) // e.g., github_com.user.repo.MyEnum
+	ed.L0.ParentFile = filedesc.SurrogateProto3
+	ed.L2.Values.List = append(ed.L2.Values.List, filedesc.EnumValue{})
+
+	// TODO: Use the presence of a UnmarshalJSON method to determine proto2?
+
+	vd := &ed.L2.Values.List[0]
+	vd.L0.FullName = ed.L0.FullName + "_UNKNOWN" // e.g., github_com.user.repo.MyEnum_UNKNOWN
+	vd.L0.ParentFile = ed.L0.ParentFile
+	vd.L0.Parent = ed
+
+	// TODO: We could use the String method to obtain some enum value names by
+	// starting at 0 and print the enum until it produces invalid identifiers.
+	// An exhaustive query is clearly impractical, but can be best-effort.
+
+	if ed, ok := aberrantEnumDescCache.LoadOrStore(t, ed); ok {
+		return ed.(pref.EnumDescriptor)
+	}
+	return ed
+}
+
+// AberrantDeriveFullName derives a fully qualified protobuf name for the given Go type
+// The provided name is not guaranteed to be stable nor universally unique.
+// It should be sufficiently unique within a program.
+//
+// This is exported for testing purposes.
+func AberrantDeriveFullName(t reflect.Type) pref.FullName {
+	sanitize := func(r rune) rune {
+		switch {
+		case r == '/':
+			return '.'
+		case 'a' <= r && r <= 'z', 'A' <= r && r <= 'Z', '0' <= r && r <= '9':
+			return r
+		default:
+			return '_'
+		}
+	}
+	prefix := strings.Map(sanitize, t.PkgPath())
+	suffix := strings.Map(sanitize, t.Name())
+	if suffix == "" {
+		suffix = fmt.Sprintf("UnknownX%X", reflect.ValueOf(t).Pointer())
+	}
+
+	ss := append(strings.Split(prefix, "."), suffix)
+	for i, s := range ss {
+		if s == "" || ('0' <= s[0] && s[0] <= '9') {
+			ss[i] = "x" + s
+		}
+	}
+	return pref.FullName(strings.Join(ss, "."))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/legacy_export.go b/vendor/google.golang.org/protobuf/internal/impl/legacy_export.go
new file mode 100644
index 0000000..e3fb0b5
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/legacy_export.go
@@ -0,0 +1,92 @@
+// 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.
+
+package impl
+
+import (
+	"encoding/binary"
+	"encoding/json"
+	"hash/crc32"
+	"math"
+	"reflect"
+
+	"google.golang.org/protobuf/internal/errors"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// These functions exist to support exported APIs in generated protobufs.
+// While these are deprecated, they cannot be removed for compatibility reasons.
+
+// LegacyEnumName returns the name of enums used in legacy code.
+func (Export) LegacyEnumName(ed pref.EnumDescriptor) string {
+	return legacyEnumName(ed)
+}
+
+// LegacyMessageTypeOf returns the protoreflect.MessageType for m,
+// with name used as the message name if necessary.
+func (Export) LegacyMessageTypeOf(m piface.MessageV1, name pref.FullName) pref.MessageType {
+	if mv := (Export{}).protoMessageV2Of(m); mv != nil {
+		return mv.ProtoReflect().Type()
+	}
+	return legacyLoadMessageType(reflect.TypeOf(m), name)
+}
+
+// UnmarshalJSONEnum unmarshals an enum from a JSON-encoded input.
+// The input can either be a string representing the enum value by name,
+// or a number representing the enum number itself.
+func (Export) UnmarshalJSONEnum(ed pref.EnumDescriptor, b []byte) (pref.EnumNumber, error) {
+	if b[0] == '"' {
+		var name pref.Name
+		if err := json.Unmarshal(b, &name); err != nil {
+			return 0, errors.New("invalid input for enum %v: %s", ed.FullName(), b)
+		}
+		ev := ed.Values().ByName(name)
+		if ev == nil {
+			return 0, errors.New("invalid value for enum %v: %s", ed.FullName(), name)
+		}
+		return ev.Number(), nil
+	} else {
+		var num pref.EnumNumber
+		if err := json.Unmarshal(b, &num); err != nil {
+			return 0, errors.New("invalid input for enum %v: %s", ed.FullName(), b)
+		}
+		return num, nil
+	}
+}
+
+// CompressGZIP compresses the input as a GZIP-encoded file.
+// The current implementation does no compression.
+func (Export) CompressGZIP(in []byte) (out []byte) {
+	// RFC 1952, section 2.3.1.
+	var gzipHeader = [10]byte{0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff}
+
+	// RFC 1951, section 3.2.4.
+	var blockHeader [5]byte
+	const maxBlockSize = math.MaxUint16
+	numBlocks := 1 + len(in)/maxBlockSize
+
+	// RFC 1952, section 2.3.1.
+	var gzipFooter [8]byte
+	binary.LittleEndian.PutUint32(gzipFooter[0:4], crc32.ChecksumIEEE(in))
+	binary.LittleEndian.PutUint32(gzipFooter[4:8], uint32(len(in)))
+
+	// Encode the input without compression using raw DEFLATE blocks.
+	out = make([]byte, 0, len(gzipHeader)+len(blockHeader)*numBlocks+len(in)+len(gzipFooter))
+	out = append(out, gzipHeader[:]...)
+	for blockHeader[0] == 0 {
+		blockSize := maxBlockSize
+		if blockSize > len(in) {
+			blockHeader[0] = 0x01 // final bit per RFC 1951, section 3.2.3.
+			blockSize = len(in)
+		}
+		binary.LittleEndian.PutUint16(blockHeader[1:3], uint16(blockSize)^0x0000)
+		binary.LittleEndian.PutUint16(blockHeader[3:5], uint16(blockSize)^0xffff)
+		out = append(out, blockHeader[:]...)
+		out = append(out, in[:blockSize]...)
+		in = in[blockSize:]
+	}
+	out = append(out, gzipFooter[:]...)
+	return out
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/legacy_extension.go b/vendor/google.golang.org/protobuf/internal/impl/legacy_extension.go
new file mode 100644
index 0000000..49e7231
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/legacy_extension.go
@@ -0,0 +1,176 @@
+// 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 impl
+
+import (
+	"reflect"
+
+	"google.golang.org/protobuf/internal/descopts"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	ptag "google.golang.org/protobuf/internal/encoding/tag"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/pragma"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+func (xi *ExtensionInfo) initToLegacy() {
+	xd := xi.desc
+	var parent piface.MessageV1
+	messageName := xd.ContainingMessage().FullName()
+	if mt, _ := preg.GlobalTypes.FindMessageByName(messageName); mt != nil {
+		// Create a new parent message and unwrap it if possible.
+		mv := mt.New().Interface()
+		t := reflect.TypeOf(mv)
+		if mv, ok := mv.(unwrapper); ok {
+			t = reflect.TypeOf(mv.protoUnwrap())
+		}
+
+		// Check whether the message implements the legacy v1 Message interface.
+		mz := reflect.Zero(t).Interface()
+		if mz, ok := mz.(piface.MessageV1); ok {
+			parent = mz
+		}
+	}
+
+	// Determine the v1 extension type, which is unfortunately not the same as
+	// the v2 ExtensionType.GoType.
+	extType := xi.goType
+	switch extType.Kind() {
+	case reflect.Bool, reflect.Int32, reflect.Int64, reflect.Uint32, reflect.Uint64, reflect.Float32, reflect.Float64, reflect.String:
+		extType = reflect.PtrTo(extType) // T -> *T for singular scalar fields
+	}
+
+	// Reconstruct the legacy enum full name.
+	var enumName string
+	if xd.Kind() == pref.EnumKind {
+		enumName = legacyEnumName(xd.Enum())
+	}
+
+	// Derive the proto file that the extension was declared within.
+	var filename string
+	if fd := xd.ParentFile(); fd != nil {
+		filename = fd.Path()
+	}
+
+	// For MessageSet extensions, the name used is the parent message.
+	name := xd.FullName()
+	if messageset.IsMessageSetExtension(xd) {
+		name = name.Parent()
+	}
+
+	xi.ExtendedType = parent
+	xi.ExtensionType = reflect.Zero(extType).Interface()
+	xi.Field = int32(xd.Number())
+	xi.Name = string(name)
+	xi.Tag = ptag.Marshal(xd, enumName)
+	xi.Filename = filename
+}
+
+// initFromLegacy initializes an ExtensionInfo from
+// the contents of the deprecated exported fields of the type.
+func (xi *ExtensionInfo) initFromLegacy() {
+	// The v1 API returns "type incomplete" descriptors where only the
+	// field number is specified. In such a case, use a placeholder.
+	if xi.ExtendedType == nil || xi.ExtensionType == nil {
+		xd := placeholderExtension{
+			name:   pref.FullName(xi.Name),
+			number: pref.FieldNumber(xi.Field),
+		}
+		xi.desc = extensionTypeDescriptor{xd, xi}
+		return
+	}
+
+	// Resolve enum or message dependencies.
+	var ed pref.EnumDescriptor
+	var md pref.MessageDescriptor
+	t := reflect.TypeOf(xi.ExtensionType)
+	isOptional := t.Kind() == reflect.Ptr && t.Elem().Kind() != reflect.Struct
+	isRepeated := t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8
+	if isOptional || isRepeated {
+		t = t.Elem()
+	}
+	switch v := reflect.Zero(t).Interface().(type) {
+	case pref.Enum:
+		ed = v.Descriptor()
+	case enumV1:
+		ed = LegacyLoadEnumDesc(t)
+	case pref.ProtoMessage:
+		md = v.ProtoReflect().Descriptor()
+	case messageV1:
+		md = LegacyLoadMessageDesc(t)
+	}
+
+	// Derive basic field information from the struct tag.
+	var evs pref.EnumValueDescriptors
+	if ed != nil {
+		evs = ed.Values()
+	}
+	fd := ptag.Unmarshal(xi.Tag, t, evs).(*filedesc.Field)
+
+	// Construct a v2 ExtensionType.
+	xd := &filedesc.Extension{L2: new(filedesc.ExtensionL2)}
+	xd.L0.ParentFile = filedesc.SurrogateProto2
+	xd.L0.FullName = pref.FullName(xi.Name)
+	xd.L1.Number = pref.FieldNumber(xi.Field)
+	xd.L1.Cardinality = fd.L1.Cardinality
+	xd.L1.Kind = fd.L1.Kind
+	xd.L2.IsPacked = fd.L1.IsPacked
+	xd.L2.Default = fd.L1.Default
+	xd.L1.Extendee = Export{}.MessageDescriptorOf(xi.ExtendedType)
+	xd.L2.Enum = ed
+	xd.L2.Message = md
+
+	// Derive real extension field name for MessageSets.
+	if messageset.IsMessageSet(xd.L1.Extendee) && md.FullName() == xd.L0.FullName {
+		xd.L0.FullName = xd.L0.FullName.Append(messageset.ExtensionName)
+	}
+
+	tt := reflect.TypeOf(xi.ExtensionType)
+	if isOptional {
+		tt = tt.Elem()
+	}
+	xi.goType = tt
+	xi.desc = extensionTypeDescriptor{xd, xi}
+}
+
+type placeholderExtension struct {
+	name   pref.FullName
+	number pref.FieldNumber
+}
+
+func (x placeholderExtension) ParentFile() pref.FileDescriptor            { return nil }
+func (x placeholderExtension) Parent() pref.Descriptor                    { return nil }
+func (x placeholderExtension) Index() int                                 { return 0 }
+func (x placeholderExtension) Syntax() pref.Syntax                        { return 0 }
+func (x placeholderExtension) Name() pref.Name                            { return x.name.Name() }
+func (x placeholderExtension) FullName() pref.FullName                    { return x.name }
+func (x placeholderExtension) IsPlaceholder() bool                        { return true }
+func (x placeholderExtension) Options() pref.ProtoMessage                 { return descopts.Field }
+func (x placeholderExtension) Number() pref.FieldNumber                   { return x.number }
+func (x placeholderExtension) Cardinality() pref.Cardinality              { return 0 }
+func (x placeholderExtension) Kind() pref.Kind                            { return 0 }
+func (x placeholderExtension) HasJSONName() bool                          { return false }
+func (x placeholderExtension) JSONName() string                           { return "[" + string(x.name) + "]" }
+func (x placeholderExtension) TextName() string                           { return "[" + string(x.name) + "]" }
+func (x placeholderExtension) HasPresence() bool                          { return false }
+func (x placeholderExtension) HasOptionalKeyword() bool                   { return false }
+func (x placeholderExtension) IsExtension() bool                          { return true }
+func (x placeholderExtension) IsWeak() bool                               { return false }
+func (x placeholderExtension) IsPacked() bool                             { return false }
+func (x placeholderExtension) IsList() bool                               { return false }
+func (x placeholderExtension) IsMap() bool                                { return false }
+func (x placeholderExtension) MapKey() pref.FieldDescriptor               { return nil }
+func (x placeholderExtension) MapValue() pref.FieldDescriptor             { return nil }
+func (x placeholderExtension) HasDefault() bool                           { return false }
+func (x placeholderExtension) Default() pref.Value                        { return pref.Value{} }
+func (x placeholderExtension) DefaultEnumValue() pref.EnumValueDescriptor { return nil }
+func (x placeholderExtension) ContainingOneof() pref.OneofDescriptor      { return nil }
+func (x placeholderExtension) ContainingMessage() pref.MessageDescriptor  { return nil }
+func (x placeholderExtension) Enum() pref.EnumDescriptor                  { return nil }
+func (x placeholderExtension) Message() pref.MessageDescriptor            { return nil }
+func (x placeholderExtension) ProtoType(pref.FieldDescriptor)             { return }
+func (x placeholderExtension) ProtoInternal(pragma.DoNotImplement)        { return }
diff --git a/vendor/google.golang.org/protobuf/internal/impl/legacy_file.go b/vendor/google.golang.org/protobuf/internal/impl/legacy_file.go
new file mode 100644
index 0000000..9ab0910
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/legacy_file.go
@@ -0,0 +1,81 @@
+// 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 impl
+
+import (
+	"bytes"
+	"compress/gzip"
+	"io/ioutil"
+	"sync"
+
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// Every enum and message type generated by protoc-gen-go since commit 2fc053c5
+// on February 25th, 2016 has had a method to get the raw descriptor.
+// Types that were not generated by protoc-gen-go or were generated prior
+// to that version are not supported.
+//
+// The []byte returned is the encoded form of a FileDescriptorProto message
+// compressed using GZIP. The []int is the path from the top-level file
+// to the specific message or enum declaration.
+type (
+	enumV1 interface {
+		EnumDescriptor() ([]byte, []int)
+	}
+	messageV1 interface {
+		Descriptor() ([]byte, []int)
+	}
+)
+
+var legacyFileDescCache sync.Map // map[*byte]protoreflect.FileDescriptor
+
+// legacyLoadFileDesc unmarshals b as a compressed FileDescriptorProto message.
+//
+// This assumes that b is immutable and that b does not refer to part of a
+// concatenated series of GZIP files (which would require shenanigans that
+// rely on the concatenation properties of both protobufs and GZIP).
+// File descriptors generated by protoc-gen-go do not rely on that property.
+func legacyLoadFileDesc(b []byte) protoreflect.FileDescriptor {
+	// Fast-path: check whether we already have a cached file descriptor.
+	if fd, ok := legacyFileDescCache.Load(&b[0]); ok {
+		return fd.(protoreflect.FileDescriptor)
+	}
+
+	// Slow-path: decompress and unmarshal the file descriptor proto.
+	zr, err := gzip.NewReader(bytes.NewReader(b))
+	if err != nil {
+		panic(err)
+	}
+	b2, err := ioutil.ReadAll(zr)
+	if err != nil {
+		panic(err)
+	}
+
+	fd := filedesc.Builder{
+		RawDescriptor: b2,
+		FileRegistry:  resolverOnly{protoregistry.GlobalFiles}, // do not register back to global registry
+	}.Build().File
+	if fd, ok := legacyFileDescCache.LoadOrStore(&b[0], fd); ok {
+		return fd.(protoreflect.FileDescriptor)
+	}
+	return fd
+}
+
+type resolverOnly struct {
+	reg *protoregistry.Files
+}
+
+func (r resolverOnly) FindFileByPath(path string) (protoreflect.FileDescriptor, error) {
+	return r.reg.FindFileByPath(path)
+}
+func (r resolverOnly) FindDescriptorByName(name protoreflect.FullName) (protoreflect.Descriptor, error) {
+	return r.reg.FindDescriptorByName(name)
+}
+func (resolverOnly) RegisterFile(protoreflect.FileDescriptor) error {
+	return nil
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/legacy_message.go b/vendor/google.golang.org/protobuf/internal/impl/legacy_message.go
new file mode 100644
index 0000000..029feee
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/legacy_message.go
@@ -0,0 +1,565 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+	"strings"
+	"sync"
+
+	"google.golang.org/protobuf/internal/descopts"
+	ptag "google.golang.org/protobuf/internal/encoding/tag"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// legacyWrapMessage wraps v as a protoreflect.Message,
+// where v must be a *struct kind and not implement the v2 API already.
+func legacyWrapMessage(v reflect.Value) pref.Message {
+	t := v.Type()
+	if t.Kind() != reflect.Ptr || t.Elem().Kind() != reflect.Struct {
+		return aberrantMessage{v: v}
+	}
+	mt := legacyLoadMessageInfo(t, "")
+	return mt.MessageOf(v.Interface())
+}
+
+// legacyLoadMessageType dynamically loads a protoreflect.Type for t,
+// where t must be not implement the v2 API already.
+// The provided name is used if it cannot be determined from the message.
+func legacyLoadMessageType(t reflect.Type, name pref.FullName) protoreflect.MessageType {
+	if t.Kind() != reflect.Ptr || t.Elem().Kind() != reflect.Struct {
+		return aberrantMessageType{t}
+	}
+	return legacyLoadMessageInfo(t, name)
+}
+
+var legacyMessageTypeCache sync.Map // map[reflect.Type]*MessageInfo
+
+// legacyLoadMessageInfo dynamically loads a *MessageInfo for t,
+// where t must be a *struct kind and not implement the v2 API already.
+// The provided name is used if it cannot be determined from the message.
+func legacyLoadMessageInfo(t reflect.Type, name pref.FullName) *MessageInfo {
+	// Fast-path: check if a MessageInfo is cached for this concrete type.
+	if mt, ok := legacyMessageTypeCache.Load(t); ok {
+		return mt.(*MessageInfo)
+	}
+
+	// Slow-path: derive message descriptor and initialize MessageInfo.
+	mi := &MessageInfo{
+		Desc:          legacyLoadMessageDesc(t, name),
+		GoReflectType: t,
+	}
+
+	var hasMarshal, hasUnmarshal bool
+	v := reflect.Zero(t).Interface()
+	if _, hasMarshal = v.(legacyMarshaler); hasMarshal {
+		mi.methods.Marshal = legacyMarshal
+
+		// We have no way to tell whether the type's Marshal method
+		// supports deterministic serialization or not, but this
+		// preserves the v1 implementation's behavior of always
+		// calling Marshal methods when present.
+		mi.methods.Flags |= piface.SupportMarshalDeterministic
+	}
+	if _, hasUnmarshal = v.(legacyUnmarshaler); hasUnmarshal {
+		mi.methods.Unmarshal = legacyUnmarshal
+	}
+	if _, hasMerge := v.(legacyMerger); hasMerge || (hasMarshal && hasUnmarshal) {
+		mi.methods.Merge = legacyMerge
+	}
+
+	if mi, ok := legacyMessageTypeCache.LoadOrStore(t, mi); ok {
+		return mi.(*MessageInfo)
+	}
+	return mi
+}
+
+var legacyMessageDescCache sync.Map // map[reflect.Type]protoreflect.MessageDescriptor
+
+// LegacyLoadMessageDesc returns an MessageDescriptor derived from the Go type,
+// which should be a *struct kind and must not implement the v2 API already.
+//
+// This is exported for testing purposes.
+func LegacyLoadMessageDesc(t reflect.Type) pref.MessageDescriptor {
+	return legacyLoadMessageDesc(t, "")
+}
+func legacyLoadMessageDesc(t reflect.Type, name pref.FullName) pref.MessageDescriptor {
+	// Fast-path: check if a MessageDescriptor is cached for this concrete type.
+	if mi, ok := legacyMessageDescCache.Load(t); ok {
+		return mi.(pref.MessageDescriptor)
+	}
+
+	// Slow-path: initialize MessageDescriptor from the raw descriptor.
+	mv := reflect.Zero(t).Interface()
+	if _, ok := mv.(pref.ProtoMessage); ok {
+		panic(fmt.Sprintf("%v already implements proto.Message", t))
+	}
+	mdV1, ok := mv.(messageV1)
+	if !ok {
+		return aberrantLoadMessageDesc(t, name)
+	}
+
+	// If this is a dynamic message type where there isn't a 1-1 mapping between
+	// Go and protobuf types, calling the Descriptor method on the zero value of
+	// the message type isn't likely to work. If it panics, swallow the panic and
+	// continue as if the Descriptor method wasn't present.
+	b, idxs := func() ([]byte, []int) {
+		defer func() {
+			recover()
+		}()
+		return mdV1.Descriptor()
+	}()
+	if b == nil {
+		return aberrantLoadMessageDesc(t, name)
+	}
+
+	// If the Go type has no fields, then this might be a proto3 empty message
+	// from before the size cache was added. If there are any fields, check to
+	// see that at least one of them looks like something we generated.
+	if t.Elem().Kind() == reflect.Struct {
+		if nfield := t.Elem().NumField(); nfield > 0 {
+			hasProtoField := false
+			for i := 0; i < nfield; i++ {
+				f := t.Elem().Field(i)
+				if f.Tag.Get("protobuf") != "" || f.Tag.Get("protobuf_oneof") != "" || strings.HasPrefix(f.Name, "XXX_") {
+					hasProtoField = true
+					break
+				}
+			}
+			if !hasProtoField {
+				return aberrantLoadMessageDesc(t, name)
+			}
+		}
+	}
+
+	md := legacyLoadFileDesc(b).Messages().Get(idxs[0])
+	for _, i := range idxs[1:] {
+		md = md.Messages().Get(i)
+	}
+	if name != "" && md.FullName() != name {
+		panic(fmt.Sprintf("mismatching message name: got %v, want %v", md.FullName(), name))
+	}
+	if md, ok := legacyMessageDescCache.LoadOrStore(t, md); ok {
+		return md.(protoreflect.MessageDescriptor)
+	}
+	return md
+}
+
+var (
+	aberrantMessageDescLock  sync.Mutex
+	aberrantMessageDescCache map[reflect.Type]protoreflect.MessageDescriptor
+)
+
+// aberrantLoadMessageDesc returns an MessageDescriptor derived from the Go type,
+// which must not implement protoreflect.ProtoMessage or messageV1.
+//
+// This is a best-effort derivation of the message descriptor using the protobuf
+// tags on the struct fields.
+func aberrantLoadMessageDesc(t reflect.Type, name pref.FullName) pref.MessageDescriptor {
+	aberrantMessageDescLock.Lock()
+	defer aberrantMessageDescLock.Unlock()
+	if aberrantMessageDescCache == nil {
+		aberrantMessageDescCache = make(map[reflect.Type]protoreflect.MessageDescriptor)
+	}
+	return aberrantLoadMessageDescReentrant(t, name)
+}
+func aberrantLoadMessageDescReentrant(t reflect.Type, name pref.FullName) pref.MessageDescriptor {
+	// Fast-path: check if an MessageDescriptor is cached for this concrete type.
+	if md, ok := aberrantMessageDescCache[t]; ok {
+		return md
+	}
+
+	// Slow-path: construct a descriptor from the Go struct type (best-effort).
+	// Cache the MessageDescriptor early on so that we can resolve internal
+	// cyclic references.
+	md := &filedesc.Message{L2: new(filedesc.MessageL2)}
+	md.L0.FullName = aberrantDeriveMessageName(t, name)
+	md.L0.ParentFile = filedesc.SurrogateProto2
+	aberrantMessageDescCache[t] = md
+
+	if t.Kind() != reflect.Ptr || t.Elem().Kind() != reflect.Struct {
+		return md
+	}
+
+	// Try to determine if the message is using proto3 by checking scalars.
+	for i := 0; i < t.Elem().NumField(); i++ {
+		f := t.Elem().Field(i)
+		if tag := f.Tag.Get("protobuf"); tag != "" {
+			switch f.Type.Kind() {
+			case reflect.Bool, reflect.Int32, reflect.Int64, reflect.Uint32, reflect.Uint64, reflect.Float32, reflect.Float64, reflect.String:
+				md.L0.ParentFile = filedesc.SurrogateProto3
+			}
+			for _, s := range strings.Split(tag, ",") {
+				if s == "proto3" {
+					md.L0.ParentFile = filedesc.SurrogateProto3
+				}
+			}
+		}
+	}
+
+	// Obtain a list of oneof wrapper types.
+	var oneofWrappers []reflect.Type
+	for _, method := range []string{"XXX_OneofFuncs", "XXX_OneofWrappers"} {
+		if fn, ok := t.MethodByName(method); ok {
+			for _, v := range fn.Func.Call([]reflect.Value{reflect.Zero(fn.Type.In(0))}) {
+				if vs, ok := v.Interface().([]interface{}); ok {
+					for _, v := range vs {
+						oneofWrappers = append(oneofWrappers, reflect.TypeOf(v))
+					}
+				}
+			}
+		}
+	}
+
+	// Obtain a list of the extension ranges.
+	if fn, ok := t.MethodByName("ExtensionRangeArray"); ok {
+		vs := fn.Func.Call([]reflect.Value{reflect.Zero(fn.Type.In(0))})[0]
+		for i := 0; i < vs.Len(); i++ {
+			v := vs.Index(i)
+			md.L2.ExtensionRanges.List = append(md.L2.ExtensionRanges.List, [2]pref.FieldNumber{
+				pref.FieldNumber(v.FieldByName("Start").Int()),
+				pref.FieldNumber(v.FieldByName("End").Int() + 1),
+			})
+			md.L2.ExtensionRangeOptions = append(md.L2.ExtensionRangeOptions, nil)
+		}
+	}
+
+	// Derive the message fields by inspecting the struct fields.
+	for i := 0; i < t.Elem().NumField(); i++ {
+		f := t.Elem().Field(i)
+		if tag := f.Tag.Get("protobuf"); tag != "" {
+			tagKey := f.Tag.Get("protobuf_key")
+			tagVal := f.Tag.Get("protobuf_val")
+			aberrantAppendField(md, f.Type, tag, tagKey, tagVal)
+		}
+		if tag := f.Tag.Get("protobuf_oneof"); tag != "" {
+			n := len(md.L2.Oneofs.List)
+			md.L2.Oneofs.List = append(md.L2.Oneofs.List, filedesc.Oneof{})
+			od := &md.L2.Oneofs.List[n]
+			od.L0.FullName = md.FullName().Append(pref.Name(tag))
+			od.L0.ParentFile = md.L0.ParentFile
+			od.L0.Parent = md
+			od.L0.Index = n
+
+			for _, t := range oneofWrappers {
+				if t.Implements(f.Type) {
+					f := t.Elem().Field(0)
+					if tag := f.Tag.Get("protobuf"); tag != "" {
+						aberrantAppendField(md, f.Type, tag, "", "")
+						fd := &md.L2.Fields.List[len(md.L2.Fields.List)-1]
+						fd.L1.ContainingOneof = od
+						od.L1.Fields.List = append(od.L1.Fields.List, fd)
+					}
+				}
+			}
+		}
+	}
+
+	return md
+}
+
+func aberrantDeriveMessageName(t reflect.Type, name pref.FullName) pref.FullName {
+	if name.IsValid() {
+		return name
+	}
+	func() {
+		defer func() { recover() }() // swallow possible nil panics
+		if m, ok := reflect.Zero(t).Interface().(interface{ XXX_MessageName() string }); ok {
+			name = pref.FullName(m.XXX_MessageName())
+		}
+	}()
+	if name.IsValid() {
+		return name
+	}
+	if t.Kind() == reflect.Ptr {
+		t = t.Elem()
+	}
+	return AberrantDeriveFullName(t)
+}
+
+func aberrantAppendField(md *filedesc.Message, goType reflect.Type, tag, tagKey, tagVal string) {
+	t := goType
+	isOptional := t.Kind() == reflect.Ptr && t.Elem().Kind() != reflect.Struct
+	isRepeated := t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8
+	if isOptional || isRepeated {
+		t = t.Elem()
+	}
+	fd := ptag.Unmarshal(tag, t, placeholderEnumValues{}).(*filedesc.Field)
+
+	// Append field descriptor to the message.
+	n := len(md.L2.Fields.List)
+	md.L2.Fields.List = append(md.L2.Fields.List, *fd)
+	fd = &md.L2.Fields.List[n]
+	fd.L0.FullName = md.FullName().Append(fd.Name())
+	fd.L0.ParentFile = md.L0.ParentFile
+	fd.L0.Parent = md
+	fd.L0.Index = n
+
+	if fd.L1.IsWeak || fd.L1.HasPacked {
+		fd.L1.Options = func() pref.ProtoMessage {
+			opts := descopts.Field.ProtoReflect().New()
+			if fd.L1.IsWeak {
+				opts.Set(opts.Descriptor().Fields().ByName("weak"), protoreflect.ValueOfBool(true))
+			}
+			if fd.L1.HasPacked {
+				opts.Set(opts.Descriptor().Fields().ByName("packed"), protoreflect.ValueOfBool(fd.L1.IsPacked))
+			}
+			return opts.Interface()
+		}
+	}
+
+	// Populate Enum and Message.
+	if fd.Enum() == nil && fd.Kind() == pref.EnumKind {
+		switch v := reflect.Zero(t).Interface().(type) {
+		case pref.Enum:
+			fd.L1.Enum = v.Descriptor()
+		default:
+			fd.L1.Enum = LegacyLoadEnumDesc(t)
+		}
+	}
+	if fd.Message() == nil && (fd.Kind() == pref.MessageKind || fd.Kind() == pref.GroupKind) {
+		switch v := reflect.Zero(t).Interface().(type) {
+		case pref.ProtoMessage:
+			fd.L1.Message = v.ProtoReflect().Descriptor()
+		case messageV1:
+			fd.L1.Message = LegacyLoadMessageDesc(t)
+		default:
+			if t.Kind() == reflect.Map {
+				n := len(md.L1.Messages.List)
+				md.L1.Messages.List = append(md.L1.Messages.List, filedesc.Message{L2: new(filedesc.MessageL2)})
+				md2 := &md.L1.Messages.List[n]
+				md2.L0.FullName = md.FullName().Append(pref.Name(strs.MapEntryName(string(fd.Name()))))
+				md2.L0.ParentFile = md.L0.ParentFile
+				md2.L0.Parent = md
+				md2.L0.Index = n
+
+				md2.L1.IsMapEntry = true
+				md2.L2.Options = func() pref.ProtoMessage {
+					opts := descopts.Message.ProtoReflect().New()
+					opts.Set(opts.Descriptor().Fields().ByName("map_entry"), protoreflect.ValueOfBool(true))
+					return opts.Interface()
+				}
+
+				aberrantAppendField(md2, t.Key(), tagKey, "", "")
+				aberrantAppendField(md2, t.Elem(), tagVal, "", "")
+
+				fd.L1.Message = md2
+				break
+			}
+			fd.L1.Message = aberrantLoadMessageDescReentrant(t, "")
+		}
+	}
+}
+
+type placeholderEnumValues struct {
+	protoreflect.EnumValueDescriptors
+}
+
+func (placeholderEnumValues) ByNumber(n pref.EnumNumber) pref.EnumValueDescriptor {
+	return filedesc.PlaceholderEnumValue(pref.FullName(fmt.Sprintf("UNKNOWN_%d", n)))
+}
+
+// legacyMarshaler is the proto.Marshaler interface superseded by protoiface.Methoder.
+type legacyMarshaler interface {
+	Marshal() ([]byte, error)
+}
+
+// legacyUnmarshaler is the proto.Unmarshaler interface superseded by protoiface.Methoder.
+type legacyUnmarshaler interface {
+	Unmarshal([]byte) error
+}
+
+// legacyMerger is the proto.Merger interface superseded by protoiface.Methoder.
+type legacyMerger interface {
+	Merge(protoiface.MessageV1)
+}
+
+var aberrantProtoMethods = &piface.Methods{
+	Marshal:   legacyMarshal,
+	Unmarshal: legacyUnmarshal,
+	Merge:     legacyMerge,
+
+	// We have no way to tell whether the type's Marshal method
+	// supports deterministic serialization or not, but this
+	// preserves the v1 implementation's behavior of always
+	// calling Marshal methods when present.
+	Flags: piface.SupportMarshalDeterministic,
+}
+
+func legacyMarshal(in piface.MarshalInput) (piface.MarshalOutput, error) {
+	v := in.Message.(unwrapper).protoUnwrap()
+	marshaler, ok := v.(legacyMarshaler)
+	if !ok {
+		return piface.MarshalOutput{}, errors.New("%T does not implement Marshal", v)
+	}
+	out, err := marshaler.Marshal()
+	if in.Buf != nil {
+		out = append(in.Buf, out...)
+	}
+	return piface.MarshalOutput{
+		Buf: out,
+	}, err
+}
+
+func legacyUnmarshal(in piface.UnmarshalInput) (piface.UnmarshalOutput, error) {
+	v := in.Message.(unwrapper).protoUnwrap()
+	unmarshaler, ok := v.(legacyUnmarshaler)
+	if !ok {
+		return piface.UnmarshalOutput{}, errors.New("%T does not implement Unmarshal", v)
+	}
+	return piface.UnmarshalOutput{}, unmarshaler.Unmarshal(in.Buf)
+}
+
+func legacyMerge(in piface.MergeInput) piface.MergeOutput {
+	// Check whether this supports the legacy merger.
+	dstv := in.Destination.(unwrapper).protoUnwrap()
+	merger, ok := dstv.(legacyMerger)
+	if ok {
+		merger.Merge(Export{}.ProtoMessageV1Of(in.Source))
+		return piface.MergeOutput{Flags: piface.MergeComplete}
+	}
+
+	// If legacy merger is unavailable, implement merge in terms of
+	// a marshal and unmarshal operation.
+	srcv := in.Source.(unwrapper).protoUnwrap()
+	marshaler, ok := srcv.(legacyMarshaler)
+	if !ok {
+		return piface.MergeOutput{}
+	}
+	dstv = in.Destination.(unwrapper).protoUnwrap()
+	unmarshaler, ok := dstv.(legacyUnmarshaler)
+	if !ok {
+		return piface.MergeOutput{}
+	}
+	if !in.Source.IsValid() {
+		// Legacy Marshal methods may not function on nil messages.
+		// Check for a typed nil source only after we confirm that
+		// legacy Marshal/Unmarshal methods are present, for
+		// consistency.
+		return piface.MergeOutput{Flags: piface.MergeComplete}
+	}
+	b, err := marshaler.Marshal()
+	if err != nil {
+		return piface.MergeOutput{}
+	}
+	err = unmarshaler.Unmarshal(b)
+	if err != nil {
+		return piface.MergeOutput{}
+	}
+	return piface.MergeOutput{Flags: piface.MergeComplete}
+}
+
+// aberrantMessageType implements MessageType for all types other than pointer-to-struct.
+type aberrantMessageType struct {
+	t reflect.Type
+}
+
+func (mt aberrantMessageType) New() pref.Message {
+	if mt.t.Kind() == reflect.Ptr {
+		return aberrantMessage{reflect.New(mt.t.Elem())}
+	}
+	return aberrantMessage{reflect.Zero(mt.t)}
+}
+func (mt aberrantMessageType) Zero() pref.Message {
+	return aberrantMessage{reflect.Zero(mt.t)}
+}
+func (mt aberrantMessageType) GoType() reflect.Type {
+	return mt.t
+}
+func (mt aberrantMessageType) Descriptor() pref.MessageDescriptor {
+	return LegacyLoadMessageDesc(mt.t)
+}
+
+// aberrantMessage implements Message for all types other than pointer-to-struct.
+//
+// When the underlying type implements legacyMarshaler or legacyUnmarshaler,
+// the aberrant Message can be marshaled or unmarshaled. Otherwise, there is
+// not much that can be done with values of this type.
+type aberrantMessage struct {
+	v reflect.Value
+}
+
+// Reset implements the v1 proto.Message.Reset method.
+func (m aberrantMessage) Reset() {
+	if mr, ok := m.v.Interface().(interface{ Reset() }); ok {
+		mr.Reset()
+		return
+	}
+	if m.v.Kind() == reflect.Ptr && !m.v.IsNil() {
+		m.v.Elem().Set(reflect.Zero(m.v.Type().Elem()))
+	}
+}
+
+func (m aberrantMessage) ProtoReflect() pref.Message {
+	return m
+}
+
+func (m aberrantMessage) Descriptor() pref.MessageDescriptor {
+	return LegacyLoadMessageDesc(m.v.Type())
+}
+func (m aberrantMessage) Type() pref.MessageType {
+	return aberrantMessageType{m.v.Type()}
+}
+func (m aberrantMessage) New() pref.Message {
+	if m.v.Type().Kind() == reflect.Ptr {
+		return aberrantMessage{reflect.New(m.v.Type().Elem())}
+	}
+	return aberrantMessage{reflect.Zero(m.v.Type())}
+}
+func (m aberrantMessage) Interface() pref.ProtoMessage {
+	return m
+}
+func (m aberrantMessage) Range(f func(pref.FieldDescriptor, pref.Value) bool) {
+	return
+}
+func (m aberrantMessage) Has(pref.FieldDescriptor) bool {
+	return false
+}
+func (m aberrantMessage) Clear(pref.FieldDescriptor) {
+	panic("invalid Message.Clear on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) Get(fd pref.FieldDescriptor) pref.Value {
+	if fd.Default().IsValid() {
+		return fd.Default()
+	}
+	panic("invalid Message.Get on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) Set(pref.FieldDescriptor, pref.Value) {
+	panic("invalid Message.Set on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) Mutable(pref.FieldDescriptor) pref.Value {
+	panic("invalid Message.Mutable on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) NewField(pref.FieldDescriptor) pref.Value {
+	panic("invalid Message.NewField on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) WhichOneof(pref.OneofDescriptor) pref.FieldDescriptor {
+	panic("invalid Message.WhichOneof descriptor on " + string(m.Descriptor().FullName()))
+}
+func (m aberrantMessage) GetUnknown() pref.RawFields {
+	return nil
+}
+func (m aberrantMessage) SetUnknown(pref.RawFields) {
+	// SetUnknown discards its input on messages which don't support unknown field storage.
+}
+func (m aberrantMessage) IsValid() bool {
+	if m.v.Kind() == reflect.Ptr {
+		return !m.v.IsNil()
+	}
+	return false
+}
+func (m aberrantMessage) ProtoMethods() *piface.Methods {
+	return aberrantProtoMethods
+}
+func (m aberrantMessage) protoUnwrap() interface{} {
+	return m.v.Interface()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/merge.go b/vendor/google.golang.org/protobuf/internal/impl/merge.go
new file mode 100644
index 0000000..c65bbc0
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/merge.go
@@ -0,0 +1,176 @@
+// Copyright 2020 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 impl
+
+import (
+	"fmt"
+	"reflect"
+
+	"google.golang.org/protobuf/proto"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+type mergeOptions struct{}
+
+func (o mergeOptions) Merge(dst, src proto.Message) {
+	proto.Merge(dst, src)
+}
+
+// merge is protoreflect.Methods.Merge.
+func (mi *MessageInfo) merge(in piface.MergeInput) piface.MergeOutput {
+	dp, ok := mi.getPointer(in.Destination)
+	if !ok {
+		return piface.MergeOutput{}
+	}
+	sp, ok := mi.getPointer(in.Source)
+	if !ok {
+		return piface.MergeOutput{}
+	}
+	mi.mergePointer(dp, sp, mergeOptions{})
+	return piface.MergeOutput{Flags: piface.MergeComplete}
+}
+
+func (mi *MessageInfo) mergePointer(dst, src pointer, opts mergeOptions) {
+	mi.init()
+	if dst.IsNil() {
+		panic(fmt.Sprintf("invalid value: merging into nil message"))
+	}
+	if src.IsNil() {
+		return
+	}
+	for _, f := range mi.orderedCoderFields {
+		if f.funcs.merge == nil {
+			continue
+		}
+		sfptr := src.Apply(f.offset)
+		if f.isPointer && sfptr.Elem().IsNil() {
+			continue
+		}
+		f.funcs.merge(dst.Apply(f.offset), sfptr, f, opts)
+	}
+	if mi.extensionOffset.IsValid() {
+		sext := src.Apply(mi.extensionOffset).Extensions()
+		dext := dst.Apply(mi.extensionOffset).Extensions()
+		if *dext == nil {
+			*dext = make(map[int32]ExtensionField)
+		}
+		for num, sx := range *sext {
+			xt := sx.Type()
+			xi := getExtensionFieldInfo(xt)
+			if xi.funcs.merge == nil {
+				continue
+			}
+			dx := (*dext)[num]
+			var dv pref.Value
+			if dx.Type() == sx.Type() {
+				dv = dx.Value()
+			}
+			if !dv.IsValid() && xi.unmarshalNeedsValue {
+				dv = xt.New()
+			}
+			dv = xi.funcs.merge(dv, sx.Value(), opts)
+			dx.Set(sx.Type(), dv)
+			(*dext)[num] = dx
+		}
+	}
+	if mi.unknownOffset.IsValid() {
+		su := mi.getUnknownBytes(src)
+		if su != nil && len(*su) > 0 {
+			du := mi.mutableUnknownBytes(dst)
+			*du = append(*du, *su...)
+		}
+	}
+}
+
+func mergeScalarValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	return src
+}
+
+func mergeBytesValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	return pref.ValueOfBytes(append(emptyBuf[:], src.Bytes()...))
+}
+
+func mergeListValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	dstl := dst.List()
+	srcl := src.List()
+	for i, llen := 0, srcl.Len(); i < llen; i++ {
+		dstl.Append(srcl.Get(i))
+	}
+	return dst
+}
+
+func mergeBytesListValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	dstl := dst.List()
+	srcl := src.List()
+	for i, llen := 0, srcl.Len(); i < llen; i++ {
+		sb := srcl.Get(i).Bytes()
+		db := append(emptyBuf[:], sb...)
+		dstl.Append(pref.ValueOfBytes(db))
+	}
+	return dst
+}
+
+func mergeMessageListValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	dstl := dst.List()
+	srcl := src.List()
+	for i, llen := 0, srcl.Len(); i < llen; i++ {
+		sm := srcl.Get(i).Message()
+		dm := proto.Clone(sm.Interface()).ProtoReflect()
+		dstl.Append(pref.ValueOfMessage(dm))
+	}
+	return dst
+}
+
+func mergeMessageValue(dst, src pref.Value, opts mergeOptions) pref.Value {
+	opts.Merge(dst.Message().Interface(), src.Message().Interface())
+	return dst
+}
+
+func mergeMessage(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+	if f.mi != nil {
+		if dst.Elem().IsNil() {
+			dst.SetPointer(pointerOfValue(reflect.New(f.mi.GoReflectType.Elem())))
+		}
+		f.mi.mergePointer(dst.Elem(), src.Elem(), opts)
+	} else {
+		dm := dst.AsValueOf(f.ft).Elem()
+		sm := src.AsValueOf(f.ft).Elem()
+		if dm.IsNil() {
+			dm.Set(reflect.New(f.ft.Elem()))
+		}
+		opts.Merge(asMessage(dm), asMessage(sm))
+	}
+}
+
+func mergeMessageSlice(dst, src pointer, f *coderFieldInfo, opts mergeOptions) {
+	for _, sp := range src.PointerSlice() {
+		dm := reflect.New(f.ft.Elem().Elem())
+		if f.mi != nil {
+			f.mi.mergePointer(pointerOfValue(dm), sp, opts)
+		} else {
+			opts.Merge(asMessage(dm), asMessage(sp.AsValueOf(f.ft.Elem().Elem())))
+		}
+		dst.AppendPointerSlice(pointerOfValue(dm))
+	}
+}
+
+func mergeBytes(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Bytes() = append(emptyBuf[:], *src.Bytes()...)
+}
+
+func mergeBytesNoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Bytes()
+	if len(v) > 0 {
+		*dst.Bytes() = append(emptyBuf[:], v...)
+	}
+}
+
+func mergeBytesSlice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.BytesSlice()
+	for _, v := range *src.BytesSlice() {
+		*ds = append(*ds, append(emptyBuf[:], v...))
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/merge_gen.go b/vendor/google.golang.org/protobuf/internal/impl/merge_gen.go
new file mode 100644
index 0000000..8816c27
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/merge_gen.go
@@ -0,0 +1,209 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package impl
+
+import ()
+
+func mergeBool(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Bool() = *src.Bool()
+}
+
+func mergeBoolNoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Bool()
+	if v != false {
+		*dst.Bool() = v
+	}
+}
+
+func mergeBoolPtr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.BoolPtr()
+	if p != nil {
+		v := *p
+		*dst.BoolPtr() = &v
+	}
+}
+
+func mergeBoolSlice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.BoolSlice()
+	ss := src.BoolSlice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeInt32(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Int32() = *src.Int32()
+}
+
+func mergeInt32NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Int32()
+	if v != 0 {
+		*dst.Int32() = v
+	}
+}
+
+func mergeInt32Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Int32Ptr()
+	if p != nil {
+		v := *p
+		*dst.Int32Ptr() = &v
+	}
+}
+
+func mergeInt32Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Int32Slice()
+	ss := src.Int32Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeUint32(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Uint32() = *src.Uint32()
+}
+
+func mergeUint32NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Uint32()
+	if v != 0 {
+		*dst.Uint32() = v
+	}
+}
+
+func mergeUint32Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Uint32Ptr()
+	if p != nil {
+		v := *p
+		*dst.Uint32Ptr() = &v
+	}
+}
+
+func mergeUint32Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Uint32Slice()
+	ss := src.Uint32Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeInt64(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Int64() = *src.Int64()
+}
+
+func mergeInt64NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Int64()
+	if v != 0 {
+		*dst.Int64() = v
+	}
+}
+
+func mergeInt64Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Int64Ptr()
+	if p != nil {
+		v := *p
+		*dst.Int64Ptr() = &v
+	}
+}
+
+func mergeInt64Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Int64Slice()
+	ss := src.Int64Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeUint64(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Uint64() = *src.Uint64()
+}
+
+func mergeUint64NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Uint64()
+	if v != 0 {
+		*dst.Uint64() = v
+	}
+}
+
+func mergeUint64Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Uint64Ptr()
+	if p != nil {
+		v := *p
+		*dst.Uint64Ptr() = &v
+	}
+}
+
+func mergeUint64Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Uint64Slice()
+	ss := src.Uint64Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeFloat32(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Float32() = *src.Float32()
+}
+
+func mergeFloat32NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Float32()
+	if v != 0 {
+		*dst.Float32() = v
+	}
+}
+
+func mergeFloat32Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Float32Ptr()
+	if p != nil {
+		v := *p
+		*dst.Float32Ptr() = &v
+	}
+}
+
+func mergeFloat32Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Float32Slice()
+	ss := src.Float32Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeFloat64(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.Float64() = *src.Float64()
+}
+
+func mergeFloat64NoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.Float64()
+	if v != 0 {
+		*dst.Float64() = v
+	}
+}
+
+func mergeFloat64Ptr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.Float64Ptr()
+	if p != nil {
+		v := *p
+		*dst.Float64Ptr() = &v
+	}
+}
+
+func mergeFloat64Slice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.Float64Slice()
+	ss := src.Float64Slice()
+	*ds = append(*ds, *ss...)
+}
+
+func mergeString(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	*dst.String() = *src.String()
+}
+
+func mergeStringNoZero(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	v := *src.String()
+	if v != "" {
+		*dst.String() = v
+	}
+}
+
+func mergeStringPtr(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	p := *src.StringPtr()
+	if p != nil {
+		v := *p
+		*dst.StringPtr() = &v
+	}
+}
+
+func mergeStringSlice(dst, src pointer, _ *coderFieldInfo, _ mergeOptions) {
+	ds := dst.StringSlice()
+	ss := src.StringSlice()
+	*ds = append(*ds, *ss...)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/message.go b/vendor/google.golang.org/protobuf/internal/impl/message.go
new file mode 100644
index 0000000..a104e28
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/message.go
@@ -0,0 +1,276 @@
+// 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 impl
+
+import (
+	"fmt"
+	"reflect"
+	"strconv"
+	"strings"
+	"sync"
+	"sync/atomic"
+
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// MessageInfo provides protobuf related functionality for a given Go type
+// that represents a message. A given instance of MessageInfo is tied to
+// exactly one Go type, which must be a pointer to a struct type.
+//
+// The exported fields must be populated before any methods are called
+// and cannot be mutated after set.
+type MessageInfo struct {
+	// GoReflectType is the underlying message Go type and must be populated.
+	GoReflectType reflect.Type // pointer to struct
+
+	// Desc is the underlying message descriptor type and must be populated.
+	Desc pref.MessageDescriptor
+
+	// Exporter must be provided in a purego environment in order to provide
+	// access to unexported fields.
+	Exporter exporter
+
+	// OneofWrappers is list of pointers to oneof wrapper struct types.
+	OneofWrappers []interface{}
+
+	initMu   sync.Mutex // protects all unexported fields
+	initDone uint32
+
+	reflectMessageInfo // for reflection implementation
+	coderMessageInfo   // for fast-path method implementations
+}
+
+// exporter is a function that returns a reference to the ith field of v,
+// where v is a pointer to a struct. It returns nil if it does not support
+// exporting the requested field (e.g., already exported).
+type exporter func(v interface{}, i int) interface{}
+
+// getMessageInfo returns the MessageInfo for any message type that
+// is generated by our implementation of protoc-gen-go (for v2 and on).
+// If it is unable to obtain a MessageInfo, it returns nil.
+func getMessageInfo(mt reflect.Type) *MessageInfo {
+	m, ok := reflect.Zero(mt).Interface().(pref.ProtoMessage)
+	if !ok {
+		return nil
+	}
+	mr, ok := m.ProtoReflect().(interface{ ProtoMessageInfo() *MessageInfo })
+	if !ok {
+		return nil
+	}
+	return mr.ProtoMessageInfo()
+}
+
+func (mi *MessageInfo) init() {
+	// This function is called in the hot path. Inline the sync.Once logic,
+	// since allocating a closure for Once.Do is expensive.
+	// Keep init small to ensure that it can be inlined.
+	if atomic.LoadUint32(&mi.initDone) == 0 {
+		mi.initOnce()
+	}
+}
+
+func (mi *MessageInfo) initOnce() {
+	mi.initMu.Lock()
+	defer mi.initMu.Unlock()
+	if mi.initDone == 1 {
+		return
+	}
+
+	t := mi.GoReflectType
+	if t.Kind() != reflect.Ptr && t.Elem().Kind() != reflect.Struct {
+		panic(fmt.Sprintf("got %v, want *struct kind", t))
+	}
+	t = t.Elem()
+
+	si := mi.makeStructInfo(t)
+	mi.makeReflectFuncs(t, si)
+	mi.makeCoderMethods(t, si)
+
+	atomic.StoreUint32(&mi.initDone, 1)
+}
+
+// getPointer returns the pointer for a message, which should be of
+// the type of the MessageInfo. If the message is of a different type,
+// it returns ok==false.
+func (mi *MessageInfo) getPointer(m pref.Message) (p pointer, ok bool) {
+	switch m := m.(type) {
+	case *messageState:
+		return m.pointer(), m.messageInfo() == mi
+	case *messageReflectWrapper:
+		return m.pointer(), m.messageInfo() == mi
+	}
+	return pointer{}, false
+}
+
+type (
+	SizeCache       = int32
+	WeakFields      = map[int32]protoreflect.ProtoMessage
+	UnknownFields   = unknownFieldsA // TODO: switch to unknownFieldsB
+	unknownFieldsA  = []byte
+	unknownFieldsB  = *[]byte
+	ExtensionFields = map[int32]ExtensionField
+)
+
+var (
+	sizecacheType       = reflect.TypeOf(SizeCache(0))
+	weakFieldsType      = reflect.TypeOf(WeakFields(nil))
+	unknownFieldsAType  = reflect.TypeOf(unknownFieldsA(nil))
+	unknownFieldsBType  = reflect.TypeOf(unknownFieldsB(nil))
+	extensionFieldsType = reflect.TypeOf(ExtensionFields(nil))
+)
+
+type structInfo struct {
+	sizecacheOffset offset
+	sizecacheType   reflect.Type
+	weakOffset      offset
+	weakType        reflect.Type
+	unknownOffset   offset
+	unknownType     reflect.Type
+	extensionOffset offset
+	extensionType   reflect.Type
+
+	fieldsByNumber        map[pref.FieldNumber]reflect.StructField
+	oneofsByName          map[pref.Name]reflect.StructField
+	oneofWrappersByType   map[reflect.Type]pref.FieldNumber
+	oneofWrappersByNumber map[pref.FieldNumber]reflect.Type
+}
+
+func (mi *MessageInfo) makeStructInfo(t reflect.Type) structInfo {
+	si := structInfo{
+		sizecacheOffset: invalidOffset,
+		weakOffset:      invalidOffset,
+		unknownOffset:   invalidOffset,
+		extensionOffset: invalidOffset,
+
+		fieldsByNumber:        map[pref.FieldNumber]reflect.StructField{},
+		oneofsByName:          map[pref.Name]reflect.StructField{},
+		oneofWrappersByType:   map[reflect.Type]pref.FieldNumber{},
+		oneofWrappersByNumber: map[pref.FieldNumber]reflect.Type{},
+	}
+
+fieldLoop:
+	for i := 0; i < t.NumField(); i++ {
+		switch f := t.Field(i); f.Name {
+		case genid.SizeCache_goname, genid.SizeCacheA_goname:
+			if f.Type == sizecacheType {
+				si.sizecacheOffset = offsetOf(f, mi.Exporter)
+				si.sizecacheType = f.Type
+			}
+		case genid.WeakFields_goname, genid.WeakFieldsA_goname:
+			if f.Type == weakFieldsType {
+				si.weakOffset = offsetOf(f, mi.Exporter)
+				si.weakType = f.Type
+			}
+		case genid.UnknownFields_goname, genid.UnknownFieldsA_goname:
+			if f.Type == unknownFieldsAType || f.Type == unknownFieldsBType {
+				si.unknownOffset = offsetOf(f, mi.Exporter)
+				si.unknownType = f.Type
+			}
+		case genid.ExtensionFields_goname, genid.ExtensionFieldsA_goname, genid.ExtensionFieldsB_goname:
+			if f.Type == extensionFieldsType {
+				si.extensionOffset = offsetOf(f, mi.Exporter)
+				si.extensionType = f.Type
+			}
+		default:
+			for _, s := range strings.Split(f.Tag.Get("protobuf"), ",") {
+				if len(s) > 0 && strings.Trim(s, "0123456789") == "" {
+					n, _ := strconv.ParseUint(s, 10, 64)
+					si.fieldsByNumber[pref.FieldNumber(n)] = f
+					continue fieldLoop
+				}
+			}
+			if s := f.Tag.Get("protobuf_oneof"); len(s) > 0 {
+				si.oneofsByName[pref.Name(s)] = f
+				continue fieldLoop
+			}
+		}
+	}
+
+	// Derive a mapping of oneof wrappers to fields.
+	oneofWrappers := mi.OneofWrappers
+	for _, method := range []string{"XXX_OneofFuncs", "XXX_OneofWrappers"} {
+		if fn, ok := reflect.PtrTo(t).MethodByName(method); ok {
+			for _, v := range fn.Func.Call([]reflect.Value{reflect.Zero(fn.Type.In(0))}) {
+				if vs, ok := v.Interface().([]interface{}); ok {
+					oneofWrappers = vs
+				}
+			}
+		}
+	}
+	for _, v := range oneofWrappers {
+		tf := reflect.TypeOf(v).Elem()
+		f := tf.Field(0)
+		for _, s := range strings.Split(f.Tag.Get("protobuf"), ",") {
+			if len(s) > 0 && strings.Trim(s, "0123456789") == "" {
+				n, _ := strconv.ParseUint(s, 10, 64)
+				si.oneofWrappersByType[tf] = pref.FieldNumber(n)
+				si.oneofWrappersByNumber[pref.FieldNumber(n)] = tf
+				break
+			}
+		}
+	}
+
+	return si
+}
+
+func (mi *MessageInfo) New() protoreflect.Message {
+	return mi.MessageOf(reflect.New(mi.GoReflectType.Elem()).Interface())
+}
+func (mi *MessageInfo) Zero() protoreflect.Message {
+	return mi.MessageOf(reflect.Zero(mi.GoReflectType).Interface())
+}
+func (mi *MessageInfo) Descriptor() protoreflect.MessageDescriptor {
+	return mi.Desc
+}
+func (mi *MessageInfo) Enum(i int) protoreflect.EnumType {
+	mi.init()
+	fd := mi.Desc.Fields().Get(i)
+	return Export{}.EnumTypeOf(mi.fieldTypes[fd.Number()])
+}
+func (mi *MessageInfo) Message(i int) protoreflect.MessageType {
+	mi.init()
+	fd := mi.Desc.Fields().Get(i)
+	switch {
+	case fd.IsWeak():
+		mt, _ := preg.GlobalTypes.FindMessageByName(fd.Message().FullName())
+		return mt
+	case fd.IsMap():
+		return mapEntryType{fd.Message(), mi.fieldTypes[fd.Number()]}
+	default:
+		return Export{}.MessageTypeOf(mi.fieldTypes[fd.Number()])
+	}
+}
+
+type mapEntryType struct {
+	desc    protoreflect.MessageDescriptor
+	valType interface{} // zero value of enum or message type
+}
+
+func (mt mapEntryType) New() protoreflect.Message {
+	return nil
+}
+func (mt mapEntryType) Zero() protoreflect.Message {
+	return nil
+}
+func (mt mapEntryType) Descriptor() protoreflect.MessageDescriptor {
+	return mt.desc
+}
+func (mt mapEntryType) Enum(i int) protoreflect.EnumType {
+	fd := mt.desc.Fields().Get(i)
+	if fd.Enum() == nil {
+		return nil
+	}
+	return Export{}.EnumTypeOf(mt.valType)
+}
+func (mt mapEntryType) Message(i int) protoreflect.MessageType {
+	fd := mt.desc.Fields().Get(i)
+	if fd.Message() == nil {
+		return nil
+	}
+	return Export{}.MessageTypeOf(mt.valType)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/message_reflect.go b/vendor/google.golang.org/protobuf/internal/impl/message_reflect.go
new file mode 100644
index 0000000..9488b72
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/message_reflect.go
@@ -0,0 +1,465 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+	"reflect"
+
+	"google.golang.org/protobuf/internal/detrand"
+	"google.golang.org/protobuf/internal/pragma"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type reflectMessageInfo struct {
+	fields map[pref.FieldNumber]*fieldInfo
+	oneofs map[pref.Name]*oneofInfo
+
+	// fieldTypes contains the zero value of an enum or message field.
+	// For lists, it contains the element type.
+	// For maps, it contains the entry value type.
+	fieldTypes map[pref.FieldNumber]interface{}
+
+	// denseFields is a subset of fields where:
+	//	0 < fieldDesc.Number() < len(denseFields)
+	// It provides faster access to the fieldInfo, but may be incomplete.
+	denseFields []*fieldInfo
+
+	// rangeInfos is a list of all fields (not belonging to a oneof) and oneofs.
+	rangeInfos []interface{} // either *fieldInfo or *oneofInfo
+
+	getUnknown   func(pointer) pref.RawFields
+	setUnknown   func(pointer, pref.RawFields)
+	extensionMap func(pointer) *extensionMap
+
+	nilMessage atomicNilMessage
+}
+
+// makeReflectFuncs generates the set of functions to support reflection.
+func (mi *MessageInfo) makeReflectFuncs(t reflect.Type, si structInfo) {
+	mi.makeKnownFieldsFunc(si)
+	mi.makeUnknownFieldsFunc(t, si)
+	mi.makeExtensionFieldsFunc(t, si)
+	mi.makeFieldTypes(si)
+}
+
+// makeKnownFieldsFunc generates functions for operations that can be performed
+// on each protobuf message field. It takes in a reflect.Type representing the
+// Go struct and matches message fields with struct fields.
+//
+// This code assumes that the struct is well-formed and panics if there are
+// any discrepancies.
+func (mi *MessageInfo) makeKnownFieldsFunc(si structInfo) {
+	mi.fields = map[pref.FieldNumber]*fieldInfo{}
+	md := mi.Desc
+	fds := md.Fields()
+	for i := 0; i < fds.Len(); i++ {
+		fd := fds.Get(i)
+		fs := si.fieldsByNumber[fd.Number()]
+		isOneof := fd.ContainingOneof() != nil && !fd.ContainingOneof().IsSynthetic()
+		if isOneof {
+			fs = si.oneofsByName[fd.ContainingOneof().Name()]
+		}
+		var fi fieldInfo
+		switch {
+		case fs.Type == nil:
+			fi = fieldInfoForMissing(fd) // never occurs for officially generated message types
+		case isOneof:
+			fi = fieldInfoForOneof(fd, fs, mi.Exporter, si.oneofWrappersByNumber[fd.Number()])
+		case fd.IsMap():
+			fi = fieldInfoForMap(fd, fs, mi.Exporter)
+		case fd.IsList():
+			fi = fieldInfoForList(fd, fs, mi.Exporter)
+		case fd.IsWeak():
+			fi = fieldInfoForWeakMessage(fd, si.weakOffset)
+		case fd.Message() != nil:
+			fi = fieldInfoForMessage(fd, fs, mi.Exporter)
+		default:
+			fi = fieldInfoForScalar(fd, fs, mi.Exporter)
+		}
+		mi.fields[fd.Number()] = &fi
+	}
+
+	mi.oneofs = map[pref.Name]*oneofInfo{}
+	for i := 0; i < md.Oneofs().Len(); i++ {
+		od := md.Oneofs().Get(i)
+		mi.oneofs[od.Name()] = makeOneofInfo(od, si, mi.Exporter)
+	}
+
+	mi.denseFields = make([]*fieldInfo, fds.Len()*2)
+	for i := 0; i < fds.Len(); i++ {
+		if fd := fds.Get(i); int(fd.Number()) < len(mi.denseFields) {
+			mi.denseFields[fd.Number()] = mi.fields[fd.Number()]
+		}
+	}
+
+	for i := 0; i < fds.Len(); {
+		fd := fds.Get(i)
+		if od := fd.ContainingOneof(); od != nil && !od.IsSynthetic() {
+			mi.rangeInfos = append(mi.rangeInfos, mi.oneofs[od.Name()])
+			i += od.Fields().Len()
+		} else {
+			mi.rangeInfos = append(mi.rangeInfos, mi.fields[fd.Number()])
+			i++
+		}
+	}
+
+	// Introduce instability to iteration order, but keep it deterministic.
+	if len(mi.rangeInfos) > 1 && detrand.Bool() {
+		i := detrand.Intn(len(mi.rangeInfos) - 1)
+		mi.rangeInfos[i], mi.rangeInfos[i+1] = mi.rangeInfos[i+1], mi.rangeInfos[i]
+	}
+}
+
+func (mi *MessageInfo) makeUnknownFieldsFunc(t reflect.Type, si structInfo) {
+	switch {
+	case si.unknownOffset.IsValid() && si.unknownType == unknownFieldsAType:
+		// Handle as []byte.
+		mi.getUnknown = func(p pointer) pref.RawFields {
+			if p.IsNil() {
+				return nil
+			}
+			return *p.Apply(mi.unknownOffset).Bytes()
+		}
+		mi.setUnknown = func(p pointer, b pref.RawFields) {
+			if p.IsNil() {
+				panic("invalid SetUnknown on nil Message")
+			}
+			*p.Apply(mi.unknownOffset).Bytes() = b
+		}
+	case si.unknownOffset.IsValid() && si.unknownType == unknownFieldsBType:
+		// Handle as *[]byte.
+		mi.getUnknown = func(p pointer) pref.RawFields {
+			if p.IsNil() {
+				return nil
+			}
+			bp := p.Apply(mi.unknownOffset).BytesPtr()
+			if *bp == nil {
+				return nil
+			}
+			return **bp
+		}
+		mi.setUnknown = func(p pointer, b pref.RawFields) {
+			if p.IsNil() {
+				panic("invalid SetUnknown on nil Message")
+			}
+			bp := p.Apply(mi.unknownOffset).BytesPtr()
+			if *bp == nil {
+				*bp = new([]byte)
+			}
+			**bp = b
+		}
+	default:
+		mi.getUnknown = func(pointer) pref.RawFields {
+			return nil
+		}
+		mi.setUnknown = func(p pointer, _ pref.RawFields) {
+			if p.IsNil() {
+				panic("invalid SetUnknown on nil Message")
+			}
+		}
+	}
+}
+
+func (mi *MessageInfo) makeExtensionFieldsFunc(t reflect.Type, si structInfo) {
+	if si.extensionOffset.IsValid() {
+		mi.extensionMap = func(p pointer) *extensionMap {
+			if p.IsNil() {
+				return (*extensionMap)(nil)
+			}
+			v := p.Apply(si.extensionOffset).AsValueOf(extensionFieldsType)
+			return (*extensionMap)(v.Interface().(*map[int32]ExtensionField))
+		}
+	} else {
+		mi.extensionMap = func(pointer) *extensionMap {
+			return (*extensionMap)(nil)
+		}
+	}
+}
+func (mi *MessageInfo) makeFieldTypes(si structInfo) {
+	md := mi.Desc
+	fds := md.Fields()
+	for i := 0; i < fds.Len(); i++ {
+		var ft reflect.Type
+		fd := fds.Get(i)
+		fs := si.fieldsByNumber[fd.Number()]
+		isOneof := fd.ContainingOneof() != nil && !fd.ContainingOneof().IsSynthetic()
+		if isOneof {
+			fs = si.oneofsByName[fd.ContainingOneof().Name()]
+		}
+		var isMessage bool
+		switch {
+		case fs.Type == nil:
+			continue // never occurs for officially generated message types
+		case isOneof:
+			if fd.Enum() != nil || fd.Message() != nil {
+				ft = si.oneofWrappersByNumber[fd.Number()].Field(0).Type
+			}
+		case fd.IsMap():
+			if fd.MapValue().Enum() != nil || fd.MapValue().Message() != nil {
+				ft = fs.Type.Elem()
+			}
+			isMessage = fd.MapValue().Message() != nil
+		case fd.IsList():
+			if fd.Enum() != nil || fd.Message() != nil {
+				ft = fs.Type.Elem()
+			}
+			isMessage = fd.Message() != nil
+		case fd.Enum() != nil:
+			ft = fs.Type
+			if fd.HasPresence() && ft.Kind() == reflect.Ptr {
+				ft = ft.Elem()
+			}
+		case fd.Message() != nil:
+			ft = fs.Type
+			if fd.IsWeak() {
+				ft = nil
+			}
+			isMessage = true
+		}
+		if isMessage && ft != nil && ft.Kind() != reflect.Ptr {
+			ft = reflect.PtrTo(ft) // never occurs for officially generated message types
+		}
+		if ft != nil {
+			if mi.fieldTypes == nil {
+				mi.fieldTypes = make(map[pref.FieldNumber]interface{})
+			}
+			mi.fieldTypes[fd.Number()] = reflect.Zero(ft).Interface()
+		}
+	}
+}
+
+type extensionMap map[int32]ExtensionField
+
+func (m *extensionMap) Range(f func(pref.FieldDescriptor, pref.Value) bool) {
+	if m != nil {
+		for _, x := range *m {
+			xd := x.Type().TypeDescriptor()
+			v := x.Value()
+			if xd.IsList() && v.List().Len() == 0 {
+				continue
+			}
+			if !f(xd, v) {
+				return
+			}
+		}
+	}
+}
+func (m *extensionMap) Has(xt pref.ExtensionType) (ok bool) {
+	if m == nil {
+		return false
+	}
+	xd := xt.TypeDescriptor()
+	x, ok := (*m)[int32(xd.Number())]
+	if !ok {
+		return false
+	}
+	switch {
+	case xd.IsList():
+		return x.Value().List().Len() > 0
+	case xd.IsMap():
+		return x.Value().Map().Len() > 0
+	case xd.Message() != nil:
+		return x.Value().Message().IsValid()
+	}
+	return true
+}
+func (m *extensionMap) Clear(xt pref.ExtensionType) {
+	delete(*m, int32(xt.TypeDescriptor().Number()))
+}
+func (m *extensionMap) Get(xt pref.ExtensionType) pref.Value {
+	xd := xt.TypeDescriptor()
+	if m != nil {
+		if x, ok := (*m)[int32(xd.Number())]; ok {
+			return x.Value()
+		}
+	}
+	return xt.Zero()
+}
+func (m *extensionMap) Set(xt pref.ExtensionType, v pref.Value) {
+	xd := xt.TypeDescriptor()
+	isValid := true
+	switch {
+	case !xt.IsValidValue(v):
+		isValid = false
+	case xd.IsList():
+		isValid = v.List().IsValid()
+	case xd.IsMap():
+		isValid = v.Map().IsValid()
+	case xd.Message() != nil:
+		isValid = v.Message().IsValid()
+	}
+	if !isValid {
+		panic(fmt.Sprintf("%v: assigning invalid value", xt.TypeDescriptor().FullName()))
+	}
+
+	if *m == nil {
+		*m = make(map[int32]ExtensionField)
+	}
+	var x ExtensionField
+	x.Set(xt, v)
+	(*m)[int32(xd.Number())] = x
+}
+func (m *extensionMap) Mutable(xt pref.ExtensionType) pref.Value {
+	xd := xt.TypeDescriptor()
+	if xd.Kind() != pref.MessageKind && xd.Kind() != pref.GroupKind && !xd.IsList() && !xd.IsMap() {
+		panic("invalid Mutable on field with non-composite type")
+	}
+	if x, ok := (*m)[int32(xd.Number())]; ok {
+		return x.Value()
+	}
+	v := xt.New()
+	m.Set(xt, v)
+	return v
+}
+
+// MessageState is a data structure that is nested as the first field in a
+// concrete message. It provides a way to implement the ProtoReflect method
+// in an allocation-free way without needing to have a shadow Go type generated
+// for every message type. This technique only works using unsafe.
+//
+//
+// Example generated code:
+//
+//	type M struct {
+//		state protoimpl.MessageState
+//
+//		Field1 int32
+//		Field2 string
+//		Field3 *BarMessage
+//		...
+//	}
+//
+//	func (m *M) ProtoReflect() protoreflect.Message {
+//		mi := &file_fizz_buzz_proto_msgInfos[5]
+//		if protoimpl.UnsafeEnabled && m != nil {
+//			ms := protoimpl.X.MessageStateOf(Pointer(m))
+//			if ms.LoadMessageInfo() == nil {
+//				ms.StoreMessageInfo(mi)
+//			}
+//			return ms
+//		}
+//		return mi.MessageOf(m)
+//	}
+//
+// The MessageState type holds a *MessageInfo, which must be atomically set to
+// the message info associated with a given message instance.
+// By unsafely converting a *M into a *MessageState, the MessageState object
+// has access to all the information needed to implement protobuf reflection.
+// It has access to the message info as its first field, and a pointer to the
+// MessageState is identical to a pointer to the concrete message value.
+//
+//
+// Requirements:
+//	• The type M must implement protoreflect.ProtoMessage.
+//	• The address of m must not be nil.
+//	• The address of m and the address of m.state must be equal,
+//	even though they are different Go types.
+type MessageState struct {
+	pragma.NoUnkeyedLiterals
+	pragma.DoNotCompare
+	pragma.DoNotCopy
+
+	atomicMessageInfo *MessageInfo
+}
+
+type messageState MessageState
+
+var (
+	_ pref.Message = (*messageState)(nil)
+	_ unwrapper    = (*messageState)(nil)
+)
+
+// messageDataType is a tuple of a pointer to the message data and
+// a pointer to the message type. It is a generalized way of providing a
+// reflective view over a message instance. The disadvantage of this approach
+// is the need to allocate this tuple of 16B.
+type messageDataType struct {
+	p  pointer
+	mi *MessageInfo
+}
+
+type (
+	messageReflectWrapper messageDataType
+	messageIfaceWrapper   messageDataType
+)
+
+var (
+	_ pref.Message      = (*messageReflectWrapper)(nil)
+	_ unwrapper         = (*messageReflectWrapper)(nil)
+	_ pref.ProtoMessage = (*messageIfaceWrapper)(nil)
+	_ unwrapper         = (*messageIfaceWrapper)(nil)
+)
+
+// MessageOf returns a reflective view over a message. The input must be a
+// pointer to a named Go struct. If the provided type has a ProtoReflect method,
+// it must be implemented by calling this method.
+func (mi *MessageInfo) MessageOf(m interface{}) pref.Message {
+	if reflect.TypeOf(m) != mi.GoReflectType {
+		panic(fmt.Sprintf("type mismatch: got %T, want %v", m, mi.GoReflectType))
+	}
+	p := pointerOfIface(m)
+	if p.IsNil() {
+		return mi.nilMessage.Init(mi)
+	}
+	return &messageReflectWrapper{p, mi}
+}
+
+func (m *messageReflectWrapper) pointer() pointer          { return m.p }
+func (m *messageReflectWrapper) messageInfo() *MessageInfo { return m.mi }
+
+// Reset implements the v1 proto.Message.Reset method.
+func (m *messageIfaceWrapper) Reset() {
+	if mr, ok := m.protoUnwrap().(interface{ Reset() }); ok {
+		mr.Reset()
+		return
+	}
+	rv := reflect.ValueOf(m.protoUnwrap())
+	if rv.Kind() == reflect.Ptr && !rv.IsNil() {
+		rv.Elem().Set(reflect.Zero(rv.Type().Elem()))
+	}
+}
+func (m *messageIfaceWrapper) ProtoReflect() pref.Message {
+	return (*messageReflectWrapper)(m)
+}
+func (m *messageIfaceWrapper) protoUnwrap() interface{} {
+	return m.p.AsIfaceOf(m.mi.GoReflectType.Elem())
+}
+
+// checkField verifies that the provided field descriptor is valid.
+// Exactly one of the returned values is populated.
+func (mi *MessageInfo) checkField(fd pref.FieldDescriptor) (*fieldInfo, pref.ExtensionType) {
+	var fi *fieldInfo
+	if n := fd.Number(); 0 < n && int(n) < len(mi.denseFields) {
+		fi = mi.denseFields[n]
+	} else {
+		fi = mi.fields[n]
+	}
+	if fi != nil {
+		if fi.fieldDesc != fd {
+			if got, want := fd.FullName(), fi.fieldDesc.FullName(); got != want {
+				panic(fmt.Sprintf("mismatching field: got %v, want %v", got, want))
+			}
+			panic(fmt.Sprintf("mismatching field: %v", fd.FullName()))
+		}
+		return fi, nil
+	}
+
+	if fd.IsExtension() {
+		if got, want := fd.ContainingMessage().FullName(), mi.Desc.FullName(); got != want {
+			// TODO: Should this be exact containing message descriptor match?
+			panic(fmt.Sprintf("extension %v has mismatching containing message: got %v, want %v", fd.FullName(), got, want))
+		}
+		if !mi.Desc.ExtensionRanges().Has(fd.Number()) {
+			panic(fmt.Sprintf("extension %v extends %v outside the extension range", fd.FullName(), mi.Desc.FullName()))
+		}
+		xtd, ok := fd.(pref.ExtensionTypeDescriptor)
+		if !ok {
+			panic(fmt.Sprintf("extension %v does not implement protoreflect.ExtensionTypeDescriptor", fd.FullName()))
+		}
+		return nil, xtd.Type()
+	}
+	panic(fmt.Sprintf("field %v is invalid", fd.FullName()))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/message_reflect_field.go b/vendor/google.golang.org/protobuf/internal/impl/message_reflect_field.go
new file mode 100644
index 0000000..343cf87
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/message_reflect_field.go
@@ -0,0 +1,543 @@
+// 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 impl
+
+import (
+	"fmt"
+	"math"
+	"reflect"
+	"sync"
+
+	"google.golang.org/protobuf/internal/flags"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+)
+
+type fieldInfo struct {
+	fieldDesc pref.FieldDescriptor
+
+	// These fields are used for protobuf reflection support.
+	has        func(pointer) bool
+	clear      func(pointer)
+	get        func(pointer) pref.Value
+	set        func(pointer, pref.Value)
+	mutable    func(pointer) pref.Value
+	newMessage func() pref.Message
+	newField   func() pref.Value
+}
+
+func fieldInfoForMissing(fd pref.FieldDescriptor) fieldInfo {
+	// This never occurs for generated message types.
+	// It implies that a hand-crafted type has missing Go fields
+	// for specific protobuf message fields.
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			return false
+		},
+		clear: func(p pointer) {
+			panic("missing Go struct field for " + string(fd.FullName()))
+		},
+		get: func(p pointer) pref.Value {
+			return fd.Default()
+		},
+		set: func(p pointer, v pref.Value) {
+			panic("missing Go struct field for " + string(fd.FullName()))
+		},
+		mutable: func(p pointer) pref.Value {
+			panic("missing Go struct field for " + string(fd.FullName()))
+		},
+		newMessage: func() pref.Message {
+			panic("missing Go struct field for " + string(fd.FullName()))
+		},
+		newField: func() pref.Value {
+			if v := fd.Default(); v.IsValid() {
+				return v
+			}
+			panic("missing Go struct field for " + string(fd.FullName()))
+		},
+	}
+}
+
+func fieldInfoForOneof(fd pref.FieldDescriptor, fs reflect.StructField, x exporter, ot reflect.Type) fieldInfo {
+	ft := fs.Type
+	if ft.Kind() != reflect.Interface {
+		panic(fmt.Sprintf("field %v has invalid type: got %v, want interface kind", fd.FullName(), ft))
+	}
+	if ot.Kind() != reflect.Struct {
+		panic(fmt.Sprintf("field %v has invalid type: got %v, want struct kind", fd.FullName(), ot))
+	}
+	if !reflect.PtrTo(ot).Implements(ft) {
+		panic(fmt.Sprintf("field %v has invalid type: %v does not implement %v", fd.FullName(), ot, ft))
+	}
+	conv := NewConverter(ot.Field(0).Type, fd)
+	isMessage := fd.Message() != nil
+
+	// TODO: Implement unsafe fast path?
+	fieldOffset := offsetOf(fs, x)
+	return fieldInfo{
+		// NOTE: The logic below intentionally assumes that oneof fields are
+		// well-formatted. That is, the oneof interface never contains a
+		// typed nil pointer to one of the wrapper structs.
+
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() || rv.Elem().Type().Elem() != ot || rv.Elem().IsNil() {
+				return false
+			}
+			return true
+		},
+		clear: func(p pointer) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() || rv.Elem().Type().Elem() != ot {
+				// NOTE: We intentionally don't check for rv.Elem().IsNil()
+				// so that (*OneofWrapperType)(nil) gets cleared to nil.
+				return
+			}
+			rv.Set(reflect.Zero(rv.Type()))
+		},
+		get: func(p pointer) pref.Value {
+			if p.IsNil() {
+				return conv.Zero()
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() || rv.Elem().Type().Elem() != ot || rv.Elem().IsNil() {
+				return conv.Zero()
+			}
+			rv = rv.Elem().Elem().Field(0)
+			return conv.PBValueOf(rv)
+		},
+		set: func(p pointer, v pref.Value) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() || rv.Elem().Type().Elem() != ot || rv.Elem().IsNil() {
+				rv.Set(reflect.New(ot))
+			}
+			rv = rv.Elem().Elem().Field(0)
+			rv.Set(conv.GoValueOf(v))
+		},
+		mutable: func(p pointer) pref.Value {
+			if !isMessage {
+				panic(fmt.Sprintf("field %v with invalid Mutable call on field with non-composite type", fd.FullName()))
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() || rv.Elem().Type().Elem() != ot || rv.Elem().IsNil() {
+				rv.Set(reflect.New(ot))
+			}
+			rv = rv.Elem().Elem().Field(0)
+			if rv.Kind() == reflect.Ptr && rv.IsNil() {
+				rv.Set(conv.GoValueOf(pref.ValueOfMessage(conv.New().Message())))
+			}
+			return conv.PBValueOf(rv)
+		},
+		newMessage: func() pref.Message {
+			return conv.New().Message()
+		},
+		newField: func() pref.Value {
+			return conv.New()
+		},
+	}
+}
+
+func fieldInfoForMap(fd pref.FieldDescriptor, fs reflect.StructField, x exporter) fieldInfo {
+	ft := fs.Type
+	if ft.Kind() != reflect.Map {
+		panic(fmt.Sprintf("field %v has invalid type: got %v, want map kind", fd.FullName(), ft))
+	}
+	conv := NewConverter(ft, fd)
+
+	// TODO: Implement unsafe fast path?
+	fieldOffset := offsetOf(fs, x)
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			return rv.Len() > 0
+		},
+		clear: func(p pointer) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			rv.Set(reflect.Zero(rv.Type()))
+		},
+		get: func(p pointer) pref.Value {
+			if p.IsNil() {
+				return conv.Zero()
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.Len() == 0 {
+				return conv.Zero()
+			}
+			return conv.PBValueOf(rv)
+		},
+		set: func(p pointer, v pref.Value) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			pv := conv.GoValueOf(v)
+			if pv.IsNil() {
+				panic(fmt.Sprintf("map field %v cannot be set with read-only value", fd.FullName()))
+			}
+			rv.Set(pv)
+		},
+		mutable: func(p pointer) pref.Value {
+			v := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if v.IsNil() {
+				v.Set(reflect.MakeMap(fs.Type))
+			}
+			return conv.PBValueOf(v)
+		},
+		newField: func() pref.Value {
+			return conv.New()
+		},
+	}
+}
+
+func fieldInfoForList(fd pref.FieldDescriptor, fs reflect.StructField, x exporter) fieldInfo {
+	ft := fs.Type
+	if ft.Kind() != reflect.Slice {
+		panic(fmt.Sprintf("field %v has invalid type: got %v, want slice kind", fd.FullName(), ft))
+	}
+	conv := NewConverter(reflect.PtrTo(ft), fd)
+
+	// TODO: Implement unsafe fast path?
+	fieldOffset := offsetOf(fs, x)
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			return rv.Len() > 0
+		},
+		clear: func(p pointer) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			rv.Set(reflect.Zero(rv.Type()))
+		},
+		get: func(p pointer) pref.Value {
+			if p.IsNil() {
+				return conv.Zero()
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type)
+			if rv.Elem().Len() == 0 {
+				return conv.Zero()
+			}
+			return conv.PBValueOf(rv)
+		},
+		set: func(p pointer, v pref.Value) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			pv := conv.GoValueOf(v)
+			if pv.IsNil() {
+				panic(fmt.Sprintf("list field %v cannot be set with read-only value", fd.FullName()))
+			}
+			rv.Set(pv.Elem())
+		},
+		mutable: func(p pointer) pref.Value {
+			v := p.Apply(fieldOffset).AsValueOf(fs.Type)
+			return conv.PBValueOf(v)
+		},
+		newField: func() pref.Value {
+			return conv.New()
+		},
+	}
+}
+
+var (
+	nilBytes   = reflect.ValueOf([]byte(nil))
+	emptyBytes = reflect.ValueOf([]byte{})
+)
+
+func fieldInfoForScalar(fd pref.FieldDescriptor, fs reflect.StructField, x exporter) fieldInfo {
+	ft := fs.Type
+	nullable := fd.HasPresence()
+	isBytes := ft.Kind() == reflect.Slice && ft.Elem().Kind() == reflect.Uint8
+	if nullable {
+		if ft.Kind() != reflect.Ptr && ft.Kind() != reflect.Slice {
+			// This never occurs for generated message types.
+			// Despite the protobuf type system specifying presence,
+			// the Go field type cannot represent it.
+			nullable = false
+		}
+		if ft.Kind() == reflect.Ptr {
+			ft = ft.Elem()
+		}
+	}
+	conv := NewConverter(ft, fd)
+
+	// TODO: Implement unsafe fast path?
+	fieldOffset := offsetOf(fs, x)
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if nullable {
+				return !rv.IsNil()
+			}
+			switch rv.Kind() {
+			case reflect.Bool:
+				return rv.Bool()
+			case reflect.Int32, reflect.Int64:
+				return rv.Int() != 0
+			case reflect.Uint32, reflect.Uint64:
+				return rv.Uint() != 0
+			case reflect.Float32, reflect.Float64:
+				return rv.Float() != 0 || math.Signbit(rv.Float())
+			case reflect.String, reflect.Slice:
+				return rv.Len() > 0
+			default:
+				panic(fmt.Sprintf("field %v has invalid type: %v", fd.FullName(), rv.Type())) // should never happen
+			}
+		},
+		clear: func(p pointer) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			rv.Set(reflect.Zero(rv.Type()))
+		},
+		get: func(p pointer) pref.Value {
+			if p.IsNil() {
+				return conv.Zero()
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if nullable {
+				if rv.IsNil() {
+					return conv.Zero()
+				}
+				if rv.Kind() == reflect.Ptr {
+					rv = rv.Elem()
+				}
+			}
+			return conv.PBValueOf(rv)
+		},
+		set: func(p pointer, v pref.Value) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if nullable && rv.Kind() == reflect.Ptr {
+				if rv.IsNil() {
+					rv.Set(reflect.New(ft))
+				}
+				rv = rv.Elem()
+			}
+			rv.Set(conv.GoValueOf(v))
+			if isBytes && rv.Len() == 0 {
+				if nullable {
+					rv.Set(emptyBytes) // preserve presence
+				} else {
+					rv.Set(nilBytes) // do not preserve presence
+				}
+			}
+		},
+		newField: func() pref.Value {
+			return conv.New()
+		},
+	}
+}
+
+func fieldInfoForWeakMessage(fd pref.FieldDescriptor, weakOffset offset) fieldInfo {
+	if !flags.ProtoLegacy {
+		panic("no support for proto1 weak fields")
+	}
+
+	var once sync.Once
+	var messageType pref.MessageType
+	lazyInit := func() {
+		once.Do(func() {
+			messageName := fd.Message().FullName()
+			messageType, _ = preg.GlobalTypes.FindMessageByName(messageName)
+			if messageType == nil {
+				panic(fmt.Sprintf("weak message %v for field %v is not linked in", messageName, fd.FullName()))
+			}
+		})
+	}
+
+	num := fd.Number()
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			_, ok := p.Apply(weakOffset).WeakFields().get(num)
+			return ok
+		},
+		clear: func(p pointer) {
+			p.Apply(weakOffset).WeakFields().clear(num)
+		},
+		get: func(p pointer) pref.Value {
+			lazyInit()
+			if p.IsNil() {
+				return pref.ValueOfMessage(messageType.Zero())
+			}
+			m, ok := p.Apply(weakOffset).WeakFields().get(num)
+			if !ok {
+				return pref.ValueOfMessage(messageType.Zero())
+			}
+			return pref.ValueOfMessage(m.ProtoReflect())
+		},
+		set: func(p pointer, v pref.Value) {
+			lazyInit()
+			m := v.Message()
+			if m.Descriptor() != messageType.Descriptor() {
+				if got, want := m.Descriptor().FullName(), messageType.Descriptor().FullName(); got != want {
+					panic(fmt.Sprintf("field %v has mismatching message descriptor: got %v, want %v", fd.FullName(), got, want))
+				}
+				panic(fmt.Sprintf("field %v has mismatching message descriptor: %v", fd.FullName(), m.Descriptor().FullName()))
+			}
+			p.Apply(weakOffset).WeakFields().set(num, m.Interface())
+		},
+		mutable: func(p pointer) pref.Value {
+			lazyInit()
+			fs := p.Apply(weakOffset).WeakFields()
+			m, ok := fs.get(num)
+			if !ok {
+				m = messageType.New().Interface()
+				fs.set(num, m)
+			}
+			return pref.ValueOfMessage(m.ProtoReflect())
+		},
+		newMessage: func() pref.Message {
+			lazyInit()
+			return messageType.New()
+		},
+		newField: func() pref.Value {
+			lazyInit()
+			return pref.ValueOfMessage(messageType.New())
+		},
+	}
+}
+
+func fieldInfoForMessage(fd pref.FieldDescriptor, fs reflect.StructField, x exporter) fieldInfo {
+	ft := fs.Type
+	conv := NewConverter(ft, fd)
+
+	// TODO: Implement unsafe fast path?
+	fieldOffset := offsetOf(fs, x)
+	return fieldInfo{
+		fieldDesc: fd,
+		has: func(p pointer) bool {
+			if p.IsNil() {
+				return false
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if fs.Type.Kind() != reflect.Ptr {
+				return !isZero(rv)
+			}
+			return !rv.IsNil()
+		},
+		clear: func(p pointer) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			rv.Set(reflect.Zero(rv.Type()))
+		},
+		get: func(p pointer) pref.Value {
+			if p.IsNil() {
+				return conv.Zero()
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			return conv.PBValueOf(rv)
+		},
+		set: func(p pointer, v pref.Value) {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			rv.Set(conv.GoValueOf(v))
+			if fs.Type.Kind() == reflect.Ptr && rv.IsNil() {
+				panic(fmt.Sprintf("field %v has invalid nil pointer", fd.FullName()))
+			}
+		},
+		mutable: func(p pointer) pref.Value {
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if fs.Type.Kind() == reflect.Ptr && rv.IsNil() {
+				rv.Set(conv.GoValueOf(conv.New()))
+			}
+			return conv.PBValueOf(rv)
+		},
+		newMessage: func() pref.Message {
+			return conv.New().Message()
+		},
+		newField: func() pref.Value {
+			return conv.New()
+		},
+	}
+}
+
+type oneofInfo struct {
+	oneofDesc pref.OneofDescriptor
+	which     func(pointer) pref.FieldNumber
+}
+
+func makeOneofInfo(od pref.OneofDescriptor, si structInfo, x exporter) *oneofInfo {
+	oi := &oneofInfo{oneofDesc: od}
+	if od.IsSynthetic() {
+		fs := si.fieldsByNumber[od.Fields().Get(0).Number()]
+		fieldOffset := offsetOf(fs, x)
+		oi.which = func(p pointer) pref.FieldNumber {
+			if p.IsNil() {
+				return 0
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() { // valid on either *T or []byte
+				return 0
+			}
+			return od.Fields().Get(0).Number()
+		}
+	} else {
+		fs := si.oneofsByName[od.Name()]
+		fieldOffset := offsetOf(fs, x)
+		oi.which = func(p pointer) pref.FieldNumber {
+			if p.IsNil() {
+				return 0
+			}
+			rv := p.Apply(fieldOffset).AsValueOf(fs.Type).Elem()
+			if rv.IsNil() {
+				return 0
+			}
+			rv = rv.Elem()
+			if rv.IsNil() {
+				return 0
+			}
+			return si.oneofWrappersByType[rv.Type().Elem()]
+		}
+	}
+	return oi
+}
+
+// isZero is identical to reflect.Value.IsZero.
+// TODO: Remove this when Go1.13 is the minimally supported Go version.
+func isZero(v reflect.Value) bool {
+	switch v.Kind() {
+	case reflect.Bool:
+		return !v.Bool()
+	case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
+		return v.Int() == 0
+	case reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64, reflect.Uintptr:
+		return v.Uint() == 0
+	case reflect.Float32, reflect.Float64:
+		return math.Float64bits(v.Float()) == 0
+	case reflect.Complex64, reflect.Complex128:
+		c := v.Complex()
+		return math.Float64bits(real(c)) == 0 && math.Float64bits(imag(c)) == 0
+	case reflect.Array:
+		for i := 0; i < v.Len(); i++ {
+			if !isZero(v.Index(i)) {
+				return false
+			}
+		}
+		return true
+	case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice, reflect.UnsafePointer:
+		return v.IsNil()
+	case reflect.String:
+		return v.Len() == 0
+	case reflect.Struct:
+		for i := 0; i < v.NumField(); i++ {
+			if !isZero(v.Field(i)) {
+				return false
+			}
+		}
+		return true
+	default:
+		panic(&reflect.ValueError{"reflect.Value.IsZero", v.Kind()})
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/message_reflect_gen.go b/vendor/google.golang.org/protobuf/internal/impl/message_reflect_gen.go
new file mode 100644
index 0000000..741d6e5
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/message_reflect_gen.go
@@ -0,0 +1,249 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package impl
+
+import (
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+func (m *messageState) Descriptor() protoreflect.MessageDescriptor {
+	return m.messageInfo().Desc
+}
+func (m *messageState) Type() protoreflect.MessageType {
+	return m.messageInfo()
+}
+func (m *messageState) New() protoreflect.Message {
+	return m.messageInfo().New()
+}
+func (m *messageState) Interface() protoreflect.ProtoMessage {
+	return m.protoUnwrap().(protoreflect.ProtoMessage)
+}
+func (m *messageState) protoUnwrap() interface{} {
+	return m.pointer().AsIfaceOf(m.messageInfo().GoReflectType.Elem())
+}
+func (m *messageState) ProtoMethods() *protoiface.Methods {
+	m.messageInfo().init()
+	return &m.messageInfo().methods
+}
+
+// ProtoMessageInfo is a pseudo-internal API for allowing the v1 code
+// to be able to retrieve a v2 MessageInfo struct.
+//
+// WARNING: This method is exempt from the compatibility promise and
+// may be removed in the future without warning.
+func (m *messageState) ProtoMessageInfo() *MessageInfo {
+	return m.messageInfo()
+}
+
+func (m *messageState) Range(f func(protoreflect.FieldDescriptor, protoreflect.Value) bool) {
+	m.messageInfo().init()
+	for _, ri := range m.messageInfo().rangeInfos {
+		switch ri := ri.(type) {
+		case *fieldInfo:
+			if ri.has(m.pointer()) {
+				if !f(ri.fieldDesc, ri.get(m.pointer())) {
+					return
+				}
+			}
+		case *oneofInfo:
+			if n := ri.which(m.pointer()); n > 0 {
+				fi := m.messageInfo().fields[n]
+				if !f(fi.fieldDesc, fi.get(m.pointer())) {
+					return
+				}
+			}
+		}
+	}
+	m.messageInfo().extensionMap(m.pointer()).Range(f)
+}
+func (m *messageState) Has(fd protoreflect.FieldDescriptor) bool {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.has(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Has(xt)
+	}
+}
+func (m *messageState) Clear(fd protoreflect.FieldDescriptor) {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		fi.clear(m.pointer())
+	} else {
+		m.messageInfo().extensionMap(m.pointer()).Clear(xt)
+	}
+}
+func (m *messageState) Get(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.get(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Get(xt)
+	}
+}
+func (m *messageState) Set(fd protoreflect.FieldDescriptor, v protoreflect.Value) {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		fi.set(m.pointer(), v)
+	} else {
+		m.messageInfo().extensionMap(m.pointer()).Set(xt, v)
+	}
+}
+func (m *messageState) Mutable(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.mutable(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Mutable(xt)
+	}
+}
+func (m *messageState) NewField(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.newField()
+	} else {
+		return xt.New()
+	}
+}
+func (m *messageState) WhichOneof(od protoreflect.OneofDescriptor) protoreflect.FieldDescriptor {
+	m.messageInfo().init()
+	if oi := m.messageInfo().oneofs[od.Name()]; oi != nil && oi.oneofDesc == od {
+		return od.Fields().ByNumber(oi.which(m.pointer()))
+	}
+	panic("invalid oneof descriptor " + string(od.FullName()) + " for message " + string(m.Descriptor().FullName()))
+}
+func (m *messageState) GetUnknown() protoreflect.RawFields {
+	m.messageInfo().init()
+	return m.messageInfo().getUnknown(m.pointer())
+}
+func (m *messageState) SetUnknown(b protoreflect.RawFields) {
+	m.messageInfo().init()
+	m.messageInfo().setUnknown(m.pointer(), b)
+}
+func (m *messageState) IsValid() bool {
+	return !m.pointer().IsNil()
+}
+
+func (m *messageReflectWrapper) Descriptor() protoreflect.MessageDescriptor {
+	return m.messageInfo().Desc
+}
+func (m *messageReflectWrapper) Type() protoreflect.MessageType {
+	return m.messageInfo()
+}
+func (m *messageReflectWrapper) New() protoreflect.Message {
+	return m.messageInfo().New()
+}
+func (m *messageReflectWrapper) Interface() protoreflect.ProtoMessage {
+	if m, ok := m.protoUnwrap().(protoreflect.ProtoMessage); ok {
+		return m
+	}
+	return (*messageIfaceWrapper)(m)
+}
+func (m *messageReflectWrapper) protoUnwrap() interface{} {
+	return m.pointer().AsIfaceOf(m.messageInfo().GoReflectType.Elem())
+}
+func (m *messageReflectWrapper) ProtoMethods() *protoiface.Methods {
+	m.messageInfo().init()
+	return &m.messageInfo().methods
+}
+
+// ProtoMessageInfo is a pseudo-internal API for allowing the v1 code
+// to be able to retrieve a v2 MessageInfo struct.
+//
+// WARNING: This method is exempt from the compatibility promise and
+// may be removed in the future without warning.
+func (m *messageReflectWrapper) ProtoMessageInfo() *MessageInfo {
+	return m.messageInfo()
+}
+
+func (m *messageReflectWrapper) Range(f func(protoreflect.FieldDescriptor, protoreflect.Value) bool) {
+	m.messageInfo().init()
+	for _, ri := range m.messageInfo().rangeInfos {
+		switch ri := ri.(type) {
+		case *fieldInfo:
+			if ri.has(m.pointer()) {
+				if !f(ri.fieldDesc, ri.get(m.pointer())) {
+					return
+				}
+			}
+		case *oneofInfo:
+			if n := ri.which(m.pointer()); n > 0 {
+				fi := m.messageInfo().fields[n]
+				if !f(fi.fieldDesc, fi.get(m.pointer())) {
+					return
+				}
+			}
+		}
+	}
+	m.messageInfo().extensionMap(m.pointer()).Range(f)
+}
+func (m *messageReflectWrapper) Has(fd protoreflect.FieldDescriptor) bool {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.has(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Has(xt)
+	}
+}
+func (m *messageReflectWrapper) Clear(fd protoreflect.FieldDescriptor) {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		fi.clear(m.pointer())
+	} else {
+		m.messageInfo().extensionMap(m.pointer()).Clear(xt)
+	}
+}
+func (m *messageReflectWrapper) Get(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.get(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Get(xt)
+	}
+}
+func (m *messageReflectWrapper) Set(fd protoreflect.FieldDescriptor, v protoreflect.Value) {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		fi.set(m.pointer(), v)
+	} else {
+		m.messageInfo().extensionMap(m.pointer()).Set(xt, v)
+	}
+}
+func (m *messageReflectWrapper) Mutable(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.mutable(m.pointer())
+	} else {
+		return m.messageInfo().extensionMap(m.pointer()).Mutable(xt)
+	}
+}
+func (m *messageReflectWrapper) NewField(fd protoreflect.FieldDescriptor) protoreflect.Value {
+	m.messageInfo().init()
+	if fi, xt := m.messageInfo().checkField(fd); fi != nil {
+		return fi.newField()
+	} else {
+		return xt.New()
+	}
+}
+func (m *messageReflectWrapper) WhichOneof(od protoreflect.OneofDescriptor) protoreflect.FieldDescriptor {
+	m.messageInfo().init()
+	if oi := m.messageInfo().oneofs[od.Name()]; oi != nil && oi.oneofDesc == od {
+		return od.Fields().ByNumber(oi.which(m.pointer()))
+	}
+	panic("invalid oneof descriptor " + string(od.FullName()) + " for message " + string(m.Descriptor().FullName()))
+}
+func (m *messageReflectWrapper) GetUnknown() protoreflect.RawFields {
+	m.messageInfo().init()
+	return m.messageInfo().getUnknown(m.pointer())
+}
+func (m *messageReflectWrapper) SetUnknown(b protoreflect.RawFields) {
+	m.messageInfo().init()
+	m.messageInfo().setUnknown(m.pointer(), b)
+}
+func (m *messageReflectWrapper) IsValid() bool {
+	return !m.pointer().IsNil()
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/pointer_reflect.go b/vendor/google.golang.org/protobuf/internal/impl/pointer_reflect.go
new file mode 100644
index 0000000..9e3ed82
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/pointer_reflect.go
@@ -0,0 +1,178 @@
+// 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 purego appengine
+
+package impl
+
+import (
+	"fmt"
+	"reflect"
+	"sync"
+)
+
+const UnsafeEnabled = false
+
+// Pointer is an opaque pointer type.
+type Pointer interface{}
+
+// offset represents the offset to a struct field, accessible from a pointer.
+// The offset is the field index into a struct.
+type offset struct {
+	index  int
+	export exporter
+}
+
+// offsetOf returns a field offset for the struct field.
+func offsetOf(f reflect.StructField, x exporter) offset {
+	if len(f.Index) != 1 {
+		panic("embedded structs are not supported")
+	}
+	if f.PkgPath == "" {
+		return offset{index: f.Index[0]} // field is already exported
+	}
+	if x == nil {
+		panic("exporter must be provided for unexported field")
+	}
+	return offset{index: f.Index[0], export: x}
+}
+
+// IsValid reports whether the offset is valid.
+func (f offset) IsValid() bool { return f.index >= 0 }
+
+// invalidOffset is an invalid field offset.
+var invalidOffset = offset{index: -1}
+
+// zeroOffset is a noop when calling pointer.Apply.
+var zeroOffset = offset{index: 0}
+
+// pointer is an abstract representation of a pointer to a struct or field.
+type pointer struct{ v reflect.Value }
+
+// pointerOf returns p as a pointer.
+func pointerOf(p Pointer) pointer {
+	return pointerOfIface(p)
+}
+
+// pointerOfValue returns v as a pointer.
+func pointerOfValue(v reflect.Value) pointer {
+	return pointer{v: v}
+}
+
+// pointerOfIface returns the pointer portion of an interface.
+func pointerOfIface(v interface{}) pointer {
+	return pointer{v: reflect.ValueOf(v)}
+}
+
+// IsNil reports whether the pointer is nil.
+func (p pointer) IsNil() bool {
+	return p.v.IsNil()
+}
+
+// Apply adds an offset to the pointer to derive a new pointer
+// to a specified field. The current pointer must be pointing at a struct.
+func (p pointer) Apply(f offset) pointer {
+	if f.export != nil {
+		if v := reflect.ValueOf(f.export(p.v.Interface(), f.index)); v.IsValid() {
+			return pointer{v: v}
+		}
+	}
+	return pointer{v: p.v.Elem().Field(f.index).Addr()}
+}
+
+// AsValueOf treats p as a pointer to an object of type t and returns the value.
+// It is equivalent to reflect.ValueOf(p.AsIfaceOf(t))
+func (p pointer) AsValueOf(t reflect.Type) reflect.Value {
+	if got := p.v.Type().Elem(); got != t {
+		panic(fmt.Sprintf("invalid type: got %v, want %v", got, t))
+	}
+	return p.v
+}
+
+// AsIfaceOf treats p as a pointer to an object of type t and returns the value.
+// It is equivalent to p.AsValueOf(t).Interface()
+func (p pointer) AsIfaceOf(t reflect.Type) interface{} {
+	return p.AsValueOf(t).Interface()
+}
+
+func (p pointer) Bool() *bool              { return p.v.Interface().(*bool) }
+func (p pointer) BoolPtr() **bool          { return p.v.Interface().(**bool) }
+func (p pointer) BoolSlice() *[]bool       { return p.v.Interface().(*[]bool) }
+func (p pointer) Int32() *int32            { return p.v.Interface().(*int32) }
+func (p pointer) Int32Ptr() **int32        { return p.v.Interface().(**int32) }
+func (p pointer) Int32Slice() *[]int32     { return p.v.Interface().(*[]int32) }
+func (p pointer) Int64() *int64            { return p.v.Interface().(*int64) }
+func (p pointer) Int64Ptr() **int64        { return p.v.Interface().(**int64) }
+func (p pointer) Int64Slice() *[]int64     { return p.v.Interface().(*[]int64) }
+func (p pointer) Uint32() *uint32          { return p.v.Interface().(*uint32) }
+func (p pointer) Uint32Ptr() **uint32      { return p.v.Interface().(**uint32) }
+func (p pointer) Uint32Slice() *[]uint32   { return p.v.Interface().(*[]uint32) }
+func (p pointer) Uint64() *uint64          { return p.v.Interface().(*uint64) }
+func (p pointer) Uint64Ptr() **uint64      { return p.v.Interface().(**uint64) }
+func (p pointer) Uint64Slice() *[]uint64   { return p.v.Interface().(*[]uint64) }
+func (p pointer) Float32() *float32        { return p.v.Interface().(*float32) }
+func (p pointer) Float32Ptr() **float32    { return p.v.Interface().(**float32) }
+func (p pointer) Float32Slice() *[]float32 { return p.v.Interface().(*[]float32) }
+func (p pointer) Float64() *float64        { return p.v.Interface().(*float64) }
+func (p pointer) Float64Ptr() **float64    { return p.v.Interface().(**float64) }
+func (p pointer) Float64Slice() *[]float64 { return p.v.Interface().(*[]float64) }
+func (p pointer) String() *string          { return p.v.Interface().(*string) }
+func (p pointer) StringPtr() **string      { return p.v.Interface().(**string) }
+func (p pointer) StringSlice() *[]string   { return p.v.Interface().(*[]string) }
+func (p pointer) Bytes() *[]byte           { return p.v.Interface().(*[]byte) }
+func (p pointer) BytesPtr() **[]byte       { return p.v.Interface().(**[]byte) }
+func (p pointer) BytesSlice() *[][]byte    { return p.v.Interface().(*[][]byte) }
+func (p pointer) WeakFields() *weakFields  { return (*weakFields)(p.v.Interface().(*WeakFields)) }
+func (p pointer) Extensions() *map[int32]ExtensionField {
+	return p.v.Interface().(*map[int32]ExtensionField)
+}
+
+func (p pointer) Elem() pointer {
+	return pointer{v: p.v.Elem()}
+}
+
+// PointerSlice copies []*T from p as a new []pointer.
+// This behavior differs from the implementation in pointer_unsafe.go.
+func (p pointer) PointerSlice() []pointer {
+	// TODO: reconsider this
+	if p.v.IsNil() {
+		return nil
+	}
+	n := p.v.Elem().Len()
+	s := make([]pointer, n)
+	for i := 0; i < n; i++ {
+		s[i] = pointer{v: p.v.Elem().Index(i)}
+	}
+	return s
+}
+
+// AppendPointerSlice appends v to p, which must be a []*T.
+func (p pointer) AppendPointerSlice(v pointer) {
+	sp := p.v.Elem()
+	sp.Set(reflect.Append(sp, v.v))
+}
+
+// SetPointer sets *p to v.
+func (p pointer) SetPointer(v pointer) {
+	p.v.Elem().Set(v.v)
+}
+
+func (Export) MessageStateOf(p Pointer) *messageState     { panic("not supported") }
+func (ms *messageState) pointer() pointer                 { panic("not supported") }
+func (ms *messageState) messageInfo() *MessageInfo        { panic("not supported") }
+func (ms *messageState) LoadMessageInfo() *MessageInfo    { panic("not supported") }
+func (ms *messageState) StoreMessageInfo(mi *MessageInfo) { panic("not supported") }
+
+type atomicNilMessage struct {
+	once sync.Once
+	m    messageReflectWrapper
+}
+
+func (m *atomicNilMessage) Init(mi *MessageInfo) *messageReflectWrapper {
+	m.once.Do(func() {
+		m.m.p = pointerOfIface(reflect.Zero(mi.GoReflectType).Interface())
+		m.m.mi = mi
+	})
+	return &m.m
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/pointer_unsafe.go b/vendor/google.golang.org/protobuf/internal/impl/pointer_unsafe.go
new file mode 100644
index 0000000..9ecf23a
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/pointer_unsafe.go
@@ -0,0 +1,174 @@
+// 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 !purego,!appengine
+
+package impl
+
+import (
+	"reflect"
+	"sync/atomic"
+	"unsafe"
+)
+
+const UnsafeEnabled = true
+
+// Pointer is an opaque pointer type.
+type Pointer unsafe.Pointer
+
+// offset represents the offset to a struct field, accessible from a pointer.
+// The offset is the byte offset to the field from the start of the struct.
+type offset uintptr
+
+// offsetOf returns a field offset for the struct field.
+func offsetOf(f reflect.StructField, x exporter) offset {
+	return offset(f.Offset)
+}
+
+// IsValid reports whether the offset is valid.
+func (f offset) IsValid() bool { return f != invalidOffset }
+
+// invalidOffset is an invalid field offset.
+var invalidOffset = ^offset(0)
+
+// zeroOffset is a noop when calling pointer.Apply.
+var zeroOffset = offset(0)
+
+// pointer is a pointer to a message struct or field.
+type pointer struct{ p unsafe.Pointer }
+
+// pointerOf returns p as a pointer.
+func pointerOf(p Pointer) pointer {
+	return pointer{p: unsafe.Pointer(p)}
+}
+
+// pointerOfValue returns v as a pointer.
+func pointerOfValue(v reflect.Value) pointer {
+	return pointer{p: unsafe.Pointer(v.Pointer())}
+}
+
+// pointerOfIface returns the pointer portion of an interface.
+func pointerOfIface(v interface{}) pointer {
+	type ifaceHeader struct {
+		Type unsafe.Pointer
+		Data unsafe.Pointer
+	}
+	return pointer{p: (*ifaceHeader)(unsafe.Pointer(&v)).Data}
+}
+
+// IsNil reports whether the pointer is nil.
+func (p pointer) IsNil() bool {
+	return p.p == nil
+}
+
+// Apply adds an offset to the pointer to derive a new pointer
+// to a specified field. The pointer must be valid and pointing at a struct.
+func (p pointer) Apply(f offset) pointer {
+	if p.IsNil() {
+		panic("invalid nil pointer")
+	}
+	return pointer{p: unsafe.Pointer(uintptr(p.p) + uintptr(f))}
+}
+
+// AsValueOf treats p as a pointer to an object of type t and returns the value.
+// It is equivalent to reflect.ValueOf(p.AsIfaceOf(t))
+func (p pointer) AsValueOf(t reflect.Type) reflect.Value {
+	return reflect.NewAt(t, p.p)
+}
+
+// AsIfaceOf treats p as a pointer to an object of type t and returns the value.
+// It is equivalent to p.AsValueOf(t).Interface()
+func (p pointer) AsIfaceOf(t reflect.Type) interface{} {
+	// TODO: Use tricky unsafe magic to directly create ifaceHeader.
+	return p.AsValueOf(t).Interface()
+}
+
+func (p pointer) Bool() *bool                           { return (*bool)(p.p) }
+func (p pointer) BoolPtr() **bool                       { return (**bool)(p.p) }
+func (p pointer) BoolSlice() *[]bool                    { return (*[]bool)(p.p) }
+func (p pointer) Int32() *int32                         { return (*int32)(p.p) }
+func (p pointer) Int32Ptr() **int32                     { return (**int32)(p.p) }
+func (p pointer) Int32Slice() *[]int32                  { return (*[]int32)(p.p) }
+func (p pointer) Int64() *int64                         { return (*int64)(p.p) }
+func (p pointer) Int64Ptr() **int64                     { return (**int64)(p.p) }
+func (p pointer) Int64Slice() *[]int64                  { return (*[]int64)(p.p) }
+func (p pointer) Uint32() *uint32                       { return (*uint32)(p.p) }
+func (p pointer) Uint32Ptr() **uint32                   { return (**uint32)(p.p) }
+func (p pointer) Uint32Slice() *[]uint32                { return (*[]uint32)(p.p) }
+func (p pointer) Uint64() *uint64                       { return (*uint64)(p.p) }
+func (p pointer) Uint64Ptr() **uint64                   { return (**uint64)(p.p) }
+func (p pointer) Uint64Slice() *[]uint64                { return (*[]uint64)(p.p) }
+func (p pointer) Float32() *float32                     { return (*float32)(p.p) }
+func (p pointer) Float32Ptr() **float32                 { return (**float32)(p.p) }
+func (p pointer) Float32Slice() *[]float32              { return (*[]float32)(p.p) }
+func (p pointer) Float64() *float64                     { return (*float64)(p.p) }
+func (p pointer) Float64Ptr() **float64                 { return (**float64)(p.p) }
+func (p pointer) Float64Slice() *[]float64              { return (*[]float64)(p.p) }
+func (p pointer) String() *string                       { return (*string)(p.p) }
+func (p pointer) StringPtr() **string                   { return (**string)(p.p) }
+func (p pointer) StringSlice() *[]string                { return (*[]string)(p.p) }
+func (p pointer) Bytes() *[]byte                        { return (*[]byte)(p.p) }
+func (p pointer) BytesPtr() **[]byte                    { return (**[]byte)(p.p) }
+func (p pointer) BytesSlice() *[][]byte                 { return (*[][]byte)(p.p) }
+func (p pointer) WeakFields() *weakFields               { return (*weakFields)(p.p) }
+func (p pointer) Extensions() *map[int32]ExtensionField { return (*map[int32]ExtensionField)(p.p) }
+
+func (p pointer) Elem() pointer {
+	return pointer{p: *(*unsafe.Pointer)(p.p)}
+}
+
+// PointerSlice loads []*T from p as a []pointer.
+// The value returned is aliased with the original slice.
+// This behavior differs from the implementation in pointer_reflect.go.
+func (p pointer) PointerSlice() []pointer {
+	// Super-tricky - p should point to a []*T where T is a
+	// message type. We load it as []pointer.
+	return *(*[]pointer)(p.p)
+}
+
+// AppendPointerSlice appends v to p, which must be a []*T.
+func (p pointer) AppendPointerSlice(v pointer) {
+	*(*[]pointer)(p.p) = append(*(*[]pointer)(p.p), v)
+}
+
+// SetPointer sets *p to v.
+func (p pointer) SetPointer(v pointer) {
+	*(*unsafe.Pointer)(p.p) = (unsafe.Pointer)(v.p)
+}
+
+// Static check that MessageState does not exceed the size of a pointer.
+const _ = uint(unsafe.Sizeof(unsafe.Pointer(nil)) - unsafe.Sizeof(MessageState{}))
+
+func (Export) MessageStateOf(p Pointer) *messageState {
+	// Super-tricky - see documentation on MessageState.
+	return (*messageState)(unsafe.Pointer(p))
+}
+func (ms *messageState) pointer() pointer {
+	// Super-tricky - see documentation on MessageState.
+	return pointer{p: unsafe.Pointer(ms)}
+}
+func (ms *messageState) messageInfo() *MessageInfo {
+	mi := ms.LoadMessageInfo()
+	if mi == nil {
+		panic("invalid nil message info; this suggests memory corruption due to a race or shallow copy on the message struct")
+	}
+	return mi
+}
+func (ms *messageState) LoadMessageInfo() *MessageInfo {
+	return (*MessageInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&ms.atomicMessageInfo))))
+}
+func (ms *messageState) StoreMessageInfo(mi *MessageInfo) {
+	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&ms.atomicMessageInfo)), unsafe.Pointer(mi))
+}
+
+type atomicNilMessage struct{ p unsafe.Pointer } // p is a *messageReflectWrapper
+
+func (m *atomicNilMessage) Init(mi *MessageInfo) *messageReflectWrapper {
+	if p := atomic.LoadPointer(&m.p); p != nil {
+		return (*messageReflectWrapper)(p)
+	}
+	w := &messageReflectWrapper{mi: mi}
+	atomic.CompareAndSwapPointer(&m.p, nil, (unsafe.Pointer)(w))
+	return (*messageReflectWrapper)(atomic.LoadPointer(&m.p))
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/validate.go b/vendor/google.golang.org/protobuf/internal/impl/validate.go
new file mode 100644
index 0000000..08cfb60
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/validate.go
@@ -0,0 +1,576 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+	"math"
+	"math/bits"
+	"reflect"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/strs"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	preg "google.golang.org/protobuf/reflect/protoregistry"
+	piface "google.golang.org/protobuf/runtime/protoiface"
+)
+
+// ValidationStatus is the result of validating the wire-format encoding of a message.
+type ValidationStatus int
+
+const (
+	// ValidationUnknown indicates that unmarshaling the message might succeed or fail.
+	// The validator was unable to render a judgement.
+	//
+	// The only causes of this status are an aberrant message type appearing somewhere
+	// in the message or a failure in the extension resolver.
+	ValidationUnknown ValidationStatus = iota + 1
+
+	// ValidationInvalid indicates that unmarshaling the message will fail.
+	ValidationInvalid
+
+	// ValidationValid indicates that unmarshaling the message will succeed.
+	ValidationValid
+)
+
+func (v ValidationStatus) String() string {
+	switch v {
+	case ValidationUnknown:
+		return "ValidationUnknown"
+	case ValidationInvalid:
+		return "ValidationInvalid"
+	case ValidationValid:
+		return "ValidationValid"
+	default:
+		return fmt.Sprintf("ValidationStatus(%d)", int(v))
+	}
+}
+
+// Validate determines whether the contents of the buffer are a valid wire encoding
+// of the message type.
+//
+// This function is exposed for testing.
+func Validate(mt pref.MessageType, in piface.UnmarshalInput) (out piface.UnmarshalOutput, _ ValidationStatus) {
+	mi, ok := mt.(*MessageInfo)
+	if !ok {
+		return out, ValidationUnknown
+	}
+	if in.Resolver == nil {
+		in.Resolver = preg.GlobalTypes
+	}
+	o, st := mi.validate(in.Buf, 0, unmarshalOptions{
+		flags:    in.Flags,
+		resolver: in.Resolver,
+	})
+	if o.initialized {
+		out.Flags |= piface.UnmarshalInitialized
+	}
+	return out, st
+}
+
+type validationInfo struct {
+	mi               *MessageInfo
+	typ              validationType
+	keyType, valType validationType
+
+	// For non-required fields, requiredBit is 0.
+	//
+	// For required fields, requiredBit's nth bit is set, where n is a
+	// unique index in the range [0, MessageInfo.numRequiredFields).
+	//
+	// If there are more than 64 required fields, requiredBit is 0.
+	requiredBit uint64
+}
+
+type validationType uint8
+
+const (
+	validationTypeOther validationType = iota
+	validationTypeMessage
+	validationTypeGroup
+	validationTypeMap
+	validationTypeRepeatedVarint
+	validationTypeRepeatedFixed32
+	validationTypeRepeatedFixed64
+	validationTypeVarint
+	validationTypeFixed32
+	validationTypeFixed64
+	validationTypeBytes
+	validationTypeUTF8String
+	validationTypeMessageSetItem
+)
+
+func newFieldValidationInfo(mi *MessageInfo, si structInfo, fd pref.FieldDescriptor, ft reflect.Type) validationInfo {
+	var vi validationInfo
+	switch {
+	case fd.ContainingOneof() != nil && !fd.ContainingOneof().IsSynthetic():
+		switch fd.Kind() {
+		case pref.MessageKind:
+			vi.typ = validationTypeMessage
+			if ot, ok := si.oneofWrappersByNumber[fd.Number()]; ok {
+				vi.mi = getMessageInfo(ot.Field(0).Type)
+			}
+		case pref.GroupKind:
+			vi.typ = validationTypeGroup
+			if ot, ok := si.oneofWrappersByNumber[fd.Number()]; ok {
+				vi.mi = getMessageInfo(ot.Field(0).Type)
+			}
+		case pref.StringKind:
+			if strs.EnforceUTF8(fd) {
+				vi.typ = validationTypeUTF8String
+			}
+		}
+	default:
+		vi = newValidationInfo(fd, ft)
+	}
+	if fd.Cardinality() == pref.Required {
+		// Avoid overflow. The required field check is done with a 64-bit mask, with
+		// any message containing more than 64 required fields always reported as
+		// potentially uninitialized, so it is not important to get a precise count
+		// of the required fields past 64.
+		if mi.numRequiredFields < math.MaxUint8 {
+			mi.numRequiredFields++
+			vi.requiredBit = 1 << (mi.numRequiredFields - 1)
+		}
+	}
+	return vi
+}
+
+func newValidationInfo(fd pref.FieldDescriptor, ft reflect.Type) validationInfo {
+	var vi validationInfo
+	switch {
+	case fd.IsList():
+		switch fd.Kind() {
+		case pref.MessageKind:
+			vi.typ = validationTypeMessage
+			if ft.Kind() == reflect.Slice {
+				vi.mi = getMessageInfo(ft.Elem())
+			}
+		case pref.GroupKind:
+			vi.typ = validationTypeGroup
+			if ft.Kind() == reflect.Slice {
+				vi.mi = getMessageInfo(ft.Elem())
+			}
+		case pref.StringKind:
+			vi.typ = validationTypeBytes
+			if strs.EnforceUTF8(fd) {
+				vi.typ = validationTypeUTF8String
+			}
+		default:
+			switch wireTypes[fd.Kind()] {
+			case protowire.VarintType:
+				vi.typ = validationTypeRepeatedVarint
+			case protowire.Fixed32Type:
+				vi.typ = validationTypeRepeatedFixed32
+			case protowire.Fixed64Type:
+				vi.typ = validationTypeRepeatedFixed64
+			}
+		}
+	case fd.IsMap():
+		vi.typ = validationTypeMap
+		switch fd.MapKey().Kind() {
+		case pref.StringKind:
+			if strs.EnforceUTF8(fd) {
+				vi.keyType = validationTypeUTF8String
+			}
+		}
+		switch fd.MapValue().Kind() {
+		case pref.MessageKind:
+			vi.valType = validationTypeMessage
+			if ft.Kind() == reflect.Map {
+				vi.mi = getMessageInfo(ft.Elem())
+			}
+		case pref.StringKind:
+			if strs.EnforceUTF8(fd) {
+				vi.valType = validationTypeUTF8String
+			}
+		}
+	default:
+		switch fd.Kind() {
+		case pref.MessageKind:
+			vi.typ = validationTypeMessage
+			if !fd.IsWeak() {
+				vi.mi = getMessageInfo(ft)
+			}
+		case pref.GroupKind:
+			vi.typ = validationTypeGroup
+			vi.mi = getMessageInfo(ft)
+		case pref.StringKind:
+			vi.typ = validationTypeBytes
+			if strs.EnforceUTF8(fd) {
+				vi.typ = validationTypeUTF8String
+			}
+		default:
+			switch wireTypes[fd.Kind()] {
+			case protowire.VarintType:
+				vi.typ = validationTypeVarint
+			case protowire.Fixed32Type:
+				vi.typ = validationTypeFixed32
+			case protowire.Fixed64Type:
+				vi.typ = validationTypeFixed64
+			case protowire.BytesType:
+				vi.typ = validationTypeBytes
+			}
+		}
+	}
+	return vi
+}
+
+func (mi *MessageInfo) validate(b []byte, groupTag protowire.Number, opts unmarshalOptions) (out unmarshalOutput, result ValidationStatus) {
+	mi.init()
+	type validationState struct {
+		typ              validationType
+		keyType, valType validationType
+		endGroup         protowire.Number
+		mi               *MessageInfo
+		tail             []byte
+		requiredMask     uint64
+	}
+
+	// Pre-allocate some slots to avoid repeated slice reallocation.
+	states := make([]validationState, 0, 16)
+	states = append(states, validationState{
+		typ: validationTypeMessage,
+		mi:  mi,
+	})
+	if groupTag > 0 {
+		states[0].typ = validationTypeGroup
+		states[0].endGroup = groupTag
+	}
+	initialized := true
+	start := len(b)
+State:
+	for len(states) > 0 {
+		st := &states[len(states)-1]
+		for len(b) > 0 {
+			// Parse the tag (field number and wire type).
+			var tag uint64
+			if b[0] < 0x80 {
+				tag = uint64(b[0])
+				b = b[1:]
+			} else if len(b) >= 2 && b[1] < 128 {
+				tag = uint64(b[0]&0x7f) + uint64(b[1])<<7
+				b = b[2:]
+			} else {
+				var n int
+				tag, n = protowire.ConsumeVarint(b)
+				if n < 0 {
+					return out, ValidationInvalid
+				}
+				b = b[n:]
+			}
+			var num protowire.Number
+			if n := tag >> 3; n < uint64(protowire.MinValidNumber) || n > uint64(protowire.MaxValidNumber) {
+				return out, ValidationInvalid
+			} else {
+				num = protowire.Number(n)
+			}
+			wtyp := protowire.Type(tag & 7)
+
+			if wtyp == protowire.EndGroupType {
+				if st.endGroup == num {
+					goto PopState
+				}
+				return out, ValidationInvalid
+			}
+			var vi validationInfo
+			switch {
+			case st.typ == validationTypeMap:
+				switch num {
+				case genid.MapEntry_Key_field_number:
+					vi.typ = st.keyType
+				case genid.MapEntry_Value_field_number:
+					vi.typ = st.valType
+					vi.mi = st.mi
+					vi.requiredBit = 1
+				}
+			case flags.ProtoLegacy && st.mi.isMessageSet:
+				switch num {
+				case messageset.FieldItem:
+					vi.typ = validationTypeMessageSetItem
+				}
+			default:
+				var f *coderFieldInfo
+				if int(num) < len(st.mi.denseCoderFields) {
+					f = st.mi.denseCoderFields[num]
+				} else {
+					f = st.mi.coderFields[num]
+				}
+				if f != nil {
+					vi = f.validation
+					if vi.typ == validationTypeMessage && vi.mi == nil {
+						// Probable weak field.
+						//
+						// TODO: Consider storing the results of this lookup somewhere
+						// rather than recomputing it on every validation.
+						fd := st.mi.Desc.Fields().ByNumber(num)
+						if fd == nil || !fd.IsWeak() {
+							break
+						}
+						messageName := fd.Message().FullName()
+						messageType, err := preg.GlobalTypes.FindMessageByName(messageName)
+						switch err {
+						case nil:
+							vi.mi, _ = messageType.(*MessageInfo)
+						case preg.NotFound:
+							vi.typ = validationTypeBytes
+						default:
+							return out, ValidationUnknown
+						}
+					}
+					break
+				}
+				// Possible extension field.
+				//
+				// TODO: We should return ValidationUnknown when:
+				//   1. The resolver is not frozen. (More extensions may be added to it.)
+				//   2. The resolver returns preg.NotFound.
+				// In this case, a type added to the resolver in the future could cause
+				// unmarshaling to begin failing. Supporting this requires some way to
+				// determine if the resolver is frozen.
+				xt, err := opts.resolver.FindExtensionByNumber(st.mi.Desc.FullName(), num)
+				if err != nil && err != preg.NotFound {
+					return out, ValidationUnknown
+				}
+				if err == nil {
+					vi = getExtensionFieldInfo(xt).validation
+				}
+			}
+			if vi.requiredBit != 0 {
+				// Check that the field has a compatible wire type.
+				// We only need to consider non-repeated field types,
+				// since repeated fields (and maps) can never be required.
+				ok := false
+				switch vi.typ {
+				case validationTypeVarint:
+					ok = wtyp == protowire.VarintType
+				case validationTypeFixed32:
+					ok = wtyp == protowire.Fixed32Type
+				case validationTypeFixed64:
+					ok = wtyp == protowire.Fixed64Type
+				case validationTypeBytes, validationTypeUTF8String, validationTypeMessage:
+					ok = wtyp == protowire.BytesType
+				case validationTypeGroup:
+					ok = wtyp == protowire.StartGroupType
+				}
+				if ok {
+					st.requiredMask |= vi.requiredBit
+				}
+			}
+
+			switch wtyp {
+			case protowire.VarintType:
+				if len(b) >= 10 {
+					switch {
+					case b[0] < 0x80:
+						b = b[1:]
+					case b[1] < 0x80:
+						b = b[2:]
+					case b[2] < 0x80:
+						b = b[3:]
+					case b[3] < 0x80:
+						b = b[4:]
+					case b[4] < 0x80:
+						b = b[5:]
+					case b[5] < 0x80:
+						b = b[6:]
+					case b[6] < 0x80:
+						b = b[7:]
+					case b[7] < 0x80:
+						b = b[8:]
+					case b[8] < 0x80:
+						b = b[9:]
+					case b[9] < 0x80 && b[9] < 2:
+						b = b[10:]
+					default:
+						return out, ValidationInvalid
+					}
+				} else {
+					switch {
+					case len(b) > 0 && b[0] < 0x80:
+						b = b[1:]
+					case len(b) > 1 && b[1] < 0x80:
+						b = b[2:]
+					case len(b) > 2 && b[2] < 0x80:
+						b = b[3:]
+					case len(b) > 3 && b[3] < 0x80:
+						b = b[4:]
+					case len(b) > 4 && b[4] < 0x80:
+						b = b[5:]
+					case len(b) > 5 && b[5] < 0x80:
+						b = b[6:]
+					case len(b) > 6 && b[6] < 0x80:
+						b = b[7:]
+					case len(b) > 7 && b[7] < 0x80:
+						b = b[8:]
+					case len(b) > 8 && b[8] < 0x80:
+						b = b[9:]
+					case len(b) > 9 && b[9] < 2:
+						b = b[10:]
+					default:
+						return out, ValidationInvalid
+					}
+				}
+				continue State
+			case protowire.BytesType:
+				var size uint64
+				if len(b) >= 1 && b[0] < 0x80 {
+					size = uint64(b[0])
+					b = b[1:]
+				} else if len(b) >= 2 && b[1] < 128 {
+					size = uint64(b[0]&0x7f) + uint64(b[1])<<7
+					b = b[2:]
+				} else {
+					var n int
+					size, n = protowire.ConsumeVarint(b)
+					if n < 0 {
+						return out, ValidationInvalid
+					}
+					b = b[n:]
+				}
+				if size > uint64(len(b)) {
+					return out, ValidationInvalid
+				}
+				v := b[:size]
+				b = b[size:]
+				switch vi.typ {
+				case validationTypeMessage:
+					if vi.mi == nil {
+						return out, ValidationUnknown
+					}
+					vi.mi.init()
+					fallthrough
+				case validationTypeMap:
+					if vi.mi != nil {
+						vi.mi.init()
+					}
+					states = append(states, validationState{
+						typ:     vi.typ,
+						keyType: vi.keyType,
+						valType: vi.valType,
+						mi:      vi.mi,
+						tail:    b,
+					})
+					b = v
+					continue State
+				case validationTypeRepeatedVarint:
+					// Packed field.
+					for len(v) > 0 {
+						_, n := protowire.ConsumeVarint(v)
+						if n < 0 {
+							return out, ValidationInvalid
+						}
+						v = v[n:]
+					}
+				case validationTypeRepeatedFixed32:
+					// Packed field.
+					if len(v)%4 != 0 {
+						return out, ValidationInvalid
+					}
+				case validationTypeRepeatedFixed64:
+					// Packed field.
+					if len(v)%8 != 0 {
+						return out, ValidationInvalid
+					}
+				case validationTypeUTF8String:
+					if !utf8.Valid(v) {
+						return out, ValidationInvalid
+					}
+				}
+			case protowire.Fixed32Type:
+				if len(b) < 4 {
+					return out, ValidationInvalid
+				}
+				b = b[4:]
+			case protowire.Fixed64Type:
+				if len(b) < 8 {
+					return out, ValidationInvalid
+				}
+				b = b[8:]
+			case protowire.StartGroupType:
+				switch {
+				case vi.typ == validationTypeGroup:
+					if vi.mi == nil {
+						return out, ValidationUnknown
+					}
+					vi.mi.init()
+					states = append(states, validationState{
+						typ:      validationTypeGroup,
+						mi:       vi.mi,
+						endGroup: num,
+					})
+					continue State
+				case flags.ProtoLegacy && vi.typ == validationTypeMessageSetItem:
+					typeid, v, n, err := messageset.ConsumeFieldValue(b, false)
+					if err != nil {
+						return out, ValidationInvalid
+					}
+					xt, err := opts.resolver.FindExtensionByNumber(st.mi.Desc.FullName(), typeid)
+					switch {
+					case err == preg.NotFound:
+						b = b[n:]
+					case err != nil:
+						return out, ValidationUnknown
+					default:
+						xvi := getExtensionFieldInfo(xt).validation
+						if xvi.mi != nil {
+							xvi.mi.init()
+						}
+						states = append(states, validationState{
+							typ:  xvi.typ,
+							mi:   xvi.mi,
+							tail: b[n:],
+						})
+						b = v
+						continue State
+					}
+				default:
+					n := protowire.ConsumeFieldValue(num, wtyp, b)
+					if n < 0 {
+						return out, ValidationInvalid
+					}
+					b = b[n:]
+				}
+			default:
+				return out, ValidationInvalid
+			}
+		}
+		if st.endGroup != 0 {
+			return out, ValidationInvalid
+		}
+		if len(b) != 0 {
+			return out, ValidationInvalid
+		}
+		b = st.tail
+	PopState:
+		numRequiredFields := 0
+		switch st.typ {
+		case validationTypeMessage, validationTypeGroup:
+			numRequiredFields = int(st.mi.numRequiredFields)
+		case validationTypeMap:
+			// If this is a map field with a message value that contains
+			// required fields, require that the value be present.
+			if st.mi != nil && st.mi.numRequiredFields > 0 {
+				numRequiredFields = 1
+			}
+		}
+		// If there are more than 64 required fields, this check will
+		// always fail and we will report that the message is potentially
+		// uninitialized.
+		if numRequiredFields > 0 && bits.OnesCount64(st.requiredMask) != numRequiredFields {
+			initialized = false
+		}
+		states = states[:len(states)-1]
+	}
+	out.n = start - len(b)
+	if initialized {
+		out.initialized = true
+	}
+	return out, ValidationValid
+}
diff --git a/vendor/google.golang.org/protobuf/internal/impl/weak.go b/vendor/google.golang.org/protobuf/internal/impl/weak.go
new file mode 100644
index 0000000..009cbef
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/impl/weak.go
@@ -0,0 +1,74 @@
+// 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.
+
+package impl
+
+import (
+	"fmt"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+// weakFields adds methods to the exported WeakFields type for internal use.
+//
+// The exported type is an alias to an unnamed type, so methods can't be
+// defined directly on it.
+type weakFields WeakFields
+
+func (w weakFields) get(num pref.FieldNumber) (pref.ProtoMessage, bool) {
+	m, ok := w[int32(num)]
+	return m, ok
+}
+
+func (w *weakFields) set(num pref.FieldNumber, m pref.ProtoMessage) {
+	if *w == nil {
+		*w = make(weakFields)
+	}
+	(*w)[int32(num)] = m
+}
+
+func (w *weakFields) clear(num pref.FieldNumber) {
+	delete(*w, int32(num))
+}
+
+func (Export) HasWeak(w WeakFields, num pref.FieldNumber) bool {
+	_, ok := w[int32(num)]
+	return ok
+}
+
+func (Export) ClearWeak(w *WeakFields, num pref.FieldNumber) {
+	delete(*w, int32(num))
+}
+
+func (Export) GetWeak(w WeakFields, num pref.FieldNumber, name pref.FullName) pref.ProtoMessage {
+	if m, ok := w[int32(num)]; ok {
+		return m
+	}
+	mt, _ := protoregistry.GlobalTypes.FindMessageByName(name)
+	if mt == nil {
+		panic(fmt.Sprintf("message %v for weak field is not linked in", name))
+	}
+	return mt.Zero().Interface()
+}
+
+func (Export) SetWeak(w *WeakFields, num pref.FieldNumber, name pref.FullName, m pref.ProtoMessage) {
+	if m != nil {
+		mt, _ := protoregistry.GlobalTypes.FindMessageByName(name)
+		if mt == nil {
+			panic(fmt.Sprintf("message %v for weak field is not linked in", name))
+		}
+		if mt != m.ProtoReflect().Type() {
+			panic(fmt.Sprintf("invalid message type for weak field: got %T, want %T", m, mt.Zero().Interface()))
+		}
+	}
+	if m == nil || !m.ProtoReflect().IsValid() {
+		delete(*w, int32(num))
+		return
+	}
+	if *w == nil {
+		*w = make(weakFields)
+	}
+	(*w)[int32(num)] = m
+}
diff --git a/vendor/google.golang.org/protobuf/internal/order/order.go b/vendor/google.golang.org/protobuf/internal/order/order.go
new file mode 100644
index 0000000..2a24953
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/order/order.go
@@ -0,0 +1,89 @@
+// Copyright 2020 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 order
+
+import (
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// FieldOrder specifies the ordering to visit message fields.
+// It is a function that reports whether x is ordered before y.
+type FieldOrder func(x, y pref.FieldDescriptor) bool
+
+var (
+	// AnyFieldOrder specifies no specific field ordering.
+	AnyFieldOrder FieldOrder = nil
+
+	// LegacyFieldOrder sorts fields in the same ordering as emitted by
+	// wire serialization in the github.com/golang/protobuf implementation.
+	LegacyFieldOrder FieldOrder = func(x, y pref.FieldDescriptor) bool {
+		ox, oy := x.ContainingOneof(), y.ContainingOneof()
+		inOneof := func(od pref.OneofDescriptor) bool {
+			return od != nil && !od.IsSynthetic()
+		}
+
+		// Extension fields sort before non-extension fields.
+		if x.IsExtension() != y.IsExtension() {
+			return x.IsExtension() && !y.IsExtension()
+		}
+		// Fields not within a oneof sort before those within a oneof.
+		if inOneof(ox) != inOneof(oy) {
+			return !inOneof(ox) && inOneof(oy)
+		}
+		// Fields in disjoint oneof sets are sorted by declaration index.
+		if ox != nil && oy != nil && ox != oy {
+			return ox.Index() < oy.Index()
+		}
+		// Fields sorted by field number.
+		return x.Number() < y.Number()
+	}
+
+	// NumberFieldOrder sorts fields by their field number.
+	NumberFieldOrder FieldOrder = func(x, y pref.FieldDescriptor) bool {
+		return x.Number() < y.Number()
+	}
+
+	// IndexNameFieldOrder sorts non-extension fields before extension fields.
+	// Non-extensions are sorted according to their declaration index.
+	// Extensions are sorted according to their full name.
+	IndexNameFieldOrder FieldOrder = func(x, y pref.FieldDescriptor) bool {
+		// Non-extension fields sort before extension fields.
+		if x.IsExtension() != y.IsExtension() {
+			return !x.IsExtension() && y.IsExtension()
+		}
+		// Extensions sorted by fullname.
+		if x.IsExtension() && y.IsExtension() {
+			return x.FullName() < y.FullName()
+		}
+		// Non-extensions sorted by declaration index.
+		return x.Index() < y.Index()
+	}
+)
+
+// KeyOrder specifies the ordering to visit map entries.
+// It is a function that reports whether x is ordered before y.
+type KeyOrder func(x, y pref.MapKey) bool
+
+var (
+	// AnyKeyOrder specifies no specific key ordering.
+	AnyKeyOrder KeyOrder = nil
+
+	// GenericKeyOrder sorts false before true, numeric keys in ascending order,
+	// and strings in lexicographical ordering according to UTF-8 codepoints.
+	GenericKeyOrder KeyOrder = func(x, y pref.MapKey) bool {
+		switch x.Interface().(type) {
+		case bool:
+			return !x.Bool() && y.Bool()
+		case int32, int64:
+			return x.Int() < y.Int()
+		case uint32, uint64:
+			return x.Uint() < y.Uint()
+		case string:
+			return x.String() < y.String()
+		default:
+			panic("invalid map key type")
+		}
+	}
+)
diff --git a/vendor/google.golang.org/protobuf/internal/order/range.go b/vendor/google.golang.org/protobuf/internal/order/range.go
new file mode 100644
index 0000000..c8090e0
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/order/range.go
@@ -0,0 +1,115 @@
+// Copyright 2020 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 order provides ordered access to messages and maps.
+package order
+
+import (
+	"sort"
+	"sync"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type messageField struct {
+	fd pref.FieldDescriptor
+	v  pref.Value
+}
+
+var messageFieldPool = sync.Pool{
+	New: func() interface{} { return new([]messageField) },
+}
+
+type (
+	// FieldRnger is an interface for visiting all fields in a message.
+	// The protoreflect.Message type implements this interface.
+	FieldRanger interface{ Range(VisitField) }
+	// VisitField is called everytime a message field is visited.
+	VisitField = func(pref.FieldDescriptor, pref.Value) bool
+)
+
+// RangeFields iterates over the fields of fs according to the specified order.
+func RangeFields(fs FieldRanger, less FieldOrder, fn VisitField) {
+	if less == nil {
+		fs.Range(fn)
+		return
+	}
+
+	// Obtain a pre-allocated scratch buffer.
+	p := messageFieldPool.Get().(*[]messageField)
+	fields := (*p)[:0]
+	defer func() {
+		if cap(fields) < 1024 {
+			*p = fields
+			messageFieldPool.Put(p)
+		}
+	}()
+
+	// Collect all fields in the message and sort them.
+	fs.Range(func(fd pref.FieldDescriptor, v pref.Value) bool {
+		fields = append(fields, messageField{fd, v})
+		return true
+	})
+	sort.Slice(fields, func(i, j int) bool {
+		return less(fields[i].fd, fields[j].fd)
+	})
+
+	// Visit the fields in the specified ordering.
+	for _, f := range fields {
+		if !fn(f.fd, f.v) {
+			return
+		}
+	}
+}
+
+type mapEntry struct {
+	k pref.MapKey
+	v pref.Value
+}
+
+var mapEntryPool = sync.Pool{
+	New: func() interface{} { return new([]mapEntry) },
+}
+
+type (
+	// EntryRanger is an interface for visiting all fields in a message.
+	// The protoreflect.Map type implements this interface.
+	EntryRanger interface{ Range(VisitEntry) }
+	// VisitEntry is called everytime a map entry is visited.
+	VisitEntry = func(pref.MapKey, pref.Value) bool
+)
+
+// RangeEntries iterates over the entries of es according to the specified order.
+func RangeEntries(es EntryRanger, less KeyOrder, fn VisitEntry) {
+	if less == nil {
+		es.Range(fn)
+		return
+	}
+
+	// Obtain a pre-allocated scratch buffer.
+	p := mapEntryPool.Get().(*[]mapEntry)
+	entries := (*p)[:0]
+	defer func() {
+		if cap(entries) < 1024 {
+			*p = entries
+			mapEntryPool.Put(p)
+		}
+	}()
+
+	// Collect all entries in the map and sort them.
+	es.Range(func(k pref.MapKey, v pref.Value) bool {
+		entries = append(entries, mapEntry{k, v})
+		return true
+	})
+	sort.Slice(entries, func(i, j int) bool {
+		return less(entries[i].k, entries[j].k)
+	})
+
+	// Visit the entries in the specified ordering.
+	for _, e := range entries {
+		if !fn(e.k, e.v) {
+			return
+		}
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/internal/pragma/pragma.go b/vendor/google.golang.org/protobuf/internal/pragma/pragma.go
new file mode 100644
index 0000000..49dc4fc
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/pragma/pragma.go
@@ -0,0 +1,29 @@
+// 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 pragma provides types that can be embedded into a struct to
+// statically enforce or prevent certain language properties.
+package pragma
+
+import "sync"
+
+// NoUnkeyedLiterals can be embedded in a struct to prevent unkeyed literals.
+type NoUnkeyedLiterals struct{}
+
+// DoNotImplement can be embedded in an interface to prevent trivial
+// implementations of the interface.
+//
+// This is useful to prevent unauthorized implementations of an interface
+// so that it can be extended in the future for any protobuf language changes.
+type DoNotImplement interface{ ProtoInternal(DoNotImplement) }
+
+// DoNotCompare can be embedded in a struct to prevent comparability.
+type DoNotCompare [0]func()
+
+// DoNotCopy can be embedded in a struct to help prevent shallow copies.
+// This does not rely on a Go language feature, but rather a special case
+// within the vet checker.
+//
+// See https://golang.org/issues/8005.
+type DoNotCopy [0]sync.Mutex
diff --git a/vendor/google.golang.org/protobuf/internal/set/ints.go b/vendor/google.golang.org/protobuf/internal/set/ints.go
new file mode 100644
index 0000000..d3d7f89
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/set/ints.go
@@ -0,0 +1,58 @@
+// 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 set provides simple set data structures for uint64s.
+package set
+
+import "math/bits"
+
+// int64s represents a set of integers within the range of 0..63.
+type int64s uint64
+
+func (bs *int64s) Len() int {
+	return bits.OnesCount64(uint64(*bs))
+}
+func (bs *int64s) Has(n uint64) bool {
+	return uint64(*bs)&(uint64(1)<<n) > 0
+}
+func (bs *int64s) Set(n uint64) {
+	*(*uint64)(bs) |= uint64(1) << n
+}
+func (bs *int64s) Clear(n uint64) {
+	*(*uint64)(bs) &^= uint64(1) << n
+}
+
+// Ints represents a set of integers within the range of 0..math.MaxUint64.
+type Ints struct {
+	lo int64s
+	hi map[uint64]struct{}
+}
+
+func (bs *Ints) Len() int {
+	return bs.lo.Len() + len(bs.hi)
+}
+func (bs *Ints) Has(n uint64) bool {
+	if n < 64 {
+		return bs.lo.Has(n)
+	}
+	_, ok := bs.hi[n]
+	return ok
+}
+func (bs *Ints) Set(n uint64) {
+	if n < 64 {
+		bs.lo.Set(n)
+		return
+	}
+	if bs.hi == nil {
+		bs.hi = make(map[uint64]struct{})
+	}
+	bs.hi[n] = struct{}{}
+}
+func (bs *Ints) Clear(n uint64) {
+	if n < 64 {
+		bs.lo.Clear(n)
+		return
+	}
+	delete(bs.hi, n)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/strs/strings.go b/vendor/google.golang.org/protobuf/internal/strs/strings.go
new file mode 100644
index 0000000..0b74e76
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/strs/strings.go
@@ -0,0 +1,196 @@
+// 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.
+
+// Package strs provides string manipulation functionality specific to protobuf.
+package strs
+
+import (
+	"go/token"
+	"strings"
+	"unicode"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// EnforceUTF8 reports whether to enforce strict UTF-8 validation.
+func EnforceUTF8(fd protoreflect.FieldDescriptor) bool {
+	if flags.ProtoLegacy {
+		if fd, ok := fd.(interface{ EnforceUTF8() bool }); ok {
+			return fd.EnforceUTF8()
+		}
+	}
+	return fd.Syntax() == protoreflect.Proto3
+}
+
+// GoCamelCase camel-cases a protobuf name for use as a Go identifier.
+//
+// If there is an interior underscore followed by a lower case letter,
+// drop the underscore and convert the letter to upper case.
+func GoCamelCase(s string) string {
+	// 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.
+	var b []byte
+	for i := 0; i < len(s); i++ {
+		c := s[i]
+		switch {
+		case c == '.' && i+1 < len(s) && isASCIILower(s[i+1]):
+			// Skip over '.' in ".{{lowercase}}".
+		case c == '.':
+			b = append(b, '_') // convert '.' to '_'
+		case c == '_' && (i == 0 || s[i-1] == '.'):
+			// Convert initial '_' to ensure we start with a capital letter.
+			// Do the same for '_' after '.' to match historic behavior.
+			b = append(b, 'X') // convert '_' to 'X'
+		case c == '_' && i+1 < len(s) && isASCIILower(s[i+1]):
+			// Skip over '_' in "_{{lowercase}}".
+		case isASCIIDigit(c):
+			b = append(b, c)
+		default:
+			// 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 -= 'a' - 'A' // convert lowercase to uppercase
+			}
+			b = append(b, c)
+
+			// Accept lower case sequence that follows.
+			for ; i+1 < len(s) && isASCIILower(s[i+1]); i++ {
+				b = append(b, s[i+1])
+			}
+		}
+	}
+	return string(b)
+}
+
+// GoSanitized converts a string to a valid Go identifier.
+func GoSanitized(s string) string {
+	// Sanitize the input to the set of valid characters,
+	// which must be '_' or be in the Unicode L or N categories.
+	s = strings.Map(func(r rune) rune {
+		if unicode.IsLetter(r) || unicode.IsDigit(r) {
+			return r
+		}
+		return '_'
+	}, s)
+
+	// Prepend '_' in the event of a Go keyword conflict or if
+	// the identifier is invalid (does not start in the Unicode L category).
+	r, _ := utf8.DecodeRuneInString(s)
+	if token.Lookup(s).IsKeyword() || !unicode.IsLetter(r) {
+		return "_" + s
+	}
+	return s
+}
+
+// JSONCamelCase converts a snake_case identifier to a camelCase identifier,
+// according to the protobuf JSON specification.
+func JSONCamelCase(s string) string {
+	var b []byte
+	var wasUnderscore bool
+	for i := 0; i < len(s); i++ { // proto identifiers are always ASCII
+		c := s[i]
+		if c != '_' {
+			if wasUnderscore && isASCIILower(c) {
+				c -= 'a' - 'A' // convert to uppercase
+			}
+			b = append(b, c)
+		}
+		wasUnderscore = c == '_'
+	}
+	return string(b)
+}
+
+// JSONSnakeCase converts a camelCase identifier to a snake_case identifier,
+// according to the protobuf JSON specification.
+func JSONSnakeCase(s string) string {
+	var b []byte
+	for i := 0; i < len(s); i++ { // proto identifiers are always ASCII
+		c := s[i]
+		if isASCIIUpper(c) {
+			b = append(b, '_')
+			c += 'a' - 'A' // convert to lowercase
+		}
+		b = append(b, c)
+	}
+	return string(b)
+}
+
+// MapEntryName derives the name of the map entry message given the field name.
+// See protoc v3.8.0: src/google/protobuf/descriptor.cc:254-276,6057
+func MapEntryName(s string) string {
+	var b []byte
+	upperNext := true
+	for _, c := range s {
+		switch {
+		case c == '_':
+			upperNext = true
+		case upperNext:
+			b = append(b, byte(unicode.ToUpper(c)))
+			upperNext = false
+		default:
+			b = append(b, byte(c))
+		}
+	}
+	b = append(b, "Entry"...)
+	return string(b)
+}
+
+// EnumValueName derives the camel-cased enum value name.
+// See protoc v3.8.0: src/google/protobuf/descriptor.cc:297-313
+func EnumValueName(s string) string {
+	var b []byte
+	upperNext := true
+	for _, c := range s {
+		switch {
+		case c == '_':
+			upperNext = true
+		case upperNext:
+			b = append(b, byte(unicode.ToUpper(c)))
+			upperNext = false
+		default:
+			b = append(b, byte(unicode.ToLower(c)))
+			upperNext = false
+		}
+	}
+	return string(b)
+}
+
+// TrimEnumPrefix trims the enum name prefix from an enum value name,
+// where the prefix is all lowercase without underscores.
+// See protoc v3.8.0: src/google/protobuf/descriptor.cc:330-375
+func TrimEnumPrefix(s, prefix string) string {
+	s0 := s // original input
+	for len(s) > 0 && len(prefix) > 0 {
+		if s[0] == '_' {
+			s = s[1:]
+			continue
+		}
+		if unicode.ToLower(rune(s[0])) != rune(prefix[0]) {
+			return s0 // no prefix match
+		}
+		s, prefix = s[1:], prefix[1:]
+	}
+	if len(prefix) > 0 {
+		return s0 // no prefix match
+	}
+	s = strings.TrimLeft(s, "_")
+	if len(s) == 0 {
+		return s0 // avoid returning empty string
+	}
+	return s
+}
+
+func isASCIILower(c byte) bool {
+	return 'a' <= c && c <= 'z'
+}
+func isASCIIUpper(c byte) bool {
+	return 'A' <= c && c <= 'Z'
+}
+func isASCIIDigit(c byte) bool {
+	return '0' <= c && c <= '9'
+}
diff --git a/vendor/google.golang.org/protobuf/internal/strs/strings_pure.go b/vendor/google.golang.org/protobuf/internal/strs/strings_pure.go
new file mode 100644
index 0000000..85e074c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/strs/strings_pure.go
@@ -0,0 +1,27 @@
+// 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 purego appengine
+
+package strs
+
+import pref "google.golang.org/protobuf/reflect/protoreflect"
+
+func UnsafeString(b []byte) string {
+	return string(b)
+}
+
+func UnsafeBytes(s string) []byte {
+	return []byte(s)
+}
+
+type Builder struct{}
+
+func (*Builder) AppendFullName(prefix pref.FullName, name pref.Name) pref.FullName {
+	return prefix.Append(name)
+}
+
+func (*Builder) MakeString(b []byte) string {
+	return string(b)
+}
diff --git a/vendor/google.golang.org/protobuf/internal/strs/strings_unsafe.go b/vendor/google.golang.org/protobuf/internal/strs/strings_unsafe.go
new file mode 100644
index 0000000..2160c70
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/strs/strings_unsafe.go
@@ -0,0 +1,94 @@
+// 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 !purego,!appengine
+
+package strs
+
+import (
+	"unsafe"
+
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+type (
+	stringHeader struct {
+		Data unsafe.Pointer
+		Len  int
+	}
+	sliceHeader struct {
+		Data unsafe.Pointer
+		Len  int
+		Cap  int
+	}
+)
+
+// UnsafeString returns an unsafe string reference of b.
+// The caller must treat the input slice as immutable.
+//
+// WARNING: Use carefully. The returned result must not leak to the end user
+// unless the input slice is provably immutable.
+func UnsafeString(b []byte) (s string) {
+	src := (*sliceHeader)(unsafe.Pointer(&b))
+	dst := (*stringHeader)(unsafe.Pointer(&s))
+	dst.Data = src.Data
+	dst.Len = src.Len
+	return s
+}
+
+// UnsafeBytes returns an unsafe bytes slice reference of s.
+// The caller must treat returned slice as immutable.
+//
+// WARNING: Use carefully. The returned result must not leak to the end user.
+func UnsafeBytes(s string) (b []byte) {
+	src := (*stringHeader)(unsafe.Pointer(&s))
+	dst := (*sliceHeader)(unsafe.Pointer(&b))
+	dst.Data = src.Data
+	dst.Len = src.Len
+	dst.Cap = src.Len
+	return b
+}
+
+// Builder builds a set of strings with shared lifetime.
+// This differs from strings.Builder, which is for building a single string.
+type Builder struct {
+	buf []byte
+}
+
+// AppendFullName is equivalent to protoreflect.FullName.Append,
+// but optimized for large batches where each name has a shared lifetime.
+func (sb *Builder) AppendFullName(prefix pref.FullName, name pref.Name) pref.FullName {
+	n := len(prefix) + len(".") + len(name)
+	if len(prefix) == 0 {
+		n -= len(".")
+	}
+	sb.grow(n)
+	sb.buf = append(sb.buf, prefix...)
+	sb.buf = append(sb.buf, '.')
+	sb.buf = append(sb.buf, name...)
+	return pref.FullName(sb.last(n))
+}
+
+// MakeString is equivalent to string(b), but optimized for large batches
+// with a shared lifetime.
+func (sb *Builder) MakeString(b []byte) string {
+	sb.grow(len(b))
+	sb.buf = append(sb.buf, b...)
+	return sb.last(len(b))
+}
+
+func (sb *Builder) grow(n int) {
+	if cap(sb.buf)-len(sb.buf) >= n {
+		return
+	}
+
+	// Unlike strings.Builder, we do not need to copy over the contents
+	// of the old buffer since our builder provides no API for
+	// retrieving previously created strings.
+	sb.buf = make([]byte, 2*(cap(sb.buf)+n))
+}
+
+func (sb *Builder) last(n int) string {
+	return UnsafeString(sb.buf[len(sb.buf)-n:])
+}
diff --git a/vendor/google.golang.org/protobuf/internal/version/version.go b/vendor/google.golang.org/protobuf/internal/version/version.go
new file mode 100644
index 0000000..14e774f
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/internal/version/version.go
@@ -0,0 +1,79 @@
+// 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.
+
+// Package version records versioning information about this module.
+package version
+
+import (
+	"fmt"
+	"strings"
+)
+
+// These constants determine the current version of this module.
+//
+//
+// For our release process, we enforce the following rules:
+//	* Tagged releases use a tag that is identical to String.
+//	* Tagged releases never reference a commit where the String
+//	contains "devel".
+//	* The set of all commits in this repository where String
+//	does not contain "devel" must have a unique String.
+//
+//
+// Steps for tagging a new release:
+//	1. Create a new CL.
+//
+//	2. Update Minor, Patch, and/or PreRelease as necessary.
+//	PreRelease must not contain the string "devel".
+//
+//	3. Since the last released minor version, have there been any changes to
+//	generator that relies on new functionality in the runtime?
+//	If yes, then increment RequiredGenerated.
+//
+//	4. Since the last released minor version, have there been any changes to
+//	the runtime that removes support for old .pb.go source code?
+//	If yes, then increment SupportMinimum.
+//
+//	5. Send out the CL for review and submit it.
+//	Note that the next CL in step 8 must be submitted after this CL
+//	without any other CLs in-between.
+//
+//	6. Tag a new version, where the tag is is the current String.
+//
+//	7. Write release notes for all notable changes
+//	between this release and the last release.
+//
+//	8. Create a new CL.
+//
+//	9. Update PreRelease to include the string "devel".
+//	For example: "" -> "devel" or "rc.1" -> "rc.1.devel"
+//
+//	10. Send out the CL for review and submit it.
+const (
+	Major      = 1
+	Minor      = 27
+	Patch      = 1
+	PreRelease = ""
+)
+
+// String formats the version string for this module in semver format.
+//
+// Examples:
+//	v1.20.1
+//	v1.21.0-rc.1
+func String() string {
+	v := fmt.Sprintf("v%d.%d.%d", Major, Minor, Patch)
+	if PreRelease != "" {
+		v += "-" + PreRelease
+
+		// TODO: Add metadata about the commit or build hash.
+		// See https://golang.org/issue/29814
+		// See https://golang.org/issue/33533
+		var metadata string
+		if strings.Contains(PreRelease, "devel") && metadata != "" {
+			v += "+" + metadata
+		}
+	}
+	return v
+}
diff --git a/vendor/google.golang.org/protobuf/proto/checkinit.go b/vendor/google.golang.org/protobuf/proto/checkinit.go
new file mode 100644
index 0000000..3e9a6a2
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/checkinit.go
@@ -0,0 +1,71 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// CheckInitialized returns an error if any required fields in m are not set.
+func CheckInitialized(m Message) error {
+	// Treat a nil message interface as an "untyped" empty message,
+	// which we assume to have no required fields.
+	if m == nil {
+		return nil
+	}
+
+	return checkInitialized(m.ProtoReflect())
+}
+
+// CheckInitialized returns an error if any required fields in m are not set.
+func checkInitialized(m protoreflect.Message) error {
+	if methods := protoMethods(m); methods != nil && methods.CheckInitialized != nil {
+		_, err := methods.CheckInitialized(protoiface.CheckInitializedInput{
+			Message: m,
+		})
+		return err
+	}
+	return checkInitializedSlow(m)
+}
+
+func checkInitializedSlow(m protoreflect.Message) error {
+	md := m.Descriptor()
+	fds := md.Fields()
+	for i, nums := 0, md.RequiredNumbers(); i < nums.Len(); i++ {
+		fd := fds.ByNumber(nums.Get(i))
+		if !m.Has(fd) {
+			return errors.RequiredNotSet(string(fd.FullName()))
+		}
+	}
+	var err error
+	m.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		switch {
+		case fd.IsList():
+			if fd.Message() == nil {
+				return true
+			}
+			for i, list := 0, v.List(); i < list.Len() && err == nil; i++ {
+				err = checkInitialized(list.Get(i).Message())
+			}
+		case fd.IsMap():
+			if fd.MapValue().Message() == nil {
+				return true
+			}
+			v.Map().Range(func(key protoreflect.MapKey, v protoreflect.Value) bool {
+				err = checkInitialized(v.Message())
+				return err == nil
+			})
+		default:
+			if fd.Message() == nil {
+				return true
+			}
+			err = checkInitialized(v.Message())
+		}
+		return err == nil
+	})
+	return err
+}
diff --git a/vendor/google.golang.org/protobuf/proto/decode.go b/vendor/google.golang.org/protobuf/proto/decode.go
new file mode 100644
index 0000000..49f9b8c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/decode.go
@@ -0,0 +1,278 @@
+// 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 proto
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// UnmarshalOptions configures the unmarshaler.
+//
+// Example usage:
+//   err := UnmarshalOptions{DiscardUnknown: true}.Unmarshal(b, m)
+type UnmarshalOptions struct {
+	pragma.NoUnkeyedLiterals
+
+	// Merge merges the input into the destination message.
+	// The default behavior is to always reset the message before unmarshaling,
+	// unless Merge is specified.
+	Merge bool
+
+	// AllowPartial accepts input for messages that will result in missing
+	// required fields. If AllowPartial is false (the default), Unmarshal will
+	// return an error if there are any missing required fields.
+	AllowPartial bool
+
+	// If DiscardUnknown is set, unknown fields are ignored.
+	DiscardUnknown bool
+
+	// Resolver is used for looking up types when unmarshaling extension fields.
+	// If nil, this defaults to using protoregistry.GlobalTypes.
+	Resolver interface {
+		FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error)
+		FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error)
+	}
+}
+
+// Unmarshal parses the wire-format message in b and places the result in m.
+// The provided message must be mutable (e.g., a non-nil pointer to a message).
+func Unmarshal(b []byte, m Message) error {
+	_, err := UnmarshalOptions{}.unmarshal(b, m.ProtoReflect())
+	return err
+}
+
+// Unmarshal parses the wire-format message in b and places the result in m.
+// The provided message must be mutable (e.g., a non-nil pointer to a message).
+func (o UnmarshalOptions) Unmarshal(b []byte, m Message) error {
+	_, err := o.unmarshal(b, m.ProtoReflect())
+	return err
+}
+
+// UnmarshalState parses a wire-format message and places the result in m.
+//
+// This method permits fine-grained control over the unmarshaler.
+// Most users should use Unmarshal instead.
+func (o UnmarshalOptions) UnmarshalState(in protoiface.UnmarshalInput) (protoiface.UnmarshalOutput, error) {
+	return o.unmarshal(in.Buf, in.Message)
+}
+
+// unmarshal is a centralized function that all unmarshal operations go through.
+// For profiling purposes, avoid changing the name of this function or
+// introducing other code paths for unmarshal that do not go through this.
+func (o UnmarshalOptions) unmarshal(b []byte, m protoreflect.Message) (out protoiface.UnmarshalOutput, err error) {
+	if o.Resolver == nil {
+		o.Resolver = protoregistry.GlobalTypes
+	}
+	if !o.Merge {
+		Reset(m.Interface())
+	}
+	allowPartial := o.AllowPartial
+	o.Merge = true
+	o.AllowPartial = true
+	methods := protoMethods(m)
+	if methods != nil && methods.Unmarshal != nil &&
+		!(o.DiscardUnknown && methods.Flags&protoiface.SupportUnmarshalDiscardUnknown == 0) {
+		in := protoiface.UnmarshalInput{
+			Message:  m,
+			Buf:      b,
+			Resolver: o.Resolver,
+		}
+		if o.DiscardUnknown {
+			in.Flags |= protoiface.UnmarshalDiscardUnknown
+		}
+		out, err = methods.Unmarshal(in)
+	} else {
+		err = o.unmarshalMessageSlow(b, m)
+	}
+	if err != nil {
+		return out, err
+	}
+	if allowPartial || (out.Flags&protoiface.UnmarshalInitialized != 0) {
+		return out, nil
+	}
+	return out, checkInitialized(m)
+}
+
+func (o UnmarshalOptions) unmarshalMessage(b []byte, m protoreflect.Message) error {
+	_, err := o.unmarshal(b, m)
+	return err
+}
+
+func (o UnmarshalOptions) unmarshalMessageSlow(b []byte, m protoreflect.Message) error {
+	md := m.Descriptor()
+	if messageset.IsMessageSet(md) {
+		return o.unmarshalMessageSet(b, m)
+	}
+	fields := md.Fields()
+	for len(b) > 0 {
+		// Parse the tag (field number and wire type).
+		num, wtyp, tagLen := protowire.ConsumeTag(b)
+		if tagLen < 0 {
+			return errDecode
+		}
+		if num > protowire.MaxValidNumber {
+			return errDecode
+		}
+
+		// Find the field descriptor for this field number.
+		fd := fields.ByNumber(num)
+		if fd == nil && md.ExtensionRanges().Has(num) {
+			extType, err := o.Resolver.FindExtensionByNumber(md.FullName(), num)
+			if err != nil && err != protoregistry.NotFound {
+				return errors.New("%v: unable to resolve extension %v: %v", md.FullName(), num, err)
+			}
+			if extType != nil {
+				fd = extType.TypeDescriptor()
+			}
+		}
+		var err error
+		if fd == nil {
+			err = errUnknown
+		} else if flags.ProtoLegacy {
+			if fd.IsWeak() && fd.Message().IsPlaceholder() {
+				err = errUnknown // weak referent is not linked in
+			}
+		}
+
+		// Parse the field value.
+		var valLen int
+		switch {
+		case err != nil:
+		case fd.IsList():
+			valLen, err = o.unmarshalList(b[tagLen:], wtyp, m.Mutable(fd).List(), fd)
+		case fd.IsMap():
+			valLen, err = o.unmarshalMap(b[tagLen:], wtyp, m.Mutable(fd).Map(), fd)
+		default:
+			valLen, err = o.unmarshalSingular(b[tagLen:], wtyp, m, fd)
+		}
+		if err != nil {
+			if err != errUnknown {
+				return err
+			}
+			valLen = protowire.ConsumeFieldValue(num, wtyp, b[tagLen:])
+			if valLen < 0 {
+				return errDecode
+			}
+			if !o.DiscardUnknown {
+				m.SetUnknown(append(m.GetUnknown(), b[:tagLen+valLen]...))
+			}
+		}
+		b = b[tagLen+valLen:]
+	}
+	return nil
+}
+
+func (o UnmarshalOptions) unmarshalSingular(b []byte, wtyp protowire.Type, m protoreflect.Message, fd protoreflect.FieldDescriptor) (n int, err error) {
+	v, n, err := o.unmarshalScalar(b, wtyp, fd)
+	if err != nil {
+		return 0, err
+	}
+	switch fd.Kind() {
+	case protoreflect.GroupKind, protoreflect.MessageKind:
+		m2 := m.Mutable(fd).Message()
+		if err := o.unmarshalMessage(v.Bytes(), m2); err != nil {
+			return n, err
+		}
+	default:
+		// Non-message scalars replace the previous value.
+		m.Set(fd, v)
+	}
+	return n, nil
+}
+
+func (o UnmarshalOptions) unmarshalMap(b []byte, wtyp protowire.Type, mapv protoreflect.Map, fd protoreflect.FieldDescriptor) (n int, err error) {
+	if wtyp != protowire.BytesType {
+		return 0, errUnknown
+	}
+	b, n = protowire.ConsumeBytes(b)
+	if n < 0 {
+		return 0, errDecode
+	}
+	var (
+		keyField = fd.MapKey()
+		valField = fd.MapValue()
+		key      protoreflect.Value
+		val      protoreflect.Value
+		haveKey  bool
+		haveVal  bool
+	)
+	switch valField.Kind() {
+	case protoreflect.GroupKind, protoreflect.MessageKind:
+		val = mapv.NewValue()
+	}
+	// Map entries are represented as a two-element message with fields
+	// containing the key and value.
+	for len(b) > 0 {
+		num, wtyp, n := protowire.ConsumeTag(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		if num > protowire.MaxValidNumber {
+			return 0, errDecode
+		}
+		b = b[n:]
+		err = errUnknown
+		switch num {
+		case genid.MapEntry_Key_field_number:
+			key, n, err = o.unmarshalScalar(b, wtyp, keyField)
+			if err != nil {
+				break
+			}
+			haveKey = true
+		case genid.MapEntry_Value_field_number:
+			var v protoreflect.Value
+			v, n, err = o.unmarshalScalar(b, wtyp, valField)
+			if err != nil {
+				break
+			}
+			switch valField.Kind() {
+			case protoreflect.GroupKind, protoreflect.MessageKind:
+				if err := o.unmarshalMessage(v.Bytes(), val.Message()); err != nil {
+					return 0, err
+				}
+			default:
+				val = v
+			}
+			haveVal = true
+		}
+		if err == errUnknown {
+			n = protowire.ConsumeFieldValue(num, wtyp, b)
+			if n < 0 {
+				return 0, errDecode
+			}
+		} else if err != nil {
+			return 0, err
+		}
+		b = b[n:]
+	}
+	// Every map entry should have entries for key and value, but this is not strictly required.
+	if !haveKey {
+		key = keyField.Default()
+	}
+	if !haveVal {
+		switch valField.Kind() {
+		case protoreflect.GroupKind, protoreflect.MessageKind:
+		default:
+			val = valField.Default()
+		}
+	}
+	mapv.Set(key.MapKey(), val)
+	return n, nil
+}
+
+// errUnknown is used internally to indicate fields which should be added
+// to the unknown field set of a message. It is never returned from an exported
+// function.
+var errUnknown = errors.New("BUG: internal error (unknown)")
+
+var errDecode = errors.New("cannot parse invalid wire-format data")
diff --git a/vendor/google.golang.org/protobuf/proto/decode_gen.go b/vendor/google.golang.org/protobuf/proto/decode_gen.go
new file mode 100644
index 0000000..301eeb2
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/decode_gen.go
@@ -0,0 +1,603 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package proto
+
+import (
+	"math"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// unmarshalScalar decodes a value of the given kind.
+//
+// Message values are decoded into a []byte which aliases the input data.
+func (o UnmarshalOptions) unmarshalScalar(b []byte, wtyp protowire.Type, fd protoreflect.FieldDescriptor) (val protoreflect.Value, n int, err error) {
+	switch fd.Kind() {
+	case protoreflect.BoolKind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfBool(protowire.DecodeBool(v)), n, nil
+	case protoreflect.EnumKind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)), n, nil
+	case protoreflect.Int32Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt32(int32(v)), n, nil
+	case protoreflect.Sint32Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))), n, nil
+	case protoreflect.Uint32Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfUint32(uint32(v)), n, nil
+	case protoreflect.Int64Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt64(int64(v)), n, nil
+	case protoreflect.Sint64Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)), n, nil
+	case protoreflect.Uint64Kind:
+		if wtyp != protowire.VarintType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfUint64(v), n, nil
+	case protoreflect.Sfixed32Kind:
+		if wtyp != protowire.Fixed32Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt32(int32(v)), n, nil
+	case protoreflect.Fixed32Kind:
+		if wtyp != protowire.Fixed32Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfUint32(uint32(v)), n, nil
+	case protoreflect.FloatKind:
+		if wtyp != protowire.Fixed32Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))), n, nil
+	case protoreflect.Sfixed64Kind:
+		if wtyp != protowire.Fixed64Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfInt64(int64(v)), n, nil
+	case protoreflect.Fixed64Kind:
+		if wtyp != protowire.Fixed64Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfUint64(v), n, nil
+	case protoreflect.DoubleKind:
+		if wtyp != protowire.Fixed64Type {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfFloat64(math.Float64frombits(v)), n, nil
+	case protoreflect.StringKind:
+		if wtyp != protowire.BytesType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		if strs.EnforceUTF8(fd) && !utf8.Valid(v) {
+			return protoreflect.Value{}, 0, errors.InvalidUTF8(string(fd.FullName()))
+		}
+		return protoreflect.ValueOfString(string(v)), n, nil
+	case protoreflect.BytesKind:
+		if wtyp != protowire.BytesType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfBytes(append(emptyBuf[:], v...)), n, nil
+	case protoreflect.MessageKind:
+		if wtyp != protowire.BytesType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfBytes(v), n, nil
+	case protoreflect.GroupKind:
+		if wtyp != protowire.StartGroupType {
+			return val, 0, errUnknown
+		}
+		v, n := protowire.ConsumeGroup(fd.Number(), b)
+		if n < 0 {
+			return val, 0, errDecode
+		}
+		return protoreflect.ValueOfBytes(v), n, nil
+	default:
+		return val, 0, errUnknown
+	}
+}
+
+func (o UnmarshalOptions) unmarshalList(b []byte, wtyp protowire.Type, list protoreflect.List, fd protoreflect.FieldDescriptor) (n int, err error) {
+	switch fd.Kind() {
+	case protoreflect.BoolKind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfBool(protowire.DecodeBool(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfBool(protowire.DecodeBool(v)))
+		return n, nil
+	case protoreflect.EnumKind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfEnum(protoreflect.EnumNumber(v)))
+		return n, nil
+	case protoreflect.Int32Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt32(int32(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt32(int32(v)))
+		return n, nil
+	case protoreflect.Sint32Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt32(int32(protowire.DecodeZigZag(v & math.MaxUint32))))
+		return n, nil
+	case protoreflect.Uint32Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfUint32(uint32(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfUint32(uint32(v)))
+		return n, nil
+	case protoreflect.Int64Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt64(int64(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt64(int64(v)))
+		return n, nil
+	case protoreflect.Sint64Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt64(protowire.DecodeZigZag(v)))
+		return n, nil
+	case protoreflect.Uint64Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeVarint(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfUint64(v))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.VarintType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeVarint(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfUint64(v))
+		return n, nil
+	case protoreflect.Sfixed32Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed32(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt32(int32(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed32Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt32(int32(v)))
+		return n, nil
+	case protoreflect.Fixed32Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed32(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfUint32(uint32(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed32Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfUint32(uint32(v)))
+		return n, nil
+	case protoreflect.FloatKind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed32(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed32Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed32(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfFloat32(math.Float32frombits(uint32(v))))
+		return n, nil
+	case protoreflect.Sfixed64Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed64(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfInt64(int64(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed64Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfInt64(int64(v)))
+		return n, nil
+	case protoreflect.Fixed64Kind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed64(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfUint64(v))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed64Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfUint64(v))
+		return n, nil
+	case protoreflect.DoubleKind:
+		if wtyp == protowire.BytesType {
+			buf, n := protowire.ConsumeBytes(b)
+			if n < 0 {
+				return 0, errDecode
+			}
+			for len(buf) > 0 {
+				v, n := protowire.ConsumeFixed64(buf)
+				if n < 0 {
+					return 0, errDecode
+				}
+				buf = buf[n:]
+				list.Append(protoreflect.ValueOfFloat64(math.Float64frombits(v)))
+			}
+			return n, nil
+		}
+		if wtyp != protowire.Fixed64Type {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeFixed64(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfFloat64(math.Float64frombits(v)))
+		return n, nil
+	case protoreflect.StringKind:
+		if wtyp != protowire.BytesType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		if strs.EnforceUTF8(fd) && !utf8.Valid(v) {
+			return 0, errors.InvalidUTF8(string(fd.FullName()))
+		}
+		list.Append(protoreflect.ValueOfString(string(v)))
+		return n, nil
+	case protoreflect.BytesKind:
+		if wtyp != protowire.BytesType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		list.Append(protoreflect.ValueOfBytes(append(emptyBuf[:], v...)))
+		return n, nil
+	case protoreflect.MessageKind:
+		if wtyp != protowire.BytesType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeBytes(b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		m := list.NewElement()
+		if err := o.unmarshalMessage(v, m.Message()); err != nil {
+			return 0, err
+		}
+		list.Append(m)
+		return n, nil
+	case protoreflect.GroupKind:
+		if wtyp != protowire.StartGroupType {
+			return 0, errUnknown
+		}
+		v, n := protowire.ConsumeGroup(fd.Number(), b)
+		if n < 0 {
+			return 0, errDecode
+		}
+		m := list.NewElement()
+		if err := o.unmarshalMessage(v, m.Message()); err != nil {
+			return 0, err
+		}
+		list.Append(m)
+		return n, nil
+	default:
+		return 0, errUnknown
+	}
+}
+
+// We append to an empty array rather than a nil []byte to get non-nil zero-length byte slices.
+var emptyBuf [0]byte
diff --git a/vendor/google.golang.org/protobuf/proto/doc.go b/vendor/google.golang.org/protobuf/proto/doc.go
new file mode 100644
index 0000000..c52d8c4
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/doc.go
@@ -0,0 +1,94 @@
+// 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.
+
+// Package proto provides functions operating on protocol buffer messages.
+//
+// For documentation on protocol buffers in general, see:
+//
+//   https://developers.google.com/protocol-buffers
+//
+// For a tutorial on using protocol buffers with Go, see:
+//
+//   https://developers.google.com/protocol-buffers/docs/gotutorial
+//
+// For a guide to generated Go protocol buffer code, see:
+//
+//   https://developers.google.com/protocol-buffers/docs/reference/go-generated
+//
+//
+// Binary serialization
+//
+// This package contains functions to convert to and from the wire format,
+// an efficient binary serialization of protocol buffers.
+//
+// • Size reports the size of a message in the wire format.
+//
+// • Marshal converts a message to the wire format.
+// The MarshalOptions type provides more control over wire marshaling.
+//
+// • Unmarshal converts a message from the wire format.
+// The UnmarshalOptions type provides more control over wire unmarshaling.
+//
+//
+// Basic message operations
+//
+// • Clone makes a deep copy of a message.
+//
+// • Merge merges the content of a message into another.
+//
+// • Equal compares two messages. For more control over comparisons
+// and detailed reporting of differences, see package
+// "google.golang.org/protobuf/testing/protocmp".
+//
+// • Reset clears the content of a message.
+//
+// • CheckInitialized reports whether all required fields in a message are set.
+//
+//
+// Optional scalar constructors
+//
+// The API for some generated messages represents optional scalar fields
+// as pointers to a value. For example, an optional string field has the
+// Go type *string.
+//
+// • Bool, Int32, Int64, Uint32, Uint64, Float32, Float64, and String
+// take a value and return a pointer to a new instance of it,
+// to simplify construction of optional field values.
+//
+// Generated enum types usually have an Enum method which performs the
+// same operation.
+//
+// Optional scalar fields are only supported in proto2.
+//
+//
+// Extension accessors
+//
+// • HasExtension, GetExtension, SetExtension, and ClearExtension
+// access extension field values in a protocol buffer message.
+//
+// Extension fields are only supported in proto2.
+//
+//
+// Related packages
+//
+// • Package "google.golang.org/protobuf/encoding/protojson" converts messages to
+// and from JSON.
+//
+// • Package "google.golang.org/protobuf/encoding/prototext" converts messages to
+// and from the text format.
+//
+// • Package "google.golang.org/protobuf/reflect/protoreflect" provides a
+// reflection interface for protocol buffer data types.
+//
+// • Package "google.golang.org/protobuf/testing/protocmp" provides features
+// to compare protocol buffer messages with the "github.com/google/go-cmp/cmp"
+// package.
+//
+// • Package "google.golang.org/protobuf/types/dynamicpb" provides a dynamic
+// message type, suitable for working with messages where the protocol buffer
+// type is only known at runtime.
+//
+// This module contains additional packages for more specialized use cases.
+// Consult the individual package documentation for details.
+package proto
diff --git a/vendor/google.golang.org/protobuf/proto/encode.go b/vendor/google.golang.org/protobuf/proto/encode.go
new file mode 100644
index 0000000..d18239c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/encode.go
@@ -0,0 +1,319 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/order"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// MarshalOptions configures the marshaler.
+//
+// Example usage:
+//   b, err := MarshalOptions{Deterministic: true}.Marshal(m)
+type MarshalOptions struct {
+	pragma.NoUnkeyedLiterals
+
+	// AllowPartial allows messages that have missing required fields to marshal
+	// without returning an error. If AllowPartial is false (the default),
+	// Marshal will return an error if there are any missing required fields.
+	AllowPartial bool
+
+	// Deterministic controls whether the same message will always be
+	// serialized to the same bytes within the same binary.
+	//
+	// Setting this option guarantees that repeated serialization of
+	// the same message will return the same bytes, and that different
+	// processes of the same binary (which may be executing on different
+	// machines) will serialize equal messages to the same bytes.
+	// It has no effect on the resulting size of the encoded message compared
+	// to a non-deterministic marshal.
+	//
+	// Note that the deterministic serialization is NOT canonical across
+	// languages. It is not guaranteed to remain stable over time. It is
+	// unstable across different builds with schema changes due to unknown
+	// fields. Users who need canonical serialization (e.g., persistent
+	// storage in a canonical form, fingerprinting, etc.) must define
+	// their own canonicalization specification and implement their own
+	// serializer rather than relying on this API.
+	//
+	// If deterministic serialization is requested, map entries will be
+	// sorted by keys in lexographical order. This is an implementation
+	// detail and subject to change.
+	Deterministic bool
+
+	// UseCachedSize indicates that the result of a previous Size call
+	// may be reused.
+	//
+	// Setting this option asserts that:
+	//
+	// 1. Size has previously been called on this message with identical
+	// options (except for UseCachedSize itself).
+	//
+	// 2. The message and all its submessages have not changed in any
+	// way since the Size call.
+	//
+	// If either of these invariants is violated,
+	// the results are undefined and may include panics or corrupted output.
+	//
+	// Implementations MAY take this option into account to provide
+	// better performance, but there is no guarantee that they will do so.
+	// There is absolutely no guarantee that Size followed by Marshal with
+	// UseCachedSize set will perform equivalently to Marshal alone.
+	UseCachedSize bool
+}
+
+// Marshal returns the wire-format encoding of m.
+func Marshal(m Message) ([]byte, error) {
+	// Treat nil message interface as an empty message; nothing to output.
+	if m == nil {
+		return nil, nil
+	}
+
+	out, err := MarshalOptions{}.marshal(nil, m.ProtoReflect())
+	if len(out.Buf) == 0 && err == nil {
+		out.Buf = emptyBytesForMessage(m)
+	}
+	return out.Buf, err
+}
+
+// Marshal returns the wire-format encoding of m.
+func (o MarshalOptions) Marshal(m Message) ([]byte, error) {
+	// Treat nil message interface as an empty message; nothing to output.
+	if m == nil {
+		return nil, nil
+	}
+
+	out, err := o.marshal(nil, m.ProtoReflect())
+	if len(out.Buf) == 0 && err == nil {
+		out.Buf = emptyBytesForMessage(m)
+	}
+	return out.Buf, err
+}
+
+// emptyBytesForMessage returns a nil buffer if and only if m is invalid,
+// otherwise it returns a non-nil empty buffer.
+//
+// This is to assist the edge-case where user-code does the following:
+//	m1.OptionalBytes, _ = proto.Marshal(m2)
+// where they expect the proto2 "optional_bytes" field to be populated
+// if any only if m2 is a valid message.
+func emptyBytesForMessage(m Message) []byte {
+	if m == nil || !m.ProtoReflect().IsValid() {
+		return nil
+	}
+	return emptyBuf[:]
+}
+
+// MarshalAppend appends the wire-format encoding of m to b,
+// returning the result.
+func (o MarshalOptions) MarshalAppend(b []byte, m Message) ([]byte, error) {
+	// Treat nil message interface as an empty message; nothing to append.
+	if m == nil {
+		return b, nil
+	}
+
+	out, err := o.marshal(b, m.ProtoReflect())
+	return out.Buf, err
+}
+
+// MarshalState returns the wire-format encoding of a message.
+//
+// This method permits fine-grained control over the marshaler.
+// Most users should use Marshal instead.
+func (o MarshalOptions) MarshalState(in protoiface.MarshalInput) (protoiface.MarshalOutput, error) {
+	return o.marshal(in.Buf, in.Message)
+}
+
+// marshal is a centralized function that all marshal operations go through.
+// For profiling purposes, avoid changing the name of this function or
+// introducing other code paths for marshal that do not go through this.
+func (o MarshalOptions) marshal(b []byte, m protoreflect.Message) (out protoiface.MarshalOutput, err error) {
+	allowPartial := o.AllowPartial
+	o.AllowPartial = true
+	if methods := protoMethods(m); methods != nil && methods.Marshal != nil &&
+		!(o.Deterministic && methods.Flags&protoiface.SupportMarshalDeterministic == 0) {
+		in := protoiface.MarshalInput{
+			Message: m,
+			Buf:     b,
+		}
+		if o.Deterministic {
+			in.Flags |= protoiface.MarshalDeterministic
+		}
+		if o.UseCachedSize {
+			in.Flags |= protoiface.MarshalUseCachedSize
+		}
+		if methods.Size != nil {
+			sout := methods.Size(protoiface.SizeInput{
+				Message: m,
+				Flags:   in.Flags,
+			})
+			if cap(b) < len(b)+sout.Size {
+				in.Buf = make([]byte, len(b), growcap(cap(b), len(b)+sout.Size))
+				copy(in.Buf, b)
+			}
+			in.Flags |= protoiface.MarshalUseCachedSize
+		}
+		out, err = methods.Marshal(in)
+	} else {
+		out.Buf, err = o.marshalMessageSlow(b, m)
+	}
+	if err != nil {
+		return out, err
+	}
+	if allowPartial {
+		return out, nil
+	}
+	return out, checkInitialized(m)
+}
+
+func (o MarshalOptions) marshalMessage(b []byte, m protoreflect.Message) ([]byte, error) {
+	out, err := o.marshal(b, m)
+	return out.Buf, err
+}
+
+// growcap scales up the capacity of a slice.
+//
+// Given a slice with a current capacity of oldcap and a desired
+// capacity of wantcap, growcap returns a new capacity >= wantcap.
+//
+// The algorithm is mostly identical to the one used by append as of Go 1.14.
+func growcap(oldcap, wantcap int) (newcap int) {
+	if wantcap > oldcap*2 {
+		newcap = wantcap
+	} else if oldcap < 1024 {
+		// The Go 1.14 runtime takes this case when len(s) < 1024,
+		// not when cap(s) < 1024. The difference doesn't seem
+		// significant here.
+		newcap = oldcap * 2
+	} else {
+		newcap = oldcap
+		for 0 < newcap && newcap < wantcap {
+			newcap += newcap / 4
+		}
+		if newcap <= 0 {
+			newcap = wantcap
+		}
+	}
+	return newcap
+}
+
+func (o MarshalOptions) marshalMessageSlow(b []byte, m protoreflect.Message) ([]byte, error) {
+	if messageset.IsMessageSet(m.Descriptor()) {
+		return o.marshalMessageSet(b, m)
+	}
+	fieldOrder := order.AnyFieldOrder
+	if o.Deterministic {
+		// TODO: This should use a more natural ordering like NumberFieldOrder,
+		// but doing so breaks golden tests that make invalid assumption about
+		// output stability of this implementation.
+		fieldOrder = order.LegacyFieldOrder
+	}
+	var err error
+	order.RangeFields(m, fieldOrder, func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		b, err = o.marshalField(b, fd, v)
+		return err == nil
+	})
+	if err != nil {
+		return b, err
+	}
+	b = append(b, m.GetUnknown()...)
+	return b, nil
+}
+
+func (o MarshalOptions) marshalField(b []byte, fd protoreflect.FieldDescriptor, value protoreflect.Value) ([]byte, error) {
+	switch {
+	case fd.IsList():
+		return o.marshalList(b, fd, value.List())
+	case fd.IsMap():
+		return o.marshalMap(b, fd, value.Map())
+	default:
+		b = protowire.AppendTag(b, fd.Number(), wireTypes[fd.Kind()])
+		return o.marshalSingular(b, fd, value)
+	}
+}
+
+func (o MarshalOptions) marshalList(b []byte, fd protoreflect.FieldDescriptor, list protoreflect.List) ([]byte, error) {
+	if fd.IsPacked() && list.Len() > 0 {
+		b = protowire.AppendTag(b, fd.Number(), protowire.BytesType)
+		b, pos := appendSpeculativeLength(b)
+		for i, llen := 0, list.Len(); i < llen; i++ {
+			var err error
+			b, err = o.marshalSingular(b, fd, list.Get(i))
+			if err != nil {
+				return b, err
+			}
+		}
+		b = finishSpeculativeLength(b, pos)
+		return b, nil
+	}
+
+	kind := fd.Kind()
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		var err error
+		b = protowire.AppendTag(b, fd.Number(), wireTypes[kind])
+		b, err = o.marshalSingular(b, fd, list.Get(i))
+		if err != nil {
+			return b, err
+		}
+	}
+	return b, nil
+}
+
+func (o MarshalOptions) marshalMap(b []byte, fd protoreflect.FieldDescriptor, mapv protoreflect.Map) ([]byte, error) {
+	keyf := fd.MapKey()
+	valf := fd.MapValue()
+	keyOrder := order.AnyKeyOrder
+	if o.Deterministic {
+		keyOrder = order.GenericKeyOrder
+	}
+	var err error
+	order.RangeEntries(mapv, keyOrder, func(key protoreflect.MapKey, value protoreflect.Value) bool {
+		b = protowire.AppendTag(b, fd.Number(), protowire.BytesType)
+		var pos int
+		b, pos = appendSpeculativeLength(b)
+
+		b, err = o.marshalField(b, keyf, key.Value())
+		if err != nil {
+			return false
+		}
+		b, err = o.marshalField(b, valf, value)
+		if err != nil {
+			return false
+		}
+		b = finishSpeculativeLength(b, pos)
+		return true
+	})
+	return b, err
+}
+
+// When encoding length-prefixed fields, we speculatively set aside some number of bytes
+// for the length, encode the data, and then encode the length (shifting the data if necessary
+// to make room).
+const speculativeLength = 1
+
+func appendSpeculativeLength(b []byte) ([]byte, int) {
+	pos := len(b)
+	b = append(b, "\x00\x00\x00\x00"[:speculativeLength]...)
+	return b, pos
+}
+
+func finishSpeculativeLength(b []byte, pos int) []byte {
+	mlen := len(b) - pos - speculativeLength
+	msiz := protowire.SizeVarint(uint64(mlen))
+	if msiz != speculativeLength {
+		for i := 0; i < msiz-speculativeLength; i++ {
+			b = append(b, 0)
+		}
+		copy(b[pos+msiz:], b[pos+speculativeLength:])
+		b = b[:pos+msiz+mlen]
+	}
+	protowire.AppendVarint(b[:pos], uint64(mlen))
+	return b
+}
diff --git a/vendor/google.golang.org/protobuf/proto/encode_gen.go b/vendor/google.golang.org/protobuf/proto/encode_gen.go
new file mode 100644
index 0000000..185dacf
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/encode_gen.go
@@ -0,0 +1,97 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package proto
+
+import (
+	"math"
+	"unicode/utf8"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+var wireTypes = map[protoreflect.Kind]protowire.Type{
+	protoreflect.BoolKind:     protowire.VarintType,
+	protoreflect.EnumKind:     protowire.VarintType,
+	protoreflect.Int32Kind:    protowire.VarintType,
+	protoreflect.Sint32Kind:   protowire.VarintType,
+	protoreflect.Uint32Kind:   protowire.VarintType,
+	protoreflect.Int64Kind:    protowire.VarintType,
+	protoreflect.Sint64Kind:   protowire.VarintType,
+	protoreflect.Uint64Kind:   protowire.VarintType,
+	protoreflect.Sfixed32Kind: protowire.Fixed32Type,
+	protoreflect.Fixed32Kind:  protowire.Fixed32Type,
+	protoreflect.FloatKind:    protowire.Fixed32Type,
+	protoreflect.Sfixed64Kind: protowire.Fixed64Type,
+	protoreflect.Fixed64Kind:  protowire.Fixed64Type,
+	protoreflect.DoubleKind:   protowire.Fixed64Type,
+	protoreflect.StringKind:   protowire.BytesType,
+	protoreflect.BytesKind:    protowire.BytesType,
+	protoreflect.MessageKind:  protowire.BytesType,
+	protoreflect.GroupKind:    protowire.StartGroupType,
+}
+
+func (o MarshalOptions) marshalSingular(b []byte, fd protoreflect.FieldDescriptor, v protoreflect.Value) ([]byte, error) {
+	switch fd.Kind() {
+	case protoreflect.BoolKind:
+		b = protowire.AppendVarint(b, protowire.EncodeBool(v.Bool()))
+	case protoreflect.EnumKind:
+		b = protowire.AppendVarint(b, uint64(v.Enum()))
+	case protoreflect.Int32Kind:
+		b = protowire.AppendVarint(b, uint64(int32(v.Int())))
+	case protoreflect.Sint32Kind:
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(int64(int32(v.Int()))))
+	case protoreflect.Uint32Kind:
+		b = protowire.AppendVarint(b, uint64(uint32(v.Uint())))
+	case protoreflect.Int64Kind:
+		b = protowire.AppendVarint(b, uint64(v.Int()))
+	case protoreflect.Sint64Kind:
+		b = protowire.AppendVarint(b, protowire.EncodeZigZag(v.Int()))
+	case protoreflect.Uint64Kind:
+		b = protowire.AppendVarint(b, v.Uint())
+	case protoreflect.Sfixed32Kind:
+		b = protowire.AppendFixed32(b, uint32(v.Int()))
+	case protoreflect.Fixed32Kind:
+		b = protowire.AppendFixed32(b, uint32(v.Uint()))
+	case protoreflect.FloatKind:
+		b = protowire.AppendFixed32(b, math.Float32bits(float32(v.Float())))
+	case protoreflect.Sfixed64Kind:
+		b = protowire.AppendFixed64(b, uint64(v.Int()))
+	case protoreflect.Fixed64Kind:
+		b = protowire.AppendFixed64(b, v.Uint())
+	case protoreflect.DoubleKind:
+		b = protowire.AppendFixed64(b, math.Float64bits(v.Float()))
+	case protoreflect.StringKind:
+		if strs.EnforceUTF8(fd) && !utf8.ValidString(v.String()) {
+			return b, errors.InvalidUTF8(string(fd.FullName()))
+		}
+		b = protowire.AppendString(b, v.String())
+	case protoreflect.BytesKind:
+		b = protowire.AppendBytes(b, v.Bytes())
+	case protoreflect.MessageKind:
+		var pos int
+		var err error
+		b, pos = appendSpeculativeLength(b)
+		b, err = o.marshalMessage(b, v.Message())
+		if err != nil {
+			return b, err
+		}
+		b = finishSpeculativeLength(b, pos)
+	case protoreflect.GroupKind:
+		var err error
+		b, err = o.marshalMessage(b, v.Message())
+		if err != nil {
+			return b, err
+		}
+		b = protowire.AppendVarint(b, protowire.EncodeTag(fd.Number(), protowire.EndGroupType))
+	default:
+		return b, errors.New("invalid kind %v", fd.Kind())
+	}
+	return b, nil
+}
diff --git a/vendor/google.golang.org/protobuf/proto/equal.go b/vendor/google.golang.org/protobuf/proto/equal.go
new file mode 100644
index 0000000..4dba2b9
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/equal.go
@@ -0,0 +1,167 @@
+// 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.
+
+package proto
+
+import (
+	"bytes"
+	"math"
+	"reflect"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	pref "google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// Equal reports whether two messages are equal.
+// If two messages marshal to the same bytes under deterministic serialization,
+// then Equal is guaranteed to report true.
+//
+// Two messages are equal if they belong to the same message descriptor,
+// have the same set of populated known and extension field values,
+// and the same set of unknown fields values. If either of the top-level
+// messages are invalid, then Equal reports true only if both are invalid.
+//
+// Scalar values are compared with the equivalent of the == operator in Go,
+// except bytes values which are compared using bytes.Equal and
+// floating point values which specially treat NaNs as equal.
+// Message values are compared by recursively calling Equal.
+// Lists are equal if each element value is also equal.
+// Maps are equal if they have the same set of keys, where the pair of values
+// for each key is also equal.
+func Equal(x, y Message) bool {
+	if x == nil || y == nil {
+		return x == nil && y == nil
+	}
+	mx := x.ProtoReflect()
+	my := y.ProtoReflect()
+	if mx.IsValid() != my.IsValid() {
+		return false
+	}
+	return equalMessage(mx, my)
+}
+
+// equalMessage compares two messages.
+func equalMessage(mx, my pref.Message) bool {
+	if mx.Descriptor() != my.Descriptor() {
+		return false
+	}
+
+	nx := 0
+	equal := true
+	mx.Range(func(fd pref.FieldDescriptor, vx pref.Value) bool {
+		nx++
+		vy := my.Get(fd)
+		equal = my.Has(fd) && equalField(fd, vx, vy)
+		return equal
+	})
+	if !equal {
+		return false
+	}
+	ny := 0
+	my.Range(func(fd pref.FieldDescriptor, vx pref.Value) bool {
+		ny++
+		return true
+	})
+	if nx != ny {
+		return false
+	}
+
+	return equalUnknown(mx.GetUnknown(), my.GetUnknown())
+}
+
+// equalField compares two fields.
+func equalField(fd pref.FieldDescriptor, x, y pref.Value) bool {
+	switch {
+	case fd.IsList():
+		return equalList(fd, x.List(), y.List())
+	case fd.IsMap():
+		return equalMap(fd, x.Map(), y.Map())
+	default:
+		return equalValue(fd, x, y)
+	}
+}
+
+// equalMap compares two maps.
+func equalMap(fd pref.FieldDescriptor, x, y pref.Map) bool {
+	if x.Len() != y.Len() {
+		return false
+	}
+	equal := true
+	x.Range(func(k pref.MapKey, vx pref.Value) bool {
+		vy := y.Get(k)
+		equal = y.Has(k) && equalValue(fd.MapValue(), vx, vy)
+		return equal
+	})
+	return equal
+}
+
+// equalList compares two lists.
+func equalList(fd pref.FieldDescriptor, x, y pref.List) bool {
+	if x.Len() != y.Len() {
+		return false
+	}
+	for i := x.Len() - 1; i >= 0; i-- {
+		if !equalValue(fd, x.Get(i), y.Get(i)) {
+			return false
+		}
+	}
+	return true
+}
+
+// equalValue compares two singular values.
+func equalValue(fd pref.FieldDescriptor, x, y pref.Value) bool {
+	switch fd.Kind() {
+	case pref.BoolKind:
+		return x.Bool() == y.Bool()
+	case pref.EnumKind:
+		return x.Enum() == y.Enum()
+	case pref.Int32Kind, pref.Sint32Kind,
+		pref.Int64Kind, pref.Sint64Kind,
+		pref.Sfixed32Kind, pref.Sfixed64Kind:
+		return x.Int() == y.Int()
+	case pref.Uint32Kind, pref.Uint64Kind,
+		pref.Fixed32Kind, pref.Fixed64Kind:
+		return x.Uint() == y.Uint()
+	case pref.FloatKind, pref.DoubleKind:
+		fx := x.Float()
+		fy := y.Float()
+		if math.IsNaN(fx) || math.IsNaN(fy) {
+			return math.IsNaN(fx) && math.IsNaN(fy)
+		}
+		return fx == fy
+	case pref.StringKind:
+		return x.String() == y.String()
+	case pref.BytesKind:
+		return bytes.Equal(x.Bytes(), y.Bytes())
+	case pref.MessageKind, pref.GroupKind:
+		return equalMessage(x.Message(), y.Message())
+	default:
+		return x.Interface() == y.Interface()
+	}
+}
+
+// equalUnknown compares unknown fields by direct comparison on the raw bytes
+// of each individual field number.
+func equalUnknown(x, y pref.RawFields) bool {
+	if len(x) != len(y) {
+		return false
+	}
+	if bytes.Equal([]byte(x), []byte(y)) {
+		return true
+	}
+
+	mx := make(map[pref.FieldNumber]pref.RawFields)
+	my := make(map[pref.FieldNumber]pref.RawFields)
+	for len(x) > 0 {
+		fnum, _, n := protowire.ConsumeField(x)
+		mx[fnum] = append(mx[fnum], x[:n]...)
+		x = x[n:]
+	}
+	for len(y) > 0 {
+		fnum, _, n := protowire.ConsumeField(y)
+		my[fnum] = append(my[fnum], y[:n]...)
+		y = y[n:]
+	}
+	return reflect.DeepEqual(mx, my)
+}
diff --git a/vendor/google.golang.org/protobuf/proto/extension.go b/vendor/google.golang.org/protobuf/proto/extension.go
new file mode 100644
index 0000000..5f293cd
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/extension.go
@@ -0,0 +1,92 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// HasExtension reports whether an extension field is populated.
+// It returns false if m is invalid or if xt does not extend m.
+func HasExtension(m Message, xt protoreflect.ExtensionType) bool {
+	// Treat nil message interface as an empty message; no populated fields.
+	if m == nil {
+		return false
+	}
+
+	// As a special-case, we reports invalid or mismatching descriptors
+	// as always not being populated (since they aren't).
+	if xt == nil || m.ProtoReflect().Descriptor() != xt.TypeDescriptor().ContainingMessage() {
+		return false
+	}
+
+	return m.ProtoReflect().Has(xt.TypeDescriptor())
+}
+
+// ClearExtension clears an extension field such that subsequent
+// HasExtension calls return false.
+// It panics if m is invalid or if xt does not extend m.
+func ClearExtension(m Message, xt protoreflect.ExtensionType) {
+	m.ProtoReflect().Clear(xt.TypeDescriptor())
+}
+
+// GetExtension retrieves the value for an extension field.
+// If the field is unpopulated, it returns the default value for
+// scalars and an immutable, empty value for lists or messages.
+// It panics if xt does not extend m.
+func GetExtension(m Message, xt protoreflect.ExtensionType) interface{} {
+	// Treat nil message interface as an empty message; return the default.
+	if m == nil {
+		return xt.InterfaceOf(xt.Zero())
+	}
+
+	return xt.InterfaceOf(m.ProtoReflect().Get(xt.TypeDescriptor()))
+}
+
+// SetExtension stores the value of an extension field.
+// It panics if m is invalid, xt does not extend m, or if type of v
+// is invalid for the specified extension field.
+func SetExtension(m Message, xt protoreflect.ExtensionType, v interface{}) {
+	xd := xt.TypeDescriptor()
+	pv := xt.ValueOf(v)
+
+	// Specially treat an invalid list, map, or message as clear.
+	isValid := true
+	switch {
+	case xd.IsList():
+		isValid = pv.List().IsValid()
+	case xd.IsMap():
+		isValid = pv.Map().IsValid()
+	case xd.Message() != nil:
+		isValid = pv.Message().IsValid()
+	}
+	if !isValid {
+		m.ProtoReflect().Clear(xd)
+		return
+	}
+
+	m.ProtoReflect().Set(xd, pv)
+}
+
+// RangeExtensions iterates over every populated extension field in m in an
+// undefined order, calling f for each extension type and value encountered.
+// It returns immediately if f returns false.
+// While iterating, mutating operations may only be performed
+// on the current extension field.
+func RangeExtensions(m Message, f func(protoreflect.ExtensionType, interface{}) bool) {
+	// Treat nil message interface as an empty message; nothing to range over.
+	if m == nil {
+		return
+	}
+
+	m.ProtoReflect().Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		if fd.IsExtension() {
+			xt := fd.(protoreflect.ExtensionTypeDescriptor).Type()
+			vi := xt.InterfaceOf(v)
+			return f(xt, vi)
+		}
+		return true
+	})
+}
diff --git a/vendor/google.golang.org/protobuf/proto/merge.go b/vendor/google.golang.org/protobuf/proto/merge.go
new file mode 100644
index 0000000..d761ab3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/merge.go
@@ -0,0 +1,139 @@
+// 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.
+
+package proto
+
+import (
+	"fmt"
+
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// Merge merges src into dst, which must be a message with the same descriptor.
+//
+// Populated scalar fields in src are copied to dst, while populated
+// singular messages in src are merged into dst by recursively calling Merge.
+// The elements of every list field in src is appended to the corresponded
+// list fields in dst. The entries of every map field in src is copied into
+// the corresponding map field in dst, possibly replacing existing entries.
+// The unknown fields of src are appended to the unknown fields of dst.
+//
+// It is semantically equivalent to unmarshaling the encoded form of src
+// into dst with the UnmarshalOptions.Merge option specified.
+func Merge(dst, src Message) {
+	// TODO: Should nil src be treated as semantically equivalent to a
+	// untyped, read-only, empty message? What about a nil dst?
+
+	dstMsg, srcMsg := dst.ProtoReflect(), src.ProtoReflect()
+	if dstMsg.Descriptor() != srcMsg.Descriptor() {
+		if got, want := dstMsg.Descriptor().FullName(), srcMsg.Descriptor().FullName(); got != want {
+			panic(fmt.Sprintf("descriptor mismatch: %v != %v", got, want))
+		}
+		panic("descriptor mismatch")
+	}
+	mergeOptions{}.mergeMessage(dstMsg, srcMsg)
+}
+
+// Clone returns a deep copy of m.
+// If the top-level message is invalid, it returns an invalid message as well.
+func Clone(m Message) Message {
+	// NOTE: Most usages of Clone assume the following properties:
+	//	t := reflect.TypeOf(m)
+	//	t == reflect.TypeOf(m.ProtoReflect().New().Interface())
+	//	t == reflect.TypeOf(m.ProtoReflect().Type().Zero().Interface())
+	//
+	// Embedding protobuf messages breaks this since the parent type will have
+	// a forwarded ProtoReflect method, but the Interface method will return
+	// the underlying embedded message type.
+	if m == nil {
+		return nil
+	}
+	src := m.ProtoReflect()
+	if !src.IsValid() {
+		return src.Type().Zero().Interface()
+	}
+	dst := src.New()
+	mergeOptions{}.mergeMessage(dst, src)
+	return dst.Interface()
+}
+
+// mergeOptions provides a namespace for merge functions, and can be
+// exported in the future if we add user-visible merge options.
+type mergeOptions struct{}
+
+func (o mergeOptions) mergeMessage(dst, src protoreflect.Message) {
+	methods := protoMethods(dst)
+	if methods != nil && methods.Merge != nil {
+		in := protoiface.MergeInput{
+			Destination: dst,
+			Source:      src,
+		}
+		out := methods.Merge(in)
+		if out.Flags&protoiface.MergeComplete != 0 {
+			return
+		}
+	}
+
+	if !dst.IsValid() {
+		panic(fmt.Sprintf("cannot merge into invalid %v message", dst.Descriptor().FullName()))
+	}
+
+	src.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		switch {
+		case fd.IsList():
+			o.mergeList(dst.Mutable(fd).List(), v.List(), fd)
+		case fd.IsMap():
+			o.mergeMap(dst.Mutable(fd).Map(), v.Map(), fd.MapValue())
+		case fd.Message() != nil:
+			o.mergeMessage(dst.Mutable(fd).Message(), v.Message())
+		case fd.Kind() == protoreflect.BytesKind:
+			dst.Set(fd, o.cloneBytes(v))
+		default:
+			dst.Set(fd, v)
+		}
+		return true
+	})
+
+	if len(src.GetUnknown()) > 0 {
+		dst.SetUnknown(append(dst.GetUnknown(), src.GetUnknown()...))
+	}
+}
+
+func (o mergeOptions) mergeList(dst, src protoreflect.List, fd protoreflect.FieldDescriptor) {
+	// Merge semantics appends to the end of the existing list.
+	for i, n := 0, src.Len(); i < n; i++ {
+		switch v := src.Get(i); {
+		case fd.Message() != nil:
+			dstv := dst.NewElement()
+			o.mergeMessage(dstv.Message(), v.Message())
+			dst.Append(dstv)
+		case fd.Kind() == protoreflect.BytesKind:
+			dst.Append(o.cloneBytes(v))
+		default:
+			dst.Append(v)
+		}
+	}
+}
+
+func (o mergeOptions) mergeMap(dst, src protoreflect.Map, fd protoreflect.FieldDescriptor) {
+	// Merge semantics replaces, rather than merges into existing entries.
+	src.Range(func(k protoreflect.MapKey, v protoreflect.Value) bool {
+		switch {
+		case fd.Message() != nil:
+			dstv := dst.NewValue()
+			o.mergeMessage(dstv.Message(), v.Message())
+			dst.Set(k, dstv)
+		case fd.Kind() == protoreflect.BytesKind:
+			dst.Set(k, o.cloneBytes(v))
+		default:
+			dst.Set(k, v)
+		}
+		return true
+	})
+}
+
+func (o mergeOptions) cloneBytes(v protoreflect.Value) protoreflect.Value {
+	return protoreflect.ValueOfBytes(append([]byte{}, v.Bytes()...))
+}
diff --git a/vendor/google.golang.org/protobuf/proto/messageset.go b/vendor/google.golang.org/protobuf/proto/messageset.go
new file mode 100644
index 0000000..312d5d4
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/messageset.go
@@ -0,0 +1,93 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/order"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+func (o MarshalOptions) sizeMessageSet(m protoreflect.Message) (size int) {
+	m.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		size += messageset.SizeField(fd.Number())
+		size += protowire.SizeTag(messageset.FieldMessage)
+		size += protowire.SizeBytes(o.size(v.Message()))
+		return true
+	})
+	size += messageset.SizeUnknown(m.GetUnknown())
+	return size
+}
+
+func (o MarshalOptions) marshalMessageSet(b []byte, m protoreflect.Message) ([]byte, error) {
+	if !flags.ProtoLegacy {
+		return b, errors.New("no support for message_set_wire_format")
+	}
+	fieldOrder := order.AnyFieldOrder
+	if o.Deterministic {
+		fieldOrder = order.NumberFieldOrder
+	}
+	var err error
+	order.RangeFields(m, fieldOrder, func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		b, err = o.marshalMessageSetField(b, fd, v)
+		return err == nil
+	})
+	if err != nil {
+		return b, err
+	}
+	return messageset.AppendUnknown(b, m.GetUnknown())
+}
+
+func (o MarshalOptions) marshalMessageSetField(b []byte, fd protoreflect.FieldDescriptor, value protoreflect.Value) ([]byte, error) {
+	b = messageset.AppendFieldStart(b, fd.Number())
+	b = protowire.AppendTag(b, messageset.FieldMessage, protowire.BytesType)
+	b = protowire.AppendVarint(b, uint64(o.Size(value.Message().Interface())))
+	b, err := o.marshalMessage(b, value.Message())
+	if err != nil {
+		return b, err
+	}
+	b = messageset.AppendFieldEnd(b)
+	return b, nil
+}
+
+func (o UnmarshalOptions) unmarshalMessageSet(b []byte, m protoreflect.Message) error {
+	if !flags.ProtoLegacy {
+		return errors.New("no support for message_set_wire_format")
+	}
+	return messageset.Unmarshal(b, false, func(num protowire.Number, v []byte) error {
+		err := o.unmarshalMessageSetField(m, num, v)
+		if err == errUnknown {
+			unknown := m.GetUnknown()
+			unknown = protowire.AppendTag(unknown, num, protowire.BytesType)
+			unknown = protowire.AppendBytes(unknown, v)
+			m.SetUnknown(unknown)
+			return nil
+		}
+		return err
+	})
+}
+
+func (o UnmarshalOptions) unmarshalMessageSetField(m protoreflect.Message, num protowire.Number, v []byte) error {
+	md := m.Descriptor()
+	if !md.ExtensionRanges().Has(num) {
+		return errUnknown
+	}
+	xt, err := o.Resolver.FindExtensionByNumber(md.FullName(), num)
+	if err == protoregistry.NotFound {
+		return errUnknown
+	}
+	if err != nil {
+		return errors.New("%v: unable to resolve extension %v: %v", md.FullName(), num, err)
+	}
+	xd := xt.TypeDescriptor()
+	if err := o.unmarshalMessage(v, m.Mutable(xd).Message()); err != nil {
+		return err
+	}
+	return nil
+}
diff --git a/vendor/google.golang.org/protobuf/proto/proto.go b/vendor/google.golang.org/protobuf/proto/proto.go
new file mode 100644
index 0000000..1f0d183
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/proto.go
@@ -0,0 +1,43 @@
+// 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 proto
+
+import (
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// Message is the top-level interface that all messages must implement.
+// It provides access to a reflective view of a message.
+// Any implementation of this interface may be used with all functions in the
+// protobuf module that accept a Message, except where otherwise specified.
+//
+// This is the v2 interface definition for protobuf messages.
+// The v1 interface definition is "github.com/golang/protobuf/proto".Message.
+//
+// To convert a v1 message to a v2 message,
+// use "github.com/golang/protobuf/proto".MessageV2.
+// To convert a v2 message to a v1 message,
+// use "github.com/golang/protobuf/proto".MessageV1.
+type Message = protoreflect.ProtoMessage
+
+// Error matches all errors produced by packages in the protobuf module.
+//
+// That is, errors.Is(err, Error) reports whether an error is produced
+// by this module.
+var Error error
+
+func init() {
+	Error = errors.Error
+}
+
+// MessageName returns the full name of m.
+// If m is nil, it returns an empty string.
+func MessageName(m Message) protoreflect.FullName {
+	if m == nil {
+		return ""
+	}
+	return m.ProtoReflect().Descriptor().FullName()
+}
diff --git a/vendor/google.golang.org/protobuf/proto/proto_methods.go b/vendor/google.golang.org/protobuf/proto/proto_methods.go
new file mode 100644
index 0000000..d8dd604
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/proto_methods.go
@@ -0,0 +1,19 @@
+// 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.
+
+// The protoreflect build tag disables use of fast-path methods.
+// +build !protoreflect
+
+package proto
+
+import (
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+const hasProtoMethods = true
+
+func protoMethods(m protoreflect.Message) *protoiface.Methods {
+	return m.ProtoMethods()
+}
diff --git a/vendor/google.golang.org/protobuf/proto/proto_reflect.go b/vendor/google.golang.org/protobuf/proto/proto_reflect.go
new file mode 100644
index 0000000..b103d43
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/proto_reflect.go
@@ -0,0 +1,19 @@
+// 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.
+
+// The protoreflect build tag disables use of fast-path methods.
+// +build protoreflect
+
+package proto
+
+import (
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+const hasProtoMethods = false
+
+func protoMethods(m protoreflect.Message) *protoiface.Methods {
+	return nil
+}
diff --git a/vendor/google.golang.org/protobuf/proto/reset.go b/vendor/google.golang.org/protobuf/proto/reset.go
new file mode 100644
index 0000000..3d7f894
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/reset.go
@@ -0,0 +1,43 @@
+// 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.
+
+package proto
+
+import (
+	"fmt"
+
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// Reset clears every field in the message.
+// The resulting message shares no observable memory with its previous state
+// other than the memory for the message itself.
+func Reset(m Message) {
+	if mr, ok := m.(interface{ Reset() }); ok && hasProtoMethods {
+		mr.Reset()
+		return
+	}
+	resetMessage(m.ProtoReflect())
+}
+
+func resetMessage(m protoreflect.Message) {
+	if !m.IsValid() {
+		panic(fmt.Sprintf("cannot reset invalid %v message", m.Descriptor().FullName()))
+	}
+
+	// Clear all known fields.
+	fds := m.Descriptor().Fields()
+	for i := 0; i < fds.Len(); i++ {
+		m.Clear(fds.Get(i))
+	}
+
+	// Clear extension fields.
+	m.Range(func(fd protoreflect.FieldDescriptor, _ protoreflect.Value) bool {
+		m.Clear(fd)
+		return true
+	})
+
+	// Clear unknown fields.
+	m.SetUnknown(nil)
+}
diff --git a/vendor/google.golang.org/protobuf/proto/size.go b/vendor/google.golang.org/protobuf/proto/size.go
new file mode 100644
index 0000000..554b9c6
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/size.go
@@ -0,0 +1,97 @@
+// 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.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+// Size returns the size in bytes of the wire-format encoding of m.
+func Size(m Message) int {
+	return MarshalOptions{}.Size(m)
+}
+
+// Size returns the size in bytes of the wire-format encoding of m.
+func (o MarshalOptions) Size(m Message) int {
+	// Treat a nil message interface as an empty message; nothing to output.
+	if m == nil {
+		return 0
+	}
+
+	return o.size(m.ProtoReflect())
+}
+
+// size is a centralized function that all size operations go through.
+// For profiling purposes, avoid changing the name of this function or
+// introducing other code paths for size that do not go through this.
+func (o MarshalOptions) size(m protoreflect.Message) (size int) {
+	methods := protoMethods(m)
+	if methods != nil && methods.Size != nil {
+		out := methods.Size(protoiface.SizeInput{
+			Message: m,
+		})
+		return out.Size
+	}
+	if methods != nil && methods.Marshal != nil {
+		// This is not efficient, but we don't have any choice.
+		// This case is mainly used for legacy types with a Marshal method.
+		out, _ := methods.Marshal(protoiface.MarshalInput{
+			Message: m,
+		})
+		return len(out.Buf)
+	}
+	return o.sizeMessageSlow(m)
+}
+
+func (o MarshalOptions) sizeMessageSlow(m protoreflect.Message) (size int) {
+	if messageset.IsMessageSet(m.Descriptor()) {
+		return o.sizeMessageSet(m)
+	}
+	m.Range(func(fd protoreflect.FieldDescriptor, v protoreflect.Value) bool {
+		size += o.sizeField(fd, v)
+		return true
+	})
+	size += len(m.GetUnknown())
+	return size
+}
+
+func (o MarshalOptions) sizeField(fd protoreflect.FieldDescriptor, value protoreflect.Value) (size int) {
+	num := fd.Number()
+	switch {
+	case fd.IsList():
+		return o.sizeList(num, fd, value.List())
+	case fd.IsMap():
+		return o.sizeMap(num, fd, value.Map())
+	default:
+		return protowire.SizeTag(num) + o.sizeSingular(num, fd.Kind(), value)
+	}
+}
+
+func (o MarshalOptions) sizeList(num protowire.Number, fd protoreflect.FieldDescriptor, list protoreflect.List) (size int) {
+	if fd.IsPacked() && list.Len() > 0 {
+		content := 0
+		for i, llen := 0, list.Len(); i < llen; i++ {
+			content += o.sizeSingular(num, fd.Kind(), list.Get(i))
+		}
+		return protowire.SizeTag(num) + protowire.SizeBytes(content)
+	}
+
+	for i, llen := 0, list.Len(); i < llen; i++ {
+		size += protowire.SizeTag(num) + o.sizeSingular(num, fd.Kind(), list.Get(i))
+	}
+	return size
+}
+
+func (o MarshalOptions) sizeMap(num protowire.Number, fd protoreflect.FieldDescriptor, mapv protoreflect.Map) (size int) {
+	mapv.Range(func(key protoreflect.MapKey, value protoreflect.Value) bool {
+		size += protowire.SizeTag(num)
+		size += protowire.SizeBytes(o.sizeField(fd.MapKey(), key.Value()) + o.sizeField(fd.MapValue(), value))
+		return true
+	})
+	return size
+}
diff --git a/vendor/google.golang.org/protobuf/proto/size_gen.go b/vendor/google.golang.org/protobuf/proto/size_gen.go
new file mode 100644
index 0000000..3cf61a8
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/size_gen.go
@@ -0,0 +1,55 @@
+// 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.
+
+// Code generated by generate-types. DO NOT EDIT.
+
+package proto
+
+import (
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+func (o MarshalOptions) sizeSingular(num protowire.Number, kind protoreflect.Kind, v protoreflect.Value) int {
+	switch kind {
+	case protoreflect.BoolKind:
+		return protowire.SizeVarint(protowire.EncodeBool(v.Bool()))
+	case protoreflect.EnumKind:
+		return protowire.SizeVarint(uint64(v.Enum()))
+	case protoreflect.Int32Kind:
+		return protowire.SizeVarint(uint64(int32(v.Int())))
+	case protoreflect.Sint32Kind:
+		return protowire.SizeVarint(protowire.EncodeZigZag(int64(int32(v.Int()))))
+	case protoreflect.Uint32Kind:
+		return protowire.SizeVarint(uint64(uint32(v.Uint())))
+	case protoreflect.Int64Kind:
+		return protowire.SizeVarint(uint64(v.Int()))
+	case protoreflect.Sint64Kind:
+		return protowire.SizeVarint(protowire.EncodeZigZag(v.Int()))
+	case protoreflect.Uint64Kind:
+		return protowire.SizeVarint(v.Uint())
+	case protoreflect.Sfixed32Kind:
+		return protowire.SizeFixed32()
+	case protoreflect.Fixed32Kind:
+		return protowire.SizeFixed32()
+	case protoreflect.FloatKind:
+		return protowire.SizeFixed32()
+	case protoreflect.Sfixed64Kind:
+		return protowire.SizeFixed64()
+	case protoreflect.Fixed64Kind:
+		return protowire.SizeFixed64()
+	case protoreflect.DoubleKind:
+		return protowire.SizeFixed64()
+	case protoreflect.StringKind:
+		return protowire.SizeBytes(len(v.String()))
+	case protoreflect.BytesKind:
+		return protowire.SizeBytes(len(v.Bytes()))
+	case protoreflect.MessageKind:
+		return protowire.SizeBytes(o.size(v.Message()))
+	case protoreflect.GroupKind:
+		return protowire.SizeGroup(num, o.size(v.Message()))
+	default:
+		return 0
+	}
+}
diff --git a/vendor/google.golang.org/protobuf/proto/wrappers.go b/vendor/google.golang.org/protobuf/proto/wrappers.go
new file mode 100644
index 0000000..653b12c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/proto/wrappers.go
@@ -0,0 +1,29 @@
+// 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.
+
+package proto
+
+// Bool stores v in a new bool value and returns a pointer to it.
+func Bool(v bool) *bool { return &v }
+
+// Int32 stores v in a new int32 value and returns a pointer to it.
+func Int32(v int32) *int32 { return &v }
+
+// Int64 stores v in a new int64 value and returns a pointer to it.
+func Int64(v int64) *int64 { return &v }
+
+// Float32 stores v in a new float32 value and returns a pointer to it.
+func Float32(v float32) *float32 { return &v }
+
+// Float64 stores v in a new float64 value and returns a pointer to it.
+func Float64(v float64) *float64 { return &v }
+
+// Uint32 stores v in a new uint32 value and returns a pointer to it.
+func Uint32(v uint32) *uint32 { return &v }
+
+// Uint64 stores v in a new uint64 value and returns a pointer to it.
+func Uint64(v uint64) *uint64 { return &v }
+
+// String stores v in a new string value and returns a pointer to it.
+func String(v string) *string { return &v }
diff --git a/vendor/google.golang.org/protobuf/reflect/protodesc/desc.go b/vendor/google.golang.org/protobuf/reflect/protodesc/desc.go
new file mode 100644
index 0000000..e4dfb12
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protodesc/desc.go
@@ -0,0 +1,276 @@
+// 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 protodesc provides functionality for converting
+// FileDescriptorProto messages to/from protoreflect.FileDescriptor values.
+//
+// The google.protobuf.FileDescriptorProto is a protobuf message that describes
+// the type information for a .proto file in a form that is easily serializable.
+// The protoreflect.FileDescriptor is a more structured representation of
+// the FileDescriptorProto message where references and remote dependencies
+// can be directly followed.
+package protodesc
+
+import (
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+
+	"google.golang.org/protobuf/types/descriptorpb"
+)
+
+// Resolver is the resolver used by NewFile to resolve dependencies.
+// The enums and messages provided must belong to some parent file,
+// which is also registered.
+//
+// It is implemented by protoregistry.Files.
+type Resolver interface {
+	FindFileByPath(string) (protoreflect.FileDescriptor, error)
+	FindDescriptorByName(protoreflect.FullName) (protoreflect.Descriptor, error)
+}
+
+// FileOptions configures the construction of file descriptors.
+type FileOptions struct {
+	pragma.NoUnkeyedLiterals
+
+	// AllowUnresolvable configures New to permissively allow unresolvable
+	// file, enum, or message dependencies. Unresolved dependencies are replaced
+	// by placeholder equivalents.
+	//
+	// The following dependencies may be left unresolved:
+	//	• Resolving an imported file.
+	//	• Resolving the type for a message field or extension field.
+	//	If the kind of the field is unknown, then a placeholder is used for both
+	//	the Enum and Message accessors on the protoreflect.FieldDescriptor.
+	//	• Resolving an enum value set as the default for an optional enum field.
+	//	If unresolvable, the protoreflect.FieldDescriptor.Default is set to the
+	//	first value in the associated enum (or zero if the also enum dependency
+	//	is also unresolvable). The protoreflect.FieldDescriptor.DefaultEnumValue
+	//	is populated with a placeholder.
+	//	• Resolving the extended message type for an extension field.
+	//	• Resolving the input or output message type for a service method.
+	//
+	// If the unresolved dependency uses a relative name,
+	// then the placeholder will contain an invalid FullName with a "*." prefix,
+	// indicating that the starting prefix of the full name is unknown.
+	AllowUnresolvable bool
+}
+
+// NewFile creates a new protoreflect.FileDescriptor from the provided
+// file descriptor message. See FileOptions.New for more information.
+func NewFile(fd *descriptorpb.FileDescriptorProto, r Resolver) (protoreflect.FileDescriptor, error) {
+	return FileOptions{}.New(fd, r)
+}
+
+// NewFiles creates a new protoregistry.Files from the provided
+// FileDescriptorSet message. See FileOptions.NewFiles for more information.
+func NewFiles(fd *descriptorpb.FileDescriptorSet) (*protoregistry.Files, error) {
+	return FileOptions{}.NewFiles(fd)
+}
+
+// New creates a new protoreflect.FileDescriptor from the provided
+// file descriptor message. The file must represent a valid proto file according
+// to protobuf semantics. The returned descriptor is a deep copy of the input.
+//
+// Any imported files, enum types, or message types referenced in the file are
+// resolved using the provided registry. When looking up an import file path,
+// the path must be unique. The newly created file descriptor is not registered
+// back into the provided file registry.
+func (o FileOptions) New(fd *descriptorpb.FileDescriptorProto, r Resolver) (protoreflect.FileDescriptor, error) {
+	if r == nil {
+		r = (*protoregistry.Files)(nil) // empty resolver
+	}
+
+	// Handle the file descriptor content.
+	f := &filedesc.File{L2: &filedesc.FileL2{}}
+	switch fd.GetSyntax() {
+	case "proto2", "":
+		f.L1.Syntax = protoreflect.Proto2
+	case "proto3":
+		f.L1.Syntax = protoreflect.Proto3
+	default:
+		return nil, errors.New("invalid syntax: %q", fd.GetSyntax())
+	}
+	f.L1.Path = fd.GetName()
+	if f.L1.Path == "" {
+		return nil, errors.New("file path must be populated")
+	}
+	f.L1.Package = protoreflect.FullName(fd.GetPackage())
+	if !f.L1.Package.IsValid() && f.L1.Package != "" {
+		return nil, errors.New("invalid package: %q", f.L1.Package)
+	}
+	if opts := fd.GetOptions(); opts != nil {
+		opts = proto.Clone(opts).(*descriptorpb.FileOptions)
+		f.L2.Options = func() protoreflect.ProtoMessage { return opts }
+	}
+
+	f.L2.Imports = make(filedesc.FileImports, len(fd.GetDependency()))
+	for _, i := range fd.GetPublicDependency() {
+		if !(0 <= i && int(i) < len(f.L2.Imports)) || f.L2.Imports[i].IsPublic {
+			return nil, errors.New("invalid or duplicate public import index: %d", i)
+		}
+		f.L2.Imports[i].IsPublic = true
+	}
+	for _, i := range fd.GetWeakDependency() {
+		if !(0 <= i && int(i) < len(f.L2.Imports)) || f.L2.Imports[i].IsWeak {
+			return nil, errors.New("invalid or duplicate weak import index: %d", i)
+		}
+		f.L2.Imports[i].IsWeak = true
+	}
+	imps := importSet{f.Path(): true}
+	for i, path := range fd.GetDependency() {
+		imp := &f.L2.Imports[i]
+		f, err := r.FindFileByPath(path)
+		if err == protoregistry.NotFound && (o.AllowUnresolvable || imp.IsWeak) {
+			f = filedesc.PlaceholderFile(path)
+		} else if err != nil {
+			return nil, errors.New("could not resolve import %q: %v", path, err)
+		}
+		imp.FileDescriptor = f
+
+		if imps[imp.Path()] {
+			return nil, errors.New("already imported %q", path)
+		}
+		imps[imp.Path()] = true
+	}
+	for i := range fd.GetDependency() {
+		imp := &f.L2.Imports[i]
+		imps.importPublic(imp.Imports())
+	}
+
+	// Handle source locations.
+	f.L2.Locations.File = f
+	for _, loc := range fd.GetSourceCodeInfo().GetLocation() {
+		var l protoreflect.SourceLocation
+		// TODO: Validate that the path points to an actual declaration?
+		l.Path = protoreflect.SourcePath(loc.GetPath())
+		s := loc.GetSpan()
+		switch len(s) {
+		case 3:
+			l.StartLine, l.StartColumn, l.EndLine, l.EndColumn = int(s[0]), int(s[1]), int(s[0]), int(s[2])
+		case 4:
+			l.StartLine, l.StartColumn, l.EndLine, l.EndColumn = int(s[0]), int(s[1]), int(s[2]), int(s[3])
+		default:
+			return nil, errors.New("invalid span: %v", s)
+		}
+		// TODO: Validate that the span information is sensible?
+		// See https://github.com/protocolbuffers/protobuf/issues/6378.
+		if false && (l.EndLine < l.StartLine || l.StartLine < 0 || l.StartColumn < 0 || l.EndColumn < 0 ||
+			(l.StartLine == l.EndLine && l.EndColumn <= l.StartColumn)) {
+			return nil, errors.New("invalid span: %v", s)
+		}
+		l.LeadingDetachedComments = loc.GetLeadingDetachedComments()
+		l.LeadingComments = loc.GetLeadingComments()
+		l.TrailingComments = loc.GetTrailingComments()
+		f.L2.Locations.List = append(f.L2.Locations.List, l)
+	}
+
+	// Step 1: Allocate and derive the names for all declarations.
+	// This copies all fields from the descriptor proto except:
+	//	google.protobuf.FieldDescriptorProto.type_name
+	//	google.protobuf.FieldDescriptorProto.default_value
+	//	google.protobuf.FieldDescriptorProto.oneof_index
+	//	google.protobuf.FieldDescriptorProto.extendee
+	//	google.protobuf.MethodDescriptorProto.input
+	//	google.protobuf.MethodDescriptorProto.output
+	var err error
+	sb := new(strs.Builder)
+	r1 := make(descsByName)
+	if f.L1.Enums.List, err = r1.initEnumDeclarations(fd.GetEnumType(), f, sb); err != nil {
+		return nil, err
+	}
+	if f.L1.Messages.List, err = r1.initMessagesDeclarations(fd.GetMessageType(), f, sb); err != nil {
+		return nil, err
+	}
+	if f.L1.Extensions.List, err = r1.initExtensionDeclarations(fd.GetExtension(), f, sb); err != nil {
+		return nil, err
+	}
+	if f.L1.Services.List, err = r1.initServiceDeclarations(fd.GetService(), f, sb); err != nil {
+		return nil, err
+	}
+
+	// Step 2: Resolve every dependency reference not handled by step 1.
+	r2 := &resolver{local: r1, remote: r, imports: imps, allowUnresolvable: o.AllowUnresolvable}
+	if err := r2.resolveMessageDependencies(f.L1.Messages.List, fd.GetMessageType()); err != nil {
+		return nil, err
+	}
+	if err := r2.resolveExtensionDependencies(f.L1.Extensions.List, fd.GetExtension()); err != nil {
+		return nil, err
+	}
+	if err := r2.resolveServiceDependencies(f.L1.Services.List, fd.GetService()); err != nil {
+		return nil, err
+	}
+
+	// Step 3: Validate every enum, message, and extension declaration.
+	if err := validateEnumDeclarations(f.L1.Enums.List, fd.GetEnumType()); err != nil {
+		return nil, err
+	}
+	if err := validateMessageDeclarations(f.L1.Messages.List, fd.GetMessageType()); err != nil {
+		return nil, err
+	}
+	if err := validateExtensionDeclarations(f.L1.Extensions.List, fd.GetExtension()); err != nil {
+		return nil, err
+	}
+
+	return f, nil
+}
+
+type importSet map[string]bool
+
+func (is importSet) importPublic(imps protoreflect.FileImports) {
+	for i := 0; i < imps.Len(); i++ {
+		if imp := imps.Get(i); imp.IsPublic {
+			is[imp.Path()] = true
+			is.importPublic(imp.Imports())
+		}
+	}
+}
+
+// NewFiles creates a new protoregistry.Files from the provided
+// FileDescriptorSet message. The descriptor set must include only
+// valid files according to protobuf semantics. The returned descriptors
+// are a deep copy of the input.
+func (o FileOptions) NewFiles(fds *descriptorpb.FileDescriptorSet) (*protoregistry.Files, error) {
+	files := make(map[string]*descriptorpb.FileDescriptorProto)
+	for _, fd := range fds.File {
+		if _, ok := files[fd.GetName()]; ok {
+			return nil, errors.New("file appears multiple times: %q", fd.GetName())
+		}
+		files[fd.GetName()] = fd
+	}
+	r := &protoregistry.Files{}
+	for _, fd := range files {
+		if err := o.addFileDeps(r, fd, files); err != nil {
+			return nil, err
+		}
+	}
+	return r, nil
+}
+func (o FileOptions) addFileDeps(r *protoregistry.Files, fd *descriptorpb.FileDescriptorProto, files map[string]*descriptorpb.FileDescriptorProto) error {
+	// Set the entry to nil while descending into a file's dependencies to detect cycles.
+	files[fd.GetName()] = nil
+	for _, dep := range fd.Dependency {
+		depfd, ok := files[dep]
+		if depfd == nil {
+			if ok {
+				return errors.New("import cycle in file: %q", dep)
+			}
+			continue
+		}
+		if err := o.addFileDeps(r, depfd, files); err != nil {
+			return err
+		}
+	}
+	// Delete the entry once dependencies are processed.
+	delete(files, fd.GetName())
+	f, err := o.New(fd, r)
+	if err != nil {
+		return err
+	}
+	return r.RegisterFile(f)
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protodesc/desc_init.go b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_init.go
new file mode 100644
index 0000000..37efda1
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_init.go
@@ -0,0 +1,248 @@
+// 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.
+
+package protodesc
+
+import (
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+
+	"google.golang.org/protobuf/types/descriptorpb"
+)
+
+type descsByName map[protoreflect.FullName]protoreflect.Descriptor
+
+func (r descsByName) initEnumDeclarations(eds []*descriptorpb.EnumDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (es []filedesc.Enum, err error) {
+	es = make([]filedesc.Enum, len(eds)) // allocate up-front to ensure stable pointers
+	for i, ed := range eds {
+		e := &es[i]
+		e.L2 = new(filedesc.EnumL2)
+		if e.L0, err = r.makeBase(e, parent, ed.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := ed.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.EnumOptions)
+			e.L2.Options = func() protoreflect.ProtoMessage { return opts }
+		}
+		for _, s := range ed.GetReservedName() {
+			e.L2.ReservedNames.List = append(e.L2.ReservedNames.List, protoreflect.Name(s))
+		}
+		for _, rr := range ed.GetReservedRange() {
+			e.L2.ReservedRanges.List = append(e.L2.ReservedRanges.List, [2]protoreflect.EnumNumber{
+				protoreflect.EnumNumber(rr.GetStart()),
+				protoreflect.EnumNumber(rr.GetEnd()),
+			})
+		}
+		if e.L2.Values.List, err = r.initEnumValuesFromDescriptorProto(ed.GetValue(), e, sb); err != nil {
+			return nil, err
+		}
+	}
+	return es, nil
+}
+
+func (r descsByName) initEnumValuesFromDescriptorProto(vds []*descriptorpb.EnumValueDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (vs []filedesc.EnumValue, err error) {
+	vs = make([]filedesc.EnumValue, len(vds)) // allocate up-front to ensure stable pointers
+	for i, vd := range vds {
+		v := &vs[i]
+		if v.L0, err = r.makeBase(v, parent, vd.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := vd.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.EnumValueOptions)
+			v.L1.Options = func() protoreflect.ProtoMessage { return opts }
+		}
+		v.L1.Number = protoreflect.EnumNumber(vd.GetNumber())
+	}
+	return vs, nil
+}
+
+func (r descsByName) initMessagesDeclarations(mds []*descriptorpb.DescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (ms []filedesc.Message, err error) {
+	ms = make([]filedesc.Message, len(mds)) // allocate up-front to ensure stable pointers
+	for i, md := range mds {
+		m := &ms[i]
+		m.L2 = new(filedesc.MessageL2)
+		if m.L0, err = r.makeBase(m, parent, md.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := md.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.MessageOptions)
+			m.L2.Options = func() protoreflect.ProtoMessage { return opts }
+			m.L1.IsMapEntry = opts.GetMapEntry()
+			m.L1.IsMessageSet = opts.GetMessageSetWireFormat()
+		}
+		for _, s := range md.GetReservedName() {
+			m.L2.ReservedNames.List = append(m.L2.ReservedNames.List, protoreflect.Name(s))
+		}
+		for _, rr := range md.GetReservedRange() {
+			m.L2.ReservedRanges.List = append(m.L2.ReservedRanges.List, [2]protoreflect.FieldNumber{
+				protoreflect.FieldNumber(rr.GetStart()),
+				protoreflect.FieldNumber(rr.GetEnd()),
+			})
+		}
+		for _, xr := range md.GetExtensionRange() {
+			m.L2.ExtensionRanges.List = append(m.L2.ExtensionRanges.List, [2]protoreflect.FieldNumber{
+				protoreflect.FieldNumber(xr.GetStart()),
+				protoreflect.FieldNumber(xr.GetEnd()),
+			})
+			var optsFunc func() protoreflect.ProtoMessage
+			if opts := xr.GetOptions(); opts != nil {
+				opts = proto.Clone(opts).(*descriptorpb.ExtensionRangeOptions)
+				optsFunc = func() protoreflect.ProtoMessage { return opts }
+			}
+			m.L2.ExtensionRangeOptions = append(m.L2.ExtensionRangeOptions, optsFunc)
+		}
+		if m.L2.Fields.List, err = r.initFieldsFromDescriptorProto(md.GetField(), m, sb); err != nil {
+			return nil, err
+		}
+		if m.L2.Oneofs.List, err = r.initOneofsFromDescriptorProto(md.GetOneofDecl(), m, sb); err != nil {
+			return nil, err
+		}
+		if m.L1.Enums.List, err = r.initEnumDeclarations(md.GetEnumType(), m, sb); err != nil {
+			return nil, err
+		}
+		if m.L1.Messages.List, err = r.initMessagesDeclarations(md.GetNestedType(), m, sb); err != nil {
+			return nil, err
+		}
+		if m.L1.Extensions.List, err = r.initExtensionDeclarations(md.GetExtension(), m, sb); err != nil {
+			return nil, err
+		}
+	}
+	return ms, nil
+}
+
+func (r descsByName) initFieldsFromDescriptorProto(fds []*descriptorpb.FieldDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (fs []filedesc.Field, err error) {
+	fs = make([]filedesc.Field, len(fds)) // allocate up-front to ensure stable pointers
+	for i, fd := range fds {
+		f := &fs[i]
+		if f.L0, err = r.makeBase(f, parent, fd.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		f.L1.IsProto3Optional = fd.GetProto3Optional()
+		if opts := fd.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.FieldOptions)
+			f.L1.Options = func() protoreflect.ProtoMessage { return opts }
+			f.L1.IsWeak = opts.GetWeak()
+			f.L1.HasPacked = opts.Packed != nil
+			f.L1.IsPacked = opts.GetPacked()
+		}
+		f.L1.Number = protoreflect.FieldNumber(fd.GetNumber())
+		f.L1.Cardinality = protoreflect.Cardinality(fd.GetLabel())
+		if fd.Type != nil {
+			f.L1.Kind = protoreflect.Kind(fd.GetType())
+		}
+		if fd.JsonName != nil {
+			f.L1.StringName.InitJSON(fd.GetJsonName())
+		}
+	}
+	return fs, nil
+}
+
+func (r descsByName) initOneofsFromDescriptorProto(ods []*descriptorpb.OneofDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (os []filedesc.Oneof, err error) {
+	os = make([]filedesc.Oneof, len(ods)) // allocate up-front to ensure stable pointers
+	for i, od := range ods {
+		o := &os[i]
+		if o.L0, err = r.makeBase(o, parent, od.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := od.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.OneofOptions)
+			o.L1.Options = func() protoreflect.ProtoMessage { return opts }
+		}
+	}
+	return os, nil
+}
+
+func (r descsByName) initExtensionDeclarations(xds []*descriptorpb.FieldDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (xs []filedesc.Extension, err error) {
+	xs = make([]filedesc.Extension, len(xds)) // allocate up-front to ensure stable pointers
+	for i, xd := range xds {
+		x := &xs[i]
+		x.L2 = new(filedesc.ExtensionL2)
+		if x.L0, err = r.makeBase(x, parent, xd.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := xd.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.FieldOptions)
+			x.L2.Options = func() protoreflect.ProtoMessage { return opts }
+			x.L2.IsPacked = opts.GetPacked()
+		}
+		x.L1.Number = protoreflect.FieldNumber(xd.GetNumber())
+		x.L1.Cardinality = protoreflect.Cardinality(xd.GetLabel())
+		if xd.Type != nil {
+			x.L1.Kind = protoreflect.Kind(xd.GetType())
+		}
+		if xd.JsonName != nil {
+			x.L2.StringName.InitJSON(xd.GetJsonName())
+		}
+	}
+	return xs, nil
+}
+
+func (r descsByName) initServiceDeclarations(sds []*descriptorpb.ServiceDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (ss []filedesc.Service, err error) {
+	ss = make([]filedesc.Service, len(sds)) // allocate up-front to ensure stable pointers
+	for i, sd := range sds {
+		s := &ss[i]
+		s.L2 = new(filedesc.ServiceL2)
+		if s.L0, err = r.makeBase(s, parent, sd.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := sd.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.ServiceOptions)
+			s.L2.Options = func() protoreflect.ProtoMessage { return opts }
+		}
+		if s.L2.Methods.List, err = r.initMethodsFromDescriptorProto(sd.GetMethod(), s, sb); err != nil {
+			return nil, err
+		}
+	}
+	return ss, nil
+}
+
+func (r descsByName) initMethodsFromDescriptorProto(mds []*descriptorpb.MethodDescriptorProto, parent protoreflect.Descriptor, sb *strs.Builder) (ms []filedesc.Method, err error) {
+	ms = make([]filedesc.Method, len(mds)) // allocate up-front to ensure stable pointers
+	for i, md := range mds {
+		m := &ms[i]
+		if m.L0, err = r.makeBase(m, parent, md.GetName(), i, sb); err != nil {
+			return nil, err
+		}
+		if opts := md.GetOptions(); opts != nil {
+			opts = proto.Clone(opts).(*descriptorpb.MethodOptions)
+			m.L1.Options = func() protoreflect.ProtoMessage { return opts }
+		}
+		m.L1.IsStreamingClient = md.GetClientStreaming()
+		m.L1.IsStreamingServer = md.GetServerStreaming()
+	}
+	return ms, nil
+}
+
+func (r descsByName) makeBase(child, parent protoreflect.Descriptor, name string, idx int, sb *strs.Builder) (filedesc.BaseL0, error) {
+	if !protoreflect.Name(name).IsValid() {
+		return filedesc.BaseL0{}, errors.New("descriptor %q has an invalid nested name: %q", parent.FullName(), name)
+	}
+
+	// Derive the full name of the child.
+	// Note that enum values are a sibling to the enum parent in the namespace.
+	var fullName protoreflect.FullName
+	if _, ok := parent.(protoreflect.EnumDescriptor); ok {
+		fullName = sb.AppendFullName(parent.FullName().Parent(), protoreflect.Name(name))
+	} else {
+		fullName = sb.AppendFullName(parent.FullName(), protoreflect.Name(name))
+	}
+	if _, ok := r[fullName]; ok {
+		return filedesc.BaseL0{}, errors.New("descriptor %q already declared", fullName)
+	}
+	r[fullName] = child
+
+	// TODO: Verify that the full name does not already exist in the resolver?
+	// This is not as critical since most usages of NewFile will register
+	// the created file back into the registry, which will perform this check.
+
+	return filedesc.BaseL0{
+		FullName:   fullName,
+		ParentFile: parent.ParentFile().(*filedesc.File),
+		Parent:     parent,
+		Index:      idx,
+	}, nil
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protodesc/desc_resolve.go b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_resolve.go
new file mode 100644
index 0000000..cebb36c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_resolve.go
@@ -0,0 +1,286 @@
+// 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.
+
+package protodesc
+
+import (
+	"google.golang.org/protobuf/internal/encoding/defval"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/reflect/protoreflect"
+	"google.golang.org/protobuf/reflect/protoregistry"
+
+	"google.golang.org/protobuf/types/descriptorpb"
+)
+
+// resolver is a wrapper around a local registry of declarations within the file
+// and the remote resolver. The remote resolver is restricted to only return
+// descriptors that have been imported.
+type resolver struct {
+	local   descsByName
+	remote  Resolver
+	imports importSet
+
+	allowUnresolvable bool
+}
+
+func (r *resolver) resolveMessageDependencies(ms []filedesc.Message, mds []*descriptorpb.DescriptorProto) (err error) {
+	for i, md := range mds {
+		m := &ms[i]
+		for j, fd := range md.GetField() {
+			f := &m.L2.Fields.List[j]
+			if f.L1.Cardinality == protoreflect.Required {
+				m.L2.RequiredNumbers.List = append(m.L2.RequiredNumbers.List, f.L1.Number)
+			}
+			if fd.OneofIndex != nil {
+				k := int(fd.GetOneofIndex())
+				if !(0 <= k && k < len(md.GetOneofDecl())) {
+					return errors.New("message field %q has an invalid oneof index: %d", f.FullName(), k)
+				}
+				o := &m.L2.Oneofs.List[k]
+				f.L1.ContainingOneof = o
+				o.L1.Fields.List = append(o.L1.Fields.List, f)
+			}
+
+			if f.L1.Kind, f.L1.Enum, f.L1.Message, err = r.findTarget(f.Kind(), f.Parent().FullName(), partialName(fd.GetTypeName()), f.IsWeak()); err != nil {
+				return errors.New("message field %q cannot resolve type: %v", f.FullName(), err)
+			}
+			if fd.DefaultValue != nil {
+				v, ev, err := unmarshalDefault(fd.GetDefaultValue(), f, r.allowUnresolvable)
+				if err != nil {
+					return errors.New("message field %q has invalid default: %v", f.FullName(), err)
+				}
+				f.L1.Default = filedesc.DefaultValue(v, ev)
+			}
+		}
+
+		if err := r.resolveMessageDependencies(m.L1.Messages.List, md.GetNestedType()); err != nil {
+			return err
+		}
+		if err := r.resolveExtensionDependencies(m.L1.Extensions.List, md.GetExtension()); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (r *resolver) resolveExtensionDependencies(xs []filedesc.Extension, xds []*descriptorpb.FieldDescriptorProto) (err error) {
+	for i, xd := range xds {
+		x := &xs[i]
+		if x.L1.Extendee, err = r.findMessageDescriptor(x.Parent().FullName(), partialName(xd.GetExtendee()), false); err != nil {
+			return errors.New("extension field %q cannot resolve extendee: %v", x.FullName(), err)
+		}
+		if x.L1.Kind, x.L2.Enum, x.L2.Message, err = r.findTarget(x.Kind(), x.Parent().FullName(), partialName(xd.GetTypeName()), false); err != nil {
+			return errors.New("extension field %q cannot resolve type: %v", x.FullName(), err)
+		}
+		if xd.DefaultValue != nil {
+			v, ev, err := unmarshalDefault(xd.GetDefaultValue(), x, r.allowUnresolvable)
+			if err != nil {
+				return errors.New("extension field %q has invalid default: %v", x.FullName(), err)
+			}
+			x.L2.Default = filedesc.DefaultValue(v, ev)
+		}
+	}
+	return nil
+}
+
+func (r *resolver) resolveServiceDependencies(ss []filedesc.Service, sds []*descriptorpb.ServiceDescriptorProto) (err error) {
+	for i, sd := range sds {
+		s := &ss[i]
+		for j, md := range sd.GetMethod() {
+			m := &s.L2.Methods.List[j]
+			m.L1.Input, err = r.findMessageDescriptor(m.Parent().FullName(), partialName(md.GetInputType()), false)
+			if err != nil {
+				return errors.New("service method %q cannot resolve input: %v", m.FullName(), err)
+			}
+			m.L1.Output, err = r.findMessageDescriptor(s.FullName(), partialName(md.GetOutputType()), false)
+			if err != nil {
+				return errors.New("service method %q cannot resolve output: %v", m.FullName(), err)
+			}
+		}
+	}
+	return nil
+}
+
+// findTarget finds an enum or message descriptor if k is an enum, message,
+// group, or unknown. If unknown, and the name could be resolved, the kind
+// returned kind is set based on the type of the resolved descriptor.
+func (r *resolver) findTarget(k protoreflect.Kind, scope protoreflect.FullName, ref partialName, isWeak bool) (protoreflect.Kind, protoreflect.EnumDescriptor, protoreflect.MessageDescriptor, error) {
+	switch k {
+	case protoreflect.EnumKind:
+		ed, err := r.findEnumDescriptor(scope, ref, isWeak)
+		if err != nil {
+			return 0, nil, nil, err
+		}
+		return k, ed, nil, nil
+	case protoreflect.MessageKind, protoreflect.GroupKind:
+		md, err := r.findMessageDescriptor(scope, ref, isWeak)
+		if err != nil {
+			return 0, nil, nil, err
+		}
+		return k, nil, md, nil
+	case 0:
+		// Handle unspecified kinds (possible with parsers that operate
+		// on a per-file basis without knowledge of dependencies).
+		d, err := r.findDescriptor(scope, ref)
+		if err == protoregistry.NotFound && (r.allowUnresolvable || isWeak) {
+			return k, filedesc.PlaceholderEnum(ref.FullName()), filedesc.PlaceholderMessage(ref.FullName()), nil
+		} else if err == protoregistry.NotFound {
+			return 0, nil, nil, errors.New("%q not found", ref.FullName())
+		} else if err != nil {
+			return 0, nil, nil, err
+		}
+		switch d := d.(type) {
+		case protoreflect.EnumDescriptor:
+			return protoreflect.EnumKind, d, nil, nil
+		case protoreflect.MessageDescriptor:
+			return protoreflect.MessageKind, nil, d, nil
+		default:
+			return 0, nil, nil, errors.New("unknown kind")
+		}
+	default:
+		if ref != "" {
+			return 0, nil, nil, errors.New("target name cannot be specified for %v", k)
+		}
+		if !k.IsValid() {
+			return 0, nil, nil, errors.New("invalid kind: %d", k)
+		}
+		return k, nil, nil, nil
+	}
+}
+
+// findDescriptor finds the descriptor by name,
+// which may be a relative name within some scope.
+//
+// Suppose the scope was "fizz.buzz" and the reference was "Foo.Bar",
+// then the following full names are searched:
+//	* fizz.buzz.Foo.Bar
+//	* fizz.Foo.Bar
+//	* Foo.Bar
+func (r *resolver) findDescriptor(scope protoreflect.FullName, ref partialName) (protoreflect.Descriptor, error) {
+	if !ref.IsValid() {
+		return nil, errors.New("invalid name reference: %q", ref)
+	}
+	if ref.IsFull() {
+		scope, ref = "", ref[1:]
+	}
+	var foundButNotImported protoreflect.Descriptor
+	for {
+		// Derive the full name to search.
+		s := protoreflect.FullName(ref)
+		if scope != "" {
+			s = scope + "." + s
+		}
+
+		// Check the current file for the descriptor.
+		if d, ok := r.local[s]; ok {
+			return d, nil
+		}
+
+		// Check the remote registry for the descriptor.
+		d, err := r.remote.FindDescriptorByName(s)
+		if err == nil {
+			// Only allow descriptors covered by one of the imports.
+			if r.imports[d.ParentFile().Path()] {
+				return d, nil
+			}
+			foundButNotImported = d
+		} else if err != protoregistry.NotFound {
+			return nil, errors.Wrap(err, "%q", s)
+		}
+
+		// Continue on at a higher level of scoping.
+		if scope == "" {
+			if d := foundButNotImported; d != nil {
+				return nil, errors.New("resolved %q, but %q is not imported", d.FullName(), d.ParentFile().Path())
+			}
+			return nil, protoregistry.NotFound
+		}
+		scope = scope.Parent()
+	}
+}
+
+func (r *resolver) findEnumDescriptor(scope protoreflect.FullName, ref partialName, isWeak bool) (protoreflect.EnumDescriptor, error) {
+	d, err := r.findDescriptor(scope, ref)
+	if err == protoregistry.NotFound && (r.allowUnresolvable || isWeak) {
+		return filedesc.PlaceholderEnum(ref.FullName()), nil
+	} else if err == protoregistry.NotFound {
+		return nil, errors.New("%q not found", ref.FullName())
+	} else if err != nil {
+		return nil, err
+	}
+	ed, ok := d.(protoreflect.EnumDescriptor)
+	if !ok {
+		return nil, errors.New("resolved %q, but it is not an enum", d.FullName())
+	}
+	return ed, nil
+}
+
+func (r *resolver) findMessageDescriptor(scope protoreflect.FullName, ref partialName, isWeak bool) (protoreflect.MessageDescriptor, error) {
+	d, err := r.findDescriptor(scope, ref)
+	if err == protoregistry.NotFound && (r.allowUnresolvable || isWeak) {
+		return filedesc.PlaceholderMessage(ref.FullName()), nil
+	} else if err == protoregistry.NotFound {
+		return nil, errors.New("%q not found", ref.FullName())
+	} else if err != nil {
+		return nil, err
+	}
+	md, ok := d.(protoreflect.MessageDescriptor)
+	if !ok {
+		return nil, errors.New("resolved %q, but it is not an message", d.FullName())
+	}
+	return md, nil
+}
+
+// partialName is the partial name. A leading dot means that the name is full,
+// otherwise the name is relative to some current scope.
+// See google.protobuf.FieldDescriptorProto.type_name.
+type partialName string
+
+func (s partialName) IsFull() bool {
+	return len(s) > 0 && s[0] == '.'
+}
+
+func (s partialName) IsValid() bool {
+	if s.IsFull() {
+		return protoreflect.FullName(s[1:]).IsValid()
+	}
+	return protoreflect.FullName(s).IsValid()
+}
+
+const unknownPrefix = "*."
+
+// FullName converts the partial name to a full name on a best-effort basis.
+// If relative, it creates an invalid full name, using a "*." prefix
+// to indicate that the start of the full name is unknown.
+func (s partialName) FullName() protoreflect.FullName {
+	if s.IsFull() {
+		return protoreflect.FullName(s[1:])
+	}
+	return protoreflect.FullName(unknownPrefix + s)
+}
+
+func unmarshalDefault(s string, fd protoreflect.FieldDescriptor, allowUnresolvable bool) (protoreflect.Value, protoreflect.EnumValueDescriptor, error) {
+	var evs protoreflect.EnumValueDescriptors
+	if fd.Enum() != nil {
+		evs = fd.Enum().Values()
+	}
+	v, ev, err := defval.Unmarshal(s, fd.Kind(), evs, defval.Descriptor)
+	if err != nil && allowUnresolvable && evs != nil && protoreflect.Name(s).IsValid() {
+		v = protoreflect.ValueOfEnum(0)
+		if evs.Len() > 0 {
+			v = protoreflect.ValueOfEnum(evs.Get(0).Number())
+		}
+		ev = filedesc.PlaceholderEnumValue(fd.Enum().FullName().Parent().Append(protoreflect.Name(s)))
+	} else if err != nil {
+		return v, ev, err
+	}
+	if fd.Syntax() == protoreflect.Proto3 {
+		return v, ev, errors.New("cannot be specified under proto3 semantics")
+	}
+	if fd.Kind() == protoreflect.MessageKind || fd.Kind() == protoreflect.GroupKind || fd.Cardinality() == protoreflect.Repeated {
+		return v, ev, errors.New("cannot be specified on composite types")
+	}
+	return v, ev, nil
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protodesc/desc_validate.go b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_validate.go
new file mode 100644
index 0000000..9af1d56
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protodesc/desc_validate.go
@@ -0,0 +1,374 @@
+// 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.
+
+package protodesc
+
+import (
+	"strings"
+	"unicode"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/internal/genid"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/reflect/protoreflect"
+
+	"google.golang.org/protobuf/types/descriptorpb"
+)
+
+func validateEnumDeclarations(es []filedesc.Enum, eds []*descriptorpb.EnumDescriptorProto) error {
+	for i, ed := range eds {
+		e := &es[i]
+		if err := e.L2.ReservedNames.CheckValid(); err != nil {
+			return errors.New("enum %q reserved names has %v", e.FullName(), err)
+		}
+		if err := e.L2.ReservedRanges.CheckValid(); err != nil {
+			return errors.New("enum %q reserved ranges has %v", e.FullName(), err)
+		}
+		if len(ed.GetValue()) == 0 {
+			return errors.New("enum %q must contain at least one value declaration", e.FullName())
+		}
+		allowAlias := ed.GetOptions().GetAllowAlias()
+		foundAlias := false
+		for i := 0; i < e.Values().Len(); i++ {
+			v1 := e.Values().Get(i)
+			if v2 := e.Values().ByNumber(v1.Number()); v1 != v2 {
+				foundAlias = true
+				if !allowAlias {
+					return errors.New("enum %q has conflicting non-aliased values on number %d: %q with %q", e.FullName(), v1.Number(), v1.Name(), v2.Name())
+				}
+			}
+		}
+		if allowAlias && !foundAlias {
+			return errors.New("enum %q allows aliases, but none were found", e.FullName())
+		}
+		if e.Syntax() == protoreflect.Proto3 {
+			if v := e.Values().Get(0); v.Number() != 0 {
+				return errors.New("enum %q using proto3 semantics must have zero number for the first value", v.FullName())
+			}
+			// Verify that value names in proto3 do not conflict if the
+			// case-insensitive prefix is removed.
+			// See protoc v3.8.0: src/google/protobuf/descriptor.cc:4991-5055
+			names := map[string]protoreflect.EnumValueDescriptor{}
+			prefix := strings.Replace(strings.ToLower(string(e.Name())), "_", "", -1)
+			for i := 0; i < e.Values().Len(); i++ {
+				v1 := e.Values().Get(i)
+				s := strs.EnumValueName(strs.TrimEnumPrefix(string(v1.Name()), prefix))
+				if v2, ok := names[s]; ok && v1.Number() != v2.Number() {
+					return errors.New("enum %q using proto3 semantics has conflict: %q with %q", e.FullName(), v1.Name(), v2.Name())
+				}
+				names[s] = v1
+			}
+		}
+
+		for j, vd := range ed.GetValue() {
+			v := &e.L2.Values.List[j]
+			if vd.Number == nil {
+				return errors.New("enum value %q must have a specified number", v.FullName())
+			}
+			if e.L2.ReservedNames.Has(v.Name()) {
+				return errors.New("enum value %q must not use reserved name", v.FullName())
+			}
+			if e.L2.ReservedRanges.Has(v.Number()) {
+				return errors.New("enum value %q must not use reserved number %d", v.FullName(), v.Number())
+			}
+		}
+	}
+	return nil
+}
+
+func validateMessageDeclarations(ms []filedesc.Message, mds []*descriptorpb.DescriptorProto) error {
+	for i, md := range mds {
+		m := &ms[i]
+
+		// Handle the message descriptor itself.
+		isMessageSet := md.GetOptions().GetMessageSetWireFormat()
+		if err := m.L2.ReservedNames.CheckValid(); err != nil {
+			return errors.New("message %q reserved names has %v", m.FullName(), err)
+		}
+		if err := m.L2.ReservedRanges.CheckValid(isMessageSet); err != nil {
+			return errors.New("message %q reserved ranges has %v", m.FullName(), err)
+		}
+		if err := m.L2.ExtensionRanges.CheckValid(isMessageSet); err != nil {
+			return errors.New("message %q extension ranges has %v", m.FullName(), err)
+		}
+		if err := (*filedesc.FieldRanges).CheckOverlap(&m.L2.ReservedRanges, &m.L2.ExtensionRanges); err != nil {
+			return errors.New("message %q reserved and extension ranges has %v", m.FullName(), err)
+		}
+		for i := 0; i < m.Fields().Len(); i++ {
+			f1 := m.Fields().Get(i)
+			if f2 := m.Fields().ByNumber(f1.Number()); f1 != f2 {
+				return errors.New("message %q has conflicting fields: %q with %q", m.FullName(), f1.Name(), f2.Name())
+			}
+		}
+		if isMessageSet && !flags.ProtoLegacy {
+			return errors.New("message %q is a MessageSet, which is a legacy proto1 feature that is no longer supported", m.FullName())
+		}
+		if isMessageSet && (m.Syntax() != protoreflect.Proto2 || m.Fields().Len() > 0 || m.ExtensionRanges().Len() == 0) {
+			return errors.New("message %q is an invalid proto1 MessageSet", m.FullName())
+		}
+		if m.Syntax() == protoreflect.Proto3 {
+			if m.ExtensionRanges().Len() > 0 {
+				return errors.New("message %q using proto3 semantics cannot have extension ranges", m.FullName())
+			}
+			// Verify that field names in proto3 do not conflict if lowercased
+			// with all underscores removed.
+			// See protoc v3.8.0: src/google/protobuf/descriptor.cc:5830-5847
+			names := map[string]protoreflect.FieldDescriptor{}
+			for i := 0; i < m.Fields().Len(); i++ {
+				f1 := m.Fields().Get(i)
+				s := strings.Replace(strings.ToLower(string(f1.Name())), "_", "", -1)
+				if f2, ok := names[s]; ok {
+					return errors.New("message %q using proto3 semantics has conflict: %q with %q", m.FullName(), f1.Name(), f2.Name())
+				}
+				names[s] = f1
+			}
+		}
+
+		for j, fd := range md.GetField() {
+			f := &m.L2.Fields.List[j]
+			if m.L2.ReservedNames.Has(f.Name()) {
+				return errors.New("message field %q must not use reserved name", f.FullName())
+			}
+			if !f.Number().IsValid() {
+				return errors.New("message field %q has an invalid number: %d", f.FullName(), f.Number())
+			}
+			if !f.Cardinality().IsValid() {
+				return errors.New("message field %q has an invalid cardinality: %d", f.FullName(), f.Cardinality())
+			}
+			if m.L2.ReservedRanges.Has(f.Number()) {
+				return errors.New("message field %q must not use reserved number %d", f.FullName(), f.Number())
+			}
+			if m.L2.ExtensionRanges.Has(f.Number()) {
+				return errors.New("message field %q with number %d in extension range", f.FullName(), f.Number())
+			}
+			if fd.Extendee != nil {
+				return errors.New("message field %q may not have extendee: %q", f.FullName(), fd.GetExtendee())
+			}
+			if f.L1.IsProto3Optional {
+				if f.Syntax() != protoreflect.Proto3 {
+					return errors.New("message field %q under proto3 optional semantics must be specified in the proto3 syntax", f.FullName())
+				}
+				if f.Cardinality() != protoreflect.Optional {
+					return errors.New("message field %q under proto3 optional semantics must have optional cardinality", f.FullName())
+				}
+				if f.ContainingOneof() != nil && f.ContainingOneof().Fields().Len() != 1 {
+					return errors.New("message field %q under proto3 optional semantics must be within a single element oneof", f.FullName())
+				}
+			}
+			if f.IsWeak() && !flags.ProtoLegacy {
+				return errors.New("message field %q is a weak field, which is a legacy proto1 feature that is no longer supported", f.FullName())
+			}
+			if f.IsWeak() && (f.Syntax() != protoreflect.Proto2 || !isOptionalMessage(f) || f.ContainingOneof() != nil) {
+				return errors.New("message field %q may only be weak for an optional message", f.FullName())
+			}
+			if f.IsPacked() && !isPackable(f) {
+				return errors.New("message field %q is not packable", f.FullName())
+			}
+			if err := checkValidGroup(f); err != nil {
+				return errors.New("message field %q is an invalid group: %v", f.FullName(), err)
+			}
+			if err := checkValidMap(f); err != nil {
+				return errors.New("message field %q is an invalid map: %v", f.FullName(), err)
+			}
+			if f.Syntax() == protoreflect.Proto3 {
+				if f.Cardinality() == protoreflect.Required {
+					return errors.New("message field %q using proto3 semantics cannot be required", f.FullName())
+				}
+				if f.Enum() != nil && !f.Enum().IsPlaceholder() && f.Enum().Syntax() != protoreflect.Proto3 {
+					return errors.New("message field %q using proto3 semantics may only depend on a proto3 enum", f.FullName())
+				}
+			}
+		}
+		seenSynthetic := false // synthetic oneofs for proto3 optional must come after real oneofs
+		for j := range md.GetOneofDecl() {
+			o := &m.L2.Oneofs.List[j]
+			if o.Fields().Len() == 0 {
+				return errors.New("message oneof %q must contain at least one field declaration", o.FullName())
+			}
+			if n := o.Fields().Len(); n-1 != (o.Fields().Get(n-1).Index() - o.Fields().Get(0).Index()) {
+				return errors.New("message oneof %q must have consecutively declared fields", o.FullName())
+			}
+
+			if o.IsSynthetic() {
+				seenSynthetic = true
+				continue
+			}
+			if !o.IsSynthetic() && seenSynthetic {
+				return errors.New("message oneof %q must be declared before synthetic oneofs", o.FullName())
+			}
+
+			for i := 0; i < o.Fields().Len(); i++ {
+				f := o.Fields().Get(i)
+				if f.Cardinality() != protoreflect.Optional {
+					return errors.New("message field %q belongs in a oneof and must be optional", f.FullName())
+				}
+				if f.IsWeak() {
+					return errors.New("message field %q belongs in a oneof and must not be a weak reference", f.FullName())
+				}
+			}
+		}
+
+		if err := validateEnumDeclarations(m.L1.Enums.List, md.GetEnumType()); err != nil {
+			return err
+		}
+		if err := validateMessageDeclarations(m.L1.Messages.List, md.GetNestedType()); err != nil {
+			return err
+		}
+		if err := validateExtensionDeclarations(m.L1.Extensions.List, md.GetExtension()); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func validateExtensionDeclarations(xs []filedesc.Extension, xds []*descriptorpb.FieldDescriptorProto) error {
+	for i, xd := range xds {
+		x := &xs[i]
+		// NOTE: Avoid using the IsValid method since extensions to MessageSet
+		// may have a field number higher than normal. This check only verifies
+		// that the number is not negative or reserved. We check again later
+		// if we know that the extendee is definitely not a MessageSet.
+		if n := x.Number(); n < 0 || (protowire.FirstReservedNumber <= n && n <= protowire.LastReservedNumber) {
+			return errors.New("extension field %q has an invalid number: %d", x.FullName(), x.Number())
+		}
+		if !x.Cardinality().IsValid() || x.Cardinality() == protoreflect.Required {
+			return errors.New("extension field %q has an invalid cardinality: %d", x.FullName(), x.Cardinality())
+		}
+		if xd.JsonName != nil {
+			// A bug in older versions of protoc would always populate the
+			// "json_name" option for extensions when it is meaningless.
+			// When it did so, it would always use the camel-cased field name.
+			if xd.GetJsonName() != strs.JSONCamelCase(string(x.Name())) {
+				return errors.New("extension field %q may not have an explicitly set JSON name: %q", x.FullName(), xd.GetJsonName())
+			}
+		}
+		if xd.OneofIndex != nil {
+			return errors.New("extension field %q may not be part of a oneof", x.FullName())
+		}
+		if md := x.ContainingMessage(); !md.IsPlaceholder() {
+			if !md.ExtensionRanges().Has(x.Number()) {
+				return errors.New("extension field %q extends %q with non-extension field number: %d", x.FullName(), md.FullName(), x.Number())
+			}
+			isMessageSet := md.Options().(*descriptorpb.MessageOptions).GetMessageSetWireFormat()
+			if isMessageSet && !isOptionalMessage(x) {
+				return errors.New("extension field %q extends MessageSet and must be an optional message", x.FullName())
+			}
+			if !isMessageSet && !x.Number().IsValid() {
+				return errors.New("extension field %q has an invalid number: %d", x.FullName(), x.Number())
+			}
+		}
+		if xd.GetOptions().GetWeak() {
+			return errors.New("extension field %q cannot be a weak reference", x.FullName())
+		}
+		if x.IsPacked() && !isPackable(x) {
+			return errors.New("extension field %q is not packable", x.FullName())
+		}
+		if err := checkValidGroup(x); err != nil {
+			return errors.New("extension field %q is an invalid group: %v", x.FullName(), err)
+		}
+		if md := x.Message(); md != nil && md.IsMapEntry() {
+			return errors.New("extension field %q cannot be a map entry", x.FullName())
+		}
+		if x.Syntax() == protoreflect.Proto3 {
+			switch x.ContainingMessage().FullName() {
+			case (*descriptorpb.FileOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.EnumOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.EnumValueOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.MessageOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.FieldOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.OneofOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.ExtensionRangeOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.ServiceOptions)(nil).ProtoReflect().Descriptor().FullName():
+			case (*descriptorpb.MethodOptions)(nil).ProtoReflect().Descriptor().FullName():
+			default:
+				return errors.New("extension field %q cannot be declared in proto3 unless extended descriptor options", x.FullName())
+			}
+		}
+	}
+	return nil
+}
+
+// isOptionalMessage reports whether this is an optional message.
+// If the kind is unknown, it is assumed to be a message.
+func isOptionalMessage(fd protoreflect.FieldDescriptor) bool {
+	return (fd.Kind() == 0 || fd.Kind() == protoreflect.MessageKind) && fd.Cardinality() == protoreflect.Optional
+}
+
+// isPackable checks whether the pack option can be specified.
+func isPackable(fd protoreflect.FieldDescriptor) bool {
+	switch fd.Kind() {
+	case protoreflect.StringKind, protoreflect.BytesKind, protoreflect.MessageKind, protoreflect.GroupKind:
+		return false
+	}
+	return fd.IsList()
+}
+
+// checkValidGroup reports whether fd is a valid group according to the same
+// rules that protoc imposes.
+func checkValidGroup(fd protoreflect.FieldDescriptor) error {
+	md := fd.Message()
+	switch {
+	case fd.Kind() != protoreflect.GroupKind:
+		return nil
+	case fd.Syntax() != protoreflect.Proto2:
+		return errors.New("invalid under proto2 semantics")
+	case md == nil || md.IsPlaceholder():
+		return errors.New("message must be resolvable")
+	case fd.FullName().Parent() != md.FullName().Parent():
+		return errors.New("message and field must be declared in the same scope")
+	case !unicode.IsUpper(rune(md.Name()[0])):
+		return errors.New("message name must start with an uppercase")
+	case fd.Name() != protoreflect.Name(strings.ToLower(string(md.Name()))):
+		return errors.New("field name must be lowercased form of the message name")
+	}
+	return nil
+}
+
+// checkValidMap checks whether the field is a valid map according to the same
+// rules that protoc imposes.
+// See protoc v3.8.0: src/google/protobuf/descriptor.cc:6045-6115
+func checkValidMap(fd protoreflect.FieldDescriptor) error {
+	md := fd.Message()
+	switch {
+	case md == nil || !md.IsMapEntry():
+		return nil
+	case fd.FullName().Parent() != md.FullName().Parent():
+		return errors.New("message and field must be declared in the same scope")
+	case md.Name() != protoreflect.Name(strs.MapEntryName(string(fd.Name()))):
+		return errors.New("incorrect implicit map entry name")
+	case fd.Cardinality() != protoreflect.Repeated:
+		return errors.New("field must be repeated")
+	case md.Fields().Len() != 2:
+		return errors.New("message must have exactly two fields")
+	case md.ExtensionRanges().Len() > 0:
+		return errors.New("message must not have any extension ranges")
+	case md.Enums().Len()+md.Messages().Len()+md.Extensions().Len() > 0:
+		return errors.New("message must not have any nested declarations")
+	}
+	kf := md.Fields().Get(0)
+	vf := md.Fields().Get(1)
+	switch {
+	case kf.Name() != genid.MapEntry_Key_field_name || kf.Number() != genid.MapEntry_Key_field_number || kf.Cardinality() != protoreflect.Optional || kf.ContainingOneof() != nil || kf.HasDefault():
+		return errors.New("invalid key field")
+	case vf.Name() != genid.MapEntry_Value_field_name || vf.Number() != genid.MapEntry_Value_field_number || vf.Cardinality() != protoreflect.Optional || vf.ContainingOneof() != nil || vf.HasDefault():
+		return errors.New("invalid value field")
+	}
+	switch kf.Kind() {
+	case protoreflect.BoolKind: // bool
+	case protoreflect.Int32Kind, protoreflect.Sint32Kind, protoreflect.Sfixed32Kind: // int32
+	case protoreflect.Int64Kind, protoreflect.Sint64Kind, protoreflect.Sfixed64Kind: // int64
+	case protoreflect.Uint32Kind, protoreflect.Fixed32Kind: // uint32
+	case protoreflect.Uint64Kind, protoreflect.Fixed64Kind: // uint64
+	case protoreflect.StringKind: // string
+	default:
+		return errors.New("invalid key kind: %v", kf.Kind())
+	}
+	if e := vf.Enum(); e != nil && e.Values().Len() > 0 && e.Values().Get(0).Number() != 0 {
+		return errors.New("map enum value must have zero number for the first value")
+	}
+	return nil
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protodesc/proto.go b/vendor/google.golang.org/protobuf/reflect/protodesc/proto.go
new file mode 100644
index 0000000..a7c5cef
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protodesc/proto.go
@@ -0,0 +1,252 @@
+// 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.
+
+package protodesc
+
+import (
+	"fmt"
+	"strings"
+
+	"google.golang.org/protobuf/internal/encoding/defval"
+	"google.golang.org/protobuf/internal/strs"
+	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/reflect/protoreflect"
+
+	"google.golang.org/protobuf/types/descriptorpb"
+)
+
+// ToFileDescriptorProto copies a protoreflect.FileDescriptor into a
+// google.protobuf.FileDescriptorProto message.
+func ToFileDescriptorProto(file protoreflect.FileDescriptor) *descriptorpb.FileDescriptorProto {
+	p := &descriptorpb.FileDescriptorProto{
+		Name:    proto.String(file.Path()),
+		Options: proto.Clone(file.Options()).(*descriptorpb.FileOptions),
+	}
+	if file.Package() != "" {
+		p.Package = proto.String(string(file.Package()))
+	}
+	for i, imports := 0, file.Imports(); i < imports.Len(); i++ {
+		imp := imports.Get(i)
+		p.Dependency = append(p.Dependency, imp.Path())
+		if imp.IsPublic {
+			p.PublicDependency = append(p.PublicDependency, int32(i))
+		}
+		if imp.IsWeak {
+			p.WeakDependency = append(p.WeakDependency, int32(i))
+		}
+	}
+	for i, locs := 0, file.SourceLocations(); i < locs.Len(); i++ {
+		loc := locs.Get(i)
+		l := &descriptorpb.SourceCodeInfo_Location{}
+		l.Path = append(l.Path, loc.Path...)
+		if loc.StartLine == loc.EndLine {
+			l.Span = []int32{int32(loc.StartLine), int32(loc.StartColumn), int32(loc.EndColumn)}
+		} else {
+			l.Span = []int32{int32(loc.StartLine), int32(loc.StartColumn), int32(loc.EndLine), int32(loc.EndColumn)}
+		}
+		l.LeadingDetachedComments = append([]string(nil), loc.LeadingDetachedComments...)
+		if loc.LeadingComments != "" {
+			l.LeadingComments = proto.String(loc.LeadingComments)
+		}
+		if loc.TrailingComments != "" {
+			l.TrailingComments = proto.String(loc.TrailingComments)
+		}
+		if p.SourceCodeInfo == nil {
+			p.SourceCodeInfo = &descriptorpb.SourceCodeInfo{}
+		}
+		p.SourceCodeInfo.Location = append(p.SourceCodeInfo.Location, l)
+
+	}
+	for i, messages := 0, file.Messages(); i < messages.Len(); i++ {
+		p.MessageType = append(p.MessageType, ToDescriptorProto(messages.Get(i)))
+	}
+	for i, enums := 0, file.Enums(); i < enums.Len(); i++ {
+		p.EnumType = append(p.EnumType, ToEnumDescriptorProto(enums.Get(i)))
+	}
+	for i, services := 0, file.Services(); i < services.Len(); i++ {
+		p.Service = append(p.Service, ToServiceDescriptorProto(services.Get(i)))
+	}
+	for i, exts := 0, file.Extensions(); i < exts.Len(); i++ {
+		p.Extension = append(p.Extension, ToFieldDescriptorProto(exts.Get(i)))
+	}
+	if syntax := file.Syntax(); syntax != protoreflect.Proto2 {
+		p.Syntax = proto.String(file.Syntax().String())
+	}
+	return p
+}
+
+// ToDescriptorProto copies a protoreflect.MessageDescriptor into a
+// google.protobuf.DescriptorProto message.
+func ToDescriptorProto(message protoreflect.MessageDescriptor) *descriptorpb.DescriptorProto {
+	p := &descriptorpb.DescriptorProto{
+		Name:    proto.String(string(message.Name())),
+		Options: proto.Clone(message.Options()).(*descriptorpb.MessageOptions),
+	}
+	for i, fields := 0, message.Fields(); i < fields.Len(); i++ {
+		p.Field = append(p.Field, ToFieldDescriptorProto(fields.Get(i)))
+	}
+	for i, exts := 0, message.Extensions(); i < exts.Len(); i++ {
+		p.Extension = append(p.Extension, ToFieldDescriptorProto(exts.Get(i)))
+	}
+	for i, messages := 0, message.Messages(); i < messages.Len(); i++ {
+		p.NestedType = append(p.NestedType, ToDescriptorProto(messages.Get(i)))
+	}
+	for i, enums := 0, message.Enums(); i < enums.Len(); i++ {
+		p.EnumType = append(p.EnumType, ToEnumDescriptorProto(enums.Get(i)))
+	}
+	for i, xranges := 0, message.ExtensionRanges(); i < xranges.Len(); i++ {
+		xrange := xranges.Get(i)
+		p.ExtensionRange = append(p.ExtensionRange, &descriptorpb.DescriptorProto_ExtensionRange{
+			Start:   proto.Int32(int32(xrange[0])),
+			End:     proto.Int32(int32(xrange[1])),
+			Options: proto.Clone(message.ExtensionRangeOptions(i)).(*descriptorpb.ExtensionRangeOptions),
+		})
+	}
+	for i, oneofs := 0, message.Oneofs(); i < oneofs.Len(); i++ {
+		p.OneofDecl = append(p.OneofDecl, ToOneofDescriptorProto(oneofs.Get(i)))
+	}
+	for i, ranges := 0, message.ReservedRanges(); i < ranges.Len(); i++ {
+		rrange := ranges.Get(i)
+		p.ReservedRange = append(p.ReservedRange, &descriptorpb.DescriptorProto_ReservedRange{
+			Start: proto.Int32(int32(rrange[0])),
+			End:   proto.Int32(int32(rrange[1])),
+		})
+	}
+	for i, names := 0, message.ReservedNames(); i < names.Len(); i++ {
+		p.ReservedName = append(p.ReservedName, string(names.Get(i)))
+	}
+	return p
+}
+
+// ToFieldDescriptorProto copies a protoreflect.FieldDescriptor into a
+// google.protobuf.FieldDescriptorProto message.
+func ToFieldDescriptorProto(field protoreflect.FieldDescriptor) *descriptorpb.FieldDescriptorProto {
+	p := &descriptorpb.FieldDescriptorProto{
+		Name:    proto.String(string(field.Name())),
+		Number:  proto.Int32(int32(field.Number())),
+		Label:   descriptorpb.FieldDescriptorProto_Label(field.Cardinality()).Enum(),
+		Options: proto.Clone(field.Options()).(*descriptorpb.FieldOptions),
+	}
+	if field.IsExtension() {
+		p.Extendee = fullNameOf(field.ContainingMessage())
+	}
+	if field.Kind().IsValid() {
+		p.Type = descriptorpb.FieldDescriptorProto_Type(field.Kind()).Enum()
+	}
+	if field.Enum() != nil {
+		p.TypeName = fullNameOf(field.Enum())
+	}
+	if field.Message() != nil {
+		p.TypeName = fullNameOf(field.Message())
+	}
+	if field.HasJSONName() {
+		// A bug in older versions of protoc would always populate the
+		// "json_name" option for extensions when it is meaningless.
+		// When it did so, it would always use the camel-cased field name.
+		if field.IsExtension() {
+			p.JsonName = proto.String(strs.JSONCamelCase(string(field.Name())))
+		} else {
+			p.JsonName = proto.String(field.JSONName())
+		}
+	}
+	if field.Syntax() == protoreflect.Proto3 && field.HasOptionalKeyword() {
+		p.Proto3Optional = proto.Bool(true)
+	}
+	if field.HasDefault() {
+		def, err := defval.Marshal(field.Default(), field.DefaultEnumValue(), field.Kind(), defval.Descriptor)
+		if err != nil && field.DefaultEnumValue() != nil {
+			def = string(field.DefaultEnumValue().Name()) // occurs for unresolved enum values
+		} else if err != nil {
+			panic(fmt.Sprintf("%v: %v", field.FullName(), err))
+		}
+		p.DefaultValue = proto.String(def)
+	}
+	if oneof := field.ContainingOneof(); oneof != nil {
+		p.OneofIndex = proto.Int32(int32(oneof.Index()))
+	}
+	return p
+}
+
+// ToOneofDescriptorProto copies a protoreflect.OneofDescriptor into a
+// google.protobuf.OneofDescriptorProto message.
+func ToOneofDescriptorProto(oneof protoreflect.OneofDescriptor) *descriptorpb.OneofDescriptorProto {
+	return &descriptorpb.OneofDescriptorProto{
+		Name:    proto.String(string(oneof.Name())),
+		Options: proto.Clone(oneof.Options()).(*descriptorpb.OneofOptions),
+	}
+}
+
+// ToEnumDescriptorProto copies a protoreflect.EnumDescriptor into a
+// google.protobuf.EnumDescriptorProto message.
+func ToEnumDescriptorProto(enum protoreflect.EnumDescriptor) *descriptorpb.EnumDescriptorProto {
+	p := &descriptorpb.EnumDescriptorProto{
+		Name:    proto.String(string(enum.Name())),
+		Options: proto.Clone(enum.Options()).(*descriptorpb.EnumOptions),
+	}
+	for i, values := 0, enum.Values(); i < values.Len(); i++ {
+		p.Value = append(p.Value, ToEnumValueDescriptorProto(values.Get(i)))
+	}
+	for i, ranges := 0, enum.ReservedRanges(); i < ranges.Len(); i++ {
+		rrange := ranges.Get(i)
+		p.ReservedRange = append(p.ReservedRange, &descriptorpb.EnumDescriptorProto_EnumReservedRange{
+			Start: proto.Int32(int32(rrange[0])),
+			End:   proto.Int32(int32(rrange[1])),
+		})
+	}
+	for i, names := 0, enum.ReservedNames(); i < names.Len(); i++ {
+		p.ReservedName = append(p.ReservedName, string(names.Get(i)))
+	}
+	return p
+}
+
+// ToEnumValueDescriptorProto copies a protoreflect.EnumValueDescriptor into a
+// google.protobuf.EnumValueDescriptorProto message.
+func ToEnumValueDescriptorProto(value protoreflect.EnumValueDescriptor) *descriptorpb.EnumValueDescriptorProto {
+	return &descriptorpb.EnumValueDescriptorProto{
+		Name:    proto.String(string(value.Name())),
+		Number:  proto.Int32(int32(value.Number())),
+		Options: proto.Clone(value.Options()).(*descriptorpb.EnumValueOptions),
+	}
+}
+
+// ToServiceDescriptorProto copies a protoreflect.ServiceDescriptor into a
+// google.protobuf.ServiceDescriptorProto message.
+func ToServiceDescriptorProto(service protoreflect.ServiceDescriptor) *descriptorpb.ServiceDescriptorProto {
+	p := &descriptorpb.ServiceDescriptorProto{
+		Name:    proto.String(string(service.Name())),
+		Options: proto.Clone(service.Options()).(*descriptorpb.ServiceOptions),
+	}
+	for i, methods := 0, service.Methods(); i < methods.Len(); i++ {
+		p.Method = append(p.Method, ToMethodDescriptorProto(methods.Get(i)))
+	}
+	return p
+}
+
+// ToMethodDescriptorProto copies a protoreflect.MethodDescriptor into a
+// google.protobuf.MethodDescriptorProto message.
+func ToMethodDescriptorProto(method protoreflect.MethodDescriptor) *descriptorpb.MethodDescriptorProto {
+	p := &descriptorpb.MethodDescriptorProto{
+		Name:       proto.String(string(method.Name())),
+		InputType:  fullNameOf(method.Input()),
+		OutputType: fullNameOf(method.Output()),
+		Options:    proto.Clone(method.Options()).(*descriptorpb.MethodOptions),
+	}
+	if method.IsStreamingClient() {
+		p.ClientStreaming = proto.Bool(true)
+	}
+	if method.IsStreamingServer() {
+		p.ServerStreaming = proto.Bool(true)
+	}
+	return p
+}
+
+func fullNameOf(d protoreflect.Descriptor) *string {
+	if d == nil {
+		return nil
+	}
+	if strings.HasPrefix(string(d.FullName()), unknownPrefix) {
+		return proto.String(string(d.FullName()[len(unknownPrefix):]))
+	}
+	return proto.String("." + string(d.FullName()))
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/methods.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/methods.go
new file mode 100644
index 0000000..6be5d16
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/methods.go
@@ -0,0 +1,77 @@
+// Copyright 2020 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 protoreflect
+
+import (
+	"google.golang.org/protobuf/internal/pragma"
+)
+
+// The following types are used by the fast-path Message.ProtoMethods method.
+//
+// To avoid polluting the public protoreflect API with types used only by
+// low-level implementations, the canonical definitions of these types are
+// in the runtime/protoiface package. The definitions here and in protoiface
+// must be kept in sync.
+type (
+	methods = struct {
+		pragma.NoUnkeyedLiterals
+		Flags            supportFlags
+		Size             func(sizeInput) sizeOutput
+		Marshal          func(marshalInput) (marshalOutput, error)
+		Unmarshal        func(unmarshalInput) (unmarshalOutput, error)
+		Merge            func(mergeInput) mergeOutput
+		CheckInitialized func(checkInitializedInput) (checkInitializedOutput, error)
+	}
+	supportFlags = uint64
+	sizeInput    = struct {
+		pragma.NoUnkeyedLiterals
+		Message Message
+		Flags   uint8
+	}
+	sizeOutput = struct {
+		pragma.NoUnkeyedLiterals
+		Size int
+	}
+	marshalInput = struct {
+		pragma.NoUnkeyedLiterals
+		Message Message
+		Buf     []byte
+		Flags   uint8
+	}
+	marshalOutput = struct {
+		pragma.NoUnkeyedLiterals
+		Buf []byte
+	}
+	unmarshalInput = struct {
+		pragma.NoUnkeyedLiterals
+		Message  Message
+		Buf      []byte
+		Flags    uint8
+		Resolver interface {
+			FindExtensionByName(field FullName) (ExtensionType, error)
+			FindExtensionByNumber(message FullName, field FieldNumber) (ExtensionType, error)
+		}
+	}
+	unmarshalOutput = struct {
+		pragma.NoUnkeyedLiterals
+		Flags uint8
+	}
+	mergeInput = struct {
+		pragma.NoUnkeyedLiterals
+		Source      Message
+		Destination Message
+	}
+	mergeOutput = struct {
+		pragma.NoUnkeyedLiterals
+		Flags uint8
+	}
+	checkInitializedInput = struct {
+		pragma.NoUnkeyedLiterals
+		Message Message
+	}
+	checkInitializedOutput = struct {
+		pragma.NoUnkeyedLiterals
+	}
+)
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/proto.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/proto.go
new file mode 100644
index 0000000..dd85915
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/proto.go
@@ -0,0 +1,504 @@
+// 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 protoreflect provides interfaces to dynamically manipulate messages.
+//
+// This package includes type descriptors which describe the structure of types
+// defined in proto source files and value interfaces which provide the
+// ability to examine and manipulate the contents of messages.
+//
+//
+// Protocol Buffer Descriptors
+//
+// Protobuf descriptors (e.g., EnumDescriptor or MessageDescriptor)
+// are immutable objects that represent protobuf type information.
+// They are wrappers around the messages declared in descriptor.proto.
+// Protobuf descriptors alone lack any information regarding Go types.
+//
+// Enums and messages generated by this module implement Enum and ProtoMessage,
+// where the Descriptor and ProtoReflect.Descriptor accessors respectively
+// return the protobuf descriptor for the values.
+//
+// The protobuf descriptor interfaces are not meant to be implemented by
+// user code since they might need to be extended in the future to support
+// additions to the protobuf language.
+// The "google.golang.org/protobuf/reflect/protodesc" package converts between
+// google.protobuf.DescriptorProto messages and protobuf descriptors.
+//
+//
+// Go Type Descriptors
+//
+// A type descriptor (e.g., EnumType or MessageType) is a constructor for
+// a concrete Go type that represents the associated protobuf descriptor.
+// There is commonly a one-to-one relationship between protobuf descriptors and
+// Go type descriptors, but it can potentially be a one-to-many relationship.
+//
+// Enums and messages generated by this module implement Enum and ProtoMessage,
+// where the Type and ProtoReflect.Type accessors respectively
+// return the protobuf descriptor for the values.
+//
+// The "google.golang.org/protobuf/types/dynamicpb" package can be used to
+// create Go type descriptors from protobuf descriptors.
+//
+//
+// Value Interfaces
+//
+// The Enum and Message interfaces provide a reflective view over an
+// enum or message instance. For enums, it provides the ability to retrieve
+// the enum value number for any concrete enum type. For messages, it provides
+// the ability to access or manipulate fields of the message.
+//
+// To convert a proto.Message to a protoreflect.Message, use the
+// former's ProtoReflect method. Since the ProtoReflect method is new to the
+// v2 message interface, it may not be present on older message implementations.
+// The "github.com/golang/protobuf/proto".MessageReflect function can be used
+// to obtain a reflective view on older messages.
+//
+//
+// Relationships
+//
+// The following diagrams demonstrate the relationships between
+// various types declared in this package.
+//
+//
+//	                       ┌───────────────────────────────────┐
+//	                       V                                   │
+//	   ┌────────────── New(n) ─────────────┐                   │
+//	   │                                   │                   │
+//	   │      ┌──── Descriptor() ──┐       │  ┌── Number() ──┐ │
+//	   │      │                    V       V  │              V │
+//	╔════════════╗  ╔════════════════╗  ╔════════╗  ╔════════════╗
+//	║  EnumType  ║  ║ EnumDescriptor ║  ║  Enum  ║  ║ EnumNumber ║
+//	╚════════════╝  ╚════════════════╝  ╚════════╝  ╚════════════╝
+//	      Λ           Λ                   │ │
+//	      │           └─── Descriptor() ──┘ │
+//	      │                                 │
+//	      └────────────────── Type() ───────┘
+//
+// • An EnumType describes a concrete Go enum type.
+// It has an EnumDescriptor and can construct an Enum instance.
+//
+// • An EnumDescriptor describes an abstract protobuf enum type.
+//
+// • An Enum is a concrete enum instance. Generated enums implement Enum.
+//
+//
+//	  ┌──────────────── New() ─────────────────┐
+//	  │                                        │
+//	  │         ┌─── Descriptor() ─────┐       │   ┌── Interface() ───┐
+//	  │         │                      V       V   │                  V
+//	╔═════════════╗  ╔═══════════════════╗  ╔═════════╗  ╔══════════════╗
+//	║ MessageType ║  ║ MessageDescriptor ║  ║ Message ║  ║ ProtoMessage ║
+//	╚═════════════╝  ╚═══════════════════╝  ╚═════════╝  ╚══════════════╝
+//	       Λ           Λ                      │ │  Λ                  │
+//	       │           └──── Descriptor() ────┘ │  └─ ProtoReflect() ─┘
+//	       │                                    │
+//	       └─────────────────── Type() ─────────┘
+//
+// • A MessageType describes a concrete Go message type.
+// It has a MessageDescriptor and can construct a Message instance.
+//
+// • A MessageDescriptor describes an abstract protobuf message type.
+//
+// • A Message is a concrete message instance. Generated messages implement
+// ProtoMessage, which can convert to/from a Message.
+//
+//
+//	      ┌── TypeDescriptor() ──┐    ┌───── Descriptor() ─────┐
+//	      │                      V    │                        V
+//	╔═══════════════╗  ╔═════════════════════════╗  ╔═════════════════════╗
+//	║ ExtensionType ║  ║ ExtensionTypeDescriptor ║  ║ ExtensionDescriptor ║
+//	╚═══════════════╝  ╚═════════════════════════╝  ╚═════════════════════╝
+//	      Λ                      │   │ Λ                      │ Λ
+//	      └─────── Type() ───────┘   │ └─── may implement ────┘ │
+//	                                 │                          │
+//	                                 └────── implements ────────┘
+//
+// • An ExtensionType describes a concrete Go implementation of an extension.
+// It has an ExtensionTypeDescriptor and can convert to/from
+// abstract Values and Go values.
+//
+// • An ExtensionTypeDescriptor is an ExtensionDescriptor
+// which also has an ExtensionType.
+//
+// • An ExtensionDescriptor describes an abstract protobuf extension field and
+// may not always be an ExtensionTypeDescriptor.
+package protoreflect
+
+import (
+	"fmt"
+	"strings"
+
+	"google.golang.org/protobuf/encoding/protowire"
+	"google.golang.org/protobuf/internal/pragma"
+)
+
+type doNotImplement pragma.DoNotImplement
+
+// ProtoMessage is the top-level interface that all proto messages implement.
+// This is declared in the protoreflect package to avoid a cyclic dependency;
+// use the proto.Message type instead, which aliases this type.
+type ProtoMessage interface{ ProtoReflect() Message }
+
+// Syntax is the language version of the proto file.
+type Syntax syntax
+
+type syntax int8 // keep exact type opaque as the int type may change
+
+const (
+	Proto2 Syntax = 2
+	Proto3 Syntax = 3
+)
+
+// IsValid reports whether the syntax is valid.
+func (s Syntax) IsValid() bool {
+	switch s {
+	case Proto2, Proto3:
+		return true
+	default:
+		return false
+	}
+}
+
+// String returns s as a proto source identifier (e.g., "proto2").
+func (s Syntax) String() string {
+	switch s {
+	case Proto2:
+		return "proto2"
+	case Proto3:
+		return "proto3"
+	default:
+		return fmt.Sprintf("<unknown:%d>", s)
+	}
+}
+
+// GoString returns s as a Go source identifier (e.g., "Proto2").
+func (s Syntax) GoString() string {
+	switch s {
+	case Proto2:
+		return "Proto2"
+	case Proto3:
+		return "Proto3"
+	default:
+		return fmt.Sprintf("Syntax(%d)", s)
+	}
+}
+
+// Cardinality determines whether a field is optional, required, or repeated.
+type Cardinality cardinality
+
+type cardinality int8 // keep exact type opaque as the int type may change
+
+// Constants as defined by the google.protobuf.Cardinality enumeration.
+const (
+	Optional Cardinality = 1 // appears zero or one times
+	Required Cardinality = 2 // appears exactly one time; invalid with Proto3
+	Repeated Cardinality = 3 // appears zero or more times
+)
+
+// IsValid reports whether the cardinality is valid.
+func (c Cardinality) IsValid() bool {
+	switch c {
+	case Optional, Required, Repeated:
+		return true
+	default:
+		return false
+	}
+}
+
+// String returns c as a proto source identifier (e.g., "optional").
+func (c Cardinality) String() string {
+	switch c {
+	case Optional:
+		return "optional"
+	case Required:
+		return "required"
+	case Repeated:
+		return "repeated"
+	default:
+		return fmt.Sprintf("<unknown:%d>", c)
+	}
+}
+
+// GoString returns c as a Go source identifier (e.g., "Optional").
+func (c Cardinality) GoString() string {
+	switch c {
+	case Optional:
+		return "Optional"
+	case Required:
+		return "Required"
+	case Repeated:
+		return "Repeated"
+	default:
+		return fmt.Sprintf("Cardinality(%d)", c)
+	}
+}
+
+// Kind indicates the basic proto kind of a field.
+type Kind kind
+
+type kind int8 // keep exact type opaque as the int type may change
+
+// Constants as defined by the google.protobuf.Field.Kind enumeration.
+const (
+	BoolKind     Kind = 8
+	EnumKind     Kind = 14
+	Int32Kind    Kind = 5
+	Sint32Kind   Kind = 17
+	Uint32Kind   Kind = 13
+	Int64Kind    Kind = 3
+	Sint64Kind   Kind = 18
+	Uint64Kind   Kind = 4
+	Sfixed32Kind Kind = 15
+	Fixed32Kind  Kind = 7
+	FloatKind    Kind = 2
+	Sfixed64Kind Kind = 16
+	Fixed64Kind  Kind = 6
+	DoubleKind   Kind = 1
+	StringKind   Kind = 9
+	BytesKind    Kind = 12
+	MessageKind  Kind = 11
+	GroupKind    Kind = 10
+)
+
+// IsValid reports whether the kind is valid.
+func (k Kind) IsValid() bool {
+	switch k {
+	case BoolKind, EnumKind,
+		Int32Kind, Sint32Kind, Uint32Kind,
+		Int64Kind, Sint64Kind, Uint64Kind,
+		Sfixed32Kind, Fixed32Kind, FloatKind,
+		Sfixed64Kind, Fixed64Kind, DoubleKind,
+		StringKind, BytesKind, MessageKind, GroupKind:
+		return true
+	default:
+		return false
+	}
+}
+
+// String returns k as a proto source identifier (e.g., "bool").
+func (k Kind) String() string {
+	switch k {
+	case BoolKind:
+		return "bool"
+	case EnumKind:
+		return "enum"
+	case Int32Kind:
+		return "int32"
+	case Sint32Kind:
+		return "sint32"
+	case Uint32Kind:
+		return "uint32"
+	case Int64Kind:
+		return "int64"
+	case Sint64Kind:
+		return "sint64"
+	case Uint64Kind:
+		return "uint64"
+	case Sfixed32Kind:
+		return "sfixed32"
+	case Fixed32Kind:
+		return "fixed32"
+	case FloatKind:
+		return "float"
+	case Sfixed64Kind:
+		return "sfixed64"
+	case Fixed64Kind:
+		return "fixed64"
+	case DoubleKind:
+		return "double"
+	case StringKind:
+		return "string"
+	case BytesKind:
+		return "bytes"
+	case MessageKind:
+		return "message"
+	case GroupKind:
+		return "group"
+	default:
+		return fmt.Sprintf("<unknown:%d>", k)
+	}
+}
+
+// GoString returns k as a Go source identifier (e.g., "BoolKind").
+func (k Kind) GoString() string {
+	switch k {
+	case BoolKind:
+		return "BoolKind"
+	case EnumKind:
+		return "EnumKind"
+	case Int32Kind:
+		return "Int32Kind"
+	case Sint32Kind:
+		return "Sint32Kind"
+	case Uint32Kind:
+		return "Uint32Kind"
+	case Int64Kind:
+		return "Int64Kind"
+	case Sint64Kind:
+		return "Sint64Kind"
+	case Uint64Kind:
+		return "Uint64Kind"
+	case Sfixed32Kind:
+		return "Sfixed32Kind"
+	case Fixed32Kind:
+		return "Fixed32Kind"
+	case FloatKind:
+		return "FloatKind"
+	case Sfixed64Kind:
+		return "Sfixed64Kind"
+	case Fixed64Kind:
+		return "Fixed64Kind"
+	case DoubleKind:
+		return "DoubleKind"
+	case StringKind:
+		return "StringKind"
+	case BytesKind:
+		return "BytesKind"
+	case MessageKind:
+		return "MessageKind"
+	case GroupKind:
+		return "GroupKind"
+	default:
+		return fmt.Sprintf("Kind(%d)", k)
+	}
+}
+
+// FieldNumber is the field number in a message.
+type FieldNumber = protowire.Number
+
+// FieldNumbers represent a list of field numbers.
+type FieldNumbers interface {
+	// Len reports the number of fields in the list.
+	Len() int
+	// Get returns the ith field number. It panics if out of bounds.
+	Get(i int) FieldNumber
+	// Has reports whether n is within the list of fields.
+	Has(n FieldNumber) bool
+
+	doNotImplement
+}
+
+// FieldRanges represent a list of field number ranges.
+type FieldRanges interface {
+	// Len reports the number of ranges in the list.
+	Len() int
+	// Get returns the ith range. It panics if out of bounds.
+	Get(i int) [2]FieldNumber // start inclusive; end exclusive
+	// Has reports whether n is within any of the ranges.
+	Has(n FieldNumber) bool
+
+	doNotImplement
+}
+
+// EnumNumber is the numeric value for an enum.
+type EnumNumber int32
+
+// EnumRanges represent a list of enum number ranges.
+type EnumRanges interface {
+	// Len reports the number of ranges in the list.
+	Len() int
+	// Get returns the ith range. It panics if out of bounds.
+	Get(i int) [2]EnumNumber // start inclusive; end inclusive
+	// Has reports whether n is within any of the ranges.
+	Has(n EnumNumber) bool
+
+	doNotImplement
+}
+
+// Name is the short name for a proto declaration. This is not the name
+// as used in Go source code, which might not be identical to the proto name.
+type Name string // e.g., "Kind"
+
+// IsValid reports whether s is a syntactically valid name.
+// An empty name is invalid.
+func (s Name) IsValid() bool {
+	return consumeIdent(string(s)) == len(s)
+}
+
+// Names represent a list of names.
+type Names interface {
+	// Len reports the number of names in the list.
+	Len() int
+	// Get returns the ith name. It panics if out of bounds.
+	Get(i int) Name
+	// Has reports whether s matches any names in the list.
+	Has(s Name) bool
+
+	doNotImplement
+}
+
+// FullName is a qualified name that uniquely identifies a proto declaration.
+// A qualified name is the concatenation of the proto package along with the
+// fully-declared name (i.e., name of parent preceding the name of the child),
+// with a '.' delimiter placed between each Name.
+//
+// This should not have any leading or trailing dots.
+type FullName string // e.g., "google.protobuf.Field.Kind"
+
+// IsValid reports whether s is a syntactically valid full name.
+// An empty full name is invalid.
+func (s FullName) IsValid() bool {
+	i := consumeIdent(string(s))
+	if i < 0 {
+		return false
+	}
+	for len(s) > i {
+		if s[i] != '.' {
+			return false
+		}
+		i++
+		n := consumeIdent(string(s[i:]))
+		if n < 0 {
+			return false
+		}
+		i += n
+	}
+	return true
+}
+
+func consumeIdent(s string) (i int) {
+	if len(s) == 0 || !isLetter(s[i]) {
+		return -1
+	}
+	i++
+	for len(s) > i && isLetterDigit(s[i]) {
+		i++
+	}
+	return i
+}
+func isLetter(c byte) bool {
+	return c == '_' || ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z')
+}
+func isLetterDigit(c byte) bool {
+	return isLetter(c) || ('0' <= c && c <= '9')
+}
+
+// Name returns the short name, which is the last identifier segment.
+// A single segment FullName is the Name itself.
+func (n FullName) Name() Name {
+	if i := strings.LastIndexByte(string(n), '.'); i >= 0 {
+		return Name(n[i+1:])
+	}
+	return Name(n)
+}
+
+// Parent returns the full name with the trailing identifier removed.
+// A single segment FullName has no parent.
+func (n FullName) Parent() FullName {
+	if i := strings.LastIndexByte(string(n), '.'); i >= 0 {
+		return n[:i]
+	}
+	return ""
+}
+
+// Append returns the qualified name appended with the provided short name.
+//
+// Invariant: n == n.Parent().Append(n.Name()) // assuming n is valid
+func (n FullName) Append(s Name) FullName {
+	if n == "" {
+		return FullName(s)
+	}
+	return n + "." + FullName(s)
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/source.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/source.go
new file mode 100644
index 0000000..121ba3a
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/source.go
@@ -0,0 +1,128 @@
+// 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.
+
+package protoreflect
+
+import (
+	"strconv"
+)
+
+// SourceLocations is a list of source locations.
+type SourceLocations interface {
+	// Len reports the number of source locations in the proto file.
+	Len() int
+	// Get returns the ith SourceLocation. It panics if out of bounds.
+	Get(int) SourceLocation
+
+	// ByPath returns the SourceLocation for the given path,
+	// returning the first location if multiple exist for the same path.
+	// If multiple locations exist for the same path,
+	// then SourceLocation.Next index can be used to identify the
+	// index of the next SourceLocation.
+	// If no location exists for this path, it returns the zero value.
+	ByPath(path SourcePath) SourceLocation
+
+	// ByDescriptor returns the SourceLocation for the given descriptor,
+	// returning the first location if multiple exist for the same path.
+	// If no location exists for this descriptor, it returns the zero value.
+	ByDescriptor(desc Descriptor) SourceLocation
+
+	doNotImplement
+}
+
+// SourceLocation describes a source location and
+// corresponds with the google.protobuf.SourceCodeInfo.Location message.
+type SourceLocation struct {
+	// Path is the path to the declaration from the root file descriptor.
+	// The contents of this slice must not be mutated.
+	Path SourcePath
+
+	// StartLine and StartColumn are the zero-indexed starting location
+	// in the source file for the declaration.
+	StartLine, StartColumn int
+	// EndLine and EndColumn are the zero-indexed ending location
+	// in the source file for the declaration.
+	// In the descriptor.proto, the end line may be omitted if it is identical
+	// to the start line. Here, it is always populated.
+	EndLine, EndColumn int
+
+	// LeadingDetachedComments are the leading detached comments
+	// for the declaration. The contents of this slice must not be mutated.
+	LeadingDetachedComments []string
+	// LeadingComments is the leading attached comment for the declaration.
+	LeadingComments string
+	// TrailingComments is the trailing attached comment for the declaration.
+	TrailingComments string
+
+	// Next is an index into SourceLocations for the next source location that
+	// has the same Path. It is zero if there is no next location.
+	Next int
+}
+
+// SourcePath identifies part of a file descriptor for a source location.
+// The SourcePath is a sequence of either field numbers or indexes into
+// a repeated field that form a path starting from the root file descriptor.
+//
+// See google.protobuf.SourceCodeInfo.Location.path.
+type SourcePath []int32
+
+// Equal reports whether p1 equals p2.
+func (p1 SourcePath) Equal(p2 SourcePath) bool {
+	if len(p1) != len(p2) {
+		return false
+	}
+	for i := range p1 {
+		if p1[i] != p2[i] {
+			return false
+		}
+	}
+	return true
+}
+
+// String formats the path in a humanly readable manner.
+// The output is guaranteed to be deterministic,
+// making it suitable for use as a key into a Go map.
+// It is not guaranteed to be stable as the exact output could change
+// in a future version of this module.
+//
+// Example output:
+//	.message_type[6].nested_type[15].field[3]
+func (p SourcePath) String() string {
+	b := p.appendFileDescriptorProto(nil)
+	for _, i := range p {
+		b = append(b, '.')
+		b = strconv.AppendInt(b, int64(i), 10)
+	}
+	return string(b)
+}
+
+type appendFunc func(*SourcePath, []byte) []byte
+
+func (p *SourcePath) appendSingularField(b []byte, name string, f appendFunc) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	b = append(b, '.')
+	b = append(b, name...)
+	*p = (*p)[1:]
+	if f != nil {
+		b = f(p, b)
+	}
+	return b
+}
+
+func (p *SourcePath) appendRepeatedField(b []byte, name string, f appendFunc) []byte {
+	b = p.appendSingularField(b, name, nil)
+	if len(*p) == 0 || (*p)[0] < 0 {
+		return b
+	}
+	b = append(b, '[')
+	b = strconv.AppendUint(b, uint64((*p)[0]), 10)
+	b = append(b, ']')
+	*p = (*p)[1:]
+	if f != nil {
+		b = f(p, b)
+	}
+	return b
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/source_gen.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/source_gen.go
new file mode 100644
index 0000000..b03c122
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/source_gen.go
@@ -0,0 +1,461 @@
+// 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.
+
+// Code generated by generate-protos. DO NOT EDIT.
+
+package protoreflect
+
+func (p *SourcePath) appendFileDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendSingularField(b, "package", nil)
+	case 3:
+		b = p.appendRepeatedField(b, "dependency", nil)
+	case 10:
+		b = p.appendRepeatedField(b, "public_dependency", nil)
+	case 11:
+		b = p.appendRepeatedField(b, "weak_dependency", nil)
+	case 4:
+		b = p.appendRepeatedField(b, "message_type", (*SourcePath).appendDescriptorProto)
+	case 5:
+		b = p.appendRepeatedField(b, "enum_type", (*SourcePath).appendEnumDescriptorProto)
+	case 6:
+		b = p.appendRepeatedField(b, "service", (*SourcePath).appendServiceDescriptorProto)
+	case 7:
+		b = p.appendRepeatedField(b, "extension", (*SourcePath).appendFieldDescriptorProto)
+	case 8:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendFileOptions)
+	case 9:
+		b = p.appendSingularField(b, "source_code_info", (*SourcePath).appendSourceCodeInfo)
+	case 12:
+		b = p.appendSingularField(b, "syntax", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendRepeatedField(b, "field", (*SourcePath).appendFieldDescriptorProto)
+	case 6:
+		b = p.appendRepeatedField(b, "extension", (*SourcePath).appendFieldDescriptorProto)
+	case 3:
+		b = p.appendRepeatedField(b, "nested_type", (*SourcePath).appendDescriptorProto)
+	case 4:
+		b = p.appendRepeatedField(b, "enum_type", (*SourcePath).appendEnumDescriptorProto)
+	case 5:
+		b = p.appendRepeatedField(b, "extension_range", (*SourcePath).appendDescriptorProto_ExtensionRange)
+	case 8:
+		b = p.appendRepeatedField(b, "oneof_decl", (*SourcePath).appendOneofDescriptorProto)
+	case 7:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendMessageOptions)
+	case 9:
+		b = p.appendRepeatedField(b, "reserved_range", (*SourcePath).appendDescriptorProto_ReservedRange)
+	case 10:
+		b = p.appendRepeatedField(b, "reserved_name", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendEnumDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendRepeatedField(b, "value", (*SourcePath).appendEnumValueDescriptorProto)
+	case 3:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendEnumOptions)
+	case 4:
+		b = p.appendRepeatedField(b, "reserved_range", (*SourcePath).appendEnumDescriptorProto_EnumReservedRange)
+	case 5:
+		b = p.appendRepeatedField(b, "reserved_name", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendServiceDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendRepeatedField(b, "method", (*SourcePath).appendMethodDescriptorProto)
+	case 3:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendServiceOptions)
+	}
+	return b
+}
+
+func (p *SourcePath) appendFieldDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 3:
+		b = p.appendSingularField(b, "number", nil)
+	case 4:
+		b = p.appendSingularField(b, "label", nil)
+	case 5:
+		b = p.appendSingularField(b, "type", nil)
+	case 6:
+		b = p.appendSingularField(b, "type_name", nil)
+	case 2:
+		b = p.appendSingularField(b, "extendee", nil)
+	case 7:
+		b = p.appendSingularField(b, "default_value", nil)
+	case 9:
+		b = p.appendSingularField(b, "oneof_index", nil)
+	case 10:
+		b = p.appendSingularField(b, "json_name", nil)
+	case 8:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendFieldOptions)
+	case 17:
+		b = p.appendSingularField(b, "proto3_optional", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendFileOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "java_package", nil)
+	case 8:
+		b = p.appendSingularField(b, "java_outer_classname", nil)
+	case 10:
+		b = p.appendSingularField(b, "java_multiple_files", nil)
+	case 20:
+		b = p.appendSingularField(b, "java_generate_equals_and_hash", nil)
+	case 27:
+		b = p.appendSingularField(b, "java_string_check_utf8", nil)
+	case 9:
+		b = p.appendSingularField(b, "optimize_for", nil)
+	case 11:
+		b = p.appendSingularField(b, "go_package", nil)
+	case 16:
+		b = p.appendSingularField(b, "cc_generic_services", nil)
+	case 17:
+		b = p.appendSingularField(b, "java_generic_services", nil)
+	case 18:
+		b = p.appendSingularField(b, "py_generic_services", nil)
+	case 42:
+		b = p.appendSingularField(b, "php_generic_services", nil)
+	case 23:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 31:
+		b = p.appendSingularField(b, "cc_enable_arenas", nil)
+	case 36:
+		b = p.appendSingularField(b, "objc_class_prefix", nil)
+	case 37:
+		b = p.appendSingularField(b, "csharp_namespace", nil)
+	case 39:
+		b = p.appendSingularField(b, "swift_prefix", nil)
+	case 40:
+		b = p.appendSingularField(b, "php_class_prefix", nil)
+	case 41:
+		b = p.appendSingularField(b, "php_namespace", nil)
+	case 44:
+		b = p.appendSingularField(b, "php_metadata_namespace", nil)
+	case 45:
+		b = p.appendSingularField(b, "ruby_package", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendSourceCodeInfo(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendRepeatedField(b, "location", (*SourcePath).appendSourceCodeInfo_Location)
+	}
+	return b
+}
+
+func (p *SourcePath) appendDescriptorProto_ExtensionRange(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "start", nil)
+	case 2:
+		b = p.appendSingularField(b, "end", nil)
+	case 3:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendExtensionRangeOptions)
+	}
+	return b
+}
+
+func (p *SourcePath) appendOneofDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendOneofOptions)
+	}
+	return b
+}
+
+func (p *SourcePath) appendMessageOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "message_set_wire_format", nil)
+	case 2:
+		b = p.appendSingularField(b, "no_standard_descriptor_accessor", nil)
+	case 3:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 7:
+		b = p.appendSingularField(b, "map_entry", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendDescriptorProto_ReservedRange(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "start", nil)
+	case 2:
+		b = p.appendSingularField(b, "end", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendEnumValueDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendSingularField(b, "number", nil)
+	case 3:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendEnumValueOptions)
+	}
+	return b
+}
+
+func (p *SourcePath) appendEnumOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 2:
+		b = p.appendSingularField(b, "allow_alias", nil)
+	case 3:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendEnumDescriptorProto_EnumReservedRange(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "start", nil)
+	case 2:
+		b = p.appendSingularField(b, "end", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendMethodDescriptorProto(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name", nil)
+	case 2:
+		b = p.appendSingularField(b, "input_type", nil)
+	case 3:
+		b = p.appendSingularField(b, "output_type", nil)
+	case 4:
+		b = p.appendSingularField(b, "options", (*SourcePath).appendMethodOptions)
+	case 5:
+		b = p.appendSingularField(b, "client_streaming", nil)
+	case 6:
+		b = p.appendSingularField(b, "server_streaming", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendServiceOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 33:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendFieldOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "ctype", nil)
+	case 2:
+		b = p.appendSingularField(b, "packed", nil)
+	case 6:
+		b = p.appendSingularField(b, "jstype", nil)
+	case 5:
+		b = p.appendSingularField(b, "lazy", nil)
+	case 3:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 10:
+		b = p.appendSingularField(b, "weak", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendUninterpretedOption(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 2:
+		b = p.appendRepeatedField(b, "name", (*SourcePath).appendUninterpretedOption_NamePart)
+	case 3:
+		b = p.appendSingularField(b, "identifier_value", nil)
+	case 4:
+		b = p.appendSingularField(b, "positive_int_value", nil)
+	case 5:
+		b = p.appendSingularField(b, "negative_int_value", nil)
+	case 6:
+		b = p.appendSingularField(b, "double_value", nil)
+	case 7:
+		b = p.appendSingularField(b, "string_value", nil)
+	case 8:
+		b = p.appendSingularField(b, "aggregate_value", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendSourceCodeInfo_Location(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendRepeatedField(b, "path", nil)
+	case 2:
+		b = p.appendRepeatedField(b, "span", nil)
+	case 3:
+		b = p.appendSingularField(b, "leading_comments", nil)
+	case 4:
+		b = p.appendSingularField(b, "trailing_comments", nil)
+	case 6:
+		b = p.appendRepeatedField(b, "leading_detached_comments", nil)
+	}
+	return b
+}
+
+func (p *SourcePath) appendExtensionRangeOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendOneofOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendEnumValueOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendMethodOptions(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 33:
+		b = p.appendSingularField(b, "deprecated", nil)
+	case 34:
+		b = p.appendSingularField(b, "idempotency_level", nil)
+	case 999:
+		b = p.appendRepeatedField(b, "uninterpreted_option", (*SourcePath).appendUninterpretedOption)
+	}
+	return b
+}
+
+func (p *SourcePath) appendUninterpretedOption_NamePart(b []byte) []byte {
+	if len(*p) == 0 {
+		return b
+	}
+	switch (*p)[0] {
+	case 1:
+		b = p.appendSingularField(b, "name_part", nil)
+	case 2:
+		b = p.appendSingularField(b, "is_extension", nil)
+	}
+	return b
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/type.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/type.go
new file mode 100644
index 0000000..8e53c44
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/type.go
@@ -0,0 +1,665 @@
+// 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 protoreflect
+
+// Descriptor provides a set of accessors that are common to every descriptor.
+// Each descriptor type wraps the equivalent google.protobuf.XXXDescriptorProto,
+// but provides efficient lookup and immutability.
+//
+// Each descriptor is comparable. Equality implies that the two types are
+// exactly identical. However, it is possible for the same semantically
+// identical proto type to be represented by multiple type descriptors.
+//
+// For example, suppose we have t1 and t2 which are both MessageDescriptors.
+// If t1 == t2, then the types are definitely equal and all accessors return
+// the same information. However, if t1 != t2, then it is still possible that
+// they still represent the same proto type (e.g., t1.FullName == t2.FullName).
+// This can occur if a descriptor type is created dynamically, or multiple
+// versions of the same proto type are accidentally linked into the Go binary.
+type Descriptor interface {
+	// ParentFile returns the parent file descriptor that this descriptor
+	// is declared within. The parent file for the file descriptor is itself.
+	//
+	// Support for this functionality is optional and may return nil.
+	ParentFile() FileDescriptor
+
+	// Parent returns the parent containing this descriptor declaration.
+	// The following shows the mapping from child type to possible parent types:
+	//
+	//	╔═════════════════════╤═══════════════════════════════════╗
+	//	║ Child type          │ Possible parent types             ║
+	//	╠═════════════════════╪═══════════════════════════════════╣
+	//	║ FileDescriptor      │ nil                               ║
+	//	║ MessageDescriptor   │ FileDescriptor, MessageDescriptor ║
+	//	║ FieldDescriptor     │ FileDescriptor, MessageDescriptor ║
+	//	║ OneofDescriptor     │ MessageDescriptor                 ║
+	//	║ EnumDescriptor      │ FileDescriptor, MessageDescriptor ║
+	//	║ EnumValueDescriptor │ EnumDescriptor                    ║
+	//	║ ServiceDescriptor   │ FileDescriptor                    ║
+	//	║ MethodDescriptor    │ ServiceDescriptor                 ║
+	//	╚═════════════════════╧═══════════════════════════════════╝
+	//
+	// Support for this functionality is optional and may return nil.
+	Parent() Descriptor
+
+	// Index returns the index of this descriptor within its parent.
+	// It returns 0 if the descriptor does not have a parent or if the parent
+	// is unknown.
+	Index() int
+
+	// Syntax is the protobuf syntax.
+	Syntax() Syntax // e.g., Proto2 or Proto3
+
+	// Name is the short name of the declaration (i.e., FullName.Name).
+	Name() Name // e.g., "Any"
+
+	// FullName is the fully-qualified name of the declaration.
+	//
+	// The FullName is a concatenation of the full name of the type that this
+	// type is declared within and the declaration name. For example,
+	// field "foo_field" in message "proto.package.MyMessage" is
+	// uniquely identified as "proto.package.MyMessage.foo_field".
+	// Enum values are an exception to the rule (see EnumValueDescriptor).
+	FullName() FullName // e.g., "google.protobuf.Any"
+
+	// IsPlaceholder reports whether type information is missing since a
+	// dependency is not resolved, in which case only name information is known.
+	//
+	// Placeholder types may only be returned by the following accessors
+	// as a result of unresolved dependencies or weak imports:
+	//
+	//	╔═══════════════════════════════════╤═════════════════════╗
+	//	║ Accessor                          │ Descriptor          ║
+	//	╠═══════════════════════════════════╪═════════════════════╣
+	//	║ FileImports.FileDescriptor        │ FileDescriptor      ║
+	//	║ FieldDescriptor.Enum              │ EnumDescriptor      ║
+	//	║ FieldDescriptor.Message           │ MessageDescriptor   ║
+	//	║ FieldDescriptor.DefaultEnumValue  │ EnumValueDescriptor ║
+	//	║ FieldDescriptor.ContainingMessage │ MessageDescriptor   ║
+	//	║ MethodDescriptor.Input            │ MessageDescriptor   ║
+	//	║ MethodDescriptor.Output           │ MessageDescriptor   ║
+	//	╚═══════════════════════════════════╧═════════════════════╝
+	//
+	// If true, only Name and FullName are valid.
+	// For FileDescriptor, the Path is also valid.
+	IsPlaceholder() bool
+
+	// Options returns the descriptor options. The caller must not modify
+	// the returned value.
+	//
+	// To avoid a dependency cycle, this function returns a proto.Message value.
+	// The proto message type returned for each descriptor type is as follows:
+	//	╔═════════════════════╤══════════════════════════════════════════╗
+	//	║ Go type             │ Protobuf message type                    ║
+	//	╠═════════════════════╪══════════════════════════════════════════╣
+	//	║ FileDescriptor      │ google.protobuf.FileOptions              ║
+	//	║ EnumDescriptor      │ google.protobuf.EnumOptions              ║
+	//	║ EnumValueDescriptor │ google.protobuf.EnumValueOptions         ║
+	//	║ MessageDescriptor   │ google.protobuf.MessageOptions           ║
+	//	║ FieldDescriptor     │ google.protobuf.FieldOptions             ║
+	//	║ OneofDescriptor     │ google.protobuf.OneofOptions             ║
+	//	║ ServiceDescriptor   │ google.protobuf.ServiceOptions           ║
+	//	║ MethodDescriptor    │ google.protobuf.MethodOptions            ║
+	//	╚═════════════════════╧══════════════════════════════════════════╝
+	//
+	// This method returns a typed nil-pointer if no options are present.
+	// The caller must import the descriptorpb package to use this.
+	Options() ProtoMessage
+
+	doNotImplement
+}
+
+// FileDescriptor describes the types in a complete proto file and
+// corresponds with the google.protobuf.FileDescriptorProto message.
+//
+// Top-level declarations:
+// EnumDescriptor, MessageDescriptor, FieldDescriptor, and/or ServiceDescriptor.
+type FileDescriptor interface {
+	Descriptor // Descriptor.FullName is identical to Package
+
+	// Path returns the file name, relative to the source tree root.
+	Path() string // e.g., "path/to/file.proto"
+	// Package returns the protobuf package namespace.
+	Package() FullName // e.g., "google.protobuf"
+
+	// Imports is a list of imported proto files.
+	Imports() FileImports
+
+	// Enums is a list of the top-level enum declarations.
+	Enums() EnumDescriptors
+	// Messages is a list of the top-level message declarations.
+	Messages() MessageDescriptors
+	// Extensions is a list of the top-level extension declarations.
+	Extensions() ExtensionDescriptors
+	// Services is a list of the top-level service declarations.
+	Services() ServiceDescriptors
+
+	// SourceLocations is a list of source locations.
+	SourceLocations() SourceLocations
+
+	isFileDescriptor
+}
+type isFileDescriptor interface{ ProtoType(FileDescriptor) }
+
+// FileImports is a list of file imports.
+type FileImports interface {
+	// Len reports the number of files imported by this proto file.
+	Len() int
+	// Get returns the ith FileImport. It panics if out of bounds.
+	Get(i int) FileImport
+
+	doNotImplement
+}
+
+// FileImport is the declaration for a proto file import.
+type FileImport struct {
+	// FileDescriptor is the file type for the given import.
+	// It is a placeholder descriptor if IsWeak is set or if a dependency has
+	// not been regenerated to implement the new reflection APIs.
+	FileDescriptor
+
+	// IsPublic reports whether this is a public import, which causes this file
+	// to alias declarations within the imported file. The intended use cases
+	// for this feature is the ability to move proto files without breaking
+	// existing dependencies.
+	//
+	// The current file and the imported file must be within proto package.
+	IsPublic bool
+
+	// IsWeak reports whether this is a weak import, which does not impose
+	// a direct dependency on the target file.
+	//
+	// Weak imports are a legacy proto1 feature. Equivalent behavior is
+	// achieved using proto2 extension fields or proto3 Any messages.
+	IsWeak bool
+}
+
+// MessageDescriptor describes a message and
+// corresponds with the google.protobuf.DescriptorProto message.
+//
+// Nested declarations:
+// FieldDescriptor, OneofDescriptor, FieldDescriptor, EnumDescriptor,
+// and/or MessageDescriptor.
+type MessageDescriptor interface {
+	Descriptor
+
+	// IsMapEntry indicates that this is an auto-generated message type to
+	// represent the entry type for a map field.
+	//
+	// Map entry messages have only two fields:
+	//	• a "key" field with a field number of 1
+	//	• a "value" field with a field number of 2
+	// The key and value types are determined by these two fields.
+	//
+	// If IsMapEntry is true, it implies that FieldDescriptor.IsMap is true
+	// for some field with this message type.
+	IsMapEntry() bool
+
+	// Fields is a list of nested field declarations.
+	Fields() FieldDescriptors
+	// Oneofs is a list of nested oneof declarations.
+	Oneofs() OneofDescriptors
+
+	// ReservedNames is a list of reserved field names.
+	ReservedNames() Names
+	// ReservedRanges is a list of reserved ranges of field numbers.
+	ReservedRanges() FieldRanges
+	// RequiredNumbers is a list of required field numbers.
+	// In Proto3, it is always an empty list.
+	RequiredNumbers() FieldNumbers
+	// ExtensionRanges is the field ranges used for extension fields.
+	// In Proto3, it is always an empty ranges.
+	ExtensionRanges() FieldRanges
+	// ExtensionRangeOptions returns the ith extension range options.
+	//
+	// To avoid a dependency cycle, this method returns a proto.Message value,
+	// which always contains a google.protobuf.ExtensionRangeOptions message.
+	// This method returns a typed nil-pointer if no options are present.
+	// The caller must import the descriptorpb package to use this.
+	ExtensionRangeOptions(i int) ProtoMessage
+
+	// Enums is a list of nested enum declarations.
+	Enums() EnumDescriptors
+	// Messages is a list of nested message declarations.
+	Messages() MessageDescriptors
+	// Extensions is a list of nested extension declarations.
+	Extensions() ExtensionDescriptors
+
+	isMessageDescriptor
+}
+type isMessageDescriptor interface{ ProtoType(MessageDescriptor) }
+
+// MessageType encapsulates a MessageDescriptor with a concrete Go implementation.
+// It is recommended that implementations of this interface also implement the
+// MessageFieldTypes interface.
+type MessageType interface {
+	// New returns a newly allocated empty message.
+	// It may return nil for synthetic messages representing a map entry.
+	New() Message
+
+	// Zero returns an empty, read-only message.
+	// It may return nil for synthetic messages representing a map entry.
+	Zero() Message
+
+	// Descriptor returns the message descriptor.
+	//
+	// Invariant: t.Descriptor() == t.New().Descriptor()
+	Descriptor() MessageDescriptor
+}
+
+// MessageFieldTypes extends a MessageType by providing type information
+// regarding enums and messages referenced by the message fields.
+type MessageFieldTypes interface {
+	MessageType
+
+	// Enum returns the EnumType for the ith field in Descriptor.Fields.
+	// It returns nil if the ith field is not an enum kind.
+	// It panics if out of bounds.
+	//
+	// Invariant: mt.Enum(i).Descriptor() == mt.Descriptor().Fields(i).Enum()
+	Enum(i int) EnumType
+
+	// Message returns the MessageType for the ith field in Descriptor.Fields.
+	// It returns nil if the ith field is not a message or group kind.
+	// It panics if out of bounds.
+	//
+	// Invariant: mt.Message(i).Descriptor() == mt.Descriptor().Fields(i).Message()
+	Message(i int) MessageType
+}
+
+// MessageDescriptors is a list of message declarations.
+type MessageDescriptors interface {
+	// Len reports the number of messages.
+	Len() int
+	// Get returns the ith MessageDescriptor. It panics if out of bounds.
+	Get(i int) MessageDescriptor
+	// ByName returns the MessageDescriptor for a message named s.
+	// It returns nil if not found.
+	ByName(s Name) MessageDescriptor
+
+	doNotImplement
+}
+
+// FieldDescriptor describes a field within a message and
+// corresponds with the google.protobuf.FieldDescriptorProto message.
+//
+// It is used for both normal fields defined within the parent message
+// (e.g., MessageDescriptor.Fields) and fields that extend some remote message
+// (e.g., FileDescriptor.Extensions or MessageDescriptor.Extensions).
+type FieldDescriptor interface {
+	Descriptor
+
+	// Number reports the unique number for this field.
+	Number() FieldNumber
+	// Cardinality reports the cardinality for this field.
+	Cardinality() Cardinality
+	// Kind reports the basic kind for this field.
+	Kind() Kind
+
+	// HasJSONName reports whether this field has an explicitly set JSON name.
+	HasJSONName() bool
+
+	// JSONName reports the name used for JSON serialization.
+	// It is usually the camel-cased form of the field name.
+	// Extension fields are represented by the full name surrounded by brackets.
+	JSONName() string
+
+	// TextName reports the name used for text serialization.
+	// It is usually the name of the field, except that groups use the name
+	// of the inlined message, and extension fields are represented by the
+	// full name surrounded by brackets.
+	TextName() string
+
+	// HasPresence reports whether the field distinguishes between unpopulated
+	// and default values.
+	HasPresence() bool
+
+	// IsExtension reports whether this is an extension field. If false,
+	// then Parent and ContainingMessage refer to the same message.
+	// Otherwise, ContainingMessage and Parent likely differ.
+	IsExtension() bool
+
+	// HasOptionalKeyword reports whether the "optional" keyword was explicitly
+	// specified in the source .proto file.
+	HasOptionalKeyword() bool
+
+	// IsWeak reports whether this is a weak field, which does not impose a
+	// direct dependency on the target type.
+	// If true, then Message returns a placeholder type.
+	IsWeak() bool
+
+	// IsPacked reports whether repeated primitive numeric kinds should be
+	// serialized using a packed encoding.
+	// If true, then it implies Cardinality is Repeated.
+	IsPacked() bool
+
+	// IsList reports whether this field represents a list,
+	// where the value type for the associated field is a List.
+	// It is equivalent to checking whether Cardinality is Repeated and
+	// that IsMap reports false.
+	IsList() bool
+
+	// IsMap reports whether this field represents a map,
+	// where the value type for the associated field is a Map.
+	// It is equivalent to checking whether Cardinality is Repeated,
+	// that the Kind is MessageKind, and that Message.IsMapEntry reports true.
+	IsMap() bool
+
+	// MapKey returns the field descriptor for the key in the map entry.
+	// It returns nil if IsMap reports false.
+	MapKey() FieldDescriptor
+
+	// MapValue returns the field descriptor for the value in the map entry.
+	// It returns nil if IsMap reports false.
+	MapValue() FieldDescriptor
+
+	// HasDefault reports whether this field has a default value.
+	HasDefault() bool
+
+	// Default returns the default value for scalar fields.
+	// For proto2, it is the default value as specified in the proto file,
+	// or the zero value if unspecified.
+	// For proto3, it is always the zero value of the scalar.
+	// The Value type is determined by the Kind.
+	Default() Value
+
+	// DefaultEnumValue returns the enum value descriptor for the default value
+	// of an enum field, and is nil for any other kind of field.
+	DefaultEnumValue() EnumValueDescriptor
+
+	// ContainingOneof is the containing oneof that this field belongs to,
+	// and is nil if this field is not part of a oneof.
+	ContainingOneof() OneofDescriptor
+
+	// ContainingMessage is the containing message that this field belongs to.
+	// For extension fields, this may not necessarily be the parent message
+	// that the field is declared within.
+	ContainingMessage() MessageDescriptor
+
+	// Enum is the enum descriptor if Kind is EnumKind.
+	// It returns nil for any other Kind.
+	Enum() EnumDescriptor
+
+	// Message is the message descriptor if Kind is
+	// MessageKind or GroupKind. It returns nil for any other Kind.
+	Message() MessageDescriptor
+
+	isFieldDescriptor
+}
+type isFieldDescriptor interface{ ProtoType(FieldDescriptor) }
+
+// FieldDescriptors is a list of field declarations.
+type FieldDescriptors interface {
+	// Len reports the number of fields.
+	Len() int
+	// Get returns the ith FieldDescriptor. It panics if out of bounds.
+	Get(i int) FieldDescriptor
+	// ByName returns the FieldDescriptor for a field named s.
+	// It returns nil if not found.
+	ByName(s Name) FieldDescriptor
+	// ByJSONName returns the FieldDescriptor for a field with s as the JSON name.
+	// It returns nil if not found.
+	ByJSONName(s string) FieldDescriptor
+	// ByTextName returns the FieldDescriptor for a field with s as the text name.
+	// It returns nil if not found.
+	ByTextName(s string) FieldDescriptor
+	// ByNumber returns the FieldDescriptor for a field numbered n.
+	// It returns nil if not found.
+	ByNumber(n FieldNumber) FieldDescriptor
+
+	doNotImplement
+}
+
+// OneofDescriptor describes a oneof field set within a given message and
+// corresponds with the google.protobuf.OneofDescriptorProto message.
+type OneofDescriptor interface {
+	Descriptor
+
+	// IsSynthetic reports whether this is a synthetic oneof created to support
+	// proto3 optional semantics. If true, Fields contains exactly one field
+	// with HasOptionalKeyword specified.
+	IsSynthetic() bool
+
+	// Fields is a list of fields belonging to this oneof.
+	Fields() FieldDescriptors
+
+	isOneofDescriptor
+}
+type isOneofDescriptor interface{ ProtoType(OneofDescriptor) }
+
+// OneofDescriptors is a list of oneof declarations.
+type OneofDescriptors interface {
+	// Len reports the number of oneof fields.
+	Len() int
+	// Get returns the ith OneofDescriptor. It panics if out of bounds.
+	Get(i int) OneofDescriptor
+	// ByName returns the OneofDescriptor for a oneof named s.
+	// It returns nil if not found.
+	ByName(s Name) OneofDescriptor
+
+	doNotImplement
+}
+
+// ExtensionDescriptor is an alias of FieldDescriptor for documentation.
+type ExtensionDescriptor = FieldDescriptor
+
+// ExtensionTypeDescriptor is an ExtensionDescriptor with an associated ExtensionType.
+type ExtensionTypeDescriptor interface {
+	ExtensionDescriptor
+
+	// Type returns the associated ExtensionType.
+	Type() ExtensionType
+
+	// Descriptor returns the plain ExtensionDescriptor without the
+	// associated ExtensionType.
+	Descriptor() ExtensionDescriptor
+}
+
+// ExtensionDescriptors is a list of field declarations.
+type ExtensionDescriptors interface {
+	// Len reports the number of fields.
+	Len() int
+	// Get returns the ith ExtensionDescriptor. It panics if out of bounds.
+	Get(i int) ExtensionDescriptor
+	// ByName returns the ExtensionDescriptor for a field named s.
+	// It returns nil if not found.
+	ByName(s Name) ExtensionDescriptor
+
+	doNotImplement
+}
+
+// ExtensionType encapsulates an ExtensionDescriptor with a concrete
+// Go implementation. The nested field descriptor must be for a extension field.
+//
+// While a normal field is a member of the parent message that it is declared
+// within (see Descriptor.Parent), an extension field is a member of some other
+// target message (see ExtensionDescriptor.Extendee) and may have no
+// relationship with the parent. However, the full name of an extension field is
+// relative to the parent that it is declared within.
+//
+// For example:
+//	syntax = "proto2";
+//	package example;
+//	message FooMessage {
+//		extensions 100 to max;
+//	}
+//	message BarMessage {
+//		extends FooMessage { optional BarMessage bar_field = 100; }
+//	}
+//
+// Field "bar_field" is an extension of FooMessage, but its full name is
+// "example.BarMessage.bar_field" instead of "example.FooMessage.bar_field".
+type ExtensionType interface {
+	// New returns a new value for the field.
+	// For scalars, this returns the default value in native Go form.
+	New() Value
+
+	// Zero returns a new value for the field.
+	// For scalars, this returns the default value in native Go form.
+	// For composite types, this returns an empty, read-only message, list, or map.
+	Zero() Value
+
+	// TypeDescriptor returns the extension type descriptor.
+	TypeDescriptor() ExtensionTypeDescriptor
+
+	// ValueOf wraps the input and returns it as a Value.
+	// ValueOf panics if the input value is invalid or not the appropriate type.
+	//
+	// ValueOf is more extensive than protoreflect.ValueOf for a given field's
+	// value as it has more type information available.
+	ValueOf(interface{}) Value
+
+	// InterfaceOf completely unwraps the Value to the underlying Go type.
+	// InterfaceOf panics if the input is nil or does not represent the
+	// appropriate underlying Go type. For composite types, it panics if the
+	// value is not mutable.
+	//
+	// InterfaceOf is able to unwrap the Value further than Value.Interface
+	// as it has more type information available.
+	InterfaceOf(Value) interface{}
+
+	// IsValidValue reports whether the Value is valid to assign to the field.
+	IsValidValue(Value) bool
+
+	// IsValidInterface reports whether the input is valid to assign to the field.
+	IsValidInterface(interface{}) bool
+}
+
+// EnumDescriptor describes an enum and
+// corresponds with the google.protobuf.EnumDescriptorProto message.
+//
+// Nested declarations:
+// EnumValueDescriptor.
+type EnumDescriptor interface {
+	Descriptor
+
+	// Values is a list of nested enum value declarations.
+	Values() EnumValueDescriptors
+
+	// ReservedNames is a list of reserved enum names.
+	ReservedNames() Names
+	// ReservedRanges is a list of reserved ranges of enum numbers.
+	ReservedRanges() EnumRanges
+
+	isEnumDescriptor
+}
+type isEnumDescriptor interface{ ProtoType(EnumDescriptor) }
+
+// EnumType encapsulates an EnumDescriptor with a concrete Go implementation.
+type EnumType interface {
+	// New returns an instance of this enum type with its value set to n.
+	New(n EnumNumber) Enum
+
+	// Descriptor returns the enum descriptor.
+	//
+	// Invariant: t.Descriptor() == t.New(0).Descriptor()
+	Descriptor() EnumDescriptor
+}
+
+// EnumDescriptors is a list of enum declarations.
+type EnumDescriptors interface {
+	// Len reports the number of enum types.
+	Len() int
+	// Get returns the ith EnumDescriptor. It panics if out of bounds.
+	Get(i int) EnumDescriptor
+	// ByName returns the EnumDescriptor for an enum named s.
+	// It returns nil if not found.
+	ByName(s Name) EnumDescriptor
+
+	doNotImplement
+}
+
+// EnumValueDescriptor describes an enum value and
+// corresponds with the google.protobuf.EnumValueDescriptorProto message.
+//
+// All other proto declarations are in the namespace of the parent.
+// However, enum values do not follow this rule and are within the namespace
+// of the parent's parent (i.e., they are a sibling of the containing enum).
+// Thus, a value named "FOO_VALUE" declared within an enum uniquely identified
+// as "proto.package.MyEnum" has a full name of "proto.package.FOO_VALUE".
+type EnumValueDescriptor interface {
+	Descriptor
+
+	// Number returns the enum value as an integer.
+	Number() EnumNumber
+
+	isEnumValueDescriptor
+}
+type isEnumValueDescriptor interface{ ProtoType(EnumValueDescriptor) }
+
+// EnumValueDescriptors is a list of enum value declarations.
+type EnumValueDescriptors interface {
+	// Len reports the number of enum values.
+	Len() int
+	// Get returns the ith EnumValueDescriptor. It panics if out of bounds.
+	Get(i int) EnumValueDescriptor
+	// ByName returns the EnumValueDescriptor for the enum value named s.
+	// It returns nil if not found.
+	ByName(s Name) EnumValueDescriptor
+	// ByNumber returns the EnumValueDescriptor for the enum value numbered n.
+	// If multiple have the same number, the first one defined is returned
+	// It returns nil if not found.
+	ByNumber(n EnumNumber) EnumValueDescriptor
+
+	doNotImplement
+}
+
+// ServiceDescriptor describes a service and
+// corresponds with the google.protobuf.ServiceDescriptorProto message.
+//
+// Nested declarations: MethodDescriptor.
+type ServiceDescriptor interface {
+	Descriptor
+
+	// Methods is a list of nested message declarations.
+	Methods() MethodDescriptors
+
+	isServiceDescriptor
+}
+type isServiceDescriptor interface{ ProtoType(ServiceDescriptor) }
+
+// ServiceDescriptors is a list of service declarations.
+type ServiceDescriptors interface {
+	// Len reports the number of services.
+	Len() int
+	// Get returns the ith ServiceDescriptor. It panics if out of bounds.
+	Get(i int) ServiceDescriptor
+	// ByName returns the ServiceDescriptor for a service named s.
+	// It returns nil if not found.
+	ByName(s Name) ServiceDescriptor
+
+	doNotImplement
+}
+
+// MethodDescriptor describes a method and
+// corresponds with the google.protobuf.MethodDescriptorProto message.
+type MethodDescriptor interface {
+	Descriptor
+
+	// Input is the input message descriptor.
+	Input() MessageDescriptor
+	// Output is the output message descriptor.
+	Output() MessageDescriptor
+	// IsStreamingClient reports whether the client streams multiple messages.
+	IsStreamingClient() bool
+	// IsStreamingServer reports whether the server streams multiple messages.
+	IsStreamingServer() bool
+
+	isMethodDescriptor
+}
+type isMethodDescriptor interface{ ProtoType(MethodDescriptor) }
+
+// MethodDescriptors is a list of method declarations.
+type MethodDescriptors interface {
+	// Len reports the number of methods.
+	Len() int
+	// Get returns the ith MethodDescriptor. It panics if out of bounds.
+	Get(i int) MethodDescriptor
+	// ByName returns the MethodDescriptor for a service method named s.
+	// It returns nil if not found.
+	ByName(s Name) MethodDescriptor
+
+	doNotImplement
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/value.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/value.go
new file mode 100644
index 0000000..f319810
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/value.go
@@ -0,0 +1,285 @@
+// 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 protoreflect
+
+import "google.golang.org/protobuf/encoding/protowire"
+
+// Enum is a reflection interface for a concrete enum value,
+// which provides type information and a getter for the enum number.
+// Enum does not provide a mutable API since enums are commonly backed by
+// Go constants, which are not addressable.
+type Enum interface {
+	// Descriptor returns enum descriptor, which contains only the protobuf
+	// type information for the enum.
+	Descriptor() EnumDescriptor
+
+	// Type returns the enum type, which encapsulates both Go and protobuf
+	// type information. If the Go type information is not needed,
+	// it is recommended that the enum descriptor be used instead.
+	Type() EnumType
+
+	// Number returns the enum value as an integer.
+	Number() EnumNumber
+}
+
+// Message is a reflective interface for a concrete message value,
+// encapsulating both type and value information for the message.
+//
+// Accessor/mutators for individual fields are keyed by FieldDescriptor.
+// For non-extension fields, the descriptor must exactly match the
+// field known by the parent message.
+// For extension fields, the descriptor must implement ExtensionTypeDescriptor,
+// extend the parent message (i.e., have the same message FullName), and
+// be within the parent's extension range.
+//
+// Each field Value can be a scalar or a composite type (Message, List, or Map).
+// See Value for the Go types associated with a FieldDescriptor.
+// Providing a Value that is invalid or of an incorrect type panics.
+type Message interface {
+	// Descriptor returns message descriptor, which contains only the protobuf
+	// type information for the message.
+	Descriptor() MessageDescriptor
+
+	// Type returns the message type, which encapsulates both Go and protobuf
+	// type information. If the Go type information is not needed,
+	// it is recommended that the message descriptor be used instead.
+	Type() MessageType
+
+	// New returns a newly allocated and mutable empty message.
+	New() Message
+
+	// Interface unwraps the message reflection interface and
+	// returns the underlying ProtoMessage interface.
+	Interface() ProtoMessage
+
+	// Range iterates over every populated field in an undefined order,
+	// calling f for each field descriptor and value encountered.
+	// Range returns immediately if f returns false.
+	// While iterating, mutating operations may only be performed
+	// on the current field descriptor.
+	Range(f func(FieldDescriptor, Value) bool)
+
+	// Has reports whether a field is populated.
+	//
+	// Some fields have the property of nullability where it is possible to
+	// distinguish between the default value of a field and whether the field
+	// was explicitly populated with the default value. Singular message fields,
+	// member fields of a oneof, and proto2 scalar fields are nullable. Such
+	// fields are populated only if explicitly set.
+	//
+	// In other cases (aside from the nullable cases above),
+	// a proto3 scalar field is populated if it contains a non-zero value, and
+	// a repeated field is populated if it is non-empty.
+	Has(FieldDescriptor) bool
+
+	// Clear clears the field such that a subsequent Has call reports false.
+	//
+	// Clearing an extension field clears both the extension type and value
+	// associated with the given field number.
+	//
+	// Clear is a mutating operation and unsafe for concurrent use.
+	Clear(FieldDescriptor)
+
+	// Get retrieves the value for a field.
+	//
+	// For unpopulated scalars, it returns the default value, where
+	// the default value of a bytes scalar is guaranteed to be a copy.
+	// For unpopulated composite types, it returns an empty, read-only view
+	// of the value; to obtain a mutable reference, use Mutable.
+	Get(FieldDescriptor) Value
+
+	// Set stores the value for a field.
+	//
+	// For a field belonging to a oneof, it implicitly clears any other field
+	// that may be currently set within the same oneof.
+	// For extension fields, it implicitly stores the provided ExtensionType.
+	// When setting a composite type, it is unspecified whether the stored value
+	// aliases the source's memory in any way. If the composite value is an
+	// empty, read-only value, then it panics.
+	//
+	// Set is a mutating operation and unsafe for concurrent use.
+	Set(FieldDescriptor, Value)
+
+	// Mutable returns a mutable reference to a composite type.
+	//
+	// If the field is unpopulated, it may allocate a composite value.
+	// For a field belonging to a oneof, it implicitly clears any other field
+	// that may be currently set within the same oneof.
+	// For extension fields, it implicitly stores the provided ExtensionType
+	// if not already stored.
+	// It panics if the field does not contain a composite type.
+	//
+	// Mutable is a mutating operation and unsafe for concurrent use.
+	Mutable(FieldDescriptor) Value
+
+	// NewField returns a new value that is assignable to the field
+	// for the given descriptor. For scalars, this returns the default value.
+	// For lists, maps, and messages, this returns a new, empty, mutable value.
+	NewField(FieldDescriptor) Value
+
+	// WhichOneof reports which field within the oneof is populated,
+	// returning nil if none are populated.
+	// It panics if the oneof descriptor does not belong to this message.
+	WhichOneof(OneofDescriptor) FieldDescriptor
+
+	// GetUnknown retrieves the entire list of unknown fields.
+	// The caller may only mutate the contents of the RawFields
+	// if the mutated bytes are stored back into the message with SetUnknown.
+	GetUnknown() RawFields
+
+	// SetUnknown stores an entire list of unknown fields.
+	// The raw fields must be syntactically valid according to the wire format.
+	// An implementation may panic if this is not the case.
+	// Once stored, the caller must not mutate the content of the RawFields.
+	// An empty RawFields may be passed to clear the fields.
+	//
+	// SetUnknown is a mutating operation and unsafe for concurrent use.
+	SetUnknown(RawFields)
+
+	// IsValid reports whether the message is valid.
+	//
+	// An invalid message is an empty, read-only value.
+	//
+	// An invalid message often corresponds to a nil pointer of the concrete
+	// message type, but the details are implementation dependent.
+	// Validity is not part of the protobuf data model, and may not
+	// be preserved in marshaling or other operations.
+	IsValid() bool
+
+	// ProtoMethods returns optional fast-path implementions of various operations.
+	// This method may return nil.
+	//
+	// The returned methods type is identical to
+	// "google.golang.org/protobuf/runtime/protoiface".Methods.
+	// Consult the protoiface package documentation for details.
+	ProtoMethods() *methods
+}
+
+// RawFields is the raw bytes for an ordered sequence of fields.
+// Each field contains both the tag (representing field number and wire type),
+// and also the wire data itself.
+type RawFields []byte
+
+// IsValid reports whether b is syntactically correct wire format.
+func (b RawFields) IsValid() bool {
+	for len(b) > 0 {
+		_, _, n := protowire.ConsumeField(b)
+		if n < 0 {
+			return false
+		}
+		b = b[n:]
+	}
+	return true
+}
+
+// List is a zero-indexed, ordered list.
+// The element Value type is determined by FieldDescriptor.Kind.
+// Providing a Value that is invalid or of an incorrect type panics.
+type List interface {
+	// Len reports the number of entries in the List.
+	// Get, Set, and Truncate panic with out of bound indexes.
+	Len() int
+
+	// Get retrieves the value at the given index.
+	// It never returns an invalid value.
+	Get(int) Value
+
+	// Set stores a value for the given index.
+	// When setting a composite type, it is unspecified whether the set
+	// value aliases the source's memory in any way.
+	//
+	// Set is a mutating operation and unsafe for concurrent use.
+	Set(int, Value)
+
+	// Append appends the provided value to the end of the list.
+	// When appending a composite type, it is unspecified whether the appended
+	// value aliases the source's memory in any way.
+	//
+	// Append is a mutating operation and unsafe for concurrent use.
+	Append(Value)
+
+	// AppendMutable appends a new, empty, mutable message value to the end
+	// of the list and returns it.
+	// It panics if the list does not contain a message type.
+	AppendMutable() Value
+
+	// Truncate truncates the list to a smaller length.
+	//
+	// Truncate is a mutating operation and unsafe for concurrent use.
+	Truncate(int)
+
+	// NewElement returns a new value for a list element.
+	// For enums, this returns the first enum value.
+	// For other scalars, this returns the zero value.
+	// For messages, this returns a new, empty, mutable value.
+	NewElement() Value
+
+	// IsValid reports whether the list is valid.
+	//
+	// An invalid list is an empty, read-only value.
+	//
+	// Validity is not part of the protobuf data model, and may not
+	// be preserved in marshaling or other operations.
+	IsValid() bool
+}
+
+// Map is an unordered, associative map.
+// The entry MapKey type is determined by FieldDescriptor.MapKey.Kind.
+// The entry Value type is determined by FieldDescriptor.MapValue.Kind.
+// Providing a MapKey or Value that is invalid or of an incorrect type panics.
+type Map interface {
+	// Len reports the number of elements in the map.
+	Len() int
+
+	// Range iterates over every map entry in an undefined order,
+	// calling f for each key and value encountered.
+	// Range calls f Len times unless f returns false, which stops iteration.
+	// While iterating, mutating operations may only be performed
+	// on the current map key.
+	Range(f func(MapKey, Value) bool)
+
+	// Has reports whether an entry with the given key is in the map.
+	Has(MapKey) bool
+
+	// Clear clears the entry associated with they given key.
+	// The operation does nothing if there is no entry associated with the key.
+	//
+	// Clear is a mutating operation and unsafe for concurrent use.
+	Clear(MapKey)
+
+	// Get retrieves the value for an entry with the given key.
+	// It returns an invalid value for non-existent entries.
+	Get(MapKey) Value
+
+	// Set stores the value for an entry with the given key.
+	// It panics when given a key or value that is invalid or the wrong type.
+	// When setting a composite type, it is unspecified whether the set
+	// value aliases the source's memory in any way.
+	//
+	// Set is a mutating operation and unsafe for concurrent use.
+	Set(MapKey, Value)
+
+	// Mutable retrieves a mutable reference to the entry for the given key.
+	// If no entry exists for the key, it creates a new, empty, mutable value
+	// and stores it as the entry for the key.
+	// It panics if the map value is not a message.
+	Mutable(MapKey) Value
+
+	// NewValue returns a new value assignable as a map value.
+	// For enums, this returns the first enum value.
+	// For other scalars, this returns the zero value.
+	// For messages, this returns a new, empty, mutable value.
+	NewValue() Value
+
+	// IsValid reports whether the map is valid.
+	//
+	// An invalid map is an empty, read-only value.
+	//
+	// An invalid message often corresponds to a nil Go map value,
+	// but the details are implementation dependent.
+	// Validity is not part of the protobuf data model, and may not
+	// be preserved in marshaling or other operations.
+	IsValid() bool
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/value_pure.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_pure.go
new file mode 100644
index 0000000..918e685
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_pure.go
@@ -0,0 +1,59 @@
+// 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 purego appengine
+
+package protoreflect
+
+import "google.golang.org/protobuf/internal/pragma"
+
+type valueType int
+
+const (
+	nilType valueType = iota
+	boolType
+	int32Type
+	int64Type
+	uint32Type
+	uint64Type
+	float32Type
+	float64Type
+	stringType
+	bytesType
+	enumType
+	ifaceType
+)
+
+// value is a union where only one type can be represented at a time.
+// This uses a distinct field for each type. This is type safe in Go, but
+// occupies more memory than necessary (72B).
+type value struct {
+	pragma.DoNotCompare // 0B
+
+	typ   valueType   // 8B
+	num   uint64      // 8B
+	str   string      // 16B
+	bin   []byte      // 24B
+	iface interface{} // 16B
+}
+
+func valueOfString(v string) Value {
+	return Value{typ: stringType, str: v}
+}
+func valueOfBytes(v []byte) Value {
+	return Value{typ: bytesType, bin: v}
+}
+func valueOfIface(v interface{}) Value {
+	return Value{typ: ifaceType, iface: v}
+}
+
+func (v Value) getString() string {
+	return v.str
+}
+func (v Value) getBytes() []byte {
+	return v.bin
+}
+func (v Value) getIface() interface{} {
+	return v.iface
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/value_union.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_union.go
new file mode 100644
index 0000000..5a34147
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_union.go
@@ -0,0 +1,411 @@
+// 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 protoreflect
+
+import (
+	"fmt"
+	"math"
+)
+
+// Value is a union where only one Go type may be set at a time.
+// The Value is used to represent all possible values a field may take.
+// The following shows which Go type is used to represent each proto Kind:
+//
+//	╔════════════╤═════════════════════════════════════╗
+//	║ Go type    │ Protobuf kind                       ║
+//	╠════════════╪═════════════════════════════════════╣
+//	║ bool       │ BoolKind                            ║
+//	║ int32      │ Int32Kind, Sint32Kind, Sfixed32Kind ║
+//	║ int64      │ Int64Kind, Sint64Kind, Sfixed64Kind ║
+//	║ uint32     │ Uint32Kind, Fixed32Kind             ║
+//	║ uint64     │ Uint64Kind, Fixed64Kind             ║
+//	║ float32    │ FloatKind                           ║
+//	║ float64    │ DoubleKind                          ║
+//	║ string     │ StringKind                          ║
+//	║ []byte     │ BytesKind                           ║
+//	║ EnumNumber │ EnumKind                            ║
+//	║ Message    │ MessageKind, GroupKind              ║
+//	╚════════════╧═════════════════════════════════════╝
+//
+// Multiple protobuf Kinds may be represented by a single Go type if the type
+// can losslessly represent the information for the proto kind. For example,
+// Int64Kind, Sint64Kind, and Sfixed64Kind are all represented by int64,
+// but use different integer encoding methods.
+//
+// The List or Map types are used if the field cardinality is repeated.
+// A field is a List if FieldDescriptor.IsList reports true.
+// A field is a Map if FieldDescriptor.IsMap reports true.
+//
+// Converting to/from a Value and a concrete Go value panics on type mismatch.
+// For example, ValueOf("hello").Int() panics because this attempts to
+// retrieve an int64 from a string.
+type Value value
+
+// The protoreflect API uses a custom Value union type instead of interface{}
+// to keep the future open for performance optimizations. Using an interface{}
+// always incurs an allocation for primitives (e.g., int64) since it needs to
+// be boxed on the heap (as interfaces can only contain pointers natively).
+// Instead, we represent the Value union as a flat struct that internally keeps
+// track of which type is set. Using unsafe, the Value union can be reduced
+// down to 24B, which is identical in size to a slice.
+//
+// The latest compiler (Go1.11) currently suffers from some limitations:
+//	• With inlining, the compiler should be able to statically prove that
+//	only one of these switch cases are taken and inline one specific case.
+//	See https://golang.org/issue/22310.
+
+// ValueOf returns a Value initialized with the concrete value stored in v.
+// This panics if the type does not match one of the allowed types in the
+// Value union.
+func ValueOf(v interface{}) Value {
+	switch v := v.(type) {
+	case nil:
+		return Value{}
+	case bool:
+		return ValueOfBool(v)
+	case int32:
+		return ValueOfInt32(v)
+	case int64:
+		return ValueOfInt64(v)
+	case uint32:
+		return ValueOfUint32(v)
+	case uint64:
+		return ValueOfUint64(v)
+	case float32:
+		return ValueOfFloat32(v)
+	case float64:
+		return ValueOfFloat64(v)
+	case string:
+		return ValueOfString(v)
+	case []byte:
+		return ValueOfBytes(v)
+	case EnumNumber:
+		return ValueOfEnum(v)
+	case Message, List, Map:
+		return valueOfIface(v)
+	case ProtoMessage:
+		panic(fmt.Sprintf("invalid proto.Message(%T) type, expected a protoreflect.Message type", v))
+	default:
+		panic(fmt.Sprintf("invalid type: %T", v))
+	}
+}
+
+// ValueOfBool returns a new boolean value.
+func ValueOfBool(v bool) Value {
+	if v {
+		return Value{typ: boolType, num: 1}
+	} else {
+		return Value{typ: boolType, num: 0}
+	}
+}
+
+// ValueOfInt32 returns a new int32 value.
+func ValueOfInt32(v int32) Value {
+	return Value{typ: int32Type, num: uint64(v)}
+}
+
+// ValueOfInt64 returns a new int64 value.
+func ValueOfInt64(v int64) Value {
+	return Value{typ: int64Type, num: uint64(v)}
+}
+
+// ValueOfUint32 returns a new uint32 value.
+func ValueOfUint32(v uint32) Value {
+	return Value{typ: uint32Type, num: uint64(v)}
+}
+
+// ValueOfUint64 returns a new uint64 value.
+func ValueOfUint64(v uint64) Value {
+	return Value{typ: uint64Type, num: v}
+}
+
+// ValueOfFloat32 returns a new float32 value.
+func ValueOfFloat32(v float32) Value {
+	return Value{typ: float32Type, num: uint64(math.Float64bits(float64(v)))}
+}
+
+// ValueOfFloat64 returns a new float64 value.
+func ValueOfFloat64(v float64) Value {
+	return Value{typ: float64Type, num: uint64(math.Float64bits(float64(v)))}
+}
+
+// ValueOfString returns a new string value.
+func ValueOfString(v string) Value {
+	return valueOfString(v)
+}
+
+// ValueOfBytes returns a new bytes value.
+func ValueOfBytes(v []byte) Value {
+	return valueOfBytes(v[:len(v):len(v)])
+}
+
+// ValueOfEnum returns a new enum value.
+func ValueOfEnum(v EnumNumber) Value {
+	return Value{typ: enumType, num: uint64(v)}
+}
+
+// ValueOfMessage returns a new Message value.
+func ValueOfMessage(v Message) Value {
+	return valueOfIface(v)
+}
+
+// ValueOfList returns a new List value.
+func ValueOfList(v List) Value {
+	return valueOfIface(v)
+}
+
+// ValueOfMap returns a new Map value.
+func ValueOfMap(v Map) Value {
+	return valueOfIface(v)
+}
+
+// IsValid reports whether v is populated with a value.
+func (v Value) IsValid() bool {
+	return v.typ != nilType
+}
+
+// Interface returns v as an interface{}.
+//
+// Invariant: v == ValueOf(v).Interface()
+func (v Value) Interface() interface{} {
+	switch v.typ {
+	case nilType:
+		return nil
+	case boolType:
+		return v.Bool()
+	case int32Type:
+		return int32(v.Int())
+	case int64Type:
+		return int64(v.Int())
+	case uint32Type:
+		return uint32(v.Uint())
+	case uint64Type:
+		return uint64(v.Uint())
+	case float32Type:
+		return float32(v.Float())
+	case float64Type:
+		return float64(v.Float())
+	case stringType:
+		return v.String()
+	case bytesType:
+		return v.Bytes()
+	case enumType:
+		return v.Enum()
+	default:
+		return v.getIface()
+	}
+}
+
+func (v Value) typeName() string {
+	switch v.typ {
+	case nilType:
+		return "nil"
+	case boolType:
+		return "bool"
+	case int32Type:
+		return "int32"
+	case int64Type:
+		return "int64"
+	case uint32Type:
+		return "uint32"
+	case uint64Type:
+		return "uint64"
+	case float32Type:
+		return "float32"
+	case float64Type:
+		return "float64"
+	case stringType:
+		return "string"
+	case bytesType:
+		return "bytes"
+	case enumType:
+		return "enum"
+	default:
+		switch v := v.getIface().(type) {
+		case Message:
+			return "message"
+		case List:
+			return "list"
+		case Map:
+			return "map"
+		default:
+			return fmt.Sprintf("<unknown: %T>", v)
+		}
+	}
+}
+
+func (v Value) panicMessage(what string) string {
+	return fmt.Sprintf("type mismatch: cannot convert %v to %s", v.typeName(), what)
+}
+
+// Bool returns v as a bool and panics if the type is not a bool.
+func (v Value) Bool() bool {
+	switch v.typ {
+	case boolType:
+		return v.num > 0
+	default:
+		panic(v.panicMessage("bool"))
+	}
+}
+
+// Int returns v as a int64 and panics if the type is not a int32 or int64.
+func (v Value) Int() int64 {
+	switch v.typ {
+	case int32Type, int64Type:
+		return int64(v.num)
+	default:
+		panic(v.panicMessage("int"))
+	}
+}
+
+// Uint returns v as a uint64 and panics if the type is not a uint32 or uint64.
+func (v Value) Uint() uint64 {
+	switch v.typ {
+	case uint32Type, uint64Type:
+		return uint64(v.num)
+	default:
+		panic(v.panicMessage("uint"))
+	}
+}
+
+// Float returns v as a float64 and panics if the type is not a float32 or float64.
+func (v Value) Float() float64 {
+	switch v.typ {
+	case float32Type, float64Type:
+		return math.Float64frombits(uint64(v.num))
+	default:
+		panic(v.panicMessage("float"))
+	}
+}
+
+// String returns v as a string. Since this method implements fmt.Stringer,
+// this returns the formatted string value for any non-string type.
+func (v Value) String() string {
+	switch v.typ {
+	case stringType:
+		return v.getString()
+	default:
+		return fmt.Sprint(v.Interface())
+	}
+}
+
+// Bytes returns v as a []byte and panics if the type is not a []byte.
+func (v Value) Bytes() []byte {
+	switch v.typ {
+	case bytesType:
+		return v.getBytes()
+	default:
+		panic(v.panicMessage("bytes"))
+	}
+}
+
+// Enum returns v as a EnumNumber and panics if the type is not a EnumNumber.
+func (v Value) Enum() EnumNumber {
+	switch v.typ {
+	case enumType:
+		return EnumNumber(v.num)
+	default:
+		panic(v.panicMessage("enum"))
+	}
+}
+
+// Message returns v as a Message and panics if the type is not a Message.
+func (v Value) Message() Message {
+	switch vi := v.getIface().(type) {
+	case Message:
+		return vi
+	default:
+		panic(v.panicMessage("message"))
+	}
+}
+
+// List returns v as a List and panics if the type is not a List.
+func (v Value) List() List {
+	switch vi := v.getIface().(type) {
+	case List:
+		return vi
+	default:
+		panic(v.panicMessage("list"))
+	}
+}
+
+// Map returns v as a Map and panics if the type is not a Map.
+func (v Value) Map() Map {
+	switch vi := v.getIface().(type) {
+	case Map:
+		return vi
+	default:
+		panic(v.panicMessage("map"))
+	}
+}
+
+// MapKey returns v as a MapKey and panics for invalid MapKey types.
+func (v Value) MapKey() MapKey {
+	switch v.typ {
+	case boolType, int32Type, int64Type, uint32Type, uint64Type, stringType:
+		return MapKey(v)
+	default:
+		panic(v.panicMessage("map key"))
+	}
+}
+
+// MapKey is used to index maps, where the Go type of the MapKey must match
+// the specified key Kind (see MessageDescriptor.IsMapEntry).
+// The following shows what Go type is used to represent each proto Kind:
+//
+//	╔═════════╤═════════════════════════════════════╗
+//	║ Go type │ Protobuf kind                       ║
+//	╠═════════╪═════════════════════════════════════╣
+//	║ bool    │ BoolKind                            ║
+//	║ int32   │ Int32Kind, Sint32Kind, Sfixed32Kind ║
+//	║ int64   │ Int64Kind, Sint64Kind, Sfixed64Kind ║
+//	║ uint32  │ Uint32Kind, Fixed32Kind             ║
+//	║ uint64  │ Uint64Kind, Fixed64Kind             ║
+//	║ string  │ StringKind                          ║
+//	╚═════════╧═════════════════════════════════════╝
+//
+// A MapKey is constructed and accessed through a Value:
+//	k := ValueOf("hash").MapKey() // convert string to MapKey
+//	s := k.String()               // convert MapKey to string
+//
+// The MapKey is a strict subset of valid types used in Value;
+// converting a Value to a MapKey with an invalid type panics.
+type MapKey value
+
+// IsValid reports whether k is populated with a value.
+func (k MapKey) IsValid() bool {
+	return Value(k).IsValid()
+}
+
+// Interface returns k as an interface{}.
+func (k MapKey) Interface() interface{} {
+	return Value(k).Interface()
+}
+
+// Bool returns k as a bool and panics if the type is not a bool.
+func (k MapKey) Bool() bool {
+	return Value(k).Bool()
+}
+
+// Int returns k as a int64 and panics if the type is not a int32 or int64.
+func (k MapKey) Int() int64 {
+	return Value(k).Int()
+}
+
+// Uint returns k as a uint64 and panics if the type is not a uint32 or uint64.
+func (k MapKey) Uint() uint64 {
+	return Value(k).Uint()
+}
+
+// String returns k as a string. Since this method implements fmt.Stringer,
+// this returns the formatted string value for any non-string type.
+func (k MapKey) String() string {
+	return Value(k).String()
+}
+
+// Value returns k as a Value.
+func (k MapKey) Value() Value {
+	return Value(k)
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoreflect/value_unsafe.go b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_unsafe.go
new file mode 100644
index 0000000..c45debd
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoreflect/value_unsafe.go
@@ -0,0 +1,98 @@
+// 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 !purego,!appengine
+
+package protoreflect
+
+import (
+	"unsafe"
+
+	"google.golang.org/protobuf/internal/pragma"
+)
+
+type (
+	stringHeader struct {
+		Data unsafe.Pointer
+		Len  int
+	}
+	sliceHeader struct {
+		Data unsafe.Pointer
+		Len  int
+		Cap  int
+	}
+	ifaceHeader struct {
+		Type unsafe.Pointer
+		Data unsafe.Pointer
+	}
+)
+
+var (
+	nilType     = typeOf(nil)
+	boolType    = typeOf(*new(bool))
+	int32Type   = typeOf(*new(int32))
+	int64Type   = typeOf(*new(int64))
+	uint32Type  = typeOf(*new(uint32))
+	uint64Type  = typeOf(*new(uint64))
+	float32Type = typeOf(*new(float32))
+	float64Type = typeOf(*new(float64))
+	stringType  = typeOf(*new(string))
+	bytesType   = typeOf(*new([]byte))
+	enumType    = typeOf(*new(EnumNumber))
+)
+
+// typeOf returns a pointer to the Go type information.
+// The pointer is comparable and equal if and only if the types are identical.
+func typeOf(t interface{}) unsafe.Pointer {
+	return (*ifaceHeader)(unsafe.Pointer(&t)).Type
+}
+
+// value is a union where only one type can be represented at a time.
+// The struct is 24B large on 64-bit systems and requires the minimum storage
+// necessary to represent each possible type.
+//
+// The Go GC needs to be able to scan variables containing pointers.
+// As such, pointers and non-pointers cannot be intermixed.
+type value struct {
+	pragma.DoNotCompare // 0B
+
+	// typ stores the type of the value as a pointer to the Go type.
+	typ unsafe.Pointer // 8B
+
+	// ptr stores the data pointer for a String, Bytes, or interface value.
+	ptr unsafe.Pointer // 8B
+
+	// num stores a Bool, Int32, Int64, Uint32, Uint64, Float32, Float64, or
+	// Enum value as a raw uint64.
+	//
+	// It is also used to store the length of a String or Bytes value;
+	// the capacity is ignored.
+	num uint64 // 8B
+}
+
+func valueOfString(v string) Value {
+	p := (*stringHeader)(unsafe.Pointer(&v))
+	return Value{typ: stringType, ptr: p.Data, num: uint64(len(v))}
+}
+func valueOfBytes(v []byte) Value {
+	p := (*sliceHeader)(unsafe.Pointer(&v))
+	return Value{typ: bytesType, ptr: p.Data, num: uint64(len(v))}
+}
+func valueOfIface(v interface{}) Value {
+	p := (*ifaceHeader)(unsafe.Pointer(&v))
+	return Value{typ: p.Type, ptr: p.Data}
+}
+
+func (v Value) getString() (x string) {
+	*(*stringHeader)(unsafe.Pointer(&x)) = stringHeader{Data: v.ptr, Len: int(v.num)}
+	return x
+}
+func (v Value) getBytes() (x []byte) {
+	*(*sliceHeader)(unsafe.Pointer(&x)) = sliceHeader{Data: v.ptr, Len: int(v.num), Cap: int(v.num)}
+	return x
+}
+func (v Value) getIface() (x interface{}) {
+	*(*ifaceHeader)(unsafe.Pointer(&x)) = ifaceHeader{Type: v.typ, Data: v.ptr}
+	return x
+}
diff --git a/vendor/google.golang.org/protobuf/reflect/protoregistry/registry.go b/vendor/google.golang.org/protobuf/reflect/protoregistry/registry.go
new file mode 100644
index 0000000..59f024c
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/reflect/protoregistry/registry.go
@@ -0,0 +1,880 @@
+// 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 protoregistry provides data structures to register and lookup
+// protobuf descriptor types.
+//
+// The Files registry contains file descriptors and provides the ability
+// to iterate over the files or lookup a specific descriptor within the files.
+// Files only contains protobuf descriptors and has no understanding of Go
+// type information that may be associated with each descriptor.
+//
+// The Types registry contains descriptor types for which there is a known
+// Go type associated with that descriptor. It provides the ability to iterate
+// over the registered types or lookup a type by name.
+package protoregistry
+
+import (
+	"fmt"
+	"os"
+	"strings"
+	"sync"
+
+	"google.golang.org/protobuf/internal/encoding/messageset"
+	"google.golang.org/protobuf/internal/errors"
+	"google.golang.org/protobuf/internal/flags"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// conflictPolicy configures the policy for handling registration conflicts.
+//
+// It can be over-written at compile time with a linker-initialized variable:
+//	go build -ldflags "-X google.golang.org/protobuf/reflect/protoregistry.conflictPolicy=warn"
+//
+// It can be over-written at program execution with an environment variable:
+//	GOLANG_PROTOBUF_REGISTRATION_CONFLICT=warn ./main
+//
+// Neither of the above are covered by the compatibility promise and
+// may be removed in a future release of this module.
+var conflictPolicy = "panic" // "panic" | "warn" | "ignore"
+
+// ignoreConflict reports whether to ignore a registration conflict
+// given the descriptor being registered and the error.
+// It is a variable so that the behavior is easily overridden in another file.
+var ignoreConflict = func(d protoreflect.Descriptor, err error) bool {
+	const env = "GOLANG_PROTOBUF_REGISTRATION_CONFLICT"
+	const faq = "https://developers.google.com/protocol-buffers/docs/reference/go/faq#namespace-conflict"
+	policy := conflictPolicy
+	if v := os.Getenv(env); v != "" {
+		policy = v
+	}
+	switch policy {
+	case "panic":
+		panic(fmt.Sprintf("%v\nSee %v\n", err, faq))
+	case "warn":
+		fmt.Fprintf(os.Stderr, "WARNING: %v\nSee %v\n\n", err, faq)
+		return true
+	case "ignore":
+		return true
+	default:
+		panic("invalid " + env + " value: " + os.Getenv(env))
+	}
+}
+
+var globalMutex sync.RWMutex
+
+// GlobalFiles is a global registry of file descriptors.
+var GlobalFiles *Files = new(Files)
+
+// GlobalTypes is the registry used by default for type lookups
+// unless a local registry is provided by the user.
+var GlobalTypes *Types = new(Types)
+
+// NotFound is a sentinel error value to indicate that the type was not found.
+//
+// Since registry lookup can happen in the critical performance path, resolvers
+// must return this exact error value, not an error wrapping it.
+var NotFound = errors.New("not found")
+
+// Files is a registry for looking up or iterating over files and the
+// descriptors contained within them.
+// The Find and Range methods are safe for concurrent use.
+type Files struct {
+	// The map of descsByName contains:
+	//	EnumDescriptor
+	//	EnumValueDescriptor
+	//	MessageDescriptor
+	//	ExtensionDescriptor
+	//	ServiceDescriptor
+	//	*packageDescriptor
+	//
+	// Note that files are stored as a slice, since a package may contain
+	// multiple files. Only top-level declarations are registered.
+	// Note that enum values are in the top-level since that are in the same
+	// scope as the parent enum.
+	descsByName map[protoreflect.FullName]interface{}
+	filesByPath map[string][]protoreflect.FileDescriptor
+	numFiles    int
+}
+
+type packageDescriptor struct {
+	files []protoreflect.FileDescriptor
+}
+
+// RegisterFile registers the provided file descriptor.
+//
+// If any descriptor within the file conflicts with the descriptor of any
+// previously registered file (e.g., two enums with the same full name),
+// then the file is not registered and an error is returned.
+//
+// It is permitted for multiple files to have the same file path.
+func (r *Files) RegisterFile(file protoreflect.FileDescriptor) error {
+	if r == GlobalFiles {
+		globalMutex.Lock()
+		defer globalMutex.Unlock()
+	}
+	if r.descsByName == nil {
+		r.descsByName = map[protoreflect.FullName]interface{}{
+			"": &packageDescriptor{},
+		}
+		r.filesByPath = make(map[string][]protoreflect.FileDescriptor)
+	}
+	path := file.Path()
+	if prev := r.filesByPath[path]; len(prev) > 0 {
+		r.checkGenProtoConflict(path)
+		err := errors.New("file %q is already registered", file.Path())
+		err = amendErrorWithCaller(err, prev[0], file)
+		if !(r == GlobalFiles && ignoreConflict(file, err)) {
+			return err
+		}
+	}
+
+	for name := file.Package(); name != ""; name = name.Parent() {
+		switch prev := r.descsByName[name]; prev.(type) {
+		case nil, *packageDescriptor:
+		default:
+			err := errors.New("file %q has a package name conflict over %v", file.Path(), name)
+			err = amendErrorWithCaller(err, prev, file)
+			if r == GlobalFiles && ignoreConflict(file, err) {
+				err = nil
+			}
+			return err
+		}
+	}
+	var err error
+	var hasConflict bool
+	rangeTopLevelDescriptors(file, func(d protoreflect.Descriptor) {
+		if prev := r.descsByName[d.FullName()]; prev != nil {
+			hasConflict = true
+			err = errors.New("file %q has a name conflict over %v", file.Path(), d.FullName())
+			err = amendErrorWithCaller(err, prev, file)
+			if r == GlobalFiles && ignoreConflict(d, err) {
+				err = nil
+			}
+		}
+	})
+	if hasConflict {
+		return err
+	}
+
+	for name := file.Package(); name != ""; name = name.Parent() {
+		if r.descsByName[name] == nil {
+			r.descsByName[name] = &packageDescriptor{}
+		}
+	}
+	p := r.descsByName[file.Package()].(*packageDescriptor)
+	p.files = append(p.files, file)
+	rangeTopLevelDescriptors(file, func(d protoreflect.Descriptor) {
+		r.descsByName[d.FullName()] = d
+	})
+	r.filesByPath[path] = append(r.filesByPath[path], file)
+	r.numFiles++
+	return nil
+}
+
+// Several well-known types were hosted in the google.golang.org/genproto module
+// but were later moved to this module. To avoid a weak dependency on the
+// genproto module (and its relatively large set of transitive dependencies),
+// we rely on a registration conflict to determine whether the genproto version
+// is too old (i.e., does not contain aliases to the new type declarations).
+func (r *Files) checkGenProtoConflict(path string) {
+	if r != GlobalFiles {
+		return
+	}
+	var prevPath string
+	const prevModule = "google.golang.org/genproto"
+	const prevVersion = "cb27e3aa (May 26th, 2020)"
+	switch path {
+	case "google/protobuf/field_mask.proto":
+		prevPath = prevModule + "/protobuf/field_mask"
+	case "google/protobuf/api.proto":
+		prevPath = prevModule + "/protobuf/api"
+	case "google/protobuf/type.proto":
+		prevPath = prevModule + "/protobuf/ptype"
+	case "google/protobuf/source_context.proto":
+		prevPath = prevModule + "/protobuf/source_context"
+	default:
+		return
+	}
+	pkgName := strings.TrimSuffix(strings.TrimPrefix(path, "google/protobuf/"), ".proto")
+	pkgName = strings.Replace(pkgName, "_", "", -1) + "pb" // e.g., "field_mask" => "fieldmaskpb"
+	currPath := "google.golang.org/protobuf/types/known/" + pkgName
+	panic(fmt.Sprintf(""+
+		"duplicate registration of %q\n"+
+		"\n"+
+		"The generated definition for this file has moved:\n"+
+		"\tfrom: %q\n"+
+		"\tto:   %q\n"+
+		"A dependency on the %q module must\n"+
+		"be at version %v or higher.\n"+
+		"\n"+
+		"Upgrade the dependency by running:\n"+
+		"\tgo get -u %v\n",
+		path, prevPath, currPath, prevModule, prevVersion, prevPath))
+}
+
+// FindDescriptorByName looks up a descriptor by the full name.
+//
+// This returns (nil, NotFound) if not found.
+func (r *Files) FindDescriptorByName(name protoreflect.FullName) (protoreflect.Descriptor, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	prefix := name
+	suffix := nameSuffix("")
+	for prefix != "" {
+		if d, ok := r.descsByName[prefix]; ok {
+			switch d := d.(type) {
+			case protoreflect.EnumDescriptor:
+				if d.FullName() == name {
+					return d, nil
+				}
+			case protoreflect.EnumValueDescriptor:
+				if d.FullName() == name {
+					return d, nil
+				}
+			case protoreflect.MessageDescriptor:
+				if d.FullName() == name {
+					return d, nil
+				}
+				if d := findDescriptorInMessage(d, suffix); d != nil && d.FullName() == name {
+					return d, nil
+				}
+			case protoreflect.ExtensionDescriptor:
+				if d.FullName() == name {
+					return d, nil
+				}
+			case protoreflect.ServiceDescriptor:
+				if d.FullName() == name {
+					return d, nil
+				}
+				if d := d.Methods().ByName(suffix.Pop()); d != nil && d.FullName() == name {
+					return d, nil
+				}
+			}
+			return nil, NotFound
+		}
+		prefix = prefix.Parent()
+		suffix = nameSuffix(name[len(prefix)+len("."):])
+	}
+	return nil, NotFound
+}
+
+func findDescriptorInMessage(md protoreflect.MessageDescriptor, suffix nameSuffix) protoreflect.Descriptor {
+	name := suffix.Pop()
+	if suffix == "" {
+		if ed := md.Enums().ByName(name); ed != nil {
+			return ed
+		}
+		for i := md.Enums().Len() - 1; i >= 0; i-- {
+			if vd := md.Enums().Get(i).Values().ByName(name); vd != nil {
+				return vd
+			}
+		}
+		if xd := md.Extensions().ByName(name); xd != nil {
+			return xd
+		}
+		if fd := md.Fields().ByName(name); fd != nil {
+			return fd
+		}
+		if od := md.Oneofs().ByName(name); od != nil {
+			return od
+		}
+	}
+	if md := md.Messages().ByName(name); md != nil {
+		if suffix == "" {
+			return md
+		}
+		return findDescriptorInMessage(md, suffix)
+	}
+	return nil
+}
+
+type nameSuffix string
+
+func (s *nameSuffix) Pop() (name protoreflect.Name) {
+	if i := strings.IndexByte(string(*s), '.'); i >= 0 {
+		name, *s = protoreflect.Name((*s)[:i]), (*s)[i+1:]
+	} else {
+		name, *s = protoreflect.Name((*s)), ""
+	}
+	return name
+}
+
+// FindFileByPath looks up a file by the path.
+//
+// This returns (nil, NotFound) if not found.
+// This returns an error if multiple files have the same path.
+func (r *Files) FindFileByPath(path string) (protoreflect.FileDescriptor, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	fds := r.filesByPath[path]
+	switch len(fds) {
+	case 0:
+		return nil, NotFound
+	case 1:
+		return fds[0], nil
+	default:
+		return nil, errors.New("multiple files named %q", path)
+	}
+}
+
+// NumFiles reports the number of registered files,
+// including duplicate files with the same name.
+func (r *Files) NumFiles() int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	return r.numFiles
+}
+
+// RangeFiles iterates over all registered files while f returns true.
+// If multiple files have the same name, RangeFiles iterates over all of them.
+// The iteration order is undefined.
+func (r *Files) RangeFiles(f func(protoreflect.FileDescriptor) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	for _, files := range r.filesByPath {
+		for _, file := range files {
+			if !f(file) {
+				return
+			}
+		}
+	}
+}
+
+// NumFilesByPackage reports the number of registered files in a proto package.
+func (r *Files) NumFilesByPackage(name protoreflect.FullName) int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	p, ok := r.descsByName[name].(*packageDescriptor)
+	if !ok {
+		return 0
+	}
+	return len(p.files)
+}
+
+// RangeFilesByPackage iterates over all registered files in a given proto package
+// while f returns true. The iteration order is undefined.
+func (r *Files) RangeFilesByPackage(name protoreflect.FullName, f func(protoreflect.FileDescriptor) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalFiles {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	p, ok := r.descsByName[name].(*packageDescriptor)
+	if !ok {
+		return
+	}
+	for _, file := range p.files {
+		if !f(file) {
+			return
+		}
+	}
+}
+
+// rangeTopLevelDescriptors iterates over all top-level descriptors in a file
+// which will be directly entered into the registry.
+func rangeTopLevelDescriptors(fd protoreflect.FileDescriptor, f func(protoreflect.Descriptor)) {
+	eds := fd.Enums()
+	for i := eds.Len() - 1; i >= 0; i-- {
+		f(eds.Get(i))
+		vds := eds.Get(i).Values()
+		for i := vds.Len() - 1; i >= 0; i-- {
+			f(vds.Get(i))
+		}
+	}
+	mds := fd.Messages()
+	for i := mds.Len() - 1; i >= 0; i-- {
+		f(mds.Get(i))
+	}
+	xds := fd.Extensions()
+	for i := xds.Len() - 1; i >= 0; i-- {
+		f(xds.Get(i))
+	}
+	sds := fd.Services()
+	for i := sds.Len() - 1; i >= 0; i-- {
+		f(sds.Get(i))
+	}
+}
+
+// MessageTypeResolver is an interface for looking up messages.
+//
+// A compliant implementation must deterministically return the same type
+// if no error is encountered.
+//
+// The Types type implements this interface.
+type MessageTypeResolver interface {
+	// FindMessageByName looks up a message by its full name.
+	// E.g., "google.protobuf.Any"
+	//
+	// This return (nil, NotFound) if not found.
+	FindMessageByName(message protoreflect.FullName) (protoreflect.MessageType, error)
+
+	// FindMessageByURL looks up a message by a URL identifier.
+	// See documentation on google.protobuf.Any.type_url for the URL format.
+	//
+	// This returns (nil, NotFound) if not found.
+	FindMessageByURL(url string) (protoreflect.MessageType, error)
+}
+
+// ExtensionTypeResolver is an interface for looking up extensions.
+//
+// A compliant implementation must deterministically return the same type
+// if no error is encountered.
+//
+// The Types type implements this interface.
+type ExtensionTypeResolver interface {
+	// FindExtensionByName looks up a extension field by the field's full name.
+	// Note that this is the full name of the field as determined by
+	// where the extension is declared and is unrelated to the full name of the
+	// message being extended.
+	//
+	// This returns (nil, NotFound) if not found.
+	FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error)
+
+	// FindExtensionByNumber looks up a extension field by the field number
+	// within some parent message, identified by full name.
+	//
+	// This returns (nil, NotFound) if not found.
+	FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error)
+}
+
+var (
+	_ MessageTypeResolver   = (*Types)(nil)
+	_ ExtensionTypeResolver = (*Types)(nil)
+)
+
+// Types is a registry for looking up or iterating over descriptor types.
+// The Find and Range methods are safe for concurrent use.
+type Types struct {
+	typesByName         typesByName
+	extensionsByMessage extensionsByMessage
+
+	numEnums      int
+	numMessages   int
+	numExtensions int
+}
+
+type (
+	typesByName         map[protoreflect.FullName]interface{}
+	extensionsByMessage map[protoreflect.FullName]extensionsByNumber
+	extensionsByNumber  map[protoreflect.FieldNumber]protoreflect.ExtensionType
+)
+
+// RegisterMessage registers the provided message type.
+//
+// If a naming conflict occurs, the type is not registered and an error is returned.
+func (r *Types) RegisterMessage(mt protoreflect.MessageType) error {
+	// Under rare circumstances getting the descriptor might recursively
+	// examine the registry, so fetch it before locking.
+	md := mt.Descriptor()
+
+	if r == GlobalTypes {
+		globalMutex.Lock()
+		defer globalMutex.Unlock()
+	}
+
+	if err := r.register("message", md, mt); err != nil {
+		return err
+	}
+	r.numMessages++
+	return nil
+}
+
+// RegisterEnum registers the provided enum type.
+//
+// If a naming conflict occurs, the type is not registered and an error is returned.
+func (r *Types) RegisterEnum(et protoreflect.EnumType) error {
+	// Under rare circumstances getting the descriptor might recursively
+	// examine the registry, so fetch it before locking.
+	ed := et.Descriptor()
+
+	if r == GlobalTypes {
+		globalMutex.Lock()
+		defer globalMutex.Unlock()
+	}
+
+	if err := r.register("enum", ed, et); err != nil {
+		return err
+	}
+	r.numEnums++
+	return nil
+}
+
+// RegisterExtension registers the provided extension type.
+//
+// If a naming conflict occurs, the type is not registered and an error is returned.
+func (r *Types) RegisterExtension(xt protoreflect.ExtensionType) error {
+	// Under rare circumstances getting the descriptor might recursively
+	// examine the registry, so fetch it before locking.
+	//
+	// A known case where this can happen: Fetching the TypeDescriptor for a
+	// legacy ExtensionDesc can consult the global registry.
+	xd := xt.TypeDescriptor()
+
+	if r == GlobalTypes {
+		globalMutex.Lock()
+		defer globalMutex.Unlock()
+	}
+
+	field := xd.Number()
+	message := xd.ContainingMessage().FullName()
+	if prev := r.extensionsByMessage[message][field]; prev != nil {
+		err := errors.New("extension number %d is already registered on message %v", field, message)
+		err = amendErrorWithCaller(err, prev, xt)
+		if !(r == GlobalTypes && ignoreConflict(xd, err)) {
+			return err
+		}
+	}
+
+	if err := r.register("extension", xd, xt); err != nil {
+		return err
+	}
+	if r.extensionsByMessage == nil {
+		r.extensionsByMessage = make(extensionsByMessage)
+	}
+	if r.extensionsByMessage[message] == nil {
+		r.extensionsByMessage[message] = make(extensionsByNumber)
+	}
+	r.extensionsByMessage[message][field] = xt
+	r.numExtensions++
+	return nil
+}
+
+func (r *Types) register(kind string, desc protoreflect.Descriptor, typ interface{}) error {
+	name := desc.FullName()
+	prev := r.typesByName[name]
+	if prev != nil {
+		err := errors.New("%v %v is already registered", kind, name)
+		err = amendErrorWithCaller(err, prev, typ)
+		if !(r == GlobalTypes && ignoreConflict(desc, err)) {
+			return err
+		}
+	}
+	if r.typesByName == nil {
+		r.typesByName = make(typesByName)
+	}
+	r.typesByName[name] = typ
+	return nil
+}
+
+// FindEnumByName looks up an enum by its full name.
+// E.g., "google.protobuf.Field.Kind".
+//
+// This returns (nil, NotFound) if not found.
+func (r *Types) FindEnumByName(enum protoreflect.FullName) (protoreflect.EnumType, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	if v := r.typesByName[enum]; v != nil {
+		if et, _ := v.(protoreflect.EnumType); et != nil {
+			return et, nil
+		}
+		return nil, errors.New("found wrong type: got %v, want enum", typeName(v))
+	}
+	return nil, NotFound
+}
+
+// FindMessageByName looks up a message by its full name,
+// e.g. "google.protobuf.Any".
+//
+// This returns (nil, NotFound) if not found.
+func (r *Types) FindMessageByName(message protoreflect.FullName) (protoreflect.MessageType, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	if v := r.typesByName[message]; v != nil {
+		if mt, _ := v.(protoreflect.MessageType); mt != nil {
+			return mt, nil
+		}
+		return nil, errors.New("found wrong type: got %v, want message", typeName(v))
+	}
+	return nil, NotFound
+}
+
+// FindMessageByURL looks up a message by a URL identifier.
+// See documentation on google.protobuf.Any.type_url for the URL format.
+//
+// This returns (nil, NotFound) if not found.
+func (r *Types) FindMessageByURL(url string) (protoreflect.MessageType, error) {
+	// This function is similar to FindMessageByName but
+	// truncates anything before and including '/' in the URL.
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	message := protoreflect.FullName(url)
+	if i := strings.LastIndexByte(url, '/'); i >= 0 {
+		message = message[i+len("/"):]
+	}
+
+	if v := r.typesByName[message]; v != nil {
+		if mt, _ := v.(protoreflect.MessageType); mt != nil {
+			return mt, nil
+		}
+		return nil, errors.New("found wrong type: got %v, want message", typeName(v))
+	}
+	return nil, NotFound
+}
+
+// FindExtensionByName looks up a extension field by the field's full name.
+// Note that this is the full name of the field as determined by
+// where the extension is declared and is unrelated to the full name of the
+// message being extended.
+//
+// This returns (nil, NotFound) if not found.
+func (r *Types) FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	if v := r.typesByName[field]; v != nil {
+		if xt, _ := v.(protoreflect.ExtensionType); xt != nil {
+			return xt, nil
+		}
+
+		// MessageSet extensions are special in that the name of the extension
+		// is the name of the message type used to extend the MessageSet.
+		// This naming scheme is used by text and JSON serialization.
+		//
+		// This feature is protected by the ProtoLegacy flag since MessageSets
+		// are a proto1 feature that is long deprecated.
+		if flags.ProtoLegacy {
+			if _, ok := v.(protoreflect.MessageType); ok {
+				field := field.Append(messageset.ExtensionName)
+				if v := r.typesByName[field]; v != nil {
+					if xt, _ := v.(protoreflect.ExtensionType); xt != nil {
+						if messageset.IsMessageSetExtension(xt.TypeDescriptor()) {
+							return xt, nil
+						}
+					}
+				}
+			}
+		}
+
+		return nil, errors.New("found wrong type: got %v, want extension", typeName(v))
+	}
+	return nil, NotFound
+}
+
+// FindExtensionByNumber looks up a extension field by the field number
+// within some parent message, identified by full name.
+//
+// This returns (nil, NotFound) if not found.
+func (r *Types) FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error) {
+	if r == nil {
+		return nil, NotFound
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	if xt, ok := r.extensionsByMessage[message][field]; ok {
+		return xt, nil
+	}
+	return nil, NotFound
+}
+
+// NumEnums reports the number of registered enums.
+func (r *Types) NumEnums() int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	return r.numEnums
+}
+
+// RangeEnums iterates over all registered enums while f returns true.
+// Iteration order is undefined.
+func (r *Types) RangeEnums(f func(protoreflect.EnumType) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	for _, typ := range r.typesByName {
+		if et, ok := typ.(protoreflect.EnumType); ok {
+			if !f(et) {
+				return
+			}
+		}
+	}
+}
+
+// NumMessages reports the number of registered messages.
+func (r *Types) NumMessages() int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	return r.numMessages
+}
+
+// RangeMessages iterates over all registered messages while f returns true.
+// Iteration order is undefined.
+func (r *Types) RangeMessages(f func(protoreflect.MessageType) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	for _, typ := range r.typesByName {
+		if mt, ok := typ.(protoreflect.MessageType); ok {
+			if !f(mt) {
+				return
+			}
+		}
+	}
+}
+
+// NumExtensions reports the number of registered extensions.
+func (r *Types) NumExtensions() int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	return r.numExtensions
+}
+
+// RangeExtensions iterates over all registered extensions while f returns true.
+// Iteration order is undefined.
+func (r *Types) RangeExtensions(f func(protoreflect.ExtensionType) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	for _, typ := range r.typesByName {
+		if xt, ok := typ.(protoreflect.ExtensionType); ok {
+			if !f(xt) {
+				return
+			}
+		}
+	}
+}
+
+// NumExtensionsByMessage reports the number of registered extensions for
+// a given message type.
+func (r *Types) NumExtensionsByMessage(message protoreflect.FullName) int {
+	if r == nil {
+		return 0
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	return len(r.extensionsByMessage[message])
+}
+
+// RangeExtensionsByMessage iterates over all registered extensions filtered
+// by a given message type while f returns true. Iteration order is undefined.
+func (r *Types) RangeExtensionsByMessage(message protoreflect.FullName, f func(protoreflect.ExtensionType) bool) {
+	if r == nil {
+		return
+	}
+	if r == GlobalTypes {
+		globalMutex.RLock()
+		defer globalMutex.RUnlock()
+	}
+	for _, xt := range r.extensionsByMessage[message] {
+		if !f(xt) {
+			return
+		}
+	}
+}
+
+func typeName(t interface{}) string {
+	switch t.(type) {
+	case protoreflect.EnumType:
+		return "enum"
+	case protoreflect.MessageType:
+		return "message"
+	case protoreflect.ExtensionType:
+		return "extension"
+	default:
+		return fmt.Sprintf("%T", t)
+	}
+}
+
+func amendErrorWithCaller(err error, prev, curr interface{}) error {
+	prevPkg := goPackage(prev)
+	currPkg := goPackage(curr)
+	if prevPkg == "" || currPkg == "" || prevPkg == currPkg {
+		return err
+	}
+	return errors.New("%s\n\tpreviously from: %q\n\tcurrently from:  %q", err, prevPkg, currPkg)
+}
+
+func goPackage(v interface{}) string {
+	switch d := v.(type) {
+	case protoreflect.EnumType:
+		v = d.Descriptor()
+	case protoreflect.MessageType:
+		v = d.Descriptor()
+	case protoreflect.ExtensionType:
+		v = d.TypeDescriptor()
+	}
+	if d, ok := v.(protoreflect.Descriptor); ok {
+		v = d.ParentFile()
+	}
+	if d, ok := v.(interface{ GoPackagePath() string }); ok {
+		return d.GoPackagePath()
+	}
+	return ""
+}
diff --git a/vendor/google.golang.org/protobuf/runtime/protoiface/legacy.go b/vendor/google.golang.org/protobuf/runtime/protoiface/legacy.go
new file mode 100644
index 0000000..c587276
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/runtime/protoiface/legacy.go
@@ -0,0 +1,15 @@
+// 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 protoiface
+
+type MessageV1 interface {
+	Reset()
+	String() string
+	ProtoMessage()
+}
+
+type ExtensionRangeV1 struct {
+	Start, End int32 // both inclusive
+}
diff --git a/vendor/google.golang.org/protobuf/runtime/protoiface/methods.go b/vendor/google.golang.org/protobuf/runtime/protoiface/methods.go
new file mode 100644
index 0000000..32c04f6
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/runtime/protoiface/methods.go
@@ -0,0 +1,167 @@
+// 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.
+
+// Package protoiface contains types referenced or implemented by messages.
+//
+// WARNING: This package should only be imported by message implementations.
+// The functionality found in this package should be accessed through
+// higher-level abstractions provided by the proto package.
+package protoiface
+
+import (
+	"google.golang.org/protobuf/internal/pragma"
+	"google.golang.org/protobuf/reflect/protoreflect"
+)
+
+// Methods is a set of optional fast-path implementations of various operations.
+type Methods = struct {
+	pragma.NoUnkeyedLiterals
+
+	// Flags indicate support for optional features.
+	Flags SupportFlags
+
+	// Size returns the size in bytes of the wire-format encoding of a message.
+	// Marshal must be provided if a custom Size is provided.
+	Size func(SizeInput) SizeOutput
+
+	// Marshal formats a message in the wire-format encoding to the provided buffer.
+	// Size should be provided if a custom Marshal is provided.
+	// It must not return an error for a partial message.
+	Marshal func(MarshalInput) (MarshalOutput, error)
+
+	// Unmarshal parses the wire-format encoding and merges the result into a message.
+	// It must not reset the target message or return an error for a partial message.
+	Unmarshal func(UnmarshalInput) (UnmarshalOutput, error)
+
+	// Merge merges the contents of a source message into a destination message.
+	Merge func(MergeInput) MergeOutput
+
+	// CheckInitialized returns an error if any required fields in the message are not set.
+	CheckInitialized func(CheckInitializedInput) (CheckInitializedOutput, error)
+}
+
+// SupportFlags indicate support for optional features.
+type SupportFlags = uint64
+
+const (
+	// SupportMarshalDeterministic reports whether MarshalOptions.Deterministic is supported.
+	SupportMarshalDeterministic SupportFlags = 1 << iota
+
+	// SupportUnmarshalDiscardUnknown reports whether UnmarshalOptions.DiscardUnknown is supported.
+	SupportUnmarshalDiscardUnknown
+)
+
+// SizeInput is input to the Size method.
+type SizeInput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Message protoreflect.Message
+	Flags   MarshalInputFlags
+}
+
+// SizeOutput is output from the Size method.
+type SizeOutput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Size int
+}
+
+// MarshalInput is input to the Marshal method.
+type MarshalInput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Message protoreflect.Message
+	Buf     []byte // output is appended to this buffer
+	Flags   MarshalInputFlags
+}
+
+// MarshalOutput is output from the Marshal method.
+type MarshalOutput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Buf []byte // contains marshaled message
+}
+
+// MarshalInputFlags configure the marshaler.
+// Most flags correspond to fields in proto.MarshalOptions.
+type MarshalInputFlags = uint8
+
+const (
+	MarshalDeterministic MarshalInputFlags = 1 << iota
+	MarshalUseCachedSize
+)
+
+// UnmarshalInput is input to the Unmarshal method.
+type UnmarshalInput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Message  protoreflect.Message
+	Buf      []byte // input buffer
+	Flags    UnmarshalInputFlags
+	Resolver interface {
+		FindExtensionByName(field protoreflect.FullName) (protoreflect.ExtensionType, error)
+		FindExtensionByNumber(message protoreflect.FullName, field protoreflect.FieldNumber) (protoreflect.ExtensionType, error)
+	}
+}
+
+// UnmarshalOutput is output from the Unmarshal method.
+type UnmarshalOutput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Flags UnmarshalOutputFlags
+}
+
+// UnmarshalInputFlags configure the unmarshaler.
+// Most flags correspond to fields in proto.UnmarshalOptions.
+type UnmarshalInputFlags = uint8
+
+const (
+	UnmarshalDiscardUnknown UnmarshalInputFlags = 1 << iota
+)
+
+// UnmarshalOutputFlags are output from the Unmarshal method.
+type UnmarshalOutputFlags = uint8
+
+const (
+	// UnmarshalInitialized may be set on return if all required fields are known to be set.
+	// If unset, then it does not necessarily indicate that the message is uninitialized,
+	// only that its status could not be confirmed.
+	UnmarshalInitialized UnmarshalOutputFlags = 1 << iota
+)
+
+// MergeInput is input to the Merge method.
+type MergeInput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Source      protoreflect.Message
+	Destination protoreflect.Message
+}
+
+// MergeOutput is output from the Merge method.
+type MergeOutput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Flags MergeOutputFlags
+}
+
+// MergeOutputFlags are output from the Merge method.
+type MergeOutputFlags = uint8
+
+const (
+	// MergeComplete reports whether the merge was performed.
+	// If unset, the merger must have made no changes to the destination.
+	MergeComplete MergeOutputFlags = 1 << iota
+)
+
+// CheckInitializedInput is input to the CheckInitialized method.
+type CheckInitializedInput = struct {
+	pragma.NoUnkeyedLiterals
+
+	Message protoreflect.Message
+}
+
+// CheckInitializedOutput is output from the CheckInitialized method.
+type CheckInitializedOutput = struct {
+	pragma.NoUnkeyedLiterals
+}
diff --git a/vendor/google.golang.org/protobuf/runtime/protoimpl/impl.go b/vendor/google.golang.org/protobuf/runtime/protoimpl/impl.go
new file mode 100644
index 0000000..4a1ab7f
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/runtime/protoimpl/impl.go
@@ -0,0 +1,44 @@
+// 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 protoimpl contains the default implementation for messages
+// generated by protoc-gen-go.
+//
+// WARNING: This package should only ever be imported by generated messages.
+// The compatibility agreement covers nothing except for functionality needed
+// to keep existing generated messages operational. Breakages that occur due
+// to unauthorized usages of this package are not the author's responsibility.
+package protoimpl
+
+import (
+	"google.golang.org/protobuf/internal/filedesc"
+	"google.golang.org/protobuf/internal/filetype"
+	"google.golang.org/protobuf/internal/impl"
+)
+
+// UnsafeEnabled specifies whether package unsafe can be used.
+const UnsafeEnabled = impl.UnsafeEnabled
+
+type (
+	// Types used by generated code in init functions.
+	DescBuilder = filedesc.Builder
+	TypeBuilder = filetype.Builder
+
+	// Types used by generated code to implement EnumType, MessageType, and ExtensionType.
+	EnumInfo      = impl.EnumInfo
+	MessageInfo   = impl.MessageInfo
+	ExtensionInfo = impl.ExtensionInfo
+
+	// Types embedded in generated messages.
+	MessageState     = impl.MessageState
+	SizeCache        = impl.SizeCache
+	WeakFields       = impl.WeakFields
+	UnknownFields    = impl.UnknownFields
+	ExtensionFields  = impl.ExtensionFields
+	ExtensionFieldV1 = impl.ExtensionField
+
+	Pointer = impl.Pointer
+)
+
+var X impl.Export
diff --git a/vendor/google.golang.org/protobuf/runtime/protoimpl/version.go b/vendor/google.golang.org/protobuf/runtime/protoimpl/version.go
new file mode 100644
index 0000000..ff094e1
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/runtime/protoimpl/version.go
@@ -0,0 +1,56 @@
+// 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.
+
+package protoimpl
+
+import (
+	"google.golang.org/protobuf/internal/version"
+)
+
+const (
+	// MaxVersion is the maximum supported version for generated .pb.go files.
+	// It is always the current version of the module.
+	MaxVersion = version.Minor
+
+	// GenVersion is the runtime version required by generated .pb.go files.
+	// This is incremented when generated code relies on new functionality
+	// in the runtime.
+	GenVersion = 20
+
+	// MinVersion is the minimum supported version for generated .pb.go files.
+	// This is incremented when the runtime drops support for old code.
+	MinVersion = 0
+)
+
+// EnforceVersion is used by code generated by protoc-gen-go
+// to statically enforce minimum and maximum versions of this package.
+// A compilation failure implies either that:
+//	* the runtime package is too old and needs to be updated OR
+//	* the generated code is too old and needs to be regenerated.
+//
+// The runtime package can be upgraded by running:
+//	go get google.golang.org/protobuf
+//
+// The generated code can be regenerated by running:
+//	protoc --go_out=${PROTOC_GEN_GO_ARGS} ${PROTO_FILES}
+//
+// Example usage by generated code:
+//	const (
+//		// Verify that this generated code is sufficiently up-to-date.
+//		_ = protoimpl.EnforceVersion(genVersion - protoimpl.MinVersion)
+//		// Verify that runtime/protoimpl is sufficiently up-to-date.
+//		_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - genVersion)
+//	)
+//
+// The genVersion is the current minor version used to generated the code.
+// This compile-time check relies on negative integer overflow of a uint
+// being a compilation failure (guaranteed by the Go specification).
+type EnforceVersion uint
+
+// This enforces the following invariant:
+//	MinVersion ≤ GenVersion ≤ MaxVersion
+const (
+	_ = EnforceVersion(GenVersion - MinVersion)
+	_ = EnforceVersion(MaxVersion - GenVersion)
+)
diff --git a/vendor/google.golang.org/protobuf/types/descriptorpb/descriptor.pb.go b/vendor/google.golang.org/protobuf/types/descriptorpb/descriptor.pb.go
new file mode 100644
index 0000000..abe4ab5
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/types/descriptorpb/descriptor.pb.go
@@ -0,0 +1,3957 @@
+// 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)
+//  Based on original Protocol Buffers design by
+//  Sanjay Ghemawat, Jeff Dean, and others.
+//
+// The messages in this file describe the definitions found in .proto files.
+// A valid .proto file can be translated directly to a FileDescriptorProto
+// without any other information (e.g. without reading its imports).
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/descriptor.proto
+
+package descriptorpb
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
+)
+
+type FieldDescriptorProto_Type int32
+
+const (
+	// 0 is reserved for errors.
+	// Order is weird for historical reasons.
+	FieldDescriptorProto_TYPE_DOUBLE FieldDescriptorProto_Type = 1
+	FieldDescriptorProto_TYPE_FLOAT  FieldDescriptorProto_Type = 2
+	// Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT64 if
+	// negative values are likely.
+	FieldDescriptorProto_TYPE_INT64  FieldDescriptorProto_Type = 3
+	FieldDescriptorProto_TYPE_UINT64 FieldDescriptorProto_Type = 4
+	// Not ZigZag encoded.  Negative numbers take 10 bytes.  Use TYPE_SINT32 if
+	// negative values are likely.
+	FieldDescriptorProto_TYPE_INT32   FieldDescriptorProto_Type = 5
+	FieldDescriptorProto_TYPE_FIXED64 FieldDescriptorProto_Type = 6
+	FieldDescriptorProto_TYPE_FIXED32 FieldDescriptorProto_Type = 7
+	FieldDescriptorProto_TYPE_BOOL    FieldDescriptorProto_Type = 8
+	FieldDescriptorProto_TYPE_STRING  FieldDescriptorProto_Type = 9
+	// Tag-delimited aggregate.
+	// Group type is deprecated and not supported in proto3. However, Proto3
+	// implementations should still be able to parse the group wire format and
+	// treat group fields as unknown fields.
+	FieldDescriptorProto_TYPE_GROUP   FieldDescriptorProto_Type = 10
+	FieldDescriptorProto_TYPE_MESSAGE FieldDescriptorProto_Type = 11 // Length-delimited aggregate.
+	// New in version 2.
+	FieldDescriptorProto_TYPE_BYTES    FieldDescriptorProto_Type = 12
+	FieldDescriptorProto_TYPE_UINT32   FieldDescriptorProto_Type = 13
+	FieldDescriptorProto_TYPE_ENUM     FieldDescriptorProto_Type = 14
+	FieldDescriptorProto_TYPE_SFIXED32 FieldDescriptorProto_Type = 15
+	FieldDescriptorProto_TYPE_SFIXED64 FieldDescriptorProto_Type = 16
+	FieldDescriptorProto_TYPE_SINT32   FieldDescriptorProto_Type = 17 // Uses ZigZag encoding.
+	FieldDescriptorProto_TYPE_SINT64   FieldDescriptorProto_Type = 18 // Uses ZigZag encoding.
+)
+
+// Enum value maps for FieldDescriptorProto_Type.
+var (
+	FieldDescriptorProto_Type_name = map[int32]string{
+		1:  "TYPE_DOUBLE",
+		2:  "TYPE_FLOAT",
+		3:  "TYPE_INT64",
+		4:  "TYPE_UINT64",
+		5:  "TYPE_INT32",
+		6:  "TYPE_FIXED64",
+		7:  "TYPE_FIXED32",
+		8:  "TYPE_BOOL",
+		9:  "TYPE_STRING",
+		10: "TYPE_GROUP",
+		11: "TYPE_MESSAGE",
+		12: "TYPE_BYTES",
+		13: "TYPE_UINT32",
+		14: "TYPE_ENUM",
+		15: "TYPE_SFIXED32",
+		16: "TYPE_SFIXED64",
+		17: "TYPE_SINT32",
+		18: "TYPE_SINT64",
+	}
+	FieldDescriptorProto_Type_value = map[string]int32{
+		"TYPE_DOUBLE":   1,
+		"TYPE_FLOAT":    2,
+		"TYPE_INT64":    3,
+		"TYPE_UINT64":   4,
+		"TYPE_INT32":    5,
+		"TYPE_FIXED64":  6,
+		"TYPE_FIXED32":  7,
+		"TYPE_BOOL":     8,
+		"TYPE_STRING":   9,
+		"TYPE_GROUP":    10,
+		"TYPE_MESSAGE":  11,
+		"TYPE_BYTES":    12,
+		"TYPE_UINT32":   13,
+		"TYPE_ENUM":     14,
+		"TYPE_SFIXED32": 15,
+		"TYPE_SFIXED64": 16,
+		"TYPE_SINT32":   17,
+		"TYPE_SINT64":   18,
+	}
+)
+
+func (x FieldDescriptorProto_Type) Enum() *FieldDescriptorProto_Type {
+	p := new(FieldDescriptorProto_Type)
+	*p = x
+	return p
+}
+
+func (x FieldDescriptorProto_Type) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FieldDescriptorProto_Type) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[0].Descriptor()
+}
+
+func (FieldDescriptorProto_Type) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[0]
+}
+
+func (x FieldDescriptorProto_Type) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *FieldDescriptorProto_Type) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = FieldDescriptorProto_Type(num)
+	return nil
+}
+
+// Deprecated: Use FieldDescriptorProto_Type.Descriptor instead.
+func (FieldDescriptorProto_Type) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{4, 0}
+}
+
+type FieldDescriptorProto_Label int32
+
+const (
+	// 0 is reserved for errors
+	FieldDescriptorProto_LABEL_OPTIONAL FieldDescriptorProto_Label = 1
+	FieldDescriptorProto_LABEL_REQUIRED FieldDescriptorProto_Label = 2
+	FieldDescriptorProto_LABEL_REPEATED FieldDescriptorProto_Label = 3
+)
+
+// Enum value maps for FieldDescriptorProto_Label.
+var (
+	FieldDescriptorProto_Label_name = map[int32]string{
+		1: "LABEL_OPTIONAL",
+		2: "LABEL_REQUIRED",
+		3: "LABEL_REPEATED",
+	}
+	FieldDescriptorProto_Label_value = map[string]int32{
+		"LABEL_OPTIONAL": 1,
+		"LABEL_REQUIRED": 2,
+		"LABEL_REPEATED": 3,
+	}
+)
+
+func (x FieldDescriptorProto_Label) Enum() *FieldDescriptorProto_Label {
+	p := new(FieldDescriptorProto_Label)
+	*p = x
+	return p
+}
+
+func (x FieldDescriptorProto_Label) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FieldDescriptorProto_Label) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[1].Descriptor()
+}
+
+func (FieldDescriptorProto_Label) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[1]
+}
+
+func (x FieldDescriptorProto_Label) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *FieldDescriptorProto_Label) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = FieldDescriptorProto_Label(num)
+	return nil
+}
+
+// Deprecated: Use FieldDescriptorProto_Label.Descriptor instead.
+func (FieldDescriptorProto_Label) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{4, 1}
+}
+
+// Generated classes can be optimized for speed or code size.
+type FileOptions_OptimizeMode int32
+
+const (
+	FileOptions_SPEED FileOptions_OptimizeMode = 1 // Generate complete code for parsing, serialization,
+	// etc.
+	FileOptions_CODE_SIZE    FileOptions_OptimizeMode = 2 // Use ReflectionOps to implement these methods.
+	FileOptions_LITE_RUNTIME FileOptions_OptimizeMode = 3 // Generate code using MessageLite and the lite runtime.
+)
+
+// Enum value maps for FileOptions_OptimizeMode.
+var (
+	FileOptions_OptimizeMode_name = map[int32]string{
+		1: "SPEED",
+		2: "CODE_SIZE",
+		3: "LITE_RUNTIME",
+	}
+	FileOptions_OptimizeMode_value = map[string]int32{
+		"SPEED":        1,
+		"CODE_SIZE":    2,
+		"LITE_RUNTIME": 3,
+	}
+)
+
+func (x FileOptions_OptimizeMode) Enum() *FileOptions_OptimizeMode {
+	p := new(FileOptions_OptimizeMode)
+	*p = x
+	return p
+}
+
+func (x FileOptions_OptimizeMode) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FileOptions_OptimizeMode) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[2].Descriptor()
+}
+
+func (FileOptions_OptimizeMode) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[2]
+}
+
+func (x FileOptions_OptimizeMode) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *FileOptions_OptimizeMode) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = FileOptions_OptimizeMode(num)
+	return nil
+}
+
+// Deprecated: Use FileOptions_OptimizeMode.Descriptor instead.
+func (FileOptions_OptimizeMode) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{10, 0}
+}
+
+type FieldOptions_CType int32
+
+const (
+	// Default mode.
+	FieldOptions_STRING       FieldOptions_CType = 0
+	FieldOptions_CORD         FieldOptions_CType = 1
+	FieldOptions_STRING_PIECE FieldOptions_CType = 2
+)
+
+// Enum value maps for FieldOptions_CType.
+var (
+	FieldOptions_CType_name = map[int32]string{
+		0: "STRING",
+		1: "CORD",
+		2: "STRING_PIECE",
+	}
+	FieldOptions_CType_value = map[string]int32{
+		"STRING":       0,
+		"CORD":         1,
+		"STRING_PIECE": 2,
+	}
+)
+
+func (x FieldOptions_CType) Enum() *FieldOptions_CType {
+	p := new(FieldOptions_CType)
+	*p = x
+	return p
+}
+
+func (x FieldOptions_CType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FieldOptions_CType) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[3].Descriptor()
+}
+
+func (FieldOptions_CType) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[3]
+}
+
+func (x FieldOptions_CType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *FieldOptions_CType) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = FieldOptions_CType(num)
+	return nil
+}
+
+// Deprecated: Use FieldOptions_CType.Descriptor instead.
+func (FieldOptions_CType) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{12, 0}
+}
+
+type FieldOptions_JSType int32
+
+const (
+	// Use the default type.
+	FieldOptions_JS_NORMAL FieldOptions_JSType = 0
+	// Use JavaScript strings.
+	FieldOptions_JS_STRING FieldOptions_JSType = 1
+	// Use JavaScript numbers.
+	FieldOptions_JS_NUMBER FieldOptions_JSType = 2
+)
+
+// Enum value maps for FieldOptions_JSType.
+var (
+	FieldOptions_JSType_name = map[int32]string{
+		0: "JS_NORMAL",
+		1: "JS_STRING",
+		2: "JS_NUMBER",
+	}
+	FieldOptions_JSType_value = map[string]int32{
+		"JS_NORMAL": 0,
+		"JS_STRING": 1,
+		"JS_NUMBER": 2,
+	}
+)
+
+func (x FieldOptions_JSType) Enum() *FieldOptions_JSType {
+	p := new(FieldOptions_JSType)
+	*p = x
+	return p
+}
+
+func (x FieldOptions_JSType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FieldOptions_JSType) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[4].Descriptor()
+}
+
+func (FieldOptions_JSType) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[4]
+}
+
+func (x FieldOptions_JSType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *FieldOptions_JSType) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = FieldOptions_JSType(num)
+	return nil
+}
+
+// Deprecated: Use FieldOptions_JSType.Descriptor instead.
+func (FieldOptions_JSType) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{12, 1}
+}
+
+// Is this method side-effect-free (or safe in HTTP parlance), or idempotent,
+// or neither? HTTP based RPC implementation may choose GET verb for safe
+// methods, and PUT verb for idempotent methods instead of the default POST.
+type MethodOptions_IdempotencyLevel int32
+
+const (
+	MethodOptions_IDEMPOTENCY_UNKNOWN MethodOptions_IdempotencyLevel = 0
+	MethodOptions_NO_SIDE_EFFECTS     MethodOptions_IdempotencyLevel = 1 // implies idempotent
+	MethodOptions_IDEMPOTENT          MethodOptions_IdempotencyLevel = 2 // idempotent, but may have side effects
+)
+
+// Enum value maps for MethodOptions_IdempotencyLevel.
+var (
+	MethodOptions_IdempotencyLevel_name = map[int32]string{
+		0: "IDEMPOTENCY_UNKNOWN",
+		1: "NO_SIDE_EFFECTS",
+		2: "IDEMPOTENT",
+	}
+	MethodOptions_IdempotencyLevel_value = map[string]int32{
+		"IDEMPOTENCY_UNKNOWN": 0,
+		"NO_SIDE_EFFECTS":     1,
+		"IDEMPOTENT":          2,
+	}
+)
+
+func (x MethodOptions_IdempotencyLevel) Enum() *MethodOptions_IdempotencyLevel {
+	p := new(MethodOptions_IdempotencyLevel)
+	*p = x
+	return p
+}
+
+func (x MethodOptions_IdempotencyLevel) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (MethodOptions_IdempotencyLevel) Descriptor() protoreflect.EnumDescriptor {
+	return file_google_protobuf_descriptor_proto_enumTypes[5].Descriptor()
+}
+
+func (MethodOptions_IdempotencyLevel) Type() protoreflect.EnumType {
+	return &file_google_protobuf_descriptor_proto_enumTypes[5]
+}
+
+func (x MethodOptions_IdempotencyLevel) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Do not use.
+func (x *MethodOptions_IdempotencyLevel) UnmarshalJSON(b []byte) error {
+	num, err := protoimpl.X.UnmarshalJSONEnum(x.Descriptor(), b)
+	if err != nil {
+		return err
+	}
+	*x = MethodOptions_IdempotencyLevel(num)
+	return nil
+}
+
+// Deprecated: Use MethodOptions_IdempotencyLevel.Descriptor instead.
+func (MethodOptions_IdempotencyLevel) EnumDescriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{17, 0}
+}
+
+// The protocol compiler can output a FileDescriptorSet containing the .proto
+// files it parses.
+type FileDescriptorSet struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	File []*FileDescriptorProto `protobuf:"bytes,1,rep,name=file" json:"file,omitempty"`
+}
+
+func (x *FileDescriptorSet) Reset() {
+	*x = FileDescriptorSet{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FileDescriptorSet) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FileDescriptorSet) ProtoMessage() {}
+
+func (x *FileDescriptorSet) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FileDescriptorSet.ProtoReflect.Descriptor instead.
+func (*FileDescriptorSet) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *FileDescriptorSet) GetFile() []*FileDescriptorProto {
+	if x != nil {
+		return x.File
+	}
+	return nil
+}
+
+// Describes a complete .proto file.
+type FileDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name    *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`       // file name, relative to root of source tree
+	Package *string `protobuf:"bytes,2,opt,name=package" json:"package,omitempty"` // e.g. "foo", "foo.bar", etc.
+	// Names of files imported by this file.
+	Dependency []string `protobuf:"bytes,3,rep,name=dependency" json:"dependency,omitempty"`
+	// Indexes of the public imported files in the dependency list above.
+	PublicDependency []int32 `protobuf:"varint,10,rep,name=public_dependency,json=publicDependency" json:"public_dependency,omitempty"`
+	// Indexes of the weak imported files in the dependency list.
+	// For Google-internal migration only. Do not use.
+	WeakDependency []int32 `protobuf:"varint,11,rep,name=weak_dependency,json=weakDependency" json:"weak_dependency,omitempty"`
+	// All top-level definitions in this file.
+	MessageType []*DescriptorProto        `protobuf:"bytes,4,rep,name=message_type,json=messageType" json:"message_type,omitempty"`
+	EnumType    []*EnumDescriptorProto    `protobuf:"bytes,5,rep,name=enum_type,json=enumType" json:"enum_type,omitempty"`
+	Service     []*ServiceDescriptorProto `protobuf:"bytes,6,rep,name=service" json:"service,omitempty"`
+	Extension   []*FieldDescriptorProto   `protobuf:"bytes,7,rep,name=extension" json:"extension,omitempty"`
+	Options     *FileOptions              `protobuf:"bytes,8,opt,name=options" json:"options,omitempty"`
+	// This field contains optional information about the original source code.
+	// You may safely remove this entire field without harming runtime
+	// functionality of the descriptors -- the information is needed only by
+	// development tools.
+	SourceCodeInfo *SourceCodeInfo `protobuf:"bytes,9,opt,name=source_code_info,json=sourceCodeInfo" json:"source_code_info,omitempty"`
+	// The syntax of the proto file.
+	// The supported values are "proto2" and "proto3".
+	Syntax *string `protobuf:"bytes,12,opt,name=syntax" json:"syntax,omitempty"`
+}
+
+func (x *FileDescriptorProto) Reset() {
+	*x = FileDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FileDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FileDescriptorProto) ProtoMessage() {}
+
+func (x *FileDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FileDescriptorProto.ProtoReflect.Descriptor instead.
+func (*FileDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *FileDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *FileDescriptorProto) GetPackage() string {
+	if x != nil && x.Package != nil {
+		return *x.Package
+	}
+	return ""
+}
+
+func (x *FileDescriptorProto) GetDependency() []string {
+	if x != nil {
+		return x.Dependency
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetPublicDependency() []int32 {
+	if x != nil {
+		return x.PublicDependency
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetWeakDependency() []int32 {
+	if x != nil {
+		return x.WeakDependency
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetMessageType() []*DescriptorProto {
+	if x != nil {
+		return x.MessageType
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetEnumType() []*EnumDescriptorProto {
+	if x != nil {
+		return x.EnumType
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetService() []*ServiceDescriptorProto {
+	if x != nil {
+		return x.Service
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetExtension() []*FieldDescriptorProto {
+	if x != nil {
+		return x.Extension
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetOptions() *FileOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetSourceCodeInfo() *SourceCodeInfo {
+	if x != nil {
+		return x.SourceCodeInfo
+	}
+	return nil
+}
+
+func (x *FileDescriptorProto) GetSyntax() string {
+	if x != nil && x.Syntax != nil {
+		return *x.Syntax
+	}
+	return ""
+}
+
+// Describes a message type.
+type DescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name           *string                           `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Field          []*FieldDescriptorProto           `protobuf:"bytes,2,rep,name=field" json:"field,omitempty"`
+	Extension      []*FieldDescriptorProto           `protobuf:"bytes,6,rep,name=extension" json:"extension,omitempty"`
+	NestedType     []*DescriptorProto                `protobuf:"bytes,3,rep,name=nested_type,json=nestedType" json:"nested_type,omitempty"`
+	EnumType       []*EnumDescriptorProto            `protobuf:"bytes,4,rep,name=enum_type,json=enumType" json:"enum_type,omitempty"`
+	ExtensionRange []*DescriptorProto_ExtensionRange `protobuf:"bytes,5,rep,name=extension_range,json=extensionRange" json:"extension_range,omitempty"`
+	OneofDecl      []*OneofDescriptorProto           `protobuf:"bytes,8,rep,name=oneof_decl,json=oneofDecl" json:"oneof_decl,omitempty"`
+	Options        *MessageOptions                   `protobuf:"bytes,7,opt,name=options" json:"options,omitempty"`
+	ReservedRange  []*DescriptorProto_ReservedRange  `protobuf:"bytes,9,rep,name=reserved_range,json=reservedRange" json:"reserved_range,omitempty"`
+	// Reserved field names, which may not be used by fields in the same message.
+	// A given name may only be reserved once.
+	ReservedName []string `protobuf:"bytes,10,rep,name=reserved_name,json=reservedName" json:"reserved_name,omitempty"`
+}
+
+func (x *DescriptorProto) Reset() {
+	*x = DescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DescriptorProto) ProtoMessage() {}
+
+func (x *DescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DescriptorProto.ProtoReflect.Descriptor instead.
+func (*DescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *DescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *DescriptorProto) GetField() []*FieldDescriptorProto {
+	if x != nil {
+		return x.Field
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetExtension() []*FieldDescriptorProto {
+	if x != nil {
+		return x.Extension
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetNestedType() []*DescriptorProto {
+	if x != nil {
+		return x.NestedType
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetEnumType() []*EnumDescriptorProto {
+	if x != nil {
+		return x.EnumType
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetExtensionRange() []*DescriptorProto_ExtensionRange {
+	if x != nil {
+		return x.ExtensionRange
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetOneofDecl() []*OneofDescriptorProto {
+	if x != nil {
+		return x.OneofDecl
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetOptions() *MessageOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetReservedRange() []*DescriptorProto_ReservedRange {
+	if x != nil {
+		return x.ReservedRange
+	}
+	return nil
+}
+
+func (x *DescriptorProto) GetReservedName() []string {
+	if x != nil {
+		return x.ReservedName
+	}
+	return nil
+}
+
+type ExtensionRangeOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+func (x *ExtensionRangeOptions) Reset() {
+	*x = ExtensionRangeOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExtensionRangeOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExtensionRangeOptions) ProtoMessage() {}
+
+func (x *ExtensionRangeOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExtensionRangeOptions.ProtoReflect.Descriptor instead.
+func (*ExtensionRangeOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{3}
+}
+
+func (x *ExtensionRangeOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+// Describes a field within a message.
+type FieldDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name   *string                     `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Number *int32                      `protobuf:"varint,3,opt,name=number" json:"number,omitempty"`
+	Label  *FieldDescriptorProto_Label `protobuf:"varint,4,opt,name=label,enum=google.protobuf.FieldDescriptorProto_Label" json:"label,omitempty"`
+	// If type_name is set, this need not be set.  If both this and type_name
+	// are set, this must be one of TYPE_ENUM, TYPE_MESSAGE or TYPE_GROUP.
+	Type *FieldDescriptorProto_Type `protobuf:"varint,5,opt,name=type,enum=google.protobuf.FieldDescriptorProto_Type" json:"type,omitempty"`
+	// For message and enum types, this is the name of the type.  If the name
+	// starts with a '.', it is fully-qualified.  Otherwise, C++-like scoping
+	// rules are used to find the type (i.e. first the nested types within this
+	// message are searched, then within the parent, on up to the root
+	// namespace).
+	TypeName *string `protobuf:"bytes,6,opt,name=type_name,json=typeName" json:"type_name,omitempty"`
+	// For extensions, this is the name of the type being extended.  It is
+	// resolved in the same manner as type_name.
+	Extendee *string `protobuf:"bytes,2,opt,name=extendee" json:"extendee,omitempty"`
+	// For numeric types, contains the original text representation of the value.
+	// For booleans, "true" or "false".
+	// For strings, contains the default text contents (not escaped in any way).
+	// For bytes, contains the C escaped value.  All bytes >= 128 are escaped.
+	// TODO(kenton):  Base-64 encode?
+	DefaultValue *string `protobuf:"bytes,7,opt,name=default_value,json=defaultValue" json:"default_value,omitempty"`
+	// If set, gives the index of a oneof in the containing type's oneof_decl
+	// list.  This field is a member of that oneof.
+	OneofIndex *int32 `protobuf:"varint,9,opt,name=oneof_index,json=oneofIndex" json:"oneof_index,omitempty"`
+	// JSON name of this field. The value is set by protocol compiler. If the
+	// user has set a "json_name" option on this field, that option's value
+	// will be used. Otherwise, it's deduced from the field's name by converting
+	// it to camelCase.
+	JsonName *string       `protobuf:"bytes,10,opt,name=json_name,json=jsonName" json:"json_name,omitempty"`
+	Options  *FieldOptions `protobuf:"bytes,8,opt,name=options" json:"options,omitempty"`
+	// If true, this is a proto3 "optional". When a proto3 field is optional, it
+	// tracks presence regardless of field type.
+	//
+	// When proto3_optional is true, this field must be belong to a oneof to
+	// signal to old proto3 clients that presence is tracked for this field. This
+	// oneof is known as a "synthetic" oneof, and this field must be its sole
+	// member (each proto3 optional field gets its own synthetic oneof). Synthetic
+	// oneofs exist in the descriptor only, and do not generate any API. Synthetic
+	// oneofs must be ordered after all "real" oneofs.
+	//
+	// For message fields, proto3_optional doesn't create any semantic change,
+	// since non-repeated message fields always track presence. However it still
+	// indicates the semantic detail of whether the user wrote "optional" or not.
+	// This can be useful for round-tripping the .proto file. For consistency we
+	// give message fields a synthetic oneof also, even though it is not required
+	// to track presence. This is especially important because the parser can't
+	// tell if a field is a message or an enum, so it must always create a
+	// synthetic oneof.
+	//
+	// Proto2 optional fields do not set this flag, because they already indicate
+	// optional with `LABEL_OPTIONAL`.
+	Proto3Optional *bool `protobuf:"varint,17,opt,name=proto3_optional,json=proto3Optional" json:"proto3_optional,omitempty"`
+}
+
+func (x *FieldDescriptorProto) Reset() {
+	*x = FieldDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FieldDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FieldDescriptorProto) ProtoMessage() {}
+
+func (x *FieldDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FieldDescriptorProto.ProtoReflect.Descriptor instead.
+func (*FieldDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{4}
+}
+
+func (x *FieldDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *FieldDescriptorProto) GetNumber() int32 {
+	if x != nil && x.Number != nil {
+		return *x.Number
+	}
+	return 0
+}
+
+func (x *FieldDescriptorProto) GetLabel() FieldDescriptorProto_Label {
+	if x != nil && x.Label != nil {
+		return *x.Label
+	}
+	return FieldDescriptorProto_LABEL_OPTIONAL
+}
+
+func (x *FieldDescriptorProto) GetType() FieldDescriptorProto_Type {
+	if x != nil && x.Type != nil {
+		return *x.Type
+	}
+	return FieldDescriptorProto_TYPE_DOUBLE
+}
+
+func (x *FieldDescriptorProto) GetTypeName() string {
+	if x != nil && x.TypeName != nil {
+		return *x.TypeName
+	}
+	return ""
+}
+
+func (x *FieldDescriptorProto) GetExtendee() string {
+	if x != nil && x.Extendee != nil {
+		return *x.Extendee
+	}
+	return ""
+}
+
+func (x *FieldDescriptorProto) GetDefaultValue() string {
+	if x != nil && x.DefaultValue != nil {
+		return *x.DefaultValue
+	}
+	return ""
+}
+
+func (x *FieldDescriptorProto) GetOneofIndex() int32 {
+	if x != nil && x.OneofIndex != nil {
+		return *x.OneofIndex
+	}
+	return 0
+}
+
+func (x *FieldDescriptorProto) GetJsonName() string {
+	if x != nil && x.JsonName != nil {
+		return *x.JsonName
+	}
+	return ""
+}
+
+func (x *FieldDescriptorProto) GetOptions() *FieldOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+func (x *FieldDescriptorProto) GetProto3Optional() bool {
+	if x != nil && x.Proto3Optional != nil {
+		return *x.Proto3Optional
+	}
+	return false
+}
+
+// Describes a oneof.
+type OneofDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name    *string       `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Options *OneofOptions `protobuf:"bytes,2,opt,name=options" json:"options,omitempty"`
+}
+
+func (x *OneofDescriptorProto) Reset() {
+	*x = OneofDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *OneofDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*OneofDescriptorProto) ProtoMessage() {}
+
+func (x *OneofDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use OneofDescriptorProto.ProtoReflect.Descriptor instead.
+func (*OneofDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{5}
+}
+
+func (x *OneofDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *OneofDescriptorProto) GetOptions() *OneofOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+// Describes an enum type.
+type EnumDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name    *string                     `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Value   []*EnumValueDescriptorProto `protobuf:"bytes,2,rep,name=value" json:"value,omitempty"`
+	Options *EnumOptions                `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
+	// Range of reserved numeric values. Reserved numeric values may not be used
+	// by enum values in the same enum declaration. Reserved ranges may not
+	// overlap.
+	ReservedRange []*EnumDescriptorProto_EnumReservedRange `protobuf:"bytes,4,rep,name=reserved_range,json=reservedRange" json:"reserved_range,omitempty"`
+	// Reserved enum value names, which may not be reused. A given name may only
+	// be reserved once.
+	ReservedName []string `protobuf:"bytes,5,rep,name=reserved_name,json=reservedName" json:"reserved_name,omitempty"`
+}
+
+func (x *EnumDescriptorProto) Reset() {
+	*x = EnumDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EnumDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EnumDescriptorProto) ProtoMessage() {}
+
+func (x *EnumDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EnumDescriptorProto.ProtoReflect.Descriptor instead.
+func (*EnumDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{6}
+}
+
+func (x *EnumDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *EnumDescriptorProto) GetValue() []*EnumValueDescriptorProto {
+	if x != nil {
+		return x.Value
+	}
+	return nil
+}
+
+func (x *EnumDescriptorProto) GetOptions() *EnumOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+func (x *EnumDescriptorProto) GetReservedRange() []*EnumDescriptorProto_EnumReservedRange {
+	if x != nil {
+		return x.ReservedRange
+	}
+	return nil
+}
+
+func (x *EnumDescriptorProto) GetReservedName() []string {
+	if x != nil {
+		return x.ReservedName
+	}
+	return nil
+}
+
+// Describes a value within an enum.
+type EnumValueDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name    *string           `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Number  *int32            `protobuf:"varint,2,opt,name=number" json:"number,omitempty"`
+	Options *EnumValueOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
+}
+
+func (x *EnumValueDescriptorProto) Reset() {
+	*x = EnumValueDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EnumValueDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EnumValueDescriptorProto) ProtoMessage() {}
+
+func (x *EnumValueDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EnumValueDescriptorProto.ProtoReflect.Descriptor instead.
+func (*EnumValueDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{7}
+}
+
+func (x *EnumValueDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *EnumValueDescriptorProto) GetNumber() int32 {
+	if x != nil && x.Number != nil {
+		return *x.Number
+	}
+	return 0
+}
+
+func (x *EnumValueDescriptorProto) GetOptions() *EnumValueOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+// Describes a service.
+type ServiceDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name    *string                  `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Method  []*MethodDescriptorProto `protobuf:"bytes,2,rep,name=method" json:"method,omitempty"`
+	Options *ServiceOptions          `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
+}
+
+func (x *ServiceDescriptorProto) Reset() {
+	*x = ServiceDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ServiceDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ServiceDescriptorProto) ProtoMessage() {}
+
+func (x *ServiceDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ServiceDescriptorProto.ProtoReflect.Descriptor instead.
+func (*ServiceDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{8}
+}
+
+func (x *ServiceDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *ServiceDescriptorProto) GetMethod() []*MethodDescriptorProto {
+	if x != nil {
+		return x.Method
+	}
+	return nil
+}
+
+func (x *ServiceDescriptorProto) GetOptions() *ServiceOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+// Describes a method of a service.
+type MethodDescriptorProto struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	// Input and output type names.  These are resolved in the same way as
+	// FieldDescriptorProto.type_name, but must refer to a message type.
+	InputType  *string        `protobuf:"bytes,2,opt,name=input_type,json=inputType" json:"input_type,omitempty"`
+	OutputType *string        `protobuf:"bytes,3,opt,name=output_type,json=outputType" json:"output_type,omitempty"`
+	Options    *MethodOptions `protobuf:"bytes,4,opt,name=options" json:"options,omitempty"`
+	// Identifies if client streams multiple client messages
+	ClientStreaming *bool `protobuf:"varint,5,opt,name=client_streaming,json=clientStreaming,def=0" json:"client_streaming,omitempty"`
+	// Identifies if server streams multiple server messages
+	ServerStreaming *bool `protobuf:"varint,6,opt,name=server_streaming,json=serverStreaming,def=0" json:"server_streaming,omitempty"`
+}
+
+// Default values for MethodDescriptorProto fields.
+const (
+	Default_MethodDescriptorProto_ClientStreaming = bool(false)
+	Default_MethodDescriptorProto_ServerStreaming = bool(false)
+)
+
+func (x *MethodDescriptorProto) Reset() {
+	*x = MethodDescriptorProto{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MethodDescriptorProto) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MethodDescriptorProto) ProtoMessage() {}
+
+func (x *MethodDescriptorProto) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MethodDescriptorProto.ProtoReflect.Descriptor instead.
+func (*MethodDescriptorProto) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{9}
+}
+
+func (x *MethodDescriptorProto) GetName() string {
+	if x != nil && x.Name != nil {
+		return *x.Name
+	}
+	return ""
+}
+
+func (x *MethodDescriptorProto) GetInputType() string {
+	if x != nil && x.InputType != nil {
+		return *x.InputType
+	}
+	return ""
+}
+
+func (x *MethodDescriptorProto) GetOutputType() string {
+	if x != nil && x.OutputType != nil {
+		return *x.OutputType
+	}
+	return ""
+}
+
+func (x *MethodDescriptorProto) GetOptions() *MethodOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+func (x *MethodDescriptorProto) GetClientStreaming() bool {
+	if x != nil && x.ClientStreaming != nil {
+		return *x.ClientStreaming
+	}
+	return Default_MethodDescriptorProto_ClientStreaming
+}
+
+func (x *MethodDescriptorProto) GetServerStreaming() bool {
+	if x != nil && x.ServerStreaming != nil {
+		return *x.ServerStreaming
+	}
+	return Default_MethodDescriptorProto_ServerStreaming
+}
+
+type FileOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Sets the Java package where classes generated from this .proto will be
+	// placed.  By default, the proto package is used, but this is often
+	// inappropriate because proto packages do not normally start with backwards
+	// domain names.
+	JavaPackage *string `protobuf:"bytes,1,opt,name=java_package,json=javaPackage" json:"java_package,omitempty"`
+	// If set, all the classes from the .proto file are wrapped in a single
+	// outer class with the given name.  This applies to both Proto1
+	// (equivalent to the old "--one_java_file" option) and Proto2 (where
+	// a .proto always translates to a single class, but you may want to
+	// explicitly choose the class name).
+	JavaOuterClassname *string `protobuf:"bytes,8,opt,name=java_outer_classname,json=javaOuterClassname" json:"java_outer_classname,omitempty"`
+	// If set true, then the Java code generator will generate a separate .java
+	// file for each top-level message, enum, and service defined in the .proto
+	// file.  Thus, these types will *not* be nested inside the outer class
+	// named by java_outer_classname.  However, the outer class will still be
+	// generated to contain the file's getDescriptor() method as well as any
+	// top-level extensions defined in the file.
+	JavaMultipleFiles *bool `protobuf:"varint,10,opt,name=java_multiple_files,json=javaMultipleFiles,def=0" json:"java_multiple_files,omitempty"`
+	// This option does nothing.
+	//
+	// Deprecated: Do not use.
+	JavaGenerateEqualsAndHash *bool `protobuf:"varint,20,opt,name=java_generate_equals_and_hash,json=javaGenerateEqualsAndHash" json:"java_generate_equals_and_hash,omitempty"`
+	// If set true, then the Java2 code generator will generate code that
+	// throws an exception whenever an attempt is made to assign a non-UTF-8
+	// byte sequence to a string field.
+	// Message reflection will do the same.
+	// However, an extension field still accepts non-UTF-8 byte sequences.
+	// This option has no effect on when used with the lite runtime.
+	JavaStringCheckUtf8 *bool                     `protobuf:"varint,27,opt,name=java_string_check_utf8,json=javaStringCheckUtf8,def=0" json:"java_string_check_utf8,omitempty"`
+	OptimizeFor         *FileOptions_OptimizeMode `protobuf:"varint,9,opt,name=optimize_for,json=optimizeFor,enum=google.protobuf.FileOptions_OptimizeMode,def=1" json:"optimize_for,omitempty"`
+	// Sets the Go package where structs generated from this .proto will be
+	// placed. If omitted, the Go package will be derived from the following:
+	//   - The basename of the package import path, if provided.
+	//   - Otherwise, the package statement in the .proto file, if present.
+	//   - Otherwise, the basename of the .proto file, without extension.
+	GoPackage *string `protobuf:"bytes,11,opt,name=go_package,json=goPackage" json:"go_package,omitempty"`
+	// Should generic services be generated in each language?  "Generic" services
+	// are not specific to any particular RPC system.  They are generated by the
+	// main code generators in each language (without additional plugins).
+	// Generic services were the only kind of service generation supported by
+	// early versions of google.protobuf.
+	//
+	// Generic services are now considered deprecated in favor of using plugins
+	// that generate code specific to your particular RPC system.  Therefore,
+	// these default to false.  Old code which depends on generic services should
+	// explicitly set them to true.
+	CcGenericServices   *bool `protobuf:"varint,16,opt,name=cc_generic_services,json=ccGenericServices,def=0" json:"cc_generic_services,omitempty"`
+	JavaGenericServices *bool `protobuf:"varint,17,opt,name=java_generic_services,json=javaGenericServices,def=0" json:"java_generic_services,omitempty"`
+	PyGenericServices   *bool `protobuf:"varint,18,opt,name=py_generic_services,json=pyGenericServices,def=0" json:"py_generic_services,omitempty"`
+	PhpGenericServices  *bool `protobuf:"varint,42,opt,name=php_generic_services,json=phpGenericServices,def=0" json:"php_generic_services,omitempty"`
+	// Is this file deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for everything in the file, or it will be completely ignored; in the very
+	// least, this is a formalization for deprecating files.
+	Deprecated *bool `protobuf:"varint,23,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// Enables the use of arenas for the proto messages in this file. This applies
+	// only to generated classes for C++.
+	CcEnableArenas *bool `protobuf:"varint,31,opt,name=cc_enable_arenas,json=ccEnableArenas,def=1" json:"cc_enable_arenas,omitempty"`
+	// Sets the objective c class prefix which is prepended to all objective c
+	// generated classes from this .proto. There is no default.
+	ObjcClassPrefix *string `protobuf:"bytes,36,opt,name=objc_class_prefix,json=objcClassPrefix" json:"objc_class_prefix,omitempty"`
+	// Namespace for generated classes; defaults to the package.
+	CsharpNamespace *string `protobuf:"bytes,37,opt,name=csharp_namespace,json=csharpNamespace" json:"csharp_namespace,omitempty"`
+	// By default Swift generators will take the proto package and CamelCase it
+	// replacing '.' with underscore and use that to prefix the types/symbols
+	// defined. When this options is provided, they will use this value instead
+	// to prefix the types/symbols defined.
+	SwiftPrefix *string `protobuf:"bytes,39,opt,name=swift_prefix,json=swiftPrefix" json:"swift_prefix,omitempty"`
+	// Sets the php class prefix which is prepended to all php generated classes
+	// from this .proto. Default is empty.
+	PhpClassPrefix *string `protobuf:"bytes,40,opt,name=php_class_prefix,json=phpClassPrefix" json:"php_class_prefix,omitempty"`
+	// Use this option to change the namespace of php generated classes. Default
+	// is empty. When this option is empty, the package name will be used for
+	// determining the namespace.
+	PhpNamespace *string `protobuf:"bytes,41,opt,name=php_namespace,json=phpNamespace" json:"php_namespace,omitempty"`
+	// Use this option to change the namespace of php generated metadata classes.
+	// Default is empty. When this option is empty, the proto file name will be
+	// used for determining the namespace.
+	PhpMetadataNamespace *string `protobuf:"bytes,44,opt,name=php_metadata_namespace,json=phpMetadataNamespace" json:"php_metadata_namespace,omitempty"`
+	// Use this option to change the package of ruby generated classes. Default
+	// is empty. When this option is not set, the package name will be used for
+	// determining the ruby package.
+	RubyPackage *string `protobuf:"bytes,45,opt,name=ruby_package,json=rubyPackage" json:"ruby_package,omitempty"`
+	// The parser stores options it doesn't recognize here.
+	// See the documentation for the "Options" section above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for FileOptions fields.
+const (
+	Default_FileOptions_JavaMultipleFiles   = bool(false)
+	Default_FileOptions_JavaStringCheckUtf8 = bool(false)
+	Default_FileOptions_OptimizeFor         = FileOptions_SPEED
+	Default_FileOptions_CcGenericServices   = bool(false)
+	Default_FileOptions_JavaGenericServices = bool(false)
+	Default_FileOptions_PyGenericServices   = bool(false)
+	Default_FileOptions_PhpGenericServices  = bool(false)
+	Default_FileOptions_Deprecated          = bool(false)
+	Default_FileOptions_CcEnableArenas      = bool(true)
+)
+
+func (x *FileOptions) Reset() {
+	*x = FileOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FileOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FileOptions) ProtoMessage() {}
+
+func (x *FileOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FileOptions.ProtoReflect.Descriptor instead.
+func (*FileOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{10}
+}
+
+func (x *FileOptions) GetJavaPackage() string {
+	if x != nil && x.JavaPackage != nil {
+		return *x.JavaPackage
+	}
+	return ""
+}
+
+func (x *FileOptions) GetJavaOuterClassname() string {
+	if x != nil && x.JavaOuterClassname != nil {
+		return *x.JavaOuterClassname
+	}
+	return ""
+}
+
+func (x *FileOptions) GetJavaMultipleFiles() bool {
+	if x != nil && x.JavaMultipleFiles != nil {
+		return *x.JavaMultipleFiles
+	}
+	return Default_FileOptions_JavaMultipleFiles
+}
+
+// Deprecated: Do not use.
+func (x *FileOptions) GetJavaGenerateEqualsAndHash() bool {
+	if x != nil && x.JavaGenerateEqualsAndHash != nil {
+		return *x.JavaGenerateEqualsAndHash
+	}
+	return false
+}
+
+func (x *FileOptions) GetJavaStringCheckUtf8() bool {
+	if x != nil && x.JavaStringCheckUtf8 != nil {
+		return *x.JavaStringCheckUtf8
+	}
+	return Default_FileOptions_JavaStringCheckUtf8
+}
+
+func (x *FileOptions) GetOptimizeFor() FileOptions_OptimizeMode {
+	if x != nil && x.OptimizeFor != nil {
+		return *x.OptimizeFor
+	}
+	return Default_FileOptions_OptimizeFor
+}
+
+func (x *FileOptions) GetGoPackage() string {
+	if x != nil && x.GoPackage != nil {
+		return *x.GoPackage
+	}
+	return ""
+}
+
+func (x *FileOptions) GetCcGenericServices() bool {
+	if x != nil && x.CcGenericServices != nil {
+		return *x.CcGenericServices
+	}
+	return Default_FileOptions_CcGenericServices
+}
+
+func (x *FileOptions) GetJavaGenericServices() bool {
+	if x != nil && x.JavaGenericServices != nil {
+		return *x.JavaGenericServices
+	}
+	return Default_FileOptions_JavaGenericServices
+}
+
+func (x *FileOptions) GetPyGenericServices() bool {
+	if x != nil && x.PyGenericServices != nil {
+		return *x.PyGenericServices
+	}
+	return Default_FileOptions_PyGenericServices
+}
+
+func (x *FileOptions) GetPhpGenericServices() bool {
+	if x != nil && x.PhpGenericServices != nil {
+		return *x.PhpGenericServices
+	}
+	return Default_FileOptions_PhpGenericServices
+}
+
+func (x *FileOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_FileOptions_Deprecated
+}
+
+func (x *FileOptions) GetCcEnableArenas() bool {
+	if x != nil && x.CcEnableArenas != nil {
+		return *x.CcEnableArenas
+	}
+	return Default_FileOptions_CcEnableArenas
+}
+
+func (x *FileOptions) GetObjcClassPrefix() string {
+	if x != nil && x.ObjcClassPrefix != nil {
+		return *x.ObjcClassPrefix
+	}
+	return ""
+}
+
+func (x *FileOptions) GetCsharpNamespace() string {
+	if x != nil && x.CsharpNamespace != nil {
+		return *x.CsharpNamespace
+	}
+	return ""
+}
+
+func (x *FileOptions) GetSwiftPrefix() string {
+	if x != nil && x.SwiftPrefix != nil {
+		return *x.SwiftPrefix
+	}
+	return ""
+}
+
+func (x *FileOptions) GetPhpClassPrefix() string {
+	if x != nil && x.PhpClassPrefix != nil {
+		return *x.PhpClassPrefix
+	}
+	return ""
+}
+
+func (x *FileOptions) GetPhpNamespace() string {
+	if x != nil && x.PhpNamespace != nil {
+		return *x.PhpNamespace
+	}
+	return ""
+}
+
+func (x *FileOptions) GetPhpMetadataNamespace() string {
+	if x != nil && x.PhpMetadataNamespace != nil {
+		return *x.PhpMetadataNamespace
+	}
+	return ""
+}
+
+func (x *FileOptions) GetRubyPackage() string {
+	if x != nil && x.RubyPackage != nil {
+		return *x.RubyPackage
+	}
+	return ""
+}
+
+func (x *FileOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type MessageOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Set true to use the old proto1 MessageSet wire format for extensions.
+	// This is provided for backwards-compatibility with the MessageSet wire
+	// format.  You should not use this for any other reason:  It's less
+	// efficient, has fewer features, and is more complicated.
+	//
+	// The message must be defined exactly as follows:
+	//   message Foo {
+	//     option message_set_wire_format = true;
+	//     extensions 4 to max;
+	//   }
+	// Note that the message cannot have any defined fields; MessageSets only
+	// have extensions.
+	//
+	// All extensions of your type must be singular messages; e.g. they cannot
+	// be int32s, enums, or repeated messages.
+	//
+	// Because this is an option, the above two restrictions are not enforced by
+	// the protocol compiler.
+	MessageSetWireFormat *bool `protobuf:"varint,1,opt,name=message_set_wire_format,json=messageSetWireFormat,def=0" json:"message_set_wire_format,omitempty"`
+	// Disables the generation of the standard "descriptor()" accessor, which can
+	// conflict with a field of the same name.  This is meant to make migration
+	// from proto1 easier; new code should avoid fields named "descriptor".
+	NoStandardDescriptorAccessor *bool `protobuf:"varint,2,opt,name=no_standard_descriptor_accessor,json=noStandardDescriptorAccessor,def=0" json:"no_standard_descriptor_accessor,omitempty"`
+	// Is this message deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for the message, or it will be completely ignored; in the very least,
+	// this is a formalization for deprecating messages.
+	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// Whether the message is an automatically generated map entry type for the
+	// maps field.
+	//
+	// For maps fields:
+	//     map<KeyType, ValueType> map_field = 1;
+	// The parsed descriptor looks like:
+	//     message MapFieldEntry {
+	//         option map_entry = true;
+	//         optional KeyType key = 1;
+	//         optional ValueType value = 2;
+	//     }
+	//     repeated MapFieldEntry map_field = 1;
+	//
+	// Implementations may choose not to generate the map_entry=true message, but
+	// use a native map in the target language to hold the keys and values.
+	// The reflection APIs in such implementations still need to work as
+	// if the field is a repeated message field.
+	//
+	// NOTE: Do not set the option in .proto files. Always use the maps syntax
+	// instead. The option should only be implicitly set by the proto compiler
+	// parser.
+	MapEntry *bool `protobuf:"varint,7,opt,name=map_entry,json=mapEntry" json:"map_entry,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for MessageOptions fields.
+const (
+	Default_MessageOptions_MessageSetWireFormat         = bool(false)
+	Default_MessageOptions_NoStandardDescriptorAccessor = bool(false)
+	Default_MessageOptions_Deprecated                   = bool(false)
+)
+
+func (x *MessageOptions) Reset() {
+	*x = MessageOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MessageOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MessageOptions) ProtoMessage() {}
+
+func (x *MessageOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MessageOptions.ProtoReflect.Descriptor instead.
+func (*MessageOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{11}
+}
+
+func (x *MessageOptions) GetMessageSetWireFormat() bool {
+	if x != nil && x.MessageSetWireFormat != nil {
+		return *x.MessageSetWireFormat
+	}
+	return Default_MessageOptions_MessageSetWireFormat
+}
+
+func (x *MessageOptions) GetNoStandardDescriptorAccessor() bool {
+	if x != nil && x.NoStandardDescriptorAccessor != nil {
+		return *x.NoStandardDescriptorAccessor
+	}
+	return Default_MessageOptions_NoStandardDescriptorAccessor
+}
+
+func (x *MessageOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_MessageOptions_Deprecated
+}
+
+func (x *MessageOptions) GetMapEntry() bool {
+	if x != nil && x.MapEntry != nil {
+		return *x.MapEntry
+	}
+	return false
+}
+
+func (x *MessageOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type FieldOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// The ctype option instructs the C++ code generator to use a different
+	// representation of the field than it normally would.  See the specific
+	// options below.  This option is not yet implemented in the open source
+	// release -- sorry, we'll try to include it in a future version!
+	Ctype *FieldOptions_CType `protobuf:"varint,1,opt,name=ctype,enum=google.protobuf.FieldOptions_CType,def=0" json:"ctype,omitempty"`
+	// The packed option can be enabled for repeated primitive fields to enable
+	// a more efficient representation on the wire. Rather than repeatedly
+	// writing the tag and type for each element, the entire array is encoded as
+	// a single length-delimited blob. In proto3, only explicit setting it to
+	// false will avoid using packed encoding.
+	Packed *bool `protobuf:"varint,2,opt,name=packed" json:"packed,omitempty"`
+	// The jstype option determines the JavaScript type used for values of the
+	// field.  The option is permitted only for 64 bit integral and fixed types
+	// (int64, uint64, sint64, fixed64, sfixed64).  A field with jstype JS_STRING
+	// is represented as JavaScript string, which avoids loss of precision that
+	// can happen when a large value is converted to a floating point JavaScript.
+	// Specifying JS_NUMBER for the jstype causes the generated JavaScript code to
+	// use the JavaScript "number" type.  The behavior of the default option
+	// JS_NORMAL is implementation dependent.
+	//
+	// This option is an enum to permit additional types to be added, e.g.
+	// goog.math.Integer.
+	Jstype *FieldOptions_JSType `protobuf:"varint,6,opt,name=jstype,enum=google.protobuf.FieldOptions_JSType,def=0" json:"jstype,omitempty"`
+	// Should this field be parsed lazily?  Lazy applies only to message-type
+	// fields.  It means that when the outer message is initially parsed, the
+	// inner message's contents will not be parsed but instead stored in encoded
+	// form.  The inner message will actually be parsed when it is first accessed.
+	//
+	// This is only a hint.  Implementations are free to choose whether to use
+	// eager or lazy parsing regardless of the value of this option.  However,
+	// setting this option true suggests that the protocol author believes that
+	// using lazy parsing on this field is worth the additional bookkeeping
+	// overhead typically needed to implement it.
+	//
+	// This option does not affect the public interface of any generated code;
+	// all method signatures remain the same.  Furthermore, thread-safety of the
+	// interface is not affected by this option; const methods remain safe to
+	// call from multiple threads concurrently, while non-const methods continue
+	// to require exclusive access.
+	//
+	//
+	// Note that implementations may choose not to check required fields within
+	// a lazy sub-message.  That is, calling IsInitialized() on the outer message
+	// may return true even if the inner message has missing required fields.
+	// This is necessary because otherwise the inner message would have to be
+	// parsed in order to perform the check, defeating the purpose of lazy
+	// parsing.  An implementation which chooses not to check required fields
+	// must be consistent about it.  That is, for any particular sub-message, the
+	// implementation must either *always* check its required fields, or *never*
+	// check its required fields, regardless of whether or not the message has
+	// been parsed.
+	Lazy *bool `protobuf:"varint,5,opt,name=lazy,def=0" json:"lazy,omitempty"`
+	// Is this field deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for accessors, or it will be completely ignored; in the very least, this
+	// is a formalization for deprecating fields.
+	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// For Google-internal migration only. Do not use.
+	Weak *bool `protobuf:"varint,10,opt,name=weak,def=0" json:"weak,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for FieldOptions fields.
+const (
+	Default_FieldOptions_Ctype      = FieldOptions_STRING
+	Default_FieldOptions_Jstype     = FieldOptions_JS_NORMAL
+	Default_FieldOptions_Lazy       = bool(false)
+	Default_FieldOptions_Deprecated = bool(false)
+	Default_FieldOptions_Weak       = bool(false)
+)
+
+func (x *FieldOptions) Reset() {
+	*x = FieldOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FieldOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FieldOptions) ProtoMessage() {}
+
+func (x *FieldOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FieldOptions.ProtoReflect.Descriptor instead.
+func (*FieldOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{12}
+}
+
+func (x *FieldOptions) GetCtype() FieldOptions_CType {
+	if x != nil && x.Ctype != nil {
+		return *x.Ctype
+	}
+	return Default_FieldOptions_Ctype
+}
+
+func (x *FieldOptions) GetPacked() bool {
+	if x != nil && x.Packed != nil {
+		return *x.Packed
+	}
+	return false
+}
+
+func (x *FieldOptions) GetJstype() FieldOptions_JSType {
+	if x != nil && x.Jstype != nil {
+		return *x.Jstype
+	}
+	return Default_FieldOptions_Jstype
+}
+
+func (x *FieldOptions) GetLazy() bool {
+	if x != nil && x.Lazy != nil {
+		return *x.Lazy
+	}
+	return Default_FieldOptions_Lazy
+}
+
+func (x *FieldOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_FieldOptions_Deprecated
+}
+
+func (x *FieldOptions) GetWeak() bool {
+	if x != nil && x.Weak != nil {
+		return *x.Weak
+	}
+	return Default_FieldOptions_Weak
+}
+
+func (x *FieldOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type OneofOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+func (x *OneofOptions) Reset() {
+	*x = OneofOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *OneofOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*OneofOptions) ProtoMessage() {}
+
+func (x *OneofOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use OneofOptions.ProtoReflect.Descriptor instead.
+func (*OneofOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{13}
+}
+
+func (x *OneofOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type EnumOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Set this option to true to allow mapping different tag names to the same
+	// value.
+	AllowAlias *bool `protobuf:"varint,2,opt,name=allow_alias,json=allowAlias" json:"allow_alias,omitempty"`
+	// Is this enum deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for the enum, or it will be completely ignored; in the very least, this
+	// is a formalization for deprecating enums.
+	Deprecated *bool `protobuf:"varint,3,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for EnumOptions fields.
+const (
+	Default_EnumOptions_Deprecated = bool(false)
+)
+
+func (x *EnumOptions) Reset() {
+	*x = EnumOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EnumOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EnumOptions) ProtoMessage() {}
+
+func (x *EnumOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EnumOptions.ProtoReflect.Descriptor instead.
+func (*EnumOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{14}
+}
+
+func (x *EnumOptions) GetAllowAlias() bool {
+	if x != nil && x.AllowAlias != nil {
+		return *x.AllowAlias
+	}
+	return false
+}
+
+func (x *EnumOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_EnumOptions_Deprecated
+}
+
+func (x *EnumOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type EnumValueOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Is this enum value deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for the enum value, or it will be completely ignored; in the very least,
+	// this is a formalization for deprecating enum values.
+	Deprecated *bool `protobuf:"varint,1,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for EnumValueOptions fields.
+const (
+	Default_EnumValueOptions_Deprecated = bool(false)
+)
+
+func (x *EnumValueOptions) Reset() {
+	*x = EnumValueOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EnumValueOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EnumValueOptions) ProtoMessage() {}
+
+func (x *EnumValueOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EnumValueOptions.ProtoReflect.Descriptor instead.
+func (*EnumValueOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{15}
+}
+
+func (x *EnumValueOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_EnumValueOptions_Deprecated
+}
+
+func (x *EnumValueOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type ServiceOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Is this service deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for the service, or it will be completely ignored; in the very least,
+	// this is a formalization for deprecating services.
+	Deprecated *bool `protobuf:"varint,33,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for ServiceOptions fields.
+const (
+	Default_ServiceOptions_Deprecated = bool(false)
+)
+
+func (x *ServiceOptions) Reset() {
+	*x = ServiceOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ServiceOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ServiceOptions) ProtoMessage() {}
+
+func (x *ServiceOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ServiceOptions.ProtoReflect.Descriptor instead.
+func (*ServiceOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{16}
+}
+
+func (x *ServiceOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_ServiceOptions_Deprecated
+}
+
+func (x *ServiceOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+type MethodOptions struct {
+	state           protoimpl.MessageState
+	sizeCache       protoimpl.SizeCache
+	unknownFields   protoimpl.UnknownFields
+	extensionFields protoimpl.ExtensionFields
+
+	// Is this method deprecated?
+	// Depending on the target platform, this can emit Deprecated annotations
+	// for the method, or it will be completely ignored; in the very least,
+	// this is a formalization for deprecating methods.
+	Deprecated       *bool                           `protobuf:"varint,33,opt,name=deprecated,def=0" json:"deprecated,omitempty"`
+	IdempotencyLevel *MethodOptions_IdempotencyLevel `protobuf:"varint,34,opt,name=idempotency_level,json=idempotencyLevel,enum=google.protobuf.MethodOptions_IdempotencyLevel,def=0" json:"idempotency_level,omitempty"`
+	// The parser stores options it doesn't recognize here. See above.
+	UninterpretedOption []*UninterpretedOption `protobuf:"bytes,999,rep,name=uninterpreted_option,json=uninterpretedOption" json:"uninterpreted_option,omitempty"`
+}
+
+// Default values for MethodOptions fields.
+const (
+	Default_MethodOptions_Deprecated       = bool(false)
+	Default_MethodOptions_IdempotencyLevel = MethodOptions_IDEMPOTENCY_UNKNOWN
+)
+
+func (x *MethodOptions) Reset() {
+	*x = MethodOptions{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[17]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MethodOptions) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MethodOptions) ProtoMessage() {}
+
+func (x *MethodOptions) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[17]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MethodOptions.ProtoReflect.Descriptor instead.
+func (*MethodOptions) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{17}
+}
+
+func (x *MethodOptions) GetDeprecated() bool {
+	if x != nil && x.Deprecated != nil {
+		return *x.Deprecated
+	}
+	return Default_MethodOptions_Deprecated
+}
+
+func (x *MethodOptions) GetIdempotencyLevel() MethodOptions_IdempotencyLevel {
+	if x != nil && x.IdempotencyLevel != nil {
+		return *x.IdempotencyLevel
+	}
+	return Default_MethodOptions_IdempotencyLevel
+}
+
+func (x *MethodOptions) GetUninterpretedOption() []*UninterpretedOption {
+	if x != nil {
+		return x.UninterpretedOption
+	}
+	return nil
+}
+
+// A message representing a option the parser does not recognize. This only
+// appears in options protos created by the compiler::Parser class.
+// DescriptorPool resolves these when building Descriptor objects. Therefore,
+// options protos in descriptor objects (e.g. returned by Descriptor::options(),
+// or produced by Descriptor::CopyTo()) will never have UninterpretedOptions
+// in them.
+type UninterpretedOption struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name []*UninterpretedOption_NamePart `protobuf:"bytes,2,rep,name=name" json:"name,omitempty"`
+	// The value of the uninterpreted option, in whatever type the tokenizer
+	// identified it as during parsing. Exactly one of these should be set.
+	IdentifierValue  *string  `protobuf:"bytes,3,opt,name=identifier_value,json=identifierValue" json:"identifier_value,omitempty"`
+	PositiveIntValue *uint64  `protobuf:"varint,4,opt,name=positive_int_value,json=positiveIntValue" json:"positive_int_value,omitempty"`
+	NegativeIntValue *int64   `protobuf:"varint,5,opt,name=negative_int_value,json=negativeIntValue" json:"negative_int_value,omitempty"`
+	DoubleValue      *float64 `protobuf:"fixed64,6,opt,name=double_value,json=doubleValue" json:"double_value,omitempty"`
+	StringValue      []byte   `protobuf:"bytes,7,opt,name=string_value,json=stringValue" json:"string_value,omitempty"`
+	AggregateValue   *string  `protobuf:"bytes,8,opt,name=aggregate_value,json=aggregateValue" json:"aggregate_value,omitempty"`
+}
+
+func (x *UninterpretedOption) Reset() {
+	*x = UninterpretedOption{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[18]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *UninterpretedOption) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*UninterpretedOption) ProtoMessage() {}
+
+func (x *UninterpretedOption) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[18]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use UninterpretedOption.ProtoReflect.Descriptor instead.
+func (*UninterpretedOption) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{18}
+}
+
+func (x *UninterpretedOption) GetName() []*UninterpretedOption_NamePart {
+	if x != nil {
+		return x.Name
+	}
+	return nil
+}
+
+func (x *UninterpretedOption) GetIdentifierValue() string {
+	if x != nil && x.IdentifierValue != nil {
+		return *x.IdentifierValue
+	}
+	return ""
+}
+
+func (x *UninterpretedOption) GetPositiveIntValue() uint64 {
+	if x != nil && x.PositiveIntValue != nil {
+		return *x.PositiveIntValue
+	}
+	return 0
+}
+
+func (x *UninterpretedOption) GetNegativeIntValue() int64 {
+	if x != nil && x.NegativeIntValue != nil {
+		return *x.NegativeIntValue
+	}
+	return 0
+}
+
+func (x *UninterpretedOption) GetDoubleValue() float64 {
+	if x != nil && x.DoubleValue != nil {
+		return *x.DoubleValue
+	}
+	return 0
+}
+
+func (x *UninterpretedOption) GetStringValue() []byte {
+	if x != nil {
+		return x.StringValue
+	}
+	return nil
+}
+
+func (x *UninterpretedOption) GetAggregateValue() string {
+	if x != nil && x.AggregateValue != nil {
+		return *x.AggregateValue
+	}
+	return ""
+}
+
+// Encapsulates information about the original source file from which a
+// FileDescriptorProto was generated.
+type SourceCodeInfo struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// A Location identifies a piece of source code in a .proto file which
+	// corresponds to a particular definition.  This information is intended
+	// to be useful to IDEs, code indexers, documentation generators, and similar
+	// tools.
+	//
+	// For example, say we have a file like:
+	//   message Foo {
+	//     optional string foo = 1;
+	//   }
+	// Let's look at just the field definition:
+	//   optional string foo = 1;
+	//   ^       ^^     ^^  ^  ^^^
+	//   a       bc     de  f  ghi
+	// We have the following locations:
+	//   span   path               represents
+	//   [a,i)  [ 4, 0, 2, 0 ]     The whole field definition.
+	//   [a,b)  [ 4, 0, 2, 0, 4 ]  The label (optional).
+	//   [c,d)  [ 4, 0, 2, 0, 5 ]  The type (string).
+	//   [e,f)  [ 4, 0, 2, 0, 1 ]  The name (foo).
+	//   [g,h)  [ 4, 0, 2, 0, 3 ]  The number (1).
+	//
+	// Notes:
+	// - A location may refer to a repeated field itself (i.e. not to any
+	//   particular index within it).  This is used whenever a set of elements are
+	//   logically enclosed in a single code segment.  For example, an entire
+	//   extend block (possibly containing multiple extension definitions) will
+	//   have an outer location whose path refers to the "extensions" repeated
+	//   field without an index.
+	// - Multiple locations may have the same path.  This happens when a single
+	//   logical declaration is spread out across multiple places.  The most
+	//   obvious example is the "extend" block again -- there may be multiple
+	//   extend blocks in the same scope, each of which will have the same path.
+	// - A location's span is not always a subset of its parent's span.  For
+	//   example, the "extendee" of an extension declaration appears at the
+	//   beginning of the "extend" block and is shared by all extensions within
+	//   the block.
+	// - Just because a location's span is a subset of some other location's span
+	//   does not mean that it is a descendant.  For example, a "group" defines
+	//   both a type and a field in a single declaration.  Thus, the locations
+	//   corresponding to the type and field and their components will overlap.
+	// - Code which tries to interpret locations should probably be designed to
+	//   ignore those that it doesn't understand, as more types of locations could
+	//   be recorded in the future.
+	Location []*SourceCodeInfo_Location `protobuf:"bytes,1,rep,name=location" json:"location,omitempty"`
+}
+
+func (x *SourceCodeInfo) Reset() {
+	*x = SourceCodeInfo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[19]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SourceCodeInfo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SourceCodeInfo) ProtoMessage() {}
+
+func (x *SourceCodeInfo) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[19]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SourceCodeInfo.ProtoReflect.Descriptor instead.
+func (*SourceCodeInfo) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{19}
+}
+
+func (x *SourceCodeInfo) GetLocation() []*SourceCodeInfo_Location {
+	if x != nil {
+		return x.Location
+	}
+	return nil
+}
+
+// Describes the relationship between generated code and its original source
+// file. A GeneratedCodeInfo message is associated with only one generated
+// source file, but may contain references to different source .proto files.
+type GeneratedCodeInfo struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// An Annotation connects some span of text in generated code to an element
+	// of its generating .proto file.
+	Annotation []*GeneratedCodeInfo_Annotation `protobuf:"bytes,1,rep,name=annotation" json:"annotation,omitempty"`
+}
+
+func (x *GeneratedCodeInfo) Reset() {
+	*x = GeneratedCodeInfo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[20]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GeneratedCodeInfo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GeneratedCodeInfo) ProtoMessage() {}
+
+func (x *GeneratedCodeInfo) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[20]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GeneratedCodeInfo.ProtoReflect.Descriptor instead.
+func (*GeneratedCodeInfo) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{20}
+}
+
+func (x *GeneratedCodeInfo) GetAnnotation() []*GeneratedCodeInfo_Annotation {
+	if x != nil {
+		return x.Annotation
+	}
+	return nil
+}
+
+type DescriptorProto_ExtensionRange struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Start   *int32                 `protobuf:"varint,1,opt,name=start" json:"start,omitempty"` // Inclusive.
+	End     *int32                 `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`     // Exclusive.
+	Options *ExtensionRangeOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
+}
+
+func (x *DescriptorProto_ExtensionRange) Reset() {
+	*x = DescriptorProto_ExtensionRange{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[21]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DescriptorProto_ExtensionRange) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DescriptorProto_ExtensionRange) ProtoMessage() {}
+
+func (x *DescriptorProto_ExtensionRange) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[21]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DescriptorProto_ExtensionRange.ProtoReflect.Descriptor instead.
+func (*DescriptorProto_ExtensionRange) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{2, 0}
+}
+
+func (x *DescriptorProto_ExtensionRange) GetStart() int32 {
+	if x != nil && x.Start != nil {
+		return *x.Start
+	}
+	return 0
+}
+
+func (x *DescriptorProto_ExtensionRange) GetEnd() int32 {
+	if x != nil && x.End != nil {
+		return *x.End
+	}
+	return 0
+}
+
+func (x *DescriptorProto_ExtensionRange) GetOptions() *ExtensionRangeOptions {
+	if x != nil {
+		return x.Options
+	}
+	return nil
+}
+
+// Range of reserved tag numbers. Reserved tag numbers may not be used by
+// fields or extension ranges in the same message. Reserved ranges may
+// not overlap.
+type DescriptorProto_ReservedRange struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Start *int32 `protobuf:"varint,1,opt,name=start" json:"start,omitempty"` // Inclusive.
+	End   *int32 `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`     // Exclusive.
+}
+
+func (x *DescriptorProto_ReservedRange) Reset() {
+	*x = DescriptorProto_ReservedRange{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[22]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DescriptorProto_ReservedRange) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DescriptorProto_ReservedRange) ProtoMessage() {}
+
+func (x *DescriptorProto_ReservedRange) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[22]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DescriptorProto_ReservedRange.ProtoReflect.Descriptor instead.
+func (*DescriptorProto_ReservedRange) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{2, 1}
+}
+
+func (x *DescriptorProto_ReservedRange) GetStart() int32 {
+	if x != nil && x.Start != nil {
+		return *x.Start
+	}
+	return 0
+}
+
+func (x *DescriptorProto_ReservedRange) GetEnd() int32 {
+	if x != nil && x.End != nil {
+		return *x.End
+	}
+	return 0
+}
+
+// Range of reserved numeric values. Reserved values may not be used by
+// entries in the same enum. Reserved ranges may not overlap.
+//
+// Note that this is distinct from DescriptorProto.ReservedRange in that it
+// is inclusive such that it can appropriately represent the entire int32
+// domain.
+type EnumDescriptorProto_EnumReservedRange struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Start *int32 `protobuf:"varint,1,opt,name=start" json:"start,omitempty"` // Inclusive.
+	End   *int32 `protobuf:"varint,2,opt,name=end" json:"end,omitempty"`     // Inclusive.
+}
+
+func (x *EnumDescriptorProto_EnumReservedRange) Reset() {
+	*x = EnumDescriptorProto_EnumReservedRange{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[23]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EnumDescriptorProto_EnumReservedRange) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EnumDescriptorProto_EnumReservedRange) ProtoMessage() {}
+
+func (x *EnumDescriptorProto_EnumReservedRange) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[23]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EnumDescriptorProto_EnumReservedRange.ProtoReflect.Descriptor instead.
+func (*EnumDescriptorProto_EnumReservedRange) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{6, 0}
+}
+
+func (x *EnumDescriptorProto_EnumReservedRange) GetStart() int32 {
+	if x != nil && x.Start != nil {
+		return *x.Start
+	}
+	return 0
+}
+
+func (x *EnumDescriptorProto_EnumReservedRange) GetEnd() int32 {
+	if x != nil && x.End != nil {
+		return *x.End
+	}
+	return 0
+}
+
+// The name of the uninterpreted option.  Each string represents a segment in
+// a dot-separated name.  is_extension is true iff a segment represents an
+// extension (denoted with parentheses in options specs in .proto files).
+// E.g.,{ ["foo", false], ["bar.baz", true], ["qux", false] } represents
+// "foo.(bar.baz).qux".
+type UninterpretedOption_NamePart struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	NamePart    *string `protobuf:"bytes,1,req,name=name_part,json=namePart" json:"name_part,omitempty"`
+	IsExtension *bool   `protobuf:"varint,2,req,name=is_extension,json=isExtension" json:"is_extension,omitempty"`
+}
+
+func (x *UninterpretedOption_NamePart) Reset() {
+	*x = UninterpretedOption_NamePart{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[24]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *UninterpretedOption_NamePart) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*UninterpretedOption_NamePart) ProtoMessage() {}
+
+func (x *UninterpretedOption_NamePart) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[24]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use UninterpretedOption_NamePart.ProtoReflect.Descriptor instead.
+func (*UninterpretedOption_NamePart) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{18, 0}
+}
+
+func (x *UninterpretedOption_NamePart) GetNamePart() string {
+	if x != nil && x.NamePart != nil {
+		return *x.NamePart
+	}
+	return ""
+}
+
+func (x *UninterpretedOption_NamePart) GetIsExtension() bool {
+	if x != nil && x.IsExtension != nil {
+		return *x.IsExtension
+	}
+	return false
+}
+
+type SourceCodeInfo_Location struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Identifies which part of the FileDescriptorProto was defined at this
+	// location.
+	//
+	// Each element is a field number or an index.  They form a path from
+	// the root FileDescriptorProto to the place where the definition.  For
+	// example, this path:
+	//   [ 4, 3, 2, 7, 1 ]
+	// refers to:
+	//   file.message_type(3)  // 4, 3
+	//       .field(7)         // 2, 7
+	//       .name()           // 1
+	// This is because FileDescriptorProto.message_type has field number 4:
+	//   repeated DescriptorProto message_type = 4;
+	// and DescriptorProto.field has field number 2:
+	//   repeated FieldDescriptorProto field = 2;
+	// and FieldDescriptorProto.name has field number 1:
+	//   optional string name = 1;
+	//
+	// Thus, the above path gives the location of a field name.  If we removed
+	// the last element:
+	//   [ 4, 3, 2, 7 ]
+	// this path refers to the whole field declaration (from the beginning
+	// of the label to the terminating semicolon).
+	Path []int32 `protobuf:"varint,1,rep,packed,name=path" json:"path,omitempty"`
+	// Always has exactly three or four elements: start line, start column,
+	// end line (optional, otherwise assumed same as start line), end column.
+	// These are packed into a single field for efficiency.  Note that line
+	// and column numbers are zero-based -- typically you will want to add
+	// 1 to each before displaying to a user.
+	Span []int32 `protobuf:"varint,2,rep,packed,name=span" json:"span,omitempty"`
+	// If this SourceCodeInfo represents a complete declaration, these are any
+	// comments appearing before and after the declaration which appear to be
+	// attached to the declaration.
+	//
+	// A series of line comments appearing on consecutive lines, with no other
+	// tokens appearing on those lines, will be treated as a single comment.
+	//
+	// leading_detached_comments will keep paragraphs of comments that appear
+	// before (but not connected to) the current element. Each paragraph,
+	// separated by empty lines, will be one comment element in the repeated
+	// field.
+	//
+	// Only the comment content is provided; comment markers (e.g. //) are
+	// stripped out.  For block comments, leading whitespace and an asterisk
+	// will be stripped from the beginning of each line other than the first.
+	// Newlines are included in the output.
+	//
+	// Examples:
+	//
+	//   optional int32 foo = 1;  // Comment attached to foo.
+	//   // Comment attached to bar.
+	//   optional int32 bar = 2;
+	//
+	//   optional string baz = 3;
+	//   // Comment attached to baz.
+	//   // Another line attached to baz.
+	//
+	//   // Comment attached to qux.
+	//   //
+	//   // Another line attached to qux.
+	//   optional double qux = 4;
+	//
+	//   // Detached comment for corge. This is not leading or trailing comments
+	//   // to qux or corge because there are blank lines separating it from
+	//   // both.
+	//
+	//   // Detached comment for corge paragraph 2.
+	//
+	//   optional string corge = 5;
+	//   /* Block comment attached
+	//    * to corge.  Leading asterisks
+	//    * will be removed. */
+	//   /* Block comment attached to
+	//    * grault. */
+	//   optional int32 grault = 6;
+	//
+	//   // ignored detached comments.
+	LeadingComments         *string  `protobuf:"bytes,3,opt,name=leading_comments,json=leadingComments" json:"leading_comments,omitempty"`
+	TrailingComments        *string  `protobuf:"bytes,4,opt,name=trailing_comments,json=trailingComments" json:"trailing_comments,omitempty"`
+	LeadingDetachedComments []string `protobuf:"bytes,6,rep,name=leading_detached_comments,json=leadingDetachedComments" json:"leading_detached_comments,omitempty"`
+}
+
+func (x *SourceCodeInfo_Location) Reset() {
+	*x = SourceCodeInfo_Location{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[25]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SourceCodeInfo_Location) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SourceCodeInfo_Location) ProtoMessage() {}
+
+func (x *SourceCodeInfo_Location) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[25]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SourceCodeInfo_Location.ProtoReflect.Descriptor instead.
+func (*SourceCodeInfo_Location) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{19, 0}
+}
+
+func (x *SourceCodeInfo_Location) GetPath() []int32 {
+	if x != nil {
+		return x.Path
+	}
+	return nil
+}
+
+func (x *SourceCodeInfo_Location) GetSpan() []int32 {
+	if x != nil {
+		return x.Span
+	}
+	return nil
+}
+
+func (x *SourceCodeInfo_Location) GetLeadingComments() string {
+	if x != nil && x.LeadingComments != nil {
+		return *x.LeadingComments
+	}
+	return ""
+}
+
+func (x *SourceCodeInfo_Location) GetTrailingComments() string {
+	if x != nil && x.TrailingComments != nil {
+		return *x.TrailingComments
+	}
+	return ""
+}
+
+func (x *SourceCodeInfo_Location) GetLeadingDetachedComments() []string {
+	if x != nil {
+		return x.LeadingDetachedComments
+	}
+	return nil
+}
+
+type GeneratedCodeInfo_Annotation struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Identifies the element in the original source .proto file. This field
+	// is formatted the same as SourceCodeInfo.Location.path.
+	Path []int32 `protobuf:"varint,1,rep,packed,name=path" json:"path,omitempty"`
+	// Identifies the filesystem path to the original source .proto.
+	SourceFile *string `protobuf:"bytes,2,opt,name=source_file,json=sourceFile" json:"source_file,omitempty"`
+	// Identifies the starting offset in bytes in the generated code
+	// that relates to the identified object.
+	Begin *int32 `protobuf:"varint,3,opt,name=begin" json:"begin,omitempty"`
+	// Identifies the ending offset in bytes in the generated code that
+	// relates to the identified offset. The end offset should be one past
+	// the last relevant byte (so the length of the text = end - begin).
+	End *int32 `protobuf:"varint,4,opt,name=end" json:"end,omitempty"`
+}
+
+func (x *GeneratedCodeInfo_Annotation) Reset() {
+	*x = GeneratedCodeInfo_Annotation{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_descriptor_proto_msgTypes[26]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GeneratedCodeInfo_Annotation) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GeneratedCodeInfo_Annotation) ProtoMessage() {}
+
+func (x *GeneratedCodeInfo_Annotation) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_descriptor_proto_msgTypes[26]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GeneratedCodeInfo_Annotation.ProtoReflect.Descriptor instead.
+func (*GeneratedCodeInfo_Annotation) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_descriptor_proto_rawDescGZIP(), []int{20, 0}
+}
+
+func (x *GeneratedCodeInfo_Annotation) GetPath() []int32 {
+	if x != nil {
+		return x.Path
+	}
+	return nil
+}
+
+func (x *GeneratedCodeInfo_Annotation) GetSourceFile() string {
+	if x != nil && x.SourceFile != nil {
+		return *x.SourceFile
+	}
+	return ""
+}
+
+func (x *GeneratedCodeInfo_Annotation) GetBegin() int32 {
+	if x != nil && x.Begin != nil {
+		return *x.Begin
+	}
+	return 0
+}
+
+func (x *GeneratedCodeInfo_Annotation) GetEnd() int32 {
+	if x != nil && x.End != nil {
+		return *x.End
+	}
+	return 0
+}
+
+var File_google_protobuf_descriptor_proto protoreflect.FileDescriptor
+
+var file_google_protobuf_descriptor_proto_rawDesc = []byte{
+	0x0a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x12, 0x0f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x22, 0x4d, 0x0a, 0x11, 0x46, 0x69, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x74, 0x12, 0x38, 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x65,
+	0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x04, 0x66, 0x69,
+	0x6c, 0x65, 0x22, 0xe4, 0x04, 0x0a, 0x13, 0x46, 0x69, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61,
+	0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18,
+	0x0a, 0x07, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x07, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x65,
+	0x6e, 0x64, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x65,
+	0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x2b, 0x0a, 0x11, 0x70, 0x75, 0x62, 0x6c,
+	0x69, 0x63, 0x5f, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x0a, 0x20,
+	0x03, 0x28, 0x05, 0x52, 0x10, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x44, 0x65, 0x70, 0x65, 0x6e,
+	0x64, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x65, 0x61, 0x6b, 0x5f, 0x64, 0x65,
+	0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0e,
+	0x77, 0x65, 0x61, 0x6b, 0x44, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x43,
+	0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54,
+	0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x09, 0x65, 0x6e, 0x75, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65,
+	0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x08, 0x65, 0x6e,
+	0x75, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f,
+	0x52, 0x07, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x43, 0x0a, 0x09, 0x65, 0x78, 0x74,
+	0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46,
+	0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72,
+	0x6f, 0x74, 0x6f, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x36,
+	0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x49, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+	0x5f, 0x63, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x1f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66,
+	0x6f, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66,
+	0x6f, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x79, 0x6e, 0x74, 0x61, 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x06, 0x73, 0x79, 0x6e, 0x74, 0x61, 0x78, 0x22, 0xb9, 0x06, 0x0a, 0x0f, 0x44, 0x65,
+	0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a,
+	0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d,
+	0x65, 0x12, 0x3b, 0x0a, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b,
+	0x32, 0x25, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x05, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x43,
+	0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x03, 0x28,
+	0x0b, 0x32, 0x25, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73,
+	0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0b, 0x6e, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x79,
+	0x70, 0x65, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x0a, 0x6e, 0x65, 0x73, 0x74,
+	0x65, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x09, 0x65, 0x6e, 0x75, 0x6d, 0x5f, 0x74,
+	0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52,
+	0x08, 0x65, 0x6e, 0x75, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x12, 0x58, 0x0a, 0x0f, 0x65, 0x78, 0x74,
+	0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x05, 0x20, 0x03,
+	0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50,
+	0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x61,
+	0x6e, 0x67, 0x65, 0x52, 0x0e, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x61,
+	0x6e, 0x67, 0x65, 0x12, 0x44, 0x0a, 0x0a, 0x6f, 0x6e, 0x65, 0x6f, 0x66, 0x5f, 0x64, 0x65, 0x63,
+	0x6c, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4f, 0x6e, 0x65, 0x6f, 0x66, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52, 0x09,
+	0x6f, 0x6e, 0x65, 0x6f, 0x66, 0x44, 0x65, 0x63, 0x6c, 0x12, 0x39, 0x0a, 0x07, 0x6f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x73,
+	0x73, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x12, 0x55, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64,
+	0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52,
+	0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x0d, 0x72, 0x65,
+	0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72,
+	0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x03,
+	0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65,
+	0x1a, 0x7a, 0x0a, 0x0e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e,
+	0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x05, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x12, 0x40, 0x0a, 0x07, 0x6f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x78,
+	0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x37, 0x0a, 0x0d,
+	0x52, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a,
+	0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x74,
+	0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05,
+	0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x7c, 0x0a, 0x15, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69,
+	0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x58,
+	0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f,
+	0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74,
+	0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80,
+	0x80, 0x80, 0x02, 0x22, 0xc1, 0x06, 0x0a, 0x14, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a, 0x04,
+	0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+	0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05,
+	0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x41, 0x0a, 0x05, 0x6c, 0x61, 0x62, 0x65,
+	0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2b, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c,
+	0x61, 0x62, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x3e, 0x0a, 0x04, 0x74,
+	0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c,
+	0x64, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f,
+	0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74,
+	0x79, 0x70, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
+	0x74, 0x79, 0x70, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x78, 0x74, 0x65,
+	0x6e, 0x64, 0x65, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x78, 0x74, 0x65,
+	0x6e, 0x64, 0x65, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x65, 0x66,
+	0x61, 0x75, 0x6c, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6f, 0x6e, 0x65,
+	0x6f, 0x66, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a,
+	0x6f, 0x6e, 0x65, 0x6f, 0x66, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1b, 0x0a, 0x09, 0x6a, 0x73,
+	0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6a,
+	0x73, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
+	0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x12, 0x27, 0x0a, 0x0f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x61, 0x6c, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x33, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x22, 0xb6, 0x02, 0x0a, 0x04, 0x54, 0x79,
+	0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c,
+	0x45, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x4c, 0x4f, 0x41,
+	0x54, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x36,
+	0x34, 0x10, 0x03, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x49, 0x4e, 0x54,
+	0x36, 0x34, 0x10, 0x04, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54,
+	0x33, 0x32, 0x10, 0x05, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x49, 0x58,
+	0x45, 0x44, 0x36, 0x34, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46,
+	0x49, 0x58, 0x45, 0x44, 0x33, 0x32, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50, 0x45,
+	0x5f, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f,
+	0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x09, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45,
+	0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x0a, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59, 0x50, 0x45,
+	0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x0b, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59,
+	0x50, 0x45, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x0c, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59,
+	0x50, 0x45, 0x5f, 0x55, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x54,
+	0x59, 0x50, 0x45, 0x5f, 0x45, 0x4e, 0x55, 0x4d, 0x10, 0x0e, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x59,
+	0x50, 0x45, 0x5f, 0x53, 0x46, 0x49, 0x58, 0x45, 0x44, 0x33, 0x32, 0x10, 0x0f, 0x12, 0x11, 0x0a,
+	0x0d, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x46, 0x49, 0x58, 0x45, 0x44, 0x36, 0x34, 0x10, 0x10,
+	0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10,
+	0x11, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x49, 0x4e, 0x54, 0x36, 0x34,
+	0x10, 0x12, 0x22, 0x43, 0x0a, 0x05, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x12, 0x0a, 0x0e, 0x4c,
+	0x41, 0x42, 0x45, 0x4c, 0x5f, 0x4f, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x10, 0x01, 0x12,
+	0x12, 0x0a, 0x0e, 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45,
+	0x44, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x50,
+	0x45, 0x41, 0x54, 0x45, 0x44, 0x10, 0x03, 0x22, 0x63, 0x0a, 0x14, 0x4f, 0x6e, 0x65, 0x6f, 0x66,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12,
+	0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,
+	0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4f, 0x6e, 0x65, 0x6f, 0x66, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe3, 0x02, 0x0a,
+	0x13, 0x45, 0x6e, 0x75, 0x6d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50,
+	0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61,
+	0x6c, 0x75, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f,
+	0x74, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x6f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75,
+	0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x12, 0x5d, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x72, 0x61,
+	0x6e, 0x67, 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+	0x45, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67,
+	0x65, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65,
+	0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d,
+	0x65, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65,
+	0x64, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x3b, 0x0a, 0x11, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x73,
+	0x65, 0x72, 0x76, 0x65, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74,
+	0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74,
+	0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x65,
+	0x6e, 0x64, 0x22, 0x83, 0x01, 0x0a, 0x18, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12,
+	0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,
+	0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x05, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x3b, 0x0a, 0x07, 0x6f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45,
+	0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52,
+	0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x72,
+	0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72,
+	0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f,
+	0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x52,
+	0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x39, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69,
+	0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x22, 0x89, 0x02, 0x0a, 0x15, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x0a, 0x04,
+	0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+	0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12,
+	0x1f, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65,
+	0x12, 0x38, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x0a, 0x10, 0x63, 0x6c,
+	0x69, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x05,
+	0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x0f, 0x63, 0x6c, 0x69,
+	0x65, 0x6e, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x12, 0x30, 0x0a, 0x10,
+	0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
+	0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x0f, 0x73,
+	0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x22, 0x91,
+	0x09, 0x0a, 0x0b, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21,
+	0x0a, 0x0c, 0x6a, 0x61, 0x76, 0x61, 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6a, 0x61, 0x76, 0x61, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67,
+	0x65, 0x12, 0x30, 0x0a, 0x14, 0x6a, 0x61, 0x76, 0x61, 0x5f, 0x6f, 0x75, 0x74, 0x65, 0x72, 0x5f,
+	0x63, 0x6c, 0x61, 0x73, 0x73, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x12, 0x6a, 0x61, 0x76, 0x61, 0x4f, 0x75, 0x74, 0x65, 0x72, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x6e,
+	0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x13, 0x6a, 0x61, 0x76, 0x61, 0x5f, 0x6d, 0x75, 0x6c, 0x74,
+	0x69, 0x70, 0x6c, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08,
+	0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x11, 0x6a, 0x61, 0x76, 0x61, 0x4d, 0x75, 0x6c,
+	0x74, 0x69, 0x70, 0x6c, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x44, 0x0a, 0x1d, 0x6a, 0x61,
+	0x76, 0x61, 0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x5f, 0x65, 0x71, 0x75, 0x61,
+	0x6c, 0x73, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x14, 0x20, 0x01, 0x28,
+	0x08, 0x42, 0x02, 0x18, 0x01, 0x52, 0x19, 0x6a, 0x61, 0x76, 0x61, 0x47, 0x65, 0x6e, 0x65, 0x72,
+	0x61, 0x74, 0x65, 0x45, 0x71, 0x75, 0x61, 0x6c, 0x73, 0x41, 0x6e, 0x64, 0x48, 0x61, 0x73, 0x68,
+	0x12, 0x3a, 0x0a, 0x16, 0x6a, 0x61, 0x76, 0x61, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f,
+	0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x75, 0x74, 0x66, 0x38, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x08,
+	0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x13, 0x6a, 0x61, 0x76, 0x61, 0x53, 0x74, 0x72,
+	0x69, 0x6e, 0x67, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x55, 0x74, 0x66, 0x38, 0x12, 0x53, 0x0a, 0x0c,
+	0x6f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x5f, 0x66, 0x6f, 0x72, 0x18, 0x09, 0x20, 0x01,
+	0x28, 0x0e, 0x32, 0x29, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x3a, 0x05, 0x53,
+	0x50, 0x45, 0x45, 0x44, 0x52, 0x0b, 0x6f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x46, 0x6f,
+	0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x6f, 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x18,
+	0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x6f, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65,
+	0x12, 0x35, 0x0a, 0x13, 0x63, 0x63, 0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x5f, 0x73,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66,
+	0x61, 0x6c, 0x73, 0x65, 0x52, 0x11, 0x63, 0x63, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x53,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x39, 0x0a, 0x15, 0x6a, 0x61, 0x76, 0x61, 0x5f,
+	0x67, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73,
+	0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x13, 0x6a,
+	0x61, 0x76, 0x61, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x73, 0x12, 0x35, 0x0a, 0x13, 0x70, 0x79, 0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63,
+	0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x3a,
+	0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x11, 0x70, 0x79, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69,
+	0x63, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x37, 0x0a, 0x14, 0x70, 0x68, 0x70,
+	0x5f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
+	0x73, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x12,
+	0x70, 0x68, 0x70, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64,
+	0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x0a, 0x64,
+	0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x2e, 0x0a, 0x10, 0x63, 0x63, 0x5f,
+	0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x65, 0x6e, 0x61, 0x73, 0x18, 0x1f, 0x20,
+	0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0e, 0x63, 0x63, 0x45, 0x6e, 0x61,
+	0x62, 0x6c, 0x65, 0x41, 0x72, 0x65, 0x6e, 0x61, 0x73, 0x12, 0x2a, 0x0a, 0x11, 0x6f, 0x62, 0x6a,
+	0x63, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x24,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6f, 0x62, 0x6a, 0x63, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x50,
+	0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x73, 0x68, 0x61, 0x72, 0x70, 0x5f,
+	0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x25, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0f, 0x63, 0x73, 0x68, 0x61, 0x72, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65,
+	0x12, 0x21, 0x0a, 0x0c, 0x73, 0x77, 0x69, 0x66, 0x74, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78,
+	0x18, 0x27, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x77, 0x69, 0x66, 0x74, 0x50, 0x72, 0x65,
+	0x66, 0x69, 0x78, 0x12, 0x28, 0x0a, 0x10, 0x70, 0x68, 0x70, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73,
+	0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x28, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x70,
+	0x68, 0x70, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x23, 0x0a,
+	0x0d, 0x70, 0x68, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x29,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x68, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61,
+	0x63, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x70, 0x68, 0x70, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61,
+	0x74, 0x61, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x2c, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x14, 0x70, 0x68, 0x70, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x4e,
+	0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x75, 0x62, 0x79,
+	0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x18, 0x2d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
+	0x72, 0x75, 0x62, 0x79, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x12, 0x58, 0x0a, 0x14, 0x75,
+	0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x6e, 0x69,
+	0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a,
+	0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x50, 0x45, 0x45, 0x44, 0x10, 0x01,
+	0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0x02, 0x12,
+	0x10, 0x0a, 0x0c, 0x4c, 0x49, 0x54, 0x45, 0x5f, 0x52, 0x55, 0x4e, 0x54, 0x49, 0x4d, 0x45, 0x10,
+	0x03, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80, 0x80, 0x80, 0x02, 0x4a, 0x04, 0x08, 0x26,
+	0x10, 0x27, 0x22, 0xd1, 0x02, 0x0a, 0x0e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3c, 0x0a, 0x17, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+	0x5f, 0x73, 0x65, 0x74, 0x5f, 0x77, 0x69, 0x72, 0x65, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x14, 0x6d,
+	0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x53, 0x65, 0x74, 0x57, 0x69, 0x72, 0x65, 0x46, 0x6f, 0x72,
+	0x6d, 0x61, 0x74, 0x12, 0x4c, 0x0a, 0x1f, 0x6e, 0x6f, 0x5f, 0x73, 0x74, 0x61, 0x6e, 0x64, 0x61,
+	0x72, 0x64, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x63,
+	0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61,
+	0x6c, 0x73, 0x65, 0x52, 0x1c, 0x6e, 0x6f, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x6f,
+	0x72, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x0a, 0x64, 0x65,
+	0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x61, 0x70, 0x5f,
+	0x65, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6d, 0x61, 0x70,
+	0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x58, 0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72,
+	0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72,
+	0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e,
+	0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x2a,
+	0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80, 0x80, 0x80, 0x02, 0x4a, 0x04, 0x08, 0x08, 0x10, 0x09,
+	0x4a, 0x04, 0x08, 0x09, 0x10, 0x0a, 0x22, 0xe2, 0x03, 0x0a, 0x0c, 0x46, 0x69, 0x65, 0x6c, 0x64,
+	0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x41, 0x0a, 0x05, 0x63, 0x74, 0x79, 0x70, 0x65,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x23, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x43, 0x54, 0x79, 0x70, 0x65, 0x3a, 0x06, 0x53, 0x54, 0x52,
+	0x49, 0x4e, 0x47, 0x52, 0x05, 0x63, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x61,
+	0x63, 0x6b, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x70, 0x61, 0x63, 0x6b,
+	0x65, 0x64, 0x12, 0x47, 0x0a, 0x06, 0x6a, 0x73, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01,
+	0x28, 0x0e, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x2e, 0x4a, 0x53, 0x54, 0x79, 0x70, 0x65, 0x3a, 0x09, 0x4a, 0x53, 0x5f, 0x4e, 0x4f, 0x52,
+	0x4d, 0x41, 0x4c, 0x52, 0x06, 0x6a, 0x73, 0x74, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x04, 0x6c,
+	0x61, 0x7a, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65,
+	0x52, 0x04, 0x6c, 0x61, 0x7a, 0x79, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63,
+	0x61, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73,
+	0x65, 0x52, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x19, 0x0a,
+	0x04, 0x77, 0x65, 0x61, 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c,
+	0x73, 0x65, 0x52, 0x04, 0x77, 0x65, 0x61, 0x6b, 0x12, 0x58, 0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e,
+	0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65,
+	0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75,
+	0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x22, 0x2f, 0x0a, 0x05, 0x43, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53,
+	0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x43, 0x4f, 0x52, 0x44, 0x10,
+	0x01, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x50, 0x49, 0x45, 0x43,
+	0x45, 0x10, 0x02, 0x22, 0x35, 0x0a, 0x06, 0x4a, 0x53, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0d, 0x0a,
+	0x09, 0x4a, 0x53, 0x5f, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09,
+	0x4a, 0x53, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x4a,
+	0x53, 0x5f, 0x4e, 0x55, 0x4d, 0x42, 0x45, 0x52, 0x10, 0x02, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10,
+	0x80, 0x80, 0x80, 0x80, 0x02, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x22, 0x73, 0x0a, 0x0c, 0x4f,
+	0x6e, 0x65, 0x6f, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x58, 0x0a, 0x14, 0x75,
+	0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x6e, 0x69,
+	0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80, 0x80, 0x80, 0x02,
+	0x22, 0xc0, 0x01, 0x0a, 0x0b, 0x45, 0x6e, 0x75, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x41, 0x6c, 0x69, 0x61,
+	0x73, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x0a, 0x64, 0x65,
+	0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x58, 0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e,
+	0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65,
+	0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75,
+	0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80, 0x80, 0x80, 0x02, 0x4a, 0x04, 0x08,
+	0x05, 0x10, 0x06, 0x22, 0x9e, 0x01, 0x0a, 0x10, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75,
+	0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72,
+	0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61,
+	0x6c, 0x73, 0x65, 0x52, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12,
+	0x58, 0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64,
+	0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24,
+	0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
+	0x2e, 0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65,
+	0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80,
+	0x80, 0x80, 0x80, 0x02, 0x22, 0x9c, 0x01, 0x0a, 0x0e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
+	0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65,
+	0x63, 0x61, 0x74, 0x65, 0x64, 0x18, 0x21, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c,
+	0x73, 0x65, 0x52, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x58,
+	0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x5f,
+	0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74,
+	0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x2a, 0x09, 0x08, 0xe8, 0x07, 0x10, 0x80, 0x80,
+	0x80, 0x80, 0x02, 0x22, 0xe0, 0x02, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61,
+	0x74, 0x65, 0x64, 0x18, 0x21, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65,
+	0x52, 0x0a, 0x64, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x12, 0x71, 0x0a, 0x11,
+	0x69, 0x64, 0x65, 0x6d, 0x70, 0x6f, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x5f, 0x6c, 0x65, 0x76, 0x65,
+	0x6c, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64,
+	0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x49, 0x64, 0x65, 0x6d, 0x70, 0x6f, 0x74, 0x65,
+	0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x3a, 0x13, 0x49, 0x44, 0x45, 0x4d, 0x50, 0x4f,
+	0x54, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x52, 0x10, 0x69,
+	0x64, 0x65, 0x6d, 0x70, 0x6f, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12,
+	0x58, 0x0a, 0x14, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64,
+	0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24,
+	0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
+	0x2e, 0x55, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x75, 0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65,
+	0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x50, 0x0a, 0x10, 0x49, 0x64, 0x65,
+	0x6d, 0x70, 0x6f, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x17, 0x0a,
+	0x13, 0x49, 0x44, 0x45, 0x4d, 0x50, 0x4f, 0x54, 0x45, 0x4e, 0x43, 0x59, 0x5f, 0x55, 0x4e, 0x4b,
+	0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x4f, 0x5f, 0x53, 0x49, 0x44,
+	0x45, 0x5f, 0x45, 0x46, 0x46, 0x45, 0x43, 0x54, 0x53, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49,
+	0x44, 0x45, 0x4d, 0x50, 0x4f, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x02, 0x2a, 0x09, 0x08, 0xe8, 0x07,
+	0x10, 0x80, 0x80, 0x80, 0x80, 0x02, 0x22, 0x9a, 0x03, 0x0a, 0x13, 0x55, 0x6e, 0x69, 0x6e, 0x74,
+	0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41,
+	0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55,
+	0x6e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x65, 0x74, 0x65, 0x64, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x50, 0x61, 0x72, 0x74, 0x52, 0x04, 0x6e, 0x61, 0x6d,
+	0x65, 0x12, 0x29, 0x0a, 0x10, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x64, 0x65,
+	0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2c, 0x0a, 0x12,
+	0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c,
+	0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69,
+	0x76, 0x65, 0x49, 0x6e, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x6e, 0x65,
+	0x67, 0x61, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65,
+	0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x6e, 0x65, 0x67, 0x61, 0x74, 0x69, 0x76, 0x65,
+	0x49, 0x6e, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62,
+	0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0b,
+	0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73,
+	0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28,
+	0x0c, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x27,
+	0x0a, 0x0f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
+	0x74, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x4a, 0x0a, 0x08, 0x4e, 0x61, 0x6d, 0x65, 0x50,
+	0x61, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74,
+	0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x61, 0x6d, 0x65, 0x50, 0x61, 0x72, 0x74,
+	0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e,
+	0x18, 0x02, 0x20, 0x02, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73,
+	0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x0e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f,
+	0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x44, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69,
+	0x6f, 0x6e, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63,
+	0x65, 0x43, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69,
+	0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xce, 0x01, 0x0a,
+	0x08, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x04, 0x70, 0x61, 0x74,
+	0x68, 0x18, 0x01, 0x20, 0x03, 0x28, 0x05, 0x42, 0x02, 0x10, 0x01, 0x52, 0x04, 0x70, 0x61, 0x74,
+	0x68, 0x12, 0x16, 0x0a, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x05, 0x42,
+	0x02, 0x10, 0x01, 0x52, 0x04, 0x73, 0x70, 0x61, 0x6e, 0x12, 0x29, 0x0a, 0x10, 0x6c, 0x65, 0x61,
+	0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x65, 0x61, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x6d,
+	0x65, 0x6e, 0x74, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x69, 0x6c, 0x69, 0x6e, 0x67,
+	0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x10, 0x74, 0x72, 0x61, 0x69, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74,
+	0x73, 0x12, 0x3a, 0x0a, 0x19, 0x6c, 0x65, 0x61, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x65, 0x74,
+	0x61, 0x63, 0x68, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06,
+	0x20, 0x03, 0x28, 0x09, 0x52, 0x17, 0x6c, 0x65, 0x61, 0x64, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x74,
+	0x61, 0x63, 0x68, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xd1, 0x01,
+	0x0a, 0x11, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x64, 0x65, 0x49,
+	0x6e, 0x66, 0x6f, 0x12, 0x4d, 0x0a, 0x0a, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f,
+	0x6e, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61,
+	0x74, 0x65, 0x64, 0x43, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x41, 0x6e, 0x6e, 0x6f,
+	0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69,
+	0x6f, 0x6e, 0x1a, 0x6d, 0x0a, 0x0a, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x12, 0x16, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x03, 0x28, 0x05, 0x42, 0x02,
+	0x10, 0x01, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72,
+	0x63, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73,
+	0x6f, 0x75, 0x72, 0x63, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x65, 0x67,
+	0x69, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x62, 0x65, 0x67, 0x69, 0x6e, 0x12,
+	0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x65, 0x6e,
+	0x64, 0x42, 0x7e, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x42, 0x10, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69,
+	0x70, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x48, 0x01, 0x5a, 0x2d, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x64,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x70, 0x62, 0xf8, 0x01, 0x01, 0xa2, 0x02,
+	0x03, 0x47, 0x50, 0x42, 0xaa, 0x02, 0x1a, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x50, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x52, 0x65, 0x66, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
+	0x6e,
+}
+
+var (
+	file_google_protobuf_descriptor_proto_rawDescOnce sync.Once
+	file_google_protobuf_descriptor_proto_rawDescData = file_google_protobuf_descriptor_proto_rawDesc
+)
+
+func file_google_protobuf_descriptor_proto_rawDescGZIP() []byte {
+	file_google_protobuf_descriptor_proto_rawDescOnce.Do(func() {
+		file_google_protobuf_descriptor_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_protobuf_descriptor_proto_rawDescData)
+	})
+	return file_google_protobuf_descriptor_proto_rawDescData
+}
+
+var file_google_protobuf_descriptor_proto_enumTypes = make([]protoimpl.EnumInfo, 6)
+var file_google_protobuf_descriptor_proto_msgTypes = make([]protoimpl.MessageInfo, 27)
+var file_google_protobuf_descriptor_proto_goTypes = []interface{}{
+	(FieldDescriptorProto_Type)(0),                // 0: google.protobuf.FieldDescriptorProto.Type
+	(FieldDescriptorProto_Label)(0),               // 1: google.protobuf.FieldDescriptorProto.Label
+	(FileOptions_OptimizeMode)(0),                 // 2: google.protobuf.FileOptions.OptimizeMode
+	(FieldOptions_CType)(0),                       // 3: google.protobuf.FieldOptions.CType
+	(FieldOptions_JSType)(0),                      // 4: google.protobuf.FieldOptions.JSType
+	(MethodOptions_IdempotencyLevel)(0),           // 5: google.protobuf.MethodOptions.IdempotencyLevel
+	(*FileDescriptorSet)(nil),                     // 6: google.protobuf.FileDescriptorSet
+	(*FileDescriptorProto)(nil),                   // 7: google.protobuf.FileDescriptorProto
+	(*DescriptorProto)(nil),                       // 8: google.protobuf.DescriptorProto
+	(*ExtensionRangeOptions)(nil),                 // 9: google.protobuf.ExtensionRangeOptions
+	(*FieldDescriptorProto)(nil),                  // 10: google.protobuf.FieldDescriptorProto
+	(*OneofDescriptorProto)(nil),                  // 11: google.protobuf.OneofDescriptorProto
+	(*EnumDescriptorProto)(nil),                   // 12: google.protobuf.EnumDescriptorProto
+	(*EnumValueDescriptorProto)(nil),              // 13: google.protobuf.EnumValueDescriptorProto
+	(*ServiceDescriptorProto)(nil),                // 14: google.protobuf.ServiceDescriptorProto
+	(*MethodDescriptorProto)(nil),                 // 15: google.protobuf.MethodDescriptorProto
+	(*FileOptions)(nil),                           // 16: google.protobuf.FileOptions
+	(*MessageOptions)(nil),                        // 17: google.protobuf.MessageOptions
+	(*FieldOptions)(nil),                          // 18: google.protobuf.FieldOptions
+	(*OneofOptions)(nil),                          // 19: google.protobuf.OneofOptions
+	(*EnumOptions)(nil),                           // 20: google.protobuf.EnumOptions
+	(*EnumValueOptions)(nil),                      // 21: google.protobuf.EnumValueOptions
+	(*ServiceOptions)(nil),                        // 22: google.protobuf.ServiceOptions
+	(*MethodOptions)(nil),                         // 23: google.protobuf.MethodOptions
+	(*UninterpretedOption)(nil),                   // 24: google.protobuf.UninterpretedOption
+	(*SourceCodeInfo)(nil),                        // 25: google.protobuf.SourceCodeInfo
+	(*GeneratedCodeInfo)(nil),                     // 26: google.protobuf.GeneratedCodeInfo
+	(*DescriptorProto_ExtensionRange)(nil),        // 27: google.protobuf.DescriptorProto.ExtensionRange
+	(*DescriptorProto_ReservedRange)(nil),         // 28: google.protobuf.DescriptorProto.ReservedRange
+	(*EnumDescriptorProto_EnumReservedRange)(nil), // 29: google.protobuf.EnumDescriptorProto.EnumReservedRange
+	(*UninterpretedOption_NamePart)(nil),          // 30: google.protobuf.UninterpretedOption.NamePart
+	(*SourceCodeInfo_Location)(nil),               // 31: google.protobuf.SourceCodeInfo.Location
+	(*GeneratedCodeInfo_Annotation)(nil),          // 32: google.protobuf.GeneratedCodeInfo.Annotation
+}
+var file_google_protobuf_descriptor_proto_depIdxs = []int32{
+	7,  // 0: google.protobuf.FileDescriptorSet.file:type_name -> google.protobuf.FileDescriptorProto
+	8,  // 1: google.protobuf.FileDescriptorProto.message_type:type_name -> google.protobuf.DescriptorProto
+	12, // 2: google.protobuf.FileDescriptorProto.enum_type:type_name -> google.protobuf.EnumDescriptorProto
+	14, // 3: google.protobuf.FileDescriptorProto.service:type_name -> google.protobuf.ServiceDescriptorProto
+	10, // 4: google.protobuf.FileDescriptorProto.extension:type_name -> google.protobuf.FieldDescriptorProto
+	16, // 5: google.protobuf.FileDescriptorProto.options:type_name -> google.protobuf.FileOptions
+	25, // 6: google.protobuf.FileDescriptorProto.source_code_info:type_name -> google.protobuf.SourceCodeInfo
+	10, // 7: google.protobuf.DescriptorProto.field:type_name -> google.protobuf.FieldDescriptorProto
+	10, // 8: google.protobuf.DescriptorProto.extension:type_name -> google.protobuf.FieldDescriptorProto
+	8,  // 9: google.protobuf.DescriptorProto.nested_type:type_name -> google.protobuf.DescriptorProto
+	12, // 10: google.protobuf.DescriptorProto.enum_type:type_name -> google.protobuf.EnumDescriptorProto
+	27, // 11: google.protobuf.DescriptorProto.extension_range:type_name -> google.protobuf.DescriptorProto.ExtensionRange
+	11, // 12: google.protobuf.DescriptorProto.oneof_decl:type_name -> google.protobuf.OneofDescriptorProto
+	17, // 13: google.protobuf.DescriptorProto.options:type_name -> google.protobuf.MessageOptions
+	28, // 14: google.protobuf.DescriptorProto.reserved_range:type_name -> google.protobuf.DescriptorProto.ReservedRange
+	24, // 15: google.protobuf.ExtensionRangeOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	1,  // 16: google.protobuf.FieldDescriptorProto.label:type_name -> google.protobuf.FieldDescriptorProto.Label
+	0,  // 17: google.protobuf.FieldDescriptorProto.type:type_name -> google.protobuf.FieldDescriptorProto.Type
+	18, // 18: google.protobuf.FieldDescriptorProto.options:type_name -> google.protobuf.FieldOptions
+	19, // 19: google.protobuf.OneofDescriptorProto.options:type_name -> google.protobuf.OneofOptions
+	13, // 20: google.protobuf.EnumDescriptorProto.value:type_name -> google.protobuf.EnumValueDescriptorProto
+	20, // 21: google.protobuf.EnumDescriptorProto.options:type_name -> google.protobuf.EnumOptions
+	29, // 22: google.protobuf.EnumDescriptorProto.reserved_range:type_name -> google.protobuf.EnumDescriptorProto.EnumReservedRange
+	21, // 23: google.protobuf.EnumValueDescriptorProto.options:type_name -> google.protobuf.EnumValueOptions
+	15, // 24: google.protobuf.ServiceDescriptorProto.method:type_name -> google.protobuf.MethodDescriptorProto
+	22, // 25: google.protobuf.ServiceDescriptorProto.options:type_name -> google.protobuf.ServiceOptions
+	23, // 26: google.protobuf.MethodDescriptorProto.options:type_name -> google.protobuf.MethodOptions
+	2,  // 27: google.protobuf.FileOptions.optimize_for:type_name -> google.protobuf.FileOptions.OptimizeMode
+	24, // 28: google.protobuf.FileOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	24, // 29: google.protobuf.MessageOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	3,  // 30: google.protobuf.FieldOptions.ctype:type_name -> google.protobuf.FieldOptions.CType
+	4,  // 31: google.protobuf.FieldOptions.jstype:type_name -> google.protobuf.FieldOptions.JSType
+	24, // 32: google.protobuf.FieldOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	24, // 33: google.protobuf.OneofOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	24, // 34: google.protobuf.EnumOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	24, // 35: google.protobuf.EnumValueOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	24, // 36: google.protobuf.ServiceOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	5,  // 37: google.protobuf.MethodOptions.idempotency_level:type_name -> google.protobuf.MethodOptions.IdempotencyLevel
+	24, // 38: google.protobuf.MethodOptions.uninterpreted_option:type_name -> google.protobuf.UninterpretedOption
+	30, // 39: google.protobuf.UninterpretedOption.name:type_name -> google.protobuf.UninterpretedOption.NamePart
+	31, // 40: google.protobuf.SourceCodeInfo.location:type_name -> google.protobuf.SourceCodeInfo.Location
+	32, // 41: google.protobuf.GeneratedCodeInfo.annotation:type_name -> google.protobuf.GeneratedCodeInfo.Annotation
+	9,  // 42: google.protobuf.DescriptorProto.ExtensionRange.options:type_name -> google.protobuf.ExtensionRangeOptions
+	43, // [43:43] is the sub-list for method output_type
+	43, // [43:43] is the sub-list for method input_type
+	43, // [43:43] is the sub-list for extension type_name
+	43, // [43:43] is the sub-list for extension extendee
+	0,  // [0:43] is the sub-list for field type_name
+}
+
+func init() { file_google_protobuf_descriptor_proto_init() }
+func file_google_protobuf_descriptor_proto_init() {
+	if File_google_protobuf_descriptor_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_protobuf_descriptor_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FileDescriptorSet); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FileDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExtensionRangeOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FieldDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*OneofDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EnumDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EnumValueDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ServiceDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MethodDescriptorProto); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FileOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MessageOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FieldOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*OneofOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EnumOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EnumValueOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ServiceOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MethodOptions); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			case 3:
+				return &v.extensionFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*UninterpretedOption); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SourceCodeInfo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GeneratedCodeInfo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DescriptorProto_ExtensionRange); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DescriptorProto_ReservedRange); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EnumDescriptorProto_EnumReservedRange); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*UninterpretedOption_NamePart); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SourceCodeInfo_Location); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_google_protobuf_descriptor_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GeneratedCodeInfo_Annotation); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_protobuf_descriptor_proto_rawDesc,
+			NumEnums:      6,
+			NumMessages:   27,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_protobuf_descriptor_proto_goTypes,
+		DependencyIndexes: file_google_protobuf_descriptor_proto_depIdxs,
+		EnumInfos:         file_google_protobuf_descriptor_proto_enumTypes,
+		MessageInfos:      file_google_protobuf_descriptor_proto_msgTypes,
+	}.Build()
+	File_google_protobuf_descriptor_proto = out.File
+	file_google_protobuf_descriptor_proto_rawDesc = nil
+	file_google_protobuf_descriptor_proto_goTypes = nil
+	file_google_protobuf_descriptor_proto_depIdxs = nil
+}
diff --git a/vendor/google.golang.org/protobuf/types/known/anypb/any.pb.go b/vendor/google.golang.org/protobuf/types/known/anypb/any.pb.go
new file mode 100644
index 0000000..8c10797
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/types/known/anypb/any.pb.go
@@ -0,0 +1,498 @@
+// 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.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/any.proto
+
+// Package anypb contains generated types for google/protobuf/any.proto.
+//
+// The Any message is a dynamic representation of any other message value.
+// It is functionally a tuple of the full name of the remote message type and
+// the serialized bytes of the remote message value.
+//
+//
+// Constructing an Any
+//
+// An Any message containing another message value is constructed using New:
+//
+//	any, err := anypb.New(m)
+//	if err != nil {
+//		... // handle error
+//	}
+//	... // make use of any
+//
+//
+// Unmarshaling an Any
+//
+// With a populated Any message, the underlying message can be serialized into
+// a remote concrete message value in a few ways.
+//
+// If the exact concrete type is known, then a new (or pre-existing) instance
+// of that message can be passed to the UnmarshalTo method:
+//
+//	m := new(foopb.MyMessage)
+//	if err := any.UnmarshalTo(m); err != nil {
+//		... // handle error
+//	}
+//	... // make use of m
+//
+// If the exact concrete type is not known, then the UnmarshalNew method can be
+// used to unmarshal the contents into a new instance of the remote message type:
+//
+//	m, err := any.UnmarshalNew()
+//	if err != nil {
+//		... // handle error
+//	}
+//	... // make use of m
+//
+// UnmarshalNew uses the global type registry to resolve the message type and
+// construct a new instance of that message to unmarshal into. In order for a
+// message type to appear in the global registry, the Go type representing that
+// protobuf message type must be linked into the Go binary. For messages
+// generated by protoc-gen-go, this is achieved through an import of the
+// generated Go package representing a .proto file.
+//
+// A common pattern with UnmarshalNew is to use a type switch with the resulting
+// proto.Message value:
+//
+//	switch m := m.(type) {
+//	case *foopb.MyMessage:
+//		... // make use of m as a *foopb.MyMessage
+//	case *barpb.OtherMessage:
+//		... // make use of m as a *barpb.OtherMessage
+//	case *bazpb.SomeMessage:
+//		... // make use of m as a *bazpb.SomeMessage
+//	}
+//
+// This pattern ensures that the generated packages containing the message types
+// listed in the case clauses are linked into the Go binary and therefore also
+// registered in the global registry.
+//
+//
+// Type checking an Any
+//
+// In order to type check whether an Any message represents some other message,
+// then use the MessageIs method:
+//
+//	if any.MessageIs((*foopb.MyMessage)(nil)) {
+//		... // make use of any, knowing that it contains a foopb.MyMessage
+//	}
+//
+// The MessageIs method can also be used with an allocated instance of the target
+// message type if the intention is to unmarshal into it if the type matches:
+//
+//	m := new(foopb.MyMessage)
+//	if any.MessageIs(m) {
+//		if err := any.UnmarshalTo(m); err != nil {
+//			... // handle error
+//		}
+//		... // make use of m
+//	}
+//
+package anypb
+
+import (
+	proto "google.golang.org/protobuf/proto"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoregistry "google.golang.org/protobuf/reflect/protoregistry"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	strings "strings"
+	sync "sync"
+)
+
+// `Any` contains an arbitrary serialized protocol buffer message along with a
+// URL that describes the type of the serialized message.
+//
+// Protobuf library provides support to pack/unpack Any values in the form
+// of utility functions or additional generated methods of the Any type.
+//
+// Example 1: Pack and unpack a message in C++.
+//
+//     Foo foo = ...;
+//     Any any;
+//     any.PackFrom(foo);
+//     ...
+//     if (any.UnpackTo(&foo)) {
+//       ...
+//     }
+//
+// Example 2: Pack and unpack a message in Java.
+//
+//     Foo foo = ...;
+//     Any any = Any.pack(foo);
+//     ...
+//     if (any.is(Foo.class)) {
+//       foo = any.unpack(Foo.class);
+//     }
+//
+//  Example 3: Pack and unpack a message in Python.
+//
+//     foo = Foo(...)
+//     any = Any()
+//     any.Pack(foo)
+//     ...
+//     if any.Is(Foo.DESCRIPTOR):
+//       any.Unpack(foo)
+//       ...
+//
+//  Example 4: Pack and unpack a message in Go
+//
+//      foo := &pb.Foo{...}
+//      any, err := anypb.New(foo)
+//      if err != nil {
+//        ...
+//      }
+//      ...
+//      foo := &pb.Foo{}
+//      if err := any.UnmarshalTo(foo); err != nil {
+//        ...
+//      }
+//
+// The pack methods provided by protobuf library will by default use
+// 'type.googleapis.com/full.type.name' as the type URL and the unpack
+// methods only use the fully qualified type name after the last '/'
+// in the type URL, for example "foo.bar.com/x/y.z" will yield type
+// name "y.z".
+//
+//
+// JSON
+// ====
+// The JSON representation of an `Any` value uses the regular
+// representation of the deserialized, embedded message, with an
+// additional field `@type` which contains the type URL. Example:
+//
+//     package google.profile;
+//     message Person {
+//       string first_name = 1;
+//       string last_name = 2;
+//     }
+//
+//     {
+//       "@type": "type.googleapis.com/google.profile.Person",
+//       "firstName": <string>,
+//       "lastName": <string>
+//     }
+//
+// If the embedded message type is well-known and has a custom JSON
+// representation, that representation will be embedded adding a field
+// `value` which holds the custom JSON in addition to the `@type`
+// field. Example (for message [google.protobuf.Duration][]):
+//
+//     {
+//       "@type": "type.googleapis.com/google.protobuf.Duration",
+//       "value": "1.212s"
+//     }
+//
+type Any struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// A URL/resource name that uniquely identifies the type of the serialized
+	// protocol buffer message. This string must contain at least
+	// one "/" character. The last segment of the URL's path must represent
+	// the fully qualified name of the type (as in
+	// `path/google.protobuf.Duration`). The name should be in a canonical form
+	// (e.g., leading "." is not accepted).
+	//
+	// In practice, teams usually precompile into the binary all types that they
+	// expect it to use in the context of Any. However, for URLs which use the
+	// scheme `http`, `https`, or no scheme, one can optionally set up a type
+	// server that maps type URLs to message definitions as follows:
+	//
+	// * If no scheme is provided, `https` is assumed.
+	// * An HTTP GET on the URL must yield a [google.protobuf.Type][]
+	//   value in binary format, or produce an error.
+	// * Applications are allowed to cache lookup results based on the
+	//   URL, or have them precompiled into a binary to avoid any
+	//   lookup. Therefore, binary compatibility needs to be preserved
+	//   on changes to types. (Use versioned type names to manage
+	//   breaking changes.)
+	//
+	// Note: this functionality is not currently available in the official
+	// protobuf release, and it is not used for type URLs beginning with
+	// type.googleapis.com.
+	//
+	// Schemes other than `http`, `https` (or the empty scheme) might be
+	// used with implementation specific semantics.
+	//
+	TypeUrl string `protobuf:"bytes,1,opt,name=type_url,json=typeUrl,proto3" json:"type_url,omitempty"`
+	// Must be a valid serialized protocol buffer of the above specified type.
+	Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+}
+
+// New marshals src into a new Any instance.
+func New(src proto.Message) (*Any, error) {
+	dst := new(Any)
+	if err := dst.MarshalFrom(src); err != nil {
+		return nil, err
+	}
+	return dst, nil
+}
+
+// MarshalFrom marshals src into dst as the underlying message
+// using the provided marshal options.
+//
+// If no options are specified, call dst.MarshalFrom instead.
+func MarshalFrom(dst *Any, src proto.Message, opts proto.MarshalOptions) error {
+	const urlPrefix = "type.googleapis.com/"
+	if src == nil {
+		return protoimpl.X.NewError("invalid nil source message")
+	}
+	b, err := opts.Marshal(src)
+	if err != nil {
+		return err
+	}
+	dst.TypeUrl = urlPrefix + string(src.ProtoReflect().Descriptor().FullName())
+	dst.Value = b
+	return nil
+}
+
+// UnmarshalTo unmarshals the underlying message from src into dst
+// using the provided unmarshal options.
+// It reports an error if dst is not of the right message type.
+//
+// If no options are specified, call src.UnmarshalTo instead.
+func UnmarshalTo(src *Any, dst proto.Message, opts proto.UnmarshalOptions) error {
+	if src == nil {
+		return protoimpl.X.NewError("invalid nil source message")
+	}
+	if !src.MessageIs(dst) {
+		got := dst.ProtoReflect().Descriptor().FullName()
+		want := src.MessageName()
+		return protoimpl.X.NewError("mismatched message type: got %q, want %q", got, want)
+	}
+	return opts.Unmarshal(src.GetValue(), dst)
+}
+
+// UnmarshalNew unmarshals the underlying message from src into dst,
+// which is newly created message using a type resolved from the type URL.
+// The message type is resolved according to opt.Resolver,
+// which should implement protoregistry.MessageTypeResolver.
+// It reports an error if the underlying message type could not be resolved.
+//
+// If no options are specified, call src.UnmarshalNew instead.
+func UnmarshalNew(src *Any, opts proto.UnmarshalOptions) (dst proto.Message, err error) {
+	if src.GetTypeUrl() == "" {
+		return nil, protoimpl.X.NewError("invalid empty type URL")
+	}
+	if opts.Resolver == nil {
+		opts.Resolver = protoregistry.GlobalTypes
+	}
+	r, ok := opts.Resolver.(protoregistry.MessageTypeResolver)
+	if !ok {
+		return nil, protoregistry.NotFound
+	}
+	mt, err := r.FindMessageByURL(src.GetTypeUrl())
+	if err != nil {
+		if err == protoregistry.NotFound {
+			return nil, err
+		}
+		return nil, protoimpl.X.NewError("could not resolve %q: %v", src.GetTypeUrl(), err)
+	}
+	dst = mt.New().Interface()
+	return dst, opts.Unmarshal(src.GetValue(), dst)
+}
+
+// MessageIs reports whether the underlying message is of the same type as m.
+func (x *Any) MessageIs(m proto.Message) bool {
+	if m == nil {
+		return false
+	}
+	url := x.GetTypeUrl()
+	name := string(m.ProtoReflect().Descriptor().FullName())
+	if !strings.HasSuffix(url, name) {
+		return false
+	}
+	return len(url) == len(name) || url[len(url)-len(name)-1] == '/'
+}
+
+// MessageName reports the full name of the underlying message,
+// returning an empty string if invalid.
+func (x *Any) MessageName() protoreflect.FullName {
+	url := x.GetTypeUrl()
+	name := protoreflect.FullName(url)
+	if i := strings.LastIndexByte(url, '/'); i >= 0 {
+		name = name[i+len("/"):]
+	}
+	if !name.IsValid() {
+		return ""
+	}
+	return name
+}
+
+// MarshalFrom marshals m into x as the underlying message.
+func (x *Any) MarshalFrom(m proto.Message) error {
+	return MarshalFrom(x, m, proto.MarshalOptions{})
+}
+
+// UnmarshalTo unmarshals the contents of the underlying message of x into m.
+// It resets m before performing the unmarshal operation.
+// It reports an error if m is not of the right message type.
+func (x *Any) UnmarshalTo(m proto.Message) error {
+	return UnmarshalTo(x, m, proto.UnmarshalOptions{})
+}
+
+// UnmarshalNew unmarshals the contents of the underlying message of x into
+// a newly allocated message of the specified type.
+// It reports an error if the underlying message type could not be resolved.
+func (x *Any) UnmarshalNew() (proto.Message, error) {
+	return UnmarshalNew(x, proto.UnmarshalOptions{})
+}
+
+func (x *Any) Reset() {
+	*x = Any{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_any_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Any) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Any) ProtoMessage() {}
+
+func (x *Any) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_any_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Any.ProtoReflect.Descriptor instead.
+func (*Any) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_any_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *Any) GetTypeUrl() string {
+	if x != nil {
+		return x.TypeUrl
+	}
+	return ""
+}
+
+func (x *Any) GetValue() []byte {
+	if x != nil {
+		return x.Value
+	}
+	return nil
+}
+
+var File_google_protobuf_any_proto protoreflect.FileDescriptor
+
+var file_google_protobuf_any_proto_rawDesc = []byte{
+	0x0a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0f, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x22, 0x36, 0x0a, 0x03,
+	0x41, 0x6e, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x75, 0x72, 0x6c, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x14,
+	0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x42, 0x76, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x42, 0x08, 0x41, 0x6e, 0x79,
+	0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x2f,
+	0x61, 0x6e, 0x79, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x47, 0x50, 0x42, 0xaa, 0x02, 0x1e, 0x47, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x57, 0x65,
+	0x6c, 0x6c, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x73, 0x62, 0x06, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_google_protobuf_any_proto_rawDescOnce sync.Once
+	file_google_protobuf_any_proto_rawDescData = file_google_protobuf_any_proto_rawDesc
+)
+
+func file_google_protobuf_any_proto_rawDescGZIP() []byte {
+	file_google_protobuf_any_proto_rawDescOnce.Do(func() {
+		file_google_protobuf_any_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_protobuf_any_proto_rawDescData)
+	})
+	return file_google_protobuf_any_proto_rawDescData
+}
+
+var file_google_protobuf_any_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_google_protobuf_any_proto_goTypes = []interface{}{
+	(*Any)(nil), // 0: google.protobuf.Any
+}
+var file_google_protobuf_any_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_google_protobuf_any_proto_init() }
+func file_google_protobuf_any_proto_init() {
+	if File_google_protobuf_any_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_protobuf_any_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Any); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_protobuf_any_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_protobuf_any_proto_goTypes,
+		DependencyIndexes: file_google_protobuf_any_proto_depIdxs,
+		MessageInfos:      file_google_protobuf_any_proto_msgTypes,
+	}.Build()
+	File_google_protobuf_any_proto = out.File
+	file_google_protobuf_any_proto_rawDesc = nil
+	file_google_protobuf_any_proto_goTypes = nil
+	file_google_protobuf_any_proto_depIdxs = nil
+}
diff --git a/vendor/google.golang.org/protobuf/types/known/durationpb/duration.pb.go b/vendor/google.golang.org/protobuf/types/known/durationpb/duration.pb.go
new file mode 100644
index 0000000..a583ca2
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/types/known/durationpb/duration.pb.go
@@ -0,0 +1,379 @@
+// 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.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/duration.proto
+
+// Package durationpb contains generated types for google/protobuf/duration.proto.
+//
+// The Duration message represents a signed span of time.
+//
+//
+// Conversion to a Go Duration
+//
+// The AsDuration method can be used to convert a Duration message to a
+// standard Go time.Duration value:
+//
+//	d := dur.AsDuration()
+//	... // make use of d as a time.Duration
+//
+// Converting to a time.Duration is a common operation so that the extensive
+// set of time-based operations provided by the time package can be leveraged.
+// See https://golang.org/pkg/time for more information.
+//
+// The AsDuration method performs the conversion on a best-effort basis.
+// Durations with denormal values (e.g., nanoseconds beyond -99999999 and
+// +99999999, inclusive; or seconds and nanoseconds with opposite signs)
+// are normalized during the conversion to a time.Duration. To manually check for
+// invalid Duration per the documented limitations in duration.proto,
+// additionally call the CheckValid method:
+//
+//	if err := dur.CheckValid(); err != nil {
+//		... // handle error
+//	}
+//
+// Note that the documented limitations in duration.proto does not protect a
+// Duration from overflowing the representable range of a time.Duration in Go.
+// The AsDuration method uses saturation arithmetic such that an overflow clamps
+// the resulting value to the closest representable value (e.g., math.MaxInt64
+// for positive overflow and math.MinInt64 for negative overflow).
+//
+//
+// Conversion from a Go Duration
+//
+// The durationpb.New function can be used to construct a Duration message
+// from a standard Go time.Duration value:
+//
+//	dur := durationpb.New(d)
+//	... // make use of d as a *durationpb.Duration
+//
+package durationpb
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	math "math"
+	reflect "reflect"
+	sync "sync"
+	time "time"
+)
+
+// A Duration represents a signed, fixed-length span of time represented
+// as a count of seconds and fractions of seconds at nanosecond
+// resolution. It is independent of any calendar and concepts like "day"
+// or "month". It is related to Timestamp in that the difference between
+// two Timestamp values is a Duration and it can be added or subtracted
+// from a Timestamp. Range is approximately +-10,000 years.
+//
+// # Examples
+//
+// Example 1: Compute Duration from two Timestamps in pseudo code.
+//
+//     Timestamp start = ...;
+//     Timestamp end = ...;
+//     Duration duration = ...;
+//
+//     duration.seconds = end.seconds - start.seconds;
+//     duration.nanos = end.nanos - start.nanos;
+//
+//     if (duration.seconds < 0 && duration.nanos > 0) {
+//       duration.seconds += 1;
+//       duration.nanos -= 1000000000;
+//     } else if (duration.seconds > 0 && duration.nanos < 0) {
+//       duration.seconds -= 1;
+//       duration.nanos += 1000000000;
+//     }
+//
+// Example 2: Compute Timestamp from Timestamp + Duration in pseudo code.
+//
+//     Timestamp start = ...;
+//     Duration duration = ...;
+//     Timestamp end = ...;
+//
+//     end.seconds = start.seconds + duration.seconds;
+//     end.nanos = start.nanos + duration.nanos;
+//
+//     if (end.nanos < 0) {
+//       end.seconds -= 1;
+//       end.nanos += 1000000000;
+//     } else if (end.nanos >= 1000000000) {
+//       end.seconds += 1;
+//       end.nanos -= 1000000000;
+//     }
+//
+// Example 3: Compute Duration from datetime.timedelta in Python.
+//
+//     td = datetime.timedelta(days=3, minutes=10)
+//     duration = Duration()
+//     duration.FromTimedelta(td)
+//
+// # JSON Mapping
+//
+// In JSON format, the Duration type is encoded as a string rather than an
+// object, where the string ends in the suffix "s" (indicating seconds) and
+// is preceded by the number of seconds, with nanoseconds expressed as
+// fractional seconds. For example, 3 seconds with 0 nanoseconds should be
+// encoded in JSON format as "3s", while 3 seconds and 1 nanosecond should
+// be expressed in JSON format as "3.000000001s", and 3 seconds and 1
+// microsecond should be expressed in JSON format as "3.000001s".
+//
+//
+type Duration struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Signed seconds of the span of time. Must be from -315,576,000,000
+	// to +315,576,000,000 inclusive. Note: these bounds are computed from:
+	// 60 sec/min * 60 min/hr * 24 hr/day * 365.25 days/year * 10000 years
+	Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"`
+	// Signed fractions of a second at nanosecond resolution of the span
+	// of time. Durations less than one second are represented with a 0
+	// `seconds` field and a positive or negative `nanos` field. For durations
+	// of one second or more, a non-zero value for the `nanos` field must be
+	// of the same sign as the `seconds` field. Must be from -999,999,999
+	// to +999,999,999 inclusive.
+	Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"`
+}
+
+// New constructs a new Duration from the provided time.Duration.
+func New(d time.Duration) *Duration {
+	nanos := d.Nanoseconds()
+	secs := nanos / 1e9
+	nanos -= secs * 1e9
+	return &Duration{Seconds: int64(secs), Nanos: int32(nanos)}
+}
+
+// AsDuration converts x to a time.Duration,
+// returning the closest duration value in the event of overflow.
+func (x *Duration) AsDuration() time.Duration {
+	secs := x.GetSeconds()
+	nanos := x.GetNanos()
+	d := time.Duration(secs) * time.Second
+	overflow := d/time.Second != time.Duration(secs)
+	d += time.Duration(nanos) * time.Nanosecond
+	overflow = overflow || (secs < 0 && nanos < 0 && d > 0)
+	overflow = overflow || (secs > 0 && nanos > 0 && d < 0)
+	if overflow {
+		switch {
+		case secs < 0:
+			return time.Duration(math.MinInt64)
+		case secs > 0:
+			return time.Duration(math.MaxInt64)
+		}
+	}
+	return d
+}
+
+// IsValid reports whether the duration is valid.
+// It is equivalent to CheckValid == nil.
+func (x *Duration) IsValid() bool {
+	return x.check() == 0
+}
+
+// CheckValid returns an error if the duration is invalid.
+// In particular, it checks whether the value is within the range of
+// -10000 years to +10000 years inclusive.
+// An error is reported for a nil Duration.
+func (x *Duration) CheckValid() error {
+	switch x.check() {
+	case invalidNil:
+		return protoimpl.X.NewError("invalid nil Duration")
+	case invalidUnderflow:
+		return protoimpl.X.NewError("duration (%v) exceeds -10000 years", x)
+	case invalidOverflow:
+		return protoimpl.X.NewError("duration (%v) exceeds +10000 years", x)
+	case invalidNanosRange:
+		return protoimpl.X.NewError("duration (%v) has out-of-range nanos", x)
+	case invalidNanosSign:
+		return protoimpl.X.NewError("duration (%v) has seconds and nanos with different signs", x)
+	default:
+		return nil
+	}
+}
+
+const (
+	_ = iota
+	invalidNil
+	invalidUnderflow
+	invalidOverflow
+	invalidNanosRange
+	invalidNanosSign
+)
+
+func (x *Duration) check() uint {
+	const absDuration = 315576000000 // 10000yr * 365.25day/yr * 24hr/day * 60min/hr * 60sec/min
+	secs := x.GetSeconds()
+	nanos := x.GetNanos()
+	switch {
+	case x == nil:
+		return invalidNil
+	case secs < -absDuration:
+		return invalidUnderflow
+	case secs > +absDuration:
+		return invalidOverflow
+	case nanos <= -1e9 || nanos >= +1e9:
+		return invalidNanosRange
+	case (secs > 0 && nanos < 0) || (secs < 0 && nanos > 0):
+		return invalidNanosSign
+	default:
+		return 0
+	}
+}
+
+func (x *Duration) Reset() {
+	*x = Duration{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_duration_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Duration) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Duration) ProtoMessage() {}
+
+func (x *Duration) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_duration_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Duration.ProtoReflect.Descriptor instead.
+func (*Duration) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_duration_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *Duration) GetSeconds() int64 {
+	if x != nil {
+		return x.Seconds
+	}
+	return 0
+}
+
+func (x *Duration) GetNanos() int32 {
+	if x != nil {
+		return x.Nanos
+	}
+	return 0
+}
+
+var File_google_protobuf_duration_proto protoreflect.FileDescriptor
+
+var file_google_protobuf_duration_proto_rawDesc = []byte{
+	0x0a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x12, 0x0f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x22, 0x3a, 0x0a, 0x08, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a,
+	0x07, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07,
+	0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6e, 0x6f, 0x73,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x42, 0x83, 0x01,
+	0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x42, 0x0d, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50,
+	0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x31, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67,
+	0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x2f, 0x64,
+	0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x70, 0x62, 0xf8, 0x01, 0x01, 0xa2, 0x02, 0x03, 0x47,
+	0x50, 0x42, 0xaa, 0x02, 0x1e, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x57, 0x65, 0x6c, 0x6c, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x54, 0x79,
+	0x70, 0x65, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_google_protobuf_duration_proto_rawDescOnce sync.Once
+	file_google_protobuf_duration_proto_rawDescData = file_google_protobuf_duration_proto_rawDesc
+)
+
+func file_google_protobuf_duration_proto_rawDescGZIP() []byte {
+	file_google_protobuf_duration_proto_rawDescOnce.Do(func() {
+		file_google_protobuf_duration_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_protobuf_duration_proto_rawDescData)
+	})
+	return file_google_protobuf_duration_proto_rawDescData
+}
+
+var file_google_protobuf_duration_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_google_protobuf_duration_proto_goTypes = []interface{}{
+	(*Duration)(nil), // 0: google.protobuf.Duration
+}
+var file_google_protobuf_duration_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_google_protobuf_duration_proto_init() }
+func file_google_protobuf_duration_proto_init() {
+	if File_google_protobuf_duration_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_protobuf_duration_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Duration); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_protobuf_duration_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_protobuf_duration_proto_goTypes,
+		DependencyIndexes: file_google_protobuf_duration_proto_depIdxs,
+		MessageInfos:      file_google_protobuf_duration_proto_msgTypes,
+	}.Build()
+	File_google_protobuf_duration_proto = out.File
+	file_google_protobuf_duration_proto_rawDesc = nil
+	file_google_protobuf_duration_proto_goTypes = nil
+	file_google_protobuf_duration_proto_depIdxs = nil
+}
diff --git a/vendor/google.golang.org/protobuf/types/known/emptypb/empty.pb.go b/vendor/google.golang.org/protobuf/types/known/emptypb/empty.pb.go
new file mode 100644
index 0000000..e7fcea3
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/types/known/emptypb/empty.pb.go
@@ -0,0 +1,168 @@
+// 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.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/empty.proto
+
+package emptypb
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
+)
+
+// A generic empty message that you can re-use to avoid defining duplicated
+// empty messages in your APIs. A typical example is to use it as the request
+// or the response type of an API method. For instance:
+//
+//     service Foo {
+//       rpc Bar(google.protobuf.Empty) returns (google.protobuf.Empty);
+//     }
+//
+// The JSON representation for `Empty` is empty JSON object `{}`.
+type Empty struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *Empty) Reset() {
+	*x = Empty{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_empty_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Empty) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Empty) ProtoMessage() {}
+
+func (x *Empty) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_empty_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Empty.ProtoReflect.Descriptor instead.
+func (*Empty) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_empty_proto_rawDescGZIP(), []int{0}
+}
+
+var File_google_protobuf_empty_proto protoreflect.FileDescriptor
+
+var file_google_protobuf_empty_proto_rawDesc = []byte{
+	0x0a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0f, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x22, 0x07,
+	0x0a, 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x7d, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x42, 0x0a,
+	0x45, 0x6d, 0x70, 0x74, 0x79, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2e, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x6b,
+	0x6e, 0x6f, 0x77, 0x6e, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x70, 0x62, 0xf8, 0x01, 0x01, 0xa2,
+	0x02, 0x03, 0x47, 0x50, 0x42, 0xaa, 0x02, 0x1e, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x50,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x57, 0x65, 0x6c, 0x6c, 0x4b, 0x6e, 0x6f, 0x77,
+	0x6e, 0x54, 0x79, 0x70, 0x65, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_google_protobuf_empty_proto_rawDescOnce sync.Once
+	file_google_protobuf_empty_proto_rawDescData = file_google_protobuf_empty_proto_rawDesc
+)
+
+func file_google_protobuf_empty_proto_rawDescGZIP() []byte {
+	file_google_protobuf_empty_proto_rawDescOnce.Do(func() {
+		file_google_protobuf_empty_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_protobuf_empty_proto_rawDescData)
+	})
+	return file_google_protobuf_empty_proto_rawDescData
+}
+
+var file_google_protobuf_empty_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_google_protobuf_empty_proto_goTypes = []interface{}{
+	(*Empty)(nil), // 0: google.protobuf.Empty
+}
+var file_google_protobuf_empty_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_google_protobuf_empty_proto_init() }
+func file_google_protobuf_empty_proto_init() {
+	if File_google_protobuf_empty_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_protobuf_empty_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Empty); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_protobuf_empty_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_protobuf_empty_proto_goTypes,
+		DependencyIndexes: file_google_protobuf_empty_proto_depIdxs,
+		MessageInfos:      file_google_protobuf_empty_proto_msgTypes,
+	}.Build()
+	File_google_protobuf_empty_proto = out.File
+	file_google_protobuf_empty_proto_rawDesc = nil
+	file_google_protobuf_empty_proto_goTypes = nil
+	file_google_protobuf_empty_proto_depIdxs = nil
+}
diff --git a/vendor/google.golang.org/protobuf/types/known/timestamppb/timestamp.pb.go b/vendor/google.golang.org/protobuf/types/known/timestamppb/timestamp.pb.go
new file mode 100644
index 0000000..c9ae921
--- /dev/null
+++ b/vendor/google.golang.org/protobuf/types/known/timestamppb/timestamp.pb.go
@@ -0,0 +1,390 @@
+// 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.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/timestamp.proto
+
+// Package timestamppb contains generated types for google/protobuf/timestamp.proto.
+//
+// The Timestamp message represents a timestamp,
+// an instant in time since the Unix epoch (January 1st, 1970).
+//
+//
+// Conversion to a Go Time
+//
+// The AsTime method can be used to convert a Timestamp message to a
+// standard Go time.Time value in UTC:
+//
+//	t := ts.AsTime()
+//	... // make use of t as a time.Time
+//
+// Converting to a time.Time is a common operation so that the extensive
+// set of time-based operations provided by the time package can be leveraged.
+// See https://golang.org/pkg/time for more information.
+//
+// The AsTime method performs the conversion on a best-effort basis. Timestamps
+// with denormal values (e.g., nanoseconds beyond 0 and 99999999, inclusive)
+// are normalized during the conversion to a time.Time. To manually check for
+// invalid Timestamps per the documented limitations in timestamp.proto,
+// additionally call the CheckValid method:
+//
+//	if err := ts.CheckValid(); err != nil {
+//		... // handle error
+//	}
+//
+//
+// Conversion from a Go Time
+//
+// The timestamppb.New function can be used to construct a Timestamp message
+// from a standard Go time.Time value:
+//
+//	ts := timestamppb.New(t)
+//	... // make use of ts as a *timestamppb.Timestamp
+//
+// In order to construct a Timestamp representing the current time, use Now:
+//
+//	ts := timestamppb.Now()
+//	... // make use of ts as a *timestamppb.Timestamp
+//
+package timestamppb
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
+	time "time"
+)
+
+// A Timestamp represents a point in time independent of any time zone or local
+// calendar, encoded as a count of seconds and fractions of seconds at
+// nanosecond resolution. The count is relative to an epoch at UTC midnight on
+// January 1, 1970, in the proleptic Gregorian calendar which extends the
+// Gregorian calendar backwards to year one.
+//
+// All minutes are 60 seconds long. Leap seconds are "smeared" so that no leap
+// second table is needed for interpretation, using a [24-hour linear
+// smear](https://developers.google.com/time/smear).
+//
+// The range is from 0001-01-01T00:00:00Z to 9999-12-31T23:59:59.999999999Z. By
+// restricting to that range, we ensure that we can convert to and from [RFC
+// 3339](https://www.ietf.org/rfc/rfc3339.txt) date strings.
+//
+// # Examples
+//
+// Example 1: Compute Timestamp from POSIX `time()`.
+//
+//     Timestamp timestamp;
+//     timestamp.set_seconds(time(NULL));
+//     timestamp.set_nanos(0);
+//
+// Example 2: Compute Timestamp from POSIX `gettimeofday()`.
+//
+//     struct timeval tv;
+//     gettimeofday(&tv, NULL);
+//
+//     Timestamp timestamp;
+//     timestamp.set_seconds(tv.tv_sec);
+//     timestamp.set_nanos(tv.tv_usec * 1000);
+//
+// Example 3: Compute Timestamp from Win32 `GetSystemTimeAsFileTime()`.
+//
+//     FILETIME ft;
+//     GetSystemTimeAsFileTime(&ft);
+//     UINT64 ticks = (((UINT64)ft.dwHighDateTime) << 32) | ft.dwLowDateTime;
+//
+//     // A Windows tick is 100 nanoseconds. Windows epoch 1601-01-01T00:00:00Z
+//     // is 11644473600 seconds before Unix epoch 1970-01-01T00:00:00Z.
+//     Timestamp timestamp;
+//     timestamp.set_seconds((INT64) ((ticks / 10000000) - 11644473600LL));
+//     timestamp.set_nanos((INT32) ((ticks % 10000000) * 100));
+//
+// Example 4: Compute Timestamp from Java `System.currentTimeMillis()`.
+//
+//     long millis = System.currentTimeMillis();
+//
+//     Timestamp timestamp = Timestamp.newBuilder().setSeconds(millis / 1000)
+//         .setNanos((int) ((millis % 1000) * 1000000)).build();
+//
+//
+// Example 5: Compute Timestamp from Java `Instant.now()`.
+//
+//     Instant now = Instant.now();
+//
+//     Timestamp timestamp =
+//         Timestamp.newBuilder().setSeconds(now.getEpochSecond())
+//             .setNanos(now.getNano()).build();
+//
+//
+// Example 6: Compute Timestamp from current time in Python.
+//
+//     timestamp = Timestamp()
+//     timestamp.GetCurrentTime()
+//
+// # JSON Mapping
+//
+// In JSON format, the Timestamp type is encoded as a string in the
+// [RFC 3339](https://www.ietf.org/rfc/rfc3339.txt) format. That is, the
+// format is "{year}-{month}-{day}T{hour}:{min}:{sec}[.{frac_sec}]Z"
+// where {year} is always expressed using four digits while {month}, {day},
+// {hour}, {min}, and {sec} are zero-padded to two digits each. The fractional
+// seconds, which can go up to 9 digits (i.e. up to 1 nanosecond resolution),
+// are optional. The "Z" suffix indicates the timezone ("UTC"); the timezone
+// is required. A proto3 JSON serializer should always use UTC (as indicated by
+// "Z") when printing the Timestamp type and a proto3 JSON parser should be
+// able to accept both UTC and other timezones (as indicated by an offset).
+//
+// For example, "2017-01-15T01:30:15.01Z" encodes 15.01 seconds past
+// 01:30 UTC on January 15, 2017.
+//
+// In JavaScript, one can convert a Date object to this format using the
+// standard
+// [toISOString()](https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Date/toISOString)
+// method. In Python, a standard `datetime.datetime` object can be converted
+// to this format using
+// [`strftime`](https://docs.python.org/2/library/time.html#time.strftime) with
+// the time format spec '%Y-%m-%dT%H:%M:%S.%fZ'. Likewise, in Java, one can use
+// the Joda Time's [`ISODateTimeFormat.dateTime()`](
+// http://www.joda.org/joda-time/apidocs/org/joda/time/format/ISODateTimeFormat.html#dateTime%2D%2D
+// ) to obtain a formatter capable of generating timestamps in this format.
+//
+//
+type Timestamp struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Represents seconds of UTC time since Unix epoch
+	// 1970-01-01T00:00:00Z. Must be from 0001-01-01T00:00:00Z to
+	// 9999-12-31T23:59:59Z inclusive.
+	Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"`
+	// Non-negative fractions of a second at nanosecond resolution. Negative
+	// second values with fractions must still have non-negative nanos values
+	// that count forward in time. Must be from 0 to 999,999,999
+	// inclusive.
+	Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"`
+}
+
+// Now constructs a new Timestamp from the current time.
+func Now() *Timestamp {
+	return New(time.Now())
+}
+
+// New constructs a new Timestamp from the provided time.Time.
+func New(t time.Time) *Timestamp {
+	return &Timestamp{Seconds: int64(t.Unix()), Nanos: int32(t.Nanosecond())}
+}
+
+// AsTime converts x to a time.Time.
+func (x *Timestamp) AsTime() time.Time {
+	return time.Unix(int64(x.GetSeconds()), int64(x.GetNanos())).UTC()
+}
+
+// IsValid reports whether the timestamp is valid.
+// It is equivalent to CheckValid == nil.
+func (x *Timestamp) IsValid() bool {
+	return x.check() == 0
+}
+
+// CheckValid returns an error if the timestamp is invalid.
+// In particular, it checks whether the value represents a date that is
+// in the range of 0001-01-01T00:00:00Z to 9999-12-31T23:59:59Z inclusive.
+// An error is reported for a nil Timestamp.
+func (x *Timestamp) CheckValid() error {
+	switch x.check() {
+	case invalidNil:
+		return protoimpl.X.NewError("invalid nil Timestamp")
+	case invalidUnderflow:
+		return protoimpl.X.NewError("timestamp (%v) before 0001-01-01", x)
+	case invalidOverflow:
+		return protoimpl.X.NewError("timestamp (%v) after 9999-12-31", x)
+	case invalidNanos:
+		return protoimpl.X.NewError("timestamp (%v) has out-of-range nanos", x)
+	default:
+		return nil
+	}
+}
+
+const (
+	_ = iota
+	invalidNil
+	invalidUnderflow
+	invalidOverflow
+	invalidNanos
+)
+
+func (x *Timestamp) check() uint {
+	const minTimestamp = -62135596800  // Seconds between 1970-01-01T00:00:00Z and 0001-01-01T00:00:00Z, inclusive
+	const maxTimestamp = +253402300799 // Seconds between 1970-01-01T00:00:00Z and 9999-12-31T23:59:59Z, inclusive
+	secs := x.GetSeconds()
+	nanos := x.GetNanos()
+	switch {
+	case x == nil:
+		return invalidNil
+	case secs < minTimestamp:
+		return invalidUnderflow
+	case secs > maxTimestamp:
+		return invalidOverflow
+	case nanos < 0 || nanos >= 1e9:
+		return invalidNanos
+	default:
+		return 0
+	}
+}
+
+func (x *Timestamp) Reset() {
+	*x = Timestamp{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_google_protobuf_timestamp_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Timestamp) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Timestamp) ProtoMessage() {}
+
+func (x *Timestamp) ProtoReflect() protoreflect.Message {
+	mi := &file_google_protobuf_timestamp_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Timestamp.ProtoReflect.Descriptor instead.
+func (*Timestamp) Descriptor() ([]byte, []int) {
+	return file_google_protobuf_timestamp_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *Timestamp) GetSeconds() int64 {
+	if x != nil {
+		return x.Seconds
+	}
+	return 0
+}
+
+func (x *Timestamp) GetNanos() int32 {
+	if x != nil {
+		return x.Nanos
+	}
+	return 0
+}
+
+var File_google_protobuf_timestamp_proto protoreflect.FileDescriptor
+
+var file_google_protobuf_timestamp_proto_rawDesc = []byte{
+	0x0a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+	0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x12, 0x0f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x22, 0x3b, 0x0a, 0x09, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12,
+	0x18, 0x0a, 0x07, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
+	0x52, 0x07, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6e,
+	0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x42,
+	0x85, 0x01, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x42, 0x0e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
+	0x6d, 0x70, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x32, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x6b, 0x6e, 0x6f, 0x77,
+	0x6e, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x70, 0x62, 0xf8, 0x01, 0x01,
+	0xa2, 0x02, 0x03, 0x47, 0x50, 0x42, 0xaa, 0x02, 0x1e, 0x47, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
+	0x50, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x57, 0x65, 0x6c, 0x6c, 0x4b, 0x6e, 0x6f,
+	0x77, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_google_protobuf_timestamp_proto_rawDescOnce sync.Once
+	file_google_protobuf_timestamp_proto_rawDescData = file_google_protobuf_timestamp_proto_rawDesc
+)
+
+func file_google_protobuf_timestamp_proto_rawDescGZIP() []byte {
+	file_google_protobuf_timestamp_proto_rawDescOnce.Do(func() {
+		file_google_protobuf_timestamp_proto_rawDescData = protoimpl.X.CompressGZIP(file_google_protobuf_timestamp_proto_rawDescData)
+	})
+	return file_google_protobuf_timestamp_proto_rawDescData
+}
+
+var file_google_protobuf_timestamp_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_google_protobuf_timestamp_proto_goTypes = []interface{}{
+	(*Timestamp)(nil), // 0: google.protobuf.Timestamp
+}
+var file_google_protobuf_timestamp_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_google_protobuf_timestamp_proto_init() }
+func file_google_protobuf_timestamp_proto_init() {
+	if File_google_protobuf_timestamp_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_google_protobuf_timestamp_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Timestamp); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_google_protobuf_timestamp_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_google_protobuf_timestamp_proto_goTypes,
+		DependencyIndexes: file_google_protobuf_timestamp_proto_depIdxs,
+		MessageInfos:      file_google_protobuf_timestamp_proto_msgTypes,
+	}.Build()
+	File_google_protobuf_timestamp_proto = out.File
+	file_google_protobuf_timestamp_proto_rawDesc = nil
+	file_google_protobuf_timestamp_proto_goTypes = nil
+	file_google_protobuf_timestamp_proto_depIdxs = nil
+}
diff --git a/vendor/gopkg.in/jcmturner/aescts.v1/.gitignore b/vendor/gopkg.in/jcmturner/aescts.v1/.gitignore
deleted file mode 100644
index a1338d6..0000000
--- a/vendor/gopkg.in/jcmturner/aescts.v1/.gitignore
+++ /dev/null
@@ -1,14 +0,0 @@
-# Binaries for programs and plugins
-*.exe
-*.dll
-*.so
-*.dylib
-
-# Test binary, build with `go test -c`
-*.test
-
-# Output of the go coverage tool, specifically when used with LiteIDE
-*.out
-
-# Project-local glide cache, RE: https://github.com/Masterminds/glide/issues/736
-.glide/
diff --git a/vendor/gopkg.in/jcmturner/aescts.v1/README.md b/vendor/gopkg.in/jcmturner/aescts.v1/README.md
deleted file mode 100644
index d1fddf3..0000000
--- a/vendor/gopkg.in/jcmturner/aescts.v1/README.md
+++ /dev/null
@@ -1,16 +0,0 @@
-# AES CBC Ciphertext Stealing
-[![GoDoc](https://godoc.org/gopkg.in/jcmturner/aescts.v1?status.svg)](https://godoc.org/gopkg.in/jcmturner/aescts.v1) [![Go Report Card](https://goreportcard.com/badge/gopkg.in/jcmturner/aescts.v1)](https://goreportcard.com/report/gopkg.in/jcmturner/aescts.v1)
-
-Encrypt and decrypt data using AES CBC Ciphertext stealing mode.
-
-Reference: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing
-
-To get the package, execute:
-```
-go get gopkg.in/jcmturner/aescts.v1
-```
-To import this package, add the following line to your code:
-```go
-import "gopkg.in/jcmturner/aescts.v1"
-
-```
\ No newline at end of file
diff --git a/vendor/gopkg.in/jcmturner/aescts.v1/aescts.go b/vendor/gopkg.in/jcmturner/aescts.v1/aescts.go
deleted file mode 100644
index 278713e..0000000
--- a/vendor/gopkg.in/jcmturner/aescts.v1/aescts.go
+++ /dev/null
@@ -1,186 +0,0 @@
-// Package aescts provides AES CBC CipherText Stealing encryption and decryption methods
-package aescts
-
-import (
-	"crypto/aes"
-	"crypto/cipher"
-	"errors"
-	"fmt"
-)
-
-// Encrypt the message with the key and the initial vector.
-// Returns: next iv, ciphertext bytes, error
-func Encrypt(key, iv, plaintext []byte) ([]byte, []byte, error) {
-	l := len(plaintext)
-
-	block, err := aes.NewCipher(key)
-	if err != nil {
-		return []byte{}, []byte{}, fmt.Errorf("Error creating cipher: %v", err)
-	}
-	mode := cipher.NewCBCEncrypter(block, iv)
-
-	m := make([]byte, len(plaintext))
-	copy(m, plaintext)
-
-	/*For consistency, ciphertext stealing is always used for the last two
-	blocks of the data to be encrypted, as in [RC5].  If the data length
-	is a multiple of the block size, this is equivalent to plain CBC mode
-	with the last two ciphertext blocks swapped.*/
-	/*The initial vector carried out from one encryption for use in a
-	subsequent encryption is the next-to-last block of the encryption
-	output; this is the encrypted form of the last plaintext block.*/
-	if l <= aes.BlockSize {
-		m, _ = zeroPad(m, aes.BlockSize)
-		mode.CryptBlocks(m, m)
-		return m, m, nil
-	}
-	if l%aes.BlockSize == 0 {
-		mode.CryptBlocks(m, m)
-		iv = m[len(m)-aes.BlockSize:]
-		rb, _ := swapLastTwoBlocks(m, aes.BlockSize)
-		return iv, rb, nil
-	}
-	m, _ = zeroPad(m, aes.BlockSize)
-	rb, pb, lb, err := tailBlocks(m, aes.BlockSize)
-	if err != nil {
-		return []byte{}, []byte{}, fmt.Errorf("Error tailing blocks: %v", err)
-	}
-	var ct []byte
-	if rb != nil {
-		// Encrpt all but the lats 2 blocks and update the rolling iv
-		mode.CryptBlocks(rb, rb)
-		iv = rb[len(rb)-aes.BlockSize:]
-		mode = cipher.NewCBCEncrypter(block, iv)
-		ct = append(ct, rb...)
-	}
-	mode.CryptBlocks(pb, pb)
-	mode = cipher.NewCBCEncrypter(block, pb)
-	mode.CryptBlocks(lb, lb)
-	// Cipher Text Stealing (CTS) - Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing
-	// Swap the last two cipher blocks
-	// Truncate the ciphertext to the length of the original plaintext
-	ct = append(ct, lb...)
-	ct = append(ct, pb...)
-	return lb, ct[:l], nil
-}
-
-// Decrypt the ciphertext with the key and the initial vector.
-func Decrypt(key, iv, ciphertext []byte) ([]byte, error) {
-	// Copy the cipher text as golang slices even when passed by value to this method can result in the backing arrays of the calling code value being updated.
-	ct := make([]byte, len(ciphertext))
-	copy(ct, ciphertext)
-	if len(ct) < aes.BlockSize {
-		return []byte{}, fmt.Errorf("Ciphertext is not large enough. It is less that one block size. Blocksize:%v; Ciphertext:%v", aes.BlockSize, len(ct))
-	}
-	// Configure the CBC
-	block, err := aes.NewCipher(key)
-	if err != nil {
-		return nil, fmt.Errorf("Error creating cipher: %v", err)
-	}
-	var mode cipher.BlockMode
-
-	//If ciphertext is multiple of blocksize we just need to swap back the last two blocks and then do CBC
-	//If the ciphertext is just one block we can't swap so we just decrypt
-	if len(ct)%aes.BlockSize == 0 {
-		if len(ct) > aes.BlockSize {
-			ct, _ = swapLastTwoBlocks(ct, aes.BlockSize)
-		}
-		mode = cipher.NewCBCDecrypter(block, iv)
-		message := make([]byte, len(ct))
-		mode.CryptBlocks(message, ct)
-		return message[:len(ct)], nil
-	}
-
-	// Cipher Text Stealing (CTS) using CBC interface. Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing
-	// Get ciphertext of the 2nd to last (penultimate) block (cpb), the last block (clb) and the rest (crb)
-	crb, cpb, clb, _ := tailBlocks(ct, aes.BlockSize)
-	v := make([]byte, len(iv), len(iv))
-	copy(v, iv)
-	var message []byte
-	if crb != nil {
-		//If there is more than just the last and the penultimate block we decrypt it and the last bloc of this becomes the iv for later
-		rb := make([]byte, len(crb))
-		mode = cipher.NewCBCDecrypter(block, v)
-		v = crb[len(crb)-aes.BlockSize:]
-		mode.CryptBlocks(rb, crb)
-		message = append(message, rb...)
-	}
-
-	// We need to modify the cipher text
-	// Decryt the 2nd to last (penultimate) block with a the original iv
-	pb := make([]byte, aes.BlockSize)
-	mode = cipher.NewCBCDecrypter(block, iv)
-	mode.CryptBlocks(pb, cpb)
-	// number of byte needed to pad
-	npb := aes.BlockSize - len(ct)%aes.BlockSize
-	//pad last block using the number of bytes needed from the tail of the plaintext 2nd to last (penultimate) block
-	clb = append(clb, pb[len(pb)-npb:]...)
-
-	// Now decrypt the last block in the penultimate position (iv will be from the crb, if the is no crb it's zeros)
-	// iv for the penultimate block decrypted in the last position becomes the modified last block
-	lb := make([]byte, aes.BlockSize)
-	mode = cipher.NewCBCDecrypter(block, v)
-	v = clb
-	mode.CryptBlocks(lb, clb)
-	message = append(message, lb...)
-
-	// Now decrypt the penultimate block in the last position (iv will be from the modified last block)
-	mode = cipher.NewCBCDecrypter(block, v)
-	mode.CryptBlocks(cpb, cpb)
-	message = append(message, cpb...)
-
-	// Truncate to the size of the original cipher text
-	return message[:len(ct)], nil
-}
-
-func tailBlocks(b []byte, c int) ([]byte, []byte, []byte, error) {
-	if len(b) <= c {
-		return []byte{}, []byte{}, []byte{}, errors.New("bytes slice is not larger than one block so cannot tail")
-	}
-	// Get size of last block
-	var lbs int
-	if l := len(b) % aes.BlockSize; l == 0 {
-		lbs = aes.BlockSize
-	} else {
-		lbs = l
-	}
-	// Get last block
-	lb := b[len(b)-lbs:]
-	// Get 2nd to last (penultimate) block
-	pb := b[len(b)-lbs-c : len(b)-lbs]
-	if len(b) > 2*c {
-		rb := b[:len(b)-lbs-c]
-		return rb, pb, lb, nil
-	}
-	return nil, pb, lb, nil
-}
-
-func swapLastTwoBlocks(b []byte, c int) ([]byte, error) {
-	rb, pb, lb, err := tailBlocks(b, c)
-	if err != nil {
-		return nil, err
-	}
-	var out []byte
-	if rb != nil {
-		out = append(out, rb...)
-	}
-	out = append(out, lb...)
-	out = append(out, pb...)
-	return out, nil
-}
-
-// zeroPad pads bytes with zeros to nearest multiple of message size m.
-func zeroPad(b []byte, m int) ([]byte, error) {
-	if m <= 0 {
-		return nil, errors.New("Invalid message block size when padding")
-	}
-	if b == nil || len(b) == 0 {
-		return nil, errors.New("Data not valid to pad: Zero size")
-	}
-	if l := len(b) % m; l != 0 {
-		n := m - l
-		z := make([]byte, n)
-		b = append(b, z...)
-	}
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/dnsutils.v1/.gitignore b/vendor/gopkg.in/jcmturner/dnsutils.v1/.gitignore
deleted file mode 100644
index a1338d6..0000000
--- a/vendor/gopkg.in/jcmturner/dnsutils.v1/.gitignore
+++ /dev/null
@@ -1,14 +0,0 @@
-# Binaries for programs and plugins
-*.exe
-*.dll
-*.so
-*.dylib
-
-# Test binary, build with `go test -c`
-*.test
-
-# Output of the go coverage tool, specifically when used with LiteIDE
-*.out
-
-# Project-local glide cache, RE: https://github.com/Masterminds/glide/issues/736
-.glide/
diff --git a/vendor/gopkg.in/jcmturner/dnsutils.v1/.travis.yml b/vendor/gopkg.in/jcmturner/dnsutils.v1/.travis.yml
deleted file mode 100644
index cab4f7b..0000000
--- a/vendor/gopkg.in/jcmturner/dnsutils.v1/.travis.yml
+++ /dev/null
@@ -1,24 +0,0 @@
-language: go
-
-go:
-  - 1.7.x
-  - 1.8.x
-  - 1.9.x
-  - master
-
-gobuild_args: -tags=integration -race
-
-sudo: required
-
-services:
-  - docker
-
-before_install:
-  - docker pull jcmturner/gokrb5:dns
-  - docker run -d -h kdc.test.gokrb5 -v /etc/localtime:/etc/localtime:ro -e "TEST_KDC_ADDR=127.0.0.1" -p 53:53 -p 53:53/udp --name dns jcmturner/gokrb5:dns
-
-before_script:
-  - sudo sed -i 's/nameserver .*/nameserver 127.0.0.1/g' /etc/resolv.conf
-
-env:
-  - DNSUTILS_OVERRIDE_NS="127.0.0.1:53"
\ No newline at end of file
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/LICENSE b/vendor/gopkg.in/jcmturner/gokrb5.v7/LICENSE
deleted file mode 100644
index 8dada3e..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/LICENSE
+++ /dev/null
@@ -1,201 +0,0 @@
-                                 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/gopkg.in/jcmturner/gokrb5.v7/client/ASExchange.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/ASExchange.go
deleted file mode 100644
index 9d1a2f3..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/ASExchange.go
+++ /dev/null
@@ -1,189 +0,0 @@
-package client
-
-import (
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/patype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// ASExchange performs an AS exchange for the client to retrieve a TGT.
-func (cl *Client) ASExchange(realm string, ASReq messages.ASReq, referral int) (messages.ASRep, error) {
-	if ok, err := cl.IsConfigured(); !ok {
-		return messages.ASRep{}, krberror.Errorf(err, krberror.ConfigError, "AS Exchange cannot be performed")
-	}
-
-	// Set PAData if required
-	err := setPAData(cl, nil, &ASReq)
-	if err != nil {
-		return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: issue with setting PAData on AS_REQ")
-	}
-
-	b, err := ASReq.Marshal()
-	if err != nil {
-		return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ")
-	}
-	var ASRep messages.ASRep
-
-	rb, err := cl.sendToKDC(b, realm)
-	if err != nil {
-		if e, ok := err.(messages.KRBError); ok {
-			switch e.ErrorCode {
-			case errorcode.KDC_ERR_PREAUTH_REQUIRED, errorcode.KDC_ERR_PREAUTH_FAILED:
-				// From now on assume this client will need to do this pre-auth and set the PAData
-				cl.settings.assumePreAuthentication = true
-				err = setPAData(cl, &e, &ASReq)
-				if err != nil {
-					return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: failed setting AS_REQ PAData for pre-authentication required")
-				}
-				b, err := ASReq.Marshal()
-				if err != nil {
-					return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ with PAData")
-				}
-				rb, err = cl.sendToKDC(b, realm)
-				if err != nil {
-					if _, ok := err.(messages.KRBError); ok {
-						return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC")
-					}
-					return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC")
-				}
-			case errorcode.KDC_ERR_WRONG_REALM:
-				// Client referral https://tools.ietf.org/html/rfc6806.html#section-7
-				if referral > 5 {
-					return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "maximum number of client referrals exceeded")
-				}
-				referral++
-				return cl.ASExchange(e.CRealm, ASReq, referral)
-			default:
-				return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC")
-			}
-		} else {
-			return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC")
-		}
-	}
-	err = ASRep.Unmarshal(rb)
-	if err != nil {
-		return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed to process the AS_REP")
-	}
-	if ok, err := ASRep.Verify(cl.Config, cl.Credentials, ASReq); !ok {
-		return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: AS_REP is not valid or client password/keytab incorrect")
-	}
-	return ASRep, nil
-}
-
-// setPAData adds pre-authentication data to the AS_REQ.
-func setPAData(cl *Client, krberr *messages.KRBError, ASReq *messages.ASReq) error {
-	if !cl.settings.DisablePAFXFAST() {
-		pa := types.PAData{PADataType: patype.PA_REQ_ENC_PA_REP}
-		ASReq.PAData = append(ASReq.PAData, pa)
-	}
-	if cl.settings.AssumePreAuthentication() {
-		// Identify the etype to use to encrypt the PA Data
-		var et etype.EType
-		var err error
-		var key types.EncryptionKey
-		if krberr == nil {
-			// This is not in response to an error from the KDC. It is preemptive or renewal
-			// There is no KRB Error that tells us the etype to use
-			etn := cl.settings.preAuthEType // Use the etype that may have previously been negotiated
-			if etn == 0 {
-				etn = int32(cl.Config.LibDefaults.PreferredPreauthTypes[0]) // Resort to config
-			}
-			et, err = crypto.GetEtype(etn)
-			if err != nil {
-				return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption")
-			}
-			key, err = cl.Key(et, nil)
-			if err != nil {
-				return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials")
-			}
-		} else {
-			// Get the etype to use from the PA data in the KRBError e-data
-			et, err = preAuthEType(krberr)
-			if err != nil {
-				return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption")
-			}
-			cl.settings.preAuthEType = et.GetETypeID() // Set the etype that has been defined for potential future use
-			key, err = cl.Key(et, krberr)
-			if err != nil {
-				return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials")
-			}
-		}
-		// Generate the PA data
-		paTSb, err := types.GetPAEncTSEncAsnMarshalled()
-		if err != nil {
-			return krberror.Errorf(err, krberror.KRBMsgError, "error creating PAEncTSEnc for Pre-Authentication")
-		}
-		//TODO (theme: KVNO from keytab) the kvno should not be hard coded to 1 as this hampers troubleshooting.
-		paEncTS, err := crypto.GetEncryptedData(paTSb, key, keyusage.AS_REQ_PA_ENC_TIMESTAMP, 1)
-		if err != nil {
-			return krberror.Errorf(err, krberror.EncryptingError, "error encrypting pre-authentication timestamp")
-		}
-		pb, err := paEncTS.Marshal()
-		if err != nil {
-			return krberror.Errorf(err, krberror.EncodingError, "error marshaling the PAEncTSEnc encrypted data")
-		}
-		pa := types.PAData{
-			PADataType:  patype.PA_ENC_TIMESTAMP,
-			PADataValue: pb,
-		}
-		// Look for and delete any exiting patype.PA_ENC_TIMESTAMP
-		for i, pa := range ASReq.PAData {
-			if pa.PADataType == patype.PA_ENC_TIMESTAMP {
-				ASReq.PAData[i] = ASReq.PAData[len(ASReq.PAData)-1]
-				ASReq.PAData = ASReq.PAData[:len(ASReq.PAData)-1]
-			}
-		}
-		ASReq.PAData = append(ASReq.PAData, pa)
-	}
-	return nil
-}
-
-// preAuthEType establishes what encryption type to use for pre-authentication from the KRBError returned from the KDC.
-func preAuthEType(krberr *messages.KRBError) (etype etype.EType, err error) {
-	//The preferred ordering of the "hint" pre-authentication data that
-	//affect client key selection is: ETYPE-INFO2, followed by ETYPE-INFO,
-	//followed by PW-SALT.
-	//A KDC SHOULD NOT send PA-PW-SALT when issuing a KRB-ERROR message
-	//that requests additional pre-authentication.  Implementation note:
-	//Some KDC implementations issue an erroneous PA-PW-SALT when issuing a
-	//KRB-ERROR message that requests additional pre-authentication.
-	//Therefore, clients SHOULD ignore a PA-PW-SALT accompanying a
-	//KRB-ERROR message that requests additional pre-authentication.
-	var etypeID int32
-	var pas types.PADataSequence
-	e := pas.Unmarshal(krberr.EData)
-	if e != nil {
-		err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling KRBError data")
-		return
-	}
-	for _, pa := range pas {
-		switch pa.PADataType {
-		case patype.PA_ETYPE_INFO2:
-			info, e := pa.GetETypeInfo2()
-			if e != nil {
-				err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO2 data")
-				return
-			}
-			etypeID = info[0].EType
-			break
-		case patype.PA_ETYPE_INFO:
-			info, e := pa.GetETypeInfo()
-			if e != nil {
-				err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO data")
-				return
-			}
-			etypeID = info[0].EType
-		}
-	}
-	etype, e = crypto.GetEtype(etypeID)
-	if e != nil {
-		err = krberror.Errorf(e, krberror.EncryptingError, "error creating etype")
-		return
-	}
-	return etype, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/TGSExchange.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/TGSExchange.go
deleted file mode 100644
index 8ad3e55..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/TGSExchange.go
+++ /dev/null
@@ -1,103 +0,0 @@
-package client
-
-import (
-	"gopkg.in/jcmturner/gokrb5.v7/iana/flags"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// TGSREQGenerateAndExchange generates the TGS_REQ and performs a TGS exchange to retrieve a ticket to the specified SPN.
-func (cl *Client) TGSREQGenerateAndExchange(spn types.PrincipalName, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, renewal bool) (tgsReq messages.TGSReq, tgsRep messages.TGSRep, err error) {
-	tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, spn, renewal)
-	if err != nil {
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: failed to generate a new TGS_REQ")
-	}
-	return cl.TGSExchange(tgsReq, kdcRealm, tgsRep.Ticket, sessionKey, 0)
-}
-
-// TGSExchange exchanges the provided TGS_REQ with the KDC to retrieve a TGS_REP.
-// Referrals are automatically handled.
-// The client's cache is updated with the ticket received.
-func (cl *Client) TGSExchange(tgsReq messages.TGSReq, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, referral int) (messages.TGSReq, messages.TGSRep, error) {
-	var tgsRep messages.TGSRep
-	b, err := tgsReq.Marshal()
-	if err != nil {
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to marshal TGS_REQ")
-	}
-	r, err := cl.sendToKDC(b, kdcRealm)
-	if err != nil {
-		if _, ok := err.(messages.KRBError); ok {
-			return tgsReq, tgsRep, krberror.Errorf(err, krberror.KDCError, "TGS Exchange Error: kerberos error response from KDC when requesting for %s", tgsReq.ReqBody.SName.PrincipalNameString())
-		}
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.NetworkingError, "TGS Exchange Error: issue sending TGS_REQ to KDC")
-	}
-	err = tgsRep.Unmarshal(r)
-	if err != nil {
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP")
-	}
-	err = tgsRep.DecryptEncPart(sessionKey)
-	if err != nil {
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP")
-	}
-	if ok, err := tgsRep.Verify(cl.Config, tgsReq); !ok {
-		return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: TGS_REP is not valid")
-	}
-
-	if tgsRep.Ticket.SName.NameString[0] == "krbtgt" && !tgsRep.Ticket.SName.Equal(tgsReq.ReqBody.SName) {
-		if referral > 5 {
-			return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: maximum number of referrals exceeded")
-		}
-		// Server referral https://tools.ietf.org/html/rfc6806.html#section-8
-		// The TGS Rep contains a TGT for another domain as the service resides in that domain.
-		cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart)
-		realm := tgsRep.Ticket.SName.NameString[len(tgsRep.Ticket.SName.NameString)-1]
-		referral++
-		if types.IsFlagSet(&tgsReq.ReqBody.KDCOptions, flags.EncTktInSkey) && len(tgsReq.ReqBody.AdditionalTickets) > 0 {
-			tgsReq, err = messages.NewUser2UserTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, tgsReq.ReqBody.SName, tgsReq.Renewal, tgsReq.ReqBody.AdditionalTickets[0])
-			if err != nil {
-				return tgsReq, tgsRep, err
-			}
-		}
-		tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, tgsReq.ReqBody.SName, tgsReq.Renewal)
-		if err != nil {
-			return tgsReq, tgsRep, err
-		}
-		return cl.TGSExchange(tgsReq, realm, tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, referral)
-	}
-	cl.cache.addEntry(
-		tgsRep.Ticket,
-		tgsRep.DecryptedEncPart.AuthTime,
-		tgsRep.DecryptedEncPart.StartTime,
-		tgsRep.DecryptedEncPart.EndTime,
-		tgsRep.DecryptedEncPart.RenewTill,
-		tgsRep.DecryptedEncPart.Key,
-	)
-	cl.Log("ticket added to cache for %s (EndTime: %v)", tgsRep.Ticket.SName.PrincipalNameString(), tgsRep.DecryptedEncPart.EndTime)
-	return tgsReq, tgsRep, err
-}
-
-// GetServiceTicket makes a request to get a service ticket for the SPN specified
-// SPN format: <SERVICE>/<FQDN> Eg. HTTP/www.example.com
-// The ticket will be added to the client's ticket cache
-func (cl *Client) GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) {
-	var tkt messages.Ticket
-	var skey types.EncryptionKey
-	if tkt, skey, ok := cl.GetCachedTicket(spn); ok {
-		// Already a valid ticket in the cache
-		return tkt, skey, nil
-	}
-	princ := types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn)
-	realm := cl.Config.ResolveRealm(princ.NameString[len(princ.NameString)-1])
-
-	tgt, skey, err := cl.sessionTGT(realm)
-	if err != nil {
-		return tkt, skey, err
-	}
-	_, tgsRep, err := cl.TGSREQGenerateAndExchange(princ, realm, tgt, skey, false)
-	if err != nil {
-		return tkt, skey, err
-	}
-	return tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/cache.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/cache.go
deleted file mode 100644
index 07b4a01..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/cache.go
+++ /dev/null
@@ -1,110 +0,0 @@
-package client
-
-import (
-	"errors"
-	"sync"
-	"time"
-
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// Cache for service tickets held by the client.
-type Cache struct {
-	Entries map[string]CacheEntry
-	mux     sync.RWMutex
-}
-
-// CacheEntry holds details for a cache entry.
-type CacheEntry struct {
-	Ticket     messages.Ticket
-	AuthTime   time.Time
-	StartTime  time.Time
-	EndTime    time.Time
-	RenewTill  time.Time
-	SessionKey types.EncryptionKey
-}
-
-// NewCache creates a new client ticket cache instance.
-func NewCache() *Cache {
-	return &Cache{
-		Entries: map[string]CacheEntry{},
-	}
-}
-
-// getEntry returns a cache entry that matches the SPN.
-func (c *Cache) getEntry(spn string) (CacheEntry, bool) {
-	c.mux.RLock()
-	defer c.mux.RUnlock()
-	e, ok := (*c).Entries[spn]
-	return e, ok
-}
-
-// addEntry adds a ticket to the cache.
-func (c *Cache) addEntry(tkt messages.Ticket, authTime, startTime, endTime, renewTill time.Time, sessionKey types.EncryptionKey) CacheEntry {
-	spn := tkt.SName.PrincipalNameString()
-	c.mux.Lock()
-	defer c.mux.Unlock()
-	(*c).Entries[spn] = CacheEntry{
-		Ticket:     tkt,
-		AuthTime:   authTime,
-		StartTime:  startTime,
-		EndTime:    endTime,
-		RenewTill:  renewTill,
-		SessionKey: sessionKey,
-	}
-	return c.Entries[spn]
-}
-
-// clear deletes all the cache entries
-func (c *Cache) clear() {
-	c.mux.Lock()
-	defer c.mux.Unlock()
-	for k := range c.Entries {
-		delete(c.Entries, k)
-	}
-}
-
-// RemoveEntry removes the cache entry for the defined SPN.
-func (c *Cache) RemoveEntry(spn string) {
-	c.mux.Lock()
-	defer c.mux.Unlock()
-	delete(c.Entries, spn)
-}
-
-// GetCachedTicket returns a ticket from the cache for the SPN.
-// Only a ticket that is currently valid will be returned.
-func (cl *Client) GetCachedTicket(spn string) (messages.Ticket, types.EncryptionKey, bool) {
-	if e, ok := cl.cache.getEntry(spn); ok {
-		//If within time window of ticket return it
-		if time.Now().UTC().After(e.StartTime) && time.Now().UTC().Before(e.EndTime) {
-			cl.Log("ticket received from cache for %s", spn)
-			return e.Ticket, e.SessionKey, true
-		} else if time.Now().UTC().Before(e.RenewTill) {
-			e, err := cl.renewTicket(e)
-			if err != nil {
-				return e.Ticket, e.SessionKey, false
-			}
-			return e.Ticket, e.SessionKey, true
-		}
-	}
-	var tkt messages.Ticket
-	var key types.EncryptionKey
-	return tkt, key, false
-}
-
-// renewTicket renews a cache entry ticket.
-// To renew from outside the client package use GetCachedTicket
-func (cl *Client) renewTicket(e CacheEntry) (CacheEntry, error) {
-	spn := e.Ticket.SName
-	_, _, err := cl.TGSREQGenerateAndExchange(spn, e.Ticket.Realm, e.Ticket, e.SessionKey, true)
-	if err != nil {
-		return e, err
-	}
-	e, ok := cl.cache.getEntry(e.Ticket.SName.PrincipalNameString())
-	if !ok {
-		return e, errors.New("ticket was not added to cache")
-	}
-	cl.Log("ticket renewed for %s (EndTime: %v)", spn.PrincipalNameString(), e.EndTime)
-	return e, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/client.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/client.go
deleted file mode 100644
index 6e4c83c..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/client.go
+++ /dev/null
@@ -1,229 +0,0 @@
-// Package client provides a client library and methods for Kerberos 5 authentication.
-package client
-
-import (
-	"errors"
-	"fmt"
-	"time"
-
-	"gopkg.in/jcmturner/gokrb5.v7/config"
-	"gopkg.in/jcmturner/gokrb5.v7/credentials"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-	"gopkg.in/jcmturner/gokrb5.v7/keytab"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// Client side configuration and state.
-type Client struct {
-	Credentials *credentials.Credentials
-	Config      *config.Config
-	settings    *Settings
-	sessions    *sessions
-	cache       *Cache
-}
-
-// NewClientWithPassword creates a new client from a password credential.
-// Set the realm to empty string to use the default realm from config.
-func NewClientWithPassword(username, realm, password string, krb5conf *config.Config, settings ...func(*Settings)) *Client {
-	creds := credentials.New(username, realm)
-	return &Client{
-		Credentials: creds.WithPassword(password),
-		Config:      krb5conf,
-		settings:    NewSettings(settings...),
-		sessions: &sessions{
-			Entries: make(map[string]*session),
-		},
-		cache: NewCache(),
-	}
-}
-
-// NewClientWithKeytab creates a new client from a keytab credential.
-func NewClientWithKeytab(username, realm string, kt *keytab.Keytab, krb5conf *config.Config, settings ...func(*Settings)) *Client {
-	creds := credentials.New(username, realm)
-	return &Client{
-		Credentials: creds.WithKeytab(kt),
-		Config:      krb5conf,
-		settings:    NewSettings(settings...),
-		sessions: &sessions{
-			Entries: make(map[string]*session),
-		},
-		cache: NewCache(),
-	}
-}
-
-// NewClientFromCCache create a client from a populated client cache.
-//
-// WARNING: A client created from CCache does not automatically renew TGTs and a failure will occur after the TGT expires.
-func NewClientFromCCache(c *credentials.CCache, krb5conf *config.Config, settings ...func(*Settings)) (*Client, error) {
-	cl := &Client{
-		Credentials: c.GetClientCredentials(),
-		Config:      krb5conf,
-		settings:    NewSettings(settings...),
-		sessions: &sessions{
-			Entries: make(map[string]*session),
-		},
-		cache: NewCache(),
-	}
-	spn := types.PrincipalName{
-		NameType:   nametype.KRB_NT_SRV_INST,
-		NameString: []string{"krbtgt", c.DefaultPrincipal.Realm},
-	}
-	cred, ok := c.GetEntry(spn)
-	if !ok {
-		return cl, errors.New("TGT not found in CCache")
-	}
-	var tgt messages.Ticket
-	err := tgt.Unmarshal(cred.Ticket)
-	if err != nil {
-		return cl, fmt.Errorf("TGT bytes in cache are not valid: %v", err)
-	}
-	cl.sessions.Entries[c.DefaultPrincipal.Realm] = &session{
-		realm:      c.DefaultPrincipal.Realm,
-		authTime:   cred.AuthTime,
-		endTime:    cred.EndTime,
-		renewTill:  cred.RenewTill,
-		tgt:        tgt,
-		sessionKey: cred.Key,
-	}
-	for _, cred := range c.GetEntries() {
-		var tkt messages.Ticket
-		err = tkt.Unmarshal(cred.Ticket)
-		if err != nil {
-			return cl, fmt.Errorf("cache entry ticket bytes are not valid: %v", err)
-		}
-		cl.cache.addEntry(
-			tkt,
-			cred.AuthTime,
-			cred.StartTime,
-			cred.EndTime,
-			cred.RenewTill,
-			cred.Key,
-		)
-	}
-	return cl, nil
-}
-
-// Key returns the client's encryption key for the specified encryption type.
-// The key can be retrieved either from the keytab or generated from the client's password.
-// If the client has both a keytab and a password defined the keytab is favoured as the source for the key
-// A KRBError can be passed in the event the KDC returns one of type KDC_ERR_PREAUTH_REQUIRED and is required to derive
-// the key for pre-authentication from the client's password. If a KRBError is not available, pass nil to this argument.
-func (cl *Client) Key(etype etype.EType, krberr *messages.KRBError) (types.EncryptionKey, error) {
-	if cl.Credentials.HasKeytab() && etype != nil {
-		return cl.Credentials.Keytab().GetEncryptionKey(cl.Credentials.CName(), cl.Credentials.Domain(), 0, etype.GetETypeID())
-	} else if cl.Credentials.HasPassword() {
-		if krberr != nil && krberr.ErrorCode == errorcode.KDC_ERR_PREAUTH_REQUIRED {
-			var pas types.PADataSequence
-			err := pas.Unmarshal(krberr.EData)
-			if err != nil {
-				return types.EncryptionKey{}, fmt.Errorf("could not get PAData from KRBError to generate key from password: %v", err)
-			}
-			key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), krberr.CName, krberr.CRealm, etype.GetETypeID(), pas)
-			return key, err
-		}
-		key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), cl.Credentials.CName(), cl.Credentials.Domain(), etype.GetETypeID(), types.PADataSequence{})
-		return key, err
-	}
-	return types.EncryptionKey{}, errors.New("credential has neither keytab or password to generate key")
-}
-
-// IsConfigured indicates if the client has the values required set.
-func (cl *Client) IsConfigured() (bool, error) {
-	if cl.Credentials.UserName() == "" {
-		return false, errors.New("client does not have a username")
-	}
-	if cl.Credentials.Domain() == "" {
-		return false, errors.New("client does not have a define realm")
-	}
-	// Client needs to have either a password, keytab or a session already (later when loading from CCache)
-	if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() {
-		authTime, _, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
-		if err != nil || authTime.IsZero() {
-			return false, errors.New("client has neither a keytab nor a password set and no session")
-		}
-	}
-	if !cl.Config.LibDefaults.DNSLookupKDC {
-		for _, r := range cl.Config.Realms {
-			if r.Realm == cl.Credentials.Domain() {
-				if len(r.KDC) > 0 {
-					return true, nil
-				}
-				return false, errors.New("client krb5 config does not have any defined KDCs for the default realm")
-			}
-		}
-	}
-	return true, nil
-}
-
-// Login the client with the KDC via an AS exchange.
-func (cl *Client) Login() error {
-	if ok, err := cl.IsConfigured(); !ok {
-		return err
-	}
-	if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() {
-		_, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
-		if err != nil {
-			return krberror.Errorf(err, krberror.KRBMsgError, "no user credentials available and error getting any existing session")
-		}
-		if time.Now().UTC().After(endTime) {
-			return krberror.NewKrberror(krberror.KRBMsgError, "cannot login, no user credentials available and no valid existing session")
-		}
-		// no credentials but there is a session with tgt already
-		return nil
-	}
-	ASReq, err := messages.NewASReqForTGT(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName())
-	if err != nil {
-		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AS_REQ")
-	}
-	ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0)
-	if err != nil {
-		return err
-	}
-	cl.addSession(ASRep.Ticket, ASRep.DecryptedEncPart)
-	return nil
-}
-
-// realmLogin obtains or renews a TGT and establishes a session for the realm specified.
-func (cl *Client) realmLogin(realm string) error {
-	if realm == cl.Credentials.Domain() {
-		return cl.Login()
-	}
-	_, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain())
-	if err != nil || time.Now().UTC().After(endTime) {
-		err := cl.Login()
-		if err != nil {
-			return fmt.Errorf("could not get valid TGT for client's realm: %v", err)
-		}
-	}
-	tgt, skey, err := cl.sessionTGT(cl.Credentials.Domain())
-	if err != nil {
-		return err
-	}
-
-	spn := types.PrincipalName{
-		NameType:   nametype.KRB_NT_SRV_INST,
-		NameString: []string{"krbtgt", realm},
-	}
-
-	_, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, false)
-	if err != nil {
-		return err
-	}
-	cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart)
-
-	return nil
-}
-
-// Destroy stops the auto-renewal of all sessions and removes the sessions and cache entries from the client.
-func (cl *Client) Destroy() {
-	creds := credentials.New("", "")
-	cl.sessions.destroy()
-	cl.cache.clear()
-	cl.Credentials = creds
-	cl.Log("client destroyed")
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/network.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/network.go
deleted file mode 100644
index 493fb2f..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/network.go
+++ /dev/null
@@ -1,224 +0,0 @@
-package client
-
-import (
-	"bytes"
-	"encoding/binary"
-	"errors"
-	"fmt"
-	"io"
-	"net"
-	"time"
-
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-)
-
-// SendToKDC performs network actions to send data to the KDC.
-func (cl *Client) sendToKDC(b []byte, realm string) ([]byte, error) {
-	var rb []byte
-	if cl.Config.LibDefaults.UDPPreferenceLimit == 1 {
-		//1 means we should always use TCP
-		rb, errtcp := cl.sendKDCTCP(realm, b)
-		if errtcp != nil {
-			if e, ok := errtcp.(messages.KRBError); ok {
-				return rb, e
-			}
-			return rb, fmt.Errorf("communication error with KDC via TCP: %v", errtcp)
-		}
-		return rb, nil
-	}
-	if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit {
-		//Try UDP first, TCP second
-		rb, errudp := cl.sendKDCUDP(realm, b)
-		if errudp != nil {
-			if e, ok := errudp.(messages.KRBError); ok && e.ErrorCode != errorcode.KRB_ERR_RESPONSE_TOO_BIG {
-				// Got a KRBError from KDC
-				// If this is not a KRB_ERR_RESPONSE_TOO_BIG we will return immediately otherwise will try TCP.
-				return rb, e
-			}
-			// Try TCP
-			r, errtcp := cl.sendKDCTCP(realm, b)
-			if errtcp != nil {
-				if e, ok := errtcp.(messages.KRBError); ok {
-					// Got a KRBError
-					return r, e
-				}
-				return r, fmt.Errorf("failed to communicate with KDC. Attempts made with UDP (%v) and then TCP (%v)", errudp, errtcp)
-			}
-			rb = r
-		}
-		return rb, nil
-	}
-	//Try TCP first, UDP second
-	rb, errtcp := cl.sendKDCTCP(realm, b)
-	if errtcp != nil {
-		if e, ok := errtcp.(messages.KRBError); ok {
-			// Got a KRBError from KDC so returning and not trying UDP.
-			return rb, e
-		}
-		rb, errudp := cl.sendKDCUDP(realm, b)
-		if errudp != nil {
-			if e, ok := errudp.(messages.KRBError); ok {
-				// Got a KRBError
-				return rb, e
-			}
-			return rb, fmt.Errorf("failed to communicate with KDC. Attempts made with TCP (%v) and then UDP (%v)", errtcp, errudp)
-		}
-	}
-	return rb, nil
-}
-
-// dialKDCTCP establishes a UDP connection to a KDC.
-func dialKDCUDP(count int, kdcs map[int]string) (*net.UDPConn, error) {
-	i := 1
-	for i <= count {
-		udpAddr, err := net.ResolveUDPAddr("udp", kdcs[i])
-		if err != nil {
-			return nil, fmt.Errorf("error resolving KDC address: %v", err)
-		}
-
-		conn, err := net.DialTimeout("udp", udpAddr.String(), 5*time.Second)
-		if err == nil {
-			if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil {
-				return nil, err
-			}
-			// conn is guaranteed to be a UDPConn
-			return conn.(*net.UDPConn), nil
-		}
-		i++
-	}
-	return nil, errors.New("error in getting a UDP connection to any of the KDCs")
-}
-
-// dialKDCTCP establishes a TCP connection to a KDC.
-func dialKDCTCP(count int, kdcs map[int]string) (*net.TCPConn, error) {
-	i := 1
-	for i <= count {
-		tcpAddr, err := net.ResolveTCPAddr("tcp", kdcs[i])
-		if err != nil {
-			return nil, fmt.Errorf("error resolving KDC address: %v", err)
-		}
-
-		conn, err := net.DialTimeout("tcp", tcpAddr.String(), 5*time.Second)
-		if err == nil {
-			if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil {
-				return nil, err
-			}
-			// conn is guaranteed to be a TCPConn
-			return conn.(*net.TCPConn), nil
-		}
-		i++
-	}
-	return nil, errors.New("error in getting a TCP connection to any of the KDCs")
-}
-
-// sendKDCUDP sends bytes to the KDC via UDP.
-func (cl *Client) sendKDCUDP(realm string, b []byte) ([]byte, error) {
-	var r []byte
-	count, kdcs, err := cl.Config.GetKDCs(realm, false)
-	if err != nil {
-		return r, err
-	}
-	conn, err := dialKDCUDP(count, kdcs)
-	if err != nil {
-		return r, err
-	}
-	r, err = cl.sendUDP(conn, b)
-	if err != nil {
-		return r, err
-	}
-	return checkForKRBError(r)
-}
-
-// sendKDCTCP sends bytes to the KDC via TCP.
-func (cl *Client) sendKDCTCP(realm string, b []byte) ([]byte, error) {
-	var r []byte
-	count, kdcs, err := cl.Config.GetKDCs(realm, true)
-	if err != nil {
-		return r, err
-	}
-	conn, err := dialKDCTCP(count, kdcs)
-	if err != nil {
-		return r, err
-	}
-	rb, err := cl.sendTCP(conn, b)
-	if err != nil {
-		return r, err
-	}
-	return checkForKRBError(rb)
-}
-
-// sendUDP sends bytes to connection over UDP.
-func (cl *Client) sendUDP(conn *net.UDPConn, b []byte) ([]byte, error) {
-	var r []byte
-	defer conn.Close()
-	_, err := conn.Write(b)
-	if err != nil {
-		return r, fmt.Errorf("error sending to (%s): %v", conn.RemoteAddr().String(), err)
-	}
-	udpbuf := make([]byte, 4096)
-	n, _, err := conn.ReadFrom(udpbuf)
-	r = udpbuf[:n]
-	if err != nil {
-		return r, fmt.Errorf("sending over UDP failed to %s: %v", conn.RemoteAddr().String(), err)
-	}
-	if len(r) < 1 {
-		return r, fmt.Errorf("no response data from %s", conn.RemoteAddr().String())
-	}
-	return r, nil
-}
-
-// sendTCP sends bytes to connection over TCP.
-func (cl *Client) sendTCP(conn *net.TCPConn, b []byte) ([]byte, error) {
-	defer conn.Close()
-	var r []byte
-	/*
-		RFC https://tools.ietf.org/html/rfc4120#section-7.2.2
-		Each request (KRB_KDC_REQ) and response (KRB_KDC_REP or KRB_ERROR)
-		sent over the TCP stream is preceded by the length of the request as
-		4 octets in network byte order.  The high bit of the length is
-		reserved for future expansion and MUST currently be set to zero.  If
-		a KDC that does not understand how to interpret a set high bit of the
-		length encoding receives a request with the high order bit of the
-		length set, it MUST return a KRB-ERROR message with the error
-		KRB_ERR_FIELD_TOOLONG and MUST close the TCP stream.
-		NB: network byte order == big endian
-	*/
-	var buf bytes.Buffer
-	err := binary.Write(&buf, binary.BigEndian, uint32(len(b)))
-	if err != nil {
-		return r, err
-	}
-	b = append(buf.Bytes(), b...)
-
-	_, err = conn.Write(b)
-	if err != nil {
-		return r, fmt.Errorf("error sending to KDC (%s): %v", conn.RemoteAddr().String(), err)
-	}
-
-	sh := make([]byte, 4, 4)
-	_, err = conn.Read(sh)
-	if err != nil {
-		return r, fmt.Errorf("error reading response size header: %v", err)
-	}
-	s := binary.BigEndian.Uint32(sh)
-
-	rb := make([]byte, s, s)
-	_, err = io.ReadFull(conn, rb)
-	if err != nil {
-		return r, fmt.Errorf("error reading response: %v", err)
-	}
-	if len(rb) < 1 {
-		return r, fmt.Errorf("no response data from KDC %s", conn.RemoteAddr().String())
-	}
-	return rb, nil
-}
-
-// checkForKRBError checks if the response bytes from the KDC are a KRBError.
-func checkForKRBError(b []byte) ([]byte, error) {
-	var KRBErr messages.KRBError
-	if err := KRBErr.Unmarshal(b); err == nil {
-		return b, KRBErr
-	}
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/passwd.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/passwd.go
deleted file mode 100644
index e6d4180..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/passwd.go
+++ /dev/null
@@ -1,95 +0,0 @@
-package client
-
-import (
-	"fmt"
-	"net"
-
-	"gopkg.in/jcmturner/gokrb5.v7/kadmin"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-)
-
-// Kpasswd server response codes.
-const (
-	KRB5_KPASSWD_SUCCESS             = 0
-	KRB5_KPASSWD_MALFORMED           = 1
-	KRB5_KPASSWD_HARDERROR           = 2
-	KRB5_KPASSWD_AUTHERROR           = 3
-	KRB5_KPASSWD_SOFTERROR           = 4
-	KRB5_KPASSWD_ACCESSDENIED        = 5
-	KRB5_KPASSWD_BAD_VERSION         = 6
-	KRB5_KPASSWD_INITIAL_FLAG_NEEDED = 7
-)
-
-// ChangePasswd changes the password of the client to the value provided.
-func (cl *Client) ChangePasswd(newPasswd string) (bool, error) {
-	ASReq, err := messages.NewASReqForChgPasswd(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName())
-	if err != nil {
-		return false, err
-	}
-	ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0)
-	if err != nil {
-		return false, err
-	}
-
-	msg, key, err := kadmin.ChangePasswdMsg(cl.Credentials.CName(), cl.Credentials.Domain(), newPasswd, ASRep.Ticket, ASRep.DecryptedEncPart.Key)
-	if err != nil {
-		return false, err
-	}
-	r, err := cl.sendToKPasswd(msg)
-	if err != nil {
-		return false, err
-	}
-	err = r.Decrypt(key)
-	if err != nil {
-		return false, err
-	}
-	if r.ResultCode != KRB5_KPASSWD_SUCCESS {
-		return false, fmt.Errorf("error response from kdamin: %s", r.Result)
-	}
-	cl.Credentials.WithPassword(newPasswd)
-	return true, nil
-}
-
-func (cl *Client) sendToKPasswd(msg kadmin.Request) (r kadmin.Reply, err error) {
-	_, kps, err := cl.Config.GetKpasswdServers(cl.Credentials.Domain(), true)
-	if err != nil {
-		return
-	}
-	addr := kps[1]
-	b, err := msg.Marshal()
-	if err != nil {
-		return
-	}
-	if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit {
-		return cl.sendKPasswdUDP(b, addr)
-	}
-	return cl.sendKPasswdTCP(b, addr)
-}
-
-func (cl *Client) sendKPasswdTCP(b []byte, kadmindAddr string) (r kadmin.Reply, err error) {
-	tcpAddr, err := net.ResolveTCPAddr("tcp", kadmindAddr)
-	if err != nil {
-		return
-	}
-	conn, err := net.DialTCP("tcp", nil, tcpAddr)
-	if err != nil {
-		return
-	}
-	rb, err := cl.sendTCP(conn, b)
-	err = r.Unmarshal(rb)
-	return
-}
-
-func (cl *Client) sendKPasswdUDP(b []byte, kadmindAddr string) (r kadmin.Reply, err error) {
-	udpAddr, err := net.ResolveUDPAddr("udp", kadmindAddr)
-	if err != nil {
-		return
-	}
-	conn, err := net.DialUDP("udp", nil, udpAddr)
-	if err != nil {
-		return
-	}
-	rb, err := cl.sendUDP(conn, b)
-	err = r.Unmarshal(rb)
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/session.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/session.go
deleted file mode 100644
index ec6c513..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/session.go
+++ /dev/null
@@ -1,255 +0,0 @@
-package client
-
-import (
-	"fmt"
-	"strings"
-	"sync"
-	"time"
-
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// sessions hold TGTs and are keyed on the realm name
-type sessions struct {
-	Entries map[string]*session
-	mux     sync.RWMutex
-}
-
-// destroy erases all sessions
-func (s *sessions) destroy() {
-	s.mux.Lock()
-	defer s.mux.Unlock()
-	for k, e := range s.Entries {
-		e.destroy()
-		delete(s.Entries, k)
-	}
-}
-
-// update replaces a session with the one provided or adds it as a new one
-func (s *sessions) update(sess *session) {
-	s.mux.Lock()
-	defer s.mux.Unlock()
-	// if a session already exists for this, cancel its auto renew.
-	if i, ok := s.Entries[sess.realm]; ok {
-		if i != sess {
-			// Session in the sessions cache is not the same as one provided.
-			// Cancel the one in the cache and add this one.
-			i.mux.Lock()
-			defer i.mux.Unlock()
-			i.cancel <- true
-			s.Entries[sess.realm] = sess
-			return
-		}
-	}
-	// No session for this realm was found so just add it
-	s.Entries[sess.realm] = sess
-}
-
-// get returns the session for the realm specified
-func (s *sessions) get(realm string) (*session, bool) {
-	s.mux.RLock()
-	defer s.mux.RUnlock()
-	sess, ok := s.Entries[realm]
-	return sess, ok
-}
-
-// session holds the TGT details for a realm
-type session struct {
-	realm                string
-	authTime             time.Time
-	endTime              time.Time
-	renewTill            time.Time
-	tgt                  messages.Ticket
-	sessionKey           types.EncryptionKey
-	sessionKeyExpiration time.Time
-	cancel               chan bool
-	mux                  sync.RWMutex
-}
-
-// AddSession adds a session for a realm with a TGT to the client's session cache.
-// A goroutine is started to automatically renew the TGT before expiry.
-func (cl *Client) addSession(tgt messages.Ticket, dep messages.EncKDCRepPart) {
-	if strings.ToLower(tgt.SName.NameString[0]) != "krbtgt" {
-		// Not a TGT
-		return
-	}
-	realm := tgt.SName.NameString[len(tgt.SName.NameString)-1]
-	s := &session{
-		realm:                realm,
-		authTime:             dep.AuthTime,
-		endTime:              dep.EndTime,
-		renewTill:            dep.RenewTill,
-		tgt:                  tgt,
-		sessionKey:           dep.Key,
-		sessionKeyExpiration: dep.KeyExpiration,
-	}
-	cl.sessions.update(s)
-	cl.enableAutoSessionRenewal(s)
-	cl.Log("TGT session added for %s (EndTime: %v)", realm, dep.EndTime)
-}
-
-// update overwrites the session details with those from the TGT and decrypted encPart
-func (s *session) update(tgt messages.Ticket, dep messages.EncKDCRepPart) {
-	s.mux.Lock()
-	defer s.mux.Unlock()
-	s.authTime = dep.AuthTime
-	s.endTime = dep.EndTime
-	s.renewTill = dep.RenewTill
-	s.tgt = tgt
-	s.sessionKey = dep.Key
-	s.sessionKeyExpiration = dep.KeyExpiration
-}
-
-// destroy will cancel any auto renewal of the session and set the expiration times to the current time
-func (s *session) destroy() {
-	s.mux.Lock()
-	defer s.mux.Unlock()
-	if s.cancel != nil {
-		s.cancel <- true
-	}
-	s.endTime = time.Now().UTC()
-	s.renewTill = s.endTime
-	s.sessionKeyExpiration = s.endTime
-}
-
-// valid informs if the TGT is still within the valid time window
-func (s *session) valid() bool {
-	s.mux.RLock()
-	defer s.mux.RUnlock()
-	t := time.Now().UTC()
-	if t.Before(s.endTime) && s.authTime.Before(t) {
-		return true
-	}
-	return false
-}
-
-// tgtDetails is a thread safe way to get the session's realm, TGT and session key values
-func (s *session) tgtDetails() (string, messages.Ticket, types.EncryptionKey) {
-	s.mux.RLock()
-	defer s.mux.RUnlock()
-	return s.realm, s.tgt, s.sessionKey
-}
-
-// timeDetails is a thread safe way to get the session's validity time values
-func (s *session) timeDetails() (string, time.Time, time.Time, time.Time, time.Time) {
-	s.mux.RLock()
-	defer s.mux.RUnlock()
-	return s.realm, s.authTime, s.endTime, s.renewTill, s.sessionKeyExpiration
-}
-
-// enableAutoSessionRenewal turns on the automatic renewal for the client's TGT session.
-func (cl *Client) enableAutoSessionRenewal(s *session) {
-	var timer *time.Timer
-	s.mux.Lock()
-	s.cancel = make(chan bool, 1)
-	s.mux.Unlock()
-	go func(s *session) {
-		for {
-			s.mux.RLock()
-			w := (s.endTime.Sub(time.Now().UTC()) * 5) / 6
-			s.mux.RUnlock()
-			if w < 0 {
-				return
-			}
-			timer = time.NewTimer(w)
-			select {
-			case <-timer.C:
-				renewal, err := cl.refreshSession(s)
-				if err != nil {
-					cl.Log("error refreshing session: %v", err)
-				}
-				if !renewal && err == nil {
-					// end this goroutine as there will have been a new login and new auto renewal goroutine created.
-					return
-				}
-			case <-s.cancel:
-				// cancel has been called. Stop the timer and exit.
-				timer.Stop()
-				return
-			}
-		}
-	}(s)
-}
-
-// renewTGT renews the client's TGT session.
-func (cl *Client) renewTGT(s *session) error {
-	realm, tgt, skey := s.tgtDetails()
-	spn := types.PrincipalName{
-		NameType:   nametype.KRB_NT_SRV_INST,
-		NameString: []string{"krbtgt", realm},
-	}
-	_, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, true)
-	if err != nil {
-		return krberror.Errorf(err, krberror.KRBMsgError, "error renewing TGT for %s", realm)
-	}
-	s.update(tgsRep.Ticket, tgsRep.DecryptedEncPart)
-	cl.sessions.update(s)
-	cl.Log("TGT session renewed for %s (EndTime: %v)", realm, tgsRep.DecryptedEncPart.EndTime)
-	return nil
-}
-
-// refreshSession updates either through renewal or creating a new login.
-// The boolean indicates if the update was a renewal.
-func (cl *Client) refreshSession(s *session) (bool, error) {
-	s.mux.RLock()
-	realm := s.realm
-	renewTill := s.renewTill
-	s.mux.RUnlock()
-	cl.Log("refreshing TGT session for %s", realm)
-	if time.Now().UTC().Before(renewTill) {
-		err := cl.renewTGT(s)
-		return true, err
-	}
-	err := cl.realmLogin(realm)
-	return false, err
-}
-
-// ensureValidSession makes sure there is a valid session for the realm
-func (cl *Client) ensureValidSession(realm string) error {
-	s, ok := cl.sessions.get(realm)
-	if ok {
-		s.mux.RLock()
-		d := s.endTime.Sub(s.authTime) / 6
-		if s.endTime.Sub(time.Now().UTC()) > d {
-			s.mux.RUnlock()
-			return nil
-		}
-		s.mux.RUnlock()
-		_, err := cl.refreshSession(s)
-		return err
-	}
-	return cl.realmLogin(realm)
-}
-
-// sessionTGTDetails is a thread safe way to get the TGT and session key values for a realm
-func (cl *Client) sessionTGT(realm string) (tgt messages.Ticket, sessionKey types.EncryptionKey, err error) {
-	err = cl.ensureValidSession(realm)
-	if err != nil {
-		return
-	}
-	s, ok := cl.sessions.get(realm)
-	if !ok {
-		err = fmt.Errorf("could not find TGT session for %s", realm)
-		return
-	}
-	_, tgt, sessionKey = s.tgtDetails()
-	return
-}
-
-func (cl *Client) sessionTimes(realm string) (authTime, endTime, renewTime, sessionExp time.Time, err error) {
-	s, ok := cl.sessions.get(realm)
-	if !ok {
-		err = fmt.Errorf("could not find TGT session for %s", realm)
-		return
-	}
-	_, authTime, endTime, renewTime, sessionExp = s.timeDetails()
-	return
-}
-
-// spnRealm resolves the realm name of a service principal name
-func (cl *Client) spnRealm(spn types.PrincipalName) string {
-	return cl.Config.ResolveRealm(spn.NameString[len(spn.NameString)-1])
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/settings.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/client/settings.go
deleted file mode 100644
index 516c823..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/client/settings.go
+++ /dev/null
@@ -1,69 +0,0 @@
-package client
-
-import "log"
-
-// Settings holds optional client settings.
-type Settings struct {
-	disablePAFXFast         bool
-	assumePreAuthentication bool
-	preAuthEType            int32
-	logger                  *log.Logger
-}
-
-// NewSettings creates a new client settings struct.
-func NewSettings(settings ...func(*Settings)) *Settings {
-	s := new(Settings)
-	for _, set := range settings {
-		set(s)
-	}
-	return s
-}
-
-// DisablePAFXFAST used to configure the client to not use PA_FX_FAST.
-//
-// s := NewSettings(DisablePAFXFAST(true))
-func DisablePAFXFAST(b bool) func(*Settings) {
-	return func(s *Settings) {
-		s.disablePAFXFast = b
-	}
-}
-
-// DisablePAFXFAST indicates is the client should disable the use of PA_FX_FAST.
-func (s *Settings) DisablePAFXFAST() bool {
-	return s.disablePAFXFast
-}
-
-// AssumePreAuthentication used to configure the client to assume pre-authentication is required.
-//
-// s := NewSettings(AssumePreAuthentication(true))
-func AssumePreAuthentication(b bool) func(*Settings) {
-	return func(s *Settings) {
-		s.disablePAFXFast = b
-	}
-}
-
-// AssumePreAuthentication indicates if the client should proactively assume using pre-authentication.
-func (s *Settings) AssumePreAuthentication() bool {
-	return s.assumePreAuthentication
-}
-
-// Logger used to configure client with a logger.
-//
-// s := NewSettings(kt, Logger(l))
-func Logger(l *log.Logger) func(*Settings) {
-	return func(s *Settings) {
-		s.logger = l
-	}
-}
-
-// Logger returns the client logger instance.
-func (s *Settings) Logger() *log.Logger {
-	return s.logger
-}
-
-// Log will write to the service's logger if it is configured.
-func (cl *Client) Log(format string, v ...interface{}) {
-	if cl.settings.Logger() != nil {
-		cl.settings.Logger().Printf(format, v...)
-	}
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/config/hosts.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/config/hosts.go
deleted file mode 100644
index a58c234..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/config/hosts.go
+++ /dev/null
@@ -1,137 +0,0 @@
-package config
-
-import (
-	"fmt"
-	"math/rand"
-	"net"
-	"strconv"
-	"strings"
-
-	"gopkg.in/jcmturner/dnsutils.v1"
-)
-
-// GetKDCs returns the count of KDCs available and a map of KDC host names keyed on preference order.
-func (c *Config) GetKDCs(realm string, tcp bool) (int, map[int]string, error) {
-	if realm == "" {
-		realm = c.LibDefaults.DefaultRealm
-	}
-	kdcs := make(map[int]string)
-	var count int
-
-	// Use DNS to resolve kerberos SRV records if configured to do so in krb5.conf.
-	if c.LibDefaults.DNSLookupKDC {
-		proto := "udp"
-		if tcp {
-			proto = "tcp"
-		}
-		c, addrs, err := dnsutils.OrderedSRV("kerberos", proto, realm)
-		if err != nil {
-			return count, kdcs, err
-		}
-		if len(addrs) < 1 {
-			return count, kdcs, fmt.Errorf("no KDC SRV records found for realm %s", realm)
-		}
-		count = c
-		for k, v := range addrs {
-			kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port))
-		}
-	} else {
-		// Get the KDCs from the krb5.conf an order them randomly for preference.
-		var ks []string
-		for _, r := range c.Realms {
-			if r.Realm == realm {
-				ks = r.KDC
-				break
-			}
-		}
-		count = len(ks)
-		if count < 1 {
-			return count, kdcs, fmt.Errorf("no KDCs defined in configuration for realm %s", realm)
-		}
-		kdcs = randServOrder(ks)
-	}
-	return count, kdcs, nil
-}
-
-// GetKpasswdServers returns the count of kpasswd servers available and a map of kpasswd host names keyed on preference order.
-// https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html#realms - see kpasswd_server section
-func (c *Config) GetKpasswdServers(realm string, tcp bool) (int, map[int]string, error) {
-	kdcs := make(map[int]string)
-	var count int
-
-	// Use DNS to resolve kerberos SRV records if configured to do so in krb5.conf.
-	if c.LibDefaults.DNSLookupKDC {
-		proto := "udp"
-		if tcp {
-			proto = "tcp"
-		}
-		c, addrs, err := dnsutils.OrderedSRV("kpasswd", proto, realm)
-		if err != nil {
-			return count, kdcs, err
-		}
-		if c < 1 {
-			c, addrs, err = dnsutils.OrderedSRV("kerberos-adm", proto, realm)
-			if err != nil {
-				return count, kdcs, err
-			}
-		}
-		if len(addrs) < 1 {
-			return count, kdcs, fmt.Errorf("no kpasswd or kadmin SRV records found for realm %s", realm)
-		}
-		count = c
-		for k, v := range addrs {
-			kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port))
-		}
-	} else {
-		// Get the KDCs from the krb5.conf an order them randomly for preference.
-		var ks []string
-		var ka []string
-		for _, r := range c.Realms {
-			if r.Realm == realm {
-				ks = r.KPasswdServer
-				ka = r.AdminServer
-				break
-			}
-		}
-		if len(ks) < 1 {
-			for _, k := range ka {
-				h, _, err := net.SplitHostPort(k)
-				if err != nil {
-					continue
-				}
-				ks = append(ks, h+":464")
-			}
-		}
-		count = len(ks)
-		if count < 1 {
-			return count, kdcs, fmt.Errorf("no kpasswd or kadmin defined in configuration for realm %s", realm)
-		}
-		kdcs = randServOrder(ks)
-	}
-	return count, kdcs, nil
-}
-
-func randServOrder(ks []string) map[int]string {
-	kdcs := make(map[int]string)
-	count := len(ks)
-	i := 1
-	if count > 1 {
-		l := len(ks)
-		for l > 0 {
-			ri := rand.Intn(l)
-			kdcs[i] = ks[ri]
-			if l > 1 {
-				// Remove the entry from the source slice by swapping with the last entry and truncating
-				ks[len(ks)-1], ks[ri] = ks[ri], ks[len(ks)-1]
-				ks = ks[:len(ks)-1]
-				l = len(ks)
-			} else {
-				l = 0
-			}
-			i++
-		}
-	} else {
-		kdcs[i] = ks[0]
-	}
-	return kdcs
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/config/krb5conf.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/config/krb5conf.go
deleted file mode 100644
index 8efe92d..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/config/krb5conf.go
+++ /dev/null
@@ -1,726 +0,0 @@
-// Package config implements KRB5 client and service configuration as described at https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html
-package config
-
-import (
-	"bufio"
-	"encoding/hex"
-	"errors"
-	"fmt"
-	"io"
-	"net"
-	"os"
-	"os/user"
-	"regexp"
-	"strconv"
-	"strings"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// Config represents the KRB5 configuration.
-type Config struct {
-	LibDefaults LibDefaults
-	Realms      []Realm
-	DomainRealm DomainRealm
-	//CaPaths
-	//AppDefaults
-	//Plugins
-}
-
-// WeakETypeList is a list of encryption types that have been deemed weak.
-const WeakETypeList = "des-cbc-crc des-cbc-md4 des-cbc-md5 des-cbc-raw des3-cbc-raw des-hmac-sha1 arcfour-hmac-exp rc4-hmac-exp arcfour-hmac-md5-exp des"
-
-// NewConfig creates a new config struct instance.
-func NewConfig() *Config {
-	d := make(DomainRealm)
-	return &Config{
-		LibDefaults: newLibDefaults(),
-		DomainRealm: d,
-	}
-}
-
-// LibDefaults represents the [libdefaults] section of the configuration.
-type LibDefaults struct {
-	AllowWeakCrypto bool //default false
-	// ap_req_checksum_type int //unlikely to support this
-	Canonicalize bool          //default false
-	CCacheType   int           //default is 4. unlikely to implement older
-	Clockskew    time.Duration //max allowed skew in seconds, default 300
-	//Default_ccache_name string // default /tmp/krb5cc_%{uid} //Not implementing as will hold in memory
-	DefaultClientKeytabName string //default /usr/local/var/krb5/user/%{euid}/client.keytab
-	DefaultKeytabName       string //default /etc/krb5.keytab
-	DefaultRealm            string
-	DefaultTGSEnctypes      []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
-	DefaultTktEnctypes      []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
-	DefaultTGSEnctypeIDs    []int32  //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
-	DefaultTktEnctypeIDs    []int32  //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
-	DNSCanonicalizeHostname bool     //default true
-	DNSLookupKDC            bool     //default false
-	DNSLookupRealm          bool
-	ExtraAddresses          []net.IP       //Not implementing yet
-	Forwardable             bool           //default false
-	IgnoreAcceptorHostname  bool           //default false
-	K5LoginAuthoritative    bool           //default false
-	K5LoginDirectory        string         //default user's home directory. Must be owned by the user or root
-	KDCDefaultOptions       asn1.BitString //default 0x00000010 (KDC_OPT_RENEWABLE_OK)
-	KDCTimeSync             int            //default 1
-	//kdc_req_checksum_type int //unlikely to implement as for very old KDCs
-	NoAddresses         bool     //default true
-	PermittedEnctypes   []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4
-	PermittedEnctypeIDs []int32
-	//plugin_base_dir string //not supporting plugins
-	PreferredPreauthTypes []int         //default “17, 16, 15, 14”, which forces libkrb5 to attempt to use PKINIT if it is supported
-	Proxiable             bool          //default false
-	RDNS                  bool          //default true
-	RealmTryDomains       int           //default -1
-	RenewLifetime         time.Duration //default 0
-	SafeChecksumType      int           //default 8
-	TicketLifetime        time.Duration //default 1 day
-	UDPPreferenceLimit    int           // 1 means to always use tcp. MIT krb5 has a default value of 1465, and it prevents user setting more than 32700.
-	VerifyAPReqNofail     bool          //default false
-}
-
-// Create a new LibDefaults struct.
-func newLibDefaults() LibDefaults {
-	uid := "0"
-	var hdir string
-	usr, _ := user.Current()
-	if usr != nil {
-		uid = usr.Uid
-		hdir = usr.HomeDir
-	}
-	opts := asn1.BitString{}
-	opts.Bytes, _ = hex.DecodeString("00000010")
-	opts.BitLength = len(opts.Bytes) * 8
-	return LibDefaults{
-		CCacheType:              4,
-		Clockskew:               time.Duration(300) * time.Second,
-		DefaultClientKeytabName: fmt.Sprintf("/usr/local/var/krb5/user/%s/client.keytab", uid),
-		DefaultKeytabName:       "/etc/krb5.keytab",
-		DefaultTGSEnctypes:      []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
-		DefaultTktEnctypes:      []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
-		DNSCanonicalizeHostname: true,
-		K5LoginDirectory:        hdir,
-		KDCDefaultOptions:       opts,
-		KDCTimeSync:             1,
-		NoAddresses:             true,
-		PermittedEnctypes:       []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"},
-		RDNS:                    true,
-		RealmTryDomains:         -1,
-		SafeChecksumType:        8,
-		TicketLifetime:          time.Duration(24) * time.Hour,
-		UDPPreferenceLimit:      1465,
-		PreferredPreauthTypes:   []int{17, 16, 15, 14},
-	}
-}
-
-// Parse the lines of the [libdefaults] section of the configuration into the LibDefaults struct.
-func (l *LibDefaults) parseLines(lines []string) error {
-	for _, line := range lines {
-		//Remove comments after the values
-		if idx := strings.IndexAny(line, "#;"); idx != -1 {
-			line = line[:idx]
-		}
-		line = strings.TrimSpace(line)
-		if line == "" {
-			continue
-		}
-		if !strings.Contains(line, "=") {
-			return InvalidErrorf("libdefaults section line (%s)", line)
-		}
-
-		p := strings.Split(line, "=")
-		key := strings.TrimSpace(strings.ToLower(p[0]))
-		switch key {
-		case "allow_weak_crypto":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.AllowWeakCrypto = v
-		case "canonicalize":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.Canonicalize = v
-		case "ccache_type":
-			p[1] = strings.TrimSpace(p[1])
-			v, err := strconv.ParseUint(p[1], 10, 32)
-			if err != nil || v < 0 || v > 4 {
-				return InvalidErrorf("libdefaults section line (%s)", line)
-			}
-			l.CCacheType = int(v)
-		case "clockskew":
-			d, err := parseDuration(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.Clockskew = d
-		case "default_client_keytab_name":
-			l.DefaultClientKeytabName = strings.TrimSpace(p[1])
-		case "default_keytab_name":
-			l.DefaultKeytabName = strings.TrimSpace(p[1])
-		case "default_realm":
-			l.DefaultRealm = strings.TrimSpace(p[1])
-		case "default_tgs_enctypes":
-			l.DefaultTGSEnctypes = strings.Fields(p[1])
-		case "default_tkt_enctypes":
-			l.DefaultTktEnctypes = strings.Fields(p[1])
-		case "dns_canonicalize_hostname":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.DNSCanonicalizeHostname = v
-		case "dns_lookup_kdc":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.DNSLookupKDC = v
-		case "dns_lookup_realm":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.DNSLookupRealm = v
-		case "extra_addresses":
-			ipStr := strings.TrimSpace(p[1])
-			for _, ip := range strings.Split(ipStr, ",") {
-				if eip := net.ParseIP(ip); eip != nil {
-					l.ExtraAddresses = append(l.ExtraAddresses, eip)
-				}
-			}
-		case "forwardable":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.Forwardable = v
-		case "ignore_acceptor_hostname":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.IgnoreAcceptorHostname = v
-		case "k5login_authoritative":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.K5LoginAuthoritative = v
-		case "k5login_directory":
-			l.K5LoginDirectory = strings.TrimSpace(p[1])
-		case "kdc_default_options":
-			v := strings.TrimSpace(p[1])
-			v = strings.Replace(v, "0x", "", -1)
-			b, err := hex.DecodeString(v)
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.KDCDefaultOptions.Bytes = b
-			l.KDCDefaultOptions.BitLength = len(b) * 8
-		case "kdc_timesync":
-			p[1] = strings.TrimSpace(p[1])
-			v, err := strconv.ParseInt(p[1], 10, 32)
-			if err != nil || v < 0 {
-				return InvalidErrorf("libdefaults section line (%s)", line)
-			}
-			l.KDCTimeSync = int(v)
-		case "noaddresses":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.NoAddresses = v
-		case "permitted_enctypes":
-			l.PermittedEnctypes = strings.Fields(p[1])
-		case "preferred_preauth_types":
-			p[1] = strings.TrimSpace(p[1])
-			t := strings.Split(p[1], ",")
-			var v []int
-			for _, s := range t {
-				i, err := strconv.ParseInt(s, 10, 32)
-				if err != nil {
-					return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-				}
-				v = append(v, int(i))
-			}
-			l.PreferredPreauthTypes = v
-		case "proxiable":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.Proxiable = v
-		case "rdns":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.RDNS = v
-		case "realm_try_domains":
-			p[1] = strings.TrimSpace(p[1])
-			v, err := strconv.ParseInt(p[1], 10, 32)
-			if err != nil || v < -1 {
-				return InvalidErrorf("libdefaults section line (%s)", line)
-			}
-			l.RealmTryDomains = int(v)
-		case "renew_lifetime":
-			d, err := parseDuration(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.RenewLifetime = d
-		case "safe_checksum_type":
-			p[1] = strings.TrimSpace(p[1])
-			v, err := strconv.ParseInt(p[1], 10, 32)
-			if err != nil || v < 0 {
-				return InvalidErrorf("libdefaults section line (%s)", line)
-			}
-			l.SafeChecksumType = int(v)
-		case "ticket_lifetime":
-			d, err := parseDuration(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.TicketLifetime = d
-		case "udp_preference_limit":
-			p[1] = strings.TrimSpace(p[1])
-			v, err := strconv.ParseUint(p[1], 10, 32)
-			if err != nil || v > 32700 {
-				return InvalidErrorf("libdefaults section line (%s)", line)
-			}
-			l.UDPPreferenceLimit = int(v)
-		case "verify_ap_req_nofail":
-			v, err := parseBoolean(p[1])
-			if err != nil {
-				return InvalidErrorf("libdefaults section line (%s): %v", line, err)
-			}
-			l.VerifyAPReqNofail = v
-		default:
-			//Ignore the line
-			continue
-		}
-	}
-	l.DefaultTGSEnctypeIDs = parseETypes(l.DefaultTGSEnctypes, l.AllowWeakCrypto)
-	l.DefaultTktEnctypeIDs = parseETypes(l.DefaultTktEnctypes, l.AllowWeakCrypto)
-	l.PermittedEnctypeIDs = parseETypes(l.PermittedEnctypes, l.AllowWeakCrypto)
-	return nil
-}
-
-// Realm represents an entry in the [realms] section of the configuration.
-type Realm struct {
-	Realm       string
-	AdminServer []string
-	//auth_to_local //Not implementing for now
-	//auth_to_local_names //Not implementing for now
-	DefaultDomain string
-	KDC           []string
-	KPasswdServer []string //default admin_server:464
-	MasterKDC     []string
-}
-
-// Parse the lines of a [realms] entry into the Realm struct.
-func (r *Realm) parseLines(name string, lines []string) (err error) {
-	r.Realm = name
-	var adminServerFinal bool
-	var KDCFinal bool
-	var kpasswdServerFinal bool
-	var masterKDCFinal bool
-	var ignore bool
-	var c int // counts the depth of blocks within brackets { }
-	for _, line := range lines {
-		if ignore && c > 0 && !strings.Contains(line, "{") && !strings.Contains(line, "}") {
-			continue
-		}
-		//Remove comments after the values
-		if idx := strings.IndexAny(line, "#;"); idx != -1 {
-			line = line[:idx]
-		}
-		line = strings.TrimSpace(line)
-		if line == "" {
-			continue
-		}
-		if !strings.Contains(line, "=") && !strings.Contains(line, "}") {
-			return InvalidErrorf("realms section line (%s)", line)
-		}
-		if strings.Contains(line, "v4_") {
-			ignore = true
-			err = UnsupportedDirective{"v4 configurations are not supported"}
-		}
-		if strings.Contains(line, "{") {
-			c++
-			if ignore {
-				continue
-			}
-		}
-		if strings.Contains(line, "}") {
-			c--
-			if c < 0 {
-				return InvalidErrorf("unpaired curly brackets")
-			}
-			if ignore {
-				if c < 1 {
-					c = 0
-					ignore = false
-				}
-				continue
-			}
-		}
-
-		p := strings.Split(line, "=")
-		key := strings.TrimSpace(strings.ToLower(p[0]))
-		v := strings.TrimSpace(p[1])
-		switch key {
-		case "admin_server":
-			appendUntilFinal(&r.AdminServer, v, &adminServerFinal)
-		case "default_domain":
-			r.DefaultDomain = v
-		case "kdc":
-			if !strings.Contains(v, ":") {
-				// No port number specified default to 88
-				if strings.HasSuffix(v, `*`) {
-					v = strings.TrimSpace(strings.TrimSuffix(v, `*`)) + ":88*"
-				} else {
-					v = strings.TrimSpace(v) + ":88"
-				}
-			}
-			appendUntilFinal(&r.KDC, v, &KDCFinal)
-		case "kpasswd_server":
-			appendUntilFinal(&r.KPasswdServer, v, &kpasswdServerFinal)
-		case "master_kdc":
-			appendUntilFinal(&r.MasterKDC, v, &masterKDCFinal)
-		default:
-			//Ignore the line
-			continue
-		}
-	}
-	//default for Kpasswd_server = admin_server:464
-	if len(r.KPasswdServer) < 1 {
-		for _, a := range r.AdminServer {
-			s := strings.Split(a, ":")
-			r.KPasswdServer = append(r.KPasswdServer, s[0]+":464")
-		}
-	}
-	return
-}
-
-// Parse the lines of the [realms] section of the configuration into an slice of Realm structs.
-func parseRealms(lines []string) (realms []Realm, err error) {
-	var name string
-	var start int
-	var c int
-	for i, l := range lines {
-		//Remove comments after the values
-		if idx := strings.IndexAny(l, "#;"); idx != -1 {
-			l = l[:idx]
-		}
-		l = strings.TrimSpace(l)
-		if l == "" {
-			continue
-		}
-		//if strings.Contains(l, "v4_") {
-		//	return nil, errors.New("v4 configurations are not supported in Realms section")
-		//}
-		if strings.Contains(l, "{") {
-			c++
-			if !strings.Contains(l, "=") {
-				return nil, fmt.Errorf("realm configuration line invalid: %s", l)
-			}
-			if c == 1 {
-				start = i
-				p := strings.Split(l, "=")
-				name = strings.TrimSpace(p[0])
-			}
-		}
-		if strings.Contains(l, "}") {
-			if c < 1 {
-				// but not started a block!!!
-				return nil, errors.New("invalid Realms section in configuration")
-			}
-			c--
-			if c == 0 {
-				var r Realm
-				e := r.parseLines(name, lines[start+1:i])
-				if e != nil {
-					if _, ok := e.(UnsupportedDirective); !ok {
-						err = e
-						return
-					}
-					err = e
-				}
-				realms = append(realms, r)
-			}
-		}
-	}
-	return
-}
-
-// DomainRealm maps the domains to realms representing the [domain_realm] section of the configuration.
-type DomainRealm map[string]string
-
-// Parse the lines of the [domain_realm] section of the configuration and add to the mapping.
-func (d *DomainRealm) parseLines(lines []string) error {
-	for _, line := range lines {
-		//Remove comments after the values
-		if idx := strings.IndexAny(line, "#;"); idx != -1 {
-			line = line[:idx]
-		}
-		if strings.TrimSpace(line) == "" {
-			continue
-		}
-		if !strings.Contains(line, "=") {
-			return InvalidErrorf("realm line (%s)", line)
-		}
-		p := strings.Split(line, "=")
-		domain := strings.TrimSpace(strings.ToLower(p[0]))
-		realm := strings.TrimSpace(p[1])
-		d.addMapping(domain, realm)
-	}
-	return nil
-}
-
-// Add a domain to realm mapping.
-func (d *DomainRealm) addMapping(domain, realm string) {
-	(*d)[domain] = realm
-}
-
-// Delete a domain to realm mapping.
-func (d *DomainRealm) deleteMapping(domain, realm string) {
-	delete(*d, domain)
-}
-
-// ResolveRealm resolves the kerberos realm for the specified domain name from the domain to realm mapping.
-// The most specific mapping is returned.
-func (c *Config) ResolveRealm(domainName string) string {
-	domainName = strings.TrimSuffix(domainName, ".")
-
-	// Try to match the entire hostname first
-	if r, ok := c.DomainRealm[domainName]; ok {
-		return r
-	}
-
-	// Try to match all DNS domain parts
-	periods := strings.Count(domainName, ".") + 1
-	for i := 2; i <= periods; i++ {
-		z := strings.SplitN(domainName, ".", i)
-		if r, ok := c.DomainRealm["."+z[len(z)-1]]; ok {
-			return r
-		}
-	}
-	return c.LibDefaults.DefaultRealm
-}
-
-// Load the KRB5 configuration from the specified file path.
-func Load(cfgPath string) (*Config, error) {
-	fh, err := os.Open(cfgPath)
-	if err != nil {
-		return nil, errors.New("configuration file could not be opened: " + cfgPath + " " + err.Error())
-	}
-	defer fh.Close()
-	scanner := bufio.NewScanner(fh)
-	return NewConfigFromScanner(scanner)
-}
-
-// NewConfigFromString creates a new Config struct from a string.
-func NewConfigFromString(s string) (*Config, error) {
-	reader := strings.NewReader(s)
-	return NewConfigFromReader(reader)
-}
-
-// NewConfigFromReader creates a new Config struct from an io.Reader.
-func NewConfigFromReader(r io.Reader) (*Config, error) {
-	scanner := bufio.NewScanner(r)
-	return NewConfigFromScanner(scanner)
-}
-
-// NewConfigFromScanner creates a new Config struct from a bufio.Scanner.
-func NewConfigFromScanner(scanner *bufio.Scanner) (*Config, error) {
-	c := NewConfig()
-	var e error
-	sections := make(map[int]string)
-	var sectionLineNum []int
-	var lines []string
-	for scanner.Scan() {
-		// Skip comments and blank lines
-		if matched, _ := regexp.MatchString(`^\s*(#|;|\n)`, scanner.Text()); matched {
-			continue
-		}
-		if matched, _ := regexp.MatchString(`^\s*\[libdefaults\]\s*`, scanner.Text()); matched {
-			sections[len(lines)] = "libdefaults"
-			sectionLineNum = append(sectionLineNum, len(lines))
-			continue
-		}
-		if matched, _ := regexp.MatchString(`^\s*\[realms\]\s*`, scanner.Text()); matched {
-			sections[len(lines)] = "realms"
-			sectionLineNum = append(sectionLineNum, len(lines))
-			continue
-		}
-		if matched, _ := regexp.MatchString(`^\s*\[domain_realm\]\s*`, scanner.Text()); matched {
-			sections[len(lines)] = "domain_realm"
-			sectionLineNum = append(sectionLineNum, len(lines))
-			continue
-		}
-		if matched, _ := regexp.MatchString(`^\s*\[.*\]\s*`, scanner.Text()); matched {
-			sections[len(lines)] = "unknown_section"
-			sectionLineNum = append(sectionLineNum, len(lines))
-			continue
-		}
-		lines = append(lines, scanner.Text())
-	}
-	for i, start := range sectionLineNum {
-		var end int
-		if i+1 >= len(sectionLineNum) {
-			end = len(lines)
-		} else {
-			end = sectionLineNum[i+1]
-		}
-		switch section := sections[start]; section {
-		case "libdefaults":
-			err := c.LibDefaults.parseLines(lines[start:end])
-			if err != nil {
-				if _, ok := err.(UnsupportedDirective); !ok {
-					return nil, fmt.Errorf("error processing libdefaults section: %v", err)
-				}
-				e = err
-			}
-		case "realms":
-			realms, err := parseRealms(lines[start:end])
-			if err != nil {
-				if _, ok := err.(UnsupportedDirective); !ok {
-					return nil, fmt.Errorf("error processing realms section: %v", err)
-				}
-				e = err
-			}
-			c.Realms = realms
-		case "domain_realm":
-			err := c.DomainRealm.parseLines(lines[start:end])
-			if err != nil {
-				if _, ok := err.(UnsupportedDirective); !ok {
-					return nil, fmt.Errorf("error processing domaain_realm section: %v", err)
-				}
-				e = err
-			}
-		default:
-			continue
-		}
-	}
-	return c, e
-}
-
-// Parse a space delimited list of ETypes into a list of EType numbers optionally filtering out weak ETypes.
-func parseETypes(s []string, w bool) []int32 {
-	var eti []int32
-	for _, et := range s {
-		if !w {
-			var weak bool
-			for _, wet := range strings.Fields(WeakETypeList) {
-				if et == wet {
-					weak = true
-					break
-				}
-			}
-			if weak {
-				continue
-			}
-		}
-		i := etypeID.EtypeSupported(et)
-		if i != 0 {
-			eti = append(eti, i)
-		}
-	}
-	return eti
-}
-
-// Parse a time duration string in the configuration to a golang time.Duration.
-func parseDuration(s string) (time.Duration, error) {
-	s = strings.Replace(strings.TrimSpace(s), " ", "", -1)
-
-	// handle Nd[NmNs]
-	if strings.Contains(s, "d") {
-		ds := strings.SplitN(s, "d", 2)
-		dn, err := strconv.ParseUint(ds[0], 10, 32)
-		if err != nil {
-			return time.Duration(0), errors.New("invalid time duration")
-		}
-		d := time.Duration(dn*24) * time.Hour
-		if ds[1] != "" {
-			dp, err := time.ParseDuration(ds[1])
-			if err != nil {
-				return time.Duration(0), errors.New("invalid time duration")
-			}
-			d = d + dp
-		}
-		return d, nil
-	}
-
-	// handle Nm[Ns]
-	d, err := time.ParseDuration(s)
-	if err == nil {
-		return d, nil
-	}
-
-	// handle N
-	v, err := strconv.ParseUint(s, 10, 32)
-	if err == nil && v > 0 {
-		return time.Duration(v) * time.Second, nil
-	}
-
-	// handle h:m[:s]
-	if strings.Contains(s, ":") {
-		t := strings.Split(s, ":")
-		if 2 > len(t) || len(t) > 3 {
-			return time.Duration(0), errors.New("invalid time duration value")
-		}
-		var i []int
-		for _, n := range t {
-			j, err := strconv.ParseInt(n, 10, 16)
-			if err != nil {
-				return time.Duration(0), errors.New("invalid time duration value")
-			}
-			i = append(i, int(j))
-		}
-		d := time.Duration(i[0])*time.Hour + time.Duration(i[1])*time.Minute
-		if len(i) == 3 {
-			d = d + time.Duration(i[2])*time.Second
-		}
-		return d, nil
-	}
-	return time.Duration(0), errors.New("invalid time duration value")
-}
-
-// Parse possible boolean values to golang bool.
-func parseBoolean(s string) (bool, error) {
-	s = strings.TrimSpace(s)
-	v, err := strconv.ParseBool(s)
-	if err == nil {
-		return v, nil
-	}
-	switch strings.ToLower(s) {
-	case "yes":
-		return true, nil
-	case "y":
-		return true, nil
-	case "no":
-		return false, nil
-	case "n":
-		return false, nil
-	}
-	return false, errors.New("invalid boolean value")
-}
-
-// Parse array of strings but stop if an asterisk is placed at the end of a line.
-func appendUntilFinal(s *[]string, value string, final *bool) {
-	if *final {
-		return
-	}
-	if last := len(value) - 1; last >= 0 && value[last] == '*' {
-		*final = true
-		value = value[:len(value)-1]
-	}
-	*s = append(*s, value)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/ccache.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/ccache.go
deleted file mode 100644
index 98ec29b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/ccache.go
+++ /dev/null
@@ -1,348 +0,0 @@
-package credentials
-
-import (
-	"bytes"
-	"encoding/binary"
-	"errors"
-	"io/ioutil"
-	"strings"
-	"time"
-	"unsafe"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-const (
-	headerFieldTagKDCOffset = 1
-)
-
-// The first byte of the file always has the value 5.
-// The value of the second byte contains the version number (1 through 4)
-// Versions 1 and 2 of the file format use native byte order for integer representations.
-// Versions 3 and 4 always use big-endian byte order
-// After the two-byte version indicator, the file has three parts:
-//   1) the header (in version 4 only)
-//   2) the default principal name
-//   3) a sequence of credentials
-
-// CCache is the file credentials cache as define here: https://web.mit.edu/kerberos/krb5-latest/doc/formats/ccache_file_format.html
-type CCache struct {
-	Version          uint8
-	Header           header
-	DefaultPrincipal principal
-	Credentials      []*Credential
-	Path             string
-}
-
-type header struct {
-	length uint16
-	fields []headerField
-}
-
-type headerField struct {
-	tag    uint16
-	length uint16
-	value  []byte
-}
-
-// Credential cache entry principal struct.
-type principal struct {
-	Realm         string
-	PrincipalName types.PrincipalName
-}
-
-// Credential holds a Kerberos client's ccache credential information.
-type Credential struct {
-	Client       principal
-	Server       principal
-	Key          types.EncryptionKey
-	AuthTime     time.Time
-	StartTime    time.Time
-	EndTime      time.Time
-	RenewTill    time.Time
-	IsSKey       bool
-	TicketFlags  asn1.BitString
-	Addresses    []types.HostAddress
-	AuthData     []types.AuthorizationDataEntry
-	Ticket       []byte
-	SecondTicket []byte
-}
-
-// LoadCCache loads a credential cache file into a CCache type.
-func LoadCCache(cpath string) (*CCache, error) {
-	c := new(CCache)
-	b, err := ioutil.ReadFile(cpath)
-	if err != nil {
-		return c, err
-	}
-	err = c.Unmarshal(b)
-	return c, err
-}
-
-// Unmarshal a byte slice of credential cache data into CCache type.
-func (c *CCache) Unmarshal(b []byte) error {
-	p := 0
-	//The first byte of the file always has the value 5
-	if int8(b[p]) != 5 {
-		return errors.New("Invalid credential cache data. First byte does not equal 5")
-	}
-	p++
-	//Get credential cache version
-	//The second byte contains the version number (1 to 4)
-	c.Version = b[p]
-	if c.Version < 1 || c.Version > 4 {
-		return errors.New("Invalid credential cache data. Keytab version is not within 1 to 4")
-	}
-	p++
-	//Version 1 or 2 of the file format uses native byte order for integer representations. Versions 3 & 4 always uses big-endian byte order
-	var endian binary.ByteOrder
-	endian = binary.BigEndian
-	if (c.Version == 1 || c.Version == 2) && isNativeEndianLittle() {
-		endian = binary.LittleEndian
-	}
-	if c.Version == 4 {
-		err := parseHeader(b, &p, c, &endian)
-		if err != nil {
-			return err
-		}
-	}
-	c.DefaultPrincipal = parsePrincipal(b, &p, c, &endian)
-	for p < len(b) {
-		cred, err := parseCredential(b, &p, c, &endian)
-		if err != nil {
-			return err
-		}
-		c.Credentials = append(c.Credentials, cred)
-	}
-	return nil
-}
-
-func parseHeader(b []byte, p *int, c *CCache, e *binary.ByteOrder) error {
-	if c.Version != 4 {
-		return errors.New("Credentials cache version is not 4 so there is no header to parse.")
-	}
-	h := header{}
-	h.length = uint16(readInt16(b, p, e))
-	for *p <= int(h.length) {
-		f := headerField{}
-		f.tag = uint16(readInt16(b, p, e))
-		f.length = uint16(readInt16(b, p, e))
-		f.value = b[*p : *p+int(f.length)]
-		*p += int(f.length)
-		if !f.valid() {
-			return errors.New("Invalid credential cache header found")
-		}
-		h.fields = append(h.fields, f)
-	}
-	c.Header = h
-	return nil
-}
-
-// Parse the Keytab bytes of a principal into a Keytab entry's principal.
-func parsePrincipal(b []byte, p *int, c *CCache, e *binary.ByteOrder) (princ principal) {
-	if c.Version != 1 {
-		//Name Type is omitted in version 1
-		princ.PrincipalName.NameType = readInt32(b, p, e)
-	}
-	nc := int(readInt32(b, p, e))
-	if c.Version == 1 {
-		//In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2
-		nc--
-	}
-	lenRealm := readInt32(b, p, e)
-	princ.Realm = string(readBytes(b, p, int(lenRealm), e))
-	for i := 0; i < nc; i++ {
-		l := readInt32(b, p, e)
-		princ.PrincipalName.NameString = append(princ.PrincipalName.NameString, string(readBytes(b, p, int(l), e)))
-	}
-	return princ
-}
-
-func parseCredential(b []byte, p *int, c *CCache, e *binary.ByteOrder) (cred *Credential, err error) {
-	cred = new(Credential)
-	cred.Client = parsePrincipal(b, p, c, e)
-	cred.Server = parsePrincipal(b, p, c, e)
-	key := types.EncryptionKey{}
-	key.KeyType = int32(readInt16(b, p, e))
-	if c.Version == 3 {
-		//repeated twice in version 3
-		key.KeyType = int32(readInt16(b, p, e))
-	}
-	key.KeyValue = readData(b, p, e)
-	cred.Key = key
-	cred.AuthTime = readTimestamp(b, p, e)
-	cred.StartTime = readTimestamp(b, p, e)
-	cred.EndTime = readTimestamp(b, p, e)
-	cred.RenewTill = readTimestamp(b, p, e)
-	if ik := readInt8(b, p, e); ik == 0 {
-		cred.IsSKey = false
-	} else {
-		cred.IsSKey = true
-	}
-	cred.TicketFlags = types.NewKrbFlags()
-	cred.TicketFlags.Bytes = readBytes(b, p, 4, e)
-	l := int(readInt32(b, p, e))
-	cred.Addresses = make([]types.HostAddress, l, l)
-	for i := range cred.Addresses {
-		cred.Addresses[i] = readAddress(b, p, e)
-	}
-	l = int(readInt32(b, p, e))
-	cred.AuthData = make([]types.AuthorizationDataEntry, l, l)
-	for i := range cred.AuthData {
-		cred.AuthData[i] = readAuthDataEntry(b, p, e)
-	}
-	cred.Ticket = readData(b, p, e)
-	cred.SecondTicket = readData(b, p, e)
-	return
-}
-
-// GetClientPrincipalName returns a PrincipalName type for the client the credentials cache is for.
-func (c *CCache) GetClientPrincipalName() types.PrincipalName {
-	return c.DefaultPrincipal.PrincipalName
-}
-
-// GetClientRealm returns the reals of the client the credentials cache is for.
-func (c *CCache) GetClientRealm() string {
-	return c.DefaultPrincipal.Realm
-}
-
-// GetClientCredentials returns a Credentials object representing the client of the credentials cache.
-func (c *CCache) GetClientCredentials() *Credentials {
-	return &Credentials{
-		username: c.DefaultPrincipal.PrincipalName.PrincipalNameString(),
-		realm:    c.GetClientRealm(),
-		cname:    c.DefaultPrincipal.PrincipalName,
-	}
-}
-
-// Contains tests if the cache contains a credential for the provided server PrincipalName
-func (c *CCache) Contains(p types.PrincipalName) bool {
-	for _, cred := range c.Credentials {
-		if cred.Server.PrincipalName.Equal(p) {
-			return true
-		}
-	}
-	return false
-}
-
-// GetEntry returns a specific credential for the PrincipalName provided.
-func (c *CCache) GetEntry(p types.PrincipalName) (*Credential, bool) {
-	cred := new(Credential)
-	var found bool
-	for i := range c.Credentials {
-		if c.Credentials[i].Server.PrincipalName.Equal(p) {
-			cred = c.Credentials[i]
-			found = true
-			break
-		}
-	}
-	if !found {
-		return cred, false
-	}
-	return cred, true
-}
-
-// GetEntries filters out configuration entries an returns a slice of credentials.
-func (c *CCache) GetEntries() []*Credential {
-	creds := make([]*Credential, 0)
-	for _, cred := range c.Credentials {
-		// Filter out configuration entries
-		if strings.HasPrefix(cred.Server.Realm, "X-CACHECONF") {
-			continue
-		}
-		creds = append(creds, cred)
-	}
-	return creds
-}
-
-func (h *headerField) valid() bool {
-	// At this time there is only one defined header field.
-	// Its tag value is 1, its length is always 8.
-	// Its contents are two 32-bit integers giving the seconds and microseconds
-	// of the time offset of the KDC relative to the client.
-	// Adding this offset to the current time on the client should give the current time on the KDC, if that offset has not changed since the initial authentication.
-
-	// Done as a switch in case other tag values are added in the future.
-	switch h.tag {
-	case headerFieldTagKDCOffset:
-		if h.length != 8 || len(h.value) != 8 {
-			return false
-		}
-		return true
-	}
-	return false
-}
-
-func readData(b []byte, p *int, e *binary.ByteOrder) []byte {
-	l := readInt32(b, p, e)
-	return readBytes(b, p, int(l), e)
-}
-
-func readAddress(b []byte, p *int, e *binary.ByteOrder) types.HostAddress {
-	a := types.HostAddress{}
-	a.AddrType = int32(readInt16(b, p, e))
-	a.Address = readData(b, p, e)
-	return a
-}
-
-func readAuthDataEntry(b []byte, p *int, e *binary.ByteOrder) types.AuthorizationDataEntry {
-	a := types.AuthorizationDataEntry{}
-	a.ADType = int32(readInt16(b, p, e))
-	a.ADData = readData(b, p, e)
-	return a
-}
-
-// Read bytes representing a timestamp.
-func readTimestamp(b []byte, p *int, e *binary.ByteOrder) time.Time {
-	return time.Unix(int64(readInt32(b, p, e)), 0)
-}
-
-// Read bytes representing an eight bit integer.
-func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8) {
-	buf := bytes.NewBuffer(b[*p : *p+1])
-	binary.Read(buf, *e, &i)
-	*p++
-	return
-}
-
-// Read bytes representing a sixteen bit integer.
-func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16) {
-	buf := bytes.NewBuffer(b[*p : *p+2])
-	binary.Read(buf, *e, &i)
-	*p += 2
-	return
-}
-
-// Read bytes representing a thirty two bit integer.
-func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32) {
-	buf := bytes.NewBuffer(b[*p : *p+4])
-	binary.Read(buf, *e, &i)
-	*p += 4
-	return
-}
-
-func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) []byte {
-	buf := bytes.NewBuffer(b[*p : *p+s])
-	r := make([]byte, s)
-	binary.Read(buf, *e, &r)
-	*p += s
-	return r
-}
-
-func isNativeEndianLittle() bool {
-	var x = 0x012345678
-	var p = unsafe.Pointer(&x)
-	var bp = (*[4]byte)(p)
-
-	var endian bool
-	if 0x01 == bp[0] {
-		endian = false
-	} else if (0x78 & 0xff) == (bp[0] & 0xff) {
-		endian = true
-	} else {
-		// Default to big endian
-		endian = false
-	}
-	return endian
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/credentials.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/credentials.go
deleted file mode 100644
index 62acab7..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/credentials/credentials.go
+++ /dev/null
@@ -1,309 +0,0 @@
-// Package credentials provides credentials management for Kerberos 5 authentication.
-package credentials
-
-import (
-	"time"
-
-	"github.com/hashicorp/go-uuid"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-	"gopkg.in/jcmturner/gokrb5.v7/keytab"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-const (
-	// AttributeKeyADCredentials assigned number for AD credentials.
-	AttributeKeyADCredentials = "gokrb5AttributeKeyADCredentials"
-)
-
-// Credentials struct for a user.
-// Contains either a keytab, password or both.
-// Keytabs are used over passwords if both are defined.
-type Credentials struct {
-	username    string
-	displayName string
-	realm       string
-	cname       types.PrincipalName
-	keytab      *keytab.Keytab
-	password    string
-	attributes  map[string]interface{}
-	validUntil  time.Time
-
-	authenticated   bool
-	human           bool
-	authTime        time.Time
-	groupMembership map[string]bool
-	sessionID       string
-}
-
-// ADCredentials contains information obtained from the PAC.
-type ADCredentials struct {
-	EffectiveName       string
-	FullName            string
-	UserID              int
-	PrimaryGroupID      int
-	LogOnTime           time.Time
-	LogOffTime          time.Time
-	PasswordLastSet     time.Time
-	GroupMembershipSIDs []string
-	LogonDomainName     string
-	LogonDomainID       string
-	LogonServer         string
-}
-
-// New creates a new Credentials instance.
-func New(username string, realm string) *Credentials {
-	uid, err := uuid.GenerateUUID()
-	if err != nil {
-		uid = "00unique-sess-ions-uuid-unavailable0"
-	}
-	return &Credentials{
-		username:        username,
-		displayName:     username,
-		realm:           realm,
-		cname:           types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, username),
-		keytab:          keytab.New(),
-		attributes:      make(map[string]interface{}),
-		groupMembership: make(map[string]bool),
-		sessionID:       uid,
-		human:           true,
-	}
-}
-
-// NewFromPrincipalName creates a new Credentials instance with the user details provides as a PrincipalName type.
-func NewFromPrincipalName(cname types.PrincipalName, realm string) *Credentials {
-	uid, err := uuid.GenerateUUID()
-	if err != nil {
-		uid = "00unique-sess-ions-uuid-unavailable0"
-	}
-	return &Credentials{
-		username:        cname.PrincipalNameString(),
-		displayName:     cname.PrincipalNameString(),
-		realm:           realm,
-		cname:           cname,
-		keytab:          keytab.New(),
-		attributes:      make(map[string]interface{}),
-		groupMembership: make(map[string]bool),
-		sessionID:       uid,
-		human:           true,
-	}
-}
-
-// WithKeytab sets the Keytab in the Credentials struct.
-func (c *Credentials) WithKeytab(kt *keytab.Keytab) *Credentials {
-	c.keytab = kt
-	c.password = ""
-	return c
-}
-
-// Keytab returns the credential's Keytab.
-func (c *Credentials) Keytab() *keytab.Keytab {
-	return c.keytab
-}
-
-// HasKeytab queries if the Credentials has a keytab defined.
-func (c *Credentials) HasKeytab() bool {
-	if c.keytab != nil && len(c.keytab.Entries) > 0 {
-		return true
-	}
-	return false
-}
-
-// WithPassword sets the password in the Credentials struct.
-func (c *Credentials) WithPassword(password string) *Credentials {
-	c.password = password
-	c.keytab = keytab.New() // clear any keytab
-	return c
-}
-
-// Password returns the credential's password.
-func (c *Credentials) Password() string {
-	return c.password
-}
-
-// HasPassword queries if the Credentials has a password defined.
-func (c *Credentials) HasPassword() bool {
-	if c.password != "" {
-		return true
-	}
-	return false
-}
-
-// SetValidUntil sets the expiry time of the credentials
-func (c *Credentials) SetValidUntil(t time.Time) {
-	c.validUntil = t
-}
-
-// SetADCredentials adds ADCredentials attributes to the credentials
-func (c *Credentials) SetADCredentials(a ADCredentials) {
-	c.SetAttribute(AttributeKeyADCredentials, a)
-	if a.FullName != "" {
-		c.SetDisplayName(a.FullName)
-	}
-	if a.EffectiveName != "" {
-		c.SetUserName(a.EffectiveName)
-	}
-	for i := range a.GroupMembershipSIDs {
-		c.AddAuthzAttribute(a.GroupMembershipSIDs[i])
-	}
-}
-
-// Methods to implement goidentity.Identity interface
-
-// UserName returns the credential's username.
-func (c *Credentials) UserName() string {
-	return c.username
-}
-
-// SetUserName sets the username value on the credential.
-func (c *Credentials) SetUserName(s string) {
-	c.username = s
-}
-
-// CName returns the credential's client principal name.
-func (c *Credentials) CName() types.PrincipalName {
-	return c.cname
-}
-
-// SetCName sets the client principal name on the credential.
-func (c *Credentials) SetCName(pn types.PrincipalName) {
-	c.cname = pn
-}
-
-// Domain returns the credential's domain.
-func (c *Credentials) Domain() string {
-	return c.realm
-}
-
-// SetDomain sets the domain value on the credential.
-func (c *Credentials) SetDomain(s string) {
-	c.realm = s
-}
-
-// Realm returns the credential's realm. Same as the domain.
-func (c *Credentials) Realm() string {
-	return c.Domain()
-}
-
-// SetRealm sets the realm value on the credential. Same as the domain
-func (c *Credentials) SetRealm(s string) {
-	c.SetDomain(s)
-}
-
-// DisplayName returns the credential's display name.
-func (c *Credentials) DisplayName() string {
-	return c.displayName
-}
-
-// SetDisplayName sets the display name value on the credential.
-func (c *Credentials) SetDisplayName(s string) {
-	c.displayName = s
-}
-
-// Human returns if the  credential represents a human or not.
-func (c *Credentials) Human() bool {
-	return c.human
-}
-
-// SetHuman sets the credential as human.
-func (c *Credentials) SetHuman(b bool) {
-	c.human = b
-}
-
-// AuthTime returns the time the credential was authenticated.
-func (c *Credentials) AuthTime() time.Time {
-	return c.authTime
-}
-
-// SetAuthTime sets the time the credential was authenticated.
-func (c *Credentials) SetAuthTime(t time.Time) {
-	c.authTime = t
-}
-
-// AuthzAttributes returns the credentials authorizing attributes.
-func (c *Credentials) AuthzAttributes() []string {
-	s := make([]string, len(c.groupMembership))
-	i := 0
-	for a := range c.groupMembership {
-		s[i] = a
-		i++
-	}
-	return s
-}
-
-// Authenticated indicates if the credential has been successfully authenticated or not.
-func (c *Credentials) Authenticated() bool {
-	return c.authenticated
-}
-
-// SetAuthenticated sets the credential as having been successfully authenticated.
-func (c *Credentials) SetAuthenticated(b bool) {
-	c.authenticated = b
-}
-
-// AddAuthzAttribute adds an authorization attribute to the credential.
-func (c *Credentials) AddAuthzAttribute(a string) {
-	c.groupMembership[a] = true
-}
-
-// RemoveAuthzAttribute removes an authorization attribute from the credential.
-func (c *Credentials) RemoveAuthzAttribute(a string) {
-	if _, ok := c.groupMembership[a]; !ok {
-		return
-	}
-	delete(c.groupMembership, a)
-}
-
-// EnableAuthzAttribute toggles an authorization attribute to an enabled state on the credential.
-func (c *Credentials) EnableAuthzAttribute(a string) {
-	if enabled, ok := c.groupMembership[a]; ok && !enabled {
-		c.groupMembership[a] = true
-	}
-}
-
-// DisableAuthzAttribute toggles an authorization attribute to a disabled state on the credential.
-func (c *Credentials) DisableAuthzAttribute(a string) {
-	if enabled, ok := c.groupMembership[a]; ok && enabled {
-		c.groupMembership[a] = false
-	}
-}
-
-// Authorized indicates if the credential has the specified authorizing attribute.
-func (c *Credentials) Authorized(a string) bool {
-	if enabled, ok := c.groupMembership[a]; ok && enabled {
-		return true
-	}
-	return false
-}
-
-// SessionID returns the credential's session ID.
-func (c *Credentials) SessionID() string {
-	return c.sessionID
-}
-
-// Expired indicates if the credential has expired.
-func (c *Credentials) Expired() bool {
-	if !c.validUntil.IsZero() && time.Now().UTC().After(c.validUntil) {
-		return true
-	}
-	return false
-}
-
-// Attributes returns the Credentials' attributes map.
-func (c *Credentials) Attributes() map[string]interface{} {
-	return c.attributes
-}
-
-// SetAttribute sets the value of an attribute.
-func (c *Credentials) SetAttribute(k string, v interface{}) {
-	c.attributes[k] = v
-}
-
-// SetAttributes replaces the attributes map with the one provided.
-func (c *Credentials) SetAttributes(a map[string]interface{}) {
-	c.attributes = a
-}
-
-// RemoveAttribute deletes an attribute from the attribute map that has the key provided.
-func (c *Credentials) RemoveAttribute(k string) {
-	delete(c.attributes, k)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha1-96.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha1-96.go
deleted file mode 100644
index 90b5df0..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha1-96.go
+++ /dev/null
@@ -1,173 +0,0 @@
-package crypto
-
-import (
-	"crypto/aes"
-	"crypto/hmac"
-	"crypto/sha1"
-	"hash"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// RFC 3962
-//+--------------------------------------------------------------------+
-//|               protocol key format        128- or 256-bit string    |
-//|                                                                    |
-//|            string-to-key function        PBKDF2+DK with variable   |
-//|                                          iteration count (see      |
-//|                                          above)                    |
-//|                                                                    |
-//|  default string-to-key parameters        00 00 10 00               |
-//|                                                                    |
-//|        key-generation seed length        key size                  |
-//|                                                                    |
-//|            random-to-key function        identity function         |
-//|                                                                    |
-//|                  hash function, H        SHA-1                     |
-//|                                                                    |
-//|               HMAC output size, h        12 octets (96 bits)       |
-//|                                                                    |
-//|             message block size, m        1 octet                   |
-//|                                                                    |
-//|  encryption/decryption functions,        AES in CBC-CTS mode       |
-//|  E and D                                 (cipher block size 16     |
-//|                                          octets), with next-to-    |
-//|                                          last block (last block    |
-//|                                          if only one) as CBC-style |
-//|                                          ivec                      |
-//+--------------------------------------------------------------------+
-//
-//+--------------------------------------------------------------------+
-//|                         encryption types                           |
-//+--------------------------------------------------------------------+
-//|         type name                  etype value          key size   |
-//+--------------------------------------------------------------------+
-//|   aes128-cts-hmac-sha1-96              17                 128      |
-//|   aes256-cts-hmac-sha1-96              18                 256      |
-//+--------------------------------------------------------------------+
-//
-//+--------------------------------------------------------------------+
-//|                          checksum types                            |
-//+--------------------------------------------------------------------+
-//|        type name                 sumtype value           length    |
-//+--------------------------------------------------------------------+
-//|    hmac-sha1-96-aes128                15                   96      |
-//|    hmac-sha1-96-aes256                16                   96      |
-//+--------------------------------------------------------------------+
-
-// Aes128CtsHmacSha96 implements Kerberos encryption type aes128-cts-hmac-sha1-96
-type Aes128CtsHmacSha96 struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e Aes128CtsHmacSha96) GetETypeID() int32 {
-	return etypeID.AES128_CTS_HMAC_SHA1_96
-}
-
-// GetHashID returns the checksum type ID number.
-func (e Aes128CtsHmacSha96) GetHashID() int32 {
-	return chksumtype.HMAC_SHA1_96_AES128
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e Aes128CtsHmacSha96) GetKeyByteSize() int {
-	return 128 / 8
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e Aes128CtsHmacSha96) GetKeySeedBitLength() int {
-	return e.GetKeyByteSize() * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e Aes128CtsHmacSha96) GetHashFunc() func() hash.Hash {
-	return sha1.New
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e Aes128CtsHmacSha96) GetMessageBlockByteSize() int {
-	return 1
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e Aes128CtsHmacSha96) GetDefaultStringToKeyParams() string {
-	return "00001000"
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e Aes128CtsHmacSha96) GetConfounderByteSize() int {
-	return aes.BlockSize
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e Aes128CtsHmacSha96) GetHMACBitLength() int {
-	return 96
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e Aes128CtsHmacSha96) GetCypherBlockBitLength() int {
-	return aes.BlockSize * 8
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e Aes128CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	return rfc3962.StringToKey(secret, salt, s2kparams, e)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e Aes128CtsHmacSha96) RandomToKey(b []byte) []byte {
-	return rfc3961.RandomToKey(b)
-}
-
-// EncryptData encrypts the data provided.
-func (e Aes128CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	return rfc3962.EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e Aes128CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	return rfc3962.EncryptMessage(key, message, usage, e)
-}
-
-// DecryptData decrypts the data provided.
-func (e Aes128CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc3962.DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e Aes128CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc3962.DecryptMessage(key, ciphertext, usage, e)
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e Aes128CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	return rfc3961.DeriveKey(protocolKey, usage, e)
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e Aes128CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	return rfc3961.DeriveRandom(protocolKey, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the plaintext message.
-func (e Aes128CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e Aes128CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e Aes128CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	c, err := e.GetChecksumHash(protocolKey, data, usage)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(chksum, c)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha256-128.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha256-128.go
deleted file mode 100644
index 49a1b07..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes128-cts-hmac-sha256-128.go
+++ /dev/null
@@ -1,135 +0,0 @@
-package crypto
-
-import (
-	"crypto/aes"
-	"crypto/hmac"
-	"crypto/sha256"
-	"hash"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// RFC https://tools.ietf.org/html/rfc8009
-
-// Aes128CtsHmacSha256128 implements Kerberos encryption type aes128-cts-hmac-sha256-128
-type Aes128CtsHmacSha256128 struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e Aes128CtsHmacSha256128) GetETypeID() int32 {
-	return etypeID.AES128_CTS_HMAC_SHA256_128
-}
-
-// GetHashID returns the checksum type ID number.
-func (e Aes128CtsHmacSha256128) GetHashID() int32 {
-	return chksumtype.HMAC_SHA256_128_AES128
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e Aes128CtsHmacSha256128) GetKeyByteSize() int {
-	return 128 / 8
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e Aes128CtsHmacSha256128) GetKeySeedBitLength() int {
-	return e.GetKeyByteSize() * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e Aes128CtsHmacSha256128) GetHashFunc() func() hash.Hash {
-	return sha256.New
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e Aes128CtsHmacSha256128) GetMessageBlockByteSize() int {
-	return 1
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e Aes128CtsHmacSha256128) GetDefaultStringToKeyParams() string {
-	return "00008000"
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e Aes128CtsHmacSha256128) GetConfounderByteSize() int {
-	return aes.BlockSize
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e Aes128CtsHmacSha256128) GetHMACBitLength() int {
-	return 128
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e Aes128CtsHmacSha256128) GetCypherBlockBitLength() int {
-	return aes.BlockSize * 8
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e Aes128CtsHmacSha256128) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	saltp := rfc8009.GetSaltP(salt, "aes128-cts-hmac-sha256-128")
-	return rfc8009.StringToKey(secret, saltp, s2kparams, e)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e Aes128CtsHmacSha256128) RandomToKey(b []byte) []byte {
-	return rfc8009.RandomToKey(b)
-}
-
-// EncryptData encrypts the data provided.
-func (e Aes128CtsHmacSha256128) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	return rfc8009.EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e Aes128CtsHmacSha256128) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	return rfc8009.EncryptMessage(key, message, usage, e)
-}
-
-// DecryptData decrypts the data provided.
-func (e Aes128CtsHmacSha256128) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc8009.DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e Aes128CtsHmacSha256128) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc8009.DecryptMessage(key, ciphertext, usage, e)
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e Aes128CtsHmacSha256128) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	return rfc8009.DeriveKey(protocolKey, usage, e), nil
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e Aes128CtsHmacSha256128) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	return rfc8009.DeriveRandom(protocolKey, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the ciphertext message.
-// The HMAC is calculated over the cipher state concatenated with the
-// AES output, instead of being calculated over the confounder and
-// plaintext.  This allows the message receiver to verify the
-// integrity of the message before decrypting the message.
-// Therefore the pt value to this interface method is not use. Pass any []byte.
-func (e Aes128CtsHmacSha256128) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	// We don't need ib just there for the interface
-	return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e Aes128CtsHmacSha256128) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e Aes128CtsHmacSha256128) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	c, err := e.GetChecksumHash(protocolKey, data, usage)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(chksum, c)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha1-96.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha1-96.go
deleted file mode 100644
index 0cdbb7e..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha1-96.go
+++ /dev/null
@@ -1,173 +0,0 @@
-package crypto
-
-import (
-	"crypto/aes"
-	"crypto/hmac"
-	"crypto/sha1"
-	"hash"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// RFC 3962
-//+--------------------------------------------------------------------+
-//|               protocol key format        128- or 256-bit string    |
-//|                                                                    |
-//|            string-to-key function        PBKDF2+DK with variable   |
-//|                                          iteration count (see      |
-//|                                          above)                    |
-//|                                                                    |
-//|  default string-to-key parameters        00 00 10 00               |
-//|                                                                    |
-//|        key-generation seed length        key size                  |
-//|                                                                    |
-//|            random-to-key function        identity function         |
-//|                                                                    |
-//|                  hash function, H        SHA-1                     |
-//|                                                                    |
-//|               HMAC output size, h        12 octets (96 bits)       |
-//|                                                                    |
-//|             message block size, m        1 octet                   |
-//|                                                                    |
-//|  encryption/decryption functions,        AES in CBC-CTS mode       |
-//|  E and D                                 (cipher block size 16     |
-//|                                          octets), with next-to-    |
-//|                                          last block (last block    |
-//|                                          if only one) as CBC-style |
-//|                                          ivec                      |
-//+--------------------------------------------------------------------+
-//
-//+--------------------------------------------------------------------+
-//|                         encryption types                           |
-//+--------------------------------------------------------------------+
-//|         type name                  etype value          key size   |
-//+--------------------------------------------------------------------+
-//|   aes128-cts-hmac-sha1-96              17                 128      |
-//|   aes256-cts-hmac-sha1-96              18                 256      |
-//+--------------------------------------------------------------------+
-//
-//+--------------------------------------------------------------------+
-//|                          checksum types                            |
-//+--------------------------------------------------------------------+
-//|        type name                 sumtype value           length    |
-//+--------------------------------------------------------------------+
-//|    hmac-sha1-96-aes128                15                   96      |
-//|    hmac-sha1-96-aes256                16                   96      |
-//+--------------------------------------------------------------------+
-
-// Aes256CtsHmacSha96 implements Kerberos encryption type aes256-cts-hmac-sha1-96
-type Aes256CtsHmacSha96 struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e Aes256CtsHmacSha96) GetETypeID() int32 {
-	return etypeID.AES256_CTS_HMAC_SHA1_96
-}
-
-// GetHashID returns the checksum type ID number.
-func (e Aes256CtsHmacSha96) GetHashID() int32 {
-	return chksumtype.HMAC_SHA1_96_AES256
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e Aes256CtsHmacSha96) GetKeyByteSize() int {
-	return 256 / 8
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e Aes256CtsHmacSha96) GetKeySeedBitLength() int {
-	return e.GetKeyByteSize() * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e Aes256CtsHmacSha96) GetHashFunc() func() hash.Hash {
-	return sha1.New
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e Aes256CtsHmacSha96) GetMessageBlockByteSize() int {
-	return 1
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e Aes256CtsHmacSha96) GetDefaultStringToKeyParams() string {
-	return "00001000"
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e Aes256CtsHmacSha96) GetConfounderByteSize() int {
-	return aes.BlockSize
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e Aes256CtsHmacSha96) GetHMACBitLength() int {
-	return 96
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e Aes256CtsHmacSha96) GetCypherBlockBitLength() int {
-	return aes.BlockSize * 8
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e Aes256CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	return rfc3962.StringToKey(secret, salt, s2kparams, e)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e Aes256CtsHmacSha96) RandomToKey(b []byte) []byte {
-	return rfc3961.RandomToKey(b)
-}
-
-// EncryptData encrypts the data provided.
-func (e Aes256CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	return rfc3962.EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e Aes256CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	return rfc3962.EncryptMessage(key, message, usage, e)
-}
-
-// DecryptData decrypts the data provided.
-func (e Aes256CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc3962.DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e Aes256CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc3962.DecryptMessage(key, ciphertext, usage, e)
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e Aes256CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	return rfc3961.DeriveKey(protocolKey, usage, e)
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e Aes256CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	return rfc3961.DeriveRandom(protocolKey, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the plaintext message.
-func (e Aes256CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e Aes256CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e Aes256CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	c, err := e.GetChecksumHash(protocolKey, data, usage)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(chksum, c)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha384-192.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha384-192.go
deleted file mode 100644
index 562b078..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/aes256-cts-hmac-sha384-192.go
+++ /dev/null
@@ -1,135 +0,0 @@
-package crypto
-
-import (
-	"crypto/aes"
-	"crypto/hmac"
-	"crypto/sha512"
-	"hash"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// RFC https://tools.ietf.org/html/rfc8009
-
-// Aes256CtsHmacSha384192 implements Kerberos encryption type aes256-cts-hmac-sha384-192
-type Aes256CtsHmacSha384192 struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e Aes256CtsHmacSha384192) GetETypeID() int32 {
-	return etypeID.AES256_CTS_HMAC_SHA384_192
-}
-
-// GetHashID returns the checksum type ID number.
-func (e Aes256CtsHmacSha384192) GetHashID() int32 {
-	return chksumtype.HMAC_SHA384_192_AES256
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e Aes256CtsHmacSha384192) GetKeyByteSize() int {
-	return 192 / 8
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e Aes256CtsHmacSha384192) GetKeySeedBitLength() int {
-	return e.GetKeyByteSize() * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e Aes256CtsHmacSha384192) GetHashFunc() func() hash.Hash {
-	return sha512.New384
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e Aes256CtsHmacSha384192) GetMessageBlockByteSize() int {
-	return 1
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e Aes256CtsHmacSha384192) GetDefaultStringToKeyParams() string {
-	return "00008000"
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e Aes256CtsHmacSha384192) GetConfounderByteSize() int {
-	return aes.BlockSize
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e Aes256CtsHmacSha384192) GetHMACBitLength() int {
-	return 192
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e Aes256CtsHmacSha384192) GetCypherBlockBitLength() int {
-	return aes.BlockSize * 8
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e Aes256CtsHmacSha384192) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	saltp := rfc8009.GetSaltP(salt, "aes256-cts-hmac-sha384-192")
-	return rfc8009.StringToKey(secret, saltp, s2kparams, e)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e Aes256CtsHmacSha384192) RandomToKey(b []byte) []byte {
-	return rfc8009.RandomToKey(b)
-}
-
-// EncryptData encrypts the data provided.
-func (e Aes256CtsHmacSha384192) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	return rfc8009.EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e Aes256CtsHmacSha384192) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	return rfc8009.EncryptMessage(key, message, usage, e)
-}
-
-// DecryptData decrypts the data provided.
-func (e Aes256CtsHmacSha384192) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc8009.DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e Aes256CtsHmacSha384192) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc8009.DecryptMessage(key, ciphertext, usage, e)
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e Aes256CtsHmacSha384192) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	return rfc8009.DeriveKey(protocolKey, usage, e), nil
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e Aes256CtsHmacSha384192) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	return rfc8009.DeriveRandom(protocolKey, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the ciphertext message.
-// The HMAC is calculated over the cipher state concatenated with the
-// AES output, instead of being calculated over the confounder and
-// plaintext.  This allows the message receiver to verify the
-// integrity of the message before decrypting the message.
-// Therefore the pt value to this interface method is not use. Pass any []byte.
-func (e Aes256CtsHmacSha384192) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	// We don't need ib just there for the interface
-	return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e Aes256CtsHmacSha384192) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e Aes256CtsHmacSha384192) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	c, err := e.GetChecksumHash(protocolKey, data, usage)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(chksum, c)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/common/common.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/common/common.go
deleted file mode 100644
index 96ae549..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/common/common.go
+++ /dev/null
@@ -1,143 +0,0 @@
-// Package common provides encryption methods common across encryption types
-package common
-
-import (
-	"bytes"
-	"crypto/hmac"
-	"encoding/binary"
-	"encoding/hex"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-// ZeroPad pads bytes with zeros to nearest multiple of message size m.
-func ZeroPad(b []byte, m int) ([]byte, error) {
-	if m <= 0 {
-		return nil, errors.New("Invalid message block size when padding")
-	}
-	if b == nil || len(b) == 0 {
-		return nil, errors.New("Data not valid to pad: Zero size")
-	}
-	if l := len(b) % m; l != 0 {
-		n := m - l
-		z := make([]byte, n)
-		b = append(b, z...)
-	}
-	return b, nil
-}
-
-// PKCS7Pad pads bytes according to RFC 2315 to nearest multiple of message size m.
-func PKCS7Pad(b []byte, m int) ([]byte, error) {
-	if m <= 0 {
-		return nil, errors.New("Invalid message block size when padding")
-	}
-	if b == nil || len(b) == 0 {
-		return nil, errors.New("Data not valid to pad: Zero size")
-	}
-	n := m - (len(b) % m)
-	pb := make([]byte, len(b)+n)
-	copy(pb, b)
-	copy(pb[len(b):], bytes.Repeat([]byte{byte(n)}, n))
-	return pb, nil
-}
-
-// PKCS7Unpad removes RFC 2315 padding from byes where message size is m.
-func PKCS7Unpad(b []byte, m int) ([]byte, error) {
-	if m <= 0 {
-		return nil, errors.New("invalid message block size when unpadding")
-	}
-	if b == nil || len(b) == 0 {
-		return nil, errors.New("padded data not valid: Zero size")
-	}
-	if len(b)%m != 0 {
-		return nil, errors.New("padded data not valid: Not multiple of message block size")
-	}
-	c := b[len(b)-1]
-	n := int(c)
-	if n == 0 || n > len(b) {
-		return nil, errors.New("padded data not valid: Data may not have been padded")
-	}
-	for i := 0; i < n; i++ {
-		if b[len(b)-n+i] != c {
-			return nil, errors.New("padded data not valid")
-		}
-	}
-	return b[:len(b)-n], nil
-}
-
-// GetHash generates the keyed hash value according to the etype's hash function.
-func GetHash(pt, key []byte, usage []byte, etype etype.EType) ([]byte, error) {
-	k, err := etype.DeriveKey(key, usage)
-	if err != nil {
-		return nil, fmt.Errorf("unable to derive key for checksum: %v", err)
-	}
-	mac := hmac.New(etype.GetHashFunc(), k)
-	p := make([]byte, len(pt))
-	copy(p, pt)
-	mac.Write(p)
-	return mac.Sum(nil)[:etype.GetHMACBitLength()/8], nil
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func GetChecksumHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) {
-	return GetHash(b, key, GetUsageKc(usage), etype)
-}
-
-// GetIntegrityHash returns a keyed integrity hash of the bytes provided.
-func GetIntegrityHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) {
-	return GetHash(b, key, GetUsageKi(usage), etype)
-}
-
-// VerifyChecksum compares the checksum of the msg bytes is the same as the checksum provided.
-func VerifyChecksum(key, chksum, msg []byte, usage uint32, etype etype.EType) bool {
-	//The ciphertext output is the concatenation of the output of the basic
-	//encryption function E and a (possibly truncated) HMAC using the
-	//specified hash function H, both applied to the plaintext with a
-	//random confounder prefix and sufficient padding to bring it to a
-	//multiple of the message block size.  When the HMAC is computed, the
-	//key is used in the protocol key form.
-	expectedMAC, _ := GetChecksumHash(msg, key, usage, etype)
-	return hmac.Equal(chksum, expectedMAC)
-}
-
-// GetUsageKc returns the checksum key usage value for the usage number un.
-//
-// RFC 3961: The "well-known constant" used for the DK function is the key usage number, expressed as four octets in big-endian order, followed by one octet indicated below.
-//
-// Kc = DK(base-key, usage | 0x99);
-func GetUsageKc(un uint32) []byte {
-	return getUsage(un, 0x99)
-}
-
-// GetUsageKe returns the encryption key usage value for the usage number un
-//
-// RFC 3961: The "well-known constant" used for the DK function is the key usage number, expressed as four octets in big-endian order, followed by one octet indicated below.
-//
-// Ke = DK(base-key, usage | 0xAA);
-func GetUsageKe(un uint32) []byte {
-	return getUsage(un, 0xAA)
-}
-
-// GetUsageKi returns the integrity key usage value for the usage number un
-//
-// RFC 3961: The "well-known constant" used for the DK function is the key usage number, expressed as four octets in big-endian order, followed by one octet indicated below.
-//
-// Ki = DK(base-key, usage | 0x55);
-func GetUsageKi(un uint32) []byte {
-	return getUsage(un, 0x55)
-}
-
-func getUsage(un uint32, o byte) []byte {
-	var buf bytes.Buffer
-	binary.Write(&buf, binary.BigEndian, un)
-	return append(buf.Bytes(), o)
-}
-
-// IterationsToS2Kparams converts the number of iterations as an integer to a string representation.
-func IterationsToS2Kparams(i uint32) string {
-	b := make([]byte, 4, 4)
-	binary.BigEndian.PutUint32(b, i)
-	return hex.EncodeToString(b)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/crypto.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/crypto.go
deleted file mode 100644
index e04e968..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/crypto.go
+++ /dev/null
@@ -1,175 +0,0 @@
-// Package crypto implements cryptographic functions for Kerberos 5 implementation.
-package crypto
-
-import (
-	"encoding/hex"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/patype"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// GetEtype returns an instances of the required etype struct for the etype ID.
-func GetEtype(id int32) (etype.EType, error) {
-	switch id {
-	case etypeID.AES128_CTS_HMAC_SHA1_96:
-		var et Aes128CtsHmacSha96
-		return et, nil
-	case etypeID.AES256_CTS_HMAC_SHA1_96:
-		var et Aes256CtsHmacSha96
-		return et, nil
-	case etypeID.AES128_CTS_HMAC_SHA256_128:
-		var et Aes128CtsHmacSha256128
-		return et, nil
-	case etypeID.AES256_CTS_HMAC_SHA384_192:
-		var et Aes256CtsHmacSha384192
-		return et, nil
-	case etypeID.DES3_CBC_SHA1_KD:
-		var et Des3CbcSha1Kd
-		return et, nil
-	case etypeID.RC4_HMAC:
-		var et RC4HMAC
-		return et, nil
-	default:
-		return nil, fmt.Errorf("unknown or unsupported EType: %d", id)
-	}
-}
-
-// GetChksumEtype returns an instances of the required etype struct for the checksum ID.
-func GetChksumEtype(id int32) (etype.EType, error) {
-	switch id {
-	case chksumtype.HMAC_SHA1_96_AES128:
-		var et Aes128CtsHmacSha96
-		return et, nil
-	case chksumtype.HMAC_SHA1_96_AES256:
-		var et Aes256CtsHmacSha96
-		return et, nil
-	case chksumtype.HMAC_SHA256_128_AES128:
-		var et Aes128CtsHmacSha256128
-		return et, nil
-	case chksumtype.HMAC_SHA384_192_AES256:
-		var et Aes256CtsHmacSha384192
-		return et, nil
-	case chksumtype.HMAC_SHA1_DES3_KD:
-		var et Des3CbcSha1Kd
-		return et, nil
-	case chksumtype.KERB_CHECKSUM_HMAC_MD5:
-		var et RC4HMAC
-		return et, nil
-	//case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED:
-	//	var et RC4HMAC
-	//	return et, nil
-	default:
-		return nil, fmt.Errorf("unknown or unsupported checksum type: %d", id)
-	}
-}
-
-// GetKeyFromPassword generates an encryption key from the principal's password.
-func GetKeyFromPassword(passwd string, cname types.PrincipalName, realm string, etypeID int32, pas types.PADataSequence) (types.EncryptionKey, etype.EType, error) {
-	var key types.EncryptionKey
-	et, err := GetEtype(etypeID)
-	if err != nil {
-		return key, et, fmt.Errorf("error getting encryption type: %v", err)
-	}
-	sk2p := et.GetDefaultStringToKeyParams()
-	var salt string
-	var paID int32
-	for _, pa := range pas {
-		switch pa.PADataType {
-		case patype.PA_PW_SALT:
-			if paID > pa.PADataType {
-				continue
-			}
-			salt = string(pa.PADataValue)
-		case patype.PA_ETYPE_INFO:
-			if paID > pa.PADataType {
-				continue
-			}
-			var eti types.ETypeInfo
-			err := eti.Unmarshal(pa.PADataValue)
-			if err != nil {
-				return key, et, fmt.Errorf("error unmashaling PA Data to PA-ETYPE-INFO2: %v", err)
-			}
-			if etypeID != eti[0].EType {
-				et, err = GetEtype(eti[0].EType)
-				if err != nil {
-					return key, et, fmt.Errorf("error getting encryption type: %v", err)
-				}
-			}
-			salt = string(eti[0].Salt)
-		case patype.PA_ETYPE_INFO2:
-			if paID > pa.PADataType {
-				continue
-			}
-			var et2 types.ETypeInfo2
-			err := et2.Unmarshal(pa.PADataValue)
-			if err != nil {
-				return key, et, fmt.Errorf("error unmashalling PA Data to PA-ETYPE-INFO2: %v", err)
-			}
-			if etypeID != et2[0].EType {
-				et, err = GetEtype(et2[0].EType)
-				if err != nil {
-					return key, et, fmt.Errorf("error getting encryption type: %v", err)
-				}
-			}
-			if len(et2[0].S2KParams) == 4 {
-				sk2p = hex.EncodeToString(et2[0].S2KParams)
-			}
-			salt = et2[0].Salt
-		}
-	}
-	if salt == "" {
-		salt = cname.GetSalt(realm)
-	}
-	k, err := et.StringToKey(passwd, salt, sk2p)
-	if err != nil {
-		return key, et, fmt.Errorf("error deriving key from string: %+v", err)
-	}
-	key = types.EncryptionKey{
-		KeyType:  etypeID,
-		KeyValue: k,
-	}
-	return key, et, nil
-}
-
-// GetEncryptedData encrypts the data provided and returns and EncryptedData type.
-// Pass a usage value of zero to use the key provided directly rather than deriving one.
-func GetEncryptedData(plainBytes []byte, key types.EncryptionKey, usage uint32, kvno int) (types.EncryptedData, error) {
-	var ed types.EncryptedData
-	et, err := GetEtype(key.KeyType)
-	if err != nil {
-		return ed, fmt.Errorf("error getting etype: %v", err)
-	}
-	_, b, err := et.EncryptMessage(key.KeyValue, plainBytes, usage)
-	if err != nil {
-		return ed, err
-	}
-
-	ed = types.EncryptedData{
-		EType:  key.KeyType,
-		Cipher: b,
-		KVNO:   kvno,
-	}
-	return ed, nil
-}
-
-// DecryptEncPart decrypts the EncryptedData.
-func DecryptEncPart(ed types.EncryptedData, key types.EncryptionKey, usage uint32) ([]byte, error) {
-	return DecryptMessage(ed.Cipher, key, usage)
-}
-
-// DecryptMessage decrypts the ciphertext and verifies the integrity.
-func DecryptMessage(ciphertext []byte, key types.EncryptionKey, usage uint32) ([]byte, error) {
-	et, err := GetEtype(key.KeyType)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error decrypting: %v", err)
-	}
-	b, err := et.DecryptMessage(key.KeyValue, ciphertext, usage)
-	if err != nil {
-		return nil, fmt.Errorf("error decrypting: %v", err)
-	}
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/des3-cbc-sha1-kd.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/des3-cbc-sha1-kd.go
deleted file mode 100644
index db3a149..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/des3-cbc-sha1-kd.go
+++ /dev/null
@@ -1,174 +0,0 @@
-package crypto
-
-import (
-	"crypto/des"
-	"crypto/hmac"
-	"crypto/sha1"
-	"errors"
-	"hash"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-//RFC: 3961 Section 6.3
-
-/*
-                 des3-cbc-hmac-sha1-kd, hmac-sha1-des3-kd
-              ------------------------------------------------
-              protocol key format     24 bytes, parity in low
-                                      bit of each
-
-              key-generation seed     21 bytes
-              length
-
-              hash function           SHA-1
-
-              HMAC output size        160 bits
-
-              message block size      8 bytes
-
-              default string-to-key   empty string
-              params
-
-              encryption and          triple-DES encrypt and
-              decryption functions    decrypt, in outer-CBC
-                                      mode (cipher block size
-                                      8 octets)
-
-              key generation functions:
-
-              random-to-key           DES3random-to-key (see
-                                      below)
-
-              string-to-key           DES3string-to-key (see
-                                      below)
-
-   The des3-cbc-hmac-sha1-kd encryption type is assigned the value
-   sixteen (16).  The hmac-sha1-des3-kd checksum algorithm is assigned a
-   checksum type number of twelve (12)*/
-
-// Des3CbcSha1Kd implements Kerberos encryption type des3-cbc-hmac-sha1-kd
-type Des3CbcSha1Kd struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e Des3CbcSha1Kd) GetETypeID() int32 {
-	return etypeID.DES3_CBC_SHA1_KD
-}
-
-// GetHashID returns the checksum type ID number.
-func (e Des3CbcSha1Kd) GetHashID() int32 {
-	return chksumtype.HMAC_SHA1_DES3_KD
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e Des3CbcSha1Kd) GetKeyByteSize() int {
-	return 24
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e Des3CbcSha1Kd) GetKeySeedBitLength() int {
-	return 21 * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e Des3CbcSha1Kd) GetHashFunc() func() hash.Hash {
-	return sha1.New
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e Des3CbcSha1Kd) GetMessageBlockByteSize() int {
-	//For traditional CBC mode with padding, it would be the underlying cipher's block size
-	return des.BlockSize
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e Des3CbcSha1Kd) GetDefaultStringToKeyParams() string {
-	var s string
-	return s
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e Des3CbcSha1Kd) GetConfounderByteSize() int {
-	return des.BlockSize
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e Des3CbcSha1Kd) GetHMACBitLength() int {
-	return e.GetHashFunc()().Size() * 8
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e Des3CbcSha1Kd) GetCypherBlockBitLength() int {
-	return des.BlockSize * 8
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e Des3CbcSha1Kd) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	if s2kparams != "" {
-		return []byte{}, errors.New("s2kparams must be an empty string")
-	}
-	return rfc3961.DES3StringToKey(secret, salt, e)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e Des3CbcSha1Kd) RandomToKey(b []byte) []byte {
-	return rfc3961.DES3RandomToKey(b)
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e Des3CbcSha1Kd) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	r, err := rfc3961.DeriveRandom(protocolKey, usage, e)
-	return r, err
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e Des3CbcSha1Kd) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	r, err := e.DeriveRandom(protocolKey, usage)
-	if err != nil {
-		return nil, err
-	}
-	return e.RandomToKey(r), nil
-}
-
-// EncryptData encrypts the data provided.
-func (e Des3CbcSha1Kd) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	return rfc3961.DES3EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e Des3CbcSha1Kd) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	return rfc3961.DES3EncryptMessage(key, message, usage, e)
-}
-
-// DecryptData decrypts the data provided.
-func (e Des3CbcSha1Kd) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc3961.DES3DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e Des3CbcSha1Kd) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc3961.DES3DecryptMessage(key, ciphertext, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the plaintext message.
-func (e Des3CbcSha1Kd) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e Des3CbcSha1Kd) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e Des3CbcSha1Kd) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	c, err := e.GetChecksumHash(protocolKey, data, usage)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(chksum, c)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/etype/etype.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/etype/etype.go
deleted file mode 100644
index ee7510e..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/etype/etype.go
+++ /dev/null
@@ -1,29 +0,0 @@
-// Package etype provides the Kerberos Encryption Type interface
-package etype
-
-import "hash"
-
-// EType is the interface defining the Encryption Type.
-type EType interface {
-	GetETypeID() int32
-	GetHashID() int32
-	GetKeyByteSize() int
-	GetKeySeedBitLength() int                                   // key-generation seed length, k
-	GetDefaultStringToKeyParams() string                        // default string-to-key parameters (s2kparams)
-	StringToKey(string, salt, s2kparams string) ([]byte, error) // string-to-key (UTF-8 string, UTF-8 string, opaque)->(protocol-key)
-	RandomToKey(b []byte) []byte                                // random-to-key (bitstring[K])->(protocol-key)
-	GetHMACBitLength() int                                      // HMAC output size, h
-	GetMessageBlockByteSize() int                               // message block size, m
-	EncryptData(key, data []byte) ([]byte, []byte, error)       // E function - encrypt (specific-key, state, octet string)->(state, octet string)
-	EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error)
-	DecryptData(key, data []byte) ([]byte, error) // D function
-	DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error)
-	GetCypherBlockBitLength() int                           // cipher block size, c
-	GetConfounderByteSize() int                             // This is the same as the cipher block size but in bytes.
-	DeriveKey(protocolKey, usage []byte) ([]byte, error)    // DK key-derivation (protocol-key, integer)->(specific-key)
-	DeriveRandom(protocolKey, usage []byte) ([]byte, error) // DR pseudo-random (protocol-key, octet-string)->(octet-string)
-	VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool
-	GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error)
-	VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool
-	GetHashFunc() func() hash.Hash
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rc4-hmac.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rc4-hmac.go
deleted file mode 100644
index 9df55ee..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rc4-hmac.go
+++ /dev/null
@@ -1,135 +0,0 @@
-package crypto
-
-import (
-	"bytes"
-	"crypto/hmac"
-	"crypto/md5"
-	"hash"
-	"io"
-
-	"golang.org/x/crypto/md4"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-//http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8u40-b25/sun/security/krb5/internal/crypto/dk/ArcFourCrypto.java#ArcFourCrypto.encrypt%28byte%5B%5D%2Cint%2Cbyte%5B%5D%2Cbyte%5B%5D%2Cbyte%5B%5D%2Cint%2Cint%29
-
-// RC4HMAC implements Kerberos encryption type aes256-cts-hmac-sha1-96
-type RC4HMAC struct {
-}
-
-// GetETypeID returns the EType ID number.
-func (e RC4HMAC) GetETypeID() int32 {
-	return etypeID.RC4_HMAC
-}
-
-// GetHashID returns the checksum type ID number.
-func (e RC4HMAC) GetHashID() int32 {
-	return chksumtype.KERB_CHECKSUM_HMAC_MD5
-}
-
-// GetKeyByteSize returns the number of bytes for key of this etype.
-func (e RC4HMAC) GetKeyByteSize() int {
-	return 16
-}
-
-// GetKeySeedBitLength returns the number of bits for the seed for key generation.
-func (e RC4HMAC) GetKeySeedBitLength() int {
-	return e.GetKeyByteSize() * 8
-}
-
-// GetHashFunc returns the hash function for this etype.
-func (e RC4HMAC) GetHashFunc() func() hash.Hash {
-	return md5.New
-}
-
-// GetMessageBlockByteSize returns the block size for the etype's messages.
-func (e RC4HMAC) GetMessageBlockByteSize() int {
-	return 1
-}
-
-// GetDefaultStringToKeyParams returns the default key derivation parameters in string form.
-func (e RC4HMAC) GetDefaultStringToKeyParams() string {
-	return ""
-}
-
-// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations.
-func (e RC4HMAC) GetConfounderByteSize() int {
-	return 8
-}
-
-// GetHMACBitLength returns the bit count size of the integrity hash.
-func (e RC4HMAC) GetHMACBitLength() int {
-	return md5.Size * 8
-}
-
-// GetCypherBlockBitLength returns the bit count size of the cypher block.
-func (e RC4HMAC) GetCypherBlockBitLength() int {
-	return 8 // doesn't really apply
-}
-
-// StringToKey returns a key derived from the string provided.
-func (e RC4HMAC) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) {
-	return rfc4757.StringToKey(secret)
-}
-
-// RandomToKey returns a key from the bytes provided.
-func (e RC4HMAC) RandomToKey(b []byte) []byte {
-	r := bytes.NewReader(b)
-	h := md4.New()
-	io.Copy(h, r)
-	return h.Sum(nil)
-}
-
-// EncryptData encrypts the data provided.
-func (e RC4HMAC) EncryptData(key, data []byte) ([]byte, []byte, error) {
-	b, err := rfc4757.EncryptData(key, data, e)
-	return []byte{}, b, err
-}
-
-// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message.
-func (e RC4HMAC) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) {
-	b, err := rfc4757.EncryptMessage(key, message, usage, false, e)
-	return []byte{}, b, err
-}
-
-// DecryptData decrypts the data provided.
-func (e RC4HMAC) DecryptData(key, data []byte) ([]byte, error) {
-	return rfc4757.DecryptData(key, data, e)
-}
-
-// DecryptMessage decrypts the message provided and verifies the integrity of the message.
-func (e RC4HMAC) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) {
-	return rfc4757.DecryptMessage(key, ciphertext, usage, false, e)
-}
-
-// DeriveKey derives a key from the protocol key based on the usage value.
-func (e RC4HMAC) DeriveKey(protocolKey, usage []byte) ([]byte, error) {
-	return rfc4757.HMAC(protocolKey, usage), nil
-}
-
-// DeriveRandom generates data needed for key generation.
-func (e RC4HMAC) DeriveRandom(protocolKey, usage []byte) ([]byte, error) {
-	return rfc3961.DeriveRandom(protocolKey, usage, e)
-}
-
-// VerifyIntegrity checks the integrity of the plaintext message.
-func (e RC4HMAC) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool {
-	return rfc4757.VerifyIntegrity(protocolKey, pt, ct, e)
-}
-
-// GetChecksumHash returns a keyed checksum hash of the bytes provided.
-func (e RC4HMAC) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) {
-	return rfc4757.Checksum(protocolKey, usage, data)
-}
-
-// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided.
-func (e RC4HMAC) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool {
-	checksum, err := rfc4757.Checksum(protocolKey, usage, data)
-	if err != nil {
-		return false
-	}
-	return hmac.Equal(checksum, chksum)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/encryption.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/encryption.go
deleted file mode 100644
index 6f550fa..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/encryption.go
+++ /dev/null
@@ -1,125 +0,0 @@
-// Package rfc3961 provides encryption and checksum methods as specified in RFC 3961
-package rfc3961
-
-import (
-	"crypto/cipher"
-	"crypto/des"
-	"crypto/hmac"
-	"crypto/rand"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-// DES3EncryptData encrypts the data provided using DES3 and methods specific to the etype provided.
-func DES3EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return nil, nil, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	data, _ = common.ZeroPad(data, e.GetMessageBlockByteSize())
-
-	block, err := des.NewTripleDESCipher(key)
-	if err != nil {
-		return nil, nil, fmt.Errorf("error creating cipher: %v", err)
-	}
-
-	//RFC 3961: initial cipher state      All bits zero
-	ivz := make([]byte, des.BlockSize)
-
-	ct := make([]byte, len(data))
-	mode := cipher.NewCBCEncrypter(block, ivz)
-	mode.CryptBlocks(ct, data)
-	return ct[len(ct)-e.GetMessageBlockByteSize():], ct, nil
-}
-
-// DES3EncryptMessage encrypts the message provided using DES3 and methods specific to the etype provided.
-// The encrypted data is concatenated with its integrity hash to create an encrypted message.
-func DES3EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
-	//confounder
-	c := make([]byte, e.GetConfounderByteSize())
-	_, err := rand.Read(c)
-	if err != nil {
-		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
-	}
-	plainBytes := append(c, message...)
-	plainBytes, _ = common.ZeroPad(plainBytes, e.GetMessageBlockByteSize())
-
-	// Derive key for encryption from usage
-	var k []byte
-	if usage != 0 {
-		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
-		if err != nil {
-			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
-		}
-	}
-
-	iv, b, err := e.EncryptData(k, plainBytes)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-
-	// Generate and append integrity hash
-	ih, err := common.GetIntegrityHash(plainBytes, key, usage, e)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-	b = append(b, ih...)
-	return iv, b, nil
-}
-
-// DES3DecryptData decrypts the data provided using DES3 and methods specific to the etype provided.
-func DES3DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-
-	if len(data) < des.BlockSize || len(data)%des.BlockSize != 0 {
-		return []byte{}, errors.New("ciphertext is not a multiple of the block size")
-	}
-	block, err := des.NewTripleDESCipher(key)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error creating cipher: %v", err)
-	}
-	pt := make([]byte, len(data))
-	ivz := make([]byte, des.BlockSize)
-	mode := cipher.NewCBCDecrypter(block, ivz)
-	mode.CryptBlocks(pt, data)
-	return pt, nil
-}
-
-// DES3DecryptMessage decrypts the message provided using DES3 and methods specific to the etype provided.
-// The integrity of the message is also verified.
-func DES3DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
-	//Derive the key
-	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
-	if err != nil {
-		return nil, fmt.Errorf("error deriving key: %v", err)
-	}
-	// Strip off the checksum from the end
-	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
-	if err != nil {
-		return nil, fmt.Errorf("error decrypting: %v", err)
-	}
-	//Verify checksum
-	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
-		return nil, errors.New("error decrypting: integrity verification failed")
-	}
-	//Remove the confounder bytes
-	return b[e.GetConfounderByteSize():], nil
-}
-
-// VerifyIntegrity verifies the integrity of cipertext bytes ct.
-func VerifyIntegrity(key, ct, pt []byte, usage uint32, etype etype.EType) bool {
-	//The ciphertext output is the concatenation of the output of the basic
-	//encryption function E and a (possibly truncated) HMAC using the
-	//specified hash function H, both applied to the plaintext with a
-	//random confounder prefix and sufficient padding to bring it to a
-	//multiple of the message block size.  When the HMAC is computed, the
-	//key is used in the protocol key form.
-	h := make([]byte, etype.GetHMACBitLength()/8)
-	copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:])
-	expectedMAC, _ := common.GetIntegrityHash(pt, key, usage, etype)
-	return hmac.Equal(h, expectedMAC)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/keyDerivation.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/keyDerivation.go
deleted file mode 100644
index 8c637a2..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/keyDerivation.go
+++ /dev/null
@@ -1,178 +0,0 @@
-package rfc3961
-
-import (
-	"bytes"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-const (
-	prfconstant = "prf"
-)
-
-// DeriveRandom implements the RFC 3961 defined function: DR(Key, Constant) = k-truncate(E(Key, Constant, initial-cipher-state)).
-//
-// key: base key or protocol key. Likely to be a key from a keytab file.
-//
-// usage: a constant.
-//
-// n: block size in bits (not bytes) - note if you use something like aes.BlockSize this is in bytes.
-//
-// k: key length / key seed length in bits. Eg. for AES256 this value is 256.
-//
-// e: the encryption etype function to use.
-func DeriveRandom(key, usage []byte, e etype.EType) ([]byte, error) {
-	n := e.GetCypherBlockBitLength()
-	k := e.GetKeySeedBitLength()
-	//Ensure the usage constant is at least the size of the cypher block size. Pass it through the nfold algorithm that will "stretch" it if needs be.
-	nFoldUsage := Nfold(usage, n)
-	//k-truncate implemented by creating a byte array the size of k (k is in bits hence /8)
-	out := make([]byte, k/8)
-
-	/*If the output	of E is shorter than k bits, it is fed back into the encryption as many times as necessary.
-	The construct is as follows (where | indicates concatenation):
-
-	K1 = E(Key, n-fold(Constant), initial-cipher-state)
-	K2 = E(Key, K1, initial-cipher-state)
-	K3 = E(Key, K2, initial-cipher-state)
-	K4 = ...
-
-	DR(Key, Constant) = k-truncate(K1 | K2 | K3 | K4 ...)*/
-	_, K, err := e.EncryptData(key, nFoldUsage)
-	if err != nil {
-		return out, err
-	}
-	for i := copy(out, K); i < len(out); {
-		_, K, _ = e.EncryptData(key, K)
-		i = i + copy(out[i:], K)
-	}
-	return out, nil
-}
-
-// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods.
-func DeriveKey(protocolKey, usage []byte, e etype.EType) ([]byte, error) {
-	r, err := e.DeriveRandom(protocolKey, usage)
-	if err != nil {
-		return nil, err
-	}
-	return e.RandomToKey(r), nil
-}
-
-// RandomToKey returns a key from the bytes provided according to the definition in RFC 3961.
-func RandomToKey(b []byte) []byte {
-	return b
-}
-
-// DES3RandomToKey returns a key from the bytes provided according to the definition in RFC 3961 for DES3 etypes.
-func DES3RandomToKey(b []byte) []byte {
-	r := fixWeakKey(stretch56Bits(b[:7]))
-	r2 := fixWeakKey(stretch56Bits(b[7:14]))
-	r = append(r, r2...)
-	r3 := fixWeakKey(stretch56Bits(b[14:21]))
-	r = append(r, r3...)
-	return r
-}
-
-// DES3StringToKey returns a key derived from the string provided according to the definition in RFC 3961 for DES3 etypes.
-func DES3StringToKey(secret, salt string, e etype.EType) ([]byte, error) {
-	s := secret + salt
-	tkey := e.RandomToKey(Nfold([]byte(s), e.GetKeySeedBitLength()))
-	return e.DeriveKey(tkey, []byte("kerberos"))
-}
-
-// PseudoRandom function as defined in RFC 3961
-func PseudoRandom(key, b []byte, e etype.EType) ([]byte, error) {
-	h := e.GetHashFunc()()
-	h.Write(b)
-	tmp := h.Sum(nil)[:e.GetMessageBlockByteSize()]
-	k, err := e.DeriveKey(key, []byte(prfconstant))
-	if err != nil {
-		return []byte{}, err
-	}
-	_, prf, err := e.EncryptData(k, tmp)
-	if err != nil {
-		return []byte{}, err
-	}
-	return prf, nil
-}
-
-func stretch56Bits(b []byte) []byte {
-	d := make([]byte, len(b), len(b))
-	copy(d, b)
-	var lb byte
-	for i, v := range d {
-		bv, nb := calcEvenParity(v)
-		d[i] = nb
-		if bv != 0 {
-			lb = lb | (1 << uint(i+1))
-		} else {
-			lb = lb &^ (1 << uint(i+1))
-		}
-	}
-	_, lb = calcEvenParity(lb)
-	d = append(d, lb)
-	return d
-}
-
-func calcEvenParity(b byte) (uint8, uint8) {
-	lowestbit := b & 0x01
-	// c counter of 1s in the first 7 bits of the byte
-	var c int
-	// Iterate over the highest 7 bits (hence p starts at 1 not zero) and count the 1s.
-	for p := 1; p < 8; p++ {
-		v := b & (1 << uint(p))
-		if v != 0 {
-			c++
-		}
-	}
-	if c%2 == 0 {
-		//Even number of 1s so set parity to 1
-		b = b | 1
-	} else {
-		//Odd number of 1s so set parity to 0
-		b = b &^ 1
-	}
-	return lowestbit, b
-}
-
-func fixWeakKey(b []byte) []byte {
-	if weak(b) {
-		b[7] ^= 0xF0
-	}
-	return b
-}
-
-func weak(b []byte) bool {
-	// weak keys from https://nvlpubs.nist.gov/nistpubs/Legacy/SP/nistspecialpublication800-67r1.pdf
-	weakKeys := [4][]byte{
-		{0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01},
-		{0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE},
-		{0xE0, 0xE0, 0xE0, 0xE0, 0xF1, 0xF1, 0xF1, 0xF1},
-		{0x1F, 0x1F, 0x1F, 0x1F, 0x0E, 0x0E, 0x0E, 0x0E},
-	}
-	semiWeakKeys := [12][]byte{
-		{0x01, 0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E},
-		{0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E, 0x01},
-		{0x01, 0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1},
-		{0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1, 0x01},
-		{0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE},
-		{0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01},
-		{0x1F, 0xE0, 0x1F, 0xE0, 0x0E, 0xF1, 0x0E, 0xF1},
-		{0xE0, 0x1F, 0xE0, 0x1F, 0xF1, 0x0E, 0xF1, 0x0E},
-		{0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E, 0xFE},
-		{0xFE, 0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E},
-		{0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1, 0xFE},
-		{0xFE, 0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1},
-	}
-	for _, k := range weakKeys {
-		if bytes.Equal(b, k) {
-			return true
-		}
-	}
-	for _, k := range semiWeakKeys {
-		if bytes.Equal(b, k) {
-			return true
-		}
-	}
-	return false
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/nfold.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/nfold.go
deleted file mode 100644
index 779d1c6..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961/nfold.go
+++ /dev/null
@@ -1,128 +0,0 @@
-package rfc3961
-
-/*
-Implementation of the n-fold algorithm as defined in RFC 3961.
-
-n-fold is an algorithm that takes m input bits and "stretches" them
-to form n output bits with equal contribution from each input bit to
-the output, as described in [Blumenthal96]:
-
-We first define a primitive called n-folding, which takes a
-variable-length input block and produces a fixed-length output
-sequence.  The intent is to give each input bit approximately
-equal weight in determining the value of each output bit.  Note
-that whenever we need to treat a string of octets as a number, the
-assumed representation is Big-Endian -- Most Significant Byte
-first.
-
-To n-fold a number X, replicate the input value to a length that
-is the least common multiple of n and the length of X.  Before
-each repetition, the input is rotated to the right by 13 bit
-positions.  The successive n-bit chunks are added together using
-1's-complement addition (that is, with end-around carry) to yield
-a n-bit result....
-*/
-
-/* Credits
-This golang implementation of nfold used the following project for help with implementation detail.
-Although their source is in java it was helpful as a reference implementation of the RFC.
-You can find the source code of their open source project along with license information below.
-We acknowledge and are grateful to these developers for their contributions to open source
-
-Project: Apache Directory (http://http://directory.apache.org/)
-https://svn.apache.org/repos/asf/directory/apacheds/tags/1.5.1/kerberos-shared/src/main/java/org/apache/directory/server/kerberos/shared/crypto/encryption/NFold.java
-License: http://www.apache.org/licenses/LICENSE-2.0
-*/
-
-// Nfold expands the key to ensure it is not smaller than one cipher block.
-// Defined in RFC 3961.
-//
-// m input bytes that will be "stretched" to the least common multiple of n bits and the bit length of m.
-func Nfold(m []byte, n int) []byte {
-	k := len(m) * 8
-
-	//Get the lowest common multiple of the two bit sizes
-	lcm := lcm(n, k)
-	relicate := lcm / k
-	var sumBytes []byte
-
-	for i := 0; i < relicate; i++ {
-		rotation := 13 * i
-		sumBytes = append(sumBytes, rotateRight(m, rotation)...)
-	}
-
-	nfold := make([]byte, n/8)
-	sum := make([]byte, n/8)
-	for i := 0; i < lcm/n; i++ {
-		for j := 0; j < n/8; j++ {
-			sum[j] = sumBytes[j+(i*len(sum))]
-		}
-		nfold = onesComplementAddition(nfold, sum)
-	}
-	return nfold
-}
-
-func onesComplementAddition(n1, n2 []byte) []byte {
-	numBits := len(n1) * 8
-	out := make([]byte, numBits/8)
-	carry := 0
-	for i := numBits - 1; i > -1; i-- {
-		n1b := getBit(&n1, i)
-		n2b := getBit(&n2, i)
-		s := n1b + n2b + carry
-
-		if s == 0 || s == 1 {
-			setBit(&out, i, s)
-			carry = 0
-		} else if s == 2 {
-			carry = 1
-		} else if s == 3 {
-			setBit(&out, i, 1)
-			carry = 1
-		}
-	}
-	if carry == 1 {
-		carryArray := make([]byte, len(n1))
-		carryArray[len(carryArray)-1] = 1
-		out = onesComplementAddition(out, carryArray)
-	}
-	return out
-}
-
-func rotateRight(b []byte, step int) []byte {
-	out := make([]byte, len(b))
-	bitLen := len(b) * 8
-	for i := 0; i < bitLen; i++ {
-		v := getBit(&b, i)
-		setBit(&out, (i+step)%bitLen, v)
-	}
-	return out
-}
-
-func lcm(x, y int) int {
-	return (x * y) / gcd(x, y)
-}
-
-func gcd(x, y int) int {
-	for y != 0 {
-		x, y = y, x%y
-	}
-	return x
-}
-
-func getBit(b *[]byte, p int) int {
-	pByte := p / 8
-	pBit := uint(p % 8)
-	vByte := (*b)[pByte]
-	vInt := int(vByte >> (8 - (pBit + 1)) & 0x0001)
-	return vInt
-}
-
-func setBit(b *[]byte, p, v int) {
-	pByte := p / 8
-	pBit := uint(p % 8)
-	oldByte := (*b)[pByte]
-	var newByte byte
-	newByte = byte(v<<(8-(pBit+1))) | oldByte
-	(*b)[pByte] = newByte
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/encryption.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/encryption.go
deleted file mode 100644
index 2be2fde..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/encryption.go
+++ /dev/null
@@ -1,89 +0,0 @@
-// Package rfc3962 provides encryption and checksum methods as specified in RFC 3962
-package rfc3962
-
-import (
-	"crypto/rand"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/aescts.v1"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 3962.
-func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	ivz := make([]byte, e.GetCypherBlockBitLength()/8)
-	return aescts.Encrypt(key, ivz, data)
-}
-
-// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 3962.
-// The encrypted data is concatenated with its integrity hash to create an encrypted message.
-func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	//confounder
-	c := make([]byte, e.GetConfounderByteSize())
-	_, err := rand.Read(c)
-	if err != nil {
-		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
-	}
-	plainBytes := append(c, message...)
-
-	// Derive key for encryption from usage
-	var k []byte
-	if usage != 0 {
-		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
-		if err != nil {
-			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
-		}
-	}
-
-	// Encrypt the data
-	iv, b, err := e.EncryptData(k, plainBytes)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-
-	// Generate and append integrity hash
-	ih, err := common.GetIntegrityHash(plainBytes, key, usage, e)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-	b = append(b, ih...)
-	return iv, b, nil
-}
-
-// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 3962.
-func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	ivz := make([]byte, e.GetCypherBlockBitLength()/8)
-	return aescts.Decrypt(key, ivz, data)
-}
-
-// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 3962.
-// The integrity of the message is also verified.
-func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
-	//Derive the key
-	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
-	if err != nil {
-		return nil, fmt.Errorf("error deriving key: %v", err)
-	}
-	// Strip off the checksum from the end
-	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
-	if err != nil {
-		return nil, err
-	}
-	//Verify checksum
-	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
-		return nil, errors.New("integrity verification failed")
-	}
-	//Remove the confounder bytes
-	return b[e.GetConfounderByteSize():], nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/keyDerivation.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/keyDerivation.go
deleted file mode 100644
index a5f45c1..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962/keyDerivation.go
+++ /dev/null
@@ -1,58 +0,0 @@
-package rfc3962
-
-import (
-	"encoding/binary"
-	"encoding/hex"
-	"errors"
-
-	"github.com/jcmturner/gofork/x/crypto/pbkdf2"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-const (
-	s2kParamsZero = 4294967296
-)
-
-// StringToKey returns a key derived from the string provided according to the definition in RFC 3961.
-func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) {
-	i, err := S2KparamsToItertions(s2kparams)
-	if err != nil {
-		return nil, err
-	}
-	return StringToKeyIter(secret, salt, i, e)
-}
-
-// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0
-func StringToPBKDF2(secret, salt string, iterations int64, e etype.EType) []byte {
-	return pbkdf2.Key64([]byte(secret), []byte(salt), iterations, int64(e.GetKeyByteSize()), e.GetHashFunc())
-}
-
-// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 3961.
-func StringToKeyIter(secret, salt string, iterations int64, e etype.EType) ([]byte, error) {
-	tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e))
-	return e.DeriveKey(tkey, []byte("kerberos"))
-}
-
-// S2KparamsToItertions converts the string representation of iterations to an integer
-func S2KparamsToItertions(s2kparams string) (int64, error) {
-	//process s2kparams string
-	//The parameter string is four octets indicating an unsigned
-	//number in big-endian order.  This is the number of iterations to be
-	//performed.  If the value is 00 00 00 00, the number of iterations to
-	//be performed is 4,294,967,296 (2**32).
-	var i uint32
-	if len(s2kparams) != 8 {
-		return int64(s2kParamsZero), errors.New("invalid s2kparams length")
-	}
-	b, err := hex.DecodeString(s2kparams)
-	if err != nil {
-		return int64(s2kParamsZero), errors.New("invalid s2kparams, cannot decode string to bytes")
-	}
-	i = binary.BigEndian.Uint32(b)
-	//buf := bytes.NewBuffer(b)
-	//err = binary.Read(buf, binary.BigEndian, &i)
-	if err != nil {
-		return int64(s2kParamsZero), errors.New("invalid s2kparams, cannot convert to big endian int32")
-	}
-	return int64(i), nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/encryption.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/encryption.go
deleted file mode 100644
index 0ec8b99..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/encryption.go
+++ /dev/null
@@ -1,80 +0,0 @@
-// Package rfc4757 provides encryption and checksum methods as specified in RFC 4757
-package rfc4757
-
-import (
-	"crypto/hmac"
-	"crypto/rand"
-	"crypto/rc4"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-)
-
-// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 4757.
-func EncryptData(key, data []byte, e etype.EType) ([]byte, error) {
-	if len(key) != e.GetKeyByteSize() {
-		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	rc4Cipher, err := rc4.NewCipher(key)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error creating RC4 cipher: %v", err)
-	}
-	ed := make([]byte, len(data))
-	copy(ed, data)
-	rc4Cipher.XORKeyStream(ed, ed)
-	rc4Cipher.Reset()
-	return ed, nil
-}
-
-// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 4757.
-func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
-	return EncryptData(key, data, e)
-}
-
-// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 4757.
-// The encrypted data is concatenated with its RC4 header containing integrity checksum and confounder to create an encrypted message.
-func EncryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) {
-	confounder := make([]byte, e.GetConfounderByteSize()) // size = 8
-	_, err := rand.Read(confounder)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error generating confounder: %v", err)
-	}
-	k1 := key
-	k2 := HMAC(k1, UsageToMSMsgType(usage))
-	toenc := append(confounder, data...)
-	chksum := HMAC(k2, toenc)
-	k3 := HMAC(k2, chksum)
-
-	ed, err := EncryptData(k3, toenc, e)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error encrypting data: %v", err)
-	}
-
-	msg := append(chksum, ed...)
-	return msg, nil
-}
-
-// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 4757.
-// The integrity of the message is also verified.
-func DecryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) {
-	checksum := data[:e.GetHMACBitLength()/8]
-	ct := data[e.GetHMACBitLength()/8:]
-	_, k2, k3 := deriveKeys(key, checksum, usage, export)
-
-	pt, err := DecryptData(k3, ct, e)
-	if err != nil {
-		return []byte{}, fmt.Errorf("error decrypting data: %v", err)
-	}
-
-	if !VerifyIntegrity(k2, pt, data, e) {
-		return []byte{}, errors.New("integrity checksum incorrect")
-	}
-	return pt[e.GetConfounderByteSize():], nil
-}
-
-// VerifyIntegrity checks the integrity checksum of the data matches that calculated from the decrypted data.
-func VerifyIntegrity(key, pt, data []byte, e etype.EType) bool {
-	chksum := HMAC(key, pt)
-	return hmac.Equal(chksum, data[:e.GetHMACBitLength()/8])
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/keyDerivation.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/keyDerivation.go
deleted file mode 100644
index 5e7ec48..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757/keyDerivation.go
+++ /dev/null
@@ -1,55 +0,0 @@
-package rfc4757
-
-import (
-	"bytes"
-	"encoding/hex"
-	"errors"
-	"fmt"
-	"io"
-
-	"golang.org/x/crypto/md4"
-)
-
-// StringToKey returns a key derived from the string provided according to the definition in RFC 4757.
-func StringToKey(secret string) ([]byte, error) {
-	b := make([]byte, len(secret)*2, len(secret)*2)
-	for i, r := range secret {
-		u := fmt.Sprintf("%04x", r)
-		c, err := hex.DecodeString(u)
-		if err != nil {
-			return []byte{}, errors.New("character could not be encoded")
-		}
-		// Swap round the two bytes to make little endian as we put into byte slice
-		b[2*i] = c[1]
-		b[2*i+1] = c[0]
-	}
-	r := bytes.NewReader(b)
-	h := md4.New()
-	_, err := io.Copy(h, r)
-	if err != nil {
-		return []byte{}, err
-	}
-	return h.Sum(nil), nil
-}
-
-func deriveKeys(key, checksum []byte, usage uint32, export bool) (k1, k2, k3 []byte) {
-	//if export {
-	//	L40 := make([]byte, 14, 14)
-	//	copy(L40, []byte(`fortybits`))
-	//	k1 = HMAC(key, L40)
-	//} else {
-	//	tb := MessageTypeBytes(usage)
-	//	k1 = HMAC(key, tb)
-	//}
-	//k2 = k1[:16]
-	//if export {
-	//	mask := []byte{0xAB,0xAB,0xAB,0xAB,0xAB,0xAB,0xAB,0xAB,0xAB}
-	//	copy(k1[7:16], mask)
-	//}
-	//k3 = HMAC(k1, checksum)
-	//return
-	k1 = key
-	k2 = HMAC(k1, UsageToMSMsgType(usage))
-	k3 = HMAC(k2, checksum)
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/encryption.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/encryption.go
deleted file mode 100644
index 86aae09..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/encryption.go
+++ /dev/null
@@ -1,128 +0,0 @@
-// Package rfc8009 provides encryption and checksum methods as specified in RFC 8009
-package rfc8009
-
-import (
-	"crypto/aes"
-	"crypto/hmac"
-	"crypto/rand"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/aescts.v1"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/common"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 8009.
-func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) {
-	kl := e.GetKeyByteSize()
-	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
-		kl = 32
-	}
-	if len(key) != kl {
-		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key))
-	}
-	ivz := make([]byte, aes.BlockSize)
-	return aescts.Encrypt(key, ivz, data)
-}
-
-// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 8009.
-// The encrypted data is concatenated with its integrity hash to create an encrypted message.
-func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) {
-	kl := e.GetKeyByteSize()
-	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
-		kl = 32
-	}
-	if len(key) != kl {
-		return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key))
-	}
-	if len(key) != e.GetKeyByteSize() {
-	}
-	//confounder
-	c := make([]byte, e.GetConfounderByteSize())
-	_, err := rand.Read(c)
-	if err != nil {
-		return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err)
-	}
-	plainBytes := append(c, message...)
-
-	// Derive key for encryption from usage
-	var k []byte
-	if usage != 0 {
-		k, err = e.DeriveKey(key, common.GetUsageKe(usage))
-		if err != nil {
-			return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err)
-		}
-	}
-
-	// Encrypt the data
-	iv, b, err := e.EncryptData(k, plainBytes)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-
-	ivz := make([]byte, e.GetConfounderByteSize())
-	ih, err := GetIntegityHash(ivz, b, key, usage, e)
-	if err != nil {
-		return iv, b, fmt.Errorf("error encrypting data: %v", err)
-	}
-	b = append(b, ih...)
-	return iv, b, nil
-}
-
-// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 8009.
-func DecryptData(key, data []byte, e etype.EType) ([]byte, error) {
-	kl := e.GetKeyByteSize()
-	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
-		kl = 32
-	}
-	if len(key) != kl {
-		return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key))
-	}
-	ivz := make([]byte, aes.BlockSize)
-	return aescts.Decrypt(key, ivz, data)
-}
-
-// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 8009.
-// The integrity of the message is also verified.
-func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) {
-	//Derive the key
-	k, err := e.DeriveKey(key, common.GetUsageKe(usage))
-	if err != nil {
-		return nil, fmt.Errorf("error deriving key: %v", err)
-	}
-	// Strip off the checksum from the end
-	b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8])
-	if err != nil {
-		return nil, err
-	}
-	//Verify checksum
-	if !e.VerifyIntegrity(key, ciphertext, b, usage) {
-		return nil, errors.New("integrity verification failed")
-	}
-	//Remove the confounder bytes
-	return b[e.GetConfounderByteSize():], nil
-}
-
-// GetIntegityHash returns a keyed integrity hash of the bytes provided as defined in RFC 8009
-func GetIntegityHash(iv, c, key []byte, usage uint32, e etype.EType) ([]byte, error) {
-	// Generate and append integrity hash
-	// The HMAC is calculated over the cipher state concatenated with the
-	// AES output, instead of being calculated over the confounder and
-	// plaintext.  This allows the message receiver to verify the
-	// integrity of the message before decrypting the message.
-	// H = HMAC(Ki, IV | C)
-	ib := append(iv, c...)
-	return common.GetIntegrityHash(ib, key, usage, e)
-}
-
-// VerifyIntegrity verifies the integrity of cipertext bytes ct.
-func VerifyIntegrity(key, ct []byte, usage uint32, etype etype.EType) bool {
-	h := make([]byte, etype.GetHMACBitLength()/8)
-	copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:])
-	ivz := make([]byte, etype.GetConfounderByteSize())
-	ib := append(ivz, ct[:len(ct)-(etype.GetHMACBitLength()/8)]...)
-	expectedMAC, _ := common.GetIntegrityHash(ib, key, usage, etype)
-	return hmac.Equal(h, expectedMAC)
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/keyDerivation.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/keyDerivation.go
deleted file mode 100644
index 90ced3b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009/keyDerivation.go
+++ /dev/null
@@ -1,144 +0,0 @@
-package rfc8009
-
-import (
-	"crypto/hmac"
-	"encoding/binary"
-	"encoding/hex"
-	"errors"
-
-	"golang.org/x/crypto/pbkdf2"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto/etype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/etypeID"
-)
-
-const (
-	s2kParamsZero = 32768
-)
-
-// DeriveRandom for key derivation as defined in RFC 8009
-func DeriveRandom(protocolKey, usage []byte, e etype.EType) ([]byte, error) {
-	h := e.GetHashFunc()()
-	return KDF_HMAC_SHA2(protocolKey, []byte("prf"), usage, h.Size(), e), nil
-}
-
-// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods.
-//
-// https://tools.ietf.org/html/rfc8009#section-5
-//
-// If the enctype is aes128-cts-hmac-sha256-128:
-// Kc = KDF-HMAC-SHA2(base-key, usage | 0x99, 128)
-// Ke = KDF-HMAC-SHA2(base-key, usage | 0xAA, 128)
-// Ki = KDF-HMAC-SHA2(base-key, usage | 0x55, 128)
-//
-// If the enctype is aes256-cts-hmac-sha384-192:
-// Kc = KDF-HMAC-SHA2(base-key, usage | 0x99, 192)
-// Ke = KDF-HMAC-SHA2(base-key, usage | 0xAA, 256)
-// Ki = KDF-HMAC-SHA2(base-key, usage | 0x55, 192)
-func DeriveKey(protocolKey, label []byte, e etype.EType) []byte {
-	var context []byte
-	var kl int
-	// Key length is longer for aes256-cts-hmac-sha384-192 is it is a Ke or from StringToKey (where label is "kerberos")
-	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
-		switch label[len(label)-1] {
-		case 0x73:
-			// 0x73 is "s" so label could be kerberos meaning StringToKey so now check if the label is "kerberos"
-			kerblabel := []byte("kerberos")
-			if len(label) != len(kerblabel) {
-				break
-			}
-			for i, b := range label {
-				if b != kerblabel[i] {
-					kl = e.GetKeySeedBitLength()
-					break
-				}
-			}
-			if kl == 0 {
-				// This is StringToKey
-				kl = 256
-			}
-		case 0xAA:
-			// This is a Ke
-			kl = 256
-		}
-	}
-	if kl == 0 {
-		kl = e.GetKeySeedBitLength()
-	}
-	return e.RandomToKey(KDF_HMAC_SHA2(protocolKey, label, context, kl, e))
-}
-
-// RandomToKey returns a key from the bytes provided according to the definition in RFC 8009.
-func RandomToKey(b []byte) []byte {
-	return b
-}
-
-// StringToKey returns a key derived from the string provided according to the definition in RFC 8009.
-func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) {
-	i, err := S2KparamsToItertions(s2kparams)
-	if err != nil {
-		return nil, err
-	}
-	return StringToKeyIter(secret, salt, i, e)
-}
-
-// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 8009.
-func StringToKeyIter(secret, salt string, iterations int, e etype.EType) ([]byte, error) {
-	tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e))
-	return e.DeriveKey(tkey, []byte("kerberos"))
-}
-
-// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0
-func StringToPBKDF2(secret, salt string, iterations int, e etype.EType) []byte {
-	kl := e.GetKeyByteSize()
-	if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 {
-		kl = 32
-	}
-	return pbkdf2.Key([]byte(secret), []byte(salt), iterations, kl, e.GetHashFunc())
-}
-
-// KDF_HMAC_SHA2 key derivation: https://tools.ietf.org/html/rfc8009#section-3
-func KDF_HMAC_SHA2(protocolKey, label, context []byte, kl int, e etype.EType) []byte {
-	//k: Length in bits of the key to be outputted, expressed in big-endian binary representation in 4 bytes.
-	k := make([]byte, 4, 4)
-	binary.BigEndian.PutUint32(k, uint32(kl))
-
-	c := make([]byte, 4, 4)
-	binary.BigEndian.PutUint32(c, uint32(1))
-	c = append(c, label...)
-	c = append(c, byte(0))
-	if len(context) > 0 {
-		c = append(c, context...)
-	}
-	c = append(c, k...)
-
-	mac := hmac.New(e.GetHashFunc(), protocolKey)
-	mac.Write(c)
-	return mac.Sum(nil)[:(kl / 8)]
-}
-
-// GetSaltP returns the salt value based on the etype name: https://tools.ietf.org/html/rfc8009#section-4
-func GetSaltP(salt, ename string) string {
-	b := []byte(ename)
-	b = append(b, byte(0))
-	b = append(b, []byte(salt)...)
-	return string(b)
-}
-
-// S2KparamsToItertions converts the string representation of iterations to an integer for RFC 8009.
-func S2KparamsToItertions(s2kparams string) (int, error) {
-	var i uint32
-	if len(s2kparams) != 8 {
-		return s2kParamsZero, errors.New("Invalid s2kparams length")
-	}
-	b, err := hex.DecodeString(s2kparams)
-	if err != nil {
-		return s2kParamsZero, errors.New("Invalid s2kparams, cannot decode string to bytes")
-	}
-	i = binary.BigEndian.Uint32(b)
-	//buf := bytes.NewBuffer(b)
-	//err = binary.Read(buf, binary.BigEndian, &i)
-	if err != nil {
-		return s2kParamsZero, errors.New("Invalid s2kparams, cannot convert to big endian int32")
-	}
-	return int(i), nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/MICToken.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/MICToken.go
deleted file mode 100644
index 856412b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/MICToken.go
+++ /dev/null
@@ -1,202 +0,0 @@
-package gssapi
-
-import (
-	"bytes"
-	"crypto/hmac"
-	"encoding/binary"
-	"encoding/hex"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-/*
-From RFC 4121, section 4.2.6.1:
-
-   Use of the GSS_GetMIC() call yields a token (referred as the MIC
-   token in this document), separate from the user data being protected,
-   which can be used to verify the integrity of that data as received.
-   The token has the following format:
-
-         Octet no   Name        Description
-         --------------------------------------------------------------
-         0..1     TOK_ID     Identification field.  Tokens emitted by
-                             GSS_GetMIC() contain the hex value 04 04
-                             expressed in big-endian order in this
-                             field.
-         2        Flags      Attributes field, as described in section
-                             4.2.2.
-         3..7     Filler     Contains five octets of hex value FF.
-         8..15    SND_SEQ    Sequence number field in clear text,
-                             expressed in big-endian order.
-         16..last SGN_CKSUM  Checksum of the "to-be-signed" data and
-                             octet 0..15, as described in section 4.2.4.
-
-   The Filler field is included in the checksum calculation for
-   simplicity.
-
-*/
-
-const (
-	// MICTokenFlagSentByAcceptor - this flag indicates the sender is the context acceptor.  When not set, it indicates the sender is the context initiator
-	MICTokenFlagSentByAcceptor = 1 << iota
-	// MICTokenFlagSealed - this flag indicates confidentiality is provided for.  It SHALL NOT be set in MIC tokens
-	MICTokenFlagSealed
-	// MICTokenFlagAcceptorSubkey - a subkey asserted by the context acceptor is used to protect the message
-	MICTokenFlagAcceptorSubkey
-)
-
-const (
-	micHdrLen = 16 // Length of the MIC Token's header
-)
-
-// MICToken represents a GSS API MIC token, as defined in RFC 4121.
-// It contains the header fields, the payload (this is not transmitted) and
-// the checksum, and provides the logic for converting to/from bytes plus
-// computing and verifying checksums
-type MICToken struct {
-	// const GSS Token ID: 0x0404
-	Flags byte // contains three flags: acceptor, sealed, acceptor subkey
-	// const Filler: 0xFF 0xFF 0xFF 0xFF 0xFF
-	SndSeqNum uint64 // sender's sequence number. big-endian
-	Payload   []byte // your data! :)
-	Checksum  []byte // checksum of { payload | header }
-}
-
-// Return the 2 bytes identifying a GSS API MIC token
-func getGSSMICTokenID() *[2]byte {
-	return &[2]byte{0x04, 0x04}
-}
-
-// Return the filler bytes used in header
-func fillerBytes() *[5]byte {
-	return &[5]byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF}
-}
-
-// Marshal the MICToken into a byte slice.
-// The payload should have been set and the checksum computed, otherwise an error is returned.
-func (mt *MICToken) Marshal() ([]byte, error) {
-	if mt.Checksum == nil {
-		return nil, errors.New("checksum has not been set")
-	}
-
-	bytes := make([]byte, micHdrLen+len(mt.Checksum))
-	copy(bytes[0:micHdrLen], mt.getMICChecksumHeader()[:])
-	copy(bytes[micHdrLen:], mt.Checksum)
-
-	return bytes, nil
-}
-
-// SetChecksum uses the passed encryption key and key usage to compute the checksum over the payload and
-// the header, and sets the Checksum field of this MICToken.
-// If the payload has not been set or the checksum has already been set, an error is returned.
-func (mt *MICToken) SetChecksum(key types.EncryptionKey, keyUsage uint32) error {
-	if mt.Checksum != nil {
-		return errors.New("checksum has already been computed")
-	}
-	checksum, err := mt.checksum(key, keyUsage)
-	if err != nil {
-		return err
-	}
-	mt.Checksum = checksum
-	return nil
-}
-
-// Compute and return the checksum of this token, computed using the passed key and key usage.
-// Confirms to RFC 4121 in that the checksum will be computed over { body | header }.
-// In the context of Kerberos MIC tokens, mostly keyusage GSSAPI_ACCEPTOR_SIGN (=23)
-// and GSSAPI_INITIATOR_SIGN (=25) will be used.
-// Note: This will NOT update the struct's Checksum field.
-func (mt *MICToken) checksum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) {
-	if mt.Payload == nil {
-		return nil, errors.New("cannot compute checksum with uninitialized payload")
-	}
-	d := make([]byte, micHdrLen+len(mt.Payload))
-	copy(d[0:], mt.Payload)
-	copy(d[len(mt.Payload):], mt.getMICChecksumHeader())
-
-	encType, err := crypto.GetEtype(key.KeyType)
-	if err != nil {
-		return nil, err
-	}
-	return encType.GetChecksumHash(key.KeyValue, d, keyUsage)
-}
-
-// Build a header suitable for a checksum computation
-func (mt *MICToken) getMICChecksumHeader() []byte {
-	header := make([]byte, micHdrLen)
-	copy(header[0:2], getGSSMICTokenID()[:])
-	header[2] = mt.Flags
-	copy(header[3:8], fillerBytes()[:])
-	binary.BigEndian.PutUint64(header[8:16], mt.SndSeqNum)
-	return header
-}
-
-// Verify computes the token's checksum with the provided key and usage,
-// and compares it to the checksum present in the token.
-// In case of any failure, (false, err) is returned, with err an explanatory error.
-func (mt *MICToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) {
-	computed, err := mt.checksum(key, keyUsage)
-	if err != nil {
-		return false, err
-	}
-	if !hmac.Equal(computed, mt.Checksum) {
-		return false, fmt.Errorf(
-			"checksum mismatch. Computed: %s, Contained in token: %s",
-			hex.EncodeToString(computed), hex.EncodeToString(mt.Checksum))
-	}
-	return true, nil
-}
-
-// Unmarshal bytes into the corresponding MICToken.
-// If expectFromAcceptor is true we expect the token to have been emitted by the gss acceptor,
-// and will check the according flag, returning an error if the token does not match the expectation.
-func (mt *MICToken) Unmarshal(b []byte, expectFromAcceptor bool) error {
-	if len(b) < micHdrLen {
-		return errors.New("bytes shorter than header length")
-	}
-	if !bytes.Equal(getGSSMICTokenID()[:], b[0:2]) {
-		return fmt.Errorf("wrong Token ID, Expected %s, was %s",
-			hex.EncodeToString(getGSSMICTokenID()[:]),
-			hex.EncodeToString(b[0:2]))
-	}
-	flags := b[2]
-	isFromAcceptor := flags&MICTokenFlagSentByAcceptor != 0
-	if isFromAcceptor && !expectFromAcceptor {
-		return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor")
-	}
-	if !isFromAcceptor && expectFromAcceptor {
-		return errors.New("unexpected acceptor flag is not set: expecting a token from the acceptor, not in the initiator")
-	}
-	if !bytes.Equal(b[3:8], fillerBytes()[:]) {
-		return fmt.Errorf("unexpected filler bytes: expecting %s, was %s",
-			hex.EncodeToString(fillerBytes()[:]),
-			hex.EncodeToString(b[3:8]))
-	}
-
-	mt.Flags = flags
-	mt.SndSeqNum = binary.BigEndian.Uint64(b[8:16])
-	mt.Checksum = b[micHdrLen:]
-	return nil
-}
-
-// NewInitiatorMICToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum.
-// Other flags are set to 0.
-// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier.
-// This is currently not supported.
-func NewInitiatorMICToken(payload []byte, key types.EncryptionKey) (*MICToken, error) {
-	token := MICToken{
-		Flags:     0x00,
-		SndSeqNum: 0,
-		Payload:   payload,
-	}
-
-	if err := token.SetChecksum(key, keyusage.GSSAPI_INITIATOR_SIGN); err != nil {
-		return nil, err
-	}
-
-	return &token, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/contextFlags.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/contextFlags.go
deleted file mode 100644
index 6634c6d..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/contextFlags.go
+++ /dev/null
@@ -1,25 +0,0 @@
-package gssapi
-
-import "github.com/jcmturner/gofork/encoding/asn1"
-
-// GSS-API context flags assigned numbers.
-const (
-	ContextFlagDeleg    = 1
-	ContextFlagMutual   = 2
-	ContextFlagReplay   = 4
-	ContextFlagSequence = 8
-	ContextFlagConf     = 16
-	ContextFlagInteg    = 32
-	ContextFlagAnon     = 64
-)
-
-// ContextFlags flags for GSSAPI
-type ContextFlags asn1.BitString
-
-// NewContextFlags creates a new ContextFlags instance.
-func NewContextFlags() ContextFlags {
-	var c ContextFlags
-	c.BitLength = 32
-	c.Bytes = make([]byte, 4)
-	return c
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/gssapi.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/gssapi.go
deleted file mode 100644
index 47754d7..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/gssapi.go
+++ /dev/null
@@ -1,199 +0,0 @@
-// Package gssapi implements Generic Security Services Application Program Interface required for SPNEGO kerberos authentication.
-package gssapi
-
-import (
-	"context"
-	"fmt"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-)
-
-// GSS-API OID names
-const (
-	// GSS-API OID names
-	OIDKRB5         OIDName = "KRB5"         // MechType OID for Kerberos 5
-	OIDMSLegacyKRB5 OIDName = "MSLegacyKRB5" // MechType OID for Kerberos 5
-	OIDSPNEGO       OIDName = "SPNEGO"
-)
-
-// GSS-API status values
-const (
-	StatusBadBindings = 1 << iota
-	StatusBadMech
-	StatusBadName
-	StatusBadNameType
-	StatusBadStatus
-	StatusBadSig
-	StatusBadMIC
-	StatusContextExpired
-	StatusCredentialsExpired
-	StatusDefectiveCredential
-	StatusDefectiveToken
-	StatusFailure
-	StatusNoContext
-	StatusNoCred
-	StatusBadQOP
-	StatusUnauthorized
-	StatusUnavailable
-	StatusDuplicateElement
-	StatusNameNotMN
-	StatusComplete
-	StatusContinueNeeded
-	StatusDuplicateToken
-	StatusOldToken
-	StatusUnseqToken
-	StatusGapToken
-)
-
-// ContextToken is an interface for a GSS-API context token.
-type ContextToken interface {
-	Marshal() ([]byte, error)
-	Unmarshal(b []byte) error
-	Verify() (bool, Status)
-	Context() context.Context
-}
-
-/*
-CREDENTIAL MANAGEMENT
-
-GSS_Acquire_cred             acquire credentials for use
-GSS_Release_cred             release credentials after use
-GSS_Inquire_cred             display information about credentials
-GSS_Add_cred                 construct credentials incrementally
-GSS_Inquire_cred_by_mech     display per-mechanism credential information
-
-CONTEXT-LEVEL CALLS
-
-GSS_Init_sec_context         initiate outbound security context
-GSS_Accept_sec_context       accept inbound security context
-GSS_Delete_sec_context       flush context when no longer needed
-GSS_Process_context_token    process received control token on context
-GSS_Context_time             indicate validity time remaining on context
-GSS_Inquire_context          display information about context
-GSS_Wrap_size_limit          determine GSS_Wrap token size limit
-GSS_Export_sec_context       transfer context to other process
-GSS_Import_sec_context       import transferred context
-
-PER-MESSAGE CALLS
-
-GSS_GetMIC                   apply integrity check, receive as token separate from message
-GSS_VerifyMIC                validate integrity check token along with message
-GSS_Wrap                     sign, optionally encrypt, encapsulate
-GSS_Unwrap                   decapsulate, decrypt if needed, validate integrity check
-
-SUPPORT CALLS
-
-GSS_Display_status           translate status codes to printable form
-GSS_Indicate_mechs           indicate mech_types supported on local system
-GSS_Compare_name             compare two names for equality
-GSS_Display_name             translate name to printable form
-GSS_Import_name              convert printable name to normalized form
-GSS_Release_name             free storage of normalized-form name
-GSS_Release_buffer           free storage of general GSS-allocated object
-GSS_Release_OID_set          free storage of OID set object
-GSS_Create_empty_OID_set     create empty OID set
-GSS_Add_OID_set_member       add member to OID set
-GSS_Test_OID_set_member      test if OID is member of OID set
-GSS_Inquire_names_for_mech   indicate name types supported by mechanism
-GSS_Inquire_mechs_for_name   indicates mechanisms supporting name type
-GSS_Canonicalize_name        translate name to per-mechanism form
-GSS_Export_name              externalize per-mechanism name
-GSS_Duplicate_name           duplicate name object
-*/
-
-// Mechanism is the GSS-API interface for authentication mechanisms.
-type Mechanism interface {
-	OID() asn1.ObjectIdentifier
-	AcquireCred() error                                               // acquire credentials for use (eg. AS exchange for KRB5)
-	InitSecContext() (ContextToken, error)                            // initiate outbound security context (eg TGS exchange builds AP_REQ to go into ContextToken to send to service)
-	AcceptSecContext(ct ContextToken) (bool, context.Context, Status) // service verifies the token server side to establish a context
-	MIC() MICToken                                                    // apply integrity check, receive as token separate from message
-	VerifyMIC(mt MICToken) (bool, error)                              // validate integrity check token along with message
-	Wrap(msg []byte) WrapToken                                        // sign, optionally encrypt, encapsulate
-	Unwrap(wt WrapToken) []byte                                       // decapsulate, decrypt if needed, validate integrity check
-}
-
-// OIDName is the type for defined GSS-API OIDs.
-type OIDName string
-
-// OID returns the OID for the provided OID name.
-func OID(o OIDName) asn1.ObjectIdentifier {
-	switch o {
-	case OIDSPNEGO:
-		return asn1.ObjectIdentifier{1, 3, 6, 1, 5, 5, 2}
-	case OIDKRB5:
-		return asn1.ObjectIdentifier{1, 2, 840, 113554, 1, 2, 2}
-	case OIDMSLegacyKRB5:
-		return asn1.ObjectIdentifier{1, 2, 840, 48018, 1, 2, 2}
-	}
-	return asn1.ObjectIdentifier{}
-}
-
-// Status is the GSS-API status and implements the error interface.
-type Status struct {
-	Code    int
-	Message string
-}
-
-// Error returns the Status description.
-func (s Status) Error() string {
-	var str string
-	switch s.Code {
-	case StatusBadBindings:
-		str = "channel binding mismatch"
-	case StatusBadMech:
-		str = "unsupported mechanism requested"
-	case StatusBadName:
-		str = "invalid name provided"
-	case StatusBadNameType:
-		str = "name of unsupported type provided"
-	case StatusBadStatus:
-		str = "invalid input status selector"
-	case StatusBadSig:
-		str = "token had invalid integrity check"
-	case StatusBadMIC:
-		str = "preferred alias for GSS_S_BAD_SIG"
-	case StatusContextExpired:
-		str = "specified security context expired"
-	case StatusCredentialsExpired:
-		str = "expired credentials detected"
-	case StatusDefectiveCredential:
-		str = "defective credential detected"
-	case StatusDefectiveToken:
-		str = "defective token detected"
-	case StatusFailure:
-		str = "failure, unspecified at GSS-API level"
-	case StatusNoContext:
-		str = "no valid security context specified"
-	case StatusNoCred:
-		str = "no valid credentials provided"
-	case StatusBadQOP:
-		str = "unsupported QOP valu"
-	case StatusUnauthorized:
-		str = "operation unauthorized"
-	case StatusUnavailable:
-		str = "operation unavailable"
-	case StatusDuplicateElement:
-		str = "duplicate credential element requested"
-	case StatusNameNotMN:
-		str = "name contains multi-mechanism elements"
-	case StatusComplete:
-		str = "normal completion"
-	case StatusContinueNeeded:
-		str = "continuation call to routine required"
-	case StatusDuplicateToken:
-		str = "duplicate per-message token detected"
-	case StatusOldToken:
-		str = "timed-out per-message token detected"
-	case StatusUnseqToken:
-		str = "reordered (early) per-message token detected"
-	case StatusGapToken:
-		str = "skipped predecessor token(s) detected"
-	default:
-		str = "unknown GSS-API error status"
-	}
-	if s.Message != "" {
-		return fmt.Sprintf("%s: %s", str, s.Message)
-	}
-	return str
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/wrapToken.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/wrapToken.go
deleted file mode 100644
index 9dbf96b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/gssapi/wrapToken.go
+++ /dev/null
@@ -1,235 +0,0 @@
-package gssapi
-
-import (
-	"bytes"
-	"crypto/hmac"
-	"encoding/binary"
-	"encoding/hex"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-/*
-From RFC 4121, section 4.2.6.2:
-
-   Use of the GSS_Wrap() call yields a token (referred as the Wrap token
-   in this document), which consists of a descriptive header, followed
-   by a body portion that contains either the input user data in
-   plaintext concatenated with the checksum, or the input user data
-   encrypted.  The GSS_Wrap() token SHALL have the following format:
-
-         Octet no   Name        Description
-         --------------------------------------------------------------
-          0..1     TOK_ID    Identification field.  Tokens emitted by
-                             GSS_Wrap() contain the hex value 05 04
-                             expressed in big-endian order in this
-                             field.
-          2        Flags     Attributes field, as described in section
-                             4.2.2.
-          3        Filler    Contains the hex value FF.
-          4..5     EC        Contains the "extra count" field, in big-
-                             endian order as described in section 4.2.3.
-          6..7     RRC       Contains the "right rotation count" in big-
-                             endian order, as described in section
-                             4.2.5.
-          8..15    SndSeqNum   Sequence number field in clear text,
-                             expressed in big-endian order.
-          16..last Data      Encrypted data for Wrap tokens with
-                             confidentiality, or plaintext data followed
-                             by the checksum for Wrap tokens without
-                             confidentiality, as described in section
-                             4.2.4.
-
-Quick notes:
-	- "EC" or "Extra Count" refers to the length of the checksum.
-	- "Flags" (complete details in section 4.2.2) is a set of bits:
-		- if bit 0 is set, it means the token was sent by the acceptor (generally the kerberized service).
-		- bit 1 indicates that the token's payload is encrypted
- 		- bit 2 indicates if the message is protected using a subkey defined by the acceptor.
-	- When computing checksums, EC and RRC MUST be set to 0.
-    - Wrap Tokens are not ASN.1 encoded.
-*/
-const (
-	HdrLen          = 16 // Length of the Wrap Token's header
-	FillerByte byte = 0xFF
-)
-
-// WrapToken represents a GSS API Wrap token, as defined in RFC 4121.
-// It contains the header fields, the payload and the checksum, and provides
-// the logic for converting to/from bytes plus computing and verifying checksums
-type WrapToken struct {
-	// const GSS Token ID: 0x0504
-	Flags byte // contains three flags: acceptor, sealed, acceptor subkey
-	// const Filler: 0xFF
-	EC        uint16 // checksum length. big-endian
-	RRC       uint16 // right rotation count. big-endian
-	SndSeqNum uint64 // sender's sequence number. big-endian
-	Payload   []byte // your data! :)
-	CheckSum  []byte // authenticated checksum of { payload | header }
-}
-
-// Return the 2 bytes identifying a GSS API Wrap token
-func getGssWrapTokenId() *[2]byte {
-	return &[2]byte{0x05, 0x04}
-}
-
-// Marshal the WrapToken into a byte slice.
-// The payload should have been set and the checksum computed, otherwise an error is returned.
-func (wt *WrapToken) Marshal() ([]byte, error) {
-	if wt.CheckSum == nil {
-		return nil, errors.New("checksum has not been set")
-	}
-	if wt.Payload == nil {
-		return nil, errors.New("payload has not been set")
-	}
-
-	pldOffset := HdrLen                    // Offset of the payload in the token
-	chkSOffset := HdrLen + len(wt.Payload) // Offset of the checksum in the token
-
-	bytes := make([]byte, chkSOffset+int(wt.EC))
-	copy(bytes[0:], getGssWrapTokenId()[:])
-	bytes[2] = wt.Flags
-	bytes[3] = FillerByte
-	binary.BigEndian.PutUint16(bytes[4:6], wt.EC)
-	binary.BigEndian.PutUint16(bytes[6:8], wt.RRC)
-	binary.BigEndian.PutUint64(bytes[8:16], wt.SndSeqNum)
-	copy(bytes[pldOffset:], wt.Payload)
-	copy(bytes[chkSOffset:], wt.CheckSum)
-	return bytes, nil
-}
-
-// SetCheckSum uses the passed encryption key and key usage to compute the checksum over the payload and
-// the header, and sets the CheckSum field of this WrapToken.
-// If the payload has not been set or the checksum has already been set, an error is returned.
-func (wt *WrapToken) SetCheckSum(key types.EncryptionKey, keyUsage uint32) error {
-	if wt.Payload == nil {
-		return errors.New("payload has not been set")
-	}
-	if wt.CheckSum != nil {
-		return errors.New("checksum has already been computed")
-	}
-	chkSum, cErr := wt.computeCheckSum(key, keyUsage)
-	if cErr != nil {
-		return cErr
-	}
-	wt.CheckSum = chkSum
-	return nil
-}
-
-// ComputeCheckSum computes and returns the checksum of this token, computed using the passed key and key usage.
-// Conforms to RFC 4121 in that the checksum will be computed over { body | header },
-// with the EC and RRC flags zeroed out.
-// In the context of Kerberos Wrap tokens, mostly keyusage GSSAPI_ACCEPTOR_SEAL (=22)
-// and GSSAPI_INITIATOR_SEAL (=24) will be used.
-// Note: This will NOT update the struct's Checksum field.
-func (wt *WrapToken) computeCheckSum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) {
-	if wt.Payload == nil {
-		return nil, errors.New("cannot compute checksum with uninitialized payload")
-	}
-	// Build a slice containing { payload | header }
-	checksumMe := make([]byte, HdrLen+len(wt.Payload))
-	copy(checksumMe[0:], wt.Payload)
-	copy(checksumMe[len(wt.Payload):], getChecksumHeader(wt.Flags, wt.SndSeqNum))
-
-	encType, err := crypto.GetEtype(key.KeyType)
-	if err != nil {
-		return nil, err
-	}
-	return encType.GetChecksumHash(key.KeyValue, checksumMe, keyUsage)
-}
-
-// Build a header suitable for a checksum computation
-func getChecksumHeader(flags byte, senderSeqNum uint64) []byte {
-	header := make([]byte, 16)
-	copy(header[0:], []byte{0x05, 0x04, flags, 0xFF, 0x00, 0x00, 0x00, 0x00})
-	binary.BigEndian.PutUint64(header[8:], senderSeqNum)
-	return header
-}
-
-// Verify computes the token's checksum with the provided key and usage,
-// and compares it to the checksum present in the token.
-// In case of any failure, (false, Err) is returned, with Err an explanatory error.
-func (wt *WrapToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) {
-	computed, cErr := wt.computeCheckSum(key, keyUsage)
-	if cErr != nil {
-		return false, cErr
-	}
-	if !hmac.Equal(computed, wt.CheckSum) {
-		return false, fmt.Errorf(
-			"checksum mismatch. Computed: %s, Contained in token: %s",
-			hex.EncodeToString(computed), hex.EncodeToString(wt.CheckSum))
-	}
-	return true, nil
-}
-
-// Unmarshal bytes into the corresponding WrapToken.
-// If expectFromAcceptor is true, we expect the token to have been emitted by the gss acceptor,
-// and will check the according flag, returning an error if the token does not match the expectation.
-func (wt *WrapToken) Unmarshal(b []byte, expectFromAcceptor bool) error {
-	// Check if we can read a whole header
-	if len(b) < 16 {
-		return errors.New("bytes shorter than header length")
-	}
-	// Is the Token ID correct?
-	if !bytes.Equal(getGssWrapTokenId()[:], b[0:2]) {
-		return fmt.Errorf("wrong Token ID. Expected %s, was %s",
-			hex.EncodeToString(getGssWrapTokenId()[:]),
-			hex.EncodeToString(b[0:2]))
-	}
-	// Check the acceptor flag
-	flags := b[2]
-	isFromAcceptor := flags&0x01 == 1
-	if isFromAcceptor && !expectFromAcceptor {
-		return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor")
-	}
-	if !isFromAcceptor && expectFromAcceptor {
-		return errors.New("expected acceptor flag is not set: expecting a token from the acceptor, not the initiator")
-	}
-	// Check the filler byte
-	if b[3] != FillerByte {
-		return fmt.Errorf("unexpected filler byte: expecting 0xFF, was %s ", hex.EncodeToString(b[3:4]))
-	}
-	checksumL := binary.BigEndian.Uint16(b[4:6])
-	// Sanity check on the checksum length
-	if int(checksumL) > len(b)-HdrLen {
-		return fmt.Errorf("inconsistent checksum length: %d bytes to parse, checksum length is %d", len(b), checksumL)
-	}
-
-	wt.Flags = flags
-	wt.EC = checksumL
-	wt.RRC = binary.BigEndian.Uint16(b[6:8])
-	wt.SndSeqNum = binary.BigEndian.Uint64(b[8:16])
-	wt.Payload = b[16 : len(b)-int(checksumL)]
-	wt.CheckSum = b[len(b)-int(checksumL):]
-	return nil
-}
-
-// NewInitiatorWrapToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum.
-// Other flags are set to 0, and the RRC and sequence number are initialized to 0.
-// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier.
-// This is currently not supported.
-func NewInitiatorWrapToken(payload []byte, key types.EncryptionKey) (*WrapToken, error) {
-	encType, err := crypto.GetEtype(key.KeyType)
-	if err != nil {
-		return nil, err
-	}
-
-	token := WrapToken{
-		Flags: 0x00, // all zeroed out (this is a token sent by the initiator)
-		// Checksum size: length of output of the HMAC function, in bytes.
-		EC:        uint16(encType.GetHMACBitLength() / 8),
-		RRC:       0,
-		SndSeqNum: 0,
-		Payload:   payload,
-	}
-
-	if err := token.SetCheckSum(key, keyusage.GSSAPI_INITIATOR_SEAL); err != nil {
-		return nil, err
-	}
-
-	return &token, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/changepasswddata.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/changepasswddata.go
deleted file mode 100644
index a3e2efd..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/changepasswddata.go
+++ /dev/null
@@ -1,23 +0,0 @@
-package kadmin
-
-import (
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// ChangePasswdData is the payload to a password change message.
-type ChangePasswdData struct {
-	NewPasswd []byte              `asn1:"explicit,tag:0"`
-	TargName  types.PrincipalName `asn1:"explicit,optional,tag:1"`
-	TargRealm string              `asn1:"generalstring,optional,explicit,tag:2"`
-}
-
-// Marshal ChangePasswdData into a byte slice.
-func (c *ChangePasswdData) Marshal() ([]byte, error) {
-	b, err := asn1.Marshal(*c)
-	if err != nil {
-		return []byte{}, err
-	}
-	//b = asn1tools.AddASNAppTag(b, asnAppTag.)
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/message.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/message.go
deleted file mode 100644
index 157fcad..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/message.go
+++ /dev/null
@@ -1,114 +0,0 @@
-package kadmin
-
-import (
-	"bytes"
-	"encoding/binary"
-	"errors"
-	"fmt"
-	"math"
-
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-const (
-	verisonHex = "ff80"
-)
-
-// Request message for changing password.
-type Request struct {
-	APREQ   messages.APReq
-	KRBPriv messages.KRBPriv
-}
-
-// Reply message for a password change.
-type Reply struct {
-	MessageLength int
-	Version       int
-	APREPLength   int
-	APREP         messages.APRep
-	KRBPriv       messages.KRBPriv
-	KRBError      messages.KRBError
-	IsKRBError    bool
-	ResultCode    uint16
-	Result        string
-}
-
-// Marshal a Request into a byte slice.
-func (m *Request) Marshal() (b []byte, err error) {
-	b = []byte{255, 128} // protocol version number: contains the hex constant 0xff80 (big-endian integer).
-	ab, e := m.APREQ.Marshal()
-	if e != nil {
-		err = fmt.Errorf("error marshaling AP_REQ: %v", e)
-		return
-	}
-	if len(ab) > math.MaxUint16 {
-		err = errors.New("length of AP_REQ greater then max Uint16 size")
-		return
-	}
-	al := make([]byte, 2)
-	binary.BigEndian.PutUint16(al, uint16(len(ab)))
-	b = append(b, al...)
-	b = append(b, ab...)
-	pb, e := m.KRBPriv.Marshal()
-	if e != nil {
-		err = fmt.Errorf("error marshaling KRB_Priv: %v", e)
-		return
-	}
-	b = append(b, pb...)
-	if len(b)+2 > math.MaxUint16 {
-		err = errors.New("length of message greater then max Uint16 size")
-		return
-	}
-	ml := make([]byte, 2)
-	binary.BigEndian.PutUint16(ml, uint16(len(b)+2))
-	b = append(ml, b...)
-	return
-}
-
-// Unmarshal a byte slice into a Reply.
-func (m *Reply) Unmarshal(b []byte) error {
-	m.MessageLength = int(binary.BigEndian.Uint16(b[0:2]))
-	m.Version = int(binary.BigEndian.Uint16(b[2:4]))
-	if m.Version != 1 {
-		return fmt.Errorf("kadmin reply has incorrect protocol version number: %d", m.Version)
-	}
-	m.APREPLength = int(binary.BigEndian.Uint16(b[4:6]))
-	if m.APREPLength != 0 {
-		err := m.APREP.Unmarshal(b[6 : 6+m.APREPLength])
-		if err != nil {
-			return err
-		}
-		err = m.KRBPriv.Unmarshal(b[6+m.APREPLength : m.MessageLength])
-		if err != nil {
-			return err
-		}
-	} else {
-		m.IsKRBError = true
-		m.KRBError.Unmarshal(b[6:m.MessageLength])
-		m.ResultCode, m.Result = parseResponse(m.KRBError.EData)
-	}
-	return nil
-}
-
-func parseResponse(b []byte) (c uint16, s string) {
-	c = binary.BigEndian.Uint16(b[0:2])
-	buf := bytes.NewBuffer(b[2:])
-	m := make([]byte, len(b)-2)
-	binary.Read(buf, binary.BigEndian, &m)
-	s = string(m)
-	return
-}
-
-// Decrypt the encrypted part of the KRBError within the change password Reply.
-func (m *Reply) Decrypt(key types.EncryptionKey) error {
-	if m.IsKRBError {
-		return m.KRBError
-	}
-	err := m.KRBPriv.DecryptEncPart(key)
-	if err != nil {
-		return err
-	}
-	m.ResultCode, m.Result = parseResponse(m.KRBPriv.DecryptedEncPart.UserData)
-	return nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/passwd.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/passwd.go
deleted file mode 100644
index 2a7491a..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/kadmin/passwd.go
+++ /dev/null
@@ -1,68 +0,0 @@
-// Package kadmin provides Kerberos administration capabilities.
-package kadmin
-
-import (
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/messages"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// ChangePasswdMsg generate a change password request and also return the key needed to decrypt the reply.
-func ChangePasswdMsg(cname types.PrincipalName, realm, password string, tkt messages.Ticket, sessionKey types.EncryptionKey) (r Request, k types.EncryptionKey, err error) {
-	// Create change password data struct and marshal to bytes
-	chgpasswd := ChangePasswdData{
-		NewPasswd: []byte(password),
-		TargName:  cname,
-		TargRealm: realm,
-	}
-	chpwdb, err := chgpasswd.Marshal()
-	if err != nil {
-		err = krberror.Errorf(err, krberror.KRBMsgError, "error marshaling change passwd data")
-		return
-	}
-
-	// Generate authenticator
-	auth, err := types.NewAuthenticator(realm, cname)
-	if err != nil {
-		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator")
-		return
-	}
-	etype, err := crypto.GetEtype(sessionKey.KeyType)
-	if err != nil {
-		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey etype")
-		return
-	}
-	err = auth.GenerateSeqNumberAndSubKey(etype.GetETypeID(), etype.GetKeyByteSize())
-	if err != nil {
-		err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey")
-		return
-	}
-	k = auth.SubKey
-
-	// Generate AP_REQ
-	APreq, err := messages.NewAPReq(tkt, sessionKey, auth)
-	if err != nil {
-		return
-	}
-
-	// Form the KRBPriv encpart data
-	kp := messages.EncKrbPrivPart{
-		UserData:       chpwdb,
-		Timestamp:      auth.CTime,
-		Usec:           auth.Cusec,
-		SequenceNumber: auth.SeqNumber,
-	}
-	kpriv := messages.NewKRBPriv(kp)
-	err = kpriv.EncryptEncPart(k)
-	if err != nil {
-		err = krberror.Errorf(err, krberror.EncryptingError, "error encrypting change passwd data")
-		return
-	}
-
-	r = Request{
-		APREQ:   APreq,
-		KRBPriv: kpriv,
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/keytab/keytab.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/keytab/keytab.go
deleted file mode 100644
index 0c7fc38..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/keytab/keytab.go
+++ /dev/null
@@ -1,369 +0,0 @@
-// Package keytab implements Kerberos keytabs: https://web.mit.edu/kerberos/krb5-devel/doc/formats/keytab_file_format.html.
-package keytab
-
-import (
-	"bytes"
-	"encoding/binary"
-	"errors"
-	"fmt"
-	"io"
-	"io/ioutil"
-	"time"
-	"unsafe"
-
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-const (
-	keytabFirstByte byte = 05
-)
-
-// Keytab struct.
-type Keytab struct {
-	version uint8
-	Entries []entry
-}
-
-// Keytab entry struct.
-type entry struct {
-	Principal principal
-	Timestamp time.Time
-	KVNO8     uint8
-	Key       types.EncryptionKey
-	KVNO      uint32
-}
-
-// Keytab entry principal struct.
-type principal struct {
-	NumComponents int16
-	Realm         string
-	Components    []string
-	NameType      int32
-}
-
-// New creates new, empty Keytab type.
-func New() *Keytab {
-	var e []entry
-	return &Keytab{
-		version: 0,
-		Entries: e,
-	}
-}
-
-// GetEncryptionKey returns the EncryptionKey from the Keytab for the newest entry with the required kvno, etype and matching principal.
-func (kt *Keytab) GetEncryptionKey(princName types.PrincipalName, realm string, kvno int, etype int32) (types.EncryptionKey, error) {
-	//TODO (theme: KVNO from keytab) this function should return the kvno too
-	var key types.EncryptionKey
-	var t time.Time
-	for _, k := range kt.Entries {
-		if k.Principal.Realm == realm && len(k.Principal.Components) == len(princName.NameString) &&
-			k.Key.KeyType == etype &&
-			(k.KVNO == uint32(kvno) || kvno == 0) &&
-			k.Timestamp.After(t) {
-			p := true
-			for i, n := range k.Principal.Components {
-				if princName.NameString[i] != n {
-					p = false
-					break
-				}
-			}
-			if p {
-				key = k.Key
-				t = k.Timestamp
-			}
-		}
-	}
-	if len(key.KeyValue) < 1 {
-		return key, fmt.Errorf("matching key not found in keytab. Looking for %v realm: %v kvno: %v etype: %v", princName.NameString, realm, kvno, etype)
-	}
-	return key, nil
-}
-
-// Create a new Keytab entry.
-func newKeytabEntry() entry {
-	var b []byte
-	return entry{
-		Principal: newPrincipal(),
-		Timestamp: time.Time{},
-		KVNO8:     0,
-		Key: types.EncryptionKey{
-			KeyType:  0,
-			KeyValue: b,
-		},
-		KVNO: 0,
-	}
-}
-
-// Create a new principal.
-func newPrincipal() principal {
-	var c []string
-	return principal{
-		NumComponents: 0,
-		Realm:         "",
-		Components:    c,
-		NameType:      0,
-	}
-}
-
-// Load a Keytab file into a Keytab type.
-func Load(ktPath string) (*Keytab, error) {
-	kt := new(Keytab)
-	b, err := ioutil.ReadFile(ktPath)
-	if err != nil {
-		return kt, err
-	}
-	err = kt.Unmarshal(b)
-	return kt, err
-}
-
-// Marshal keytab into byte slice
-func (kt *Keytab) Marshal() ([]byte, error) {
-	b := []byte{keytabFirstByte, kt.version}
-	for _, e := range kt.Entries {
-		eb, err := e.marshal(int(kt.version))
-		if err != nil {
-			return b, err
-		}
-		b = append(b, eb...)
-	}
-	return b, nil
-}
-
-// Write the keytab bytes to io.Writer.
-// Returns the number of bytes written
-func (kt *Keytab) Write(w io.Writer) (int, error) {
-	b, err := kt.Marshal()
-	if err != nil {
-		return 0, fmt.Errorf("error marshaling keytab: %v", err)
-	}
-	return w.Write(b)
-}
-
-// Unmarshal byte slice of Keytab data into Keytab type.
-func (kt *Keytab) Unmarshal(b []byte) error {
-	//The first byte of the file always has the value 5
-	if b[0] != keytabFirstByte {
-		return errors.New("invalid keytab data. First byte does not equal 5")
-	}
-	//Get keytab version
-	//The 2nd byte contains the version number (1 or 2)
-	kt.version = b[1]
-	if kt.version != 1 && kt.version != 2 {
-		return errors.New("invalid keytab data. Keytab version is neither 1 nor 2")
-	}
-	//Version 1 of the file format uses native byte order for integer representations. Version 2 always uses big-endian byte order
-	var endian binary.ByteOrder
-	endian = binary.BigEndian
-	if kt.version == 1 && isNativeEndianLittle() {
-		endian = binary.LittleEndian
-	}
-	/*
-		After the two-byte version indicator, the file contains a sequence of signed 32-bit record lengths followed by key records or holes.
-		A positive record length indicates a valid key entry whose size is equal to or less than the record length.
-		A negative length indicates a zero-filled hole whose size is the inverse of the length.
-		A length of 0 indicates the end of the file.
-	*/
-	// n tracks position in the byte array
-	n := 2
-	l := readInt32(b, &n, &endian)
-	for l != 0 {
-		if l < 0 {
-			//Zero padded so skip over
-			l = l * -1
-			n = n + int(l)
-		} else {
-			//fmt.Printf("Bytes for entry: %v\n", b[n:n+int(l)])
-			eb := b[n : n+int(l)]
-			n = n + int(l)
-			ke := newKeytabEntry()
-			// p keeps track as to where we are in the byte stream
-			var p int
-			parsePrincipal(eb, &p, kt, &ke, &endian)
-			ke.Timestamp = readTimestamp(eb, &p, &endian)
-			ke.KVNO8 = uint8(readInt8(eb, &p, &endian))
-			ke.Key.KeyType = int32(readInt16(eb, &p, &endian))
-			kl := int(readInt16(eb, &p, &endian))
-			ke.Key.KeyValue = readBytes(eb, &p, kl, &endian)
-			//The 32-bit key version overrides the 8-bit key version.
-			// To determine if it is present, the implementation must check that at least 4 bytes remain in the record after the other fields are read,
-			// and that the value of the 32-bit integer contained in those bytes is non-zero.
-			if len(eb)-p >= 4 {
-				// The 32-bit key may be present
-				ke.KVNO = uint32(readInt32(eb, &p, &endian))
-			}
-			if ke.KVNO == 0 {
-				// Handles if the value from the last 4 bytes was zero and also if there are not the 4 bytes present. Makes sense to put the same value here as KVNO8
-				ke.KVNO = uint32(ke.KVNO8)
-			}
-			// Add the entry to the keytab
-			kt.Entries = append(kt.Entries, ke)
-		}
-		// Check if there are still 4 bytes left to read
-		if n > len(b) || len(b[n:]) < 4 {
-			break
-		}
-		// Read the size of the next entry
-		l = readInt32(b, &n, &endian)
-	}
-	return nil
-}
-
-func (e entry) marshal(v int) ([]byte, error) {
-	var b []byte
-	pb, err := e.Principal.marshal(v)
-	if err != nil {
-		return b, err
-	}
-	b = append(b, pb...)
-
-	var endian binary.ByteOrder
-	endian = binary.BigEndian
-	if v == 1 && isNativeEndianLittle() {
-		endian = binary.LittleEndian
-	}
-
-	t := make([]byte, 9)
-	endian.PutUint32(t[0:4], uint32(e.Timestamp.Unix()))
-	t[4] = e.KVNO8
-	endian.PutUint16(t[5:7], uint16(e.Key.KeyType))
-	endian.PutUint16(t[7:9], uint16(len(e.Key.KeyValue)))
-	b = append(b, t...)
-
-	buf := new(bytes.Buffer)
-	err = binary.Write(buf, endian, e.Key.KeyValue)
-	if err != nil {
-		return b, err
-	}
-	b = append(b, buf.Bytes()...)
-
-	t = make([]byte, 4)
-	endian.PutUint32(t, e.KVNO)
-	b = append(b, t...)
-
-	// Add the length header
-	t = make([]byte, 4)
-	endian.PutUint32(t, uint32(len(b)))
-	b = append(t, b...)
-	return b, nil
-}
-
-// Parse the Keytab bytes of a principal into a Keytab entry's principal.
-func parsePrincipal(b []byte, p *int, kt *Keytab, ke *entry, e *binary.ByteOrder) error {
-	ke.Principal.NumComponents = readInt16(b, p, e)
-	if kt.version == 1 {
-		//In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2
-		ke.Principal.NumComponents--
-	}
-	lenRealm := readInt16(b, p, e)
-	ke.Principal.Realm = string(readBytes(b, p, int(lenRealm), e))
-	for i := 0; i < int(ke.Principal.NumComponents); i++ {
-		l := readInt16(b, p, e)
-		ke.Principal.Components = append(ke.Principal.Components, string(readBytes(b, p, int(l), e)))
-	}
-	if kt.version != 1 {
-		//Name Type is omitted in version 1
-		ke.Principal.NameType = readInt32(b, p, e)
-	}
-	return nil
-}
-
-func (p principal) marshal(v int) ([]byte, error) {
-	//var b []byte
-	b := make([]byte, 2)
-	var endian binary.ByteOrder
-	endian = binary.BigEndian
-	if v == 1 && isNativeEndianLittle() {
-		endian = binary.LittleEndian
-	}
-	endian.PutUint16(b[0:], uint16(p.NumComponents))
-	realm, err := marshalString(p.Realm, v)
-	if err != nil {
-		return b, err
-	}
-	b = append(b, realm...)
-	for _, c := range p.Components {
-		cb, err := marshalString(c, v)
-		if err != nil {
-			return b, err
-		}
-		b = append(b, cb...)
-	}
-	if v != 1 {
-		t := make([]byte, 4)
-		endian.PutUint32(t, uint32(p.NameType))
-		b = append(b, t...)
-	}
-	return b, nil
-}
-
-func marshalString(s string, v int) ([]byte, error) {
-	sb := []byte(s)
-	b := make([]byte, 2)
-	var endian binary.ByteOrder
-	endian = binary.BigEndian
-	if v == 1 && isNativeEndianLittle() {
-		endian = binary.LittleEndian
-	}
-	endian.PutUint16(b[0:], uint16(len(sb)))
-	buf := new(bytes.Buffer)
-	err := binary.Write(buf, endian, sb)
-	if err != nil {
-		return b, err
-	}
-	b = append(b, buf.Bytes()...)
-	return b, err
-}
-
-// Read bytes representing a timestamp.
-func readTimestamp(b []byte, p *int, e *binary.ByteOrder) time.Time {
-	return time.Unix(int64(readInt32(b, p, e)), 0)
-}
-
-// Read bytes representing an eight bit integer.
-func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8) {
-	buf := bytes.NewBuffer(b[*p : *p+1])
-	binary.Read(buf, *e, &i)
-	*p++
-	return
-}
-
-// Read bytes representing a sixteen bit integer.
-func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16) {
-	buf := bytes.NewBuffer(b[*p : *p+2])
-	binary.Read(buf, *e, &i)
-	*p += 2
-	return
-}
-
-// Read bytes representing a thirty two bit integer.
-func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32) {
-	buf := bytes.NewBuffer(b[*p : *p+4])
-	binary.Read(buf, *e, &i)
-	*p += 4
-	return
-}
-
-func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) []byte {
-	buf := bytes.NewBuffer(b[*p : *p+s])
-	r := make([]byte, s)
-	binary.Read(buf, *e, &r)
-	*p += s
-	return r
-}
-
-func isNativeEndianLittle() bool {
-	var x = 0x012345678
-	var p = unsafe.Pointer(&x)
-	var bp = (*[4]byte)(p)
-
-	var endian bool
-	if 0x01 == bp[0] {
-		endian = false
-	} else if (0x78 & 0xff) == (bp[0] & 0xff) {
-		endian = true
-	} else {
-		// Default to big endian
-		endian = false
-	}
-	return endian
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/krberror/error.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/krberror/error.go
deleted file mode 100644
index d591bde..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/krberror/error.go
+++ /dev/null
@@ -1,67 +0,0 @@
-// Package krberror provides error type and functions for gokrb5.
-package krberror
-
-import (
-	"fmt"
-	"strings"
-)
-
-// Error type descriptions.
-const (
-	separator       = " < "
-	EncodingError   = "Encoding_Error"
-	NetworkingError = "Networking_Error"
-	DecryptingError = "Decrypting_Error"
-	EncryptingError = "Encrypting_Error"
-	ChksumError     = "Checksum_Error"
-	KRBMsgError     = "KRBMessage_Handling_Error"
-	ConfigError     = "Configuration_Error"
-	KDCError        = "KDC_Error"
-)
-
-// Krberror is an error type for gokrb5
-type Krberror struct {
-	RootCause string
-	EText     []string
-}
-
-// Error function to implement the error interface.
-func (e Krberror) Error() string {
-	return fmt.Sprintf("[Root cause: %s] ", e.RootCause) + strings.Join(e.EText, separator)
-}
-
-// Add another error statement to the error.
-func (e *Krberror) Add(et string, s string) {
-	e.EText = append([]string{fmt.Sprintf("%s: %s", et, s)}, e.EText...)
-}
-
-// NewKrberror creates a new instance of Krberror.
-func NewKrberror(et, s string) Krberror {
-	return Krberror{
-		RootCause: et,
-		EText:     []string{s},
-	}
-}
-
-// Errorf appends to or creates a new Krberror.
-func Errorf(err error, et, format string, a ...interface{}) Krberror {
-	if e, ok := err.(Krberror); ok {
-		e.Add(et, fmt.Sprintf(format, a...))
-		return e
-	}
-	return NewErrorf(et, format+": %s", append(a, err)...)
-}
-
-// NewErrorf creates a new Krberror from a formatted string.
-func NewErrorf(et, format string, a ...interface{}) Krberror {
-	var s string
-	if len(a) > 0 {
-		s = fmt.Sprintf("%s: %s", et, fmt.Sprintf(format, a...))
-	} else {
-		s = fmt.Sprintf("%s: %s", et, format)
-	}
-	return Krberror{
-		RootCause: et,
-		EText:     []string{s},
-	}
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APRep.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APRep.go
deleted file mode 100644
index 9c244f0..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APRep.go
+++ /dev/null
@@ -1,64 +0,0 @@
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-/*
-AP-REP          ::= [APPLICATION 15] SEQUENCE {
-pvno            [0] INTEGER (5),
-msg-type        [1] INTEGER (15),
-enc-part        [2] EncryptedData -- EncAPRepPart
-}
-
-EncAPRepPart    ::= [APPLICATION 27] SEQUENCE {
-        ctime           [0] KerberosTime,
-        cusec           [1] Microseconds,
-        subkey          [2] EncryptionKey OPTIONAL,
-        seq-number      [3] UInt32 OPTIONAL
-}
-*/
-
-// APRep implements RFC 4120 KRB_AP_REP: https://tools.ietf.org/html/rfc4120#section-5.5.2.
-type APRep struct {
-	PVNO    int                 `asn1:"explicit,tag:0"`
-	MsgType int                 `asn1:"explicit,tag:1"`
-	EncPart types.EncryptedData `asn1:"explicit,tag:2"`
-}
-
-// EncAPRepPart is the encrypted part of KRB_AP_REP.
-type EncAPRepPart struct {
-	CTime          time.Time           `asn1:"generalized,explicit,tag:0"`
-	Cusec          int                 `asn1:"explicit,tag:1"`
-	Subkey         types.EncryptionKey `asn1:"optional,explicit,tag:2"`
-	SequenceNumber int64               `asn1:"optional,explicit,tag:3"`
-}
-
-// Unmarshal bytes b into the APRep struct.
-func (a *APRep) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREP))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	expectedMsgType := msgtype.KRB_AP_REP
-	if a.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_AP_REP. Expected: %v; Actual: %v", expectedMsgType, a.MsgType)
-	}
-	return nil
-}
-
-// Unmarshal bytes b into the APRep encrypted part struct.
-func (a *EncAPRepPart) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncAPRepPart))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "AP_REP unmarshal error")
-	}
-	return nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APReq.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APReq.go
deleted file mode 100644
index e1ed4ae..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/APReq.go
+++ /dev/null
@@ -1,220 +0,0 @@
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/keytab"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-/*AP-REQ          ::= [APPLICATION 14] SEQUENCE {
-pvno            [0] INTEGER (5),
-msg-type        [1] INTEGER (14),
-ap-options      [2] APOptions,
-ticket          [3] Ticket,
-authenticator   [4] EncryptedData -- Authenticator
-}
-
-APOptions       ::= KerberosFlags
--- reserved(0),
--- use-session-key(1),
--- mutual-required(2)*/
-
-type marshalAPReq struct {
-	PVNO      int            `asn1:"explicit,tag:0"`
-	MsgType   int            `asn1:"explicit,tag:1"`
-	APOptions asn1.BitString `asn1:"explicit,tag:2"`
-	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
-	Ticket                 asn1.RawValue       `asn1:"explicit,tag:3"`
-	EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"`
-}
-
-// APReq implements RFC 4120 KRB_AP_REQ: https://tools.ietf.org/html/rfc4120#section-5.5.1.
-type APReq struct {
-	PVNO                   int                 `asn1:"explicit,tag:0"`
-	MsgType                int                 `asn1:"explicit,tag:1"`
-	APOptions              asn1.BitString      `asn1:"explicit,tag:2"`
-	Ticket                 Ticket              `asn1:"explicit,tag:3"`
-	EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"`
-	Authenticator          types.Authenticator `asn1:"optional"`
-}
-
-// NewAPReq generates a new KRB_AP_REQ struct.
-func NewAPReq(tkt Ticket, sessionKey types.EncryptionKey, auth types.Authenticator) (APReq, error) {
-	var a APReq
-	ed, err := encryptAuthenticator(auth, sessionKey, tkt)
-	if err != nil {
-		return a, krberror.Errorf(err, krberror.KRBMsgError, "error creating Authenticator for AP_REQ")
-	}
-	a = APReq{
-		PVNO:                   iana.PVNO,
-		MsgType:                msgtype.KRB_AP_REQ,
-		APOptions:              types.NewKrbFlags(),
-		Ticket:                 tkt,
-		EncryptedAuthenticator: ed,
-	}
-	return a, nil
-}
-
-// Encrypt Authenticator
-func encryptAuthenticator(a types.Authenticator, sessionKey types.EncryptionKey, tkt Ticket) (types.EncryptedData, error) {
-	var ed types.EncryptedData
-	m, err := a.Marshal()
-	if err != nil {
-		return ed, krberror.Errorf(err, krberror.EncodingError, "marshaling error of EncryptedData form of Authenticator")
-	}
-	usage := authenticatorKeyUsage(tkt.SName)
-	ed, err = crypto.GetEncryptedData(m, sessionKey, uint32(usage), tkt.EncPart.KVNO)
-	if err != nil {
-		return ed, krberror.Errorf(err, krberror.EncryptingError, "error encrypting Authenticator")
-	}
-	return ed, nil
-}
-
-// DecryptAuthenticator decrypts the Authenticator within the AP_REQ.
-// sessionKey may simply be the key within the decrypted EncPart of the ticket within the AP_REQ.
-func (a *APReq) DecryptAuthenticator(sessionKey types.EncryptionKey) error {
-	usage := authenticatorKeyUsage(a.Ticket.SName)
-	ab, e := crypto.DecryptEncPart(a.EncryptedAuthenticator, sessionKey, uint32(usage))
-	if e != nil {
-		return fmt.Errorf("error decrypting authenticator: %v", e)
-	}
-	err := a.Authenticator.Unmarshal(ab)
-	if err != nil {
-		return fmt.Errorf("error unmarshaling authenticator: %v", err)
-	}
-	return nil
-}
-
-func authenticatorKeyUsage(pn types.PrincipalName) int {
-	if pn.NameString[0] == "krbtgt" {
-		return keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR
-	}
-	return keyusage.AP_REQ_AUTHENTICATOR
-}
-
-// Unmarshal bytes b into the APReq struct.
-func (a *APReq) Unmarshal(b []byte) error {
-	var m marshalAPReq
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREQ))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "unmarshal error of AP_REQ")
-	}
-	if m.MsgType != msgtype.KRB_AP_REQ {
-		return NewKRBError(types.PrincipalName{}, "", errorcode.KRB_AP_ERR_MSG_TYPE, errorcode.Lookup(errorcode.KRB_AP_ERR_MSG_TYPE))
-	}
-	a.PVNO = m.PVNO
-	a.MsgType = m.MsgType
-	a.APOptions = m.APOptions
-	a.EncryptedAuthenticator = m.EncryptedAuthenticator
-	a.Ticket, err = unmarshalTicket(m.Ticket.Bytes)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "unmarshaling error of Ticket within AP_REQ")
-	}
-	return nil
-}
-
-// Marshal APReq struct.
-func (a *APReq) Marshal() ([]byte, error) {
-	m := marshalAPReq{
-		PVNO:                   a.PVNO,
-		MsgType:                a.MsgType,
-		APOptions:              a.APOptions,
-		EncryptedAuthenticator: a.EncryptedAuthenticator,
-	}
-	var b []byte
-	b, err := a.Ticket.Marshal()
-	if err != nil {
-		return b, err
-	}
-	m.Ticket = asn1.RawValue{
-		Class:      asn1.ClassContextSpecific,
-		IsCompound: true,
-		Tag:        3,
-		Bytes:      b,
-	}
-	mk, err := asn1.Marshal(m)
-	if err != nil {
-		return mk, krberror.Errorf(err, krberror.EncodingError, "marshaling error of AP_REQ")
-	}
-	mk = asn1tools.AddASNAppTag(mk, asnAppTag.APREQ)
-	return mk, nil
-}
-
-// Verify an AP_REQ using service's keytab, spn and max acceptable clock skew duration.
-// The service ticket encrypted part and authenticator will be decrypted as part of this operation.
-func (a *APReq) Verify(kt *keytab.Keytab, d time.Duration, cAddr types.HostAddress) (bool, error) {
-	// Decrypt ticket's encrypted part with service key
-	//TODO decrypt with service's session key from its TGT is use-to-user. Need to figure out how to get TGT.
-	//if types.IsFlagSet(&a.APOptions, flags.APOptionUseSessionKey) {
-	//	//If the USE-SESSION-KEY flag is set in the ap-options field, it indicates to
-	//	//the server that user-to-user authentication is in use, and that the ticket
-	//	//is encrypted in the session key from the server's TGT rather than in the server's secret key.
-	//	err := a.Ticket.Decrypt(tgt.DecryptedEncPart.Key)
-	//	if err != nil {
-	//		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of ticket provided using session key")
-	//	}
-	//} else {
-	//	// Because it is possible for the server to be registered in multiple
-	//	// realms, with different keys in each, the srealm field in the
-	//	// unencrypted portion of the ticket in the KRB_AP_REQ is used to
-	//	// specify which secret key the server should use to decrypt that
-	//	// ticket.The KRB_AP_ERR_NOKEY error code is returned if the server
-	//	// doesn't have the proper key to decipher the ticket.
-	//	// The ticket is decrypted using the version of the server's key
-	//	// specified by the ticket.
-	//	err := a.Ticket.DecryptEncPart(*kt, &a.Ticket.SName)
-	//	if err != nil {
-	//		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided")
-	//	}
-	//}
-	err := a.Ticket.DecryptEncPart(kt, &a.Ticket.SName)
-	if err != nil {
-		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided")
-	}
-
-	// Check time validity of ticket
-	ok, err := a.Ticket.Valid(d)
-	if err != nil || !ok {
-		return ok, err
-	}
-
-	// Check client's address is listed in the client addresses in the ticket
-	if len(a.Ticket.DecryptedEncPart.CAddr) > 0 {
-		//The addresses in the ticket (if any) are then searched for an address matching the operating-system reported
-		//address of the client.  If no match is found or the server insists on ticket addresses but none are present in
-		//the ticket, the KRB_AP_ERR_BADADDR error is returned.
-		if !types.HostAddressesContains(a.Ticket.DecryptedEncPart.CAddr, cAddr) {
-			return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADADDR, "client address not within the list contained in the service ticket")
-		}
-	}
-
-	// Decrypt authenticator with session key from ticket's encrypted part
-	err = a.DecryptAuthenticator(a.Ticket.DecryptedEncPart.Key)
-	if err != nil {
-		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BAD_INTEGRITY, "could not decrypt authenticator")
-	}
-
-	// Check CName in authenticator is the same as that in the ticket
-	if !a.Authenticator.CName.Equal(a.Ticket.DecryptedEncPart.CName) {
-		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADMATCH, "CName in Authenticator does not match that in service ticket")
-	}
-
-	// Check the clock skew between the client and the service server
-	ct := a.Authenticator.CTime.Add(time.Duration(a.Authenticator.Cusec) * time.Microsecond)
-	t := time.Now().UTC()
-	if t.Sub(ct) > d || ct.Sub(t) > d {
-		return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_SKEW, fmt.Sprintf("clock skew with client too large. greater than %v seconds", d))
-	}
-	return true, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCRep.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCRep.go
deleted file mode 100644
index 76c89c3..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCRep.go
+++ /dev/null
@@ -1,312 +0,0 @@
-package messages
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.4.2
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/config"
-	"gopkg.in/jcmturner/gokrb5.v7/credentials"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/flags"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/patype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-type marshalKDCRep struct {
-	PVNO    int                  `asn1:"explicit,tag:0"`
-	MsgType int                  `asn1:"explicit,tag:1"`
-	PAData  types.PADataSequence `asn1:"explicit,optional,tag:2"`
-	CRealm  string               `asn1:"generalstring,explicit,tag:3"`
-	CName   types.PrincipalName  `asn1:"explicit,tag:4"`
-	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
-	Ticket  asn1.RawValue       `asn1:"explicit,tag:5"`
-	EncPart types.EncryptedData `asn1:"explicit,tag:6"`
-}
-
-// KDCRepFields represents the KRB_KDC_REP fields.
-type KDCRepFields struct {
-	PVNO             int
-	MsgType          int
-	PAData           []types.PAData
-	CRealm           string
-	CName            types.PrincipalName
-	Ticket           Ticket
-	EncPart          types.EncryptedData
-	DecryptedEncPart EncKDCRepPart
-}
-
-// ASRep implements RFC 4120 KRB_AS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2.
-type ASRep struct {
-	KDCRepFields
-}
-
-// TGSRep implements RFC 4120 KRB_TGS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2.
-type TGSRep struct {
-	KDCRepFields
-}
-
-// EncKDCRepPart is the encrypted part of KRB_KDC_REP.
-type EncKDCRepPart struct {
-	Key           types.EncryptionKey  `asn1:"explicit,tag:0"`
-	LastReqs      []LastReq            `asn1:"explicit,tag:1"`
-	Nonce         int                  `asn1:"explicit,tag:2"`
-	KeyExpiration time.Time            `asn1:"generalized,explicit,optional,tag:3"`
-	Flags         asn1.BitString       `asn1:"explicit,tag:4"`
-	AuthTime      time.Time            `asn1:"generalized,explicit,tag:5"`
-	StartTime     time.Time            `asn1:"generalized,explicit,optional,tag:6"`
-	EndTime       time.Time            `asn1:"generalized,explicit,tag:7"`
-	RenewTill     time.Time            `asn1:"generalized,explicit,optional,tag:8"`
-	SRealm        string               `asn1:"generalstring,explicit,tag:9"`
-	SName         types.PrincipalName  `asn1:"explicit,tag:10"`
-	CAddr         []types.HostAddress  `asn1:"explicit,optional,tag:11"`
-	EncPAData     types.PADataSequence `asn1:"explicit,optional,tag:12"`
-}
-
-// LastReq part of KRB_KDC_REP.
-type LastReq struct {
-	LRType  int32     `asn1:"explicit,tag:0"`
-	LRValue time.Time `asn1:"generalized,explicit,tag:1"`
-}
-
-// Unmarshal bytes b into the ASRep struct.
-func (k *ASRep) Unmarshal(b []byte) error {
-	var m marshalKDCRep
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREP))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	if m.MsgType != msgtype.KRB_AS_REP {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an AS_REP. Expected: %v; Actual: %v", msgtype.KRB_AS_REP, m.MsgType)
-	}
-	//Process the raw ticket within
-	tkt, err := unmarshalTicket(m.Ticket.Bytes)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within AS_REP")
-	}
-	k.KDCRepFields = KDCRepFields{
-		PVNO:    m.PVNO,
-		MsgType: m.MsgType,
-		PAData:  m.PAData,
-		CRealm:  m.CRealm,
-		CName:   m.CName,
-		Ticket:  tkt,
-		EncPart: m.EncPart,
-	}
-	return nil
-}
-
-// Unmarshal bytes b into the TGSRep struct.
-func (k *TGSRep) Unmarshal(b []byte) error {
-	var m marshalKDCRep
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREP))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	if m.MsgType != msgtype.KRB_TGS_REP {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an TGS_REP. Expected: %v; Actual: %v", msgtype.KRB_TGS_REP, m.MsgType)
-	}
-	//Process the raw ticket within
-	tkt, err := unmarshalTicket(m.Ticket.Bytes)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within TGS_REP")
-	}
-	k.KDCRepFields = KDCRepFields{
-		PVNO:    m.PVNO,
-		MsgType: m.MsgType,
-		PAData:  m.PAData,
-		CRealm:  m.CRealm,
-		CName:   m.CName,
-		Ticket:  tkt,
-		EncPart: m.EncPart,
-	}
-	return nil
-}
-
-// Unmarshal bytes b into encrypted part of KRB_KDC_REP.
-func (e *EncKDCRepPart) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncASRepPart))
-	if err != nil {
-		// Try using tag 26
-		/* Ref: RFC 4120
-		Compatibility note: Some implementations unconditionally send an
-		encrypted EncTGSRepPart (application tag number 26) in this field
-		regardless of whether the reply is a AS-REP or a TGS-REP.  In the
-		interest of compatibility, implementors MAY relax the check on the
-		tag number of the decrypted ENC-PART.*/
-		_, err = asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncTGSRepPart))
-		if err != nil {
-			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part within KDC_REP")
-		}
-	}
-	return nil
-}
-
-// DecryptEncPart decrypts the encrypted part of an AS_REP.
-func (k *ASRep) DecryptEncPart(c *credentials.Credentials) (types.EncryptionKey, error) {
-	var key types.EncryptionKey
-	var err error
-	if c.HasKeytab() {
-		key, err = c.Keytab().GetEncryptionKey(k.CName, k.CRealm, k.EncPart.KVNO, k.EncPart.EType)
-		if err != nil {
-			return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
-		}
-	}
-	if c.HasPassword() {
-		key, _, err = crypto.GetKeyFromPassword(c.Password(), k.CName, k.CRealm, k.EncPart.EType, k.PAData)
-		if err != nil {
-			return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
-		}
-	}
-	if !c.HasKeytab() && !c.HasPassword() {
-		return key, krberror.NewErrorf(krberror.DecryptingError, "no secret available in credentials to perform decryption of AS_REP encrypted part")
-	}
-	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.AS_REP_ENCPART)
-	if err != nil {
-		return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part")
-	}
-	var denc EncKDCRepPart
-	err = denc.Unmarshal(b)
-	if err != nil {
-		return key, krberror.Errorf(err, krberror.EncodingError, "error unmarshaling decrypted encpart of AS_REP")
-	}
-	k.DecryptedEncPart = denc
-	return key, nil
-}
-
-// Verify checks the validity of AS_REP message.
-func (k *ASRep) Verify(cfg *config.Config, creds *credentials.Credentials, asReq ASReq) (bool, error) {
-	//Ref RFC 4120 Section 3.1.5
-	if k.CName.NameType != asReq.ReqBody.CName.NameType || k.CName.NameString == nil {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", asReq.ReqBody.CName, k.CName)
-	}
-	for i := range k.CName.NameString {
-		if k.CName.NameString[i] != asReq.ReqBody.CName.NameString[i] {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", asReq.ReqBody.CName, k.CName)
-		}
-	}
-	if k.CRealm != asReq.ReqBody.Realm {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "CRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.CRealm)
-	}
-	key, err := k.DecryptEncPart(creds)
-	if err != nil {
-		return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting EncPart of AS_REP")
-	}
-	if k.DecryptedEncPart.Nonce != asReq.ReqBody.Nonce {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request")
-	}
-	if k.DecryptedEncPart.SName.NameType != asReq.ReqBody.SName.NameType || k.DecryptedEncPart.SName.NameString == nil {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", asReq.ReqBody.SName, k.DecryptedEncPart.SName)
-	}
-	for i := range k.CName.NameString {
-		if k.DecryptedEncPart.SName.NameString[i] != asReq.ReqBody.SName.NameString[i] {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %+v; Reply: %+v", asReq.ReqBody.SName, k.DecryptedEncPart.SName)
-		}
-	}
-	if k.DecryptedEncPart.SRealm != asReq.ReqBody.Realm {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.DecryptedEncPart.SRealm)
-	}
-	if len(asReq.ReqBody.Addresses) > 0 {
-		if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, asReq.ReqBody.Addresses) {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the AS_REP does not match those listed in the AS_REQ")
-		}
-	}
-	t := time.Now().UTC()
-	if t.Sub(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(t) > cfg.LibDefaults.Clockskew {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds", cfg.LibDefaults.Clockskew.Seconds())
-	}
-	// RFC 6806 https://tools.ietf.org/html/rfc6806.html#section-11
-	if asReq.PAData.Contains(patype.PA_REQ_ENC_PA_REP) && types.IsFlagSet(&k.DecryptedEncPart.Flags, flags.EncPARep) {
-		if len(k.DecryptedEncPart.EncPAData) < 2 || !k.DecryptedEncPart.EncPAData.Contains(patype.PA_FX_FAST) {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "KDC did not respond appropriately to FAST negotiation")
-		}
-		for _, pa := range k.DecryptedEncPart.EncPAData {
-			if pa.PADataType == patype.PA_REQ_ENC_PA_REP {
-				var pafast types.PAReqEncPARep
-				err := pafast.Unmarshal(pa.PADataValue)
-				if err != nil {
-					return false, krberror.Errorf(err, krberror.EncodingError, "KDC FAST negotiation response error, could not unmarshal PA_REQ_ENC_PA_REP")
-				}
-				etype, err := crypto.GetChksumEtype(pafast.ChksumType)
-				if err != nil {
-					return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response error")
-				}
-				ab, _ := asReq.Marshal()
-				if !etype.VerifyChecksum(key.KeyValue, ab, pafast.Chksum, keyusage.KEY_USAGE_AS_REQ) {
-					return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response checksum invalid")
-				}
-			}
-		}
-	}
-	return true, nil
-}
-
-// DecryptEncPart decrypts the encrypted part of an TGS_REP.
-func (k *TGSRep) DecryptEncPart(key types.EncryptionKey) error {
-	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.TGS_REP_ENCPART_SESSION_KEY)
-	if err != nil {
-		return krberror.Errorf(err, krberror.DecryptingError, "error decrypting TGS_REP EncPart")
-	}
-	var denc EncKDCRepPart
-	err = denc.Unmarshal(b)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part")
-	}
-	k.DecryptedEncPart = denc
-	return nil
-}
-
-// Verify checks the validity of the TGS_REP message.
-func (k *TGSRep) Verify(cfg *config.Config, tgsReq TGSReq) (bool, error) {
-	if k.CName.NameType != tgsReq.ReqBody.CName.NameType || k.CName.NameString == nil {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "CName type in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.CName, k.CName)
-	}
-	for i := range k.CName.NameString {
-		if k.CName.NameString[i] != tgsReq.ReqBody.CName.NameString[i] {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.CName, k.CName)
-		}
-	}
-	if k.Ticket.Realm != tgsReq.ReqBody.Realm {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "realm in response ticket does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.Ticket.Realm)
-	}
-	if k.DecryptedEncPart.Nonce != tgsReq.ReqBody.Nonce {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request")
-	}
-	//if k.Ticket.SName.NameType != tgsReq.ReqBody.SName.NameType || k.Ticket.SName.NameString == nil {
-	//	return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.Ticket.SName)
-	//}
-	//for i := range k.Ticket.SName.NameString {
-	//	if k.Ticket.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] {
-	//		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.Ticket.SName)
-	//	}
-	//}
-	//if k.DecryptedEncPart.SName.NameType != tgsReq.ReqBody.SName.NameType || k.DecryptedEncPart.SName.NameString == nil {
-	//	return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName)
-	//}
-	//for i := range k.DecryptedEncPart.SName.NameString {
-	//	if k.DecryptedEncPart.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] {
-	//		return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName)
-	//	}
-	//}
-	if k.DecryptedEncPart.SRealm != tgsReq.ReqBody.Realm {
-		return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.DecryptedEncPart.SRealm)
-	}
-	if len(k.DecryptedEncPart.CAddr) > 0 {
-		if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, tgsReq.ReqBody.Addresses) {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the TGS_REP does not match those listed in the TGS_REQ")
-		}
-	}
-	if time.Since(k.DecryptedEncPart.StartTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.StartTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew {
-		if time.Since(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew {
-			return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds.", cfg.LibDefaults.Clockskew.Seconds())
-		}
-	}
-	return true, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCReq.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCReq.go
deleted file mode 100644
index f75ddc4..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KDCReq.go
+++ /dev/null
@@ -1,432 +0,0 @@
-package messages
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.4.1
-
-import (
-	"crypto/rand"
-	"fmt"
-	"math"
-	"math/big"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/config"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/flags"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/patype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-type marshalKDCReq struct {
-	PVNO    int                  `asn1:"explicit,tag:1"`
-	MsgType int                  `asn1:"explicit,tag:2"`
-	PAData  types.PADataSequence `asn1:"explicit,optional,tag:3"`
-	ReqBody asn1.RawValue        `asn1:"explicit,tag:4"`
-}
-
-// KDCReqFields represents the KRB_KDC_REQ fields.
-type KDCReqFields struct {
-	PVNO    int
-	MsgType int
-	PAData  types.PADataSequence
-	ReqBody KDCReqBody
-	Renewal bool
-}
-
-// ASReq implements RFC 4120 KRB_AS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1.
-type ASReq struct {
-	KDCReqFields
-}
-
-// TGSReq implements RFC 4120 KRB_TGS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1.
-type TGSReq struct {
-	KDCReqFields
-}
-
-type marshalKDCReqBody struct {
-	KDCOptions  asn1.BitString      `asn1:"explicit,tag:0"`
-	CName       types.PrincipalName `asn1:"explicit,optional,tag:1"`
-	Realm       string              `asn1:"generalstring,explicit,tag:2"`
-	SName       types.PrincipalName `asn1:"explicit,optional,tag:3"`
-	From        time.Time           `asn1:"generalized,explicit,optional,tag:4"`
-	Till        time.Time           `asn1:"generalized,explicit,tag:5"`
-	RTime       time.Time           `asn1:"generalized,explicit,optional,tag:6"`
-	Nonce       int                 `asn1:"explicit,tag:7"`
-	EType       []int32             `asn1:"explicit,tag:8"`
-	Addresses   []types.HostAddress `asn1:"explicit,optional,tag:9"`
-	EncAuthData types.EncryptedData `asn1:"explicit,optional,tag:10"`
-	// Ticket needs to be a raw value as it is wrapped in an APPLICATION tag
-	AdditionalTickets asn1.RawValue `asn1:"explicit,optional,tag:11"`
-}
-
-// KDCReqBody implements the KRB_KDC_REQ request body.
-type KDCReqBody struct {
-	KDCOptions        asn1.BitString      `asn1:"explicit,tag:0"`
-	CName             types.PrincipalName `asn1:"explicit,optional,tag:1"`
-	Realm             string              `asn1:"generalstring,explicit,tag:2"`
-	SName             types.PrincipalName `asn1:"explicit,optional,tag:3"`
-	From              time.Time           `asn1:"generalized,explicit,optional,tag:4"`
-	Till              time.Time           `asn1:"generalized,explicit,tag:5"`
-	RTime             time.Time           `asn1:"generalized,explicit,optional,tag:6"`
-	Nonce             int                 `asn1:"explicit,tag:7"`
-	EType             []int32             `asn1:"explicit,tag:8"`
-	Addresses         []types.HostAddress `asn1:"explicit,optional,tag:9"`
-	EncAuthData       types.EncryptedData `asn1:"explicit,optional,tag:10"`
-	AdditionalTickets []Ticket            `asn1:"explicit,optional,tag:11"`
-}
-
-// NewASReqForTGT generates a new KRB_AS_REQ struct for a TGT request.
-func NewASReqForTGT(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) {
-	sname := types.PrincipalName{
-		NameType:   nametype.KRB_NT_SRV_INST,
-		NameString: []string{"krbtgt", realm},
-	}
-	return NewASReq(realm, c, cname, sname)
-}
-
-// NewASReqForChgPasswd generates a new KRB_AS_REQ struct for a change password request.
-func NewASReqForChgPasswd(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) {
-	sname := types.PrincipalName{
-		NameType:   nametype.KRB_NT_PRINCIPAL,
-		NameString: []string{"kadmin", "changepw"},
-	}
-	return NewASReq(realm, c, cname, sname)
-}
-
-// NewASReq generates a new KRB_AS_REQ struct for a given SNAME.
-func NewASReq(realm string, c *config.Config, cname, sname types.PrincipalName) (ASReq, error) {
-	nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32))
-	if err != nil {
-		return ASReq{}, err
-	}
-	t := time.Now().UTC()
-	// Copy the default options to make this thread safe
-	kopts := types.NewKrbFlags()
-	copy(kopts.Bytes, c.LibDefaults.KDCDefaultOptions.Bytes)
-	kopts.BitLength = c.LibDefaults.KDCDefaultOptions.BitLength
-	a := ASReq{
-		KDCReqFields{
-			PVNO:    iana.PVNO,
-			MsgType: msgtype.KRB_AS_REQ,
-			PAData:  types.PADataSequence{},
-			ReqBody: KDCReqBody{
-				KDCOptions: kopts,
-				Realm:      realm,
-				CName:      cname,
-				SName:      sname,
-				Till:       t.Add(c.LibDefaults.TicketLifetime),
-				Nonce:      int(nonce.Int64()),
-				EType:      c.LibDefaults.DefaultTktEnctypeIDs,
-			},
-		},
-	}
-	if c.LibDefaults.Forwardable {
-		types.SetFlag(&a.ReqBody.KDCOptions, flags.Forwardable)
-	}
-	if c.LibDefaults.Canonicalize {
-		types.SetFlag(&a.ReqBody.KDCOptions, flags.Canonicalize)
-	}
-	if c.LibDefaults.Proxiable {
-		types.SetFlag(&a.ReqBody.KDCOptions, flags.Proxiable)
-	}
-	if c.LibDefaults.RenewLifetime != 0 {
-		types.SetFlag(&a.ReqBody.KDCOptions, flags.Renewable)
-		a.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime)
-		a.ReqBody.RTime = t.Add(time.Duration(48) * time.Hour)
-	}
-	if !c.LibDefaults.NoAddresses {
-		ha, err := types.LocalHostAddresses()
-		if err != nil {
-			return a, fmt.Errorf("could not get local addresses: %v", err)
-		}
-		ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...)
-		a.ReqBody.Addresses = ha
-	}
-	return a, nil
-}
-
-// NewTGSReq generates a new KRB_TGS_REQ struct.
-func NewTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, tgt Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool) (TGSReq, error) {
-	a, err := tgsReq(cname, sname, kdcRealm, renewal, c)
-	if err != nil {
-		return a, err
-	}
-	err = a.setPAData(tgt, sessionKey)
-	return a, err
-}
-
-// NewUser2UserTGSReq returns a TGS-REQ suitable for user-to-user authentication (https://tools.ietf.org/html/rfc4120#section-3.7)
-func NewUser2UserTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, clientTGT Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool, verifyingTGT Ticket) (TGSReq, error) {
-	a, err := tgsReq(cname, sname, kdcRealm, renewal, c)
-	if err != nil {
-		return a, err
-	}
-	a.ReqBody.AdditionalTickets = []Ticket{verifyingTGT}
-	types.SetFlag(&a.ReqBody.KDCOptions, flags.EncTktInSkey)
-	err = a.setPAData(clientTGT, sessionKey)
-	return a, err
-}
-
-// tgsReq populates the fields for a TGS_REQ
-func tgsReq(cname, sname types.PrincipalName, kdcRealm string, renewal bool, c *config.Config) (TGSReq, error) {
-	nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32))
-	if err != nil {
-		return TGSReq{}, err
-	}
-	t := time.Now().UTC()
-	k := KDCReqFields{
-		PVNO:    iana.PVNO,
-		MsgType: msgtype.KRB_TGS_REQ,
-		ReqBody: KDCReqBody{
-			KDCOptions: types.NewKrbFlags(),
-			Realm:      kdcRealm,
-			CName:      cname, // Add the CName to make validation of the reply easier
-			SName:      sname,
-			Till:       t.Add(c.LibDefaults.TicketLifetime),
-			Nonce:      int(nonce.Int64()),
-			EType:      c.LibDefaults.DefaultTGSEnctypeIDs,
-		},
-		Renewal: renewal,
-	}
-	if c.LibDefaults.Forwardable {
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Forwardable)
-	}
-	if c.LibDefaults.Canonicalize {
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Canonicalize)
-	}
-	if c.LibDefaults.Proxiable {
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Proxiable)
-	}
-	if c.LibDefaults.RenewLifetime > time.Duration(0) {
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable)
-		k.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime)
-	}
-	if !c.LibDefaults.NoAddresses {
-		ha, err := types.LocalHostAddresses()
-		if err != nil {
-			return TGSReq{}, fmt.Errorf("could not get local addresses: %v", err)
-		}
-		ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...)
-		k.ReqBody.Addresses = ha
-	}
-	if renewal {
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renew)
-		types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable)
-	}
-	return TGSReq{
-		k,
-	}, nil
-}
-
-func (k *TGSReq) setPAData(tgt Ticket, sessionKey types.EncryptionKey) error {
-	// Marshal the request and calculate checksum
-	b, err := k.ReqBody.Marshal()
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error marshaling TGS_REQ body")
-	}
-	etype, err := crypto.GetEtype(sessionKey.KeyType)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncryptingError, "error getting etype to encrypt authenticator")
-	}
-	cb, err := etype.GetChecksumHash(sessionKey.KeyValue, b, keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR_CHKSUM)
-	if err != nil {
-		return krberror.Errorf(err, krberror.ChksumError, "error getting etype checksum hash")
-	}
-
-	// Form PAData for TGS_REQ
-	// Create authenticator
-	auth, err := types.NewAuthenticator(tgt.Realm, k.ReqBody.CName)
-	if err != nil {
-		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator")
-	}
-	auth.Cksum = types.Checksum{
-		CksumType: etype.GetHashID(),
-		Checksum:  cb,
-	}
-	// Create AP_REQ
-	apReq, err := NewAPReq(tgt, sessionKey, auth)
-	if err != nil {
-		return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AP_REQ")
-	}
-	apb, err := apReq.Marshal()
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error marshaling AP_REQ for pre-authentication data")
-	}
-	k.PAData = types.PADataSequence{
-		types.PAData{
-			PADataType:  patype.PA_TGS_REQ,
-			PADataValue: apb,
-		},
-	}
-	return nil
-}
-
-// Unmarshal bytes b into the ASReq struct.
-func (k *ASReq) Unmarshal(b []byte) error {
-	var m marshalKDCReq
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREQ))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling AS_REQ")
-	}
-	expectedMsgType := msgtype.KRB_AS_REQ
-	if m.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a AS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
-	}
-	var reqb KDCReqBody
-	err = reqb.Unmarshal(m.ReqBody.Bytes)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error processing AS_REQ body")
-	}
-	k.MsgType = m.MsgType
-	k.PAData = m.PAData
-	k.PVNO = m.PVNO
-	k.ReqBody = reqb
-	return nil
-}
-
-// Unmarshal bytes b into the TGSReq struct.
-func (k *TGSReq) Unmarshal(b []byte) error {
-	var m marshalKDCReq
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREQ))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling TGS_REQ")
-	}
-	expectedMsgType := msgtype.KRB_TGS_REQ
-	if m.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a TGS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
-	}
-	var reqb KDCReqBody
-	err = reqb.Unmarshal(m.ReqBody.Bytes)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error processing TGS_REQ body")
-	}
-	k.MsgType = m.MsgType
-	k.PAData = m.PAData
-	k.PVNO = m.PVNO
-	k.ReqBody = reqb
-	return nil
-}
-
-// Unmarshal bytes b into the KRB_KDC_REQ body struct.
-func (k *KDCReqBody) Unmarshal(b []byte) error {
-	var m marshalKDCReqBody
-	_, err := asn1.Unmarshal(b, &m)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling KDC_REQ body")
-	}
-	k.KDCOptions = m.KDCOptions
-	if len(k.KDCOptions.Bytes) < 4 {
-		tb := make([]byte, 4-len(k.KDCOptions.Bytes))
-		k.KDCOptions.Bytes = append(tb, k.KDCOptions.Bytes...)
-		k.KDCOptions.BitLength = len(k.KDCOptions.Bytes) * 8
-	}
-	k.CName = m.CName
-	k.Realm = m.Realm
-	k.SName = m.SName
-	k.From = m.From
-	k.Till = m.Till
-	k.RTime = m.RTime
-	k.Nonce = m.Nonce
-	k.EType = m.EType
-	k.Addresses = m.Addresses
-	k.EncAuthData = m.EncAuthData
-	if len(m.AdditionalTickets.Bytes) > 0 {
-		k.AdditionalTickets, err = unmarshalTicketsSequence(m.AdditionalTickets)
-		if err != nil {
-			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling additional tickets")
-		}
-	}
-	return nil
-}
-
-// Marshal ASReq struct.
-func (k *ASReq) Marshal() ([]byte, error) {
-	m := marshalKDCReq{
-		PVNO:    k.PVNO,
-		MsgType: k.MsgType,
-		PAData:  k.PAData,
-	}
-	b, err := k.ReqBody.Marshal()
-	if err != nil {
-		var mk []byte
-		return mk, err
-	}
-	m.ReqBody = asn1.RawValue{
-		Class:      asn1.ClassContextSpecific,
-		IsCompound: true,
-		Tag:        4,
-		Bytes:      b,
-	}
-	mk, err := asn1.Marshal(m)
-	if err != nil {
-		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ")
-	}
-	mk = asn1tools.AddASNAppTag(mk, asnAppTag.ASREQ)
-	return mk, nil
-}
-
-// Marshal TGSReq struct.
-func (k *TGSReq) Marshal() ([]byte, error) {
-	m := marshalKDCReq{
-		PVNO:    k.PVNO,
-		MsgType: k.MsgType,
-		PAData:  k.PAData,
-	}
-	b, err := k.ReqBody.Marshal()
-	if err != nil {
-		var mk []byte
-		return mk, err
-	}
-	m.ReqBody = asn1.RawValue{
-		Class:      asn1.ClassContextSpecific,
-		IsCompound: true,
-		Tag:        4,
-		Bytes:      b,
-	}
-	mk, err := asn1.Marshal(m)
-	if err != nil {
-		return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ")
-	}
-	mk = asn1tools.AddASNAppTag(mk, asnAppTag.TGSREQ)
-	return mk, nil
-}
-
-// Marshal KRB_KDC_REQ body struct.
-func (k *KDCReqBody) Marshal() ([]byte, error) {
-	var b []byte
-	m := marshalKDCReqBody{
-		KDCOptions:  k.KDCOptions,
-		CName:       k.CName,
-		Realm:       k.Realm,
-		SName:       k.SName,
-		From:        k.From,
-		Till:        k.Till,
-		RTime:       k.RTime,
-		Nonce:       k.Nonce,
-		EType:       k.EType,
-		Addresses:   k.Addresses,
-		EncAuthData: k.EncAuthData,
-	}
-	rawtkts, err := MarshalTicketSequence(k.AdditionalTickets)
-	if err != nil {
-		return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body additional tickets")
-	}
-	//The asn1.rawValue needs the tag setting on it for where it is in the KDCReqBody
-	rawtkts.Tag = 11
-	if len(rawtkts.Bytes) > 0 {
-		m.AdditionalTickets = rawtkts
-	}
-	b, err = asn1.Marshal(m)
-	if err != nil {
-		return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body")
-	}
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBCred.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBCred.go
deleted file mode 100644
index 380cf80..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBCred.go
+++ /dev/null
@@ -1,102 +0,0 @@
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-type marshalKRBCred struct {
-	PVNO    int                 `asn1:"explicit,tag:0"`
-	MsgType int                 `asn1:"explicit,tag:1"`
-	Tickets asn1.RawValue       `asn1:"explicit,tag:2"`
-	EncPart types.EncryptedData `asn1:"explicit,tag:3"`
-}
-
-// KRBCred implements RFC 4120 KRB_CRED: https://tools.ietf.org/html/rfc4120#section-5.8.1.
-type KRBCred struct {
-	PVNO             int
-	MsgType          int
-	Tickets          []Ticket
-	EncPart          types.EncryptedData
-	DecryptedEncPart EncKrbCredPart
-}
-
-// EncKrbCredPart is the encrypted part of KRB_CRED.
-type EncKrbCredPart struct {
-	TicketInfo []KrbCredInfo     `asn1:"explicit,tag:0"`
-	Nouce      int               `asn1:"optional,explicit,tag:1"`
-	Timestamp  time.Time         `asn1:"generalized,optional,explicit,tag:2"`
-	Usec       int               `asn1:"optional,explicit,tag:3"`
-	SAddress   types.HostAddress `asn1:"optional,explicit,tag:4"`
-	RAddress   types.HostAddress `asn1:"optional,explicit,tag:5"`
-}
-
-// KrbCredInfo is the KRB_CRED_INFO part of KRB_CRED.
-type KrbCredInfo struct {
-	Key       types.EncryptionKey `asn1:"explicit,tag:0"`
-	PRealm    string              `asn1:"generalstring,optional,explicit,tag:1"`
-	PName     types.PrincipalName `asn1:"optional,explicit,tag:2"`
-	Flags     asn1.BitString      `asn1:"optional,explicit,tag:3"`
-	AuthTime  time.Time           `asn1:"generalized,optional,explicit,tag:4"`
-	StartTime time.Time           `asn1:"generalized,optional,explicit,tag:5"`
-	EndTime   time.Time           `asn1:"generalized,optional,explicit,tag:6"`
-	RenewTill time.Time           `asn1:"generalized,optional,explicit,tag:7"`
-	SRealm    string              `asn1:"optional,explicit,ia5,tag:8"`
-	SName     types.PrincipalName `asn1:"optional,explicit,tag:9"`
-	CAddr     types.HostAddresses `asn1:"optional,explicit,tag:10"`
-}
-
-// Unmarshal bytes b into the KRBCred struct.
-func (k *KRBCred) Unmarshal(b []byte) error {
-	var m marshalKRBCred
-	_, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBCred))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	expectedMsgType := msgtype.KRB_CRED
-	if m.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_CRED. Expected: %v; Actual: %v", expectedMsgType, m.MsgType)
-	}
-	k.PVNO = m.PVNO
-	k.MsgType = m.MsgType
-	k.EncPart = m.EncPart
-	if len(m.Tickets.Bytes) > 0 {
-		k.Tickets, err = unmarshalTicketsSequence(m.Tickets)
-		if err != nil {
-			return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling tickets within KRB_CRED")
-		}
-	}
-	return nil
-}
-
-// DecryptEncPart decrypts the encrypted part of a KRB_CRED.
-func (k *KRBCred) DecryptEncPart(key types.EncryptionKey) error {
-	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_CRED_ENCPART)
-	if err != nil {
-		return krberror.Errorf(err, krberror.DecryptingError, "error decrypting KRB_CRED EncPart")
-	}
-	var denc EncKrbCredPart
-	err = denc.Unmarshal(b)
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part of KRB_CRED")
-	}
-	k.DecryptedEncPart = denc
-	return nil
-}
-
-// Unmarshal bytes b into the encrypted part of KRB_CRED.
-func (k *EncKrbCredPart) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbCredPart))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling EncKrbCredPart")
-	}
-	return nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBError.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBError.go
deleted file mode 100644
index 5aa9def..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBError.go
+++ /dev/null
@@ -1,83 +0,0 @@
-// Package messages implements Kerberos 5 message types and methods.
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// KRBError implements RFC 4120 KRB_ERROR: https://tools.ietf.org/html/rfc4120#section-5.9.1.
-type KRBError struct {
-	PVNO      int                 `asn1:"explicit,tag:0"`
-	MsgType   int                 `asn1:"explicit,tag:1"`
-	CTime     time.Time           `asn1:"generalized,optional,explicit,tag:2"`
-	Cusec     int                 `asn1:"optional,explicit,tag:3"`
-	STime     time.Time           `asn1:"generalized,explicit,tag:4"`
-	Susec     int                 `asn1:"explicit,tag:5"`
-	ErrorCode int32               `asn1:"explicit,tag:6"`
-	CRealm    string              `asn1:"generalstring,optional,explicit,tag:7"`
-	CName     types.PrincipalName `asn1:"optional,explicit,tag:8"`
-	Realm     string              `asn1:"generalstring,explicit,tag:9"`
-	SName     types.PrincipalName `asn1:"explicit,tag:10"`
-	EText     string              `asn1:"generalstring,optional,explicit,tag:11"`
-	EData     []byte              `asn1:"optional,explicit,tag:12"`
-}
-
-// NewKRBError creates a new KRBError.
-func NewKRBError(sname types.PrincipalName, realm string, code int32, etext string) KRBError {
-	t := time.Now().UTC()
-	return KRBError{
-		PVNO:      iana.PVNO,
-		MsgType:   msgtype.KRB_ERROR,
-		STime:     t,
-		Susec:     int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
-		ErrorCode: code,
-		SName:     sname,
-		Realm:     realm,
-		EText:     etext,
-	}
-}
-
-// Unmarshal bytes b into the KRBError struct.
-func (k *KRBError) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBError))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "KRB_ERROR unmarshal error")
-	}
-	expectedMsgType := msgtype.KRB_ERROR
-	if k.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_ERROR. Expected: %v; Actual: %v", expectedMsgType, k.MsgType)
-	}
-	return nil
-}
-
-// Error method implementing error interface on KRBError struct.
-func (k KRBError) Error() string {
-	etxt := fmt.Sprintf("KRB Error: %s", errorcode.Lookup(k.ErrorCode))
-	if k.EText != "" {
-		etxt = fmt.Sprintf("%s - %s", etxt, k.EText)
-	}
-	return etxt
-}
-
-func processUnmarshalReplyError(b []byte, err error) error {
-	switch err.(type) {
-	case asn1.StructuralError:
-		var krberr KRBError
-		tmperr := krberr.Unmarshal(b)
-		if tmperr != nil {
-			return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply")
-		}
-		return krberr
-	default:
-		return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply")
-	}
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBPriv.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBPriv.go
deleted file mode 100644
index ebc5d3d..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBPriv.go
+++ /dev/null
@@ -1,108 +0,0 @@
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// KRBPriv implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.7.1.
-type KRBPriv struct {
-	PVNO             int                 `asn1:"explicit,tag:0"`
-	MsgType          int                 `asn1:"explicit,tag:1"`
-	EncPart          types.EncryptedData `asn1:"explicit,tag:3"`
-	DecryptedEncPart EncKrbPrivPart      `asn1:"optional,omitempty"` // Not part of ASN1 bytes so marked as optional so unmarshalling works
-}
-
-// EncKrbPrivPart is the encrypted part of KRB_PRIV.
-type EncKrbPrivPart struct {
-	UserData       []byte            `asn1:"explicit,tag:0"`
-	Timestamp      time.Time         `asn1:"generalized,optional,explicit,tag:1"`
-	Usec           int               `asn1:"optional,explicit,tag:2"`
-	SequenceNumber int64             `asn1:"optional,explicit,tag:3"`
-	SAddress       types.HostAddress `asn1:"explicit,tag:4"`
-	RAddress       types.HostAddress `asn1:"optional,explicit,tag:5"`
-}
-
-// NewKRBPriv returns a new KRBPriv type.
-func NewKRBPriv(part EncKrbPrivPart) KRBPriv {
-	return KRBPriv{
-		PVNO:             iana.PVNO,
-		MsgType:          msgtype.KRB_PRIV,
-		DecryptedEncPart: part,
-	}
-}
-
-// Unmarshal bytes b into the KRBPriv struct.
-func (k *KRBPriv) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBPriv))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	expectedMsgType := msgtype.KRB_PRIV
-	if k.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_PRIV. Expected: %v; Actual: %v", expectedMsgType, k.MsgType)
-	}
-	return nil
-}
-
-// Unmarshal bytes b into the EncKrbPrivPart struct.
-func (k *EncKrbPrivPart) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbPrivPart))
-	if err != nil {
-		return krberror.Errorf(err, krberror.EncodingError, "KRB_PRIV unmarshal error")
-	}
-	return nil
-}
-
-// Marshal the KRBPriv.
-func (k *KRBPriv) Marshal() ([]byte, error) {
-	tk := KRBPriv{
-		PVNO:    k.PVNO,
-		MsgType: k.MsgType,
-		EncPart: k.EncPart,
-	}
-	b, err := asn1.Marshal(tk)
-	if err != nil {
-		return []byte{}, err
-	}
-	b = asn1tools.AddASNAppTag(b, asnAppTag.KRBPriv)
-	return b, nil
-}
-
-// EncryptEncPart encrypts the DecryptedEncPart within the KRBPriv.
-// Use to prepare for marshaling.
-func (k *KRBPriv) EncryptEncPart(key types.EncryptionKey) error {
-	b, err := asn1.Marshal(k.DecryptedEncPart)
-	if err != nil {
-		return err
-	}
-	b = asn1tools.AddASNAppTag(b, asnAppTag.EncKrbPrivPart)
-	k.EncPart, err = crypto.GetEncryptedData(b, key, keyusage.KRB_PRIV_ENCPART, 1)
-	if err != nil {
-		return err
-	}
-	return nil
-}
-
-// DecryptEncPart decrypts the encrypted part of the KRBPriv message.
-func (k *KRBPriv) DecryptEncPart(key types.EncryptionKey) error {
-	b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_PRIV_ENCPART)
-	if err != nil {
-		return fmt.Errorf("error decrypting KRBPriv EncPart: %v", err)
-	}
-	err = k.DecryptedEncPart.Unmarshal(b)
-	if err != nil {
-		return fmt.Errorf("error unmarshaling encrypted part: %v", err)
-	}
-	return nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBSafe.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBSafe.go
deleted file mode 100644
index 9c5acc1..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/KRBSafe.go
+++ /dev/null
@@ -1,61 +0,0 @@
-package messages
-
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/msgtype"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-/*
-KRB-SAFE        ::= [APPLICATION 20] SEQUENCE {
-	pvno            [0] INTEGER (5),
-	msg-type        [1] INTEGER (20),
-	safe-body       [2] KRB-SAFE-BODY,
-	cksum           [3] Checksum
-}
-
-KRB-SAFE-BODY   ::= SEQUENCE {
-	user-data       [0] OCTET STRING,
-	timestamp       [1] KerberosTime OPTIONAL,
-	usec            [2] Microseconds OPTIONAL,
-	seq-number      [3] UInt32 OPTIONAL,
-	s-address       [4] HostAddress,
-	r-address       [5] HostAddress OPTIONAL
-}
-*/
-
-// KRBSafe implements RFC 4120 KRB_SAFE: https://tools.ietf.org/html/rfc4120#section-5.6.1.
-type KRBSafe struct {
-	PVNO     int            `asn1:"explicit,tag:0"`
-	MsgType  int            `asn1:"explicit,tag:1"`
-	SafeBody KRBSafeBody    `asn1:"explicit,tag:2"`
-	Cksum    types.Checksum `asn1:"explicit,tag:3"`
-}
-
-// KRBSafeBody implements the KRB_SAFE_BODY of KRB_SAFE.
-type KRBSafeBody struct {
-	UserData       []byte            `asn1:"explicit,tag:0"`
-	Timestamp      time.Time         `asn1:"generalized,optional,explicit,tag:1"`
-	Usec           int               `asn1:"optional,explicit,tag:2"`
-	SequenceNumber int64             `asn1:"optional,explicit,tag:3"`
-	SAddress       types.HostAddress `asn1:"explicit,tag:4"`
-	RAddress       types.HostAddress `asn1:"optional,explicit,tag:5"`
-}
-
-// Unmarshal bytes b into the KRBSafe struct.
-func (s *KRBSafe) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, s, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBSafe))
-	if err != nil {
-		return processUnmarshalReplyError(b, err)
-	}
-	expectedMsgType := msgtype.KRB_SAFE
-	if s.MsgType != expectedMsgType {
-		return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_SAFE. Expected: %v; Actual: %v", expectedMsgType, s.MsgType)
-	}
-	return nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/Ticket.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/Ticket.go
deleted file mode 100644
index 49664b8..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/messages/Ticket.go
+++ /dev/null
@@ -1,265 +0,0 @@
-package messages
-
-import (
-	"crypto/rand"
-	"fmt"
-	"log"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/adtype"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/errorcode"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/flags"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/keytab"
-	"gopkg.in/jcmturner/gokrb5.v7/krberror"
-	"gopkg.in/jcmturner/gokrb5.v7/pac"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-)
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.3
-
-// Ticket implements the Kerberos ticket.
-type Ticket struct {
-	TktVNO           int                 `asn1:"explicit,tag:0"`
-	Realm            string              `asn1:"generalstring,explicit,tag:1"`
-	SName            types.PrincipalName `asn1:"explicit,tag:2"`
-	EncPart          types.EncryptedData `asn1:"explicit,tag:3"`
-	DecryptedEncPart EncTicketPart       `asn1:"optional"` // Not part of ASN1 bytes so marked as optional so unmarshalling works
-}
-
-// EncTicketPart is the encrypted part of the Ticket.
-type EncTicketPart struct {
-	Flags             asn1.BitString          `asn1:"explicit,tag:0"`
-	Key               types.EncryptionKey     `asn1:"explicit,tag:1"`
-	CRealm            string                  `asn1:"generalstring,explicit,tag:2"`
-	CName             types.PrincipalName     `asn1:"explicit,tag:3"`
-	Transited         TransitedEncoding       `asn1:"explicit,tag:4"`
-	AuthTime          time.Time               `asn1:"generalized,explicit,tag:5"`
-	StartTime         time.Time               `asn1:"generalized,explicit,optional,tag:6"`
-	EndTime           time.Time               `asn1:"generalized,explicit,tag:7"`
-	RenewTill         time.Time               `asn1:"generalized,explicit,optional,tag:8"`
-	CAddr             types.HostAddresses     `asn1:"explicit,optional,tag:9"`
-	AuthorizationData types.AuthorizationData `asn1:"explicit,optional,tag:10"`
-}
-
-// TransitedEncoding part of the ticket's encrypted part.
-type TransitedEncoding struct {
-	TRType   int32  `asn1:"explicit,tag:0"`
-	Contents []byte `asn1:"explicit,tag:1"`
-}
-
-// NewTicket creates a new Ticket instance.
-func NewTicket(cname types.PrincipalName, crealm string, sname types.PrincipalName, srealm string, flags asn1.BitString, sktab *keytab.Keytab, eTypeID int32, kvno int, authTime, startTime, endTime, renewTill time.Time) (Ticket, types.EncryptionKey, error) {
-	etype, err := crypto.GetEtype(eTypeID)
-	if err != nil {
-		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting etype for new ticket")
-	}
-	ks := etype.GetKeyByteSize()
-	kv := make([]byte, ks, ks)
-	rand.Read(kv)
-	sessionKey := types.EncryptionKey{
-		KeyType:  eTypeID,
-		KeyValue: kv,
-	}
-	etp := EncTicketPart{
-		Flags:     flags,
-		Key:       sessionKey,
-		CRealm:    crealm,
-		CName:     cname,
-		Transited: TransitedEncoding{},
-		AuthTime:  authTime,
-		StartTime: startTime,
-		EndTime:   endTime,
-		RenewTill: renewTill,
-	}
-	b, err := asn1.Marshal(etp)
-	if err != nil {
-		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncodingError, "error marshalling ticket encpart")
-	}
-	b = asn1tools.AddASNAppTag(b, asnAppTag.EncTicketPart)
-	skey, err := sktab.GetEncryptionKey(sname, srealm, kvno, eTypeID)
-	if err != nil {
-		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting encryption key for new ticket")
-	}
-	ed, err := crypto.GetEncryptedData(b, skey, keyusage.KDC_REP_TICKET, kvno)
-	if err != nil {
-		return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error encrypting ticket encpart")
-	}
-	tkt := Ticket{
-		TktVNO:  iana.PVNO,
-		Realm:   srealm,
-		SName:   sname,
-		EncPart: ed,
-	}
-	return tkt, sessionKey, nil
-}
-
-// Unmarshal bytes b into a Ticket struct.
-func (t *Ticket) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.Ticket))
-	return err
-}
-
-// Marshal the Ticket.
-func (t *Ticket) Marshal() ([]byte, error) {
-	b, err := asn1.Marshal(*t)
-	if err != nil {
-		return nil, err
-	}
-	b = asn1tools.AddASNAppTag(b, asnAppTag.Ticket)
-	return b, nil
-}
-
-// Unmarshal bytes b into the EncTicketPart struct.
-func (t *EncTicketPart) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.EncTicketPart))
-	return err
-}
-
-// unmarshalTicket returns a ticket from the bytes provided.
-func unmarshalTicket(b []byte) (t Ticket, err error) {
-	err = t.Unmarshal(b)
-	return
-}
-
-// UnmarshalTicketsSequence returns a slice of Tickets from a raw ASN1 value.
-func unmarshalTicketsSequence(in asn1.RawValue) ([]Ticket, error) {
-	//This is a workaround to a asn1 decoding issue in golang - https://github.com/golang/go/issues/17321. It's not pretty I'm afraid
-	//We pull out raw values from the larger raw value (that is actually the data of the sequence of raw values) and track our position moving along the data.
-	b := in.Bytes
-	// Ignore the head of the asn1 stream (1 byte for tag and those for the length) as this is what tells us its a sequence but we're handling it ourselves
-	p := 1 + asn1tools.GetNumberBytesInLengthHeader(in.Bytes)
-	var tkts []Ticket
-	var raw asn1.RawValue
-	for p < (len(b)) {
-		_, err := asn1.UnmarshalWithParams(b[p:], &raw, fmt.Sprintf("application,tag:%d", asnAppTag.Ticket))
-		if err != nil {
-			return nil, fmt.Errorf("unmarshaling sequence of tickets failed getting length of ticket: %v", err)
-		}
-		t, err := unmarshalTicket(b[p:])
-		if err != nil {
-			return nil, fmt.Errorf("unmarshaling sequence of tickets failed: %v", err)
-		}
-		p += len(raw.FullBytes)
-		tkts = append(tkts, t)
-	}
-	MarshalTicketSequence(tkts)
-	return tkts, nil
-}
-
-// MarshalTicketSequence marshals a slice of Tickets returning an ASN1 raw value containing the ticket sequence.
-func MarshalTicketSequence(tkts []Ticket) (asn1.RawValue, error) {
-	raw := asn1.RawValue{
-		Class:      2,
-		IsCompound: true,
-	}
-	if len(tkts) < 1 {
-		// There are no tickets to marshal
-		return raw, nil
-	}
-	var btkts []byte
-	for i, t := range tkts {
-		b, err := t.Marshal()
-		if err != nil {
-			return raw, fmt.Errorf("error marshaling ticket number %d in sequence of tickets", i+1)
-		}
-		btkts = append(btkts, b...)
-	}
-	// The ASN1 wrapping consists of 2 bytes:
-	// 1st byte -> Identifier Octet - In this case an OCTET STRING (ASN TAG
-	// 2nd byte -> The length (this will be the size indicated in the input bytes + 2 for the additional bytes we add here.
-	// Application Tag:
-	//| Byte:       | 8                            | 7                          | 6                                         | 5 | 4 | 3 | 2 | 1             |
-	//| Value:      | 0                            | 1                          | 1                                         | From the RFC spec 4120        |
-	//| Explanation | Defined by the ASN1 encoding rules for an application tag | A value of 1 indicates a constructed type | The ASN Application tag value |
-	btkts = append(asn1tools.MarshalLengthBytes(len(btkts)), btkts...)
-	btkts = append([]byte{byte(32 + asn1.TagSequence)}, btkts...)
-	raw.Bytes = btkts
-	// If we need to create the full bytes then identifier octet is "context-specific" = 128 + "constructed" + 32 + the wrapping explicit tag (11)
-	//fmt.Fprintf(os.Stderr, "mRaw fb: %v\n", raw.FullBytes)
-	return raw, nil
-}
-
-// DecryptEncPart decrypts the encrypted part of the ticket.
-// The sname argument can be used to specify which service principal's key should be used to decrypt the ticket.
-// If nil is passed as the sname then the service principal specified within the ticket it used.
-func (t *Ticket) DecryptEncPart(keytab *keytab.Keytab, sname *types.PrincipalName) error {
-	if sname == nil {
-		sname = &t.SName
-	}
-	key, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType)
-	if err != nil {
-		return NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err))
-	}
-	return t.Decrypt(key)
-}
-
-// Decrypt decrypts the encrypted part of the ticket using the key provided.
-func (t *Ticket) Decrypt(key types.EncryptionKey) error {
-	b, err := crypto.DecryptEncPart(t.EncPart, key, keyusage.KDC_REP_TICKET)
-	if err != nil {
-		return fmt.Errorf("error decrypting Ticket EncPart: %v", err)
-	}
-	var denc EncTicketPart
-	err = denc.Unmarshal(b)
-	if err != nil {
-		return fmt.Errorf("error unmarshaling encrypted part: %v", err)
-	}
-	t.DecryptedEncPart = denc
-	return nil
-}
-
-// GetPACType returns a Microsoft PAC that has been extracted from the ticket and processed.
-func (t *Ticket) GetPACType(keytab *keytab.Keytab, sname *types.PrincipalName, l *log.Logger) (bool, pac.PACType, error) {
-	var isPAC bool
-	for _, ad := range t.DecryptedEncPart.AuthorizationData {
-		if ad.ADType == adtype.ADIfRelevant {
-			var ad2 types.AuthorizationData
-			err := ad2.Unmarshal(ad.ADData)
-			if err != nil {
-				l.Printf("PAC authorization data could not be unmarshaled: %v", err)
-				continue
-			}
-			if ad2[0].ADType == adtype.ADWin2KPAC {
-				isPAC = true
-				var p pac.PACType
-				err = p.Unmarshal(ad2[0].ADData)
-				if err != nil {
-					return isPAC, p, fmt.Errorf("error unmarshaling PAC: %v", err)
-				}
-				if sname == nil {
-					sname = &t.SName
-				}
-				key, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType)
-				if err != nil {
-					return isPAC, p, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err))
-				}
-				err = p.ProcessPACInfoBuffers(key, l)
-				return isPAC, p, err
-			}
-		}
-	}
-	return isPAC, pac.PACType{}, nil
-}
-
-// Valid checks it the ticket is currently valid. Max duration passed endtime passed in as argument.
-func (t *Ticket) Valid(d time.Duration) (bool, error) {
-	// Check for future tickets or invalid tickets
-	time := time.Now().UTC()
-	if t.DecryptedEncPart.StartTime.Sub(time) > d || types.IsFlagSet(&t.DecryptedEncPart.Flags, flags.Invalid) {
-		return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_NYV, "service ticket provided is not yet valid")
-	}
-
-	// Check for expired ticket
-	if time.Sub(t.DecryptedEncPart.EndTime) > d {
-		return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_EXPIRED, "service ticket provided has expired")
-	}
-
-	return true, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_claims.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_claims.go
deleted file mode 100644
index 612979e..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_claims.go
+++ /dev/null
@@ -1,33 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx
-
-// ClientClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh536365.aspx
-type ClientClaimsInfo struct {
-	ClaimsSetMetadata mstypes.ClaimsSetMetadata
-	ClaimsSet         mstypes.ClaimsSet
-}
-
-// Unmarshal bytes into the ClientClaimsInfo struct
-func (k *ClientClaimsInfo) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	m := new(mstypes.ClaimsSetMetadata)
-	err = dec.Decode(m)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err)
-	}
-	k.ClaimsSetMetadata = *m
-	k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet()
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_info.go
deleted file mode 100644
index ad5212d..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/client_info.go
+++ /dev/null
@@ -1,31 +0,0 @@
-package pac
-
-import (
-	"bytes"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-)
-
-// ClientInfo implements https://msdn.microsoft.com/en-us/library/cc237951.aspx
-type ClientInfo struct {
-	ClientID   mstypes.FileTime // A FILETIME structure in little-endian format that contains the Kerberos initial ticket-granting ticket TGT authentication time
-	NameLength uint16           // An unsigned 16-bit integer in little-endian format that specifies the length, in bytes, of the Name field.
-	Name       string           // An array of 16-bit Unicode characters in little-endian format that contains the client's account name.
-}
-
-// Unmarshal bytes into the ClientInfo struct
-func (k *ClientInfo) Unmarshal(b []byte) (err error) {
-	//The PAC_CLIENT_INFO structure is a simple structure that is not NDR-encoded.
-	r := mstypes.NewReader(bytes.NewReader(b))
-
-	k.ClientID, err = r.FileTime()
-	if err != nil {
-		return
-	}
-	k.NameLength, err = r.Uint16()
-	if err != nil {
-		return
-	}
-	k.Name, err = r.UTF16String(int(k.NameLength))
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/credentials_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/credentials_info.go
deleted file mode 100644
index a8c2c3c..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/credentials_info.go
+++ /dev/null
@@ -1,86 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// https://msdn.microsoft.com/en-us/library/cc237931.aspx
-
-// CredentialsInfo implements https://msdn.microsoft.com/en-us/library/cc237953.aspx
-type CredentialsInfo struct {
-	Version                    uint32 // A 32-bit unsigned integer in little-endian format that defines the version. MUST be 0x00000000.
-	EType                      uint32
-	PACCredentialDataEncrypted []byte // Key usage number for encryption: KERB_NON_KERB_SALT (16)
-	PACCredentialData          CredentialData
-}
-
-// Unmarshal bytes into the CredentialsInfo struct
-func (c *CredentialsInfo) Unmarshal(b []byte, k types.EncryptionKey) (err error) {
-	//The CredentialsInfo structure is a simple structure that is not NDR-encoded.
-	r := mstypes.NewReader(bytes.NewReader(b))
-
-	c.Version, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	if c.Version != 0 {
-		err = errors.New("credentials info version is not zero")
-		return
-	}
-	c.EType, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	c.PACCredentialDataEncrypted, err = r.ReadBytes(len(b) - 8)
-
-	err = c.DecryptEncPart(k)
-	if err != nil {
-		err = fmt.Errorf("error decrypting PAC Credentials Data: %v", err)
-		return
-	}
-	return
-}
-
-// DecryptEncPart decrypts the encrypted part of the CredentialsInfo.
-func (c *CredentialsInfo) DecryptEncPart(k types.EncryptionKey) error {
-	if k.KeyType != int32(c.EType) {
-		return fmt.Errorf("key provided is not the correct type. Type needed: %d, type provided: %d", c.EType, k.KeyType)
-	}
-	pt, err := crypto.DecryptMessage(c.PACCredentialDataEncrypted, k, keyusage.KERB_NON_KERB_SALT)
-	if err != nil {
-		return err
-	}
-	err = c.PACCredentialData.Unmarshal(pt)
-	if err != nil {
-		return err
-	}
-	return nil
-}
-
-// CredentialData implements https://msdn.microsoft.com/en-us/library/cc237952.aspx
-// This structure is encrypted prior to being encoded in any other structures.
-// Encryption is performed by first serializing the data structure via Network Data Representation (NDR) encoding, as specified in [MS-RPCE].
-// Once serialized, the data is encrypted using the key and cryptographic system selected through the AS protocol and the KRB_AS_REP message
-// Fields (for capturing this information) and cryptographic parameters are specified in PAC_CREDENTIAL_INFO (section 2.6.1).
-type CredentialData struct {
-	CredentialCount uint32
-	Credentials     []SECPKGSupplementalCred // Size is the value of CredentialCount
-}
-
-// Unmarshal converts the bytes provided into a CredentialData type.
-func (c *CredentialData) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	err = dec.Decode(c)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_claims.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_claims.go
deleted file mode 100644
index c2299bb..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_claims.go
+++ /dev/null
@@ -1,33 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx
-
-// DeviceClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh554226.aspx
-type DeviceClaimsInfo struct {
-	ClaimsSetMetadata mstypes.ClaimsSetMetadata
-	ClaimsSet         mstypes.ClaimsSet
-}
-
-// Unmarshal bytes into the ClientClaimsInfo struct
-func (k *DeviceClaimsInfo) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	m := new(mstypes.ClaimsSetMetadata)
-	err = dec.Decode(m)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err)
-	}
-	k.ClaimsSetMetadata = *m
-	k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet()
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_info.go
deleted file mode 100644
index 51be207..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/device_info.go
+++ /dev/null
@@ -1,32 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// DeviceInfo implements https://msdn.microsoft.com/en-us/library/hh536402.aspx
-type DeviceInfo struct {
-	UserID            uint32                          // A 32-bit unsigned integer that contains the RID of the account. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account.
-	PrimaryGroupID    uint32                          // A 32-bit unsigned integer that contains the RID for the primary group to which this account belongs.
-	AccountDomainID   mstypes.RPCSID                  `ndr:"pointer"` // A SID structure that contains the SID for the domain of the account.This member is used in conjunction with the UserId, and GroupIds members to create the user and group SIDs for the client.
-	AccountGroupCount uint32                          // A 32-bit unsigned integer that contains the number of groups within the account domain to which the account belongs
-	AccountGroupIDs   []mstypes.GroupMembership       `ndr:"pointer,conformant"` // A pointer to a list of GROUP_MEMBERSHIP (section 2.2.2) structures that contains the groups to which the account belongs in the account domain. The number of groups in this list MUST be equal to GroupCount.
-	SIDCount          uint32                          // A 32-bit unsigned integer that contains the total number of SIDs present in the ExtraSids member.
-	ExtraSIDs         []mstypes.KerbSidAndAttributes  `ndr:"pointer,conformant"` // A pointer to a list of KERB_SID_AND_ATTRIBUTES structures that contain a list of SIDs corresponding to groups not in domains. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account.
-	DomainGroupCount  uint32                          // A 32-bit unsigned integer that contains the number of domains with groups to which the account belongs.
-	DomainGroup       []mstypes.DomainGroupMembership `ndr:"pointer,conformant"` // A pointer to a list of DOMAIN_GROUP_MEMBERSHIP structures (section 2.2.3) that contains the domains to which the account belongs to a group. The number of sets in this list MUST be equal to DomainCount.
-}
-
-// Unmarshal bytes into the DeviceInfo struct
-func (k *DeviceInfo) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	err = dec.Decode(k)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling DeviceInfo: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/kerb_validation_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/kerb_validation_info.go
deleted file mode 100644
index 9dd69d2..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/kerb_validation_info.go
+++ /dev/null
@@ -1,115 +0,0 @@
-// Package pac implements Microsoft Privilege Attribute Certificate (PAC) processing.
-package pac
-
-import (
-	"bytes"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// KERB_VALIDATION_INFO flags.
-const (
-	USERFLAG_GUEST                                    = 31 // Authentication was done via the GUEST account; no password was used.
-	USERFLAG_NO_ENCRYPTION_AVAILABLE                  = 30 // No encryption is available.
-	USERFLAG_LAN_MANAGER_KEY                          = 28 // LAN Manager key was used for authentication.
-	USERFLAG_SUB_AUTH                                 = 25 // Sub-authentication used; session key came from the sub-authentication package.
-	USERFLAG_EXTRA_SIDS                               = 26 // Indicates that the ExtraSids field is populated and contains additional SIDs.
-	USERFLAG_MACHINE_ACCOUNT                          = 24 // Indicates that the account is a machine account.
-	USERFLAG_DC_NTLM2                                 = 23 // Indicates that the domain controller understands NTLMv2.
-	USERFLAG_RESOURCE_GROUPIDS                        = 22 // Indicates that the ResourceGroupIds field is populated.
-	USERFLAG_PROFILEPATH                              = 21 // Indicates that ProfilePath is populated.
-	USERFLAG_NTLM2_NTCHALLENGERESP                    = 20 // The NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation.
-	USERFLAG_LM2_LMCHALLENGERESP                      = 19 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation.
-	USERFLAG_AUTH_LMCHALLENGERESP_KEY_NTCHALLENGERESP = 18 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and the NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used session key generation.
-)
-
-// KerbValidationInfo implement https://msdn.microsoft.com/en-us/library/cc237948.aspx
-// The KERB_VALIDATION_INFO structure defines the user's logon and authorization information
-// provided by the DC. The KERB_VALIDATION_INFO structure is a subset of the
-// NETLOGON_VALIDATION_SAM_INFO4 structure ([MS-NRPC] section 2.2.1.4.13).
-// It is a subset due to historical reasons and to the use of the common Active Directory to generate this information.
-// The KERB_VALIDATION_INFO structure is marshaled by RPC [MS-RPCE].
-type KerbValidationInfo struct {
-	LogOnTime              mstypes.FileTime
-	LogOffTime             mstypes.FileTime
-	KickOffTime            mstypes.FileTime
-	PasswordLastSet        mstypes.FileTime
-	PasswordCanChange      mstypes.FileTime
-	PasswordMustChange     mstypes.FileTime
-	EffectiveName          mstypes.RPCUnicodeString
-	FullName               mstypes.RPCUnicodeString
-	LogonScript            mstypes.RPCUnicodeString
-	ProfilePath            mstypes.RPCUnicodeString
-	HomeDirectory          mstypes.RPCUnicodeString
-	HomeDirectoryDrive     mstypes.RPCUnicodeString
-	LogonCount             uint16
-	BadPasswordCount       uint16
-	UserID                 uint32
-	PrimaryGroupID         uint32
-	GroupCount             uint32
-	GroupIDs               []mstypes.GroupMembership `ndr:"pointer,conformant"`
-	UserFlags              uint32
-	UserSessionKey         mstypes.UserSessionKey
-	LogonServer            mstypes.RPCUnicodeString
-	LogonDomainName        mstypes.RPCUnicodeString
-	LogonDomainID          mstypes.RPCSID `ndr:"pointer"`
-	Reserved1              [2]uint32      // Has 2 elements
-	UserAccountControl     uint32
-	SubAuthStatus          uint32
-	LastSuccessfulILogon   mstypes.FileTime
-	LastFailedILogon       mstypes.FileTime
-	FailedILogonCount      uint32
-	Reserved3              uint32
-	SIDCount               uint32
-	ExtraSIDs              []mstypes.KerbSidAndAttributes `ndr:"pointer,conformant"`
-	ResourceGroupDomainSID mstypes.RPCSID                 `ndr:"pointer"`
-	ResourceGroupCount     uint32
-	ResourceGroupIDs       []mstypes.GroupMembership `ndr:"pointer,conformant"`
-}
-
-// Unmarshal bytes into the DeviceInfo struct
-func (k *KerbValidationInfo) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	err = dec.Decode(k)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err)
-	}
-	return
-}
-
-// GetGroupMembershipSIDs returns a slice of strings containing the group membership SIDs found in the PAC.
-func (k *KerbValidationInfo) GetGroupMembershipSIDs() []string {
-	var g []string
-	lSID := k.LogonDomainID.String()
-	for i := range k.GroupIDs {
-		g = append(g, fmt.Sprintf("%s-%d", lSID, k.GroupIDs[i].RelativeID))
-	}
-	for _, s := range k.ExtraSIDs {
-		var exists = false
-		for _, es := range g {
-			if es == s.SID.String() {
-				exists = true
-				break
-			}
-		}
-		if !exists {
-			g = append(g, s.SID.String())
-		}
-	}
-	for _, r := range k.ResourceGroupIDs {
-		var exists = false
-		s := fmt.Sprintf("%s-%d", k.ResourceGroupDomainSID.String(), r.RelativeID)
-		for _, es := range g {
-			if es == s {
-				exists = true
-				break
-			}
-		}
-		if !exists {
-			g = append(g, s)
-		}
-	}
-	return g
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/pac_type.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/pac_type.go
deleted file mode 100644
index c73fd06..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/pac_type.go
+++ /dev/null
@@ -1,251 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"errors"
-	"fmt"
-	"log"
-
-	"gopkg.in/jcmturner/gokrb5.v7/crypto"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/keyusage"
-	"gopkg.in/jcmturner/gokrb5.v7/types"
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-)
-
-const (
-	infoTypeKerbValidationInfo     uint32 = 1
-	infoTypeCredentials            uint32 = 2
-	infoTypePACServerSignatureData uint32 = 6
-	infoTypePACKDCSignatureData    uint32 = 7
-	infoTypePACClientInfo          uint32 = 10
-	infoTypeS4UDelegationInfo      uint32 = 11
-	infoTypeUPNDNSInfo             uint32 = 12
-	infoTypePACClientClaimsInfo    uint32 = 13
-	infoTypePACDeviceInfo          uint32 = 14
-	infoTypePACDeviceClaimsInfo    uint32 = 15
-)
-
-// PACType implements: https://msdn.microsoft.com/en-us/library/cc237950.aspx
-type PACType struct {
-	CBuffers           uint32
-	Version            uint32
-	Buffers            []InfoBuffer
-	Data               []byte
-	KerbValidationInfo *KerbValidationInfo
-	CredentialsInfo    *CredentialsInfo
-	ServerChecksum     *SignatureData
-	KDCChecksum        *SignatureData
-	ClientInfo         *ClientInfo
-	S4UDelegationInfo  *S4UDelegationInfo
-	UPNDNSInfo         *UPNDNSInfo
-	ClientClaimsInfo   *ClientClaimsInfo
-	DeviceInfo         *DeviceInfo
-	DeviceClaimsInfo   *DeviceClaimsInfo
-	ZeroSigData        []byte
-}
-
-// InfoBuffer implements the PAC Info Buffer: https://msdn.microsoft.com/en-us/library/cc237954.aspx
-type InfoBuffer struct {
-	ULType       uint32 // A 32-bit unsigned integer in little-endian format that describes the type of data present in the buffer contained at Offset.
-	CBBufferSize uint32 // A 32-bit unsigned integer in little-endian format that contains the size, in bytes, of the buffer in the PAC located at Offset.
-	Offset       uint64 // A 64-bit unsigned integer in little-endian format that contains the offset to the beginning of the buffer, in bytes, from the beginning of the PACTYPE structure. The data offset MUST be a multiple of eight. The following sections specify the format of each type of element.
-}
-
-// Unmarshal bytes into the PACType struct
-func (pac *PACType) Unmarshal(b []byte) (err error) {
-	pac.Data = b
-	zb := make([]byte, len(b), len(b))
-	copy(zb, b)
-	pac.ZeroSigData = zb
-	r := mstypes.NewReader(bytes.NewReader(b))
-	pac.CBuffers, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	pac.Version, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	buf := make([]InfoBuffer, pac.CBuffers, pac.CBuffers)
-	for i := range buf {
-		buf[i].ULType, err = r.Uint32()
-		if err != nil {
-			return
-		}
-		buf[i].CBBufferSize, err = r.Uint32()
-		if err != nil {
-			return
-		}
-		buf[i].Offset, err = r.Uint64()
-		if err != nil {
-			return
-		}
-	}
-	pac.Buffers = buf
-	return nil
-}
-
-// ProcessPACInfoBuffers processes the PAC Info Buffers.
-// https://msdn.microsoft.com/en-us/library/cc237954.aspx
-func (pac *PACType) ProcessPACInfoBuffers(key types.EncryptionKey, l *log.Logger) error {
-	for _, buf := range pac.Buffers {
-		p := make([]byte, buf.CBBufferSize, buf.CBBufferSize)
-		copy(p, pac.Data[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)])
-		switch buf.ULType {
-		case infoTypeKerbValidationInfo:
-			if pac.KerbValidationInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k KerbValidationInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				return fmt.Errorf("error processing KerbValidationInfo: %v", err)
-			}
-			pac.KerbValidationInfo = &k
-		case infoTypeCredentials:
-			// Currently PAC parsing is only useful on the service side in gokrb5
-			// The CredentialsInfo are only useful when gokrb5 has implemented RFC4556 and only applied on the client side.
-			// Skipping CredentialsInfo - will be revisited under RFC4556 implementation.
-			continue
-			//if pac.CredentialsInfo != nil {
-			//	//Must ignore subsequent buffers of this type
-			//	continue
-			//}
-			//var k CredentialsInfo
-			//err := k.Unmarshal(p, key) // The encryption key used is the AS reply key only available to the client.
-			//if err != nil {
-			//	return fmt.Errorf("error processing CredentialsInfo: %v", err)
-			//}
-			//pac.CredentialsInfo = &k
-		case infoTypePACServerSignatureData:
-			if pac.ServerChecksum != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k SignatureData
-			zb, err := k.Unmarshal(p)
-			copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb)
-			if err != nil {
-				return fmt.Errorf("error processing ServerChecksum: %v", err)
-			}
-			pac.ServerChecksum = &k
-		case infoTypePACKDCSignatureData:
-			if pac.KDCChecksum != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k SignatureData
-			zb, err := k.Unmarshal(p)
-			copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb)
-			if err != nil {
-				return fmt.Errorf("error processing KDCChecksum: %v", err)
-			}
-			pac.KDCChecksum = &k
-		case infoTypePACClientInfo:
-			if pac.ClientInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k ClientInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				return fmt.Errorf("error processing ClientInfo: %v", err)
-			}
-			pac.ClientInfo = &k
-		case infoTypeS4UDelegationInfo:
-			if pac.S4UDelegationInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k S4UDelegationInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				l.Printf("could not process S4U_DelegationInfo: %v", err)
-				continue
-			}
-			pac.S4UDelegationInfo = &k
-		case infoTypeUPNDNSInfo:
-			if pac.UPNDNSInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k UPNDNSInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				l.Printf("could not process UPN_DNSInfo: %v", err)
-				continue
-			}
-			pac.UPNDNSInfo = &k
-		case infoTypePACClientClaimsInfo:
-			if pac.ClientClaimsInfo != nil || len(p) < 1 {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k ClientClaimsInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				l.Printf("could not process ClientClaimsInfo: %v", err)
-				continue
-			}
-			pac.ClientClaimsInfo = &k
-		case infoTypePACDeviceInfo:
-			if pac.DeviceInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k DeviceInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				l.Printf("could not process DeviceInfo: %v", err)
-				continue
-			}
-			pac.DeviceInfo = &k
-		case infoTypePACDeviceClaimsInfo:
-			if pac.DeviceClaimsInfo != nil {
-				//Must ignore subsequent buffers of this type
-				continue
-			}
-			var k DeviceClaimsInfo
-			err := k.Unmarshal(p)
-			if err != nil {
-				l.Printf("could not process DeviceClaimsInfo: %v", err)
-				continue
-			}
-			pac.DeviceClaimsInfo = &k
-		}
-	}
-
-	if ok, err := pac.verify(key); !ok {
-		return err
-	}
-
-	return nil
-}
-
-func (pac *PACType) verify(key types.EncryptionKey) (bool, error) {
-	if pac.KerbValidationInfo == nil {
-		return false, errors.New("PAC Info Buffers does not contain a KerbValidationInfo")
-	}
-	if pac.ServerChecksum == nil {
-		return false, errors.New("PAC Info Buffers does not contain a ServerChecksum")
-	}
-	if pac.KDCChecksum == nil {
-		return false, errors.New("PAC Info Buffers does not contain a KDCChecksum")
-	}
-	if pac.ClientInfo == nil {
-		return false, errors.New("PAC Info Buffers does not contain a ClientInfo")
-	}
-	etype, err := crypto.GetChksumEtype(int32(pac.ServerChecksum.SignatureType))
-	if err != nil {
-		return false, err
-	}
-	if ok := etype.VerifyChecksum(key.KeyValue,
-		pac.ZeroSigData,
-		pac.ServerChecksum.Signature,
-		keyusage.KERB_NON_KERB_CKSUM_SALT); !ok {
-		return false, errors.New("PAC service checksum verification failed")
-	}
-
-	return true, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/s4u_delegation_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/s4u_delegation_info.go
deleted file mode 100644
index 614ee85..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/s4u_delegation_info.go
+++ /dev/null
@@ -1,26 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// S4UDelegationInfo implements https://msdn.microsoft.com/en-us/library/cc237944.aspx
-type S4UDelegationInfo struct {
-	S4U2proxyTarget      mstypes.RPCUnicodeString // The name of the principal to whom the application can forward the ticket.
-	TransitedListSize    uint32
-	S4UTransitedServices []mstypes.RPCUnicodeString `ndr:"pointer,conformant"` // List of all services that have been delegated through by this client and subsequent services or servers.. Size is value of TransitedListSize
-}
-
-// Unmarshal bytes into the S4UDelegationInfo struct
-func (k *S4UDelegationInfo) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	err = dec.Decode(k)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling S4UDelegationInfo: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/signature_data.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/signature_data.go
deleted file mode 100644
index 7e0fce1..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/signature_data.go
+++ /dev/null
@@ -1,79 +0,0 @@
-package pac
-
-import (
-	"bytes"
-
-	"gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype"
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-)
-
-/*
-https://msdn.microsoft.com/en-us/library/cc237955.aspx
-
-The Key Usage Value MUST be KERB_NON_KERB_CKSUM_SALT (17) [MS-KILE] (section 3.1.5.9).
-
-Server Signature (SignatureType = 0x00000006)
-https://msdn.microsoft.com/en-us/library/cc237957.aspx
-The KDC will use the long-term key that the KDC shares with the server, so that the server can verify this signature on receiving a PAC.
-The server signature is a keyed hash [RFC4757] of the entire PAC message, with the Signature fields of both PAC_SIGNATURE_DATA structures set to zero.
-The key used to protect the ciphertext part of the response is used.
-The checksum type corresponds to the key unless the key is DES, in which case the KERB_CHECKSUM_HMAC_MD5 key is used.
-The resulting hash value is then placed in the Signature field of the server's PAC_SIGNATURE_DATA structure.
-
-KDC Signature (SignatureType = 0x00000007)
-https://msdn.microsoft.com/en-us/library/dd357117.aspx
-The KDC will use KDC (krbtgt) key [RFC4120], so that other KDCs can verify this signature on receiving a PAC.
-The KDC signature is a keyed hash [RFC4757] of the Server Signature field in the PAC message.
-The cryptographic system that is used to calculate the checksum depends on which system the KDC supports, as defined below:
-- Supports RC4-HMAC --> KERB_CHECKSUM_HMAC_MD5
-- Does not support RC4-HMAC and supports AES256 --> HMAC_SHA1_96_AES256
-- Does not support RC4-HMAC or AES256-CTS-HMAC-SHA1-96, and supports AES128-CTS-HMAC-SHA1-96 --> HMAC_SHA1_96_AES128
-- Does not support RC4-HMAC, AES128-CTS-HMAC-SHA1-96 or AES256-CTS-HMAC-SHA1-96 -->  None. The checksum operation will fail.
-*/
-
-// SignatureData implements https://msdn.microsoft.com/en-us/library/cc237955.aspx
-type SignatureData struct {
-	SignatureType  uint32 // A 32-bit unsigned integer value in little-endian format that defines the cryptographic system used to calculate the checksum. This MUST be one of the following checksum types: KERB_CHECKSUM_HMAC_MD5 (signature size = 16), HMAC_SHA1_96_AES128 (signature size = 12), HMAC_SHA1_96_AES256 (signature size = 12).
-	Signature      []byte // Size depends on the type. See comment above.
-	RODCIdentifier uint16 // A 16-bit unsigned integer value in little-endian format that contains the first 16 bits of the key version number ([MS-KILE] section 3.1.5.8) when the KDC is an RODC. When the KDC is not an RODC, this field does not exist.
-}
-
-// Unmarshal bytes into the SignatureData struct
-func (k *SignatureData) Unmarshal(b []byte) (rb []byte, err error) {
-	r := mstypes.NewReader(bytes.NewReader(b))
-
-	k.SignatureType, err = r.Uint32()
-	if err != nil {
-		return
-	}
-
-	var c int
-	switch k.SignatureType {
-	case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED:
-		c = 16
-	case uint32(chksumtype.HMAC_SHA1_96_AES128):
-		c = 12
-	case uint32(chksumtype.HMAC_SHA1_96_AES256):
-		c = 12
-	}
-	k.Signature, err = r.ReadBytes(c)
-	if err != nil {
-		return
-	}
-
-	// When the KDC is not an Read Only Domain Controller (RODC), this field does not exist.
-	if len(b) >= 4+c+2 {
-		k.RODCIdentifier, err = r.Uint16()
-		if err != nil {
-			return
-		}
-	}
-
-	// Create bytes with zeroed signature needed for checksum verification
-	rb = make([]byte, len(b), len(b))
-	copy(rb, b)
-	z := make([]byte, len(b), len(b))
-	copy(rb[4:4+c], z)
-
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/supplemental_cred.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/supplemental_cred.go
deleted file mode 100644
index 5f4f93c..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/supplemental_cred.go
+++ /dev/null
@@ -1,90 +0,0 @@
-package pac
-
-import (
-	"bytes"
-	"encoding/binary"
-	"errors"
-	"fmt"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-const (
-	// NTLMSupCredLMOWF indicates that the LM OWF member is present and valid.
-	NTLMSupCredLMOWF uint32 = 31
-	// NTLMSupCredNTOWF indicates that the NT OWF member is present and valid.
-	NTLMSupCredNTOWF uint32 = 30
-)
-
-// NTLMSupplementalCred implements https://msdn.microsoft.com/en-us/library/cc237949.aspx
-type NTLMSupplementalCred struct {
-	Version    uint32 // A 32-bit unsigned integer that defines the credential version.This field MUST be 0x00000000.
-	Flags      uint32
-	LMPassword []byte // A 16-element array of unsigned 8-bit integers that define the LM OWF. The LMPassword member MUST be ignored if the L flag is not set in the Flags member.
-	NTPassword []byte // A 16-element array of unsigned 8-bit integers that define the NT OWF. The NTPassword member MUST be ignored if the N flag is not set in the Flags member.
-}
-
-// Unmarshal converts the bytes provided into a NTLMSupplementalCred.
-func (c *NTLMSupplementalCred) Unmarshal(b []byte) (err error) {
-	r := mstypes.NewReader(bytes.NewReader(b))
-	c.Version, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	if c.Version != 0 {
-		err = errors.New("NTLMSupplementalCred version is not zero")
-		return
-	}
-	c.Flags, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	if isFlagSet(c.Flags, NTLMSupCredLMOWF) {
-		c.LMPassword, err = r.ReadBytes(16)
-		if err != nil {
-			return
-		}
-	}
-	if isFlagSet(c.Flags, NTLMSupCredNTOWF) {
-		c.NTPassword, err = r.ReadBytes(16)
-		if err != nil {
-			return
-		}
-	}
-	return
-}
-
-// isFlagSet tests if a flag is set in the uint32 little endian flag
-func isFlagSet(f uint32, i uint32) bool {
-	//Which byte?
-	b := int(i / 8)
-	//Which bit in byte
-	p := uint(7 - (int(i) - 8*b))
-	fb := make([]byte, 4)
-	binary.LittleEndian.PutUint32(fb, f)
-	if fb[b]&(1<<p) != 0 {
-		return true
-	}
-	return false
-}
-
-// SECPKGSupplementalCred implements https://msdn.microsoft.com/en-us/library/cc237956.aspx
-// The SECPKG_SUPPLEMENTAL_CRED structure defines the name of the security package that requires
-// supplemental credentials and the credential buffer for that package.
-// The SECPKG_SUPPLEMENTAL_CRED structure is marshaled by RPC.
-type SECPKGSupplementalCred struct {
-	PackageName    mstypes.RPCUnicodeString
-	CredentialSize uint32
-	Credentials    []uint8 `ndr:"pointer,conformant"` // Is a ptr. Size is the value of CredentialSize
-}
-
-// Unmarshal converts the bytes provided into a SECPKGSupplementalCred.
-func (c *SECPKGSupplementalCred) Unmarshal(b []byte) (err error) {
-	dec := ndr.NewDecoder(bytes.NewReader(b))
-	err = dec.Decode(c)
-	if err != nil {
-		err = fmt.Errorf("error unmarshaling SECPKGSupplementalCred: %v", err)
-	}
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/upn_dns_info.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/upn_dns_info.go
deleted file mode 100644
index ee0e6bf..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/pac/upn_dns_info.go
+++ /dev/null
@@ -1,73 +0,0 @@
-package pac
-
-import (
-	"bytes"
-
-	"gopkg.in/jcmturner/rpc.v1/mstypes"
-)
-
-// UPNDNSInfo implements https://msdn.microsoft.com/en-us/library/dd240468.aspx
-type UPNDNSInfo struct {
-	UPNLength           uint16 // An unsigned 16-bit integer in little-endian format that specifies the length, in bytes, of the UPN field.
-	UPNOffset           uint16 // An unsigned 16-bit integer in little-endian format that contains the offset to the beginning of the buffer, in bytes, from the beginning of the UPN_DNS_INFO structure.
-	DNSDomainNameLength uint16
-	DNSDomainNameOffset uint16
-	Flags               uint32
-	UPN                 string
-	DNSDomain           string
-}
-
-const (
-	upnNoUPNAttr = 31 // The user account object does not have the userPrincipalName attribute ([MS-ADA3] section 2.349) set. A UPN constructed by concatenating the user name with the DNS domain name of the account domain is provided.
-)
-
-// Unmarshal bytes into the UPN_DNSInfo struct
-func (k *UPNDNSInfo) Unmarshal(b []byte) (err error) {
-	//The UPN_DNS_INFO structure is a simple structure that is not NDR-encoded.
-	r := mstypes.NewReader(bytes.NewReader(b))
-	k.UPNLength, err = r.Uint16()
-	if err != nil {
-		return
-	}
-	k.UPNOffset, err = r.Uint16()
-	if err != nil {
-		return
-	}
-	k.DNSDomainNameLength, err = r.Uint16()
-	if err != nil {
-		return
-	}
-	k.DNSDomainNameOffset, err = r.Uint16()
-	if err != nil {
-		return
-	}
-	k.Flags, err = r.Uint32()
-	if err != nil {
-		return
-	}
-	ub := mstypes.NewReader(bytes.NewReader(b[k.UPNOffset : k.UPNOffset+k.UPNLength]))
-	db := mstypes.NewReader(bytes.NewReader(b[k.DNSDomainNameOffset : k.DNSDomainNameOffset+k.DNSDomainNameLength]))
-
-	u := make([]rune, k.UPNLength/2, k.UPNLength/2)
-	for i := 0; i < len(u); i++ {
-		var r uint16
-		r, err = ub.Uint16()
-		if err != nil {
-			return
-		}
-		u[i] = rune(r)
-	}
-	k.UPN = string(u)
-	d := make([]rune, k.DNSDomainNameLength/2, k.DNSDomainNameLength/2)
-	for i := 0; i < len(d); i++ {
-		var r uint16
-		r, err = db.Uint16()
-		if err != nil {
-			return
-		}
-		d[i] = rune(r)
-	}
-	k.DNSDomain = string(d)
-
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Authenticator.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Authenticator.go
deleted file mode 100644
index 500e034..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Authenticator.go
+++ /dev/null
@@ -1,100 +0,0 @@
-// Package types provides Kerberos 5 data types.
-package types
-
-import (
-	"crypto/rand"
-	"fmt"
-	"math"
-	"math/big"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/asn1tools"
-	"gopkg.in/jcmturner/gokrb5.v7/iana"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag"
-)
-
-/*Authenticator   ::= [APPLICATION 2] SEQUENCE  {
-authenticator-vno       [0] INTEGER (5),
-crealm                  [1] Realm,
-cname                   [2] PrincipalName,
-cksum                   [3] Checksum OPTIONAL,
-cusec                   [4] Microseconds,
-ctime                   [5] KerberosTime,
-subkey                  [6] EncryptionKey OPTIONAL,
-seq-number              [7] UInt32 OPTIONAL,
-authorization-data      [8] AuthorizationData OPTIONAL
-}
-
-   cksum
-      This field contains a checksum of the application data that
-      accompanies the KRB_AP_REQ, computed using a key usage value of 10
-      in normal application exchanges, or 6 when used in the TGS-REQ
-      PA-TGS-REQ AP-DATA field.
-
-*/
-
-// Authenticator - A record containing information that can be shown to have been recently generated using the session key known only by the client and server.
-// https://tools.ietf.org/html/rfc4120#section-5.5.1
-type Authenticator struct {
-	AVNO              int               `asn1:"explicit,tag:0"`
-	CRealm            string            `asn1:"generalstring,explicit,tag:1"`
-	CName             PrincipalName     `asn1:"explicit,tag:2"`
-	Cksum             Checksum          `asn1:"explicit,optional,tag:3"`
-	Cusec             int               `asn1:"explicit,tag:4"`
-	CTime             time.Time         `asn1:"generalized,explicit,tag:5"`
-	SubKey            EncryptionKey     `asn1:"explicit,optional,tag:6"`
-	SeqNumber         int64             `asn1:"explicit,optional,tag:7"`
-	AuthorizationData AuthorizationData `asn1:"explicit,optional,tag:8"`
-}
-
-// NewAuthenticator creates a new Authenticator.
-func NewAuthenticator(realm string, cname PrincipalName) (Authenticator, error) {
-	seq, err := rand.Int(rand.Reader, big.NewInt(math.MaxUint32))
-	if err != nil {
-		return Authenticator{}, err
-	}
-	t := time.Now().UTC()
-	return Authenticator{
-		AVNO:      iana.PVNO,
-		CRealm:    realm,
-		CName:     cname,
-		Cksum:     Checksum{},
-		Cusec:     int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
-		CTime:     t,
-		SeqNumber: seq.Int64(),
-	}, nil
-}
-
-// GenerateSeqNumberAndSubKey sets the Authenticator's sequence number and subkey.
-func (a *Authenticator) GenerateSeqNumberAndSubKey(keyType int32, keySize int) error {
-	seq, err := rand.Int(rand.Reader, big.NewInt(math.MaxUint32))
-	if err != nil {
-		return err
-	}
-	a.SeqNumber = seq.Int64()
-	//Generate subkey value
-	sk := make([]byte, keySize, keySize)
-	rand.Read(sk)
-	a.SubKey = EncryptionKey{
-		KeyType:  keyType,
-		KeyValue: sk,
-	}
-	return nil
-}
-
-// Unmarshal bytes into the Authenticator.
-func (a *Authenticator) Unmarshal(b []byte) error {
-	_, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.Authenticator))
-	return err
-}
-
-// Marshal the Authenticator.
-func (a *Authenticator) Marshal() ([]byte, error) {
-	b, err := asn1.Marshal(*a)
-	if err != nil {
-		return nil, err
-	}
-	b = asn1tools.AddASNAppTag(b, asnAppTag.Authenticator)
-	return b, nil
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/AuthorizationData.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/AuthorizationData.go
deleted file mode 100644
index c944800..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/AuthorizationData.go
+++ /dev/null
@@ -1,123 +0,0 @@
-package types
-
-import (
-	"github.com/jcmturner/gofork/encoding/asn1"
-)
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.6
-
-/*
-AuthorizationData
-
--- NOTE: AuthorizationData is always used as an OPTIONAL field and
--- should not be empty.
-AuthorizationData       ::= SEQUENCE OF SEQUENCE {
-ad-type         [0] Int32,
-ad-data         [1] OCTET STRING
-}
-
-ad-data
-This field contains authorization data to be interpreted according
-to the value of the corresponding ad-type field.
-
-ad-type
-	This field specifies the format for the ad-data subfield.  All
-negative values are reserved for local use.  Non-negative values
-are reserved for registered use.
-
-Each sequence of type and data is referred to as an authorization
-element.  Elements MAY be application specific; however, there is a
-common set of recursive elements that should be understood by all
-implementations.  These elements contain other elements embedded
-within them, and the interpretation of the encapsulating element
-determines which of the embedded elements must be interpreted, and
-which may be ignored.
-
-These common authorization data elements are recursively defined,
-meaning that the ad-data for these types will itself contain a
-sequence of authorization data whose interpretation is affected by
-the encapsulating element.  Depending on the meaning of the
-encapsulating element, the encapsulated elements may be ignored,
-might be interpreted as issued directly by the KDC, or might be
-stored in a separate plaintext part of the ticket.  The types of the
-encapsulating elements are specified as part of the Kerberos
-specification because the behavior based on these values should be
-understood across implementations, whereas other elements need only
-be understood by the applications that they affect.
-
-Authorization data elements are considered critical if present in a
-ticket or authenticator.  If an unknown authorization data element
-type is received by a server either in an AP-REQ or in a ticket
-contained in an AP-REQ, then, unless it is encapsulated in a known
-authorization data element amending the criticality of the elements
-it contains, authentication MUST fail.  Authorization data is
-intended to restrict the use of a ticket.  If the service cannot
-determine whether the restriction applies to that service, then a
-security weakness may result if the ticket can be used for that
-service.  Authorization elements that are optional can be enclosed in
-an AD-IF-RELEVANT element.
-
-In the definitions that follow, the value of the ad-type for the
-element will be specified as the least significant part of the
-subsection number, and the value of the ad-data will be as shown in
-the ASN.1 structure that follows the subsection heading.
-
-   Contents of ad-data                ad-type
-
-   DER encoding of AD-IF-RELEVANT        1
-
-   DER encoding of AD-KDCIssued          4
-
-   DER encoding of AD-AND-OR             5
-
-   DER encoding of AD-MANDATORY-FOR-KDC  8
-
-*/
-
-// AuthorizationData implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6
-type AuthorizationData []AuthorizationDataEntry
-
-// AuthorizationDataEntry implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6
-type AuthorizationDataEntry struct {
-	ADType int32  `asn1:"explicit,tag:0"`
-	ADData []byte `asn1:"explicit,tag:1"`
-}
-
-// ADIfRelevant implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.1
-type ADIfRelevant AuthorizationData
-
-// ADKDCIssued implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.2
-type ADKDCIssued struct {
-	ADChecksum Checksum          `asn1:"explicit,tag:0"`
-	IRealm     string            `asn1:"optional,generalstring,explicit,tag:1"`
-	Isname     PrincipalName     `asn1:"optional,explicit,tag:2"`
-	Elements   AuthorizationData `asn1:"explicit,tag:3"`
-}
-
-// ADAndOr implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.3
-type ADAndOr struct {
-	ConditionCount int32             `asn1:"explicit,tag:0"`
-	Elements       AuthorizationData `asn1:"explicit,tag:1"`
-}
-
-// ADMandatoryForKDC implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.6.4
-type ADMandatoryForKDC AuthorizationData
-
-// Unmarshal bytes into the ADKDCIssued.
-func (a *ADKDCIssued) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the AuthorizationData.
-func (a *AuthorizationData) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the AuthorizationDataEntry.
-func (a *AuthorizationDataEntry) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Cryptosystem.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Cryptosystem.go
deleted file mode 100644
index 7e8b4ab..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/Cryptosystem.go
+++ /dev/null
@@ -1,55 +0,0 @@
-package types
-
-import (
-	"github.com/jcmturner/gofork/encoding/asn1"
-)
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.9
-
-// EncryptedData implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
-type EncryptedData struct {
-	EType  int32  `asn1:"explicit,tag:0"`
-	KVNO   int    `asn1:"explicit,optional,tag:1"`
-	Cipher []byte `asn1:"explicit,tag:2"`
-}
-
-// EncryptionKey implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
-// AKA KeyBlock
-type EncryptionKey struct {
-	KeyType  int32  `asn1:"explicit,tag:0"`
-	KeyValue []byte `asn1:"explicit,tag:1"`
-}
-
-// Checksum implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.9
-type Checksum struct {
-	CksumType int32  `asn1:"explicit,tag:0"`
-	Checksum  []byte `asn1:"explicit,tag:1"`
-}
-
-// Unmarshal bytes into the EncryptedData.
-func (a *EncryptedData) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Marshal the EncryptedData.
-func (a *EncryptedData) Marshal() ([]byte, error) {
-	edb, err := asn1.Marshal(*a)
-	if err != nil {
-		return edb, err
-	}
-	return edb, nil
-}
-
-// Unmarshal bytes into the EncryptionKey.
-func (a *EncryptionKey) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the Checksum.
-func (a *Checksum) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/HostAddress.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/HostAddress.go
deleted file mode 100644
index 2f6a5a7..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/HostAddress.go
+++ /dev/null
@@ -1,204 +0,0 @@
-package types
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.5
-
-import (
-	"bytes"
-	"fmt"
-	"net"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/addrtype"
-)
-
-/*
-HostAddress and HostAddresses
-
-HostAddress     ::= SEQUENCE  {
-	addr-type       [0] Int32,
-	address         [1] OCTET STRING
-}
-
--- NOTE: HostAddresses is always used as an OPTIONAL field and
--- should not be empty.
-HostAddresses   -- NOTE: subtly different from rfc1510,
-		-- but has a value mapping and encodes the same
-	::= SEQUENCE OF HostAddress
-
-The host address encodings consist of two fields:
-
-addr-type
-	This field specifies the type of address that follows.  Pre-
-	defined values for this field are specified in Section 7.5.3.
-
-address
-	This field encodes a single address of type addr-type.
-*/
-
-// HostAddresses implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.5
-type HostAddresses []HostAddress
-
-// HostAddress implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.5
-type HostAddress struct {
-	AddrType int32  `asn1:"explicit,tag:0"`
-	Address  []byte `asn1:"explicit,tag:1"`
-}
-
-// GetHostAddress returns a HostAddress struct from a string in the format <hostname>:<port>
-func GetHostAddress(s string) (HostAddress, error) {
-	var h HostAddress
-	cAddr, _, err := net.SplitHostPort(s)
-	if err != nil {
-		return h, fmt.Errorf("invalid format of client address: %v", err)
-	}
-	ip := net.ParseIP(cAddr)
-	var ht int32
-	if ip.To4() != nil {
-		ht = addrtype.IPv4
-		ip = ip.To4()
-	} else if ip.To16() != nil {
-		ht = addrtype.IPv6
-		ip = ip.To16()
-	} else {
-		return h, fmt.Errorf("could not determine client's address types: %v", err)
-	}
-	h = HostAddress{
-		AddrType: ht,
-		Address:  ip,
-	}
-	return h, nil
-}
-
-// GetAddress returns a string representation of the HostAddress.
-func (h *HostAddress) GetAddress() (string, error) {
-	var b []byte
-	_, err := asn1.Unmarshal(h.Address, &b)
-	return string(b), err
-}
-
-// LocalHostAddresses returns a HostAddresses struct for the local machines interface IP addresses.
-func LocalHostAddresses() (ha HostAddresses, err error) {
-	ifs, err := net.Interfaces()
-	if err != nil {
-		return
-	}
-	for _, iface := range ifs {
-		if iface.Flags&net.FlagLoopback != 0 || iface.Flags&net.FlagUp == 0 {
-			// Interface is either loopback of not up
-			continue
-		}
-		addrs, err := iface.Addrs()
-		if err != nil {
-			continue
-		}
-		for _, addr := range addrs {
-			var ip net.IP
-			switch v := addr.(type) {
-			case *net.IPNet:
-				ip = v.IP
-			case *net.IPAddr:
-				ip = v.IP
-			}
-			var a HostAddress
-			if ip.To16() == nil {
-				//neither IPv4 or IPv6
-				continue
-			}
-			if ip.To4() != nil {
-				//Is IPv4
-				a.AddrType = addrtype.IPv4
-				a.Address = ip.To4()
-			} else {
-				a.AddrType = addrtype.IPv6
-				a.Address = ip.To16()
-			}
-			ha = append(ha, a)
-		}
-	}
-	return ha, nil
-}
-
-// HostAddressesFromNetIPs returns a HostAddresses type from a slice of net.IP
-func HostAddressesFromNetIPs(ips []net.IP) (ha HostAddresses) {
-	for _, ip := range ips {
-		ha = append(ha, HostAddressFromNetIP(ip))
-	}
-	return ha
-}
-
-// HostAddressFromNetIP returns a HostAddress type from a net.IP
-func HostAddressFromNetIP(ip net.IP) HostAddress {
-	if ip.To4() != nil {
-		//Is IPv4
-		return HostAddress{
-			AddrType: addrtype.IPv4,
-			Address:  ip.To4(),
-		}
-	}
-	return HostAddress{
-		AddrType: addrtype.IPv6,
-		Address:  ip.To16(),
-	}
-}
-
-// HostAddressesEqual tests if two HostAddress slices are equal.
-func HostAddressesEqual(h, a []HostAddress) bool {
-	if len(h) != len(a) {
-		return false
-	}
-	for _, e := range a {
-		var found bool
-		for _, i := range h {
-			if e.Equal(i) {
-				found = true
-				break
-			}
-		}
-		if !found {
-			return false
-		}
-	}
-	return true
-}
-
-// HostAddressesContains tests if a HostAddress is contained in a HostAddress slice.
-func HostAddressesContains(h []HostAddress, a HostAddress) bool {
-	for _, e := range h {
-		if e.Equal(a) {
-			return true
-		}
-	}
-	return false
-}
-
-// Equal tests if the HostAddress is equal to another HostAddress provided.
-func (h *HostAddress) Equal(a HostAddress) bool {
-	if h.AddrType != a.AddrType {
-		return false
-	}
-	return bytes.Equal(h.Address, a.Address)
-}
-
-// Contains tests if a HostAddress is contained within the HostAddresses struct.
-func (h *HostAddresses) Contains(a HostAddress) bool {
-	for _, e := range *h {
-		if e.Equal(a) {
-			return true
-		}
-	}
-	return false
-}
-
-// Equal tests if a HostAddress slice is equal to the HostAddresses struct.
-func (h *HostAddresses) Equal(a []HostAddress) bool {
-	if len(*h) != len(a) {
-		return false
-	}
-	for _, e := range a {
-		if !h.Contains(e) {
-			return false
-		}
-	}
-	return true
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/KerberosFlags.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/KerberosFlags.go
deleted file mode 100644
index bd75d5b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/KerberosFlags.go
+++ /dev/null
@@ -1,124 +0,0 @@
-package types
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.8
-
-import (
-	"github.com/jcmturner/gofork/encoding/asn1"
-)
-
-/*
-KerberosFlags
-
-For several message types, a specific constrained bit string type,
-KerberosFlags, is used.
-
-KerberosFlags   ::= BIT STRING (SIZE (32..MAX))
--- minimum number of bits shall be sent,
--- but no fewer than 32
-
-Compatibility note: The following paragraphs describe a change from
-the RFC 1510 description of bit strings that would result in
-incompatility in the case of an implementation that strictly
-conformed to ASN.1 DER and RFC 1510.
-
-ASN.1 bit strings have multiple uses.  The simplest use of a bit
-string is to contain a vector of bits, with no particular meaning
-attached to individual bits.  This vector of bits is not necessarily
-a multiple of eight bits long.  The use in Kerberos of a bit string
-as a compact boolean vector wherein each element has a distinct
-meaning poses some problems.  The natural notation for a compact
-boolean vector is the ASN.1 "NamedBit" notation, and the DER require
-that encodings of a bit string using "NamedBit" notation exclude any
-trailing zero bits.  This truncation is easy to neglect, especially
-given C language implementations that naturally choose to store
-boolean vectors as 32-bit integers.
-
-For example, if the notation for KDCOptions were to include the
-"NamedBit" notation, as in RFC 1510, and a KDCOptions value to be
-encoded had only the "forwardable" (bit number one) bit set, the DER
-encoding MUST include only two bits: the first reserved bit
-("reserved", bit number zero, value zero) and the one-valued bit (bit
-number one) for "forwardable".
-
-Most existing implementations of Kerberos unconditionally send 32
-bits on the wire when encoding bit strings used as boolean vectors.
-This behavior violates the ASN.1 syntax used for flag values in RFC
-1510, but it occurs on such a widely installed base that the protocol
-description is being modified to accommodate it.
-
-Consequently, this document removes the "NamedBit" notations for
-individual bits, relegating them to comments.  The size constraint on
-the KerberosFlags type requires that at least 32 bits be encoded at
-all times, though a lenient implementation MAY choose to accept fewer
-than 32 bits and to treat the missing bits as set to zero.
-
-Currently, no uses of KerberosFlags specify more than 32 bits' worth
-of flags, although future revisions of this document may do so.  When
-more than 32 bits are to be transmitted in a KerberosFlags value,
-future revisions to this document will likely specify that the
-smallest number of bits needed to encode the highest-numbered one-
-valued bit should be sent.  This is somewhat similar to the DER
-encoding of a bit string that is declared with the "NamedBit"
-notation.
-*/
-
-// NewKrbFlags returns an ASN1 BitString struct of the right size for KrbFlags.
-func NewKrbFlags() asn1.BitString {
-	f := asn1.BitString{}
-	f.Bytes = make([]byte, 4)
-	f.BitLength = len(f.Bytes) * 8
-	return f
-}
-
-// SetFlags sets the flags of an ASN1 BitString.
-func SetFlags(f *asn1.BitString, j []int) {
-	for _, i := range j {
-		SetFlag(f, i)
-	}
-}
-
-// SetFlag sets a flag in an ASN1 BitString.
-func SetFlag(f *asn1.BitString, i int) {
-	for l := len(f.Bytes); l < 4; l++ {
-		(*f).Bytes = append((*f).Bytes, byte(0))
-		(*f).BitLength = len((*f).Bytes) * 8
-	}
-	//Which byte?
-	b := i / 8
-	//Which bit in byte
-	p := uint(7 - (i - 8*b))
-	(*f).Bytes[b] = (*f).Bytes[b] | (1 << p)
-}
-
-// UnsetFlags unsets flags in an ASN1 BitString.
-func UnsetFlags(f *asn1.BitString, j []int) {
-	for _, i := range j {
-		UnsetFlag(f, i)
-	}
-}
-
-// UnsetFlag unsets a flag in an ASN1 BitString.
-func UnsetFlag(f *asn1.BitString, i int) {
-	for l := len(f.Bytes); l < 4; l++ {
-		(*f).Bytes = append((*f).Bytes, byte(0))
-		(*f).BitLength = len((*f).Bytes) * 8
-	}
-	//Which byte?
-	b := i / 8
-	//Which bit in byte
-	p := uint(7 - (i - 8*b))
-	(*f).Bytes[b] = (*f).Bytes[b] &^ (1 << p)
-}
-
-// IsFlagSet tests if a flag is set in the ASN1 BitString.
-func IsFlagSet(f *asn1.BitString, i int) bool {
-	//Which byte?
-	b := i / 8
-	//Which bit in byte
-	p := uint(7 - (i - 8*b))
-	if (*f).Bytes[b]&(1<<p) != 0 {
-		return true
-	}
-	return false
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PAData.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PAData.go
deleted file mode 100644
index 484ec5b..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PAData.go
+++ /dev/null
@@ -1,155 +0,0 @@
-package types
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.7
-import (
-	"fmt"
-	"time"
-
-	"github.com/jcmturner/gofork/encoding/asn1"
-	"gopkg.in/jcmturner/gokrb5.v7/iana/patype"
-)
-
-// PAData implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7
-type PAData struct {
-	PADataType  int32  `asn1:"explicit,tag:1"`
-	PADataValue []byte `asn1:"explicit,tag:2"`
-}
-
-// PADataSequence implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7
-type PADataSequence []PAData
-
-// MethodData implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.9.1
-type MethodData []PAData
-
-// PAEncTimestamp implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.2
-type PAEncTimestamp EncryptedData
-
-// PAEncTSEnc implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.2
-type PAEncTSEnc struct {
-	PATimestamp time.Time `asn1:"generalized,explicit,tag:0"`
-	PAUSec      int       `asn1:"explicit,optional,tag:1"`
-}
-
-// Contains tests if a PADataSequence contains PA Data of a certain type.
-func (pas *PADataSequence) Contains(patype int32) bool {
-	for _, pa := range *pas {
-		if pa.PADataType == patype {
-			return true
-		}
-	}
-	return false
-}
-
-// GetPAEncTSEncAsnMarshalled returns the bytes of a PAEncTSEnc.
-func GetPAEncTSEncAsnMarshalled() ([]byte, error) {
-	t := time.Now().UTC()
-	p := PAEncTSEnc{
-		PATimestamp: t,
-		PAUSec:      int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)),
-	}
-	b, err := asn1.Marshal(p)
-	if err != nil {
-		return b, fmt.Errorf("error mashaling PAEncTSEnc: %v", err)
-	}
-	return b, nil
-}
-
-// ETypeInfoEntry implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.4
-type ETypeInfoEntry struct {
-	EType int32  `asn1:"explicit,tag:0"`
-	Salt  []byte `asn1:"explicit,optional,tag:1"`
-}
-
-// ETypeInfo implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.4
-type ETypeInfo []ETypeInfoEntry
-
-// ETypeInfo2Entry implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.5
-type ETypeInfo2Entry struct {
-	EType     int32  `asn1:"explicit,tag:0"`
-	Salt      string `asn1:"explicit,optional,generalstring,tag:1"`
-	S2KParams []byte `asn1:"explicit,optional,tag:2"`
-}
-
-// ETypeInfo2 implements RFC 4120 types: https://tools.ietf.org/html/rfc4120#section-5.2.7.5
-type ETypeInfo2 []ETypeInfo2Entry
-
-// PAReqEncPARep PA Data Type
-type PAReqEncPARep struct {
-	ChksumType int32  `asn1:"explicit,tag:0"`
-	Chksum     []byte `asn1:"explicit,tag:1"`
-}
-
-// Unmarshal bytes into the PAData
-func (pa *PAData) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, pa)
-	return err
-}
-
-// Unmarshal bytes into the PADataSequence
-func (pas *PADataSequence) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, pas)
-	return err
-}
-
-// Unmarshal bytes into the PAReqEncPARep
-func (pa *PAReqEncPARep) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, pa)
-	return err
-}
-
-// Unmarshal bytes into the PAEncTimestamp
-func (pa *PAEncTimestamp) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, pa)
-	return err
-}
-
-// Unmarshal bytes into the PAEncTSEnc
-func (pa *PAEncTSEnc) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, pa)
-	return err
-}
-
-// Unmarshal bytes into the ETypeInfo
-func (a *ETypeInfo) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the ETypeInfoEntry
-func (a *ETypeInfoEntry) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the ETypeInfo2
-func (a *ETypeInfo2) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// Unmarshal bytes into the ETypeInfo2Entry
-func (a *ETypeInfo2Entry) Unmarshal(b []byte) error {
-	_, err := asn1.Unmarshal(b, a)
-	return err
-}
-
-// GetETypeInfo returns an ETypeInfo from the PAData.
-func (pa *PAData) GetETypeInfo() (d ETypeInfo, err error) {
-	if pa.PADataType != patype.PA_ETYPE_INFO {
-		err = fmt.Errorf("PAData does not contain PA EType Info data. TypeID Expected: %v; Actual: %v", patype.PA_ETYPE_INFO, pa.PADataType)
-		return
-	}
-	_, err = asn1.Unmarshal(pa.PADataValue, &d)
-	return
-}
-
-// GetETypeInfo2 returns an ETypeInfo2 from the PAData.
-func (pa *PAData) GetETypeInfo2() (d ETypeInfo2, err error) {
-	if pa.PADataType != patype.PA_ETYPE_INFO2 {
-		err = fmt.Errorf("PAData does not contain PA EType Info 2 data. TypeID Expected: %v; Actual: %v", patype.PA_ETYPE_INFO2, pa.PADataType)
-		return
-	}
-	_, err = asn1.Unmarshal(pa.PADataValue, &d)
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PrincipalName.go b/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PrincipalName.go
deleted file mode 100644
index fa6d3ce..0000000
--- a/vendor/gopkg.in/jcmturner/gokrb5.v7/types/PrincipalName.go
+++ /dev/null
@@ -1,64 +0,0 @@
-package types
-
-import (
-	"strings"
-
-	"gopkg.in/jcmturner/gokrb5.v7/iana/nametype"
-)
-
-// Reference: https://www.ietf.org/rfc/rfc4120.txt
-// Section: 5.2.2
-
-// PrincipalName implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.2.2
-type PrincipalName struct {
-	NameType   int32    `asn1:"explicit,tag:0"`
-	NameString []string `asn1:"generalstring,explicit,tag:1"`
-}
-
-// NewPrincipalName creates a new PrincipalName from the name type int32 and name string provided.
-func NewPrincipalName(ntype int32, spn string) PrincipalName {
-	return PrincipalName{
-		NameType:   ntype,
-		NameString: strings.Split(spn, "/"),
-	}
-}
-
-// GetSalt returns a salt derived from the PrincipalName.
-func (pn PrincipalName) GetSalt(realm string) string {
-	var sb []byte
-	sb = append(sb, realm...)
-	for _, n := range pn.NameString {
-		sb = append(sb, n...)
-	}
-	return string(sb)
-}
-
-// Equal tests if the PrincipalName is equal to the one provided.
-func (pn PrincipalName) Equal(n PrincipalName) bool {
-	//https://tools.ietf.org/html/rfc4120#section-6.2 - the name type is not significant when checking for equivalence
-	for i, s := range pn.NameString {
-		if n.NameString[i] != s {
-			return false
-		}
-	}
-	return true
-}
-
-// PrincipalNameString returns the PrincipalName in string form.
-func (pn PrincipalName) PrincipalNameString() string {
-	return strings.Join(pn.NameString, "/")
-}
-
-// ParseSPNString will parse a string in the format <service>/<name>@<realm>
-// a PrincipalName type will be returned with the name type set to KRB_NT_PRINCIPAL(1)
-// and the realm will be returned as a string. If the "@<realm>" suffix
-// is not included in the SPN then the value of realm string returned will be ""
-func ParseSPNString(spn string) (pn PrincipalName, realm string) {
-	if strings.Contains(spn, "@") {
-		s := strings.Split(spn, "@")
-		realm = s[len(s)-1]
-		spn = strings.TrimSuffix(spn, "@"+realm)
-	}
-	pn = NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn)
-	return
-}
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/claims.go b/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/claims.go
deleted file mode 100644
index 2b375da..0000000
--- a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/claims.go
+++ /dev/null
@@ -1,134 +0,0 @@
-package mstypes
-
-import (
-	"bytes"
-	"errors"
-
-	"gopkg.in/jcmturner/rpc.v1/ndr"
-)
-
-// Compression format assigned numbers.
-const (
-	CompressionFormatNone       uint16 = 0
-	CompressionFormatLZNT1      uint16 = 2
-	CompressionFormatXPress     uint16 = 3
-	CompressionFormatXPressHuff uint16 = 4
-)
-
-// ClaimsSourceTypeAD https://msdn.microsoft.com/en-us/library/hh553809.aspx
-const ClaimsSourceTypeAD uint16 = 1
-
-// Claim Type assigned numbers
-const (
-	ClaimTypeIDInt64    uint16 = 1
-	ClaimTypeIDUInt64   uint16 = 2
-	ClaimTypeIDString   uint16 = 3
-	ClaimsTypeIDBoolean uint16 = 6
-)
-
-// ClaimsBlob implements https://msdn.microsoft.com/en-us/library/hh554119.aspx
-type ClaimsBlob struct {
-	Size        uint32
-	EncodedBlob EncodedBlob
-}
-
-// EncodedBlob are the bytes of the encoded Claims
-type EncodedBlob []byte
-
-// Size returns the size of the bytes of the encoded Claims
-func (b EncodedBlob) Size(c interface{}) int {
-	cb := c.(ClaimsBlob)
-	return int(cb.Size)
-}
-
-// ClaimsSetMetadata implements https://msdn.microsoft.com/en-us/library/hh554073.aspx
-type ClaimsSetMetadata struct {
-	ClaimsSetSize             uint32
-	ClaimsSetBytes            []byte `ndr:"pointer,conformant"`
-	CompressionFormat         uint16 // Enum see constants for options
-	UncompressedClaimsSetSize uint32
-	ReservedType              uint16
-	ReservedFieldSize         uint32
-	ReservedField             []byte `ndr:"pointer,conformant"`
-}
-
-// ClaimsSet reads the ClaimsSet type from the NDR encoded ClaimsSetBytes in the ClaimsSetMetadata
-func (m *ClaimsSetMetadata) ClaimsSet() (c ClaimsSet, err error) {
-	if len(m.ClaimsSetBytes) < 1 {
-		err = errors.New("no bytes available for ClaimsSet")
-		return
-	}
-	// TODO switch statement to decompress ClaimsSetBytes
-	if m.CompressionFormat != CompressionFormatNone {
-		err = errors.New("compressed ClaimsSet not currently supported")
-		return
-	}
-	dec := ndr.NewDecoder(bytes.NewReader(m.ClaimsSetBytes))
-	err = dec.Decode(&c)
-	return
-}
-
-// ClaimsSet implements https://msdn.microsoft.com/en-us/library/hh554122.aspx
-type ClaimsSet struct {
-	ClaimsArrayCount  uint32
-	ClaimsArrays      []ClaimsArray `ndr:"pointer,conformant"`
-	ReservedType      uint16
-	ReservedFieldSize uint32
-	ReservedField     []byte `ndr:"pointer,conformant"`
-}
-
-// ClaimsArray implements https://msdn.microsoft.com/en-us/library/hh536458.aspx
-type ClaimsArray struct {
-	ClaimsSourceType uint16
-	ClaimsCount      uint32
-	ClaimEntries     []ClaimEntry `ndr:"pointer,conformant"`
-}
-
-// ClaimEntry is a NDR union that implements https://msdn.microsoft.com/en-us/library/hh536374.aspx
-type ClaimEntry struct {
-	ID         string           `ndr:"pointer,conformant,varying"`
-	Type       uint16           `ndr:"unionTag"`
-	TypeInt64  ClaimTypeInt64   `ndr:"unionField"`
-	TypeUInt64 ClaimTypeUInt64  `ndr:"unionField"`
-	TypeString ClaimTypeString  `ndr:"unionField"`
-	TypeBool   ClaimTypeBoolean `ndr:"unionField"`
-}
-
-// SwitchFunc is the ClaimEntry union field selection function
-func (u ClaimEntry) SwitchFunc(_ interface{}) string {
-	switch u.Type {
-	case ClaimTypeIDInt64:
-		return "TypeInt64"
-	case ClaimTypeIDUInt64:
-		return "TypeUInt64"
-	case ClaimTypeIDString:
-		return "TypeString"
-	case ClaimsTypeIDBoolean:
-		return "TypeBool"
-	}
-	return ""
-}
-
-// ClaimTypeInt64 is a claim of type int64
-type ClaimTypeInt64 struct {
-	ValueCount uint32
-	Value      []int64 `ndr:"pointer,conformant"`
-}
-
-// ClaimTypeUInt64 is a claim of type uint64
-type ClaimTypeUInt64 struct {
-	ValueCount uint32
-	Value      []uint64 `ndr:"pointer,conformant"`
-}
-
-// ClaimTypeString is a claim of type string
-type ClaimTypeString struct {
-	ValueCount uint32
-	Value      []LPWSTR `ndr:"pointer,conformant"`
-}
-
-// ClaimTypeBoolean is a claim of type bool
-type ClaimTypeBoolean struct {
-	ValueCount uint32
-	Value      []bool `ndr:"pointer,conformant"`
-}
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/common.go b/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/common.go
deleted file mode 100644
index 62cac28..0000000
--- a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/common.go
+++ /dev/null
@@ -1,10 +0,0 @@
-package mstypes
-
-// LPWSTR implements https://msdn.microsoft.com/en-us/library/cc230355.aspx
-type LPWSTR struct {
-	Value string `ndr:"pointer,conformant,varying"`
-}
-
-func (s *LPWSTR) String() string {
-	return s.Value
-}
diff --git a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/sid.go b/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/sid.go
deleted file mode 100644
index 98a9c5a..0000000
--- a/vendor/gopkg.in/jcmturner/rpc.v1/mstypes/sid.go
+++ /dev/null
@@ -1,32 +0,0 @@
-package mstypes
-
-import (
-	"encoding/binary"
-	"encoding/hex"
-	"fmt"
-)
-
-// RPCSID implements https://msdn.microsoft.com/en-us/library/cc230364.aspx
-type RPCSID struct {
-	Revision            uint8    // An 8-bit unsigned integer that specifies the revision level of the SID. This value MUST be set to 0x01.
-	SubAuthorityCount   uint8    // An 8-bit unsigned integer that specifies the number of elements in the SubAuthority array. The maximum number of elements allowed is 15.
-	IdentifierAuthority [6]byte  // An RPC_SID_IDENTIFIER_AUTHORITY structure that indicates the authority under which the SID was created. It describes the entity that created the SID. The Identifier Authority value {0,0,0,0,0,5} denotes SIDs created by the NT SID authority.
-	SubAuthority        []uint32 `ndr:"conformant"` // A variable length array of unsigned 32-bit integers that uniquely identifies a principal relative to the IdentifierAuthority. Its length is determined by SubAuthorityCount.
-}
-
-// String returns the string representation of the RPC_SID.
-func (s *RPCSID) String() string {
-	var str string
-	b := append(make([]byte, 2, 2), s.IdentifierAuthority[:]...)
-	// For a strange reason this is read big endian: https://msdn.microsoft.com/en-us/library/dd302645.aspx
-	i := binary.BigEndian.Uint64(b)
-	if i >= 4294967296 {
-		str = fmt.Sprintf("S-1-0x%s", hex.EncodeToString(s.IdentifierAuthority[:]))
-	} else {
-		str = fmt.Sprintf("S-1-%d", i)
-	}
-	for _, sub := range s.SubAuthority {
-		str = fmt.Sprintf("%s-%d", str, sub)
-	}
-	return str
-}
diff --git a/vendor/gopkg.in/yaml.v3/.travis.yml b/vendor/gopkg.in/yaml.v3/.travis.yml
deleted file mode 100644
index 04d4dae..0000000
--- a/vendor/gopkg.in/yaml.v3/.travis.yml
+++ /dev/null
@@ -1,16 +0,0 @@
-language: go
-
-go:
-    - "1.4.x"
-    - "1.5.x"
-    - "1.6.x"
-    - "1.7.x"
-    - "1.8.x"
-    - "1.9.x"
-    - "1.10.x"
-    - "1.11.x"
-    - "1.12.x"
-    - "1.13.x"
-    - "tip"
-
-go_import_path: gopkg.in/yaml.v3
diff --git a/vendor/gopkg.in/yaml.v3/apic.go b/vendor/gopkg.in/yaml.v3/apic.go
index 65846e6..ae7d049 100644
--- a/vendor/gopkg.in/yaml.v3/apic.go
+++ b/vendor/gopkg.in/yaml.v3/apic.go
@@ -108,6 +108,7 @@
 		raw_buffer: make([]byte, 0, output_raw_buffer_size),
 		states:     make([]yaml_emitter_state_t, 0, initial_stack_size),
 		events:     make([]yaml_event_t, 0, initial_queue_size),
+		best_width: -1,
 	}
 }
 
diff --git a/vendor/gopkg.in/yaml.v3/decode.go b/vendor/gopkg.in/yaml.v3/decode.go
index be63169..df36e3a 100644
--- a/vendor/gopkg.in/yaml.v3/decode.go
+++ b/vendor/gopkg.in/yaml.v3/decode.go
@@ -35,6 +35,7 @@
 	doc      *Node
 	anchors  map[string]*Node
 	doneInit bool
+	textless bool
 }
 
 func newParser(b []byte) *parser {
@@ -108,14 +109,18 @@
 func (p *parser) fail() {
 	var where string
 	var line int
-	if p.parser.problem_mark.line != 0 {
+	if p.parser.context_mark.line != 0 {
+		line = p.parser.context_mark.line
+		// Scanner errors don't iterate line before returning error
+		if p.parser.error == yaml_SCANNER_ERROR {
+			line++
+		}
+	} else if p.parser.problem_mark.line != 0 {
 		line = p.parser.problem_mark.line
 		// Scanner errors don't iterate line before returning error
 		if p.parser.error == yaml_SCANNER_ERROR {
 			line++
 		}
-	} else if p.parser.context_mark.line != 0 {
-		line = p.parser.context_mark.line
 	}
 	if line != 0 {
 		where = "line " + strconv.Itoa(line) + ": "
@@ -169,17 +174,20 @@
 	} else if kind == ScalarNode {
 		tag, _ = resolve("", value)
 	}
-	return &Node{
-		Kind:        kind,
-		Tag:         tag,
-		Value:       value,
-		Style:       style,
-		Line:        p.event.start_mark.line + 1,
-		Column:      p.event.start_mark.column + 1,
-		HeadComment: string(p.event.head_comment),
-		LineComment: string(p.event.line_comment),
-		FootComment: string(p.event.foot_comment),
+	n := &Node{
+		Kind:  kind,
+		Tag:   tag,
+		Value: value,
+		Style: style,
 	}
+	if !p.textless {
+		n.Line = p.event.start_mark.line + 1
+		n.Column = p.event.start_mark.column + 1
+		n.HeadComment = string(p.event.head_comment)
+		n.LineComment = string(p.event.line_comment)
+		n.FootComment = string(p.event.foot_comment)
+	}
+	return n
 }
 
 func (p *parser) parseChild(parent *Node) *Node {
@@ -497,8 +505,13 @@
 		good = d.mapping(n, out)
 	case SequenceNode:
 		good = d.sequence(n, out)
+	case 0:
+		if n.IsZero() {
+			return d.null(out)
+		}
+		fallthrough
 	default:
-		panic("internal error: unknown node kind: " + strconv.Itoa(int(n.Kind)))
+		failf("cannot decode node with unknown kind %d", n.Kind)
 	}
 	return good
 }
@@ -533,6 +546,17 @@
 	}
 }
 
+func (d *decoder) null(out reflect.Value) bool {
+	if out.CanAddr() {
+		switch out.Kind() {
+		case reflect.Interface, reflect.Ptr, reflect.Map, reflect.Slice:
+			out.Set(reflect.Zero(out.Type()))
+			return true
+		}
+	}
+	return false
+}
+
 func (d *decoder) scalar(n *Node, out reflect.Value) bool {
 	var tag string
 	var resolved interface{}
@@ -550,14 +574,7 @@
 		}
 	}
 	if resolved == nil {
-		if out.CanAddr() {
-			switch out.Kind() {
-			case reflect.Interface, reflect.Ptr, reflect.Map, reflect.Slice:
-				out.Set(reflect.Zero(out.Type()))
-				return true
-			}
-		}
-		return false
+		return d.null(out)
 	}
 	if resolvedv := reflect.ValueOf(resolved); out.Type() == resolvedv.Type() {
 		// We've resolved to exactly the type we want, so use that.
@@ -791,8 +808,10 @@
 		}
 	}
 
+	mapIsNew := false
 	if out.IsNil() {
 		out.Set(reflect.MakeMap(outt))
+		mapIsNew = true
 	}
 	for i := 0; i < l; i += 2 {
 		if isMerge(n.Content[i]) {
@@ -809,7 +828,7 @@
 				failf("invalid map key: %#v", k.Interface())
 			}
 			e := reflect.New(et).Elem()
-			if d.unmarshal(n.Content[i+1], e) {
+			if d.unmarshal(n.Content[i+1], e) || n.Content[i+1].ShortTag() == nullTag && (mapIsNew || !out.MapIndex(k).IsValid()) {
 				out.SetMapIndex(k, e)
 			}
 		}
diff --git a/vendor/gopkg.in/yaml.v3/emitterc.go b/vendor/gopkg.in/yaml.v3/emitterc.go
index ab2a066..0f47c9c 100644
--- a/vendor/gopkg.in/yaml.v3/emitterc.go
+++ b/vendor/gopkg.in/yaml.v3/emitterc.go
@@ -235,10 +235,13 @@
 			emitter.indent = 0
 		}
 	} else if !indentless {
-		emitter.indent += emitter.best_indent
-		// [Go] If inside a block sequence item, discount the space taken by the indicator.
-		if emitter.best_indent > 2 && emitter.states[len(emitter.states)-1] == yaml_EMIT_BLOCK_SEQUENCE_ITEM_STATE {
-			emitter.indent -= 2
+		// [Go] This was changed so that indentations are more regular.
+		if emitter.states[len(emitter.states)-1] == yaml_EMIT_BLOCK_SEQUENCE_ITEM_STATE {
+			// The first indent inside a sequence will just skip the "- " indicator.
+			emitter.indent += 2
+		} else {
+			// Everything else aligns to the chosen indentation.
+			emitter.indent = emitter.best_indent*((emitter.indent+emitter.best_indent)/emitter.best_indent)
 		}
 	}
 	return true
@@ -725,16 +728,9 @@
 // Expect a block item node.
 func yaml_emitter_emit_block_sequence_item(emitter *yaml_emitter_t, event *yaml_event_t, first bool) bool {
 	if first {
-		// [Go] The original logic here would not indent the sequence when inside a mapping.
-		// In Go we always indent it, but take the sequence indicator out of the indentation.
-		indentless := emitter.best_indent == 2 && emitter.mapping_context && (emitter.column == 0 || !emitter.indention)
-		original := emitter.indent
-		if !yaml_emitter_increase_indent(emitter, false, indentless) {
+		if !yaml_emitter_increase_indent(emitter, false, false) {
 			return false
 		}
-		if emitter.indent > original+2 {
-			emitter.indent -= 2
-		}
 	}
 	if event.typ == yaml_SEQUENCE_END_EVENT {
 		emitter.indent = emitter.indents[len(emitter.indents)-1]
@@ -785,6 +781,13 @@
 	if !yaml_emitter_write_indent(emitter) {
 		return false
 	}
+	if len(emitter.line_comment) > 0 {
+		// [Go] A line comment was provided for the key. That's unusual as the
+		//      scanner associates line comments with the value. Either way,
+		//      save the line comment and render it appropriately later.
+		emitter.key_line_comment = emitter.line_comment
+		emitter.line_comment = nil
+	}
 	if yaml_emitter_check_simple_key(emitter) {
 		emitter.states = append(emitter.states, yaml_EMIT_BLOCK_MAPPING_SIMPLE_VALUE_STATE)
 		return yaml_emitter_emit_node(emitter, event, false, false, true, true)
@@ -810,6 +813,27 @@
 			return false
 		}
 	}
+	if len(emitter.key_line_comment) > 0 {
+		// [Go] Line comments are generally associated with the value, but when there's
+		//      no value on the same line as a mapping key they end up attached to the
+		//      key itself.
+		if event.typ == yaml_SCALAR_EVENT {
+			if len(emitter.line_comment) == 0 {
+				// A scalar is coming and it has no line comments by itself yet,
+				// so just let it handle the line comment as usual. If it has a
+				// line comment, we can't have both so the one from the key is lost.
+				emitter.line_comment = emitter.key_line_comment
+				emitter.key_line_comment = nil
+			}
+		} else if event.sequence_style() != yaml_FLOW_SEQUENCE_STYLE && (event.typ == yaml_MAPPING_START_EVENT || event.typ == yaml_SEQUENCE_START_EVENT) {
+			// An indented block follows, so write the comment right now.
+			emitter.line_comment, emitter.key_line_comment = emitter.key_line_comment, emitter.line_comment
+			if !yaml_emitter_process_line_comment(emitter) {
+				return false
+			}
+			emitter.line_comment, emitter.key_line_comment = emitter.key_line_comment, emitter.line_comment
+		}
+	}
 	emitter.states = append(emitter.states, yaml_EMIT_BLOCK_MAPPING_KEY_STATE)
 	if !yaml_emitter_emit_node(emitter, event, false, false, true, false) {
 		return false
@@ -823,6 +847,10 @@
 	return true
 }
 
+func yaml_emitter_silent_nil_event(emitter *yaml_emitter_t, event *yaml_event_t) bool {
+	return event.typ == yaml_SCALAR_EVENT && event.implicit && !emitter.canonical && len(emitter.scalar_data.value) == 0
+}
+
 // Expect a node.
 func yaml_emitter_emit_node(emitter *yaml_emitter_t, event *yaml_event_t,
 	root bool, sequence bool, mapping bool, simple_key bool) bool {
@@ -1866,7 +1894,7 @@
 	if !yaml_emitter_write_block_scalar_hints(emitter, value) {
 		return false
 	}
-	if !put_break(emitter) {
+	if !yaml_emitter_process_line_comment(emitter) {
 		return false
 	}
 	//emitter.indention = true
@@ -1903,10 +1931,10 @@
 	if !yaml_emitter_write_block_scalar_hints(emitter, value) {
 		return false
 	}
-
-	if !put_break(emitter) {
+	if !yaml_emitter_process_line_comment(emitter) {
 		return false
 	}
+
 	//emitter.indention = true
 	emitter.whitespace = true
 
diff --git a/vendor/gopkg.in/yaml.v3/encode.go b/vendor/gopkg.in/yaml.v3/encode.go
index 1f37271..de9e72a 100644
--- a/vendor/gopkg.in/yaml.v3/encode.go
+++ b/vendor/gopkg.in/yaml.v3/encode.go
@@ -119,6 +119,14 @@
 	case *Node:
 		e.nodev(in)
 		return
+	case Node:
+		if !in.CanAddr() {
+			var n = reflect.New(in.Type()).Elem()
+			n.Set(in)
+			in = n
+		}
+		e.nodev(in.Addr())
+		return
 	case time.Time:
 		e.timev(tag, in)
 		return
@@ -422,18 +430,23 @@
 }
 
 func (e *encoder) node(node *Node, tail string) {
+	// Zero nodes behave as nil.
+	if node.Kind == 0 && node.IsZero() {
+		e.nilv()
+		return
+	}
+
 	// If the tag was not explicitly requested, and dropping it won't change the
 	// implicit tag of the value, don't include it in the presentation.
 	var tag = node.Tag
 	var stag = shortTag(tag)
-	var rtag string
 	var forceQuoting bool
 	if tag != "" && node.Style&TaggedStyle == 0 {
 		if node.Kind == ScalarNode {
 			if stag == strTag && node.Style&(SingleQuotedStyle|DoubleQuotedStyle|LiteralStyle|FoldedStyle) != 0 {
 				tag = ""
 			} else {
-				rtag, _ = resolve("", node.Value)
+				rtag, _ := resolve("", node.Value)
 				if rtag == stag {
 					tag = ""
 				} else if stag == strTag {
@@ -442,6 +455,7 @@
 				}
 			}
 		} else {
+			var rtag string
 			switch node.Kind {
 			case MappingNode:
 				rtag = mapTag
@@ -471,7 +485,7 @@
 		if node.Style&FlowStyle != 0 {
 			style = yaml_FLOW_SEQUENCE_STYLE
 		}
-		e.must(yaml_sequence_start_event_initialize(&e.event, []byte(node.Anchor), []byte(tag), tag == "", style))
+		e.must(yaml_sequence_start_event_initialize(&e.event, []byte(node.Anchor), []byte(longTag(tag)), tag == "", style))
 		e.event.head_comment = []byte(node.HeadComment)
 		e.emit()
 		for _, node := range node.Content {
@@ -487,7 +501,7 @@
 		if node.Style&FlowStyle != 0 {
 			style = yaml_FLOW_MAPPING_STYLE
 		}
-		yaml_mapping_start_event_initialize(&e.event, []byte(node.Anchor), []byte(tag), tag == "", style)
+		yaml_mapping_start_event_initialize(&e.event, []byte(node.Anchor), []byte(longTag(tag)), tag == "", style)
 		e.event.tail_comment = []byte(tail)
 		e.event.head_comment = []byte(node.HeadComment)
 		e.emit()
@@ -528,11 +542,11 @@
 	case ScalarNode:
 		value := node.Value
 		if !utf8.ValidString(value) {
-			if tag == binaryTag {
+			if stag == binaryTag {
 				failf("explicitly tagged !!binary data must be base64-encoded")
 			}
-			if tag != "" {
-				failf("cannot marshal invalid UTF-8 data as %s", shortTag(tag))
+			if stag != "" {
+				failf("cannot marshal invalid UTF-8 data as %s", stag)
 			}
 			// It can't be encoded directly as YAML so use a binary tag
 			// and encode it as base64.
@@ -557,5 +571,7 @@
 		}
 
 		e.emitScalar(value, node.Anchor, tag, style, []byte(node.HeadComment), []byte(node.LineComment), []byte(node.FootComment), []byte(tail))
+	default:
+		failf("cannot encode node with unknown kind %d", node.Kind)
 	}
 }
diff --git a/vendor/gopkg.in/yaml.v3/parserc.go b/vendor/gopkg.in/yaml.v3/parserc.go
index aea9050..ac66fcc 100644
--- a/vendor/gopkg.in/yaml.v3/parserc.go
+++ b/vendor/gopkg.in/yaml.v3/parserc.go
@@ -648,6 +648,10 @@
 			implicit:   implicit,
 			style:      yaml_style_t(yaml_BLOCK_MAPPING_STYLE),
 		}
+		if parser.stem_comment != nil {
+			event.head_comment = parser.stem_comment
+			parser.stem_comment = nil
+		}
 		return true
 	}
 	if len(anchor) > 0 || len(tag) > 0 {
@@ -694,25 +698,13 @@
 
 	if token.typ == yaml_BLOCK_ENTRY_TOKEN {
 		mark := token.end_mark
-		prior_head := len(parser.head_comment)
+		prior_head_len := len(parser.head_comment)
 		skip_token(parser)
+		yaml_parser_split_stem_comment(parser, prior_head_len)
 		token = peek_token(parser)
 		if token == nil {
 			return false
 		}
-		if prior_head > 0 && token.typ == yaml_BLOCK_SEQUENCE_START_TOKEN {
-			// [Go] It's a sequence under a sequence entry, so the former head comment
-			//      is for the list itself, not the first list item under it.
-			parser.stem_comment = parser.head_comment[:prior_head]
-			if len(parser.head_comment) == prior_head {
-				parser.head_comment = nil
-			} else {
-				// Copy suffix to prevent very strange bugs if someone ever appends
-				// further bytes to the prefix in the stem_comment slice above.
-				parser.head_comment = append([]byte(nil), parser.head_comment[prior_head+1:]...)
-			}
-
-		}
 		if token.typ != yaml_BLOCK_ENTRY_TOKEN && token.typ != yaml_BLOCK_END_TOKEN {
 			parser.states = append(parser.states, yaml_PARSE_BLOCK_SEQUENCE_ENTRY_STATE)
 			return yaml_parser_parse_node(parser, event, true, false)
@@ -754,7 +746,9 @@
 
 	if token.typ == yaml_BLOCK_ENTRY_TOKEN {
 		mark := token.end_mark
+		prior_head_len := len(parser.head_comment)
 		skip_token(parser)
+		yaml_parser_split_stem_comment(parser, prior_head_len)
 		token = peek_token(parser)
 		if token == nil {
 			return false
@@ -780,6 +774,32 @@
 	return true
 }
 
+// Split stem comment from head comment.
+//
+// When a sequence or map is found under a sequence entry, the former head comment
+// is assigned to the underlying sequence or map as a whole, not the individual
+// sequence or map entry as would be expected otherwise. To handle this case the
+// previous head comment is moved aside as the stem comment.
+func yaml_parser_split_stem_comment(parser *yaml_parser_t, stem_len int) {
+	if stem_len == 0 {
+		return
+	}
+
+	token := peek_token(parser)
+	if token.typ != yaml_BLOCK_SEQUENCE_START_TOKEN && token.typ != yaml_BLOCK_MAPPING_START_TOKEN {
+		return
+	}
+
+	parser.stem_comment = parser.head_comment[:stem_len]
+	if len(parser.head_comment) == stem_len {
+		parser.head_comment = nil
+	} else {
+		// Copy suffix to prevent very strange bugs if someone ever appends
+		// further bytes to the prefix in the stem_comment slice above.
+		parser.head_comment = append([]byte(nil), parser.head_comment[stem_len+1:]...)
+	}
+}
+
 // Parse the productions:
 // block_mapping        ::= BLOCK-MAPPING_START
 //                          *******************
diff --git a/vendor/gopkg.in/yaml.v3/scannerc.go b/vendor/gopkg.in/yaml.v3/scannerc.go
index 57e954c..ca00701 100644
--- a/vendor/gopkg.in/yaml.v3/scannerc.go
+++ b/vendor/gopkg.in/yaml.v3/scannerc.go
@@ -749,6 +749,11 @@
 		if !ok {
 			return
 		}
+		if len(parser.tokens) > 0 && parser.tokens[len(parser.tokens)-1].typ == yaml_BLOCK_ENTRY_TOKEN {
+			// Sequence indicators alone have no line comments. It becomes
+			// a head comment for whatever follows.
+			return
+		}
 		if !yaml_parser_scan_line_comment(parser, comment_mark) {
 			ok = false
 			return
@@ -2255,10 +2260,9 @@
 		}
 	}
 	if parser.buffer[parser.buffer_pos] == '#' {
-		// TODO Test this and then re-enable it.
-		//if !yaml_parser_scan_line_comment(parser, start_mark) {
-		//	return false
-		//}
+		if !yaml_parser_scan_line_comment(parser, start_mark) {
+			return false
+		}
 		for !is_breakz(parser.buffer, parser.buffer_pos) {
 			skip(parser)
 			if parser.unread < 1 && !yaml_parser_update_buffer(parser, 1) {
@@ -2856,13 +2860,12 @@
 						return false
 					}
 					skip_line(parser)
-				} else {
-					if parser.mark.index >= seen {
-						if len(text) == 0 {
-							start_mark = parser.mark
-						}
-						text = append(text, parser.buffer[parser.buffer_pos])
+				} else if parser.mark.index >= seen {
+					if len(text) == 0 {
+						start_mark = parser.mark
 					}
+					text = read(parser, text)
+				} else {
 					skip(parser)
 				}
 			}
@@ -2888,6 +2891,10 @@
 
 	var token_mark = token.start_mark
 	var start_mark yaml_mark_t
+	var next_indent = parser.indent
+	if next_indent < 0 {
+		next_indent = 0
+	}
 
 	var recent_empty = false
 	var first_empty = parser.newlines <= 1
@@ -2919,15 +2926,18 @@
 			continue
 		}
 		c := parser.buffer[parser.buffer_pos+peek]
-		if is_breakz(parser.buffer, parser.buffer_pos+peek) || parser.flow_level > 0 && (c == ']' || c == '}') {
+		var close_flow = parser.flow_level > 0 && (c == ']' || c == '}')
+		if close_flow || is_breakz(parser.buffer, parser.buffer_pos+peek) {
 			// Got line break or terminator.
-			if !recent_empty {
-				if first_empty && (start_mark.line == foot_line || start_mark.column-1 < parser.indent) {
+			if close_flow || !recent_empty {
+				if close_flow || first_empty && (start_mark.line == foot_line && token.typ != yaml_VALUE_TOKEN || start_mark.column-1 < next_indent) {
 					// This is the first empty line and there were no empty lines before,
 					// so this initial part of the comment is a foot of the prior token
 					// instead of being a head for the following one. Split it up.
+					// Alternatively, this might also be the last comment inside a flow
+					// scope, so it must be a footer.
 					if len(text) > 0 {
-						if start_mark.column-1 < parser.indent {
+						if start_mark.column-1 < next_indent {
 							// If dedented it's unrelated to the prior token.
 							token_mark = start_mark
 						}
@@ -2958,7 +2968,7 @@
 			continue
 		}
 
-		if len(text) > 0 && column < parser.indent+1 && column != start_mark.column {
+		if len(text) > 0 && (close_flow || column-1 < next_indent && column != start_mark.column) {
 			// The comment at the different indentation is a foot of the
 			// preceding data rather than a head of the upcoming one.
 			parser.comments = append(parser.comments, yaml_comment_t{
@@ -2999,10 +3009,9 @@
 					return false
 				}
 				skip_line(parser)
+			} else if parser.mark.index >= seen {
+				text = read(parser, text)
 			} else {
-				if parser.mark.index >= seen {
-					text = append(text, parser.buffer[parser.buffer_pos])
-				}
 				skip(parser)
 			}
 		}
@@ -3010,6 +3019,10 @@
 		peek = 0
 		column = 0
 		line = parser.mark.line
+		next_indent = parser.indent
+		if next_indent < 0 {
+			next_indent = 0
+		}
 	}
 
 	if len(text) > 0 {
diff --git a/vendor/gopkg.in/yaml.v3/yaml.go b/vendor/gopkg.in/yaml.v3/yaml.go
index b5d35a5..8cec6da 100644
--- a/vendor/gopkg.in/yaml.v3/yaml.go
+++ b/vendor/gopkg.in/yaml.v3/yaml.go
@@ -89,7 +89,7 @@
 	return unmarshal(in, out, false)
 }
 
-// A Decorder reads and decodes YAML values from an input stream.
+// A Decoder reads and decodes YAML values from an input stream.
 type Decoder struct {
 	parser      *parser
 	knownFields bool
@@ -194,7 +194,7 @@
 //                  Zero valued structs will be omitted if all their public
 //                  fields are zero, unless they implement an IsZero
 //                  method (see the IsZeroer interface type), in which
-//                  case the field will be included if that method returns true.
+//                  case the field will be excluded if IsZero returns true.
 //
 //     flow         Marshal using a flow style (useful for structs,
 //                  sequences and maps).
@@ -252,6 +252,24 @@
 	return nil
 }
 
+// Encode encodes value v and stores its representation in n.
+//
+// See the documentation for Marshal for details about the
+// conversion of Go values into YAML.
+func (n *Node) Encode(v interface{}) (err error) {
+	defer handleErr(&err)
+	e := newEncoder()
+	defer e.destroy()
+	e.marshalDoc("", reflect.ValueOf(v))
+	e.finish()
+	p := newParser(e.out)
+	p.textless = true
+	defer p.destroy()
+	doc := p.parse()
+	*n = *doc.Content[0]
+	return nil
+}
+
 // SetIndent changes the used indentation used when encoding.
 func (e *Encoder) SetIndent(spaces int) {
 	if spaces < 0 {
@@ -328,6 +346,12 @@
 // and maps, Node is an intermediate representation that allows detailed
 // control over the content being decoded or encoded.
 //
+// It's worth noting that although Node offers access into details such as
+// line numbers, colums, and comments, the content when re-encoded will not
+// have its original textual representation preserved. An effort is made to
+// render the data plesantly, and to preserve comments near the data they
+// describe, though.
+//
 // Values that make use of the Node type interact with the yaml package in the
 // same way any other type would do, by encoding and decoding yaml data
 // directly or indirectly into them.
@@ -391,6 +415,13 @@
 	Column int
 }
 
+// IsZero returns whether the node has all of its fields unset.
+func (n *Node) IsZero() bool {
+	return n.Kind == 0 && n.Style == 0 && n.Tag == "" && n.Value == "" && n.Anchor == "" && n.Alias == nil && n.Content == nil &&
+		n.HeadComment == "" && n.LineComment == "" && n.FootComment == "" && n.Line == 0 && n.Column == 0
+}
+
+
 // LongTag returns the long form of the tag that indicates the data type for
 // the node. If the Tag field isn't explicitly defined, one will be computed
 // based on the node properties.
@@ -418,6 +449,11 @@
 		case ScalarNode:
 			tag, _ := resolve("", n.Value)
 			return tag
+		case 0:
+			// Special case to make the zero value convenient.
+			if n.IsZero() {
+				return nullTag
+			}
 		}
 		return ""
 	}
diff --git a/vendor/gopkg.in/yaml.v3/yamlh.go b/vendor/gopkg.in/yaml.v3/yamlh.go
index 2719cfb..7c6d007 100644
--- a/vendor/gopkg.in/yaml.v3/yamlh.go
+++ b/vendor/gopkg.in/yaml.v3/yamlh.go
@@ -787,6 +787,8 @@
 	foot_comment []byte
 	tail_comment []byte
 
+	key_line_comment []byte
+
 	// Dumper stuff
 
 	opened bool // If the stream was already opened?
diff --git a/vendor/modules.txt b/vendor/modules.txt
index 9fcc410..308412b 100644
--- a/vendor/modules.txt
+++ b/vendor/modules.txt
@@ -1,6 +1,4 @@
-# github.com/DataDog/zstd v1.4.1
-github.com/DataDog/zstd
-# github.com/Shopify/sarama v1.23.1
+# github.com/Shopify/sarama v1.29.1
 github.com/Shopify/sarama
 # github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1
 github.com/aead/cmac
@@ -10,16 +8,30 @@
 github.com/boguslaw-wojcik/crc32a
 # github.com/bsm/sarama-cluster v2.1.15+incompatible
 github.com/bsm/sarama-cluster
-# github.com/buraksezer/consistent v0.0.0-20191006190839-693edf70fd72
-github.com/buraksezer/consistent
-# github.com/cespare/xxhash v1.1.0
-github.com/cespare/xxhash
 # github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73
 ## explicit
 github.com/cevaris/ordered_map
-# github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a
+# github.com/coreos/etcd v3.3.25+incompatible
+github.com/coreos/etcd/auth/authpb
+github.com/coreos/etcd/clientv3/balancer
+github.com/coreos/etcd/clientv3/balancer/connectivity
+github.com/coreos/etcd/clientv3/balancer/picker
+github.com/coreos/etcd/clientv3/balancer/resolver/endpoint
+github.com/coreos/etcd/clientv3/credentials
+github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes
+github.com/coreos/etcd/etcdserver/etcdserverpb
+github.com/coreos/etcd/mvcc/mvccpb
+github.com/coreos/etcd/pkg/logutil
+github.com/coreos/etcd/pkg/systemd
+github.com/coreos/etcd/pkg/types
+github.com/coreos/etcd/raft
+github.com/coreos/etcd/raft/raftpb
+github.com/coreos/etcd/version
+# github.com/coreos/go-semver v0.3.0
+github.com/coreos/go-semver/semver
+# github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf
 github.com/coreos/go-systemd/journal
-# github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea
+# github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f
 github.com/coreos/pkg/capnslog
 # github.com/davecgh/go-spew v1.1.1
 github.com/davecgh/go-spew/spew
@@ -36,7 +48,7 @@
 github.com/gogo/protobuf/gogoproto
 github.com/gogo/protobuf/proto
 github.com/gogo/protobuf/protoc-gen-gogo/descriptor
-# github.com/golang/protobuf v1.3.2
+# github.com/golang/protobuf v1.5.2
 ## explicit
 github.com/golang/protobuf/proto
 github.com/golang/protobuf/protoc-gen-go/descriptor
@@ -45,19 +57,67 @@
 github.com/golang/protobuf/ptypes/duration
 github.com/golang/protobuf/ptypes/empty
 github.com/golang/protobuf/ptypes/timestamp
-# github.com/golang/snappy v0.0.1
+# github.com/golang/snappy v0.0.3
 github.com/golang/snappy
 # github.com/google/gopacket v1.1.17
 ## explicit
 github.com/google/gopacket
 github.com/google/gopacket/layers
-# github.com/google/uuid v1.1.1
+# github.com/google/uuid v1.3.0
 github.com/google/uuid
-# github.com/hashicorp/go-uuid v1.0.1
+# github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
+github.com/grpc-ecosystem/go-grpc-middleware
+github.com/grpc-ecosystem/go-grpc-middleware/tags
+github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing
+github.com/grpc-ecosystem/go-grpc-middleware/util/metautils
+# github.com/hashicorp/go-uuid v1.0.2
 github.com/hashicorp/go-uuid
+# github.com/jcmturner/aescts/v2 v2.0.0
+github.com/jcmturner/aescts/v2
+# github.com/jcmturner/dnsutils/v2 v2.0.0
+github.com/jcmturner/dnsutils/v2
 # github.com/jcmturner/gofork v1.0.0
 github.com/jcmturner/gofork/encoding/asn1
 github.com/jcmturner/gofork/x/crypto/pbkdf2
+# github.com/jcmturner/gokrb5/v8 v8.4.2
+github.com/jcmturner/gokrb5/v8/asn1tools
+github.com/jcmturner/gokrb5/v8/client
+github.com/jcmturner/gokrb5/v8/config
+github.com/jcmturner/gokrb5/v8/credentials
+github.com/jcmturner/gokrb5/v8/crypto
+github.com/jcmturner/gokrb5/v8/crypto/common
+github.com/jcmturner/gokrb5/v8/crypto/etype
+github.com/jcmturner/gokrb5/v8/crypto/rfc3961
+github.com/jcmturner/gokrb5/v8/crypto/rfc3962
+github.com/jcmturner/gokrb5/v8/crypto/rfc4757
+github.com/jcmturner/gokrb5/v8/crypto/rfc8009
+github.com/jcmturner/gokrb5/v8/gssapi
+github.com/jcmturner/gokrb5/v8/iana
+github.com/jcmturner/gokrb5/v8/iana/addrtype
+github.com/jcmturner/gokrb5/v8/iana/adtype
+github.com/jcmturner/gokrb5/v8/iana/asnAppTag
+github.com/jcmturner/gokrb5/v8/iana/chksumtype
+github.com/jcmturner/gokrb5/v8/iana/errorcode
+github.com/jcmturner/gokrb5/v8/iana/etypeID
+github.com/jcmturner/gokrb5/v8/iana/flags
+github.com/jcmturner/gokrb5/v8/iana/keyusage
+github.com/jcmturner/gokrb5/v8/iana/msgtype
+github.com/jcmturner/gokrb5/v8/iana/nametype
+github.com/jcmturner/gokrb5/v8/iana/patype
+github.com/jcmturner/gokrb5/v8/kadmin
+github.com/jcmturner/gokrb5/v8/keytab
+github.com/jcmturner/gokrb5/v8/krberror
+github.com/jcmturner/gokrb5/v8/messages
+github.com/jcmturner/gokrb5/v8/pac
+github.com/jcmturner/gokrb5/v8/types
+# github.com/jcmturner/rpc/v2 v2.0.3
+github.com/jcmturner/rpc/v2/mstypes
+github.com/jcmturner/rpc/v2/ndr
+# github.com/klauspost/compress v1.12.2
+github.com/klauspost/compress/fse
+github.com/klauspost/compress/huff0
+github.com/klauspost/compress/zstd
+github.com/klauspost/compress/zstd/internal/xxhash
 # github.com/looplab/fsm v0.2.0
 ## explicit
 github.com/looplab/fsm
@@ -65,50 +125,50 @@
 ## explicit
 github.com/opencord/omci-lib-go
 github.com/opencord/omci-lib-go/generated
-# github.com/opencord/voltha-lib-go/v5 v5.0.5
+# github.com/opencord/voltha-lib-go/v7 v7.0.0
 ## explicit
-github.com/opencord/voltha-lib-go/v5/pkg/adapters
-github.com/opencord/voltha-lib-go/v5/pkg/adapters/adapterif
-github.com/opencord/voltha-lib-go/v5/pkg/adapters/common
-github.com/opencord/voltha-lib-go/v5/pkg/config
-github.com/opencord/voltha-lib-go/v5/pkg/db
-github.com/opencord/voltha-lib-go/v5/pkg/db/kvstore
-github.com/opencord/voltha-lib-go/v5/pkg/events
-github.com/opencord/voltha-lib-go/v5/pkg/events/eventif
-github.com/opencord/voltha-lib-go/v5/pkg/flows
-github.com/opencord/voltha-lib-go/v5/pkg/kafka
-github.com/opencord/voltha-lib-go/v5/pkg/log
-github.com/opencord/voltha-lib-go/v5/pkg/meters
-github.com/opencord/voltha-lib-go/v5/pkg/probe
-github.com/opencord/voltha-lib-go/v5/pkg/version
-# github.com/opencord/voltha-protos/v4 v4.2.0
+github.com/opencord/voltha-lib-go/v7/pkg/config
+github.com/opencord/voltha-lib-go/v7/pkg/db
+github.com/opencord/voltha-lib-go/v7/pkg/db/kvstore
+github.com/opencord/voltha-lib-go/v7/pkg/events
+github.com/opencord/voltha-lib-go/v7/pkg/events/eventif
+github.com/opencord/voltha-lib-go/v7/pkg/flows
+github.com/opencord/voltha-lib-go/v7/pkg/grpc
+github.com/opencord/voltha-lib-go/v7/pkg/kafka
+github.com/opencord/voltha-lib-go/v7/pkg/log
+github.com/opencord/voltha-lib-go/v7/pkg/meters
+github.com/opencord/voltha-lib-go/v7/pkg/probe
+github.com/opencord/voltha-lib-go/v7/pkg/version
+# github.com/opencord/voltha-protos/v5 v5.0.0
 ## explicit
-github.com/opencord/voltha-protos/v4/go/common
-github.com/opencord/voltha-protos/v4/go/ext/config
-github.com/opencord/voltha-protos/v4/go/extension
-github.com/opencord/voltha-protos/v4/go/inter_container
-github.com/opencord/voltha-protos/v4/go/omci
-github.com/opencord/voltha-protos/v4/go/openflow_13
-github.com/opencord/voltha-protos/v4/go/openolt
-github.com/opencord/voltha-protos/v4/go/tech_profile
-github.com/opencord/voltha-protos/v4/go/voltha
-# github.com/opentracing/opentracing-go v1.1.0
+github.com/opencord/voltha-protos/v5/go/adapter_services
+github.com/opencord/voltha-protos/v5/go/common
+github.com/opencord/voltha-protos/v5/go/core
+github.com/opencord/voltha-protos/v5/go/ext/config
+github.com/opencord/voltha-protos/v5/go/extension
+github.com/opencord/voltha-protos/v5/go/inter_container
+github.com/opencord/voltha-protos/v5/go/omci
+github.com/opencord/voltha-protos/v5/go/openflow_13
+github.com/opencord/voltha-protos/v5/go/openolt
+github.com/opencord/voltha-protos/v5/go/tech_profile
+github.com/opencord/voltha-protos/v5/go/voltha
+# github.com/opentracing/opentracing-go v1.2.0
 github.com/opentracing/opentracing-go
 github.com/opentracing/opentracing-go/ext
 github.com/opentracing/opentracing-go/log
-# github.com/pierrec/lz4 v2.3.0+incompatible
+# github.com/pierrec/lz4 v2.6.0+incompatible
 github.com/pierrec/lz4
 github.com/pierrec/lz4/internal/xxh32
-# github.com/pkg/errors v0.8.1
+# github.com/pkg/errors v0.9.1
 github.com/pkg/errors
 # github.com/pmezard/go-difflib v1.0.0
 github.com/pmezard/go-difflib/difflib
-# github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563
+# github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475
 github.com/rcrowley/go-metrics
-# github.com/stretchr/testify v1.6.1
+# github.com/stretchr/testify v1.7.0
 ## explicit
 github.com/stretchr/testify/assert
-# github.com/uber/jaeger-client-go v2.23.1+incompatible
+# github.com/uber/jaeger-client-go v2.29.1+incompatible
 github.com/uber/jaeger-client-go
 github.com/uber/jaeger-client-go/config
 github.com/uber/jaeger-client-go/internal/baggage
@@ -127,43 +187,26 @@
 github.com/uber/jaeger-client-go/thrift-gen/zipkincore
 github.com/uber/jaeger-client-go/transport
 github.com/uber/jaeger-client-go/utils
-# github.com/uber/jaeger-lib v2.2.0+incompatible
+# github.com/uber/jaeger-lib v2.4.1+incompatible
 github.com/uber/jaeger-lib/metrics
-# go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522
-go.etcd.io/etcd/auth/authpb
+# go.etcd.io/etcd v3.3.25+incompatible
 go.etcd.io/etcd/clientv3
-go.etcd.io/etcd/clientv3/balancer
-go.etcd.io/etcd/clientv3/balancer/connectivity
-go.etcd.io/etcd/clientv3/balancer/picker
-go.etcd.io/etcd/clientv3/balancer/resolver/endpoint
-go.etcd.io/etcd/clientv3/credentials
 go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes
-go.etcd.io/etcd/etcdserver/etcdserverpb
-go.etcd.io/etcd/mvcc/mvccpb
-go.etcd.io/etcd/pkg/logutil
-go.etcd.io/etcd/pkg/systemd
-go.etcd.io/etcd/pkg/types
-go.etcd.io/etcd/raft
-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.uber.org/atomic v1.4.0
+# go.uber.org/atomic v1.7.0
 go.uber.org/atomic
-# go.uber.org/multierr v1.2.0
+# go.uber.org/multierr v1.6.0
 go.uber.org/multierr
-# go.uber.org/zap v1.10.0
+# go.uber.org/zap v1.18.1
 go.uber.org/zap
 go.uber.org/zap/buffer
 go.uber.org/zap/internal/bufferpool
 go.uber.org/zap/internal/color
 go.uber.org/zap/internal/exit
 go.uber.org/zap/zapcore
-# golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9
+# golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e
 golang.org/x/crypto/md4
 golang.org/x/crypto/pbkdf2
-# golang.org/x/net v0.0.0-20201021035429-f5854403a974
-golang.org/x/net/context
+# golang.org/x/net v0.0.0-20210614182718-04defd469f4e
 golang.org/x/net/http/httpguts
 golang.org/x/net/http2
 golang.org/x/net/http2/hpack
@@ -172,18 +215,18 @@
 golang.org/x/net/internal/timeseries
 golang.org/x/net/proxy
 golang.org/x/net/trace
-# golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f
+# golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1
 golang.org/x/sys/internal/unsafeheader
 golang.org/x/sys/unix
-# golang.org/x/text v0.3.3
+# golang.org/x/text v0.3.6
 golang.org/x/text/secure/bidirule
 golang.org/x/text/transform
 golang.org/x/text/unicode/bidi
 golang.org/x/text/unicode/norm
-# google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c
+# google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0
 google.golang.org/genproto/googleapis/api/annotations
 google.golang.org/genproto/googleapis/rpc/status
-# google.golang.org/grpc v1.25.1
+# google.golang.org/grpc v1.41.0 => google.golang.org/grpc v1.25.1
 ## explicit
 google.golang.org/grpc
 google.golang.org/grpc/backoff
@@ -215,6 +258,8 @@
 google.golang.org/grpc/metadata
 google.golang.org/grpc/naming
 google.golang.org/grpc/peer
+google.golang.org/grpc/reflection
+google.golang.org/grpc/reflection/grpc_reflection_v1alpha
 google.golang.org/grpc/resolver
 google.golang.org/grpc/resolver/dns
 google.golang.org/grpc/resolver/passthrough
@@ -222,43 +267,40 @@
 google.golang.org/grpc/stats
 google.golang.org/grpc/status
 google.golang.org/grpc/tap
-# gopkg.in/jcmturner/aescts.v1 v1.0.1
-gopkg.in/jcmturner/aescts.v1
-# gopkg.in/jcmturner/dnsutils.v1 v1.0.1
-gopkg.in/jcmturner/dnsutils.v1
-# gopkg.in/jcmturner/gokrb5.v7 v7.3.0
-gopkg.in/jcmturner/gokrb5.v7/asn1tools
-gopkg.in/jcmturner/gokrb5.v7/client
-gopkg.in/jcmturner/gokrb5.v7/config
-gopkg.in/jcmturner/gokrb5.v7/credentials
-gopkg.in/jcmturner/gokrb5.v7/crypto
-gopkg.in/jcmturner/gokrb5.v7/crypto/common
-gopkg.in/jcmturner/gokrb5.v7/crypto/etype
-gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961
-gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962
-gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757
-gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009
-gopkg.in/jcmturner/gokrb5.v7/gssapi
-gopkg.in/jcmturner/gokrb5.v7/iana
-gopkg.in/jcmturner/gokrb5.v7/iana/addrtype
-gopkg.in/jcmturner/gokrb5.v7/iana/adtype
-gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag
-gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype
-gopkg.in/jcmturner/gokrb5.v7/iana/errorcode
-gopkg.in/jcmturner/gokrb5.v7/iana/etypeID
-gopkg.in/jcmturner/gokrb5.v7/iana/flags
-gopkg.in/jcmturner/gokrb5.v7/iana/keyusage
-gopkg.in/jcmturner/gokrb5.v7/iana/msgtype
-gopkg.in/jcmturner/gokrb5.v7/iana/nametype
-gopkg.in/jcmturner/gokrb5.v7/iana/patype
-gopkg.in/jcmturner/gokrb5.v7/kadmin
-gopkg.in/jcmturner/gokrb5.v7/keytab
-gopkg.in/jcmturner/gokrb5.v7/krberror
-gopkg.in/jcmturner/gokrb5.v7/messages
-gopkg.in/jcmturner/gokrb5.v7/pac
-gopkg.in/jcmturner/gokrb5.v7/types
-# gopkg.in/jcmturner/rpc.v1 v1.1.0
-gopkg.in/jcmturner/rpc.v1/mstypes
-gopkg.in/jcmturner/rpc.v1/ndr
-# gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c
+# google.golang.org/protobuf v1.27.1
+google.golang.org/protobuf/encoding/prototext
+google.golang.org/protobuf/encoding/protowire
+google.golang.org/protobuf/internal/descfmt
+google.golang.org/protobuf/internal/descopts
+google.golang.org/protobuf/internal/detrand
+google.golang.org/protobuf/internal/encoding/defval
+google.golang.org/protobuf/internal/encoding/messageset
+google.golang.org/protobuf/internal/encoding/tag
+google.golang.org/protobuf/internal/encoding/text
+google.golang.org/protobuf/internal/errors
+google.golang.org/protobuf/internal/filedesc
+google.golang.org/protobuf/internal/filetype
+google.golang.org/protobuf/internal/flags
+google.golang.org/protobuf/internal/genid
+google.golang.org/protobuf/internal/impl
+google.golang.org/protobuf/internal/order
+google.golang.org/protobuf/internal/pragma
+google.golang.org/protobuf/internal/set
+google.golang.org/protobuf/internal/strs
+google.golang.org/protobuf/internal/version
+google.golang.org/protobuf/proto
+google.golang.org/protobuf/reflect/protodesc
+google.golang.org/protobuf/reflect/protoreflect
+google.golang.org/protobuf/reflect/protoregistry
+google.golang.org/protobuf/runtime/protoiface
+google.golang.org/protobuf/runtime/protoimpl
+google.golang.org/protobuf/types/descriptorpb
+google.golang.org/protobuf/types/known/anypb
+google.golang.org/protobuf/types/known/durationpb
+google.golang.org/protobuf/types/known/emptypb
+google.golang.org/protobuf/types/known/timestamppb
+# gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b
 gopkg.in/yaml.v3
+# github.com/coreos/bbolt v1.3.4 => go.etcd.io/bbolt v1.3.4
+# go.etcd.io/bbolt v1.3.4 => github.com/coreos/bbolt v1.3.4
+# google.golang.org/grpc => google.golang.org/grpc v1.25.1